From 09b6642e459384ccd5ac140c92cf1a89bb013d6e Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sat, 22 Feb 2025 12:33:32 +0800 Subject: [PATCH 01/57] refactor(stream): scan wal is driven by time, instead of insert events. --- source/dnode/vnode/src/inc/vnodeInt.h | 2 +- source/dnode/vnode/src/tq/tq.c | 24 +- source/dnode/vnode/src/tq/tqPush.c | 14 -- source/dnode/vnode/src/tq/tqStreamTask.c | 242 +++++++++++---------- source/dnode/vnode/src/tq/tqUtil.c | 4 + source/dnode/vnode/src/tqCommon/tqCommon.c | 10 +- source/libs/stream/src/streamHb.c | 2 +- source/libs/stream/src/streamMeta.c | 2 +- 8 files changed, 156 insertions(+), 144 deletions(-) diff --git a/source/dnode/vnode/src/inc/vnodeInt.h b/source/dnode/vnode/src/inc/vnodeInt.h index 5bf0a9b199..a76fc17377 100644 --- a/source/dnode/vnode/src/inc/vnodeInt.h +++ b/source/dnode/vnode/src/inc/vnodeInt.h @@ -239,7 +239,7 @@ void tqClose(STQ*); int tqPushMsg(STQ*, tmsg_t msgType); int tqRegisterPushHandle(STQ* pTq, void* handle, SRpcMsg* pMsg); void tqUnregisterPushHandle(STQ* pTq, void* pHandle); -int tqScanWalAsync(STQ* pTq, bool ckPause); +int tqScanWalAsync(STQ* pTq); int32_t tqStopStreamTasksAsync(STQ* pTq); int32_t tqProcessTaskCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg* pRsp); int32_t tqProcessTaskCheckpointReadyMsg(STQ* pTq, SRpcMsg* pMsg); diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 119edb47bc..fc173eb691 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -920,12 +920,12 @@ static void doStartFillhistoryStep2(SStreamTask* pTask, SStreamTask* pStreamTask // now the fill-history task starts to scan data from wal files. code = streamTaskHandleEvent(pTask->status.pSM, TASK_EVENT_SCANHIST_DONE); - if (code == TSDB_CODE_SUCCESS) { - code = tqScanWalAsync(pTq, false); - if (code) { - tqError("vgId:%d failed to start scan wal file, code:%s", vgId, tstrerror(code)); - } - } +// if (code == TSDB_CODE_SUCCESS) { +// code = tqScanWalAsync(pTq, false); +// if (code) { +// tqError("vgId:%d failed to start scan wal file, code:%s", vgId, tstrerror(code)); +// } +// } } } @@ -1122,12 +1122,12 @@ int32_t tqProcessTaskRunReq(STQ* pTq, SRpcMsg* pMsg) { } // let's continue scan data in the wal files - if (req.reqType >= 0 || req.reqType == STREAM_EXEC_T_RESUME_TASK) { - code = tqScanWalAsync(pTq, false); // it's ok to failed - if (code) { - tqError("vgId:%d failed to start scan wal file, code:%s", pTq->pStreamMeta->vgId, tstrerror(code)); - } - } +// if (req.reqType >= 0 || req.reqType == STREAM_EXEC_T_RESUME_TASK) { +// code = tqScanWalAsync(pTq, false); // it's ok to failed +// if (code) { +// tqError("vgId:%d failed to start scan wal file, code:%s", pTq->pStreamMeta->vgId, tstrerror(code)); +// } +// } return code; } diff --git a/source/dnode/vnode/src/tq/tqPush.c b/source/dnode/vnode/src/tq/tqPush.c index 2b2667773a..fc83343c99 100644 --- a/source/dnode/vnode/src/tq/tqPush.c +++ b/source/dnode/vnode/src/tq/tqPush.c @@ -49,20 +49,6 @@ int32_t tqPushMsg(STQ* pTq, tmsg_t msgType) { } } - streamMetaRLock(pTq->pStreamMeta); - int32_t numOfTasks = streamMetaGetNumOfTasks(pTq->pStreamMeta); - streamMetaRUnLock(pTq->pStreamMeta); - -// tqTrace("vgId:%d handle submit, restore:%d, numOfTasks:%d", TD_VID(pTq->pVnode), pTq->pVnode->restored, numOfTasks); - - // push data for stream processing: - // 1. the vnode has already been restored. - // 2. the vnode should be the leader. - // 3. the stream is not suspended yet. - if ((!tsDisableStream) && (numOfTasks > 0)) { - code = tqScanWalAsync(pTq, true); - } - return code; } diff --git a/source/dnode/vnode/src/tq/tqStreamTask.c b/source/dnode/vnode/src/tq/tqStreamTask.c index 9ea84830f1..33a32617ef 100644 --- a/source/dnode/vnode/src/tq/tqStreamTask.c +++ b/source/dnode/vnode/src/tq/tqStreamTask.c @@ -17,22 +17,20 @@ #include "vnd.h" #define MAX_REPEAT_SCAN_THRESHOLD 3 -#define SCAN_WAL_IDLE_DURATION 500 // idle for 500ms to do next wal scan +#define SCAN_WAL_IDLE_DURATION 500 // idle for 500ms to do next wal scan typedef struct SBuildScanWalMsgParam { int64_t metaId; - int32_t numOfTasks; - int8_t restored; +// int8_t restored; SMsgCb msgCb; } SBuildScanWalMsgParam; -static int32_t doScanWalForAllTasks(SStreamMeta* pStreamMeta); +static int32_t doScanWalForAllTasks(SStreamMeta* pStreamMeta, int32_t* pNumOfTasks); static int32_t setWalReaderStartOffset(SStreamTask* pTask, int32_t vgId); static bool handleFillhistoryScanComplete(SStreamTask* pTask, int64_t ver); static bool taskReadyForDataFromWal(SStreamTask* pTask); static int32_t doPutDataIntoInputQ(SStreamTask* pTask, int64_t maxVer, int32_t* numOfItems, bool* pSucc); -static int32_t tqScanWalInFuture(STQ* pTq, int32_t numOfTasks, int32_t idleDuration); -static int32_t doScanWalAsync(STQ* pTq, bool ckPause); +static int32_t tqScanWalInFuture(STQ* pTq); // extract data blocks(submit/delete) from WAL, and add them into the input queue for all the sources tasks. int32_t tqScanWal(STQ* pTq) { @@ -40,44 +38,38 @@ int32_t tqScanWal(STQ* pTq) { int32_t vgId = pMeta->vgId; int64_t st = taosGetTimestampMs(); int32_t numOfTasks = 0; + int64_t el = 0; + int32_t code = 0; - tqDebug("vgId:%d continue to check if data in wal are available, scanCounter:%d", vgId, pMeta->scanInfo.scanCounter); - - // check all tasks - int32_t code = doScanWalForAllTasks(pMeta); - if (code) { - tqError("vgId:%d failed to start all tasks, try next time, code:%s", vgId, tstrerror(code)); + int32_t old = atomic_val_compare_exchange_32(&pMeta->scanInfo.scanCounter, 0, 1); + if (old == 0) { + tqDebug("vgId:%d try to scan wal to extract data", vgId); + } else { + tqDebug("vgId:%d already in wal scan, abort", vgId); return code; } - streamMetaWLock(pMeta); - int32_t times = (--pMeta->scanInfo.scanCounter); - if (times < 0) { - tqError("vgId:%d invalid scan counter:%d, reset to 0", vgId, times); - times = 0; - } - - numOfTasks = taosArrayGetSize(pMeta->pTaskList); - streamMetaWUnLock(pMeta); - - int64_t el = (taosGetTimestampMs() - st); - tqDebug("vgId:%d scan wal for stream tasks completed, elapsed time:%" PRId64 " ms", vgId, el); - - if (times > 0) { - tqDebug("vgId:%d scan wal for stream tasks for %d times in %dms", vgId, times, SCAN_WAL_IDLE_DURATION); - code = tqScanWalInFuture(pTq, numOfTasks, SCAN_WAL_IDLE_DURATION); - if (code) { - tqError("vgId:%d sched scan wal in %dms failed, ignore this failure", vgId, SCAN_WAL_IDLE_DURATION); - } + // check all tasks + code = doScanWalForAllTasks(pMeta, &numOfTasks); + + el = (taosGetTimestampMs() - st); + if (code) { + tqError("vgId:%d failed to scan wal for all tasks, try next time, elapsed time:%" PRId64 "ms code:%s", vgId, el, + tstrerror(code)); + } else { + tqDebug("vgId:%d scan wal for stream tasks completed, elapsed time:%" PRId64 "ms, next scan start in %dms", vgId, + el, SCAN_WAL_IDLE_DURATION); } + atomic_store_32(&pMeta->scanInfo.scanCounter, 0); return code; } static void doStartScanWal(void* param, void* tmrId) { - int32_t vgId = 0; - int32_t code = 0; - + int32_t vgId = 0; + int32_t code = 0; + SVnode* pVnode = NULL; + int32_t numOfTasks = 0; SBuildScanWalMsgParam* pParam = (SBuildScanWalMsgParam*)param; SStreamMeta* pMeta = taosAcquireRef(streamMetaRefPool, pParam->metaId); @@ -102,71 +94,93 @@ static void doStartScanWal(void* param, void* tmrId) { vgId = pMeta->vgId; - tqDebug("vgId:%d create msg to start wal scan, numOfTasks:%d, vnd restored:%d", vgId, pParam->numOfTasks, - pParam->restored); -#if 0 - // wait for the vnode is freed, and invalid read may occur. + if (pMeta->role == NODE_ROLE_FOLLOWER) { + tqDebug("vgId:%d not leader, role:%d not scan wal anymore", vgId, pMeta->role); + + code = taosReleaseRef(streamMetaRefPool, pParam->metaId); + if (code == TSDB_CODE_SUCCESS) { + tqDebug("vgId:%d jump out of scan wal timer since not leader", vgId); + } else { + tqError("vgId:%d failed to release ref for streamMeta, rid:%" PRId64 " code:%s", vgId, pParam->metaId, + tstrerror(code)); + } + + taosMemFree(pParam); + return; + } + + if (pMeta->startInfo.startAllTasks) { + tqTrace("vgId:%d in restart procedure, not ready to scan wal", vgId); + goto _end; + } + + streamMetaRLock(pMeta); + numOfTasks = taosArrayGetSize(pMeta->pTaskList); + streamMetaRUnLock(pMeta); + + if (numOfTasks == 0) { + goto _end; + } + + tqTrace("vgId:%d create msg to start wal scan, numOfTasks:%d", vgId, numOfTasks); + + // #if 0 + // wait for the vnode is freed, and invalid read may occur. taosMsleep(10000); -#endif + // #endif code = streamTaskSchedTask(&pParam->msgCb, vgId, 0, 0, STREAM_EXEC_T_EXTRACT_WAL_DATA); if (code) { tqError("vgId:%d failed sched task to scan wal, code:%s", vgId, tstrerror(code)); } +_end: + streamTmrStart(doStartScanWal, SCAN_WAL_IDLE_DURATION, pParam, &pMeta->scanInfo.scanTimer, &pMeta->scanInfo.scanTimer, + vgId, "scan-wal"); + code = taosReleaseRef(streamMetaRefPool, pParam->metaId); if (code) { tqError("vgId:%d failed to release ref for streamMeta, rid:%" PRId64 " code:%s", vgId, pParam->metaId, tstrerror(code)); } - - taosMemoryFree(pParam); } -int32_t tqScanWalInFuture(STQ* pTq, int32_t numOfTasks, int32_t idleDuration) { +int32_t tqScanWalAsync(STQ* pTq) { SStreamMeta* pMeta = pTq->pStreamMeta; int32_t code = 0; int32_t vgId = TD_VID(pTq->pVnode); tmr_h pTimer = NULL; SBuildScanWalMsgParam* pParam = NULL; + // 1. the vnode should be the leader. + // 2. the stream isn't disabled + if ((pMeta->role == NODE_ROLE_FOLLOWER) || tsDisableStream) { + tqInfo("vgId:%d follower node or stream disabled, not scan wal", vgId); + return TSDB_CODE_SUCCESS; + } + pParam = taosMemoryMalloc(sizeof(SBuildScanWalMsgParam)); if (pParam == NULL) { + tqError("vgId:%d failed to start scan wal, stream not executes, code:%s", vgId, tstrerror(code)); return terrno; } pParam->metaId = pMeta->rid; - pParam->numOfTasks = numOfTasks; - pParam->restored = pTq->pVnode->restored; pParam->msgCb = pTq->pVnode->msgCb; code = streamTimerGetInstance(&pTimer); if (code) { - tqError("vgId:%d failed to get tmr ctrl during sched scan wal", vgId); + tqFatal("vgId:%d failed to get tmr ctrl during sched scan wal", vgId); taosMemoryFree(pParam); } else { - streamTmrStart(doStartScanWal, idleDuration, pParam, pTimer, &pMeta->scanInfo.scanTimer, vgId, "scan-wal-fut"); + // todo: start in 1sec for the first time + streamTmrStart(doStartScanWal, 1000, pParam, pTimer, &pMeta->scanInfo.scanTimer, vgId, + "scan-wal"); } return code; } -int32_t tqScanWalAsync(STQ* pTq, bool ckPause) { - SStreamMeta* pMeta = pTq->pStreamMeta; - bool alreadyRestored = pTq->pVnode->restored; - int32_t code = 0; - - // do not launch the stream tasks, if it is a follower or not restored vnode. - if (!(vnodeIsRoleLeader(pTq->pVnode) && alreadyRestored)) { - return TSDB_CODE_SUCCESS; - } - - streamMetaWLock(pMeta); - code = doScanWalAsync(pTq, ckPause); - streamMetaWUnLock(pMeta); - return code; -} - int32_t tqStopStreamTasksAsync(STQ* pTq) { SStreamMeta* pMeta = pTq->pStreamMeta; int32_t vgId = pMeta->vgId; @@ -347,13 +361,10 @@ int32_t doPutDataIntoInputQ(SStreamTask* pTask, int64_t maxVer, int32_t* numOfIt return code; } -int32_t doScanWalForAllTasks(SStreamMeta* pStreamMeta) { +int32_t doScanWalForAllTasks(SStreamMeta* pStreamMeta, int32_t* pNumOfTasks) { int32_t vgId = pStreamMeta->vgId; SArray* pTaskList = NULL; - int32_t numOfTasks = taosArrayGetSize(pStreamMeta->pTaskList); - if (numOfTasks == 0) { - return TSDB_CODE_SUCCESS; - } + int32_t numOfTasks = 0; // clone the task list, to avoid the task update during scan wal files streamMetaWLock(pStreamMeta); @@ -364,10 +375,13 @@ int32_t doScanWalForAllTasks(SStreamMeta* pStreamMeta) { return terrno; } - tqDebug("vgId:%d start to check wal to extract new submit block for %d tasks", vgId, numOfTasks); - // update the new task number numOfTasks = taosArrayGetSize(pTaskList); + if (pNumOfTasks != NULL) { + *pNumOfTasks = numOfTasks; + } + + tqDebug("vgId:%d start to check wal to extract new submit block for %d tasks", vgId, numOfTasks); for (int32_t i = 0; i < numOfTasks; ++i) { STaskId* pTaskId = taosArrayGet(pTaskList, i); @@ -426,51 +440,59 @@ int32_t doScanWalForAllTasks(SStreamMeta* pStreamMeta) { return TSDB_CODE_SUCCESS; } -int32_t doScanWalAsync(STQ* pTq, bool ckPause) { - SStreamMeta* pMeta = pTq->pStreamMeta; - bool alreadyRestored = pTq->pVnode->restored; - int32_t vgId = pMeta->vgId; - int32_t numOfTasks = taosArrayGetSize(pMeta->pTaskList); - - if (numOfTasks == 0) { - tqDebug("vgId:%d no stream tasks existed to run", vgId); - return 0; - } - - if (pMeta->startInfo.startAllTasks) { - tqTrace("vgId:%d in restart procedure, not scan wal", vgId); - return 0; - } - - pMeta->scanInfo.scanCounter += 1; - if (pMeta->scanInfo.scanCounter > MAX_REPEAT_SCAN_THRESHOLD) { - pMeta->scanInfo.scanCounter = MAX_REPEAT_SCAN_THRESHOLD; - } - - if (pMeta->scanInfo.scanCounter > 1) { - tqDebug("vgId:%d wal read task has been launched, remain scan times:%d", vgId, pMeta->scanInfo.scanCounter); - return 0; - } - - int32_t numOfPauseTasks = pMeta->numOfPausedTasks; - if (ckPause && numOfTasks == numOfPauseTasks) { - tqDebug("vgId:%d ignore all submit, all streams had been paused, reset the walScanCounter", vgId); - - // reset the counter value, since we do not launch the scan wal operation. - pMeta->scanInfo.scanCounter = 0; - return 0; - } - - tqDebug("vgId:%d create msg to start wal scan to launch stream tasks, numOfTasks:%d, vnd restored:%d", vgId, - numOfTasks, alreadyRestored); - - return streamTaskSchedTask(&pTq->pVnode->msgCb, vgId, 0, 0, STREAM_EXEC_T_EXTRACT_WAL_DATA); -} +// int32_t doScanWalAsync(STQ* pTq) { +// SStreamMeta* pMeta = pTq->pStreamMeta; +// bool alreadyRestored = pTq->pVnode->restored; +// int32_t vgId = pMeta->vgId; +// int32_t code = 0; +// int32_t numOfTasks = taosArrayGetSize(pMeta->pTaskList); +// +// // if (numOfTasks == 0) { +//// tqDebug("vgId:%d no stream tasks existed to run", vgId); +//// return 0; +//// } +// +//// if (pMeta->startInfo.startAllTasks) { +//// tqTrace("vgId:%d in restart procedure, not scan wal", vgId); +//// return 0; +//// } +// +//// pMeta->scanInfo.scanCounter += 1; +//// if (pMeta->scanInfo.scanCounter > MAX_REPEAT_SCAN_THRESHOLD) { +//// pMeta->scanInfo.scanCounter = MAX_REPEAT_SCAN_THRESHOLD; +//// } +// +//// if (pMeta->scanInfo.scanCounter > 1) { +//// tqDebug("vgId:%d wal read task has been launched, remain scan times:%d", vgId, pMeta->scanInfo.scanCounter); +//// return 0; +//// } +// +//// int32_t numOfPauseTasks = pMeta->numOfPausedTasks; +//// if (ckPause && numOfTasks == numOfPauseTasks) { +//// tqDebug("vgId:%d ignore all submit, all streams had been paused, reset the walScanCounter", vgId); +//// +//// reset the counter value, since we do not launch the scan wal operation. +//// pMeta->scanInfo.scanCounter = 0; +//// return 0; +//// } +// +//// tqDebug("vgId:%d create msg to start wal scan to launch stream tasks, numOfTasks:%d, vnd restored:%d", vgId, +//// numOfTasks, alreadyRestored); +//// +//// code = streamTaskSchedTask(&pTq->pVnode->msgCb, vgId, 0, 0, STREAM_EXEC_T_EXTRACT_WAL_DATA); +//// if (code) { +//// tqError("vgId:%d failed create msg to scan data in wal, retry in %dms", vgId, SCAN_WAL_IDLE_DURATION); +//// } +// +// code = tqScanWalInFuture(pTq, numOfTasks, SCAN_WAL_IDLE_DURATION); +// if (code) { +// tqError("vgId:%d sched scan wal in %dms failed, ignore this failure", vgId, SCAN_WAL_IDLE_DURATION); +// } +//} void streamMetaFreeTQDuringScanWalError(STQ* pTq) { SBuildScanWalMsgParam* p = taosMemoryCalloc(1, sizeof(SBuildScanWalMsgParam)); p->metaId = pTq->pStreamMeta->rid; - p->numOfTasks = 0; doStartScanWal(p, 0); } \ No newline at end of file diff --git a/source/dnode/vnode/src/tq/tqUtil.c b/source/dnode/vnode/src/tq/tqUtil.c index 197a45cdb9..6f001981fb 100644 --- a/source/dnode/vnode/src/tq/tqUtil.c +++ b/source/dnode/vnode/src/tq/tqUtil.c @@ -47,6 +47,10 @@ END: void tqUpdateNodeStage(STQ* pTq, bool isLeader) { SSyncState state = syncGetState(pTq->pVnode->sync); streamMetaUpdateStageRole(pTq->pStreamMeta, state.term, isLeader); + + if (isLeader) { + tqScanWalAsync(pTq); + } } static int32_t tqInitTaosxRsp(SMqDataRsp* pRsp, STqOffsetVal pOffset) { diff --git a/source/dnode/vnode/src/tqCommon/tqCommon.c b/source/dnode/vnode/src/tqCommon/tqCommon.c index 77632d328e..7fc2881881 100644 --- a/source/dnode/vnode/src/tqCommon/tqCommon.c +++ b/source/dnode/vnode/src/tqCommon/tqCommon.c @@ -938,10 +938,10 @@ int32_t tqStartTaskCompleteCallback(SStreamMeta* pMeta) { streamMetaWUnLock(pMeta); - if (scanWal && (vgId != SNODE_HANDLE)) { - tqDebug("vgId:%d start scan wal for executing tasks", vgId); - code = tqScanWalAsync(pMeta->ahandle, true); - } +// if (scanWal && (vgId != SNODE_HANDLE)) { +// tqDebug("vgId:%d start scan wal for executing tasks", vgId); +// code = tqScanWalAsync(pMeta->ahandle, true); +// } return code; } @@ -1170,7 +1170,7 @@ static int32_t tqProcessTaskResumeImpl(void* handle, SStreamTask* pTask, int64_t pTask->hTaskInfo.operatorOpen = false; code = streamStartScanHistoryAsync(pTask, igUntreated); } else if (level == TASK_LEVEL__SOURCE && (streamQueueGetNumOfItems(pTask->inputq.queue) == 0)) { - code = tqScanWalAsync((STQ*)handle, false); +// code = tqScanWalAsync((STQ*)handle, false); } else { code = streamTrySchedExec(pTask); } diff --git a/source/libs/stream/src/streamHb.c b/source/libs/stream/src/streamHb.c index 7c157bb05e..36a6488709 100644 --- a/source/libs/stream/src/streamHb.c +++ b/source/libs/stream/src/streamHb.c @@ -341,7 +341,7 @@ void streamMetaHbToMnode(void* param, void* tmrId) { if (code == TSDB_CODE_SUCCESS) { stInfo("vgId:%d role:%d not leader not send hb to mnode", vgId, role); } else { - stError("vgId:%d role:%d not leader not send hb to mnodefailed to release the meta rid:%" PRId64, vgId, role, rid); + stError("vgId:%d role:%d not leader not send hb to mnode, failed to release meta rid:%" PRId64, vgId, role, rid); } // taosMemoryFree(param); return; diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index ddd0201460..733bae1097 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -1320,7 +1320,7 @@ void streamMetaUpdateStageRole(SStreamMeta* pMeta, int64_t stage, bool isLeader) // mark the sign to send msg before close all tasks // 1. for leader vnode, always send msg before closing - // 2. for follower vnode, if it's is changed from leader, also sending msg before closing. + // 2. for follower vnode, if it's changed from leader, also sending msg before closing. if (pMeta->role == NODE_ROLE_LEADER) { pMeta->sendMsgBeforeClosing = true; } From b5fdb441937eb46967eb6ebadf218960ba7ea116 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sat, 22 Feb 2025 12:35:37 +0800 Subject: [PATCH 02/57] refactor(stream): remove comments. --- source/dnode/vnode/src/tq/tqStreamTask.c | 50 ------------------------ 1 file changed, 50 deletions(-) diff --git a/source/dnode/vnode/src/tq/tqStreamTask.c b/source/dnode/vnode/src/tq/tqStreamTask.c index 33a32617ef..72749d31c9 100644 --- a/source/dnode/vnode/src/tq/tqStreamTask.c +++ b/source/dnode/vnode/src/tq/tqStreamTask.c @@ -440,56 +440,6 @@ int32_t doScanWalForAllTasks(SStreamMeta* pStreamMeta, int32_t* pNumOfTasks) { return TSDB_CODE_SUCCESS; } -// int32_t doScanWalAsync(STQ* pTq) { -// SStreamMeta* pMeta = pTq->pStreamMeta; -// bool alreadyRestored = pTq->pVnode->restored; -// int32_t vgId = pMeta->vgId; -// int32_t code = 0; -// int32_t numOfTasks = taosArrayGetSize(pMeta->pTaskList); -// -// // if (numOfTasks == 0) { -//// tqDebug("vgId:%d no stream tasks existed to run", vgId); -//// return 0; -//// } -// -//// if (pMeta->startInfo.startAllTasks) { -//// tqTrace("vgId:%d in restart procedure, not scan wal", vgId); -//// return 0; -//// } -// -//// pMeta->scanInfo.scanCounter += 1; -//// if (pMeta->scanInfo.scanCounter > MAX_REPEAT_SCAN_THRESHOLD) { -//// pMeta->scanInfo.scanCounter = MAX_REPEAT_SCAN_THRESHOLD; -//// } -// -//// if (pMeta->scanInfo.scanCounter > 1) { -//// tqDebug("vgId:%d wal read task has been launched, remain scan times:%d", vgId, pMeta->scanInfo.scanCounter); -//// return 0; -//// } -// -//// int32_t numOfPauseTasks = pMeta->numOfPausedTasks; -//// if (ckPause && numOfTasks == numOfPauseTasks) { -//// tqDebug("vgId:%d ignore all submit, all streams had been paused, reset the walScanCounter", vgId); -//// -//// reset the counter value, since we do not launch the scan wal operation. -//// pMeta->scanInfo.scanCounter = 0; -//// return 0; -//// } -// -//// tqDebug("vgId:%d create msg to start wal scan to launch stream tasks, numOfTasks:%d, vnd restored:%d", vgId, -//// numOfTasks, alreadyRestored); -//// -//// code = streamTaskSchedTask(&pTq->pVnode->msgCb, vgId, 0, 0, STREAM_EXEC_T_EXTRACT_WAL_DATA); -//// if (code) { -//// tqError("vgId:%d failed create msg to scan data in wal, retry in %dms", vgId, SCAN_WAL_IDLE_DURATION); -//// } -// -// code = tqScanWalInFuture(pTq, numOfTasks, SCAN_WAL_IDLE_DURATION); -// if (code) { -// tqError("vgId:%d sched scan wal in %dms failed, ignore this failure", vgId, SCAN_WAL_IDLE_DURATION); -// } -//} - void streamMetaFreeTQDuringScanWalError(STQ* pTq) { SBuildScanWalMsgParam* p = taosMemoryCalloc(1, sizeof(SBuildScanWalMsgParam)); p->metaId = pTq->pStreamMeta->rid; From 9bd8b532ad1b454df2148b521948ed9e146163d0 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sat, 22 Feb 2025 12:33:32 +0800 Subject: [PATCH 03/57] fix(stream): update function return value. --- source/dnode/vnode/src/inc/vnodeInt.h | 2 +- source/dnode/vnode/src/tq/tqStreamTask.c | 4 +--- source/dnode/vnode/src/tq/tqUtil.c | 2 +- 3 files changed, 3 insertions(+), 5 deletions(-) diff --git a/source/dnode/vnode/src/inc/vnodeInt.h b/source/dnode/vnode/src/inc/vnodeInt.h index a76fc17377..6b3cf47d13 100644 --- a/source/dnode/vnode/src/inc/vnodeInt.h +++ b/source/dnode/vnode/src/inc/vnodeInt.h @@ -239,7 +239,7 @@ void tqClose(STQ*); int tqPushMsg(STQ*, tmsg_t msgType); int tqRegisterPushHandle(STQ* pTq, void* handle, SRpcMsg* pMsg); void tqUnregisterPushHandle(STQ* pTq, void* pHandle); -int tqScanWalAsync(STQ* pTq); +void tqScanWalAsync(STQ* pTq); int32_t tqStopStreamTasksAsync(STQ* pTq); int32_t tqProcessTaskCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg* pRsp); int32_t tqProcessTaskCheckpointReadyMsg(STQ* pTq, SRpcMsg* pMsg); diff --git a/source/dnode/vnode/src/tq/tqStreamTask.c b/source/dnode/vnode/src/tq/tqStreamTask.c index 72749d31c9..aa8151d7c2 100644 --- a/source/dnode/vnode/src/tq/tqStreamTask.c +++ b/source/dnode/vnode/src/tq/tqStreamTask.c @@ -145,7 +145,7 @@ _end: } } -int32_t tqScanWalAsync(STQ* pTq) { +void tqScanWalAsync(STQ* pTq) { SStreamMeta* pMeta = pTq->pStreamMeta; int32_t code = 0; int32_t vgId = TD_VID(pTq->pVnode); @@ -177,8 +177,6 @@ int32_t tqScanWalAsync(STQ* pTq) { streamTmrStart(doStartScanWal, 1000, pParam, pTimer, &pMeta->scanInfo.scanTimer, vgId, "scan-wal"); } - - return code; } int32_t tqStopStreamTasksAsync(STQ* pTq) { diff --git a/source/dnode/vnode/src/tq/tqUtil.c b/source/dnode/vnode/src/tq/tqUtil.c index 6f001981fb..217859394a 100644 --- a/source/dnode/vnode/src/tq/tqUtil.c +++ b/source/dnode/vnode/src/tq/tqUtil.c @@ -49,7 +49,7 @@ void tqUpdateNodeStage(STQ* pTq, bool isLeader) { streamMetaUpdateStageRole(pTq->pStreamMeta, state.term, isLeader); if (isLeader) { - tqScanWalAsync(pTq); + int32_t code = tqScanWalAsync(pTq); } } From 6c92474afd67f863dc86d06aa8253fb7693645db Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sat, 22 Feb 2025 14:16:46 +0800 Subject: [PATCH 04/57] fix(stream): fix syntax error. --- source/dnode/vnode/src/tq/tqStreamTask.c | 4 ++-- source/dnode/vnode/src/tq/tqUtil.c | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/source/dnode/vnode/src/tq/tqStreamTask.c b/source/dnode/vnode/src/tq/tqStreamTask.c index aa8151d7c2..d4b4e3e90e 100644 --- a/source/dnode/vnode/src/tq/tqStreamTask.c +++ b/source/dnode/vnode/src/tq/tqStreamTask.c @@ -156,13 +156,13 @@ void tqScanWalAsync(STQ* pTq) { // 2. the stream isn't disabled if ((pMeta->role == NODE_ROLE_FOLLOWER) || tsDisableStream) { tqInfo("vgId:%d follower node or stream disabled, not scan wal", vgId); - return TSDB_CODE_SUCCESS; + return; } pParam = taosMemoryMalloc(sizeof(SBuildScanWalMsgParam)); if (pParam == NULL) { tqError("vgId:%d failed to start scan wal, stream not executes, code:%s", vgId, tstrerror(code)); - return terrno; + return; } pParam->metaId = pMeta->rid; diff --git a/source/dnode/vnode/src/tq/tqUtil.c b/source/dnode/vnode/src/tq/tqUtil.c index 217859394a..6f001981fb 100644 --- a/source/dnode/vnode/src/tq/tqUtil.c +++ b/source/dnode/vnode/src/tq/tqUtil.c @@ -49,7 +49,7 @@ void tqUpdateNodeStage(STQ* pTq, bool isLeader) { streamMetaUpdateStageRole(pTq->pStreamMeta, state.term, isLeader); if (isLeader) { - int32_t code = tqScanWalAsync(pTq); + tqScanWalAsync(pTq); } } From a8d50f28a60d7f8a146487422ad686320ecb4f8b Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sat, 22 Feb 2025 14:19:44 +0800 Subject: [PATCH 05/57] fix(stream): disable error injection. --- source/dnode/vnode/src/tq/tqStreamTask.c | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/source/dnode/vnode/src/tq/tqStreamTask.c b/source/dnode/vnode/src/tq/tqStreamTask.c index d4b4e3e90e..06b496f136 100644 --- a/source/dnode/vnode/src/tq/tqStreamTask.c +++ b/source/dnode/vnode/src/tq/tqStreamTask.c @@ -124,10 +124,10 @@ static void doStartScanWal(void* param, void* tmrId) { tqTrace("vgId:%d create msg to start wal scan, numOfTasks:%d", vgId, numOfTasks); - // #if 0 + #if 0 // wait for the vnode is freed, and invalid read may occur. taosMsleep(10000); - // #endif + #endif code = streamTaskSchedTask(&pParam->msgCb, vgId, 0, 0, STREAM_EXEC_T_EXTRACT_WAL_DATA); if (code) { From 034c907ee148e8eb95523f6e9603b64073c92e74 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sun, 23 Feb 2025 01:38:05 +0800 Subject: [PATCH 06/57] refactor: do some internal refactor. --- include/libs/stream/tstream.h | 4 +- source/dnode/vnode/src/tq/tq.c | 19 ++----- source/dnode/vnode/src/tq/tqStreamTask.c | 66 ++++++++++++++++++------ source/libs/stream/src/streamHb.c | 4 +- source/libs/stream/src/streamMeta.c | 10 +++- 5 files changed, 68 insertions(+), 35 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index a4d89dcdcc..4707ee959a 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -483,8 +483,10 @@ typedef struct STaskUpdateInfo { } STaskUpdateInfo; typedef struct SScanWalInfo { - int32_t scanCounter; + int32_t scanSentinel; tmr_h scanTimer; + int64_t lastScanTs; + int32_t tickCounter; } SScanWalInfo; typedef struct SFatalErrInfo { diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index fc173eb691..63727e5c45 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1113,23 +1113,14 @@ int32_t tqProcessTaskRunReq(STQ* pTq, SRpcMsg* pMsg) { // extracted submit data from wal files for all tasks if (req.reqType == STREAM_EXEC_T_EXTRACT_WAL_DATA) { return tqScanWal(pTq); - } + } else { + code = tqStreamTaskProcessRunReq(pTq->pStreamMeta, pMsg, vnodeIsRoleLeader(pTq->pVnode)); + if (code) { + tqError("vgId:%d failed to create task run req, code:%s", TD_VID(pTq->pVnode), tstrerror(code)); + } - code = tqStreamTaskProcessRunReq(pTq->pStreamMeta, pMsg, vnodeIsRoleLeader(pTq->pVnode)); - if (code) { - tqError("vgId:%d failed to create task run req, code:%s", TD_VID(pTq->pVnode), tstrerror(code)); return code; } - - // let's continue scan data in the wal files -// if (req.reqType >= 0 || req.reqType == STREAM_EXEC_T_RESUME_TASK) { -// code = tqScanWalAsync(pTq, false); // it's ok to failed -// if (code) { -// tqError("vgId:%d failed to start scan wal file, code:%s", pTq->pStreamMeta->vgId, tstrerror(code)); -// } -// } - - return code; } int32_t tqProcessTaskDispatchReq(STQ* pTq, SRpcMsg* pMsg) { diff --git a/source/dnode/vnode/src/tq/tqStreamTask.c b/source/dnode/vnode/src/tq/tqStreamTask.c index 06b496f136..08c0be49bc 100644 --- a/source/dnode/vnode/src/tq/tqStreamTask.c +++ b/source/dnode/vnode/src/tq/tqStreamTask.c @@ -16,12 +16,11 @@ #include "tq.h" #include "vnd.h" -#define MAX_REPEAT_SCAN_THRESHOLD 3 -#define SCAN_WAL_IDLE_DURATION 500 // idle for 500ms to do next wal scan +#define SCAN_WAL_IDLE_DURATION 250 // idle for 500ms to do next wal scan +#define SCAN_WAL_WAIT_COUNT 2 typedef struct SBuildScanWalMsgParam { int64_t metaId; -// int8_t restored; SMsgCb msgCb; } SBuildScanWalMsgParam; @@ -30,7 +29,6 @@ static int32_t setWalReaderStartOffset(SStreamTask* pTask, int32_t vgId); static bool handleFillhistoryScanComplete(SStreamTask* pTask, int64_t ver); static bool taskReadyForDataFromWal(SStreamTask* pTask); static int32_t doPutDataIntoInputQ(SStreamTask* pTask, int64_t maxVer, int32_t* numOfItems, bool* pSucc); -static int32_t tqScanWalInFuture(STQ* pTq); // extract data blocks(submit/delete) from WAL, and add them into the input queue for all the sources tasks. int32_t tqScanWal(STQ* pTq) { @@ -41,7 +39,7 @@ int32_t tqScanWal(STQ* pTq) { int64_t el = 0; int32_t code = 0; - int32_t old = atomic_val_compare_exchange_32(&pMeta->scanInfo.scanCounter, 0, 1); + int32_t old = atomic_val_compare_exchange_32(&pMeta->scanInfo.scanSentinel, 0, 1); if (old == 0) { tqDebug("vgId:%d try to scan wal to extract data", vgId); } else { @@ -49,27 +47,44 @@ int32_t tqScanWal(STQ* pTq) { return code; } + // the scan wal interval less than 200, not scan, actually. + if ((pMeta->scanInfo.lastScanTs > st) && (pMeta->scanInfo.lastScanTs - st < 200)) { + tqDebug("vgId:%d scan wal less than 200ms, do nothing", vgId); + atomic_store_32(&pMeta->scanInfo.scanSentinel, 0); + return code; + } + // check all tasks code = doScanWalForAllTasks(pMeta, &numOfTasks); - el = (taosGetTimestampMs() - st); + pMeta->scanInfo.lastScanTs = taosGetTimestampMs(); + el = (pMeta->scanInfo.lastScanTs - st); + if (code) { tqError("vgId:%d failed to scan wal for all tasks, try next time, elapsed time:%" PRId64 "ms code:%s", vgId, el, tstrerror(code)); } else { - tqDebug("vgId:%d scan wal for stream tasks completed, elapsed time:%" PRId64 "ms, next scan start in %dms", vgId, - el, SCAN_WAL_IDLE_DURATION); + tqDebug("vgId:%d scan wal for stream tasks completed, elapsed time:%" PRId64 "ms", vgId, el); } - atomic_store_32(&pMeta->scanInfo.scanCounter, 0); + atomic_store_32(&pMeta->scanInfo.scanSentinel, 0); return code; } +static bool waitEnoughDuration(SStreamMeta* pMeta) { + if ((++pMeta->scanInfo.tickCounter) >= SCAN_WAL_WAIT_COUNT) { + pMeta->scanInfo.tickCounter = 0; + return true; + } + + return false; +} + static void doStartScanWal(void* param, void* tmrId) { int32_t vgId = 0; int32_t code = 0; - SVnode* pVnode = NULL; int32_t numOfTasks = 0; + tmr_h pTimer = NULL; SBuildScanWalMsgParam* pParam = (SBuildScanWalMsgParam*)param; SStreamMeta* pMeta = taosAcquireRef(streamMetaRefPool, pParam->metaId); @@ -79,10 +94,12 @@ static void doStartScanWal(void* param, void* tmrId) { return; } + vgId = pMeta->vgId; + if (pMeta->closeFlag) { code = taosReleaseRef(streamMetaRefPool, pParam->metaId); if (code == TSDB_CODE_SUCCESS) { - tqDebug("vgId:%d jump out of scan wal timer since closed", vgId); + tqInfo("vgId:%d jump out of scan wal timer since closed", vgId); } else { tqError("vgId:%d failed to release ref for streamMeta, rid:%" PRId64 " code:%s", vgId, pParam->metaId, tstrerror(code)); @@ -92,9 +109,7 @@ static void doStartScanWal(void* param, void* tmrId) { return; } - vgId = pMeta->vgId; - - if (pMeta->role == NODE_ROLE_FOLLOWER) { + if (pMeta->role != NODE_ROLE_LEADER) { tqDebug("vgId:%d not leader, role:%d not scan wal anymore", vgId, pMeta->role); code = taosReleaseRef(streamMetaRefPool, pParam->metaId); @@ -114,6 +129,24 @@ static void doStartScanWal(void* param, void* tmrId) { goto _end; } + code = streamTimerGetInstance(&pTimer); + if (code) { + tqFatal("vgId:%d failed to get tmr ctrl during sched scan wal, not scan wal, code:%s", vgId, tstrerror(code)); + taosMemoryFree(pParam); + return; + } + + if (!waitEnoughDuration(pMeta)) { + streamTmrStart(doStartScanWal, SCAN_WAL_IDLE_DURATION, pParam, pTimer, &pMeta->scanInfo.scanTimer, vgId, + "scan-wal"); + code = taosReleaseRef(streamMetaRefPool, pParam->metaId); + if (code) { + tqError("vgId:%d failed to release ref for streamMeta, rid:%" PRId64 " code:%s", vgId, pParam->metaId, + tstrerror(code)); + } + return; + } + streamMetaRLock(pMeta); numOfTasks = taosArrayGetSize(pMeta->pTaskList); streamMetaRUnLock(pMeta); @@ -135,8 +168,9 @@ static void doStartScanWal(void* param, void* tmrId) { } _end: - streamTmrStart(doStartScanWal, SCAN_WAL_IDLE_DURATION, pParam, &pMeta->scanInfo.scanTimer, &pMeta->scanInfo.scanTimer, - vgId, "scan-wal"); + + streamTmrStart(doStartScanWal, SCAN_WAL_IDLE_DURATION, pParam, pTimer, &pMeta->scanInfo.scanTimer, vgId, "scan-wal"); + tqDebug("vgId:%d scan-wal will start in %dms", vgId, SCAN_WAL_IDLE_DURATION*SCAN_WAL_WAIT_COUNT); code = taosReleaseRef(streamMetaRefPool, pParam->metaId); if (code) { diff --git a/source/libs/stream/src/streamHb.c b/source/libs/stream/src/streamHb.c index 36a6488709..90ebd47ac6 100644 --- a/source/libs/stream/src/streamHb.c +++ b/source/libs/stream/src/streamHb.c @@ -327,7 +327,7 @@ void streamMetaHbToMnode(void* param, void* tmrId) { pMeta->pHbInfo->hbStart = 0; code = taosReleaseRef(streamMetaRefPool, rid); if (code == TSDB_CODE_SUCCESS) { - stDebug("vgId:%d jump out of meta timer", vgId); + stInfo("vgId:%d jump out of meta timer since closed", vgId); } else { stError("vgId:%d jump out of meta timer, failed to release the meta rid:%" PRId64, vgId, rid); } @@ -413,7 +413,7 @@ void destroyMetaHbInfo(SMetaHbInfo* pInfo) { void streamMetaWaitForHbTmrQuit(SStreamMeta* pMeta) { // wait for the stream meta hb function stopping if (pMeta->role == NODE_ROLE_LEADER) { - taosMsleep(2 * META_HB_CHECK_INTERVAL); + taosMsleep(3 * META_HB_CHECK_INTERVAL); stDebug("vgId:%d wait for meta to stop timer", pMeta->vgId); } } diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 733bae1097..7bb9f5db7b 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -427,7 +427,7 @@ int32_t streamMetaOpen(const char* path, void* ahandle, FTaskBuild buildTaskFn, pMeta->pTaskList = taosArrayInit(4, sizeof(SStreamTaskId)); TSDB_CHECK_NULL(pMeta->pTaskList, code, lino, _err, terrno); - pMeta->scanInfo.scanCounter = 0; + pMeta->scanInfo.scanSentinel = 0; pMeta->vgId = vgId; pMeta->ahandle = ahandle; pMeta->buildTaskFn = buildTaskFn; @@ -1213,8 +1213,8 @@ void streamMetaNotifyClose(SStreamMeta* pMeta) { vgId, (pMeta->role == NODE_ROLE_LEADER), startTs, sendCount); // wait for the stream meta hb function stopping - streamMetaWaitForHbTmrQuit(pMeta); pMeta->closeFlag = true; + streamMetaWaitForHbTmrQuit(pMeta); stDebug("vgId:%d start to check all tasks for closing", vgId); int64_t st = taosGetTimestampMs(); @@ -1253,6 +1253,12 @@ void streamMetaNotifyClose(SStreamMeta* pMeta) { double el = (taosGetTimestampMs() - st) / 1000.0; stDebug("vgId:%d stop all %d task(s) completed, elapsed time:%.2f Sec.", pMeta->vgId, numOfTasks, el); + + if (pMeta->scanInfo.scanTimer != NULL) { + streamTmrStop(pMeta->scanInfo.scanTimer); + pMeta->scanInfo.scanTimer = NULL; + } + streamMetaRUnLock(pMeta); } From e378b5cb8ae212a0e4744ea64f17a2eb7ba2a817 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sun, 23 Feb 2025 03:42:44 +0800 Subject: [PATCH 07/57] fix(stream): adjust scan timer --- source/dnode/vnode/src/tq/tqStreamTask.c | 3 +-- source/libs/stream/src/streamMeta.c | 4 ++-- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/source/dnode/vnode/src/tq/tqStreamTask.c b/source/dnode/vnode/src/tq/tqStreamTask.c index 08c0be49bc..68e895684e 100644 --- a/source/dnode/vnode/src/tq/tqStreamTask.c +++ b/source/dnode/vnode/src/tq/tqStreamTask.c @@ -207,8 +207,7 @@ void tqScanWalAsync(STQ* pTq) { tqFatal("vgId:%d failed to get tmr ctrl during sched scan wal", vgId); taosMemoryFree(pParam); } else { - // todo: start in 1sec for the first time - streamTmrStart(doStartScanWal, 1000, pParam, pTimer, &pMeta->scanInfo.scanTimer, vgId, + streamTmrStart(doStartScanWal, SCAN_WAL_IDLE_DURATION, pParam, pTimer, &pMeta->scanInfo.scanTimer, vgId, "scan-wal"); } } diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 7bb9f5db7b..eff4783072 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -1336,11 +1336,11 @@ void streamMetaUpdateStageRole(SStreamMeta* pMeta, int64_t stage, bool isLeader) if (isLeader) { stInfo("vgId:%d update meta stage:%" PRId64 ", prev:%" PRId64 " leader:%d, start to send Hb, rid:%" PRId64, - pMeta->vgId, prevStage, stage, isLeader, pMeta->rid); + pMeta->vgId, stage, prevStage, isLeader, pMeta->rid); streamMetaStartHb(pMeta); } else { stInfo("vgId:%d update meta stage:%" PRId64 " prev:%" PRId64 " leader:%d sendMsg beforeClosing:%d", pMeta->vgId, - prevStage, stage, isLeader, pMeta->sendMsgBeforeClosing); + stage, prevStage, isLeader, pMeta->sendMsgBeforeClosing); } } From 00edb8e6124e83419508437011193120d9b29eaa Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sun, 23 Feb 2025 16:25:17 +0800 Subject: [PATCH 08/57] fix(stream): adjust init tmr position. --- source/dnode/vnode/src/tq/tqStreamTask.c | 13 ++++++------- 1 file changed, 6 insertions(+), 7 deletions(-) diff --git a/source/dnode/vnode/src/tq/tqStreamTask.c b/source/dnode/vnode/src/tq/tqStreamTask.c index 68e895684e..640b6bdc63 100644 --- a/source/dnode/vnode/src/tq/tqStreamTask.c +++ b/source/dnode/vnode/src/tq/tqStreamTask.c @@ -95,6 +95,12 @@ static void doStartScanWal(void* param, void* tmrId) { } vgId = pMeta->vgId; + code = streamTimerGetInstance(&pTimer); + if (code) { + tqFatal("vgId:%d failed to get tmr ctrl during sched scan wal, not scan wal, code:%s", vgId, tstrerror(code)); + taosMemoryFree(pParam); + return; + } if (pMeta->closeFlag) { code = taosReleaseRef(streamMetaRefPool, pParam->metaId); @@ -129,13 +135,6 @@ static void doStartScanWal(void* param, void* tmrId) { goto _end; } - code = streamTimerGetInstance(&pTimer); - if (code) { - tqFatal("vgId:%d failed to get tmr ctrl during sched scan wal, not scan wal, code:%s", vgId, tstrerror(code)); - taosMemoryFree(pParam); - return; - } - if (!waitEnoughDuration(pMeta)) { streamTmrStart(doStartScanWal, SCAN_WAL_IDLE_DURATION, pParam, pTimer, &pMeta->scanInfo.scanTimer, vgId, "scan-wal"); From 8bcc8399e63e96402920709dac060c2537ccb4d9 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sun, 23 Feb 2025 15:36:26 +0800 Subject: [PATCH 09/57] fix(stream): reset value. --- source/libs/stream/src/streamMeta.c | 3 +++ 1 file changed, 3 insertions(+) diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index eff4783072..8b0db33536 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -428,6 +428,9 @@ int32_t streamMetaOpen(const char* path, void* ahandle, FTaskBuild buildTaskFn, TSDB_CHECK_NULL(pMeta->pTaskList, code, lino, _err, terrno); pMeta->scanInfo.scanSentinel = 0; + pMeta->scanInfo.lastScanTs = 0; + pMeta->scanInfo.tickCounter = 0; + pMeta->vgId = vgId; pMeta->ahandle = ahandle; pMeta->buildTaskFn = buildTaskFn; From 514a63ab37f558cd9645956255ad243fa60593c9 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 24 Feb 2025 00:40:16 +0800 Subject: [PATCH 10/57] fix(stream): fix the error in check scan interval --- source/dnode/vnode/src/tq/tqStreamTask.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/dnode/vnode/src/tq/tqStreamTask.c b/source/dnode/vnode/src/tq/tqStreamTask.c index 640b6bdc63..b34ea78f64 100644 --- a/source/dnode/vnode/src/tq/tqStreamTask.c +++ b/source/dnode/vnode/src/tq/tqStreamTask.c @@ -48,7 +48,7 @@ int32_t tqScanWal(STQ* pTq) { } // the scan wal interval less than 200, not scan, actually. - if ((pMeta->scanInfo.lastScanTs > st) && (pMeta->scanInfo.lastScanTs - st < 200)) { + if ((pMeta->scanInfo.lastScanTs > 0) && (st - pMeta->scanInfo.lastScanTs < 200)) { tqDebug("vgId:%d scan wal less than 200ms, do nothing", vgId); atomic_store_32(&pMeta->scanInfo.scanSentinel, 0); return code; From 97071f1d291d9a526e1373ade03a2a9e61e63e9b Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 24 Feb 2025 09:42:15 +0800 Subject: [PATCH 11/57] other: add some logs. --- source/dnode/mnode/impl/src/mndStream.c | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index 11fd2d4f65..18919b3ace 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -2613,7 +2613,7 @@ int32_t mndProcessConsensusInTmr(SRpcMsg *pMsg) { int64_t now = taosGetTimestampMs(); bool allReady = true; SArray *pNodeSnapshot = NULL; - int32_t maxAllowedTrans = 50; + int32_t maxAllowedTrans = 20; int32_t numOfTrans = 0; int32_t code = 0; void *pIter = NULL; @@ -2700,6 +2700,7 @@ int32_t mndProcessConsensusInTmr(SRpcMsg *pMsg) { return TSDB_CODE_FAILED; } + // todo: check for redundant consensus-checkpoint trans, if this kinds of trans repeatly failed. code = mndCreateSetConsensusChkptIdTrans(pMnode, pStream, pe->req.taskId, chkId, pe->req.startTs); if (code != TSDB_CODE_SUCCESS && code != TSDB_CODE_ACTION_IN_PROGRESS) { mError("failed to create consensus-checkpoint trans, stream:0x%" PRIx64, pStream->uid); From 3923707826f4558e7f3416215f36ac586985eac1 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 26 Feb 2025 00:13:44 +0800 Subject: [PATCH 12/57] fix(stream): fix memory leak. --- source/libs/stream/src/streamCheckpoint.c | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/source/libs/stream/src/streamCheckpoint.c b/source/libs/stream/src/streamCheckpoint.c index c53e1a19a3..c839495f18 100644 --- a/source/libs/stream/src/streamCheckpoint.c +++ b/source/libs/stream/src/streamCheckpoint.c @@ -1581,6 +1581,11 @@ int32_t streamTaskSendNegotiateChkptIdMsg(SStreamTask* pTask) { streamFreeTaskState(pTask, p); pTask->pBackend = NULL; } + + if (pTask->exec.pExecutor != NULL) { + qDestroyTask(pTask->exec.pExecutor); + pTask->exec.pExecutor = NULL; + } return 0; } From 047ba9ae85928e926d873f4f42f784d966324563 Mon Sep 17 00:00:00 2001 From: chenhaoran Date: Wed, 26 Feb 2025 20:40:41 +0800 Subject: [PATCH 13/57] feat: add rsync to Dockerfile --- packaging/docker/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/packaging/docker/Dockerfile b/packaging/docker/Dockerfile index a67724d5a8..e2ba30bca6 100644 --- a/packaging/docker/Dockerfile +++ b/packaging/docker/Dockerfile @@ -12,7 +12,7 @@ ENV TINI_VERSION v0.19.0 ADD https://github.com/krallin/tini/releases/download/${TINI_VERSION}/tini-${cpuType} /tini ENV DEBIAN_FRONTEND=noninteractive WORKDIR /root/ -RUN tar -zxf ${pkgFile} && cd /root/${dirName}/ && /bin/bash install.sh -e no && cd /root && rm /root/${pkgFile} && rm -rf /root/${dirName} && apt-get update && apt-get install -y locales tzdata netcat curl gdb vim tmux less net-tools valgrind && locale-gen en_US.UTF-8 && apt-get clean && rm -rf /var/lib/apt/lists/ && chmod +x /tini +RUN tar -zxf ${pkgFile} && cd /root/${dirName}/ && /bin/bash install.sh -e no && cd /root && rm /root/${pkgFile} && rm -rf /root/${dirName} && apt-get update && apt-get install -y locales tzdata netcat curl gdb vim tmux less net-tools valgrind rsync && locale-gen en_US.UTF-8 && apt-get clean && rm -rf /var/lib/apt/lists/ && chmod +x /tini ENV LD_LIBRARY_PATH="$LD_LIBRARY_PATH:/usr/lib" \ LC_CTYPE=en_US.UTF-8 \ From c14ba08d7cf0921f9845df87dfd4d6e4a8a6c7d7 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 13 Feb 2025 23:41:06 +0800 Subject: [PATCH 14/57] refactor(stream): drop task in async ways --- include/dnode/vnode/tqCommon.h | 2 +- include/libs/executor/executor.h | 2 +- include/libs/stream/tstream.h | 10 ++++-- source/dnode/snode/src/snode.c | 2 +- source/dnode/vnode/src/sma/smaRollup.c | 2 +- source/dnode/vnode/src/tq/tq.c | 2 +- source/dnode/vnode/src/tqCommon/tqCommon.c | 34 ++++++++++++------ source/libs/executor/src/executor.c | 14 ++++++-- source/libs/stream/src/streamCheckpoint.c | 24 ++++++------- source/libs/stream/src/streamMeta.c | 42 ++++++++++++++++++---- source/libs/stream/src/streamTask.c | 8 +++-- 11 files changed, 99 insertions(+), 43 deletions(-) diff --git a/include/dnode/vnode/tqCommon.h b/include/dnode/vnode/tqCommon.h index 4d5e18520c..69a11638b1 100644 --- a/include/dnode/vnode/tqCommon.h +++ b/include/dnode/vnode/tqCommon.h @@ -34,7 +34,7 @@ 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); -int32_t tqStreamTaskProcessDropReq(SStreamMeta* pMeta, char* msg, int32_t msgLen); +int32_t tqStreamTaskProcessDropReq(SStreamMeta* pMeta, SMsgCb* cb, char* msg, int32_t msgLen); int32_t tqStreamTaskProcessRunReq(SStreamMeta* pMeta, SRpcMsg* pMsg, bool isLeader); int32_t tqStartTaskCompleteCallback(SStreamMeta* pMeta); int32_t tqStreamTasksGetTotalNum(SStreamMeta* pMeta); diff --git a/include/libs/executor/executor.h b/include/libs/executor/executor.h index 883c5f7b99..5b4224d575 100644 --- a/include/libs/executor/executor.h +++ b/include/libs/executor/executor.h @@ -180,7 +180,7 @@ void qCleanExecTaskBlockBuf(qTaskInfo_t tinfo); */ int32_t qAsyncKillTask(qTaskInfo_t tinfo, int32_t rspCode); -int32_t qKillTask(qTaskInfo_t tinfo, int32_t rspCode); +int32_t qKillTask(qTaskInfo_t tinfo, int32_t rspCode, int64_t waiting); bool qTaskIsExecuting(qTaskInfo_t qinfo); diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index a4d89dcdcc..cc12566389 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -58,6 +58,7 @@ extern "C" { #define STREAM_EXEC_T_STOP_ALL_TASKS (-5) #define STREAM_EXEC_T_RESUME_TASK (-6) #define STREAM_EXEC_T_ADD_FAILED_TASK (-7) +#define STREAM_EXEC_T_DROP_ONE_TASK (-8) typedef struct SStreamTask SStreamTask; typedef struct SStreamQueue SStreamQueue; @@ -752,15 +753,20 @@ void streamMetaCleanup(); int32_t streamMetaOpen(const char* path, void* ahandle, FTaskBuild expandFunc, FTaskExpand expandTaskFn, int32_t vgId, int64_t stage, startComplete_fn_t fn, SStreamMeta** pMeta); void streamMetaClose(SStreamMeta* streamMeta); -int32_t streamMetaSaveTask(SStreamMeta* pMeta, SStreamTask* pTask); // save to stream meta store -int32_t streamMetaRemoveTask(SStreamMeta* pMeta, STaskId* pKey); + +int32_t streamMetaSaveTaskInMeta(SStreamMeta* pMeta, SStreamTask* pTask); // save to stream meta store +int32_t streamMetaRemoveTaskInMeta(SStreamMeta* pMeta, STaskId* pKey); + int32_t streamMetaRegisterTask(SStreamMeta* pMeta, int64_t ver, SStreamTask* pTask, bool* pAdded); int32_t streamMetaUnregisterTask(SStreamMeta* pMeta, int64_t streamId, int32_t taskId); int32_t streamMetaGetNumOfTasks(SStreamMeta* pMeta); +int32_t streamMetaDropTask(SStreamMeta* pMeta, int64_t streamId, int32_t taskId); + int32_t streamMetaAcquireTaskNoLock(SStreamMeta* pMeta, int64_t streamId, int32_t taskId, SStreamTask** pTask); int32_t streamMetaAcquireTaskUnsafe(SStreamMeta* pMeta, STaskId* pId, SStreamTask** pTask); int32_t streamMetaAcquireTask(SStreamMeta* pMeta, int64_t streamId, int32_t taskId, SStreamTask** pTask); void streamMetaReleaseTask(SStreamMeta* pMeta, SStreamTask* pTask); + void streamMetaClear(SStreamMeta* pMeta); void streamMetaInitBackend(SStreamMeta* pMeta); int32_t streamMetaCommit(SStreamMeta* pMeta); diff --git a/source/dnode/snode/src/snode.c b/source/dnode/snode/src/snode.c index 6eee8c510b..6899140af9 100644 --- a/source/dnode/snode/src/snode.c +++ b/source/dnode/snode/src/snode.c @@ -155,7 +155,7 @@ int32_t sndProcessWriteMsg(SSnode *pSnode, SRpcMsg *pMsg, SRpcMsg *pRsp) { } case TDMT_STREAM_TASK_DROP: - return tqStreamTaskProcessDropReq(pSnode->pMeta, pMsg->pCont, pMsg->contLen); + return tqStreamTaskProcessDropReq(pSnode->pMeta, &pSnode->msgCb, pMsg->pCont, pMsg->contLen); case TDMT_VND_STREAM_TASK_UPDATE: return tqStreamTaskProcessUpdateReq(pSnode->pMeta, &pSnode->msgCb, pMsg, true); case TDMT_VND_STREAM_TASK_RESET: diff --git a/source/dnode/vnode/src/sma/smaRollup.c b/source/dnode/vnode/src/sma/smaRollup.c index 7d83dbcf84..1f755f816e 100644 --- a/source/dnode/vnode/src/sma/smaRollup.c +++ b/source/dnode/vnode/src/sma/smaRollup.c @@ -1302,7 +1302,7 @@ _checkpoint: } streamMetaWLock(pMeta); - if ((code = streamMetaSaveTask(pMeta, pTask)) != 0) { + if ((code = streamMetaSaveTaskInMeta(pMeta, pTask)) != 0) { streamMetaWUnLock(pMeta); taosHashCancelIterate(pInfoHash, infoHash); TSDB_CHECK_CODE(code, lino, _exit); diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 1c0f73249c..bd2a7207f2 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1118,7 +1118,7 @@ int32_t tqProcessTaskDispatchRsp(STQ* pTq, SRpcMsg* pMsg) { } int32_t tqProcessTaskDropReq(STQ* pTq, char* msg, int32_t msgLen) { - return tqStreamTaskProcessDropReq(pTq->pStreamMeta, msg, msgLen); + return tqStreamTaskProcessDropReq(pTq->pStreamMeta, &pTq->pVnode->msgCb, msg, msgLen); } int32_t tqProcessTaskUpdateCheckpointReq(STQ* pTq, char* msg, int32_t msgLen) { diff --git a/source/dnode/vnode/src/tqCommon/tqCommon.c b/source/dnode/vnode/src/tqCommon/tqCommon.c index 77632d328e..bfcbbf6c74 100644 --- a/source/dnode/vnode/src/tqCommon/tqCommon.c +++ b/source/dnode/vnode/src/tqCommon/tqCommon.c @@ -260,13 +260,13 @@ int32_t tqStreamTaskProcessUpdateReq(SStreamMeta* pMeta, SMsgCb* cb, SRpcMsg* pM // stream do update the nodeEp info, write it into stream meta. if (updated) { tqDebug("s-task:%s vgId:%d save task after update epset, and stop task", idstr, vgId); - code = streamMetaSaveTask(pMeta, pTask); + code = streamMetaSaveTaskInMeta(pMeta, pTask); if (code) { tqError("s-task:%s vgId:%d failed to save task, code:%s", idstr, vgId, tstrerror(code)); } if (pHTask != NULL) { - code = streamMetaSaveTask(pMeta, pHTask); + code = streamMetaSaveTaskInMeta(pMeta, pHTask); if (code) { tqError("s-task:%s vgId:%d failed to save related history task, code:%s", idstr, vgId, tstrerror(code)); } @@ -688,7 +688,7 @@ int32_t tqStreamTaskProcessDeployReq(SStreamMeta* pMeta, SMsgCb* cb, int64_t sve return code; } -int32_t tqStreamTaskProcessDropReq(SStreamMeta* pMeta, char* msg, int32_t msgLen) { +int32_t tqStreamTaskProcessDropReq(SStreamMeta* pMeta, SMsgCb* cb, char* msg, int32_t msgLen) { SVDropStreamTaskReq* pReq = (SVDropStreamTaskReq*)msg; int32_t code = 0; int32_t vgId = pMeta->vgId; @@ -720,29 +720,40 @@ int32_t tqStreamTaskProcessDropReq(SStreamMeta* pMeta, char* msg, int32_t msgLen // drop the related fill-history task firstly if (hTaskId.taskId != 0 && hTaskId.streamId != 0) { - tqDebug("s-task:0x%x vgId:%d drop rel fill-history task:0x%x firstly", pReq->taskId, vgId, (int32_t)hTaskId.taskId); - code = streamMetaUnregisterTask(pMeta, hTaskId.streamId, hTaskId.taskId); + code = streamTaskSchedTask(cb, vgId, hTaskId.streamId, hTaskId.taskId, STREAM_EXEC_T_DROP_ONE_TASK); if (code) { - tqDebug("s-task:0x%x vgId:%d drop rel fill-history task:0x%x failed", pReq->taskId, vgId, + tqError("s-task:0x%x vgId:%d failed to create msg to drop rel fill-history task:0x%x, code:%s", pReq->taskId, + vgId, (int32_t)hTaskId.taskId, tstrerror(code)); + } else { + tqDebug("s-task:0x%x vgId:%d create msg to drop rel fill-history task:0x%x succ", pReq->taskId, vgId, (int32_t)hTaskId.taskId); } } // drop the stream task now - code = streamMetaUnregisterTask(pMeta, pReq->streamId, pReq->taskId); + code = streamTaskSchedTask(cb, vgId, pReq->streamId, pReq->taskId, STREAM_EXEC_T_DROP_ONE_TASK); if (code) { - tqDebug("s-task:0x%x vgId:%d drop task failed", pReq->taskId, vgId); + tqError("s-task:0x%x vgId:%d failed to create msg to drop task, code:%s", pReq->taskId, vgId, tstrerror(code)); + } else { + tqDebug("s-task:0x%x vgId:%d create msg to drop succ", pReq->taskId, vgId); } +// code = streamMetaUnregisterTask(pMeta, pReq->streamId, pReq->taskId); +// if (code) { +// tqDebug("s-task:0x%x vgId:%d drop task failed", pReq->taskId, vgId); +// } + // commit the update - int32_t numOfTasks = streamMetaGetNumOfTasks(pMeta); - tqDebug("vgId:%d task:0x%x dropped, remain tasks:%d", vgId, pReq->taskId, numOfTasks); +// int32_t numOfTasks = streamMetaGetNumOfTasks(pMeta); +// tqDebug("vgId:%d task:0x%x dropped, remain tasks:%d", vgId, pReq->taskId, numOfTasks); if (streamMetaCommit(pMeta) < 0) { // persist to disk } streamMetaWUnLock(pMeta); + + tqDebug("vgId:%d process drop task:0x%x async completed", vgId, pReq->taskId); return 0; // always return success } @@ -857,6 +868,9 @@ int32_t tqStreamTaskProcessRunReq(SStreamMeta* pMeta, SRpcMsg* pMsg, bool isLead } else if (type == STREAM_EXEC_T_ADD_FAILED_TASK) { code = streamMetaAddFailedTask(pMeta, req.streamId, req.taskId); return code; + } else if (type == STREAM_EXEC_T_DROP_ONE_TASK) { + code = streamMetaDropTask(pMeta, req.streamId, req.taskId); + return code; } else if (type == STREAM_EXEC_T_RESUME_TASK) { // task resume to run after idle for a while SStreamTask* pTask = NULL; code = streamMetaAcquireTask(pMeta, req.streamId, req.taskId, &pTask); diff --git a/source/libs/executor/src/executor.c b/source/libs/executor/src/executor.c index 1386b0b82f..3d60e4d797 100644 --- a/source/libs/executor/src/executor.c +++ b/source/libs/executor/src/executor.c @@ -972,20 +972,28 @@ int32_t qAsyncKillTask(qTaskInfo_t qinfo, int32_t rspCode) { return TSDB_CODE_SUCCESS; } -int32_t qKillTask(qTaskInfo_t tinfo, int32_t rspCode) { +int32_t qKillTask(qTaskInfo_t tinfo, int32_t rspCode, int64_t waitingDuration) { + int64_t st = taosGetTimestampMs(); SExecTaskInfo* pTaskInfo = (SExecTaskInfo*)tinfo; if (pTaskInfo == NULL) { return TSDB_CODE_QRY_INVALID_QHANDLE; } - qDebug("%s sync killed execTask", GET_TASKID(pTaskInfo)); + qDebug("%s sync killed execTask, and waiting for %.2fs", GET_TASKID(pTaskInfo), waitingDuration/1000.0); setTaskKilled(pTaskInfo, TSDB_CODE_TSC_QUERY_KILLED); while (1) { taosWLockLatch(&pTaskInfo->lock); if (qTaskIsExecuting(pTaskInfo)) { // let's wait for 100 ms and try again taosWUnLockLatch(&pTaskInfo->lock); - taosMsleep(100); + + taosMsleep(200); + + int64_t d = taosGetTimestampMs() - st; + if (d >= waitingDuration && waitingDuration >= 0) { + qWarn("%s waiting more than %.2fs, not wait anymore", GET_TASKID(pTaskInfo), waitingDuration/1000.0); + return TSDB_CODE_SUCCESS; + } } else { // not running now pTaskInfo->code = rspCode; taosWUnLockLatch(&pTaskInfo->lock); diff --git a/source/libs/stream/src/streamCheckpoint.c b/source/libs/stream/src/streamCheckpoint.c index 1a49e50547..d62165a72c 100644 --- a/source/libs/stream/src/streamCheckpoint.c +++ b/source/libs/stream/src/streamCheckpoint.c @@ -625,14 +625,10 @@ int32_t streamTaskUpdateTaskCheckpointInfo(SStreamTask* pTask, bool restored, SV { // destroy the related fill-history tasks // drop task should not in the meta-lock, and drop the related fill-history task now if (pReq->dropRelHTask) { - code = streamMetaUnregisterTask(pMeta, pReq->hStreamId, pReq->hTaskId); - int32_t numOfTasks = streamMetaGetNumOfTasks(pMeta); - stDebug("s-task:%s vgId:%d related fill-history task:0x%x dropped in update checkpointInfo, remain tasks:%d", - id, vgId, pReq->taskId, numOfTasks); - } - - if (pReq->dropRelHTask) { - code = streamMetaCommit(pMeta); + code = streamTaskSchedTask(pTask->pMsgCb, vgId, pReq->hStreamId, pReq->hTaskId, STREAM_EXEC_T_DROP_ONE_TASK); + if (code) { + stError("s-task:%s failed to create msg to drop related fill-history task, code:%s", id, tstrerror(code)); + } } } @@ -697,7 +693,7 @@ int32_t streamTaskUpdateTaskCheckpointInfo(SStreamTask* pTask, bool restored, SV pTask->status.taskStatus = TASK_STATUS__READY; - code = streamMetaSaveTask(pMeta, pTask); + code = streamMetaSaveTaskInMeta(pMeta, pTask); streamMutexUnlock(&pTask->lock); if (code != TSDB_CODE_SUCCESS) { @@ -708,10 +704,12 @@ int32_t streamTaskUpdateTaskCheckpointInfo(SStreamTask* pTask, bool restored, SV // drop task should not in the meta-lock, and drop the related fill-history task now if (pReq->dropRelHTask) { - code = streamMetaUnregisterTask(pMeta, pReq->hStreamId, pReq->hTaskId); - int32_t numOfTasks = streamMetaGetNumOfTasks(pMeta); - stDebug("s-task:%s vgId:%d related fill-history task:0x%x dropped, remain tasks:%d", id, vgId, - (int32_t)pReq->hTaskId, numOfTasks); + code = streamTaskSchedTask(pTask->pMsgCb, vgId, pReq->hStreamId, pReq->hTaskId, STREAM_EXEC_T_DROP_ONE_TASK); + if (code) { + stError("s-task:%s failed to create msg to drop related fill-history task, code:%s", id, tstrerror(code)); + } else { + stDebug("s-task:%s vgId:%d create msg to drop related fill-history task:0x%x", id, vgId, (int32_t)pReq->hTaskId); + } } code = streamMetaCommit(pMeta); diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index ddd0201460..ce1c228b7b 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -632,7 +632,7 @@ void streamMetaCloseImpl(void* arg) { } // todo let's check the status for each task -int32_t streamMetaSaveTask(SStreamMeta* pMeta, SStreamTask* pTask) { +int32_t streamMetaSaveTaskInMeta(SStreamMeta* pMeta, SStreamTask* pTask) { int32_t vgId = pTask->pMeta->vgId; void* buf = NULL; int32_t len; @@ -682,7 +682,7 @@ int32_t streamMetaSaveTask(SStreamMeta* pMeta, SStreamTask* pTask) { return code; } -int32_t streamMetaRemoveTask(SStreamMeta* pMeta, STaskId* pTaskId) { +int32_t streamMetaRemoveTaskInMeta(SStreamMeta* pMeta, STaskId* pTaskId) { int64_t key[2] = {pTaskId->streamId, pTaskId->taskId}; int32_t code = tdbTbDelete(pMeta->pTaskDb, key, STREAM_TASK_KEY_LEN, pMeta->txn); if (code != 0) { @@ -705,7 +705,7 @@ int32_t streamMetaRegisterTask(SStreamMeta* pMeta, int64_t ver, SStreamTask* pTa void* p = taosHashGet(pMeta->pTasksMap, &id, sizeof(id)); if (p != NULL) { - stDebug("s-task:%" PRIx64 " already exist in meta, no need to register", id.taskId); + stDebug("s-task:0x%" PRIx64 " already exist in meta, no need to register", id.taskId); tFreeStreamTask(pTask); return code; } @@ -735,7 +735,7 @@ int32_t streamMetaRegisterTask(SStreamMeta* pMeta, int64_t ver, SStreamTask* pTa return code; } - if ((code = streamMetaSaveTask(pMeta, pTask)) != 0) { + if ((code = streamMetaSaveTaskInMeta(pMeta, pTask)) != 0) { int32_t unused = taosHashRemove(pMeta->pTasksMap, &id, sizeof(id)); void* pUnused = taosArrayPop(pMeta->pTaskList); @@ -885,6 +885,8 @@ static void doRemoveIdFromList(SArray* pTaskList, int32_t num, SStreamTaskId* id static int32_t streamTaskSendTransSuccessMsg(SStreamTask* pTask, void* param) { int32_t code = 0; + int32_t waitingDuration = 5000; + if (pTask->info.taskLevel == TASK_LEVEL__SOURCE) { code = streamTaskSendCheckpointSourceRsp(pTask); if (code) { @@ -895,7 +897,7 @@ static int32_t streamTaskSendTransSuccessMsg(SStreamTask* pTask, void* param) { // let's kill the query procedure within stream, to end it ASAP. if (pTask->info.taskLevel != TASK_LEVEL__SINK && pTask->exec.pExecutor != NULL) { - code = qKillTask(pTask->exec.pExecutor, TSDB_CODE_SUCCESS); + code = qKillTask(pTask->exec.pExecutor, TSDB_CODE_SUCCESS, 5000); if (code != TSDB_CODE_SUCCESS) { stError("s-task:%s failed to kill task related query handle, code:%s", pTask->id.idStr, tstrerror(code)); } @@ -932,7 +934,7 @@ int32_t streamMetaUnregisterTask(SStreamMeta* pMeta, int64_t streamId, int32_t t code = taosHashRemove(pMeta->pTasksMap, &id, sizeof(id)); doRemoveIdFromList(pMeta->pTaskList, (int32_t)taosArrayGetSize(pMeta->pTaskList), &pTask->id); - code = streamMetaRemoveTask(pMeta, &id); + code = streamMetaRemoveTaskInMeta(pMeta, &id); if (code) { stError("vgId:%d failed to remove task:0x%" PRIx64 ", code:%s", pMeta->vgId, id.taskId, tstrerror(code)); } @@ -963,6 +965,32 @@ int32_t streamMetaUnregisterTask(SStreamMeta* pMeta, int64_t streamId, int32_t t return 0; } +int32_t streamMetaDropTask(SStreamMeta* pMeta, int64_t streamId, int32_t taskId) { + SStreamTask* pTask = NULL; + int32_t code = 0; + int32_t vgId = pMeta->vgId; + int32_t numOfTasks = 0; + + streamMetaWLock(pMeta); + + code = streamMetaUnregisterTask(pMeta, streamId, taskId); + numOfTasks = streamMetaGetNumOfTasks(pMeta); + if (code) { + stError("vgId:%d failed to drop task:0x%x, code:%s", vgId, taskId, tstrerror(code)); + } + + code = streamMetaCommit(pMeta); + if (code) { + stError("vgId:%d failed to commit after drop task:0x%x, code:%s", vgId, taskId, tstrerror(code)); + } else { + stDebug("s-task:0x%"PRIx64"-0x%x vgId:%d dropped, remain tasks:%d", streamId, taskId, pMeta->vgId, numOfTasks); + } + + streamMetaWUnLock(pMeta); + + return code; +} + int32_t streamMetaBegin(SStreamMeta* pMeta) { streamMetaWLock(pMeta); int32_t code = tdbBegin(pMeta->db, &pMeta->txn, tdbDefaultMalloc, tdbDefaultFree, NULL, @@ -1185,7 +1213,7 @@ void streamMetaLoadAllTasks(SStreamMeta* pMeta) { if (taosArrayGetSize(pRecycleList) > 0) { for (int32_t i = 0; i < taosArrayGetSize(pRecycleList); ++i) { STaskId* pId = taosArrayGet(pRecycleList, i); - code = streamMetaRemoveTask(pMeta, pId); + code = streamMetaRemoveTaskInMeta(pMeta, pId); if (code) { stError("s-task:0x%" PRIx64 " failed to remove task, code:%s", pId->taskId, tstrerror(code)); } diff --git a/source/libs/stream/src/streamTask.c b/source/libs/stream/src/streamTask.c index d27ed520c6..f0b288f096 100644 --- a/source/libs/stream/src/streamTask.c +++ b/source/libs/stream/src/streamTask.c @@ -703,7 +703,7 @@ int32_t streamTaskStop(SStreamTask* pTask) { } if (pTask->info.taskLevel != TASK_LEVEL__SINK && pTask->exec.pExecutor != NULL) { - code = qKillTask(pTask->exec.pExecutor, TSDB_CODE_SUCCESS); + code = qKillTask(pTask->exec.pExecutor, TSDB_CODE_SUCCESS, 5000); if (code != TSDB_CODE_SUCCESS) { stError("s-task:%s failed to kill task related query handle, code:%s", id, tstrerror(code)); } @@ -862,7 +862,7 @@ int32_t streamTaskClearHTaskAttr(SStreamTask* pTask, int32_t resetRelHalt) { pStreamTask->status.taskStatus = TASK_STATUS__READY; } - code = streamMetaSaveTask(pMeta, pStreamTask); + code = streamMetaSaveTaskInMeta(pMeta, pStreamTask); streamMutexUnlock(&(pStreamTask->lock)); streamMetaReleaseTask(pMeta, pStreamTask); @@ -1025,7 +1025,7 @@ static int32_t taskPauseCallback(SStreamTask* pTask, void* param) { // in case of fill-history task, stop the tsdb file scan operation. if (pTask->info.fillHistory == 1) { void* pExecutor = pTask->exec.pExecutor; - code = qKillTask(pExecutor, TSDB_CODE_SUCCESS); + code = qKillTask(pExecutor, TSDB_CODE_SUCCESS, 10000); } stDebug("vgId:%d s-task:%s set pause flag and pause task", pMeta->vgId, pTask->id.idStr); @@ -1287,6 +1287,8 @@ const char* streamTaskGetExecType(int32_t type) { return "resume-task-from-idle"; case STREAM_EXEC_T_ADD_FAILED_TASK: return "record-start-failed-task"; + case STREAM_EXEC_T_DROP_ONE_TASK: + return "drop-one-task"; case 0: return "exec-all-tasks"; default: From 7319f544c2e9eb879a98569d9032e06549509c05 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 13 Feb 2025 23:43:07 +0800 Subject: [PATCH 15/57] refactor(stream): drop task in async ways --- source/dnode/vnode/src/tqCommon/tqCommon.c | 2 +- tests/ci/func.txt | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/source/dnode/vnode/src/tqCommon/tqCommon.c b/source/dnode/vnode/src/tqCommon/tqCommon.c index bfcbbf6c74..29b9141759 100644 --- a/source/dnode/vnode/src/tqCommon/tqCommon.c +++ b/source/dnode/vnode/src/tqCommon/tqCommon.c @@ -720,7 +720,7 @@ int32_t tqStreamTaskProcessDropReq(SStreamMeta* pMeta, SMsgCb* cb, char* msg, in // drop the related fill-history task firstly if (hTaskId.taskId != 0 && hTaskId.streamId != 0) { - code = streamTaskSchedTask(cb, vgId, hTaskId.streamId, hTaskId.taskId, STREAM_EXEC_T_DROP_ONE_TASK); + code = streamTaskSchedTask(cb, vgId, hTaskId.streamId, (int32_t)hTaskId.taskId, STREAM_EXEC_T_DROP_ONE_TASK); if (code) { tqError("s-task:0x%x vgId:%d failed to create msg to drop rel fill-history task:0x%x, code:%s", pReq->taskId, vgId, (int32_t)hTaskId.taskId, tstrerror(code)); diff --git a/tests/ci/func.txt b/tests/ci/func.txt index 45d4fb1c11..c724568537 100644 --- a/tests/ci/func.txt +++ b/tests/ci/func.txt @@ -79,7 +79,7 @@ (void)streamMetaAddFailedTask (void)streamMetaAddTaskLaunchResult (void)streamMetaCommit -(void)streamMetaRemoveTask +(void)streamMetaRemoveTaskInMeta (void)streamMetaSendHbHelper (void)streamMetaStartAllTasks (void)streamMetaStartOneTask From 45b0e47a1931c550cd9bbd91adc8659631209496 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Fri, 14 Feb 2025 18:34:20 +0800 Subject: [PATCH 16/57] refactor(stream): add long exec stream queue for history tasks in step1 and re-calculate task execution. --- include/common/tmsgcb.h | 1 + include/util/tworker.h | 2 +- source/dnode/mgmt/mgmt_vnode/inc/vmInt.h | 4 ++ source/dnode/mgmt/mgmt_vnode/src/vmHandle.c | 18 ++++--- source/dnode/mgmt/mgmt_vnode/src/vmInt.c | 8 ++- source/dnode/mgmt/mgmt_vnode/src/vmWorker.c | 59 ++++++++++++++++++--- source/dnode/vnode/inc/vnode.h | 1 + source/dnode/vnode/src/vnd/vnodeSvr.c | 22 ++++++-- source/libs/executor/src/projectoperator.c | 8 --- source/libs/stream/src/streamExec.c | 2 +- source/libs/stream/src/streamStartHistory.c | 2 +- source/util/src/tworker.c | 7 ++- 12 files changed, 99 insertions(+), 35 deletions(-) diff --git a/include/common/tmsgcb.h b/include/common/tmsgcb.h index c934cb6961..2847f4278a 100644 --- a/include/common/tmsgcb.h +++ b/include/common/tmsgcb.h @@ -38,6 +38,7 @@ typedef enum { STREAM_QUEUE, ARB_QUEUE, STREAM_CTRL_QUEUE, + STREAM_LONG_EXEC_QUEUE, QUEUE_MAX, } EQueueType; diff --git a/include/util/tworker.h b/include/util/tworker.h index a3ba7dba6d..bc0dde1a37 100644 --- a/include/util/tworker.h +++ b/include/util/tworker.h @@ -76,7 +76,7 @@ void tQWorkerFreeQueue(SQWorkerPool *pool, STaosQueue *queue); int32_t tAutoQWorkerInit(SAutoQWorkerPool *pool); void tAutoQWorkerCleanup(SAutoQWorkerPool *pool); -STaosQueue *tAutoQWorkerAllocQueue(SAutoQWorkerPool *pool, void *ahandle, FItem fp); +STaosQueue *tAutoQWorkerAllocQueue(SAutoQWorkerPool *pool, void *ahandle, FItem fp, int32_t minNum); void tAutoQWorkerFreeQueue(SAutoQWorkerPool *pool, STaosQueue *queue); int32_t tWWorkerInit(SWWorkerPool *pool); diff --git a/source/dnode/mgmt/mgmt_vnode/inc/vmInt.h b/source/dnode/mgmt/mgmt_vnode/inc/vmInt.h index 84f5149624..9b4c11d6ae 100644 --- a/source/dnode/mgmt/mgmt_vnode/inc/vmInt.h +++ b/source/dnode/mgmt/mgmt_vnode/inc/vmInt.h @@ -32,6 +32,7 @@ typedef struct SVnodeMgmt { const char *name; SQueryAutoQWorkerPool queryPool; SAutoQWorkerPool streamPool; + SAutoQWorkerPool streamLongExecPool; SWWorkerPool streamCtrlPool; SWWorkerPool fetchPool; SSingleWorker mgmtWorker; @@ -75,6 +76,7 @@ typedef struct { STaosQueue *pQueryQ; STaosQueue *pStreamQ; STaosQueue *pStreamCtrlQ; + STaosQueue *pStreamLongExecQ; STaosQueue *pFetchQ; STaosQueue *pMultiMgmQ; } SVnodeObj; @@ -137,6 +139,8 @@ int32_t vmPutMsgToQueryQueue(SVnodeMgmt *pMgmt, SRpcMsg *pMsg); int32_t vmPutMsgToFetchQueue(SVnodeMgmt *pMgmt, SRpcMsg *pMsg); int32_t vmPutMsgToStreamQueue(SVnodeMgmt *pMgmt, SRpcMsg *pMsg); int32_t vmPutMsgToStreamCtrlQueue(SVnodeMgmt *pMgmt, SRpcMsg *pMsg); +int32_t vmPutMsgToStreamLongExecQueue(SVnodeMgmt *pMgmt, SRpcMsg *pMsg); + int32_t vmPutMsgToMergeQueue(SVnodeMgmt *pMgmt, SRpcMsg *pMsg); int32_t vmPutMsgToMgmtQueue(SVnodeMgmt *pMgmt, SRpcMsg *pMsg); int32_t vmPutMsgToMultiMgmtQueue(SVnodeMgmt *pMgmt, SRpcMsg *pMsg); diff --git a/source/dnode/mgmt/mgmt_vnode/src/vmHandle.c b/source/dnode/mgmt/mgmt_vnode/src/vmHandle.c index 234d4f41e1..1dea7d3cad 100644 --- a/source/dnode/mgmt/mgmt_vnode/src/vmHandle.c +++ b/source/dnode/mgmt/mgmt_vnode/src/vmHandle.c @@ -1008,27 +1008,29 @@ SArray *vmGetMsgHandles() { if (dmSetMgmtHandle(pArray, TDMT_STREAM_TASK_RUN, vmPutMsgToStreamQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_STREAM_TASK_DISPATCH, vmPutMsgToStreamCtrlQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_STREAM_TASK_DISPATCH_RSP, vmPutMsgToStreamCtrlQueue, 0) == NULL) goto _OVER; - if (dmSetMgmtHandle(pArray, TDMT_STREAM_RETRIEVE, vmPutMsgToStreamQueue, 0) == NULL) goto _OVER; - if (dmSetMgmtHandle(pArray, TDMT_STREAM_RETRIEVE_RSP, vmPutMsgToStreamQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_VND_STREAM_TASK_CHECK, vmPutMsgToStreamCtrlQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_VND_STREAM_TASK_CHECK_RSP, vmPutMsgToStreamCtrlQueue, 0) == NULL) goto _OVER; - if (dmSetMgmtHandle(pArray, TDMT_STREAM_TASK_PAUSE, vmPutMsgToWriteQueue, 0) == NULL) goto _OVER; - if (dmSetMgmtHandle(pArray, TDMT_STREAM_TASK_RESUME, vmPutMsgToWriteQueue, 0) == NULL) goto _OVER; - if (dmSetMgmtHandle(pArray, TDMT_STREAM_TASK_STOP, vmPutMsgToWriteQueue, 0) == NULL) goto _OVER; - if (dmSetMgmtHandle(pArray, TDMT_VND_STREAM_CHECK_POINT_SOURCE, vmPutMsgToWriteQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_STREAM_TASK_CHECKPOINT_READY, vmPutMsgToStreamCtrlQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_STREAM_TASK_CHECKPOINT_READY_RSP, vmPutMsgToStreamCtrlQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_STREAM_RETRIEVE_TRIGGER, vmPutMsgToStreamCtrlQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_STREAM_RETRIEVE_TRIGGER_RSP, vmPutMsgToStreamCtrlQueue, 0) == NULL) goto _OVER; - if (dmSetMgmtHandle(pArray, TDMT_VND_STREAM_TASK_UPDATE, vmPutMsgToWriteQueue, 0) == NULL) goto _OVER; - if (dmSetMgmtHandle(pArray, TDMT_VND_STREAM_TASK_RESET, vmPutMsgToWriteQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_MND_STREAM_HEARTBEAT_RSP, vmPutMsgToStreamCtrlQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_MND_STREAM_REQ_CHKPT_RSP, vmPutMsgToStreamCtrlQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_MND_STREAM_CHKPT_REPORT_RSP, vmPutMsgToStreamCtrlQueue, 0) == NULL) goto _OVER; + if (dmSetMgmtHandle(pArray, TDMT_VND_STREAM_SCAN_HISTORY, vmPutMsgToStreamLongExecQueue, 0) == NULL) goto _OVER; + if (dmSetMgmtHandle(pArray, TDMT_VND_GET_STREAM_PROGRESS, vmPutMsgToStreamQueue, 0) == NULL) goto _OVER; + if (dmSetMgmtHandle(pArray, TDMT_STREAM_RETRIEVE, vmPutMsgToStreamQueue, 0) == NULL) goto _OVER; + if (dmSetMgmtHandle(pArray, TDMT_STREAM_RETRIEVE_RSP, 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_STREAM_TASK_PAUSE, vmPutMsgToWriteQueue, 0) == NULL) goto _OVER; + if (dmSetMgmtHandle(pArray, TDMT_STREAM_TASK_RESUME, vmPutMsgToWriteQueue, 0) == NULL) goto _OVER; + if (dmSetMgmtHandle(pArray, TDMT_STREAM_TASK_STOP, vmPutMsgToWriteQueue, 0) == NULL) goto _OVER; + if (dmSetMgmtHandle(pArray, TDMT_VND_STREAM_CHECK_POINT_SOURCE, vmPutMsgToWriteQueue, 0) == NULL) goto _OVER; + if (dmSetMgmtHandle(pArray, TDMT_VND_STREAM_TASK_UPDATE, vmPutMsgToWriteQueue, 0) == NULL) goto _OVER; + if (dmSetMgmtHandle(pArray, TDMT_VND_STREAM_TASK_RESET, 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/mgmt/mgmt_vnode/src/vmInt.c b/source/dnode/mgmt/mgmt_vnode/src/vmInt.c index d71e0b02c4..6f30977e10 100644 --- a/source/dnode/mgmt/mgmt_vnode/src/vmInt.c +++ b/source/dnode/mgmt/mgmt_vnode/src/vmInt.c @@ -398,10 +398,14 @@ void vmCloseVnode(SVnodeMgmt *pMgmt, SVnodeObj *pVnode, bool commitAndRemoveWal, dInfo("vgId:%d, wait for vnode stream queue:%p is empty, %d remains", pVnode->vgId, pVnode->pStreamQ, taosQueueItemSize(pVnode->pStreamQ)); - while (!taosQueueEmpty(pVnode->pStreamQ)) taosMsleep(10); + while (!taosQueueEmpty(pVnode->pStreamQ)) taosMsleep(50); dInfo("vgId:%d, wait for vnode stream ctrl queue:%p is empty", pVnode->vgId, pVnode->pStreamCtrlQ); - while (!taosQueueEmpty(pVnode->pStreamCtrlQ)) taosMsleep(10); + while (!taosQueueEmpty(pVnode->pStreamCtrlQ)) taosMsleep(50); + + dInfo("vgId:%d, wait for vnode stream long-exec queue:%p is empty, %d remains", pVnode->vgId, + pVnode->pStreamLongExecQ, taosQueueItemSize(pVnode->pStreamLongExecQ)); + while (!taosQueueEmpty(pVnode->pStreamLongExecQ)) taosMsleep(50); dInfo("vgId:%d, all vnode queues is empty", pVnode->vgId); diff --git a/source/dnode/mgmt/mgmt_vnode/src/vmWorker.c b/source/dnode/mgmt/mgmt_vnode/src/vmWorker.c index b398bdf242..5acd06bbda 100644 --- a/source/dnode/mgmt/mgmt_vnode/src/vmWorker.c +++ b/source/dnode/mgmt/mgmt_vnode/src/vmWorker.c @@ -150,7 +150,7 @@ static void vmProcessStreamCtrlQueue(SQueueInfo *pInfo, STaosQall* pQall, int32_ SRpcMsg *pMsg = pItem; const STraceId *trace = &pMsg->info.traceId; - dGTrace("vgId:%d, msg:%p get from vnode-ctrl-stream queue", pVnode->vgId, pMsg); + dGTrace("vgId:%d, msg:%p get from vnode-stream-ctrl queue", pVnode->vgId, pMsg); code = vnodeProcessStreamCtrlMsg(pVnode->pImpl, pMsg, pInfo); if (code != 0) { terrno = code; @@ -165,6 +165,26 @@ static void vmProcessStreamCtrlQueue(SQueueInfo *pInfo, STaosQall* pQall, int32_ } } +static void vmProcessStreamLongExecQueue(SQueueInfo *pInfo, SRpcMsg *pMsg) { + SVnodeObj *pVnode = pInfo->ahandle; + const STraceId *trace = &pMsg->info.traceId; + int32_t code = 0; + + dGTrace("vgId:%d, msg:%p get from vnode-stream long-exec queue", pVnode->vgId, pMsg); + + code = vnodeProcessStreamLongExecMsg(pVnode->pImpl, pMsg, pInfo); + if (code != 0) { + terrno = code; + dGError("vgId:%d, msg:%p failed to process stream msg %s since %s", pVnode->vgId, pMsg, TMSG_INFO(pMsg->msgType), + tstrerror(code)); + vmSendRsp(pMsg, code); + } + + dGTrace("vgId:%d, msg:%p is freed, code:0x%x", pVnode->vgId, pMsg, code); + rpcFreeCont(pMsg->pCont); + taosFreeQitem(pMsg); +} + static void vmProcessFetchQueue(SQueueInfo *pInfo, STaosQall *qall, int32_t numOfMsgs) { SVnodeObj *pVnode = pInfo->ahandle; SRpcMsg *pMsg = NULL; @@ -274,9 +294,13 @@ static int32_t vmPutMsgToQueue(SVnodeMgmt *pMgmt, SRpcMsg *pMsg, EQueueType qtyp code = taosWriteQitem(pVnode->pStreamQ, pMsg); break; case STREAM_CTRL_QUEUE: - dGTrace("vgId:%d, msg:%p put into vnode-ctrl-stream queue", pVnode->vgId, pMsg); + dGTrace("vgId:%d, msg:%p put into vnode-stream-ctrl queue", pVnode->vgId, pMsg); code = taosWriteQitem(pVnode->pStreamCtrlQ, pMsg); break; + case STREAM_LONG_EXEC_QUEUE: + dGTrace("vgId:%d, msg:%p put into vnode-stream-long-exec queue", pVnode->vgId, pMsg); + code = taosWriteQitem(pVnode->pStreamLongExecQ, pMsg); + break; case FETCH_QUEUE: dGTrace("vgId:%d, msg:%p put into vnode-fetch queue", pVnode->vgId, pMsg); code = taosWriteQitem(pVnode->pFetchQ, pMsg); @@ -335,6 +359,8 @@ int32_t vmPutMsgToStreamQueue(SVnodeMgmt *pMgmt, SRpcMsg *pMsg) { return vmPutMs int32_t vmPutMsgToStreamCtrlQueue(SVnodeMgmt *pMgmt, SRpcMsg *pMsg) { return vmPutMsgToQueue(pMgmt, pMsg, STREAM_CTRL_QUEUE); } +int32_t vmPutMsgToStreamLongExecQueue(SVnodeMgmt *pMgmt, SRpcMsg *pMsg) { return vmPutMsgToQueue(pMgmt, pMsg, STREAM_LONG_EXEC_QUEUE); } + int32_t vmPutMsgToMultiMgmtQueue(SVnodeMgmt *pMgmt, SRpcMsg *pMsg) { const STraceId *trace = &pMsg->info.traceId; dGTrace("msg:%p, put into vnode-multi-mgmt queue", pMsg); @@ -409,6 +435,10 @@ int32_t vmGetQueueSize(SVnodeMgmt *pMgmt, int32_t vgId, EQueueType qtype) { break; case STREAM_CTRL_QUEUE: size = taosQueueItemSize(pVnode->pStreamCtrlQ); + break; + case STREAM_LONG_EXEC_QUEUE: + size = taosQueueItemSize(pVnode->pStreamLongExecQ); + break; default: break; } @@ -451,13 +481,16 @@ int32_t vmAllocQueue(SVnodeMgmt *pMgmt, SVnodeObj *pVnode) { } pVnode->pQueryQ = tQueryAutoQWorkerAllocQueue(&pMgmt->queryPool, pVnode, (FItem)vmProcessQueryQueue); - pVnode->pStreamQ = tAutoQWorkerAllocQueue(&pMgmt->streamPool, pVnode, (FItem)vmProcessStreamQueue); pVnode->pFetchQ = tWWorkerAllocQueue(&pMgmt->fetchPool, pVnode, (FItems)vmProcessFetchQueue); + + // init stream msg processing queue family + pVnode->pStreamQ = tAutoQWorkerAllocQueue(&pMgmt->streamPool, pVnode, (FItem)vmProcessStreamQueue, 2); pVnode->pStreamCtrlQ = tWWorkerAllocQueue(&pMgmt->streamCtrlPool, pVnode, (FItems)vmProcessStreamCtrlQueue); + pVnode->pStreamLongExecQ = tAutoQWorkerAllocQueue(&pMgmt->streamLongExecPool, pVnode, (FItem)vmProcessStreamLongExecQueue, 1); if (pVnode->pWriteW.queue == NULL || pVnode->pSyncW.queue == NULL || pVnode->pSyncRdW.queue == NULL || pVnode->pApplyW.queue == NULL || pVnode->pQueryQ == NULL || pVnode->pStreamQ == NULL || pVnode->pFetchQ == NULL - || pVnode->pStreamCtrlQ == NULL) { + || pVnode->pStreamCtrlQ == NULL || pVnode->pStreamLongExecQ == NULL) { return TSDB_CODE_OUT_OF_MEMORY; } @@ -473,6 +506,7 @@ int32_t vmAllocQueue(SVnodeMgmt *pMgmt, SVnodeObj *pVnode) { dInfo("vgId:%d, fetch-queue:%p is alloced, thread:%08" PRId64, pVnode->vgId, pVnode->pFetchQ, taosQueueGetThreadId(pVnode->pFetchQ)); dInfo("vgId:%d, stream-queue:%p is alloced", pVnode->vgId, pVnode->pStreamQ); + dInfo("vgId:%d, stream-long-exec-queue:%p is alloced", pVnode->vgId, pVnode->pStreamLongExecQ); dInfo("vgId:%d, stream-ctrl-queue:%p is alloced, thread:%08" PRId64, pVnode->vgId, pVnode->pStreamCtrlQ, taosQueueGetThreadId(pVnode->pStreamCtrlQ)); return 0; @@ -481,17 +515,22 @@ int32_t vmAllocQueue(SVnodeMgmt *pMgmt, SVnodeObj *pVnode) { void vmFreeQueue(SVnodeMgmt *pMgmt, SVnodeObj *pVnode) { tQueryAutoQWorkerFreeQueue(&pMgmt->queryPool, pVnode->pQueryQ); tAutoQWorkerFreeQueue(&pMgmt->streamPool, pVnode->pStreamQ); + tAutoQWorkerFreeQueue(&pMgmt->streamLongExecPool, pVnode->pStreamLongExecQ); tWWorkerFreeQueue(&pMgmt->streamCtrlPool, pVnode->pStreamCtrlQ); tWWorkerFreeQueue(&pMgmt->fetchPool, pVnode->pFetchQ); pVnode->pQueryQ = NULL; + pVnode->pFetchQ = NULL; + pVnode->pStreamQ = NULL; pVnode->pStreamCtrlQ = NULL; - pVnode->pFetchQ = NULL; + pVnode->pStreamLongExecQ = NULL; + dDebug("vgId:%d, queue is freed", pVnode->vgId); } int32_t vmStartWorker(SVnodeMgmt *pMgmt) { - int32_t code = 0; + int32_t code = 0; + SQueryAutoQWorkerPool *pQPool = &pMgmt->queryPool; pQPool->name = "vnode-query"; pQPool->min = tsNumOfVnodeQueryThreads; @@ -505,8 +544,13 @@ int32_t vmStartWorker(SVnodeMgmt *pMgmt) { pStreamPool->ratio = tsRatioOfVnodeStreamThreads; if ((code = tAutoQWorkerInit(pStreamPool)) != 0) return code; + SAutoQWorkerPool *pLongExecPool = &pMgmt->streamLongExecPool; + pLongExecPool->name = "vnode-stream-long-exec"; + pLongExecPool->ratio = tsRatioOfVnodeStreamThreads/3; + if ((code = tAutoQWorkerInit(pLongExecPool)) != 0) return code; + SWWorkerPool *pStreamCtrlPool = &pMgmt->streamCtrlPool; - pStreamCtrlPool->name = "vnode-ctrl-stream"; + pStreamCtrlPool->name = "vnode-stream-ctrl"; pStreamCtrlPool->max = 1; if ((code = tWWorkerInit(pStreamCtrlPool)) != 0) return code; @@ -541,6 +585,7 @@ int32_t vmStartWorker(SVnodeMgmt *pMgmt) { void vmStopWorker(SVnodeMgmt *pMgmt) { tQueryAutoQWorkerCleanup(&pMgmt->queryPool); tAutoQWorkerCleanup(&pMgmt->streamPool); + tAutoQWorkerCleanup(&pMgmt->streamLongExecPool); tWWorkerCleanup(&pMgmt->streamCtrlPool); tWWorkerCleanup(&pMgmt->fetchPool); dDebug("vnode workers are closed"); diff --git a/source/dnode/vnode/inc/vnode.h b/source/dnode/vnode/inc/vnode.h index 50d75c4838..871a8c06e1 100644 --- a/source/dnode/vnode/inc/vnode.h +++ b/source/dnode/vnode/inc/vnode.h @@ -113,6 +113,7 @@ int32_t vnodeProcessQueryMsg(SVnode *pVnode, SRpcMsg *pMsg, SQueueInfo *pInfo); int32_t vnodeProcessFetchMsg(SVnode *pVnode, SRpcMsg *pMsg, SQueueInfo *pInfo); int32_t vnodeProcessStreamMsg(SVnode *pVnode, SRpcMsg *pMsg, SQueueInfo *pInfo); int32_t vnodeProcessStreamCtrlMsg(SVnode *pVnode, SRpcMsg *pMsg, SQueueInfo *pInfo); +int32_t vnodeProcessStreamLongExecMsg(SVnode *pVnode, SRpcMsg *pMsg, SQueueInfo *pInfo); void vnodeProposeWriteMsg(SQueueInfo *pInfo, STaosQall *qall, int32_t numOfMsgs); void vnodeApplyWriteMsg(SQueueInfo *pInfo, STaosQall *qall, int32_t numOfMsgs); void vnodeProposeCommitOnNeed(SVnode *pVnode, bool atExit); diff --git a/source/dnode/vnode/src/vnd/vnodeSvr.c b/source/dnode/vnode/src/vnd/vnodeSvr.c index 6190f4b0a7..f37dd94106 100644 --- a/source/dnode/vnode/src/vnd/vnodeSvr.c +++ b/source/dnode/vnode/src/vnd/vnodeSvr.c @@ -931,9 +931,7 @@ int32_t vnodeProcessFetchMsg(SVnode *pVnode, SRpcMsg *pMsg, SQueueInfo *pInfo) { int32_t vnodeProcessStreamMsg(SVnode *pVnode, SRpcMsg *pMsg, SQueueInfo *pInfo) { vTrace("vgId:%d, msg:%p in stream queue is processing", pVnode->config.vgId, pMsg); - if ((pMsg->msgType == TDMT_SCH_FETCH || pMsg->msgType == TDMT_VND_TABLE_META || pMsg->msgType == TDMT_VND_TABLE_CFG || - pMsg->msgType == TDMT_VND_BATCH_META) && - !syncIsReadyForRead(pVnode->sync)) { + if (!syncIsReadyForRead(pVnode->sync)) { vnodeRedirectRpcMsg(pVnode, pMsg, terrno); return 0; } @@ -945,8 +943,6 @@ int32_t vnodeProcessStreamMsg(SVnode *pVnode, SRpcMsg *pMsg, SQueueInfo *pInfo) return tqProcessTaskRetrieveReq(pVnode->pTq, pMsg); case TDMT_STREAM_RETRIEVE_RSP: return tqProcessTaskRetrieveRsp(pVnode->pTq, pMsg); - case TDMT_VND_STREAM_SCAN_HISTORY: - return tqProcessTaskScanHistory(pVnode->pTq, pMsg); case TDMT_VND_GET_STREAM_PROGRESS: return tqStreamProgressRetrieveReq(pVnode->pTq, pMsg); default: @@ -993,6 +989,22 @@ int32_t vnodeProcessStreamCtrlMsg(SVnode *pVnode, SRpcMsg *pMsg, SQueueInfo *pIn } } +int32_t vnodeProcessStreamLongExecMsg(SVnode *pVnode, SRpcMsg *pMsg, SQueueInfo *pInfo) { + vTrace("vgId:%d, msg:%p in stream long exec queue is processing", pVnode->config.vgId, pMsg); + if (!syncIsReadyForRead(pVnode->sync)) { + vnodeRedirectRpcMsg(pVnode, pMsg, terrno); + return 0; + } + + switch (pMsg->msgType) { + case TDMT_VND_STREAM_SCAN_HISTORY: + return tqProcessTaskScanHistory(pVnode->pTq, pMsg); + default: + vError("unknown msg type:%d in stream long exec queue", pMsg->msgType); + return TSDB_CODE_APP_ERROR; + } +} + void smaHandleRes(void *pVnode, int64_t smaId, const SArray *data) { int32_t code = tdProcessTSmaInsert(((SVnode *)pVnode)->pSma, smaId, (const char *)data); if (code) { diff --git a/source/libs/executor/src/projectoperator.c b/source/libs/executor/src/projectoperator.c index cb91bae691..0aab1511a4 100644 --- a/source/libs/executor/src/projectoperator.c +++ b/source/libs/executor/src/projectoperator.c @@ -42,9 +42,7 @@ typedef struct SIndefOperatorInfo { } SIndefOperatorInfo; static int32_t doGenerateSourceData(SOperatorInfo* pOperator); -static SSDataBlock* doProjectOperation1(SOperatorInfo* pOperator); static int32_t doProjectOperation(SOperatorInfo* pOperator, SSDataBlock** pResBlock); -static SSDataBlock* doApplyIndefinitFunction1(SOperatorInfo* pOperator); static int32_t doApplyIndefinitFunction(SOperatorInfo* pOperator, SSDataBlock** pResBlock); static int32_t setRowTsColumnOutputInfo(SqlFunctionCtx* pCtx, int32_t numOfCols, SArray** pResList); static int32_t setFunctionResultOutput(SOperatorInfo* pOperator, SOptrBasicInfo* pInfo, SAggSupporter* pSup, @@ -557,12 +555,6 @@ static void doHandleDataBlock(SOperatorInfo* pOperator, SSDataBlock* pBlock, SOp } } -SSDataBlock* doApplyIndefinitFunction1(SOperatorInfo* pOperator) { - SSDataBlock* pResBlock = NULL; - pOperator->pTaskInfo->code = doApplyIndefinitFunction(pOperator, &pResBlock); - return pResBlock; -} - int32_t doApplyIndefinitFunction(SOperatorInfo* pOperator, SSDataBlock** pResBlock) { QRY_PARAM_CHECK(pResBlock); SIndefOperatorInfo* pIndefInfo = pOperator->info; diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index 267dc88807..0528e336bc 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -878,7 +878,7 @@ static int32_t doStreamExecTask(SStreamTask* pTask) { } double el = (taosGetTimestampMs() - st) / 1000.0; - if (el > 5.0) { // elapsed more than 5 sec, not occupy the CPU anymore + if (el > 2.0) { // elapsed more than 5 sec, not occupy the CPU anymore stDebug("s-task:%s occupy more than 5.0s, release the exec threads and idle for 500ms", id); streamTaskSetIdleInfo(pTask, 500); return code; diff --git a/source/libs/stream/src/streamStartHistory.c b/source/libs/stream/src/streamStartHistory.c index 54a8929123..f8b1b5ecbc 100644 --- a/source/libs/stream/src/streamStartHistory.c +++ b/source/libs/stream/src/streamStartHistory.c @@ -76,7 +76,7 @@ int32_t streamStartScanHistoryAsync(SStreamTask* pTask, int8_t igUntreated) { memcpy(serializedReq, &req, len); SRpcMsg rpcMsg = {.contLen = len, .pCont = serializedReq, .msgType = TDMT_VND_STREAM_SCAN_HISTORY}; - return tmsgPutToQueue(pTask->pMsgCb, STREAM_QUEUE, &rpcMsg); + return tmsgPutToQueue(pTask->pMsgCb, STREAM_LONG_EXEC_QUEUE, &rpcMsg); } void streamExecScanHistoryInFuture(SStreamTask* pTask, int32_t idleDuration) { diff --git a/source/util/src/tworker.c b/source/util/src/tworker.c index dbd8cb159e..469f98fcf0 100644 --- a/source/util/src/tworker.c +++ b/source/util/src/tworker.c @@ -256,7 +256,7 @@ static void *tAutoQWorkerThreadFp(SQueueWorker *worker) { return NULL; } -STaosQueue *tAutoQWorkerAllocQueue(SAutoQWorkerPool *pool, void *ahandle, FItem fp) { +STaosQueue *tAutoQWorkerAllocQueue(SAutoQWorkerPool *pool, void *ahandle, FItem fp, int32_t minNum) { int32_t code; STaosQueue *queue; @@ -280,7 +280,10 @@ STaosQueue *tAutoQWorkerAllocQueue(SAutoQWorkerPool *pool, void *ahandle, FItem int32_t queueNum = taosGetQueueNumber(pool->qset); int32_t curWorkerNum = taosArrayGetSize(pool->workers); int32_t dstWorkerNum = ceilf(queueNum * pool->ratio); - if (dstWorkerNum < 2) dstWorkerNum = 2; + + if (dstWorkerNum < minNum) { + dstWorkerNum = minNum; + } // spawn a thread to process queue while (curWorkerNum < dstWorkerNum) { From 012ee53d356113a36eca43084ae8859ede166007 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sat, 15 Feb 2025 23:20:05 +0800 Subject: [PATCH 17/57] fix(stream): drop task in synch model in write thread. --- include/libs/stream/tstream.h | 2 +- source/dnode/vnode/src/tqCommon/tqCommon.c | 27 ++++++----------- source/libs/executor/src/executor.c | 34 +++++++++++----------- source/libs/stream/src/streamCheckpoint.c | 32 ++++++++++---------- source/libs/stream/src/streamMeta.c | 26 ++++++++--------- source/libs/stream/src/streamTask.c | 4 +-- 6 files changed, 59 insertions(+), 66 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index cc12566389..1f12b8addd 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -58,7 +58,7 @@ extern "C" { #define STREAM_EXEC_T_STOP_ALL_TASKS (-5) #define STREAM_EXEC_T_RESUME_TASK (-6) #define STREAM_EXEC_T_ADD_FAILED_TASK (-7) -#define STREAM_EXEC_T_DROP_ONE_TASK (-8) +#define STREAM_EXEC_T_STOP_ONE_TASK (-8) typedef struct SStreamTask SStreamTask; typedef struct SStreamQueue SStreamQueue; diff --git a/source/dnode/vnode/src/tqCommon/tqCommon.c b/source/dnode/vnode/src/tqCommon/tqCommon.c index 29b9141759..817868a1d6 100644 --- a/source/dnode/vnode/src/tqCommon/tqCommon.c +++ b/source/dnode/vnode/src/tqCommon/tqCommon.c @@ -720,40 +720,31 @@ int32_t tqStreamTaskProcessDropReq(SStreamMeta* pMeta, SMsgCb* cb, char* msg, in // drop the related fill-history task firstly if (hTaskId.taskId != 0 && hTaskId.streamId != 0) { - code = streamTaskSchedTask(cb, vgId, hTaskId.streamId, (int32_t)hTaskId.taskId, STREAM_EXEC_T_DROP_ONE_TASK); + tqDebug("s-task:0x%x vgId:%d drop rel fill-history task:0x%x firstly", pReq->taskId, vgId, (int32_t)hTaskId.taskId); + code = streamMetaUnregisterTask(pMeta, hTaskId.streamId, hTaskId.taskId); if (code) { - tqError("s-task:0x%x vgId:%d failed to create msg to drop rel fill-history task:0x%x, code:%s", pReq->taskId, - vgId, (int32_t)hTaskId.taskId, tstrerror(code)); - } else { - tqDebug("s-task:0x%x vgId:%d create msg to drop rel fill-history task:0x%x succ", pReq->taskId, vgId, + tqDebug("s-task:0x%x vgId:%d drop rel fill-history task:0x%x failed", pReq->taskId, vgId, (int32_t)hTaskId.taskId); } } // drop the stream task now - code = streamTaskSchedTask(cb, vgId, pReq->streamId, pReq->taskId, STREAM_EXEC_T_DROP_ONE_TASK); + code = streamMetaUnregisterTask(pMeta, pReq->streamId, pReq->taskId); if (code) { - tqError("s-task:0x%x vgId:%d failed to create msg to drop task, code:%s", pReq->taskId, vgId, tstrerror(code)); - } else { - tqDebug("s-task:0x%x vgId:%d create msg to drop succ", pReq->taskId, vgId); + tqDebug("s-task:0x%x vgId:%d drop task failed", pReq->taskId, vgId); } -// code = streamMetaUnregisterTask(pMeta, pReq->streamId, pReq->taskId); -// if (code) { -// tqDebug("s-task:0x%x vgId:%d drop task failed", pReq->taskId, vgId); -// } - // commit the update -// int32_t numOfTasks = streamMetaGetNumOfTasks(pMeta); -// tqDebug("vgId:%d task:0x%x dropped, remain tasks:%d", vgId, pReq->taskId, numOfTasks); + int32_t numOfTasks = streamMetaGetNumOfTasks(pMeta); + tqDebug("vgId:%d task:0x%x dropped, remain tasks:%d", vgId, pReq->taskId, numOfTasks); if (streamMetaCommit(pMeta) < 0) { // persist to disk } streamMetaWUnLock(pMeta); + tqDebug("vgId:%d process drop task:0x%x completed", vgId, pReq->taskId); - tqDebug("vgId:%d process drop task:0x%x async completed", vgId, pReq->taskId); return 0; // always return success } @@ -868,7 +859,7 @@ int32_t tqStreamTaskProcessRunReq(SStreamMeta* pMeta, SRpcMsg* pMsg, bool isLead } else if (type == STREAM_EXEC_T_ADD_FAILED_TASK) { code = streamMetaAddFailedTask(pMeta, req.streamId, req.taskId); return code; - } else if (type == STREAM_EXEC_T_DROP_ONE_TASK) { + } else if (type == STREAM_EXEC_T_STOP_ONE_TASK) { code = streamMetaDropTask(pMeta, req.streamId, req.taskId); return code; } else if (type == STREAM_EXEC_T_RESUME_TASK) { // task resume to run after idle for a while diff --git a/source/libs/executor/src/executor.c b/source/libs/executor/src/executor.c index 3d60e4d797..f271679e37 100644 --- a/source/libs/executor/src/executor.c +++ b/source/libs/executor/src/executor.c @@ -982,24 +982,24 @@ int32_t qKillTask(qTaskInfo_t tinfo, int32_t rspCode, int64_t waitingDuration) { qDebug("%s sync killed execTask, and waiting for %.2fs", GET_TASKID(pTaskInfo), waitingDuration/1000.0); setTaskKilled(pTaskInfo, TSDB_CODE_TSC_QUERY_KILLED); - while (1) { - taosWLockLatch(&pTaskInfo->lock); - if (qTaskIsExecuting(pTaskInfo)) { // let's wait for 100 ms and try again - taosWUnLockLatch(&pTaskInfo->lock); - - taosMsleep(200); - - int64_t d = taosGetTimestampMs() - st; - if (d >= waitingDuration && waitingDuration >= 0) { - qWarn("%s waiting more than %.2fs, not wait anymore", GET_TASKID(pTaskInfo), waitingDuration/1000.0); - return TSDB_CODE_SUCCESS; - } - } else { // not running now - pTaskInfo->code = rspCode; - taosWUnLockLatch(&pTaskInfo->lock); +// while (1) { +// taosWLockLatch(&pTaskInfo->lock); +// if (qTaskIsExecuting(pTaskInfo)) { // let's wait for 100 ms and try again +// taosWUnLockLatch(&pTaskInfo->lock); +// +// taosMsleep(200); +// +// int64_t d = taosGetTimestampMs() - st; +// if (d >= waitingDuration && waitingDuration >= 0) { +// qWarn("%s waiting more than %.2fs, not wait anymore", GET_TASKID(pTaskInfo), waitingDuration/1000.0); +// return TSDB_CODE_SUCCESS; +// } +// } else { // not running now +// pTaskInfo->code = rspCode; +// taosWUnLockLatch(&pTaskInfo->lock); return TSDB_CODE_SUCCESS; - } - } +// } +// } } bool qTaskIsExecuting(qTaskInfo_t qinfo) { diff --git a/source/libs/stream/src/streamCheckpoint.c b/source/libs/stream/src/streamCheckpoint.c index d62165a72c..a43cdd0b85 100644 --- a/source/libs/stream/src/streamCheckpoint.c +++ b/source/libs/stream/src/streamCheckpoint.c @@ -622,15 +622,19 @@ int32_t streamTaskUpdateTaskCheckpointInfo(SStreamTask* pTask, bool restored, SV pReq->transId); streamMutexUnlock(&pTask->lock); - { // destroy the related fill-history tasks - // drop task should not in the meta-lock, and drop the related fill-history task now - if (pReq->dropRelHTask) { - code = streamTaskSchedTask(pTask->pMsgCb, vgId, pReq->hStreamId, pReq->hTaskId, STREAM_EXEC_T_DROP_ONE_TASK); - if (code) { - stError("s-task:%s failed to create msg to drop related fill-history task, code:%s", id, tstrerror(code)); - } - } - } + { // destroy the related fill-history tasks + // drop task should not in the meta-lock, and drop the related fill-history task now + if (pReq->dropRelHTask) { + code = streamMetaUnregisterTask(pMeta, pReq->hStreamId, pReq->hTaskId); + int32_t numOfTasks = streamMetaGetNumOfTasks(pMeta); + stDebug("s-task:%s vgId:%d related fill-history task:0x%x dropped in update checkpointInfo, remain tasks:%d", + id, vgId, pReq->taskId, numOfTasks); + } + + if (pReq->dropRelHTask) { + code = streamMetaCommit(pMeta); + } + } // always return true return TSDB_CODE_SUCCESS; @@ -704,12 +708,10 @@ int32_t streamTaskUpdateTaskCheckpointInfo(SStreamTask* pTask, bool restored, SV // drop task should not in the meta-lock, and drop the related fill-history task now if (pReq->dropRelHTask) { - code = streamTaskSchedTask(pTask->pMsgCb, vgId, pReq->hStreamId, pReq->hTaskId, STREAM_EXEC_T_DROP_ONE_TASK); - if (code) { - stError("s-task:%s failed to create msg to drop related fill-history task, code:%s", id, tstrerror(code)); - } else { - stDebug("s-task:%s vgId:%d create msg to drop related fill-history task:0x%x", id, vgId, (int32_t)pReq->hTaskId); - } + code = streamMetaUnregisterTask(pMeta, pReq->hStreamId, pReq->hTaskId); + int32_t numOfTasks = streamMetaGetNumOfTasks(pMeta); + stDebug("s-task:%s vgId:%d related fill-history task:0x%x dropped, remain tasks:%d", id, vgId, + (int32_t)pReq->hTaskId, numOfTasks); } code = streamMetaCommit(pMeta); diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index ce1c228b7b..036cb43b12 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -965,7 +965,7 @@ int32_t streamMetaUnregisterTask(SStreamMeta* pMeta, int64_t streamId, int32_t t return 0; } -int32_t streamMetaDropTask(SStreamMeta* pMeta, int64_t streamId, int32_t taskId) { +int32_t streamMetaStopTask(SStreamMeta* pMeta, int64_t streamId, int32_t taskId) { SStreamTask* pTask = NULL; int32_t code = 0; int32_t vgId = pMeta->vgId; @@ -973,18 +973,18 @@ int32_t streamMetaDropTask(SStreamMeta* pMeta, int64_t streamId, int32_t taskId) streamMetaWLock(pMeta); - code = streamMetaUnregisterTask(pMeta, streamId, taskId); - numOfTasks = streamMetaGetNumOfTasks(pMeta); - if (code) { - stError("vgId:%d failed to drop task:0x%x, code:%s", vgId, taskId, tstrerror(code)); - } - - code = streamMetaCommit(pMeta); - if (code) { - stError("vgId:%d failed to commit after drop task:0x%x, code:%s", vgId, taskId, tstrerror(code)); - } else { - stDebug("s-task:0x%"PRIx64"-0x%x vgId:%d dropped, remain tasks:%d", streamId, taskId, pMeta->vgId, numOfTasks); - } +// code = streamMetaUnregisterTask(pMeta, streamId, taskId); +// numOfTasks = streamMetaGetNumOfTasks(pMeta); +// if (code) { +// stError("vgId:%d failed to drop task:0x%x, code:%s", vgId, taskId, tstrerror(code)); +// } +// +// code = streamMetaCommit(pMeta); +// if (code) { +// stError("vgId:%d failed to commit after drop task:0x%x, code:%s", vgId, taskId, tstrerror(code)); +// } else { +// stDebug("s-task:0x%"PRIx64"-0x%x vgId:%d dropped, remain tasks:%d", streamId, taskId, pMeta->vgId, numOfTasks); +// } streamMetaWUnLock(pMeta); diff --git a/source/libs/stream/src/streamTask.c b/source/libs/stream/src/streamTask.c index f0b288f096..e4e8a37b37 100644 --- a/source/libs/stream/src/streamTask.c +++ b/source/libs/stream/src/streamTask.c @@ -1287,8 +1287,8 @@ const char* streamTaskGetExecType(int32_t type) { return "resume-task-from-idle"; case STREAM_EXEC_T_ADD_FAILED_TASK: return "record-start-failed-task"; - case STREAM_EXEC_T_DROP_ONE_TASK: - return "drop-one-task"; + case STREAM_EXEC_T_STOP_ONE_TASK: + return "stop-one-task"; case 0: return "exec-all-tasks"; default: From bf1f162aa0fb0f1131e31c258c22e4cf28b08241 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 17 Feb 2025 10:06:09 +0800 Subject: [PATCH 18/57] fix(stream): drop task in synch model in write thread. --- include/dnode/vnode/tqCommon.h | 2 +- include/libs/executor/executor.h | 2 +- include/libs/stream/tstream.h | 2 +- source/dnode/snode/src/snode.c | 2 +- source/dnode/vnode/src/tq/tq.c | 2 +- source/dnode/vnode/src/tqCommon/tqCommon.c | 4 +- source/libs/executor/src/executor.c | 49 +++++++++++++--------- source/libs/stream/src/streamMeta.c | 4 +- source/libs/stream/src/streamStartTask.c | 1 - 9 files changed, 38 insertions(+), 30 deletions(-) diff --git a/include/dnode/vnode/tqCommon.h b/include/dnode/vnode/tqCommon.h index 69a11638b1..4d5e18520c 100644 --- a/include/dnode/vnode/tqCommon.h +++ b/include/dnode/vnode/tqCommon.h @@ -34,7 +34,7 @@ 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); -int32_t tqStreamTaskProcessDropReq(SStreamMeta* pMeta, SMsgCb* cb, char* msg, int32_t msgLen); +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); diff --git a/include/libs/executor/executor.h b/include/libs/executor/executor.h index 5b4224d575..988a650d6c 100644 --- a/include/libs/executor/executor.h +++ b/include/libs/executor/executor.h @@ -180,7 +180,7 @@ void qCleanExecTaskBlockBuf(qTaskInfo_t tinfo); */ int32_t qAsyncKillTask(qTaskInfo_t tinfo, int32_t rspCode); -int32_t qKillTask(qTaskInfo_t tinfo, int32_t rspCode, int64_t waiting); +int32_t qKillTask(qTaskInfo_t tinfo, int32_t rspCode, int64_t waitDuration); bool qTaskIsExecuting(qTaskInfo_t qinfo); diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 1f12b8addd..b7d560e060 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -760,7 +760,6 @@ int32_t streamMetaRemoveTaskInMeta(SStreamMeta* pMeta, STaskId* pKey); int32_t streamMetaRegisterTask(SStreamMeta* pMeta, int64_t ver, SStreamTask* pTask, bool* pAdded); int32_t streamMetaUnregisterTask(SStreamMeta* pMeta, int64_t streamId, int32_t taskId); int32_t streamMetaGetNumOfTasks(SStreamMeta* pMeta); -int32_t streamMetaDropTask(SStreamMeta* pMeta, int64_t streamId, int32_t taskId); int32_t streamMetaAcquireTaskNoLock(SStreamMeta* pMeta, int64_t streamId, int32_t taskId, SStreamTask** pTask); int32_t streamMetaAcquireTaskUnsafe(SStreamMeta* pMeta, STaskId* pId, SStreamTask** pTask); @@ -800,6 +799,7 @@ void streamMetaLoadAllTasks(SStreamMeta* pMeta); int32_t streamMetaStartAllTasks(SStreamMeta* pMeta); int32_t streamMetaStopAllTasks(SStreamMeta* pMeta); int32_t streamMetaStartOneTask(SStreamMeta* pMeta, int64_t streamId, int32_t taskId); +int32_t streamMetaStopOneTask(SStreamMeta* pMeta, int64_t streamId, int32_t taskId); bool streamMetaAllTasksReady(const SStreamMeta* pMeta); int32_t streamTaskSendNegotiateChkptIdMsg(SStreamTask* pTask); int32_t streamTaskCheckIfReqConsenChkptId(SStreamTask* pTask, int64_t ts); diff --git a/source/dnode/snode/src/snode.c b/source/dnode/snode/src/snode.c index 6899140af9..6eee8c510b 100644 --- a/source/dnode/snode/src/snode.c +++ b/source/dnode/snode/src/snode.c @@ -155,7 +155,7 @@ int32_t sndProcessWriteMsg(SSnode *pSnode, SRpcMsg *pMsg, SRpcMsg *pRsp) { } case TDMT_STREAM_TASK_DROP: - return tqStreamTaskProcessDropReq(pSnode->pMeta, &pSnode->msgCb, pMsg->pCont, pMsg->contLen); + return tqStreamTaskProcessDropReq(pSnode->pMeta, pMsg->pCont, pMsg->contLen); case TDMT_VND_STREAM_TASK_UPDATE: return tqStreamTaskProcessUpdateReq(pSnode->pMeta, &pSnode->msgCb, pMsg, true); case TDMT_VND_STREAM_TASK_RESET: diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index bd2a7207f2..1c0f73249c 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1118,7 +1118,7 @@ int32_t tqProcessTaskDispatchRsp(STQ* pTq, SRpcMsg* pMsg) { } int32_t tqProcessTaskDropReq(STQ* pTq, char* msg, int32_t msgLen) { - return tqStreamTaskProcessDropReq(pTq->pStreamMeta, &pTq->pVnode->msgCb, msg, msgLen); + return tqStreamTaskProcessDropReq(pTq->pStreamMeta, msg, msgLen); } int32_t tqProcessTaskUpdateCheckpointReq(STQ* pTq, char* msg, int32_t msgLen) { diff --git a/source/dnode/vnode/src/tqCommon/tqCommon.c b/source/dnode/vnode/src/tqCommon/tqCommon.c index 817868a1d6..76f0fd6bc6 100644 --- a/source/dnode/vnode/src/tqCommon/tqCommon.c +++ b/source/dnode/vnode/src/tqCommon/tqCommon.c @@ -688,7 +688,7 @@ int32_t tqStreamTaskProcessDeployReq(SStreamMeta* pMeta, SMsgCb* cb, int64_t sve return code; } -int32_t tqStreamTaskProcessDropReq(SStreamMeta* pMeta, SMsgCb* cb, char* msg, int32_t msgLen) { +int32_t tqStreamTaskProcessDropReq(SStreamMeta* pMeta, char* msg, int32_t msgLen) { SVDropStreamTaskReq* pReq = (SVDropStreamTaskReq*)msg; int32_t code = 0; int32_t vgId = pMeta->vgId; @@ -860,7 +860,7 @@ int32_t tqStreamTaskProcessRunReq(SStreamMeta* pMeta, SRpcMsg* pMsg, bool isLead code = streamMetaAddFailedTask(pMeta, req.streamId, req.taskId); return code; } else if (type == STREAM_EXEC_T_STOP_ONE_TASK) { - code = streamMetaDropTask(pMeta, req.streamId, req.taskId); + code = streamMetaStopOneTask(pMeta, req.streamId, req.taskId); return code; } else if (type == STREAM_EXEC_T_RESUME_TASK) { // task resume to run after idle for a while SStreamTask* pTask = NULL; diff --git a/source/libs/executor/src/executor.c b/source/libs/executor/src/executor.c index f271679e37..1aa114a02a 100644 --- a/source/libs/executor/src/executor.c +++ b/source/libs/executor/src/executor.c @@ -972,34 +972,43 @@ int32_t qAsyncKillTask(qTaskInfo_t qinfo, int32_t rspCode) { return TSDB_CODE_SUCCESS; } -int32_t qKillTask(qTaskInfo_t tinfo, int32_t rspCode, int64_t waitingDuration) { +int32_t qKillTask(qTaskInfo_t tinfo, int32_t rspCode, int64_t waitDuration) { int64_t st = taosGetTimestampMs(); SExecTaskInfo* pTaskInfo = (SExecTaskInfo*)tinfo; if (pTaskInfo == NULL) { return TSDB_CODE_QRY_INVALID_QHANDLE; } - qDebug("%s sync killed execTask, and waiting for %.2fs", GET_TASKID(pTaskInfo), waitingDuration/1000.0); + if (waitDuration > 0) { + qDebug("%s sync killed execTask, and waiting for %.2fs", GET_TASKID(pTaskInfo), waitDuration/1000.0); + } else { + qDebug("%s async killed execTask", GET_TASKID(pTaskInfo)); + } + setTaskKilled(pTaskInfo, TSDB_CODE_TSC_QUERY_KILLED); -// while (1) { -// taosWLockLatch(&pTaskInfo->lock); -// if (qTaskIsExecuting(pTaskInfo)) { // let's wait for 100 ms and try again -// taosWUnLockLatch(&pTaskInfo->lock); -// -// taosMsleep(200); -// -// int64_t d = taosGetTimestampMs() - st; -// if (d >= waitingDuration && waitingDuration >= 0) { -// qWarn("%s waiting more than %.2fs, not wait anymore", GET_TASKID(pTaskInfo), waitingDuration/1000.0); -// return TSDB_CODE_SUCCESS; -// } -// } else { // not running now -// pTaskInfo->code = rspCode; -// taosWUnLockLatch(&pTaskInfo->lock); - return TSDB_CODE_SUCCESS; -// } -// } + if (waitDuration > 0) { + while (1) { + taosWLockLatch(&pTaskInfo->lock); + if (qTaskIsExecuting(pTaskInfo)) { // let's wait for 100 ms and try again + taosWUnLockLatch(&pTaskInfo->lock); + + taosMsleep(200); + + int64_t d = taosGetTimestampMs() - st; + if (d >= waitDuration && waitDuration >= 0) { + qWarn("%s waiting more than %.2fs, not wait anymore", GET_TASKID(pTaskInfo), waitDuration / 1000.0); + return TSDB_CODE_SUCCESS; + } + } else { // not running now + pTaskInfo->code = rspCode; + taosWUnLockLatch(&pTaskInfo->lock); + return TSDB_CODE_SUCCESS; + } + } + } + + return TSDB_CODE_SUCCESS; } bool qTaskIsExecuting(qTaskInfo_t qinfo) { diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 036cb43b12..c5a5c70115 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -897,7 +897,7 @@ static int32_t streamTaskSendTransSuccessMsg(SStreamTask* pTask, void* param) { // let's kill the query procedure within stream, to end it ASAP. if (pTask->info.taskLevel != TASK_LEVEL__SINK && pTask->exec.pExecutor != NULL) { - code = qKillTask(pTask->exec.pExecutor, TSDB_CODE_SUCCESS, 5000); + code = qKillTask(pTask->exec.pExecutor, TSDB_CODE_SUCCESS, -1); if (code != TSDB_CODE_SUCCESS) { stError("s-task:%s failed to kill task related query handle, code:%s", pTask->id.idStr, tstrerror(code)); } @@ -965,7 +965,7 @@ int32_t streamMetaUnregisterTask(SStreamMeta* pMeta, int64_t streamId, int32_t t return 0; } -int32_t streamMetaStopTask(SStreamMeta* pMeta, int64_t streamId, int32_t taskId) { +int32_t streamMetaStopOneTask(SStreamMeta* pMeta, int64_t streamId, int32_t taskId) { SStreamTask* pTask = NULL; int32_t code = 0; int32_t vgId = pMeta->vgId; diff --git a/source/libs/stream/src/streamStartTask.c b/source/libs/stream/src/streamStartTask.c index 9c16ff036e..d9ca506849 100644 --- a/source/libs/stream/src/streamStartTask.c +++ b/source/libs/stream/src/streamStartTask.c @@ -447,7 +447,6 @@ int32_t streamMetaStopAllTasks(SStreamMeta* pMeta) { continue; } - int64_t refId = pTask->id.refId; int32_t ret = streamTaskStop(pTask); if (ret) { stError("s-task:0x%x failed to stop task, code:%s", pTaskId->taskId, tstrerror(ret)); From 899d2cef207f1c74be9e29c0331d199a7f8912b5 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 13 Feb 2025 23:41:06 +0800 Subject: [PATCH 19/57] fix(stream): not discard stream msg --- source/dnode/vnode/src/vnd/vnodeSvr.c | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/source/dnode/vnode/src/vnd/vnodeSvr.c b/source/dnode/vnode/src/vnd/vnodeSvr.c index f37dd94106..6562504988 100644 --- a/source/dnode/vnode/src/vnd/vnodeSvr.c +++ b/source/dnode/vnode/src/vnd/vnodeSvr.c @@ -931,7 +931,9 @@ int32_t vnodeProcessFetchMsg(SVnode *pVnode, SRpcMsg *pMsg, SQueueInfo *pInfo) { int32_t vnodeProcessStreamMsg(SVnode *pVnode, SRpcMsg *pMsg, SQueueInfo *pInfo) { vTrace("vgId:%d, msg:%p in stream queue is processing", pVnode->config.vgId, pMsg); - if (!syncIsReadyForRead(pVnode->sync)) { + if ((pMsg->msgType == TDMT_SCH_FETCH || pMsg->msgType == TDMT_VND_TABLE_META || pMsg->msgType == TDMT_VND_TABLE_CFG || + pMsg->msgType == TDMT_VND_BATCH_META) && + !syncIsReadyForRead(pVnode->sync)) { vnodeRedirectRpcMsg(pVnode, pMsg, terrno); return 0; } From 644d9b98bad8de922822b9651b19ea0f8b6cca9e Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 27 Feb 2025 00:46:42 +0800 Subject: [PATCH 20/57] refactor(stream): remove unused variables. --- source/libs/stream/src/streamMeta.c | 1 - 1 file changed, 1 deletion(-) diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index c5a5c70115..5d605c29a5 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -885,7 +885,6 @@ static void doRemoveIdFromList(SArray* pTaskList, int32_t num, SStreamTaskId* id static int32_t streamTaskSendTransSuccessMsg(SStreamTask* pTask, void* param) { int32_t code = 0; - int32_t waitingDuration = 5000; if (pTask->info.taskLevel == TASK_LEVEL__SOURCE) { code = streamTaskSendCheckpointSourceRsp(pTask); From 73e8720b0a8cbae2c90fb5874019dcfaca6ec493 Mon Sep 17 00:00:00 2001 From: WANG Xu Date: Wed, 26 Feb 2025 21:09:16 +0800 Subject: [PATCH 21/57] refactor: break RUN command into multiple lines Signed-off-by: WANG Xu --- packaging/docker/Dockerfile | 32 ++++++++++++++++++++++++++++---- 1 file changed, 28 insertions(+), 4 deletions(-) diff --git a/packaging/docker/Dockerfile b/packaging/docker/Dockerfile index e2ba30bca6..c0aadac376 100644 --- a/packaging/docker/Dockerfile +++ b/packaging/docker/Dockerfile @@ -8,18 +8,42 @@ ARG cpuType RUN echo ${pkgFile} && echo ${dirName} COPY ${pkgFile} /root/ + ENV TINI_VERSION v0.19.0 -ADD https://github.com/krallin/tini/releases/download/${TINI_VERSION}/tini-${cpuType} /tini ENV DEBIAN_FRONTEND=noninteractive -WORKDIR /root/ -RUN tar -zxf ${pkgFile} && cd /root/${dirName}/ && /bin/bash install.sh -e no && cd /root && rm /root/${pkgFile} && rm -rf /root/${dirName} && apt-get update && apt-get install -y locales tzdata netcat curl gdb vim tmux less net-tools valgrind rsync && locale-gen en_US.UTF-8 && apt-get clean && rm -rf /var/lib/apt/lists/ && chmod +x /tini + +ADD https://github.com/krallin/tini/releases/download/${TINI_VERSION}/tini-${cpuType} /tini +RUN chmod +x /tini + +RUN tar -zxf ${pkgFile} && \ + /bin/bash /root/${dirName}/install.sh -e no && \ + rm /root/${pkgFile} && \ + rm -rf /root/${dirName} && \ + apt-get update && \ + apt-get install -y --no-install-recommends \ + locales \ + tzdata \ + netcat \ + curl \ + gdb \ + vim \ + tmux \ + less \ + net-tools \ + valgrind \ + rsync && \ + apt-get clean && \ + rm -rf /var/lib/apt/lists/* && \ + locale-gen en_US.UTF-8 ENV LD_LIBRARY_PATH="$LD_LIBRARY_PATH:/usr/lib" \ LC_CTYPE=en_US.UTF-8 \ LANG=en_US.UTF-8 \ LC_ALL=en_US.UTF-8 + COPY ./bin/* /usr/bin/ ENTRYPOINT ["/tini", "--", "/usr/bin/entrypoint.sh"] CMD ["taosd"] -VOLUME [ "/var/lib/taos", "/var/log/taos", "/corefile" ] + +VOLUME [ "/var/lib/taos", "/var/log/taos", "/corefile" ] \ No newline at end of file From d279aaa22cd1111aa49db7fb63c99886bf67af94 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 27 Feb 2025 11:44:50 +0800 Subject: [PATCH 22/57] test(stream): sleep a little bit longer. --- tests/script/tsim/stream/tag.sim | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/script/tsim/stream/tag.sim b/tests/script/tsim/stream/tag.sim index f293f4ac05..9f4c62e747 100644 --- a/tests/script/tsim/stream/tag.sim +++ b/tests/script/tsim/stream/tag.sim @@ -26,7 +26,7 @@ sql insert into t1 values(1648791223000,0,1,1,1.0); sql insert into t1 values(1648791223001,9,2,2,1.1); sql insert into t1 values(1648791223009,0,3,3,1.0); -sleep 300 +sleep 1000 sql select * from streamt; if $data01 != 3 then From 7b56df0adce996fdf9697b2bc21841531ba79c56 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 27 Feb 2025 14:51:26 +0800 Subject: [PATCH 23/57] refactor(stream): create task epset update trans out of lock, to avoid blocking mnode-read thread. --- source/dnode/mnode/impl/src/mndStream.c | 134 ++++++++++++++---------- 1 file changed, 80 insertions(+), 54 deletions(-) diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index 55974eb9c7..220319122c 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -1901,15 +1901,16 @@ static int32_t mndProcessResumeStreamReq(SRpcMsg *pReq) { return TSDB_CODE_ACTION_IN_PROGRESS; } -static int32_t mndProcessVgroupChange(SMnode *pMnode, SVgroupChangeInfo *pChangeInfo, bool includeAllNodes) { +static int32_t mndProcessVgroupChange(SMnode *pMnode, SVgroupChangeInfo *pChangeInfo, bool includeAllNodes, STrans** pUpdateTrans) { SSdb *pSdb = pMnode->pSdb; - SStreamObj *pStream = NULL; void *pIter = NULL; STrans *pTrans = NULL; int32_t code = 0; + *pUpdateTrans = NULL; // conflict check for nodeUpdate trans, here we randomly chose one stream to add into the trans pool while (1) { + SStreamObj *pStream = NULL; pIter = sdbFetch(pSdb, SDB_STREAM, pIter, (void **)&pStream); if (pIter == NULL) { break; @@ -1926,6 +1927,7 @@ static int32_t mndProcessVgroupChange(SMnode *pMnode, SVgroupChangeInfo *pChange } while (1) { + SStreamObj *pStream = NULL; pIter = sdbFetch(pSdb, SDB_STREAM, pIter, (void **)&pStream); if (pIter == NULL) { break; @@ -1946,7 +1948,7 @@ static int32_t mndProcessVgroupChange(SMnode *pMnode, SVgroupChangeInfo *pChange void *p1 = taosHashGet(pChangeInfo->pDBMap, pStream->targetDb, strlen(pStream->targetDb)); void *p2 = taosHashGet(pChangeInfo->pDBMap, pStream->sourceDb, strlen(pStream->sourceDb)); if (p1 == NULL && p2 == NULL) { - mDebug("stream:0x%" PRIx64 " %s not involved nodeUpdate, ignore", pStream->uid, pStream->name); + mDebug("stream:0x%" PRIx64 " %s not involved in nodeUpdate, ignore", pStream->uid, pStream->name); sdbRelease(pSdb, pStream); continue; } @@ -1981,20 +1983,7 @@ static int32_t mndProcessVgroupChange(SMnode *pMnode, SVgroupChangeInfo *pChange } // no need to build the trans to handle the vgroup update - if (pTrans == NULL) { - return 0; - } - - code = mndTransPrepare(pMnode, pTrans); - if (code != TSDB_CODE_SUCCESS && code != TSDB_CODE_ACTION_IN_PROGRESS) { - mError("trans:%d, failed to prepare update stream trans since %s", pTrans->id, tstrerror(code)); - sdbRelease(pMnode->pSdb, pStream); - mndTransDrop(pTrans); - return code; - } - - sdbRelease(pMnode->pSdb, pStream); - mndTransDrop(pTrans); + *pUpdateTrans = pTrans; return code; } @@ -2076,7 +2065,7 @@ static int32_t refreshNodeListFromExistedStreams(SMnode *pMnode, SArray *pNodeLi taosHashCleanup(pHash); - mDebug("numOfNodes:%d for stream after extract nodeInfo from stream", (int32_t)taosArrayGetSize(pNodeList)); + mDebug("numOfvNodes:%d get after extracting nodeInfo from all streams", (int32_t)taosArrayGetSize(pNodeList)); return code; } @@ -2100,14 +2089,49 @@ static void addAllDbsIntoHashmap(SHashObj *pDBMap, SSdb *pSdb) { } } +static int32_t doProcessNodeCheckHelp(SArray *pNodeSnapshot, SMnode *pMnode, SVgroupChangeInfo *pChangeInfo, + bool *pUpdateAllVgroups) { + int32_t code = removeExpiredNodeEntryAndTaskInBuf(pNodeSnapshot); + if (code) { + mDebug("failed to remove expired node entry in buf, code:%s", tstrerror(code)); + return code; + } + + code = mndFindChangedNodeInfo(pMnode, execInfo.pNodeList, pNodeSnapshot, pChangeInfo); + if (code) { + mDebug("failed to find changed vnode(s) during vnode(s) check, code:%s", tstrerror(code)); + return code; + } + + { + if (execInfo.role == NODE_ROLE_LEADER && execInfo.switchFromFollower) { + mInfo("rollback all stream due to mnode leader/follower switch by using nodeUpdate trans"); + *pUpdateAllVgroups = true; + execInfo.switchFromFollower = false; // reset the flag + addAllDbsIntoHashmap(pChangeInfo->pDBMap, pMnode->pSdb); + } + } + + if (taosArrayGetSize(pChangeInfo->pUpdateNodeList) > 0 || (*pUpdateAllVgroups)) { + // kill current active checkpoint transaction, since the transaction is vnode wide. + killAllCheckpointTrans(pMnode, pChangeInfo); + } else { + mDebug("no update found in vnode(s) list"); + } + + return code; +} + // this function runs by only one thread, so it is not multi-thread safe static int32_t mndProcessNodeCheckReq(SRpcMsg *pMsg) { - int32_t code = 0; - bool allReady = true; - SArray *pNodeSnapshot = NULL; - SMnode *pMnode = pMsg->info.node; - int64_t ts = taosGetTimestampSec(); - bool updateAllVgroups = false; + int32_t code = 0; + bool allReady = true; + SArray *pNodeSnapshot = NULL; + SMnode *pMnode = pMsg->info.node; + int64_t tsms = taosGetTimestampMs(); + int64_t ts = tsms / 1000; + bool updateAllVgroups = false; + SVgroupChangeInfo changeInfo = {0}; int32_t old = atomic_val_compare_exchange_32(&mndNodeCheckSentinel, 0, 1); if (old != 0) { @@ -2115,7 +2139,7 @@ static int32_t mndProcessNodeCheckReq(SRpcMsg *pMsg) { return 0; } - mDebug("start to do node changing check"); + mDebug("start to do node changing check, ts:%" PRId64, tsms); streamMutexLock(&execInfo.lock); int32_t numOfNodes = extractStreamNodeList(pMnode); @@ -2141,58 +2165,60 @@ static int32_t mndProcessNodeCheckReq(SRpcMsg *pMsg) { } streamMutexLock(&execInfo.lock); + code = doProcessNodeCheckHelp(pNodeSnapshot, pMnode, &changeInfo, &updateAllVgroups); + streamMutexUnlock(&execInfo.lock); - code = removeExpiredNodeEntryAndTaskInBuf(pNodeSnapshot); if (code) { goto _end; } - SVgroupChangeInfo changeInfo = {0}; - code = mndFindChangedNodeInfo(pMnode, execInfo.pNodeList, pNodeSnapshot, &changeInfo); - if (code) { - goto _end; - } - - { - if (execInfo.role == NODE_ROLE_LEADER && execInfo.switchFromFollower) { - mInfo("rollback all stream due to mnode leader/follower switch by using nodeUpdate trans"); - updateAllVgroups = true; - execInfo.switchFromFollower = false; // reset the flag - addAllDbsIntoHashmap(changeInfo.pDBMap, pMnode->pSdb); - } - } - if (taosArrayGetSize(changeInfo.pUpdateNodeList) > 0 || updateAllVgroups) { - // kill current active checkpoint transaction, since the transaction is vnode wide. - killAllCheckpointTrans(pMnode, &changeInfo); - code = mndProcessVgroupChange(pMnode, &changeInfo, updateAllVgroups); + mDebug("vnode(s) change detected, build trans to update stream task epsets"); + + STrans *pTrans = NULL; + + streamMutexLock(&execInfo.lock); + code = mndProcessVgroupChange(pMnode, &changeInfo, updateAllVgroups, &pTrans); + streamMutexUnlock(&execInfo.lock); + + // NOTE: sync trans out of lock + if (code == 0 && pTrans != NULL) { + code = mndTransPrepare(pMnode, pTrans); + if (code != TSDB_CODE_SUCCESS && code != TSDB_CODE_ACTION_IN_PROGRESS) { + mError("trans:%d, failed to prepare update stream trans since %s", pTrans->id, tstrerror(code)); + } + + mndTransDrop(pTrans); + } // keep the new vnode snapshot if success if (code == TSDB_CODE_SUCCESS || code == TSDB_CODE_ACTION_IN_PROGRESS) { + streamMutexLock(&execInfo.lock); + code = refreshNodeListFromExistedStreams(pMnode, execInfo.pNodeList); + int32_t num = (int)taosArrayGetSize(execInfo.pNodeList); + if (code == 0) { + execInfo.ts = ts; + mDebug("create trans successfully, update cached node list, numOfNodes:%d", num); + } + + streamMutexUnlock(&execInfo.lock); + if (code) { mError("failed to extract node list from stream, code:%s", tstrerror(code)); goto _end; } - - execInfo.ts = ts; - mDebug("create trans successfully, update cached node list, numOfNodes:%d", - (int)taosArrayGetSize(execInfo.pNodeList)); - } else { - mError("unexpected code during create nodeUpdate trans, code:%s", tstrerror(code)); } - } else { - mDebug("no update found in nodeList"); } mndDestroyVgroupChangeInfo(&changeInfo); _end: - streamMutexUnlock(&execInfo.lock); taosArrayDestroy(pNodeSnapshot); - mDebug("end to do stream task node change checking"); + mDebug("end to do stream task node change checking, elapsed time:%" PRId64 "ms", taosGetTimestampMs() - tsms); atomic_store_32(&mndNodeCheckSentinel, 0); + return 0; } From 6bfa1acaeba5e00df253fd80ea5fbcfaf8bc47da Mon Sep 17 00:00:00 2001 From: dmchen Date: Wed, 26 Feb 2025 16:55:06 +0800 Subject: [PATCH 24/57] fix/remove-sync-heartbeat-lock --- source/libs/sync/inc/syncPipeline.h | 1 + source/libs/sync/src/syncPipeline.c | 16 +++++++++++++--- 2 files changed, 14 insertions(+), 3 deletions(-) diff --git a/source/libs/sync/inc/syncPipeline.h b/source/libs/sync/inc/syncPipeline.h index eeb24d2f16..147f8a67ae 100644 --- a/source/libs/sync/inc/syncPipeline.h +++ b/source/libs/sync/inc/syncPipeline.h @@ -40,6 +40,7 @@ typedef struct SSyncLogReplMgr { int32_t retryBackoff; int32_t peerId; int32_t sendCount; + TdThreadMutex mutex; } SSyncLogReplMgr; typedef struct SSyncLogBufEntry { diff --git a/source/libs/sync/src/syncPipeline.c b/source/libs/sync/src/syncPipeline.c index 18252db9ee..ceaba1e643 100644 --- a/source/libs/sync/src/syncPipeline.c +++ b/source/libs/sync/src/syncPipeline.c @@ -1140,26 +1140,29 @@ int32_t syncLogReplRecover(SSyncLogReplMgr* pMgr, SSyncNode* pNode, SyncAppendEn int32_t syncLogReplProcessHeartbeatReply(SSyncLogReplMgr* pMgr, SSyncNode* pNode, SyncHeartbeatReply* pMsg) { SSyncLogBuffer* pBuf = pNode->pLogBuf; - (void)taosThreadMutexLock(&pBuf->mutex); + (void)taosThreadMutexLock(&pMgr->mutex); if (pMsg->startTime != 0 && pMsg->startTime != pMgr->peerStartTime) { sInfo("vgId:%d, reset sync log repl in heartbeat. peer:%" PRIx64 ", start time:%" PRId64 ", old:%" PRId64 "", pNode->vgId, pMsg->srcId.addr, pMsg->startTime, pMgr->peerStartTime); syncLogReplReset(pMgr); pMgr->peerStartTime = pMsg->startTime; } - (void)taosThreadMutexUnlock(&pBuf->mutex); + (void)taosThreadMutexUnlock(&pMgr->mutex); return 0; } int32_t syncLogReplProcessReply(SSyncLogReplMgr* pMgr, SSyncNode* pNode, SyncAppendEntriesReply* pMsg) { SSyncLogBuffer* pBuf = pNode->pLogBuf; - (void)taosThreadMutexLock(&pBuf->mutex); + (void)taosThreadMutexLock(&pMgr->mutex); if (pMsg->startTime != pMgr->peerStartTime) { sInfo("vgId:%d, reset sync log repl in appendlog reply. peer:%" PRIx64 ", start time:%" PRId64 ", old:%" PRId64, pNode->vgId, pMsg->srcId.addr, pMsg->startTime, pMgr->peerStartTime); syncLogReplReset(pMgr); pMgr->peerStartTime = pMsg->startTime; } + (void)taosThreadMutexUnlock(&pMgr->mutex); + + (void)taosThreadMutexLock(&pBuf->mutex); int32_t code = 0; if (pMgr->restored) { @@ -1324,6 +1327,12 @@ SSyncLogReplMgr* syncLogReplCreate() { return NULL; } + int32_t code = taosThreadMutexInit(&pMgr->mutex, NULL); + if (code) { + terrno = code; + return NULL; + } + return pMgr; } @@ -1331,6 +1340,7 @@ void syncLogReplDestroy(SSyncLogReplMgr* pMgr) { if (pMgr == NULL) { return; } + taosThreadMutexDestroy(&pMgr->mutex); taosMemoryFree(pMgr); return; } From 4dcb984b022a8c198697e5809138dd56d4aa65af Mon Sep 17 00:00:00 2001 From: dmchen Date: Wed, 26 Feb 2025 18:41:17 +0800 Subject: [PATCH 25/57] fix/remove-sync-heartbeat-lock-fix-check --- source/libs/sync/src/syncPipeline.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/libs/sync/src/syncPipeline.c b/source/libs/sync/src/syncPipeline.c index ceaba1e643..8d81a03344 100644 --- a/source/libs/sync/src/syncPipeline.c +++ b/source/libs/sync/src/syncPipeline.c @@ -1340,7 +1340,7 @@ void syncLogReplDestroy(SSyncLogReplMgr* pMgr) { if (pMgr == NULL) { return; } - taosThreadMutexDestroy(&pMgr->mutex); + (void)taosThreadMutexDestroy(&pMgr->mutex); taosMemoryFree(pMgr); return; } From 1f6c4df4222508f90e64bee771a413be401c1683 Mon Sep 17 00:00:00 2001 From: dmchen Date: Thu, 27 Feb 2025 11:02:18 +0800 Subject: [PATCH 26/57] fix/move-status-to-new-queue --- include/common/tmsgcb.h | 1 + source/dnode/mgmt/mgmt_dnode/src/dmHandle.c | 7 ++++--- source/dnode/mgmt/mgmt_mnode/inc/mmInt.h | 2 ++ source/dnode/mgmt/mgmt_mnode/src/mmHandle.c | 2 +- source/dnode/mgmt/mgmt_mnode/src/mmWorker.c | 20 +++++++++++++++++++ source/dnode/mgmt/mgmt_qnode/src/qmWorker.c | 1 + source/dnode/mnode/impl/test/trans/trans2.cpp | 3 ++- source/libs/sync/inc/syncUtil.h | 4 ++-- 8 files changed, 33 insertions(+), 7 deletions(-) diff --git a/include/common/tmsgcb.h b/include/common/tmsgcb.h index c934cb6961..56f1219b1b 100644 --- a/include/common/tmsgcb.h +++ b/include/common/tmsgcb.h @@ -31,6 +31,7 @@ typedef enum { QUERY_QUEUE, FETCH_QUEUE, READ_QUEUE, + STATUS_QUEUE, WRITE_QUEUE, APPLY_QUEUE, SYNC_QUEUE, diff --git a/source/dnode/mgmt/mgmt_dnode/src/dmHandle.c b/source/dnode/mgmt/mgmt_dnode/src/dmHandle.c index 637713d2f9..ef0106cdbb 100644 --- a/source/dnode/mgmt/mgmt_dnode/src/dmHandle.c +++ b/source/dnode/mgmt/mgmt_dnode/src/dmHandle.c @@ -267,7 +267,7 @@ void dmSendStatusReq(SDnodeMgmt *pMgmt) { code = rpcSendRecvWithTimeout(pMgmt->msgCb.statusRpc, &epSet, &rpcMsg, &rpcRsp, &epUpdated, tsStatusInterval * 5 * 1000); if (code != 0) { - dError("failed to send status req since %s", tstrerror(code)); + dError("failed to SendRecv with timeout %d status req since %s", tsStatusInterval * 5 * 1000, tstrerror(code)); return; } @@ -275,7 +275,8 @@ void dmSendStatusReq(SDnodeMgmt *pMgmt) { dmRotateMnodeEpSet(pMgmt->pData); char tbuf[512]; dmEpSetToStr(tbuf, sizeof(tbuf), &epSet); - dError("failed to send status req since %s, epSet:%s, inUse:%d", tstrerror(rpcRsp.code), tbuf, epSet.inUse); + dInfo("Rotate mnode ep set since failed to SendRecv status req %s, epSet:%s, inUse:%d", tstrerror(rpcRsp.code), + tbuf, epSet.inUse); } else { if (epUpdated == 1) { dmSetMnodeEpSet(pMgmt->pData, &epSet); @@ -403,7 +404,7 @@ void dmSendConfigReq(SDnodeMgmt *pMgmt) { code = rpcSendRecvWithTimeout(pMgmt->msgCb.statusRpc, &epSet, &rpcMsg, &rpcRsp, &epUpdated, tsStatusInterval * 5 * 1000); if (code != 0) { - dError("failed to send status req since %s", tstrerror(code)); + dError("failed to SendRecv config req with timeout %d since %s", tsStatusInterval * 5 * 1000, tstrerror(code)); return; } if (rpcRsp.code != 0) { diff --git a/source/dnode/mgmt/mgmt_mnode/inc/mmInt.h b/source/dnode/mgmt/mgmt_mnode/inc/mmInt.h index bc2977fe31..2d7aa35c13 100644 --- a/source/dnode/mgmt/mgmt_mnode/inc/mmInt.h +++ b/source/dnode/mgmt/mgmt_mnode/inc/mmInt.h @@ -32,6 +32,7 @@ typedef struct SMnodeMgmt { SSingleWorker queryWorker; SSingleWorker fetchWorker; SSingleWorker readWorker; + SSingleWorker statusWorker; SSingleWorker writeWorker; SSingleWorker arbWorker; SSingleWorker syncWorker; @@ -58,6 +59,7 @@ int32_t mmPutMsgToArbQueue(SMnodeMgmt *pMgmt, SRpcMsg *pMsg); int32_t mmPutMsgToSyncQueue(SMnodeMgmt *pMgmt, SRpcMsg *pMsg); int32_t mmPutMsgToSyncRdQueue(SMnodeMgmt *pMgmt, SRpcMsg *pMsg); int32_t mmPutMsgToReadQueue(SMnodeMgmt *pMgmt, SRpcMsg *pMsg); +int32_t mmPutMsgToStatusQueue(SMnodeMgmt *pMgmt, SRpcMsg *pMsg); int32_t mmPutMsgToQueryQueue(SMnodeMgmt *pMgmt, SRpcMsg *pMsg); int32_t mmPutMsgToFetchQueue(SMnodeMgmt *pMgmt, SRpcMsg *pMsg); int32_t mmPutMsgToQueue(SMnodeMgmt *pMgmt, EQueueType qtype, SRpcMsg *pRpc); diff --git a/source/dnode/mgmt/mgmt_mnode/src/mmHandle.c b/source/dnode/mgmt/mgmt_mnode/src/mmHandle.c index 80ef0d31de..0031042497 100644 --- a/source/dnode/mgmt/mgmt_mnode/src/mmHandle.c +++ b/source/dnode/mgmt/mgmt_mnode/src/mmHandle.c @@ -202,7 +202,7 @@ SArray *mmGetMsgHandles() { if (dmSetMgmtHandle(pArray, TDMT_MND_KILL_QUERY, mmPutMsgToWriteQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_MND_KILL_CONN, mmPutMsgToWriteQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_MND_HEARTBEAT, mmPutMsgToReadQueue, 0) == NULL) goto _OVER; - if (dmSetMgmtHandle(pArray, TDMT_MND_STATUS, mmPutMsgToReadQueue, 0) == NULL) goto _OVER; + if (dmSetMgmtHandle(pArray, TDMT_MND_STATUS, mmPutMsgToStatusQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_MND_CONFIG, mmPutMsgToWriteQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_MND_NOTIFY, mmPutMsgToReadQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_MND_SYSTABLE_RETRIEVE, mmPutMsgToReadQueue, 0) == NULL) goto _OVER; diff --git a/source/dnode/mgmt/mgmt_mnode/src/mmWorker.c b/source/dnode/mgmt/mgmt_mnode/src/mmWorker.c index cfdc09f29e..cb82ddae2e 100644 --- a/source/dnode/mgmt/mgmt_mnode/src/mmWorker.c +++ b/source/dnode/mgmt/mgmt_mnode/src/mmWorker.c @@ -133,6 +133,10 @@ int32_t mmPutMsgToReadQueue(SMnodeMgmt *pMgmt, SRpcMsg *pMsg) { return mmPutMsgToWorker(pMgmt, &pMgmt->readWorker, pMsg); } +int32_t mmPutMsgToStatusQueue(SMnodeMgmt *pMgmt, SRpcMsg *pMsg) { + return mmPutMsgToWorker(pMgmt, &pMgmt->statusWorker, pMsg); +} + int32_t mmPutMsgToQueryQueue(SMnodeMgmt *pMgmt, SRpcMsg *pMsg) { int32_t code = 0; if (NULL == pMgmt->pMnode) { @@ -172,6 +176,9 @@ int32_t mmPutMsgToQueue(SMnodeMgmt *pMgmt, EQueueType qtype, SRpcMsg *pRpc) { case READ_QUEUE: pWorker = &pMgmt->readWorker; break; + case STATUS_QUEUE: + pWorker = &pMgmt->statusWorker; + break; case ARB_QUEUE: pWorker = &pMgmt->arbWorker; break; @@ -246,6 +253,18 @@ int32_t mmStartWorker(SMnodeMgmt *pMgmt) { return code; } + SSingleWorkerCfg stautsCfg = { + .min = 1, + .max = 1, + .name = "mnode-status", + .fp = (FItem)mmProcessRpcMsg, + .param = pMgmt, + }; + if ((code = tSingleWorkerInit(&pMgmt->statusWorker, &stautsCfg)) != 0) { + dError("failed to start mnode-status worker since %s", tstrerror(code)); + return code; + } + SSingleWorkerCfg wCfg = { .min = 1, .max = 1, @@ -304,6 +323,7 @@ void mmStopWorker(SMnodeMgmt *pMgmt) { tSingleWorkerCleanup(&pMgmt->queryWorker); tSingleWorkerCleanup(&pMgmt->fetchWorker); tSingleWorkerCleanup(&pMgmt->readWorker); + tSingleWorkerCleanup(&pMgmt->statusWorker); tSingleWorkerCleanup(&pMgmt->writeWorker); tSingleWorkerCleanup(&pMgmt->arbWorker); tSingleWorkerCleanup(&pMgmt->syncWorker); diff --git a/source/dnode/mgmt/mgmt_qnode/src/qmWorker.c b/source/dnode/mgmt/mgmt_qnode/src/qmWorker.c index ab5b70079a..950da46a40 100644 --- a/source/dnode/mgmt/mgmt_qnode/src/qmWorker.c +++ b/source/dnode/mgmt/mgmt_qnode/src/qmWorker.c @@ -71,6 +71,7 @@ int32_t qmPutRpcMsgToQueue(SQnodeMgmt *pMgmt, EQueueType qtype, SRpcMsg *pRpc) { code = taosWriteQitem(pMgmt->queryWorker.queue, pMsg); return code; case READ_QUEUE: + case STATUS_QUEUE: case FETCH_QUEUE: dTrace("msg:%p, is created and will put into qnode-fetch queue, len:%d", pMsg, pRpc->contLen); code = taosWriteQitem(pMgmt->fetchWorker.queue, pMsg); diff --git a/source/dnode/mnode/impl/test/trans/trans2.cpp b/source/dnode/mnode/impl/test/trans/trans2.cpp index b73f07c778..e520fce111 100644 --- a/source/dnode/mnode/impl/test/trans/trans2.cpp +++ b/source/dnode/mnode/impl/test/trans/trans2.cpp @@ -65,7 +65,8 @@ class MndTestTrans2 : public ::testing::Test { msgCb.sendRspFp = sendRsp; msgCb.queueFps[SYNC_QUEUE] = putToQueue; msgCb.queueFps[WRITE_QUEUE] = putToQueue; - msgCb.queueFps[READ_QUEUE] = putToQueue; + msgCb.queueFps[READ_QUEUE] = putToQueue; + msgCb.queueFps[STATUS_QUEUE] = putToQueue; msgCb.mgmt = (SMgmtWrapper *)(&msgCb); // hack tmsgSetDefault(&msgCb); diff --git a/source/libs/sync/inc/syncUtil.h b/source/libs/sync/inc/syncUtil.h index 7b71491f47..e7110c0964 100644 --- a/source/libs/sync/inc/syncUtil.h +++ b/source/libs/sync/inc/syncUtil.h @@ -50,8 +50,8 @@ extern "C" { #define sNError(pNode, ...) if (sDebugFlag & DEBUG_ERROR) { syncPrintNodeLog("SYN ERROR ", DEBUG_ERROR, 255, true, pNode, __VA_ARGS__); } #define sNWarn(pNode, ...) if (sDebugFlag & DEBUG_WARN) { syncPrintNodeLog("SYN WARN ", DEBUG_WARN, 255, true, pNode, __VA_ARGS__); } #define sNInfo(pNode, ...) if (sDebugFlag & DEBUG_INFO) { syncPrintNodeLog("SYN ", DEBUG_INFO, 255, true, pNode, __VA_ARGS__); } -#define sNDebug(pNode, ...) if (sDebugFlag & DEBUG_DEBUG) { syncPrintNodeLog("SYN ", DEBUG_DEBUG, sDebugFlag, false, pNode, __VA_ARGS__); } -#define sNTrace(pNode, ...) if (sDebugFlag & DEBUG_TRACE) { syncPrintNodeLog("SYN ", DEBUG_TRACE, sDebugFlag, false, pNode, __VA_ARGS__); } +#define sNDebug(pNode, ...) if (sDebugFlag & DEBUG_DEBUG) { syncPrintNodeLog("SYN ", DEBUG_DEBUG, sDebugFlag, true, pNode, __VA_ARGS__); } +#define sNTrace(pNode, ...) if (sDebugFlag & DEBUG_TRACE) { syncPrintNodeLog("SYN ", DEBUG_TRACE, sDebugFlag, true, pNode, __VA_ARGS__); } #define sSFatal(pSender, ...) if (sDebugFlag & DEBUG_FATAL) { syncPrintSnapshotSenderLog("SYN FATAL ", DEBUG_FATAL, 255, pSender, __VA_ARGS__); } #define sSError(pSender, ...) if (sDebugFlag & DEBUG_ERROR) { syncPrintSnapshotSenderLog("SYN ERROR ", DEBUG_ERROR, 255, pSender, __VA_ARGS__); } From 7eeb800213a95c57504e27196117267643771552 Mon Sep 17 00:00:00 2001 From: dmchen Date: Thu, 27 Feb 2025 15:22:33 +0800 Subject: [PATCH 27/57] fix:add-status-timestamp --- include/common/tmsg.h | 1 + include/common/ttime.h | 2 +- source/common/src/msg/tmsg.c | 6 +++++ source/common/src/ttime.c | 28 +++++++++++++++++++++ source/dnode/mgmt/mgmt_dnode/src/dmHandle.c | 1 + source/dnode/mnode/impl/src/mndDnode.c | 8 ++++-- source/libs/sync/src/syncUtil.c | 2 +- 7 files changed, 44 insertions(+), 4 deletions(-) diff --git a/include/common/tmsg.h b/include/common/tmsg.h index 0b6a8b3f1b..38fdc52513 100644 --- a/include/common/tmsg.h +++ b/include/common/tmsg.h @@ -1839,6 +1839,7 @@ typedef struct { int32_t statusSeq; int64_t ipWhiteVer; int64_t analVer; + int64_t timestamp; } SStatusReq; int32_t tSerializeSStatusReq(void* buf, int32_t bufLen, SStatusReq* pReq); diff --git a/include/common/ttime.h b/include/common/ttime.h index 47875752d5..77a3041b2b 100644 --- a/include/common/ttime.h +++ b/include/common/ttime.h @@ -82,7 +82,7 @@ int32_t convertStringToTimestamp(int16_t type, char* inputData, int64_t timePrec int32_t getDuration(int64_t val, char unit, int64_t* result, int32_t timePrecision); int32_t taosFormatUtcTime(char* buf, int32_t bufLen, int64_t ts, int32_t precision); - +char* formatTimestampLocal(char* buf, int64_t val, int precision); struct STm { struct tm tm; int64_t fsec; // in NANOSECOND diff --git a/source/common/src/msg/tmsg.c b/source/common/src/msg/tmsg.c index f1ebcb3f42..cc635885c0 100644 --- a/source/common/src/msg/tmsg.c +++ b/source/common/src/msg/tmsg.c @@ -1471,6 +1471,8 @@ int32_t tSerializeSStatusReq(void *buf, int32_t bufLen, SStatusReq *pReq) { TAOS_CHECK_EXIT(tEncodeI64(&encoder, pReq->analVer)); TAOS_CHECK_EXIT(tSerializeSMonitorParas(&encoder, &pReq->clusterCfg.monitorParas)); + TAOS_CHECK_EXIT(tEncodeI64(&encoder, pReq->timestamp)); + tEndEncode(&encoder); _exit: @@ -1600,6 +1602,10 @@ int32_t tDeserializeSStatusReq(void *buf, int32_t bufLen, SStatusReq *pReq) { TAOS_CHECK_EXIT(tDeserializeSMonitorParas(&decoder, &pReq->clusterCfg.monitorParas)); } + if (!tDecodeIsEnd(&decoder)) { + TAOS_CHECK_EXIT(tDecodeI64(&decoder, &pReq->timestamp)); + } + tEndDecode(&decoder); _exit: diff --git a/source/common/src/ttime.c b/source/common/src/ttime.c index 9746fea034..22ab106bb3 100644 --- a/source/common/src/ttime.c +++ b/source/common/src/ttime.c @@ -1000,6 +1000,34 @@ int32_t taosFormatUtcTime(char* buf, int32_t bufLen, int64_t t, int32_t precisio TAOS_RETURN(TSDB_CODE_SUCCESS); } +char* formatTimestampLocal(char* buf, int64_t val, int precision) { + time_t tt; + if (precision == TSDB_TIME_PRECISION_MICRO) { + tt = (time_t)(val / 1000000); + } + if (precision == TSDB_TIME_PRECISION_NANO) { + tt = (time_t)(val / 1000000000); + } else { + tt = (time_t)(val / 1000); + } + + struct tm tm; + if (taosLocalTime(&tt, &tm, NULL, 0, NULL) == NULL) { + return NULL; + } + size_t pos = taosStrfTime(buf, 32, "%Y-%m-%d %H:%M:%S", &tm); + + if (precision == TSDB_TIME_PRECISION_MICRO) { + sprintf(buf + pos, ".%06d", (int)(val % 1000000)); + } else if (precision == TSDB_TIME_PRECISION_NANO) { + sprintf(buf + pos, ".%09d", (int)(val % 1000000000)); + } else { + sprintf(buf + pos, ".%03d", (int)(val % 1000)); + } + + return buf; +} + int32_t taosTs2Tm(int64_t ts, int32_t precision, struct STm* tm, timezone_t tz) { tm->fsec = ts % TICK_PER_SECOND[precision] * (TICK_PER_SECOND[TSDB_TIME_PRECISION_NANO] / TICK_PER_SECOND[precision]); time_t t = ts / TICK_PER_SECOND[precision]; diff --git a/source/dnode/mgmt/mgmt_dnode/src/dmHandle.c b/source/dnode/mgmt/mgmt_dnode/src/dmHandle.c index ef0106cdbb..653a54592a 100644 --- a/source/dnode/mgmt/mgmt_dnode/src/dmHandle.c +++ b/source/dnode/mgmt/mgmt_dnode/src/dmHandle.c @@ -167,6 +167,7 @@ static void dmProcessStatusRsp(SDnodeMgmt *pMgmt, SRpcMsg *pRsp) { void dmSendStatusReq(SDnodeMgmt *pMgmt) { int32_t code = 0; SStatusReq req = {0}; + req.timestamp = taosGetTimestampMs(); dDebug("send status req to mnode, statusSeq:%d, begin to mgnt lock", pMgmt->statusSeq); (void)taosThreadRwlockRdlock(&pMgmt->pData->lock); diff --git a/source/dnode/mnode/impl/src/mndDnode.c b/source/dnode/mnode/impl/src/mndDnode.c index ca119191eb..dac38dc4db 100644 --- a/source/dnode/mnode/impl/src/mndDnode.c +++ b/source/dnode/mnode/impl/src/mndDnode.c @@ -756,8 +756,12 @@ static int32_t mndProcessStatusReq(SRpcMsg *pReq) { bool needCheck = !online || dnodeChanged || reboot || supportVnodesChanged || analVerChanged || pMnode->ipWhiteVer != statusReq.ipWhiteVer || encryptKeyChanged || enableWhiteListChanged; const STraceId *trace = &pReq->info.traceId; - mGTrace("dnode:%d, status received, accessTimes:%d check:%d online:%d reboot:%d changed:%d statusSeq:%d", pDnode->id, - pDnode->accessTimes, needCheck, online, reboot, dnodeChanged, statusReq.statusSeq); + char timestamp[TD_TIME_STR_LEN] = {0}; + if (mDebugFlag & DEBUG_TRACE) (void)formatTimestampLocal(timestamp, statusReq.timestamp, TSDB_TIME_PRECISION_MILLI); + mGTrace( + "dnode:%d, status received, accessTimes:%d check:%d online:%d reboot:%d changed:%d statusSeq:%d " + "timestamp:%s", + pDnode->id, pDnode->accessTimes, needCheck, online, reboot, dnodeChanged, statusReq.statusSeq, timestamp); if (reboot) { tsGrantHBInterval = GRANT_HEART_BEAT_MIN; diff --git a/source/libs/sync/src/syncUtil.c b/source/libs/sync/src/syncUtil.c index 38fd660cfd..5a2571a291 100644 --- a/source/libs/sync/src/syncUtil.c +++ b/source/libs/sync/src/syncUtil.c @@ -116,7 +116,7 @@ static void syncPrintTime(bool formatTime, int32_t* len, int64_t tsMs, int32_t i if (formatTime) { char pBuf[TD_TIME_STR_LEN] = {0}; if (tsMs > 0) { - if (taosFormatUtcTime(pBuf, TD_TIME_STR_LEN, tsMs, TSDB_TIME_PRECISION_MILLI) != 0) { + if (formatTimestampLocal(pBuf, tsMs, TSDB_TIME_PRECISION_MILLI) == NULL) { pBuf[0] = '\0'; } } From 4dd0622594de8c37905b0a7bccebcbdad7cc5f3b Mon Sep 17 00:00:00 2001 From: dmchen Date: Thu, 27 Feb 2025 19:05:06 +0800 Subject: [PATCH 28/57] fix:decouple get mload --- source/dnode/mgmt/mgmt_dnode/src/dmHandle.c | 23 ++++++++++++++------- source/dnode/mnode/impl/src/mndMain.c | 1 + 2 files changed, 17 insertions(+), 7 deletions(-) diff --git a/source/dnode/mgmt/mgmt_dnode/src/dmHandle.c b/source/dnode/mgmt/mgmt_dnode/src/dmHandle.c index 653a54592a..49f634443f 100644 --- a/source/dnode/mgmt/mgmt_dnode/src/dmHandle.c +++ b/source/dnode/mgmt/mgmt_dnode/src/dmHandle.c @@ -25,6 +25,7 @@ extern SConfig *tsCfg; SMonVloadInfo tsVinfo = {0}; +SMnodeLoad tsMLoad = {0}; static void dmUpdateDnodeCfg(SDnodeMgmt *pMgmt, SDnodeCfg *pCfg) { int32_t code = 0; @@ -208,23 +209,22 @@ void dmSendStatusReq(SDnodeMgmt *pMgmt) { memcpy(req.clusterCfg.charset, tsCharset, TD_LOCALE_LEN); (void)taosThreadRwlockUnlock(&pMgmt->pData->lock); - dDebug("send status req to mnode, statusSeq:%d, begin to get vnode loads", pMgmt->statusSeq); + dDebug("send status req to mnode, statusSeq:%d, begin to get vnode and loads", pMgmt->statusSeq); if (taosThreadMutexLock(&pMgmt->pData->statusInfolock) != 0) { dError("failed to lock status info lock"); return; } + req.pVloads = tsVinfo.pVloads; tsVinfo.pVloads = NULL; + + req.mload = tsMLoad; + if (taosThreadMutexUnlock(&pMgmt->pData->statusInfolock) != 0) { dError("failed to unlock status info lock"); return; } - dDebug("send status req to mnode, statusSeq:%d, begin to get mnode loads", pMgmt->statusSeq); - SMonMloadInfo minfo = {0}; - (*pMgmt->getMnodeLoadsFp)(&minfo); - req.mload = minfo.load; - dDebug("send status req to mnode, statusSeq:%d, begin to get qnode loads", pMgmt->statusSeq); (*pMgmt->getQnodeLoadsFp)(&req.qload); @@ -418,12 +418,18 @@ void dmSendConfigReq(SDnodeMgmt *pMgmt) { void dmUpdateStatusInfo(SDnodeMgmt *pMgmt) { SMonVloadInfo vinfo = {0}; dDebug("begin to get vnode loads"); - (*pMgmt->getVnodeLoadsFp)(&vinfo); + (*pMgmt->getVnodeLoadsFp)(&vinfo); // dmGetVnodeLoads + + dDebug("begin to get mnode loads"); + SMonMloadInfo minfo = {0}; + (*pMgmt->getMnodeLoadsFp)(&minfo); // dmGetMnodeLoads + dDebug("begin to lock status info"); if (taosThreadMutexLock(&pMgmt->pData->statusInfolock) != 0) { dError("failed to lock status info lock"); return; } + if (tsVinfo.pVloads == NULL) { tsVinfo.pVloads = vinfo.pVloads; vinfo.pVloads = NULL; @@ -431,6 +437,9 @@ void dmUpdateStatusInfo(SDnodeMgmt *pMgmt) { taosArrayDestroy(vinfo.pVloads); vinfo.pVloads = NULL; } + + tsMLoad = minfo.load; + if (taosThreadMutexUnlock(&pMgmt->pData->statusInfolock) != 0) { dError("failed to unlock status info lock"); return; diff --git a/source/dnode/mnode/impl/src/mndMain.c b/source/dnode/mnode/impl/src/mndMain.c index b241af5adb..09314c9e63 100644 --- a/source/dnode/mnode/impl/src/mndMain.c +++ b/source/dnode/mnode/impl/src/mndMain.c @@ -1174,6 +1174,7 @@ int32_t mndGetMonitorInfo(SMnode *pMnode, SMonClusterInfo *pClusterInfo, SMonVgr } int32_t mndGetLoad(SMnode *pMnode, SMnodeLoad *pLoad) { + mTrace("mnode get load"); SSyncState state = syncGetState(pMnode->syncMgmt.sync); pLoad->syncState = state.state; pLoad->syncRestore = state.restored; From b6707fbbd207ff64c5721f7c2681db0d49e1a6ad Mon Sep 17 00:00:00 2001 From: dmchen Date: Wed, 26 Feb 2025 16:55:06 +0800 Subject: [PATCH 29/57] refactor(stream): move the generate checkpoint procedure out of lock, to avoid blocking heartbeat, and resulting in leader/follower switch. --- source/libs/stream/src/streamCheckpoint.c | 5 ++++- source/libs/stream/src/streamExec.c | 4 +++- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/source/libs/stream/src/streamCheckpoint.c b/source/libs/stream/src/streamCheckpoint.c index c53e1a19a3..41e3d40542 100644 --- a/source/libs/stream/src/streamCheckpoint.c +++ b/source/libs/stream/src/streamCheckpoint.c @@ -910,9 +910,12 @@ int32_t streamTaskBuildCheckpoint(SStreamTask* pTask) { int64_t startTs = pTask->chkInfo.startTs; int64_t ckId = pTask->chkInfo.pActiveInfo->activeId; const char* id = pTask->id.idStr; - bool dropRelHTask = (streamTaskGetPrevStatus(pTask) == TASK_STATUS__HALT); SStreamMeta* pMeta = pTask->pMeta; + streamMutexLock(&pTask->lock); + bool dropRelHTask = (streamTaskGetPrevStatus(pTask) == TASK_STATUS__HALT); + streamMutexUnlock(&pTask->lock); + // sink task does not need to save the status, and generated the checkpoint if (pTask->info.taskLevel != TASK_LEVEL__SINK) { stDebug("s-task:%s level:%d start gen checkpoint, checkpointId:%" PRId64, id, pTask->info.taskLevel, ckId); diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index 267dc88807..999a6b5a4c 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -698,6 +698,8 @@ static int32_t doHandleChkptBlock(SStreamTask* pTask) { streamMutexLock(&pTask->lock); SStreamTaskState pState = streamTaskGetStatus(pTask); + streamMutexUnlock(&pTask->lock); + if (pState.state == TASK_STATUS__CK) { // todo other thread may change the status stDebug("s-task:%s checkpoint block received, set status:%s", id, pState.name); code = streamTaskBuildCheckpoint(pTask); // ignore this error msg, and continue @@ -715,7 +717,7 @@ static int32_t doHandleChkptBlock(SStreamTask* pTask) { } } - streamMutexUnlock(&pTask->lock); +// streamMutexUnlock(&pTask->lock); return code; } From ebcdb1292f7cd9591c66cb39ea53d50330362482 Mon Sep 17 00:00:00 2001 From: haoranchen Date: Fri, 28 Feb 2025 11:38:07 +0800 Subject: [PATCH 30/57] fix: update error link in 02-database.md --- docs/zh/14-reference/03-taos-sql/02-database.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/zh/14-reference/03-taos-sql/02-database.md b/docs/zh/14-reference/03-taos-sql/02-database.md index 35cb99a3dd..3742b7c571 100644 --- a/docs/zh/14-reference/03-taos-sql/02-database.md +++ b/docs/zh/14-reference/03-taos-sql/02-database.md @@ -67,7 +67,7 @@ database_option: { - DURATION:数据文件存储数据的时间跨度。可以使用加单位的表示形式,如 DURATION 100h、DURATION 10d 等,支持 m(分钟)、h(小时)和 d(天)三个单位。不加时间单位时默认单位为天,如 DURATION 50 表示 50 天。 - MAXROWS:文件块中记录的最大条数,默认为 4096 条。 - MINROWS:文件块中记录的最小条数,默认为 100 条。 -- KEEP:表示数据文件保存的天数,缺省值为 3650,取值范围 [1, 365000],且必须大于或等于 3 倍的 DURATION 参数值。数据库会自动删除保存时间超过 KEEP 值的数据从而释放存储空间。KEEP 可以使用加单位的表示形式,如 KEEP 100h、KEEP 10d 等,支持 m(分钟)、h(小时)和 d(天)三个单位。也可以不写单位,如 KEEP 50,此时默认单位为天。企业版支持[多级存储](../../operation/planning/#%E5%A4%9A%E7%BA%A7%E5%AD%98%E5%82%A8)功能, 因此, 可以设置多个保存时间(多个以英文逗号分隔,最多 3 个,满足 keep 0 \<= keep 1 \<= keep 2,如 KEEP 100h,100d,3650d); 社区版不支持多级存储功能(即使配置了多个保存时间, 也不会生效, KEEP 会取最大的保存时间)。了解更多,请点击 [关于主键时间戳](https://docs.taosdata.com/reference/taos-sql/insert/) +- KEEP:表示数据文件保存的天数,缺省值为 3650,取值范围 [1, 365000],且必须大于或等于 3 倍的 DURATION 参数值。数据库会自动删除保存时间超过 KEEP 值的数据从而释放存储空间。KEEP 可以使用加单位的表示形式,如 KEEP 100h、KEEP 10d 等,支持 m(分钟)、h(小时)和 d(天)三个单位。也可以不写单位,如 KEEP 50,此时默认单位为天。企业版支持[多级存储](https://docs.taosdata.com/operation/planning/#%E5%A4%9A%E7%BA%A7%E5%AD%98%E5%82%A8)功能, 因此, 可以设置多个保存时间(多个以英文逗号分隔,最多 3 个,满足 keep 0 \<= keep 1 \<= keep 2,如 KEEP 100h,100d,3650d); 社区版不支持多级存储功能(即使配置了多个保存时间, 也不会生效, KEEP 会取最大的保存时间)。了解更多,请点击 [关于主键时间戳](https://docs.taosdata.com/reference/taos-sql/insert/) - KEEP_TIME_OFFSET:自 3.2.0.0 版本生效。删除或迁移保存时间超过 KEEP 值的数据的延迟执行时间,默认值为 0 (小时)。在数据文件保存时间超过 KEEP 后,删除或迁移操作不会立即执行,而会额外等待本参数指定的时间间隔,以实现与业务高峰期错开的目的。 - STT_TRIGGER:表示落盘文件触发文件合并的个数。对于少表高频写入场景,此参数建议使用默认配置;而对于多表低频写入场景,此参数建议配置较大的值。 From aa7cb880bb9b3c285f1eade915decf3d60d39a85 Mon Sep 17 00:00:00 2001 From: dmchen Date: Fri, 28 Feb 2025 11:52:27 +0800 Subject: [PATCH 31/57] fix/remove-get-queue-size-when-print-heartbeat-log --- source/dnode/mgmt/mgmt_vnode/inc/vmInt.h | 2 +- source/dnode/mgmt/mgmt_vnode/src/vmFile.c | 18 +++++------ source/dnode/mgmt/mgmt_vnode/src/vmHandle.c | 12 ++++---- source/dnode/mgmt/mgmt_vnode/src/vmInt.c | 34 ++++++++++----------- source/libs/sync/src/syncUtil.c | 30 +++++++++++------- source/libs/transport/src/tmsgcb.c | 2 +- 6 files changed, 53 insertions(+), 45 deletions(-) diff --git a/source/dnode/mgmt/mgmt_vnode/inc/vmInt.h b/source/dnode/mgmt/mgmt_vnode/inc/vmInt.h index 84f5149624..3d415e63c6 100644 --- a/source/dnode/mgmt/mgmt_vnode/inc/vmInt.h +++ b/source/dnode/mgmt/mgmt_vnode/inc/vmInt.h @@ -39,7 +39,7 @@ typedef struct SVnodeMgmt { SHashObj *runngingHash; SHashObj *closedHash; SHashObj *creatingHash; - TdThreadRwlock lock; + TdThreadRwlock hashLock; TdThreadMutex mutex; SVnodesStat state; STfs *pTfs; diff --git a/source/dnode/mgmt/mgmt_vnode/src/vmFile.c b/source/dnode/mgmt/mgmt_vnode/src/vmFile.c index dbef048c23..cb14155b1c 100644 --- a/source/dnode/mgmt/mgmt_vnode/src/vmFile.c +++ b/source/dnode/mgmt/mgmt_vnode/src/vmFile.c @@ -20,7 +20,7 @@ #define MAX_CONTENT_LEN 2 * 1024 * 1024 int32_t vmGetAllVnodeListFromHash(SVnodeMgmt *pMgmt, int32_t *numOfVnodes, SVnodeObj ***ppVnodes) { - (void)taosThreadRwlockRdlock(&pMgmt->lock); + (void)taosThreadRwlockRdlock(&pMgmt->hashLock); int32_t num = 0; int32_t size = taosHashGetSize(pMgmt->runngingHash); @@ -28,7 +28,7 @@ int32_t vmGetAllVnodeListFromHash(SVnodeMgmt *pMgmt, int32_t *numOfVnodes, SVnod size += closedSize; SVnodeObj **pVnodes = taosMemoryCalloc(size, sizeof(SVnodeObj *)); if (pVnodes == NULL) { - (void)taosThreadRwlockUnlock(&pMgmt->lock); + (void)taosThreadRwlockUnlock(&pMgmt->hashLock); return terrno; } @@ -60,7 +60,7 @@ int32_t vmGetAllVnodeListFromHash(SVnodeMgmt *pMgmt, int32_t *numOfVnodes, SVnod } } - (void)taosThreadRwlockUnlock(&pMgmt->lock); + (void)taosThreadRwlockUnlock(&pMgmt->hashLock); *numOfVnodes = num; *ppVnodes = pVnodes; @@ -68,7 +68,7 @@ int32_t vmGetAllVnodeListFromHash(SVnodeMgmt *pMgmt, int32_t *numOfVnodes, SVnod } int32_t vmGetAllVnodeListFromHashWithCreating(SVnodeMgmt *pMgmt, int32_t *numOfVnodes, SVnodeObj ***ppVnodes) { - (void)taosThreadRwlockRdlock(&pMgmt->lock); + (void)taosThreadRwlockRdlock(&pMgmt->hashLock); int32_t num = 0; int32_t size = taosHashGetSize(pMgmt->runngingHash); @@ -76,7 +76,7 @@ int32_t vmGetAllVnodeListFromHashWithCreating(SVnodeMgmt *pMgmt, int32_t *numOfV size += creatingSize; SVnodeObj **pVnodes = taosMemoryCalloc(size, sizeof(SVnodeObj *)); if (pVnodes == NULL) { - (void)taosThreadRwlockUnlock(&pMgmt->lock); + (void)taosThreadRwlockUnlock(&pMgmt->hashLock); return terrno; } @@ -107,7 +107,7 @@ int32_t vmGetAllVnodeListFromHashWithCreating(SVnodeMgmt *pMgmt, int32_t *numOfV taosHashCancelIterate(pMgmt->creatingHash, pIter); } } - (void)taosThreadRwlockUnlock(&pMgmt->lock); + (void)taosThreadRwlockUnlock(&pMgmt->hashLock); *numOfVnodes = num; *ppVnodes = pVnodes; @@ -116,13 +116,13 @@ int32_t vmGetAllVnodeListFromHashWithCreating(SVnodeMgmt *pMgmt, int32_t *numOfV } int32_t vmGetVnodeListFromHash(SVnodeMgmt *pMgmt, int32_t *numOfVnodes, SVnodeObj ***ppVnodes) { - (void)taosThreadRwlockRdlock(&pMgmt->lock); + (void)taosThreadRwlockRdlock(&pMgmt->hashLock); int32_t num = 0; int32_t size = taosHashGetSize(pMgmt->runngingHash); SVnodeObj **pVnodes = taosMemoryCalloc(size, sizeof(SVnodeObj *)); if (pVnodes == NULL) { - (void)taosThreadRwlockUnlock(&pMgmt->lock); + (void)taosThreadRwlockUnlock(&pMgmt->hashLock); return terrno; } @@ -140,7 +140,7 @@ int32_t vmGetVnodeListFromHash(SVnodeMgmt *pMgmt, int32_t *numOfVnodes, SVnodeOb } } - (void)taosThreadRwlockUnlock(&pMgmt->lock); + (void)taosThreadRwlockUnlock(&pMgmt->hashLock); *numOfVnodes = num; *ppVnodes = pVnodes; diff --git a/source/dnode/mgmt/mgmt_vnode/src/vmHandle.c b/source/dnode/mgmt/mgmt_vnode/src/vmHandle.c index 234d4f41e1..846b76d3ce 100644 --- a/source/dnode/mgmt/mgmt_vnode/src/vmHandle.c +++ b/source/dnode/mgmt/mgmt_vnode/src/vmHandle.c @@ -25,7 +25,7 @@ void vmGetVnodeLoads(SVnodeMgmt *pMgmt, SMonVloadInfo *pInfo, bool isReset) { tfsUpdateSize(pMgmt->pTfs); - (void)taosThreadRwlockRdlock(&pMgmt->lock); + (void)taosThreadRwlockRdlock(&pMgmt->hashLock); void *pIter = taosHashIterate(pMgmt->runngingHash, NULL); while (pIter) { @@ -46,14 +46,14 @@ void vmGetVnodeLoads(SVnodeMgmt *pMgmt, SMonVloadInfo *pInfo, bool isReset) { pIter = taosHashIterate(pMgmt->runngingHash, pIter); } - (void)taosThreadRwlockUnlock(&pMgmt->lock); + (void)taosThreadRwlockUnlock(&pMgmt->hashLock); } void vmGetVnodeLoadsLite(SVnodeMgmt *pMgmt, SMonVloadInfo *pInfo) { pInfo->pVloads = taosArrayInit(pMgmt->state.totalVnodes, sizeof(SVnodeLoadLite)); if (!pInfo->pVloads) return; - (void)taosThreadRwlockRdlock(&pMgmt->lock); + (void)taosThreadRwlockRdlock(&pMgmt->hashLock); void *pIter = taosHashIterate(pMgmt->runngingHash, NULL); while (pIter) { @@ -74,7 +74,7 @@ void vmGetVnodeLoadsLite(SVnodeMgmt *pMgmt, SMonVloadInfo *pInfo) { pIter = taosHashIterate(pMgmt->runngingHash, pIter); } - (void)taosThreadRwlockUnlock(&pMgmt->lock); + (void)taosThreadRwlockUnlock(&pMgmt->hashLock); } void vmGetMonitorInfo(SVnodeMgmt *pMgmt, SMonVmInfo *pInfo) { @@ -137,7 +137,7 @@ void vmCleanExpriedSamples(SVnodeMgmt *pMgmt) { dError("failed to get vgroup ids"); return; } - (void)taosThreadRwlockRdlock(&pMgmt->lock); + (void)taosThreadRwlockRdlock(&pMgmt->hashLock); for (int i = 0; i < list_size; i++) { int32_t vgroup_id = vgroup_ids[i]; void *vnode = taosHashGet(pMgmt->runngingHash, &vgroup_id, sizeof(int32_t)); @@ -148,7 +148,7 @@ void vmCleanExpriedSamples(SVnodeMgmt *pMgmt) { } } } - (void)taosThreadRwlockUnlock(&pMgmt->lock); + (void)taosThreadRwlockUnlock(&pMgmt->hashLock); if (vgroup_ids) taosMemoryFree(vgroup_ids); if (keys) taosMemoryFree(keys); return; diff --git a/source/dnode/mgmt/mgmt_vnode/src/vmInt.c b/source/dnode/mgmt/mgmt_vnode/src/vmInt.c index d71e0b02c4..f4925c348c 100644 --- a/source/dnode/mgmt/mgmt_vnode/src/vmInt.c +++ b/source/dnode/mgmt/mgmt_vnode/src/vmInt.c @@ -24,12 +24,12 @@ int32_t vmGetPrimaryDisk(SVnodeMgmt *pMgmt, int32_t vgId) { int32_t diskId = -1; SVnodeObj *pVnode = NULL; - (void)taosThreadRwlockRdlock(&pMgmt->lock); + (void)taosThreadRwlockRdlock(&pMgmt->hashLock); int32_t r = taosHashGetDup(pMgmt->runngingHash, &vgId, sizeof(int32_t), (void *)&pVnode); if (pVnode != NULL) { diskId = pVnode->diskPrimary; } - (void)taosThreadRwlockUnlock(&pMgmt->lock); + (void)taosThreadRwlockUnlock(&pMgmt->hashLock); return diskId; } @@ -62,7 +62,7 @@ static int32_t vmRegisterCreatingState(SVnodeMgmt *pMgmt, int32_t vgId, int32_t pCreatingVnode->vgId = vgId; pCreatingVnode->diskPrimary = diskId; - code = taosThreadRwlockWrlock(&pMgmt->lock); + code = taosThreadRwlockWrlock(&pMgmt->hashLock); if (code != 0) { taosMemoryFree(pCreatingVnode); return code; @@ -75,7 +75,7 @@ static int32_t vmRegisterCreatingState(SVnodeMgmt *pMgmt, int32_t vgId, int32_t taosMemoryFree(pCreatingVnode); } - int32_t r = taosThreadRwlockUnlock(&pMgmt->lock); + int32_t r = taosThreadRwlockUnlock(&pMgmt->hashLock); if (r != 0) { dError("vgId:%d, failed to unlock since %s", vgId, tstrerror(r)); } @@ -86,7 +86,7 @@ static int32_t vmRegisterCreatingState(SVnodeMgmt *pMgmt, int32_t vgId, int32_t static void vmUnRegisterCreatingState(SVnodeMgmt *pMgmt, int32_t vgId) { SVnodeObj *pOld = NULL; - (void)taosThreadRwlockWrlock(&pMgmt->lock); + (void)taosThreadRwlockWrlock(&pMgmt->hashLock); int32_t r = taosHashGetDup(pMgmt->creatingHash, &vgId, sizeof(int32_t), (void *)&pOld); if (r != 0) { dError("vgId:%d, failed to get vnode from creating Hash", vgId); @@ -96,7 +96,7 @@ static void vmUnRegisterCreatingState(SVnodeMgmt *pMgmt, int32_t vgId) { if (r != 0) { dError("vgId:%d, failed to remove vnode from creatingHash", vgId); } - (void)taosThreadRwlockUnlock(&pMgmt->lock); + (void)taosThreadRwlockUnlock(&pMgmt->hashLock); if (pOld) { dTrace("vgId:%d, free vnode pOld:%p", vgId, &pOld); @@ -205,7 +205,7 @@ void vmCleanPrimaryDisk(SVnodeMgmt *pMgmt, int32_t vgId) { vmUnRegisterCreatingS SVnodeObj *vmAcquireVnodeImpl(SVnodeMgmt *pMgmt, int32_t vgId, bool strict) { SVnodeObj *pVnode = NULL; - (void)taosThreadRwlockRdlock(&pMgmt->lock); + (void)taosThreadRwlockRdlock(&pMgmt->hashLock); int32_t r = taosHashGetDup(pMgmt->runngingHash, &vgId, sizeof(int32_t), (void *)&pVnode); if (pVnode == NULL || strict && (pVnode->dropped || pVnode->failed)) { terrno = TSDB_CODE_VND_INVALID_VGROUP_ID; @@ -214,7 +214,7 @@ SVnodeObj *vmAcquireVnodeImpl(SVnodeMgmt *pMgmt, int32_t vgId, bool strict) { int32_t refCount = atomic_add_fetch_32(&pVnode->refCount, 1); dTrace("vgId:%d, acquire vnode, vnode:%p, ref:%d", pVnode->vgId, pVnode, refCount); } - (void)taosThreadRwlockUnlock(&pMgmt->lock); + (void)taosThreadRwlockUnlock(&pMgmt->hashLock); return pVnode; } @@ -334,10 +334,10 @@ int32_t vmOpenVnode(SVnodeMgmt *pMgmt, SWrapperCfg *pCfg, SVnode *pImpl) { pVnode->failed = 1; } - (void)taosThreadRwlockWrlock(&pMgmt->lock); + (void)taosThreadRwlockWrlock(&pMgmt->hashLock); int32_t code = vmRegisterRunningState(pMgmt, pVnode); vmUnRegisterClosedState(pMgmt, pVnode); - (void)taosThreadRwlockUnlock(&pMgmt->lock); + (void)taosThreadRwlockUnlock(&pMgmt->hashLock); return code; } @@ -350,15 +350,15 @@ void vmCloseVnode(SVnodeMgmt *pMgmt, SVnodeObj *pVnode, bool commitAndRemoveWal, vnodeProposeCommitOnNeed(pVnode->pImpl, atExit); } - (void)taosThreadRwlockWrlock(&pMgmt->lock); + (void)taosThreadRwlockWrlock(&pMgmt->hashLock); vmUnRegisterRunningState(pMgmt, pVnode->vgId); if (keepClosed) { if (vmRegisterClosedState(pMgmt, pVnode) != 0) { - (void)taosThreadRwlockUnlock(&pMgmt->lock); + (void)taosThreadRwlockUnlock(&pMgmt->hashLock); return; }; } - (void)taosThreadRwlockUnlock(&pMgmt->lock); + (void)taosThreadRwlockUnlock(&pMgmt->hashLock); vmReleaseVnode(pMgmt, pVnode); @@ -450,14 +450,14 @@ _closed: void vmCloseFailedVnode(SVnodeMgmt *pMgmt, int32_t vgId) { int32_t r = 0; - r = taosThreadRwlockWrlock(&pMgmt->lock); + r = taosThreadRwlockWrlock(&pMgmt->hashLock); if (r != 0) { dError("vgId:%d, failed to lock since %s", vgId, tstrerror(r)); } if (r == 0) { vmUnRegisterRunningState(pMgmt, vgId); } - r = taosThreadRwlockUnlock(&pMgmt->lock); + r = taosThreadRwlockUnlock(&pMgmt->hashLock); if (r != 0) { dError("vgId:%d, failed to unlock since %s", vgId, tstrerror(r)); } @@ -792,7 +792,7 @@ static void vmCleanup(SVnodeMgmt *pMgmt) { vmCloseVnodes(pMgmt); vmStopWorker(pMgmt); vnodeCleanup(); - (void)taosThreadRwlockDestroy(&pMgmt->lock); + (void)taosThreadRwlockDestroy(&pMgmt->hashLock); (void)taosThreadMutexDestroy(&pMgmt->mutex); (void)taosThreadMutexDestroy(&pMgmt->fileLock); taosMemoryFree(pMgmt); @@ -880,7 +880,7 @@ static int32_t vmInit(SMgmtInputOpt *pInput, SMgmtOutputOpt *pOutput) { pMgmt->msgCb.qsizeFp = (GetQueueSizeFp)vmGetQueueSize; pMgmt->msgCb.mgmt = pMgmt; - code = taosThreadRwlockInit(&pMgmt->lock, NULL); + code = taosThreadRwlockInit(&pMgmt->hashLock, NULL); if (code != 0) { code = TAOS_SYSTEM_ERROR(errno); goto _OVER; diff --git a/source/libs/sync/src/syncUtil.c b/source/libs/sync/src/syncUtil.c index 5a2571a291..44e8d97aca 100644 --- a/source/libs/sync/src/syncUtil.c +++ b/source/libs/sync/src/syncUtil.c @@ -215,7 +215,7 @@ void syncPrintNodeLog(const char* flags, ELogLevel level, int32_t dflag, bool fo SSnapshot snapshot = {.data = NULL, .lastApplyIndex = -1, .lastApplyTerm = 0}; if (pNode->pFsm != NULL && pNode->pFsm->FpGetSnapshotInfo != NULL) { - (void)pNode->pFsm->FpGetSnapshotInfo(pNode->pFsm, &snapshot); + (void)pNode->pFsm->FpGetSnapshotInfo(pNode->pFsm, &snapshot); // vnodeSyncGetSnapshotInfo } SyncIndex logLastIndex = SYNC_INDEX_INVALID; @@ -253,13 +253,15 @@ void syncPrintNodeLog(const char* flags, ELogLevel level, int32_t dflag, bool fo va_end(argpointer); int32_t aqItems = 0; + /* if (pNode != NULL && pNode->pFsm != NULL && pNode->pFsm->FpApplyQueueItems != NULL) { - aqItems = pNode->pFsm->FpApplyQueueItems(pNode->pFsm); + aqItems = pNode->pFsm->FpApplyQueueItems(pNode->pFsm); // vnodeApplyQueueItems } + */ // restore error code terrno = errCode; - SyncIndex appliedIndex = pNode->pFsm->FpAppliedIndexCb(pNode->pFsm); + SyncIndex appliedIndex = pNode->pFsm->FpAppliedIndexCb(pNode->pFsm); // vnodeSyncAppliedIndex if (pNode != NULL) { taosPrintLog( @@ -426,19 +428,25 @@ void syncLogSendHeartbeat(SSyncNode* pSyncNode, const SyncHeartbeat* pMsg, bool } void syncLogRecvHeartbeat(SSyncNode* pSyncNode, const SyncHeartbeat* pMsg, int64_t timeDiff, const char* s) { + char pBuf[TD_TIME_STR_LEN] = {0}; + if (pMsg->timeStamp > 0) { + if (formatTimestampLocal(pBuf, pMsg->timeStamp, TSDB_TIME_PRECISION_MILLI) == NULL) { + pBuf[0] = '\0'; + } + } if (timeDiff > SYNC_HEARTBEAT_SLOW_MS) { pSyncNode->hbSlowNum++; sNTrace(pSyncNode, - "recv sync-heartbeat from dnode:%d slow {term:%" PRId64 ", commit-index:%" PRId64 ", min-match:%" PRId64 - ", ts:%" PRId64 "}, QID:%s, net elapsed:%" PRId64, - DID(&pMsg->srcId), pMsg->term, pMsg->commitIndex, pMsg->minMatchIndex, pMsg->timeStamp, s, timeDiff); + "recv sync-heartbeat from dnode:%d slow(%d ms) {term:%" PRId64 ", commit-index:%" PRId64 + ", min-match:%" PRId64 ", ts:%s}, QID:%s, net elapsed:%" PRId64 "ms", + DID(&pMsg->srcId), SYNC_HEARTBEAT_SLOW_MS, pMsg->term, pMsg->commitIndex, pMsg->minMatchIndex, pBuf, s, timeDiff); + } else { + sNTrace(pSyncNode, + "recv sync-heartbeat from dnode:%d {term:%" PRId64 ", commit-index:%" PRId64 ", min-match:%" PRId64 + ", ts:%s}, QID:%s, net elapsed:%" PRId64 "ms", + DID(&pMsg->srcId), pMsg->term, pMsg->commitIndex, pMsg->minMatchIndex, pBuf, s, timeDiff); } - - sNTrace(pSyncNode, - "recv sync-heartbeat from dnode:%d {term:%" PRId64 ", commit-index:%" PRId64 ", min-match:%" PRId64 - ", ts:%" PRId64 "}, QID:%s, net elapsed:%" PRId64, - DID(&pMsg->srcId), pMsg->term, pMsg->commitIndex, pMsg->minMatchIndex, pMsg->timeStamp, s, timeDiff); } void syncLogSendHeartbeatReply(SSyncNode* pSyncNode, const SyncHeartbeatReply* pMsg, const char* s) { diff --git a/source/libs/transport/src/tmsgcb.c b/source/libs/transport/src/tmsgcb.c index e87011f097..0632e21d20 100644 --- a/source/libs/transport/src/tmsgcb.c +++ b/source/libs/transport/src/tmsgcb.c @@ -37,7 +37,7 @@ int32_t tmsgPutToQueue(const SMsgCb* msgcb, EQueueType qtype, SRpcMsg* pMsg) { } int32_t tmsgGetQueueSize(const SMsgCb* msgcb, int32_t vgId, EQueueType qtype) { - return (*msgcb->qsizeFp)(msgcb->mgmt, vgId, qtype); + return (*msgcb->qsizeFp)(msgcb->mgmt, vgId, qtype); // vmGetQueueSize } int32_t tmsgSendReq(const SEpSet* epSet, SRpcMsg* pMsg) { From 67853b4a5461d72739f49f853f1f6abaa3f85b12 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sun, 26 Jan 2025 18:19:37 +0800 Subject: [PATCH 32/57] enh(stream): add checkpoint queue for source tasks. --- include/libs/stream/tstream.h | 1 + source/libs/stream/inc/streamInt.h | 2 + source/libs/stream/src/streamQueue.c | 136 ++++++++++++++++++++++----- 3 files changed, 113 insertions(+), 26 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index c4c0aaf742..5188e3c667 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -136,6 +136,7 @@ enum { enum { STREAM_QUEUE__SUCESS = 1, STREAM_QUEUE__FAILED, + STREAM_QUEUE__CHKPTFAILED, STREAM_QUEUE__PROCESSING, }; diff --git a/source/libs/stream/inc/streamInt.h b/source/libs/stream/inc/streamInt.h index d9778a6a05..0e5aaac58c 100644 --- a/source/libs/stream/inc/streamInt.h +++ b/source/libs/stream/inc/streamInt.h @@ -144,6 +144,8 @@ struct SStreamQueue { STaosQall* qall; void* qItem; int8_t status; + STaosQueue* pChkptQueue; + void* qChkptItem; }; struct SStreamQueueItem { diff --git a/source/libs/stream/src/streamQueue.c b/source/libs/stream/src/streamQueue.c index 401aa7530d..2a1332c0c4 100644 --- a/source/libs/stream/src/streamQueue.c +++ b/source/libs/stream/src/streamQueue.c @@ -47,7 +47,9 @@ static void streamQueueCleanup(SStreamQueue* pQueue) { int32_t streamQueueOpen(int64_t cap, SStreamQueue** pQ) { *pQ = NULL; + int32_t code = 0; + int32_t lino = 0; SStreamQueue* pQueue = taosMemoryCalloc(1, sizeof(SStreamQueue)); if (pQueue == NULL) { @@ -55,24 +57,26 @@ int32_t streamQueueOpen(int64_t cap, SStreamQueue** pQ) { } code = taosOpenQueue(&pQueue->pQueue); - if (code) { - taosMemoryFreeClear(pQueue); - return code; - } + TSDB_CHECK_CODE(code, lino, _error); code = taosAllocateQall(&pQueue->qall); - if (code) { - taosCloseQueue(pQueue->pQueue); - taosMemoryFree(pQueue); - return code; - } + TSDB_CHECK_CODE(code, lino, _error); + + code = taosOpenQueue(&pQueue->pChkptQueue); + TSDB_CHECK_CODE(code, lino, _error); pQueue->status = STREAM_QUEUE__SUCESS; + taosSetQueueCapacity(pQueue->pQueue, cap); taosSetQueueMemoryCapacity(pQueue->pQueue, cap * 1024); *pQ = pQueue; return code; + +_error: + streamQueueClose(pQueue, 0); + stError("failed to open stream queue at line:%d, code:%s", lino, tstrerror(code)); + return code; } void streamQueueClose(SStreamQueue* pQueue, int32_t taskId) { @@ -82,6 +86,11 @@ void streamQueueClose(SStreamQueue* pQueue, int32_t taskId) { taosFreeQall(pQueue->qall); taosCloseQueue(pQueue->pQueue); + pQueue->pQueue = NULL; + + taosCloseQueue(pQueue->pChkptQueue); + pQueue->pChkptQueue = NULL; + taosMemoryFree(pQueue); } @@ -94,6 +103,46 @@ void streamQueueNextItem(SStreamQueue* pQueue, SStreamQueueItem** pItem) { } else { pQueue->qItem = NULL; (void) taosGetQitem(pQueue->qall, &pQueue->qItem); + + if (pQueue->qItem == NULL) { + (void) taosReadAllQitems(pQueue->pQueue, pQueue->qall); + (void) taosGetQitem(pQueue->qall, &pQueue->qItem); + } + + *pItem = streamQueueCurItem(pQueue); + } +} + +void streamQueueNextItemInSourceQ(SStreamQueue* pQueue, SStreamQueueItem** pItem, ETaskStatus status) { + *pItem = NULL; + int8_t flag = atomic_exchange_8(&pQueue->status, STREAM_QUEUE__PROCESSING); + + if (flag == STREAM_QUEUE__CHKPTFAILED) { + *pItem = pQueue->qChkptItem; + } else { + pQueue->qChkptItem = NULL; + taosReadQitem(pQueue->pChkptQueue, (void**) &pQueue->qChkptItem); + if (pQueue->qChkptItem != NULL) { + stDebug("read data from checkpoint queue, status:%d", status); + + *pItem = pQueue->qChkptItem; + return; + } + + // if in checkpoint status, not read data from ordinary input q. + if (status == TASK_STATUS__CK) { + stDebug("in checkpoint status, not ready data in normal queue"); + return; + } + } + + // let's try the ordinary input q + if (flag == STREAM_QUEUE__FAILED) { + *pItem = streamQueueCurItem(pQueue); + } else { + pQueue->qItem = NULL; + (void) taosGetQitem(pQueue->qall, &pQueue->qItem); + if (pQueue->qItem == NULL) { (void) taosReadAllQitems(pQueue->pQueue, pQueue->qall); (void) taosGetQitem(pQueue->qall, &pQueue->qItem); @@ -110,6 +159,7 @@ void streamQueueProcessSuccess(SStreamQueue* queue) { } queue->qItem = NULL; + queue->qChkptItem = NULL; atomic_store_8(&queue->status, STREAM_QUEUE__SUCESS); } @@ -121,6 +171,14 @@ void streamQueueProcessFail(SStreamQueue* queue) { atomic_store_8(&queue->status, STREAM_QUEUE__FAILED); } +void streamQueueGetSourceChkptFailed(SStreamQueue* pQueue) { + if (atomic_load_8(&pQueue->status) != STREAM_QUEUE__PROCESSING) { + stError("invalid queue status:%d, expect:%d", atomic_load_8(&pQueue->status), STREAM_QUEUE__PROCESSING); + return; + } + atomic_store_8(&pQueue->status, STREAM_QUEUE__CHKPTFAILED); +} + bool streamQueueIsFull(const SStreamQueue* pQueue) { int32_t numOfItems = streamQueueGetNumOfItems(pQueue); if (numOfItems >= STREAM_TASK_QUEUE_CAPACITY) { @@ -175,8 +233,9 @@ const char* streamQueueItemGetTypeStr(int32_t type) { EExtractDataCode streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueItem** pInput, int32_t* numOfBlocks, int32_t* blockSize) { - const char* id = pTask->id.idStr; - int32_t taskLevel = pTask->info.taskLevel; + const char* id = pTask->id.idStr; + int32_t taskLevel = pTask->info.taskLevel; + SStreamQueue* pQueue = pTask->inputq.queue; *pInput = NULL; *numOfBlocks = 0; @@ -189,13 +248,19 @@ EExtractDataCode streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueIte } while (1) { - if (streamTaskShouldPause(pTask) || streamTaskShouldStop(pTask)) { - stDebug("s-task:%s task should pause, extract input blocks:%d", id, *numOfBlocks); + ETaskStatus status = streamTaskGetStatus(pTask).state; + if (status == TASK_STATUS__PAUSE || status == TASK_STATUS__STOP || status == TASK_STATUS__STOP) { + stDebug("s-task:%s task should pause/stop, extract input blocks:%d", id, *numOfBlocks); return EXEC_CONTINUE; } SStreamQueueItem* qItem = NULL; - streamQueueNextItem(pTask->inputq.queue, (SStreamQueueItem**)&qItem); + if (taskLevel == TASK_LEVEL__SOURCE) { + streamQueueNextItemInSourceQ(pQueue, &qItem, status); + } else { + streamQueueNextItem(pQueue, &qItem); + } + if (qItem == NULL) { // restore the token to bucket if (*numOfBlocks > 0) { @@ -225,14 +290,19 @@ EExtractDataCode streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueIte *numOfBlocks = 1; *pInput = qItem; return EXEC_CONTINUE; - } else { // previous existed blocks needs to be handle, before handle the checkpoint msg block + } else { // previous existed blocks needs to be handled, before handle the checkpoint msg block stDebug("s-task:%s %s msg extracted, handle previous blocks, numOfBlocks:%d", id, p, *numOfBlocks); *blockSize = streamQueueItemGetSize(*pInput); if (taskLevel == TASK_LEVEL__SINK) { streamTaskConsumeQuota(pTask->outputInfo.pTokenBucket, *blockSize); } - streamQueueProcessFail(pTask->inputq.queue); + if ((type == STREAM_INPUT__CHECKPOINT_TRIGGER || type == STREAM_INPUT__CHECKPOINT) && + (taskLevel == TASK_LEVEL__SOURCE)) { + streamQueueGetSourceChkptFailed(pQueue); + } else { + streamQueueProcessFail(pQueue); + } return EXEC_CONTINUE; } } else { @@ -252,7 +322,7 @@ EExtractDataCode streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueIte streamTaskConsumeQuota(pTask->outputInfo.pTokenBucket, *blockSize); } - streamQueueProcessFail(pTask->inputq.queue); + streamQueueProcessFail(pQueue); return EXEC_CONTINUE; } @@ -260,7 +330,7 @@ EExtractDataCode streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueIte } *numOfBlocks += 1; - streamQueueProcessSuccess(pTask->inputq.queue); + streamQueueProcessSuccess(pQueue); if (*numOfBlocks >= MAX_STREAM_EXEC_BATCH_NUM) { stDebug("s-task:%s batch size limit:%d reached, start to process blocks", id, MAX_STREAM_EXEC_BATCH_NUM); @@ -279,6 +349,7 @@ EExtractDataCode streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueIte int32_t streamTaskPutDataIntoInputQ(SStreamTask* pTask, SStreamQueueItem* pItem) { int8_t type = pItem->type; STaosQueue* pQueue = pTask->inputq.queue->pQueue; + int32_t level = pTask->info.taskLevel; int32_t total = streamQueueGetNumOfItems(pTask->inputq.queue) + 1; if (type == STREAM_INPUT__DATA_SUBMIT) { @@ -326,15 +397,28 @@ int32_t streamTaskPutDataIntoInputQ(SStreamTask* pTask, SStreamQueueItem* pItem) stDebug("s-task:%s blockdata enqueue, total in queue:%d, size:%.2fMiB", pTask->id.idStr, total, size); } else if (type == STREAM_INPUT__CHECKPOINT || type == STREAM_INPUT__CHECKPOINT_TRIGGER || type == STREAM_INPUT__TRANS_STATE || type == STREAM_INPUT__DATA_RETRIEVE) { - int32_t code = taosWriteQitem(pQueue, pItem); - if (code != TSDB_CODE_SUCCESS) { - streamFreeQitem(pItem); - return code; - } - double size = SIZE_IN_MiB(taosQueueMemorySize(pQueue)); - stDebug("s-task:%s level:%d %s blockdata enqueue, total in queue:%d, size:%.2fMiB", pTask->id.idStr, - pTask->info.taskLevel, streamQueueItemGetTypeStr(type), total, size); + int32_t code = 0; + if ((type == STREAM_INPUT__CHECKPOINT_TRIGGER || type == STREAM_INPUT__CHECKPOINT) && level == TASK_LEVEL__SOURCE) { + STaosQueue* pChkptQ = pTask->inputq.queue->pChkptQueue; + code = taosWriteQitem(pChkptQ, pItem); + + double size = SIZE_IN_MiB(taosQueueMemorySize(pChkptQ)); + int32_t num = taosQueueItemSize(pChkptQ); + + stDebug("s-task:%s level:%d %s checkpoint enqueue ctrl queue, total in queue:%d, size:%.2fMiB, data queue:%d", + pTask->id.idStr, pTask->info.taskLevel, streamQueueItemGetTypeStr(type), num, size, (total - 1)); + } else { + code = taosWriteQitem(pQueue, pItem); + if (code != TSDB_CODE_SUCCESS) { + streamFreeQitem(pItem); + return code; + } + + double size = SIZE_IN_MiB(taosQueueMemorySize(pQueue)); + stDebug("s-task:%s level:%d %s blockdata enqueue, total in queue:%d, size:%.2fMiB", pTask->id.idStr, + pTask->info.taskLevel, streamQueueItemGetTypeStr(type), total, size); + } } else if (type == STREAM_INPUT__GET_RES) { // use the default memory limit, refactor later. int32_t code = taosWriteQitem(pQueue, pItem); From 24c6fe727ff72ce32ff771b168d02f35cdfdc547 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sun, 26 Jan 2025 21:50:51 +0800 Subject: [PATCH 33/57] refactor(stream): do some internal refactor. --- source/libs/stream/src/streamQueue.c | 52 +++++++++++++++------------- 1 file changed, 28 insertions(+), 24 deletions(-) diff --git a/source/libs/stream/src/streamQueue.c b/source/libs/stream/src/streamQueue.c index 2a1332c0c4..6a9a1ac880 100644 --- a/source/libs/stream/src/streamQueue.c +++ b/source/libs/stream/src/streamQueue.c @@ -119,37 +119,41 @@ void streamQueueNextItemInSourceQ(SStreamQueue* pQueue, SStreamQueueItem** pItem if (flag == STREAM_QUEUE__CHKPTFAILED) { *pItem = pQueue->qChkptItem; - } else { - pQueue->qChkptItem = NULL; - taosReadQitem(pQueue->pChkptQueue, (void**) &pQueue->qChkptItem); - if (pQueue->qChkptItem != NULL) { - stDebug("read data from checkpoint queue, status:%d", status); + ASSERT(status != TASK_STATUS__CK && pQueue->qItem == NULL); + return; + } - *pItem = pQueue->qChkptItem; - return; - } + if (flag == STREAM_QUEUE__FAILED) { + *pItem = pQueue->qItem; + ASSERT(status != TASK_STATUS__CK && pQueue->qChkptItem == NULL); + return; + } - // if in checkpoint status, not read data from ordinary input q. - if (status == TASK_STATUS__CK) { - stDebug("in checkpoint status, not ready data in normal queue"); - return; - } + pQueue->qChkptItem = NULL; + taosReadQitem(pQueue->pChkptQueue, (void**)&pQueue->qChkptItem); + if (pQueue->qChkptItem != NULL) { + stDebug("read data from checkpoint queue, status:%d", status); + + *pItem = pQueue->qChkptItem; + return; + } + + // if in checkpoint status, not read data from ordinary input q. + if (status == TASK_STATUS__CK) { + stDebug("in checkpoint status, not ready data in normal queue"); + return; } // let's try the ordinary input q - if (flag == STREAM_QUEUE__FAILED) { - *pItem = streamQueueCurItem(pQueue); - } else { - pQueue->qItem = NULL; - (void) taosGetQitem(pQueue->qall, &pQueue->qItem); + pQueue->qItem = NULL; + (void)taosGetQitem(pQueue->qall, &pQueue->qItem); - if (pQueue->qItem == NULL) { - (void) taosReadAllQitems(pQueue->pQueue, pQueue->qall); - (void) taosGetQitem(pQueue->qall, &pQueue->qItem); - } - - *pItem = streamQueueCurItem(pQueue); + if (pQueue->qItem == NULL) { + (void)taosReadAllQitems(pQueue->pQueue, pQueue->qall); + (void)taosGetQitem(pQueue->qall, &pQueue->qItem); } + + *pItem = streamQueueCurItem(pQueue); } void streamQueueProcessSuccess(SStreamQueue* queue) { From f56aeaf1bd06d3a60da5fa0a7d929257d1da2348 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sun, 26 Jan 2025 22:57:06 +0800 Subject: [PATCH 34/57] fix(stream): fix error in deciding exec should quit or not. --- source/libs/stream/src/streamExec.c | 28 +++++++++++++++++++++++++--- source/libs/stream/src/streamQueue.c | 9 ++++----- 2 files changed, 29 insertions(+), 8 deletions(-) diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index 054f88ec3d..ef3fb11866 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -915,8 +915,31 @@ bool streamTaskReadyToRun(const SStreamTask* pTask, char** pStatus) { } } +static bool shouldNotCont(SStreamTask* pTask) { + int32_t level = pTask->info.taskLevel; + SStreamQueue* pQueue = pTask->inputq.queue; + ETaskStatus status = streamTaskGetStatus(pTask).state; + + // 1. task should jump out + bool quit = (status == TASK_STATUS__STOP) || (status == TASK_STATUS__PAUSE) || (status == TASK_STATUS__DROPPING); + + // 2. checkpoint procedure, the source task's checkpoint queue is empty, not read from ordinary queue + bool notCkCont = + (taosQueueItemSize(pQueue->pChkptQueue) == 0) && (level == TASK_LEVEL__SOURCE) && (status == TASK_STATUS__CK); + + // 3. no data in ordinary queue + int32_t numOfItems = streamQueueGetNumOfItems(pQueue); + + if ((numOfItems == 0) || quit || notCkCont) { + return true; + } else { + return false; + } +} + int32_t streamResumeTask(SStreamTask* pTask) { const char* id = pTask->id.idStr; + int32_t level = pTask->info.taskLevel; int32_t code = 0; if (pTask->status.schedStatus != TASK_SCHED_STATUS__ACTIVE) { @@ -929,11 +952,10 @@ int32_t streamResumeTask(SStreamTask* pTask) { if (code) { stError("s-task:%s failed to exec stream task, code:%s, continue", id, tstrerror(code)); } - // check if continue + streamMutexLock(&pTask->lock); - int32_t numOfItems = streamQueueGetNumOfItems(pTask->inputq.queue); - if ((numOfItems == 0) || streamTaskShouldStop(pTask) || streamTaskShouldPause(pTask)) { + if (shouldNotCont(pTask)) { atomic_store_8(&pTask->status.schedStatus, TASK_SCHED_STATUS__INACTIVE); streamTaskClearSchedIdleInfo(pTask); streamMutexUnlock(&pTask->lock); diff --git a/source/libs/stream/src/streamQueue.c b/source/libs/stream/src/streamQueue.c index 6a9a1ac880..3d0d01ce0e 100644 --- a/source/libs/stream/src/streamQueue.c +++ b/source/libs/stream/src/streamQueue.c @@ -113,7 +113,7 @@ void streamQueueNextItem(SStreamQueue* pQueue, SStreamQueueItem** pItem) { } } -void streamQueueNextItemInSourceQ(SStreamQueue* pQueue, SStreamQueueItem** pItem, ETaskStatus status) { +void streamQueueNextItemInSourceQ(SStreamQueue* pQueue, SStreamQueueItem** pItem, ETaskStatus status, const char* id) { *pItem = NULL; int8_t flag = atomic_exchange_8(&pQueue->status, STREAM_QUEUE__PROCESSING); @@ -132,15 +132,14 @@ void streamQueueNextItemInSourceQ(SStreamQueue* pQueue, SStreamQueueItem** pItem pQueue->qChkptItem = NULL; taosReadQitem(pQueue->pChkptQueue, (void**)&pQueue->qChkptItem); if (pQueue->qChkptItem != NULL) { - stDebug("read data from checkpoint queue, status:%d", status); - + stDebug("s-task:%s read data from checkpoint queue, status:%d", id, status); *pItem = pQueue->qChkptItem; return; } // if in checkpoint status, not read data from ordinary input q. if (status == TASK_STATUS__CK) { - stDebug("in checkpoint status, not ready data in normal queue"); + stDebug("s-task:%s in checkpoint status, not read data in block queue, status:%d", id, status); return; } @@ -260,7 +259,7 @@ EExtractDataCode streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueIte SStreamQueueItem* qItem = NULL; if (taskLevel == TASK_LEVEL__SOURCE) { - streamQueueNextItemInSourceQ(pQueue, &qItem, status); + streamQueueNextItemInSourceQ(pQueue, &qItem, status, id); } else { streamQueueNextItem(pQueue, &qItem); } From ed03b3a22cc3e70e7919e9d81c61d462b32893fb Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 27 Jan 2025 00:50:36 +0800 Subject: [PATCH 35/57] fix(stream): try starting task even the inputQ is full. --- source/dnode/vnode/src/tq/tqStreamTask.c | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/source/dnode/vnode/src/tq/tqStreamTask.c b/source/dnode/vnode/src/tq/tqStreamTask.c index b34ea78f64..88bbc30b7b 100644 --- a/source/dnode/vnode/src/tq/tqStreamTask.c +++ b/source/dnode/vnode/src/tq/tqStreamTask.c @@ -317,9 +317,10 @@ bool taskReadyForDataFromWal(SStreamTask* pTask) { return false; } - // check if input queue is full or not + // check whether input queue is full or not if (streamQueueIsFull(pTask->inputq.queue)) { - tqTrace("s-task:%s input queue is full, do nothing", pTask->id.idStr); + tqTrace("s-task:%s input queue is full, launch task without scanning wal", pTask->id.idStr); + streamTrySchedExec(pTask); return false; } From cb9a1a852d58aee1bb743e1cf1cd901ce03e28da Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 28 Jan 2025 01:38:16 +0800 Subject: [PATCH 36/57] refactor: update some logs. --- source/libs/stream/src/streamDispatch.c | 2 ++ 1 file changed, 2 insertions(+) diff --git a/source/libs/stream/src/streamDispatch.c b/source/libs/stream/src/streamDispatch.c index b7039d372d..b093f808c0 100644 --- a/source/libs/stream/src/streamDispatch.c +++ b/source/libs/stream/src/streamDispatch.c @@ -1098,6 +1098,8 @@ static void chkptReadyMsgSendMonitorFn(void* param, void* tmrId) { pActiveInfo = pTask->chkInfo.pActiveInfo; pTmrInfo = &pActiveInfo->chkptReadyMsgTmr; + stDebug("s-task:%s acquire task, refId:%" PRId64, id, taskRefId); + // check the status every 100ms if (streamTaskShouldStop(pTask)) { streamCleanBeforeQuitTmr(pTmrInfo, param); From 34a439d78bb322ad2ec5c3814eae072053855430 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 5 Feb 2025 10:59:29 +0800 Subject: [PATCH 37/57] fix(stream): fix error in the check of continuing execution condition. --- source/libs/stream/src/streamExec.c | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index ef3fb11866..3555515f75 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -924,16 +924,20 @@ static bool shouldNotCont(SStreamTask* pTask) { bool quit = (status == TASK_STATUS__STOP) || (status == TASK_STATUS__PAUSE) || (status == TASK_STATUS__DROPPING); // 2. checkpoint procedure, the source task's checkpoint queue is empty, not read from ordinary queue - bool notCkCont = - (taosQueueItemSize(pQueue->pChkptQueue) == 0) && (level == TASK_LEVEL__SOURCE) && (status == TASK_STATUS__CK); + bool emptyCkQueue = (taosQueueItemSize(pQueue->pChkptQueue) == 0) && (level == TASK_LEVEL__SOURCE); // 3. no data in ordinary queue - int32_t numOfItems = streamQueueGetNumOfItems(pQueue); + int32_t emptyBlockQueue = (streamQueueGetNumOfItems(pQueue) == 0); - if ((numOfItems == 0) || quit || notCkCont) { + if (quit) { return true; } else { - return false; + if (status == TASK_STATUS__CK) { + // in checkpoint procedure, we only check whether the controller queue is empty or not + return emptyCkQueue; + } else { // otherwise, if the block queue is empty, not continue. + return emptyBlockQueue; + } } } From 732a9d4b32b9cb36be64b3bbe674e242f1a90307 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 5 Feb 2025 13:35:23 +0800 Subject: [PATCH 38/57] fix(stream): fix error in the check of continuing execution condition. --- source/libs/stream/src/streamExec.c | 8 ++++---- source/libs/stream/src/streamQueue.c | 2 +- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index 3555515f75..1015917f61 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -924,19 +924,19 @@ static bool shouldNotCont(SStreamTask* pTask) { bool quit = (status == TASK_STATUS__STOP) || (status == TASK_STATUS__PAUSE) || (status == TASK_STATUS__DROPPING); // 2. checkpoint procedure, the source task's checkpoint queue is empty, not read from ordinary queue - bool emptyCkQueue = (taosQueueItemSize(pQueue->pChkptQueue) == 0) && (level == TASK_LEVEL__SOURCE); + bool emptyCkQueue = (taosQueueItemSize(pQueue->pChkptQueue) == 0); // 3. no data in ordinary queue - int32_t emptyBlockQueue = (streamQueueGetNumOfItems(pQueue) == 0); + bool emptyBlockQueue = (streamQueueGetNumOfItems(pQueue) == 0); if (quit) { return true; } else { - if (status == TASK_STATUS__CK) { + if (status == TASK_STATUS__CK && level == TASK_LEVEL__SOURCE) { // in checkpoint procedure, we only check whether the controller queue is empty or not return emptyCkQueue; } else { // otherwise, if the block queue is empty, not continue. - return emptyBlockQueue; + return emptyBlockQueue && emptyCkQueue; } } } diff --git a/source/libs/stream/src/streamQueue.c b/source/libs/stream/src/streamQueue.c index 3d0d01ce0e..f68dd1452f 100644 --- a/source/libs/stream/src/streamQueue.c +++ b/source/libs/stream/src/streamQueue.c @@ -402,7 +402,7 @@ int32_t streamTaskPutDataIntoInputQ(SStreamTask* pTask, SStreamQueueItem* pItem) type == STREAM_INPUT__TRANS_STATE || type == STREAM_INPUT__DATA_RETRIEVE) { int32_t code = 0; - if ((type == STREAM_INPUT__CHECKPOINT_TRIGGER || type == STREAM_INPUT__CHECKPOINT) && level == TASK_LEVEL__SOURCE) { + if ((type == STREAM_INPUT__CHECKPOINT_TRIGGER || type == STREAM_INPUT__CHECKPOINT) && (level == TASK_LEVEL__SOURCE)) { STaosQueue* pChkptQ = pTask->inputq.queue->pChkptQueue; code = taosWriteQitem(pChkptQ, pItem); From e60321196bf7f360ac5a1630648048a4f075371a Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sun, 9 Feb 2025 22:29:41 +0800 Subject: [PATCH 39/57] fix(stream): remove invalid assert. --- source/libs/stream/src/streamQueue.c | 2 -- 1 file changed, 2 deletions(-) diff --git a/source/libs/stream/src/streamQueue.c b/source/libs/stream/src/streamQueue.c index f68dd1452f..c7b0bc8f11 100644 --- a/source/libs/stream/src/streamQueue.c +++ b/source/libs/stream/src/streamQueue.c @@ -119,13 +119,11 @@ void streamQueueNextItemInSourceQ(SStreamQueue* pQueue, SStreamQueueItem** pItem if (flag == STREAM_QUEUE__CHKPTFAILED) { *pItem = pQueue->qChkptItem; - ASSERT(status != TASK_STATUS__CK && pQueue->qItem == NULL); return; } if (flag == STREAM_QUEUE__FAILED) { *pItem = pQueue->qItem; - ASSERT(status != TASK_STATUS__CK && pQueue->qChkptItem == NULL); return; } From 3e55f8edfed67b32a5ccdcc584a329b7ba606ef2 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sun, 23 Feb 2025 10:43:56 +0800 Subject: [PATCH 40/57] fix(stream): follower nodes not restart tasks. --- include/dnode/vnode/tqCommon.h | 2 +- source/dnode/snode/src/snode.c | 2 +- source/dnode/vnode/src/tq/tq.c | 3 ++- source/dnode/vnode/src/tqCommon/tqCommon.c | 31 ++++++++++++++-------- 4 files changed, 24 insertions(+), 14 deletions(-) diff --git a/include/dnode/vnode/tqCommon.h b/include/dnode/vnode/tqCommon.h index 4d5e18520c..d4ca0aba62 100644 --- a/include/dnode/vnode/tqCommon.h +++ b/include/dnode/vnode/tqCommon.h @@ -19,7 +19,7 @@ // message process int32_t tqStreamTaskStartAsync(SStreamMeta* pMeta, SMsgCb* cb, bool restart); int32_t tqStreamStartOneTaskAsync(SStreamMeta* pMeta, SMsgCb* cb, int64_t streamId, int32_t taskId); -int32_t tqStreamTaskProcessUpdateReq(SStreamMeta* pMeta, SMsgCb* cb, SRpcMsg* pMsg, bool restored); +int32_t tqStreamTaskProcessUpdateReq(SStreamMeta* pMeta, SMsgCb* cb, SRpcMsg* pMsg, bool restored, bool isLeader); int32_t tqStreamTaskProcessDispatchReq(SStreamMeta* pMeta, SRpcMsg* pMsg); int32_t tqStreamTaskProcessDispatchRsp(SStreamMeta* pMeta, SRpcMsg* pMsg); int32_t tqStreamTaskProcessRetrieveReq(SStreamMeta* pMeta, SRpcMsg* pMsg); diff --git a/source/dnode/snode/src/snode.c b/source/dnode/snode/src/snode.c index 6eee8c510b..b15ac447b0 100644 --- a/source/dnode/snode/src/snode.c +++ b/source/dnode/snode/src/snode.c @@ -157,7 +157,7 @@ int32_t sndProcessWriteMsg(SSnode *pSnode, SRpcMsg *pMsg, SRpcMsg *pRsp) { case TDMT_STREAM_TASK_DROP: return tqStreamTaskProcessDropReq(pSnode->pMeta, pMsg->pCont, pMsg->contLen); case TDMT_VND_STREAM_TASK_UPDATE: - return tqStreamTaskProcessUpdateReq(pSnode->pMeta, &pSnode->msgCb, pMsg, true); + return tqStreamTaskProcessUpdateReq(pSnode->pMeta, &pSnode->msgCb, pMsg, true, true); case TDMT_VND_STREAM_TASK_RESET: return tqStreamTaskProcessTaskResetReq(pSnode->pMeta, pMsg->pCont); case TDMT_STREAM_TASK_PAUSE: diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 63727e5c45..e1298b11ea 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1364,7 +1364,8 @@ int32_t tqProcessTaskCheckpointReadyMsg(STQ* pTq, SRpcMsg* pMsg) { } int32_t tqProcessTaskUpdateReq(STQ* pTq, SRpcMsg* pMsg) { - return tqStreamTaskProcessUpdateReq(pTq->pStreamMeta, &pTq->pVnode->msgCb, pMsg, pTq->pVnode->restored); + return tqStreamTaskProcessUpdateReq(pTq->pStreamMeta, &pTq->pVnode->msgCb, pMsg, + pTq->pVnode->restored, (pTq->pStreamMeta->role == NODE_ROLE_LEADER)); } int32_t tqProcessTaskResetReq(STQ* pTq, SRpcMsg* pMsg) { diff --git a/source/dnode/vnode/src/tqCommon/tqCommon.c b/source/dnode/vnode/src/tqCommon/tqCommon.c index 2c48ada0fa..fb3582c5ff 100644 --- a/source/dnode/vnode/src/tqCommon/tqCommon.c +++ b/source/dnode/vnode/src/tqCommon/tqCommon.c @@ -139,7 +139,7 @@ int32_t tqStreamStartOneTaskAsync(SStreamMeta* pMeta, SMsgCb* cb, int64_t stream } // this is to process request from transaction, always return true. -int32_t tqStreamTaskProcessUpdateReq(SStreamMeta* pMeta, SMsgCb* cb, SRpcMsg* pMsg, bool restored) { +int32_t tqStreamTaskProcessUpdateReq(SStreamMeta* pMeta, SMsgCb* cb, SRpcMsg* pMsg, bool restored, bool isLeader) { int32_t vgId = pMeta->vgId; char* msg = POINTER_SHIFT(pMsg->pCont, sizeof(SMsgHead)); int32_t len = pMsg->contLen - sizeof(SMsgHead); @@ -298,14 +298,19 @@ int32_t tqStreamTaskProcessUpdateReq(SStreamMeta* pMeta, SMsgCb* cb, SRpcMsg* pM int32_t numOfTasks = streamMetaGetNumOfTasks(pMeta); int32_t updateTasks = taosHashGetSize(pMeta->updateInfo.pTasks); - if (restored) { + if (restored && isLeader) { 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, - updateTasks, (numOfTasks - updateTasks)); + if (isLeader) { + tqDebug("vgId:%d closed tasks:%d, unclosed:%d, all tasks will be started when nodeEp update completed", vgId, + updateTasks, (numOfTasks - updateTasks)); + } else { + tqDebug("vgId:%d closed tasks:%d, unclosed:%d, follower not restart tasks", vgId, updateTasks, + (numOfTasks - updateTasks)); + } } else { if ((code = streamMetaCommit(pMeta)) < 0) { // always return true @@ -316,17 +321,21 @@ int32_t tqStreamTaskProcessUpdateReq(SStreamMeta* pMeta, SMsgCb* cb, SRpcMsg* pM streamMetaClearSetUpdateTaskListComplete(pMeta); - if (!restored) { - 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 (isLeader) { + if (!restored) { + 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 taosMSleep(5000);// for test purpose, to trigger the leader election #endif - code = tqStreamTaskStartAsync(pMeta, cb, true); - if (code) { - tqError("vgId:%d async start all tasks, failed, code:%s", vgId, tstrerror(code)); + code = tqStreamTaskStartAsync(pMeta, cb, true); + if (code) { + tqError("vgId:%d async start all tasks, failed, code:%s", vgId, tstrerror(code)); + } } + } else { + tqDebug("vgId:%d follower nodes not restart tasks", vgId); } } From 14ef25101dddd37307b6fc75d6ee10f84eb887e5 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sun, 23 Feb 2025 15:37:59 +0800 Subject: [PATCH 41/57] fix(stream): adjust log. --- source/dnode/vnode/src/tq/tqStreamTask.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/dnode/vnode/src/tq/tqStreamTask.c b/source/dnode/vnode/src/tq/tqStreamTask.c index 88bbc30b7b..57579eecb4 100644 --- a/source/dnode/vnode/src/tq/tqStreamTask.c +++ b/source/dnode/vnode/src/tq/tqStreamTask.c @@ -131,7 +131,7 @@ static void doStartScanWal(void* param, void* tmrId) { } if (pMeta->startInfo.startAllTasks) { - tqTrace("vgId:%d in restart procedure, not ready to scan wal", vgId); + tqDebug("vgId:%d in restart procedure, not ready to scan wal", vgId); goto _end; } From 951b469b10c6a0e306662a35d904b9638df611bb Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sun, 23 Feb 2025 16:18:19 +0800 Subject: [PATCH 42/57] fix(stream): add some logs. --- source/dnode/vnode/src/tq/tqStreamTask.c | 2 ++ 1 file changed, 2 insertions(+) diff --git a/source/dnode/vnode/src/tq/tqStreamTask.c b/source/dnode/vnode/src/tq/tqStreamTask.c index 57579eecb4..1e021a8d5b 100644 --- a/source/dnode/vnode/src/tq/tqStreamTask.c +++ b/source/dnode/vnode/src/tq/tqStreamTask.c @@ -87,6 +87,8 @@ static void doStartScanWal(void* param, void* tmrId) { tmr_h pTimer = NULL; SBuildScanWalMsgParam* pParam = (SBuildScanWalMsgParam*)param; + tqDebug("start to do scan wal in tmr, metaRid:%" PRId64, pParam->metaId); + SStreamMeta* pMeta = taosAcquireRef(streamMetaRefPool, pParam->metaId); if (pMeta == NULL) { tqError("metaRid:%" PRId64 " not valid now, stream meta has been freed", pParam->metaId); From 6e1206bbc11ef6e2bb0f7da5284c9a86bc14b731 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sun, 23 Feb 2025 16:18:49 +0800 Subject: [PATCH 43/57] fix(stream): add some logs. --- source/dnode/vnode/src/tq/tqStreamTask.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/dnode/vnode/src/tq/tqStreamTask.c b/source/dnode/vnode/src/tq/tqStreamTask.c index 1e021a8d5b..c8862b03ad 100644 --- a/source/dnode/vnode/src/tq/tqStreamTask.c +++ b/source/dnode/vnode/src/tq/tqStreamTask.c @@ -156,7 +156,7 @@ static void doStartScanWal(void* param, void* tmrId) { goto _end; } - tqTrace("vgId:%d create msg to start wal scan, numOfTasks:%d", vgId, numOfTasks); + tqDebug("vgId:%d create msg to start wal scan, numOfTasks:%d", vgId, numOfTasks); #if 0 // wait for the vnode is freed, and invalid read may occur. From b23f20a4507c78673b848175946acc5c4a3c3576 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 25 Feb 2025 14:29:49 +0800 Subject: [PATCH 44/57] fix(stream): fix memory leak in checkpt queue. --- source/libs/stream/src/streamQueue.c | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/source/libs/stream/src/streamQueue.c b/source/libs/stream/src/streamQueue.c index c7b0bc8f11..300b3bdb4d 100644 --- a/source/libs/stream/src/streamQueue.c +++ b/source/libs/stream/src/streamQueue.c @@ -32,11 +32,12 @@ typedef struct SQueueReader { static bool streamTaskExtractAvailableToken(STokenBucket* pBucket, const char* id); static void streamTaskPutbackToken(STokenBucket* pBucket); static void streamTaskConsumeQuota(STokenBucket* pBucket, int32_t bytes); +static void streamQueueNextItemInSourceQ(SStreamQueue* pQueue, SStreamQueueItem** pItem, ETaskStatus status, const char* id); static void streamQueueCleanup(SStreamQueue* pQueue) { SStreamQueueItem* qItem = NULL; while (1) { - streamQueueNextItem(pQueue, &qItem); + streamQueueNextItemInSourceQ(pQueue, &qItem, TASK_STATUS__READY, NULL); if (qItem == NULL) { break; } @@ -250,7 +251,7 @@ EExtractDataCode streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueIte while (1) { ETaskStatus status = streamTaskGetStatus(pTask).state; - if (status == TASK_STATUS__PAUSE || status == TASK_STATUS__STOP || status == TASK_STATUS__STOP) { + if (status == TASK_STATUS__PAUSE || status == TASK_STATUS__STOP) { stDebug("s-task:%s task should pause/stop, extract input blocks:%d", id, *numOfBlocks); return EXEC_CONTINUE; } From 00e02280b81eed943e3843f3d8feb3b160b26cd7 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Fri, 28 Feb 2025 14:01:26 +0800 Subject: [PATCH 45/57] fix(stream): adjust the error code position. --- include/libs/stream/tstream.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 5188e3c667..0fa32acfbb 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -136,8 +136,8 @@ enum { enum { STREAM_QUEUE__SUCESS = 1, STREAM_QUEUE__FAILED, - STREAM_QUEUE__CHKPTFAILED, STREAM_QUEUE__PROCESSING, + STREAM_QUEUE__CHKPTFAILED, }; typedef enum EStreamTaskEvent { From 3892a98e2ee61619fadbcf1f30ff74c4f0c6a824 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Fri, 28 Feb 2025 14:09:45 +0800 Subject: [PATCH 46/57] fix(stream): check return values. --- source/libs/stream/src/streamQueue.c | 16 +++++++++++++--- 1 file changed, 13 insertions(+), 3 deletions(-) diff --git a/source/libs/stream/src/streamQueue.c b/source/libs/stream/src/streamQueue.c index 300b3bdb4d..954e41f288 100644 --- a/source/libs/stream/src/streamQueue.c +++ b/source/libs/stream/src/streamQueue.c @@ -144,11 +144,21 @@ void streamQueueNextItemInSourceQ(SStreamQueue* pQueue, SStreamQueueItem** pItem // let's try the ordinary input q pQueue->qItem = NULL; - (void)taosGetQitem(pQueue->qall, &pQueue->qItem); + int32_t code = taosGetQitem(pQueue->qall, &pQueue->qItem); + if (code) { + stError("s-task:%s failed to get item in inputq, code:%s", id, tstrerror(code)); + } if (pQueue->qItem == NULL) { - (void)taosReadAllQitems(pQueue->pQueue, pQueue->qall); - (void)taosGetQitem(pQueue->qall, &pQueue->qItem); + code = taosReadAllQitems(pQueue->pQueue, pQueue->qall); + if (code) { + stError("s-task:%s failed to get all items in inputq, code:%s", id, tstrerror(code)); + } + + code = taosGetQitem(pQueue->qall, &pQueue->qItem); + if (code) { + stError("s-task:%s failed to get item in inputq, code:%s", id, tstrerror(code)); + } } *pItem = streamQueueCurItem(pQueue); From 1462f64ce209b1628b505333003ea5b427a9c4e0 Mon Sep 17 00:00:00 2001 From: facetosea <285808407@qq.com> Date: Fri, 28 Feb 2025 14:15:41 +0800 Subject: [PATCH 47/57] fix: different lenth in uion --- source/libs/parser/src/parTranslater.c | 52 +++++++++++++++ tests/army/query/queryBugs.py | 88 +++++++++++++++++++++++++- 2 files changed, 139 insertions(+), 1 deletion(-) diff --git a/source/libs/parser/src/parTranslater.c b/source/libs/parser/src/parTranslater.c index be4882935f..3511636bd9 100755 --- a/source/libs/parser/src/parTranslater.c +++ b/source/libs/parser/src/parTranslater.c @@ -7406,6 +7406,52 @@ static SNode* createSetOperProject(const char* pTableAlias, SNode* pNode) { return (SNode*)pCol; } +static bool isUionOperator(SNode* pNode) { + return QUERY_NODE_SET_OPERATOR == nodeType(pNode) && (((SSetOperator*)pNode)->opType == SET_OP_TYPE_UNION || + ((SSetOperator*)pNode)->opType == SET_OP_TYPE_UNION_ALL); +} + +static int32_t pushdownCastForUnion(STranslateContext* pCxt, SNode* pNode, SExprNode* pExpr, int pos) { + int32_t code = TSDB_CODE_SUCCESS; + if (isUionOperator(pNode)) { + SSetOperator* pSetOperator = (SSetOperator*)pNode; + SNodeList* pLeftProjections = getProjectList(pSetOperator->pLeft); + SNodeList* pRightProjections = getProjectList(pSetOperator->pRight); + if (LIST_LENGTH(pLeftProjections) != LIST_LENGTH(pRightProjections)) { + return generateSyntaxErrMsg(&pCxt->msgBuf, TSDB_CODE_PAR_INCORRECT_NUM_OF_COL); + } + + SNode* pLeft = NULL; + SNode* pRight = NULL; + int32_t index = 0; + FORBOTH(pLeft, pLeftProjections, pRight, pRightProjections) { + ++index; + if (index < pos) { + continue; + } + SNode* pRightFunc = NULL; + code = createCastFunc(pCxt, pRight, pExpr->resType, &pRightFunc); + if (TSDB_CODE_SUCCESS != code || NULL == pRightFunc) { + return code; + } + REPLACE_LIST2_NODE(pRightFunc); + code = pushdownCastForUnion(pCxt, pSetOperator->pRight, (SExprNode*)pRightFunc, index); + if (TSDB_CODE_SUCCESS != code ) return code; + + SNode* pLeftFunc = NULL; + code = createCastFunc(pCxt, pLeft, pExpr->resType, &pLeftFunc); + if (TSDB_CODE_SUCCESS != code || NULL == pLeftFunc) { + return code; + } + REPLACE_LIST1_NODE(pLeftFunc); + code = pushdownCastForUnion(pCxt, pSetOperator->pLeft, (SExprNode*)pLeftFunc, index); + if (TSDB_CODE_SUCCESS != code ) return code; + break; + } + } + return TSDB_CODE_SUCCESS; +} + static int32_t translateSetOperProject(STranslateContext* pCxt, SSetOperator* pSetOperator) { SNodeList* pLeftProjections = getProjectList(pSetOperator->pLeft); SNodeList* pRightProjections = getProjectList(pSetOperator->pRight); @@ -7415,9 +7461,11 @@ static int32_t translateSetOperProject(STranslateContext* pCxt, SSetOperator* pS SNode* pLeft = NULL; SNode* pRight = NULL; + int32_t index = 0; FORBOTH(pLeft, pLeftProjections, pRight, pRightProjections) { SExprNode* pLeftExpr = (SExprNode*)pLeft; SExprNode* pRightExpr = (SExprNode*)pRight; + ++index; int32_t comp = dataTypeComp(&pLeftExpr->resType, &pRightExpr->resType); if (comp > 0) { SNode* pRightFunc = NULL; @@ -7427,6 +7475,8 @@ static int32_t translateSetOperProject(STranslateContext* pCxt, SSetOperator* pS } REPLACE_LIST2_NODE(pRightFunc); pRightExpr = (SExprNode*)pRightFunc; + code = pushdownCastForUnion(pCxt, pSetOperator->pRight, pRightExpr, index); + if (TSDB_CODE_SUCCESS != code ) return code; } else if (comp < 0) { SNode* pLeftFunc = NULL; int32_t code = createCastFunc(pCxt, pLeft, pRightExpr->resType, &pLeftFunc); @@ -7439,6 +7489,8 @@ static int32_t translateSetOperProject(STranslateContext* pCxt, SSetOperator* pS snprintf(pLeftFuncExpr->userAlias, sizeof(pLeftFuncExpr->userAlias), "%s", pLeftExpr->userAlias); pLeft = pLeftFunc; pLeftExpr = pLeftFuncExpr; + code = pushdownCastForUnion(pCxt, pSetOperator->pLeft, pLeftExpr, index); + if (TSDB_CODE_SUCCESS != code ) return code; } snprintf(pRightExpr->aliasName, sizeof(pRightExpr->aliasName), "%s", pLeftExpr->aliasName); SNode* pProj = createSetOperProject(pSetOperator->stmtName, pLeft); diff --git a/tests/army/query/queryBugs.py b/tests/army/query/queryBugs.py index cd61b8c620..e6b20addb9 100644 --- a/tests/army/query/queryBugs.py +++ b/tests/army/query/queryBugs.py @@ -325,12 +325,97 @@ class TDTestCase(TBase): tdSql.query("select * from t1 where ts > '2025-01-01 00:00:00';") tdSql.checkRows(0) + def FIX_TS_6058(self): + tdSql.execute("create database iot_60j_production_eqp;") + tdSql.execute("create table iot_60j_production_eqp.realtime_data_collections (device_time TIMESTAMP, item_value VARCHAR(64), \ + upload_time TIMESTAMP) tags(bu_id VARCHAR(64), district_id VARCHAR(64), factory_id VARCHAR(64), production_line_id VARCHAR(64), \ + production_processes_id VARCHAR(64), work_center_id VARCHAR(64), station_id VARCHAR(64), device_name VARCHAR(64), item_name VARCHAR(64));") + + sub1 = " SELECT '实际速度' as name, 0 as rank, '当月' as cycle,\ + CASE \ + WHEN COUNT(item_value) = 0 THEN NULL\ + ELSE AVG(CAST(item_value AS double))\ + END AS item_value\ + FROM iot_60j_production_eqp.realtime_data_collections\ + WHERE device_time >= TO_TIMESTAMP(CONCAT(substring(TO_CHAR(today ,'YYYY-MM-dd'), 1,7), '-01 00:00:00'), 'YYYY-mm-dd')\ + AND item_name = 'Premixer_SpindleMotor_ActualSpeed' " + + sub2 = " SELECT '实际速度' as name, 3 as rank, TO_CHAR(TODAY(),'YYYY-MM-dd') as cycle,\ + CASE \ + WHEN COUNT(item_value) = 0 THEN NULL\ + ELSE AVG(CAST(item_value AS double))\ + END AS item_value\ + FROM iot_60j_production_eqp.realtime_data_collections\ + WHERE device_time >= TODAY()-1d and device_time <= now()\ + AND item_name = 'Premixer_SpindleMotor_ActualSpeed' " + + sub3 = " SELECT '设定速度' as name, 1 as rank, CAST(CONCAT('WEEK-',CAST(WEEKOFYEAR(TODAY()-1w) as VARCHAR)) as VARCHAR) as cycle,\ + CASE \ + WHEN COUNT(item_value) = 0 THEN NULL\ + ELSE AVG(CAST(item_value AS double))\ + END AS item_value\ + FROM iot_60j_production_eqp.realtime_data_collections\ + where \ + item_name = 'Premixer_SpindleMotor_SettingSpeed'\ + AND (\ + (WEEKDAY(now) = 0 AND device_time >= today()-8d and device_time <= today()-1d) OR\ + (WEEKDAY(now) = 1 AND device_time >= today()-9d and device_time <= today()-2d) OR\ + (WEEKDAY(now) = 2 AND device_time >= today()-10d and device_time <= today()-3d) OR\ + (WEEKDAY(now) = 3 AND device_time >= today()-11d and device_time <= today()-4d) OR\ + (WEEKDAY(now) = 4 AND device_time >= today()-12d and device_time <= today()-5d) OR\ + (WEEKDAY(now) = 5 AND device_time >= today()-13d and device_time <= today()-6d) OR\ + (WEEKDAY(now) = 6 AND device_time >= today()-14d and device_time <= today()-7d)\ + ) " + + sub4 = " SELECT '设定速度2' as name, 1 as rank, CAST(CONCAT('WEEK-',CAST(WEEKOFYEAR(TODAY()-1w) as VARCHAR)) as VARCHAR(5000)) as cycle,\ + CASE \ + WHEN COUNT(item_value) = 0 THEN NULL\ + ELSE AVG(CAST(item_value AS double))\ + END AS item_value\ + FROM iot_60j_production_eqp.realtime_data_collections\ + where \ + item_name = 'Premixer_SpindleMotor_SettingSpeed'\ + AND (\ + (WEEKDAY(now) = 0 AND device_time >= today()-8d and device_time <= today()-1d) OR\ + (WEEKDAY(now) = 1 AND device_time >= today()-9d and device_time <= today()-2d) OR\ + (WEEKDAY(now) = 2 AND device_time >= today()-10d and device_time <= today()-3d) OR\ + (WEEKDAY(now) = 3 AND device_time >= today()-11d and device_time <= today()-4d) OR\ + (WEEKDAY(now) = 4 AND device_time >= today()-12d and device_time <= today()-5d) OR\ + (WEEKDAY(now) = 5 AND device_time >= today()-13d and device_time <= today()-6d) OR\ + (WEEKDAY(now) = 6 AND device_time >= today()-14d and device_time <= today()-7d)\ + ) " + for uiontype in ["union" ,"union all"]: + repeatLines = 1 + if uiontype == "union": + repeatLines = 0 + for i in range(1, 10): + tdLog.debug(f"test: realtime_data_collections {i} times...") + tdSql.query(f"select name,cycle,item_value from ( {sub1} {uiontype} {sub2} {uiontype} {sub3}) order by rank,name,cycle;", queryTimes = 1) + tdSql.checkRows(3) + tdSql.query(f"select name,cycle,item_value from ( {sub1} {uiontype} {sub2} {uiontype} {sub4}) order by rank,name,cycle;", queryTimes = 1) + tdSql.checkRows(3) + tdSql.query(f"select name,cycle,item_value from ( {sub3} {uiontype} {sub2} {uiontype} {sub1}) order by rank,name,cycle;", queryTimes = 1) + tdSql.checkRows(3) + tdSql.query(f"select name,cycle,item_value from ( {sub3} {uiontype} {sub2} {uiontype} {sub1}) order by rank,name,cycle;", queryTimes = 1) + tdSql.checkRows(3) + tdSql.query(f"select name,cycle,item_value from ( {sub2} {uiontype} {sub4} {uiontype} {sub1}) order by rank,name,cycle;", queryTimes = 1) + tdSql.checkRows(3) + tdSql.query(f"select name,cycle,item_value from ( {sub3} {uiontype} {sub2} {uiontype} {sub1} {uiontype} {sub4}) order by rank,name,cycle;", queryTimes = 1) + tdSql.checkRows(4) + tdSql.query(f"select name,cycle,item_value from ( {sub2} {uiontype} {sub3} {uiontype} {sub1} {uiontype} {sub4}) order by rank,name,cycle;", queryTimes = 1) + tdSql.checkRows(4) + tdSql.query(f"select name,cycle,item_value from ( {sub3} {uiontype} {sub4} {uiontype} {sub1} {uiontype} {sub2}) order by rank,name,cycle;", queryTimes = 1) + tdSql.checkRows(4) + tdSql.query(f"select name,cycle,item_value from ( {sub3} {uiontype} {sub4} {uiontype} {sub1} {uiontype} {sub2} {uiontype} {sub4}) order by rank,name,cycle;", queryTimes = 1) + tdSql.checkRows(4 + repeatLines) + tdSql.query(f"select name,cycle,item_value from ( {sub3} {uiontype} {sub2} {uiontype} {sub1} {uiontype} {sub2} {uiontype} {sub4}) order by rank,name,cycle;", queryTimes = 1) + tdSql.checkRows(4 + repeatLines) + # run def run(self): tdLog.debug(f"start to excute {__file__}") self.ts5946() - # TD BUGS self.FIX_TD_30686() self.FIX_TD_31684() @@ -340,6 +425,7 @@ class TDTestCase(TBase): self.FIX_TS_5143() self.FIX_TS_5239() self.FIX_TS_5984() + self.FIX_TS_6058() tdLog.success(f"{__file__} successfully executed") From fac18aa80694b7766068da973da946047f4d1c95 Mon Sep 17 00:00:00 2001 From: Feng Chao Date: Fri, 28 Feb 2025 14:52:57 +0800 Subject: [PATCH 48/57] ci: add linux/mac ci build and test workflow --- .github/workflows/taosd-ci.yml | 316 ++++++++++++++++++++++++++ .github/workflows/taosd-doc-build.yml | 74 +----- 2 files changed, 327 insertions(+), 63 deletions(-) create mode 100644 .github/workflows/taosd-ci.yml diff --git a/.github/workflows/taosd-ci.yml b/.github/workflows/taosd-ci.yml new file mode 100644 index 0000000000..022e4bcab2 --- /dev/null +++ b/.github/workflows/taosd-ci.yml @@ -0,0 +1,316 @@ + +name: TDengine CI Pipeline + +on: + pull_request: + branches: + - 'main' + - '3.0' + - '3.1' + paths-ignore: + - 'packaging/**' + - 'docs/**' + repository_dispatch: + types: [run-tests] + +concurrency: + group: ${{ github.workflow }}-${{ github.ref }} + cancel-in-progress: true + +env: + CONTAINER_NAME: 'taosd-test' + WKDIR: '/var/lib/jenkins/workspace' + WK: '/var/lib/jenkins/workspace/TDinternal' + WKC: '/var/lib/jenkins/workspace/TDinternal/community' + +jobs: + fetch-parameters: + runs-on: + group: CI + labels: [self-hosted, Linux, X64, testing] + outputs: + tdinternal: ${{ steps.parameters.outputs.tdinternal }} + run_function_test: ${{ steps.parameters.outputs.run_function_test }} + run_tdgpt_test: ${{ steps.parameters.outputs.run_tdgpt_test }} + source_branch: ${{ steps.parameters.outputs.source_branch }} + target_branch: ${{ steps.parameters.outputs.target_branch }} + pr_number: ${{ steps.parameters.outputs.pr_number }} + steps: + - name: Determine trigger source and fetch parameters + id: parameters + run: | + set -euo pipefail + # check the trigger source and get branch information + if [ "${{ github.event_name }}" == "repository_dispatch" ]; then + tdinternal="true" + source_branch=${{ github.event.client_payload.tdinternal_source_branch }} + target_branch=${{ github.event.client_payload.tdinternal_target_branch }} + pr_number=${{ github.event.client_payload.tdinternal_pr_number }} + run_tdgpt_test="true" + run_function_test="true" + else + tdinternal="false" + source_branch=${{ github.event.pull_request.head.ref }} + target_branch=${{ github.event.pull_request.base.ref }} + pr_number=${{ github.event.pull_request.number }} + + # check whether to run tdgpt test cases + cd ${{ env.WKC }} + changed_files_non_doc=$(git --no-pager diff --name-only FETCH_HEAD `git merge-base FETCH_HEAD $target_branch`|grep -v "^docs/en/"|grep -v "^docs/zh/"|grep -v ".md$" | tr '\n' ' ' || :) + + if [[ "$changed_files_non_doc" != '' && "$changed_files_non_doc" =~ /forecastoperator.c|anomalywindowoperator.c|tanalytics.h|tanalytics.c|tdgpt_cases.task|analytics/ ]]; then + run_tdgpt_test="true" + else + run_tdgpt_test="false" + fi + + # check whether to run function test cases + changed_files_non_tdgpt=$(git --no-pager diff --name-only FETCH_HEAD `git merge-base FETCH_HEAD $target_branch`|grep -v "^docs/en/"|grep -v "^docs/zh/"|grep -v ".md$" | grep -Ev "forecastoperator.c|anomalywindowoperator.c|tanalytics.h|tanalytics.c|tdgpt_cases.task|analytics" | tr '\n' ' ' ||:) + if [ $changed_files_non_tdgpt != '' ]; then + run_function_test="true" + else + run_function_test="false" + fi + fi + + echo "tdinternal=$tdinternal" >> $GITHUB_OUTPUT + echo "run_function_test=$run_function_test" >> $GITHUB_OUTPUT + echo "run_tdgpt_test=$run_tdgpt_test" >> $GITHUB_OUTPUT + echo "source_branch=$source_branch" >> $GITHUB_OUTPUT + echo "target_branch=$target_branch" >> $GITHUB_OUTPUT + echo "pr_number=$pr_number" >> $GITHUB_OUTPUT + + run-tests-on-linux: + needs: fetch-parameters + runs-on: + group: CI + labels: [self-hosted, Linux, X64, testing] + timeout-minutes: 200 + env: + IS_TDINTERNAL: ${{ needs.fetch-parameters.outputs.tdinternal }} + RUN_RUNCTION_TEST: ${{ needs.fetch-parameters.outputs.run_function_test }} + RUN_TDGPT_TEST: ${{ needs.fetch-parameters.outputs.run_tdgpt_tests }} + SOURCE_BRANCH: ${{ needs.fetch-parameters.outputs.source_branch }} + TARGET_BRANCH: ${{ needs.fetch-parameters.outputs.target_branch }} + PR_NUMBER: ${{ needs.fetch-parameters.outputs.pr_number }} + steps: + - name: Output the environment information + run: | + echo "::group::Environment Info" + date + hostname + env + echo "Runner: ${{ runner.name }}" + echo "Trigger Source from TDinternal: ${{ env.IS_TDINTERNAL }}" + echo "Workspace: ${{ env.WKDIR }}" + git --version + echo "${{ env.WKDIR }}/restore.sh -p ${{ env.PR_NUMBER }} -n ${{ github.run_number }} -c ${{ env.CONTAINER_NAME }}" + echo "::endgroup::" + + - name: Prepare repositories + run: | + set -euo pipefail + prepare_environment() { + cd "$1" + git reset --hard + git clean -f + git remote prune origin + git fetch + git checkout "$2" + } + prepare_environment "${{ env.WK }}" "${{ env.TARGET_BRANCH }}" + prepare_environment "${{ env.WKC }}" "${{ env.TARGET_BRANCH }}" + + - name: Get latest codes and logs for TDinternal PR + if: ${{ env.IS_TDINTERNAL == 'true' }} + run: | + cd ${{ env.WK }} + git pull >/dev/null + git log -5 + echo "`date "+%Y%m%d-%H%M%S"` TDinternalTest/${{ env.PR_NUMBER }}:${{ github.run_number }}:${{ env.TARGET_BRANCH }}" >>${{ env.WKDIR }}/jenkins.log + echo "CHANGE_BRANCH:${{ env.SOURCE_BRANCH }}" >>${{ env.WKDIR }}/jenkins.log + echo "TDinternal log: `git log -5`" >>${{ env.WKDIR }}/jenkins.log + git fetch origin +refs/pull/${{ env.PR_NUMBER }}/merge + git checkout -qf FETCH_HEAD + git log -5 + echo "TDinternal log merged: `git log -5`" >>${{ env.WKDIR }}/jenkins.log + cd ${{ env.WKC }} + git remote prune origin + git pull >/dev/null + git log -5 + echo "community log: `git log -5`" >>${{ env.WKDIR }}/jenkins.log + - name: Get latest codes and logs for TDengine PR + if: ${{ env.IS_TDINTERNAL == 'false' }} + run: | + cd ${{ env.WKC }} + git remote prune origin + git pull >/dev/null + git log -5 + echo "`date "+%Y%m%d-%H%M%S"` TDengineTest/${{ env.PR_NUMBER }}:${{ github.run_number }}:${{ env.TARGET_BRANCH }}" >>${{ env.WKDIR }}/jenkins.log + echo "CHANGE_BRANCH:${{ env.SOURCE_BRANCH }}" >>${{ env.WKDIR }}/jenkins.log + echo "community log: `git log -5`" >>${{ env.WKDIR }}/jenkins.log + git fetch origin +refs/pull/${{ env.PR_NUMBER }}/merge + git checkout -qf FETCH_HEAD + git log -5 + echo "community log merged: `git log -5`" >>${{ env.WKDIR }}/jenkins.log + cd ${{ env.WK }} + git pull >/dev/null + git log -5 + echo "TDinternal log: `git log -5`" >>${{ env.WKDIR }}/jenkins.log + - name: Update submodule + run: | + cd ${{ env.WKC }} + git submodule update --init --recursive + - name: Output the 'file_no_doc_changed' information to the file + if: ${{ env.IS_TDINTERNAL == 'false' }} + run: | + mkdir -p ${{ env.WKDIR }}/tmp/${{ env.PR_NUMBER }}_${{ github.run_number }} + changed_files_non_doc=$(git --no-pager diff --name-only FETCH_HEAD `git merge-base FETCH_HEAD ${{ env.TARGET_BRANCH }}`|grep -v "^docs/en/"|grep -v "^docs/zh/"|grep -v ".md$" | tr '\n' ' ' || :) + echo $changed_files_non_doc > ${{ env.WKDIR }}/tmp/${{ env.PR_NUMBER }}_${{ github.run_number }}/docs_changed.txt + - name: Check assert testing + run: | + cd ${{ env.WKC }}/tests/parallel_test + ./run_check_assert_container.sh -d ${{ env.WKDIR }} + - name: Check void function testing + run: | + cd ${{ env.WKC }}/tests/parallel_test + ./run_check_void_container.sh -d ${{ env.WKDIR }} + - name: Build docker container + run: | + date + rm -rf ${{ env.WKC }}/debug + cd ${{ env.WKC }}/tests/parallel_test + time ./container_build.sh -w ${{ env.WKDIR }} -e + - name: Get parameters for testing + id: get_param + run: | + log_server_file="/home/log_server.json" + timeout_cmd="" + extra_param="" + + if [ -f "$log_server_file" ]; then + log_server_enabled=$(jq '.enabled' "$log_server_file") + timeout_param=$(jq '.timeout' "$log_server_file") + if [ "$timeout_param" != "null" ] && [ "$timeout_param" != "0" ]; then + timeout_cmd="timeout $timeout_param" + fi + + if [ "$log_server_enabled" == "1" ]; then + log_server=$(jq '.server' "$log_server_file" | sed 's/\\\"//g') + if [ "$log_server" != "null" ] && [ "$log_server" != "" ]; then + extra_param="-w $log_server" + fi + fi + fi + echo "timeout_cmd=$timeout_cmd" >> $GITHUB_OUTPUT + echo "extra_param=$extra_param" >> $GITHUB_OUTPUT + - name: Run function returns with a null pointer scan testing + run: | + cd ${{ env.WKC }}/tests/parallel_test + ./run_scan_container.sh -d ${{ env.WKDIR }} -b ${{ env.PR_NUMBER }}_${{ github.run_number }} -f ${{ env.WKDIR }}/tmp/${{ env.PR_NUMBER }}_${{ github.run_number }}/docs_changed.txt ${{ steps.get_param.outputs.extra_param }} + - name: Run tdgpt test cases + if: ${{ env.IS_TDINTERNAL }} == 'false' && ${{ env.RUN_TDGPT_TEST }} == 'true' + run: | + cd ${{ env.WKC }}/tests/parallel_test + export DEFAULT_RETRY_TIME=2 + date + timeout 600 time ./run.sh -e -m /home/m.json -t tdgpt_cases.task -b ${{ env.PR_NUMBER }}_${{ github.run_number }} -l ${{ env.WKDIR }}/log -o 300 ${{ steps.get_param.outputs.extra_param }} + - name: Run function test cases + if: ${{ env.RUN_RUNCTION_TEST }} == 'true' + run: | + cd ${{ env.WKC }}/tests/parallel_test + export DEFAULT_RETRY_TIME=2 + date + ${{ steps.get_param.outputs.timeout_cmd }} time ./run.sh -e -m /home/m.json -t cases.task -b ${{ env.PR_NUMBER }}_${{ github.run_number }} -l ${{ env.WKDIR }}/log -o 1200 ${{ steps.get_param.outputs.extra_param }} + + run-tests-on-mac: + needs: fetch-parameters + if: ${{ needs.fetch-parameters.outputs.run_function_test == 'false' }} + runs-on: + group: CI + labels: [self-hosted, macOS, ARM64, testing] + timeout-minutes: 60 + env: + IS_TDINTERNAL: ${{ needs.fetch-parameters.outputs.tdinternal }} + SOURCE_BRANCH: ${{ needs.fetch-parameters.outputs.source_branch }} + TARGET_BRANCH: ${{ needs.fetch-parameters.outputs.target_branch }} + PR_NUMBER: ${{ needs.fetch-parameters.outputs.pr_number }} + steps: + - name: Output the environment information + run: | + echo "::group::Environment Info" + date + hostname + env + echo "Runner: ${{ runner.name }}" + echo "Trigger Source from TDinternal: ${{ env.IS_TDINTERNAL }}" + echo "Workspace: ${{ env.WKDIR }}" + git --version + echo "${{ env.WKDIR }}/restore.sh -p ${{ env.PR_NUMBER }} -n ${{ github.run_number }} -c ${{ env.CONTAINER_NAME }}" + echo "::endgroup::" + - name: Prepare repositories + run: | + set -euo pipefail + prepare_environment() { + cd "$1" + git reset --hard + git clean -f + git remote prune origin + git fetch + git checkout "$2" + } + prepare_environment "${{ env.WK }}" "${{ env.TARGET_BRANCH }}" + prepare_environment "${{ env.WKC }}" "${{ env.TARGET_BRANCH }}" + - name: Get latest codes and logs for TDinternal PR + if: ${{ env.IS_TDINTERNAL == 'true' }} + run: | + cd ${{ env.WK }} + git pull >/dev/null + git log -5 + echo "`date "+%Y%m%d-%H%M%S"` TDinternalTest/${{ env.PR_NUMBER }}:${{ github.run_number }}:${{ env.TARGET_BRANCH }}" >>${{ env.WKDIR }}/jenkins.log + echo "CHANGE_BRANCH:${{ env.SOURCE_BRANCH }}" >>${{ env.WKDIR }}/jenkins.log + echo "TDinternal log: `git log -5`" >>${{ env.WKDIR }}/jenkins.log + git fetch origin +refs/pull/${{ env.PR_NUMBER }}/merge + git checkout -qf FETCH_HEAD + git log -5 + echo "TDinternal log merged: `git log -5`" >>${{ env.WKDIR }}/jenkins.log + cd ${{ env.WKC }} + git remote prune origin + git pull >/dev/null + git log -5 + echo "community log: `git log -5`" >>${{ env.WKDIR }}/jenkins.log + - name: Get latest codes and logs for TDengine PR + if: ${{ env.IS_TDINTERNAL == 'false' }} + run: | + cd ${{ env.WKC }} + git remote prune origin + git pull >/dev/null + git log -5 + echo "`date "+%Y%m%d-%H%M%S"` TDengineTest/${{ env.PR_NUMBER }}:${{ github.run_number }}:${{ env.TARGET_BRANCH }}" >>${{ env.WKDIR }}/jenkins.log + echo "CHANGE_BRANCH:${{ env.SOURCE_BRANCH }}" >>${{ env.WKDIR }}/jenkins.log + echo "community log: `git log -5`" >>${{ env.WKDIR }}/jenkins.log + git fetch origin +refs/pull/${{ env.PR_NUMBER }}/merge + git checkout -qf FETCH_HEAD + git log -5 + echo "community log merged: `git log -5`" >>${{ env.WKDIR }}/jenkins.log + cd ${{ env.WK }} + git pull >/dev/null + git log -5 + echo "TDinternal log: `git log -5`" >>${{ env.WKDIR }}/jenkins.log + - name: Update submodule + run: | + cd ${{ env.WKC }} + git submodule update --init --recursive + - name: Run tests + run: | + date + cd ${{ env.WK }} + rm -rf debug + mkdir debug + cd ${{ env.WK }}/debug + echo $PATH + echo "PATH=/opt/homebrew/bin:$PATH" >> $GITHUB_ENV + cmake .. -DBUILD_TEST=true -DBUILD_HTTPS=false -DCMAKE_BUILD_TYPE=Release + make -j10 + ctest -j10 || exit 7 + date diff --git a/.github/workflows/taosd-doc-build.yml b/.github/workflows/taosd-doc-build.yml index 52a35fc3d5..1dffaa0aa7 100644 --- a/.github/workflows/taosd-doc-build.yml +++ b/.github/workflows/taosd-doc-build.yml @@ -1,17 +1,14 @@ name: TDengine Doc Build on: - workflow_call: - inputs: - target_branch: - description: "Target branch name of for building the document" - required: true - type: string - - target_pr_number: - description: "PR number of target branch to merge for building the document" - required: true - type: string + pull_request: + branches: + - 'main' + - '3.0' + - '3.1' + paths: + - 'docs/**' + - '*.md' env: DOC_WKC: "/root/doc_ci_work" @@ -21,81 +18,32 @@ env: TOOLS_REPO: "taos-tools" jobs: - check: - runs-on: - group: CI - labels: [self-hosted, doc-build] - outputs: - changed_files_zh: ${{ steps.set_output.outputs.changed_files_zh }} - changed_files_en: ${{ steps.set_output.outputs.changed_files_en }} - changed_files_non_doc: ${{ steps.set_output.outputs.changed_files_non_doc }} - changed_files_non_tdgpt: ${{ steps.set_output.outputs.changed_files_non_tdgpt }} - steps: - - name: Get the latest document contents from the repository - run: | - set -e - # ./.github/scripts/update_repo.sh ${{ env.DOC_WKC }}/${{ env.TD_REPO }} ${{ inputs.target_branch }} ${{ inputs.target_pr_number }} - cd ${{ env.DOC_WKC }}/${{ env.TD_REPO }} - git reset --hard - git clean -f - git remote prune origin - git fetch - git checkout ${{ inputs.target_branch }} - git pull >/dev/null - git fetch origin +refs/pull/${{ inputs.target_pr_number }}/merge - git checkout -qf FETCH_HEAD - - name: Check whether the document is changed and set output variables - id: set_output - run: | - set -e - cd ${{ env.DOC_WKC }}/${{ env.TD_REPO }} - changed_files_zh=$(git --no-pager diff --name-only FETCH_HEAD `git merge-base FETCH_HEAD ${{ inputs.tartget_branch }}`| grep "^docs/zh/" | tr '\n' ' ' || :) - changed_files_en=$(git --no-pager diff --name-only FETCH_HEAD `git merge-base FETCH_HEAD ${{ inputs.tartget_branch }}`| grep "^docs/en/" | tr '\n' ' ' || :) - changed_files_non_doc=$(git --no-pager diff --name-only FETCH_HEAD `git merge-base FETCH_HEAD ${{ inputs.tartget_branch }}`|grep -v "^docs/en/"|grep -v "^docs/zh/"|grep -v ".md$" | tr '\n' ' ' || :) - changed_files_non_tdgpt=$(git --no-pager diff --name-only FETCH_HEAD `git merge-base FETCH_HEAD ${{ inputs.tartget_branch }}`|grep -v "^docs/en/"|grep -v "^docs/zh/"|grep -v ".md$" | grep -Ev "forecastoperator.c|anomalywindowoperator.c|tanalytics.h|tanalytics.c|tdgpt_cases.task|analytics" | tr '\n' ' ' ||:) - echo "changed_files_zh=${changed_files_zh}" >> $GITHUB_OUTPUT - echo "changed_files_en=${changed_files_en}" >> $GITHUB_OUTPUT - echo "changed_files_non_doc=${changed_files_non_doc}" >> $GITHUB_OUTPUT - echo "changed_files_non_tdgpt=${changed_files_non_tdgpt}" >> $GITHUB_OUTPUT - - build: - needs: check + build-doc: runs-on: group: CI labels: [self-hosted, doc-build] - if: ${{ needs.check.outputs.changed_files_zh != '' || needs.check.outputs.changed_files_en != '' }} - steps: - name: Get the latest document contents run: | set -e - #./.github/scripts/update_repo.sh ${{ env.DOC_WKC }}/${{ env.TD_REPO }} ${{ inputs.target_branch }} ${{ inputs.target_pr_number }} cd ${{ env.DOC_WKC }}/${{ env.TD_REPO }} git reset --hard git clean -f git remote prune origin git fetch - git checkout ${{ inputs.target_branch }} + git checkout ${{ github.event.pull_request.base.ref }} git pull >/dev/null - git fetch origin +refs/pull/${{ inputs.target_pr_number }}/merge + git fetch origin +refs/pull/${{ github.event.pull_request.number }}/merge git checkout -qf FETCH_HEAD - name: Build the chinese document - if: ${{ needs.check.outputs.changed_files_zh != '' }} run: | cd ${{ env.DOC_WKC }}/${{ env.ZH_DOC_REPO }} yarn ass local yarn build - name: Build the english document - if: ${{ needs.check.outputs.changed_files_en != '' }} run: | cd ${{ env.DOC_WKC }}/${{ env.EN_DOC_REPO }} yarn ass local yarn build - - outputs: - changed_files_zh: ${{ needs.check.outputs.changed_files_zh }} - changed_files_en: ${{ needs.check.outputs.changed_files_en }} - changed_files_non_doc: ${{ needs.check.outputs.changed_files_non_doc }} - changed_files_non_tdgpt: ${{ needs.check.outputs.changed_files_non_tdgpt }} From 5f6ecab85443cd161ea13d39fc00911e0c6144ad Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Fri, 28 Feb 2025 15:05:00 +0800 Subject: [PATCH 49/57] fix(stream): check the return values. --- source/dnode/vnode/src/tq/tqStreamTask.c | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/source/dnode/vnode/src/tq/tqStreamTask.c b/source/dnode/vnode/src/tq/tqStreamTask.c index c8862b03ad..7c65f805e9 100644 --- a/source/dnode/vnode/src/tq/tqStreamTask.c +++ b/source/dnode/vnode/src/tq/tqStreamTask.c @@ -322,7 +322,10 @@ bool taskReadyForDataFromWal(SStreamTask* pTask) { // check whether input queue is full or not if (streamQueueIsFull(pTask->inputq.queue)) { tqTrace("s-task:%s input queue is full, launch task without scanning wal", pTask->id.idStr); - streamTrySchedExec(pTask); + int32_t code = streamTrySchedExec(pTask); + if (code) { + tqError("s-task:%s failed to start task while inputQ is full", pTask->id.idStr); + } return false; } From 12249bb51271356a9027eb7466f3479f1dc3b37c Mon Sep 17 00:00:00 2001 From: dmchen Date: Fri, 28 Feb 2025 16:02:18 +0800 Subject: [PATCH 50/57] fix:remove-lock-from-heartbeat-log --- source/libs/sync/inc/syncUtil.h | 9 +++ source/libs/sync/src/syncUtil.c | 133 +++++++++++++++++++++++++------- 2 files changed, 115 insertions(+), 27 deletions(-) diff --git a/source/libs/sync/inc/syncUtil.h b/source/libs/sync/inc/syncUtil.h index e7110c0964..95ea75731c 100644 --- a/source/libs/sync/inc/syncUtil.h +++ b/source/libs/sync/inc/syncUtil.h @@ -53,6 +53,13 @@ extern "C" { #define sNDebug(pNode, ...) if (sDebugFlag & DEBUG_DEBUG) { syncPrintNodeLog("SYN ", DEBUG_DEBUG, sDebugFlag, true, pNode, __VA_ARGS__); } #define sNTrace(pNode, ...) if (sDebugFlag & DEBUG_TRACE) { syncPrintNodeLog("SYN ", DEBUG_TRACE, sDebugFlag, true, pNode, __VA_ARGS__); } +#define sHFatal(pNode, ...) if (sDebugFlag & DEBUG_FATAL) { syncPrintHbLog("SYN FATAL ", DEBUG_FATAL, 255, true, pNode, __VA_ARGS__); } +#define sHError(pNode, ...) if (sDebugFlag & DEBUG_ERROR) { syncPrintHbLog("SYN ERROR ", DEBUG_ERROR, 255, true, pNode, __VA_ARGS__); } +#define sHWarn(pNode, ...) if (sDebugFlag & DEBUG_WARN) { syncPrintHbLog("SYN WARN ", DEBUG_WARN, 255, true, pNode, __VA_ARGS__); } +#define sHInfo(pNode, ...) if (sDebugFlag & DEBUG_INFO) { syncPrintHbLog("SYN ", DEBUG_INFO, 255, true, pNode, __VA_ARGS__); } +#define sHDebug(pNode, ...) if (sDebugFlag & DEBUG_DEBUG) { syncPrintHbLog("SYN ", DEBUG_DEBUG, sDebugFlag, true, pNode, __VA_ARGS__); } +#define sHTrace(pNode, ...) if (sDebugFlag & DEBUG_TRACE) { syncPrintHbLog("SYN ", DEBUG_TRACE, sDebugFlag, true, pNode, __VA_ARGS__); } + #define sSFatal(pSender, ...) if (sDebugFlag & DEBUG_FATAL) { syncPrintSnapshotSenderLog("SYN FATAL ", DEBUG_FATAL, 255, pSender, __VA_ARGS__); } #define sSError(pSender, ...) if (sDebugFlag & DEBUG_ERROR) { syncPrintSnapshotSenderLog("SYN ERROR ", DEBUG_ERROR, 255, pSender, __VA_ARGS__); } #define sSWarn(pSender, ...) if (sDebugFlag & DEBUG_WARN) { syncPrintSnapshotSenderLog("SYN WARN ", DEBUG_WARN, 255, pSender, __VA_ARGS__); } @@ -87,6 +94,8 @@ void syncUtilGenerateArbToken(int32_t nodeId, int32_t groupId, char* buf); void syncPrintNodeLog(const char* flags, ELogLevel level, int32_t dflag, bool formatTime, SSyncNode* pNode, const char* format, ...); +void syncPrintHbLog(const char* flags, ELogLevel level, int32_t dflag, bool formatTime, SSyncNode* pNode, + const char* format, ...); void syncPrintSnapshotSenderLog(const char* flags, ELogLevel level, int32_t dflag, SSyncSnapshotSender* pSender, const char* format, ...); void syncPrintSnapshotReceiverLog(const char* flags, ELogLevel level, int32_t dflag, SSyncSnapshotReceiver* pReceiver, diff --git a/source/libs/sync/src/syncUtil.c b/source/libs/sync/src/syncUtil.c index 44e8d97aca..435586041a 100644 --- a/source/libs/sync/src/syncUtil.c +++ b/source/libs/sync/src/syncUtil.c @@ -253,11 +253,9 @@ void syncPrintNodeLog(const char* flags, ELogLevel level, int32_t dflag, bool fo va_end(argpointer); int32_t aqItems = 0; - /* if (pNode != NULL && pNode->pFsm != NULL && pNode->pFsm->FpApplyQueueItems != NULL) { aqItems = pNode->pFsm->FpApplyQueueItems(pNode->pFsm); // vnodeApplyQueueItems } - */ // restore error code terrno = errCode; @@ -272,15 +270,71 @@ void syncPrintNodeLog(const char* flags, ELogLevel level, int32_t dflag, bool fo ", elect-times:%d, as-leader-times:%d, as-assigned-leader-times:%d, cfg-ch-times:%d, hb-slow:%d, hbr-slow:%d, " "aq-items:%d, snaping:%" PRId64 ", replicas:%d, last-cfg:%" PRId64 ", chging:%d, restore:%d, quorum:%d, elect-lc-timer:%" PRId64 ", hb:%" PRId64 - ", buffer:%s, repl-mgrs:%s, members:%s, send hb:%s, recv hb:%s, recv hb-reply:%s, arb-token:%s, msg[sent:%d, recv:%d, slow-recev:%d]", + ", buffer:%s, repl-mgrs:%s, members:%s, send hb:%s, recv hb:%s, recv hb-reply:%s, arb-token:%s, msg[sent:%d, " + "recv:%d, slow-recv:%d]", pNode->vgId, eventLog, syncStr(pNode->state), currentTerm, pNode->commitIndex, pNode->assignedCommitIndex, appliedIndex, logBeginIndex, logLastIndex, pNode->minMatchIndex, snapshot.lastApplyIndex, snapshot.lastApplyTerm, pNode->electNum, pNode->becomeLeaderNum, pNode->becomeAssignedLeaderNum, pNode->configChangeNum, pNode->hbSlowNum, pNode->hbrSlowNum, aqItems, pNode->snapshottingIndex, pNode->replicaNum, pNode->raftCfg.lastConfigIndex, pNode->changing, pNode->restoreFinish, syncNodeDynamicQuorum(pNode), pNode->electTimerLogicClock, pNode->heartbeatTimerLogicClockUser, bufferStatesStr, - replMgrStatesStr, cfgStr, sentHbTimeStr, hbTimeStr, hbrTimeStr, pNode->arbToken, pNode->sendCount, pNode->recvCount, - pNode->slowCount); + replMgrStatesStr, cfgStr, sentHbTimeStr, hbTimeStr, hbrTimeStr, pNode->arbToken, pNode->sendCount, + pNode->recvCount, pNode->slowCount); + } +} + +void syncPrintHbLog(const char* flags, ELogLevel level, int32_t dflag, bool formatTime, SSyncNode* pNode, + const char* format, ...) { + if (pNode == NULL || pNode->pLogStore == NULL) return; + int64_t currentTerm = raftStoreGetTerm(pNode); + + // save error code, otherwise it will be overwritten + int32_t errCode = terrno; + + int32_t cacheHit = pNode->pLogStore->cacheHit; + int32_t cacheMiss = pNode->pLogStore->cacheMiss; + + char cfgStr[1024] = ""; + syncCfg2SimpleStr(&pNode->raftCfg.cfg, cfgStr, sizeof(cfgStr)); + + char replMgrStatesStr[1024] = ""; + syncLogReplStates2Str(pNode, replMgrStatesStr, sizeof(replMgrStatesStr)); + + char bufferStatesStr[256] = ""; + syncLogBufferStates2Str(pNode, bufferStatesStr, sizeof(bufferStatesStr)); + + char hbrTimeStr[256] = ""; + syncHearbeatReplyTime2Str(pNode, hbrTimeStr, sizeof(hbrTimeStr), formatTime); + + char hbTimeStr[256] = ""; + syncHearbeatTime2Str(pNode, hbTimeStr, sizeof(hbTimeStr), formatTime); + + char sentHbTimeStr[512] = ""; + syncSentHearbeatTime2Str(pNode, sentHbTimeStr, sizeof(sentHbTimeStr), formatTime); + + char eventLog[512]; // {0}; + va_list argpointer; + va_start(argpointer, format); + int32_t writeLen = vsnprintf(eventLog, sizeof(eventLog), format, argpointer); + va_end(argpointer); + + terrno = errCode; + + if (pNode != NULL) { + taosPrintLog( + flags, level, dflag, + "vgId:%d, %s, sync:%s, term:%" PRIu64 ", commit-index:%" PRId64 ", assigned-index:%" PRId64 ", min:%" PRId64 + ", elect-times:%d, as-leader-times:%d, as-assigned-leader-times:%d, cfg-ch-times:%d, hb-slow:%d, hbr-slow:%d, " + ", snaping:%" PRId64 ", replicas:%d, last-cfg:%" PRId64 + ", chging:%d, restore:%d, quorum:%d, elect-lc-timer:%" PRId64 ", hb:%" PRId64 + ", buffer:%s, repl-mgrs:%s, members:%s, send hb:%s, recv hb:%s, recv hb-reply:%s, arb-token:%s, msg[sent:%d, " + "recv:%d, slow-recv:%d]", + pNode->vgId, eventLog, syncStr(pNode->state), currentTerm, pNode->commitIndex, pNode->assignedCommitIndex, + pNode->minMatchIndex, pNode->electNum, pNode->becomeLeaderNum, pNode->becomeAssignedLeaderNum, + pNode->configChangeNum, pNode->hbSlowNum, pNode->hbrSlowNum, pNode->snapshottingIndex, pNode->replicaNum, + pNode->raftCfg.lastConfigIndex, pNode->changing, pNode->restoreFinish, syncNodeDynamicQuorum(pNode), + pNode->electTimerLogicClock, pNode->heartbeatTimerLogicClockUser, bufferStatesStr, replMgrStatesStr, cfgStr, + sentHbTimeStr, hbTimeStr, hbrTimeStr, pNode->arbToken, pNode->sendCount, pNode->recvCount, pNode->slowCount); } } @@ -414,12 +468,12 @@ void syncLogRecvAppendEntriesReply(SSyncNode* pSyncNode, const SyncAppendEntries void syncLogSendHeartbeat(SSyncNode* pSyncNode, const SyncHeartbeat* pMsg, bool printX, int64_t timerElapsed, int64_t execTime) { if (printX) { - sNTrace(pSyncNode, + sHTrace(pSyncNode, "send sync-heartbeat to dnode:%d {term:%" PRId64 ", commit-index:%" PRId64 ", min-match:%" PRId64 ", ts:%" PRId64 "}, x", DID(&pMsg->destId), pMsg->term, pMsg->commitIndex, pMsg->minMatchIndex, pMsg->timeStamp); } else { - sNTrace(pSyncNode, + sHTrace(pSyncNode, "send sync-heartbeat to dnode:%d {term:%" PRId64 ", commit-index:%" PRId64 ", min-match:%" PRId64 ", ts:%" PRId64 "}, timer-elapsed:%" PRId64 ", next-exec:%" PRId64, DID(&pMsg->destId), pMsg->term, pMsg->commitIndex, pMsg->minMatchIndex, pMsg->timeStamp, timerElapsed, @@ -428,29 +482,39 @@ void syncLogSendHeartbeat(SSyncNode* pSyncNode, const SyncHeartbeat* pMsg, bool } void syncLogRecvHeartbeat(SSyncNode* pSyncNode, const SyncHeartbeat* pMsg, int64_t timeDiff, const char* s) { - char pBuf[TD_TIME_STR_LEN] = {0}; - if (pMsg->timeStamp > 0) { - if (formatTimestampLocal(pBuf, pMsg->timeStamp, TSDB_TIME_PRECISION_MILLI) == NULL) { - pBuf[0] = '\0'; - } - } if (timeDiff > SYNC_HEARTBEAT_SLOW_MS) { pSyncNode->hbSlowNum++; - sNTrace(pSyncNode, + char pBuf[TD_TIME_STR_LEN] = {0}; + if (pMsg->timeStamp > 0) { + if (formatTimestampLocal(pBuf, pMsg->timeStamp, TSDB_TIME_PRECISION_MILLI) == NULL) { + pBuf[0] = '\0'; + } + } + + sHError(pSyncNode, "recv sync-heartbeat from dnode:%d slow(%d ms) {term:%" PRId64 ", commit-index:%" PRId64 ", min-match:%" PRId64 ", ts:%s}, QID:%s, net elapsed:%" PRId64 "ms", - DID(&pMsg->srcId), SYNC_HEARTBEAT_SLOW_MS, pMsg->term, pMsg->commitIndex, pMsg->minMatchIndex, pBuf, s, timeDiff); + DID(&pMsg->srcId), SYNC_HEARTBEAT_SLOW_MS, pMsg->term, pMsg->commitIndex, pMsg->minMatchIndex, pBuf, s, + timeDiff); } else { - sNTrace(pSyncNode, - "recv sync-heartbeat from dnode:%d {term:%" PRId64 ", commit-index:%" PRId64 ", min-match:%" PRId64 - ", ts:%s}, QID:%s, net elapsed:%" PRId64 "ms", - DID(&pMsg->srcId), pMsg->term, pMsg->commitIndex, pMsg->minMatchIndex, pBuf, s, timeDiff); + if (sDebugFlag & DEBUG_TRACE) { + char pBuf[TD_TIME_STR_LEN] = {0}; + if (pMsg->timeStamp > 0) { + if (formatTimestampLocal(pBuf, pMsg->timeStamp, TSDB_TIME_PRECISION_MILLI) == NULL) { + pBuf[0] = '\0'; + } + } + sHTrace(pSyncNode, + "recv sync-heartbeat from dnode:%d {term:%" PRId64 ", commit-index:%" PRId64 ", min-match:%" PRId64 + ", ts:%s}, QID:%s, net elapsed:%" PRId64 "ms", + DID(&pMsg->srcId), pMsg->term, pMsg->commitIndex, pMsg->minMatchIndex, pBuf, s, timeDiff); + } } } void syncLogSendHeartbeatReply(SSyncNode* pSyncNode, const SyncHeartbeatReply* pMsg, const char* s) { - sNTrace(pSyncNode, "send sync-heartbeat-reply from dnode:%d {term:%" PRId64 ", ts:%" PRId64 "}, %s", + sHTrace(pSyncNode, "send sync-heartbeat-reply from dnode:%d {term:%" PRId64 ", ts:%" PRId64 "}, %s", DID(&pMsg->destId), pMsg->term, pMsg->timeStamp, s); } @@ -458,14 +522,29 @@ void syncLogRecvHeartbeatReply(SSyncNode* pSyncNode, const SyncHeartbeatReply* p if (timeDiff > SYNC_HEARTBEAT_REPLY_SLOW_MS) { pSyncNode->hbrSlowNum++; - sNTrace(pSyncNode, - "recv sync-heartbeat-reply from dnode:%d slow {term:%" PRId64 ", ts:%" PRId64 "}, %s, net elapsed:%" PRId64, - DID(&pMsg->srcId), pMsg->term, pMsg->timeStamp, s, timeDiff); - } + char pBuf[TD_TIME_STR_LEN] = {0}; + if (pMsg->timeStamp > 0) { + if (formatTimestampLocal(pBuf, pMsg->timeStamp, TSDB_TIME_PRECISION_MILLI) == NULL) { + pBuf[0] = '\0'; + } + } - sNTrace(pSyncNode, - "recv sync-heartbeat-reply from dnode:%d {term:%" PRId64 ", ts:%" PRId64 "}, %s, net elapsed:%" PRId64, - DID(&pMsg->srcId), pMsg->term, pMsg->timeStamp, s, timeDiff); + sHError(pSyncNode, + "recv sync-heartbeat-reply from dnode:%d slow(%d ms) {term:%" PRId64 ", ts:%s}, %s, net elapsed:%" PRId64, + DID(&pMsg->srcId), SYNC_HEARTBEAT_REPLY_SLOW_MS, pMsg->term, pBuf, s, timeDiff); + } else { + if (sDebugFlag & DEBUG_TRACE) { + char pBuf[TD_TIME_STR_LEN] = {0}; + if (pMsg->timeStamp > 0) { + if (formatTimestampLocal(pBuf, pMsg->timeStamp, TSDB_TIME_PRECISION_MILLI) == NULL) { + pBuf[0] = '\0'; + } + } + sHTrace(pSyncNode, + "recv sync-heartbeat-reply from dnode:%d {term:%" PRId64 ", ts:%" PRId64 "}, %s, net elapsed:%" PRId64, + DID(&pMsg->srcId), pMsg->term, pMsg->timeStamp, s, timeDiff); + } + } } void syncLogSendSyncSnapshotSend(SSyncNode* pSyncNode, const SyncSnapshotSend* pMsg, const char* s) { From 887444669c346ea1e71c97bc9131cc6cba5a1e1b Mon Sep 17 00:00:00 2001 From: facetosea <285808407@qq.com> Date: Fri, 28 Feb 2025 15:44:45 +0800 Subject: [PATCH 51/57] fix: error log --- source/client/src/clientImpl.c | 1 + source/client/src/clientMain.c | 1 + source/client/src/clientMsgHandler.c | 2 + source/libs/qcom/src/querymsg.c | 105 ++++++++++++-------------- source/libs/scheduler/src/scheduler.c | 1 + 5 files changed, 52 insertions(+), 58 deletions(-) diff --git a/source/client/src/clientImpl.c b/source/client/src/clientImpl.c index c2a199e9c1..e1817f3e19 100644 --- a/source/client/src/clientImpl.c +++ b/source/client/src/clientImpl.c @@ -2232,6 +2232,7 @@ static int32_t doConvertJson(SReqResultInfo* pResultInfo) { int32_t blockVersion = *(int32_t*)p; int32_t dataLen = estimateJsonLen(pResultInfo); if (dataLen <= 0) { + tscError("doConvertJson error: estimateJsonLen failed"); return TSDB_CODE_TSC_INTERNAL_ERROR; } diff --git a/source/client/src/clientMain.c b/source/client/src/clientMain.c index 27e21d9a58..8c867f5160 100644 --- a/source/client/src/clientMain.c +++ b/source/client/src/clientMain.c @@ -888,6 +888,7 @@ int taos_select_db(TAOS *taos, const char *db) { if (db == NULL || strlen(db) == 0) { releaseTscObj(*(int64_t *)taos); + tscError("invalid parameter for %s", db == NULL ? "db is NULL" : "db is empty"); terrno = TSDB_CODE_TSC_INVALID_INPUT; return terrno; } diff --git a/source/client/src/clientMsgHandler.c b/source/client/src/clientMsgHandler.c index 58ba39864d..a9f5472d1f 100644 --- a/source/client/src/clientMsgHandler.c +++ b/source/client/src/clientMsgHandler.c @@ -372,11 +372,13 @@ int32_t processUseDbRsp(void* param, SDataBuf* pMsg, int32_t code) { int32_t processCreateSTableRsp(void* param, SDataBuf* pMsg, int32_t code) { if (pMsg == NULL) { + tscError("processCreateSTableRsp: invalid input param, pMsg is NULL"); return TSDB_CODE_TSC_INVALID_INPUT; } if (param == NULL) { taosMemoryFree(pMsg->pEpSet); taosMemoryFree(pMsg->pData); + tscError("processCreateSTableRsp: invalid input param, param is NULL"); return TSDB_CODE_TSC_INVALID_INPUT; } diff --git a/source/libs/qcom/src/querymsg.c b/source/libs/qcom/src/querymsg.c index 0ee61726e3..b72edba483 100644 --- a/source/libs/qcom/src/querymsg.c +++ b/source/libs/qcom/src/querymsg.c @@ -91,10 +91,8 @@ int32_t queryBuildTableMetaReqMsg(void *input, char **msg, int32_t msgSize, int3 if (NULL == pBuf) { return terrno; } - if(tSerializeSTableInfoReq(pBuf, bufLen, &infoReq) < 0) - { - return TSDB_CODE_TSC_INVALID_INPUT; - } + int32_t ret = tSerializeSTableInfoReq(pBuf, bufLen, &infoReq); + if (ret < 0) return ret; *msg = pBuf; *msgLen = bufLen; @@ -121,10 +119,8 @@ int32_t queryBuildUseDbMsg(void *input, char **msg, int32_t msgSize, int32_t *ms if (NULL == pBuf) { return terrno; } - if(tSerializeSUseDbReq(pBuf, bufLen, &usedbReq) < 0) - { - return TSDB_CODE_TSC_INVALID_INPUT; - } + int32_t ret = tSerializeSUseDbReq(pBuf, bufLen, &usedbReq); + if (ret < 0) return ret; *msg = pBuf; *msgLen = bufLen; @@ -144,10 +140,9 @@ int32_t queryBuildQnodeListMsg(void *input, char **msg, int32_t msgSize, int32_t if (NULL == pBuf) { return terrno; } - if(tSerializeSQnodeListReq(pBuf, bufLen, &qnodeListReq) < 0) - { - return TSDB_CODE_TSC_INVALID_INPUT; - } + + int32_t ret = tSerializeSQnodeListReq(pBuf, bufLen, &qnodeListReq); + if (ret < 0) return ret; *msg = pBuf; *msgLen = bufLen; @@ -167,10 +162,8 @@ int32_t queryBuildDnodeListMsg(void *input, char **msg, int32_t msgSize, int32_t if (NULL == pBuf) { return terrno; } - if(tSerializeSDnodeListReq(pBuf, bufLen, &dnodeListReq) < 0) - { - return TSDB_CODE_TSC_INVALID_INPUT; - } + int32_t ret = tSerializeSDnodeListReq(pBuf, bufLen, &dnodeListReq); + if (ret < 0) return ret; *msg = pBuf; *msgLen = bufLen; @@ -189,10 +182,8 @@ int32_t queryBuildGetSerVerMsg(void *input, char **msg, int32_t msgSize, int32_t if (NULL == pBuf) { return terrno; } - if(tSerializeSServerVerReq(pBuf, bufLen, &req) < 0) - { - return TSDB_CODE_TSC_INVALID_INPUT; - } + int32_t ret = tSerializeSServerVerReq(pBuf, bufLen, &req); + if (ret < 0) return ret; *msg = pBuf; *msgLen = bufLen; @@ -213,10 +204,8 @@ int32_t queryBuildGetDBCfgMsg(void *input, char **msg, int32_t msgSize, int32_t if (NULL == pBuf) { return terrno; } - if(tSerializeSDbCfgReq(pBuf, bufLen, &dbCfgReq) < 0) - { - return TSDB_CODE_TSC_INVALID_INPUT; - } + int32_t ret = tSerializeSDbCfgReq(pBuf, bufLen, &dbCfgReq); + if (ret < 0) return ret; *msg = pBuf; *msgLen = bufLen; @@ -237,10 +226,8 @@ int32_t queryBuildGetIndexMsg(void *input, char **msg, int32_t msgSize, int32_t if (NULL == pBuf) { return terrno; } - if(tSerializeSUserIndexReq(pBuf, bufLen, &indexReq) < 0) - { - return TSDB_CODE_TSC_INVALID_INPUT; - } + int32_t ret = tSerializeSUserIndexReq(pBuf, bufLen, &indexReq); + if (ret < 0) return ret; *msg = pBuf; *msgLen = bufLen; @@ -272,10 +259,10 @@ int32_t queryBuildRetrieveFuncMsg(void *input, char **msg, int32_t msgSize, int3 taosArrayDestroy(funcReq.pFuncNames); return terrno; } - if(tSerializeSRetrieveFuncReq(pBuf, bufLen, &funcReq) < 0) - { + int32_t ret = tSerializeSRetrieveFuncReq(pBuf, bufLen, &funcReq); + if (ret < 0) { taosArrayDestroy(funcReq.pFuncNames); - return TSDB_CODE_TSC_INVALID_INPUT; + return ret; } taosArrayDestroy(funcReq.pFuncNames); @@ -299,9 +286,8 @@ int32_t queryBuildGetUserAuthMsg(void *input, char **msg, int32_t msgSize, int32 if (NULL == pBuf) { return terrno; } - if (tSerializeSGetUserAuthReq(pBuf, bufLen, &req) < 0) { - return TSDB_CODE_TSC_INVALID_INPUT; - } + int32_t ret = tSerializeSGetUserAuthReq(pBuf, bufLen, &req); + if (ret < 0) return ret; *msg = pBuf; *msgLen = bufLen; @@ -322,10 +308,8 @@ int32_t queryBuildGetTbIndexMsg(void *input, char **msg, int32_t msgSize, int32_ if (NULL == pBuf) { return terrno; } - if(tSerializeSTableIndexReq(pBuf, bufLen, &indexReq) < 0) - { - return TSDB_CODE_TSC_INVALID_INPUT; - } + int32_t ret = tSerializeSTableIndexReq(pBuf, bufLen, &indexReq); + if (ret < 0) return ret; *msg = pBuf; *msgLen = bufLen; @@ -349,10 +333,8 @@ int32_t queryBuildGetTbCfgMsg(void *input, char **msg, int32_t msgSize, int32_t if (NULL == pBuf) { return terrno; } - if(tSerializeSTableCfgReq(pBuf, bufLen, &cfgReq) < 0) - { - return TSDB_CODE_TSC_INVALID_INPUT; - } + int32_t ret = tSerializeSTableCfgReq(pBuf, bufLen, &cfgReq); + if (ret < 0) return ret; *msg = pBuf; *msgLen = bufLen; @@ -373,10 +355,8 @@ int32_t queryBuildGetViewMetaMsg(void *input, char **msg, int32_t msgSize, int32 if (NULL == pBuf) { return terrno; } - if(tSerializeSViewMetaReq(pBuf, bufLen, &req) < 0) - { - return TSDB_CODE_TSC_INVALID_INPUT; - } + int32_t ret = tSerializeSViewMetaReq(pBuf, bufLen, &req); + if (ret < 0) return ret; *msg = pBuf; *msgLen = bufLen; @@ -398,10 +378,8 @@ int32_t queryBuildGetTableTSMAMsg(void *input, char **msg, int32_t msgSize, int3 if (NULL == pBuf) { return terrno; } - if(tSerializeTableTSMAInfoReq(pBuf, bufLen, &req) < 0) - { - return TSDB_CODE_TSC_INVALID_INPUT; - } + int32_t ret = tSerializeTableTSMAInfoReq(pBuf, bufLen, &req); + if (ret < 0) return ret; *msg = pBuf; *msgLen = bufLen; @@ -424,10 +402,8 @@ int32_t queryBuildGetTSMAMsg(void *input, char **msg, int32_t msgSize, int32_t * { return terrno; } - if(tSerializeTableTSMAInfoReq(pBuf, bufLen, &req) < 0) - { - return TSDB_CODE_TSC_INVALID_INPUT; - } + int32_t ret = tSerializeTableTSMAInfoReq(pBuf, bufLen, &req); + if(ret < 0) return ret; *msg = pBuf; *msgLen = bufLen; @@ -445,10 +421,8 @@ int32_t queryBuildGetStreamProgressMsg(void* input, char** msg, int32_t msgSize, return terrno; } - if(tSerializeStreamProgressReq(pBuf, len, input) < 0) - { - return TSDB_CODE_TSC_INVALID_INPUT; - } + int32_t ret = tSerializeStreamProgressReq(pBuf, len, input); + if (ret < 0) return ret; *msg = pBuf; *msgLen = len; @@ -462,6 +436,7 @@ int32_t queryProcessUseDBRsp(void *output, char *msg, int32_t msgSize) { if (NULL == output || NULL == msg || msgSize <= 0) { code = TSDB_CODE_TSC_INVALID_INPUT; + qError("invalid input param, output:%p, msg:%p, msgSize:%d", output, msg, msgSize); goto PROCESS_USEDB_OVER; } @@ -673,6 +648,7 @@ int32_t queryProcessTableMetaRsp(void *output, char *msg, int32_t msgSize) { STableMetaRsp metaRsp = {0}; if (NULL == output || NULL == msg || msgSize <= 0) { + qError("queryProcessTableMetaRsp: invalid input param, output:%p, msg:%p, msgSize:%d", output, msg, msgSize); code = TSDB_CODE_TSC_INVALID_INPUT; goto PROCESS_META_OVER; } @@ -729,6 +705,7 @@ static int32_t queryProcessTableNameRsp(void *output, char *msg, int32_t msgSize STableMetaRsp metaRsp = {0}; if (NULL == output || NULL == msg || msgSize <= 0) { + qError("queryProcessTableNameRsp: invalid input param, output:%p, msg:%p, msgSize:%d", output, msg, msgSize); code = TSDB_CODE_TSC_INVALID_INPUT; goto PROCESS_NAME_OVER; } @@ -785,6 +762,7 @@ int32_t queryProcessQnodeListRsp(void *output, char *msg, int32_t msgSize) { int32_t code = 0; if (NULL == output || NULL == msg || msgSize <= 0) { + qError("queryProcessQnodeListRsp: invalid input param, output:%p, msg:%p, msgSize:%d", output, msg, msgSize); code = TSDB_CODE_TSC_INVALID_INPUT; return code; } @@ -804,6 +782,7 @@ int32_t queryProcessDnodeListRsp(void *output, char *msg, int32_t msgSize) { int32_t code = 0; if (NULL == output || NULL == msg || msgSize <= 0) { + qError("queryProcessDnodeListRsp: invalid input param, output:%p, msg:%p, msgSize:%d", output, msg, msgSize); code = TSDB_CODE_TSC_INVALID_INPUT; return code; } @@ -824,6 +803,7 @@ int32_t queryProcessGetSerVerRsp(void *output, char *msg, int32_t msgSize) { int32_t code = 0; if (NULL == output || NULL == msg || msgSize <= 0) { + qError("queryProcessGetSerVerRsp: invalid input param, output:%p, msg:%p, msgSize:%d", output, msg, msgSize); code = TSDB_CODE_TSC_INVALID_INPUT; return code; } @@ -846,6 +826,7 @@ int32_t queryProcessGetDbCfgRsp(void *output, char *msg, int32_t msgSize) { SDbCfgRsp out = {0}; if (NULL == output || NULL == msg || msgSize <= 0) { + qError("queryProcessGetDbCfgRsp: invalid input param, output:%p, msg:%p, msgSize:%d", output, msg, msgSize); return TSDB_CODE_TSC_INVALID_INPUT; } @@ -863,6 +844,7 @@ int32_t queryProcessGetIndexRsp(void *output, char *msg, int32_t msgSize) { SUserIndexRsp out = {0}; if (NULL == output || NULL == msg || msgSize <= 0) { + qError("queryProcessGetIndexRsp: invalid input param, output:%p, msg:%p, msgSize:%d", output, msg, msgSize); return TSDB_CODE_TSC_INVALID_INPUT; } @@ -880,6 +862,7 @@ int32_t queryProcessRetrieveFuncRsp(void *output, char *msg, int32_t msgSize) { SRetrieveFuncRsp out = {0}; if (NULL == output || NULL == msg || msgSize <= 0) { + qError("queryProcessRetrieveFuncRsp: invalid input param, output:%p, msg:%p, msgSize:%d", output, msg, msgSize); return TSDB_CODE_TSC_INVALID_INPUT; } @@ -904,6 +887,7 @@ int32_t queryProcessRetrieveFuncRsp(void *output, char *msg, int32_t msgSize) { int32_t queryProcessGetUserAuthRsp(void *output, char *msg, int32_t msgSize) { if (NULL == output || NULL == msg || msgSize <= 0) { + qError("queryProcessGetUserAuthRsp: invalid input param, output:%p, msg:%p, msgSize:%d", output, msg, msgSize); return TSDB_CODE_TSC_INVALID_INPUT; } @@ -917,6 +901,7 @@ int32_t queryProcessGetUserAuthRsp(void *output, char *msg, int32_t msgSize) { int32_t queryProcessGetTbIndexRsp(void *output, char *msg, int32_t msgSize) { if (NULL == output || NULL == msg || msgSize <= 0) { + qError("queryProcessGetTbIndexRsp: invalid input param, output:%p, msg:%p, msgSize:%d", output, msg, msgSize); return TSDB_CODE_TSC_INVALID_INPUT; } @@ -931,6 +916,7 @@ int32_t queryProcessGetTbIndexRsp(void *output, char *msg, int32_t msgSize) { int32_t queryProcessGetTbCfgRsp(void *output, char *msg, int32_t msgSize) { if (NULL == output || NULL == msg || msgSize <= 0) { + qError("queryProcessGetTbCfgRsp: invalid input param, output:%p, msg:%p, msgSize:%d", output, msg, msgSize); return TSDB_CODE_TSC_INVALID_INPUT; } @@ -952,6 +938,7 @@ int32_t queryProcessGetTbCfgRsp(void *output, char *msg, int32_t msgSize) { int32_t queryProcessGetViewMetaRsp(void *output, char *msg, int32_t msgSize) { if (NULL == output || NULL == msg || msgSize <= 0) { + qError("queryProcessGetViewMetaRsp: invalid input param, output:%p, msg:%p, msgSize:%d", output, msg, msgSize); return TSDB_CODE_TSC_INVALID_INPUT; } @@ -975,6 +962,7 @@ int32_t queryProcessGetViewMetaRsp(void *output, char *msg, int32_t msgSize) { int32_t queryProcessGetTbTSMARsp(void* output, char* msg, int32_t msgSize) { if (NULL == output || NULL == msg || msgSize <= 0) { + qError("queryProcessGetTbTSMARsp: invalid input param, output:%p, msg:%p, msgSize:%d", output, msg, msgSize); return TSDB_CODE_TSC_INVALID_INPUT; } @@ -988,6 +976,7 @@ int32_t queryProcessGetTbTSMARsp(void* output, char* msg, int32_t msgSize) { int32_t queryProcessStreamProgressRsp(void* output, char* msg, int32_t msgSize) { if (!output || !msg || msgSize <= 0) { + qError("queryProcessStreamProgressRsp: invalid input param, output:%p, msg:%p, msgSize:%d", output, msg, msgSize); return TSDB_CODE_TSC_INVALID_INPUT; } diff --git a/source/libs/scheduler/src/scheduler.c b/source/libs/scheduler/src/scheduler.c index 21659d9cd5..c2c6116250 100644 --- a/source/libs/scheduler/src/scheduler.c +++ b/source/libs/scheduler/src/scheduler.c @@ -148,6 +148,7 @@ int32_t schedulerUpdatePolicy(int32_t policy) { qDebug("schedule policy updated to %d", schMgmt.cfg.schPolicy); break; default: + qError("invalid schedule policy %d", policy); SCH_RET(TSDB_CODE_TSC_INVALID_INPUT); } From d2c505a9b9ecca32cf4954d3222f10979114280e Mon Sep 17 00:00:00 2001 From: dmchen Date: Fri, 28 Feb 2025 17:32:54 +0800 Subject: [PATCH 52/57] fix:remove-lock-from-log-sync-heartbeat --- source/libs/sync/inc/syncRaftStore.h | 1 + source/libs/sync/src/syncRaftStore.c | 10 ++++++ source/libs/sync/src/syncUtil.c | 31 ++++++++++++------- tests/system-test/6-cluster/5dnode2mnode.py | 1 + .../4dnode1mnode_basic_createDb_replica1.py | 1 + 5 files changed, 32 insertions(+), 12 deletions(-) diff --git a/source/libs/sync/inc/syncRaftStore.h b/source/libs/sync/inc/syncRaftStore.h index 38a8ed234b..f45dccffd8 100644 --- a/source/libs/sync/inc/syncRaftStore.h +++ b/source/libs/sync/inc/syncRaftStore.h @@ -35,6 +35,7 @@ void raftStoreClearVote(SSyncNode *pNode); void raftStoreNextTerm(SSyncNode *pNode); void raftStoreSetTerm(SSyncNode *pNode, SyncTerm term); SyncTerm raftStoreGetTerm(SSyncNode *pNode); +SyncTerm raftStoreTryGetTerm(SSyncNode *pNode); #ifdef __cplusplus } diff --git a/source/libs/sync/src/syncRaftStore.c b/source/libs/sync/src/syncRaftStore.c index c61be4356c..f2f0bf35c2 100644 --- a/source/libs/sync/src/syncRaftStore.c +++ b/source/libs/sync/src/syncRaftStore.c @@ -213,3 +213,13 @@ SyncTerm raftStoreGetTerm(SSyncNode *pNode) { (void)taosThreadMutexUnlock(&pNode->raftStore.mutex); return term; } + +SyncTerm raftStoreTryGetTerm(SSyncNode *pNode) { + SyncTerm term = 0; + if (taosThreadMutexTryLock(&pNode->raftStore.mutex) == 0) { + term = pNode->raftStore.currentTerm; + (void)taosThreadMutexUnlock(&pNode->raftStore.mutex); + } + + return term; +} diff --git a/source/libs/sync/src/syncUtil.c b/source/libs/sync/src/syncUtil.c index 435586041a..b20f0d8aeb 100644 --- a/source/libs/sync/src/syncUtil.c +++ b/source/libs/sync/src/syncUtil.c @@ -286,7 +286,7 @@ void syncPrintNodeLog(const char* flags, ELogLevel level, int32_t dflag, bool fo void syncPrintHbLog(const char* flags, ELogLevel level, int32_t dflag, bool formatTime, SSyncNode* pNode, const char* format, ...) { if (pNode == NULL || pNode->pLogStore == NULL) return; - int64_t currentTerm = raftStoreGetTerm(pNode); + int64_t currentTerm = raftStoreTryGetTerm(pNode); // save error code, otherwise it will be overwritten int32_t errCode = terrno; @@ -467,17 +467,24 @@ void syncLogRecvAppendEntriesReply(SSyncNode* pSyncNode, const SyncAppendEntries void syncLogSendHeartbeat(SSyncNode* pSyncNode, const SyncHeartbeat* pMsg, bool printX, int64_t timerElapsed, int64_t execTime) { - if (printX) { - sHTrace(pSyncNode, - "send sync-heartbeat to dnode:%d {term:%" PRId64 ", commit-index:%" PRId64 ", min-match:%" PRId64 - ", ts:%" PRId64 "}, x", - DID(&pMsg->destId), pMsg->term, pMsg->commitIndex, pMsg->minMatchIndex, pMsg->timeStamp); - } else { - sHTrace(pSyncNode, - "send sync-heartbeat to dnode:%d {term:%" PRId64 ", commit-index:%" PRId64 ", min-match:%" PRId64 - ", ts:%" PRId64 "}, timer-elapsed:%" PRId64 ", next-exec:%" PRId64, - DID(&pMsg->destId), pMsg->term, pMsg->commitIndex, pMsg->minMatchIndex, pMsg->timeStamp, timerElapsed, - execTime); + if (sDebugFlag & DEBUG_TRACE) { + char pBuf[TD_TIME_STR_LEN] = {0}; + if (pMsg->timeStamp > 0) { + if (formatTimestampLocal(pBuf, pMsg->timeStamp, TSDB_TIME_PRECISION_MILLI) == NULL) { + pBuf[0] = '\0'; + } + } + if (printX) { + sHTrace(pSyncNode, + "send sync-heartbeat to dnode:%d {term:%" PRId64 ", commit-index:%" PRId64 ", min-match:%" PRId64 + ", ts:%s}, x", + DID(&pMsg->destId), pMsg->term, pMsg->commitIndex, pMsg->minMatchIndex, pBuf); + } else { + sHTrace(pSyncNode, + "send sync-heartbeat to dnode:%d {term:%" PRId64 ", commit-index:%" PRId64 ", min-match:%" PRId64 + ", ts:%s}, timer-elapsed:%" PRId64 ", next-exec:%" PRId64, + DID(&pMsg->destId), pMsg->term, pMsg->commitIndex, pMsg->minMatchIndex, pBuf, timerElapsed, execTime); + } } } diff --git a/tests/system-test/6-cluster/5dnode2mnode.py b/tests/system-test/6-cluster/5dnode2mnode.py index aa9c3fc053..82e27e6dc4 100644 --- a/tests/system-test/6-cluster/5dnode2mnode.py +++ b/tests/system-test/6-cluster/5dnode2mnode.py @@ -48,6 +48,7 @@ class TDTestCase: tdSql.checkData(4,1,'%s:6430'%self.host) tdSql.checkData(0,4,'ready') tdSql.checkData(4,4,'ready') + time.sleep(1) tdSql.query("select * from information_schema.ins_mnodes;") tdSql.checkData(0,1,'%s:6030'%self.host) tdSql.checkData(0,2,'leader') diff --git a/tests/system-test/6-cluster/vnode/4dnode1mnode_basic_createDb_replica1.py b/tests/system-test/6-cluster/vnode/4dnode1mnode_basic_createDb_replica1.py index fb00fc0846..ab4d2ef990 100644 --- a/tests/system-test/6-cluster/vnode/4dnode1mnode_basic_createDb_replica1.py +++ b/tests/system-test/6-cluster/vnode/4dnode1mnode_basic_createDb_replica1.py @@ -42,6 +42,7 @@ class TDTestCase: return buildPath def check_setup_cluster_status(self): + time.sleep(1) tdSql.query("select * from information_schema.ins_mnodes") for mnode in tdSql.queryResult: name = mnode[1] From d7560846b168036512a23dcd8621bef7f0b966d1 Mon Sep 17 00:00:00 2001 From: WANG Xu Date: Fri, 28 Feb 2025 18:07:10 +0800 Subject: [PATCH 53/57] Update .github/workflows/taosd-ci.yml --- .github/workflows/taosd-ci.yml | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/.github/workflows/taosd-ci.yml b/.github/workflows/taosd-ci.yml index 022e4bcab2..5be958adf7 100644 --- a/.github/workflows/taosd-ci.yml +++ b/.github/workflows/taosd-ci.yml @@ -1,5 +1,4 @@ - -name: TDengine CI Pipeline +name: TDengine CI Test on: pull_request: From de472b755d732921fd04e508d7e93787e5808f1d Mon Sep 17 00:00:00 2001 From: Ping Xiao Date: Fri, 28 Feb 2025 20:35:00 +0800 Subject: [PATCH 54/57] test:update 3.3.5.8 release notes in office web --- cmake/cmake.version | 2 +- docs/en/28-releases/01-tdengine.md | 4 ++ docs/en/28-releases/03-notes/3.3.5.8.md | 66 +++++++++++++++++++++++++ docs/zh/28-releases/01-tdengine.md | 4 ++ docs/zh/28-releases/03-notes/3.3.5.8.md | 65 ++++++++++++++++++++++++ 5 files changed, 140 insertions(+), 1 deletion(-) create mode 100755 docs/en/28-releases/03-notes/3.3.5.8.md create mode 100755 docs/zh/28-releases/03-notes/3.3.5.8.md diff --git a/cmake/cmake.version b/cmake/cmake.version index 52730e84e5..622bd49fdf 100644 --- a/cmake/cmake.version +++ b/cmake/cmake.version @@ -2,7 +2,7 @@ IF (DEFINED VERNUMBER) SET(TD_VER_NUMBER ${VERNUMBER}) ELSE () - SET(TD_VER_NUMBER "3.3.5.2.alpha") + SET(TD_VER_NUMBER "3.3.5.8.alpha") ENDIF () IF (DEFINED VERCOMPATIBLE) diff --git a/docs/en/28-releases/01-tdengine.md b/docs/en/28-releases/01-tdengine.md index 9f4246c7a0..b2bfb30cfa 100644 --- a/docs/en/28-releases/01-tdengine.md +++ b/docs/en/28-releases/01-tdengine.md @@ -25,6 +25,10 @@ Download links for TDengine 3.x version installation packages are as follows: import Release from "/components/ReleaseV3"; +## 3.3.5.8 + + + ## 3.3.5.2 diff --git a/docs/en/28-releases/03-notes/3.3.5.8.md b/docs/en/28-releases/03-notes/3.3.5.8.md new file mode 100755 index 0000000000..be75272545 --- /dev/null +++ b/docs/en/28-releases/03-notes/3.3.5.8.md @@ -0,0 +1,66 @@ +--- +title: TDengine 3.3.5.8 Release Notes +sidebar_label: 3.3.5.8 +description: Version 3.3.5.8 Notes +slug: /release-history/release-notes/3.3.5.8 +--- + +## Features + 1. feat: suppport tmq subscription with ONLY META in JDBC + 2. feat: support multiple-line SQL editor in Grafana + 3. feat: add support for VARBINARY/GEOMETRY in ODBC + 4. feat: support TDengine with ODBC dirver in Excel + 5. feat: taosX agent use specific port range in local connection + +## Enhancements + 1. enh: websocket handle consumer error when tmq polled nothing + 2. enh: JDBC add support for unsigned integers + 3. enh: expose global.written_concurrent configuration for kafka/mqtt/csv in Explorer + 4. enh: support integration with TDgpt in community version + 5. enh: support BinaryRowData type in flink + 6. enh: in stmt2 SQL statements, the LIMIT clause supports the use of ? as a parameter placeholder + 7. enh: enable compression via websocket in taosX backup + 8. enh: ODBC support SQL_ROWSET_SIZE in SQLSetStmtAttr + 9. enh: expose num.of.consumers/writters configurations in Explorer + 10. enh: Add connector files to the macOS installation package. + 11. enh: handle errors when poll result is null in rust connector + 12. enh: tsbs support csv output format + 13. enh: add Classified Connections Counts table in TDinsight + 14. enh: use consist float precision in explorer and tao shell + 15. enh: flink table support update/delete + 16. enh: taosX agent will resume connection when taosX server disconnected for long time + +## Fixes + 1. fix: explorer support signup email with dot `.` + 2. fix: flock syscall error on aws cloud storage in taosAdapter + 3. fix: modify boolean tag values in sub-tables results in erroneous metadata from data subscriptions. + 4. fix: allow spaces in columns of csv in explorer datain + 5. fix: resolved the issue of high CPU usage by the stmtbind thread when the system is in an idle state + 6. fix: health state tick to idle when no data consumed + 7. fix: fix security issues in JDBC sample code + 8. fix: fix upgrade compaibility issue of taosX + 9. fix: ODBC core when set SQL_ATTR_TXN_ISOLATION with SQLSetConnectAttr + 10. fix: received/processed_messages should be reset when task rerun + 11. fix: when restoring data using taosX, it may crash if the database is not specified + 12. fix: when creating a database, the keep_time_offset options supports suffixes h (hours) and d (days) for time values + 13. fix: potential deadlocks while drop stream + 14. fix: failed to write data in a dual-replica database when a single dnode is disconnected from the network + 15. fix: when querying the information_schema.ins_tables table, a "Sync leader is unreachable" error may be triggered if the Leader of the mnode changes. + 16. fix: the time-filtering query results involving composite primary keys were incorrect after data compact + 17. fix: when the join condition of the primary key column is not a simple equality condition, it may lead to incorrect JOIN results + 18. fix: error caused by cursor.fetchmany with custom length in python taosws + 19. fix: the issue where the "show grants" command returned an incorrect number of columns + 20. fix: unexpected backup points before schedule executing + 21. fix: taosX task does not restart after interrupted + 22. fix: jdbc select server_version() caused mem high-usage + 23. fix: when using the WHERE tbname IN () statement, executing LAST query may cause taosd crash if the subtables filtered out do not belong to the same super table + 24. fix: after taosd exits abnormally and is restarted, if the WAL that has not been written to the data file is too large, it may cause an OOM error during startup + 25. fix: when using interp interpolation, if the select list contains string constants or string tags, the returned string content may be incomplete.[#29353](https://github.com/taosdata/TDengine/issues/29353) + 26. fix: when performing a JOIN query on a super table, using a subquery as the right table may lead to missing results + 27. fix: syntax error while use DISTINCT and ORDER BY together.[#29263](https://github.com/taosdata/TDengine/issues/29263) + 28. fix: when using the CAST function to convert a floating-point number to a binary and then performing a comparison, the result may be inaccurate due to loss of precision[#29382](https://github.com/taosdata/TDengine/issues/29382) + 29. fix: after upgrading from version 3.3.4 to 3.3.5, the taosd service fails to start properly if the configured charset does not exist in the system + 30. fix: websocket api timing field should not be negtive + 31. fix: duplicates backup points in taosX + 32. fix: configuration item s3BucketName was incorrectly set as a global variable, leading to failures while file uploads to S3. + diff --git a/docs/zh/28-releases/01-tdengine.md b/docs/zh/28-releases/01-tdengine.md index 88c07a89f4..435d5819ce 100644 --- a/docs/zh/28-releases/01-tdengine.md +++ b/docs/zh/28-releases/01-tdengine.md @@ -24,6 +24,10 @@ TDengine 3.x 各版本安装包下载链接如下: import Release from "/components/ReleaseV3"; +## 3.3.5.8 + + + ## 3.3.5.2 diff --git a/docs/zh/28-releases/03-notes/3.3.5.8.md b/docs/zh/28-releases/03-notes/3.3.5.8.md new file mode 100755 index 0000000000..07f9cf060b --- /dev/null +++ b/docs/zh/28-releases/03-notes/3.3.5.8.md @@ -0,0 +1,65 @@ +--- +title: 3.3.5.8 版本说明 +sidebar_label: 3.3.5.8 +description: 3.3.5.8 版本说明 +--- + +## 特性 + 1. 特性:JDBC 支持 ONLY META 订阅 + 2. 特性:Grafana 插件 SQL 编辑支持多行 + 3. 特性:ODBC 支持 VARBINARY/GEOMETRY 类型 + 4. 特性:支持 Excel 通过 ODBC 驱动连接 TDengine + 5. 特性:taosX agent 支持指定本地端口 + +## 优化 + 1. 优化:WebSocket 连接订阅消息为空时,支持获取错误 + 2. 优化:JDBC 支持无符号整数 + 3. 优化:MQTT/Kafka/CSV 新增写并发参数 + 4. 优化:开源版本 TDengine 支持与 TDgpt 对接 + 5. 优化:Flink 连接器支持 BinaryRowData 数据传输类型 + 6. 优化:参数绑定 SQL 语句中,LIMIT 子句支持使用 ? 作为参数占位符 + 7. 优化:taosX 备份开启 WebSocket 压缩 + 8. 优化:ODBC SQLSetStmtAttr 支持 SQL_ROWSET_SIZE 属性 + 9. 优化:TMQ 数据同步任务新增 Number Of Writters/Consumers 等参数 + 10. 优化:mac 安装包增加连接器文件 + 11. 优化:Rust 连接器支持订阅结果为空时的错误处理 + 12. 优化:TSBS 支持生成 CSV 文件 + 13. 优化:TDinsight 添加连接分类信息 + 14. 优化:Explorer float 显示精度与 taos shell 一致 + 15. 优化:Flink 连接器 Table 支持更新和删除操作 + 16. 优化:taosX Agent 在 taosX 无法连接时可重试恢复 + +## 修复 + 1. 修复:Explorer 注册邮箱支持包含 "." + 2. 修复:AWS 云存储下 taosAdapter flock 失败 + 3. 修复:修改子表中布尔类型标签的值时,数据订阅的元数据返回结果存在错误 + 4. 修复:Explorer 导入 CSV 时列包含空格时预览将失效 + 5. 修复:解决 stmtbind 线程在系统处于空闲状态时 CPU 占用过高的问题 + 6. 修复:数据源任务不再处理数据时健康状态恢复为 Idle + 7. 修复:JDBC 示例代码安全漏洞 + 8. 修复:taosX 平滑升级 + 9. 修复:ODBC 调用 SQLSetConnectAttr 设置 SQL_ATTR_TXN_ISOLATION 时 core + 10. 修复:received/processed_messages 当前运行指标重启未清零 + 11. 修复:使用 taosX 恢复数据时,若未指定数据库,系统可能崩溃 + 12. 修复:创建数据库时,keep_time_offset 选项支持使用后缀 h和 d 来指定时间值 + 13. 修复:删除流计算时可能的死锁 + 14. 修复:双副本数据库在某个 dnode 断网后写入数据失败的问题 + 15. 修复:查询 information_schema.ins_tables 表时,如果 mnode 的 Leader 发生变更,可能会触发 Sync leader is unreachable 错误 + 16. 修复:数据重新整理后,涉及复合主键的时间过滤查询结果出现错误的问题 + 17. 修复:当主键列的连接条件为非简单等值条件时,可能导致 JOIN 结果出现错误 + 18. 修复:Python WebSocket 连接器 Cusor.fetchmany 接口自定义长度问题 + 19. 修复:Show Grants 命令返回的列数目不正确的问题 + 20. 修复:备份计划未启动时查看备份点列表不符合预期 + 21. 修复:taosX 任务写入中断后未重新启动 + 22. 修复:JDBC select server_version() 结果未释放导致内存持续增长 + 23. 修复:在使用 WHERE tbname IN () 语句时,如果筛选出的子表不属于同一超级表,执行 LAST 查询可能会导致 taosd 崩溃 + 24. 修复:taosd 异常退出并再次启动后,如果未进入数据文件的 WAL 过大,可能导致启动时 oom + 25. 修复:在使用 interp 插值填充时,如果 select list 中包含字符串常量或字符串标签列,返回的字符串内容可能会出现缺失的情况[#29353](https://github.com/taosdata/TDengine/issues/29353) + 26. 修复:在超级表上执行 JOIN 查询时,将子查询用作右表可能会导致结果缺失 + 27. 修复:同时使用 DISTINCT 和 ORDER BY 关键字时出现的语法错误问题[#29263](https://github.com/taosdata/TDengine/issues/29263) + 28. 修复:使用 CAST 函数将浮点数转换为字符串后进行比较时,可能会因精度丢失而导致结果不准确[#29382](https://github.com/taosdata/TDengine/issues/29382) + 29. 修复:在从 3.3.4 版本升级到 3.3.5 版本后,如果配置的字符集在系统中不存在,taosd 服务将无法正常启动 + 30. 修复:websocket 接口 timing 字段有时为负值 + 31. 修复:taosX 备份任务显示备份点重复 + 32. 修复:配置项 s3BucketName 被误设为全局变量参数,导致文件上传到 S3 失败 + From ccd4b19c5f5d3e3e2cbfed1f94591a043caa3cbf Mon Sep 17 00:00:00 2001 From: dmchen Date: Sat, 1 Mar 2025 00:52:11 +0800 Subject: [PATCH 55/57] fix:remove-send-status-lock --- source/dnode/mgmt/mgmt_dnode/src/dmHandle.c | 49 ++++++++++++++------- 1 file changed, 34 insertions(+), 15 deletions(-) diff --git a/source/dnode/mgmt/mgmt_dnode/src/dmHandle.c b/source/dnode/mgmt/mgmt_dnode/src/dmHandle.c index 49f634443f..4d0135c07f 100644 --- a/source/dnode/mgmt/mgmt_dnode/src/dmHandle.c +++ b/source/dnode/mgmt/mgmt_dnode/src/dmHandle.c @@ -26,6 +26,7 @@ extern SConfig *tsCfg; SMonVloadInfo tsVinfo = {0}; SMnodeLoad tsMLoad = {0}; +SDnodeData tsDnodeData = {0}; static void dmUpdateDnodeCfg(SDnodeMgmt *pMgmt, SDnodeCfg *pCfg) { int32_t code = 0; @@ -170,22 +171,27 @@ void dmSendStatusReq(SDnodeMgmt *pMgmt) { SStatusReq req = {0}; req.timestamp = taosGetTimestampMs(); - dDebug("send status req to mnode, statusSeq:%d, begin to mgnt lock", pMgmt->statusSeq); - (void)taosThreadRwlockRdlock(&pMgmt->pData->lock); + dDebug("send status req to mnode, statusSeq:%d, begin to mgnt statusInfolock", pMgmt->statusSeq); + if (taosThreadMutexLock(&pMgmt->pData->statusInfolock) != 0) { + dError("failed to lock status info lock"); + return; + } + + dDebug("send status req to mnode, statusSeq:%d, begin to get dnode info", pMgmt->statusSeq); req.sver = tsVersion; - req.dnodeVer = pMgmt->pData->dnodeVer; - req.dnodeId = pMgmt->pData->dnodeId; - req.clusterId = pMgmt->pData->clusterId; + req.dnodeVer = tsDnodeData.dnodeVer; + req.dnodeId = tsDnodeData.dnodeId; + req.clusterId = tsDnodeData.clusterId; if (req.clusterId == 0) req.dnodeId = 0; - req.rebootTime = pMgmt->pData->rebootTime; - req.updateTime = pMgmt->pData->updateTime; + req.rebootTime = tsDnodeData.rebootTime; + req.updateTime = tsDnodeData.updateTime; req.numOfCores = tsNumOfCores; req.numOfSupportVnodes = tsNumOfSupportVnodes; req.numOfDiskCfg = tsDiskCfgNum; req.memTotal = tsTotalMemoryKB * 1024; req.memAvail = req.memTotal - tsQueueMemoryAllowed - tsApplyMemoryAllowed - 16 * 1024 * 1024; tstrncpy(req.dnodeEp, tsLocalEp, TSDB_EP_LEN); - tstrncpy(req.machineId, pMgmt->pData->machineId, TSDB_MACHINE_ID_LEN + 1); + tstrncpy(req.machineId, tsDnodeData.machineId, TSDB_MACHINE_ID_LEN + 1); req.clusterCfg.statusInterval = tsStatusInterval; req.clusterCfg.checkTime = 0; @@ -207,17 +213,13 @@ void dmSendStatusReq(SDnodeMgmt *pMgmt) { memcpy(req.clusterCfg.timezone, tsTimezoneStr, TD_TIMEZONE_LEN); memcpy(req.clusterCfg.locale, tsLocale, TD_LOCALE_LEN); memcpy(req.clusterCfg.charset, tsCharset, TD_LOCALE_LEN); - (void)taosThreadRwlockUnlock(&pMgmt->pData->lock); - dDebug("send status req to mnode, statusSeq:%d, begin to get vnode and loads", pMgmt->statusSeq); - if (taosThreadMutexLock(&pMgmt->pData->statusInfolock) != 0) { - dError("failed to lock status info lock"); - return; - } + dDebug("send status req to mnode, statusSeq:%d, begin to get vnode loads", pMgmt->statusSeq); req.pVloads = tsVinfo.pVloads; tsVinfo.pVloads = NULL; + dDebug("send status req to mnode, statusSeq:%d, begin to get mnode loads", pMgmt->statusSeq); req.mload = tsMLoad; if (taosThreadMutexUnlock(&pMgmt->pData->statusInfolock) != 0) { @@ -416,8 +418,19 @@ void dmSendConfigReq(SDnodeMgmt *pMgmt) { } void dmUpdateStatusInfo(SDnodeMgmt *pMgmt) { - SMonVloadInfo vinfo = {0}; + dDebug("begin to get dnode info"); + SDnodeData dnodeData = {0}; + (void)taosThreadRwlockRdlock(&pMgmt->pData->lock); + dnodeData.dnodeVer = pMgmt->pData->dnodeVer; + dnodeData.dnodeId = pMgmt->pData->dnodeId; + dnodeData.clusterId = pMgmt->pData->clusterId; + dnodeData.rebootTime = pMgmt->pData->rebootTime; + dnodeData.updateTime = pMgmt->pData->updateTime; + tstrncpy(dnodeData.machineId, pMgmt->pData->machineId, TSDB_MACHINE_ID_LEN + 1); + (void)taosThreadRwlockUnlock(&pMgmt->pData->lock); + dDebug("begin to get vnode loads"); + SMonVloadInfo vinfo = {0}; (*pMgmt->getVnodeLoadsFp)(&vinfo); // dmGetVnodeLoads dDebug("begin to get mnode loads"); @@ -429,6 +442,12 @@ void dmUpdateStatusInfo(SDnodeMgmt *pMgmt) { dError("failed to lock status info lock"); return; } + tsDnodeData.dnodeVer = dnodeData.dnodeVer; + tsDnodeData.dnodeId = dnodeData.dnodeId; + tsDnodeData.clusterId = dnodeData.clusterId; + tsDnodeData.rebootTime = dnodeData.rebootTime; + tsDnodeData.updateTime = dnodeData.updateTime; + tstrncpy(tsDnodeData.machineId, dnodeData.machineId, TSDB_MACHINE_ID_LEN + 1); if (tsVinfo.pVloads == NULL) { tsVinfo.pVloads = vinfo.pVloads; From b0f62f4f91357d298f09b8d2a5f0ba8cc508f2aa Mon Sep 17 00:00:00 2001 From: dmchen Date: Sat, 1 Mar 2025 12:50:28 +0800 Subject: [PATCH 56/57] fix:statusseq --- source/dnode/mgmt/mgmt_dnode/src/dmHandle.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/dnode/mgmt/mgmt_dnode/src/dmHandle.c b/source/dnode/mgmt/mgmt_dnode/src/dmHandle.c index 4d0135c07f..b1a910d246 100644 --- a/source/dnode/mgmt/mgmt_dnode/src/dmHandle.c +++ b/source/dnode/mgmt/mgmt_dnode/src/dmHandle.c @@ -170,6 +170,7 @@ void dmSendStatusReq(SDnodeMgmt *pMgmt) { int32_t code = 0; SStatusReq req = {0}; req.timestamp = taosGetTimestampMs(); + pMgmt->statusSeq++; dDebug("send status req to mnode, statusSeq:%d, begin to mgnt statusInfolock", pMgmt->statusSeq); if (taosThreadMutexLock(&pMgmt->pData->statusInfolock) != 0) { @@ -230,7 +231,6 @@ void dmSendStatusReq(SDnodeMgmt *pMgmt) { dDebug("send status req to mnode, statusSeq:%d, begin to get qnode loads", pMgmt->statusSeq); (*pMgmt->getQnodeLoadsFp)(&req.qload); - pMgmt->statusSeq++; req.statusSeq = pMgmt->statusSeq; req.ipWhiteVer = pMgmt->pData->ipWhiteVer; req.analVer = taosAnalGetVersion(); From 74562db0e577b86f32a794f4127d1b25b6aee082 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sun, 2 Mar 2025 15:36:29 +0800 Subject: [PATCH 57/57] fix(stream): set empty string instead of null ptr. --- source/libs/stream/src/streamQueue.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/libs/stream/src/streamQueue.c b/source/libs/stream/src/streamQueue.c index 954e41f288..d8a1c68c07 100644 --- a/source/libs/stream/src/streamQueue.c +++ b/source/libs/stream/src/streamQueue.c @@ -37,7 +37,7 @@ static void streamQueueNextItemInSourceQ(SStreamQueue* pQueue, SStreamQueueItem* static void streamQueueCleanup(SStreamQueue* pQueue) { SStreamQueueItem* qItem = NULL; while (1) { - streamQueueNextItemInSourceQ(pQueue, &qItem, TASK_STATUS__READY, NULL); + streamQueueNextItemInSourceQ(pQueue, &qItem, TASK_STATUS__READY, ""); if (qItem == NULL) { break; }