From 27cb3638c22abf2882bee1686862e759d05731e5 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 3 Jul 2024 09:20:37 +0800 Subject: [PATCH 01/22] fix(stream): add more check before launching update stream task nodeEp --- source/dnode/mnode/impl/src/mndStream.c | 2 +- source/dnode/mnode/impl/src/mndStreamUtil.c | 18 ++++++++++++++++-- 2 files changed, 17 insertions(+), 3 deletions(-) diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index a137c10ed5..54b24fe0b4 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -2309,7 +2309,7 @@ static int32_t mndProcessNodeCheckReq(SRpcMsg *pMsg) { taosThreadMutexUnlock(&execInfo.lock); if (numOfNodes == 0) { - mDebug("end to do stream task node change checking, no vgroup exists, do nothing"); + mDebug("end to do stream task(s) node change checking, no stream tasks exist, do nothing"); execInfo.ts = ts; atomic_store_32(&mndNodeCheckSentinel, 0); return 0; diff --git a/source/dnode/mnode/impl/src/mndStreamUtil.c b/source/dnode/mnode/impl/src/mndStreamUtil.c index 8fb5bc8a99..a15b817784 100644 --- a/source/dnode/mnode/impl/src/mndStreamUtil.c +++ b/source/dnode/mnode/impl/src/mndStreamUtil.c @@ -84,6 +84,7 @@ SArray *mndTakeVgroupSnapshot(SMnode *pMnode, bool *allReady) { SSdb *pSdb = pMnode->pSdb; void *pIter = NULL; SVgObj *pVgroup = NULL; + int32_t replica = -1; // do the replica check *allReady = true; SArray *pVgroupListSnapshot = taosArrayInit(4, sizeof(SNodeEntry)); @@ -97,6 +98,17 @@ SArray *mndTakeVgroupSnapshot(SMnode *pMnode, bool *allReady) { SNodeEntry entry = {.nodeId = pVgroup->vgId, .hbTimestamp = pVgroup->updateTime}; entry.epset = mndGetVgroupEpset(pMnode, pVgroup); + if (replica == -1) { + replica = pVgroup->replica; + } else { + if (replica != pVgroup->replica) { + mInfo("vgId:%d replica:%d inconsistent with other vgroups replica:%d, not ready for stream operations", + pVgroup->vgId); + *allReady = false; + break; + } + } + // if not all ready till now, no need to check the remaining vgroups. if (*allReady) { for (int32_t i = 0; i < pVgroup->replica; ++i) { @@ -107,8 +119,10 @@ SArray *mndTakeVgroupSnapshot(SMnode *pMnode, bool *allReady) { } ESyncState state = pVgroup->vnodeGid[i].syncState; - if (state == TAOS_SYNC_STATE_OFFLINE || state == TAOS_SYNC_STATE_ERROR) { - mInfo("vgId:%d offline/err, not ready for checkpoint or other operations", pVgroup->vgId); + if (state == TAOS_SYNC_STATE_OFFLINE || state == TAOS_SYNC_STATE_ERROR || state == TAOS_SYNC_STATE_LEARNER || + state == TAOS_SYNC_STATE_CANDIDATE) { + mInfo("vgId:%d state:%d , not ready for checkpoint or other operations, not check other vgroups", + pVgroup->vgId, state); *allReady = false; break; } From 653f7a1a434403882a2dc94f6ea59875f9665097 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 4 Jul 2024 13:55:50 +0800 Subject: [PATCH 02/22] fix(stream): refactor the checkpoint consensus policy. --- include/common/tmsgdef.h | 1 + include/libs/stream/streamMsg.h | 4 +- include/libs/stream/tstream.h | 5 +- source/dnode/mnode/impl/inc/mndStream.h | 10 +- source/dnode/mnode/impl/src/mndMain.c | 14 +- source/dnode/mnode/impl/src/mndStream.c | 195 +++++++++++++++----- source/dnode/mnode/impl/src/mndStreamUtil.c | 132 ++++--------- source/dnode/snode/src/snode.c | 4 +- source/dnode/vnode/src/tqCommon/tqCommon.c | 8 +- source/libs/stream/src/streamCheckStatus.c | 6 +- source/libs/stream/src/streamCheckpoint.c | 8 - source/libs/stream/src/streamHb.c | 2 + 12 files changed, 223 insertions(+), 166 deletions(-) diff --git a/include/common/tmsgdef.h b/include/common/tmsgdef.h index c92649f1f7..19fe34fe01 100644 --- a/include/common/tmsgdef.h +++ b/include/common/tmsgdef.h @@ -251,6 +251,7 @@ TD_DEF_MSG_TYPE(TDMT_MND_STREAM_UPDATE_CHKPT_EVT, "stream-update-chkpt-evt", NULL, NULL) TD_DEF_MSG_TYPE(TDMT_MND_STREAM_CHKPT_REPORT, "stream-chkpt-report", NULL, NULL) TD_DEF_MSG_TYPE(TDMT_MND_STREAM_CHKPT_CONSEN, "stream-chkpt-consen", NULL, NULL) + TD_DEF_MSG_TYPE(TDMT_MND_STREAM_CONSEN_TIMER, "stream-consen-tmr", NULL, NULL) TD_DEF_MSG_TYPE(TDMT_MND_MAX_MSG, "mnd-max", NULL, NULL) TD_CLOSE_MSG_SEG(TDMT_END_MND_MSG) diff --git a/include/libs/stream/streamMsg.h b/include/libs/stream/streamMsg.h index b69032330d..8b6ca2c5cd 100644 --- a/include/libs/stream/streamMsg.h +++ b/include/libs/stream/streamMsg.h @@ -242,8 +242,8 @@ typedef struct { typedef struct SCheckpointConsensusEntry { SRestoreCheckpointInfo req; - SRpcMsg rsp; - int64_t ts; + SRpcHandleInfo rspInfo; + int64_t ts; } SCheckpointConsensusEntry; #ifdef __cplusplus diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index e98039d2fe..66b9db47e2 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -616,8 +616,9 @@ typedef struct SStreamTaskState { typedef struct SCheckpointConsensusInfo { SArray* pTaskList; - int64_t checkpointId; - int64_t genTs; +// int64_t checkpointId; +// int64_t genTs; + int32_t numOfTasks; } SCheckpointConsensusInfo; int32_t streamSetupScheduleTrigger(SStreamTask* pTask); diff --git a/source/dnode/mnode/impl/inc/mndStream.h b/source/dnode/mnode/impl/inc/mndStream.h index b261f89057..a86e06b486 100644 --- a/source/dnode/mnode/impl/inc/mndStream.h +++ b/source/dnode/mnode/impl/inc/mndStream.h @@ -133,7 +133,8 @@ int32_t mndCreateStreamResetStatusTrans(SMnode *pMnode, SStreamObj *pStream) int32_t mndStreamSetUpdateChkptAction(SMnode *pMnode, STrans *pTrans, SStreamObj *pStream); int32_t mndCreateStreamChkptInfoUpdateTrans(SMnode *pMnode, SStreamObj *pStream, SArray *pChkptInfoList); int32_t mndScanCheckpointReportInfo(SRpcMsg *pReq); -int32_t mndSendConsensusCheckpointIdRsp(SArray* pList, int64_t checkpointId); +int32_t mndSendQuickConsensusChkptIdRsp(SRestoreCheckpointInfo *pReq, int32_t code, int64_t streamId, + int64_t checkpointId, SRpcHandleInfo *pRpcInfo); void removeTasksInBuf(SArray *pTaskIds, SStreamExecInfo *pExecInfo); @@ -146,10 +147,9 @@ void mndInitStreamExecInfo(SMnode *pMnode, SStreamExecInfo *pExecInf int32_t removeExpiredNodeEntryAndTaskInBuf(SArray *pNodeSnapshot); void removeStreamTasksInBuf(SStreamObj *pStream, SStreamExecInfo *pExecNode); -SCheckpointConsensusInfo *mndGetConsensusInfo(SHashObj *pHash, int64_t streamId); -void mndAddConsensusTasks(SCheckpointConsensusInfo *pInfo, const SRestoreCheckpointInfo *pRestoreInfo, SRpcMsg *pMsg); -int64_t mndGetConsensusCheckpointId(SCheckpointConsensusInfo *pInfo, SStreamObj *pStream); -bool mndAllTaskSendCheckpointId(SCheckpointConsensusInfo *pInfo, int32_t numOfTasks, int32_t* pTotal); +SCheckpointConsensusInfo *mndGetConsensusInfo(SHashObj *pHash, int64_t streamId, int32_t numOfTasks); +void mndAddConsensusTasks(SCheckpointConsensusInfo *pInfo, const SRestoreCheckpointInfo *pRestoreInfo, + SRpcHandleInfo *pRpcInfo); void mndClearConsensusRspEntry(SCheckpointConsensusInfo *pInfo); int32_t doSendConsensusCheckpointRsp(SRestoreCheckpointInfo *pInfo, SRpcMsg *pMsg, int64_t checkpointId); int64_t mndClearConsensusCheckpointId(SHashObj* pHash, int64_t streamId); diff --git a/source/dnode/mnode/impl/src/mndMain.c b/source/dnode/mnode/impl/src/mndMain.c index cad8c6d745..3acb727910 100644 --- a/source/dnode/mnode/impl/src/mndMain.c +++ b/source/dnode/mnode/impl/src/mndMain.c @@ -177,6 +177,15 @@ static void mndStreamCheckNode(SMnode *pMnode) { } } +static void mndStreamConsensusChkpt(SMnode *pMnode) { + int32_t contLen = 0; + void *pReq = mndBuildTimerMsg(&contLen); + if (pReq != NULL) { + SRpcMsg rpcMsg = {.msgType = TDMT_MND_STREAM_CONSEN_TIMER, .pCont = pReq, .contLen = contLen}; + tmsgPutToQueue(&pMnode->msgCb, WRITE_QUEUE, &rpcMsg); + } +} + static void mndPullupTelem(SMnode *pMnode) { mTrace("pullup telem msg"); int32_t contLen = 0; @@ -308,7 +317,6 @@ static int32_t minCronTime() { min = TMIN(min, tsCompactPullupInterval); min = TMIN(min, tsMqRebalanceInterval); min = TMIN(min, tsStreamCheckpointInterval); - min = TMIN(min, 6); // checkpointRemain min = TMIN(min, tsStreamNodeCheckInterval); min = TMIN(min, tsArbHeartBeatIntervalSec); min = TMIN(min, tsArbCheckSyncIntervalSec); @@ -353,6 +361,10 @@ void mndDoTimerPullupTask(SMnode *pMnode, int64_t sec) { mndStreamCheckNode(pMnode); } + if (sec % 5 == 0) { + mndStreamConsensusChkpt(pMnode); + } + if (sec % tsTelemInterval == (TMIN(60, (tsTelemInterval - 1)))) { mndPullupTelem(pMnode); } diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index 54b24fe0b4..a4c03ab3e0 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -59,7 +59,8 @@ static int32_t mndProcessNodeCheckReq(SRpcMsg *pMsg); static int32_t extractNodeListFromStream(SMnode *pMnode, SArray *pNodeList); static int32_t mndProcessStreamReqCheckpoint(SRpcMsg *pReq); static int32_t mndProcessCheckpointReport(SRpcMsg *pReq); -static int32_t mndProcessConsensusCheckpointId(SRpcMsg *pReq); +static int32_t mndProcessConsensusCheckpointId(SRpcMsg *pMsg); +static int32_t mndProcessConsensusInTmr(SRpcMsg *pMsg); static SVgroupChangeInfo mndFindChangedNodeInfo(SMnode *pMnode, const SArray *pPrevNodeList, const SArray *pNodeList); @@ -123,6 +124,7 @@ int32_t mndInitStream(SMnode *pMnode) { mndSetMsgHandle(pMnode, TDMT_STREAM_TASK_REPORT_CHECKPOINT, mndTransProcessRsp); mndSetMsgHandle(pMnode, TDMT_MND_STREAM_HEARTBEAT, mndProcessStreamHb); mndSetMsgHandle(pMnode, TDMT_MND_STREAM_NODECHANGE_CHECK, mndProcessNodeCheckReq); + mndSetMsgHandle(pMnode, TDMT_MND_STREAM_CONSEN_TIMER, mndProcessConsensusInTmr); mndSetMsgHandle(pMnode, TDMT_MND_PAUSE_STREAM, mndProcessPauseStreamReq); mndSetMsgHandle(pMnode, TDMT_MND_RESUME_STREAM, mndProcessResumeStreamReq); @@ -803,7 +805,7 @@ static int32_t mndProcessCreateStreamReq(SRpcMsg *pReq) { taosThreadMutexLock(&execInfo.lock); mDebug("stream stream:%s start to register tasks into task nodeList and set initial checkpointId", createReq.name); saveTaskAndNodeInfoIntoBuf(&streamObj, &execInfo); - mndRegisterConsensusChkptId(execInfo.pStreamConsensus, streamObj.uid); +// mndRegisterConsensusChkptId(execInfo.pStreamConsensus, streamObj.uid); taosThreadMutexUnlock(&execInfo.lock); // execute creation @@ -2625,12 +2627,42 @@ int32_t mndProcessCheckpointReport(SRpcMsg *pReq) { return 0; } -static int32_t mndProcessConsensusCheckpointId(SRpcMsg *pReq) { - SMnode *pMnode = pReq->info.node; +static int64_t getConsensusId(int64_t streamId, int32_t numOfTasks, bool* pAllEqual) { + int32_t num = 0; + int64_t chkId = INT64_MAX; + *pAllEqual = true; + + for(int32_t i = 0; i < taosArrayGetSize(execInfo.pTaskList); ++i) { + STaskId* p = taosArrayGet(execInfo.pTaskList, i); + if (p->streamId != streamId) { + continue; + } + + num += 1; + STaskStatusEntry* pe = taosHashGet(execInfo.pTaskMap, p, sizeof(*p)); + + if (chkId != INT64_MAX && chkId != pe->checkpointInfo.latestId) { + *pAllEqual = false; + } + + if (chkId > pe->checkpointInfo.latestId) { + chkId = pe->checkpointInfo.latestId; + } + } + + if (num < numOfTasks) { // not all task send info to mnode through hbMsg, no valid checkpoint Id + return -1; + } + + return chkId; +} + +static int32_t mndProcessConsensusCheckpointId(SRpcMsg *pMsg) { + SMnode *pMnode = pMsg->info.node; SDecoder decoder = {0}; SRestoreCheckpointInfo req = {0}; - tDecoderInit(&decoder, pReq->pCont, pReq->contLen); + tDecoderInit(&decoder, pMsg->pCont, pMsg->contLen); if (tDecodeRestoreCheckpointInfo(&decoder, &req)) { tDecoderClear(&decoder); @@ -2647,80 +2679,155 @@ static int32_t mndProcessConsensusCheckpointId(SRpcMsg *pReq) { taosThreadMutexLock(&execInfo.lock); SStreamObj *pStream = mndGetStreamObj(pMnode, req.streamId); + + // mnode handle the create stream transaction too slow may cause this problem if (pStream == NULL) { - mWarn("failed to find the stream:0x%" PRIx64 ", not handle checkpoint-report, try to acquire in buf", req.streamId); + mWarn("failed to find the stream:0x%" PRIx64 ", not handle consensus-checkpointId", req.streamId); // not in meta-store yet, try to acquire the task in exec buffer // the checkpoint req arrives too soon before the completion of the create stream trans. STaskId id = {.streamId = req.streamId, .taskId = req.taskId}; void *p = taosHashGet(execInfo.pTaskMap, &id, sizeof(id)); if (p == NULL) { - mError("failed to find the stream:0x%" PRIx64 " in buf, not handle the checkpoint-report", req.streamId); + mError("failed to find the stream:0x%" PRIx64 " in buf, not handle consensus-checkpointId", req.streamId); terrno = TSDB_CODE_MND_STREAM_NOT_EXIST; + + mndSendQuickConsensusChkptIdRsp(&req, terrno, req.streamId, 0, &pMsg->info); + taosThreadMutexUnlock(&execInfo.lock); + pMsg->info.handle = NULL; // disable auto rsp return -1; } else { mDebug("s-task:0x%" PRIx64 "-0x%x in buf not in mnode/meta, create stream trans may not complete yet", req.streamId, req.taskId); + // todo wait for stream is created } } + mInfo("vgId:%d meta-stored checkpointId for stream:0x%" PRIx64 " %s is:%" PRId64, req.nodeId, req.streamId, + pStream->name, pStream->checkpointId); + int32_t numOfTasks = (pStream == NULL) ? 0 : mndGetNumOfStreamTasks(pStream); - - SCheckpointConsensusInfo *pInfo = mndGetConsensusInfo(execInfo.pStreamConsensus, req.streamId); - - int64_t ckId = mndGetConsensusCheckpointId(pInfo, pStream); - if (ckId != -1) { // consensus checkpoint id already exist - SRpcMsg rsp = {0}; - rsp.code = 0; - rsp.info = pReq->info; - rsp.contLen = sizeof(SRestoreCheckpointInfoRsp) + sizeof(SMsgHead); - rsp.pCont = rpcMallocCont(rsp.contLen); - - SMsgHead *pHead = rsp.pCont; - pHead->vgId = htonl(req.nodeId); - - mDebug("stream:0x%" PRIx64 " consensus-checkpointId:%" PRId64 " exists, return directly", req.streamId, ckId); - doSendConsensusCheckpointRsp(&req, &rsp, ckId); + if ((pStream != NULL) && (pStream->checkpointId == 0)) { // not generated checkpoint yet, return 0 directly + mndSendQuickConsensusChkptIdRsp(&req, TSDB_CODE_SUCCESS, req.streamId, 0, &pMsg->info); taosThreadMutexUnlock(&execInfo.lock); - pReq->info.handle = NULL; // disable auto rsp - + pMsg->info.handle = NULL; // disable auto rsp return TSDB_CODE_SUCCESS; } - mndAddConsensusTasks(pInfo, &req, pReq); + bool allEqual = true; + int64_t chkId = getConsensusId(req.streamId, numOfTasks, &allEqual); - int32_t total = 0; - if (mndAllTaskSendCheckpointId(pInfo, numOfTasks, &total)) { // all tasks has send the reqs - // start transaction to set the checkpoint id - int64_t checkpointId = mndGetConsensusCheckpointId(pInfo, pStream); - mInfo("stream:0x%" PRIx64 " %s all %d tasks send latest checkpointId, the consensus-checkpointId is:%" PRId64 - " will be issued soon", - req.streamId, pStream->name, numOfTasks, checkpointId); + // some tasks not send hbMsg to mnode yet, wait for 5s. + if (chkId == -1) { + mDebug( + "not all task send hbMsg yet, add into list and wait for 10s to check the consensus-checkpointId again, " + "s-task:0x%x", req.taskId); + SCheckpointConsensusInfo *pInfo = mndGetConsensusInfo(execInfo.pStreamConsensus, req.streamId, numOfTasks); + mndAddConsensusTasks(pInfo, &req, &pMsg->info); + taosThreadMutexUnlock(&execInfo.lock); - // start the checkpoint consensus trans - int32_t code = mndSendConsensusCheckpointIdRsp(pInfo->pTaskList, checkpointId); - if (code == TSDB_CODE_SUCCESS) { - mndClearConsensusRspEntry(pInfo); - mDebug("clear all waiting for rsp entry for stream:0x%" PRIx64, req.streamId); - } else { - mDebug("stream:0x%" PRIx64 " not start send consensus-checkpointId msg, due to not all task ready", req.streamId); - } - } else { - mDebug("stream:0x%" PRIx64 " %d/%d tasks send consensus-checkpointId info", req.streamId, total, numOfTasks); + pMsg->info.handle = NULL; // disable auto rsp + return 0; } + if (chkId == req.checkpointId) { + mDebug("vgId:%d stream:0x%" PRIx64 " %s consensus-checkpointId is:%" PRId64, req.nodeId, req.streamId, + pStream->name, pStream->checkpointId); + mndSendQuickConsensusChkptIdRsp(&req, TSDB_CODE_SUCCESS, req.streamId, chkId, &pMsg->info); + + taosThreadMutexUnlock(&execInfo.lock); + pMsg->info.handle = NULL; // disable auto rsp + return 0; + } + + // wait for 5s and check again + SCheckpointConsensusInfo *pInfo = mndGetConsensusInfo(execInfo.pStreamConsensus, req.streamId, numOfTasks); + mndAddConsensusTasks(pInfo, &req, &pMsg->info); + if (pStream != NULL) { mndReleaseStream(pMnode, pStream); } taosThreadMutexUnlock(&execInfo.lock); - pReq->info.handle = NULL; // disable auto rsp + pMsg->info.handle = NULL; // disable auto rsp return 0; } +int32_t mndProcessConsensusInTmr(SRpcMsg *pMsg) { + int64_t now = taosGetTimestampMs(); + int64_t streamId = -1; // todo: fix only one + + mDebug("start to process consensus-checkpointId in tmr"); + taosThreadMutexLock(&execInfo.lock); + + void *pIter = NULL; + while ((pIter = taosHashIterate(execInfo.pStreamConsensus, pIter)) != NULL) { + SCheckpointConsensusInfo *pInfo = (SCheckpointConsensusInfo *)pIter; + + int32_t j = 0; + int32_t num = taosArrayGetSize(pInfo->pTaskList); + + SArray *pList = taosArrayInit(4, sizeof(int32_t)); + + for (; j < num; ++j) { + SCheckpointConsensusEntry *pe = taosArrayGet(pInfo->pTaskList, j); + + if ((now - pe->ts) > 10 * 1000) { + bool allEqual = true; + int64_t chkId = getConsensusId(pe->req.streamId, pInfo->numOfTasks, &allEqual); + if (chkId == -1) { + mDebug("tasks send hbMsg for stream:0x%" PRIx64 ", wait for next round", pe->req.streamId); + break; + } + + if (allEqual) { + mDebug("all has identical checkpointId for stream:0x%"PRIx64" send checkpointId to s-task:0x%x", + pe->req.streamId, pe->req.taskId); + + mndSendQuickConsensusChkptIdRsp(&pe->req, TSDB_CODE_SUCCESS, pe->req.streamId, chkId, &pe->rspInfo); + } else { + ASSERT(chkId <= pe->req.checkpointId); + mndSendQuickConsensusChkptIdRsp(&pe->req, TSDB_CODE_SUCCESS, pe->req.streamId, chkId, &pe->rspInfo); + } + + taosArrayPush(pList, &pe->req.taskId); + streamId = pe->req.streamId; + } else { + mDebug("s-task:0x%x sendTs:%" PRId64 " wait %2.fs already, wait for next round to check", pe->req.taskId, + (now - pe->ts)/ 1000.0, pe->ts); + } + } + + if (taosArrayGetSize(pList) > 0) { + for (int32_t i = 0; i < taosArrayGetSize(pList); ++i) { + int32_t *taskId = taosArrayGet(pList, i); + for (int32_t k = 0; k < taosArrayGetSize(pInfo->pTaskList); ++k) { + SCheckpointConsensusEntry *pe = taosArrayGet(pInfo->pTaskList, k); + if (pe->req.taskId == *taskId) { + taosArrayRemove(pInfo->pTaskList, k); + break; + } + } + } + } + + taosArrayDestroy(pList); + + if (taosArrayGetSize(pInfo->pTaskList) == 0) { + mndClearConsensusRspEntry(pInfo); + mndClearConsensusCheckpointId(execInfo.pStreamConsensus, streamId); + } + } + + taosThreadMutexUnlock(&execInfo.lock); + + mDebug("end to process consensus-checkpointId in tmr"); + return TSDB_CODE_SUCCESS; +} + static int32_t mndProcessCreateStreamReqFromMNode(SRpcMsg *pReq) { int32_t code = mndProcessCreateStreamReq(pReq); if (code != 0 && code != TSDB_CODE_ACTION_IN_PROGRESS) { diff --git a/source/dnode/mnode/impl/src/mndStreamUtil.c b/source/dnode/mnode/impl/src/mndStreamUtil.c index a15b817784..9ee820925c 100644 --- a/source/dnode/mnode/impl/src/mndStreamUtil.c +++ b/source/dnode/mnode/impl/src/mndStreamUtil.c @@ -848,7 +848,7 @@ int32_t doSendConsensusCheckpointRsp(SRestoreCheckpointInfo* pInfo, SRpcMsg* pMs return -1; } - int32_t tlen = sizeof(SMsgHead) + blen; + int32_t tlen = sizeof(SMsgHead) + blen; void *abuf = POINTER_SHIFT(pMsg->pCont, sizeof(SMsgHead)); SEncoder encoder; tEncoderInit(&encoder, abuf, tlen); @@ -863,22 +863,30 @@ int32_t doSendConsensusCheckpointRsp(SRestoreCheckpointInfo* pInfo, SRpcMsg* pMs return code; } -int32_t mndSendConsensusCheckpointIdRsp(SArray* pInfoList, int64_t checkpointId) { - for(int32_t i = 0; i < taosArrayGetSize(pInfoList); ++i) { - SCheckpointConsensusEntry* pInfo = taosArrayGet(pInfoList, i); - doSendConsensusCheckpointRsp(&pInfo->req, &pInfo->rsp, checkpointId); - } - return 0; +int32_t mndSendQuickConsensusChkptIdRsp(SRestoreCheckpointInfo *pReq, int32_t code, int64_t streamId, + int64_t checkpointId, SRpcHandleInfo *pRpcInfo) { + SRpcMsg rsp = {.code = code, .info = *pRpcInfo, .contLen = sizeof(SRestoreCheckpointInfoRsp) + sizeof(SMsgHead)}; + rsp.pCont = rpcMallocCont(rsp.contLen); + + SMsgHead *pHead = rsp.pCont; + pHead->vgId = htonl(pReq->nodeId); + + mDebug("stream:0x%" PRIx64 " consensus-checkpointId:%" PRId64 " exists, s-task:0x%x send to vnode", + streamId, checkpointId, pReq->taskId); + return doSendConsensusCheckpointRsp(pReq, &rsp, checkpointId); } -SCheckpointConsensusInfo* mndGetConsensusInfo(SHashObj* pHash, int64_t streamId) { +SCheckpointConsensusInfo* mndGetConsensusInfo(SHashObj* pHash, int64_t streamId, int32_t numOfTasks) { void* pInfo = taosHashGet(pHash, &streamId, sizeof(streamId)); if (pInfo != NULL) { return (SCheckpointConsensusInfo*)pInfo; } SCheckpointConsensusInfo p = { - .genTs = -1, .checkpointId = -1, .pTaskList = taosArrayInit(4, sizeof(SCheckpointConsensusEntry))}; + .pTaskList = taosArrayInit(4, sizeof(SCheckpointConsensusEntry)), + .numOfTasks = numOfTasks, + }; + taosHashPut(pHash, &streamId, sizeof(streamId), &p, sizeof(p)); void* pChkptInfo = (SCheckpointConsensusInfo*)taosHashGet(pHash, &streamId, sizeof(streamId)); @@ -887,87 +895,15 @@ SCheckpointConsensusInfo* mndGetConsensusInfo(SHashObj* pHash, int64_t streamId) // no matter existed or not, add the request into info list anyway, since we need to send rsp mannually // discard the msg may lead to the lost of connections. -void mndAddConsensusTasks(SCheckpointConsensusInfo *pInfo, const SRestoreCheckpointInfo *pRestoreInfo, SRpcMsg *pMsg) { - SCheckpointConsensusEntry info = {0}; +void mndAddConsensusTasks(SCheckpointConsensusInfo *pInfo, const SRestoreCheckpointInfo *pRestoreInfo, SRpcHandleInfo* pRpcInfo) { + SCheckpointConsensusEntry info = {.ts = taosGetTimestampMs(), .rspInfo = *pRpcInfo}; memcpy(&info.req, pRestoreInfo, sizeof(info.req)); - info.rsp.code = 0; - info.rsp.info = pMsg->info; - info.rsp.contLen = sizeof(SRestoreCheckpointInfoRsp) + sizeof(SMsgHead); - info.rsp.pCont = rpcMallocCont(info.rsp.contLen); - - SMsgHead *pHead = info.rsp.pCont; - pHead->vgId = htonl(pRestoreInfo->nodeId); - taosArrayPush(pInfo->pTaskList, &info); -} -static int32_t entryComparFn(const void* p1, const void* p2) { - const SCheckpointConsensusEntry* pe1 = p1; - const SCheckpointConsensusEntry* pe2 = p2; - - if (pe1->req.taskId == pe2->req.taskId) { - return 0; - } - - return pe1->req.taskId < pe2->req.taskId? -1:1; -} - -bool mndAllTaskSendCheckpointId(SCheckpointConsensusInfo* pInfo, int32_t numOfTasks, int32_t* pTotal) { - int32_t numOfExisted = taosArrayGetSize(pInfo->pTaskList); - if (numOfExisted < numOfTasks) { - if (pTotal != NULL) { - *pTotal = numOfExisted; - } - return false; - } - - taosArraySort(pInfo->pTaskList, entryComparFn); - - int32_t num = 1; - int32_t taskId = ((SCheckpointConsensusEntry*)taosArrayGet(pInfo->pTaskList, 0))->req.taskId; - for(int32_t i = 1; i < taosArrayGetSize(pInfo->pTaskList); ++i) { - SCheckpointConsensusEntry* pe = taosArrayGet(pInfo->pTaskList, i); - if (pe->req.taskId != taskId) { - num += 1; - taskId = pe->req.taskId; - } - } - - if (pTotal != NULL) { - *pTotal = num; - } - - ASSERT(num <= numOfTasks); - return num == numOfTasks; -} - -int64_t mndGetConsensusCheckpointId(SCheckpointConsensusInfo* pInfo, SStreamObj* pStream) { - if (pInfo->genTs > 0) { // there is no checkpoint ever generated if the checkpointId is 0. - mDebug("existed consensus-checkpointId:%" PRId64 " for stream:0x%" PRIx64 " %s exist, and return", - pInfo->checkpointId, pStream->uid, pStream->name); - return pInfo->checkpointId; - } - - int32_t numOfTasks = mndGetNumOfStreamTasks(pStream); - if (!mndAllTaskSendCheckpointId(pInfo, numOfTasks, NULL)) { - return -1; - } - - int64_t checkpointId = INT64_MAX; - - for (int32_t i = 0; i < taosArrayGetSize(pInfo->pTaskList); ++i) { - SCheckpointConsensusEntry *pEntry = taosArrayGet(pInfo->pTaskList, i); - if (pEntry->req.checkpointId < checkpointId) { - checkpointId = pEntry->req.checkpointId; - mTrace("stream:0x%" PRIx64 " %s task:0x%x vgId:%d latest checkpointId:%" PRId64, pStream->uid, pStream->name, - pEntry->req.taskId, pEntry->req.nodeId, pEntry->req.checkpointId); - } - } - - pInfo->checkpointId = checkpointId; - pInfo->genTs = taosGetTimestampMs(); - return checkpointId; + int32_t num = taosArrayGetSize(pInfo->pTaskList); + mDebug("s-task:0x%x added into consensus-checkpointId list, stream:0x%" PRIx64 " total waiting:%d", + pRestoreInfo->taskId, pRestoreInfo->streamId, num); } void mndClearConsensusRspEntry(SCheckpointConsensusInfo* pInfo) { @@ -982,15 +918,15 @@ int64_t mndClearConsensusCheckpointId(SHashObj* pHash, int64_t streamId) { return TSDB_CODE_SUCCESS; } -int32_t mndRegisterConsensusChkptId(SHashObj* pHash, int64_t streamId) { - void* pInfo = taosHashGet(pHash, &streamId, sizeof(streamId)); - ASSERT(pInfo == NULL); - - SCheckpointConsensusInfo p = {.genTs = taosGetTimestampMs(), .checkpointId = 0, .pTaskList = NULL}; - taosHashPut(pHash, &streamId, sizeof(streamId), &p, sizeof(p)); - - SCheckpointConsensusInfo* pChkptInfo = (SCheckpointConsensusInfo*)taosHashGet(pHash, &streamId, sizeof(streamId)); - ASSERT(pChkptInfo->genTs > 0 && pChkptInfo->checkpointId == 0); - mDebug("s-task:0x%" PRIx64 " set the initial consensus-checkpointId:0", streamId); - return TSDB_CODE_SUCCESS; -} \ No newline at end of file +//int32_t mndRegisterConsensusChkptId(SHashObj* pHash, int64_t streamId) { +// void* pInfo = taosHashGet(pHash, &streamId, sizeof(streamId)); +// ASSERT(pInfo == NULL); +// +// SCheckpointConsensusInfo p = {.genTs = taosGetTimestampMs(), .checkpointId = 0, .pTaskList = NULL}; +// taosHashPut(pHash, &streamId, sizeof(streamId), &p, sizeof(p)); +// +// SCheckpointConsensusInfo* pChkptInfo = (SCheckpointConsensusInfo*)taosHashGet(pHash, &streamId, sizeof(streamId)); +// ASSERT(pChkptInfo->genTs > 0 && pChkptInfo->checkpointId == 0); +// mDebug("s-task:0x%" PRIx64 " set the initial consensus-checkpointId:0", streamId); +// return TSDB_CODE_SUCCESS; +//} \ No newline at end of file diff --git a/source/dnode/snode/src/snode.c b/source/dnode/snode/src/snode.c index 9686fd3789..69a7bc7ba4 100644 --- a/source/dnode/snode/src/snode.c +++ b/source/dnode/snode/src/snode.c @@ -43,14 +43,14 @@ int32_t sndBuildStreamTask(SSnode *pSnode, SStreamTask *pTask, int64_t nextProce char *p = streamTaskGetStatus(pTask)->name; if (pTask->info.fillHistory) { - sndInfo("vgId:%d expand stream task, s-task:%s, checkpointId:%" PRId64 " checkpointVer:%" PRId64 + sndInfo("vgId:%d build stream task, s-task:%s, checkpointId:%" PRId64 " checkpointVer:%" PRId64 " nextProcessVer:%" PRId64 " child id:%d, level:%d, status:%s fill-history:%d, related stream task:0x%x trigger:%" PRId64 " ms", SNODE_HANDLE, pTask->id.idStr, pChkInfo->checkpointId, pChkInfo->checkpointVer, pChkInfo->nextProcessVer, pTask->info.selfChildId, pTask->info.taskLevel, p, pTask->info.fillHistory, (int32_t)pTask->streamTaskId.taskId, pTask->info.delaySchedParam); } else { - sndInfo("vgId:%d expand stream task, s-task:%s, checkpointId:%" PRId64 " checkpointVer:%" PRId64 + sndInfo("vgId:%d build stream task, s-task:%s, checkpointId:%" PRId64 " checkpointVer:%" PRId64 " nextProcessVer:%" PRId64 " child id:%d, level:%d, status:%s fill-history:%d, related fill-task:0x%x trigger:%" PRId64 " ms", SNODE_HANDLE, pTask->id.idStr, pChkInfo->checkpointId, pChkInfo->checkpointVer, pChkInfo->nextProcessVer, diff --git a/source/dnode/vnode/src/tqCommon/tqCommon.c b/source/dnode/vnode/src/tqCommon/tqCommon.c index 668e178d2d..04a658a30c 100644 --- a/source/dnode/vnode/src/tqCommon/tqCommon.c +++ b/source/dnode/vnode/src/tqCommon/tqCommon.c @@ -279,7 +279,10 @@ int32_t tqStreamTaskProcessUpdateReq(SStreamMeta* pMeta, SMsgCb* cb, SRpcMsg* pM int32_t numOfTasks = streamMetaGetNumOfTasks(pMeta); int32_t updateTasks = taosHashGetSize(pMeta->updateInfo.pTasks); - pMeta->startInfo.tasksWillRestart = 1; + if (restored) { + tqDebug("vgId:%d s-task:0x%x update epset transId:%d, set the restart flag", vgId, req.taskId, req.transId); + pMeta->startInfo.tasksWillRestart = 1; + } if (updateTasks < numOfTasks) { tqDebug("vgId:%d closed tasks:%d, unclosed:%d, all tasks will be started when nodeEp update completed", vgId, @@ -292,8 +295,7 @@ int32_t tqStreamTaskProcessUpdateReq(SStreamMeta* pMeta, SMsgCb* cb, SRpcMsg* pM streamMetaClearUpdateTaskList(pMeta); if (!restored) { - tqDebug("vgId:%d vnode restore not completed, not start the tasks, clear the start after nodeUpdate flag", vgId); - pMeta->startInfo.tasksWillRestart = 0; + tqDebug("vgId:%d vnode restore not completed, not start all tasks", vgId); } else { tqDebug("vgId:%d all %d task(s) nodeEp updated and closed, transId:%d", vgId, numOfTasks, req.transId); #if 0 diff --git a/source/libs/stream/src/streamCheckStatus.c b/source/libs/stream/src/streamCheckStatus.c index 8778e3314a..22d336a549 100644 --- a/source/libs/stream/src/streamCheckStatus.c +++ b/source/libs/stream/src/streamCheckStatus.c @@ -666,7 +666,11 @@ void rspMonitorFn(void* param, void* tmrId) { stDebug("s-task:%s status:%s vgId:%d quit from monitor check-rsp tmr, ref:%d", id, pStat->name, vgId, ref); streamTaskCompleteCheckRsp(pInfo, true, id); - addDownstreamFailedStatusResultAsync(pTask->pMsgCb, vgId, pTask->id.streamId, pTask->id.taskId); + + // not record the failed of the current task if try to close current vnode + if (!pMeta->closeFlag) { + addDownstreamFailedStatusResultAsync(pTask->pMsgCb, vgId, pTask->id.streamId, pTask->id.taskId); + } streamMetaReleaseTask(pMeta, pTask); return; diff --git a/source/libs/stream/src/streamCheckpoint.c b/source/libs/stream/src/streamCheckpoint.c index bc973f17d7..1195362ab3 100644 --- a/source/libs/stream/src/streamCheckpoint.c +++ b/source/libs/stream/src/streamCheckpoint.c @@ -447,14 +447,6 @@ int32_t streamTaskUpdateTaskCheckpointInfo(SStreamTask* pTask, bool restored, SV SStreamTaskState* pStatus = streamTaskGetStatus(pTask); - if (restored && (pStatus->state != TASK_STATUS__CK) && (pMeta->role == NODE_ROLE_LEADER)) { - stDebug("s-task:0x%x vgId:%d restored:%d status:%s not update checkpoint-info, checkpointId:%" PRId64 "->%" PRId64 - " failed", - pReq->taskId, vgId, restored, pStatus->name, pInfo->checkpointId, pReq->checkpointId); - taosThreadMutexUnlock(&pTask->lock); - return TSDB_CODE_STREAM_TASK_IVLD_STATUS; - } - if (!restored) { // during restore procedure, do update checkpoint-info stDebug("s-task:%s vgId:%d status:%s update the checkpoint-info during restore, checkpointId:%" PRId64 "->%" PRId64 " checkpointVer:%" PRId64 "->%" PRId64 " checkpointTs:%" PRId64 "->%" PRId64, diff --git a/source/libs/stream/src/streamHb.c b/source/libs/stream/src/streamHb.c index d6411e25f2..691ec44672 100644 --- a/source/libs/stream/src/streamHb.c +++ b/source/libs/stream/src/streamHb.c @@ -168,7 +168,9 @@ int32_t streamMetaSendHbHelper(SStreamMeta* pMeta) { continue; } + taosThreadMutexLock(&(*pTask)->lock); STaskStatusEntry entry = streamTaskGetStatusEntry(*pTask); + taosThreadMutexUnlock(&(*pTask)->lock); entry.inputRate = entry.inputQUsed * 100.0 / (2 * STREAM_TASK_QUEUE_CAPACITY_IN_SIZE); if ((*pTask)->info.taskLevel == TASK_LEVEL__SINK) { From 47b0a0464e62e1458a87799ac5800b451929fd4d Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 4 Jul 2024 15:21:16 +0800 Subject: [PATCH 03/22] fix(stream): send checkpoint-source-rsp to mnode before reset task status. --- include/libs/stream/tstream.h | 1 + source/dnode/mnode/impl/src/mndStream.c | 3 --- source/dnode/vnode/src/tqCommon/tqCommon.c | 4 +++- source/libs/stream/src/streamCheckpoint.c | 16 ++++++++++++++++ 4 files changed, 20 insertions(+), 4 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 66b9db47e2..f24f7da7c3 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -801,6 +801,7 @@ int32_t streamTaskBroadcastRetrieveReq(SStreamTask* pTask, SStreamRetrieveReq* r void streamTaskSendRetrieveRsp(SStreamRetrieveReq* pReq, SRpcMsg* pRsp); int32_t streamProcessHeartbeatRsp(SStreamMeta* pMeta, SMStreamHbRspMsg* pRsp); +int32_t streamTaskSendPreparedCheckpointsourceRsp(SStreamTask* pTask); #ifdef __cplusplus diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index a4c03ab3e0..536dfab331 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -1239,9 +1239,6 @@ static int32_t mndProcessStreamCheckpoint(SRpcMsg *pReq) { code = mndProcessStreamCheckpointTrans(pMnode, p, checkpointId, 1, true); sdbRelease(pSdb, p); - // clear the consensus checkpoint info - mndClearConsensusCheckpointId(execInfo.pStreamConsensus, p->uid); - if (code != -1) { started += 1; diff --git a/source/dnode/vnode/src/tqCommon/tqCommon.c b/source/dnode/vnode/src/tqCommon/tqCommon.c index 04a658a30c..1999134754 100644 --- a/source/dnode/vnode/src/tqCommon/tqCommon.c +++ b/source/dnode/vnode/src/tqCommon/tqCommon.c @@ -228,6 +228,9 @@ int32_t tqStreamTaskProcessUpdateReq(SStreamMeta* pMeta, SMsgCb* cb, SRpcMsg* pM } updated = streamTaskUpdateEpsetInfo(pTask, req.pNodeList); + + // send the checkpoint-source-rsp for source task to end the checkpoint trans in mnode + streamTaskSendPreparedCheckpointsourceRsp(pTask); streamTaskResetStatus(pTask); streamTaskStopMonitorCheckRsp(&pTask->taskCheckInfo, pTask->id.idStr); @@ -264,7 +267,6 @@ int32_t tqStreamTaskProcessUpdateReq(SStreamMeta* pMeta, SMsgCb* cb, SRpcMsg* pM tqDebug("s-task:%s vgId:%d not save task since not update epset actually, stop task", idstr, vgId); } - // stop streamTaskStop(pTask); if (ppHTask != NULL) { streamTaskStop(*ppHTask); diff --git a/source/libs/stream/src/streamCheckpoint.c b/source/libs/stream/src/streamCheckpoint.c index 1195362ab3..969c2e1795 100644 --- a/source/libs/stream/src/streamCheckpoint.c +++ b/source/libs/stream/src/streamCheckpoint.c @@ -1146,4 +1146,20 @@ int32_t streamTaskSendRestoreChkptMsg(SStreamTask* pTask) { tmsgSendReq(&pTask->info.mnodeEpset, &msg); return 0; +} + +int32_t streamTaskSendPreparedCheckpointsourceRsp(SStreamTask* pTask) { + int32_t code = 0; + if (pTask->info.taskLevel != TASK_LEVEL__SOURCE) { + return code; + } + + taosThreadMutexLock(&pTask->lock); + SStreamTaskState* p = streamTaskGetStatus(pTask); + if (p->state == TASK_STATUS__CK) { + code = streamTaskSendCheckpointSourceRsp(pTask); + } + taosThreadMutexUnlock(&pTask->lock); + + return code; } \ No newline at end of file From c35c634977390d64165a9f4bd03b0ccfc1855f8a Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 4 Jul 2024 17:08:32 +0800 Subject: [PATCH 04/22] fix(stream): add flag to disable the concurrently started consensus-checkpointId procedure. --- include/libs/stream/tstream.h | 4 ++-- source/libs/stream/src/streamCheckpoint.c | 11 +++++++++++ source/libs/stream/src/streamMeta.c | 23 +++++++++++++++-------- 3 files changed, 28 insertions(+), 10 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index f24f7da7c3..093a21c999 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -272,9 +272,8 @@ typedef struct SCheckpointInfo { int64_t checkpointTime; // latest checkpoint time int64_t processedVer; int64_t nextProcessVer; // current offset in WAL, not serialize it - + int64_t msgVer; SActiveCheckpointInfo* pActiveInfo; - int64_t msgVer; } SCheckpointInfo; typedef struct SStreamStatus { @@ -289,6 +288,7 @@ typedef struct SStreamStatus { int32_t inScanHistorySentinel; bool appendTranstateBlock; // has append the transfer state data block already bool removeBackendFiles; // remove backend files on disk when free stream tasks + bool sendConsensusChkptId; } SStreamStatus; typedef struct SDataRange { diff --git a/source/libs/stream/src/streamCheckpoint.c b/source/libs/stream/src/streamCheckpoint.c index 969c2e1795..0f39ca7213 100644 --- a/source/libs/stream/src/streamCheckpoint.c +++ b/source/libs/stream/src/streamCheckpoint.c @@ -1107,6 +1107,17 @@ int32_t streamTaskSendRestoreChkptMsg(SStreamTask* pTask) { const char* id = pTask->id.idStr; SCheckpointInfo* pInfo = &pTask->chkInfo; + taosThreadMutexLock(&pTask->lock); + if (pTask->status.sendConsensusChkptId == true) { + stDebug("s-task:%s already start to consensus-checkpointId, not start again before it completed", id); + taosThreadMutexUnlock(&pTask->lock); + return TSDB_CODE_SUCCESS; + } else { + pTask->status.sendConsensusChkptId = true; + } + + taosThreadMutexUnlock(&pTask->lock); + ASSERT(pTask->pBackend == NULL); SRestoreCheckpointInfo req = { diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 7b94f642e2..19cb2f7854 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -1198,7 +1198,7 @@ int32_t streamMetaStartAllTasks(SStreamMeta* pMeta) { continue; } - if ((pTask->pBackend == NULL) && (pTask->info.fillHistory == 1 || HAS_RELATED_FILLHISTORY_TASK(pTask))) { + if ((pTask->pBackend == NULL) && ((pTask->info.fillHistory == 1) || HAS_RELATED_FILLHISTORY_TASK(pTask))) { code = pMeta->expandTaskFn(pTask); if (code != TSDB_CODE_SUCCESS) { stError("s-task:0x%x vgId:%d failed to expand stream backend", pTaskId->taskId, vgId); @@ -1392,17 +1392,24 @@ int32_t streamMetaAddTaskLaunchResult(SStreamMeta* pMeta, int64_t streamId, int3 streamMetaWLock(pMeta); - if (pStartInfo->startAllTasks != 1) { - int64_t el = endTs - startTs; - stDebug("vgId:%d not start all task(s), not record status, s-task:0x%x launch succ:%d elapsed time:%" PRId64 "ms", - pMeta->vgId, taskId, ready, el); + SStreamTask** p = taosHashGet(pMeta->pTasksMap, &id, sizeof(id)); + if (p == NULL) { // task does not exists in current vnode, not record the complete info + stError("vgId:%d s-task:0x%x not exists discard the check downstream info", pMeta->vgId, taskId); streamMetaWUnLock(pMeta); return 0; } - void* p = taosHashGet(pMeta->pTasksMap, &id, sizeof(id)); - if (p == NULL) { // task does not exists in current vnode, not record the complete info - stError("vgId:%d s-task:0x%x not exists discard the check downstream info", pMeta->vgId, taskId); + // clear the send consensus-checkpointId flag + taosThreadMutexLock(&(*p)->lock); + (*p)->status.sendConsensusChkptId = false; + taosThreadMutexUnlock(&(*p)->lock); + + if (pStartInfo->startAllTasks != 1) { + int64_t el = endTs - startTs; + stDebug( + "vgId:%d not in start all task(s) process, not record launch result status, s-task:0x%x launch succ:%d elapsed " + "time:%" PRId64 "ms", + pMeta->vgId, taskId, ready, el); streamMetaWUnLock(pMeta); return 0; } From e6defda0d06bfee1841cbab56a7daa2d567895d3 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 4 Jul 2024 17:48:58 +0800 Subject: [PATCH 05/22] fix(stream): check for checkpoint interrpution in sendReady monitor. --- source/libs/stream/src/streamCheckpoint.c | 6 ++++-- source/libs/stream/src/streamDispatch.c | 13 ++++++++++++- 2 files changed, 16 insertions(+), 3 deletions(-) diff --git a/source/libs/stream/src/streamCheckpoint.c b/source/libs/stream/src/streamCheckpoint.c index 0f39ca7213..d5f7d6ef21 100644 --- a/source/libs/stream/src/streamCheckpoint.c +++ b/source/libs/stream/src/streamCheckpoint.c @@ -405,12 +405,14 @@ int32_t streamTaskProcessCheckpointReadyRsp(SStreamTask* pTask, int32_t upstream void streamTaskClearCheckInfo(SStreamTask* pTask, bool clearChkpReadyMsg) { pTask->chkInfo.startTs = 0; // clear the recorded start time - - streamTaskClearActiveInfo(pTask->chkInfo.pActiveInfo); streamTaskOpenAllUpstreamInput(pTask); // open inputQ for all upstream tasks + + taosThreadMutexLock(&pTask->chkInfo.pActiveInfo->lock); + streamTaskClearActiveInfo(pTask->chkInfo.pActiveInfo); if (clearChkpReadyMsg) { streamClearChkptReadyMsg(pTask->chkInfo.pActiveInfo); } + taosThreadMutexUnlock(&pTask->chkInfo.pActiveInfo->lock); } int32_t streamTaskUpdateTaskCheckpointInfo(SStreamTask* pTask, bool restored, SVUpdateCheckpointInfoReq* pReq) { diff --git a/source/libs/stream/src/streamDispatch.c b/source/libs/stream/src/streamDispatch.c index 83e73e8c88..5164e20ec9 100644 --- a/source/libs/stream/src/streamDispatch.c +++ b/source/libs/stream/src/streamDispatch.c @@ -813,9 +813,20 @@ static void checkpointReadyMsgSendMonitorFn(void* param, void* tmrId) { taosThreadMutexLock(&pActiveInfo->lock); SArray* pList = pActiveInfo->pReadyMsgList; + int32_t num = taosArrayGetSize(pList); + + // active checkpoint info is cleared for now + if ((pActiveInfo->activeId == 0) && (pActiveInfo->transId == 0) && (num == 0) && (pTask->chkInfo.startTs == 0)) { + taosThreadMutexUnlock(&pActiveInfo->lock); + int32_t ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1); + stWarn("s-task:0x%x vgId:%d active checkpoint may failed, quit from readyMsg send tmr, ref:%d", id, vgId, ref); + + streamMetaReleaseTask(pTask->pMeta, pTask); + return; + } + SArray* pNotRspList = taosArrayInit(4, sizeof(int32_t)); - int32_t num = taosArrayGetSize(pList); ASSERT(taosArrayGetSize(pTask->upstreamInfo.pList) == num); for (int32_t i = 0; i < num; ++i) { From 998421e5ad79315892fe5ef639e2baf2f30b6237 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 4 Jul 2024 17:49:59 +0800 Subject: [PATCH 06/22] fix(stream): update some logs. --- source/libs/stream/src/streamDispatch.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/libs/stream/src/streamDispatch.c b/source/libs/stream/src/streamDispatch.c index 5164e20ec9..8bf0ccca53 100644 --- a/source/libs/stream/src/streamDispatch.c +++ b/source/libs/stream/src/streamDispatch.c @@ -819,7 +819,7 @@ static void checkpointReadyMsgSendMonitorFn(void* param, void* tmrId) { if ((pActiveInfo->activeId == 0) && (pActiveInfo->transId == 0) && (num == 0) && (pTask->chkInfo.startTs == 0)) { taosThreadMutexUnlock(&pActiveInfo->lock); int32_t ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1); - stWarn("s-task:0x%x vgId:%d active checkpoint may failed, quit from readyMsg send tmr, ref:%d", id, vgId, ref); + stWarn("s-task:0x%x vgId:%d active checkpoint may be cleared, quit from readyMsg send tmr, ref:%d", id, vgId, ref); streamMetaReleaseTask(pTask->pMeta, pTask); return; From db4a00c74ed4ac0f58e360af579ce8ec5580884f Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Fri, 5 Jul 2024 08:55:40 +0800 Subject: [PATCH 07/22] fix(stream): not restart for reset task status. --- source/dnode/mnode/impl/src/mndStream.c | 8 ++++---- source/dnode/vnode/src/tqCommon/tqCommon.c | 8 ++++---- source/libs/stream/src/streamCheckStatus.c | 3 ++- 3 files changed, 10 insertions(+), 9 deletions(-) diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index 536dfab331..f5b944de45 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -2701,8 +2701,8 @@ static int32_t mndProcessConsensusCheckpointId(SRpcMsg *pMsg) { } } - mInfo("vgId:%d meta-stored checkpointId for stream:0x%" PRIx64 " %s is:%" PRId64, req.nodeId, req.streamId, - pStream->name, pStream->checkpointId); + mInfo("vgId:%d stream:0x%" PRIx64 " %s meta-stored checkpointId:%" PRId64 " stream:0x%" PRIx64 " %s", req.nodeId, + req.streamId, pStream->name, pStream->checkpointId); int32_t numOfTasks = (pStream == NULL) ? 0 : mndGetNumOfStreamTasks(pStream); if ((pStream != NULL) && (pStream->checkpointId == 0)) { // not generated checkpoint yet, return 0 directly @@ -2730,8 +2730,8 @@ static int32_t mndProcessConsensusCheckpointId(SRpcMsg *pMsg) { } if (chkId == req.checkpointId) { - mDebug("vgId:%d stream:0x%" PRIx64 " %s consensus-checkpointId is:%" PRId64, req.nodeId, req.streamId, - pStream->name, pStream->checkpointId); + mDebug("vgId:%d stream:0x%" PRIx64 " %s consensus-checkpointId is:%" PRId64 ", meta-stored checkpointId:%" PRId64, + req.nodeId, req.streamId, pStream->name, chkId, pStream->checkpointId); mndSendQuickConsensusChkptIdRsp(&req, TSDB_CODE_SUCCESS, req.streamId, chkId, &pMsg->info); taosThreadMutexUnlock(&execInfo.lock); diff --git a/source/dnode/vnode/src/tqCommon/tqCommon.c b/source/dnode/vnode/src/tqCommon/tqCommon.c index 1999134754..c40332ff39 100644 --- a/source/dnode/vnode/src/tqCommon/tqCommon.c +++ b/source/dnode/vnode/src/tqCommon/tqCommon.c @@ -871,7 +871,6 @@ int32_t tqStreamTaskProcessTaskResetReq(SStreamMeta* pMeta, SRpcMsg* pMsg) { tqDebug("s-task:%s receive task-reset msg from mnode, reset status and ready for data processing", pTask->id.idStr); taosThreadMutexLock(&pTask->lock); - streamTaskClearCheckInfo(pTask, true); // clear flag set during do checkpoint, and open inputQ for all upstream tasks @@ -886,9 +885,10 @@ int32_t tqStreamTaskProcessTaskResetReq(SStreamMeta* pMeta, SRpcMsg* pMsg) { streamTaskSetStatusReady(pTask); } else if (pState->state == TASK_STATUS__UNINIT) { - tqDebug("s-task:%s start task by checking downstream tasks", pTask->id.idStr); - ASSERT(pTask->status.downstreamReady == 0); - tqStreamTaskRestoreCheckpoint(pMeta, pTask->id.streamId, pTask->id.taskId); +// tqDebug("s-task:%s start task by checking downstream tasks", pTask->id.idStr); +// ASSERT(pTask->status.downstreamReady == 0); +// tqStreamTaskRestoreCheckpoint(pMeta, pTask->id.streamId, pTask->id.taskId); + tqDebug("s-task:%s status:%s do nothing after receiving reset-task from mnode", pTask->id.idStr, pState->name); } else { tqDebug("s-task:%s status:%s do nothing after receiving reset-task from mnode", pTask->id.idStr, pState->name); } diff --git a/source/libs/stream/src/streamCheckStatus.c b/source/libs/stream/src/streamCheckStatus.c index 22d336a549..2d8fe4a0e1 100644 --- a/source/libs/stream/src/streamCheckStatus.c +++ b/source/libs/stream/src/streamCheckStatus.c @@ -668,6 +668,7 @@ void rspMonitorFn(void* param, void* tmrId) { streamTaskCompleteCheckRsp(pInfo, true, id); // not record the failed of the current task if try to close current vnode + // otherwise, the put of message operation may incur invalid read of message queue. if (!pMeta->closeFlag) { addDownstreamFailedStatusResultAsync(pTask->pMsgCb, vgId, pTask->id.streamId, pTask->id.taskId); } @@ -676,7 +677,7 @@ void rspMonitorFn(void* param, void* tmrId) { return; } - if (state == TASK_STATUS__DROPPING || state == TASK_STATUS__READY || state == TASK_STATUS__PAUSE) { + if (state == TASK_STATUS__DROPPING || state == TASK_STATUS__READY) { int32_t ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1); stDebug("s-task:%s status:%s vgId:%d quit from monitor check-rsp tmr, ref:%d", id, pStat->name, vgId, ref); From 78993d9c55cef8bbbb6346f15a3da1ac6c8d022c Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Fri, 5 Jul 2024 09:00:01 +0800 Subject: [PATCH 08/22] fix(stream): update logs. --- source/dnode/mnode/impl/src/mndStream.c | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index f5b944de45..a1016ad96c 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -2701,8 +2701,8 @@ static int32_t mndProcessConsensusCheckpointId(SRpcMsg *pMsg) { } } - mInfo("vgId:%d stream:0x%" PRIx64 " %s meta-stored checkpointId:%" PRId64 " stream:0x%" PRIx64 " %s", req.nodeId, - req.streamId, pStream->name, pStream->checkpointId); + mInfo("vgId:%d stream:0x%" PRIx64 " %s meta-stored checkpointId:%" PRId64, req.nodeId, req.streamId, pStream->name, + pStream->checkpointId); int32_t numOfTasks = (pStream == NULL) ? 0 : mndGetNumOfStreamTasks(pStream); if ((pStream != NULL) && (pStream->checkpointId == 0)) { // not generated checkpoint yet, return 0 directly From 8e6bb176c21675e8eb0d762347f931784d7c6403 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Fri, 5 Jul 2024 16:17:20 +0800 Subject: [PATCH 09/22] fix(stream): use trans to set the consensus-checkpoint id --- include/common/tmsgdef.h | 3 +- include/dnode/vnode/tqCommon.h | 7 +- include/libs/stream/streamMsg.h | 10 -- include/libs/stream/tstream.h | 3 +- source/dnode/mgmt/mgmt_mnode/src/mmHandle.c | 3 +- source/dnode/mgmt/mgmt_snode/src/smHandle.c | 3 +- source/dnode/mgmt/mgmt_vnode/src/vmHandle.c | 3 +- source/dnode/mnode/impl/inc/mndStream.h | 9 +- source/dnode/mnode/impl/src/mndStream.c | 120 ++++++++++---------- source/dnode/mnode/impl/src/mndStreamUtil.c | 102 +++++++++++++---- source/dnode/snode/src/snode.c | 10 +- source/dnode/vnode/src/inc/vnodeInt.h | 1 + source/dnode/vnode/src/tq/tq.c | 10 +- source/dnode/vnode/src/tqCommon/tqCommon.c | 25 ++-- source/dnode/vnode/src/vnd/vnodeSvr.c | 12 +- source/libs/stream/src/streamCheckpoint.c | 2 +- source/libs/stream/src/streamMsg.c | 20 ---- 17 files changed, 193 insertions(+), 150 deletions(-) diff --git a/include/common/tmsgdef.h b/include/common/tmsgdef.h index 19fe34fe01..7621615278 100644 --- a/include/common/tmsgdef.h +++ b/include/common/tmsgdef.h @@ -250,7 +250,7 @@ TD_DEF_MSG_TYPE(TDMT_MND_DROP_TB_WITH_TSMA, "drop-tb-with-tsma", NULL, NULL) TD_DEF_MSG_TYPE(TDMT_MND_STREAM_UPDATE_CHKPT_EVT, "stream-update-chkpt-evt", NULL, NULL) TD_DEF_MSG_TYPE(TDMT_MND_STREAM_CHKPT_REPORT, "stream-chkpt-report", NULL, NULL) - TD_DEF_MSG_TYPE(TDMT_MND_STREAM_CHKPT_CONSEN, "stream-chkpt-consen", NULL, NULL) + TD_DEF_MSG_TYPE(TDMT_MND_STREAM_REQ_CONSEN_CHKPT, "stream-req-consen-chkpt", NULL, NULL) TD_DEF_MSG_TYPE(TDMT_MND_STREAM_CONSEN_TIMER, "stream-consen-tmr", NULL, NULL) TD_DEF_MSG_TYPE(TDMT_MND_MAX_MSG, "mnd-max", NULL, NULL) TD_CLOSE_MSG_SEG(TDMT_END_MND_MSG) @@ -342,6 +342,7 @@ TD_DEF_MSG_TYPE(TDMT_STREAM_CREATE, "stream-create", NULL, NULL) TD_DEF_MSG_TYPE(TDMT_STREAM_DROP, "stream-drop", NULL, NULL) TD_DEF_MSG_TYPE(TDMT_STREAM_RETRIEVE_TRIGGER, "stream-retri-trigger", NULL, NULL) + TD_DEF_MSG_TYPE(TDMT_STREAM_CONSEN_CHKPT, "stream-consen-chkpt", NULL, NULL) TD_CLOSE_MSG_SEG(TDMT_STREAM_MSG) TD_NEW_MSG_SEG(TDMT_MON_MSG) //5 << 8 diff --git a/include/dnode/vnode/tqCommon.h b/include/dnode/vnode/tqCommon.h index 566e8dbbd8..4d5e18520c 100644 --- a/include/dnode/vnode/tqCommon.h +++ b/include/dnode/vnode/tqCommon.h @@ -29,7 +29,8 @@ int32_t tqStreamTaskProcessCheckpointReadyMsg(SStreamMeta* pMeta, SRpcMsg* pMsg) int32_t tqStreamProcessStreamHbRsp(SStreamMeta* pMeta, SRpcMsg* pMsg); int32_t tqStreamProcessReqCheckpointRsp(SStreamMeta* pMeta, SRpcMsg* pMsg); int32_t tqStreamProcessChkptReportRsp(SStreamMeta* pMeta, SRpcMsg* pMsg); -int32_t tqStreamProcessConsensusChkptRsp(SStreamMeta* pMeta, SRpcMsg* pMsg); +int32_t tqStreamProcessConsensusChkptRsp2(SStreamMeta* pMeta, SRpcMsg* pMsg); +int32_t tqStreamTaskProcessConsenChkptIdReq(SStreamMeta* pMeta, SRpcMsg* pMsg); int32_t tqStreamProcessCheckpointReadyRsp(SStreamMeta* pMeta, SRpcMsg* pMsg); int32_t tqStreamTaskProcessDeployReq(SStreamMeta* pMeta, SMsgCb* cb, int64_t sversion, char* msg, int32_t msgLen, bool isLeader, bool restored); @@ -37,12 +38,12 @@ int32_t tqStreamTaskProcessDropReq(SStreamMeta* pMeta, char* msg, int32_t msgLen int32_t tqStreamTaskProcessRunReq(SStreamMeta* pMeta, SRpcMsg* pMsg, bool isLeader); int32_t tqStartTaskCompleteCallback(SStreamMeta* pMeta); int32_t tqStreamTasksGetTotalNum(SStreamMeta* pMeta); -int32_t tqStreamTaskProcessTaskResetReq(SStreamMeta* pMeta, SRpcMsg* pMsg); +int32_t tqStreamTaskProcessTaskResetReq(SStreamMeta* pMeta, char* msg); int32_t tqStreamTaskProcessRetrieveTriggerReq(SStreamMeta* pMeta, SRpcMsg* pMsg); int32_t tqStreamTaskProcessRetrieveTriggerRsp(SStreamMeta* pMeta, SRpcMsg* pMsg); int32_t tqStreamTaskProcessTaskPauseReq(SStreamMeta* pMeta, char* pMsg); int32_t tqStreamTaskProcessTaskResumeReq(void* handle, int64_t sversion, char* pMsg, bool fromVnode); -int32_t tqStreamTaskProcessUpdateCheckpointReq(SStreamMeta* pMeta, bool restored, char* msg, int32_t msgLen); +int32_t tqStreamTaskProcessUpdateCheckpointReq(SStreamMeta* pMeta, bool restored, char* msg); void tqSetRestoreVersionInfo(SStreamTask* pTask); int32_t tqExpandStreamTask(SStreamTask* pTask); diff --git a/include/libs/stream/streamMsg.h b/include/libs/stream/streamMsg.h index 8b6ca2c5cd..bdb8ff7f8e 100644 --- a/include/libs/stream/streamMsg.h +++ b/include/libs/stream/streamMsg.h @@ -223,16 +223,6 @@ typedef struct SRestoreCheckpointInfo { int32_t tEncodeRestoreCheckpointInfo (SEncoder* pEncoder, const SRestoreCheckpointInfo* pReq); int32_t tDecodeRestoreCheckpointInfo(SDecoder* pDecoder, SRestoreCheckpointInfo* pReq); -typedef struct SRestoreCheckpointInfoRsp { - int64_t streamId; - int64_t checkpointId; - int64_t startTs; - int32_t taskId; -} SRestoreCheckpointInfoRsp; - -int32_t tEncodeRestoreCheckpointInfoRsp(SEncoder* pCoder, const SRestoreCheckpointInfoRsp* pInfo); -int32_t tDecodeRestoreCheckpointInfoRsp(SDecoder* pCoder, SRestoreCheckpointInfoRsp* pInfo); - typedef struct { SMsgHead head; int64_t streamId; diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 093a21c999..d0feebf814 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -616,9 +616,8 @@ typedef struct SStreamTaskState { typedef struct SCheckpointConsensusInfo { SArray* pTaskList; -// int64_t checkpointId; -// int64_t genTs; int32_t numOfTasks; + int64_t streamId; } SCheckpointConsensusInfo; int32_t streamSetupScheduleTrigger(SStreamTask* pTask); diff --git a/source/dnode/mgmt/mgmt_mnode/src/mmHandle.c b/source/dnode/mgmt/mgmt_mnode/src/mmHandle.c index 677e19d4c1..f3763ef0c5 100644 --- a/source/dnode/mgmt/mgmt_mnode/src/mmHandle.c +++ b/source/dnode/mgmt/mgmt_mnode/src/mmHandle.c @@ -233,6 +233,7 @@ SArray *mmGetMsgHandles() { if (dmSetMgmtHandle(pArray, TDMT_STREAM_TASK_RESUME_RSP, mmPutMsgToWriteQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_STREAM_TASK_STOP_RSP, mmPutMsgToWriteQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_STREAM_TASK_UPDATE_CHKPT_RSP, mmPutMsgToWriteQueue, 0) == NULL) goto _OVER; + if (dmSetMgmtHandle(pArray, TDMT_STREAM_CONSEN_CHKPT_RSP, mmPutMsgToWriteQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_STREAM_CREATE, mmPutMsgToWriteQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_STREAM_DROP, mmPutMsgToWriteQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_STREAM_CREATE_RSP, mmPutMsgToWriteQueue, 0) == NULL) goto _OVER; @@ -242,7 +243,7 @@ SArray *mmGetMsgHandles() { if (dmSetMgmtHandle(pArray, TDMT_VND_STREAM_TASK_RESET_RSP, mmPutMsgToWriteQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_MND_STREAM_HEARTBEAT, mmPutMsgToReadQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_MND_STREAM_CHKPT_REPORT, mmPutMsgToWriteQueue, 0) == NULL) goto _OVER; - if (dmSetMgmtHandle(pArray, TDMT_MND_STREAM_CHKPT_CONSEN, mmPutMsgToWriteQueue, 0) == NULL) goto _OVER; + if (dmSetMgmtHandle(pArray, TDMT_MND_STREAM_REQ_CONSEN_CHKPT, mmPutMsgToWriteQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_MND_STREAM_REQ_CHKPT, mmPutMsgToWriteQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_VND_KILL_COMPACT_RSP, mmPutMsgToWriteQueue, 0) == NULL) goto _OVER; diff --git a/source/dnode/mgmt/mgmt_snode/src/smHandle.c b/source/dnode/mgmt/mgmt_snode/src/smHandle.c index 7a0189b7c1..3d0587a11b 100644 --- a/source/dnode/mgmt/mgmt_snode/src/smHandle.c +++ b/source/dnode/mgmt/mgmt_snode/src/smHandle.c @@ -76,6 +76,7 @@ SArray *smGetMsgHandles() { if (dmSetMgmtHandle(pArray, TDMT_VND_STREAM_TASK_UPDATE, smPutNodeMsgToMgmtQueue, 1) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_STREAM_TASK_DEPLOY, smPutNodeMsgToMgmtQueue, 1) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_STREAM_TASK_UPDATE_CHKPT, smPutNodeMsgToMgmtQueue, 1) == NULL) goto _OVER; + if (dmSetMgmtHandle(pArray, TDMT_STREAM_CONSEN_CHKPT, smPutNodeMsgToMgmtQueue, 1) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_STREAM_TASK_DROP, smPutNodeMsgToMgmtQueue, 1) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_STREAM_TASK_RUN, smPutNodeMsgToStreamQueue, 1) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_STREAM_TASK_DISPATCH, smPutNodeMsgToStreamQueue, 1) == NULL) goto _OVER; @@ -96,7 +97,7 @@ SArray *smGetMsgHandles() { if (dmSetMgmtHandle(pArray, TDMT_MND_STREAM_HEARTBEAT_RSP, smPutNodeMsgToStreamQueue, 1) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_MND_STREAM_REQ_CHKPT_RSP, smPutNodeMsgToStreamQueue, 1) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_MND_STREAM_CHKPT_REPORT_RSP, smPutNodeMsgToStreamQueue, 1) == NULL) goto _OVER; - if (dmSetMgmtHandle(pArray, TDMT_MND_STREAM_CHKPT_CONSEN_RSP, smPutNodeMsgToMgmtQueue, 1) == NULL) goto _OVER; + if (dmSetMgmtHandle(pArray, TDMT_MND_STREAM_REQ_CONSEN_CHKPT_RSP, smPutNodeMsgToStreamQueue, 1) == NULL) goto _OVER; code = 0; _OVER: diff --git a/source/dnode/mgmt/mgmt_vnode/src/vmHandle.c b/source/dnode/mgmt/mgmt_vnode/src/vmHandle.c index 001696aecc..7d35fd71b7 100644 --- a/source/dnode/mgmt/mgmt_vnode/src/vmHandle.c +++ b/source/dnode/mgmt/mgmt_vnode/src/vmHandle.c @@ -972,10 +972,11 @@ SArray *vmGetMsgHandles() { if (dmSetMgmtHandle(pArray, TDMT_MND_STREAM_HEARTBEAT_RSP, vmPutMsgToStreamQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_MND_STREAM_REQ_CHKPT_RSP, vmPutMsgToStreamQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_MND_STREAM_CHKPT_REPORT_RSP, vmPutMsgToStreamQueue, 0) == NULL) goto _OVER; - if (dmSetMgmtHandle(pArray, TDMT_MND_STREAM_CHKPT_CONSEN_RSP, vmPutMsgToWriteQueue, 0) == NULL) goto _OVER; + if (dmSetMgmtHandle(pArray, TDMT_MND_STREAM_REQ_CONSEN_CHKPT_RSP, vmPutMsgToStreamQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_VND_GET_STREAM_PROGRESS, vmPutMsgToStreamQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_STREAM_TASK_UPDATE_CHKPT, vmPutMsgToWriteQueue, 0) == NULL) goto _OVER; + if (dmSetMgmtHandle(pArray, TDMT_STREAM_CONSEN_CHKPT, vmPutMsgToWriteQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_VND_ALTER_REPLICA, vmPutMsgToMgmtQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_VND_ALTER_CONFIG, vmPutMsgToWriteQueue, 0) == NULL) goto _OVER; diff --git a/source/dnode/mnode/impl/inc/mndStream.h b/source/dnode/mnode/impl/inc/mndStream.h index a86e06b486..7b4270462f 100644 --- a/source/dnode/mnode/impl/inc/mndStream.h +++ b/source/dnode/mnode/impl/inc/mndStream.h @@ -83,7 +83,7 @@ typedef struct SOrphanTask { typedef struct { SMsgHead head; -} SMStreamReqCheckpointRsp, SMStreamUpdateChkptRsp; +} SMStreamReqCheckpointRsp, SMStreamUpdateChkptRsp, SMStreamReqConsensChkptRsp; typedef struct STaskChkptInfo { int32_t nodeId; @@ -133,9 +133,8 @@ int32_t mndCreateStreamResetStatusTrans(SMnode *pMnode, SStreamObj *pStream) int32_t mndStreamSetUpdateChkptAction(SMnode *pMnode, STrans *pTrans, SStreamObj *pStream); int32_t mndCreateStreamChkptInfoUpdateTrans(SMnode *pMnode, SStreamObj *pStream, SArray *pChkptInfoList); int32_t mndScanCheckpointReportInfo(SRpcMsg *pReq); -int32_t mndSendQuickConsensusChkptIdRsp(SRestoreCheckpointInfo *pReq, int32_t code, int64_t streamId, - int64_t checkpointId, SRpcHandleInfo *pRpcInfo); - +int32_t mndCreateSetConsensusChkptIdTrans(SMnode *pMnode, SStreamObj *pStream, int32_t taskId, int64_t checkpointId, + int64_t ts); void removeTasksInBuf(SArray *pTaskIds, SStreamExecInfo *pExecInfo); SStreamTaskIter *createStreamTaskIter(SStreamObj *pStream); @@ -151,9 +150,7 @@ SCheckpointConsensusInfo *mndGetConsensusInfo(SHashObj *pHash, int64_t streamId, void mndAddConsensusTasks(SCheckpointConsensusInfo *pInfo, const SRestoreCheckpointInfo *pRestoreInfo, SRpcHandleInfo *pRpcInfo); void mndClearConsensusRspEntry(SCheckpointConsensusInfo *pInfo); -int32_t doSendConsensusCheckpointRsp(SRestoreCheckpointInfo *pInfo, SRpcMsg *pMsg, int64_t checkpointId); int64_t mndClearConsensusCheckpointId(SHashObj* pHash, int64_t streamId); -int32_t mndRegisterConsensusChkptId(SHashObj* pHash, int64_t streamId); #ifdef __cplusplus } diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index a1016ad96c..a8dc1e42bb 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -61,6 +61,7 @@ static int32_t mndProcessStreamReqCheckpoint(SRpcMsg *pReq); static int32_t mndProcessCheckpointReport(SRpcMsg *pReq); static int32_t mndProcessConsensusCheckpointId(SRpcMsg *pMsg); static int32_t mndProcessConsensusInTmr(SRpcMsg *pMsg); +static void doSendQuickRsp(SRpcHandleInfo *pInfo, int32_t msgSize, int32_t vgId, int32_t code); static SVgroupChangeInfo mndFindChangedNodeInfo(SMnode *pMnode, const SArray *pPrevNodeList, const SArray *pNodeList); @@ -107,6 +108,7 @@ int32_t mndInitStream(SMnode *pMnode) { mndSetMsgHandle(pMnode, TDMT_VND_STREAM_TASK_UPDATE_RSP, mndTransProcessRsp); mndSetMsgHandle(pMnode, TDMT_VND_STREAM_TASK_RESET_RSP, mndTransProcessRsp); mndSetMsgHandle(pMnode, TDMT_STREAM_TASK_UPDATE_CHKPT_RSP, mndTransProcessRsp); + mndSetMsgHandle(pMnode, TDMT_STREAM_CONSEN_CHKPT_RSP, mndTransProcessRsp); // for msgs inside mnode // TODO change the name @@ -119,7 +121,7 @@ int32_t mndInitStream(SMnode *pMnode) { mndSetMsgHandle(pMnode, TDMT_MND_STREAM_BEGIN_CHECKPOINT, mndProcessStreamCheckpoint); mndSetMsgHandle(pMnode, TDMT_MND_STREAM_REQ_CHKPT, mndProcessStreamReqCheckpoint); mndSetMsgHandle(pMnode, TDMT_MND_STREAM_CHKPT_REPORT, mndProcessCheckpointReport); - mndSetMsgHandle(pMnode, TDMT_MND_STREAM_CHKPT_CONSEN, mndProcessConsensusCheckpointId); + mndSetMsgHandle(pMnode, TDMT_MND_STREAM_REQ_CONSEN_CHKPT, mndProcessConsensusCheckpointId); mndSetMsgHandle(pMnode, TDMT_MND_STREAM_UPDATE_CHKPT_EVT, mndScanCheckpointReportInfo); mndSetMsgHandle(pMnode, TDMT_STREAM_TASK_REPORT_CHECKPOINT, mndTransProcessRsp); mndSetMsgHandle(pMnode, TDMT_MND_STREAM_HEARTBEAT, mndProcessStreamHb); @@ -2611,23 +2613,14 @@ int32_t mndProcessCheckpointReport(SRpcMsg *pReq) { taosThreadMutexUnlock(&execInfo.lock); - { - SRpcMsg rsp = {.code = 0, .info = pReq->info, .contLen = sizeof(SMStreamUpdateChkptRsp)}; - rsp.pCont = rpcMallocCont(rsp.contLen); - SMsgHead *pHead = rsp.pCont; - pHead->vgId = htonl(req.nodeId); - - tmsgSendRsp(&rsp); - pReq->info.handle = NULL; // disable auto rsp - } - + doSendQuickRsp(&pReq->info, sizeof(SMStreamUpdateChkptRsp), req.nodeId, TSDB_CODE_SUCCESS); return 0; } -static int64_t getConsensusId(int64_t streamId, int32_t numOfTasks, bool* pAllEqual) { +static int64_t getConsensusId(int64_t streamId, int32_t numOfTasks, int32_t* pExistedTasks) { int32_t num = 0; int64_t chkId = INT64_MAX; - *pAllEqual = true; + *pExistedTasks = 0; for(int32_t i = 0; i < taosArrayGetSize(execInfo.pTaskList); ++i) { STaskId* p = taosArrayGet(execInfo.pTaskList, i); @@ -2637,16 +2630,12 @@ static int64_t getConsensusId(int64_t streamId, int32_t numOfTasks, bool* pAllEq num += 1; STaskStatusEntry* pe = taosHashGet(execInfo.pTaskMap, p, sizeof(*p)); - - if (chkId != INT64_MAX && chkId != pe->checkpointInfo.latestId) { - *pAllEqual = false; - } - if (chkId > pe->checkpointInfo.latestId) { chkId = pe->checkpointInfo.latestId; } } + *pExistedTasks = num; if (num < numOfTasks) { // not all task send info to mnode through hbMsg, no valid checkpoint Id return -1; } @@ -2654,6 +2643,16 @@ static int64_t getConsensusId(int64_t streamId, int32_t numOfTasks, bool* pAllEq return chkId; } +static void doSendQuickRsp(SRpcHandleInfo *pInfo, int32_t msgSize, int32_t vgId, int32_t code) { + SRpcMsg rsp = {.code = code, .info = *pInfo, .contLen = msgSize}; + rsp.pCont = rpcMallocCont(rsp.contLen); + SMsgHead *pHead = rsp.pCont; + pHead->vgId = htonl(vgId); + + tmsgSendRsp(&rsp); + pInfo->handle = NULL; // disable auto rsp +} + static int32_t mndProcessConsensusCheckpointId(SRpcMsg *pMsg) { SMnode *pMnode = pMsg->info.node; SDecoder decoder = {0}; @@ -2675,9 +2674,8 @@ static int32_t mndProcessConsensusCheckpointId(SRpcMsg *pMsg) { // register to the stream task done map, if all tasks has sent this kinds of message, start the checkpoint trans. taosThreadMutexLock(&execInfo.lock); - SStreamObj *pStream = mndGetStreamObj(pMnode, req.streamId); - // mnode handle the create stream transaction too slow may cause this problem + SStreamObj *pStream = mndGetStreamObj(pMnode, req.streamId); if (pStream == NULL) { mWarn("failed to find the stream:0x%" PRIx64 ", not handle consensus-checkpointId", req.streamId); @@ -2688,11 +2686,9 @@ static int32_t mndProcessConsensusCheckpointId(SRpcMsg *pMsg) { if (p == NULL) { mError("failed to find the stream:0x%" PRIx64 " in buf, not handle consensus-checkpointId", req.streamId); terrno = TSDB_CODE_MND_STREAM_NOT_EXIST; - - mndSendQuickConsensusChkptIdRsp(&req, terrno, req.streamId, 0, &pMsg->info); - taosThreadMutexUnlock(&execInfo.lock); - pMsg->info.handle = NULL; // disable auto rsp + + doSendQuickRsp(&pMsg->info, sizeof(SMStreamReqConsensChkptRsp), req.nodeId, terrno); return -1; } else { mDebug("s-task:0x%" PRIx64 "-0x%x in buf not in mnode/meta, create stream trans may not complete yet", @@ -2706,36 +2702,35 @@ static int32_t mndProcessConsensusCheckpointId(SRpcMsg *pMsg) { int32_t numOfTasks = (pStream == NULL) ? 0 : mndGetNumOfStreamTasks(pStream); if ((pStream != NULL) && (pStream->checkpointId == 0)) { // not generated checkpoint yet, return 0 directly - mndSendQuickConsensusChkptIdRsp(&req, TSDB_CODE_SUCCESS, req.streamId, 0, &pMsg->info); - taosThreadMutexUnlock(&execInfo.lock); - pMsg->info.handle = NULL; // disable auto rsp + mndCreateSetConsensusChkptIdTrans(pMnode, pStream, req.taskId, 0, req.startTs); + + doSendQuickRsp(&pMsg->info, sizeof(SMStreamReqConsensChkptRsp), req.nodeId, terrno); return TSDB_CODE_SUCCESS; } - bool allEqual = true; - int64_t chkId = getConsensusId(req.streamId, numOfTasks, &allEqual); + int32_t num = 0; + int64_t chkId = getConsensusId(req.streamId, numOfTasks, &num); // some tasks not send hbMsg to mnode yet, wait for 5s. if (chkId == -1) { - mDebug( - "not all task send hbMsg yet, add into list and wait for 10s to check the consensus-checkpointId again, " - "s-task:0x%x", req.taskId); + mDebug("not all(%d/%d) task(s) send hbMsg yet, wait for a while and check again, s-task:0x%x", req.taskId, num, + numOfTasks); SCheckpointConsensusInfo *pInfo = mndGetConsensusInfo(execInfo.pStreamConsensus, req.streamId, numOfTasks); mndAddConsensusTasks(pInfo, &req, &pMsg->info); - taosThreadMutexUnlock(&execInfo.lock); - pMsg->info.handle = NULL; // disable auto rsp + taosThreadMutexUnlock(&execInfo.lock); + doSendQuickRsp(&pMsg->info, sizeof(SMStreamReqConsensChkptRsp), req.nodeId, terrno); return 0; } if (chkId == req.checkpointId) { mDebug("vgId:%d stream:0x%" PRIx64 " %s consensus-checkpointId is:%" PRId64 ", meta-stored checkpointId:%" PRId64, req.nodeId, req.streamId, pStream->name, chkId, pStream->checkpointId); - mndSendQuickConsensusChkptIdRsp(&req, TSDB_CODE_SUCCESS, req.streamId, chkId, &pMsg->info); + mndCreateSetConsensusChkptIdTrans(pMnode, pStream, req.taskId, chkId, req.startTs); taosThreadMutexUnlock(&execInfo.lock); - pMsg->info.handle = NULL; // disable auto rsp + doSendQuickRsp(&pMsg->info, sizeof(SMStreamReqConsensChkptRsp), req.nodeId, terrno); return 0; } @@ -2748,14 +2743,14 @@ static int32_t mndProcessConsensusCheckpointId(SRpcMsg *pMsg) { } taosThreadMutexUnlock(&execInfo.lock); - pMsg->info.handle = NULL; // disable auto rsp - + doSendQuickRsp(&pMsg->info, sizeof(SMStreamReqConsensChkptRsp), req.nodeId, terrno); return 0; } int32_t mndProcessConsensusInTmr(SRpcMsg *pMsg) { + SMnode *pMnode = pMsg->info.node; int64_t now = taosGetTimestampMs(); - int64_t streamId = -1; // todo: fix only one + SArray *pStreamList = taosArrayInit(4, sizeof(int64_t)); mDebug("start to process consensus-checkpointId in tmr"); taosThreadMutexLock(&execInfo.lock); @@ -2764,40 +2759,42 @@ int32_t mndProcessConsensusInTmr(SRpcMsg *pMsg) { while ((pIter = taosHashIterate(execInfo.pStreamConsensus, pIter)) != NULL) { SCheckpointConsensusInfo *pInfo = (SCheckpointConsensusInfo *)pIter; - int32_t j = 0; + int64_t streamId = -1; int32_t num = taosArrayGetSize(pInfo->pTaskList); - SArray *pList = taosArrayInit(4, sizeof(int32_t)); - for (; j < num; ++j) { - SCheckpointConsensusEntry *pe = taosArrayGet(pInfo->pTaskList, j); + SStreamObj *pStream = mndGetStreamObj(pMnode, pInfo->streamId); + if (pStream == NULL) { // stream has been dropped already + mDebug("stream:0x%"PRIx64" dropped already, continue", pInfo->streamId); + continue; + } - if ((now - pe->ts) > 10 * 1000) { - bool allEqual = true; - int64_t chkId = getConsensusId(pe->req.streamId, pInfo->numOfTasks, &allEqual); + for (int32_t j = 0; j < num; ++j) { + SCheckpointConsensusEntry *pe = taosArrayGet(pInfo->pTaskList, j); + streamId = pe->req.streamId; + + if ((now - pe->ts) >= 10 * 1000) { + int32_t existed = 0; + int64_t chkId = getConsensusId(pe->req.streamId, pInfo->numOfTasks, &existed); if (chkId == -1) { - mDebug("tasks send hbMsg for stream:0x%" PRIx64 ", wait for next round", pe->req.streamId); + mDebug("not all(%d/%d) task(s) send hbMsg yet, wait for a while and check again, s-task:0x%x", existed, + pInfo->numOfTasks, pe->req.taskId); break; } - if (allEqual) { - mDebug("all has identical checkpointId for stream:0x%"PRIx64" send checkpointId to s-task:0x%x", - pe->req.streamId, pe->req.taskId); - - mndSendQuickConsensusChkptIdRsp(&pe->req, TSDB_CODE_SUCCESS, pe->req.streamId, chkId, &pe->rspInfo); - } else { - ASSERT(chkId <= pe->req.checkpointId); - mndSendQuickConsensusChkptIdRsp(&pe->req, TSDB_CODE_SUCCESS, pe->req.streamId, chkId, &pe->rspInfo); - } + ASSERT(chkId <= pe->req.checkpointId); + mndCreateSetConsensusChkptIdTrans(pMnode, pStream, pe->req.taskId, chkId, pe->req.startTs); taosArrayPush(pList, &pe->req.taskId); streamId = pe->req.streamId; } else { mDebug("s-task:0x%x sendTs:%" PRId64 " wait %2.fs already, wait for next round to check", pe->req.taskId, - (now - pe->ts)/ 1000.0, pe->ts); + (now - pe->ts) / 1000.0, pe->ts); } } + mndReleaseStream(pMnode, pStream); + if (taosArrayGetSize(pList) > 0) { for (int32_t i = 0; i < taosArrayGetSize(pList); ++i) { int32_t *taskId = taosArrayGet(pList, i); @@ -2815,12 +2812,19 @@ int32_t mndProcessConsensusInTmr(SRpcMsg *pMsg) { if (taosArrayGetSize(pInfo->pTaskList) == 0) { mndClearConsensusRspEntry(pInfo); - mndClearConsensusCheckpointId(execInfo.pStreamConsensus, streamId); + ASSERT(streamId != -1); + taosArrayPush(pStreamList, &streamId); } } + for (int32_t i = 0; i < taosArrayGetSize(pStreamList); ++i) { + int64_t *pStreamId = (int64_t *)taosArrayGet(pStreamList, i); + mndClearConsensusCheckpointId(execInfo.pStreamConsensus, *pStreamId); + } + taosThreadMutexUnlock(&execInfo.lock); + taosArrayDestroy(pStreamList); mDebug("end to process consensus-checkpointId in tmr"); return TSDB_CODE_SUCCESS; } diff --git a/source/dnode/mnode/impl/src/mndStreamUtil.c b/source/dnode/mnode/impl/src/mndStreamUtil.c index 9ee820925c..7d9b9e4571 100644 --- a/source/dnode/mnode/impl/src/mndStreamUtil.c +++ b/source/dnode/mnode/impl/src/mndStreamUtil.c @@ -651,6 +651,7 @@ void removeTasksInBuf(SArray *pTaskIds, SStreamExecInfo* pExecInfo) { void removeStreamTasksInBuf(SStreamObj *pStream, SStreamExecInfo *pExecNode) { taosThreadMutexLock(&pExecNode->lock); + // 1. remove task entries SStreamTaskIter *pIter = createStreamTaskIter(pStream); while (streamTaskIterNextTask(pIter)) { SStreamTask *pTask = streamTaskIterGetCurrent(pIter); @@ -660,8 +661,11 @@ void removeStreamTasksInBuf(SStreamObj *pStream, SStreamExecInfo *pExecNode) { } ASSERT(taosHashGetSize(pExecNode->pTaskMap) == taosArrayGetSize(pExecNode->pTaskList)); - taosThreadMutexUnlock(&pExecNode->lock); + // 2. remove stream entry in consensus hash table + mndClearConsensusCheckpointId(execInfo.pStreamConsensus, pStream->uid); + + taosThreadMutexUnlock(&pExecNode->lock); destroyStreamTaskIter(pIter); } @@ -835,45 +839,98 @@ int32_t mndScanCheckpointReportInfo(SRpcMsg *pReq) { return TSDB_CODE_SUCCESS; } -int32_t doSendConsensusCheckpointRsp(SRestoreCheckpointInfo* pInfo, SRpcMsg* pMsg, int64_t checkpointId) { +static int32_t mndStreamSetChkptIdAction(SMnode *pMnode, STrans *pTrans, SStreamTask* pTask, int64_t checkpointId, int64_t ts) { + SRestoreCheckpointInfo req = { + .taskId = pTask->id.taskId, + .streamId = pTask->id.streamId, + .checkpointId = checkpointId, + .startTs = ts, + .nodeId = pTask->info.nodeId, + }; + int32_t code = 0; int32_t blen; - - SRestoreCheckpointInfoRsp req = { - .streamId = pInfo->streamId, .taskId = pInfo->taskId, .checkpointId = checkpointId, .startTs = pInfo->startTs}; - - tEncodeSize(tEncodeRestoreCheckpointInfoRsp, &req, blen, code); + tEncodeSize(tEncodeRestoreCheckpointInfo, &req, blen, code); if (code < 0) { terrno = TSDB_CODE_OUT_OF_MEMORY; return -1; } - int32_t tlen = sizeof(SMsgHead) + blen; - void *abuf = POINTER_SHIFT(pMsg->pCont, sizeof(SMsgHead)); + int32_t tlen = sizeof(SMsgHead) + blen; + + void *pBuf = taosMemoryMalloc(tlen); + if (pBuf == NULL) { + terrno = TSDB_CODE_OUT_OF_MEMORY; + return -1; + } + + void *abuf = POINTER_SHIFT(pBuf, sizeof(SMsgHead)); SEncoder encoder; tEncoderInit(&encoder, abuf, tlen); - tEncodeRestoreCheckpointInfoRsp(&encoder, &req); + tEncodeRestoreCheckpointInfo(&encoder, &req); - SMsgHead *pMsgHead = (SMsgHead *)pMsg->pCont; + SMsgHead *pMsgHead = (SMsgHead *)pBuf; pMsgHead->contLen = htonl(tlen); - pMsgHead->vgId = htonl(pInfo->nodeId); + pMsgHead->vgId = htonl(pTask->info.nodeId); + tEncoderClear(&encoder); - tmsgSendRsp(pMsg); + SEpSet epset = {0}; + bool hasEpset = false; + code = extractNodeEpset(pMnode, &epset, &hasEpset, pTask->id.taskId, pTask->info.nodeId); + if (code != TSDB_CODE_SUCCESS || !hasEpset) { + taosMemoryFree(pBuf); + return code; + } + + code = setTransAction(pTrans, pBuf, tlen, TDMT_STREAM_CONSEN_CHKPT, &epset, 0, TSDB_CODE_VND_INVALID_VGROUP_ID); + if (code != TSDB_CODE_SUCCESS) { + taosMemoryFree(pBuf); + } + return code; } -int32_t mndSendQuickConsensusChkptIdRsp(SRestoreCheckpointInfo *pReq, int32_t code, int64_t streamId, - int64_t checkpointId, SRpcHandleInfo *pRpcInfo) { - SRpcMsg rsp = {.code = code, .info = *pRpcInfo, .contLen = sizeof(SRestoreCheckpointInfoRsp) + sizeof(SMsgHead)}; - rsp.pCont = rpcMallocCont(rsp.contLen); +int32_t mndCreateSetConsensusChkptIdTrans(SMnode *pMnode, SStreamObj *pStream, int32_t taskId, int64_t checkpointId, + int64_t ts) { + char msg[128] = {0}; + snprintf(msg, tListLen(msg), "set consen-chkpt-id for task:0x%x", taskId); - SMsgHead *pHead = rsp.pCont; - pHead->vgId = htonl(pReq->nodeId); + STrans *pTrans = doCreateTrans(pMnode, pStream, NULL, TRN_CONFLICT_NOTHING, MND_STREAM_CHKPT_CONSEN_NAME, msg); + if (pTrans == NULL) { + return terrno; + } - mDebug("stream:0x%" PRIx64 " consensus-checkpointId:%" PRId64 " exists, s-task:0x%x send to vnode", - streamId, checkpointId, pReq->taskId); - return doSendConsensusCheckpointRsp(pReq, &rsp, checkpointId); + STaskId id = {.streamId = pStream->uid, .taskId = taskId}; + SStreamTask *pTask = mndGetStreamTask(&id, pStream); + ASSERT(pTask); + + /*int32_t code = */ mndStreamRegisterTrans(pTrans, MND_STREAM_CHKPT_CONSEN_NAME, pStream->uid); + int32_t code = mndStreamSetChkptIdAction(pMnode, pTrans, pTask, checkpointId, ts); + if (code != 0) { + sdbRelease(pMnode->pSdb, pStream); + mndTransDrop(pTrans); + return code; + } + + code = mndPersistTransLog(pStream, pTrans, SDB_STATUS_READY); + if (code != TSDB_CODE_SUCCESS) { + sdbRelease(pMnode->pSdb, pStream); + mndTransDrop(pTrans); + return -1; + } + + if (mndTransPrepare(pMnode, pTrans) != 0) { + mError("trans:%d, failed to prepare set consensus-chkptId trans since %s", pTrans->id, terrstr()); + sdbRelease(pMnode->pSdb, pStream); + mndTransDrop(pTrans); + return -1; + } + + sdbRelease(pMnode->pSdb, pStream); + mndTransDrop(pTrans); + + return TSDB_CODE_ACTION_IN_PROGRESS; } SCheckpointConsensusInfo* mndGetConsensusInfo(SHashObj* pHash, int64_t streamId, int32_t numOfTasks) { @@ -885,6 +942,7 @@ SCheckpointConsensusInfo* mndGetConsensusInfo(SHashObj* pHash, int64_t streamId, SCheckpointConsensusInfo p = { .pTaskList = taosArrayInit(4, sizeof(SCheckpointConsensusEntry)), .numOfTasks = numOfTasks, + .streamId = streamId, }; taosHashPut(pHash, &streamId, sizeof(streamId), &p, sizeof(p)); diff --git a/source/dnode/snode/src/snode.c b/source/dnode/snode/src/snode.c index 69a7bc7ba4..60e57e8a2f 100644 --- a/source/dnode/snode/src/snode.c +++ b/source/dnode/snode/src/snode.c @@ -128,6 +128,8 @@ int32_t sndProcessStreamMsg(SSnode *pSnode, SRpcMsg *pMsg) { return tqStreamTaskProcessRetrieveTriggerReq(pSnode->pMeta, pMsg); case TDMT_STREAM_RETRIEVE_TRIGGER_RSP: return tqStreamTaskProcessRetrieveTriggerRsp(pSnode->pMeta, pMsg); + case TDMT_MND_STREAM_REQ_CONSEN_CHKPT_RSP: + return tqStreamProcessConsensusChkptRsp2(pSnode->pMeta, pMsg); default: sndError("invalid snode msg:%d", pMsg->msgType); ASSERT(0); @@ -149,15 +151,15 @@ int32_t sndProcessWriteMsg(SSnode *pSnode, SRpcMsg *pMsg, SRpcMsg *pRsp) { case TDMT_VND_STREAM_TASK_UPDATE: return tqStreamTaskProcessUpdateReq(pSnode->pMeta, &pSnode->msgCb, pMsg, true); case TDMT_VND_STREAM_TASK_RESET: - return tqStreamTaskProcessTaskResetReq(pSnode->pMeta, pMsg); + return tqStreamTaskProcessTaskResetReq(pSnode->pMeta, pMsg->pCont); case TDMT_STREAM_TASK_PAUSE: return tqStreamTaskProcessTaskPauseReq(pSnode->pMeta, pMsg->pCont); case TDMT_STREAM_TASK_RESUME: return tqStreamTaskProcessTaskResumeReq(pSnode->pMeta, pMsg->info.conn.applyIndex, pMsg->pCont, false); case TDMT_STREAM_TASK_UPDATE_CHKPT: - return tqStreamTaskProcessUpdateCheckpointReq(pSnode->pMeta, true, pMsg->pCont, pMsg->contLen); - case TDMT_MND_STREAM_CHKPT_CONSEN_RSP: - return tqStreamProcessConsensusChkptRsp(pSnode->pMeta, pMsg); + return tqStreamTaskProcessUpdateCheckpointReq(pSnode->pMeta, true, pMsg->pCont); + case TDMT_STREAM_CONSEN_CHKPT: + return tqStreamTaskProcessConsenChkptIdReq(pSnode->pMeta, pMsg); default: ASSERT(0); } diff --git a/source/dnode/vnode/src/inc/vnodeInt.h b/source/dnode/vnode/src/inc/vnodeInt.h index 1bec226489..4a47e08730 100644 --- a/source/dnode/vnode/src/inc/vnodeInt.h +++ b/source/dnode/vnode/src/inc/vnodeInt.h @@ -298,6 +298,7 @@ int32_t tqProcessTaskRetrieveRsp(STQ* pTq, SRpcMsg* pMsg); int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg); int32_t tqStreamProgressRetrieveReq(STQ* pTq, SRpcMsg* pMsg); int32_t tqProcessTaskUpdateCheckpointReq(STQ* pTq, char* msg, int32_t msgLen); +int32_t tqProcessTaskConsenChkptIdReq(STQ* pTq, SRpcMsg* pMsg); // sma int32_t smaInit(); diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 0a64b9c165..ac57a003c5 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1016,7 +1016,11 @@ int32_t tqProcessTaskDropReq(STQ* pTq, char* msg, int32_t msgLen) { } int32_t tqProcessTaskUpdateCheckpointReq(STQ* pTq, char* msg, int32_t msgLen) { - return tqStreamTaskProcessUpdateCheckpointReq(pTq->pStreamMeta, pTq->pVnode->restored, msg, msgLen); + return tqStreamTaskProcessUpdateCheckpointReq(pTq->pStreamMeta, pTq->pVnode->restored, msg); +} + +int32_t tqProcessTaskConsenChkptIdReq(STQ* pTq, SRpcMsg* pMsg) { + return tqStreamTaskProcessConsenChkptIdReq(pTq->pStreamMeta, pMsg); } int32_t tqProcessTaskPauseReq(STQ* pTq, int64_t sversion, char* msg, int32_t msgLen) { @@ -1239,7 +1243,7 @@ int32_t tqProcessTaskUpdateReq(STQ* pTq, SRpcMsg* pMsg) { } int32_t tqProcessTaskResetReq(STQ* pTq, SRpcMsg* pMsg) { - return tqStreamTaskProcessTaskResetReq(pTq->pStreamMeta, pMsg); + return tqStreamTaskProcessTaskResetReq(pTq->pStreamMeta, pMsg->pCont); } int32_t tqProcessTaskRetrieveTriggerReq(STQ* pTq, SRpcMsg* pMsg) { @@ -1277,5 +1281,5 @@ int32_t tqProcessTaskChkptReportRsp(STQ* pTq, SRpcMsg* pMsg) { } int32_t tqProcessTaskConsensusChkptRsp(STQ* pTq, SRpcMsg* pMsg) { - return tqStreamProcessConsensusChkptRsp(pTq->pStreamMeta, pMsg); + return tqStreamProcessConsensusChkptRsp2(pTq->pStreamMeta, pMsg); } diff --git a/source/dnode/vnode/src/tqCommon/tqCommon.c b/source/dnode/vnode/src/tqCommon/tqCommon.c index c40332ff39..cb480d09bb 100644 --- a/source/dnode/vnode/src/tqCommon/tqCommon.c +++ b/source/dnode/vnode/src/tqCommon/tqCommon.c @@ -670,7 +670,7 @@ int32_t tqStreamTaskProcessDropReq(SStreamMeta* pMeta, char* msg, int32_t msgLen return 0; } -int32_t tqStreamTaskProcessUpdateCheckpointReq(SStreamMeta* pMeta, bool restored, char* msg, int32_t msgLen) { +int32_t tqStreamTaskProcessUpdateCheckpointReq(SStreamMeta* pMeta, bool restored, char* msg) { SVUpdateCheckpointInfoReq* pReq = (SVUpdateCheckpointInfoReq*)msg; int32_t vgId = pMeta->vgId; @@ -858,8 +858,8 @@ int32_t tqStartTaskCompleteCallback(SStreamMeta* pMeta) { return TSDB_CODE_SUCCESS; } -int32_t tqStreamTaskProcessTaskResetReq(SStreamMeta* pMeta, SRpcMsg* pMsg) { - SVPauseStreamTaskReq* pReq = (SVPauseStreamTaskReq*)pMsg->pCont; +int32_t tqStreamTaskProcessTaskResetReq(SStreamMeta* pMeta, char* pMsg) { + SVPauseStreamTaskReq* pReq = (SVPauseStreamTaskReq*)pMsg; SStreamTask* pTask = streamMetaAcquireTask(pMeta, pReq->streamId, pReq->taskId); if (pTask == NULL) { @@ -1115,6 +1115,8 @@ int32_t tqStreamProcessReqCheckpointRsp(SStreamMeta* pMeta, SRpcMsg* pMsg) { ret int32_t tqStreamProcessChkptReportRsp(SStreamMeta* pMeta, SRpcMsg* pMsg) { return doProcessDummyRspMsg(pMeta, pMsg); } +int32_t tqStreamProcessConsensusChkptRsp2(SStreamMeta* pMeta, SRpcMsg* pMsg) { return doProcessDummyRspMsg(pMeta, pMsg); } + int32_t tqStreamProcessCheckpointReadyRsp(SStreamMeta* pMeta, SRpcMsg* pMsg) { SMStreamCheckpointReadyRspMsg* pRsp = pMsg->pCont; @@ -1130,22 +1132,21 @@ int32_t tqStreamProcessCheckpointReadyRsp(SStreamMeta* pMeta, SRpcMsg* pMsg) { return TSDB_CODE_SUCCESS; } -int32_t tqStreamProcessConsensusChkptRsp(SStreamMeta* pMeta, SRpcMsg* pMsg) { +int32_t tqStreamTaskProcessConsenChkptIdReq(SStreamMeta* pMeta, SRpcMsg* pMsg) { int32_t vgId = pMeta->vgId; + int32_t code = 0; + char* msg = POINTER_SHIFT(pMsg->pCont, sizeof(SMsgHead)); int32_t len = pMsg->contLen - sizeof(SMsgHead); - SRpcMsg rsp = {.info = pMsg->info, .code = TSDB_CODE_SUCCESS}; int64_t now = taosGetTimestampMs(); - SRestoreCheckpointInfoRsp req = {0}; + SRestoreCheckpointInfo req = {0}; SDecoder decoder; tDecoderInit(&decoder, (uint8_t*)msg, len); - rsp.info.handle = NULL; - if (tDecodeRestoreCheckpointInfoRsp(&decoder, &req) < 0) { - // rsp.code = TSDB_CODE_MSG_DECODE_ERROR; // disable it temporarily - tqError("vgId:%d failed to decode restore task checkpointId, code:%s", vgId, tstrerror(rsp.code)); + if (tDecodeRestoreCheckpointInfo(&decoder, &req) < 0) { + tqError("vgId:%d failed to decode set consensus checkpointId req, code:%s", vgId, tstrerror(code)); tDecoderClear(&decoder); return TSDB_CODE_SUCCESS; } @@ -1154,7 +1155,7 @@ int32_t tqStreamProcessConsensusChkptRsp(SStreamMeta* pMeta, SRpcMsg* pMsg) { SStreamTask* pTask = streamMetaAcquireTask(pMeta, req.streamId, req.taskId); if (pTask == NULL) { - tqError("vgId:%d process restore checkpointId req, failed to acquire task:0x%x, it may have been dropped already", + tqError("vgId:%d process set consensus checkpointId req, failed to acquire task:0x%x, it may have been dropped already", pMeta->vgId, req.taskId); streamMetaAddFailedTask(pMeta, req.streamId, req.taskId); return TSDB_CODE_SUCCESS; @@ -1182,7 +1183,7 @@ int32_t tqStreamProcessConsensusChkptRsp(SStreamMeta* pMeta, SRpcMsg* pMsg) { pTask->chkInfo.checkpointId = req.checkpointId; tqSetRestoreVersionInfo(pTask); } else { - tqDebug("s-task:%s vgId:%d consensus-checkpointId:%" PRId64 " equals to current checkpointId, no need to update", + tqDebug("s-task:%s vgId:%d consensus-checkpointId:%" PRId64 " equals to current checkpointId, not update", pTask->id.idStr, vgId, req.checkpointId); } diff --git a/source/dnode/vnode/src/vnd/vnodeSvr.c b/source/dnode/vnode/src/vnd/vnodeSvr.c index 3757cd00bc..04839c3357 100644 --- a/source/dnode/vnode/src/vnd/vnodeSvr.c +++ b/source/dnode/vnode/src/vnd/vnodeSvr.c @@ -630,6 +630,11 @@ int32_t vnodeProcessWriteMsg(SVnode *pVnode, SRpcMsg *pMsg, int64_t ver, SRpcMsg goto _err; } } break; + case TDMT_STREAM_CONSEN_CHKPT: { + if (pVnode->restored) { + tqProcessTaskConsenChkptIdReq(pVnode->pTq, pMsg); + } + } break; case TDMT_STREAM_TASK_PAUSE: { if (pVnode->restored && vnodeIsLeader(pVnode) && tqProcessTaskPauseReq(pVnode->pTq, ver, pMsg->pCont, pMsg->contLen) < 0) { @@ -647,11 +652,6 @@ int32_t vnodeProcessWriteMsg(SVnode *pVnode, SRpcMsg *pMsg, int64_t ver, SRpcMsg tqProcessTaskResetReq(pVnode->pTq, pMsg); } } break; - case TDMT_MND_STREAM_CHKPT_CONSEN_RSP: { - if (pVnode->restored) { - tqProcessTaskConsensusChkptRsp(pVnode->pTq, pMsg); - } - } break; case TDMT_VND_ALTER_CONFIRM: needCommit = pVnode->config.hashChange; if (vnodeProcessAlterConfirmReq(pVnode, ver, pReq, len, pRsp) < 0) { @@ -861,6 +861,8 @@ int32_t vnodeProcessStreamMsg(SVnode *pVnode, SRpcMsg *pMsg, SQueueInfo *pInfo) return tqStreamProgressRetrieveReq(pVnode->pTq, pMsg); case TDMT_MND_STREAM_CHKPT_REPORT_RSP: return tqProcessTaskChkptReportRsp(pVnode->pTq, pMsg); + case TDMT_MND_STREAM_REQ_CONSEN_CHKPT_RSP: + return tqProcessTaskConsensusChkptRsp(pVnode->pTq, pMsg); default: vError("unknown msg type:%d in stream queue", pMsg->msgType); return TSDB_CODE_APP_ERROR; diff --git a/source/libs/stream/src/streamCheckpoint.c b/source/libs/stream/src/streamCheckpoint.c index d5f7d6ef21..5cd084e6a2 100644 --- a/source/libs/stream/src/streamCheckpoint.c +++ b/source/libs/stream/src/streamCheckpoint.c @@ -1153,7 +1153,7 @@ int32_t streamTaskSendRestoreChkptMsg(SStreamTask* pTask) { tEncoderClear(&encoder); SRpcMsg msg = {0}; - initRpcMsg(&msg, TDMT_MND_STREAM_CHKPT_CONSEN, buf, tlen); + initRpcMsg(&msg, TDMT_MND_STREAM_REQ_CONSEN_CHKPT, buf, tlen); stDebug("s-task:%s vgId:%d send latest checkpointId:%" PRId64 " to mnode to get the consensus checkpointId", id, vgId, pInfo->checkpointId); diff --git a/source/libs/stream/src/streamMsg.c b/source/libs/stream/src/streamMsg.c index e0435156e2..f02f661143 100644 --- a/source/libs/stream/src/streamMsg.c +++ b/source/libs/stream/src/streamMsg.c @@ -650,23 +650,3 @@ int32_t tDecodeRestoreCheckpointInfo(SDecoder* pDecoder, SRestoreCheckpointInfo* tEndDecode(pDecoder); return 0; } - -int32_t tEncodeRestoreCheckpointInfoRsp(SEncoder* pCoder, const SRestoreCheckpointInfoRsp* pInfo) { - if (tStartEncode(pCoder) < 0) return -1; - if (tEncodeI64(pCoder, pInfo->startTs) < 0) return -1; - if (tEncodeI64(pCoder, pInfo->streamId) < 0) return -1; - if (tEncodeI32(pCoder, pInfo->taskId) < 0) return -1; - if (tEncodeI64(pCoder, pInfo->checkpointId) < 0) return -1; - tEndEncode(pCoder); - return 0; -} - -int32_t tDecodeRestoreCheckpointInfoRsp(SDecoder* pCoder, SRestoreCheckpointInfoRsp* pInfo) { - if (tStartDecode(pCoder) < 0) return -1; - if (tDecodeI64(pCoder, &pInfo->startTs) < 0) return -1; - if (tDecodeI64(pCoder, &pInfo->streamId) < 0) return -1; - if (tDecodeI32(pCoder, &pInfo->taskId) < 0) return -1; - if (tDecodeI64(pCoder, &pInfo->checkpointId) < 0) return -1; - tEndDecode(pCoder); - return 0; -} \ No newline at end of file From 6a1555e893ccd11419b90ff43133bec6f5b7efdd Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Fri, 5 Jul 2024 18:06:00 +0800 Subject: [PATCH 10/22] refactor: do some internal refactor. --- source/dnode/mnode/impl/src/mndStream.c | 1 + 1 file changed, 1 insertion(+) diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index a8dc1e42bb..00909cb36d 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -1138,6 +1138,7 @@ static int32_t mndCheckTaskAndNodeStatus(SMnode *pMnode) { mDebug("s-task:0x%" PRIx64 "-0x%x (nodeId:%d) status:%s, checkpoint not issued", pEntry->id.streamId, (int32_t)pEntry->id.taskId, pEntry->nodeId, streamTaskGetStatusStr(pEntry->status)); ready = false; + break; } if (pEntry->hTaskId != 0) { From ce4153b6fcb125059c5d96a5ba364a0732ba90e2 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 8 Jul 2024 09:05:33 +0800 Subject: [PATCH 11/22] fix(stream): use hb to send the consens-checkpointid req. --- include/libs/stream/tstream.h | 18 ++++++----- source/dnode/mnode/impl/inc/mndStream.h | 3 +- source/dnode/mnode/impl/src/mndStream.c | 6 ++-- source/dnode/mnode/impl/src/mndStreamHb.c | 35 +++++++++++++++++++-- source/dnode/mnode/impl/src/mndStreamUtil.c | 15 +++++++-- source/dnode/vnode/src/tqCommon/tqCommon.c | 5 +-- source/libs/stream/src/streamCheckpoint.c | 4 ++- source/libs/stream/src/streamHb.c | 6 ++++ source/libs/stream/src/streamMeta.c | 3 +- source/libs/stream/src/streamMsg.c | 2 ++ source/libs/stream/src/streamTask.c | 1 + 11 files changed, 76 insertions(+), 22 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index d0feebf814..5c61265c01 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -289,6 +289,7 @@ typedef struct SStreamStatus { bool appendTranstateBlock; // has append the transfer state data block already bool removeBackendFiles; // remove backend files on disk when free stream tasks bool sendConsensusChkptId; + bool requireConsensusChkptId; } SStreamStatus; typedef struct SDataRange { @@ -568,14 +569,15 @@ typedef struct { } SStreamScanHistoryReq; typedef struct STaskCkptInfo { - int64_t latestId; // saved checkpoint id - int64_t latestVer; // saved checkpoint ver - int64_t latestTime; // latest checkpoint time - int64_t latestSize; // latest checkpoint size - int8_t remoteBackup; // latest checkpoint backup done - int64_t activeId; // current active checkpoint id - int32_t activeTransId; // checkpoint trans id - int8_t failed; // denote if the checkpoint is failed or not + int64_t latestId; // saved checkpoint id + int64_t latestVer; // saved checkpoint ver + int64_t latestTime; // latest checkpoint time + int64_t latestSize; // latest checkpoint size + int8_t remoteBackup; // latest checkpoint backup done + int64_t activeId; // current active checkpoint id + int32_t activeTransId; // checkpoint trans id + int8_t failed; // denote if the checkpoint is failed or not + int8_t consensusChkptId; // required the consensus-checkpointId } STaskCkptInfo; typedef struct STaskStatusEntry { diff --git a/source/dnode/mnode/impl/inc/mndStream.h b/source/dnode/mnode/impl/inc/mndStream.h index 7b4270462f..0b6b6a9ef2 100644 --- a/source/dnode/mnode/impl/inc/mndStream.h +++ b/source/dnode/mnode/impl/inc/mndStream.h @@ -147,8 +147,7 @@ int32_t removeExpiredNodeEntryAndTaskInBuf(SArray *pNodeSnapshot); void removeStreamTasksInBuf(SStreamObj *pStream, SStreamExecInfo *pExecNode); SCheckpointConsensusInfo *mndGetConsensusInfo(SHashObj *pHash, int64_t streamId, int32_t numOfTasks); -void mndAddConsensusTasks(SCheckpointConsensusInfo *pInfo, const SRestoreCheckpointInfo *pRestoreInfo, - SRpcHandleInfo *pRpcInfo); +void mndAddConsensusTasks(SCheckpointConsensusInfo *pInfo, const SRestoreCheckpointInfo *pRestoreInfo); void mndClearConsensusRspEntry(SCheckpointConsensusInfo *pInfo); int64_t mndClearConsensusCheckpointId(SHashObj* pHash, int64_t streamId); diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index 00909cb36d..cd99714395 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -2718,7 +2718,7 @@ static int32_t mndProcessConsensusCheckpointId(SRpcMsg *pMsg) { mDebug("not all(%d/%d) task(s) send hbMsg yet, wait for a while and check again, s-task:0x%x", req.taskId, num, numOfTasks); SCheckpointConsensusInfo *pInfo = mndGetConsensusInfo(execInfo.pStreamConsensus, req.streamId, numOfTasks); - mndAddConsensusTasks(pInfo, &req, &pMsg->info); + mndAddConsensusTasks(pInfo, &req); taosThreadMutexUnlock(&execInfo.lock); doSendQuickRsp(&pMsg->info, sizeof(SMStreamReqConsensChkptRsp), req.nodeId, terrno); @@ -2737,7 +2737,7 @@ static int32_t mndProcessConsensusCheckpointId(SRpcMsg *pMsg) { // wait for 5s and check again SCheckpointConsensusInfo *pInfo = mndGetConsensusInfo(execInfo.pStreamConsensus, req.streamId, numOfTasks); - mndAddConsensusTasks(pInfo, &req, &pMsg->info); + mndAddConsensusTasks(pInfo, &req); if (pStream != NULL) { mndReleaseStream(pMnode, pStream); @@ -2789,7 +2789,7 @@ int32_t mndProcessConsensusInTmr(SRpcMsg *pMsg) { taosArrayPush(pList, &pe->req.taskId); streamId = pe->req.streamId; } else { - mDebug("s-task:0x%x sendTs:%" PRId64 " wait %2.fs already, wait for next round to check", pe->req.taskId, + mDebug("s-task:0x%x sendTs:%" PRId64 " wait %.2fs already, wait for next round to check", pe->req.taskId, (now - pe->ts) / 1000.0, pe->ts); } } diff --git a/source/dnode/mnode/impl/src/mndStreamHb.c b/source/dnode/mnode/impl/src/mndStreamHb.c index c7f97b4a62..8a374c99ef 100644 --- a/source/dnode/mnode/impl/src/mndStreamHb.c +++ b/source/dnode/mnode/impl/src/mndStreamHb.c @@ -246,7 +246,7 @@ int32_t mndProcessStreamHb(SRpcMsg *pReq) { } tDecoderClear(&decoder); - mTrace("receive stream-meta hb from vgId:%d, active numOfTasks:%d, msgId:%d", req.vgId, req.numOfTasks, req.msgId); + mDebug("receive stream-meta hb from vgId:%d, active numOfTasks:%d, msgId:%d", req.vgId, req.numOfTasks, req.msgId); pFailedChkpt = taosArrayInit(4, sizeof(SFailedCheckpointInfo)); pOrphanTasks = taosArrayInit(4, sizeof(SOrphanTask)); @@ -284,6 +284,23 @@ int32_t mndProcessStreamHb(SRpcMsg *pReq) { continue; } + STaskCkptInfo *pChkInfo = &p->checkpointInfo; + if (pChkInfo->consensusChkptId != 0) { + SRestoreCheckpointInfo cp = { + .streamId = p->id.streamId, + .taskId = p->id.taskId, + .checkpointId = p->checkpointInfo.latestId, + .startTs = pTaskEntry->startTime, + }; + + SStreamObj *pStream = mndGetStreamObj(pMnode, p->id.streamId); + int32_t numOfTasks = mndGetNumOfStreamTasks(pStream); + + SCheckpointConsensusInfo *pInfo = mndGetConsensusInfo(execInfo.pStreamConsensus, p->id.streamId, numOfTasks); + mndAddConsensusTasks(pInfo, &cp); + mndReleaseStream(pMnode, pStream); + } + if (pTaskEntry->stage != p->stage && pTaskEntry->stage != -1) { updateStageInfo(pTaskEntry, p->stage); if (pTaskEntry->nodeId == SNODE_HANDLE) { @@ -292,7 +309,6 @@ int32_t mndProcessStreamHb(SRpcMsg *pReq) { } else { streamTaskStatusCopy(pTaskEntry, p); - STaskCkptInfo *pChkInfo = &p->checkpointInfo; if ((pChkInfo->activeId != 0) && pChkInfo->failed) { mError("stream task:0x%" PRIx64 " checkpointId:%" PRIx64 " transId:%d failed, kill it", p->id.taskId, pChkInfo->activeId, pChkInfo->activeTransId); @@ -304,6 +320,21 @@ int32_t mndProcessStreamHb(SRpcMsg *pReq) { // remove failed trans from pChkptStreams taosHashRemove(execInfo.pChkptStreams, &p->id.streamId, sizeof(p->id.streamId)); } + +/* if (pChkInfo->consensusChkptId != 0) { + SRestoreCheckpointInfo cp = { + .streamId = p->id.streamId, + .taskId = p->id.taskId, + .checkpointId = p->checkpointInfo.latestId, + .startTs = pTaskEntry->startTime, + }; + + SStreamObj* pStream = mndGetStreamObj(pMnode, p->id.streamId); + int32_t numOfTasks = mndGetNumOfStreamTasks(pStream); + SCheckpointConsensusInfo *pInfo = mndGetConsensusInfo(execInfo.pStreamConsensus, p->id.streamId, numOfTasks); + mndAddConsensusTasks(pInfo, &cp, NULL); + mndReleaseStream(pMnode, pStream); + }*/ } if (p->status == pTaskEntry->status) { diff --git a/source/dnode/mnode/impl/src/mndStreamUtil.c b/source/dnode/mnode/impl/src/mndStreamUtil.c index 7d9b9e4571..c4797957c2 100644 --- a/source/dnode/mnode/impl/src/mndStreamUtil.c +++ b/source/dnode/mnode/impl/src/mndStreamUtil.c @@ -953,12 +953,21 @@ SCheckpointConsensusInfo* mndGetConsensusInfo(SHashObj* pHash, int64_t streamId, // no matter existed or not, add the request into info list anyway, since we need to send rsp mannually // discard the msg may lead to the lost of connections. -void mndAddConsensusTasks(SCheckpointConsensusInfo *pInfo, const SRestoreCheckpointInfo *pRestoreInfo, SRpcHandleInfo* pRpcInfo) { - SCheckpointConsensusEntry info = {.ts = taosGetTimestampMs(), .rspInfo = *pRpcInfo}; +void mndAddConsensusTasks(SCheckpointConsensusInfo *pInfo, const SRestoreCheckpointInfo *pRestoreInfo) { + SCheckpointConsensusEntry info = {.ts = taosGetTimestampMs()}; memcpy(&info.req, pRestoreInfo, sizeof(info.req)); - taosArrayPush(pInfo->pTaskList, &info); + for (int32_t i = 0; i < taosArrayGetSize(pInfo->pTaskList); ++i) { + SCheckpointConsensusEntry *p = taosArrayGet(pInfo->pTaskList, i); + if (p->req.taskId == info.req.taskId) { + mDebug("s-task:0x%x already in consensus-checkpointId list for stream:0x%" PRIx64 + ", ignore this, total existed:%d", + pRestoreInfo->taskId, pRestoreInfo->streamId, (int32_t)taosArrayGetSize(pInfo->pTaskList)); + return; + } + } + taosArrayPush(pInfo->pTaskList, &info); int32_t num = taosArrayGetSize(pInfo->pTaskList); mDebug("s-task:0x%x added into consensus-checkpointId list, stream:0x%" PRIx64 " total waiting:%d", pRestoreInfo->taskId, pRestoreInfo->streamId, num); diff --git a/source/dnode/vnode/src/tqCommon/tqCommon.c b/source/dnode/vnode/src/tqCommon/tqCommon.c index cb480d09bb..dc55acbb5c 100644 --- a/source/dnode/vnode/src/tqCommon/tqCommon.c +++ b/source/dnode/vnode/src/tqCommon/tqCommon.c @@ -742,6 +742,7 @@ static int32_t restartStreamTasks(SStreamMeta* pMeta, bool isLeader) { streamMetaStartAllTasks(pMeta); } else { streamMetaResetStartInfo(&pMeta->startInfo, pMeta->vgId); + pMeta->startInfo.restartCount = 0; streamMetaWUnLock(pMeta); tqInfo("vgId:%d, follower node not start stream tasks or stream is disabled", vgId); } @@ -1160,7 +1161,7 @@ int32_t tqStreamTaskProcessConsenChkptIdReq(SStreamMeta* pMeta, SRpcMsg* pMsg) { streamMetaAddFailedTask(pMeta, req.streamId, req.taskId); return TSDB_CODE_SUCCESS; } - +#if 0 // discard the rsp, since it is expired. if (req.startTs < pTask->execInfo.created) { tqWarn("s-task:%s vgId:%d create time:%" PRId64 " recv expired consensus checkpointId:%" PRId64 @@ -1170,7 +1171,7 @@ int32_t tqStreamTaskProcessConsenChkptIdReq(SStreamMeta* pMeta, SRpcMsg* pMsg) { streamMetaReleaseTask(pMeta, pTask); return TSDB_CODE_SUCCESS; } - +#endif tqDebug("s-task:%s vgId:%d checkpointId:%" PRId64 " restore to consensus-checkpointId:%" PRId64 " from mnode", pTask->id.idStr, vgId, pTask->chkInfo.checkpointId, req.checkpointId); diff --git a/source/libs/stream/src/streamCheckpoint.c b/source/libs/stream/src/streamCheckpoint.c index 5cd084e6a2..cdb5bf0b50 100644 --- a/source/libs/stream/src/streamCheckpoint.c +++ b/source/libs/stream/src/streamCheckpoint.c @@ -1121,7 +1121,8 @@ int32_t streamTaskSendRestoreChkptMsg(SStreamTask* pTask) { taosThreadMutexUnlock(&pTask->lock); ASSERT(pTask->pBackend == NULL); - + pTask->status.requireConsensusChkptId = true; +#if 0 SRestoreCheckpointInfo req = { .streamId = pTask->id.streamId, .taskId = pTask->id.taskId, @@ -1158,6 +1159,7 @@ int32_t streamTaskSendRestoreChkptMsg(SStreamTask* pTask) { pInfo->checkpointId); tmsgSendReq(&pTask->info.mnodeEpset, &msg); +#endif return 0; } diff --git a/source/libs/stream/src/streamHb.c b/source/libs/stream/src/streamHb.c index 691ec44672..6b1a1aca92 100644 --- a/source/libs/stream/src/streamHb.c +++ b/source/libs/stream/src/streamHb.c @@ -194,6 +194,12 @@ int32_t streamMetaSendHbHelper(SStreamMeta* pMeta) { } } + if ((*pTask)->status.requireConsensusChkptId) { + entry.checkpointInfo.consensusChkptId = 1; + (*pTask)->status.requireConsensusChkptId = false; + stDebug("s-task:%s vgId:%d set the require consensus-checkpointId in hbMsg", (*pTask)->id.idStr, pMeta->vgId); + } + if ((*pTask)->exec.pWalReader != NULL) { entry.processedVer = walReaderGetCurrentVer((*pTask)->exec.pWalReader) - 1; if (entry.processedVer < 0) { diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 19cb2f7854..e7fdb7ae2a 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -1031,10 +1031,11 @@ void streamMetaResetStartInfo(STaskStartInfo* pStartInfo, int32_t vgId) { taosHashClear(pStartInfo->pFailedTaskSet); pStartInfo->tasksWillRestart = 0; pStartInfo->readyTs = 0; + pStartInfo->elapsedTime = 0; // reset the sentinel flag value to be 0 pStartInfo->startAllTasks = 0; - stDebug("vgId:%d clear all start-all-task info", vgId); + stDebug("vgId:%d clear start-all-task info", vgId); } void streamMetaRLock(SStreamMeta* pMeta) { diff --git a/source/libs/stream/src/streamMsg.c b/source/libs/stream/src/streamMsg.c index f02f661143..f10296f6ff 100644 --- a/source/libs/stream/src/streamMsg.c +++ b/source/libs/stream/src/streamMsg.c @@ -349,6 +349,7 @@ int32_t tEncodeStreamHbMsg(SEncoder* pEncoder, const SStreamHbMsg* pReq) { if (tEncodeI64(pEncoder, ps->checkpointInfo.latestTime) < 0) return -1; if (tEncodeI64(pEncoder, ps->checkpointInfo.latestSize) < 0) return -1; if (tEncodeI8(pEncoder, ps->checkpointInfo.remoteBackup) < 0) return -1; + if (tEncodeI8(pEncoder, ps->checkpointInfo.consensusChkptId) < 0) return -1; if (tEncodeI64(pEncoder, ps->startTime) < 0) return -1; if (tEncodeI64(pEncoder, ps->startCheckpointId) < 0) return -1; if (tEncodeI64(pEncoder, ps->startCheckpointVer) < 0) return -1; @@ -403,6 +404,7 @@ int32_t tDecodeStreamHbMsg(SDecoder* pDecoder, SStreamHbMsg* pReq) { if (tDecodeI64(pDecoder, &entry.checkpointInfo.latestTime) < 0) return -1; if (tDecodeI64(pDecoder, &entry.checkpointInfo.latestSize) < 0) return -1; if (tDecodeI8(pDecoder, &entry.checkpointInfo.remoteBackup) < 0) return -1; + if (tDecodeI8(pDecoder, &entry.checkpointInfo.consensusChkptId) < 0) return -1; if (tDecodeI64(pDecoder, &entry.startTime) < 0) return -1; if (tDecodeI64(pDecoder, &entry.startCheckpointId) < 0) return -1; if (tDecodeI64(pDecoder, &entry.startCheckpointVer) < 0) return -1; diff --git a/source/libs/stream/src/streamTask.c b/source/libs/stream/src/streamTask.c index 1decfe198a..5506ed2d45 100644 --- a/source/libs/stream/src/streamTask.c +++ b/source/libs/stream/src/streamTask.c @@ -848,6 +848,7 @@ STaskStatusEntry streamTaskGetStatusEntry(SStreamTask* pTask) { .checkpointInfo.latestTime = pTask->chkInfo.checkpointTime, .checkpointInfo.latestSize = 0, .checkpointInfo.remoteBackup = 0, + .checkpointInfo.consensusChkptId = 0, .hTaskId = pTask->hTaskInfo.id.taskId, .procsTotal = SIZE_IN_MiB(pExecInfo->inputDataSize), .outputTotal = SIZE_IN_MiB(pExecInfo->outputDataSize), From 647f9f47ef6602509d5330391965cf9b532cd275 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 8 Jul 2024 10:32:23 +0800 Subject: [PATCH 12/22] refactor: do some internal refactor. --- include/common/tmsgdef.h | 1 - include/libs/stream/streamMsg.h | 1 - source/dnode/mgmt/mgmt_mnode/src/mmHandle.c | 1 - source/dnode/mgmt/mgmt_snode/src/smHandle.c | 1 - source/dnode/mgmt/mgmt_vnode/src/vmHandle.c | 1 - source/dnode/mnode/impl/src/mndStream.c | 1 - source/dnode/mnode/impl/src/mndStreamUtil.c | 2 +- source/dnode/snode/src/snode.c | 2 -- source/dnode/vnode/src/vnd/vnodeSvr.c | 2 -- 9 files changed, 1 insertion(+), 11 deletions(-) diff --git a/include/common/tmsgdef.h b/include/common/tmsgdef.h index 7621615278..3515df3127 100644 --- a/include/common/tmsgdef.h +++ b/include/common/tmsgdef.h @@ -250,7 +250,6 @@ TD_DEF_MSG_TYPE(TDMT_MND_DROP_TB_WITH_TSMA, "drop-tb-with-tsma", NULL, NULL) TD_DEF_MSG_TYPE(TDMT_MND_STREAM_UPDATE_CHKPT_EVT, "stream-update-chkpt-evt", NULL, NULL) TD_DEF_MSG_TYPE(TDMT_MND_STREAM_CHKPT_REPORT, "stream-chkpt-report", NULL, NULL) - TD_DEF_MSG_TYPE(TDMT_MND_STREAM_REQ_CONSEN_CHKPT, "stream-req-consen-chkpt", NULL, NULL) TD_DEF_MSG_TYPE(TDMT_MND_STREAM_CONSEN_TIMER, "stream-consen-tmr", NULL, NULL) TD_DEF_MSG_TYPE(TDMT_MND_MAX_MSG, "mnd-max", NULL, NULL) TD_CLOSE_MSG_SEG(TDMT_END_MND_MSG) diff --git a/include/libs/stream/streamMsg.h b/include/libs/stream/streamMsg.h index bdb8ff7f8e..b253054dfe 100644 --- a/include/libs/stream/streamMsg.h +++ b/include/libs/stream/streamMsg.h @@ -232,7 +232,6 @@ typedef struct { typedef struct SCheckpointConsensusEntry { SRestoreCheckpointInfo req; - SRpcHandleInfo rspInfo; int64_t ts; } SCheckpointConsensusEntry; diff --git a/source/dnode/mgmt/mgmt_mnode/src/mmHandle.c b/source/dnode/mgmt/mgmt_mnode/src/mmHandle.c index f3763ef0c5..9b987b3237 100644 --- a/source/dnode/mgmt/mgmt_mnode/src/mmHandle.c +++ b/source/dnode/mgmt/mgmt_mnode/src/mmHandle.c @@ -243,7 +243,6 @@ SArray *mmGetMsgHandles() { if (dmSetMgmtHandle(pArray, TDMT_VND_STREAM_TASK_RESET_RSP, mmPutMsgToWriteQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_MND_STREAM_HEARTBEAT, mmPutMsgToReadQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_MND_STREAM_CHKPT_REPORT, mmPutMsgToWriteQueue, 0) == NULL) goto _OVER; - if (dmSetMgmtHandle(pArray, TDMT_MND_STREAM_REQ_CONSEN_CHKPT, mmPutMsgToWriteQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_MND_STREAM_REQ_CHKPT, mmPutMsgToWriteQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_VND_KILL_COMPACT_RSP, mmPutMsgToWriteQueue, 0) == NULL) goto _OVER; diff --git a/source/dnode/mgmt/mgmt_snode/src/smHandle.c b/source/dnode/mgmt/mgmt_snode/src/smHandle.c index 3d0587a11b..5c2f54fd10 100644 --- a/source/dnode/mgmt/mgmt_snode/src/smHandle.c +++ b/source/dnode/mgmt/mgmt_snode/src/smHandle.c @@ -97,7 +97,6 @@ SArray *smGetMsgHandles() { if (dmSetMgmtHandle(pArray, TDMT_MND_STREAM_HEARTBEAT_RSP, smPutNodeMsgToStreamQueue, 1) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_MND_STREAM_REQ_CHKPT_RSP, smPutNodeMsgToStreamQueue, 1) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_MND_STREAM_CHKPT_REPORT_RSP, smPutNodeMsgToStreamQueue, 1) == NULL) goto _OVER; - if (dmSetMgmtHandle(pArray, TDMT_MND_STREAM_REQ_CONSEN_CHKPT_RSP, smPutNodeMsgToStreamQueue, 1) == NULL) goto _OVER; code = 0; _OVER: diff --git a/source/dnode/mgmt/mgmt_vnode/src/vmHandle.c b/source/dnode/mgmt/mgmt_vnode/src/vmHandle.c index 7d35fd71b7..fbe1925e3f 100644 --- a/source/dnode/mgmt/mgmt_vnode/src/vmHandle.c +++ b/source/dnode/mgmt/mgmt_vnode/src/vmHandle.c @@ -972,7 +972,6 @@ SArray *vmGetMsgHandles() { if (dmSetMgmtHandle(pArray, TDMT_MND_STREAM_HEARTBEAT_RSP, vmPutMsgToStreamQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_MND_STREAM_REQ_CHKPT_RSP, vmPutMsgToStreamQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_MND_STREAM_CHKPT_REPORT_RSP, vmPutMsgToStreamQueue, 0) == NULL) goto _OVER; - if (dmSetMgmtHandle(pArray, TDMT_MND_STREAM_REQ_CONSEN_CHKPT_RSP, vmPutMsgToStreamQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_VND_GET_STREAM_PROGRESS, vmPutMsgToStreamQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_STREAM_TASK_UPDATE_CHKPT, vmPutMsgToWriteQueue, 0) == NULL) goto _OVER; diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index cd99714395..4faa8bdb58 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -121,7 +121,6 @@ int32_t mndInitStream(SMnode *pMnode) { mndSetMsgHandle(pMnode, TDMT_MND_STREAM_BEGIN_CHECKPOINT, mndProcessStreamCheckpoint); mndSetMsgHandle(pMnode, TDMT_MND_STREAM_REQ_CHKPT, mndProcessStreamReqCheckpoint); mndSetMsgHandle(pMnode, TDMT_MND_STREAM_CHKPT_REPORT, mndProcessCheckpointReport); - mndSetMsgHandle(pMnode, TDMT_MND_STREAM_REQ_CONSEN_CHKPT, mndProcessConsensusCheckpointId); mndSetMsgHandle(pMnode, TDMT_MND_STREAM_UPDATE_CHKPT_EVT, mndScanCheckpointReportInfo); mndSetMsgHandle(pMnode, TDMT_STREAM_TASK_REPORT_CHECKPOINT, mndTransProcessRsp); mndSetMsgHandle(pMnode, TDMT_MND_STREAM_HEARTBEAT, mndProcessStreamHb); diff --git a/source/dnode/mnode/impl/src/mndStreamUtil.c b/source/dnode/mnode/impl/src/mndStreamUtil.c index c4797957c2..430bfcc3a2 100644 --- a/source/dnode/mnode/impl/src/mndStreamUtil.c +++ b/source/dnode/mnode/impl/src/mndStreamUtil.c @@ -103,7 +103,7 @@ SArray *mndTakeVgroupSnapshot(SMnode *pMnode, bool *allReady) { } else { if (replica != pVgroup->replica) { mInfo("vgId:%d replica:%d inconsistent with other vgroups replica:%d, not ready for stream operations", - pVgroup->vgId); + pVgroup->vgId, pVgroup->replica, replica); *allReady = false; break; } diff --git a/source/dnode/snode/src/snode.c b/source/dnode/snode/src/snode.c index 60e57e8a2f..cfa24b2430 100644 --- a/source/dnode/snode/src/snode.c +++ b/source/dnode/snode/src/snode.c @@ -128,8 +128,6 @@ int32_t sndProcessStreamMsg(SSnode *pSnode, SRpcMsg *pMsg) { return tqStreamTaskProcessRetrieveTriggerReq(pSnode->pMeta, pMsg); case TDMT_STREAM_RETRIEVE_TRIGGER_RSP: return tqStreamTaskProcessRetrieveTriggerRsp(pSnode->pMeta, pMsg); - case TDMT_MND_STREAM_REQ_CONSEN_CHKPT_RSP: - return tqStreamProcessConsensusChkptRsp2(pSnode->pMeta, pMsg); default: sndError("invalid snode msg:%d", pMsg->msgType); ASSERT(0); diff --git a/source/dnode/vnode/src/vnd/vnodeSvr.c b/source/dnode/vnode/src/vnd/vnodeSvr.c index 04839c3357..9dcf7e53c7 100644 --- a/source/dnode/vnode/src/vnd/vnodeSvr.c +++ b/source/dnode/vnode/src/vnd/vnodeSvr.c @@ -861,8 +861,6 @@ int32_t vnodeProcessStreamMsg(SVnode *pVnode, SRpcMsg *pMsg, SQueueInfo *pInfo) return tqStreamProgressRetrieveReq(pVnode->pTq, pMsg); case TDMT_MND_STREAM_CHKPT_REPORT_RSP: return tqProcessTaskChkptReportRsp(pVnode->pTq, pMsg); - case TDMT_MND_STREAM_REQ_CONSEN_CHKPT_RSP: - return tqProcessTaskConsensusChkptRsp(pVnode->pTq, pMsg); default: vError("unknown msg type:%d in stream queue", pMsg->msgType); return TSDB_CODE_APP_ERROR; From 6e15c16cf722d6bb8aff2c3b4edf8591289445d0 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 8 Jul 2024 14:33:28 +0800 Subject: [PATCH 13/22] fix(stream): check the set consensus-checkpointId ts. --- include/libs/stream/tstream.h | 3 +- source/dnode/mnode/impl/src/mndStream.c | 225 +++++++++++--------- source/dnode/mnode/impl/src/mndStreamHb.c | 17 +- source/dnode/mnode/impl/src/mndStreamUtil.c | 13 +- source/dnode/vnode/src/tqCommon/tqCommon.c | 4 +- source/libs/stream/src/streamMsg.c | 2 + source/libs/stream/src/streamTask.c | 1 + 7 files changed, 135 insertions(+), 130 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 5c61265c01..e275c1511d 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -578,6 +578,7 @@ typedef struct STaskCkptInfo { int32_t activeTransId; // checkpoint trans id int8_t failed; // denote if the checkpoint is failed or not int8_t consensusChkptId; // required the consensus-checkpointId + int64_t consensusTs; // } STaskCkptInfo; typedef struct STaskStatusEntry { @@ -588,8 +589,6 @@ typedef struct STaskStatusEntry { int32_t nodeId; SVersionRange verRange; // start/end version in WAL, only valid for source task int64_t processedVer; // only valid for source task - bool inputQChanging; // inputQ is changing or not - int64_t inputQUnchangeCounter; double inputQUsed; // in MiB double inputRate; double procsThroughput; // duration between one element put into input queue and being processed. diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index 4faa8bdb58..d2252d1bee 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -59,7 +59,7 @@ static int32_t mndProcessNodeCheckReq(SRpcMsg *pMsg); static int32_t extractNodeListFromStream(SMnode *pMnode, SArray *pNodeList); static int32_t mndProcessStreamReqCheckpoint(SRpcMsg *pReq); static int32_t mndProcessCheckpointReport(SRpcMsg *pReq); -static int32_t mndProcessConsensusCheckpointId(SRpcMsg *pMsg); +//static int32_t mndProcessConsensusCheckpointId(SRpcMsg *pMsg); static int32_t mndProcessConsensusInTmr(SRpcMsg *pMsg); static void doSendQuickRsp(SRpcHandleInfo *pInfo, int32_t msgSize, int32_t vgId, int32_t code); @@ -2617,10 +2617,11 @@ int32_t mndProcessCheckpointReport(SRpcMsg *pReq) { return 0; } -static int64_t getConsensusId(int64_t streamId, int32_t numOfTasks, int32_t* pExistedTasks) { +static int64_t getConsensusId(int64_t streamId, int32_t numOfTasks, int32_t* pExistedTasks, bool *pAllSame) { int32_t num = 0; int64_t chkId = INT64_MAX; *pExistedTasks = 0; + *pAllSame = true; for(int32_t i = 0; i < taosArrayGetSize(execInfo.pTaskList); ++i) { STaskId* p = taosArrayGet(execInfo.pTaskList, i); @@ -2631,6 +2632,9 @@ static int64_t getConsensusId(int64_t streamId, int32_t numOfTasks, int32_t* pEx num += 1; STaskStatusEntry* pe = taosHashGet(execInfo.pTaskMap, p, sizeof(*p)); if (chkId > pe->checkpointInfo.latestId) { + if (chkId != INT64_MAX) { + *pAllSame = false; + } chkId = pe->checkpointInfo.latestId; } } @@ -2653,99 +2657,99 @@ static void doSendQuickRsp(SRpcHandleInfo *pInfo, int32_t msgSize, int32_t vgId, pInfo->handle = NULL; // disable auto rsp } -static int32_t mndProcessConsensusCheckpointId(SRpcMsg *pMsg) { - SMnode *pMnode = pMsg->info.node; - SDecoder decoder = {0}; - - SRestoreCheckpointInfo req = {0}; - tDecoderInit(&decoder, pMsg->pCont, pMsg->contLen); - - if (tDecodeRestoreCheckpointInfo(&decoder, &req)) { - tDecoderClear(&decoder); - terrno = TSDB_CODE_INVALID_MSG; - mError("invalid task consensus-checkpoint msg received"); - return -1; - } - tDecoderClear(&decoder); - - mDebug("receive stream task consensus-checkpoint msg, vgId:%d, s-task:0x%" PRIx64 "-0x%x, checkpointId:%" PRId64, - req.nodeId, req.streamId, req.taskId, req.checkpointId); - - // register to the stream task done map, if all tasks has sent this kinds of message, start the checkpoint trans. - taosThreadMutexLock(&execInfo.lock); - - // mnode handle the create stream transaction too slow may cause this problem - SStreamObj *pStream = mndGetStreamObj(pMnode, req.streamId); - if (pStream == NULL) { - mWarn("failed to find the stream:0x%" PRIx64 ", not handle consensus-checkpointId", req.streamId); - - // not in meta-store yet, try to acquire the task in exec buffer - // the checkpoint req arrives too soon before the completion of the create stream trans. - STaskId id = {.streamId = req.streamId, .taskId = req.taskId}; - void *p = taosHashGet(execInfo.pTaskMap, &id, sizeof(id)); - if (p == NULL) { - mError("failed to find the stream:0x%" PRIx64 " in buf, not handle consensus-checkpointId", req.streamId); - terrno = TSDB_CODE_MND_STREAM_NOT_EXIST; - taosThreadMutexUnlock(&execInfo.lock); - - doSendQuickRsp(&pMsg->info, sizeof(SMStreamReqConsensChkptRsp), req.nodeId, terrno); - return -1; - } else { - mDebug("s-task:0x%" PRIx64 "-0x%x in buf not in mnode/meta, create stream trans may not complete yet", - req.streamId, req.taskId); - // todo wait for stream is created - } - } - - mInfo("vgId:%d stream:0x%" PRIx64 " %s meta-stored checkpointId:%" PRId64, req.nodeId, req.streamId, pStream->name, - pStream->checkpointId); - - int32_t numOfTasks = (pStream == NULL) ? 0 : mndGetNumOfStreamTasks(pStream); - if ((pStream != NULL) && (pStream->checkpointId == 0)) { // not generated checkpoint yet, return 0 directly - taosThreadMutexUnlock(&execInfo.lock); - mndCreateSetConsensusChkptIdTrans(pMnode, pStream, req.taskId, 0, req.startTs); - - doSendQuickRsp(&pMsg->info, sizeof(SMStreamReqConsensChkptRsp), req.nodeId, terrno); - return TSDB_CODE_SUCCESS; - } - - int32_t num = 0; - int64_t chkId = getConsensusId(req.streamId, numOfTasks, &num); - - // some tasks not send hbMsg to mnode yet, wait for 5s. - if (chkId == -1) { - mDebug("not all(%d/%d) task(s) send hbMsg yet, wait for a while and check again, s-task:0x%x", req.taskId, num, - numOfTasks); - SCheckpointConsensusInfo *pInfo = mndGetConsensusInfo(execInfo.pStreamConsensus, req.streamId, numOfTasks); - mndAddConsensusTasks(pInfo, &req); - - taosThreadMutexUnlock(&execInfo.lock); - doSendQuickRsp(&pMsg->info, sizeof(SMStreamReqConsensChkptRsp), req.nodeId, terrno); - return 0; - } - - if (chkId == req.checkpointId) { - mDebug("vgId:%d stream:0x%" PRIx64 " %s consensus-checkpointId is:%" PRId64 ", meta-stored checkpointId:%" PRId64, - req.nodeId, req.streamId, pStream->name, chkId, pStream->checkpointId); - mndCreateSetConsensusChkptIdTrans(pMnode, pStream, req.taskId, chkId, req.startTs); - - taosThreadMutexUnlock(&execInfo.lock); - doSendQuickRsp(&pMsg->info, sizeof(SMStreamReqConsensChkptRsp), req.nodeId, terrno); - return 0; - } - - // wait for 5s and check again - SCheckpointConsensusInfo *pInfo = mndGetConsensusInfo(execInfo.pStreamConsensus, req.streamId, numOfTasks); - mndAddConsensusTasks(pInfo, &req); - - if (pStream != NULL) { - mndReleaseStream(pMnode, pStream); - } - - taosThreadMutexUnlock(&execInfo.lock); - doSendQuickRsp(&pMsg->info, sizeof(SMStreamReqConsensChkptRsp), req.nodeId, terrno); - return 0; -} +//static int32_t mndProcessConsensusCheckpointId(SRpcMsg *pMsg) { +// SMnode *pMnode = pMsg->info.node; +// SDecoder decoder = {0}; +// +// SRestoreCheckpointInfo req = {0}; +// tDecoderInit(&decoder, pMsg->pCont, pMsg->contLen); +// +// if (tDecodeRestoreCheckpointInfo(&decoder, &req)) { +// tDecoderClear(&decoder); +// terrno = TSDB_CODE_INVALID_MSG; +// mError("invalid task consensus-checkpoint msg received"); +// return -1; +// } +// tDecoderClear(&decoder); +// +// mDebug("receive stream task consensus-checkpoint msg, vgId:%d, s-task:0x%" PRIx64 "-0x%x, checkpointId:%" PRId64, +// req.nodeId, req.streamId, req.taskId, req.checkpointId); +// +// // register to the stream task done map, if all tasks has sent this kinds of message, start the checkpoint trans. +// taosThreadMutexLock(&execInfo.lock); +// +// // mnode handle the create stream transaction too slow may cause this problem +// SStreamObj *pStream = mndGetStreamObj(pMnode, req.streamId); +// if (pStream == NULL) { +// mWarn("failed to find the stream:0x%" PRIx64 ", not handle consensus-checkpointId", req.streamId); +// +// // not in meta-store yet, try to acquire the task in exec buffer +// // the checkpoint req arrives too soon before the completion of the create stream trans. +// STaskId id = {.streamId = req.streamId, .taskId = req.taskId}; +// void *p = taosHashGet(execInfo.pTaskMap, &id, sizeof(id)); +// if (p == NULL) { +// mError("failed to find the stream:0x%" PRIx64 " in buf, not handle consensus-checkpointId", req.streamId); +// terrno = TSDB_CODE_MND_STREAM_NOT_EXIST; +// taosThreadMutexUnlock(&execInfo.lock); +// +// doSendQuickRsp(&pMsg->info, sizeof(SMStreamReqConsensChkptRsp), req.nodeId, terrno); +// return -1; +// } else { +// mDebug("s-task:0x%" PRIx64 "-0x%x in buf not in mnode/meta, create stream trans may not complete yet", +// req.streamId, req.taskId); +// // todo wait for stream is created +// } +// } +// +// mInfo("vgId:%d stream:0x%" PRIx64 " %s meta-stored checkpointId:%" PRId64, req.nodeId, req.streamId, pStream->name, +// pStream->checkpointId); +// +// int32_t numOfTasks = (pStream == NULL) ? 0 : mndGetNumOfStreamTasks(pStream); +// if ((pStream != NULL) && (pStream->checkpointId == 0)) { // not generated checkpoint yet, return 0 directly +// taosThreadMutexUnlock(&execInfo.lock); +// mndCreateSetConsensusChkptIdTrans(pMnode, pStream, req.taskId, 0, req.startTs); +// +// doSendQuickRsp(&pMsg->info, sizeof(SMStreamReqConsensChkptRsp), req.nodeId, terrno); +// return TSDB_CODE_SUCCESS; +// } +// +// int32_t num = 0; +// int64_t chkId = getConsensusId(req.streamId, numOfTasks, &num); +// +// // some tasks not send hbMsg to mnode yet, wait for 5s. +// if (chkId == -1) { +// mDebug("not all(%d/%d) task(s) send hbMsg yet, wait for a while and check again, s-task:0x%x", req.taskId, num, +// numOfTasks); +// SCheckpointConsensusInfo *pInfo = mndGetConsensusInfo(execInfo.pStreamConsensus, req.streamId, numOfTasks); +// mndAddConsensusTasks(pInfo, &req); +// +// taosThreadMutexUnlock(&execInfo.lock); +// doSendQuickRsp(&pMsg->info, sizeof(SMStreamReqConsensChkptRsp), req.nodeId, terrno); +// return 0; +// } +// +// if (chkId == req.checkpointId) { +// mDebug("vgId:%d stream:0x%" PRIx64 " %s consensus-checkpointId is:%" PRId64 ", meta-stored checkpointId:%" PRId64, +// req.nodeId, req.streamId, pStream->name, chkId, pStream->checkpointId); +// mndCreateSetConsensusChkptIdTrans(pMnode, pStream, req.taskId, chkId, req.startTs); +// +// taosThreadMutexUnlock(&execInfo.lock); +// doSendQuickRsp(&pMsg->info, sizeof(SMStreamReqConsensChkptRsp), req.nodeId, terrno); +// return 0; +// } +// +// // wait for 5s and check again +// SCheckpointConsensusInfo *pInfo = mndGetConsensusInfo(execInfo.pStreamConsensus, req.streamId, numOfTasks); +// mndAddConsensusTasks(pInfo, &req); +// +// if (pStream != NULL) { +// mndReleaseStream(pMnode, pStream); +// } +// +// taosThreadMutexUnlock(&execInfo.lock); +// doSendQuickRsp(&pMsg->info, sizeof(SMStreamReqConsensChkptRsp), req.nodeId, terrno); +// return 0; +//} int32_t mndProcessConsensusInTmr(SRpcMsg *pMsg) { SMnode *pMnode = pMsg->info.node; @@ -2753,6 +2757,15 @@ int32_t mndProcessConsensusInTmr(SRpcMsg *pMsg) { SArray *pStreamList = taosArrayInit(4, sizeof(int64_t)); mDebug("start to process consensus-checkpointId in tmr"); + + bool allReady = true; + SArray *pNodeSnapshot = mndTakeVgroupSnapshot(pMnode, &allReady); + taosArrayDestroy(pNodeSnapshot); + if (!allReady) { + mWarn("not all vnodes are ready, end to process the consensus-checkpointId in tmr process"); + return 0; + } + taosThreadMutexLock(&execInfo.lock); void *pIter = NULL; @@ -2765,7 +2778,8 @@ int32_t mndProcessConsensusInTmr(SRpcMsg *pMsg) { SStreamObj *pStream = mndGetStreamObj(pMnode, pInfo->streamId); if (pStream == NULL) { // stream has been dropped already - mDebug("stream:0x%"PRIx64" dropped already, continue", pInfo->streamId); + mDebug("stream:0x%" PRIx64 " dropped already, continue", pInfo->streamId); + taosArrayDestroy(pList); continue; } @@ -2773,15 +2787,18 @@ int32_t mndProcessConsensusInTmr(SRpcMsg *pMsg) { SCheckpointConsensusEntry *pe = taosArrayGet(pInfo->pTaskList, j); streamId = pe->req.streamId; - if ((now - pe->ts) >= 10 * 1000) { - int32_t existed = 0; - int64_t chkId = getConsensusId(pe->req.streamId, pInfo->numOfTasks, &existed); - if (chkId == -1) { - mDebug("not all(%d/%d) task(s) send hbMsg yet, wait for a while and check again, s-task:0x%x", existed, - pInfo->numOfTasks, pe->req.taskId); - break; - } + int32_t existed = 0; + bool allSame = true; + int64_t chkId = getConsensusId(pe->req.streamId, pInfo->numOfTasks, &existed, &allSame); + if (chkId == -1) { + mDebug("not all(%d/%d) task(s) send hbMsg yet, wait for a while and check again, s-task:0x%x", existed, + pInfo->numOfTasks, pe->req.taskId); + break; + } + if (((now - pe->ts) >= 10 * 1000) || allSame) { + mDebug("s-task:0x%x sendTs:%" PRId64 " wait %.2fs and all tasks have same checkpointId", pe->req.taskId, + (now - pe->ts) / 1000.0, pe->ts); ASSERT(chkId <= pe->req.checkpointId); mndCreateSetConsensusChkptIdTrans(pMnode, pStream, pe->req.taskId, chkId, pe->req.startTs); diff --git a/source/dnode/mnode/impl/src/mndStreamHb.c b/source/dnode/mnode/impl/src/mndStreamHb.c index 8a374c99ef..1452ac77d2 100644 --- a/source/dnode/mnode/impl/src/mndStreamHb.c +++ b/source/dnode/mnode/impl/src/mndStreamHb.c @@ -290,7 +290,7 @@ int32_t mndProcessStreamHb(SRpcMsg *pReq) { .streamId = p->id.streamId, .taskId = p->id.taskId, .checkpointId = p->checkpointInfo.latestId, - .startTs = pTaskEntry->startTime, + .startTs = pChkInfo->consensusTs, }; SStreamObj *pStream = mndGetStreamObj(pMnode, p->id.streamId); @@ -320,21 +320,6 @@ int32_t mndProcessStreamHb(SRpcMsg *pReq) { // remove failed trans from pChkptStreams taosHashRemove(execInfo.pChkptStreams, &p->id.streamId, sizeof(p->id.streamId)); } - -/* if (pChkInfo->consensusChkptId != 0) { - SRestoreCheckpointInfo cp = { - .streamId = p->id.streamId, - .taskId = p->id.taskId, - .checkpointId = p->checkpointInfo.latestId, - .startTs = pTaskEntry->startTime, - }; - - SStreamObj* pStream = mndGetStreamObj(pMnode, p->id.streamId); - int32_t numOfTasks = mndGetNumOfStreamTasks(pStream); - SCheckpointConsensusInfo *pInfo = mndGetConsensusInfo(execInfo.pStreamConsensus, p->id.streamId, numOfTasks); - mndAddConsensusTasks(pInfo, &cp, NULL); - mndReleaseStream(pMnode, pStream); - }*/ } if (p->status == pTaskEntry->status) { diff --git a/source/dnode/mnode/impl/src/mndStreamUtil.c b/source/dnode/mnode/impl/src/mndStreamUtil.c index 430bfcc3a2..be17082200 100644 --- a/source/dnode/mnode/impl/src/mndStreamUtil.c +++ b/source/dnode/mnode/impl/src/mndStreamUtil.c @@ -87,7 +87,7 @@ SArray *mndTakeVgroupSnapshot(SMnode *pMnode, bool *allReady) { int32_t replica = -1; // do the replica check *allReady = true; - SArray *pVgroupListSnapshot = taosArrayInit(4, sizeof(SNodeEntry)); + SArray *pVgroupList = taosArrayInit(4, sizeof(SNodeEntry)); while (1) { pIter = sdbFetch(pSdb, SDB_VGROUP, pIter, (void **)&pVgroup); @@ -133,7 +133,7 @@ SArray *mndTakeVgroupSnapshot(SMnode *pMnode, bool *allReady) { epsetToStr(&entry.epset, buf, tListLen(buf)); mDebug("take node snapshot, nodeId:%d %s", entry.nodeId, buf); - taosArrayPush(pVgroupListSnapshot, &entry); + taosArrayPush(pVgroupList, &entry); sdbRelease(pSdb, pVgroup); } @@ -152,11 +152,11 @@ SArray *mndTakeVgroupSnapshot(SMnode *pMnode, bool *allReady) { epsetToStr(&entry.epset, buf, tListLen(buf)); mDebug("take snode snapshot, nodeId:%d %s", entry.nodeId, buf); - taosArrayPush(pVgroupListSnapshot, &entry); + taosArrayPush(pVgroupList, &entry); sdbRelease(pSdb, pObj); } - return pVgroupListSnapshot; + return pVgroupList; } SStreamObj *mndGetStreamObj(SMnode *pMnode, int64_t streamId) { @@ -960,9 +960,10 @@ void mndAddConsensusTasks(SCheckpointConsensusInfo *pInfo, const SRestoreCheckpo for (int32_t i = 0; i < taosArrayGetSize(pInfo->pTaskList); ++i) { SCheckpointConsensusEntry *p = taosArrayGet(pInfo->pTaskList, i); if (p->req.taskId == info.req.taskId) { - mDebug("s-task:0x%x already in consensus-checkpointId list for stream:0x%" PRIx64 - ", ignore this, total existed:%d", + mDebug("s-task:0x%x already in consensus-checkpointId list for stream:0x%" PRIx64 ", update ts %" PRId64 + "->%" PRId64 " total existed:%d", pRestoreInfo->taskId, pRestoreInfo->streamId, (int32_t)taosArrayGetSize(pInfo->pTaskList)); + p->req.startTs = info.req.startTs; return; } } diff --git a/source/dnode/vnode/src/tqCommon/tqCommon.c b/source/dnode/vnode/src/tqCommon/tqCommon.c index dc55acbb5c..a94c17f735 100644 --- a/source/dnode/vnode/src/tqCommon/tqCommon.c +++ b/source/dnode/vnode/src/tqCommon/tqCommon.c @@ -1161,7 +1161,7 @@ int32_t tqStreamTaskProcessConsenChkptIdReq(SStreamMeta* pMeta, SRpcMsg* pMsg) { streamMetaAddFailedTask(pMeta, req.streamId, req.taskId); return TSDB_CODE_SUCCESS; } -#if 0 + // discard the rsp, since it is expired. if (req.startTs < pTask->execInfo.created) { tqWarn("s-task:%s vgId:%d create time:%" PRId64 " recv expired consensus checkpointId:%" PRId64 @@ -1171,7 +1171,7 @@ int32_t tqStreamTaskProcessConsenChkptIdReq(SStreamMeta* pMeta, SRpcMsg* pMsg) { streamMetaReleaseTask(pMeta, pTask); return TSDB_CODE_SUCCESS; } -#endif + tqDebug("s-task:%s vgId:%d checkpointId:%" PRId64 " restore to consensus-checkpointId:%" PRId64 " from mnode", pTask->id.idStr, vgId, pTask->chkInfo.checkpointId, req.checkpointId); diff --git a/source/libs/stream/src/streamMsg.c b/source/libs/stream/src/streamMsg.c index f10296f6ff..1bc91d6984 100644 --- a/source/libs/stream/src/streamMsg.c +++ b/source/libs/stream/src/streamMsg.c @@ -350,6 +350,7 @@ int32_t tEncodeStreamHbMsg(SEncoder* pEncoder, const SStreamHbMsg* pReq) { if (tEncodeI64(pEncoder, ps->checkpointInfo.latestSize) < 0) return -1; if (tEncodeI8(pEncoder, ps->checkpointInfo.remoteBackup) < 0) return -1; if (tEncodeI8(pEncoder, ps->checkpointInfo.consensusChkptId) < 0) return -1; + if (tEncodeI64(pEncoder, ps->checkpointInfo.consensusTs) < 0) return -1; if (tEncodeI64(pEncoder, ps->startTime) < 0) return -1; if (tEncodeI64(pEncoder, ps->startCheckpointId) < 0) return -1; if (tEncodeI64(pEncoder, ps->startCheckpointVer) < 0) return -1; @@ -405,6 +406,7 @@ int32_t tDecodeStreamHbMsg(SDecoder* pDecoder, SStreamHbMsg* pReq) { if (tDecodeI64(pDecoder, &entry.checkpointInfo.latestSize) < 0) return -1; if (tDecodeI8(pDecoder, &entry.checkpointInfo.remoteBackup) < 0) return -1; if (tDecodeI8(pDecoder, &entry.checkpointInfo.consensusChkptId) < 0) return -1; + if (tDecodeI64(pDecoder, &entry.checkpointInfo.consensusTs) < 0) return -1; if (tDecodeI64(pDecoder, &entry.startTime) < 0) return -1; if (tDecodeI64(pDecoder, &entry.startCheckpointId) < 0) return -1; if (tDecodeI64(pDecoder, &entry.startCheckpointVer) < 0) return -1; diff --git a/source/libs/stream/src/streamTask.c b/source/libs/stream/src/streamTask.c index 5506ed2d45..f72a5dd434 100644 --- a/source/libs/stream/src/streamTask.c +++ b/source/libs/stream/src/streamTask.c @@ -849,6 +849,7 @@ STaskStatusEntry streamTaskGetStatusEntry(SStreamTask* pTask) { .checkpointInfo.latestSize = 0, .checkpointInfo.remoteBackup = 0, .checkpointInfo.consensusChkptId = 0, + .checkpointInfo.consensusTs = taosGetTimestampMs(), .hTaskId = pTask->hTaskInfo.id.taskId, .procsTotal = SIZE_IN_MiB(pExecInfo->inputDataSize), .outputTotal = SIZE_IN_MiB(pExecInfo->outputDataSize), From 0cbf88f021ebe431f5e80c8ca4b606888529ec15 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 8 Jul 2024 14:51:08 +0800 Subject: [PATCH 14/22] fix(stream): fix memory leak. --- source/dnode/mnode/impl/src/mndStream.c | 1 + 1 file changed, 1 insertion(+) diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index d2252d1bee..77a6cb64b2 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -2763,6 +2763,7 @@ int32_t mndProcessConsensusInTmr(SRpcMsg *pMsg) { taosArrayDestroy(pNodeSnapshot); if (!allReady) { mWarn("not all vnodes are ready, end to process the consensus-checkpointId in tmr process"); + taosArrayDestroy(pStreamList); return 0; } From 56b7ec3d89556e5ecaff31fe42b079491bd3f89a Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 8 Jul 2024 15:10:00 +0800 Subject: [PATCH 15/22] fix(stream): update some logs. --- source/dnode/mnode/impl/src/mndStreamUtil.c | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/source/dnode/mnode/impl/src/mndStreamUtil.c b/source/dnode/mnode/impl/src/mndStreamUtil.c index be17082200..e14d2a0557 100644 --- a/source/dnode/mnode/impl/src/mndStreamUtil.c +++ b/source/dnode/mnode/impl/src/mndStreamUtil.c @@ -962,7 +962,8 @@ void mndAddConsensusTasks(SCheckpointConsensusInfo *pInfo, const SRestoreCheckpo if (p->req.taskId == info.req.taskId) { mDebug("s-task:0x%x already in consensus-checkpointId list for stream:0x%" PRIx64 ", update ts %" PRId64 "->%" PRId64 " total existed:%d", - pRestoreInfo->taskId, pRestoreInfo->streamId, (int32_t)taosArrayGetSize(pInfo->pTaskList)); + pRestoreInfo->taskId, pRestoreInfo->streamId, p->req.startTs, info.req.startTs, + (int32_t)taosArrayGetSize(pInfo->pTaskList)); p->req.startTs = info.req.startTs; return; } From 7cfdf0c14da8f74d8a7854b5a30540854818787e Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 8 Jul 2024 15:43:29 +0800 Subject: [PATCH 16/22] fix(stream): not check the total number of sub tasks with different status. --- source/libs/stream/src/streamCheckStatus.c | 42 +++++++++++----------- 1 file changed, 21 insertions(+), 21 deletions(-) diff --git a/source/libs/stream/src/streamCheckStatus.c b/source/libs/stream/src/streamCheckStatus.c index 2d8fe4a0e1..226a06be7e 100644 --- a/source/libs/stream/src/streamCheckStatus.c +++ b/source/libs/stream/src/streamCheckStatus.c @@ -703,31 +703,31 @@ void rspMonitorFn(void* param, void* tmrId) { if (pStat->state == TASK_STATUS__UNINIT) { getCheckRspStatus(pInfo, timeoutDuration, &numOfReady, &numOfFault, &numOfNotRsp, pTimeoutList, pNotReadyList, id); + + numOfNotReady = (int32_t)taosArrayGetSize(pNotReadyList); + numOfTimeout = (int32_t)taosArrayGetSize(pTimeoutList); + + // fault tasks detected, not try anymore + ASSERT((numOfReady + numOfFault + numOfNotReady + numOfTimeout + numOfNotRsp) == total); + if (numOfFault > 0) { + int32_t ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1); + stDebug( + "s-task:%s status:%s vgId:%d all rsp. quit from monitor rsp tmr, since vnode-transfer/leader-change/restart " + "detected, total:%d, notRsp:%d, notReady:%d, fault:%d, timeout:%d, ready:%d ref:%d", + id, pStat->name, vgId, total, numOfNotRsp, numOfNotReady, numOfFault, numOfTimeout, numOfReady, ref); + + streamTaskCompleteCheckRsp(pInfo, false, id); + taosThreadMutexUnlock(&pInfo->checkInfoLock); + streamMetaReleaseTask(pMeta, pTask); + + taosArrayDestroy(pNotReadyList); + taosArrayDestroy(pTimeoutList); + return; + } } else { // unexpected status stError("s-task:%s unexpected task status:%s during waiting for check rsp", id, pStat->name); } - numOfNotReady = (int32_t)taosArrayGetSize(pNotReadyList); - numOfTimeout = (int32_t)taosArrayGetSize(pTimeoutList); - - // fault tasks detected, not try anymore - ASSERT((numOfReady + numOfFault + numOfNotReady + numOfTimeout + numOfNotRsp) == total); - if (numOfFault > 0) { - int32_t ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1); - stDebug( - "s-task:%s status:%s vgId:%d all rsp. quit from monitor rsp tmr, since vnode-transfer/leader-change/restart " - "detected, total:%d, notRsp:%d, notReady:%d, fault:%d, timeout:%d, ready:%d ref:%d", - id, pStat->name, vgId, total, numOfNotRsp, numOfNotReady, numOfFault, numOfTimeout, numOfReady, ref); - - streamTaskCompleteCheckRsp(pInfo, false, id); - taosThreadMutexUnlock(&pInfo->checkInfoLock); - streamMetaReleaseTask(pMeta, pTask); - - taosArrayDestroy(pNotReadyList); - taosArrayDestroy(pTimeoutList); - return; - } - // checking of downstream tasks has been stopped by other threads if (pInfo->stopCheckProcess == 1) { int32_t ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1); From de7e25f259042f668962188eaf6e9cfd6a467c7f Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 8 Jul 2024 16:57:13 +0800 Subject: [PATCH 17/22] fix(stream): fix race condition in handling the hbMsg rsp. --- source/libs/stream/src/streamHb.c | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/source/libs/stream/src/streamHb.c b/source/libs/stream/src/streamHb.c index 6b1a1aca92..16cb23de10 100644 --- a/source/libs/stream/src/streamHb.c +++ b/source/libs/stream/src/streamHb.c @@ -332,7 +332,7 @@ int32_t streamProcessHeartbeatRsp(SStreamMeta* pMeta, SMStreamHbRspMsg* pRsp) { stDebug("vgId:%d process hbMsg rsp, msgId:%d rsp confirmed", pMeta->vgId, pRsp->msgId); SMetaHbInfo* pInfo = pMeta->pHbInfo; - streamMetaRLock(pMeta); + streamMetaWLock(pMeta); // current waiting rsp recved if (pRsp->msgId == pInfo->hbCount) { @@ -345,6 +345,6 @@ int32_t streamProcessHeartbeatRsp(SStreamMeta* pMeta, SMStreamHbRspMsg* pRsp) { stWarn("vgId:%d recv expired hb rsp, msgId:%d, discarded", pMeta->vgId, pRsp->msgId); } - streamMetaRUnLock(pMeta); + streamMetaWUnLock(pMeta); return TSDB_CODE_SUCCESS; } \ No newline at end of file From 50a2ef08bdbd62f1aecc5223fef9ce9801d4cbf7 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 8 Jul 2024 17:10:34 +0800 Subject: [PATCH 18/22] fix(stream): update some logs. --- source/dnode/mnode/impl/src/mndStreamUtil.c | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/source/dnode/mnode/impl/src/mndStreamUtil.c b/source/dnode/mnode/impl/src/mndStreamUtil.c index e14d2a0557..0e498f20f6 100644 --- a/source/dnode/mnode/impl/src/mndStreamUtil.c +++ b/source/dnode/mnode/impl/src/mndStreamUtil.c @@ -971,8 +971,9 @@ void mndAddConsensusTasks(SCheckpointConsensusInfo *pInfo, const SRestoreCheckpo taosArrayPush(pInfo->pTaskList, &info); int32_t num = taosArrayGetSize(pInfo->pTaskList); - mDebug("s-task:0x%x added into consensus-checkpointId list, stream:0x%" PRIx64 " total waiting:%d", - pRestoreInfo->taskId, pRestoreInfo->streamId, num); + mDebug("s-task:0x%x checkpointId:%" PRId64 " added into consensus-checkpointId list, stream:0x%" PRIx64 + " waiting tasks:%d", + pRestoreInfo->taskId, pRestoreInfo->checkpointId, pRestoreInfo->streamId, num); } void mndClearConsensusRspEntry(SCheckpointConsensusInfo* pInfo) { From bdced636b3e611718ff096fd740e650454d5fd91 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 8 Jul 2024 18:20:35 +0800 Subject: [PATCH 19/22] fix(stream): free task state when stopping stream tasks. --- include/libs/stream/tstream.h | 3 ++- source/libs/stream/src/streamTask.c | 18 +++++++++++------- source/libs/stream/src/streamTaskSm.c | 22 ++++++++++++++-------- 3 files changed, 27 insertions(+), 16 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index e275c1511d..f867a82cbb 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -529,10 +529,11 @@ typedef int32_t (*__state_trans_user_fn)(SStreamTask*, void* param); SStreamTask* tNewStreamTask(int64_t streamId, int8_t taskLevel, SEpSet* pEpset, bool fillHistory, int64_t triggerParam, SArray* pTaskList, bool hasFillhistory, int8_t subtableWithoutMd5); +void tFreeStreamTask(SStreamTask* pTask); int32_t tEncodeStreamTask(SEncoder* pEncoder, const SStreamTask* pTask); int32_t tDecodeStreamTask(SDecoder* pDecoder, SStreamTask* pTask); -void tFreeStreamTask(SStreamTask* pTask); int32_t streamTaskInit(SStreamTask* pTask, SStreamMeta* pMeta, SMsgCb* pMsgCb, int64_t ver); +void streamFreeTaskState(SStreamTask* pTask, ETaskStatus status); int32_t tDecodeStreamTaskChkInfo(SDecoder* pDecoder, SCheckpointInfo* pChkpInfo); int32_t tDecodeStreamTaskId(SDecoder* pDecoder, STaskId* pTaskId); diff --git a/source/libs/stream/src/streamTask.c b/source/libs/stream/src/streamTask.c index f72a5dd434..9bcad87264 100644 --- a/source/libs/stream/src/streamTask.c +++ b/source/libs/stream/src/streamTask.c @@ -268,13 +268,7 @@ void tFreeStreamTask(SStreamTask* pTask) { } streamTaskCleanupCheckInfo(&pTask->taskCheckInfo); - - if (pTask->pState) { - stDebug("s-task:0x%x start to free task state", taskId); - streamStateClose(pTask->pState, status1 == TASK_STATUS__DROPPING); - taskDbRemoveRef(pTask->pBackend); - pTask->pBackend = NULL; - } + streamFreeTaskState(pTask, status1); if (pTask->pNameMap) { tSimpleHashCleanup(pTask->pNameMap); @@ -311,6 +305,16 @@ void tFreeStreamTask(SStreamTask* pTask) { stDebug("s-task:0x%x free task completed", taskId); } +void streamFreeTaskState(SStreamTask* pTask, ETaskStatus status) { + if (pTask->pState != NULL) { + stDebug("s-task:0x%x start to free task state", pTask->id.taskId); + streamStateClose(pTask->pState, status == TASK_STATUS__DROPPING); + taskDbRemoveRef(pTask->pBackend); + pTask->pBackend = NULL; + pTask->pState = NULL; + } +} + static void setInitialVersionInfo(SStreamTask* pTask, int64_t ver) { SCheckpointInfo* pChkInfo = &pTask->chkInfo; SDataRange* pRange = &pTask->dataRange; diff --git a/source/libs/stream/src/streamTaskSm.c b/source/libs/stream/src/streamTaskSm.c index 85d3e0068a..f2bd99cdaf 100644 --- a/source/libs/stream/src/streamTaskSm.c +++ b/source/libs/stream/src/streamTaskSm.c @@ -79,6 +79,12 @@ static int32_t attachWaitedEvent(SStreamTask* pTask, SFutureHandleEventInfo* pEv return 0; } +static int32_t stopTaskSuccFn(SStreamTask* pTask) { + SStreamTaskSM* pSM = pTask->status.pSM; + streamFreeTaskState(pTask, pSM->current.state); + return TSDB_CODE_SUCCESS; +} + int32_t streamTaskInitStatus(SStreamTask* pTask) { pTask->execInfo.checkTs = taosGetTimestampMs(); stDebug("s-task:%s start init, and check downstream tasks, set the init ts:%" PRId64, pTask->id.idStr, @@ -634,21 +640,21 @@ void doInitStateTransferTable(void) { // resume is completed by restore status of state-machine // stop related event - trans = createStateTransform(TASK_STATUS__READY, TASK_STATUS__STOP, TASK_EVENT_STOP, NULL, NULL, NULL); + trans = createStateTransform(TASK_STATUS__READY, TASK_STATUS__STOP, TASK_EVENT_STOP, NULL, stopTaskSuccFn, NULL); taosArrayPush(streamTaskSMTrans, &trans); - trans = createStateTransform(TASK_STATUS__DROPPING, TASK_STATUS__STOP, TASK_EVENT_STOP, NULL, NULL, NULL); + trans = createStateTransform(TASK_STATUS__DROPPING, TASK_STATUS__STOP, TASK_EVENT_STOP, NULL, stopTaskSuccFn, NULL); taosArrayPush(streamTaskSMTrans, &trans); - trans = createStateTransform(TASK_STATUS__UNINIT, TASK_STATUS__STOP, TASK_EVENT_STOP, NULL, NULL, NULL); + trans = createStateTransform(TASK_STATUS__UNINIT, TASK_STATUS__STOP, TASK_EVENT_STOP, NULL, stopTaskSuccFn, NULL); taosArrayPush(streamTaskSMTrans, &trans); - trans = createStateTransform(TASK_STATUS__STOP, TASK_STATUS__STOP, TASK_EVENT_STOP, NULL, NULL, NULL); + trans = createStateTransform(TASK_STATUS__STOP, TASK_STATUS__STOP, TASK_EVENT_STOP, NULL, stopTaskSuccFn, NULL); taosArrayPush(streamTaskSMTrans, &trans); - trans = createStateTransform(TASK_STATUS__SCAN_HISTORY, TASK_STATUS__STOP, TASK_EVENT_STOP, NULL, NULL, NULL); + trans = createStateTransform(TASK_STATUS__SCAN_HISTORY, TASK_STATUS__STOP, TASK_EVENT_STOP, NULL, stopTaskSuccFn, NULL); taosArrayPush(streamTaskSMTrans, &trans); - trans = createStateTransform(TASK_STATUS__HALT, TASK_STATUS__STOP, TASK_EVENT_STOP, NULL, NULL, NULL); + trans = createStateTransform(TASK_STATUS__HALT, TASK_STATUS__STOP, TASK_EVENT_STOP, NULL, stopTaskSuccFn, NULL); taosArrayPush(streamTaskSMTrans, &trans); - trans = createStateTransform(TASK_STATUS__PAUSE, TASK_STATUS__STOP, TASK_EVENT_STOP, NULL, NULL, NULL); + trans = createStateTransform(TASK_STATUS__PAUSE, TASK_STATUS__STOP, TASK_EVENT_STOP, NULL, stopTaskSuccFn, NULL); taosArrayPush(streamTaskSMTrans, &trans); - trans = createStateTransform(TASK_STATUS__CK, TASK_STATUS__STOP, TASK_EVENT_STOP, NULL, NULL, NULL); + trans = createStateTransform(TASK_STATUS__CK, TASK_STATUS__STOP, TASK_EVENT_STOP, NULL, stopTaskSuccFn, NULL); taosArrayPush(streamTaskSMTrans, &trans); // dropping related event From c71413c2aa3b927252231880f3f7f00af8e053e8 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 9 Jul 2024 14:52:15 +0800 Subject: [PATCH 20/22] fix(stream): discard the repeat send consensus-checkpointId msg. --- include/libs/stream/streamMsg.h | 1 + include/libs/stream/tstream.h | 1 + source/dnode/mnode/impl/src/mndStreamUtil.c | 1 + source/dnode/vnode/src/tqCommon/tqCommon.c | 17 +++++++++++++---- source/libs/stream/src/streamMsg.c | 2 ++ 5 files changed, 18 insertions(+), 4 deletions(-) diff --git a/include/libs/stream/streamMsg.h b/include/libs/stream/streamMsg.h index b253054dfe..34921daac3 100644 --- a/include/libs/stream/streamMsg.h +++ b/include/libs/stream/streamMsg.h @@ -216,6 +216,7 @@ typedef struct SRestoreCheckpointInfo { int64_t startTs; int64_t streamId; int64_t checkpointId; // latest checkpoint id + int32_t transId; // transaction id of the update the consensus-checkpointId transaction int32_t taskId; int32_t nodeId; } SRestoreCheckpointInfo; diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index f867a82cbb..5ba0ce454c 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -273,6 +273,7 @@ typedef struct SCheckpointInfo { int64_t processedVer; int64_t nextProcessVer; // current offset in WAL, not serialize it int64_t msgVer; + int32_t consensusTransId;// consensus checkpoint id SActiveCheckpointInfo* pActiveInfo; } SCheckpointInfo; diff --git a/source/dnode/mnode/impl/src/mndStreamUtil.c b/source/dnode/mnode/impl/src/mndStreamUtil.c index 0e498f20f6..c4adbd0fc3 100644 --- a/source/dnode/mnode/impl/src/mndStreamUtil.c +++ b/source/dnode/mnode/impl/src/mndStreamUtil.c @@ -846,6 +846,7 @@ static int32_t mndStreamSetChkptIdAction(SMnode *pMnode, STrans *pTrans, SStream .checkpointId = checkpointId, .startTs = ts, .nodeId = pTask->info.nodeId, + .transId = pTrans->id, }; int32_t code = 0; diff --git a/source/dnode/vnode/src/tqCommon/tqCommon.c b/source/dnode/vnode/src/tqCommon/tqCommon.c index a94c17f735..1f3c049211 100644 --- a/source/dnode/vnode/src/tqCommon/tqCommon.c +++ b/source/dnode/vnode/src/tqCommon/tqCommon.c @@ -1178,16 +1178,25 @@ int32_t tqStreamTaskProcessConsenChkptIdReq(SStreamMeta* pMeta, SRpcMsg* pMsg) { taosThreadMutexLock(&pTask->lock); ASSERT(pTask->chkInfo.checkpointId >= req.checkpointId); + if (pTask->chkInfo.consensusTransId >= req.transId) { + tqDebug("s-task:%s vgId:%d latest consensus transId:%d, expired consensus trans:%d, discard", + pTask->id.idStr, vgId, pTask->chkInfo.consensusTransId, req.transId); + taosThreadMutexUnlock(&pTask->lock); + streamMetaReleaseTask(pMeta, pTask); + return TSDB_CODE_SUCCESS; + } + if (pTask->chkInfo.checkpointId != req.checkpointId) { - tqDebug("s-task:%s vgId:%d update the checkpoint from %" PRId64 " to %" PRId64, pTask->id.idStr, vgId, - pTask->chkInfo.checkpointId, req.checkpointId); + tqDebug("s-task:%s vgId:%d update the checkpoint from %" PRId64 " to %" PRId64" transId:%d", pTask->id.idStr, vgId, + pTask->chkInfo.checkpointId, req.checkpointId, req.transId); pTask->chkInfo.checkpointId = req.checkpointId; tqSetRestoreVersionInfo(pTask); } else { - tqDebug("s-task:%s vgId:%d consensus-checkpointId:%" PRId64 " equals to current checkpointId, not update", - pTask->id.idStr, vgId, req.checkpointId); + tqDebug("s-task:%s vgId:%d consensus-checkpointId:%" PRId64 " equals to current id, transId:%d not update", + pTask->id.idStr, vgId, req.checkpointId, req.transId); } + pTask->chkInfo.consensusTransId = req.transId; taosThreadMutexUnlock(&pTask->lock); if (pMeta->role == NODE_ROLE_LEADER) { diff --git a/source/libs/stream/src/streamMsg.c b/source/libs/stream/src/streamMsg.c index 1bc91d6984..40582b5144 100644 --- a/source/libs/stream/src/streamMsg.c +++ b/source/libs/stream/src/streamMsg.c @@ -638,6 +638,7 @@ int32_t tEncodeRestoreCheckpointInfo (SEncoder* pEncoder, const SRestoreCheckpoi if (tEncodeI64(pEncoder, pReq->startTs) < 0) return -1; if (tEncodeI64(pEncoder, pReq->streamId) < 0) return -1; if (tEncodeI64(pEncoder, pReq->checkpointId) < 0) return -1; + if (tEncodeI32(pEncoder, pReq->transId) < 0) return -1; if (tEncodeI32(pEncoder, pReq->taskId) < 0) return -1; if (tEncodeI32(pEncoder, pReq->nodeId) < 0) return -1; tEndEncode(pEncoder); @@ -649,6 +650,7 @@ int32_t tDecodeRestoreCheckpointInfo(SDecoder* pDecoder, SRestoreCheckpointInfo* if (tDecodeI64(pDecoder, &pReq->startTs) < 0) return -1; if (tDecodeI64(pDecoder, &pReq->streamId) < 0) return -1; if (tDecodeI64(pDecoder, &pReq->checkpointId) < 0) return -1; + if (tDecodeI32(pDecoder, &pReq->transId) < 0) return -1; if (tDecodeI32(pDecoder, &pReq->taskId) < 0) return -1; if (tDecodeI32(pDecoder, &pReq->nodeId) < 0) return -1; tEndDecode(pDecoder); From 773f70a907e9ee84d4f8d9c8ff072aabfec68f52 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 9 Jul 2024 15:55:56 +0800 Subject: [PATCH 21/22] fix(stream): fix syntax error. --- source/libs/stream/src/streamDispatch.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/libs/stream/src/streamDispatch.c b/source/libs/stream/src/streamDispatch.c index 8bf0ccca53..617adaa016 100644 --- a/source/libs/stream/src/streamDispatch.c +++ b/source/libs/stream/src/streamDispatch.c @@ -819,7 +819,7 @@ static void checkpointReadyMsgSendMonitorFn(void* param, void* tmrId) { if ((pActiveInfo->activeId == 0) && (pActiveInfo->transId == 0) && (num == 0) && (pTask->chkInfo.startTs == 0)) { taosThreadMutexUnlock(&pActiveInfo->lock); int32_t ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1); - stWarn("s-task:0x%x vgId:%d active checkpoint may be cleared, quit from readyMsg send tmr, ref:%d", id, vgId, ref); + stWarn("s-task:%s vgId:%d active checkpoint may be cleared, quit from readyMsg send tmr, ref:%d", id, vgId, ref); streamMetaReleaseTask(pTask->pMeta, pTask); return; From 8db4722be475f5ce7162b9a0a7424acffbf16787 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 9 Jul 2024 16:08:35 +0800 Subject: [PATCH 22/22] fix(stream):fix syntax error. --- source/dnode/mnode/impl/src/mndStream.c | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index a515ed1d7d..415d1ff9f0 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -2829,7 +2829,7 @@ int32_t mndProcessConsensusInTmr(SRpcMsg *pMsg) { if (((now - pe->ts) >= 10 * 1000) || allSame) { mDebug("s-task:0x%x sendTs:%" PRId64 " wait %.2fs and all tasks have same checkpointId", pe->req.taskId, - (now - pe->ts) / 1000.0, pe->ts); + pe->req.startTs, (now - pe->ts) / 1000.0); ASSERT(chkId <= pe->req.checkpointId); mndCreateSetConsensusChkptIdTrans(pMnode, pStream, pe->req.taskId, chkId, pe->req.startTs); @@ -2837,7 +2837,7 @@ int32_t mndProcessConsensusInTmr(SRpcMsg *pMsg) { streamId = pe->req.streamId; } else { mDebug("s-task:0x%x sendTs:%" PRId64 " wait %.2fs already, wait for next round to check", pe->req.taskId, - (now - pe->ts) / 1000.0, pe->ts); + pe->req.startTs, (now - pe->ts) / 1000.0); } }