From a936a7a584f01b7dcf98ec98e7591f5df167e74a Mon Sep 17 00:00:00 2001 From: lichuang Date: Tue, 16 Nov 2021 15:41:08 +0800 Subject: [PATCH 01/44] [TD-10645][raft]add node map --- source/libs/sync/inc/sync_raft_node_map.h | 36 ++++++++++ source/libs/sync/inc/sync_raft_proto.h | 1 + source/libs/sync/inc/sync_raft_quorum_joint.h | 7 +- source/libs/sync/inc/sync_type.h | 7 +- source/libs/sync/src/raft.c | 8 ++- .../sync/src/raft_handle_election_message.c | 18 ----- source/libs/sync/src/raft_replication.c | 6 +- .../libs/sync/src/sync_raft_config_change.c | 4 +- .../{raft_election.c => sync_raft_election.c} | 55 ++++++++++++---- source/libs/sync/src/sync_raft_impl.c | 5 +- source/libs/sync/src/sync_raft_node_map.c | 66 +++++++++++++++++++ .../sync/src/sync_raft_progress_tracker.c | 4 ++ source/libs/sync/src/sync_raft_quorum_joint.c | 14 ++-- .../libs/sync/src/sync_raft_quorum_majority.c | 1 + 14 files changed, 174 insertions(+), 58 deletions(-) create mode 100644 source/libs/sync/inc/sync_raft_node_map.h rename source/libs/sync/src/{raft_election.c => sync_raft_election.c} (56%) create mode 100644 source/libs/sync/src/sync_raft_node_map.c diff --git a/source/libs/sync/inc/sync_raft_node_map.h b/source/libs/sync/inc/sync_raft_node_map.h new file mode 100644 index 0000000000..bfb5f68489 --- /dev/null +++ b/source/libs/sync/inc/sync_raft_node_map.h @@ -0,0 +1,36 @@ +/* + * 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_NODE_MAP_H +#define _TD_LIBS_SYNC_RAFT_NODE_MAP_H + +#include "sync.h" +#include "sync_type.h" + +// TODO: is TSDB_MAX_REPLICA enough? +struct SSyncRaftNodeMap { + int32_t replica; + SyncNodeId nodeId[TSDB_MAX_REPLICA]; +}; + +bool syncRaftIsInNodeMap(const SSyncRaftNodeMap* nodeMap, SyncNodeId nodeId); + +void syncRaftCopyNodeMap(const SSyncRaftNodeMap* nodeMap, SSyncRaftNodeMap* to); + +void syncRaftUnionNodeMap(const SSyncRaftNodeMap* nodeMap, SSyncRaftNodeMap* to); + +void syncRaftAddToNodeMap(SSyncRaftNodeMap* nodeMap, SyncNodeId nodeId); + +#endif /* _TD_LIBS_SYNC_RAFT_NODE_MAP_H */ \ No newline at end of file diff --git a/source/libs/sync/inc/sync_raft_proto.h b/source/libs/sync/inc/sync_raft_proto.h index c131e91139..dd153e8dad 100644 --- a/source/libs/sync/inc/sync_raft_proto.h +++ b/source/libs/sync/inc/sync_raft_proto.h @@ -17,6 +17,7 @@ #define TD_SYNC_RAFT_PROTO_H #include "sync_type.h" +#include "sync_raft_node_map.h" typedef enum ESyncRaftConfChangeType { SYNC_RAFT_Conf_AddNode = 0, diff --git a/source/libs/sync/inc/sync_raft_quorum_joint.h b/source/libs/sync/inc/sync_raft_quorum_joint.h index 0ef002fe1a..0637a9be9a 100644 --- a/source/libs/sync/inc/sync_raft_quorum_joint.h +++ b/source/libs/sync/inc/sync_raft_quorum_joint.h @@ -19,6 +19,7 @@ #include "taosdef.h" #include "sync.h" #include "sync_type.h" +#include "sync_raft_node_map.h" /** * SSyncRaftQuorumJointConfig is a configuration of two groups of (possibly overlapping) @@ -36,8 +37,6 @@ typedef struct SSyncRaftQuorumJointConfig { **/ ESyncRaftVoteType syncRaftVoteResult(SSyncRaftQuorumJointConfig* config, const ESyncRaftVoteType* votes); -bool syncRaftIsInNodeMap(const SSyncRaftNodeMap* nodeMap, SyncNodeId nodeId); - static FORCE_INLINE bool syncRaftJointConfigInOutgoing(const SSyncRaftQuorumJointConfig* config, SyncNodeId id) { return syncRaftIsInNodeMap(&config->outgoing, id); } @@ -59,7 +58,9 @@ static FORCE_INLINE const SSyncRaftNodeMap* syncRaftJointConfigOutgoing(const SS } static FORCE_INLINE void syncRaftJointConfigClearOutgoing(SSyncRaftQuorumJointConfig* config) { - memset(&config->outgoing, 0, sizeof(SSyncCluster)); + memset(&config->outgoing, 0, sizeof(SSyncRaftNodeMap)); } +void syncRaftJointConfigIDS(const SSyncRaftQuorumJointConfig* config, SSyncRaftNodeMap* nodeMap); + #endif /* _TD_LIBS_SYNC_RAFT_QUORUM_JOINT_H */ diff --git a/source/libs/sync/inc/sync_type.h b/source/libs/sync/inc/sync_type.h index cb938c7319..6d29e019cc 100644 --- a/source/libs/sync/inc/sync_type.h +++ b/source/libs/sync/inc/sync_type.h @@ -32,6 +32,8 @@ typedef struct SSyncRaftProgress SSyncRaftProgress; typedef struct SSyncRaftProgressMap SSyncRaftProgressMap; typedef struct SSyncRaftProgressTrackerConfig SSyncRaftProgressTrackerConfig; +typedef struct SSyncRaftNodeMap SSyncRaftNodeMap; + typedef struct SSyncRaftProgressTracker SSyncRaftProgressTracker; typedef struct SSyncRaftChanger SSyncRaftChanger; @@ -68,11 +70,6 @@ typedef struct SSyncClusterConfig { const SSyncCluster* cluster; } SSyncClusterConfig; -typedef struct { - int32_t replica; - SyncNodeId nodeId[TSDB_MAX_REPLICA]; -} SSyncRaftNodeMap; - typedef enum { SYNC_RAFT_CAMPAIGN_PRE_ELECTION = 0, SYNC_RAFT_CAMPAIGN_ELECTION = 1, diff --git a/source/libs/sync/src/raft.c b/source/libs/sync/src/raft.c index 23351277c4..39e0377545 100644 --- a/source/libs/sync/src/raft.c +++ b/source/libs/sync/src/raft.c @@ -140,6 +140,7 @@ int32_t syncRaftStep(SSyncRaft* pRaft, const SSyncMessage* pMsg) { int32_t syncRaftTick(SSyncRaft* pRaft) { pRaft->currentTick += 1; + pRaft->tickFp(pRaft); return 0; } @@ -212,8 +213,11 @@ static void switchToConfig(SSyncRaft* pRaft, const SSyncRaftProgressTrackerConfi // If the the leadTransferee was removed or demoted, abort the leadership transfer. SyncNodeId leadTransferee = pRaft->leadTransferee; - if (leadTransferee != SYNC_NON_NODE_ID && !syncRaftIsInNodeMap(&pRaft->tracker->config.voters, leadTransferee)) { - abortLeaderTransfer(pRaft); + if (leadTransferee != SYNC_NON_NODE_ID) { + if (!syncRaftIsInNodeMap(&pRaft->tracker->config.voters.incoming, leadTransferee) && + !syncRaftIsInNodeMap(&pRaft->tracker->config.voters.outgoing, leadTransferee)) { + abortLeaderTransfer(pRaft); + } } } } diff --git a/source/libs/sync/src/raft_handle_election_message.c b/source/libs/sync/src/raft_handle_election_message.c index e536fc67c0..a58c8ba5cf 100644 --- a/source/libs/sync/src/raft_handle_election_message.c +++ b/source/libs/sync/src/raft_handle_election_message.c @@ -19,24 +19,6 @@ #include "raft_message.h" int syncRaftHandleElectionMessage(SSyncRaft* pRaft, const SSyncMessage* pMsg) { - if (pRaft->state == TAOS_SYNC_STATE_LEADER) { - syncDebug("[%d:%d] ignoring RAFT_MSG_INTERNAL_ELECTION because already leader", pRaft->selfGroupId, pRaft->selfId); - return 0; - } - - if (!syncRaftIsPromotable(pRaft)) { - syncDebug("[%d:%d] is unpromotable and can not campaign", pRaft->selfGroupId, pRaft->selfId); - return 0; - } - // if there is pending uncommitted config,cannot start election - if (syncRaftLogNumOfPendingConf(pRaft->log) > 0 && syncRaftHasUnappliedLog(pRaft->log)) { - syncWarn("[%d:%d] cannot syncRaftStartElection at term %" PRId64 " since there are still pending configuration changes to apply", - pRaft->selfGroupId, pRaft->selfId, pRaft->term); - return 0; - } - - syncInfo("[%d:%d] is starting a new election at term %" PRId64 "", pRaft->selfGroupId, pRaft->selfId, pRaft->term); - if (pRaft->preVote) { syncRaftStartElection(pRaft, SYNC_RAFT_CAMPAIGN_PRE_ELECTION); } else { diff --git a/source/libs/sync/src/raft_replication.c b/source/libs/sync/src/raft_replication.c index c19fcd1e68..74f40179c6 100644 --- a/source/libs/sync/src/raft_replication.c +++ b/source/libs/sync/src/raft_replication.c @@ -22,7 +22,7 @@ static bool sendSnapshot(SSyncRaft* pRaft, SSyncRaftProgress* progress); static bool sendAppendEntries(SSyncRaft* pRaft, SSyncRaftProgress* progress, SyncIndex prevIndex, SyncTerm prevTerm, - const SSyncRaftEntry *entries, int nEntry); + SSyncRaftEntry *entries, int nEntry); // syncRaftReplicate sends an append RPC with new entries to the given peer, // if necessary. Returns true if a message was sent. The sendIfEmpty @@ -68,7 +68,7 @@ static bool sendSnapshot(SSyncRaft* pRaft, SSyncRaftProgress* progress) { static bool sendAppendEntries(SSyncRaft* pRaft, SSyncRaftProgress* progress, SyncIndex prevIndex, SyncTerm prevTerm, - const SSyncRaftEntry *entries, int nEntry) { + SSyncRaftEntry *entries, int nEntry) { SyncIndex lastIndex; SyncTerm logTerm = prevTerm; SNodeInfo* pNode = &(pRaft->cluster.nodeInfo[progress->selfIndex]); @@ -87,7 +87,7 @@ static bool sendAppendEntries(SSyncRaft* pRaft, SSyncRaftProgress* progress, case PROGRESS_STATE_REPLICATE: lastIndex = entries[nEntry - 1].index; syncRaftProgressOptimisticNextIndex(progress, lastIndex); - syncRaftInflightAdd(&progress->inflights, lastIndex); + syncRaftInflightAdd(progress->inflights, lastIndex); break; case PROGRESS_STATE_PROBE: progress->probeSent = true; diff --git a/source/libs/sync/src/sync_raft_config_change.c b/source/libs/sync/src/sync_raft_config_change.c index 4e7f2190ea..288fdc465e 100644 --- a/source/libs/sync/src/sync_raft_config_change.c +++ b/source/libs/sync/src/sync_raft_config_change.c @@ -359,7 +359,7 @@ static void makeLearner(SSyncRaftChanger* changer, SSyncRaftProgressTrackerConfi // be turned into a learner in LeaveJoint(). // // Otherwise, add a regular learner right away. - bool inOutgoing = syncRaftJointConfigInCluster(&config->voters.outgoing, id); + bool inOutgoing = syncRaftIsInNodeMap(&config->voters.outgoing, id); if (inOutgoing) { nilAwareAdd(&config->learnersNext, id); } else { @@ -381,7 +381,7 @@ static void removeNodeId(SSyncRaftChanger* changer, SSyncRaftProgressTrackerConf nilAwareDelete(&config->learnersNext, id); // If the peer is still a voter in the outgoing config, keep the Progress. - bool inOutgoing = syncRaftJointConfigInCluster(&config->voters.outgoing, id); + bool inOutgoing = syncRaftIsInNodeMap(&config->voters.outgoing, id); if (!inOutgoing) { syncRaftRemoveFromProgressMap(progressMap, id); } diff --git a/source/libs/sync/src/raft_election.c b/source/libs/sync/src/sync_raft_election.c similarity index 56% rename from source/libs/sync/src/raft_election.c rename to source/libs/sync/src/sync_raft_election.c index eb310c31ec..74c3e09dae 100644 --- a/source/libs/sync/src/raft_election.c +++ b/source/libs/sync/src/sync_raft_election.c @@ -17,15 +17,42 @@ #include "raft.h" #include "raft_log.h" #include "raft_message.h" +#include "sync_raft_progress_tracker.h" + +static void campaign(SSyncRaft* pRaft, ESyncRaftElectionType cType); void syncRaftStartElection(SSyncRaft* pRaft, ESyncRaftElectionType cType) { - SyncTerm term; + if (pRaft->state == TAOS_SYNC_STATE_LEADER) { + syncDebug("[%d:%d] ignoring RAFT_MSG_INTERNAL_ELECTION because already leader", pRaft->selfGroupId, pRaft->selfId); + return; + } + + if (!syncRaftIsPromotable(pRaft)) { + syncWarn("[%d:%d] is unpromotable and can not campaign", pRaft->selfGroupId, pRaft->selfId); + return; + } + + // if there is pending uncommitted config,cannot start election + if (syncRaftLogNumOfPendingConf(pRaft->log) > 0 && syncRaftHasUnappliedLog(pRaft->log)) { + syncWarn("[%d:%d] cannot syncRaftStartElection at term %" PRId64 " since there are still pending configuration changes to apply", + pRaft->selfGroupId, pRaft->selfId, pRaft->term); + return; + } + + syncInfo("[%d:%d] is starting a new election at term %" PRId64 "", pRaft->selfGroupId, pRaft->selfId, pRaft->term); + + campaign(pRaft, cType); +} + +// campaign transitions the raft instance to candidate state. This must only be +// called after verifying that this is a legitimate transition. +static void campaign(SSyncRaft* pRaft, ESyncRaftElectionType cType) { bool preVote; - ESyncRaftMessageType voteMsgType; + SyncTerm term; if (syncRaftIsPromotable(pRaft)) { syncDebug("[%d:%d] is unpromotable; campaign() should have been called", pRaft->selfGroupId, pRaft->selfId); - return 0; + return; } if (cType == SYNC_RAFT_CAMPAIGN_PRE_ELECTION) { @@ -35,7 +62,6 @@ void syncRaftStartElection(SSyncRaft* pRaft, ESyncRaftElectionType cType) { term = pRaft->term + 1; } else { syncRaftBecomeCandidate(pRaft); - voteMsgType = RAFT_MSG_VOTE; term = pRaft->term; preVote = false; } @@ -43,10 +69,8 @@ void syncRaftStartElection(SSyncRaft* pRaft, ESyncRaftElectionType cType) { int quorum = syncRaftQuorum(pRaft); ESyncRaftVoteResult result = syncRaftPollVote(pRaft, pRaft->selfId, preVote, true, NULL, NULL); if (result == SYNC_RAFT_VOTE_WON) { - /** - * We won the election after voting for ourselves (which must mean that - * this is a single-node cluster). Advance to the next state. - **/ + // We won the election after voting for ourselves (which must mean that + // this is a single-node cluster). Advance to the next state. if (cType == SYNC_RAFT_CAMPAIGN_PRE_ELECTION) { syncRaftStartElection(pRaft, SYNC_RAFT_CAMPAIGN_ELECTION); } else { @@ -59,12 +83,17 @@ void syncRaftStartElection(SSyncRaft* pRaft, ESyncRaftElectionType cType) { int i; SyncIndex lastIndex = syncRaftLogLastIndex(pRaft->log); SyncTerm lastTerm = syncRaftLogLastTerm(pRaft->log); - for (i = 0; i < pRaft->cluster.replica; ++i) { - if (i == pRaft->cluster.selfIndex) { + SSyncRaftNodeMap nodeMap; + syncRaftJointConfigIDS(&pRaft->tracker->config.voters, &nodeMap); + for (i = 0; i < TSDB_MAX_REPLICA; ++i) { + SyncNodeId nodeId = nodeMap.nodeId[i]; + if (nodeId == SYNC_NON_NODE_ID) { continue; } - SyncNodeId nodeId = pRaft->cluster.nodeInfo[i].nodeId; + if (nodeId == pRaft->selfId) { + continue; + } SSyncMessage* pMsg = syncNewVoteMsg(pRaft->selfGroupId, pRaft->selfId, term, cType, lastIndex, lastTerm); @@ -72,9 +101,9 @@ void syncRaftStartElection(SSyncRaft* pRaft, ESyncRaftElectionType cType) { continue; } - syncInfo("[%d:%d] [logterm: %" PRId64 ", index: %" PRId64 "] sent %d request to %d at term %" PRId64 "", + syncInfo("[%d:%d] [logterm: %" PRId64 ", index: %" PRId64 "] sent vote request to %d at term %" PRId64 "", pRaft->selfGroupId, pRaft->selfId, lastTerm, - lastIndex, voteMsgType, nodeId, pRaft->term); + lastIndex, nodeId, pRaft->term); pRaft->io.send(pMsg, &(pRaft->cluster.nodeInfo[i])); } diff --git a/source/libs/sync/src/sync_raft_impl.c b/source/libs/sync/src/sync_raft_impl.c index d65e03c64f..ab2db10230 100644 --- a/source/libs/sync/src/sync_raft_impl.c +++ b/source/libs/sync/src/sync_raft_impl.c @@ -234,9 +234,7 @@ static int stepLeader(SSyncRaft* pRaft, const SSyncMessage* pMsg) { return 0; } -/** - * tickElection is run by followers and candidates per tick. - **/ +// tickElection is run by followers and candidates after r.electionTimeout. static void tickElection(SSyncRaft* pRaft) { pRaft->electionElapsed += 1; @@ -254,6 +252,7 @@ static void tickElection(SSyncRaft* pRaft) { syncRaftStep(pRaft, syncInitElectionMsg(&msg, pRaft->selfId)); } +// tickHeartbeat is run by leaders to send a MsgBeat after r.heartbeatTimeout. static void tickHeartbeat(SSyncRaft* pRaft) { } diff --git a/source/libs/sync/src/sync_raft_node_map.c b/source/libs/sync/src/sync_raft_node_map.c new file mode 100644 index 0000000000..e13c808075 --- /dev/null +++ b/source/libs/sync/src/sync_raft_node_map.c @@ -0,0 +1,66 @@ +/* + * 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 "sync_raft_node_map.h" +#include "sync_type.h" + +bool syncRaftIsInNodeMap(const SSyncRaftNodeMap* nodeMap, SyncNodeId nodeId) { + int i; + + for (i = 0; i < TSDB_MAX_REPLICA; ++i) { + if (nodeId == nodeMap->nodeId[i]) { + return true; + } + } + + return false; +} + +void syncRaftCopyNodeMap(const SSyncRaftNodeMap* nodeMap, SSyncRaftNodeMap* to) { + memcpy(to, nodeMap, sizeof(SSyncRaftNodeMap)); +} + +void syncRaftUnionNodeMap(const SSyncRaftNodeMap* nodeMap, SSyncRaftNodeMap* to) { + int i, j, m; + + for (i = 0; i < TSDB_MAX_REPLICA; ++i) { + SyncNodeId id = nodeMap->nodeId[i]; + if (id == SYNC_NON_NODE_ID) { + continue; + } + + syncRaftAddToNodeMap(to, id); + } +} + +void syncRaftAddToNodeMap(SSyncRaftNodeMap* nodeMap, SyncNodeId nodeId) { + assert(nodeMap->replica < TSDB_MAX_REPLICA); + + int i, j; + for (i = 0, j = -1; i < TSDB_MAX_REPLICA; ++i) { + SyncNodeId id = nodeMap->nodeId[i]; + if (id == SYNC_NON_NODE_ID) { + if (j == -1) j = i; + continue; + } + if (id == nodeId) { + return; + } + } + + assert(j != -1); + nodeMap->nodeId[j] = nodeId; + nodeMap->replica += 1; +} \ No newline at end of file diff --git a/source/libs/sync/src/sync_raft_progress_tracker.c b/source/libs/sync/src/sync_raft_progress_tracker.c index ea7f1ae4f5..f43414127d 100644 --- a/source/libs/sync/src/sync_raft_progress_tracker.c +++ b/source/libs/sync/src/sync_raft_progress_tracker.c @@ -49,6 +49,10 @@ void syncRaftCloneTrackerConfig(const SSyncRaftProgressTrackerConfig* from, SSyn } +int syncRaftCheckProgress(const SSyncRaftProgressTrackerConfig* config, SSyncRaftProgressMap* progressMap) { + return 0; +} + /** * syncRaftTallyVotes returns the number of granted and rejected Votes, and whether the * election outcome is known. diff --git a/source/libs/sync/src/sync_raft_quorum_joint.c b/source/libs/sync/src/sync_raft_quorum_joint.c index fa663b6fc3..8a99574d68 100644 --- a/source/libs/sync/src/sync_raft_quorum_joint.c +++ b/source/libs/sync/src/sync_raft_quorum_joint.c @@ -13,6 +13,7 @@ * along with this program. If not, see . */ +#include "sync_raft_node_map.h" #include "sync_raft_quorum_majority.h" #include "sync_raft_quorum_joint.h" #include "sync_raft_quorum.h" @@ -71,15 +72,10 @@ void syncRaftJointConfigRemoveFromIncoming(SSyncRaftQuorumJointConfig* config, S assert(config->incoming.replica >= 0); } +void syncRaftJointConfigIDS(const SSyncRaftQuorumJointConfig* config, SSyncRaftNodeMap* nodeMap) { + int i, j, m; -bool syncRaftIsInNodeMap(const SSyncRaftNodeMap* nodeMap, SyncNodeId nodeId) { - int i; + syncRaftCopyNodeMap(&config->incoming, nodeMap); - for (i = 0; i < TSDB_MAX_REPLICA; ++i) { - if (nodeId == nodeMap->nodeId[i]) { - return true; - } - } - - return false; + syncRaftUnionNodeMap(&config->outgoing, nodeMap); } \ No newline at end of file diff --git a/source/libs/sync/src/sync_raft_quorum_majority.c b/source/libs/sync/src/sync_raft_quorum_majority.c index 73eb378e09..8ff5752b97 100644 --- a/source/libs/sync/src/sync_raft_quorum_majority.c +++ b/source/libs/sync/src/sync_raft_quorum_majority.c @@ -15,6 +15,7 @@ #include "sync_raft_quorum.h" #include "sync_raft_quorum_majority.h" +#include "sync_raft_node_map.h" /** * syncRaftMajorityVoteResult takes a mapping of voters to yes/no (true/false) votes and returns From 0c65b84886ede5d64c434f5d106ceb0851f69856 Mon Sep 17 00:00:00 2001 From: lichuang Date: Wed, 17 Nov 2021 08:29:24 +0800 Subject: [PATCH 02/44] [TD-10645][raft]add node map --- source/libs/sync/inc/raft.h | 4 ++- source/libs/sync/inc/raft_configuration.h | 27 ------------------- source/libs/sync/inc/sync_raft_impl.h | 2 ++ source/libs/sync/src/raft.c | 13 ++++++--- source/libs/sync/src/raft_configuration.c | 25 ----------------- .../src/raft_handle_append_entries_message.c | 9 +++---- .../libs/sync/src/raft_handle_vote_message.c | 9 ++++--- .../sync/src/raft_handle_vote_resp_message.c | 6 ++--- source/libs/sync/src/raft_replication.c | 7 +++-- source/libs/sync/src/sync_raft_election.c | 2 +- source/libs/sync/src/sync_raft_impl.c | 26 ++++++++++++------ 11 files changed, 50 insertions(+), 80 deletions(-) delete mode 100644 source/libs/sync/inc/raft_configuration.h delete mode 100644 source/libs/sync/src/raft_configuration.c diff --git a/source/libs/sync/inc/raft.h b/source/libs/sync/inc/raft.h index 5b6efb95e5..6fa6c6e346 100644 --- a/source/libs/sync/inc/raft.h +++ b/source/libs/sync/inc/raft.h @@ -18,6 +18,7 @@ #include "sync.h" #include "sync_type.h" +#include "thash.h" #include "raft_message.h" #include "sync_raft_impl.h" #include "sync_raft_quorum.h" @@ -43,7 +44,8 @@ struct SSyncRaft { // owner sync node SSyncNode* pNode; - SSyncCluster cluster; + // hash map nodeId -> SNodeInfo* + SHashObj* nodeInfoMap; int selfIndex; SyncNodeId selfId; diff --git a/source/libs/sync/inc/raft_configuration.h b/source/libs/sync/inc/raft_configuration.h deleted file mode 100644 index ac9bbb5e55..0000000000 --- a/source/libs/sync/inc/raft_configuration.h +++ /dev/null @@ -1,27 +0,0 @@ -/* - * 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_CONFIGURATION_H -#define _TD_LIBS_SYNC_RAFT_CONFIGURATION_H - -#include "sync.h" -#include "sync_type.h" - -// return -1 if cannot find this id -int syncRaftConfigurationIndexOfNode(SSyncRaft *pRaft, SyncNodeId id); - -int syncRaftConfigurationVoterCount(SSyncRaft *pRaft); - -#endif /* _TD_LIBS_SYNC_RAFT_CONFIGURATION_H */ \ No newline at end of file diff --git a/source/libs/sync/inc/sync_raft_impl.h b/source/libs/sync/inc/sync_raft_impl.h index bd77978c28..a8615f17eb 100644 --- a/source/libs/sync/inc/sync_raft_impl.h +++ b/source/libs/sync/inc/sync_raft_impl.h @@ -51,4 +51,6 @@ void syncRaftLoadState(SSyncRaft* pRaft, const SSyncServerState* serverState); void syncRaftBroadcastAppend(SSyncRaft* pRaft); +SNodeInfo* syncRaftGetNodeById(SSyncRaft *pRaft, SyncNodeId id); + #endif /* _TD_LIBS_SYNC_RAFT_IMPL_H */ diff --git a/source/libs/sync/src/raft.c b/source/libs/sync/src/raft.c index 39e0377545..d39e047492 100644 --- a/source/libs/sync/src/raft.c +++ b/source/libs/sync/src/raft.c @@ -14,7 +14,7 @@ */ #include "raft.h" -#include "raft_configuration.h" +#include "sync_raft_impl.h" #include "raft_log.h" #include "sync_raft_restore.h" #include "raft_replication.h" @@ -59,6 +59,11 @@ int32_t syncRaftStart(SSyncRaft* pRaft, const SSyncInfo* pInfo) { logStore = &(pRaft->logStore); fsm = &(pRaft->fsm); + pRaft->nodeInfoMap = taosHashInit(TSDB_MAX_REPLICA, taosGetDefaultHashFunction(TSDB_DATA_TYPE_INT), true, HASH_ENTRY_LOCK); + if (pRaft->nodeInfoMap == NULL) { + return -1; + } + // init progress tracker pRaft->tracker = syncRaftOpenProgressTracker(); if (pRaft->tracker == NULL) { @@ -290,8 +295,8 @@ static bool preHandleOldTermMessage(SSyncRaft* pRaft, const SSyncMessage* pMsg) * but it will not receive MsgApp or MsgHeartbeat, so it will not create * disruptive term increases **/ - int peerIndex = syncRaftConfigurationIndexOfNode(pRaft, pMsg->from); - if (peerIndex < 0) { + SNodeInfo* pNode = syncRaftGetNodeById(pRaft, pMsg->from); + if (pNode == NULL) { return true; } SSyncMessage* msg = syncNewEmptyAppendRespMsg(pRaft->selfGroupId, pRaft->selfId, pRaft->term); @@ -299,7 +304,7 @@ static bool preHandleOldTermMessage(SSyncRaft* pRaft, const SSyncMessage* pMsg) return true; } - pRaft->io.send(msg, &(pRaft->cluster.nodeInfo[peerIndex])); + pRaft->io.send(msg, pNode); } else { // ignore other cases syncInfo("[%d:%d] [term:%" PRId64 "] ignored a %d message with lower term from %d [term:%" PRId64 "]", diff --git a/source/libs/sync/src/raft_configuration.c b/source/libs/sync/src/raft_configuration.c deleted file mode 100644 index e16cb34989..0000000000 --- a/source/libs/sync/src/raft_configuration.c +++ /dev/null @@ -1,25 +0,0 @@ -/* - * 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 "raft_configuration.h" -#include "raft.h" - -int syncRaftConfigurationIndexOfNode(SSyncRaft *pRaft, SyncNodeId id) { - return (int)(id); -} - -int syncRaftConfigurationVoterCount(SSyncRaft *pRaft) { - return pRaft->cluster.replica; -} \ No newline at end of file diff --git a/source/libs/sync/src/raft_handle_append_entries_message.c b/source/libs/sync/src/raft_handle_append_entries_message.c index 4797b6ce03..92ebfe75f5 100644 --- a/source/libs/sync/src/raft_handle_append_entries_message.c +++ b/source/libs/sync/src/raft_handle_append_entries_message.c @@ -16,15 +16,14 @@ #include "syncInt.h" #include "raft.h" #include "raft_log.h" -#include "raft_configuration.h" +#include "sync_raft_impl.h" #include "raft_message.h" int syncRaftHandleAppendEntriesMessage(SSyncRaft* pRaft, const SSyncMessage* pMsg) { const RaftMsg_Append_Entries *appendEntries = &(pMsg->appendEntries); - int peerIndex = syncRaftConfigurationIndexOfNode(pRaft, pMsg->from); - - if (peerIndex < 0) { + SNodeInfo* pNode = syncRaftGetNodeById(pRaft, pMsg->from); + if (pNode == NULL) { return 0; } @@ -44,6 +43,6 @@ int syncRaftHandleAppendEntriesMessage(SSyncRaft* pRaft, const SSyncMessage* pMs pRaft->selfGroupId, pRaft->selfId, pMsg->from, appendEntries->index); out: - pRaft->io.send(pRespMsg, &(pRaft->cluster.nodeInfo[peerIndex])); + pRaft->io.send(pRespMsg, pNode); return 0; } \ No newline at end of file diff --git a/source/libs/sync/src/raft_handle_vote_message.c b/source/libs/sync/src/raft_handle_vote_message.c index 709e319c3e..9997c5226d 100644 --- a/source/libs/sync/src/raft_handle_vote_message.c +++ b/source/libs/sync/src/raft_handle_vote_message.c @@ -15,7 +15,7 @@ #include "syncInt.h" #include "raft.h" -#include "raft_configuration.h" +#include "sync_raft_impl.h" #include "raft_log.h" #include "raft_message.h" @@ -23,10 +23,11 @@ static bool canGrantVoteMessage(SSyncRaft* pRaft, const SSyncMessage* pMsg); int syncRaftHandleVoteMessage(SSyncRaft* pRaft, const SSyncMessage* pMsg) { SSyncMessage* pRespMsg; - int voteIndex = syncRaftConfigurationIndexOfNode(pRaft, pMsg->from); - if (voteIndex == -1) { + SNodeInfo* pNode = syncRaftGetNodeById(pRaft, pMsg->from); + if (pNode == NULL) { return 0; } + bool grant; SyncIndex lastIndex = syncRaftLogLastIndex(pRaft->log); SyncTerm lastTerm = syncRaftLogLastTerm(pRaft->log); @@ -42,7 +43,7 @@ int syncRaftHandleVoteMessage(SSyncRaft* pRaft, const SSyncMessage* pMsg) { grant ? "grant" : "reject", pMsg->from, pMsg->vote.lastTerm, pMsg->vote.lastIndex, pRaft->term); - pRaft->io.send(pRespMsg, &(pRaft->cluster.nodeInfo[voteIndex])); + pRaft->io.send(pRespMsg, pNode); return 0; } diff --git a/source/libs/sync/src/raft_handle_vote_resp_message.c b/source/libs/sync/src/raft_handle_vote_resp_message.c index 1781205ec0..744d654cc5 100644 --- a/source/libs/sync/src/raft_handle_vote_resp_message.c +++ b/source/libs/sync/src/raft_handle_vote_resp_message.c @@ -15,7 +15,7 @@ #include "syncInt.h" #include "raft.h" -#include "raft_configuration.h" +#include "sync_raft_impl.h" #include "raft_message.h" int syncRaftHandleVoteRespMessage(SSyncRaft* pRaft, const SSyncMessage* pMsg) { @@ -25,8 +25,8 @@ int syncRaftHandleVoteRespMessage(SSyncRaft* pRaft, const SSyncMessage* pMsg) { assert(pRaft->state == TAOS_SYNC_STATE_CANDIDATE); - voterIndex = syncRaftConfigurationIndexOfNode(pRaft, pMsg->from); - if (voterIndex == -1) { + SNodeInfo* pNode = syncRaftGetNodeById(pRaft, pMsg->from); + if (pNode == NULL) { syncError("[%d:%d] recv vote resp from unknown server %d", pRaft->selfGroupId, pRaft->selfId, pMsg->from); return 0; } diff --git a/source/libs/sync/src/raft_replication.c b/source/libs/sync/src/raft_replication.c index 74f40179c6..228d8195f6 100644 --- a/source/libs/sync/src/raft_replication.c +++ b/source/libs/sync/src/raft_replication.c @@ -69,9 +69,12 @@ static bool sendSnapshot(SSyncRaft* pRaft, SSyncRaftProgress* progress) { static bool sendAppendEntries(SSyncRaft* pRaft, SSyncRaftProgress* progress, SyncIndex prevIndex, SyncTerm prevTerm, SSyncRaftEntry *entries, int nEntry) { + SNodeInfo* pNode = syncRaftGetNodeById(pRaft, progress->id); + if (pNode == NULL) { + return false; + } SyncIndex lastIndex; - SyncTerm logTerm = prevTerm; - SNodeInfo* pNode = &(pRaft->cluster.nodeInfo[progress->selfIndex]); + SyncTerm logTerm = prevTerm; SSyncMessage* msg = syncNewAppendMsg(pRaft->selfGroupId, pRaft->selfId, pRaft->term, prevIndex, prevTerm, pRaft->log->commitIndex, diff --git a/source/libs/sync/src/sync_raft_election.c b/source/libs/sync/src/sync_raft_election.c index 74c3e09dae..b5649d5c5e 100644 --- a/source/libs/sync/src/sync_raft_election.c +++ b/source/libs/sync/src/sync_raft_election.c @@ -105,6 +105,6 @@ static void campaign(SSyncRaft* pRaft, ESyncRaftElectionType cType) { pRaft->selfGroupId, pRaft->selfId, lastTerm, lastIndex, nodeId, pRaft->term); - pRaft->io.send(pMsg, &(pRaft->cluster.nodeInfo[i])); + //pRaft->io.send(pMsg, &(pRaft->cluster.nodeInfo[i])); } } \ No newline at end of file diff --git a/source/libs/sync/src/sync_raft_impl.c b/source/libs/sync/src/sync_raft_impl.c index ab2db10230..6ec0c6c089 100644 --- a/source/libs/sync/src/sync_raft_impl.c +++ b/source/libs/sync/src/sync_raft_impl.c @@ -14,7 +14,7 @@ */ #include "raft.h" -#include "raft_configuration.h" +#include "sync_raft_impl.h" #include "raft_log.h" #include "raft_replication.h" #include "sync_raft_progress_tracker.h" @@ -123,15 +123,16 @@ bool syncRaftIsPastElectionTimeout(SSyncRaft* pRaft) { } int syncRaftQuorum(SSyncRaft* pRaft) { - return pRaft->cluster.replica / 2 + 1; + return 0; + //return pRaft->cluster.replica / 2 + 1; } ESyncRaftVoteResult syncRaftPollVote(SSyncRaft* pRaft, SyncNodeId id, bool preVote, bool grant, int* rejected, int *granted) { - int voterIndex = syncRaftConfigurationIndexOfNode(pRaft, id); - if (voterIndex == -1) { - return SYNC_RAFT_VOTE_PENDING; + SNodeInfo* pNode = syncRaftGetNodeById(pRaft, id); + if (pNode == NULL) { + return true; } if (grant) { @@ -142,7 +143,7 @@ ESyncRaftVoteResult syncRaftPollVote(SSyncRaft* pRaft, SyncNodeId id, pRaft->selfGroupId, pRaft->selfId, preVote, id, pRaft->term); } - syncRaftRecordVote(pRaft->tracker, voterIndex, grant); + syncRaftRecordVote(pRaft->tracker, pNode->nodeId, grant); return syncRaftTallyVotes(pRaft->tracker, rejected, granted); } /* @@ -154,7 +155,7 @@ ESyncRaftVoteResult syncRaftPollVote(SSyncRaft* pRaft, SyncNodeId id, pRaft->selfGroupId, pRaft->selfId, id, pRaft->term); } - int voteIndex = syncRaftConfigurationIndexOfNode(pRaft, id); + int voteIndex = syncRaftGetNodeById(pRaft, id); assert(voteIndex < pRaft->cluster.replica && voteIndex >= 0); assert(pRaft->candidateState.votes[voteIndex] == SYNC_RAFT_VOTE_RESP_UNKNOWN); @@ -198,6 +199,15 @@ void syncRaftBroadcastAppend(SSyncRaft* pRaft) { syncRaftProgressVisit(pRaft->tracker, visitProgressSendAppend, pRaft); } +SNodeInfo* syncRaftGetNodeById(SSyncRaft *pRaft, SyncNodeId id) { + SNodeInfo **ppNode = taosHashGet(pRaft->nodeInfoMap, &id, sizeof(SNodeInfo)); + if (ppNode != NULL) { + return *ppNode; + } + + return NULL; +} + static int convertClear(SSyncRaft* pRaft) { } @@ -269,7 +279,7 @@ static void appendEntries(SSyncRaft* pRaft, SSyncRaftEntry* entries, int n) { syncRaftLogAppend(pRaft->log, entries, n); - SSyncRaftProgress* progress = &(pRaft->tracker->progressMap.progress[pRaft->cluster.selfIndex]); + SSyncRaftProgress* progress = &(pRaft->tracker->progressMap.progress[pRaft->selfIndex]); syncRaftProgressMaybeUpdate(progress, lastIndex); // Regardless of syncRaftMaybeCommit's return, our caller will call bcastAppend. syncRaftMaybeCommit(pRaft); From 1d874657f7b982b6f29c5177b59f6ed6e9c4d8a9 Mon Sep 17 00:00:00 2001 From: lichuang Date: Wed, 17 Nov 2021 09:12:54 +0800 Subject: [PATCH 03/44] [TD-10645][raft]add node map --- .../libs/sync/inc/sync_raft_progress_tracker.h | 7 +++++-- source/libs/sync/inc/sync_raft_quorum_joint.h | 3 ++- .../libs/sync/inc/sync_raft_quorum_majority.h | 3 ++- source/libs/sync/inc/sync_type.h | 3 --- source/libs/sync/src/sync.c | 6 +++--- source/libs/sync/src/sync_raft_election.c | 7 ++++++- source/libs/sync/src/sync_raft_impl.c | 2 +- .../libs/sync/src/sync_raft_progress_tracker.c | 17 ++++++++++------- source/libs/sync/src/sync_raft_quorum_joint.c | 6 +++--- .../libs/sync/src/sync_raft_quorum_majority.c | 10 +++++++--- 10 files changed, 39 insertions(+), 25 deletions(-) diff --git a/source/libs/sync/inc/sync_raft_progress_tracker.h b/source/libs/sync/inc/sync_raft_progress_tracker.h index 8adb0b4736..3a448290c8 100644 --- a/source/libs/sync/inc/sync_raft_progress_tracker.h +++ b/source/libs/sync/inc/sync_raft_progress_tracker.h @@ -21,6 +21,7 @@ #include "sync_raft_quorum_joint.h" #include "sync_raft_progress.h" #include "sync_raft_proto.h" +#include "thash.h" struct SSyncRaftProgressTrackerConfig { SSyncRaftQuorumJointConfig voters; @@ -83,7 +84,9 @@ struct SSyncRaftProgressTracker { SSyncRaftProgressMap progressMap; - ESyncRaftVoteType votes[TSDB_MAX_REPLICA]; + // nodeid -> ESyncRaftVoteType map + SHashObj* votesMap; + int maxInflightMsgs; }; @@ -98,7 +101,7 @@ void syncRaftProgressVisit(SSyncRaftProgressTracker*, visitProgressFp visit, voi * syncRaftRecordVote records that the node with the given id voted for this Raft * instance if v == true (and declined it otherwise). **/ -void syncRaftRecordVote(SSyncRaftProgressTracker* tracker, int i, bool grant); +void syncRaftRecordVote(SSyncRaftProgressTracker* tracker, SyncNodeId id, bool grant); void syncRaftCloneTrackerConfig(const SSyncRaftProgressTrackerConfig* config, SSyncRaftProgressTrackerConfig* result); diff --git a/source/libs/sync/inc/sync_raft_quorum_joint.h b/source/libs/sync/inc/sync_raft_quorum_joint.h index 0637a9be9a..4a5b749a0e 100644 --- a/source/libs/sync/inc/sync_raft_quorum_joint.h +++ b/source/libs/sync/inc/sync_raft_quorum_joint.h @@ -20,6 +20,7 @@ #include "sync.h" #include "sync_type.h" #include "sync_raft_node_map.h" +#include "thash.h" /** * SSyncRaftQuorumJointConfig is a configuration of two groups of (possibly overlapping) @@ -35,7 +36,7 @@ typedef struct SSyncRaftQuorumJointConfig { * a result indicating whether the vote is pending, lost, or won. A joint quorum * requires both majority quorums to vote in favor. **/ -ESyncRaftVoteType syncRaftVoteResult(SSyncRaftQuorumJointConfig* config, const ESyncRaftVoteType* votes); +ESyncRaftVoteType syncRaftVoteResult(SSyncRaftQuorumJointConfig* config, SHashObj* votesMap); static FORCE_INLINE bool syncRaftJointConfigInOutgoing(const SSyncRaftQuorumJointConfig* config, SyncNodeId id) { return syncRaftIsInNodeMap(&config->outgoing, id); diff --git a/source/libs/sync/inc/sync_raft_quorum_majority.h b/source/libs/sync/inc/sync_raft_quorum_majority.h index 0512a4dc87..38df40147a 100644 --- a/source/libs/sync/inc/sync_raft_quorum_majority.h +++ b/source/libs/sync/inc/sync_raft_quorum_majority.h @@ -19,6 +19,7 @@ #include "sync.h" #include "sync_type.h" #include "sync_raft_quorum.h" +#include "thash.h" /** * syncRaftMajorityVoteResult takes a mapping of voters to yes/no (true/false) votes and returns @@ -26,6 +27,6 @@ * yes/no has been reached), won (a quorum of yes has been reached), or lost (a * quorum of no has been reached). **/ -ESyncRaftVoteResult syncRaftMajorityVoteResult(SSyncRaftNodeMap* config, const ESyncRaftVoteType* votes); +ESyncRaftVoteResult syncRaftMajorityVoteResult(SSyncRaftNodeMap* config, SHashObj* votesMap); #endif /* _TD_LIBS_SYNC_RAFT_QUORUM_MAJORITY_H */ diff --git a/source/libs/sync/inc/sync_type.h b/source/libs/sync/inc/sync_type.h index 6d29e019cc..fcb0940609 100644 --- a/source/libs/sync/inc/sync_type.h +++ b/source/libs/sync/inc/sync_type.h @@ -77,9 +77,6 @@ typedef enum { } ESyncRaftElectionType; typedef enum { - // the init vote resp status - SYNC_RAFT_VOTE_RESP_UNKNOWN = 0, - // grant the vote request SYNC_RAFT_VOTE_RESP_GRANT = 1, diff --git a/source/libs/sync/src/sync.c b/source/libs/sync/src/sync.c index 2f25828d5d..06af8ff6c2 100644 --- a/source/libs/sync/src/sync.c +++ b/source/libs/sync/src/sync.c @@ -99,7 +99,7 @@ void syncCleanUp() { SSyncNode* syncStart(const SSyncInfo* pInfo) { pthread_mutex_lock(&gSyncManager->mutex); - SSyncNode **ppNode = taosHashGet(gSyncManager->vgroupTable, &pInfo->vgId, sizeof(SyncGroupId)); + SSyncNode **ppNode = taosHashGet(gSyncManager->vgroupTable, &pInfo->vgId, sizeof(SyncGroupId*)); if (ppNode != NULL) { syncInfo("vgroup %d already exist", pInfo->vgId); pthread_mutex_unlock(&gSyncManager->mutex); @@ -140,7 +140,7 @@ SSyncNode* syncStart(const SSyncInfo* pInfo) { void syncStop(const SSyncNode* pNode) { pthread_mutex_lock(&gSyncManager->mutex); - SSyncNode **ppNode = taosHashGet(gSyncManager->vgroupTable, &pNode->vgId, sizeof(SyncGroupId)); + SSyncNode **ppNode = taosHashGet(gSyncManager->vgroupTable, &pNode->vgId, sizeof(SyncGroupId*)); if (ppNode == NULL) { syncInfo("vgroup %d not exist", pNode->vgId); pthread_mutex_unlock(&gSyncManager->mutex); @@ -288,7 +288,7 @@ static void *syncWorkerMain(void *argv) { static void syncNodeTick(void *param, void *tmrId) { SyncGroupId vgId = (SyncGroupId)param; - SSyncNode **ppNode = taosHashGet(gSyncManager->vgroupTable, &vgId, sizeof(SyncGroupId)); + SSyncNode **ppNode = taosHashGet(gSyncManager->vgroupTable, &vgId, sizeof(SyncGroupId*)); if (ppNode == NULL) { return; } diff --git a/source/libs/sync/src/sync_raft_election.c b/source/libs/sync/src/sync_raft_election.c index b5649d5c5e..4423e1ac3c 100644 --- a/source/libs/sync/src/sync_raft_election.c +++ b/source/libs/sync/src/sync_raft_election.c @@ -95,6 +95,11 @@ static void campaign(SSyncRaft* pRaft, ESyncRaftElectionType cType) { continue; } + SNodeInfo* pNode = syncRaftGetNodeById(pRaft, nodeId); + if (pNode == NULL) { + continue; + } + SSyncMessage* pMsg = syncNewVoteMsg(pRaft->selfGroupId, pRaft->selfId, term, cType, lastIndex, lastTerm); if (pMsg == NULL) { @@ -105,6 +110,6 @@ static void campaign(SSyncRaft* pRaft, ESyncRaftElectionType cType) { pRaft->selfGroupId, pRaft->selfId, lastTerm, lastIndex, nodeId, pRaft->term); - //pRaft->io.send(pMsg, &(pRaft->cluster.nodeInfo[i])); + pRaft->io.send(pMsg, pNode); } } \ No newline at end of file diff --git a/source/libs/sync/src/sync_raft_impl.c b/source/libs/sync/src/sync_raft_impl.c index 6ec0c6c089..73a02c4b80 100644 --- a/source/libs/sync/src/sync_raft_impl.c +++ b/source/libs/sync/src/sync_raft_impl.c @@ -200,7 +200,7 @@ void syncRaftBroadcastAppend(SSyncRaft* pRaft) { } SNodeInfo* syncRaftGetNodeById(SSyncRaft *pRaft, SyncNodeId id) { - SNodeInfo **ppNode = taosHashGet(pRaft->nodeInfoMap, &id, sizeof(SNodeInfo)); + SNodeInfo **ppNode = taosHashGet(pRaft->nodeInfoMap, &id, sizeof(SyncNodeId*)); if (ppNode != NULL) { return *ppNode; } diff --git a/source/libs/sync/src/sync_raft_progress_tracker.c b/source/libs/sync/src/sync_raft_progress_tracker.c index f43414127d..1407df059a 100644 --- a/source/libs/sync/src/sync_raft_progress_tracker.c +++ b/source/libs/sync/src/sync_raft_progress_tracker.c @@ -26,7 +26,7 @@ SSyncRaftProgressTracker* syncRaftOpenProgressTracker() { } void syncRaftResetVotes(SSyncRaftProgressTracker* tracker) { - memset(tracker->votes, SYNC_RAFT_VOTE_RESP_UNKNOWN, sizeof(ESyncRaftVoteType) * TSDB_MAX_REPLICA); + taosHashClear(tracker->votesMap); } void syncRaftProgressVisit(SSyncRaftProgressTracker* tracker, visitProgressFp visit, void* arg) { @@ -37,12 +37,14 @@ void syncRaftProgressVisit(SSyncRaftProgressTracker* tracker, visitProgressFp vi } } -void syncRaftRecordVote(SSyncRaftProgressTracker* tracker, int i, bool grant) { - if (tracker->votes[i] != SYNC_RAFT_VOTE_RESP_UNKNOWN) { +void syncRaftRecordVote(SSyncRaftProgressTracker* tracker, SyncNodeId id, bool grant) { + ESyncRaftVoteType* pType = taosHashGet(tracker->votesMap, &id, sizeof(SyncNodeId*)); + if (pType != NULL) { return; } - tracker->votes[i] = grant ? SYNC_RAFT_VOTE_RESP_GRANT : SYNC_RAFT_VOTE_RESP_REJECT; + ESyncRaftVoteType type = grant ? SYNC_RAFT_VOTE_RESP_GRANT : SYNC_RAFT_VOTE_RESP_REJECT; + taosHashPut(tracker->votesMap, &id, sizeof(SyncNodeId), &type, sizeof(ESyncRaftVoteType*)); } void syncRaftCloneTrackerConfig(const SSyncRaftProgressTrackerConfig* from, SSyncRaftProgressTrackerConfig* to) { @@ -68,11 +70,12 @@ ESyncRaftVoteResult syncRaftTallyVotes(SSyncRaftProgressTracker* tracker, int* r continue; } - if (tracker->votes[i] == SYNC_RAFT_VOTE_RESP_UNKNOWN) { + ESyncRaftVoteType* pType = taosHashGet(tracker->votesMap, &progress->id, sizeof(SyncNodeId*)); + if (pType == NULL) { continue; } - if (tracker->votes[i] == SYNC_RAFT_VOTE_RESP_GRANT) { + if (*pType == SYNC_RAFT_VOTE_RESP_GRANT) { g++; } else { r++; @@ -81,7 +84,7 @@ ESyncRaftVoteResult syncRaftTallyVotes(SSyncRaftProgressTracker* tracker, int* r if (rejected) *rejected = r; if (granted) *granted = g; - return syncRaftVoteResult(&(tracker->config.voters), tracker->votes); + return syncRaftVoteResult(&(tracker->config.voters), tracker->votesMap); } void syncRaftConfigState(const SSyncRaftProgressTracker* tracker, SSyncConfigState* cs) { diff --git a/source/libs/sync/src/sync_raft_quorum_joint.c b/source/libs/sync/src/sync_raft_quorum_joint.c index 8a99574d68..2383d7ee63 100644 --- a/source/libs/sync/src/sync_raft_quorum_joint.c +++ b/source/libs/sync/src/sync_raft_quorum_joint.c @@ -23,9 +23,9 @@ * a result indicating whether the vote is pending, lost, or won. A joint quorum * requires both majority quorums to vote in favor. **/ -ESyncRaftVoteType syncRaftVoteResult(SSyncRaftQuorumJointConfig* config, const ESyncRaftVoteType* votes) { - ESyncRaftVoteResult r1 = syncRaftMajorityVoteResult(&(config->incoming), votes); - ESyncRaftVoteResult r2 = syncRaftMajorityVoteResult(&(config->outgoing), votes); +ESyncRaftVoteType syncRaftVoteResult(SSyncRaftQuorumJointConfig* config, SHashObj* votesMap) { + ESyncRaftVoteResult r1 = syncRaftMajorityVoteResult(&(config->incoming), votesMap); + ESyncRaftVoteResult r2 = syncRaftMajorityVoteResult(&(config->outgoing), votesMap); if (r1 == r2) { // If they agree, return the agreed state. diff --git a/source/libs/sync/src/sync_raft_quorum_majority.c b/source/libs/sync/src/sync_raft_quorum_majority.c index 8ff5752b97..014a8c7303 100644 --- a/source/libs/sync/src/sync_raft_quorum_majority.c +++ b/source/libs/sync/src/sync_raft_quorum_majority.c @@ -23,7 +23,7 @@ * yes/no has been reached), won (a quorum of yes has been reached), or lost (a * quorum of no has been reached). **/ -ESyncRaftVoteResult syncRaftMajorityVoteResult(SSyncRaftNodeMap* config, const ESyncRaftVoteType* votes) { +ESyncRaftVoteResult syncRaftMajorityVoteResult(SSyncRaftNodeMap* config, SHashObj* votesMap) { if (config->replica == 0) { return SYNC_RAFT_VOTE_WON; } @@ -34,9 +34,13 @@ ESyncRaftVoteResult syncRaftMajorityVoteResult(SSyncRaftNodeMap* config, const E continue; } - if (votes[i] == SYNC_RAFT_VOTE_RESP_UNKNOWN) { + const ESyncRaftVoteType* pType = taosHashGet(votesMap, &config->nodeId[i], sizeof(SyncNodeId*)); + if (pType == NULL) { missing += 1; - } else if (votes[i] == SYNC_RAFT_VOTE_RESP_GRANT) { + continue; + } + + if (*pType == SYNC_RAFT_VOTE_RESP_GRANT) { g +=1; } else { r += 1; From df2530f969949ea74190626f1ab6bc0cfa200b61 Mon Sep 17 00:00:00 2001 From: lichuang Date: Wed, 17 Nov 2021 11:04:38 +0800 Subject: [PATCH 04/44] [TD-10645][raft]refactor node and progress map --- source/libs/sync/inc/sync_raft_node_map.h | 16 ++- source/libs/sync/inc/sync_raft_progress.h | 15 +-- source/libs/sync/src/raft.c | 4 +- .../libs/sync/src/sync_raft_config_change.c | 115 +++++++----------- source/libs/sync/src/sync_raft_election.c | 10 +- source/libs/sync/src/sync_raft_node_map.c | 69 +++++------ source/libs/sync/src/sync_raft_progress.c | 49 +++----- .../sync/src/sync_raft_progress_tracker.c | 11 +- source/libs/sync/src/sync_raft_quorum_joint.c | 29 +---- 9 files changed, 141 insertions(+), 177 deletions(-) diff --git a/source/libs/sync/inc/sync_raft_node_map.h b/source/libs/sync/inc/sync_raft_node_map.h index bfb5f68489..5d43e42061 100644 --- a/source/libs/sync/inc/sync_raft_node_map.h +++ b/source/libs/sync/inc/sync_raft_node_map.h @@ -16,15 +16,18 @@ #ifndef _TD_LIBS_SYNC_RAFT_NODE_MAP_H #define _TD_LIBS_SYNC_RAFT_NODE_MAP_H +#include "thash.h" #include "sync.h" #include "sync_type.h" -// TODO: is TSDB_MAX_REPLICA enough? struct SSyncRaftNodeMap { - int32_t replica; - SyncNodeId nodeId[TSDB_MAX_REPLICA]; + SHashObj* nodeIdMap; }; +void syncRaftInitNodeMap(SSyncRaftNodeMap* nodeMap); + +void syncRaftClearNodeMap(SSyncRaftNodeMap* nodeMap); + bool syncRaftIsInNodeMap(const SSyncRaftNodeMap* nodeMap, SyncNodeId nodeId); void syncRaftCopyNodeMap(const SSyncRaftNodeMap* nodeMap, SSyncRaftNodeMap* to); @@ -33,4 +36,11 @@ void syncRaftUnionNodeMap(const SSyncRaftNodeMap* nodeMap, SSyncRaftNodeMap* to) void syncRaftAddToNodeMap(SSyncRaftNodeMap* nodeMap, SyncNodeId nodeId); +void syncRaftRemoveFromNodeMap(SSyncRaftNodeMap* nodeMap, SyncNodeId nodeId); + +int32_t syncRaftNodeMapSize(const SSyncRaftNodeMap* nodeMap); + +// return true if reach the end +bool syncRaftIterateNodeMap(const SSyncRaftNodeMap* nodeMap, SyncNodeId *pId); + #endif /* _TD_LIBS_SYNC_RAFT_NODE_MAP_H */ \ No newline at end of file diff --git a/source/libs/sync/inc/sync_raft_progress.h b/source/libs/sync/inc/sync_raft_progress.h index 173608a40a..3d2995ed77 100644 --- a/source/libs/sync/inc/sync_raft_progress.h +++ b/source/libs/sync/inc/sync_raft_progress.h @@ -18,6 +18,7 @@ #include "sync_type.h" #include "sync_raft_inflights.h" +#include "thash.h" /** * State defines how the leader should interact with the follower. @@ -69,8 +70,7 @@ static const char* kProgressStateString[] = { * progresses of all followers, and sends entries to the follower based on its progress. **/ struct SSyncRaftProgress { - // index in raft cluster config - int selfIndex; + SyncGroupId groupId; SyncNodeId id; @@ -139,10 +139,10 @@ struct SSyncRaftProgress { }; struct SSyncRaftProgressMap { - SSyncRaftProgress progress[TSDB_MAX_REPLICA]; + // map nodeId -> SSyncRaftProgress* + SHashObj* progressMap; }; - static FORCE_INLINE const char* syncRaftProgressStateString(const SSyncRaftProgress* progress) { return kProgressStateString[progress->state]; } @@ -221,9 +221,9 @@ static FORCE_INLINE bool syncRaftProgressRecentActive(SSyncRaftProgress* progres return progress->recentActive; } -int syncRaftFindProgressIndexByNodeId(const SSyncRaftProgressMap* progressMap, SyncNodeId id); +SSyncRaftProgress* syncRaftFindProgressByNodeId(const SSyncRaftProgressMap* progressMap, SyncNodeId id); -int syncRaftAddToProgressMap(SSyncRaftProgressMap* progressMap, SyncNodeId id); +int syncRaftAddToProgressMap(SSyncRaftProgressMap* progressMap, SSyncRaftProgress* progress); void syncRaftRemoveFromProgressMap(SSyncRaftProgressMap* progressMap, SyncNodeId id); @@ -236,7 +236,8 @@ void syncRaftProgressBecomeSnapshot(SSyncRaftProgress* progress, SyncIndex snaps void syncRaftCopyProgress(const SSyncRaftProgress* from, SSyncRaftProgress* to); -void syncRaftProgressMapCopy(const SSyncRaftProgressMap* from, SSyncRaftProgressMap* to); +// return true if reach the end +bool syncRaftIterateProgressMap(const SSyncRaftNodeMap* nodeMap, SSyncRaftProgress *pProgress); #if 0 diff --git a/source/libs/sync/src/raft.c b/source/libs/sync/src/raft.c index d39e047492..b2170e0b68 100644 --- a/source/libs/sync/src/raft.c +++ b/source/libs/sync/src/raft.c @@ -175,7 +175,7 @@ static void switchToConfig(SSyncRaft* pRaft, const SSyncRaftProgressTrackerConfi SSyncRaftProgress* progress = NULL; syncRaftConfigState(pRaft->tracker, cs); - i = syncRaftFindProgressIndexByNodeId(&pRaft->tracker->progressMap, selfId); + i = syncRaftFindProgressByNodeId(&pRaft->tracker->progressMap, selfId); exist = (i != -1); // Update whether the node itself is a learner, resetting to false when the @@ -202,7 +202,7 @@ static void switchToConfig(SSyncRaft* pRaft, const SSyncRaftProgressTrackerConfi // The remaining steps only make sense if this node is the leader and there // are other nodes. - if (pRaft->state != TAOS_SYNC_STATE_LEADER || cs->voters.replica == 0) { + if (pRaft->state != TAOS_SYNC_STATE_LEADER || syncRaftNodeMapSize(&cs->voters) == 0) { return; } diff --git a/source/libs/sync/src/sync_raft_config_change.c b/source/libs/sync/src/sync_raft_config_change.c index 288fdc465e..ad261dd277 100644 --- a/source/libs/sync/src/sync_raft_config_change.c +++ b/source/libs/sync/src/sync_raft_config_change.c @@ -101,7 +101,7 @@ int syncRaftChangerEnterJoint(SSyncRaftChanger* changer, bool autoLeave, const S return -1; } - if(config->voters.incoming.replica == 0) { + if(syncRaftNodeMapSize(&config->voters.incoming) == 0) { // We allow adding nodes to an empty config for convenience (testing and // bootstrap), but you can't enter a joint state. syncError("can't make a zero-voter config joint"); @@ -112,7 +112,7 @@ int syncRaftChangerEnterJoint(SSyncRaftChanger* changer, bool autoLeave, const S syncRaftJointConfigClearOutgoing(&config->voters); // Copy incoming to outgoing. - memcpy(&config->voters.outgoing, &config->voters.incoming, sizeof(SSyncCluster)); + syncRaftCopyNodeMap(&config->voters.incoming, &config->voters.outgoing); ret = applyConfig(changer, config, progressMap, css); if (ret != 0) { @@ -129,13 +129,12 @@ int syncRaftChangerEnterJoint(SSyncRaftChanger* changer, bool autoLeave, const S static int checkAndCopy(SSyncRaftChanger* changer, SSyncRaftProgressTrackerConfig* config, SSyncRaftProgressMap* progressMap) { syncRaftCloneTrackerConfig(&changer->tracker->config, config); int i; - for (i = 0; i < TSDB_MAX_REPLICA; ++i) { - SSyncRaftProgress* progress = &(changer->tracker->progressMap.progress[i]); - if (progress->id == SYNC_NON_NODE_ID) { - continue; - } - syncRaftCopyProgress(progress, &(progressMap->progress[i])); + + SSyncRaftProgress* pProgress = NULL; + while (!syncRaftIterateProgressMap(&changer->tracker->progressMap, pProgress)) { + syncRaftAddToProgressMap(progressMap, pProgress); } + return checkAndReturn(config, progressMap); } @@ -158,33 +157,44 @@ static int checkInvariants(SSyncRaftProgressTrackerConfig* config, SSyncRaftProg return ret; } - int i; // Any staged learner was staged because it could not be directly added due // to a conflicting voter in the outgoing config. - for (i = 0; i < TSDB_MAX_REPLICA; ++i) { - if (!syncRaftJointConfigInOutgoing(&config->voters, config->learnersNext.nodeId[i])) { + SyncNodeId* pNodeId = NULL; + while (!syncRaftIterateNodeMap(&config->learnersNext, pNodeId)) { + SyncNodeId nodeId = *pNodeId; + if (!syncRaftJointConfigInOutgoing(&config->voters, nodeId)) { return -1; } - if (progressMap->progress[i].id != SYNC_NON_NODE_ID && progressMap->progress[i].isLearner) { - syncError("%d is in LearnersNext, but is already marked as learner", progressMap->progress[i].id); + SSyncRaftProgress* progress = syncRaftFindProgressByNodeId(progressMap, nodeId); + assert(progress); + assert(progress->id == nodeId); + if (progress->isLearner) { + syncError("[%d:%d] is in LearnersNext, but is already marked as learner", progress->groupId, nodeId); return -1; } } + // Conversely Learners and Voters doesn't intersect at all. - for (i = 0; i < TSDB_MAX_REPLICA; ++i) { - if (syncRaftJointConfigInIncoming(&config->voters, config->learners.nodeId[i])) { - syncError("%d is in Learners and voter.incoming", progressMap->progress[i].id); + SyncNodeId* pNodeId = NULL; + while (!syncRaftIterateNodeMap(&config->learners, pNodeId)) { + SyncNodeId nodeId = *pNodeId; + if (syncRaftJointConfigInIncoming(&config->voters, nodeId)) { + syncError("%d is in Learners and voter.incoming", nodeId); return -1; } - if (progressMap->progress[i].id != SYNC_NON_NODE_ID && !progressMap->progress[i].isLearner) { - syncError("%d is in Learners, but is not marked as learner", progressMap->progress[i].id); + SSyncRaftProgress* progress = syncRaftFindProgressByNodeId(progressMap, nodeId); + assert(progress); + assert(progress->id == nodeId); + + if (!progress->isLearner) { + syncError("[%d:%d] is in Learners, but is not marked as learner", progress->groupId, nodeId); return -1; } } if (!hasJointConfig(config)) { // We enforce that empty maps are nil instead of zero. - if (config->learnersNext.replica > 0) { + if (syncRaftNodeMapSize(&config->learnersNext)) { syncError("cfg.LearnersNext must be nil when not joint"); return -1; } @@ -198,7 +208,7 @@ static int checkInvariants(SSyncRaftProgressTrackerConfig* config, SSyncRaftProg } static bool hasJointConfig(const SSyncRaftProgressTrackerConfig* config) { - return config->voters.outgoing.replica > 0; + return syncRaftNodeMapSize(&config->voters.outgoing) > 0; } static int applyConfig(SSyncRaftChanger* changer, SSyncRaftProgressTrackerConfig* config, @@ -227,7 +237,7 @@ static int applyConfig(SSyncRaftChanger* changer, SSyncRaftProgressTrackerConfig } } - if (config->voters.incoming.replica == 0) { + if (syncRaftNodeMapSize(&config->voters.incoming) == 0) { syncError("removed all voters"); return -1; } @@ -251,15 +261,10 @@ static int symDiff(const SSyncRaftNodeMap* l, const SSyncRaftNodeMap* r) { const SSyncRaftNodeMap* p0 = pp[0]; const SSyncRaftNodeMap* p1 = pp[1]; - for (j0 = 0; j0 < TSDB_MAX_REPLICA; ++j0) { - SyncNodeId id = p0->nodeId[j0]; - if (id == SYNC_NON_NODE_ID) { - continue; - } - for (j1 = 0; j1 < p1->replica; ++j1) { - if (p1->nodeId[j1] != SYNC_NON_NODE_ID && p1->nodeId[j1] != id) { - n+=1; - } + SyncNodeId* pNodeId; + while (!syncRaftIterateNodeMap(p0, pNodeId)) { + if (!syncRaftIsInNodeMap(p1, *pNodeId)) { + n+=1; } } } @@ -274,47 +279,23 @@ static void initProgress(SSyncRaftChanger* changer, SSyncRaftProgressTrackerConf // nilAwareDelete deletes from a map, nil'ing the map itself if it is empty after. static void nilAwareDelete(SSyncRaftNodeMap* nodeMap, SyncNodeId id) { - int i; - for (i = 0; i < TSDB_MAX_REPLICA; ++i) { - if (nodeMap->nodeId[i] == id) { - nodeMap->replica -= 1; - nodeMap->nodeId[i] = SYNC_NON_NODE_ID; - break; - } - } - - assert(nodeMap->replica >= 0); + syncRaftRemoveFromNodeMap(nodeMap, id); } // nilAwareAdd populates a map entry, creating the map if necessary. static void nilAwareAdd(SSyncRaftNodeMap* nodeMap, SyncNodeId id) { - int i, j; - for (i = 0, j = -1; i < TSDB_MAX_REPLICA; ++i) { - if (nodeMap->nodeId[i] == id) { - return; - } - if (j == -1 && nodeMap->nodeId[i] == SYNC_NON_NODE_ID) { - j = i; - } - } - - assert(j != -1); - nodeMap->nodeId[j] = id; - nodeMap->replica += 1; + syncRaftAddToNodeMap(nodeMap, id); } // makeVoter adds or promotes the given ID to be a voter in the incoming // majority config. static void makeVoter(SSyncRaftChanger* changer, SSyncRaftProgressTrackerConfig* config, SSyncRaftProgressMap* progressMap, SyncNodeId id) { - int i = syncRaftFindProgressIndexByNodeId(progressMap, id); - if (i == -1) { + SSyncRaftProgress* progress = syncRaftFindProgressByNodeId(progressMap, id); + if (progress == -1) { initProgress(changer, config, progressMap, id, false); - i = syncRaftFindProgressIndexByNodeId(progressMap, id); + return; } - - assert(i != -1); - SSyncRaftProgress* progress = &(progressMap->progress[i]); progress->isLearner = false; nilAwareDelete(&config->learners, id); @@ -337,14 +318,12 @@ static void makeVoter(SSyncRaftChanger* changer, SSyncRaftProgressTrackerConfig* // LeaveJoint(). static void makeLearner(SSyncRaftChanger* changer, SSyncRaftProgressTrackerConfig* config, SSyncRaftProgressMap* progressMap, SyncNodeId id) { - int i = syncRaftFindProgressIndexByNodeId(progressMap, id); - if (i == -1) { + SSyncRaftProgress* progress = syncRaftFindProgressByNodeId(progressMap, id); + if (progress == NULL) { initProgress(changer, config, progressMap, id, false); - i = syncRaftFindProgressIndexByNodeId(progressMap, id); + return; } - - assert(i != -1); - SSyncRaftProgress* progress = &(progressMap->progress[i]); + if (progress->isLearner) { return; } @@ -352,7 +331,7 @@ static void makeLearner(SSyncRaftChanger* changer, SSyncRaftProgressTrackerConfi removeNodeId(changer, config, progressMap, id); // ... but save the Progress. - syncRaftAddToProgressMap(progressMap, id); + syncRaftAddToProgressMap(progressMap, progress); // Use LearnersNext if we can't add the learner to Learners directly, i.e. // if the peer is still tracked as a voter in the outgoing config. It will @@ -371,8 +350,8 @@ static void makeLearner(SSyncRaftChanger* changer, SSyncRaftProgressTrackerConfi // removeNodeId this peer as a voter or learner from the incoming config. static void removeNodeId(SSyncRaftChanger* changer, SSyncRaftProgressTrackerConfig* config, SSyncRaftProgressMap* progressMap, SyncNodeId id) { - int i = syncRaftFindProgressIndexByNodeId(progressMap, id); - if (i == -1) { + SSyncRaftProgress* progress = syncRaftFindProgressByNodeId(progressMap, id); + if (progress == NULL) { return; } diff --git a/source/libs/sync/src/sync_raft_election.c b/source/libs/sync/src/sync_raft_election.c index 4423e1ac3c..d4013bbc08 100644 --- a/source/libs/sync/src/sync_raft_election.c +++ b/source/libs/sync/src/sync_raft_election.c @@ -85,8 +85,14 @@ static void campaign(SSyncRaft* pRaft, ESyncRaftElectionType cType) { SyncTerm lastTerm = syncRaftLogLastTerm(pRaft->log); SSyncRaftNodeMap nodeMap; syncRaftJointConfigIDS(&pRaft->tracker->config.voters, &nodeMap); - for (i = 0; i < TSDB_MAX_REPLICA; ++i) { - SyncNodeId nodeId = nodeMap.nodeId[i]; + SyncNodeId *pNodeId = NULL; + while (true) { + syncRaftIterateNodeMap(&nodeMap, &pNodeId); + if (pNodeId == NULL || *pNodeId == NULL) { + break; + } + + SyncNodeId nodeId = *pNodeId; if (nodeId == SYNC_NON_NODE_ID) { continue; } diff --git a/source/libs/sync/src/sync_raft_node_map.c b/source/libs/sync/src/sync_raft_node_map.c index e13c808075..5dacc01d1a 100644 --- a/source/libs/sync/src/sync_raft_node_map.c +++ b/source/libs/sync/src/sync_raft_node_map.c @@ -16,51 +16,52 @@ #include "sync_raft_node_map.h" #include "sync_type.h" +void syncRaftInitNodeMap(SSyncRaftNodeMap* nodeMap) { + nodeMap->nodeIdMap = taosHashInit(TSDB_MAX_REPLICA, taosGetDefaultHashFunction(TSDB_DATA_TYPE_INT), true, HASH_ENTRY_LOCK); +} + +void syncRaftClearNodeMap(SSyncRaftNodeMap* nodeMap) { + taosHashClear(nodeMap->nodeIdMap); +} + bool syncRaftIsInNodeMap(const SSyncRaftNodeMap* nodeMap, SyncNodeId nodeId) { - int i; - - for (i = 0; i < TSDB_MAX_REPLICA; ++i) { - if (nodeId == nodeMap->nodeId[i]) { - return true; - } + SyncNodeId** ppId = (SyncNodeId**)taosHashGet(nodeMap->nodeIdMap, &nodeId, sizeof(SyncNodeId*)); + if (ppId == NULL) { + return false; } - - return false; + return true; } void syncRaftCopyNodeMap(const SSyncRaftNodeMap* nodeMap, SSyncRaftNodeMap* to) { - memcpy(to, nodeMap, sizeof(SSyncRaftNodeMap)); + SyncNodeId** ppId = (SyncNodeId**)taosHashIterate(nodeMap->nodeIdMap, NULL); + while (ppId) { + taosHashPut(to->nodeIdMap, ppId, sizeof(SyncNodeId*), ppId, sizeof(SyncNodeId*)); + ppId = taosHashIterate(nodeMap->nodeIdMap, ppId); + } +} + +bool syncRaftIterateNodeMap(const SSyncRaftNodeMap* nodeMap, SyncNodeId *pId) { + SyncNodeId **ppId = taosHashIterate(nodeMap->nodeIdMap, pId); + if (ppId == NULL) { + return true; + } + + *pId = *(*ppId); + return false; } void syncRaftUnionNodeMap(const SSyncRaftNodeMap* nodeMap, SSyncRaftNodeMap* to) { - int i, j, m; - - for (i = 0; i < TSDB_MAX_REPLICA; ++i) { - SyncNodeId id = nodeMap->nodeId[i]; - if (id == SYNC_NON_NODE_ID) { - continue; - } - - syncRaftAddToNodeMap(to, id); - } + syncRaftCopyNodeMap(nodeMap, to); } void syncRaftAddToNodeMap(SSyncRaftNodeMap* nodeMap, SyncNodeId nodeId) { - assert(nodeMap->replica < TSDB_MAX_REPLICA); + taosHashPut(nodeMap->nodeIdMap, &nodeId, sizeof(SyncNodeId*), &nodeId, sizeof(SyncNodeId*)); +} - int i, j; - for (i = 0, j = -1; i < TSDB_MAX_REPLICA; ++i) { - SyncNodeId id = nodeMap->nodeId[i]; - if (id == SYNC_NON_NODE_ID) { - if (j == -1) j = i; - continue; - } - if (id == nodeId) { - return; - } - } +void syncRaftRemoveFromNodeMap(SSyncRaftNodeMap* nodeMap, SyncNodeId nodeId) { + taosHashRemove(nodeMap->nodeIdMap, &nodeId, sizeof(SyncNodeId*)); +} - assert(j != -1); - nodeMap->nodeId[j] = nodeId; - nodeMap->replica += 1; +int32_t syncRaftNodeMapSize(const SSyncRaftNodeMap* nodeMap) { + return taosHashGetSize(nodeMap); } \ No newline at end of file diff --git a/source/libs/sync/src/sync_raft_progress.c b/source/libs/sync/src/sync_raft_progress.c index a53aae93d0..436250e594 100644 --- a/source/libs/sync/src/sync_raft_progress.c +++ b/source/libs/sync/src/sync_raft_progress.c @@ -112,42 +112,21 @@ bool syncRaftProgressIsPaused(SSyncRaftProgress* progress) { } } -int syncRaftFindProgressIndexByNodeId(const SSyncRaftProgressMap* progressMap, SyncNodeId id) { - int i; - for (i = 0; i < TSDB_MAX_REPLICA; ++i) { - if (progressMap->progress[i].id == id) { - return i; - } +SSyncRaftProgress* syncRaftFindProgressByNodeId(const SSyncRaftProgressMap* progressMap, SyncNodeId id) { + SSyncRaftProgress** ppProgress = (SSyncRaftProgress**)taosHashGet(progressMap, &id, sizeof(SyncNodeId*)); + if (ppProgress == NULL) { + return NULL; } - return -1; + + return *ppProgress; } -int syncRaftAddToProgressMap(SSyncRaftProgressMap* progressMap, SyncNodeId id) { - int i, j; - - for (i = 0, j = -1; i < TSDB_MAX_REPLICA; ++i) { - if (progressMap->progress[i].id == id) { - return i; - } - if (j == -1 && progressMap->progress[i].id == SYNC_NON_NODE_ID) { - j = i; - } - } - - assert(j != -1); - - progressMap->progress[i].id = id; +int syncRaftAddToProgressMap(SSyncRaftProgressMap* progressMap, SSyncRaftProgress* progress) { + taosHashPut(progressMap->progressMap, &progress->id, sizeof(SyncNodeId*), &progress, sizeof(SSyncRaftProgress*)); } void syncRaftRemoveFromProgressMap(SSyncRaftProgressMap* progressMap, SyncNodeId id) { - int i; - - for (i = 0; i < TSDB_MAX_REPLICA; ++i) { - if (progressMap->progress[i].id == id) { - progressMap->progress[i].id = SYNC_NON_NODE_ID; - break; - } - } + taosHashRemove(progressMap->progressMap, &id, sizeof(SyncNodeId*)); } bool syncRaftProgressIsUptodate(SSyncRaft* pRaft, SSyncRaftProgress* progress) { @@ -188,7 +167,17 @@ void syncRaftProgressBecomeSnapshot(SSyncRaftProgress* progress, SyncIndex snaps } void syncRaftCopyProgress(const SSyncRaftProgress* progress, SSyncRaftProgress* out) { + memcpy(out, progress, sizeof(SSyncRaftProgress)); +} +bool syncRaftIterateProgressMap(const SSyncRaftNodeMap* nodeMap, SSyncRaftProgress *pProgress) { + SSyncRaftProgress **ppProgress = taosHashIterate(nodeMap->nodeIdMap, pProgress); + if (ppProgress == NULL) { + return true; + } + + *pProgress = *(*ppProgress); + return false; } /** diff --git a/source/libs/sync/src/sync_raft_progress_tracker.c b/source/libs/sync/src/sync_raft_progress_tracker.c index 1407df059a..3dd0a5ffe1 100644 --- a/source/libs/sync/src/sync_raft_progress_tracker.c +++ b/source/libs/sync/src/sync_raft_progress_tracker.c @@ -22,6 +22,9 @@ SSyncRaftProgressTracker* syncRaftOpenProgressTracker() { return NULL; } + syncRaftInitNodeMap(&tracker->config.learners); + syncRaftInitNodeMap(&tracker->config.learnersNext); + return tracker; } @@ -88,8 +91,8 @@ ESyncRaftVoteResult syncRaftTallyVotes(SSyncRaftProgressTracker* tracker, int* r } void syncRaftConfigState(const SSyncRaftProgressTracker* tracker, SSyncConfigState* cs) { - memcpy(&cs->voters, &tracker->config.voters.incoming, sizeof(SSyncRaftNodeMap)); - memcpy(&cs->votersOutgoing, &tracker->config.voters.outgoing, sizeof(SSyncRaftNodeMap)); - memcpy(&cs->learners, &tracker->config.learners, sizeof(SSyncRaftNodeMap)); - memcpy(&cs->learnersNext, &tracker->config.learnersNext, sizeof(SSyncRaftNodeMap)); + syncRaftCopyNodeMap(&cs->voters, &tracker->config.voters.incoming); + syncRaftCopyNodeMap(&cs->votersOutgoing, &tracker->config.voters.outgoing); + syncRaftCopyNodeMap(&cs->learners, &tracker->config.learners); + syncRaftCopyNodeMap(&cs->learnersNext, &tracker->config.learnersNext); } \ No newline at end of file diff --git a/source/libs/sync/src/sync_raft_quorum_joint.c b/source/libs/sync/src/sync_raft_quorum_joint.c index 2383d7ee63..6eecfbd9e5 100644 --- a/source/libs/sync/src/sync_raft_quorum_joint.c +++ b/source/libs/sync/src/sync_raft_quorum_joint.c @@ -42,39 +42,14 @@ ESyncRaftVoteType syncRaftVoteResult(SSyncRaftQuorumJointConfig* config, SHashOb } void syncRaftJointConfigAddToIncoming(SSyncRaftQuorumJointConfig* config, SyncNodeId id) { - int i, min; - - for (i = 0, min = -1; i < TSDB_MAX_REPLICA; ++i) { - if (config->incoming.nodeId[i] == id) { - return; - } - if (min == -1 && config->incoming.nodeId[i] == SYNC_NON_NODE_ID) { - min = i; - } - } - - assert(min != -1); - config->incoming.nodeId[min] = id; - config->incoming.replica += 1; + syncRaftAddToNodeMap(&config->incoming, id); } void syncRaftJointConfigRemoveFromIncoming(SSyncRaftQuorumJointConfig* config, SyncNodeId id) { - int i; - - for (i = 0; i < TSDB_MAX_REPLICA; ++i) { - if (config->incoming.nodeId[i] == id) { - config->incoming.replica -= 1; - config->incoming.nodeId[i] = SYNC_NON_NODE_ID; - break; - } - } - - assert(config->incoming.replica >= 0); + syncRaftRemoveFromNodeMap(&config->incoming, id); } void syncRaftJointConfigIDS(const SSyncRaftQuorumJointConfig* config, SSyncRaftNodeMap* nodeMap) { - int i, j, m; - syncRaftCopyNodeMap(&config->incoming, nodeMap); syncRaftUnionNodeMap(&config->outgoing, nodeMap); From da029f32a77703f9f8dc35067c7c562cbfa657b0 Mon Sep 17 00:00:00 2001 From: lichuang Date: Wed, 17 Nov 2021 12:06:06 +0800 Subject: [PATCH 05/44] [TD-10645][raft]refactor node and progress map --- source/libs/sync/inc/sync_raft_node_map.h | 2 +- source/libs/sync/inc/sync_raft_quorum_joint.h | 14 +- .../libs/sync/src/sync_raft_config_change.c | 323 +++++++++--------- source/libs/sync/src/sync_raft_node_map.c | 6 +- 4 files changed, 184 insertions(+), 161 deletions(-) diff --git a/source/libs/sync/inc/sync_raft_node_map.h b/source/libs/sync/inc/sync_raft_node_map.h index 5d43e42061..285717ed78 100644 --- a/source/libs/sync/inc/sync_raft_node_map.h +++ b/source/libs/sync/inc/sync_raft_node_map.h @@ -30,7 +30,7 @@ void syncRaftClearNodeMap(SSyncRaftNodeMap* nodeMap); bool syncRaftIsInNodeMap(const SSyncRaftNodeMap* nodeMap, SyncNodeId nodeId); -void syncRaftCopyNodeMap(const SSyncRaftNodeMap* nodeMap, SSyncRaftNodeMap* to); +void syncRaftCopyNodeMap(const SSyncRaftNodeMap* from, SSyncRaftNodeMap* to); void syncRaftUnionNodeMap(const SSyncRaftNodeMap* nodeMap, SSyncRaftNodeMap* to); diff --git a/source/libs/sync/inc/sync_raft_quorum_joint.h b/source/libs/sync/inc/sync_raft_quorum_joint.h index 4a5b749a0e..59d1fadc4a 100644 --- a/source/libs/sync/inc/sync_raft_quorum_joint.h +++ b/source/libs/sync/inc/sync_raft_quorum_joint.h @@ -59,7 +59,19 @@ static FORCE_INLINE const SSyncRaftNodeMap* syncRaftJointConfigOutgoing(const SS } static FORCE_INLINE void syncRaftJointConfigClearOutgoing(SSyncRaftQuorumJointConfig* config) { - memset(&config->outgoing, 0, sizeof(SSyncRaftNodeMap)); + syncRaftClearNodeMap(&config->outgoing); +} + +static FORCE_INLINE bool syncRaftJointConfigIsIncomingEmpty(const SSyncRaftQuorumJointConfig* config) { + return syncRaftNodeMapSize(&config->incoming) == 0; +} + +static FORCE_INLINE bool syncRaftJointConfigIsOutgoingEmpty(const SSyncRaftQuorumJointConfig* config) { + return syncRaftNodeMapSize(&config->outgoing) == 0; +} + +static FORCE_INLINE bool syncRaftJointConfigIsInOutgoing(const SSyncRaftQuorumJointConfig* config, SyncNodeId id) { + return syncRaftIsInNodeMap(&config->outgoing, id); } void syncRaftJointConfigIDS(const SSyncRaftQuorumJointConfig* config, SSyncRaftNodeMap* nodeMap); diff --git a/source/libs/sync/src/sync_raft_config_change.c b/source/libs/sync/src/sync_raft_config_change.c index ad261dd277..0fe31b9f08 100644 --- a/source/libs/sync/src/sync_raft_config_change.c +++ b/source/libs/sync/src/sync_raft_config_change.c @@ -40,7 +40,57 @@ static void makeVoter(SSyncRaftChanger* changer, SSyncRaftProgressTrackerConfig* static void makeLearner(SSyncRaftChanger* changer, SSyncRaftProgressTrackerConfig* config, SSyncRaftProgressMap* progressMap, SyncNodeId id); static void removeNodeId(SSyncRaftChanger* changer, SSyncRaftProgressTrackerConfig* config, - SSyncRaftProgressMap* progressMap, SyncNodeId id); + SSyncRaftProgressMap* progressMap, SyncNodeId id); + +// EnterJoint verifies that the outgoing (=right) majority config of the joint +// config is empty and initializes it with a copy of the incoming (=left) +// majority config. That is, it transitions from +// +// (1 2 3)&&() +// to +// (1 2 3)&&(1 2 3). +// +// The supplied changes are then applied to the incoming majority config, +// resulting in a joint configuration that in terms of the Raft thesis[1] +// (Section 4.3) corresponds to `C_{new,old}`. +// +// [1]: https://github.com/ongardie/dissertation/blob/master/online-trim.pdf +int syncRaftChangerEnterJoint(SSyncRaftChanger* changer, bool autoLeave, const SSyncConfChangeSingleArray* css, + SSyncRaftProgressTrackerConfig* config, SSyncRaftProgressMap* progressMap) { + int ret; + + ret = checkAndCopy(changer, config, progressMap); + if (ret != 0) { + return ret; + } + + if (hasJointConfig(config)) { + syncError("config is already joint"); + return -1; + } + + if(syncRaftJointConfigIsIncomingEmpty(&config->voters) == 0) { + // We allow adding nodes to an empty config for convenience (testing and + // bootstrap), but you can't enter a joint state. + syncError("can't make a zero-voter config joint"); + return -1; + } + + // Clear the outgoing config. + syncRaftJointConfigClearOutgoing(&config->voters); + + // Copy incoming to outgoing. + syncRaftCopyNodeMap(&config->voters.incoming, &config->voters.outgoing); + + ret = applyConfig(changer, config, progressMap, css); + if (ret != 0) { + return ret; + } + + config->autoLeave = autoLeave; + return checkAndReturn(config, progressMap); +} + // syncRaftChangerSimpleConfig carries out a series of configuration changes that (in aggregate) // mutates the incoming majority config Voters[0] by at most one. This method // will return an error if that is not the case, if the resulting quorum is @@ -75,52 +125,131 @@ int syncRaftChangerSimpleConfig(SSyncRaftChanger* changer, const SSyncConfChange return checkAndReturn(config, progressMap); } -// EnterJoint verifies that the outgoing (=right) majority config of the joint -// config is empty and initializes it with a copy of the incoming (=left) -// majority config. That is, it transitions from -// -// (1 2 3)&&() -// to -// (1 2 3)&&(1 2 3). -// -// The supplied changes are then applied to the incoming majority config, -// resulting in a joint configuration that in terms of the Raft thesis[1] -// (Section 4.3) corresponds to `C_{new,old}`. -// -// [1]: https://github.com/ongardie/dissertation/blob/master/online-trim.pdf -int syncRaftChangerEnterJoint(SSyncRaftChanger* changer, bool autoLeave, const SSyncConfChangeSingleArray* css, - SSyncRaftProgressTrackerConfig* config, SSyncRaftProgressMap* progressMap) { - int ret; +// apply a change to the configuration. By convention, changes to voters are +// always made to the incoming majority config Voters[0]. Voters[1] is either +// empty or preserves the outgoing majority configuration while in a joint state. +static int applyConfig(SSyncRaftChanger* changer, SSyncRaftProgressTrackerConfig* config, + SSyncRaftProgressMap* progressMap, const SSyncConfChangeSingleArray* css) { + int i; - ret = checkAndCopy(changer, config, progressMap); - if (ret != 0) { - return ret; + for (i = 0; i < css->n; ++i) { + const SSyncConfChangeSingle* cs = &(css->changes[i]); + if (cs->nodeId == SYNC_NON_NODE_ID) { + continue; + } + + ESyncRaftConfChangeType type = cs->type; + switch (type) { + case SYNC_RAFT_Conf_AddNode: + makeVoter(changer, config, progressMap, cs->nodeId); + break; + case SYNC_RAFT_Conf_AddLearnerNode: + makeLearner(changer, config, progressMap, cs->nodeId); + break; + case SYNC_RAFT_Conf_RemoveNode: + removeNodeId(changer, config, progressMap, cs->nodeId); + break; + case SYNC_RAFT_Conf_UpdateNode: + break; + } } - if (hasJointConfig(config)) { - syncError("config is already joint"); + + if (syncRaftJointConfigIsIncomingEmpty(&config->voters)) { + syncError("removed all voters"); return -1; } - if(syncRaftNodeMapSize(&config->voters.incoming) == 0) { - // We allow adding nodes to an empty config for convenience (testing and - // bootstrap), but you can't enter a joint state. - syncError("can't make a zero-voter config joint"); - return -1; + return 0; +} + + +// makeVoter adds or promotes the given ID to be a voter in the incoming +// majority config. +static void makeVoter(SSyncRaftChanger* changer, SSyncRaftProgressTrackerConfig* config, + SSyncRaftProgressMap* progressMap, SyncNodeId id) { + SSyncRaftProgress* progress = syncRaftFindProgressByNodeId(progressMap, id); + if (progress == -1) { + initProgress(changer, config, progressMap, id, false); + return; } - // Clear the outgoing config. - syncRaftJointConfigClearOutgoing(&config->voters); + progress->isLearner = false; + nilAwareDelete(&config->learners, id); + nilAwareDelete(&config->learnersNext, id); + syncRaftJointConfigAddToIncoming(&config->voters, id); +} - // Copy incoming to outgoing. - syncRaftCopyNodeMap(&config->voters.incoming, &config->voters.outgoing); - - ret = applyConfig(changer, config, progressMap, css); - if (ret != 0) { - return ret; +// makeLearner makes the given ID a learner or stages it to be a learner once +// an active joint configuration is exited. +// +// The former happens when the peer is not a part of the outgoing config, in +// which case we either add a new learner or demote a voter in the incoming +// config. +// +// The latter case occurs when the configuration is joint and the peer is a +// voter in the outgoing config. In that case, we do not want to add the peer +// as a learner because then we'd have to track a peer as a voter and learner +// simultaneously. Instead, we add the learner to LearnersNext, so that it will +// be added to Learners the moment the outgoing config is removed by +// LeaveJoint(). +static void makeLearner(SSyncRaftChanger* changer, SSyncRaftProgressTrackerConfig* config, + SSyncRaftProgressMap* progressMap, SyncNodeId id) { + SSyncRaftProgress* progress = syncRaftFindProgressByNodeId(progressMap, id); + if (progress == NULL) { + initProgress(changer, config, progressMap, id, true); + return; } - config->autoLeave = autoLeave; - return checkAndReturn(config, progressMap); + if (progress->isLearner) { + return; + } + // Remove any existing voter in the incoming config... + removeNodeId(changer, config, progressMap, id); + + // ... but save the Progress. + syncRaftAddToProgressMap(progressMap, progress); + + // Use LearnersNext if we can't add the learner to Learners directly, i.e. + // if the peer is still tracked as a voter in the outgoing config. It will + // be turned into a learner in LeaveJoint(). + // + // Otherwise, add a regular learner right away. + bool inInOutgoing = syncRaftJointConfigIsInOutgoing(&config->voters, id); + if (inInOutgoing) { + nilAwareAdd(&config->learnersNext, id); + } else { + nilAwareAdd(&config->learners, id); + progress->isLearner = true; + } +} + +// removeNodeId this peer as a voter or learner from the incoming config. +static void removeNodeId(SSyncRaftChanger* changer, SSyncRaftProgressTrackerConfig* config, + SSyncRaftProgressMap* progressMap, SyncNodeId id) { + SSyncRaftProgress* progress = syncRaftFindProgressByNodeId(progressMap, id); + if (progress == NULL) { + return; + } + + syncRaftJointConfigRemoveFromIncoming(&config->voters, id); + nilAwareDelete(&config->learners, id); + nilAwareDelete(&config->learnersNext, id); + + // If the peer is still a voter in the outgoing config, keep the Progress. + bool inInOutgoing = syncRaftJointConfigIsInOutgoing(&config->voters, id); + if (!inInOutgoing) { + syncRaftRemoveFromProgressMap(progressMap, id); + } +} + +// initProgress initializes a new progress for the given node or learner. +static void initProgress(SSyncRaftChanger* changer, SSyncRaftProgressTrackerConfig* config, + SSyncRaftProgressMap* progressMap, SyncNodeId id, bool isLearner) { + if (!isLearner) { + syncRaftJointConfigAddToIncoming(&config->voters, id); + } else { + nilAwareAdd(&config->learners, id); + } } // checkAndCopy copies the tracker's config and progress map (deeply enough for @@ -208,41 +337,7 @@ static int checkInvariants(SSyncRaftProgressTrackerConfig* config, SSyncRaftProg } static bool hasJointConfig(const SSyncRaftProgressTrackerConfig* config) { - return syncRaftNodeMapSize(&config->voters.outgoing) > 0; -} - -static int applyConfig(SSyncRaftChanger* changer, SSyncRaftProgressTrackerConfig* config, - SSyncRaftProgressMap* progressMap, const SSyncConfChangeSingleArray* css) { - int i; - - for (i = 0; i < css->n; ++i) { - const SSyncConfChangeSingle* cs = &(css->changes[i]); - if (cs->nodeId == SYNC_NON_NODE_ID) { - continue; - } - - ESyncRaftConfChangeType type = cs->type; - switch (type) { - case SYNC_RAFT_Conf_AddNode: - makeVoter(changer, config, progressMap, cs->nodeId); - break; - case SYNC_RAFT_Conf_AddLearnerNode: - makeLearner(changer, config, progressMap, cs->nodeId); - break; - case SYNC_RAFT_Conf_RemoveNode: - removeNodeId(changer, config, progressMap, cs->nodeId); - break; - case SYNC_RAFT_Conf_UpdateNode: - break; - } - } - - if (syncRaftNodeMapSize(&config->voters.incoming) == 0) { - syncError("removed all voters"); - return -1; - } - - return 0; + return !syncRaftJointConfigIsOutgoingEmpty(&config->voters); } // symdiff returns the count of the symmetric difference between the sets of @@ -272,11 +367,6 @@ static int symDiff(const SSyncRaftNodeMap* l, const SSyncRaftNodeMap* r) { return n; } -static void initProgress(SSyncRaftChanger* changer, SSyncRaftProgressTrackerConfig* config, - SSyncRaftProgressMap* progressMap, SyncNodeId id, bool isLearner) { - -} - // nilAwareDelete deletes from a map, nil'ing the map itself if it is empty after. static void nilAwareDelete(SSyncRaftNodeMap* nodeMap, SyncNodeId id) { syncRaftRemoveFromNodeMap(nodeMap, id); @@ -285,83 +375,4 @@ static void nilAwareDelete(SSyncRaftNodeMap* nodeMap, SyncNodeId id) { // nilAwareAdd populates a map entry, creating the map if necessary. static void nilAwareAdd(SSyncRaftNodeMap* nodeMap, SyncNodeId id) { syncRaftAddToNodeMap(nodeMap, id); -} - -// makeVoter adds or promotes the given ID to be a voter in the incoming -// majority config. -static void makeVoter(SSyncRaftChanger* changer, SSyncRaftProgressTrackerConfig* config, - SSyncRaftProgressMap* progressMap, SyncNodeId id) { - SSyncRaftProgress* progress = syncRaftFindProgressByNodeId(progressMap, id); - if (progress == -1) { - initProgress(changer, config, progressMap, id, false); - return; - } - - progress->isLearner = false; - nilAwareDelete(&config->learners, id); - nilAwareDelete(&config->learnersNext, id); - syncRaftJointConfigAddToIncoming(&config->voters, id); -} - -// makeLearner makes the given ID a learner or stages it to be a learner once -// an active joint configuration is exited. -// -// The former happens when the peer is not a part of the outgoing config, in -// which case we either add a new learner or demote a voter in the incoming -// config. -// -// The latter case occurs when the configuration is joint and the peer is a -// voter in the outgoing config. In that case, we do not want to add the peer -// as a learner because then we'd have to track a peer as a voter and learner -// simultaneously. Instead, we add the learner to LearnersNext, so that it will -// be added to Learners the moment the outgoing config is removed by -// LeaveJoint(). -static void makeLearner(SSyncRaftChanger* changer, SSyncRaftProgressTrackerConfig* config, - SSyncRaftProgressMap* progressMap, SyncNodeId id) { - SSyncRaftProgress* progress = syncRaftFindProgressByNodeId(progressMap, id); - if (progress == NULL) { - initProgress(changer, config, progressMap, id, false); - return; - } - - if (progress->isLearner) { - return; - } - // Remove any existing voter in the incoming config... - removeNodeId(changer, config, progressMap, id); - - // ... but save the Progress. - syncRaftAddToProgressMap(progressMap, progress); - - // Use LearnersNext if we can't add the learner to Learners directly, i.e. - // if the peer is still tracked as a voter in the outgoing config. It will - // be turned into a learner in LeaveJoint(). - // - // Otherwise, add a regular learner right away. - bool inOutgoing = syncRaftIsInNodeMap(&config->voters.outgoing, id); - if (inOutgoing) { - nilAwareAdd(&config->learnersNext, id); - } else { - nilAwareAdd(&config->learners, id); - progress->isLearner = true; - } -} - -// removeNodeId this peer as a voter or learner from the incoming config. -static void removeNodeId(SSyncRaftChanger* changer, SSyncRaftProgressTrackerConfig* config, - SSyncRaftProgressMap* progressMap, SyncNodeId id) { - SSyncRaftProgress* progress = syncRaftFindProgressByNodeId(progressMap, id); - if (progress == NULL) { - return; - } - - syncRaftJointConfigRemoveFromIncoming(&config->voters, id); - nilAwareDelete(&config->learners, id); - nilAwareDelete(&config->learnersNext, id); - - // If the peer is still a voter in the outgoing config, keep the Progress. - bool inOutgoing = syncRaftIsInNodeMap(&config->voters.outgoing, id); - if (!inOutgoing) { - syncRaftRemoveFromProgressMap(progressMap, id); - } } \ No newline at end of file diff --git a/source/libs/sync/src/sync_raft_node_map.c b/source/libs/sync/src/sync_raft_node_map.c index 5dacc01d1a..9adb3844f5 100644 --- a/source/libs/sync/src/sync_raft_node_map.c +++ b/source/libs/sync/src/sync_raft_node_map.c @@ -32,11 +32,11 @@ bool syncRaftIsInNodeMap(const SSyncRaftNodeMap* nodeMap, SyncNodeId nodeId) { return true; } -void syncRaftCopyNodeMap(const SSyncRaftNodeMap* nodeMap, SSyncRaftNodeMap* to) { - SyncNodeId** ppId = (SyncNodeId**)taosHashIterate(nodeMap->nodeIdMap, NULL); +void syncRaftCopyNodeMap(const SSyncRaftNodeMap* from, SSyncRaftNodeMap* to) { + SyncNodeId** ppId = (SyncNodeId**)taosHashIterate(from->nodeIdMap, NULL); while (ppId) { taosHashPut(to->nodeIdMap, ppId, sizeof(SyncNodeId*), ppId, sizeof(SyncNodeId*)); - ppId = taosHashIterate(nodeMap->nodeIdMap, ppId); + ppId = taosHashIterate(from->nodeIdMap, ppId); } } From 98a6b1918c1fa5270d5d3742c877594bbef192d4 Mon Sep 17 00:00:00 2001 From: lichuang Date: Wed, 17 Nov 2021 15:50:57 +0800 Subject: [PATCH 06/44] [TD-10645][raft]refactor node and progress map --- source/libs/sync/inc/raft.h | 1 - source/libs/sync/inc/sync_raft_node_map.h | 2 + source/libs/sync/inc/sync_raft_progress.h | 8 +- .../sync/inc/sync_raft_progress_tracker.h | 5 +- source/libs/sync/inc/sync_type.h | 2 + source/libs/sync/src/raft.c | 9 +-- .../libs/sync/src/sync_raft_config_change.c | 79 +++++++++++++------ source/libs/sync/src/sync_raft_election.c | 7 +- source/libs/sync/src/sync_raft_impl.c | 10 +-- source/libs/sync/src/sync_raft_node_map.c | 14 +++- source/libs/sync/src/sync_raft_progress.c | 29 +++++-- .../sync/src/sync_raft_progress_tracker.c | 32 +++++--- .../libs/sync/src/sync_raft_quorum_majority.c | 30 +++---- source/libs/sync/src/sync_raft_restore.c | 74 ++++++++--------- 14 files changed, 180 insertions(+), 122 deletions(-) diff --git a/source/libs/sync/inc/raft.h b/source/libs/sync/inc/raft.h index 6fa6c6e346..129f0f4dbc 100644 --- a/source/libs/sync/inc/raft.h +++ b/source/libs/sync/inc/raft.h @@ -47,7 +47,6 @@ struct SSyncRaft { // hash map nodeId -> SNodeInfo* SHashObj* nodeInfoMap; - int selfIndex; SyncNodeId selfId; SyncGroupId selfGroupId; diff --git a/source/libs/sync/inc/sync_raft_node_map.h b/source/libs/sync/inc/sync_raft_node_map.h index 285717ed78..15e559a733 100644 --- a/source/libs/sync/inc/sync_raft_node_map.h +++ b/source/libs/sync/inc/sync_raft_node_map.h @@ -43,4 +43,6 @@ int32_t syncRaftNodeMapSize(const SSyncRaftNodeMap* nodeMap); // return true if reach the end bool syncRaftIterateNodeMap(const SSyncRaftNodeMap* nodeMap, SyncNodeId *pId); +bool syncRaftIsAllInProgressMap(const SSyncRaftNodeMap* nodeMap, const SSyncRaftProgressMap* progressMap); + #endif /* _TD_LIBS_SYNC_RAFT_NODE_MAP_H */ \ No newline at end of file diff --git a/source/libs/sync/inc/sync_raft_progress.h b/source/libs/sync/inc/sync_raft_progress.h index 3d2995ed77..5664cfd15e 100644 --- a/source/libs/sync/inc/sync_raft_progress.h +++ b/source/libs/sync/inc/sync_raft_progress.h @@ -147,7 +147,7 @@ static FORCE_INLINE const char* syncRaftProgressStateString(const SSyncRaftProgr return kProgressStateString[progress->state]; } -void syncRaftInitProgress(int i, SSyncRaft* pRaft, SSyncRaftProgress* progress); +void syncRaftResetProgress(SSyncRaft* pRaft, SSyncRaftProgress* progress); /** * syncRaftProgressBecomeProbe transitions into StateProbe. Next is reset to Match+1 or, @@ -227,6 +227,8 @@ int syncRaftAddToProgressMap(SSyncRaftProgressMap* progressMap, SSyncRaftProgres void syncRaftRemoveFromProgressMap(SSyncRaftProgressMap* progressMap, SyncNodeId id); +bool syncRaftIsInProgressMap(SSyncRaftProgressMap* progressMap, SyncNodeId id); + /** * return true if progress's log is up-todate **/ @@ -237,7 +239,9 @@ void syncRaftProgressBecomeSnapshot(SSyncRaftProgress* progress, SyncIndex snaps void syncRaftCopyProgress(const SSyncRaftProgress* from, SSyncRaftProgress* to); // return true if reach the end -bool syncRaftIterateProgressMap(const SSyncRaftNodeMap* nodeMap, SSyncRaftProgress *pProgress); +bool syncRaftIterateProgressMap(const SSyncRaftProgressMap* progressMap, SSyncRaftProgress *pProgress); + +bool syncRaftVisitProgressMap(SSyncRaftProgressMap* progressMap, visitProgressFp fp, void* arg); #if 0 diff --git a/source/libs/sync/inc/sync_raft_progress_tracker.h b/source/libs/sync/inc/sync_raft_progress_tracker.h index 3a448290c8..35daf8139c 100644 --- a/source/libs/sync/inc/sync_raft_progress_tracker.h +++ b/source/libs/sync/inc/sync_raft_progress_tracker.h @@ -88,13 +88,14 @@ struct SSyncRaftProgressTracker { SHashObj* votesMap; int maxInflightMsgs; + + SSyncRaft* pRaft; }; -SSyncRaftProgressTracker* syncRaftOpenProgressTracker(); +SSyncRaftProgressTracker* syncRaftOpenProgressTracker(SSyncRaft* pRaft); void syncRaftResetVotes(SSyncRaftProgressTracker*); -typedef void (*visitProgressFp)(int i, SSyncRaftProgress* progress, void* arg); void syncRaftProgressVisit(SSyncRaftProgressTracker*, visitProgressFp visit, void* arg); /** diff --git a/source/libs/sync/inc/sync_type.h b/source/libs/sync/inc/sync_type.h index fcb0940609..e00700d724 100644 --- a/source/libs/sync/inc/sync_type.h +++ b/source/libs/sync/inc/sync_type.h @@ -84,4 +84,6 @@ typedef enum { SYNC_RAFT_VOTE_RESP_REJECT = 2, } ESyncRaftVoteType; +typedef void (*visitProgressFp)(SSyncRaftProgress* progress, void* arg); + #endif /* _TD_LIBS_SYNC_TYPE_H */ diff --git a/source/libs/sync/src/raft.c b/source/libs/sync/src/raft.c index b2170e0b68..85c330ece3 100644 --- a/source/libs/sync/src/raft.c +++ b/source/libs/sync/src/raft.c @@ -65,7 +65,7 @@ int32_t syncRaftStart(SSyncRaft* pRaft, const SSyncInfo* pInfo) { } // init progress tracker - pRaft->tracker = syncRaftOpenProgressTracker(); + pRaft->tracker = syncRaftOpenProgressTracker(pRaft); if (pRaft->tracker == NULL) { return -1; } @@ -157,7 +157,7 @@ static int deserializeClusterStateFromBuffer(SSyncConfigState* cluster, const ch return 0; } -static void visitProgressMaybeSendAppend(int i, SSyncRaftProgress* progress, void* arg) { +static void visitProgressMaybeSendAppend(SSyncRaftProgress* progress, void* arg) { syncRaftReplicate(arg, progress, false); } @@ -175,13 +175,12 @@ static void switchToConfig(SSyncRaft* pRaft, const SSyncRaftProgressTrackerConfi SSyncRaftProgress* progress = NULL; syncRaftConfigState(pRaft->tracker, cs); - i = syncRaftFindProgressByNodeId(&pRaft->tracker->progressMap, selfId); - exist = (i != -1); + progress = syncRaftFindProgressByNodeId(&pRaft->tracker->progressMap, selfId); + exist = (progress != NULL); // Update whether the node itself is a learner, resetting to false when the // node is removed. if (exist) { - progress = &pRaft->tracker->progressMap.progress[i]; pRaft->isLearner = progress->isLearner; } else { pRaft->isLearner = false; diff --git a/source/libs/sync/src/sync_raft_config_change.c b/source/libs/sync/src/sync_raft_config_change.c index 0fe31b9f08..1f7aab064f 100644 --- a/source/libs/sync/src/sync_raft_config_change.c +++ b/source/libs/sync/src/sync_raft_config_change.c @@ -13,6 +13,7 @@ * along with this program. If not, see . */ +#include "raft.h" #include "syncInt.h" #include "sync_raft_config_change.h" #include "sync_raft_progress.h" @@ -168,7 +169,7 @@ static int applyConfig(SSyncRaftChanger* changer, SSyncRaftProgressTrackerConfig static void makeVoter(SSyncRaftChanger* changer, SSyncRaftProgressTrackerConfig* config, SSyncRaftProgressMap* progressMap, SyncNodeId id) { SSyncRaftProgress* progress = syncRaftFindProgressByNodeId(progressMap, id); - if (progress == -1) { + if (progress == NULL) { initProgress(changer, config, progressMap, id, false); return; } @@ -250,31 +251,34 @@ static void initProgress(SSyncRaftChanger* changer, SSyncRaftProgressTrackerConf } else { nilAwareAdd(&config->learners, id); } -} -// checkAndCopy copies the tracker's config and progress map (deeply enough for -// the purposes of the Changer) and returns those copies. It returns an error -// if checkInvariants does. -static int checkAndCopy(SSyncRaftChanger* changer, SSyncRaftProgressTrackerConfig* config, SSyncRaftProgressMap* progressMap) { - syncRaftCloneTrackerConfig(&changer->tracker->config, config); - int i; + SSyncRaftProgress* pProgress = (SSyncRaftProgress*)malloc(sizeof(SSyncRaftProgress)); + assert (pProgress != NULL); + *pProgress = (SSyncRaftProgress) { + // Initializing the Progress with the last index means that the follower + // can be probed (with the last index). + // + // TODO(tbg): seems awfully optimistic. Using the first index would be + // better. The general expectation here is that the follower has no log + // at all (and will thus likely need a snapshot), though the app may + // have applied a snapshot out of band before adding the replica (thus + // making the first index the better choice). + .id = id, + .groupId = changer->tracker->pRaft->selfGroupId, + .nextIndex = changer->lastIndex, + .matchIndex = 0, + .state = PROGRESS_STATE_PROBE, + .pendingSnapshotIndex = 0, + .probeSent = false, + .inflights = syncRaftOpenInflights(changer->tracker->maxInflightMsgs), + .isLearner = isLearner, + // When a node is first added, we should mark it as recently active. + // Otherwise, CheckQuorum may cause us to step down if it is invoked + // before the added node has had a chance to communicate with us. + .recentActive = true, + }; - SSyncRaftProgress* pProgress = NULL; - while (!syncRaftIterateProgressMap(&changer->tracker->progressMap, pProgress)) { - syncRaftAddToProgressMap(progressMap, pProgress); - } - - return checkAndReturn(config, progressMap); -} - -// checkAndReturn calls checkInvariants on the input and returns either the -// resulting error or the input. -static int checkAndReturn(SSyncRaftProgressTrackerConfig* config, SSyncRaftProgressMap* progressMap) { - if (checkInvariants(config, progressMap) != 0) { - return -1; - } - - return 0; + syncRaftAddToProgressMap(progressMap, pProgress); } // checkInvariants makes sure that the config and progress are compatible with @@ -304,7 +308,7 @@ static int checkInvariants(SSyncRaftProgressTrackerConfig* config, SSyncRaftProg } // Conversely Learners and Voters doesn't intersect at all. - SyncNodeId* pNodeId = NULL; + pNodeId = NULL; while (!syncRaftIterateNodeMap(&config->learners, pNodeId)) { SyncNodeId nodeId = *pNodeId; if (syncRaftJointConfigInIncoming(&config->voters, nodeId)) { @@ -336,6 +340,31 @@ static int checkInvariants(SSyncRaftProgressTrackerConfig* config, SSyncRaftProg return 0; } +// checkAndCopy copies the tracker's config and progress map (deeply enough for +// the purposes of the Changer) and returns those copies. It returns an error +// if checkInvariants does. +static int checkAndCopy(SSyncRaftChanger* changer, SSyncRaftProgressTrackerConfig* config, SSyncRaftProgressMap* progressMap) { + syncRaftCloneTrackerConfig(&changer->tracker->config, config); + int i; + + SSyncRaftProgress* pProgress = NULL; + while (!syncRaftIterateProgressMap(&changer->tracker->progressMap, pProgress)) { + syncRaftAddToProgressMap(progressMap, pProgress); + } + + return checkAndReturn(config, progressMap); +} + +// checkAndReturn calls checkInvariants on the input and returns either the +// resulting error or the input. +static int checkAndReturn(SSyncRaftProgressTrackerConfig* config, SSyncRaftProgressMap* progressMap) { + if (checkInvariants(config, progressMap) != 0) { + return -1; + } + + return 0; +} + static bool hasJointConfig(const SSyncRaftProgressTrackerConfig* config) { return !syncRaftJointConfigIsOutgoingEmpty(&config->voters); } diff --git a/source/libs/sync/src/sync_raft_election.c b/source/libs/sync/src/sync_raft_election.c index d4013bbc08..6d36d38267 100644 --- a/source/libs/sync/src/sync_raft_election.c +++ b/source/libs/sync/src/sync_raft_election.c @@ -86,12 +86,7 @@ static void campaign(SSyncRaft* pRaft, ESyncRaftElectionType cType) { SSyncRaftNodeMap nodeMap; syncRaftJointConfigIDS(&pRaft->tracker->config.voters, &nodeMap); SyncNodeId *pNodeId = NULL; - while (true) { - syncRaftIterateNodeMap(&nodeMap, &pNodeId); - if (pNodeId == NULL || *pNodeId == NULL) { - break; - } - + while (!syncRaftIterateNodeMap(&nodeMap, pNodeId)) { SyncNodeId nodeId = *pNodeId; if (nodeId == SYNC_NON_NODE_ID) { continue; diff --git a/source/libs/sync/src/sync_raft_impl.c b/source/libs/sync/src/sync_raft_impl.c index 73a02c4b80..2093bcb046 100644 --- a/source/libs/sync/src/sync_raft_impl.c +++ b/source/libs/sync/src/sync_raft_impl.c @@ -186,7 +186,7 @@ void syncRaftLoadState(SSyncRaft* pRaft, const SSyncServerState* serverState) { pRaft->voteFor = serverState->voteFor; } -static void visitProgressSendAppend(int i, SSyncRaftProgress* progress, void* arg) { +static void visitProgressSendAppend(SSyncRaftProgress* progress, void* arg) { SSyncRaft* pRaft = (SSyncRaft*)arg; if (pRaft->selfId == progress->id) { return; @@ -279,7 +279,7 @@ static void appendEntries(SSyncRaft* pRaft, SSyncRaftEntry* entries, int n) { syncRaftLogAppend(pRaft->log, entries, n); - SSyncRaftProgress* progress = &(pRaft->tracker->progressMap.progress[pRaft->selfIndex]); + SSyncRaftProgress* progress = syncRaftFindProgressByNodeId(&pRaft->tracker->progressMap, pRaft->selfId); syncRaftProgressMaybeUpdate(progress, lastIndex); // Regardless of syncRaftMaybeCommit's return, our caller will call bcastAppend. syncRaftMaybeCommit(pRaft); @@ -316,8 +316,8 @@ static void abortLeaderTransfer(SSyncRaft* pRaft) { pRaft->leadTransferee = SYNC_NON_NODE_ID; } -static void initProgress(int i, SSyncRaftProgress* progress, void* arg) { - syncRaftInitProgress(i, (SSyncRaft*)arg, progress); +static void resetProgress(SSyncRaftProgress* progress, void* arg) { + syncRaftResetProgress((SSyncRaft*)arg, progress); } static void resetRaft(SSyncRaft* pRaft, SyncTerm term) { @@ -336,7 +336,7 @@ static void resetRaft(SSyncRaft* pRaft, SyncTerm term) { abortLeaderTransfer(pRaft); syncRaftResetVotes(pRaft->tracker); - syncRaftProgressVisit(pRaft->tracker, initProgress, pRaft); + syncRaftProgressVisit(pRaft->tracker, resetProgress, pRaft); pRaft->pendingConfigIndex = 0; pRaft->uncommittedSize = 0; diff --git a/source/libs/sync/src/sync_raft_node_map.c b/source/libs/sync/src/sync_raft_node_map.c index 9adb3844f5..022f3b2dcb 100644 --- a/source/libs/sync/src/sync_raft_node_map.c +++ b/source/libs/sync/src/sync_raft_node_map.c @@ -15,6 +15,7 @@ #include "sync_raft_node_map.h" #include "sync_type.h" +#include "sync_raft_progress.h" void syncRaftInitNodeMap(SSyncRaftNodeMap* nodeMap) { nodeMap->nodeIdMap = taosHashInit(TSDB_MAX_REPLICA, taosGetDefaultHashFunction(TSDB_DATA_TYPE_INT), true, HASH_ENTRY_LOCK); @@ -50,6 +51,17 @@ bool syncRaftIterateNodeMap(const SSyncRaftNodeMap* nodeMap, SyncNodeId *pId) { return false; } +bool syncRaftIsAllInProgressMap(const SSyncRaftNodeMap* nodeMap, const SSyncRaftProgressMap* progressMap) { + SyncNodeId *pId = NULL; + while (!syncRaftIterateNodeMap(nodeMap, pId)) { + if (!syncRaftIsInProgressMap(progressMap, *pId)) { + return false; + } + } + + return true; +} + void syncRaftUnionNodeMap(const SSyncRaftNodeMap* nodeMap, SSyncRaftNodeMap* to) { syncRaftCopyNodeMap(nodeMap, to); } @@ -63,5 +75,5 @@ void syncRaftRemoveFromNodeMap(SSyncRaftNodeMap* nodeMap, SyncNodeId nodeId) { } int32_t syncRaftNodeMapSize(const SSyncRaftNodeMap* nodeMap) { - return taosHashGetSize(nodeMap); + return taosHashGetSize(nodeMap->nodeIdMap); } \ No newline at end of file diff --git a/source/libs/sync/src/sync_raft_progress.c b/source/libs/sync/src/sync_raft_progress.c index 436250e594..e63d37cee9 100644 --- a/source/libs/sync/src/sync_raft_progress.c +++ b/source/libs/sync/src/sync_raft_progress.c @@ -25,13 +25,16 @@ static void probeAcked(SSyncRaftProgress* progress); static void resumeProgress(SSyncRaftProgress* progress); -void syncRaftInitProgress(int i, SSyncRaft* pRaft, SSyncRaftProgress* progress) { +void syncRaftResetProgress(SSyncRaft* pRaft, SSyncRaftProgress* progress) { + if (progress->inflights) { + syncRaftCloseInflights(progress->inflights); + } SSyncRaftInflights* inflights = syncRaftOpenInflights(pRaft->tracker->maxInflightMsgs); if (inflights == NULL) { return; } *progress = (SSyncRaftProgress) { - .matchIndex = i == pRaft->selfIndex ? syncRaftLogLastIndex(pRaft->log) : 0, + .matchIndex = progress->id == pRaft->selfId ? syncRaftLogLastIndex(pRaft->log) : 0, .nextIndex = syncRaftLogLastIndex(pRaft->log) + 1, .inflights = inflights, .isLearner = false, @@ -113,7 +116,7 @@ bool syncRaftProgressIsPaused(SSyncRaftProgress* progress) { } SSyncRaftProgress* syncRaftFindProgressByNodeId(const SSyncRaftProgressMap* progressMap, SyncNodeId id) { - SSyncRaftProgress** ppProgress = (SSyncRaftProgress**)taosHashGet(progressMap, &id, sizeof(SyncNodeId*)); + SSyncRaftProgress** ppProgress = (SSyncRaftProgress**)taosHashGet(progressMap->progressMap, &id, sizeof(SyncNodeId*)); if (ppProgress == NULL) { return NULL; } @@ -126,9 +129,18 @@ int syncRaftAddToProgressMap(SSyncRaftProgressMap* progressMap, SSyncRaftProgres } void syncRaftRemoveFromProgressMap(SSyncRaftProgressMap* progressMap, SyncNodeId id) { + SSyncRaftProgress** ppProgress = (SSyncRaftProgress**)taosHashGet(progressMap->progressMap, &id, sizeof(SyncNodeId*)); + if (ppProgress == NULL) { + return; + } + free(*ppProgress); taosHashRemove(progressMap->progressMap, &id, sizeof(SyncNodeId*)); } +bool syncRaftIsInProgressMap(SSyncRaftProgressMap* progressMap, SyncNodeId id) { + return taosHashGet(progressMap->progressMap, &id, sizeof(SyncNodeId*)) != NULL; +} + bool syncRaftProgressIsUptodate(SSyncRaft* pRaft, SSyncRaftProgress* progress) { return syncRaftLogLastIndex(pRaft->log) + 1 == progress->nextIndex; } @@ -170,8 +182,8 @@ void syncRaftCopyProgress(const SSyncRaftProgress* progress, SSyncRaftProgress* memcpy(out, progress, sizeof(SSyncRaftProgress)); } -bool syncRaftIterateProgressMap(const SSyncRaftNodeMap* nodeMap, SSyncRaftProgress *pProgress) { - SSyncRaftProgress **ppProgress = taosHashIterate(nodeMap->nodeIdMap, pProgress); +bool syncRaftIterateProgressMap(const SSyncRaftProgressMap* progressMap, SSyncRaftProgress *pProgress) { + SSyncRaftProgress **ppProgress = taosHashIterate(progressMap->progressMap, pProgress); if (ppProgress == NULL) { return true; } @@ -180,6 +192,13 @@ bool syncRaftIterateProgressMap(const SSyncRaftNodeMap* nodeMap, SSyncRaftProgre return false; } +bool syncRaftVisitProgressMap(SSyncRaftProgressMap* progressMap, visitProgressFp fp, void* arg) { + SSyncRaftProgress *pProgress; + while (!syncRaftIterateProgressMap(progressMap, pProgress)) { + fp(pProgress, arg); + } +} + /** * ResetState moves the Progress into the specified State, resetting ProbeSent, * PendingSnapshot, and Inflights. diff --git a/source/libs/sync/src/sync_raft_progress_tracker.c b/source/libs/sync/src/sync_raft_progress_tracker.c index 3dd0a5ffe1..e6a016b7cf 100644 --- a/source/libs/sync/src/sync_raft_progress_tracker.c +++ b/source/libs/sync/src/sync_raft_progress_tracker.c @@ -16,7 +16,7 @@ #include "sync_raft_progress_tracker.h" #include "sync_raft_proto.h" -SSyncRaftProgressTracker* syncRaftOpenProgressTracker() { +SSyncRaftProgressTracker* syncRaftOpenProgressTracker(SSyncRaft* pRaft) { SSyncRaftProgressTracker* tracker = (SSyncRaftProgressTracker*)malloc(sizeof(SSyncRaftProgressTracker)); if (tracker == NULL) { return NULL; @@ -24,6 +24,7 @@ SSyncRaftProgressTracker* syncRaftOpenProgressTracker() { syncRaftInitNodeMap(&tracker->config.learners); syncRaftInitNodeMap(&tracker->config.learnersNext); + tracker->pRaft = pRaft; return tracker; } @@ -33,11 +34,7 @@ void syncRaftResetVotes(SSyncRaftProgressTracker* tracker) { } void syncRaftProgressVisit(SSyncRaftProgressTracker* tracker, visitProgressFp visit, void* arg) { - int i; - for (i = 0; i < TSDB_MAX_REPLICA; ++i) { - SSyncRaftProgress* progress = &(tracker->progressMap.progress[i]); - visit(i, progress, arg); - } + syncRaftVisitProgressMap(&tracker->progressMap, visit, arg); } void syncRaftRecordVote(SSyncRaftProgressTracker* tracker, SyncNodeId id, bool grant) { @@ -51,10 +48,20 @@ void syncRaftRecordVote(SSyncRaftProgressTracker* tracker, SyncNodeId id, bool g } void syncRaftCloneTrackerConfig(const SSyncRaftProgressTrackerConfig* from, SSyncRaftProgressTrackerConfig* to) { - + memcpy(to, from, sizeof(SSyncRaftProgressTrackerConfig)); } int syncRaftCheckProgress(const SSyncRaftProgressTrackerConfig* config, SSyncRaftProgressMap* progressMap) { + // NB: intentionally allow the empty config. In production we'll never see a + // non-empty config (we prevent it from being created) but we will need to + // be able to *create* an initial config, for example during bootstrap (or + // during tests). Instead of having to hand-code this, we allow + // transitioning from an empty config into any other legal and non-empty + // config. + if (!syncRaftIsAllInProgressMap(&config->voters.incoming, progressMap)) return -1; + if (!syncRaftIsAllInProgressMap(&config->voters.outgoing, progressMap)) return -1; + if (!syncRaftIsAllInProgressMap(&config->learners, progressMap)) return -1; + if (!syncRaftIsAllInProgressMap(&config->learnersNext, progressMap)) return -1; return 0; } @@ -67,8 +74,7 @@ ESyncRaftVoteResult syncRaftTallyVotes(SSyncRaftProgressTracker* tracker, int* r SSyncRaftProgress* progress; int r, g; - for (i = 0, r = 0, g = 0; i < TSDB_MAX_REPLICA; ++i) { - progress = &(tracker->progressMap.progress[i]); + while (!syncRaftIterateProgressMap(&tracker->progressMap, progress)) { if (progress->id == SYNC_NON_NODE_ID) { continue; } @@ -91,8 +97,8 @@ ESyncRaftVoteResult syncRaftTallyVotes(SSyncRaftProgressTracker* tracker, int* r } void syncRaftConfigState(const SSyncRaftProgressTracker* tracker, SSyncConfigState* cs) { - syncRaftCopyNodeMap(&cs->voters, &tracker->config.voters.incoming); - syncRaftCopyNodeMap(&cs->votersOutgoing, &tracker->config.voters.outgoing); - syncRaftCopyNodeMap(&cs->learners, &tracker->config.learners); - syncRaftCopyNodeMap(&cs->learnersNext, &tracker->config.learnersNext); + syncRaftCopyNodeMap(&tracker->config.voters.incoming, &cs->voters); + syncRaftCopyNodeMap(&tracker->config.voters.outgoing, &cs->votersOutgoing); + syncRaftCopyNodeMap(&tracker->config.learners, &cs->learners); + syncRaftCopyNodeMap(&tracker->config.learnersNext, &cs->learnersNext); } \ No newline at end of file diff --git a/source/libs/sync/src/sync_raft_quorum_majority.c b/source/libs/sync/src/sync_raft_quorum_majority.c index 014a8c7303..ff5ba64876 100644 --- a/source/libs/sync/src/sync_raft_quorum_majority.c +++ b/source/libs/sync/src/sync_raft_quorum_majority.c @@ -17,24 +17,24 @@ #include "sync_raft_quorum_majority.h" #include "sync_raft_node_map.h" -/** - * syncRaftMajorityVoteResult takes a mapping of voters to yes/no (true/false) votes and returns - * a result indicating whether the vote is pending (i.e. neither a quorum of - * yes/no has been reached), won (a quorum of yes has been reached), or lost (a - * quorum of no has been reached). - **/ +// VoteResult takes a mapping of voters to yes/no (true/false) votes and returns +// a result indicating whether the vote is pending (i.e. neither a quorum of +// yes/no has been reached), won (a quorum of yes has been reached), or lost (a +// quorum of no has been reached). ESyncRaftVoteResult syncRaftMajorityVoteResult(SSyncRaftNodeMap* config, SHashObj* votesMap) { - if (config->replica == 0) { + int n = syncRaftNodeMapSize(config); + if (n == 0) { + // By convention, the elections on an empty config win. This comes in + // handy with joint quorums because it'll make a half-populated joint + // quorum behave like a majority quorum. return SYNC_RAFT_VOTE_WON; } int i, g, r, missing; - for (i = g = r = missing = 0; i < TSDB_MAX_REPLICA; ++i) { - if (config->nodeId[i] == SYNC_NON_NODE_ID) { - continue; - } - - const ESyncRaftVoteType* pType = taosHashGet(votesMap, &config->nodeId[i], sizeof(SyncNodeId*)); + i = g = r = missing = 0; + SyncNodeId* pId = NULL; + while (!syncRaftIterateNodeMap(config, pId)) { + const ESyncRaftVoteType* pType = taosHashGet(votesMap, pId, sizeof(SyncNodeId*)); if (pType == NULL) { missing += 1; continue; @@ -47,11 +47,11 @@ ESyncRaftVoteResult syncRaftMajorityVoteResult(SSyncRaftNodeMap* config, SHashOb } } - int quorum = config->replica / 2 + 1; + int quorum = n / 2 + 1; if (g >= quorum) { return SYNC_RAFT_VOTE_WON; } - if (r + missing >= quorum) { + if (g + missing >= quorum) { return SYNC_RAFT_VOTE_PENDING; } diff --git a/source/libs/sync/src/sync_raft_restore.c b/source/libs/sync/src/sync_raft_restore.c index 01bc7da7eb..17269254bd 100644 --- a/source/libs/sync/src/sync_raft_restore.c +++ b/source/libs/sync/src/sync_raft_restore.c @@ -17,6 +17,7 @@ #include "sync_raft_restore.h" #include "sync_raft_progress_tracker.h" +static void addToConfChangeSingleArray(SSyncConfChangeSingleArray* out, int* i, const SSyncRaftNodeMap* nodeMap, ESyncRaftConfChangeType t); static int toConfChangeSingle(const SSyncConfigState* cs, SSyncConfChangeSingleArray* out, SSyncConfChangeSingleArray* in); // syncRaftRestoreConfig takes a Changer (which must represent an empty configuration), and @@ -82,6 +83,18 @@ out: return ret; } +static void addToConfChangeSingleArray(SSyncConfChangeSingleArray* out, int* i, const SSyncRaftNodeMap* nodeMap, ESyncRaftConfChangeType t) { + SyncNodeId* pId = NULL; + + while (!syncRaftIterateNodeMap(nodeMap, pId)) { + out->changes[*i] = (SSyncConfChangeSingle) { + .type = t, + .nodeId = *pId, + }; + *i += 1; + } +} + // toConfChangeSingle translates a conf state into 1) a slice of operations creating // first the config that will become the outgoing one, and then the incoming one, and // b) another slice that, when applied to the config resulted from 1), represents the @@ -91,13 +104,16 @@ static int toConfChangeSingle(const SSyncConfigState* cs, SSyncConfChangeSingleA out->n = in->n = 0; - out->n = cs->votersOutgoing.replica; + out->n = syncRaftNodeMapSize(&cs->votersOutgoing); out->changes = (SSyncConfChangeSingle*)malloc(sizeof(SSyncConfChangeSingle) * out->n); if (out->changes == NULL) { out->n = 0; return -1; } - in->n = cs->votersOutgoing.replica + cs->voters.replica + cs->learners.replica + cs->learnersNext.replica; + in->n = syncRaftNodeMapSize(&cs->votersOutgoing) + + syncRaftNodeMapSize(&cs->voters) + + syncRaftNodeMapSize(&cs->learners) + + syncRaftNodeMapSize(&cs->learnersNext); out->changes = (SSyncConfChangeSingle*)malloc(sizeof(SSyncConfChangeSingle) * in->n); if (in->changes == NULL) { in->n = 0; @@ -132,50 +148,24 @@ static int toConfChangeSingle(const SSyncConfigState* cs, SSyncConfChangeSingleA // // as desired. - for (i = 0; i < cs->votersOutgoing.replica; ++i) { - // If there are outgoing voters, first add them one by one so that the - // (non-joint) config has them all. - out->changes[i] = (SSyncConfChangeSingle) { - .type = SYNC_RAFT_Conf_AddNode, - .nodeId = cs->votersOutgoing.nodeId[i], - }; - } + // If there are outgoing voters, first add them one by one so that the + // (non-joint) config has them all. + i = 0; + addToConfChangeSingleArray(out, &i, &cs->votersOutgoing, SYNC_RAFT_Conf_AddNode); + assert(i == out->n); // We're done constructing the outgoing slice, now on to the incoming one // (which will apply on top of the config created by the outgoing slice). - + i = 0; + // First, we'll remove all of the outgoing voters. - int j = 0; - for (i = 0; i < cs->votersOutgoing.replica; ++i) { - in->changes[j] = (SSyncConfChangeSingle) { - .type = SYNC_RAFT_Conf_RemoveNode, - .nodeId = cs->votersOutgoing.nodeId[i], - }; - j += 1; - } + addToConfChangeSingleArray(in, &i, &cs->votersOutgoing, SYNC_RAFT_Conf_RemoveNode); + // Then we'll add the incoming voters and learners. - for (i = 0; i < cs->voters.replica; ++i) { - in->changes[j] = (SSyncConfChangeSingle) { - .type = SYNC_RAFT_Conf_AddNode, - .nodeId = cs->voters.nodeId[i], - }; - j += 1; - } - for (i = 0; i < cs->learners.replica; ++i) { - in->changes[j] = (SSyncConfChangeSingle) { - .type = SYNC_RAFT_Conf_AddLearnerNode, - .nodeId = cs->learners.nodeId[i], - }; - j += 1; - } - // Same for LearnersNext; these are nodes we want to be learners but which - // are currently voters in the outgoing config. - for (i = 0; i < cs->learnersNext.replica; ++i) { - in->changes[j] = (SSyncConfChangeSingle) { - .type = SYNC_RAFT_Conf_AddLearnerNode, - .nodeId = cs->learnersNext.nodeId[i], - }; - j += 1; - } + addToConfChangeSingleArray(in, &i, &cs->voters, SYNC_RAFT_Conf_AddNode); + addToConfChangeSingleArray(in, &i, &cs->learners, SYNC_RAFT_Conf_AddLearnerNode); + addToConfChangeSingleArray(in, &i, &cs->learnersNext, SYNC_RAFT_Conf_AddLearnerNode); + assert(i == in->n); + return 0; } \ No newline at end of file From 7e2590f1087014d631e93bfaec349a0d51fbded1 Mon Sep 17 00:00:00 2001 From: lichuang Date: Thu, 18 Nov 2021 17:19:33 +0800 Subject: [PATCH 07/44] [TD-10645][raft]refactor node and progress map --- .../libs/sync/inc/sync_raft_config_change.h | 5 +++ source/libs/sync/inc/sync_raft_node_map.h | 3 +- source/libs/sync/inc/sync_raft_progress.h | 8 ++++ .../sync/inc/sync_raft_progress_tracker.h | 9 +++- source/libs/sync/inc/sync_raft_proto.h | 15 +++++++ source/libs/sync/inc/sync_raft_quorum_joint.h | 2 + source/libs/sync/inc/sync_raft_restore.h | 3 +- source/libs/sync/src/raft.c | 13 +++++- .../libs/sync/src/sync_raft_config_change.c | 18 ++++---- source/libs/sync/src/sync_raft_node_map.c | 6 ++- source/libs/sync/src/sync_raft_progress.c | 45 ++++++++++++++++++- .../sync/src/sync_raft_progress_tracker.c | 27 ++++++++--- source/libs/sync/src/sync_raft_quorum_joint.c | 10 +++++ source/libs/sync/src/sync_raft_restore.c | 25 +++++++---- 14 files changed, 159 insertions(+), 30 deletions(-) diff --git a/source/libs/sync/inc/sync_raft_config_change.h b/source/libs/sync/inc/sync_raft_config_change.h index a54a7544fe..75a29f35e8 100644 --- a/source/libs/sync/inc/sync_raft_config_change.h +++ b/source/libs/sync/inc/sync_raft_config_change.h @@ -33,6 +33,11 @@ struct SSyncRaftChanger { typedef int (*configChangeFp)(SSyncRaftChanger* changer, const SSyncConfChangeSingleArray* css, SSyncRaftProgressTrackerConfig* config, SSyncRaftProgressMap* progressMap); +// Simple carries out a series of configuration changes that (in aggregate) +// mutates the incoming majority config Voters[0] by at most one. This method +// will return an error if that is not the case, if the resulting quorum is +// zero, or if the configuration is in a joint state (i.e. if there is an +// outgoing configuration). int syncRaftChangerSimpleConfig(SSyncRaftChanger* changer, const SSyncConfChangeSingleArray* css, SSyncRaftProgressTrackerConfig* config, SSyncRaftProgressMap* progressMap); diff --git a/source/libs/sync/inc/sync_raft_node_map.h b/source/libs/sync/inc/sync_raft_node_map.h index 15e559a733..2de4887bf4 100644 --- a/source/libs/sync/inc/sync_raft_node_map.h +++ b/source/libs/sync/inc/sync_raft_node_map.h @@ -25,6 +25,7 @@ struct SSyncRaftNodeMap { }; void syncRaftInitNodeMap(SSyncRaftNodeMap* nodeMap); +void syncRaftFreeNodeMap(SSyncRaftNodeMap* nodeMap); void syncRaftClearNodeMap(SSyncRaftNodeMap* nodeMap); @@ -43,6 +44,6 @@ int32_t syncRaftNodeMapSize(const SSyncRaftNodeMap* nodeMap); // return true if reach the end bool syncRaftIterateNodeMap(const SSyncRaftNodeMap* nodeMap, SyncNodeId *pId); -bool syncRaftIsAllInProgressMap(const SSyncRaftNodeMap* nodeMap, const SSyncRaftProgressMap* progressMap); +bool syncRaftIsAllNodeInProgressMap(SSyncRaftNodeMap* nodeMap, SSyncRaftProgressMap* progressMap); #endif /* _TD_LIBS_SYNC_RAFT_NODE_MAP_H */ \ No newline at end of file diff --git a/source/libs/sync/inc/sync_raft_progress.h b/source/libs/sync/inc/sync_raft_progress.h index 5664cfd15e..7d80ce5438 100644 --- a/source/libs/sync/inc/sync_raft_progress.h +++ b/source/libs/sync/inc/sync_raft_progress.h @@ -74,6 +74,8 @@ struct SSyncRaftProgress { SyncNodeId id; + int16_t refCount; + SyncIndex nextIndex; SyncIndex matchIndex; @@ -221,6 +223,12 @@ static FORCE_INLINE bool syncRaftProgressRecentActive(SSyncRaftProgress* progres return progress->recentActive; } +void syncRaftInitProgressMap(SSyncRaftProgressMap* progressMap); +void syncRaftFreeProgressMap(SSyncRaftProgressMap* progressMap); + +void syncRaftClearProgressMap(SSyncRaftProgressMap* progressMap); +void syncRaftCopyProgressMap(SSyncRaftProgressMap* from, SSyncRaftProgressMap* to); + SSyncRaftProgress* syncRaftFindProgressByNodeId(const SSyncRaftProgressMap* progressMap, SyncNodeId id); int syncRaftAddToProgressMap(SSyncRaftProgressMap* progressMap, SSyncRaftProgress* progress); diff --git a/source/libs/sync/inc/sync_raft_progress_tracker.h b/source/libs/sync/inc/sync_raft_progress_tracker.h index 35daf8139c..ff69b7b1d1 100644 --- a/source/libs/sync/inc/sync_raft_progress_tracker.h +++ b/source/libs/sync/inc/sync_raft_progress_tracker.h @@ -94,6 +94,11 @@ struct SSyncRaftProgressTracker { SSyncRaftProgressTracker* syncRaftOpenProgressTracker(SSyncRaft* pRaft); +void syncRaftInitTrackConfig(SSyncRaftProgressTrackerConfig* config); +void syncRaftFreeTrackConfig(SSyncRaftProgressTrackerConfig* config); + +void syncRaftFreeTrackConfig(SSyncRaftProgressTrackerConfig* config); + void syncRaftResetVotes(SSyncRaftProgressTracker*); void syncRaftProgressVisit(SSyncRaftProgressTracker*, visitProgressFp visit, void* arg); @@ -104,9 +109,9 @@ void syncRaftProgressVisit(SSyncRaftProgressTracker*, visitProgressFp visit, voi **/ void syncRaftRecordVote(SSyncRaftProgressTracker* tracker, SyncNodeId id, bool grant); -void syncRaftCloneTrackerConfig(const SSyncRaftProgressTrackerConfig* config, SSyncRaftProgressTrackerConfig* result); +void syncRaftCopyTrackerConfig(const SSyncRaftProgressTrackerConfig* from, SSyncRaftProgressTrackerConfig* to); -int syncRaftCheckProgress(const SSyncRaftProgressTrackerConfig* config, SSyncRaftProgressMap* progressMap); +int syncRaftCheckTrackerConfigInProgress(SSyncRaftProgressTrackerConfig* config, SSyncRaftProgressMap* progressMap); /** * syncRaftTallyVotes returns the number of granted and rejected Votes, and whether the diff --git a/source/libs/sync/inc/sync_raft_proto.h b/source/libs/sync/inc/sync_raft_proto.h index dd153e8dad..29371e328d 100644 --- a/source/libs/sync/inc/sync_raft_proto.h +++ b/source/libs/sync/inc/sync_raft_proto.h @@ -59,4 +59,19 @@ typedef struct SSyncConfigState { bool autoLeave; } SSyncConfigState; +static FORCE_INLINE bool syncRaftConfArrayIsEmpty(const SSyncConfChangeSingleArray* ary) { + return ary->n == 0; +} + +static FORCE_INLINE void syncRaftInitConfArray(SSyncConfChangeSingleArray* ary) { + *ary = (SSyncConfChangeSingleArray) { + .changes = NULL, + .n = 0, + }; +} + +static FORCE_INLINE void syncRaftFreeConfArray(SSyncConfChangeSingleArray* ary) { + if (ary->changes != NULL) free(ary->changes); +} + #endif /* TD_SYNC_RAFT_PROTO_H */ diff --git a/source/libs/sync/inc/sync_raft_quorum_joint.h b/source/libs/sync/inc/sync_raft_quorum_joint.h index 59d1fadc4a..92cddaaec1 100644 --- a/source/libs/sync/inc/sync_raft_quorum_joint.h +++ b/source/libs/sync/inc/sync_raft_quorum_joint.h @@ -38,6 +38,8 @@ typedef struct SSyncRaftQuorumJointConfig { **/ ESyncRaftVoteType syncRaftVoteResult(SSyncRaftQuorumJointConfig* config, SHashObj* votesMap); +void syncRaftInitQuorumJointConfig(SSyncRaftQuorumJointConfig* config); + static FORCE_INLINE bool syncRaftJointConfigInOutgoing(const SSyncRaftQuorumJointConfig* config, SyncNodeId id) { return syncRaftIsInNodeMap(&config->outgoing, id); } diff --git a/source/libs/sync/inc/sync_raft_restore.h b/source/libs/sync/inc/sync_raft_restore.h index 38eadb00c7..df4448cab8 100644 --- a/source/libs/sync/inc/sync_raft_restore.h +++ b/source/libs/sync/inc/sync_raft_restore.h @@ -27,6 +27,7 @@ // the Changer only needs a ProgressMap (not a whole Tracker) at which point // this can just take LastIndex and MaxInflight directly instead and cook up // the results from that alone. -int syncRaftRestoreConfig(SSyncRaftChanger* changer, const SSyncConfigState* cs); +int syncRaftRestoreConfig(SSyncRaftChanger* changer, const SSyncConfigState* cs, + SSyncRaftProgressTrackerConfig* config, SSyncRaftProgressMap* progressMap); #endif /* TD_SYNC_RAFT_RESTORE_H */ diff --git a/source/libs/sync/src/raft.c b/source/libs/sync/src/raft.c index 85c330ece3..3b4c9e5f36 100644 --- a/source/libs/sync/src/raft.c +++ b/source/libs/sync/src/raft.c @@ -101,11 +101,22 @@ int32_t syncRaftStart(SSyncRaft* pRaft, const SSyncInfo* pInfo) { .tracker = pRaft->tracker, .lastIndex = syncRaftLogLastIndex(pRaft->log), }; - if (syncRaftRestoreConfig(&changer, &confState) < 0) { + SSyncRaftProgressTrackerConfig config; + SSyncRaftProgressMap progressMap; + + if (syncRaftRestoreConfig(&changer, &confState, &config, &progressMap) < 0) { syncError("syncRaftRestoreConfig for vgid %d fail", pInfo->vgId); return -1; } + // save restored config and progress map to tracker + syncRaftCopyProgressMap(&progressMap, &pRaft->tracker->progressMap); + syncRaftCopyTrackerConfig(&config, &pRaft->tracker->config); + + // free progress map and config + syncRaftFreeProgressMap(&progressMap); + syncRaftFreeTrackConfig(&config); + if (!syncRaftIsEmptyServerState(&serverState)) { syncRaftLoadState(pRaft, &serverState); } diff --git a/source/libs/sync/src/sync_raft_config_change.c b/source/libs/sync/src/sync_raft_config_change.c index 1f7aab064f..de790b5876 100644 --- a/source/libs/sync/src/sync_raft_config_change.c +++ b/source/libs/sync/src/sync_raft_config_change.c @@ -92,7 +92,7 @@ int syncRaftChangerEnterJoint(SSyncRaftChanger* changer, bool autoLeave, const S return checkAndReturn(config, progressMap); } -// syncRaftChangerSimpleConfig carries out a series of configuration changes that (in aggregate) +// Simple carries out a series of configuration changes that (in aggregate) // mutates the incoming majority config Voters[0] by at most one. This method // will return an error if that is not the case, if the resulting quorum is // zero, or if the configuration is in a joint state (i.e. if there is an @@ -275,7 +275,8 @@ static void initProgress(SSyncRaftChanger* changer, SSyncRaftProgressTrackerConf // When a node is first added, we should mark it as recently active. // Otherwise, CheckQuorum may cause us to step down if it is invoked // before the added node has had a chance to communicate with us. - .recentActive = true, + .recentActive = true, + .refCount = 0, }; syncRaftAddToProgressMap(progressMap, pProgress); @@ -285,7 +286,7 @@ static void initProgress(SSyncRaftChanger* changer, SSyncRaftProgressTrackerConf // each other. This is used to check both what the Changer is initialized with, // as well as what it returns. static int checkInvariants(SSyncRaftProgressTrackerConfig* config, SSyncRaftProgressMap* progressMap) { - int ret = syncRaftCheckProgress(config, progressMap); + int ret = syncRaftCheckTrackerConfigInProgress(config, progressMap); if (ret != 0) { return ret; } @@ -296,6 +297,7 @@ static int checkInvariants(SSyncRaftProgressTrackerConfig* config, SSyncRaftProg while (!syncRaftIterateNodeMap(&config->learnersNext, pNodeId)) { SyncNodeId nodeId = *pNodeId; if (!syncRaftJointConfigInOutgoing(&config->voters, nodeId)) { + syncError("[%d] is in LearnersNext, but not outgoing", nodeId); return -1; } SSyncRaftProgress* progress = syncRaftFindProgressByNodeId(progressMap, nodeId); @@ -311,8 +313,8 @@ static int checkInvariants(SSyncRaftProgressTrackerConfig* config, SSyncRaftProg pNodeId = NULL; while (!syncRaftIterateNodeMap(&config->learners, pNodeId)) { SyncNodeId nodeId = *pNodeId; - if (syncRaftJointConfigInIncoming(&config->voters, nodeId)) { - syncError("%d is in Learners and voter.incoming", nodeId); + if (syncRaftJointConfigInOutgoing(&config->voters, nodeId)) { + syncError("%d is in Learners and outgoing", nodeId); return -1; } SSyncRaftProgress* progress = syncRaftFindProgressByNodeId(progressMap, nodeId); @@ -327,7 +329,7 @@ static int checkInvariants(SSyncRaftProgressTrackerConfig* config, SSyncRaftProg if (!hasJointConfig(config)) { // We enforce that empty maps are nil instead of zero. - if (syncRaftNodeMapSize(&config->learnersNext)) { + if (syncRaftNodeMapSize(&config->learnersNext) > 0) { syncError("cfg.LearnersNext must be nil when not joint"); return -1; } @@ -344,8 +346,8 @@ static int checkInvariants(SSyncRaftProgressTrackerConfig* config, SSyncRaftProg // the purposes of the Changer) and returns those copies. It returns an error // if checkInvariants does. static int checkAndCopy(SSyncRaftChanger* changer, SSyncRaftProgressTrackerConfig* config, SSyncRaftProgressMap* progressMap) { - syncRaftCloneTrackerConfig(&changer->tracker->config, config); - int i; + syncRaftCopyTrackerConfig(&changer->tracker->config, config); + syncRaftClearProgressMap(progressMap); SSyncRaftProgress* pProgress = NULL; while (!syncRaftIterateProgressMap(&changer->tracker->progressMap, pProgress)) { diff --git a/source/libs/sync/src/sync_raft_node_map.c b/source/libs/sync/src/sync_raft_node_map.c index 022f3b2dcb..1c54d32b59 100644 --- a/source/libs/sync/src/sync_raft_node_map.c +++ b/source/libs/sync/src/sync_raft_node_map.c @@ -21,6 +21,10 @@ void syncRaftInitNodeMap(SSyncRaftNodeMap* nodeMap) { nodeMap->nodeIdMap = taosHashInit(TSDB_MAX_REPLICA, taosGetDefaultHashFunction(TSDB_DATA_TYPE_INT), true, HASH_ENTRY_LOCK); } +void syncRaftFreeNodeMap(SSyncRaftNodeMap* nodeMap) { + taosHashCleanup(nodeMap->nodeIdMap); +} + void syncRaftClearNodeMap(SSyncRaftNodeMap* nodeMap) { taosHashClear(nodeMap->nodeIdMap); } @@ -51,7 +55,7 @@ bool syncRaftIterateNodeMap(const SSyncRaftNodeMap* nodeMap, SyncNodeId *pId) { return false; } -bool syncRaftIsAllInProgressMap(const SSyncRaftNodeMap* nodeMap, const SSyncRaftProgressMap* progressMap) { +bool syncRaftIsAllNodeInProgressMap(SSyncRaftNodeMap* nodeMap, SSyncRaftProgressMap* progressMap) { SyncNodeId *pId = NULL; while (!syncRaftIterateNodeMap(nodeMap, pId)) { if (!syncRaftIsInProgressMap(progressMap, *pId)) { diff --git a/source/libs/sync/src/sync_raft_progress.c b/source/libs/sync/src/sync_raft_progress.c index e63d37cee9..65676655ec 100644 --- a/source/libs/sync/src/sync_raft_progress.c +++ b/source/libs/sync/src/sync_raft_progress.c @@ -20,6 +20,11 @@ #include "sync.h" #include "syncInt.h" +static void copyProgress(SSyncRaftProgress* progress, void* arg); + +static void refProgress(SSyncRaftProgress* progress); +static void unrefProgress(SSyncRaftProgress* progress, void*); + static void resetProgressState(SSyncRaftProgress* progress, ESyncRaftProgressState state); static void probeAcked(SSyncRaftProgress* progress); @@ -125,6 +130,7 @@ SSyncRaftProgress* syncRaftFindProgressByNodeId(const SSyncRaftProgressMap* prog } int syncRaftAddToProgressMap(SSyncRaftProgressMap* progressMap, SSyncRaftProgress* progress) { + refProgress(progress); taosHashPut(progressMap->progressMap, &progress->id, sizeof(SyncNodeId*), &progress, sizeof(SSyncRaftProgress*)); } @@ -133,7 +139,8 @@ void syncRaftRemoveFromProgressMap(SSyncRaftProgressMap* progressMap, SyncNodeId if (ppProgress == NULL) { return; } - free(*ppProgress); + unrefProgress(*ppProgress, NULL); + taosHashRemove(progressMap->progressMap, &id, sizeof(SyncNodeId*)); } @@ -182,6 +189,23 @@ void syncRaftCopyProgress(const SSyncRaftProgress* progress, SSyncRaftProgress* memcpy(out, progress, sizeof(SSyncRaftProgress)); } +void syncRaftInitProgressMap(SSyncRaftProgressMap* progressMap) { + progressMap->progressMap = taosHashInit(TSDB_MAX_REPLICA, taosGetDefaultHashFunction(TSDB_DATA_TYPE_INT), true, HASH_ENTRY_LOCK); +} + +void syncRaftFreeProgressMap(SSyncRaftProgressMap* progressMap) { + syncRaftVisitProgressMap(progressMap, unrefProgress, NULL); + taosHashCleanup(progressMap->progressMap); +} + +void syncRaftClearProgressMap(SSyncRaftProgressMap* progressMap) { + taosHashClear(progressMap->progressMap); +} + +void syncRaftCopyProgressMap(SSyncRaftProgressMap* from, SSyncRaftProgressMap* to) { + syncRaftVisitProgressMap(from, copyProgress, to); +} + bool syncRaftIterateProgressMap(const SSyncRaftProgressMap* progressMap, SSyncRaftProgress *pProgress) { SSyncRaftProgress **ppProgress = taosHashIterate(progressMap->progressMap, pProgress); if (ppProgress == NULL) { @@ -199,6 +223,25 @@ bool syncRaftVisitProgressMap(SSyncRaftProgressMap* progressMap, visitProgressFp } } +static void copyProgress(SSyncRaftProgress* progress, void* arg) { + assert(progress->refCount > 0); + SSyncRaftProgressMap* to = (SSyncRaftProgressMap*)arg; + syncRaftAddToProgressMap(to, progress); +} + +static void refProgress(SSyncRaftProgress* progress) { + progress->refCount += 1; +} + +static void unrefProgress(SSyncRaftProgress* progress, void* arg) { + (void)arg; + progress->refCount -= 1; + assert(progress->refCount >= 0); + if (progress->refCount == 0) { + free(progress); + } +} + /** * ResetState moves the Progress into the specified State, resetting ProbeSent, * PendingSnapshot, and Inflights. diff --git a/source/libs/sync/src/sync_raft_progress_tracker.c b/source/libs/sync/src/sync_raft_progress_tracker.c index e6a016b7cf..60e3ccea6a 100644 --- a/source/libs/sync/src/sync_raft_progress_tracker.c +++ b/source/libs/sync/src/sync_raft_progress_tracker.c @@ -22,13 +22,26 @@ SSyncRaftProgressTracker* syncRaftOpenProgressTracker(SSyncRaft* pRaft) { return NULL; } - syncRaftInitNodeMap(&tracker->config.learners); + syncRaftInitTrackConfig(&tracker->config); syncRaftInitNodeMap(&tracker->config.learnersNext); tracker->pRaft = pRaft; return tracker; } +void syncRaftInitTrackConfig(SSyncRaftProgressTrackerConfig* config) { + syncRaftInitNodeMap(&config->learners); + syncRaftInitNodeMap(&config->learnersNext); + syncRaftInitQuorumJointConfig(&config->voters); + config->autoLeave = false; +} + +void syncRaftFreeTrackConfig(SSyncRaftProgressTrackerConfig* config) { + syncRaftFreeNodeMap(&config->learners); + syncRaftFreeNodeMap(&config->learnersNext); + syncRaftFreeQuorumJointConfig(&config->voters); +} + void syncRaftResetVotes(SSyncRaftProgressTracker* tracker) { taosHashClear(tracker->votesMap); } @@ -47,21 +60,21 @@ void syncRaftRecordVote(SSyncRaftProgressTracker* tracker, SyncNodeId id, bool g taosHashPut(tracker->votesMap, &id, sizeof(SyncNodeId), &type, sizeof(ESyncRaftVoteType*)); } -void syncRaftCloneTrackerConfig(const SSyncRaftProgressTrackerConfig* from, SSyncRaftProgressTrackerConfig* to) { +void syncRaftCopyTrackerConfig(const SSyncRaftProgressTrackerConfig* from, SSyncRaftProgressTrackerConfig* to) { memcpy(to, from, sizeof(SSyncRaftProgressTrackerConfig)); } -int syncRaftCheckProgress(const SSyncRaftProgressTrackerConfig* config, SSyncRaftProgressMap* progressMap) { +int syncRaftCheckTrackerConfigInProgress(SSyncRaftProgressTrackerConfig* config, SSyncRaftProgressMap* progressMap) { // NB: intentionally allow the empty config. In production we'll never see a // non-empty config (we prevent it from being created) but we will need to // be able to *create* an initial config, for example during bootstrap (or // during tests). Instead of having to hand-code this, we allow // transitioning from an empty config into any other legal and non-empty // config. - if (!syncRaftIsAllInProgressMap(&config->voters.incoming, progressMap)) return -1; - if (!syncRaftIsAllInProgressMap(&config->voters.outgoing, progressMap)) return -1; - if (!syncRaftIsAllInProgressMap(&config->learners, progressMap)) return -1; - if (!syncRaftIsAllInProgressMap(&config->learnersNext, progressMap)) return -1; + if (!syncRaftIsAllNodeInProgressMap(&config->voters.incoming, progressMap)) return -1; + if (!syncRaftIsAllNodeInProgressMap(&config->voters.outgoing, progressMap)) return -1; + if (!syncRaftIsAllNodeInProgressMap(&config->learners, progressMap)) return -1; + if (!syncRaftIsAllNodeInProgressMap(&config->learnersNext, progressMap)) return -1; return 0; } diff --git a/source/libs/sync/src/sync_raft_quorum_joint.c b/source/libs/sync/src/sync_raft_quorum_joint.c index 6eecfbd9e5..500bd908c0 100644 --- a/source/libs/sync/src/sync_raft_quorum_joint.c +++ b/source/libs/sync/src/sync_raft_quorum_joint.c @@ -41,6 +41,16 @@ ESyncRaftVoteType syncRaftVoteResult(SSyncRaftQuorumJointConfig* config, SHashOb return SYNC_RAFT_VOTE_PENDING; } +void syncRaftInitQuorumJointConfig(SSyncRaftQuorumJointConfig* config) { + syncRaftInitNodeMap(&config->incoming); + syncRaftInitNodeMap(&config->outgoing); +} + +void syncRaftFreeQuorumJointConfig(SSyncRaftQuorumJointConfig* config) { + syncRaftFreeNodeMap(&config->incoming); + syncRaftFreeNodeMap(&config->outgoing); +} + void syncRaftJointConfigAddToIncoming(SSyncRaftQuorumJointConfig* config, SyncNodeId id) { syncRaftAddToNodeMap(&config->incoming, id); } diff --git a/source/libs/sync/src/sync_raft_restore.c b/source/libs/sync/src/sync_raft_restore.c index 17269254bd..d1acd3e8e9 100644 --- a/source/libs/sync/src/sync_raft_restore.c +++ b/source/libs/sync/src/sync_raft_restore.c @@ -28,21 +28,26 @@ static int toConfChangeSingle(const SSyncConfigState* cs, SSyncConfChangeSingleA // the Changer only needs a ProgressMap (not a whole Tracker) at which point // this can just take LastIndex and MaxInflight directly instead and cook up // the results from that alone. -int syncRaftRestoreConfig(SSyncRaftChanger* changer, const SSyncConfigState* cs) { +int syncRaftRestoreConfig(SSyncRaftChanger* changer, const SSyncConfigState* cs, + SSyncRaftProgressTrackerConfig* config, SSyncRaftProgressMap* progressMap) { SSyncConfChangeSingleArray outgoing; SSyncConfChangeSingleArray incoming; SSyncConfChangeSingleArray css; SSyncRaftProgressTracker* tracker = changer->tracker; - SSyncRaftProgressTrackerConfig* config = &tracker->config; - SSyncRaftProgressMap* progressMap = &tracker->progressMap; int i, ret; + syncRaftInitConfArray(&outgoing); + syncRaftInitConfArray(&incoming); + + syncRaftInitTrackConfig(config); + syncRaftInitProgressMap(progressMap); + ret = toConfChangeSingle(cs, &outgoing, &incoming); if (ret != 0) { goto out; } - if (outgoing.n == 0) { + if (syncRaftConfArrayIsEmpty(&outgoing)) { // No outgoing config, so just apply the incoming changes one by one. for (i = 0; i < incoming.n; ++i) { css = (SSyncConfChangeSingleArray) { @@ -53,6 +58,9 @@ int syncRaftRestoreConfig(SSyncRaftChanger* changer, const SSyncConfigState* cs) if (ret != 0) { goto out; } + + syncRaftCopyTrackerConfig(config, &changer->tracker->config); + syncRaftCopyProgressMap(progressMap, &changer->tracker->progressMap); } } else { // The ConfState describes a joint configuration. @@ -69,6 +77,8 @@ int syncRaftRestoreConfig(SSyncRaftChanger* changer, const SSyncConfigState* cs) if (ret != 0) { goto out; } + syncRaftCopyTrackerConfig(config, &changer->tracker->config); + syncRaftCopyProgressMap(progressMap, &changer->tracker->progressMap); } ret = syncRaftChangerEnterJoint(changer, cs->autoLeave, &incoming, config, progressMap); @@ -78,8 +88,9 @@ int syncRaftRestoreConfig(SSyncRaftChanger* changer, const SSyncConfigState* cs) } out: - if (incoming.n != 0) free(incoming.changes); - if (outgoing.n != 0) free(outgoing.changes); + syncRaftFreeConfArray(&incoming); + syncRaftFreeConfArray(&outgoing); + return ret; } @@ -102,8 +113,6 @@ static void addToConfChangeSingleArray(SSyncConfChangeSingleArray* out, int* i, static int toConfChangeSingle(const SSyncConfigState* cs, SSyncConfChangeSingleArray* out, SSyncConfChangeSingleArray* in) { int i; - out->n = in->n = 0; - out->n = syncRaftNodeMapSize(&cs->votersOutgoing); out->changes = (SSyncConfChangeSingle*)malloc(sizeof(SSyncConfChangeSingle) * out->n); if (out->changes == NULL) { From ce654f835a5b31cff3fbcfe5c52b488689a0335f Mon Sep 17 00:00:00 2001 From: lichuang Date: Fri, 19 Nov 2021 10:18:56 +0800 Subject: [PATCH 08/44] [TD-10645][raft]refactor node and progress map --- source/libs/sync/inc/sync_const.h | 25 +++ source/libs/sync/inc/sync_raft_inflights.h | 41 ++--- source/libs/sync/inc/sync_raft_node_map.h | 4 +- source/libs/sync/inc/sync_raft_progress.h | 159 ++++++++---------- .../sync/inc/sync_raft_progress_tracker.h | 24 ++- source/libs/sync/inc/sync_raft_quorum_joint.h | 25 +-- .../libs/sync/inc/sync_raft_quorum_majority.h | 4 + source/libs/sync/inc/sync_type.h | 2 + source/libs/sync/src/sync_raft_election.c | 2 +- source/libs/sync/src/sync_raft_inflights.c | 25 +-- source/libs/sync/src/sync_raft_node_map.c | 11 +- source/libs/sync/src/sync_raft_progress.c | 155 +++++------------ .../sync/src/sync_raft_progress_tracker.c | 67 ++++++-- source/libs/sync/src/sync_raft_quorum_joint.c | 11 +- .../libs/sync/src/sync_raft_quorum_majority.c | 68 +++++++- 15 files changed, 333 insertions(+), 290 deletions(-) create mode 100644 source/libs/sync/inc/sync_const.h diff --git a/source/libs/sync/inc/sync_const.h b/source/libs/sync/inc/sync_const.h new file mode 100644 index 0000000000..b49c17f82e --- /dev/null +++ b/source/libs/sync/inc/sync_const.h @@ -0,0 +1,25 @@ +/* + * 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_CONST_H +#define _TD_LIBS_SYNC_CONST_H + +#include "sync.h" + +static int kSyncRaftMaxInflghtMsgs = 20; + +static SyncIndex kMaxCommitIndex = UINT64_MAX; + +#endif /* _TD_LIBS_SYNC_CONST_H */ diff --git a/source/libs/sync/inc/sync_raft_inflights.h b/source/libs/sync/inc/sync_raft_inflights.h index 6d249c9274..627bf9a26f 100644 --- a/source/libs/sync/inc/sync_raft_inflights.h +++ b/source/libs/sync/inc/sync_raft_inflights.h @@ -18,54 +18,47 @@ #include "sync.h" -/** - * SSyncRaftInflights limits the number of MsgApp (represented by the largest index - * contained within) sent to followers but not yet acknowledged by them. Callers - * use syncRaftInflightFull() to check whether more messages can be sent, - * call syncRaftInflightAdd() whenever they are sending a new append, - * and release "quota" via FreeLE() whenever an ack is received. -**/ +// Inflights limits the number of MsgApp (represented by the largest index +// contained within) sent to followers but not yet acknowledged by them. Callers +// use Full() to check whether more messages can be sent, call Add() whenever +// they are sending a new append, and release "quota" via FreeLE() whenever an +// ack is received. typedef struct SSyncRaftInflights { - /* the starting index in the buffer */ + // the starting index in the buffer int start; - /* number of inflights in the buffer */ + // number of inflights in the buffer int count; - /* the size of the buffer */ + // the size of the buffer int size; - /** - * buffer contains the index of the last entry - * inside one message. - **/ + // buffer contains the index of the last entry + // inside one message. SyncIndex* buffer; } SSyncRaftInflights; SSyncRaftInflights* syncRaftOpenInflights(int size); void syncRaftCloseInflights(SSyncRaftInflights*); +// reset frees all inflights. static FORCE_INLINE void syncRaftInflightReset(SSyncRaftInflights* inflights) { inflights->count = 0; inflights->start = 0; } +// Full returns true if no more messages can be sent at the moment. static FORCE_INLINE bool syncRaftInflightFull(SSyncRaftInflights* inflights) { return inflights->count == inflights->size; } -/** - * syncRaftInflightAdd notifies the Inflights that a new message with the given index is being - * dispatched. syncRaftInflightFull() must be called prior to syncRaftInflightAdd() - * to verify that there is room for one more message, - * and consecutive calls to add syncRaftInflightAdd() must provide a - * monotonic sequence of indexes. - **/ +// Add notifies the Inflights that a new message with the given index is being +// dispatched. Full() must be called prior to Add() to verify that there is room +// for one more message, and consecutive calls to add Add() must provide a +// monotonic sequence of indexes. void syncRaftInflightAdd(SSyncRaftInflights* inflights, SyncIndex inflightIndex); -/** - * syncRaftInflightFreeLE frees the inflights smaller or equal to the given `to` flight. - **/ +// FreeLE frees the inflights smaller or equal to the given `to` flight. void syncRaftInflightFreeLE(SSyncRaftInflights* inflights, SyncIndex toIndex); /** diff --git a/source/libs/sync/inc/sync_raft_node_map.h b/source/libs/sync/inc/sync_raft_node_map.h index 2de4887bf4..b4cf04056d 100644 --- a/source/libs/sync/inc/sync_raft_node_map.h +++ b/source/libs/sync/inc/sync_raft_node_map.h @@ -31,9 +31,9 @@ void syncRaftClearNodeMap(SSyncRaftNodeMap* nodeMap); bool syncRaftIsInNodeMap(const SSyncRaftNodeMap* nodeMap, SyncNodeId nodeId); -void syncRaftCopyNodeMap(const SSyncRaftNodeMap* from, SSyncRaftNodeMap* to); +void syncRaftCopyNodeMap(SSyncRaftNodeMap* from, SSyncRaftNodeMap* to); -void syncRaftUnionNodeMap(const SSyncRaftNodeMap* nodeMap, SSyncRaftNodeMap* to); +void syncRaftUnionNodeMap(SSyncRaftNodeMap* nodeMap, SSyncRaftNodeMap* to); void syncRaftAddToNodeMap(SSyncRaftNodeMap* nodeMap, SyncNodeId nodeId); diff --git a/source/libs/sync/inc/sync_raft_progress.h b/source/libs/sync/inc/sync_raft_progress.h index 7d80ce5438..32c21281cd 100644 --- a/source/libs/sync/inc/sync_raft_progress.h +++ b/source/libs/sync/inc/sync_raft_progress.h @@ -65,10 +65,13 @@ static const char* kProgressStateString[] = { "Snapshot", }; -/** - * Progress represents a follower’s progress in the view of the leader. Leader maintains - * progresses of all followers, and sends entries to the follower based on its progress. - **/ +// Progress represents a follower’s progress in the view of the leader. Leader +// maintains progresses of all followers, and sends entries to the follower +// based on its progress. +// +// NB(tbg): Progress is basically a state machine whose transitions are mostly +// strewn around `*raft.raft`. Additionally, some fields are only used when in a +// certain State. All of this isn't ideal. struct SSyncRaftProgress { SyncGroupId groupId; @@ -80,63 +83,53 @@ struct SSyncRaftProgress { SyncIndex matchIndex; - /** - * State defines how the leader should interact with the follower. - * - * When in StateProbe, leader sends at most one replication message - * per heartbeat interval. It also probes actual progress of the follower. - * - * When in StateReplicate, leader optimistically increases next - * to the latest entry sent after sending replication message. This is - * an optimized state for fast replicating log entries to the follower. - * - * When in StateSnapshot, leader should have sent out snapshot - * before and stops sending any replication message. - **/ + // State defines how the leader should interact with the follower. + // + // When in StateProbe, leader sends at most one replication message + // per heartbeat interval. It also probes actual progress of the follower. + // + // When in StateReplicate, leader optimistically increases next + // to the latest entry sent after sending replication message. This is + // an optimized state for fast replicating log entries to the follower. + // + // When in StateSnapshot, leader should have sent out snapshot + // before and stops sending any replication message. ESyncRaftProgressState state; - /** - * pendingSnapshotIndex is used in PROGRESS_STATE_SNAPSHOT. - * If there is a pending snapshot, the pendingSnapshotIndex will be set to the - * index of the snapshot. If pendingSnapshotIndex is set, the replication process of - * this Progress will be paused. raft will not resend snapshot until the pending one - * is reported to be failed. - **/ + // PendingSnapshot is used in StateSnapshot. + // If there is a pending snapshot, the pendingSnapshot will be set to the + // index of the snapshot. If pendingSnapshot is set, the replication process of + // this Progress will be paused. raft will not resend snapshot until the pending one + // is reported to be failed. SyncIndex pendingSnapshotIndex; - /** - * recentActive is true if the progress is recently active. Receiving any messages - * from the corresponding follower indicates the progress is active. - * RecentActive can be reset to false after an election timeout. - **/ + // RecentActive is true if the progress is recently active. Receiving any messages + // from the corresponding follower indicates the progress is active. + // RecentActive can be reset to false after an election timeout. + // + // TODO(tbg): the leader should always have this set to true. bool recentActive; - /** - * probeSent is used while this follower is in StateProbe. When probeSent is - * true, raft should pause sending replication message to this peer until - * probeSent is reset. See ProbeAcked() and IsPaused(). - **/ + // ProbeSent is used while this follower is in StateProbe. When ProbeSent is + // true, raft should pause sending replication message to this peer until + // ProbeSent is reset. See ProbeAcked() and IsPaused(). bool probeSent; - /** - * inflights is a sliding window for the inflight messages. - * Each inflight message contains one or more log entries. - * The max number of entries per message is defined in raft config as MaxSizePerMsg. - * Thus inflight effectively limits both the number of inflight messages - * and the bandwidth each Progress can use. - * When inflights is Full, no more message should be sent. - * When a leader sends out a message, the index of the last - * entry should be added to inflights. The index MUST be added - * into inflights in order. - * When a leader receives a reply, the previous inflights should - * be freed by calling inflights.FreeLE with the index of the last - * received entry. - **/ + // Inflights is a sliding window for the inflight messages. + // Each inflight message contains one or more log entries. + // The max number of entries per message is defined in raft config as MaxSizePerMsg. + // Thus inflight effectively limits both the number of inflight messages + // and the bandwidth each Progress can use. + // When inflights is Full, no more message should be sent. + // When a leader sends out a message, the index of the last + // entry should be added to inflights. The index MUST be added + // into inflights in order. + // When a leader receives a reply, the previous inflights should + // be freed by calling inflights.FreeLE with the index of the last + // received entry. SSyncRaftInflights* inflights; - /** - * IsLearner is true if this progress is tracked for a learner. - **/ + // IsLearner is true if this progress is tracked for a learner. bool isLearner; }; @@ -151,56 +144,44 @@ static FORCE_INLINE const char* syncRaftProgressStateString(const SSyncRaftProgr void syncRaftResetProgress(SSyncRaft* pRaft, SSyncRaftProgress* progress); -/** - * syncRaftProgressBecomeProbe transitions into StateProbe. Next is reset to Match+1 or, - * optionally and if larger, the index of the pending snapshot. - **/ +// BecomeProbe transitions into StateProbe. Next is reset to Match+1 or, +// optionally and if larger, the index of the pending snapshot. void syncRaftProgressBecomeProbe(SSyncRaftProgress* progress); -/** - * syncRaftProgressBecomeReplicate transitions into StateReplicate, resetting Next to Match+1. - **/ +// BecomeReplicate transitions into StateReplicate, resetting Next to Match+1. void syncRaftProgressBecomeReplicate(SSyncRaftProgress* progress); -/** - * syncRaftProgressMaybeUpdate is called when an MsgAppResp arrives from the follower, with the - * index acked by it. The method returns false if the given n index comes from - * an outdated message. Otherwise it updates the progress and returns true. - **/ +// MaybeUpdate is called when an MsgAppResp arrives from the follower, with the +// index acked by it. The method returns false if the given n index comes from +// an outdated message. Otherwise it updates the progress and returns true. bool syncRaftProgressMaybeUpdate(SSyncRaftProgress* progress, SyncIndex lastIndex); -/** - * syncRaftProgressOptimisticNextIndex signals that appends all the way up to and including index n - * are in-flight. As a result, Next is increased to n+1. - **/ +// OptimisticUpdate signals that appends all the way up to and including index n +// are in-flight. As a result, Next is increased to n+1. static FORCE_INLINE void syncRaftProgressOptimisticNextIndex(SSyncRaftProgress* progress, SyncIndex nextIndex) { progress->nextIndex = nextIndex + 1; } -/** - * syncRaftProgressMaybeDecrTo adjusts the Progress to the receipt of a MsgApp rejection. The - * arguments are the index of the append message rejected by the follower, and - * the hint that we want to decrease to. - * - * Rejections can happen spuriously as messages are sent out of order or - * duplicated. In such cases, the rejection pertains to an index that the - * Progress already knows were previously acknowledged, and false is returned - * without changing the Progress. - * - * If the rejection is genuine, Next is lowered sensibly, and the Progress is - * cleared for sending log entries. -**/ +// MaybeDecrTo adjusts the Progress to the receipt of a MsgApp rejection. The +// arguments are the index of the append message rejected by the follower, and +// the hint that we want to decrease to. +// +// Rejections can happen spuriously as messages are sent out of order or +// duplicated. In such cases, the rejection pertains to an index that the +// Progress already knows were previously acknowledged, and false is returned +// without changing the Progress. +// +// If the rejection is genuine, Next is lowered sensibly, and the Progress is +// cleared for sending log entries. bool syncRaftProgressMaybeDecrTo(SSyncRaftProgress* progress, SyncIndex rejected, SyncIndex matchHint); -/** - * syncRaftProgressIsPaused returns whether sending log entries to this node has been throttled. - * This is done when a node has rejected recent MsgApps, is currently waiting - * for a snapshot, or has reached the MaxInflightMsgs limit. In normal - * operation, this is false. A throttled node will be contacted less frequently - * until it has reached a state in which it's able to accept a steady stream of - * log entries again. - **/ +// IsPaused returns whether sending log entries to this node has been throttled. +// This is done when a node has rejected recent MsgApps, is currently waiting +// for a snapshot, or has reached the MaxInflightMsgs limit. In normal +// operation, this is false. A throttled node will be contacted less frequently +// until it has reached a state in which it's able to accept a steady stream of +// log entries again. bool syncRaftProgressIsPaused(SSyncRaftProgress* progress); static FORCE_INLINE SyncIndex syncRaftProgressNextIndex(SSyncRaftProgress* progress) { @@ -242,6 +223,8 @@ bool syncRaftIsInProgressMap(SSyncRaftProgressMap* progressMap, SyncNodeId id); **/ bool syncRaftProgressIsUptodate(SSyncRaft* pRaft, SSyncRaftProgress* progress); +// BecomeSnapshot moves the Progress to StateSnapshot with the specified pending +// snapshot index. void syncRaftProgressBecomeSnapshot(SSyncRaftProgress* progress, SyncIndex snapshotIndex); void syncRaftCopyProgress(const SSyncRaftProgress* from, SSyncRaftProgress* to); diff --git a/source/libs/sync/inc/sync_raft_progress_tracker.h b/source/libs/sync/inc/sync_raft_progress_tracker.h index ff69b7b1d1..0a3c7dd6fc 100644 --- a/source/libs/sync/inc/sync_raft_progress_tracker.h +++ b/source/libs/sync/inc/sync_raft_progress_tracker.h @@ -23,6 +23,7 @@ #include "sync_raft_proto.h" #include "thash.h" +// Config reflects the configuration tracked in a ProgressTracker. struct SSyncRaftProgressTrackerConfig { SSyncRaftQuorumJointConfig voters; @@ -99,27 +100,32 @@ void syncRaftFreeTrackConfig(SSyncRaftProgressTrackerConfig* config); void syncRaftFreeTrackConfig(SSyncRaftProgressTrackerConfig* config); +// ResetVotes prepares for a new round of vote counting via recordVote. void syncRaftResetVotes(SSyncRaftProgressTracker*); void syncRaftProgressVisit(SSyncRaftProgressTracker*, visitProgressFp visit, void* arg); -/** - * syncRaftRecordVote records that the node with the given id voted for this Raft - * instance if v == true (and declined it otherwise). - **/ +// RecordVote records that the node with the given id voted for this Raft +// instance if v == true (and declined it otherwise). void syncRaftRecordVote(SSyncRaftProgressTracker* tracker, SyncNodeId id, bool grant); void syncRaftCopyTrackerConfig(const SSyncRaftProgressTrackerConfig* from, SSyncRaftProgressTrackerConfig* to); int syncRaftCheckTrackerConfigInProgress(SSyncRaftProgressTrackerConfig* config, SSyncRaftProgressMap* progressMap); -/** - * syncRaftTallyVotes returns the number of granted and rejected Votes, and whether the - * election outcome is known. - **/ +// TallyVotes returns the number of granted and rejected Votes, and whether the +// election outcome is known. ESyncRaftVoteResult syncRaftTallyVotes(SSyncRaftProgressTracker* tracker, int* rejected, int *granted); -void syncRaftConfigState(const SSyncRaftProgressTracker* tracker, SSyncConfigState* cs); +void syncRaftConfigState(SSyncRaftProgressTracker* tracker, SSyncConfigState* cs); + +// Committed returns the largest log index known to be committed based on what +// the voting members of the group have acknowledged. +SyncIndex syncRaftCommittedIndex(SSyncRaftProgressTracker* tracker); + +// QuorumActive returns true if the quorum is active from the view of the local +// raft state machine. Otherwise, it returns false. +bool syncRaftQuorumActive(SSyncRaftProgressTracker* tracker); bool syncRaftIsInNodeMap(const SSyncRaftNodeMap* nodeMap, SyncNodeId nodeId); diff --git a/source/libs/sync/inc/sync_raft_quorum_joint.h b/source/libs/sync/inc/sync_raft_quorum_joint.h index 92cddaaec1..9d5f10ab51 100644 --- a/source/libs/sync/inc/sync_raft_quorum_joint.h +++ b/source/libs/sync/inc/sync_raft_quorum_joint.h @@ -22,20 +22,25 @@ #include "sync_raft_node_map.h" #include "thash.h" -/** - * SSyncRaftQuorumJointConfig is a configuration of two groups of (possibly overlapping) - * majority configurations. Decisions require the support of both majorities. - **/ +// JointConfig is a configuration of two groups of (possibly overlapping) +// majority configurations. Decisions require the support of both majorities. typedef struct SSyncRaftQuorumJointConfig { SSyncRaftNodeMap outgoing; SSyncRaftNodeMap incoming; } SSyncRaftQuorumJointConfig; -/** - * syncRaftVoteResult takes a mapping of voters to yes/no (true/false) votes and returns - * a result indicating whether the vote is pending, lost, or won. A joint quorum - * requires both majority quorums to vote in favor. - **/ +// IDs returns a newly initialized map representing the set of voters present +// in the joint configuration. +void syncRaftJointConfigIDs(SSyncRaftQuorumJointConfig* config, SSyncRaftNodeMap* nodeMap); + +// CommittedIndex returns the largest committed index for the given joint +// quorum. An index is jointly committed if it is committed in both constituent +// majorities. +SyncIndex syncRaftJointConfigCommittedIndex(const SSyncRaftQuorumJointConfig* config, matchAckIndexerFp indexer, void* arg); + +// VoteResult takes a mapping of voters to yes/no (true/false) votes and returns +// a result indicating whether the vote is pending, lost, or won. A joint quorum +// requires both majority quorums to vote in favor. ESyncRaftVoteType syncRaftVoteResult(SSyncRaftQuorumJointConfig* config, SHashObj* votesMap); void syncRaftInitQuorumJointConfig(SSyncRaftQuorumJointConfig* config); @@ -76,6 +81,4 @@ static FORCE_INLINE bool syncRaftJointConfigIsInOutgoing(const SSyncRaftQuorumJo return syncRaftIsInNodeMap(&config->outgoing, id); } -void syncRaftJointConfigIDS(const SSyncRaftQuorumJointConfig* config, SSyncRaftNodeMap* nodeMap); - #endif /* _TD_LIBS_SYNC_RAFT_QUORUM_JOINT_H */ diff --git a/source/libs/sync/inc/sync_raft_quorum_majority.h b/source/libs/sync/inc/sync_raft_quorum_majority.h index 38df40147a..399bd71db8 100644 --- a/source/libs/sync/inc/sync_raft_quorum_majority.h +++ b/source/libs/sync/inc/sync_raft_quorum_majority.h @@ -29,4 +29,8 @@ **/ ESyncRaftVoteResult syncRaftMajorityVoteResult(SSyncRaftNodeMap* config, SHashObj* votesMap); +// CommittedIndex computes the committed index from those supplied via the +// provided AckedIndexer (for the active config). +SyncIndex syncRaftMajorityConfigCommittedIndex(const SSyncRaftNodeMap* config, matchAckIndexerFp indexer, void* arg); + #endif /* _TD_LIBS_SYNC_RAFT_QUORUM_MAJORITY_H */ diff --git a/source/libs/sync/inc/sync_type.h b/source/libs/sync/inc/sync_type.h index e00700d724..9c4bc9e63c 100644 --- a/source/libs/sync/inc/sync_type.h +++ b/source/libs/sync/inc/sync_type.h @@ -86,4 +86,6 @@ typedef enum { typedef void (*visitProgressFp)(SSyncRaftProgress* progress, void* arg); +typedef void (*matchAckIndexerFp)(SyncNodeId id, void* arg, SyncIndex* index); + #endif /* _TD_LIBS_SYNC_TYPE_H */ diff --git a/source/libs/sync/src/sync_raft_election.c b/source/libs/sync/src/sync_raft_election.c index 6d36d38267..d961978be2 100644 --- a/source/libs/sync/src/sync_raft_election.c +++ b/source/libs/sync/src/sync_raft_election.c @@ -84,7 +84,7 @@ static void campaign(SSyncRaft* pRaft, ESyncRaftElectionType cType) { SyncIndex lastIndex = syncRaftLogLastIndex(pRaft->log); SyncTerm lastTerm = syncRaftLogLastTerm(pRaft->log); SSyncRaftNodeMap nodeMap; - syncRaftJointConfigIDS(&pRaft->tracker->config.voters, &nodeMap); + syncRaftJointConfigIDs(&pRaft->tracker->config.voters, &nodeMap); SyncNodeId *pNodeId = NULL; while (!syncRaftIterateNodeMap(&nodeMap, pNodeId)) { SyncNodeId nodeId = *pNodeId; diff --git a/source/libs/sync/src/sync_raft_inflights.c b/source/libs/sync/src/sync_raft_inflights.c index 3d740b5a9e..7b97aca014 100644 --- a/source/libs/sync/src/sync_raft_inflights.c +++ b/source/libs/sync/src/sync_raft_inflights.c @@ -40,19 +40,16 @@ void syncRaftCloseInflights(SSyncRaftInflights* inflights) { free(inflights); } -/** - * syncRaftInflightAdd notifies the Inflights that a new message with the given index is being - * dispatched. syncRaftInflightFull() must be called prior to syncRaftInflightAdd() - * to verify that there is room for one more message, - * and consecutive calls to add syncRaftInflightAdd() must provide a - * monotonic sequence of indexes. - **/ +// Add notifies the Inflights that a new message with the given index is being +// dispatched. Full() must be called prior to Add() to verify that there is room +// for one more message, and consecutive calls to add Add() must provide a +// monotonic sequence of indexes. void syncRaftInflightAdd(SSyncRaftInflights* inflights, SyncIndex inflightIndex) { assert(!syncRaftInflightFull(inflights)); int next = inflights->start + inflights->count; int size = inflights->size; - /* is next wrapped around buffer? */ + if (next >= size) { next -= size; } @@ -61,12 +58,10 @@ void syncRaftInflightAdd(SSyncRaftInflights* inflights, SyncIndex inflightIndex) inflights->count++; } -/** - * syncRaftInflightFreeLE frees the inflights smaller or equal to the given `to` flight. - **/ +// FreeLE frees the inflights smaller or equal to the given `to` flight. void syncRaftInflightFreeLE(SSyncRaftInflights* inflights, SyncIndex toIndex) { if (inflights->count == 0 || toIndex < inflights->buffer[inflights->start]) { - /* out of the left side of the window */ + // out of the left side of the window return; } @@ -95,10 +90,8 @@ void syncRaftInflightFreeLE(SSyncRaftInflights* inflights, SyncIndex toIndex) { } } -/** - * syncRaftInflightFreeFirstOne releases the first inflight. - * This is a no-op if nothing is inflight. - **/ +// FreeFirstOne releases the first inflight. This is a no-op if nothing is +// inflight. void syncRaftInflightFreeFirstOne(SSyncRaftInflights* inflights) { syncRaftInflightFreeLE(inflights, inflights->buffer[inflights->start]); } diff --git a/source/libs/sync/src/sync_raft_node_map.c b/source/libs/sync/src/sync_raft_node_map.c index 1c54d32b59..642eebe65b 100644 --- a/source/libs/sync/src/sync_raft_node_map.c +++ b/source/libs/sync/src/sync_raft_node_map.c @@ -37,11 +37,10 @@ bool syncRaftIsInNodeMap(const SSyncRaftNodeMap* nodeMap, SyncNodeId nodeId) { return true; } -void syncRaftCopyNodeMap(const SSyncRaftNodeMap* from, SSyncRaftNodeMap* to) { - SyncNodeId** ppId = (SyncNodeId**)taosHashIterate(from->nodeIdMap, NULL); - while (ppId) { - taosHashPut(to->nodeIdMap, ppId, sizeof(SyncNodeId*), ppId, sizeof(SyncNodeId*)); - ppId = taosHashIterate(from->nodeIdMap, ppId); +void syncRaftCopyNodeMap(SSyncRaftNodeMap* from, SSyncRaftNodeMap* to) { + SyncNodeId *pId = NULL; + while (!syncRaftIterateNodeMap(from, pId)) { + taosHashPut(to->nodeIdMap, &pId, sizeof(SyncNodeId*), &pId, sizeof(SyncNodeId*)); } } @@ -66,7 +65,7 @@ bool syncRaftIsAllNodeInProgressMap(SSyncRaftNodeMap* nodeMap, SSyncRaftProgress return true; } -void syncRaftUnionNodeMap(const SSyncRaftNodeMap* nodeMap, SSyncRaftNodeMap* to) { +void syncRaftUnionNodeMap(SSyncRaftNodeMap* nodeMap, SSyncRaftNodeMap* to) { syncRaftCopyNodeMap(nodeMap, to); } diff --git a/source/libs/sync/src/sync_raft_progress.c b/source/libs/sync/src/sync_raft_progress.c index 65676655ec..a3ab93c0fc 100644 --- a/source/libs/sync/src/sync_raft_progress.c +++ b/source/libs/sync/src/sync_raft_progress.c @@ -47,11 +47,9 @@ void syncRaftResetProgress(SSyncRaft* pRaft, SSyncRaftProgress* progress) { }; } -/** - * syncRaftProgressMaybeUpdate is called when an MsgAppResp arrives from the follower, with the - * index acked by it. The method returns false if the given n index comes from - * an outdated message. Otherwise it updates the progress and returns true. - **/ +// MaybeUpdate is called when an MsgAppResp arrives from the follower, with the +// index acked by it. The method returns false if the given n index comes from +// an outdated message. Otherwise it updates the progress and returns true. bool syncRaftProgressMaybeUpdate(SSyncRaftProgress* progress, SyncIndex lastIndex) { bool updated = false; @@ -66,27 +64,36 @@ bool syncRaftProgressMaybeUpdate(SSyncRaftProgress* progress, SyncIndex lastInde return updated; } +// MaybeDecrTo adjusts the Progress to the receipt of a MsgApp rejection. The +// arguments are the index of the append message rejected by the follower, and +// the hint that we want to decrease to. +// +// Rejections can happen spuriously as messages are sent out of order or +// duplicated. In such cases, the rejection pertains to an index that the +// Progress already knows were previously acknowledged, and false is returned +// without changing the Progress. +// +// If the rejection is genuine, Next is lowered sensibly, and the Progress is +// cleared for sending log entries. bool syncRaftProgressMaybeDecrTo(SSyncRaftProgress* progress, SyncIndex rejected, SyncIndex matchHint) { if (progress->state == PROGRESS_STATE_REPLICATE) { - /** - * the rejection must be stale if the progress has matched and "rejected" - * is smaller than "match". - **/ + // The rejection must be stale if the progress has matched and "rejected" + // is smaller than "match". if (rejected <= progress->matchIndex) { syncDebug("match index is up to date,ignore"); return false; } - /* directly decrease next to match + 1 */ + // Directly decrease next to match + 1. + // + // TODO(tbg): why not use matchHint if it's larger? progress->nextIndex = progress->matchIndex + 1; return true; } - /** - * The rejection must be stale if "rejected" does not match next - 1. This - * is because non-replicating followers are probed one entry at a time. - **/ + // The rejection must be stale if "rejected" does not match next - 1. This + // is because non-replicating followers are probed one entry at a time. if (rejected != progress->nextIndex - 1) { syncDebug("rejected index %" PRId64 " different from next index %" PRId64 " -> ignore" , rejected, progress->nextIndex); @@ -99,14 +106,12 @@ bool syncRaftProgressMaybeDecrTo(SSyncRaftProgress* progress, return true; } -/** - * syncRaftProgressIsPaused returns whether sending log entries to this node has been throttled. - * This is done when a node has rejected recent MsgApps, is currently waiting - * for a snapshot, or has reached the MaxInflightMsgs limit. In normal - * operation, this is false. A throttled node will be contacted less frequently - * until it has reached a state in which it's able to accept a steady stream of - * log entries again. - **/ +// IsPaused returns whether sending log entries to this node has been throttled. +// This is done when a node has rejected recent MsgApps, is currently waiting +// for a snapshot, or has reached the MaxInflightMsgs limit. In normal +// operation, this is false. A throttled node will be contacted less frequently +// until it has reached a state in which it's able to accept a steady stream of +// log entries again. bool syncRaftProgressIsPaused(SSyncRaftProgress* progress) { switch (progress->state) { case PROGRESS_STATE_PROBE: @@ -152,16 +157,12 @@ bool syncRaftProgressIsUptodate(SSyncRaft* pRaft, SSyncRaftProgress* progress) { return syncRaftLogLastIndex(pRaft->log) + 1 == progress->nextIndex; } -/** - * syncRaftProgressBecomeProbe transitions into StateProbe. Next is reset to Match+1 or, - * optionally and if larger, the index of the pending snapshot. - **/ +// BecomeProbe transitions into StateProbe. Next is reset to Match+1 or, +// optionally and if larger, the index of the pending snapshot. void syncRaftProgressBecomeProbe(SSyncRaftProgress* progress) { - /** - * If the original state is ProgressStateSnapshot, progress knows that - * the pending snapshot has been sent to this peer successfully, then - * probes from pendingSnapshot + 1. - **/ + // If the original state is StateSnapshot, progress knows that + // the pending snapshot has been sent to this peer successfully, then + // probes from pendingSnapshot + 1. if (progress->state == PROGRESS_STATE_SNAPSHOT) { SyncIndex pendingSnapshotIndex = progress->pendingSnapshotIndex; resetProgressState(progress, PROGRESS_STATE_PROBE); @@ -172,14 +173,14 @@ void syncRaftProgressBecomeProbe(SSyncRaftProgress* progress) { } } -/** - * syncRaftProgressBecomeReplicate transitions into StateReplicate, resetting Next to Match+1. - **/ +// BecomeReplicate transitions into StateReplicate, resetting Next to Match+1. void syncRaftProgressBecomeReplicate(SSyncRaftProgress* progress) { resetProgressState(progress, PROGRESS_STATE_REPLICATE); progress->nextIndex = progress->matchIndex + 1; } +// BecomeSnapshot moves the Progress to StateSnapshot with the specified pending +// snapshot index. void syncRaftProgressBecomeSnapshot(SSyncRaftProgress* progress, SyncIndex snapshotIndex) { resetProgressState(progress, PROGRESS_STATE_SNAPSHOT); progress->pendingSnapshotIndex = snapshotIndex; @@ -242,10 +243,8 @@ static void unrefProgress(SSyncRaftProgress* progress, void* arg) { } } -/** - * ResetState moves the Progress into the specified State, resetting ProbeSent, - * PendingSnapshot, and Inflights. - **/ +// ResetState moves the Progress into the specified State, resetting ProbeSent, +// PendingSnapshot, and Inflights. static void resetProgressState(SSyncRaftProgress* progress, ESyncRaftProgressState state) { progress->probeSent = false; progress->pendingSnapshotIndex = 0; @@ -253,83 +252,9 @@ static void resetProgressState(SSyncRaftProgress* progress, ESyncRaftProgressSta syncRaftInflightReset(progress->inflights); } -/** - * probeAcked is called when this peer has accepted an append. It resets - * ProbeSent to signal that additional append messages should be sent without - * further delay. - **/ +// ProbeAcked is called when this peer has accepted an append. It resets +// ProbeSent to signal that additional append messages should be sent without +// further delay. static void probeAcked(SSyncRaftProgress* progress) { progress->probeSent = false; } - -#if 0 - -SyncIndex syncRaftProgressNextIndex(SSyncRaft* pRaft, int i) { - return pRaft->leaderState.progress[i].nextIndex; -} - -SyncIndex syncRaftProgressMatchIndex(SSyncRaft* pRaft, int i) { - return pRaft->leaderState.progress[i].matchIndex; -} - -void syncRaftProgressUpdateLastSend(SSyncRaft* pRaft, int i) { - pRaft->leaderState.progress[i].lastSend = pRaft->io.time(pRaft); -} - -void syncRaftProgressUpdateSnapshotLastSend(SSyncRaft* pRaft, int i) { - pRaft->leaderState.progress[i].lastSendSnapshot = pRaft->io.time(pRaft); -} - -bool syncRaftProgressResetRecentRecv(SSyncRaft* pRaft, int i) { - SSyncRaftProgress* progress = &(pRaft->leaderState.progress[i]); - bool prev = progress->recentRecv; - progress->recentRecv = false; - return prev; -} - -void syncRaftProgressMarkRecentRecv(SSyncRaft* pRaft, int i) { - pRaft->leaderState.progress[i].recentRecv = true; -} - -bool syncRaftProgressGetRecentRecv(SSyncRaft* pRaft, int i) { - return pRaft->leaderState.progress[i].recentRecv; -} - -void syncRaftProgressBecomeSnapshot(SSyncRaft* pRaft, int i) { - SSyncRaftProgress* progress = &(pRaft->leaderState.progress[i]); - resetProgressState(progress, PROGRESS_STATE_SNAPSHOT); - progress->pendingSnapshotIndex = raftLogSnapshotIndex(pRaft->log); -} - -void syncRaftProgressBecomeProbe(SSyncRaft* pRaft, int i) { - SSyncRaftProgress* progress = &(pRaft->leaderState.progress[i]); - - if (progress->state == PROGRESS_STATE_SNAPSHOT) { - assert(progress->pendingSnapshotIndex > 0); - SyncIndex pendingSnapshotIndex = progress->pendingSnapshotIndex; - resetProgressState(progress, PROGRESS_STATE_PROBE); - progress->nextIndex = max(progress->matchIndex + 1, pendingSnapshotIndex); - } else { - resetProgressState(progress, PROGRESS_STATE_PROBE); - progress->nextIndex = progress->matchIndex + 1; - } -} - -void syncRaftProgressBecomeReplicate(SSyncRaft* pRaft, int i) { - resetProgressState(pRaft->leaderState.progress, PROGRESS_STATE_REPLICATE); - pRaft->leaderState.progress->nextIndex = pRaft->leaderState.progress->matchIndex + 1; -} - -void syncRaftProgressAbortSnapshot(SSyncRaft* pRaft, int i) { - SSyncRaftProgress* progress = &(pRaft->leaderState.progress[i]); - progress->pendingSnapshotIndex = 0; - progress->state = PROGRESS_STATE_PROBE; -} - -ESyncRaftProgressState syncRaftProgressState(SSyncRaft* pRaft, int i) { - return pRaft->leaderState.progress[i].state; -} - - - -#endif \ No newline at end of file diff --git a/source/libs/sync/src/sync_raft_progress_tracker.c b/source/libs/sync/src/sync_raft_progress_tracker.c index 60e3ccea6a..e0b4afae21 100644 --- a/source/libs/sync/src/sync_raft_progress_tracker.c +++ b/source/libs/sync/src/sync_raft_progress_tracker.c @@ -13,6 +13,8 @@ * along with this program. If not, see . */ +#include "raft.h" +#include "sync_const.h" #include "sync_raft_progress_tracker.h" #include "sync_raft_proto.h" @@ -22,9 +24,11 @@ SSyncRaftProgressTracker* syncRaftOpenProgressTracker(SSyncRaft* pRaft) { return NULL; } + tracker->votesMap = taosHashInit(TSDB_MAX_REPLICA, taosGetDefaultHashFunction(TSDB_DATA_TYPE_INT), true, HASH_ENTRY_LOCK); + syncRaftInitTrackConfig(&tracker->config); - syncRaftInitNodeMap(&tracker->config.learnersNext); tracker->pRaft = pRaft; + tracker->maxInflightMsgs = kSyncRaftMaxInflghtMsgs; return tracker; } @@ -39,9 +43,11 @@ void syncRaftInitTrackConfig(SSyncRaftProgressTrackerConfig* config) { void syncRaftFreeTrackConfig(SSyncRaftProgressTrackerConfig* config) { syncRaftFreeNodeMap(&config->learners); syncRaftFreeNodeMap(&config->learnersNext); - syncRaftFreeQuorumJointConfig(&config->voters); + syncRaftFreeNodeMap(&config->voters.incoming); + syncRaftFreeNodeMap(&config->voters.outgoing); } +// ResetVotes prepares for a new round of vote counting via recordVote. void syncRaftResetVotes(SSyncRaftProgressTracker* tracker) { taosHashClear(tracker->votesMap); } @@ -50,14 +56,15 @@ void syncRaftProgressVisit(SSyncRaftProgressTracker* tracker, visitProgressFp vi syncRaftVisitProgressMap(&tracker->progressMap, visit, arg); } +// RecordVote records that the node with the given id voted for this Raft +// instance if v == true (and declined it otherwise). void syncRaftRecordVote(SSyncRaftProgressTracker* tracker, SyncNodeId id, bool grant) { ESyncRaftVoteType* pType = taosHashGet(tracker->votesMap, &id, sizeof(SyncNodeId*)); if (pType != NULL) { return; } - ESyncRaftVoteType type = grant ? SYNC_RAFT_VOTE_RESP_GRANT : SYNC_RAFT_VOTE_RESP_REJECT; - taosHashPut(tracker->votesMap, &id, sizeof(SyncNodeId), &type, sizeof(ESyncRaftVoteType*)); + taosHashPut(tracker->votesMap, &id, sizeof(SyncNodeId), &grant, sizeof(bool*)); } void syncRaftCopyTrackerConfig(const SSyncRaftProgressTrackerConfig* from, SSyncRaftProgressTrackerConfig* to) { @@ -78,26 +85,27 @@ int syncRaftCheckTrackerConfigInProgress(SSyncRaftProgressTrackerConfig* config, return 0; } -/** - * syncRaftTallyVotes returns the number of granted and rejected Votes, and whether the - * election outcome is known. - **/ +// TallyVotes returns the number of granted and rejected Votes, and whether the +// election outcome is known. ESyncRaftVoteResult syncRaftTallyVotes(SSyncRaftProgressTracker* tracker, int* rejected, int *granted) { - int i; - SSyncRaftProgress* progress; + SSyncRaftProgress* progress = NULL; int r, g; + // Make sure to populate granted/rejected correctly even if the Votes slice + // contains members no longer part of the configuration. This doesn't really + // matter in the way the numbers are used (they're informational), but might + // as well get it right. while (!syncRaftIterateProgressMap(&tracker->progressMap, progress)) { if (progress->id == SYNC_NON_NODE_ID) { continue; } - ESyncRaftVoteType* pType = taosHashGet(tracker->votesMap, &progress->id, sizeof(SyncNodeId*)); - if (pType == NULL) { + bool* v = taosHashGet(tracker->votesMap, &progress->id, sizeof(SyncNodeId*)); + if (v == NULL) { continue; } - if (*pType == SYNC_RAFT_VOTE_RESP_GRANT) { + if (*v) { g++; } else { r++; @@ -109,9 +117,40 @@ ESyncRaftVoteResult syncRaftTallyVotes(SSyncRaftProgressTracker* tracker, int* r return syncRaftVoteResult(&(tracker->config.voters), tracker->votesMap); } -void syncRaftConfigState(const SSyncRaftProgressTracker* tracker, SSyncConfigState* cs) { +void syncRaftConfigState(SSyncRaftProgressTracker* tracker, SSyncConfigState* cs) { syncRaftCopyNodeMap(&tracker->config.voters.incoming, &cs->voters); syncRaftCopyNodeMap(&tracker->config.voters.outgoing, &cs->votersOutgoing); syncRaftCopyNodeMap(&tracker->config.learners, &cs->learners); syncRaftCopyNodeMap(&tracker->config.learnersNext, &cs->learnersNext); + cs->autoLeave = tracker->config.autoLeave; +} + +static void matchAckIndexer(SyncNodeId id, void* arg, SyncIndex* index) { + SSyncRaftProgressTracker* tracker = (SSyncRaftProgressTracker*)arg; + SSyncRaftProgress* progress = syncRaftFindProgressByNodeId(&tracker->progressMap, id); + if (progress == NULL) { + *index = 0; + return; + } + *index = progress->matchIndex; +} + +// Committed returns the largest log index known to be committed based on what +// the voting members of the group have acknowledged. +SyncIndex syncRaftCommittedIndex(SSyncRaftProgressTracker* tracker) { + return syncRaftJointConfigCommittedIndex(&tracker->config.voters, matchAckIndexer, tracker); +} + +static void visitProgressActive(SSyncRaftProgress* progress, void* arg) { + SHashObj* votesMap = (SHashObj*)arg; + taosHashPut(votesMap, &progress->id, sizeof(SyncNodeId), &progress->recentActive, sizeof(bool)); +} + +// QuorumActive returns true if the quorum is active from the view of the local +// raft state machine. Otherwise, it returns false. +bool syncRaftQuorumActive(SSyncRaftProgressTracker* tracker) { + SHashObj* votesMap = taosHashInit(TSDB_MAX_REPLICA, taosGetDefaultHashFunction(TSDB_DATA_TYPE_INT), true, HASH_ENTRY_LOCK); + syncRaftVisitProgressMap(&tracker->progressMap, visitProgressActive, votesMap); + + return syncRaftVoteResult(&tracker->config.voters, votesMap) == SYNC_RAFT_VOTE_WON; } \ No newline at end of file diff --git a/source/libs/sync/src/sync_raft_quorum_joint.c b/source/libs/sync/src/sync_raft_quorum_joint.c index 500bd908c0..70c078b6f5 100644 --- a/source/libs/sync/src/sync_raft_quorum_joint.c +++ b/source/libs/sync/src/sync_raft_quorum_joint.c @@ -59,8 +59,17 @@ void syncRaftJointConfigRemoveFromIncoming(SSyncRaftQuorumJointConfig* config, S syncRaftRemoveFromNodeMap(&config->incoming, id); } -void syncRaftJointConfigIDS(const SSyncRaftQuorumJointConfig* config, SSyncRaftNodeMap* nodeMap) { +void syncRaftJointConfigIDs(SSyncRaftQuorumJointConfig* config, SSyncRaftNodeMap* nodeMap) { syncRaftCopyNodeMap(&config->incoming, nodeMap); syncRaftUnionNodeMap(&config->outgoing, nodeMap); +} + +SyncIndex syncRaftJointConfigCommittedIndex(const SSyncRaftQuorumJointConfig* config, matchAckIndexerFp indexer, void* arg) { + SyncIndex index0, index1; + + index0 = syncRaftMajorityConfigCommittedIndex(&config->incoming, indexer, arg); + index1 = syncRaftMajorityConfigCommittedIndex(&config->outgoing, indexer, arg); + + return index0 < index1 ? index0 : index1; } \ No newline at end of file diff --git a/source/libs/sync/src/sync_raft_quorum_majority.c b/source/libs/sync/src/sync_raft_quorum_majority.c index ff5ba64876..313f213cda 100644 --- a/source/libs/sync/src/sync_raft_quorum_majority.c +++ b/source/libs/sync/src/sync_raft_quorum_majority.c @@ -13,6 +13,7 @@ * along with this program. If not, see . */ +#include "sync_const.h" #include "sync_raft_quorum.h" #include "sync_raft_quorum_majority.h" #include "sync_raft_node_map.h" @@ -34,13 +35,13 @@ ESyncRaftVoteResult syncRaftMajorityVoteResult(SSyncRaftNodeMap* config, SHashOb i = g = r = missing = 0; SyncNodeId* pId = NULL; while (!syncRaftIterateNodeMap(config, pId)) { - const ESyncRaftVoteType* pType = taosHashGet(votesMap, pId, sizeof(SyncNodeId*)); - if (pType == NULL) { + const bool* v = (const bool*)taosHashGet(votesMap, pId, sizeof(SyncNodeId*)); + if (v == NULL) { missing += 1; continue; } - if (*pType == SYNC_RAFT_VOTE_RESP_GRANT) { + if (*v) { g +=1; } else { r += 1; @@ -56,4 +57,65 @@ ESyncRaftVoteResult syncRaftMajorityVoteResult(SSyncRaftNodeMap* config, SHashOb } return SYNC_RAFT_VOTE_LOST; +} + +int compSyncIndex(const void * elem1, const void * elem2) { + SyncIndex index1 = *((SyncIndex*)elem1); + SyncIndex index2 = *((SyncIndex*)elem1); + if (index1 > index2) return 1; + if (index1 < index2) return -1; + return 0; +} + +SyncIndex syncRaftMajorityConfigCommittedIndex(const SSyncRaftNodeMap* config, matchAckIndexerFp indexer, void* arg) { + int n = syncRaftNodeMapSize(config); + if (n == 0) { + // This plays well with joint quorums which, when one half is the zero + // MajorityConfig, should behave like the other half. + return kMaxCommitIndex; + } + + // Use an on-stack slice to collect the committed indexes when n <= 7 + // (otherwise we alloc). The alternative is to stash a slice on + // MajorityConfig, but this impairs usability (as is, MajorityConfig is just + // a map, and that's nice). The assumption is that running with a + // replication factor of >7 is rare, and in cases in which it happens + // performance is a lesser concern (additionally the performance + // implications of an allocation here are far from drastic). + SyncIndex* srt = NULL; + SyncIndex srk[TSDB_MAX_REPLICA]; + if (n > TSDB_MAX_REPLICA) { + srt = (SyncIndex*)malloc(sizeof(SyncIndex) * n); + if (srt == NULL) { + return kMaxCommitIndex; + } + } else { + srt = &srk[0]; + } + + // Fill the slice with the indexes observed. Any unused slots will be + // left as zero; these correspond to voters that may report in, but + // haven't yet. We fill from the right (since the zeroes will end up on + // the left after sorting below anyway). + SyncNodeId *pId = NULL; + int i = 0; + SyncIndex index; + while (!syncRaftIterateNodeMap(config, pId)) { + indexer(*pId, arg, &index); + srt[i++] = index; + } + + // Sort by index. Use a bespoke algorithm (copied from the stdlib's sort + // package) to keep srt on the stack. + qsort(srt, n, sizeof(SyncIndex), compSyncIndex); + + // The smallest index into the array for which the value is acked by a + // quorum. In other words, from the end of the slice, move n/2+1 to the + // left (accounting for zero-indexing). + index = srt[n - (n/2 + 1)]; + if (srt != &srk[0]) { + free(srt); + } + + return index; } \ No newline at end of file From 68e6b82a6692533838a95f20d3bfe1ec82746f70 Mon Sep 17 00:00:00 2001 From: lichuang Date: Fri, 19 Nov 2021 11:31:54 +0800 Subject: [PATCH 09/44] [TD-10645][raft]add vote resp process --- source/libs/sync/inc/raft_replication.h | 4 +- source/libs/sync/inc/sync_raft_impl.h | 2 + source/libs/sync/src/raft.c | 2 +- .../libs/sync/src/raft_handle_vote_message.c | 16 +++---- .../sync/src/raft_handle_vote_resp_message.c | 6 ++- source/libs/sync/src/raft_replication.c | 4 +- source/libs/sync/src/sync_raft_election.c | 12 +++--- source/libs/sync/src/sync_raft_impl.c | 42 +++++++++++++++---- 8 files changed, 59 insertions(+), 29 deletions(-) diff --git a/source/libs/sync/inc/raft_replication.h b/source/libs/sync/inc/raft_replication.h index d0e55ef10e..180a2db61f 100644 --- a/source/libs/sync/inc/raft_replication.h +++ b/source/libs/sync/inc/raft_replication.h @@ -20,11 +20,11 @@ #include "syncInt.h" #include "sync_type.h" -// syncRaftReplicate sends an append RPC with new entries to the given peer, +// syncRaftMaybeSendAppend sends an append RPC with new entries to the given peer, // if necessary. Returns true if a message was sent. The sendIfEmpty // argument controls whether messages with no entries will be sent // ("empty" messages are useful to convey updated Commit indexes, but // are undesirable when we're sending multiple messages in a batch). -bool syncRaftReplicate(SSyncRaft* pRaft, SSyncRaftProgress* progress, bool sendIfEmpty); +bool syncRaftMaybeSendAppend(SSyncRaft* pRaft, SSyncRaftProgress* progress, bool sendIfEmpty); #endif /* TD_SYNC_RAFT_REPLICATION_H */ diff --git a/source/libs/sync/inc/sync_raft_impl.h b/source/libs/sync/inc/sync_raft_impl.h index a8615f17eb..1a6c13f65f 100644 --- a/source/libs/sync/inc/sync_raft_impl.h +++ b/source/libs/sync/inc/sync_raft_impl.h @@ -28,6 +28,8 @@ void syncRaftBecomeLeader(SSyncRaft* pRaft); void syncRaftStartElection(SSyncRaft* pRaft, ESyncRaftElectionType cType); +void syncRaftCampaign(SSyncRaft* pRaft, ESyncRaftElectionType cType); + void syncRaftTriggerHeartbeat(SSyncRaft* pRaft); void syncRaftRandomizedElectionTimeout(SSyncRaft* pRaft); diff --git a/source/libs/sync/src/raft.c b/source/libs/sync/src/raft.c index 3b4c9e5f36..72b0d268a8 100644 --- a/source/libs/sync/src/raft.c +++ b/source/libs/sync/src/raft.c @@ -169,7 +169,7 @@ static int deserializeClusterStateFromBuffer(SSyncConfigState* cluster, const ch } static void visitProgressMaybeSendAppend(SSyncRaftProgress* progress, void* arg) { - syncRaftReplicate(arg, progress, false); + syncRaftMaybeSendAppend(arg, progress, false); } // switchToConfig reconfigures this node to use the provided configuration. It diff --git a/source/libs/sync/src/raft_handle_vote_message.c b/source/libs/sync/src/raft_handle_vote_message.c index 9997c5226d..0219e39df9 100644 --- a/source/libs/sync/src/raft_handle_vote_message.c +++ b/source/libs/sync/src/raft_handle_vote_message.c @@ -48,12 +48,14 @@ int syncRaftHandleVoteMessage(SSyncRaft* pRaft, const SSyncMessage* pMsg) { } static bool canGrantVoteMessage(SSyncRaft* pRaft, const SSyncMessage* pMsg) { - if (!(pRaft->voteFor == SYNC_NON_NODE_ID || pMsg->term > pRaft->term || pRaft->voteFor == pMsg->from)) { - return false; - } - if (!syncRaftLogIsUptodate(pRaft->log, pMsg->vote.lastIndex, pMsg->vote.lastTerm)) { - return false; - } + bool canVote = + // We can vote if this is a repeat of a vote we've already cast... + pRaft->voteFor == pMsg->from || + // ...we haven't voted and we don't think there's a leader yet in this term... + (pRaft->voteFor == SYNC_NON_NODE_ID && pRaft->leaderId == SYNC_NON_NODE_ID) || + // ...or this is a PreVote for a future term... + (pMsg->vote.cType == SYNC_RAFT_CAMPAIGN_PRE_ELECTION && pMsg->term > pRaft->term); - return true; + // ...and we believe the candidate is up to date. + return canVote && syncRaftLogIsUptodate(pRaft->log, pMsg->vote.lastIndex, pMsg->vote.lastTerm); } \ No newline at end of file diff --git a/source/libs/sync/src/raft_handle_vote_resp_message.c b/source/libs/sync/src/raft_handle_vote_resp_message.c index 744d654cc5..87a5cfcd15 100644 --- a/source/libs/sync/src/raft_handle_vote_resp_message.c +++ b/source/libs/sync/src/raft_handle_vote_resp_message.c @@ -45,12 +45,14 @@ int syncRaftHandleVoteRespMessage(SSyncRaft* pRaft, const SSyncMessage* pMsg) { if (result == SYNC_RAFT_VOTE_WON) { if (pRaft->candidateState.inPreVote) { - syncRaftStartElection(pRaft, SYNC_RAFT_CAMPAIGN_ELECTION); + syncRaftCampaign(pRaft, SYNC_RAFT_CAMPAIGN_ELECTION); } else { syncRaftBecomeLeader(pRaft); - + syncRaftBroadcastAppend(pRaft); } } else if (result == SYNC_RAFT_VOTE_LOST) { + // pb.MsgPreVoteResp contains future term of pre-candidate + // m.Term > r.Term; reuse r.Term syncRaftBecomeFollower(pRaft, pRaft->term, SYNC_NON_NODE_ID); } diff --git a/source/libs/sync/src/raft_replication.c b/source/libs/sync/src/raft_replication.c index 228d8195f6..c8c2d2c379 100644 --- a/source/libs/sync/src/raft_replication.c +++ b/source/libs/sync/src/raft_replication.c @@ -24,12 +24,12 @@ static bool sendAppendEntries(SSyncRaft* pRaft, SSyncRaftProgress* progress, SyncIndex prevIndex, SyncTerm prevTerm, SSyncRaftEntry *entries, int nEntry); -// syncRaftReplicate sends an append RPC with new entries to the given peer, +// maybeSendAppend sends an append RPC with new entries to the given peer, // if necessary. Returns true if a message was sent. The sendIfEmpty // argument controls whether messages with no entries will be sent // ("empty" messages are useful to convey updated Commit indexes, but // are undesirable when we're sending multiple messages in a batch). -bool syncRaftReplicate(SSyncRaft* pRaft, SSyncRaftProgress* progress, bool sendIfEmpty) { +bool syncRaftMaybeSendAppend(SSyncRaft* pRaft, SSyncRaftProgress* progress, bool sendIfEmpty) { assert(pRaft->state == TAOS_SYNC_STATE_LEADER); SyncNodeId nodeId = progress->id; diff --git a/source/libs/sync/src/sync_raft_election.c b/source/libs/sync/src/sync_raft_election.c index d961978be2..fe2e0fd9d3 100644 --- a/source/libs/sync/src/sync_raft_election.c +++ b/source/libs/sync/src/sync_raft_election.c @@ -19,8 +19,6 @@ #include "raft_message.h" #include "sync_raft_progress_tracker.h" -static void campaign(SSyncRaft* pRaft, ESyncRaftElectionType cType); - void syncRaftStartElection(SSyncRaft* pRaft, ESyncRaftElectionType cType) { if (pRaft->state == TAOS_SYNC_STATE_LEADER) { syncDebug("[%d:%d] ignoring RAFT_MSG_INTERNAL_ELECTION because already leader", pRaft->selfGroupId, pRaft->selfId); @@ -28,7 +26,7 @@ void syncRaftStartElection(SSyncRaft* pRaft, ESyncRaftElectionType cType) { } if (!syncRaftIsPromotable(pRaft)) { - syncWarn("[%d:%d] is unpromotable and can not campaign", pRaft->selfGroupId, pRaft->selfId); + syncWarn("[%d:%d] is unpromotable and can not syncRaftCampaign", pRaft->selfGroupId, pRaft->selfId); return; } @@ -41,17 +39,17 @@ void syncRaftStartElection(SSyncRaft* pRaft, ESyncRaftElectionType cType) { syncInfo("[%d:%d] is starting a new election at term %" PRId64 "", pRaft->selfGroupId, pRaft->selfId, pRaft->term); - campaign(pRaft, cType); + syncRaftCampaign(pRaft, cType); } -// campaign transitions the raft instance to candidate state. This must only be +// syncRaftCampaign transitions the raft instance to candidate state. This must only be // called after verifying that this is a legitimate transition. -static void campaign(SSyncRaft* pRaft, ESyncRaftElectionType cType) { +void syncRaftCampaign(SSyncRaft* pRaft, ESyncRaftElectionType cType) { bool preVote; SyncTerm term; if (syncRaftIsPromotable(pRaft)) { - syncDebug("[%d:%d] is unpromotable; campaign() should have been called", pRaft->selfGroupId, pRaft->selfId); + syncDebug("[%d:%d] is unpromotable; syncRaftCampaign() should have been called", pRaft->selfGroupId, pRaft->selfId); return; } diff --git a/source/libs/sync/src/sync_raft_impl.c b/source/libs/sync/src/sync_raft_impl.c index 2093bcb046..4d8222e826 100644 --- a/source/libs/sync/src/sync_raft_impl.c +++ b/source/libs/sync/src/sync_raft_impl.c @@ -25,6 +25,8 @@ static int stepFollower(SSyncRaft* pRaft, const SSyncMessage* pMsg); static int stepCandidate(SSyncRaft* pRaft, const SSyncMessage* pMsg); static int stepLeader(SSyncRaft* pRaft, const SSyncMessage* pMsg); +static bool increaseUncommittedSize(SSyncRaft* pRaft, SSyncRaftEntry* entries, int n); + static int triggerAll(SSyncRaft* pRaft); static void tickElection(SSyncRaft* pRaft); @@ -82,13 +84,22 @@ void syncRaftBecomeLeader(SSyncRaft* pRaft) { resetRaft(pRaft, pRaft->term); pRaft->leaderId = pRaft->leaderId; pRaft->state = TAOS_SYNC_STATE_LEADER; - // TODO: check if there is pending config log - int nPendingConf = syncRaftLogNumOfPendingConf(pRaft->log); - if (nPendingConf > 1) { - syncFatal("unexpected multiple uncommitted config entry"); - } - syncInfo("[%d:%d] became leader at term %" PRId64 "", pRaft->selfGroupId, pRaft->selfId, pRaft->term); + SSyncRaftProgress* progress = syncRaftFindProgressByNodeId(&pRaft->tracker->progressMap, pRaft->selfId); + assert(progress != NULL); + // Followers enter replicate mode when they've been successfully probed + // (perhaps after having received a snapshot as a result). The leader is + // trivially in this state. Note that r.reset() has initialized this + // progress with the last index already. + syncRaftProgressBecomeReplicate(progress); + + // Conservatively set the pendingConfIndex to the last index in the + // log. There may or may not be a pending config change, but it's + // safe to delay any future proposals until we commit all our + // pending log entries, and scanning the entire tail of the log + // could be expensive. + SyncIndex lastIndex = syncRaftLogLastIndex(pRaft->log); + pRaft->pendingConfigIndex = lastIndex; // after become leader, send a no-op log SSyncRaftEntry* entry = (SSyncRaftEntry*)malloc(sizeof(SSyncRaftEntry)); @@ -103,6 +114,7 @@ void syncRaftBecomeLeader(SSyncRaft* pRaft) { }; appendEntries(pRaft, entry, 1); //syncRaftTriggerHeartbeat(pRaft); + syncInfo("[%d:%d] became leader at term %" PRId64 "", pRaft->selfGroupId, pRaft->selfId, pRaft->term); } void syncRaftTriggerHeartbeat(SSyncRaft* pRaft) { @@ -192,9 +204,11 @@ static void visitProgressSendAppend(SSyncRaftProgress* progress, void* arg) { return; } - syncRaftReplicate(arg, progress, true); + syncRaftMaybeSendAppend(arg, progress, true); } +// bcastAppend sends RPC, with entries to all peers that are not up-to-date +// according to the progress recorded in r.prs. void syncRaftBroadcastAppend(SSyncRaft* pRaft) { syncRaftProgressVisit(pRaft->tracker, visitProgressSendAppend, pRaft); } @@ -267,6 +281,11 @@ static void tickHeartbeat(SSyncRaft* pRaft) { } +// TODO +static bool increaseUncommittedSize(SSyncRaft* pRaft, SSyncRaftEntry* entries, int n) { + return false; +} + static void appendEntries(SSyncRaft* pRaft, SSyncRaftEntry* entries, int n) { SyncIndex lastIndex = syncRaftLogLastIndex(pRaft->log); SyncTerm term = pRaft->term; @@ -277,9 +296,16 @@ static void appendEntries(SSyncRaft* pRaft, SSyncRaftEntry* entries, int n) { entries[i].index = lastIndex + 1 + i; } + // Track the size of this uncommitted proposal. + if (!increaseUncommittedSize(pRaft, entries, n)) { + // Drop the proposal. + return; + } + syncRaftLogAppend(pRaft->log, entries, n); SSyncRaftProgress* progress = syncRaftFindProgressByNodeId(&pRaft->tracker->progressMap, pRaft->selfId); + assert(progress != NULL); syncRaftProgressMaybeUpdate(progress, lastIndex); // Regardless of syncRaftMaybeCommit's return, our caller will call bcastAppend. syncRaftMaybeCommit(pRaft); @@ -306,7 +332,7 @@ static int triggerAll(SSyncRaft* pRaft) { continue; } - syncRaftReplicate(pRaft, pRaft->tracker->progressMap.progress[i], true); + syncRaftMaybeSendAppend(pRaft, pRaft->tracker->progressMap.progress[i], true); } #endif return 0; From 72ec932491d194f36aa892655fb9bf1f403c14b3 Mon Sep 17 00:00:00 2001 From: Shengliang Guan Date: Fri, 19 Nov 2021 14:46:06 +0800 Subject: [PATCH 10/44] refact transaction --- include/dnode/mnode/sdb/sdb.h | 24 +- source/dnode/mgmt/inc/dnodeInt.h | 2 +- source/dnode/mgmt/src/dnodeInt.c | 10 +- source/dnode/mgmt/src/dnodeMnode.c | 9 +- source/dnode/mgmt/src/dnodeTransport.c | 19 +- source/dnode/mnode/CMakeLists.txt | 1 - source/dnode/mnode/impl/CMakeLists.txt | 1 - source/dnode/mnode/impl/inc/mnodeDef.h | 22 + source/dnode/mnode/impl/inc/mnodeInt.h | 14 +- .../dnode/mnode/impl/inc/mnodeTrans.h | 22 +- source/dnode/mnode/impl/src/mnode.c | 22 +- source/dnode/mnode/impl/src/mnodeSync.c | 1 + source/dnode/mnode/impl/src/mnodeTrans.c | 502 ++++++++++++++++++ source/dnode/mnode/impl/src/mnodeUser.c | 9 +- source/dnode/mnode/sdb/src/sdbFile.c | 3 +- source/dnode/mnode/transaction/CMakeLists.txt | 15 - source/dnode/mnode/transaction/inc/trnInt.h | 70 --- source/dnode/mnode/transaction/src/trn.c | 233 -------- source/dnode/mnode/transaction/src/trnExec.c | 189 ------- source/libs/transport/src/rpcMain.c | 6 +- 20 files changed, 615 insertions(+), 559 deletions(-) rename include/dnode/mnode/transaction/trn.h => source/dnode/mnode/impl/inc/mnodeTrans.h (77%) create mode 100644 source/dnode/mnode/impl/src/mnodeTrans.c delete mode 100644 source/dnode/mnode/transaction/CMakeLists.txt delete mode 100644 source/dnode/mnode/transaction/inc/trnInt.h delete mode 100644 source/dnode/mnode/transaction/src/trn.c delete mode 100644 source/dnode/mnode/transaction/src/trnExec.c diff --git a/include/dnode/mnode/sdb/sdb.h b/include/dnode/mnode/sdb/sdb.h index 90c5ef0c4a..cb514cef53 100644 --- a/include/dnode/mnode/sdb/sdb.h +++ b/include/dnode/mnode/sdb/sdb.h @@ -56,39 +56,39 @@ extern "C" { dataPos += valLen; \ } -#define SDB_SET_INT64(pData, dataPos, val) \ +#define SDB_SET_INT64(pRaw, dataPos, val) \ { \ if (sdbSetRawInt64(pRaw, dataPos, val) != 0) { \ - sdbFreeRaw(pRaw); \ + sdbFreeRaw(pRaw); \ return NULL; \ - }; \ + } \ dataPos += sizeof(int64_t); \ } -#define SDB_SET_INT32(pData, dataPos, val) \ +#define SDB_SET_INT32(pRaw, dataPos, val) \ { \ if (sdbSetRawInt32(pRaw, dataPos, val) != 0) { \ - sdbFreeRaw(pRaw); \ + sdbFreeRaw(pRaw); \ return NULL; \ - }; \ + } \ dataPos += sizeof(int32_t); \ } -#define SDB_SET_INT8(pData, dataPos, val) \ +#define SDB_SET_INT8(pRaw, dataPos, val) \ { \ if (sdbSetRawInt8(pRaw, dataPos, val) != 0) { \ - sdbFreeRaw(pRaw); \ + sdbFreeRaw(pRaw); \ return NULL; \ - }; \ + } \ dataPos += sizeof(int8_t); \ } #define SDB_SET_BINARY(pRaw, dataPos, val, valLen) \ { \ if (sdbSetRawBinary(pRaw, dataPos, val, valLen) != 0) { \ - sdbFreeRaw(pRaw); \ + sdbFreeRaw(pRaw); \ return NULL; \ - }; \ + } \ dataPos += valLen; \ } @@ -97,7 +97,7 @@ extern "C" { if (sdbSetRawDataLen(pRaw, dataLen) != 0) { \ sdbFreeRaw(pRaw); \ return NULL; \ - }; \ + } \ } typedef struct SSdbRaw SSdbRaw; diff --git a/source/dnode/mgmt/inc/dnodeInt.h b/source/dnode/mgmt/inc/dnodeInt.h index 906455dce4..a4226a47b8 100644 --- a/source/dnode/mgmt/inc/dnodeInt.h +++ b/source/dnode/mgmt/inc/dnodeInt.h @@ -41,7 +41,7 @@ int32_t dnodeInit(); void dnodeCleanup(); EDnStat dnodeGetRunStat(); -void dnodeSetRunStat(); +void dnodeSetRunStat(EDnStat stat); void dnodeReportStartup(char *name, char *desc); void dnodeReportStartupFinished(char *name, char *desc); diff --git a/source/dnode/mgmt/src/dnodeInt.c b/source/dnode/mgmt/src/dnodeInt.c index 2674e107fd..eee4bac050 100644 --- a/source/dnode/mgmt/src/dnodeInt.c +++ b/source/dnode/mgmt/src/dnodeInt.c @@ -33,7 +33,10 @@ static struct { EDnStat dnodeGetRunStat() { return tsInt.runStat; } -void dnodeSetRunStat(EDnStat stat) { tsInt.runStat = stat; } +void dnodeSetRunStat(EDnStat stat) { + dDebug("runstat set to %d", stat); + tsInt.runStat = stat; +} void dnodeReportStartup(char *name, char *desc) { SStartupMsg *pStartup = &tsInt.startup; @@ -99,7 +102,6 @@ static int32_t dnodeInitDir() { } static int32_t dnodeInitMain() { - tsInt.runStat = DN_RUN_STAT_STOPPED; tscEmbedded = 1; taosIgnSIGPIPE(); taosBlockSIGPIPE(); @@ -147,7 +149,9 @@ static void dnodeCleanupMain() { int32_t dnodeInit() { SSteps *steps = taosStepInit(10, dnodeReportStartup); if (steps == NULL) return -1; - +#if 1 + dnodeSetRunStat(DN_RUN_STAT_RUNNING); +#endif taosStepAdd(steps, "dnode-main", dnodeInitMain, dnodeCleanupMain); taosStepAdd(steps, "dnode-rpc", rpcInit, rpcCleanup); taosStepAdd(steps, "dnode-tfs", NULL, NULL); diff --git a/source/dnode/mgmt/src/dnodeMnode.c b/source/dnode/mgmt/src/dnodeMnode.c index 48cc1cb20d..b6d54a28d2 100644 --- a/source/dnode/mgmt/src/dnodeMnode.c +++ b/source/dnode/mgmt/src/dnodeMnode.c @@ -330,17 +330,22 @@ static void dnodeProcessMnodeApplyQueue(void *unused, SMnodeMsg *pMsg) { mnodePr static void dnodeProcessMnodeSyncQueue(void *unused, SMnodeMsg *pMsg) { mnodeProcessMsg(pMsg, MN_MSG_TYPE_SYNC); } static int32_t dnodeWriteMnodeMsgToQueue(taos_queue pQueue, SRpcMsg *pRpcMsg) { - int32_t code = 0; + int32_t code = 0; + SMnodeMsg *pMsg = NULL; if (pQueue == NULL) { code = TSDB_CODE_DND_MSG_NOT_PROCESSED; } else { - SMnodeMsg *pMsg = mnodeInitMsg(pRpcMsg); + pMsg = mnodeInitMsg(pRpcMsg); if (pMsg == NULL) { code = terrno; } } + if (code == 0) { + code = taosWriteQitem(pQueue, pMsg); + } + if (code != TSDB_CODE_SUCCESS) { SRpcMsg rsp = {.handle = pRpcMsg->handle, .code = code}; rpcSendResponse(&rsp); diff --git a/source/dnode/mgmt/src/dnodeTransport.c b/source/dnode/mgmt/src/dnodeTransport.c index 475470b574..ed5650ace8 100644 --- a/source/dnode/mgmt/src/dnodeTransport.c +++ b/source/dnode/mgmt/src/dnodeTransport.c @@ -314,6 +314,8 @@ static int32_t dnodeRetrieveUserAuthInfo(char *user, char *spi, char *encrypt, c } static int32_t dnodeInitShellServer() { + dnodeInitMsgFp(); + int32_t numOfThreads = (int32_t)((tsNumOfCores * tsNumOfThreadsPerCore) / 2.0); if (numOfThreads < 1) { numOfThreads = 1; @@ -336,6 +338,17 @@ static int32_t dnodeInitShellServer() { return -1; } +#if 1 + SRpcMsg rpcMsg = {0}; + rpcMsg.msgType = TSDB_MSG_TYPE_CREATE_USER; + rpcMsg.contLen = sizeof(SCreateUserMsg); + rpcMsg.pCont = rpcMallocCont(rpcMsg.contLen); + SCreateUserMsg *pMsg = (SCreateUserMsg*)rpcMsg.pCont; + strcpy(pMsg->user, "u1"); + strcpy(pMsg->pass, "up1"); + dnodeProcessShellReq(&rpcMsg, NULL); + +#endif dInfo("dnode shell rpc server is initialized"); return 0; } @@ -369,7 +382,11 @@ void dnodeCleanupTrans() { dnodeCleanupClient(); } -void dnodeSendMsgToDnode(SEpSet *epSet, SRpcMsg *rpcMsg) { rpcSendRequest(tsTrans.clientRpc, epSet, rpcMsg, NULL); } +void dnodeSendMsgToDnode(SEpSet *epSet, SRpcMsg *rpcMsg) { + #if 0 + rpcSendRequest(tsTrans.clientRpc, epSet, rpcMsg, NULL); + #endif + } void dnodeSendMsgToMnode(SRpcMsg *rpcMsg) { SEpSet epSet = {0}; diff --git a/source/dnode/mnode/CMakeLists.txt b/source/dnode/mnode/CMakeLists.txt index 6de5f06476..45a5af7566 100644 --- a/source/dnode/mnode/CMakeLists.txt +++ b/source/dnode/mnode/CMakeLists.txt @@ -1,3 +1,2 @@ add_subdirectory(impl) add_subdirectory(sdb) -add_subdirectory(transaction) diff --git a/source/dnode/mnode/impl/CMakeLists.txt b/source/dnode/mnode/impl/CMakeLists.txt index 4c9d44b39e..49c9d54513 100644 --- a/source/dnode/mnode/impl/CMakeLists.txt +++ b/source/dnode/mnode/impl/CMakeLists.txt @@ -8,7 +8,6 @@ target_include_directories( target_link_libraries( mnode PRIVATE sdb - PRIVATE transaction PUBLIC transport PUBLIC cjson ) \ No newline at end of file diff --git a/source/dnode/mnode/impl/inc/mnodeDef.h b/source/dnode/mnode/impl/inc/mnodeDef.h index b6449ecfe7..ccdba13006 100644 --- a/source/dnode/mnode/impl/inc/mnodeDef.h +++ b/source/dnode/mnode/impl/inc/mnodeDef.h @@ -76,6 +76,28 @@ typedef enum { MN_AUTH_MAX } EMnAuthOp; +typedef enum { + TRN_STAGE_PREPARE = 1, + TRN_STAGE_EXECUTE = 2, + TRN_STAGE_COMMIT = 3, + TRN_STAGE_ROLLBACK = 4, + TRN_STAGE_RETRY = 5 +} ETrnStage; + +typedef enum { TRN_POLICY_ROLLBACK = 1, TRN_POLICY_RETRY = 2 } ETrnPolicy; + + +typedef struct STrans { + int32_t id; + ETrnStage stage; + ETrnPolicy policy; + void *rpcHandle; + SArray *redoLogs; + SArray *undoLogs; + SArray *commitLogs; + SArray *redoActions; + SArray *undoActions; +} STrans; typedef struct SClusterObj { diff --git a/source/dnode/mnode/impl/inc/mnodeInt.h b/source/dnode/mnode/impl/inc/mnodeInt.h index 96803ba4a5..373be6aa84 100644 --- a/source/dnode/mnode/impl/inc/mnodeInt.h +++ b/source/dnode/mnode/impl/inc/mnodeInt.h @@ -18,13 +18,23 @@ #include "mnodeDef.h" #include "sdb.h" -#include "trn.h" +#include "tstep.h" #ifdef __cplusplus extern "C" { #endif -typedef void (*MnodeRpcFp)(SMnodeMsg *pMsg); +typedef int32_t (*MnodeRpcFp)(SMnodeMsg *pMsg); + +typedef struct SMnode { + int32_t dnodeId; + int64_t clusterId; + tmr_h timer; + SSteps *pInitSteps; + SSteps *pStartSteps; + SMnodePara para; + MnodeRpcFp msgFp[TSDB_MSG_TYPE_MAX]; +} SMnode; tmr_h mnodeGetTimer(); int32_t mnodeGetDnodeId(); diff --git a/include/dnode/mnode/transaction/trn.h b/source/dnode/mnode/impl/inc/mnodeTrans.h similarity index 77% rename from include/dnode/mnode/transaction/trn.h rename to source/dnode/mnode/impl/inc/mnodeTrans.h index 8ba043de12..2abe101dfd 100644 --- a/include/dnode/mnode/transaction/trn.h +++ b/source/dnode/mnode/impl/inc/mnodeTrans.h @@ -13,25 +13,20 @@ * along with this program. If not, see . */ -#ifndef _TD_TRANSACTION_H_ -#define _TD_TRANSACTION_H_ +#ifndef _TD_TRANSACTION_INT_H_ +#define _TD_TRANSACTION_INT_H_ -#include "sdb.h" -#include "taosmsg.h" +#include "mnodeInt.h" #ifdef __cplusplus extern "C" { #endif -typedef struct STrans STrans; -typedef enum { TRN_POLICY_ROLLBACK = 1, TRN_POLICY_RETRY = 2 } ETrnPolicy; +int32_t mnodeInitTrans(); +void mnodeCleanupTrans(); -int32_t trnInit(); -void trnCleanup(); - -STrans *trnCreate(ETrnPolicy); +STrans *trnCreate(ETrnPolicy policy, void *rpcHandle); void trnDrop(STrans *pTrans); -void trnSetRpcHandle(STrans *pTrans, void *rpcHandle); int32_t trnAppendRedoLog(STrans *pTrans, SSdbRaw *pRaw); int32_t trnAppendUndoLog(STrans *pTrans, SSdbRaw *pRaw); int32_t trnAppendCommitLog(STrans *pTrans, SSdbRaw *pRaw); @@ -42,8 +37,11 @@ int32_t trnPrepare(STrans *pTrans, int32_t (*syncfp)(SSdbRaw *pRaw, void *pData) int32_t trnApply(SSdbRaw *pRaw, void *pData, int32_t code); int32_t trnExecute(int32_t tranId); +SSdbRaw *trnActionEncode(STrans *pTrans); +SSdbRow *trnActionDecode(SSdbRaw *pRaw); + #ifdef __cplusplus } #endif -#endif /*_TD_TRANSACTION_H_*/ +#endif /*_TD_TRANSACTION_INT_H_*/ diff --git a/source/dnode/mnode/impl/src/mnode.c b/source/dnode/mnode/impl/src/mnode.c index f52e60dbad..ae909917de 100644 --- a/source/dnode/mnode/impl/src/mnode.c +++ b/source/dnode/mnode/impl/src/mnode.c @@ -16,7 +16,6 @@ #define _DEFAULT_SOURCE #include "os.h" #include "tglobal.h" -#include "tstep.h" #include "tqueue.h" #include "mnodeAcct.h" #include "mnodeAuth.h" @@ -34,16 +33,9 @@ #include "mnodeTelem.h" #include "mnodeUser.h" #include "mnodeVgroup.h" +#include "mnodeTrans.h" -static struct { - int32_t dnodeId; - int64_t clusterId; - tmr_h timer; - SSteps *pInitSteps; - SSteps *pStartSteps; - SMnodePara para; - MnodeRpcFp msgFp[TSDB_MSG_TYPE_MAX]; -} tsMint; +SMnode tsMint = {0}; int32_t mnodeGetDnodeId() { return tsMint.para.dnodeId; } @@ -116,7 +108,7 @@ static int32_t mnodeAllocInitSteps() { struct SSteps *steps = taosStepInit(16, NULL); if (steps == NULL) return -1; - if (taosStepAdd(steps, "mnode-trans", trnInit, trnCleanup) != 0) return -1; + if (taosStepAdd(steps, "mnode-trans", mnodeInitTrans, mnodeCleanupTrans) != 0) return -1; if (taosStepAdd(steps, "mnode-cluster", mnodeInitCluster, mnodeCleanupCluster) != 0) return -1; if (taosStepAdd(steps, "mnode-dnode", mnodeInitDnode, mnodeCleanupDnode) != 0) return -1; if (taosStepAdd(steps, "mnode-mnode", mnodeInitMnode, mnodeCleanupMnode) != 0) return -1; @@ -224,10 +216,14 @@ void mnodeCleanupMsg(SMnodeMsg *pMsg) { static void mnodeProcessRpcMsg(SMnodeMsg *pMsg) { int32_t msgType = pMsg->rpcMsg.msgType; - if (tsMint.msgFp[msgType] == NULL) { + MnodeRpcFp fp = tsMint.msgFp[msgType]; + if (fp == NULL) { } - (*tsMint.msgFp[msgType])(pMsg); + int32_t code = (fp)(pMsg); + if (code != 0) { + assert(code); + } } void mnodeSetMsgFp(int32_t msgType, MnodeRpcFp fp) { diff --git a/source/dnode/mnode/impl/src/mnodeSync.c b/source/dnode/mnode/impl/src/mnodeSync.c index fd34793172..6e4084ffa6 100644 --- a/source/dnode/mnode/impl/src/mnodeSync.c +++ b/source/dnode/mnode/impl/src/mnodeSync.c @@ -16,6 +16,7 @@ #define _DEFAULT_SOURCE #include "os.h" #include "mnodeInt.h" +#include "mnodeTrans.h" int32_t mnodeInitSync() { return 0; } void mnodeCleanUpSync() {} diff --git a/source/dnode/mnode/impl/src/mnodeTrans.c b/source/dnode/mnode/impl/src/mnodeTrans.c new file mode 100644 index 0000000000..4cd6bf7bdb --- /dev/null +++ b/source/dnode/mnode/impl/src/mnodeTrans.c @@ -0,0 +1,502 @@ +/* + * Copyright (c) 2019 TAOS Data, Inc. + * + * This program is free software: you can use, redistribute, and/or modify + * it under the terms of the GNU Affero General Public License, version 3 + * or later ("AGPL"), as published by the Free Software Foundation. + * + * This program is distributed in the hope that it will be useful, but WITHOUT + * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or + * FITNESS FOR A PARTICULAR PURPOSE. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + */ + +#define _DEFAULT_SOURCE +#include "mnodeTrans.h" +#include "trpc.h" + +#define SDB_TRANS_VER 1 +#define TRN_DEFAULT_ARRAY_SIZE 8 + +SSdbRaw *trnActionEncode(STrans *pTrans) { + int32_t rawDataLen = 10 * sizeof(int32_t); + int32_t redoLogNum = taosArrayGetSize(pTrans->redoLogs); + int32_t undoLogNum = taosArrayGetSize(pTrans->undoLogs); + int32_t commitLogNum = taosArrayGetSize(pTrans->commitLogs); + int32_t redoActionNum = taosArrayGetSize(pTrans->redoActions); + int32_t undoActionNum = taosArrayGetSize(pTrans->undoActions); + + for (int32_t index = 0; index < redoLogNum; ++index) { + SSdbRaw *pTmp = taosArrayGet(pTrans->redoLogs, index); + rawDataLen += sdbGetRawTotalSize(pTmp); + } + + for (int32_t index = 0; index < undoLogNum; ++index) { + SSdbRaw *pTmp = taosArrayGet(pTrans->undoLogs, index); + rawDataLen += sdbGetRawTotalSize(pTmp); + } + + for (int32_t index = 0; index < commitLogNum; ++index) { + SSdbRaw *pTmp = taosArrayGet(pTrans->commitLogs, index); + rawDataLen += sdbGetRawTotalSize(pTmp); + } + + SSdbRaw *pRaw = sdbAllocRaw(SDB_TRANS, SDB_TRANS_VER, rawDataLen); + if (pRaw == NULL) { + mError("trn:%d, failed to alloc raw since %s", pTrans->id, terrstr()); + return NULL; + } + + int32_t dataPos = 0; + SDB_SET_INT32(pRaw, dataPos, pTrans->id) + SDB_SET_INT8(pRaw, dataPos, pTrans->stage) + SDB_SET_INT8(pRaw, dataPos, pTrans->policy) + SDB_SET_INT32(pRaw, dataPos, redoLogNum) + SDB_SET_INT32(pRaw, dataPos, undoLogNum) + SDB_SET_INT32(pRaw, dataPos, commitLogNum) + SDB_SET_INT32(pRaw, dataPos, redoActionNum) + SDB_SET_INT32(pRaw, dataPos, undoActionNum) + + for (int32_t index = 0; index < redoLogNum; ++index) { + SSdbRaw *pTmp = taosArrayGet(pTrans->redoLogs, index); + int32_t len = sdbGetRawTotalSize(pTmp); + SDB_SET_INT32(pRaw, dataPos, len) + SDB_SET_BINARY(pRaw, dataPos, (void *)pTmp, len) + } + + for (int32_t index = 0; index < undoLogNum; ++index) { + SSdbRaw *pTmp = taosArrayGet(pTrans->undoLogs, index); + int32_t len = sdbGetRawTotalSize(pTmp); + SDB_SET_INT32(pRaw, dataPos, len) + SDB_SET_BINARY(pRaw, dataPos, (void *)pTmp, len) + } + + for (int32_t index = 0; index < commitLogNum; ++index) { + SSdbRaw *pTmp = taosArrayGet(pTrans->commitLogs, index); + int32_t len = sdbGetRawTotalSize(pTmp); + SDB_SET_INT32(pRaw, dataPos, len) + SDB_SET_BINARY(pRaw, dataPos, (void *)pTmp, len) + } + + mDebug("trn:%d, is encoded as raw:%p, len:%d", pTrans->id, pRaw, dataPos); + return pRaw; +} + +SSdbRow *trnActionDecode(SSdbRaw *pRaw) { + int8_t sver = 0; + if (sdbGetRawSoftVer(pRaw, &sver) != 0) { + mError("failed to get soft ver from raw:%p since %s", pRaw, terrstr()); + return NULL; + } + + if (sver != SDB_TRANS_VER) { + terrno = TSDB_CODE_SDB_INVALID_DATA_VER; + mError("failed to get check soft ver from raw:%p since %s", pRaw, terrstr()); + return NULL; + } + + SSdbRow *pRow = sdbAllocRow(sizeof(STrans)); + STrans *pTrans = sdbGetRowObj(pRow); + if (pTrans == NULL) { + mError("failed to alloc trans from raw:%p since %s", pRaw, terrstr()); + return NULL; + } + + pTrans->redoLogs = taosArrayInit(TRN_DEFAULT_ARRAY_SIZE, sizeof(void *)); + pTrans->undoLogs = taosArrayInit(TRN_DEFAULT_ARRAY_SIZE, sizeof(void *)); + pTrans->commitLogs = taosArrayInit(TRN_DEFAULT_ARRAY_SIZE, sizeof(void *)); + pTrans->redoActions = taosArrayInit(TRN_DEFAULT_ARRAY_SIZE, sizeof(void *)); + pTrans->undoActions = taosArrayInit(TRN_DEFAULT_ARRAY_SIZE, sizeof(void *)); + + if (pTrans->redoLogs == NULL || pTrans->undoLogs == NULL || pTrans->commitLogs == NULL || + pTrans->redoActions == NULL || pTrans->undoActions == NULL) { + terrno = TSDB_CODE_OUT_OF_MEMORY; + mDebug("trn:%d, failed to create array while parsed from raw:%p", pTrans->id, pRaw); + return NULL; + } + + int32_t redoLogNum = 0; + int32_t undoLogNum = 0; + int32_t commitLogNum = 0; + int32_t redoActionNum = 0; + int32_t undoActionNum = 0; + + int32_t dataPos = 0; + SDB_GET_INT32(pRaw, pRow, dataPos, &pTrans->id) + SDB_GET_INT8(pRaw, pRow, dataPos, (int8_t *)&pTrans->stage) + SDB_GET_INT8(pRaw, pRow, dataPos, (int8_t *)&pTrans->policy) + SDB_GET_INT32(pRaw, pRow, dataPos, &redoLogNum) + SDB_GET_INT32(pRaw, pRow, dataPos, &undoLogNum) + SDB_GET_INT32(pRaw, pRow, dataPos, &commitLogNum) + SDB_GET_INT32(pRaw, pRow, dataPos, &redoActionNum) + SDB_GET_INT32(pRaw, pRow, dataPos, &undoActionNum) + + int32_t code = 0; + for (int32_t index = 0; index < redoLogNum; ++index) { + int32_t dataLen = 0; + SDB_GET_INT32(pRaw, pRow, dataPos, &dataLen) + + char *pData = malloc(dataLen); + SDB_GET_BINARY(pRaw, pRow, dataPos, pData, dataLen); + void *ret = taosArrayPush(pTrans->redoLogs, pData); + if (ret == NULL) { + code = TSDB_CODE_OUT_OF_MEMORY; + break; + } + } + + if (code != 0) { + terrno = code; + mError("trn:%d, failed to parse from raw:%p since %s", pTrans->id, pRaw, terrstr()); + trnDrop(pTrans); + return NULL; + } + + mDebug("trn:%d, is parsed from raw:%p", pTrans->id, pRaw); + return pRow; +} + +static int32_t trnActionInsert(STrans *pTrans) { + SArray *pArray = pTrans->redoLogs; + int32_t arraySize = taosArrayGetSize(pArray); + + for (int32_t index = 0; index < arraySize; ++index) { + SSdbRaw *pRaw = taosArrayGet(pArray, index); + int32_t code = sdbWrite(pRaw); + if (code != 0) { + mError("trn:%d, failed to write raw:%p to sdb since %s", pTrans->id, pRaw, terrstr()); + return code; + } + } + + mDebug("trn:%d, write to sdb", pTrans->id); + return 0; +} + +static int32_t trnActionDelete(STrans *pTrans) { + SArray *pArray = pTrans->redoLogs; + int32_t arraySize = taosArrayGetSize(pArray); + + for (int32_t index = 0; index < arraySize; ++index) { + SSdbRaw *pRaw = taosArrayGet(pArray, index); + int32_t code = sdbWrite(pRaw); + if (code != 0) { + mError("trn:%d, failed to write raw:%p to sdb since %s", pTrans->id, pRaw, terrstr()); + return code; + } + } + + mDebug("trn:%d, delete from sdb", pTrans->id); + return 0; +} + +static int32_t trnActionUpdate(STrans *pTrans, STrans *pDstTrans) { + assert(true); + SArray *pArray = pTrans->redoLogs; + int32_t arraySize = taosArrayGetSize(pArray); + + for (int32_t index = 0; index < arraySize; ++index) { + SSdbRaw *pRaw = taosArrayGet(pArray, index); + int32_t code = sdbWrite(pRaw); + if (code != 0) { + mError("trn:%d, failed to write raw:%p to sdb since %s", pTrans->id, pRaw, terrstr()); + return code; + } + } + + pTrans->stage = pDstTrans->stage; + mDebug("trn:%d, update in sdb", pTrans->id); + return 0; +} + +static int32_t trnGenerateTransId() { return 1; } + +STrans *trnCreate(ETrnPolicy policy, void *rpcHandle) { + STrans *pTrans = calloc(1, sizeof(STrans)); + if (pTrans == NULL) { + terrno = TSDB_CODE_OUT_OF_MEMORY; + mError("failed to create transaction since %s", terrstr()); + return NULL; + } + + pTrans->id = trnGenerateTransId(); + pTrans->stage = TRN_STAGE_PREPARE; + pTrans->policy = policy; + pTrans->rpcHandle = rpcHandle; + pTrans->redoLogs = taosArrayInit(TRN_DEFAULT_ARRAY_SIZE, sizeof(void *)); + pTrans->undoLogs = taosArrayInit(TRN_DEFAULT_ARRAY_SIZE, sizeof(void *)); + pTrans->commitLogs = taosArrayInit(TRN_DEFAULT_ARRAY_SIZE, sizeof(void *)); + pTrans->redoActions = taosArrayInit(TRN_DEFAULT_ARRAY_SIZE, sizeof(void *)); + pTrans->undoActions = taosArrayInit(TRN_DEFAULT_ARRAY_SIZE, sizeof(void *)); + + if (pTrans->redoLogs == NULL || pTrans->undoLogs == NULL || pTrans->commitLogs == NULL || + pTrans->redoActions == NULL || pTrans->undoActions == NULL) { + terrno = TSDB_CODE_OUT_OF_MEMORY; + mError("failed to create transaction since %s", terrstr()); + return NULL; + } + + mDebug("trn:%d, is created, %p", pTrans->id, pTrans); + return pTrans; +} + +static void trnDropArray(SArray *pArray) { + for (int32_t index = 0; index < pArray->size; ++index) { + SSdbRaw *pRaw = taosArrayGet(pArray, index); + tfree(pRaw); + } + + taosArrayDestroy(pArray); +} + +void trnDrop(STrans *pTrans) { + trnDropArray(pTrans->redoLogs); + trnDropArray(pTrans->undoLogs); + trnDropArray(pTrans->commitLogs); + trnDropArray(pTrans->redoActions); + trnDropArray(pTrans->undoActions); + + mDebug("trn:%d, is dropped, %p", pTrans->id, pTrans); + tfree(pTrans); +} + +void trnSetRpcHandle(STrans *pTrans, void *rpcHandle) { + pTrans->rpcHandle = rpcHandle; + mTrace("trn:%d, set rpc handle:%p", pTrans->id, rpcHandle); +} + +static int32_t trnAppendArray(SArray *pArray, SSdbRaw *pRaw) { + if (pArray == NULL || pRaw == NULL) { + terrno = TSDB_CODE_OUT_OF_MEMORY; + return -1; + } + + void *ptr = taosArrayPush(pArray, pRaw); + if (ptr == NULL) { + terrno = TSDB_CODE_OUT_OF_MEMORY; + return -1; + } + + return 0; +} + +int32_t trnAppendRedoLog(STrans *pTrans, SSdbRaw *pRaw) { + int32_t code = trnAppendArray(pTrans->redoLogs, pRaw); + mTrace("trn:%d, raw:%p append to redo logs, code:%d", pTrans->id, pRaw, code); + return code; +} + +int32_t trnAppendUndoLog(STrans *pTrans, SSdbRaw *pRaw) { + int32_t code = trnAppendArray(pTrans->undoLogs, pRaw); + mTrace("trn:%d, raw:%p append to undo logs, code:%d", pTrans->id, pRaw, code); + return code; +} + +int32_t trnAppendCommitLog(STrans *pTrans, SSdbRaw *pRaw) { + int32_t code = trnAppendArray(pTrans->commitLogs, pRaw); + mTrace("trn:%d, raw:%p append to commit logs, code:%d", pTrans->id, pRaw, code); + return code; +} + +int32_t trnAppendRedoAction(STrans *pTrans, SEpSet *pEpSet, void *pMsg) { + int32_t code = trnAppendArray(pTrans->redoActions, pMsg); + mTrace("trn:%d, msg:%p append to redo actions", pTrans->id, pMsg); + return code; +} + +int32_t trnAppendUndoAction(STrans *pTrans, SEpSet *pEpSet, void *pMsg) { + int32_t code = trnAppendArray(pTrans->undoActions, pMsg); + mTrace("trn:%d, msg:%p append to undo actions", pTrans->id, pMsg); + return code; +} + +int32_t mnodeInitTrans() { + SSdbTable table = {.sdbType = SDB_TRANS, + .keyType = SDB_KEY_INT32, + .encodeFp = (SdbEncodeFp)trnActionEncode, + .decodeFp = (SdbDecodeFp)trnActionDecode, + .insertFp = (SdbInsertFp)trnActionInsert, + .updateFp = (SdbUpdateFp)trnActionUpdate, + .deleteFp = (SdbDeleteFp)trnActionDelete}; + sdbSetTable(table); + + mInfo("trn module is initialized"); + return 0; +} + +void mnodeCleanupTrans() { mInfo("trn module is cleaned up"); } + + +int32_t trnPrepare(STrans *pTrans, int32_t (*syncfp)(SSdbRaw *pRaw, void *pData)) { + if (syncfp == NULL) return -1; + + SSdbRaw *pRaw = trnActionEncode(pTrans); + if (pRaw == NULL) { + mError("trn:%d, failed to decode trans since %s", pTrans->id, terrstr()); + return -1; + } + sdbSetRawStatus(pRaw, SDB_STATUS_CREATING); + + if (sdbWrite(pRaw) != 0) { + mError("trn:%d, failed to write trans since %s", pTrans->id, terrstr()); + return -1; + } + + if ((*syncfp)(pRaw, pTrans->rpcHandle) != 0) { + mError("trn:%d, failed to sync trans since %s", pTrans->id, terrstr()); + return -1; + } + + return 0; +} + +static void trnSendRpcRsp(void *rpcHandle, int32_t code) { + if (rpcHandle != NULL) { + SRpcMsg rspMsg = {.handle = rpcHandle, .code = terrno}; + rpcSendResponse(&rspMsg); + } +} + +int32_t trnApply(SSdbRaw *pRaw, void *pData, int32_t code) { + if (code != 0) { + trnSendRpcRsp(pData, terrno); + return 0; + } + + if (sdbWrite(pData) != 0) { + code = terrno; + trnSendRpcRsp(pData, code); + terrno = code; + return -1; + } + + return 0; +} + +static int32_t trnExecuteArray(SArray *pArray) { + for (int32_t index = 0; index < pArray->size; ++index) { + SSdbRaw *pRaw = taosArrayGetP(pArray, index); + if (sdbWrite(pRaw) != 0) { + return -1; + } + } + + return 0; +} + +static int32_t trnExecuteRedoLogs(STrans *pTrans) { return trnExecuteArray(pTrans->redoLogs); } + +static int32_t trnExecuteUndoLogs(STrans *pTrans) { return trnExecuteArray(pTrans->undoLogs); } + +static int32_t trnExecuteCommitLogs(STrans *pTrans) { return trnExecuteArray(pTrans->commitLogs); } + +static int32_t trnExecuteRedoActions(STrans *pTrans) { return trnExecuteArray(pTrans->redoActions); } + +static int32_t trnExecuteUndoActions(STrans *pTrans) { return trnExecuteArray(pTrans->undoActions); } + +static int32_t trnPerformPrepareStage(STrans *pTrans) { + if (trnExecuteRedoLogs(pTrans) == 0) { + pTrans->stage = TRN_STAGE_EXECUTE; + return 0; + } else { + pTrans->stage = TRN_STAGE_ROLLBACK; + return -1; + } +} + +static int32_t trnPerformExecuteStage(STrans *pTrans) { + int32_t code = trnExecuteRedoActions(pTrans); + + if (code == 0) { + pTrans->stage = TRN_STAGE_COMMIT; + return 0; + } else if (code == TSDB_CODE_MND_ACTION_IN_PROGRESS) { + return -1; + } else { + if (pTrans->policy == TRN_POLICY_RETRY) { + pTrans->stage = TRN_STAGE_RETRY; + } else { + pTrans->stage = TRN_STAGE_ROLLBACK; + } + return 0; + } +} + +static int32_t trnPerformCommitStage(STrans *pTrans) { + if (trnExecuteCommitLogs(pTrans) == 0) { + pTrans->stage = TRN_STAGE_EXECUTE; + return 0; + } else { + pTrans->stage = TRN_STAGE_ROLLBACK; + return -1; + } +} + +static int32_t trnPerformRollbackStage(STrans *pTrans) { + if (trnExecuteCommitLogs(pTrans) == 0) { + pTrans->stage = TRN_STAGE_EXECUTE; + return 0; + } else { + pTrans->stage = TRN_STAGE_ROLLBACK; + return -1; + } +} + +static int32_t trnPerformRetryStage(STrans *pTrans) { + if (trnExecuteCommitLogs(pTrans) == 0) { + pTrans->stage = TRN_STAGE_EXECUTE; + return 0; + } else { + pTrans->stage = TRN_STAGE_ROLLBACK; + return -1; + } +} + +int32_t trnExecute(int32_t tranId) { + int32_t code = 0; + + STrans *pTrans = sdbAcquire(SDB_TRANS, &tranId); + if (pTrans == NULL) { + return -1; + } + + if (pTrans->stage == TRN_STAGE_PREPARE) { + if (trnPerformPrepareStage(pTrans) != 0) { + sdbRelease(pTrans); + return -1; + } + } + + if (pTrans->stage == TRN_STAGE_EXECUTE) { + if (trnPerformExecuteStage(pTrans) != 0) { + sdbRelease(pTrans); + return -1; + } + } + + if (pTrans->stage == TRN_STAGE_COMMIT) { + if (trnPerformCommitStage(pTrans) != 0) { + sdbRelease(pTrans); + return -1; + } + } + + if (pTrans->stage == TRN_STAGE_ROLLBACK) { + if (trnPerformRollbackStage(pTrans) != 0) { + sdbRelease(pTrans); + return -1; + } + } + + if (pTrans->stage == TRN_STAGE_RETRY) { + if (trnPerformRetryStage(pTrans) != 0) { + sdbRelease(pTrans); + return -1; + } + } + + sdbRelease(pTrans); + return 0; +} \ No newline at end of file diff --git a/source/dnode/mnode/impl/src/mnodeUser.c b/source/dnode/mnode/impl/src/mnodeUser.c index 63aa171238..b8e5706484 100644 --- a/source/dnode/mnode/impl/src/mnodeUser.c +++ b/source/dnode/mnode/impl/src/mnodeUser.c @@ -18,6 +18,7 @@ #include "os.h" #include "tglobal.h" #include "tkey.h" +#include "mnodeTrans.h" #define SDB_USER_VER 1 @@ -142,12 +143,12 @@ static int32_t mnodeCreateUser(char *acct, char *user, char *pass, SMnodeMsg *pM userObj.updateTime = userObj.createdTime; userObj.rootAuth = 0; - STrans *pTrans = trnCreate(TRN_POLICY_ROLLBACK); + STrans *pTrans = trnCreate(TRN_POLICY_ROLLBACK, pMsg->rpcMsg.handle); if (pTrans == NULL) return -1; - trnSetRpcHandle(pTrans, pMsg->rpcMsg.handle); SSdbRaw *pRedoRaw = mnodeUserActionEncode(&userObj); if (pRedoRaw == NULL || trnAppendRedoLog(pTrans, pRedoRaw) != 0) { + mError("failed to append redo log since %s", terrstr()); trnDrop(pTrans); return -1; } @@ -155,6 +156,7 @@ static int32_t mnodeCreateUser(char *acct, char *user, char *pass, SMnodeMsg *pM SSdbRaw *pUndoRaw = mnodeUserActionEncode(&userObj); if (pUndoRaw == NULL || trnAppendUndoLog(pTrans, pUndoRaw) != 0) { + mError("failed to append undo log since %s", terrstr()); trnDrop(pTrans); return -1; } @@ -162,6 +164,7 @@ static int32_t mnodeCreateUser(char *acct, char *user, char *pass, SMnodeMsg *pM SSdbRaw *pCommitRaw = mnodeUserActionEncode(&userObj); if (pCommitRaw == NULL || trnAppendCommitLog(pTrans, pCommitRaw) != 0) { + mError("failed to append commit log since %s", terrstr()); trnDrop(pTrans); return -1; } @@ -228,6 +231,8 @@ int32_t mnodeInitUser() { .deleteFp = (SdbDeleteFp)mnodeUserActionDelete}; sdbSetTable(table); + mnodeSetMsgFp(TSDB_MSG_TYPE_CREATE_USER, mnodeProcessCreateUserMsg); + return 0; } diff --git a/source/dnode/mnode/sdb/src/sdbFile.c b/source/dnode/mnode/sdb/src/sdbFile.c index 69c82c77f1..169b2cf809 100644 --- a/source/dnode/mnode/sdb/src/sdbFile.c +++ b/source/dnode/mnode/sdb/src/sdbFile.c @@ -45,7 +45,7 @@ static int32_t sdbCreateDir() { static int32_t sdbRunDeployFp() { mDebug("start to run deploy functions"); - for (int32_t i = SDB_START; i < SDB_MAX; ++i) { + for (int32_t i = SDB_MAX - 1; i > SDB_START; --i) { SdbDeployFp fp = tsSdb.deployFps[i]; if (fp == NULL) continue; if ((*fp)() != 0) { @@ -54,6 +54,7 @@ static int32_t sdbRunDeployFp() { } } + mDebug("end of run deploy functions"); return 0; } diff --git a/source/dnode/mnode/transaction/CMakeLists.txt b/source/dnode/mnode/transaction/CMakeLists.txt deleted file mode 100644 index d35a8c9b3f..0000000000 --- a/source/dnode/mnode/transaction/CMakeLists.txt +++ /dev/null @@ -1,15 +0,0 @@ -aux_source_directory(src MNODE_SRC) -add_library(transaction ${MNODE_SRC}) -target_include_directories( - transaction - PUBLIC "${CMAKE_SOURCE_DIR}/include/dnode/mnode/transaction" - private "${CMAKE_CURRENT_SOURCE_DIR}/inc" -) -target_link_libraries( - transaction - PRIVATE os - PRIVATE common - PRIVATE util - PRIVATE sdb - PRIVATE transport -) diff --git a/source/dnode/mnode/transaction/inc/trnInt.h b/source/dnode/mnode/transaction/inc/trnInt.h deleted file mode 100644 index 771217dcc0..0000000000 --- a/source/dnode/mnode/transaction/inc/trnInt.h +++ /dev/null @@ -1,70 +0,0 @@ -/* - * 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_TRANSACTION_INT_H_ -#define _TD_TRANSACTION_INT_H_ - -#include "os.h" -#include "trn.h" -#include "tglobal.h" -#include "tarray.h" -#include "tlog.h" - -#ifdef __cplusplus -extern "C" { -#endif - -#define mFatal(...) { if (mDebugFlag & DEBUG_FATAL) { taosPrintLog("MND FATAL ", 255, __VA_ARGS__); }} -#define mError(...) { if (mDebugFlag & DEBUG_ERROR) { taosPrintLog("MND ERROR ", 255, __VA_ARGS__); }} -#define mWarn(...) { if (mDebugFlag & DEBUG_WARN) { taosPrintLog("MND WARN ", 255, __VA_ARGS__); }} -#define mInfo(...) { if (mDebugFlag & DEBUG_INFO) { taosPrintLog("MND ", 255, __VA_ARGS__); }} -#define mDebug(...) { if (mDebugFlag & DEBUG_DEBUG) { taosPrintLog("MND ", mDebugFlag, __VA_ARGS__); }} -#define mTrace(...) { if (mDebugFlag & DEBUG_TRACE) { taosPrintLog("MND ", mDebugFlag, __VA_ARGS__); }} - -#define TRN_VER 1 -#define TRN_DEFAULT_ARRAY_SIZE 8 - -typedef enum { - TRN_STAGE_PREPARE = 1, - TRN_STAGE_EXECUTE = 2, - TRN_STAGE_COMMIT = 3, - TRN_STAGE_ROLLBACK = 4, - TRN_STAGE_RETRY = 5 -} ETrnStage; - -typedef struct STrans { - int32_t id; - int8_t stage; - int8_t policy; - void *rpcHandle; - SArray *redoLogs; - SArray *undoLogs; - SArray *commitLogs; - SArray *redoActions; - SArray *undoActions; -} STrans; - -SSdbRaw *trnActionEncode(STrans *pTrans); -STrans *trnActionDecode(SSdbRaw *pRaw); -int32_t trnActionInsert(STrans *pTrans); -int32_t trnActionDelete(STrans *pTrans); -int32_t trnActionUpdate(STrans *pSrcTrans, STrans *pDstTrans); -int32_t trnGenerateTransId(); - -#ifdef __cplusplus -} -#endif - -#endif /*_TD_TRANSACTION_INT_H_*/ diff --git a/source/dnode/mnode/transaction/src/trn.c b/source/dnode/mnode/transaction/src/trn.c deleted file mode 100644 index 0d7c1a061e..0000000000 --- a/source/dnode/mnode/transaction/src/trn.c +++ /dev/null @@ -1,233 +0,0 @@ -/* - * Copyright (c) 2019 TAOS Data, Inc. - * - * This program is free software: you can use, redistribute, and/or modify - * it under the terms of the GNU Affero General Public License, version 3 - * or later ("AGPL"), as published by the Free Software Foundation. - * - * This program is distributed in the hope that it will be useful, but WITHOUT - * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or - * FITNESS FOR A PARTICULAR PURPOSE. - * - * You should have received a copy of the GNU Affero General Public License - * along with this program. If not, see . - */ - -#define _DEFAULT_SOURCE -#include "trnInt.h" - -#define SDB_TRANS_VER 1 - -SSdbRaw *trnActionEncode(STrans *pTrans) { - int32_t rawDataLen = 10 * sizeof(int32_t); - int32_t redoLogNum = taosArrayGetSize(pTrans->redoLogs); - int32_t undoLogNum = taosArrayGetSize(pTrans->undoLogs); - int32_t commitLogNum = taosArrayGetSize(pTrans->commitLogs); - int32_t redoActionNum = taosArrayGetSize(pTrans->redoActions); - int32_t undoActionNum = taosArrayGetSize(pTrans->undoActions); - - for (int32_t index = 0; index < redoLogNum; ++index) { - SSdbRaw *pRaw = taosArrayGet(pTrans->redoLogs, index); - rawDataLen += sdbGetRawTotalSize(pRaw); - } - - for (int32_t index = 0; index < undoLogNum; ++index) { - SSdbRaw *pRaw = taosArrayGet(pTrans->undoLogs, index); - rawDataLen += sdbGetRawTotalSize(pRaw); - } - - for (int32_t index = 0; index < commitLogNum; ++index) { - SSdbRaw *pRaw = taosArrayGet(pTrans->commitLogs, index); - rawDataLen += sdbGetRawTotalSize(pRaw); - } - - SSdbRaw *pRaw = sdbAllocRaw(SDB_TRANS, SDB_TRANS_VER, rawDataLen); - if (pRaw == NULL) return NULL; - - int32_t dataPos = 0; - SDB_SET_INT32(pData, dataPos, pTrans->id) - SDB_SET_INT8(pData, dataPos, pTrans->stage) - SDB_SET_INT8(pData, dataPos, pTrans->policy) - SDB_SET_INT32(pData, dataPos, redoLogNum) - SDB_SET_INT32(pData, dataPos, undoLogNum) - SDB_SET_INT32(pData, dataPos, commitLogNum) - SDB_SET_INT32(pData, dataPos, redoActionNum) - SDB_SET_INT32(pData, dataPos, undoActionNum) - SDB_SET_DATALEN(pRaw, dataPos); - - return pRaw; -} - -STrans *trnActionDecode(SSdbRaw *pRaw) { - int8_t sver = 0; - if (sdbGetRawSoftVer(pRaw, &sver) != 0) return NULL; - - if (sver != SDB_TRANS_VER) { - terrno = TSDB_CODE_SDB_INVALID_DATA_VER; - return NULL; - } - - SSdbRow *pRow = sdbAllocRow(sizeof(STrans)); - STrans *pTrans = sdbGetRowObj(pRow); - if (pTrans == NULL) return NULL; - - int32_t redoLogNum = 0; - int32_t undoLogNum = 0; - int32_t commitLogNum = 0; - int32_t redoActionNum = 0; - int32_t undoActionNum = 0; - - int32_t dataPos = 0; - SDB_GET_INT32(pRaw, pRow, dataPos, &pTrans->id) - SDB_GET_INT8(pRaw, pRow, dataPos, &pTrans->stage) - SDB_GET_INT8(pRaw, pRow, dataPos, &pTrans->policy) - SDB_GET_INT32(pRaw, pRow, dataPos, &redoLogNum) - SDB_GET_INT32(pRaw, pRow, dataPos, &undoLogNum) - SDB_GET_INT32(pRaw, pRow, dataPos, &commitLogNum) - SDB_GET_INT32(pRaw, pRow, dataPos, &redoActionNum) - SDB_GET_INT32(pRaw, pRow, dataPos, &undoActionNum) - - for (int32_t index = 0; index < redoLogNum; ++index) { - int32_t dataLen = 0; - SDB_GET_INT32(pRaw, pRow, dataPos, &dataLen) - - char *pData = malloc(dataLen); - SDB_GET_BINARY(pRaw, pRow, dataPos, pData, dataLen); - void *ret = taosArrayPush(pTrans->redoLogs, pData); - if (ret == NULL) { - terrno = TSDB_CODE_OUT_OF_MEMORY; - break; - } - } - - // if (code != 0) { - // trnDrop(pTrans); - // terrno = code; - // return NULL; - // } - - return pTrans; -} - -int32_t trnActionInsert(STrans *pTrans) { - SArray *pArray = pTrans->redoLogs; - int32_t arraySize = taosArrayGetSize(pArray); - - for (int32_t index = 0; index < arraySize; ++index) { - SSdbRaw *pRaw = taosArrayGetP(pArray, index); - int32_t code = sdbWrite(pRaw); - if (code != 0) { - return code; - } - } - - return 0; -} - -int32_t trnActionDelete(STrans *pTrans) { - SArray *pArray = pTrans->redoLogs; - int32_t arraySize = taosArrayGetSize(pArray); - - for (int32_t index = 0; index < arraySize; ++index) { - SSdbRaw *pRaw = taosArrayGetP(pArray, index); - int32_t code = sdbWrite(pRaw); - if (code != 0) { - return code; - } - } - - return 0; -} - -int32_t trnActionUpdate(STrans *pSrcTrans, STrans *pDstTrans) { return 0; } - -int32_t trnGenerateTransId() { return 1; } - -STrans *trnCreate(ETrnPolicy policy) { - STrans *pTrans = calloc(1, sizeof(STrans)); - if (pTrans == NULL) { - terrno = TSDB_CODE_OUT_OF_MEMORY; - return NULL; - } - - pTrans->id = trnGenerateTransId(); - pTrans->stage = TRN_STAGE_PREPARE; - pTrans->policy = policy; - pTrans->redoLogs = taosArrayInit(TRN_DEFAULT_ARRAY_SIZE, sizeof(void *)); - pTrans->undoLogs = taosArrayInit(TRN_DEFAULT_ARRAY_SIZE, sizeof(void *)); - pTrans->commitLogs = taosArrayInit(TRN_DEFAULT_ARRAY_SIZE, sizeof(void *)); - pTrans->redoActions = taosArrayInit(TRN_DEFAULT_ARRAY_SIZE, sizeof(void *)); - pTrans->undoActions = taosArrayInit(TRN_DEFAULT_ARRAY_SIZE, sizeof(void *)); - - if (pTrans->redoLogs == NULL || pTrans->undoLogs == NULL || pTrans->commitLogs == NULL || - pTrans->redoActions == NULL || pTrans->undoActions == NULL) { - terrno = TSDB_CODE_OUT_OF_MEMORY; - return NULL; - } - - return pTrans; -} - -static void trnDropArray(SArray *pArray) { - for (int32_t index = 0; index < pArray->size; ++index) { - SSdbRaw *pRaw = taosArrayGetP(pArray, index); - tfree(pRaw); - } - - taosArrayDestroy(pArray); -} - -void trnDrop(STrans *pTrans) { - trnDropArray(pTrans->redoLogs); - trnDropArray(pTrans->undoLogs); - trnDropArray(pTrans->commitLogs); - trnDropArray(pTrans->redoActions); - trnDropArray(pTrans->undoActions); - tfree(pTrans); -} - -void trnSetRpcHandle(STrans *pTrans, void *rpcHandle) { pTrans->rpcHandle = rpcHandle; } - -static int32_t trnAppendArray(SArray *pArray, SSdbRaw *pRaw) { - if (pArray == NULL || pRaw == NULL) { - terrno = TSDB_CODE_OUT_OF_MEMORY; - return -1; - } - - void *ptr = taosArrayPush(pArray, &pRaw); - if (ptr == NULL) { - terrno = TSDB_CODE_OUT_OF_MEMORY; - return -1; - } - - return 0; -} - -int32_t trnAppendRedoLog(STrans *pTrans, SSdbRaw *pRaw) { return trnAppendArray(pTrans->redoLogs, pRaw); } - -int32_t trnAppendUndoLog(STrans *pTrans, SSdbRaw *pRaw) { return trnAppendArray(pTrans->undoLogs, pRaw); } - -int32_t trnAppendCommitLog(STrans *pTrans, SSdbRaw *pRaw) { return trnAppendArray(pTrans->commitLogs, pRaw); } - -int32_t trnAppendRedoAction(STrans *pTrans, SEpSet *pEpSet, void *pMsg) { - return trnAppendArray(pTrans->redoActions, pMsg); -} - -int32_t trnAppendUndoAction(STrans *pTrans, SEpSet *pEpSet, void *pMsg) { - return trnAppendArray(pTrans->undoActions, pMsg); -} - -int32_t trnInit() { - SSdbTable table = {.sdbType = SDB_TRANS, - .keyType = SDB_KEY_INT32, - .encodeFp = (SdbEncodeFp)trnActionEncode, - .decodeFp = (SdbDecodeFp)trnActionDecode, - .insertFp = (SdbInsertFp)trnActionInsert, - .updateFp = (SdbUpdateFp)trnActionUpdate, - .deleteFp = (SdbDeleteFp)trnActionDelete}; - sdbSetTable(table); - - return 0; -} - -void trnCleanup() {} diff --git a/source/dnode/mnode/transaction/src/trnExec.c b/source/dnode/mnode/transaction/src/trnExec.c deleted file mode 100644 index fc15c16225..0000000000 --- a/source/dnode/mnode/transaction/src/trnExec.c +++ /dev/null @@ -1,189 +0,0 @@ -/* - * Copyright (c) 2019 TAOS Data, Inc. - * - * This program is free software: you can use, redistribute, and/or modify - * it under the terms of the GNU Affero General Public License, version 3 - * or later ("AGPL"), as published by the Free Software Foundation. - * - * This program is distributed in the hope that it will be useful, but WITHOUT - * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or - * FITNESS FOR A PARTICULAR PURPOSE. - * - * You should have received a copy of the GNU Affero General Public License - * along with this program. If not, see . - */ - -#define _DEFAULT_SOURCE -#include "trnInt.h" -#include "trpc.h" - -int32_t trnPrepare(STrans *pTrans, int32_t (*syncfp)(SSdbRaw *pRaw, void *pData)) { - if (syncfp == NULL) return -1; - - SSdbRaw *pRaw = trnActionEncode(pTrans); - if (pRaw == NULL) { - mError("tranId:%d, failed to decode trans since %s", pTrans->id, terrstr()); - return -1; - } - - if (sdbWrite(pRaw) != 0) { - mError("tranId:%d, failed to write trans since %s", pTrans->id, terrstr()); - return -1; - } - - if ((*syncfp)(pRaw, pTrans->rpcHandle) != 0) { - mError("tranId:%d, failed to sync trans since %s", pTrans->id, terrstr()); - return -1; - } - - return 0; -} - -static void trnSendRpcRsp(void *rpcHandle, int32_t code) { - if (rpcHandle != NULL) { - SRpcMsg rspMsg = {.handle = rpcHandle, .code = terrno}; - rpcSendResponse(&rspMsg); - } -} - -int32_t trnApply(SSdbRaw *pRaw, void *pData, int32_t code) { - if (code != 0) { - trnSendRpcRsp(pData, terrno); - return 0; - } - - if (sdbWrite(pData) != 0) { - code = terrno; - trnSendRpcRsp(pData, code); - terrno = code; - return -1; - } - - return 0; -} - -static int32_t trnExecuteArray(SArray *pArray) { - for (int32_t index = 0; index < pArray->size; ++index) { - SSdbRaw *pRaw = taosArrayGetP(pArray, index); - if (sdbWrite(pRaw) != 0) { - return -1; - } - } - - return 0; -} - -static int32_t trnExecuteRedoLogs(STrans *pTrans) { return trnExecuteArray(pTrans->redoLogs); } - -static int32_t trnExecuteUndoLogs(STrans *pTrans) { return trnExecuteArray(pTrans->undoLogs); } - -static int32_t trnExecuteCommitLogs(STrans *pTrans) { return trnExecuteArray(pTrans->commitLogs); } - -static int32_t trnExecuteRedoActions(STrans *pTrans) { return trnExecuteArray(pTrans->redoActions); } - -static int32_t trnExecuteUndoActions(STrans *pTrans) { return trnExecuteArray(pTrans->undoActions); } - -static int32_t trnPerformPrepareStage(STrans *pTrans) { - if (trnExecuteRedoLogs(pTrans) == 0) { - pTrans->stage = TRN_STAGE_EXECUTE; - return 0; - } else { - pTrans->stage = TRN_STAGE_ROLLBACK; - return -1; - } -} - -static int32_t trnPerformExecuteStage(STrans *pTrans) { - int32_t code = trnExecuteRedoActions(pTrans); - - if (code == 0) { - pTrans->stage = TRN_STAGE_COMMIT; - return 0; - } else if (code == TSDB_CODE_MND_ACTION_IN_PROGRESS) { - return -1; - } else { - if (pTrans->policy == TRN_POLICY_RETRY) { - pTrans->stage = TRN_STAGE_RETRY; - } else { - pTrans->stage = TRN_STAGE_ROLLBACK; - } - return 0; - } -} - -static int32_t trnPerformCommitStage(STrans *pTrans) { - if (trnExecuteCommitLogs(pTrans) == 0) { - pTrans->stage = TRN_STAGE_EXECUTE; - return 0; - } else { - pTrans->stage = TRN_STAGE_ROLLBACK; - return -1; - } -} - -static int32_t trnPerformRollbackStage(STrans *pTrans) { - if (trnExecuteCommitLogs(pTrans) == 0) { - pTrans->stage = TRN_STAGE_EXECUTE; - return 0; - } else { - pTrans->stage = TRN_STAGE_ROLLBACK; - return -1; - } -} - -static int32_t trnPerformRetryStage(STrans *pTrans) { - if (trnExecuteCommitLogs(pTrans) == 0) { - pTrans->stage = TRN_STAGE_EXECUTE; - return 0; - } else { - pTrans->stage = TRN_STAGE_ROLLBACK; - return -1; - } -} - -int32_t trnExecute(int32_t tranId) { - int32_t code = 0; - - STrans *pTrans = sdbAcquire(SDB_TRANS, &tranId); - if (pTrans == NULL) { - return -1; - } - - if (pTrans->stage == TRN_STAGE_PREPARE) { - if (trnPerformPrepareStage(pTrans) != 0) { - sdbRelease(pTrans); - return -1; - } - } - - if (pTrans->stage == TRN_STAGE_EXECUTE) { - if (trnPerformExecuteStage(pTrans) != 0) { - sdbRelease(pTrans); - return -1; - } - } - - if (pTrans->stage == TRN_STAGE_COMMIT) { - if (trnPerformCommitStage(pTrans) != 0) { - sdbRelease(pTrans); - return -1; - } - } - - if (pTrans->stage == TRN_STAGE_ROLLBACK) { - if (trnPerformRollbackStage(pTrans) != 0) { - sdbRelease(pTrans); - return -1; - } - } - - if (pTrans->stage == TRN_STAGE_RETRY) { - if (trnPerformRetryStage(pTrans) != 0) { - sdbRelease(pTrans); - return -1; - } - } - - sdbRelease(pTrans); - return 0; -} \ No newline at end of file diff --git a/source/libs/transport/src/rpcMain.c b/source/libs/transport/src/rpcMain.c index 1aafc880be..934a8dd6ab 100644 --- a/source/libs/transport/src/rpcMain.c +++ b/source/libs/transport/src/rpcMain.c @@ -505,14 +505,18 @@ void rpcSendRedirectRsp(void *thandle, const SEpSet *pEpSet) { } int rpcGetConnInfo(void *thandle, SRpcConnInfo *pInfo) { +#if 0 SRpcConn *pConn = (SRpcConn *)thandle; if (pConn->user[0] == 0) return -1; pInfo->clientIp = pConn->peerIp; pInfo->clientPort = pConn->peerPort; // pInfo->serverIp = pConn->destIp; - + tstrncpy(pInfo->user, pConn->user, sizeof(pInfo->user)); +#else + strcpy(pInfo->user, "root"); +#endif return 0; } From 7f1a4b9c6c2765e7736199cfd5c9a037e3ee3201 Mon Sep 17 00:00:00 2001 From: Shengliang Guan Date: Fri, 19 Nov 2021 17:15:24 +0800 Subject: [PATCH 11/44] remove global variables --- include/dnode/mnode/mnode.h | 22 ++++++---- include/dnode/mnode/sdb/sdb.h | 2 + include/dnode/vnode/vnode.h | 12 ++++-- source/dnode/mgmt/inc/dnodeDnode.h | 2 +- source/dnode/mgmt/inc/dnodeInt.h | 3 ++ source/dnode/mgmt/inc/dnodeTransport.h | 4 +- source/dnode/mgmt/src/dnodeDnode.c | 4 +- source/dnode/mgmt/src/dnodeMnode.c | 17 +++++--- source/dnode/mgmt/src/dnodeTransport.c | 6 +-- source/dnode/mgmt/src/dnodeVnodes.c | 8 ++-- source/dnode/mnode/impl/inc/mnodeInt.h | 23 +++++++++-- source/dnode/mnode/impl/src/mnode.c | 56 +++++++++++++++++--------- source/dnode/mnode/sdb/inc/sdbInt.h | 6 +-- source/dnode/mnode/sdb/src/sdb.c | 2 +- 14 files changed, 113 insertions(+), 54 deletions(-) diff --git a/include/dnode/mnode/mnode.h b/include/dnode/mnode/mnode.h index e03d3ffd18..1ef3bd579f 100644 --- a/include/dnode/mnode/mnode.h +++ b/include/dnode/mnode/mnode.h @@ -43,16 +43,24 @@ typedef struct { int64_t compStorage; } SMnodeLoad; +typedef struct SMnode SMnode; +typedef struct SServer SServer; + +typedef void (*SendMsgToDnodeFp)(SServer *pServer, struct SEpSet *epSet, struct SRpcMsg *rpcMsg); +typedef void (*SendMsgToMnodeFp)(SServer *pServer, struct SRpcMsg *rpcMsg); +typedef void (*SendRedirectMsgFp)(SServer *pServer, struct SRpcMsg *rpcMsg, bool forShell); +typedef int32_t (*PutMsgToMnodeQFp)(SServer *pServer, SMnodeMsg *pMsg); + typedef struct { - int32_t dnodeId; - int64_t clusterId; - void (*SendMsgToDnode)(struct SEpSet *epSet, struct SRpcMsg *rpcMsg); - void (*SendMsgToMnode)(struct SRpcMsg *rpcMsg); - void (*SendRedirectMsg)(struct SRpcMsg *rpcMsg, bool forShell); - int32_t (*PutMsgIntoApplyQueue)(SMnodeMsg *pMsg); + int32_t dnodeId; + int64_t clusterId; + PutMsgToMnodeQFp putMsgToApplyMsgFp; + SendMsgToDnodeFp sendMsgToDnodeFp; + SendMsgToMnodeFp sendMsgToMnodeFp; + SendRedirectMsgFp sendRedirectMsgFp; } SMnodePara; -int32_t mnodeInit(SMnodePara para); +SMnode* mnodeCreate(SMnodePara para); void mnodeCleanup(); int32_t mnodeDeploy(SMnodeCfg *pCfg); diff --git a/include/dnode/mnode/sdb/sdb.h b/include/dnode/mnode/sdb/sdb.h index cb514cef53..784672e0ec 100644 --- a/include/dnode/mnode/sdb/sdb.h +++ b/include/dnode/mnode/sdb/sdb.h @@ -144,6 +144,8 @@ typedef struct { SdbDeleteFp deleteFp; } SSdbTable; +typedef struct SSdb SSdb; + int32_t sdbInit(); void sdbCleanup(); void sdbSetTable(SSdbTable table); diff --git a/include/dnode/vnode/vnode.h b/include/dnode/vnode/vnode.h index 52470d60a9..419c9dfcfc 100644 --- a/include/dnode/vnode/vnode.h +++ b/include/dnode/vnode/vnode.h @@ -184,10 +184,16 @@ typedef struct { SRpcMsg rpcMsg[]; } SVnodeMsg; +typedef struct SServer SServer; +typedef void (*SendMsgToDnodeFp)(SServer *pServer, struct SEpSet *epSet, struct SRpcMsg *rpcMsg); +typedef void (*SendMsgToMnodeFp)(SServer *pServer, struct SRpcMsg *rpcMsg); +typedef void (*SendRedirectMsgFp)(SServer *pServer, struct SRpcMsg *rpcMsg, bool forShell); +typedef int32_t (*PutMsgToVnodeQFp)(SServer *pServer, int32_t vgId, SVnodeMsg *pMsg); + typedef struct { - void (*SendMsgToDnode)(SEpSet *pEpSet, SRpcMsg *pMsg); - void (*SendMsgToMnode)(SRpcMsg *pMsg); - int32_t (*PutMsgIntoApplyQueue)(int32_t vgId, SVnodeMsg *pMsg); + PutMsgToVnodeQFp putMsgToApplyQueueFp; + SendMsgToDnodeFp sendMsgToDnodeFp; + SendMsgToMnodeFp sendMsgToMnodeFp; } SVnodePara; int32_t vnodeInit(SVnodePara); diff --git a/source/dnode/mgmt/inc/dnodeDnode.h b/source/dnode/mgmt/inc/dnodeDnode.h index 2ca1368e63..0d1e93e60f 100644 --- a/source/dnode/mgmt/inc/dnodeDnode.h +++ b/source/dnode/mgmt/inc/dnodeDnode.h @@ -30,7 +30,7 @@ int64_t dnodeGetClusterId(); void dnodeGetDnodeEp(int32_t dnodeId, char *epstr, char *fqdn, uint16_t *port); void dnodeGetMnodeEpSetForPeer(SEpSet *epSet); void dnodeGetMnodeEpSetForShell(SEpSet *epSet); -void dnodeSendRedirectMsg(SRpcMsg *rpcMsg, bool forShell); +void dnodeSendRedirectMsg(SServer *pServer, SRpcMsg *rpcMsg, bool forShell); #ifdef __cplusplus } diff --git a/source/dnode/mgmt/inc/dnodeInt.h b/source/dnode/mgmt/inc/dnodeInt.h index a4226a47b8..48da1ee558 100644 --- a/source/dnode/mgmt/inc/dnodeInt.h +++ b/source/dnode/mgmt/inc/dnodeInt.h @@ -37,6 +37,9 @@ extern int32_t dDebugFlag; typedef enum { DN_RUN_STAT_INIT, DN_RUN_STAT_RUNNING, DN_RUN_STAT_STOPPED } EDnStat; typedef void (*MsgFp)(SRpcMsg *pMsg, SEpSet *pEpSet); +typedef struct SServer { +} SServer; + int32_t dnodeInit(); void dnodeCleanup(); diff --git a/source/dnode/mgmt/inc/dnodeTransport.h b/source/dnode/mgmt/inc/dnodeTransport.h index 95ca1b81e5..4a9518fe09 100644 --- a/source/dnode/mgmt/inc/dnodeTransport.h +++ b/source/dnode/mgmt/inc/dnodeTransport.h @@ -23,8 +23,8 @@ extern "C" { int32_t dnodeInitTrans(); void dnodeCleanupTrans(); -void dnodeSendMsgToMnode(SRpcMsg *rpcMsg); -void dnodeSendMsgToDnode(SEpSet *epSet, SRpcMsg *rpcMsg); +void dnodeSendMsgToMnode(SServer *pServer, SRpcMsg *rpcMsg); +void dnodeSendMsgToDnode(SServer *pServer, SEpSet *epSet, SRpcMsg *rpcMsg); #ifdef __cplusplus } diff --git a/source/dnode/mgmt/src/dnodeDnode.c b/source/dnode/mgmt/src/dnodeDnode.c index 7843075b30..8a326c72d5 100644 --- a/source/dnode/mgmt/src/dnodeDnode.c +++ b/source/dnode/mgmt/src/dnodeDnode.c @@ -78,7 +78,7 @@ void dnodeGetMnodeEpSetForShell(SEpSet *pEpSet) { pthread_mutex_unlock(&tsDnode.mutex); } -void dnodeSendRedirectMsg(SRpcMsg *pMsg, bool forShell) { +void dnodeSendRedirectMsg(SServer *pServer, SRpcMsg *pMsg, bool forShell) { int32_t msgType = pMsg->msgType; SEpSet epSet = {0}; @@ -383,7 +383,7 @@ static void dnodeSendStatusMsg() { contLen = sizeof(SStatusMsg) + pStatus->vnodeLoads.num * sizeof(SVnodeLoad); SRpcMsg rpcMsg = {.pCont = pStatus, .contLen = contLen, .msgType = TSDB_MSG_TYPE_STATUS}; - dnodeSendMsgToMnode(&rpcMsg); + dnodeSendMsgToMnode(NULL, &rpcMsg); } static void dnodeUpdateCfg(SDnodeCfg *pCfg) { diff --git a/source/dnode/mgmt/src/dnodeMnode.c b/source/dnode/mgmt/src/dnodeMnode.c index b6d54a28d2..232af96897 100644 --- a/source/dnode/mgmt/src/dnodeMnode.c +++ b/source/dnode/mgmt/src/dnodeMnode.c @@ -38,6 +38,7 @@ static struct { taos_queue pSyncQ; taos_queue pMgmtQ; SSteps *pSteps; + SMnode *pMnode; SRWLatch latch; } tsMnode = {0}; @@ -360,7 +361,7 @@ void dnodeProcessMnodeWriteMsg(SRpcMsg *pMsg, SEpSet *pEpSet) { dnodeWriteMnodeMsgToQueue(tsMnode.pWriteQ, pMsg); dnodeReleaseMnode(); } else { - dnodeSendRedirectMsg(pMsg, 0); + dnodeSendRedirectMsg(NULL, pMsg, 0); } } @@ -381,7 +382,7 @@ void dnodeProcessMnodeReadMsg(SRpcMsg *pMsg, SEpSet *pEpSet) { dnodeWriteMnodeMsgToQueue(tsMnode.pReadQ, pMsg); dnodeReleaseMnode(); } else { - dnodeSendRedirectMsg(pMsg, 0); + dnodeSendRedirectMsg(NULL, pMsg, 0); } } @@ -505,11 +506,15 @@ static int32_t dnodeInitMnodeModule() { SMnodePara para; para.dnodeId = dnodeGetDnodeId(); para.clusterId = dnodeGetClusterId(); - para.SendMsgToDnode = dnodeSendMsgToDnode; - para.SendMsgToMnode = dnodeSendMsgToMnode; - para.SendRedirectMsg = dnodeSendRedirectMsg; + para.sendMsgToDnodeFp = dnodeSendMsgToDnode; + para.sendMsgToMnodeFp = dnodeSendMsgToMnode; + para.sendMsgToMnodeFp = dnodeSendRedirectMsg; - return mnodeInit(para); + tsMnode.pMnode = mnodeCreate(para); + if (tsMnode.pMnode != NULL) { + return -1; + } + return 0; } static void dnodeCleanupMnodeModule() { mnodeCleanup(); } diff --git a/source/dnode/mgmt/src/dnodeTransport.c b/source/dnode/mgmt/src/dnodeTransport.c index ed5650ace8..b3263aadca 100644 --- a/source/dnode/mgmt/src/dnodeTransport.c +++ b/source/dnode/mgmt/src/dnodeTransport.c @@ -382,14 +382,14 @@ void dnodeCleanupTrans() { dnodeCleanupClient(); } -void dnodeSendMsgToDnode(SEpSet *epSet, SRpcMsg *rpcMsg) { +void dnodeSendMsgToDnode(SServer *pServer, SEpSet *epSet, SRpcMsg *rpcMsg) { #if 0 rpcSendRequest(tsTrans.clientRpc, epSet, rpcMsg, NULL); #endif } -void dnodeSendMsgToMnode(SRpcMsg *rpcMsg) { +void dnodeSendMsgToMnode(SServer *pServer, SRpcMsg *rpcMsg) { SEpSet epSet = {0}; dnodeGetMnodeEpSetForPeer(&epSet); - dnodeSendMsgToDnode(&epSet, rpcMsg); + dnodeSendMsgToDnode(NULL, &epSet, rpcMsg); } \ No newline at end of file diff --git a/source/dnode/mgmt/src/dnodeVnodes.c b/source/dnode/mgmt/src/dnodeVnodes.c index 4ec9e1dc60..bd15850c42 100644 --- a/source/dnode/mgmt/src/dnodeVnodes.c +++ b/source/dnode/mgmt/src/dnodeVnodes.c @@ -815,7 +815,7 @@ void dnodeProcessVnodeFetchMsg(SRpcMsg *pMsg, SEpSet *pEpSet) { } } -static int32_t dnodePutMsgIntoVnodeApplyQueue(int32_t vgId, SVnodeMsg *pMsg) { +static int32_t dnodePutMsgIntoVnodeApplyQueue(SServer *pServer, int32_t vgId, SVnodeMsg *pMsg) { SVnodeObj *pVnode = dnodeAcquireVnode(vgId); if (pVnode == NULL) { return terrno; @@ -973,9 +973,9 @@ static void dnodeCleanupVnodeSyncWorker() { tMWorkerCleanup(&tsVnodes.syncPool); static int32_t dnodeInitVnodeModule() { SVnodePara para; - para.SendMsgToDnode = dnodeSendMsgToDnode; - para.SendMsgToMnode = dnodeSendMsgToMnode; - para.PutMsgIntoApplyQueue = dnodePutMsgIntoVnodeApplyQueue; + para.sendMsgToDnodeFp = dnodeSendMsgToDnode; + para.sendMsgToMnodeFp = dnodeSendMsgToMnode; + para.putMsgToApplyQueueFp = dnodePutMsgIntoVnodeApplyQueue; return vnodeInit(para); } diff --git a/source/dnode/mnode/impl/inc/mnodeInt.h b/source/dnode/mnode/impl/inc/mnodeInt.h index 373be6aa84..b0005acc20 100644 --- a/source/dnode/mnode/impl/inc/mnodeInt.h +++ b/source/dnode/mnode/impl/inc/mnodeInt.h @@ -26,6 +26,16 @@ extern "C" { typedef int32_t (*MnodeRpcFp)(SMnodeMsg *pMsg); +typedef struct SMnodeBak { + int32_t dnodeId; + int64_t clusterId; + tmr_h timer; + SSteps *pInitSteps; + SSteps *pStartSteps; + SMnodePara para; + MnodeRpcFp msgFp[TSDB_MSG_TYPE_MAX]; +} SMnodeBak; + typedef struct SMnode { int32_t dnodeId; int64_t clusterId; @@ -34,15 +44,22 @@ typedef struct SMnode { SSteps *pStartSteps; SMnodePara para; MnodeRpcFp msgFp[TSDB_MSG_TYPE_MAX]; + + struct SSdb *pSdb; + struct SServer *pServer; + PutMsgToMnodeQFp putMsgToApplyMsgFp; + SendMsgToDnodeFp sendMsgToDnodeFp; + SendMsgToMnodeFp sendMsgToMnodeFp; + SendRedirectMsgFp sendRedirectMsgFp; } SMnode; tmr_h mnodeGetTimer(); int32_t mnodeGetDnodeId(); int64_t mnodeGetClusterId(); -void mnodeSendMsgToDnode(struct SEpSet *epSet, struct SRpcMsg *rpcMsg); -void mnodeSendMsgToMnode(struct SRpcMsg *rpcMsg); -void mnodeSendRedirectMsg(struct SRpcMsg *rpcMsg, bool forShell); +void mnodeSendMsgToDnode(SMnode *pMnode, struct SEpSet *epSet, struct SRpcMsg *rpcMsg); +void mnodeSendMsgToMnode(SMnode *pMnode, struct SRpcMsg *rpcMsg); +void mnodeSendRedirectMsg(SMnode *pMnode, struct SRpcMsg *rpcMsg, bool forShell); void mnodeSetMsgFp(int32_t msgType, MnodeRpcFp fp); diff --git a/source/dnode/mnode/impl/src/mnode.c b/source/dnode/mnode/impl/src/mnode.c index ae909917de..9ea4ebe0e6 100644 --- a/source/dnode/mnode/impl/src/mnode.c +++ b/source/dnode/mnode/impl/src/mnode.c @@ -35,17 +35,26 @@ #include "mnodeVgroup.h" #include "mnodeTrans.h" -SMnode tsMint = {0}; +SMnodeBak tsMint = {0}; int32_t mnodeGetDnodeId() { return tsMint.para.dnodeId; } int64_t mnodeGetClusterId() { return tsMint.para.clusterId; } -void mnodeSendMsgToDnode(struct SEpSet *epSet, struct SRpcMsg *rpcMsg) { (*tsMint.para.SendMsgToDnode)(epSet, rpcMsg); } +void mnodeSendMsgToDnode(SMnode *pMnode, struct SEpSet *epSet, struct SRpcMsg *rpcMsg) { + assert(pMnode); + (*pMnode->sendMsgToDnodeFp)(pMnode->pServer, epSet, rpcMsg); +} -void mnodeSendMsgToMnode(struct SRpcMsg *rpcMsg) { return (*tsMint.para.SendMsgToMnode)(rpcMsg); } +void mnodeSendMsgToMnode(SMnode *pMnode, struct SRpcMsg *rpcMsg) { + assert(pMnode); + (*pMnode->sendMsgToMnodeFp)(pMnode->pServer, rpcMsg); +} -void mnodeSendRedirectMsg(struct SRpcMsg *rpcMsg, bool forShell) { (*tsMint.para.SendRedirectMsg)(rpcMsg, forShell); } +void mnodeSendRedirectMsg(SMnode *pMnode, struct SRpcMsg *rpcMsg, bool forShell) { + assert(pMnode); + (*pMnode->sendRedirectMsgFp)(pMnode->pServer, rpcMsg, forShell); +} static int32_t mnodeInitTimer() { if (tsMint.timer == NULL) { @@ -68,35 +77,40 @@ static void mnodeCleanupTimer() { tmr_h mnodeGetTimer() { return tsMint.timer; } -static int32_t mnodeSetPara(SMnodePara para) { - tsMint.para = para; +static int32_t mnodeSetPara(SMnode *pMnode, SMnodePara para) { + pMnode->dnodeId = para.dnodeId; + pMnode->clusterId = para.clusterId; + pMnode->putMsgToApplyMsgFp = para.putMsgToApplyMsgFp; + pMnode->sendMsgToDnodeFp = para.sendMsgToDnodeFp; + pMnode->sendMsgToMnodeFp = para.sendMsgToMnodeFp; + pMnode->sendRedirectMsgFp = para.sendRedirectMsgFp; - if (tsMint.para.SendMsgToDnode == NULL) { + if (pMnode->sendMsgToDnodeFp == NULL) { terrno = TSDB_CODE_MND_APP_ERROR; return -1; } - if (tsMint.para.SendMsgToMnode == NULL) { + if (pMnode->sendMsgToMnodeFp == NULL) { terrno = TSDB_CODE_MND_APP_ERROR; return -1; } - if (tsMint.para.SendRedirectMsg == NULL) { + if (pMnode->sendRedirectMsgFp == NULL) { terrno = TSDB_CODE_MND_APP_ERROR; return -1; } - if (tsMint.para.PutMsgIntoApplyQueue == NULL) { + if (pMnode->putMsgToApplyMsgFp == NULL) { terrno = TSDB_CODE_MND_APP_ERROR; return -1; } - if (tsMint.para.dnodeId < 0) { + if (pMnode->dnodeId < 0) { terrno = TSDB_CODE_MND_APP_ERROR; return -1; } - if (tsMint.para.clusterId < 0) { + if (pMnode->clusterId < 0) { terrno = TSDB_CODE_MND_APP_ERROR; return -1; } @@ -142,23 +156,27 @@ static int32_t mnodeAllocStartSteps() { return 0; } -int32_t mnodeInit(SMnodePara para) { - if (mnodeSetPara(para) != 0) { +SMnode *mnodeCreate(SMnodePara para) { + SMnode *pMnode = calloc(1, sizeof(SMnode)); + + if (mnodeSetPara(pMnode, para) != 0) { + free(pMnode); mError("failed to init mnode para since %s", terrstr()); - return -1; + return NULL; } if (mnodeAllocInitSteps() != 0) { mError("failed to alloc init steps since %s", terrstr()); - return -1; + return NULL; } if (mnodeAllocStartSteps() != 0) { mError("failed to alloc start steps since %s", terrstr()); - return -1; + return NULL; } - return taosStepExec(tsMint.pInitSteps); + taosStepExec(tsMint.pInitSteps); + return NULL; } void mnodeCleanup() { taosStepCleanup(tsMint.pInitSteps); } @@ -234,7 +252,7 @@ void mnodeSetMsgFp(int32_t msgType, MnodeRpcFp fp) { void mnodeProcessMsg(SMnodeMsg *pMsg, EMnMsgType msgType) { if (!mnodeIsMaster()) { - mnodeSendRedirectMsg(&pMsg->rpcMsg, true); + mnodeSendRedirectMsg(NULL, &pMsg->rpcMsg, true); mnodeCleanupMsg(pMsg); return; } diff --git a/source/dnode/mnode/sdb/inc/sdbInt.h b/source/dnode/mnode/sdb/inc/sdbInt.h index aa0b3c8a58..2b3c577ba9 100644 --- a/source/dnode/mnode/sdb/inc/sdbInt.h +++ b/source/dnode/mnode/sdb/inc/sdbInt.h @@ -52,7 +52,7 @@ typedef struct SSdbRow { char pObj[]; } SSdbRow; -typedef struct { +typedef struct SSdb { char *currDir; char *syncDir; char *tmpDir; @@ -67,9 +67,9 @@ typedef struct { SdbDeployFp deployFps[SDB_MAX]; SdbEncodeFp encodeFps[SDB_MAX]; SdbDecodeFp decodeFps[SDB_MAX]; -} SSdbMgr; +} SSdb; -extern SSdbMgr tsSdb; +extern SSdb tsSdb; int32_t sdbWriteImp(SSdbRaw *pRaw); diff --git a/source/dnode/mnode/sdb/src/sdb.c b/source/dnode/mnode/sdb/src/sdb.c index 8e9b7fbecc..83496f3794 100644 --- a/source/dnode/mnode/sdb/src/sdb.c +++ b/source/dnode/mnode/sdb/src/sdb.c @@ -17,7 +17,7 @@ #include "sdbInt.h" #include "tglobal.h" -SSdbMgr tsSdb = {0}; +SSdb tsSdb = {0}; int32_t sdbInit() { char path[PATH_MAX + 100]; From 2f1bb2815f0bead617797cd7f0bcba43892eed75 Mon Sep 17 00:00:00 2001 From: Shengliang Guan Date: Sun, 21 Nov 2021 00:13:35 +0800 Subject: [PATCH 12/44] add daemon --- include/dnode/mgmt/dnode.h | 46 +++ include/dnode/mnode/mnode.h | 147 +++++++-- include/dnode/vnode/vnode.h | 10 +- source/dnode/CMakeLists.txt | 2 +- source/dnode/mgmt/CMakeLists.txt | 18 +- source/dnode/mgmt/daemon/CMakeLists.txt | 8 + .../mgmt/{src/dnode.c => daemon/src/daemon.c} | 21 +- source/dnode/mgmt/impl/CMakeLists.txt | 16 + source/dnode/mgmt/{ => impl}/inc/dnodeDnode.h | 6 +- source/dnode/mgmt/impl/inc/dnodeInt.h | 97 ++++++ source/dnode/mgmt/{ => impl}/inc/dnodeMnode.h | 0 .../mgmt/{ => impl}/inc/dnodeTransport.h | 4 +- .../dnode/mgmt/{ => impl}/inc/dnodeVnodes.h | 0 source/dnode/mgmt/{ => impl}/src/dnodeDnode.c | 282 +++++++++--------- source/dnode/mgmt/impl/src/dnodeInt.c | 260 ++++++++++++++++ source/dnode/mgmt/{ => impl}/src/dnodeMnode.c | 32 +- .../mgmt/{ => impl}/src/dnodeTransport.c | 12 +- .../dnode/mgmt/{ => impl}/src/dnodeVnodes.c | 2 +- source/dnode/mgmt/inc/dnodeInt.h | 57 ---- source/dnode/mgmt/src/dnodeInt.c | 181 ----------- source/dnode/mnode/impl/inc/mnodeInt.h | 21 +- source/dnode/mnode/impl/src/mnode.c | 114 +++---- source/dnode/mnode/impl/src/mnodeAuth.c | 2 +- source/dnode/mnode/impl/src/mnodeTelem.c | 2 +- 24 files changed, 790 insertions(+), 550 deletions(-) create mode 100644 include/dnode/mgmt/dnode.h create mode 100644 source/dnode/mgmt/daemon/CMakeLists.txt rename source/dnode/mgmt/{src/dnode.c => daemon/src/daemon.c} (78%) create mode 100644 source/dnode/mgmt/impl/CMakeLists.txt rename source/dnode/mgmt/{ => impl}/inc/dnodeDnode.h (87%) create mode 100644 source/dnode/mgmt/impl/inc/dnodeInt.h rename source/dnode/mgmt/{ => impl}/inc/dnodeMnode.h (100%) rename source/dnode/mgmt/{ => impl}/inc/dnodeTransport.h (85%) rename source/dnode/mgmt/{ => impl}/inc/dnodeVnodes.h (100%) rename source/dnode/mgmt/{ => impl}/src/dnodeDnode.c (65%) create mode 100644 source/dnode/mgmt/impl/src/dnodeInt.c rename source/dnode/mgmt/{ => impl}/src/dnodeMnode.c (96%) rename source/dnode/mgmt/{ => impl}/src/dnodeTransport.c (97%) rename source/dnode/mgmt/{ => impl}/src/dnodeVnodes.c (99%) delete mode 100644 source/dnode/mgmt/inc/dnodeInt.h delete mode 100644 source/dnode/mgmt/src/dnodeInt.c diff --git a/include/dnode/mgmt/dnode.h b/include/dnode/mgmt/dnode.h new file mode 100644 index 0000000000..5002ee37b0 --- /dev/null +++ b/include/dnode/mgmt/dnode.h @@ -0,0 +1,46 @@ +/* + * 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_DNODE_H_ +#define _TD_DNODE_H_ + +#ifdef __cplusplus +extern "C" { +#endif + +/* ------------------------ TYPES EXPOSED ------------------------ */ +typedef struct SDnode SDnode; + +/* ------------------------ SDnode ------------------------ */ +/** + * @brief Initialize and start the dnode. + * + * @param cfgPath Config file path. + * @return SDnode* The dnode object. + */ +SDnode *dnodeInit(const char *cfgPath); + +/** + * @brief Stop and cleanup dnode. + * + * @param pDnode The dnode object to close. + */ +void dnodeCleanup(SDnode *pDnode); + +#ifdef __cplusplus +} +#endif + +#endif /*_TD_DNODE_H_*/ diff --git a/include/dnode/mnode/mnode.h b/include/dnode/mnode/mnode.h index 1ef3bd579f..0071296bc1 100644 --- a/include/dnode/mnode/mnode.h +++ b/include/dnode/mnode/mnode.h @@ -20,17 +20,16 @@ extern "C" { #endif -typedef enum { MN_MSG_TYPE_WRITE = 1, MN_MSG_TYPE_APPLY, MN_MSG_TYPE_SYNC, MN_MSG_TYPE_READ } EMnMsgType; - +/* ------------------------ TYPES EXPOSED ------------------------ */ +typedef struct SDnode SDnode; +typedef struct SMnode SMnode; typedef struct SMnodeMsg SMnodeMsg; +typedef void (*SendMsgToDnodeFp)(SDnode *pDnode, struct SEpSet *epSet, struct SRpcMsg *rpcMsg); +typedef void (*SendMsgToMnodeFp)(SDnode *pDnode, struct SRpcMsg *rpcMsg); +typedef void (*SendRedirectMsgFp)(SDnode *pDnode, struct SRpcMsg *rpcMsg, bool forShell); +typedef int32_t (*PutMsgToMnodeQFp)(SDnode *pDnode, SMnodeMsg *pMsg); -typedef struct { - int8_t replica; - int8_t selfIndex; - SReplica replicas[TSDB_MAX_REPLICA]; -} SMnodeCfg; - -typedef struct { +typedef struct SMnodeLoad { int64_t numOfDnode; int64_t numOfMnode; int64_t numOfVgroup; @@ -43,38 +42,126 @@ typedef struct { int64_t compStorage; } SMnodeLoad; -typedef struct SMnode SMnode; -typedef struct SServer SServer; - -typedef void (*SendMsgToDnodeFp)(SServer *pServer, struct SEpSet *epSet, struct SRpcMsg *rpcMsg); -typedef void (*SendMsgToMnodeFp)(SServer *pServer, struct SRpcMsg *rpcMsg); -typedef void (*SendRedirectMsgFp)(SServer *pServer, struct SRpcMsg *rpcMsg, bool forShell); -typedef int32_t (*PutMsgToMnodeQFp)(SServer *pServer, SMnodeMsg *pMsg); - typedef struct { int32_t dnodeId; int64_t clusterId; + int8_t replica; + int8_t selfIndex; + SReplica replicas[TSDB_MAX_REPLICA]; + struct SServer *pServer; PutMsgToMnodeQFp putMsgToApplyMsgFp; SendMsgToDnodeFp sendMsgToDnodeFp; SendMsgToMnodeFp sendMsgToMnodeFp; SendRedirectMsgFp sendRedirectMsgFp; -} SMnodePara; +} SMnodeOptions; -SMnode* mnodeCreate(SMnodePara para); -void mnodeCleanup(); +/* ------------------------ SMnode ------------------------ */ +/** + * @brief Open a mnode. + * + * @param path Path of the mnode + * @param pOptions Options of the mnode + * @return SMnode* The mnode object + */ +SMnode *mnodeOpen(const char *path, const SMnodeOptions *pOptions); -int32_t mnodeDeploy(SMnodeCfg *pCfg); -void mnodeUnDeploy(); -int32_t mnodeStart(SMnodeCfg *pCfg); -int32_t mnodeAlter(SMnodeCfg *pCfg); -void mnodeStop(); +/** + * @brief Close a mnode + * + * @param pMnode The mnode object to close + */ +void mnodeClose(SMnode *pMnode); -int32_t mnodeGetLoad(SMnodeLoad *pLoad); -int32_t mnodeRetriveAuth(char *user, char *spi, char *encrypt, char *secret, char *ckey); +/** + * @brief Close a mnode + * + * @param pMnode The mnode object to close + * @param pOptions Options of the mnode + * @return int32_t 0 for success, -1 for failure + */ +int32_t mnodeAlter(SMnode *pMnode, const SMnodeOptions *pOptions); -SMnodeMsg *mnodeInitMsg(SRpcMsg *pRpcMsg); -void mnodeCleanupMsg(SMnodeMsg *pMsg); -void mnodeProcessMsg(SMnodeMsg *pMsg, EMnMsgType msgType); +/** + * @brief Drop a mnode. + * + * @param path Path of the mnode. + */ +void mnodeDestroy(const char *path); + +/** + * @brief Get mnode statistics info + * + * @param pMnode The mnode object + * @param pLoad Statistics of the mnode. + * @return int32_t 0 for success, -1 for failure + */ +int32_t mnodeGetLoad(SMnode *pMnode, SMnodeLoad *pLoad); + +/** + * @brief Get user authentication info + * + * @param pMnode The mnode object + * @param user + * @param spi + * @param encrypt + * @param secret + * @param ckey + * @return int32_t 0 for success, -1 for failure + */ +int32_t mnodeRetriveAuth(SMnode *pMnode, char *user, char *spi, char *encrypt, char *secret, char *ckey); + +/** + * @brief Initialize mnode msg + * + * @param pMnode The mnode object + * @param pMsg The request rpc msg + * @return int32_t The created mnode msg + */ +SMnodeMsg *mnodeInitMsg(SMnode *pMnode, SRpcMsg *pRpcMsg); + +/** + * @brief Cleanup mnode msg + * + * @param pMnode The mnode object + * @param pMsg The request msg + */ +void mnodeCleanupMsg(SMnode *pMnode, SMnodeMsg *pMsg); + +/** + * @brief Process the read request + * + * @param pMnode The mnode object + * @param pMsg The request msg + * @return int32_t 0 for success, -1 for failure + */ +void mnodeProcessReadMsg(SMnode *pMnode, SMnodeMsg *pMsg); + +/** + * @brief Process the write request + * + * @param pMnode The mnode object + * @param pMsg The request msg + * @return int32_t 0 for success, -1 for failure + */ +void mnodeProcessWriteMsg(SMnode *pMnode, SMnodeMsg *pMsg); + +/** + * @brief Process the sync request + * + * @param pMnode The mnode object + * @param pMsg The request msg + * @return int32_t 0 for success, -1 for failure + */ +void mnodeProcessSyncMsg(SMnode *pMnode, SMnodeMsg *pMsg); + +/** + * @brief Process the apply request + * + * @param pMnode The mnode object + * @param pMsg The request msg + * @return int32_t 0 for success, -1 for failure + */ +void mnodeProcessApplyMsg(SMnode *pMnode, SMnodeMsg *pMsg); #ifdef __cplusplus } diff --git a/include/dnode/vnode/vnode.h b/include/dnode/vnode/vnode.h index 419c9dfcfc..36f6a3b6fb 100644 --- a/include/dnode/vnode/vnode.h +++ b/include/dnode/vnode/vnode.h @@ -184,11 +184,11 @@ typedef struct { SRpcMsg rpcMsg[]; } SVnodeMsg; -typedef struct SServer SServer; -typedef void (*SendMsgToDnodeFp)(SServer *pServer, struct SEpSet *epSet, struct SRpcMsg *rpcMsg); -typedef void (*SendMsgToMnodeFp)(SServer *pServer, struct SRpcMsg *rpcMsg); -typedef void (*SendRedirectMsgFp)(SServer *pServer, struct SRpcMsg *rpcMsg, bool forShell); -typedef int32_t (*PutMsgToVnodeQFp)(SServer *pServer, int32_t vgId, SVnodeMsg *pMsg); +typedef struct SDnode SDnode; +typedef void (*SendMsgToDnodeFp)(SDnode *pDnode, struct SEpSet *epSet, struct SRpcMsg *rpcMsg); +typedef void (*SendMsgToMnodeFp)(SDnode *pDnode, struct SRpcMsg *rpcMsg); +typedef void (*SendRedirectMsgFp)(SDnode *pDnode, struct SRpcMsg *rpcMsg, bool forShell); +typedef int32_t (*PutMsgToVnodeQFp)(SDnode *pDnode, int32_t vgId, SVnodeMsg *pMsg); typedef struct { PutMsgToVnodeQFp putMsgToApplyQueueFp; diff --git a/source/dnode/CMakeLists.txt b/source/dnode/CMakeLists.txt index d719a2d106..af132dea80 100644 --- a/source/dnode/CMakeLists.txt +++ b/source/dnode/CMakeLists.txt @@ -1,4 +1,4 @@ add_subdirectory(mnode) add_subdirectory(vnode) add_subdirectory(qnode) -add_subdirectory(mgmt) +add_subdirectory(mgmt) \ No newline at end of file diff --git a/source/dnode/mgmt/CMakeLists.txt b/source/dnode/mgmt/CMakeLists.txt index 194c317991..64e8980219 100644 --- a/source/dnode/mgmt/CMakeLists.txt +++ b/source/dnode/mgmt/CMakeLists.txt @@ -1,16 +1,2 @@ -aux_source_directory(src DNODE_SRC) -add_executable(taosd ${DNODE_SRC}) -target_link_libraries( - taosd - PUBLIC cjson - PUBLIC mnode - PUBLIC vnode - PUBLIC wal - PUBLIC sync - PUBLIC taos -) -target_include_directories( - taosd - PUBLIC "${CMAKE_SOURCE_DIR}/include/dnode" - private "${CMAKE_CURRENT_SOURCE_DIR}/inc" -) +add_subdirectory(daemon) +add_subdirectory(impl) \ No newline at end of file diff --git a/source/dnode/mgmt/daemon/CMakeLists.txt b/source/dnode/mgmt/daemon/CMakeLists.txt new file mode 100644 index 0000000000..f1ce726d85 --- /dev/null +++ b/source/dnode/mgmt/daemon/CMakeLists.txt @@ -0,0 +1,8 @@ +aux_source_directory(src DAEMON_SRC) +add_executable(taosd ${DAEMON_SRC}) +target_link_libraries( + taosd + PUBLIC dnode + PUBLIC util + PUBLIC os +) diff --git a/source/dnode/mgmt/src/dnode.c b/source/dnode/mgmt/daemon/src/daemon.c similarity index 78% rename from source/dnode/mgmt/src/dnode.c rename to source/dnode/mgmt/daemon/src/daemon.c index 1fbeb1e732..720d1589c2 100644 --- a/source/dnode/mgmt/src/dnode.c +++ b/source/dnode/mgmt/daemon/src/daemon.c @@ -14,10 +14,14 @@ */ #define _DEFAULT_SOURCE -#include "dnodeInt.h" +#include "dnode.h" +#include "os.h" +#include "ulog.h" static bool stop = false; + static void sigintHandler(int32_t signum, void *info, void *ctx) { stop = true; } + static void setSignalHandler() { taosSetSignal(SIGTERM, sigintHandler); taosSetSignal(SIGHUP, sigintHandler); @@ -27,20 +31,23 @@ static void setSignalHandler() { } int main(int argc, char const *argv[]) { - setSignalHandler(); + const char *path = "/etc/taos"; - int32_t code = dnodeInit(); - if (code != 0) { - dInfo("Failed to start TDengine, please check the log at:%s", tsLogDir); + SDnode *pDnode = dnodeInit(path); + if (pDnode == NULL) { + uInfo("Failed to start TDengine, please check the log at %s", tsLogDir); exit(EXIT_FAILURE); } + uInfo("Started TDengine service successfully."); + + setSignalHandler(); while (!stop) { taosMsleep(100); } - dInfo("TDengine is shut down!"); - dnodeCleanup(); + uInfo("TDengine is shut down!"); + dnodeCleanup(pDnode); return 0; } diff --git a/source/dnode/mgmt/impl/CMakeLists.txt b/source/dnode/mgmt/impl/CMakeLists.txt new file mode 100644 index 0000000000..b061d75731 --- /dev/null +++ b/source/dnode/mgmt/impl/CMakeLists.txt @@ -0,0 +1,16 @@ +aux_source_directory(src DNODE_SRC) +add_library(dnode ${DNODE_SRC}) +target_link_libraries( + dnode + PUBLIC cjson + PUBLIC mnode + PUBLIC vnode + PUBLIC wal + PUBLIC sync + PUBLIC taos +) +target_include_directories( + dnode + PUBLIC "${CMAKE_SOURCE_DIR}/include/dnode/mgmt" + private "${CMAKE_CURRENT_SOURCE_DIR}/inc" +) diff --git a/source/dnode/mgmt/inc/dnodeDnode.h b/source/dnode/mgmt/impl/inc/dnodeDnode.h similarity index 87% rename from source/dnode/mgmt/inc/dnodeDnode.h rename to source/dnode/mgmt/impl/inc/dnodeDnode.h index 0d1e93e60f..87dc0fdb9b 100644 --- a/source/dnode/mgmt/inc/dnodeDnode.h +++ b/source/dnode/mgmt/impl/inc/dnodeDnode.h @@ -21,8 +21,8 @@ extern "C" { #endif #include "dnodeInt.h" -int32_t dnodeInitDnode(); -void dnodeCleanupDnode(); +int32_t dnodeInitDnode(SDnode *pDnode); +void dnodeCleanupDnode(SDnode *pDnode); void dnodeProcessDnodeMsg(SRpcMsg *pMsg, SEpSet *pEpSet); int32_t dnodeGetDnodeId(); @@ -30,7 +30,7 @@ int64_t dnodeGetClusterId(); void dnodeGetDnodeEp(int32_t dnodeId, char *epstr, char *fqdn, uint16_t *port); void dnodeGetMnodeEpSetForPeer(SEpSet *epSet); void dnodeGetMnodeEpSetForShell(SEpSet *epSet); -void dnodeSendRedirectMsg(SServer *pServer, SRpcMsg *rpcMsg, bool forShell); +void dnodeSendRedirectMsg(SDnode *pDnode, SRpcMsg *rpcMsg, bool forShell); #ifdef __cplusplus } diff --git a/source/dnode/mgmt/impl/inc/dnodeInt.h b/source/dnode/mgmt/impl/inc/dnodeInt.h new file mode 100644 index 0000000000..8944755268 --- /dev/null +++ b/source/dnode/mgmt/impl/inc/dnodeInt.h @@ -0,0 +1,97 @@ +/* + * 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_DNODE_INT_H_ +#define _TD_DNODE_INT_H_ + +#ifdef __cplusplus +extern "C" { +#endif +#include "os.h" +#include "taosmsg.h" +#include "tglobal.h" +#include "thash.h" +#include "tlog.h" +#include "trpc.h" +#include "tthread.h" +#include "ttime.h" + +extern int32_t dDebugFlag; + +#define dFatal(...) { if (dDebugFlag & DEBUG_FATAL) { taosPrintLog("SRV FATAL ", 255, __VA_ARGS__); }} +#define dError(...) { if (dDebugFlag & DEBUG_ERROR) { taosPrintLog("SRV ERROR ", 255, __VA_ARGS__); }} +#define dWarn(...) { if (dDebugFlag & DEBUG_WARN) { taosPrintLog("SRV WARN ", 255, __VA_ARGS__); }} +#define dInfo(...) { if (dDebugFlag & DEBUG_INFO) { taosPrintLog("SRV ", 255, __VA_ARGS__); }} +#define dDebug(...) { if (dDebugFlag & DEBUG_DEBUG) { taosPrintLog("SRV ", dDebugFlag, __VA_ARGS__); }} +#define dTrace(...) { if (dDebugFlag & DEBUG_TRACE) { taosPrintLog("SRV ", dDebugFlag, __VA_ARGS__); }} + +typedef enum { DN_STAT_INIT, DN_STAT_RUNNING, DN_STAT_STOPPED } EStat; +typedef void (*MsgFp)(SRpcMsg *pMsg, SEpSet *pEpSet); + +typedef struct { + char *dnode; + char *mnode; + char *vnodes; +} SDnodeDir; + +typedef struct { + int32_t dnodeId; + int64_t clusterId; + SDnodeEps *dnodeEps; + SHashObj *dnodeHash; + SEpSet mnodeEpSetForShell; + SEpSet mnodeEpSetForPeer; + char *file; + uint32_t rebootTime; + int8_t dropped; + int8_t threadStop; + pthread_t *threadId; + pthread_mutex_t mutex; +} SDnodeDnode; + +typedef struct { +} SDnodeMnode; + +typedef struct { +} SDnodeVnodes; + +typedef struct { + void *peerRpc; + void *shellRpc; + void *clientRpc; +} SDnodeTrans; + +typedef struct SDnode { + EStat stat; + SDnodeDir dir; + SDnodeDnode dnode; + SDnodeVnodes vnodes; + SDnodeMnode mnode; + SDnodeTrans trans; + SStartupMsg startup; +} SDnode; + +EStat dnodeGetStat(SDnode *pDnode); +void dnodeSetStat(SDnode *pDnode, EStat stat); +char *dnodeStatStr(EStat stat); + +void dnodeReportStartup(SDnode *pDnode, char *name, char *desc); +void dnodeGetStartup(SDnode *pDnode, SStartupMsg *pStartup); + +#ifdef __cplusplus +} +#endif + +#endif /*_TD_DNODE_INT_H_*/ \ No newline at end of file diff --git a/source/dnode/mgmt/inc/dnodeMnode.h b/source/dnode/mgmt/impl/inc/dnodeMnode.h similarity index 100% rename from source/dnode/mgmt/inc/dnodeMnode.h rename to source/dnode/mgmt/impl/inc/dnodeMnode.h diff --git a/source/dnode/mgmt/inc/dnodeTransport.h b/source/dnode/mgmt/impl/inc/dnodeTransport.h similarity index 85% rename from source/dnode/mgmt/inc/dnodeTransport.h rename to source/dnode/mgmt/impl/inc/dnodeTransport.h index 4a9518fe09..7d3f4be1ff 100644 --- a/source/dnode/mgmt/inc/dnodeTransport.h +++ b/source/dnode/mgmt/impl/inc/dnodeTransport.h @@ -23,8 +23,8 @@ extern "C" { int32_t dnodeInitTrans(); void dnodeCleanupTrans(); -void dnodeSendMsgToMnode(SServer *pServer, SRpcMsg *rpcMsg); -void dnodeSendMsgToDnode(SServer *pServer, SEpSet *epSet, SRpcMsg *rpcMsg); +void dnodeSendMsgToMnode(SDnode *pDnode, SRpcMsg *rpcMsg); +void dnodeSendMsgToDnode(SDnode *pDnode, SEpSet *epSet, SRpcMsg *rpcMsg); #ifdef __cplusplus } diff --git a/source/dnode/mgmt/inc/dnodeVnodes.h b/source/dnode/mgmt/impl/inc/dnodeVnodes.h similarity index 100% rename from source/dnode/mgmt/inc/dnodeVnodes.h rename to source/dnode/mgmt/impl/inc/dnodeVnodes.h diff --git a/source/dnode/mgmt/src/dnodeDnode.c b/source/dnode/mgmt/impl/src/dnodeDnode.c similarity index 65% rename from source/dnode/mgmt/src/dnodeDnode.c rename to source/dnode/mgmt/impl/src/dnodeDnode.c index 8a326c72d5..ec60116ce6 100644 --- a/source/dnode/mgmt/src/dnodeDnode.c +++ b/source/dnode/mgmt/impl/src/dnodeDnode.c @@ -18,67 +18,49 @@ #include "dnodeTransport.h" #include "dnodeVnodes.h" #include "cJSON.h" -#include "thash.h" -#include "tthread.h" -#include "ttime.h" - -static struct { - int32_t dnodeId; - int64_t clusterId; - SDnodeEps *dnodeEps; - SHashObj *dnodeHash; - SEpSet mnodeEpSetForShell; - SEpSet mnodeEpSetForPeer; - char file[PATH_MAX + 20]; - uint32_t rebootTime; - int8_t dropped; - int8_t threadStop; - pthread_t *threadId; - pthread_mutex_t mutex; -} tsDnode = {0}; int32_t dnodeGetDnodeId() { int32_t dnodeId = 0; - pthread_mutex_lock(&tsDnode.mutex); - dnodeId = tsDnode.dnodeId; - pthread_mutex_unlock(&tsDnode.mutex); + pthread_mutex_lock(&pDnode->mutex); + dnodeId = pDnode->dnodeId; + pthread_mutex_unlock(&pDnode->mutex); return dnodeId; } int64_t dnodeGetClusterId() { int64_t clusterId = 0; - pthread_mutex_lock(&tsDnode.mutex); - clusterId = tsDnode.clusterId; - pthread_mutex_unlock(&tsDnode.mutex); + pthread_mutex_lock(&pDnode->mutex); + clusterId = pDnode->clusterId; + pthread_mutex_unlock(&pDnode->mutex); return clusterId; } void dnodeGetDnodeEp(int32_t dnodeId, char *ep, char *fqdn, uint16_t *port) { - pthread_mutex_lock(&tsDnode.mutex); + pthread_mutex_lock(&pDnode->mutex); - SDnodeEp *pEp = taosHashGet(tsDnode.dnodeHash, &dnodeId, sizeof(int32_t)); + SDnodeEp *pEp = taosHashGet(pDnode->dnodeHash, &dnodeId, sizeof(int32_t)); if (pEp != NULL) { if (port) *port = pEp->dnodePort; if (fqdn) tstrncpy(fqdn, pEp->dnodeFqdn, TSDB_FQDN_LEN); if (ep) snprintf(ep, TSDB_EP_LEN, "%s:%u", pEp->dnodeFqdn, pEp->dnodePort); } - pthread_mutex_unlock(&tsDnode.mutex); + pthread_mutex_unlock(&pDnode->mutex); } void dnodeGetMnodeEpSetForPeer(SEpSet *pEpSet) { - pthread_mutex_lock(&tsDnode.mutex); - *pEpSet = tsDnode.mnodeEpSetForPeer; - pthread_mutex_unlock(&tsDnode.mutex); + pthread_mutex_lock(&pDnode->mutex); + *pEpSet = pDnode->mnodeEpSetForPeer; + pthread_mutex_unlock(&pDnode->mutex); } void dnodeGetMnodeEpSetForShell(SEpSet *pEpSet) { - pthread_mutex_lock(&tsDnode.mutex); - *pEpSet = tsDnode.mnodeEpSetForShell; - pthread_mutex_unlock(&tsDnode.mutex); + pthread_mutex_lock(&pDnode->mutex); + *pEpSet = pDnode->mnodeEpSetForShell; + pthread_mutex_unlock(&pDnode->mutex); } -void dnodeSendRedirectMsg(SServer *pServer, SRpcMsg *pMsg, bool forShell) { +void dnodeSendRedirectMsg(SDnode *pDnode, SRpcMsg *pMsg, bool forShell) { int32_t msgType = pMsg->msgType; SEpSet epSet = {0}; @@ -106,7 +88,7 @@ void dnodeSendRedirectMsg(SServer *pServer, SRpcMsg *pMsg, bool forShell) { rpcSendRedirectRsp(pMsg->handle, &epSet); } -static void dnodeUpdateMnodeEpSet(SEpSet *pEpSet) { +static void dnodeUpdateMnodeEpSet(SDnodeDnode *pDnode, SEpSet *pEpSet) { if (pEpSet == NULL || pEpSet->numOfEps <= 0) { dError("mnode is changed, but content is invalid, discard it"); return; @@ -114,22 +96,22 @@ static void dnodeUpdateMnodeEpSet(SEpSet *pEpSet) { dInfo("mnode is changed, num:%d use:%d", pEpSet->numOfEps, pEpSet->inUse); } - pthread_mutex_lock(&tsDnode.mutex); + pthread_mutex_lock(&pDnode->mutex); - tsDnode.mnodeEpSetForPeer = *pEpSet; + pDnode->mnodeEpSetForPeer = *pEpSet; for (int32_t i = 0; i < pEpSet->numOfEps; ++i) { pEpSet->port[i] -= TSDB_PORT_DNODEDNODE; dInfo("mnode index:%d %s:%u", i, pEpSet->fqdn[i], pEpSet->port[i]); } - tsDnode.mnodeEpSetForShell = *pEpSet; + pDnode->mnodeEpSetForShell = *pEpSet; - pthread_mutex_unlock(&tsDnode.mutex); + pthread_mutex_unlock(&pDnode->mutex); } static void dnodePrintDnodes() { - dDebug("print dnode endpoint list, num:%d", tsDnode.dnodeEps->dnodeNum); - for (int32_t i = 0; i < tsDnode.dnodeEps->dnodeNum; i++) { - SDnodeEp *ep = &tsDnode.dnodeEps->dnodeEps[i]; + dDebug("print dnode endpoint list, num:%d", pDnode->dnodeEps->dnodeNum); + for (int32_t i = 0; i < pDnode->dnodeEps->dnodeNum; i++) { + SDnodeEp *ep = &pDnode->dnodeEps->dnodeEps[i]; dDebug("dnode:%d, fqdn:%s port:%u isMnode:%d", ep->dnodeId, ep->dnodeFqdn, ep->dnodePort, ep->isMnode); } } @@ -138,36 +120,36 @@ static void dnodeResetDnodes(SDnodeEps *pEps) { assert(pEps != NULL); int32_t size = sizeof(SDnodeEps) + pEps->dnodeNum * sizeof(SDnodeEp); - if (pEps->dnodeNum > tsDnode.dnodeEps->dnodeNum) { + if (pEps->dnodeNum > pDnode->dnodeEps->dnodeNum) { SDnodeEps *tmp = calloc(1, size); if (tmp == NULL) return; - tfree(tsDnode.dnodeEps); - tsDnode.dnodeEps = tmp; + tfree(pDnode->dnodeEps); + pDnode->dnodeEps = tmp; } - if (tsDnode.dnodeEps != pEps) { - memcpy(tsDnode.dnodeEps, pEps, size); + if (pDnode->dnodeEps != pEps) { + memcpy(pDnode->dnodeEps, pEps, size); } - tsDnode.mnodeEpSetForPeer.inUse = 0; - tsDnode.mnodeEpSetForShell.inUse = 0; + pDnode->mnodeEpSetForPeer.inUse = 0; + pDnode->mnodeEpSetForShell.inUse = 0; int32_t mIndex = 0; - for (int32_t i = 0; i < tsDnode.dnodeEps->dnodeNum; i++) { - SDnodeEp *ep = &tsDnode.dnodeEps->dnodeEps[i]; + for (int32_t i = 0; i < pDnode->dnodeEps->dnodeNum; i++) { + SDnodeEp *ep = &pDnode->dnodeEps->dnodeEps[i]; if (!ep->isMnode) continue; if (mIndex >= TSDB_MAX_REPLICA) continue; - strcpy(tsDnode.mnodeEpSetForShell.fqdn[mIndex], ep->dnodeFqdn); - strcpy(tsDnode.mnodeEpSetForPeer.fqdn[mIndex], ep->dnodeFqdn); - tsDnode.mnodeEpSetForShell.port[mIndex] = ep->dnodePort; - tsDnode.mnodeEpSetForShell.port[mIndex] = ep->dnodePort + tsDnodeDnodePort; + strcpy(pDnode->mnodeEpSetForShell.fqdn[mIndex], ep->dnodeFqdn); + strcpy(pDnode->mnodeEpSetForPeer.fqdn[mIndex], ep->dnodeFqdn); + pDnode->mnodeEpSetForShell.port[mIndex] = ep->dnodePort; + pDnode->mnodeEpSetForShell.port[mIndex] = ep->dnodePort + tsDnodeDnodePort; mIndex++; } - for (int32_t i = 0; i < tsDnode.dnodeEps->dnodeNum; ++i) { - SDnodeEp *ep = &tsDnode.dnodeEps->dnodeEps[i]; - taosHashPut(tsDnode.dnodeHash, &ep->dnodeId, sizeof(int32_t), ep, sizeof(SDnodeEp)); + for (int32_t i = 0; i < pDnode->dnodeEps->dnodeNum; ++i) { + SDnodeEp *ep = &pDnode->dnodeEps->dnodeEps[i]; + taosHashPut(pDnode->dnodeHash, &ep->dnodeId, sizeof(int32_t), ep, sizeof(SDnodeEp)); } dnodePrintDnodes(); @@ -175,16 +157,16 @@ static void dnodeResetDnodes(SDnodeEps *pEps) { static bool dnodeIsEpChanged(int32_t dnodeId, char *epStr) { bool changed = false; - pthread_mutex_lock(&tsDnode.mutex); + pthread_mutex_lock(&pDnode->mutex); - SDnodeEp *pEp = taosHashGet(tsDnode.dnodeHash, &dnodeId, sizeof(int32_t)); + SDnodeEp *pEp = taosHashGet(pDnode->dnodeHash, &dnodeId, sizeof(int32_t)); if (pEp != NULL) { char epSaved[TSDB_EP_LEN + 1]; snprintf(epSaved, TSDB_EP_LEN, "%s:%u", pEp->dnodeFqdn, pEp->dnodePort); changed = strcmp(epStr, epSaved) != 0; } - pthread_mutex_unlock(&tsDnode.mutex); + pthread_mutex_unlock(&pDnode->mutex); return changed; } @@ -195,101 +177,101 @@ static int32_t dnodeReadDnodes() { cJSON *root = NULL; FILE *fp = NULL; - fp = fopen(tsDnode.file, "r"); + fp = fopen(pDnode->file, "r"); if (!fp) { - dDebug("file %s not exist", tsDnode.file); + dDebug("file %s not exist", pDnode->file); goto PRASE_DNODE_OVER; } len = (int32_t)fread(content, 1, maxLen, fp); if (len <= 0) { - dError("failed to read %s since content is null", tsDnode.file); + dError("failed to read %s since content is null", pDnode->file); goto PRASE_DNODE_OVER; } content[len] = 0; root = cJSON_Parse(content); if (root == NULL) { - dError("failed to read %s since invalid json format", tsDnode.file); + dError("failed to read %s since invalid json format", pDnode->file); goto PRASE_DNODE_OVER; } cJSON *dnodeId = cJSON_GetObjectItem(root, "dnodeId"); if (!dnodeId || dnodeId->type != cJSON_String) { - dError("failed to read %s since dnodeId not found", tsDnode.file); + dError("failed to read %s since dnodeId not found", pDnode->file); goto PRASE_DNODE_OVER; } - tsDnode.dnodeId = atoi(dnodeId->valuestring); + pDnode->dnodeId = atoi(dnodeId->valuestring); cJSON *clusterId = cJSON_GetObjectItem(root, "clusterId"); if (!clusterId || clusterId->type != cJSON_String) { - dError("failed to read %s since clusterId not found", tsDnode.file); + dError("failed to read %s since clusterId not found", pDnode->file); goto PRASE_DNODE_OVER; } - tsDnode.clusterId = atoll(clusterId->valuestring); + pDnode->clusterId = atoll(clusterId->valuestring); cJSON *dropped = cJSON_GetObjectItem(root, "dropped"); if (!dropped || dropped->type != cJSON_String) { - dError("failed to read %s since dropped not found", tsDnode.file); + dError("failed to read %s since dropped not found", pDnode->file); goto PRASE_DNODE_OVER; } - tsDnode.dropped = atoi(dropped->valuestring); + pDnode->dropped = atoi(dropped->valuestring); cJSON *dnodeInfos = cJSON_GetObjectItem(root, "dnodeInfos"); if (!dnodeInfos || dnodeInfos->type != cJSON_Array) { - dError("failed to read %s since dnodeInfos not found", tsDnode.file); + dError("failed to read %s since dnodeInfos not found", pDnode->file); goto PRASE_DNODE_OVER; } int32_t dnodeInfosSize = cJSON_GetArraySize(dnodeInfos); if (dnodeInfosSize <= 0) { - dError("failed to read %s since dnodeInfos size:%d invalid", tsDnode.file, dnodeInfosSize); + dError("failed to read %s since dnodeInfos size:%d invalid", pDnode->file, dnodeInfosSize); goto PRASE_DNODE_OVER; } - tsDnode.dnodeEps = calloc(1, dnodeInfosSize * sizeof(SDnodeEp) + sizeof(SDnodeEps)); - if (tsDnode.dnodeEps == NULL) { + pDnode->dnodeEps = calloc(1, dnodeInfosSize * sizeof(SDnodeEp) + sizeof(SDnodeEps)); + if (pDnode->dnodeEps == NULL) { dError("failed to calloc dnodeEpList since %s", strerror(errno)); goto PRASE_DNODE_OVER; } - tsDnode.dnodeEps->dnodeNum = dnodeInfosSize; + pDnode->dnodeEps->dnodeNum = dnodeInfosSize; for (int32_t i = 0; i < dnodeInfosSize; ++i) { cJSON *dnodeInfo = cJSON_GetArrayItem(dnodeInfos, i); if (dnodeInfo == NULL) break; - SDnodeEp *pEp = &tsDnode.dnodeEps->dnodeEps[i]; + SDnodeEp *pEp = &pDnode->dnodeEps->dnodeEps[i]; cJSON *dnodeId = cJSON_GetObjectItem(dnodeInfo, "dnodeId"); if (!dnodeId || dnodeId->type != cJSON_String) { - dError("failed to read %s, dnodeId not found", tsDnode.file); + dError("failed to read %s, dnodeId not found", pDnode->file); goto PRASE_DNODE_OVER; } pEp->dnodeId = atoi(dnodeId->valuestring); cJSON *isMnode = cJSON_GetObjectItem(dnodeInfo, "isMnode"); if (!isMnode || isMnode->type != cJSON_String) { - dError("failed to read %s, isMnode not found", tsDnode.file); + dError("failed to read %s, isMnode not found", pDnode->file); goto PRASE_DNODE_OVER; } pEp->isMnode = atoi(isMnode->valuestring); cJSON *dnodeFqdn = cJSON_GetObjectItem(dnodeInfo, "dnodeFqdn"); if (!dnodeFqdn || dnodeFqdn->type != cJSON_String || dnodeFqdn->valuestring == NULL) { - dError("failed to read %s, dnodeFqdn not found", tsDnode.file); + dError("failed to read %s, dnodeFqdn not found", pDnode->file); goto PRASE_DNODE_OVER; } tstrncpy(pEp->dnodeFqdn, dnodeFqdn->valuestring, TSDB_FQDN_LEN); cJSON *dnodePort = cJSON_GetObjectItem(dnodeInfo, "dnodePort"); if (!dnodePort || dnodePort->type != cJSON_String) { - dError("failed to read %s, dnodePort not found", tsDnode.file); + dError("failed to read %s, dnodePort not found", pDnode->file); goto PRASE_DNODE_OVER; } pEp->dnodePort = atoi(dnodePort->valuestring); } - dInfo("succcessed to read file %s", tsDnode.file); + dInfo("succcessed to read file %s", pDnode->file); dnodePrintDnodes(); PRASE_DNODE_OVER: @@ -297,28 +279,28 @@ PRASE_DNODE_OVER: if (root != NULL) cJSON_Delete(root); if (fp != NULL) fclose(fp); - if (dnodeIsEpChanged(tsDnode.dnodeId, tsLocalEp)) { - dError("localEp %s different with %s and need reconfigured", tsLocalEp, tsDnode.file); + if (dnodeIsEpChanged(pDnode->dnodeId, tsLocalEp)) { + dError("localEp %s different with %s and need reconfigured", tsLocalEp, pDnode->file); return -1; } - if (tsDnode.dnodeEps == NULL) { - tsDnode.dnodeEps = calloc(1, sizeof(SDnodeEps) + sizeof(SDnodeEp)); - tsDnode.dnodeEps->dnodeNum = 1; - tsDnode.dnodeEps->dnodeEps[0].dnodePort = tsServerPort; - tstrncpy(tsDnode.dnodeEps->dnodeEps[0].dnodeFqdn, tsLocalFqdn, TSDB_FQDN_LEN); + if (pDnode->dnodeEps == NULL) { + pDnode->dnodeEps = calloc(1, sizeof(SDnodeEps) + sizeof(SDnodeEp)); + pDnode->dnodeEps->dnodeNum = 1; + pDnode->dnodeEps->dnodeEps[0].dnodePort = tsServerPort; + tstrncpy(pDnode->dnodeEps->dnodeEps[0].dnodeFqdn, tsLocalFqdn, TSDB_FQDN_LEN); } - dnodeResetDnodes(tsDnode.dnodeEps); + dnodeResetDnodes(pDnode->dnodeEps); terrno = 0; return 0; } static int32_t dnodeWriteDnodes() { - FILE *fp = fopen(tsDnode.file, "w"); + FILE *fp = fopen(pDnode->file, "w"); if (!fp) { - dError("failed to write %s since %s", tsDnode.file, strerror(errno)); + dError("failed to write %s since %s", pDnode->file, strerror(errno)); return -1; } @@ -327,17 +309,17 @@ static int32_t dnodeWriteDnodes() { char *content = calloc(1, maxLen + 1); len += snprintf(content + len, maxLen - len, "{\n"); - len += snprintf(content + len, maxLen - len, " \"dnodeId\": \"%d\",\n", tsDnode.dnodeId); - len += snprintf(content + len, maxLen - len, " \"clusterId\": \"%" PRId64 "\",\n", tsDnode.clusterId); - len += snprintf(content + len, maxLen - len, " \"dropped\": \"%d\",\n", tsDnode.dropped); + len += snprintf(content + len, maxLen - len, " \"dnodeId\": \"%d\",\n", pDnode->dnodeId); + len += snprintf(content + len, maxLen - len, " \"clusterId\": \"%" PRId64 "\",\n", pDnode->clusterId); + len += snprintf(content + len, maxLen - len, " \"dropped\": \"%d\",\n", pDnode->dropped); len += snprintf(content + len, maxLen - len, " \"dnodeInfos\": [{\n"); - for (int32_t i = 0; i < tsDnode.dnodeEps->dnodeNum; ++i) { - SDnodeEp *ep = &tsDnode.dnodeEps->dnodeEps[i]; + for (int32_t i = 0; i < pDnode->dnodeEps->dnodeNum; ++i) { + SDnodeEp *ep = &pDnode->dnodeEps->dnodeEps[i]; len += snprintf(content + len, maxLen - len, " \"dnodeId\": \"%d\",\n", ep->dnodeId); len += snprintf(content + len, maxLen - len, " \"isMnode\": \"%d\",\n", ep->isMnode); len += snprintf(content + len, maxLen - len, " \"dnodeFqdn\": \"%s\",\n", ep->dnodeFqdn); len += snprintf(content + len, maxLen - len, " \"dnodePort\": \"%u\"\n", ep->dnodePort); - if (i < tsDnode.dnodeEps->dnodeNum - 1) { + if (i < pDnode->dnodeEps->dnodeNum - 1) { len += snprintf(content + len, maxLen - len, " },{\n"); } else { len += snprintf(content + len, maxLen - len, " }]\n"); @@ -351,7 +333,7 @@ static int32_t dnodeWriteDnodes() { free(content); terrno = 0; - dInfo("successed to write %s", tsDnode.file); + dInfo("successed to write %s", pDnode->file); return 0; } @@ -367,7 +349,7 @@ static void dnodeSendStatusMsg() { pStatus->sversion = htonl(tsVersion); pStatus->dnodeId = htonl(dnodeGetDnodeId()); pStatus->clusterId = htobe64(dnodeGetClusterId()); - pStatus->rebootTime = htonl(tsDnode.rebootTime); + pStatus->rebootTime = htonl(pDnode->rebootTime); pStatus->numOfCores = htonl(tsNumOfCores); tstrncpy(pStatus->dnodeEp, tsLocalEp, TSDB_EP_LEN); @@ -387,37 +369,37 @@ static void dnodeSendStatusMsg() { } static void dnodeUpdateCfg(SDnodeCfg *pCfg) { - if (tsDnode.dnodeId == 0) return; - if (tsDnode.dropped) return; + if (pDnode->dnodeId == 0) return; + if (pDnode->dropped) return; - pthread_mutex_lock(&tsDnode.mutex); + pthread_mutex_lock(&pDnode->mutex); - tsDnode.dnodeId = pCfg->dnodeId; - tsDnode.clusterId = pCfg->clusterId; - tsDnode.dropped = pCfg->dropped; + pDnode->dnodeId = pCfg->dnodeId; + pDnode->clusterId = pCfg->clusterId; + pDnode->dropped = pCfg->dropped; dInfo("dnodeId is set to %d, clusterId is set to %" PRId64, pCfg->dnodeId, pCfg->clusterId); dnodeWriteDnodes(); - pthread_mutex_unlock(&tsDnode.mutex); + pthread_mutex_unlock(&pDnode->mutex); } static void dnodeUpdateDnodeEps(SDnodeEps *pEps) { if (pEps == NULL || pEps->dnodeNum <= 0) return; - pthread_mutex_lock(&tsDnode.mutex); + pthread_mutex_lock(&pDnode->mutex); - if (pEps->dnodeNum != tsDnode.dnodeEps->dnodeNum) { + if (pEps->dnodeNum != pDnode->dnodeEps->dnodeNum) { dnodeResetDnodes(pEps); dnodeWriteDnodes(); } else { int32_t size = pEps->dnodeNum * sizeof(SDnodeEp) + sizeof(SDnodeEps); - if (memcmp(tsDnode.dnodeEps, pEps, size) != 0) { + if (memcmp(pDnode->dnodeEps, pEps, size) != 0) { dnodeResetDnodes(pEps); dnodeWriteDnodes(); } } - pthread_mutex_unlock(&tsDnode.mutex); + pthread_mutex_unlock(&pDnode->mutex); } static void dnodeProcessStatusRsp(SRpcMsg *pMsg) { @@ -455,7 +437,7 @@ static void dnodeProcessStartupReq(SRpcMsg *pMsg) { dInfo("startup msg is received, cont:%s", (char *)pMsg->pCont); SStartupMsg *pStartup = rpcMallocCont(sizeof(SStartupMsg)); - dnodeGetStartup(pStartup); + dnodeGetStartup(NULL, pStartup); dInfo("startup msg is sent, step:%s desc:%s finished:%d", pStartup->name, pStartup->desc, pStartup->finished); @@ -467,8 +449,8 @@ static void dnodeProcessStartupReq(SRpcMsg *pMsg) { static void *dnodeThreadRoutine(void *param) { int32_t ms = tsStatusInterval * 1000; - while (!tsDnode.threadStop) { - if (dnodeGetRunStat() != DN_RUN_STAT_RUNNING) { + while (!pDnode->threadStop) { + if (dnodeGetStat() != DN_STAT_RUNNING) { continue; } else { dnodeSendStatusMsg(); @@ -477,31 +459,41 @@ static void *dnodeThreadRoutine(void *param) { } } -int32_t dnodeInitDnode() { - tsDnode.dnodeId = 0; - tsDnode.clusterId = 0; - tsDnode.dnodeEps = NULL; - snprintf(tsDnode.file, sizeof(tsDnode.file), "%s/dnode.json", tsDnodeDir); - tsDnode.rebootTime = taosGetTimestampSec(); - tsDnode.dropped = 0; - pthread_mutex_init(&tsDnode.mutex, NULL); - tsDnode.threadStop = false; +int32_t dnodeInitDnode(SDnode *pServer) { + SDnodeDnode *pDnode = &pServer->dnode; - tsDnode.dnodeHash = taosHashInit(4, taosGetDefaultHashFunction(TSDB_DATA_TYPE_INT), true, HASH_ENTRY_LOCK); - if (tsDnode.dnodeHash == NULL) { + char path[PATH_MAX]; + snprintf(path, PATH_MAX, "%s/dnode.json", pServer->dir.dnode); + pDnode->file = strdup(path); + if (pDnode->file == NULL) { + terrno = TSDB_CODE_OUT_OF_MEMORY; + return -1; + } + + pDnode->dnodeId = 0; + pDnode->clusterId = 0; + pDnode->dnodeEps = NULL; + + pDnode->rebootTime = taosGetTimestampSec(); + pDnode->dropped = 0; + pthread_mutex_init(&pDnode->mutex, NULL); + pDnode->threadStop = false; + + pDnode->dnodeHash = taosHashInit(4, taosGetDefaultHashFunction(TSDB_DATA_TYPE_INT), true, HASH_ENTRY_LOCK); + if (pDnode->dnodeHash == NULL) { dError("failed to init dnode hash"); return TSDB_CODE_DND_OUT_OF_MEMORY; } - tsDnode.threadId = taosCreateThread(dnodeThreadRoutine, NULL); - if (tsDnode.threadId == NULL) { + pDnode->threadId = taosCreateThread(dnodeThreadRoutine, NULL); + if (pDnode->threadId == NULL) { dError("failed to init dnode thread"); return TSDB_CODE_DND_OUT_OF_MEMORY; } int32_t code = dnodeReadDnodes(); if (code != 0) { - dError("failed to read file:%s since %s", tsDnode.file, tstrerror(code)); + dError("failed to read file:%s since %s", pDnode->file, tstrerror(code)); return code; } @@ -509,36 +501,38 @@ int32_t dnodeInitDnode() { return 0; } -void dnodeCleanupDnode() { - if (tsDnode.threadId != NULL) { - tsDnode.threadStop = true; - taosDestoryThread(tsDnode.threadId); - tsDnode.threadId = NULL; +void dnodeCleanupDnode(SDnode *pServer) { + SDnodeDnode *pDnode = &pServer->dnode; + + if (pDnode->threadId != NULL) { + pDnode->threadStop = true; + taosDestoryThread(pDnode->threadId); + pDnode->threadId = NULL; } - pthread_mutex_lock(&tsDnode.mutex); + pthread_mutex_lock(&pDnode->mutex); - if (tsDnode.dnodeEps != NULL) { - free(tsDnode.dnodeEps); - tsDnode.dnodeEps = NULL; + if (pDnode->dnodeEps != NULL) { + free(pDnode->dnodeEps); + pDnode->dnodeEps = NULL; } - if (tsDnode.dnodeHash) { - taosHashCleanup(tsDnode.dnodeHash); - tsDnode.dnodeHash = NULL; + if (pDnode->dnodeHash) { + taosHashCleanup(pDnode->dnodeHash); + pDnode->dnodeHash = NULL; } - pthread_mutex_unlock(&tsDnode.mutex); - pthread_mutex_destroy(&tsDnode.mutex); + pthread_mutex_unlock(&pDnode->mutex); + pthread_mutex_destroy(&pDnode->mutex); dInfo("dnode-dnode is cleaned up"); } -void dnodeProcessDnodeMsg(SRpcMsg *pMsg, SEpSet *pEpSet) { +void dnodeProcessDnodeMsg(SDnode *pDnode, SRpcMsg *pMsg, SEpSet *pEpSet) { int32_t msgType = pMsg->msgType; if (msgType == TSDB_MSG_TYPE_STATUS_RSP && pEpSet) { - dnodeUpdateMnodeEpSet(pEpSet); + dnodeUpdateMnodeEpSet(&pDnode->dnode, pEpSet); } switch (msgType) { diff --git a/source/dnode/mgmt/impl/src/dnodeInt.c b/source/dnode/mgmt/impl/src/dnodeInt.c new file mode 100644 index 0000000000..8641a54def --- /dev/null +++ b/source/dnode/mgmt/impl/src/dnodeInt.c @@ -0,0 +1,260 @@ +/* + * Copyright (c) 2019 TAOS Data, Inc. + * + * This program is free software: you can use, redistribute, and/or modify + * it under the terms of the GNU Affero General Public License, version 3 + * or later ("AGPL"), as published by the Free Software Foundation. + * + * This program is distributed in the hope that it will be useful, but WITHOUT + * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or + * FITNESS FOR A PARTICULAR PURPOSE. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + */ + +#define _DEFAULT_SOURCE +#include "dnodeDnode.h" +#include "dnodeMnode.h" +#include "dnodeTransport.h" +#include "dnodeVnodes.h" +#include "sync.h" +#include "tcache.h" +#include "tconfig.h" +#include "tnote.h" +#include "tstep.h" +#include "wal.h" + +EStat dnodeGetStat(SDnode *pDnode) { return pDnode->stat; } + +void dnodeSetStat(SDnode *pDnode, EStat stat) { + dDebug("dnode stat set from %s to %s", dnodeStatStr(pDnode->stat), dnodeStatStr(stat)); + pDnode->stat = stat; +} + +char *dnodeStatStr(EStat stat) { + switch (stat) { + case DN_STAT_INIT: + return "init"; + case DN_STAT_RUNNING: + return "running"; + case DN_STAT_STOPPED: + return "stopped"; + default: + return "unknown"; + } +} + +void dnodeReportStartup(SDnode *pDnode, char *name, char *desc) { + SStartupMsg *pStartup = &pDnode->startup; + tstrncpy(pStartup->name, name, strlen(pStartup->name)); + tstrncpy(pStartup->desc, desc, strlen(pStartup->desc)); + pStartup->finished = 0; +} + +void dnodeGetStartup(SDnode *pDnode, SStartupMsg *pStartup) { + memcpy(pStartup, &pDnode->startup, sizeof(SStartupMsg); + pStartup->finished = (dnodeGetStat(pDnode) == DN_STAT_RUNNING); +} + +static int32_t dnodeCheckRunning(char *dataDir) { + char filepath[PATH_MAX] = {0}; + snprintf(filepath, sizeof(filepath), "%s/.running", dataDir); + + FileFd fd = taosOpenFileCreateWriteTrunc(filepath); + if (fd < 0) { + dError("failed to open lock file:%s since %s, quit", filepath, strerror(errno)); + terrno = TAOS_SYSTEM_ERROR(errno); + return -1; + } + + int32_t ret = taosLockFile(fd); + if (ret != 0) { + dError("failed to lock file:%s since %s, quit", filepath, strerror(errno)); + terrno = TAOS_SYSTEM_ERROR(errno); + taosCloseFile(fd); + return -1; + } + + return 0; +} + +static int32_t dnodeInitDisk(SDnode *pDnode, char *dataDir) { + char path[PATH_MAX]; + snprintf(path, PATH_MAX, "%s/mnode", dataDir); + pDnode->dir.mnode = strdup(path); + + sprintf(path, PATH_MAX, "%s/vnode", dataDir); + pDnode->dir.vnodes = strdup(path); + + sprintf(path, PATH_MAX, "%s/dnode", dataDir); + pDnode->dir.dnode = strdup(path); + + if (pDnode->dir.mnode == NULL || pDnode->dir.vnodes == NULL || pDnode->dir.dnode == NULL) { + dError("failed to malloc dir object"); + terrno = TSDB_CODE_OUT_OF_MEMORY; + return -1; + } + + if (!taosMkDir(pDnode->dir.dnode)) { + dError("failed to create dir:%s since %s", pDnode->dir.dnode, strerror(errno)); + terrno = TAOS_SYSTEM_ERROR(errno); + return -1; + } + + if (!taosMkDir(pDnode->dir.mnode)) { + dError("failed to create dir:%s since %s", pDnode->dir.mnode, strerror(errno)); + terrno = TAOS_SYSTEM_ERROR(errno); + return -1; + } + + if (!taosMkDir(pDnode->dir.vnodes)) { + dError("failed to create dir:%s since %s", pDnode->dir.vnodes, strerror(errno)); + terrno = TAOS_SYSTEM_ERROR(errno); + return -1; + } + + if (dnodeCheckRunning(dataDir) != 0) { + return -1; + } + + return 0; +} + +static int32_t dnodeInitEnv(SDnode *pDnode, const char *cfgPath) { + taosIgnSIGPIPE(); + taosBlockSIGPIPE(); + taosResolveCRC(); + taosInitGlobalCfg(); + taosReadGlobalLogCfg(); + taosSetCoreDump(tsEnableCoreFile); + + if (!taosMkDir(tsLogDir)) { + printf("failed to create dir: %s, reason: %s\n", tsLogDir, strerror(errno)); + return -1; + } + + char temp[TSDB_FILENAME_LEN]; + sprintf(temp, "%s/taosdlog", tsLogDir); + if (taosInitLog(temp, tsNumOfLogLines, 1) < 0) { + dError("failed to init log file\n"); + return -1; + } + + if (!taosReadGlobalCfg()) { + taosPrintGlobalCfg(); + dError("TDengine read global config failed"); + return -1; + } + + taosInitNotes(); + + if (taosCheckGlobalCfg() != 0) { + dError("TDengine check global config failed"); + return -1; + } + + if (dnodeInitDisk(pDnode, tsDataDir) != 0) { + dError("TDengine failed to init directory"); + return -1; + } + + return 0; +} + +static void dnodeCleanupEnv(SDnode *pDnode) { + if (pDnode->dir.mnode != NULL) { + tfree(pDnode->dir.mnode); + } + + if (pDnode->dir.vnodes != NULL) { + tfree(pDnode->dir.vnodes); + } + + if (pDnode->dir.dnode != NULL) { + tfree(pDnode->dir.dnode); + } + + taosCloseLog(); + taosStopCacheRefreshWorker(); +} + +SDnode *dnodeInit(const char *cfgPath) { + SDnode *pDnode = calloc(1, sizeof(pDnode)); + if (pDnode == NULL) { + dError("failed to create dnode object"); + terrno = TSDB_CODE_OUT_OF_MEMORY; + return NULL; + } + + dInfo("start to initialize TDengine"); + dnodeSetStat(pDnode, DN_STAT_INIT); + + if (dnodeInitEnv(pDnode, cfgPath) != 0) { + dError("failed to init env"); + dnodeCleanup(pDnode); + return NULL; + } + + if (rpcInit() != 0) { + dError("failed to init rpc env"); + dnodeCleanup(pDnode); + return NULL; + } + + if (walInit() != 0) { + dError("failed to init wal env"); + dnodeCleanup(pDnode); + return NULL; + } + + if (dnodeInitDnode(pDnode) != 0) { + dError("failed to init dnode"); + dnodeCleanup(pDnode); + return NULL; + } + + if (dnodeInitVnodes(pDnode) != 0) { + dError("failed to init vnodes"); + dnodeCleanup(pDnode); + return NULL; + } + + if (dnodeInitMnode(pDnode) != 0) { + dError("failed to init mnode"); + dnodeCleanup(pDnode); + return NULL; + } + + if (dnodeInitTrans(pDnode) != 0) { + dError("failed to init transport"); + dnodeCleanup(pDnode); + return NULL; + } + + dnodeSetStat(pDnode, DN_STAT_RUNNING); + dnodeReportStartup(pDnode, "TDengine", "initialized successfully"); + dInfo("TDengine is initialized successfully"); + + return 0; +} + +void dnodeCleanup(SDnode *pDnode) { + if (dnodeGetStat(pDnode) == DN_STAT_STOPPED) { + dError("dnode is shutting down"); + return; + } + + dInfo("start to cleanup TDengine"); + dnodeSetStat(pDnode, DN_STAT_STOPPED); + dnodeCleanupTrans(pDnode); + dnodeCleanupMnode(pDnode); + dnodeCleanupVnodes(pDnode); + dnodeCleanupDnode(pDnode); + walCleanUp(); + rpcCleanup(); + + dInfo("TDengine is cleaned up successfully"); + dnodeCleanupEnv(pDnode); + free(pDnode); +} diff --git a/source/dnode/mgmt/src/dnodeMnode.c b/source/dnode/mgmt/impl/src/dnodeMnode.c similarity index 96% rename from source/dnode/mgmt/src/dnodeMnode.c rename to source/dnode/mgmt/impl/src/dnodeMnode.c index 232af96897..06b28aeea9 100644 --- a/source/dnode/mgmt/src/dnodeMnode.c +++ b/source/dnode/mgmt/impl/src/dnodeMnode.c @@ -51,19 +51,24 @@ static void dnodeFreeMnodeApplyQueue(); static int32_t dnodeAllocMnodeSyncQueue(); static void dnodeFreeMnodeSyncQueue(); -static int32_t dnodeAcquireMnode() { +static SMnode *dnodeAcquireMnode() { + SMnode *pMnode = NULL; taosRLockLatch(&tsMnode.latch); - int32_t code = tsMnode.deployed ? 0 : TSDB_CODE_DND_MNODE_NOT_DEPLOYED; - if (code == 0) { + if (tsMnode.deployed) { atomic_add_fetch_32(&tsMnode.refCount, 1); + pMnode = tsMnode.pMnode; } taosRUnLockLatch(&tsMnode.latch); - return code; + return pMnode; } -static void dnodeReleaseMnode() { atomic_sub_fetch_32(&tsMnode.refCount, 1); } +static void dnodeReleaseMnode(SMnode *pMnode) { + taosRLockLatch(&tsMnode.latch); + atomic_sub_fetch_32(&tsMnode.refCount, 1); + taosRUnLockLatch(&tsMnode.latch); +} static int32_t dnodeReadMnodeFile() { int32_t code = TSDB_CODE_DND_READ_MNODE_FILE_ERROR; @@ -503,12 +508,12 @@ static void dnodeCleanupMnodeSyncWorker() { tWorkerCleanup(&tsMnode.syncPool); } static int32_t dnodeInitMnodeModule() { taosInitRWLatch(&tsMnode.latch); - SMnodePara para; + SMnodeOptions para; para.dnodeId = dnodeGetDnodeId(); para.clusterId = dnodeGetClusterId(); para.sendMsgToDnodeFp = dnodeSendMsgToDnode; para.sendMsgToMnodeFp = dnodeSendMsgToMnode; - para.sendMsgToMnodeFp = dnodeSendRedirectMsg; + para.sendRedirectMsgFp = dnodeSendRedirectMsg; tsMnode.pMnode = mnodeCreate(para); if (tsMnode.pMnode != NULL) { @@ -517,7 +522,7 @@ static int32_t dnodeInitMnodeModule() { return 0; } -static void dnodeCleanupMnodeModule() { mnodeCleanup(); } +static void dnodeCleanupMnodeModule() { mnodeDrop(NULL); } static bool dnodeNeedDeployMnode() { if (dnodeGetDnodeId() > 0) return false; @@ -590,13 +595,14 @@ void dnodeCleanupMnode() { } int32_t dnodeGetUserAuthFromMnode(char *user, char *spi, char *encrypt, char *secret, char *ckey) { - int32_t code = dnodeAcquireMnode(); - if (code != 0) { + SMnode *pMnode = dnodeAcquireMnode(); + if (pMnode == NULL) { dTrace("failed to get user auth since mnode not deployed"); - return code; + terrno = TSDB_CODE_DND_MNODE_NOT_DEPLOYED; + return -1; } - code = mnodeRetriveAuth(user, spi, encrypt, secret, ckey); - dnodeReleaseMnode(); + int32_t code = mnodeRetriveAuth(pMnode, user, spi, encrypt, secret, ckey); + dnodeReleaseMnode(pMnode); return code; } \ No newline at end of file diff --git a/source/dnode/mgmt/src/dnodeTransport.c b/source/dnode/mgmt/impl/src/dnodeTransport.c similarity index 97% rename from source/dnode/mgmt/src/dnodeTransport.c rename to source/dnode/mgmt/impl/src/dnodeTransport.c index b3263aadca..c1e8955625 100644 --- a/source/dnode/mgmt/src/dnodeTransport.c +++ b/source/dnode/mgmt/impl/src/dnodeTransport.c @@ -135,7 +135,7 @@ static void dnodeProcessPeerReq(SRpcMsg *pMsg, SEpSet *pEpSet) { return; } - if (dnodeGetRunStat() != DN_RUN_STAT_RUNNING) { + if (dnodeGetStat() != DN_STAT_RUNNING) { rspMsg.code = TSDB_CODE_APP_NOT_READY; rpcSendResponse(&rspMsg); rpcFreeCont(pMsg->pCont); @@ -193,7 +193,7 @@ static void dnodeCleanupPeerServer() { static void dnodeProcessPeerRsp(SRpcMsg *pMsg, SEpSet *pEpSet) { int32_t msgType = pMsg->msgType; - if (dnodeGetRunStat() == DN_RUN_STAT_STOPPED) { + if (dnodeGetStat() == DN_STAT_STOPPED) { if (pMsg == NULL || pMsg->pCont == NULL) return; dTrace("RPC %p, peer rsp:%s is ignored since dnode is stopping", pMsg->handle, taosMsg[msgType]); rpcFreeCont(pMsg->pCont); @@ -248,13 +248,13 @@ static void dnodeProcessShellReq(SRpcMsg *pMsg, SEpSet *pEpSet) { SRpcMsg rspMsg = {.handle = pMsg->handle}; int32_t msgType = pMsg->msgType; - if (dnodeGetRunStat() == DN_RUN_STAT_STOPPED) { + if (dnodeGetStat() == DN_STAT_STOPPED) { dError("RPC %p, shell req:%s is ignored since dnode exiting", pMsg->handle, taosMsg[msgType]); rspMsg.code = TSDB_CODE_DND_EXITING; rpcSendResponse(&rspMsg); rpcFreeCont(pMsg->pCont); return; - } else if (dnodeGetRunStat() != DN_RUN_STAT_RUNNING) { + } else if (dnodeGetStat() != DN_STAT_RUNNING) { dError("RPC %p, shell req:%s is ignored since dnode not running", pMsg->handle, taosMsg[msgType]); rspMsg.code = TSDB_CODE_APP_NOT_READY; rpcSendResponse(&rspMsg); @@ -382,13 +382,13 @@ void dnodeCleanupTrans() { dnodeCleanupClient(); } -void dnodeSendMsgToDnode(SServer *pServer, SEpSet *epSet, SRpcMsg *rpcMsg) { +void dnodeSendMsgToDnode(SDnode *pDnode, SEpSet *epSet, SRpcMsg *rpcMsg) { #if 0 rpcSendRequest(tsTrans.clientRpc, epSet, rpcMsg, NULL); #endif } -void dnodeSendMsgToMnode(SServer *pServer, SRpcMsg *rpcMsg) { +void dnodeSendMsgToMnode(SDnode *pDnode, SRpcMsg *rpcMsg) { SEpSet epSet = {0}; dnodeGetMnodeEpSetForPeer(&epSet); dnodeSendMsgToDnode(NULL, &epSet, rpcMsg); diff --git a/source/dnode/mgmt/src/dnodeVnodes.c b/source/dnode/mgmt/impl/src/dnodeVnodes.c similarity index 99% rename from source/dnode/mgmt/src/dnodeVnodes.c rename to source/dnode/mgmt/impl/src/dnodeVnodes.c index bd15850c42..7eaa82ba93 100644 --- a/source/dnode/mgmt/src/dnodeVnodes.c +++ b/source/dnode/mgmt/impl/src/dnodeVnodes.c @@ -815,7 +815,7 @@ void dnodeProcessVnodeFetchMsg(SRpcMsg *pMsg, SEpSet *pEpSet) { } } -static int32_t dnodePutMsgIntoVnodeApplyQueue(SServer *pServer, int32_t vgId, SVnodeMsg *pMsg) { +static int32_t dnodePutMsgIntoVnodeApplyQueue(SDnode *pDnode, int32_t vgId, SVnodeMsg *pMsg) { SVnodeObj *pVnode = dnodeAcquireVnode(vgId); if (pVnode == NULL) { return terrno; diff --git a/source/dnode/mgmt/inc/dnodeInt.h b/source/dnode/mgmt/inc/dnodeInt.h deleted file mode 100644 index 48da1ee558..0000000000 --- a/source/dnode/mgmt/inc/dnodeInt.h +++ /dev/null @@ -1,57 +0,0 @@ -/* - * 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_DNODE_INT_H_ -#define _TD_DNODE_INT_H_ - -#ifdef __cplusplus -extern "C" { -#endif -#include "os.h" -#include "taosmsg.h" -#include "tglobal.h" -#include "tlog.h" -#include "trpc.h" - -extern int32_t dDebugFlag; - -#define dFatal(...) { if (dDebugFlag & DEBUG_FATAL) { taosPrintLog("DND FATAL ", 255, __VA_ARGS__); }} -#define dError(...) { if (dDebugFlag & DEBUG_ERROR) { taosPrintLog("DND ERROR ", 255, __VA_ARGS__); }} -#define dWarn(...) { if (dDebugFlag & DEBUG_WARN) { taosPrintLog("DND WARN ", 255, __VA_ARGS__); }} -#define dInfo(...) { if (dDebugFlag & DEBUG_INFO) { taosPrintLog("DND ", 255, __VA_ARGS__); }} -#define dDebug(...) { if (dDebugFlag & DEBUG_DEBUG) { taosPrintLog("DND ", dDebugFlag, __VA_ARGS__); }} -#define dTrace(...) { if (dDebugFlag & DEBUG_TRACE) { taosPrintLog("DND ", dDebugFlag, __VA_ARGS__); }} - -typedef enum { DN_RUN_STAT_INIT, DN_RUN_STAT_RUNNING, DN_RUN_STAT_STOPPED } EDnStat; -typedef void (*MsgFp)(SRpcMsg *pMsg, SEpSet *pEpSet); - -typedef struct SServer { -} SServer; - -int32_t dnodeInit(); -void dnodeCleanup(); - -EDnStat dnodeGetRunStat(); -void dnodeSetRunStat(EDnStat stat); - -void dnodeReportStartup(char *name, char *desc); -void dnodeReportStartupFinished(char *name, char *desc); -void dnodeGetStartup(SStartupMsg *); - -#ifdef __cplusplus -} -#endif - -#endif /*_TD_DNODE_INT_H_*/ \ No newline at end of file diff --git a/source/dnode/mgmt/src/dnodeInt.c b/source/dnode/mgmt/src/dnodeInt.c deleted file mode 100644 index eee4bac050..0000000000 --- a/source/dnode/mgmt/src/dnodeInt.c +++ /dev/null @@ -1,181 +0,0 @@ -/* - * Copyright (c) 2019 TAOS Data, Inc. - * - * This program is free software: you can use, redistribute, and/or modify - * it under the terms of the GNU Affero General Public License, version 3 - * or later ("AGPL"), as published by the Free Software Foundation. - * - * This program is distributed in the hope that it will be useful, but WITHOUT - * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or - * FITNESS FOR A PARTICULAR PURPOSE. - * - * You should have received a copy of the GNU Affero General Public License - * along with this program. If not, see . - */ - -#define _DEFAULT_SOURCE -#include "dnodeDnode.h" -#include "dnodeMnode.h" -#include "dnodeTransport.h" -#include "dnodeVnodes.h" -#include "sync.h" -#include "tcache.h" -#include "tconfig.h" -#include "tnote.h" -#include "tstep.h" -#include "wal.h" - -static struct { - SStartupMsg startup; - EDnStat runStat; - SSteps *steps; -} tsInt; - -EDnStat dnodeGetRunStat() { return tsInt.runStat; } - -void dnodeSetRunStat(EDnStat stat) { - dDebug("runstat set to %d", stat); - tsInt.runStat = stat; -} - -void dnodeReportStartup(char *name, char *desc) { - SStartupMsg *pStartup = &tsInt.startup; - tstrncpy(pStartup->name, name, strlen(pStartup->name)); - tstrncpy(pStartup->desc, desc, strlen(pStartup->desc)); - pStartup->finished = 0; -} - -void dnodeReportStartupFinished(char *name, char *desc) { - SStartupMsg *pStartup = &tsInt.startup; - tstrncpy(pStartup->name, name, strlen(pStartup->name)); - tstrncpy(pStartup->desc, desc, strlen(pStartup->desc)); - pStartup->finished = 1; -} - -void dnodeGetStartup(SStartupMsg *pStartup) { memcpy(pStartup, &tsInt.startup, sizeof(SStartupMsg)); } - -static int32_t dnodeCheckRunning(char *dir) { - char filepath[256] = {0}; - snprintf(filepath, sizeof(filepath), "%s/.running", dir); - - FileFd fd = taosOpenFileCreateWriteTrunc(filepath); - if (fd < 0) { - dError("failed to open lock file:%s since %s, quit", filepath, strerror(errno)); - return -1; - } - - int32_t ret = taosLockFile(fd); - if (ret != 0) { - dError("failed to lock file:%s since %s, quit", filepath, strerror(errno)); - taosCloseFile(fd); - return -1; - } - - return 0; -} - -static int32_t dnodeInitDir() { - sprintf(tsMnodeDir, "%s/mnode", tsDataDir); - sprintf(tsVnodeDir, "%s/vnode", tsDataDir); - sprintf(tsDnodeDir, "%s/dnode", tsDataDir); - - if (!taosMkDir(tsDnodeDir)) { - dError("failed to create dir:%s since %s", tsDnodeDir, strerror(errno)); - return -1; - } - - if (!taosMkDir(tsMnodeDir)) { - dError("failed to create dir:%s since %s", tsMnodeDir, strerror(errno)); - return -1; - } - - if (!taosMkDir(tsVnodeDir)) { - dError("failed to create dir:%s since %s", tsVnodeDir, strerror(errno)); - return -1; - } - - if (dnodeCheckRunning(tsDnodeDir) != 0) { - return -1; - } - - return 0; -} - -static int32_t dnodeInitMain() { - tscEmbedded = 1; - taosIgnSIGPIPE(); - taosBlockSIGPIPE(); - taosResolveCRC(); - taosInitGlobalCfg(); - taosReadGlobalLogCfg(); - taosSetCoreDump(tsEnableCoreFile); - - if (!taosMkDir(tsLogDir)) { - printf("failed to create dir: %s, reason: %s\n", tsLogDir, strerror(errno)); - return -1; - } - - char temp[TSDB_FILENAME_LEN]; - sprintf(temp, "%s/taosdlog", tsLogDir); - if (taosInitLog(temp, tsNumOfLogLines, 1) < 0) { - printf("failed to init log file\n"); - } - - if (!taosReadGlobalCfg()) { - taosPrintGlobalCfg(); - dError("TDengine read global config failed"); - return -1; - } - - dInfo("start to initialize TDengine"); - - taosInitNotes(); - - if (taosCheckGlobalCfg() != 0) { - return -1; - } - - dnodeInitDir(); - - return 0; -} - -static void dnodeCleanupMain() { - taos_cleanup(); - taosCloseLog(); - taosStopCacheRefreshWorker(); -} - -int32_t dnodeInit() { - SSteps *steps = taosStepInit(10, dnodeReportStartup); - if (steps == NULL) return -1; -#if 1 - dnodeSetRunStat(DN_RUN_STAT_RUNNING); -#endif - taosStepAdd(steps, "dnode-main", dnodeInitMain, dnodeCleanupMain); - taosStepAdd(steps, "dnode-rpc", rpcInit, rpcCleanup); - taosStepAdd(steps, "dnode-tfs", NULL, NULL); - taosStepAdd(steps, "dnode-wal", walInit, walCleanUp); - //taosStepAdd(steps, "dnode-sync", syncInit, syncCleanUp); - taosStepAdd(steps, "dnode-dnode", dnodeInitDnode, dnodeCleanupDnode); - taosStepAdd(steps, "dnode-vnodes", dnodeInitVnodes, dnodeCleanupVnodes); - taosStepAdd(steps, "dnode-mnode", dnodeInitMnode, dnodeCleanupMnode); - taosStepAdd(steps, "dnode-trans", dnodeInitTrans, dnodeCleanupTrans); - - tsInt.steps = steps; - taosStepExec(tsInt.steps); - - dnodeSetRunStat(DN_RUN_STAT_RUNNING); - dnodeReportStartupFinished("TDengine", "initialized successfully"); - dInfo("TDengine is initialized successfully"); - - return 0; -} - -void dnodeCleanup() { - if (dnodeGetRunStat() != DN_RUN_STAT_STOPPED) { - dnodeSetRunStat(DN_RUN_STAT_STOPPED); - taosStepCleanup(tsInt.steps); - tsInt.steps = NULL; - } -} diff --git a/source/dnode/mnode/impl/inc/mnodeInt.h b/source/dnode/mnode/impl/inc/mnodeInt.h index b0005acc20..43af281f27 100644 --- a/source/dnode/mnode/impl/inc/mnodeInt.h +++ b/source/dnode/mnode/impl/inc/mnodeInt.h @@ -32,21 +32,22 @@ typedef struct SMnodeBak { tmr_h timer; SSteps *pInitSteps; SSteps *pStartSteps; - SMnodePara para; + SMnodeOptions para; MnodeRpcFp msgFp[TSDB_MSG_TYPE_MAX]; } SMnodeBak; typedef struct SMnode { - int32_t dnodeId; - int64_t clusterId; - tmr_h timer; - SSteps *pInitSteps; - SSteps *pStartSteps; - SMnodePara para; - MnodeRpcFp msgFp[TSDB_MSG_TYPE_MAX]; - + int32_t dnodeId; + int64_t clusterId; + int8_t replica; + int8_t selfIndex; + SReplica replicas[TSDB_MAX_REPLICA]; + tmr_h timer; + SSteps *pInitSteps; + SSteps *pStartSteps; struct SSdb *pSdb; - struct SServer *pServer; + struct SDnode *pServer; + MnodeRpcFp msgFp[TSDB_MSG_TYPE_MAX]; PutMsgToMnodeQFp putMsgToApplyMsgFp; SendMsgToDnodeFp sendMsgToDnodeFp; SendMsgToMnodeFp sendMsgToMnodeFp; diff --git a/source/dnode/mnode/impl/src/mnode.c b/source/dnode/mnode/impl/src/mnode.c index 9ea4ebe0e6..bc4718ee5b 100644 --- a/source/dnode/mnode/impl/src/mnode.c +++ b/source/dnode/mnode/impl/src/mnode.c @@ -77,40 +77,20 @@ static void mnodeCleanupTimer() { tmr_h mnodeGetTimer() { return tsMint.timer; } -static int32_t mnodeSetPara(SMnode *pMnode, SMnodePara para) { - pMnode->dnodeId = para.dnodeId; - pMnode->clusterId = para.clusterId; - pMnode->putMsgToApplyMsgFp = para.putMsgToApplyMsgFp; - pMnode->sendMsgToDnodeFp = para.sendMsgToDnodeFp; - pMnode->sendMsgToMnodeFp = para.sendMsgToMnodeFp; - pMnode->sendRedirectMsgFp = para.sendRedirectMsgFp; +static int32_t mnodeSetOptions(SMnode *pMnode, const SMnodeOptions *pOptions) { + pMnode->dnodeId = pOptions->dnodeId; + pMnode->clusterId = pOptions->clusterId; + pMnode->replica = pOptions->replica; + pMnode->selfIndex = pOptions->selfIndex; + memcpy(&pMnode->replicas, pOptions->replicas, sizeof(SReplica) * TSDB_MAX_REPLICA); + pMnode->pServer = pOptions->pServer; + pMnode->putMsgToApplyMsgFp = pOptions->putMsgToApplyMsgFp; + pMnode->sendMsgToDnodeFp = pOptions->sendMsgToDnodeFp; + pMnode->sendMsgToMnodeFp = pOptions->sendMsgToMnodeFp; + pMnode->sendRedirectMsgFp = pOptions->sendRedirectMsgFp; - if (pMnode->sendMsgToDnodeFp == NULL) { - terrno = TSDB_CODE_MND_APP_ERROR; - return -1; - } - - if (pMnode->sendMsgToMnodeFp == NULL) { - terrno = TSDB_CODE_MND_APP_ERROR; - return -1; - } - - if (pMnode->sendRedirectMsgFp == NULL) { - terrno = TSDB_CODE_MND_APP_ERROR; - return -1; - } - - if (pMnode->putMsgToApplyMsgFp == NULL) { - terrno = TSDB_CODE_MND_APP_ERROR; - return -1; - } - - if (pMnode->dnodeId < 0) { - terrno = TSDB_CODE_MND_APP_ERROR; - return -1; - } - - if (pMnode->clusterId < 0) { + if (pMnode->sendMsgToDnodeFp == NULL || pMnode->sendMsgToMnodeFp == NULL || pMnode->sendRedirectMsgFp == NULL || + pMnode->putMsgToApplyMsgFp == NULL || pMnode->dnodeId < 0 || pMnode->clusterId < 0) { terrno = TSDB_CODE_MND_APP_ERROR; return -1; } @@ -156,12 +136,12 @@ static int32_t mnodeAllocStartSteps() { return 0; } -SMnode *mnodeCreate(SMnodePara para) { +SMnode *mnodeOpen(const char *path, const SMnodeOptions *pOptions) { SMnode *pMnode = calloc(1, sizeof(SMnode)); - if (mnodeSetPara(pMnode, para) != 0) { + if (mnodeSetOptions(pMnode, pOptions) != 0) { free(pMnode); - mError("failed to init mnode para since %s", terrstr()); + mError("failed to init mnode options since %s", terrstr()); return NULL; } @@ -175,35 +155,31 @@ SMnode *mnodeCreate(SMnodePara para) { return NULL; } - taosStepExec(tsMint.pInitSteps); - return NULL; -} + taosStepExec(tsMint.pInitSteps); -void mnodeCleanup() { taosStepCleanup(tsMint.pInitSteps); } - -int32_t mnodeDeploy(SMnodeCfg *pCfg) { if (tsMint.para.dnodeId <= 0 && tsMint.para.clusterId <= 0) { if (sdbDeploy() != 0) { mError("failed to deploy sdb since %s", terrstr()); - return -1; + return NULL; + } else { + mInfo("mnode is deployed"); } } - mDebug("mnode is deployed"); - return 0; + taosStepExec(tsMint.pStartSteps); + + return pMnode; } -void mnodeUnDeploy() { sdbUnDeploy(); } +void mnodeClose(SMnode *pMnode) { free(pMnode); } -int32_t mnodeStart(SMnodeCfg *pCfg) { return taosStepExec(tsMint.pStartSteps); } +int32_t mnodeAlter(SMnode *pMnode, const SMnodeOptions *pOptions) { return 0; } -int32_t mnodeAlter(SMnodeCfg *pCfg) { return 0; } +void mnodeDestroy(const char *path) { sdbUnDeploy(); } -void mnodeStop() { taosStepCleanup(tsMint.pStartSteps); } +int32_t mnodeGetLoad(SMnode *pMnode, SMnodeLoad *pLoad) { return 0; } -int32_t mnodeGetLoad(SMnodeLoad *pLoad) { return 0; } - -SMnodeMsg *mnodeInitMsg(SRpcMsg *pRpcMsg) { +SMnodeMsg *mnodeInitMsg(SMnode *pMnode, SRpcMsg *pRpcMsg) { SMnodeMsg *pMsg = taosAllocateQitem(sizeof(SMnodeMsg)); if (pMsg == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; @@ -211,7 +187,7 @@ SMnodeMsg *mnodeInitMsg(SRpcMsg *pRpcMsg) { } if (rpcGetConnInfo(pRpcMsg->handle, &pMsg->conn) != 0) { - mnodeCleanupMsg(pMsg); + mnodeCleanupMsg(pMnode, pMsg); mError("can not get user from conn:%p", pMsg->rpcMsg.handle); terrno = TSDB_CODE_MND_NO_USER_FROM_CONN; return NULL; @@ -223,7 +199,7 @@ SMnodeMsg *mnodeInitMsg(SRpcMsg *pRpcMsg) { return pMsg; } -void mnodeCleanupMsg(SMnodeMsg *pMsg) { +void mnodeCleanupMsg(SMnode *pMnode, SMnodeMsg *pMsg) { if (pMsg->pUser != NULL) { sdbRelease(pMsg->pUser); } @@ -232,6 +208,12 @@ void mnodeCleanupMsg(SMnodeMsg *pMsg) { } static void mnodeProcessRpcMsg(SMnodeMsg *pMsg) { + if (!mnodeIsMaster()) { + mnodeSendRedirectMsg(NULL, &pMsg->rpcMsg, true); + mnodeCleanupMsg(NULL, pMsg); + return; + } + int32_t msgType = pMsg->rpcMsg.msgType; MnodeRpcFp fp = tsMint.msgFp[msgType]; @@ -250,25 +232,13 @@ void mnodeSetMsgFp(int32_t msgType, MnodeRpcFp fp) { } } -void mnodeProcessMsg(SMnodeMsg *pMsg, EMnMsgType msgType) { - if (!mnodeIsMaster()) { - mnodeSendRedirectMsg(NULL, &pMsg->rpcMsg, true); - mnodeCleanupMsg(pMsg); - return; - } +void mnodeProcessReadMsg(SMnode *pMnode, SMnodeMsg *pMsg) { mnodeProcessRpcMsg(pMsg); } - switch (msgType) { - case MN_MSG_TYPE_READ: - case MN_MSG_TYPE_WRITE: - case MN_MSG_TYPE_SYNC: - mnodeProcessRpcMsg(pMsg); - break; - case MN_MSG_TYPE_APPLY: - break; - default: - break; - } -} +void mnodeProcessWriteMsg(SMnode *pMnode, SMnodeMsg *pMsg) { mnodeProcessRpcMsg(pMsg); } + +void mnodeProcessSyncMsg(SMnode *pMnode, SMnodeMsg *pMsg) { mnodeProcessRpcMsg(pMsg); } + +void mnodeProcessApplyMsg(SMnode *pMnode, SMnodeMsg *pMsg) {} #if 0 diff --git a/source/dnode/mnode/impl/src/mnodeAuth.c b/source/dnode/mnode/impl/src/mnodeAuth.c index bb3289ebeb..ddd2b91ff3 100644 --- a/source/dnode/mnode/impl/src/mnodeAuth.c +++ b/source/dnode/mnode/impl/src/mnodeAuth.c @@ -20,7 +20,7 @@ int32_t mnodeInitAuth() { return 0; } void mnodeCleanupAuth() {} -int32_t mnodeRetriveAuth(char *user, char *spi, char *encrypt, char *secret, char *ckey) { +int32_t mnodeRetriveAuth(SMnode *pMnode, char *user, char *spi, char *encrypt, char *secret, char *ckey) { if (strcmp(user, TSDB_NETTEST_USER) == 0) { char pass[32] = {0}; taosEncryptPass((uint8_t *)user, strlen(user), pass); diff --git a/source/dnode/mnode/impl/src/mnodeTelem.c b/source/dnode/mnode/impl/src/mnodeTelem.c index ef1ac10eb6..206b94a6c7 100644 --- a/source/dnode/mnode/impl/src/mnodeTelem.c +++ b/source/dnode/mnode/impl/src/mnodeTelem.c @@ -174,7 +174,7 @@ static void mnodeAddVersionInfo(SBufferWriter* bw) { static void mnodeAddRuntimeInfo(SBufferWriter* bw) { SMnodeLoad load = {0}; - if (mnodeGetLoad(&load) != 0) { + if (mnodeGetLoad(NULL, &load) != 0) { return; } From b0c147f05770bf1560a3115aa81724e47d9d74e0 Mon Sep 17 00:00:00 2001 From: lichuang Date: Mon, 22 Nov 2021 10:04:47 +0800 Subject: [PATCH 13/44] [TD-10645][raft]add vote resp process --- source/libs/sync/inc/raft_log.h | 2 -- source/libs/sync/src/sync_raft_impl.c | 2 +- 2 files changed, 1 insertion(+), 3 deletions(-) diff --git a/source/libs/sync/inc/raft_log.h b/source/libs/sync/inc/raft_log.h index dc10c59b28..117ed42c2c 100644 --- a/source/libs/sync/inc/raft_log.h +++ b/source/libs/sync/inc/raft_log.h @@ -39,8 +39,6 @@ struct SSyncRaftLog { SyncIndex commitIndex; SyncIndex appliedIndex; - - }; SSyncRaftLog* syncRaftLogOpen(); diff --git a/source/libs/sync/src/sync_raft_impl.c b/source/libs/sync/src/sync_raft_impl.c index 4d8222e826..3050bb2c8a 100644 --- a/source/libs/sync/src/sync_raft_impl.c +++ b/source/libs/sync/src/sync_raft_impl.c @@ -247,7 +247,7 @@ static int stepCandidate(SSyncRaft* pRaft, const SSyncMessage* pMsg) { syncRaftHandleVoteRespMessage(pRaft, pMsg); return 0; } else if (msgType == RAFT_MSG_APPEND) { - syncRaftBecomeFollower(pRaft, pRaft->term, pMsg->from); + syncRaftBecomeFollower(pRaft, pMsg->term, pMsg->from); syncRaftHandleAppendEntriesMessage(pRaft, pMsg); } return 0; From 759b077ecf667184e2f185ba195d08da2caa4dcc Mon Sep 17 00:00:00 2001 From: Liu Jicong Date: Mon, 22 Nov 2021 14:56:11 +0800 Subject: [PATCH 14/44] add walhandle for integration --- include/dnode/vnode/tq/tq.h | 118 +++++++++++++++--- include/libs/wal/wal.h | 6 +- source/dnode/vnode/tq/inc/tqMetaStore.h | 87 +------------ source/dnode/vnode/tq/src/tq.c | 95 +++++++++++--- source/dnode/vnode/tq/src/tqMetaStore.c | 48 +++---- .../dnode/vnode/tq/test/tqSerializerTest.cpp | 13 ++ source/libs/wal/src/wal.c | 12 +- 7 files changed, 233 insertions(+), 146 deletions(-) create mode 100644 source/dnode/vnode/tq/test/tqSerializerTest.cpp diff --git a/include/dnode/vnode/tq/tq.h b/include/dnode/vnode/tq/tq.h index 7a21b08aaf..2785b6de96 100644 --- a/include/dnode/vnode/tq/tq.h +++ b/include/dnode/vnode/tq/tq.h @@ -109,11 +109,10 @@ typedef struct TqTopicVhandle { #define TQ_BUFFER_SIZE 8 -// TODO: define a serializer and deserializer typedef struct TqBufferItem { int64_t offset; // executors are identical but not concurrent - // so it must be a copy in each item + // so there must be a copy in each item void* executor; int64_t size; void* content; @@ -156,23 +155,111 @@ typedef struct TqQueryMsg { typedef struct TqLogReader { void* logHandle; - int32_t (*walRead)(void* logHandle, void** data, int64_t ver); - int64_t (*walGetFirstVer)(void* logHandle); - int64_t (*walGetSnapshotVer)(void* logHandle); - int64_t (*walGetLastVer)(void* logHandle); + int32_t (*logRead)(void* logHandle, void** data, int64_t ver); + int64_t (*logGetFirstVer)(void* logHandle); + int64_t (*logGetSnapshotVer)(void* logHandle); + int64_t (*logGetLastVer)(void* logHandle); } TqLogReader; typedef struct TqConfig { // TODO } TqConfig; +typedef struct TqMemRef { + SMemAllocatorFactory *pAlloctorFactory; + SMemAllocator *pAllocator; +} TqMemRef; + +typedef struct TqSerializedHead { + int16_t ver; + int16_t action; + int32_t checksum; + int64_t ssize; + char content[]; +} TqSerializedHead; + +typedef int (*TqSerializeFun)(const void* pObj, TqSerializedHead** ppHead); +typedef const void* (*TqDeserializeFun)(const TqSerializedHead* pHead, void** ppObj); +typedef void (*TqDeleteFun)(void*); + +#define TQ_BUCKET_MASK 0xFF +#define TQ_BUCKET_SIZE 256 + +#define TQ_PAGE_SIZE 4096 +//key + offset + size +#define TQ_IDX_SIZE 24 +//4096 / 24 +#define TQ_MAX_IDX_ONE_PAGE 170 +//24 * 170 +#define TQ_IDX_PAGE_BODY_SIZE 4080 +//4096 - 4080 +#define TQ_IDX_PAGE_HEAD_SIZE 16 + +#define TQ_ACTION_CONST 0 +#define TQ_ACTION_INUSE 1 +#define TQ_ACTION_INUSE_CONT 2 +#define TQ_ACTION_INTXN 3 + +#define TQ_SVER 0 + +//TODO: inplace mode is not implemented +#define TQ_UPDATE_INPLACE 0 +#define TQ_UPDATE_APPEND 1 + +#define TQ_DUP_INTXN_REWRITE 0 +#define TQ_DUP_INTXN_REJECT 2 + +static inline bool TqUpdateAppend(int32_t tqConfigFlag) { + return tqConfigFlag & TQ_UPDATE_APPEND; +} + +static inline bool TqDupIntxnReject(int32_t tqConfigFlag) { + return tqConfigFlag & TQ_DUP_INTXN_REJECT; +} + +static const int8_t TQ_CONST_DELETE = TQ_ACTION_CONST; +#define TQ_DELETE_TOKEN (void*)&TQ_CONST_DELETE + +typedef struct TqMetaHandle { + int64_t key; + int64_t offset; + int64_t serializedSize; + void* valueInUse; + void* valueInTxn; +} TqMetaHandle; + +typedef struct TqMetaList { + TqMetaHandle handle; + struct TqMetaList* next; + //struct TqMetaList* inTxnPrev; + //struct TqMetaList* inTxnNext; + struct TqMetaList* unpersistPrev; + struct TqMetaList* unpersistNext; +} TqMetaList; + +typedef struct TqMetaStore { + TqMetaList* bucket[TQ_BUCKET_SIZE]; + //a table head + TqMetaList* unpersistHead; + //TODO:temporaral use, to be replaced by unified tfile + int fileFd; + //TODO:temporaral use, to be replaced by unified tfile + int idxFd; + char* dirPath; + int32_t tqConfigFlag; + TqSerializeFun pSerializer; + TqDeserializeFun pDeserializer; + TqDeleteFun pDeleter; +} TqMetaStore; + typedef struct STQ { // the collection of group handle // the handle of kvstore - const char* path; + char* path; TqConfig* tqConfig; TqLogReader* tqLogReader; - SMemAllocatorFactory* allocFac; + TqMemRef tqMemRef; + TqMetaStore* tqMeta; } STQ; // open in each vnode @@ -187,7 +274,7 @@ int tqConsume(STQ*, TmqConsumeReq*); TqGroupHandle* tqGetGroupHandle(STQ*, int64_t cId); -int tqOpenTCGroup(STQ*, int64_t topicId, int64_t cgId, int64_t cId); +TqGroupHandle* tqOpenTCGroup(STQ*, int64_t topicId, int64_t cgId, int64_t cId); int tqCloseTCGroup(STQ*, int64_t topicId, int64_t cgId, int64_t cId); int tqMoveOffsetToNext(TqGroupHandle*); int tqResetOffset(STQ*, int64_t topicId, int64_t cgId, int64_t offset); @@ -195,18 +282,9 @@ int tqRegisterContext(TqGroupHandle*, void* ahandle); int tqLaunchQuery(TqGroupHandle*); int tqSendLaunchQuery(TqGroupHandle*); -int tqSerializeGroupHandle(TqGroupHandle* gHandle, void** ppBytes); -void* tqSerializeListHandle(TqListHandle* listHandle, void* ptr); -void* tqSerializeBufHandle(TqBufferHandle* bufHandle, void* ptr); -void* tqSerializeBufItem(TqBufferItem* bufItem, void* ptr); +int tqSerializeGroupHandle(const TqGroupHandle* gHandle, TqSerializedHead** ppHead); -const void* tqDeserializeGroupHandle(const void* pBytes, TqGroupHandle* ghandle); -const void* tqDeserializeBufHandle(const void* pBytes, TqBufferHandle* bufHandle); -const void* tqDeserializeBufItem(const void* pBytes, TqBufferItem* bufItem); - -int tqGetGHandleSSize(const TqGroupHandle* gHandle); -int tqBufHandleSSize(); -int tqBufItemSSize(); +const void* tqDeserializeGroupHandle(const TqSerializedHead* pHead, TqGroupHandle** gHandle); #ifdef __cplusplus } diff --git a/include/libs/wal/wal.h b/include/libs/wal/wal.h index 0829782310..ba37e6880b 100644 --- a/include/libs/wal/wal.h +++ b/include/libs/wal/wal.h @@ -44,8 +44,10 @@ typedef struct { EWalType walLevel; // wal level } SWalCfg; -struct SWal; -typedef struct SWal SWal; // WAL HANDLE +typedef struct SWal { + int8_t unused; +} SWal; // WAL HANDLE + typedef int32_t (*FWalWrite)(void *ahandle, void *pHead, int32_t qtype, void *pMsg); // module initialization diff --git a/source/dnode/vnode/tq/inc/tqMetaStore.h b/source/dnode/vnode/tq/inc/tqMetaStore.h index 63e48625d9..b9e702a89a 100644 --- a/source/dnode/vnode/tq/inc/tqMetaStore.h +++ b/source/dnode/vnode/tq/inc/tqMetaStore.h @@ -17,97 +17,22 @@ #define _TQ_META_STORE_H_ #include "os.h" - +#include "tq.h" #ifdef __cplusplus extern "C" { #endif -#define TQ_BUCKET_MASK 0xFF -#define TQ_BUCKET_SIZE 256 - -#define TQ_PAGE_SIZE 4096 -//key + offset + size -#define TQ_IDX_SIZE 24 -//4096 / 24 -#define TQ_MAX_IDX_ONE_PAGE 170 -//24 * 170 -#define TQ_IDX_PAGE_BODY_SIZE 4080 -//4096 - 4080 -#define TQ_IDX_PAGE_HEAD_SIZE 16 - -#define TQ_ACTION_CONST 0 -#define TQ_ACTION_INUSE 1 -#define TQ_ACTION_INUSE_CONT 2 -#define TQ_ACTION_INTXN 3 - -#define TQ_SVER 0 - -//TODO: inplace mode is not implemented -#define TQ_UPDATE_INPLACE 0 -#define TQ_UPDATE_APPEND 1 - -#define TQ_DUP_INTXN_REWRITE 0 -#define TQ_DUP_INTXN_REJECT 2 - -static inline bool TqUpdateAppend(int32_t tqConfigFlag) { - return tqConfigFlag & TQ_UPDATE_APPEND; -} - -static inline bool TqDupIntxnReject(int32_t tqConfigFlag) { - return tqConfigFlag & TQ_DUP_INTXN_REJECT; -} - -static const int8_t TQ_CONST_DELETE = TQ_ACTION_CONST; -#define TQ_DELETE_TOKEN (void*)&TQ_CONST_DELETE - -typedef struct TqSerializedHead { - int16_t ver; - int16_t action; - int32_t checksum; - int64_t ssize; - char content[]; -} TqSerializedHead; - -typedef struct TqMetaHandle { - int64_t key; - int64_t offset; - int64_t serializedSize; - void* valueInUse; - void* valueInTxn; -} TqMetaHandle; - -typedef struct TqMetaList { - TqMetaHandle handle; - struct TqMetaList* next; - //struct TqMetaList* inTxnPrev; - //struct TqMetaList* inTxnNext; - struct TqMetaList* unpersistPrev; - struct TqMetaList* unpersistNext; -} TqMetaList; - -typedef struct TqMetaStore { - TqMetaList* bucket[TQ_BUCKET_SIZE]; - //a table head - TqMetaList* unpersistHead; - int fileFd; //TODO:temporaral use, to be replaced by unified tfile - int idxFd; //TODO:temporaral use, to be replaced by unified tfile - char* dirPath; - int32_t tqConfigFlag; - int (*serializer)(const void* pObj, TqSerializedHead** ppHead); - const void* (*deserializer)(const TqSerializedHead* pHead, void** ppObj); - void (*deleter)(void*); -} TqMetaStore; TqMetaStore* tqStoreOpen(const char* path, - int serializer(const void* pObj, TqSerializedHead** ppHead), - const void* deserializer(const TqSerializedHead* pHead, void** ppObj), - void deleter(void* pObj), - int32_t tqConfigFlag + TqSerializeFun pSerializer, + TqDeserializeFun pDeserializer, + TqDeleteFun pDeleter, + int32_t tqConfigFlag ); int32_t tqStoreClose(TqMetaStore*); //int32_t tqStoreDelete(TqMetaStore*); -//int32_t TqStoreCommitAll(TqMetaStore*); +//int32_t tqStoreCommitAll(TqMetaStore*); int32_t tqStorePersist(TqMetaStore*); //clean deleted idx and data from persistent file int32_t tqStoreCompact(TqMetaStore*); diff --git a/source/dnode/vnode/tq/src/tq.c b/source/dnode/vnode/tq/src/tq.c index cf98e3e1a4..c010042b8c 100644 --- a/source/dnode/vnode/tq/src/tq.c +++ b/source/dnode/vnode/tq/src/tq.c @@ -14,6 +14,7 @@ */ #include "tqInt.h" +#include "tqMetaStore.h" //static //read next version data @@ -24,6 +25,46 @@ // int tqGetgHandleSSize(const TqGroupHandle *gHandle); +int tqBufHandleSSize(); +int tqBufItemSSize(); + +TqGroupHandle* tqFindHandle(STQ* pTq, int64_t topicId, int64_t cgId, int64_t cId) { + TqGroupHandle* gHandle; + return NULL; +} + +void* tqSerializeListHandle(TqListHandle* listHandle, void* ptr); +void* tqSerializeBufHandle(TqBufferHandle* bufHandle, void* ptr); +void* tqSerializeBufItem(TqBufferItem* bufItem, void* ptr); + +const void* tqDeserializeBufHandle(const void* pBytes, TqBufferHandle* bufHandle); +const void* tqDeserializeBufItem(const void* pBytes, TqBufferItem* bufItem); + +STQ* tqOpen(const char* path, TqConfig* tqConfig, TqLogReader* tqLogReader, SMemAllocatorFactory *allocFac) { + STQ* pTq = malloc(sizeof(STQ)); + if(pTq == NULL) { + //TODO: memory error + return NULL; + } + strcpy(pTq->path, path); + pTq->tqConfig = tqConfig; + pTq->tqLogReader = tqLogReader; + pTq->tqMemRef.pAlloctorFactory = allocFac; + pTq->tqMemRef.pAllocator = allocFac->create(); + if(pTq->tqMemRef.pAllocator == NULL) { + //TODO + } + pTq->tqMeta = tqStoreOpen(path, + (TqSerializeFun)tqSerializeGroupHandle, + (TqDeserializeFun)tqDeserializeGroupHandle, + free, + 0); + if(pTq->tqMeta == NULL) { + //TODO: free STQ + return NULL; + } + return pTq; +} static int tqProtoCheck(TmqMsgHead *pMsg) { return pMsg->protoVer == 0; @@ -83,14 +124,29 @@ static int tqCommitTCGroup(TqGroupHandle* handle) { int tqCreateTCGroup(STQ *pTq, int64_t topicId, int64_t cgId, int64_t cId, TqGroupHandle** handle) { //create in disk + TqGroupHandle* gHandle = (TqGroupHandle*)malloc(sizeof(TqGroupHandle)); + if(gHandle == NULL) { + //TODO + return -1; + } + memset(gHandle, 0, sizeof(TqGroupHandle)); + return 0; } -int tqOpenTCGroup(STQ* pTq, int64_t topicId, int64_t cgId, int64_t cId) { - //look up in disk +TqGroupHandle* tqOpenTCGroup(STQ* pTq, int64_t topicId, int64_t cgId, int64_t cId) { + TqGroupHandle* gHandle = tqHandleGet(pTq->tqMeta, cId); + if(gHandle == NULL) { + int code = tqCreateTCGroup(pTq, topicId, cgId, cId, &gHandle); + if(code != 0) { + //TODO + return NULL; + } + } + //create //open - return 0; + return gHandle; } int tqCloseTCGroup(STQ* pTq, int64_t topicId, int64_t cgId, int64_t cId) { @@ -207,16 +263,20 @@ int tqConsume(STQ* pTq, TmqConsumeReq* pMsg) { return 0; } -int tqSerializeGroupHandle(TqGroupHandle *gHandle, void** ppBytes) { +int tqSerializeGroupHandle(const TqGroupHandle *gHandle, TqSerializedHead** ppHead) { //calculate size - int sz = tqGetgHandleSSize(gHandle); - void* ptr = realloc(*ppBytes, sz); - if(ptr == NULL) { - free(ppBytes); - //TODO: memory err - return -1; + int sz = tqGetgHandleSSize(gHandle) + sizeof(TqSerializedHead); + if(sz > (*ppHead)->ssize) { + void* tmpPtr = realloc(*ppHead, sz); + if(tmpPtr == NULL) { + free(*ppHead); + //TODO: memory err + return -1; + } + *ppHead = tmpPtr; + (*ppHead)->ssize = sz; } - *ppBytes = ptr; + void* ptr = (*ppHead)->content; //do serialization *(int64_t*)ptr = gHandle->cId; ptr = POINTER_SHIFT(ptr, sizeof(int64_t)); @@ -261,8 +321,9 @@ void* tqSerializeBufItem(TqBufferItem *bufItem, void* ptr) { return ptr; } -const void* tqDeserializeGroupHandle(const void* pBytes, TqGroupHandle *gHandle) { - const void* ptr = pBytes; +const void* tqDeserializeGroupHandle(const TqSerializedHead* pHead, TqGroupHandle **ppGHandle) { + TqGroupHandle *gHandle = *ppGHandle; + const void* ptr = pHead->content; gHandle->cId = *(int64_t*)ptr; ptr = POINTER_SHIFT(ptr, sizeof(int64_t)); gHandle->cgId = *(int64_t*)ptr; @@ -317,15 +378,15 @@ const void* tqDeserializeBufItem(const void* pBytes, TqBufferItem *bufItem) { //TODO: make this a macro int tqGetgHandleSSize(const TqGroupHandle *gHandle) { - return sizeof(int64_t) * 2 - + sizeof(int32_t) + return sizeof(int64_t) * 2 //cId + cgId + + sizeof(int32_t) //topicNum + gHandle->topicNum * tqBufHandleSSize(); } //TODO: make this a macro int tqBufHandleSSize() { - return sizeof(int64_t) * 2 - + sizeof(int32_t) * 2 + return sizeof(int64_t) * 2 // nextConsumeOffset + topicId + + sizeof(int32_t) * 2 // head + tail + TQ_BUFFER_SIZE * tqBufItemSSize(); } diff --git a/source/dnode/vnode/tq/src/tqMetaStore.c b/source/dnode/vnode/tq/src/tqMetaStore.c index f8202941bb..71d1e8d890 100644 --- a/source/dnode/vnode/tq/src/tqMetaStore.c +++ b/source/dnode/vnode/tq/src/tqMetaStore.c @@ -69,10 +69,10 @@ static inline int tqReadLastPage(int fd, TqIdxPageBuf* pBuf) { } TqMetaStore* tqStoreOpen(const char* path, - int serializer(const void* pObj, TqSerializedHead** ppHead), - const void* deserializer(const TqSerializedHead* pHead, void** ppObj), - void deleter(void* pObj), - int32_t tqConfigFlag + TqSerializeFun serializer, + TqDeserializeFun deserializer, + TqDeleteFun deleter, + int32_t tqConfigFlag ) { TqMetaStore* pMeta = malloc(sizeof(TqMetaStore)); if(pMeta == NULL) { @@ -127,9 +127,9 @@ TqMetaStore* tqStoreOpen(const char* path, pMeta->fileFd = fileFd; - pMeta->serializer = serializer; - pMeta->deserializer = deserializer; - pMeta->deleter = deleter; + pMeta->pSerializer = serializer; + pMeta->pDeserializer = deserializer; + pMeta->pDeleter = deleter; pMeta->tqConfigFlag = tqConfigFlag; //read idx file and load into memory @@ -171,25 +171,25 @@ TqMetaStore* tqStoreOpen(const char* path, } if(serializedObj->action == TQ_ACTION_INUSE) { if(serializedObj->ssize != sizeof(TqSerializedHead)) { - pMeta->deserializer(serializedObj, &pNode->handle.valueInUse); + pMeta->pDeserializer(serializedObj, &pNode->handle.valueInUse); } else { pNode->handle.valueInUse = TQ_DELETE_TOKEN; } } else if(serializedObj->action == TQ_ACTION_INTXN) { if(serializedObj->ssize != sizeof(TqSerializedHead)) { - pMeta->deserializer(serializedObj, &pNode->handle.valueInTxn); + pMeta->pDeserializer(serializedObj, &pNode->handle.valueInTxn); } else { pNode->handle.valueInTxn = TQ_DELETE_TOKEN; } } else if(serializedObj->action == TQ_ACTION_INUSE_CONT) { if(serializedObj->ssize != sizeof(TqSerializedHead)) { - pMeta->deserializer(serializedObj, &pNode->handle.valueInUse); + pMeta->pDeserializer(serializedObj, &pNode->handle.valueInUse); } else { pNode->handle.valueInUse = TQ_DELETE_TOKEN; } TqSerializedHead* ptr = POINTER_SHIFT(serializedObj, serializedObj->ssize); if(ptr->ssize != sizeof(TqSerializedHead)) { - pMeta->deserializer(ptr, &pNode->handle.valueInTxn); + pMeta->pDeserializer(ptr, &pNode->handle.valueInTxn); } else { pNode->handle.valueInTxn = TQ_DELETE_TOKEN; } @@ -225,11 +225,11 @@ TqMetaStore* tqStoreOpen(const char* path, if(pBucketNode) { if(pBucketNode->handle.valueInUse && pBucketNode->handle.valueInUse != TQ_DELETE_TOKEN) { - pMeta->deleter(pBucketNode->handle.valueInUse); + pMeta->pDeleter(pBucketNode->handle.valueInUse); } if(pBucketNode->handle.valueInTxn && pBucketNode->handle.valueInTxn != TQ_DELETE_TOKEN) { - pMeta->deleter(pBucketNode->handle.valueInTxn); + pMeta->pDeleter(pBucketNode->handle.valueInTxn); } free(pBucketNode); } @@ -253,11 +253,11 @@ int32_t tqStoreClose(TqMetaStore* pMeta) { ASSERT(pNode->unpersistPrev == NULL); if(pNode->handle.valueInTxn && pNode->handle.valueInTxn != TQ_DELETE_TOKEN) { - pMeta->deleter(pNode->handle.valueInTxn); + pMeta->pDeleter(pNode->handle.valueInTxn); } if(pNode->handle.valueInUse && pNode->handle.valueInUse != TQ_DELETE_TOKEN) { - pMeta->deleter(pNode->handle.valueInUse); + pMeta->pDeleter(pNode->handle.valueInUse); } TqMetaList* next = pNode->next; free(pNode); @@ -280,11 +280,11 @@ int32_t tqStoreDelete(TqMetaStore* pMeta) { while(pNode) { if(pNode->handle.valueInTxn && pNode->handle.valueInTxn != TQ_DELETE_TOKEN) { - pMeta->deleter(pNode->handle.valueInTxn); + pMeta->pDeleter(pNode->handle.valueInTxn); } if(pNode->handle.valueInUse && pNode->handle.valueInUse != TQ_DELETE_TOKEN) { - pMeta->deleter(pNode->handle.valueInUse); + pMeta->pDeleter(pNode->handle.valueInUse); } TqMetaList* next = pNode->next; free(pNode); @@ -338,7 +338,7 @@ int32_t tqStorePersist(TqMetaStore* pMeta) { if(pNode->handle.valueInUse == TQ_DELETE_TOKEN) { pSHead->ssize = sizeof(TqSerializedHead); } else { - pMeta->serializer(pNode->handle.valueInUse, &pSHead); + pMeta->pSerializer(pNode->handle.valueInUse, &pSHead); } nBytes = write(pMeta->fileFd, pSHead, pSHead->ssize); ASSERT(nBytes == pSHead->ssize); @@ -349,7 +349,7 @@ int32_t tqStorePersist(TqMetaStore* pMeta) { if(pNode->handle.valueInTxn == TQ_DELETE_TOKEN) { pSHead->ssize = sizeof(TqSerializedHead); } else { - pMeta->serializer(pNode->handle.valueInTxn, &pSHead); + pMeta->pSerializer(pNode->handle.valueInTxn, &pSHead); } int nBytesTxn = write(pMeta->fileFd, pSHead, pSHead->ssize); ASSERT(nBytesTxn == pSHead->ssize); @@ -423,7 +423,7 @@ static int32_t tqHandlePutCommitted(TqMetaStore* pMeta, int64_t key, void* value //TODO: think about thread safety if(pNode->handle.valueInUse && pNode->handle.valueInUse != TQ_DELETE_TOKEN) { - pMeta->deleter(pNode->handle.valueInUse); + pMeta->pDeleter(pNode->handle.valueInUse); } //change pointer ownership pNode->handle.valueInUse = value; @@ -496,7 +496,7 @@ static inline int32_t tqHandlePutImpl(TqMetaStore* pMeta, int64_t key, void* val return -2; } if(pNode->handle.valueInTxn != TQ_DELETE_TOKEN) { - pMeta->deleter(pNode->handle.valueInTxn); + pMeta->pDeleter(pNode->handle.valueInTxn); } } pNode->handle.valueInTxn = value; @@ -562,7 +562,7 @@ int32_t tqHandleCommit(TqMetaStore* pMeta, int64_t key) { } if(pNode->handle.valueInUse && pNode->handle.valueInUse != TQ_DELETE_TOKEN) { - pMeta->deleter(pNode->handle.valueInUse); + pMeta->pDeleter(pNode->handle.valueInUse); } pNode->handle.valueInUse = pNode->handle.valueInTxn; pNode->handle.valueInTxn = NULL; @@ -582,7 +582,7 @@ int32_t tqHandleAbort(TqMetaStore* pMeta, int64_t key) { if(pNode->handle.key == key) { if(pNode->handle.valueInTxn) { if(pNode->handle.valueInTxn != TQ_DELETE_TOKEN) { - pMeta->deleter(pNode->handle.valueInTxn); + pMeta->pDeleter(pNode->handle.valueInTxn); } pNode->handle.valueInTxn = NULL; tqLinkUnpersist(pMeta, pNode); @@ -602,7 +602,7 @@ int32_t tqHandleDel(TqMetaStore* pMeta, int64_t key) { while(pNode) { if(pNode->handle.valueInTxn != TQ_DELETE_TOKEN) { if(pNode->handle.valueInTxn) { - pMeta->deleter(pNode->handle.valueInTxn); + pMeta->pDeleter(pNode->handle.valueInTxn); } pNode->handle.valueInTxn = TQ_DELETE_TOKEN; tqLinkUnpersist(pMeta, pNode); diff --git a/source/dnode/vnode/tq/test/tqSerializerTest.cpp b/source/dnode/vnode/tq/test/tqSerializerTest.cpp new file mode 100644 index 0000000000..0d76322c17 --- /dev/null +++ b/source/dnode/vnode/tq/test/tqSerializerTest.cpp @@ -0,0 +1,13 @@ +#include +#include +#include +#include + +#include "tq.h" + +using namespace std; + +TEST(TqSerializerTest, basicTest) { + TqGroupHandle* gHandle = (TqGroupHandle*)malloc(sizeof(TqGroupHandle)); + +} diff --git a/source/libs/wal/src/wal.c b/source/libs/wal/src/wal.c index f25c127f3f..c107a94f3f 100644 --- a/source/libs/wal/src/wal.c +++ b/source/libs/wal/src/wal.c @@ -19,11 +19,19 @@ int32_t walInit() { return 0; } void walCleanUp() {} -SWal *walOpen(char *path, SWalCfg *pCfg) { return NULL; } +SWal *walOpen(char *path, SWalCfg *pCfg) { + SWal* pWal = malloc(sizeof(SWal)); + if(pWal == NULL) { + return NULL; + } + return pWal; +} int32_t walAlter(SWal *pWal, SWalCfg *pCfg) { return 0; } -void walClose(SWal *pWal) {} +void walClose(SWal *pWal) { + if(pWal) free(pWal); +} void walFsync(SWal *pWal, bool force) {} From 044b02bb60d5e3853ecf36f4c25f13eee61c4dda Mon Sep 17 00:00:00 2001 From: Liu Jicong Date: Tue, 23 Nov 2021 15:46:23 +0800 Subject: [PATCH 15/44] add wal implementation --- include/libs/wal/wal.h | 48 ++++++- include/os/os.h | 1 + include/util/tlog.h | 2 + include/util/tnote.h | 2 +- include/util/ulog.h | 15 ++- source/libs/wal/CMakeLists.txt | 1 + source/libs/wal/inc/walInt.h | 4 +- source/libs/wal/src/wal.c | 55 ++++---- {src => source/libs}/wal/src/walMgmt.c | 25 ++-- {src => source/libs}/wal/src/walUtil.c | 2 +- {src => source/libs}/wal/src/walWrite.c | 170 ++++-------------------- source/libs/wal/test/walTests.cpp | 137 +++++++++++++++++++ source/util/src/tconfig.c | 6 +- source/util/src/tlog.c | 2 + src/wal/inc/walInt.h | 68 ---------- src/wal/test/waltest.c | 137 ------------------- 16 files changed, 265 insertions(+), 410 deletions(-) rename {src => source/libs}/wal/src/walMgmt.c (91%) rename {src => source/libs}/wal/src/walUtil.c (99%) rename {src => source/libs}/wal/src/walWrite.c (73%) delete mode 100644 src/wal/inc/walInt.h delete mode 100644 src/wal/test/waltest.c diff --git a/include/libs/wal/wal.h b/include/libs/wal/wal.h index ba37e6880b..94346d705e 100644 --- a/include/libs/wal/wal.h +++ b/include/libs/wal/wal.h @@ -16,11 +16,21 @@ #define _TD_WAL_H_ #include "os.h" - +#include "tdef.h" +#include "tlog.h" #ifdef __cplusplus extern "C" { #endif +extern int32_t wDebugFlag; + +#define wFatal(...) { if (wDebugFlag & DEBUG_FATAL) { taosPrintLog("WAL FATAL ", 255, __VA_ARGS__); }} +#define wError(...) { if (wDebugFlag & DEBUG_ERROR) { taosPrintLog("WAL ERROR ", 255, __VA_ARGS__); }} +#define wWarn(...) { if (wDebugFlag & DEBUG_WARN) { taosPrintLog("WAL WARN ", 255, __VA_ARGS__); }} +#define wInfo(...) { if (wDebugFlag & DEBUG_INFO) { taosPrintLog("WAL ", 255, __VA_ARGS__); }} +#define wDebug(...) { if (wDebugFlag & DEBUG_DEBUG) { taosPrintLog("WAL ", wDebugFlag, __VA_ARGS__); }} +#define wTrace(...) { if (wDebugFlag & DEBUG_TRACE) { taosPrintLog("WAL ", wDebugFlag, __VA_ARGS__); }} + typedef enum { TAOS_WAL_NOLOG = 0, TAOS_WAL_WRITE = 1, @@ -28,9 +38,8 @@ typedef enum { } EWalType; typedef struct { - int8_t msgType; - int8_t sver; // sver 2 for WAL SDataRow/SMemRow compatibility - int8_t reserved[2]; + int8_t sver; + int8_t reserved[3]; int32_t len; int64_t version; uint32_t signature; @@ -44,11 +53,33 @@ typedef struct { EWalType walLevel; // wal level } SWalCfg; +#define WAL_PREFIX "wal" +#define WAL_PREFIX_LEN 3 +#define WAL_REFRESH_MS 1000 +#define WAL_MAX_SIZE (TSDB_MAX_WAL_SIZE + sizeof(SWalHead) + 16) +#define WAL_SIGNATURE ((uint32_t)(0xFAFBFDFE)) +#define WAL_PATH_LEN (TSDB_FILENAME_LEN + 12) +#define WAL_FILE_LEN (WAL_PATH_LEN + 32) +#define WAL_FILE_NUM 1 // 3 + typedef struct SWal { - int8_t unused; + int64_t version; + int64_t fileId; + int64_t rId; + int64_t tfd; + int32_t vgId; + int32_t keep; + int32_t level; + int32_t fsyncPeriod; + int32_t fsyncSeq; + int8_t stop; + int8_t reseved[3]; + char path[WAL_PATH_LEN]; + char name[WAL_FILE_LEN]; + pthread_mutex_t mutex; } SWal; // WAL HANDLE -typedef int32_t (*FWalWrite)(void *ahandle, void *pHead, int32_t qtype, void *pMsg); +typedef int32_t (*FWalWrite)(void *ahandle, void *pHead, void *pMsg); // module initialization int32_t walInit(); @@ -82,6 +113,11 @@ int64_t walGetSnapshotVer(SWal *); int64_t walGetLastVer(SWal *); // int32_t walDataCorrupted(SWal*); +//internal +int32_t walGetNextFile(SWal *pWal, int64_t *nextFileId); +int32_t walGetOldFile(SWal *pWal, int64_t curFileId, int32_t minDiff, int64_t *oldFileId); +int32_t walGetNewFile(SWal *pWal, int64_t *newFileId); + #ifdef __cplusplus } #endif diff --git a/include/os/os.h b/include/os/os.h index 44ce6f81ec..53a6cef96a 100644 --- a/include/os/os.h +++ b/include/os/os.h @@ -46,6 +46,7 @@ extern "C" { #include #include #include +#include #include #include "osAtomic.h" diff --git a/include/util/tlog.h b/include/util/tlog.h index 5c6e59b103..2ee60e4324 100644 --- a/include/util/tlog.h +++ b/include/util/tlog.h @@ -16,6 +16,8 @@ #ifndef _TD_UTIL_LOG_H #define _TD_UTIL_LOG_H +#include "os.h" + #ifdef __cplusplus extern "C" { #endif diff --git a/include/util/tnote.h b/include/util/tnote.h index e4f28d8cff..f17857a32b 100644 --- a/include/util/tnote.h +++ b/include/util/tnote.h @@ -53,7 +53,7 @@ void taosNotePrintBuffer(SNoteObj *pNote, char *buffer, int32_t len); } #define nInfo(buffer, len) \ - if (tscEmbedded == 1) { \ + if (tscEmbeddedInUtil == 1) { \ taosNotePrintBuffer(&tsInfoNote, buffer, len); \ } diff --git a/include/util/ulog.h b/include/util/ulog.h index ba59f4eb79..89d9f89476 100644 --- a/include/util/ulog.h +++ b/include/util/ulog.h @@ -20,20 +20,21 @@ extern "C" { #endif +#include "os.h" #include "tlog.h" extern int32_t uDebugFlag; -extern int8_t tscEmbedded; +extern int8_t tscEmbeddedInUtil; -#define uFatal(...) { if (uDebugFlag & DEBUG_FATAL) { taosPrintLog("UTL FATAL", tscEmbedded ? 255 : uDebugFlag, __VA_ARGS__); }} -#define uError(...) { if (uDebugFlag & DEBUG_ERROR) { taosPrintLog("UTL ERROR ", tscEmbedded ? 255 : uDebugFlag, __VA_ARGS__); }} -#define uWarn(...) { if (uDebugFlag & DEBUG_WARN) { taosPrintLog("UTL WARN ", tscEmbedded ? 255 : uDebugFlag, __VA_ARGS__); }} -#define uInfo(...) { if (uDebugFlag & DEBUG_INFO) { taosPrintLog("UTL ", tscEmbedded ? 255 : uDebugFlag, __VA_ARGS__); }} +#define uFatal(...) { if (uDebugFlag & DEBUG_FATAL) { taosPrintLog("UTL FATAL", tscEmbeddedInUtil ? 255 : uDebugFlag, __VA_ARGS__); }} +#define uError(...) { if (uDebugFlag & DEBUG_ERROR) { taosPrintLog("UTL ERROR ", tscEmbeddedInUtil ? 255 : uDebugFlag, __VA_ARGS__); }} +#define uWarn(...) { if (uDebugFlag & DEBUG_WARN) { taosPrintLog("UTL WARN ", tscEmbeddedInUtil ? 255 : uDebugFlag, __VA_ARGS__); }} +#define uInfo(...) { if (uDebugFlag & DEBUG_INFO) { taosPrintLog("UTL ", tscEmbeddedInUtil ? 255 : uDebugFlag, __VA_ARGS__); }} #define uDebug(...) { if (uDebugFlag & DEBUG_DEBUG) { taosPrintLog("UTL ", uDebugFlag, __VA_ARGS__); }} #define uTrace(...) { if (uDebugFlag & DEBUG_TRACE) { taosPrintLog("UTL ", uDebugFlag, __VA_ARGS__); }} -#define pError(...) { taosPrintLog("APP ERROR ", 255, __VA_ARGS__); } -#define pPrint(...) { taosPrintLog("APP ", 255, __VA_ARGS__); } +#define pError(...) { taosPrintLog("APP ERROR ", 255, __VA_ARGS__); } +#define pPrint(...) { taosPrintLog("APP ", 255, __VA_ARGS__); } #ifdef __cplusplus } diff --git a/source/libs/wal/CMakeLists.txt b/source/libs/wal/CMakeLists.txt index 4af8bac7f9..e5697415f1 100644 --- a/source/libs/wal/CMakeLists.txt +++ b/source/libs/wal/CMakeLists.txt @@ -9,4 +9,5 @@ target_include_directories( target_link_libraries( wal PUBLIC os + PUBLIC util ) diff --git a/source/libs/wal/inc/walInt.h b/source/libs/wal/inc/walInt.h index 3cf38a5ffc..f5f944b12b 100644 --- a/source/libs/wal/inc/walInt.h +++ b/source/libs/wal/inc/walInt.h @@ -16,6 +16,8 @@ #ifndef _TD_WAL_INT_H_ #define _TD_WAL_INT_H_ +#include "wal.h" + #ifdef __cplusplus extern "C" { #endif @@ -24,4 +26,4 @@ extern "C" { } #endif -#endif /*_TD_WAL_INT_H_*/ \ No newline at end of file +#endif /*_TD_WAL_INT_H_*/ diff --git a/source/libs/wal/src/wal.c b/source/libs/wal/src/wal.c index c107a94f3f..05d81e0867 100644 --- a/source/libs/wal/src/wal.c +++ b/source/libs/wal/src/wal.c @@ -15,40 +15,35 @@ #include "wal.h" -int32_t walInit() { return 0; } - -void walCleanUp() {} - -SWal *walOpen(char *path, SWalCfg *pCfg) { - SWal* pWal = malloc(sizeof(SWal)); - if(pWal == NULL) { - return NULL; - } - return pWal; -} - -int32_t walAlter(SWal *pWal, SWalCfg *pCfg) { return 0; } - -void walClose(SWal *pWal) { - if(pWal) free(pWal); -} - -void walFsync(SWal *pWal, bool force) {} - -int64_t walWrite(SWal *pWal, int64_t index, void *body, int32_t bodyLen) { +int32_t walCommit(SWal *pWal, int64_t ver) { return 0; } -int32_t walCommit(SWal *pWal, int64_t ver) { return 0; } +int32_t walRollback(SWal *pWal, int64_t ver) { + return 0; +} -int32_t walRollback(SWal *pWal, int64_t ver) { return 0; } - -int32_t walPrune(SWal *pWal, int64_t ver) { return 0; } +int32_t walPrune(SWal *pWal, int64_t ver) { + return 0; +} -int32_t walRead(SWal *, SWalHead **, int64_t ver); -int32_t walReadWithFp(SWal *, FWalWrite writeFp, int64_t verStart, int32_t readNum); +int32_t walRead(SWal *pWal, SWalHead **ppHead, int64_t ver) { + return 0; +} -int64_t walGetFirstVer(SWal *); -int64_t walGetSnapshotVer(SWal *); -int64_t walGetLastVer(SWal *); +int32_t walReadWithFp(SWal *pWal, FWalWrite writeFp, int64_t verStart, int32_t readNum) { + return 0; +} + +int64_t walGetFirstVer(SWal *pWal) { + return 0; +} + +int64_t walGetSnapshotVer(SWal *pWal) { + return 0; +} + +int64_t walGetLastVer(SWal *pWal) { + return 0; +} diff --git a/src/wal/src/walMgmt.c b/source/libs/wal/src/walMgmt.c similarity index 91% rename from src/wal/src/walMgmt.c rename to source/libs/wal/src/walMgmt.c index dbff08d730..2bc12b374c 100644 --- a/src/wal/src/walMgmt.c +++ b/source/libs/wal/src/walMgmt.c @@ -18,7 +18,6 @@ #include "taoserror.h" #include "tref.h" #include "tfile.h" -#include "twal.h" #include "walInt.h" typedef struct { @@ -62,8 +61,8 @@ void walCleanUp() { wInfo("wal module is cleaned up"); } -void *walOpen(char *path, SWalCfg *pCfg) { - SWal *pWal = tcalloc(1, sizeof(SWal)); +SWal *walOpen(char *path, SWalCfg *pCfg) { + SWal *pWal = malloc(sizeof(SWal)); if (pWal == NULL) { terrno = TAOS_SYSTEM_ERROR(errno); return NULL; @@ -73,7 +72,7 @@ void *walOpen(char *path, SWalCfg *pCfg) { pWal->tfd = -1; pWal->fileId = -1; pWal->level = pCfg->walLevel; - pWal->keep = pCfg->keep; + /*pWal->keep = pCfg->keep;*/ pWal->fsyncPeriod = pCfg->fsyncPeriod; tstrncpy(pWal->path, path, sizeof(pWal->path)); pthread_mutex_init(&pWal->mutex, NULL); @@ -86,8 +85,8 @@ void *walOpen(char *path, SWalCfg *pCfg) { return NULL; } - pWal->rid = taosAddRef(tsWal.refId, pWal); - if (pWal->rid < 0) { + pWal->rId = taosAddRef(tsWal.refId, pWal); + if (pWal->rId < 0) { walFreeObj(pWal); return NULL; } @@ -97,9 +96,8 @@ void *walOpen(char *path, SWalCfg *pCfg) { return pWal; } -int32_t walAlter(void *handle, SWalCfg *pCfg) { - if (handle == NULL) return TSDB_CODE_WAL_APP_ERROR; - SWal *pWal = handle; +int32_t walAlter(SWal *pWal, SWalCfg *pCfg) { + if (pWal == NULL) return TSDB_CODE_WAL_APP_ERROR; if (pWal->level == pCfg->walLevel && pWal->fsyncPeriod == pCfg->fsyncPeriod) { wDebug("vgId:%d, old walLevel:%d fsync:%d, new walLevel:%d fsync:%d not change", pWal->vgId, pWal->level, @@ -128,14 +126,13 @@ void walStop(void *handle) { wDebug("vgId:%d, stop write wal", pWal->vgId); } -void walClose(void *handle) { - if (handle == NULL) return; +void walClose(SWal *pWal) { + if (pWal == NULL) return; - SWal *pWal = handle; pthread_mutex_lock(&pWal->mutex); tfClose(pWal->tfd); pthread_mutex_unlock(&pWal->mutex); - taosRemoveRef(tsWal.refId, pWal->rid); + taosRemoveRef(tsWal.refId, pWal->rId); } static int32_t walInitObj(SWal *pWal) { @@ -186,7 +183,7 @@ static void walFsyncAll() { wError("vgId:%d, file:%s, failed to fsync since %s", pWal->vgId, pWal->name, strerror(code)); } } - pWal = taosIterateRef(tsWal.refId, pWal->rid); + pWal = taosIterateRef(tsWal.refId, pWal->rId); } } diff --git a/src/wal/src/walUtil.c b/source/libs/wal/src/walUtil.c similarity index 99% rename from src/wal/src/walUtil.c rename to source/libs/wal/src/walUtil.c index e4d9a555b3..c88cc918fe 100644 --- a/src/wal/src/walUtil.c +++ b/source/libs/wal/src/walUtil.c @@ -115,4 +115,4 @@ int32_t walGetNewFile(SWal *pWal, int64_t *newFileId) { wTrace("vgId:%d, path:%s, newFileId:%" PRId64, pWal->vgId, pWal->path, *newFileId); return 0; -} \ No newline at end of file +} diff --git a/src/wal/src/walWrite.c b/source/libs/wal/src/walWrite.c similarity index 73% rename from src/wal/src/walWrite.c rename to source/libs/wal/src/walWrite.c index cae4291eb8..023b1c4a48 100644 --- a/src/wal/src/walWrite.c +++ b/source/libs/wal/src/walWrite.c @@ -14,13 +14,11 @@ */ #define _DEFAULT_SOURCE -#define TAOS_RANDOM_FILE_FAIL_TEST + #include "os.h" #include "taoserror.h" -#include "taosmsg.h" #include "tchecksum.h" #include "tfile.h" -#include "twal.h" #include "walInt.h" static int32_t walRestoreWalFile(SWal *pWal, void *pVnode, FWalWrite writeFp, char *name, int64_t fileId); @@ -43,12 +41,12 @@ int32_t walRenew(void *handle) { wDebug("vgId:%d, file:%s, it is closed while renew", pWal->vgId, pWal->name); } - if (pWal->keep == TAOS_WAL_KEEP) { - pWal->fileId = 0; - } else { - if (walGetNewFile(pWal, &pWal->fileId) != 0) pWal->fileId = 0; - pWal->fileId++; - } + /*if (pWal->keep == TAOS_WAL_KEEP) {*/ + /*pWal->fileId = 0;*/ + /*} else {*/ + /*if (walGetNewFile(pWal, &pWal->fileId) != 0) pWal->fileId = 0;*/ + /*pWal->fileId++;*/ + /*}*/ snprintf(pWal->name, sizeof(pWal->name), "%s/%s%" PRId64, pWal->path, WAL_PREFIX, pWal->fileId); pWal->tfd = tfOpenCreateWrite(pWal->name); @@ -68,7 +66,7 @@ int32_t walRenew(void *handle) { void walRemoveOneOldFile(void *handle) { SWal *pWal = handle; if (pWal == NULL) return; - if (pWal->keep == TAOS_WAL_KEEP) return; + /*if (pWal->keep == TAOS_WAL_KEEP) return;*/ if (!tfValid(pWal->tfd)) return; pthread_mutex_lock(&pWal->mutex); @@ -117,7 +115,7 @@ void walRemoveAllOldFiles(void *handle) { static void walUpdateChecksum(SWalHead *pHead) { pHead->sver = 2; pHead->cksum = 0; - pHead->cksum = taosCalcChecksum(0, (uint8_t *)pHead, sizeof(*pHead) + pHead->len); + pHead->cksum = taosCalcChecksum(0, (uint8_t *)pHead, sizeof(SWalHead) + pHead->len); } static int walValidateChecksum(SWalHead *pHead) { @@ -134,10 +132,14 @@ static int walValidateChecksum(SWalHead *pHead) { #endif -int32_t walWrite(void *handle, SWalHead *pHead) { - if (handle == NULL) return -1; +int64_t walWrite(SWal *pWal, int64_t index, void *body, int32_t bodyLen) { + if (pWal == NULL) return -1; - SWal * pWal = handle; + SWalHead *pHead = malloc(sizeof(SWalHead) + bodyLen); + if(pHead == NULL) { + return -1; + } + pHead->version = index; int32_t code = 0; // no wal @@ -146,6 +148,9 @@ int32_t walWrite(void *handle, SWalHead *pHead) { if (pHead->version <= pWal->version) return 0; pHead->signature = WAL_SIGNATURE; + pHead->len = bodyLen; + memcpy(pHead->cont, body, bodyLen); + #if defined(WAL_CHECKSUM_WHOLE) walUpdateChecksum(pHead); #else @@ -173,8 +178,7 @@ int32_t walWrite(void *handle, SWalHead *pHead) { return code; } -void walFsync(void *handle, bool forceFsync) { - SWal *pWal = handle; +void walFsync(SWal *pWal, bool forceFsync) { if (pWal == NULL || !tfValid(pWal->tfd)) return; if (forceFsync || (pWal->level == TAOS_WAL_FSYNC && pWal->fsyncPeriod == 0)) { @@ -211,7 +215,7 @@ int32_t walRestore(void *handle, void *pVnode, FWalWrite writeFp) { count++; } - if (pWal->keep != TAOS_WAL_KEEP) return TSDB_CODE_SUCCESS; + /*if (pWal->keep != TAOS_WAL_KEEP) return TSDB_CODE_SUCCESS;*/ if (count == 0) { wDebug("vgId:%d, wal file not exist, renew it", pWal->vgId); @@ -307,119 +311,10 @@ static int32_t walSkipCorruptedRecord(SWal *pWal, SWalHead *pHead, int64_t tfd, return TSDB_CODE_WAL_FILE_CORRUPTED; } -// Add SMemRowType ahead of SDataRow -static void expandSubmitBlk(SSubmitBlk *pDest, SSubmitBlk *pSrc, int32_t *lenExpand) { - // copy the header firstly - memcpy(pDest, pSrc, sizeof(SSubmitBlk)); - - int32_t nRows = htons(pDest->numOfRows); - int32_t dataLen = htonl(pDest->dataLen); - - if ((nRows <= 0) || (dataLen <= 0)) { - return; - } - - char *pDestData = pDest->data; - char *pSrcData = pSrc->data; - for (int32_t i = 0; i < nRows; ++i) { - memRowSetType(pDestData, SMEM_ROW_DATA); - memcpy(memRowDataBody(pDestData), pSrcData, dataRowLen(pSrcData)); - pDestData = POINTER_SHIFT(pDestData, memRowTLen(pDestData)); - pSrcData = POINTER_SHIFT(pSrcData, dataRowLen(pSrcData)); - ++(*lenExpand); - } - pDest->dataLen = htonl(dataLen + nRows * sizeof(uint8_t)); -} - -// Check SDataRow by comparing the SDataRow len and SSubmitBlk dataLen -static bool walIsSDataRow(void *pBlkData, int nRows, int32_t dataLen) { - if ((nRows <= 0) || (dataLen <= 0)) { - return true; - } - int32_t len = 0, kvLen = 0; - for (int i = 0; i < nRows; ++i) { - len += dataRowLen(pBlkData); - if (len > dataLen) { - return false; - } - - /** - * For SDataRow between version [2.1.5.0 and 2.1.6.X], it would never conflict. - * For SKVRow between version [2.1.5.0 and 2.1.6.X], it may conflict in below scenario - * - with 1st type byte 0x01 and sversion 0x0101(257), thus do further check - */ - if (dataRowLen(pBlkData) == 257) { - SMemRow memRow = pBlkData; - SKVRow kvRow = memRowKvBody(memRow); - int nCols = kvRowNCols(kvRow); - uint16_t calcTsOffset = (uint16_t)(TD_KV_ROW_HEAD_SIZE + sizeof(SColIdx) * nCols); - uint16_t realTsOffset = (kvRowColIdx(kvRow))->offset; - if (calcTsOffset == realTsOffset) { - kvLen += memRowKvTLen(memRow); - } - } - pBlkData = POINTER_SHIFT(pBlkData, dataRowLen(pBlkData)); - } - if (len != dataLen) { - return false; - } - if (kvLen == dataLen) { - return false; - } - return true; -} -// for WAL SMemRow/SDataRow compatibility -static int walSMemRowCheck(SWalHead *pHead) { - if ((pHead->sver < 2) && (pHead->msgType == TSDB_MSG_TYPE_SUBMIT)) { - SSubmitMsg *pMsg = (SSubmitMsg *)pHead->cont; - int32_t numOfBlocks = htonl(pMsg->numOfBlocks); - if (numOfBlocks <= 0) { - return 0; - } - - int32_t nTotalRows = 0; - SSubmitBlk *pBlk = (SSubmitBlk *)pMsg->blocks; - for (int32_t i = 0; i < numOfBlocks; ++i) { - int32_t dataLen = htonl(pBlk->dataLen); - int32_t nRows = htons(pBlk->numOfRows); - nTotalRows += nRows; - if (!walIsSDataRow(pBlk->data, nRows, dataLen)) { - return 0; - } - pBlk = (SSubmitBlk *)POINTER_SHIFT(pBlk, sizeof(SSubmitBlk) + dataLen); - } - ASSERT(nTotalRows >= 0); - SWalHead *pWalHead = (SWalHead *)calloc(sizeof(SWalHead) + pHead->len + nTotalRows * sizeof(uint8_t), 1); - if (pWalHead == NULL) { - return -1; - } - - memcpy(pWalHead, pHead, sizeof(SWalHead) + sizeof(SSubmitMsg)); - - SSubmitMsg *pDestMsg = (SSubmitMsg *)pWalHead->cont; - SSubmitBlk *pDestBlks = (SSubmitBlk *)pDestMsg->blocks; - SSubmitBlk *pSrcBlks = (SSubmitBlk *)pMsg->blocks; - int32_t lenExpand = 0; - for (int32_t i = 0; i < numOfBlocks; ++i) { - expandSubmitBlk(pDestBlks, pSrcBlks, &lenExpand); - pDestBlks = POINTER_SHIFT(pDestBlks, htonl(pDestBlks->dataLen) + sizeof(SSubmitBlk)); - pSrcBlks = POINTER_SHIFT(pSrcBlks, htonl(pSrcBlks->dataLen) + sizeof(SSubmitBlk)); - } - if (lenExpand > 0) { - pDestMsg->header.contLen = htonl(pDestMsg->length) + lenExpand; - pDestMsg->length = htonl(pDestMsg->header.contLen); - pWalHead->len = pWalHead->len + lenExpand; - } - - memcpy(pHead, pWalHead, sizeof(SWalHead) + pWalHead->len); - tfree(pWalHead); - } - return 0; -} static int32_t walRestoreWalFile(SWal *pWal, void *pVnode, FWalWrite writeFp, char *name, int64_t fileId) { int32_t size = WAL_MAX_SIZE; - void * buffer = tmalloc(size); + void * buffer = malloc(size); if (buffer == NULL) { wError("vgId:%d, file:%s, failed to open for restore since %s", pWal->vgId, name, strerror(errno)); return TAOS_SYSTEM_ERROR(errno); @@ -541,14 +436,7 @@ static int32_t walRestoreWalFile(SWal *pWal, void *pVnode, FWalWrite writeFp, ch pWal->version = pHead->version; // wInfo("writeFp: %ld", offset); - if (0 != walSMemRowCheck(pHead)) { - wError("vgId:%d, restore wal, fileId:%" PRId64 " hver:%" PRIu64 " wver:%" PRIu64 " len:%d offset:%" PRId64, - pWal->vgId, fileId, pHead->version, pWal->version, pHead->len, offset); - tfClose(tfd); - tfree(buffer); - return TAOS_SYSTEM_ERROR(errno); - } - (*writeFp)(pVnode, pHead, TAOS_QTYPE_WAL, NULL); + (*writeFp)(pVnode, pHead, NULL); } tfClose(tfd); @@ -558,9 +446,8 @@ static int32_t walRestoreWalFile(SWal *pWal, void *pVnode, FWalWrite writeFp, ch return code; } -uint64_t walGetVersion(twalh param) { - SWal *pWal = param; - if (pWal == 0) return 0; +uint64_t walGetVersion(SWal *pWal) { + if (pWal == NULL) return 0; return pWal->version; } @@ -570,10 +457,9 @@ uint64_t walGetVersion(twalh param) { // Some new wal record cannot be written to the wal file in dnode1 for wal version not reset, then fversion and the record in wal file may inconsistent, // At this time, if dnode2 down, dnode1 switched to master. After dnode2 start and restore data from dnode1, data loss will occur -void walResetVersion(twalh param, uint64_t newVer) { - SWal *pWal = param; - if (pWal == 0) return; +void walResetVersion(SWal *pWal, uint64_t newVer) { + if (pWal == NULL) return; wInfo("vgId:%d, version reset from %" PRIu64 " to %" PRIu64, pWal->vgId, pWal->version, newVer); pWal->version = newVer; -} \ No newline at end of file +} diff --git a/source/libs/wal/test/walTests.cpp b/source/libs/wal/test/walTests.cpp index e69de29bb2..505728fbe4 100644 --- a/source/libs/wal/test/walTests.cpp +++ b/source/libs/wal/test/walTests.cpp @@ -0,0 +1,137 @@ +/* + * Copyright (c) 2019 TAOS Data, Inc. + * + * This program is free software: you can use, redistribute, and/or modify + * it under the terms of the GNU Affero General Public License, version 3 + * or later ("AGPL"), as published by the Free Software Foundation. + * + * This program is distributed in the hope that it will be useful, but WITHOUT + * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or + * FITNESS FOR A PARTICULAR PURPOSE. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + */ + +//#define _DEFAULT_SOURCE +#include "os.h" +#include "tutil.h" +#include "tglobal.h" +#include "tlog.h" +#include "twal.h" +#include "tfile.h" + +int64_t ver = 0; +void *pWal = NULL; + +int writeToQueue(void *pVnode, void *data, int type, void *pMsg) { + // do nothing + SWalHead *pHead = data; + + if (pHead->version > ver) + ver = pHead->version; + + walWrite(pWal, pHead); + + return 0; +} + +int main(int argc, char *argv[]) { + char path[128] = "/tmp/wal"; + int level = 2; + int total = 5; + int rows = 10000; + int size = 128; + int keep = 0; + + for (int i=1; iversion = ++ver; + pHead->len = size; + walWrite(pWal, pHead); + } + + printf("renew a wal, i:%d\n", i); + walRenew(pWal); + } + + printf("%d wal files are written\n", total); + + int64_t index = 0; + char name[256]; + + while (1) { + int code = walGetWalFile(pWal, name, &index); + if (code == -1) { + printf("failed to get wal file, index:%" PRId64 "\n", index); + break; + } + + printf("index:%" PRId64 " wal:%s\n", index, name); + if (code == 0) break; + } + + getchar(); + + walClose(pWal); + walCleanUp(); + tfCleanup(); + + return 0; +} diff --git a/source/util/src/tconfig.c b/source/util/src/tconfig.c index 6b3f08a446..1ca29f798a 100644 --- a/source/util/src/tconfig.c +++ b/source/util/src/tconfig.c @@ -402,7 +402,7 @@ void taosPrintGlobalCfg() { for (int i = 0; i < tsGlobalConfigNum; ++i) { SGlobalCfg *cfg = tsGlobalConfig + i; - if (tscEmbedded == 0 && !(cfg->cfgType & TSDB_CFG_CTYPE_B_CLIENT)) continue; + if (tscEmbeddedInUtil == 0 && !(cfg->cfgType & TSDB_CFG_CTYPE_B_CLIENT)) continue; if (cfg->cfgType & TSDB_CFG_CTYPE_B_NOT_PRINT) continue; int optionLen = (int)strlen(cfg->option); @@ -487,7 +487,7 @@ void taosDumpGlobalCfg() { printf("==================================\n"); for (int i = 0; i < tsGlobalConfigNum; ++i) { SGlobalCfg *cfg = tsGlobalConfig + i; - if (tscEmbedded == 0 && !(cfg->cfgType & TSDB_CFG_CTYPE_B_CLIENT)) continue; + if (tscEmbeddedInUtil == 0 && !(cfg->cfgType & TSDB_CFG_CTYPE_B_CLIENT)) continue; if (cfg->cfgType & TSDB_CFG_CTYPE_B_NOT_PRINT) continue; if (!(cfg->cfgType & TSDB_CFG_CTYPE_B_SHOW)) continue; @@ -499,7 +499,7 @@ void taosDumpGlobalCfg() { for (int i = 0; i < tsGlobalConfigNum; ++i) { SGlobalCfg *cfg = tsGlobalConfig + i; - if (tscEmbedded == 0 && !(cfg->cfgType & TSDB_CFG_CTYPE_B_CLIENT)) continue; + if (tscEmbeddedInUtil == 0 && !(cfg->cfgType & TSDB_CFG_CTYPE_B_CLIENT)) continue; if (cfg->cfgType & TSDB_CFG_CTYPE_B_NOT_PRINT) continue; if (cfg->cfgType & TSDB_CFG_CTYPE_B_SHOW) continue; diff --git a/source/util/src/tlog.c b/source/util/src/tlog.c index 2fb84656b6..e6cc3a53af 100644 --- a/source/util/src/tlog.c +++ b/source/util/src/tlog.c @@ -68,6 +68,8 @@ typedef struct { pthread_mutex_t logMutex; } SLogObj; +int8_t tscEmbeddedInUtil = 0; + int32_t tsLogKeepDays = 0; int8_t tsAsyncLog = 1; float tsTotalLogDirGB = 0; diff --git a/src/wal/inc/walInt.h b/src/wal/inc/walInt.h deleted file mode 100644 index 890b404ce9..0000000000 --- a/src/wal/inc/walInt.h +++ /dev/null @@ -1,68 +0,0 @@ -/* - * 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 TDENGINE_WAL_INT_H -#define TDENGINE_WAL_INT_H - -#ifdef __cplusplus -extern "C" { -#endif - -#include "tlog.h" - -extern int32_t wDebugFlag; - -#define wFatal(...) { if (wDebugFlag & DEBUG_FATAL) { taosPrintLog("WAL FATAL ", 255, __VA_ARGS__); }} -#define wError(...) { if (wDebugFlag & DEBUG_ERROR) { taosPrintLog("WAL ERROR ", 255, __VA_ARGS__); }} -#define wWarn(...) { if (wDebugFlag & DEBUG_WARN) { taosPrintLog("WAL WARN ", 255, __VA_ARGS__); }} -#define wInfo(...) { if (wDebugFlag & DEBUG_INFO) { taosPrintLog("WAL ", 255, __VA_ARGS__); }} -#define wDebug(...) { if (wDebugFlag & DEBUG_DEBUG) { taosPrintLog("WAL ", wDebugFlag, __VA_ARGS__); }} -#define wTrace(...) { if (wDebugFlag & DEBUG_TRACE) { taosPrintLog("WAL ", wDebugFlag, __VA_ARGS__); }} - -#define WAL_PREFIX "wal" -#define WAL_PREFIX_LEN 3 -#define WAL_REFRESH_MS 1000 -#define WAL_MAX_SIZE (TSDB_MAX_WAL_SIZE + sizeof(SWalHead) + 16) -#define WAL_SIGNATURE ((uint32_t)(0xFAFBFDFE)) -#define WAL_PATH_LEN (TSDB_FILENAME_LEN + 12) -#define WAL_FILE_LEN (WAL_PATH_LEN + 32) -#define WAL_FILE_NUM 1 // 3 - -typedef struct { - uint64_t version; - int64_t fileId; - int64_t rid; - int64_t tfd; - int32_t vgId; - int32_t keep; - int32_t level; - int32_t fsyncPeriod; - int32_t fsyncSeq; - int8_t stop; - int8_t reserved[3]; - char path[WAL_PATH_LEN]; - char name[WAL_FILE_LEN]; - pthread_mutex_t mutex; -} SWal; - -int32_t walGetNextFile(SWal *pWal, int64_t *nextFileId); -int32_t walGetOldFile(SWal *pWal, int64_t curFileId, int32_t minDiff, int64_t *oldFileId); -int32_t walGetNewFile(SWal *pWal, int64_t *newFileId); - -#ifdef __cplusplus -} -#endif - -#endif diff --git a/src/wal/test/waltest.c b/src/wal/test/waltest.c deleted file mode 100644 index 505728fbe4..0000000000 --- a/src/wal/test/waltest.c +++ /dev/null @@ -1,137 +0,0 @@ -/* - * Copyright (c) 2019 TAOS Data, Inc. - * - * This program is free software: you can use, redistribute, and/or modify - * it under the terms of the GNU Affero General Public License, version 3 - * or later ("AGPL"), as published by the Free Software Foundation. - * - * This program is distributed in the hope that it will be useful, but WITHOUT - * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or - * FITNESS FOR A PARTICULAR PURPOSE. - * - * You should have received a copy of the GNU Affero General Public License - * along with this program. If not, see . - */ - -//#define _DEFAULT_SOURCE -#include "os.h" -#include "tutil.h" -#include "tglobal.h" -#include "tlog.h" -#include "twal.h" -#include "tfile.h" - -int64_t ver = 0; -void *pWal = NULL; - -int writeToQueue(void *pVnode, void *data, int type, void *pMsg) { - // do nothing - SWalHead *pHead = data; - - if (pHead->version > ver) - ver = pHead->version; - - walWrite(pWal, pHead); - - return 0; -} - -int main(int argc, char *argv[]) { - char path[128] = "/tmp/wal"; - int level = 2; - int total = 5; - int rows = 10000; - int size = 128; - int keep = 0; - - for (int i=1; iversion = ++ver; - pHead->len = size; - walWrite(pWal, pHead); - } - - printf("renew a wal, i:%d\n", i); - walRenew(pWal); - } - - printf("%d wal files are written\n", total); - - int64_t index = 0; - char name[256]; - - while (1) { - int code = walGetWalFile(pWal, name, &index); - if (code == -1) { - printf("failed to get wal file, index:%" PRId64 "\n", index); - break; - } - - printf("index:%" PRId64 " wal:%s\n", index, name); - if (code == 0) break; - } - - getchar(); - - walClose(pWal); - walCleanUp(); - tfCleanup(); - - return 0; -} From fa0d64f14b427a61daf62c40736369cae4320ff3 Mon Sep 17 00:00:00 2001 From: Shengliang Guan Date: Mon, 22 Nov 2021 10:03:41 +0800 Subject: [PATCH 16/44] add daemon lib --- include/common/taosmsg.h | 16 +- include/dnode/mgmt/dnode.h | 26 +- include/dnode/mnode/mnode.h | 21 +- include/dnode/vnode/vnode.h | 8 +- include/libs/transport/trpc.h | 10 +- include/util/taoserror.h | 41 +- source/dnode/mgmt/daemon/src/daemon.c | 183 ++- .../impl/inc/{dnodeVnodes.h => dndDnode.h} | 25 +- .../mgmt/impl/inc/{dnodeInt.h => dndInt.h} | 98 +- .../impl/inc/{dnodeDnode.h => dndMnode.h} | 25 +- .../inc/{dnodeTransport.h => dndTransport.h} | 16 +- .../impl/inc/{dnodeMnode.h => dndVnodes.h} | 24 +- .../impl/src/{dnodeDnode.c => dndDnode.c} | 434 ++++--- .../mgmt/impl/src/{dnodeInt.c => dndInt.c} | 145 +-- source/dnode/mgmt/impl/src/dndMnode.c | 816 ++++++++++++ source/dnode/mgmt/impl/src/dndTransport.c | 328 +++++ source/dnode/mgmt/impl/src/dndVnodes.c | 1119 +++++++++++++++++ source/dnode/mgmt/impl/src/dnodeMnode.c | 608 --------- source/dnode/mgmt/impl/src/dnodeTransport.c | 395 ------ source/dnode/mgmt/impl/src/dnodeVnodes.c | 1029 --------------- source/dnode/mnode/impl/inc/mnodeDef.h | 4 +- source/dnode/mnode/impl/src/mnode.c | 8 +- source/libs/transport/src/rpcMain.c | 24 +- source/util/src/terror.c | 25 +- 24 files changed, 2920 insertions(+), 2508 deletions(-) rename source/dnode/mgmt/impl/inc/{dnodeVnodes.h => dndDnode.h} (53%) rename source/dnode/mgmt/impl/inc/{dnodeInt.h => dndInt.h} (54%) rename source/dnode/mgmt/impl/inc/{dnodeDnode.h => dndMnode.h} (53%) rename source/dnode/mgmt/impl/inc/{dnodeTransport.h => dndTransport.h} (68%) rename source/dnode/mgmt/impl/inc/{dnodeMnode.h => dndVnodes.h} (52%) rename source/dnode/mgmt/impl/src/{dnodeDnode.c => dndDnode.c} (51%) rename source/dnode/mgmt/impl/src/{dnodeInt.c => dndInt.c} (56%) create mode 100644 source/dnode/mgmt/impl/src/dndMnode.c create mode 100644 source/dnode/mgmt/impl/src/dndTransport.c create mode 100644 source/dnode/mgmt/impl/src/dndVnodes.c delete mode 100644 source/dnode/mgmt/impl/src/dnodeMnode.c delete mode 100644 source/dnode/mgmt/impl/src/dnodeTransport.c delete mode 100644 source/dnode/mgmt/impl/src/dnodeVnodes.c diff --git a/include/common/taosmsg.h b/include/common/taosmsg.h index cf5a3c8e05..e70a8539c3 100644 --- a/include/common/taosmsg.h +++ b/include/common/taosmsg.h @@ -672,16 +672,16 @@ typedef struct { } SDnodeCfg; typedef struct { - int32_t dnodeId; + int32_t id; int8_t isMnode; int8_t reserved; - uint16_t dnodePort; - char dnodeFqdn[TSDB_FQDN_LEN]; + uint16_t port; + char fqdn[TSDB_FQDN_LEN]; } SDnodeEp; typedef struct { - int32_t dnodeNum; - SDnodeEp dnodeEps[]; + int32_t num; + SDnodeEp eps[]; } SDnodeEps; typedef struct { @@ -820,9 +820,9 @@ typedef struct { } SCreateDnodeMsg, SDropDnodeMsg; typedef struct { - int32_t dnodeId; - int8_t replica; - int8_t reserved[3]; + int32_t dnodeId; + int8_t replica; + int8_t reserved[3]; SReplica replicas[TSDB_MAX_REPLICA]; } SCreateMnodeMsg, SAlterMnodeMsg, SDropMnodeMsg; diff --git a/include/dnode/mgmt/dnode.h b/include/dnode/mgmt/dnode.h index 5002ee37b0..a1a94bb10b 100644 --- a/include/dnode/mgmt/dnode.h +++ b/include/dnode/mgmt/dnode.h @@ -16,6 +16,8 @@ #ifndef _TD_DNODE_H_ #define _TD_DNODE_H_ +#include "tdef.h" + #ifdef __cplusplus extern "C" { #endif @@ -23,6 +25,24 @@ extern "C" { /* ------------------------ TYPES EXPOSED ------------------------ */ typedef struct SDnode SDnode; +typedef struct { + int32_t sver; + int32_t numOfCores; + float numOfThreadsPerCore; + float ratioOfQueryCores; + int32_t maxShellConns; + int32_t shellActivityTimer; + int32_t statusInterval; + uint16_t serverPort; + char dataDir[PATH_MAX]; + char localEp[TSDB_EP_LEN]; + char localFqdn[TSDB_FQDN_LEN]; + char firstEp[TSDB_EP_LEN]; + char timezone[TSDB_TIMEZONE_LEN]; + char locale[TSDB_LOCALE_LEN]; + char charset[TSDB_LOCALE_LEN]; +} SDnodeOpt; + /* ------------------------ SDnode ------------------------ */ /** * @brief Initialize and start the dnode. @@ -30,14 +50,14 @@ typedef struct SDnode SDnode; * @param cfgPath Config file path. * @return SDnode* The dnode object. */ -SDnode *dnodeInit(const char *cfgPath); +SDnode *dndInit(SDnodeOpt *pOptions); /** * @brief Stop and cleanup dnode. * - * @param pDnode The dnode object to close. + * @param pDnd The dnode object to close. */ -void dnodeCleanup(SDnode *pDnode); +void dndCleanup(SDnode *pDnd); #ifdef __cplusplus } diff --git a/include/dnode/mnode/mnode.h b/include/dnode/mnode/mnode.h index 0071296bc1..98aefc6db3 100644 --- a/include/dnode/mnode/mnode.h +++ b/include/dnode/mnode/mnode.h @@ -24,10 +24,10 @@ extern "C" { typedef struct SDnode SDnode; typedef struct SMnode SMnode; typedef struct SMnodeMsg SMnodeMsg; -typedef void (*SendMsgToDnodeFp)(SDnode *pDnode, struct SEpSet *epSet, struct SRpcMsg *rpcMsg); -typedef void (*SendMsgToMnodeFp)(SDnode *pDnode, struct SRpcMsg *rpcMsg); -typedef void (*SendRedirectMsgFp)(SDnode *pDnode, struct SRpcMsg *rpcMsg, bool forShell); -typedef int32_t (*PutMsgToMnodeQFp)(SDnode *pDnode, SMnodeMsg *pMsg); +typedef void (*SendMsgToDnodeFp)(SDnode *pDnd, struct SEpSet *epSet, struct SRpcMsg *rpcMsg); +typedef void (*SendMsgToMnodeFp)(SDnode *pDnd, struct SRpcMsg *rpcMsg); +typedef void (*SendRedirectMsgFp)(SDnode *pDnd, struct SRpcMsg *rpcMsg, bool forShell); +typedef int32_t (*PutMsgToMnodeQFp)(SDnode *pDnd, SMnodeMsg *pMsg); typedef struct SMnodeLoad { int64_t numOfDnode; @@ -48,7 +48,7 @@ typedef struct { int8_t replica; int8_t selfIndex; SReplica replicas[TSDB_MAX_REPLICA]; - struct SServer *pServer; + struct SDnode *pDnode; PutMsgToMnodeQFp putMsgToApplyMsgFp; SendMsgToDnodeFp sendMsgToDnodeFp; SendMsgToMnodeFp sendMsgToMnodeFp; @@ -122,10 +122,17 @@ SMnodeMsg *mnodeInitMsg(SMnode *pMnode, SRpcMsg *pRpcMsg); /** * @brief Cleanup mnode msg * - * @param pMnode The mnode object * @param pMsg The request msg */ -void mnodeCleanupMsg(SMnode *pMnode, SMnodeMsg *pMsg); +void mnodeCleanupMsg(SMnodeMsg *pMsg); + +/** + * @brief Cleanup mnode msg + * + * @param pMsg The request msg + * @param code The error code + */ +void mnodeSendRsp(SMnodeMsg *pMsg, int32_t code); /** * @brief Process the read request diff --git a/include/dnode/vnode/vnode.h b/include/dnode/vnode/vnode.h index 36f6a3b6fb..586cb49d0f 100644 --- a/include/dnode/vnode/vnode.h +++ b/include/dnode/vnode/vnode.h @@ -185,10 +185,10 @@ typedef struct { } SVnodeMsg; typedef struct SDnode SDnode; -typedef void (*SendMsgToDnodeFp)(SDnode *pDnode, struct SEpSet *epSet, struct SRpcMsg *rpcMsg); -typedef void (*SendMsgToMnodeFp)(SDnode *pDnode, struct SRpcMsg *rpcMsg); -typedef void (*SendRedirectMsgFp)(SDnode *pDnode, struct SRpcMsg *rpcMsg, bool forShell); -typedef int32_t (*PutMsgToVnodeQFp)(SDnode *pDnode, int32_t vgId, SVnodeMsg *pMsg); +typedef void (*SendMsgToDnodeFp)(SDnode *pDnd, struct SEpSet *epSet, struct SRpcMsg *rpcMsg); +typedef void (*SendMsgToMnodeFp)(SDnode *pDnd, struct SRpcMsg *rpcMsg); +typedef void (*SendRedirectMsgFp)(SDnode *pDnd, struct SRpcMsg *rpcMsg, bool forShell); +typedef int32_t (*PutMsgToVnodeQFp)(SDnode *pDnd, int32_t vgId, SVnodeMsg *pMsg); typedef struct { PutMsgToVnodeQFp putMsgToApplyQueueFp; diff --git a/include/libs/transport/trpc.h b/include/libs/transport/trpc.h index 5460ae5401..2a8a7aad4b 100644 --- a/include/libs/transport/trpc.h +++ b/include/libs/transport/trpc.h @@ -51,7 +51,7 @@ typedef struct SRpcMsg { } SRpcMsg; typedef struct SRpcInit { - uint16_t localPort; // local port + uint16_t localPort; // local port char *label; // for debug purpose int numOfThreads; // number of threads to handle connections int sessions; // number of sessions allowed @@ -66,10 +66,12 @@ typedef struct SRpcInit { char *ckey; // ciphering key // call back to process incoming msg, code shall be ignored by server app - void (*cfp)(SRpcMsg *, SEpSet *); + void (*cfp)(void *parent, SRpcMsg *, SEpSet *); - // call back to retrieve the client auth info, for server app only - int (*afp)(char *tableId, char *spi, char *encrypt, char *secret, char *ckey); + // call back to retrieve the client auth info, for server app only + int (*afp)(void *parent, char *tableId, char *spi, char *encrypt, char *secret, char *ckey); + + void *parent; } SRpcInit; int32_t rpcInit(); diff --git a/include/util/taoserror.h b/include/util/taoserror.h index 022c3e9096..304fb56a6a 100644 --- a/include/util/taoserror.h +++ b/include/util/taoserror.h @@ -68,12 +68,13 @@ int32_t* taosGetErrno(); #define TSDB_CODE_FILE_CORRUPTED TAOS_DEF_ERROR_CODE(0, 0x0106) #define TSDB_CODE_CHECKSUM_ERROR TAOS_DEF_ERROR_CODE(0, 0x0107) #define TSDB_CODE_INVALID_MSG TAOS_DEF_ERROR_CODE(0, 0x0108) -#define TSDB_CODE_REF_NO_MEMORY TAOS_DEF_ERROR_CODE(0, 0x0109) -#define TSDB_CODE_REF_FULL TAOS_DEF_ERROR_CODE(0, 0x010A) -#define TSDB_CODE_REF_ID_REMOVED TAOS_DEF_ERROR_CODE(0, 0x010B) -#define TSDB_CODE_REF_INVALID_ID TAOS_DEF_ERROR_CODE(0, 0x010C) -#define TSDB_CODE_REF_ALREADY_EXIST TAOS_DEF_ERROR_CODE(0, 0x010D) -#define TSDB_CODE_REF_NOT_EXIST TAOS_DEF_ERROR_CODE(0, 0x010E) +#define TSDB_CODE_MSG_NOT_PROCESSED TAOS_DEF_ERROR_CODE(0, 0x0109) +#define TSDB_CODE_REF_NO_MEMORY TAOS_DEF_ERROR_CODE(0, 0x0110) +#define TSDB_CODE_REF_FULL TAOS_DEF_ERROR_CODE(0, 0x0111) +#define TSDB_CODE_REF_ID_REMOVED TAOS_DEF_ERROR_CODE(0, 0x0112) +#define TSDB_CODE_REF_INVALID_ID TAOS_DEF_ERROR_CODE(0, 0x0113) +#define TSDB_CODE_REF_ALREADY_EXIST TAOS_DEF_ERROR_CODE(0, 0x0114) +#define TSDB_CODE_REF_NOT_EXIST TAOS_DEF_ERROR_CODE(0, 0x0115) //client #define TSDB_CODE_TSC_INVALID_OPERATION TAOS_DEF_ERROR_CODE(0, 0x0200) //"Invalid Operation") @@ -223,20 +224,20 @@ int32_t* taosGetErrno(); #define TSDB_CODE_MND_TOPIC_ALREADY_EXIST TAOS_DEF_ERROR_CODE(0, 0x0395) //"Topic already exists) // dnode -#define TSDB_CODE_DND_MSG_NOT_PROCESSED TAOS_DEF_ERROR_CODE(0, 0x0400) //"Message not processed") -#define TSDB_CODE_DND_OUT_OF_MEMORY TAOS_DEF_ERROR_CODE(0, 0x0401) //"Dnode out of memory") -#define TSDB_CODE_DND_MNODE_ID_NOT_MATCH_DNODE TAOS_DEF_ERROR_CODE(0, 0x0402) //"Mnode Id not match Dnode") -#define TSDB_CODE_DND_MNODE_ALREADY_DEPLOYED TAOS_DEF_ERROR_CODE(0, 0x0403) //"Mnode already deployed") -#define TSDB_CODE_DND_MNODE_NOT_DEPLOYED TAOS_DEF_ERROR_CODE(0, 0x0404) //"Mnode not deployed") -#define TSDB_CODE_DND_READ_MNODE_FILE_ERROR TAOS_DEF_ERROR_CODE(0, 0x0405) //"Read mnode.json error") -#define TSDB_CODE_DND_WRITE_MNODE_FILE_ERROR TAOS_DEF_ERROR_CODE(0, 0x0406) //"Write mnode.json error") -#define TSDB_CODE_DND_NO_WRITE_ACCESS TAOS_DEF_ERROR_CODE(0, 0x0407) //"No permission for disk files in dnode") -#define TSDB_CODE_DND_INVALID_MSG_LEN TAOS_DEF_ERROR_CODE(0, 0x0408) //"Invalid message length") -#define TSDB_CODE_DND_ACTION_IN_PROGRESS TAOS_DEF_ERROR_CODE(0, 0x0409) //"Action in progress") -#define TSDB_CODE_DND_TOO_MANY_VNODES TAOS_DEF_ERROR_CODE(0, 0x040A) //"Too many vnode directories") -#define TSDB_CODE_DND_EXITING TAOS_DEF_ERROR_CODE(0, 0x040B) //"Dnode is exiting" -#define TSDB_CODE_DND_PARSE_VNODE_FILE_ERROR TAOS_DEF_ERROR_CODE(0, 0x040C) //"Parse vnodes.json error") -#define TSDB_CODE_DND_PARSE_DNODE_FILE_ERROR TAOS_DEF_ERROR_CODE(0, 0x040D) //"Parse dnodes.json error") +#define TSDB_CODE_DND_ACTION_IN_PROGRESS TAOS_DEF_ERROR_CODE(0, 0x0400) +#define TSDB_CODE_DND_EXITING TAOS_DEF_ERROR_CODE(0, 0x0401) +#define TSDB_CODE_DND_INVALID_MSG_LEN TAOS_DEF_ERROR_CODE(0, 0x0402) +#define TSDB_CODE_DND_DNODE_READ_FILE_ERROR TAOS_DEF_ERROR_CODE(0, 0x0410) +#define TSDB_CODE_DND_DNODE_WRITE_FILE_ERROR TAOS_DEF_ERROR_CODE(0, 0x0411) +#define TSDB_CODE_DND_MNODE_ALREADY_DEPLOYED TAOS_DEF_ERROR_CODE(0, 0x0420) +#define TSDB_CODE_DND_MNODE_NOT_DEPLOYED TAOS_DEF_ERROR_CODE(0, 0x0421) +#define TSDB_CODE_DND_MNODE_ID_INVALID TAOS_DEF_ERROR_CODE(0, 0x0422) +#define TSDB_CODE_DND_MNODE_ID_NOT_FOUND TAOS_DEF_ERROR_CODE(0, 0x0423) +#define TSDB_CODE_DND_MNODE_READ_FILE_ERROR TAOS_DEF_ERROR_CODE(0, 0x0424) +#define TSDB_CODE_DND_MNODE_WRITE_FILE_ERROR TAOS_DEF_ERROR_CODE(0, 0x0425) +#define TSDB_CODE_DND_VNODE_TOO_MANY_VNODES TAOS_DEF_ERROR_CODE(0, 0x0430) +#define TSDB_CODE_DND_VNODE_READ_FILE_ERROR TAOS_DEF_ERROR_CODE(0, 0x0431) +#define TSDB_CODE_DND_VNODE_WRITE_FILE_ERROR TAOS_DEF_ERROR_CODE(0, 0x0432) // vnode #define TSDB_CODE_VND_ACTION_IN_PROGRESS TAOS_DEF_ERROR_CODE(0, 0x0500) //"Action in progress") diff --git a/source/dnode/mgmt/daemon/src/daemon.c b/source/dnode/mgmt/daemon/src/daemon.c index 720d1589c2..326dfae4af 100644 --- a/source/dnode/mgmt/daemon/src/daemon.c +++ b/source/dnode/mgmt/daemon/src/daemon.c @@ -16,38 +16,173 @@ #define _DEFAULT_SOURCE #include "dnode.h" #include "os.h" +#include "tglobal.h" #include "ulog.h" -static bool stop = false; +static struct { + bool stop; + bool dumpConfig; + bool generateGrant; + bool printAuth; + bool printVersion; + char configDir[PATH_MAX]; +} global = {0}; -static void sigintHandler(int32_t signum, void *info, void *ctx) { stop = true; } +void dmnSigintHandle(int signum, void *info, void *ctx) { global.stop = true; } -static void setSignalHandler() { - taosSetSignal(SIGTERM, sigintHandler); - taosSetSignal(SIGHUP, sigintHandler); - taosSetSignal(SIGINT, sigintHandler); - taosSetSignal(SIGABRT, sigintHandler); - taosSetSignal(SIGBREAK, sigintHandler); +void dmnSetSignalHandle() { + taosSetSignal(SIGTERM, dmnSigintHandle); + taosSetSignal(SIGHUP, dmnSigintHandle); + taosSetSignal(SIGINT, dmnSigintHandle); + taosSetSignal(SIGABRT, dmnSigintHandle); + taosSetSignal(SIGBREAK, dmnSigintHandle); } -int main(int argc, char const *argv[]) { - const char *path = "/etc/taos"; +int dmnParseOpts(int argc, char const *argv[]) { + tstrncpy(global.configDir, "/etc/taos", PATH_MAX); - SDnode *pDnode = dnodeInit(path); - if (pDnode == NULL) { - uInfo("Failed to start TDengine, please check the log at %s", tsLogDir); - exit(EXIT_FAILURE); + for (int i = 1; i < argc; ++i) { + if (strcmp(argv[i], "-c") == 0) { + if (i < argc - 1) { + if (strlen(argv[++i]) >= PATH_MAX) { + printf("config file path overflow"); + return -1; + } + tstrncpy(global.configDir, argv[i], PATH_MAX); + } else { + printf("'-c' requires a parameter, default:%s\n", configDir); + return -1; + } + } else if (strcmp(argv[i], "-C") == 0) { + global.dumpConfig = true; + } else if (strcmp(argv[i], "-k") == 0) { + global.generateGrant = true; + } else if (strcmp(argv[i], "-A") == 0) { + global.printAuth = true; + } else if (strcmp(argv[i], "-V") == 0) { + global.printVersion = true; + } else { + } } - uInfo("Started TDengine service successfully."); - - setSignalHandler(); - while (!stop) { - taosMsleep(100); - } - - uInfo("TDengine is shut down!"); - dnodeCleanup(pDnode); - return 0; } + +void dmnGenerateGrant() { grantParseParameter(); } + +void dmnPrintVersion() { +#ifdef TD_ENTERPRISE + char *versionStr = "enterprise"; +#else + char *versionStr = "community"; +#endif + printf("%s version: %s compatible_version: %s\n", versionStr, version, compatible_version); + printf("gitinfo: %s\n", gitinfo); + printf("gitinfoI: %s\n", gitinfoOfInternal); + printf("builuInfo: %s\n", buildinfo); +} + +int dmnReadConfig(const char *path) { + taosIgnSIGPIPE(); + taosBlockSIGPIPE(); + taosResolveCRC(); + taosInitGlobalCfg(); + taosReadGlobalLogCfg(); + + if (taosMkDir(tsLogDir) != 0) { + printf("failed to create dir: %s, reason: %s\n", tsLogDir, strerror(errno)); + return -1; + } + + char temp[PATH_MAX]; + snprintf(temp, PATH_MAX, "%s/taosdlog", tsLogDir); + if (taosInitLog(temp, tsNumOfLogLines, 1) != 0) { + printf("failed to init log file\n"); + return -1; + } + + if (taosInitNotes() != 0) { + printf("failed to init log file\n"); + return -1; + } + + if (taosReadGlobalCfg() != 0) { + uError("failed to read global config"); + return -1; + } + + if (taosCheckGlobalCfg() != 0) { + uError("failed to check global config"); + return -1; + } + + taosSetCoreDump(tsEnableCoreFile); + return 0; +} + +void dmnDumpConfig() { taosDumpGlobalCfg(); } + +void dmnWaitSignal() { + dmnSetSignalHandle(); + while (!global.stop) { + taosMsleep(100); + } +} + +void dmnInitOption(SDnodeOpt *pOpt) { + pOpt->sver = tsVersion; + pOpt->numOfCores = tsNumOfCores; + pOpt->statusInterval = tsStatusInterval; + pOpt->serverPort = tsServerPort; + tstrncpy(pOpt->localEp, tsLocalEp, TSDB_EP_LEN); + tstrncpy(pOpt->localFqdn, tsLocalEp, TSDB_FQDN_LEN); + tstrncpy(pOpt->timezone, tsLocalEp, TSDB_TIMEZONE_LEN); + tstrncpy(pOpt->locale, tsLocalEp, TSDB_LOCALE_LEN); + tstrncpy(pOpt->charset, tsLocalEp, TSDB_LOCALE_LEN); +} + +int dmnRunDnode() { + SDnodeOpt opt = {0}; + dmnInitOption(&opt); + + SDnode *pDnd = dndInit(&opt); + if (pDnd == NULL) { + uInfo("Failed to start TDengine, please check the log at %s", tsLogDir); + return -1; + } + + uInfo("Started TDengine service successfully."); + dmnWaitSignal(); + uInfo("TDengine is shut down!"); + + dndCleanup(pDnd); + taosCloseLog(); + return 0; +} + +int main(int argc, char const *argv[]) { + if (dmnParseOpts(argc, argv) != 0) { + return -1; + } + + if (global.generateGrant) { + dmnGenerateGrant(); + return 0; + } + + if (global.printVersion) { + dmnPrintVersion(); + return 0; + } + + if (dmnReadConfig(global.configDir) != 0) { + return -1; + } + + if (global.dumpConfig) { + dmnDumpConfig(); + return 0; + } + + return dmnRunDnode(); +} diff --git a/source/dnode/mgmt/impl/inc/dnodeVnodes.h b/source/dnode/mgmt/impl/inc/dndDnode.h similarity index 53% rename from source/dnode/mgmt/impl/inc/dnodeVnodes.h rename to source/dnode/mgmt/impl/inc/dndDnode.h index 31eae049ab..ef16b1c8f0 100644 --- a/source/dnode/mgmt/impl/inc/dnodeVnodes.h +++ b/source/dnode/mgmt/impl/inc/dndDnode.h @@ -13,26 +13,27 @@ * along with this program. If not, see . */ -#ifndef _TD_DNODE_VNODES_H_ -#define _TD_DNODE_VNODES_H_ +#ifndef _TD_DND_DNODE_H_ +#define _TD_DND_DNODE_H_ #ifdef __cplusplus extern "C" { #endif -#include "dnodeInt.h" +#include "dndInt.h" -int32_t dnodeInitVnodes(); -void dnodeCleanupVnodes(); -void dnodeGetVnodeLoads(SVnodeLoads *pVloads); +int32_t dndInitDnode(SDnode *pDnd); +void dndCleanupDnode(SDnode *pDnd); +void dndProcessDnodeReq(SDnode *pDnd, SRpcMsg *pMsg, SEpSet *pEpSet); +void dndProcessDnodeRsp(SDnode *pDnd, SRpcMsg *pMsg, SEpSet *pEpSet); -void dnodeProcessVnodeMgmtMsg(SRpcMsg *pMsg, SEpSet *pEpSet); -void dnodeProcessVnodeWriteMsg(SRpcMsg *pMsg, SEpSet *pEpSet); -void dnodeProcessVnodeSyncMsg(SRpcMsg *pMsg, SEpSet *pEpSet); -void dnodeProcessVnodeQueryMsg(SRpcMsg *pMsg, SEpSet *pEpSet); -void dnodeProcessVnodeFetchMsg(SRpcMsg *pMsg, SEpSet *pEpSet); +int32_t dndGetDnodeId(SDnode *pDnd); +int64_t dndGetClusterId(SDnode *pDnd); +void dndGetDnodeEp(SDnode *pDnd, int32_t dnodeId, char *pEp, char *pFqdn, uint16_t *pPort); +void dndGetMnodeEpSet(SDnode *pDnd, SEpSet *pEpSet); +void dndSendRedirectMsg(SDnode *pDnd, SRpcMsg *pMsg, bool forShell); #ifdef __cplusplus } #endif -#endif /*_TD_DNODE_VNODES_H_*/ \ No newline at end of file +#endif /*_TD_DND_DNODE_H_*/ \ No newline at end of file diff --git a/source/dnode/mgmt/impl/inc/dnodeInt.h b/source/dnode/mgmt/impl/inc/dndInt.h similarity index 54% rename from source/dnode/mgmt/impl/inc/dnodeInt.h rename to source/dnode/mgmt/impl/inc/dndInt.h index 8944755268..966781426b 100644 --- a/source/dnode/mgmt/impl/inc/dnodeInt.h +++ b/source/dnode/mgmt/impl/inc/dndInt.h @@ -13,20 +13,27 @@ * along with this program. If not, see . */ -#ifndef _TD_DNODE_INT_H_ -#define _TD_DNODE_INT_H_ +#ifndef _TD_DND_INT_H_ +#define _TD_DND_INT_H_ #ifdef __cplusplus extern "C" { #endif + +#include "cJSON.h" #include "os.h" #include "taosmsg.h" -#include "tglobal.h" #include "thash.h" +#include "tlockfree.h" #include "tlog.h" +#include "tqueue.h" #include "trpc.h" #include "tthread.h" #include "ttime.h" +#include "tworker.h" +#include "mnode.h" +#include "vnode.h" +#include "dnode.h" extern int32_t dDebugFlag; @@ -37,8 +44,8 @@ extern int32_t dDebugFlag; #define dDebug(...) { if (dDebugFlag & DEBUG_DEBUG) { taosPrintLog("SRV ", dDebugFlag, __VA_ARGS__); }} #define dTrace(...) { if (dDebugFlag & DEBUG_TRACE) { taosPrintLog("SRV ", dDebugFlag, __VA_ARGS__); }} -typedef enum { DN_STAT_INIT, DN_STAT_RUNNING, DN_STAT_STOPPED } EStat; -typedef void (*MsgFp)(SRpcMsg *pMsg, SEpSet *pEpSet); +typedef enum { DND_STAT_INIT, DND_STAT_RUNNING, DND_STAT_STOPPED } EStat; +typedef void (*DndMsgFp)(SDnode *pDnode, SRpcMsg *pMsg, SEpSet *pEps); typedef struct { char *dnode; @@ -48,50 +55,75 @@ typedef struct { typedef struct { int32_t dnodeId; - int64_t clusterId; - SDnodeEps *dnodeEps; - SHashObj *dnodeHash; - SEpSet mnodeEpSetForShell; - SEpSet mnodeEpSetForPeer; - char *file; uint32_t rebootTime; - int8_t dropped; - int8_t threadStop; + int32_t dropped; + int64_t clusterId; + SEpSet shellEpSet; + SEpSet peerEpSet; + char *file; + SHashObj *dnodeHash; + SDnodeEps *dnodeEps; pthread_t *threadId; pthread_mutex_t mutex; -} SDnodeDnode; +} SDnodeMgmt; typedef struct { -} SDnodeMnode; + int32_t refCount; + int8_t deployed; + int8_t dropped; + SWorkerPool mgmtPool; + SWorkerPool readPool; + SWorkerPool writePool; + SWorkerPool syncPool; + taos_queue pReadQ; + taos_queue pWriteQ; + taos_queue pApplyQ; + taos_queue pSyncQ; + taos_queue pMgmtQ; + char *file; + SMnode *pMnode; + SRWLatch latch; +} SMnodeMgmt; typedef struct { -} SDnodeVnodes; + SHashObj *hash; + SWorkerPool mgmtPool; + SWorkerPool queryPool; + SWorkerPool fetchPool; + SMWorkerPool syncPool; + SMWorkerPool writePool; + taos_queue pMgmtQ; + int32_t openVnodes; + int32_t totalVnodes; + SRWLatch latch; +} SVnodesMgmt; typedef struct { - void *peerRpc; - void *shellRpc; - void *clientRpc; -} SDnodeTrans; + void *serverRpc; + void *clientRpc; + DndMsgFp msgFp[TSDB_MSG_TYPE_MAX]; +} STransMgmt; typedef struct SDnode { - EStat stat; - SDnodeDir dir; - SDnodeDnode dnode; - SDnodeVnodes vnodes; - SDnodeMnode mnode; - SDnodeTrans trans; - SStartupMsg startup; + EStat stat; + SDnodeOpt opt; + SDnodeDir dir; + SDnodeMgmt d; + SMnodeMgmt m; + SVnodesMgmt vmgmt; + STransMgmt t; + SStartupMsg startup; } SDnode; -EStat dnodeGetStat(SDnode *pDnode); -void dnodeSetStat(SDnode *pDnode, EStat stat); -char *dnodeStatStr(EStat stat); +EStat dndGetStat(SDnode *pDnode); +void dndSetStat(SDnode *pDnode, EStat stat); +char *dndStatStr(EStat stat); -void dnodeReportStartup(SDnode *pDnode, char *name, char *desc); -void dnodeGetStartup(SDnode *pDnode, SStartupMsg *pStartup); +void dndReportStartup(SDnode *pDnode, char *name, char *desc); +void dndGetStartup(SDnode *pDnode, SStartupMsg *pStartup); #ifdef __cplusplus } #endif -#endif /*_TD_DNODE_INT_H_*/ \ No newline at end of file +#endif /*_TD_DND_INT_H_*/ \ No newline at end of file diff --git a/source/dnode/mgmt/impl/inc/dnodeDnode.h b/source/dnode/mgmt/impl/inc/dndMnode.h similarity index 53% rename from source/dnode/mgmt/impl/inc/dnodeDnode.h rename to source/dnode/mgmt/impl/inc/dndMnode.h index 87dc0fdb9b..67c51e51a8 100644 --- a/source/dnode/mgmt/impl/inc/dnodeDnode.h +++ b/source/dnode/mgmt/impl/inc/dndMnode.h @@ -13,27 +13,24 @@ * along with this program. If not, see . */ -#ifndef _TD_DNODE_DNODE_H_ -#define _TD_DNODE_DNODE_H_ +#ifndef _TD_DND_MNODE_H_ +#define _TD_DND_MNODE_H_ #ifdef __cplusplus extern "C" { #endif -#include "dnodeInt.h" +#include "dndInt.h" -int32_t dnodeInitDnode(SDnode *pDnode); -void dnodeCleanupDnode(SDnode *pDnode); -void dnodeProcessDnodeMsg(SRpcMsg *pMsg, SEpSet *pEpSet); - -int32_t dnodeGetDnodeId(); -int64_t dnodeGetClusterId(); -void dnodeGetDnodeEp(int32_t dnodeId, char *epstr, char *fqdn, uint16_t *port); -void dnodeGetMnodeEpSetForPeer(SEpSet *epSet); -void dnodeGetMnodeEpSetForShell(SEpSet *epSet); -void dnodeSendRedirectMsg(SDnode *pDnode, SRpcMsg *rpcMsg, bool forShell); +int32_t dndInitMnode(SDnode *pDnode); +void dndCleanupMnode(SDnode *pDnode); +int32_t dndGetUserAuthFromMnode(SDnode *pDnode, char *user, char *spi, char *encrypt, char *secret, char *ckey); +void dndProcessMnodeMgmtMsg(SDnode *pDnode, SRpcMsg *pMsg, SEpSet *pEpSet); +void dndProcessMnodeReadMsg(SDnode *pDnode, SRpcMsg *pMsg, SEpSet *pEpSet); +void dndProcessMnodeWriteMsg(SDnode *pDnode, SRpcMsg *pMsg, SEpSet *pEpSet); +void dndProcessMnodeSyncMsg(SDnode *pDnode, SRpcMsg *pMsg, SEpSet *pEpSet); #ifdef __cplusplus } #endif -#endif /*_TD_DNODE_DNODE_H_*/ \ No newline at end of file +#endif /*_TD_DND_MNODE_H_*/ \ No newline at end of file diff --git a/source/dnode/mgmt/impl/inc/dnodeTransport.h b/source/dnode/mgmt/impl/inc/dndTransport.h similarity index 68% rename from source/dnode/mgmt/impl/inc/dnodeTransport.h rename to source/dnode/mgmt/impl/inc/dndTransport.h index 7d3f4be1ff..312da69fa2 100644 --- a/source/dnode/mgmt/impl/inc/dnodeTransport.h +++ b/source/dnode/mgmt/impl/inc/dndTransport.h @@ -13,21 +13,21 @@ * along with this program. If not, see . */ -#ifndef _TD_DNODE_TRANSPORT_H_ -#define _TD_DNODE_TRANSPORT_H_ +#ifndef _TD_DND_TRANSPORT_H_ +#define _TD_DND_TRANSPORT_H_ #ifdef __cplusplus extern "C" { #endif -#include "dnodeInt.h" +#include "dndInt.h" -int32_t dnodeInitTrans(); -void dnodeCleanupTrans(); -void dnodeSendMsgToMnode(SDnode *pDnode, SRpcMsg *rpcMsg); -void dnodeSendMsgToDnode(SDnode *pDnode, SEpSet *epSet, SRpcMsg *rpcMsg); +int32_t dndInitTrans(SDnode *pDnode); +void dndCleanupTrans(SDnode *pDnode); +void dndSendMsgToMnode(SDnode *pDnode, SRpcMsg *pRpcMsg); +void dndSendMsgToDnode(SDnode *pDnode, SEpSet *pEpSet, SRpcMsg *pRpcMsg); #ifdef __cplusplus } #endif -#endif /*_TD_DNODE_TRANSPORT_H_*/ +#endif /*_TD_DND_TRANSPORT_H_*/ diff --git a/source/dnode/mgmt/impl/inc/dnodeMnode.h b/source/dnode/mgmt/impl/inc/dndVnodes.h similarity index 52% rename from source/dnode/mgmt/impl/inc/dnodeMnode.h rename to source/dnode/mgmt/impl/inc/dndVnodes.h index 9f52e586fe..35f99ee73b 100644 --- a/source/dnode/mgmt/impl/inc/dnodeMnode.h +++ b/source/dnode/mgmt/impl/inc/dndVnodes.h @@ -13,25 +13,25 @@ * along with this program. If not, see . */ -#ifndef _TD_DNODE_MNODE_H_ -#define _TD_DNODE_MNODE_H_ +#ifndef _TD_DND_VNODES_H_ +#define _TD_DND_VNODES_H_ #ifdef __cplusplus extern "C" { #endif -#include "dnodeInt.h" +#include "dndInt.h" -int32_t dnodeInitMnode(); -void dnodeCleanupMnode(); -int32_t dnodeGetUserAuthFromMnode(char *user, char *spi, char *encrypt, char *secret, char *ckey); - -void dnodeProcessMnodeMgmtMsg(SRpcMsg *pMsg, SEpSet *pEpSet); -void dnodeProcessMnodeReadMsg(SRpcMsg *pMsg, SEpSet *pEpSet); -void dnodeProcessMnodeWriteMsg(SRpcMsg *pMsg, SEpSet *pEpSet); -void dnodeProcessMnodeSyncMsg(SRpcMsg *pMsg, SEpSet *pEpSet); +int32_t dndInitVnodes(SDnode *pDnode); +void dndCleanupVnodes(SDnode *pDnode); +void dndGetVnodeLoads(SDnode *pDnode, SVnodeLoads *pVloads); +void dndProcessVnodeMgmtMsg(SDnode *pDnode, SRpcMsg *pMsg, SEpSet *pEpSet); +void dndProcessVnodeWriteMsg(SDnode *pDnode, SRpcMsg *pMsg, SEpSet *pEpSet); +void dndProcessVnodeSyncMsg(SDnode *pDnode, SRpcMsg *pMsg, SEpSet *pEpSet); +void dndProcessVnodeQueryMsg(SDnode *pDnode, SRpcMsg *pMsg, SEpSet *pEpSet); +void dndProcessVnodeFetchMsg(SDnode *pDnode, SRpcMsg *pMsg, SEpSet *pEpSet); #ifdef __cplusplus } #endif -#endif /*_TD_DNODE_MNODE_H_*/ \ No newline at end of file +#endif /*_TD_DND_VNODES_H_*/ \ No newline at end of file diff --git a/source/dnode/mgmt/impl/src/dnodeDnode.c b/source/dnode/mgmt/impl/src/dndDnode.c similarity index 51% rename from source/dnode/mgmt/impl/src/dnodeDnode.c rename to source/dnode/mgmt/impl/src/dndDnode.c index ec60116ce6..378d76e046 100644 --- a/source/dnode/mgmt/impl/src/dnodeDnode.c +++ b/source/dnode/mgmt/impl/src/dndDnode.c @@ -14,69 +14,76 @@ */ #define _DEFAULT_SOURCE -#include "dnodeDnode.h" -#include "dnodeTransport.h" -#include "dnodeVnodes.h" -#include "cJSON.h" +#include "dndDnode.h" +#include "dndTransport.h" +#include "dndVnodes.h" -int32_t dnodeGetDnodeId() { - int32_t dnodeId = 0; - pthread_mutex_lock(&pDnode->mutex); - dnodeId = pDnode->dnodeId; - pthread_mutex_unlock(&pDnode->mutex); +static inline void dndLockDnode(SDnode *pDnd) { pthread_mutex_lock(&pDnd->d.mutex); } + +static inline void dndUnLockDnode(SDnode *pDnd) { pthread_mutex_unlock(&pDnd->d.mutex); } + +int32_t dndGetDnodeId(SDnode *pDnd) { + dndLockDnode(pDnd); + int32_t dnodeId = pDnd->d.dnodeId; + dndUnLockDnode(pDnd); return dnodeId; } -int64_t dnodeGetClusterId() { - int64_t clusterId = 0; - pthread_mutex_lock(&pDnode->mutex); - clusterId = pDnode->clusterId; - pthread_mutex_unlock(&pDnode->mutex); +int64_t dndGetClusterId(SDnode *pDnd) { + dndLockDnode(pDnd); + int64_t clusterId = pDnd->d.clusterId; + dndUnLockDnode(pDnd); return clusterId; } -void dnodeGetDnodeEp(int32_t dnodeId, char *ep, char *fqdn, uint16_t *port) { - pthread_mutex_lock(&pDnode->mutex); +void dndGetDnodeEp(SDnode *pDnd, int32_t dnodeId, char *pEp, char *pFqdn, uint16_t *pPort) { + dndLockDnode(pDnd); - SDnodeEp *pEp = taosHashGet(pDnode->dnodeHash, &dnodeId, sizeof(int32_t)); - if (pEp != NULL) { - if (port) *port = pEp->dnodePort; - if (fqdn) tstrncpy(fqdn, pEp->dnodeFqdn, TSDB_FQDN_LEN); - if (ep) snprintf(ep, TSDB_EP_LEN, "%s:%u", pEp->dnodeFqdn, pEp->dnodePort); + SDnodeEp *pDnodeEp = taosHashGet(pDnd->d.dnodeHash, &dnodeId, sizeof(int32_t)); + if (pDnodeEp != NULL) { + if (pPort != NULL) { + *pPort = pDnodeEp->port; + } + if (pFqdn != NULL) { + tstrncpy(pFqdn, pDnodeEp->fqdn, TSDB_FQDN_LEN); + } + if (pEp != NULL) { + snprintf(pEp, TSDB_EP_LEN, "%s:%u", pDnodeEp->fqdn, pDnodeEp->port); + } } - pthread_mutex_unlock(&pDnode->mutex); + dndUnLockDnode(pDnd); } -void dnodeGetMnodeEpSetForPeer(SEpSet *pEpSet) { - pthread_mutex_lock(&pDnode->mutex); - *pEpSet = pDnode->mnodeEpSetForPeer; - pthread_mutex_unlock(&pDnode->mutex); +void dndGetMnodeEpSet(SDnode *pDnd, SEpSet *pEpSet) { + dndLockDnode(pDnd); + *pEpSet = pDnd->d.peerEpSet; + dndUnLockDnode(pDnd); } -void dnodeGetMnodeEpSetForShell(SEpSet *pEpSet) { - pthread_mutex_lock(&pDnode->mutex); - *pEpSet = pDnode->mnodeEpSetForShell; - pthread_mutex_unlock(&pDnode->mutex); +void dndGetShellEpSet(SDnode *pDnd, SEpSet *pEpSet) { + dndLockDnode(pDnd); + *pEpSet = pDnd->d.shellEpSet; + dndUnLockDnode(pDnd); } -void dnodeSendRedirectMsg(SDnode *pDnode, SRpcMsg *pMsg, bool forShell) { +void dndSendRedirectMsg(SDnode *pDnd, SRpcMsg *pMsg, bool forShell) { int32_t msgType = pMsg->msgType; SEpSet epSet = {0}; if (forShell) { - dnodeGetMnodeEpSetForShell(&epSet); + dndGetShellEpSet(pDnd, &epSet); } else { - dnodeGetMnodeEpSetForPeer(&epSet); + dndGetMnodeEpSet(pDnd, &epSet); } dDebug("RPC %p, msg:%s is redirected, num:%d use:%d", pMsg->handle, taosMsg[msgType], epSet.numOfEps, epSet.inUse); for (int32_t i = 0; i < epSet.numOfEps; ++i) { dDebug("mnode index:%d %s:%u", i, epSet.fqdn[i], epSet.port[i]); - if (strcmp(epSet.fqdn[i], tsLocalFqdn) == 0) { - if ((epSet.port[i] == tsServerPort + TSDB_PORT_DNODEDNODE && !forShell) || - (epSet.port[i] == tsServerPort && forShell)) { + if (strcmp(epSet.fqdn[i], pDnd->opt.localFqdn) == 0) { + if ((epSet.port[i] == pDnd->opt.serverPort + TSDB_PORT_DNODEDNODE && !forShell) || + (epSet.port[i] == pDnd->opt.serverPort && forShell)) { epSet.inUse = (i + 1) % epSet.numOfEps; dDebug("mnode index:%d %s:%d set inUse to %d", i, epSet.fqdn[i], epSet.port[i], epSet.inUse); } @@ -88,39 +95,37 @@ void dnodeSendRedirectMsg(SDnode *pDnode, SRpcMsg *pMsg, bool forShell) { rpcSendRedirectRsp(pMsg->handle, &epSet); } -static void dnodeUpdateMnodeEpSet(SDnodeDnode *pDnode, SEpSet *pEpSet) { - if (pEpSet == NULL || pEpSet->numOfEps <= 0) { - dError("mnode is changed, but content is invalid, discard it"); - return; - } else { - dInfo("mnode is changed, num:%d use:%d", pEpSet->numOfEps, pEpSet->inUse); - } +static void dndUpdateMnodeEpSet(SDnode *pDnd, SEpSet *pEpSet) { + dInfo("mnode is changed, num:%d use:%d", pEpSet->numOfEps, pEpSet->inUse); - pthread_mutex_lock(&pDnode->mutex); + dndLockDnode(pDnd); - pDnode->mnodeEpSetForPeer = *pEpSet; + pDnd->d.peerEpSet = *pEpSet; for (int32_t i = 0; i < pEpSet->numOfEps; ++i) { pEpSet->port[i] -= TSDB_PORT_DNODEDNODE; dInfo("mnode index:%d %s:%u", i, pEpSet->fqdn[i], pEpSet->port[i]); } - pDnode->mnodeEpSetForShell = *pEpSet; + pDnd->d.shellEpSet = *pEpSet; - pthread_mutex_unlock(&pDnode->mutex); + dndUnLockDnode(pDnd); } -static void dnodePrintDnodes() { - dDebug("print dnode endpoint list, num:%d", pDnode->dnodeEps->dnodeNum); - for (int32_t i = 0; i < pDnode->dnodeEps->dnodeNum; i++) { - SDnodeEp *ep = &pDnode->dnodeEps->dnodeEps[i]; - dDebug("dnode:%d, fqdn:%s port:%u isMnode:%d", ep->dnodeId, ep->dnodeFqdn, ep->dnodePort, ep->isMnode); +static void dndPrintDnodes(SDnode *pDnd) { + SDnodeMgmt *pDnode = &pDnd->d; + + dDebug("print dnode endpoint list, num:%d", pDnode->dnodeEps->num); + for (int32_t i = 0; i < pDnode->dnodeEps->num; i++) { + SDnodeEp *pEp = &pDnode->dnodeEps->eps[i]; + dDebug("dnode:%d, fqdn:%s port:%u isMnode:%d", pEp->id, pEp->fqdn, pEp->port, pEp->isMnode); } } -static void dnodeResetDnodes(SDnodeEps *pEps) { - assert(pEps != NULL); - int32_t size = sizeof(SDnodeEps) + pEps->dnodeNum * sizeof(SDnodeEp); +static void dndResetDnodes(SDnode *pDnd, SDnodeEps *pDnodeEps) { + SDnodeMgmt *pDnode = &pDnd->d; - if (pEps->dnodeNum > pDnode->dnodeEps->dnodeNum) { + int32_t size = sizeof(SDnodeEps) + pDnodeEps->num * sizeof(SDnodeEp); + + if (pDnodeEps->num > pDnode->dnodeEps->num) { SDnodeEps *tmp = calloc(1, size); if (tmp == NULL) return; @@ -128,49 +133,51 @@ static void dnodeResetDnodes(SDnodeEps *pEps) { pDnode->dnodeEps = tmp; } - if (pDnode->dnodeEps != pEps) { - memcpy(pDnode->dnodeEps, pEps, size); + if (pDnode->dnodeEps != pDnodeEps) { + memcpy(pDnode->dnodeEps, pDnodeEps, size); } - pDnode->mnodeEpSetForPeer.inUse = 0; - pDnode->mnodeEpSetForShell.inUse = 0; + pDnode->peerEpSet.inUse = 0; + pDnode->shellEpSet.inUse = 0; int32_t mIndex = 0; - for (int32_t i = 0; i < pDnode->dnodeEps->dnodeNum; i++) { - SDnodeEp *ep = &pDnode->dnodeEps->dnodeEps[i]; - if (!ep->isMnode) continue; + for (int32_t i = 0; i < pDnode->dnodeEps->num; i++) { + SDnodeEp *pDnodeEp = &pDnode->dnodeEps->eps[i]; + if (!pDnodeEp->isMnode) continue; if (mIndex >= TSDB_MAX_REPLICA) continue; - strcpy(pDnode->mnodeEpSetForShell.fqdn[mIndex], ep->dnodeFqdn); - strcpy(pDnode->mnodeEpSetForPeer.fqdn[mIndex], ep->dnodeFqdn); - pDnode->mnodeEpSetForShell.port[mIndex] = ep->dnodePort; - pDnode->mnodeEpSetForShell.port[mIndex] = ep->dnodePort + tsDnodeDnodePort; + strcpy(pDnode->shellEpSet.fqdn[mIndex], pDnodeEp->fqdn); + strcpy(pDnode->peerEpSet.fqdn[mIndex], pDnodeEp->fqdn); + pDnode->shellEpSet.port[mIndex] = pDnodeEp->port; + pDnode->shellEpSet.port[mIndex] = pDnodeEp->port + TSDB_PORT_DNODEDNODE; mIndex++; } - for (int32_t i = 0; i < pDnode->dnodeEps->dnodeNum; ++i) { - SDnodeEp *ep = &pDnode->dnodeEps->dnodeEps[i]; - taosHashPut(pDnode->dnodeHash, &ep->dnodeId, sizeof(int32_t), ep, sizeof(SDnodeEp)); + for (int32_t i = 0; i < pDnode->dnodeEps->num; ++i) { + SDnodeEp *pDnodeEp = &pDnode->dnodeEps->eps[i]; + taosHashPut(pDnode->dnodeHash, &pDnodeEp->id, sizeof(int32_t), pDnodeEp, sizeof(SDnodeEp)); } - dnodePrintDnodes(); + dndPrintDnodes(pDnd); } -static bool dnodeIsEpChanged(int32_t dnodeId, char *epStr) { +static bool dndIsEpChanged(SDnode *pDnd, int32_t dnodeId) { bool changed = false; - pthread_mutex_lock(&pDnode->mutex); + dndLockDnode(pDnd); - SDnodeEp *pEp = taosHashGet(pDnode->dnodeHash, &dnodeId, sizeof(int32_t)); - if (pEp != NULL) { - char epSaved[TSDB_EP_LEN + 1]; - snprintf(epSaved, TSDB_EP_LEN, "%s:%u", pEp->dnodeFqdn, pEp->dnodePort); - changed = strcmp(epStr, epSaved) != 0; + SDnodeEp *pDnodeEp = taosHashGet(pDnd->d.dnodeHash, &dnodeId, sizeof(int32_t)); + if (pDnodeEp != NULL) { + char epstr[TSDB_EP_LEN + 1]; + snprintf(epstr, TSDB_EP_LEN, "%s:%u", pDnodeEp->fqdn, pDnodeEp->port); + changed = strcmp(pDnd->opt.localEp, epstr) != 0; } - pthread_mutex_unlock(&pDnode->mutex); + dndUnLockDnode(pDnd); return changed; } -static int32_t dnodeReadDnodes() { +static int32_t dndReadDnodes(SDnode *pDnd) { + SDnodeMgmt *pDnode = &pDnd->d; + int32_t len = 0; int32_t maxLen = 30000; char *content = calloc(1, maxLen + 1); @@ -234,70 +241,72 @@ static int32_t dnodeReadDnodes() { dError("failed to calloc dnodeEpList since %s", strerror(errno)); goto PRASE_DNODE_OVER; } - pDnode->dnodeEps->dnodeNum = dnodeInfosSize; + pDnode->dnodeEps->num = dnodeInfosSize; for (int32_t i = 0; i < dnodeInfosSize; ++i) { cJSON *dnodeInfo = cJSON_GetArrayItem(dnodeInfos, i); if (dnodeInfo == NULL) break; - SDnodeEp *pEp = &pDnode->dnodeEps->dnodeEps[i]; + SDnodeEp *pDnodeEp = &pDnode->dnodeEps->eps[i]; cJSON *dnodeId = cJSON_GetObjectItem(dnodeInfo, "dnodeId"); if (!dnodeId || dnodeId->type != cJSON_String) { dError("failed to read %s, dnodeId not found", pDnode->file); goto PRASE_DNODE_OVER; } - pEp->dnodeId = atoi(dnodeId->valuestring); + pDnodeEp->id = atoi(dnodeId->valuestring); cJSON *isMnode = cJSON_GetObjectItem(dnodeInfo, "isMnode"); if (!isMnode || isMnode->type != cJSON_String) { dError("failed to read %s, isMnode not found", pDnode->file); goto PRASE_DNODE_OVER; } - pEp->isMnode = atoi(isMnode->valuestring); + pDnodeEp->isMnode = atoi(isMnode->valuestring); cJSON *dnodeFqdn = cJSON_GetObjectItem(dnodeInfo, "dnodeFqdn"); if (!dnodeFqdn || dnodeFqdn->type != cJSON_String || dnodeFqdn->valuestring == NULL) { dError("failed to read %s, dnodeFqdn not found", pDnode->file); goto PRASE_DNODE_OVER; } - tstrncpy(pEp->dnodeFqdn, dnodeFqdn->valuestring, TSDB_FQDN_LEN); + tstrncpy(pDnodeEp->fqdn, dnodeFqdn->valuestring, TSDB_FQDN_LEN); cJSON *dnodePort = cJSON_GetObjectItem(dnodeInfo, "dnodePort"); if (!dnodePort || dnodePort->type != cJSON_String) { dError("failed to read %s, dnodePort not found", pDnode->file); goto PRASE_DNODE_OVER; } - pEp->dnodePort = atoi(dnodePort->valuestring); + pDnodeEp->port = atoi(dnodePort->valuestring); } dInfo("succcessed to read file %s", pDnode->file); - dnodePrintDnodes(); + dndPrintDnodes(pDnd); PRASE_DNODE_OVER: if (content != NULL) free(content); if (root != NULL) cJSON_Delete(root); if (fp != NULL) fclose(fp); - if (dnodeIsEpChanged(pDnode->dnodeId, tsLocalEp)) { - dError("localEp %s different with %s and need reconfigured", tsLocalEp, pDnode->file); + if (dndIsEpChanged(pDnd, pDnode->dnodeId)) { + dError("localEp %s different with %s and need reconfigured", pDnd->opt.localEp, pDnode->file); return -1; } if (pDnode->dnodeEps == NULL) { pDnode->dnodeEps = calloc(1, sizeof(SDnodeEps) + sizeof(SDnodeEp)); - pDnode->dnodeEps->dnodeNum = 1; - pDnode->dnodeEps->dnodeEps[0].dnodePort = tsServerPort; - tstrncpy(pDnode->dnodeEps->dnodeEps[0].dnodeFqdn, tsLocalFqdn, TSDB_FQDN_LEN); + pDnode->dnodeEps->num = 1; + pDnode->dnodeEps->eps[0].port = pDnd->opt.serverPort; + tstrncpy(pDnode->dnodeEps->eps[0].fqdn, pDnd->opt.localFqdn, TSDB_FQDN_LEN); } - dnodeResetDnodes(pDnode->dnodeEps); + dndResetDnodes(pDnd, pDnode->dnodeEps); terrno = 0; return 0; } -static int32_t dnodeWriteDnodes() { +static int32_t dndWriteDnodes(SDnode *pDnd) { + SDnodeMgmt *pDnode = &pDnd->d; + FILE *fp = fopen(pDnode->file, "w"); if (!fp) { dError("failed to write %s since %s", pDnode->file, strerror(errno)); @@ -313,13 +322,13 @@ static int32_t dnodeWriteDnodes() { len += snprintf(content + len, maxLen - len, " \"clusterId\": \"%" PRId64 "\",\n", pDnode->clusterId); len += snprintf(content + len, maxLen - len, " \"dropped\": \"%d\",\n", pDnode->dropped); len += snprintf(content + len, maxLen - len, " \"dnodeInfos\": [{\n"); - for (int32_t i = 0; i < pDnode->dnodeEps->dnodeNum; ++i) { - SDnodeEp *ep = &pDnode->dnodeEps->dnodeEps[i]; - len += snprintf(content + len, maxLen - len, " \"dnodeId\": \"%d\",\n", ep->dnodeId); - len += snprintf(content + len, maxLen - len, " \"isMnode\": \"%d\",\n", ep->isMnode); - len += snprintf(content + len, maxLen - len, " \"dnodeFqdn\": \"%s\",\n", ep->dnodeFqdn); - len += snprintf(content + len, maxLen - len, " \"dnodePort\": \"%u\"\n", ep->dnodePort); - if (i < pDnode->dnodeEps->dnodeNum - 1) { + for (int32_t i = 0; i < pDnode->dnodeEps->num; ++i) { + SDnodeEp *pDnodeEp = &pDnode->dnodeEps->eps[i]; + len += snprintf(content + len, maxLen - len, " \"dnodeId\": \"%d\",\n", pDnodeEp->id); + len += snprintf(content + len, maxLen - len, " \"isMnode\": \"%d\",\n", pDnodeEp->isMnode); + len += snprintf(content + len, maxLen - len, " \"dnodeFqdn\": \"%s\",\n", pDnodeEp->fqdn); + len += snprintf(content + len, maxLen - len, " \"dnodePort\": \"%u\"\n", pDnodeEp->port); + if (i < pDnode->dnodeEps->num - 1) { len += snprintf(content + len, maxLen - len, " },{\n"); } else { len += snprintf(content + len, maxLen - len, " }]\n"); @@ -337,107 +346,114 @@ static int32_t dnodeWriteDnodes() { return 0; } -static void dnodeSendStatusMsg() { - int32_t contLen = sizeof(SStatusMsg) + TSDB_MAX_VNODES * sizeof(SVnodeLoad); - +static void dndSendStatusMsg(SDnode *pDnd) { + int32_t contLen = sizeof(SStatusMsg) + TSDB_MAX_VNODES * sizeof(SVnodeLoad); SStatusMsg *pStatus = rpcMallocCont(contLen); if (pStatus == NULL) { dError("failed to malloc status message"); return; } - pStatus->sversion = htonl(tsVersion); - pStatus->dnodeId = htonl(dnodeGetDnodeId()); - pStatus->clusterId = htobe64(dnodeGetClusterId()); - pStatus->rebootTime = htonl(pDnode->rebootTime); - pStatus->numOfCores = htonl(tsNumOfCores); - tstrncpy(pStatus->dnodeEp, tsLocalEp, TSDB_EP_LEN); - - pStatus->clusterCfg.statusInterval = htonl(tsStatusInterval); + dndLockDnode(pDnd); + pStatus->sversion = htonl(pDnd->opt.sver); + pStatus->dnodeId = htonl(pDnd->d.dnodeId); + pStatus->clusterId = htobe64(pDnd->d.clusterId); + pStatus->rebootTime = htonl(pDnd->d.rebootTime); + pStatus->numOfCores = htonl(pDnd->opt.numOfCores); + tstrncpy(pStatus->dnodeEp, pDnd->opt.localEp, TSDB_EP_LEN); + pStatus->clusterCfg.statusInterval = htonl(pDnd->opt.statusInterval); + tstrncpy(pStatus->clusterCfg.timezone, pDnd->opt.timezone, TSDB_TIMEZONE_LEN); + tstrncpy(pStatus->clusterCfg.locale, pDnd->opt.locale, TSDB_LOCALE_LEN); + tstrncpy(pStatus->clusterCfg.charset, pDnd->opt.charset, TSDB_LOCALE_LEN); pStatus->clusterCfg.checkTime = 0; - tstrncpy(pStatus->clusterCfg.timezone, tsTimezone, TSDB_TIMEZONE_LEN); - tstrncpy(pStatus->clusterCfg.locale, tsLocale, TSDB_LOCALE_LEN); - tstrncpy(pStatus->clusterCfg.charset, tsCharset, TSDB_LOCALE_LEN); char timestr[32] = "1970-01-01 00:00:00.00"; (void)taosParseTime(timestr, &pStatus->clusterCfg.checkTime, (int32_t)strlen(timestr), TSDB_TIME_PRECISION_MILLI, 0); + dndUnLockDnode(pDnd); - dnodeGetVnodeLoads(&pStatus->vnodeLoads); + dndGetVnodeLoads(pDnd, &pStatus->vnodeLoads); contLen = sizeof(SStatusMsg) + pStatus->vnodeLoads.num * sizeof(SVnodeLoad); SRpcMsg rpcMsg = {.pCont = pStatus, .contLen = contLen, .msgType = TSDB_MSG_TYPE_STATUS}; - dnodeSendMsgToMnode(NULL, &rpcMsg); + dndSendMsgToMnode(pDnd, &rpcMsg); } -static void dnodeUpdateCfg(SDnodeCfg *pCfg) { - if (pDnode->dnodeId == 0) return; - if (pDnode->dropped) return; +static void dndUpdateDnodeCfg(SDnode *pDnd, SDnodeCfg *pCfg) { + SDnodeMgmt *pDnode = &pDnd->d; + if (pDnode->dnodeId != 0 && pDnode->dropped != pCfg->dropped) return; - pthread_mutex_lock(&pDnode->mutex); + dndLockDnode(pDnd); pDnode->dnodeId = pCfg->dnodeId; pDnode->clusterId = pCfg->clusterId; pDnode->dropped = pCfg->dropped; - dInfo("dnodeId is set to %d, clusterId is set to %" PRId64, pCfg->dnodeId, pCfg->clusterId); + dInfo("set dnodeId:%d clusterId:%" PRId64 " dropped:%d", pCfg->dnodeId, pCfg->clusterId, pCfg->dropped); - dnodeWriteDnodes(); - pthread_mutex_unlock(&pDnode->mutex); + dndWriteDnodes(pDnd); + dndUnLockDnode(pDnd); } -static void dnodeUpdateDnodeEps(SDnodeEps *pEps) { - if (pEps == NULL || pEps->dnodeNum <= 0) return; +static void dndUpdateDnodeEps(SDnode *pDnd, SDnodeEps *pDnodeEps) { + if (pDnodeEps == NULL || pDnodeEps->num <= 0) return; - pthread_mutex_lock(&pDnode->mutex); + dndLockDnode(pDnd); - if (pEps->dnodeNum != pDnode->dnodeEps->dnodeNum) { - dnodeResetDnodes(pEps); - dnodeWriteDnodes(); + if (pDnodeEps->num != pDnd->d.dnodeEps->num) { + dndResetDnodes(pDnd, pDnodeEps); + dndWriteDnodes(pDnd); } else { - int32_t size = pEps->dnodeNum * sizeof(SDnodeEp) + sizeof(SDnodeEps); - if (memcmp(pDnode->dnodeEps, pEps, size) != 0) { - dnodeResetDnodes(pEps); - dnodeWriteDnodes(); + int32_t size = pDnodeEps->num * sizeof(SDnodeEp) + sizeof(SDnodeEps); + if (memcmp(pDnd->d.dnodeEps, pDnodeEps, size) != 0) { + dndResetDnodes(pDnd, pDnodeEps); + dndWriteDnodes(pDnd); } } - pthread_mutex_unlock(&pDnode->mutex); + dndUnLockDnode(pDnd); } -static void dnodeProcessStatusRsp(SRpcMsg *pMsg) { +static void dndProcessStatusRsp(SDnode *pDnd, SRpcMsg *pMsg, SEpSet *pEpSet) { + if (pEpSet && pEpSet->numOfEps > 0) { + dndUpdateMnodeEpSet(pDnd, pEpSet); + } + if (pMsg->code != TSDB_CODE_SUCCESS) return; SStatusRsp *pStatusRsp = pMsg->pCont; - - SDnodeCfg *pCfg = &pStatusRsp->dnodeCfg; + SDnodeCfg *pCfg = &pStatusRsp->dnodeCfg; pCfg->dnodeId = htonl(pCfg->dnodeId); pCfg->clusterId = htobe64(pCfg->clusterId); - dnodeUpdateCfg(pCfg); + dndUpdateDnodeCfg(pDnd, pCfg); if (pCfg->dropped) return; - SDnodeEps *pEps = &pStatusRsp->dnodeEps; - pEps->dnodeNum = htonl(pEps->dnodeNum); - for (int32_t i = 0; i < pEps->dnodeNum; ++i) { - pEps->dnodeEps[i].dnodeId = htonl(pEps->dnodeEps[i].dnodeId); - pEps->dnodeEps[i].dnodePort = htons(pEps->dnodeEps[i].dnodePort); + SDnodeEps *pDnodeEps = &pStatusRsp->dnodeEps; + pDnodeEps->num = htonl(pDnodeEps->num); + for (int32_t i = 0; i < pDnodeEps->num; ++i) { + pDnodeEps->eps[i].id = htonl(pDnodeEps->eps[i].id); + pDnodeEps->eps[i].port = htons(pDnodeEps->eps[i].port); } - dnodeUpdateDnodeEps(pEps); + dndUpdateDnodeEps(pDnd, pDnodeEps); } -static void dnodeProcessConfigDnodeReq(SRpcMsg *pMsg) { +static void dndProcessAuthRsp(SDnode *pDnd, SRpcMsg *pMsg, SEpSet *pEpSet) { assert(1); } + +static void dndProcessGrantRsp(SDnode *pDnd, SRpcMsg *pMsg, SEpSet *pEpSet) { assert(1); } + +static void dndProcessConfigDnodeReq(SDnode *pDnd, SRpcMsg *pMsg) { SCfgDnodeMsg *pCfg = pMsg->pCont; - int32_t code = taosCfgDynamicOptions(pCfg->config); + int32_t code = TSDB_CODE_OPS_NOT_SUPPORT; SRpcMsg rspMsg = {.handle = pMsg->handle, .pCont = NULL, .contLen = 0, .code = code}; rpcSendResponse(&rspMsg); rpcFreeCont(pMsg->pCont); } -static void dnodeProcessStartupReq(SRpcMsg *pMsg) { - dInfo("startup msg is received, cont:%s", (char *)pMsg->pCont); +static void dndProcessStartupReq(SDnode *pDnd, SRpcMsg *pMsg) { + dInfo("startup msg is received"); SStartupMsg *pStartup = rpcMallocCont(sizeof(SStartupMsg)); - dnodeGetStartup(NULL, pStartup); + dndGetStartup(pDnd, pStartup); dInfo("startup msg is sent, step:%s desc:%s finished:%d", pStartup->name, pStartup->desc, pStartup->finished); @@ -447,108 +463,120 @@ static void dnodeProcessStartupReq(SRpcMsg *pMsg) { } static void *dnodeThreadRoutine(void *param) { - int32_t ms = tsStatusInterval * 1000; + SDnode *pDnd = param; + int32_t ms = pDnd->opt.statusInterval * 1000; - while (!pDnode->threadStop) { - if (dnodeGetStat() != DN_STAT_RUNNING) { - continue; - } else { - dnodeSendStatusMsg(); - } + while (true) { taosMsleep(ms); + if (dndGetStat(pDnd) != DND_STAT_RUNNING) { + continue; + } + + pthread_testcancel(); + dndSendStatusMsg(pDnd); } } -int32_t dnodeInitDnode(SDnode *pServer) { - SDnodeDnode *pDnode = &pServer->dnode; +int32_t dndInitDnode(SDnode *pDnd) { + SDnodeMgmt *pDnode = &pDnd->d; + + pDnode->dnodeId = 0; + pDnode->rebootTime = taosGetTimestampSec(); + pDnode->dropped = 0; + pDnode->clusterId = 0; char path[PATH_MAX]; - snprintf(path, PATH_MAX, "%s/dnode.json", pServer->dir.dnode); + snprintf(path, PATH_MAX, "%s/dnode.json", pDnd->dir.dnode); pDnode->file = strdup(path); if (pDnode->file == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; return -1; } - pDnode->dnodeId = 0; - pDnode->clusterId = 0; - pDnode->dnodeEps = NULL; - - pDnode->rebootTime = taosGetTimestampSec(); - pDnode->dropped = 0; - pthread_mutex_init(&pDnode->mutex, NULL); - pDnode->threadStop = false; - pDnode->dnodeHash = taosHashInit(4, taosGetDefaultHashFunction(TSDB_DATA_TYPE_INT), true, HASH_ENTRY_LOCK); if (pDnode->dnodeHash == NULL) { dError("failed to init dnode hash"); - return TSDB_CODE_DND_OUT_OF_MEMORY; + terrno = TSDB_CODE_OUT_OF_MEMORY; + return -1; } - pDnode->threadId = taosCreateThread(dnodeThreadRoutine, NULL); + if (dndReadDnodes(pDnd) != 0) { + dError("failed to read file:%s since %s", pDnode->file, terrstr()); + return -1; + } + + pthread_mutex_init(&pDnode->mutex, NULL); + + pDnode->threadId = taosCreateThread(dnodeThreadRoutine, pDnd); if (pDnode->threadId == NULL) { dError("failed to init dnode thread"); - return TSDB_CODE_DND_OUT_OF_MEMORY; + terrno = TSDB_CODE_OUT_OF_MEMORY; + return -1; } - int32_t code = dnodeReadDnodes(); - if (code != 0) { - dError("failed to read file:%s since %s", pDnode->file, tstrerror(code)); - return code; - } - - dInfo("dnode-dnode is initialized"); + dInfo("dnd-dnode is initialized"); return 0; } -void dnodeCleanupDnode(SDnode *pServer) { - SDnodeDnode *pDnode = &pServer->dnode; +void dndCleanupDnode(SDnode *pDnd) { + SDnodeMgmt *pDnode = &pDnd->d; if (pDnode->threadId != NULL) { - pDnode->threadStop = true; taosDestoryThread(pDnode->threadId); pDnode->threadId = NULL; } - pthread_mutex_lock(&pDnode->mutex); + dndLockDnode(pDnd); if (pDnode->dnodeEps != NULL) { free(pDnode->dnodeEps); pDnode->dnodeEps = NULL; } - if (pDnode->dnodeHash) { + if (pDnode->dnodeHash != NULL) { taosHashCleanup(pDnode->dnodeHash); pDnode->dnodeHash = NULL; } - pthread_mutex_unlock(&pDnode->mutex); - pthread_mutex_destroy(&pDnode->mutex); - - dInfo("dnode-dnode is cleaned up"); -} - -void dnodeProcessDnodeMsg(SDnode *pDnode, SRpcMsg *pMsg, SEpSet *pEpSet) { - int32_t msgType = pMsg->msgType; - - if (msgType == TSDB_MSG_TYPE_STATUS_RSP && pEpSet) { - dnodeUpdateMnodeEpSet(&pDnode->dnode, pEpSet); + if (pDnode->file != NULL) { + free(pDnode->file); + pDnode->file = NULL; } - switch (msgType) { + dndUnLockDnode(pDnd); + pthread_mutex_destroy(&pDnode->mutex); + + dInfo("dnd-dnode is cleaned up"); +} + +void dndProcessDnodeReq(SDnode *pDnd, SRpcMsg *pMsg, SEpSet *pEpSet) { + switch (pMsg->msgType) { case TSDB_MSG_TYPE_NETWORK_TEST: - dnodeProcessStartupReq(pMsg); + dndProcessStartupReq(pDnd, pMsg); break; case TSDB_MSG_TYPE_CONFIG_DNODE_IN: - dnodeProcessConfigDnodeReq(pMsg); - break; - case TSDB_MSG_TYPE_STATUS_RSP: - dnodeProcessStatusRsp(pMsg); + dndProcessConfigDnodeReq(pDnd, pMsg); break; default: - dError("RPC %p, %s not processed", pMsg->handle, taosMsg[msgType]); - SRpcMsg rspMsg = {.handle = pMsg->handle, .code = TSDB_CODE_DND_MSG_NOT_PROCESSED}; + dError("RPC %p, dnode req:%s not processed", pMsg->handle, taosMsg[pMsg->msgType]); + SRpcMsg rspMsg = {.handle = pMsg->handle, .code = TSDB_CODE_MSG_NOT_PROCESSED}; rpcSendResponse(&rspMsg); rpcFreeCont(pMsg->pCont); } } + +void dndProcessDnodeRsp(SDnode *pDnd, SRpcMsg *pMsg, SEpSet *pEpSet) { + switch (pMsg->msgType) { + case TSDB_MSG_TYPE_STATUS_RSP: + dndProcessStatusRsp(pDnd, pMsg, pEpSet); + break; + case TSDB_MSG_TYPE_AUTH_RSP: + dndProcessAuthRsp(pDnd, pMsg, pEpSet); + break; + case TSDB_MSG_TYPE_GRANT_RSP: + dndProcessGrantRsp(pDnd, pMsg, pEpSet); + break; + default: + dError("RPC %p, dnode rsp:%s not processed", pMsg->handle, taosMsg[pMsg->msgType]); + } +} diff --git a/source/dnode/mgmt/impl/src/dnodeInt.c b/source/dnode/mgmt/impl/src/dndInt.c similarity index 56% rename from source/dnode/mgmt/impl/src/dnodeInt.c rename to source/dnode/mgmt/impl/src/dndInt.c index 8641a54def..9e77da61b4 100644 --- a/source/dnode/mgmt/impl/src/dnodeInt.c +++ b/source/dnode/mgmt/impl/src/dndInt.c @@ -14,50 +14,47 @@ */ #define _DEFAULT_SOURCE -#include "dnodeDnode.h" -#include "dnodeMnode.h" -#include "dnodeTransport.h" -#include "dnodeVnodes.h" +#include "dndDnode.h" +#include "dndMnode.h" +#include "dndTransport.h" +#include "dndVnodes.h" #include "sync.h" #include "tcache.h" -#include "tconfig.h" -#include "tnote.h" -#include "tstep.h" #include "wal.h" -EStat dnodeGetStat(SDnode *pDnode) { return pDnode->stat; } +EStat dndGetStat(SDnode *pDnode) { return pDnode->stat; } -void dnodeSetStat(SDnode *pDnode, EStat stat) { - dDebug("dnode stat set from %s to %s", dnodeStatStr(pDnode->stat), dnodeStatStr(stat)); +void dndSetStat(SDnode *pDnode, EStat stat) { + dDebug("dnode stat set from %s to %s", dndStatStr(pDnode->stat), dndStatStr(stat)); pDnode->stat = stat; } -char *dnodeStatStr(EStat stat) { +char *dndStatStr(EStat stat) { switch (stat) { - case DN_STAT_INIT: + case DND_STAT_INIT: return "init"; - case DN_STAT_RUNNING: + case DND_STAT_RUNNING: return "running"; - case DN_STAT_STOPPED: + case DND_STAT_STOPPED: return "stopped"; default: return "unknown"; } } -void dnodeReportStartup(SDnode *pDnode, char *name, char *desc) { +void dndReportStartup(SDnode *pDnode, char *name, char *desc) { SStartupMsg *pStartup = &pDnode->startup; tstrncpy(pStartup->name, name, strlen(pStartup->name)); tstrncpy(pStartup->desc, desc, strlen(pStartup->desc)); pStartup->finished = 0; } -void dnodeGetStartup(SDnode *pDnode, SStartupMsg *pStartup) { - memcpy(pStartup, &pDnode->startup, sizeof(SStartupMsg); - pStartup->finished = (dnodeGetStat(pDnode) == DN_STAT_RUNNING); +void dndGetStartup(SDnode *pDnode, SStartupMsg *pStartup) { + memcpy(pStartup, &pDnode->startup, sizeof(SStartupMsg)); + pStartup->finished = (dndGetStat(pDnode) == DND_STAT_RUNNING); } -static int32_t dnodeCheckRunning(char *dataDir) { +static int32_t dndCheckRunning(char *dataDir) { char filepath[PATH_MAX] = {0}; snprintf(filepath, sizeof(filepath), "%s/.running", dataDir); @@ -79,15 +76,19 @@ static int32_t dnodeCheckRunning(char *dataDir) { return 0; } -static int32_t dnodeInitDisk(SDnode *pDnode, char *dataDir) { - char path[PATH_MAX]; - snprintf(path, PATH_MAX, "%s/mnode", dataDir); +static int32_t dndInitEnv(SDnode *pDnode, SDnodeOpt *pOptions) { + if (dndCheckRunning(pOptions->dataDir) != 0) { + return -1; + } + + char path[PATH_MAX + 100]; + snprintf(path, sizeof(path), "%s%smnode", pOptions->dataDir, TD_DIRSEP); pDnode->dir.mnode = strdup(path); - sprintf(path, PATH_MAX, "%s/vnode", dataDir); + snprintf(path, sizeof(path), "%s%svnode", pOptions->dataDir, TD_DIRSEP); pDnode->dir.vnodes = strdup(path); - sprintf(path, PATH_MAX, "%s/dnode", dataDir); + snprintf(path, sizeof(path), "%s%sdnode", pOptions->dataDir, TD_DIRSEP); pDnode->dir.dnode = strdup(path); if (pDnode->dir.mnode == NULL || pDnode->dir.vnodes == NULL || pDnode->dir.dnode == NULL) { @@ -114,55 +115,10 @@ static int32_t dnodeInitDisk(SDnode *pDnode, char *dataDir) { return -1; } - if (dnodeCheckRunning(dataDir) != 0) { - return -1; - } - return 0; } -static int32_t dnodeInitEnv(SDnode *pDnode, const char *cfgPath) { - taosIgnSIGPIPE(); - taosBlockSIGPIPE(); - taosResolveCRC(); - taosInitGlobalCfg(); - taosReadGlobalLogCfg(); - taosSetCoreDump(tsEnableCoreFile); - - if (!taosMkDir(tsLogDir)) { - printf("failed to create dir: %s, reason: %s\n", tsLogDir, strerror(errno)); - return -1; - } - - char temp[TSDB_FILENAME_LEN]; - sprintf(temp, "%s/taosdlog", tsLogDir); - if (taosInitLog(temp, tsNumOfLogLines, 1) < 0) { - dError("failed to init log file\n"); - return -1; - } - - if (!taosReadGlobalCfg()) { - taosPrintGlobalCfg(); - dError("TDengine read global config failed"); - return -1; - } - - taosInitNotes(); - - if (taosCheckGlobalCfg() != 0) { - dError("TDengine check global config failed"); - return -1; - } - - if (dnodeInitDisk(pDnode, tsDataDir) != 0) { - dError("TDengine failed to init directory"); - return -1; - } - - return 0; -} - -static void dnodeCleanupEnv(SDnode *pDnode) { +static void dndCleanupEnv(SDnode *pDnode) { if (pDnode->dir.mnode != NULL) { tfree(pDnode->dir.mnode); } @@ -175,11 +131,10 @@ static void dnodeCleanupEnv(SDnode *pDnode) { tfree(pDnode->dir.dnode); } - taosCloseLog(); taosStopCacheRefreshWorker(); } -SDnode *dnodeInit(const char *cfgPath) { +SDnode *dndInit(SDnodeOpt *pOptions) { SDnode *pDnode = calloc(1, sizeof(pDnode)); if (pDnode == NULL) { dError("failed to create dnode object"); @@ -188,73 +143,73 @@ SDnode *dnodeInit(const char *cfgPath) { } dInfo("start to initialize TDengine"); - dnodeSetStat(pDnode, DN_STAT_INIT); + dndSetStat(pDnode, DND_STAT_INIT); - if (dnodeInitEnv(pDnode, cfgPath) != 0) { + if (dndInitEnv(pDnode, pOptions) != 0) { dError("failed to init env"); - dnodeCleanup(pDnode); + dndCleanup(pDnode); return NULL; } if (rpcInit() != 0) { dError("failed to init rpc env"); - dnodeCleanup(pDnode); + dndCleanup(pDnode); return NULL; } if (walInit() != 0) { dError("failed to init wal env"); - dnodeCleanup(pDnode); + dndCleanup(pDnode); return NULL; } - if (dnodeInitDnode(pDnode) != 0) { + if (dndInitDnode(pDnode) != 0) { dError("failed to init dnode"); - dnodeCleanup(pDnode); + dndCleanup(pDnode); return NULL; } - if (dnodeInitVnodes(pDnode) != 0) { + if (dndInitVnodes(pDnode) != 0) { dError("failed to init vnodes"); - dnodeCleanup(pDnode); + dndCleanup(pDnode); return NULL; } - if (dnodeInitMnode(pDnode) != 0) { + if (dndInitMnode(pDnode) != 0) { dError("failed to init mnode"); - dnodeCleanup(pDnode); + dndCleanup(pDnode); return NULL; } - if (dnodeInitTrans(pDnode) != 0) { + if (dndInitTrans(pDnode) != 0) { dError("failed to init transport"); - dnodeCleanup(pDnode); + dndCleanup(pDnode); return NULL; } - dnodeSetStat(pDnode, DN_STAT_RUNNING); - dnodeReportStartup(pDnode, "TDengine", "initialized successfully"); + dndSetStat(pDnode, DND_STAT_RUNNING); + dndReportStartup(pDnode, "TDengine", "initialized successfully"); dInfo("TDengine is initialized successfully"); return 0; } -void dnodeCleanup(SDnode *pDnode) { - if (dnodeGetStat(pDnode) == DN_STAT_STOPPED) { +void dndCleanup(SDnode *pDnode) { + if (dndGetStat(pDnode) == DND_STAT_STOPPED) { dError("dnode is shutting down"); return; } dInfo("start to cleanup TDengine"); - dnodeSetStat(pDnode, DN_STAT_STOPPED); - dnodeCleanupTrans(pDnode); - dnodeCleanupMnode(pDnode); - dnodeCleanupVnodes(pDnode); - dnodeCleanupDnode(pDnode); + dndSetStat(pDnode, DND_STAT_STOPPED); + dndCleanupTrans(pDnode); + dndCleanupMnode(pDnode); + dndCleanupVnodes(pDnode); + dndCleanupDnode(pDnode); walCleanUp(); rpcCleanup(); dInfo("TDengine is cleaned up successfully"); - dnodeCleanupEnv(pDnode); + dndCleanupEnv(pDnode); free(pDnode); } diff --git a/source/dnode/mgmt/impl/src/dndMnode.c b/source/dnode/mgmt/impl/src/dndMnode.c new file mode 100644 index 0000000000..5f3e48d8a1 --- /dev/null +++ b/source/dnode/mgmt/impl/src/dndMnode.c @@ -0,0 +1,816 @@ +/* + * Copyright (c) 2019 TAOS Data, Inc. + * + * This program is free software: you can use, redistribute, and/or modify + * it under the terms of the GNU Affero General Public License, version 3 + * or later ("AGPL"), as published by the Free Software Foundation. + * + * This program is distributed in the hope that it will be useful, but WITHOUT + * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or + * FITNESS FOR A PARTICULAR PURPOSE. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + */ + +#define _DEFAULT_SOURCE +#include "dndMnode.h" +#include "dndDnode.h" +#include "dndTransport.h" + +static int32_t dndInitMnodeReadWorker(SDnode *pDnode); +static int32_t dndInitMnodeWriteWorker(SDnode *pDnode); +static int32_t dndInitMnodeSyncWorker(SDnode *pDnode); +static int32_t dndInitMnodeMgmtWorker(SDnode *pDnode); +static void dndCleanupMnodeReadWorker(SDnode *pDnode); +static void dndCleanupMnodeWriteWorker(SDnode *pDnode); +static void dndCleanupMnodeSyncWorker(SDnode *pDnode); +static void dndCleanupMnodeMgmtWorker(SDnode *pDnode); +static int32_t dndAllocMnodeReadQueue(SDnode *pDnode); +static int32_t dndAllocMnodeWriteQueue(SDnode *pDnode); +static int32_t dndAllocMnodeApplyQueue(SDnode *pDnode); +static int32_t dndAllocMnodeSyncQueue(SDnode *pDnode); +static int32_t dndAllocMnodeMgmtQueue(SDnode *pDnode); +static void dndFreeMnodeReadQueue(SDnode *pDnode); +static void dndFreeMnodeWriteQueue(SDnode *pDnode); +static void dndFreeMnodeApplyQueue(SDnode *pDnode); +static void dndFreeMnodeSyncQueue(SDnode *pDnode); +static void dndFreeMnodeMgmtQueue(SDnode *pDnode); + +static void dndProcessMnodeReadQueue(SDnode *pDnode, SMnodeMsg *pMsg); +static void dndProcessMnodeWriteQueue(SDnode *pDnode, SMnodeMsg *pMsg); +static void dndProcessMnodeApplyQueue(SDnode *pDnode, SMnodeMsg *pMsg); +static void dndProcessMnodeSyncQueue(SDnode *pDnode, SMnodeMsg *pMsg); +static void dndProcessMnodeMgmtQueue(SDnode *pDnode, SRpcMsg *pMsg); +static int32_t dndWriteMnodeMsgToQueue(SMnode *pMnode, taos_queue pQueue, SRpcMsg *pRpcMsg); +void dndProcessMnodeReadMsg(SDnode *pDnode, SRpcMsg *pMsg, SEpSet *pEpSet); +void dndProcessMnodeWriteMsg(SDnode *pDnode, SRpcMsg *pMsg, SEpSet *pEpSet); +void dndProcessMnodeSyncMsg(SDnode *pDnode, SRpcMsg *pMsg, SEpSet *pEpSet); +void dndProcessMnodeMgmtMsg(SDnode *pDnode, SRpcMsg *pMsg, SEpSet *pEpSet); +static int32_t dndPutMsgIntoMnodeApplyQueue(SDnode *pDnode, SMnodeMsg *pMsg); + +static int32_t dndStartMnodeWorker(SDnode *pDnode); +static void dndStopMnodeWorker(SDnode *pDnode); + +static SMnode *dndAcquireMnode(SDnode *pDnode); +static void dndReleaseMnode(SDnode *pDnode, SMnode *pMnode); + +static int32_t dndReadMnodeFile(SDnode *pDnode); +static int32_t dndWriteMnodeFile(SDnode *pDnode); + +static int32_t dndOpenMnode(SDnode *pDnode, SMnodeOptions *pOptions); +static int32_t dndAlterMnode(SDnode *pDnode, SMnodeOptions *pOptions); +static int32_t dndDropMnode(SDnode *pDnode); + +static int32_t dndProcessCreateMnodeReq(SDnode *pDnode, SRpcMsg *pRpcMsg); +static int32_t dndProcessAlterMnodeReq(SDnode *pDnode, SRpcMsg *pRpcMsg); +static int32_t dndProcessDropMnodeReq(SDnode *pDnode, SRpcMsg *pRpcMsg); + +static SMnode *dndAcquireMnode(SDnode *pDnode) { + SMnodeMgmt *pMgmt = &pDnode->m; + SMnode *pMnode = NULL; + int32_t refCount = 0; + + taosRLockLatch(&pMgmt->latch); + if (pMgmt->deployed && !pMgmt->dropped) { + refCount = atomic_add_fetch_32(&pMgmt->refCount, 1); + pMnode = pMgmt->pMnode; + } else { + terrno = TSDB_CODE_DND_MNODE_NOT_DEPLOYED; + } + taosRUnLockLatch(&pMgmt->latch); + + dTrace("acquire mnode, refCount:%d", refCount); + return pMnode; +} + +static void dndReleaseMnode(SDnode *pDnode, SMnode *pMnode) { + SMnodeMgmt *pMgmt = &pDnode->m; + int32_t refCount = 0; + + taosRLockLatch(&pMgmt->latch); + if (pMnode != NULL) { + refCount = atomic_sub_fetch_32(&pMgmt->refCount, 1); + } + taosRUnLockLatch(&pMgmt->latch); + + dTrace("release mnode, refCount:%d", refCount); +} + +static int32_t dndReadMnodeFile(SDnode *pDnode) { + SMnodeMgmt *pMgmt = &pDnode->m; + int32_t code = TSDB_CODE_DND_MNODE_READ_FILE_ERROR; + int32_t len = 0; + int32_t maxLen = 300; + char *content = calloc(1, maxLen + 1); + cJSON *root = NULL; + + FILE *fp = fopen(pMgmt->file, "r"); + if (fp == NULL) { + dDebug("file %s not exist", pMgmt->file); + code = 0; + goto PRASE_MNODE_OVER; + } + + len = (int32_t)fread(content, 1, maxLen, fp); + if (len <= 0) { + dError("failed to read %s since content is null", pMgmt->file); + goto PRASE_MNODE_OVER; + } + + content[len] = 0; + root = cJSON_Parse(content); + if (root == NULL) { + dError("failed to read %s since invalid json format", pMgmt->file); + goto PRASE_MNODE_OVER; + } + + cJSON *deployed = cJSON_GetObjectItem(root, "deployed"); + if (!deployed || deployed->type != cJSON_String) { + dError("failed to read %s since deployed not found", pMgmt->file); + goto PRASE_MNODE_OVER; + } + pMgmt->deployed = atoi(deployed->valuestring); + + cJSON *dropped = cJSON_GetObjectItem(root, "dropped"); + if (!dropped || dropped->type != cJSON_String) { + dError("failed to read %s since dropped not found", pMgmt->file); + goto PRASE_MNODE_OVER; + } + pMgmt->dropped = atoi(dropped->valuestring); + + code = 0; + dInfo("succcessed to read file %s", pMgmt->file); + +PRASE_MNODE_OVER: + if (content != NULL) free(content); + if (root != NULL) cJSON_Delete(root); + if (fp != NULL) fclose(fp); + + terrno = code; + return code; +} + +static int32_t dndWriteMnodeFile(SDnode *pDnode) { + SMnodeMgmt *pMgmt = &pDnode->m; + char file[PATH_MAX + 20] = {0}; + snprintf(file, sizeof(file), "%s.bak", pMgmt->file); + + FILE *fp = fopen(file, "w"); + if (fp != NULL) { + terrno = TSDB_CODE_DND_MNODE_WRITE_FILE_ERROR; + dError("failed to write %s since %s", file, terrstr()); + return -1; + } + + int32_t len = 0; + int32_t maxLen = 300; + char *content = calloc(1, maxLen + 1); + + len += snprintf(content + len, maxLen - len, "{\n"); + len += snprintf(content + len, maxLen - len, " \"deployed\": \"%d\",\n", pMgmt->deployed); + len += snprintf(content + len, maxLen - len, " \"dropped\": \"%d\"\n", pMgmt->dropped); + len += snprintf(content + len, maxLen - len, "}\n"); + + fwrite(content, 1, len, fp); + taosFsyncFile(fileno(fp)); + fclose(fp); + free(content); + + if (taosRenameFile(file, pMgmt->file) != 0) { + terrno = TSDB_CODE_DND_MNODE_WRITE_FILE_ERROR; + dError("failed to rename %s since %s", pMgmt->file, terrstr()); + return -1; + } + + dInfo("successed to write %s", pMgmt->file); + return 0; +} + +static int32_t dndStartMnodeWorker(SDnode *pDnode) { + if (dndAllocMnodeReadQueue(pDnode) != 0) { + dError("failed to alloc mnode read queue since %s", terrstr()); + return -1; + } + + if (dndAllocMnodeWriteQueue(pDnode) != 0) { + dError("failed to alloc mnode write queue since %s", terrstr()); + return -1; + } + + if (dndAllocMnodeApplyQueue(pDnode) != 0) { + dError("failed to alloc mnode apply queue since %s", terrstr()); + return -1; + } + + if (dndAllocMnodeSyncQueue(pDnode) != 0) { + dError("failed to alloc mnode sync queue since %s", terrstr()); + return -1; + } + + return 0; +} + +static void dndStopMnodeWorker(SDnode *pDnode) { + SMnodeMgmt *pMgmt = &pDnode->m; + + taosWLockLatch(&pMgmt->latch); + pMgmt->deployed = 0; + pMgmt->pMnode = NULL; + taosWUnLockLatch(&pMgmt->latch); + + while (pMgmt->refCount > 1) taosMsleep(10); + while (!taosQueueEmpty(pMgmt->pReadQ)) taosMsleep(10); + while (!taosQueueEmpty(pMgmt->pApplyQ)) taosMsleep(10); + while (!taosQueueEmpty(pMgmt->pWriteQ)) taosMsleep(10); + while (!taosQueueEmpty(pMgmt->pSyncQ)) taosMsleep(10); + + dndFreeMnodeReadQueue(pDnode); + dndFreeMnodeWriteQueue(pDnode); + dndFreeMnodeApplyQueue(pDnode); + dndFreeMnodeSyncQueue(pDnode); + + dndCleanupMnodeReadWorker(pDnode); + dndCleanupMnodeWriteWorker(pDnode); + dndCleanupMnodeSyncWorker(pDnode); +} + +static bool dndNeedDeployMnode(SDnode *pDnode) { + if (dndGetDnodeId(pDnode) > 0) { + return false; + } + + if (dndGetClusterId(pDnode) > 0) { + return false; + } + if (strcmp(pDnode->opt.localEp, pDnode->opt.firstEp) != 0) { + return false; + } + + return true; +} + +static void dndInitMnodeOptions(SDnode *pDnode, SMnodeOptions *pOptions) { + pOptions->pDnode = pDnode; + pOptions->sendMsgToDnodeFp = dndSendMsgToDnode; + pOptions->sendMsgToMnodeFp = dndSendMsgToMnode; + pOptions->sendRedirectMsgFp = dndSendRedirectMsg; + pOptions->putMsgToApplyMsgFp = dndPutMsgIntoMnodeApplyQueue; +} + +static int32_t dndBuildMnodeOptions(SDnode *pDnode, SMnodeOptions *pOptions, SCreateMnodeMsg *pMsg) { + dndInitMnodeOptions(pDnode, pOptions); + + if (pMsg == NULL) { + pOptions->dnodeId = 1; + pOptions->clusterId = 1234; + pOptions->replica = 1; + pOptions->selfIndex = 0; + SReplica *pReplica = &pOptions->replicas[0]; + pReplica->id = 1; + pReplica->port = pDnode->opt.serverPort; + tstrncpy(pReplica->fqdn, pDnode->opt.localFqdn, TSDB_FQDN_LEN); + } else { + pOptions->dnodeId = dndGetDnodeId(pDnode); + pOptions->clusterId = dndGetClusterId(pDnode); + pOptions->selfIndex = -1; + pOptions->replica = pMsg->replica; + for (int32_t index = 0; index < pMsg->replica; ++index) { + SReplica *pReplica = &pOptions->replicas[index]; + pReplica->id = pMsg->replicas[index].id; + pReplica->port = pMsg->replicas[index].port; + tstrncpy(pReplica->fqdn, pMsg->replicas[index].fqdn, TSDB_FQDN_LEN); + if (pReplica->id == pOptions->dnodeId) { + pOptions->selfIndex = index; + } + } + } + + if (pOptions->selfIndex == -1) { + terrno = TSDB_CODE_DND_MNODE_ID_NOT_FOUND; + dError("failed to build mnode options since %s", terrstr()); + return -1; + } + + return 0; +} + +static int32_t dndOpenMnode(SDnode *pDnode, SMnodeOptions *pOptions) { + SMnodeMgmt *pMgmt = &pDnode->m; + + int32_t code = dndStartMnodeWorker(pDnode); + if (code != 0) { + dError("failed to start mnode worker since %s", terrstr()); + return code; + } + + SMnode *pMnode = mnodeOpen(pDnode->dir.mnode, pOptions); + if (pMnode == NULL) { + dError("failed to open mnode since %s", terrstr()); + code = terrno; + dndStopMnodeWorker(pDnode); + terrno = code; + return code; + } + + if (dndWriteMnodeFile(pDnode) != 0) { + dError("failed to write mnode file since %s", terrstr()); + code = terrno; + dndStopMnodeWorker(pDnode); + mnodeClose(pMnode); + mnodeDestroy(pDnode->dir.mnode); + terrno = code; + return code; + } + + taosWLockLatch(&pMgmt->latch); + pMgmt->pMnode = pMnode; + pMgmt->deployed = 1; + taosWUnLockLatch(&pMgmt->latch); + + return 0; +} + +static int32_t dndAlterMnode(SDnode *pDnode, SMnodeOptions *pOptions) { + SMnodeMgmt *pMgmt = &pDnode->m; + + SMnode *pMnode = dndAcquireMnode(pDnode); + if (pMnode == NULL) { + dError("failed to alter mnode since %s", terrstr()); + return -1; + } + + if (mnodeAlter(pMnode, pOptions) != 0) { + dError("failed to alter mnode since %s", terrstr()); + dndReleaseMnode(pDnode, pMnode); + return -1; + } + + dndReleaseMnode(pDnode, pMnode); + return 0; +} + +static int32_t dndDropMnode(SDnode *pDnode) { + SMnodeMgmt *pMgmt = &pDnode->m; + + SMnode *pMnode = dndAcquireMnode(pDnode); + if (pMnode == NULL) { + dError("failed to drop mnode since %s", terrstr()); + return -1; + } + + taosRLockLatch(&pMgmt->latch); + pMgmt->dropped = 1; + taosRUnLockLatch(&pMgmt->latch); + + if (dndWriteMnodeFile(pDnode) != 0) { + taosRLockLatch(&pMgmt->latch); + pMgmt->dropped = 0; + taosRUnLockLatch(&pMgmt->latch); + + dndReleaseMnode(pDnode, pMnode); + dError("failed to drop mnode since %s", terrstr()); + return -1; + } + + dndStopMnodeWorker(pDnode); + dndWriteMnodeFile(pDnode); + mnodeClose(pMnode); + mnodeDestroy(pDnode->dir.mnode); + + return 0; +} + +static SCreateMnodeMsg *dndParseCreateMnodeMsg(SRpcMsg *pRpcMsg) { + SCreateMnodeMsg *pMsg = pRpcMsg->pCont; + pMsg->dnodeId = htonl(pMsg->dnodeId); + for (int32_t i = 0; i < pMsg->replica; ++i) { + pMsg->replicas[i].id = htonl(pMsg->replicas[i].id); + pMsg->replicas[i].port = htons(pMsg->replicas[i].port); + } + + return pMsg; +} + +static int32_t dndProcessCreateMnodeReq(SDnode *pDnode, SRpcMsg *pRpcMsg) { + SCreateMnodeMsg *pMsg = dndParseCreateMnodeMsg(pRpcMsg->pCont); + + if (pMsg->dnodeId != dndGetDnodeId(pDnode)) { + terrno = TSDB_CODE_DND_MNODE_ID_INVALID; + return -1; + } else { + SMnodeOptions option = {0}; + if (dndBuildMnodeOptions(pDnode, &option, pMsg) != 0) { + return -1; + } + return dndOpenMnode(pDnode, &option); + } +} + +static int32_t dndProcessAlterMnodeReq(SDnode *pDnode, SRpcMsg *pRpcMsg) { + SAlterMnodeMsg *pMsg = dndParseCreateMnodeMsg(pRpcMsg->pCont); + + if (pMsg->dnodeId != dndGetDnodeId(pDnode)) { + terrno = TSDB_CODE_DND_MNODE_ID_INVALID; + return -1; + } else { + SMnodeOptions option = {0}; + if (dndBuildMnodeOptions(pDnode, &option, pMsg) != 0) { + return -1; + } + return dndAlterMnode(pDnode, &option); + } +} + +static int32_t dndProcessDropMnodeReq(SDnode *pDnode, SRpcMsg *pRpcMsg) { + SDropMnodeMsg *pMsg = dndParseCreateMnodeMsg(pRpcMsg->pCont); + + if (pMsg->dnodeId != dndGetDnodeId(pDnode)) { + terrno = TSDB_CODE_DND_MNODE_ID_INVALID; + return -1; + } else { + return dndDropMnode(pDnode); + } +} + +static void dndProcessMnodeMgmtQueue(SDnode *pDnode, SRpcMsg *pMsg) { + int32_t code = 0; + + switch (pMsg->msgType) { + case TSDB_MSG_TYPE_CREATE_MNODE_IN: + code = dndProcessCreateMnodeReq(pDnode, pMsg); + break; + case TSDB_MSG_TYPE_ALTER_MNODE_IN: + code = dndProcessAlterMnodeReq(pDnode, pMsg); + break; + case TSDB_MSG_TYPE_DROP_MNODE_IN: + code = dndProcessDropMnodeReq(pDnode, pMsg); + break; + default: + code = TSDB_CODE_MSG_NOT_PROCESSED; + break; + } + + SRpcMsg rsp = {.code = code, .handle = pMsg->handle}; + rpcSendResponse(&rsp); + rpcFreeCont(pMsg->pCont); + taosFreeQitem(pMsg); +} + +static void dndProcessMnodeReadQueue(SDnode *pDnode, SMnodeMsg *pMsg) { + SMnodeMgmt *pMgmt = &pDnode->m; + + SMnode *pMnode = dndAcquireMnode(pDnode); + if (pMnode != NULL) { + mnodeProcessReadMsg(pMnode, pMsg); + dndReleaseMnode(pDnode, pMnode); + } else { + mnodeSendRsp(pMsg, terrno); + } + + mnodeCleanupMsg(pMsg); +} + +static void dndProcessMnodeWriteQueue(SDnode *pDnode, SMnodeMsg *pMsg) { + SMnodeMgmt *pMgmt = &pDnode->m; + + SMnode *pMnode = dndAcquireMnode(pDnode); + if (pMnode != NULL) { + mnodeProcessWriteMsg(pMnode, pMsg); + dndReleaseMnode(pDnode, pMnode); + } else { + mnodeSendRsp(pMsg, terrno); + } + + mnodeCleanupMsg(pMsg); +} + +static void dndProcessMnodeApplyQueue(SDnode *pDnode, SMnodeMsg *pMsg) { + SMnodeMgmt *pMgmt = &pDnode->m; + + SMnode *pMnode = dndAcquireMnode(pDnode); + if (pMnode != NULL) { + mnodeProcessApplyMsg(pMnode, pMsg); + dndReleaseMnode(pDnode, pMnode); + } else { + mnodeSendRsp(pMsg, terrno); + } + + mnodeCleanupMsg(pMsg); +} + +static void dndProcessMnodeSyncQueue(SDnode *pDnode, SMnodeMsg *pMsg) { + SMnodeMgmt *pMgmt = &pDnode->m; + + SMnode *pMnode = dndAcquireMnode(pDnode); + if (pMnode != NULL) { + mnodeProcessSyncMsg(pMnode, pMsg); + dndReleaseMnode(pDnode, pMnode); + } else { + mnodeSendRsp(pMsg, terrno); + } + + mnodeCleanupMsg(pMsg); +} + +static int32_t dndWriteMnodeMsgToQueue(SMnode *pMnode, taos_queue pQueue, SRpcMsg *pRpcMsg) { + assert(pQueue); + + SMnodeMsg *pMsg = mnodeInitMsg(pMnode, pRpcMsg); + if (pMsg == NULL) { + terrno = TSDB_CODE_OUT_OF_MEMORY; + return -1; + } + + if (taosWriteQitem(pQueue, pMsg) != 0) { + mnodeCleanupMsg(pMsg); + terrno = TSDB_CODE_OUT_OF_MEMORY; + return -1; + } + + return 0; +} + +void dndProcessMnodeMgmtMsg(SDnode *pDnode, SRpcMsg *pRpcMsg, SEpSet *pEpSet) { + SMnodeMgmt *pMgmt = &pDnode->m; + SMnode *pMnode = dndAcquireMnode(pDnode); + + SRpcMsg *pMsg = taosAllocateQitem(sizeof(SRpcMsg)); + if (pMsg == NULL || taosWriteQitem(pMgmt->pMgmtQ, pMsg) != 0) { + SRpcMsg rsp = {.handle = pRpcMsg->handle, .code = TSDB_CODE_OUT_OF_MEMORY}; + rpcSendResponse(&rsp); + rpcFreeCont(pRpcMsg->pCont); + taosFreeQitem(pMsg); + } +} + +void dndProcessMnodeWriteMsg(SDnode *pDnode, SRpcMsg *pMsg, SEpSet *pEpSet) { + SMnodeMgmt *pMgmt = &pDnode->m; + SMnode *pMnode = dndAcquireMnode(pDnode); + if (pMnode == NULL || dndWriteMnodeMsgToQueue(pMnode, pMgmt->pWriteQ, pMsg) != 0) { + SRpcMsg rsp = {.handle = pMsg->handle, .code = terrno}; + rpcSendResponse(&rsp); + rpcFreeCont(pMsg->pCont); + } + + dndReleaseMnode(pDnode, pMnode); +} + +void dndProcessMnodeSyncMsg(SDnode *pDnode, SRpcMsg *pMsg, SEpSet *pEpSet) { + SMnodeMgmt *pMgmt = &pDnode->m; + SMnode *pMnode = dndAcquireMnode(pDnode); + if (pMnode == NULL || dndWriteMnodeMsgToQueue(pMnode, pMgmt->pSyncQ, pMsg) != 0) { + SRpcMsg rsp = {.handle = pMsg->handle, .code = terrno}; + rpcSendResponse(&rsp); + rpcFreeCont(pMsg->pCont); + } + + dndReleaseMnode(pDnode, pMnode); +} + +void dndProcessMnodeReadMsg(SDnode *pDnode, SRpcMsg *pMsg, SEpSet *pEpSet) { + SMnodeMgmt *pMgmt = &pDnode->m; + SMnode *pMnode = dndAcquireMnode(pDnode); + if (pMnode == NULL || dndWriteMnodeMsgToQueue(pMnode, pMgmt->pSyncQ, pMsg) != 0) { + SRpcMsg rsp = {.handle = pMsg->handle, .code = terrno}; + rpcSendResponse(&rsp); + rpcFreeCont(pMsg->pCont); + } + + dndReleaseMnode(pDnode, pMnode); +} + +static int32_t dndPutMsgIntoMnodeApplyQueue(SDnode *pDnode, SMnodeMsg *pMsg) { + SMnodeMgmt *pMgmt = &pDnode->m; + + SMnode *pMnode = dndAcquireMnode(pDnode); + if (pMnode == NULL) { + return -1; + } + + int32_t code = taosWriteQitem(pMgmt->pApplyQ, pMsg); + dndReleaseMnode(pDnode, pMnode); + return code; +} + +static int32_t dndAllocMnodeMgmtQueue(SDnode *pDnode) { + SMnodeMgmt *pMgmt = &pDnode->m; + pMgmt->pMgmtQ = tWorkerAllocQueue(&pMgmt->mgmtPool, NULL, (FProcessItem)dndProcessMnodeMgmtQueue); + if (pMgmt->pMgmtQ == NULL) { + terrno = TSDB_CODE_OUT_OF_MEMORY; + return -1; + } + return 0; +} + +static void dndFreeMnodeMgmtQueue(SDnode *pDnode) { + SMnodeMgmt *pMgmt = &pDnode->m; + tWorkerFreeQueue(&pMgmt->mgmtPool, pMgmt->pMgmtQ); + pMgmt->pMgmtQ = NULL; +} + +static int32_t dndInitMnodeMgmtWorker(SDnode *pDnode) { + SMnodeMgmt *pMgmt = &pDnode->m; + SWorkerPool *pPool = &pMgmt->mgmtPool; + pPool->name = "mnode-mgmt"; + pPool->min = 1; + pPool->max = 1; + if (tWorkerInit(pPool) != 0) { + terrno = TSDB_CODE_VND_OUT_OF_MEMORY; + return -1; + } + + return 0; +} + +static void dndCleanupMnodeMgmtWorker(SDnode *pDnode) { + SMnodeMgmt *pMgmt = &pDnode->m; + ; + tWorkerCleanup(&pMgmt->mgmtPool); +} + +static int32_t dndAllocMnodeReadQueue(SDnode *pDnode) { + SMnodeMgmt *pMgmt = &pDnode->m; + pMgmt->pReadQ = tWorkerAllocQueue(&pMgmt->readPool, NULL, (FProcessItem)dndProcessMnodeReadQueue); + if (pMgmt->pReadQ == NULL) { + terrno = TSDB_CODE_OUT_OF_MEMORY; + return -1; + } + return 0; +} + +static void dndFreeMnodeReadQueue(SDnode *pDnode) { + SMnodeMgmt *pMgmt = &pDnode->m; + tWorkerFreeQueue(&pMgmt->readPool, pMgmt->pReadQ); + pMgmt->pReadQ = NULL; +} + +static int32_t dndInitMnodeReadWorker(SDnode *pDnode) { + SMnodeMgmt *pMgmt = &pDnode->m; + SWorkerPool *pPool = &pMgmt->readPool; + pPool->name = "mnode-read"; + pPool->min = 0; + pPool->max = 1; + if (tWorkerInit(pPool) != 0) { + terrno = TSDB_CODE_VND_OUT_OF_MEMORY; + return -1; + } + + return 0; +} + +static void dndCleanupMnodeReadWorker(SDnode *pDnode) { + SMnodeMgmt *pMgmt = &pDnode->m; + tWorkerCleanup(&pMgmt->readPool); +} + +static int32_t dndAllocMnodeWriteQueue(SDnode *pDnode) { + SMnodeMgmt *pMgmt = &pDnode->m; + pMgmt->pWriteQ = tWorkerAllocQueue(&pMgmt->writePool, NULL, (FProcessItem)dndProcessMnodeWriteQueue); + if (pMgmt->pWriteQ == NULL) { + terrno = TSDB_CODE_OUT_OF_MEMORY; + return -1; + } + return 0; +} + +static void dndFreeMnodeWriteQueue(SDnode *pDnode) { + SMnodeMgmt *pMgmt = &pDnode->m; + tWorkerFreeQueue(&pMgmt->writePool, pMgmt->pWriteQ); + pMgmt->pWriteQ = NULL; +} + +static int32_t dndAllocMnodeApplyQueue(SDnode *pDnode) { + SMnodeMgmt *pMgmt = &pDnode->m; + pMgmt->pApplyQ = tWorkerAllocQueue(&pMgmt->writePool, NULL, (FProcessItem)dndProcessMnodeApplyQueue); + if (pMgmt->pApplyQ == NULL) { + terrno = TSDB_CODE_OUT_OF_MEMORY; + return -1; + } + return 0; +} + +static void dndFreeMnodeApplyQueue(SDnode *pDnode) { + SMnodeMgmt *pMgmt = &pDnode->m; + tWorkerFreeQueue(&pMgmt->writePool, pMgmt->pApplyQ); + pMgmt->pApplyQ = NULL; +} + +static int32_t dndInitMnodeWriteWorker(SDnode *pDnode) { + SMnodeMgmt *pMgmt = &pDnode->m; + SWorkerPool *pPool = &pMgmt->writePool; + pPool->name = "mnode-write"; + pPool->min = 0; + pPool->max = 1; + if (tWorkerInit(pPool) != 0) { + terrno = TSDB_CODE_VND_OUT_OF_MEMORY; + return -1; + } + + return 0; +} + +static void dndCleanupMnodeWriteWorker(SDnode *pDnode) { + SMnodeMgmt *pMgmt = &pDnode->m; + tWorkerCleanup(&pMgmt->writePool); +} + +static int32_t dndAllocMnodeSyncQueue(SDnode *pDnode) { + SMnodeMgmt *pMgmt = &pDnode->m; + pMgmt->pSyncQ = tWorkerAllocQueue(&pMgmt->syncPool, NULL, (FProcessItem)dndProcessMnodeSyncQueue); + if (pMgmt->pSyncQ == NULL) { + terrno = TSDB_CODE_OUT_OF_MEMORY; + return -1; + } + return 0; +} + +static void dndFreeMnodeSyncQueue(SDnode *pDnode) { + SMnodeMgmt *pMgmt = &pDnode->m; + tWorkerFreeQueue(&pMgmt->syncPool, pMgmt->pSyncQ); + pMgmt->pSyncQ = NULL; +} + +static int32_t dndInitMnodeSyncWorker(SDnode *pDnode) { + SMnodeMgmt *pMgmt = &pDnode->m; + SWorkerPool *pPool = &pMgmt->syncPool; + pPool->name = "mnode-sync"; + pPool->min = 0; + pPool->max = 1; + return tWorkerInit(pPool); +} + +static void dndCleanupMnodeSyncWorker(SDnode *pDnode) { + SMnodeMgmt *pMgmt = &pDnode->m; + tWorkerCleanup(&pMgmt->syncPool); +} + +int32_t dndInitMnode(SDnode *pDnode) { + dInfo("dnode-mnode start to init"); + SMnodeMgmt *pMgmt = &pDnode->m; + taosInitRWLatch(&pMgmt->latch); + + if (dndInitMnodeMgmtWorker(pDnode) != 0) { + terrno = TSDB_CODE_OUT_OF_MEMORY; + return -1; + } + + char path[PATH_MAX]; + snprintf(path, PATH_MAX, "%s/mnode.json", pDnode->dir.dnode); + pMgmt->file = strdup(path); + if (pMgmt->file == NULL) { + terrno = TSDB_CODE_OUT_OF_MEMORY; + return -1; + } + + if (dndReadMnodeFile(pDnode) != 0) { + return -1; + } + + if (pMgmt->dropped) { + dInfo("mnode has been deployed and needs to be deleted"); + mnodeDestroy(pDnode->dir.mnode); + return 0; + } + + if (!pMgmt->deployed) { + bool needDeploy = dndNeedDeployMnode(pDnode); + if (!needDeploy) { + dDebug("mnode does not need to be deployed"); + return 0; + } + + dInfo("start to deploy mnode"); + } else { + dInfo("start to open mnode"); + } + + SMnodeOptions option = {0}; + dndInitMnodeOptions(pDnode, &option); + return dndOpenMnode(pDnode, &option); +} + +void dndCleanupMnode(SDnode *pDnode) { + SMnodeMgmt *pMgmt = &pDnode->m; + + dInfo("dnode-mnode start to clean up"); + dndStopMnodeWorker(pDnode); + dndCleanupMnodeMgmtWorker(pDnode); + tfree(pMgmt->file); + dInfo("dnode-mnode is cleaned up"); +} + +int32_t dndGetUserAuthFromMnode(SDnode *pDnode, char *user, char *spi, char *encrypt, char *secret, char *ckey) { + SMnodeMgmt *pMgmt = &pDnode->m; + + SMnode *pMnode = dndAcquireMnode(pDnode); + if (pMnode == NULL) { + terrno = TSDB_CODE_APP_NOT_READY; + dTrace("failed to get user auth since %s", terrstr()); + return -1; + } + + int32_t code = mnodeRetriveAuth(pMnode, user, spi, encrypt, secret, ckey); + dndReleaseMnode(pDnode, pMnode); + return code; +} diff --git a/source/dnode/mgmt/impl/src/dndTransport.c b/source/dnode/mgmt/impl/src/dndTransport.c new file mode 100644 index 0000000000..679e3ef5f9 --- /dev/null +++ b/source/dnode/mgmt/impl/src/dndTransport.c @@ -0,0 +1,328 @@ +/* + * 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 . + */ + +/* this file is mainly responsible for the communication between DNODEs. Each + * dnode works as both server and client. Dnode may send status, grant, config + * messages to mnode, mnode may send create/alter/drop table/vnode messages + * to dnode. All theses messages are handled from here + */ + +#define _DEFAULT_SOURCE +#include "dndTransport.h" +#include "dndDnode.h" +#include "dndMnode.h" +#include "dndVnodes.h" + +static void dndInitMsgFp(STransMgmt *pMgmt) { + // msg from client to dnode + pMgmt->msgFp[TSDB_MSG_TYPE_SUBMIT] = dndProcessVnodeWriteMsg; + pMgmt->msgFp[TSDB_MSG_TYPE_QUERY] = dndProcessVnodeQueryMsg; + pMgmt->msgFp[TSDB_MSG_TYPE_FETCH] = dndProcessVnodeFetchMsg; + pMgmt->msgFp[TSDB_MSG_TYPE_CREATE_TABLE] = dndProcessVnodeWriteMsg; + pMgmt->msgFp[TSDB_MSG_TYPE_DROP_TABLE] = dndProcessVnodeWriteMsg; + pMgmt->msgFp[TSDB_MSG_TYPE_ALTER_TABLE] = dndProcessVnodeWriteMsg; + pMgmt->msgFp[TSDB_MSG_TYPE_UPDATE_TAG_VAL] = dndProcessVnodeWriteMsg; + pMgmt->msgFp[TSDB_MSG_TYPE_TABLE_META] = dndProcessVnodeQueryMsg; + pMgmt->msgFp[TSDB_MSG_TYPE_TABLES_META] = dndProcessVnodeQueryMsg; + pMgmt->msgFp[TSDB_MSG_TYPE_MQ_QUERY] = dndProcessVnodeQueryMsg; + pMgmt->msgFp[TSDB_MSG_TYPE_MQ_CONSUME] = dndProcessVnodeQueryMsg; + pMgmt->msgFp[TSDB_MSG_TYPE_MQ_CONNECT] = dndProcessVnodeWriteMsg; + pMgmt->msgFp[TSDB_MSG_TYPE_MQ_DISCONNECT] = dndProcessVnodeWriteMsg; + pMgmt->msgFp[TSDB_MSG_TYPE_MQ_ACK] = dndProcessVnodeWriteMsg; + pMgmt->msgFp[TSDB_MSG_TYPE_MQ_RESET] = dndProcessVnodeWriteMsg; + + // msg from client to mnode + pMgmt->msgFp[TSDB_MSG_TYPE_CONNECT] = dndProcessMnodeReadMsg; + pMgmt->msgFp[TSDB_MSG_TYPE_CREATE_ACCT] = dndProcessMnodeWriteMsg; + pMgmt->msgFp[TSDB_MSG_TYPE_ALTER_ACCT] = dndProcessMnodeWriteMsg; + pMgmt->msgFp[TSDB_MSG_TYPE_DROP_ACCT] = dndProcessMnodeWriteMsg; + pMgmt->msgFp[TSDB_MSG_TYPE_CREATE_USER] = dndProcessMnodeWriteMsg; + pMgmt->msgFp[TSDB_MSG_TYPE_ALTER_USER] = dndProcessMnodeWriteMsg; + pMgmt->msgFp[TSDB_MSG_TYPE_DROP_USER] = dndProcessMnodeWriteMsg; + pMgmt->msgFp[TSDB_MSG_TYPE_CREATE_DNODE] = dndProcessMnodeWriteMsg; + pMgmt->msgFp[TSDB_MSG_TYPE_CONFIG_DNODE] = dndProcessMnodeWriteMsg; + pMgmt->msgFp[TSDB_MSG_TYPE_DROP_DNODE] = dndProcessMnodeWriteMsg; + pMgmt->msgFp[TSDB_MSG_TYPE_CREATE_DB] = dndProcessMnodeWriteMsg; + pMgmt->msgFp[TSDB_MSG_TYPE_DROP_DB] = dndProcessMnodeWriteMsg; + pMgmt->msgFp[TSDB_MSG_TYPE_USE_DB] = dndProcessMnodeWriteMsg; + pMgmt->msgFp[TSDB_MSG_TYPE_ALTER_DB] = dndProcessMnodeWriteMsg; + pMgmt->msgFp[TSDB_MSG_TYPE_SYNC_DB] = dndProcessMnodeWriteMsg; + pMgmt->msgFp[TSDB_MSG_TYPE_CREATE_TOPIC] = dndProcessMnodeWriteMsg; + pMgmt->msgFp[TSDB_MSG_TYPE_DROP_TOPIC] = dndProcessMnodeWriteMsg; + pMgmt->msgFp[TSDB_MSG_TYPE_ALTER_TOPIC] = dndProcessMnodeWriteMsg; + pMgmt->msgFp[TSDB_MSG_TYPE_CREATE_FUNCTION] = dndProcessMnodeWriteMsg; + pMgmt->msgFp[TSDB_MSG_TYPE_ALTER_FUNCTION] = dndProcessMnodeWriteMsg; + pMgmt->msgFp[TSDB_MSG_TYPE_DROP_FUNCTION] = dndProcessMnodeWriteMsg; + pMgmt->msgFp[TSDB_MSG_TYPE_CREATE_STABLE] = dndProcessMnodeWriteMsg; + pMgmt->msgFp[TSDB_MSG_TYPE_ALTER_STABLE] = dndProcessMnodeWriteMsg; + pMgmt->msgFp[TSDB_MSG_TYPE_DROP_STABLE] = dndProcessMnodeWriteMsg; + pMgmt->msgFp[TSDB_MSG_TYPE_STABLE_VGROUP] = dndProcessMnodeReadMsg; + pMgmt->msgFp[TSDB_MSG_TYPE_KILL_QUERY] = dndProcessMnodeWriteMsg; + pMgmt->msgFp[TSDB_MSG_TYPE_KILL_CONN] = dndProcessMnodeWriteMsg; + pMgmt->msgFp[TSDB_MSG_TYPE_HEARTBEAT] = dndProcessMnodeReadMsg; + pMgmt->msgFp[TSDB_MSG_TYPE_SHOW] = dndProcessMnodeReadMsg; + pMgmt->msgFp[TSDB_MSG_TYPE_SHOW_RETRIEVE] = dndProcessMnodeReadMsg; + pMgmt->msgFp[TSDB_MSG_TYPE_SHOW_RETRIEVE_FUNC] = dndProcessMnodeReadMsg; + pMgmt->msgFp[TSDB_MSG_TYPE_COMPACT_VNODE] = dndProcessMnodeWriteMsg; + + // message from client to dnode + pMgmt->msgFp[TSDB_MSG_TYPE_NETWORK_TEST] = dndProcessDnodeReq; + + // message from mnode to vnode + pMgmt->msgFp[TSDB_MSG_TYPE_CREATE_STABLE_IN] = dndProcessVnodeWriteMsg; + pMgmt->msgFp[TSDB_MSG_TYPE_CREATE_STABLE_IN_RSP] = dndProcessMnodeWriteMsg; + pMgmt->msgFp[TSDB_MSG_TYPE_ALTER_STABLE_IN] = dndProcessVnodeWriteMsg; + pMgmt->msgFp[TSDB_MSG_TYPE_ALTER_STABLE_IN_RSP] = dndProcessMnodeWriteMsg; + pMgmt->msgFp[TSDB_MSG_TYPE_DROP_STABLE_IN] = dndProcessVnodeWriteMsg; + pMgmt->msgFp[TSDB_MSG_TYPE_DROP_STABLE_IN_RSP] = dndProcessMnodeWriteMsg; + + // message from mnode to dnode + pMgmt->msgFp[TSDB_MSG_TYPE_CREATE_VNODE_IN] = dndProcessVnodeMgmtMsg; + pMgmt->msgFp[TSDB_MSG_TYPE_CREATE_VNODE_IN_RSP] = dndProcessMnodeWriteMsg; + pMgmt->msgFp[TSDB_MSG_TYPE_ALTER_VNODE_IN] = dndProcessVnodeMgmtMsg; + pMgmt->msgFp[TSDB_MSG_TYPE_ALTER_VNODE_IN_RSP] = dndProcessMnodeWriteMsg; + pMgmt->msgFp[TSDB_MSG_TYPE_DROP_VNODE_IN] = dndProcessVnodeMgmtMsg; + pMgmt->msgFp[TSDB_MSG_TYPE_DROP_VNODE_IN_RSP] = dndProcessMnodeWriteMsg; + pMgmt->msgFp[TSDB_MSG_TYPE_SYNC_VNODE_IN] = dndProcessVnodeMgmtMsg; + pMgmt->msgFp[TSDB_MSG_TYPE_SYNC_VNODE_IN_RSP] = dndProcessMnodeWriteMsg; + pMgmt->msgFp[TSDB_MSG_TYPE_AUTH_VNODE_IN] = dndProcessVnodeMgmtMsg; + pMgmt->msgFp[TSDB_MSG_TYPE_AUTH_VNODE_IN_RSP] = dndProcessMnodeWriteMsg; + pMgmt->msgFp[TSDB_MSG_TYPE_COMPACT_VNODE_IN] = dndProcessVnodeMgmtMsg; + pMgmt->msgFp[TSDB_MSG_TYPE_COMPACT_VNODE_IN_RSP] = dndProcessMnodeWriteMsg; + pMgmt->msgFp[TSDB_MSG_TYPE_CREATE_MNODE_IN] = dndProcessMnodeMgmtMsg; + pMgmt->msgFp[TSDB_MSG_TYPE_CREATE_MNODE_IN_RSP] = dndProcessMnodeWriteMsg; + pMgmt->msgFp[TSDB_MSG_TYPE_ALTER_MNODE_IN] = dndProcessMnodeMgmtMsg; + pMgmt->msgFp[TSDB_MSG_TYPE_ALTER_MNODE_IN_RSP] = dndProcessMnodeWriteMsg; + pMgmt->msgFp[TSDB_MSG_TYPE_DROP_MNODE_IN] = dndProcessMnodeMgmtMsg; + pMgmt->msgFp[TSDB_MSG_TYPE_DROP_MNODE_IN_RSP] = dndProcessMnodeWriteMsg; + pMgmt->msgFp[TSDB_MSG_TYPE_CONFIG_DNODE_IN] = dndProcessDnodeReq; + pMgmt->msgFp[TSDB_MSG_TYPE_CONFIG_DNODE_IN_RSP] = dndProcessMnodeWriteMsg; + + // message from dnode to mnode + pMgmt->msgFp[TSDB_MSG_TYPE_AUTH] = dndProcessMnodeReadMsg; + pMgmt->msgFp[TSDB_MSG_TYPE_AUTH_RSP] = dndProcessDnodeRsp; + pMgmt->msgFp[TSDB_MSG_TYPE_GRANT] = dndProcessMnodeWriteMsg; + pMgmt->msgFp[TSDB_MSG_TYPE_GRANT_RSP] = dndProcessDnodeRsp; + pMgmt->msgFp[TSDB_MSG_TYPE_STATUS] = dndProcessMnodeWriteMsg; + pMgmt->msgFp[TSDB_MSG_TYPE_STATUS_RSP] = dndProcessDnodeRsp; +} + +static void dndProcessResponse(void *parent, SRpcMsg *pMsg, SEpSet *pEpSet) { + SDnode *pDnode = parent; + STransMgmt *pMgmt = &pDnode->t; + + int32_t msgType = pMsg->msgType; + + if (dndGetStat(pDnode) == DND_STAT_STOPPED) { + if (pMsg == NULL || pMsg->pCont == NULL) return; + dTrace("RPC %p, rsp:%s is ignored since dnode is stopping", pMsg->handle, taosMsg[msgType]); + rpcFreeCont(pMsg->pCont); + return; + } + + DndMsgFp fp = pMgmt->msgFp[msgType]; + if (fp != NULL) { + dTrace("RPC %p, rsp:%s will be processed, code:%s", pMsg->handle, taosMsg[msgType], tstrerror(pMsg->code)); + (*fp)(pDnode, pMsg, pEpSet); + } else { + dError("RPC %p, rsp:%s not processed", pMsg->handle, taosMsg[msgType]); + rpcFreeCont(pMsg->pCont); + } +} + +static int32_t dndInitClient(SDnode *pDnode) { + STransMgmt *pMgmt = &pDnode->t; + + SRpcInit rpcInit; + memset(&rpcInit, 0, sizeof(rpcInit)); + rpcInit.label = "DND-C"; + rpcInit.numOfThreads = 1; + rpcInit.cfp = dndProcessResponse; + rpcInit.sessions = TSDB_MAX_VNODES << 4; + rpcInit.connType = TAOS_CONN_CLIENT; + rpcInit.idleTime = pDnode->opt.shellActivityTimer * 1000; + rpcInit.user = "-internal"; + rpcInit.ckey = "-key"; + rpcInit.secret = "-secret"; + + pMgmt->clientRpc = rpcOpen(&rpcInit); + if (pMgmt->clientRpc == NULL) { + dError("failed to init rpc client"); + return -1; + } + + return 0; +} + +static void dndCleanupClient(SDnode *pDnode) { + STransMgmt *pMgmt = &pDnode->t; + if (pMgmt->clientRpc) { + rpcClose(pMgmt->clientRpc); + pMgmt->clientRpc = NULL; + dInfo("dnode peer rpc client is closed"); + } +} + +static void dndProcessRequest(void *param, SRpcMsg *pMsg, SEpSet *pEpSet) { + SDnode *pDnode = param; + STransMgmt *pMgmt = &pDnode->t; + + int32_t msgType = pMsg->msgType; + if (msgType == TSDB_MSG_TYPE_NETWORK_TEST) { + dndProcessDnodeReq(pDnode, pMsg, pEpSet); + return; + } + + if (dndGetStat(pDnode) == DND_STAT_STOPPED) { + dError("RPC %p, req:%s is ignored since dnode exiting", pMsg->handle, taosMsg[msgType]); + SRpcMsg rspMsg = {.handle = pMsg->handle, .code = TSDB_CODE_DND_EXITING}; + rpcSendResponse(&rspMsg); + rpcFreeCont(pMsg->pCont); + return; + } else if (dndGetStat(pDnode) != DND_STAT_RUNNING) { + dError("RPC %p, req:%s is ignored since dnode not running", pMsg->handle, taosMsg[msgType]); + SRpcMsg rspMsg = {.handle = pMsg->handle, .code = TSDB_CODE_APP_NOT_READY}; + rpcSendResponse(&rspMsg); + rpcFreeCont(pMsg->pCont); + return; + } + + if (pMsg->pCont == NULL) { + SRpcMsg rspMsg = {.handle = pMsg->handle, .code = TSDB_CODE_DND_INVALID_MSG_LEN}; + rpcSendResponse(&rspMsg); + return; + } + + DndMsgFp fp = pMgmt->msgFp[msgType]; + if (fp != NULL) { + dTrace("RPC %p, req:%s will be processed", pMsg->handle, taosMsg[msgType]); + (*fp)(pDnode, pMsg, pEpSet); + } else { + dError("RPC %p, req:%s is not processed", pMsg->handle, taosMsg[msgType]); + SRpcMsg rspMsg = {.handle = pMsg->handle, .code = TSDB_CODE_MSG_NOT_PROCESSED}; + rpcSendResponse(&rspMsg); + rpcFreeCont(pMsg->pCont); + } +} + +static void dndSendMsgToMnodeRecv(SDnode *pDnode, SRpcMsg *pRpcMsg, SRpcMsg *pRpcRsp) { + STransMgmt *pMgmt = &pDnode->t; + + SEpSet epSet = {0}; + dndGetMnodeEpSet(pDnode, &epSet); + rpcSendRecv(pMgmt->clientRpc, &epSet, pRpcMsg, pRpcRsp); +} + +static int32_t dndRetrieveUserAuthInfo(void *parent, char *user, char *spi, char *encrypt, char *secret, char *ckey) { + SDnode *pDnode = parent; + + if (dndGetUserAuthFromMnode(pDnode, user, spi, encrypt, secret, ckey) != 0) { + if (terrno != TSDB_CODE_APP_NOT_READY) { + dTrace("failed to get user auth from mnode since %s", terrstr()); + return -1; + } + } + + dDebug("user:%s, send auth msg to mnodes", user); + + SAuthMsg *pMsg = rpcMallocCont(sizeof(SAuthMsg)); + tstrncpy(pMsg->user, user, TSDB_USER_LEN); + + SRpcMsg rpcMsg = {.pCont = pMsg, .contLen = sizeof(SAuthMsg), .msgType = TSDB_MSG_TYPE_AUTH}; + SRpcMsg rpcRsp = {0}; + dndSendMsgToMnodeRecv(pDnode, &rpcMsg, &rpcRsp); + + if (rpcRsp.code != 0) { + terrno = rpcRsp.code; + dError("user:%s, failed to get user auth from mnodes since %s", user, terrstr()); + } else { + SAuthRsp *pRsp = rpcRsp.pCont; + memcpy(secret, pRsp->secret, TSDB_KEY_LEN); + memcpy(ckey, pRsp->ckey, TSDB_KEY_LEN); + *spi = pRsp->spi; + *encrypt = pRsp->encrypt; + dDebug("user:%s, success to get user auth from mnodes", user); + } + + rpcFreeCont(rpcRsp.pCont); + return rpcRsp.code; +} + +static int32_t dndInitServer(SDnode *pDnode) { + STransMgmt *pMgmt = &pDnode->t; + dndInitMsgFp(pMgmt); + + int32_t numOfThreads = (int32_t)((pDnode->opt.numOfCores * pDnode->opt.numOfThreadsPerCore) / 2.0); + if (numOfThreads < 1) { + numOfThreads = 1; + } + + SRpcInit rpcInit; + memset(&rpcInit, 0, sizeof(rpcInit)); + rpcInit.localPort = pDnode->opt.serverPort; + rpcInit.label = "DND-S"; + rpcInit.numOfThreads = numOfThreads; + rpcInit.cfp = dndProcessRequest; + rpcInit.sessions = pDnode->opt.maxShellConns; + rpcInit.connType = TAOS_CONN_SERVER; + rpcInit.idleTime = pDnode->opt.shellActivityTimer * 1000; + rpcInit.afp = dndRetrieveUserAuthInfo; + + pMgmt->serverRpc = rpcOpen(&rpcInit); + if (pMgmt->serverRpc == NULL) { + dError("failed to init rpc server"); + return -1; + } + + return 0; +} + +static void dndCleanupServer(SDnode *pDnode) { + STransMgmt *pMgmt = &pDnode->t; + if (pMgmt->serverRpc) { + rpcClose(pMgmt->serverRpc); + pMgmt->serverRpc = NULL; + } +} + +int32_t dndInitTrans(SDnode *pDnode) { + if (dndInitClient(pDnode) != 0) { + return -1; + } + + if (dndInitServer(pDnode) != 0) { + return -1; + } + + dInfo("dnode-transport is initialized"); + return 0; +} + +void dndCleanupTrans(SDnode *pDnode) { + dndCleanupServer(pDnode); + dndCleanupClient(pDnode); + dInfo("dnode-transport is cleaned up"); +} + +void dndSendMsgToDnode(SDnode *pDnode, SEpSet *pEpSet, SRpcMsg *pMsg) { + STransMgmt *pMgmt = &pDnode->t; + rpcSendRequest(pMgmt->clientRpc, pEpSet, pMsg, NULL); +} + +void dndSendMsgToMnode(SDnode *pDnode, SRpcMsg *pMsg) { + SEpSet epSet = {0}; + dndGetMnodeEpSet(pDnode, &epSet); + dndSendMsgToDnode(pDnode, &epSet, pMsg); +} \ No newline at end of file diff --git a/source/dnode/mgmt/impl/src/dndVnodes.c b/source/dnode/mgmt/impl/src/dndVnodes.c new file mode 100644 index 0000000000..d5e94106a7 --- /dev/null +++ b/source/dnode/mgmt/impl/src/dndVnodes.c @@ -0,0 +1,1119 @@ +/* + * Copyright (c) 2019 TAOS Data, Inc. + * + * This program is free software: you can use, redistribute, and/or modify + * it under the terms of the GNU Affero General Public License, version 3 + * or later ("AGPL"), as published by the Free Software Foundation. + * + * This program is distributed in the hope that it will be useful, but WITHOUT + * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or + * FITNESS FOR A PARTICULAR PURPOSE. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + */ + +#define _DEFAULT_SOURCE +#include "dndVnodes.h" +#include "dndTransport.h" + +typedef struct { + int32_t vgId; + int32_t refCount; + int8_t dropped; + int8_t accessState; + SVnode *pImpl; + taos_queue pWriteQ; + taos_queue pSyncQ; + taos_queue pApplyQ; + taos_queue pQueryQ; + taos_queue pFetchQ; +} SVnodeObj; + +typedef struct { + int32_t vnodeNum; + int32_t opened; + int32_t failed; + int32_t threadIndex; + pthread_t *pThreadId; + SVnodeObj *pVnodes; + SDnode *pDnode; +} SVnodeThread; + +static int32_t dndInitVnodeReadWorker(SDnode *pDnode); +static int32_t dndInitVnodeWriteWorker(SDnode *pDnode); +static int32_t dndInitVnodeSyncWorker(SDnode *pDnode); +static int32_t dndInitVnodeMgmtWorker(SDnode *pDnode); +static void dndCleanupVnodeReadWorker(SDnode *pDnode); +static void dndCleanupVnodeWriteWorker(SDnode *pDnode); +static void dndCleanupVnodeSyncWorker(SDnode *pDnode); +static void dndCleanupVnodeMgmtWorker(SDnode *pDnode); +static int32_t dndAllocVnodeQueryQueue(SDnode *pDnode, SVnodeObj *pVnode); +static int32_t dndAllocVnodeFetchQueue(SDnode *pDnode, SVnodeObj *pVnode); +static int32_t dndAllocVnodeWriteQueue(SDnode *pDnode, SVnodeObj *pVnode); +static int32_t dndAllocVnodeApplyQueue(SDnode *pDnode, SVnodeObj *pVnode); +static int32_t dndAllocVnodeSyncQueue(SDnode *pDnode, SVnodeObj *pVnode); +static void dndFreeVnodeQueryQueue(SDnode *pDnode, SVnodeObj *pVnode); +static void dndFreeVnodeFetchQueue(SDnode *pDnode, SVnodeObj *pVnode); +static void dndFreeVnodeWriteQueue(SDnode *pDnode, SVnodeObj *pVnode); +static void dndFreeVnodeApplyQueue(SDnode *pDnode, SVnodeObj *pVnode); +static void dndFreeVnodeSyncQueue(SDnode *pDnode, SVnodeObj *pVnode); + +static void dndProcessVnodeQueryQueue(SVnodeObj *pVnode, SVnodeMsg *pMsg); +static void dndProcessVnodeFetchQueue(SVnodeObj *pVnode, SVnodeMsg *pMsg); +static void dndProcessVnodeWriteQueue(SVnodeObj *pVnode, taos_qall qall, int32_t numOfMsgs); +static void dndProcessVnodeApplyQueue(SVnodeObj *pVnode, taos_qall qall, int32_t numOfMsgs); +static void dndProcessVnodeSyncQueue(SVnodeObj *pVnode, taos_qall qall, int32_t numOfMsgs); +static void dndProcessVnodeMgmtQueue(SDnode *pDnode, SRpcMsg *pMsg); +void dndProcessVnodeQueryMsg(SDnode *pDnode, SRpcMsg *pMsg, SEpSet *pEpSet); +void dndProcessVnodeFetchMsg(SDnode *pDnode, SRpcMsg *pMsg, SEpSet *pEpSet); +void dndProcessVnodeWriteMsg(SDnode *pDnode, SRpcMsg *pMsg, SEpSet *pEpSet); +void dndProcessVnodeSyncMsg(SDnode *pDnode, SRpcMsg *pMsg, SEpSet *pEpSet); +void dndProcessVnodeMgmtMsg(SDnode *pDnode, SRpcMsg *pMsg, SEpSet *pEpSet); +static int32_t dndPutMsgIntoVnodeApplyQueue(SDnode *pDnode, int32_t vgId, SVnodeMsg *pMsg); + +static SVnodeObj *dndAcquireVnode(SDnode *pDnode, int32_t vgId); +static void dndReleaseVnode(SDnode *pDnode, SVnodeObj *pVnode); +static int32_t dndCreateVnodeWrapper(SDnode *pDnode, int32_t vgId, SVnode *pImpl); +static void dndDropVnodeWrapper(SDnode *pDnode, SVnodeObj *pVnode); +static SVnodeObj **dndGetVnodesFromHash(SDnode *pDnode, int32_t *numOfVnodes); +static int32_t dndGetVnodesFromFile(SDnode *pDnode, SVnodeObj **ppVnodes, int32_t *numOfVnodes); +static int32_t dndWriteVnodesToFile(SDnode *pDnode); + +static int32_t dndCreateVnode(SDnode *pDnode, int32_t vgId, SVnodeCfg *pCfg); +static int32_t dndDropVnode(SDnode *pDnode, SVnodeObj *pVnode); +static int32_t dndOpenVnodes(SDnode *pDnode); +static void dndCloseVnodes(SDnode *pDnode); + +static int32_t dndProcessCreateVnodeReq(SDnode *pDnode, SRpcMsg *rpcMsg); +static int32_t dndProcessAlterVnodeReq(SDnode *pDnode, SRpcMsg *rpcMsg); +static int32_t dndProcessDropVnodeReq(SDnode *pDnode, SRpcMsg *rpcMsg); +static int32_t dndProcessAuthVnodeReq(SDnode *pDnode, SRpcMsg *rpcMsg); +static int32_t dndProcessSyncVnodeReq(SDnode *pDnode, SRpcMsg *rpcMsg); +static int32_t dndProcessCompactVnodeReq(SDnode *pDnode, SRpcMsg *rpcMsg); + +static SVnodeObj *dndAcquireVnode(SDnode *pDnode, int32_t vgId) { + SVnodesMgmt *pMgmt = &pDnode->vmgmt; + SVnodeObj *pVnode = NULL; + int32_t refCount = 0; + + taosRLockLatch(&pMgmt->latch); + taosHashGetClone(pMgmt->hash, &vgId, sizeof(int32_t), (void *)&pVnode); + if (pVnode == NULL) { + terrno = TSDB_CODE_VND_INVALID_VGROUP_ID; + } else { + refCount = atomic_add_fetch_32(&pVnode->refCount, 1); + } + taosRUnLockLatch(&pMgmt->latch); + + dTrace("vgId:%d, acquire vnode, refCount:%d", pVnode->vgId, refCount); + return pVnode; +} + +static void dndReleaseVnode(SDnode *pDnode, SVnodeObj *pVnode) { + SVnodesMgmt *pMgmt = &pDnode->vmgmt; + int32_t refCount = 0; + + taosRLockLatch(&pMgmt->latch); + if (pVnode != NULL) { + refCount = atomic_sub_fetch_32(&pVnode->refCount, 1); + } + taosRUnLockLatch(&pMgmt->latch); + + if (pVnode != NULL) { + dTrace("vgId:%d, release vnode, refCount:%d", pVnode->vgId, refCount); + } +} + +static int32_t dndCreateVnodeWrapper(SDnode *pDnode, int32_t vgId, SVnode *pImpl) { + SVnodesMgmt *pMgmt = &pDnode->vmgmt; + SVnodeObj *pVnode = calloc(1, sizeof(SVnodeObj)); + if (pVnode == NULL) { + terrno = TSDB_CODE_OUT_OF_MEMORY; + return -1; + } + + pVnode->vgId = vgId; + pVnode->refCount = 0; + pVnode->dropped = 0; + pVnode->accessState = TSDB_VN_ALL_ACCCESS; + pVnode->pImpl = pImpl; + + if (dndAllocVnodeQueryQueue(pDnode, pVnode) != 0) { + return -1; + } + + if (dndAllocVnodeFetchQueue(pDnode, pVnode) != 0) { + return -1; + } + + if (dndAllocVnodeWriteQueue(pDnode, pVnode) != 0) { + return -1; + } + + if (dndAllocVnodeApplyQueue(pDnode, pVnode) != 0) { + return -1; + } + + if (dndAllocVnodeSyncQueue(pDnode, pVnode) != 0) { + return -1; + } + + taosWLockLatch(&pMgmt->latch); + int32_t code = taosHashPut(pMgmt->hash, &vgId, sizeof(int32_t), &pVnode, sizeof(SVnodeObj *)); + taosWUnLockLatch(&pMgmt->latch); + + if (code != 0) { + terrno = TSDB_CODE_OUT_OF_MEMORY; + } + return code; +} + +static void dndDropVnodeWrapper(SDnode *pDnode, SVnodeObj *pVnode) { + SVnodesMgmt *pMgmt = &pDnode->vmgmt; + taosWLockLatch(&pMgmt->latch); + taosHashRemove(pMgmt->hash, &pVnode->vgId, sizeof(int32_t)); + taosWUnLockLatch(&pMgmt->latch); + + dndReleaseVnode(pDnode, pVnode); + while (pVnode->refCount > 0) taosMsleep(10); + while (!taosQueueEmpty(pVnode->pWriteQ)) taosMsleep(10); + while (!taosQueueEmpty(pVnode->pSyncQ)) taosMsleep(10); + while (!taosQueueEmpty(pVnode->pApplyQ)) taosMsleep(10); + while (!taosQueueEmpty(pVnode->pQueryQ)) taosMsleep(10); + while (!taosQueueEmpty(pVnode->pFetchQ)) taosMsleep(10); + + dndFreeVnodeQueryQueue(pDnode, pVnode); + dndFreeVnodeFetchQueue(pDnode, pVnode); + dndFreeVnodeWriteQueue(pDnode, pVnode); + dndFreeVnodeApplyQueue(pDnode, pVnode); + dndFreeVnodeSyncQueue(pDnode, pVnode); +} + +static SVnodeObj **dndGetVnodesFromHash(SDnode *pDnode, int32_t *numOfVnodes) { + SVnodesMgmt *pMgmt = &pDnode->vmgmt; + taosRLockLatch(&pMgmt->latch); + + int32_t num = 0; + int32_t size = taosHashGetSize(pMgmt->hash); + SVnodeObj **pVnodes = calloc(size, sizeof(SVnodeObj *)); + + void *pIter = taosHashIterate(pMgmt->hash, NULL); + while (pIter) { + SVnodeObj **ppVnode = pIter; + SVnodeObj *pVnode = *ppVnode; + if (pVnode) { + num++; + if (num < size) { + int32_t refCount = atomic_add_fetch_32(&pVnode->refCount, 1); + dTrace("vgId:%d, acquire vnode, refCount:%d", pVnode->vgId, refCount); + pVnodes[num] = (*ppVnode); + } + } + pIter = taosHashIterate(pMgmt->hash, pIter); + } + + taosRUnLockLatch(&pMgmt->latch); + *numOfVnodes = num; + + return pVnodes; +} + +static int32_t dndGetVnodesFromFile(SDnode *pDnode, SVnodeObj **ppVnodes, int32_t *numOfVnodes) { + int32_t code = TSDB_CODE_DND_VNODE_READ_FILE_ERROR; + int32_t len = 0; + int32_t maxLen = 30000; + char *content = calloc(1, maxLen + 1); + cJSON *root = NULL; + FILE *fp = NULL; + char file[PATH_MAX + 20] = {0}; + SVnodeObj *pVnodes = NULL; + + snprintf(file, PATH_MAX + 20, "%s/vnodes.json", pDnode->dir.vnodes); + + fp = fopen(file, "r"); + if (!fp) { + dDebug("file %s not exist", file); + code = 0; + goto PRASE_VNODE_OVER; + } + + len = (int32_t)fread(content, 1, maxLen, fp); + if (len <= 0) { + dError("failed to read %s since content is null", file); + goto PRASE_VNODE_OVER; + } + + content[len] = 0; + root = cJSON_Parse(content); + if (root == NULL) { + dError("failed to read %s since invalid json format", file); + goto PRASE_VNODE_OVER; + } + + cJSON *vnodes = cJSON_GetObjectItem(root, "vnodes"); + if (!vnodes || vnodes->type != cJSON_Array) { + dError("failed to read %s since vnodes not found", file); + goto PRASE_VNODE_OVER; + } + + int32_t vnodesNum = cJSON_GetArraySize(vnodes); + if (vnodesNum <= 0) { + dError("failed to read %s since vnodes size:%d invalid", file, vnodesNum); + goto PRASE_VNODE_OVER; + } + + pVnodes = calloc(vnodesNum, sizeof(SVnodeObj)); + if (pVnodes == NULL) { + dError("failed to read %s since out of memory", file); + goto PRASE_VNODE_OVER; + } + + for (int32_t i = 0; i < vnodesNum; ++i) { + cJSON *vnode = cJSON_GetArrayItem(vnodes, i); + SVnodeObj *pVnode = &pVnodes[i]; + + cJSON *vgId = cJSON_GetObjectItem(vnode, "vgId"); + if (!vgId || vgId->type != cJSON_String) { + dError("failed to read %s since vgId not found", file); + goto PRASE_VNODE_OVER; + } + pVnode->vgId = atoi(vgId->valuestring); + + cJSON *dropped = cJSON_GetObjectItem(vnode, "dropped"); + if (!dropped || dropped->type != cJSON_String) { + dError("failed to read %s since dropped not found", file); + goto PRASE_VNODE_OVER; + } + pVnode->dropped = atoi(vnode->valuestring); + } + + code = 0; + dInfo("succcessed to read file %s", file); + +PRASE_VNODE_OVER: + if (content != NULL) free(content); + if (root != NULL) cJSON_Delete(root); + if (fp != NULL) fclose(fp); + + return code; +} + +static int32_t dndWriteVnodesToFile(SDnode *pDnode) { + char file[PATH_MAX + 20] = {0}; + char realfile[PATH_MAX + 20] = {0}; + snprintf(file, PATH_MAX + 20, "%s/vnodes.json.bak", pDnode->dir.vnodes); + snprintf(realfile, PATH_MAX + 20, "%s/vnodes.json", pDnode->dir.vnodes); + + FILE *fp = fopen(file, "w"); + if (fp != NULL) { + terrno = TAOS_SYSTEM_ERROR(errno); + dError("failed to write %s since %s", file, terrstr()); + return -1; + } + + int32_t len = 0; + int32_t maxLen = 30000; + char *content = calloc(1, maxLen + 1); + int32_t numOfVnodes = 0; + SVnodeObj **pVnodes = dndGetVnodesFromHash(pDnode, &numOfVnodes); + + len += snprintf(content + len, maxLen - len, "{\n"); + len += snprintf(content + len, maxLen - len, " \"vnodes\": [{\n"); + for (int32_t i = 0; i < numOfVnodes; ++i) { + SVnodeObj *pVnode = pVnodes[i]; + len += snprintf(content + len, maxLen - len, " \"vgId\": \"%d\",\n", pVnode->vgId); + len += snprintf(content + len, maxLen - len, " \"dropped\": \"%d\"\n", pVnode->dropped); + if (i < numOfVnodes - 1) { + len += snprintf(content + len, maxLen - len, " },{\n"); + } else { + len += snprintf(content + len, maxLen - len, " }]\n"); + } + } + len += snprintf(content + len, maxLen - len, "}\n"); + + fwrite(content, 1, len, fp); + taosFsyncFile(fileno(fp)); + fclose(fp); + free(content); + terrno = 0; + + for (int32_t i = 0; i < numOfVnodes; ++i) { + SVnodeObj *pVnode = pVnodes[i]; + dndReleaseVnode(pDnode, pVnode); + } + + if (pVnodes != NULL) { + free(pVnodes); + } + + dInfo("successed to write %s", file); + return taosRenameFile(file, realfile); +} + +static int32_t dndCreateVnode(SDnode *pDnode, int32_t vgId, SVnodeCfg *pCfg) { + char path[PATH_MAX + 20] = {0}; + snprintf(path, sizeof(path), "%s/vnode%d", pDnode->dir.vnodes, vgId); + SVnode *pImpl = vnodeCreate(vgId, path, pCfg); + + if (pImpl == NULL) { + return -1; + } + + int32_t code = dndCreateVnodeWrapper(pDnode, vgId, pImpl); + if (code != 0) { + vnodeDrop(pImpl); + terrno = code; + return code; + } + + code = dndWriteVnodesToFile(pDnode); + if (code != 0) { + vnodeDrop(pImpl); + terrno = code; + return code; + } + + return 0; +} + +static int32_t dndDropVnode(SDnode *pDnode, SVnodeObj *pVnode) { + pVnode->dropped = 1; + if (dndWriteVnodesToFile(pDnode) != 0) { + pVnode->dropped = 0; + return -1; + } + + dndDropVnodeWrapper(pDnode, pVnode); + vnodeDrop(pVnode->pImpl); + dndWriteVnodesToFile(pDnode); + return 0; +} + +static void *dnodeOpenVnodeFunc(void *param) { + SVnodeThread *pThread = param; + SDnode *pDnode = pThread->pDnode; + SVnodesMgmt *pMgmt = &pDnode->vmgmt; + + dDebug("thread:%d, start to open %d vnodes", pThread->threadIndex, pThread->vnodeNum); + setThreadName("open-vnodes"); + + for (int32_t v = 0; v < pThread->vnodeNum; ++v) { + SVnodeObj *pVnode = &pThread->pVnodes[v]; + + char stepDesc[TSDB_STEP_DESC_LEN] = {0}; + snprintf(stepDesc, TSDB_STEP_DESC_LEN, "vgId:%d, start to restore, %d of %d have been opened", pVnode->vgId, + pMgmt->openVnodes, pMgmt->totalVnodes); + dndReportStartup(pDnode, "open-vnodes", stepDesc); + + char path[PATH_MAX + 20] = {0}; + snprintf(path, sizeof(path), "%s/vnode%d", pDnode->dir.vnodes, pVnode->vgId); + SVnode *pImpl = vnodeOpen(path, NULL); + if (pImpl == NULL) { + dError("vgId:%d, failed to open vnode by thread:%d", pVnode->vgId, pThread->threadIndex); + pThread->failed++; + } else { + dndCreateVnodeWrapper(pDnode, pVnode->vgId, pImpl); + dDebug("vgId:%d, is opened by thread:%d", pVnode->vgId, pThread->threadIndex); + pThread->opened++; + } + + atomic_add_fetch_32(&pMgmt->openVnodes, 1); + } + + dDebug("thread:%d, total vnodes:%d, opened:%d failed:%d", pThread->threadIndex, pThread->vnodeNum, pThread->opened, + pThread->failed); + return NULL; +} + +static int32_t dndOpenVnodes(SDnode *pDnode) { + SVnodesMgmt *pMgmt = &pDnode->vmgmt; + taosInitRWLatch(&pMgmt->latch); + + pMgmt->hash = taosHashInit(TSDB_MIN_VNODES, taosGetDefaultHashFunction(TSDB_DATA_TYPE_INT), true, HASH_NO_LOCK); + if (pMgmt->hash == NULL) { + dError("failed to init vnode hash"); + terrno = TSDB_CODE_VND_OUT_OF_MEMORY; + return -1; + } + + SVnodeObj *pVnodes = NULL; + int32_t numOfVnodes = 0; + if (dndGetVnodesFromFile(pDnode, &pVnodes, &numOfVnodes) != 0) { + dInfo("failed to get vnode list from disk since %s", terrstr()); + return -1; + } + + pMgmt->totalVnodes = numOfVnodes; + + int32_t threadNum = tsNumOfCores; + int32_t vnodesPerThread = numOfVnodes / threadNum + 1; + + SVnodeThread *threads = calloc(threadNum, sizeof(SVnodeThread)); + for (int32_t t = 0; t < threadNum; ++t) { + threads[t].threadIndex = t; + threads[t].pVnodes = calloc(vnodesPerThread, sizeof(SVnodeObj)); + } + + for (int32_t v = 0; v < numOfVnodes; ++v) { + int32_t t = v % threadNum; + SVnodeThread *pThread = &threads[t]; + pThread->pVnodes[pThread->vnodeNum++] = pVnodes[v]; + } + + dInfo("start %d threads to open %d vnodes", threadNum, numOfVnodes); + + for (int32_t t = 0; t < threadNum; ++t) { + SVnodeThread *pThread = &threads[t]; + if (pThread->vnodeNum == 0) continue; + + pThread->pThreadId = taosCreateThread(dnodeOpenVnodeFunc, pThread); + if (pThread->pThreadId == NULL) { + dError("thread:%d, failed to create thread to open vnode, reason:%s", pThread->threadIndex, strerror(errno)); + } + } + + for (int32_t t = 0; t < threadNum; ++t) { + SVnodeThread *pThread = &threads[t]; + taosDestoryThread(pThread->pThreadId); + pThread->pThreadId = NULL; + free(pThread->pVnodes); + } + free(threads); + + if (pMgmt->openVnodes != pMgmt->totalVnodes) { + dError("there are total vnodes:%d, opened:%d", pMgmt->totalVnodes, pMgmt->openVnodes); + return -1; + } else { + dInfo("total vnodes:%d open successfully", pMgmt->totalVnodes); + return 0; + } +} + +static void dndCloseVnodes(SDnode *pDnode) { + SVnodesMgmt *pMgmt = &pDnode->vmgmt; + + int32_t numOfVnodes = 0; + SVnodeObj **pVnodes = dndGetVnodesFromHash(pDnode, &numOfVnodes); + + for (int32_t i = 0; i < numOfVnodes; ++i) { + dndDropVnodeWrapper(pDnode, pVnodes[i]); + } + + if (pVnodes != NULL) { + free(pVnodes); + } + + if (pMgmt->hash != NULL) { + taosHashCleanup(pMgmt->hash); + pMgmt->hash = NULL; + } + + dInfo("total vnodes:%d are all closed", numOfVnodes); +} + +static int32_t dndParseCreateVnodeReq(SRpcMsg *rpcMsg, int32_t *vgId, SVnodeCfg *pCfg) { + SCreateVnodeMsg *pCreate = rpcMsg->pCont; + *vgId = htonl(pCreate->vgId); + + tstrncpy(pCfg->db, pCreate->db, TSDB_FULL_DB_NAME_LEN); + pCfg->cacheBlockSize = htonl(pCreate->cacheBlockSize); + pCfg->totalBlocks = htonl(pCreate->totalBlocks); + pCfg->daysPerFile = htonl(pCreate->daysPerFile); + pCfg->daysToKeep0 = htonl(pCreate->daysToKeep0); + pCfg->daysToKeep1 = htonl(pCreate->daysToKeep1); + pCfg->daysToKeep2 = htonl(pCreate->daysToKeep2); + pCfg->minRowsPerFileBlock = htonl(pCreate->minRowsPerFileBlock); + pCfg->maxRowsPerFileBlock = htonl(pCreate->maxRowsPerFileBlock); + pCfg->precision = pCreate->precision; + pCfg->compression = pCreate->compression; + pCfg->cacheLastRow = pCreate->cacheLastRow; + pCfg->update = pCreate->update; + pCfg->quorum = pCreate->quorum; + pCfg->replica = pCreate->replica; + pCfg->walLevel = pCreate->walLevel; + pCfg->fsyncPeriod = htonl(pCreate->fsyncPeriod); + + for (int32_t i = 0; i < pCfg->replica; ++i) { + pCfg->replicas[i].port = htons(pCreate->replicas[i].port); + tstrncpy(pCfg->replicas[i].fqdn, pCreate->replicas[i].fqdn, TSDB_FQDN_LEN); + } + + return 0; +} + +static SDropVnodeMsg *vnodeParseDropVnodeReq(SRpcMsg *rpcMsg) { + SDropVnodeMsg *pDrop = rpcMsg->pCont; + pDrop->vgId = htonl(pDrop->vgId); + return pDrop; +} + +static SAuthVnodeMsg *vnodeParseAuthVnodeReq(SRpcMsg *rpcMsg) { + SAuthVnodeMsg *pAuth = rpcMsg->pCont; + pAuth->vgId = htonl(pAuth->vgId); + return pAuth; +} + +static int32_t dndProcessCreateVnodeReq(SDnode *pDnode, SRpcMsg *rpcMsg) { + SVnodeCfg vnodeCfg = {0}; + int32_t vgId = 0; + + dndParseCreateVnodeReq(rpcMsg, &vgId, &vnodeCfg); + dDebug("vgId:%d, create vnode req is received", vgId); + + SVnodeObj *pVnode = dndAcquireVnode(pDnode, vgId); + if (pVnode != NULL) { + dDebug("vgId:%d, already exist, return success", vgId); + dndReleaseVnode(pDnode, pVnode); + return 0; + } + + if (dndCreateVnode(pDnode, vgId, &vnodeCfg) != 0) { + dError("vgId:%d, failed to create vnode since %s", vgId, terrstr()); + return terrno; + } + + return 0; +} + +static int32_t dndProcessAlterVnodeReq(SDnode *pDnode, SRpcMsg *rpcMsg) { + SVnodeCfg vnodeCfg = {0}; + int32_t vgId = 0; + + dndParseCreateVnodeReq(rpcMsg, &vgId, &vnodeCfg); + dDebug("vgId:%d, alter vnode req is received", vgId); + + SVnodeObj *pVnode = dndAcquireVnode(pDnode, vgId); + if (pVnode == NULL) { + dDebug("vgId:%d, failed to alter vnode since %s", vgId, terrstr()); + return terrno; + } + + if (vnodeAlter(pVnode->pImpl, &vnodeCfg) != 0) { + dError("vgId:%d, failed to alter vnode since %s", vgId, terrstr()); + dndReleaseVnode(pDnode, pVnode); + return terrno; + } + + dndReleaseVnode(pDnode, pVnode); + return 0; +} + +static int32_t dndProcessDropVnodeReq(SDnode *pDnode, SRpcMsg *rpcMsg) { + SDropVnodeMsg *pDrop = vnodeParseDropVnodeReq(rpcMsg); + + int32_t vgId = pDrop->vgId; + dDebug("vgId:%d, drop vnode req is received", vgId); + + SVnodeObj *pVnode = dndAcquireVnode(pDnode, vgId); + if (pVnode == NULL) { + dDebug("vgId:%d, failed to drop since %s", vgId, terrstr()); + return terrno; + } + + if (dndDropVnode(pDnode, pVnode) != 0) { + dError("vgId:%d, failed to drop vnode since %s", vgId, terrstr()); + dndReleaseVnode(pDnode, pVnode); + return terrno; + } + + return 0; +} + +static int32_t dndProcessAuthVnodeReq(SDnode *pDnode, SRpcMsg *rpcMsg) { + SAuthVnodeMsg *pAuth = (SAuthVnodeMsg *)vnodeParseAuthVnodeReq(rpcMsg); + + int32_t code = 0; + int32_t vgId = pAuth->vgId; + dDebug("vgId:%d, auth vnode req is received", vgId); + + SVnodeObj *pVnode = dndAcquireVnode(pDnode, vgId); + if (pVnode == NULL) { + dDebug("vgId:%d, failed to auth since %s", vgId, terrstr()); + return terrno; + } + + pVnode->accessState = pAuth->accessState; + dndReleaseVnode(pDnode, pVnode); + return 0; +} + +static int32_t dndProcessSyncVnodeReq(SDnode *pDnode, SRpcMsg *rpcMsg) { + SAuthVnodeMsg *pAuth = (SAuthVnodeMsg *)vnodeParseAuthVnodeReq(rpcMsg); + + int32_t vgId = pAuth->vgId; + dDebug("vgId:%d, auth vnode req is received", vgId); + + SVnodeObj *pVnode = dndAcquireVnode(pDnode, vgId); + if (pVnode == NULL) { + dDebug("vgId:%d, failed to auth since %s", vgId, terrstr()); + return terrno; + } + + if (vnodeSync(pVnode->pImpl) != 0) { + dError("vgId:%d, failed to auth vnode since %s", vgId, terrstr()); + dndReleaseVnode(pDnode, pVnode); + return terrno; + } + + dndReleaseVnode(pDnode, pVnode); + return 0; +} + +static int32_t dndProcessCompactVnodeReq(SDnode *pDnode, SRpcMsg *rpcMsg) { + SCompactVnodeMsg *pCompact = (SCompactVnodeMsg *)vnodeParseDropVnodeReq(rpcMsg); + + int32_t vgId = pCompact->vgId; + dDebug("vgId:%d, compact vnode req is received", vgId); + + SVnodeObj *pVnode = dndAcquireVnode(pDnode, vgId); + if (pVnode == NULL) { + dDebug("vgId:%d, failed to compact since %s", vgId, terrstr()); + return terrno; + } + + if (vnodeCompact(pVnode->pImpl) != 0) { + dError("vgId:%d, failed to compact vnode since %s", vgId, terrstr()); + dndReleaseVnode(pDnode, pVnode); + return terrno; + } + + dndReleaseVnode(pDnode, pVnode); + return 0; +} + +static void dndProcessVnodeMgmtQueue(SDnode *pDnode, SRpcMsg *pMsg) { + int32_t code = 0; + + switch (pMsg->msgType) { + case TSDB_MSG_TYPE_CREATE_VNODE_IN: + code = dndProcessCreateVnodeReq(pDnode, pMsg); + break; + case TSDB_MSG_TYPE_ALTER_VNODE_IN: + code = dndProcessAlterVnodeReq(pDnode, pMsg); + break; + case TSDB_MSG_TYPE_DROP_VNODE_IN: + code = dndProcessDropVnodeReq(pDnode, pMsg); + break; + case TSDB_MSG_TYPE_AUTH_VNODE_IN: + code = dndProcessAuthVnodeReq(pDnode, pMsg); + break; + case TSDB_MSG_TYPE_SYNC_VNODE_IN: + code = dndProcessSyncVnodeReq(pDnode, pMsg); + break; + case TSDB_MSG_TYPE_COMPACT_VNODE_IN: + code = dndProcessCompactVnodeReq(pDnode, pMsg); + break; + default: + code = TSDB_CODE_MSG_NOT_PROCESSED; + break; + } + + if (code != 0) { + SRpcMsg rsp = {.code = code, .handle = pMsg->handle}; + rpcSendResponse(&rsp); + rpcFreeCont(pMsg->pCont); + taosFreeQitem(pMsg); + } +} + +static void dndProcessVnodeQueryQueue(SVnodeObj *pVnode, SVnodeMsg *pMsg) { + vnodeProcessMsg(pVnode->pImpl, pMsg, VN_MSG_TYPE_QUERY); +} + +static void dndProcessVnodeFetchQueue(SVnodeObj *pVnode, SVnodeMsg *pMsg) { + vnodeProcessMsg(pVnode->pImpl, pMsg, VN_MSG_TYPE_FETCH); +} + +static void dndProcessVnodeWriteQueue(SVnodeObj *pVnode, taos_qall qall, int32_t numOfMsgs) { + SVnodeMsg *pMsg = vnodeInitMsg(numOfMsgs); + SRpcMsg *pRpcMsg = NULL; + + for (int32_t i = 0; i < numOfMsgs; ++i) { + taosGetQitem(qall, (void **)&pRpcMsg); + vnodeAppendMsg(pMsg, pRpcMsg); + taosFreeQitem(pRpcMsg); + } + + vnodeProcessMsg(pVnode->pImpl, pMsg, VN_MSG_TYPE_WRITE); +} + +static void dndProcessVnodeApplyQueue(SVnodeObj *pVnode, taos_qall qall, int32_t numOfMsgs) { + SVnodeMsg *pMsg = NULL; + for (int32_t i = 0; i < numOfMsgs; ++i) { + taosGetQitem(qall, (void **)&pMsg); + vnodeProcessMsg(pVnode->pImpl, pMsg, VN_MSG_TYPE_APPLY); + } +} + +static void dndProcessVnodeSyncQueue(SVnodeObj *pVnode, taos_qall qall, int32_t numOfMsgs) { + SVnodeMsg *pMsg = NULL; + for (int32_t i = 0; i < numOfMsgs; ++i) { + taosGetQitem(qall, (void **)&pMsg); + vnodeProcessMsg(pVnode->pImpl, pMsg, VN_MSG_TYPE_SYNC); + } +} + +static int32_t dndWriteRpcMsgToVnodeQueue(taos_queue pQueue, SRpcMsg *pRpcMsg) { + int32_t code = 0; + + if (pQueue == NULL) { + code = TSDB_CODE_MSG_NOT_PROCESSED; + } else { + SRpcMsg *pMsg = taosAllocateQitem(sizeof(SRpcMsg)); + if (pMsg == NULL) { + code = TSDB_CODE_OUT_OF_MEMORY; + } else { + *pMsg = *pRpcMsg; + if (taosWriteQitem(pQueue, pMsg) != 0) { + code = TSDB_CODE_OUT_OF_MEMORY; + } + } + } + + if (code != TSDB_CODE_SUCCESS) { + SRpcMsg rsp = {.handle = pRpcMsg->handle, .code = code}; + rpcSendResponse(&rsp); + rpcFreeCont(pRpcMsg->pCont); + } +} + +static int32_t dndWriteVnodeMsgToVnodeQueue(taos_queue pQueue, SRpcMsg *pRpcMsg) { + int32_t code = 0; + + if (pQueue == NULL) { + code = TSDB_CODE_MSG_NOT_PROCESSED; + } else { + SVnodeMsg *pMsg = vnodeInitMsg(1); + if (pMsg == NULL) { + code = TSDB_CODE_OUT_OF_MEMORY; + } else { + if (vnodeAppendMsg(pMsg, pRpcMsg) != 0) { + code = terrno; + } else { + if (taosWriteQitem(pQueue, pMsg) != 0) { + code = TSDB_CODE_OUT_OF_MEMORY; + } + } + } + } + + if (code != TSDB_CODE_SUCCESS) { + SRpcMsg rsp = {.handle = pRpcMsg->handle, .code = code}; + rpcSendResponse(&rsp); + rpcFreeCont(pRpcMsg->pCont); + } +} + +static SVnodeObj *dndAcquireVnodeFromMsg(SDnode *pDnode, SRpcMsg *pMsg) { + SMsgHead *pHead = (SMsgHead *)pMsg->pCont; + pHead->vgId = htonl(pHead->vgId); + + SVnodeObj *pVnode = dndAcquireVnode(pDnode, pHead->vgId); + if (pVnode == NULL) { + SRpcMsg rsp = {.handle = pMsg->handle, .code = terrno}; + rpcSendResponse(&rsp); + rpcFreeCont(pMsg->pCont); + } + + return pVnode; +} + +void dndProcessVnodeMgmtMsg(SDnode *pDnode, SRpcMsg *pMsg, SEpSet *pEpSet) { + SVnodesMgmt *pMgmt = &pDnode->vmgmt; + dndWriteRpcMsgToVnodeQueue(pMgmt->pMgmtQ, pMsg); +} + +void dndProcessVnodeWriteMsg(SDnode *pDnode, SRpcMsg *pMsg, SEpSet *pEpSet) { + SVnodeObj *pVnode = dndAcquireVnodeFromMsg(pDnode, pMsg); + if (pVnode != NULL) { + dndWriteRpcMsgToVnodeQueue(pVnode->pWriteQ, pMsg); + dndReleaseVnode(pDnode, pVnode); + } +} + +void dndProcessVnodeSyncMsg(SDnode *pDnode, SRpcMsg *pMsg, SEpSet *pEpSet) { + SVnodeObj *pVnode = dndAcquireVnodeFromMsg(pDnode, pMsg); + if (pVnode != NULL) { + dndWriteVnodeMsgToVnodeQueue(pVnode->pSyncQ, pMsg); + dndReleaseVnode(pDnode, pVnode); + } +} + +void dndProcessVnodeQueryMsg(SDnode *pDnode, SRpcMsg *pMsg, SEpSet *pEpSet) { + SVnodeObj *pVnode = dndAcquireVnodeFromMsg(pDnode, pMsg); + if (pVnode != NULL) { + dndWriteVnodeMsgToVnodeQueue(pVnode->pQueryQ, pMsg); + dndReleaseVnode(pDnode, pVnode); + } +} + +void dndProcessVnodeFetchMsg(SDnode *pDnode, SRpcMsg *pMsg, SEpSet *pEpSet) { + SVnodeObj *pVnode = dndAcquireVnodeFromMsg(pDnode, pMsg); + if (pVnode != NULL) { + dndWriteVnodeMsgToVnodeQueue(pVnode->pFetchQ, pMsg); + dndReleaseVnode(pDnode, pVnode); + } +} + +static int32_t dndPutMsgIntoVnodeApplyQueue(SDnode *pDnode, int32_t vgId, SVnodeMsg *pMsg) { + SVnodeObj *pVnode = dndAcquireVnode(pDnode, vgId); + if (pVnode == NULL) { + return -1; + } + + int32_t code = taosWriteQitem(pVnode->pApplyQ, pMsg); + dndReleaseVnode(pDnode, pVnode); + return code; +} + +static int32_t dndInitVnodeMgmtWorker(SDnode *pDnode) { + SVnodesMgmt *pMgmt = &pDnode->vmgmt; + SWorkerPool *pPool = &pMgmt->mgmtPool; + pPool->name = "vnode-mgmt"; + pPool->min = 1; + pPool->max = 1; + if (tWorkerInit(pPool) != 0) { + terrno = TSDB_CODE_VND_OUT_OF_MEMORY; + return -1; + } + + pMgmt->pMgmtQ = tWorkerAllocQueue(pPool, pDnode, (FProcessItem)dndProcessVnodeMgmtQueue); + if (pMgmt->pMgmtQ == NULL) { + terrno = TSDB_CODE_VND_OUT_OF_MEMORY; + return -1; + } + + return 0; +} + +static void dndCleanupVnodeMgmtWorker(SDnode *pDnode) { + SVnodesMgmt *pMgmt = &pDnode->vmgmt; + tWorkerFreeQueue(&pMgmt->mgmtPool, pMgmt->pMgmtQ); + tWorkerCleanup(&pMgmt->mgmtPool); + pMgmt->pMgmtQ = NULL; +} + +static int32_t dndAllocVnodeQueryQueue(SDnode *pDnode, SVnodeObj *pVnode) { + SVnodesMgmt *pMgmt = &pDnode->vmgmt; + pVnode->pQueryQ = tWorkerAllocQueue(&pMgmt->queryPool, pVnode, (FProcessItem)dndProcessVnodeQueryQueue); + if (pVnode->pQueryQ == NULL) { + terrno = TSDB_CODE_OUT_OF_MEMORY; + return -1; + } + + return 0; +} + +static void dndFreeVnodeQueryQueue(SDnode *pDnode, SVnodeObj *pVnode) { + SVnodesMgmt *pMgmt = &pDnode->vmgmt; + tWorkerFreeQueue(&pMgmt->queryPool, pVnode->pQueryQ); + pVnode->pQueryQ = NULL; +} + +static int32_t dndAllocVnodeFetchQueue(SDnode *pDnode, SVnodeObj *pVnode) { + SVnodesMgmt *pMgmt = &pDnode->vmgmt; + pVnode->pFetchQ = tWorkerAllocQueue(&pMgmt->fetchPool, pVnode, (FProcessItem)dndProcessVnodeFetchQueue); + if (pVnode->pFetchQ == NULL) { + terrno = TSDB_CODE_OUT_OF_MEMORY; + return -1; + } + return 0; +} + +static void dndFreeVnodeFetchQueue(SDnode *pDnode, SVnodeObj *pVnode) { + SVnodesMgmt *pMgmt = &pDnode->vmgmt; + tWorkerFreeQueue(&pMgmt->fetchPool, pVnode->pFetchQ); + pVnode->pFetchQ = NULL; +} + +static int32_t dndInitVnodeReadWorker(SDnode *pDnode) { + SVnodesMgmt *pMgmt = &pDnode->vmgmt; + + int32_t maxFetchThreads = 4; + float threadsForQuery = MAX(pDnode->opt.numOfCores * pDnode->opt.ratioOfQueryCores, 1); + + SWorkerPool *pPool = &pMgmt->queryPool; + pPool->name = "vnode-query"; + pPool->min = (int32_t)threadsForQuery; + pPool->max = pPool->min; + if (tWorkerInit(pPool) != 0) { + return TSDB_CODE_VND_OUT_OF_MEMORY; + } + + pPool = &pMgmt->fetchPool; + pPool->name = "vnode-fetch"; + pPool->min = MIN(maxFetchThreads, pDnode->opt.numOfCores); + pPool->max = pPool->min; + if (tWorkerInit(pPool) != 0) { + TSDB_CODE_VND_OUT_OF_MEMORY; + } + + return 0; +} + +static void dndCleanupVnodeReadWorker(SDnode *pDnode) { + SVnodesMgmt *pMgmt = &pDnode->vmgmt; + tWorkerCleanup(&pMgmt->fetchPool); + tWorkerCleanup(&pMgmt->queryPool); +} + +static int32_t dndAllocVnodeWriteQueue(SDnode *pDnode, SVnodeObj *pVnode) { + SVnodesMgmt *pMgmt = &pDnode->vmgmt; + pVnode->pWriteQ = tMWorkerAllocQueue(&pMgmt->writePool, pVnode, (FProcessItems)dndProcessVnodeWriteQueue); + if (pVnode->pWriteQ == NULL) { + terrno = TSDB_CODE_OUT_OF_MEMORY; + return -1; + } + + return 0; +} + +static void dndFreeVnodeWriteQueue(SDnode *pDnode, SVnodeObj *pVnode) { + SVnodesMgmt *pMgmt = &pDnode->vmgmt; + tMWorkerFreeQueue(&pMgmt->writePool, pVnode->pWriteQ); + pVnode->pWriteQ = NULL; +} + +static int32_t dndAllocVnodeApplyQueue(SDnode *pDnode, SVnodeObj *pVnode) { + SVnodesMgmt *pMgmt = &pDnode->vmgmt; + pVnode->pApplyQ = tMWorkerAllocQueue(&pMgmt->writePool, pVnode, (FProcessItems)dndProcessVnodeApplyQueue); + if (pVnode->pApplyQ == NULL) { + terrno = TSDB_CODE_OUT_OF_MEMORY; + return -1; + } + + return 0; +} + +static void dndFreeVnodeApplyQueue(SDnode *pDnode, SVnodeObj *pVnode) { + SVnodesMgmt *pMgmt = &pDnode->vmgmt; + tMWorkerFreeQueue(&pMgmt->writePool, pVnode->pApplyQ); + pVnode->pApplyQ = NULL; +} + +static int32_t dndInitVnodeWriteWorker(SDnode *pDnode) { + SVnodesMgmt *pMgmt = &pDnode->vmgmt; + SMWorkerPool *pPool = &pMgmt->writePool; + pPool->name = "vnode-write"; + pPool->max = tsNumOfCores; + if (tMWorkerInit(pPool) != 0) { + terrno = TSDB_CODE_VND_OUT_OF_MEMORY; + return -1; + } + + return 0; +} + +static void dndCleanupVnodeWriteWorker(SDnode *pDnode) { + SVnodesMgmt *pMgmt = &pDnode->vmgmt; + tMWorkerCleanup(&pMgmt->writePool); +} + +static int32_t dndAllocVnodeSyncQueue(SDnode *pDnode, SVnodeObj *pVnode) { + SVnodesMgmt *pMgmt = &pDnode->vmgmt; + pVnode->pSyncQ = tMWorkerAllocQueue(&pMgmt->writePool, pVnode, (FProcessItems)dndProcessVnodeSyncQueue); + if (pVnode->pSyncQ == NULL) { + terrno = TSDB_CODE_OUT_OF_MEMORY; + return -1; + } + + return 0; +} + +static void dndFreeVnodeSyncQueue(SDnode *pDnode, SVnodeObj *pVnode) { + SVnodesMgmt *pMgmt = &pDnode->vmgmt; + tMWorkerFreeQueue(&pMgmt->writePool, pVnode->pSyncQ); + pVnode->pSyncQ = NULL; +} + +static int32_t dndInitVnodeSyncWorker(SDnode *pDnode) { + int32_t maxThreads = tsNumOfCores / 2; + if (maxThreads < 1) maxThreads = 1; + + SVnodesMgmt *pMgmt = &pDnode->vmgmt; + SMWorkerPool *pPool = &pMgmt->writePool; + pPool->name = "vnode-sync"; + pPool->max = maxThreads; + if (tMWorkerInit(pPool) != 0) { + terrno = TSDB_CODE_VND_OUT_OF_MEMORY; + return -1; + } + + return 0; +} + +static void dndCleanupVnodeSyncWorker(SDnode *pDnode) { + SVnodesMgmt *pMgmt = &pDnode->vmgmt; + tMWorkerCleanup(&pMgmt->syncPool); +} + +int32_t dndInitVnodes(SDnode *pDnode) { + dInfo("dnode-vnodes start to init"); + + if (dndInitVnodeReadWorker(pDnode) != 0) { + dError("failed to init vnodes read worker since %s", terrstr()); + return -1; + } + + if (dndInitVnodeWriteWorker(pDnode) != 0) { + dError("failed to init vnodes write worker since %s", terrstr()); + return -1; + } + + if (dndInitVnodeSyncWorker(pDnode) != 0) { + dError("failed to init vnodes sync worker since %s", terrstr()); + return -1; + } + + if (dndInitVnodeMgmtWorker(pDnode) != 0) { + dError("failed to init vnodes mgmt worker since %s", terrstr()); + return -1; + } + + if (dndOpenVnodes(pDnode) != 0) { + dError("failed to open vnodes since %s", terrstr()); + return -1; + } + + dInfo("dnode-vnodes is initialized"); + return 0; +} + +void dndCleanupVnodes(SDnode *pDnode) { + dInfo("dnode-vnodes start to clean up"); + dndCloseVnodes(pDnode); + dndCleanupVnodeReadWorker(pDnode); + dndCleanupVnodeWriteWorker(pDnode); + dndCleanupVnodeSyncWorker(pDnode); + dndCleanupVnodeMgmtWorker(pDnode); + dInfo("dnode-vnodes is cleaned up"); +} + +void dndGetVnodeLoads(SDnode *pDnode, SVnodeLoads *pLoads) { + SVnodesMgmt *pMgmt = &pDnode->vmgmt; + + taosRLockLatch(&pMgmt->latch); + pLoads->num = taosHashGetSize(pMgmt->hash); + + int32_t v = 0; + void *pIter = taosHashIterate(pMgmt->hash, NULL); + while (pIter) { + SVnodeObj **ppVnode = pIter; + if (ppVnode == NULL || *ppVnode == NULL) continue; + + SVnodeObj *pVnode = *ppVnode; + SVnodeLoad *pLoad = &pLoads->data[v++]; + + vnodeGetLoad(pVnode->pImpl, pLoad); + pLoad->vgId = htonl(pLoad->vgId); + pLoad->totalStorage = htobe64(pLoad->totalStorage); + pLoad->compStorage = htobe64(pLoad->compStorage); + pLoad->pointsWritten = htobe64(pLoad->pointsWritten); + pLoad->tablesNum = htobe64(pLoad->tablesNum); + + pIter = taosHashIterate(pMgmt->hash, pIter); + } + + taosRUnLockLatch(&pMgmt->latch); +} diff --git a/source/dnode/mgmt/impl/src/dnodeMnode.c b/source/dnode/mgmt/impl/src/dnodeMnode.c deleted file mode 100644 index 06b28aeea9..0000000000 --- a/source/dnode/mgmt/impl/src/dnodeMnode.c +++ /dev/null @@ -1,608 +0,0 @@ -/* - * Copyright (c) 2019 TAOS Data, Inc. - * - * This program is free software: you can use, redistribute, and/or modify - * it under the terms of the GNU Affero General Public License, version 3 - * or later ("AGPL"), as published by the Free Software Foundation. - * - * This program is distributed in the hope that it will be useful, but WITHOUT - * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or - * FITNESS FOR A PARTICULAR PURPOSE. - * - * You should have received a copy of the GNU Affero General Public License - * along with this program. If not, see . - */ - -#define _DEFAULT_SOURCE -#include "dnodeMnode.h" -#include "cJSON.h" -#include "dnodeDnode.h" -#include "dnodeTransport.h" -#include "mnode.h" -#include "tlockfree.h" -#include "tqueue.h" -#include "tstep.h" -#include "tworker.h" - -static struct { - int32_t refCount; - int8_t deployed; - int8_t dropped; - SWorkerPool mgmtPool; - SWorkerPool readPool; - SWorkerPool writePool; - SWorkerPool syncPool; - taos_queue pReadQ; - taos_queue pWriteQ; - taos_queue pApplyQ; - taos_queue pSyncQ; - taos_queue pMgmtQ; - SSteps *pSteps; - SMnode *pMnode; - SRWLatch latch; -} tsMnode = {0}; - -static int32_t dnodeAllocMnodeReadQueue(); -static void dnodeFreeMnodeReadQueue(); -static int32_t dnodeAllocMnodeWriteQueue(); -static void dnodeFreeMnodeWriteQueue(); -static int32_t dnodeAllocMnodeApplyQueue(); -static void dnodeFreeMnodeApplyQueue(); -static int32_t dnodeAllocMnodeSyncQueue(); -static void dnodeFreeMnodeSyncQueue(); - -static SMnode *dnodeAcquireMnode() { - SMnode *pMnode = NULL; - taosRLockLatch(&tsMnode.latch); - - if (tsMnode.deployed) { - atomic_add_fetch_32(&tsMnode.refCount, 1); - pMnode = tsMnode.pMnode; - } - - taosRUnLockLatch(&tsMnode.latch); - return pMnode; -} - -static void dnodeReleaseMnode(SMnode *pMnode) { - taosRLockLatch(&tsMnode.latch); - atomic_sub_fetch_32(&tsMnode.refCount, 1); - taosRUnLockLatch(&tsMnode.latch); -} - -static int32_t dnodeReadMnodeFile() { - int32_t code = TSDB_CODE_DND_READ_MNODE_FILE_ERROR; - int32_t len = 0; - int32_t maxLen = 300; - char *content = calloc(1, maxLen + 1); - cJSON *root = NULL; - FILE *fp = NULL; - char file[PATH_MAX + 20] = {0}; - - snprintf(file, sizeof(file), "%s/mnode.json", tsDnodeDir); - fp = fopen(file, "r"); - if (!fp) { - dDebug("file %s not exist", file); - code = 0; - goto PRASE_MNODE_OVER; - } - - len = (int32_t)fread(content, 1, maxLen, fp); - if (len <= 0) { - dError("failed to read %s since content is null", file); - goto PRASE_MNODE_OVER; - } - - content[len] = 0; - root = cJSON_Parse(content); - if (root == NULL) { - dError("failed to read %s since invalid json format", file); - goto PRASE_MNODE_OVER; - } - - cJSON *deployed = cJSON_GetObjectItem(root, "deployed"); - if (!deployed || deployed->type != cJSON_String) { - dError("failed to read %s since deployed not found", file); - goto PRASE_MNODE_OVER; - } - tsMnode.deployed = atoi(deployed->valuestring); - - cJSON *dropped = cJSON_GetObjectItem(root, "dropped"); - if (!dropped || dropped->type != cJSON_String) { - dError("failed to read %s since dropped not found", file); - goto PRASE_MNODE_OVER; - } - tsMnode.dropped = atoi(dropped->valuestring); - - code = 0; - dInfo("succcessed to read file %s", file); - -PRASE_MNODE_OVER: - if (content != NULL) free(content); - if (root != NULL) cJSON_Delete(root); - if (fp != NULL) fclose(fp); - - return code; -} - -static int32_t dnodeWriteMnodeFile() { - char file[PATH_MAX + 20] = {0}; - char realfile[PATH_MAX + 20] = {0}; - snprintf(file, sizeof(file), "%s/mnode.json.bak", tsDnodeDir); - snprintf(realfile, sizeof(realfile), "%s/mnode.json", tsDnodeDir); - - FILE *fp = fopen(file, "w"); - if (!fp) { - dError("failed to write %s since %s", file, strerror(errno)); - return TSDB_CODE_DND_WRITE_MNODE_FILE_ERROR; - } - - int32_t len = 0; - int32_t maxLen = 300; - char *content = calloc(1, maxLen + 1); - - len += snprintf(content + len, maxLen - len, "{\n"); - len += snprintf(content + len, maxLen - len, " \"deployed\": \"%d\",\n", tsMnode.deployed); - len += snprintf(content + len, maxLen - len, " \"dropped\": \"%d\"\n", tsMnode.dropped); - len += snprintf(content + len, maxLen - len, "}\n"); - - fwrite(content, 1, len, fp); - taosFsyncFile(fileno(fp)); - fclose(fp); - free(content); - - int32_t code = taosRenameFile(file, realfile); - if (code != 0) { - dError("failed to rename %s since %s", file, tstrerror(code)); - return TSDB_CODE_DND_WRITE_MNODE_FILE_ERROR; - } - - dInfo("successed to write %s", realfile); - return 0; -} - -static int32_t dnodeStartMnode() { - int32_t code = dnodeAllocMnodeReadQueue(); - if (code != 0) { - return code; - } - - code = dnodeAllocMnodeWriteQueue(); - if (code != 0) { - return code; - } - - code = dnodeAllocMnodeApplyQueue(); - if (code != 0) { - return code; - } - - code = dnodeAllocMnodeSyncQueue(); - if (code != 0) { - return code; - } - - taosWLockLatch(&tsMnode.latch); - tsMnode.deployed = 1; - taosWUnLockLatch(&tsMnode.latch); - - return mnodeStart(NULL); -} - -static void dnodeStopMnode() { - taosWLockLatch(&tsMnode.latch); - tsMnode.deployed = 0; - taosWUnLockLatch(&tsMnode.latch); - - dnodeReleaseMnode(); - - while (tsMnode.refCount > 0) taosMsleep(10); - while (!taosQueueEmpty(tsMnode.pReadQ)) taosMsleep(10); - while (!taosQueueEmpty(tsMnode.pApplyQ)) taosMsleep(10); - while (!taosQueueEmpty(tsMnode.pWriteQ)) taosMsleep(10); - while (!taosQueueEmpty(tsMnode.pSyncQ)) taosMsleep(10); - - dnodeFreeMnodeReadQueue(); - dnodeFreeMnodeWriteQueue(); - dnodeFreeMnodeApplyQueue(); - dnodeFreeMnodeSyncQueue(); -} - -static int32_t dnodeUnDeployMnode() { - tsMnode.dropped = 1; - int32_t code = dnodeWriteMnodeFile(); - if (code != 0) { - tsMnode.dropped = 0; - dError("failed to undeploy mnode since %s", tstrerror(code)); - return code; - } - - dnodeStopMnode(); - mnodeUnDeploy(); - dnodeWriteMnodeFile(); - - return code; -} - -static int32_t dnodeDeployMnode(SMnodeCfg *pCfg) { - int32_t code = mnodeDeploy(pCfg); - if (code != 0) { - dError("failed to deploy mnode since %s", tstrerror(code)); - return code; - } - - code = dnodeStartMnode(); - if (code != 0) { - dnodeUnDeployMnode(); - dError("failed to deploy mnode since %s", tstrerror(code)); - return code; - } - - code = dnodeWriteMnodeFile(); - if (code != 0) { - dnodeUnDeployMnode(); - dError("failed to deploy mnode since %s", tstrerror(code)); - return code; - } - - dInfo("deploy mnode success"); - return code; -} - -static int32_t dnodeAlterMnode(SMnodeCfg *pCfg) { - int32_t code = dnodeAcquireMnode(); - if (code == 0) { - code = mnodeAlter(pCfg); - dnodeReleaseMnode(); - } - return code; -} - -static SCreateMnodeMsg *dnodeParseCreateMnodeMsg(SRpcMsg *pRpcMsg) { - SCreateMnodeMsg *pMsg = pRpcMsg->pCont; - pMsg->dnodeId = htonl(pMsg->dnodeId); - for (int32_t i = 0; i < pMsg->replica; ++i) { - pMsg->replicas[i].port = htons(pMsg->replicas[i].port); - } - return pMsg; -} - -static int32_t dnodeProcessCreateMnodeReq(SRpcMsg *pRpcMsg) { - SAlterMnodeMsg *pMsg = (SAlterMnodeMsg *)dnodeParseCreateMnodeMsg(pRpcMsg->pCont); - - if (pMsg->dnodeId != dnodeGetDnodeId()) { - return TSDB_CODE_DND_MNODE_ID_NOT_MATCH_DNODE; - } else { - SMnodeCfg cfg = {0}; - cfg.replica = pMsg->replica; - memcpy(cfg.replicas, pMsg->replicas, sizeof(SReplica) * sizeof(TSDB_MAX_REPLICA)); - return dnodeDeployMnode(&cfg); - } -} - -static int32_t dnodeProcessAlterMnodeReq(SRpcMsg *pRpcMsg) { - SAlterMnodeMsg *pMsg = (SAlterMnodeMsg *)dnodeParseCreateMnodeMsg(pRpcMsg->pCont); - if (pMsg->dnodeId != dnodeGetDnodeId()) { - return TSDB_CODE_DND_MNODE_ID_NOT_MATCH_DNODE; - } else { - SMnodeCfg cfg = {0}; - cfg.replica = pMsg->replica; - memcpy(cfg.replicas, pMsg->replicas, sizeof(SReplica) * sizeof(TSDB_MAX_REPLICA)); - return dnodeAlterMnode(&cfg); - } -} - -static int32_t dnodeProcessDropMnodeReq(SRpcMsg *pMsg) { - SAlterMnodeMsg *pCfg = pMsg->pCont; - pCfg->dnodeId = htonl(pCfg->dnodeId); - - if (pCfg->dnodeId != dnodeGetDnodeId()) { - return TSDB_CODE_DND_MNODE_ID_NOT_MATCH_DNODE; - } else { - return dnodeUnDeployMnode(); - } -} - -static void dnodeProcessMnodeMgmtQueue(void *unused, SRpcMsg *pMsg) { - int32_t code = 0; - - switch (pMsg->msgType) { - case TSDB_MSG_TYPE_CREATE_MNODE_IN: - code = dnodeProcessCreateMnodeReq(pMsg); - break; - case TSDB_MSG_TYPE_ALTER_MNODE_IN: - code = dnodeProcessAlterMnodeReq(pMsg); - break; - case TSDB_MSG_TYPE_DROP_MNODE_IN: - code = dnodeProcessDropMnodeReq(pMsg); - break; - default: - code = TSDB_CODE_DND_MSG_NOT_PROCESSED; - break; - } - - SRpcMsg rsp = {.code = code, .handle = pMsg->handle}; - rpcSendResponse(&rsp); - rpcFreeCont(pMsg->pCont); - taosFreeQitem(pMsg); -} - -static void dnodeProcessMnodeReadQueue(void *unused, SMnodeMsg *pMsg) { mnodeProcessMsg(pMsg, MN_MSG_TYPE_READ); } - -static void dnodeProcessMnodeWriteQueue(void *unused, SMnodeMsg *pMsg) { mnodeProcessMsg(pMsg, MN_MSG_TYPE_WRITE); } - -static void dnodeProcessMnodeApplyQueue(void *unused, SMnodeMsg *pMsg) { mnodeProcessMsg(pMsg, MN_MSG_TYPE_APPLY); } - -static void dnodeProcessMnodeSyncQueue(void *unused, SMnodeMsg *pMsg) { mnodeProcessMsg(pMsg, MN_MSG_TYPE_SYNC); } - -static int32_t dnodeWriteMnodeMsgToQueue(taos_queue pQueue, SRpcMsg *pRpcMsg) { - int32_t code = 0; - SMnodeMsg *pMsg = NULL; - - if (pQueue == NULL) { - code = TSDB_CODE_DND_MSG_NOT_PROCESSED; - } else { - pMsg = mnodeInitMsg(pRpcMsg); - if (pMsg == NULL) { - code = terrno; - } - } - - if (code == 0) { - code = taosWriteQitem(pQueue, pMsg); - } - - if (code != TSDB_CODE_SUCCESS) { - SRpcMsg rsp = {.handle = pRpcMsg->handle, .code = code}; - rpcSendResponse(&rsp); - rpcFreeCont(pRpcMsg->pCont); - } -} - -void dnodeProcessMnodeMgmtMsg(SRpcMsg *pMsg, SEpSet *pEpSet) { dnodeWriteMnodeMsgToQueue(tsMnode.pMgmtQ, pMsg); } - -void dnodeProcessMnodeWriteMsg(SRpcMsg *pMsg, SEpSet *pEpSet) { - if (dnodeAcquireMnode() == 0) { - dnodeWriteMnodeMsgToQueue(tsMnode.pWriteQ, pMsg); - dnodeReleaseMnode(); - } else { - dnodeSendRedirectMsg(NULL, pMsg, 0); - } -} - -void dnodeProcessMnodeSyncMsg(SRpcMsg *pMsg, SEpSet *pEpSet) { - int32_t code = dnodeAcquireMnode(); - if (code == 0) { - dnodeWriteMnodeMsgToQueue(tsMnode.pSyncQ, pMsg); - dnodeReleaseMnode(); - } else { - SRpcMsg rsp = {.handle = pMsg->handle, .code = code}; - rpcSendResponse(&rsp); - rpcFreeCont(pMsg->pCont); - } -} - -void dnodeProcessMnodeReadMsg(SRpcMsg *pMsg, SEpSet *pEpSet) { - if (dnodeAcquireMnode() == 0) { - dnodeWriteMnodeMsgToQueue(tsMnode.pReadQ, pMsg); - dnodeReleaseMnode(); - } else { - dnodeSendRedirectMsg(NULL, pMsg, 0); - } -} - -static int32_t dnodePutMsgIntoMnodeApplyQueue(SMnodeMsg *pMsg) { - int32_t code = dnodeAcquireMnode(); - if (code != 0) return code; - - code = taosWriteQitem(tsMnode.pApplyQ, pMsg); - dnodeReleaseMnode(); - return code; -} - -static int32_t dnodeAllocMnodeMgmtQueue() { - tsMnode.pMgmtQ = tWorkerAllocQueue(&tsMnode.mgmtPool, NULL, (FProcessItem)dnodeProcessMnodeMgmtQueue); - if (tsMnode.pMgmtQ == NULL) { - return TSDB_CODE_DND_OUT_OF_MEMORY; - } - return 0; -} - -static void dnodeFreeMnodeMgmtQueue() { - tWorkerFreeQueue(&tsMnode.mgmtPool, tsMnode.pMgmtQ); - tsMnode.pMgmtQ = NULL; -} - -static int32_t dnodeInitMnodeMgmtWorker() { - SWorkerPool *pPool = &tsMnode.mgmtPool; - pPool->name = "mnode-mgmt"; - pPool->min = 1; - pPool->max = 1; - return tWorkerInit(pPool); -} - -static void dnodeCleanupMnodeMgmtWorker() { tWorkerCleanup(&tsMnode.mgmtPool); } - -static int32_t dnodeAllocMnodeReadQueue() { - tsMnode.pReadQ = tWorkerAllocQueue(&tsMnode.readPool, NULL, (FProcessItem)dnodeProcessMnodeReadQueue); - if (tsMnode.pReadQ == NULL) { - return TSDB_CODE_DND_OUT_OF_MEMORY; - } - return 0; -} - -static void dnodeFreeMnodeReadQueue() { - tWorkerFreeQueue(&tsMnode.readPool, tsMnode.pReadQ); - tsMnode.pReadQ = NULL; -} - -static int32_t dnodeInitMnodeReadWorker() { - SWorkerPool *pPool = &tsMnode.readPool; - pPool->name = "mnode-read"; - pPool->min = 0; - pPool->max = 1; - return tWorkerInit(pPool); -} - -static void dnodeCleanupMnodeReadWorker() { tWorkerCleanup(&tsMnode.readPool); } - -static int32_t dnodeAllocMnodeWriteQueue() { - tsMnode.pWriteQ = tWorkerAllocQueue(&tsMnode.writePool, NULL, (FProcessItem)dnodeProcessMnodeWriteQueue); - if (tsMnode.pWriteQ == NULL) { - return TSDB_CODE_DND_OUT_OF_MEMORY; - } - return 0; -} - -static void dnodeFreeMnodeWriteQueue() { - tWorkerFreeQueue(&tsMnode.writePool, tsMnode.pWriteQ); - tsMnode.pWriteQ = NULL; -} - -static int32_t dnodeAllocMnodeApplyQueue() { - tsMnode.pApplyQ = tWorkerAllocQueue(&tsMnode.writePool, NULL, (FProcessItem)dnodeProcessMnodeApplyQueue); - if (tsMnode.pApplyQ == NULL) { - return TSDB_CODE_DND_OUT_OF_MEMORY; - } - return 0; -} - -static void dnodeFreeMnodeApplyQueue() { - tWorkerFreeQueue(&tsMnode.writePool, tsMnode.pApplyQ); - tsMnode.pApplyQ = NULL; -} - -static int32_t dnodeInitMnodeWriteWorker() { - SWorkerPool *pPool = &tsMnode.writePool; - pPool->name = "mnode-write"; - pPool->min = 0; - pPool->max = 1; - return tWorkerInit(pPool); -} - -static void dnodeCleanupMnodeWriteWorker() { tWorkerCleanup(&tsMnode.writePool); } - -static int32_t dnodeAllocMnodeSyncQueue() { - tsMnode.pSyncQ = tWorkerAllocQueue(&tsMnode.syncPool, NULL, (FProcessItem)dnodeProcessMnodeSyncQueue); - if (tsMnode.pSyncQ == NULL) { - return TSDB_CODE_DND_OUT_OF_MEMORY; - } - return 0; -} - -static void dnodeFreeMnodeSyncQueue() { - tWorkerFreeQueue(&tsMnode.syncPool, tsMnode.pSyncQ); - tsMnode.pSyncQ = NULL; -} - -static int32_t dnodeInitMnodeSyncWorker() { - SWorkerPool *pPool = &tsMnode.syncPool; - pPool->name = "mnode-sync"; - pPool->min = 0; - pPool->max = 1; - return tWorkerInit(pPool); -} - -static void dnodeCleanupMnodeSyncWorker() { tWorkerCleanup(&tsMnode.syncPool); } - -static int32_t dnodeInitMnodeModule() { - taosInitRWLatch(&tsMnode.latch); - - SMnodeOptions para; - para.dnodeId = dnodeGetDnodeId(); - para.clusterId = dnodeGetClusterId(); - para.sendMsgToDnodeFp = dnodeSendMsgToDnode; - para.sendMsgToMnodeFp = dnodeSendMsgToMnode; - para.sendRedirectMsgFp = dnodeSendRedirectMsg; - - tsMnode.pMnode = mnodeCreate(para); - if (tsMnode.pMnode != NULL) { - return -1; - } - return 0; -} - -static void dnodeCleanupMnodeModule() { mnodeDrop(NULL); } - -static bool dnodeNeedDeployMnode() { - if (dnodeGetDnodeId() > 0) return false; - if (dnodeGetClusterId() > 0) return false; - if (strcmp(tsFirst, tsLocalEp) != 0) return false; - return true; -} - -static int32_t dnodeOpenMnode() { - int32_t code = dnodeReadMnodeFile(); - if (code != 0) { - dError("failed to read open mnode since %s", tstrerror(code)); - return code; - } - - if (tsMnode.dropped) { - dInfo("mnode already dropped, undeploy it"); - return dnodeUnDeployMnode(); - } - - if (!tsMnode.deployed) { - bool needDeploy = dnodeNeedDeployMnode(); - if (!needDeploy) return 0; - - dInfo("start to deploy mnode"); - SMnodeCfg cfg = {.replica = 1}; - cfg.replicas[0].port = tsServerPort; - tstrncpy(cfg.replicas[0].fqdn, tsLocalFqdn, TSDB_FQDN_LEN); - code = dnodeDeployMnode(&cfg); - } else { - dInfo("start to open mnode"); - return dnodeStartMnode(); - } -} - -static void dnodeCloseMnode() { - if (dnodeAcquireMnode() == 0) { - dnodeStopMnode(); - } -} - -int32_t dnodeInitMnode() { - dInfo("dnode-mnode start to init"); - - SSteps *pSteps = taosStepInit(6, dnodeReportStartup); - taosStepAdd(pSteps, "dnode-mnode-env", dnodeInitMnodeModule, dnodeCleanupMnodeModule); - taosStepAdd(pSteps, "dnode-mnode-mgmt", dnodeInitMnodeMgmtWorker, dnodeCleanupMnodeMgmtWorker); - taosStepAdd(pSteps, "dnode-mnode-read", dnodeInitMnodeReadWorker, dnodeCleanupMnodeReadWorker); - taosStepAdd(pSteps, "dnode-mnode-write", dnodeInitMnodeWriteWorker, dnodeCleanupMnodeWriteWorker); - taosStepAdd(pSteps, "dnode-mnode-sync", dnodeInitMnodeSyncWorker, dnodeCleanupMnodeSyncWorker); - taosStepAdd(pSteps, "dnode-mnode", dnodeOpenMnode, dnodeCloseMnode); - - tsMnode.pSteps = pSteps; - int32_t code = taosStepExec(pSteps); - - if (code != 0) { - dError("dnode-mnode init failed since %s", tstrerror(code)); - } else { - dInfo("dnode-mnode is initialized"); - } -} - -void dnodeCleanupMnode() { - if (tsMnode.pSteps == NULL) { - dInfo("dnode-mnode start to clean up"); - taosStepCleanup(tsMnode.pSteps); - tsMnode.pSteps = NULL; - dInfo("dnode-mnode is cleaned up"); - } -} - -int32_t dnodeGetUserAuthFromMnode(char *user, char *spi, char *encrypt, char *secret, char *ckey) { - SMnode *pMnode = dnodeAcquireMnode(); - if (pMnode == NULL) { - dTrace("failed to get user auth since mnode not deployed"); - terrno = TSDB_CODE_DND_MNODE_NOT_DEPLOYED; - return -1; - } - - int32_t code = mnodeRetriveAuth(pMnode, user, spi, encrypt, secret, ckey); - dnodeReleaseMnode(pMnode); - return code; -} \ No newline at end of file diff --git a/source/dnode/mgmt/impl/src/dnodeTransport.c b/source/dnode/mgmt/impl/src/dnodeTransport.c deleted file mode 100644 index c1e8955625..0000000000 --- a/source/dnode/mgmt/impl/src/dnodeTransport.c +++ /dev/null @@ -1,395 +0,0 @@ -/* - * 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 . - */ - -/* this file is mainly responsible for the communication between DNODEs. Each - * dnode works as both server and client. Dnode may send status, grant, config - * messages to mnode, mnode may send create/alter/drop table/vnode messages - * to dnode. All theses messages are handled from here - */ - -#define _DEFAULT_SOURCE -#include "dnodeTransport.h" -#include "dnodeDnode.h" -#include "dnodeMnode.h" -#include "dnodeVnodes.h" - -static struct { - void *peerRpc; - void *shellRpc; - void *clientRpc; - MsgFp msgFp[TSDB_MSG_TYPE_MAX]; -} tsTrans; - -static void dnodeInitMsgFp() { - // msg from client to dnode - tsTrans.msgFp[TSDB_MSG_TYPE_SUBMIT] = dnodeProcessVnodeWriteMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_QUERY] = dnodeProcessVnodeQueryMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_FETCH] = dnodeProcessVnodeFetchMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_CREATE_TABLE] = dnodeProcessVnodeWriteMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_DROP_TABLE] = dnodeProcessVnodeWriteMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_ALTER_TABLE] = dnodeProcessVnodeWriteMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_UPDATE_TAG_VAL] = dnodeProcessVnodeWriteMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_TABLE_META] = dnodeProcessVnodeQueryMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_TABLES_META] = dnodeProcessVnodeQueryMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_MQ_QUERY] = dnodeProcessVnodeQueryMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_MQ_CONSUME] = dnodeProcessVnodeQueryMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_MQ_CONNECT] = dnodeProcessVnodeWriteMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_MQ_DISCONNECT] = dnodeProcessVnodeWriteMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_MQ_ACK] = dnodeProcessVnodeWriteMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_MQ_RESET] = dnodeProcessVnodeWriteMsg; - - // msg from client to mnode - tsTrans.msgFp[TSDB_MSG_TYPE_CONNECT] = dnodeProcessMnodeReadMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_CREATE_ACCT] = dnodeProcessMnodeWriteMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_ALTER_ACCT] = dnodeProcessMnodeWriteMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_DROP_ACCT] = dnodeProcessMnodeWriteMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_CREATE_USER] = dnodeProcessMnodeWriteMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_ALTER_USER] = dnodeProcessMnodeWriteMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_DROP_USER] = dnodeProcessMnodeWriteMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_CREATE_DNODE] = dnodeProcessMnodeWriteMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_CONFIG_DNODE] = dnodeProcessMnodeWriteMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_DROP_DNODE] = dnodeProcessMnodeWriteMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_CREATE_DB] = dnodeProcessMnodeWriteMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_DROP_DB] = dnodeProcessMnodeWriteMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_USE_DB] = dnodeProcessMnodeWriteMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_ALTER_DB] = dnodeProcessMnodeWriteMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_SYNC_DB] = dnodeProcessMnodeWriteMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_CREATE_TOPIC] = dnodeProcessMnodeWriteMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_DROP_TOPIC] = dnodeProcessMnodeWriteMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_ALTER_TOPIC] = dnodeProcessMnodeWriteMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_CREATE_FUNCTION] = dnodeProcessMnodeWriteMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_ALTER_FUNCTION] = dnodeProcessMnodeWriteMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_DROP_FUNCTION] = dnodeProcessMnodeWriteMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_CREATE_STABLE] = dnodeProcessMnodeWriteMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_ALTER_STABLE] = dnodeProcessMnodeWriteMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_DROP_STABLE] = dnodeProcessMnodeWriteMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_STABLE_VGROUP] = dnodeProcessMnodeReadMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_KILL_QUERY] = dnodeProcessMnodeWriteMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_KILL_CONN] = dnodeProcessMnodeWriteMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_HEARTBEAT] = dnodeProcessMnodeReadMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_SHOW] = dnodeProcessMnodeReadMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_SHOW_RETRIEVE] = dnodeProcessMnodeReadMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_SHOW_RETRIEVE_FUNC] = dnodeProcessMnodeReadMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_COMPACT_VNODE] = dnodeProcessMnodeWriteMsg; - - // message from client to dnode - tsTrans.msgFp[TSDB_MSG_TYPE_NETWORK_TEST] = dnodeProcessDnodeMsg; - - // message from mnode to vnode - tsTrans.msgFp[TSDB_MSG_TYPE_CREATE_STABLE_IN] = dnodeProcessVnodeWriteMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_CREATE_STABLE_IN_RSP] = dnodeProcessMnodeWriteMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_ALTER_STABLE_IN] = dnodeProcessVnodeWriteMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_ALTER_STABLE_IN_RSP] = dnodeProcessMnodeWriteMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_DROP_STABLE_IN] = dnodeProcessVnodeWriteMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_DROP_STABLE_IN] = dnodeProcessMnodeWriteMsg; - - // message from mnode to dnode - tsTrans.msgFp[TSDB_MSG_TYPE_CREATE_VNODE_IN] = dnodeProcessVnodeMgmtMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_CREATE_VNODE_IN_RSP] = dnodeProcessMnodeWriteMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_ALTER_VNODE_IN] = dnodeProcessVnodeMgmtMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_ALTER_VNODE_IN_RSP] = dnodeProcessMnodeWriteMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_DROP_VNODE_IN] = dnodeProcessVnodeMgmtMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_DROP_VNODE_IN_RSP] = dnodeProcessMnodeWriteMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_SYNC_VNODE_IN] = dnodeProcessVnodeMgmtMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_SYNC_VNODE_IN_RSP] = dnodeProcessMnodeWriteMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_AUTH_VNODE_IN] = dnodeProcessVnodeMgmtMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_AUTH_VNODE_IN_RSP] = dnodeProcessMnodeWriteMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_COMPACT_VNODE_IN] = dnodeProcessVnodeMgmtMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_COMPACT_VNODE_IN_RSP] = dnodeProcessMnodeWriteMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_CREATE_MNODE_IN] = dnodeProcessMnodeMgmtMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_CREATE_MNODE_IN_RSP] = dnodeProcessMnodeWriteMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_ALTER_MNODE_IN] = dnodeProcessMnodeMgmtMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_ALTER_MNODE_IN_RSP] = dnodeProcessMnodeWriteMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_DROP_MNODE_IN] = dnodeProcessMnodeMgmtMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_DROP_MNODE_IN_RSP] = dnodeProcessMnodeWriteMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_CONFIG_DNODE_IN] = dnodeProcessDnodeMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_CONFIG_DNODE_IN_RSP] = dnodeProcessMnodeWriteMsg; - - // message from dnode to mnode - tsTrans.msgFp[TSDB_MSG_TYPE_AUTH] = dnodeProcessMnodeReadMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_AUTH_RSP] = dnodeProcessDnodeMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_GRANT] = dnodeProcessMnodeWriteMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_GRANT_RSP] = dnodeProcessDnodeMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_STATUS] = dnodeProcessMnodeWriteMsg; - tsTrans.msgFp[TSDB_MSG_TYPE_STATUS_RSP] = dnodeProcessDnodeMsg; -} - -static void dnodeProcessPeerReq(SRpcMsg *pMsg, SEpSet *pEpSet) { - SRpcMsg rspMsg = {.handle = pMsg->handle}; - int32_t msgType = pMsg->msgType; - - if (msgType == TSDB_MSG_TYPE_NETWORK_TEST) { - dnodeProcessDnodeMsg(pMsg, pEpSet); - return; - } - - if (dnodeGetStat() != DN_STAT_RUNNING) { - rspMsg.code = TSDB_CODE_APP_NOT_READY; - rpcSendResponse(&rspMsg); - rpcFreeCont(pMsg->pCont); - dTrace("RPC %p, peer req:%s is ignored since dnode not running", pMsg->handle, taosMsg[msgType]); - return; - } - - if (pMsg->pCont == NULL) { - rspMsg.code = TSDB_CODE_DND_INVALID_MSG_LEN; - rpcSendResponse(&rspMsg); - return; - } - - MsgFp fp = tsTrans.msgFp[msgType]; - if (fp != NULL) { - dTrace("RPC %p, peer req:%s will be processed", pMsg->handle, taosMsg[msgType]); - (*fp)(pMsg, pEpSet); - } else { - dError("RPC %p, peer req:%s not processed", pMsg->handle, taosMsg[msgType]); - rspMsg.code = TSDB_CODE_DND_MSG_NOT_PROCESSED; - rpcSendResponse(&rspMsg); - rpcFreeCont(pMsg->pCont); - } -} - -static int32_t dnodeInitPeerServer() { - SRpcInit rpcInit; - memset(&rpcInit, 0, sizeof(rpcInit)); - rpcInit.localPort = tsDnodeDnodePort; - rpcInit.label = "DND-S"; - rpcInit.numOfThreads = 1; - rpcInit.cfp = dnodeProcessPeerReq; - rpcInit.sessions = TSDB_MAX_VNODES << 4; - rpcInit.connType = TAOS_CONN_SERVER; - rpcInit.idleTime = tsShellActivityTimer * 1000; - - tsTrans.peerRpc = rpcOpen(&rpcInit); - if (tsTrans.peerRpc == NULL) { - dError("failed to init peer rpc server"); - return -1; - } - - dInfo("dnode peer rpc server is initialized"); - return 0; -} - -static void dnodeCleanupPeerServer() { - if (tsTrans.peerRpc) { - rpcClose(tsTrans.peerRpc); - tsTrans.peerRpc = NULL; - dInfo("dnode peer server is closed"); - } -} - -static void dnodeProcessPeerRsp(SRpcMsg *pMsg, SEpSet *pEpSet) { - int32_t msgType = pMsg->msgType; - - if (dnodeGetStat() == DN_STAT_STOPPED) { - if (pMsg == NULL || pMsg->pCont == NULL) return; - dTrace("RPC %p, peer rsp:%s is ignored since dnode is stopping", pMsg->handle, taosMsg[msgType]); - rpcFreeCont(pMsg->pCont); - return; - } - - MsgFp fp = tsTrans.msgFp[msgType]; - if (fp != NULL) { - dTrace("RPC %p, peer rsp:%s will be processed, code:%s", pMsg->handle, taosMsg[msgType], tstrerror(pMsg->code)); - (*fp)(pMsg, pEpSet); - } else { - dDebug("RPC %p, peer rsp:%s not processed", pMsg->handle, taosMsg[msgType]); - } - - rpcFreeCont(pMsg->pCont); -} - -static int32_t dnodeInitClient() { - char secret[TSDB_KEY_LEN] = "secret"; - - SRpcInit rpcInit; - memset(&rpcInit, 0, sizeof(rpcInit)); - rpcInit.label = "DND-C"; - rpcInit.numOfThreads = 1; - rpcInit.cfp = dnodeProcessPeerRsp; - rpcInit.sessions = TSDB_MAX_VNODES << 4; - rpcInit.connType = TAOS_CONN_CLIENT; - rpcInit.idleTime = tsShellActivityTimer * 1000; - rpcInit.user = "t"; - rpcInit.ckey = "key"; - rpcInit.secret = secret; - - tsTrans.clientRpc = rpcOpen(&rpcInit); - if (tsTrans.clientRpc == NULL) { - dError("failed to init peer rpc client"); - return -1; - } - - dInfo("dnode peer rpc client is initialized"); - return 0; -} - -static void dnodeCleanupClient() { - if (tsTrans.clientRpc) { - rpcClose(tsTrans.clientRpc); - tsTrans.clientRpc = NULL; - dInfo("dnode peer rpc client is closed"); - } -} - -static void dnodeProcessShellReq(SRpcMsg *pMsg, SEpSet *pEpSet) { - SRpcMsg rspMsg = {.handle = pMsg->handle}; - int32_t msgType = pMsg->msgType; - - if (dnodeGetStat() == DN_STAT_STOPPED) { - dError("RPC %p, shell req:%s is ignored since dnode exiting", pMsg->handle, taosMsg[msgType]); - rspMsg.code = TSDB_CODE_DND_EXITING; - rpcSendResponse(&rspMsg); - rpcFreeCont(pMsg->pCont); - return; - } else if (dnodeGetStat() != DN_STAT_RUNNING) { - dError("RPC %p, shell req:%s is ignored since dnode not running", pMsg->handle, taosMsg[msgType]); - rspMsg.code = TSDB_CODE_APP_NOT_READY; - rpcSendResponse(&rspMsg); - rpcFreeCont(pMsg->pCont); - return; - } - - if (pMsg->pCont == NULL) { - rspMsg.code = TSDB_CODE_DND_INVALID_MSG_LEN; - rpcSendResponse(&rspMsg); - return; - } - - MsgFp fp = tsTrans.msgFp[msgType]; - if (fp != NULL) { - dTrace("RPC %p, shell req:%s will be processed", pMsg->handle, taosMsg[msgType]); - (*fp)(pMsg, pEpSet); - } else { - dError("RPC %p, shell req:%s is not processed", pMsg->handle, taosMsg[msgType]); - rspMsg.code = TSDB_CODE_DND_MSG_NOT_PROCESSED; - rpcSendResponse(&rspMsg); - rpcFreeCont(pMsg->pCont); - } -} - -static void dnodeSendMsgToMnodeRecv(SRpcMsg *rpcMsg, SRpcMsg *rpcRsp) { - SEpSet epSet = {0}; - dnodeGetMnodeEpSetForPeer(&epSet); - rpcSendRecv(tsTrans.clientRpc, &epSet, rpcMsg, rpcRsp); -} - -static int32_t dnodeRetrieveUserAuthInfo(char *user, char *spi, char *encrypt, char *secret, char *ckey) { - int32_t code = dnodeGetUserAuthFromMnode(user, spi, encrypt, secret, ckey); - if (code != TSDB_CODE_APP_NOT_READY) return code; - - SAuthMsg *pMsg = rpcMallocCont(sizeof(SAuthMsg)); - tstrncpy(pMsg->user, user, sizeof(pMsg->user)); - - dDebug("user:%s, send auth msg to mnodes", user); - SRpcMsg rpcMsg = {.pCont = pMsg, .contLen = sizeof(SAuthMsg), .msgType = TSDB_MSG_TYPE_AUTH}; - SRpcMsg rpcRsp = {0}; - dnodeSendMsgToMnodeRecv(&rpcMsg, &rpcRsp); - - if (rpcRsp.code != 0) { - dError("user:%s, auth msg received from mnodes, error:%s", user, tstrerror(rpcRsp.code)); - } else { - dDebug("user:%s, auth msg received from mnodes", user); - SAuthRsp *pRsp = rpcRsp.pCont; - memcpy(secret, pRsp->secret, TSDB_KEY_LEN); - memcpy(ckey, pRsp->ckey, TSDB_KEY_LEN); - *spi = pRsp->spi; - *encrypt = pRsp->encrypt; - } - - rpcFreeCont(rpcRsp.pCont); - return rpcRsp.code; -} - -static int32_t dnodeInitShellServer() { - dnodeInitMsgFp(); - - int32_t numOfThreads = (int32_t)((tsNumOfCores * tsNumOfThreadsPerCore) / 2.0); - if (numOfThreads < 1) { - numOfThreads = 1; - } - - SRpcInit rpcInit; - memset(&rpcInit, 0, sizeof(rpcInit)); - rpcInit.localPort = tsDnodeShellPort; - rpcInit.label = "SHELL"; - rpcInit.numOfThreads = numOfThreads; - rpcInit.cfp = dnodeProcessShellReq; - rpcInit.sessions = tsMaxShellConns; - rpcInit.connType = TAOS_CONN_SERVER; - rpcInit.idleTime = tsShellActivityTimer * 1000; - rpcInit.afp = dnodeRetrieveUserAuthInfo; - - tsTrans.shellRpc = rpcOpen(&rpcInit); - if (tsTrans.shellRpc == NULL) { - dError("failed to init shell rpc server"); - return -1; - } - -#if 1 - SRpcMsg rpcMsg = {0}; - rpcMsg.msgType = TSDB_MSG_TYPE_CREATE_USER; - rpcMsg.contLen = sizeof(SCreateUserMsg); - rpcMsg.pCont = rpcMallocCont(rpcMsg.contLen); - SCreateUserMsg *pMsg = (SCreateUserMsg*)rpcMsg.pCont; - strcpy(pMsg->user, "u1"); - strcpy(pMsg->pass, "up1"); - dnodeProcessShellReq(&rpcMsg, NULL); - -#endif - dInfo("dnode shell rpc server is initialized"); - return 0; -} - -static void dnodeCleanupShellServer() { - if (tsTrans.shellRpc) { - rpcClose(tsTrans.shellRpc); - tsTrans.shellRpc = NULL; - } -} - -int32_t dnodeInitTrans() { - if (dnodeInitClient() != 0) { - return -1; - } - - if (dnodeInitPeerServer() != 0) { - return -1; - } - - if (dnodeInitShellServer() != 0) { - return -1; - } - - return 0; -} - -void dnodeCleanupTrans() { - dnodeCleanupShellServer(); - dnodeCleanupPeerServer(); - dnodeCleanupClient(); -} - -void dnodeSendMsgToDnode(SDnode *pDnode, SEpSet *epSet, SRpcMsg *rpcMsg) { - #if 0 - rpcSendRequest(tsTrans.clientRpc, epSet, rpcMsg, NULL); - #endif - } - -void dnodeSendMsgToMnode(SDnode *pDnode, SRpcMsg *rpcMsg) { - SEpSet epSet = {0}; - dnodeGetMnodeEpSetForPeer(&epSet); - dnodeSendMsgToDnode(NULL, &epSet, rpcMsg); -} \ No newline at end of file diff --git a/source/dnode/mgmt/impl/src/dnodeVnodes.c b/source/dnode/mgmt/impl/src/dnodeVnodes.c deleted file mode 100644 index 7eaa82ba93..0000000000 --- a/source/dnode/mgmt/impl/src/dnodeVnodes.c +++ /dev/null @@ -1,1029 +0,0 @@ -/* - * Copyright (c) 2019 TAOS Data, Inc. - * - * This program is free software: you can use, redistribute, and/or modify - * it under the terms of the GNU Affero General Public License, version 3 - * or later ("AGPL"), as published by the Free Software Foundation. - * - * This program is distributed in the hope that it will be useful, but WITHOUT - * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or - * FITNESS FOR A PARTICULAR PURPOSE. - * - * You should have received a copy of the GNU Affero General Public License - * along with this program. If not, see . - */ - -#define _DEFAULT_SOURCE -#include "dnodeVnodes.h" -#include "dnodeTransport.h" -#include "cJSON.h" -#include "thash.h" -#include "tlockfree.h" -#include "tqueue.h" -#include "tstep.h" -#include "tthread.h" -#include "tworker.h" -#include "vnode.h" - -typedef struct { - int32_t vgId; - int32_t refCount; - int8_t dropped; - int8_t accessState; - SVnode *pImpl; - taos_queue pWriteQ; - taos_queue pSyncQ; - taos_queue pApplyQ; - taos_queue pQueryQ; - taos_queue pFetchQ; -} SVnodeObj; - -typedef struct { - pthread_t *threadId; - int32_t threadIndex; - int32_t failed; - int32_t opened; - int32_t vnodeNum; - SVnodeObj *pVnodes; -} SVThread; - -static struct { - SHashObj *hash; - SWorkerPool mgmtPool; - SWorkerPool queryPool; - SWorkerPool fetchPool; - SMWorkerPool syncPool; - SMWorkerPool writePool; - taos_queue pMgmtQ; - SSteps *pSteps; - int32_t openVnodes; - int32_t totalVnodes; - SRWLatch latch; -} tsVnodes; - -static int32_t dnodeAllocVnodeQueryQueue(SVnodeObj *pVnode); -static void dnodeFreeVnodeQueryQueue(SVnodeObj *pVnode); -static int32_t dnodeAllocVnodeFetchQueue(SVnodeObj *pVnode); -static void dnodeFreeVnodeFetchQueue(SVnodeObj *pVnode); -static int32_t dnodeAllocVnodeWriteQueue(SVnodeObj *pVnode); -static void dnodeFreeVnodeWriteQueue(SVnodeObj *pVnode); -static int32_t dnodeAllocVnodeApplyQueue(SVnodeObj *pVnode); -static void dnodeFreeVnodeApplyQueue(SVnodeObj *pVnode); -static int32_t dnodeAllocVnodeSyncQueue(SVnodeObj *pVnode); -static void dnodeFreeVnodeSyncQueue(SVnodeObj *pVnode); - -static SVnodeObj *dnodeAcquireVnode(int32_t vgId) { - SVnodeObj *pVnode = NULL; - int32_t refCount = 0; - - taosRLockLatch(&tsVnodes.latch); - taosHashGetClone(tsVnodes.hash, &vgId, sizeof(int32_t), (void *)&pVnode); - if (pVnode == NULL) { - terrno = TSDB_CODE_VND_INVALID_VGROUP_ID; - } else { - refCount = atomic_add_fetch_32(&pVnode->refCount, 1); - } - taosRUnLockLatch(&tsVnodes.latch); - - dTrace("vgId:%d, accquire vnode, refCount:%d", pVnode->vgId, refCount); - return pVnode; -} - -static void dnodeReleaseVnode(SVnodeObj *pVnode) { - int32_t refCount = atomic_sub_fetch_32(&pVnode->refCount, 1); - dTrace("vgId:%d, release vnode, refCount:%d", pVnode->vgId, refCount); -} - -static int32_t dnodeCreateVnodeWrapper(int32_t vgId, SVnode *pImpl) { - SVnodeObj *pVnode = calloc(1, sizeof(SVnodeObj)); - if (pVnode == NULL) { - return TSDB_CODE_DND_OUT_OF_MEMORY; - } - - pVnode->vgId = vgId; - pVnode->refCount = 0; - pVnode->dropped = 0; - pVnode->accessState = TSDB_VN_ALL_ACCCESS; - pVnode->pImpl = pImpl; - - int32_t code = dnodeAllocVnodeQueryQueue(pVnode); - if (code != 0) { - return code; - } - - code = dnodeAllocVnodeFetchQueue(pVnode); - if (code != 0) { - return code; - } - - code = dnodeAllocVnodeWriteQueue(pVnode); - if (code != 0) { - return code; - } - - code = dnodeAllocVnodeApplyQueue(pVnode); - if (code != 0) { - return code; - } - - code = dnodeAllocVnodeSyncQueue(pVnode); - if (code != 0) { - return code; - } - - taosWLockLatch(&tsVnodes.latch); - code = taosHashPut(tsVnodes.hash, &vgId, sizeof(int32_t), &pVnode, sizeof(SVnodeObj *)); - taosWUnLockLatch(&tsVnodes.latch); - - return code; -} - -static void dnodeDropVnodeWrapper(SVnodeObj *pVnode) { - taosWLockLatch(&tsVnodes.latch); - taosHashRemove(tsVnodes.hash, &pVnode->vgId, sizeof(int32_t)); - taosWUnLockLatch(&tsVnodes.latch); - - // wait all queue empty - dnodeReleaseVnode(pVnode); - while (pVnode->refCount > 0) taosMsleep(10); - while (!taosQueueEmpty(pVnode->pWriteQ)) taosMsleep(10); - while (!taosQueueEmpty(pVnode->pSyncQ)) taosMsleep(10); - while (!taosQueueEmpty(pVnode->pApplyQ)) taosMsleep(10); - while (!taosQueueEmpty(pVnode->pQueryQ)) taosMsleep(10); - while (!taosQueueEmpty(pVnode->pFetchQ)) taosMsleep(10); - - dnodeFreeVnodeQueryQueue(pVnode); - dnodeFreeVnodeFetchQueue(pVnode); - dnodeFreeVnodeWriteQueue(pVnode); - dnodeFreeVnodeApplyQueue(pVnode); - dnodeFreeVnodeSyncQueue(pVnode); -} - -static SVnodeObj **dnodeGetVnodesFromHash(int32_t *numOfVnodes) { - taosRLockLatch(&tsVnodes.latch); - - int32_t num = 0; - int32_t size = taosHashGetSize(tsVnodes.hash); - SVnodeObj **pVnodes = calloc(size, sizeof(SVnodeObj *)); - - void *pIter = taosHashIterate(tsVnodes.hash, NULL); - while (pIter) { - SVnodeObj **ppVnode = pIter; - SVnodeObj *pVnode = *ppVnode; - if (pVnode) { - num++; - if (num < size) { - int32_t refCount = atomic_add_fetch_32(&pVnode->refCount, 1); - dTrace("vgId:%d, accquire vnode, refCount:%d", pVnode->vgId, refCount); - pVnodes[num] = (*ppVnode); - } - } - pIter = taosHashIterate(tsVnodes.hash, pIter); - } - - taosRUnLockLatch(&tsVnodes.latch); - *numOfVnodes = num; - - return pVnodes; -} - -static int32_t dnodeGetVnodesFromFile(SVnodeObj **ppVnodes, int32_t *numOfVnodes) { - int32_t code = TSDB_CODE_DND_PARSE_VNODE_FILE_ERROR; - int32_t len = 0; - int32_t maxLen = 30000; - char *content = calloc(1, maxLen + 1); - cJSON *root = NULL; - FILE *fp = NULL; - char file[PATH_MAX + 20] = {0}; - SVnodeObj *pVnodes = NULL; - - snprintf(file, PATH_MAX + 20, "%s/vnodes.json", tsVnodeDir); - - fp = fopen(file, "r"); - if (!fp) { - dDebug("file %s not exist", file); - code = 0; - goto PRASE_VNODE_OVER; - } - - len = (int32_t)fread(content, 1, maxLen, fp); - if (len <= 0) { - dError("failed to read %s since content is null", file); - goto PRASE_VNODE_OVER; - } - - content[len] = 0; - root = cJSON_Parse(content); - if (root == NULL) { - dError("failed to read %s since invalid json format", file); - goto PRASE_VNODE_OVER; - } - - cJSON *vnodes = cJSON_GetObjectItem(root, "vnodes"); - if (!vnodes || vnodes->type != cJSON_Array) { - dError("failed to read %s since vnodes not found", file); - goto PRASE_VNODE_OVER; - } - - int32_t vnodesNum = cJSON_GetArraySize(vnodes); - if (vnodesNum <= 0) { - dError("failed to read %s since vnodes size:%d invalid", file, vnodesNum); - goto PRASE_VNODE_OVER; - } - - pVnodes = calloc(vnodesNum, sizeof(SVnodeObj)); - if (pVnodes == NULL) { - dError("failed to read %s since out of memory", file); - goto PRASE_VNODE_OVER; - } - - for (int32_t i = 0; i < vnodesNum; ++i) { - cJSON *vnode = cJSON_GetArrayItem(vnodes, i); - SVnodeObj *pVnode = &pVnodes[i]; - - cJSON *vgId = cJSON_GetObjectItem(vnode, "vgId"); - if (!vgId || vgId->type != cJSON_String) { - dError("failed to read %s since vgId not found", file); - goto PRASE_VNODE_OVER; - } - pVnode->vgId = atoi(vgId->valuestring); - - cJSON *dropped = cJSON_GetObjectItem(vnode, "dropped"); - if (!dropped || dropped->type != cJSON_String) { - dError("failed to read %s since dropped not found", file); - goto PRASE_VNODE_OVER; - } - pVnode->dropped = atoi(vnode->valuestring); - } - - code = 0; - dInfo("succcessed to read file %s", file); - -PRASE_VNODE_OVER: - if (content != NULL) free(content); - if (root != NULL) cJSON_Delete(root); - if (fp != NULL) fclose(fp); - - return code; -} - -static int32_t dnodeWriteVnodesToFile() { - char file[PATH_MAX + 20] = {0}; - char realfile[PATH_MAX + 20] = {0}; - snprintf(file, PATH_MAX + 20, "%s/vnodes.json.bak", tsVnodeDir); - snprintf(realfile, PATH_MAX + 20, "%s/vnodes.json", tsVnodeDir); - - FILE *fp = fopen(file, "w"); - if (!fp) { - dError("failed to write %s since %s", file, strerror(errno)); - return -1; - } - - int32_t len = 0; - int32_t maxLen = 30000; - char *content = calloc(1, maxLen + 1); - int32_t numOfVnodes = 0; - SVnodeObj **pVnodes = dnodeGetVnodesFromHash(&numOfVnodes); - - len += snprintf(content + len, maxLen - len, "{\n"); - len += snprintf(content + len, maxLen - len, " \"vnodes\": [{\n"); - for (int32_t i = 0; i < numOfVnodes; ++i) { - SVnodeObj *pVnode = pVnodes[i]; - len += snprintf(content + len, maxLen - len, " \"vgId\": \"%d\",\n", pVnode->vgId); - len += snprintf(content + len, maxLen - len, " \"dropped\": \"%d\"\n", pVnode->dropped); - if (i < numOfVnodes - 1) { - len += snprintf(content + len, maxLen - len, " },{\n"); - } else { - len += snprintf(content + len, maxLen - len, " }]\n"); - } - } - len += snprintf(content + len, maxLen - len, "}\n"); - - fwrite(content, 1, len, fp); - taosFsyncFile(fileno(fp)); - fclose(fp); - free(content); - terrno = 0; - - for (int32_t i = 0; i < numOfVnodes; ++i) { - SVnodeObj *pVnode = pVnodes[i]; - dnodeReleaseVnode(pVnode); - } - - if (pVnodes != NULL) { - free(pVnodes); - } - - dInfo("successed to write %s", file); - return taosRenameFile(file, realfile); -} - -static int32_t dnodeCreateVnode(int32_t vgId, SVnodeCfg *pCfg) { - int32_t code = 0; - - char path[PATH_MAX + 20] = {0}; - snprintf(path, sizeof(path),"%s/vnode%d", tsVnodeDir, vgId); - SVnode *pImpl = vnodeCreate(vgId, path, pCfg); - - if (pImpl == NULL) { - code = terrno; - return code; - } - - code = dnodeCreateVnodeWrapper(vgId, pImpl); - if (code != 0) { - vnodeDrop(pImpl); - return code; - } - - code = dnodeWriteVnodesToFile(); - if (code != 0) { - vnodeDrop(pImpl); - return code; - } - - return code; -} - -static int32_t dnodeDropVnode(SVnodeObj *pVnode) { - pVnode->dropped = 1; - - int32_t code = dnodeWriteVnodesToFile(); - if (code != 0) { - pVnode->dropped = 0; - return code; - } - - dnodeDropVnodeWrapper(pVnode); - vnodeDrop(pVnode->pImpl); - dnodeWriteVnodesToFile(); - return 0; -} - -static void *dnodeOpenVnodeFunc(void *param) { - SVThread *pThread = param; - - dDebug("thread:%d, start to open %d vnodes", pThread->threadIndex, pThread->vnodeNum); - setThreadName("open-vnodes"); - - for (int32_t v = 0; v < pThread->vnodeNum; ++v) { - SVnodeObj *pVnode = &pThread->pVnodes[v]; - - char stepDesc[TSDB_STEP_DESC_LEN] = {0}; - snprintf(stepDesc, TSDB_STEP_DESC_LEN, "vgId:%d, start to restore, %d of %d have been opened", pVnode->vgId, - tsVnodes.openVnodes, tsVnodes.totalVnodes); - dnodeReportStartup("open-vnodes", stepDesc); - - char path[PATH_MAX + 20] = {0}; - snprintf(path, sizeof(path),"%s/vnode%d", tsVnodeDir, pVnode->vgId); - SVnode *pImpl = vnodeOpen(path, NULL); - if (pImpl == NULL) { - dError("vgId:%d, failed to open vnode by thread:%d", pVnode->vgId, pThread->threadIndex); - pThread->failed++; - } else { - dnodeCreateVnodeWrapper(pVnode->vgId, pImpl); - dDebug("vgId:%d, is opened by thread:%d", pVnode->vgId, pThread->threadIndex); - pThread->opened++; - } - - atomic_add_fetch_32(&tsVnodes.openVnodes, 1); - } - - dDebug("thread:%d, total vnodes:%d, opened:%d failed:%d", pThread->threadIndex, pThread->vnodeNum, pThread->opened, - pThread->failed); - return NULL; -} - -static int32_t dnodeOpenVnodes() { - taosInitRWLatch(&tsVnodes.latch); - - tsVnodes.hash = taosHashInit(TSDB_MIN_VNODES, taosGetDefaultHashFunction(TSDB_DATA_TYPE_INT), true, HASH_ENTRY_LOCK); - if (tsVnodes.hash == NULL) { - dError("failed to init vnode hash"); - return TSDB_CODE_VND_OUT_OF_MEMORY; - } - - SVnodeObj *pVnodes = NULL; - int32_t numOfVnodes = 0; - int32_t code = dnodeGetVnodesFromFile(&pVnodes, &numOfVnodes); - if (code != TSDB_CODE_SUCCESS) { - dInfo("failed to get vnode list from disk since %s", tstrerror(code)); - return code; - } - - tsVnodes.totalVnodes = numOfVnodes; - - int32_t threadNum = tsNumOfCores; - int32_t vnodesPerThread = numOfVnodes / threadNum + 1; - - SVThread *threads = calloc(threadNum, sizeof(SVThread)); - for (int32_t t = 0; t < threadNum; ++t) { - threads[t].threadIndex = t; - threads[t].pVnodes = calloc(vnodesPerThread, sizeof(SVnodeObj)); - } - - for (int32_t v = 0; v < numOfVnodes; ++v) { - int32_t t = v % threadNum; - SVThread *pThread = &threads[t]; - pThread->pVnodes[pThread->vnodeNum++] = pVnodes[v]; - } - - dInfo("start %d threads to open %d vnodes", threadNum, numOfVnodes); - - for (int32_t t = 0; t < threadNum; ++t) { - SVThread *pThread = &threads[t]; - if (pThread->vnodeNum == 0) continue; - - pThread->threadId = taosCreateThread(dnodeOpenVnodeFunc, pThread); - if (pThread->threadId == NULL) { - dError("thread:%d, failed to create thread to open vnode, reason:%s", pThread->threadIndex, strerror(errno)); - } - } - - for (int32_t t = 0; t < threadNum; ++t) { - SVThread *pThread = &threads[t]; - taosDestoryThread(pThread->threadId); - pThread->threadId = NULL; - free(pThread->pVnodes); - } - free(threads); - - if (tsVnodes.openVnodes != tsVnodes.totalVnodes) { - dError("there are total vnodes:%d, opened:%d", tsVnodes.totalVnodes, tsVnodes.openVnodes); - return -1; - } else { - dInfo("total vnodes:%d open successfully", tsVnodes.totalVnodes); - } - - return TSDB_CODE_SUCCESS; -} - -static void dnodeCloseVnodes() { - int32_t numOfVnodes = 0; - SVnodeObj **pVnodes = dnodeGetVnodesFromHash(&numOfVnodes); - - for (int32_t i = 0; i < numOfVnodes; ++i) { - dnodeDropVnodeWrapper(pVnodes[i]); - } - if (pVnodes != NULL) { - free(pVnodes); - } - - if (tsVnodes.hash != NULL) { - taosHashCleanup(tsVnodes.hash); - tsVnodes.hash = NULL; - } - - dInfo("total vnodes:%d are all closed", numOfVnodes); -} - -static int32_t dnodeParseCreateVnodeReq(SRpcMsg *rpcMsg, int32_t *vgId, SVnodeCfg *pCfg) { - SCreateVnodeMsg *pCreate = rpcMsg->pCont; - *vgId = htonl(pCreate->vgId); - - tstrncpy(pCfg->db, pCreate->db, TSDB_FULL_DB_NAME_LEN); - pCfg->cacheBlockSize = htonl(pCreate->cacheBlockSize); - pCfg->totalBlocks = htonl(pCreate->totalBlocks); - pCfg->daysPerFile = htonl(pCreate->daysPerFile); - pCfg->daysToKeep0 = htonl(pCreate->daysToKeep0); - pCfg->daysToKeep1 = htonl(pCreate->daysToKeep1); - pCfg->daysToKeep2 = htonl(pCreate->daysToKeep2); - pCfg->minRowsPerFileBlock = htonl(pCreate->minRowsPerFileBlock); - pCfg->maxRowsPerFileBlock = htonl(pCreate->maxRowsPerFileBlock); - pCfg->precision = pCreate->precision; - pCfg->compression = pCreate->compression; - pCfg->cacheLastRow = pCreate->cacheLastRow; - pCfg->update = pCreate->update; - pCfg->quorum = pCreate->quorum; - pCfg->replica = pCreate->replica; - pCfg->walLevel = pCreate->walLevel; - pCfg->fsyncPeriod = htonl(pCreate->fsyncPeriod); - - for (int32_t i = 0; i < pCfg->replica; ++i) { - pCfg->replicas[i].port = htons(pCreate->replicas[i].port); - tstrncpy(pCfg->replicas[i].fqdn, pCreate->replicas[i].fqdn, TSDB_FQDN_LEN); - } - - return 0; -} - -static SDropVnodeMsg *vnodeParseDropVnodeReq(SRpcMsg *rpcMsg) { - SDropVnodeMsg *pDrop = rpcMsg->pCont; - pDrop->vgId = htonl(pDrop->vgId); - return pDrop; -} - -static SAuthVnodeMsg *vnodeParseAuthVnodeReq(SRpcMsg *rpcMsg) { - SAuthVnodeMsg *pAuth = rpcMsg->pCont; - pAuth->vgId = htonl(pAuth->vgId); - return pAuth; -} - -static int32_t vnodeProcessCreateVnodeReq(SRpcMsg *rpcMsg) { - SVnodeCfg vnodeCfg = {0}; - int32_t vgId = 0; - - dnodeParseCreateVnodeReq(rpcMsg, &vgId, &vnodeCfg); - dDebug("vgId:%d, create vnode req is received", vgId); - - SVnodeObj *pVnode = dnodeAcquireVnode(vgId); - if (pVnode != NULL) { - dDebug("vgId:%d, already exist, return success", vgId); - dnodeReleaseVnode(pVnode); - return 0; - } - - int32_t code = dnodeCreateVnode(vgId, &vnodeCfg); - if (code != 0) { - dError("vgId:%d, failed to create vnode since %s", vgId, tstrerror(code)); - } - - return code; -} - -static int32_t vnodeProcessAlterVnodeReq(SRpcMsg *rpcMsg) { - SVnodeCfg vnodeCfg = {0}; - int32_t vgId = 0; - int32_t code = 0; - - dnodeParseCreateVnodeReq(rpcMsg, &vgId, &vnodeCfg); - dDebug("vgId:%d, alter vnode req is received", vgId); - - SVnodeObj *pVnode = dnodeAcquireVnode(vgId); - if (pVnode == NULL) { - code = terrno; - dDebug("vgId:%d, failed to alter vnode since %s", vgId, tstrerror(code)); - return code; - } - - code = vnodeAlter(pVnode->pImpl, &vnodeCfg); - if (code != 0) { - dError("vgId:%d, failed to alter vnode since %s", vgId, tstrerror(code)); - } - - dnodeReleaseVnode(pVnode); - return code; -} - -static int32_t vnodeProcessDropVnodeReq(SRpcMsg *rpcMsg) { - SDropVnodeMsg *pDrop = vnodeParseDropVnodeReq(rpcMsg); - - int32_t code = 0; - int32_t vgId = pDrop->vgId; - dDebug("vgId:%d, drop vnode req is received", vgId); - - SVnodeObj *pVnode = dnodeAcquireVnode(vgId); - if (pVnode == NULL) { - code = terrno; - dDebug("vgId:%d, failed to drop since %s", vgId, tstrerror(code)); - return code; - } - - code = dnodeDropVnode(pVnode); - if (code != 0) { - dnodeReleaseVnode(pVnode); - dError("vgId:%d, failed to drop vnode since %s", vgId, tstrerror(code)); - } - - return code; -} - -static int32_t vnodeProcessAuthVnodeReq(SRpcMsg *rpcMsg) { - SAuthVnodeMsg *pAuth = (SAuthVnodeMsg *)vnodeParseAuthVnodeReq(rpcMsg); - - int32_t code = 0; - int32_t vgId = pAuth->vgId; - dDebug("vgId:%d, auth vnode req is received", vgId); - - SVnodeObj *pVnode = dnodeAcquireVnode(vgId); - if (pVnode == NULL) { - code = terrno; - dDebug("vgId:%d, failed to auth since %s", vgId, tstrerror(code)); - return code; - } - - pVnode->accessState = pAuth->accessState; - dnodeReleaseVnode(pVnode); - return code; -} - -static int32_t vnodeProcessSyncVnodeReq(SRpcMsg *rpcMsg) { - SAuthVnodeMsg *pAuth = (SAuthVnodeMsg *)vnodeParseAuthVnodeReq(rpcMsg); - - int32_t code = 0; - int32_t vgId = pAuth->vgId; - dDebug("vgId:%d, auth vnode req is received", vgId); - - SVnodeObj *pVnode = dnodeAcquireVnode(vgId); - if (pVnode == NULL) { - code = terrno; - dDebug("vgId:%d, failed to auth since %s", vgId, tstrerror(code)); - return code; - } - - code = vnodeSync(pVnode->pImpl); - if (code != 0) { - dError("vgId:%d, failed to auth vnode since %s", vgId, tstrerror(code)); - } - - dnodeReleaseVnode(pVnode); - return code; -} - -static int32_t vnodeProcessCompactVnodeReq(SRpcMsg *rpcMsg) { - SCompactVnodeMsg *pCompact = (SCompactVnodeMsg *)vnodeParseDropVnodeReq(rpcMsg); - - int32_t code = 0; - int32_t vgId = pCompact->vgId; - dDebug("vgId:%d, compact vnode req is received", vgId); - - SVnodeObj *pVnode = dnodeAcquireVnode(vgId); - if (pVnode == NULL) { - code = terrno; - dDebug("vgId:%d, failed to compact since %s", vgId, tstrerror(code)); - return code; - } - - code = vnodeCompact(pVnode->pImpl); - if (code != 0) { - dError("vgId:%d, failed to compact vnode since %s", vgId, tstrerror(code)); - } - - dnodeReleaseVnode(pVnode); - return code; -} - -static void dnodeProcessVnodeMgmtQueue(void *unused, SRpcMsg *pMsg) { - int32_t code = 0; - - switch (pMsg->msgType) { - case TSDB_MSG_TYPE_CREATE_VNODE_IN: - code = vnodeProcessCreateVnodeReq(pMsg); - break; - case TSDB_MSG_TYPE_ALTER_VNODE_IN: - code = vnodeProcessAlterVnodeReq(pMsg); - break; - case TSDB_MSG_TYPE_DROP_VNODE_IN: - code = vnodeProcessDropVnodeReq(pMsg); - break; - case TSDB_MSG_TYPE_AUTH_VNODE_IN: - code = vnodeProcessAuthVnodeReq(pMsg); - break; - case TSDB_MSG_TYPE_SYNC_VNODE_IN: - code = vnodeProcessSyncVnodeReq(pMsg); - break; - case TSDB_MSG_TYPE_COMPACT_VNODE_IN: - code = vnodeProcessCompactVnodeReq(pMsg); - break; - default: - code = TSDB_CODE_DND_MSG_NOT_PROCESSED; - break; - } - - SRpcMsg rsp = {.code = code, .handle = pMsg->handle}; - rpcSendResponse(&rsp); - rpcFreeCont(pMsg->pCont); - taosFreeQitem(pMsg); -} - -static void dnodeProcessVnodeQueryQueue(SVnodeObj *pVnode, SVnodeMsg *pMsg) { - vnodeProcessMsg(pVnode->pImpl, pMsg, VN_MSG_TYPE_QUERY); -} - -static void dnodeProcessVnodeFetchQueue(SVnodeObj *pVnode, SVnodeMsg *pMsg) { - vnodeProcessMsg(pVnode->pImpl, pMsg, VN_MSG_TYPE_FETCH); -} - -static void dnodeProcessVnodeWriteQueue(SVnodeObj *pVnode, taos_qall qall, int32_t numOfMsgs) { - SVnodeMsg *pMsg = vnodeInitMsg(numOfMsgs); - SRpcMsg *pRpcMsg = NULL; - - for (int32_t i = 0; i < numOfMsgs; ++i) { - taosGetQitem(qall, (void **)&pRpcMsg); - vnodeAppendMsg(pMsg, pRpcMsg); - taosFreeQitem(pRpcMsg); - } - - vnodeProcessMsg(pVnode->pImpl, pMsg, VN_MSG_TYPE_WRITE); -} - -static void dnodeProcessVnodeApplyQueue(SVnodeObj *pVnode, taos_qall qall, int32_t numOfMsgs) { - SVnodeMsg *pMsg = NULL; - for (int32_t i = 0; i < numOfMsgs; ++i) { - taosGetQitem(qall, (void **)&pMsg); - vnodeProcessMsg(pVnode->pImpl, pMsg, VN_MSG_TYPE_APPLY); - } -} - -static void dnodeProcessVnodeSyncQueue(SVnodeObj *pVnode, taos_qall qall, int32_t numOfMsgs) { - SVnodeMsg *pMsg = NULL; - for (int32_t i = 0; i < numOfMsgs; ++i) { - taosGetQitem(qall, (void **)&pMsg); - vnodeProcessMsg(pVnode->pImpl, pMsg, VN_MSG_TYPE_SYNC); - } -} - -static int32_t dnodeWriteRpcMsgToVnodeQueue(taos_queue pQueue, SRpcMsg *pRpcMsg) { - int32_t code = 0; - - if (pQueue == NULL) { - code = TSDB_CODE_DND_MSG_NOT_PROCESSED; - } else { - SRpcMsg *pMsg = taosAllocateQitem(sizeof(SRpcMsg)); - if (pMsg == NULL) { - code = TSDB_CODE_DND_OUT_OF_MEMORY; - } else { - *pMsg = *pRpcMsg; - code = taosWriteQitem(pQueue, pMsg); - } - } - - if (code != TSDB_CODE_SUCCESS) { - SRpcMsg rsp = {.handle = pRpcMsg->handle, .code = code}; - rpcSendResponse(&rsp); - rpcFreeCont(pRpcMsg->pCont); - } -} - -static int32_t dnodeWriteVnodeMsgToVnodeQueue(taos_queue pQueue, SRpcMsg *pRpcMsg) { - int32_t code = 0; - - if (pQueue == NULL) { - code = TSDB_CODE_DND_MSG_NOT_PROCESSED; - } else { - SVnodeMsg *pMsg = vnodeInitMsg(1); - if (pMsg == NULL) { - code = TSDB_CODE_DND_OUT_OF_MEMORY; - } else { - vnodeAppendMsg(pMsg, pRpcMsg); - code = taosWriteQitem(pQueue, pMsg); - } - } - - if (code != TSDB_CODE_SUCCESS) { - SRpcMsg rsp = {.handle = pRpcMsg->handle, .code = code}; - rpcSendResponse(&rsp); - rpcFreeCont(pRpcMsg->pCont); - } -} - -static SVnodeObj *dnodeAcquireVnodeFromMsg(SRpcMsg *pMsg) { - SMsgHead *pHead = (SMsgHead *)pMsg->pCont; - pHead->vgId = htonl(pHead->vgId); - - SVnodeObj *pVnode = dnodeAcquireVnode(pHead->vgId); - if (pVnode == NULL) { - SRpcMsg rsp = {.handle = pMsg->handle, .code = terrno}; - rpcSendResponse(&rsp); - rpcFreeCont(pMsg->pCont); - } - - return pVnode; -} - -void dnodeProcessVnodeMgmtMsg(SRpcMsg *pMsg, SEpSet *pEpSet) { dnodeWriteRpcMsgToVnodeQueue(tsVnodes.pMgmtQ, pMsg); } - -void dnodeProcessVnodeWriteMsg(SRpcMsg *pMsg, SEpSet *pEpSet) { - SVnodeObj *pVnode = dnodeAcquireVnodeFromMsg(pMsg); - if (pVnode != NULL) { - dnodeWriteRpcMsgToVnodeQueue(pVnode->pWriteQ, pMsg); - dnodeReleaseVnode(pVnode); - } -} - -void dnodeProcessVnodeSyncMsg(SRpcMsg *pMsg, SEpSet *pEpSet) { - SVnodeObj *pVnode = dnodeAcquireVnodeFromMsg(pMsg); - if (pVnode != NULL) { - dnodeWriteVnodeMsgToVnodeQueue(pVnode->pSyncQ, pMsg); - dnodeReleaseVnode(pVnode); - } -} - -void dnodeProcessVnodeQueryMsg(SRpcMsg *pMsg, SEpSet *pEpSet) { - SVnodeObj *pVnode = dnodeAcquireVnodeFromMsg(pMsg); - if (pVnode != NULL) { - dnodeWriteVnodeMsgToVnodeQueue(pVnode->pQueryQ, pMsg); - dnodeReleaseVnode(pVnode); - } -} - -void dnodeProcessVnodeFetchMsg(SRpcMsg *pMsg, SEpSet *pEpSet) { - SVnodeObj *pVnode = dnodeAcquireVnodeFromMsg(pMsg); - if (pVnode != NULL) { - dnodeWriteVnodeMsgToVnodeQueue(pVnode->pFetchQ, pMsg); - dnodeReleaseVnode(pVnode); - } -} - -static int32_t dnodePutMsgIntoVnodeApplyQueue(SDnode *pDnode, int32_t vgId, SVnodeMsg *pMsg) { - SVnodeObj *pVnode = dnodeAcquireVnode(vgId); - if (pVnode == NULL) { - return terrno; - } - - int32_t code = taosWriteQitem(pVnode->pApplyQ, pMsg); - dnodeReleaseVnode(pVnode); - return code; -} - -static int32_t dnodeInitVnodeMgmtWorker() { - SWorkerPool *pPool = &tsVnodes.mgmtPool; - pPool->name = "vnode-mgmt"; - pPool->min = 1; - pPool->max = 1; - if (tWorkerInit(pPool) != 0) { - return TSDB_CODE_VND_OUT_OF_MEMORY; - } - - tsVnodes.pMgmtQ = tWorkerAllocQueue(pPool, NULL, (FProcessItem)dnodeProcessVnodeMgmtQueue); - if (tsVnodes.pMgmtQ == NULL) { - return TSDB_CODE_VND_OUT_OF_MEMORY; - } - - return 0; -} - -static void dnodeCleanupVnodeMgmtWorker() { - tWorkerFreeQueue(&tsVnodes.mgmtPool, tsVnodes.pMgmtQ); - tWorkerCleanup(&tsVnodes.mgmtPool); - tsVnodes.pMgmtQ = NULL; -} - -static int32_t dnodeAllocVnodeQueryQueue(SVnodeObj *pVnode) { - pVnode->pQueryQ = tWorkerAllocQueue(&tsVnodes.queryPool, pVnode, (FProcessItem)dnodeProcessVnodeQueryQueue); - if (pVnode->pQueryQ == NULL) { - return TSDB_CODE_DND_OUT_OF_MEMORY; - } - return 0; -} - -static void dnodeFreeVnodeQueryQueue(SVnodeObj *pVnode) { - tWorkerFreeQueue(&tsVnodes.queryPool, pVnode->pQueryQ); - pVnode->pQueryQ = NULL; -} - -static int32_t dnodeAllocVnodeFetchQueue(SVnodeObj *pVnode) { - pVnode->pFetchQ = tWorkerAllocQueue(&tsVnodes.fetchPool, pVnode, (FProcessItem)dnodeProcessVnodeFetchQueue); - if (pVnode->pFetchQ == NULL) { - return TSDB_CODE_DND_OUT_OF_MEMORY; - } - return 0; -} - -static void dnodeFreeVnodeFetchQueue(SVnodeObj *pVnode) { - tWorkerFreeQueue(&tsVnodes.fetchPool, pVnode->pFetchQ); - pVnode->pFetchQ = NULL; -} - -static int32_t dnodeInitVnodeReadWorker() { - int32_t maxFetchThreads = 4; - float threadsForQuery = MAX(tsNumOfCores * tsRatioOfQueryCores, 1); - - SWorkerPool *pPool = &tsVnodes.queryPool; - pPool->name = "vnode-query"; - pPool->min = (int32_t)threadsForQuery; - pPool->max = pPool->min; - if (tWorkerInit(pPool) != 0) { - return TSDB_CODE_VND_OUT_OF_MEMORY; - } - - pPool = &tsVnodes.fetchPool; - pPool->name = "vnode-fetch"; - pPool->min = MIN(maxFetchThreads, tsNumOfCores); - pPool->max = pPool->min; - if (tWorkerInit(pPool) != 0) { - TSDB_CODE_VND_OUT_OF_MEMORY; - } - - return 0; -} - -static void dnodeCleanupVnodeReadWorker() { - tWorkerCleanup(&tsVnodes.fetchPool); - tWorkerCleanup(&tsVnodes.queryPool); -} - -static int32_t dnodeAllocVnodeWriteQueue(SVnodeObj *pVnode) { - pVnode->pWriteQ = tMWorkerAllocQueue(&tsVnodes.writePool, pVnode, (FProcessItems)dnodeProcessVnodeWriteQueue); - if (pVnode->pWriteQ == NULL) { - return TSDB_CODE_DND_OUT_OF_MEMORY; - } - return 0; -} - -static void dnodeFreeVnodeWriteQueue(SVnodeObj *pVnode) { - tMWorkerFreeQueue(&tsVnodes.writePool, pVnode->pWriteQ); - pVnode->pWriteQ = NULL; -} - -static int32_t dnodeAllocVnodeApplyQueue(SVnodeObj *pVnode) { - pVnode->pApplyQ = tMWorkerAllocQueue(&tsVnodes.writePool, pVnode, (FProcessItems)dnodeProcessVnodeApplyQueue); - if (pVnode->pApplyQ == NULL) { - return TSDB_CODE_DND_OUT_OF_MEMORY; - } - return 0; -} - -static void dnodeFreeVnodeApplyQueue(SVnodeObj *pVnode) { - tMWorkerFreeQueue(&tsVnodes.writePool, pVnode->pApplyQ); - pVnode->pApplyQ = NULL; -} - -static int32_t dnodeInitVnodeWriteWorker() { - SMWorkerPool *pPool = &tsVnodes.writePool; - pPool->name = "vnode-write"; - pPool->max = tsNumOfCores; - if (tMWorkerInit(pPool) != 0) { - return TSDB_CODE_VND_OUT_OF_MEMORY; - } - - return 0; -} - -static void dnodeCleanupVnodeWriteWorker() { tMWorkerCleanup(&tsVnodes.writePool); } - -static int32_t dnodeAllocVnodeSyncQueue(SVnodeObj *pVnode) { - pVnode->pSyncQ = tMWorkerAllocQueue(&tsVnodes.writePool, pVnode, (FProcessItems)dnodeProcessVnodeSyncQueue); - if (pVnode->pSyncQ == NULL) { - return TSDB_CODE_DND_OUT_OF_MEMORY; - } - return 0; -} - -static void dnodeFreeVnodeSyncQueue(SVnodeObj *pVnode) { - tMWorkerFreeQueue(&tsVnodes.writePool, pVnode->pSyncQ); - pVnode->pSyncQ = NULL; -} - -static int32_t dnodeInitVnodeSyncWorker() { - int32_t maxThreads = tsNumOfCores / 2; - if (maxThreads < 1) maxThreads = 1; - - SMWorkerPool *pPool = &tsVnodes.writePool; - pPool->name = "vnode-sync"; - pPool->max = maxThreads; - if (tMWorkerInit(pPool) != 0) { - return TSDB_CODE_VND_OUT_OF_MEMORY; - } - - return 0; -} - -static void dnodeCleanupVnodeSyncWorker() { tMWorkerCleanup(&tsVnodes.syncPool); } - -static int32_t dnodeInitVnodeModule() { - SVnodePara para; - para.sendMsgToDnodeFp = dnodeSendMsgToDnode; - para.sendMsgToMnodeFp = dnodeSendMsgToMnode; - para.putMsgToApplyQueueFp = dnodePutMsgIntoVnodeApplyQueue; - - return vnodeInit(para); -} - -int32_t dnodeInitVnodes() { - dInfo("dnode-vnodes start to init"); - - SSteps *pSteps = taosStepInit(6, dnodeReportStartup); - taosStepAdd(pSteps, "dnode-vnode-env", dnodeInitVnodeModule, vnodeCleanup); - taosStepAdd(pSteps, "dnode-vnode-mgmt", dnodeInitVnodeMgmtWorker, dnodeCleanupVnodeMgmtWorker); - taosStepAdd(pSteps, "dnode-vnode-read", dnodeInitVnodeReadWorker, dnodeCleanupVnodeReadWorker); - taosStepAdd(pSteps, "dnode-vnode-write", dnodeInitVnodeWriteWorker, dnodeCleanupVnodeWriteWorker); - taosStepAdd(pSteps, "dnode-vnode-sync", dnodeInitVnodeSyncWorker, dnodeCleanupVnodeSyncWorker); - taosStepAdd(pSteps, "dnode-vnodes", dnodeOpenVnodes, dnodeCleanupVnodes); - - tsVnodes.pSteps = pSteps; - return taosStepExec(pSteps); -} - -void dnodeCleanupVnodes() { - if (tsVnodes.pSteps != NULL) { - dInfo("dnode-vnodes start to clean up"); - taosStepCleanup(tsVnodes.pSteps); - tsVnodes.pSteps = NULL; - dInfo("dnode-vnodes is cleaned up"); - } -} - -void dnodeGetVnodeLoads(SVnodeLoads *pLoads) { - pLoads->num = taosHashGetSize(tsVnodes.hash); - - int32_t v = 0; - void *pIter = taosHashIterate(tsVnodes.hash, NULL); - while (pIter) { - SVnodeObj **ppVnode = pIter; - if (ppVnode == NULL) continue; - - SVnodeObj *pVnode = *ppVnode; - if (pVnode == NULL) continue; - - SVnodeLoad *pLoad = &pLoads->data[v++]; - vnodeGetLoad(pVnode->pImpl, pLoad); - pLoad->vgId = htonl(pLoad->vgId); - pLoad->totalStorage = htobe64(pLoad->totalStorage); - pLoad->compStorage = htobe64(pLoad->compStorage); - pLoad->pointsWritten = htobe64(pLoad->pointsWritten); - pLoad->tablesNum = htobe64(pLoad->tablesNum); - - pIter = taosHashIterate(tsVnodes.hash, pIter); - } -} diff --git a/source/dnode/mnode/impl/inc/mnodeDef.h b/source/dnode/mnode/impl/inc/mnodeDef.h index ccdba13006..4b4c4abdb3 100644 --- a/source/dnode/mnode/impl/inc/mnodeDef.h +++ b/source/dnode/mnode/impl/inc/mnodeDef.h @@ -131,7 +131,7 @@ typedef struct SMnodeObj { int64_t roleTime; int64_t createdTime; int64_t updateTime; - SDnodeObj *pDnode; + SDnodeObj *pDnd; } SMnodeObj; typedef struct { @@ -215,7 +215,7 @@ typedef struct SDbObj { typedef struct { int32_t dnodeId; int8_t role; - SDnodeObj *pDnode; + SDnodeObj *pDnd; } SVnodeGid; typedef struct SVgObj { diff --git a/source/dnode/mnode/impl/src/mnode.c b/source/dnode/mnode/impl/src/mnode.c index bc4718ee5b..877ec1431c 100644 --- a/source/dnode/mnode/impl/src/mnode.c +++ b/source/dnode/mnode/impl/src/mnode.c @@ -83,7 +83,7 @@ static int32_t mnodeSetOptions(SMnode *pMnode, const SMnodeOptions *pOptions) { pMnode->replica = pOptions->replica; pMnode->selfIndex = pOptions->selfIndex; memcpy(&pMnode->replicas, pOptions->replicas, sizeof(SReplica) * TSDB_MAX_REPLICA); - pMnode->pServer = pOptions->pServer; + pMnode->pServer = pOptions->pDnode; pMnode->putMsgToApplyMsgFp = pOptions->putMsgToApplyMsgFp; pMnode->sendMsgToDnodeFp = pOptions->sendMsgToDnodeFp; pMnode->sendMsgToMnodeFp = pOptions->sendMsgToMnodeFp; @@ -187,7 +187,7 @@ SMnodeMsg *mnodeInitMsg(SMnode *pMnode, SRpcMsg *pRpcMsg) { } if (rpcGetConnInfo(pRpcMsg->handle, &pMsg->conn) != 0) { - mnodeCleanupMsg(pMnode, pMsg); + mnodeCleanupMsg(pMsg); mError("can not get user from conn:%p", pMsg->rpcMsg.handle); terrno = TSDB_CODE_MND_NO_USER_FROM_CONN; return NULL; @@ -199,7 +199,7 @@ SMnodeMsg *mnodeInitMsg(SMnode *pMnode, SRpcMsg *pRpcMsg) { return pMsg; } -void mnodeCleanupMsg(SMnode *pMnode, SMnodeMsg *pMsg) { +void mnodeCleanupMsg(SMnodeMsg *pMsg) { if (pMsg->pUser != NULL) { sdbRelease(pMsg->pUser); } @@ -210,7 +210,7 @@ void mnodeCleanupMsg(SMnode *pMnode, SMnodeMsg *pMsg) { static void mnodeProcessRpcMsg(SMnodeMsg *pMsg) { if (!mnodeIsMaster()) { mnodeSendRedirectMsg(NULL, &pMsg->rpcMsg, true); - mnodeCleanupMsg(NULL, pMsg); + mnodeCleanupMsg(pMsg); return; } diff --git a/source/libs/transport/src/rpcMain.c b/source/libs/transport/src/rpcMain.c index 934a8dd6ab..a2041c76fc 100644 --- a/source/libs/transport/src/rpcMain.c +++ b/source/libs/transport/src/rpcMain.c @@ -54,10 +54,11 @@ typedef struct { char secret[TSDB_KEY_LEN]; // secret for the link char ckey[TSDB_KEY_LEN]; // ciphering key - void (*cfp)(SRpcMsg *, SEpSet *); - int (*afp)(char *user, char *spi, char *encrypt, char *secret, char *ckey); + void (*cfp)(void *parent, SRpcMsg *, SEpSet *); + int (*afp)(void *parent, char *user, char *spi, char *encrypt, char *secret, char *ckey); int32_t refCount; + void *parent; void *idPool; // handle to ID pool void *tmrCtrl; // handle to timer SHashObj *hash; // handle returned by hash utility @@ -260,6 +261,7 @@ void *rpcOpen(const SRpcInit *pInit) { pRpc->spi = pInit->spi; pRpc->cfp = pInit->cfp; pRpc->afp = pInit->afp; + pRpc->parent = pInit->parent; pRpc->refCount = 1; atomic_add_fetch_32(&tsRpcNum, 1); @@ -744,7 +746,7 @@ static SRpcConn *rpcAllocateServerConn(SRpcInfo *pRpc, SRecvInfo *pRecv) { if (pConn->user[0] == 0) { terrno = TSDB_CODE_RPC_AUTH_REQUIRED; } else { - terrno = (*pRpc->afp)(pConn->user, &pConn->spi, &pConn->encrypt, pConn->secret, pConn->ckey); + terrno = (*pRpc->afp)(pRpc->parent, pConn->user, &pConn->spi, &pConn->encrypt, pConn->secret, pConn->ckey); } if (terrno != 0) { @@ -1024,8 +1026,8 @@ static void doRpcReportBrokenLinkToServer(void *param, void *id) { SRpcMsg *pRpcMsg = (SRpcMsg *)(param); SRpcConn *pConn = (SRpcConn *)(pRpcMsg->handle); SRpcInfo *pRpc = pConn->pRpc; - (*(pRpc->cfp))(pRpcMsg, NULL); - free(pRpcMsg); + (*(pRpc->cfp))(pRpc->parent, pRpcMsg, NULL); + free(pRpcMsg); } static void rpcReportBrokenLinkToServer(SRpcConn *pConn) { SRpcInfo *pRpc = pConn->pRpc; @@ -1137,9 +1139,9 @@ static void rpcNotifyClient(SRpcReqContext *pContext, SRpcMsg *pMsg) { // for asynchronous API SEpSet *pEpSet = NULL; if (pContext->epSet.inUse != pContext->oldInUse || pContext->redirect) - pEpSet = &pContext->epSet; + pEpSet = &pContext->epSet; - (*pRpc->cfp)(pMsg, pEpSet); + (*pRpc->cfp)(pRpc->parent, pMsg, pEpSet); } // free the request message @@ -1155,15 +1157,15 @@ static void rpcProcessIncomingMsg(SRpcConn *pConn, SRpcHead *pHead, SRpcReqConte rpcMsg.contLen = rpcContLenFromMsg(pHead->msgLen); rpcMsg.pCont = pHead->content; rpcMsg.msgType = pHead->msgType; - rpcMsg.code = pHead->code; - - if ( rpcIsReq(pHead->msgType) ) { + rpcMsg.code = pHead->code; + + if (rpcIsReq(pHead->msgType)) { rpcMsg.ahandle = pConn->ahandle; rpcMsg.handle = pConn; rpcAddRef(pRpc); // add the refCount for requests // notify the server app - (*(pRpc->cfp))(&rpcMsg, NULL); + (*(pRpc->cfp))(pRpc->parent, &rpcMsg, NULL); } else { // it's a response rpcMsg.handle = pContext; diff --git a/source/util/src/terror.c b/source/util/src/terror.c index 0b10274c00..6d4f4eb6a7 100644 --- a/source/util/src/terror.c +++ b/source/util/src/terror.c @@ -78,6 +78,7 @@ TAOS_DEFINE_ERROR(TSDB_CODE_MEMORY_CORRUPTED, "Memory corrupted") TAOS_DEFINE_ERROR(TSDB_CODE_FILE_CORRUPTED, "Data file corrupted") TAOS_DEFINE_ERROR(TSDB_CODE_CHECKSUM_ERROR, "Checksum error") TAOS_DEFINE_ERROR(TSDB_CODE_INVALID_MSG, "Invalid config message") +TAOS_DEFINE_ERROR(TSDB_CODE_MSG_NOT_PROCESSED, "Message not processed") TAOS_DEFINE_ERROR(TSDB_CODE_REF_NO_MEMORY, "Ref out of memory") TAOS_DEFINE_ERROR(TSDB_CODE_REF_FULL, "too many Ref Objs") TAOS_DEFINE_ERROR(TSDB_CODE_REF_ID_REMOVED, "Ref ID is removed") @@ -235,20 +236,20 @@ TAOS_DEFINE_ERROR(TSDB_CODE_MND_INVALID_TOPIC_PARTITONS, "Invalid topic partito TAOS_DEFINE_ERROR(TSDB_CODE_MND_TOPIC_ALREADY_EXIST, "Topic already exists") // dnode -TAOS_DEFINE_ERROR(TSDB_CODE_DND_MSG_NOT_PROCESSED, "Message not processed") -TAOS_DEFINE_ERROR(TSDB_CODE_DND_OUT_OF_MEMORY, "Dnode out of memory") -TAOS_DEFINE_ERROR(TSDB_CODE_DND_MNODE_ID_NOT_MATCH_DNODE, "Mnode Id not match Dnode") +TAOS_DEFINE_ERROR(TSDB_CODE_DND_ACTION_IN_PROGRESS, "Action in progress") +TAOS_DEFINE_ERROR(TSDB_CODE_DND_EXITING, "Dnode is exiting") +TAOS_DEFINE_ERROR(TSDB_CODE_DND_INVALID_MSG_LEN, "Invalid message length") +TAOS_DEFINE_ERROR(TSDB_CODE_DND_DNODE_READ_FILE_ERROR, "Read dnode.json error") +TAOS_DEFINE_ERROR(TSDB_CODE_DND_DNODE_WRITE_FILE_ERROR, "Write dnode.json error") TAOS_DEFINE_ERROR(TSDB_CODE_DND_MNODE_ALREADY_DEPLOYED, "Mnode already deployed") TAOS_DEFINE_ERROR(TSDB_CODE_DND_MNODE_NOT_DEPLOYED, "Mnode not deployed") -TAOS_DEFINE_ERROR(TSDB_CODE_DND_READ_MNODE_FILE_ERROR, "Read mnode.json error") -TAOS_DEFINE_ERROR(TSDB_CODE_DND_WRITE_MNODE_FILE_ERROR, "Write mnode.json error") -TAOS_DEFINE_ERROR(TSDB_CODE_DND_NO_WRITE_ACCESS, "No permission for disk files in dnode") -TAOS_DEFINE_ERROR(TSDB_CODE_DND_INVALID_MSG_LEN, "Invalid message length") -TAOS_DEFINE_ERROR(TSDB_CODE_DND_ACTION_IN_PROGRESS, "Action in progress") -TAOS_DEFINE_ERROR(TSDB_CODE_DND_TOO_MANY_VNODES, "Too many vnode directories") -TAOS_DEFINE_ERROR(TSDB_CODE_DND_EXITING, "Dnode is exiting") -TAOS_DEFINE_ERROR(TSDB_CODE_DND_PARSE_VNODE_FILE_ERROR, "Parse vnodes.json error") -TAOS_DEFINE_ERROR(TSDB_CODE_DND_PARSE_DNODE_FILE_ERROR, "Parse dnodes.json error") +TAOS_DEFINE_ERROR(TSDB_CODE_DND_MNODE_ID_INVALID, "Mnode Id invalid") +TAOS_DEFINE_ERROR(TSDB_CODE_DND_MNODE_ID_NOT_FOUND, "Mnode Id not found") +TAOS_DEFINE_ERROR(TSDB_CODE_DND_MNODE_READ_FILE_ERROR, "Read mnode.json error") +TAOS_DEFINE_ERROR(TSDB_CODE_DND_MNODE_WRITE_FILE_ERROR, "Write mnode.json error") +TAOS_DEFINE_ERROR(TSDB_CODE_DND_VNODE_TOO_MANY_VNODES, "Too many vnode directories") +TAOS_DEFINE_ERROR(TSDB_CODE_DND_VNODE_READ_FILE_ERROR, "Read vnodes.json error") +TAOS_DEFINE_ERROR(TSDB_CODE_DND_VNODE_WRITE_FILE_ERROR, "Write vnodes.json error") // vnode TAOS_DEFINE_ERROR(TSDB_CODE_VND_ACTION_IN_PROGRESS, "Action in progress") From 6c626909113e4580beac3f74ade258430de6a408 Mon Sep 17 00:00:00 2001 From: Shengliang Guan Date: Tue, 23 Nov 2021 18:20:57 +0800 Subject: [PATCH 17/44] fix compile errors --- include/util/tnote.h | 6 +++--- source/dnode/mgmt/daemon/src/daemon.c | 11 +++++++---- source/dnode/mgmt/impl/CMakeLists.txt | 2 +- source/dnode/mgmt/impl/src/dndInt.c | 5 +++++ source/dnode/mnode/impl/src/mnode.c | 2 ++ source/util/src/tnote.c | 5 +++-- 6 files changed, 21 insertions(+), 10 deletions(-) diff --git a/include/util/tnote.h b/include/util/tnote.h index f17857a32b..e613ec7e41 100644 --- a/include/util/tnote.h +++ b/include/util/tnote.h @@ -38,9 +38,9 @@ extern SNoteObj tsHttpNote; extern SNoteObj tsTscNote; extern SNoteObj tsInfoNote; -void taosInitNotes(); -void taosNotePrint(SNoteObj* pNote, const char* const format, ...); -void taosNotePrintBuffer(SNoteObj *pNote, char *buffer, int32_t len); +int32_t taosInitNotes(); +void taosNotePrint(SNoteObj* pNote, const char* const format, ...); +void taosNotePrintBuffer(SNoteObj* pNote, char* buffer, int32_t len); #define nPrintHttp(...) \ if (tsHttpEnableRecordSql) { \ diff --git a/source/dnode/mgmt/daemon/src/daemon.c b/source/dnode/mgmt/daemon/src/daemon.c index 326dfae4af..e160de62d9 100644 --- a/source/dnode/mgmt/daemon/src/daemon.c +++ b/source/dnode/mgmt/daemon/src/daemon.c @@ -16,7 +16,9 @@ #define _DEFAULT_SOURCE #include "dnode.h" #include "os.h" +#include "tconfig.h" #include "tglobal.h" +#include "tnote.h" #include "ulog.h" static struct { @@ -68,7 +70,11 @@ int dmnParseOpts(int argc, char const *argv[]) { return 0; } -void dmnGenerateGrant() { grantParseParameter(); } +void dmnGenerateGrant() { +#if 0 + grantParseParameter(); +#endif +} void dmnPrintVersion() { #ifdef TD_ENTERPRISE @@ -83,9 +89,6 @@ void dmnPrintVersion() { } int dmnReadConfig(const char *path) { - taosIgnSIGPIPE(); - taosBlockSIGPIPE(); - taosResolveCRC(); taosInitGlobalCfg(); taosReadGlobalLogCfg(); diff --git a/source/dnode/mgmt/impl/CMakeLists.txt b/source/dnode/mgmt/impl/CMakeLists.txt index b061d75731..d0830484ed 100644 --- a/source/dnode/mgmt/impl/CMakeLists.txt +++ b/source/dnode/mgmt/impl/CMakeLists.txt @@ -1,5 +1,5 @@ aux_source_directory(src DNODE_SRC) -add_library(dnode ${DNODE_SRC}) +add_library(dnode STATIC ${DNODE_SRC}) target_link_libraries( dnode PUBLIC cjson diff --git a/source/dnode/mgmt/impl/src/dndInt.c b/source/dnode/mgmt/impl/src/dndInt.c index 9e77da61b4..9f575ddb4d 100644 --- a/source/dnode/mgmt/impl/src/dndInt.c +++ b/source/dnode/mgmt/impl/src/dndInt.c @@ -21,6 +21,7 @@ #include "sync.h" #include "tcache.h" #include "wal.h" +#include "tcrc32c.h" EStat dndGetStat(SDnode *pDnode) { return pDnode->stat; } @@ -135,6 +136,10 @@ static void dndCleanupEnv(SDnode *pDnode) { } SDnode *dndInit(SDnodeOpt *pOptions) { + taosIgnSIGPIPE(); + taosBlockSIGPIPE(); + taosResolveCRC(); + SDnode *pDnode = calloc(1, sizeof(pDnode)); if (pDnode == NULL) { dError("failed to create dnode object"); diff --git a/source/dnode/mnode/impl/src/mnode.c b/source/dnode/mnode/impl/src/mnode.c index 877ec1431c..5eff8a37ca 100644 --- a/source/dnode/mnode/impl/src/mnode.c +++ b/source/dnode/mnode/impl/src/mnode.c @@ -207,6 +207,8 @@ void mnodeCleanupMsg(SMnodeMsg *pMsg) { taosFreeQitem(pMsg); } +void mnodeSendRsp(SMnodeMsg *pMsg, int32_t code) {} + static void mnodeProcessRpcMsg(SMnodeMsg *pMsg) { if (!mnodeIsMaster()) { mnodeSendRedirectMsg(NULL, &pMsg->rpcMsg, true); diff --git a/source/util/src/tnote.c b/source/util/src/tnote.c index 50b1e0002d..5606ab248d 100644 --- a/source/util/src/tnote.c +++ b/source/util/src/tnote.c @@ -40,7 +40,7 @@ static void taosInitNote(int32_t numOfLines, int32_t maxNotes, SNoteObj *pNote, taosNotePrint(pNote, "=================================================="); } -void taosInitNotes() { +int32_t taosInitNotes() { char name[TSDB_FILENAME_LEN * 2] = {0}; #if 0 @@ -58,7 +58,8 @@ void taosInitNotes() { snprintf(name, TSDB_FILENAME_LEN * 2, "%s/taosinfo", tsLogDir); taosInitNote(tsNumOfLogLines, 1, &tsInfoNote, name); } -#endif +#endif + return 0; } static bool taosLockNote(int32_t fd, SNoteObj *pNote) { From 94eac805b99762db1289bcba9ec50f5d33eb747f Mon Sep 17 00:00:00 2001 From: Shengliang Guan Date: Tue, 23 Nov 2021 19:54:17 +0800 Subject: [PATCH 18/44] TD-10430 refact osDir.h and daemon module --- include/dnode/mgmt/dnode.h | 110 ++++++++++++++++++++---- include/os/osDir.h | 12 +-- source/common/src/tglobal.c | 2 +- source/dnode/mgmt/daemon/src/daemon.c | 12 +-- source/dnode/mgmt/impl/src/dndInt.c | 6 +- source/dnode/mnode/sdb/src/sdbFile.c | 6 +- source/dnode/vnode/tq/src/tqMetaStore.c | 2 +- source/libs/wal/src/walMgmt.c | 2 +- source/os/src/osDir.c | 18 ++-- source/util/src/tconfig.c | 2 +- src/client/src/tscSub.c | 2 +- src/tfs/src/tfs.c | 2 +- 12 files changed, 125 insertions(+), 51 deletions(-) diff --git a/include/dnode/mgmt/dnode.h b/include/dnode/mgmt/dnode.h index a1a94bb10b..7dd7730443 100644 --- a/include/dnode/mgmt/dnode.h +++ b/include/dnode/mgmt/dnode.h @@ -26,38 +26,112 @@ extern "C" { typedef struct SDnode SDnode; typedef struct { - int32_t sver; - int32_t numOfCores; - float numOfThreadsPerCore; - float ratioOfQueryCores; - int32_t maxShellConns; - int32_t shellActivityTimer; - int32_t statusInterval; + /** + * @brief software version of the program. + * + */ + int32_t sver; + + /** + * @brief num of CPU cores. + * + */ + int32_t numOfCores; + + /** + * @brief number of threads per CPU core. + * + */ + float numOfThreadsPerCore; + + /** + * @brief the proportion of total CPU cores available for query processing. + * + */ + float ratioOfQueryCores; + + /** + * @brief max number of connections allowed in dnode. + * + */ + int32_t maxShellConns; + + /** + * @brief time interval of heart beat from shell to dnode, seconds. + * + */ + int32_t shellActivityTimer; + + /** + * @brief time interval of dnode status reporting to mnode, seconds, for cluster only. + * + */ + int32_t statusInterval; + + /** + * @brief first port number for the connection (12 continuous UDP/TCP port number are used). + * + */ uint16_t serverPort; - char dataDir[PATH_MAX]; - char localEp[TSDB_EP_LEN]; - char localFqdn[TSDB_FQDN_LEN]; - char firstEp[TSDB_EP_LEN]; - char timezone[TSDB_TIMEZONE_LEN]; - char locale[TSDB_LOCALE_LEN]; - char charset[TSDB_LOCALE_LEN]; + + /** + * @brief data file's directory. + * + */ + char dataDir[PATH_MAX]; + + /** + * @brief local endpoint. + * + */ + char localEp[TSDB_EP_LEN]; + + /** + * @brieflocal fully qualified domain name (FQDN). + * + */ + char localFqdn[TSDB_FQDN_LEN]; + + /** + * @brief first fully qualified domain name (FQDN) for TDengine system. + * + */ + char firstEp[TSDB_EP_LEN]; + + /** + * @brief system time zone. + * + */ + char timezone[TSDB_TIMEZONE_LEN]; + + /** + * @brief system locale. + * + */ + char locale[TSDB_LOCALE_LEN]; + + /** + * @briefdefault system charset. + * + */ + char charset[TSDB_LOCALE_LEN]; } SDnodeOpt; /* ------------------------ SDnode ------------------------ */ /** * @brief Initialize and start the dnode. * - * @param cfgPath Config file path. + * @param pOptions Options of the dnode. * @return SDnode* The dnode object. */ SDnode *dndInit(SDnodeOpt *pOptions); /** - * @brief Stop and cleanup dnode. + * @brief Stop and cleanup the dnode. * - * @param pDnd The dnode object to close. + * @param pDnode The dnode object to close. */ -void dndCleanup(SDnode *pDnd); +void dndCleanup(SDnode *pDnode); #ifdef __cplusplus } diff --git a/include/os/osDir.h b/include/os/osDir.h index 8aefaa171a..35c294af39 100644 --- a/include/os/osDir.h +++ b/include/os/osDir.h @@ -20,12 +20,12 @@ extern "C" { #endif -void taosRemoveDir(const char *dirname); -bool taosDirExist(char *dirname); -bool taosMkDir(const char *dirname); -void taosRemoveOldFiles(char *dirname, int32_t keepDays); -bool taosExpandDir(char *dirname, char *outname, int32_t maxlen); -bool taosRealPath(char *dirname, int32_t maxlen); +void taosRemoveDir(const char *dirname); +int32_t taosDirExist(char *dirname); +int32_t taosMkDir(const char *dirname); +void taosRemoveOldFiles(char *dirname, int32_t keepDays); +int32_t taosExpandDir(char *dirname, char *outname, int32_t maxlen); +int32_t taosRealPath(char *dirname, int32_t maxlen); #ifdef __cplusplus } diff --git a/source/common/src/tglobal.c b/source/common/src/tglobal.c index a19f6ec04b..501936d354 100644 --- a/source/common/src/tglobal.c +++ b/source/common/src/tglobal.c @@ -1679,7 +1679,7 @@ int32_t taosCheckGlobalCfg() { taosCheckDataDirCfg(); - if (!taosDirExist(tsTempDir)) { + if (taosDirExist(tsTempDir) != 0) { return -1; } diff --git a/source/dnode/mgmt/daemon/src/daemon.c b/source/dnode/mgmt/daemon/src/daemon.c index e160de62d9..a57db1f778 100644 --- a/source/dnode/mgmt/daemon/src/daemon.c +++ b/source/dnode/mgmt/daemon/src/daemon.c @@ -40,7 +40,7 @@ void dmnSetSignalHandle() { taosSetSignal(SIGBREAK, dmnSigintHandle); } -int dmnParseOpts(int argc, char const *argv[]) { +int dmnParseOption(int argc, char const *argv[]) { tstrncpy(global.configDir, "/etc/taos", PATH_MAX); for (int i = 1; i < argc; ++i) { @@ -52,7 +52,7 @@ int dmnParseOpts(int argc, char const *argv[]) { } tstrncpy(global.configDir, argv[i], PATH_MAX); } else { - printf("'-c' requires a parameter, default:%s\n", configDir); + printf("'-c' requires a parameter, default is %s\n", configDir); return -1; } } else if (strcmp(argv[i], "-C") == 0) { @@ -78,11 +78,11 @@ void dmnGenerateGrant() { void dmnPrintVersion() { #ifdef TD_ENTERPRISE - char *versionStr = "enterprise"; + char *releaseName = "enterprise"; #else - char *versionStr = "community"; + char *releaseName = "community"; #endif - printf("%s version: %s compatible_version: %s\n", versionStr, version, compatible_version); + printf("%s version: %s compatible_version: %s\n", releaseName, version, compatible_version); printf("gitinfo: %s\n", gitinfo); printf("gitinfoI: %s\n", gitinfoOfInternal); printf("builuInfo: %s\n", buildinfo); @@ -164,7 +164,7 @@ int dmnRunDnode() { } int main(int argc, char const *argv[]) { - if (dmnParseOpts(argc, argv) != 0) { + if (dmnParseOption(argc, argv) != 0) { return -1; } diff --git a/source/dnode/mgmt/impl/src/dndInt.c b/source/dnode/mgmt/impl/src/dndInt.c index 9f575ddb4d..f4cee3f6fd 100644 --- a/source/dnode/mgmt/impl/src/dndInt.c +++ b/source/dnode/mgmt/impl/src/dndInt.c @@ -98,19 +98,19 @@ static int32_t dndInitEnv(SDnode *pDnode, SDnodeOpt *pOptions) { return -1; } - if (!taosMkDir(pDnode->dir.dnode)) { + if (taosMkDir(pDnode->dir.dnode) != 0) { dError("failed to create dir:%s since %s", pDnode->dir.dnode, strerror(errno)); terrno = TAOS_SYSTEM_ERROR(errno); return -1; } - if (!taosMkDir(pDnode->dir.mnode)) { + if (taosMkDir(pDnode->dir.mnode) != 0) { dError("failed to create dir:%s since %s", pDnode->dir.mnode, strerror(errno)); terrno = TAOS_SYSTEM_ERROR(errno); return -1; } - if (!taosMkDir(pDnode->dir.vnodes)) { + if (taosMkDir(pDnode->dir.vnodes) != 0) { dError("failed to create dir:%s since %s", pDnode->dir.vnodes, strerror(errno)); terrno = TAOS_SYSTEM_ERROR(errno); return -1; diff --git a/source/dnode/mnode/sdb/src/sdbFile.c b/source/dnode/mnode/sdb/src/sdbFile.c index 169b2cf809..d77ec6ef8f 100644 --- a/source/dnode/mnode/sdb/src/sdbFile.c +++ b/source/dnode/mnode/sdb/src/sdbFile.c @@ -21,19 +21,19 @@ static int32_t sdbCreateDir() { mDebug("start to create mnode at %s", tsMnodeDir); - if (!taosMkDir(tsSdb.currDir)) { + if (taosMkDir(tsSdb.currDir) != 0) { terrno = TAOS_SYSTEM_ERROR(errno); mError("failed to create dir:%s since %s", tsSdb.currDir, terrstr()); return -1; } - if (!taosMkDir(tsSdb.syncDir)) { + if (taosMkDir(tsSdb.syncDir) != 0) { terrno = TAOS_SYSTEM_ERROR(errno); mError("failed to create dir:%s since %s", tsSdb.syncDir, terrstr()); return -1; } - if (!taosMkDir(tsSdb.tmpDir)) { + if (taosMkDir(tsSdb.tmpDir) != 0) { terrno = TAOS_SYSTEM_ERROR(errno); mError("failed to create dir:%s since %s", tsSdb.tmpDir, terrstr()); return -1; diff --git a/source/dnode/vnode/tq/src/tqMetaStore.c b/source/dnode/vnode/tq/src/tqMetaStore.c index 71d1e8d890..81b48191bc 100644 --- a/source/dnode/vnode/tq/src/tqMetaStore.c +++ b/source/dnode/vnode/tq/src/tqMetaStore.c @@ -92,7 +92,7 @@ TqMetaStore* tqStoreOpen(const char* path, char name[pathLen+10]; strcpy(name, path); - if(!taosDirExist(name) && !taosMkDir(name)) { + if (taosDirExist(name) != 0 && taosMkDir(name) != 0) { ASSERT(false); } strcat(name, "/" TQ_IDX_NAME); diff --git a/source/libs/wal/src/walMgmt.c b/source/libs/wal/src/walMgmt.c index 2bc12b374c..917723b166 100644 --- a/source/libs/wal/src/walMgmt.c +++ b/source/libs/wal/src/walMgmt.c @@ -136,7 +136,7 @@ void walClose(SWal *pWal) { } static int32_t walInitObj(SWal *pWal) { - if (!taosMkDir(pWal->path)) { + if (taosMkDir(pWal->path) != 0) { wError("vgId:%d, path:%s, failed to create directory since %s", pWal->vgId, pWal->path, strerror(errno)); return TAOS_SYSTEM_ERROR(errno); } diff --git a/source/os/src/osDir.c b/source/os/src/osDir.c index bf9446c578..cfa0028925 100644 --- a/source/os/src/osDir.c +++ b/source/os/src/osDir.c @@ -58,15 +58,15 @@ void taosRemoveDir(const char *dirname) { printf("dir:%s is removed\n", dirname); } -bool taosDirExist(char *dirname) { return access(dirname, F_OK) == 0; } +int32_t taosDirExist(char *dirname) { return access(dirname, F_OK); } -bool taosMkDir(const char *dirname) { +int32_t taosMkDir(const char *dirname) { int32_t code = mkdir(dirname, 0755); if (code < 0 && errno == EEXIST) { - return true; + return 0; } - return code == 0; + return code; } void taosRemoveOldFiles(char *dirname, int32_t keepDays) { @@ -112,12 +112,12 @@ void taosRemoveOldFiles(char *dirname, int32_t keepDays) { rmdir(dirname); } -bool taosExpandDir(char *dirname, char *outname, int32_t maxlen) { +int32_t taosExpandDir(char *dirname, char *outname, int32_t maxlen) { wordexp_t full_path; if (0 != wordexp(dirname, &full_path, 0)) { printf("failed to expand path:%s since %s", dirname, strerror(errno)); wordfree(&full_path); - return false; + return -1; } if (full_path.we_wordv != NULL && full_path.we_wordv[0] != NULL) { @@ -126,16 +126,16 @@ bool taosExpandDir(char *dirname, char *outname, int32_t maxlen) { wordfree(&full_path); - return true; + return 0; } -bool taosRealPath(char *dirname, int32_t maxlen) { +int32_t taosRealPath(char *dirname, int32_t maxlen) { char tmp[PATH_MAX] = {0}; if (realpath(dirname, tmp) != NULL) { strncpy(dirname, tmp, maxlen); } - return true; + return 0; } #endif diff --git a/source/util/src/tconfig.c b/source/util/src/tconfig.c index 1ca29f798a..d249297e8b 100644 --- a/source/util/src/tconfig.c +++ b/source/util/src/tconfig.c @@ -154,7 +154,7 @@ static bool taosReadDirectoryConfig(SGlobalCfg *cfg, char *input_value) { taosExpandDir(input_value, option, cfg->ptrLength); taosRealPath(option, cfg->ptrLength); - if (!taosMkDir(option)) { + if (taosMkDir(option) != 0) { uError("config option:%s, input value:%s, directory not exist, create fail:%s", cfg->option, input_value, strerror(errno)); return false; diff --git a/src/client/src/tscSub.c b/src/client/src/tscSub.c index a0af98a9d2..6464224c5e 100644 --- a/src/client/src/tscSub.c +++ b/src/client/src/tscSub.c @@ -371,7 +371,7 @@ void tscSaveSubscriptionProgress(void* sub) { char path[256]; sprintf(path, "%s/subscribe", tsDataDir); - if (!taosMkDir(path)) { + if (taosMkDir(path) != 0) { tscError("failed to create subscribe dir: %s", path); } diff --git a/src/tfs/src/tfs.c b/src/tfs/src/tfs.c index b8e8972d93..547f862c20 100644 --- a/src/tfs/src/tfs.c +++ b/src/tfs/src/tfs.c @@ -252,7 +252,7 @@ int tfsMkdirAt(const char *rname, int level, int id) { char aname[TMPNAME_LEN]; snprintf(aname, TMPNAME_LEN, "%s/%s", DISK_DIR(pDisk), rname); - if (!taosMkDir(aname)) { + if (taosMkDir(aname) != 0) { terrno = TAOS_SYSTEM_ERROR(errno); return -1; } From 68af0bf32aa931bbe0233a69f6ffdfd68a7653ef Mon Sep 17 00:00:00 2001 From: Shengliang Guan Date: Tue, 23 Nov 2021 20:01:05 +0800 Subject: [PATCH 19/44] TD-10430 minor changes for daemon module --- include/util/tconfig.h | 8 +++--- source/dnode/mgmt/daemon/src/daemon.c | 36 ++++++++++++++++----------- source/util/src/tconfig.c | 6 ++--- src/client/src/tscSystem.c | 2 +- src/kit/shell/src/shellMain.c | 2 +- 5 files changed, 30 insertions(+), 24 deletions(-) diff --git a/include/util/tconfig.h b/include/util/tconfig.h index bc56d21b3a..15100423d5 100644 --- a/include/util/tconfig.h +++ b/include/util/tconfig.h @@ -82,10 +82,10 @@ extern SGlobalCfg tsGlobalConfig[]; extern int32_t tsGlobalConfigNum; extern char * tsCfgStatusStr[]; -void taosReadGlobalLogCfg(); -bool taosReadGlobalCfg(); -void taosPrintGlobalCfg(); -void taosDumpGlobalCfg(); +void taosReadGlobalLogCfg(); +int32_t taosReadGlobalCfg(); +void taosPrintGlobalCfg(); +void taosDumpGlobalCfg(); void taosInitConfigOption(SGlobalCfg cfg); SGlobalCfg *taosGetConfigOption(const char *option); diff --git a/source/dnode/mgmt/daemon/src/daemon.c b/source/dnode/mgmt/daemon/src/daemon.c index a57db1f778..4d1116466c 100644 --- a/source/dnode/mgmt/daemon/src/daemon.c +++ b/source/dnode/mgmt/daemon/src/daemon.c @@ -132,24 +132,30 @@ void dmnWaitSignal() { } } -void dmnInitOption(SDnodeOpt *pOpt) { - pOpt->sver = tsVersion; - pOpt->numOfCores = tsNumOfCores; - pOpt->statusInterval = tsStatusInterval; - pOpt->serverPort = tsServerPort; - tstrncpy(pOpt->localEp, tsLocalEp, TSDB_EP_LEN); - tstrncpy(pOpt->localFqdn, tsLocalEp, TSDB_FQDN_LEN); - tstrncpy(pOpt->timezone, tsLocalEp, TSDB_TIMEZONE_LEN); - tstrncpy(pOpt->locale, tsLocalEp, TSDB_LOCALE_LEN); - tstrncpy(pOpt->charset, tsLocalEp, TSDB_LOCALE_LEN); +void dmnInitOption(SDnodeOpt *pOption) { + pOption->sver = tsVersion; + pOption->numOfCores = tsNumOfCores; + pOption->numOfThreadsPerCore = tsNumOfThreadsPerCore; + pOption->ratioOfQueryCores = tsRatioOfQueryCores; + pOption->maxShellConns = tsMaxShellConns; + pOption->shellActivityTimer = tsShellActivityTimer; + pOption->statusInterval = tsStatusInterval; + pOption->serverPort = tsServerPort; + tstrncpy(pOption->dataDir, tsDataDir, TSDB_EP_LEN); + tstrncpy(pOption->localEp, tsLocalEp, TSDB_EP_LEN); + tstrncpy(pOption->localFqdn, tsLocalEp, TSDB_FQDN_LEN); + tstrncpy(pOption->firstEp, tsFirst, TSDB_FQDN_LEN); + tstrncpy(pOption->timezone, tsLocalEp, TSDB_TIMEZONE_LEN); + tstrncpy(pOption->locale, tsLocalEp, TSDB_LOCALE_LEN); + tstrncpy(pOption->charset, tsLocalEp, TSDB_LOCALE_LEN); } int dmnRunDnode() { - SDnodeOpt opt = {0}; - dmnInitOption(&opt); + SDnodeOpt option = {0}; + dmnInitOption(&option); - SDnode *pDnd = dndInit(&opt); - if (pDnd == NULL) { + SDnode *pDnode = dndInit(&option); + if (pDnode == NULL) { uInfo("Failed to start TDengine, please check the log at %s", tsLogDir); return -1; } @@ -158,7 +164,7 @@ int dmnRunDnode() { dmnWaitSignal(); uInfo("TDengine is shut down!"); - dndCleanup(pDnd); + dndCleanup(pDnode); taosCloseLog(); return 0; } diff --git a/source/util/src/tconfig.c b/source/util/src/tconfig.c index d249297e8b..8a6f389366 100644 --- a/source/util/src/tconfig.c +++ b/source/util/src/tconfig.c @@ -335,7 +335,7 @@ void taosReadGlobalLogCfg() { fclose(fp); } -bool taosReadGlobalCfg() { +int32_t taosReadGlobalCfg() { char * line, *option, *value, *value2, *value3; int olen, vlen, vlen2, vlen3; char fileName[PATH_MAX] = {0}; @@ -345,7 +345,7 @@ bool taosReadGlobalCfg() { if (fp == NULL) { fp = fopen(configDir, "r"); if (fp == NULL) { - return false; + return -1; } } @@ -393,7 +393,7 @@ bool taosReadGlobalCfg() { // taosSetAllDebugFlag(); // } - return true; + return 0; } void taosPrintGlobalCfg() { diff --git a/src/client/src/tscSystem.c b/src/client/src/tscSystem.c index 0ca8e49fa7..badc97ba35 100644 --- a/src/client/src/tscSystem.c +++ b/src/client/src/tscSystem.c @@ -148,7 +148,7 @@ void taos_init_imp(void) { } taosReadGlobalCfg(); - if (taosCheckGlobalCfg()) { + if (taosCheckGlobalCfg() != 0) { tscInitRes = -1; return; } diff --git a/src/kit/shell/src/shellMain.c b/src/kit/shell/src/shellMain.c index 5c9dc0995d..29713686ea 100644 --- a/src/kit/shell/src/shellMain.c +++ b/src/kit/shell/src/shellMain.c @@ -106,7 +106,7 @@ int main(int argc, char* argv[]) { taosInitGlobalCfg(); taosReadGlobalLogCfg(); - if (!taosReadGlobalCfg()) { + if (taosReadGlobalCfg() ! =0) { printf("TDengine read global config failed"); exit(EXIT_FAILURE); } From 4c99c6eb0ecbf7d676dee2c4f022312d71a9e8e2 Mon Sep 17 00:00:00 2001 From: Shengliang Guan Date: Wed, 24 Nov 2021 14:20:18 +0800 Subject: [PATCH 20/44] rename file --- source/dnode/mgmt/impl/src/{dndInt.c => dnode.c} | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename source/dnode/mgmt/impl/src/{dndInt.c => dnode.c} (100%) diff --git a/source/dnode/mgmt/impl/src/dndInt.c b/source/dnode/mgmt/impl/src/dnode.c similarity index 100% rename from source/dnode/mgmt/impl/src/dndInt.c rename to source/dnode/mgmt/impl/src/dnode.c From bbb280a770be2aad8f88ae76d27b9a7fe6143183 Mon Sep 17 00:00:00 2001 From: Shengliang Guan Date: Fri, 26 Nov 2021 10:26:08 +0800 Subject: [PATCH 21/44] minor changes --- source/dnode/mgmt/impl/inc/dndInt.h | 14 +++++++------- source/dnode/mgmt/impl/src/dnode.c | 10 +++++----- 2 files changed, 12 insertions(+), 12 deletions(-) diff --git a/source/dnode/mgmt/impl/inc/dndInt.h b/source/dnode/mgmt/impl/inc/dndInt.h index 966781426b..fba589c73a 100644 --- a/source/dnode/mgmt/impl/inc/dndInt.h +++ b/source/dnode/mgmt/impl/inc/dndInt.h @@ -37,12 +37,12 @@ extern "C" { extern int32_t dDebugFlag; -#define dFatal(...) { if (dDebugFlag & DEBUG_FATAL) { taosPrintLog("SRV FATAL ", 255, __VA_ARGS__); }} -#define dError(...) { if (dDebugFlag & DEBUG_ERROR) { taosPrintLog("SRV ERROR ", 255, __VA_ARGS__); }} -#define dWarn(...) { if (dDebugFlag & DEBUG_WARN) { taosPrintLog("SRV WARN ", 255, __VA_ARGS__); }} -#define dInfo(...) { if (dDebugFlag & DEBUG_INFO) { taosPrintLog("SRV ", 255, __VA_ARGS__); }} -#define dDebug(...) { if (dDebugFlag & DEBUG_DEBUG) { taosPrintLog("SRV ", dDebugFlag, __VA_ARGS__); }} -#define dTrace(...) { if (dDebugFlag & DEBUG_TRACE) { taosPrintLog("SRV ", dDebugFlag, __VA_ARGS__); }} +#define dFatal(...) { if (dDebugFlag & DEBUG_FATAL) { taosPrintLog("DND FATAL ", 255, __VA_ARGS__); }} +#define dError(...) { if (dDebugFlag & DEBUG_ERROR) { taosPrintLog("DND ERROR ", 255, __VA_ARGS__); }} +#define dWarn(...) { if (dDebugFlag & DEBUG_WARN) { taosPrintLog("DND WARN ", 255, __VA_ARGS__); }} +#define dInfo(...) { if (dDebugFlag & DEBUG_INFO) { taosPrintLog("DND ", 255, __VA_ARGS__); }} +#define dDebug(...) { if (dDebugFlag & DEBUG_DEBUG) { taosPrintLog("DND ", dDebugFlag, __VA_ARGS__); }} +#define dTrace(...) { if (dDebugFlag & DEBUG_TRACE) { taosPrintLog("DND ", dDebugFlag, __VA_ARGS__); }} typedef enum { DND_STAT_INIT, DND_STAT_RUNNING, DND_STAT_STOPPED } EStat; typedef void (*DndMsgFp)(SDnode *pDnode, SRpcMsg *pMsg, SEpSet *pEps); @@ -119,7 +119,7 @@ EStat dndGetStat(SDnode *pDnode); void dndSetStat(SDnode *pDnode, EStat stat); char *dndStatStr(EStat stat); -void dndReportStartup(SDnode *pDnode, char *name, char *desc); +void dndReportStartup(SDnode *pDnode, char *pName, char *pDesc); void dndGetStartup(SDnode *pDnode, SStartupMsg *pStartup); #ifdef __cplusplus diff --git a/source/dnode/mgmt/impl/src/dnode.c b/source/dnode/mgmt/impl/src/dnode.c index f4cee3f6fd..1261136fd3 100644 --- a/source/dnode/mgmt/impl/src/dnode.c +++ b/source/dnode/mgmt/impl/src/dnode.c @@ -20,8 +20,8 @@ #include "dndVnodes.h" #include "sync.h" #include "tcache.h" -#include "wal.h" #include "tcrc32c.h" +#include "wal.h" EStat dndGetStat(SDnode *pDnode) { return pDnode->stat; } @@ -43,10 +43,10 @@ char *dndStatStr(EStat stat) { } } -void dndReportStartup(SDnode *pDnode, char *name, char *desc) { +void dndReportStartup(SDnode *pDnode, char *pName, char *pDesc) { SStartupMsg *pStartup = &pDnode->startup; - tstrncpy(pStartup->name, name, strlen(pStartup->name)); - tstrncpy(pStartup->desc, desc, strlen(pStartup->desc)); + tstrncpy(pStartup->name, pName, TSDB_STEP_NAME_LEN); + tstrncpy(pStartup->desc, pDesc, TSDB_STEP_DESC_LEN); pStartup->finished = 0; } @@ -61,7 +61,7 @@ static int32_t dndCheckRunning(char *dataDir) { FileFd fd = taosOpenFileCreateWriteTrunc(filepath); if (fd < 0) { - dError("failed to open lock file:%s since %s, quit", filepath, strerror(errno)); + dError("failed to lock file:%s since %s, quit", filepath, strerror(errno)); terrno = TAOS_SYSTEM_ERROR(errno); return -1; } From 04ea23584f789bd5650b8f6dc845828616dd1269 Mon Sep 17 00:00:00 2001 From: Shengliang Guan Date: Fri, 26 Nov 2021 11:13:45 +0800 Subject: [PATCH 22/44] refact dnode --- include/common/tglobal.h | 4 - include/dnode/mnode/mnode.h | 2 +- include/dnode/vnode/vnode.h | 2 +- include/util/tdef.h | 6 - source/common/src/tglobal.c | 9 - source/dnode/mgmt/impl/inc/dndDnode.h | 2 +- source/dnode/mgmt/impl/inc/dndInt.h | 26 +- source/dnode/mgmt/impl/src/dndDnode.c | 411 +++++++++++++------------- source/dnode/mnode/impl/src/mnode.c | 2 +- 9 files changed, 215 insertions(+), 249 deletions(-) diff --git a/include/common/tglobal.h b/include/common/tglobal.h index f3fce8becd..f478e96766 100644 --- a/include/common/tglobal.h +++ b/include/common/tglobal.h @@ -28,10 +28,6 @@ extern char tsSecond[]; extern char tsLocalFqdn[]; extern char tsLocalEp[]; extern uint16_t tsServerPort; -extern uint16_t tsDnodeShellPort; -extern uint16_t tsDnodeDnodePort; -extern uint16_t tsSyncPort; -extern uint16_t tsArbitratorPort; extern int32_t tsStatusInterval; extern int32_t tsNumOfMnodes; extern int8_t tsEnableVnodeBak; diff --git a/include/dnode/mnode/mnode.h b/include/dnode/mnode/mnode.h index 98aefc6db3..b7b05f896f 100644 --- a/include/dnode/mnode/mnode.h +++ b/include/dnode/mnode/mnode.h @@ -26,7 +26,7 @@ typedef struct SMnode SMnode; typedef struct SMnodeMsg SMnodeMsg; typedef void (*SendMsgToDnodeFp)(SDnode *pDnd, struct SEpSet *epSet, struct SRpcMsg *rpcMsg); typedef void (*SendMsgToMnodeFp)(SDnode *pDnd, struct SRpcMsg *rpcMsg); -typedef void (*SendRedirectMsgFp)(SDnode *pDnd, struct SRpcMsg *rpcMsg, bool forShell); +typedef void (*SendRedirectMsgFp)(SDnode *pDnd, struct SRpcMsg *rpcMsg); typedef int32_t (*PutMsgToMnodeQFp)(SDnode *pDnd, SMnodeMsg *pMsg); typedef struct SMnodeLoad { diff --git a/include/dnode/vnode/vnode.h b/include/dnode/vnode/vnode.h index 586cb49d0f..8dc01b2a8a 100644 --- a/include/dnode/vnode/vnode.h +++ b/include/dnode/vnode/vnode.h @@ -187,7 +187,7 @@ typedef struct { typedef struct SDnode SDnode; typedef void (*SendMsgToDnodeFp)(SDnode *pDnd, struct SEpSet *epSet, struct SRpcMsg *rpcMsg); typedef void (*SendMsgToMnodeFp)(SDnode *pDnd, struct SRpcMsg *rpcMsg); -typedef void (*SendRedirectMsgFp)(SDnode *pDnd, struct SRpcMsg *rpcMsg, bool forShell); +typedef void (*SendRedirectMsgFp)(SDnode *pDnd, struct SRpcMsg *rpcMsg); typedef int32_t (*PutMsgToVnodeQFp)(SDnode *pDnd, int32_t vgId, SVnodeMsg *pMsg); typedef struct { diff --git a/include/util/tdef.h b/include/util/tdef.h index d61a3e7188..76df8887a0 100644 --- a/include/util/tdef.h +++ b/include/util/tdef.h @@ -358,12 +358,6 @@ do { \ #define TSDB_DEFAULT_STABLES_HASH_SIZE 100 #define TSDB_DEFAULT_CTABLES_HASH_SIZE 20000 -#define TSDB_PORT_DNODESHELL 0 -#define TSDB_PORT_DNODEDNODE 5 -#define TSDB_PORT_SYNC 10 -#define TSDB_PORT_HTTP 11 -#define TSDB_PORT_ARBITRATOR 12 - #define TSDB_MAX_WAL_SIZE (1024*1024*3) #define TSDB_ARB_DUMMY_TIME 4765104000000 // 2121-01-01 00:00:00.000, :P diff --git a/source/common/src/tglobal.c b/source/common/src/tglobal.c index 501936d354..8d57218a64 100644 --- a/source/common/src/tglobal.c +++ b/source/common/src/tglobal.c @@ -33,10 +33,6 @@ char tsArbitrator[TSDB_EP_LEN] = {0}; char tsLocalFqdn[TSDB_FQDN_LEN] = {0}; char tsLocalEp[TSDB_EP_LEN] = {0}; // Local End Point, hostname:port uint16_t tsServerPort = 6030; -uint16_t tsDnodeShellPort = 6030; // udp[6035-6039] tcp[6035] -uint16_t tsDnodeDnodePort = 6035; // udp/tcp -uint16_t tsSyncPort = 6040; -uint16_t tsArbitratorPort = 6042; int32_t tsStatusInterval = 1; // second int32_t tsNumOfMnodes = 1; int8_t tsEnableVnodeBak = 1; @@ -1726,11 +1722,6 @@ int32_t taosCheckGlobalCfg() { } } - tsDnodeShellPort = tsServerPort + TSDB_PORT_DNODESHELL; // udp[6035-6039] tcp[6035] - tsDnodeDnodePort = tsServerPort + TSDB_PORT_DNODEDNODE; // udp/tcp - tsSyncPort = tsServerPort + TSDB_PORT_SYNC; - tsHttpPort = tsServerPort + TSDB_PORT_HTTP; - if (tsQueryBufferSize >= 0) { tsQueryBufferSizeBytes = tsQueryBufferSize * 1048576UL; } diff --git a/source/dnode/mgmt/impl/inc/dndDnode.h b/source/dnode/mgmt/impl/inc/dndDnode.h index ef16b1c8f0..590a9611e1 100644 --- a/source/dnode/mgmt/impl/inc/dndDnode.h +++ b/source/dnode/mgmt/impl/inc/dndDnode.h @@ -30,7 +30,7 @@ int32_t dndGetDnodeId(SDnode *pDnd); int64_t dndGetClusterId(SDnode *pDnd); void dndGetDnodeEp(SDnode *pDnd, int32_t dnodeId, char *pEp, char *pFqdn, uint16_t *pPort); void dndGetMnodeEpSet(SDnode *pDnd, SEpSet *pEpSet); -void dndSendRedirectMsg(SDnode *pDnd, SRpcMsg *pMsg, bool forShell); +void dndSendRedirectMsg(SDnode *pDnd, SRpcMsg *pMsg); #ifdef __cplusplus } diff --git a/source/dnode/mgmt/impl/inc/dndInt.h b/source/dnode/mgmt/impl/inc/dndInt.h index fba589c73a..258f40aad6 100644 --- a/source/dnode/mgmt/impl/inc/dndInt.h +++ b/source/dnode/mgmt/impl/inc/dndInt.h @@ -31,9 +31,10 @@ extern "C" { #include "tthread.h" #include "ttime.h" #include "tworker.h" + +#include "dnode.h" #include "mnode.h" #include "vnode.h" -#include "dnode.h" extern int32_t dDebugFlag; @@ -54,17 +55,16 @@ typedef struct { } SDnodeDir; typedef struct { - int32_t dnodeId; - uint32_t rebootTime; - int32_t dropped; - int64_t clusterId; - SEpSet shellEpSet; - SEpSet peerEpSet; - char *file; - SHashObj *dnodeHash; - SDnodeEps *dnodeEps; - pthread_t *threadId; - pthread_mutex_t mutex; + int32_t dnodeId; + int32_t dropped; + uint32_t rebootTime; + int64_t clusterId; + SEpSet mnodeEpSet; + char *file; + SHashObj *dnodeHash; + SDnodeEps *dnodeEps; + pthread_t *threadId; + SRWLatch latch; } SDnodeMgmt; typedef struct { @@ -108,7 +108,7 @@ typedef struct SDnode { EStat stat; SDnodeOpt opt; SDnodeDir dir; - SDnodeMgmt d; + SDnodeMgmt dmgmt; SMnodeMgmt m; SVnodesMgmt vmgmt; STransMgmt t; diff --git a/source/dnode/mgmt/impl/src/dndDnode.c b/source/dnode/mgmt/impl/src/dndDnode.c index 378d76e046..962ce2b73d 100644 --- a/source/dnode/mgmt/impl/src/dndDnode.c +++ b/source/dnode/mgmt/impl/src/dndDnode.c @@ -18,28 +18,32 @@ #include "dndTransport.h" #include "dndVnodes.h" -static inline void dndLockDnode(SDnode *pDnd) { pthread_mutex_lock(&pDnd->d.mutex); } +static inline void dndRLockDnode(SDnode *pDnode) { taosRLockLatch(&pDnode->dmgmt.latch); } -static inline void dndUnLockDnode(SDnode *pDnd) { pthread_mutex_unlock(&pDnd->d.mutex); } +static inline void dndRUnLockDnode(SDnode *pDnode) { taosRUnLockLatch(&pDnode->dmgmt.latch); } -int32_t dndGetDnodeId(SDnode *pDnd) { - dndLockDnode(pDnd); - int32_t dnodeId = pDnd->d.dnodeId; - dndUnLockDnode(pDnd); +static inline void dndWLockDnode(SDnode *pDnode) { taosWLockLatch(&pDnode->dmgmt.latch); } + +static inline void dndWUnLockDnode(SDnode *pDnode) { taosWUnLockLatch(&pDnode->dmgmt.latch); } + +int32_t dndGetDnodeId(SDnode *pDnode) { + dndRLockDnode(pDnode); + int32_t dnodeId = pDnode->dmgmt.dnodeId; + dndRUnLockDnode(pDnode); return dnodeId; } -int64_t dndGetClusterId(SDnode *pDnd) { - dndLockDnode(pDnd); - int64_t clusterId = pDnd->d.clusterId; - dndUnLockDnode(pDnd); +int64_t dndGetClusterId(SDnode *pDnode) { + dndRLockDnode(pDnode); + int64_t clusterId = pDnode->dmgmt.clusterId; + dndRUnLockDnode(pDnode); return clusterId; } -void dndGetDnodeEp(SDnode *pDnd, int32_t dnodeId, char *pEp, char *pFqdn, uint16_t *pPort) { - dndLockDnode(pDnd); +void dndGetDnodeEp(SDnode *pDnode, int32_t dnodeId, char *pEp, char *pFqdn, uint16_t *pPort) { + dndRLockDnode(pDnode); - SDnodeEp *pDnodeEp = taosHashGet(pDnd->d.dnodeHash, &dnodeId, sizeof(int32_t)); + SDnodeEp *pDnodeEp = taosHashGet(pDnode->dmgmt.dnodeHash, &dnodeId, sizeof(int32_t)); if (pDnodeEp != NULL) { if (pPort != NULL) { *pPort = pDnodeEp->port; @@ -52,41 +56,26 @@ void dndGetDnodeEp(SDnode *pDnd, int32_t dnodeId, char *pEp, char *pFqdn, uint16 } } - dndUnLockDnode(pDnd); + dndRUnLockDnode(pDnode); } -void dndGetMnodeEpSet(SDnode *pDnd, SEpSet *pEpSet) { - dndLockDnode(pDnd); - *pEpSet = pDnd->d.peerEpSet; - dndUnLockDnode(pDnd); +void dndGetMnodeEpSet(SDnode *pDnode, SEpSet *pEpSet) { + dndRLockDnode(pDnode); + *pEpSet = pDnode->dmgmt.mnodeEpSet; + dndRUnLockDnode(pDnode); } -void dndGetShellEpSet(SDnode *pDnd, SEpSet *pEpSet) { - dndLockDnode(pDnd); - *pEpSet = pDnd->d.shellEpSet; - dndUnLockDnode(pDnd); -} - -void dndSendRedirectMsg(SDnode *pDnd, SRpcMsg *pMsg, bool forShell) { +void dndSendRedirectMsg(SDnode *pDnode, SRpcMsg *pMsg) { int32_t msgType = pMsg->msgType; SEpSet epSet = {0}; - if (forShell) { - dndGetShellEpSet(pDnd, &epSet); - } else { - dndGetMnodeEpSet(pDnd, &epSet); - } - - dDebug("RPC %p, msg:%s is redirected, num:%d use:%d", pMsg->handle, taosMsg[msgType], epSet.numOfEps, epSet.inUse); + dndGetMnodeEpSet(pDnode, &epSet); + dDebug("RPC %p, msg:%s is redirected, num:%d inUse:%d", pMsg->handle, taosMsg[msgType], epSet.numOfEps, epSet.inUse); for (int32_t i = 0; i < epSet.numOfEps; ++i) { dDebug("mnode index:%d %s:%u", i, epSet.fqdn[i], epSet.port[i]); - if (strcmp(epSet.fqdn[i], pDnd->opt.localFqdn) == 0) { - if ((epSet.port[i] == pDnd->opt.serverPort + TSDB_PORT_DNODEDNODE && !forShell) || - (epSet.port[i] == pDnd->opt.serverPort && forShell)) { - epSet.inUse = (i + 1) % epSet.numOfEps; - dDebug("mnode index:%d %s:%d set inUse to %d", i, epSet.fqdn[i], epSet.port[i], epSet.inUse); - } + if (strcmp(epSet.fqdn[i], pDnode->opt.localFqdn) == 0 && epSet.port[i] == pDnode->opt.serverPort) { + epSet.inUse = (i + 1) % epSet.numOfEps; } epSet.port[i] = htons(epSet.port[i]); @@ -96,220 +85,218 @@ void dndSendRedirectMsg(SDnode *pDnd, SRpcMsg *pMsg, bool forShell) { } static void dndUpdateMnodeEpSet(SDnode *pDnd, SEpSet *pEpSet) { - dInfo("mnode is changed, num:%d use:%d", pEpSet->numOfEps, pEpSet->inUse); + dInfo("mnode is changed, num:%d inUse:%d", pEpSet->numOfEps, pEpSet->inUse); - dndLockDnode(pDnd); + dndWLockDnode(pDnd); - pDnd->d.peerEpSet = *pEpSet; + pDnd->dmgmt.mnodeEpSet = *pEpSet; for (int32_t i = 0; i < pEpSet->numOfEps; ++i) { - pEpSet->port[i] -= TSDB_PORT_DNODEDNODE; dInfo("mnode index:%d %s:%u", i, pEpSet->fqdn[i], pEpSet->port[i]); } - pDnd->d.shellEpSet = *pEpSet; - dndUnLockDnode(pDnd); + dndWUnLockDnode(pDnd); } -static void dndPrintDnodes(SDnode *pDnd) { - SDnodeMgmt *pDnode = &pDnd->d; +static void dndPrintDnodes(SDnode *pDnode) { + SDnodeMgmt *pMgmt = &pDnode->dmgmt; - dDebug("print dnode endpoint list, num:%d", pDnode->dnodeEps->num); - for (int32_t i = 0; i < pDnode->dnodeEps->num; i++) { - SDnodeEp *pEp = &pDnode->dnodeEps->eps[i]; + dDebug("print dnode ep list, num:%d", pMgmt->dnodeEps->num); + for (int32_t i = 0; i < pMgmt->dnodeEps->num; i++) { + SDnodeEp *pEp = &pMgmt->dnodeEps->eps[i]; dDebug("dnode:%d, fqdn:%s port:%u isMnode:%d", pEp->id, pEp->fqdn, pEp->port, pEp->isMnode); } } -static void dndResetDnodes(SDnode *pDnd, SDnodeEps *pDnodeEps) { - SDnodeMgmt *pDnode = &pDnd->d; +static void dndResetDnodes(SDnode *pDnode, SDnodeEps *pDnodeEps) { + SDnodeMgmt *pMgmt = &pDnode->dmgmt; int32_t size = sizeof(SDnodeEps) + pDnodeEps->num * sizeof(SDnodeEp); - - if (pDnodeEps->num > pDnode->dnodeEps->num) { + if (pDnodeEps->num > pMgmt->dnodeEps->num) { SDnodeEps *tmp = calloc(1, size); if (tmp == NULL) return; - tfree(pDnode->dnodeEps); - pDnode->dnodeEps = tmp; + tfree(pMgmt->dnodeEps); + pMgmt->dnodeEps = tmp; } - if (pDnode->dnodeEps != pDnodeEps) { - memcpy(pDnode->dnodeEps, pDnodeEps, size); + if (pMgmt->dnodeEps != pDnodeEps) { + memcpy(pMgmt->dnodeEps, pDnodeEps, size); } - pDnode->peerEpSet.inUse = 0; - pDnode->shellEpSet.inUse = 0; + pMgmt->mnodeEpSet.inUse = 0; int32_t mIndex = 0; - for (int32_t i = 0; i < pDnode->dnodeEps->num; i++) { - SDnodeEp *pDnodeEp = &pDnode->dnodeEps->eps[i]; + for (int32_t i = 0; i < pMgmt->dnodeEps->num; i++) { + SDnodeEp *pDnodeEp = &pMgmt->dnodeEps->eps[i]; if (!pDnodeEp->isMnode) continue; if (mIndex >= TSDB_MAX_REPLICA) continue; - strcpy(pDnode->shellEpSet.fqdn[mIndex], pDnodeEp->fqdn); - strcpy(pDnode->peerEpSet.fqdn[mIndex], pDnodeEp->fqdn); - pDnode->shellEpSet.port[mIndex] = pDnodeEp->port; - pDnode->shellEpSet.port[mIndex] = pDnodeEp->port + TSDB_PORT_DNODEDNODE; + strcpy(pMgmt->mnodeEpSet.fqdn[mIndex], pDnodeEp->fqdn); + pMgmt->mnodeEpSet.port[mIndex] = pDnodeEp->port; mIndex++; } - for (int32_t i = 0; i < pDnode->dnodeEps->num; ++i) { - SDnodeEp *pDnodeEp = &pDnode->dnodeEps->eps[i]; - taosHashPut(pDnode->dnodeHash, &pDnodeEp->id, sizeof(int32_t), pDnodeEp, sizeof(SDnodeEp)); + for (int32_t i = 0; i < pMgmt->dnodeEps->num; ++i) { + SDnodeEp *pDnodeEp = &pMgmt->dnodeEps->eps[i]; + taosHashPut(pMgmt->dnodeHash, &pDnodeEp->id, sizeof(int32_t), pDnodeEp, sizeof(SDnodeEp)); } - dndPrintDnodes(pDnd); + dndPrintDnodes(pDnode); } -static bool dndIsEpChanged(SDnode *pDnd, int32_t dnodeId) { +static bool dndIsEpChanged(SDnode *pDnode, int32_t dnodeId, char *pEp) { bool changed = false; - dndLockDnode(pDnd); + dndRLockDnode(pDnode); - SDnodeEp *pDnodeEp = taosHashGet(pDnd->d.dnodeHash, &dnodeId, sizeof(int32_t)); + SDnodeEp *pDnodeEp = taosHashGet(pDnode->dmgmt.dnodeHash, &dnodeId, sizeof(int32_t)); if (pDnodeEp != NULL) { char epstr[TSDB_EP_LEN + 1]; snprintf(epstr, TSDB_EP_LEN, "%s:%u", pDnodeEp->fqdn, pDnodeEp->port); - changed = strcmp(pDnd->opt.localEp, epstr) != 0; + changed = strcmp(pEp, epstr) != 0; } - dndUnLockDnode(pDnd); + dndRUnLockDnode(pDnode); return changed; } -static int32_t dndReadDnodes(SDnode *pDnd) { - SDnodeMgmt *pDnode = &pDnd->d; +static int32_t dndReadDnodes(SDnode *pDnode) { + SDnodeMgmt *pMgmt = &pDnode->dmgmt; + int32_t code = TSDB_CODE_DND_DNODE_READ_FILE_ERROR; int32_t len = 0; int32_t maxLen = 30000; char *content = calloc(1, maxLen + 1); cJSON *root = NULL; FILE *fp = NULL; - fp = fopen(pDnode->file, "r"); - if (!fp) { - dDebug("file %s not exist", pDnode->file); + fp = fopen(pMgmt->file, "r"); + if (fp == NULL) { + dDebug("file %s not exist", pMgmt->file); + code = 0; goto PRASE_DNODE_OVER; } len = (int32_t)fread(content, 1, maxLen, fp); if (len <= 0) { - dError("failed to read %s since content is null", pDnode->file); + dError("failed to read %s since content is null", pMgmt->file); goto PRASE_DNODE_OVER; } content[len] = 0; root = cJSON_Parse(content); if (root == NULL) { - dError("failed to read %s since invalid json format", pDnode->file); + dError("failed to read %s since invalid json format", pMgmt->file); goto PRASE_DNODE_OVER; } cJSON *dnodeId = cJSON_GetObjectItem(root, "dnodeId"); if (!dnodeId || dnodeId->type != cJSON_String) { - dError("failed to read %s since dnodeId not found", pDnode->file); + dError("failed to read %s since dnodeId not found", pMgmt->file); goto PRASE_DNODE_OVER; } - pDnode->dnodeId = atoi(dnodeId->valuestring); + pMgmt->dnodeId = atoi(dnodeId->valuestring); cJSON *clusterId = cJSON_GetObjectItem(root, "clusterId"); if (!clusterId || clusterId->type != cJSON_String) { - dError("failed to read %s since clusterId not found", pDnode->file); + dError("failed to read %s since clusterId not found", pMgmt->file); goto PRASE_DNODE_OVER; } - pDnode->clusterId = atoll(clusterId->valuestring); + pMgmt->clusterId = atoll(clusterId->valuestring); cJSON *dropped = cJSON_GetObjectItem(root, "dropped"); if (!dropped || dropped->type != cJSON_String) { - dError("failed to read %s since dropped not found", pDnode->file); + dError("failed to read %s since dropped not found", pMgmt->file); goto PRASE_DNODE_OVER; } - pDnode->dropped = atoi(dropped->valuestring); + pMgmt->dropped = atoi(dropped->valuestring); cJSON *dnodeInfos = cJSON_GetObjectItem(root, "dnodeInfos"); if (!dnodeInfos || dnodeInfos->type != cJSON_Array) { - dError("failed to read %s since dnodeInfos not found", pDnode->file); + dError("failed to read %s since dnodeInfos not found", pMgmt->file); goto PRASE_DNODE_OVER; } int32_t dnodeInfosSize = cJSON_GetArraySize(dnodeInfos); if (dnodeInfosSize <= 0) { - dError("failed to read %s since dnodeInfos size:%d invalid", pDnode->file, dnodeInfosSize); + dError("failed to read %s since dnodeInfos size:%d invalid", pMgmt->file, dnodeInfosSize); goto PRASE_DNODE_OVER; } - pDnode->dnodeEps = calloc(1, dnodeInfosSize * sizeof(SDnodeEp) + sizeof(SDnodeEps)); - if (pDnode->dnodeEps == NULL) { + pMgmt->dnodeEps = calloc(1, dnodeInfosSize * sizeof(SDnodeEp) + sizeof(SDnodeEps)); + if (pMgmt->dnodeEps == NULL) { dError("failed to calloc dnodeEpList since %s", strerror(errno)); goto PRASE_DNODE_OVER; } - pDnode->dnodeEps->num = dnodeInfosSize; + pMgmt->dnodeEps->num = dnodeInfosSize; for (int32_t i = 0; i < dnodeInfosSize; ++i) { cJSON *dnodeInfo = cJSON_GetArrayItem(dnodeInfos, i); if (dnodeInfo == NULL) break; - SDnodeEp *pDnodeEp = &pDnode->dnodeEps->eps[i]; + SDnodeEp *pDnodeEp = &pMgmt->dnodeEps->eps[i]; cJSON *dnodeId = cJSON_GetObjectItem(dnodeInfo, "dnodeId"); if (!dnodeId || dnodeId->type != cJSON_String) { - dError("failed to read %s, dnodeId not found", pDnode->file); + dError("failed to read %s, dnodeId not found", pMgmt->file); goto PRASE_DNODE_OVER; } pDnodeEp->id = atoi(dnodeId->valuestring); cJSON *isMnode = cJSON_GetObjectItem(dnodeInfo, "isMnode"); if (!isMnode || isMnode->type != cJSON_String) { - dError("failed to read %s, isMnode not found", pDnode->file); + dError("failed to read %s, isMnode not found", pMgmt->file); goto PRASE_DNODE_OVER; } pDnodeEp->isMnode = atoi(isMnode->valuestring); cJSON *dnodeFqdn = cJSON_GetObjectItem(dnodeInfo, "dnodeFqdn"); if (!dnodeFqdn || dnodeFqdn->type != cJSON_String || dnodeFqdn->valuestring == NULL) { - dError("failed to read %s, dnodeFqdn not found", pDnode->file); + dError("failed to read %s, dnodeFqdn not found", pMgmt->file); goto PRASE_DNODE_OVER; } tstrncpy(pDnodeEp->fqdn, dnodeFqdn->valuestring, TSDB_FQDN_LEN); cJSON *dnodePort = cJSON_GetObjectItem(dnodeInfo, "dnodePort"); if (!dnodePort || dnodePort->type != cJSON_String) { - dError("failed to read %s, dnodePort not found", pDnode->file); + dError("failed to read %s, dnodePort not found", pMgmt->file); goto PRASE_DNODE_OVER; } pDnodeEp->port = atoi(dnodePort->valuestring); } - dInfo("succcessed to read file %s", pDnode->file); - dndPrintDnodes(pDnd); + code = 0; + dInfo("succcessed to read file %s", pMgmt->file); + dndPrintDnodes(pDnode); PRASE_DNODE_OVER: if (content != NULL) free(content); if (root != NULL) cJSON_Delete(root); if (fp != NULL) fclose(fp); - if (dndIsEpChanged(pDnd, pDnode->dnodeId)) { - dError("localEp %s different with %s and need reconfigured", pDnd->opt.localEp, pDnode->file); + if (dndIsEpChanged(pDnode, pMgmt->dnodeId, pDnode->opt.localEp)) { + dError("localEp %s different with %s and need reconfigured", pDnode->opt.localEp, pMgmt->file); return -1; } - if (pDnode->dnodeEps == NULL) { - pDnode->dnodeEps = calloc(1, sizeof(SDnodeEps) + sizeof(SDnodeEp)); - pDnode->dnodeEps->num = 1; - pDnode->dnodeEps->eps[0].port = pDnd->opt.serverPort; - tstrncpy(pDnode->dnodeEps->eps[0].fqdn, pDnd->opt.localFqdn, TSDB_FQDN_LEN); + if (pMgmt->dnodeEps == NULL) { + pMgmt->dnodeEps = calloc(1, sizeof(SDnodeEps) + sizeof(SDnodeEp)); + pMgmt->dnodeEps->num = 1; + pMgmt->dnodeEps->eps[0].port = pDnode->opt.serverPort; + tstrncpy(pMgmt->dnodeEps->eps[0].fqdn, pDnode->opt.localFqdn, TSDB_FQDN_LEN); } - dndResetDnodes(pDnd, pDnode->dnodeEps); + dndResetDnodes(pDnode, pMgmt->dnodeEps); terrno = 0; return 0; } -static int32_t dndWriteDnodes(SDnode *pDnd) { - SDnodeMgmt *pDnode = &pDnd->d; +static int32_t dndWriteDnodes(SDnode *pDnode) { + SDnodeMgmt *pMgmt = &pDnode->dmgmt; - FILE *fp = fopen(pDnode->file, "w"); - if (!fp) { - dError("failed to write %s since %s", pDnode->file, strerror(errno)); + FILE *fp = fopen(pMgmt->file, "w"); + if (fp == NULL) { + dError("failed to write %s since %s", pMgmt->file, strerror(errno)); + terrno = TAOS_SYSTEM_ERROR(errno); return -1; } @@ -318,17 +305,17 @@ static int32_t dndWriteDnodes(SDnode *pDnd) { char *content = calloc(1, maxLen + 1); len += snprintf(content + len, maxLen - len, "{\n"); - len += snprintf(content + len, maxLen - len, " \"dnodeId\": \"%d\",\n", pDnode->dnodeId); - len += snprintf(content + len, maxLen - len, " \"clusterId\": \"%" PRId64 "\",\n", pDnode->clusterId); - len += snprintf(content + len, maxLen - len, " \"dropped\": \"%d\",\n", pDnode->dropped); + len += snprintf(content + len, maxLen - len, " \"dnodeId\": \"%d\",\n", pMgmt->dnodeId); + len += snprintf(content + len, maxLen - len, " \"clusterId\": \"%" PRId64 "\",\n", pMgmt->clusterId); + len += snprintf(content + len, maxLen - len, " \"dropped\": \"%d\",\n", pMgmt->dropped); len += snprintf(content + len, maxLen - len, " \"dnodeInfos\": [{\n"); - for (int32_t i = 0; i < pDnode->dnodeEps->num; ++i) { - SDnodeEp *pDnodeEp = &pDnode->dnodeEps->eps[i]; + for (int32_t i = 0; i < pMgmt->dnodeEps->num; ++i) { + SDnodeEp *pDnodeEp = &pMgmt->dnodeEps->eps[i]; len += snprintf(content + len, maxLen - len, " \"dnodeId\": \"%d\",\n", pDnodeEp->id); len += snprintf(content + len, maxLen - len, " \"isMnode\": \"%d\",\n", pDnodeEp->isMnode); len += snprintf(content + len, maxLen - len, " \"dnodeFqdn\": \"%s\",\n", pDnodeEp->fqdn); len += snprintf(content + len, maxLen - len, " \"dnodePort\": \"%u\"\n", pDnodeEp->port); - if (i < pDnode->dnodeEps->num - 1) { + if (i < pMgmt->dnodeEps->num - 1) { len += snprintf(content + len, maxLen - len, " },{\n"); } else { len += snprintf(content + len, maxLen - len, " }]\n"); @@ -342,105 +329,105 @@ static int32_t dndWriteDnodes(SDnode *pDnd) { free(content); terrno = 0; - dInfo("successed to write %s", pDnode->file); + dInfo("successed to write %s", pMgmt->file); return 0; } -static void dndSendStatusMsg(SDnode *pDnd) { - int32_t contLen = sizeof(SStatusMsg) + TSDB_MAX_VNODES * sizeof(SVnodeLoad); +static void dndSendStatusMsg(SDnode *pDnode) { + int32_t contLen = sizeof(SStatusMsg) + TSDB_MAX_VNODES * sizeof(SVnodeLoad); + SStatusMsg *pStatus = rpcMallocCont(contLen); if (pStatus == NULL) { dError("failed to malloc status message"); return; } - dndLockDnode(pDnd); - pStatus->sversion = htonl(pDnd->opt.sver); - pStatus->dnodeId = htonl(pDnd->d.dnodeId); - pStatus->clusterId = htobe64(pDnd->d.clusterId); - pStatus->rebootTime = htonl(pDnd->d.rebootTime); - pStatus->numOfCores = htonl(pDnd->opt.numOfCores); - tstrncpy(pStatus->dnodeEp, pDnd->opt.localEp, TSDB_EP_LEN); - pStatus->clusterCfg.statusInterval = htonl(pDnd->opt.statusInterval); - tstrncpy(pStatus->clusterCfg.timezone, pDnd->opt.timezone, TSDB_TIMEZONE_LEN); - tstrncpy(pStatus->clusterCfg.locale, pDnd->opt.locale, TSDB_LOCALE_LEN); - tstrncpy(pStatus->clusterCfg.charset, pDnd->opt.charset, TSDB_LOCALE_LEN); + dndRLockDnode(pDnode); + pStatus->sversion = htonl(pDnode->opt.sver); + pStatus->dnodeId = htonl(pDnode->dmgmt.dnodeId); + pStatus->clusterId = htobe64(pDnode->dmgmt.clusterId); + pStatus->rebootTime = htonl(pDnode->dmgmt.rebootTime); + pStatus->numOfCores = htonl(pDnode->opt.numOfCores); + tstrncpy(pStatus->dnodeEp, pDnode->opt.localEp, TSDB_EP_LEN); + pStatus->clusterCfg.statusInterval = htonl(pDnode->opt.statusInterval); + tstrncpy(pStatus->clusterCfg.timezone, pDnode->opt.timezone, TSDB_TIMEZONE_LEN); + tstrncpy(pStatus->clusterCfg.locale, pDnode->opt.locale, TSDB_LOCALE_LEN); + tstrncpy(pStatus->clusterCfg.charset, pDnode->opt.charset, TSDB_LOCALE_LEN); pStatus->clusterCfg.checkTime = 0; char timestr[32] = "1970-01-01 00:00:00.00"; (void)taosParseTime(timestr, &pStatus->clusterCfg.checkTime, (int32_t)strlen(timestr), TSDB_TIME_PRECISION_MILLI, 0); - dndUnLockDnode(pDnd); + dndRUnLockDnode(pDnode); - dndGetVnodeLoads(pDnd, &pStatus->vnodeLoads); + dndGetVnodeLoads(pDnode, &pStatus->vnodeLoads); contLen = sizeof(SStatusMsg) + pStatus->vnodeLoads.num * sizeof(SVnodeLoad); SRpcMsg rpcMsg = {.pCont = pStatus, .contLen = contLen, .msgType = TSDB_MSG_TYPE_STATUS}; - dndSendMsgToMnode(pDnd, &rpcMsg); + dndSendMsgToMnode(pDnode, &rpcMsg); } -static void dndUpdateDnodeCfg(SDnode *pDnd, SDnodeCfg *pCfg) { - SDnodeMgmt *pDnode = &pDnd->d; - if (pDnode->dnodeId != 0 && pDnode->dropped != pCfg->dropped) return; +static void dndUpdateDnodeCfg(SDnode *pDnode, SDnodeCfg *pCfg) { + SDnodeMgmt *pMgmt = &pDnode->dmgmt; + if (pMgmt->dnodeId == 0 || pMgmt->dropped != pCfg->dropped) { + dInfo("set dnodeId:%d clusterId:%" PRId64 " dropped:%d", pCfg->dnodeId, pCfg->clusterId, pCfg->dropped); - dndLockDnode(pDnd); - - pDnode->dnodeId = pCfg->dnodeId; - pDnode->clusterId = pCfg->clusterId; - pDnode->dropped = pCfg->dropped; - dInfo("set dnodeId:%d clusterId:%" PRId64 " dropped:%d", pCfg->dnodeId, pCfg->clusterId, pCfg->dropped); - - dndWriteDnodes(pDnd); - dndUnLockDnode(pDnd); + dndWLockDnode(pDnode); + pMgmt->dnodeId = pCfg->dnodeId; + pMgmt->clusterId = pCfg->clusterId; + pMgmt->dropped = pCfg->dropped; + (void)dndWriteDnodes(pDnode); + dndWUnLockDnode(pDnode); + } } -static void dndUpdateDnodeEps(SDnode *pDnd, SDnodeEps *pDnodeEps) { +static void dndUpdateDnodeEps(SDnode *pDnode, SDnodeEps *pDnodeEps) { if (pDnodeEps == NULL || pDnodeEps->num <= 0) return; - dndLockDnode(pDnd); + dndWLockDnode(pDnode); - if (pDnodeEps->num != pDnd->d.dnodeEps->num) { - dndResetDnodes(pDnd, pDnodeEps); - dndWriteDnodes(pDnd); + if (pDnodeEps->num != pDnode->dmgmt.dnodeEps->num) { + dndResetDnodes(pDnode, pDnodeEps); + dndWriteDnodes(pDnode); } else { int32_t size = pDnodeEps->num * sizeof(SDnodeEp) + sizeof(SDnodeEps); - if (memcmp(pDnd->d.dnodeEps, pDnodeEps, size) != 0) { - dndResetDnodes(pDnd, pDnodeEps); - dndWriteDnodes(pDnd); + if (memcmp(pDnode->dmgmt.dnodeEps, pDnodeEps, size) != 0) { + dndResetDnodes(pDnode, pDnodeEps); + dndWriteDnodes(pDnode); } } - dndUnLockDnode(pDnd); + dndWUnLockDnode(pDnode); } -static void dndProcessStatusRsp(SDnode *pDnd, SRpcMsg *pMsg, SEpSet *pEpSet) { +static void dndProcessStatusRsp(SDnode *pDnode, SRpcMsg *pMsg, SEpSet *pEpSet) { if (pEpSet && pEpSet->numOfEps > 0) { - dndUpdateMnodeEpSet(pDnd, pEpSet); + dndUpdateMnodeEpSet(pDnode, pEpSet); } if (pMsg->code != TSDB_CODE_SUCCESS) return; - SStatusRsp *pStatusRsp = pMsg->pCont; - SDnodeCfg *pCfg = &pStatusRsp->dnodeCfg; + SStatusRsp *pRsp = pMsg->pCont; + SDnodeCfg *pCfg = &pRsp->dnodeCfg; pCfg->dnodeId = htonl(pCfg->dnodeId); pCfg->clusterId = htobe64(pCfg->clusterId); - dndUpdateDnodeCfg(pDnd, pCfg); + dndUpdateDnodeCfg(pDnode, pCfg); if (pCfg->dropped) return; - SDnodeEps *pDnodeEps = &pStatusRsp->dnodeEps; + SDnodeEps *pDnodeEps = &pRsp->dnodeEps; pDnodeEps->num = htonl(pDnodeEps->num); for (int32_t i = 0; i < pDnodeEps->num; ++i) { pDnodeEps->eps[i].id = htonl(pDnodeEps->eps[i].id); pDnodeEps->eps[i].port = htons(pDnodeEps->eps[i].port); } - dndUpdateDnodeEps(pDnd, pDnodeEps); + dndUpdateDnodeEps(pDnode, pDnodeEps); } -static void dndProcessAuthRsp(SDnode *pDnd, SRpcMsg *pMsg, SEpSet *pEpSet) { assert(1); } +static void dndProcessAuthRsp(SDnode *pDnode, SRpcMsg *pMsg, SEpSet *pEpSet) { assert(1); } -static void dndProcessGrantRsp(SDnode *pDnd, SRpcMsg *pMsg, SEpSet *pEpSet) { assert(1); } +static void dndProcessGrantRsp(SDnode *pDnode, SRpcMsg *pMsg, SEpSet *pEpSet) { assert(1); } -static void dndProcessConfigDnodeReq(SDnode *pDnd, SRpcMsg *pMsg) { +static void dndProcessConfigDnodeReq(SDnode *pDnode, SRpcMsg *pMsg) { SCfgDnodeMsg *pCfg = pMsg->pCont; int32_t code = TSDB_CODE_OPS_NOT_SUPPORT; @@ -449,11 +436,11 @@ static void dndProcessConfigDnodeReq(SDnode *pDnd, SRpcMsg *pMsg) { rpcFreeCont(pMsg->pCont); } -static void dndProcessStartupReq(SDnode *pDnd, SRpcMsg *pMsg) { +static void dndProcessStartupReq(SDnode *pDnode, SRpcMsg *pMsg) { dInfo("startup msg is received"); SStartupMsg *pStartup = rpcMallocCont(sizeof(SStartupMsg)); - dndGetStartup(pDnd, pStartup); + dndGetStartup(pDnode, pStartup); dInfo("startup msg is sent, step:%s desc:%s finished:%d", pStartup->name, pStartup->desc, pStartup->finished); @@ -463,52 +450,52 @@ static void dndProcessStartupReq(SDnode *pDnd, SRpcMsg *pMsg) { } static void *dnodeThreadRoutine(void *param) { - SDnode *pDnd = param; - int32_t ms = pDnd->opt.statusInterval * 1000; + SDnode *pDnode = param; + int32_t ms = pDnode->opt.statusInterval * 1000; while (true) { taosMsleep(ms); - if (dndGetStat(pDnd) != DND_STAT_RUNNING) { + if (dndGetStat(pDnode) != DND_STAT_RUNNING) { continue; } pthread_testcancel(); - dndSendStatusMsg(pDnd); + dndSendStatusMsg(pDnode); } } -int32_t dndInitDnode(SDnode *pDnd) { - SDnodeMgmt *pDnode = &pDnd->d; +int32_t dndInitDnode(SDnode *pDnode) { + SDnodeMgmt *pMgmt = &pDnode->dmgmt; - pDnode->dnodeId = 0; - pDnode->rebootTime = taosGetTimestampSec(); - pDnode->dropped = 0; - pDnode->clusterId = 0; + pMgmt->dnodeId = 0; + pMgmt->rebootTime = taosGetTimestampSec(); + pMgmt->dropped = 0; + pMgmt->clusterId = 0; char path[PATH_MAX]; - snprintf(path, PATH_MAX, "%s/dnode.json", pDnd->dir.dnode); - pDnode->file = strdup(path); - if (pDnode->file == NULL) { + snprintf(path, PATH_MAX, "%s/dnode.json", pDnode->dir.dnode); + pMgmt->file = strdup(path); + if (pMgmt->file == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; return -1; } - pDnode->dnodeHash = taosHashInit(4, taosGetDefaultHashFunction(TSDB_DATA_TYPE_INT), true, HASH_ENTRY_LOCK); - if (pDnode->dnodeHash == NULL) { + pMgmt->dnodeHash = taosHashInit(4, taosGetDefaultHashFunction(TSDB_DATA_TYPE_INT), true, HASH_ENTRY_LOCK); + if (pMgmt->dnodeHash == NULL) { dError("failed to init dnode hash"); terrno = TSDB_CODE_OUT_OF_MEMORY; return -1; } - if (dndReadDnodes(pDnd) != 0) { - dError("failed to read file:%s since %s", pDnode->file, terrstr()); + if (dndReadDnodes(pDnode) != 0) { + dError("failed to read file:%s since %s", pMgmt->file, terrstr()); return -1; } - pthread_mutex_init(&pDnode->mutex, NULL); + taosInitRWLatch(&pMgmt->latch); - pDnode->threadId = taosCreateThread(dnodeThreadRoutine, pDnd); - if (pDnode->threadId == NULL) { + pMgmt->threadId = taosCreateThread(dnodeThreadRoutine, pDnode); + if (pMgmt->threadId == NULL) { dError("failed to init dnode thread"); terrno = TSDB_CODE_OUT_OF_MEMORY; return -1; @@ -518,44 +505,42 @@ int32_t dndInitDnode(SDnode *pDnd) { return 0; } -void dndCleanupDnode(SDnode *pDnd) { - SDnodeMgmt *pDnode = &pDnd->d; +void dndCleanupDnode(SDnode *pDnode) { + SDnodeMgmt *pMgmt = &pDnode->dmgmt; - if (pDnode->threadId != NULL) { - taosDestoryThread(pDnode->threadId); - pDnode->threadId = NULL; + if (pMgmt->threadId != NULL) { + taosDestoryThread(pMgmt->threadId); + pMgmt->threadId = NULL; } - dndLockDnode(pDnd); + dndWLockDnode(pDnode); - if (pDnode->dnodeEps != NULL) { - free(pDnode->dnodeEps); - pDnode->dnodeEps = NULL; + if (pMgmt->dnodeEps != NULL) { + free(pMgmt->dnodeEps); + pMgmt->dnodeEps = NULL; } - if (pDnode->dnodeHash != NULL) { - taosHashCleanup(pDnode->dnodeHash); - pDnode->dnodeHash = NULL; + if (pMgmt->dnodeHash != NULL) { + taosHashCleanup(pMgmt->dnodeHash); + pMgmt->dnodeHash = NULL; } - if (pDnode->file != NULL) { - free(pDnode->file); - pDnode->file = NULL; + if (pMgmt->file != NULL) { + free(pMgmt->file); + pMgmt->file = NULL; } - dndUnLockDnode(pDnd); - pthread_mutex_destroy(&pDnode->mutex); - + dndWUnLockDnode(pDnode); dInfo("dnd-dnode is cleaned up"); } -void dndProcessDnodeReq(SDnode *pDnd, SRpcMsg *pMsg, SEpSet *pEpSet) { +void dndProcessDnodeReq(SDnode *pDnode, SRpcMsg *pMsg, SEpSet *pEpSet) { switch (pMsg->msgType) { case TSDB_MSG_TYPE_NETWORK_TEST: - dndProcessStartupReq(pDnd, pMsg); + dndProcessStartupReq(pDnode, pMsg); break; case TSDB_MSG_TYPE_CONFIG_DNODE_IN: - dndProcessConfigDnodeReq(pDnd, pMsg); + dndProcessConfigDnodeReq(pDnode, pMsg); break; default: dError("RPC %p, dnode req:%s not processed", pMsg->handle, taosMsg[pMsg->msgType]); @@ -565,16 +550,16 @@ void dndProcessDnodeReq(SDnode *pDnd, SRpcMsg *pMsg, SEpSet *pEpSet) { } } -void dndProcessDnodeRsp(SDnode *pDnd, SRpcMsg *pMsg, SEpSet *pEpSet) { +void dndProcessDnodeRsp(SDnode *pDnode, SRpcMsg *pMsg, SEpSet *pEpSet) { switch (pMsg->msgType) { case TSDB_MSG_TYPE_STATUS_RSP: - dndProcessStatusRsp(pDnd, pMsg, pEpSet); + dndProcessStatusRsp(pDnode, pMsg, pEpSet); break; case TSDB_MSG_TYPE_AUTH_RSP: - dndProcessAuthRsp(pDnd, pMsg, pEpSet); + dndProcessAuthRsp(pDnode, pMsg, pEpSet); break; case TSDB_MSG_TYPE_GRANT_RSP: - dndProcessGrantRsp(pDnd, pMsg, pEpSet); + dndProcessGrantRsp(pDnode, pMsg, pEpSet); break; default: dError("RPC %p, dnode rsp:%s not processed", pMsg->handle, taosMsg[pMsg->msgType]); diff --git a/source/dnode/mnode/impl/src/mnode.c b/source/dnode/mnode/impl/src/mnode.c index 5eff8a37ca..bd89476cef 100644 --- a/source/dnode/mnode/impl/src/mnode.c +++ b/source/dnode/mnode/impl/src/mnode.c @@ -53,7 +53,7 @@ void mnodeSendMsgToMnode(SMnode *pMnode, struct SRpcMsg *rpcMsg) { void mnodeSendRedirectMsg(SMnode *pMnode, struct SRpcMsg *rpcMsg, bool forShell) { assert(pMnode); - (*pMnode->sendRedirectMsgFp)(pMnode->pServer, rpcMsg, forShell); + (*pMnode->sendRedirectMsgFp)(pMnode->pServer, rpcMsg); } static int32_t mnodeInitTimer() { From b6c28a4914f2418194b1c79c8a85af0c971f7887 Mon Sep 17 00:00:00 2001 From: Shengliang Guan Date: Fri, 26 Nov 2021 11:42:56 +0800 Subject: [PATCH 23/44] update authentications --- source/dnode/mgmt/impl/inc/dndInt.h | 4 +- source/dnode/mgmt/impl/src/dndDnode.c | 8 +-- source/dnode/mgmt/impl/src/dndMnode.c | 76 +++++++++++----------- source/dnode/mgmt/impl/src/dndTransport.c | 78 +++++++++++++++++------ source/dnode/mnode/impl/src/mnodeAuth.c | 11 ---- 5 files changed, 101 insertions(+), 76 deletions(-) diff --git a/source/dnode/mgmt/impl/inc/dndInt.h b/source/dnode/mgmt/impl/inc/dndInt.h index 258f40aad6..4094871bcd 100644 --- a/source/dnode/mgmt/impl/inc/dndInt.h +++ b/source/dnode/mgmt/impl/inc/dndInt.h @@ -109,9 +109,9 @@ typedef struct SDnode { SDnodeOpt opt; SDnodeDir dir; SDnodeMgmt dmgmt; - SMnodeMgmt m; + SMnodeMgmt mmgmt; SVnodesMgmt vmgmt; - STransMgmt t; + STransMgmt tmgmt; SStartupMsg startup; } SDnode; diff --git a/source/dnode/mgmt/impl/src/dndDnode.c b/source/dnode/mgmt/impl/src/dndDnode.c index 962ce2b73d..fcafaf828f 100644 --- a/source/dnode/mgmt/impl/src/dndDnode.c +++ b/source/dnode/mgmt/impl/src/dndDnode.c @@ -84,17 +84,17 @@ void dndSendRedirectMsg(SDnode *pDnode, SRpcMsg *pMsg) { rpcSendRedirectRsp(pMsg->handle, &epSet); } -static void dndUpdateMnodeEpSet(SDnode *pDnd, SEpSet *pEpSet) { +static void dndUpdateMnodeEpSet(SDnode *pDnode, SEpSet *pEpSet) { dInfo("mnode is changed, num:%d inUse:%d", pEpSet->numOfEps, pEpSet->inUse); - dndWLockDnode(pDnd); + dndWLockDnode(pDnode); - pDnd->dmgmt.mnodeEpSet = *pEpSet; + pDnode->dmgmt.mnodeEpSet = *pEpSet; for (int32_t i = 0; i < pEpSet->numOfEps; ++i) { dInfo("mnode index:%d %s:%u", i, pEpSet->fqdn[i], pEpSet->port[i]); } - dndWUnLockDnode(pDnd); + dndWUnLockDnode(pDnode); } static void dndPrintDnodes(SDnode *pDnode) { diff --git a/source/dnode/mgmt/impl/src/dndMnode.c b/source/dnode/mgmt/impl/src/dndMnode.c index 5f3e48d8a1..d581c7761c 100644 --- a/source/dnode/mgmt/impl/src/dndMnode.c +++ b/source/dnode/mgmt/impl/src/dndMnode.c @@ -67,7 +67,7 @@ static int32_t dndProcessAlterMnodeReq(SDnode *pDnode, SRpcMsg *pRpcMsg); static int32_t dndProcessDropMnodeReq(SDnode *pDnode, SRpcMsg *pRpcMsg); static SMnode *dndAcquireMnode(SDnode *pDnode) { - SMnodeMgmt *pMgmt = &pDnode->m; + SMnodeMgmt *pMgmt = &pDnode->mmgmt; SMnode *pMnode = NULL; int32_t refCount = 0; @@ -85,7 +85,7 @@ static SMnode *dndAcquireMnode(SDnode *pDnode) { } static void dndReleaseMnode(SDnode *pDnode, SMnode *pMnode) { - SMnodeMgmt *pMgmt = &pDnode->m; + SMnodeMgmt *pMgmt = &pDnode->mmgmt; int32_t refCount = 0; taosRLockLatch(&pMgmt->latch); @@ -98,7 +98,7 @@ static void dndReleaseMnode(SDnode *pDnode, SMnode *pMnode) { } static int32_t dndReadMnodeFile(SDnode *pDnode) { - SMnodeMgmt *pMgmt = &pDnode->m; + SMnodeMgmt *pMgmt = &pDnode->mmgmt; int32_t code = TSDB_CODE_DND_MNODE_READ_FILE_ERROR; int32_t len = 0; int32_t maxLen = 300; @@ -152,7 +152,7 @@ PRASE_MNODE_OVER: } static int32_t dndWriteMnodeFile(SDnode *pDnode) { - SMnodeMgmt *pMgmt = &pDnode->m; + SMnodeMgmt *pMgmt = &pDnode->mmgmt; char file[PATH_MAX + 20] = {0}; snprintf(file, sizeof(file), "%s.bak", pMgmt->file); @@ -212,7 +212,7 @@ static int32_t dndStartMnodeWorker(SDnode *pDnode) { } static void dndStopMnodeWorker(SDnode *pDnode) { - SMnodeMgmt *pMgmt = &pDnode->m; + SMnodeMgmt *pMgmt = &pDnode->mmgmt; taosWLockLatch(&pMgmt->latch); pMgmt->deployed = 0; @@ -296,7 +296,7 @@ static int32_t dndBuildMnodeOptions(SDnode *pDnode, SMnodeOptions *pOptions, SCr } static int32_t dndOpenMnode(SDnode *pDnode, SMnodeOptions *pOptions) { - SMnodeMgmt *pMgmt = &pDnode->m; + SMnodeMgmt *pMgmt = &pDnode->mmgmt; int32_t code = dndStartMnodeWorker(pDnode); if (code != 0) { @@ -332,7 +332,7 @@ static int32_t dndOpenMnode(SDnode *pDnode, SMnodeOptions *pOptions) { } static int32_t dndAlterMnode(SDnode *pDnode, SMnodeOptions *pOptions) { - SMnodeMgmt *pMgmt = &pDnode->m; + SMnodeMgmt *pMgmt = &pDnode->mmgmt; SMnode *pMnode = dndAcquireMnode(pDnode); if (pMnode == NULL) { @@ -351,7 +351,7 @@ static int32_t dndAlterMnode(SDnode *pDnode, SMnodeOptions *pOptions) { } static int32_t dndDropMnode(SDnode *pDnode) { - SMnodeMgmt *pMgmt = &pDnode->m; + SMnodeMgmt *pMgmt = &pDnode->mmgmt; SMnode *pMnode = dndAcquireMnode(pDnode); if (pMnode == NULL) { @@ -458,7 +458,7 @@ static void dndProcessMnodeMgmtQueue(SDnode *pDnode, SRpcMsg *pMsg) { } static void dndProcessMnodeReadQueue(SDnode *pDnode, SMnodeMsg *pMsg) { - SMnodeMgmt *pMgmt = &pDnode->m; + SMnodeMgmt *pMgmt = &pDnode->mmgmt; SMnode *pMnode = dndAcquireMnode(pDnode); if (pMnode != NULL) { @@ -472,7 +472,7 @@ static void dndProcessMnodeReadQueue(SDnode *pDnode, SMnodeMsg *pMsg) { } static void dndProcessMnodeWriteQueue(SDnode *pDnode, SMnodeMsg *pMsg) { - SMnodeMgmt *pMgmt = &pDnode->m; + SMnodeMgmt *pMgmt = &pDnode->mmgmt; SMnode *pMnode = dndAcquireMnode(pDnode); if (pMnode != NULL) { @@ -486,7 +486,7 @@ static void dndProcessMnodeWriteQueue(SDnode *pDnode, SMnodeMsg *pMsg) { } static void dndProcessMnodeApplyQueue(SDnode *pDnode, SMnodeMsg *pMsg) { - SMnodeMgmt *pMgmt = &pDnode->m; + SMnodeMgmt *pMgmt = &pDnode->mmgmt; SMnode *pMnode = dndAcquireMnode(pDnode); if (pMnode != NULL) { @@ -500,7 +500,7 @@ static void dndProcessMnodeApplyQueue(SDnode *pDnode, SMnodeMsg *pMsg) { } static void dndProcessMnodeSyncQueue(SDnode *pDnode, SMnodeMsg *pMsg) { - SMnodeMgmt *pMgmt = &pDnode->m; + SMnodeMgmt *pMgmt = &pDnode->mmgmt; SMnode *pMnode = dndAcquireMnode(pDnode); if (pMnode != NULL) { @@ -532,7 +532,7 @@ static int32_t dndWriteMnodeMsgToQueue(SMnode *pMnode, taos_queue pQueue, SRpcMs } void dndProcessMnodeMgmtMsg(SDnode *pDnode, SRpcMsg *pRpcMsg, SEpSet *pEpSet) { - SMnodeMgmt *pMgmt = &pDnode->m; + SMnodeMgmt *pMgmt = &pDnode->mmgmt; SMnode *pMnode = dndAcquireMnode(pDnode); SRpcMsg *pMsg = taosAllocateQitem(sizeof(SRpcMsg)); @@ -545,7 +545,7 @@ void dndProcessMnodeMgmtMsg(SDnode *pDnode, SRpcMsg *pRpcMsg, SEpSet *pEpSet) { } void dndProcessMnodeWriteMsg(SDnode *pDnode, SRpcMsg *pMsg, SEpSet *pEpSet) { - SMnodeMgmt *pMgmt = &pDnode->m; + SMnodeMgmt *pMgmt = &pDnode->mmgmt; SMnode *pMnode = dndAcquireMnode(pDnode); if (pMnode == NULL || dndWriteMnodeMsgToQueue(pMnode, pMgmt->pWriteQ, pMsg) != 0) { SRpcMsg rsp = {.handle = pMsg->handle, .code = terrno}; @@ -557,7 +557,7 @@ void dndProcessMnodeWriteMsg(SDnode *pDnode, SRpcMsg *pMsg, SEpSet *pEpSet) { } void dndProcessMnodeSyncMsg(SDnode *pDnode, SRpcMsg *pMsg, SEpSet *pEpSet) { - SMnodeMgmt *pMgmt = &pDnode->m; + SMnodeMgmt *pMgmt = &pDnode->mmgmt; SMnode *pMnode = dndAcquireMnode(pDnode); if (pMnode == NULL || dndWriteMnodeMsgToQueue(pMnode, pMgmt->pSyncQ, pMsg) != 0) { SRpcMsg rsp = {.handle = pMsg->handle, .code = terrno}; @@ -569,7 +569,7 @@ void dndProcessMnodeSyncMsg(SDnode *pDnode, SRpcMsg *pMsg, SEpSet *pEpSet) { } void dndProcessMnodeReadMsg(SDnode *pDnode, SRpcMsg *pMsg, SEpSet *pEpSet) { - SMnodeMgmt *pMgmt = &pDnode->m; + SMnodeMgmt *pMgmt = &pDnode->mmgmt; SMnode *pMnode = dndAcquireMnode(pDnode); if (pMnode == NULL || dndWriteMnodeMsgToQueue(pMnode, pMgmt->pSyncQ, pMsg) != 0) { SRpcMsg rsp = {.handle = pMsg->handle, .code = terrno}; @@ -581,7 +581,7 @@ void dndProcessMnodeReadMsg(SDnode *pDnode, SRpcMsg *pMsg, SEpSet *pEpSet) { } static int32_t dndPutMsgIntoMnodeApplyQueue(SDnode *pDnode, SMnodeMsg *pMsg) { - SMnodeMgmt *pMgmt = &pDnode->m; + SMnodeMgmt *pMgmt = &pDnode->mmgmt; SMnode *pMnode = dndAcquireMnode(pDnode); if (pMnode == NULL) { @@ -594,7 +594,7 @@ static int32_t dndPutMsgIntoMnodeApplyQueue(SDnode *pDnode, SMnodeMsg *pMsg) { } static int32_t dndAllocMnodeMgmtQueue(SDnode *pDnode) { - SMnodeMgmt *pMgmt = &pDnode->m; + SMnodeMgmt *pMgmt = &pDnode->mmgmt; pMgmt->pMgmtQ = tWorkerAllocQueue(&pMgmt->mgmtPool, NULL, (FProcessItem)dndProcessMnodeMgmtQueue); if (pMgmt->pMgmtQ == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; @@ -604,13 +604,13 @@ static int32_t dndAllocMnodeMgmtQueue(SDnode *pDnode) { } static void dndFreeMnodeMgmtQueue(SDnode *pDnode) { - SMnodeMgmt *pMgmt = &pDnode->m; + SMnodeMgmt *pMgmt = &pDnode->mmgmt; tWorkerFreeQueue(&pMgmt->mgmtPool, pMgmt->pMgmtQ); pMgmt->pMgmtQ = NULL; } static int32_t dndInitMnodeMgmtWorker(SDnode *pDnode) { - SMnodeMgmt *pMgmt = &pDnode->m; + SMnodeMgmt *pMgmt = &pDnode->mmgmt; SWorkerPool *pPool = &pMgmt->mgmtPool; pPool->name = "mnode-mgmt"; pPool->min = 1; @@ -624,13 +624,13 @@ static int32_t dndInitMnodeMgmtWorker(SDnode *pDnode) { } static void dndCleanupMnodeMgmtWorker(SDnode *pDnode) { - SMnodeMgmt *pMgmt = &pDnode->m; + SMnodeMgmt *pMgmt = &pDnode->mmgmt; ; tWorkerCleanup(&pMgmt->mgmtPool); } static int32_t dndAllocMnodeReadQueue(SDnode *pDnode) { - SMnodeMgmt *pMgmt = &pDnode->m; + SMnodeMgmt *pMgmt = &pDnode->mmgmt; pMgmt->pReadQ = tWorkerAllocQueue(&pMgmt->readPool, NULL, (FProcessItem)dndProcessMnodeReadQueue); if (pMgmt->pReadQ == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; @@ -640,13 +640,13 @@ static int32_t dndAllocMnodeReadQueue(SDnode *pDnode) { } static void dndFreeMnodeReadQueue(SDnode *pDnode) { - SMnodeMgmt *pMgmt = &pDnode->m; + SMnodeMgmt *pMgmt = &pDnode->mmgmt; tWorkerFreeQueue(&pMgmt->readPool, pMgmt->pReadQ); pMgmt->pReadQ = NULL; } static int32_t dndInitMnodeReadWorker(SDnode *pDnode) { - SMnodeMgmt *pMgmt = &pDnode->m; + SMnodeMgmt *pMgmt = &pDnode->mmgmt; SWorkerPool *pPool = &pMgmt->readPool; pPool->name = "mnode-read"; pPool->min = 0; @@ -660,12 +660,12 @@ static int32_t dndInitMnodeReadWorker(SDnode *pDnode) { } static void dndCleanupMnodeReadWorker(SDnode *pDnode) { - SMnodeMgmt *pMgmt = &pDnode->m; + SMnodeMgmt *pMgmt = &pDnode->mmgmt; tWorkerCleanup(&pMgmt->readPool); } static int32_t dndAllocMnodeWriteQueue(SDnode *pDnode) { - SMnodeMgmt *pMgmt = &pDnode->m; + SMnodeMgmt *pMgmt = &pDnode->mmgmt; pMgmt->pWriteQ = tWorkerAllocQueue(&pMgmt->writePool, NULL, (FProcessItem)dndProcessMnodeWriteQueue); if (pMgmt->pWriteQ == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; @@ -675,13 +675,13 @@ static int32_t dndAllocMnodeWriteQueue(SDnode *pDnode) { } static void dndFreeMnodeWriteQueue(SDnode *pDnode) { - SMnodeMgmt *pMgmt = &pDnode->m; + SMnodeMgmt *pMgmt = &pDnode->mmgmt; tWorkerFreeQueue(&pMgmt->writePool, pMgmt->pWriteQ); pMgmt->pWriteQ = NULL; } static int32_t dndAllocMnodeApplyQueue(SDnode *pDnode) { - SMnodeMgmt *pMgmt = &pDnode->m; + SMnodeMgmt *pMgmt = &pDnode->mmgmt; pMgmt->pApplyQ = tWorkerAllocQueue(&pMgmt->writePool, NULL, (FProcessItem)dndProcessMnodeApplyQueue); if (pMgmt->pApplyQ == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; @@ -691,13 +691,13 @@ static int32_t dndAllocMnodeApplyQueue(SDnode *pDnode) { } static void dndFreeMnodeApplyQueue(SDnode *pDnode) { - SMnodeMgmt *pMgmt = &pDnode->m; + SMnodeMgmt *pMgmt = &pDnode->mmgmt; tWorkerFreeQueue(&pMgmt->writePool, pMgmt->pApplyQ); pMgmt->pApplyQ = NULL; } static int32_t dndInitMnodeWriteWorker(SDnode *pDnode) { - SMnodeMgmt *pMgmt = &pDnode->m; + SMnodeMgmt *pMgmt = &pDnode->mmgmt; SWorkerPool *pPool = &pMgmt->writePool; pPool->name = "mnode-write"; pPool->min = 0; @@ -711,12 +711,12 @@ static int32_t dndInitMnodeWriteWorker(SDnode *pDnode) { } static void dndCleanupMnodeWriteWorker(SDnode *pDnode) { - SMnodeMgmt *pMgmt = &pDnode->m; + SMnodeMgmt *pMgmt = &pDnode->mmgmt; tWorkerCleanup(&pMgmt->writePool); } static int32_t dndAllocMnodeSyncQueue(SDnode *pDnode) { - SMnodeMgmt *pMgmt = &pDnode->m; + SMnodeMgmt *pMgmt = &pDnode->mmgmt; pMgmt->pSyncQ = tWorkerAllocQueue(&pMgmt->syncPool, NULL, (FProcessItem)dndProcessMnodeSyncQueue); if (pMgmt->pSyncQ == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; @@ -726,13 +726,13 @@ static int32_t dndAllocMnodeSyncQueue(SDnode *pDnode) { } static void dndFreeMnodeSyncQueue(SDnode *pDnode) { - SMnodeMgmt *pMgmt = &pDnode->m; + SMnodeMgmt *pMgmt = &pDnode->mmgmt; tWorkerFreeQueue(&pMgmt->syncPool, pMgmt->pSyncQ); pMgmt->pSyncQ = NULL; } static int32_t dndInitMnodeSyncWorker(SDnode *pDnode) { - SMnodeMgmt *pMgmt = &pDnode->m; + SMnodeMgmt *pMgmt = &pDnode->mmgmt; SWorkerPool *pPool = &pMgmt->syncPool; pPool->name = "mnode-sync"; pPool->min = 0; @@ -741,13 +741,13 @@ static int32_t dndInitMnodeSyncWorker(SDnode *pDnode) { } static void dndCleanupMnodeSyncWorker(SDnode *pDnode) { - SMnodeMgmt *pMgmt = &pDnode->m; + SMnodeMgmt *pMgmt = &pDnode->mmgmt; tWorkerCleanup(&pMgmt->syncPool); } int32_t dndInitMnode(SDnode *pDnode) { dInfo("dnode-mnode start to init"); - SMnodeMgmt *pMgmt = &pDnode->m; + SMnodeMgmt *pMgmt = &pDnode->mmgmt; taosInitRWLatch(&pMgmt->latch); if (dndInitMnodeMgmtWorker(pDnode) != 0) { @@ -791,7 +791,7 @@ int32_t dndInitMnode(SDnode *pDnode) { } void dndCleanupMnode(SDnode *pDnode) { - SMnodeMgmt *pMgmt = &pDnode->m; + SMnodeMgmt *pMgmt = &pDnode->mmgmt; dInfo("dnode-mnode start to clean up"); dndStopMnodeWorker(pDnode); @@ -801,7 +801,7 @@ void dndCleanupMnode(SDnode *pDnode) { } int32_t dndGetUserAuthFromMnode(SDnode *pDnode, char *user, char *spi, char *encrypt, char *secret, char *ckey) { - SMnodeMgmt *pMgmt = &pDnode->m; + SMnodeMgmt *pMgmt = &pDnode->mmgmt; SMnode *pMnode = dndAcquireMnode(pDnode); if (pMnode == NULL) { diff --git a/source/dnode/mgmt/impl/src/dndTransport.c b/source/dnode/mgmt/impl/src/dndTransport.c index 679e3ef5f9..d5f52bac8b 100644 --- a/source/dnode/mgmt/impl/src/dndTransport.c +++ b/source/dnode/mgmt/impl/src/dndTransport.c @@ -25,6 +25,10 @@ #include "dndMnode.h" #include "dndVnodes.h" +#define INTERNAL_USER "_internal" +#define INTERNAL_CKEY "_key" +#define INTERNAL_SECRET "_secret" + static void dndInitMsgFp(STransMgmt *pMgmt) { // msg from client to dnode pMgmt->msgFp[TSDB_MSG_TYPE_SUBMIT] = dndProcessVnodeWriteMsg; @@ -121,7 +125,7 @@ static void dndInitMsgFp(STransMgmt *pMgmt) { static void dndProcessResponse(void *parent, SRpcMsg *pMsg, SEpSet *pEpSet) { SDnode *pDnode = parent; - STransMgmt *pMgmt = &pDnode->t; + STransMgmt *pMgmt = &pDnode->tmgmt; int32_t msgType = pMsg->msgType; @@ -143,19 +147,19 @@ static void dndProcessResponse(void *parent, SRpcMsg *pMsg, SEpSet *pEpSet) { } static int32_t dndInitClient(SDnode *pDnode) { - STransMgmt *pMgmt = &pDnode->t; + STransMgmt *pMgmt = &pDnode->tmgmt; SRpcInit rpcInit; memset(&rpcInit, 0, sizeof(rpcInit)); rpcInit.label = "DND-C"; rpcInit.numOfThreads = 1; rpcInit.cfp = dndProcessResponse; - rpcInit.sessions = TSDB_MAX_VNODES << 4; + rpcInit.sessions = 8; rpcInit.connType = TAOS_CONN_CLIENT; rpcInit.idleTime = pDnode->opt.shellActivityTimer * 1000; - rpcInit.user = "-internal"; - rpcInit.ckey = "-key"; - rpcInit.secret = "-secret"; + rpcInit.user = INTERNAL_USER; + rpcInit.ckey = INTERNAL_CKEY; + rpcInit.secret = INTERNAL_SECRET; pMgmt->clientRpc = rpcOpen(&rpcInit); if (pMgmt->clientRpc == NULL) { @@ -167,7 +171,7 @@ static int32_t dndInitClient(SDnode *pDnode) { } static void dndCleanupClient(SDnode *pDnode) { - STransMgmt *pMgmt = &pDnode->t; + STransMgmt *pMgmt = &pDnode->tmgmt; if (pMgmt->clientRpc) { rpcClose(pMgmt->clientRpc); pMgmt->clientRpc = NULL; @@ -176,8 +180,8 @@ static void dndCleanupClient(SDnode *pDnode) { } static void dndProcessRequest(void *param, SRpcMsg *pMsg, SEpSet *pEpSet) { - SDnode *pDnode = param; - STransMgmt *pMgmt = &pDnode->t; + SDnode *pDnode = param; + STransMgmt *pMgmt = &pDnode->tmgmt; int32_t msgType = pMsg->msgType; if (msgType == TSDB_MSG_TYPE_NETWORK_TEST) { @@ -218,24 +222,56 @@ static void dndProcessRequest(void *param, SRpcMsg *pMsg, SEpSet *pEpSet) { } static void dndSendMsgToMnodeRecv(SDnode *pDnode, SRpcMsg *pRpcMsg, SRpcMsg *pRpcRsp) { - STransMgmt *pMgmt = &pDnode->t; + STransMgmt *pMgmt = &pDnode->tmgmt; SEpSet epSet = {0}; dndGetMnodeEpSet(pDnode, &epSet); rpcSendRecv(pMgmt->clientRpc, &epSet, pRpcMsg, pRpcRsp); } +static int32_t dndAuthInternalMsg(SDnode *pDnode, char *user, char *spi, char *encrypt, char *secret, char *ckey) { + if (strcmp(user, INTERNAL_USER) == 0) { + // A simple temporary implementation + char pass[32] = {0}; + taosEncryptPass((uint8_t *)(INTERNAL_SECRET), strlen(INTERNAL_SECRET), pass); + memcpy(secret, pass, TSDB_KEY_LEN); + *spi = 0; + *encrypt = 0; + *ckey = 0; + return 0; + } else if (strcmp(user, TSDB_NETTEST_USER) == 0) { + // A simple temporary implementation + char pass[32] = {0}; + taosEncryptPass((uint8_t *)(TSDB_NETTEST_USER), strlen(TSDB_NETTEST_USER), pass); + memcpy(secret, pass, TSDB_KEY_LEN); + *spi = 0; + *encrypt = 0; + *ckey = 0; + return 0; + } else { + return -1; + } +} + static int32_t dndRetrieveUserAuthInfo(void *parent, char *user, char *spi, char *encrypt, char *secret, char *ckey) { SDnode *pDnode = parent; - if (dndGetUserAuthFromMnode(pDnode, user, spi, encrypt, secret, ckey) != 0) { - if (terrno != TSDB_CODE_APP_NOT_READY) { - dTrace("failed to get user auth from mnode since %s", terrstr()); - return -1; - } + if (dndAuthInternalMsg(parent, user, spi, encrypt, secret, ckey) == 0) { + dTrace("get internal auth success"); + return 0; } - dDebug("user:%s, send auth msg to mnodes", user); + if (dndGetUserAuthFromMnode(pDnode, user, spi, encrypt, secret, ckey) == 0) { + dTrace("get auth from internal mnode"); + return 0; + } + + if (terrno != TSDB_CODE_APP_NOT_READY) { + dTrace("failed to get user auth from internal mnode since %s", terrstr()); + return -1; + } + + dDebug("user:%s, send auth msg to other mnodes", user); SAuthMsg *pMsg = rpcMallocCont(sizeof(SAuthMsg)); tstrncpy(pMsg->user, user, TSDB_USER_LEN); @@ -246,14 +282,14 @@ static int32_t dndRetrieveUserAuthInfo(void *parent, char *user, char *spi, char if (rpcRsp.code != 0) { terrno = rpcRsp.code; - dError("user:%s, failed to get user auth from mnodes since %s", user, terrstr()); + dError("user:%s, failed to get user auth from other mnodes since %s", user, terrstr()); } else { SAuthRsp *pRsp = rpcRsp.pCont; memcpy(secret, pRsp->secret, TSDB_KEY_LEN); memcpy(ckey, pRsp->ckey, TSDB_KEY_LEN); *spi = pRsp->spi; *encrypt = pRsp->encrypt; - dDebug("user:%s, success to get user auth from mnodes", user); + dDebug("user:%s, success to get user auth from other mnodes", user); } rpcFreeCont(rpcRsp.pCont); @@ -261,7 +297,7 @@ static int32_t dndRetrieveUserAuthInfo(void *parent, char *user, char *spi, char } static int32_t dndInitServer(SDnode *pDnode) { - STransMgmt *pMgmt = &pDnode->t; + STransMgmt *pMgmt = &pDnode->tmgmt; dndInitMsgFp(pMgmt); int32_t numOfThreads = (int32_t)((pDnode->opt.numOfCores * pDnode->opt.numOfThreadsPerCore) / 2.0); @@ -290,7 +326,7 @@ static int32_t dndInitServer(SDnode *pDnode) { } static void dndCleanupServer(SDnode *pDnode) { - STransMgmt *pMgmt = &pDnode->t; + STransMgmt *pMgmt = &pDnode->tmgmt; if (pMgmt->serverRpc) { rpcClose(pMgmt->serverRpc); pMgmt->serverRpc = NULL; @@ -317,7 +353,7 @@ void dndCleanupTrans(SDnode *pDnode) { } void dndSendMsgToDnode(SDnode *pDnode, SEpSet *pEpSet, SRpcMsg *pMsg) { - STransMgmt *pMgmt = &pDnode->t; + STransMgmt *pMgmt = &pDnode->tmgmt; rpcSendRequest(pMgmt->clientRpc, pEpSet, pMsg, NULL); } diff --git a/source/dnode/mnode/impl/src/mnodeAuth.c b/source/dnode/mnode/impl/src/mnodeAuth.c index ddd2b91ff3..f8e704d16d 100644 --- a/source/dnode/mnode/impl/src/mnodeAuth.c +++ b/source/dnode/mnode/impl/src/mnodeAuth.c @@ -21,16 +21,5 @@ int32_t mnodeInitAuth() { return 0; } void mnodeCleanupAuth() {} int32_t mnodeRetriveAuth(SMnode *pMnode, char *user, char *spi, char *encrypt, char *secret, char *ckey) { - if (strcmp(user, TSDB_NETTEST_USER) == 0) { - char pass[32] = {0}; - taosEncryptPass((uint8_t *)user, strlen(user), pass); - *spi = 0; - *encrypt = 0; - *ckey = 0; - memcpy(secret, pass, TSDB_KEY_LEN); - mDebug("nettest user is authorized"); - return 0; - } - return 0; } \ No newline at end of file From 0d57a4beb70e0d1192c1768b4b22d4a02534172b Mon Sep 17 00:00:00 2001 From: Shengliang Guan Date: Fri, 26 Nov 2021 14:15:22 +0800 Subject: [PATCH 24/44] TD-11265 invalid write in dnode.c --- include/dnode/mnode/mnode.h | 8 ++++---- include/dnode/vnode/vnode.h | 8 ++++---- source/dnode/mgmt/daemon/src/daemon.c | 12 ++++++------ source/dnode/mgmt/impl/inc/dndDnode.h | 18 +++++++++--------- source/dnode/mgmt/impl/src/dndDnode.c | 2 +- source/dnode/mgmt/impl/src/dnode.c | 8 ++++---- source/dnode/mnode/impl/inc/mnodeDef.h | 4 ++-- 7 files changed, 30 insertions(+), 30 deletions(-) diff --git a/include/dnode/mnode/mnode.h b/include/dnode/mnode/mnode.h index b7b05f896f..725bdaec3c 100644 --- a/include/dnode/mnode/mnode.h +++ b/include/dnode/mnode/mnode.h @@ -24,10 +24,10 @@ extern "C" { typedef struct SDnode SDnode; typedef struct SMnode SMnode; typedef struct SMnodeMsg SMnodeMsg; -typedef void (*SendMsgToDnodeFp)(SDnode *pDnd, struct SEpSet *epSet, struct SRpcMsg *rpcMsg); -typedef void (*SendMsgToMnodeFp)(SDnode *pDnd, struct SRpcMsg *rpcMsg); -typedef void (*SendRedirectMsgFp)(SDnode *pDnd, struct SRpcMsg *rpcMsg); -typedef int32_t (*PutMsgToMnodeQFp)(SDnode *pDnd, SMnodeMsg *pMsg); +typedef void (*SendMsgToDnodeFp)(SDnode *pDnode, struct SEpSet *epSet, struct SRpcMsg *rpcMsg); +typedef void (*SendMsgToMnodeFp)(SDnode *pDnode, struct SRpcMsg *rpcMsg); +typedef void (*SendRedirectMsgFp)(SDnode *pDnode, struct SRpcMsg *rpcMsg); +typedef int32_t (*PutMsgToMnodeQFp)(SDnode *pDnode, SMnodeMsg *pMsg); typedef struct SMnodeLoad { int64_t numOfDnode; diff --git a/include/dnode/vnode/vnode.h b/include/dnode/vnode/vnode.h index 8dc01b2a8a..3f6705fac6 100644 --- a/include/dnode/vnode/vnode.h +++ b/include/dnode/vnode/vnode.h @@ -185,10 +185,10 @@ typedef struct { } SVnodeMsg; typedef struct SDnode SDnode; -typedef void (*SendMsgToDnodeFp)(SDnode *pDnd, struct SEpSet *epSet, struct SRpcMsg *rpcMsg); -typedef void (*SendMsgToMnodeFp)(SDnode *pDnd, struct SRpcMsg *rpcMsg); -typedef void (*SendRedirectMsgFp)(SDnode *pDnd, struct SRpcMsg *rpcMsg); -typedef int32_t (*PutMsgToVnodeQFp)(SDnode *pDnd, int32_t vgId, SVnodeMsg *pMsg); +typedef void (*SendMsgToDnodeFp)(SDnode *pDnode, struct SEpSet *epSet, struct SRpcMsg *rpcMsg); +typedef void (*SendMsgToMnodeFp)(SDnode *pDnode, struct SRpcMsg *rpcMsg); +typedef void (*SendRedirectMsgFp)(SDnode *pDnode, struct SRpcMsg *rpcMsg); +typedef int32_t (*PutMsgToVnodeQFp)(SDnode *pDnode, int32_t vgId, SVnodeMsg *pMsg); typedef struct { PutMsgToVnodeQFp putMsgToApplyQueueFp; diff --git a/source/dnode/mgmt/daemon/src/daemon.c b/source/dnode/mgmt/daemon/src/daemon.c index 4d1116466c..429b097fb8 100644 --- a/source/dnode/mgmt/daemon/src/daemon.c +++ b/source/dnode/mgmt/daemon/src/daemon.c @@ -141,13 +141,13 @@ void dmnInitOption(SDnodeOpt *pOption) { pOption->shellActivityTimer = tsShellActivityTimer; pOption->statusInterval = tsStatusInterval; pOption->serverPort = tsServerPort; - tstrncpy(pOption->dataDir, tsDataDir, TSDB_EP_LEN); + tstrncpy(pOption->dataDir, tsDataDir, PATH_MAX); tstrncpy(pOption->localEp, tsLocalEp, TSDB_EP_LEN); - tstrncpy(pOption->localFqdn, tsLocalEp, TSDB_FQDN_LEN); - tstrncpy(pOption->firstEp, tsFirst, TSDB_FQDN_LEN); - tstrncpy(pOption->timezone, tsLocalEp, TSDB_TIMEZONE_LEN); - tstrncpy(pOption->locale, tsLocalEp, TSDB_LOCALE_LEN); - tstrncpy(pOption->charset, tsLocalEp, TSDB_LOCALE_LEN); + tstrncpy(pOption->localFqdn, tsLocalFqdn, TSDB_FQDN_LEN); + tstrncpy(pOption->firstEp, tsFirst, TSDB_EP_LEN); + tstrncpy(pOption->timezone, tsTimezone, TSDB_TIMEZONE_LEN); + tstrncpy(pOption->locale, tsLocale, TSDB_LOCALE_LEN); + tstrncpy(pOption->charset, tsCharset, TSDB_LOCALE_LEN); } int dmnRunDnode() { diff --git a/source/dnode/mgmt/impl/inc/dndDnode.h b/source/dnode/mgmt/impl/inc/dndDnode.h index 590a9611e1..4bb4cad8cc 100644 --- a/source/dnode/mgmt/impl/inc/dndDnode.h +++ b/source/dnode/mgmt/impl/inc/dndDnode.h @@ -21,16 +21,16 @@ extern "C" { #endif #include "dndInt.h" -int32_t dndInitDnode(SDnode *pDnd); -void dndCleanupDnode(SDnode *pDnd); -void dndProcessDnodeReq(SDnode *pDnd, SRpcMsg *pMsg, SEpSet *pEpSet); -void dndProcessDnodeRsp(SDnode *pDnd, SRpcMsg *pMsg, SEpSet *pEpSet); +int32_t dndInitDnode(SDnode *pDnode); +void dndCleanupDnode(SDnode *pDnode); +void dndProcessDnodeReq(SDnode *pDnode, SRpcMsg *pMsg, SEpSet *pEpSet); +void dndProcessDnodeRsp(SDnode *pDnode, SRpcMsg *pMsg, SEpSet *pEpSet); -int32_t dndGetDnodeId(SDnode *pDnd); -int64_t dndGetClusterId(SDnode *pDnd); -void dndGetDnodeEp(SDnode *pDnd, int32_t dnodeId, char *pEp, char *pFqdn, uint16_t *pPort); -void dndGetMnodeEpSet(SDnode *pDnd, SEpSet *pEpSet); -void dndSendRedirectMsg(SDnode *pDnd, SRpcMsg *pMsg); +int32_t dndGetDnodeId(SDnode *pDnode); +int64_t dndGetClusterId(SDnode *pDnode); +void dndGetDnodeEp(SDnode *pDnode, int32_t dnodeId, char *pEp, char *pFqdn, uint16_t *pPort); +void dndGetMnodeEpSet(SDnode *pDnode, SEpSet *pEpSet); +void dndSendRedirectMsg(SDnode *pDnode, SRpcMsg *pMsg); #ifdef __cplusplus } diff --git a/source/dnode/mgmt/impl/src/dndDnode.c b/source/dnode/mgmt/impl/src/dndDnode.c index fcafaf828f..ef30503494 100644 --- a/source/dnode/mgmt/impl/src/dndDnode.c +++ b/source/dnode/mgmt/impl/src/dndDnode.c @@ -480,7 +480,7 @@ int32_t dndInitDnode(SDnode *pDnode) { return -1; } - pMgmt->dnodeHash = taosHashInit(4, taosGetDefaultHashFunction(TSDB_DATA_TYPE_INT), true, HASH_ENTRY_LOCK); + pMgmt->dnodeHash = taosHashInit(4, taosGetDefaultHashFunction(TSDB_DATA_TYPE_INT), true, HASH_NO_LOCK); if (pMgmt->dnodeHash == NULL) { dError("failed to init dnode hash"); terrno = TSDB_CODE_OUT_OF_MEMORY; diff --git a/source/dnode/mgmt/impl/src/dnode.c b/source/dnode/mgmt/impl/src/dnode.c index 1261136fd3..0bfcf5d721 100644 --- a/source/dnode/mgmt/impl/src/dnode.c +++ b/source/dnode/mgmt/impl/src/dnode.c @@ -84,13 +84,13 @@ static int32_t dndInitEnv(SDnode *pDnode, SDnodeOpt *pOptions) { char path[PATH_MAX + 100]; snprintf(path, sizeof(path), "%s%smnode", pOptions->dataDir, TD_DIRSEP); - pDnode->dir.mnode = strdup(path); + pDnode->dir.mnode = tstrdup(path); snprintf(path, sizeof(path), "%s%svnode", pOptions->dataDir, TD_DIRSEP); - pDnode->dir.vnodes = strdup(path); + pDnode->dir.vnodes = tstrdup(path); snprintf(path, sizeof(path), "%s%sdnode", pOptions->dataDir, TD_DIRSEP); - pDnode->dir.dnode = strdup(path); + pDnode->dir.dnode = tstrdup(path); if (pDnode->dir.mnode == NULL || pDnode->dir.vnodes == NULL || pDnode->dir.dnode == NULL) { dError("failed to malloc dir object"); @@ -140,7 +140,7 @@ SDnode *dndInit(SDnodeOpt *pOptions) { taosBlockSIGPIPE(); taosResolveCRC(); - SDnode *pDnode = calloc(1, sizeof(pDnode)); + SDnode *pDnode = calloc(1, sizeof(SDnode)); if (pDnode == NULL) { dError("failed to create dnode object"); terrno = TSDB_CODE_OUT_OF_MEMORY; diff --git a/source/dnode/mnode/impl/inc/mnodeDef.h b/source/dnode/mnode/impl/inc/mnodeDef.h index 4b4c4abdb3..ccdba13006 100644 --- a/source/dnode/mnode/impl/inc/mnodeDef.h +++ b/source/dnode/mnode/impl/inc/mnodeDef.h @@ -131,7 +131,7 @@ typedef struct SMnodeObj { int64_t roleTime; int64_t createdTime; int64_t updateTime; - SDnodeObj *pDnd; + SDnodeObj *pDnode; } SMnodeObj; typedef struct { @@ -215,7 +215,7 @@ typedef struct SDbObj { typedef struct { int32_t dnodeId; int8_t role; - SDnodeObj *pDnd; + SDnodeObj *pDnode; } SVnodeGid; typedef struct SVgObj { From 9f75ad065284275a8478e01ed9db39c6542fa9fe Mon Sep 17 00:00:00 2001 From: Liu Jicong Date: Fri, 26 Nov 2021 14:22:16 +0800 Subject: [PATCH 25/44] refactor wal --- include/libs/wal/wal.h | 68 +++++++++----- source/libs/wal/inc/walInt.h | 3 + source/libs/wal/src/wal.c | 3 +- source/libs/wal/src/walIndex.c | 85 +++++++++++++++++ source/libs/wal/src/walMgmt.c | 55 +++++------ source/libs/wal/src/walWrite.c | 162 +++++++++++---------------------- 6 files changed, 206 insertions(+), 170 deletions(-) create mode 100644 source/libs/wal/src/walIndex.c diff --git a/include/libs/wal/wal.h b/include/libs/wal/wal.h index 94346d705e..4d5b4d977a 100644 --- a/include/libs/wal/wal.h +++ b/include/libs/wal/wal.h @@ -53,45 +53,63 @@ typedef struct { EWalType walLevel; // wal level } SWalCfg; -#define WAL_PREFIX "wal" -#define WAL_PREFIX_LEN 3 -#define WAL_REFRESH_MS 1000 -#define WAL_MAX_SIZE (TSDB_MAX_WAL_SIZE + sizeof(SWalHead) + 16) -#define WAL_SIGNATURE ((uint32_t)(0xFAFBFDFE)) -#define WAL_PATH_LEN (TSDB_FILENAME_LEN + 12) -#define WAL_FILE_LEN (WAL_PATH_LEN + 32) -#define WAL_FILE_NUM 1 // 3 +#define WAL_PREFIX "wal" +#define WAL_LOG_SUFFIX "log" +#define WAL_INDEX_SUFFIX "idx" +#define WAL_PREFIX_LEN 3 +#define WAL_REFRESH_MS 1000 +#define WAL_MAX_SIZE (TSDB_MAX_WAL_SIZE + sizeof(SWalHead) + 16) +#define WAL_SIGNATURE ((uint32_t)(0xFAFBFDFEUL)) +#define WAL_PATH_LEN (TSDB_FILENAME_LEN + 12) +#define WAL_FILE_LEN (WAL_PATH_LEN + 32) +//#define WAL_FILE_NUM 1 // 3 + +#define WAL_CUR_POS_READ_ONLY 1 +#define WAL_CUR_FILE_READ_ONLY 2 typedef struct SWal { - int64_t version; - int64_t fileId; - int64_t rId; - int64_t tfd; - int32_t vgId; - int32_t keep; - int32_t level; - int32_t fsyncPeriod; + // cfg + int32_t vgId; + int32_t fsyncPeriod; // millisecond + EWalType level; + //reference + int64_t refId; + //current tfd + int64_t curLogTfd; + int64_t curIdxTfd; + //current version + int64_t curVersion; + int64_t curOffset; + //current file version + int64_t curFileFirstVersion; + int64_t curFileLastVersion; + //wal fileset version + int64_t firstVersion; + int64_t snapshotVersion; + int64_t lastVersion; + //fsync status int32_t fsyncSeq; - int8_t stop; - int8_t reseved[3]; - char path[WAL_PATH_LEN]; - char name[WAL_FILE_LEN]; + //ctl + int32_t curStatus; pthread_mutex_t mutex; + //path + char path[WAL_PATH_LEN]; } SWal; // WAL HANDLE -typedef int32_t (*FWalWrite)(void *ahandle, void *pHead, void *pMsg); +typedef int32_t (*FWalWrite)(void *ahandle, void *pHead); // module initialization int32_t walInit(); void walCleanUp(); // handle open and ctl -SWal *walOpen(char *path, SWalCfg *pCfg); +SWal *walOpen(const char *path, SWalCfg *pCfg); +void walStop(SWal *pWal); int32_t walAlter(SWal *, SWalCfg *pCfg); void walClose(SWal *); // write -// int64_t walWriteWithMsgType(SWal*, int8_t msgType, void* body, int32_t bodyLen); +//int64_t walWriteWithMsgType(SWal*, int8_t msgType, void* body, int32_t bodyLen); int64_t walWrite(SWal *, int64_t index, void *body, int32_t bodyLen); int64_t walWriteBatch(SWal *, void **bodies, int32_t *bodyLen, int32_t batchSize); @@ -101,7 +119,8 @@ int32_t walCommit(SWal *, int64_t ver); // truncate after int32_t walRollback(SWal *, int64_t ver); // notify that previous log can be pruned safely -int32_t walPrune(SWal *, int64_t ver); +int32_t walTakeSnapshot(SWal *, int64_t ver); +//int32_t walDataCorrupted(SWal*); // read int32_t walRead(SWal *, SWalHead **, int64_t ver); @@ -111,7 +130,6 @@ int32_t walReadWithFp(SWal *, FWalWrite writeFp, int64_t verStart, int32_t readN int64_t walGetFirstVer(SWal *); int64_t walGetSnapshotVer(SWal *); int64_t walGetLastVer(SWal *); -// int32_t walDataCorrupted(SWal*); //internal int32_t walGetNextFile(SWal *pWal, int64_t *nextFileId); diff --git a/source/libs/wal/inc/walInt.h b/source/libs/wal/inc/walInt.h index f5f944b12b..42ede49c6b 100644 --- a/source/libs/wal/inc/walInt.h +++ b/source/libs/wal/inc/walInt.h @@ -22,6 +22,9 @@ extern "C" { #endif +int walRotate(SWal* pWal); +int walGetFile(SWal* pWal, int32_t version); + #ifdef __cplusplus } #endif diff --git a/source/libs/wal/src/wal.c b/source/libs/wal/src/wal.c index 05d81e0867..59f9c48814 100644 --- a/source/libs/wal/src/wal.c +++ b/source/libs/wal/src/wal.c @@ -23,11 +23,10 @@ int32_t walRollback(SWal *pWal, int64_t ver) { return 0; } -int32_t walPrune(SWal *pWal, int64_t ver) { +int32_t walTakeSnapshot(SWal *pWal, int64_t ver) { return 0; } - int32_t walRead(SWal *pWal, SWalHead **ppHead, int64_t ver) { return 0; } diff --git a/source/libs/wal/src/walIndex.c b/source/libs/wal/src/walIndex.c new file mode 100644 index 0000000000..e1fa8c72dd --- /dev/null +++ b/source/libs/wal/src/walIndex.c @@ -0,0 +1,85 @@ +/* + * Copyright (c) 2019 TAOS Data, Inc. + * + * This program is free software: you can use, redistribute, and/or modify + * it under the terms of the GNU Affero General Public License, version 3 + * or later ("AGPL"), as published by the Free Software Foundation. + * + * This program is distributed in the hope that it will be useful, but WITHOUT + * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or + * FITNESS FOR A PARTICULAR PURPOSE. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + */ + +#define _DEFAULT_SOURCE +#include "os.h" +#include "taoserror.h" +#include "tref.h" +#include "tfile.h" +#include "walInt.h" + +int walSetCurVerImpl(SWal *pWal, int64_t ver) { + //close old file + //iterate all files + //open right file + //set cur version, cur file version and cur status + return 0; +} + +int walSetCurVer(SWal *pWal, int64_t ver) { + if(ver > pWal->lastVersion + 1) { + //TODO: some records are skipped + return -1; + } + if(ver < pWal->firstVersion) { + //TODO: try to seek pruned log + return -1; + } + if(ver < pWal->snapshotVersion) { + //TODO: seek snapshotted log + } + if(ver < pWal->curFileFirstVersion || (pWal->curFileLastVersion != -1 && ver > pWal->curFileLastVersion)) { + //back up to avoid inconsistency + int64_t curVersion = pWal->curVersion; + int64_t curOffset = pWal->curOffset; + int64_t curFileFirstVersion = pWal->curFileFirstVersion; + int64_t curFileLastVersion = pWal->curFileLastVersion; + if(walSetCurVerImpl(pWal, ver) < 0) { + //TODO: errno + pWal->curVersion = curVersion; + pWal->curOffset = curOffset; + pWal->curFileFirstVersion = curFileFirstVersion; + pWal->curFileLastVersion = curFileLastVersion; + return -1; + } + } + + return 0; +} + +int walWriteIndex(SWal *pWal, int64_t ver, int64_t offset) { + int code = 0; + //get index file + if(!tfValid(pWal->curIdxTfd)) { + code = TAOS_SYSTEM_ERROR(errno); + wError("vgId:%d, file:%"PRId64".idx, failed to open since %s", pWal->vgId, pWal->curFileFirstVersion, strerror(errno)); + } + if(pWal->curVersion != ver) { + if(walSetCurVer(pWal, ver) != 0) { + //TODO: some records are skipped + return -1; + } + } + //check file checksum + //append index + return 0; +} + +int walRotateIndex(SWal *pWal) { + //check file checksum + //create new file + //switch file + return 0; +} diff --git a/source/libs/wal/src/walMgmt.c b/source/libs/wal/src/walMgmt.c index 2bc12b374c..4168c21a6e 100644 --- a/source/libs/wal/src/walMgmt.c +++ b/source/libs/wal/src/walMgmt.c @@ -21,7 +21,7 @@ #include "walInt.h" typedef struct { - int32_t refId; + int32_t refSetId; int32_t seq; int8_t stop; pthread_t thread; @@ -36,7 +36,7 @@ static void walFreeObj(void *pWal); int32_t walInit() { int32_t code = 0; - tsWal.refId = taosOpenRef(TSDB_MIN_VNODES, walFreeObj); + tsWal.refSetId = taosOpenRef(TSDB_MIN_VNODES, walFreeObj); code = pthread_mutex_init(&tsWal.mutex, NULL); if (code) { @@ -45,23 +45,23 @@ int32_t walInit() { } code = walCreateThread(); - if (code != TSDB_CODE_SUCCESS) { + if (code != 0) { wError("failed to init wal module since %s", tstrerror(code)); return code; } - wInfo("wal module is initialized, rsetId:%d", tsWal.refId); + wInfo("wal module is initialized, rsetId:%d", tsWal.refSetId); return code; } void walCleanUp() { walStopThread(); - taosCloseRef(tsWal.refId); + taosCloseRef(tsWal.refSetId); pthread_mutex_destroy(&tsWal.mutex); wInfo("wal module is cleaned up"); } -SWal *walOpen(char *path, SWalCfg *pCfg) { +SWal *walOpen(const char *path, SWalCfg *pCfg) { SWal *pWal = malloc(sizeof(SWal)); if (pWal == NULL) { terrno = TAOS_SYSTEM_ERROR(errno); @@ -69,10 +69,9 @@ SWal *walOpen(char *path, SWalCfg *pCfg) { } pWal->vgId = pCfg->vgId; - pWal->tfd = -1; - pWal->fileId = -1; + pWal->curLogTfd = -1; + /*pWal->curFileId = -1;*/ pWal->level = pCfg->walLevel; - /*pWal->keep = pCfg->keep;*/ pWal->fsyncPeriod = pCfg->fsyncPeriod; tstrncpy(pWal->path, path, sizeof(pWal->path)); pthread_mutex_init(&pWal->mutex, NULL); @@ -80,13 +79,13 @@ SWal *walOpen(char *path, SWalCfg *pCfg) { pWal->fsyncSeq = pCfg->fsyncPeriod / 1000; if (pWal->fsyncSeq <= 0) pWal->fsyncSeq = 1; - if (walInitObj(pWal) != TSDB_CODE_SUCCESS) { + if (walInitObj(pWal) != 0) { walFreeObj(pWal); return NULL; } - pWal->rId = taosAddRef(tsWal.refId, pWal); - if (pWal->rId < 0) { + pWal->refId = taosAddRef(tsWal.refSetId, pWal); + if (pWal->refId < 0) { walFreeObj(pWal); return NULL; } @@ -102,7 +101,7 @@ int32_t walAlter(SWal *pWal, SWalCfg *pCfg) { if (pWal->level == pCfg->walLevel && pWal->fsyncPeriod == pCfg->fsyncPeriod) { wDebug("vgId:%d, old walLevel:%d fsync:%d, new walLevel:%d fsync:%d not change", pWal->vgId, pWal->level, pWal->fsyncPeriod, pCfg->walLevel, pCfg->fsyncPeriod); - return TSDB_CODE_SUCCESS; + return 0; } wInfo("vgId:%d, change old walLevel:%d fsync:%d, new walLevel:%d fsync:%d", pWal->vgId, pWal->level, @@ -113,26 +112,16 @@ int32_t walAlter(SWal *pWal, SWalCfg *pCfg) { pWal->fsyncSeq = pCfg->fsyncPeriod / 1000; if (pWal->fsyncSeq <= 0) pWal->fsyncSeq = 1; - return TSDB_CODE_SUCCESS; -} - -void walStop(void *handle) { - if (handle == NULL) return; - SWal *pWal = handle; - - pthread_mutex_lock(&pWal->mutex); - pWal->stop = 1; - pthread_mutex_unlock(&pWal->mutex); - wDebug("vgId:%d, stop write wal", pWal->vgId); + return 0; } void walClose(SWal *pWal) { if (pWal == NULL) return; pthread_mutex_lock(&pWal->mutex); - tfClose(pWal->tfd); + tfClose(pWal->curLogTfd); pthread_mutex_unlock(&pWal->mutex); - taosRemoveRef(tsWal.refId, pWal->rId); + taosRemoveRef(tsWal.refSetId, pWal->refId); } static int32_t walInitObj(SWal *pWal) { @@ -142,14 +131,14 @@ static int32_t walInitObj(SWal *pWal) { } wDebug("vgId:%d, object is initialized", pWal->vgId); - return TSDB_CODE_SUCCESS; + return 0; } static void walFreeObj(void *wal) { SWal *pWal = wal; wDebug("vgId:%d, wal:%p is freed", pWal->vgId, pWal); - tfClose(pWal->tfd); + tfClose(pWal->curLogTfd); pthread_mutex_destroy(&pWal->mutex); tfree(pWal); } @@ -174,16 +163,16 @@ static void walUpdateSeq() { } static void walFsyncAll() { - SWal *pWal = taosIterateRef(tsWal.refId, 0); + SWal *pWal = taosIterateRef(tsWal.refSetId, 0); while (pWal) { if (walNeedFsync(pWal)) { wTrace("vgId:%d, do fsync, level:%d seq:%d rseq:%d", pWal->vgId, pWal->level, pWal->fsyncSeq, tsWal.seq); - int32_t code = tfFsync(pWal->tfd); + int32_t code = tfFsync(pWal->curLogTfd); if (code != 0) { - wError("vgId:%d, file:%s, failed to fsync since %s", pWal->vgId, pWal->name, strerror(code)); + wError("vgId:%d, file:%"PRId64".log, failed to fsync since %s", pWal->vgId, pWal->curFileFirstVersion, strerror(code)); } } - pWal = taosIterateRef(tsWal.refId, pWal->rId); + pWal = taosIterateRef(tsWal.refSetId, pWal->refId); } } @@ -216,7 +205,7 @@ static int32_t walCreateThread() { pthread_attr_destroy(&thAttr); wDebug("wal thread is launched, thread:0x%08" PRIx64, taosGetPthreadId(tsWal.thread)); - return TSDB_CODE_SUCCESS; + return 0; } static void walStopThread() { diff --git a/source/libs/wal/src/walWrite.c b/source/libs/wal/src/walWrite.c index 023b1c4a48..a8123f9c25 100644 --- a/source/libs/wal/src/walWrite.c +++ b/source/libs/wal/src/walWrite.c @@ -21,6 +21,7 @@ #include "tfile.h" #include "walInt.h" +#if 0 static int32_t walRestoreWalFile(SWal *pWal, void *pVnode, FWalWrite writeFp, char *name, int64_t fileId); int32_t walRenew(void *handle) { @@ -29,16 +30,16 @@ int32_t walRenew(void *handle) { SWal * pWal = handle; int32_t code = 0; - if (pWal->stop) { - wDebug("vgId:%d, do not create a new wal file", pWal->vgId); - return 0; - } + /*if (pWal->stop) {*/ + /*wDebug("vgId:%d, do not create a new wal file", pWal->vgId);*/ + /*return 0;*/ + /*}*/ pthread_mutex_lock(&pWal->mutex); - if (tfValid(pWal->tfd)) { - tfClose(pWal->tfd); - wDebug("vgId:%d, file:%s, it is closed while renew", pWal->vgId, pWal->name); + if (tfValid(pWal->logTfd)) { + tfClose(pWal->logTfd); + wDebug("vgId:%d, file:%s, it is closed while renew", pWal->vgId, pWal->logName); } /*if (pWal->keep == TAOS_WAL_KEEP) {*/ @@ -48,14 +49,14 @@ int32_t walRenew(void *handle) { /*pWal->fileId++;*/ /*}*/ - snprintf(pWal->name, sizeof(pWal->name), "%s/%s%" PRId64, pWal->path, WAL_PREFIX, pWal->fileId); - pWal->tfd = tfOpenCreateWrite(pWal->name); + snprintf(pWal->logName, sizeof(pWal->logName), "%s/%s%" PRId64, pWal->path, WAL_PREFIX, pWal->curFileId); + pWal->logTfd = tfOpenCreateWrite(pWal->logName); - if (!tfValid(pWal->tfd)) { + if (!tfValid(pWal->logTfd)) { code = TAOS_SYSTEM_ERROR(errno); - wError("vgId:%d, file:%s, failed to open since %s", pWal->vgId, pWal->name, strerror(errno)); + wError("vgId:%d, file:%s, failed to open since %s", pWal->vgId, pWal->logName, strerror(errno)); } else { - wDebug("vgId:%d, file:%s, it is created and open while renew", pWal->vgId, pWal->name); + wDebug("vgId:%d, file:%s, it is created and open while renew", pWal->vgId, pWal->logName); } pthread_mutex_unlock(&pWal->mutex); @@ -67,13 +68,13 @@ void walRemoveOneOldFile(void *handle) { SWal *pWal = handle; if (pWal == NULL) return; /*if (pWal->keep == TAOS_WAL_KEEP) return;*/ - if (!tfValid(pWal->tfd)) return; + if (!tfValid(pWal->logTfd)) return; pthread_mutex_lock(&pWal->mutex); // remove the oldest wal file int64_t oldFileId = -1; - if (walGetOldFile(pWal, pWal->fileId, WAL_FILE_NUM, &oldFileId) == 0) { + if (walGetOldFile(pWal, pWal->curFileId, WAL_FILE_NUM, &oldFileId) == 0) { char walName[WAL_FILE_LEN] = {0}; snprintf(walName, sizeof(walName), "%s/%s%" PRId64, pWal->path, WAL_PREFIX, oldFileId); @@ -95,26 +96,24 @@ void walRemoveAllOldFiles(void *handle) { pthread_mutex_lock(&pWal->mutex); - tfClose(pWal->tfd); - wDebug("vgId:%d, file:%s, it is closed before remove all wals", pWal->vgId, pWal->name); + tfClose(pWal->logTfd); + wDebug("vgId:%d, file:%s, it is closed before remove all wals", pWal->vgId, pWal->logName); while (walGetNextFile(pWal, &fileId) >= 0) { - snprintf(pWal->name, sizeof(pWal->name), "%s/%s%" PRId64, pWal->path, WAL_PREFIX, fileId); + snprintf(pWal->logName, sizeof(pWal->logName), "%s/%s%" PRId64, pWal->path, WAL_PREFIX, fileId); - if (remove(pWal->name) < 0) { - wError("vgId:%d, wal:%p file:%s, failed to remove since %s", pWal->vgId, pWal, pWal->name, strerror(errno)); + if (remove(pWal->logName) < 0) { + wError("vgId:%d, wal:%p file:%s, failed to remove since %s", pWal->vgId, pWal, pWal->logName, strerror(errno)); } else { - wInfo("vgId:%d, wal:%p file:%s, it is removed", pWal->vgId, pWal, pWal->name); + wInfo("vgId:%d, wal:%p file:%s, it is removed", pWal->vgId, pWal, pWal->logName); } } pthread_mutex_unlock(&pWal->mutex); } - -#if defined(WAL_CHECKSUM_WHOLE) +#endif static void walUpdateChecksum(SWalHead *pHead) { pHead->sver = 2; - pHead->cksum = 0; pHead->cksum = taosCalcChecksum(0, (uint8_t *)pHead, sizeof(SWalHead) + pHead->len); } @@ -130,8 +129,6 @@ static int walValidateChecksum(SWalHead *pHead) { return 0; } -#endif - int64_t walWrite(SWal *pWal, int64_t index, void *body, int32_t bodyLen) { if (pWal == NULL) return -1; @@ -143,32 +140,27 @@ int64_t walWrite(SWal *pWal, int64_t index, void *body, int32_t bodyLen) { int32_t code = 0; // no wal - if (!tfValid(pWal->tfd)) return 0; + if (!tfValid(pWal->curLogTfd)) return 0; if (pWal->level == TAOS_WAL_NOLOG) return 0; - if (pHead->version <= pWal->version) return 0; + if (pHead->version <= pWal->curVersion) return 0; pHead->signature = WAL_SIGNATURE; pHead->len = bodyLen; memcpy(pHead->cont, body, bodyLen); -#if defined(WAL_CHECKSUM_WHOLE) walUpdateChecksum(pHead); -#else - pHead->sver = 0; - taosCalcChecksumAppend(0, (uint8_t *)pHead, sizeof(SWalHead)); -#endif int32_t contLen = pHead->len + sizeof(SWalHead); pthread_mutex_lock(&pWal->mutex); - if (tfWrite(pWal->tfd, pHead, contLen) != contLen) { + if (tfWrite(pWal->curLogTfd, pHead, contLen) != contLen) { code = TAOS_SYSTEM_ERROR(errno); - wError("vgId:%d, file:%s, failed to write since %s", pWal->vgId, pWal->name, strerror(errno)); + wError("vgId:%d, file:%"PRId64".log, failed to write since %s", pWal->vgId, pWal->curFileFirstVersion, strerror(errno)); } else { - wTrace("vgId:%d, write wal, fileId:%" PRId64 " tfd:%" PRId64 " hver:%" PRId64 " wver:%" PRIu64 " len:%d", pWal->vgId, - pWal->fileId, pWal->tfd, pHead->version, pWal->version, pHead->len); - pWal->version = pHead->version; + /*wTrace("vgId:%d, write wal, fileId:%" PRId64 " tfd:%" PRId64 " hver:%" PRId64 " wver:%" PRIu64 " len:%d", pWal->vgId,*/ + /*pWal->curFileId, pWal->logTfd, pHead->version, pWal->curVersion, pHead->len);*/ + pWal->curVersion = pHead->version; } pthread_mutex_unlock(&pWal->mutex); @@ -179,16 +171,17 @@ int64_t walWrite(SWal *pWal, int64_t index, void *body, int32_t bodyLen) { } void walFsync(SWal *pWal, bool forceFsync) { - if (pWal == NULL || !tfValid(pWal->tfd)) return; + if (pWal == NULL || !tfValid(pWal->curLogTfd)) return; if (forceFsync || (pWal->level == TAOS_WAL_FSYNC && pWal->fsyncPeriod == 0)) { - wTrace("vgId:%d, fileId:%" PRId64 ", do fsync", pWal->vgId, pWal->fileId); - if (tfFsync(pWal->tfd) < 0) { - wError("vgId:%d, fileId:%" PRId64 ", fsync failed since %s", pWal->vgId, pWal->fileId, strerror(errno)); + wTrace("vgId:%d, fileId:%"PRId64".log, do fsync", pWal->vgId, pWal->curFileFirstVersion); + if (tfFsync(pWal->curLogTfd) < 0) { + wError("vgId:%d, file:%"PRId64".log, fsync failed since %s", pWal->vgId, pWal->curFileFirstVersion, strerror(errno)); } } } +#if 0 int32_t walRestore(void *handle, void *pVnode, FWalWrite writeFp) { if (handle == NULL) return -1; @@ -198,10 +191,10 @@ int32_t walRestore(void *handle, void *pVnode, FWalWrite writeFp) { int64_t fileId = -1; while ((code = walGetNextFile(pWal, &fileId)) >= 0) { - if (fileId == pWal->fileId) continue; + /*if (fileId == pWal->curFileId) continue;*/ char walName[WAL_FILE_LEN]; - snprintf(walName, sizeof(pWal->name), "%s/%s%" PRId64, pWal->path, WAL_PREFIX, fileId); + snprintf(walName, sizeof(pWal->logName), "%s/%s%" PRId64, pWal->path, WAL_PREFIX, fileId); wInfo("vgId:%d, file:%s, will be restored", pWal->vgId, walName); code = walRestoreWalFile(pWal, pVnode, writeFp, walName, fileId); @@ -210,7 +203,7 @@ int32_t walRestore(void *handle, void *pVnode, FWalWrite writeFp) { continue; } - wInfo("vgId:%d, file:%s, restore success, wver:%" PRIu64, pWal->vgId, walName, pWal->version); + wInfo("vgId:%d, file:%s, restore success, wver:%" PRIu64, pWal->vgId, walName, pWal->curVersion); count++; } @@ -222,14 +215,14 @@ int32_t walRestore(void *handle, void *pVnode, FWalWrite writeFp) { return walRenew(pWal); } else { // open the existing WAL file in append mode - pWal->fileId = 0; - snprintf(pWal->name, sizeof(pWal->name), "%s/%s%" PRId64, pWal->path, WAL_PREFIX, pWal->fileId); - pWal->tfd = tfOpenCreateWriteAppend(pWal->name); - if (!tfValid(pWal->tfd)) { - wError("vgId:%d, file:%s, failed to open since %s", pWal->vgId, pWal->name, strerror(errno)); + /*pWal->curFileId = 0;*/ + snprintf(pWal->logName, sizeof(pWal->logName), "%s/%s%" PRId64, pWal->path, WAL_PREFIX, pWal->curFileId); + pWal->logTfd = tfOpenCreateWriteAppend(pWal->logName); + if (!tfValid(pWal->logTfd)) { + wError("vgId:%d, file:%s, failed to open since %s", pWal->vgId, pWal->logName, strerror(errno)); return TAOS_SYSTEM_ERROR(errno); } - wDebug("vgId:%d, file:%s, it is created and open while restore", pWal->vgId, pWal->name); + wDebug("vgId:%d, file:%s, it is created and open while restore", pWal->vgId, pWal->logName); } return TSDB_CODE_SUCCESS; @@ -246,14 +239,15 @@ int32_t walGetWalFile(void *handle, char *fileName, int64_t *fileId) { int32_t code = walGetNextFile(pWal, fileId); if (code >= 0) { sprintf(fileName, "wal/%s%" PRId64, WAL_PREFIX, *fileId); - code = (*fileId == pWal->fileId) ? 0 : 1; + /*code = (*fileId == pWal->curFileId) ? 0 : 1;*/ } - wDebug("vgId:%d, get wal file, code:%d curId:%" PRId64 " outId:%" PRId64, pWal->vgId, code, pWal->fileId, *fileId); + wDebug("vgId:%d, get wal file, code:%d curId:%" PRId64 " outId:%" PRId64, pWal->vgId, code, pWal->curFileId, *fileId); pthread_mutex_unlock(&(pWal->mutex)); return code; } +#endif static void walFtruncate(SWal *pWal, int64_t tfd, int64_t offset) { tfFtruncate(tfd, offset); @@ -279,13 +273,6 @@ static int32_t walSkipCorruptedRecord(SWal *pWal, SWalHead *pHead, int64_t tfd, continue; } -#if defined(WAL_CHECKSUM_WHOLE) - if (pHead->sver == 0 && walValidateChecksum(pHead)) { - wInfo("vgId:%d, wal head cksum check passed, offset:%" PRId64, pWal->vgId, pos); - *offset = pos; - return TSDB_CODE_SUCCESS; - } - if (pHead->sver >= 1) { if (tfRead(tfd, pHead->cont, pHead->len) < pHead->len) { wError("vgId:%d, read to end of corrupted wal file, offset:%" PRId64, pWal->vgId, pos); @@ -298,15 +285,6 @@ static int32_t walSkipCorruptedRecord(SWal *pWal, SWalHead *pHead, int64_t tfd, return TSDB_CODE_SUCCESS; } } - -#else - if (taosCheckChecksumWhole((uint8_t *)pHead, sizeof(SWalHead))) { - wInfo("vgId:%d, wal head cksum check passed, offset:%" PRId64, pWal->vgId, pos); - *offset = pos; - return TSDB_CODE_SUCCESS; - } - -#endif } return TSDB_CODE_WAL_FILE_CORRUPTED; @@ -349,7 +327,6 @@ static int32_t walRestoreWalFile(SWal *pWal, void *pVnode, FWalWrite writeFp, ch break; } -#if defined(WAL_CHECKSUM_WHOLE) if ((pHead->sver == 0 && !walValidateChecksum(pHead)) || pHead->sver < 0 || pHead->sver > 2) { wError("vgId:%d, file:%s, wal head cksum is messed up, hver:%" PRIu64 " len:%d offset:%" PRId64, pWal->vgId, name, pHead->version, pHead->len, offset); @@ -393,50 +370,15 @@ static int32_t walRestoreWalFile(SWal *pWal, void *pVnode, FWalWrite writeFp, ch } } -#else - if (!taosCheckChecksumWhole((uint8_t *)pHead, sizeof(SWalHead))) { - wError("vgId:%d, file:%s, wal head cksum is messed up, hver:%" PRIu64 " len:%d offset:%" PRId64, pWal->vgId, name, - pHead->version, pHead->len, offset); - code = walSkipCorruptedRecord(pWal, pHead, tfd, &offset); - if (code != TSDB_CODE_SUCCESS) { - walFtruncate(pWal, tfd, offset); - break; - } - } - - if (pHead->len < 0 || pHead->len > size - sizeof(SWalHead)) { - wError("vgId:%d, file:%s, wal head len out of range, hver:%" PRIu64 " len:%d offset:%" PRId64, pWal->vgId, name, - pHead->version, pHead->len, offset); - code = walSkipCorruptedRecord(pWal, pHead, tfd, &offset); - if (code != TSDB_CODE_SUCCESS) { - walFtruncate(pWal, tfd, offset); - break; - } - } - - ret = (int32_t)tfRead(tfd, pHead->cont, pHead->len); - if (ret < 0) { - wError("vgId:%d, file:%s, failed to read wal body since %s", pWal->vgId, name, strerror(errno)); - code = TAOS_SYSTEM_ERROR(errno); - break; - } - - if (ret < pHead->len) { - wError("vgId:%d, file:%s, failed to read wal body, ret:%d len:%d", pWal->vgId, name, ret, pHead->len); - offset += sizeof(SWalHead); - continue; - } - -#endif offset = offset + sizeof(SWalHead) + pHead->len; wTrace("vgId:%d, restore wal, fileId:%" PRId64 " hver:%" PRIu64 " wver:%" PRIu64 " len:%d offset:%" PRId64, - pWal->vgId, fileId, pHead->version, pWal->version, pHead->len, offset); + pWal->vgId, fileId, pHead->version, pWal->curVersion, pHead->len, offset); - pWal->version = pHead->version; + pWal->curVersion = pHead->version; // wInfo("writeFp: %ld", offset); - (*writeFp)(pVnode, pHead, NULL); + (*writeFp)(pVnode, pHead); } tfClose(tfd); @@ -449,7 +391,7 @@ static int32_t walRestoreWalFile(SWal *pWal, void *pVnode, FWalWrite writeFp, ch uint64_t walGetVersion(SWal *pWal) { if (pWal == NULL) return 0; - return pWal->version; + return pWal->curVersion; } // Wal version in slave (dnode1) must be reset. @@ -459,7 +401,7 @@ uint64_t walGetVersion(SWal *pWal) { void walResetVersion(SWal *pWal, uint64_t newVer) { if (pWal == NULL) return; - wInfo("vgId:%d, version reset from %" PRIu64 " to %" PRIu64, pWal->vgId, pWal->version, newVer); + wInfo("vgId:%d, version reset from %" PRIu64 " to %" PRIu64, pWal->vgId, pWal->curVersion, newVer); - pWal->version = newVer; + pWal->curVersion = newVer; } From 7906e78257565977d4a5ac1f7a81ebfe267dd657 Mon Sep 17 00:00:00 2001 From: Shengliang Guan Date: Fri, 26 Nov 2021 14:43:56 +0800 Subject: [PATCH 26/44] TD-11265 refact dndVnodes --- include/dnode/mgmt/dnode.h | 2 +- include/dnode/vnode/vnode.h | 18 ----------- source/dnode/mgmt/daemon/src/daemon.c | 2 +- source/dnode/mgmt/impl/src/dndDnode.c | 13 ++++---- source/dnode/mgmt/impl/src/dndMnode.c | 6 ++-- source/dnode/mgmt/impl/src/dndVnodes.c | 44 +++++++++++++++++--------- source/dnode/mgmt/impl/src/dnode.c | 1 + source/dnode/vnode/impl/src/vnodeInt.c | 7 ++-- 8 files changed, 43 insertions(+), 50 deletions(-) diff --git a/include/dnode/mgmt/dnode.h b/include/dnode/mgmt/dnode.h index 7dd7730443..f43fe107fe 100644 --- a/include/dnode/mgmt/dnode.h +++ b/include/dnode/mgmt/dnode.h @@ -78,7 +78,7 @@ typedef struct { * @brief data file's directory. * */ - char dataDir[PATH_MAX]; + char dataDir[TSDB_FILENAME_LEN]; /** * @brief local endpoint. diff --git a/include/dnode/vnode/vnode.h b/include/dnode/vnode/vnode.h index 3f6705fac6..1edd93f509 100644 --- a/include/dnode/vnode/vnode.h +++ b/include/dnode/vnode/vnode.h @@ -184,27 +184,9 @@ typedef struct { SRpcMsg rpcMsg[]; } SVnodeMsg; -typedef struct SDnode SDnode; -typedef void (*SendMsgToDnodeFp)(SDnode *pDnode, struct SEpSet *epSet, struct SRpcMsg *rpcMsg); -typedef void (*SendMsgToMnodeFp)(SDnode *pDnode, struct SRpcMsg *rpcMsg); -typedef void (*SendRedirectMsgFp)(SDnode *pDnode, struct SRpcMsg *rpcMsg); -typedef int32_t (*PutMsgToVnodeQFp)(SDnode *pDnode, int32_t vgId, SVnodeMsg *pMsg); - -typedef struct { - PutMsgToVnodeQFp putMsgToApplyQueueFp; - SendMsgToDnodeFp sendMsgToDnodeFp; - SendMsgToMnodeFp sendMsgToMnodeFp; -} SVnodePara; - -int32_t vnodeInit(SVnodePara); -void vnodeCleanup(); - int32_t vnodeAlter(SVnode *pVnode, const SVnodeCfg *pCfg); -SVnode *vnodeCreate(int32_t vgId, const char *path, const SVnodeCfg *pCfg); -void vnodeDrop(SVnode *pVnode); int32_t vnodeCompact(SVnode *pVnode); int32_t vnodeSync(SVnode *pVnode); - int32_t vnodeGetLoad(SVnode *pVnode, SVnodeLoad *pLoad); SVnodeMsg *vnodeInitMsg(int32_t msgNum); diff --git a/source/dnode/mgmt/daemon/src/daemon.c b/source/dnode/mgmt/daemon/src/daemon.c index 429b097fb8..effaec66a8 100644 --- a/source/dnode/mgmt/daemon/src/daemon.c +++ b/source/dnode/mgmt/daemon/src/daemon.c @@ -141,7 +141,7 @@ void dmnInitOption(SDnodeOpt *pOption) { pOption->shellActivityTimer = tsShellActivityTimer; pOption->statusInterval = tsStatusInterval; pOption->serverPort = tsServerPort; - tstrncpy(pOption->dataDir, tsDataDir, PATH_MAX); + tstrncpy(pOption->dataDir, tsDataDir, TSDB_FILENAME_LEN); tstrncpy(pOption->localEp, tsLocalEp, TSDB_EP_LEN); tstrncpy(pOption->localFqdn, tsLocalFqdn, TSDB_FQDN_LEN); tstrncpy(pOption->firstEp, tsFirst, TSDB_EP_LEN); diff --git a/source/dnode/mgmt/impl/src/dndDnode.c b/source/dnode/mgmt/impl/src/dndDnode.c index ef30503494..fa601a0d99 100644 --- a/source/dnode/mgmt/impl/src/dndDnode.c +++ b/source/dnode/mgmt/impl/src/dndDnode.c @@ -455,12 +455,11 @@ static void *dnodeThreadRoutine(void *param) { while (true) { taosMsleep(ms); - if (dndGetStat(pDnode) != DND_STAT_RUNNING) { - continue; - } - pthread_testcancel(); - dndSendStatusMsg(pDnode); + + if (dndGetStat(pDnode) == DND_STAT_RUNNING) { + dndSendStatusMsg(pDnode); + } } } @@ -501,7 +500,7 @@ int32_t dndInitDnode(SDnode *pDnode) { return -1; } - dInfo("dnd-dnode is initialized"); + dInfo("dnode-dnode is initialized"); return 0; } @@ -531,7 +530,7 @@ void dndCleanupDnode(SDnode *pDnode) { } dndWUnLockDnode(pDnode); - dInfo("dnd-dnode is cleaned up"); + dInfo("dnode-dnode is cleaned up"); } void dndProcessDnodeReq(SDnode *pDnode, SRpcMsg *pMsg, SEpSet *pEpSet) { diff --git a/source/dnode/mgmt/impl/src/dndMnode.c b/source/dnode/mgmt/impl/src/dndMnode.c index d581c7761c..9b3435a49e 100644 --- a/source/dnode/mgmt/impl/src/dndMnode.c +++ b/source/dnode/mgmt/impl/src/dndMnode.c @@ -616,7 +616,7 @@ static int32_t dndInitMnodeMgmtWorker(SDnode *pDnode) { pPool->min = 1; pPool->max = 1; if (tWorkerInit(pPool) != 0) { - terrno = TSDB_CODE_VND_OUT_OF_MEMORY; + terrno = TSDB_CODE_OUT_OF_MEMORY; return -1; } @@ -652,7 +652,7 @@ static int32_t dndInitMnodeReadWorker(SDnode *pDnode) { pPool->min = 0; pPool->max = 1; if (tWorkerInit(pPool) != 0) { - terrno = TSDB_CODE_VND_OUT_OF_MEMORY; + terrno = TSDB_CODE_OUT_OF_MEMORY; return -1; } @@ -703,7 +703,7 @@ static int32_t dndInitMnodeWriteWorker(SDnode *pDnode) { pPool->min = 0; pPool->max = 1; if (tWorkerInit(pPool) != 0) { - terrno = TSDB_CODE_VND_OUT_OF_MEMORY; + terrno = TSDB_CODE_OUT_OF_MEMORY; return -1; } diff --git a/source/dnode/mgmt/impl/src/dndVnodes.c b/source/dnode/mgmt/impl/src/dndVnodes.c index d5e94106a7..ac3e55ffa7 100644 --- a/source/dnode/mgmt/impl/src/dndVnodes.c +++ b/source/dnode/mgmt/impl/src/dndVnodes.c @@ -22,6 +22,7 @@ typedef struct { int32_t refCount; int8_t dropped; int8_t accessState; + char *path; SVnode *pImpl; taos_queue pWriteQ; taos_queue pSyncQ; @@ -74,7 +75,7 @@ static int32_t dndPutMsgIntoVnodeApplyQueue(SDnode *pDnode, int32_t vgId, SVnode static SVnodeObj *dndAcquireVnode(SDnode *pDnode, int32_t vgId); static void dndReleaseVnode(SDnode *pDnode, SVnodeObj *pVnode); -static int32_t dndCreateVnodeWrapper(SDnode *pDnode, int32_t vgId, SVnode *pImpl); +static int32_t dndCreateVnodeWrapper(SDnode *pDnode, int32_t vgId, char *path, SVnode *pImpl); static void dndDropVnodeWrapper(SDnode *pDnode, SVnodeObj *pVnode); static SVnodeObj **dndGetVnodesFromHash(SDnode *pDnode, int32_t *numOfVnodes); static int32_t dndGetVnodesFromFile(SDnode *pDnode, SVnodeObj **ppVnodes, int32_t *numOfVnodes); @@ -125,7 +126,7 @@ static void dndReleaseVnode(SDnode *pDnode, SVnodeObj *pVnode) { } } -static int32_t dndCreateVnodeWrapper(SDnode *pDnode, int32_t vgId, SVnode *pImpl) { +static int32_t dndCreateVnodeWrapper(SDnode *pDnode, int32_t vgId, char *path, SVnode *pImpl) { SVnodesMgmt *pMgmt = &pDnode->vmgmt; SVnodeObj *pVnode = calloc(1, sizeof(SVnodeObj)); if (pVnode == NULL) { @@ -139,6 +140,12 @@ static int32_t dndCreateVnodeWrapper(SDnode *pDnode, int32_t vgId, SVnode *pImpl pVnode->accessState = TSDB_VN_ALL_ACCCESS; pVnode->pImpl = pImpl; + pVnode->path = tstrdup(path); + if (pVnode->path == NULL) { + terrno = TSDB_CODE_OUT_OF_MEMORY; + return -1; + } + if (dndAllocVnodeQueryQueue(pDnode, pVnode) != 0) { return -1; } @@ -354,22 +361,25 @@ static int32_t dndWriteVnodesToFile(SDnode *pDnode) { static int32_t dndCreateVnode(SDnode *pDnode, int32_t vgId, SVnodeCfg *pCfg) { char path[PATH_MAX + 20] = {0}; snprintf(path, sizeof(path), "%s/vnode%d", pDnode->dir.vnodes, vgId); - SVnode *pImpl = vnodeCreate(vgId, path, pCfg); + // SVnode *pImpl = vnodeCreate(vgId, path, pCfg); + SVnode *pImpl = vnodeOpen(path, NULL); if (pImpl == NULL) { return -1; } - int32_t code = dndCreateVnodeWrapper(pDnode, vgId, pImpl); + int32_t code = dndCreateVnodeWrapper(pDnode, vgId, path, pImpl); if (code != 0) { - vnodeDrop(pImpl); + vnodeClose(pImpl); + vnodeDestroy(path); terrno = code; return code; } code = dndWriteVnodesToFile(pDnode); if (code != 0) { - vnodeDrop(pImpl); + vnodeClose(pImpl); + vnodeDestroy(path); terrno = code; return code; } @@ -385,7 +395,8 @@ static int32_t dndDropVnode(SDnode *pDnode, SVnodeObj *pVnode) { } dndDropVnodeWrapper(pDnode, pVnode); - vnodeDrop(pVnode->pImpl); + vnodeClose(pVnode->pImpl); + vnodeDestroy(pVnode->path); dndWriteVnodesToFile(pDnode); return 0; } @@ -413,7 +424,7 @@ static void *dnodeOpenVnodeFunc(void *param) { dError("vgId:%d, failed to open vnode by thread:%d", pVnode->vgId, pThread->threadIndex); pThread->failed++; } else { - dndCreateVnodeWrapper(pDnode, pVnode->vgId, pImpl); + dndCreateVnodeWrapper(pDnode, pVnode->vgId, path, pImpl); dDebug("vgId:%d, is opened by thread:%d", pVnode->vgId, pThread->threadIndex); pThread->opened++; } @@ -433,7 +444,7 @@ static int32_t dndOpenVnodes(SDnode *pDnode) { pMgmt->hash = taosHashInit(TSDB_MIN_VNODES, taosGetDefaultHashFunction(TSDB_DATA_TYPE_INT), true, HASH_NO_LOCK); if (pMgmt->hash == NULL) { dError("failed to init vnode hash"); - terrno = TSDB_CODE_VND_OUT_OF_MEMORY; + terrno = TSDB_CODE_OUT_OF_MEMORY; return -1; } @@ -874,13 +885,13 @@ static int32_t dndInitVnodeMgmtWorker(SDnode *pDnode) { pPool->min = 1; pPool->max = 1; if (tWorkerInit(pPool) != 0) { - terrno = TSDB_CODE_VND_OUT_OF_MEMORY; + terrno = TSDB_CODE_OUT_OF_MEMORY; return -1; } pMgmt->pMgmtQ = tWorkerAllocQueue(pPool, pDnode, (FProcessItem)dndProcessVnodeMgmtQueue); if (pMgmt->pMgmtQ == NULL) { - terrno = TSDB_CODE_VND_OUT_OF_MEMORY; + terrno = TSDB_CODE_OUT_OF_MEMORY; return -1; } @@ -918,6 +929,7 @@ static int32_t dndAllocVnodeFetchQueue(SDnode *pDnode, SVnodeObj *pVnode) { terrno = TSDB_CODE_OUT_OF_MEMORY; return -1; } + return 0; } @@ -938,7 +950,8 @@ static int32_t dndInitVnodeReadWorker(SDnode *pDnode) { pPool->min = (int32_t)threadsForQuery; pPool->max = pPool->min; if (tWorkerInit(pPool) != 0) { - return TSDB_CODE_VND_OUT_OF_MEMORY; + terrno = TSDB_CODE_OUT_OF_MEMORY; + return -1; } pPool = &pMgmt->fetchPool; @@ -946,7 +959,8 @@ static int32_t dndInitVnodeReadWorker(SDnode *pDnode) { pPool->min = MIN(maxFetchThreads, pDnode->opt.numOfCores); pPool->max = pPool->min; if (tWorkerInit(pPool) != 0) { - TSDB_CODE_VND_OUT_OF_MEMORY; + terrno = TSDB_CODE_OUT_OF_MEMORY; + return -1; } return 0; @@ -998,7 +1012,7 @@ static int32_t dndInitVnodeWriteWorker(SDnode *pDnode) { pPool->name = "vnode-write"; pPool->max = tsNumOfCores; if (tMWorkerInit(pPool) != 0) { - terrno = TSDB_CODE_VND_OUT_OF_MEMORY; + terrno = TSDB_CODE_OUT_OF_MEMORY; return -1; } @@ -1036,7 +1050,7 @@ static int32_t dndInitVnodeSyncWorker(SDnode *pDnode) { pPool->name = "vnode-sync"; pPool->max = maxThreads; if (tMWorkerInit(pPool) != 0) { - terrno = TSDB_CODE_VND_OUT_OF_MEMORY; + terrno = TSDB_CODE_OUT_OF_MEMORY; return -1; } diff --git a/source/dnode/mgmt/impl/src/dnode.c b/source/dnode/mgmt/impl/src/dnode.c index 0bfcf5d721..aa0070cfa9 100644 --- a/source/dnode/mgmt/impl/src/dnode.c +++ b/source/dnode/mgmt/impl/src/dnode.c @@ -116,6 +116,7 @@ static int32_t dndInitEnv(SDnode *pDnode, SDnodeOpt *pOptions) { return -1; } + memcpy(&pDnode->opt, pOptions, sizeof(SDnodeOpt)); return 0; } diff --git a/source/dnode/vnode/impl/src/vnodeInt.c b/source/dnode/vnode/impl/src/vnodeInt.c index 2cbdf318a2..8a6fc8bf5e 100644 --- a/source/dnode/vnode/impl/src/vnodeInt.c +++ b/source/dnode/vnode/impl/src/vnodeInt.c @@ -17,9 +17,6 @@ #include "vnodeInt.h" #include "tqueue.h" -int32_t vnodeInit(SVnodePara para) { return 0; } -void vnodeCleanup() {} - int32_t vnodeAlter(SVnode *pVnode, const SVnodeCfg *pCfg) { return 0; } SVnode *vnodeCreate(int32_t vgId, const char *path, const SVnodeCfg *pCfg) { return NULL; } void vnodeDrop(SVnode *pVnode) {} @@ -31,7 +28,7 @@ int32_t vnodeGetLoad(SVnode *pVnode, SVnodeLoad *pLoad) { return 0; } SVnodeMsg *vnodeInitMsg(int32_t msgNum) { SVnodeMsg *pMsg = taosAllocateQitem(msgNum * sizeof(SRpcMsg *) + sizeof(SVnodeMsg)); if (pMsg == NULL) { - terrno = TSDB_CODE_VND_OUT_OF_MEMORY; + terrno = TSDB_CODE_OUT_OF_MEMORY; return NULL; } else { pMsg->allocNum = msgNum; @@ -41,7 +38,7 @@ SVnodeMsg *vnodeInitMsg(int32_t msgNum) { int32_t vnodeAppendMsg(SVnodeMsg *pMsg, SRpcMsg *pRpcMsg) { if (pMsg->curNum >= pMsg->allocNum) { - return TSDB_CODE_VND_OUT_OF_MEMORY; + return TSDB_CODE_OUT_OF_MEMORY; } pMsg->rpcMsg[pMsg->curNum++] = *pRpcMsg; From aad478d34f218bbd45ec11e4d900dde505b9aad3 Mon Sep 17 00:00:00 2001 From: Shengliang Guan Date: Fri, 26 Nov 2021 16:28:38 +0800 Subject: [PATCH 27/44] TD-11265 refact dndMnode --- include/dnode/mgmt/dnode.h | 4 +- include/dnode/mnode/mnode.h | 10 +-- source/dnode/mgmt/impl/src/dndMnode.c | 107 ++++++++++++++----------- source/dnode/mgmt/impl/src/dndVnodes.c | 2 +- source/dnode/mgmt/impl/src/dnode.c | 16 ++-- source/dnode/mnode/impl/inc/mnodeInt.h | 2 +- source/dnode/mnode/impl/src/mnode.c | 28 +++---- 7 files changed, 91 insertions(+), 78 deletions(-) diff --git a/include/dnode/mgmt/dnode.h b/include/dnode/mgmt/dnode.h index f43fe107fe..fe9560d427 100644 --- a/include/dnode/mgmt/dnode.h +++ b/include/dnode/mgmt/dnode.h @@ -121,10 +121,10 @@ typedef struct { /** * @brief Initialize and start the dnode. * - * @param pOptions Options of the dnode. + * @param pOption Option of the dnode. * @return SDnode* The dnode object. */ -SDnode *dndInit(SDnodeOpt *pOptions); +SDnode *dndInit(SDnodeOpt *pOption); /** * @brief Stop and cleanup the dnode. diff --git a/include/dnode/mnode/mnode.h b/include/dnode/mnode/mnode.h index 725bdaec3c..824eb24191 100644 --- a/include/dnode/mnode/mnode.h +++ b/include/dnode/mnode/mnode.h @@ -53,17 +53,17 @@ typedef struct { SendMsgToDnodeFp sendMsgToDnodeFp; SendMsgToMnodeFp sendMsgToMnodeFp; SendRedirectMsgFp sendRedirectMsgFp; -} SMnodeOptions; +} SMnodeOpt; /* ------------------------ SMnode ------------------------ */ /** * @brief Open a mnode. * * @param path Path of the mnode - * @param pOptions Options of the mnode + * @param pOption Option of the mnode * @return SMnode* The mnode object */ -SMnode *mnodeOpen(const char *path, const SMnodeOptions *pOptions); +SMnode *mnodeOpen(const char *path, const SMnodeOpt *pOption); /** * @brief Close a mnode @@ -76,10 +76,10 @@ void mnodeClose(SMnode *pMnode); * @brief Close a mnode * * @param pMnode The mnode object to close - * @param pOptions Options of the mnode + * @param pOption Options of the mnode * @return int32_t 0 for success, -1 for failure */ -int32_t mnodeAlter(SMnode *pMnode, const SMnodeOptions *pOptions); +int32_t mnodeAlter(SMnode *pMnode, const SMnodeOpt *pOption); /** * @brief Drop a mnode. diff --git a/source/dnode/mgmt/impl/src/dndMnode.c b/source/dnode/mgmt/impl/src/dndMnode.c index 9b3435a49e..0a764af8dc 100644 --- a/source/dnode/mgmt/impl/src/dndMnode.c +++ b/source/dnode/mgmt/impl/src/dndMnode.c @@ -58,8 +58,8 @@ static void dndReleaseMnode(SDnode *pDnode, SMnode *pMnode); static int32_t dndReadMnodeFile(SDnode *pDnode); static int32_t dndWriteMnodeFile(SDnode *pDnode); -static int32_t dndOpenMnode(SDnode *pDnode, SMnodeOptions *pOptions); -static int32_t dndAlterMnode(SDnode *pDnode, SMnodeOptions *pOptions); +static int32_t dndOpenMnode(SDnode *pDnode, SMnodeOpt *pOption); +static int32_t dndAlterMnode(SDnode *pDnode, SMnodeOpt *pOption); static int32_t dndDropMnode(SDnode *pDnode); static int32_t dndProcessCreateMnodeReq(SDnode *pDnode, SRpcMsg *pRpcMsg); @@ -243,6 +243,7 @@ static bool dndNeedDeployMnode(SDnode *pDnode) { if (dndGetClusterId(pDnode) > 0) { return false; } + if (strcmp(pDnode->opt.localEp, pDnode->opt.firstEp) != 0) { return false; } @@ -250,43 +251,49 @@ static bool dndNeedDeployMnode(SDnode *pDnode) { return true; } -static void dndInitMnodeOptions(SDnode *pDnode, SMnodeOptions *pOptions) { - pOptions->pDnode = pDnode; - pOptions->sendMsgToDnodeFp = dndSendMsgToDnode; - pOptions->sendMsgToMnodeFp = dndSendMsgToMnode; - pOptions->sendRedirectMsgFp = dndSendRedirectMsg; - pOptions->putMsgToApplyMsgFp = dndPutMsgIntoMnodeApplyQueue; +static void dndInitMnodeOption(SDnode *pDnode, SMnodeOpt *pOption) { + pOption->pDnode = pDnode; + pOption->sendMsgToDnodeFp = dndSendMsgToDnode; + pOption->sendMsgToMnodeFp = dndSendMsgToMnode; + pOption->sendRedirectMsgFp = dndSendRedirectMsg; + pOption->putMsgToApplyMsgFp = dndPutMsgIntoMnodeApplyQueue; + pOption->dnodeId = dndGetDnodeId(pDnode); + pOption->clusterId = dndGetClusterId(pDnode); } -static int32_t dndBuildMnodeOptions(SDnode *pDnode, SMnodeOptions *pOptions, SCreateMnodeMsg *pMsg) { - dndInitMnodeOptions(pDnode, pOptions); +static void dndBuildMnodeDeployOption(SDnode *pDnode, SMnodeOpt *pOption) { + dndInitMnodeOption(pDnode, pOption); + pOption->replica = 1; + pOption->selfIndex = 0; + SReplica *pReplica = &pOption->replicas[0]; + pReplica->id = 1; + pReplica->port = pDnode->opt.serverPort; + tstrncpy(pReplica->fqdn, pDnode->opt.localFqdn, TSDB_FQDN_LEN); +} - if (pMsg == NULL) { - pOptions->dnodeId = 1; - pOptions->clusterId = 1234; - pOptions->replica = 1; - pOptions->selfIndex = 0; - SReplica *pReplica = &pOptions->replicas[0]; - pReplica->id = 1; - pReplica->port = pDnode->opt.serverPort; - tstrncpy(pReplica->fqdn, pDnode->opt.localFqdn, TSDB_FQDN_LEN); - } else { - pOptions->dnodeId = dndGetDnodeId(pDnode); - pOptions->clusterId = dndGetClusterId(pDnode); - pOptions->selfIndex = -1; - pOptions->replica = pMsg->replica; - for (int32_t index = 0; index < pMsg->replica; ++index) { - SReplica *pReplica = &pOptions->replicas[index]; - pReplica->id = pMsg->replicas[index].id; - pReplica->port = pMsg->replicas[index].port; - tstrncpy(pReplica->fqdn, pMsg->replicas[index].fqdn, TSDB_FQDN_LEN); - if (pReplica->id == pOptions->dnodeId) { - pOptions->selfIndex = index; - } +static void dndBuildMnodeOpenOption(SDnode *pDnode, SMnodeOpt *pOption) { + dndInitMnodeOption(pDnode, pOption); + pOption->replica = 0; +} + +static int32_t dndBuildMnodeOptionFromMsg(SDnode *pDnode, SMnodeOpt *pOption, SCreateMnodeMsg *pMsg) { + dndInitMnodeOption(pDnode, pOption); + pOption->dnodeId = dndGetDnodeId(pDnode); + pOption->clusterId = dndGetClusterId(pDnode); + + pOption->replica = pMsg->replica; + pOption->selfIndex = -1; + for (int32_t index = 0; index < pMsg->replica; ++index) { + SReplica *pReplica = &pOption->replicas[index]; + pReplica->id = pMsg->replicas[index].id; + pReplica->port = pMsg->replicas[index].port; + tstrncpy(pReplica->fqdn, pMsg->replicas[index].fqdn, TSDB_FQDN_LEN); + if (pReplica->id == pOption->dnodeId) { + pOption->selfIndex = index; } } - if (pOptions->selfIndex == -1) { + if (pOption->selfIndex == -1) { terrno = TSDB_CODE_DND_MNODE_ID_NOT_FOUND; dError("failed to build mnode options since %s", terrstr()); return -1; @@ -295,7 +302,7 @@ static int32_t dndBuildMnodeOptions(SDnode *pDnode, SMnodeOptions *pOptions, SCr return 0; } -static int32_t dndOpenMnode(SDnode *pDnode, SMnodeOptions *pOptions) { +static int32_t dndOpenMnode(SDnode *pDnode, SMnodeOpt *pOption) { SMnodeMgmt *pMgmt = &pDnode->mmgmt; int32_t code = dndStartMnodeWorker(pDnode); @@ -304,7 +311,7 @@ static int32_t dndOpenMnode(SDnode *pDnode, SMnodeOptions *pOptions) { return code; } - SMnode *pMnode = mnodeOpen(pDnode->dir.mnode, pOptions); + SMnode *pMnode = mnodeOpen(pDnode->dir.mnode, pOption); if (pMnode == NULL) { dError("failed to open mnode since %s", terrstr()); code = terrno; @@ -331,7 +338,7 @@ static int32_t dndOpenMnode(SDnode *pDnode, SMnodeOptions *pOptions) { return 0; } -static int32_t dndAlterMnode(SDnode *pDnode, SMnodeOptions *pOptions) { +static int32_t dndAlterMnode(SDnode *pDnode, SMnodeOpt *pOption) { SMnodeMgmt *pMgmt = &pDnode->mmgmt; SMnode *pMnode = dndAcquireMnode(pDnode); @@ -340,7 +347,7 @@ static int32_t dndAlterMnode(SDnode *pDnode, SMnodeOptions *pOptions) { return -1; } - if (mnodeAlter(pMnode, pOptions) != 0) { + if (mnodeAlter(pMnode, pOption) != 0) { dError("failed to alter mnode since %s", terrstr()); dndReleaseMnode(pDnode, pMnode); return -1; @@ -399,8 +406,8 @@ static int32_t dndProcessCreateMnodeReq(SDnode *pDnode, SRpcMsg *pRpcMsg) { terrno = TSDB_CODE_DND_MNODE_ID_INVALID; return -1; } else { - SMnodeOptions option = {0}; - if (dndBuildMnodeOptions(pDnode, &option, pMsg) != 0) { + SMnodeOpt option = {0}; + if (dndBuildMnodeOptionFromMsg(pDnode, &option, pMsg) != 0) { return -1; } return dndOpenMnode(pDnode, &option); @@ -414,8 +421,8 @@ static int32_t dndProcessAlterMnodeReq(SDnode *pDnode, SRpcMsg *pRpcMsg) { terrno = TSDB_CODE_DND_MNODE_ID_INVALID; return -1; } else { - SMnodeOptions option = {0}; - if (dndBuildMnodeOptions(pDnode, &option, pMsg) != 0) { + SMnodeOpt option = {0}; + if (dndBuildMnodeOptionFromMsg(pDnode, &option, pMsg) != 0) { return -1; } return dndAlterMnode(pDnode, &option); @@ -625,7 +632,6 @@ static int32_t dndInitMnodeMgmtWorker(SDnode *pDnode) { static void dndCleanupMnodeMgmtWorker(SDnode *pDnode) { SMnodeMgmt *pMgmt = &pDnode->mmgmt; - ; tWorkerCleanup(&pMgmt->mgmtPool); } @@ -737,7 +743,12 @@ static int32_t dndInitMnodeSyncWorker(SDnode *pDnode) { pPool->name = "mnode-sync"; pPool->min = 0; pPool->max = 1; - return tWorkerInit(pPool); + if (tWorkerInit(pPool) != 0) { + terrno = TSDB_CODE_OUT_OF_MEMORY; + return -1; + } + + return 0; } static void dndCleanupMnodeSyncWorker(SDnode *pDnode) { @@ -781,13 +792,15 @@ int32_t dndInitMnode(SDnode *pDnode) { } dInfo("start to deploy mnode"); + SMnodeOpt option = {0}; + dndBuildMnodeDeployOption(pDnode, &option); + return dndOpenMnode(pDnode, &option); } else { dInfo("start to open mnode"); + SMnodeOpt option = {0}; + dndBuildMnodeOpenOption(pDnode, &option); + return dndOpenMnode(pDnode, &option); } - - SMnodeOptions option = {0}; - dndInitMnodeOptions(pDnode, &option); - return dndOpenMnode(pDnode, &option); } void dndCleanupMnode(SDnode *pDnode) { diff --git a/source/dnode/mgmt/impl/src/dndVnodes.c b/source/dnode/mgmt/impl/src/dndVnodes.c index ac3e55ffa7..fd66695e32 100644 --- a/source/dnode/mgmt/impl/src/dndVnodes.c +++ b/source/dnode/mgmt/impl/src/dndVnodes.c @@ -239,7 +239,7 @@ static int32_t dndGetVnodesFromFile(SDnode *pDnode, SVnodeObj **ppVnodes, int32_ snprintf(file, PATH_MAX + 20, "%s/vnodes.json", pDnode->dir.vnodes); fp = fopen(file, "r"); - if (!fp) { + if (fp == NULL) { dDebug("file %s not exist", file); code = 0; goto PRASE_VNODE_OVER; diff --git a/source/dnode/mgmt/impl/src/dnode.c b/source/dnode/mgmt/impl/src/dnode.c index aa0070cfa9..8d72f83200 100644 --- a/source/dnode/mgmt/impl/src/dnode.c +++ b/source/dnode/mgmt/impl/src/dnode.c @@ -77,19 +77,19 @@ static int32_t dndCheckRunning(char *dataDir) { return 0; } -static int32_t dndInitEnv(SDnode *pDnode, SDnodeOpt *pOptions) { - if (dndCheckRunning(pOptions->dataDir) != 0) { +static int32_t dndInitEnv(SDnode *pDnode, SDnodeOpt *pOption) { + if (dndCheckRunning(pOption->dataDir) != 0) { return -1; } char path[PATH_MAX + 100]; - snprintf(path, sizeof(path), "%s%smnode", pOptions->dataDir, TD_DIRSEP); + snprintf(path, sizeof(path), "%s%smnode", pOption->dataDir, TD_DIRSEP); pDnode->dir.mnode = tstrdup(path); - snprintf(path, sizeof(path), "%s%svnode", pOptions->dataDir, TD_DIRSEP); + snprintf(path, sizeof(path), "%s%svnode", pOption->dataDir, TD_DIRSEP); pDnode->dir.vnodes = tstrdup(path); - snprintf(path, sizeof(path), "%s%sdnode", pOptions->dataDir, TD_DIRSEP); + snprintf(path, sizeof(path), "%s%sdnode", pOption->dataDir, TD_DIRSEP); pDnode->dir.dnode = tstrdup(path); if (pDnode->dir.mnode == NULL || pDnode->dir.vnodes == NULL || pDnode->dir.dnode == NULL) { @@ -116,7 +116,7 @@ static int32_t dndInitEnv(SDnode *pDnode, SDnodeOpt *pOptions) { return -1; } - memcpy(&pDnode->opt, pOptions, sizeof(SDnodeOpt)); + memcpy(&pDnode->opt, pOption, sizeof(SDnodeOpt)); return 0; } @@ -136,7 +136,7 @@ static void dndCleanupEnv(SDnode *pDnode) { taosStopCacheRefreshWorker(); } -SDnode *dndInit(SDnodeOpt *pOptions) { +SDnode *dndInit(SDnodeOpt *pOption) { taosIgnSIGPIPE(); taosBlockSIGPIPE(); taosResolveCRC(); @@ -151,7 +151,7 @@ SDnode *dndInit(SDnodeOpt *pOptions) { dInfo("start to initialize TDengine"); dndSetStat(pDnode, DND_STAT_INIT); - if (dndInitEnv(pDnode, pOptions) != 0) { + if (dndInitEnv(pDnode, pOption) != 0) { dError("failed to init env"); dndCleanup(pDnode); return NULL; diff --git a/source/dnode/mnode/impl/inc/mnodeInt.h b/source/dnode/mnode/impl/inc/mnodeInt.h index 43af281f27..7f7f91a3af 100644 --- a/source/dnode/mnode/impl/inc/mnodeInt.h +++ b/source/dnode/mnode/impl/inc/mnodeInt.h @@ -32,7 +32,7 @@ typedef struct SMnodeBak { tmr_h timer; SSteps *pInitSteps; SSteps *pStartSteps; - SMnodeOptions para; + SMnodeOpt para; MnodeRpcFp msgFp[TSDB_MSG_TYPE_MAX]; } SMnodeBak; diff --git a/source/dnode/mnode/impl/src/mnode.c b/source/dnode/mnode/impl/src/mnode.c index bd89476cef..43dd57bbf8 100644 --- a/source/dnode/mnode/impl/src/mnode.c +++ b/source/dnode/mnode/impl/src/mnode.c @@ -77,17 +77,17 @@ static void mnodeCleanupTimer() { tmr_h mnodeGetTimer() { return tsMint.timer; } -static int32_t mnodeSetOptions(SMnode *pMnode, const SMnodeOptions *pOptions) { - pMnode->dnodeId = pOptions->dnodeId; - pMnode->clusterId = pOptions->clusterId; - pMnode->replica = pOptions->replica; - pMnode->selfIndex = pOptions->selfIndex; - memcpy(&pMnode->replicas, pOptions->replicas, sizeof(SReplica) * TSDB_MAX_REPLICA); - pMnode->pServer = pOptions->pDnode; - pMnode->putMsgToApplyMsgFp = pOptions->putMsgToApplyMsgFp; - pMnode->sendMsgToDnodeFp = pOptions->sendMsgToDnodeFp; - pMnode->sendMsgToMnodeFp = pOptions->sendMsgToMnodeFp; - pMnode->sendRedirectMsgFp = pOptions->sendRedirectMsgFp; +static int32_t mnodeSetOptions(SMnode *pMnode, const SMnodeOpt *pOption) { + pMnode->dnodeId = pOption->dnodeId; + pMnode->clusterId = pOption->clusterId; + pMnode->replica = pOption->replica; + pMnode->selfIndex = pOption->selfIndex; + memcpy(&pMnode->replicas, pOption->replicas, sizeof(SReplica) * TSDB_MAX_REPLICA); + pMnode->pServer = pOption->pDnode; + pMnode->putMsgToApplyMsgFp = pOption->putMsgToApplyMsgFp; + pMnode->sendMsgToDnodeFp = pOption->sendMsgToDnodeFp; + pMnode->sendMsgToMnodeFp = pOption->sendMsgToMnodeFp; + pMnode->sendRedirectMsgFp = pOption->sendRedirectMsgFp; if (pMnode->sendMsgToDnodeFp == NULL || pMnode->sendMsgToMnodeFp == NULL || pMnode->sendRedirectMsgFp == NULL || pMnode->putMsgToApplyMsgFp == NULL || pMnode->dnodeId < 0 || pMnode->clusterId < 0) { @@ -136,10 +136,10 @@ static int32_t mnodeAllocStartSteps() { return 0; } -SMnode *mnodeOpen(const char *path, const SMnodeOptions *pOptions) { +SMnode *mnodeOpen(const char *path, const SMnodeOpt *pOption) { SMnode *pMnode = calloc(1, sizeof(SMnode)); - if (mnodeSetOptions(pMnode, pOptions) != 0) { + if (mnodeSetOptions(pMnode, pOption) != 0) { free(pMnode); mError("failed to init mnode options since %s", terrstr()); return NULL; @@ -173,7 +173,7 @@ SMnode *mnodeOpen(const char *path, const SMnodeOptions *pOptions) { void mnodeClose(SMnode *pMnode) { free(pMnode); } -int32_t mnodeAlter(SMnode *pMnode, const SMnodeOptions *pOptions) { return 0; } +int32_t mnodeAlter(SMnode *pMnode, const SMnodeOpt *pOption) { return 0; } void mnodeDestroy(const char *path) { sdbUnDeploy(); } From 8bf34d89607ffedb0c4a09d3fd1271d945e73cf3 Mon Sep 17 00:00:00 2001 From: Shengliang Guan Date: Sat, 27 Nov 2021 00:25:31 +0800 Subject: [PATCH 28/44] TD-11265 save mnode replicas in mnode.json --- source/dnode/mgmt/impl/inc/dndInt.h | 3 ++ source/dnode/mgmt/impl/src/dndMnode.c | 76 +++++++++++++++++++++++++-- 2 files changed, 74 insertions(+), 5 deletions(-) diff --git a/source/dnode/mgmt/impl/inc/dndInt.h b/source/dnode/mgmt/impl/inc/dndInt.h index 4094871bcd..106f192856 100644 --- a/source/dnode/mgmt/impl/inc/dndInt.h +++ b/source/dnode/mgmt/impl/inc/dndInt.h @@ -71,6 +71,9 @@ typedef struct { int32_t refCount; int8_t deployed; int8_t dropped; + int8_t replica; + int8_t selfIndex; + SReplica replicas[TSDB_MAX_REPLICA]; SWorkerPool mgmtPool; SWorkerPool readPool; SWorkerPool writePool; diff --git a/source/dnode/mgmt/impl/src/dndMnode.c b/source/dnode/mgmt/impl/src/dndMnode.c index 0a764af8dc..fa35e9b573 100644 --- a/source/dnode/mgmt/impl/src/dndMnode.c +++ b/source/dnode/mgmt/impl/src/dndMnode.c @@ -101,7 +101,7 @@ static int32_t dndReadMnodeFile(SDnode *pDnode) { SMnodeMgmt *pMgmt = &pDnode->mmgmt; int32_t code = TSDB_CODE_DND_MNODE_READ_FILE_ERROR; int32_t len = 0; - int32_t maxLen = 300; + int32_t maxLen = 4096; char *content = calloc(1, maxLen + 1); cJSON *root = NULL; @@ -139,6 +139,46 @@ static int32_t dndReadMnodeFile(SDnode *pDnode) { } pMgmt->dropped = atoi(dropped->valuestring); + cJSON *nodes = cJSON_GetObjectItem(root, "nodes"); + if (!nodes || nodes->type != cJSON_Array) { + dError("failed to read %s since nodes not found", pMgmt->file); + goto PRASE_MNODE_OVER; + } + + pMgmt->replica = cJSON_GetArraySize(nodes); + if (pMgmt->replica <= 0 || pMgmt->replica > TSDB_MAX_REPLICA) { + dError("failed to read %s since nodes size %d invalid", pMgmt->file, pMgmt->replica); + goto PRASE_MNODE_OVER; + } + + for (int32_t i = 0; i < pMgmt->replica; ++i) { + cJSON *node = cJSON_GetArrayItem(nodes, i); + if (node == NULL) break; + + SReplica *pReplica = &pMgmt->replicas[i]; + + cJSON *id = cJSON_GetObjectItem(node, "id"); + if (!id || id->type != cJSON_String || id->valuestring == NULL) { + dError("failed to read %s since id not found", pMgmt->file); + goto PRASE_MNODE_OVER; + } + pReplica->id = atoi(id->valuestring); + + cJSON *fqdn = cJSON_GetObjectItem(node, "fqdn"); + if (!fqdn || fqdn->type != cJSON_String || fqdn->valuestring == NULL) { + dError("failed to read %s since fqdn not found", pMgmt->file); + goto PRASE_MNODE_OVER; + } + tstrncpy(pReplica->fqdn, fqdn->valuestring, TSDB_FQDN_LEN); + + cJSON *port = cJSON_GetObjectItem(node, "port"); + if (!port || port->type != cJSON_String || port->valuestring == NULL) { + dError("failed to read %s since port not found", pMgmt->file); + goto PRASE_MNODE_OVER; + } + pReplica->port = atoi(port->valuestring); + } + code = 0; dInfo("succcessed to read file %s", pMgmt->file); @@ -153,7 +193,8 @@ PRASE_MNODE_OVER: static int32_t dndWriteMnodeFile(SDnode *pDnode) { SMnodeMgmt *pMgmt = &pDnode->mmgmt; - char file[PATH_MAX + 20] = {0}; + + char file[PATH_MAX + 20] = {0}; snprintf(file, sizeof(file), "%s.bak", pMgmt->file); FILE *fp = fopen(file, "w"); @@ -164,12 +205,25 @@ static int32_t dndWriteMnodeFile(SDnode *pDnode) { } int32_t len = 0; - int32_t maxLen = 300; + int32_t maxLen = 4096; char *content = calloc(1, maxLen + 1); len += snprintf(content + len, maxLen - len, "{\n"); len += snprintf(content + len, maxLen - len, " \"deployed\": \"%d\",\n", pMgmt->deployed); - len += snprintf(content + len, maxLen - len, " \"dropped\": \"%d\"\n", pMgmt->dropped); + + len += snprintf(content + len, maxLen - len, " \"dropped\": \"%d\",\n", pMgmt->dropped); + len += snprintf(content + len, maxLen - len, " \"nodes\": [{\n"); + for (int32_t i = 0; i < pMgmt->replica; ++i) { + SReplica *pReplica = &pMgmt->replicas[i]; + len += snprintf(content + len, maxLen - len, " \"id\": \"%d\",\n", pReplica->id); + len += snprintf(content + len, maxLen - len, " \"fqdn\": \"%s\",\n", pReplica->fqdn); + len += snprintf(content + len, maxLen - len, " \"port\": \"%u\"\n", pReplica->port); + if (i < pMgmt->replica - 1) { + len += snprintf(content + len, maxLen - len, " },{\n"); + } else { + len += snprintf(content + len, maxLen - len, " }]\n"); + } + } len += snprintf(content + len, maxLen - len, "}\n"); fwrite(content, 1, len, fp); @@ -269,11 +323,19 @@ static void dndBuildMnodeDeployOption(SDnode *pDnode, SMnodeOpt *pOption) { pReplica->id = 1; pReplica->port = pDnode->opt.serverPort; tstrncpy(pReplica->fqdn, pDnode->opt.localFqdn, TSDB_FQDN_LEN); + + SMnodeMgmt *pMgmt = &pDnode->mmgmt; + pMgmt->selfIndex = pOption->selfIndex; + pMgmt->replica = pOption->replica; + memcpy(&pMgmt->replicas, pOption->replicas, sizeof(SReplica) * TSDB_MAX_REPLICA); } static void dndBuildMnodeOpenOption(SDnode *pDnode, SMnodeOpt *pOption) { dndInitMnodeOption(pDnode, pOption); - pOption->replica = 0; + SMnodeMgmt *pMgmt = &pDnode->mmgmt; + pOption->selfIndex = pMgmt->selfIndex; + pOption->replica = pMgmt->replica; + memcpy(&pOption->replicas, pMgmt->replicas, sizeof(SReplica) * TSDB_MAX_REPLICA); } static int32_t dndBuildMnodeOptionFromMsg(SDnode *pDnode, SMnodeOpt *pOption, SCreateMnodeMsg *pMsg) { @@ -299,6 +361,10 @@ static int32_t dndBuildMnodeOptionFromMsg(SDnode *pDnode, SMnodeOpt *pOption, SC return -1; } + SMnodeMgmt *pMgmt = &pDnode->mmgmt; + pMgmt->selfIndex = pOption->selfIndex; + pMgmt->replica = pOption->replica; + memcpy(&pMgmt->replicas, pOption->replicas, sizeof(SReplica) * TSDB_MAX_REPLICA); return 0; } From 7330b0f8c870e39da440f5b5156806a6651ed135 Mon Sep 17 00:00:00 2001 From: Shengliang Guan Date: Sat, 27 Nov 2021 14:57:16 +0800 Subject: [PATCH 29/44] remove printf --- source/dnode/mgmt/impl/src/dndMnode.c | 21 +++++- source/dnode/mnode/impl/src/mnode.c | 2 +- source/os/src/osDir.c | 10 +-- source/os/src/osFile.c | 4 +- source/os/src/osSocket.c | 96 +++++++++++++-------------- source/os/src/osString.c | 2 +- source/os/src/osSysinfo.c | 90 ++++++++++++------------- source/os/src/osSystem.c | 10 +-- source/os/src/osTimer.c | 14 ++-- 9 files changed, 134 insertions(+), 115 deletions(-) diff --git a/source/dnode/mgmt/impl/src/dndMnode.c b/source/dnode/mgmt/impl/src/dndMnode.c index fa35e9b573..a4f6d845fd 100644 --- a/source/dnode/mgmt/impl/src/dndMnode.c +++ b/source/dnode/mgmt/impl/src/dndMnode.c @@ -198,7 +198,7 @@ static int32_t dndWriteMnodeFile(SDnode *pDnode) { snprintf(file, sizeof(file), "%s.bak", pMgmt->file); FILE *fp = fopen(file, "w"); - if (fp != NULL) { + if (fp == NULL) { terrno = TSDB_CODE_DND_MNODE_WRITE_FILE_ERROR; dError("failed to write %s since %s", file, terrstr()); return -1; @@ -242,6 +242,21 @@ static int32_t dndWriteMnodeFile(SDnode *pDnode) { } static int32_t dndStartMnodeWorker(SDnode *pDnode) { + if (dndInitMnodeReadWorker(pDnode) != 0) { + dError("failed to start mnode read worker since %s", terrstr()); + return -1; + } + + if (dndInitMnodeWriteWorker(pDnode) != 0) { + dError("failed to start mnode write worker since %s", terrstr()); + return -1; + } + + if (dndInitMnodeSyncWorker(pDnode) != 0) { + dError("failed to start mnode sync worker since %s", terrstr()); + return -1; + } + if (dndAllocMnodeReadQueue(pDnode) != 0) { dError("failed to alloc mnode read queue since %s", terrstr()); return -1; @@ -708,6 +723,7 @@ static int32_t dndAllocMnodeReadQueue(SDnode *pDnode) { terrno = TSDB_CODE_OUT_OF_MEMORY; return -1; } + return 0; } @@ -743,6 +759,7 @@ static int32_t dndAllocMnodeWriteQueue(SDnode *pDnode) { terrno = TSDB_CODE_OUT_OF_MEMORY; return -1; } + return 0; } @@ -759,6 +776,7 @@ static int32_t dndAllocMnodeApplyQueue(SDnode *pDnode) { terrno = TSDB_CODE_OUT_OF_MEMORY; return -1; } + return 0; } @@ -794,6 +812,7 @@ static int32_t dndAllocMnodeSyncQueue(SDnode *pDnode) { terrno = TSDB_CODE_OUT_OF_MEMORY; return -1; } + return 0; } diff --git a/source/dnode/mnode/impl/src/mnode.c b/source/dnode/mnode/impl/src/mnode.c index 43dd57bbf8..8fc5de588f 100644 --- a/source/dnode/mnode/impl/src/mnode.c +++ b/source/dnode/mnode/impl/src/mnode.c @@ -31,9 +31,9 @@ #include "mnodeStable.h" #include "mnodeSync.h" #include "mnodeTelem.h" +#include "mnodeTrans.h" #include "mnodeUser.h" #include "mnodeVgroup.h" -#include "mnodeTrans.h" SMnodeBak tsMint = {0}; diff --git a/source/os/src/osDir.c b/source/os/src/osDir.c index cfa0028925..070fe4a6c8 100644 --- a/source/os/src/osDir.c +++ b/source/os/src/osDir.c @@ -48,14 +48,14 @@ void taosRemoveDir(const char *dirname) { taosRemoveDir(filename); } else { (void)remove(filename); - printf("file:%s is removed\n", filename); + //printf("file:%s is removed\n", filename); } } closedir(dir); rmdir(dirname); - printf("dir:%s is removed\n", dirname); + //printf("dir:%s is removed\n", dirname); } int32_t taosDirExist(char *dirname) { return access(dirname, F_OK); } @@ -101,9 +101,9 @@ void taosRemoveOldFiles(char *dirname, int32_t keepDays) { int32_t days = (int32_t)(ABS(sec - fileSec) / 86400 + 1); if (days > keepDays) { (void)remove(filename); - printf("file:%s is removed, days:%d keepDays:%d", filename, days, keepDays); + //printf("file:%s is removed, days:%d keepDays:%d", filename, days, keepDays); } else { - printf("file:%s won't be removed, days:%d keepDays:%d", filename, days, keepDays); + //printf("file:%s won't be removed, days:%d keepDays:%d", filename, days, keepDays); } } } @@ -115,7 +115,7 @@ void taosRemoveOldFiles(char *dirname, int32_t keepDays) { int32_t taosExpandDir(char *dirname, char *outname, int32_t maxlen) { wordexp_t full_path; if (0 != wordexp(dirname, &full_path, 0)) { - printf("failed to expand path:%s since %s", dirname, strerror(errno)); + //printf("failed to expand path:%s since %s", dirname, strerror(errno)); wordfree(&full_path); return -1; } diff --git a/source/os/src/osFile.c b/source/os/src/osFile.c index b5d30d7c25..bdf3a53dcb 100644 --- a/source/os/src/osFile.c +++ b/source/os/src/osFile.c @@ -404,14 +404,14 @@ int32_t taosRenameFile(char *oldName, char *newName) { #if defined(_TD_WINDOWS_64) || defined(_TD_WINDOWS_32) int32_t code = MoveFileEx(oldName, newName, MOVEFILE_REPLACE_EXISTING | MOVEFILE_COPY_ALLOWED); if (code < 0) { - printf("failed to rename file %s to %s, reason:%s", oldName, newName, strerror(errno)); + //printf("failed to rename file %s to %s, reason:%s", oldName, newName, strerror(errno)); } return code; #else int32_t code = rename(oldName, newName); if (code < 0) { - printf("failed to rename file %s to %s, reason:%s", oldName, newName, strerror(errno)); + //printf("failed to rename file %s to %s, reason:%s", oldName, newName, strerror(errno)); } return code; diff --git a/source/os/src/osSocket.c b/source/os/src/osSocket.c index fb1aeebe1c..e12f9493b3 100644 --- a/source/os/src/osSocket.c +++ b/source/os/src/osSocket.c @@ -95,7 +95,7 @@ void taosShutDownSocketWR(SOCKET fd) { int32_t taosSetNonblocking(SOCKET sock, int32_t on) { int32_t flags = 0; if ((flags = fcntl(sock, F_GETFL, 0)) < 0) { - printf("fcntl(F_GETFL) error: %d (%s)\n", errno, strerror(errno)); + //printf("fcntl(F_GETFL) error: %d (%s)\n", errno, strerror(errno)); return 1; } @@ -105,7 +105,7 @@ int32_t taosSetNonblocking(SOCKET sock, int32_t on) { flags &= ~O_NONBLOCK; if ((flags = fcntl(sock, F_SETFL, flags)) < 0) { - printf("fcntl(F_SETFL) error: %d (%s)\n", errno, strerror(errno)); + //printf("fcntl(F_SETFL) error: %d (%s)\n", errno, strerror(errno)); return 1; } @@ -120,7 +120,7 @@ void taosBlockSIGPIPE() { sigaddset(&signal_mask, SIGPIPE); int32_t rc = pthread_sigmask(SIG_BLOCK, &signal_mask, NULL); if (rc != 0) { - printf("failed to block SIGPIPE"); + //printf("failed to block SIGPIPE"); } } @@ -130,7 +130,7 @@ void taosSetMaskSIGPIPE() { sigaddset(&signal_mask, SIGPIPE); int32_t rc = pthread_sigmask(SIG_SETMASK, &signal_mask, NULL); if (rc != 0) { - printf("failed to setmask SIGPIPE"); + //printf("failed to setmask SIGPIPE"); } } @@ -277,7 +277,7 @@ int32_t taosGetFqdn(char *fqdn) { char hostname[1024]; hostname[1023] = '\0'; if (gethostname(hostname, 1023) == -1) { - printf("failed to get hostname, reason:%s", strerror(errno)); + //printf("failed to get hostname, reason:%s", strerror(errno)); return -1; } @@ -294,7 +294,7 @@ int32_t taosGetFqdn(char *fqdn) { #endif // __APPLE__ int32_t ret = getaddrinfo(hostname, NULL, &hints, &result); if (!result) { - printf("failed to get fqdn, code:%d, reason:%s", ret, gai_strerror(ret)); + //printf("failed to get fqdn, code:%d, reason:%s", ret, gai_strerror(ret)); return -1; } @@ -326,12 +326,12 @@ uint32_t taosGetIpv4FromFqdn(const char *fqdn) { } else { #ifdef EAI_SYSTEM if (ret == EAI_SYSTEM) { - printf("failed to get the ip address, fqdn:%s, since:%s", fqdn, strerror(errno)); + //printf("failed to get the ip address, fqdn:%s, since:%s", fqdn, strerror(errno)); } else { - printf("failed to get the ip address, fqdn:%s, since:%s", fqdn, gai_strerror(ret)); + //printf("failed to get the ip address, fqdn:%s, since:%s", fqdn, gai_strerror(ret)); } #else - printf("failed to get the ip address, fqdn:%s, since:%s", fqdn, gai_strerror(ret)); + //printf("failed to get the ip address, fqdn:%s, since:%s", fqdn, gai_strerror(ret)); #endif return 0xFFFFFFFF; } @@ -437,13 +437,13 @@ int32_t taosNonblockwrite(SOCKET fd, char *ptr, int32_t nbytes) { FD_SET(fd, &fset); if ((nready = select((int32_t)(fd + 1), NULL, &fset, NULL, &tv)) == 0) { errno = ETIMEDOUT; - printf("fd %d timeout, no enough space to write", fd); + //printf("fd %d timeout, no enough space to write", fd); break; } else if (nready < 0) { if (errno == EINTR) continue; - printf("select error, %d (%s)", errno, strerror(errno)); + //printf("select error, %d (%s)", errno, strerror(errno)); return -1; } @@ -451,7 +451,7 @@ int32_t taosNonblockwrite(SOCKET fd, char *ptr, int32_t nbytes) { if (nwritten <= 0) { if (errno == EAGAIN || errno == EINTR) continue; - printf("write error, %d (%s)", errno, strerror(errno)); + //printf("write error, %d (%s)", errno, strerror(errno)); return -1; } @@ -477,21 +477,21 @@ int32_t taosReadn(SOCKET fd, char *ptr, int32_t nbytes) { FD_SET(fd, &fset); if ((nready = select((int32_t)(fd + 1), NULL, &fset, NULL, &tv)) == 0) { errno = ETIMEDOUT; - printf("fd %d timeout\n", fd); + //printf("fd %d timeout\n", fd); break; } else if (nready < 0) { if (errno == EINTR) continue; - printf("select error, %d (%s)", errno, strerror(errno)); + //printf("select error, %d (%s)", errno, strerror(errno)); return -1; } if ((nread = (int32_t)taosReadSocket(fd, ptr, (size_t)nleft)) < 0) { if (errno == EINTR) continue; - printf("read error, %d (%s)", errno, strerror(errno)); + //printf("read error, %d (%s)", errno, strerror(errno)); return -1; } else if (nread == 0) { - printf("fd %d EOF", fd); + //printf("fd %d EOF", fd); break; // EOF } @@ -507,7 +507,7 @@ SOCKET taosOpenUdpSocket(uint32_t ip, uint16_t port) { SOCKET sockFd; int32_t bufSize = 1024000; - printf("open udp socket:0x%x:%hu", ip, port); + //printf("open udp socket:0x%x:%hu", ip, port); memset((char *)&localAddr, 0, sizeof(localAddr)); localAddr.sin_family = AF_INET; @@ -515,26 +515,26 @@ SOCKET taosOpenUdpSocket(uint32_t ip, uint16_t port) { localAddr.sin_port = (uint16_t)htons(port); if ((sockFd = socket(AF_INET, SOCK_DGRAM, 0)) <= 2) { - printf("failed to open udp socket: %d (%s)", errno, strerror(errno)); + //printf("failed to open udp socket: %d (%s)", errno, strerror(errno)); taosCloseSocketNoCheck(sockFd); return -1; } if (taosSetSockOpt(sockFd, SOL_SOCKET, SO_SNDBUF, (void *)&bufSize, sizeof(bufSize)) != 0) { - printf("failed to set the send buffer size for UDP socket\n"); + //printf("failed to set the send buffer size for UDP socket\n"); taosCloseSocket(sockFd); return -1; } if (taosSetSockOpt(sockFd, SOL_SOCKET, SO_RCVBUF, (void *)&bufSize, sizeof(bufSize)) != 0) { - printf("failed to set the receive buffer size for UDP socket\n"); + //printf("failed to set the receive buffer size for UDP socket\n"); taosCloseSocket(sockFd); return -1; } /* bind socket to local address */ if (bind(sockFd, (struct sockaddr *)&localAddr, sizeof(localAddr)) < 0) { - printf("failed to bind udp socket: %d (%s), 0x%x:%hu", errno, strerror(errno), ip, port); + //printf("failed to bind udp socket: %d (%s), 0x%x:%hu", errno, strerror(errno), ip, port); taosCloseSocket(sockFd); return -1; } @@ -551,7 +551,7 @@ SOCKET taosOpenTcpClientSocket(uint32_t destIp, uint16_t destPort, uint32_t clie sockFd = socket(PF_INET, SOCK_STREAM, IPPROTO_TCP); if (sockFd <= 2) { - printf("failed to open the socket: %d (%s)", errno, strerror(errno)); + //printf("failed to open the socket: %d (%s)", errno, strerror(errno)); taosCloseSocketNoCheck(sockFd); return -1; } @@ -559,19 +559,19 @@ SOCKET taosOpenTcpClientSocket(uint32_t destIp, uint16_t destPort, uint32_t clie /* set REUSEADDR option, so the portnumber can be re-used */ int32_t reuse = 1; if (taosSetSockOpt(sockFd, SOL_SOCKET, SO_REUSEADDR, (void *)&reuse, sizeof(reuse)) < 0) { - printf("setsockopt SO_REUSEADDR failed: %d (%s)", errno, strerror(errno)); + //printf("setsockopt SO_REUSEADDR failed: %d (%s)", errno, strerror(errno)); taosCloseSocket(sockFd); return -1; } if (taosSetSockOpt(sockFd, SOL_SOCKET, SO_SNDBUF, (void *)&bufSize, sizeof(bufSize)) != 0) { - printf("failed to set the send buffer size for TCP socket\n"); + //printf("failed to set the send buffer size for TCP socket\n"); taosCloseSocket(sockFd); return -1; } if (taosSetSockOpt(sockFd, SOL_SOCKET, SO_RCVBUF, (void *)&bufSize, sizeof(bufSize)) != 0) { - printf("failed to set the receive buffer size for TCP socket\n"); + //printf("failed to set the receive buffer size for TCP socket\n"); taosCloseSocket(sockFd); return -1; } @@ -584,8 +584,8 @@ SOCKET taosOpenTcpClientSocket(uint32_t destIp, uint16_t destPort, uint32_t clie /* bind socket to client address */ if (bind(sockFd, (struct sockaddr *)&clientAddr, sizeof(clientAddr)) < 0) { - printf("bind tcp client socket failed, client(0x%x:0), dest(0x%x:%d), reason:(%s)", clientIp, destIp, destPort, - strerror(errno)); + //printf("bind tcp client socket failed, client(0x%x:0), dest(0x%x:%d), reason:(%s)", clientIp, destIp, destPort, + // strerror(errno)); taosCloseSocket(sockFd); return -1; } @@ -601,7 +601,7 @@ SOCKET taosOpenTcpClientSocket(uint32_t destIp, uint16_t destPort, uint32_t clie ret = connect(sockFd, (struct sockaddr *)&serverAddr, sizeof(serverAddr)); if (ret == -1) { if (errno == EHOSTUNREACH) { - printf("failed to connect socket, ip:0x%x, port:%hu(%s)", destIp, destPort, strerror(errno)); + //printf("failed to connect socket, ip:0x%x, port:%hu(%s)", destIp, destPort, strerror(errno)); taosCloseSocket(sockFd); return -1; } else if (errno == EINPROGRESS || errno == EAGAIN || errno == EWOULDBLOCK) { @@ -612,19 +612,19 @@ SOCKET taosOpenTcpClientSocket(uint32_t destIp, uint16_t destPort, uint32_t clie int res = poll(wfd, 1, TCP_CONN_TIMEOUT); if (res == -1 || res == 0) { - printf("failed to connect socket, ip:0x%x, port:%hu(poll error/conn timeout)", destIp, destPort); + //printf("failed to connect socket, ip:0x%x, port:%hu(poll error/conn timeout)", destIp, destPort); taosCloseSocket(sockFd); // return -1; } int optVal = -1, optLen = sizeof(int); if ((0 != taosGetSockOpt(sockFd, SOL_SOCKET, SO_ERROR, &optVal, &optLen)) || (optVal != 0)) { - printf("failed to connect socket, ip:0x%x, port:%hu(connect host error)", destIp, destPort); + //printf("failed to connect socket, ip:0x%x, port:%hu(connect host error)", destIp, destPort); taosCloseSocket(sockFd); // return -1; } ret = 0; } else { // Other error - printf("failed to connect socket, ip:0x%x, port:%hu(target host cannot be reached)", destIp, destPort); + //printf("failed to connect socket, ip:0x%x, port:%hu(target host cannot be reached)", destIp, destPort); taosCloseSocket(sockFd); // return -1; } @@ -636,7 +636,7 @@ SOCKET taosOpenTcpClientSocket(uint32_t destIp, uint16_t destPort, uint32_t clie #endif if (ret != 0) { - printf("failed to connect socket, ip:0x%x, port:%hu(%s)", destIp, destPort, strerror(errno)); + //printf("failed to connect socket, ip:0x%x, port:%hu(%s)", destIp, destPort, strerror(errno)); taosCloseSocket(sockFd); sockFd = -1; } else { @@ -649,7 +649,7 @@ SOCKET taosOpenTcpClientSocket(uint32_t destIp, uint16_t destPort, uint32_t clie int32_t taosKeepTcpAlive(SOCKET sockFd) { int32_t alive = 1; if (taosSetSockOpt(sockFd, SOL_SOCKET, SO_KEEPALIVE, (void *)&alive, sizeof(alive)) < 0) { - printf("fd:%d setsockopt SO_KEEPALIVE failed: %d (%s)", sockFd, errno, strerror(errno)); + //printf("fd:%d setsockopt SO_KEEPALIVE failed: %d (%s)", sockFd, errno, strerror(errno)); taosCloseSocket(sockFd); return -1; } @@ -658,21 +658,21 @@ int32_t taosKeepTcpAlive(SOCKET sockFd) { // all fails on macosx int32_t probes = 3; if (taosSetSockOpt(sockFd, SOL_TCP, TCP_KEEPCNT, (void *)&probes, sizeof(probes)) < 0) { - printf("fd:%d setsockopt SO_KEEPCNT failed: %d (%s)", sockFd, errno, strerror(errno)); + //printf("fd:%d setsockopt SO_KEEPCNT failed: %d (%s)", sockFd, errno, strerror(errno)); taosCloseSocket(sockFd); return -1; } int32_t alivetime = 10; if (taosSetSockOpt(sockFd, SOL_TCP, TCP_KEEPIDLE, (void *)&alivetime, sizeof(alivetime)) < 0) { - printf("fd:%d setsockopt SO_KEEPIDLE failed: %d (%s)", sockFd, errno, strerror(errno)); + //printf("fd:%d setsockopt SO_KEEPIDLE failed: %d (%s)", sockFd, errno, strerror(errno)); taosCloseSocket(sockFd); return -1; } int32_t interval = 3; if (taosSetSockOpt(sockFd, SOL_TCP, TCP_KEEPINTVL, (void *)&interval, sizeof(interval)) < 0) { - printf("fd:%d setsockopt SO_KEEPINTVL failed: %d (%s)", sockFd, errno, strerror(errno)); + //printf("fd:%d setsockopt SO_KEEPINTVL failed: %d (%s)", sockFd, errno, strerror(errno)); taosCloseSocket(sockFd); return -1; } @@ -680,7 +680,7 @@ int32_t taosKeepTcpAlive(SOCKET sockFd) { int32_t nodelay = 1; if (taosSetSockOpt(sockFd, IPPROTO_TCP, TCP_NODELAY, (void *)&nodelay, sizeof(nodelay)) < 0) { - printf("fd:%d setsockopt TCP_NODELAY failed %d (%s)", sockFd, errno, strerror(errno)); + //printf("fd:%d setsockopt TCP_NODELAY failed %d (%s)", sockFd, errno, strerror(errno)); taosCloseSocket(sockFd); return -1; } @@ -689,7 +689,7 @@ int32_t taosKeepTcpAlive(SOCKET sockFd) { linger.l_onoff = 1; linger.l_linger = 3; if (taosSetSockOpt(sockFd, SOL_SOCKET, SO_LINGER, (void *)&linger, sizeof(linger)) < 0) { - printf("setsockopt SO_LINGER failed: %d (%s)", errno, strerror(errno)); + //printf("setsockopt SO_LINGER failed: %d (%s)", errno, strerror(errno)); taosCloseSocket(sockFd); return -1; } @@ -702,7 +702,7 @@ SOCKET taosOpenTcpServerSocket(uint32_t ip, uint16_t port) { SOCKET sockFd; int32_t reuse; - printf("open tcp server socket:0x%x:%hu", ip, port); + //printf("open tcp server socket:0x%x:%hu", ip, port); bzero((char *)&serverAdd, sizeof(serverAdd)); serverAdd.sin_family = AF_INET; @@ -710,7 +710,7 @@ SOCKET taosOpenTcpServerSocket(uint32_t ip, uint16_t port) { serverAdd.sin_port = (uint16_t)htons(port); if ((sockFd = socket(AF_INET, SOCK_STREAM, IPPROTO_TCP)) <= 2) { - printf("failed to open TCP socket: %d (%s)", errno, strerror(errno)); + //printf("failed to open TCP socket: %d (%s)", errno, strerror(errno)); taosCloseSocketNoCheck(sockFd); return -1; } @@ -718,26 +718,26 @@ SOCKET taosOpenTcpServerSocket(uint32_t ip, uint16_t port) { /* set REUSEADDR option, so the portnumber can be re-used */ reuse = 1; if (taosSetSockOpt(sockFd, SOL_SOCKET, SO_REUSEADDR, (void *)&reuse, sizeof(reuse)) < 0) { - printf("setsockopt SO_REUSEADDR failed: %d (%s)", errno, strerror(errno)); + //printf("setsockopt SO_REUSEADDR failed: %d (%s)", errno, strerror(errno)); taosCloseSocket(sockFd); return -1; } /* bind socket to server address */ if (bind(sockFd, (struct sockaddr *)&serverAdd, sizeof(serverAdd)) < 0) { - printf("bind tcp server socket failed, 0x%x:%hu(%s)", ip, port, strerror(errno)); + //printf("bind tcp server socket failed, 0x%x:%hu(%s)", ip, port, strerror(errno)); taosCloseSocket(sockFd); return -1; } if (taosKeepTcpAlive(sockFd) < 0) { - printf("failed to set tcp server keep-alive option, 0x%x:%hu(%s)", ip, port, strerror(errno)); + //printf("failed to set tcp server keep-alive option, 0x%x:%hu(%s)", ip, port, strerror(errno)); taosCloseSocket(sockFd); return -1; } if (listen(sockFd, 1024) < 0) { - printf("listen tcp server socket failed, 0x%x:%hu(%s)", ip, port, strerror(errno)); + //printf("listen tcp server socket failed, 0x%x:%hu(%s)", ip, port, strerror(errno)); taosCloseSocket(sockFd); return -1; } @@ -767,16 +767,16 @@ int64_t taosCopyFds(SOCKET sfd, int32_t dfd, int64_t len) { int64_t retLen = taosReadMsg(sfd, temp, (int32_t)readLen); if (readLen != retLen) { - printf("read error, readLen:%" PRId64 " retLen:%" PRId64 " len:%" PRId64 " leftLen:%" PRId64 ", reason:%s", - readLen, retLen, len, leftLen, strerror(errno)); + //printf("read error, readLen:%" PRId64 " retLen:%" PRId64 " len:%" PRId64 " leftLen:%" PRId64 ", reason:%s", + // readLen, retLen, len, leftLen, strerror(errno)); return -1; } writeLen = taosWriteMsg(dfd, temp, (int32_t)readLen); if (readLen != writeLen) { - printf("copy error, readLen:%" PRId64 " writeLen:%" PRId64 " len:%" PRId64 " leftLen:%" PRId64 ", reason:%s", - readLen, writeLen, len, leftLen, strerror(errno)); + //printf("copy error, readLen:%" PRId64 " writeLen:%" PRId64 " len:%" PRId64 " leftLen:%" PRId64 ", reason:%s", + // readLen, writeLen, len, leftLen, strerror(errno)); return -1; } diff --git a/source/os/src/osString.c b/source/os/src/osString.c index 10606a3d7b..8054dc42be 100644 --- a/source/os/src/osString.c +++ b/source/os/src/osString.c @@ -277,7 +277,7 @@ char *strsep(char **stringp, const char *delim) { char *getpass(const char *prefix) { static char passwd[TSDB_KEY_LEN] = {0}; memset(passwd, 0, TSDB_KEY_LEN); - printf("%s", prefix); + //printf("%s", prefix); int32_t index = 0; char ch; diff --git a/source/os/src/osSysinfo.c b/source/os/src/osSysinfo.c index 0344507f5e..e37e059b7d 100644 --- a/source/os/src/osSysinfo.c +++ b/source/os/src/osSysinfo.c @@ -134,7 +134,7 @@ int32_t taosGetDiskSize(char *dataDir, SysDiskSize *diskSize) { diskSize->used = (int64_t)(i64TotalBytes - i64FreeBytes); return 0; } else { - printf("failed to get disk size, dataDir:%s errno:%s", tsDataDir, strerror(errno)); + //printf("failed to get disk size, dataDir:%s errno:%s", tsDataDir, strerror(errno)); terrno = TAOS_SYSTEM_ERROR(errno); return -1; } @@ -205,12 +205,12 @@ void taosGetSystemInfo() { } void taosKillSystem() { - printf("function taosKillSystem, exit!"); + //printf("function taosKillSystem, exit!"); exit(0); } int taosSystem(const char *cmd) { - printf("taosSystem not support"); + //printf("taosSystem not support"); return -1; } @@ -280,7 +280,7 @@ static void taosGetSystemTimezone() { { int n = readlink("/etc/localtime", buf, sizeof(buf)); if (n < 0) { - printf("read /etc/localtime error, reason:%s", strerror(errno)); + //printf("read /etc/localtime error, reason:%s", strerror(errno)); return; } buf[n] = '\0'; @@ -294,7 +294,7 @@ static void taosGetSystemTimezone() { } } if (!tz || 0 == strchr(tz, '/')) { - printf("parsing /etc/localtime failed"); + //printf("parsing /etc/localtime failed"); return; } @@ -321,7 +321,7 @@ static void taosGetSystemTimezone() { -timezone / 3600); // cfg_timezone->cfgStatus = TAOS_CFG_CSTATUS_DEFAULT; - printf("timezone not configured, set to system default:%s", tsTimezone); + //printf("timezone not configured, set to system default:%s", tsTimezone); } /* @@ -348,11 +348,11 @@ static void taosGetSystemLocale() { // get and set default locale locale = setlocale(LC_CTYPE, ""); if (locale == NULL) { - printf("can't get locale from system, set it to en_US.UTF-8 since error:%d:%s", errno, strerror(errno)); + //printf("can't get locale from system, set it to en_US.UTF-8 since error:%d:%s", errno, strerror(errno)); strcpy(tsLocale, "en_US.UTF-8"); } else { tstrncpy(tsLocale, locale, TSDB_LOCALE_LEN); - printf("locale not configured, set to system default:%s", tsLocale); + //printf("locale not configured, set to system default:%s", tsLocale); } /* if user does not specify the charset, extract it from locale */ @@ -364,15 +364,15 @@ static void taosGetSystemLocale() { // get and set default locale tstrncpy(tsCharset, revisedCharset, TSDB_LOCALE_LEN); free(revisedCharset); - printf("charset not configured, set to system default:%s", tsCharset); + //printf("charset not configured, set to system default:%s", tsCharset); } else { strcpy(tsCharset, "UTF-8"); - printf("can't get locale and charset from system, set it to UTF-8"); + //printf("can't get locale and charset from system, set it to UTF-8"); } } void taosKillSystem() { - printf("function taosKillSystem, exit!"); + //printf("function taosKillSystem, exit!"); exit(0); } @@ -432,7 +432,7 @@ bool taosGetSysMemory(float *memoryUsedMB) { } int taosSystem(const char *cmd) { - printf("un support funtion"); + //printf("un support funtion"); return -1; } @@ -441,7 +441,7 @@ void taosSetCoreDump() {} int32_t taosGetDiskSize(char *dataDir, SysDiskSize *diskSize) { struct statvfs info; if (statvfs(dataDir, &info)) { - printf("failed to get disk size, dataDir:%s errno:%s", tsDataDir, strerror(errno)); + //printf("failed to get disk size, dataDir:%s errno:%s", tsDataDir, strerror(errno)); terrno = TAOS_SYSTEM_ERROR(errno); return -1; } else { @@ -535,7 +535,7 @@ bool taosGetSysMemory(float *memoryUsedMB) { bool taosGetProcMemory(float *memoryUsedMB) { FILE *fp = fopen(tsProcMemFile, "r"); if (fp == NULL) { - printf("open file:%s failed", tsProcMemFile); + //printf("open file:%s failed", tsProcMemFile); return false; } @@ -555,7 +555,7 @@ bool taosGetProcMemory(float *memoryUsedMB) { } if (line == NULL) { - printf("read file:%s failed", tsProcMemFile); + //printf("read file:%s failed", tsProcMemFile); fclose(fp); return false; } @@ -573,7 +573,7 @@ bool taosGetProcMemory(float *memoryUsedMB) { static bool taosGetSysCpuInfo(SysCpuInfo *cpuInfo) { FILE *fp = fopen(tsSysCpuFile, "r"); if (fp == NULL) { - printf("open file:%s failed", tsSysCpuFile); + //printf("open file:%s failed", tsSysCpuFile); return false; } @@ -581,7 +581,7 @@ static bool taosGetSysCpuInfo(SysCpuInfo *cpuInfo) { char * line = NULL; ssize_t _bytes = getline(&line, &len, fp); if ((_bytes < 0) || (line == NULL)) { - printf("read file:%s failed", tsSysCpuFile); + //printf("read file:%s failed", tsSysCpuFile); fclose(fp); return false; } @@ -598,7 +598,7 @@ static bool taosGetSysCpuInfo(SysCpuInfo *cpuInfo) { static bool taosGetProcCpuInfo(ProcCpuInfo *cpuInfo) { FILE *fp = fopen(tsProcCpuFile, "r"); if (fp == NULL) { - printf("open file:%s failed", tsProcCpuFile); + //printf("open file:%s failed", tsProcCpuFile); return false; } @@ -606,7 +606,7 @@ static bool taosGetProcCpuInfo(ProcCpuInfo *cpuInfo) { char * line = NULL; ssize_t _bytes = getline(&line, &len, fp); if ((_bytes < 0) || (line == NULL)) { - printf("read file:%s failed", tsProcCpuFile); + //printf("read file:%s failed", tsProcCpuFile); fclose(fp); return false; } @@ -642,7 +642,7 @@ static void taosGetSystemTimezone() { int len = fread(buf, 64, 1, f); if (len < 64 && ferror(f)) { fclose(f); - printf("read /etc/timezone error, reason:%s", strerror(errno)); + //printf("read /etc/timezone error, reason:%s", strerror(errno)); return; } @@ -681,7 +681,7 @@ static void taosGetSystemTimezone() { snprintf(tsTimezone, TSDB_TIMEZONE_LEN, "%s (%s, %s%02d00)", buf, tzname[daylight], tz >= 0 ? "+" : "-", abs(tz)); // cfg_timezone->cfgStatus = TAOS_CFG_CSTATUS_DEFAULT; - printf("timezone not configured, set to system default:%s", tsTimezone); + //printf("timezone not configured, set to system default:%s", tsTimezone); } /* @@ -707,11 +707,11 @@ static void taosGetSystemLocale() { // get and set default locale locale = setlocale(LC_CTYPE, ""); if (locale == NULL) { - printf("can't get locale from system, set it to en_US.UTF-8 since error:%d:%s", errno, strerror(errno)); + //printf("can't get locale from system, set it to en_US.UTF-8 since error:%d:%s", errno, strerror(errno)); strcpy(tsLocale, "en_US.UTF-8"); } else { tstrncpy(tsLocale, locale, TSDB_LOCALE_LEN); - printf("locale not configured, set to system default:%s", tsLocale); + //printf("locale not configured, set to system default:%s", tsLocale); } /* if user does not specify the charset, extract it from locale */ @@ -723,10 +723,10 @@ static void taosGetSystemLocale() { // get and set default locale tstrncpy(tsCharset, revisedCharset, TSDB_LOCALE_LEN); free(revisedCharset); - printf("charset not configured, set to system default:%s", tsCharset); + //printf("charset not configured, set to system default:%s", tsCharset); } else { strcpy(tsCharset, "UTF-8"); - printf("can't get locale and charset from system, set it to UTF-8"); + //printf("can't get locale and charset from system, set it to UTF-8"); } } @@ -774,7 +774,7 @@ bool taosGetCpuUsage(float *sysCpuUsage, float *procCpuUsage) { int32_t taosGetDiskSize(char *dataDir, SysDiskSize *diskSize) { struct statvfs info; if (statvfs(dataDir, &info)) { - printf("failed to get disk size, dataDir:%s errno:%s", dataDir, strerror(errno)); + //printf("failed to get disk size, dataDir:%s errno:%s", dataDir, strerror(errno)); return -1; } else { diskSize->tsize = info.f_blocks * info.f_frsize; @@ -788,7 +788,7 @@ bool taosGetCardInfo(int64_t *bytes, int64_t *rbytes, int64_t *tbytes) { *bytes = 0; FILE *fp = fopen(tsSysNetFile, "r"); if (fp == NULL) { - printf("open file:%s failed", tsSysNetFile); + //printf("open file:%s failed", tsSysNetFile); return false; } @@ -864,7 +864,7 @@ bool taosGetBandSpeed(float *bandSpeedKb) { double totalBytes = (double)(curBytes - lastBytes) / 1024 * 8; // Kb *bandSpeedKb = (float)(totalBytes / (double)(curTime - lastTime)); - // printf("bandwidth lastBytes:%ld, lastTime:%ld, curBytes:%ld, curTime:%ld, + // //printf("bandwidth lastBytes:%ld, lastTime:%ld, curBytes:%ld, curTime:%ld, // speed:%f", lastBytes, lastTime, curBytes, curTime, *bandSpeed); lastTime = curTime; @@ -876,7 +876,7 @@ bool taosGetBandSpeed(float *bandSpeedKb) { bool taosReadProcIO(int64_t *rchars, int64_t *wchars) { FILE *fp = fopen(tsProcIOFile, "r"); if (fp == NULL) { - printf("open file:%s failed", tsProcIOFile); + //printf("open file:%s failed", tsProcIOFile); return false; } @@ -909,7 +909,7 @@ bool taosReadProcIO(int64_t *rchars, int64_t *wchars) { fclose(fp); if (readIndex < 2) { - printf("read file:%s failed", tsProcIOFile); + //printf("read file:%s failed", tsProcIOFile); return false; } @@ -964,7 +964,7 @@ void taosGetSystemInfo() { void taosKillSystem() { // SIGINT - printf("taosd will shut down soon"); + //printf("taosd will shut down soon"); kill(tsProcId, 2); } @@ -973,22 +973,22 @@ int taosSystem(const char *cmd) { int res; char buf[1024]; if (cmd == NULL) { - printf("taosSystem cmd is NULL!"); + //printf("taosSystem cmd is NULL!"); return -1; } if ((fp = popen(cmd, "r")) == NULL) { - printf("popen cmd:%s error: %s", cmd, strerror(errno)); + //printf("popen cmd:%s error: %s", cmd, strerror(errno)); return -1; } else { while (fgets(buf, sizeof(buf), fp)) { - printf("popen result:%s", buf); + //printf("popen result:%s", buf); } if ((res = pclose(fp)) == -1) { - printf("close popen file pointer fp error!"); + //printf("close popen file pointer fp error!"); } else { - printf("popen res is :%d", res); + //printf("popen res is :%d", res); } return res; @@ -1003,14 +1003,14 @@ void taosSetCoreDump(bool enable) { struct rlimit rlim_new; if (getrlimit(RLIMIT_CORE, &rlim) == 0) { #ifndef _ALPINE - printf("the old unlimited para: rlim_cur=%" PRIu64 ", rlim_max=%" PRIu64, rlim.rlim_cur, rlim.rlim_max); + //printf("the old unlimited para: rlim_cur=%" PRIu64 ", rlim_max=%" PRIu64, rlim.rlim_cur, rlim.rlim_max); #else - printf("the old unlimited para: rlim_cur=%llu, rlim_max=%llu", rlim.rlim_cur, rlim.rlim_max); + //printf("the old unlimited para: rlim_cur=%llu, rlim_max=%llu", rlim.rlim_cur, rlim.rlim_max); #endif rlim_new.rlim_cur = RLIM_INFINITY; rlim_new.rlim_max = RLIM_INFINITY; if (setrlimit(RLIMIT_CORE, &rlim_new) != 0) { - printf("set unlimited fail, error: %s", strerror(errno)); + //printf("set unlimited fail, error: %s", strerror(errno)); rlim_new.rlim_cur = rlim.rlim_max; rlim_new.rlim_max = rlim.rlim_max; (void)setrlimit(RLIMIT_CORE, &rlim_new); @@ -1019,9 +1019,9 @@ void taosSetCoreDump(bool enable) { if (getrlimit(RLIMIT_CORE, &rlim) == 0) { #ifndef _ALPINE - printf("the new unlimited para: rlim_cur=%" PRIu64 ", rlim_max=%" PRIu64, rlim.rlim_cur, rlim.rlim_max); + //printf("the new unlimited para: rlim_cur=%" PRIu64 ", rlim_max=%" PRIu64, rlim.rlim_cur, rlim.rlim_max); #else - printf("the new unlimited para: rlim_cur=%llu, rlim_max=%llu", rlim.rlim_cur, rlim.rlim_max); + //printf("the new unlimited para: rlim_cur=%llu, rlim_max=%llu", rlim.rlim_cur, rlim.rlim_max); #endif } @@ -1047,10 +1047,10 @@ void taosSetCoreDump(bool enable) { old_len = sizeof(old_usespid); if (syscall(SYS__sysctl, &args) == -1) { - printf("_sysctl(kern_core_uses_pid) set fail: %s", strerror(errno)); + //printf("_sysctl(kern_core_uses_pid) set fail: %s", strerror(errno)); } - printf("The old core_uses_pid[%" PRIu64 "]: %d", old_len, old_usespid); + //printf("The old core_uses_pid[%" PRIu64 "]: %d", old_len, old_usespid); old_usespid = 0; old_len = 0; @@ -1063,10 +1063,10 @@ void taosSetCoreDump(bool enable) { old_len = sizeof(old_usespid); if (syscall(SYS__sysctl, &args) == -1) { - printf("_sysctl(kern_core_uses_pid) get fail: %s", strerror(errno)); + //printf("_sysctl(kern_core_uses_pid) get fail: %s", strerror(errno)); } - printf("The new core_uses_pid[%" PRIu64 "]: %d", old_len, old_usespid); + //printf("The new core_uses_pid[%" PRIu64 "]: %d", old_len, old_usespid); #endif } diff --git a/source/os/src/osSystem.c b/source/os/src/osSystem.c index 1a57e88c58..717cae0fbd 100644 --- a/source/os/src/osSystem.c +++ b/source/os/src/osSystem.c @@ -87,11 +87,11 @@ int taosSetConsoleEcho(bool on) { void* taosLoadDll(const char* filename) { void* handle = dlopen(filename, RTLD_LAZY); if (!handle) { - printf("load dll:%s failed, error:%s", filename, dlerror()); + //printf("load dll:%s failed, error:%s", filename, dlerror()); return NULL; } - printf("dll %s loaded", filename); + //printf("dll %s loaded", filename); return handle; } @@ -101,11 +101,11 @@ void* taosLoadSym(void* handle, char* name) { char* error = NULL; if ((error = dlerror()) != NULL) { - printf("load sym:%s failed, error:%s", name, dlerror()); + //printf("load sym:%s failed, error:%s", name, dlerror()); return NULL; } - printf("sym %s loaded", name); + //printf("sym %s loaded", name); return sym; } @@ -133,7 +133,7 @@ int taosSetConsoleEcho(bool on) { err = tcsetattr(STDIN_FILENO, TCSAFLUSH, &term); if (err == -1 || err == EINTR) { - printf("Cannot set the attribution of the terminal"); + //printf("Cannot set the attribution of the terminal"); return -1; } diff --git a/source/os/src/osTimer.c b/source/os/src/osTimer.c index b1bf1bcd2d..7e542ef80f 100644 --- a/source/os/src/osTimer.c +++ b/source/os/src/osTimer.c @@ -170,7 +170,7 @@ static void *taosProcessAlarmSignal(void *tharg) { sevent.sigev_signo = SIGALRM; if (timer_create(CLOCK_REALTIME, &sevent, &timerId) == -1) { - printf("Failed to create timer"); + //printf("Failed to create timer"); } pthread_cleanup_push(taosDeleteTimer, &timerId); @@ -182,17 +182,17 @@ static void *taosProcessAlarmSignal(void *tharg) { ts.it_interval.tv_nsec = 1000000 * MSECONDS_PER_TICK; if (timer_settime(timerId, 0, &ts, NULL)) { - printf("Failed to init timer"); + //printf("Failed to init timer"); return NULL; } int signo; while (!stopTimer) { if (sigwait(&sigset, &signo)) { - printf("Failed to wait signal: number %d", signo); + //printf("Failed to wait signal: number %d", signo); continue; } - /* printf("Signal handling: number %d ......\n", signo); */ + /* //printf("Signal handling: number %d ......\n", signo); */ callback(0); } @@ -208,10 +208,10 @@ int taosInitTimer(void (*callback)(int), int ms) { int code = pthread_create(&timerThread, &tattr, taosProcessAlarmSignal, callback); pthread_attr_destroy(&tattr); if (code != 0) { - printf("failed to create timer thread"); + //printf("failed to create timer thread"); return -1; } else { - printf("timer thread:0x%08" PRIx64 " is created", taosGetPthreadId(timerThread)); + //printf("timer thread:0x%08" PRIx64 " is created", taosGetPthreadId(timerThread)); } return 0; @@ -220,7 +220,7 @@ int taosInitTimer(void (*callback)(int), int ms) { void taosUninitTimer() { stopTimer = true; - printf("join timer thread:0x%08" PRIx64, taosGetPthreadId(timerThread)); + //printf("join timer thread:0x%08" PRIx64, taosGetPthreadId(timerThread)); pthread_join(timerThread, NULL); } From c049678ade258de74d9b5b2cc6db2c9a5dc8fe4e Mon Sep 17 00:00:00 2001 From: Shengliang Guan Date: Sat, 27 Nov 2021 15:57:54 +0800 Subject: [PATCH 30/44] TD-11265 fix deadlock while quit taosd --- include/util/tworker.h | 19 +++++++++++-------- source/dnode/mgmt/daemon/src/daemon.c | 5 ++++- source/dnode/mgmt/impl/inc/dndInt.h | 22 +++++++++++----------- source/dnode/mgmt/impl/src/dndMnode.c | 12 ++++++++---- source/dnode/mgmt/impl/src/dndTransport.c | 8 +++++++- source/dnode/mgmt/impl/src/dnode.c | 2 +- source/util/src/tqueue.c | 2 +- source/util/src/tworker.c | 4 ++-- 8 files changed, 45 insertions(+), 29 deletions(-) diff --git a/include/util/tworker.h b/include/util/tworker.h index 9b0fe4f3a5..2e5852cbba 100644 --- a/include/util/tworker.h +++ b/include/util/tworker.h @@ -22,10 +22,13 @@ extern "C" { #endif +typedef struct SWorkerPool SWorkerPool; +typedef struct SMWorkerPool SMWorkerPool; + typedef struct SWorker { - int32_t id; // worker ID - pthread_t thread; // thread - struct SWorkerPool *pool; + int32_t id; // worker ID + pthread_t thread; // thread + SWorkerPool *pool; } SWorker; typedef struct SWorkerPool { @@ -39,11 +42,11 @@ typedef struct SWorkerPool { } SWorkerPool; typedef struct SMWorker { - int32_t id; // worker id - pthread_t thread; // thread - taos_qall qall; - taos_qset qset; // queue set - struct SMWorkerPool *pool; + int32_t id; // worker id + pthread_t thread; // thread + taos_qall qall; + taos_qset qset; // queue set + SMWorkerPool *pool; } SMWorker; typedef struct SMWorkerPool { diff --git a/source/dnode/mgmt/daemon/src/daemon.c b/source/dnode/mgmt/daemon/src/daemon.c index effaec66a8..a0ca0dd390 100644 --- a/source/dnode/mgmt/daemon/src/daemon.c +++ b/source/dnode/mgmt/daemon/src/daemon.c @@ -30,7 +30,10 @@ static struct { char configDir[PATH_MAX]; } global = {0}; -void dmnSigintHandle(int signum, void *info, void *ctx) { global.stop = true; } +void dmnSigintHandle(int signum, void *info, void *ctx) { + uError("singal:%d is received", signum); + global.stop = true; +} void dmnSetSignalHandle() { taosSetSignal(SIGTERM, dmnSigintHandle); diff --git a/source/dnode/mgmt/impl/inc/dndInt.h b/source/dnode/mgmt/impl/inc/dndInt.h index 106f192856..39243a1795 100644 --- a/source/dnode/mgmt/impl/inc/dndInt.h +++ b/source/dnode/mgmt/impl/inc/dndInt.h @@ -74,31 +74,31 @@ typedef struct { int8_t replica; int8_t selfIndex; SReplica replicas[TSDB_MAX_REPLICA]; - SWorkerPool mgmtPool; - SWorkerPool readPool; - SWorkerPool writePool; - SWorkerPool syncPool; + char *file; + SMnode *pMnode; + SRWLatch latch; taos_queue pReadQ; taos_queue pWriteQ; taos_queue pApplyQ; taos_queue pSyncQ; taos_queue pMgmtQ; - char *file; - SMnode *pMnode; - SRWLatch latch; + SWorkerPool mgmtPool; + SWorkerPool readPool; + SWorkerPool writePool; + SWorkerPool syncPool; } SMnodeMgmt; typedef struct { SHashObj *hash; + int32_t openVnodes; + int32_t totalVnodes; + SRWLatch latch; + taos_queue pMgmtQ; SWorkerPool mgmtPool; SWorkerPool queryPool; SWorkerPool fetchPool; SMWorkerPool syncPool; SMWorkerPool writePool; - taos_queue pMgmtQ; - int32_t openVnodes; - int32_t totalVnodes; - SRWLatch latch; } SVnodesMgmt; typedef struct { diff --git a/source/dnode/mgmt/impl/src/dndMnode.c b/source/dnode/mgmt/impl/src/dndMnode.c index a4f6d845fd..fe3accdd84 100644 --- a/source/dnode/mgmt/impl/src/dndMnode.c +++ b/source/dnode/mgmt/impl/src/dndMnode.c @@ -294,14 +294,14 @@ static void dndStopMnodeWorker(SDnode *pDnode) { while (!taosQueueEmpty(pMgmt->pWriteQ)) taosMsleep(10); while (!taosQueueEmpty(pMgmt->pSyncQ)) taosMsleep(10); + dndCleanupMnodeReadWorker(pDnode); + dndCleanupMnodeWriteWorker(pDnode); + dndCleanupMnodeSyncWorker(pDnode); + dndFreeMnodeReadQueue(pDnode); dndFreeMnodeWriteQueue(pDnode); dndFreeMnodeApplyQueue(pDnode); dndFreeMnodeSyncQueue(pDnode); - - dndCleanupMnodeReadWorker(pDnode); - dndCleanupMnodeWriteWorker(pDnode); - dndCleanupMnodeSyncWorker(pDnode); } static bool dndNeedDeployMnode(SDnode *pDnode) { @@ -714,6 +714,7 @@ static int32_t dndInitMnodeMgmtWorker(SDnode *pDnode) { static void dndCleanupMnodeMgmtWorker(SDnode *pDnode) { SMnodeMgmt *pMgmt = &pDnode->mmgmt; tWorkerCleanup(&pMgmt->mgmtPool); + dDebug("mnode mgmt worker is stopped"); } static int32_t dndAllocMnodeReadQueue(SDnode *pDnode) { @@ -750,6 +751,7 @@ static int32_t dndInitMnodeReadWorker(SDnode *pDnode) { static void dndCleanupMnodeReadWorker(SDnode *pDnode) { SMnodeMgmt *pMgmt = &pDnode->mmgmt; tWorkerCleanup(&pMgmt->readPool); + dDebug("mnode read worker is stopped"); } static int32_t dndAllocMnodeWriteQueue(SDnode *pDnode) { @@ -803,6 +805,7 @@ static int32_t dndInitMnodeWriteWorker(SDnode *pDnode) { static void dndCleanupMnodeWriteWorker(SDnode *pDnode) { SMnodeMgmt *pMgmt = &pDnode->mmgmt; tWorkerCleanup(&pMgmt->writePool); + dDebug("mnode write worker is stopped"); } static int32_t dndAllocMnodeSyncQueue(SDnode *pDnode) { @@ -839,6 +842,7 @@ static int32_t dndInitMnodeSyncWorker(SDnode *pDnode) { static void dndCleanupMnodeSyncWorker(SDnode *pDnode) { SMnodeMgmt *pMgmt = &pDnode->mmgmt; tWorkerCleanup(&pMgmt->syncPool); + dDebug("mnode sync worker is stopped"); } int32_t dndInitMnode(SDnode *pDnode) { diff --git a/source/dnode/mgmt/impl/src/dndTransport.c b/source/dnode/mgmt/impl/src/dndTransport.c index d5f52bac8b..c3940cd3cc 100644 --- a/source/dnode/mgmt/impl/src/dndTransport.c +++ b/source/dnode/mgmt/impl/src/dndTransport.c @@ -160,6 +160,7 @@ static int32_t dndInitClient(SDnode *pDnode) { rpcInit.user = INTERNAL_USER; rpcInit.ckey = INTERNAL_CKEY; rpcInit.secret = INTERNAL_SECRET; + rpcInit.parent = pDnode; pMgmt->clientRpc = rpcOpen(&rpcInit); if (pMgmt->clientRpc == NULL) { @@ -167,6 +168,7 @@ static int32_t dndInitClient(SDnode *pDnode) { return -1; } + dDebug("dnode rpc client is initialized"); return 0; } @@ -175,7 +177,7 @@ static void dndCleanupClient(SDnode *pDnode) { if (pMgmt->clientRpc) { rpcClose(pMgmt->clientRpc); pMgmt->clientRpc = NULL; - dInfo("dnode peer rpc client is closed"); + dDebug("dnode rpc client is closed"); } } @@ -315,6 +317,7 @@ static int32_t dndInitServer(SDnode *pDnode) { rpcInit.connType = TAOS_CONN_SERVER; rpcInit.idleTime = pDnode->opt.shellActivityTimer * 1000; rpcInit.afp = dndRetrieveUserAuthInfo; + rpcInit.parent = pDnode; pMgmt->serverRpc = rpcOpen(&rpcInit); if (pMgmt->serverRpc == NULL) { @@ -322,6 +325,7 @@ static int32_t dndInitServer(SDnode *pDnode) { return -1; } + dDebug("dnode rpc server is initialized"); return 0; } @@ -330,6 +334,7 @@ static void dndCleanupServer(SDnode *pDnode) { if (pMgmt->serverRpc) { rpcClose(pMgmt->serverRpc); pMgmt->serverRpc = NULL; + dDebug("dnode rpc server is closed"); } } @@ -347,6 +352,7 @@ int32_t dndInitTrans(SDnode *pDnode) { } void dndCleanupTrans(SDnode *pDnode) { + dInfo("dnode-transport start to clean up"); dndCleanupServer(pDnode); dndCleanupClient(pDnode); dInfo("dnode-transport is cleaned up"); diff --git a/source/dnode/mgmt/impl/src/dnode.c b/source/dnode/mgmt/impl/src/dnode.c index 8d72f83200..23c9ee0ebf 100644 --- a/source/dnode/mgmt/impl/src/dnode.c +++ b/source/dnode/mgmt/impl/src/dnode.c @@ -197,7 +197,7 @@ SDnode *dndInit(SDnodeOpt *pOption) { dndReportStartup(pDnode, "TDengine", "initialized successfully"); dInfo("TDengine is initialized successfully"); - return 0; + return pDnode; } void dndCleanup(SDnode *pDnode) { diff --git a/source/util/src/tqueue.c b/source/util/src/tqueue.c index 93008f7114..04bc0c8dc8 100644 --- a/source/util/src/tqueue.c +++ b/source/util/src/tqueue.c @@ -107,7 +107,7 @@ bool taosQueueEmpty(taos_queue param) { if (queue->head == NULL && queue->tail == NULL) { empty = true; } - pthread_mutex_destroy(&queue->mutex); + pthread_mutex_unlock(&queue->mutex); return empty; } diff --git a/source/util/src/tworker.c b/source/util/src/tworker.c index 136bc40482..11972e84cb 100644 --- a/source/util/src/tworker.c +++ b/source/util/src/tworker.c @@ -50,7 +50,7 @@ void tWorkerCleanup(SWorkerPool *pool) { } } - free(pool->workers); + tfree(pool->workers); taosCloseQset(pool->qset); pthread_mutex_destroy(&pool->mutex); @@ -159,7 +159,7 @@ void tMWorkerCleanup(SMWorkerPool *pool) { } } - free(pool->workers); + tfree(pool->workers); pthread_mutex_destroy(&pool->mutex); uInfo("worker:%s is closed", pool->name); From c3a25985a73ea65e9500f01b0e643494194a4eff Mon Sep 17 00:00:00 2001 From: Shengliang Guan Date: Sat, 27 Nov 2021 16:20:47 +0800 Subject: [PATCH 31/44] change lock type in dndDnode --- source/dnode/mgmt/impl/src/dndDnode.c | 81 ++++++++++++----------- source/dnode/mgmt/impl/src/dndTransport.c | 2 + source/dnode/mgmt/impl/src/dnode.c | 5 +- 3 files changed, 46 insertions(+), 42 deletions(-) diff --git a/source/dnode/mgmt/impl/src/dndDnode.c b/source/dnode/mgmt/impl/src/dndDnode.c index fa601a0d99..0f47c5b409 100644 --- a/source/dnode/mgmt/impl/src/dndDnode.c +++ b/source/dnode/mgmt/impl/src/dndDnode.c @@ -18,32 +18,27 @@ #include "dndTransport.h" #include "dndVnodes.h" -static inline void dndRLockDnode(SDnode *pDnode) { taosRLockLatch(&pDnode->dmgmt.latch); } - -static inline void dndRUnLockDnode(SDnode *pDnode) { taosRUnLockLatch(&pDnode->dmgmt.latch); } - -static inline void dndWLockDnode(SDnode *pDnode) { taosWLockLatch(&pDnode->dmgmt.latch); } - -static inline void dndWUnLockDnode(SDnode *pDnode) { taosWUnLockLatch(&pDnode->dmgmt.latch); } - int32_t dndGetDnodeId(SDnode *pDnode) { - dndRLockDnode(pDnode); - int32_t dnodeId = pDnode->dmgmt.dnodeId; - dndRUnLockDnode(pDnode); + SDnodeMgmt *pMgmt = &pDnode->dmgmt; + taosRLockLatch(&pMgmt->latch); + int32_t dnodeId = pMgmt->dnodeId; + taosRUnLockLatch(&pMgmt->latch); return dnodeId; } int64_t dndGetClusterId(SDnode *pDnode) { - dndRLockDnode(pDnode); - int64_t clusterId = pDnode->dmgmt.clusterId; - dndRUnLockDnode(pDnode); + SDnodeMgmt *pMgmt = &pDnode->dmgmt; + taosRLockLatch(&pMgmt->latch); + int64_t clusterId = pMgmt->clusterId; + taosRUnLockLatch(&pMgmt->latch); return clusterId; } void dndGetDnodeEp(SDnode *pDnode, int32_t dnodeId, char *pEp, char *pFqdn, uint16_t *pPort) { - dndRLockDnode(pDnode); + SDnodeMgmt *pMgmt = &pDnode->dmgmt; + taosRLockLatch(&pMgmt->latch); - SDnodeEp *pDnodeEp = taosHashGet(pDnode->dmgmt.dnodeHash, &dnodeId, sizeof(int32_t)); + SDnodeEp *pDnodeEp = taosHashGet(pMgmt->dnodeHash, &dnodeId, sizeof(int32_t)); if (pDnodeEp != NULL) { if (pPort != NULL) { *pPort = pDnodeEp->port; @@ -56,13 +51,14 @@ void dndGetDnodeEp(SDnode *pDnode, int32_t dnodeId, char *pEp, char *pFqdn, uint } } - dndRUnLockDnode(pDnode); + taosRUnLockLatch(&pMgmt->latch); } void dndGetMnodeEpSet(SDnode *pDnode, SEpSet *pEpSet) { - dndRLockDnode(pDnode); - *pEpSet = pDnode->dmgmt.mnodeEpSet; - dndRUnLockDnode(pDnode); + SDnodeMgmt *pMgmt = &pDnode->dmgmt; + taosRLockLatch(&pMgmt->latch); + *pEpSet = pMgmt->mnodeEpSet; + taosRUnLockLatch(&pMgmt->latch); } void dndSendRedirectMsg(SDnode *pDnode, SRpcMsg *pMsg) { @@ -87,14 +83,15 @@ void dndSendRedirectMsg(SDnode *pDnode, SRpcMsg *pMsg) { static void dndUpdateMnodeEpSet(SDnode *pDnode, SEpSet *pEpSet) { dInfo("mnode is changed, num:%d inUse:%d", pEpSet->numOfEps, pEpSet->inUse); - dndWLockDnode(pDnode); + SDnodeMgmt *pMgmt = &pDnode->dmgmt; + taosWLockLatch(&pMgmt->latch); - pDnode->dmgmt.mnodeEpSet = *pEpSet; + pMgmt->mnodeEpSet = *pEpSet; for (int32_t i = 0; i < pEpSet->numOfEps; ++i) { dInfo("mnode index:%d %s:%u", i, pEpSet->fqdn[i], pEpSet->port[i]); } - dndWUnLockDnode(pDnode); + taosWUnLockLatch(&pMgmt->latch); } static void dndPrintDnodes(SDnode *pDnode) { @@ -145,16 +142,18 @@ static void dndResetDnodes(SDnode *pDnode, SDnodeEps *pDnodeEps) { static bool dndIsEpChanged(SDnode *pDnode, int32_t dnodeId, char *pEp) { bool changed = false; - dndRLockDnode(pDnode); - SDnodeEp *pDnodeEp = taosHashGet(pDnode->dmgmt.dnodeHash, &dnodeId, sizeof(int32_t)); + SDnodeMgmt *pMgmt = &pDnode->dmgmt; + taosRLockLatch(&pMgmt->latch); + + SDnodeEp *pDnodeEp = taosHashGet(pMgmt->dnodeHash, &dnodeId, sizeof(int32_t)); if (pDnodeEp != NULL) { char epstr[TSDB_EP_LEN + 1]; snprintf(epstr, TSDB_EP_LEN, "%s:%u", pDnodeEp->fqdn, pDnodeEp->port); changed = strcmp(pEp, epstr) != 0; } - dndRUnLockDnode(pDnode); + taosRUnLockLatch(&pMgmt->latch); return changed; } @@ -342,11 +341,14 @@ static void dndSendStatusMsg(SDnode *pDnode) { return; } - dndRLockDnode(pDnode); + bool changed = false; + + SDnodeMgmt *pMgmt = &pDnode->dmgmt; + taosRLockLatch(&pMgmt->latch); pStatus->sversion = htonl(pDnode->opt.sver); - pStatus->dnodeId = htonl(pDnode->dmgmt.dnodeId); - pStatus->clusterId = htobe64(pDnode->dmgmt.clusterId); - pStatus->rebootTime = htonl(pDnode->dmgmt.rebootTime); + pStatus->dnodeId = htonl(pMgmt->dnodeId); + pStatus->clusterId = htobe64(pMgmt->clusterId); + pStatus->rebootTime = htonl(pMgmt->rebootTime); pStatus->numOfCores = htonl(pDnode->opt.numOfCores); tstrncpy(pStatus->dnodeEp, pDnode->opt.localEp, TSDB_EP_LEN); pStatus->clusterCfg.statusInterval = htonl(pDnode->opt.statusInterval); @@ -356,7 +358,7 @@ static void dndSendStatusMsg(SDnode *pDnode) { pStatus->clusterCfg.checkTime = 0; char timestr[32] = "1970-01-01 00:00:00.00"; (void)taosParseTime(timestr, &pStatus->clusterCfg.checkTime, (int32_t)strlen(timestr), TSDB_TIME_PRECISION_MILLI, 0); - dndRUnLockDnode(pDnode); + taosRUnLockLatch(&pMgmt->latch); dndGetVnodeLoads(pDnode, &pStatus->vnodeLoads); contLen = sizeof(SStatusMsg) + pStatus->vnodeLoads.num * sizeof(SVnodeLoad); @@ -370,32 +372,33 @@ static void dndUpdateDnodeCfg(SDnode *pDnode, SDnodeCfg *pCfg) { if (pMgmt->dnodeId == 0 || pMgmt->dropped != pCfg->dropped) { dInfo("set dnodeId:%d clusterId:%" PRId64 " dropped:%d", pCfg->dnodeId, pCfg->clusterId, pCfg->dropped); - dndWLockDnode(pDnode); + taosWLockLatch(&pMgmt->latch); pMgmt->dnodeId = pCfg->dnodeId; pMgmt->clusterId = pCfg->clusterId; pMgmt->dropped = pCfg->dropped; (void)dndWriteDnodes(pDnode); - dndWUnLockDnode(pDnode); + taosWUnLockLatch(&pMgmt->latch); } } static void dndUpdateDnodeEps(SDnode *pDnode, SDnodeEps *pDnodeEps) { if (pDnodeEps == NULL || pDnodeEps->num <= 0) return; - dndWLockDnode(pDnode); + SDnodeMgmt *pMgmt = &pDnode->dmgmt; + taosWLockLatch(&pMgmt->latch); - if (pDnodeEps->num != pDnode->dmgmt.dnodeEps->num) { + if (pDnodeEps->num != pMgmt->dnodeEps->num) { dndResetDnodes(pDnode, pDnodeEps); dndWriteDnodes(pDnode); } else { int32_t size = pDnodeEps->num * sizeof(SDnodeEp) + sizeof(SDnodeEps); - if (memcmp(pDnode->dmgmt.dnodeEps, pDnodeEps, size) != 0) { + if (memcmp(pMgmt->dnodeEps, pDnodeEps, size) != 0) { dndResetDnodes(pDnode, pDnodeEps); dndWriteDnodes(pDnode); } } - dndWUnLockDnode(pDnode); + taosWUnLockLatch(&pMgmt->latch); } static void dndProcessStatusRsp(SDnode *pDnode, SRpcMsg *pMsg, SEpSet *pEpSet) { @@ -512,7 +515,7 @@ void dndCleanupDnode(SDnode *pDnode) { pMgmt->threadId = NULL; } - dndWLockDnode(pDnode); + taosWLockLatch(&pMgmt->latch); if (pMgmt->dnodeEps != NULL) { free(pMgmt->dnodeEps); @@ -529,7 +532,7 @@ void dndCleanupDnode(SDnode *pDnode) { pMgmt->file = NULL; } - dndWUnLockDnode(pDnode); + taosWUnLockLatch(&pMgmt->latch); dInfo("dnode-dnode is cleaned up"); } diff --git a/source/dnode/mgmt/impl/src/dndTransport.c b/source/dnode/mgmt/impl/src/dndTransport.c index c3940cd3cc..0364c4db38 100644 --- a/source/dnode/mgmt/impl/src/dndTransport.c +++ b/source/dnode/mgmt/impl/src/dndTransport.c @@ -187,6 +187,7 @@ static void dndProcessRequest(void *param, SRpcMsg *pMsg, SEpSet *pEpSet) { int32_t msgType = pMsg->msgType; if (msgType == TSDB_MSG_TYPE_NETWORK_TEST) { + dTrace("RPC %p, network test req will be processed", pMsg->handle); dndProcessDnodeReq(pDnode, pMsg, pEpSet); return; } @@ -206,6 +207,7 @@ static void dndProcessRequest(void *param, SRpcMsg *pMsg, SEpSet *pEpSet) { } if (pMsg->pCont == NULL) { + dTrace("RPC %p, req:%s not processed since content is null", pMsg->handle, taosMsg[msgType]); SRpcMsg rspMsg = {.handle = pMsg->handle, .code = TSDB_CODE_DND_INVALID_MSG_LEN}; rpcSendResponse(&rspMsg); return; diff --git a/source/dnode/mgmt/impl/src/dnode.c b/source/dnode/mgmt/impl/src/dnode.c index 23c9ee0ebf..1159bbb4c5 100644 --- a/source/dnode/mgmt/impl/src/dnode.c +++ b/source/dnode/mgmt/impl/src/dnode.c @@ -26,7 +26,7 @@ EStat dndGetStat(SDnode *pDnode) { return pDnode->stat; } void dndSetStat(SDnode *pDnode, EStat stat) { - dDebug("dnode stat set from %s to %s", dndStatStr(pDnode->stat), dndStatStr(stat)); + dDebug("dnode status set from %s to %s", dndStatStr(pDnode->stat), dndStatStr(stat)); pDnode->stat = stat; } @@ -214,8 +214,7 @@ void dndCleanup(SDnode *pDnode) { dndCleanupDnode(pDnode); walCleanUp(); rpcCleanup(); - - dInfo("TDengine is cleaned up successfully"); dndCleanupEnv(pDnode); free(pDnode); + dInfo("TDengine is cleaned up successfully"); } From c4306958c788d5b828336cfee34387e16e33ded2 Mon Sep 17 00:00:00 2001 From: Shengliang Guan Date: Sat, 27 Nov 2021 17:23:57 +0800 Subject: [PATCH 32/44] TD-11265 fix transport in dnode-mgmt --- include/dnode/mnode/mnode.h | 12 +++---- source/dnode/mgmt/impl/src/dndDnode.c | 3 ++ source/dnode/mgmt/impl/src/dndMnode.c | 38 ++++++++++++++--------- source/dnode/mgmt/impl/src/dndTransport.c | 6 ++-- source/dnode/mnode/impl/src/mnode.c | 8 ++--- 5 files changed, 37 insertions(+), 30 deletions(-) diff --git a/include/dnode/mnode/mnode.h b/include/dnode/mnode/mnode.h index 824eb24191..2ffbd395e8 100644 --- a/include/dnode/mnode/mnode.h +++ b/include/dnode/mnode/mnode.h @@ -137,38 +137,34 @@ void mnodeSendRsp(SMnodeMsg *pMsg, int32_t code); /** * @brief Process the read request * - * @param pMnode The mnode object * @param pMsg The request msg * @return int32_t 0 for success, -1 for failure */ -void mnodeProcessReadMsg(SMnode *pMnode, SMnodeMsg *pMsg); +void mnodeProcessReadMsg(SMnodeMsg *pMsg); /** * @brief Process the write request * - * @param pMnode The mnode object * @param pMsg The request msg * @return int32_t 0 for success, -1 for failure */ -void mnodeProcessWriteMsg(SMnode *pMnode, SMnodeMsg *pMsg); +void mnodeProcessWriteMsg(SMnodeMsg *pMsg); /** * @brief Process the sync request * - * @param pMnode The mnode object * @param pMsg The request msg * @return int32_t 0 for success, -1 for failure */ -void mnodeProcessSyncMsg(SMnode *pMnode, SMnodeMsg *pMsg); +void mnodeProcessSyncMsg(SMnodeMsg *pMsg); /** * @brief Process the apply request * - * @param pMnode The mnode object * @param pMsg The request msg * @return int32_t 0 for success, -1 for failure */ -void mnodeProcessApplyMsg(SMnode *pMnode, SMnodeMsg *pMsg); +void mnodeProcessApplyMsg(SMnodeMsg *pMsg); #ifdef __cplusplus } diff --git a/source/dnode/mgmt/impl/src/dndDnode.c b/source/dnode/mgmt/impl/src/dndDnode.c index 0f47c5b409..9e70bf1ae9 100644 --- a/source/dnode/mgmt/impl/src/dndDnode.c +++ b/source/dnode/mgmt/impl/src/dndDnode.c @@ -121,12 +121,14 @@ static void dndResetDnodes(SDnode *pDnode, SDnodeEps *pDnodeEps) { } pMgmt->mnodeEpSet.inUse = 0; + pMgmt->mnodeEpSet.numOfEps = 0; int32_t mIndex = 0; for (int32_t i = 0; i < pMgmt->dnodeEps->num; i++) { SDnodeEp *pDnodeEp = &pMgmt->dnodeEps->eps[i]; if (!pDnodeEp->isMnode) continue; if (mIndex >= TSDB_MAX_REPLICA) continue; + pMgmt->mnodeEpSet.numOfEps++; strcpy(pMgmt->mnodeEpSet.fqdn[mIndex], pDnodeEp->fqdn); pMgmt->mnodeEpSet.port[mIndex] = pDnodeEp->port; mIndex++; @@ -279,6 +281,7 @@ PRASE_DNODE_OVER: if (pMgmt->dnodeEps == NULL) { pMgmt->dnodeEps = calloc(1, sizeof(SDnodeEps) + sizeof(SDnodeEp)); pMgmt->dnodeEps->num = 1; + pMgmt->dnodeEps->eps[0].isMnode = 1; pMgmt->dnodeEps->eps[0].port = pDnode->opt.serverPort; tstrncpy(pMgmt->dnodeEps->eps[0].fqdn, pDnode->opt.localFqdn, TSDB_FQDN_LEN); } diff --git a/source/dnode/mgmt/impl/src/dndMnode.c b/source/dnode/mgmt/impl/src/dndMnode.c index fe3accdd84..d42891f1e4 100644 --- a/source/dnode/mgmt/impl/src/dndMnode.c +++ b/source/dnode/mgmt/impl/src/dndMnode.c @@ -80,7 +80,9 @@ static SMnode *dndAcquireMnode(SDnode *pDnode) { } taosRUnLockLatch(&pMgmt->latch); - dTrace("acquire mnode, refCount:%d", refCount); + if (pMnode != NULL) { + dTrace("acquire mnode, refCount:%d", refCount); + } return pMnode; } @@ -94,7 +96,9 @@ static void dndReleaseMnode(SDnode *pDnode, SMnode *pMnode) { } taosRUnLockLatch(&pMgmt->latch); - dTrace("release mnode, refCount:%d", refCount); + if (pMnode != NULL) { + dTrace("release mnode, refCount:%d", refCount); + } } static int32_t dndReadMnodeFile(SDnode *pDnode) { @@ -550,7 +554,7 @@ static void dndProcessMnodeReadQueue(SDnode *pDnode, SMnodeMsg *pMsg) { SMnode *pMnode = dndAcquireMnode(pDnode); if (pMnode != NULL) { - mnodeProcessReadMsg(pMnode, pMsg); + mnodeProcessReadMsg(pMsg); dndReleaseMnode(pDnode, pMnode); } else { mnodeSendRsp(pMsg, terrno); @@ -564,7 +568,7 @@ static void dndProcessMnodeWriteQueue(SDnode *pDnode, SMnodeMsg *pMsg) { SMnode *pMnode = dndAcquireMnode(pDnode); if (pMnode != NULL) { - mnodeProcessWriteMsg(pMnode, pMsg); + mnodeProcessWriteMsg(pMsg); dndReleaseMnode(pDnode, pMnode); } else { mnodeSendRsp(pMsg, terrno); @@ -578,7 +582,7 @@ static void dndProcessMnodeApplyQueue(SDnode *pDnode, SMnodeMsg *pMsg) { SMnode *pMnode = dndAcquireMnode(pDnode); if (pMnode != NULL) { - mnodeProcessApplyMsg(pMnode, pMsg); + mnodeProcessApplyMsg(pMsg); dndReleaseMnode(pDnode, pMnode); } else { mnodeSendRsp(pMsg, terrno); @@ -592,7 +596,7 @@ static void dndProcessMnodeSyncQueue(SDnode *pDnode, SMnodeMsg *pMsg) { SMnode *pMnode = dndAcquireMnode(pDnode); if (pMnode != NULL) { - mnodeProcessSyncMsg(pMnode, pMsg); + mnodeProcessSyncMsg(pMsg); dndReleaseMnode(pDnode, pMnode); } else { mnodeSendRsp(pMsg, terrno); @@ -683,7 +687,7 @@ static int32_t dndPutMsgIntoMnodeApplyQueue(SDnode *pDnode, SMnodeMsg *pMsg) { static int32_t dndAllocMnodeMgmtQueue(SDnode *pDnode) { SMnodeMgmt *pMgmt = &pDnode->mmgmt; - pMgmt->pMgmtQ = tWorkerAllocQueue(&pMgmt->mgmtPool, NULL, (FProcessItem)dndProcessMnodeMgmtQueue); + pMgmt->pMgmtQ = tWorkerAllocQueue(&pMgmt->mgmtPool, pDnode, (FProcessItem)dndProcessMnodeMgmtQueue); if (pMgmt->pMgmtQ == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; return -1; @@ -708,18 +712,19 @@ static int32_t dndInitMnodeMgmtWorker(SDnode *pDnode) { return -1; } + dDebug("mnode mgmt worker is initialized"); return 0; } static void dndCleanupMnodeMgmtWorker(SDnode *pDnode) { SMnodeMgmt *pMgmt = &pDnode->mmgmt; tWorkerCleanup(&pMgmt->mgmtPool); - dDebug("mnode mgmt worker is stopped"); + dDebug("mnode mgmt worker is closed"); } static int32_t dndAllocMnodeReadQueue(SDnode *pDnode) { SMnodeMgmt *pMgmt = &pDnode->mmgmt; - pMgmt->pReadQ = tWorkerAllocQueue(&pMgmt->readPool, NULL, (FProcessItem)dndProcessMnodeReadQueue); + pMgmt->pReadQ = tWorkerAllocQueue(&pMgmt->readPool, pDnode, (FProcessItem)dndProcessMnodeReadQueue); if (pMgmt->pReadQ == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; return -1; @@ -745,18 +750,19 @@ static int32_t dndInitMnodeReadWorker(SDnode *pDnode) { return -1; } + dDebug("mnode read worker is initialized"); return 0; } static void dndCleanupMnodeReadWorker(SDnode *pDnode) { SMnodeMgmt *pMgmt = &pDnode->mmgmt; tWorkerCleanup(&pMgmt->readPool); - dDebug("mnode read worker is stopped"); + dDebug("mnode read worker is closed"); } static int32_t dndAllocMnodeWriteQueue(SDnode *pDnode) { SMnodeMgmt *pMgmt = &pDnode->mmgmt; - pMgmt->pWriteQ = tWorkerAllocQueue(&pMgmt->writePool, NULL, (FProcessItem)dndProcessMnodeWriteQueue); + pMgmt->pWriteQ = tWorkerAllocQueue(&pMgmt->writePool, pDnode, (FProcessItem)dndProcessMnodeWriteQueue); if (pMgmt->pWriteQ == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; return -1; @@ -773,7 +779,7 @@ static void dndFreeMnodeWriteQueue(SDnode *pDnode) { static int32_t dndAllocMnodeApplyQueue(SDnode *pDnode) { SMnodeMgmt *pMgmt = &pDnode->mmgmt; - pMgmt->pApplyQ = tWorkerAllocQueue(&pMgmt->writePool, NULL, (FProcessItem)dndProcessMnodeApplyQueue); + pMgmt->pApplyQ = tWorkerAllocQueue(&pMgmt->writePool, pDnode, (FProcessItem)dndProcessMnodeApplyQueue); if (pMgmt->pApplyQ == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; return -1; @@ -799,18 +805,19 @@ static int32_t dndInitMnodeWriteWorker(SDnode *pDnode) { return -1; } + dDebug("mnode write worker is initialized"); return 0; } static void dndCleanupMnodeWriteWorker(SDnode *pDnode) { SMnodeMgmt *pMgmt = &pDnode->mmgmt; tWorkerCleanup(&pMgmt->writePool); - dDebug("mnode write worker is stopped"); + dDebug("mnode write worker is closed"); } static int32_t dndAllocMnodeSyncQueue(SDnode *pDnode) { SMnodeMgmt *pMgmt = &pDnode->mmgmt; - pMgmt->pSyncQ = tWorkerAllocQueue(&pMgmt->syncPool, NULL, (FProcessItem)dndProcessMnodeSyncQueue); + pMgmt->pSyncQ = tWorkerAllocQueue(&pMgmt->syncPool, pDnode, (FProcessItem)dndProcessMnodeSyncQueue); if (pMgmt->pSyncQ == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; return -1; @@ -836,13 +843,14 @@ static int32_t dndInitMnodeSyncWorker(SDnode *pDnode) { return -1; } + dDebug("mnode sync worker is initialized"); return 0; } static void dndCleanupMnodeSyncWorker(SDnode *pDnode) { SMnodeMgmt *pMgmt = &pDnode->mmgmt; tWorkerCleanup(&pMgmt->syncPool); - dDebug("mnode sync worker is stopped"); + dDebug("mnode sync worker is closed"); } int32_t dndInitMnode(SDnode *pDnode) { diff --git a/source/dnode/mgmt/impl/src/dndTransport.c b/source/dnode/mgmt/impl/src/dndTransport.c index 0364c4db38..eee2e36c0f 100644 --- a/source/dnode/mgmt/impl/src/dndTransport.c +++ b/source/dnode/mgmt/impl/src/dndTransport.c @@ -261,12 +261,12 @@ static int32_t dndRetrieveUserAuthInfo(void *parent, char *user, char *spi, char SDnode *pDnode = parent; if (dndAuthInternalMsg(parent, user, spi, encrypt, secret, ckey) == 0) { - dTrace("get internal auth success"); + // dTrace("get internal auth success"); return 0; } if (dndGetUserAuthFromMnode(pDnode, user, spi, encrypt, secret, ckey) == 0) { - dTrace("get auth from internal mnode"); + // dTrace("get auth from internal mnode"); return 0; } @@ -275,7 +275,7 @@ static int32_t dndRetrieveUserAuthInfo(void *parent, char *user, char *spi, char return -1; } - dDebug("user:%s, send auth msg to other mnodes", user); + // dDebug("user:%s, send auth msg to other mnodes", user); SAuthMsg *pMsg = rpcMallocCont(sizeof(SAuthMsg)); tstrncpy(pMsg->user, user, TSDB_USER_LEN); diff --git a/source/dnode/mnode/impl/src/mnode.c b/source/dnode/mnode/impl/src/mnode.c index 8fc5de588f..128a834729 100644 --- a/source/dnode/mnode/impl/src/mnode.c +++ b/source/dnode/mnode/impl/src/mnode.c @@ -234,13 +234,13 @@ void mnodeSetMsgFp(int32_t msgType, MnodeRpcFp fp) { } } -void mnodeProcessReadMsg(SMnode *pMnode, SMnodeMsg *pMsg) { mnodeProcessRpcMsg(pMsg); } +void mnodeProcessReadMsg(SMnodeMsg *pMsg) { mnodeProcessRpcMsg(pMsg); } -void mnodeProcessWriteMsg(SMnode *pMnode, SMnodeMsg *pMsg) { mnodeProcessRpcMsg(pMsg); } +void mnodeProcessWriteMsg(SMnodeMsg *pMsg) { mnodeProcessRpcMsg(pMsg); } -void mnodeProcessSyncMsg(SMnode *pMnode, SMnodeMsg *pMsg) { mnodeProcessRpcMsg(pMsg); } +void mnodeProcessSyncMsg(SMnodeMsg *pMsg) { mnodeProcessRpcMsg(pMsg); } -void mnodeProcessApplyMsg(SMnode *pMnode, SMnodeMsg *pMsg) {} +void mnodeProcessApplyMsg(SMnodeMsg *pMsg) {} #if 0 From 5cebfe33f4e8c4f6398dc304e93434cafa1d4f7e Mon Sep 17 00:00:00 2001 From: Shengliang Guan Date: Sat, 27 Nov 2021 22:46:12 +0800 Subject: [PATCH 33/44] minor changes in mnode --- source/dnode/mnode/impl/inc/mnodeInt.h | 21 ++++++++++----------- source/dnode/mnode/impl/src/mnode.c | 8 ++++---- source/dnode/mnode/impl/src/mnodeUser.c | 4 ++-- 3 files changed, 16 insertions(+), 17 deletions(-) diff --git a/source/dnode/mnode/impl/inc/mnodeInt.h b/source/dnode/mnode/impl/inc/mnodeInt.h index 7f7f91a3af..16322eeb55 100644 --- a/source/dnode/mnode/impl/inc/mnodeInt.h +++ b/source/dnode/mnode/impl/inc/mnodeInt.h @@ -24,16 +24,16 @@ extern "C" { #endif -typedef int32_t (*MnodeRpcFp)(SMnodeMsg *pMsg); +typedef int32_t (*MndMsgFp)(SMnode *pMnode, SMnodeMsg *pMsg); typedef struct SMnodeBak { - int32_t dnodeId; - int64_t clusterId; - tmr_h timer; - SSteps *pInitSteps; - SSteps *pStartSteps; - SMnodeOpt para; - MnodeRpcFp msgFp[TSDB_MSG_TYPE_MAX]; + int32_t dnodeId; + int64_t clusterId; + tmr_h timer; + SSteps *pInitSteps; + SSteps *pStartSteps; + SMnodeOpt para; + MndMsgFp msgFp[TSDB_MSG_TYPE_MAX]; } SMnodeBak; typedef struct SMnode { @@ -47,7 +47,7 @@ typedef struct SMnode { SSteps *pStartSteps; struct SSdb *pSdb; struct SDnode *pServer; - MnodeRpcFp msgFp[TSDB_MSG_TYPE_MAX]; + MndMsgFp msgFp[TSDB_MSG_TYPE_MAX]; PutMsgToMnodeQFp putMsgToApplyMsgFp; SendMsgToDnodeFp sendMsgToDnodeFp; SendMsgToMnodeFp sendMsgToMnodeFp; @@ -61,8 +61,7 @@ int64_t mnodeGetClusterId(); void mnodeSendMsgToDnode(SMnode *pMnode, struct SEpSet *epSet, struct SRpcMsg *rpcMsg); void mnodeSendMsgToMnode(SMnode *pMnode, struct SRpcMsg *rpcMsg); void mnodeSendRedirectMsg(SMnode *pMnode, struct SRpcMsg *rpcMsg, bool forShell); - -void mnodeSetMsgFp(int32_t msgType, MnodeRpcFp fp); +void mnodeSetMsgHandle(SMnode *pMnode, int32_t msgType, MndMsgFp fp); #ifdef __cplusplus } diff --git a/source/dnode/mnode/impl/src/mnode.c b/source/dnode/mnode/impl/src/mnode.c index 128a834729..6ea47b41fc 100644 --- a/source/dnode/mnode/impl/src/mnode.c +++ b/source/dnode/mnode/impl/src/mnode.c @@ -218,18 +218,18 @@ static void mnodeProcessRpcMsg(SMnodeMsg *pMsg) { int32_t msgType = pMsg->rpcMsg.msgType; - MnodeRpcFp fp = tsMint.msgFp[msgType]; + MndMsgFp fp = tsMint.msgFp[msgType]; if (fp == NULL) { } - int32_t code = (fp)(pMsg); + int32_t code = (*fp)(NULL, pMsg); if (code != 0) { assert(code); } } -void mnodeSetMsgFp(int32_t msgType, MnodeRpcFp fp) { - if (msgType > 0 || msgType < TSDB_MSG_TYPE_MAX) { +void mnodeSetMsgHandle(SMnode *pMnode, int32_t msgType, MndMsgFp fp) { + if (msgType >= 0 && msgType < TSDB_MSG_TYPE_MAX) { tsMint.msgFp[msgType] = fp; } } diff --git a/source/dnode/mnode/impl/src/mnodeUser.c b/source/dnode/mnode/impl/src/mnodeUser.c index b8e5706484..e67fedd5eb 100644 --- a/source/dnode/mnode/impl/src/mnodeUser.c +++ b/source/dnode/mnode/impl/src/mnodeUser.c @@ -179,7 +179,7 @@ static int32_t mnodeCreateUser(char *acct, char *user, char *pass, SMnodeMsg *pM return 0; } -static int32_t mnodeProcessCreateUserMsg(SMnodeMsg *pMsg) { +static int32_t mnodeProcessCreateUserMsg(SMnode *pMnode, SMnodeMsg *pMsg) { SCreateUserMsg *pCreate = pMsg->rpcMsg.pCont; if (pCreate->user[0] == 0) { @@ -231,7 +231,7 @@ int32_t mnodeInitUser() { .deleteFp = (SdbDeleteFp)mnodeUserActionDelete}; sdbSetTable(table); - mnodeSetMsgFp(TSDB_MSG_TYPE_CREATE_USER, mnodeProcessCreateUserMsg); + mnodeSetMsgHandle(NULL, TSDB_MSG_TYPE_CREATE_USER, mnodeProcessCreateUserMsg); return 0; } From ad361ee1df8ea3abe9e492172454245ea6a5d564 Mon Sep 17 00:00:00 2001 From: Shengliang Guan Date: Sat, 27 Nov 2021 22:56:18 +0800 Subject: [PATCH 34/44] rename files --- .../mnode/impl/inc/{mnodeAcct.h => mndAcct.h} | 2 +- .../mnode/impl/inc/{mnodeAuth.h => mndAuth.h} | 2 +- .../impl/inc/{mnodeBalance.h => mndBalance.h} | 2 +- .../impl/inc/{mnodeCluster.h => mndCluster.h} | 2 +- .../mnode/impl/inc/{mnodeDb.h => mndDb.h} | 2 +- .../mnode/impl/inc/{mnodeDef.h => mndDef.h} | 0 .../impl/inc/{mnodeDnode.h => mndDnode.h} | 2 +- .../mnode/impl/inc/{mnodeFunc.h => mndFunc.h} | 2 +- .../mnode/impl/inc/{mnodeInt.h => mndInt.h} | 2 +- .../impl/inc/{mnodeMnode.h => mndMnode.h} | 2 +- .../mnode/impl/inc/{mnodeOper.h => mndOper.h} | 0 .../impl/inc/{mnodeProfile.h => mndProfile.h} | 2 +- .../mnode/impl/inc/{mnodeShow.h => mndShow.h} | 2 +- .../impl/inc/{mnodeStable.h => mndStable.h} | 2 +- .../mnode/impl/inc/{mnodeSync.h => mndSync.h} | 2 +- .../impl/inc/{mnodeTelem.h => mndTelem.h} | 2 +- .../impl/inc/{mnodeTrans.h => mndTrans.h} | 2 +- .../mnode/impl/inc/{mnodeUser.h => mndUser.h} | 2 +- .../impl/inc/{mnodeVgroup.h => mndVgroup.h} | 2 +- .../mnode/impl/src/{mnodeAcct.c => mndAcct.c} | 2 +- .../mnode/impl/src/{mnodeAuth.c => mndAuth.c} | 2 +- .../impl/src/{mnodeBalance.c => mndBalance.c} | 2 +- .../impl/src/{mnodeCluster.c => mndCluster.c} | 2 +- .../mnode/impl/src/{mnodeDb.c => mndDb.c} | 2 +- .../impl/src/{mnodeDnode.c => mndDnode.c} | 2 +- .../mnode/impl/src/{mnodeFunc.c => mndFunc.c} | 2 +- .../impl/src/{mnodeMnode.c => mndMnode.c} | 2 +- .../mnode/impl/src/{mnodeOper.c => mndOper.c} | 2 +- .../impl/src/{mnodeProfile.c => mndProfile.c} | 2 +- .../mnode/impl/src/{mnodeShow.c => mndShow.c} | 2 +- .../impl/src/{mnodeStable.c => mndStable.c} | 2 +- .../mnode/impl/src/{mnodeSync.c => mndSync.c} | 4 +-- .../impl/src/{mnodeTelem.c => mndTelem.c} | 4 +-- .../impl/src/{mnodeTrans.c => mndTrans.c} | 2 +- .../mnode/impl/src/{mnodeUser.c => mndUser.c} | 4 +-- .../impl/src/{mnodeVgroup.c => mndVgroup.c} | 2 +- source/dnode/mnode/impl/src/mnode.c | 34 +++++++++---------- 37 files changed, 54 insertions(+), 54 deletions(-) rename source/dnode/mnode/impl/inc/{mnodeAcct.h => mndAcct.h} (97%) rename source/dnode/mnode/impl/inc/{mnodeAuth.h => mndAuth.h} (97%) rename source/dnode/mnode/impl/inc/{mnodeBalance.h => mndBalance.h} (97%) rename source/dnode/mnode/impl/inc/{mnodeCluster.h => mndCluster.h} (97%) rename source/dnode/mnode/impl/inc/{mnodeDb.h => mndDb.h} (97%) rename source/dnode/mnode/impl/inc/{mnodeDef.h => mndDef.h} (100%) rename source/dnode/mnode/impl/inc/{mnodeDnode.h => mndDnode.h} (97%) rename source/dnode/mnode/impl/inc/{mnodeFunc.h => mndFunc.h} (97%) rename source/dnode/mnode/impl/inc/{mnodeInt.h => mndInt.h} (98%) rename source/dnode/mnode/impl/inc/{mnodeMnode.h => mndMnode.h} (97%) rename source/dnode/mnode/impl/inc/{mnodeOper.h => mndOper.h} (100%) rename source/dnode/mnode/impl/inc/{mnodeProfile.h => mndProfile.h} (97%) rename source/dnode/mnode/impl/inc/{mnodeShow.h => mndShow.h} (97%) rename source/dnode/mnode/impl/inc/{mnodeStable.h => mndStable.h} (97%) rename source/dnode/mnode/impl/inc/{mnodeSync.h => mndSync.h} (97%) rename source/dnode/mnode/impl/inc/{mnodeTelem.h => mndTelem.h} (97%) rename source/dnode/mnode/impl/inc/{mnodeTrans.h => mndTrans.h} (98%) rename source/dnode/mnode/impl/inc/{mnodeUser.h => mndUser.h} (97%) rename source/dnode/mnode/impl/inc/{mnodeVgroup.h => mndVgroup.h} (97%) rename source/dnode/mnode/impl/src/{mnodeAcct.c => mndAcct.c} (99%) rename source/dnode/mnode/impl/src/{mnodeAuth.c => mndAuth.c} (97%) rename source/dnode/mnode/impl/src/{mnodeBalance.c => mndBalance.c} (97%) rename source/dnode/mnode/impl/src/{mnodeCluster.c => mndCluster.c} (97%) rename source/dnode/mnode/impl/src/{mnodeDb.c => mndDb.c} (97%) rename source/dnode/mnode/impl/src/{mnodeDnode.c => mndDnode.c} (97%) rename source/dnode/mnode/impl/src/{mnodeFunc.c => mndFunc.c} (97%) rename source/dnode/mnode/impl/src/{mnodeMnode.c => mndMnode.c} (97%) rename source/dnode/mnode/impl/src/{mnodeOper.c => mndOper.c} (97%) rename source/dnode/mnode/impl/src/{mnodeProfile.c => mndProfile.c} (97%) rename source/dnode/mnode/impl/src/{mnodeShow.c => mndShow.c} (97%) rename source/dnode/mnode/impl/src/{mnodeStable.c => mndStable.c} (97%) rename source/dnode/mnode/impl/src/{mnodeSync.c => mndSync.c} (95%) rename source/dnode/mnode/impl/src/{mnodeTelem.c => mndTelem.c} (99%) rename source/dnode/mnode/impl/src/{mnodeTrans.c => mndTrans.c} (99%) rename source/dnode/mnode/impl/src/{mnodeUser.c => mndUser.c} (99%) rename source/dnode/mnode/impl/src/{mnodeVgroup.c => mndVgroup.c} (97%) diff --git a/source/dnode/mnode/impl/inc/mnodeAcct.h b/source/dnode/mnode/impl/inc/mndAcct.h similarity index 97% rename from source/dnode/mnode/impl/inc/mnodeAcct.h rename to source/dnode/mnode/impl/inc/mndAcct.h index 9491410ddf..a45d865c03 100644 --- a/source/dnode/mnode/impl/inc/mnodeAcct.h +++ b/source/dnode/mnode/impl/inc/mndAcct.h @@ -16,7 +16,7 @@ #ifndef _TD_MNODE_ACCT_H_ #define _TD_MNODE_ACCT_H_ -#include "mnodeInt.h" +#include "mndInt.h" #ifdef __cplusplus extern "C" { diff --git a/source/dnode/mnode/impl/inc/mnodeAuth.h b/source/dnode/mnode/impl/inc/mndAuth.h similarity index 97% rename from source/dnode/mnode/impl/inc/mnodeAuth.h rename to source/dnode/mnode/impl/inc/mndAuth.h index fc828aca27..a759e0169e 100644 --- a/source/dnode/mnode/impl/inc/mnodeAuth.h +++ b/source/dnode/mnode/impl/inc/mndAuth.h @@ -16,7 +16,7 @@ #ifndef _TD_MNODE_AUTH_H_ #define _TD_MNODE_AUTH_H_ -#include "mnodeInt.h" +#include "mndInt.h" #ifdef __cplusplus extern "C" { diff --git a/source/dnode/mnode/impl/inc/mnodeBalance.h b/source/dnode/mnode/impl/inc/mndBalance.h similarity index 97% rename from source/dnode/mnode/impl/inc/mnodeBalance.h rename to source/dnode/mnode/impl/inc/mndBalance.h index c851d58c30..84de9bcb32 100644 --- a/source/dnode/mnode/impl/inc/mnodeBalance.h +++ b/source/dnode/mnode/impl/inc/mndBalance.h @@ -16,7 +16,7 @@ #ifndef _TD_MNODE_BALANCE_H_ #define _TD_MNODE_BALANCE_H_ -#include "mnodeInt.h" +#include "mndInt.h" #ifdef __cplusplus extern "C" { diff --git a/source/dnode/mnode/impl/inc/mnodeCluster.h b/source/dnode/mnode/impl/inc/mndCluster.h similarity index 97% rename from source/dnode/mnode/impl/inc/mnodeCluster.h rename to source/dnode/mnode/impl/inc/mndCluster.h index 06001a7757..f9047e6e25 100644 --- a/source/dnode/mnode/impl/inc/mnodeCluster.h +++ b/source/dnode/mnode/impl/inc/mndCluster.h @@ -16,7 +16,7 @@ #ifndef _TD_MNODE_CLUSTER_H_ #define _TD_MNODE_CLUSTER_H_ -#include "mnodeInt.h" +#include "mndInt.h" #ifdef __cplusplus extern "C" { diff --git a/source/dnode/mnode/impl/inc/mnodeDb.h b/source/dnode/mnode/impl/inc/mndDb.h similarity index 97% rename from source/dnode/mnode/impl/inc/mnodeDb.h rename to source/dnode/mnode/impl/inc/mndDb.h index ba16235348..1ab4844873 100644 --- a/source/dnode/mnode/impl/inc/mnodeDb.h +++ b/source/dnode/mnode/impl/inc/mndDb.h @@ -16,7 +16,7 @@ #ifndef _TD_MNODE_DATABASE_H_ #define _TD_MNODE_DATABASE_H_ -#include "mnodeInt.h" +#include "mndInt.h" #ifdef __cplusplus extern "C" { diff --git a/source/dnode/mnode/impl/inc/mnodeDef.h b/source/dnode/mnode/impl/inc/mndDef.h similarity index 100% rename from source/dnode/mnode/impl/inc/mnodeDef.h rename to source/dnode/mnode/impl/inc/mndDef.h diff --git a/source/dnode/mnode/impl/inc/mnodeDnode.h b/source/dnode/mnode/impl/inc/mndDnode.h similarity index 97% rename from source/dnode/mnode/impl/inc/mnodeDnode.h rename to source/dnode/mnode/impl/inc/mndDnode.h index ccbd359f38..7b25061b95 100644 --- a/source/dnode/mnode/impl/inc/mnodeDnode.h +++ b/source/dnode/mnode/impl/inc/mndDnode.h @@ -16,7 +16,7 @@ #ifndef _TD_MNODE_DNODE_H_ #define _TD_MNODE_DNODE_H_ -#include "mnodeInt.h" +#include "mndInt.h" #ifdef __cplusplus extern "C" { diff --git a/source/dnode/mnode/impl/inc/mnodeFunc.h b/source/dnode/mnode/impl/inc/mndFunc.h similarity index 97% rename from source/dnode/mnode/impl/inc/mnodeFunc.h rename to source/dnode/mnode/impl/inc/mndFunc.h index 18154d1c00..ff1ebde8b0 100644 --- a/source/dnode/mnode/impl/inc/mnodeFunc.h +++ b/source/dnode/mnode/impl/inc/mndFunc.h @@ -16,7 +16,7 @@ #ifndef _TD_MNODE_FUNC_H_ #define _TD_MNODE_FUNC_H_ -#include "mnodeInt.h" +#include "mndInt.h" #ifdef __cplusplus extern "C" { diff --git a/source/dnode/mnode/impl/inc/mnodeInt.h b/source/dnode/mnode/impl/inc/mndInt.h similarity index 98% rename from source/dnode/mnode/impl/inc/mnodeInt.h rename to source/dnode/mnode/impl/inc/mndInt.h index 16322eeb55..17db21fe18 100644 --- a/source/dnode/mnode/impl/inc/mnodeInt.h +++ b/source/dnode/mnode/impl/inc/mndInt.h @@ -16,7 +16,7 @@ #ifndef _TD_MNODE_INT_H_ #define _TD_MNODE_INT_H_ -#include "mnodeDef.h" +#include "mndDef.h" #include "sdb.h" #include "tstep.h" diff --git a/source/dnode/mnode/impl/inc/mnodeMnode.h b/source/dnode/mnode/impl/inc/mndMnode.h similarity index 97% rename from source/dnode/mnode/impl/inc/mnodeMnode.h rename to source/dnode/mnode/impl/inc/mndMnode.h index 0c8069a917..9e7f1766a6 100644 --- a/source/dnode/mnode/impl/inc/mnodeMnode.h +++ b/source/dnode/mnode/impl/inc/mndMnode.h @@ -16,7 +16,7 @@ #ifndef _TD_MNODE_MNODE_H_ #define _TD_MNODE_MNODE_H_ -#include "mnodeInt.h" +#include "mndInt.h" #ifdef __cplusplus extern "C" { diff --git a/source/dnode/mnode/impl/inc/mnodeOper.h b/source/dnode/mnode/impl/inc/mndOper.h similarity index 100% rename from source/dnode/mnode/impl/inc/mnodeOper.h rename to source/dnode/mnode/impl/inc/mndOper.h diff --git a/source/dnode/mnode/impl/inc/mnodeProfile.h b/source/dnode/mnode/impl/inc/mndProfile.h similarity index 97% rename from source/dnode/mnode/impl/inc/mnodeProfile.h rename to source/dnode/mnode/impl/inc/mndProfile.h index b3735a02a3..405a6f8dac 100644 --- a/source/dnode/mnode/impl/inc/mnodeProfile.h +++ b/source/dnode/mnode/impl/inc/mndProfile.h @@ -16,7 +16,7 @@ #ifndef _TD_MNODE_PROFILE_H_ #define _TD_MNODE_PROFILE_H_ -#include "mnodeInt.h" +#include "mndInt.h" #ifdef __cplusplus extern "C" { diff --git a/source/dnode/mnode/impl/inc/mnodeShow.h b/source/dnode/mnode/impl/inc/mndShow.h similarity index 97% rename from source/dnode/mnode/impl/inc/mnodeShow.h rename to source/dnode/mnode/impl/inc/mndShow.h index 50beee2897..5045fde1a8 100644 --- a/source/dnode/mnode/impl/inc/mnodeShow.h +++ b/source/dnode/mnode/impl/inc/mndShow.h @@ -16,7 +16,7 @@ #ifndef _TD_MNODE_SHOW_H_ #define _TD_MNODE_SHOW_H_ -#include "mnodeInt.h" +#include "mndInt.h" #ifdef __cplusplus extern "C" { diff --git a/source/dnode/mnode/impl/inc/mnodeStable.h b/source/dnode/mnode/impl/inc/mndStable.h similarity index 97% rename from source/dnode/mnode/impl/inc/mnodeStable.h rename to source/dnode/mnode/impl/inc/mndStable.h index e81fa74d0f..f55d7737de 100644 --- a/source/dnode/mnode/impl/inc/mnodeStable.h +++ b/source/dnode/mnode/impl/inc/mndStable.h @@ -16,7 +16,7 @@ #ifndef _TD_MNODE_STABLE_H_ #define _TD_MNODE_STABLE_H_ -#include "mnodeInt.h" +#include "mndInt.h" #ifdef __cplusplus extern "C" { diff --git a/source/dnode/mnode/impl/inc/mnodeSync.h b/source/dnode/mnode/impl/inc/mndSync.h similarity index 97% rename from source/dnode/mnode/impl/inc/mnodeSync.h rename to source/dnode/mnode/impl/inc/mndSync.h index 380ad36e23..8877bc9c82 100644 --- a/source/dnode/mnode/impl/inc/mnodeSync.h +++ b/source/dnode/mnode/impl/inc/mndSync.h @@ -16,7 +16,7 @@ #ifndef _TD_MNODE_SYNC_H_ #define _TD_MNODE_SYNC_H_ -#include "mnodeInt.h" +#include "mndInt.h" #ifdef __cplusplus extern "C" { diff --git a/source/dnode/mnode/impl/inc/mnodeTelem.h b/source/dnode/mnode/impl/inc/mndTelem.h similarity index 97% rename from source/dnode/mnode/impl/inc/mnodeTelem.h rename to source/dnode/mnode/impl/inc/mndTelem.h index 9122775a31..bb2e6635b1 100644 --- a/source/dnode/mnode/impl/inc/mnodeTelem.h +++ b/source/dnode/mnode/impl/inc/mndTelem.h @@ -19,7 +19,7 @@ #ifdef __cplusplus extern "C" { #endif -#include "mnodeInt.h" +#include "mndInt.h" int32_t mnodeInitTelem(); void mnodeCleanupTelem(); diff --git a/source/dnode/mnode/impl/inc/mnodeTrans.h b/source/dnode/mnode/impl/inc/mndTrans.h similarity index 98% rename from source/dnode/mnode/impl/inc/mnodeTrans.h rename to source/dnode/mnode/impl/inc/mndTrans.h index 2abe101dfd..ba6b9fff79 100644 --- a/source/dnode/mnode/impl/inc/mnodeTrans.h +++ b/source/dnode/mnode/impl/inc/mndTrans.h @@ -16,7 +16,7 @@ #ifndef _TD_TRANSACTION_INT_H_ #define _TD_TRANSACTION_INT_H_ -#include "mnodeInt.h" +#include "mndInt.h" #ifdef __cplusplus extern "C" { diff --git a/source/dnode/mnode/impl/inc/mnodeUser.h b/source/dnode/mnode/impl/inc/mndUser.h similarity index 97% rename from source/dnode/mnode/impl/inc/mnodeUser.h rename to source/dnode/mnode/impl/inc/mndUser.h index 144dd532d9..47fc32c634 100644 --- a/source/dnode/mnode/impl/inc/mnodeUser.h +++ b/source/dnode/mnode/impl/inc/mndUser.h @@ -16,7 +16,7 @@ #ifndef _TD_MNODE_USER_H_ #define _TD_MNODE_USER_H_ -#include "mnodeInt.h" +#include "mndInt.h" #ifdef __cplusplus extern "C" { diff --git a/source/dnode/mnode/impl/inc/mnodeVgroup.h b/source/dnode/mnode/impl/inc/mndVgroup.h similarity index 97% rename from source/dnode/mnode/impl/inc/mnodeVgroup.h rename to source/dnode/mnode/impl/inc/mndVgroup.h index ebd98bb798..d6e5e00ba8 100644 --- a/source/dnode/mnode/impl/inc/mnodeVgroup.h +++ b/source/dnode/mnode/impl/inc/mndVgroup.h @@ -16,7 +16,7 @@ #ifndef _TD_MNODE_VGROUP_H_ #define _TD_MNODE_VGROUP_H_ -#include "mnodeInt.h" +#include "mndInt.h" #ifdef __cplusplus extern "C" { diff --git a/source/dnode/mnode/impl/src/mnodeAcct.c b/source/dnode/mnode/impl/src/mndAcct.c similarity index 99% rename from source/dnode/mnode/impl/src/mnodeAcct.c rename to source/dnode/mnode/impl/src/mndAcct.c index 6f5c498ed2..b503a27b11 100644 --- a/source/dnode/mnode/impl/src/mnodeAcct.c +++ b/source/dnode/mnode/impl/src/mndAcct.c @@ -14,7 +14,7 @@ */ #define _DEFAULT_SOURCE -#include "mnodeInt.h" +#include "mndInt.h" #define SDB_ACCT_VER 1 diff --git a/source/dnode/mnode/impl/src/mnodeAuth.c b/source/dnode/mnode/impl/src/mndAuth.c similarity index 97% rename from source/dnode/mnode/impl/src/mnodeAuth.c rename to source/dnode/mnode/impl/src/mndAuth.c index f8e704d16d..bb50645cee 100644 --- a/source/dnode/mnode/impl/src/mnodeAuth.c +++ b/source/dnode/mnode/impl/src/mndAuth.c @@ -15,7 +15,7 @@ #define _DEFAULT_SOURCE #include "os.h" -#include "mnodeAuth.h" +#include "mndAuth.h" int32_t mnodeInitAuth() { return 0; } void mnodeCleanupAuth() {} diff --git a/source/dnode/mnode/impl/src/mnodeBalance.c b/source/dnode/mnode/impl/src/mndBalance.c similarity index 97% rename from source/dnode/mnode/impl/src/mnodeBalance.c rename to source/dnode/mnode/impl/src/mndBalance.c index a9542ab457..f3acb9b4b3 100644 --- a/source/dnode/mnode/impl/src/mnodeBalance.c +++ b/source/dnode/mnode/impl/src/mndBalance.c @@ -15,7 +15,7 @@ #define _DEFAULT_SOURCE #include "os.h" -#include "mnodeInt.h" +#include "mndInt.h" int32_t mnodeInitBalance() { return 0; } void mnodeCleanupBalance() {} \ No newline at end of file diff --git a/source/dnode/mnode/impl/src/mnodeCluster.c b/source/dnode/mnode/impl/src/mndCluster.c similarity index 97% rename from source/dnode/mnode/impl/src/mnodeCluster.c rename to source/dnode/mnode/impl/src/mndCluster.c index 8ce26270e2..f1dbe1df7d 100644 --- a/source/dnode/mnode/impl/src/mnodeCluster.c +++ b/source/dnode/mnode/impl/src/mndCluster.c @@ -15,7 +15,7 @@ #define _DEFAULT_SOURCE #include "os.h" -#include "mnodeInt.h" +#include "mndInt.h" int32_t mnodeInitCluster() { return 0; } void mnodeCleanupCluster() {} \ No newline at end of file diff --git a/source/dnode/mnode/impl/src/mnodeDb.c b/source/dnode/mnode/impl/src/mndDb.c similarity index 97% rename from source/dnode/mnode/impl/src/mnodeDb.c rename to source/dnode/mnode/impl/src/mndDb.c index da5e3e5542..c8aa347c7b 100644 --- a/source/dnode/mnode/impl/src/mnodeDb.c +++ b/source/dnode/mnode/impl/src/mndDb.c @@ -15,7 +15,7 @@ #define _DEFAULT_SOURCE #include "os.h" -#include "mnodeInt.h" +#include "mndInt.h" int32_t mnodeInitDb() { return 0; } void mnodeCleanupDb() {} \ No newline at end of file diff --git a/source/dnode/mnode/impl/src/mnodeDnode.c b/source/dnode/mnode/impl/src/mndDnode.c similarity index 97% rename from source/dnode/mnode/impl/src/mnodeDnode.c rename to source/dnode/mnode/impl/src/mndDnode.c index 36766f72a8..61cd4c01bc 100644 --- a/source/dnode/mnode/impl/src/mnodeDnode.c +++ b/source/dnode/mnode/impl/src/mndDnode.c @@ -15,7 +15,7 @@ #define _DEFAULT_SOURCE #include "os.h" -#include "mnodeInt.h" +#include "mndInt.h" int32_t mnodeInitDnode() { return 0; } void mnodeCleanupDnode() {} \ No newline at end of file diff --git a/source/dnode/mnode/impl/src/mnodeFunc.c b/source/dnode/mnode/impl/src/mndFunc.c similarity index 97% rename from source/dnode/mnode/impl/src/mnodeFunc.c rename to source/dnode/mnode/impl/src/mndFunc.c index 2d8628c2be..19a8b8dd0e 100644 --- a/source/dnode/mnode/impl/src/mnodeFunc.c +++ b/source/dnode/mnode/impl/src/mndFunc.c @@ -15,7 +15,7 @@ #define _DEFAULT_SOURCE #include "os.h" -#include "mnodeInt.h" +#include "mndInt.h" int32_t mnodeInitFunc() { return 0; } void mnodeCleanupFunc() {} \ No newline at end of file diff --git a/source/dnode/mnode/impl/src/mnodeMnode.c b/source/dnode/mnode/impl/src/mndMnode.c similarity index 97% rename from source/dnode/mnode/impl/src/mnodeMnode.c rename to source/dnode/mnode/impl/src/mndMnode.c index 60e1627ad4..da4d77b440 100644 --- a/source/dnode/mnode/impl/src/mnodeMnode.c +++ b/source/dnode/mnode/impl/src/mndMnode.c @@ -15,7 +15,7 @@ #define _DEFAULT_SOURCE #include "os.h" -#include "mnodeInt.h" +#include "mndInt.h" int32_t mnodeInitMnode() { return 0; } void mnodeCleanupMnode() {} diff --git a/source/dnode/mnode/impl/src/mnodeOper.c b/source/dnode/mnode/impl/src/mndOper.c similarity index 97% rename from source/dnode/mnode/impl/src/mnodeOper.c rename to source/dnode/mnode/impl/src/mndOper.c index 35f4db0261..d25748882a 100644 --- a/source/dnode/mnode/impl/src/mnodeOper.c +++ b/source/dnode/mnode/impl/src/mndOper.c @@ -15,7 +15,7 @@ #define _DEFAULT_SOURCE #include "os.h" -#include "mnodeInt.h" +#include "mndInt.h" int32_t mnodeInitOper() { return 0; } void mnodeCleanupOper() {} \ No newline at end of file diff --git a/source/dnode/mnode/impl/src/mnodeProfile.c b/source/dnode/mnode/impl/src/mndProfile.c similarity index 97% rename from source/dnode/mnode/impl/src/mnodeProfile.c rename to source/dnode/mnode/impl/src/mndProfile.c index 9cf9e184cf..64378b0093 100644 --- a/source/dnode/mnode/impl/src/mnodeProfile.c +++ b/source/dnode/mnode/impl/src/mndProfile.c @@ -15,7 +15,7 @@ #define _DEFAULT_SOURCE #include "os.h" -#include "mnodeInt.h" +#include "mndInt.h" int32_t mnodeInitProfile() { return 0; } void mnodeCleanupProfile() {} \ No newline at end of file diff --git a/source/dnode/mnode/impl/src/mnodeShow.c b/source/dnode/mnode/impl/src/mndShow.c similarity index 97% rename from source/dnode/mnode/impl/src/mnodeShow.c rename to source/dnode/mnode/impl/src/mndShow.c index 9918ad28e0..338c199afc 100644 --- a/source/dnode/mnode/impl/src/mnodeShow.c +++ b/source/dnode/mnode/impl/src/mndShow.c @@ -15,7 +15,7 @@ #define _DEFAULT_SOURCE #include "os.h" -#include "mnodeInt.h" +#include "mndInt.h" int32_t mnodeInitShow() { return 0; } void mnodeCleanUpShow() {} \ No newline at end of file diff --git a/source/dnode/mnode/impl/src/mnodeStable.c b/source/dnode/mnode/impl/src/mndStable.c similarity index 97% rename from source/dnode/mnode/impl/src/mnodeStable.c rename to source/dnode/mnode/impl/src/mndStable.c index 87e887a11b..fe0fc59d25 100644 --- a/source/dnode/mnode/impl/src/mnodeStable.c +++ b/source/dnode/mnode/impl/src/mndStable.c @@ -15,7 +15,7 @@ #define _DEFAULT_SOURCE #include "os.h" -#include "mnodeInt.h" +#include "mndInt.h" int32_t mnodeInitStable() { return 0; } void mnodeCleanupStable() {} \ No newline at end of file diff --git a/source/dnode/mnode/impl/src/mnodeSync.c b/source/dnode/mnode/impl/src/mndSync.c similarity index 95% rename from source/dnode/mnode/impl/src/mnodeSync.c rename to source/dnode/mnode/impl/src/mndSync.c index 6e4084ffa6..4ba0c7dc27 100644 --- a/source/dnode/mnode/impl/src/mnodeSync.c +++ b/source/dnode/mnode/impl/src/mndSync.c @@ -15,8 +15,8 @@ #define _DEFAULT_SOURCE #include "os.h" -#include "mnodeInt.h" -#include "mnodeTrans.h" +#include "mndInt.h" +#include "mndTrans.h" int32_t mnodeInitSync() { return 0; } void mnodeCleanUpSync() {} diff --git a/source/dnode/mnode/impl/src/mnodeTelem.c b/source/dnode/mnode/impl/src/mndTelem.c similarity index 99% rename from source/dnode/mnode/impl/src/mnodeTelem.c rename to source/dnode/mnode/impl/src/mndTelem.c index 206b94a6c7..50934f05e4 100644 --- a/source/dnode/mnode/impl/src/mnodeTelem.c +++ b/source/dnode/mnode/impl/src/mndTelem.c @@ -14,10 +14,10 @@ */ #define _DEFAULT_SOURCE -#include "mnodeTelem.h" +#include "mndTelem.h" #include "tbuffer.h" #include "tglobal.h" -#include "mnodeSync.h" +#include "mndSync.h" #define TELEMETRY_SERVER "telemetry.taosdata.com" #define TELEMETRY_PORT 80 diff --git a/source/dnode/mnode/impl/src/mnodeTrans.c b/source/dnode/mnode/impl/src/mndTrans.c similarity index 99% rename from source/dnode/mnode/impl/src/mnodeTrans.c rename to source/dnode/mnode/impl/src/mndTrans.c index 4cd6bf7bdb..2ea889903e 100644 --- a/source/dnode/mnode/impl/src/mnodeTrans.c +++ b/source/dnode/mnode/impl/src/mndTrans.c @@ -14,7 +14,7 @@ */ #define _DEFAULT_SOURCE -#include "mnodeTrans.h" +#include "mndTrans.h" #include "trpc.h" #define SDB_TRANS_VER 1 diff --git a/source/dnode/mnode/impl/src/mnodeUser.c b/source/dnode/mnode/impl/src/mndUser.c similarity index 99% rename from source/dnode/mnode/impl/src/mnodeUser.c rename to source/dnode/mnode/impl/src/mndUser.c index e67fedd5eb..e8cc6df378 100644 --- a/source/dnode/mnode/impl/src/mnodeUser.c +++ b/source/dnode/mnode/impl/src/mndUser.c @@ -14,11 +14,11 @@ */ #define _DEFAULT_SOURCE -#include "mnodeSync.h" +#include "mndSync.h" #include "os.h" #include "tglobal.h" #include "tkey.h" -#include "mnodeTrans.h" +#include "mndTrans.h" #define SDB_USER_VER 1 diff --git a/source/dnode/mnode/impl/src/mnodeVgroup.c b/source/dnode/mnode/impl/src/mndVgroup.c similarity index 97% rename from source/dnode/mnode/impl/src/mnodeVgroup.c rename to source/dnode/mnode/impl/src/mndVgroup.c index 6f9df1f69f..134603ce27 100644 --- a/source/dnode/mnode/impl/src/mnodeVgroup.c +++ b/source/dnode/mnode/impl/src/mndVgroup.c @@ -15,7 +15,7 @@ #define _DEFAULT_SOURCE #include "os.h" -#include "mnodeInt.h" +#include "mndInt.h" int32_t mnodeInitVgroup() { return 0; } void mnodeCleanupVgroup() {} \ No newline at end of file diff --git a/source/dnode/mnode/impl/src/mnode.c b/source/dnode/mnode/impl/src/mnode.c index 6ea47b41fc..30eb85b9d1 100644 --- a/source/dnode/mnode/impl/src/mnode.c +++ b/source/dnode/mnode/impl/src/mnode.c @@ -17,23 +17,23 @@ #include "os.h" #include "tglobal.h" #include "tqueue.h" -#include "mnodeAcct.h" -#include "mnodeAuth.h" -#include "mnodeBalance.h" -#include "mnodeCluster.h" -#include "mnodeDb.h" -#include "mnodeDnode.h" -#include "mnodeFunc.h" -#include "mnodeMnode.h" -#include "mnodeOper.h" -#include "mnodeProfile.h" -#include "mnodeShow.h" -#include "mnodeStable.h" -#include "mnodeSync.h" -#include "mnodeTelem.h" -#include "mnodeTrans.h" -#include "mnodeUser.h" -#include "mnodeVgroup.h" +#include "mndAcct.h" +#include "mndAuth.h" +#include "mndBalance.h" +#include "mndCluster.h" +#include "mndDb.h" +#include "mndDnode.h" +#include "mndFunc.h" +#include "mndMnode.h" +#include "mndOper.h" +#include "mndProfile.h" +#include "mndShow.h" +#include "mndStable.h" +#include "mndSync.h" +#include "mndTelem.h" +#include "mndTrans.h" +#include "mndUser.h" +#include "mndVgroup.h" SMnodeBak tsMint = {0}; From a17d9811795303e77c0ec5d2b0e12a04b0f258e5 Mon Sep 17 00:00:00 2001 From: Shengliang Guan Date: Sat, 27 Nov 2021 23:02:20 +0800 Subject: [PATCH 35/44] rename some defs --- include/dnode/mnode/mnode.h | 6 +- source/dnode/mnode/impl/inc/mndAcct.h | 6 +- source/dnode/mnode/impl/inc/mndAuth.h | 6 +- source/dnode/mnode/impl/inc/mndBalance.h | 6 +- source/dnode/mnode/impl/inc/mndCluster.h | 6 +- source/dnode/mnode/impl/inc/mndDb.h | 6 +- source/dnode/mnode/impl/inc/mndDef.h | 6 +- source/dnode/mnode/impl/inc/mndDnode.h | 6 +- source/dnode/mnode/impl/inc/mndFunc.h | 6 +- source/dnode/mnode/impl/inc/mndInt.h | 6 +- source/dnode/mnode/impl/inc/mndMnode.h | 6 +- source/dnode/mnode/impl/inc/mndOper.h | 6 +- source/dnode/mnode/impl/inc/mndProfile.h | 6 +- source/dnode/mnode/impl/inc/mndShow.h | 6 +- source/dnode/mnode/impl/inc/mndStable.h | 6 +- source/dnode/mnode/impl/inc/mndSync.h | 6 +- source/dnode/mnode/impl/inc/mndTelem.h | 6 +- source/dnode/mnode/impl/inc/mndUser.h | 6 +- source/dnode/mnode/impl/inc/mndVgroup.h | 6 +- source/dnode/mnode/impl/inc/mnodeInt.h | 70 ++++++++++++++++++++++++ source/dnode/mnode/impl/inc/mnodeUser.h | 32 +++++++++++ 21 files changed, 159 insertions(+), 57 deletions(-) create mode 100644 source/dnode/mnode/impl/inc/mnodeInt.h create mode 100644 source/dnode/mnode/impl/inc/mnodeUser.h diff --git a/include/dnode/mnode/mnode.h b/include/dnode/mnode/mnode.h index 2ffbd395e8..fa357fc814 100644 --- a/include/dnode/mnode/mnode.h +++ b/include/dnode/mnode/mnode.h @@ -13,8 +13,8 @@ * along with this program. If not, see . */ -#ifndef _TD_MNODE_H_ -#define _TD_MNODE_H_ +#ifndef _TD_MND_H_ +#define _TD_MND_H_ #ifdef __cplusplus extern "C" { @@ -170,4 +170,4 @@ void mnodeProcessApplyMsg(SMnodeMsg *pMsg); } #endif -#endif /*_TD_MNODE_H_*/ +#endif /*_TD_MND_H_*/ diff --git a/source/dnode/mnode/impl/inc/mndAcct.h b/source/dnode/mnode/impl/inc/mndAcct.h index a45d865c03..e52739104c 100644 --- a/source/dnode/mnode/impl/inc/mndAcct.h +++ b/source/dnode/mnode/impl/inc/mndAcct.h @@ -13,8 +13,8 @@ * along with this program. If not, see . */ -#ifndef _TD_MNODE_ACCT_H_ -#define _TD_MNODE_ACCT_H_ +#ifndef _TD_MND_ACCT_H_ +#define _TD_MND_ACCT_H_ #include "mndInt.h" @@ -29,4 +29,4 @@ void mnodeCleanupAcct(); } #endif -#endif /*_TD_MNODE_ACCT_H_*/ +#endif /*_TD_MND_ACCT_H_*/ diff --git a/source/dnode/mnode/impl/inc/mndAuth.h b/source/dnode/mnode/impl/inc/mndAuth.h index a759e0169e..cba1682fa5 100644 --- a/source/dnode/mnode/impl/inc/mndAuth.h +++ b/source/dnode/mnode/impl/inc/mndAuth.h @@ -13,8 +13,8 @@ * along with this program. If not, see . */ -#ifndef _TD_MNODE_AUTH_H_ -#define _TD_MNODE_AUTH_H_ +#ifndef _TD_MND_AUTH_H_ +#define _TD_MND_AUTH_H_ #include "mndInt.h" @@ -29,4 +29,4 @@ void mnodeCleanupAuth(); } #endif -#endif /*_TD_MNODE_AUTH_H_*/ +#endif /*_TD_MND_AUTH_H_*/ diff --git a/source/dnode/mnode/impl/inc/mndBalance.h b/source/dnode/mnode/impl/inc/mndBalance.h index 84de9bcb32..3ec572d32e 100644 --- a/source/dnode/mnode/impl/inc/mndBalance.h +++ b/source/dnode/mnode/impl/inc/mndBalance.h @@ -13,8 +13,8 @@ * along with this program. If not, see . */ -#ifndef _TD_MNODE_BALANCE_H_ -#define _TD_MNODE_BALANCE_H_ +#ifndef _TD_MND_BALANCE_H_ +#define _TD_MND_BALANCE_H_ #include "mndInt.h" @@ -29,4 +29,4 @@ void mnodeCleanupBalance(); } #endif -#endif /*_TD_MNODE_BALANCE_H_*/ +#endif /*_TD_MND_BALANCE_H_*/ diff --git a/source/dnode/mnode/impl/inc/mndCluster.h b/source/dnode/mnode/impl/inc/mndCluster.h index f9047e6e25..cb01a9e414 100644 --- a/source/dnode/mnode/impl/inc/mndCluster.h +++ b/source/dnode/mnode/impl/inc/mndCluster.h @@ -13,8 +13,8 @@ * along with this program. If not, see . */ -#ifndef _TD_MNODE_CLUSTER_H_ -#define _TD_MNODE_CLUSTER_H_ +#ifndef _TD_MND_CLUSTER_H_ +#define _TD_MND_CLUSTER_H_ #include "mndInt.h" @@ -29,4 +29,4 @@ void mnodeCleanupCluster(); } #endif -#endif /*_TD_MNODE_CLUSTER_H_*/ +#endif /*_TD_MND_CLUSTER_H_*/ diff --git a/source/dnode/mnode/impl/inc/mndDb.h b/source/dnode/mnode/impl/inc/mndDb.h index 1ab4844873..d694badb54 100644 --- a/source/dnode/mnode/impl/inc/mndDb.h +++ b/source/dnode/mnode/impl/inc/mndDb.h @@ -13,8 +13,8 @@ * along with this program. If not, see . */ -#ifndef _TD_MNODE_DATABASE_H_ -#define _TD_MNODE_DATABASE_H_ +#ifndef _TD_MND_DATABASE_H_ +#define _TD_MND_DATABASE_H_ #include "mndInt.h" @@ -29,4 +29,4 @@ void mnodeCleanupDb(); } #endif -#endif /*_TD_MNODE_DATABASE_H_*/ +#endif /*_TD_MND_DATABASE_H_*/ diff --git a/source/dnode/mnode/impl/inc/mndDef.h b/source/dnode/mnode/impl/inc/mndDef.h index ccdba13006..baed0b44e4 100644 --- a/source/dnode/mnode/impl/inc/mndDef.h +++ b/source/dnode/mnode/impl/inc/mndDef.h @@ -13,8 +13,8 @@ * along with this program. If not, see . */ -#ifndef _TD_MNODE_DEF_H_ -#define _TD_MNODE_DEF_H_ +#ifndef _TD_MND_DEF_H_ +#define _TD_MND_DEF_H_ #include "os.h" #include "taosmsg.h" @@ -305,4 +305,4 @@ typedef struct SMnodeMsg { } #endif -#endif /*_TD_MNODE_DEF_H_*/ +#endif /*_TD_MND_DEF_H_*/ diff --git a/source/dnode/mnode/impl/inc/mndDnode.h b/source/dnode/mnode/impl/inc/mndDnode.h index 7b25061b95..5babb99a42 100644 --- a/source/dnode/mnode/impl/inc/mndDnode.h +++ b/source/dnode/mnode/impl/inc/mndDnode.h @@ -13,8 +13,8 @@ * along with this program. If not, see . */ -#ifndef _TD_MNODE_DNODE_H_ -#define _TD_MNODE_DNODE_H_ +#ifndef _TD_MND_DNODE_H_ +#define _TD_MND_DNODE_H_ #include "mndInt.h" @@ -29,4 +29,4 @@ void mnodeCleanupDnode(); } #endif -#endif /*_TD_MNODE_DNODE_H_*/ +#endif /*_TD_MND_DNODE_H_*/ diff --git a/source/dnode/mnode/impl/inc/mndFunc.h b/source/dnode/mnode/impl/inc/mndFunc.h index ff1ebde8b0..22ee7018f2 100644 --- a/source/dnode/mnode/impl/inc/mndFunc.h +++ b/source/dnode/mnode/impl/inc/mndFunc.h @@ -13,8 +13,8 @@ * along with this program. If not, see . */ -#ifndef _TD_MNODE_FUNC_H_ -#define _TD_MNODE_FUNC_H_ +#ifndef _TD_MND_FUNC_H_ +#define _TD_MND_FUNC_H_ #include "mndInt.h" @@ -29,4 +29,4 @@ void mnodeCleanupFunc(); } #endif -#endif /*_TD_MNODE_FUNC_H_*/ +#endif /*_TD_MND_FUNC_H_*/ diff --git a/source/dnode/mnode/impl/inc/mndInt.h b/source/dnode/mnode/impl/inc/mndInt.h index 17db21fe18..62f591a26e 100644 --- a/source/dnode/mnode/impl/inc/mndInt.h +++ b/source/dnode/mnode/impl/inc/mndInt.h @@ -13,8 +13,8 @@ * along with this program. If not, see . */ -#ifndef _TD_MNODE_INT_H_ -#define _TD_MNODE_INT_H_ +#ifndef _TD_MND_INT_H_ +#define _TD_MND_INT_H_ #include "mndDef.h" #include "sdb.h" @@ -67,4 +67,4 @@ void mnodeSetMsgHandle(SMnode *pMnode, int32_t msgType, MndMsgFp fp); } #endif -#endif /*_TD_MNODE_INT_H_*/ +#endif /*_TD_MND_INT_H_*/ diff --git a/source/dnode/mnode/impl/inc/mndMnode.h b/source/dnode/mnode/impl/inc/mndMnode.h index 9e7f1766a6..4c20cbf833 100644 --- a/source/dnode/mnode/impl/inc/mndMnode.h +++ b/source/dnode/mnode/impl/inc/mndMnode.h @@ -13,8 +13,8 @@ * along with this program. If not, see . */ -#ifndef _TD_MNODE_MNODE_H_ -#define _TD_MNODE_MNODE_H_ +#ifndef _TD_MND_MNODE_H_ +#define _TD_MND_MNODE_H_ #include "mndInt.h" @@ -31,4 +31,4 @@ void mnodeGetMnodeEpSetForShell(SEpSet *epSet, bool redirect); } #endif -#endif /*_TD_MNODE_MNODE_H_*/ +#endif /*_TD_MND_MNODE_H_*/ diff --git a/source/dnode/mnode/impl/inc/mndOper.h b/source/dnode/mnode/impl/inc/mndOper.h index 4392422b23..7ab396a019 100644 --- a/source/dnode/mnode/impl/inc/mndOper.h +++ b/source/dnode/mnode/impl/inc/mndOper.h @@ -13,8 +13,8 @@ * along with this program. If not, see . */ -#ifndef _TD_MNODE_OPER_H_ -#define _TD_MNODE_OPER_H_ +#ifndef _TD_MND_OPER_H_ +#define _TD_MND_OPER_H_ #ifdef __cplusplus extern "C" { @@ -27,4 +27,4 @@ void mnodeCleanupOper(); } #endif -#endif /*_TD_MNODE_OPER_H_*/ +#endif /*_TD_MND_OPER_H_*/ diff --git a/source/dnode/mnode/impl/inc/mndProfile.h b/source/dnode/mnode/impl/inc/mndProfile.h index 405a6f8dac..d2b402227b 100644 --- a/source/dnode/mnode/impl/inc/mndProfile.h +++ b/source/dnode/mnode/impl/inc/mndProfile.h @@ -13,8 +13,8 @@ * along with this program. If not, see . */ -#ifndef _TD_MNODE_PROFILE_H_ -#define _TD_MNODE_PROFILE_H_ +#ifndef _TD_MND_PROFILE_H_ +#define _TD_MND_PROFILE_H_ #include "mndInt.h" @@ -29,4 +29,4 @@ void mnodeCleanupProfile(); } #endif -#endif /*_TD_MNODE_PROFILE_H_*/ +#endif /*_TD_MND_PROFILE_H_*/ diff --git a/source/dnode/mnode/impl/inc/mndShow.h b/source/dnode/mnode/impl/inc/mndShow.h index 5045fde1a8..cc3bd55cf4 100644 --- a/source/dnode/mnode/impl/inc/mndShow.h +++ b/source/dnode/mnode/impl/inc/mndShow.h @@ -13,8 +13,8 @@ * along with this program. If not, see . */ -#ifndef _TD_MNODE_SHOW_H_ -#define _TD_MNODE_SHOW_H_ +#ifndef _TD_MND_SHOW_H_ +#define _TD_MND_SHOW_H_ #include "mndInt.h" @@ -29,4 +29,4 @@ void mnodeCleanUpShow(); } #endif -#endif /*_TD_MNODE_SHOW_H_*/ +#endif /*_TD_MND_SHOW_H_*/ diff --git a/source/dnode/mnode/impl/inc/mndStable.h b/source/dnode/mnode/impl/inc/mndStable.h index f55d7737de..b2469aaaa8 100644 --- a/source/dnode/mnode/impl/inc/mndStable.h +++ b/source/dnode/mnode/impl/inc/mndStable.h @@ -13,8 +13,8 @@ * along with this program. If not, see . */ -#ifndef _TD_MNODE_STABLE_H_ -#define _TD_MNODE_STABLE_H_ +#ifndef _TD_MND_STABLE_H_ +#define _TD_MND_STABLE_H_ #include "mndInt.h" @@ -29,4 +29,4 @@ void mnodeCleanupStable(); } #endif -#endif /*_TD_MNODE_STABLE_H_*/ +#endif /*_TD_MND_STABLE_H_*/ diff --git a/source/dnode/mnode/impl/inc/mndSync.h b/source/dnode/mnode/impl/inc/mndSync.h index 8877bc9c82..863aaccf0c 100644 --- a/source/dnode/mnode/impl/inc/mndSync.h +++ b/source/dnode/mnode/impl/inc/mndSync.h @@ -13,8 +13,8 @@ * along with this program. If not, see . */ -#ifndef _TD_MNODE_SYNC_H_ -#define _TD_MNODE_SYNC_H_ +#ifndef _TD_MND_SYNC_H_ +#define _TD_MND_SYNC_H_ #include "mndInt.h" @@ -32,4 +32,4 @@ bool mnodeIsMaster(); } #endif -#endif /*_TD_MNODE_SYNC_H_*/ +#endif /*_TD_MND_SYNC_H_*/ diff --git a/source/dnode/mnode/impl/inc/mndTelem.h b/source/dnode/mnode/impl/inc/mndTelem.h index bb2e6635b1..f2a11a3d32 100644 --- a/source/dnode/mnode/impl/inc/mndTelem.h +++ b/source/dnode/mnode/impl/inc/mndTelem.h @@ -13,8 +13,8 @@ * along with this program. If not, see . */ -#ifndef _TD_MNODE_TELEMETRY_H_ -#define _TD_MNODE_TELEMETRY_H_ +#ifndef _TD_MND_TELEMETRY_H_ +#define _TD_MND_TELEMETRY_H_ #ifdef __cplusplus extern "C" { @@ -28,4 +28,4 @@ void mnodeCleanupTelem(); } #endif -#endif /*_TD_MNODE_TELEMETRY_H_*/ +#endif /*_TD_MND_TELEMETRY_H_*/ diff --git a/source/dnode/mnode/impl/inc/mndUser.h b/source/dnode/mnode/impl/inc/mndUser.h index 47fc32c634..adc7a0db13 100644 --- a/source/dnode/mnode/impl/inc/mndUser.h +++ b/source/dnode/mnode/impl/inc/mndUser.h @@ -13,8 +13,8 @@ * along with this program. If not, see . */ -#ifndef _TD_MNODE_USER_H_ -#define _TD_MNODE_USER_H_ +#ifndef _TD_MND_USER_H_ +#define _TD_MND_USER_H_ #include "mndInt.h" @@ -29,4 +29,4 @@ void mnodeCleanupUser(); } #endif -#endif /*_TD_MNODE_USER_H_*/ +#endif /*_TD_MND_USER_H_*/ diff --git a/source/dnode/mnode/impl/inc/mndVgroup.h b/source/dnode/mnode/impl/inc/mndVgroup.h index d6e5e00ba8..18c2ea7ecf 100644 --- a/source/dnode/mnode/impl/inc/mndVgroup.h +++ b/source/dnode/mnode/impl/inc/mndVgroup.h @@ -13,8 +13,8 @@ * along with this program. If not, see . */ -#ifndef _TD_MNODE_VGROUP_H_ -#define _TD_MNODE_VGROUP_H_ +#ifndef _TD_MND_VGROUP_H_ +#define _TD_MND_VGROUP_H_ #include "mndInt.h" @@ -29,4 +29,4 @@ void mnodeCleanupVgroup(); } #endif -#endif /*_TD_MNODE_VGROUP_H_*/ +#endif /*_TD_MND_VGROUP_H_*/ diff --git a/source/dnode/mnode/impl/inc/mnodeInt.h b/source/dnode/mnode/impl/inc/mnodeInt.h new file mode 100644 index 0000000000..62f591a26e --- /dev/null +++ b/source/dnode/mnode/impl/inc/mnodeInt.h @@ -0,0 +1,70 @@ +/* + * 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_MND_INT_H_ +#define _TD_MND_INT_H_ + +#include "mndDef.h" +#include "sdb.h" +#include "tstep.h" + +#ifdef __cplusplus +extern "C" { +#endif + +typedef int32_t (*MndMsgFp)(SMnode *pMnode, SMnodeMsg *pMsg); + +typedef struct SMnodeBak { + int32_t dnodeId; + int64_t clusterId; + tmr_h timer; + SSteps *pInitSteps; + SSteps *pStartSteps; + SMnodeOpt para; + MndMsgFp msgFp[TSDB_MSG_TYPE_MAX]; +} SMnodeBak; + +typedef struct SMnode { + int32_t dnodeId; + int64_t clusterId; + int8_t replica; + int8_t selfIndex; + SReplica replicas[TSDB_MAX_REPLICA]; + tmr_h timer; + SSteps *pInitSteps; + SSteps *pStartSteps; + struct SSdb *pSdb; + struct SDnode *pServer; + MndMsgFp msgFp[TSDB_MSG_TYPE_MAX]; + PutMsgToMnodeQFp putMsgToApplyMsgFp; + SendMsgToDnodeFp sendMsgToDnodeFp; + SendMsgToMnodeFp sendMsgToMnodeFp; + SendRedirectMsgFp sendRedirectMsgFp; +} SMnode; + +tmr_h mnodeGetTimer(); +int32_t mnodeGetDnodeId(); +int64_t mnodeGetClusterId(); + +void mnodeSendMsgToDnode(SMnode *pMnode, struct SEpSet *epSet, struct SRpcMsg *rpcMsg); +void mnodeSendMsgToMnode(SMnode *pMnode, struct SRpcMsg *rpcMsg); +void mnodeSendRedirectMsg(SMnode *pMnode, struct SRpcMsg *rpcMsg, bool forShell); +void mnodeSetMsgHandle(SMnode *pMnode, int32_t msgType, MndMsgFp fp); + +#ifdef __cplusplus +} +#endif + +#endif /*_TD_MND_INT_H_*/ diff --git a/source/dnode/mnode/impl/inc/mnodeUser.h b/source/dnode/mnode/impl/inc/mnodeUser.h new file mode 100644 index 0000000000..adc7a0db13 --- /dev/null +++ b/source/dnode/mnode/impl/inc/mnodeUser.h @@ -0,0 +1,32 @@ +/* + * 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_MND_USER_H_ +#define _TD_MND_USER_H_ + +#include "mndInt.h" + +#ifdef __cplusplus +extern "C" { +#endif + +int32_t mnodeInitUser(); +void mnodeCleanupUser(); + +#ifdef __cplusplus +} +#endif + +#endif /*_TD_MND_USER_H_*/ From 6ee8832271103f33baa7f6759661283cd87eec88 Mon Sep 17 00:00:00 2001 From: Shengliang Guan Date: Mon, 29 Nov 2021 12:09:18 +0800 Subject: [PATCH 36/44] TD-10431 fix compile errors in mnode --- include/dnode/mnode/mnode.h | 28 +-- source/dnode/mgmt/impl/src/dndMnode.c | 45 ++-- source/dnode/mnode/impl/inc/mndAcct.h | 4 +- source/dnode/mnode/impl/inc/mndAuth.h | 4 +- source/dnode/mnode/impl/inc/mndBalance.h | 4 +- source/dnode/mnode/impl/inc/mndCluster.h | 4 +- source/dnode/mnode/impl/inc/mndDb.h | 4 +- source/dnode/mnode/impl/inc/mndDef.h | 96 ++++--- source/dnode/mnode/impl/inc/mndDnode.h | 4 +- source/dnode/mnode/impl/inc/mndFunc.h | 4 +- source/dnode/mnode/impl/inc/mndInt.h | 41 ++- source/dnode/mnode/impl/inc/mndMnode.h | 4 +- source/dnode/mnode/impl/inc/mndOper.h | 4 +- source/dnode/mnode/impl/inc/mndProfile.h | 4 +- source/dnode/mnode/impl/inc/mndShow.h | 4 +- source/dnode/mnode/impl/inc/mndStable.h | 4 +- source/dnode/mnode/impl/inc/mndSync.h | 4 +- source/dnode/mnode/impl/inc/mndTelem.h | 4 +- source/dnode/mnode/impl/inc/mndTrans.h | 4 +- source/dnode/mnode/impl/inc/mndUser.h | 4 +- source/dnode/mnode/impl/inc/mndVgroup.h | 4 +- source/dnode/mnode/impl/inc/mnodeInt.h | 70 ------ source/dnode/mnode/impl/inc/mnodeUser.h | 32 --- source/dnode/mnode/impl/src/mndAcct.c | 4 +- source/dnode/mnode/impl/src/mndAuth.c | 6 +- source/dnode/mnode/impl/src/mndBalance.c | 4 +- source/dnode/mnode/impl/src/mndCluster.c | 4 +- source/dnode/mnode/impl/src/mndDb.c | 4 +- source/dnode/mnode/impl/src/mndDnode.c | 4 +- source/dnode/mnode/impl/src/mndFunc.c | 4 +- source/dnode/mnode/impl/src/mndMnode.c | 4 +- source/dnode/mnode/impl/src/mndOper.c | 4 +- source/dnode/mnode/impl/src/mndProfile.c | 4 +- source/dnode/mnode/impl/src/mndShow.c | 4 +- source/dnode/mnode/impl/src/mndStable.c | 4 +- source/dnode/mnode/impl/src/mndSync.c | 4 +- source/dnode/mnode/impl/src/mndTelem.c | 8 +- source/dnode/mnode/impl/src/mndTrans.c | 4 +- source/dnode/mnode/impl/src/mndUser.c | 6 +- source/dnode/mnode/impl/src/mndVgroup.c | 4 +- source/dnode/mnode/impl/src/mnode.c | 307 ++++++++++++++--------- 41 files changed, 364 insertions(+), 399 deletions(-) delete mode 100644 source/dnode/mnode/impl/inc/mnodeInt.h delete mode 100644 source/dnode/mnode/impl/inc/mnodeUser.h diff --git a/include/dnode/mnode/mnode.h b/include/dnode/mnode/mnode.h index fa357fc814..9d4c27a9dc 100644 --- a/include/dnode/mnode/mnode.h +++ b/include/dnode/mnode/mnode.h @@ -48,7 +48,7 @@ typedef struct { int8_t replica; int8_t selfIndex; SReplica replicas[TSDB_MAX_REPLICA]; - struct SDnode *pDnode; + SDnode *pDnode; PutMsgToMnodeQFp putMsgToApplyMsgFp; SendMsgToDnodeFp sendMsgToDnodeFp; SendMsgToMnodeFp sendMsgToMnodeFp; @@ -63,14 +63,14 @@ typedef struct { * @param pOption Option of the mnode * @return SMnode* The mnode object */ -SMnode *mnodeOpen(const char *path, const SMnodeOpt *pOption); +SMnode *mndOpen(const char *path, const SMnodeOpt *pOption); /** * @brief Close a mnode * * @param pMnode The mnode object to close */ -void mnodeClose(SMnode *pMnode); +void mndClose(SMnode *pMnode); /** * @brief Close a mnode @@ -79,14 +79,14 @@ void mnodeClose(SMnode *pMnode); * @param pOption Options of the mnode * @return int32_t 0 for success, -1 for failure */ -int32_t mnodeAlter(SMnode *pMnode, const SMnodeOpt *pOption); +int32_t mndAlter(SMnode *pMnode, const SMnodeOpt *pOption); /** * @brief Drop a mnode. * * @param path Path of the mnode. */ -void mnodeDestroy(const char *path); +void mndDestroy(const char *path); /** * @brief Get mnode statistics info @@ -95,7 +95,7 @@ void mnodeDestroy(const char *path); * @param pLoad Statistics of the mnode. * @return int32_t 0 for success, -1 for failure */ -int32_t mnodeGetLoad(SMnode *pMnode, SMnodeLoad *pLoad); +int32_t mndGetLoad(SMnode *pMnode, SMnodeLoad *pLoad); /** * @brief Get user authentication info @@ -108,7 +108,7 @@ int32_t mnodeGetLoad(SMnode *pMnode, SMnodeLoad *pLoad); * @param ckey * @return int32_t 0 for success, -1 for failure */ -int32_t mnodeRetriveAuth(SMnode *pMnode, char *user, char *spi, char *encrypt, char *secret, char *ckey); +int32_t mndRetriveAuth(SMnode *pMnode, char *user, char *spi, char *encrypt, char *secret, char *ckey); /** * @brief Initialize mnode msg @@ -117,14 +117,14 @@ int32_t mnodeRetriveAuth(SMnode *pMnode, char *user, char *spi, char *encrypt, c * @param pMsg The request rpc msg * @return int32_t The created mnode msg */ -SMnodeMsg *mnodeInitMsg(SMnode *pMnode, SRpcMsg *pRpcMsg); +SMnodeMsg *mndInitMsg(SMnode *pMnode, SRpcMsg *pRpcMsg); /** * @brief Cleanup mnode msg * * @param pMsg The request msg */ -void mnodeCleanupMsg(SMnodeMsg *pMsg); +void mndCleanupMsg(SMnodeMsg *pMsg); /** * @brief Cleanup mnode msg @@ -132,7 +132,7 @@ void mnodeCleanupMsg(SMnodeMsg *pMsg); * @param pMsg The request msg * @param code The error code */ -void mnodeSendRsp(SMnodeMsg *pMsg, int32_t code); +void mndSendRsp(SMnodeMsg *pMsg, int32_t code); /** * @brief Process the read request @@ -140,7 +140,7 @@ void mnodeSendRsp(SMnodeMsg *pMsg, int32_t code); * @param pMsg The request msg * @return int32_t 0 for success, -1 for failure */ -void mnodeProcessReadMsg(SMnodeMsg *pMsg); +void mndProcessReadMsg(SMnodeMsg *pMsg); /** * @brief Process the write request @@ -148,7 +148,7 @@ void mnodeProcessReadMsg(SMnodeMsg *pMsg); * @param pMsg The request msg * @return int32_t 0 for success, -1 for failure */ -void mnodeProcessWriteMsg(SMnodeMsg *pMsg); +void mndProcessWriteMsg(SMnodeMsg *pMsg); /** * @brief Process the sync request @@ -156,7 +156,7 @@ void mnodeProcessWriteMsg(SMnodeMsg *pMsg); * @param pMsg The request msg * @return int32_t 0 for success, -1 for failure */ -void mnodeProcessSyncMsg(SMnodeMsg *pMsg); +void mndProcessSyncMsg(SMnodeMsg *pMsg); /** * @brief Process the apply request @@ -164,7 +164,7 @@ void mnodeProcessSyncMsg(SMnodeMsg *pMsg); * @param pMsg The request msg * @return int32_t 0 for success, -1 for failure */ -void mnodeProcessApplyMsg(SMnodeMsg *pMsg); +void mndProcessApplyMsg(SMnodeMsg *pMsg); #ifdef __cplusplus } diff --git a/source/dnode/mgmt/impl/src/dndMnode.c b/source/dnode/mgmt/impl/src/dndMnode.c index d42891f1e4..f2ef420cc0 100644 --- a/source/dnode/mgmt/impl/src/dndMnode.c +++ b/source/dnode/mgmt/impl/src/dndMnode.c @@ -396,7 +396,7 @@ static int32_t dndOpenMnode(SDnode *pDnode, SMnodeOpt *pOption) { return code; } - SMnode *pMnode = mnodeOpen(pDnode->dir.mnode, pOption); + SMnode *pMnode = mndOpen(pDnode->dir.mnode, pOption); if (pMnode == NULL) { dError("failed to open mnode since %s", terrstr()); code = terrno; @@ -409,8 +409,8 @@ static int32_t dndOpenMnode(SDnode *pDnode, SMnodeOpt *pOption) { dError("failed to write mnode file since %s", terrstr()); code = terrno; dndStopMnodeWorker(pDnode); - mnodeClose(pMnode); - mnodeDestroy(pDnode->dir.mnode); + mndClose(pMnode); + mndDestroy(pDnode->dir.mnode); terrno = code; return code; } @@ -432,7 +432,7 @@ static int32_t dndAlterMnode(SDnode *pDnode, SMnodeOpt *pOption) { return -1; } - if (mnodeAlter(pMnode, pOption) != 0) { + if (mndAlter(pMnode, pOption) != 0) { dError("failed to alter mnode since %s", terrstr()); dndReleaseMnode(pDnode, pMnode); return -1; @@ -467,8 +467,8 @@ static int32_t dndDropMnode(SDnode *pDnode) { dndStopMnodeWorker(pDnode); dndWriteMnodeFile(pDnode); - mnodeClose(pMnode); - mnodeDestroy(pDnode->dir.mnode); + mndClose(pMnode); + mndDestroy(pDnode->dir.mnode); return 0; } @@ -495,6 +495,7 @@ static int32_t dndProcessCreateMnodeReq(SDnode *pDnode, SRpcMsg *pRpcMsg) { if (dndBuildMnodeOptionFromMsg(pDnode, &option, pMsg) != 0) { return -1; } + return dndOpenMnode(pDnode, &option); } } @@ -554,13 +555,13 @@ static void dndProcessMnodeReadQueue(SDnode *pDnode, SMnodeMsg *pMsg) { SMnode *pMnode = dndAcquireMnode(pDnode); if (pMnode != NULL) { - mnodeProcessReadMsg(pMsg); + mndProcessReadMsg(pMsg); dndReleaseMnode(pDnode, pMnode); } else { - mnodeSendRsp(pMsg, terrno); + mndSendRsp(pMsg, terrno); } - mnodeCleanupMsg(pMsg); + mndCleanupMsg(pMsg); } static void dndProcessMnodeWriteQueue(SDnode *pDnode, SMnodeMsg *pMsg) { @@ -568,13 +569,13 @@ static void dndProcessMnodeWriteQueue(SDnode *pDnode, SMnodeMsg *pMsg) { SMnode *pMnode = dndAcquireMnode(pDnode); if (pMnode != NULL) { - mnodeProcessWriteMsg(pMsg); + mndProcessWriteMsg(pMsg); dndReleaseMnode(pDnode, pMnode); } else { - mnodeSendRsp(pMsg, terrno); + mndSendRsp(pMsg, terrno); } - mnodeCleanupMsg(pMsg); + mndCleanupMsg(pMsg); } static void dndProcessMnodeApplyQueue(SDnode *pDnode, SMnodeMsg *pMsg) { @@ -582,13 +583,13 @@ static void dndProcessMnodeApplyQueue(SDnode *pDnode, SMnodeMsg *pMsg) { SMnode *pMnode = dndAcquireMnode(pDnode); if (pMnode != NULL) { - mnodeProcessApplyMsg(pMsg); + mndProcessApplyMsg(pMsg); dndReleaseMnode(pDnode, pMnode); } else { - mnodeSendRsp(pMsg, terrno); + mndSendRsp(pMsg, terrno); } - mnodeCleanupMsg(pMsg); + mndCleanupMsg(pMsg); } static void dndProcessMnodeSyncQueue(SDnode *pDnode, SMnodeMsg *pMsg) { @@ -596,26 +597,26 @@ static void dndProcessMnodeSyncQueue(SDnode *pDnode, SMnodeMsg *pMsg) { SMnode *pMnode = dndAcquireMnode(pDnode); if (pMnode != NULL) { - mnodeProcessSyncMsg(pMsg); + mndProcessSyncMsg(pMsg); dndReleaseMnode(pDnode, pMnode); } else { - mnodeSendRsp(pMsg, terrno); + mndSendRsp(pMsg, terrno); } - mnodeCleanupMsg(pMsg); + mndCleanupMsg(pMsg); } static int32_t dndWriteMnodeMsgToQueue(SMnode *pMnode, taos_queue pQueue, SRpcMsg *pRpcMsg) { assert(pQueue); - SMnodeMsg *pMsg = mnodeInitMsg(pMnode, pRpcMsg); + SMnodeMsg *pMsg = mndInitMsg(pMnode, pRpcMsg); if (pMsg == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; return -1; } if (taosWriteQitem(pQueue, pMsg) != 0) { - mnodeCleanupMsg(pMsg); + mndCleanupMsg(pMsg); terrno = TSDB_CODE_OUT_OF_MEMORY; return -1; } @@ -877,7 +878,7 @@ int32_t dndInitMnode(SDnode *pDnode) { if (pMgmt->dropped) { dInfo("mnode has been deployed and needs to be deleted"); - mnodeDestroy(pDnode->dir.mnode); + mndDestroy(pDnode->dir.mnode); return 0; } @@ -920,7 +921,7 @@ int32_t dndGetUserAuthFromMnode(SDnode *pDnode, char *user, char *spi, char *enc return -1; } - int32_t code = mnodeRetriveAuth(pMnode, user, spi, encrypt, secret, ckey); + int32_t code = mndRetriveAuth(pMnode, user, spi, encrypt, secret, ckey); dndReleaseMnode(pDnode, pMnode); return code; } diff --git a/source/dnode/mnode/impl/inc/mndAcct.h b/source/dnode/mnode/impl/inc/mndAcct.h index e52739104c..3289c24110 100644 --- a/source/dnode/mnode/impl/inc/mndAcct.h +++ b/source/dnode/mnode/impl/inc/mndAcct.h @@ -22,8 +22,8 @@ extern "C" { #endif -int32_t mnodeInitAcct(); -void mnodeCleanupAcct(); +int32_t mndInitAcct(); +void mndCleanupAcct(); #ifdef __cplusplus } diff --git a/source/dnode/mnode/impl/inc/mndAuth.h b/source/dnode/mnode/impl/inc/mndAuth.h index cba1682fa5..3f0deeec6c 100644 --- a/source/dnode/mnode/impl/inc/mndAuth.h +++ b/source/dnode/mnode/impl/inc/mndAuth.h @@ -22,8 +22,8 @@ extern "C" { #endif -int32_t mnodeInitAuth(); -void mnodeCleanupAuth(); +int32_t mndInitAuth(); +void mndCleanupAuth(); #ifdef __cplusplus } diff --git a/source/dnode/mnode/impl/inc/mndBalance.h b/source/dnode/mnode/impl/inc/mndBalance.h index 3ec572d32e..66d85eb2c3 100644 --- a/source/dnode/mnode/impl/inc/mndBalance.h +++ b/source/dnode/mnode/impl/inc/mndBalance.h @@ -22,8 +22,8 @@ extern "C" { #endif -int32_t mnodeInitBalance(); -void mnodeCleanupBalance(); +int32_t mndInitBalance(); +void mndCleanupBalance(); #ifdef __cplusplus } diff --git a/source/dnode/mnode/impl/inc/mndCluster.h b/source/dnode/mnode/impl/inc/mndCluster.h index cb01a9e414..d8dba66a63 100644 --- a/source/dnode/mnode/impl/inc/mndCluster.h +++ b/source/dnode/mnode/impl/inc/mndCluster.h @@ -22,8 +22,8 @@ extern "C" { #endif -int32_t mnodeInitCluster(); -void mnodeCleanupCluster(); +int32_t mndInitCluster(); +void mndCleanupCluster(); #ifdef __cplusplus } diff --git a/source/dnode/mnode/impl/inc/mndDb.h b/source/dnode/mnode/impl/inc/mndDb.h index d694badb54..c9a888a115 100644 --- a/source/dnode/mnode/impl/inc/mndDb.h +++ b/source/dnode/mnode/impl/inc/mndDb.h @@ -22,8 +22,8 @@ extern "C" { #endif -int32_t mnodeInitDb(); -void mnodeCleanupDb(); +int32_t mndInitDb(); +void mndCleanupDb(); #ifdef __cplusplus } diff --git a/source/dnode/mnode/impl/inc/mndDef.h b/source/dnode/mnode/impl/inc/mndDef.h index baed0b44e4..c0d02d3fc4 100644 --- a/source/dnode/mnode/impl/inc/mndDef.h +++ b/source/dnode/mnode/impl/inc/mndDef.h @@ -39,42 +39,34 @@ extern int32_t mDebugFlag; #define mDebug(...) { if (mDebugFlag & DEBUG_DEBUG) { taosPrintLog("MND ", mDebugFlag, __VA_ARGS__); }} #define mTrace(...) { if (mDebugFlag & DEBUG_TRACE) { taosPrintLog("MND ", mDebugFlag, __VA_ARGS__); }} -// #define mLError(...) { monSaveLog(2, __VA_ARGS__); mError(__VA_ARGS__) } -// #define mLWarn(...) { monSaveLog(1, __VA_ARGS__); mWarn(__VA_ARGS__) } -// #define mLInfo(...) { monSaveLog(0, __VA_ARGS__); mInfo(__VA_ARGS__) } - -#define mLError(...) {mError(__VA_ARGS__) } -#define mLWarn(...) {mWarn(__VA_ARGS__) } -#define mLInfo(...) {mInfo(__VA_ARGS__) } - -typedef struct SClusterObj SClusterObj; -typedef struct SDnodeObj SDnodeObj; -typedef struct SMnodeObj SMnodeObj; -typedef struct SAcctObj SAcctObj; -typedef struct SUserObj SUserObj; -typedef struct SDbObj SDbObj; -typedef struct SVgObj SVgObj; -typedef struct SSTableObj SSTableObj; -typedef struct SFuncObj SFuncObj; -typedef struct SOperObj SOperObj; +typedef struct SClusterObj SClusterObj; +typedef struct SDnodeObj SDnodeObj; +typedef struct SMnodeObj SMnodeObj; +typedef struct SAcctObj SAcctObj; +typedef struct SUserObj SUserObj; +typedef struct SDbObj SDbObj; +typedef struct SVgObj SVgObj; +typedef struct SSTableObj SSTableObj; +typedef struct SFuncObj SFuncObj; +typedef struct SOperObj SOperObj; typedef enum { - MN_AUTH_ACCT_START = 0, - MN_AUTH_ACCT_USER, - MN_AUTH_ACCT_DNODE, - MN_AUTH_ACCT_MNODE, - MN_AUTH_ACCT_DB, - MN_AUTH_ACCT_TABLE, - MN_AUTH_ACCT_MAX -} EMnAuthAcct; + MND_AUTH_ACCT_START = 0, + MND_AUTH_ACCT_USER, + MND_AUTH_ACCT_DNODE, + MND_AUTH_ACCT_MNODE, + MND_AUTH_ACCT_DB, + MND_AUTH_ACCT_TABLE, + MND_AUTH_ACCT_MAX +} EAuthAcct; typedef enum { - MN_AUTH_OP_START = 0, - MN_AUTH_OP_CREATE_USER, - MN_AUTH_OP_ALTER_USER, - MN_AUTH_OP_DROP_USER, - MN_AUTH_MAX -} EMnAuthOp; + MND_AUTH_OP_START = 0, + MND_AUTH_OP_CREATE_USER, + MND_AUTH_OP_ALTER_USER, + MND_AUTH_OP_DROP_USER, + MND_AUTH_MAX +} EAuthOp; typedef enum { TRN_STAGE_PREPARE = 1, @@ -86,7 +78,6 @@ typedef enum { typedef enum { TRN_POLICY_ROLLBACK = 1, TRN_POLICY_RETRY = 2 } ETrnPolicy; - typedef struct STrans { int32_t id; ETrnStage stage; @@ -99,7 +90,6 @@ typedef struct STrans { SArray *undoActions; } STrans; - typedef struct SClusterObj { int64_t id; char uid[TSDB_CLUSTER_ID_LEN]; @@ -202,6 +192,7 @@ typedef struct SDbObj { int64_t createdTime; int64_t updateTime; SDbCfg cfg; + int64_t uid; int8_t status; int32_t numOfVgroups; int32_t numOfTables; @@ -240,13 +231,13 @@ typedef struct SVgObj { } SVgObj; typedef struct SSTableObj { - char tableId[TSDB_TABLE_NAME_LEN]; - uint64_t uid; - int64_t createdTime; - int64_t updateTime; - int32_t numOfColumns; // used by normal table - int32_t numOfTags; - SSchema * schema; + char tableId[TSDB_TABLE_NAME_LEN]; + uint64_t uid; + int64_t createdTime; + int64_t updateTime; + int32_t numOfColumns; // used by normal table + int32_t numOfTags; + SSchema *schema; } SSTableObj; typedef struct SFuncObj { @@ -284,21 +275,22 @@ typedef struct { typedef struct { int32_t len; void *rsp; -} SMnRsp; +} SMnodeRsp; typedef struct SMnodeMsg { + SMnode *pMnode; void (*fp)(SMnodeMsg *pMsg, int32_t code); SRpcConnInfo conn; - SUserObj *pUser; - int16_t received; - int16_t successed; - int16_t expected; - int16_t retry; - int32_t code; - int64_t createdTime; - SMnRsp rpcRsp; - SRpcMsg rpcMsg; - char pCont[]; + SUserObj *pUser; + int16_t received; + int16_t successed; + int16_t expected; + int16_t retry; + int32_t code; + int64_t createdTime; + SMnodeRsp rpcRsp; + SRpcMsg rpcMsg; + char pCont[]; } SMnodeMsg; #ifdef __cplusplus diff --git a/source/dnode/mnode/impl/inc/mndDnode.h b/source/dnode/mnode/impl/inc/mndDnode.h index 5babb99a42..0c484b0724 100644 --- a/source/dnode/mnode/impl/inc/mndDnode.h +++ b/source/dnode/mnode/impl/inc/mndDnode.h @@ -22,8 +22,8 @@ extern "C" { #endif -int32_t mnodeInitDnode(); -void mnodeCleanupDnode(); +int32_t mndInitDnode(); +void mndCleanupDnode(); #ifdef __cplusplus } diff --git a/source/dnode/mnode/impl/inc/mndFunc.h b/source/dnode/mnode/impl/inc/mndFunc.h index 22ee7018f2..3e17aae389 100644 --- a/source/dnode/mnode/impl/inc/mndFunc.h +++ b/source/dnode/mnode/impl/inc/mndFunc.h @@ -22,8 +22,8 @@ extern "C" { #endif -int32_t mnodeInitFunc(); -void mnodeCleanupFunc(); +int32_t mndInitFunc(); +void mndCleanupFunc(); #ifdef __cplusplus } diff --git a/source/dnode/mnode/impl/inc/mndInt.h b/source/dnode/mnode/impl/inc/mndInt.h index 62f591a26e..ac9c715adc 100644 --- a/source/dnode/mnode/impl/inc/mndInt.h +++ b/source/dnode/mnode/impl/inc/mndInt.h @@ -18,23 +18,21 @@ #include "mndDef.h" #include "sdb.h" -#include "tstep.h" +#include "tqueue.h" #ifdef __cplusplus extern "C" { #endif typedef int32_t (*MndMsgFp)(SMnode *pMnode, SMnodeMsg *pMsg); +typedef int32_t (*MndInitFp)(SMnode *pMnode); +typedef void (*MndCleanupFp)(SMnode *pMnode); -typedef struct SMnodeBak { - int32_t dnodeId; - int64_t clusterId; - tmr_h timer; - SSteps *pInitSteps; - SSteps *pStartSteps; - SMnodeOpt para; - MndMsgFp msgFp[TSDB_MSG_TYPE_MAX]; -} SMnodeBak; +typedef struct { + const char *name; + MndInitFp initFp; + MndCleanupFp cleanupFp; +} SMnodeStep; typedef struct SMnode { int32_t dnodeId; @@ -43,25 +41,24 @@ typedef struct SMnode { int8_t selfIndex; SReplica replicas[TSDB_MAX_REPLICA]; tmr_h timer; - SSteps *pInitSteps; - SSteps *pStartSteps; - struct SSdb *pSdb; - struct SDnode *pServer; + SSdb *pSdb; + SDnode *pDnode; + SArray steps; MndMsgFp msgFp[TSDB_MSG_TYPE_MAX]; - PutMsgToMnodeQFp putMsgToApplyMsgFp; SendMsgToDnodeFp sendMsgToDnodeFp; SendMsgToMnodeFp sendMsgToMnodeFp; SendRedirectMsgFp sendRedirectMsgFp; + PutMsgToMnodeQFp putMsgToApplyMsgFp; } SMnode; -tmr_h mnodeGetTimer(); -int32_t mnodeGetDnodeId(); -int64_t mnodeGetClusterId(); +tmr_h mndGetTimer(SMnode *pMnode); +int32_t mndGetDnodeId(SMnode *pMnode); +int64_t mndGetClusterId(SMnode *pMnode); -void mnodeSendMsgToDnode(SMnode *pMnode, struct SEpSet *epSet, struct SRpcMsg *rpcMsg); -void mnodeSendMsgToMnode(SMnode *pMnode, struct SRpcMsg *rpcMsg); -void mnodeSendRedirectMsg(SMnode *pMnode, struct SRpcMsg *rpcMsg, bool forShell); -void mnodeSetMsgHandle(SMnode *pMnode, int32_t msgType, MndMsgFp fp); +void mndSendMsgToDnode(SMnode *pMnode, SEpSet *pEpSet, SRpcMsg *rpcMsg); +void mndSendMsgToMnode(SMnode *pMnode, SRpcMsg *pMsg); +void mndSendRedirectMsg(SMnode *pMnode, SRpcMsg *pMsg); +void mndSetMsgHandle(SMnode *pMnode, int32_t msgType, MndMsgFp fp); #ifdef __cplusplus } diff --git a/source/dnode/mnode/impl/inc/mndMnode.h b/source/dnode/mnode/impl/inc/mndMnode.h index 4c20cbf833..7e969baf37 100644 --- a/source/dnode/mnode/impl/inc/mndMnode.h +++ b/source/dnode/mnode/impl/inc/mndMnode.h @@ -22,8 +22,8 @@ extern "C" { #endif -int32_t mnodeInitMnode(); -void mnodeCleanupMnode(); +int32_t mndInitMnode(); +void mndCleanupMnode(); void mnodeGetMnodeEpSetForPeer(SEpSet *epSet, bool redirect); void mnodeGetMnodeEpSetForShell(SEpSet *epSet, bool redirect); diff --git a/source/dnode/mnode/impl/inc/mndOper.h b/source/dnode/mnode/impl/inc/mndOper.h index 7ab396a019..d07567635d 100644 --- a/source/dnode/mnode/impl/inc/mndOper.h +++ b/source/dnode/mnode/impl/inc/mndOper.h @@ -20,8 +20,8 @@ extern "C" { #endif -int32_t mnodeInitOper(); -void mnodeCleanupOper(); +int32_t mndInitOper(); +void mndCleanupOper(); #ifdef __cplusplus } diff --git a/source/dnode/mnode/impl/inc/mndProfile.h b/source/dnode/mnode/impl/inc/mndProfile.h index d2b402227b..4433478caa 100644 --- a/source/dnode/mnode/impl/inc/mndProfile.h +++ b/source/dnode/mnode/impl/inc/mndProfile.h @@ -22,8 +22,8 @@ extern "C" { #endif -int32_t mnodeInitProfile(); -void mnodeCleanupProfile(); +int32_t mndInitProfile(); +void mndCleanupProfile(); #ifdef __cplusplus } diff --git a/source/dnode/mnode/impl/inc/mndShow.h b/source/dnode/mnode/impl/inc/mndShow.h index cc3bd55cf4..2161c81b74 100644 --- a/source/dnode/mnode/impl/inc/mndShow.h +++ b/source/dnode/mnode/impl/inc/mndShow.h @@ -22,8 +22,8 @@ extern "C" { #endif -int32_t mnodeInitShow(); -void mnodeCleanUpShow(); +int32_t mndInitShow(); +void mndCleanupShow(); #ifdef __cplusplus } diff --git a/source/dnode/mnode/impl/inc/mndStable.h b/source/dnode/mnode/impl/inc/mndStable.h index b2469aaaa8..429f4d947a 100644 --- a/source/dnode/mnode/impl/inc/mndStable.h +++ b/source/dnode/mnode/impl/inc/mndStable.h @@ -22,8 +22,8 @@ extern "C" { #endif -int32_t mnodeInitStable(); -void mnodeCleanupStable(); +int32_t mndInitStable(); +void mndCleanupStable(); #ifdef __cplusplus } diff --git a/source/dnode/mnode/impl/inc/mndSync.h b/source/dnode/mnode/impl/inc/mndSync.h index 863aaccf0c..7d7f9d69d0 100644 --- a/source/dnode/mnode/impl/inc/mndSync.h +++ b/source/dnode/mnode/impl/inc/mndSync.h @@ -22,8 +22,8 @@ extern "C" { #endif -int32_t mnodeInitSync(); -void mnodeCleanUpSync(); +int32_t mndInitSync(); +void mndCleanupSync(); int32_t mnodeSyncPropose(SSdbRaw *pRaw, void *pData); bool mnodeIsMaster(); diff --git a/source/dnode/mnode/impl/inc/mndTelem.h b/source/dnode/mnode/impl/inc/mndTelem.h index f2a11a3d32..ccee629aad 100644 --- a/source/dnode/mnode/impl/inc/mndTelem.h +++ b/source/dnode/mnode/impl/inc/mndTelem.h @@ -21,8 +21,8 @@ extern "C" { #endif #include "mndInt.h" -int32_t mnodeInitTelem(); -void mnodeCleanupTelem(); +int32_t mndInitTelem(); +void mndCleanupTelem(); #ifdef __cplusplus } diff --git a/source/dnode/mnode/impl/inc/mndTrans.h b/source/dnode/mnode/impl/inc/mndTrans.h index ba6b9fff79..42a5c391b4 100644 --- a/source/dnode/mnode/impl/inc/mndTrans.h +++ b/source/dnode/mnode/impl/inc/mndTrans.h @@ -22,8 +22,8 @@ extern "C" { #endif -int32_t mnodeInitTrans(); -void mnodeCleanupTrans(); +int32_t mndInitTrans(); +void mndCleanupTrans(); STrans *trnCreate(ETrnPolicy policy, void *rpcHandle); void trnDrop(STrans *pTrans); diff --git a/source/dnode/mnode/impl/inc/mndUser.h b/source/dnode/mnode/impl/inc/mndUser.h index adc7a0db13..19554ccd37 100644 --- a/source/dnode/mnode/impl/inc/mndUser.h +++ b/source/dnode/mnode/impl/inc/mndUser.h @@ -22,8 +22,8 @@ extern "C" { #endif -int32_t mnodeInitUser(); -void mnodeCleanupUser(); +int32_t mndInitUser(); +void mndCleanupUser(); #ifdef __cplusplus } diff --git a/source/dnode/mnode/impl/inc/mndVgroup.h b/source/dnode/mnode/impl/inc/mndVgroup.h index 18c2ea7ecf..bf2cb60092 100644 --- a/source/dnode/mnode/impl/inc/mndVgroup.h +++ b/source/dnode/mnode/impl/inc/mndVgroup.h @@ -22,8 +22,8 @@ extern "C" { #endif -int32_t mnodeInitVgroup(); -void mnodeCleanupVgroup(); +int32_t mndInitVgroup(); +void mndCleanupVgroup(); #ifdef __cplusplus } diff --git a/source/dnode/mnode/impl/inc/mnodeInt.h b/source/dnode/mnode/impl/inc/mnodeInt.h deleted file mode 100644 index 62f591a26e..0000000000 --- a/source/dnode/mnode/impl/inc/mnodeInt.h +++ /dev/null @@ -1,70 +0,0 @@ -/* - * 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_MND_INT_H_ -#define _TD_MND_INT_H_ - -#include "mndDef.h" -#include "sdb.h" -#include "tstep.h" - -#ifdef __cplusplus -extern "C" { -#endif - -typedef int32_t (*MndMsgFp)(SMnode *pMnode, SMnodeMsg *pMsg); - -typedef struct SMnodeBak { - int32_t dnodeId; - int64_t clusterId; - tmr_h timer; - SSteps *pInitSteps; - SSteps *pStartSteps; - SMnodeOpt para; - MndMsgFp msgFp[TSDB_MSG_TYPE_MAX]; -} SMnodeBak; - -typedef struct SMnode { - int32_t dnodeId; - int64_t clusterId; - int8_t replica; - int8_t selfIndex; - SReplica replicas[TSDB_MAX_REPLICA]; - tmr_h timer; - SSteps *pInitSteps; - SSteps *pStartSteps; - struct SSdb *pSdb; - struct SDnode *pServer; - MndMsgFp msgFp[TSDB_MSG_TYPE_MAX]; - PutMsgToMnodeQFp putMsgToApplyMsgFp; - SendMsgToDnodeFp sendMsgToDnodeFp; - SendMsgToMnodeFp sendMsgToMnodeFp; - SendRedirectMsgFp sendRedirectMsgFp; -} SMnode; - -tmr_h mnodeGetTimer(); -int32_t mnodeGetDnodeId(); -int64_t mnodeGetClusterId(); - -void mnodeSendMsgToDnode(SMnode *pMnode, struct SEpSet *epSet, struct SRpcMsg *rpcMsg); -void mnodeSendMsgToMnode(SMnode *pMnode, struct SRpcMsg *rpcMsg); -void mnodeSendRedirectMsg(SMnode *pMnode, struct SRpcMsg *rpcMsg, bool forShell); -void mnodeSetMsgHandle(SMnode *pMnode, int32_t msgType, MndMsgFp fp); - -#ifdef __cplusplus -} -#endif - -#endif /*_TD_MND_INT_H_*/ diff --git a/source/dnode/mnode/impl/inc/mnodeUser.h b/source/dnode/mnode/impl/inc/mnodeUser.h deleted file mode 100644 index adc7a0db13..0000000000 --- a/source/dnode/mnode/impl/inc/mnodeUser.h +++ /dev/null @@ -1,32 +0,0 @@ -/* - * 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_MND_USER_H_ -#define _TD_MND_USER_H_ - -#include "mndInt.h" - -#ifdef __cplusplus -extern "C" { -#endif - -int32_t mnodeInitUser(); -void mnodeCleanupUser(); - -#ifdef __cplusplus -} -#endif - -#endif /*_TD_MND_USER_H_*/ diff --git a/source/dnode/mnode/impl/src/mndAcct.c b/source/dnode/mnode/impl/src/mndAcct.c index b503a27b11..c8087dbc2d 100644 --- a/source/dnode/mnode/impl/src/mndAcct.c +++ b/source/dnode/mnode/impl/src/mndAcct.c @@ -101,7 +101,7 @@ static int32_t mnodeCreateDefaultAcct() { return sdbWrite(pRaw); } -int32_t mnodeInitAcct() { +int32_t mndInitAcct() { SSdbTable table = {.sdbType = SDB_ACCT, .keyType = SDB_KEY_BINARY, .deployFp = (SdbDeployFp)mnodeCreateDefaultAcct, @@ -115,4 +115,4 @@ int32_t mnodeInitAcct() { return 0; } -void mnodeCleanupAcct() {} +void mndCleanupAcct() {} diff --git a/source/dnode/mnode/impl/src/mndAuth.c b/source/dnode/mnode/impl/src/mndAuth.c index bb50645cee..d37c7bd4da 100644 --- a/source/dnode/mnode/impl/src/mndAuth.c +++ b/source/dnode/mnode/impl/src/mndAuth.c @@ -17,9 +17,9 @@ #include "os.h" #include "mndAuth.h" -int32_t mnodeInitAuth() { return 0; } -void mnodeCleanupAuth() {} +int32_t mndInitAuth() { return 0; } +void mndCleanupAuth() {} -int32_t mnodeRetriveAuth(SMnode *pMnode, char *user, char *spi, char *encrypt, char *secret, char *ckey) { +int32_t mndRetriveAuth(SMnode *pMnode, char *user, char *spi, char *encrypt, char *secret, char *ckey) { return 0; } \ No newline at end of file diff --git a/source/dnode/mnode/impl/src/mndBalance.c b/source/dnode/mnode/impl/src/mndBalance.c index f3acb9b4b3..7b919af3f8 100644 --- a/source/dnode/mnode/impl/src/mndBalance.c +++ b/source/dnode/mnode/impl/src/mndBalance.c @@ -17,5 +17,5 @@ #include "os.h" #include "mndInt.h" -int32_t mnodeInitBalance() { return 0; } -void mnodeCleanupBalance() {} \ No newline at end of file +int32_t mndInitBalance() { return 0; } +void mndCleanupBalance() {} \ No newline at end of file diff --git a/source/dnode/mnode/impl/src/mndCluster.c b/source/dnode/mnode/impl/src/mndCluster.c index f1dbe1df7d..f926959b9f 100644 --- a/source/dnode/mnode/impl/src/mndCluster.c +++ b/source/dnode/mnode/impl/src/mndCluster.c @@ -17,5 +17,5 @@ #include "os.h" #include "mndInt.h" -int32_t mnodeInitCluster() { return 0; } -void mnodeCleanupCluster() {} \ No newline at end of file +int32_t mndInitCluster() { return 0; } +void mndCleanupCluster() {} \ No newline at end of file diff --git a/source/dnode/mnode/impl/src/mndDb.c b/source/dnode/mnode/impl/src/mndDb.c index c8aa347c7b..4c0e54f436 100644 --- a/source/dnode/mnode/impl/src/mndDb.c +++ b/source/dnode/mnode/impl/src/mndDb.c @@ -17,5 +17,5 @@ #include "os.h" #include "mndInt.h" -int32_t mnodeInitDb() { return 0; } -void mnodeCleanupDb() {} \ No newline at end of file +int32_t mndInitDb() { return 0; } +void mndCleanupDb() {} \ No newline at end of file diff --git a/source/dnode/mnode/impl/src/mndDnode.c b/source/dnode/mnode/impl/src/mndDnode.c index 61cd4c01bc..f9329e871f 100644 --- a/source/dnode/mnode/impl/src/mndDnode.c +++ b/source/dnode/mnode/impl/src/mndDnode.c @@ -17,5 +17,5 @@ #include "os.h" #include "mndInt.h" -int32_t mnodeInitDnode() { return 0; } -void mnodeCleanupDnode() {} \ No newline at end of file +int32_t mndInitDnode() { return 0; } +void mndCleanupDnode() {} \ No newline at end of file diff --git a/source/dnode/mnode/impl/src/mndFunc.c b/source/dnode/mnode/impl/src/mndFunc.c index 19a8b8dd0e..a84178ea6e 100644 --- a/source/dnode/mnode/impl/src/mndFunc.c +++ b/source/dnode/mnode/impl/src/mndFunc.c @@ -17,5 +17,5 @@ #include "os.h" #include "mndInt.h" -int32_t mnodeInitFunc() { return 0; } -void mnodeCleanupFunc() {} \ No newline at end of file +int32_t mndInitFunc() { return 0; } +void mndCleanupFunc() {} \ No newline at end of file diff --git a/source/dnode/mnode/impl/src/mndMnode.c b/source/dnode/mnode/impl/src/mndMnode.c index da4d77b440..ce38049a36 100644 --- a/source/dnode/mnode/impl/src/mndMnode.c +++ b/source/dnode/mnode/impl/src/mndMnode.c @@ -17,8 +17,8 @@ #include "os.h" #include "mndInt.h" -int32_t mnodeInitMnode() { return 0; } -void mnodeCleanupMnode() {} +int32_t mndInitMnode() { return 0; } +void mndCleanupMnode() {} void mnodeGetMnodeEpSetForPeer(SEpSet *epSet, bool redirect) {} void mnodeGetMnodeEpSetForShell(SEpSet *epSet, bool redirect) {} \ No newline at end of file diff --git a/source/dnode/mnode/impl/src/mndOper.c b/source/dnode/mnode/impl/src/mndOper.c index d25748882a..aa3a1d5e25 100644 --- a/source/dnode/mnode/impl/src/mndOper.c +++ b/source/dnode/mnode/impl/src/mndOper.c @@ -17,5 +17,5 @@ #include "os.h" #include "mndInt.h" -int32_t mnodeInitOper() { return 0; } -void mnodeCleanupOper() {} \ No newline at end of file +int32_t mndInitOper() { return 0; } +void mndCleanupOper() {} \ No newline at end of file diff --git a/source/dnode/mnode/impl/src/mndProfile.c b/source/dnode/mnode/impl/src/mndProfile.c index 64378b0093..7c86778eb4 100644 --- a/source/dnode/mnode/impl/src/mndProfile.c +++ b/source/dnode/mnode/impl/src/mndProfile.c @@ -17,5 +17,5 @@ #include "os.h" #include "mndInt.h" -int32_t mnodeInitProfile() { return 0; } -void mnodeCleanupProfile() {} \ No newline at end of file +int32_t mndInitProfile() { return 0; } +void mndCleanupProfile() {} \ No newline at end of file diff --git a/source/dnode/mnode/impl/src/mndShow.c b/source/dnode/mnode/impl/src/mndShow.c index 338c199afc..3f97470fd4 100644 --- a/source/dnode/mnode/impl/src/mndShow.c +++ b/source/dnode/mnode/impl/src/mndShow.c @@ -17,5 +17,5 @@ #include "os.h" #include "mndInt.h" -int32_t mnodeInitShow() { return 0; } -void mnodeCleanUpShow() {} \ No newline at end of file +int32_t mndInitShow() { return 0; } +void mndCleanupShow() {} \ No newline at end of file diff --git a/source/dnode/mnode/impl/src/mndStable.c b/source/dnode/mnode/impl/src/mndStable.c index fe0fc59d25..0a8320c9fa 100644 --- a/source/dnode/mnode/impl/src/mndStable.c +++ b/source/dnode/mnode/impl/src/mndStable.c @@ -17,5 +17,5 @@ #include "os.h" #include "mndInt.h" -int32_t mnodeInitStable() { return 0; } -void mnodeCleanupStable() {} \ No newline at end of file +int32_t mndInitStable() { return 0; } +void mndCleanupStable() {} \ No newline at end of file diff --git a/source/dnode/mnode/impl/src/mndSync.c b/source/dnode/mnode/impl/src/mndSync.c index 4ba0c7dc27..8db79e1e32 100644 --- a/source/dnode/mnode/impl/src/mndSync.c +++ b/source/dnode/mnode/impl/src/mndSync.c @@ -18,8 +18,8 @@ #include "mndInt.h" #include "mndTrans.h" -int32_t mnodeInitSync() { return 0; } -void mnodeCleanUpSync() {} +int32_t mndInitSync() { return 0; } +void mndCleanupSync() {} int32_t mnodeSyncPropose(SSdbRaw *pRaw, void *pData) { trnApply(pData, pData, 0); diff --git a/source/dnode/mnode/impl/src/mndTelem.c b/source/dnode/mnode/impl/src/mndTelem.c index 50934f05e4..e9748ec12e 100644 --- a/source/dnode/mnode/impl/src/mndTelem.c +++ b/source/dnode/mnode/impl/src/mndTelem.c @@ -174,7 +174,7 @@ static void mnodeAddVersionInfo(SBufferWriter* bw) { static void mnodeAddRuntimeInfo(SBufferWriter* bw) { SMnodeLoad load = {0}; - if (mnodeGetLoad(NULL, &load) != 0) { + if (mndGetLoad(NULL, &load) != 0) { return; } @@ -203,7 +203,7 @@ static void mnodeSendTelemetryReport() { return; } - int64_t clusterId = mnodeGetClusterId(); + int64_t clusterId = mndGetClusterId(NULL); char clusterIdStr[20] = {0}; snprintf(clusterIdStr, sizeof(clusterIdStr), "%" PRId64, clusterId); @@ -278,7 +278,7 @@ static void mnodeGetEmail(char* filepath) { taosCloseFile(fd); } -int32_t mnodeInitTelem() { +int32_t mndInitTelem() { tsTelem.enable = tsEnableTelemetryReporting; if (!tsTelem.enable) return 0; @@ -303,7 +303,7 @@ int32_t mnodeInitTelem() { return 0; } -void mnodeCleanupTelem() { +void mndCleanupTelem() { if (!tsTelem.enable) return; if (taosCheckPthreadValid(tsTelem.thread)) { diff --git a/source/dnode/mnode/impl/src/mndTrans.c b/source/dnode/mnode/impl/src/mndTrans.c index 2ea889903e..ed5a2a7b14 100644 --- a/source/dnode/mnode/impl/src/mndTrans.c +++ b/source/dnode/mnode/impl/src/mndTrans.c @@ -312,7 +312,7 @@ int32_t trnAppendUndoAction(STrans *pTrans, SEpSet *pEpSet, void *pMsg) { return code; } -int32_t mnodeInitTrans() { +int32_t mndInitTrans() { SSdbTable table = {.sdbType = SDB_TRANS, .keyType = SDB_KEY_INT32, .encodeFp = (SdbEncodeFp)trnActionEncode, @@ -326,7 +326,7 @@ int32_t mnodeInitTrans() { return 0; } -void mnodeCleanupTrans() { mInfo("trn module is cleaned up"); } +void mndCleanupTrans() { mInfo("trn module is cleaned up"); } int32_t trnPrepare(STrans *pTrans, int32_t (*syncfp)(SSdbRaw *pRaw, void *pData)) { diff --git a/source/dnode/mnode/impl/src/mndUser.c b/source/dnode/mnode/impl/src/mndUser.c index e8cc6df378..a51133d427 100644 --- a/source/dnode/mnode/impl/src/mndUser.c +++ b/source/dnode/mnode/impl/src/mndUser.c @@ -220,7 +220,7 @@ static int32_t mnodeProcessCreateUserMsg(SMnode *pMnode, SMnodeMsg *pMsg) { return TSDB_CODE_MND_ACTION_IN_PROGRESS; } -int32_t mnodeInitUser() { +int32_t mndInitUser() { SSdbTable table = {.sdbType = SDB_USER, .keyType = SDB_KEY_BINARY, .deployFp = (SdbDeployFp)mnodeCreateDefaultUsers, @@ -231,9 +231,9 @@ int32_t mnodeInitUser() { .deleteFp = (SdbDeleteFp)mnodeUserActionDelete}; sdbSetTable(table); - mnodeSetMsgHandle(NULL, TSDB_MSG_TYPE_CREATE_USER, mnodeProcessCreateUserMsg); + mndSetMsgHandle(NULL, TSDB_MSG_TYPE_CREATE_USER, mnodeProcessCreateUserMsg); return 0; } -void mnodeCleanupUser() {} \ No newline at end of file +void mndCleanupUser() {} \ No newline at end of file diff --git a/source/dnode/mnode/impl/src/mndVgroup.c b/source/dnode/mnode/impl/src/mndVgroup.c index 134603ce27..f7af6e5fa8 100644 --- a/source/dnode/mnode/impl/src/mndVgroup.c +++ b/source/dnode/mnode/impl/src/mndVgroup.c @@ -17,5 +17,5 @@ #include "os.h" #include "mndInt.h" -int32_t mnodeInitVgroup() { return 0; } -void mnodeCleanupVgroup() {} \ No newline at end of file +int32_t mndInitVgroup() { return 0; } +void mndCleanupVgroup() {} \ No newline at end of file diff --git a/source/dnode/mnode/impl/src/mnode.c b/source/dnode/mnode/impl/src/mnode.c index 30eb85b9d1..426c86f728 100644 --- a/source/dnode/mnode/impl/src/mnode.c +++ b/source/dnode/mnode/impl/src/mnode.c @@ -14,9 +14,6 @@ */ #define _DEFAULT_SOURCE -#include "os.h" -#include "tglobal.h" -#include "tqueue.h" #include "mndAcct.h" #include "mndAuth.h" #include "mndBalance.h" @@ -35,55 +32,70 @@ #include "mndUser.h" #include "mndVgroup.h" -SMnodeBak tsMint = {0}; - -int32_t mnodeGetDnodeId() { return tsMint.para.dnodeId; } - -int64_t mnodeGetClusterId() { return tsMint.para.clusterId; } - -void mnodeSendMsgToDnode(SMnode *pMnode, struct SEpSet *epSet, struct SRpcMsg *rpcMsg) { - assert(pMnode); - (*pMnode->sendMsgToDnodeFp)(pMnode->pServer, epSet, rpcMsg); +int32_t mndGetDnodeId(SMnode *pMnode) { + if (pMnode != NULL) { + return pMnode->dnodeId; + } + return -1; } -void mnodeSendMsgToMnode(SMnode *pMnode, struct SRpcMsg *rpcMsg) { - assert(pMnode); - (*pMnode->sendMsgToMnodeFp)(pMnode->pServer, rpcMsg); +int64_t mndGetClusterId(SMnode *pMnode) { + if (pMnode != NULL) { + return pMnode->clusterId; + } + return -1; } -void mnodeSendRedirectMsg(SMnode *pMnode, struct SRpcMsg *rpcMsg, bool forShell) { - assert(pMnode); - (*pMnode->sendRedirectMsgFp)(pMnode->pServer, rpcMsg); +void mndSendMsgToDnode(SMnode *pMnode, SEpSet *pEpSet, SRpcMsg *pMsg) { + if (pMnode != NULL && pMnode->sendMsgToDnodeFp != NULL) { + (*pMnode->sendMsgToDnodeFp)(pMnode->pDnode, pEpSet, pMsg); + } } -static int32_t mnodeInitTimer() { - if (tsMint.timer == NULL) { - tsMint.timer = taosTmrInit(tsMaxShellConns, 200, 3600000, "MND"); +void mndSendMsgToMnode(SMnode *pMnode, SRpcMsg *pMsg) { + if (pMnode != NULL && pMnode->sendMsgToMnodeFp != NULL) { + (*pMnode->sendMsgToMnodeFp)(pMnode->pDnode, pMsg); + } +} + +void mndSendRedirectMsg(SMnode *pMnode, SRpcMsg *pMsg) { + if (pMnode != NULL && pMnode->sendRedirectMsgFp != NULL) { + (*pMnode->sendRedirectMsgFp)(pMnode->pDnode, pMsg); + } +} + +static int32_t mndInitTimer(SMnode *pMnode) { + if (pMnode->timer == NULL) { + pMnode->timer = taosTmrInit(5000, 200, 3600000, "MND"); } - if (tsMint.timer == NULL) { + if (pMnode->timer == NULL) { return -1; } return 0; } -static void mnodeCleanupTimer() { - if (tsMint.timer != NULL) { - taosTmrCleanUp(tsMint.timer); - tsMint.timer = NULL; +static void mndCleanupTimer(SMnode *pMnode) { + if (pMnode->timer != NULL) { + taosTmrCleanUp(pMnode->timer); + pMnode->timer = NULL; } } -tmr_h mnodeGetTimer() { return tsMint.timer; } +tmr_h mndGetTimer(SMnode *pMnode) { + if (pMnode != NULL) { + return pMnode->timer; + } +} -static int32_t mnodeSetOptions(SMnode *pMnode, const SMnodeOpt *pOption) { +static int32_t mndSetOptions(SMnode *pMnode, const SMnodeOpt *pOption) { pMnode->dnodeId = pOption->dnodeId; pMnode->clusterId = pOption->clusterId; pMnode->replica = pOption->replica; pMnode->selfIndex = pOption->selfIndex; memcpy(&pMnode->replicas, pOption->replicas, sizeof(SReplica) * TSDB_MAX_REPLICA); - pMnode->pServer = pOption->pDnode; + pMnode->pDnode = pOption->pDnode; pMnode->putMsgToApplyMsgFp = pOption->putMsgToApplyMsgFp; pMnode->sendMsgToDnodeFp = pOption->sendMsgToDnodeFp; pMnode->sendMsgToMnodeFp = pOption->sendMsgToMnodeFp; @@ -98,88 +110,143 @@ static int32_t mnodeSetOptions(SMnode *pMnode, const SMnodeOpt *pOption) { return 0; } -static int32_t mnodeAllocInitSteps() { - struct SSteps *steps = taosStepInit(16, NULL); - if (steps == NULL) return -1; +static int32_t mndAllocStep(SMnode *pMnode, char *name, MndInitFp initFp, MndCleanupFp cleanupFp) { + SMnodeStep step = {0}; + step.name = name; + step.initFp = initFp; + step.cleanupFp = cleanupFp; + if (taosArrayPush(&pMnode->steps, &step) != NULL) { + terrno = TSDB_CODE_OUT_OF_MEMORY; + mError("failed to alloc step:%s since %s", name, terrstr()); + return -1; + } - if (taosStepAdd(steps, "mnode-trans", mnodeInitTrans, mnodeCleanupTrans) != 0) return -1; - if (taosStepAdd(steps, "mnode-cluster", mnodeInitCluster, mnodeCleanupCluster) != 0) return -1; - if (taosStepAdd(steps, "mnode-dnode", mnodeInitDnode, mnodeCleanupDnode) != 0) return -1; - if (taosStepAdd(steps, "mnode-mnode", mnodeInitMnode, mnodeCleanupMnode) != 0) return -1; - if (taosStepAdd(steps, "mnode-acct", mnodeInitAcct, mnodeCleanupAcct) != 0) return -1; - if (taosStepAdd(steps, "mnode-auth", mnodeInitAuth, mnodeCleanupAuth) != 0) return -1; - if (taosStepAdd(steps, "mnode-user", mnodeInitUser, mnodeCleanupUser) != 0) return -1; - if (taosStepAdd(steps, "mnode-db", mnodeInitDb, mnodeCleanupDb) != 0) return -1; - if (taosStepAdd(steps, "mnode-vgroup", mnodeInitVgroup, mnodeCleanupVgroup) != 0) return -1; - if (taosStepAdd(steps, "mnode-stable", mnodeInitStable, mnodeCleanupStable) != 0) return -1; - if (taosStepAdd(steps, "mnode-func", mnodeInitFunc, mnodeCleanupFunc) != 0) return -1; - if (taosStepAdd(steps, "mnode-sdb", sdbInit, sdbCleanup) != 0) return -1; - - tsMint.pInitSteps = steps; return 0; } -static int32_t mnodeAllocStartSteps() { - struct SSteps *steps = taosStepInit(8, NULL); - if (steps == NULL) return -1; +static int32_t mndInitSteps(SMnode *pMnode) { + if (mndAllocStep(pMnode, "mnode-trans", mndInitTrans, mndCleanupTrans) != 0) return -1; + if (mndAllocStep(pMnode, "mnode-cluster", mndInitCluster, mndCleanupCluster) != 0) return -1; + if (mndAllocStep(pMnode, "mnode-dnode", mndInitDnode, mndCleanupDnode) != 0) return -1; + if (mndAllocStep(pMnode, "mnode-mnode", mndInitMnode, mndCleanupMnode) != 0) return -1; + if (mndAllocStep(pMnode, "mnode-acct", mndInitAcct, mndCleanupAcct) != 0) return -1; + if (mndAllocStep(pMnode, "mnode-auth", mndInitAuth, mndCleanupAuth) != 0) return -1; + if (mndAllocStep(pMnode, "mnode-user", mndInitUser, mndCleanupUser) != 0) return -1; + if (mndAllocStep(pMnode, "mnode-db", mndInitDb, mndCleanupDb) != 0) return -1; + if (mndAllocStep(pMnode, "mnode-vgroup", mndInitVgroup, mndCleanupVgroup) != 0) return -1; + if (mndAllocStep(pMnode, "mnode-stable", mndInitStable, mndCleanupStable) != 0) return -1; + if (mndAllocStep(pMnode, "mnode-func", mndInitFunc, mndCleanupFunc) != 0) return -1; + if (mndAllocStep(pMnode, "mnode-sdb", sdbInit, sdbCleanup) != 0) return -1; - taosStepAdd(steps, "mnode-timer", mnodeInitTimer, NULL); - taosStepAdd(steps, "mnode-sdb-file", sdbOpen, sdbClose); - taosStepAdd(steps, "mnode-balance", mnodeInitBalance, mnodeCleanupBalance); - taosStepAdd(steps, "mnode-profile", mnodeInitProfile, mnodeCleanupProfile); - taosStepAdd(steps, "mnode-show", mnodeInitShow, mnodeCleanUpShow); - taosStepAdd(steps, "mnode-sync", mnodeInitSync, mnodeCleanUpSync); - taosStepAdd(steps, "mnode-telem", mnodeInitTelem, mnodeCleanupTelem); - taosStepAdd(steps, "mnode-timer", NULL, mnodeCleanupTimer); + if (pMnode->replica == 1) { + if (mndAllocStep(pMnode, "mnode-deploy-sdb", sdbDeploy, sdbClose) != 0) return -1; + } else { + if (mndAllocStep(pMnode, "mnode-open-sdb", sdbOpen, sdbClose) != 0) return -1; + } + + if (mndAllocStep(pMnode, "mnode-timer", mndInitTimer, NULL) != 0) return -1; + if (mndAllocStep(pMnode, "mnode-sdb-file", sdbOpen, sdbClose) != 0) return -1; + if (mndAllocStep(pMnode, "mnode-balance", mndInitBalance, mndCleanupBalance) != 0) return -1; + if (mndAllocStep(pMnode, "mnode-profile", mndInitProfile, mndCleanupProfile) != 0) return -1; + if (mndAllocStep(pMnode, "mnode-show", mndInitShow, mndCleanupShow) != 0) return -1; + if (mndAllocStep(pMnode, "mnode-sync", mndInitSync, mndCleanupSync) != 0) return -1; + if (mndAllocStep(pMnode, "mnode-telem", mndInitTelem, mndCleanupTelem) != 0) return -1; + if (mndAllocStep(pMnode, "mnode-timer", NULL, mndCleanupTimer) != 0) return -1; - tsMint.pStartSteps = steps; return 0; } -SMnode *mnodeOpen(const char *path, const SMnodeOpt *pOption) { - SMnode *pMnode = calloc(1, sizeof(SMnode)); - - if (mnodeSetOptions(pMnode, pOption) != 0) { - free(pMnode); - mError("failed to init mnode options since %s", terrstr()); - return NULL; +static void mndCleanupSteps(SMnode *pMnode, int32_t pos) { + if (pos == -1) { + pos = taosArrayGetSize(&pMnode->steps); } - if (mnodeAllocInitSteps() != 0) { - mError("failed to alloc init steps since %s", terrstr()); - return NULL; - } - - if (mnodeAllocStartSteps() != 0) { - mError("failed to alloc start steps since %s", terrstr()); - return NULL; - } - - taosStepExec(tsMint.pInitSteps); - - if (tsMint.para.dnodeId <= 0 && tsMint.para.clusterId <= 0) { - if (sdbDeploy() != 0) { - mError("failed to deploy sdb since %s", terrstr()); - return NULL; - } else { - mInfo("mnode is deployed"); + for (int32_t s = pos; s >= 0; s--) { + SMnodeStep *pStep = taosArrayGet(&pMnode->steps, pos); + mDebug("step:%s will cleanup", pStep->name); + if (pStep->cleanupFp != NULL) { + (*pStep->cleanupFp)(pMnode); } } - taosStepExec(tsMint.pStartSteps); + taosArrayClear(&pMnode->steps); +} +static int32_t mndExecSteps(SMnode *pMnode) { + int32_t size = taosArrayGetSize(&pMnode->steps); + for (int32_t pos = 0; pos < size; pos++) { + SMnodeStep *pStep = taosArrayGet(&pMnode->steps, pos); + if (pStep->initFp == NULL) continue; + + // (*pMnode->reportProgress)(pStep->name, "start initialize"); + + int32_t code = (*pStep->initFp)(pMnode); + if (code != 0) { + mError("step:%s exec failed since %s, start to cleanup", pStep->name, tstrerror(code)); + mndCleanupSteps(pMnode, pos); + terrno = code; + return code; + } else { + mDebug("step:%s is initialized", pStep->name); + } + + // (*pMnode->reportProgress)(pStep->name, "initialize completed"); + } +} + +SMnode *mndOpen(const char *path, const SMnodeOpt *pOption) { + SMnode *pMnode = calloc(1, sizeof(SMnode)); + + int32_t code = mndSetOptions(pMnode, pOption); + if (code != 0) { + mndClose(pMnode); + terrno = code; + mError("failed to set mnode options since %s", terrstr()); + return NULL; + } + + code = mndInitSteps(pMnode); + if (code != 0) { + mndClose(pMnode); + terrno = code; + mError("failed to int steps since %s", terrstr()); + return NULL; + } + + code = mndExecSteps(pMnode); + if (code != 0) { + mndClose(pMnode); + terrno = code; + mError("failed to execute steps since %s", terrstr()); + return NULL; + } + + mDebug("mnode:%p object is created", pMnode); return pMnode; } -void mnodeClose(SMnode *pMnode) { free(pMnode); } +void mndClose(SMnode *pMnode) { + mndCleanupSteps(pMnode, -1); + free(pMnode); + mDebug("mnode:%p object is cleaned up", pMnode); +} -int32_t mnodeAlter(SMnode *pMnode, const SMnodeOpt *pOption) { return 0; } +int32_t mndAlter(SMnode *pMnode, const SMnodeOpt *pOption) { + assert(1); + return 0; +} -void mnodeDestroy(const char *path) { sdbUnDeploy(); } +void mndDestroy(const char *path) { + mDebug("mnode in %s will be destroyed", path); + sdbUnDeploy(); +} -int32_t mnodeGetLoad(SMnode *pMnode, SMnodeLoad *pLoad) { return 0; } +int32_t mndGetLoad(SMnode *pMnode, SMnodeLoad *pLoad) { + assert(1); + return 0; +} -SMnodeMsg *mnodeInitMsg(SMnode *pMnode, SRpcMsg *pRpcMsg) { +SMnodeMsg *mndInitMsg(SMnode *pMnode, SRpcMsg *pRpcMsg) { SMnodeMsg *pMsg = taosAllocateQitem(sizeof(SMnodeMsg)); if (pMsg == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; @@ -187,7 +254,7 @@ SMnodeMsg *mnodeInitMsg(SMnode *pMnode, SRpcMsg *pRpcMsg) { } if (rpcGetConnInfo(pRpcMsg->handle, &pMsg->conn) != 0) { - mnodeCleanupMsg(pMsg); + mndCleanupMsg(pMsg); mError("can not get user from conn:%p", pMsg->rpcMsg.handle); terrno = TSDB_CODE_MND_NO_USER_FROM_CONN; return NULL; @@ -199,7 +266,7 @@ SMnodeMsg *mnodeInitMsg(SMnode *pMnode, SRpcMsg *pRpcMsg) { return pMsg; } -void mnodeCleanupMsg(SMnodeMsg *pMsg) { +void mndCleanupMsg(SMnodeMsg *pMsg) { if (pMsg->pUser != NULL) { sdbRelease(pMsg->pUser); } @@ -207,40 +274,50 @@ void mnodeCleanupMsg(SMnodeMsg *pMsg) { taosFreeQitem(pMsg); } -void mnodeSendRsp(SMnodeMsg *pMsg, int32_t code) {} +void mndSendRsp(SMnodeMsg *pMsg, int32_t code) {} -static void mnodeProcessRpcMsg(SMnodeMsg *pMsg) { - if (!mnodeIsMaster()) { - mnodeSendRedirectMsg(NULL, &pMsg->rpcMsg, true); - mnodeCleanupMsg(pMsg); +static void mndProcessRpcMsg(SMnodeMsg *pMsg) { + SMnode *pMnode = pMsg->pMnode; + + if (!mnodeIsMaster(pMnode)) { + mndSendRedirectMsg(pMnode, &pMsg->rpcMsg); + mndCleanupMsg(pMsg); return; } - int32_t msgType = pMsg->rpcMsg.msgType; - - MndMsgFp fp = tsMint.msgFp[msgType]; + int32_t msgType = pMsg->rpcMsg.msgType; + MndMsgFp fp = pMnode->msgFp[msgType]; if (fp == NULL) { + mError("RPC %p, req:%s is not processed", pMsg->rpcMsg.handle, taosMsg[msgType]); + SRpcMsg rspMsg = {.handle = pMsg->rpcMsg.handle, .code = TSDB_CODE_MSG_NOT_PROCESSED}; + rpcSendResponse(&rspMsg); + mndCleanupMsg(pMsg); + return; } - int32_t code = (*fp)(NULL, pMsg); + int32_t code = (*fp)(pMnode, pMsg); if (code != 0) { - assert(code); + mError("RPC %p, req:%s processed error since %s", pMsg->rpcMsg.handle, taosMsg[msgType], tstrerror(code)); + SRpcMsg rspMsg = {.handle = pMsg->rpcMsg.handle, .code = TSDB_CODE_MSG_NOT_PROCESSED}; + rpcSendResponse(&rspMsg); } + + mndCleanupMsg(pMsg); } -void mnodeSetMsgHandle(SMnode *pMnode, int32_t msgType, MndMsgFp fp) { +void mndSetMsgHandle(SMnode *pMnode, int32_t msgType, MndMsgFp fp) { if (msgType >= 0 && msgType < TSDB_MSG_TYPE_MAX) { - tsMint.msgFp[msgType] = fp; + pMnode->msgFp[msgType] = fp; } } -void mnodeProcessReadMsg(SMnodeMsg *pMsg) { mnodeProcessRpcMsg(pMsg); } +void mndProcessReadMsg(SMnodeMsg *pMsg) { mndProcessRpcMsg(pMsg); } -void mnodeProcessWriteMsg(SMnodeMsg *pMsg) { mnodeProcessRpcMsg(pMsg); } +void mndProcessWriteMsg(SMnodeMsg *pMsg) { mndProcessRpcMsg(pMsg); } -void mnodeProcessSyncMsg(SMnodeMsg *pMsg) { mnodeProcessRpcMsg(pMsg); } +void mndProcessSyncMsg(SMnodeMsg *pMsg) { mndProcessRpcMsg(pMsg); } -void mnodeProcessApplyMsg(SMnodeMsg *pMsg) {} +void mndProcessApplyMsg(SMnodeMsg *pMsg) {} #if 0 @@ -256,7 +333,7 @@ static void mnodeProcessWriteReq(SMnodeMsg *pMsg, void *unused) { } if (!mnodeIsMaster()) { - SMnRsp *rpcRsp = &pMsg->rpcRsp; + SMnodeRsp *rpcRsp = &pMsg->rpcRsp; SEpSet *epSet = rpcMallocCont(sizeof(SEpSet)); mnodeGetMnodeEpSetForShell(epSet, true); rpcRsp->rsp = epSet; @@ -278,7 +355,7 @@ static void mnodeProcessWriteReq(SMnodeMsg *pMsg, void *unused) { code = (*tsMworker.writeMsgFp[msgType])(pMsg); PROCESS_WRITE_REQ_END: - mnodeSendRsp(pMsg, code); + mndSendRsp(pMsg, code); } static void mnodeProcessReadReq(SMnodeMsg *pMsg, void *unused) { @@ -293,7 +370,7 @@ static void mnodeProcessReadReq(SMnodeMsg *pMsg, void *unused) { } if (!mnodeIsMaster()) { - SMnRsp *rpcRsp = &pMsg->rpcRsp; + SMnodeRsp *rpcRsp = &pMsg->rpcRsp; SEpSet *epSet = rpcMallocCont(sizeof(SEpSet)); if (!epSet) { code = TSDB_CODE_OUT_OF_MEMORY; @@ -319,7 +396,7 @@ static void mnodeProcessReadReq(SMnodeMsg *pMsg, void *unused) { code = (*tsMworker.readMsgFp[msgType])(pMsg); PROCESS_READ_REQ_END: - mnodeSendRsp(pMsg, code); + mndSendRsp(pMsg, code); } static void mnodeProcessPeerReq(SMnodeMsg *pMsg, void *unused) { @@ -334,7 +411,7 @@ static void mnodeProcessPeerReq(SMnodeMsg *pMsg, void *unused) { } if (!mnodeIsMaster()) { - SMnRsp *rpcRsp = &pMsg->rpcRsp; + SMnodeRsp *rpcRsp = &pMsg->rpcRsp; SEpSet *epSet = rpcMallocCont(sizeof(SEpSet)); mnodeGetMnodeEpSetForPeer(epSet, true); rpcRsp->rsp = epSet; @@ -356,7 +433,7 @@ static void mnodeProcessPeerReq(SMnodeMsg *pMsg, void *unused) { code = (*tsMworker.peerReqFp[msgType])(pMsg); PROCESS_PEER_REQ_END: - mnodeSendRsp(pMsg, code); + mndSendRsp(pMsg, code); } static void mnodeProcessPeerRsp(SMnodeMsg *pMsg, void *unused) { @@ -365,7 +442,7 @@ static void mnodeProcessPeerRsp(SMnodeMsg *pMsg, void *unused) { if (!mnodeIsMaster()) { mError("msg:%p, ahandle:%p type:%s not processed for not master", pRpcMsg, pRpcMsg->ahandle, taosMsg[msgType]); - mnodeCleanupMsg2(pMsg); + mndCleanupMsg2(pMsg); } if (tsMworker.peerRspFp[msgType]) { @@ -374,6 +451,6 @@ static void mnodeProcessPeerRsp(SMnodeMsg *pMsg, void *unused) { mError("msg:%p, ahandle:%p type:%s is not processed", pRpcMsg, pRpcMsg->ahandle, taosMsg[msgType]); } - mnodeCleanupMsg2(pMsg); + mndCleanupMsg2(pMsg); } #endif \ No newline at end of file From e63637fcce2029fb51d29f8265471c5ff95565b3 Mon Sep 17 00:00:00 2001 From: Shengliang Guan Date: Mon, 29 Nov 2021 13:19:00 +0800 Subject: [PATCH 37/44] TD-10431 rename some functions --- source/dnode/mnode/impl/inc/mndMnode.h | 4 +- source/dnode/mnode/impl/inc/mndSync.h | 5 +- source/dnode/mnode/impl/src/mndMnode.c | 4 +- source/dnode/mnode/impl/src/mndSync.c | 4 +- source/dnode/mnode/impl/src/mndTelem.c | 98 +++++++------- source/dnode/mnode/impl/src/mndUser.c | 54 ++++---- source/dnode/mnode/impl/src/mnode.c | 180 +++++-------------------- 7 files changed, 111 insertions(+), 238 deletions(-) diff --git a/source/dnode/mnode/impl/inc/mndMnode.h b/source/dnode/mnode/impl/inc/mndMnode.h index 7e969baf37..260d238c65 100644 --- a/source/dnode/mnode/impl/inc/mndMnode.h +++ b/source/dnode/mnode/impl/inc/mndMnode.h @@ -24,8 +24,8 @@ extern "C" { int32_t mndInitMnode(); void mndCleanupMnode(); -void mnodeGetMnodeEpSetForPeer(SEpSet *epSet, bool redirect); -void mnodeGetMnodeEpSetForShell(SEpSet *epSet, bool redirect); +void mndGetMnodeEpSetForPeer(SEpSet *epSet, bool redirect); +void mndGetMnodeEpSetForShell(SEpSet *epSet, bool redirect); #ifdef __cplusplus } diff --git a/source/dnode/mnode/impl/inc/mndSync.h b/source/dnode/mnode/impl/inc/mndSync.h index 7d7f9d69d0..af091882a3 100644 --- a/source/dnode/mnode/impl/inc/mndSync.h +++ b/source/dnode/mnode/impl/inc/mndSync.h @@ -24,9 +24,8 @@ extern "C" { int32_t mndInitSync(); void mndCleanupSync(); -int32_t mnodeSyncPropose(SSdbRaw *pRaw, void *pData); - -bool mnodeIsMaster(); +bool mndIsMaster(); +int32_t mndSyncPropose(SSdbRaw *pRaw, void *pData); #ifdef __cplusplus } diff --git a/source/dnode/mnode/impl/src/mndMnode.c b/source/dnode/mnode/impl/src/mndMnode.c index ce38049a36..66df205f6e 100644 --- a/source/dnode/mnode/impl/src/mndMnode.c +++ b/source/dnode/mnode/impl/src/mndMnode.c @@ -20,5 +20,5 @@ int32_t mndInitMnode() { return 0; } void mndCleanupMnode() {} -void mnodeGetMnodeEpSetForPeer(SEpSet *epSet, bool redirect) {} -void mnodeGetMnodeEpSetForShell(SEpSet *epSet, bool redirect) {} \ No newline at end of file +void mndGetMnodeEpSetForPeer(SEpSet *epSet, bool redirect) {} +void mndGetMnodeEpSetForShell(SEpSet *epSet, bool redirect) {} \ No newline at end of file diff --git a/source/dnode/mnode/impl/src/mndSync.c b/source/dnode/mnode/impl/src/mndSync.c index 8db79e1e32..89098e7ec0 100644 --- a/source/dnode/mnode/impl/src/mndSync.c +++ b/source/dnode/mnode/impl/src/mndSync.c @@ -21,10 +21,10 @@ int32_t mndInitSync() { return 0; } void mndCleanupSync() {} -int32_t mnodeSyncPropose(SSdbRaw *pRaw, void *pData) { +int32_t mndSyncPropose(SSdbRaw *pRaw, void *pData) { trnApply(pData, pData, 0); free(pData); return 0; } -bool mnodeIsMaster() { return true; } \ No newline at end of file +bool mndIsMaster() { return true; } \ No newline at end of file diff --git a/source/dnode/mnode/impl/src/mndTelem.c b/source/dnode/mnode/impl/src/mndTelem.c index e9748ec12e..f5f5464822 100644 --- a/source/dnode/mnode/impl/src/mndTelem.c +++ b/source/dnode/mnode/impl/src/mndTelem.c @@ -36,9 +36,9 @@ static struct { char email[TSDB_FQDN_LEN]; } tsTelem; -static void mnodeBeginObject(SBufferWriter* bw) { tbufWriteChar(bw, '{'); } +static void mndBeginObject(SBufferWriter* bw) { tbufWriteChar(bw, '{'); } -static void mnodeCloseObject(SBufferWriter* bw) { +static void mndCloseObject(SBufferWriter* bw) { size_t len = tbufTell(bw); if (tbufGetData(bw, false)[len - 1] == ',') { tbufWriteCharAt(bw, len - 1, '}'); @@ -64,14 +64,14 @@ static void closeArray(SBufferWriter* bw) { } #endif -static void mnodeWriteString(SBufferWriter* bw, const char* str) { +static void mndWriteString(SBufferWriter* bw, const char* str) { tbufWriteChar(bw, '"'); tbufWrite(bw, str, strlen(str)); tbufWriteChar(bw, '"'); } -static void mnodeAddIntField(SBufferWriter* bw, const char* k, int64_t v) { - mnodeWriteString(bw, k); +static void mndAddIntField(SBufferWriter* bw, const char* k, int64_t v) { + mndWriteString(bw, k); tbufWriteChar(bw, ':'); char buf[32]; sprintf(buf, "%" PRId64, v); @@ -79,14 +79,14 @@ static void mnodeAddIntField(SBufferWriter* bw, const char* k, int64_t v) { tbufWriteChar(bw, ','); } -static void mnodeAddStringField(SBufferWriter* bw, const char* k, const char* v) { - mnodeWriteString(bw, k); +static void mndAddStringField(SBufferWriter* bw, const char* k, const char* v) { + mndWriteString(bw, k); tbufWriteChar(bw, ':'); - mnodeWriteString(bw, v); + mndWriteString(bw, v); tbufWriteChar(bw, ','); } -static void mnodeAddCpuInfo(SBufferWriter* bw) { +static void mndAddCpuInfo(SBufferWriter* bw) { char* line = NULL; size_t size = 0; int32_t done = 0; @@ -100,11 +100,11 @@ static void mnodeAddCpuInfo(SBufferWriter* bw) { line[size - 1] = '\0'; if (((done & 1) == 0) && strncmp(line, "model name", 10) == 0) { const char* v = strchr(line, ':') + 2; - mnodeAddStringField(bw, "cpuModel", v); + mndAddStringField(bw, "cpuModel", v); done |= 1; } else if (((done & 2) == 0) && strncmp(line, "cpu cores", 9) == 0) { const char* v = strchr(line, ':') + 2; - mnodeWriteString(bw, "numOfCpu"); + mndWriteString(bw, "numOfCpu"); tbufWriteChar(bw, ':'); tbufWrite(bw, v, strlen(v)); tbufWriteChar(bw, ','); @@ -116,7 +116,7 @@ static void mnodeAddCpuInfo(SBufferWriter* bw) { fclose(fp); } -static void mnodeAddOsInfo(SBufferWriter* bw) { +static void mndAddOsInfo(SBufferWriter* bw) { char* line = NULL; size_t size = 0; @@ -133,7 +133,7 @@ static void mnodeAddOsInfo(SBufferWriter* bw) { p++; line[size - 2] = 0; } - mnodeAddStringField(bw, "os", p); + mndAddStringField(bw, "os", p); break; } } @@ -142,7 +142,7 @@ static void mnodeAddOsInfo(SBufferWriter* bw) { fclose(fp); } -static void mnodeAddMemoryInfo(SBufferWriter* bw) { +static void mndAddMemoryInfo(SBufferWriter* bw) { char* line = NULL; size_t size = 0; @@ -156,7 +156,7 @@ static void mnodeAddMemoryInfo(SBufferWriter* bw) { if (strncmp(line, "MemTotal", 8) == 0) { const char* p = strchr(line, ':') + 1; while (*p == ' ') p++; - mnodeAddStringField(bw, "memory", p); + mndAddStringField(bw, "memory", p); break; } } @@ -165,32 +165,32 @@ static void mnodeAddMemoryInfo(SBufferWriter* bw) { fclose(fp); } -static void mnodeAddVersionInfo(SBufferWriter* bw) { - mnodeAddStringField(bw, "version", version); - mnodeAddStringField(bw, "buildInfo", buildinfo); - mnodeAddStringField(bw, "gitInfo", gitinfo); - mnodeAddStringField(bw, "email", tsTelem.email); +static void mndAddVersionInfo(SBufferWriter* bw) { + mndAddStringField(bw, "version", version); + mndAddStringField(bw, "buildInfo", buildinfo); + mndAddStringField(bw, "gitInfo", gitinfo); + mndAddStringField(bw, "email", tsTelem.email); } -static void mnodeAddRuntimeInfo(SBufferWriter* bw) { +static void mndAddRuntimeInfo(SBufferWriter* bw) { SMnodeLoad load = {0}; if (mndGetLoad(NULL, &load) != 0) { return; } - mnodeAddIntField(bw, "numOfDnode", load.numOfDnode); - mnodeAddIntField(bw, "numOfMnode", load.numOfMnode); - mnodeAddIntField(bw, "numOfVgroup", load.numOfVgroup); - mnodeAddIntField(bw, "numOfDatabase", load.numOfDatabase); - mnodeAddIntField(bw, "numOfSuperTable", load.numOfSuperTable); - mnodeAddIntField(bw, "numOfChildTable", load.numOfChildTable); - mnodeAddIntField(bw, "numOfColumn", load.numOfColumn); - mnodeAddIntField(bw, "numOfPoint", load.totalPoints); - mnodeAddIntField(bw, "totalStorage", load.totalStorage); - mnodeAddIntField(bw, "compStorage", load.compStorage); + mndAddIntField(bw, "numOfDnode", load.numOfDnode); + mndAddIntField(bw, "numOfMnode", load.numOfMnode); + mndAddIntField(bw, "numOfVgroup", load.numOfVgroup); + mndAddIntField(bw, "numOfDatabase", load.numOfDatabase); + mndAddIntField(bw, "numOfSuperTable", load.numOfSuperTable); + mndAddIntField(bw, "numOfChildTable", load.numOfChildTable); + mndAddIntField(bw, "numOfColumn", load.numOfColumn); + mndAddIntField(bw, "numOfPoint", load.totalPoints); + mndAddIntField(bw, "totalStorage", load.totalStorage); + mndAddIntField(bw, "compStorage", load.compStorage); } -static void mnodeSendTelemetryReport() { +static void mndSendTelemetryReport() { char buf[128] = {0}; uint32_t ip = taosGetIpv4FromFqdn(TELEMETRY_SERVER); if (ip == 0xffffffff) { @@ -208,15 +208,15 @@ static void mnodeSendTelemetryReport() { snprintf(clusterIdStr, sizeof(clusterIdStr), "%" PRId64, clusterId); SBufferWriter bw = tbufInitWriter(NULL, false); - mnodeBeginObject(&bw); - mnodeAddStringField(&bw, "instanceId", clusterIdStr); - mnodeAddIntField(&bw, "reportVersion", 1); - mnodeAddOsInfo(&bw); - mnodeAddCpuInfo(&bw); - mnodeAddMemoryInfo(&bw); - mnodeAddVersionInfo(&bw); - mnodeAddRuntimeInfo(&bw); - mnodeCloseObject(&bw); + mndBeginObject(&bw); + mndAddStringField(&bw, "instanceId", clusterIdStr); + mndAddIntField(&bw, "reportVersion", 1); + mndAddOsInfo(&bw); + mndAddCpuInfo(&bw); + mndAddMemoryInfo(&bw); + mndAddVersionInfo(&bw); + mndAddRuntimeInfo(&bw); + mndCloseObject(&bw); const char* header = "POST /report HTTP/1.1\n" @@ -240,12 +240,12 @@ static void mnodeSendTelemetryReport() { taosCloseSocket(fd); } -static void* mnodeTelemThreadFp(void* param) { +static void* mndTelemThreadFp(void* param) { struct timespec end = {0}; clock_gettime(CLOCK_REALTIME, &end); end.tv_sec += 300; // wait 5 minutes before send first report - setThreadName("mnode-telem"); + setThreadName("mnd-telem"); while (!tsTelem.exit) { int32_t r = 0; @@ -256,8 +256,8 @@ static void* mnodeTelemThreadFp(void* param) { if (r == 0) break; if (r != ETIMEDOUT) continue; - if (mnodeIsMaster()) { - mnodeSendTelemetryReport(); + if (mndIsMaster()) { + mndSendTelemetryReport(); } end.tv_sec += REPORT_INTERVAL; } @@ -265,7 +265,7 @@ static void* mnodeTelemThreadFp(void* param) { return NULL; } -static void mnodeGetEmail(char* filepath) { +static void mndGetEmail(char* filepath) { int32_t fd = taosOpenFileRead(filepath); if (fd < 0) { return; @@ -287,19 +287,19 @@ int32_t mndInitTelem() { pthread_cond_init(&tsTelem.cond, NULL); tsTelem.email[0] = 0; - mnodeGetEmail("/usr/local/taos/email"); + mndGetEmail("/usr/local/taos/email"); pthread_attr_t attr; pthread_attr_init(&attr); pthread_attr_setdetachstate(&attr, PTHREAD_CREATE_JOINABLE); - int32_t code = pthread_create(&tsTelem.thread, &attr, mnodeTelemThreadFp, NULL); + int32_t code = pthread_create(&tsTelem.thread, &attr, mndTelemThreadFp, NULL); pthread_attr_destroy(&attr); if (code != 0) { mTrace("failed to create telemetry thread since :%s", strerror(code)); } - mInfo("mnode telemetry is initialized"); + mInfo("mnd telemetry is initialized"); return 0; } diff --git a/source/dnode/mnode/impl/src/mndUser.c b/source/dnode/mnode/impl/src/mndUser.c index a51133d427..32c2c459e5 100644 --- a/source/dnode/mnode/impl/src/mndUser.c +++ b/source/dnode/mnode/impl/src/mndUser.c @@ -15,14 +15,12 @@ #define _DEFAULT_SOURCE #include "mndSync.h" -#include "os.h" -#include "tglobal.h" #include "tkey.h" #include "mndTrans.h" #define SDB_USER_VER 1 -static SSdbRaw *mnodeUserActionEncode(SUserObj *pUser) { +static SSdbRaw *mndUserActionEncode(SUserObj *pUser) { SSdbRaw *pRaw = sdbAllocRaw(SDB_USER, SDB_USER_VER, sizeof(SAcctObj)); if (pRaw == NULL) return NULL; @@ -38,7 +36,7 @@ static SSdbRaw *mnodeUserActionEncode(SUserObj *pUser) { return pRaw; } -static SSdbRow *mnodeUserActionDecode(SSdbRaw *pRaw) { +static SSdbRow *mndUserActionDecode(SSdbRaw *pRaw) { int8_t sver = 0; if (sdbGetRawSoftVer(pRaw, &sver) != 0) return NULL; @@ -62,7 +60,7 @@ static SSdbRow *mnodeUserActionDecode(SSdbRaw *pRaw) { return pRow; } -static int32_t mnodeUserActionInsert(SUserObj *pUser) { +static int32_t mndUserActionInsert(SUserObj *pUser) { pUser->prohibitDbHash = taosHashInit(8, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY), true, HASH_ENTRY_LOCK); if (pUser->prohibitDbHash == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; @@ -78,7 +76,7 @@ static int32_t mnodeUserActionInsert(SUserObj *pUser) { return 0; } -static int32_t mnodeUserActionDelete(SUserObj *pUser) { +static int32_t mndUserActionDelete(SUserObj *pUser) { if (pUser->prohibitDbHash) { taosHashCleanup(pUser->prohibitDbHash); pUser->prohibitDbHash = NULL; @@ -92,14 +90,14 @@ static int32_t mnodeUserActionDelete(SUserObj *pUser) { return 0; } -static int32_t mnodeUserActionUpdate(SUserObj *pSrcUser, SUserObj *pDstUser) { +static int32_t mndUserActionUpdate(SUserObj *pSrcUser, SUserObj *pDstUser) { SUserObj tObj; int32_t len = (int32_t)((int8_t *)tObj.prohibitDbHash - (int8_t *)&tObj); memcpy(pDstUser, pSrcUser, len); return 0; } -static int32_t mnodeCreateDefaultUser(char *acct, char *user, char *pass) { +static int32_t mndCreateDefaultUser(char *acct, char *user, char *pass) { SUserObj userObj = {0}; tstrncpy(userObj.user, user, TSDB_USER_LEN); tstrncpy(userObj.acct, acct, TSDB_USER_LEN); @@ -111,30 +109,26 @@ static int32_t mnodeCreateDefaultUser(char *acct, char *user, char *pass) { userObj.rootAuth = 1; } - SSdbRaw *pRaw = mnodeUserActionEncode(&userObj); + SSdbRaw *pRaw = mndUserActionEncode(&userObj); if (pRaw == NULL) return -1; sdbSetRawStatus(pRaw, SDB_STATUS_READY); return sdbWrite(pRaw); } -static int32_t mnodeCreateDefaultUsers() { - if (mnodeCreateDefaultUser(TSDB_DEFAULT_USER, TSDB_DEFAULT_USER, TSDB_DEFAULT_PASS) != 0) { +static int32_t mndCreateDefaultUsers() { + if (mndCreateDefaultUser(TSDB_DEFAULT_USER, TSDB_DEFAULT_USER, TSDB_DEFAULT_PASS) != 0) { return -1; } - if (mnodeCreateDefaultUser(TSDB_DEFAULT_USER, "monitor", tsInternalPass) != 0) { - return -1; - } - - if (mnodeCreateDefaultUser(TSDB_DEFAULT_USER, "_" TSDB_DEFAULT_USER, tsInternalPass) != 0) { + if (mndCreateDefaultUser(TSDB_DEFAULT_USER, "_" TSDB_DEFAULT_USER, TSDB_DEFAULT_PASS) != 0) { return -1; } return 0; } -static int32_t mnodeCreateUser(char *acct, char *user, char *pass, SMnodeMsg *pMsg) { +static int32_t mndCreateUser(char *acct, char *user, char *pass, SMnodeMsg *pMsg) { SUserObj userObj = {0}; tstrncpy(userObj.user, user, TSDB_USER_LEN); tstrncpy(userObj.acct, acct, TSDB_USER_LEN); @@ -146,7 +140,7 @@ static int32_t mnodeCreateUser(char *acct, char *user, char *pass, SMnodeMsg *pM STrans *pTrans = trnCreate(TRN_POLICY_ROLLBACK, pMsg->rpcMsg.handle); if (pTrans == NULL) return -1; - SSdbRaw *pRedoRaw = mnodeUserActionEncode(&userObj); + SSdbRaw *pRedoRaw = mndUserActionEncode(&userObj); if (pRedoRaw == NULL || trnAppendRedoLog(pTrans, pRedoRaw) != 0) { mError("failed to append redo log since %s", terrstr()); trnDrop(pTrans); @@ -154,7 +148,7 @@ static int32_t mnodeCreateUser(char *acct, char *user, char *pass, SMnodeMsg *pM } sdbSetRawStatus(pRedoRaw, SDB_STATUS_CREATING); - SSdbRaw *pUndoRaw = mnodeUserActionEncode(&userObj); + SSdbRaw *pUndoRaw = mndUserActionEncode(&userObj); if (pUndoRaw == NULL || trnAppendUndoLog(pTrans, pUndoRaw) != 0) { mError("failed to append undo log since %s", terrstr()); trnDrop(pTrans); @@ -162,7 +156,7 @@ static int32_t mnodeCreateUser(char *acct, char *user, char *pass, SMnodeMsg *pM } sdbSetRawStatus(pUndoRaw, SDB_STATUS_DROPPED); - SSdbRaw *pCommitRaw = mnodeUserActionEncode(&userObj); + SSdbRaw *pCommitRaw = mndUserActionEncode(&userObj); if (pCommitRaw == NULL || trnAppendCommitLog(pTrans, pCommitRaw) != 0) { mError("failed to append commit log since %s", terrstr()); trnDrop(pTrans); @@ -170,7 +164,7 @@ static int32_t mnodeCreateUser(char *acct, char *user, char *pass, SMnodeMsg *pM } sdbSetRawStatus(pCommitRaw, SDB_STATUS_READY); - if (trnPrepare(pTrans, mnodeSyncPropose) != 0) { + if (trnPrepare(pTrans, mndSyncPropose) != 0) { trnDrop(pTrans); return -1; } @@ -179,7 +173,7 @@ static int32_t mnodeCreateUser(char *acct, char *user, char *pass, SMnodeMsg *pM return 0; } -static int32_t mnodeProcessCreateUserMsg(SMnode *pMnode, SMnodeMsg *pMsg) { +static int32_t mndProcessCreateUserMsg(SMnode *pMnode, SMnodeMsg *pMsg) { SCreateUserMsg *pCreate = pMsg->rpcMsg.pCont; if (pCreate->user[0] == 0) { @@ -209,7 +203,7 @@ static int32_t mnodeProcessCreateUserMsg(SMnode *pMnode, SMnodeMsg *pMsg) { return -1; } - int32_t code = mnodeCreateUser(pOperUser->acct, pCreate->user, pCreate->pass, pMsg); + int32_t code = mndCreateUser(pOperUser->acct, pCreate->user, pCreate->pass, pMsg); sdbRelease(pOperUser); if (code != 0) { @@ -223,15 +217,15 @@ static int32_t mnodeProcessCreateUserMsg(SMnode *pMnode, SMnodeMsg *pMsg) { int32_t mndInitUser() { SSdbTable table = {.sdbType = SDB_USER, .keyType = SDB_KEY_BINARY, - .deployFp = (SdbDeployFp)mnodeCreateDefaultUsers, - .encodeFp = (SdbEncodeFp)mnodeUserActionEncode, - .decodeFp = (SdbDecodeFp)mnodeUserActionDecode, - .insertFp = (SdbInsertFp)mnodeUserActionInsert, - .updateFp = (SdbUpdateFp)mnodeUserActionUpdate, - .deleteFp = (SdbDeleteFp)mnodeUserActionDelete}; + .deployFp = (SdbDeployFp)mndCreateDefaultUsers, + .encodeFp = (SdbEncodeFp)mndUserActionEncode, + .decodeFp = (SdbDecodeFp)mndUserActionDecode, + .insertFp = (SdbInsertFp)mndUserActionInsert, + .updateFp = (SdbUpdateFp)mndUserActionUpdate, + .deleteFp = (SdbDeleteFp)mndUserActionDelete}; sdbSetTable(table); - mndSetMsgHandle(NULL, TSDB_MSG_TYPE_CREATE_USER, mnodeProcessCreateUserMsg); + mndSetMsgHandle(NULL, TSDB_MSG_TYPE_CREATE_USER, mndProcessCreateUserMsg); return 0; } diff --git a/source/dnode/mnode/impl/src/mnode.c b/source/dnode/mnode/impl/src/mnode.c index 426c86f728..369994c2bb 100644 --- a/source/dnode/mnode/impl/src/mnode.c +++ b/source/dnode/mnode/impl/src/mnode.c @@ -278,28 +278,44 @@ void mndSendRsp(SMnodeMsg *pMsg, int32_t code) {} static void mndProcessRpcMsg(SMnodeMsg *pMsg) { SMnode *pMnode = pMsg->pMnode; + int32_t code = 0; + int32_t msgType = pMsg->rpcMsg.msgType; + void *ahandle = pMsg->rpcMsg.ahandle; + bool isReq = (msgType % 2 == 1); - if (!mnodeIsMaster(pMnode)) { - mndSendRedirectMsg(pMnode, &pMsg->rpcMsg); - mndCleanupMsg(pMsg); - return; + if (isReq && !mndIsMaster(pMnode)) { + code = TSDB_CODE_APP_NOT_READY; + goto PROCESS_RPC_END; + } + + if (isReq && pMsg->rpcMsg.pCont == NULL) { + mError("msg:%p, app:%p type:%s content is null", pMsg, ahandle, taosMsg[msgType]); + code = TSDB_CODE_MND_INVALID_MSG_LEN; + goto PROCESS_RPC_END; } - int32_t msgType = pMsg->rpcMsg.msgType; MndMsgFp fp = pMnode->msgFp[msgType]; if (fp == NULL) { - mError("RPC %p, req:%s is not processed", pMsg->rpcMsg.handle, taosMsg[msgType]); - SRpcMsg rspMsg = {.handle = pMsg->rpcMsg.handle, .code = TSDB_CODE_MSG_NOT_PROCESSED}; - rpcSendResponse(&rspMsg); - mndCleanupMsg(pMsg); - return; + mError("msg:%p, app:%p type:%s not processed", pMsg, ahandle, taosMsg[msgType]); + code = TSDB_CODE_MSG_NOT_PROCESSED; + goto PROCESS_RPC_END; } - int32_t code = (*fp)(pMnode, pMsg); + code = (*fp)(pMnode, pMsg); if (code != 0) { - mError("RPC %p, req:%s processed error since %s", pMsg->rpcMsg.handle, taosMsg[msgType], tstrerror(code)); - SRpcMsg rspMsg = {.handle = pMsg->rpcMsg.handle, .code = TSDB_CODE_MSG_NOT_PROCESSED}; - rpcSendResponse(&rspMsg); + mError("msg:%p, app:%p type:%s failed to process since %s", pMsg, ahandle, taosMsg[msgType], tstrerror(code)); + goto PROCESS_RPC_END; + } + +PROCESS_RPC_END: + if (isReq) { + if (code == TSDB_CODE_APP_NOT_READY) { + mndSendRedirectMsg(pMnode, &pMsg->rpcMsg); + } else if (code != 0) { + SRpcMsg rspMsg = {.handle = pMsg->rpcMsg.handle, .code = code}; + rpcSendResponse(&rspMsg); + } else { + } } mndCleanupMsg(pMsg); @@ -318,139 +334,3 @@ void mndProcessWriteMsg(SMnodeMsg *pMsg) { mndProcessRpcMsg(pMsg); } void mndProcessSyncMsg(SMnodeMsg *pMsg) { mndProcessRpcMsg(pMsg); } void mndProcessApplyMsg(SMnodeMsg *pMsg) {} - -#if 0 - -static void mnodeProcessWriteReq(SMnodeMsg *pMsg, void *unused) { - int32_t msgType = pMsg->rpcMsg.msgType; - void *ahandle = pMsg->rpcMsg.ahandle; - int32_t code = 0; - - if (pMsg->rpcMsg.pCont == NULL) { - mError("msg:%p, app:%p type:%s content is null", pMsg, ahandle, taosMsg[msgType]); - code = TSDB_CODE_MND_INVALID_MSG_LEN; - goto PROCESS_WRITE_REQ_END; - } - - if (!mnodeIsMaster()) { - SMnodeRsp *rpcRsp = &pMsg->rpcRsp; - SEpSet *epSet = rpcMallocCont(sizeof(SEpSet)); - mnodeGetMnodeEpSetForShell(epSet, true); - rpcRsp->rsp = epSet; - rpcRsp->len = sizeof(SEpSet); - - mDebug("msg:%p, app:%p type:%s in write queue, is redirected, numOfEps:%d inUse:%d", pMsg, ahandle, - taosMsg[msgType], epSet->numOfEps, epSet->inUse); - - code = TSDB_CODE_RPC_REDIRECT; - goto PROCESS_WRITE_REQ_END; - } - - if (tsMworker.writeMsgFp[msgType] == NULL) { - mError("msg:%p, app:%p type:%s not processed", pMsg, ahandle, taosMsg[msgType]); - code = TSDB_CODE_MND_MSG_NOT_PROCESSED; - goto PROCESS_WRITE_REQ_END; - } - - code = (*tsMworker.writeMsgFp[msgType])(pMsg); - -PROCESS_WRITE_REQ_END: - mndSendRsp(pMsg, code); -} - -static void mnodeProcessReadReq(SMnodeMsg *pMsg, void *unused) { - int32_t msgType = pMsg->rpcMsg.msgType; - void *ahandle = pMsg->rpcMsg.ahandle; - int32_t code = 0; - - if (pMsg->rpcMsg.pCont == NULL) { - mError("msg:%p, app:%p type:%s in mread queue, content is null", pMsg, ahandle, taosMsg[msgType]); - code = TSDB_CODE_MND_INVALID_MSG_LEN; - goto PROCESS_READ_REQ_END; - } - - if (!mnodeIsMaster()) { - SMnodeRsp *rpcRsp = &pMsg->rpcRsp; - SEpSet *epSet = rpcMallocCont(sizeof(SEpSet)); - if (!epSet) { - code = TSDB_CODE_OUT_OF_MEMORY; - goto PROCESS_READ_REQ_END; - } - mnodeGetMnodeEpSetForShell(epSet, true); - rpcRsp->rsp = epSet; - rpcRsp->len = sizeof(SEpSet); - - mDebug("msg:%p, app:%p type:%s in mread queue is redirected, numOfEps:%d inUse:%d", pMsg, ahandle, taosMsg[msgType], - epSet->numOfEps, epSet->inUse); - code = TSDB_CODE_RPC_REDIRECT; - goto PROCESS_READ_REQ_END; - } - - if (tsMworker.readMsgFp[msgType] == NULL) { - mError("msg:%p, app:%p type:%s in mread queue, not processed", pMsg, ahandle, taosMsg[msgType]); - code = TSDB_CODE_MND_MSG_NOT_PROCESSED; - goto PROCESS_READ_REQ_END; - } - - mTrace("msg:%p, app:%p type:%s will be processed in mread queue", pMsg, ahandle, taosMsg[msgType]); - code = (*tsMworker.readMsgFp[msgType])(pMsg); - -PROCESS_READ_REQ_END: - mndSendRsp(pMsg, code); -} - -static void mnodeProcessPeerReq(SMnodeMsg *pMsg, void *unused) { - int32_t msgType = pMsg->rpcMsg.msgType; - void *ahandle = pMsg->rpcMsg.ahandle; - int32_t code = 0; - - if (pMsg->rpcMsg.pCont == NULL) { - mError("msg:%p, ahandle:%p type:%s in mpeer queue, content is null", pMsg, ahandle, taosMsg[msgType]); - code = TSDB_CODE_MND_INVALID_MSG_LEN; - goto PROCESS_PEER_REQ_END; - } - - if (!mnodeIsMaster()) { - SMnodeRsp *rpcRsp = &pMsg->rpcRsp; - SEpSet *epSet = rpcMallocCont(sizeof(SEpSet)); - mnodeGetMnodeEpSetForPeer(epSet, true); - rpcRsp->rsp = epSet; - rpcRsp->len = sizeof(SEpSet); - - mDebug("msg:%p, ahandle:%p type:%s in mpeer queue is redirected, numOfEps:%d inUse:%d", pMsg, ahandle, - taosMsg[msgType], epSet->numOfEps, epSet->inUse); - - code = TSDB_CODE_RPC_REDIRECT; - goto PROCESS_PEER_REQ_END; - } - - if (tsMworker.peerReqFp[msgType] == NULL) { - mError("msg:%p, ahandle:%p type:%s in mpeer queue, not processed", pMsg, ahandle, taosMsg[msgType]); - code = TSDB_CODE_MND_MSG_NOT_PROCESSED; - goto PROCESS_PEER_REQ_END; - } - - code = (*tsMworker.peerReqFp[msgType])(pMsg); - -PROCESS_PEER_REQ_END: - mndSendRsp(pMsg, code); -} - -static void mnodeProcessPeerRsp(SMnodeMsg *pMsg, void *unused) { - int32_t msgType = pMsg->rpcMsg.msgType; - SRpcMsg *pRpcMsg = &pMsg->rpcMsg; - - if (!mnodeIsMaster()) { - mError("msg:%p, ahandle:%p type:%s not processed for not master", pRpcMsg, pRpcMsg->ahandle, taosMsg[msgType]); - mndCleanupMsg2(pMsg); - } - - if (tsMworker.peerRspFp[msgType]) { - (*tsMworker.peerRspFp[msgType])(pRpcMsg); - } else { - mError("msg:%p, ahandle:%p type:%s is not processed", pRpcMsg, pRpcMsg->ahandle, taosMsg[msgType]); - } - - mndCleanupMsg2(pMsg); -} -#endif \ No newline at end of file From 35b0c693331bf8ce9ec9c8572c8ba9522205fda4 Mon Sep 17 00:00:00 2001 From: Shengliang Guan Date: Mon, 29 Nov 2021 13:29:13 +0800 Subject: [PATCH 38/44] TD-10431 rename some funcs --- source/dnode/mnode/impl/inc/mndAcct.h | 4 ++-- source/dnode/mnode/impl/inc/mndAuth.h | 4 ++-- source/dnode/mnode/impl/inc/mndBalance.h | 4 ++-- source/dnode/mnode/impl/inc/mndCluster.h | 4 ++-- source/dnode/mnode/impl/inc/mndDb.h | 4 ++-- source/dnode/mnode/impl/inc/mndDnode.h | 4 ++-- source/dnode/mnode/impl/inc/mndFunc.h | 4 ++-- source/dnode/mnode/impl/inc/mndMnode.h | 4 ++-- source/dnode/mnode/impl/inc/mndOper.h | 4 ++-- source/dnode/mnode/impl/inc/mndProfile.h | 4 ++-- source/dnode/mnode/impl/inc/mndShow.h | 4 ++-- source/dnode/mnode/impl/inc/mndStable.h | 4 ++-- source/dnode/mnode/impl/inc/mndSync.h | 6 +++--- source/dnode/mnode/impl/inc/mndTelem.h | 4 ++-- source/dnode/mnode/impl/inc/mndTrans.h | 4 ++-- source/dnode/mnode/impl/inc/mndUser.h | 4 ++-- source/dnode/mnode/impl/inc/mndVgroup.h | 4 ++-- source/dnode/mnode/impl/src/mndAcct.c | 4 ++-- source/dnode/mnode/impl/src/mndAuth.c | 4 ++-- source/dnode/mnode/impl/src/mndBalance.c | 4 ++-- source/dnode/mnode/impl/src/mndCluster.c | 4 ++-- source/dnode/mnode/impl/src/mndDb.c | 4 ++-- source/dnode/mnode/impl/src/mndDnode.c | 4 ++-- source/dnode/mnode/impl/src/mndFunc.c | 4 ++-- source/dnode/mnode/impl/src/mndMnode.c | 4 ++-- source/dnode/mnode/impl/src/mndOper.c | 4 ++-- source/dnode/mnode/impl/src/mndProfile.c | 4 ++-- source/dnode/mnode/impl/src/mndShow.c | 4 ++-- source/dnode/mnode/impl/src/mndStable.c | 4 ++-- source/dnode/mnode/impl/src/mndSync.c | 4 ++-- source/dnode/mnode/impl/src/mndTelem.c | 6 +++--- source/dnode/mnode/impl/src/mndTrans.c | 4 ++-- source/dnode/mnode/impl/src/mndUser.c | 6 +++--- source/dnode/mnode/impl/src/mndVgroup.c | 4 ++-- 34 files changed, 71 insertions(+), 71 deletions(-) diff --git a/source/dnode/mnode/impl/inc/mndAcct.h b/source/dnode/mnode/impl/inc/mndAcct.h index 3289c24110..2073944cc1 100644 --- a/source/dnode/mnode/impl/inc/mndAcct.h +++ b/source/dnode/mnode/impl/inc/mndAcct.h @@ -22,8 +22,8 @@ extern "C" { #endif -int32_t mndInitAcct(); -void mndCleanupAcct(); +int32_t mndInitAcct(SMnode *pMnode); +void mndCleanupAcct(SMnode *pMnode); #ifdef __cplusplus } diff --git a/source/dnode/mnode/impl/inc/mndAuth.h b/source/dnode/mnode/impl/inc/mndAuth.h index 3f0deeec6c..b426ce23cf 100644 --- a/source/dnode/mnode/impl/inc/mndAuth.h +++ b/source/dnode/mnode/impl/inc/mndAuth.h @@ -22,8 +22,8 @@ extern "C" { #endif -int32_t mndInitAuth(); -void mndCleanupAuth(); +int32_t mndInitAuth(SMnode *pMnode); +void mndCleanupAuth(SMnode *pMnode); #ifdef __cplusplus } diff --git a/source/dnode/mnode/impl/inc/mndBalance.h b/source/dnode/mnode/impl/inc/mndBalance.h index 66d85eb2c3..fa3679d033 100644 --- a/source/dnode/mnode/impl/inc/mndBalance.h +++ b/source/dnode/mnode/impl/inc/mndBalance.h @@ -22,8 +22,8 @@ extern "C" { #endif -int32_t mndInitBalance(); -void mndCleanupBalance(); +int32_t mndInitBalance(SMnode *pMnode); +void mndCleanupBalance(SMnode *pMnode); #ifdef __cplusplus } diff --git a/source/dnode/mnode/impl/inc/mndCluster.h b/source/dnode/mnode/impl/inc/mndCluster.h index d8dba66a63..efc1ada67d 100644 --- a/source/dnode/mnode/impl/inc/mndCluster.h +++ b/source/dnode/mnode/impl/inc/mndCluster.h @@ -22,8 +22,8 @@ extern "C" { #endif -int32_t mndInitCluster(); -void mndCleanupCluster(); +int32_t mndInitCluster(SMnode *pMnode); +void mndCleanupCluster(SMnode *pMnode); #ifdef __cplusplus } diff --git a/source/dnode/mnode/impl/inc/mndDb.h b/source/dnode/mnode/impl/inc/mndDb.h index c9a888a115..acccb62603 100644 --- a/source/dnode/mnode/impl/inc/mndDb.h +++ b/source/dnode/mnode/impl/inc/mndDb.h @@ -22,8 +22,8 @@ extern "C" { #endif -int32_t mndInitDb(); -void mndCleanupDb(); +int32_t mndInitDb(SMnode *pMnode); +void mndCleanupDb(SMnode *pMnode); #ifdef __cplusplus } diff --git a/source/dnode/mnode/impl/inc/mndDnode.h b/source/dnode/mnode/impl/inc/mndDnode.h index 0c484b0724..d7bfdba122 100644 --- a/source/dnode/mnode/impl/inc/mndDnode.h +++ b/source/dnode/mnode/impl/inc/mndDnode.h @@ -22,8 +22,8 @@ extern "C" { #endif -int32_t mndInitDnode(); -void mndCleanupDnode(); +int32_t mndInitDnode(SMnode *pMnode); +void mndCleanupDnode(SMnode *pMnode); #ifdef __cplusplus } diff --git a/source/dnode/mnode/impl/inc/mndFunc.h b/source/dnode/mnode/impl/inc/mndFunc.h index 3e17aae389..a531ef31a4 100644 --- a/source/dnode/mnode/impl/inc/mndFunc.h +++ b/source/dnode/mnode/impl/inc/mndFunc.h @@ -22,8 +22,8 @@ extern "C" { #endif -int32_t mndInitFunc(); -void mndCleanupFunc(); +int32_t mndInitFunc(SMnode *pMnode); +void mndCleanupFunc(SMnode *pMnode); #ifdef __cplusplus } diff --git a/source/dnode/mnode/impl/inc/mndMnode.h b/source/dnode/mnode/impl/inc/mndMnode.h index 260d238c65..c57e1d42a5 100644 --- a/source/dnode/mnode/impl/inc/mndMnode.h +++ b/source/dnode/mnode/impl/inc/mndMnode.h @@ -22,8 +22,8 @@ extern "C" { #endif -int32_t mndInitMnode(); -void mndCleanupMnode(); +int32_t mndInitMnode(SMnode *pMnode); +void mndCleanupMnode(SMnode *pMnode); void mndGetMnodeEpSetForPeer(SEpSet *epSet, bool redirect); void mndGetMnodeEpSetForShell(SEpSet *epSet, bool redirect); diff --git a/source/dnode/mnode/impl/inc/mndOper.h b/source/dnode/mnode/impl/inc/mndOper.h index d07567635d..5ad5059a0f 100644 --- a/source/dnode/mnode/impl/inc/mndOper.h +++ b/source/dnode/mnode/impl/inc/mndOper.h @@ -20,8 +20,8 @@ extern "C" { #endif -int32_t mndInitOper(); -void mndCleanupOper(); +int32_t mndInitOper(SMnode *pMnode); +void mndCleanupOper(SMnode *pMnode); #ifdef __cplusplus } diff --git a/source/dnode/mnode/impl/inc/mndProfile.h b/source/dnode/mnode/impl/inc/mndProfile.h index 4433478caa..df857be73e 100644 --- a/source/dnode/mnode/impl/inc/mndProfile.h +++ b/source/dnode/mnode/impl/inc/mndProfile.h @@ -22,8 +22,8 @@ extern "C" { #endif -int32_t mndInitProfile(); -void mndCleanupProfile(); +int32_t mndInitProfile(SMnode *pMnode); +void mndCleanupProfile(SMnode *pMnode); #ifdef __cplusplus } diff --git a/source/dnode/mnode/impl/inc/mndShow.h b/source/dnode/mnode/impl/inc/mndShow.h index 2161c81b74..06c18cb029 100644 --- a/source/dnode/mnode/impl/inc/mndShow.h +++ b/source/dnode/mnode/impl/inc/mndShow.h @@ -22,8 +22,8 @@ extern "C" { #endif -int32_t mndInitShow(); -void mndCleanupShow(); +int32_t mndInitShow(SMnode *pMnode); +void mndCleanupShow(SMnode *pMnode); #ifdef __cplusplus } diff --git a/source/dnode/mnode/impl/inc/mndStable.h b/source/dnode/mnode/impl/inc/mndStable.h index 429f4d947a..c7767a59e4 100644 --- a/source/dnode/mnode/impl/inc/mndStable.h +++ b/source/dnode/mnode/impl/inc/mndStable.h @@ -22,8 +22,8 @@ extern "C" { #endif -int32_t mndInitStable(); -void mndCleanupStable(); +int32_t mndInitStable(SMnode *pMnode); +void mndCleanupStable(SMnode *pMnode); #ifdef __cplusplus } diff --git a/source/dnode/mnode/impl/inc/mndSync.h b/source/dnode/mnode/impl/inc/mndSync.h index af091882a3..3e45b0adb1 100644 --- a/source/dnode/mnode/impl/inc/mndSync.h +++ b/source/dnode/mnode/impl/inc/mndSync.h @@ -22,9 +22,9 @@ extern "C" { #endif -int32_t mndInitSync(); -void mndCleanupSync(); -bool mndIsMaster(); +int32_t mndInitSync(SMnode *pMnode); +void mndCleanupSync(SMnode *pMnode); +bool mndIsMaster(SMnode *pMnode); int32_t mndSyncPropose(SSdbRaw *pRaw, void *pData); #ifdef __cplusplus diff --git a/source/dnode/mnode/impl/inc/mndTelem.h b/source/dnode/mnode/impl/inc/mndTelem.h index ccee629aad..4cbf0cdc77 100644 --- a/source/dnode/mnode/impl/inc/mndTelem.h +++ b/source/dnode/mnode/impl/inc/mndTelem.h @@ -21,8 +21,8 @@ extern "C" { #endif #include "mndInt.h" -int32_t mndInitTelem(); -void mndCleanupTelem(); +int32_t mndInitTelem(SMnode *pMnode); +void mndCleanupTelem(SMnode *pMnode); #ifdef __cplusplus } diff --git a/source/dnode/mnode/impl/inc/mndTrans.h b/source/dnode/mnode/impl/inc/mndTrans.h index 42a5c391b4..4443dd2e1c 100644 --- a/source/dnode/mnode/impl/inc/mndTrans.h +++ b/source/dnode/mnode/impl/inc/mndTrans.h @@ -22,8 +22,8 @@ extern "C" { #endif -int32_t mndInitTrans(); -void mndCleanupTrans(); +int32_t mndInitTrans(SMnode *pMnode); +void mndCleanupTrans(SMnode *pMnode); STrans *trnCreate(ETrnPolicy policy, void *rpcHandle); void trnDrop(STrans *pTrans); diff --git a/source/dnode/mnode/impl/inc/mndUser.h b/source/dnode/mnode/impl/inc/mndUser.h index 19554ccd37..ce570773bd 100644 --- a/source/dnode/mnode/impl/inc/mndUser.h +++ b/source/dnode/mnode/impl/inc/mndUser.h @@ -22,8 +22,8 @@ extern "C" { #endif -int32_t mndInitUser(); -void mndCleanupUser(); +int32_t mndInitUser(SMnode *pMnode); +void mndCleanupUser(SMnode *pMnode); #ifdef __cplusplus } diff --git a/source/dnode/mnode/impl/inc/mndVgroup.h b/source/dnode/mnode/impl/inc/mndVgroup.h index bf2cb60092..c75bdb5949 100644 --- a/source/dnode/mnode/impl/inc/mndVgroup.h +++ b/source/dnode/mnode/impl/inc/mndVgroup.h @@ -22,8 +22,8 @@ extern "C" { #endif -int32_t mndInitVgroup(); -void mndCleanupVgroup(); +int32_t mndInitVgroup(SMnode *pMnode); +void mndCleanupVgroup(SMnode *pMnode); #ifdef __cplusplus } diff --git a/source/dnode/mnode/impl/src/mndAcct.c b/source/dnode/mnode/impl/src/mndAcct.c index c8087dbc2d..0ee290943e 100644 --- a/source/dnode/mnode/impl/src/mndAcct.c +++ b/source/dnode/mnode/impl/src/mndAcct.c @@ -101,7 +101,7 @@ static int32_t mnodeCreateDefaultAcct() { return sdbWrite(pRaw); } -int32_t mndInitAcct() { +int32_t mndInitAcct(SMnode *pMnode) { SSdbTable table = {.sdbType = SDB_ACCT, .keyType = SDB_KEY_BINARY, .deployFp = (SdbDeployFp)mnodeCreateDefaultAcct, @@ -115,4 +115,4 @@ int32_t mndInitAcct() { return 0; } -void mndCleanupAcct() {} +void mndCleanupAcct(SMnode *pMnode) {} diff --git a/source/dnode/mnode/impl/src/mndAuth.c b/source/dnode/mnode/impl/src/mndAuth.c index d37c7bd4da..bcefbfde21 100644 --- a/source/dnode/mnode/impl/src/mndAuth.c +++ b/source/dnode/mnode/impl/src/mndAuth.c @@ -17,8 +17,8 @@ #include "os.h" #include "mndAuth.h" -int32_t mndInitAuth() { return 0; } -void mndCleanupAuth() {} +int32_t mndInitAuth(SMnode *pMnode) { return 0; } +void mndCleanupAuth(SMnode *pMnode) {} int32_t mndRetriveAuth(SMnode *pMnode, char *user, char *spi, char *encrypt, char *secret, char *ckey) { return 0; diff --git a/source/dnode/mnode/impl/src/mndBalance.c b/source/dnode/mnode/impl/src/mndBalance.c index 7b919af3f8..6ce89b96a3 100644 --- a/source/dnode/mnode/impl/src/mndBalance.c +++ b/source/dnode/mnode/impl/src/mndBalance.c @@ -17,5 +17,5 @@ #include "os.h" #include "mndInt.h" -int32_t mndInitBalance() { return 0; } -void mndCleanupBalance() {} \ No newline at end of file +int32_t mndInitBalance(SMnode *pMnode) { return 0; } +void mndCleanupBalance(SMnode *pMnode) {} \ No newline at end of file diff --git a/source/dnode/mnode/impl/src/mndCluster.c b/source/dnode/mnode/impl/src/mndCluster.c index f926959b9f..98e9e70229 100644 --- a/source/dnode/mnode/impl/src/mndCluster.c +++ b/source/dnode/mnode/impl/src/mndCluster.c @@ -17,5 +17,5 @@ #include "os.h" #include "mndInt.h" -int32_t mndInitCluster() { return 0; } -void mndCleanupCluster() {} \ No newline at end of file +int32_t mndInitCluster(SMnode *pMnode) { return 0; } +void mndCleanupCluster(SMnode *pMnode) {} \ No newline at end of file diff --git a/source/dnode/mnode/impl/src/mndDb.c b/source/dnode/mnode/impl/src/mndDb.c index 4c0e54f436..3a5a221677 100644 --- a/source/dnode/mnode/impl/src/mndDb.c +++ b/source/dnode/mnode/impl/src/mndDb.c @@ -17,5 +17,5 @@ #include "os.h" #include "mndInt.h" -int32_t mndInitDb() { return 0; } -void mndCleanupDb() {} \ No newline at end of file +int32_t mndInitDb(SMnode *pMnode) { return 0; } +void mndCleanupDb(SMnode *pMnode) {} \ No newline at end of file diff --git a/source/dnode/mnode/impl/src/mndDnode.c b/source/dnode/mnode/impl/src/mndDnode.c index f9329e871f..11f3dc1ee9 100644 --- a/source/dnode/mnode/impl/src/mndDnode.c +++ b/source/dnode/mnode/impl/src/mndDnode.c @@ -17,5 +17,5 @@ #include "os.h" #include "mndInt.h" -int32_t mndInitDnode() { return 0; } -void mndCleanupDnode() {} \ No newline at end of file +int32_t mndInitDnode(SMnode *pMnode) { return 0; } +void mndCleanupDnode(SMnode *pMnode) {} \ No newline at end of file diff --git a/source/dnode/mnode/impl/src/mndFunc.c b/source/dnode/mnode/impl/src/mndFunc.c index a84178ea6e..d859da029f 100644 --- a/source/dnode/mnode/impl/src/mndFunc.c +++ b/source/dnode/mnode/impl/src/mndFunc.c @@ -17,5 +17,5 @@ #include "os.h" #include "mndInt.h" -int32_t mndInitFunc() { return 0; } -void mndCleanupFunc() {} \ No newline at end of file +int32_t mndInitFunc(SMnode *pMnode) { return 0; } +void mndCleanupFunc(SMnode *pMnode) {} \ No newline at end of file diff --git a/source/dnode/mnode/impl/src/mndMnode.c b/source/dnode/mnode/impl/src/mndMnode.c index 66df205f6e..c2e05687c7 100644 --- a/source/dnode/mnode/impl/src/mndMnode.c +++ b/source/dnode/mnode/impl/src/mndMnode.c @@ -17,8 +17,8 @@ #include "os.h" #include "mndInt.h" -int32_t mndInitMnode() { return 0; } -void mndCleanupMnode() {} +int32_t mndInitMnode(SMnode *pMnode) { return 0; } +void mndCleanupMnode(SMnode *pMnode) {} void mndGetMnodeEpSetForPeer(SEpSet *epSet, bool redirect) {} void mndGetMnodeEpSetForShell(SEpSet *epSet, bool redirect) {} \ No newline at end of file diff --git a/source/dnode/mnode/impl/src/mndOper.c b/source/dnode/mnode/impl/src/mndOper.c index aa3a1d5e25..4416a1fc2d 100644 --- a/source/dnode/mnode/impl/src/mndOper.c +++ b/source/dnode/mnode/impl/src/mndOper.c @@ -17,5 +17,5 @@ #include "os.h" #include "mndInt.h" -int32_t mndInitOper() { return 0; } -void mndCleanupOper() {} \ No newline at end of file +int32_t mndInitOper(SMnode *pMnode) { return 0; } +void mndCleanupOper(SMnode *pMnode) {} \ No newline at end of file diff --git a/source/dnode/mnode/impl/src/mndProfile.c b/source/dnode/mnode/impl/src/mndProfile.c index 7c86778eb4..845e50210a 100644 --- a/source/dnode/mnode/impl/src/mndProfile.c +++ b/source/dnode/mnode/impl/src/mndProfile.c @@ -17,5 +17,5 @@ #include "os.h" #include "mndInt.h" -int32_t mndInitProfile() { return 0; } -void mndCleanupProfile() {} \ No newline at end of file +int32_t mndInitProfile(SMnode *pMnode) { return 0; } +void mndCleanupProfile(SMnode *pMnode) {} \ No newline at end of file diff --git a/source/dnode/mnode/impl/src/mndShow.c b/source/dnode/mnode/impl/src/mndShow.c index 3f97470fd4..9938e95a73 100644 --- a/source/dnode/mnode/impl/src/mndShow.c +++ b/source/dnode/mnode/impl/src/mndShow.c @@ -17,5 +17,5 @@ #include "os.h" #include "mndInt.h" -int32_t mndInitShow() { return 0; } -void mndCleanupShow() {} \ No newline at end of file +int32_t mndInitShow(SMnode *pMnode) { return 0; } +void mndCleanupShow(SMnode *pMnode) {} \ No newline at end of file diff --git a/source/dnode/mnode/impl/src/mndStable.c b/source/dnode/mnode/impl/src/mndStable.c index 0a8320c9fa..e54bb17451 100644 --- a/source/dnode/mnode/impl/src/mndStable.c +++ b/source/dnode/mnode/impl/src/mndStable.c @@ -17,5 +17,5 @@ #include "os.h" #include "mndInt.h" -int32_t mndInitStable() { return 0; } -void mndCleanupStable() {} \ No newline at end of file +int32_t mndInitStable(SMnode *pMnode) { return 0; } +void mndCleanupStable(SMnode *pMnode) {} \ No newline at end of file diff --git a/source/dnode/mnode/impl/src/mndSync.c b/source/dnode/mnode/impl/src/mndSync.c index 89098e7ec0..a3452d39c9 100644 --- a/source/dnode/mnode/impl/src/mndSync.c +++ b/source/dnode/mnode/impl/src/mndSync.c @@ -18,8 +18,8 @@ #include "mndInt.h" #include "mndTrans.h" -int32_t mndInitSync() { return 0; } -void mndCleanupSync() {} +int32_t mndInitSync(SMnode *pMnode) { return 0; } +void mndCleanupSync(SMnode *pMnode) {} int32_t mndSyncPropose(SSdbRaw *pRaw, void *pData) { trnApply(pData, pData, 0); diff --git a/source/dnode/mnode/impl/src/mndTelem.c b/source/dnode/mnode/impl/src/mndTelem.c index f5f5464822..f7fafc7095 100644 --- a/source/dnode/mnode/impl/src/mndTelem.c +++ b/source/dnode/mnode/impl/src/mndTelem.c @@ -256,7 +256,7 @@ static void* mndTelemThreadFp(void* param) { if (r == 0) break; if (r != ETIMEDOUT) continue; - if (mndIsMaster()) { + if (mndIsMaster(NULL)) { mndSendTelemetryReport(); } end.tv_sec += REPORT_INTERVAL; @@ -278,7 +278,7 @@ static void mndGetEmail(char* filepath) { taosCloseFile(fd); } -int32_t mndInitTelem() { +int32_t mndInitTelem(SMnode *pMnode) { tsTelem.enable = tsEnableTelemetryReporting; if (!tsTelem.enable) return 0; @@ -303,7 +303,7 @@ int32_t mndInitTelem() { return 0; } -void mndCleanupTelem() { +void mndCleanupTelem(SMnode *pMnode) { if (!tsTelem.enable) return; if (taosCheckPthreadValid(tsTelem.thread)) { diff --git a/source/dnode/mnode/impl/src/mndTrans.c b/source/dnode/mnode/impl/src/mndTrans.c index ed5a2a7b14..f2fe420013 100644 --- a/source/dnode/mnode/impl/src/mndTrans.c +++ b/source/dnode/mnode/impl/src/mndTrans.c @@ -312,7 +312,7 @@ int32_t trnAppendUndoAction(STrans *pTrans, SEpSet *pEpSet, void *pMsg) { return code; } -int32_t mndInitTrans() { +int32_t mndInitTrans(SMnode *pMnode) { SSdbTable table = {.sdbType = SDB_TRANS, .keyType = SDB_KEY_INT32, .encodeFp = (SdbEncodeFp)trnActionEncode, @@ -326,7 +326,7 @@ int32_t mndInitTrans() { return 0; } -void mndCleanupTrans() { mInfo("trn module is cleaned up"); } +void mndCleanupTrans(SMnode *pMnode) { mInfo("trn module is cleaned up"); } int32_t trnPrepare(STrans *pTrans, int32_t (*syncfp)(SSdbRaw *pRaw, void *pData)) { diff --git a/source/dnode/mnode/impl/src/mndUser.c b/source/dnode/mnode/impl/src/mndUser.c index 32c2c459e5..bfc2da9118 100644 --- a/source/dnode/mnode/impl/src/mndUser.c +++ b/source/dnode/mnode/impl/src/mndUser.c @@ -214,7 +214,7 @@ static int32_t mndProcessCreateUserMsg(SMnode *pMnode, SMnodeMsg *pMsg) { return TSDB_CODE_MND_ACTION_IN_PROGRESS; } -int32_t mndInitUser() { +int32_t mndInitUser(SMnode *pMnode) { SSdbTable table = {.sdbType = SDB_USER, .keyType = SDB_KEY_BINARY, .deployFp = (SdbDeployFp)mndCreateDefaultUsers, @@ -225,9 +225,9 @@ int32_t mndInitUser() { .deleteFp = (SdbDeleteFp)mndUserActionDelete}; sdbSetTable(table); - mndSetMsgHandle(NULL, TSDB_MSG_TYPE_CREATE_USER, mndProcessCreateUserMsg); + mndSetMsgHandle(pMnode, TSDB_MSG_TYPE_CREATE_USER, mndProcessCreateUserMsg); return 0; } -void mndCleanupUser() {} \ No newline at end of file +void mndCleanupUser(SMnode *pMnode) {} \ No newline at end of file diff --git a/source/dnode/mnode/impl/src/mndVgroup.c b/source/dnode/mnode/impl/src/mndVgroup.c index f7af6e5fa8..b98468b63e 100644 --- a/source/dnode/mnode/impl/src/mndVgroup.c +++ b/source/dnode/mnode/impl/src/mndVgroup.c @@ -17,5 +17,5 @@ #include "os.h" #include "mndInt.h" -int32_t mndInitVgroup() { return 0; } -void mndCleanupVgroup() {} \ No newline at end of file +int32_t mndInitVgroup(SMnode *pMnode) { return 0; } +void mndCleanupVgroup(SMnode *pMnode) {} \ No newline at end of file From 18c871a9f17b741d51901d3ab120adf80172a330 Mon Sep 17 00:00:00 2001 From: Shengliang Guan Date: Mon, 29 Nov 2021 14:14:03 +0800 Subject: [PATCH 39/44] TD-10431 add global variables in sdb --- include/dnode/mnode/sdb/sdb.h | 14 ++- source/dnode/mnode/impl/inc/mndInt.h | 1 + source/dnode/mnode/impl/src/mndAcct.c | 2 +- source/dnode/mnode/impl/src/mndTrans.c | 2 +- source/dnode/mnode/impl/src/mndUser.c | 2 +- source/dnode/mnode/impl/src/mnode.c | 130 ++++++++++++++++--------- source/dnode/mnode/sdb/src/sdb.c | 76 ++++++++------- source/dnode/mnode/sdb/src/sdbFile.c | 40 ++++---- 8 files changed, 159 insertions(+), 108 deletions(-) diff --git a/include/dnode/mnode/sdb/sdb.h b/include/dnode/mnode/sdb/sdb.h index 784672e0ec..3f746f2b29 100644 --- a/include/dnode/mnode/sdb/sdb.h +++ b/include/dnode/mnode/sdb/sdb.h @@ -126,6 +126,10 @@ typedef enum { SDB_MAX = 12 } ESdbType; +typedef struct SSdbOpt { + const char *path; +} SSdbOpt; + typedef int32_t (*SdbInsertFp)(void *pObj); typedef int32_t (*SdbUpdateFp)(void *pSrcObj, void *pDstObj); typedef int32_t (*SdbDeleteFp)(void *pObj); @@ -146,12 +150,12 @@ typedef struct { typedef struct SSdb SSdb; -int32_t sdbInit(); -void sdbCleanup(); -void sdbSetTable(SSdbTable table); +SSdb *sdbOpen(SSdbOpt *pOption); +void sdbClose(SSdb *pSdb); +void sdbSetTable(SSdb *pSdb, SSdbTable table); -int32_t sdbOpen(); -void sdbClose(); +// int32_t sdbOpen(); +// void sdbClose(); int32_t sdbWrite(SSdbRaw *pRaw); int32_t sdbDeploy(); diff --git a/source/dnode/mnode/impl/inc/mndInt.h b/source/dnode/mnode/impl/inc/mndInt.h index ac9c715adc..8a870091d4 100644 --- a/source/dnode/mnode/impl/inc/mndInt.h +++ b/source/dnode/mnode/impl/inc/mndInt.h @@ -43,6 +43,7 @@ typedef struct SMnode { tmr_h timer; SSdb *pSdb; SDnode *pDnode; + char *path; SArray steps; MndMsgFp msgFp[TSDB_MSG_TYPE_MAX]; SendMsgToDnodeFp sendMsgToDnodeFp; diff --git a/source/dnode/mnode/impl/src/mndAcct.c b/source/dnode/mnode/impl/src/mndAcct.c index 0ee290943e..98434482c8 100644 --- a/source/dnode/mnode/impl/src/mndAcct.c +++ b/source/dnode/mnode/impl/src/mndAcct.c @@ -110,8 +110,8 @@ int32_t mndInitAcct(SMnode *pMnode) { .insertFp = (SdbInsertFp)mnodeAcctActionInsert, .updateFp = (SdbUpdateFp)mnodeAcctActionUpdate, .deleteFp = (SdbDeleteFp)mnodeAcctActionDelete}; - sdbSetTable(table); + sdbSetTable(pMnode->pSdb, table); return 0; } diff --git a/source/dnode/mnode/impl/src/mndTrans.c b/source/dnode/mnode/impl/src/mndTrans.c index f2fe420013..c1c2d9b3d3 100644 --- a/source/dnode/mnode/impl/src/mndTrans.c +++ b/source/dnode/mnode/impl/src/mndTrans.c @@ -320,7 +320,7 @@ int32_t mndInitTrans(SMnode *pMnode) { .insertFp = (SdbInsertFp)trnActionInsert, .updateFp = (SdbUpdateFp)trnActionUpdate, .deleteFp = (SdbDeleteFp)trnActionDelete}; - sdbSetTable(table); + sdbSetTable(pMnode->pSdb, table); mInfo("trn module is initialized"); return 0; diff --git a/source/dnode/mnode/impl/src/mndUser.c b/source/dnode/mnode/impl/src/mndUser.c index bfc2da9118..c50b220126 100644 --- a/source/dnode/mnode/impl/src/mndUser.c +++ b/source/dnode/mnode/impl/src/mndUser.c @@ -223,7 +223,7 @@ int32_t mndInitUser(SMnode *pMnode) { .insertFp = (SdbInsertFp)mndUserActionInsert, .updateFp = (SdbUpdateFp)mndUserActionUpdate, .deleteFp = (SdbDeleteFp)mndUserActionDelete}; - sdbSetTable(table); + sdbSetTable(pMnode->pSdb, table); mndSetMsgHandle(pMnode, TSDB_MSG_TYPE_CREATE_USER, mndProcessCreateUserMsg); diff --git a/source/dnode/mnode/impl/src/mnode.c b/source/dnode/mnode/impl/src/mnode.c index 369994c2bb..4459dc2572 100644 --- a/source/dnode/mnode/impl/src/mnode.c +++ b/source/dnode/mnode/impl/src/mnode.c @@ -46,6 +46,12 @@ int64_t mndGetClusterId(SMnode *pMnode) { return -1; } +tmr_h mndGetTimer(SMnode *pMnode) { + if (pMnode != NULL) { + return pMnode->timer; + } +} + void mndSendMsgToDnode(SMnode *pMnode, SEpSet *pEpSet, SRpcMsg *pMsg) { if (pMnode != NULL && pMnode->sendMsgToDnodeFp != NULL) { (*pMnode->sendMsgToDnodeFp)(pMnode->pDnode, pEpSet, pMsg); @@ -83,33 +89,42 @@ static void mndCleanupTimer(SMnode *pMnode) { } } -tmr_h mndGetTimer(SMnode *pMnode) { - if (pMnode != NULL) { - return pMnode->timer; +static int32_t mnodeCreateDir(SMnode *pMnode, const char *path) { + pMnode->path = strdup(path); + if (pMnode->path == NULL) { + terrno = TSDB_CODE_OUT_OF_MEMORY; + return terrno; } + + if (taosMkDir(pMnode->path) != 0) { + terrno = TAOS_SYSTEM_ERROR(errno); + return terrno; + } + + return 0; } -static int32_t mndSetOptions(SMnode *pMnode, const SMnodeOpt *pOption) { - pMnode->dnodeId = pOption->dnodeId; - pMnode->clusterId = pOption->clusterId; - pMnode->replica = pOption->replica; - pMnode->selfIndex = pOption->selfIndex; - memcpy(&pMnode->replicas, pOption->replicas, sizeof(SReplica) * TSDB_MAX_REPLICA); - pMnode->pDnode = pOption->pDnode; - pMnode->putMsgToApplyMsgFp = pOption->putMsgToApplyMsgFp; - pMnode->sendMsgToDnodeFp = pOption->sendMsgToDnodeFp; - pMnode->sendMsgToMnodeFp = pOption->sendMsgToMnodeFp; - pMnode->sendRedirectMsgFp = pOption->sendRedirectMsgFp; +static int32_t mndInitSdb(SMnode *pMnode) { + SSdbOpt opt = {0}; + opt.path = pMnode->path; - if (pMnode->sendMsgToDnodeFp == NULL || pMnode->sendMsgToMnodeFp == NULL || pMnode->sendRedirectMsgFp == NULL || - pMnode->putMsgToApplyMsgFp == NULL || pMnode->dnodeId < 0 || pMnode->clusterId < 0) { - terrno = TSDB_CODE_MND_APP_ERROR; + pMnode->pSdb = sdbOpen(&opt); + if (pMnode->pSdb == NULL) { return -1; } return 0; } +static int32_t mndDeploySdb(SMnode *pMnode) { return sdbDeploy(pMnode->pSdb); } + +static void mndCleanupSdb(SMnode *pMnode) { + if (pMnode->pSdb) { + sdbClose(pMnode->pSdb); + pMnode->pSdb = NULL; + } +} + static int32_t mndAllocStep(SMnode *pMnode, char *name, MndInitFp initFp, MndCleanupFp cleanupFp) { SMnodeStep step = {0}; step.name = name; @@ -125,33 +140,28 @@ static int32_t mndAllocStep(SMnode *pMnode, char *name, MndInitFp initFp, MndCle } static int32_t mndInitSteps(SMnode *pMnode) { - if (mndAllocStep(pMnode, "mnode-trans", mndInitTrans, mndCleanupTrans) != 0) return -1; - if (mndAllocStep(pMnode, "mnode-cluster", mndInitCluster, mndCleanupCluster) != 0) return -1; - if (mndAllocStep(pMnode, "mnode-dnode", mndInitDnode, mndCleanupDnode) != 0) return -1; - if (mndAllocStep(pMnode, "mnode-mnode", mndInitMnode, mndCleanupMnode) != 0) return -1; - if (mndAllocStep(pMnode, "mnode-acct", mndInitAcct, mndCleanupAcct) != 0) return -1; - if (mndAllocStep(pMnode, "mnode-auth", mndInitAuth, mndCleanupAuth) != 0) return -1; - if (mndAllocStep(pMnode, "mnode-user", mndInitUser, mndCleanupUser) != 0) return -1; - if (mndAllocStep(pMnode, "mnode-db", mndInitDb, mndCleanupDb) != 0) return -1; - if (mndAllocStep(pMnode, "mnode-vgroup", mndInitVgroup, mndCleanupVgroup) != 0) return -1; - if (mndAllocStep(pMnode, "mnode-stable", mndInitStable, mndCleanupStable) != 0) return -1; - if (mndAllocStep(pMnode, "mnode-func", mndInitFunc, mndCleanupFunc) != 0) return -1; - if (mndAllocStep(pMnode, "mnode-sdb", sdbInit, sdbCleanup) != 0) return -1; - - if (pMnode->replica == 1) { - if (mndAllocStep(pMnode, "mnode-deploy-sdb", sdbDeploy, sdbClose) != 0) return -1; - } else { - if (mndAllocStep(pMnode, "mnode-open-sdb", sdbOpen, sdbClose) != 0) return -1; + if (mndAllocStep(pMnode, "mnode-trans", mndInitTrans, mndCleanupTrans) != 0) return terrno; + if (mndAllocStep(pMnode, "mnode-cluster", mndInitCluster, mndCleanupCluster) != 0) return terrno; + if (mndAllocStep(pMnode, "mnode-dnode", mndInitDnode, mndCleanupDnode) != 0) return terrno; + if (mndAllocStep(pMnode, "mnode-mnode", mndInitMnode, mndCleanupMnode) != 0) return terrno; + if (mndAllocStep(pMnode, "mnode-acct", mndInitAcct, mndCleanupAcct) != 0) return terrno; + if (mndAllocStep(pMnode, "mnode-auth", mndInitAuth, mndCleanupAuth) != 0) return terrno; + if (mndAllocStep(pMnode, "mnode-user", mndInitUser, mndCleanupUser) != 0) return terrno; + if (mndAllocStep(pMnode, "mnode-db", mndInitDb, mndCleanupDb) != 0) return terrno; + if (mndAllocStep(pMnode, "mnode-vgroup", mndInitVgroup, mndCleanupVgroup) != 0) return terrno; + if (mndAllocStep(pMnode, "mnode-stable", mndInitStable, mndCleanupStable) != 0) return terrno; + if (mndAllocStep(pMnode, "mnode-func", mndInitFunc, mndCleanupFunc) != 0) return terrno; + if (mndAllocStep(pMnode, "mnode-sdb", mndInitSdb, mndCleanupSdb) != 0) return terrno; + if (pMnode->clusterId <= 0) { + if (mndAllocStep(pMnode, "mnode-deploy", mndDeploySdb, NULL) != 0) return terrno; } - - if (mndAllocStep(pMnode, "mnode-timer", mndInitTimer, NULL) != 0) return -1; - if (mndAllocStep(pMnode, "mnode-sdb-file", sdbOpen, sdbClose) != 0) return -1; - if (mndAllocStep(pMnode, "mnode-balance", mndInitBalance, mndCleanupBalance) != 0) return -1; - if (mndAllocStep(pMnode, "mnode-profile", mndInitProfile, mndCleanupProfile) != 0) return -1; - if (mndAllocStep(pMnode, "mnode-show", mndInitShow, mndCleanupShow) != 0) return -1; - if (mndAllocStep(pMnode, "mnode-sync", mndInitSync, mndCleanupSync) != 0) return -1; - if (mndAllocStep(pMnode, "mnode-telem", mndInitTelem, mndCleanupTelem) != 0) return -1; - if (mndAllocStep(pMnode, "mnode-timer", NULL, mndCleanupTimer) != 0) return -1; + if (mndAllocStep(pMnode, "mnode-timer", mndInitTimer, NULL) != 0) return terrno; + if (mndAllocStep(pMnode, "mnode-balance", mndInitBalance, mndCleanupBalance) != 0) return terrno; + if (mndAllocStep(pMnode, "mnode-profile", mndInitProfile, mndCleanupProfile) != 0) return terrno; + if (mndAllocStep(pMnode, "mnode-show", mndInitShow, mndCleanupShow) != 0) return terrno; + if (mndAllocStep(pMnode, "mnode-sync", mndInitSync, mndCleanupSync) != 0) return terrno; + if (mndAllocStep(pMnode, "mnode-telem", mndInitTelem, mndCleanupTelem) != 0) return terrno; + if (mndAllocStep(pMnode, "mnode-timer", NULL, mndCleanupTimer) != 0) return terrno; return 0; } @@ -194,10 +204,39 @@ static int32_t mndExecSteps(SMnode *pMnode) { } } +static int32_t mndSetOptions(SMnode *pMnode, const SMnodeOpt *pOption) { + pMnode->dnodeId = pOption->dnodeId; + pMnode->clusterId = pOption->clusterId; + pMnode->replica = pOption->replica; + pMnode->selfIndex = pOption->selfIndex; + memcpy(&pMnode->replicas, pOption->replicas, sizeof(SReplica) * TSDB_MAX_REPLICA); + pMnode->pDnode = pOption->pDnode; + pMnode->putMsgToApplyMsgFp = pOption->putMsgToApplyMsgFp; + pMnode->sendMsgToDnodeFp = pOption->sendMsgToDnodeFp; + pMnode->sendMsgToMnodeFp = pOption->sendMsgToMnodeFp; + pMnode->sendRedirectMsgFp = pOption->sendRedirectMsgFp; + + if (pMnode->sendMsgToDnodeFp == NULL || pMnode->sendMsgToMnodeFp == NULL || pMnode->sendRedirectMsgFp == NULL || + pMnode->putMsgToApplyMsgFp == NULL || pMnode->dnodeId < 0 || pMnode->clusterId < 0) { + terrno = TSDB_CODE_MND_APP_ERROR; + return terrno; + } + + return 0; +} + SMnode *mndOpen(const char *path, const SMnodeOpt *pOption) { SMnode *pMnode = calloc(1, sizeof(SMnode)); - int32_t code = mndSetOptions(pMnode, pOption); + int32_t code = mnodeCreateDir(pMnode, path); + if (code != 0) { + mError("failed to set mnode options since %s", terrstr()); + mndClose(pMnode); + terrno = code; + return NULL; + } + + code = mndSetOptions(pMnode, pOption); if (code != 0) { mndClose(pMnode); terrno = code; @@ -227,7 +266,8 @@ SMnode *mndOpen(const char *path, const SMnodeOpt *pOption) { void mndClose(SMnode *pMnode) { mndCleanupSteps(pMnode, -1); - free(pMnode); + tfree(pMnode->path); + tfree(pMnode); mDebug("mnode:%p object is cleaned up", pMnode); } diff --git a/source/dnode/mnode/sdb/src/sdb.c b/source/dnode/mnode/sdb/src/sdb.c index 83496f3794..f7af8fb4bd 100644 --- a/source/dnode/mnode/sdb/src/sdb.c +++ b/source/dnode/mnode/sdb/src/sdb.c @@ -19,27 +19,31 @@ SSdb tsSdb = {0}; -int32_t sdbInit() { +SSdb *sdbOpen(SSdbOpt *pOption) { + SSdb *pSdb = calloc(1, sizeof(SSdb)); + if (pSdb == NULL) { + terrno = TSDB_CODE_OUT_OF_MEMORY; + return NULL; + } + char path[PATH_MAX + 100]; - - snprintf(path, PATH_MAX + 100, "%s%scur%s", tsMnodeDir, TD_DIRSEP, TD_DIRSEP); - tsSdb.currDir = strdup(path); - - snprintf(path, PATH_MAX + 100, "%s%ssync%s", tsMnodeDir, TD_DIRSEP, TD_DIRSEP); - tsSdb.syncDir = strdup(path); - - snprintf(path, PATH_MAX + 100, "%s%stmp%s", tsMnodeDir, TD_DIRSEP, TD_DIRSEP); - tsSdb.tmpDir = strdup(path); - - if (tsSdb.currDir == NULL || tsSdb.currDir == NULL || tsSdb.currDir == NULL) { - return TSDB_CODE_OUT_OF_MEMORY; + snprintf(path, PATH_MAX + 100, "%s%scur%s", pOption->path, TD_DIRSEP, TD_DIRSEP); + pSdb->currDir = strdup(path); + snprintf(path, PATH_MAX + 100, "%s%ssync%s", pOption->path, TD_DIRSEP, TD_DIRSEP); + pSdb->syncDir = strdup(path); + snprintf(path, PATH_MAX + 100, "%s%stmp%s", pOption->path, TD_DIRSEP, TD_DIRSEP); + pSdb->tmpDir = strdup(path); + if (pSdb->currDir == NULL || pSdb->currDir == NULL || pSdb->currDir == NULL) { + sdbClose(pSdb); + terrno = TSDB_CODE_OUT_OF_MEMORY; + return NULL; } for (int32_t i = 0; i < SDB_MAX; ++i) { int32_t type; - if (tsSdb.keyTypes[i] == SDB_KEY_INT32) { + if (pSdb->keyTypes[i] == SDB_KEY_INT32) { type = TSDB_DATA_TYPE_INT; - } else if (tsSdb.keyTypes[i] == SDB_KEY_INT64) { + } else if (pSdb->keyTypes[i] == SDB_KEY_INT64) { type = TSDB_DATA_TYPE_BIGINT; } else { type = TSDB_DATA_TYPE_BINARY; @@ -47,45 +51,47 @@ int32_t sdbInit() { SHashObj *hash = taosHashInit(64, taosGetDefaultHashFunction(type), true, HASH_NO_LOCK); if (hash == NULL) { - return TSDB_CODE_OUT_OF_MEMORY; + sdbClose(pSdb); + terrno = TSDB_CODE_OUT_OF_MEMORY; + return NULL; } - tsSdb.hashObjs[i] = hash; - taosInitRWLatch(&tsSdb.locks[i]); + pSdb->hashObjs[i] = hash; + taosInitRWLatch(&pSdb->locks[i]); } return 0; } -void sdbCleanup() { - if (tsSdb.currDir != NULL) { - tfree(tsSdb.currDir); +void sdbClose(SSdb *pSdb) { + if (pSdb->currDir != NULL) { + tfree(pSdb->currDir); } - if (tsSdb.syncDir != NULL) { - tfree(tsSdb.syncDir); + if (pSdb->syncDir != NULL) { + tfree(pSdb->syncDir); } - if (tsSdb.tmpDir != NULL) { - tfree(tsSdb.tmpDir); + if (pSdb->tmpDir != NULL) { + tfree(pSdb->tmpDir); } for (int32_t i = 0; i < SDB_MAX; ++i) { - SHashObj *hash = tsSdb.hashObjs[i]; + SHashObj *hash = pSdb->hashObjs[i]; if (hash != NULL) { taosHashCleanup(hash); } - tsSdb.hashObjs[i] = NULL; + pSdb->hashObjs[i] = NULL; } } -void sdbSetTable(SSdbTable table) { +void sdbSetTable(SSdb *pSdb, SSdbTable table) { ESdbType sdb = table.sdbType; - tsSdb.keyTypes[sdb] = table.keyType; - tsSdb.insertFps[sdb] = table.insertFp; - tsSdb.updateFps[sdb] = table.updateFp; - tsSdb.deleteFps[sdb] = table.deleteFp; - tsSdb.deployFps[sdb] = table.deployFp; - tsSdb.encodeFps[sdb] = table.encodeFp; - tsSdb.decodeFps[sdb] = table.decodeFp; + pSdb->keyTypes[sdb] = table.keyType; + pSdb->insertFps[sdb] = table.insertFp; + pSdb->updateFps[sdb] = table.updateFp; + pSdb->deleteFps[sdb] = table.deleteFp; + pSdb->deployFps[sdb] = table.deployFp; + pSdb->encodeFps[sdb] = table.encodeFp; + pSdb->decodeFps[sdb] = table.decodeFp; } \ No newline at end of file diff --git a/source/dnode/mnode/sdb/src/sdbFile.c b/source/dnode/mnode/sdb/src/sdbFile.c index d77ec6ef8f..1764dff226 100644 --- a/source/dnode/mnode/sdb/src/sdbFile.c +++ b/source/dnode/mnode/sdb/src/sdbFile.c @@ -212,29 +212,29 @@ static int32_t sdbWriteDataFile() { return code; } -int32_t sdbOpen() { - mDebug("start to read mnode file"); +// int32_t sdbOpen() { +// mDebug("start to read mnode file"); - if (sdbReadDataFile() != 0) { - return -1; - } +// if (sdbReadDataFile() != 0) { +// return -1; +// } - return 0; -} +// return 0; +// } -void sdbClose() { - if (tsSdb.curVer != tsSdb.lastCommitVer) { - mDebug("start to write mnode file"); - sdbWriteDataFile(); - } +// void sdbClose() { +// if (tsSdb.curVer != tsSdb.lastCommitVer) { +// mDebug("start to write mnode file"); +// sdbWriteDataFile(); +// } - for (int32_t i = 0; i < SDB_MAX; ++i) { - SHashObj *hash = tsSdb.hashObjs[i]; - if (hash != NULL) { - taosHashClear(hash); - } - } -} +// for (int32_t i = 0; i < SDB_MAX; ++i) { +// SHashObj *hash = tsSdb.hashObjs[i]; +// if (hash != NULL) { +// taosHashClear(hash); +// } +// } +// } int32_t sdbDeploy() { if (sdbCreateDir() != 0) { @@ -249,7 +249,7 @@ int32_t sdbDeploy() { return -1; } - sdbClose(); + // sdbClose(); return 0; } From 1bf566a701d0f730fe9205327e62816c1eaaf4e9 Mon Sep 17 00:00:00 2001 From: Shengliang Guan Date: Mon, 29 Nov 2021 15:53:02 +0800 Subject: [PATCH 40/44] TD-10431 remove global variables --- include/dnode/mnode/sdb/sdb.h | 44 +++---- source/dnode/mnode/impl/inc/mndDef.h | 1 + source/dnode/mnode/impl/inc/mndTrans.h | 6 +- source/dnode/mnode/impl/src/mndAcct.c | 16 +-- source/dnode/mnode/impl/src/mndSync.c | 6 +- source/dnode/mnode/impl/src/mndTrans.c | 50 +++---- source/dnode/mnode/impl/src/mndUser.c | 34 ++--- source/dnode/mnode/impl/src/mnode.c | 4 +- source/dnode/mnode/sdb/inc/sdbInt.h | 6 +- source/dnode/mnode/sdb/src/sdb.c | 38 +++++- source/dnode/mnode/sdb/src/sdbFile.c | 172 +++++++++++-------------- source/dnode/mnode/sdb/src/sdbHash.c | 121 +++++++++-------- source/dnode/mnode/sdb/src/sdbRow.c | 2 +- 13 files changed, 251 insertions(+), 249 deletions(-) diff --git a/include/dnode/mnode/sdb/sdb.h b/include/dnode/mnode/sdb/sdb.h index 3f746f2b29..2c2d24268e 100644 --- a/include/dnode/mnode/sdb/sdb.h +++ b/include/dnode/mnode/sdb/sdb.h @@ -126,14 +126,11 @@ typedef enum { SDB_MAX = 12 } ESdbType; -typedef struct SSdbOpt { - const char *path; -} SSdbOpt; - -typedef int32_t (*SdbInsertFp)(void *pObj); -typedef int32_t (*SdbUpdateFp)(void *pSrcObj, void *pDstObj); -typedef int32_t (*SdbDeleteFp)(void *pObj); -typedef int32_t (*SdbDeployFp)(); +typedef struct SSdb SSdb; +typedef int32_t (*SdbInsertFp)(SSdb *pSdb, void *pObj); +typedef int32_t (*SdbUpdateFp)(SSdb *pSdb, void *pSrcObj, void *pDstObj); +typedef int32_t (*SdbDeleteFp)(SSdb *pSdb, void *pObj); +typedef int32_t (*SdbDeployFp)(SSdb*pSdb); typedef SSdbRow *(*SdbDecodeFp)(SSdbRaw *pRaw); typedef SSdbRaw *(*SdbEncodeFp)(void *pObj); @@ -148,26 +145,23 @@ typedef struct { SdbDeleteFp deleteFp; } SSdbTable; -typedef struct SSdb SSdb; +typedef struct SSdbOpt { + const char *path; +} SSdbOpt; -SSdb *sdbOpen(SSdbOpt *pOption); -void sdbClose(SSdb *pSdb); -void sdbSetTable(SSdb *pSdb, SSdbTable table); +SSdb *sdbOpen(SSdbOpt *pOption); +void sdbClose(SSdb *pSdb); +int32_t sdbDeploy(SSdb *pSdb); +void sdbSetTable(SSdb *pSdb, SSdbTable table); +int32_t sdbWrite(SSdb *pSdb, SSdbRaw *pRaw); -// int32_t sdbOpen(); -// void sdbClose(); -int32_t sdbWrite(SSdbRaw *pRaw); +void *sdbAcquire(SSdb *pSdb, ESdbType type, void *pKey); +void sdbRelease(SSdb *pSdb, void *pObj); +void *sdbFetch(SSdb *pSdb, ESdbType type, void *pIter, void **ppObj); +void sdbCancelFetch(SSdb *pSdb, void *pIter); +int32_t sdbGetSize(SSdb *pSdb, ESdbType type); -int32_t sdbDeploy(); -void sdbUnDeploy(); - -void *sdbAcquire(ESdbType sdb, void *pKey); -void sdbRelease(void *pObj); -void *sdbFetch(ESdbType sdb, void *pIter, void **ppObj); -void sdbCancelFetch(void *pIter); -int32_t sdbGetSize(ESdbType sdb); - -SSdbRaw *sdbAllocRaw(ESdbType sdb, int8_t sver, int32_t dataLen); +SSdbRaw *sdbAllocRaw(ESdbType type, int8_t sver, int32_t dataLen); void sdbFreeRaw(SSdbRaw *pRaw); int32_t sdbSetRawInt8(SSdbRaw *pRaw, int32_t dataPos, int8_t val); int32_t sdbSetRawInt32(SSdbRaw *pRaw, int32_t dataPos, int32_t val); diff --git a/source/dnode/mnode/impl/inc/mndDef.h b/source/dnode/mnode/impl/inc/mndDef.h index c0d02d3fc4..9b8c6eccc8 100644 --- a/source/dnode/mnode/impl/inc/mndDef.h +++ b/source/dnode/mnode/impl/inc/mndDef.h @@ -82,6 +82,7 @@ typedef struct STrans { int32_t id; ETrnStage stage; ETrnPolicy policy; + SMnode *pMnode; void *rpcHandle; SArray *redoLogs; SArray *undoLogs; diff --git a/source/dnode/mnode/impl/inc/mndTrans.h b/source/dnode/mnode/impl/inc/mndTrans.h index 4443dd2e1c..0a6395c900 100644 --- a/source/dnode/mnode/impl/inc/mndTrans.h +++ b/source/dnode/mnode/impl/inc/mndTrans.h @@ -25,7 +25,7 @@ extern "C" { int32_t mndInitTrans(SMnode *pMnode); void mndCleanupTrans(SMnode *pMnode); -STrans *trnCreate(ETrnPolicy policy, void *rpcHandle); +STrans *trnCreate(SMnode *pMnode, ETrnPolicy policy, void *rpcHandle); void trnDrop(STrans *pTrans); int32_t trnAppendRedoLog(STrans *pTrans, SSdbRaw *pRaw); int32_t trnAppendUndoLog(STrans *pTrans, SSdbRaw *pRaw); @@ -34,8 +34,8 @@ int32_t trnAppendRedoAction(STrans *pTrans, SEpSet *, void *pMsg); int32_t trnAppendUndoAction(STrans *pTrans, SEpSet *, void *pMsg); int32_t trnPrepare(STrans *pTrans, int32_t (*syncfp)(SSdbRaw *pRaw, void *pData)); -int32_t trnApply(SSdbRaw *pRaw, void *pData, int32_t code); -int32_t trnExecute(int32_t tranId); +int32_t trnApply(SMnode *pMnode, SSdbRaw *pRaw, void *pData, int32_t code); +int32_t trnExecute(SSdb *pSdb, int32_t tranId); SSdbRaw *trnActionEncode(STrans *pTrans); SSdbRow *trnActionDecode(SSdbRaw *pRaw); diff --git a/source/dnode/mnode/impl/src/mndAcct.c b/source/dnode/mnode/impl/src/mndAcct.c index 98434482c8..b52c7d1e8e 100644 --- a/source/dnode/mnode/impl/src/mndAcct.c +++ b/source/dnode/mnode/impl/src/mndAcct.c @@ -48,10 +48,10 @@ static SSdbRow *mnodeAcctActionDecode(SSdbRaw *pRaw) { return NULL; } - SSdbRow *pRow = sdbAllocRow(sizeof(SAcctObj)); + SSdbRow *pRow = sdbAllocRow(sizeof(SAcctObj)); SAcctObj *pAcct = sdbGetRowObj(pRow); if (pAcct == NULL) return NULL; - + int32_t dataPos = 0; SDB_GET_BINARY(pRaw, pRow, dataPos, pAcct->acct, TSDB_USER_LEN) SDB_GET_INT64(pRaw, pRow, dataPos, &pAcct->createdTime) @@ -68,18 +68,18 @@ static SSdbRow *mnodeAcctActionDecode(SSdbRaw *pRaw) { return pRow; } -static int32_t mnodeAcctActionInsert(SAcctObj *pAcct) { return 0; } +static int32_t mnodeAcctActionInsert(SSdb *pSdb, SAcctObj *pAcct) { return 0; } -static int32_t mnodeAcctActionDelete(SAcctObj *pAcct) { return 0; } +static int32_t mnodeAcctActionDelete(SSdb *pSdb, SAcctObj *pAcct) { return 0; } -static int32_t mnodeAcctActionUpdate(SAcctObj *pSrcAcct, SAcctObj *pDstAcct) { +static int32_t mnodeAcctActionUpdate(SSdb *pSdb, SAcctObj *pSrcAcct, SAcctObj *pDstAcct) { SAcctObj tObj; int32_t len = (int32_t)((int8_t *)&tObj.info - (int8_t *)&tObj); memcpy(pDstAcct, pSrcAcct, len); return 0; } -static int32_t mnodeCreateDefaultAcct() { +static int32_t mnodeCreateDefaultAcct(SSdb *pSdb) { int32_t code = 0; SAcctObj acctObj = {0}; @@ -98,13 +98,13 @@ static int32_t mnodeCreateDefaultAcct() { if (pRaw == NULL) return -1; sdbSetRawStatus(pRaw, SDB_STATUS_READY); - return sdbWrite(pRaw); + return sdbWrite(pSdb, pRaw); } int32_t mndInitAcct(SMnode *pMnode) { SSdbTable table = {.sdbType = SDB_ACCT, .keyType = SDB_KEY_BINARY, - .deployFp = (SdbDeployFp)mnodeCreateDefaultAcct, + .deployFp = mnodeCreateDefaultAcct, .encodeFp = (SdbEncodeFp)mnodeAcctActionEncode, .decodeFp = (SdbDecodeFp)mnodeAcctActionDecode, .insertFp = (SdbInsertFp)mnodeAcctActionInsert, diff --git a/source/dnode/mnode/impl/src/mndSync.c b/source/dnode/mnode/impl/src/mndSync.c index a3452d39c9..3535dcb677 100644 --- a/source/dnode/mnode/impl/src/mndSync.c +++ b/source/dnode/mnode/impl/src/mndSync.c @@ -21,10 +21,10 @@ int32_t mndInitSync(SMnode *pMnode) { return 0; } void mndCleanupSync(SMnode *pMnode) {} -int32_t mndSyncPropose(SSdbRaw *pRaw, void *pData) { - trnApply(pData, pData, 0); +int32_t mndSyncPropose(SMnode *pMnode, SSdbRaw *pRaw, void *pData) { + trnApply(pMnode, pData, pData, 0); free(pData); return 0; } -bool mndIsMaster() { return true; } \ No newline at end of file +bool mndIsMaster(SMnode *pMnode) { return true; } \ No newline at end of file diff --git a/source/dnode/mnode/impl/src/mndTrans.c b/source/dnode/mnode/impl/src/mndTrans.c index c1c2d9b3d3..168cb27757 100644 --- a/source/dnode/mnode/impl/src/mndTrans.c +++ b/source/dnode/mnode/impl/src/mndTrans.c @@ -158,13 +158,13 @@ SSdbRow *trnActionDecode(SSdbRaw *pRaw) { return pRow; } -static int32_t trnActionInsert(STrans *pTrans) { +static int32_t trnActionInsert(SSdb *pSdb, STrans *pTrans) { SArray *pArray = pTrans->redoLogs; int32_t arraySize = taosArrayGetSize(pArray); for (int32_t index = 0; index < arraySize; ++index) { SSdbRaw *pRaw = taosArrayGet(pArray, index); - int32_t code = sdbWrite(pRaw); + int32_t code = sdbWrite(pSdb, pRaw); if (code != 0) { mError("trn:%d, failed to write raw:%p to sdb since %s", pTrans->id, pRaw, terrstr()); return code; @@ -175,13 +175,13 @@ static int32_t trnActionInsert(STrans *pTrans) { return 0; } -static int32_t trnActionDelete(STrans *pTrans) { +static int32_t trnActionDelete(SSdb *pSdb, STrans *pTrans) { SArray *pArray = pTrans->redoLogs; int32_t arraySize = taosArrayGetSize(pArray); for (int32_t index = 0; index < arraySize; ++index) { SSdbRaw *pRaw = taosArrayGet(pArray, index); - int32_t code = sdbWrite(pRaw); + int32_t code = sdbWrite(pSdb, pRaw); if (code != 0) { mError("trn:%d, failed to write raw:%p to sdb since %s", pTrans->id, pRaw, terrstr()); return code; @@ -192,14 +192,14 @@ static int32_t trnActionDelete(STrans *pTrans) { return 0; } -static int32_t trnActionUpdate(STrans *pTrans, STrans *pDstTrans) { +static int32_t trnActionUpdate(SSdb *pSdb, STrans *pTrans, STrans *pDstTrans) { assert(true); SArray *pArray = pTrans->redoLogs; int32_t arraySize = taosArrayGetSize(pArray); for (int32_t index = 0; index < arraySize; ++index) { SSdbRaw *pRaw = taosArrayGet(pArray, index); - int32_t code = sdbWrite(pRaw); + int32_t code = sdbWrite(pSdb, pRaw); if (code != 0) { mError("trn:%d, failed to write raw:%p to sdb since %s", pTrans->id, pRaw, terrstr()); return code; @@ -213,7 +213,7 @@ static int32_t trnActionUpdate(STrans *pTrans, STrans *pDstTrans) { static int32_t trnGenerateTransId() { return 1; } -STrans *trnCreate(ETrnPolicy policy, void *rpcHandle) { +STrans *trnCreate(SMnode *pMnode, ETrnPolicy policy, void *rpcHandle) { STrans *pTrans = calloc(1, sizeof(STrans)); if (pTrans == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; @@ -339,7 +339,7 @@ int32_t trnPrepare(STrans *pTrans, int32_t (*syncfp)(SSdbRaw *pRaw, void *pData) } sdbSetRawStatus(pRaw, SDB_STATUS_CREATING); - if (sdbWrite(pRaw) != 0) { + if (sdbWrite(pTrans->pMnode->pSdb, pRaw) != 0) { mError("trn:%d, failed to write trans since %s", pTrans->id, terrstr()); return -1; } @@ -359,13 +359,13 @@ static void trnSendRpcRsp(void *rpcHandle, int32_t code) { } } -int32_t trnApply(SSdbRaw *pRaw, void *pData, int32_t code) { +int32_t trnApply(SMnode *pMnode, SSdbRaw *pRaw, void *pData, int32_t code) { if (code != 0) { trnSendRpcRsp(pData, terrno); return 0; } - if (sdbWrite(pData) != 0) { + if (sdbWrite(pMnode->pSdb, pData) != 0) { code = terrno; trnSendRpcRsp(pData, code); terrno = code; @@ -375,10 +375,10 @@ int32_t trnApply(SSdbRaw *pRaw, void *pData, int32_t code) { return 0; } -static int32_t trnExecuteArray(SArray *pArray) { +static int32_t trnExecuteArray(SMnode *pMnode, SArray *pArray) { for (int32_t index = 0; index < pArray->size; ++index) { SSdbRaw *pRaw = taosArrayGetP(pArray, index); - if (sdbWrite(pRaw) != 0) { + if (sdbWrite(pMnode->pSdb, pRaw) != 0) { return -1; } } @@ -386,15 +386,15 @@ static int32_t trnExecuteArray(SArray *pArray) { return 0; } -static int32_t trnExecuteRedoLogs(STrans *pTrans) { return trnExecuteArray(pTrans->redoLogs); } +static int32_t trnExecuteRedoLogs(STrans *pTrans) { return trnExecuteArray(pTrans->pMnode, pTrans->redoLogs); } -static int32_t trnExecuteUndoLogs(STrans *pTrans) { return trnExecuteArray(pTrans->undoLogs); } +static int32_t trnExecuteUndoLogs(STrans *pTrans) { return trnExecuteArray(pTrans->pMnode, pTrans->undoLogs); } -static int32_t trnExecuteCommitLogs(STrans *pTrans) { return trnExecuteArray(pTrans->commitLogs); } +static int32_t trnExecuteCommitLogs(STrans *pTrans) { return trnExecuteArray(pTrans->pMnode, pTrans->commitLogs); } -static int32_t trnExecuteRedoActions(STrans *pTrans) { return trnExecuteArray(pTrans->redoActions); } +static int32_t trnExecuteRedoActions(STrans *pTrans) { return trnExecuteArray(pTrans->pMnode, pTrans->redoActions); } -static int32_t trnExecuteUndoActions(STrans *pTrans) { return trnExecuteArray(pTrans->undoActions); } +static int32_t trnExecuteUndoActions(STrans *pTrans) { return trnExecuteArray(pTrans->pMnode, pTrans->undoActions); } static int32_t trnPerformPrepareStage(STrans *pTrans) { if (trnExecuteRedoLogs(pTrans) == 0) { @@ -454,49 +454,49 @@ static int32_t trnPerformRetryStage(STrans *pTrans) { } } -int32_t trnExecute(int32_t tranId) { +int32_t trnExecute(SSdb *pSdb, int32_t tranId) { int32_t code = 0; - STrans *pTrans = sdbAcquire(SDB_TRANS, &tranId); + STrans *pTrans = sdbAcquire(pSdb, SDB_TRANS, &tranId); if (pTrans == NULL) { return -1; } if (pTrans->stage == TRN_STAGE_PREPARE) { if (trnPerformPrepareStage(pTrans) != 0) { - sdbRelease(pTrans); + sdbRelease(pSdb, pTrans); return -1; } } if (pTrans->stage == TRN_STAGE_EXECUTE) { if (trnPerformExecuteStage(pTrans) != 0) { - sdbRelease(pTrans); + sdbRelease(pSdb, pTrans); return -1; } } if (pTrans->stage == TRN_STAGE_COMMIT) { if (trnPerformCommitStage(pTrans) != 0) { - sdbRelease(pTrans); + sdbRelease(pSdb, pTrans); return -1; } } if (pTrans->stage == TRN_STAGE_ROLLBACK) { if (trnPerformRollbackStage(pTrans) != 0) { - sdbRelease(pTrans); + sdbRelease(pSdb, pTrans); return -1; } } if (pTrans->stage == TRN_STAGE_RETRY) { if (trnPerformRetryStage(pTrans) != 0) { - sdbRelease(pTrans); + sdbRelease(pSdb, pTrans); return -1; } } - sdbRelease(pTrans); + sdbRelease(pSdb, pTrans); return 0; } \ No newline at end of file diff --git a/source/dnode/mnode/impl/src/mndUser.c b/source/dnode/mnode/impl/src/mndUser.c index c50b220126..95c1a75ced 100644 --- a/source/dnode/mnode/impl/src/mndUser.c +++ b/source/dnode/mnode/impl/src/mndUser.c @@ -60,14 +60,14 @@ static SSdbRow *mndUserActionDecode(SSdbRaw *pRaw) { return pRow; } -static int32_t mndUserActionInsert(SUserObj *pUser) { +static int32_t mndUserActionInsert(SSdb *pSdb, SUserObj *pUser) { pUser->prohibitDbHash = taosHashInit(8, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY), true, HASH_ENTRY_LOCK); if (pUser->prohibitDbHash == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; return -1; } - pUser->pAcct = sdbAcquire(SDB_ACCT, pUser->acct); + pUser->pAcct = sdbAcquire(pSdb, SDB_ACCT, pUser->acct); if (pUser->pAcct == NULL) { terrno = TSDB_CODE_MND_ACCT_NOT_EXIST; return -1; @@ -76,28 +76,28 @@ static int32_t mndUserActionInsert(SUserObj *pUser) { return 0; } -static int32_t mndUserActionDelete(SUserObj *pUser) { +static int32_t mndUserActionDelete(SSdb *pSdb, SUserObj *pUser) { if (pUser->prohibitDbHash) { taosHashCleanup(pUser->prohibitDbHash); pUser->prohibitDbHash = NULL; } if (pUser->acct != NULL) { - sdbRelease(pUser->pAcct); + sdbRelease(pSdb, pUser->pAcct); pUser->pAcct = NULL; } return 0; } -static int32_t mndUserActionUpdate(SUserObj *pSrcUser, SUserObj *pDstUser) { +static int32_t mndUserActionUpdate(SSdb *pSdb, SUserObj *pSrcUser, SUserObj *pDstUser) { SUserObj tObj; int32_t len = (int32_t)((int8_t *)tObj.prohibitDbHash - (int8_t *)&tObj); memcpy(pDstUser, pSrcUser, len); return 0; } -static int32_t mndCreateDefaultUser(char *acct, char *user, char *pass) { +static int32_t mndCreateDefaultUser(SSdb *pSdb, char *acct, char *user, char *pass) { SUserObj userObj = {0}; tstrncpy(userObj.user, user, TSDB_USER_LEN); tstrncpy(userObj.acct, acct, TSDB_USER_LEN); @@ -113,22 +113,22 @@ static int32_t mndCreateDefaultUser(char *acct, char *user, char *pass) { if (pRaw == NULL) return -1; sdbSetRawStatus(pRaw, SDB_STATUS_READY); - return sdbWrite(pRaw); + return sdbWrite(pSdb, pRaw); } -static int32_t mndCreateDefaultUsers() { - if (mndCreateDefaultUser(TSDB_DEFAULT_USER, TSDB_DEFAULT_USER, TSDB_DEFAULT_PASS) != 0) { +static int32_t mndCreateDefaultUsers(SSdb *pSdb) { + if (mndCreateDefaultUser(pSdb, TSDB_DEFAULT_USER, TSDB_DEFAULT_USER, TSDB_DEFAULT_PASS) != 0) { return -1; } - if (mndCreateDefaultUser(TSDB_DEFAULT_USER, "_" TSDB_DEFAULT_USER, TSDB_DEFAULT_PASS) != 0) { + if (mndCreateDefaultUser(pSdb, TSDB_DEFAULT_USER, "_" TSDB_DEFAULT_USER, TSDB_DEFAULT_PASS) != 0) { return -1; } return 0; } -static int32_t mndCreateUser(char *acct, char *user, char *pass, SMnodeMsg *pMsg) { +static int32_t mndCreateUser(SMnode *pMnode, char *acct, char *user, char *pass, SMnodeMsg *pMsg) { SUserObj userObj = {0}; tstrncpy(userObj.user, user, TSDB_USER_LEN); tstrncpy(userObj.acct, acct, TSDB_USER_LEN); @@ -137,7 +137,7 @@ static int32_t mndCreateUser(char *acct, char *user, char *pass, SMnodeMsg *pMsg userObj.updateTime = userObj.createdTime; userObj.rootAuth = 0; - STrans *pTrans = trnCreate(TRN_POLICY_ROLLBACK, pMsg->rpcMsg.handle); + STrans *pTrans = trnCreate(pMnode, TRN_POLICY_ROLLBACK, pMsg->rpcMsg.handle); if (pTrans == NULL) return -1; SSdbRaw *pRedoRaw = mndUserActionEncode(&userObj); @@ -188,23 +188,23 @@ static int32_t mndProcessCreateUserMsg(SMnode *pMnode, SMnodeMsg *pMsg) { return -1; } - SUserObj *pUser = sdbAcquire(SDB_USER, pCreate->user); + SUserObj *pUser = sdbAcquire(pMnode->pSdb, SDB_USER, pCreate->user); if (pUser != NULL) { - sdbRelease(pUser); + sdbRelease(pMnode->pSdb, pUser); terrno = TSDB_CODE_MND_USER_ALREADY_EXIST; mError("user:%s, failed to create since %s", pCreate->user, terrstr()); return -1; } - SUserObj *pOperUser = sdbAcquire(SDB_USER, pMsg->conn.user); + SUserObj *pOperUser = sdbAcquire(pMnode->pSdb, SDB_USER, pMsg->conn.user); if (pOperUser == NULL) { terrno = TSDB_CODE_MND_NO_USER_FROM_CONN; mError("user:%s, failed to create since %s", pCreate->user, terrstr()); return -1; } - int32_t code = mndCreateUser(pOperUser->acct, pCreate->user, pCreate->pass, pMsg); - sdbRelease(pOperUser); + int32_t code = mndCreateUser(pMnode, pOperUser->acct, pCreate->user, pCreate->pass, pMsg); + sdbRelease(pMnode->pSdb, pOperUser); if (code != 0) { mError("user:%s, failed to create since %s", pCreate->user, terrstr()); diff --git a/source/dnode/mnode/impl/src/mnode.c b/source/dnode/mnode/impl/src/mnode.c index 4459dc2572..9217fa7194 100644 --- a/source/dnode/mnode/impl/src/mnode.c +++ b/source/dnode/mnode/impl/src/mnode.c @@ -278,7 +278,7 @@ int32_t mndAlter(SMnode *pMnode, const SMnodeOpt *pOption) { void mndDestroy(const char *path) { mDebug("mnode in %s will be destroyed", path); - sdbUnDeploy(); + taosRemoveDir(path); } int32_t mndGetLoad(SMnode *pMnode, SMnodeLoad *pLoad) { @@ -308,7 +308,7 @@ SMnodeMsg *mndInitMsg(SMnode *pMnode, SRpcMsg *pRpcMsg) { void mndCleanupMsg(SMnodeMsg *pMsg) { if (pMsg->pUser != NULL) { - sdbRelease(pMsg->pUser); + sdbRelease(pMsg->pMnode->pSdb, pMsg->pUser); } taosFreeQitem(pMsg); diff --git a/source/dnode/mnode/sdb/inc/sdbInt.h b/source/dnode/mnode/sdb/inc/sdbInt.h index 2b3c577ba9..5c25349b5c 100644 --- a/source/dnode/mnode/sdb/inc/sdbInt.h +++ b/source/dnode/mnode/sdb/inc/sdbInt.h @@ -69,9 +69,9 @@ typedef struct SSdb { SdbDecodeFp decodeFps[SDB_MAX]; } SSdb; -extern SSdb tsSdb; - -int32_t sdbWriteImp(SSdbRaw *pRaw); +int32_t sdbReadFile(SSdb *pSdb); +int32_t sdbWriteFile(SSdb *pSdb); +int32_t sdbWriteRaw(SSdb *pSdb, SSdbRaw *pRaw); #ifdef __cplusplus } diff --git a/source/dnode/mnode/sdb/src/sdb.c b/source/dnode/mnode/sdb/src/sdb.c index f7af8fb4bd..c4c6a553f7 100644 --- a/source/dnode/mnode/sdb/src/sdb.c +++ b/source/dnode/mnode/sdb/src/sdb.c @@ -15,27 +15,28 @@ #define _DEFAULT_SOURCE #include "sdbInt.h" -#include "tglobal.h" - -SSdb tsSdb = {0}; SSdb *sdbOpen(SSdbOpt *pOption) { + mDebug("start to open sdb in %s", pOption->path); + SSdb *pSdb = calloc(1, sizeof(SSdb)); if (pSdb == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; + mError("failed to open sdb since %s", terrstr()); return NULL; } char path[PATH_MAX + 100]; - snprintf(path, PATH_MAX + 100, "%s%scur%s", pOption->path, TD_DIRSEP, TD_DIRSEP); + snprintf(path, PATH_MAX + 100, "%s%scur", pOption->path, TD_DIRSEP); pSdb->currDir = strdup(path); - snprintf(path, PATH_MAX + 100, "%s%ssync%s", pOption->path, TD_DIRSEP, TD_DIRSEP); + snprintf(path, PATH_MAX + 100, "%s%ssync", pOption->path, TD_DIRSEP); pSdb->syncDir = strdup(path); - snprintf(path, PATH_MAX + 100, "%s%stmp%s", pOption->path, TD_DIRSEP, TD_DIRSEP); + snprintf(path, PATH_MAX + 100, "%s%stmp", pOption->path, TD_DIRSEP); pSdb->tmpDir = strdup(path); if (pSdb->currDir == NULL || pSdb->currDir == NULL || pSdb->currDir == NULL) { sdbClose(pSdb); terrno = TSDB_CODE_OUT_OF_MEMORY; + mError("failed to open sdb since %s", terrstr()); return NULL; } @@ -53,6 +54,7 @@ SSdb *sdbOpen(SSdbOpt *pOption) { if (hash == NULL) { sdbClose(pSdb); terrno = TSDB_CODE_OUT_OF_MEMORY; + mError("failed to open sdb since %s", terrstr()); return NULL; } @@ -60,10 +62,27 @@ SSdb *sdbOpen(SSdbOpt *pOption) { taosInitRWLatch(&pSdb->locks[i]); } - return 0; + int32_t code = sdbReadFile(pSdb); + if (code != 0) { + sdbClose(pSdb); + terrno = code; + mError("failed to open sdb since %s", terrstr()); + return NULL; + } + + mDebug("sdb open successfully"); + return pSdb; } void sdbClose(SSdb *pSdb) { + mDebug("start to close sdb"); + + if (pSdb->curVer != pSdb->lastCommitVer) { + mDebug("start to write sdb file since curVer:% " PRId64 " and lastCommitVer:%" PRId64 " inequal", pSdb->curVer, + pSdb->lastCommitVer); + sdbWriteFile(pSdb); + } + if (pSdb->currDir != NULL) { tfree(pSdb->currDir); } @@ -79,10 +98,13 @@ void sdbClose(SSdb *pSdb) { for (int32_t i = 0; i < SDB_MAX; ++i) { SHashObj *hash = pSdb->hashObjs[i]; if (hash != NULL) { + taosHashClear(hash); taosHashCleanup(hash); } pSdb->hashObjs[i] = NULL; } + + mDebug("sdb is closed"); } void sdbSetTable(SSdb *pSdb, SSdbTable table) { @@ -94,4 +116,6 @@ void sdbSetTable(SSdb *pSdb, SSdbTable table) { pSdb->deployFps[sdb] = table.deployFp; pSdb->encodeFps[sdb] = table.encodeFp; pSdb->decodeFps[sdb] = table.decodeFp; + + mDebug("set sdb handle of table %d", pSdb, table); } \ No newline at end of file diff --git a/source/dnode/mnode/sdb/src/sdbFile.c b/source/dnode/mnode/sdb/src/sdbFile.c index 1764dff226..fbbb602882 100644 --- a/source/dnode/mnode/sdb/src/sdbFile.c +++ b/source/dnode/mnode/sdb/src/sdbFile.c @@ -15,71 +15,75 @@ #define _DEFAULT_SOURCE #include "sdbInt.h" -#include "tglobal.h" #include "tchecksum.h" -static int32_t sdbCreateDir() { - mDebug("start to create mnode at %s", tsMnodeDir); - - if (taosMkDir(tsSdb.currDir) != 0) { - terrno = TAOS_SYSTEM_ERROR(errno); - mError("failed to create dir:%s since %s", tsSdb.currDir, terrstr()); - return -1; +static int32_t sdbCreateDir(SSdb *pSdb) { + int32_t code = taosMkDir(pSdb->currDir); + if (code != 0) { + code = TAOS_SYSTEM_ERROR(errno); + mError("failed to create dir:%s since %s", pSdb->currDir, tstrerror(code)); + return code; } - if (taosMkDir(tsSdb.syncDir) != 0) { - terrno = TAOS_SYSTEM_ERROR(errno); - mError("failed to create dir:%s since %s", tsSdb.syncDir, terrstr()); - return -1; + code = taosMkDir(pSdb->syncDir); + if (code != 0) { + code = TAOS_SYSTEM_ERROR(errno); + mError("failed to create dir:%s since %s", pSdb->syncDir, tstrerror(code)); + return code; } - if (taosMkDir(tsSdb.tmpDir) != 0) { - terrno = TAOS_SYSTEM_ERROR(errno); - mError("failed to create dir:%s since %s", tsSdb.tmpDir, terrstr()); - return -1; + code = taosMkDir(pSdb->tmpDir); + if (code != 0) { + code = TAOS_SYSTEM_ERROR(errno); + mError("failed to create dir:%s since %s", pSdb->tmpDir, tstrerror(code)); + return code; } return 0; } -static int32_t sdbRunDeployFp() { - mDebug("start to run deploy functions"); +static int32_t sdbRunDeployFp(SSdb *pSdb) { + mDebug("start to run sdb deploy functions"); for (int32_t i = SDB_MAX - 1; i > SDB_START; --i) { - SdbDeployFp fp = tsSdb.deployFps[i]; + SdbDeployFp fp = pSdb->deployFps[i]; if (fp == NULL) continue; - if ((*fp)() != 0) { - mError("failed to deploy sdb:%d since %s", i, terrstr()); - return -1; + + int32_t code = (*fp)(pSdb); + if (code != 0) { + mError("failed to deploy sdb:%d since %s", i, tstrerror(code)); + return code; } } - mDebug("end of run deploy functions"); + mDebug("sdb deploy functions run finished"); return 0; } -static int32_t sdbReadDataFile() { - SSdbRaw *pRaw = malloc(SDB_MAX_SIZE); - if (pRaw == NULL) { - terrno = TSDB_CODE_OUT_OF_MEMORY; - return -1; - } - - char file[PATH_MAX] = {0}; - snprintf(file, sizeof(file), "%ssdb.data", tsSdb.currDir); - FileFd fd = taosOpenFileRead(file); - if (fd <= 0) { - free(pRaw); - terrno = TAOS_SYSTEM_ERROR(errno); - mError("failed to open file:%s for read since %s", file, terrstr()); - return -1; - } - +int32_t sdbReadFile(SSdb *pSdb) { int64_t offset = 0; int32_t code = 0; int32_t readLen = 0; int64_t ret = 0; + SSdbRaw *pRaw = malloc(SDB_MAX_SIZE); + if (pRaw == NULL) { + code = TSDB_CODE_OUT_OF_MEMORY; + mError("failed read file since %s", tstrerror(code)); + return code; + } + + char file[PATH_MAX] = {0}; + snprintf(file, sizeof(file), "%s%ssdb.data", pSdb->currDir, TD_DIRSEP); + + FileFd fd = taosOpenFileRead(file); + if (fd <= 0) { + free(pRaw); + code = TAOS_SYSTEM_ERROR(errno); + mError("failed to read file:%s since %s", file, tstrerror(code)); + return code; + } + while (1) { readLen = sizeof(SSdbRaw); ret = taosReadFile(fd, pRaw, readLen); @@ -118,7 +122,7 @@ static int32_t sdbReadDataFile() { break; } - code = sdbWriteImp(pRaw); + code = sdbWriteRaw(pSdb, pRaw); if (code != 0) { mError("failed to read file:%s since %s", file, terrstr()); goto PARSE_SDB_DATA_ERROR; @@ -130,29 +134,31 @@ static int32_t sdbReadDataFile() { PARSE_SDB_DATA_ERROR: taosCloseFile(fd); sdbFreeRaw(pRaw); - terrno = code; + return code; } -static int32_t sdbWriteDataFile() { +int32_t sdbWriteFile(SSdb *pSdb) { + int32_t code = 0; + char tmpfile[PATH_MAX] = {0}; - snprintf(tmpfile, sizeof(tmpfile), "%ssdb.data", tsSdb.tmpDir); + snprintf(tmpfile, sizeof(tmpfile), "%ssdb.data", pSdb->tmpDir); + char curfile[PATH_MAX] = {0}; + snprintf(curfile, sizeof(curfile), "%ssdb.data", pSdb->currDir); FileFd fd = taosOpenFileCreateWrite(tmpfile); if (fd <= 0) { - terrno = TAOS_SYSTEM_ERROR(errno); - mError("failed to open file:%s for write since %s", tmpfile, terrstr()); - return -1; + code = TAOS_SYSTEM_ERROR(errno); + mError("failed to open file:%s for write since %s", tmpfile, tstrerror(code)); + return code; } - int32_t code = 0; - for (int32_t i = SDB_MAX - 1; i > SDB_START; --i) { - SdbEncodeFp encodeFp = tsSdb.encodeFps[i]; + SdbEncodeFp encodeFp = pSdb->encodeFps[i]; if (encodeFp == NULL) continue; - SHashObj *hash = tsSdb.hashObjs[i]; - SRWLatch *pLock = &tsSdb.locks[i]; + SHashObj *hash = pSdb->hashObjs[i]; + SRWLatch *pLock = &pSdb->locks[i]; taosWLockLatch(pLock); SSdbRow **ppRow = taosHashIterate(hash, NULL); @@ -192,68 +198,46 @@ static int32_t sdbWriteDataFile() { if (code == 0) { code = taosFsyncFile(fd); + if (code != 0) { + code = TAOS_SYSTEM_ERROR(errno); + mError("failed to write file:%s since %s", tmpfile, tstrerror(code)); + } } taosCloseFile(fd); if (code == 0) { - char curfile[PATH_MAX] = {0}; - snprintf(curfile, sizeof(curfile), "%ssdb.data", tsSdb.currDir); code = taosRenameFile(tmpfile, curfile); + if (code != 0) { + code = TAOS_SYSTEM_ERROR(errno); + mError("failed to write file:%s since %s", curfile, tstrerror(code)); + } } if (code != 0) { - terrno = code; - mError("failed to write sdb file since %s", terrstr()); + mError("failed to write file:%s since %s", curfile, tstrerror(code)); } else { - mDebug("write sdb file successfully"); + mDebug("write file:%s successfully", curfile); } return code; } -// int32_t sdbOpen() { -// mDebug("start to read mnode file"); - -// if (sdbReadDataFile() != 0) { -// return -1; -// } - -// return 0; -// } - -// void sdbClose() { -// if (tsSdb.curVer != tsSdb.lastCommitVer) { -// mDebug("start to write mnode file"); -// sdbWriteDataFile(); -// } - -// for (int32_t i = 0; i < SDB_MAX; ++i) { -// SHashObj *hash = tsSdb.hashObjs[i]; -// if (hash != NULL) { -// taosHashClear(hash); -// } -// } -// } - -int32_t sdbDeploy() { - if (sdbCreateDir() != 0) { - return -1; +int32_t sdbDeploy(SSdb *pSdb) { + int32_t code = sdbCreateDir(pSdb); + if (code != 0) { + return code; } - if (sdbRunDeployFp() != 0) { - return -1; + code = sdbRunDeployFp(pSdb); + if (code != 0) { + return code; } - if (sdbWriteDataFile() != 0) { - return -1; + code = sdbWriteFile(pSdb); + if (code != 0) { + return code; } - // sdbClose(); return 0; } - -void sdbUnDeploy() { - mDebug("start to undeploy mnode"); - taosRemoveDir(tsMnodeDir); -} diff --git a/source/dnode/mnode/sdb/src/sdbHash.c b/source/dnode/mnode/sdb/src/sdbHash.c index 9c19e7f1a2..1ac983d538 100644 --- a/source/dnode/mnode/sdb/src/sdbHash.c +++ b/source/dnode/mnode/sdb/src/sdbHash.c @@ -15,15 +15,14 @@ #define _DEFAULT_SOURCE #include "sdbInt.h" -#include "tglobal.h" -static SHashObj *sdbGetHash(int32_t sdb) { - if (sdb >= SDB_MAX || sdb <= SDB_START) { +static SHashObj *sdbGetHash(SSdb *pSdb, int32_t type) { + if (type >= SDB_MAX || type <= SDB_START) { terrno = TSDB_CODE_SDB_INVALID_TABLE_TYPE; return NULL; } - SHashObj *hash = tsSdb.hashObjs[sdb]; + SHashObj *hash = pSdb->hashObjs[type]; if (hash == NULL) { terrno = TSDB_CODE_SDB_APP_ERROR; return NULL; @@ -32,9 +31,9 @@ static SHashObj *sdbGetHash(int32_t sdb) { return hash; } -static int32_t sdbGetkeySize(ESdbType sdb, void *pKey) { +static int32_t sdbGetkeySize(SSdb *pSdb, ESdbType type, void *pKey) { int32_t keySize; - EKeyType keyType = tsSdb.keyTypes[sdb]; + EKeyType keyType = pSdb->keyTypes[type]; if (keyType == SDB_KEY_INT32) { keySize = sizeof(int32_t); @@ -47,77 +46,81 @@ static int32_t sdbGetkeySize(ESdbType sdb, void *pKey) { return keySize; } -static int32_t sdbInsertRow(SHashObj *hash, SSdbRaw *pRaw, SSdbRow *pRow, int32_t keySize) { - SRWLatch *pLock = &tsSdb.locks[pRow->sdb]; +static int32_t sdbInsertRow(SSdb *pSdb, SHashObj *hash, SSdbRaw *pRaw, SSdbRow *pRow, int32_t keySize) { + int32_t code = 0; + + SRWLatch *pLock = &pSdb->locks[pRow->sdb]; taosWLockLatch(pLock); SSdbRow *pDstRow = taosHashGet(hash, pRow->pObj, keySize); if (pDstRow != NULL) { - terrno = TSDB_CODE_SDB_OBJ_ALREADY_THERE; taosWUnLockLatch(pLock); sdbFreeRow(pRow); - return -1; + return TSDB_CODE_SDB_OBJ_ALREADY_THERE; } pRow->refCount = 1; pRow->status = pRaw->status; if (taosHashPut(hash, pRow->pObj, keySize, &pRow, sizeof(void *)) != 0) { - terrno = TSDB_CODE_OUT_OF_MEMORY; taosWUnLockLatch(pLock); sdbFreeRow(pRow); - return -1; + return TSDB_CODE_SDB_OBJ_ALREADY_THERE; } taosWUnLockLatch(pLock); - SdbInsertFp insertFp = tsSdb.insertFps[pRow->sdb]; + SdbInsertFp insertFp = pSdb->insertFps[pRow->sdb]; if (insertFp != NULL) { - if ((*insertFp)(pRow->pObj) != 0) { + code = (*insertFp)(pSdb, pRow->pObj); + if (code != 0) { taosWLockLatch(pLock); taosHashRemove(hash, pRow->pObj, keySize); taosWUnLockLatch(pLock); sdbFreeRow(pRow); - return -1; + return code; } } return 0; } -static int32_t sdbUpdateRow(SHashObj *hash, SSdbRaw *pRaw, SSdbRow *pRow, int32_t keySize) { - SRWLatch *pLock = &tsSdb.locks[pRow->sdb]; +static int32_t sdbUpdateRow(SSdb *pSdb, SHashObj *hash, SSdbRaw *pRaw, SSdbRow *pRow, int32_t keySize) { + int32_t code = 0; + + SRWLatch *pLock = &pSdb->locks[pRow->sdb]; taosRLockLatch(pLock); SSdbRow **ppDstRow = taosHashGet(hash, pRow->pObj, keySize); if (ppDstRow == NULL || *ppDstRow == NULL) { taosRUnLockLatch(pLock); - return sdbInsertRow(hash, pRaw, pRow, keySize); + return sdbInsertRow(pSdb, hash, pRaw, pRow, keySize); } SSdbRow *pDstRow = *ppDstRow; pRow->status = pRaw->status; taosRUnLockLatch(pLock); - SdbUpdateFp updateFp = tsSdb.updateFps[pRow->sdb]; + SdbUpdateFp updateFp = pSdb->updateFps[pRow->sdb]; if (updateFp != NULL) { - (*updateFp)(pRow->pObj, pDstRow->pObj); + code = (*updateFp)(pSdb, pRow->pObj, pDstRow->pObj); } sdbFreeRow(pRow); - return 0; + return code; } -static int32_t sdbDeleteRow(SHashObj *hash, SSdbRaw *pRaw, SSdbRow *pRow, int32_t keySize) { - SRWLatch *pLock = &tsSdb.locks[pRow->sdb]; +static int32_t sdbDeleteRow(SSdb *pSdb, SHashObj *hash, SSdbRaw *pRaw, SSdbRow *pRow, int32_t keySize) { + int32_t code = 0; + + SRWLatch *pLock = &pSdb->locks[pRow->sdb]; taosWLockLatch(pLock); SSdbRow **ppDstRow = taosHashGet(hash, pRow->pObj, keySize); if (ppDstRow == NULL || *ppDstRow == NULL) { - terrno = TSDB_CODE_SDB_OBJ_NOT_THERE; taosWUnLockLatch(pLock); sdbFreeRow(pRow); - return -1; + return TSDB_CODE_SDB_OBJ_NOT_THERE; } SSdbRow *pDstRow = *ppDstRow; @@ -125,71 +128,67 @@ static int32_t sdbDeleteRow(SHashObj *hash, SSdbRaw *pRaw, SSdbRow *pRow, int32_ taosHashRemove(hash, pDstRow->pObj, keySize); taosWUnLockLatch(pLock); - SdbDeleteFp deleteFp = tsSdb.deleteFps[pDstRow->sdb]; + SdbDeleteFp deleteFp = pSdb->deleteFps[pDstRow->sdb]; if (deleteFp != NULL) { - (void)(*deleteFp)(pDstRow->pObj); + code = (*deleteFp)(pSdb, pDstRow->pObj); } - sdbRelease(pDstRow->pObj); + sdbRelease(pSdb, pDstRow->pObj); sdbFreeRow(pRow); - return 0; + return code; } -int32_t sdbWriteImp(SSdbRaw *pRaw) { - SHashObj *hash = sdbGetHash(pRaw->sdb); - if (hash == NULL) return -1; +int32_t sdbWriteRaw(SSdb *pSdb, SSdbRaw *pRaw) { + SHashObj *hash = sdbGetHash(pSdb, pRaw->sdb); + if (hash == NULL) return terrno; - SdbDecodeFp decodeFp = tsSdb.decodeFps[pRaw->sdb]; + SdbDecodeFp decodeFp = pSdb->decodeFps[pRaw->sdb]; SSdbRow *pRow = (*decodeFp)(pRaw); if (pRow == NULL) { - terrno = TSDB_CODE_SDB_INVALID_DATA_CONTENT; - return -1; + return terrno; } pRow->sdb = pRaw->sdb; - int32_t keySize = sdbGetkeySize(pRow->sdb, pRow->pObj); - int32_t code = -1; + int32_t keySize = sdbGetkeySize(pSdb, pRow->sdb, pRow->pObj); + int32_t code = TSDB_CODE_SDB_INVALID_ACTION_TYPE; switch (pRaw->status) { case SDB_STATUS_CREATING: - code = sdbInsertRow(hash, pRaw, pRow, keySize); + code = sdbInsertRow(pSdb, hash, pRaw, pRow, keySize); break; case SDB_STATUS_READY: case SDB_STATUS_DROPPING: - code = sdbUpdateRow(hash, pRaw, pRow, keySize); + code = sdbUpdateRow(pSdb, hash, pRaw, pRow, keySize); break; case SDB_STATUS_DROPPED: - code = sdbDeleteRow(hash, pRaw, pRow, keySize); - break; - default: - terrno = TSDB_CODE_SDB_INVALID_ACTION_TYPE; + code = sdbDeleteRow(pSdb, hash, pRaw, pRow, keySize); break; } return code; } -int32_t sdbWrite(SSdbRaw *pRaw) { - int32_t code = sdbWriteImp(pRaw); +int32_t sdbWrite(SSdb *pSdb, SSdbRaw *pRaw) { + int32_t code = sdbWriteRaw(pSdb, pRaw); sdbFreeRaw(pRaw); return code; } -void *sdbAcquire(ESdbType sdb, void *pKey) { - SHashObj *hash = sdbGetHash(sdb); +void *sdbAcquire(SSdb *pSdb, ESdbType type, void *pKey) { + SHashObj *hash = sdbGetHash(pSdb, type); if (hash == NULL) return NULL; void *pRet = NULL; - int32_t keySize = sdbGetkeySize(sdb, pKey); + int32_t keySize = sdbGetkeySize(pSdb, type, pKey); - SRWLatch *pLock = &tsSdb.locks[sdb]; + SRWLatch *pLock = &pSdb->locks[type]; taosRLockLatch(pLock); SSdbRow **ppRow = taosHashGet(hash, pKey, keySize); if (ppRow == NULL || *ppRow == NULL) { - terrno = TSDB_CODE_SDB_OBJ_NOT_THERE; taosRUnLockLatch(pLock); + terrno = TSDB_CODE_SDB_OBJ_NOT_THERE; return NULL; } @@ -214,13 +213,13 @@ void *sdbAcquire(ESdbType sdb, void *pKey) { return pRet; } -void sdbRelease(void *pObj) { +void sdbRelease(SSdb *pSdb, void *pObj) { if (pObj == NULL) return; SSdbRow *pRow = (SSdbRow *)((char *)pObj - sizeof(SSdbRow)); if (pRow->sdb >= SDB_MAX || pRow->sdb <= SDB_START) return; - SRWLatch *pLock = &tsSdb.locks[pRow->sdb]; + SRWLatch *pLock = &pSdb->locks[pRow->sdb]; taosRLockLatch(pLock); int32_t ref = atomic_sub_fetch_32(&pRow->refCount, 1); @@ -231,11 +230,11 @@ void sdbRelease(void *pObj) { taosRUnLockLatch(pLock); } -void *sdbFetch(ESdbType sdb, void *pIter, void **ppObj) { - SHashObj *hash = sdbGetHash(sdb); +void *sdbFetch(SSdb *pSdb, ESdbType type, void *pIter, void **ppObj) { + SHashObj *hash = sdbGetHash(pSdb, type); if (hash == NULL) return NULL; - SRWLatch *pLock = &tsSdb.locks[sdb]; + SRWLatch *pLock = &pSdb->locks[type]; taosRLockLatch(pLock); SSdbRow **ppRow = taosHashIterate(hash, ppRow); @@ -255,23 +254,23 @@ void *sdbFetch(ESdbType sdb, void *pIter, void **ppObj) { return ppRow; } -void sdbCancelFetch(void *pIter) { +void sdbCancelFetch(SSdb *pSdb, void *pIter) { if (pIter == NULL) return; SSdbRow *pRow = *(SSdbRow **)pIter; - SHashObj *hash = sdbGetHash(pRow->sdb); + SHashObj *hash = sdbGetHash(pSdb, pRow->sdb); if (hash == NULL) return; - SRWLatch *pLock = &tsSdb.locks[pRow->sdb]; + SRWLatch *pLock = &pSdb->locks[pRow->sdb]; taosRLockLatch(pLock); taosHashCancelIterate(hash, pIter); taosRUnLockLatch(pLock); } -int32_t sdbGetSize(ESdbType sdb) { - SHashObj *hash = sdbGetHash(sdb); +int32_t sdbGetSize(SSdb *pSdb, ESdbType type) { + SHashObj *hash = sdbGetHash(pSdb, type); if (hash == NULL) return 0; - SRWLatch *pLock = &tsSdb.locks[sdb]; + SRWLatch *pLock = &pSdb->locks[type]; taosRLockLatch(pLock); int32_t size = taosHashGetSize(hash); taosRUnLockLatch(pLock); diff --git a/source/dnode/mnode/sdb/src/sdbRow.c b/source/dnode/mnode/sdb/src/sdbRow.c index 68a0faa7b9..ec1dcf39e6 100644 --- a/source/dnode/mnode/sdb/src/sdbRow.c +++ b/source/dnode/mnode/sdb/src/sdbRow.c @@ -35,4 +35,4 @@ void *sdbGetRowObj(SSdbRow *pRow) { return pRow->pObj; } -void sdbFreeRow(SSdbRow *pRow) { free(pRow); } +void sdbFreeRow(SSdbRow *pRow) { tfree(pRow); } From c7451a9949e1775a90de264f1133e27aff79cace Mon Sep 17 00:00:00 2001 From: Shengliang Guan Date: Mon, 29 Nov 2021 16:02:37 +0800 Subject: [PATCH 41/44] TD-10431 rename variables --- source/dnode/mnode/impl/inc/mndTrans.h | 24 +++++------ source/dnode/mnode/impl/src/mndSync.c | 2 +- source/dnode/mnode/impl/src/mndTrans.c | 58 +++++++++++++------------- source/dnode/mnode/impl/src/mndUser.c | 20 ++++----- 4 files changed, 52 insertions(+), 52 deletions(-) diff --git a/source/dnode/mnode/impl/inc/mndTrans.h b/source/dnode/mnode/impl/inc/mndTrans.h index 0a6395c900..745c0a0447 100644 --- a/source/dnode/mnode/impl/inc/mndTrans.h +++ b/source/dnode/mnode/impl/inc/mndTrans.h @@ -25,20 +25,20 @@ extern "C" { int32_t mndInitTrans(SMnode *pMnode); void mndCleanupTrans(SMnode *pMnode); -STrans *trnCreate(SMnode *pMnode, ETrnPolicy policy, void *rpcHandle); -void trnDrop(STrans *pTrans); -int32_t trnAppendRedoLog(STrans *pTrans, SSdbRaw *pRaw); -int32_t trnAppendUndoLog(STrans *pTrans, SSdbRaw *pRaw); -int32_t trnAppendCommitLog(STrans *pTrans, SSdbRaw *pRaw); -int32_t trnAppendRedoAction(STrans *pTrans, SEpSet *, void *pMsg); -int32_t trnAppendUndoAction(STrans *pTrans, SEpSet *, void *pMsg); +STrans *mndTransCreate(SMnode *pMnode, ETrnPolicy policy, void *rpcHandle); +void mndTransDrop(STrans *pTrans); +int32_t mndTransAppendRedolog(STrans *pTrans, SSdbRaw *pRaw); +int32_t mndTransAppendUndolog(STrans *pTrans, SSdbRaw *pRaw); +int32_t mndTransAppendCommitlog(STrans *pTrans, SSdbRaw *pRaw); +int32_t mndTransAppendRedoAction(STrans *pTrans, SEpSet *, void *pMsg); +int32_t mndTransAppendUndoAction(STrans *pTrans, SEpSet *, void *pMsg); -int32_t trnPrepare(STrans *pTrans, int32_t (*syncfp)(SSdbRaw *pRaw, void *pData)); -int32_t trnApply(SMnode *pMnode, SSdbRaw *pRaw, void *pData, int32_t code); -int32_t trnExecute(SSdb *pSdb, int32_t tranId); +int32_t mndTransPrepare(STrans *pTrans, int32_t (*syncfp)(SSdbRaw *pRaw, void *pData)); +int32_t mndTransApply(SMnode *pMnode, SSdbRaw *pRaw, void *pData, int32_t code); +int32_t mndTransExecute(SSdb *pSdb, int32_t tranId); -SSdbRaw *trnActionEncode(STrans *pTrans); -SSdbRow *trnActionDecode(SSdbRaw *pRaw); +SSdbRaw *mndTransActionEncode(STrans *pTrans); +SSdbRow *mndTransActionDecode(SSdbRaw *pRaw); #ifdef __cplusplus } diff --git a/source/dnode/mnode/impl/src/mndSync.c b/source/dnode/mnode/impl/src/mndSync.c index 3535dcb677..7231915311 100644 --- a/source/dnode/mnode/impl/src/mndSync.c +++ b/source/dnode/mnode/impl/src/mndSync.c @@ -22,7 +22,7 @@ int32_t mndInitSync(SMnode *pMnode) { return 0; } void mndCleanupSync(SMnode *pMnode) {} int32_t mndSyncPropose(SMnode *pMnode, SSdbRaw *pRaw, void *pData) { - trnApply(pMnode, pData, pData, 0); + mndTransApply(pMnode, pData, pData, 0); free(pData); return 0; } diff --git a/source/dnode/mnode/impl/src/mndTrans.c b/source/dnode/mnode/impl/src/mndTrans.c index 168cb27757..91d0f20013 100644 --- a/source/dnode/mnode/impl/src/mndTrans.c +++ b/source/dnode/mnode/impl/src/mndTrans.c @@ -20,7 +20,7 @@ #define SDB_TRANS_VER 1 #define TRN_DEFAULT_ARRAY_SIZE 8 -SSdbRaw *trnActionEncode(STrans *pTrans) { +SSdbRaw *mndTransActionEncode(STrans *pTrans) { int32_t rawDataLen = 10 * sizeof(int32_t); int32_t redoLogNum = taosArrayGetSize(pTrans->redoLogs); int32_t undoLogNum = taosArrayGetSize(pTrans->undoLogs); @@ -84,7 +84,7 @@ SSdbRaw *trnActionEncode(STrans *pTrans) { return pRaw; } -SSdbRow *trnActionDecode(SSdbRaw *pRaw) { +SSdbRow *mndTransActionDecode(SSdbRaw *pRaw) { int8_t sver = 0; if (sdbGetRawSoftVer(pRaw, &sver) != 0) { mError("failed to get soft ver from raw:%p since %s", pRaw, terrstr()); @@ -150,7 +150,7 @@ SSdbRow *trnActionDecode(SSdbRaw *pRaw) { if (code != 0) { terrno = code; mError("trn:%d, failed to parse from raw:%p since %s", pTrans->id, pRaw, terrstr()); - trnDrop(pTrans); + mndTransDrop(pTrans); return NULL; } @@ -158,7 +158,7 @@ SSdbRow *trnActionDecode(SSdbRaw *pRaw) { return pRow; } -static int32_t trnActionInsert(SSdb *pSdb, STrans *pTrans) { +static int32_t mndTransActionInsert(SSdb *pSdb, STrans *pTrans) { SArray *pArray = pTrans->redoLogs; int32_t arraySize = taosArrayGetSize(pArray); @@ -175,7 +175,7 @@ static int32_t trnActionInsert(SSdb *pSdb, STrans *pTrans) { return 0; } -static int32_t trnActionDelete(SSdb *pSdb, STrans *pTrans) { +static int32_t mndTransActionDelete(SSdb *pSdb, STrans *pTrans) { SArray *pArray = pTrans->redoLogs; int32_t arraySize = taosArrayGetSize(pArray); @@ -192,7 +192,7 @@ static int32_t trnActionDelete(SSdb *pSdb, STrans *pTrans) { return 0; } -static int32_t trnActionUpdate(SSdb *pSdb, STrans *pTrans, STrans *pDstTrans) { +static int32_t mndTransActionUpdate(SSdb *pSdb, STrans *pTrans, STrans *pDstTrans) { assert(true); SArray *pArray = pTrans->redoLogs; int32_t arraySize = taosArrayGetSize(pArray); @@ -213,7 +213,7 @@ static int32_t trnActionUpdate(SSdb *pSdb, STrans *pTrans, STrans *pDstTrans) { static int32_t trnGenerateTransId() { return 1; } -STrans *trnCreate(SMnode *pMnode, ETrnPolicy policy, void *rpcHandle) { +STrans *mndTransCreate(SMnode *pMnode, ETrnPolicy policy, void *rpcHandle) { STrans *pTrans = calloc(1, sizeof(STrans)); if (pTrans == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; @@ -251,7 +251,7 @@ static void trnDropArray(SArray *pArray) { taosArrayDestroy(pArray); } -void trnDrop(STrans *pTrans) { +void mndTransDrop(STrans *pTrans) { trnDropArray(pTrans->redoLogs); trnDropArray(pTrans->undoLogs); trnDropArray(pTrans->commitLogs); @@ -262,12 +262,12 @@ void trnDrop(STrans *pTrans) { tfree(pTrans); } -void trnSetRpcHandle(STrans *pTrans, void *rpcHandle) { +void mndTransSetRpcHandle(STrans *pTrans, void *rpcHandle) { pTrans->rpcHandle = rpcHandle; mTrace("trn:%d, set rpc handle:%p", pTrans->id, rpcHandle); } -static int32_t trnAppendArray(SArray *pArray, SSdbRaw *pRaw) { +static int32_t mndTransAppendArray(SArray *pArray, SSdbRaw *pRaw) { if (pArray == NULL || pRaw == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; return -1; @@ -282,32 +282,32 @@ static int32_t trnAppendArray(SArray *pArray, SSdbRaw *pRaw) { return 0; } -int32_t trnAppendRedoLog(STrans *pTrans, SSdbRaw *pRaw) { - int32_t code = trnAppendArray(pTrans->redoLogs, pRaw); +int32_t mndTransAppendRedolog(STrans *pTrans, SSdbRaw *pRaw) { + int32_t code = mndTransAppendArray(pTrans->redoLogs, pRaw); mTrace("trn:%d, raw:%p append to redo logs, code:%d", pTrans->id, pRaw, code); return code; } -int32_t trnAppendUndoLog(STrans *pTrans, SSdbRaw *pRaw) { - int32_t code = trnAppendArray(pTrans->undoLogs, pRaw); +int32_t mndTransAppendUndolog(STrans *pTrans, SSdbRaw *pRaw) { + int32_t code = mndTransAppendArray(pTrans->undoLogs, pRaw); mTrace("trn:%d, raw:%p append to undo logs, code:%d", pTrans->id, pRaw, code); return code; } -int32_t trnAppendCommitLog(STrans *pTrans, SSdbRaw *pRaw) { - int32_t code = trnAppendArray(pTrans->commitLogs, pRaw); +int32_t mndTransAppendCommitlog(STrans *pTrans, SSdbRaw *pRaw) { + int32_t code = mndTransAppendArray(pTrans->commitLogs, pRaw); mTrace("trn:%d, raw:%p append to commit logs, code:%d", pTrans->id, pRaw, code); return code; } -int32_t trnAppendRedoAction(STrans *pTrans, SEpSet *pEpSet, void *pMsg) { - int32_t code = trnAppendArray(pTrans->redoActions, pMsg); +int32_t mndTransAppendRedoAction(STrans *pTrans, SEpSet *pEpSet, void *pMsg) { + int32_t code = mndTransAppendArray(pTrans->redoActions, pMsg); mTrace("trn:%d, msg:%p append to redo actions", pTrans->id, pMsg); return code; } -int32_t trnAppendUndoAction(STrans *pTrans, SEpSet *pEpSet, void *pMsg) { - int32_t code = trnAppendArray(pTrans->undoActions, pMsg); +int32_t mndTransAppendUndoAction(STrans *pTrans, SEpSet *pEpSet, void *pMsg) { + int32_t code = mndTransAppendArray(pTrans->undoActions, pMsg); mTrace("trn:%d, msg:%p append to undo actions", pTrans->id, pMsg); return code; } @@ -315,11 +315,11 @@ int32_t trnAppendUndoAction(STrans *pTrans, SEpSet *pEpSet, void *pMsg) { int32_t mndInitTrans(SMnode *pMnode) { SSdbTable table = {.sdbType = SDB_TRANS, .keyType = SDB_KEY_INT32, - .encodeFp = (SdbEncodeFp)trnActionEncode, - .decodeFp = (SdbDecodeFp)trnActionDecode, - .insertFp = (SdbInsertFp)trnActionInsert, - .updateFp = (SdbUpdateFp)trnActionUpdate, - .deleteFp = (SdbDeleteFp)trnActionDelete}; + .encodeFp = (SdbEncodeFp)mndTransActionEncode, + .decodeFp = (SdbDecodeFp)mndTransActionDecode, + .insertFp = (SdbInsertFp)mndTransActionInsert, + .updateFp = (SdbUpdateFp)mndTransActionUpdate, + .deleteFp = (SdbDeleteFp)mndTransActionDelete}; sdbSetTable(pMnode->pSdb, table); mInfo("trn module is initialized"); @@ -329,10 +329,10 @@ int32_t mndInitTrans(SMnode *pMnode) { void mndCleanupTrans(SMnode *pMnode) { mInfo("trn module is cleaned up"); } -int32_t trnPrepare(STrans *pTrans, int32_t (*syncfp)(SSdbRaw *pRaw, void *pData)) { +int32_t mndTransPrepare(STrans *pTrans, int32_t (*syncfp)(SSdbRaw *pRaw, void *pData)) { if (syncfp == NULL) return -1; - SSdbRaw *pRaw = trnActionEncode(pTrans); + SSdbRaw *pRaw = mndTransActionEncode(pTrans); if (pRaw == NULL) { mError("trn:%d, failed to decode trans since %s", pTrans->id, terrstr()); return -1; @@ -359,7 +359,7 @@ static void trnSendRpcRsp(void *rpcHandle, int32_t code) { } } -int32_t trnApply(SMnode *pMnode, SSdbRaw *pRaw, void *pData, int32_t code) { +int32_t mndTransApply(SMnode *pMnode, SSdbRaw *pRaw, void *pData, int32_t code) { if (code != 0) { trnSendRpcRsp(pData, terrno); return 0; @@ -454,7 +454,7 @@ static int32_t trnPerformRetryStage(STrans *pTrans) { } } -int32_t trnExecute(SSdb *pSdb, int32_t tranId) { +int32_t mndTransExecute(SSdb *pSdb, int32_t tranId) { int32_t code = 0; STrans *pTrans = sdbAcquire(pSdb, SDB_TRANS, &tranId); diff --git a/source/dnode/mnode/impl/src/mndUser.c b/source/dnode/mnode/impl/src/mndUser.c index 95c1a75ced..b762a5cddf 100644 --- a/source/dnode/mnode/impl/src/mndUser.c +++ b/source/dnode/mnode/impl/src/mndUser.c @@ -137,39 +137,39 @@ static int32_t mndCreateUser(SMnode *pMnode, char *acct, char *user, char *pass, userObj.updateTime = userObj.createdTime; userObj.rootAuth = 0; - STrans *pTrans = trnCreate(pMnode, TRN_POLICY_ROLLBACK, pMsg->rpcMsg.handle); + STrans *pTrans = mndTransCreate(pMnode, TRN_POLICY_ROLLBACK, pMsg->rpcMsg.handle); if (pTrans == NULL) return -1; SSdbRaw *pRedoRaw = mndUserActionEncode(&userObj); - if (pRedoRaw == NULL || trnAppendRedoLog(pTrans, pRedoRaw) != 0) { + if (pRedoRaw == NULL || mndTransAppendRedolog(pTrans, pRedoRaw) != 0) { mError("failed to append redo log since %s", terrstr()); - trnDrop(pTrans); + mndTransDrop(pTrans); return -1; } sdbSetRawStatus(pRedoRaw, SDB_STATUS_CREATING); SSdbRaw *pUndoRaw = mndUserActionEncode(&userObj); - if (pUndoRaw == NULL || trnAppendUndoLog(pTrans, pUndoRaw) != 0) { + if (pUndoRaw == NULL || mndTransAppendUndolog(pTrans, pUndoRaw) != 0) { mError("failed to append undo log since %s", terrstr()); - trnDrop(pTrans); + mndTransDrop(pTrans); return -1; } sdbSetRawStatus(pUndoRaw, SDB_STATUS_DROPPED); SSdbRaw *pCommitRaw = mndUserActionEncode(&userObj); - if (pCommitRaw == NULL || trnAppendCommitLog(pTrans, pCommitRaw) != 0) { + if (pCommitRaw == NULL || mndTransAppendCommitlog(pTrans, pCommitRaw) != 0) { mError("failed to append commit log since %s", terrstr()); - trnDrop(pTrans); + mndTransDrop(pTrans); return -1; } sdbSetRawStatus(pCommitRaw, SDB_STATUS_READY); - if (trnPrepare(pTrans, mndSyncPropose) != 0) { - trnDrop(pTrans); + if (mndTransPrepare(pTrans, mndSyncPropose) != 0) { + mndTransDrop(pTrans); return -1; } - trnDrop(pTrans); + mndTransDrop(pTrans); return 0; } From 67814005086eef58b9a3f453e9acd63aa25520ea Mon Sep 17 00:00:00 2001 From: Shengliang Guan Date: Mon, 29 Nov 2021 19:35:42 +0800 Subject: [PATCH 42/44] TD-10431 fix bug in sdb --- include/dnode/mnode/sdb/sdb.h | 15 +-- source/dnode/mgmt/impl/src/dndDnode.c | 2 +- source/dnode/mnode/impl/inc/mndInt.h | 4 +- source/dnode/mnode/impl/src/mndAcct.c | 3 +- source/dnode/mnode/impl/src/mndTrans.c | 7 +- source/dnode/mnode/impl/src/mndUser.c | 3 +- source/dnode/mnode/impl/src/mnode.c | 150 ++++++++++++++++--------- source/dnode/mnode/sdb/inc/sdbInt.h | 5 +- source/dnode/mnode/sdb/src/sdb.c | 73 ++++++------ source/dnode/mnode/sdb/src/sdbFile.c | 77 ++++++------- source/dnode/mnode/sdb/src/sdbHash.c | 28 ++--- source/dnode/mnode/sdb/src/sdbRaw.c | 4 +- 12 files changed, 200 insertions(+), 171 deletions(-) diff --git a/include/dnode/mnode/sdb/sdb.h b/include/dnode/mnode/sdb/sdb.h index 2c2d24268e..e5667e352f 100644 --- a/include/dnode/mnode/sdb/sdb.h +++ b/include/dnode/mnode/sdb/sdb.h @@ -114,14 +114,14 @@ typedef enum { SDB_START = 0, SDB_TRANS = 1, SDB_CLUSTER = 2, - SDB_DNODE = 3, - SDB_MNODE = 4, + SDB_MNODE = 3, + SDB_DNODE = 4, SDB_USER = 5, SDB_AUTH = 6, SDB_ACCT = 7, - SDB_DB = 8, SDB_VGROUP = 9, - SDB_STABLE = 10, + SDB_STABLE = 9, + SDB_DB = 10, SDB_FUNC = 11, SDB_MAX = 12 } ESdbType; @@ -149,10 +149,11 @@ typedef struct SSdbOpt { const char *path; } SSdbOpt; -SSdb *sdbOpen(SSdbOpt *pOption); -void sdbClose(SSdb *pSdb); +SSdb *sdbInit(SSdbOpt *pOption); +void sdbCleanup(SSdb *pSdb); +int32_t sdbSetTable(SSdb *pSdb, SSdbTable table); int32_t sdbDeploy(SSdb *pSdb); -void sdbSetTable(SSdb *pSdb, SSdbTable table); +int32_t sdbReadFile(SSdb *pSdb); int32_t sdbWrite(SSdb *pSdb, SSdbRaw *pRaw); void *sdbAcquire(SSdb *pSdb, ESdbType type, void *pKey); diff --git a/source/dnode/mgmt/impl/src/dndDnode.c b/source/dnode/mgmt/impl/src/dndDnode.c index 9e70bf1ae9..4265c8a3cd 100644 --- a/source/dnode/mgmt/impl/src/dndDnode.c +++ b/source/dnode/mgmt/impl/src/dndDnode.c @@ -464,7 +464,7 @@ static void *dnodeThreadRoutine(void *param) { pthread_testcancel(); if (dndGetStat(pDnode) == DND_STAT_RUNNING) { - dndSendStatusMsg(pDnode); + // dndSendStatusMsg(pDnode); } } } diff --git a/source/dnode/mnode/impl/inc/mndInt.h b/source/dnode/mnode/impl/inc/mndInt.h index 8a870091d4..2c7e597774 100644 --- a/source/dnode/mnode/impl/inc/mndInt.h +++ b/source/dnode/mnode/impl/inc/mndInt.h @@ -41,10 +41,10 @@ typedef struct SMnode { int8_t selfIndex; SReplica replicas[TSDB_MAX_REPLICA]; tmr_h timer; + char *path; SSdb *pSdb; SDnode *pDnode; - char *path; - SArray steps; + SArray *pSteps; MndMsgFp msgFp[TSDB_MSG_TYPE_MAX]; SendMsgToDnodeFp sendMsgToDnodeFp; SendMsgToMnodeFp sendMsgToMnodeFp; diff --git a/source/dnode/mnode/impl/src/mndAcct.c b/source/dnode/mnode/impl/src/mndAcct.c index b52c7d1e8e..dd91de1c95 100644 --- a/source/dnode/mnode/impl/src/mndAcct.c +++ b/source/dnode/mnode/impl/src/mndAcct.c @@ -111,8 +111,7 @@ int32_t mndInitAcct(SMnode *pMnode) { .updateFp = (SdbUpdateFp)mnodeAcctActionUpdate, .deleteFp = (SdbDeleteFp)mnodeAcctActionDelete}; - sdbSetTable(pMnode->pSdb, table); - return 0; + return sdbSetTable(pMnode->pSdb, table); } void mndCleanupAcct(SMnode *pMnode) {} diff --git a/source/dnode/mnode/impl/src/mndTrans.c b/source/dnode/mnode/impl/src/mndTrans.c index 91d0f20013..9333fe3f57 100644 --- a/source/dnode/mnode/impl/src/mndTrans.c +++ b/source/dnode/mnode/impl/src/mndTrans.c @@ -320,14 +320,11 @@ int32_t mndInitTrans(SMnode *pMnode) { .insertFp = (SdbInsertFp)mndTransActionInsert, .updateFp = (SdbUpdateFp)mndTransActionUpdate, .deleteFp = (SdbDeleteFp)mndTransActionDelete}; - sdbSetTable(pMnode->pSdb, table); - mInfo("trn module is initialized"); - return 0; + return sdbSetTable(pMnode->pSdb, table); } -void mndCleanupTrans(SMnode *pMnode) { mInfo("trn module is cleaned up"); } - +void mndCleanupTrans(SMnode *pMnode) {} int32_t mndTransPrepare(STrans *pTrans, int32_t (*syncfp)(SSdbRaw *pRaw, void *pData)) { if (syncfp == NULL) return -1; diff --git a/source/dnode/mnode/impl/src/mndUser.c b/source/dnode/mnode/impl/src/mndUser.c index b762a5cddf..d63e3662e0 100644 --- a/source/dnode/mnode/impl/src/mndUser.c +++ b/source/dnode/mnode/impl/src/mndUser.c @@ -223,11 +223,10 @@ int32_t mndInitUser(SMnode *pMnode) { .insertFp = (SdbInsertFp)mndUserActionInsert, .updateFp = (SdbUpdateFp)mndUserActionUpdate, .deleteFp = (SdbDeleteFp)mndUserActionDelete}; - sdbSetTable(pMnode->pSdb, table); mndSetMsgHandle(pMnode, TSDB_MSG_TYPE_CREATE_USER, mndProcessCreateUserMsg); - return 0; + return sdbSetTable(pMnode->pSdb, table); } void mndCleanupUser(SMnode *pMnode) {} \ No newline at end of file diff --git a/source/dnode/mnode/impl/src/mnode.c b/source/dnode/mnode/impl/src/mnode.c index 9217fa7194..bc2ed92197 100644 --- a/source/dnode/mnode/impl/src/mnode.c +++ b/source/dnode/mnode/impl/src/mnode.c @@ -36,6 +36,7 @@ int32_t mndGetDnodeId(SMnode *pMnode) { if (pMnode != NULL) { return pMnode->dnodeId; } + return -1; } @@ -43,6 +44,7 @@ int64_t mndGetClusterId(SMnode *pMnode) { if (pMnode != NULL) { return pMnode->clusterId; } + return -1; } @@ -50,6 +52,8 @@ tmr_h mndGetTimer(SMnode *pMnode) { if (pMnode != NULL) { return pMnode->timer; } + + return NULL; } void mndSendMsgToDnode(SMnode *pMnode, SEpSet *pEpSet, SRpcMsg *pMsg) { @@ -76,6 +80,7 @@ static int32_t mndInitTimer(SMnode *pMnode) { } if (pMnode->timer == NULL) { + terrno = TSDB_CODE_OUT_OF_MEMORY; return -1; } @@ -93,12 +98,12 @@ static int32_t mnodeCreateDir(SMnode *pMnode, const char *path) { pMnode->path = strdup(path); if (pMnode->path == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; - return terrno; + return -1; } if (taosMkDir(pMnode->path) != 0) { terrno = TAOS_SYSTEM_ERROR(errno); - return terrno; + return -1; } return 0; @@ -108,7 +113,7 @@ static int32_t mndInitSdb(SMnode *pMnode) { SSdbOpt opt = {0}; opt.path = pMnode->path; - pMnode->pSdb = sdbOpen(&opt); + pMnode->pSdb = sdbInit(&opt); if (pMnode->pSdb == NULL) { return -1; } @@ -117,10 +122,11 @@ static int32_t mndInitSdb(SMnode *pMnode) { } static int32_t mndDeploySdb(SMnode *pMnode) { return sdbDeploy(pMnode->pSdb); } +static int32_t mndReadSdb(SMnode *pMnode) { return sdbReadFile(pMnode->pSdb); } static void mndCleanupSdb(SMnode *pMnode) { if (pMnode->pSdb) { - sdbClose(pMnode->pSdb); + sdbCleanup(pMnode->pSdb); pMnode->pSdb = NULL; } } @@ -130,9 +136,8 @@ static int32_t mndAllocStep(SMnode *pMnode, char *name, MndInitFp initFp, MndCle step.name = name; step.initFp = initFp; step.cleanupFp = cleanupFp; - if (taosArrayPush(&pMnode->steps, &step) != NULL) { + if (taosArrayPush(pMnode->pSteps, &step) == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; - mError("failed to alloc step:%s since %s", name, terrstr()); return -1; } @@ -140,68 +145,73 @@ static int32_t mndAllocStep(SMnode *pMnode, char *name, MndInitFp initFp, MndCle } static int32_t mndInitSteps(SMnode *pMnode) { - if (mndAllocStep(pMnode, "mnode-trans", mndInitTrans, mndCleanupTrans) != 0) return terrno; - if (mndAllocStep(pMnode, "mnode-cluster", mndInitCluster, mndCleanupCluster) != 0) return terrno; - if (mndAllocStep(pMnode, "mnode-dnode", mndInitDnode, mndCleanupDnode) != 0) return terrno; - if (mndAllocStep(pMnode, "mnode-mnode", mndInitMnode, mndCleanupMnode) != 0) return terrno; - if (mndAllocStep(pMnode, "mnode-acct", mndInitAcct, mndCleanupAcct) != 0) return terrno; - if (mndAllocStep(pMnode, "mnode-auth", mndInitAuth, mndCleanupAuth) != 0) return terrno; - if (mndAllocStep(pMnode, "mnode-user", mndInitUser, mndCleanupUser) != 0) return terrno; - if (mndAllocStep(pMnode, "mnode-db", mndInitDb, mndCleanupDb) != 0) return terrno; - if (mndAllocStep(pMnode, "mnode-vgroup", mndInitVgroup, mndCleanupVgroup) != 0) return terrno; - if (mndAllocStep(pMnode, "mnode-stable", mndInitStable, mndCleanupStable) != 0) return terrno; - if (mndAllocStep(pMnode, "mnode-func", mndInitFunc, mndCleanupFunc) != 0) return terrno; - if (mndAllocStep(pMnode, "mnode-sdb", mndInitSdb, mndCleanupSdb) != 0) return terrno; + if (mndAllocStep(pMnode, "mnode-sdb", mndInitSdb, mndCleanupSdb) != 0) return -1; + if (mndAllocStep(pMnode, "mnode-trans", mndInitTrans, mndCleanupTrans) != 0) return -1; + if (mndAllocStep(pMnode, "mnode-cluster", mndInitCluster, mndCleanupCluster) != 0) return -1; + if (mndAllocStep(pMnode, "mnode-dnode", mndInitDnode, mndCleanupDnode) != 0) return -1; + if (mndAllocStep(pMnode, "mnode-mnode", mndInitMnode, mndCleanupMnode) != 0) return -1; + if (mndAllocStep(pMnode, "mnode-acct", mndInitAcct, mndCleanupAcct) != 0) return -1; + if (mndAllocStep(pMnode, "mnode-auth", mndInitAuth, mndCleanupAuth) != 0) return -1; + if (mndAllocStep(pMnode, "mnode-user", mndInitUser, mndCleanupUser) != 0) return -1; + if (mndAllocStep(pMnode, "mnode-db", mndInitDb, mndCleanupDb) != 0) return -1; + if (mndAllocStep(pMnode, "mnode-vgroup", mndInitVgroup, mndCleanupVgroup) != 0) return -1; + if (mndAllocStep(pMnode, "mnode-stable", mndInitStable, mndCleanupStable) != 0) return -1; + if (mndAllocStep(pMnode, "mnode-func", mndInitFunc, mndCleanupFunc) != 0) return -1; if (pMnode->clusterId <= 0) { - if (mndAllocStep(pMnode, "mnode-deploy", mndDeploySdb, NULL) != 0) return terrno; + if (mndAllocStep(pMnode, "mnode-sdb-deploy", mndDeploySdb, NULL) != 0) return -1; + } else { + if (mndAllocStep(pMnode, "mnode-sdb-read", mndReadSdb, NULL) != 0) return -1; } - if (mndAllocStep(pMnode, "mnode-timer", mndInitTimer, NULL) != 0) return terrno; - if (mndAllocStep(pMnode, "mnode-balance", mndInitBalance, mndCleanupBalance) != 0) return terrno; - if (mndAllocStep(pMnode, "mnode-profile", mndInitProfile, mndCleanupProfile) != 0) return terrno; - if (mndAllocStep(pMnode, "mnode-show", mndInitShow, mndCleanupShow) != 0) return terrno; - if (mndAllocStep(pMnode, "mnode-sync", mndInitSync, mndCleanupSync) != 0) return terrno; - if (mndAllocStep(pMnode, "mnode-telem", mndInitTelem, mndCleanupTelem) != 0) return terrno; - if (mndAllocStep(pMnode, "mnode-timer", NULL, mndCleanupTimer) != 0) return terrno; + if (mndAllocStep(pMnode, "mnode-timer", mndInitTimer, NULL) != 0) return -1; + if (mndAllocStep(pMnode, "mnode-balance", mndInitBalance, mndCleanupBalance) != 0) return -1; + if (mndAllocStep(pMnode, "mnode-profile", mndInitProfile, mndCleanupProfile) != 0) return -1; + if (mndAllocStep(pMnode, "mnode-show", mndInitShow, mndCleanupShow) != 0) return -1; + if (mndAllocStep(pMnode, "mnode-sync", mndInitSync, mndCleanupSync) != 0) return -1; + if (mndAllocStep(pMnode, "mnode-telem", mndInitTelem, mndCleanupTelem) != 0) return -1; + if (mndAllocStep(pMnode, "mnode-timer", NULL, mndCleanupTimer) != 0) return -1; return 0; } static void mndCleanupSteps(SMnode *pMnode, int32_t pos) { + if (pMnode->pSteps == NULL) return; + if (pos == -1) { - pos = taosArrayGetSize(&pMnode->steps); + pos = taosArrayGetSize(pMnode->pSteps); } for (int32_t s = pos; s >= 0; s--) { - SMnodeStep *pStep = taosArrayGet(&pMnode->steps, pos); + SMnodeStep *pStep = taosArrayGet(pMnode->pSteps, pos); mDebug("step:%s will cleanup", pStep->name); if (pStep->cleanupFp != NULL) { (*pStep->cleanupFp)(pMnode); } } - taosArrayClear(&pMnode->steps); + taosArrayClear(pMnode->pSteps); + pMnode->pSteps = NULL; } static int32_t mndExecSteps(SMnode *pMnode) { - int32_t size = taosArrayGetSize(&pMnode->steps); + int32_t size = taosArrayGetSize(pMnode->pSteps); for (int32_t pos = 0; pos < size; pos++) { - SMnodeStep *pStep = taosArrayGet(&pMnode->steps, pos); + SMnodeStep *pStep = taosArrayGet(pMnode->pSteps, pos); if (pStep->initFp == NULL) continue; // (*pMnode->reportProgress)(pStep->name, "start initialize"); - int32_t code = (*pStep->initFp)(pMnode); - if (code != 0) { - mError("step:%s exec failed since %s, start to cleanup", pStep->name, tstrerror(code)); + if ((*pStep->initFp)(pMnode) != 0) { + mError("step:%s exec failed since %s, start to cleanup", pStep->name, terrstr()); mndCleanupSteps(pMnode, pos); - terrno = code; - return code; + return -1; } else { mDebug("step:%s is initialized", pStep->name); } // (*pMnode->reportProgress)(pStep->name, "initialize completed"); } + + return 0; } static int32_t mndSetOptions(SMnode *pMnode, const SMnodeOpt *pOption) { @@ -217,20 +227,40 @@ static int32_t mndSetOptions(SMnode *pMnode, const SMnodeOpt *pOption) { pMnode->sendRedirectMsgFp = pOption->sendRedirectMsgFp; if (pMnode->sendMsgToDnodeFp == NULL || pMnode->sendMsgToMnodeFp == NULL || pMnode->sendRedirectMsgFp == NULL || - pMnode->putMsgToApplyMsgFp == NULL || pMnode->dnodeId < 0 || pMnode->clusterId < 0) { + pMnode->putMsgToApplyMsgFp == NULL) { terrno = TSDB_CODE_MND_APP_ERROR; - return terrno; + return -1; + } + + if (pMnode->dnodeId < 0 || pMnode->clusterId < 0) { + terrno = TSDB_CODE_MND_APP_ERROR; + return -1; } return 0; } SMnode *mndOpen(const char *path, const SMnodeOpt *pOption) { + mDebug("start to open mnode in %s", path); + SMnode *pMnode = calloc(1, sizeof(SMnode)); + if (pMnode == NULL) { + terrno = TSDB_CODE_OUT_OF_MEMORY; + mError("failed to open mnode since %s", terrstr()); + return NULL; + } + + pMnode->pSteps = taosArrayInit(24, sizeof(SMnodeStep)); + if (pMnode->pSteps == NULL) { + free(pMnode); + terrno = TSDB_CODE_OUT_OF_MEMORY; + mError("failed to open mnode since %s", terrstr()); + return NULL; + } int32_t code = mnodeCreateDir(pMnode, path); - if (code != 0) { - mError("failed to set mnode options since %s", terrstr()); + if (mnodeCreateDir(pMnode, path) != 0) { + mError("failed to open mnode since %s", tstrerror(code)); mndClose(pMnode); terrno = code; return NULL; @@ -238,51 +268,66 @@ SMnode *mndOpen(const char *path, const SMnodeOpt *pOption) { code = mndSetOptions(pMnode, pOption); if (code != 0) { + mError("failed to open mnode since %s", tstrerror(code)); mndClose(pMnode); terrno = code; - mError("failed to set mnode options since %s", terrstr()); return NULL; } code = mndInitSteps(pMnode); if (code != 0) { + mError("failed to open mnode since %s", tstrerror(code)); mndClose(pMnode); terrno = code; - mError("failed to int steps since %s", terrstr()); return NULL; } code = mndExecSteps(pMnode); if (code != 0) { + mError("failed to open mnode since %s", tstrerror(code)); mndClose(pMnode); terrno = code; - mError("failed to execute steps since %s", terrstr()); return NULL; } - mDebug("mnode:%p object is created", pMnode); + mDebug("mnode open successfully "); return pMnode; } void mndClose(SMnode *pMnode) { - mndCleanupSteps(pMnode, -1); - tfree(pMnode->path); - tfree(pMnode); - mDebug("mnode:%p object is cleaned up", pMnode); + if (pMnode != NULL) { + mDebug("start to close mnode"); + mndCleanupSteps(pMnode, -1); + tfree(pMnode->path); + tfree(pMnode); + mDebug("mnode is closed"); + } } int32_t mndAlter(SMnode *pMnode, const SMnodeOpt *pOption) { - assert(1); + mDebug("start to alter mnode"); + mDebug("mnode is altered"); return 0; } void mndDestroy(const char *path) { - mDebug("mnode in %s will be destroyed", path); + mDebug("start to destroy mnode at %s", path); taosRemoveDir(path); + mDebug("mnode is destroyed"); } int32_t mndGetLoad(SMnode *pMnode, SMnodeLoad *pLoad) { - assert(1); + pLoad->numOfDnode = 0; + pLoad->numOfMnode = 0; + pLoad->numOfVgroup = 0; + pLoad->numOfDatabase = 0; + pLoad->numOfSuperTable = 0; + pLoad->numOfChildTable = 0; + pLoad->numOfColumn = 0; + pLoad->totalPoints = 0; + pLoad->totalStorage = 0; + pLoad->compStorage = 0; + return 0; } @@ -343,7 +388,8 @@ static void mndProcessRpcMsg(SMnodeMsg *pMsg) { code = (*fp)(pMnode, pMsg); if (code != 0) { - mError("msg:%p, app:%p type:%s failed to process since %s", pMsg, ahandle, taosMsg[msgType], tstrerror(code)); + code = terrno; + mError("msg:%p, app:%p type:%s failed to process since %s", pMsg, ahandle, taosMsg[msgType], terrstr()); goto PROCESS_RPC_END; } diff --git a/source/dnode/mnode/sdb/inc/sdbInt.h b/source/dnode/mnode/sdb/inc/sdbInt.h index 5c25349b5c..6162617568 100644 --- a/source/dnode/mnode/sdb/inc/sdbInt.h +++ b/source/dnode/mnode/sdb/inc/sdbInt.h @@ -37,7 +37,7 @@ extern "C" { #define SDB_MAX_SIZE (32 * 1024) typedef struct SSdbRaw { - int8_t sdb; + int8_t type; int8_t sver; int8_t status; int8_t reserved; @@ -46,7 +46,7 @@ typedef struct SSdbRaw { } SSdbRaw; typedef struct SSdbRow { - ESdbType sdb; + ESdbType type; ESdbStatus status; int32_t refCount; char pObj[]; @@ -69,7 +69,6 @@ typedef struct SSdb { SdbDecodeFp decodeFps[SDB_MAX]; } SSdb; -int32_t sdbReadFile(SSdb *pSdb); int32_t sdbWriteFile(SSdb *pSdb); int32_t sdbWriteRaw(SSdb *pSdb, SSdbRaw *pRaw); diff --git a/source/dnode/mnode/sdb/src/sdb.c b/source/dnode/mnode/sdb/src/sdb.c index c4c6a553f7..4f1267498c 100644 --- a/source/dnode/mnode/sdb/src/sdb.c +++ b/source/dnode/mnode/sdb/src/sdb.c @@ -16,70 +16,43 @@ #define _DEFAULT_SOURCE #include "sdbInt.h" -SSdb *sdbOpen(SSdbOpt *pOption) { - mDebug("start to open sdb in %s", pOption->path); +SSdb *sdbInit(SSdbOpt *pOption) { + mDebug("start to init sdb in %s", pOption->path); SSdb *pSdb = calloc(1, sizeof(SSdb)); if (pSdb == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; - mError("failed to open sdb since %s", terrstr()); + mError("failed to init sdb since %s", terrstr()); return NULL; } char path[PATH_MAX + 100]; - snprintf(path, PATH_MAX + 100, "%s%scur", pOption->path, TD_DIRSEP); + snprintf(path, PATH_MAX + 100, "%s", pOption->path); pSdb->currDir = strdup(path); snprintf(path, PATH_MAX + 100, "%s%ssync", pOption->path, TD_DIRSEP); pSdb->syncDir = strdup(path); snprintf(path, PATH_MAX + 100, "%s%stmp", pOption->path, TD_DIRSEP); pSdb->tmpDir = strdup(path); if (pSdb->currDir == NULL || pSdb->currDir == NULL || pSdb->currDir == NULL) { - sdbClose(pSdb); + sdbCleanup(pSdb); terrno = TSDB_CODE_OUT_OF_MEMORY; - mError("failed to open sdb since %s", terrstr()); + mError("failed to init sdb since %s", terrstr()); return NULL; } for (int32_t i = 0; i < SDB_MAX; ++i) { - int32_t type; - if (pSdb->keyTypes[i] == SDB_KEY_INT32) { - type = TSDB_DATA_TYPE_INT; - } else if (pSdb->keyTypes[i] == SDB_KEY_INT64) { - type = TSDB_DATA_TYPE_BIGINT; - } else { - type = TSDB_DATA_TYPE_BINARY; - } - - SHashObj *hash = taosHashInit(64, taosGetDefaultHashFunction(type), true, HASH_NO_LOCK); - if (hash == NULL) { - sdbClose(pSdb); - terrno = TSDB_CODE_OUT_OF_MEMORY; - mError("failed to open sdb since %s", terrstr()); - return NULL; - } - - pSdb->hashObjs[i] = hash; taosInitRWLatch(&pSdb->locks[i]); } - int32_t code = sdbReadFile(pSdb); - if (code != 0) { - sdbClose(pSdb); - terrno = code; - mError("failed to open sdb since %s", terrstr()); - return NULL; - } - - mDebug("sdb open successfully"); + mDebug("sdb init successfully"); return pSdb; } -void sdbClose(SSdb *pSdb) { - mDebug("start to close sdb"); +void sdbCleanup(SSdb *pSdb) { + mDebug("start to cleanup sdb"); if (pSdb->curVer != pSdb->lastCommitVer) { - mDebug("start to write sdb file since curVer:% " PRId64 " and lastCommitVer:%" PRId64 " inequal", pSdb->curVer, - pSdb->lastCommitVer); + mDebug("write sdb file for curVer:% " PRId64 " and lastVer:%" PRId64, pSdb->curVer, pSdb->lastCommitVer); sdbWriteFile(pSdb); } @@ -104,10 +77,10 @@ void sdbClose(SSdb *pSdb) { pSdb->hashObjs[i] = NULL; } - mDebug("sdb is closed"); + mDebug("sdb is cleaned up"); } -void sdbSetTable(SSdb *pSdb, SSdbTable table) { +int32_t sdbSetTable(SSdb *pSdb, SSdbTable table) { ESdbType sdb = table.sdbType; pSdb->keyTypes[sdb] = table.keyType; pSdb->insertFps[sdb] = table.insertFp; @@ -117,5 +90,25 @@ void sdbSetTable(SSdb *pSdb, SSdbTable table) { pSdb->encodeFps[sdb] = table.encodeFp; pSdb->decodeFps[sdb] = table.decodeFp; - mDebug("set sdb handle of table %d", pSdb, table); + for (int32_t i = 0; i < SDB_MAX; ++i) { + int32_t type; + if (pSdb->keyTypes[i] == SDB_KEY_INT32) { + type = TSDB_DATA_TYPE_INT; + } else if (pSdb->keyTypes[i] == SDB_KEY_INT64) { + type = TSDB_DATA_TYPE_BIGINT; + } else { + type = TSDB_DATA_TYPE_BINARY; + } + + SHashObj *hash = taosHashInit(64, taosGetDefaultHashFunction(type), true, HASH_NO_LOCK); + if (hash == NULL) { + terrno = TSDB_CODE_OUT_OF_MEMORY; + return -1; + } + + pSdb->hashObjs[i] = hash; + taosInitRWLatch(&pSdb->locks[i]); + } + + return 0; } \ No newline at end of file diff --git a/source/dnode/mnode/sdb/src/sdbFile.c b/source/dnode/mnode/sdb/src/sdbFile.c index fbbb602882..37fcdc19ef 100644 --- a/source/dnode/mnode/sdb/src/sdbFile.c +++ b/source/dnode/mnode/sdb/src/sdbFile.c @@ -18,45 +18,41 @@ #include "tchecksum.h" static int32_t sdbCreateDir(SSdb *pSdb) { - int32_t code = taosMkDir(pSdb->currDir); - if (code != 0) { - code = TAOS_SYSTEM_ERROR(errno); - mError("failed to create dir:%s since %s", pSdb->currDir, tstrerror(code)); - return code; + if (taosMkDir(pSdb->currDir) != 0) { + terrno = TAOS_SYSTEM_ERROR(errno); + mError("failed to create dir:%s since %s", pSdb->currDir, terrstr()); + return -1; } - code = taosMkDir(pSdb->syncDir); - if (code != 0) { - code = TAOS_SYSTEM_ERROR(errno); - mError("failed to create dir:%s since %s", pSdb->syncDir, tstrerror(code)); - return code; + if (taosMkDir(pSdb->syncDir) != 0) { + terrno = TAOS_SYSTEM_ERROR(errno); + mError("failed to create dir:%s since %s", pSdb->syncDir, terrstr()); + return -1; } - code = taosMkDir(pSdb->tmpDir); - if (code != 0) { - code = TAOS_SYSTEM_ERROR(errno); - mError("failed to create dir:%s since %s", pSdb->tmpDir, tstrerror(code)); - return code; + if (taosMkDir(pSdb->tmpDir) != 0) { + terrno = TAOS_SYSTEM_ERROR(errno); + mError("failed to create dir:%s since %s", pSdb->tmpDir, terrstr()); + return -1; } return 0; } static int32_t sdbRunDeployFp(SSdb *pSdb) { - mDebug("start to run sdb deploy functions"); + mDebug("start to deploy sdb"); for (int32_t i = SDB_MAX - 1; i > SDB_START; --i) { SdbDeployFp fp = pSdb->deployFps[i]; if (fp == NULL) continue; - int32_t code = (*fp)(pSdb); - if (code != 0) { - mError("failed to deploy sdb:%d since %s", i, tstrerror(code)); - return code; + if ((*fp)(pSdb) != 0) { + mError("failed to deploy sdb:%d since %s", i, terrstr()); + return -1; } } - mDebug("sdb deploy functions run finished"); + mDebug("sdb deploy successfully"); return 0; } @@ -68,9 +64,9 @@ int32_t sdbReadFile(SSdb *pSdb) { SSdbRaw *pRaw = malloc(SDB_MAX_SIZE); if (pRaw == NULL) { - code = TSDB_CODE_OUT_OF_MEMORY; - mError("failed read file since %s", tstrerror(code)); - return code; + terrno = TSDB_CODE_OUT_OF_MEMORY; + mError("failed read file since %s", terrstr()); + return -1; } char file[PATH_MAX] = {0}; @@ -79,9 +75,9 @@ int32_t sdbReadFile(SSdb *pSdb) { FileFd fd = taosOpenFileRead(file); if (fd <= 0) { free(pRaw); - code = TAOS_SYSTEM_ERROR(errno); - mError("failed to read file:%s since %s", file, tstrerror(code)); - return code; + terrno = TAOS_SYSTEM_ERROR(errno); + mError("failed to read file:%s since %s", file, terrstr()); + return -1; } while (1) { @@ -135,6 +131,7 @@ PARSE_SDB_DATA_ERROR: taosCloseFile(fd); sdbFreeRaw(pRaw); + terrno = code; return code; } @@ -142,15 +139,15 @@ int32_t sdbWriteFile(SSdb *pSdb) { int32_t code = 0; char tmpfile[PATH_MAX] = {0}; - snprintf(tmpfile, sizeof(tmpfile), "%ssdb.data", pSdb->tmpDir); + snprintf(tmpfile, sizeof(tmpfile), "%s%ssdb.data", pSdb->tmpDir, TD_DIRSEP); char curfile[PATH_MAX] = {0}; - snprintf(curfile, sizeof(curfile), "%ssdb.data", pSdb->currDir); + snprintf(curfile, sizeof(curfile), "%s%ssdb.data", pSdb->currDir, TD_DIRSEP); FileFd fd = taosOpenFileCreateWrite(tmpfile); if (fd <= 0) { - code = TAOS_SYSTEM_ERROR(errno); - mError("failed to open file:%s for write since %s", tmpfile, tstrerror(code)); - return code; + terrno = TAOS_SYSTEM_ERROR(errno); + mError("failed to open file:%s for write since %s", tmpfile, terrstr()); + return -1; } for (int32_t i = SDB_MAX - 1; i > SDB_START; --i) { @@ -220,23 +217,21 @@ int32_t sdbWriteFile(SSdb *pSdb) { mDebug("write file:%s successfully", curfile); } + terrno = code; return code; } int32_t sdbDeploy(SSdb *pSdb) { - int32_t code = sdbCreateDir(pSdb); - if (code != 0) { - return code; + if (sdbCreateDir(pSdb) != 0) { + return -1; } - code = sdbRunDeployFp(pSdb); - if (code != 0) { - return code; + if (sdbRunDeployFp(pSdb) != 0) { + return -1; } - code = sdbWriteFile(pSdb); - if (code != 0) { - return code; + if (sdbWriteFile(pSdb) != 0) { + return -1; } return 0; diff --git a/source/dnode/mnode/sdb/src/sdbHash.c b/source/dnode/mnode/sdb/src/sdbHash.c index 1ac983d538..0db5c0beb6 100644 --- a/source/dnode/mnode/sdb/src/sdbHash.c +++ b/source/dnode/mnode/sdb/src/sdbHash.c @@ -49,7 +49,7 @@ static int32_t sdbGetkeySize(SSdb *pSdb, ESdbType type, void *pKey) { static int32_t sdbInsertRow(SSdb *pSdb, SHashObj *hash, SSdbRaw *pRaw, SSdbRow *pRow, int32_t keySize) { int32_t code = 0; - SRWLatch *pLock = &pSdb->locks[pRow->sdb]; + SRWLatch *pLock = &pSdb->locks[pRow->type]; taosWLockLatch(pLock); SSdbRow *pDstRow = taosHashGet(hash, pRow->pObj, keySize); @@ -70,7 +70,7 @@ static int32_t sdbInsertRow(SSdb *pSdb, SHashObj *hash, SSdbRaw *pRaw, SSdbRow * taosWUnLockLatch(pLock); - SdbInsertFp insertFp = pSdb->insertFps[pRow->sdb]; + SdbInsertFp insertFp = pSdb->insertFps[pRow->type]; if (insertFp != NULL) { code = (*insertFp)(pSdb, pRow->pObj); if (code != 0) { @@ -88,7 +88,7 @@ static int32_t sdbInsertRow(SSdb *pSdb, SHashObj *hash, SSdbRaw *pRaw, SSdbRow * static int32_t sdbUpdateRow(SSdb *pSdb, SHashObj *hash, SSdbRaw *pRaw, SSdbRow *pRow, int32_t keySize) { int32_t code = 0; - SRWLatch *pLock = &pSdb->locks[pRow->sdb]; + SRWLatch *pLock = &pSdb->locks[pRow->type]; taosRLockLatch(pLock); SSdbRow **ppDstRow = taosHashGet(hash, pRow->pObj, keySize); @@ -101,7 +101,7 @@ static int32_t sdbUpdateRow(SSdb *pSdb, SHashObj *hash, SSdbRaw *pRaw, SSdbRow * pRow->status = pRaw->status; taosRUnLockLatch(pLock); - SdbUpdateFp updateFp = pSdb->updateFps[pRow->sdb]; + SdbUpdateFp updateFp = pSdb->updateFps[pRow->type]; if (updateFp != NULL) { code = (*updateFp)(pSdb, pRow->pObj, pDstRow->pObj); } @@ -113,7 +113,7 @@ static int32_t sdbUpdateRow(SSdb *pSdb, SHashObj *hash, SSdbRaw *pRaw, SSdbRow * static int32_t sdbDeleteRow(SSdb *pSdb, SHashObj *hash, SSdbRaw *pRaw, SSdbRow *pRow, int32_t keySize) { int32_t code = 0; - SRWLatch *pLock = &pSdb->locks[pRow->sdb]; + SRWLatch *pLock = &pSdb->locks[pRow->type]; taosWLockLatch(pLock); SSdbRow **ppDstRow = taosHashGet(hash, pRow->pObj, keySize); @@ -128,7 +128,7 @@ static int32_t sdbDeleteRow(SSdb *pSdb, SHashObj *hash, SSdbRaw *pRaw, SSdbRow * taosHashRemove(hash, pDstRow->pObj, keySize); taosWUnLockLatch(pLock); - SdbDeleteFp deleteFp = pSdb->deleteFps[pDstRow->sdb]; + SdbDeleteFp deleteFp = pSdb->deleteFps[pDstRow->type]; if (deleteFp != NULL) { code = (*deleteFp)(pSdb, pDstRow->pObj); } @@ -139,18 +139,18 @@ static int32_t sdbDeleteRow(SSdb *pSdb, SHashObj *hash, SSdbRaw *pRaw, SSdbRow * } int32_t sdbWriteRaw(SSdb *pSdb, SSdbRaw *pRaw) { - SHashObj *hash = sdbGetHash(pSdb, pRaw->sdb); + SHashObj *hash = sdbGetHash(pSdb, pRaw->type); if (hash == NULL) return terrno; - SdbDecodeFp decodeFp = pSdb->decodeFps[pRaw->sdb]; + SdbDecodeFp decodeFp = pSdb->decodeFps[pRaw->type]; SSdbRow *pRow = (*decodeFp)(pRaw); if (pRow == NULL) { return terrno; } - pRow->sdb = pRaw->sdb; + pRow->type = pRaw->type; - int32_t keySize = sdbGetkeySize(pSdb, pRow->sdb, pRow->pObj); + int32_t keySize = sdbGetkeySize(pSdb, pRow->type, pRow->pObj); int32_t code = TSDB_CODE_SDB_INVALID_ACTION_TYPE; switch (pRaw->status) { @@ -217,9 +217,9 @@ void sdbRelease(SSdb *pSdb, void *pObj) { if (pObj == NULL) return; SSdbRow *pRow = (SSdbRow *)((char *)pObj - sizeof(SSdbRow)); - if (pRow->sdb >= SDB_MAX || pRow->sdb <= SDB_START) return; + if (pRow->type >= SDB_MAX || pRow->type <= SDB_START) return; - SRWLatch *pLock = &pSdb->locks[pRow->sdb]; + SRWLatch *pLock = &pSdb->locks[pRow->type]; taosRLockLatch(pLock); int32_t ref = atomic_sub_fetch_32(&pRow->refCount, 1); @@ -257,10 +257,10 @@ void *sdbFetch(SSdb *pSdb, ESdbType type, void *pIter, void **ppObj) { void sdbCancelFetch(SSdb *pSdb, void *pIter) { if (pIter == NULL) return; SSdbRow *pRow = *(SSdbRow **)pIter; - SHashObj *hash = sdbGetHash(pSdb, pRow->sdb); + SHashObj *hash = sdbGetHash(pSdb, pRow->type); if (hash == NULL) return; - SRWLatch *pLock = &pSdb->locks[pRow->sdb]; + SRWLatch *pLock = &pSdb->locks[pRow->type]; taosRLockLatch(pLock); taosHashCancelIterate(hash, pIter); taosRUnLockLatch(pLock); diff --git a/source/dnode/mnode/sdb/src/sdbRaw.c b/source/dnode/mnode/sdb/src/sdbRaw.c index 68a0abc859..2abff74168 100644 --- a/source/dnode/mnode/sdb/src/sdbRaw.c +++ b/source/dnode/mnode/sdb/src/sdbRaw.c @@ -16,14 +16,14 @@ #define _DEFAULT_SOURCE #include "sdbInt.h" -SSdbRaw *sdbAllocRaw(ESdbType sdb, int8_t sver, int32_t dataLen) { +SSdbRaw *sdbAllocRaw(ESdbType type, int8_t sver, int32_t dataLen) { SSdbRaw *pRaw = calloc(1, dataLen + sizeof(SSdbRaw)); if (pRaw == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; return NULL; } - pRaw->sdb = sdb; + pRaw->type = type; pRaw->sver = sver; pRaw->dataLen = dataLen; return pRaw; From 939c3b3a26a8e0f6dfbf005d337d6e79bcb50f2d Mon Sep 17 00:00:00 2001 From: Shengliang Guan Date: Mon, 29 Nov 2021 20:24:16 +0800 Subject: [PATCH 43/44] TD-10431 interface of mnode --- include/dnode/mnode/mnode.h | 194 +++++++++++++++++++------ include/dnode/mnode/sdb/sdb.h | 144 ++++++++++++++++-- source/dnode/mgmt/impl/CMakeLists.txt | 2 +- source/dnode/mnode/impl/CMakeLists.txt | 6 +- source/dnode/mnode/sdb/CMakeLists.txt | 2 +- 5 files changed, 289 insertions(+), 59 deletions(-) diff --git a/include/dnode/mnode/mnode.h b/include/dnode/mnode/mnode.h index 9d4c27a9dc..5066b881b5 100644 --- a/include/dnode/mnode/mnode.h +++ b/include/dnode/mnode/mnode.h @@ -30,28 +30,132 @@ typedef void (*SendRedirectMsgFp)(SDnode *pDnode, struct SRpcMsg *rpcMsg); typedef int32_t (*PutMsgToMnodeQFp)(SDnode *pDnode, SMnodeMsg *pMsg); typedef struct SMnodeLoad { + /** + * @brief the number of dnodes in cluster. + * + */ int64_t numOfDnode; + + /** + * @brief the number of mnodes in cluster. + * + */ int64_t numOfMnode; + + /** + * @brief the number of vgroups in cluster. + * + */ int64_t numOfVgroup; + + /** + * @brief the number of databases in cluster. + * + */ int64_t numOfDatabase; + + /** + * @brief the number of super tables in cluster. + * + */ int64_t numOfSuperTable; + + /** + * @brief the number of child tables in cluster. + * + */ int64_t numOfChildTable; + + /** + * @brief the number of normal tables in cluster. + * + */ + int64_t numOfNormalTable; + + /** + * @brief the number of numOfTimeseries in cluster. + * + */ int64_t numOfColumn; + + /** + * @brief total points written in cluster. + * + */ int64_t totalPoints; + + /** + * @brief total storage in cluster. + * + */ int64_t totalStorage; + + /** + * @brief total compressed storage in cluster. + * + */ int64_t compStorage; } SMnodeLoad; typedef struct { - int32_t dnodeId; - int64_t clusterId; - int8_t replica; - int8_t selfIndex; - SReplica replicas[TSDB_MAX_REPLICA]; - SDnode *pDnode; - PutMsgToMnodeQFp putMsgToApplyMsgFp; - SendMsgToDnodeFp sendMsgToDnodeFp; - SendMsgToMnodeFp sendMsgToMnodeFp; + /** + * @brief dnodeId of this mnode. + * + */ + int32_t dnodeId; + + /** + * @brief clusterId of this mnode. + * + */ + int64_t clusterId; + + /** + * @brief replica num of this mnode. + * + */ + int8_t replica; + + /** + * @brief self index in the array of replicas. + * + */ + int8_t selfIndex; + + /** + * @brief detail replica information of this mnode. + * + */ + SReplica replicas[TSDB_MAX_REPLICA]; + + /** + * @brief the parent dnode of this mnode. + * + */ + SDnode *pDnode; + + /** + * @brief put apply msg to the write queue in dnode. + * + */ + PutMsgToMnodeQFp putMsgToApplyMsgFp; + + /** + * @brief the callback function while send msg to dnode. + * + */ + SendMsgToDnodeFp sendMsgToDnodeFp; + + /** + * @brief the callback function while send msg to mnode. + * + */ + SendMsgToMnodeFp sendMsgToMnodeFp; + + /** + * @brief the callback function while send redirect msg to clients or peers. + * + */ SendRedirectMsgFp sendRedirectMsgFp; } SMnodeOpt; @@ -59,25 +163,25 @@ typedef struct { /** * @brief Open a mnode. * - * @param path Path of the mnode - * @param pOption Option of the mnode - * @return SMnode* The mnode object + * @param path Path of the mnode. + * @param pOption Option of the mnode. + * @return SMnode* The mnode object. */ SMnode *mndOpen(const char *path, const SMnodeOpt *pOption); /** - * @brief Close a mnode + * @brief Close a mnode. * - * @param pMnode The mnode object to close + * @param pMnode The mnode object to close. */ void mndClose(SMnode *pMnode); /** - * @brief Close a mnode + * @brief Close a mnode. * - * @param pMnode The mnode object to close - * @param pOption Options of the mnode - * @return int32_t 0 for success, -1 for failure + * @param pMnode The mnode object to close. + * @param pOption Options of the mnode. + * @return int32_t 0 for success, -1 for failure. */ int32_t mndAlter(SMnode *pMnode, const SMnodeOpt *pOption); @@ -89,80 +193,80 @@ int32_t mndAlter(SMnode *pMnode, const SMnodeOpt *pOption); void mndDestroy(const char *path); /** - * @brief Get mnode statistics info + * @brief Get mnode statistics info. * - * @param pMnode The mnode object + * @param pMnode The mnode object. * @param pLoad Statistics of the mnode. - * @return int32_t 0 for success, -1 for failure + * @return int32_t 0 for success, -1 for failure. */ int32_t mndGetLoad(SMnode *pMnode, SMnodeLoad *pLoad); /** - * @brief Get user authentication info + * @brief Get user authentication info. * - * @param pMnode The mnode object + * @param pMnode The mnode object. * @param user * @param spi * @param encrypt * @param secret * @param ckey - * @return int32_t 0 for success, -1 for failure + * @return int32_t 0 for success, -1 for failure. */ int32_t mndRetriveAuth(SMnode *pMnode, char *user, char *spi, char *encrypt, char *secret, char *ckey); /** - * @brief Initialize mnode msg + * @brief Initialize mnode msg. * - * @param pMnode The mnode object - * @param pMsg The request rpc msg - * @return int32_t The created mnode msg + * @param pMnode The mnode object. + * @param pMsg The request rpc msg. + * @return int32_t The created mnode msg. */ SMnodeMsg *mndInitMsg(SMnode *pMnode, SRpcMsg *pRpcMsg); /** - * @brief Cleanup mnode msg + * @brief Cleanup mnode msg. * - * @param pMsg The request msg + * @param pMsg The request msg. */ void mndCleanupMsg(SMnodeMsg *pMsg); /** - * @brief Cleanup mnode msg + * @brief Cleanup mnode msg. * - * @param pMsg The request msg - * @param code The error code + * @param pMsg The request msg. + * @param code The error code. */ void mndSendRsp(SMnodeMsg *pMsg, int32_t code); /** - * @brief Process the read request + * @brief Process the read request. * - * @param pMsg The request msg - * @return int32_t 0 for success, -1 for failure + * @param pMsg The request msg. + * @return int32_t 0 for success, -1 for failure. */ void mndProcessReadMsg(SMnodeMsg *pMsg); /** - * @brief Process the write request + * @brief Process the write request. * - * @param pMsg The request msg - * @return int32_t 0 for success, -1 for failure + * @param pMsg The request msg. + * @return int32_t 0 for success, -1 for failure. */ void mndProcessWriteMsg(SMnodeMsg *pMsg); /** - * @brief Process the sync request + * @brief Process the sync request. * - * @param pMsg The request msg - * @return int32_t 0 for success, -1 for failure + * @param pMsg The request msg. + * @return int32_t 0 for success, -1 for failure. */ void mndProcessSyncMsg(SMnodeMsg *pMsg); /** - * @brief Process the apply request + * @brief Process the apply request. * - * @param pMsg The request msg - * @return int32_t 0 for success, -1 for failure + * @param pMsg The request msg. + * @return int32_t 0 for success, -1 for failure. */ void mndProcessApplyMsg(SMnodeMsg *pMsg); diff --git a/include/dnode/mnode/sdb/sdb.h b/include/dnode/mnode/sdb/sdb.h index e5667e352f..3648af5fcb 100644 --- a/include/dnode/mnode/sdb/sdb.h +++ b/include/dnode/mnode/sdb/sdb.h @@ -130,36 +130,162 @@ typedef struct SSdb SSdb; typedef int32_t (*SdbInsertFp)(SSdb *pSdb, void *pObj); typedef int32_t (*SdbUpdateFp)(SSdb *pSdb, void *pSrcObj, void *pDstObj); typedef int32_t (*SdbDeleteFp)(SSdb *pSdb, void *pObj); -typedef int32_t (*SdbDeployFp)(SSdb*pSdb); +typedef int32_t (*SdbDeployFp)(SSdb *pSdb); typedef SSdbRow *(*SdbDecodeFp)(SSdbRaw *pRaw); typedef SSdbRaw *(*SdbEncodeFp)(void *pObj); typedef struct { - ESdbType sdbType; - EKeyType keyType; + /** + * @brief The sdb type of the table. + * + */ + ESdbType sdbType; + + /** + * @brief The key type of the table. + * + */ + EKeyType keyType; + + /** + * @brief The callback function when the table is first deployed. + * + */ SdbDeployFp deployFp; + + /** + * @brief Encode one row of the table into rawdata. + * + */ SdbEncodeFp encodeFp; + + /** + * @brief Decode one row of the table from rawdata. + * + */ SdbDecodeFp decodeFp; + + /** + * @brief The callback function when insert a row to sdb. + * + */ SdbInsertFp insertFp; + + /** + * @brief The callback function when undate a row in sdb. + * + */ SdbUpdateFp updateFp; + + /** + * @brief The callback function when delete a row from sdb. + * + */ SdbDeleteFp deleteFp; } SSdbTable; typedef struct SSdbOpt { + /** + * @brief The path of the sdb file. + * + */ const char *path; } SSdbOpt; -SSdb *sdbInit(SSdbOpt *pOption); -void sdbCleanup(SSdb *pSdb); +/** + * @brief Initialize and start the sdb. + * + * @param pOption Option of the sdb. + * @return SSdb* The sdb object. + */ +SSdb *sdbInit(SSdbOpt *pOption); + +/** + * @brief Stop and cleanup the sdb. + * + * @param pSdb The sdb object to close. + */ +void sdbCleanup(SSdb *pSdb); + +/** + * @brief Set the properties of sdb table. + * + * @param pSdb The sdb object. + * @param table The properties of the table. + * @return int32_t 0 for success, -1 for failure. + */ int32_t sdbSetTable(SSdb *pSdb, SSdbTable table); + +/** + * @brief Set the initial rows of sdb. + * + * @param pSdb The sdb object. + * @return int32_t 0 for success, -1 for failure. + */ int32_t sdbDeploy(SSdb *pSdb); + +/** + * @brief Load sdb from file. + * + * @param pSdb The sdb object. + * @return int32_t 0 for success, -1 for failure. + */ int32_t sdbReadFile(SSdb *pSdb); + +/** + * @brief Parse and write raw data to sdb. + * + * @param pSdb The sdb object. + * @param pRaw The raw data. + * @return int32_t 0 for success, -1 for failure. + */ int32_t sdbWrite(SSdb *pSdb, SSdbRaw *pRaw); -void *sdbAcquire(SSdb *pSdb, ESdbType type, void *pKey); -void sdbRelease(SSdb *pSdb, void *pObj); -void *sdbFetch(SSdb *pSdb, ESdbType type, void *pIter, void **ppObj); -void sdbCancelFetch(SSdb *pSdb, void *pIter); +/** + * @brief Acquire a row from sdb + * + * @param pSdb The sdb object. + * @param type The type of the row. + * @param pKey The key value of the row. + * @return void* The object of the row. + */ +void *sdbAcquire(SSdb *pSdb, ESdbType type, void *pKey); + +/** + * @brief Release a row from sdb. + * + * @param pSdb The sdb object. + * @param pObj The object of the row. + */ +void sdbRelease(SSdb *pSdb, void *pObj); + +/** + * @brief Traverse a sdb table + * + * @param pSdb The sdb object. + * @param type The type of the table. + * @param type The initial iterator of the table. + * @param pObj The object of the row just fetched. + * @return void* The next iterator of the table. + */ +void *sdbFetch(SSdb *pSdb, ESdbType type, void *pIter, void **ppObj); + +/** + * @brief Cancel a traversal + * + * @param pSdb The sdb object. + * @param pIter The iterator of the table. + * @param type The initial iterator of table. + */ +void sdbCancelFetch(SSdb *pSdb, void *pIter); + +/** + * @brief Get the number of rows in the table + * + * @param pSdb The sdb object. + * @param pIter The type of the table. + * @record int32_t The number of rows in the table + */ int32_t sdbGetSize(SSdb *pSdb, ESdbType type); SSdbRaw *sdbAllocRaw(ESdbType type, int8_t sver, int32_t dataLen); diff --git a/source/dnode/mgmt/impl/CMakeLists.txt b/source/dnode/mgmt/impl/CMakeLists.txt index d0830484ed..3eadf24164 100644 --- a/source/dnode/mgmt/impl/CMakeLists.txt +++ b/source/dnode/mgmt/impl/CMakeLists.txt @@ -12,5 +12,5 @@ target_link_libraries( target_include_directories( dnode PUBLIC "${CMAKE_SOURCE_DIR}/include/dnode/mgmt" - private "${CMAKE_CURRENT_SOURCE_DIR}/inc" + PRIVATE "${CMAKE_CURRENT_SOURCE_DIR}/inc" ) diff --git a/source/dnode/mnode/impl/CMakeLists.txt b/source/dnode/mnode/impl/CMakeLists.txt index 49c9d54513..6fecb457c0 100644 --- a/source/dnode/mnode/impl/CMakeLists.txt +++ b/source/dnode/mnode/impl/CMakeLists.txt @@ -3,11 +3,11 @@ add_library(mnode ${MNODE_SRC}) target_include_directories( mnode PUBLIC "${CMAKE_SOURCE_DIR}/include/dnode/mnode" - private "${CMAKE_CURRENT_SOURCE_DIR}/inc" + PRIVATE "${CMAKE_CURRENT_SOURCE_DIR}/inc" ) target_link_libraries( mnode PRIVATE sdb - PUBLIC transport - PUBLIC cjson + PRIVATE transport + PRIVATE cjson ) \ No newline at end of file diff --git a/source/dnode/mnode/sdb/CMakeLists.txt b/source/dnode/mnode/sdb/CMakeLists.txt index c5a0b7299a..b6620f8be4 100644 --- a/source/dnode/mnode/sdb/CMakeLists.txt +++ b/source/dnode/mnode/sdb/CMakeLists.txt @@ -3,7 +3,7 @@ add_library(sdb ${MNODE_SRC}) target_include_directories( sdb PUBLIC "${CMAKE_SOURCE_DIR}/include/dnode/mnode/sdb" - private "${CMAKE_CURRENT_SOURCE_DIR}/inc" + PRIVATE "${CMAKE_CURRENT_SOURCE_DIR}/inc" ) target_link_libraries( sdb From a63b2e873414a39bc54d000ed05d5db55a3470ba Mon Sep 17 00:00:00 2001 From: Shengliang Guan Date: Mon, 29 Nov 2021 20:26:00 +0800 Subject: [PATCH 44/44] TD-10431 remove index vars while use for synatx --- source/dnode/mgmt/impl/src/dndMnode.c | 12 +++---- source/dnode/mnode/impl/src/mndTrans.c | 46 +++++++++++++------------- 2 files changed, 29 insertions(+), 29 deletions(-) diff --git a/source/dnode/mgmt/impl/src/dndMnode.c b/source/dnode/mgmt/impl/src/dndMnode.c index f2ef420cc0..ca48642899 100644 --- a/source/dnode/mgmt/impl/src/dndMnode.c +++ b/source/dnode/mgmt/impl/src/dndMnode.c @@ -364,13 +364,13 @@ static int32_t dndBuildMnodeOptionFromMsg(SDnode *pDnode, SMnodeOpt *pOption, SC pOption->replica = pMsg->replica; pOption->selfIndex = -1; - for (int32_t index = 0; index < pMsg->replica; ++index) { - SReplica *pReplica = &pOption->replicas[index]; - pReplica->id = pMsg->replicas[index].id; - pReplica->port = pMsg->replicas[index].port; - tstrncpy(pReplica->fqdn, pMsg->replicas[index].fqdn, TSDB_FQDN_LEN); + for (int32_t i = 0; i < pMsg->replica; ++i) { + SReplica *pReplica = &pOption->replicas[i]; + pReplica->id = pMsg->replicas[i].id; + pReplica->port = pMsg->replicas[i].port; + tstrncpy(pReplica->fqdn, pMsg->replicas[i].fqdn, TSDB_FQDN_LEN); if (pReplica->id == pOption->dnodeId) { - pOption->selfIndex = index; + pOption->selfIndex = i; } } diff --git a/source/dnode/mnode/impl/src/mndTrans.c b/source/dnode/mnode/impl/src/mndTrans.c index 9333fe3f57..7eab16895c 100644 --- a/source/dnode/mnode/impl/src/mndTrans.c +++ b/source/dnode/mnode/impl/src/mndTrans.c @@ -28,18 +28,18 @@ SSdbRaw *mndTransActionEncode(STrans *pTrans) { int32_t redoActionNum = taosArrayGetSize(pTrans->redoActions); int32_t undoActionNum = taosArrayGetSize(pTrans->undoActions); - for (int32_t index = 0; index < redoLogNum; ++index) { - SSdbRaw *pTmp = taosArrayGet(pTrans->redoLogs, index); + for (int32_t i = 0; i < redoLogNum; ++i) { + SSdbRaw *pTmp = taosArrayGet(pTrans->redoLogs, i); rawDataLen += sdbGetRawTotalSize(pTmp); } - for (int32_t index = 0; index < undoLogNum; ++index) { - SSdbRaw *pTmp = taosArrayGet(pTrans->undoLogs, index); + for (int32_t i = 0; i < undoLogNum; ++i) { + SSdbRaw *pTmp = taosArrayGet(pTrans->undoLogs, i); rawDataLen += sdbGetRawTotalSize(pTmp); } - for (int32_t index = 0; index < commitLogNum; ++index) { - SSdbRaw *pTmp = taosArrayGet(pTrans->commitLogs, index); + for (int32_t i = 0; i < commitLogNum; ++i) { + SSdbRaw *pTmp = taosArrayGet(pTrans->commitLogs, i); rawDataLen += sdbGetRawTotalSize(pTmp); } @@ -59,22 +59,22 @@ SSdbRaw *mndTransActionEncode(STrans *pTrans) { SDB_SET_INT32(pRaw, dataPos, redoActionNum) SDB_SET_INT32(pRaw, dataPos, undoActionNum) - for (int32_t index = 0; index < redoLogNum; ++index) { - SSdbRaw *pTmp = taosArrayGet(pTrans->redoLogs, index); + for (int32_t i = 0; i < redoLogNum; ++i) { + SSdbRaw *pTmp = taosArrayGet(pTrans->redoLogs, i); int32_t len = sdbGetRawTotalSize(pTmp); SDB_SET_INT32(pRaw, dataPos, len) SDB_SET_BINARY(pRaw, dataPos, (void *)pTmp, len) } - for (int32_t index = 0; index < undoLogNum; ++index) { - SSdbRaw *pTmp = taosArrayGet(pTrans->undoLogs, index); + for (int32_t i = 0; i < undoLogNum; ++i) { + SSdbRaw *pTmp = taosArrayGet(pTrans->undoLogs, i); int32_t len = sdbGetRawTotalSize(pTmp); SDB_SET_INT32(pRaw, dataPos, len) SDB_SET_BINARY(pRaw, dataPos, (void *)pTmp, len) } - for (int32_t index = 0; index < commitLogNum; ++index) { - SSdbRaw *pTmp = taosArrayGet(pTrans->commitLogs, index); + for (int32_t i = 0; i < commitLogNum; ++i) { + SSdbRaw *pTmp = taosArrayGet(pTrans->commitLogs, i); int32_t len = sdbGetRawTotalSize(pTmp); SDB_SET_INT32(pRaw, dataPos, len) SDB_SET_BINARY(pRaw, dataPos, (void *)pTmp, len) @@ -134,7 +134,7 @@ SSdbRow *mndTransActionDecode(SSdbRaw *pRaw) { SDB_GET_INT32(pRaw, pRow, dataPos, &undoActionNum) int32_t code = 0; - for (int32_t index = 0; index < redoLogNum; ++index) { + for (int32_t i = 0; i < redoLogNum; ++i) { int32_t dataLen = 0; SDB_GET_INT32(pRaw, pRow, dataPos, &dataLen) @@ -162,8 +162,8 @@ static int32_t mndTransActionInsert(SSdb *pSdb, STrans *pTrans) { SArray *pArray = pTrans->redoLogs; int32_t arraySize = taosArrayGetSize(pArray); - for (int32_t index = 0; index < arraySize; ++index) { - SSdbRaw *pRaw = taosArrayGet(pArray, index); + for (int32_t i = 0; i < arraySize; ++i) { + SSdbRaw *pRaw = taosArrayGet(pArray, i); int32_t code = sdbWrite(pSdb, pRaw); if (code != 0) { mError("trn:%d, failed to write raw:%p to sdb since %s", pTrans->id, pRaw, terrstr()); @@ -179,8 +179,8 @@ static int32_t mndTransActionDelete(SSdb *pSdb, STrans *pTrans) { SArray *pArray = pTrans->redoLogs; int32_t arraySize = taosArrayGetSize(pArray); - for (int32_t index = 0; index < arraySize; ++index) { - SSdbRaw *pRaw = taosArrayGet(pArray, index); + for (int32_t i = 0; i < arraySize; ++i) { + SSdbRaw *pRaw = taosArrayGet(pArray, i); int32_t code = sdbWrite(pSdb, pRaw); if (code != 0) { mError("trn:%d, failed to write raw:%p to sdb since %s", pTrans->id, pRaw, terrstr()); @@ -197,8 +197,8 @@ static int32_t mndTransActionUpdate(SSdb *pSdb, STrans *pTrans, STrans *pDstTran SArray *pArray = pTrans->redoLogs; int32_t arraySize = taosArrayGetSize(pArray); - for (int32_t index = 0; index < arraySize; ++index) { - SSdbRaw *pRaw = taosArrayGet(pArray, index); + for (int32_t i = 0; i < arraySize; ++i) { + SSdbRaw *pRaw = taosArrayGet(pArray, i); int32_t code = sdbWrite(pSdb, pRaw); if (code != 0) { mError("trn:%d, failed to write raw:%p to sdb since %s", pTrans->id, pRaw, terrstr()); @@ -243,8 +243,8 @@ STrans *mndTransCreate(SMnode *pMnode, ETrnPolicy policy, void *rpcHandle) { } static void trnDropArray(SArray *pArray) { - for (int32_t index = 0; index < pArray->size; ++index) { - SSdbRaw *pRaw = taosArrayGet(pArray, index); + for (int32_t i = 0; i < pArray->size; ++i) { + SSdbRaw *pRaw = taosArrayGet(pArray, i); tfree(pRaw); } @@ -373,8 +373,8 @@ int32_t mndTransApply(SMnode *pMnode, SSdbRaw *pRaw, void *pData, int32_t code) } static int32_t trnExecuteArray(SMnode *pMnode, SArray *pArray) { - for (int32_t index = 0; index < pArray->size; ++index) { - SSdbRaw *pRaw = taosArrayGetP(pArray, index); + for (int32_t i = 0; i < pArray->size; ++i) { + SSdbRaw *pRaw = taosArrayGetP(pArray, i); if (sdbWrite(pMnode->pSdb, pRaw) != 0) { return -1; }