From a678e722ceee3e37c797b1d6085decda5442455f Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 13 Sep 2023 09:19:59 +0800 Subject: [PATCH 001/158] fix(stream): limit the token bucket capacity --- source/libs/stream/src/streamTask.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/libs/stream/src/streamTask.c b/source/libs/stream/src/streamTask.c index 663deca171..84e373172f 100644 --- a/source/libs/stream/src/streamTask.c +++ b/source/libs/stream/src/streamTask.c @@ -385,7 +385,7 @@ int32_t streamTaskInit(SStreamTask* pTask, SStreamMeta* pMeta, SMsgCb* pMsgCb, i pTask->dataRange.range.minVer = ver; pTask->pMsgCb = pMsgCb; - streamTaskInitTokenBucket(&pTask->tokenBucket, 150, 100); + streamTaskInitTokenBucket(&pTask->tokenBucket, 100, 100); taosThreadMutexInit(&pTask->lock, NULL); streamTaskOpenAllUpstreamInput(pTask); From 01d104706efde7e3ecd39c2fd0bf09e1820b3ed6 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 13 Sep 2023 11:02:38 +0800 Subject: [PATCH 002/158] refactor(stream): remove unnecessary streamschedTask function to improve the performance. --- source/libs/stream/src/stream.c | 3 +-- source/libs/stream/src/streamExec.c | 25 +++++++++++++------------ 2 files changed, 14 insertions(+), 14 deletions(-) diff --git a/source/libs/stream/src/stream.c b/source/libs/stream/src/stream.c index d1bf6a91c5..d16822be60 100644 --- a/source/libs/stream/src/stream.c +++ b/source/libs/stream/src/stream.c @@ -256,8 +256,7 @@ int32_t streamProcessDispatchMsg(SStreamTask* pTask, SStreamDispatchReq* pReq, S } tDeleteStreamDispatchReq(pReq); - streamSchedExec(pTask); - + streamTryExec(pTask); return 0; } diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index f03a6a32d4..16fc54d8be 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -621,19 +621,20 @@ int32_t streamTryExec(SStreamTask* pTask) { const char* id = pTask->id.idStr; if (schedStatus == TASK_SCHED_STATUS__WAITING) { - int32_t code = streamExecForAll(pTask); - if (code < 0) { // todo this status shoudl be removed - atomic_store_8(&pTask->status.schedStatus, TASK_SCHED_STATUS__FAILED); - return -1; - } + while (1) { + int32_t code = streamExecForAll(pTask); + if (code < 0) { // todo this status shoudl be removed + atomic_store_8(&pTask->status.schedStatus, TASK_SCHED_STATUS__FAILED); + return -1; + } - atomic_store_8(&pTask->status.schedStatus, TASK_SCHED_STATUS__INACTIVE); - qDebug("s-task:%s exec completed, status:%s, sched-status:%d", id, streamGetTaskStatusStr(pTask->status.taskStatus), - pTask->status.schedStatus); - - if (!(taosQueueEmpty(pTask->inputInfo.queue->pQueue) || streamTaskShouldStop(&pTask->status) || - streamTaskShouldPause(&pTask->status))) { - streamSchedExec(pTask); + if (taosQueueEmpty(pTask->inputInfo.queue->pQueue) || streamTaskShouldStop(&pTask->status) || + streamTaskShouldPause(&pTask->status)) { + atomic_store_8(&pTask->status.schedStatus, TASK_SCHED_STATUS__INACTIVE); + qDebug("s-task:%s exec completed, status:%s, sched-status:%d", id, + streamGetTaskStatusStr(pTask->status.taskStatus), pTask->status.schedStatus); + return 0; + } } } else { qDebug("s-task:%s already started to exec by other thread, status:%s, sched-status:%d", id, From aaa12ed4f85c6b22381cab26a848147bc6321f15 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 13 Sep 2023 14:00:34 +0800 Subject: [PATCH 003/158] refactor: do some internal refactor about the sched status set --- include/libs/stream/tstream.h | 4 ++++ source/dnode/vnode/src/tq/tq.c | 16 ++++++------- source/libs/stream/src/stream.c | 11 +++++---- source/libs/stream/src/streamExec.c | 9 +++---- source/libs/stream/src/streamRecover.c | 2 +- source/libs/stream/src/streamTask.c | 33 +++++++++++++++++++++++++- 6 files changed, 56 insertions(+), 19 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 3eb624f932..a5baf33612 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -401,6 +401,7 @@ typedef struct SStreamMeta { int32_t vgId; int64_t stage; SRWLatch lock; +// TdThreadRwlock lock; int32_t walScanCounter; void* streamBackend; int64_t streamBackendRid; @@ -660,6 +661,9 @@ int32_t streamTaskLaunchScanHistory(SStreamTask* pTask); int32_t streamTaskCheckStatus(SStreamTask* pTask, int32_t upstreamTaskId, int32_t vgId, int64_t stage); int32_t streamTaskUpdateEpsetInfo(SStreamTask* pTask, SArray* pNodeList); void streamTaskResetUpstreamStageInfo(SStreamTask* pTask); +int8_t streamTaskSetSchedStatusWait(SStreamTask* pTask); +int8_t streamTaskSetSchedStatusActive(SStreamTask* pTask); +int8_t streamTaskSetSchedStatusInActive(SStreamTask* pTask); int32_t streamTaskStop(SStreamTask* pTask); int32_t streamSendCheckRsp(const SStreamMeta* pMeta, const SStreamTaskCheckReq* pReq, SStreamTaskCheckRsp* pRsp, diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 58544090e2..7186adc2c4 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1046,8 +1046,7 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { } // we have to continue retrying to successfully execute the scan history task. - int8_t schedStatus = atomic_val_compare_exchange_8(&pTask->status.schedStatus, TASK_SCHED_STATUS__INACTIVE, - TASK_SCHED_STATUS__WAITING); + int8_t schedStatus = streamTaskSetSchedStatusWait(pTask); if (schedStatus != TASK_SCHED_STATUS__INACTIVE) { tqError( "s-task:%s failed to start scan-history in first stream time window since already started, unexpected " @@ -1064,9 +1063,8 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { streamScanHistoryData(pTask); if (pTask->status.taskStatus == TASK_STATUS__PAUSE) { double el = (taosGetTimestampMs() - pTask->tsInfo.step1Start) / 1000.0; - tqDebug("s-task:%s is paused in the step1, elapsed time:%.2fs, sched-status:%d", pTask->id.idStr, el, - TASK_SCHED_STATUS__INACTIVE); - atomic_store_8(&pTask->status.schedStatus, TASK_SCHED_STATUS__INACTIVE); + int8_t status = streamTaskSetSchedStatusInActive(pTask); + tqDebug("s-task:%s is paused in the step1, elapsed time:%.2fs, sched-status:%d", pTask->id.idStr, el, status); streamMetaReleaseTask(pMeta, pTask); return 0; } @@ -1106,8 +1104,8 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { } // now we can stop the stream task execution - int64_t latestVer = 0; + taosThreadMutexLock(&pStreamTask->lock); streamTaskHalt(pStreamTask); tqDebug("s-task:%s level:%d sched-status:%d is halt by fill-history task:%s", pStreamTask->id.idStr, @@ -1141,7 +1139,7 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { tqDebug("s-task:%s wal reader start scan WAL verRange:%" PRId64 "-%" PRId64 ", set sched-status:%d", id, dstVer, pTask->dataRange.range.maxVer, TASK_SCHED_STATUS__INACTIVE); - atomic_store_8(&pTask->status.schedStatus, TASK_SCHED_STATUS__INACTIVE); + /*int8_t status = */streamTaskSetSchedStatusInActive(pTask); // set the fill-history task to be normal if (pTask->info.fillHistory == 1 && !streamTaskShouldStop(&pTask->status)) { @@ -1308,9 +1306,9 @@ int32_t tqProcessTaskRunReq(STQ* pTq, SRpcMsg* pMsg) { pTask->chkInfo.nextProcessVer); streamProcessRunReq(pTask); } else { - atomic_store_8(&pTask->status.schedStatus, TASK_SCHED_STATUS__INACTIVE); + int8_t status = streamTaskSetSchedStatusInActive(pTask); tqDebug("vgId:%d s-task:%s ignore run req since not in ready state, status:%s, sched-status:%d", vgId, - pTask->id.idStr, streamGetTaskStatusStr(st), pTask->status.schedStatus); + pTask->id.idStr, streamGetTaskStatusStr(st), status); } streamMetaReleaseTask(pTq->pStreamMeta, pTask); diff --git a/source/libs/stream/src/stream.c b/source/libs/stream/src/stream.c index d16822be60..1b4de5e6c4 100644 --- a/source/libs/stream/src/stream.c +++ b/source/libs/stream/src/stream.c @@ -108,14 +108,13 @@ int32_t streamSetupScheduleTrigger(SStreamTask* pTask) { } int32_t streamSchedExec(SStreamTask* pTask) { - int8_t schedStatus = atomic_val_compare_exchange_8(&pTask->status.schedStatus, TASK_SCHED_STATUS__INACTIVE, - TASK_SCHED_STATUS__WAITING); + int8_t schedStatus = streamTaskSetSchedStatusWait(pTask); if (schedStatus == TASK_SCHED_STATUS__INACTIVE) { SStreamTaskRunReq* pRunReq = rpcMallocCont(sizeof(SStreamTaskRunReq)); if (pRunReq == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; - atomic_store_8(&pTask->status.schedStatus, TASK_SCHED_STATUS__INACTIVE); + /*int8_t status = */streamTaskSetSchedStatusInActive(pTask); qError("failed to create msg to aunch s-task:%s, reason out of memory", pTask->id.idStr); return -1; } @@ -256,7 +255,11 @@ int32_t streamProcessDispatchMsg(SStreamTask* pTask, SStreamDispatchReq* pReq, S } tDeleteStreamDispatchReq(pReq); - streamTryExec(pTask); + + int8_t schedStatus = streamTaskSetSchedStatusWait(pTask); + if (schedStatus == TASK_SCHED_STATUS__INACTIVE) { + streamTryExec(pTask); + } return 0; } diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index 16fc54d8be..b981fcb41a 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -509,7 +509,7 @@ int32_t streamProcessTranstateBlock(SStreamTask* pTask, SStreamDataBlock* pBlock code = streamTransferStateToStreamTask(pTask); if (code != TSDB_CODE_SUCCESS) { - atomic_store_8(&pTask->status.schedStatus, TASK_SCHED_STATUS__INACTIVE); + /*int8_t status = */streamTaskSetSchedStatusInActive(pTask); } } else { qDebug("s-task:%s sink task does not transfer state", id); @@ -615,11 +615,9 @@ bool streamTaskIsIdle(const SStreamTask* pTask) { int32_t streamTryExec(SStreamTask* pTask) { // this function may be executed by multi-threads, so status check is required. - int8_t schedStatus = - atomic_val_compare_exchange_8(&pTask->status.schedStatus, TASK_SCHED_STATUS__WAITING, TASK_SCHED_STATUS__ACTIVE); - const char* id = pTask->id.idStr; + int8_t schedStatus = streamTaskSetSchedStatusActive(pTask); if (schedStatus == TASK_SCHED_STATUS__WAITING) { while (1) { int32_t code = streamExecForAll(pTask); @@ -628,9 +626,12 @@ int32_t streamTryExec(SStreamTask* pTask) { return -1; } + taosThreadMutexLock(&pTask->lock); if (taosQueueEmpty(pTask->inputInfo.queue->pQueue) || streamTaskShouldStop(&pTask->status) || streamTaskShouldPause(&pTask->status)) { atomic_store_8(&pTask->status.schedStatus, TASK_SCHED_STATUS__INACTIVE); + taosThreadMutexUnlock(&pTask->lock); + qDebug("s-task:%s exec completed, status:%s, sched-status:%d", id, streamGetTaskStatusStr(pTask->status.taskStatus), pTask->status.schedStatus); return 0; diff --git a/source/libs/stream/src/streamRecover.c b/source/libs/stream/src/streamRecover.c index 54d5957900..8b2a800576 100644 --- a/source/libs/stream/src/streamRecover.c +++ b/source/libs/stream/src/streamRecover.c @@ -480,7 +480,7 @@ int32_t streamProcessScanHistoryFinishRsp(SStreamTask* pTask) { // execute in the scan history complete call back msg, ready to process data from inputQ streamSetStatusNormal(pTask); - atomic_store_8(&pTask->status.schedStatus, TASK_SCHED_STATUS__INACTIVE); + streamTaskSetSchedStatusInActive(pTask); taosWLockLatch(&pMeta->lock); streamMetaSaveTask(pMeta, pTask); diff --git a/source/libs/stream/src/streamTask.c b/source/libs/stream/src/streamTask.c index 84e373172f..71a9a3102c 100644 --- a/source/libs/stream/src/streamTask.c +++ b/source/libs/stream/src/streamTask.c @@ -332,7 +332,6 @@ void tFreeStreamTask(SStreamTask* pTask) { } pTask->pReadyMsgList = taosArrayDestroy(pTask->pReadyMsgList); - taosThreadMutexDestroy(&pTask->lock); if (pTask->msgInfo.pData != NULL) { destroyStreamDataBlock(pTask->msgInfo.pData); pTask->msgInfo.pData = NULL; @@ -553,3 +552,35 @@ void streamTaskResetUpstreamStageInfo(SStreamTask* pTask) { qDebug("s-task:%s reset all upstream tasks stage info", pTask->id.idStr); } + +int8_t streamTaskSetSchedStatusWait(SStreamTask* pTask) { + taosThreadMutexLock(&pTask->lock); + int8_t status = pTask->status.schedStatus; + if (status == TASK_SCHED_STATUS__INACTIVE) { + pTask->status.schedStatus = TASK_SCHED_STATUS__WAITING; + } + taosThreadMutexUnlock(&pTask->lock); + + return status; +} + +int8_t streamTaskSetSchedStatusActive(SStreamTask* pTask) { + taosThreadMutexLock(&pTask->lock); + int8_t status = pTask->status.schedStatus; + if (status == TASK_SCHED_STATUS__WAITING) { + pTask->status.schedStatus = TASK_SCHED_STATUS__ACTIVE; + } + taosThreadMutexUnlock(&pTask->lock); + + return status; +} + +int8_t streamTaskSetSchedStatusInActive(SStreamTask* pTask) { + taosThreadMutexLock(&pTask->lock); + int8_t status = pTask->status.schedStatus; + ASSERT(status == TASK_SCHED_STATUS__WAITING || status == TASK_SCHED_STATUS__ACTIVE); + pTask->status.schedStatus = TASK_SCHED_STATUS__INACTIVE; + taosThreadMutexUnlock(&pTask->lock); + + return status; +} From 283118bf684f455d6707119f743c22ba4458807d Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 13 Sep 2023 14:11:41 +0800 Subject: [PATCH 004/158] fix(stream):fix missing unlock. --- source/libs/stream/src/streamExec.c | 1 + 1 file changed, 1 insertion(+) diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index b981fcb41a..94dfc09314 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -636,6 +636,7 @@ int32_t streamTryExec(SStreamTask* pTask) { streamGetTaskStatusStr(pTask->status.taskStatus), pTask->status.schedStatus); return 0; } + taosThreadMutexUnlock(&pTask->lock); } } else { qDebug("s-task:%s already started to exec by other thread, status:%s, sched-status:%d", id, From be09c173ab743d2224187a50ddb9b1da9f6aa236 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 14 Sep 2023 09:06:10 +0800 Subject: [PATCH 005/158] fix:fix coverity issue and remove unnecessary files. --- source/dnode/mnode/impl/src/mndStream.c | 60 - source/dnode/vnode/inc/vnode.h | 2 +- source/dnode/vnode/src/inc/tsdb.h | 6 +- source/dnode/vnode/src/tsdb/tsdbMergeTree.c | 101 - source/dnode/vnode/src/tsdb/tsdbRead.c | 5611 ------------------- source/dnode/vnode/src/tsdb/tsdbRead2.c | 24 +- source/libs/parser/src/parTranslater.c | 4 +- 7 files changed, 24 insertions(+), 5784 deletions(-) delete mode 100644 source/dnode/vnode/src/tsdb/tsdbRead.c diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index 049b4e737a..d3162a143d 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -2327,65 +2327,5 @@ int32_t mndProcessStreamHb(SRpcMsg *pReq) { taosThreadMutexUnlock(&execNodeList.lock); taosArrayDestroy(req.pTaskStatus); - - // bool nodeChanged = false; - // SArray* pList = taosArrayInit(4, sizeof(int32_t)); - /* - // record the timeout node - for(int32_t i = 0; i < taosArrayGetSize(execNodeList.pNodeEntryList); ++i) { - SNodeEntry* pEntry = taosArrayGet(execNodeList.pNodeEntryList, i); - int64_t duration = now - pEntry->hbTimestamp; - if (duration > MND_STREAM_HB_INTERVAL) { // execNode timeout, try next - taosArrayPush(pList, &pEntry); - mWarn("nodeId:%d stream node timeout, since last hb:%"PRId64"s", pEntry->nodeId, duration); - continue; - } - - if (pEntry->nodeId != req.vgId) { - continue; - } - - pEntry->hbTimestamp = now; - - // check epset to identify whether the node has been transferred to other dnodes. - // node the epset is changed, which means the node transfer has occurred for this node. - // if (!isEpsetEqual(&pEntry->epset, &req.epset)) { - // nodeChanged = true; - // break; - // } - } - - // todo handle the node timeout case. Once the vnode is off-line, we should check the dnode status from mnode, - // to identify whether the dnode is truely offline or not. - - // handle the node changed case - if (!nodeChanged) { - return TSDB_CODE_SUCCESS; - } - - int32_t nodeId = req.vgId; - - {// check all streams that involved this vnode should update the epset info - SStreamObj *pStream = NULL; - void *pIter = NULL; - while (1) { - pIter = sdbFetch(pSdb, SDB_STREAM, pIter, (void **)&pStream); - if (pIter == NULL) { - break; - } - - // update the related upstream and downstream tasks, todo remove this, no need this function - taosWLockLatch(&pStream->lock); - // streamTaskUpdateEpInfo(pStream->tasks, req.vgId, &req.epset); - // streamTaskUpdateEpInfo(pStream->pHTasksList, req.vgId, &req.epset); - taosWUnLockLatch(&pStream->lock); - - // code = createStreamUpdateTrans(pMnode, pStream, nodeId, ); - // if (code != TSDB_CODE_SUCCESS) { - // todo - //// } - // } - } - */ return TSDB_CODE_SUCCESS; } diff --git a/source/dnode/vnode/inc/vnode.h b/source/dnode/vnode/inc/vnode.h index e15f5f911d..8976e00792 100644 --- a/source/dnode/vnode/inc/vnode.h +++ b/source/dnode/vnode/inc/vnode.h @@ -186,7 +186,7 @@ int64_t tsdbGetNumOfRowsInMemTable2(STsdbReader *pHandle); void *tsdbGetIdx2(SMeta *pMeta); void *tsdbGetIvtIdx2(SMeta *pMeta); uint64_t tsdbGetReaderMaxVersion2(STsdbReader *pReader); -void tsdbReaderSetCloseFlag2(STsdbReader *pReader); +void tsdbReaderSetCloseFlag(STsdbReader *pReader); int64_t tsdbGetLastTimestamp2(SVnode *pVnode, void *pTableList, int32_t numOfTables, const char *pIdStr); //====================================================================================================================== diff --git a/source/dnode/vnode/src/inc/tsdb.h b/source/dnode/vnode/src/inc/tsdb.h index ab6a7fb88b..8337b11ce5 100644 --- a/source/dnode/vnode/src/inc/tsdb.h +++ b/source/dnode/vnode/src/inc/tsdb.h @@ -302,12 +302,11 @@ int32_t tsdbDelFReaderClose(SDelFReader **ppReader); int32_t tsdbReadDelDatav1(SDelFReader *pReader, SDelIdx *pDelIdx, SArray *aDelData, int64_t maxVer); int32_t tsdbReadDelData(SDelFReader *pReader, SDelIdx *pDelIdx, SArray *aDelData); int32_t tsdbReadDelIdx(SDelFReader *pReader, SArray *aDelIdx); -// tsdbRead.c ============================================================================================== -int32_t tsdbTakeReadSnap(STsdbReader *pReader, _query_reseek_func_t reseek, STsdbReadSnap **ppSnap); -void tsdbUntakeReadSnap(STsdbReader *pReader, STsdbReadSnap *pSnap, bool proactive); +// tsdbRead.c ============================================================================================== int32_t tsdbTakeReadSnap2(STsdbReader *pReader, _query_reseek_func_t reseek, STsdbReadSnap **ppSnap); void tsdbUntakeReadSnap2(STsdbReader *pReader, STsdbReadSnap *pSnap, bool proactive); + // tsdbMerge.c ============================================================================================== int32_t tsdbMerge(void *arg); @@ -830,7 +829,6 @@ bool tMergeTreeNext(SMergeTree *pMTree); bool tMergeTreeIgnoreEarlierTs(SMergeTree *pMTree); void tMergeTreeClose(SMergeTree *pMTree); -SSttBlockLoadInfo *tCreateLastBlockLoadInfo(STSchema *pSchema, int16_t *colList, int32_t numOfCols, int32_t numOfStt); SSttBlockLoadInfo *tCreateOneLastBlockLoadInfo(STSchema *pSchema, int16_t *colList, int32_t numOfCols); void resetLastBlockLoadInfo(SSttBlockLoadInfo *pLoadInfo); void getSttBlockLoadInfo(SSttBlockLoadInfo *pLoadInfo, SSttBlockLoadCostInfo *pLoadCost); diff --git a/source/dnode/vnode/src/tsdb/tsdbMergeTree.c b/source/dnode/vnode/src/tsdb/tsdbMergeTree.c index 4927b1539b..4705d95c0e 100644 --- a/source/dnode/vnode/src/tsdb/tsdbMergeTree.c +++ b/source/dnode/vnode/src/tsdb/tsdbMergeTree.c @@ -22,38 +22,6 @@ static void tLDataIterClose2(SLDataIter *pIter); // SLDataIter ================================================= -SSttBlockLoadInfo *tCreateLastBlockLoadInfo(STSchema *pSchema, int16_t *colList, int32_t numOfCols, - int32_t numOfSttTrigger) { - SSttBlockLoadInfo *pLoadInfo = taosMemoryCalloc(numOfSttTrigger, sizeof(SSttBlockLoadInfo)); - if (pLoadInfo == NULL) { - terrno = TSDB_CODE_OUT_OF_MEMORY; - return NULL; - } - - for (int32_t i = 0; i < numOfSttTrigger; ++i) { - pLoadInfo[i].blockIndex[0] = -1; - pLoadInfo[i].blockIndex[1] = -1; - pLoadInfo[i].currentLoadBlockIndex = 1; - - int32_t code = tBlockDataCreate(&pLoadInfo[i].blockData[0]); - if (code) { - terrno = code; - } - - code = tBlockDataCreate(&pLoadInfo[i].blockData[1]); - if (code) { - terrno = code; - } - - pLoadInfo[i].aSttBlk = taosArrayInit(4, sizeof(SSttBlk)); - pLoadInfo[i].pSchema = pSchema; - pLoadInfo[i].colIds = colList; - pLoadInfo[i].numOfCols = numOfCols; - } - - return pLoadInfo; -} - SSttBlockLoadInfo *tCreateOneLastBlockLoadInfo(STSchema *pSchema, int16_t *colList, int32_t numOfCols) { SSttBlockLoadInfo *pLoadInfo = taosMemoryCalloc(1, sizeof(SSttBlockLoadInfo)); if (pLoadInfo == NULL) { @@ -83,25 +51,6 @@ SSttBlockLoadInfo *tCreateOneLastBlockLoadInfo(STSchema *pSchema, int16_t *colLi return pLoadInfo; } -void resetLastBlockLoadInfo(SSttBlockLoadInfo *pLoadInfo) { - for (int32_t i = 0; i < 1; ++i) { - pLoadInfo[i].currentLoadBlockIndex = 1; - pLoadInfo[i].blockIndex[0] = -1; - pLoadInfo[i].blockIndex[1] = -1; - - taosArrayClear(pLoadInfo[i].aSttBlk); - - pLoadInfo[i].cost.loadBlocks = 0; - pLoadInfo[i].cost.blockElapsedTime = 0; - pLoadInfo[i].cost.statisElapsedTime = 0; - pLoadInfo[i].cost.loadStatisBlocks = 0; - pLoadInfo[i].statisBlockIndex = -1; - tStatisBlockDestroy(pLoadInfo[i].statisBlock); - - pLoadInfo[i].sttBlockLoaded = false; - } -} - void getSttBlockLoadInfo(SSttBlockLoadInfo *pLoadInfo, SSttBlockLoadCostInfo* pLoadCost) { for (int32_t i = 0; i < 1; ++i) { pLoadCost->blockElapsedTime += pLoadInfo[i].cost.blockElapsedTime; @@ -309,12 +258,6 @@ static int32_t binarySearchForStartRowIndex(uint64_t *uidList, int32_t num, uint } } -int32_t tLDataIterOpen(struct SLDataIter *pIter, SDataFReader *pReader, int32_t iStt, int8_t backward, uint64_t suid, - uint64_t uid, STimeWindow *pTimeWindow, SVersionRange *pRange, SSttBlockLoadInfo *pBlockLoadInfo, - const char *idStr, bool strictTimeRange) { - return 0; -} - static int32_t extractSttBlockInfo(SLDataIter *pIter, const TSttBlkArray *pArray, SSttBlockLoadInfo *pBlockLoadInfo, uint64_t suid) { if (TARRAY2_SIZE(pArray) <= 0) { @@ -767,50 +710,6 @@ static FORCE_INLINE int32_t tLDataIterDescCmprFn(const SRBTreeNode *p1, const SR return -1 * tLDataIterCmprFn(p1, p2); } -int32_t tMergeTreeOpen(SMergeTree *pMTree, int8_t backward, SDataFReader *pFReader, uint64_t suid, uint64_t uid, - STimeWindow *pTimeWindow, SVersionRange *pVerRange, SSttBlockLoadInfo *pBlockLoadInfo, - bool destroyLoadInfo, const char *idStr, bool strictTimeRange, SLDataIter *pLDataIter) { - int32_t code = TSDB_CODE_SUCCESS; - - pMTree->backward = backward; - pMTree->pIter = NULL; - pMTree->idStr = idStr; - - if (!pMTree->backward) { // asc - tRBTreeCreate(&pMTree->rbt, tLDataIterCmprFn); - } else { // desc - tRBTreeCreate(&pMTree->rbt, tLDataIterDescCmprFn); - } - - pMTree->pLoadInfo = pBlockLoadInfo; - pMTree->destroyLoadInfo = destroyLoadInfo; - pMTree->ignoreEarlierTs = false; - - for (int32_t i = 0; i < pFReader->pSet->nSttF; ++i) { // open all last file - memset(&pLDataIter[i], 0, sizeof(SLDataIter)); - code = tLDataIterOpen(&pLDataIter[i], pFReader, i, pMTree->backward, suid, uid, pTimeWindow, pVerRange, - &pMTree->pLoadInfo[i], pMTree->idStr, strictTimeRange); - if (code != TSDB_CODE_SUCCESS) { - goto _end; - } - - bool hasVal = tLDataIterNextRow(&pLDataIter[i], pMTree->idStr); - if (hasVal) { - tMergeTreeAddIter(pMTree, &pLDataIter[i]); - } else { - if (!pMTree->ignoreEarlierTs) { - pMTree->ignoreEarlierTs = pLDataIter[i].ignoreEarlierTs; - } - } - } - - return code; - -_end: - tMergeTreeClose(pMTree); - return code; -} - int32_t tMergeTreeOpen2(SMergeTree *pMTree, SMergeTreeConf *pConf) { int32_t code = TSDB_CODE_SUCCESS; diff --git a/source/dnode/vnode/src/tsdb/tsdbRead.c b/source/dnode/vnode/src/tsdb/tsdbRead.c deleted file mode 100644 index c02cff3aa9..0000000000 --- a/source/dnode/vnode/src/tsdb/tsdbRead.c +++ /dev/null @@ -1,5611 +0,0 @@ -/* - * Copyright (c) 2019 TAOS Data, Inc. - * - * This program is free software: you can use, redistribute, and/or modify - * it under the terms of the GNU Affero General Public License, version 3 - * or later ("AGPL"), as published by the Free Software Foundation. - * - * This program is distributed in the hope that it will be useful, but WITHOUT - * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or - * FITNESS FOR A PARTICULAR PURPOSE. - * - * You should have received a copy of the GNU Affero General Public License - * along with this program. If not, see . - */ - -#include "osDef.h" -#include "tsdb.h" -#include "tsimplehash.h" - -#define ASCENDING_TRAVERSE(o) (o == TSDB_ORDER_ASC) -#define getCurrentKeyInLastBlock(_r) ((_r)->currentKey) - -typedef enum { - READER_STATUS_SUSPEND = 0x1, - READER_STATUS_NORMAL = 0x2, -} EReaderStatus; - -typedef enum { - EXTERNAL_ROWS_PREV = 0x1, - EXTERNAL_ROWS_MAIN = 0x2, - EXTERNAL_ROWS_NEXT = 0x3, -} EContentData; -/* -typedef enum { - READ_MODE_COUNT_ONLY = 0x1, - READ_MODE_ALL, -} EReadMode; -*/ -typedef struct { - STbDataIter* iter; - int32_t index; - bool hasVal; -} SIterInfo; - -typedef struct { - int32_t numOfBlocks; - int32_t numOfLastFiles; -} SBlockNumber; - -typedef struct SBlockIndex { - int32_t ordinalIndex; - int64_t inFileOffset; - STimeWindow window; // todo replace it with overlap flag. -} SBlockIndex; - -typedef struct STableBlockScanInfo { - uint64_t uid; - TSKEY lastKey; - TSKEY lastKeyInStt; // last accessed key in stt - SMapData mapData; // block info (compressed) - SArray* pBlockList; // block data index list, SArray - SIterInfo iter; // mem buffer skip list iterator - SIterInfo iiter; // imem buffer skip list iterator - SArray* delSkyline; // delete info for this table - int32_t fileDelIndex; // file block delete index - int32_t sttBlockDelIndex; // delete index for last block - bool iterInit; // whether to initialize the in-memory skip list iterator or not -} STableBlockScanInfo; - -typedef struct SBlockOrderWrapper { - int64_t uid; - int64_t offset; -} SBlockOrderWrapper; - -typedef struct SBlockOrderSupporter { - SBlockOrderWrapper** pDataBlockInfo; - int32_t* indexPerTable; - int32_t* numOfBlocksPerTable; - int32_t numOfTables; -} SBlockOrderSupporter; - -typedef struct SIOCostSummary { - int64_t numOfBlocks; - double blockLoadTime; - double buildmemBlock; - int64_t headFileLoad; - double headFileLoadTime; - int64_t smaDataLoad; - double smaLoadTime; - int64_t sttStatisBlockLoad; - int64_t sttBlockLoad; - double sttBlockLoadTime; - int64_t composedBlocks; - double buildComposedBlockTime; - double createScanInfoList; - SSttBlockLoadCostInfo sttCost; - double initDelSkylineIterTime; -} SIOCostSummary; - -typedef struct SBlockLoadSuppInfo { - SArray* pColAgg; - SColumnDataAgg tsColAgg; - int16_t* colId; - int16_t* slotId; - int32_t numOfCols; - char** buildBuf; // build string tmp buffer, todo remove it later after all string format being updated. - bool smaValid; // the sma on all queried columns are activated -} SBlockLoadSuppInfo; - -typedef struct SLastBlockReader { - STimeWindow window; - SVersionRange verRange; - int32_t order; - uint64_t uid; - SMergeTree mergeTree; - SSttBlockLoadInfo* pInfo; - int64_t currentKey; -} SLastBlockReader; - -typedef struct SFilesetIter { - int32_t numOfFiles; // number of total files - int32_t index; // current accessed index in the list - SArray* pFileList; // data file list - int32_t order; - SLastBlockReader* pLastBlockReader; // last file block reader -} SFilesetIter; - -typedef struct SFileDataBlockInfo { - // index position in STableBlockScanInfo in order to check whether neighbor block overlaps with it - uint64_t uid; - int32_t tbBlockIdx; -} SFileDataBlockInfo; - -typedef struct SDataBlockIter { - int32_t numOfBlocks; - int32_t index; - SArray* blockList; // SArray - int32_t order; - SDataBlk block; // current SDataBlk data - SSHashObj* pTableMap; -} SDataBlockIter; - -typedef struct SFileBlockDumpInfo { - int32_t totalRows; - int32_t rowIndex; - int64_t lastKey; - bool allDumped; -} SFileBlockDumpInfo; - -typedef struct STableUidList { - uint64_t* tableUidList; // access table uid list in uid ascending order list - int32_t currentIndex; // index in table uid list -} STableUidList; - -typedef struct SReaderStatus { - bool loadFromFile; // check file stage - bool composedDataBlock; // the returned data block is a composed block or not - bool mapDataCleaned; // mapData has been cleaned up alreay or not - SSHashObj* pTableMap; // SHash - STableBlockScanInfo** pTableIter; // table iterator used in building in-memory buffer data blocks. - STableUidList uidList; // check tables in uid order, to avoid the repeatly load of blocks in STT. - SFileBlockDumpInfo fBlockDumpInfo; - SDFileSet* pCurrentFileset; // current opened file set - SBlockData fileBlockData; - SFilesetIter fileIter; - SDataBlockIter blockIter; - SLDataIter* pLDataIter; - SRowMerger merger; - SColumnInfoData* pPrimaryTsCol; // primary time stamp output col info data -} SReaderStatus; - -typedef struct SBlockInfoBuf { - int32_t currentIndex; - SArray* pData; - int32_t numPerBucket; - int32_t numOfTables; -} SBlockInfoBuf; - -typedef struct STsdbReaderAttr { - STSchema* pSchema; - EReadMode readMode; - uint64_t rowsNum; - STimeWindow window; - bool freeBlock; - SVersionRange verRange; - int16_t order; -} STsdbReaderAttr; - -typedef struct SResultBlockInfo { - SSDataBlock* pResBlock; - bool freeBlock; - int64_t capacity; -} SResultBlockInfo; - -struct STsdbReader { - STsdb* pTsdb; - SVersionRange verRange; - TdThreadMutex readerMutex; - EReaderStatus flag; - int32_t code; - uint64_t suid; - int16_t order; - EReadMode readMode; - uint64_t rowsNum; - STimeWindow window; // the primary query time window that applies to all queries - SResultBlockInfo resBlockInfo; - SReaderStatus status; - char* idStr; // query info handle, for debug purpose - int32_t type; // query type: 1. retrieve all data blocks, 2. retrieve direct prev|next rows - SBlockLoadSuppInfo suppInfo; - STsdbReadSnap* pReadSnap; - SIOCostSummary cost; - SHashObj** pIgnoreTables; - STSchema* pSchema; // the newest version schema - SSHashObj* pSchemaMap; // keep the retrieved schema info, to avoid the overhead by repeatly load schema - SDataFReader* pFileReader; // the file reader - SDelFReader* pDelFReader; // the del file reader - SArray* pDelIdx; // del file block index; - SBlockInfoBuf blockInfoBuf; - EContentData step; - STsdbReader* innerReader[2]; -}; - -static SFileDataBlockInfo* getCurrentBlockInfo(SDataBlockIter* pBlockIter); -static int buildDataBlockFromBufImpl(STableBlockScanInfo* pBlockScanInfo, int64_t endKey, int32_t capacity, - STsdbReader* pReader); -static TSDBROW* getValidMemRow(SIterInfo* pIter, const SArray* pDelList, STsdbReader* pReader); -static int32_t doMergeRowsInFileBlocks(SBlockData* pBlockData, STableBlockScanInfo* pScanInfo, STsdbReader* pReader); -static int32_t doMergeRowsInLastBlock(SLastBlockReader* pLastBlockReader, STableBlockScanInfo* pScanInfo, int64_t ts, - SRowMerger* pMerger, SVersionRange* pVerRange, const char* id); -static int32_t doMergeRowsInBuf(SIterInfo* pIter, uint64_t uid, int64_t ts, SArray* pDelList, STsdbReader* pReader); -static int32_t doAppendRowFromTSRow(SSDataBlock* pBlock, STsdbReader* pReader, SRow* pTSRow, - STableBlockScanInfo* pScanInfo); -static int32_t doAppendRowFromFileBlock(SSDataBlock* pResBlock, STsdbReader* pReader, SBlockData* pBlockData, - int32_t rowIndex); -static void setComposedBlockFlag(STsdbReader* pReader, bool composed); -static bool hasBeenDropped(const SArray* pDelList, int32_t* index, int64_t key, int64_t ver, int32_t order, - SVersionRange* pVerRange); - -static int32_t doMergeMemTableMultiRows(TSDBROW* pRow, uint64_t uid, SIterInfo* pIter, SArray* pDelList, - TSDBROW* pResRow, STsdbReader* pReader, bool* freeTSRow); -static int32_t doMergeMemIMemRows(TSDBROW* pRow, TSDBROW* piRow, STableBlockScanInfo* pBlockScanInfo, - STsdbReader* pReader, SRow** pTSRow); -static int32_t mergeRowsInFileBlocks(SBlockData* pBlockData, STableBlockScanInfo* pBlockScanInfo, int64_t key, - STsdbReader* pReader); - -static int32_t initDelSkylineIterator(STableBlockScanInfo* pBlockScanInfo, STsdbReader* pReader, STbData* pMemTbData, - STbData* piMemTbData); -static STsdb* getTsdbByRetentions(SVnode* pVnode, TSKEY winSKey, SRetention* retentions, const char* idstr, - int8_t* pLevel); -static SVersionRange getQueryVerRange(SVnode* pVnode, SQueryTableDataCond* pCond, const char* id); -static bool hasDataInLastBlock(SLastBlockReader* pLastBlockReader); -static int32_t doBuildDataBlock(STsdbReader* pReader); -static TSDBKEY getCurrentKeyInBuf(STableBlockScanInfo* pScanInfo, STsdbReader* pReader); -static bool hasDataInFileBlock(const SBlockData* pBlockData, const SFileBlockDumpInfo* pDumpInfo); -static void initBlockDumpInfo(STsdbReader* pReader, SDataBlockIter* pBlockIter); -static int32_t getInitialDelIndex(const SArray* pDelSkyline, int32_t order); - -static STableBlockScanInfo* getTableBlockScanInfo(SSHashObj* pTableMap, uint64_t uid, const char* id); - -static bool outOfTimeWindow(int64_t ts, STimeWindow* pWindow) { return (ts > pWindow->ekey) || (ts < pWindow->skey); } - -static int32_t setColumnIdSlotList(SBlockLoadSuppInfo* pSupInfo, SColumnInfo* pCols, const int32_t* pSlotIdList, - int32_t numOfCols) { - pSupInfo->smaValid = true; - pSupInfo->numOfCols = numOfCols; - pSupInfo->colId = taosMemoryMalloc(numOfCols * (sizeof(int16_t) * 2 + POINTER_BYTES)); - if (pSupInfo->colId == NULL) { - taosMemoryFree(pSupInfo->colId); - return TSDB_CODE_OUT_OF_MEMORY; - } - - pSupInfo->slotId = (int16_t*)((char*)pSupInfo->colId + (sizeof(int16_t) * numOfCols)); - pSupInfo->buildBuf = (char**)((char*)pSupInfo->slotId + (sizeof(int16_t) * numOfCols)); - for (int32_t i = 0; i < numOfCols; ++i) { - pSupInfo->colId[i] = pCols[i].colId; - pSupInfo->slotId[i] = pSlotIdList[i]; - - if (IS_VAR_DATA_TYPE(pCols[i].type)) { - pSupInfo->buildBuf[i] = taosMemoryMalloc(pCols[i].bytes); - } else { - pSupInfo->buildBuf[i] = NULL; - } - } - - return TSDB_CODE_SUCCESS; -} - -static int32_t updateBlockSMAInfo(STSchema* pSchema, SBlockLoadSuppInfo* pSupInfo) { - int32_t i = 0, j = 0; - - if (j < pSupInfo->numOfCols && PRIMARYKEY_TIMESTAMP_COL_ID == pSupInfo->colId[j]) { - j += 1; - } - - while (i < pSchema->numOfCols && j < pSupInfo->numOfCols) { - STColumn* pTCol = &pSchema->columns[i]; - if (pTCol->colId == pSupInfo->colId[j]) { - if (!IS_BSMA_ON(pTCol)) { - pSupInfo->smaValid = false; - return TSDB_CODE_SUCCESS; - } - - i += 1; - j += 1; - } else if (pTCol->colId < pSupInfo->colId[j]) { - // do nothing - i += 1; - } else { - return TSDB_CODE_INVALID_PARA; - } - } - - return TSDB_CODE_SUCCESS; -} - -static int32_t initBlockScanInfoBuf(SBlockInfoBuf* pBuf, int32_t numOfTables) { - int32_t num = numOfTables / pBuf->numPerBucket; - int32_t remainder = numOfTables % pBuf->numPerBucket; - if (pBuf->pData == NULL) { - pBuf->pData = taosArrayInit(num + 1, POINTER_BYTES); - } - - for (int32_t i = 0; i < num; ++i) { - char* p = taosMemoryCalloc(pBuf->numPerBucket, sizeof(STableBlockScanInfo)); - if (p == NULL) { - return TSDB_CODE_OUT_OF_MEMORY; - } - - taosArrayPush(pBuf->pData, &p); - } - - if (remainder > 0) { - char* p = taosMemoryCalloc(remainder, sizeof(STableBlockScanInfo)); - if (p == NULL) { - return TSDB_CODE_OUT_OF_MEMORY; - } - taosArrayPush(pBuf->pData, &p); - } - - pBuf->numOfTables = numOfTables; - - return TSDB_CODE_SUCCESS; -} - -static int32_t ensureBlockScanInfoBuf(SBlockInfoBuf* pBuf, int32_t numOfTables) { - if (numOfTables <= pBuf->numOfTables) { - return TSDB_CODE_SUCCESS; - } - - if (pBuf->numOfTables > 0) { - STableBlockScanInfo** p = (STableBlockScanInfo**)taosArrayPop(pBuf->pData); - taosMemoryFree(*p); - pBuf->numOfTables /= pBuf->numPerBucket; - } - - int32_t num = (numOfTables - pBuf->numOfTables) / pBuf->numPerBucket; - int32_t remainder = (numOfTables - pBuf->numOfTables) % pBuf->numPerBucket; - if (pBuf->pData == NULL) { - pBuf->pData = taosArrayInit(num + 1, POINTER_BYTES); - } - - for (int32_t i = 0; i < num; ++i) { - char* p = taosMemoryCalloc(pBuf->numPerBucket, sizeof(STableBlockScanInfo)); - if (p == NULL) { - return TSDB_CODE_OUT_OF_MEMORY; - } - - taosArrayPush(pBuf->pData, &p); - } - - if (remainder > 0) { - char* p = taosMemoryCalloc(remainder, sizeof(STableBlockScanInfo)); - if (p == NULL) { - return TSDB_CODE_OUT_OF_MEMORY; - } - taosArrayPush(pBuf->pData, &p); - } - - pBuf->numOfTables = numOfTables; - - return TSDB_CODE_SUCCESS; -} - -static void clearBlockScanInfoBuf(SBlockInfoBuf* pBuf) { - size_t num = taosArrayGetSize(pBuf->pData); - for (int32_t i = 0; i < num; ++i) { - char** p = taosArrayGet(pBuf->pData, i); - taosMemoryFree(*p); - } - - taosArrayDestroy(pBuf->pData); -} - -static void* getPosInBlockInfoBuf(SBlockInfoBuf* pBuf, int32_t index) { - int32_t bucketIndex = index / pBuf->numPerBucket; - char** pBucket = taosArrayGet(pBuf->pData, bucketIndex); - return (*pBucket) + (index % pBuf->numPerBucket) * sizeof(STableBlockScanInfo); -} - -static int32_t uidComparFunc(const void* p1, const void* p2) { - uint64_t pu1 = *(uint64_t*)p1; - uint64_t pu2 = *(uint64_t*)p2; - if (pu1 == pu2) { - return 0; - } else { - return (pu1 < pu2) ? -1 : 1; - } -} - -// NOTE: speedup the whole processing by preparing the buffer for STableBlockScanInfo in batch model -static SSHashObj* createDataBlockScanInfo(STsdbReader* pTsdbReader, SBlockInfoBuf* pBuf, const STableKeyInfo* idList, - STableUidList* pUidList, int32_t numOfTables) { - // allocate buffer in order to load data blocks from file - // todo use simple hash instead, optimize the memory consumption - SSHashObj* pTableMap = tSimpleHashInit(numOfTables, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BIGINT)); - if (pTableMap == NULL) { - return NULL; - } - - int64_t st = taosGetTimestampUs(); - initBlockScanInfoBuf(pBuf, numOfTables); - - pUidList->tableUidList = taosMemoryMalloc(numOfTables * sizeof(uint64_t)); - if (pUidList->tableUidList == NULL) { - tSimpleHashCleanup(pTableMap); - return NULL; - } - - pUidList->currentIndex = 0; - - for (int32_t j = 0; j < numOfTables; ++j) { - STableBlockScanInfo* pScanInfo = getPosInBlockInfoBuf(pBuf, j); - - pScanInfo->uid = idList[j].uid; - pUidList->tableUidList[j] = idList[j].uid; - - if (ASCENDING_TRAVERSE(pTsdbReader->order)) { - int64_t skey = pTsdbReader->window.skey; - pScanInfo->lastKey = (skey > INT64_MIN) ? (skey - 1) : skey; - pScanInfo->lastKeyInStt = skey; - } else { - int64_t ekey = pTsdbReader->window.ekey; - pScanInfo->lastKey = (ekey < INT64_MAX) ? (ekey + 1) : ekey; - pScanInfo->lastKeyInStt = ekey; - } - - tSimpleHashPut(pTableMap, &pScanInfo->uid, sizeof(uint64_t), &pScanInfo, POINTER_BYTES); - tsdbTrace("%p check table uid:%" PRId64 " from lastKey:%" PRId64 " %s", pTsdbReader, pScanInfo->uid, - pScanInfo->lastKey, pTsdbReader->idStr); - } - - taosSort(pUidList->tableUidList, numOfTables, sizeof(uint64_t), uidComparFunc); - - pTsdbReader->cost.createScanInfoList = (taosGetTimestampUs() - st) / 1000.0; - tsdbDebug("%p create %d tables scan-info, size:%.2f Kb, elapsed time:%.2f ms, %s", pTsdbReader, numOfTables, - (sizeof(STableBlockScanInfo) * numOfTables) / 1024.0, pTsdbReader->cost.createScanInfoList, - pTsdbReader->idStr); - - return pTableMap; -} - -static void resetAllDataBlockScanInfo(SSHashObj* pTableMap, int64_t ts, int32_t step) { - void* p = NULL; - int32_t iter = 0; - - while ((p = tSimpleHashIterate(pTableMap, p, &iter)) != NULL) { - STableBlockScanInfo* pInfo = *(STableBlockScanInfo**)p; - - pInfo->iterInit = false; - pInfo->iter.hasVal = false; - pInfo->iiter.hasVal = false; - - if (pInfo->iter.iter != NULL) { - pInfo->iter.iter = tsdbTbDataIterDestroy(pInfo->iter.iter); - } - - if (pInfo->iiter.iter != NULL) { - pInfo->iiter.iter = tsdbTbDataIterDestroy(pInfo->iiter.iter); - } - - pInfo->delSkyline = taosArrayDestroy(pInfo->delSkyline); - pInfo->lastKey = ts; - pInfo->lastKeyInStt = ts + step; - } -} - -static void clearBlockScanInfo(STableBlockScanInfo* p) { - p->iterInit = false; - - p->iter.hasVal = false; - p->iiter.hasVal = false; - - if (p->iter.iter != NULL) { - p->iter.iter = tsdbTbDataIterDestroy(p->iter.iter); - } - - if (p->iiter.iter != NULL) { - p->iiter.iter = tsdbTbDataIterDestroy(p->iiter.iter); - } - - p->delSkyline = taosArrayDestroy(p->delSkyline); - p->pBlockList = taosArrayDestroy(p->pBlockList); - tMapDataClear(&p->mapData); -} - -static void destroyAllBlockScanInfo(SSHashObj* pTableMap) { - void* p = NULL; - int32_t iter = 0; - - while ((p = tSimpleHashIterate(pTableMap, p, &iter)) != NULL) { - clearBlockScanInfo(*(STableBlockScanInfo**)p); - } - - tSimpleHashCleanup(pTableMap); -} - -static bool isEmptyQueryTimeWindow(STimeWindow* pWindow) { return pWindow->skey > pWindow->ekey; } - -// Update the query time window according to the data time to live(TTL) information, in order to avoid to return -// the expired data to client, even it is queried already. -static STimeWindow updateQueryTimeWindow(STsdb* pTsdb, STimeWindow* pWindow) { - STsdbKeepCfg* pCfg = &pTsdb->keepCfg; - - int64_t now = taosGetTimestamp(pCfg->precision); - int64_t earilyTs = now - (tsTickPerMin[pCfg->precision] * pCfg->keep2) + 1; // needs to add one tick - - STimeWindow win = *pWindow; - if (win.skey < earilyTs) { - win.skey = earilyTs; - } - - return win; -} - -// init file iterator -static int32_t initFilesetIterator(SFilesetIter* pIter, SArray* aDFileSet, STsdbReader* pReader) { - size_t numOfFileset = taosArrayGetSize(aDFileSet); - - pIter->index = ASCENDING_TRAVERSE(pReader->order) ? -1 : numOfFileset; - pIter->order = pReader->order; - pIter->pFileList = aDFileSet; - pIter->numOfFiles = numOfFileset; - - if (pIter->pLastBlockReader == NULL) { - pIter->pLastBlockReader = taosMemoryCalloc(1, sizeof(struct SLastBlockReader)); - if (pIter->pLastBlockReader == NULL) { - int32_t code = TSDB_CODE_OUT_OF_MEMORY; - tsdbError("failed to prepare the last block iterator, since:%s %s", tstrerror(code), pReader->idStr); - return code; - } - } - - SLastBlockReader* pLReader = pIter->pLastBlockReader; - pLReader->order = pReader->order; - pLReader->window = pReader->window; - pLReader->verRange = pReader->verRange; - - pLReader->uid = 0; - tMergeTreeClose(&pLReader->mergeTree); - - if (pLReader->pInfo == NULL) { - // here we ignore the first column, which is always be the primary timestamp column - SBlockLoadSuppInfo* pInfo = &pReader->suppInfo; - - int32_t numOfStt = pReader->pTsdb->pVnode->config.sttTrigger; - pLReader->pInfo = tCreateLastBlockLoadInfo(pReader->pSchema, &pInfo->colId[1], pInfo->numOfCols - 1, numOfStt); - if (pLReader->pInfo == NULL) { - tsdbDebug("init fileset iterator failed, code:%s %s", tstrerror(terrno), pReader->idStr); - return terrno; - } - } - - tsdbDebug("init fileset iterator, total files:%d %s", pIter->numOfFiles, pReader->idStr); - return TSDB_CODE_SUCCESS; -} - -static int32_t filesetIteratorNext(SFilesetIter* pIter, STsdbReader* pReader, bool* hasNext) { - bool asc = ASCENDING_TRAVERSE(pIter->order); - int32_t step = asc ? 1 : -1; - pIter->index += step; - int32_t code = 0; - - if ((asc && pIter->index >= pIter->numOfFiles) || ((!asc) && pIter->index < 0)) { - *hasNext = false; - return TSDB_CODE_SUCCESS; - } - - SIOCostSummary* pCost = &pReader->cost; - getSttBlockLoadInfo(pIter->pLastBlockReader->pInfo, &pCost->sttCost); - - pIter->pLastBlockReader->uid = 0; - tMergeTreeClose(&pIter->pLastBlockReader->mergeTree); - resetLastBlockLoadInfo(pIter->pLastBlockReader->pInfo); - - // check file the time range of coverage - STimeWindow win = {0}; - - while (1) { - if (pReader->pFileReader != NULL) { - tsdbDataFReaderClose(&pReader->pFileReader); - } - - pReader->status.pCurrentFileset = (SDFileSet*)taosArrayGet(pIter->pFileList, pIter->index); - - code = tsdbDataFReaderOpen(&pReader->pFileReader, pReader->pTsdb, pReader->status.pCurrentFileset); - if (code != TSDB_CODE_SUCCESS) { - goto _err; - } - - pReader->cost.headFileLoad += 1; - - int32_t fid = pReader->status.pCurrentFileset->fid; - tsdbFidKeyRange(fid, pReader->pTsdb->keepCfg.days, pReader->pTsdb->keepCfg.precision, &win.skey, &win.ekey); - - // current file are no longer overlapped with query time window, ignore remain files - if ((asc && win.skey > pReader->window.ekey) || (!asc && win.ekey < pReader->window.skey)) { - tsdbDebug("%p remain files are not qualified for qrange:%" PRId64 "-%" PRId64 ", ignore, %s", pReader, - pReader->window.skey, pReader->window.ekey, pReader->idStr); - *hasNext = false; - return TSDB_CODE_SUCCESS; - } - - if ((asc && (win.ekey < pReader->window.skey)) || ((!asc) && (win.skey > pReader->window.ekey))) { - pIter->index += step; - if ((asc && pIter->index >= pIter->numOfFiles) || ((!asc) && pIter->index < 0)) { - *hasNext = false; - return TSDB_CODE_SUCCESS; - } - continue; - } - - tsdbDebug("%p file found fid:%d for qrange:%" PRId64 "-%" PRId64 ", %s", pReader, fid, pReader->window.skey, - pReader->window.ekey, pReader->idStr); - *hasNext = true; - return TSDB_CODE_SUCCESS; - } - -_err: - *hasNext = false; - return code; -} - -static void resetDataBlockIterator(SDataBlockIter* pIter, int32_t order) { - pIter->order = order; - pIter->index = -1; - pIter->numOfBlocks = 0; - if (pIter->blockList == NULL) { - pIter->blockList = taosArrayInit(4, sizeof(SFileDataBlockInfo)); - } else { - taosArrayClear(pIter->blockList); - } -} - -static void cleanupDataBlockIterator(SDataBlockIter* pIter) { taosArrayDestroy(pIter->blockList); } - -static void initReaderStatus(SReaderStatus* pStatus) { - pStatus->pTableIter = NULL; - pStatus->loadFromFile = true; -} - -static SSDataBlock* createResBlock(SQueryTableDataCond* pCond, int32_t capacity) { - SSDataBlock* pResBlock = createDataBlock(); - if (pResBlock == NULL) { - terrno = TSDB_CODE_OUT_OF_MEMORY; - return NULL; - } - - for (int32_t i = 0; i < pCond->numOfCols; ++i) { - SColumnInfoData colInfo = {0}; - colInfo.info = pCond->colList[i]; - blockDataAppendColInfo(pResBlock, &colInfo); - } - - int32_t code = blockDataEnsureCapacity(pResBlock, capacity); - if (code != TSDB_CODE_SUCCESS) { - terrno = code; - taosMemoryFree(pResBlock); - return NULL; - } - return pResBlock; -} - -static int32_t tsdbInitReaderLock(STsdbReader* pReader) { - int32_t code = -1; - qTrace("tsdb/read: %p, pre-init read mutex: %p, code: %d", pReader, &pReader->readerMutex, code); - - code = taosThreadMutexInit(&pReader->readerMutex, NULL); - - qTrace("tsdb/read: %p, post-init read mutex: %p, code: %d", pReader, &pReader->readerMutex, code); - - return code; -} - -static int32_t tsdbUninitReaderLock(STsdbReader* pReader) { - int32_t code = -1; - qTrace("tsdb/read: %p, pre-uninit read mutex: %p, code: %d", pReader, &pReader->readerMutex, code); - - code = taosThreadMutexDestroy(&pReader->readerMutex); - - qTrace("tsdb/read: %p, post-uninit read mutex: %p, code: %d", pReader, &pReader->readerMutex, code); - - return code; -} - -static int32_t tsdbAcquireReader(STsdbReader* pReader) { - int32_t code = -1; - qTrace("tsdb/read: %p, pre-take read mutex: %p, code: %d", pReader, &pReader->readerMutex, code); - - code = taosThreadMutexLock(&pReader->readerMutex); - - qTrace("tsdb/read: %p, post-take read mutex: %p, code: %d", pReader, &pReader->readerMutex, code); - - return code; -} - -static int32_t tsdbTryAcquireReader(STsdbReader* pReader) { - int32_t code = -1; - qTrace("tsdb/read: %p, pre-trytake read mutex: %p, code: %d", pReader, &pReader->readerMutex, code); - - code = taosThreadMutexTryLock(&pReader->readerMutex); - - qTrace("tsdb/read: %p, post-trytake read mutex: %p, code: %d", pReader, &pReader->readerMutex, code); - - return code; -} - -static int32_t tsdbReleaseReader(STsdbReader* pReader) { - int32_t code = -1; - qTrace("tsdb/read: %p, pre-untake read mutex: %p, code: %d", pReader, &pReader->readerMutex, code); - - code = taosThreadMutexUnlock(&pReader->readerMutex); - - qTrace("tsdb/read: %p, post-untake read mutex: %p, code: %d", pReader, &pReader->readerMutex, code); - - return code; -} - -void tsdbReleaseDataBlock(STsdbReader* pReader) { - SReaderStatus* pStatus = &pReader->status; - if (!pStatus->composedDataBlock) { - tsdbReleaseReader(pReader); - } -} - -static int32_t initResBlockInfo(SResultBlockInfo* pResBlockInfo, int64_t capacity, SSDataBlock* pResBlock, - SQueryTableDataCond* pCond) { - pResBlockInfo->capacity = capacity; - pResBlockInfo->pResBlock = pResBlock; - terrno = 0; - - if (pResBlockInfo->pResBlock == NULL) { - pResBlockInfo->freeBlock = true; - pResBlockInfo->pResBlock = createResBlock(pCond, pResBlockInfo->capacity); - } else { - pResBlockInfo->freeBlock = false; - } - - return terrno; -} - -static int32_t tsdbReaderCreate(SVnode* pVnode, SQueryTableDataCond* pCond, void** ppReader, int32_t capacity, - SSDataBlock* pResBlock, const char* idstr) { - int32_t code = 0; - int8_t level = 0; - STsdbReader* pReader = (STsdbReader*)taosMemoryCalloc(1, sizeof(*pReader)); - if (pReader == NULL) { - code = TSDB_CODE_OUT_OF_MEMORY; - goto _end; - } - - if (VND_IS_TSMA(pVnode)) { - tsdbDebug("vgId:%d, tsma is selected to query, %s", TD_VID(pVnode), idstr); - } - - initReaderStatus(&pReader->status); - - pReader->pTsdb = getTsdbByRetentions(pVnode, pCond->twindows.skey, pVnode->config.tsdbCfg.retentions, idstr, &level); - pReader->suid = pCond->suid; - pReader->order = pCond->order; - - pReader->idStr = (idstr != NULL) ? taosStrdup(idstr) : NULL; - pReader->verRange = getQueryVerRange(pVnode, pCond, idstr); - pReader->type = pCond->type; - pReader->window = updateQueryTimeWindow(pReader->pTsdb, &pCond->twindows); - pReader->blockInfoBuf.numPerBucket = 1000; // 1000 tables per bucket - - code = initResBlockInfo(&pReader->resBlockInfo, capacity, pResBlock, pCond); - if (code != TSDB_CODE_SUCCESS) { - goto _end; - } - - if (pCond->numOfCols <= 0) { - tsdbError("vgId:%d, invalid column number %d in query cond, %s", TD_VID(pVnode), pCond->numOfCols, idstr); - code = TSDB_CODE_INVALID_PARA; - goto _end; - } - - // allocate buffer in order to load data blocks from file - SBlockLoadSuppInfo* pSup = &pReader->suppInfo; - pSup->pColAgg = taosArrayInit(pCond->numOfCols, sizeof(SColumnDataAgg)); - if (pSup->pColAgg == NULL) { - code = TSDB_CODE_OUT_OF_MEMORY; - goto _end; - } - - pSup->tsColAgg.colId = PRIMARYKEY_TIMESTAMP_COL_ID; - setColumnIdSlotList(pSup, pCond->colList, pCond->pSlotList, pCond->numOfCols); - - code = tBlockDataCreate(&pReader->status.fileBlockData); - if (code != TSDB_CODE_SUCCESS) { - terrno = code; - goto _end; - } - - if (pReader->suppInfo.colId[0] != PRIMARYKEY_TIMESTAMP_COL_ID) { - tsdbError("the first column isn't primary timestamp, %d, %s", pReader->suppInfo.colId[0], pReader->idStr); - code = TSDB_CODE_INVALID_PARA; - goto _end; - } - - pReader->status.pPrimaryTsCol = taosArrayGet(pReader->resBlockInfo.pResBlock->pDataBlock, pSup->slotId[0]); - int32_t type = pReader->status.pPrimaryTsCol->info.type; - if (type != TSDB_DATA_TYPE_TIMESTAMP) { - tsdbError("the first column isn't primary timestamp in result block, actual: %s, %s", tDataTypes[type].name, - pReader->idStr); - code = TSDB_CODE_INVALID_PARA; - goto _end; - } - - tsdbInitReaderLock(pReader); - - *ppReader = pReader; - return code; - -_end: - tsdbReaderClose(pReader); - *ppReader = NULL; - return code; -} - -static int32_t doLoadBlockIndex(STsdbReader* pReader, SDataFReader* pFileReader, SArray* pIndexList) { - int64_t st = taosGetTimestampUs(); - LRUHandle* handle = NULL; - int32_t code = tsdbCacheGetBlockIdx(pFileReader->pTsdb->biCache, pFileReader, &handle); - if (code != TSDB_CODE_SUCCESS || handle == NULL) { - goto _end; - } - - int32_t numOfTables = tSimpleHashGetSize(pReader->status.pTableMap); - - SArray* aBlockIdx = (SArray*)taosLRUCacheValue(pFileReader->pTsdb->biCache, handle); - size_t num = taosArrayGetSize(aBlockIdx); - if (num == 0) { - tsdbBICacheRelease(pFileReader->pTsdb->biCache, handle); - return TSDB_CODE_SUCCESS; - } - - // todo binary search to the start position - int64_t et1 = taosGetTimestampUs(); - - SBlockIdx* pBlockIdx = NULL; - STableUidList* pList = &pReader->status.uidList; - - int32_t i = 0, j = 0; - while (i < num && j < numOfTables) { - pBlockIdx = (SBlockIdx*)taosArrayGet(aBlockIdx, i); - if (pBlockIdx->suid != pReader->suid) { - i += 1; - continue; - } - - if (pBlockIdx->uid < pList->tableUidList[j]) { - i += 1; - continue; - } - - if (pBlockIdx->uid > pList->tableUidList[j]) { - j += 1; - continue; - } - - if (pBlockIdx->uid == pList->tableUidList[j]) { - // this block belongs to a table that is not queried. - STableBlockScanInfo* pScanInfo = getTableBlockScanInfo(pReader->status.pTableMap, pBlockIdx->uid, pReader->idStr); - if (pScanInfo == NULL) { - tsdbBICacheRelease(pFileReader->pTsdb->biCache, handle); - return terrno; - } - - if (pScanInfo->pBlockList == NULL) { - pScanInfo->pBlockList = taosArrayInit(4, sizeof(SBlockIndex)); - } - - taosArrayPush(pIndexList, pBlockIdx); - - i += 1; - j += 1; - } - } - - int64_t et2 = taosGetTimestampUs(); - tsdbDebug("load block index for %d/%d tables completed, elapsed time:%.2f ms, set blockIdx:%.2f ms, size:%.2f Kb %s", - numOfTables, (int32_t)num, (et1 - st) / 1000.0, (et2 - et1) / 1000.0, num * sizeof(SBlockIdx) / 1024.0, - pReader->idStr); - - pReader->cost.headFileLoadTime += (et1 - st) / 1000.0; - -_end: - tsdbBICacheRelease(pFileReader->pTsdb->biCache, handle); - return code; -} - -static void doCleanupTableScanInfo(STableBlockScanInfo* pScanInfo) { - // reset the index in last block when handing a new file - tMapDataClear(&pScanInfo->mapData); - taosArrayClear(pScanInfo->pBlockList); -} - -static void cleanupTableScanInfo(SReaderStatus* pStatus) { - if (pStatus->mapDataCleaned) { - return; - } - - SSHashObj* pTableMap = pStatus->pTableMap; - STableBlockScanInfo** px = NULL; - int32_t iter = 0; - - while (1) { - px = tSimpleHashIterate(pTableMap, px, &iter); - if (px == NULL) { - break; - } - - doCleanupTableScanInfo(*px); - } - - pStatus->mapDataCleaned = true; -} - -static int32_t doLoadFileBlock(STsdbReader* pReader, SArray* pIndexList, SBlockNumber* pBlockNum, - SArray* pTableScanInfoList) { - size_t sizeInDisk = 0; - size_t numOfTables = taosArrayGetSize(pIndexList); - - int64_t st = taosGetTimestampUs(); - cleanupTableScanInfo(&pReader->status); - - // set the flag for the new file - pReader->status.mapDataCleaned = false; - for (int32_t i = 0; i < numOfTables; ++i) { - SBlockIdx* pBlockIdx = taosArrayGet(pIndexList, i); - STableBlockScanInfo* pScanInfo = getTableBlockScanInfo(pReader->status.pTableMap, pBlockIdx->uid, pReader->idStr); - if (pScanInfo == NULL) { - return terrno; - } - - tMapDataReset(&pScanInfo->mapData); - tsdbReadDataBlk(pReader->pFileReader, pBlockIdx, &pScanInfo->mapData); - taosArrayEnsureCap(pScanInfo->pBlockList, pScanInfo->mapData.nItem); - - sizeInDisk += pScanInfo->mapData.nData; - - int32_t step = ASCENDING_TRAVERSE(pReader->order) ? 1 : -1; - STimeWindow w = pReader->window; - if (ASCENDING_TRAVERSE(pReader->order)) { - w.skey = pScanInfo->lastKey + step; - } else { - w.ekey = pScanInfo->lastKey + step; - } - - if (isEmptyQueryTimeWindow(&w)) { - continue; - } - - SDataBlk block = {0}; - for (int32_t j = 0; j < pScanInfo->mapData.nItem; ++j) { - tGetDataBlk(pScanInfo->mapData.pData + pScanInfo->mapData.aOffset[j], &block); - - // 1. time range check - // if (block.minKey.ts > pReader->window.ekey || block.maxKey.ts < pReader->window.skey) { - if (block.minKey.ts > w.ekey || block.maxKey.ts < w.skey) { - continue; - } - - // 2. version range check - if (block.minVer > pReader->verRange.maxVer || block.maxVer < pReader->verRange.minVer) { - continue; - } - - SBlockIndex bIndex = {.ordinalIndex = j, .inFileOffset = block.aSubBlock->offset}; - bIndex.window = (STimeWindow){.skey = block.minKey.ts, .ekey = block.maxKey.ts}; - - void* p1 = taosArrayPush(pScanInfo->pBlockList, &bIndex); - if (p1 == NULL) { - tMapDataClear(&pScanInfo->mapData); - return TSDB_CODE_OUT_OF_MEMORY; - } - - pBlockNum->numOfBlocks += 1; - } - - if (taosArrayGetSize(pScanInfo->pBlockList) > 0) { - taosArrayPush(pTableScanInfoList, &pScanInfo); - } - } - - pBlockNum->numOfLastFiles = pReader->pFileReader->pSet->nSttF; - int32_t total = pBlockNum->numOfLastFiles + pBlockNum->numOfBlocks; - - double el = (taosGetTimestampUs() - st) / 1000.0; - tsdbDebug( - "load block of %ld tables completed, blocks:%d in %d tables, last-files:%d, block-info-size:%.2f Kb, elapsed " - "time:%.2f ms %s", - numOfTables, pBlockNum->numOfBlocks, (int32_t)taosArrayGetSize(pTableScanInfoList), pBlockNum->numOfLastFiles, - sizeInDisk / 1000.0, el, pReader->idStr); - - pReader->cost.numOfBlocks += total; - pReader->cost.headFileLoadTime += el; - - return TSDB_CODE_SUCCESS; -} - -static void setBlockAllDumped(SFileBlockDumpInfo* pDumpInfo, int64_t maxKey, int32_t order) { - int32_t step = ASCENDING_TRAVERSE(order) ? 1 : -1; - pDumpInfo->allDumped = true; - pDumpInfo->lastKey = maxKey + step; -} - -static int32_t doCopyColVal(SColumnInfoData* pColInfoData, int32_t rowIndex, int32_t colIndex, SColVal* pColVal, - SBlockLoadSuppInfo* pSup) { - if (IS_VAR_DATA_TYPE(pColVal->type)) { - if (!COL_VAL_IS_VALUE(pColVal)) { - colDataSetNULL(pColInfoData, rowIndex); - } else { - varDataSetLen(pSup->buildBuf[colIndex], pColVal->value.nData); - if (pColVal->value.nData > pColInfoData->info.bytes) { - tsdbWarn("column cid:%d actual data len %d is bigger than schema len %d", pColVal->cid, pColVal->value.nData, - pColInfoData->info.bytes); - return TSDB_CODE_TDB_INVALID_TABLE_SCHEMA_VER; - } - if (pColVal->value.nData > 0) { // pData may be null, if nData is 0 - memcpy(varDataVal(pSup->buildBuf[colIndex]), pColVal->value.pData, pColVal->value.nData); - } - - colDataSetVal(pColInfoData, rowIndex, pSup->buildBuf[colIndex], false); - } - } else { - colDataSetVal(pColInfoData, rowIndex, (const char*)&pColVal->value, !COL_VAL_IS_VALUE(pColVal)); - } - - return TSDB_CODE_SUCCESS; -} - -static SFileDataBlockInfo* getCurrentBlockInfo(SDataBlockIter* pBlockIter) { - size_t num = taosArrayGetSize(pBlockIter->blockList); - if (num == 0) { - ASSERT(pBlockIter->numOfBlocks == num); - return NULL; - } - - SFileDataBlockInfo* pBlockInfo = taosArrayGet(pBlockIter->blockList, pBlockIter->index); - return pBlockInfo; -} - -static SDataBlk* getCurrentBlock(SDataBlockIter* pBlockIter) { return &pBlockIter->block; } - -static int doBinarySearchKey(TSKEY* keyList, int num, int pos, TSKEY key, int order) { - // start end position - int s, e; - s = pos; - - // check - ASSERT(pos >= 0 && pos < num && num > 0); - if (order == TSDB_ORDER_ASC) { - // find the first position which is smaller than the key - e = num - 1; - if (key < keyList[pos]) return -1; - while (1) { - // check can return - if (key >= keyList[e]) return e; - if (key <= keyList[s]) return s; - if (e - s <= 1) return s; - - // change start or end position - int mid = s + (e - s + 1) / 2; - if (keyList[mid] > key) - e = mid; - else if (keyList[mid] < key) - s = mid; - else - return mid; - } - } else { // DESC - // find the first position which is bigger than the key - e = 0; - if (key > keyList[pos]) return -1; - while (1) { - // check can return - if (key <= keyList[e]) return e; - if (key >= keyList[s]) return s; - if (s - e <= 1) return s; - - // change start or end position - int mid = s - (s - e + 1) / 2; - if (keyList[mid] < key) - e = mid; - else if (keyList[mid] > key) - s = mid; - else - return mid; - } - } -} - -static int32_t getEndPosInDataBlock(STsdbReader* pReader, SBlockData* pBlockData, SDataBlk* pBlock, int32_t pos) { - // NOTE: reverse the order to find the end position in data block - int32_t endPos = -1; - bool asc = ASCENDING_TRAVERSE(pReader->order); - - if (asc && pReader->window.ekey >= pBlock->maxKey.ts) { - endPos = pBlock->nRow - 1; - } else if (!asc && pReader->window.skey <= pBlock->minKey.ts) { - endPos = 0; - } else { - int64_t key = asc ? pReader->window.ekey : pReader->window.skey; - endPos = doBinarySearchKey(pBlockData->aTSKEY, pBlock->nRow, pos, key, pReader->order); - } - - if ((pReader->verRange.maxVer >= pBlock->minVer && pReader->verRange.maxVer < pBlock->maxVer) || - (pReader->verRange.minVer <= pBlock->maxVer && pReader->verRange.minVer > pBlock->minVer)) { - int32_t i = endPos; - - if (asc) { - for (; i >= 0; --i) { - if (pBlockData->aVersion[i] <= pReader->verRange.maxVer) { - break; - } - } - } else { - for (; i < pBlock->nRow; ++i) { - if (pBlockData->aVersion[i] >= pReader->verRange.minVer) { - break; - } - } - } - - endPos = i; - } - - return endPos; -} - -static void copyPrimaryTsCol(const SBlockData* pBlockData, SFileBlockDumpInfo* pDumpInfo, SColumnInfoData* pColData, - int32_t dumpedRows, bool asc) { - if (asc) { - memcpy(pColData->pData, &pBlockData->aTSKEY[pDumpInfo->rowIndex], dumpedRows * sizeof(int64_t)); - } else { - int32_t startIndex = pDumpInfo->rowIndex - dumpedRows + 1; - memcpy(pColData->pData, &pBlockData->aTSKEY[startIndex], dumpedRows * sizeof(int64_t)); - - // todo: opt perf by extract the loop - // reverse the array list - int32_t mid = dumpedRows >> 1u; - int64_t* pts = (int64_t*)pColData->pData; - for (int32_t j = 0; j < mid; ++j) { - int64_t t = pts[j]; - pts[j] = pts[dumpedRows - j - 1]; - pts[dumpedRows - j - 1] = t; - } - } -} - -// a faster version of copy procedure. -static void copyNumericCols(const SColData* pData, SFileBlockDumpInfo* pDumpInfo, SColumnInfoData* pColData, - int32_t dumpedRows, bool asc) { - uint8_t* p = NULL; - if (asc) { - p = pData->pData + tDataTypes[pData->type].bytes * pDumpInfo->rowIndex; - } else { - int32_t startIndex = pDumpInfo->rowIndex - dumpedRows + 1; - p = pData->pData + tDataTypes[pData->type].bytes * startIndex; - } - - int32_t step = asc ? 1 : -1; - - // make sure it is aligned to 8bit, the allocated memory address is aligned to 256bit - // ASSERT((((uint64_t)pColData->pData) & (0x8 - 1)) == 0); - - // 1. copy data in a batch model - memcpy(pColData->pData, p, dumpedRows * tDataTypes[pData->type].bytes); - - // 2. reverse the array list in case of descending order scan data block - if (!asc) { - switch (pColData->info.type) { - case TSDB_DATA_TYPE_TIMESTAMP: - case TSDB_DATA_TYPE_DOUBLE: - case TSDB_DATA_TYPE_BIGINT: - case TSDB_DATA_TYPE_UBIGINT: { - int32_t mid = dumpedRows >> 1u; - int64_t* pts = (int64_t*)pColData->pData; - for (int32_t j = 0; j < mid; ++j) { - int64_t t = pts[j]; - pts[j] = pts[dumpedRows - j - 1]; - pts[dumpedRows - j - 1] = t; - } - break; - } - - case TSDB_DATA_TYPE_BOOL: - case TSDB_DATA_TYPE_TINYINT: - case TSDB_DATA_TYPE_UTINYINT: { - int32_t mid = dumpedRows >> 1u; - int8_t* pts = (int8_t*)pColData->pData; - for (int32_t j = 0; j < mid; ++j) { - int8_t t = pts[j]; - pts[j] = pts[dumpedRows - j - 1]; - pts[dumpedRows - j - 1] = t; - } - break; - } - - case TSDB_DATA_TYPE_SMALLINT: - case TSDB_DATA_TYPE_USMALLINT: { - int32_t mid = dumpedRows >> 1u; - int16_t* pts = (int16_t*)pColData->pData; - for (int32_t j = 0; j < mid; ++j) { - int64_t t = pts[j]; - pts[j] = pts[dumpedRows - j - 1]; - pts[dumpedRows - j - 1] = t; - } - break; - } - - case TSDB_DATA_TYPE_FLOAT: - case TSDB_DATA_TYPE_INT: - case TSDB_DATA_TYPE_UINT: { - int32_t mid = dumpedRows >> 1u; - int32_t* pts = (int32_t*)pColData->pData; - for (int32_t j = 0; j < mid; ++j) { - int32_t t = pts[j]; - pts[j] = pts[dumpedRows - j - 1]; - pts[dumpedRows - j - 1] = t; - } - break; - } - } - } - - // 3. if the null value exists, check items one-by-one - if (pData->flag != HAS_VALUE) { - int32_t rowIndex = 0; - - for (int32_t j = pDumpInfo->rowIndex; rowIndex < dumpedRows; j += step, rowIndex++) { - uint8_t v = tColDataGetBitValue(pData, j); - if (v == 0 || v == 1) { - colDataSetNull_f(pColData->nullbitmap, rowIndex); - pColData->hasNull = true; - } - } - } -} - -static int32_t copyBlockDataToSDataBlock(STsdbReader* pReader) { - SReaderStatus* pStatus = &pReader->status; - SDataBlockIter* pBlockIter = &pStatus->blockIter; - SBlockLoadSuppInfo* pSupInfo = &pReader->suppInfo; - SFileBlockDumpInfo* pDumpInfo = &pReader->status.fBlockDumpInfo; - - SBlockData* pBlockData = &pStatus->fileBlockData; - SFileDataBlockInfo* pBlockInfo = getCurrentBlockInfo(pBlockIter); - SDataBlk* pBlock = getCurrentBlock(pBlockIter); - SSDataBlock* pResBlock = pReader->resBlockInfo.pResBlock; - int32_t numOfOutputCols = pSupInfo->numOfCols; - int32_t code = TSDB_CODE_SUCCESS; - - SColVal cv = {0}; - int64_t st = taosGetTimestampUs(); - bool asc = ASCENDING_TRAVERSE(pReader->order); - int32_t step = asc ? 1 : -1; - - // no data exists, return directly. - if (pBlockData->nRow == 0 || pBlockData->aTSKEY == 0) { - tsdbWarn("%p no need to copy since no data in blockData, table uid:%" PRIu64 " has been dropped, %s", pReader, - pBlockInfo->uid, pReader->idStr); - pResBlock->info.rows = 0; - return 0; - } - - // row index of dump info remain the initial position, let's find the appropriate start position. - if ((pDumpInfo->rowIndex == 0 && asc) || (pDumpInfo->rowIndex == pBlock->nRow - 1 && (!asc))) { - if (asc && pReader->window.skey <= pBlock->minKey.ts && pReader->verRange.minVer <= pBlock->minVer) { - // pDumpInfo->rowIndex = 0; - } else if (!asc && pReader->window.ekey >= pBlock->maxKey.ts && pReader->verRange.maxVer >= pBlock->maxVer) { - // pDumpInfo->rowIndex = pBlock->nRow - 1; - } else { // find the appropriate the start position in current block, and set it to be the current rowIndex - int32_t pos = asc ? pBlock->nRow - 1 : 0; - int32_t order = asc ? TSDB_ORDER_DESC : TSDB_ORDER_ASC; - int64_t key = asc ? pReader->window.skey : pReader->window.ekey; - pDumpInfo->rowIndex = doBinarySearchKey(pBlockData->aTSKEY, pBlock->nRow, pos, key, order); - - if (pDumpInfo->rowIndex < 0) { - tsdbError( - "%p failed to locate the start position in current block, global index:%d, table index:%d, brange:%" PRId64 - "-%" PRId64 ", minVer:%" PRId64 ", maxVer:%" PRId64 " %s", - pReader, pBlockIter->index, pBlockInfo->tbBlockIdx, pBlock->minKey.ts, pBlock->maxKey.ts, pBlock->minVer, - pBlock->maxVer, pReader->idStr); - return TSDB_CODE_INVALID_PARA; - } - - ASSERT(pReader->verRange.minVer <= pBlock->maxVer && pReader->verRange.maxVer >= pBlock->minVer); - - // find the appropriate start position that satisfies the version requirement. - if ((pReader->verRange.maxVer >= pBlock->minVer && pReader->verRange.maxVer < pBlock->maxVer) || - (pReader->verRange.minVer <= pBlock->maxVer && pReader->verRange.minVer > pBlock->minVer)) { - int32_t i = pDumpInfo->rowIndex; - if (asc) { - for (; i < pBlock->nRow; ++i) { - if (pBlockData->aVersion[i] >= pReader->verRange.minVer) { - break; - } - } - } else { - for (; i >= 0; --i) { - if (pBlockData->aVersion[i] <= pReader->verRange.maxVer) { - break; - } - } - } - - pDumpInfo->rowIndex = i; - } - } - } - - // time window check - int32_t endIndex = getEndPosInDataBlock(pReader, pBlockData, pBlock, pDumpInfo->rowIndex); - if (endIndex == -1) { - setBlockAllDumped(pDumpInfo, pReader->window.ekey, pReader->order); - return TSDB_CODE_SUCCESS; - } - - endIndex += step; - int32_t dumpedRows = asc ? (endIndex - pDumpInfo->rowIndex) : (pDumpInfo->rowIndex - endIndex); - if (dumpedRows > pReader->resBlockInfo.capacity) { // output buffer check - dumpedRows = pReader->resBlockInfo.capacity; - } else if (dumpedRows <= 0) { // no qualified rows in current data block, abort directly. - setBlockAllDumped(pDumpInfo, pReader->window.ekey, pReader->order); - return TSDB_CODE_SUCCESS; - } - - int32_t i = 0; - int32_t rowIndex = 0; - - SColumnInfoData* pColData = taosArrayGet(pResBlock->pDataBlock, pSupInfo->slotId[i]); - if (pSupInfo->colId[i] == PRIMARYKEY_TIMESTAMP_COL_ID) { - copyPrimaryTsCol(pBlockData, pDumpInfo, pColData, dumpedRows, asc); - i += 1; - } - - int32_t colIndex = 0; - int32_t num = pBlockData->nColData; - while (i < numOfOutputCols && colIndex < num) { - rowIndex = 0; - - SColData* pData = tBlockDataGetColDataByIdx(pBlockData, colIndex); - if (pData->cid < pSupInfo->colId[i]) { - colIndex += 1; - } else if (pData->cid == pSupInfo->colId[i]) { - pColData = taosArrayGet(pResBlock->pDataBlock, pSupInfo->slotId[i]); - - if (pData->flag == HAS_NONE || pData->flag == HAS_NULL || pData->flag == (HAS_NULL | HAS_NONE)) { - colDataSetNNULL(pColData, 0, dumpedRows); - } else { - if (IS_MATHABLE_TYPE(pColData->info.type)) { - copyNumericCols(pData, pDumpInfo, pColData, dumpedRows, asc); - } else { // varchar/nchar type - for (int32_t j = pDumpInfo->rowIndex; rowIndex < dumpedRows; j += step) { - tColDataGetValue(pData, j, &cv); - code = doCopyColVal(pColData, rowIndex++, i, &cv, pSupInfo); - if (code) { - return code; - } - } - } - } - - colIndex += 1; - i += 1; - } else { // the specified column does not exist in file block, fill with null data - pColData = taosArrayGet(pResBlock->pDataBlock, pSupInfo->slotId[i]); - colDataSetNNULL(pColData, 0, dumpedRows); - i += 1; - } - } - - // fill the mis-matched columns with null value - while (i < numOfOutputCols) { - pColData = taosArrayGet(pResBlock->pDataBlock, pSupInfo->slotId[i]); - colDataSetNNULL(pColData, 0, dumpedRows); - i += 1; - } - - pResBlock->info.dataLoad = 1; - pResBlock->info.rows = dumpedRows; - pDumpInfo->rowIndex += step * dumpedRows; - - // check if current block are all handled - if (pDumpInfo->rowIndex >= 0 && pDumpInfo->rowIndex < pBlock->nRow) { - int64_t ts = pBlockData->aTSKEY[pDumpInfo->rowIndex]; - if (outOfTimeWindow(ts, &pReader->window)) { // the remain data has out of query time window, ignore current block - setBlockAllDumped(pDumpInfo, ts, pReader->order); - } - } else { - int64_t ts = asc ? pBlock->maxKey.ts : pBlock->minKey.ts; - setBlockAllDumped(pDumpInfo, ts, pReader->order); - } - - double elapsedTime = (taosGetTimestampUs() - st) / 1000.0; - pReader->cost.blockLoadTime += elapsedTime; - - int32_t unDumpedRows = asc ? pBlock->nRow - pDumpInfo->rowIndex : pDumpInfo->rowIndex + 1; - tsdbDebug("%p copy file block to sdatablock, global index:%d, table index:%d, brange:%" PRId64 "-%" PRId64 - ", rows:%d, remain:%d, minVer:%" PRId64 ", maxVer:%" PRId64 ", uid:%" PRIu64 " elapsed time:%.2f ms, %s", - pReader, pBlockIter->index, pBlockInfo->tbBlockIdx, pBlock->minKey.ts, pBlock->maxKey.ts, dumpedRows, - unDumpedRows, pBlock->minVer, pBlock->maxVer, pBlockInfo->uid, elapsedTime, pReader->idStr); - - return TSDB_CODE_SUCCESS; -} - -static FORCE_INLINE STSchema* getTableSchemaImpl(STsdbReader* pReader, uint64_t uid) { - ASSERT(pReader->pSchema == NULL); - - int32_t code = metaGetTbTSchemaEx(pReader->pTsdb->pVnode->pMeta, pReader->suid, uid, -1, &pReader->pSchema); - if (code != TSDB_CODE_SUCCESS || pReader->pSchema == NULL) { - terrno = code; - tsdbError("failed to get table schema, uid:%" PRIu64 ", it may have been dropped, ver:-1, %s", uid, pReader->idStr); - return NULL; - } - - code = tsdbRowMergerInit(&pReader->status.merger, pReader->pSchema); - if (code != TSDB_CODE_SUCCESS) { - terrno = code; - tsdbError("failed to init merger, code:%s, %s", tstrerror(code), pReader->idStr); - return NULL; - } - - return pReader->pSchema; -} - -static int32_t doLoadFileBlockData(STsdbReader* pReader, SDataBlockIter* pBlockIter, SBlockData* pBlockData, - uint64_t uid) { - int32_t code = 0; - STSchema* pSchema = pReader->pSchema; - int64_t st = taosGetTimestampUs(); - - tBlockDataReset(pBlockData); - - if (pReader->pSchema == NULL) { - pSchema = getTableSchemaImpl(pReader, uid); - if (pSchema == NULL) { - tsdbDebug("%p table uid:%" PRIu64 " has been dropped, no data existed, %s", pReader, uid, pReader->idStr); - return code; - } - } - - SBlockLoadSuppInfo* pSup = &pReader->suppInfo; - TABLEID tid = {.suid = pReader->suid, .uid = uid}; - code = tBlockDataInit(pBlockData, &tid, pSchema, &pSup->colId[1], pSup->numOfCols - 1); - if (code != TSDB_CODE_SUCCESS) { - return code; - } - - SFileDataBlockInfo* pBlockInfo = getCurrentBlockInfo(pBlockIter); - SFileBlockDumpInfo* pDumpInfo = &pReader->status.fBlockDumpInfo; - - SDataBlk* pBlock = getCurrentBlock(pBlockIter); - code = tsdbReadDataBlock(pReader->pFileReader, pBlock, pBlockData); - if (code != TSDB_CODE_SUCCESS) { - tsdbError("%p error occurs in loading file block, global index:%d, table index:%d, brange:%" PRId64 "-%" PRId64 - ", rows:%d, code:%s %s", - pReader, pBlockIter->index, pBlockInfo->tbBlockIdx, pBlock->minKey.ts, pBlock->maxKey.ts, pBlock->nRow, - tstrerror(code), pReader->idStr); - return code; - } - - double elapsedTime = (taosGetTimestampUs() - st) / 1000.0; - - tsdbDebug("%p load file block into buffer, global index:%d, index in table block list:%d, brange:%" PRId64 "-%" PRId64 - ", rows:%d, minVer:%" PRId64 ", maxVer:%" PRId64 ", elapsed time:%.2f ms, %s", - pReader, pBlockIter->index, pBlockInfo->tbBlockIdx, pBlock->minKey.ts, pBlock->maxKey.ts, pBlock->nRow, - pBlock->minVer, pBlock->maxVer, elapsedTime, pReader->idStr); - - pReader->cost.blockLoadTime += elapsedTime; - pDumpInfo->allDumped = false; - - return TSDB_CODE_SUCCESS; -} - -static void cleanupBlockOrderSupporter(SBlockOrderSupporter* pSup) { - taosMemoryFreeClear(pSup->numOfBlocksPerTable); - taosMemoryFreeClear(pSup->indexPerTable); - - for (int32_t i = 0; i < pSup->numOfTables; ++i) { - SBlockOrderWrapper* pBlockInfo = pSup->pDataBlockInfo[i]; - taosMemoryFreeClear(pBlockInfo); - } - - taosMemoryFreeClear(pSup->pDataBlockInfo); -} - -static int32_t initBlockOrderSupporter(SBlockOrderSupporter* pSup, int32_t numOfTables) { - pSup->numOfBlocksPerTable = taosMemoryCalloc(1, sizeof(int32_t) * numOfTables); - pSup->indexPerTable = taosMemoryCalloc(1, sizeof(int32_t) * numOfTables); - pSup->pDataBlockInfo = taosMemoryCalloc(1, POINTER_BYTES * numOfTables); - - if (pSup->numOfBlocksPerTable == NULL || pSup->indexPerTable == NULL || pSup->pDataBlockInfo == NULL) { - cleanupBlockOrderSupporter(pSup); - return TSDB_CODE_OUT_OF_MEMORY; - } - - return TSDB_CODE_SUCCESS; -} - -static int32_t fileDataBlockOrderCompar(const void* pLeft, const void* pRight, void* param) { - int32_t leftIndex = *(int32_t*)pLeft; - int32_t rightIndex = *(int32_t*)pRight; - - SBlockOrderSupporter* pSupporter = (SBlockOrderSupporter*)param; - - int32_t leftTableBlockIndex = pSupporter->indexPerTable[leftIndex]; - int32_t rightTableBlockIndex = pSupporter->indexPerTable[rightIndex]; - - if (leftTableBlockIndex > pSupporter->numOfBlocksPerTable[leftIndex]) { - /* left block is empty */ - return 1; - } else if (rightTableBlockIndex > pSupporter->numOfBlocksPerTable[rightIndex]) { - /* right block is empty */ - return -1; - } - - SBlockOrderWrapper* pLeftBlock = &pSupporter->pDataBlockInfo[leftIndex][leftTableBlockIndex]; - SBlockOrderWrapper* pRightBlock = &pSupporter->pDataBlockInfo[rightIndex][rightTableBlockIndex]; - - return pLeftBlock->offset > pRightBlock->offset ? 1 : -1; -} - -static int32_t doSetCurrentBlock(SDataBlockIter* pBlockIter, const char* idStr) { - SFileDataBlockInfo* pBlockInfo = getCurrentBlockInfo(pBlockIter); - if (pBlockInfo != NULL) { - STableBlockScanInfo* pScanInfo = getTableBlockScanInfo(pBlockIter->pTableMap, pBlockInfo->uid, idStr); - if (pScanInfo == NULL) { - return terrno; - } - - SBlockIndex* pIndex = taosArrayGet(pScanInfo->pBlockList, pBlockInfo->tbBlockIdx); - tMapDataGetItemByIdx(&pScanInfo->mapData, pIndex->ordinalIndex, &pBlockIter->block, tGetDataBlk); - } - -#if 0 - qDebug("check file block, table uid:%"PRIu64" index:%d offset:%"PRId64", ", pScanInfo->uid, *mapDataIndex, pBlockIter->block.aSubBlock[0].offset); -#endif - - return TSDB_CODE_SUCCESS; -} - -static int32_t initBlockIterator(STsdbReader* pReader, SDataBlockIter* pBlockIter, int32_t numOfBlocks, - SArray* pTableList) { - bool asc = ASCENDING_TRAVERSE(pReader->order); - - SBlockOrderSupporter sup = {0}; - pBlockIter->numOfBlocks = numOfBlocks; - taosArrayClear(pBlockIter->blockList); - pBlockIter->pTableMap = pReader->status.pTableMap; - - // access data blocks according to the offset of each block in asc/desc order. - int32_t numOfTables = taosArrayGetSize(pTableList); - - int64_t st = taosGetTimestampUs(); - int32_t code = initBlockOrderSupporter(&sup, numOfTables); - if (code != TSDB_CODE_SUCCESS) { - return code; - } - - int32_t cnt = 0; - - for (int32_t i = 0; i < numOfTables; ++i) { - STableBlockScanInfo* pTableScanInfo = taosArrayGetP(pTableList, i); - ASSERT(pTableScanInfo->pBlockList != NULL && taosArrayGetSize(pTableScanInfo->pBlockList) > 0); - - size_t num = taosArrayGetSize(pTableScanInfo->pBlockList); - sup.numOfBlocksPerTable[sup.numOfTables] = num; - - char* buf = taosMemoryMalloc(sizeof(SBlockOrderWrapper) * num); - if (buf == NULL) { - cleanupBlockOrderSupporter(&sup); - return TSDB_CODE_OUT_OF_MEMORY; - } - - sup.pDataBlockInfo[sup.numOfTables] = (SBlockOrderWrapper*)buf; - - for (int32_t k = 0; k < num; ++k) { - SBlockIndex* pIndex = taosArrayGet(pTableScanInfo->pBlockList, k); - sup.pDataBlockInfo[sup.numOfTables][k] = - (SBlockOrderWrapper){.uid = pTableScanInfo->uid, .offset = pIndex->inFileOffset}; - cnt++; - } - - sup.numOfTables += 1; - } - - if (numOfBlocks != cnt && sup.numOfTables != numOfTables) { - cleanupBlockOrderSupporter(&sup); - return TSDB_CODE_INVALID_PARA; - } - - // since there is only one table qualified, blocks are not sorted - if (sup.numOfTables == 1) { - for (int32_t i = 0; i < numOfBlocks; ++i) { - SFileDataBlockInfo blockInfo = {.uid = sup.pDataBlockInfo[0][i].uid, .tbBlockIdx = i}; - taosArrayPush(pBlockIter->blockList, &blockInfo); - } - - int64_t et = taosGetTimestampUs(); - tsdbDebug("%p create blocks info struct completed for one table, %d blocks not sorted, elapsed time:%.2f ms %s", - pReader, numOfBlocks, (et - st) / 1000.0, pReader->idStr); - - pBlockIter->index = asc ? 0 : (numOfBlocks - 1); - cleanupBlockOrderSupporter(&sup); - doSetCurrentBlock(pBlockIter, pReader->idStr); - return TSDB_CODE_SUCCESS; - } - - tsdbDebug("%p create data blocks info struct completed, %d blocks in %d tables %s", pReader, cnt, sup.numOfTables, - pReader->idStr); - - SMultiwayMergeTreeInfo* pTree = NULL; - - uint8_t ret = tMergeTreeCreate(&pTree, sup.numOfTables, &sup, fileDataBlockOrderCompar); - if (ret != TSDB_CODE_SUCCESS) { - cleanupBlockOrderSupporter(&sup); - return TSDB_CODE_OUT_OF_MEMORY; - } - - int32_t numOfTotal = 0; - while (numOfTotal < cnt) { - int32_t pos = tMergeTreeGetChosenIndex(pTree); - int32_t index = sup.indexPerTable[pos]++; - - SFileDataBlockInfo blockInfo = {.uid = sup.pDataBlockInfo[pos][index].uid, .tbBlockIdx = index}; - taosArrayPush(pBlockIter->blockList, &blockInfo); - - // set data block index overflow, in order to disable the offset comparator - if (sup.indexPerTable[pos] >= sup.numOfBlocksPerTable[pos]) { - sup.indexPerTable[pos] = sup.numOfBlocksPerTable[pos] + 1; - } - - numOfTotal += 1; - tMergeTreeAdjust(pTree, tMergeTreeGetAdjustIndex(pTree)); - } - - int64_t et = taosGetTimestampUs(); - tsdbDebug("%p %d data blocks access order completed, elapsed time:%.2f ms %s", pReader, numOfBlocks, - (et - st) / 1000.0, pReader->idStr); - cleanupBlockOrderSupporter(&sup); - taosMemoryFree(pTree); - - pBlockIter->index = asc ? 0 : (numOfBlocks - 1); - doSetCurrentBlock(pBlockIter, pReader->idStr); - - return TSDB_CODE_SUCCESS; -} - -static bool blockIteratorNext(SDataBlockIter* pBlockIter, const char* idStr) { - bool asc = ASCENDING_TRAVERSE(pBlockIter->order); - - int32_t step = asc ? 1 : -1; - if ((pBlockIter->index >= pBlockIter->numOfBlocks - 1 && asc) || (pBlockIter->index <= 0 && (!asc))) { - return false; - } - - pBlockIter->index += step; - doSetCurrentBlock(pBlockIter, idStr); - - return true; -} - -/** - * This is an two rectangles overlap cases. - */ -static int32_t dataBlockPartiallyRequired(STimeWindow* pWindow, SVersionRange* pVerRange, SDataBlk* pBlock) { - return (pWindow->ekey < pBlock->maxKey.ts && pWindow->ekey >= pBlock->minKey.ts) || - (pWindow->skey > pBlock->minKey.ts && pWindow->skey <= pBlock->maxKey.ts) || - (pVerRange->minVer > pBlock->minVer && pVerRange->minVer <= pBlock->maxVer) || - (pVerRange->maxVer < pBlock->maxVer && pVerRange->maxVer >= pBlock->minVer); -} - -static bool getNeighborBlockOfSameTable(SFileDataBlockInfo* pBlockInfo, STableBlockScanInfo* pTableBlockScanInfo, - int32_t* nextIndex, int32_t order, SBlockIndex* pBlockIndex) { - bool asc = ASCENDING_TRAVERSE(order); - if (asc && pBlockInfo->tbBlockIdx >= taosArrayGetSize(pTableBlockScanInfo->pBlockList) - 1) { - return false; - } - - if (!asc && pBlockInfo->tbBlockIdx == 0) { - return false; - } - - int32_t step = asc ? 1 : -1; - *nextIndex = pBlockInfo->tbBlockIdx + step; - *pBlockIndex = *(SBlockIndex*)taosArrayGet(pTableBlockScanInfo->pBlockList, *nextIndex); - // tMapDataGetItemByIdx(&pTableBlockScanInfo->mapData, pIndex->ordinalIndex, pBlock, tGetDataBlk); - return true; -} - -static int32_t findFileBlockInfoIndex(SDataBlockIter* pBlockIter, SFileDataBlockInfo* pFBlockInfo) { - int32_t step = ASCENDING_TRAVERSE(pBlockIter->order) ? 1 : -1; - int32_t index = pBlockIter->index; - - while (index < pBlockIter->numOfBlocks && index >= 0) { - SFileDataBlockInfo* pFBlock = taosArrayGet(pBlockIter->blockList, index); - if (pFBlock->uid == pFBlockInfo->uid && pFBlock->tbBlockIdx == pFBlockInfo->tbBlockIdx) { - return index; - } - - index += step; - } - - return -1; -} - -static int32_t setFileBlockActiveInBlockIter(SDataBlockIter* pBlockIter, int32_t index, int32_t step) { - if (index < 0 || index >= pBlockIter->numOfBlocks) { - return -1; - } - - SFileDataBlockInfo fblock = *(SFileDataBlockInfo*)taosArrayGet(pBlockIter->blockList, index); - pBlockIter->index += step; - - if (index != pBlockIter->index) { - taosArrayRemove(pBlockIter->blockList, index); - taosArrayInsert(pBlockIter->blockList, pBlockIter->index, &fblock); - - SFileDataBlockInfo* pBlockInfo = taosArrayGet(pBlockIter->blockList, pBlockIter->index); - ASSERT(pBlockInfo->uid == fblock.uid && pBlockInfo->tbBlockIdx == fblock.tbBlockIdx); - } - - doSetCurrentBlock(pBlockIter, ""); - return TSDB_CODE_SUCCESS; -} - -// todo: this attribute could be acquired during extractin the global ordered block list. -static bool overlapWithNeighborBlock(SDataBlk* pBlock, SBlockIndex* pNeighborBlockIndex, int32_t order) { - // it is the last block in current file, no chance to overlap with neighbor blocks. - if (ASCENDING_TRAVERSE(order)) { - return pBlock->maxKey.ts == pNeighborBlockIndex->window.skey; - } else { - return pBlock->minKey.ts == pNeighborBlockIndex->window.ekey; - } -} - -static bool bufferDataInFileBlockGap(int32_t order, TSDBKEY key, SDataBlk* pBlock) { - bool ascScan = ASCENDING_TRAVERSE(order); - - return (ascScan && (key.ts != TSKEY_INITIAL_VAL && key.ts <= pBlock->minKey.ts)) || - (!ascScan && (key.ts != TSKEY_INITIAL_VAL && key.ts >= pBlock->maxKey.ts)); -} - -static bool keyOverlapFileBlock(TSDBKEY key, SDataBlk* pBlock, SVersionRange* pVerRange) { - return (key.ts >= pBlock->minKey.ts && key.ts <= pBlock->maxKey.ts) && (pBlock->maxVer >= pVerRange->minVer) && - (pBlock->minVer <= pVerRange->maxVer); -} - -static bool doCheckforDatablockOverlap(STableBlockScanInfo* pBlockScanInfo, const SDataBlk* pBlock, - int32_t startIndex) { - size_t num = taosArrayGetSize(pBlockScanInfo->delSkyline); - - for (int32_t i = startIndex; i < num; i += 1) { - TSDBKEY* p = taosArrayGet(pBlockScanInfo->delSkyline, i); - if (p->ts >= pBlock->minKey.ts && p->ts <= pBlock->maxKey.ts) { - if (p->version >= pBlock->minVer) { - return true; - } - } else if (p->ts < pBlock->minKey.ts) { // p->ts < pBlock->minKey.ts - if (p->version >= pBlock->minVer) { - if (i < num - 1) { - TSDBKEY* pnext = taosArrayGet(pBlockScanInfo->delSkyline, i + 1); - if (pnext->ts >= pBlock->minKey.ts) { - return true; - } - } else { // it must be the last point - ASSERT(p->version == 0); - } - } - } else { // (p->ts > pBlock->maxKey.ts) { - return false; - } - } - - return false; -} - -static bool overlapWithDelSkyline(STableBlockScanInfo* pBlockScanInfo, const SDataBlk* pBlock, int32_t order) { - if (pBlockScanInfo->delSkyline == NULL) { - return false; - } - - // ts is not overlap - TSDBKEY* pFirst = taosArrayGet(pBlockScanInfo->delSkyline, 0); - TSDBKEY* pLast = taosArrayGetLast(pBlockScanInfo->delSkyline); - if (pBlock->minKey.ts > pLast->ts || pBlock->maxKey.ts < pFirst->ts) { - return false; - } - - // version is not overlap - if (ASCENDING_TRAVERSE(order)) { - return doCheckforDatablockOverlap(pBlockScanInfo, pBlock, pBlockScanInfo->fileDelIndex); - } else { - int32_t index = pBlockScanInfo->fileDelIndex; - while (1) { - TSDBKEY* p = taosArrayGet(pBlockScanInfo->delSkyline, index); - if (p->ts > pBlock->minKey.ts && index > 0) { - index -= 1; - } else { // find the first point that is smaller than the minKey.ts of dataBlock. - if (p->ts == pBlock->minKey.ts && p->version < pBlock->maxVer && index > 0) { - index -= 1; - } - break; - } - } - - return doCheckforDatablockOverlap(pBlockScanInfo, pBlock, index); - } -} - -typedef struct { - bool overlapWithNeighborBlock; - bool hasDupTs; - bool overlapWithDelInfo; - bool overlapWithLastBlock; - bool overlapWithKeyInBuf; - bool partiallyRequired; - bool moreThanCapcity; -} SDataBlockToLoadInfo; - -static void getBlockToLoadInfo(SDataBlockToLoadInfo* pInfo, SFileDataBlockInfo* pBlockInfo, SDataBlk* pBlock, - STableBlockScanInfo* pScanInfo, TSDBKEY keyInBuf, SLastBlockReader* pLastBlockReader, - STsdbReader* pReader) { - int32_t neighborIndex = 0; - SBlockIndex bIndex = {0}; - - bool hasNeighbor = getNeighborBlockOfSameTable(pBlockInfo, pScanInfo, &neighborIndex, pReader->order, &bIndex); - - // overlap with neighbor - if (hasNeighbor) { - pInfo->overlapWithNeighborBlock = overlapWithNeighborBlock(pBlock, &bIndex, pReader->order); - } - - // has duplicated ts of different version in this block - pInfo->hasDupTs = (pBlock->nSubBlock == 1) ? pBlock->hasDup : true; - pInfo->overlapWithDelInfo = overlapWithDelSkyline(pScanInfo, pBlock, pReader->order); - - if (hasDataInLastBlock(pLastBlockReader)) { - int64_t tsLast = getCurrentKeyInLastBlock(pLastBlockReader); - pInfo->overlapWithLastBlock = !(pBlock->maxKey.ts < tsLast || pBlock->minKey.ts > tsLast); - } - - pInfo->moreThanCapcity = pBlock->nRow > pReader->resBlockInfo.capacity; - pInfo->partiallyRequired = dataBlockPartiallyRequired(&pReader->window, &pReader->verRange, pBlock); - pInfo->overlapWithKeyInBuf = keyOverlapFileBlock(keyInBuf, pBlock, &pReader->verRange); -} - -// 1. the version of all rows should be less than the endVersion -// 2. current block should not overlap with next neighbor block -// 3. current timestamp should not be overlap with each other -// 4. output buffer should be large enough to hold all rows in current block -// 5. delete info should not overlap with current block data -// 6. current block should not contain the duplicated ts -static bool fileBlockShouldLoad(STsdbReader* pReader, SFileDataBlockInfo* pBlockInfo, SDataBlk* pBlock, - STableBlockScanInfo* pScanInfo, TSDBKEY keyInBuf, SLastBlockReader* pLastBlockReader) { - SDataBlockToLoadInfo info = {0}; - getBlockToLoadInfo(&info, pBlockInfo, pBlock, pScanInfo, keyInBuf, pLastBlockReader, pReader); - - bool loadDataBlock = - (info.overlapWithNeighborBlock || info.hasDupTs || info.partiallyRequired || info.overlapWithKeyInBuf || - info.moreThanCapcity || info.overlapWithDelInfo || info.overlapWithLastBlock); - - // log the reason why load the datablock for profile - if (loadDataBlock) { - tsdbDebug("%p uid:%" PRIu64 - " need to load the datablock, overlapneighbor:%d, hasDup:%d, partiallyRequired:%d, " - "overlapWithKey:%d, greaterThanBuf:%d, overlapWithDel:%d, overlapWithlastBlock:%d, %s", - pReader, pBlockInfo->uid, info.overlapWithNeighborBlock, info.hasDupTs, info.partiallyRequired, - info.overlapWithKeyInBuf, info.moreThanCapcity, info.overlapWithDelInfo, info.overlapWithLastBlock, - pReader->idStr); - } - - return loadDataBlock; -} - -static bool isCleanFileDataBlock(STsdbReader* pReader, SFileDataBlockInfo* pBlockInfo, SDataBlk* pBlock, - STableBlockScanInfo* pScanInfo, TSDBKEY keyInBuf, SLastBlockReader* pLastBlockReader) { - SDataBlockToLoadInfo info = {0}; - getBlockToLoadInfo(&info, pBlockInfo, pBlock, pScanInfo, keyInBuf, pLastBlockReader, pReader); - bool isCleanFileBlock = !(info.overlapWithNeighborBlock || info.hasDupTs || info.overlapWithKeyInBuf || - info.overlapWithDelInfo || info.overlapWithLastBlock); - return isCleanFileBlock; -} - -static int32_t buildDataBlockFromBuf(STsdbReader* pReader, STableBlockScanInfo* pBlockScanInfo, int64_t endKey) { - if (!(pBlockScanInfo->iiter.hasVal || pBlockScanInfo->iter.hasVal)) { - return TSDB_CODE_SUCCESS; - } - - SSDataBlock* pBlock = pReader->resBlockInfo.pResBlock; - - int64_t st = taosGetTimestampUs(); - int32_t code = buildDataBlockFromBufImpl(pBlockScanInfo, endKey, pReader->resBlockInfo.capacity, pReader); - - blockDataUpdateTsWindow(pBlock, pReader->suppInfo.slotId[0]); - pBlock->info.id.uid = pBlockScanInfo->uid; - - setComposedBlockFlag(pReader, true); - - double elapsedTime = (taosGetTimestampUs() - st) / 1000.0; - tsdbDebug("%p build data block from cache completed, elapsed time:%.2f ms, numOfRows:%" PRId64 ", brange:%" PRId64 - " - %" PRId64 ", uid:%" PRIu64 ", %s", - pReader, elapsedTime, pBlock->info.rows, pBlock->info.window.skey, pBlock->info.window.ekey, - pBlockScanInfo->uid, pReader->idStr); - - pReader->cost.buildmemBlock += elapsedTime; - return code; -} - -static bool tryCopyDistinctRowFromFileBlock(STsdbReader* pReader, SBlockData* pBlockData, int64_t key, - SFileBlockDumpInfo* pDumpInfo, bool* copied) { - // opt version - // 1. it is not a border point - // 2. the direct next point is not an duplicated timestamp - int32_t code = TSDB_CODE_SUCCESS; - - *copied = false; - bool asc = (pReader->order == TSDB_ORDER_ASC); - if ((pDumpInfo->rowIndex < pDumpInfo->totalRows - 1 && asc) || (pDumpInfo->rowIndex > 0 && (!asc))) { - int32_t step = pReader->order == TSDB_ORDER_ASC ? 1 : -1; - - int64_t nextKey = pBlockData->aTSKEY[pDumpInfo->rowIndex + step]; - if (nextKey != key) { // merge is not needed - code = doAppendRowFromFileBlock(pReader->resBlockInfo.pResBlock, pReader, pBlockData, pDumpInfo->rowIndex); - if (code) { - return code; - } - pDumpInfo->rowIndex += step; - *copied = true; - } - } - - return code; -} - -static bool nextRowFromLastBlocks(SLastBlockReader* pLastBlockReader, STableBlockScanInfo* pScanInfo, - SVersionRange* pVerRange) { - int32_t step = ASCENDING_TRAVERSE(pLastBlockReader->order) ? 1 : -1; - - while (1) { - bool hasVal = tMergeTreeNext(&pLastBlockReader->mergeTree); - if (!hasVal) { // the next value will be the accessed key in stt - pScanInfo->lastKeyInStt += step; - return false; - } - - TSDBROW* pRow = tMergeTreeGetRow(&pLastBlockReader->mergeTree); - int64_t key = pRow->pBlockData->aTSKEY[pRow->iRow]; - int64_t ver = pRow->pBlockData->aVersion[pRow->iRow]; - - pLastBlockReader->currentKey = key; - pScanInfo->lastKeyInStt = key; - - if (!hasBeenDropped(pScanInfo->delSkyline, &pScanInfo->sttBlockDelIndex, key, ver, pLastBlockReader->order, - pVerRange)) { - return true; - } - } -} - -static bool tryCopyDistinctRowFromSttBlock(TSDBROW* fRow, SLastBlockReader* pLastBlockReader, - STableBlockScanInfo* pScanInfo, int64_t ts, STsdbReader* pReader, - bool* copied) { - int32_t code = TSDB_CODE_SUCCESS; - - *copied = false; - - bool hasVal = nextRowFromLastBlocks(pLastBlockReader, pScanInfo, &pReader->verRange); - if (hasVal) { - int64_t next1 = getCurrentKeyInLastBlock(pLastBlockReader); - if (next1 != ts) { - code = doAppendRowFromFileBlock(pReader->resBlockInfo.pResBlock, pReader, fRow->pBlockData, fRow->iRow); - if (code) { - return code; - } - - *copied = true; - return code; - } - } else { - code = doAppendRowFromFileBlock(pReader->resBlockInfo.pResBlock, pReader, fRow->pBlockData, fRow->iRow); - if (code) { - return code; - } - - *copied = true; - return code; - } - - return code; -} - -static FORCE_INLINE STSchema* doGetSchemaForTSRow(int32_t sversion, STsdbReader* pReader, uint64_t uid) { - // always set the newest schema version in pReader->pSchema - if (pReader->pSchema == NULL) { - STSchema* ps = getTableSchemaImpl(pReader, uid); - if (ps == NULL) { - return NULL; - } - } - - if (pReader->pSchema && sversion == pReader->pSchema->version) { - return pReader->pSchema; - } - - void** p = tSimpleHashGet(pReader->pSchemaMap, &sversion, sizeof(sversion)); - if (p != NULL) { - return *(STSchema**)p; - } - - STSchema* ptr = NULL; - int32_t code = metaGetTbTSchemaEx(pReader->pTsdb->pVnode->pMeta, pReader->suid, uid, sversion, &ptr); - if (code != TSDB_CODE_SUCCESS) { - terrno = code; - return NULL; - } else { - code = tSimpleHashPut(pReader->pSchemaMap, &sversion, sizeof(sversion), &ptr, POINTER_BYTES); - if (code != TSDB_CODE_SUCCESS) { - terrno = code; - return NULL; - } - return ptr; - } -} - -static int32_t doMergeBufAndFileRows(STsdbReader* pReader, STableBlockScanInfo* pBlockScanInfo, TSDBROW* pRow, - SIterInfo* pIter, int64_t key, SLastBlockReader* pLastBlockReader) { - SRowMerger* pMerger = &pReader->status.merger; - SRow* pTSRow = NULL; - SBlockData* pBlockData = &pReader->status.fileBlockData; - SFileBlockDumpInfo* pDumpInfo = &pReader->status.fBlockDumpInfo; - - int64_t tsLast = INT64_MIN; - if (hasDataInLastBlock(pLastBlockReader)) { - tsLast = getCurrentKeyInLastBlock(pLastBlockReader); - } - - TSDBKEY k = TSDBROW_KEY(pRow); - TSDBROW fRow = tsdbRowFromBlockData(pBlockData, pDumpInfo->rowIndex); - - // merge is not initialized yet, due to the fact that the pReader->pSchema is not initialized - if (pMerger->pArray == NULL) { - ASSERT(pReader->pSchema == NULL); - STSchema* ps = getTableSchemaImpl(pReader, pBlockScanInfo->uid); - if (ps == NULL) { - return terrno; - } - } - - int64_t minKey = 0; - if (pReader->order == TSDB_ORDER_ASC) { - minKey = INT64_MAX; // chosen the minimum value - if (minKey > tsLast && hasDataInLastBlock(pLastBlockReader)) { - minKey = tsLast; - } - - if (minKey > k.ts) { - minKey = k.ts; - } - - if (minKey > key && hasDataInFileBlock(pBlockData, pDumpInfo)) { - minKey = key; - } - } else { - minKey = INT64_MIN; - if (minKey < tsLast && hasDataInLastBlock(pLastBlockReader)) { - minKey = tsLast; - } - - if (minKey < k.ts) { - minKey = k.ts; - } - - if (minKey < key && hasDataInFileBlock(pBlockData, pDumpInfo)) { - minKey = key; - } - } - - // todo remove init - bool init = false; - - // ASC: file block ---> last block -----> imem -----> mem - // DESC: mem -----> imem -----> last block -----> file block - if (pReader->order == TSDB_ORDER_ASC) { - if (minKey == key) { - init = true; - int32_t code = tsdbRowMergerAdd(pMerger, &fRow, pReader->pSchema); - if (code != TSDB_CODE_SUCCESS) { - return code; - } - doMergeRowsInFileBlocks(pBlockData, pBlockScanInfo, pReader); - } - - if (minKey == tsLast) { - TSDBROW* fRow1 = tMergeTreeGetRow(&pLastBlockReader->mergeTree); - if (init) { - tsdbRowMergerAdd(pMerger, fRow1, NULL); - } else { - init = true; - int32_t code = tsdbRowMergerAdd(pMerger, fRow1, pReader->pSchema); - if (code != TSDB_CODE_SUCCESS) { - return code; - } - } - doMergeRowsInLastBlock(pLastBlockReader, pBlockScanInfo, tsLast, pMerger, &pReader->verRange, pReader->idStr); - } - - if (minKey == k.ts) { - STSchema* pSchema = doGetSchemaForTSRow(TSDBROW_SVERSION(pRow), pReader, pBlockScanInfo->uid); - if (pSchema == NULL) { - return terrno; - } - if (init) { - tsdbRowMergerAdd(pMerger, pRow, pSchema); - } else { - init = true; - int32_t code = tsdbRowMergerAdd(pMerger, pRow, pSchema); - if (code != TSDB_CODE_SUCCESS) { - return code; - } - } - int32_t code = doMergeRowsInBuf(pIter, pBlockScanInfo->uid, k.ts, pBlockScanInfo->delSkyline, pReader); - if (code != TSDB_CODE_SUCCESS) { - return code; - } - } - } else { - if (minKey == k.ts) { - init = true; - STSchema* pSchema = doGetSchemaForTSRow(TSDBROW_SVERSION(pRow), pReader, pBlockScanInfo->uid); - if (pSchema == NULL) { - return terrno; - } - - int32_t code = tsdbRowMergerAdd(pMerger, pRow, pSchema); - if (code != TSDB_CODE_SUCCESS) { - return code; - } - - code = doMergeRowsInBuf(pIter, pBlockScanInfo->uid, k.ts, pBlockScanInfo->delSkyline, pReader); - if (code != TSDB_CODE_SUCCESS || pMerger->pTSchema == NULL) { - return code; - } - } - - if (minKey == tsLast) { - TSDBROW* fRow1 = tMergeTreeGetRow(&pLastBlockReader->mergeTree); - if (init) { - tsdbRowMergerAdd(pMerger, fRow1, NULL); - } else { - init = true; - int32_t code = tsdbRowMergerAdd(pMerger, fRow1, pReader->pSchema); - if (code != TSDB_CODE_SUCCESS) { - return code; - } - } - doMergeRowsInLastBlock(pLastBlockReader, pBlockScanInfo, tsLast, pMerger, &pReader->verRange, pReader->idStr); - } - - if (minKey == key) { - if (init) { - tsdbRowMergerAdd(pMerger, &fRow, NULL); - } else { - init = true; - int32_t code = tsdbRowMergerAdd(pMerger, &fRow, pReader->pSchema); - if (code != TSDB_CODE_SUCCESS) { - return code; - } - } - doMergeRowsInFileBlocks(pBlockData, pBlockScanInfo, pReader); - } - } - - int32_t code = tsdbRowMergerGetRow(pMerger, &pTSRow); - if (code != TSDB_CODE_SUCCESS) { - return code; - } - - code = doAppendRowFromTSRow(pReader->resBlockInfo.pResBlock, pReader, pTSRow, pBlockScanInfo); - - taosMemoryFree(pTSRow); - tsdbRowMergerClear(pMerger); - - return code; -} - -static int32_t doMergeFileBlockAndLastBlock(SLastBlockReader* pLastBlockReader, STsdbReader* pReader, - STableBlockScanInfo* pBlockScanInfo, SBlockData* pBlockData, - bool mergeBlockData) { - SRowMerger* pMerger = &pReader->status.merger; - SFileBlockDumpInfo* pDumpInfo = &pReader->status.fBlockDumpInfo; - - int64_t tsLastBlock = getCurrentKeyInLastBlock(pLastBlockReader); - bool copied = false; - int32_t code = TSDB_CODE_SUCCESS; - SRow* pTSRow = NULL; - TSDBROW* pRow = tMergeTreeGetRow(&pLastBlockReader->mergeTree); - - // create local variable to hold the row value - TSDBROW fRow = {.iRow = pRow->iRow, .type = TSDBROW_COL_FMT, .pBlockData = pRow->pBlockData}; - - tsdbTrace("fRow ptr:%p, %d, uid:%" PRIu64 ", %s", pRow->pBlockData, pRow->iRow, pLastBlockReader->uid, - pReader->idStr); - - // only last block exists - if ((!mergeBlockData) || (tsLastBlock != pBlockData->aTSKEY[pDumpInfo->rowIndex])) { - code = tryCopyDistinctRowFromSttBlock(&fRow, pLastBlockReader, pBlockScanInfo, tsLastBlock, pReader, &copied); - if (code) { - return code; - } - - if (copied) { - pBlockScanInfo->lastKey = tsLastBlock; - return TSDB_CODE_SUCCESS; - } else { - code = tsdbRowMergerAdd(pMerger, &fRow, pReader->pSchema); - if (code != TSDB_CODE_SUCCESS) { - return code; - } - - TSDBROW* pRow1 = tMergeTreeGetRow(&pLastBlockReader->mergeTree); - tsdbRowMergerAdd(pMerger, pRow1, NULL); - doMergeRowsInLastBlock(pLastBlockReader, pBlockScanInfo, tsLastBlock, pMerger, &pReader->verRange, - pReader->idStr); - - code = tsdbRowMergerGetRow(pMerger, &pTSRow); - if (code != TSDB_CODE_SUCCESS) { - return code; - } - - code = doAppendRowFromTSRow(pReader->resBlockInfo.pResBlock, pReader, pTSRow, pBlockScanInfo); - - taosMemoryFree(pTSRow); - tsdbRowMergerClear(pMerger); - - if (code != TSDB_CODE_SUCCESS) { - return code; - } - } - } else { // not merge block data - code = tsdbRowMergerAdd(pMerger, &fRow, pReader->pSchema); - if (code != TSDB_CODE_SUCCESS) { - return code; - } - - doMergeRowsInLastBlock(pLastBlockReader, pBlockScanInfo, tsLastBlock, pMerger, &pReader->verRange, pReader->idStr); - - // merge with block data if ts == key - if (tsLastBlock == pBlockData->aTSKEY[pDumpInfo->rowIndex]) { - doMergeRowsInFileBlocks(pBlockData, pBlockScanInfo, pReader); - } - - code = tsdbRowMergerGetRow(pMerger, &pTSRow); - if (code != TSDB_CODE_SUCCESS) { - return code; - } - - code = doAppendRowFromTSRow(pReader->resBlockInfo.pResBlock, pReader, pTSRow, pBlockScanInfo); - - taosMemoryFree(pTSRow); - tsdbRowMergerClear(pMerger); - - if (code != TSDB_CODE_SUCCESS) { - return code; - } - } - - return TSDB_CODE_SUCCESS; -} - -static int32_t mergeFileBlockAndLastBlock(STsdbReader* pReader, SLastBlockReader* pLastBlockReader, int64_t key, - STableBlockScanInfo* pBlockScanInfo, SBlockData* pBlockData) { - SFileBlockDumpInfo* pDumpInfo = &pReader->status.fBlockDumpInfo; - SRowMerger* pMerger = &pReader->status.merger; - - // merge is not initialized yet, due to the fact that the pReader->pSchema is not initialized - if (pMerger->pArray == NULL) { - ASSERT(pReader->pSchema == NULL); - STSchema* ps = getTableSchemaImpl(pReader, pBlockScanInfo->uid); - if (ps == NULL) { - return terrno; - } - } - - if (hasDataInFileBlock(pBlockData, pDumpInfo)) { - // no last block available, only data block exists - if (!hasDataInLastBlock(pLastBlockReader)) { - return mergeRowsInFileBlocks(pBlockData, pBlockScanInfo, key, pReader); - } - - // row in last file block - TSDBROW fRow = tsdbRowFromBlockData(pBlockData, pDumpInfo->rowIndex); - int64_t ts = getCurrentKeyInLastBlock(pLastBlockReader); - ASSERT(ts >= key); - - if (ASCENDING_TRAVERSE(pReader->order)) { - if (key < ts) { // imem, mem are all empty, file blocks (data blocks and last block) exist - return mergeRowsInFileBlocks(pBlockData, pBlockScanInfo, key, pReader); - } else if (key == ts) { - SRow* pTSRow = NULL; - int32_t code = tsdbRowMergerAdd(pMerger, &fRow, pReader->pSchema); - if (code != TSDB_CODE_SUCCESS) { - return code; - } - - doMergeRowsInFileBlocks(pBlockData, pBlockScanInfo, pReader); - - TSDBROW* pRow1 = tMergeTreeGetRow(&pLastBlockReader->mergeTree); - tsdbRowMergerAdd(pMerger, pRow1, NULL); - - doMergeRowsInLastBlock(pLastBlockReader, pBlockScanInfo, ts, pMerger, &pReader->verRange, pReader->idStr); - - code = tsdbRowMergerGetRow(pMerger, &pTSRow); - if (code != TSDB_CODE_SUCCESS) { - return code; - } - - code = doAppendRowFromTSRow(pReader->resBlockInfo.pResBlock, pReader, pTSRow, pBlockScanInfo); - - taosMemoryFree(pTSRow); - tsdbRowMergerClear(pMerger); - return code; - } else { - return TSDB_CODE_SUCCESS; - } - } else { // desc order - return doMergeFileBlockAndLastBlock(pLastBlockReader, pReader, pBlockScanInfo, pBlockData, true); - } - } else { // only last block exists - return doMergeFileBlockAndLastBlock(pLastBlockReader, pReader, pBlockScanInfo, NULL, false); - } -} - -static int32_t doMergeMultiLevelRows(STsdbReader* pReader, STableBlockScanInfo* pBlockScanInfo, SBlockData* pBlockData, - SLastBlockReader* pLastBlockReader) { - SRowMerger* pMerger = &pReader->status.merger; - SRow* pTSRow = NULL; - int32_t code = TSDB_CODE_SUCCESS; - SFileBlockDumpInfo* pDumpInfo = &pReader->status.fBlockDumpInfo; - SArray* pDelList = pBlockScanInfo->delSkyline; - - TSDBROW* pRow = getValidMemRow(&pBlockScanInfo->iter, pDelList, pReader); - TSDBROW* piRow = getValidMemRow(&pBlockScanInfo->iiter, pDelList, pReader); - - int64_t tsLast = INT64_MIN; - if (hasDataInLastBlock(pLastBlockReader)) { - tsLast = getCurrentKeyInLastBlock(pLastBlockReader); - } - - int64_t key = hasDataInFileBlock(pBlockData, pDumpInfo) ? pBlockData->aTSKEY[pDumpInfo->rowIndex] : INT64_MIN; - - TSDBKEY k = TSDBROW_KEY(pRow); - TSDBKEY ik = TSDBROW_KEY(piRow); - STSchema* pSchema = doGetSchemaForTSRow(TSDBROW_SVERSION(pRow), pReader, pBlockScanInfo->uid); - if (pSchema == NULL) { - return code; - } - - STSchema* piSchema = doGetSchemaForTSRow(TSDBROW_SVERSION(piRow), pReader, pBlockScanInfo->uid); - if (piSchema == NULL) { - return code; - } - - // merge is not initialized yet, due to the fact that the pReader->pSchema is not initialized - if (pMerger->pArray == NULL) { - ASSERT(pReader->pSchema == NULL); - STSchema* ps = getTableSchemaImpl(pReader, pBlockScanInfo->uid); - if (ps == NULL) { - return terrno; - } - } - - int64_t minKey = 0; - if (ASCENDING_TRAVERSE(pReader->order)) { - minKey = INT64_MAX; // let's find the minimum - if (minKey > k.ts) { - minKey = k.ts; - } - - if (minKey > ik.ts) { - minKey = ik.ts; - } - - if (minKey > key && hasDataInFileBlock(pBlockData, pDumpInfo)) { - minKey = key; - } - - if (minKey > tsLast && hasDataInLastBlock(pLastBlockReader)) { - minKey = tsLast; - } - } else { - minKey = INT64_MIN; // let find the maximum ts value - if (minKey < k.ts) { - minKey = k.ts; - } - - if (minKey < ik.ts) { - minKey = ik.ts; - } - - if (minKey < key && hasDataInFileBlock(pBlockData, pDumpInfo)) { - minKey = key; - } - - if (minKey < tsLast && hasDataInLastBlock(pLastBlockReader)) { - minKey = tsLast; - } - } - - bool init = false; - - // ASC: file block -----> last block -----> imem -----> mem - // DESC: mem -----> imem -----> last block -----> file block - if (ASCENDING_TRAVERSE(pReader->order)) { - if (minKey == key) { - init = true; - TSDBROW fRow = tsdbRowFromBlockData(pBlockData, pDumpInfo->rowIndex); - code = tsdbRowMergerAdd(pMerger, &fRow, pReader->pSchema); - if (code != TSDB_CODE_SUCCESS) { - return code; - } - - doMergeRowsInFileBlocks(pBlockData, pBlockScanInfo, pReader); - } - - if (minKey == tsLast) { - TSDBROW* pRow1 = tMergeTreeGetRow(&pLastBlockReader->mergeTree); - if (init) { - tsdbRowMergerAdd(pMerger, pRow1, NULL); - } else { - init = true; - code = tsdbRowMergerAdd(pMerger, pRow1, pReader->pSchema); - if (code != TSDB_CODE_SUCCESS) { - return code; - } - } - - doMergeRowsInLastBlock(pLastBlockReader, pBlockScanInfo, tsLast, pMerger, &pReader->verRange, pReader->idStr); - } - - if (minKey == ik.ts) { - if (init) { - tsdbRowMergerAdd(pMerger, piRow, piSchema); - } else { - init = true; - code = tsdbRowMergerAdd(pMerger, piRow, piSchema); - if (code != TSDB_CODE_SUCCESS) { - return code; - } - } - - code = doMergeRowsInBuf(&pBlockScanInfo->iiter, pBlockScanInfo->uid, ik.ts, pBlockScanInfo->delSkyline, pReader); - if (code != TSDB_CODE_SUCCESS) { - return code; - } - } - - if (minKey == k.ts) { - if (init) { - tsdbRowMergerAdd(pMerger, pRow, pSchema); - } else { - // STSchema* pSchema = doGetSchemaForTSRow(TSDBROW_SVERSION(pRow), pReader, pBlockScanInfo->uid); - code = tsdbRowMergerAdd(pMerger, pRow, pSchema); - if (code != TSDB_CODE_SUCCESS) { - return code; - } - } - code = doMergeRowsInBuf(&pBlockScanInfo->iter, pBlockScanInfo->uid, k.ts, pBlockScanInfo->delSkyline, pReader); - if (code != TSDB_CODE_SUCCESS) { - return code; - } - } - } else { - if (minKey == k.ts) { - init = true; - code = tsdbRowMergerAdd(pMerger, pRow, pSchema); - if (code != TSDB_CODE_SUCCESS) { - return code; - } - - code = doMergeRowsInBuf(&pBlockScanInfo->iter, pBlockScanInfo->uid, k.ts, pBlockScanInfo->delSkyline, pReader); - if (code != TSDB_CODE_SUCCESS) { - return code; - } - } - - if (minKey == ik.ts) { - if (init) { - tsdbRowMergerAdd(pMerger, piRow, piSchema); - } else { - init = true; - code = tsdbRowMergerAdd(pMerger, piRow, piSchema); - if (code != TSDB_CODE_SUCCESS) { - return code; - } - } - code = doMergeRowsInBuf(&pBlockScanInfo->iiter, pBlockScanInfo->uid, ik.ts, pBlockScanInfo->delSkyline, pReader); - if (code != TSDB_CODE_SUCCESS) { - return code; - } - } - - if (minKey == tsLast) { - TSDBROW* pRow1 = tMergeTreeGetRow(&pLastBlockReader->mergeTree); - if (init) { - tsdbRowMergerAdd(pMerger, pRow1, NULL); - } else { - init = true; - code = tsdbRowMergerAdd(pMerger, pRow1, pReader->pSchema); - if (code != TSDB_CODE_SUCCESS) { - return code; - } - } - doMergeRowsInLastBlock(pLastBlockReader, pBlockScanInfo, tsLast, pMerger, &pReader->verRange, pReader->idStr); - } - - if (minKey == key) { - TSDBROW fRow = tsdbRowFromBlockData(pBlockData, pDumpInfo->rowIndex); - if (!init) { - code = tsdbRowMergerAdd(pMerger, &fRow, pReader->pSchema); - if (code != TSDB_CODE_SUCCESS) { - return code; - } - } else { - tsdbRowMergerAdd(pMerger, &fRow, NULL); - } - doMergeRowsInFileBlocks(pBlockData, pBlockScanInfo, pReader); - } - } - - code = tsdbRowMergerGetRow(pMerger, &pTSRow); - if (code != TSDB_CODE_SUCCESS) { - return code; - } - - code = doAppendRowFromTSRow(pReader->resBlockInfo.pResBlock, pReader, pTSRow, pBlockScanInfo); - - taosMemoryFree(pTSRow); - tsdbRowMergerClear(pMerger); - return code; -} - -static int32_t initMemDataIterator(STableBlockScanInfo* pBlockScanInfo, STsdbReader* pReader) { - if (pBlockScanInfo->iterInit) { - return TSDB_CODE_SUCCESS; - } - - int32_t code = TSDB_CODE_SUCCESS; - - TSDBKEY startKey = {0}; - if (ASCENDING_TRAVERSE(pReader->order)) { - // startKey = (TSDBKEY){.ts = pReader->window.skey, .version = pReader->verRange.minVer}; - startKey = (TSDBKEY){.ts = pBlockScanInfo->lastKey + 1, .version = pReader->verRange.minVer}; - } else { - // startKey = (TSDBKEY){.ts = pReader->window.ekey, .version = pReader->verRange.maxVer}; - startKey = (TSDBKEY){.ts = pBlockScanInfo->lastKey - 1, .version = pReader->verRange.maxVer}; - } - - int32_t backward = (!ASCENDING_TRAVERSE(pReader->order)); - int64_t st = 0; - - STbData* d = NULL; - if (pReader->pReadSnap->pMem != NULL) { - d = tsdbGetTbDataFromMemTable(pReader->pReadSnap->pMem, pReader->suid, pBlockScanInfo->uid); - if (d != NULL) { - code = tsdbTbDataIterCreate(d, &startKey, backward, &pBlockScanInfo->iter.iter); - if (code == TSDB_CODE_SUCCESS) { - pBlockScanInfo->iter.hasVal = (tsdbTbDataIterGet(pBlockScanInfo->iter.iter) != NULL); - - tsdbDebug("%p uid:%" PRIu64 ", check data in mem from skey:%" PRId64 ", order:%d, ts range in buf:%" PRId64 - "-%" PRId64 " %s", - pReader, pBlockScanInfo->uid, startKey.ts, pReader->order, d->minKey, d->maxKey, pReader->idStr); - } else { - tsdbError("%p uid:%" PRIu64 ", failed to create iterator for imem, code:%s, %s", pReader, pBlockScanInfo->uid, - tstrerror(code), pReader->idStr); - return code; - } - } - } else { - tsdbDebug("%p uid:%" PRIu64 ", no data in mem, %s", pReader, pBlockScanInfo->uid, pReader->idStr); - } - - STbData* di = NULL; - if (pReader->pReadSnap->pIMem != NULL) { - di = tsdbGetTbDataFromMemTable(pReader->pReadSnap->pIMem, pReader->suid, pBlockScanInfo->uid); - if (di != NULL) { - code = tsdbTbDataIterCreate(di, &startKey, backward, &pBlockScanInfo->iiter.iter); - if (code == TSDB_CODE_SUCCESS) { - pBlockScanInfo->iiter.hasVal = (tsdbTbDataIterGet(pBlockScanInfo->iiter.iter) != NULL); - - tsdbDebug("%p uid:%" PRIu64 ", check data in imem from skey:%" PRId64 ", order:%d, ts range in buf:%" PRId64 - "-%" PRId64 " %s", - pReader, pBlockScanInfo->uid, startKey.ts, pReader->order, di->minKey, di->maxKey, pReader->idStr); - } else { - tsdbError("%p uid:%" PRIu64 ", failed to create iterator for mem, code:%s, %s", pReader, pBlockScanInfo->uid, - tstrerror(code), pReader->idStr); - return code; - } - } - } else { - tsdbDebug("%p uid:%" PRIu64 ", no data in imem, %s", pReader, pBlockScanInfo->uid, pReader->idStr); - } - - st = taosGetTimestampUs(); - initDelSkylineIterator(pBlockScanInfo, pReader, d, di); - pReader->cost.initDelSkylineIterTime += (taosGetTimestampUs() - st) / 1000.0; - - pBlockScanInfo->iterInit = true; - return TSDB_CODE_SUCCESS; -} - -static bool isValidFileBlockRow(SBlockData* pBlockData, SFileBlockDumpInfo* pDumpInfo, - STableBlockScanInfo* pBlockScanInfo, STsdbReader* pReader) { - // it is an multi-table data block - if (pBlockData->aUid != NULL) { - uint64_t uid = pBlockData->aUid[pDumpInfo->rowIndex]; - if (uid != pBlockScanInfo->uid) { // move to next row - return false; - } - } - - // check for version and time range - int64_t ver = pBlockData->aVersion[pDumpInfo->rowIndex]; - if (ver > pReader->verRange.maxVer || ver < pReader->verRange.minVer) { - return false; - } - - int64_t ts = pBlockData->aTSKEY[pDumpInfo->rowIndex]; - if (ts > pReader->window.ekey || ts < pReader->window.skey) { - return false; - } - - if (hasBeenDropped(pBlockScanInfo->delSkyline, &pBlockScanInfo->fileDelIndex, ts, ver, pReader->order, - &pReader->verRange)) { - return false; - } - - return true; -} - -static bool initLastBlockReader(SLastBlockReader* pLBlockReader, STableBlockScanInfo* pScanInfo, STsdbReader* pReader) { - // the last block reader has been initialized for this table. - if (pLBlockReader->uid == pScanInfo->uid) { - return hasDataInLastBlock(pLBlockReader); - } - - if (pLBlockReader->uid != 0) { - tMergeTreeClose(&pLBlockReader->mergeTree); - } - - initMemDataIterator(pScanInfo, pReader); - pLBlockReader->uid = pScanInfo->uid; - - STimeWindow w = pLBlockReader->window; - if (ASCENDING_TRAVERSE(pLBlockReader->order)) { - w.skey = pScanInfo->lastKeyInStt; - } else { - w.ekey = pScanInfo->lastKeyInStt; - } - - tsdbDebug("init last block reader, window:%" PRId64 "-%" PRId64 ", uid:%" PRIu64 ", %s", w.skey, w.ekey, - pScanInfo->uid, pReader->idStr); - int32_t code = tMergeTreeOpen(&pLBlockReader->mergeTree, (pLBlockReader->order == TSDB_ORDER_DESC), - pReader->pFileReader, pReader->suid, pScanInfo->uid, &w, &pLBlockReader->verRange, - pLBlockReader->pInfo, false, pReader->idStr, false, pReader->status.pLDataIter); - if (code != TSDB_CODE_SUCCESS) { - return false; - } - - return nextRowFromLastBlocks(pLBlockReader, pScanInfo, &pReader->verRange); -} - -static bool hasDataInLastBlock(SLastBlockReader* pLastBlockReader) { return pLastBlockReader->mergeTree.pIter != NULL; } - -bool hasDataInFileBlock(const SBlockData* pBlockData, const SFileBlockDumpInfo* pDumpInfo) { - if ((pBlockData->nRow > 0) && (pBlockData->nRow != pDumpInfo->totalRows)) { - return false; // this is an invalid result. - } - return pBlockData->nRow > 0 && (!pDumpInfo->allDumped); -} - -int32_t mergeRowsInFileBlocks(SBlockData* pBlockData, STableBlockScanInfo* pBlockScanInfo, int64_t key, - STsdbReader* pReader) { - SRowMerger* pMerger = &pReader->status.merger; - SFileBlockDumpInfo* pDumpInfo = &pReader->status.fBlockDumpInfo; - bool copied = false; - int32_t code = tryCopyDistinctRowFromFileBlock(pReader, pBlockData, key, pDumpInfo, &copied); - if (code) { - return code; - } - - // merge is not initialized yet, due to the fact that the pReader->pSchema is not initialized - if (pMerger->pArray == NULL) { - ASSERT(pReader->pSchema == NULL); - STSchema* ps = getTableSchemaImpl(pReader, pBlockScanInfo->uid); - if (ps == NULL) { - return terrno; - } - } - - if (copied) { - pBlockScanInfo->lastKey = key; - return TSDB_CODE_SUCCESS; - } else { - TSDBROW fRow = tsdbRowFromBlockData(pBlockData, pDumpInfo->rowIndex); - - SRow* pTSRow = NULL; - code = tsdbRowMergerAdd(pMerger, &fRow, pReader->pSchema); - if (code != TSDB_CODE_SUCCESS) { - return code; - } - - doMergeRowsInFileBlocks(pBlockData, pBlockScanInfo, pReader); - code = tsdbRowMergerGetRow(pMerger, &pTSRow); - if (code != TSDB_CODE_SUCCESS) { - return code; - } - - code = doAppendRowFromTSRow(pReader->resBlockInfo.pResBlock, pReader, pTSRow, pBlockScanInfo); - - taosMemoryFree(pTSRow); - tsdbRowMergerClear(pMerger); - return code; - } -} - -static int32_t buildComposedDataBlockImpl(STsdbReader* pReader, STableBlockScanInfo* pBlockScanInfo, - SBlockData* pBlockData, SLastBlockReader* pLastBlockReader) { - SFileBlockDumpInfo* pDumpInfo = &pReader->status.fBlockDumpInfo; - - TSDBROW *pRow = NULL, *piRow = NULL; - int64_t key = (pBlockData->nRow > 0 && (!pDumpInfo->allDumped)) ? pBlockData->aTSKEY[pDumpInfo->rowIndex] : INT64_MIN; - if (pBlockScanInfo->iter.hasVal) { - pRow = getValidMemRow(&pBlockScanInfo->iter, pBlockScanInfo->delSkyline, pReader); - } - - if (pBlockScanInfo->iiter.hasVal) { - piRow = getValidMemRow(&pBlockScanInfo->iiter, pBlockScanInfo->delSkyline, pReader); - } - - // two levels of mem-table does contain the valid rows - if (pRow != NULL && piRow != NULL) { - return doMergeMultiLevelRows(pReader, pBlockScanInfo, pBlockData, pLastBlockReader); - } - - // imem + file + last block - if (pBlockScanInfo->iiter.hasVal) { - return doMergeBufAndFileRows(pReader, pBlockScanInfo, piRow, &pBlockScanInfo->iiter, key, pLastBlockReader); - } - - // mem + file + last block - if (pBlockScanInfo->iter.hasVal) { - return doMergeBufAndFileRows(pReader, pBlockScanInfo, pRow, &pBlockScanInfo->iter, key, pLastBlockReader); - } - - // files data blocks + last block - return mergeFileBlockAndLastBlock(pReader, pLastBlockReader, key, pBlockScanInfo, pBlockData); -} - -static int32_t loadNeighborIfOverlap(SFileDataBlockInfo* pBlockInfo, STableBlockScanInfo* pBlockScanInfo, - STsdbReader* pReader, bool* loadNeighbor) { - int32_t code = TSDB_CODE_SUCCESS; - int32_t step = ASCENDING_TRAVERSE(pReader->order) ? 1 : -1; - int32_t nextIndex = -1; - SBlockIndex nxtBIndex = {0}; - - *loadNeighbor = false; - SDataBlk* pBlock = getCurrentBlock(&pReader->status.blockIter); - - bool hasNeighbor = getNeighborBlockOfSameTable(pBlockInfo, pBlockScanInfo, &nextIndex, pReader->order, &nxtBIndex); - if (!hasNeighbor) { // do nothing - return code; - } - - if (overlapWithNeighborBlock(pBlock, &nxtBIndex, pReader->order)) { // load next block - SReaderStatus* pStatus = &pReader->status; - SDataBlockIter* pBlockIter = &pStatus->blockIter; - - // 1. find the next neighbor block in the scan block list - SFileDataBlockInfo fb = {.uid = pBlockInfo->uid, .tbBlockIdx = nextIndex}; - int32_t neighborIndex = findFileBlockInfoIndex(pBlockIter, &fb); - - // 2. remove it from the scan block list - setFileBlockActiveInBlockIter(pBlockIter, neighborIndex, step); - - // 3. load the neighbor block, and set it to be the currently accessed file data block - code = doLoadFileBlockData(pReader, pBlockIter, &pStatus->fileBlockData, pBlockInfo->uid); - if (code != TSDB_CODE_SUCCESS) { - return code; - } - - // 4. check the data values - initBlockDumpInfo(pReader, pBlockIter); - *loadNeighbor = true; - } - - return code; -} - -static void updateComposedBlockInfo(STsdbReader* pReader, double el, STableBlockScanInfo* pBlockScanInfo) { - SSDataBlock* pResBlock = pReader->resBlockInfo.pResBlock; - - pResBlock->info.id.uid = (pBlockScanInfo != NULL) ? pBlockScanInfo->uid : 0; - pResBlock->info.dataLoad = 1; - blockDataUpdateTsWindow(pResBlock, pReader->suppInfo.slotId[0]); - - setComposedBlockFlag(pReader, true); - - pReader->cost.composedBlocks += 1; - pReader->cost.buildComposedBlockTime += el; -} - -static int32_t buildComposedDataBlock(STsdbReader* pReader) { - int32_t code = TSDB_CODE_SUCCESS; - - SSDataBlock* pResBlock = pReader->resBlockInfo.pResBlock; - - SFileDataBlockInfo* pBlockInfo = getCurrentBlockInfo(&pReader->status.blockIter); - SLastBlockReader* pLastBlockReader = pReader->status.fileIter.pLastBlockReader; - - bool asc = ASCENDING_TRAVERSE(pReader->order); - int64_t st = taosGetTimestampUs(); - int32_t step = asc ? 1 : -1; - double el = 0; - SDataBlk* pBlock = getCurrentBlock(&pReader->status.blockIter); - SFileBlockDumpInfo* pDumpInfo = &pReader->status.fBlockDumpInfo; - - STableBlockScanInfo* pBlockScanInfo = NULL; - if (pBlockInfo != NULL) { - if (pReader->pIgnoreTables && taosHashGet(*pReader->pIgnoreTables, &pBlockInfo->uid, sizeof(pBlockInfo->uid))) { - setBlockAllDumped(pDumpInfo, pBlock->maxKey.ts, pReader->order); - return code; - } - - pBlockScanInfo = getTableBlockScanInfo(pReader->status.pTableMap, pBlockInfo->uid, pReader->idStr); - if (pBlockScanInfo == NULL) { - goto _end; - } - - TSDBKEY keyInBuf = getCurrentKeyInBuf(pBlockScanInfo, pReader); - - // it is a clean block, load it directly - if (isCleanFileDataBlock(pReader, pBlockInfo, pBlock, pBlockScanInfo, keyInBuf, pLastBlockReader) && - pBlock->nRow <= pReader->resBlockInfo.capacity) { - if (asc || (!hasDataInLastBlock(pLastBlockReader) && (pBlock->maxKey.ts > keyInBuf.ts))) { - code = copyBlockDataToSDataBlock(pReader); - if (code) { - goto _end; - } - - // record the last key value - pBlockScanInfo->lastKey = asc ? pBlock->maxKey.ts : pBlock->minKey.ts; - goto _end; - } - } - } else { // file blocks not exist - pBlockScanInfo = *pReader->status.pTableIter; - if (pReader->pIgnoreTables && - taosHashGet(*pReader->pIgnoreTables, &pBlockScanInfo->uid, sizeof(pBlockScanInfo->uid))) { - setBlockAllDumped(pDumpInfo, pBlock->maxKey.ts, pReader->order); - return code; - } - } - - SBlockData* pBlockData = &pReader->status.fileBlockData; - - while (1) { - bool hasBlockData = false; - { - while (pBlockData->nRow > 0 && - pBlockData->uid == pBlockScanInfo->uid) { // find the first qualified row in data block - if (isValidFileBlockRow(pBlockData, pDumpInfo, pBlockScanInfo, pReader)) { - hasBlockData = true; - break; - } - - pDumpInfo->rowIndex += step; - - pBlock = getCurrentBlock(&pReader->status.blockIter); - if (pDumpInfo->rowIndex >= pBlock->nRow || pDumpInfo->rowIndex < 0) { - pBlockInfo = getCurrentBlockInfo(&pReader->status.blockIter); // NOTE: get the new block info - - // continue check for the next file block if the last ts in the current block - // is overlapped with the next neighbor block - bool loadNeighbor = false; - code = loadNeighborIfOverlap(pBlockInfo, pBlockScanInfo, pReader, &loadNeighbor); - if ((!loadNeighbor) || (code != 0)) { - setBlockAllDumped(pDumpInfo, pBlock->maxKey.ts, pReader->order); - break; - } - } - } - } - - // no data in last block and block, no need to proceed. - if (hasBlockData == false) { - break; - } - - code = buildComposedDataBlockImpl(pReader, pBlockScanInfo, pBlockData, pLastBlockReader); - if (code) { - goto _end; - } - - // currently loaded file data block is consumed - if ((pBlockData->nRow > 0) && (pDumpInfo->rowIndex >= pBlockData->nRow || pDumpInfo->rowIndex < 0)) { - pBlock = getCurrentBlock(&pReader->status.blockIter); - setBlockAllDumped(pDumpInfo, pBlock->maxKey.ts, pReader->order); - break; - } - - if (pResBlock->info.rows >= pReader->resBlockInfo.capacity) { - break; - } - } - -_end: - el = (taosGetTimestampUs() - st) / 1000.0; - updateComposedBlockInfo(pReader, el, pBlockScanInfo); - - if (pResBlock->info.rows > 0) { - tsdbDebug("%p uid:%" PRIu64 ", composed data block created, brange:%" PRIu64 "-%" PRIu64 " rows:%" PRId64 - ", elapsed time:%.2f ms %s", - pReader, pResBlock->info.id.uid, pResBlock->info.window.skey, pResBlock->info.window.ekey, - pResBlock->info.rows, el, pReader->idStr); - } - - return code; -} - -void setComposedBlockFlag(STsdbReader* pReader, bool composed) { pReader->status.composedDataBlock = composed; } - -int32_t getInitialDelIndex(const SArray* pDelSkyline, int32_t order) { - if (pDelSkyline == NULL) { - return 0; - } - - return ASCENDING_TRAVERSE(order) ? 0 : taosArrayGetSize(pDelSkyline) - 1; -} - -int32_t initDelSkylineIterator(STableBlockScanInfo* pBlockScanInfo, STsdbReader* pReader, STbData* pMemTbData, - STbData* piMemTbData) { - if (pBlockScanInfo->delSkyline != NULL) { - return TSDB_CODE_SUCCESS; - } - - int32_t code = 0; - SArray* pDelData = taosArrayInit(4, sizeof(SDelData)); - - SDelFile* pDelFile = pReader->pReadSnap->fs.pDelFile; - if (pDelFile && taosArrayGetSize(pReader->pDelIdx) > 0) { - SDelIdx idx = {.suid = pReader->suid, .uid = pBlockScanInfo->uid}; - SDelIdx* pIdx = taosArraySearch(pReader->pDelIdx, &idx, tCmprDelIdx, TD_EQ); - - if (pIdx != NULL) { - code = tsdbReadDelDatav1(pReader->pDelFReader, pIdx, pDelData, pReader->verRange.maxVer); - } - if (code != TSDB_CODE_SUCCESS) { - goto _err; - } - } - - SDelData* p = NULL; - if (pMemTbData != NULL) { - p = pMemTbData->pHead; - while (p) { - if (p->version <= pReader->verRange.maxVer) { - taosArrayPush(pDelData, p); - } - - p = p->pNext; - } - } - - if (piMemTbData != NULL) { - p = piMemTbData->pHead; - while (p) { - if (p->version <= pReader->verRange.maxVer) { - taosArrayPush(pDelData, p); - } - p = p->pNext; - } - } - - if (taosArrayGetSize(pDelData) > 0) { - pBlockScanInfo->delSkyline = taosArrayInit(4, sizeof(TSDBKEY)); - code = tsdbBuildDeleteSkyline(pDelData, 0, (int32_t)(taosArrayGetSize(pDelData) - 1), pBlockScanInfo->delSkyline); - } - - taosArrayDestroy(pDelData); - int32_t index = getInitialDelIndex(pBlockScanInfo->delSkyline, pReader->order); - - pBlockScanInfo->iter.index = index; - pBlockScanInfo->iiter.index = index; - pBlockScanInfo->fileDelIndex = index; - pBlockScanInfo->sttBlockDelIndex = index; - - return code; - -_err: - taosArrayDestroy(pDelData); - return code; -} - -TSDBKEY getCurrentKeyInBuf(STableBlockScanInfo* pScanInfo, STsdbReader* pReader) { - bool asc = ASCENDING_TRAVERSE(pReader->order); - // TSKEY initialVal = asc? TSKEY_MIN:TSKEY_MAX; - - TSDBKEY key = {.ts = TSKEY_INITIAL_VAL}, ikey = {.ts = TSKEY_INITIAL_VAL}; - - bool hasKey = false, hasIKey = false; - TSDBROW* pRow = getValidMemRow(&pScanInfo->iter, pScanInfo->delSkyline, pReader); - if (pRow != NULL) { - hasKey = true; - key = TSDBROW_KEY(pRow); - } - - TSDBROW* pIRow = getValidMemRow(&pScanInfo->iiter, pScanInfo->delSkyline, pReader); - if (pIRow != NULL) { - hasIKey = true; - ikey = TSDBROW_KEY(pIRow); - } - - if (hasKey) { - if (hasIKey) { // has data in mem & imem - if (asc) { - return key.ts <= ikey.ts ? key : ikey; - } else { - return key.ts <= ikey.ts ? ikey : key; - } - } else { // no data in imem - return key; - } - } else { - // no data in mem & imem, return the initial value - // only imem has data, return ikey - return ikey; - } -} - -static int32_t moveToNextFile(STsdbReader* pReader, SBlockNumber* pBlockNum, SArray* pTableList) { - SReaderStatus* pStatus = &pReader->status; - pBlockNum->numOfBlocks = 0; - pBlockNum->numOfLastFiles = 0; - - size_t numOfTables = tSimpleHashGetSize(pReader->status.pTableMap); - SArray* pIndexList = taosArrayInit(numOfTables, sizeof(SBlockIdx)); - - while (1) { - // only check here, since the iterate data in memory is very fast. - if (pReader->code != TSDB_CODE_SUCCESS) { - tsdbWarn("tsdb reader is stopped ASAP, code:%s, %s", strerror(pReader->code), pReader->idStr); - taosArrayDestroy(pIndexList); - return pReader->code; - } - - bool hasNext = false; - int32_t code = filesetIteratorNext(&pStatus->fileIter, pReader, &hasNext); - if (code != TSDB_CODE_SUCCESS) { - taosArrayDestroy(pIndexList); - return code; - } - - if (!hasNext) { // no data files on disk - break; - } - - taosArrayClear(pIndexList); - code = doLoadBlockIndex(pReader, pReader->pFileReader, pIndexList); - if (code != TSDB_CODE_SUCCESS) { - taosArrayDestroy(pIndexList); - return code; - } - - if (taosArrayGetSize(pIndexList) > 0 || pReader->pFileReader->pSet->nSttF > 0) { - code = doLoadFileBlock(pReader, pIndexList, pBlockNum, pTableList); - if (code != TSDB_CODE_SUCCESS) { - taosArrayDestroy(pIndexList); - return code; - } - - if (pBlockNum->numOfBlocks + pBlockNum->numOfLastFiles > 0) { - break; - } - } - - // no blocks in current file, try next files - } - - taosArrayDestroy(pIndexList); - - if (pReader->pReadSnap != NULL) { - SDelFile* pDelFile = pReader->pReadSnap->fs.pDelFile; - if (pReader->pDelFReader == NULL && pDelFile != NULL) { - int32_t code = tsdbDelFReaderOpen(&pReader->pDelFReader, pDelFile, pReader->pTsdb); - if (code != TSDB_CODE_SUCCESS) { - return code; - } - - pReader->pDelIdx = taosArrayInit(4, sizeof(SDelIdx)); - if (pReader->pDelIdx == NULL) { - code = TSDB_CODE_OUT_OF_MEMORY; - return code; - } - - code = tsdbReadDelIdx(pReader->pDelFReader, pReader->pDelIdx); - if (code != TSDB_CODE_SUCCESS) { - taosArrayDestroy(pReader->pDelIdx); - return code; - } - } - } - - return TSDB_CODE_SUCCESS; -} - -static void resetTableListIndex(SReaderStatus* pStatus) { - STableUidList* pList = &pStatus->uidList; - - pList->currentIndex = 0; - uint64_t uid = pList->tableUidList[0]; - pStatus->pTableIter = tSimpleHashGet(pStatus->pTableMap, &uid, sizeof(uid)); -} - -static bool moveToNextTable(STableUidList* pOrderedCheckInfo, SReaderStatus* pStatus) { - pOrderedCheckInfo->currentIndex += 1; - if (pOrderedCheckInfo->currentIndex >= tSimpleHashGetSize(pStatus->pTableMap)) { - pStatus->pTableIter = NULL; - return false; - } - - uint64_t uid = pOrderedCheckInfo->tableUidList[pOrderedCheckInfo->currentIndex]; - pStatus->pTableIter = tSimpleHashGet(pStatus->pTableMap, &uid, sizeof(uid)); - return (pStatus->pTableIter != NULL); -} - -static int32_t doLoadLastBlockSequentially(STsdbReader* pReader) { - SReaderStatus* pStatus = &pReader->status; - SLastBlockReader* pLastBlockReader = pStatus->fileIter.pLastBlockReader; - STableUidList* pUidList = &pStatus->uidList; - int32_t code = TSDB_CODE_SUCCESS; - - if (tSimpleHashGetSize(pStatus->pTableMap) == 0) { - return TSDB_CODE_SUCCESS; - } - - SSDataBlock* pResBlock = pReader->resBlockInfo.pResBlock; - - while (1) { - if (pReader->code != TSDB_CODE_SUCCESS) { - tsdbWarn("tsdb reader is stopped ASAP, code:%s, %s", strerror(pReader->code), pReader->idStr); - return pReader->code; - } - - // load the last data block of current table - STableBlockScanInfo* pScanInfo = *(STableBlockScanInfo**)pStatus->pTableIter; - if (pScanInfo == NULL) { - tsdbError("table Iter is null, invalid pScanInfo, try next table %s", pReader->idStr); - bool hasNexTable = moveToNextTable(pUidList, pStatus); - if (!hasNexTable) { - return TSDB_CODE_SUCCESS; - } - - continue; - } - - if (pReader->pIgnoreTables && taosHashGet(*pReader->pIgnoreTables, &pScanInfo->uid, sizeof(pScanInfo->uid))) { - // reset the index in last block when handing a new file - doCleanupTableScanInfo(pScanInfo); - pStatus->mapDataCleaned = true; - - bool hasNexTable = moveToNextTable(pUidList, pStatus); - if (!hasNexTable) { - return TSDB_CODE_SUCCESS; - } - - continue; - } - - // reset the index in last block when handing a new file - doCleanupTableScanInfo(pScanInfo); - pStatus->mapDataCleaned = true; - - bool hasDataInLastFile = initLastBlockReader(pLastBlockReader, pScanInfo, pReader); - if (!hasDataInLastFile) { - bool hasNexTable = moveToNextTable(pUidList, pStatus); - if (!hasNexTable) { - return TSDB_CODE_SUCCESS; - } - - continue; - } - - int64_t st = taosGetTimestampUs(); - while (1) { - bool hasBlockLData = hasDataInLastBlock(pLastBlockReader); - - // no data in last block and block, no need to proceed. - if (hasBlockLData == false) { - break; - } - - code = buildComposedDataBlockImpl(pReader, pScanInfo, &pReader->status.fileBlockData, pLastBlockReader); - if (code) { - return code; - } - - if (pResBlock->info.rows >= pReader->resBlockInfo.capacity) { - break; - } - } - - double el = (taosGetTimestampUs() - st) / 1000.0; - updateComposedBlockInfo(pReader, el, pScanInfo); - - if (pResBlock->info.rows > 0) { - tsdbDebug("%p uid:%" PRIu64 ", composed data block created, brange:%" PRIu64 "-%" PRIu64 " rows:%" PRId64 - ", elapsed time:%.2f ms %s", - pReader, pResBlock->info.id.uid, pResBlock->info.window.skey, pResBlock->info.window.ekey, - pResBlock->info.rows, el, pReader->idStr); - return TSDB_CODE_SUCCESS; - } - - // current table is exhausted, let's try next table - bool hasNexTable = moveToNextTable(pUidList, pStatus); - if (!hasNexTable) { - return TSDB_CODE_SUCCESS; - } - } -} - -static int32_t doBuildDataBlock(STsdbReader* pReader) { - int32_t code = TSDB_CODE_SUCCESS; - - SReaderStatus* pStatus = &pReader->status; - SDataBlockIter* pBlockIter = &pStatus->blockIter; - STableBlockScanInfo* pScanInfo = NULL; - SFileDataBlockInfo* pBlockInfo = getCurrentBlockInfo(pBlockIter); - SLastBlockReader* pLastBlockReader = pReader->status.fileIter.pLastBlockReader; - SDataBlk* pBlock = getCurrentBlock(pBlockIter); - - if (pReader->pIgnoreTables && taosHashGet(*pReader->pIgnoreTables, &pBlockInfo->uid, sizeof(pBlockInfo->uid))) { - setBlockAllDumped(&pStatus->fBlockDumpInfo, pBlock->maxKey.ts, pReader->order); - return code; - } - - if (pReader->code != TSDB_CODE_SUCCESS) { - return pReader->code; - } - - pScanInfo = getTableBlockScanInfo(pReader->status.pTableMap, pBlockInfo->uid, pReader->idStr); - if (pScanInfo == NULL) { - return terrno; - } - - initLastBlockReader(pLastBlockReader, pScanInfo, pReader); - TSDBKEY keyInBuf = getCurrentKeyInBuf(pScanInfo, pReader); - - if (fileBlockShouldLoad(pReader, pBlockInfo, pBlock, pScanInfo, keyInBuf, pLastBlockReader)) { - code = doLoadFileBlockData(pReader, pBlockIter, &pStatus->fileBlockData, pScanInfo->uid); - if (code != TSDB_CODE_SUCCESS) { - return code; - } - - // build composed data block - code = buildComposedDataBlock(pReader); - } else if (bufferDataInFileBlockGap(pReader->order, keyInBuf, pBlock)) { - // data in memory that are earlier than current file block - // rows in buffer should be less than the file block in asc, greater than file block in desc - int64_t endKey = (ASCENDING_TRAVERSE(pReader->order)) ? pBlock->minKey.ts : pBlock->maxKey.ts; - code = buildDataBlockFromBuf(pReader, pScanInfo, endKey); - } else { - if (hasDataInLastBlock(pLastBlockReader) && !ASCENDING_TRAVERSE(pReader->order)) { - // only return the rows in last block - int64_t tsLast = getCurrentKeyInLastBlock(pLastBlockReader); - ASSERT(tsLast >= pBlock->maxKey.ts); - - SBlockData* pBData = &pReader->status.fileBlockData; - tBlockDataReset(pBData); - - SSDataBlock* pResBlock = pReader->resBlockInfo.pResBlock; - tsdbDebug("load data in last block firstly, due to desc scan data, %s", pReader->idStr); - - int64_t st = taosGetTimestampUs(); - - while (1) { - bool hasBlockLData = hasDataInLastBlock(pLastBlockReader); - - // no data in last block and block, no need to proceed. - if (hasBlockLData == false) { - break; - } - - code = buildComposedDataBlockImpl(pReader, pScanInfo, &pReader->status.fileBlockData, pLastBlockReader); - if (code) { - return code; - } - - if (pResBlock->info.rows >= pReader->resBlockInfo.capacity) { - break; - } - } - - double el = (taosGetTimestampUs() - st) / 1000.0; - updateComposedBlockInfo(pReader, el, pScanInfo); - - if (pResBlock->info.rows > 0) { - tsdbDebug("%p uid:%" PRIu64 ", composed data block created, brange:%" PRIu64 "-%" PRIu64 " rows:%" PRId64 - ", elapsed time:%.2f ms %s", - pReader, pResBlock->info.id.uid, pResBlock->info.window.skey, pResBlock->info.window.ekey, - pResBlock->info.rows, el, pReader->idStr); - } - } else { // whole block is required, return it directly - SDataBlockInfo* pInfo = &pReader->resBlockInfo.pResBlock->info; - pInfo->rows = pBlock->nRow; - pInfo->id.uid = pScanInfo->uid; - pInfo->dataLoad = 0; - pInfo->window = (STimeWindow){.skey = pBlock->minKey.ts, .ekey = pBlock->maxKey.ts}; - setComposedBlockFlag(pReader, false); - setBlockAllDumped(&pStatus->fBlockDumpInfo, pBlock->maxKey.ts, pReader->order); - - // update the last key for the corresponding table - pScanInfo->lastKey = ASCENDING_TRAVERSE(pReader->order) ? pInfo->window.ekey : pInfo->window.skey; - tsdbDebug("%p uid:%" PRIu64 - " clean file block retrieved from file, global index:%d, " - "table index:%d, rows:%d, brange:%" PRId64 "-%" PRId64 ", %s", - pReader, pScanInfo->uid, pBlockIter->index, pBlockInfo->tbBlockIdx, pBlock->nRow, pBlock->minKey.ts, - pBlock->maxKey.ts, pReader->idStr); - } - } - - return (pReader->code != TSDB_CODE_SUCCESS) ? pReader->code : code; -} - -static int32_t doSumFileBlockRows(STsdbReader* pReader, SDataFReader* pFileReader) { - int64_t st = taosGetTimestampUs(); - LRUHandle* handle = NULL; - int32_t code = tsdbCacheGetBlockIdx(pFileReader->pTsdb->biCache, pFileReader, &handle); - if (code != TSDB_CODE_SUCCESS || handle == NULL) { - goto _end; - } - - int32_t numOfTables = tSimpleHashGetSize(pReader->status.pTableMap); - - SArray* aBlockIdx = (SArray*)taosLRUCacheValue(pFileReader->pTsdb->biCache, handle); - size_t num = taosArrayGetSize(aBlockIdx); - if (num == 0) { - tsdbBICacheRelease(pFileReader->pTsdb->biCache, handle); - return TSDB_CODE_SUCCESS; - } - - SBlockIdx* pBlockIdx = NULL; - for (int32_t i = 0; i < num; ++i) { - pBlockIdx = (SBlockIdx*)taosArrayGet(aBlockIdx, i); - if (pBlockIdx->suid != pReader->suid) { - continue; - } - - STableBlockScanInfo** p = tSimpleHashGet(pReader->status.pTableMap, &pBlockIdx->uid, sizeof(pBlockIdx->uid)); - if (p == NULL) { - continue; - } - - STableBlockScanInfo* pScanInfo = *p; - tMapDataReset(&pScanInfo->mapData); - tsdbReadDataBlk(pReader->pFileReader, pBlockIdx, &pScanInfo->mapData); - - SDataBlk block = {0}; - for (int32_t j = 0; j < pScanInfo->mapData.nItem; ++j) { - tGetDataBlk(pScanInfo->mapData.pData + pScanInfo->mapData.aOffset[j], &block); - pReader->rowsNum += block.nRow; - } - } - -_end: - tsdbBICacheRelease(pFileReader->pTsdb->biCache, handle); - return code; -} - -static int32_t doSumSttBlockRows(STsdbReader* pReader) { - int32_t code = TSDB_CODE_SUCCESS; - SLastBlockReader* pLastBlockReader = pReader->status.fileIter.pLastBlockReader; - SSttBlockLoadInfo* pBlockLoadInfo = NULL; - - for (int32_t i = 0; i < pReader->pFileReader->pSet->nSttF; ++i) { // open all last file - pBlockLoadInfo = &pLastBlockReader->pInfo[i]; - - code = tsdbReadSttBlk(pReader->pFileReader, i, pBlockLoadInfo->aSttBlk); - if (code) { - return code; - } - - size_t size = taosArrayGetSize(pBlockLoadInfo->aSttBlk); - if (size >= 1) { - SSttBlk* pStart = taosArrayGet(pBlockLoadInfo->aSttBlk, 0); - SSttBlk* pEnd = taosArrayGet(pBlockLoadInfo->aSttBlk, size - 1); - - // all identical - if (pStart->suid == pEnd->suid) { - if (pStart->suid != pReader->suid) { - // no qualified stt block existed - taosArrayClear(pBlockLoadInfo->aSttBlk); - continue; - } - for (int32_t j = 0; j < size; ++j) { - SSttBlk* p = taosArrayGet(pBlockLoadInfo->aSttBlk, j); - pReader->rowsNum += p->nRow; - } - } else { - for (int32_t j = 0; j < size; ++j) { - SSttBlk* p = taosArrayGet(pBlockLoadInfo->aSttBlk, j); - uint64_t s = p->suid; - if (s < pReader->suid) { - continue; - } - - if (s == pReader->suid) { - pReader->rowsNum += p->nRow; - } else if (s > pReader->suid) { - break; - } - } - } - } - } - - return code; -} - -static int32_t readRowsCountFromFiles(STsdbReader* pReader) { - int32_t code = TSDB_CODE_SUCCESS; - - while (1) { - bool hasNext = false; - code = filesetIteratorNext(&pReader->status.fileIter, pReader, &hasNext); - if (code) { - return code; - } - - if (!hasNext) { // no data files on disk - break; - } - - code = doSumFileBlockRows(pReader, pReader->pFileReader); - if (code != TSDB_CODE_SUCCESS) { - return code; - } - - code = doSumSttBlockRows(pReader); - if (code != TSDB_CODE_SUCCESS) { - return code; - } - } - - pReader->status.loadFromFile = false; - - return code; -} - -static int32_t readRowsCountFromMem(STsdbReader* pReader) { - int32_t code = TSDB_CODE_SUCCESS; - int64_t memNum = 0, imemNum = 0; - if (pReader->pReadSnap->pMem != NULL) { - tsdbMemTableCountRows(pReader->pReadSnap->pMem, pReader->status.pTableMap, &memNum); - } - - if (pReader->pReadSnap->pIMem != NULL) { - tsdbMemTableCountRows(pReader->pReadSnap->pIMem, pReader->status.pTableMap, &imemNum); - } - - pReader->rowsNum += memNum + imemNum; - - return code; -} - -static int32_t buildBlockFromBufferSequentially(STsdbReader* pReader) { - SReaderStatus* pStatus = &pReader->status; - STableUidList* pUidList = &pStatus->uidList; - - while (1) { - if (pReader->code != TSDB_CODE_SUCCESS) { - tsdbWarn("tsdb reader is stopped ASAP, code:%s, %s", strerror(pReader->code), pReader->idStr); - return pReader->code; - } - - STableBlockScanInfo** pBlockScanInfo = pStatus->pTableIter; - if (pReader->pIgnoreTables && - taosHashGet(*pReader->pIgnoreTables, &(*pBlockScanInfo)->uid, sizeof((*pBlockScanInfo)->uid))) { - bool hasNexTable = moveToNextTable(pUidList, pStatus); - if (!hasNexTable) { - return TSDB_CODE_SUCCESS; - } - pBlockScanInfo = pStatus->pTableIter; - } - - initMemDataIterator(*pBlockScanInfo, pReader); - - int64_t endKey = (ASCENDING_TRAVERSE(pReader->order)) ? INT64_MAX : INT64_MIN; - int32_t code = buildDataBlockFromBuf(pReader, *pBlockScanInfo, endKey); - if (code != TSDB_CODE_SUCCESS) { - return code; - } - - if (pReader->resBlockInfo.pResBlock->info.rows > 0) { - return TSDB_CODE_SUCCESS; - } - - // current table is exhausted, let's try next table - bool hasNexTable = moveToNextTable(pUidList, pStatus); - if (!hasNexTable) { - return TSDB_CODE_SUCCESS; - } - } -} - -// set the correct start position in case of the first/last file block, according to the query time window -static void initBlockDumpInfo(STsdbReader* pReader, SDataBlockIter* pBlockIter) { - int64_t lastKey = ASCENDING_TRAVERSE(pReader->order) ? INT64_MIN : INT64_MAX; - SDataBlk* pBlock = getCurrentBlock(pBlockIter); - SFileDataBlockInfo* pBlockInfo = getCurrentBlockInfo(pBlockIter); - if (pBlockInfo) { - STableBlockScanInfo* pScanInfo = tSimpleHashGet(pBlockIter->pTableMap, &pBlockInfo->uid, sizeof(pBlockInfo->uid)); - if (pScanInfo) { - lastKey = pScanInfo->lastKey; - } - } - SReaderStatus* pStatus = &pReader->status; - - SFileBlockDumpInfo* pDumpInfo = &pStatus->fBlockDumpInfo; - - pDumpInfo->totalRows = pBlock->nRow; - pDumpInfo->allDumped = false; - pDumpInfo->rowIndex = ASCENDING_TRAVERSE(pReader->order) ? 0 : pBlock->nRow - 1; - pDumpInfo->lastKey = lastKey; -} - -static int32_t initForFirstBlockInFile(STsdbReader* pReader, SDataBlockIter* pBlockIter) { - SBlockNumber num = {0}; - SArray* pTableList = taosArrayInit(40, POINTER_BYTES); - - int32_t code = moveToNextFile(pReader, &num, pTableList); - if (code != TSDB_CODE_SUCCESS) { - taosArrayDestroy(pTableList); - return code; - } - - // all data files are consumed, try data in buffer - if (num.numOfBlocks + num.numOfLastFiles == 0) { - pReader->status.loadFromFile = false; - taosArrayDestroy(pTableList); - return code; - } - - // initialize the block iterator for a new fileset - if (num.numOfBlocks > 0) { - code = initBlockIterator(pReader, pBlockIter, num.numOfBlocks, pTableList); - } else { // no block data, only last block exists - tBlockDataReset(&pReader->status.fileBlockData); - resetDataBlockIterator(pBlockIter, pReader->order); - resetTableListIndex(&pReader->status); - } - - // set the correct start position according to the query time window - initBlockDumpInfo(pReader, pBlockIter); - taosArrayDestroy(pTableList); - return code; -} - -static bool fileBlockPartiallyRead(SFileBlockDumpInfo* pDumpInfo, bool asc) { - return (!pDumpInfo->allDumped) && - ((pDumpInfo->rowIndex > 0 && asc) || (pDumpInfo->rowIndex < (pDumpInfo->totalRows - 1) && (!asc))); -} - -typedef enum { - TSDB_READ_RETURN = 0x1, - TSDB_READ_CONTINUE = 0x2, -} ERetrieveType; - -static ERetrieveType doReadDataFromLastFiles(STsdbReader* pReader) { - int32_t code = TSDB_CODE_SUCCESS; - SSDataBlock* pResBlock = pReader->resBlockInfo.pResBlock; - SDataBlockIter* pBlockIter = &pReader->status.blockIter; - - while (1) { - terrno = 0; - - code = doLoadLastBlockSequentially(pReader); - if (code != TSDB_CODE_SUCCESS) { - terrno = code; - return TSDB_READ_RETURN; - } - - if (pResBlock->info.rows > 0) { - return TSDB_READ_RETURN; - } - - // all data blocks are checked in this last block file, now let's try the next file - ASSERT(pReader->status.pTableIter == NULL); - code = initForFirstBlockInFile(pReader, pBlockIter); - - // error happens or all the data files are completely checked - if ((code != TSDB_CODE_SUCCESS) || (pReader->status.loadFromFile == false)) { - terrno = code; - return TSDB_READ_RETURN; - } - - if (pBlockIter->numOfBlocks > 0) { // there are data blocks existed. - return TSDB_READ_CONTINUE; - } else { // all blocks in data file are checked, let's check the data in last files - resetTableListIndex(&pReader->status); - } - } -} - -static int32_t buildBlockFromFiles(STsdbReader* pReader) { - int32_t code = TSDB_CODE_SUCCESS; - bool asc = ASCENDING_TRAVERSE(pReader->order); - - SDataBlockIter* pBlockIter = &pReader->status.blockIter; - SSDataBlock* pResBlock = pReader->resBlockInfo.pResBlock; - - if (pBlockIter->numOfBlocks == 0) { - // let's try to extract data from stt files. - ERetrieveType type = doReadDataFromLastFiles(pReader); - if (type == TSDB_READ_RETURN) { - return terrno; - } - - code = doBuildDataBlock(pReader); - if (code != TSDB_CODE_SUCCESS || pResBlock->info.rows > 0) { - return code; - } - } - - while (1) { - SFileBlockDumpInfo* pDumpInfo = &pReader->status.fBlockDumpInfo; - - if (fileBlockPartiallyRead(pDumpInfo, asc)) { // file data block is partially loaded - code = buildComposedDataBlock(pReader); - } else { - // current block are exhausted, try the next file block - if (pDumpInfo->allDumped) { - // try next data block in current file - bool hasNext = blockIteratorNext(&pReader->status.blockIter, pReader->idStr); - if (hasNext) { // check for the next block in the block accessed order list - initBlockDumpInfo(pReader, pBlockIter); - } else { - // all data blocks in files are checked, let's check the data in last files. - ASSERT(pReader->status.pCurrentFileset->nSttF > 0); - - // data blocks in current file are exhausted, let's try the next file now - SBlockData* pBlockData = &pReader->status.fileBlockData; - if (pBlockData->uid != 0) { - tBlockDataClear(pBlockData); - } - - tBlockDataReset(pBlockData); - resetDataBlockIterator(pBlockIter, pReader->order); - resetTableListIndex(&pReader->status); - - ERetrieveType type = doReadDataFromLastFiles(pReader); - if (type == TSDB_READ_RETURN) { - return terrno; - } - } - } - - code = doBuildDataBlock(pReader); - } - - if (code != TSDB_CODE_SUCCESS || pResBlock->info.rows > 0) { - return code; - } - } -} - -static STsdb* getTsdbByRetentions(SVnode* pVnode, TSKEY winSKey, SRetention* retentions, const char* idStr, - int8_t* pLevel) { - if (VND_IS_RSMA(pVnode)) { - int8_t level = 0; - int8_t precision = pVnode->config.tsdbCfg.precision; - int64_t now = taosGetTimestamp(precision); - int64_t offset = tsQueryRsmaTolerance * ((precision == TSDB_TIME_PRECISION_MILLI) ? 1L - : (precision == TSDB_TIME_PRECISION_MICRO) ? 1000L - : 1000000L); - - for (int8_t i = 0; i < TSDB_RETENTION_MAX; ++i) { - SRetention* pRetention = retentions + level; - if (pRetention->keep <= 0) { - if (level > 0) { - --level; - } - break; - } - if ((now - pRetention->keep) <= (winSKey + offset)) { - break; - } - ++level; - } - - const char* str = (idStr != NULL) ? idStr : ""; - - if (level == TSDB_RETENTION_L0) { - *pLevel = TSDB_RETENTION_L0; - tsdbDebug("vgId:%d, rsma level %d is selected to query %s", TD_VID(pVnode), TSDB_RETENTION_L0, str); - return VND_RSMA0(pVnode); - } else if (level == TSDB_RETENTION_L1) { - *pLevel = TSDB_RETENTION_L1; - tsdbDebug("vgId:%d, rsma level %d is selected to query %s", TD_VID(pVnode), TSDB_RETENTION_L1, str); - return VND_RSMA1(pVnode); - } else { - *pLevel = TSDB_RETENTION_L2; - tsdbDebug("vgId:%d, rsma level %d is selected to query %s", TD_VID(pVnode), TSDB_RETENTION_L2, str); - return VND_RSMA2(pVnode); - } - } - - return VND_TSDB(pVnode); -} - -SVersionRange getQueryVerRange(SVnode* pVnode, SQueryTableDataCond* pCond, const char* id) { - int64_t startVer = (pCond->startVersion == -1) ? 0 : pCond->startVersion; - - int64_t endVer = 0; - if (pCond->endVersion == -1) { - // user not specified end version, set current maximum version of vnode as the endVersion - endVer = pVnode->state.applied; - } else { - endVer = (pCond->endVersion > pVnode->state.applied) ? pVnode->state.applied : pCond->endVersion; - } - - tsdbDebug("queried verRange:%" PRId64 "-%" PRId64 ", revised query verRange:%" PRId64 "-%" PRId64 ", %s", - pCond->startVersion, pCond->endVersion, startVer, endVer, id); - - return (SVersionRange){.minVer = startVer, .maxVer = endVer}; -} - -bool hasBeenDropped(const SArray* pDelList, int32_t* index, int64_t key, int64_t ver, int32_t order, - SVersionRange* pVerRange) { - if (pDelList == NULL) { - return false; - } - - size_t num = taosArrayGetSize(pDelList); - bool asc = ASCENDING_TRAVERSE(order); - int32_t step = asc ? 1 : -1; - - if (asc) { - if (*index >= num - 1) { - TSDBKEY* last = taosArrayGetLast(pDelList); - ASSERT(key >= last->ts); - - if (key > last->ts) { - return false; - } else if (key == last->ts) { - TSDBKEY* prev = taosArrayGet(pDelList, num - 2); - return (prev->version >= ver && prev->version <= pVerRange->maxVer && prev->version >= pVerRange->minVer); - } - } else { - TSDBKEY* pCurrent = taosArrayGet(pDelList, *index); - TSDBKEY* pNext = taosArrayGet(pDelList, (*index) + 1); - - if (key < pCurrent->ts) { - return false; - } - - if (pCurrent->ts <= key && pNext->ts >= key && pCurrent->version >= ver && - pVerRange->maxVer >= pCurrent->version) { - return true; - } - - while (pNext->ts <= key && (*index) < num - 1) { - (*index) += 1; - - if ((*index) < num - 1) { - pCurrent = taosArrayGet(pDelList, *index); - pNext = taosArrayGet(pDelList, (*index) + 1); - - // it is not a consecutive deletion range, ignore it - if (pCurrent->version == 0 && pNext->version > 0) { - continue; - } - - if (pCurrent->ts <= key && pNext->ts >= key && pCurrent->version >= ver && - pVerRange->maxVer >= pCurrent->version) { - return true; - } - } - } - - return false; - } - } else { - if (*index <= 0) { - TSDBKEY* pFirst = taosArrayGet(pDelList, 0); - - if (key < pFirst->ts) { - return false; - } else if (key == pFirst->ts) { - return pFirst->version >= ver; - } else { - ASSERT(0); - } - } else { - TSDBKEY* pCurrent = taosArrayGet(pDelList, *index); - TSDBKEY* pPrev = taosArrayGet(pDelList, (*index) - 1); - - if (key > pCurrent->ts) { - return false; - } - - if (pPrev->ts <= key && pCurrent->ts >= key && pPrev->version >= ver) { - return true; - } - - while (pPrev->ts >= key && (*index) > 1) { - (*index) += step; - - if ((*index) >= 1) { - pCurrent = taosArrayGet(pDelList, *index); - pPrev = taosArrayGet(pDelList, (*index) - 1); - - // it is not a consecutive deletion range, ignore it - if (pCurrent->version > 0 && pPrev->version == 0) { - continue; - } - - if (pPrev->ts <= key && pCurrent->ts >= key && pPrev->version >= ver) { - return true; - } - } - } - - return false; - } - } - - return false; -} - -TSDBROW* getValidMemRow(SIterInfo* pIter, const SArray* pDelList, STsdbReader* pReader) { - if (!pIter->hasVal) { - return NULL; - } - - TSDBROW* pRow = tsdbTbDataIterGet(pIter->iter); - TSDBKEY key = TSDBROW_KEY(pRow); - - if (outOfTimeWindow(key.ts, &pReader->window)) { - pIter->hasVal = false; - return NULL; - } - - // it is a valid data version - if ((key.version <= pReader->verRange.maxVer && key.version >= pReader->verRange.minVer) && - (!hasBeenDropped(pDelList, &pIter->index, key.ts, key.version, pReader->order, &pReader->verRange))) { - return pRow; - } - - while (1) { - pIter->hasVal = tsdbTbDataIterNext(pIter->iter); - if (!pIter->hasVal) { - return NULL; - } - - pRow = tsdbTbDataIterGet(pIter->iter); - - key = TSDBROW_KEY(pRow); - if (outOfTimeWindow(key.ts, &pReader->window)) { - pIter->hasVal = false; - return NULL; - } - - if (key.version <= pReader->verRange.maxVer && key.version >= pReader->verRange.minVer && - (!hasBeenDropped(pDelList, &pIter->index, key.ts, key.version, pReader->order, &pReader->verRange))) { - return pRow; - } - } -} - -int32_t doMergeRowsInBuf(SIterInfo* pIter, uint64_t uid, int64_t ts, SArray* pDelList, STsdbReader* pReader) { - SRowMerger* pMerger = &pReader->status.merger; - - while (1) { - pIter->hasVal = tsdbTbDataIterNext(pIter->iter); - if (!pIter->hasVal) { - break; - } - - // data exists but not valid - TSDBROW* pRow = getValidMemRow(pIter, pDelList, pReader); - if (pRow == NULL) { - break; - } - - // ts is not identical, quit - TSDBKEY k = TSDBROW_KEY(pRow); - if (k.ts != ts) { - break; - } - - if (pRow->type == TSDBROW_ROW_FMT) { - STSchema* pTSchema = doGetSchemaForTSRow(TSDBROW_SVERSION(pRow), pReader, uid); - if (pTSchema == NULL) { - return terrno; - } - - tsdbRowMergerAdd(pMerger, pRow, pTSchema); - } else { // column format - tsdbRowMergerAdd(pMerger, pRow, NULL); - } - } - - return TSDB_CODE_SUCCESS; -} - -static int32_t doMergeRowsInFileBlockImpl(SBlockData* pBlockData, int32_t rowIndex, int64_t key, SRowMerger* pMerger, - SVersionRange* pVerRange, int32_t step) { - while (rowIndex < pBlockData->nRow && rowIndex >= 0 && pBlockData->aTSKEY[rowIndex] == key) { - if (pBlockData->aVersion[rowIndex] > pVerRange->maxVer || pBlockData->aVersion[rowIndex] < pVerRange->minVer) { - rowIndex += step; - continue; - } - - TSDBROW fRow = tsdbRowFromBlockData(pBlockData, rowIndex); - tsdbRowMergerAdd(pMerger, &fRow, NULL); - rowIndex += step; - } - - return rowIndex; -} - -typedef enum { - CHECK_FILEBLOCK_CONT = 0x1, - CHECK_FILEBLOCK_QUIT = 0x2, -} CHECK_FILEBLOCK_STATE; - -static int32_t checkForNeighborFileBlock(STsdbReader* pReader, STableBlockScanInfo* pScanInfo, SDataBlk* pBlock, - SFileDataBlockInfo* pFBlock, SRowMerger* pMerger, int64_t key, - CHECK_FILEBLOCK_STATE* state) { - SFileBlockDumpInfo* pDumpInfo = &pReader->status.fBlockDumpInfo; - SBlockData* pBlockData = &pReader->status.fileBlockData; - bool asc = ASCENDING_TRAVERSE(pReader->order); - - *state = CHECK_FILEBLOCK_QUIT; - int32_t step = ASCENDING_TRAVERSE(pReader->order) ? 1 : -1; - - bool loadNeighbor = true; - int32_t code = loadNeighborIfOverlap(pFBlock, pScanInfo, pReader, &loadNeighbor); - - if (loadNeighbor && (code == TSDB_CODE_SUCCESS)) { - pDumpInfo->rowIndex = - doMergeRowsInFileBlockImpl(pBlockData, pDumpInfo->rowIndex, key, pMerger, &pReader->verRange, step); - if ((pDumpInfo->rowIndex >= pDumpInfo->totalRows && asc) || (pDumpInfo->rowIndex < 0 && !asc)) { - *state = CHECK_FILEBLOCK_CONT; - } - } - - return code; -} - -int32_t doMergeRowsInFileBlocks(SBlockData* pBlockData, STableBlockScanInfo* pScanInfo, STsdbReader* pReader) { - SFileBlockDumpInfo* pDumpInfo = &pReader->status.fBlockDumpInfo; - - SRowMerger* pMerger = &pReader->status.merger; - bool asc = ASCENDING_TRAVERSE(pReader->order); - int64_t key = pBlockData->aTSKEY[pDumpInfo->rowIndex]; - int32_t step = asc ? 1 : -1; - - pDumpInfo->rowIndex += step; - if ((pDumpInfo->rowIndex <= pBlockData->nRow - 1 && asc) || (pDumpInfo->rowIndex >= 0 && !asc)) { - pDumpInfo->rowIndex = - doMergeRowsInFileBlockImpl(pBlockData, pDumpInfo->rowIndex, key, pMerger, &pReader->verRange, step); - } - - // all rows are consumed, let's try next file block - if ((pDumpInfo->rowIndex >= pBlockData->nRow && asc) || (pDumpInfo->rowIndex < 0 && !asc)) { - while (1) { - CHECK_FILEBLOCK_STATE st; - - SFileDataBlockInfo* pFileBlockInfo = getCurrentBlockInfo(&pReader->status.blockIter); - SDataBlk* pCurrentBlock = getCurrentBlock(&pReader->status.blockIter); - if (pFileBlockInfo == NULL) { - st = CHECK_FILEBLOCK_QUIT; - break; - } - - checkForNeighborFileBlock(pReader, pScanInfo, pCurrentBlock, pFileBlockInfo, pMerger, key, &st); - if (st == CHECK_FILEBLOCK_QUIT) { - break; - } - } - } - - return TSDB_CODE_SUCCESS; -} - -int32_t doMergeRowsInLastBlock(SLastBlockReader* pLastBlockReader, STableBlockScanInfo* pScanInfo, int64_t ts, - SRowMerger* pMerger, SVersionRange* pVerRange, const char* idStr) { - while (nextRowFromLastBlocks(pLastBlockReader, pScanInfo, pVerRange)) { - int64_t next1 = getCurrentKeyInLastBlock(pLastBlockReader); - if (next1 == ts) { - TSDBROW* pRow1 = tMergeTreeGetRow(&pLastBlockReader->mergeTree); - tsdbRowMergerAdd(pMerger, pRow1, NULL); - } else { - tsdbTrace("uid:%" PRIu64 " last del index:%d, del range:%d, lastKeyInStt:%" PRId64 ", %s", pScanInfo->uid, - pScanInfo->sttBlockDelIndex, (int32_t)taosArrayGetSize(pScanInfo->delSkyline), pScanInfo->lastKeyInStt, - idStr); - break; - } - } - - return TSDB_CODE_SUCCESS; -} - -int32_t doMergeMemTableMultiRows(TSDBROW* pRow, uint64_t uid, SIterInfo* pIter, SArray* pDelList, TSDBROW* pResRow, - STsdbReader* pReader, bool* freeTSRow) { - TSDBROW* pNextRow = NULL; - TSDBROW current = *pRow; - - { // if the timestamp of the next valid row has a different ts, return current row directly - pIter->hasVal = tsdbTbDataIterNext(pIter->iter); - - if (!pIter->hasVal) { - *pResRow = *pRow; - *freeTSRow = false; - return TSDB_CODE_SUCCESS; - } else { // has next point in mem/imem - pNextRow = getValidMemRow(pIter, pDelList, pReader); - if (pNextRow == NULL) { - *pResRow = current; - *freeTSRow = false; - return TSDB_CODE_SUCCESS; - } - - if (TSDBROW_TS(¤t) != TSDBROW_TS(pNextRow)) { - *pResRow = current; - *freeTSRow = false; - return TSDB_CODE_SUCCESS; - } - } - } - - terrno = 0; - int32_t code = 0; - - // start to merge duplicated rows - if (current.type == TSDBROW_ROW_FMT) { - // get the correct schema for data in memory - STSchema* pTSchema = doGetSchemaForTSRow(TSDBROW_SVERSION(¤t), pReader, uid); - if (pTSchema == NULL) { - return terrno; - } - - code = tsdbRowMergerAdd(&pReader->status.merger, ¤t, pTSchema); - if (code != TSDB_CODE_SUCCESS) { - return code; - } - - STSchema* pTSchema1 = doGetSchemaForTSRow(TSDBROW_SVERSION(pNextRow), pReader, uid); - if (pTSchema1 == NULL) { - return terrno; - } - - tsdbRowMergerAdd(&pReader->status.merger, pNextRow, pTSchema1); - } else { // let's merge rows in file block - code = tsdbRowMergerAdd(&pReader->status.merger, ¤t, pReader->pSchema); - if (code != TSDB_CODE_SUCCESS) { - return code; - } - - tsdbRowMergerAdd(&pReader->status.merger, pNextRow, NULL); - } - - code = doMergeRowsInBuf(pIter, uid, TSDBROW_TS(¤t), pDelList, pReader); - if (code != TSDB_CODE_SUCCESS) { - return code; - } - - code = tsdbRowMergerGetRow(&pReader->status.merger, &pResRow->pTSRow); - if (code != TSDB_CODE_SUCCESS) { - return code; - } - - pResRow->type = TSDBROW_ROW_FMT; - tsdbRowMergerClear(&pReader->status.merger); - *freeTSRow = true; - - return TSDB_CODE_SUCCESS; -} - -int32_t doMergeMemIMemRows(TSDBROW* pRow, TSDBROW* piRow, STableBlockScanInfo* pBlockScanInfo, STsdbReader* pReader, - SRow** pTSRow) { - SRowMerger* pMerger = &pReader->status.merger; - - TSDBKEY k = TSDBROW_KEY(pRow); - TSDBKEY ik = TSDBROW_KEY(piRow); - STSchema* pSchema = doGetSchemaForTSRow(TSDBROW_SVERSION(pRow), pReader, pBlockScanInfo->uid); - if (pSchema == NULL) { - return terrno; - } - - STSchema* piSchema = doGetSchemaForTSRow(TSDBROW_SVERSION(piRow), pReader, pBlockScanInfo->uid); - if (piSchema == NULL) { - return terrno; - } - - if (ASCENDING_TRAVERSE(pReader->order)) { // ascending order imem --> mem - int32_t code = tsdbRowMergerAdd(&pReader->status.merger, piRow, piSchema); - if (code != TSDB_CODE_SUCCESS) { - return code; - } - - code = doMergeRowsInBuf(&pBlockScanInfo->iiter, pBlockScanInfo->uid, ik.ts, pBlockScanInfo->delSkyline, pReader); - if (code != TSDB_CODE_SUCCESS) { - return code; - } - - tsdbRowMergerAdd(&pReader->status.merger, pRow, pSchema); - code = doMergeRowsInBuf(&pBlockScanInfo->iter, pBlockScanInfo->uid, k.ts, pBlockScanInfo->delSkyline, pReader); - if (code != TSDB_CODE_SUCCESS) { - return code; - } - - } else { - int32_t code = tsdbRowMergerAdd(&pReader->status.merger, pRow, pSchema); - if (code != TSDB_CODE_SUCCESS || pMerger->pTSchema == NULL) { - return code; - } - - code = doMergeRowsInBuf(&pBlockScanInfo->iter, pBlockScanInfo->uid, k.ts, pBlockScanInfo->delSkyline, pReader); - if (code != TSDB_CODE_SUCCESS) { - return code; - } - - tsdbRowMergerAdd(&pReader->status.merger, piRow, piSchema); - code = doMergeRowsInBuf(&pBlockScanInfo->iiter, pBlockScanInfo->uid, ik.ts, pBlockScanInfo->delSkyline, pReader); - if (code != TSDB_CODE_SUCCESS) { - return code; - } - } - - int32_t code = tsdbRowMergerGetRow(pMerger, pTSRow); - tsdbRowMergerClear(pMerger); - return code; -} - -int32_t tsdbGetNextRowInMem(STableBlockScanInfo* pBlockScanInfo, STsdbReader* pReader, TSDBROW* pResRow, int64_t endKey, - bool* freeTSRow) { - TSDBROW* pRow = getValidMemRow(&pBlockScanInfo->iter, pBlockScanInfo->delSkyline, pReader); - TSDBROW* piRow = getValidMemRow(&pBlockScanInfo->iiter, pBlockScanInfo->delSkyline, pReader); - SArray* pDelList = pBlockScanInfo->delSkyline; - uint64_t uid = pBlockScanInfo->uid; - - // todo refactor - bool asc = ASCENDING_TRAVERSE(pReader->order); - if (pBlockScanInfo->iter.hasVal) { - TSDBKEY k = TSDBROW_KEY(pRow); - if ((k.ts >= endKey && asc) || (k.ts <= endKey && !asc)) { - pRow = NULL; - } - } - - if (pBlockScanInfo->iiter.hasVal) { - TSDBKEY k = TSDBROW_KEY(piRow); - if ((k.ts >= endKey && asc) || (k.ts <= endKey && !asc)) { - piRow = NULL; - } - } - - if (pBlockScanInfo->iter.hasVal && pBlockScanInfo->iiter.hasVal && pRow != NULL && piRow != NULL) { - TSDBKEY k = TSDBROW_KEY(pRow); - TSDBKEY ik = TSDBROW_KEY(piRow); - - int32_t code = TSDB_CODE_SUCCESS; - if (ik.ts != k.ts) { - if (((ik.ts < k.ts) && asc) || ((ik.ts > k.ts) && (!asc))) { // ik.ts < k.ts - code = doMergeMemTableMultiRows(piRow, uid, &pBlockScanInfo->iiter, pDelList, pResRow, pReader, freeTSRow); - } else if (((k.ts < ik.ts) && asc) || ((k.ts > ik.ts) && (!asc))) { - code = doMergeMemTableMultiRows(pRow, uid, &pBlockScanInfo->iter, pDelList, pResRow, pReader, freeTSRow); - } - } else { // ik.ts == k.ts - *freeTSRow = true; - pResRow->type = TSDBROW_ROW_FMT; - code = doMergeMemIMemRows(pRow, piRow, pBlockScanInfo, pReader, &pResRow->pTSRow); - if (code != TSDB_CODE_SUCCESS) { - return code; - } - } - - return code; - } - - if (pBlockScanInfo->iter.hasVal && pRow != NULL) { - return doMergeMemTableMultiRows(pRow, pBlockScanInfo->uid, &pBlockScanInfo->iter, pDelList, pResRow, pReader, - freeTSRow); - } - - if (pBlockScanInfo->iiter.hasVal && piRow != NULL) { - return doMergeMemTableMultiRows(piRow, uid, &pBlockScanInfo->iiter, pDelList, pResRow, pReader, freeTSRow); - } - - return TSDB_CODE_SUCCESS; -} - -int32_t doAppendRowFromTSRow(SSDataBlock* pBlock, STsdbReader* pReader, SRow* pTSRow, STableBlockScanInfo* pScanInfo) { - int32_t outputRowIndex = pBlock->info.rows; - int64_t uid = pScanInfo->uid; - int32_t code = TSDB_CODE_SUCCESS; - - SBlockLoadSuppInfo* pSupInfo = &pReader->suppInfo; - STSchema* pSchema = doGetSchemaForTSRow(pTSRow->sver, pReader, uid); - if (pSchema == NULL) { - return terrno; - } - - SColVal colVal = {0}; - int32_t i = 0, j = 0; - - if (pSupInfo->colId[i] == PRIMARYKEY_TIMESTAMP_COL_ID) { - SColumnInfoData* pColData = taosArrayGet(pBlock->pDataBlock, pSupInfo->slotId[i]); - ((int64_t*)pColData->pData)[outputRowIndex] = pTSRow->ts; - i += 1; - } - - while (i < pSupInfo->numOfCols && j < pSchema->numOfCols) { - col_id_t colId = pSupInfo->colId[i]; - - if (colId == pSchema->columns[j].colId) { - SColumnInfoData* pColInfoData = taosArrayGet(pBlock->pDataBlock, pSupInfo->slotId[i]); - - tRowGet(pTSRow, pSchema, j, &colVal); - code = doCopyColVal(pColInfoData, outputRowIndex, i, &colVal, pSupInfo); - if (code) { - return code; - } - i += 1; - j += 1; - } else if (colId < pSchema->columns[j].colId) { - SColumnInfoData* pColInfoData = taosArrayGet(pBlock->pDataBlock, pSupInfo->slotId[i]); - - colDataSetNULL(pColInfoData, outputRowIndex); - i += 1; - } else if (colId > pSchema->columns[j].colId) { - j += 1; - } - } - - // set null value since current column does not exist in the "pSchema" - while (i < pSupInfo->numOfCols) { - SColumnInfoData* pColInfoData = taosArrayGet(pBlock->pDataBlock, pSupInfo->slotId[i]); - colDataSetNULL(pColInfoData, outputRowIndex); - i += 1; - } - - pBlock->info.dataLoad = 1; - pBlock->info.rows += 1; - pScanInfo->lastKey = pTSRow->ts; - return TSDB_CODE_SUCCESS; -} - -int32_t doAppendRowFromFileBlock(SSDataBlock* pResBlock, STsdbReader* pReader, SBlockData* pBlockData, - int32_t rowIndex) { - int32_t i = 0, j = 0; - int32_t outputRowIndex = pResBlock->info.rows; - int32_t code = TSDB_CODE_SUCCESS; - - SBlockLoadSuppInfo* pSupInfo = &pReader->suppInfo; - ((int64_t*)pReader->status.pPrimaryTsCol->pData)[outputRowIndex] = pBlockData->aTSKEY[rowIndex]; - i += 1; - - SColVal cv = {0}; - int32_t numOfInputCols = pBlockData->nColData; - int32_t numOfOutputCols = pSupInfo->numOfCols; - - while (i < numOfOutputCols && j < numOfInputCols) { - SColData* pData = tBlockDataGetColDataByIdx(pBlockData, j); - if (pData->cid < pSupInfo->colId[i]) { - j += 1; - continue; - } - - SColumnInfoData* pCol = TARRAY_GET_ELEM(pResBlock->pDataBlock, pSupInfo->slotId[i]); - if (pData->cid == pSupInfo->colId[i]) { - tColDataGetValue(pData, rowIndex, &cv); - code = doCopyColVal(pCol, outputRowIndex, i, &cv, pSupInfo); - if (code) { - return code; - } - j += 1; - } else if (pData->cid > pCol->info.colId) { - // the specified column does not exist in file block, fill with null data - colDataSetNULL(pCol, outputRowIndex); - } - - i += 1; - } - - while (i < numOfOutputCols) { - SColumnInfoData* pCol = taosArrayGet(pResBlock->pDataBlock, pSupInfo->slotId[i]); - colDataSetNULL(pCol, outputRowIndex); - i += 1; - } - - pResBlock->info.dataLoad = 1; - pResBlock->info.rows += 1; - return TSDB_CODE_SUCCESS; -} - -int32_t buildDataBlockFromBufImpl(STableBlockScanInfo* pBlockScanInfo, int64_t endKey, int32_t capacity, - STsdbReader* pReader) { - SSDataBlock* pBlock = pReader->resBlockInfo.pResBlock; - int32_t code = TSDB_CODE_SUCCESS; - - do { - // SRow* pTSRow = NULL; - TSDBROW row = {.type = -1}; - bool freeTSRow = false; - tsdbGetNextRowInMem(pBlockScanInfo, pReader, &row, endKey, &freeTSRow); - if (row.type == -1) { - break; - } - - if (row.type == TSDBROW_ROW_FMT) { - code = doAppendRowFromTSRow(pBlock, pReader, row.pTSRow, pBlockScanInfo); - - if (freeTSRow) { - taosMemoryFree(row.pTSRow); - } - - if (code) { - return code; - } - } else { - code = doAppendRowFromFileBlock(pBlock, pReader, row.pBlockData, row.iRow); - if (code) { - break; - } - } - - // no data in buffer, return immediately - if (!(pBlockScanInfo->iter.hasVal || pBlockScanInfo->iiter.hasVal)) { - break; - } - - if (pBlock->info.rows >= capacity) { - break; - } - } while (1); - - return code; -} - -// TODO refactor: with createDataBlockScanInfo -int32_t tsdbSetTableList(STsdbReader* pReader, const void* pTableList, int32_t num) { - int32_t size = tSimpleHashGetSize(pReader->status.pTableMap); - - STableBlockScanInfo** p = NULL; - int32_t iter = 0; - - while ((p = tSimpleHashIterate(pReader->status.pTableMap, p, &iter)) != NULL) { - clearBlockScanInfo(*p); - } - - if (size < num) { - int32_t code = ensureBlockScanInfoBuf(&pReader->blockInfoBuf, num); - if (code) { - return code; - } - - char* p1 = taosMemoryRealloc(pReader->status.uidList.tableUidList, sizeof(uint64_t) * num); - if (p1 == NULL) { - return TSDB_CODE_OUT_OF_MEMORY; - } - - pReader->status.uidList.tableUidList = (uint64_t*)p1; - } - - tSimpleHashClear(pReader->status.pTableMap); - STableUidList* pUidList = &pReader->status.uidList; - pUidList->currentIndex = 0; - - STableKeyInfo* pList = (STableKeyInfo*)pTableList; - for (int32_t i = 0; i < num; ++i) { - STableBlockScanInfo* pInfo = getPosInBlockInfoBuf(&pReader->blockInfoBuf, i); - pInfo->uid = pList[i].uid; - pUidList->tableUidList[i] = pList[i].uid; - - // todo extract method - if (ASCENDING_TRAVERSE(pReader->order)) { - int64_t skey = pReader->window.skey; - pInfo->lastKey = (skey > INT64_MIN) ? (skey - 1) : skey; - pInfo->lastKeyInStt = skey; - } else { - int64_t ekey = pReader->window.ekey; - pInfo->lastKey = (ekey < INT64_MAX) ? (ekey + 1) : ekey; - pInfo->lastKeyInStt = ekey; - } - - tSimpleHashPut(pReader->status.pTableMap, &pInfo->uid, sizeof(uint64_t), &pInfo, POINTER_BYTES); - } - - return TDB_CODE_SUCCESS; -} - -void* tsdbGetIdx(SMeta* pMeta) { - if (pMeta == NULL) { - return NULL; - } - return metaGetIdx(pMeta); -} - -void* tsdbGetIvtIdx(SMeta* pMeta) { - if (pMeta == NULL) { - return NULL; - } - return metaGetIvtIdx(pMeta); -} - -uint64_t tsdbGetReaderMaxVersion(STsdbReader* pReader) { return pReader->verRange.maxVer; } - -static int32_t doOpenReaderImpl(STsdbReader* pReader) { - SReaderStatus* pStatus = &pReader->status; - SDataBlockIter* pBlockIter = &pStatus->blockIter; - - initFilesetIterator(&pStatus->fileIter, pReader->pReadSnap->fs.aDFileSet, pReader); - resetDataBlockIterator(&pStatus->blockIter, pReader->order); - - int32_t code = TSDB_CODE_SUCCESS; - if (pStatus->fileIter.numOfFiles == 0) { - pStatus->loadFromFile = false; - } else if (READ_MODE_COUNT_ONLY == pReader->readMode) { - // DO NOTHING - } else { - code = initForFirstBlockInFile(pReader, pBlockIter); - } - - if (!pStatus->loadFromFile) { - resetTableListIndex(pStatus); - } - - return code; -} - -static void freeSchemaFunc(void* param) { - void** p = (void**)param; - taosMemoryFreeClear(*p); -} - -// ====================================== EXPOSED APIs ====================================== -int32_t tsdbReaderOpen(void* pVnode, SQueryTableDataCond* pCond, void* pTableList, int32_t numOfTables, - SSDataBlock* pResBlock, void** ppReader, const char* idstr, bool countOnly, - SHashObj** pIgnoreTables) { - STimeWindow window = pCond->twindows; - SVnodeCfg* pConf = &(((SVnode*)pVnode)->config); - - int32_t capacity = pConf->tsdbCfg.maxRows; - if (pResBlock != NULL) { - blockDataEnsureCapacity(pResBlock, capacity); - } - - int32_t code = tsdbReaderCreate(pVnode, pCond, ppReader, capacity, pResBlock, idstr); - if (code != TSDB_CODE_SUCCESS) { - goto _err; - } - - // check for query time window - STsdbReader* pReader = *ppReader; - if (isEmptyQueryTimeWindow(&pReader->window) && pCond->type == TIMEWINDOW_RANGE_CONTAINED) { - tsdbDebug("%p query window not overlaps with the data set, no result returned, %s", pReader, pReader->idStr); - return TSDB_CODE_SUCCESS; - } - - if (pCond->type == TIMEWINDOW_RANGE_EXTERNAL) { - // update the SQueryTableDataCond to create inner reader - int32_t order = pCond->order; - if (order == TSDB_ORDER_ASC) { - pCond->twindows.ekey = window.skey - 1; - pCond->twindows.skey = INT64_MIN; - pCond->order = TSDB_ORDER_DESC; - } else { - pCond->twindows.skey = window.ekey + 1; - pCond->twindows.ekey = INT64_MAX; - pCond->order = TSDB_ORDER_ASC; - } - - // here we only need one more row, so the capacity is set to be ONE. - code = tsdbReaderCreate(pVnode, pCond, (void**)&((STsdbReader*)pReader)->innerReader[0], 1, pResBlock, idstr); - if (code != TSDB_CODE_SUCCESS) { - goto _err; - } - - if (order == TSDB_ORDER_ASC) { - pCond->twindows.skey = window.ekey + 1; - pCond->twindows.ekey = INT64_MAX; - } else { - pCond->twindows.skey = INT64_MIN; - pCond->twindows.ekey = window.ekey - 1; - } - pCond->order = order; - - code = tsdbReaderCreate(pVnode, pCond, (void**)&((STsdbReader*)pReader)->innerReader[1], 1, pResBlock, idstr); - if (code != TSDB_CODE_SUCCESS) { - goto _err; - } - } - - // NOTE: the endVersion in pCond is the data version not schema version, so pCond->endVersion is not correct here. - // no valid error code set in metaGetTbTSchema, so let's set the error code here. - // we should proceed in case of tmq processing. - if (pCond->suid != 0) { - pReader->pSchema = metaGetTbTSchema(pReader->pTsdb->pVnode->pMeta, pReader->suid, -1, 1); - if (pReader->pSchema == NULL) { - tsdbError("failed to get table schema, suid:%" PRIu64 ", ver:-1, %s", pReader->suid, pReader->idStr); - } - } else if (numOfTables > 0) { - STableKeyInfo* pKey = pTableList; - pReader->pSchema = metaGetTbTSchema(pReader->pTsdb->pVnode->pMeta, pKey->uid, -1, 1); - if (pReader->pSchema == NULL) { - tsdbError("failed to get table schema, uid:%" PRIu64 ", ver:-1, %s", pKey->uid, pReader->idStr); - } - } - - if (pReader->pSchema != NULL) { - tsdbRowMergerInit(&pReader->status.merger, pReader->pSchema); - } - - pReader->pSchemaMap = tSimpleHashInit(8, taosFastHash); - if (pReader->pSchemaMap == NULL) { - tsdbError("failed init schema hash for reader %s", pReader->idStr); - code = TSDB_CODE_OUT_OF_MEMORY; - goto _err; - } - - tSimpleHashSetFreeFp(pReader->pSchemaMap, freeSchemaFunc); - if (pReader->pSchema != NULL) { - code = updateBlockSMAInfo(pReader->pSchema, &pReader->suppInfo); - if (code != TSDB_CODE_SUCCESS) { - goto _err; - } - } - - STsdbReader* p = (pReader->innerReader[0] != NULL) ? pReader->innerReader[0] : pReader; - pReader->status.pTableMap = - createDataBlockScanInfo(p, &pReader->blockInfoBuf, pTableList, &pReader->status.uidList, numOfTables); - if (pReader->status.pTableMap == NULL) { - *ppReader = NULL; - code = TSDB_CODE_OUT_OF_MEMORY; - goto _err; - } - - pReader->status.pLDataIter = taosMemoryCalloc(pConf->sttTrigger, sizeof(SLDataIter)); - if (pReader->status.pLDataIter == NULL) { - terrno = TSDB_CODE_OUT_OF_MEMORY; - goto _err; - } - - pReader->flag = READER_STATUS_SUSPEND; - - if (countOnly) { - pReader->readMode = READ_MODE_COUNT_ONLY; - } - - pReader->pIgnoreTables = pIgnoreTables; - - tsdbDebug("%p total numOfTable:%d, window:%" PRId64 " - %" PRId64 ", verRange:%" PRId64 " - %" PRId64 - " in this query %s", - pReader, numOfTables, pReader->window.skey, pReader->window.ekey, pReader->verRange.minVer, - pReader->verRange.maxVer, pReader->idStr); - - return code; - -_err: - tsdbError("failed to create data reader, code:%s %s", tstrerror(code), idstr); - tsdbReaderClose(*ppReader); - *ppReader = NULL; // reset the pointer value. - return code; -} - -static void clearSharedPtr(STsdbReader* p) { - p->status.pLDataIter = NULL; - p->status.pTableMap = NULL; - p->status.uidList.tableUidList = NULL; - p->pReadSnap = NULL; - p->pSchema = NULL; - p->pSchemaMap = NULL; -} - -static void setSharedPtr(STsdbReader* pDst, const STsdbReader* pSrc) { - pDst->status.pTableMap = pSrc->status.pTableMap; - pDst->status.pLDataIter = pSrc->status.pLDataIter; - pDst->status.uidList = pSrc->status.uidList; - pDst->pSchema = pSrc->pSchema; - pDst->pSchemaMap = pSrc->pSchemaMap; - pDst->pReadSnap = pSrc->pReadSnap; - - if (pDst->pSchema) { - tsdbRowMergerInit(&pDst->status.merger, pDst->pSchema); - } -} - -void tsdbReaderClose(STsdbReader* pReader) { - if (pReader == NULL) { - return; - } - - tsdbAcquireReader(pReader); - - { - if (pReader->innerReader[0] != NULL || pReader->innerReader[1] != NULL) { - STsdbReader* p = pReader->innerReader[0]; - clearSharedPtr(p); - - p = pReader->innerReader[1]; - clearSharedPtr(p); - - tsdbReaderClose(pReader->innerReader[0]); - tsdbReaderClose(pReader->innerReader[1]); - } - } - - SBlockLoadSuppInfo* pSupInfo = &pReader->suppInfo; - - taosArrayDestroy(pSupInfo->pColAgg); - for (int32_t i = 0; i < pSupInfo->numOfCols; ++i) { - if (pSupInfo->buildBuf[i] != NULL) { - taosMemoryFreeClear(pSupInfo->buildBuf[i]); - } - } - - if (pReader->resBlockInfo.freeBlock) { - pReader->resBlockInfo.pResBlock = blockDataDestroy(pReader->resBlockInfo.pResBlock); - } - - taosMemoryFree(pSupInfo->colId); - tBlockDataDestroy(&pReader->status.fileBlockData); - cleanupDataBlockIterator(&pReader->status.blockIter); - - size_t numOfTables = tSimpleHashGetSize(pReader->status.pTableMap); - if (pReader->status.pTableMap != NULL) { - destroyAllBlockScanInfo(pReader->status.pTableMap); - clearBlockScanInfoBuf(&pReader->blockInfoBuf); - } - - if (pReader->pFileReader != NULL) { - tsdbDataFReaderClose(&pReader->pFileReader); - } - - if (pReader->pDelFReader != NULL) { - tsdbDelFReaderClose(&pReader->pDelFReader); - } - - if (pReader->pDelIdx != NULL) { - taosArrayDestroy(pReader->pDelIdx); - pReader->pDelIdx = NULL; - } - - qTrace("tsdb/reader-close: %p, untake snapshot", pReader); - tsdbUntakeReadSnap(pReader, pReader->pReadSnap, true); - pReader->pReadSnap = NULL; - - tsdbReleaseReader(pReader); - - tsdbUninitReaderLock(pReader); - - taosMemoryFreeClear(pReader->status.pLDataIter); - taosMemoryFreeClear(pReader->status.uidList.tableUidList); - SIOCostSummary* pCost = &pReader->cost; - - SFilesetIter* pFilesetIter = &pReader->status.fileIter; - if (pFilesetIter->pLastBlockReader != NULL) { - SLastBlockReader* pLReader = pFilesetIter->pLastBlockReader; - tMergeTreeClose(&pLReader->mergeTree); - - getSttBlockLoadInfo(pLReader->pInfo, &pCost->sttCost); - - pLReader->pInfo = destroyLastBlockLoadInfo(pLReader->pInfo); - taosMemoryFree(pLReader); - } - - tsdbDebug( - "%p :io-cost summary: head-file:%" PRIu64 ", head-file time:%.2f ms, SMA:%" PRId64 - " SMA-time:%.2f ms, fileBlocks:%" PRId64 - ", fileBlocks-load-time:%.2f ms, " - "build in-memory-block-time:%.2f ms, lastBlocks:%" PRId64 ", lastBlocks-time:%.2f ms, composed-blocks:%" PRId64 - ", composed-blocks-time:%.2fms, STableBlockScanInfo size:%.2f Kb, createTime:%.2f ms,initDelSkylineIterTime:%.2f " - "ms, %s", - pReader, pCost->headFileLoad, pCost->headFileLoadTime, pCost->smaDataLoad, pCost->smaLoadTime, pCost->numOfBlocks, - pCost->blockLoadTime, pCost->buildmemBlock, pCost->sttBlockLoad, pCost->sttBlockLoadTime, pCost->composedBlocks, - pCost->buildComposedBlockTime, numOfTables * sizeof(STableBlockScanInfo) / 1000.0, pCost->createScanInfoList, - pCost->initDelSkylineIterTime, pReader->idStr); - - taosMemoryFree(pReader->idStr); - - tsdbRowMergerCleanup(&pReader->status.merger); - taosMemoryFree(pReader->pSchema); - - tSimpleHashCleanup(pReader->pSchemaMap); - taosMemoryFreeClear(pReader); -} - -int32_t tsdbReaderSuspend(STsdbReader* pReader) { - int32_t code = 0; - - // save reader's base state & reset top state to be reconstructed from base state - SReaderStatus* pStatus = &pReader->status; - STableBlockScanInfo* pBlockScanInfo = NULL; - - if (pStatus->loadFromFile) { - SFileDataBlockInfo* pBlockInfo = getCurrentBlockInfo(&pReader->status.blockIter); - if (pBlockInfo != NULL) { - pBlockScanInfo = getTableBlockScanInfo(pStatus->pTableMap, pBlockInfo->uid, pReader->idStr); - if (pBlockScanInfo == NULL) { - goto _err; - } - } else { - pBlockScanInfo = *pStatus->pTableIter; - } - - tsdbDataFReaderClose(&pReader->pFileReader); - - // resetDataBlockScanInfo excluding lastKey - STableBlockScanInfo** p = NULL; - int32_t iter = 0; - - while ((p = tSimpleHashIterate(pStatus->pTableMap, p, &iter)) != NULL) { - STableBlockScanInfo* pInfo = *(STableBlockScanInfo**)p; - - pInfo->iterInit = false; - pInfo->iter.hasVal = false; - pInfo->iiter.hasVal = false; - - if (pInfo->iter.iter != NULL) { - pInfo->iter.iter = tsdbTbDataIterDestroy(pInfo->iter.iter); - } - - if (pInfo->iiter.iter != NULL) { - pInfo->iiter.iter = tsdbTbDataIterDestroy(pInfo->iiter.iter); - } - - pInfo->delSkyline = taosArrayDestroy(pInfo->delSkyline); - } - } else { - // resetDataBlockScanInfo excluding lastKey - STableBlockScanInfo** p = NULL; - int32_t iter = 0; - - while ((p = tSimpleHashIterate(pStatus->pTableMap, p, &iter)) != NULL) { - STableBlockScanInfo* pInfo = *(STableBlockScanInfo**)p; - - pInfo->iterInit = false; - pInfo->iter.hasVal = false; - pInfo->iiter.hasVal = false; - - if (pInfo->iter.iter != NULL) { - pInfo->iter.iter = tsdbTbDataIterDestroy(pInfo->iter.iter); - } - - if (pInfo->iiter.iter != NULL) { - pInfo->iiter.iter = tsdbTbDataIterDestroy(pInfo->iiter.iter); - } - - pInfo->delSkyline = taosArrayDestroy(pInfo->delSkyline); - } - - pBlockScanInfo = pStatus->pTableIter == NULL ? NULL : *pStatus->pTableIter; - if (pBlockScanInfo) { - // save lastKey to restore memory iterator - STimeWindow w = pReader->resBlockInfo.pResBlock->info.window; - pBlockScanInfo->lastKey = ASCENDING_TRAVERSE(pReader->order) ? w.ekey : w.skey; - - // reset current current table's data block scan info, - pBlockScanInfo->iterInit = false; - pBlockScanInfo->iter.hasVal = false; - pBlockScanInfo->iiter.hasVal = false; - if (pBlockScanInfo->iter.iter != NULL) { - pBlockScanInfo->iter.iter = tsdbTbDataIterDestroy(pBlockScanInfo->iter.iter); - } - - if (pBlockScanInfo->iiter.iter != NULL) { - pBlockScanInfo->iiter.iter = tsdbTbDataIterDestroy(pBlockScanInfo->iiter.iter); - } - - pBlockScanInfo->pBlockList = taosArrayDestroy(pBlockScanInfo->pBlockList); - tMapDataClear(&pBlockScanInfo->mapData); - // TODO: keep skyline for reuse - pBlockScanInfo->delSkyline = taosArrayDestroy(pBlockScanInfo->delSkyline); - } - } - - tsdbUntakeReadSnap(pReader, pReader->pReadSnap, false); - pReader->pReadSnap = NULL; - pReader->flag = READER_STATUS_SUSPEND; - - tsdbDebug("reader: %p suspended uid %" PRIu64 " in this query %s", pReader, pBlockScanInfo ? pBlockScanInfo->uid : 0, - pReader->idStr); - return code; - -_err: - tsdbError("failed to suspend data reader, code:%s %s", tstrerror(code), pReader->idStr); - return code; -} - -static int32_t tsdbSetQueryReseek(void* pQHandle) { - int32_t code = 0; - STsdbReader* pReader = pQHandle; - - code = tsdbTryAcquireReader(pReader); - if (code == 0) { - if (pReader->flag == READER_STATUS_SUSPEND) { - tsdbReleaseReader(pReader); - return code; - } - - tsdbReaderSuspend(pReader); - - tsdbReleaseReader(pReader); - - return code; - } else if (code == EBUSY) { - return TSDB_CODE_VND_QUERY_BUSY; - } else { - terrno = TAOS_SYSTEM_ERROR(code); - return TSDB_CODE_FAILED; - } -} - -int32_t tsdbReaderResume(STsdbReader* pReader) { - int32_t code = 0; - - STableBlockScanInfo** pBlockScanInfo = pReader->status.pTableIter; - - // restore reader's state - // task snapshot - int32_t numOfTables = tSimpleHashGetSize(pReader->status.pTableMap); - if (numOfTables > 0) { - qTrace("tsdb/reader: %p, take snapshot", pReader); - code = tsdbTakeReadSnap(pReader, tsdbSetQueryReseek, &pReader->pReadSnap); - if (code != TSDB_CODE_SUCCESS) { - goto _err; - } - - if (pReader->type == TIMEWINDOW_RANGE_CONTAINED) { - code = doOpenReaderImpl(pReader); - if (code != TSDB_CODE_SUCCESS) { - return code; - } - } else { - STsdbReader* pPrevReader = pReader->innerReader[0]; - STsdbReader* pNextReader = pReader->innerReader[1]; - - // we need only one row - pPrevReader->resBlockInfo.capacity = 1; - setSharedPtr(pPrevReader, pReader); - - pNextReader->resBlockInfo.capacity = 1; - setSharedPtr(pNextReader, pReader); - - code = doOpenReaderImpl(pPrevReader); - if (code != TSDB_CODE_SUCCESS) { - return code; - } - } - } - - pReader->flag = READER_STATUS_NORMAL; - tsdbDebug("reader: %p resumed uid %" PRIu64 ", numOfTable:%" PRId32 ", in this query %s", pReader, - pBlockScanInfo ? (*pBlockScanInfo)->uid : 0, numOfTables, pReader->idStr); - return code; - -_err: - tsdbError("failed to resume data reader, code:%s %s", tstrerror(code), pReader->idStr); - return code; -} - -static bool tsdbReadRowsCountOnly(STsdbReader* pReader) { - int32_t code = TSDB_CODE_SUCCESS; - SSDataBlock* pBlock = pReader->resBlockInfo.pResBlock; - - if (pReader->status.loadFromFile == false) { - return false; - } - - code = readRowsCountFromFiles(pReader); - if (code != TSDB_CODE_SUCCESS) { - return false; - } - - code = readRowsCountFromMem(pReader); - if (code != TSDB_CODE_SUCCESS) { - return false; - } - - pBlock->info.rows = pReader->rowsNum; - pBlock->info.id.uid = 0; - pBlock->info.dataLoad = 0; - - pReader->rowsNum = 0; - - return pBlock->info.rows > 0; -} - -static int32_t doTsdbNextDataBlock(STsdbReader* pReader, bool* hasNext) { - int32_t code = TSDB_CODE_SUCCESS; - - // cleanup the data that belongs to the previous data block - SSDataBlock* pBlock = pReader->resBlockInfo.pResBlock; - blockDataCleanup(pBlock); - - *hasNext = false; - - SReaderStatus* pStatus = &pReader->status; - if (tSimpleHashGetSize(pStatus->pTableMap) == 0) { - return code; - } - - if (READ_MODE_COUNT_ONLY == pReader->readMode) { - return tsdbReadRowsCountOnly(pReader); - } - - if (pStatus->loadFromFile) { - code = buildBlockFromFiles(pReader); - if (code != TSDB_CODE_SUCCESS) { - return code; - } - - if (pBlock->info.rows <= 0) { - resetTableListIndex(&pReader->status); - code = buildBlockFromBufferSequentially(pReader); - } - } else { // no data in files, let's try the buffer - code = buildBlockFromBufferSequentially(pReader); - } - - *hasNext = pBlock->info.rows > 0; - - return code; -} - -int32_t tsdbNextDataBlock(STsdbReader* pReader, bool* hasNext) { - int32_t code = TSDB_CODE_SUCCESS; - - *hasNext = false; - - if (isEmptyQueryTimeWindow(&pReader->window) || pReader->step == EXTERNAL_ROWS_NEXT || - pReader->code != TSDB_CODE_SUCCESS) { - return (pReader->code != TSDB_CODE_SUCCESS) ? pReader->code : code; - } - - SReaderStatus* pStatus = &pReader->status; - - code = tsdbAcquireReader(pReader); - qTrace("tsdb/read: %p, take read mutex, code: %d", pReader, code); - - if (pReader->flag == READER_STATUS_SUSPEND) { - code = tsdbReaderResume(pReader); - if (code != TSDB_CODE_SUCCESS) { - tsdbReleaseReader(pReader); - return code; - } - } - - if (pReader->innerReader[0] != NULL && pReader->step == 0) { - code = doTsdbNextDataBlock(pReader->innerReader[0], hasNext); - if (code) { - tsdbReleaseReader(pReader); - return code; - } - - pReader->step = EXTERNAL_ROWS_PREV; - if (*hasNext) { - pStatus = &pReader->innerReader[0]->status; - if (pStatus->composedDataBlock) { - qTrace("tsdb/read: %p, unlock read mutex", pReader); - tsdbReleaseReader(pReader); - } - - return code; - } - } - - if (pReader->step == EXTERNAL_ROWS_PREV) { - // prepare for the main scan - code = doOpenReaderImpl(pReader); - int32_t step = 1; - resetAllDataBlockScanInfo(pReader->status.pTableMap, pReader->innerReader[0]->window.ekey, step); - - if (code != TSDB_CODE_SUCCESS) { - return code; - } - - pReader->step = EXTERNAL_ROWS_MAIN; - } - - code = doTsdbNextDataBlock(pReader, hasNext); - if (code != TSDB_CODE_SUCCESS) { - tsdbReleaseReader(pReader); - return code; - } - - if (*hasNext) { - if (pStatus->composedDataBlock) { - qTrace("tsdb/read: %p, unlock read mutex", pReader); - tsdbReleaseReader(pReader); - } - - return code; - } - - if (pReader->step == EXTERNAL_ROWS_MAIN && pReader->innerReader[1] != NULL) { - // prepare for the next row scan - int32_t step = -1; - code = doOpenReaderImpl(pReader->innerReader[1]); - resetAllDataBlockScanInfo(pReader->innerReader[1]->status.pTableMap, pReader->window.ekey, step); - if (code != TSDB_CODE_SUCCESS) { - return code; - } - - code = doTsdbNextDataBlock(pReader->innerReader[1], hasNext); - if (code != TSDB_CODE_SUCCESS) { - tsdbReleaseReader(pReader); - return code; - } - - pReader->step = EXTERNAL_ROWS_NEXT; - if (*hasNext) { - pStatus = &pReader->innerReader[1]->status; - if (pStatus->composedDataBlock) { - qTrace("tsdb/read: %p, unlock read mutex", pReader); - tsdbReleaseReader(pReader); - } - - return code; - } - } - - qTrace("tsdb/read: %p, unlock read mutex", pReader); - tsdbReleaseReader(pReader); - - return code; -} - -static bool doFillNullColSMA(SBlockLoadSuppInfo* pSup, int32_t numOfRows, int32_t numOfCols, SColumnDataAgg* pTsAgg) { - bool hasNullSMA = false; - // do fill all null column value SMA info - int32_t i = 0, j = 0; - int32_t size = (int32_t)taosArrayGetSize(pSup->pColAgg); - taosArrayInsert(pSup->pColAgg, 0, pTsAgg); - size++; - - while (j < numOfCols && i < size) { - SColumnDataAgg* pAgg = taosArrayGet(pSup->pColAgg, i); - if (pAgg->colId == pSup->colId[j]) { - i += 1; - j += 1; - } else if (pAgg->colId < pSup->colId[j]) { - i += 1; - } else if (pSup->colId[j] < pAgg->colId) { - if (pSup->colId[j] != PRIMARYKEY_TIMESTAMP_COL_ID) { - SColumnDataAgg nullColAgg = {.colId = pSup->colId[j], .numOfNull = numOfRows}; - taosArrayInsert(pSup->pColAgg, i, &nullColAgg); - i += 1; - size++; - hasNullSMA = true; - } - j += 1; - } - } - - while (j < numOfCols) { - if (pSup->colId[j] != PRIMARYKEY_TIMESTAMP_COL_ID) { - SColumnDataAgg nullColAgg = {.colId = pSup->colId[j], .numOfNull = numOfRows}; - taosArrayInsert(pSup->pColAgg, i, &nullColAgg); - i += 1; - hasNullSMA = true; - } - j++; - } - - return hasNullSMA; -} - -int32_t tsdbRetrieveDatablockSMA(STsdbReader* pReader, SSDataBlock* pDataBlock, bool* allHave, bool* hasNullSMA) { - SColumnDataAgg*** pBlockSMA = &pDataBlock->pBlockAgg; - - int32_t code = 0; - *allHave = false; - *pBlockSMA = NULL; - - if (pReader->type == TIMEWINDOW_RANGE_EXTERNAL) { - return TSDB_CODE_SUCCESS; - } - - // there is no statistics data for composed block - if (pReader->status.composedDataBlock || (!pReader->suppInfo.smaValid)) { - return TSDB_CODE_SUCCESS; - } - - SFileDataBlockInfo* pFBlock = getCurrentBlockInfo(&pReader->status.blockIter); - SBlockLoadSuppInfo* pSup = &pReader->suppInfo; - - if (pReader->resBlockInfo.pResBlock->info.id.uid != pFBlock->uid) { - return TSDB_CODE_SUCCESS; - } - - int64_t st = taosGetTimestampUs(); - - SDataBlk* pBlock = getCurrentBlock(&pReader->status.blockIter); - if (tDataBlkHasSma(pBlock)) { - code = tsdbReadBlockSma(pReader->pFileReader, pBlock, pSup->pColAgg); - if (code != TSDB_CODE_SUCCESS) { - tsdbDebug("vgId:%d, failed to load block SMA for uid %" PRIu64 ", code:%s, %s", 0, pFBlock->uid, tstrerror(code), - pReader->idStr); - return code; - } - } else { - *pBlockSMA = NULL; - return TSDB_CODE_SUCCESS; - } - - *allHave = true; - - // always load the first primary timestamp column data - SColumnDataAgg* pTsAgg = &pSup->tsColAgg; - - pTsAgg->numOfNull = 0; - pTsAgg->colId = PRIMARYKEY_TIMESTAMP_COL_ID; - pTsAgg->min = pReader->resBlockInfo.pResBlock->info.window.skey; - pTsAgg->max = pReader->resBlockInfo.pResBlock->info.window.ekey; - - // update the number of NULL data rows - size_t numOfCols = pSup->numOfCols; - - // ensure capacity - if (pDataBlock->pDataBlock) { - size_t colsNum = taosArrayGetSize(pDataBlock->pDataBlock); - taosArrayEnsureCap(pSup->pColAgg, colsNum); - } - - SSDataBlock* pResBlock = pReader->resBlockInfo.pResBlock; - if (pResBlock->pBlockAgg == NULL) { - size_t num = taosArrayGetSize(pResBlock->pDataBlock); - pResBlock->pBlockAgg = taosMemoryCalloc(num, POINTER_BYTES); - } - - // do fill all null column value SMA info - if (doFillNullColSMA(pSup, pBlock->nRow, numOfCols, pTsAgg)) { - *hasNullSMA = true; - return TSDB_CODE_SUCCESS; - } - size_t size = taosArrayGetSize(pSup->pColAgg); - - int32_t i = 0, j = 0; - while (j < numOfCols && i < size) { - SColumnDataAgg* pAgg = taosArrayGet(pSup->pColAgg, i); - if (pAgg->colId == pSup->colId[j]) { - pResBlock->pBlockAgg[pSup->slotId[j]] = pAgg; - i += 1; - j += 1; - } else if (pAgg->colId < pSup->colId[j]) { - i += 1; - } else if (pSup->colId[j] < pAgg->colId) { - pResBlock->pBlockAgg[pSup->slotId[j]] = NULL; - *allHave = false; - j += 1; - } - } - - *pBlockSMA = pResBlock->pBlockAgg; - pReader->cost.smaDataLoad += 1; - - double elapsedTime = (taosGetTimestampUs() - st) / 1000.0; - pReader->cost.smaLoadTime += elapsedTime; - - tsdbDebug("vgId:%d, succeed to load block SMA for uid %" PRIu64 ", %s", 0, pFBlock->uid, pReader->idStr); - return code; -} - -STableBlockScanInfo* getTableBlockScanInfo(SSHashObj* pTableMap, uint64_t uid, const char* id) { - STableBlockScanInfo** p = tSimpleHashGet(pTableMap, &uid, sizeof(uid)); - if (p == NULL || *p == NULL) { - terrno = TSDB_CODE_INVALID_PARA; - int32_t size = tSimpleHashGetSize(pTableMap); - tsdbError("failed to locate the uid:%" PRIu64 " in query table uid list, total tables:%d, %s", uid, size, id); - return NULL; - } - - return *p; -} - -static SSDataBlock* doRetrieveDataBlock(STsdbReader* pReader) { - SReaderStatus* pStatus = &pReader->status; - int32_t code = TSDB_CODE_SUCCESS; - SFileDataBlockInfo* pBlockInfo = getCurrentBlockInfo(&pStatus->blockIter); - - if (pReader->code != TSDB_CODE_SUCCESS) { - return NULL; - } - - STableBlockScanInfo* pBlockScanInfo = getTableBlockScanInfo(pStatus->pTableMap, pBlockInfo->uid, pReader->idStr); - if (pBlockScanInfo == NULL) { - return NULL; - } - - code = doLoadFileBlockData(pReader, &pStatus->blockIter, &pStatus->fileBlockData, pBlockScanInfo->uid); - if (code != TSDB_CODE_SUCCESS) { - tBlockDataDestroy(&pStatus->fileBlockData); - terrno = code; - return NULL; - } - - code = copyBlockDataToSDataBlock(pReader); - if (code != TSDB_CODE_SUCCESS) { - tBlockDataDestroy(&pStatus->fileBlockData); - terrno = code; - return NULL; - } - - return pReader->resBlockInfo.pResBlock; -} - -SSDataBlock* tsdbRetrieveDataBlock(STsdbReader* pReader, SArray* pIdList) { - STsdbReader* pTReader = pReader; - if (pReader->type == TIMEWINDOW_RANGE_EXTERNAL) { - if (pReader->step == EXTERNAL_ROWS_PREV) { - pTReader = pReader->innerReader[0]; - } else if (pReader->step == EXTERNAL_ROWS_NEXT) { - pTReader = pReader->innerReader[1]; - } - } - - SReaderStatus* pStatus = &pTReader->status; - if (pStatus->composedDataBlock) { - return pTReader->resBlockInfo.pResBlock; - } - - SSDataBlock* ret = doRetrieveDataBlock(pTReader); - - qTrace("tsdb/read-retrieve: %p, unlock read mutex", pReader); - tsdbReleaseReader(pReader); - - return ret; -} - -int32_t tsdbReaderReset(STsdbReader* pReader, SQueryTableDataCond* pCond) { - int32_t code = TSDB_CODE_SUCCESS; - - qTrace("tsdb/reader-reset: %p, take read mutex", pReader); - tsdbAcquireReader(pReader); - - if (pReader->flag == READER_STATUS_SUSPEND) { - code = tsdbReaderResume(pReader); - if (code != TSDB_CODE_SUCCESS) { - tsdbReleaseReader(pReader); - return code; - } - } - - if (isEmptyQueryTimeWindow(&pReader->window) || pReader->pReadSnap == NULL) { - tsdbDebug("tsdb reader reset return %p, %s", pReader->pReadSnap, pReader->idStr); - tsdbReleaseReader(pReader); - return TSDB_CODE_SUCCESS; - } - - SReaderStatus* pStatus = &pReader->status; - SDataBlockIter* pBlockIter = &pStatus->blockIter; - - pReader->order = pCond->order; - pReader->type = TIMEWINDOW_RANGE_CONTAINED; - pStatus->loadFromFile = true; - pStatus->pTableIter = NULL; - pReader->window = updateQueryTimeWindow(pReader->pTsdb, &pCond->twindows); - - // allocate buffer in order to load data blocks from file - memset(&pReader->suppInfo.tsColAgg, 0, sizeof(SColumnDataAgg)); - - pReader->suppInfo.tsColAgg.colId = PRIMARYKEY_TIMESTAMP_COL_ID; - tsdbDataFReaderClose(&pReader->pFileReader); - - int32_t numOfTables = tSimpleHashGetSize(pStatus->pTableMap); - - initFilesetIterator(&pStatus->fileIter, pReader->pReadSnap->fs.aDFileSet, pReader); - resetDataBlockIterator(pBlockIter, pReader->order); - resetTableListIndex(&pReader->status); - - bool asc = ASCENDING_TRAVERSE(pReader->order); - int32_t step = asc ? 1 : -1; - int64_t ts = asc ? pReader->window.skey - 1 : pReader->window.ekey + 1; - resetAllDataBlockScanInfo(pStatus->pTableMap, ts, step); - - // no data in files, let's try buffer in memory - if (pStatus->fileIter.numOfFiles == 0) { - pStatus->loadFromFile = false; - resetTableListIndex(pStatus); - } else { - code = initForFirstBlockInFile(pReader, pBlockIter); - if (code != TSDB_CODE_SUCCESS) { - tsdbError("%p reset reader failed, numOfTables:%d, query range:%" PRId64 " - %" PRId64 " in query %s", pReader, - numOfTables, pReader->window.skey, pReader->window.ekey, pReader->idStr); - - tsdbReleaseReader(pReader); - return code; - } - } - - tsdbDebug("%p reset reader, suid:%" PRIu64 ", numOfTables:%d, skey:%" PRId64 ", query range:%" PRId64 " - %" PRId64 - " in query %s", - pReader, pReader->suid, numOfTables, pCond->twindows.skey, pReader->window.skey, pReader->window.ekey, - pReader->idStr); - - tsdbReleaseReader(pReader); - - return code; -} - -static int32_t getBucketIndex(int32_t startRow, int32_t bucketRange, int32_t numOfRows, int32_t numOfBucket) { - if (numOfRows < startRow) { - return 0; - } - int32_t bucketIndex = ((numOfRows - startRow) / bucketRange); - if (bucketIndex == numOfBucket) { - bucketIndex -= 1; - } - return bucketIndex; -} - -int32_t tsdbGetFileBlocksDistInfo(STsdbReader* pReader, STableBlockDistInfo* pTableBlockInfo) { - int32_t code = TSDB_CODE_SUCCESS; - pTableBlockInfo->totalSize = 0; - pTableBlockInfo->totalRows = 0; - pTableBlockInfo->numOfVgroups = 1; - - const int32_t numOfBuckets = 20.0; - - // find the start data block in file - tsdbAcquireReader(pReader); - if (pReader->flag == READER_STATUS_SUSPEND) { - code = tsdbReaderResume(pReader); - if (code != TSDB_CODE_SUCCESS) { - tsdbReleaseReader(pReader); - return code; - } - } - SReaderStatus* pStatus = &pReader->status; - - STsdbCfg* pc = &pReader->pTsdb->pVnode->config.tsdbCfg; - pTableBlockInfo->defMinRows = pc->minRows; - pTableBlockInfo->defMaxRows = pc->maxRows; - - int32_t bucketRange = ceil(((double)(pc->maxRows - pc->minRows)) / numOfBuckets); - - pTableBlockInfo->numOfFiles += 1; - - int32_t numOfTables = (int32_t)tSimpleHashGetSize(pStatus->pTableMap); - int defaultRows = 4096; - - SDataBlockIter* pBlockIter = &pStatus->blockIter; - pTableBlockInfo->numOfFiles += pStatus->fileIter.numOfFiles; - - if (pBlockIter->numOfBlocks > 0) { - pTableBlockInfo->numOfBlocks += pBlockIter->numOfBlocks; - } - - pTableBlockInfo->numOfTables = numOfTables; - bool hasNext = (pBlockIter->numOfBlocks > 0); - - while (true) { - if (hasNext) { - SDataBlk* pBlock = getCurrentBlock(pBlockIter); - - int32_t numOfRows = pBlock->nRow; - pTableBlockInfo->totalRows += numOfRows; - - if (numOfRows > pTableBlockInfo->maxRows) { - pTableBlockInfo->maxRows = numOfRows; - } - - if (numOfRows < pTableBlockInfo->minRows) { - pTableBlockInfo->minRows = numOfRows; - } - - if (numOfRows < defaultRows) { - pTableBlockInfo->numOfSmallBlocks += 1; - } - - pTableBlockInfo->totalSize += pBlock->aSubBlock[0].szBlock; - - int32_t bucketIndex = getBucketIndex(pTableBlockInfo->defMinRows, bucketRange, numOfRows, numOfBuckets); - pTableBlockInfo->blockRowsHisto[bucketIndex]++; - - hasNext = blockIteratorNext(&pStatus->blockIter, pReader->idStr); - } else { - code = initForFirstBlockInFile(pReader, pBlockIter); - if ((code != TSDB_CODE_SUCCESS) || (pStatus->loadFromFile == false)) { - break; - } - - pTableBlockInfo->numOfBlocks += pBlockIter->numOfBlocks; - hasNext = (pBlockIter->numOfBlocks > 0); - } - - // tsdbDebug("%p %d blocks found in file for %d table(s), fid:%d, %s", pReader, numOfBlocks, numOfTables, - // pReader->pFileGroup->fid, pReader->idStr); - } - tsdbReleaseReader(pReader); - return code; -} - -int64_t tsdbGetNumOfRowsInMemTable(STsdbReader* pReader) { - int32_t code = TSDB_CODE_SUCCESS; - int64_t rows = 0; - - SReaderStatus* pStatus = &pReader->status; - tsdbAcquireReader(pReader); - if (pReader->flag == READER_STATUS_SUSPEND) { - code = tsdbReaderResume(pReader); - if (code != TSDB_CODE_SUCCESS) { - tsdbReleaseReader(pReader); - return code; - } - } - - int32_t iter = 0; - pStatus->pTableIter = tSimpleHashIterate(pStatus->pTableMap, NULL, &iter); - - while (pStatus->pTableIter != NULL) { - STableBlockScanInfo* pBlockScanInfo = *(STableBlockScanInfo**)pStatus->pTableIter; - - STbData* d = NULL; - if (pReader->pReadSnap->pMem != NULL) { - d = tsdbGetTbDataFromMemTable(pReader->pReadSnap->pMem, pReader->suid, pBlockScanInfo->uid); - if (d != NULL) { - rows += tsdbGetNRowsInTbData(d); - } - } - - STbData* di = NULL; - if (pReader->pReadSnap->pIMem != NULL) { - di = tsdbGetTbDataFromMemTable(pReader->pReadSnap->pIMem, pReader->suid, pBlockScanInfo->uid); - if (di != NULL) { - rows += tsdbGetNRowsInTbData(di); - } - } - - // current table is exhausted, let's try the next table - pStatus->pTableIter = tSimpleHashIterate(pStatus->pTableMap, pStatus->pTableIter, &iter); - } - - tsdbReleaseReader(pReader); - - return rows; -} - -int32_t tsdbGetTableSchema(void* pVnode, int64_t uid, STSchema** pSchema, int64_t* suid) { - SMetaReader mr = {0}; - metaReaderDoInit(&mr, ((SVnode*)pVnode)->pMeta, 0); - int32_t code = metaReaderGetTableEntryByUidCache(&mr, uid); - if (code != TSDB_CODE_SUCCESS) { - terrno = TSDB_CODE_TDB_INVALID_TABLE_ID; - metaReaderClear(&mr); - return terrno; - } - - *suid = 0; - - // only child table and ordinary table is allowed, super table is not allowed. - if (mr.me.type == TSDB_CHILD_TABLE) { - tDecoderClear(&mr.coder); - *suid = mr.me.ctbEntry.suid; - code = metaReaderGetTableEntryByUidCache(&mr, *suid); - if (code != TSDB_CODE_SUCCESS) { - terrno = TSDB_CODE_TDB_INVALID_TABLE_ID; - metaReaderClear(&mr); - return terrno; - } - } else if (mr.me.type == TSDB_NORMAL_TABLE) { // do nothing - } else { - terrno = TSDB_CODE_INVALID_PARA; - metaReaderClear(&mr); - return terrno; - } - - metaReaderClear(&mr); - - // get the newest table schema version - code = metaGetTbTSchemaEx(((SVnode*)pVnode)->pMeta, *suid, uid, -1, pSchema); - return code; -} - -int32_t tsdbTakeReadSnap(STsdbReader* pReader, _query_reseek_func_t reseek, STsdbReadSnap** ppSnap) { - int32_t code = 0; - STsdb* pTsdb = pReader->pTsdb; - SVersionRange* pRange = &pReader->verRange; - - // alloc - STsdbReadSnap* pSnap = (STsdbReadSnap*)taosMemoryCalloc(1, sizeof(*pSnap)); - if (pSnap == NULL) { - code = TSDB_CODE_OUT_OF_MEMORY; - goto _exit; - } - - // lock - taosThreadRwlockRdlock(&pTsdb->rwLock); - - // take snapshot - if (pTsdb->mem && (pRange->minVer <= pTsdb->mem->maxVer && pRange->maxVer >= pTsdb->mem->minVer)) { - pSnap->pMem = pTsdb->mem; - pSnap->pNode = taosMemoryMalloc(sizeof(*pSnap->pNode)); - if (pSnap->pNode == NULL) { - taosThreadRwlockUnlock(&pTsdb->rwLock); - code = TSDB_CODE_OUT_OF_MEMORY; - goto _exit; - } - pSnap->pNode->pQHandle = pReader; - pSnap->pNode->reseek = reseek; - - tsdbRefMemTable(pTsdb->mem, pSnap->pNode); - } - - if (pTsdb->imem && (pRange->minVer <= pTsdb->imem->maxVer && pRange->maxVer >= pTsdb->imem->minVer)) { - pSnap->pIMem = pTsdb->imem; - pSnap->pINode = taosMemoryMalloc(sizeof(*pSnap->pINode)); - if (pSnap->pINode == NULL) { - taosThreadRwlockUnlock(&pTsdb->rwLock); - code = TSDB_CODE_OUT_OF_MEMORY; - goto _exit; - } - pSnap->pINode->pQHandle = pReader; - pSnap->pINode->reseek = reseek; - - tsdbRefMemTable(pTsdb->imem, pSnap->pINode); - } - - // fs - code = tsdbFSRef(pTsdb, &pSnap->fs); - if (code) { - taosThreadRwlockUnlock(&pTsdb->rwLock); - goto _exit; - } - - // unlock - taosThreadRwlockUnlock(&pTsdb->rwLock); - - tsdbTrace("vgId:%d, take read snapshot", TD_VID(pTsdb->pVnode)); - -_exit: - if (code) { - *ppSnap = NULL; - if (pSnap) { - if (pSnap->pNode) taosMemoryFree(pSnap->pNode); - if (pSnap->pINode) taosMemoryFree(pSnap->pINode); - taosMemoryFree(pSnap); - } - } else { - *ppSnap = pSnap; - } - return code; -} - -void tsdbUntakeReadSnap(STsdbReader* pReader, STsdbReadSnap* pSnap, bool proactive) { - STsdb* pTsdb = pReader->pTsdb; - - if (pSnap) { - if (pSnap->pMem) { - tsdbUnrefMemTable(pSnap->pMem, pSnap->pNode, proactive); - } - - if (pSnap->pIMem) { - tsdbUnrefMemTable(pSnap->pIMem, pSnap->pINode, proactive); - } - - tsdbFSUnref(pTsdb, &pSnap->fs); - if (pSnap->pNode) taosMemoryFree(pSnap->pNode); - if (pSnap->pINode) taosMemoryFree(pSnap->pINode); - taosMemoryFree(pSnap); - } - tsdbTrace("vgId:%d, untake read snapshot", TD_VID(pTsdb->pVnode)); -} - -// if failed, do nothing -void tsdbReaderSetId(STsdbReader* pReader, const char* idstr) { - taosMemoryFreeClear(pReader->idStr); - pReader->idStr = taosStrdup(idstr); -} - -void tsdbReaderSetCloseFlag(STsdbReader* pReader) { /*pReader->code = TSDB_CODE_TSC_QUERY_CANCELLED;*/ } diff --git a/source/dnode/vnode/src/tsdb/tsdbRead2.c b/source/dnode/vnode/src/tsdb/tsdbRead2.c index e4aba7011c..5b8e5a5b87 100644 --- a/source/dnode/vnode/src/tsdb/tsdbRead2.c +++ b/source/dnode/vnode/src/tsdb/tsdbRead2.c @@ -2760,6 +2760,7 @@ static int32_t doSumFileBlockRows(STsdbReader* pReader, SDataFReader* pFileReade goto _end; } +#if 0 int32_t numOfTables = tSimpleHashGetSize(pReader->status.pTableMap); SArray* aBlockIdx = (SArray*)taosLRUCacheValue(pFileReader->pTsdb->biCache, handle); @@ -2788,6 +2789,7 @@ static int32_t doSumFileBlockRows(STsdbReader* pReader, SDataFReader* pFileReade // pReader->rowsNum += block.nRow; // } } +#endif _end: tsdbBICacheRelease(pFileReader->pTsdb->biCache, handle); @@ -4453,7 +4455,11 @@ static void doFillNullColSMA(SBlockLoadSuppInfo* pSup, int32_t numOfRows, int32_ // do fill all null column value SMA info int32_t i = 0, j = 0; int32_t size = (int32_t)TARRAY2_SIZE(&pSup->colAggArray); - TARRAY2_INSERT_PTR(&pSup->colAggArray, 0, pTsAgg); + int32_t code = TARRAY2_INSERT_PTR(&pSup->colAggArray, 0, pTsAgg); + if (code != TSDB_CODE_SUCCESS) { + return; + } + size++; while (j < numOfCols && i < size) { @@ -4466,7 +4472,11 @@ static void doFillNullColSMA(SBlockLoadSuppInfo* pSup, int32_t numOfRows, int32_ } else if (pSup->colId[j] < pAgg->colId) { if (pSup->colId[j] != PRIMARYKEY_TIMESTAMP_COL_ID) { SColumnDataAgg nullColAgg = {.colId = pSup->colId[j], .numOfNull = numOfRows}; - TARRAY2_INSERT_PTR(&pSup->colAggArray, i, &nullColAgg); + code = TARRAY2_INSERT_PTR(&pSup->colAggArray, i, &nullColAgg); + if (code != TSDB_CODE_SUCCESS) { + return; + } + i += 1; size++; } @@ -4477,7 +4487,11 @@ static void doFillNullColSMA(SBlockLoadSuppInfo* pSup, int32_t numOfRows, int32_ while (j < numOfCols) { if (pSup->colId[j] != PRIMARYKEY_TIMESTAMP_COL_ID) { SColumnDataAgg nullColAgg = {.colId = pSup->colId[j], .numOfNull = numOfRows}; - TARRAY2_INSERT_PTR(&pSup->colAggArray, i, &nullColAgg); + code = TARRAY2_INSERT_PTR(&pSup->colAggArray, i, &nullColAgg); + if (code != TSDB_CODE_SUCCESS) { + return; + } + i += 1; } j++; @@ -4835,7 +4849,7 @@ int64_t tsdbGetNumOfRowsInMemTable2(STsdbReader* pReader) { return rows; } -int32_t tsdbGetTableSchema2(void* pVnode, int64_t uid, STSchema** pSchema, int64_t* suid) { +int32_t tsdbGetTableSchema(void* pVnode, int64_t uid, STSchema** pSchema, int64_t* suid) { SMetaReader mr = {0}; metaReaderDoInit(&mr, ((SVnode*)pVnode)->pMeta, 0); int32_t code = metaReaderGetTableEntryByUidCache(&mr, uid); @@ -4970,4 +4984,4 @@ void tsdbReaderSetId2(STsdbReader* pReader, const char* idstr) { pReader->status.fileIter.pLastBlockReader->mergeTree.idStr = pReader->idStr; } -void tsdbReaderSetCloseFlag2(STsdbReader* pReader) { pReader->code = TSDB_CODE_TSC_QUERY_CANCELLED; } +void tsdbReaderSetCloseFlag(STsdbReader* pReader) { /*pReader->code = TSDB_CODE_TSC_QUERY_CANCELLED;*/ } diff --git a/source/libs/parser/src/parTranslater.c b/source/libs/parser/src/parTranslater.c index 3a6829da56..d4e899cb23 100644 --- a/source/libs/parser/src/parTranslater.c +++ b/source/libs/parser/src/parTranslater.c @@ -7011,8 +7011,8 @@ static int32_t createLastTsSelectStmt(char* pDb, char* pTable, STableMeta* pMeta return TSDB_CODE_OUT_OF_MEMORY; } - strcpy(col->tableAlias, pTable); - strcpy(col->colName, pMeta->schema[0].name); + tstrncpy(col->tableAlias, pTable, tListLen(col->tableAlias)); + tstrncpy(col->colName, pMeta->schema[0].name, tListLen(col->colName)); SNodeList* pParamterList = nodesMakeList(); if (NULL == pParamterList) { nodesDestroyNode((SNode*)col); From c6355fcc2f8bffd1f5d1274003cc8f7fdeb50d99 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 14 Sep 2023 09:25:35 +0800 Subject: [PATCH 006/158] fix(tsdb): check return code. --- source/dnode/vnode/src/tsdb/tsdbRead2.c | 3 +++ 1 file changed, 3 insertions(+) diff --git a/source/dnode/vnode/src/tsdb/tsdbRead2.c b/source/dnode/vnode/src/tsdb/tsdbRead2.c index 5b8e5a5b87..9299264136 100644 --- a/source/dnode/vnode/src/tsdb/tsdbRead2.c +++ b/source/dnode/vnode/src/tsdb/tsdbRead2.c @@ -452,6 +452,9 @@ static int32_t doLoadBlockIndex(STsdbReader* pReader, SDataFileReader* pFileRead const TBrinBlkArray* pBlkArray = NULL; int32_t code = tsdbDataFileReadBrinBlk(pFileReader, &pBlkArray); + if (code != TSDB_CODE_SUCCESS) { + return code; + } #if 0 LRUHandle* handle = NULL; From 1256eafddb2be460b4bd75aec6a975b8e1d68eec Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 14 Sep 2023 14:09:05 +0800 Subject: [PATCH 007/158] other: merge stream fix. --- include/libs/stream/tstream.h | 2 +- source/common/src/tglobal.c | 2 +- source/dnode/mnode/impl/src/mndStream.c | 10 ++--- source/dnode/vnode/src/tq/tq.c | 37 +++++++++---------- source/dnode/vnode/src/tq/tqStreamStateSnap.c | 2 +- source/dnode/vnode/src/tq/tqStreamTask.c | 7 ++-- source/dnode/vnode/src/vnd/vnodeSync.c | 6 +-- source/libs/stream/src/stream.c | 1 - source/libs/stream/src/streamMeta.c | 2 +- source/libs/stream/src/streamQueue.c | 2 +- source/libs/stream/src/streamTask.c | 21 +++++++++-- 11 files changed, 48 insertions(+), 44 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index a5baf33612..60043d4df6 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -718,7 +718,7 @@ int32_t streamMetaGetNumOfTasks(SStreamMeta* pMeta); int32_t streamMetaGetNumOfStreamTasks(SStreamMeta* pMeta); SStreamTask* streamMetaAcquireTask(SStreamMeta* pMeta, int64_t streamId, int32_t taskId); void streamMetaReleaseTask(SStreamMeta* pMeta, SStreamTask* pTask); -int32_t streamMetaReopen(SStreamMeta* pMeta, int64_t chkpId); +int32_t streamMetaReopen(SStreamMeta* pMeta); int32_t streamMetaCommit(SStreamMeta* pMeta); int32_t streamMetaLoadAllTasks(SStreamMeta* pMeta); void streamMetaNotifyClose(SStreamMeta* pMeta); diff --git a/source/common/src/tglobal.c b/source/common/src/tglobal.c index 90812a66b2..1069c5830e 100644 --- a/source/common/src/tglobal.c +++ b/source/common/src/tglobal.c @@ -241,7 +241,7 @@ int32_t tsTtlBatchDropNum = 10000; // number of tables dropped per batch // internal int32_t tsTransPullupInterval = 2; int32_t tsMqRebalanceInterval = 2; -int32_t tsStreamCheckpointTickInterval = 600; +int32_t tsStreamCheckpointTickInterval = 30; int32_t tsStreamNodeCheckInterval = 10; int32_t tsTtlUnit = 86400; int32_t tsTtlPushIntervalSec = 10; diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index d3162a143d..6d44d55d25 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -65,9 +65,6 @@ static int32_t mndProcessDropStreamReq(SRpcMsg *pReq); static int32_t mndProcessStreamCheckpointTmr(SRpcMsg *pReq); static int32_t mndProcessStreamDoCheckpoint(SRpcMsg *pReq); static int32_t mndProcessStreamHb(SRpcMsg *pReq); -static int32_t mndProcessRecoverStreamReq(SRpcMsg *pReq); -static int32_t mndProcessStreamMetaReq(SRpcMsg *pReq); -static int32_t mndGetStreamMeta(SRpcMsg *pReq, SShowObj *pShow, STableMetaRsp *pMeta); static int32_t mndRetrieveStream(SRpcMsg *pReq, SShowObj *pShow, SSDataBlock *pBlock, int32_t rows); static void mndCancelGetNextStream(SMnode *pMnode, void *pIter); static int32_t mndRetrieveStreamTask(SRpcMsg *pReq, SShowObj *pShow, SSDataBlock *pBlock, int32_t rows); @@ -1063,8 +1060,7 @@ static int32_t mndBuildStreamCheckpointSourceReq2(void **pBuf, int32_t *pLen, in // return -1; // } -static int32_t mndAddStreamCheckpointToTrans(STrans *pTrans, SStreamObj *pStream, SMnode *pMnode, - int64_t checkpointId) { +static int32_t mndAddStreamCheckpointToTrans(STrans *pTrans, SStreamObj *pStream, SMnode *pMnode, int64_t chkptId) { taosWLockLatch(&pStream->lock); int32_t totLevel = taosArrayGetSize(pStream->tasks); @@ -1088,7 +1084,7 @@ static int32_t mndAddStreamCheckpointToTrans(STrans *pTrans, SStreamObj *pStream void *buf; int32_t tlen; - if (mndBuildStreamCheckpointSourceReq2(&buf, &tlen, pTask->info.nodeId, checkpointId, pTask->id.streamId, + if (mndBuildStreamCheckpointSourceReq2(&buf, &tlen, pTask->info.nodeId, chkptId, pTask->id.streamId, pTask->id.taskId) < 0) { mndReleaseVgroup(pMnode, pVgObj); taosWUnLockLatch(&pStream->lock); @@ -1109,7 +1105,7 @@ static int32_t mndAddStreamCheckpointToTrans(STrans *pTrans, SStreamObj *pStream } } - pStream->checkpointId = checkpointId; + pStream->checkpointId = chkptId; pStream->checkpointFreq = taosGetTimestampMs(); atomic_store_64(&pStream->currentTick, 0); // 3. commit log: stream checkpoint info diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 7186adc2c4..ab6e0d1171 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1698,9 +1698,12 @@ int32_t tqProcessTaskUpdateReq(STQ* pTq, SRpcMsg* pMsg) { if (tDecodeStreamTaskUpdateMsg(&decoder, &req) < 0) { rsp.code = TSDB_CODE_MSG_DECODE_ERROR; tqError("vgId:%d failed to decode task update msg, code:%s", vgId, tstrerror(rsp.code)); - goto _end; + tDecoderClear(&decoder); + return rsp.code; } + tDecoderClear(&decoder); + // update the nodeEpset when it exists taosWLockLatch(&pMeta->lock); @@ -1713,7 +1716,7 @@ int32_t tqProcessTaskUpdateReq(STQ* pTq, SRpcMsg* pMsg) { req.taskId); rsp.code = TSDB_CODE_SUCCESS; taosWUnLockLatch(&pMeta->lock); - goto _end; + return rsp.code; } SStreamTask* pTask = *ppTask; @@ -1753,37 +1756,32 @@ int32_t tqProcessTaskUpdateReq(STQ* pTq, SRpcMsg* pMsg) { streamTaskStop(*ppHTask); } - tqDebug("s-task:%s task nodeEp update completed", pTask->id.idStr); - pMeta->closedTask += 1; if (ppHTask != NULL) { + tqDebug("s-task:%s task nodeEp update completed, streamTask and related fill-history task closed", pTask->id.idStr); pMeta->closedTask += 1; + } else { + tqDebug("s-task:%s task nodeEp update completed, streamTask closed", pTask->id.idStr); } + rsp.code = 0; + // possibly only handle the stream task. int32_t numOfTasks = streamMetaGetNumOfTasks(pMeta); - allStopped = (pMeta->closedTask == numOfTasks); - if (allStopped) { - pMeta->closedTask = 0; + if (pMeta->closedTask < numOfTasks) { + tqDebug("vgId:%d closed tasks:%d, unclosed:%d", vgId, pMeta->closedTask, (numOfTasks - pMeta->closedTask)); + taosWUnLockLatch(&pMeta->lock); } else { - tqDebug("vgId:%d closed tasks:%d, not closed:%d", vgId, pMeta->closedTask, (numOfTasks - pMeta->closedTask)); - } - - taosWUnLockLatch(&pMeta->lock); - -_end: - tDecoderClear(&decoder); - - if (allStopped) { + pMeta->closedTask = 0; if (!pTq->pVnode->restored) { tqDebug("vgId:%d vnode restore not completed, not restart the tasks", vgId); + taosWUnLockLatch(&pMeta->lock); } else { - tqDebug("vgId:%d all tasks are stopped, restart them", vgId); - taosWLockLatch(&pMeta->lock); + tqDebug("vgId:%d tasks are all updated and stopped, restart them", vgId); terrno = 0; - int32_t code = streamMetaReopen(pMeta, 0); + int32_t code = streamMetaReopen(pMeta); if (code != 0) { tqError("vgId:%d failed to reopen stream meta", vgId); taosWUnLockLatch(&pMeta->lock); @@ -1807,4 +1805,3 @@ _end: return rsp.code; } - diff --git a/source/dnode/vnode/src/tq/tqStreamStateSnap.c b/source/dnode/vnode/src/tq/tqStreamStateSnap.c index 4a1b3961cd..a016498980 100644 --- a/source/dnode/vnode/src/tq/tqStreamStateSnap.c +++ b/source/dnode/vnode/src/tq/tqStreamStateSnap.c @@ -168,7 +168,7 @@ int32_t streamStateSnapWriterClose(SStreamStateWriter* pWriter, int8_t rollback) } int32_t streamStateRebuildFromSnap(SStreamStateWriter* pWriter, int64_t chkpId) { tqDebug("vgId:%d, vnode %s start to rebuild stream-state", TD_VID(pWriter->pTq->pVnode), STREAM_STATE_TRANSFER); - int32_t code = streamMetaReopen(pWriter->pTq->pStreamMeta, chkpId); + int32_t code = streamMetaReopen(pWriter->pTq->pStreamMeta); if (code == 0) { code = streamStateLoadTasks(pWriter); } diff --git a/source/dnode/vnode/src/tq/tqStreamTask.c b/source/dnode/vnode/src/tq/tqStreamTask.c index 1ac2ddb9cb..255f71bf30 100644 --- a/source/dnode/vnode/src/tq/tqStreamTask.c +++ b/source/dnode/vnode/src/tq/tqStreamTask.c @@ -201,8 +201,7 @@ int32_t tqStopStreamTasks(STQ* pTq) { int32_t vgId = TD_VID(pTq->pVnode); int32_t numOfTasks = taosArrayGetSize(pMeta->pTaskList); - tqDebug("vgId:%d start to stop all %d stream task(s)", vgId, numOfTasks); - + tqDebug("vgId:%d stop all %d stream task(s)", vgId, numOfTasks); if (numOfTasks == 0) { return TSDB_CODE_SUCCESS; } @@ -232,7 +231,7 @@ int32_t tqStartStreamTasks(STQ* pTq) { int32_t vgId = TD_VID(pTq->pVnode); int32_t numOfTasks = taosArrayGetSize(pMeta->pTaskList); - tqDebug("vgId:%d start to stop all %d stream task(s)", vgId, numOfTasks); + tqDebug("vgId:%d start all %d stream task(s)", vgId, numOfTasks); if (numOfTasks == 0) { return TSDB_CODE_SUCCESS; @@ -314,7 +313,7 @@ void handleFillhistoryScanComplete(SStreamTask* pTask, int64_t ver) { double el = (taosGetTimestampMs() - pTask->tsInfo.step2Start) / 1000.0; qDebug("s-task:%s scan-history from WAL stage(step 2) ended, elapsed time:%.2fs", id, el); /*int32_t code = */streamTaskPutTranstateIntoInputQ(pTask); - /*int32_t code = */ streamSchedExec(pTask); + /*int32_t code = */streamSchedExec(pTask); } else { qWarn("s-task:%s fill-history scan WAL, nextProcessVer:%" PRId64 " out of the maximum ver:%" PRId64 ", not scan wal", id, ver, maxVer); diff --git a/source/dnode/vnode/src/vnd/vnodeSync.c b/source/dnode/vnode/src/vnd/vnodeSync.c index 43850ebfee..3a7a60fcbb 100644 --- a/source/dnode/vnode/src/vnd/vnodeSync.c +++ b/source/dnode/vnode/src/vnd/vnodeSync.c @@ -553,13 +553,11 @@ static void vnodeRestoreFinish(const SSyncFSM *pFsm, const SyncIndex commitIdx) pVnode->restored = true; if (vnodeIsRoleLeader(pVnode)) { - vInfo("vgId:%d, sync restore finished, start to launch stream tasks", vgId); - // start to restore all stream tasks if (tsDisableStream) { - vInfo("vgId:%d, not launch stream tasks, since stream tasks are disabled", vgId); + vInfo("vgId:%d, sync restore finished, not launch stream tasks, since stream tasks are disabled", vgId); } else { - vInfo("vgId:%d start to launch stream tasks", pVnode->config.vgId); + vInfo("vgId:%d sync restore finished, start to launch stream tasks", pVnode->config.vgId); tqStartStreamTasks(pVnode->pTq); tqCheckAndRunStreamTaskAsync(pVnode->pTq); } diff --git a/source/libs/stream/src/stream.c b/source/libs/stream/src/stream.c index 1b4de5e6c4..5a7e14c629 100644 --- a/source/libs/stream/src/stream.c +++ b/source/libs/stream/src/stream.c @@ -109,7 +109,6 @@ int32_t streamSetupScheduleTrigger(SStreamTask* pTask) { int32_t streamSchedExec(SStreamTask* pTask) { int8_t schedStatus = streamTaskSetSchedStatusWait(pTask); - if (schedStatus == TASK_SCHED_STATUS__INACTIVE) { SStreamTaskRunReq* pRunReq = rpcMallocCont(sizeof(SStreamTaskRunReq)); if (pRunReq == NULL) { diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 652ef7cde7..55d9a46b11 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -211,7 +211,7 @@ _err: return NULL; } -int32_t streamMetaReopen(SStreamMeta* pMeta, int64_t chkpId) { +int32_t streamMetaReopen(SStreamMeta* pMeta) { streamMetaClear(pMeta); pMeta->streamBackendRid = -1; diff --git a/source/libs/stream/src/streamQueue.c b/source/libs/stream/src/streamQueue.c index 29ca351a6b..a9d0c3b77e 100644 --- a/source/libs/stream/src/streamQueue.c +++ b/source/libs/stream/src/streamQueue.c @@ -395,7 +395,7 @@ int32_t streamTaskPutDataIntoOutputQ(SStreamTask* pTask, SStreamDataBlock* pBloc } int32_t streamTaskInitTokenBucket(STokenBucket* pBucket, int32_t cap, int32_t rate) { - if (cap < 100 || rate < 50 || pBucket == NULL) { + if (cap < 50 || rate < 50 || pBucket == NULL) { qError("failed to init sink task bucket, cap:%d, rate:%d", cap, rate); return TSDB_CODE_INVALID_PARA; } diff --git a/source/libs/stream/src/streamTask.c b/source/libs/stream/src/streamTask.c index 71a9a3102c..af550f86cb 100644 --- a/source/libs/stream/src/streamTask.c +++ b/source/libs/stream/src/streamTask.c @@ -384,8 +384,22 @@ int32_t streamTaskInit(SStreamTask* pTask, SStreamMeta* pMeta, SMsgCb* pMsgCb, i pTask->dataRange.range.minVer = ver; pTask->pMsgCb = pMsgCb; - streamTaskInitTokenBucket(&pTask->tokenBucket, 100, 100); - taosThreadMutexInit(&pTask->lock, NULL); + streamTaskInitTokenBucket(&pTask->tokenBucket, 50, 50); + + TdThreadMutexAttr attr = {0}; + int ret = taosThreadMutexAttrInit(&attr); + if (ret != 0) { + qError("s-task:%s init mutex attr failed, code:%s", pTask->id.idStr, tstrerror(ret)); + return ret; + } + + ret = taosThreadMutexAttrSetType(&attr, PTHREAD_MUTEX_RECURSIVE); + if (ret != 0) { + qError("s-task:%s set mutex attr recursive, code:%s", pTask->id.idStr, tstrerror(ret)); + return ret; + } + + taosThreadMutexInit(&pTask->lock, &attr); streamTaskOpenAllUpstreamInput(pTask); return TSDB_CODE_SUCCESS; @@ -578,7 +592,8 @@ int8_t streamTaskSetSchedStatusActive(SStreamTask* pTask) { int8_t streamTaskSetSchedStatusInActive(SStreamTask* pTask) { taosThreadMutexLock(&pTask->lock); int8_t status = pTask->status.schedStatus; - ASSERT(status == TASK_SCHED_STATUS__WAITING || status == TASK_SCHED_STATUS__ACTIVE); + ASSERT(status == TASK_SCHED_STATUS__WAITING || status == TASK_SCHED_STATUS__ACTIVE || + status == TASK_SCHED_STATUS__INACTIVE); pTask->status.schedStatus = TASK_SCHED_STATUS__INACTIVE; taosThreadMutexUnlock(&pTask->lock); From 6fa54789cb40a08be8bea5a28a6532aa3d4b534f Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 14 Sep 2023 17:22:31 +0800 Subject: [PATCH 008/158] fix(stream): disable follower send hb to mnode. and do some internal refactor. --- include/libs/stream/tstream.h | 6 ++++- source/dnode/vnode/src/inc/tq.h | 2 +- source/dnode/vnode/src/tq/tq.c | 17 ++++++++----- source/dnode/vnode/src/tq/tqStreamTask.c | 13 ++++------ source/dnode/vnode/src/tq/tqUtil.c | 11 +++++--- source/dnode/vnode/src/vnd/vnodeSync.c | 22 +++++++++++++--- source/libs/stream/src/streamMeta.c | 32 ++++++++++++++++++------ 7 files changed, 73 insertions(+), 30 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 60043d4df6..932a6d951b 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -400,6 +400,8 @@ typedef struct SStreamMeta { FTaskExpand* expandFunc; int32_t vgId; int64_t stage; + bool leader; + int8_t taskWillbeLaunched; SRWLatch lock; // TdThreadRwlock lock; int32_t walScanCounter; @@ -408,7 +410,8 @@ typedef struct SStreamMeta { SHashObj* pTaskBackendUnique; TdThreadMutex backendMutex; SMetaHbInfo hbInfo; - int32_t closedTask; + SHashObj* pUpdateTaskList; +// int32_t closedTask; int32_t totalTasks; // this value should be increased when a new task is added into the meta int32_t chkptNotReadyTasks; int64_t rid; @@ -722,6 +725,7 @@ int32_t streamMetaReopen(SStreamMeta* pMeta); int32_t streamMetaCommit(SStreamMeta* pMeta); int32_t streamMetaLoadAllTasks(SStreamMeta* pMeta); void streamMetaNotifyClose(SStreamMeta* pMeta); +void streamMetaStartHb(SStreamMeta* pMeta); // checkpoint int32_t streamProcessCheckpointSourceReq(SStreamTask* pTask, SStreamCheckpointSourceReq* pReq); diff --git a/source/dnode/vnode/src/inc/tq.h b/source/dnode/vnode/src/inc/tq.h index c6a424666c..4e73a481c8 100644 --- a/source/dnode/vnode/src/inc/tq.h +++ b/source/dnode/vnode/src/inc/tq.h @@ -174,7 +174,7 @@ int32_t tqExtractDataForMq(STQ* pTq, STqHandle* pHandle, const SMqPollReq* pRequ int32_t tqDoSendDataRsp(const SRpcHandleInfo* pRpcHandleInfo, const SMqDataRsp* pRsp, int32_t epoch, int64_t consumerId, int32_t type, int64_t sver, int64_t ever); int32_t tqInitDataRsp(SMqDataRsp* pRsp, STqOffsetVal pOffset); -void tqUpdateNodeStage(STQ* pTq); +void tqUpdateNodeStage(STQ* pTq, bool isLeader); #ifdef __cplusplus } diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index ab6e0d1171..fa734096c9 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1756,10 +1756,10 @@ int32_t tqProcessTaskUpdateReq(STQ* pTq, SRpcMsg* pMsg) { streamTaskStop(*ppHTask); } - pMeta->closedTask += 1; + taosHashPut(pMeta->pUpdateTaskList, &pTask->id, sizeof(pTask->id), NULL, 0); if (ppHTask != NULL) { tqDebug("s-task:%s task nodeEp update completed, streamTask and related fill-history task closed", pTask->id.idStr); - pMeta->closedTask += 1; + taosHashPut(pMeta->pUpdateTaskList, &(*ppHTask)->id, sizeof(pTask->id), NULL, 0); } else { tqDebug("s-task:%s task nodeEp update completed, streamTask closed", pTask->id.idStr); } @@ -1768,11 +1768,14 @@ int32_t tqProcessTaskUpdateReq(STQ* pTq, SRpcMsg* pMsg) { // possibly only handle the stream task. int32_t numOfTasks = streamMetaGetNumOfTasks(pMeta); - if (pMeta->closedTask < numOfTasks) { - tqDebug("vgId:%d closed tasks:%d, unclosed:%d", vgId, pMeta->closedTask, (numOfTasks - pMeta->closedTask)); + int32_t updateTasks = taosHashGetSize(pMeta->pUpdateTaskList); + if (updateTasks < numOfTasks) { + pMeta->taskWillbeLaunched = 1; + + tqDebug("vgId:%d closed tasks:%d, unclosed:%d", vgId, updateTasks, (numOfTasks - updateTasks)); taosWUnLockLatch(&pMeta->lock); } else { - pMeta->closedTask = 0; + taosHashClear(pMeta->pUpdateTaskList); if (!pTq->pVnode->restored) { tqDebug("vgId:%d vnode restore not completed, not restart the tasks", vgId); @@ -1794,12 +1797,14 @@ int32_t tqProcessTaskUpdateReq(STQ* pTq, SRpcMsg* pMsg) { return -1; } - taosWUnLockLatch(&pMeta->lock); if (vnodeIsRoleLeader(pTq->pVnode) && !tsDisableStream) { vInfo("vgId:%d, restart all stream tasks", vgId); tqStartStreamTasks(pTq); tqCheckAndRunStreamTaskAsync(pTq); } + + pMeta->taskWillbeLaunched = 0; + taosWUnLockLatch(&pMeta->lock); } } diff --git a/source/dnode/vnode/src/tq/tqStreamTask.c b/source/dnode/vnode/src/tq/tqStreamTask.c index 255f71bf30..8c45aa4f8c 100644 --- a/source/dnode/vnode/src/tq/tqStreamTask.c +++ b/source/dnode/vnode/src/tq/tqStreamTask.c @@ -111,12 +111,12 @@ int32_t tqCheckAndRunStreamTaskAsync(STQ* pTq) { int32_t vgId = TD_VID(pTq->pVnode); SStreamMeta* pMeta = pTq->pStreamMeta; - taosWLockLatch(&pMeta->lock); +// taosWLockLatch(&pMeta->lock); int32_t numOfTasks = taosArrayGetSize(pMeta->pTaskList); if (numOfTasks == 0) { tqDebug("vgId:%d no stream tasks existed to run", vgId); - taosWUnLockLatch(&pMeta->lock); +// taosWUnLockLatch(&pMeta->lock); return 0; } @@ -124,7 +124,7 @@ int32_t tqCheckAndRunStreamTaskAsync(STQ* pTq) { if (pRunReq == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; tqError("vgId:%d failed to create msg to start wal scanning to launch stream tasks, code:%s", vgId, terrstr()); - taosWUnLockLatch(&pMeta->lock); +// taosWUnLockLatch(&pMeta->lock); return -1; } @@ -135,7 +135,7 @@ int32_t tqCheckAndRunStreamTaskAsync(STQ* pTq) { SRpcMsg msg = {.msgType = TDMT_STREAM_TASK_RUN, .pCont = pRunReq, .contLen = sizeof(SStreamTaskRunReq)}; tmsgPutToQueue(&pTq->pVnode->msgCb, STREAM_QUEUE, &msg); - taosWUnLockLatch(&pMeta->lock); +// taosWUnLockLatch(&pMeta->lock); return 0; } @@ -237,8 +237,6 @@ int32_t tqStartStreamTasks(STQ* pTq) { return TSDB_CODE_SUCCESS; } - taosWLockLatch(&pMeta->lock); - for (int32_t i = 0; i < numOfTasks; ++i) { SStreamTaskId* pTaskId = taosArrayGet(pMeta->pTaskList, i); @@ -246,12 +244,11 @@ int32_t tqStartStreamTasks(STQ* pTq) { SStreamTask** pTask = taosHashGet(pMeta->pTasks, key, sizeof(key)); int8_t status = (*pTask)->status.taskStatus; - if (status == TASK_STATUS__STOP) { + if (status == TASK_STATUS__STOP && (*pTask)->info.fillHistory != 1) { streamSetStatusNormal(*pTask); } } - taosWUnLockLatch(&pMeta->lock); return 0; } diff --git a/source/dnode/vnode/src/tq/tqUtil.c b/source/dnode/vnode/src/tq/tqUtil.c index 04695c1f63..f10f87b6b7 100644 --- a/source/dnode/vnode/src/tq/tqUtil.c +++ b/source/dnode/vnode/src/tq/tqUtil.c @@ -36,10 +36,15 @@ int32_t tqInitDataRsp(SMqDataRsp* pRsp, STqOffsetVal pOffset) { return 0; } -void tqUpdateNodeStage(STQ* pTq) { +void tqUpdateNodeStage(STQ* pTq, bool isLeader) { SSyncState state = syncGetState(pTq->pVnode->sync); - pTq->pStreamMeta->stage = state.term; - tqDebug("vgId:%d update the meta stage to be:%"PRId64, pTq->pStreamMeta->vgId, pTq->pStreamMeta->stage); + SStreamMeta* pMeta = pTq->pStreamMeta; + tqDebug("vgId:%d update the meta stage:%"PRId64", prev:%"PRId64" leader:%d", pMeta->vgId, state.term, pMeta->stage, isLeader); + pMeta->stage = state.term; + pMeta->leader = isLeader; + if (isLeader) { + streamMetaStartHb(pMeta); + } } static int32_t tqInitTaosxRsp(STaosxRsp* pRsp, STqOffsetVal pOffset) { diff --git a/source/dnode/vnode/src/vnd/vnodeSync.c b/source/dnode/vnode/src/vnd/vnodeSync.c index 3a7a60fcbb..9a4dfc8c11 100644 --- a/source/dnode/vnode/src/vnd/vnodeSync.c +++ b/source/dnode/vnode/src/vnd/vnodeSync.c @@ -549,9 +549,20 @@ static void vnodeRestoreFinish(const SSyncFSM *pFsm, const SyncIndex commitIdx) ASSERT(commitIdx == vnodeSyncAppliedIndex(pFsm)); walApplyVer(pVnode->pWal, commitIdx); - pVnode->restored = true; + if (!pVnode->pTq->pStreamMeta->taskWillbeLaunched) { + vInfo("vgId:%d, sync restore finished, stream tasks will be launched by other thread", vgId); + return; + } + + taosWLockLatch(&pVnode->pTq->pStreamMeta->lock); + if (!pVnode->pTq->pStreamMeta->taskWillbeLaunched) { + vInfo("vgId:%d, sync restore finished, stream tasks will be launched by other thread", vgId); + taosWUnLockLatch(&pVnode->pTq->pStreamMeta->lock); + return; + } + if (vnodeIsRoleLeader(pVnode)) { // start to restore all stream tasks if (tsDisableStream) { @@ -564,6 +575,8 @@ static void vnodeRestoreFinish(const SSyncFSM *pFsm, const SyncIndex commitIdx) } else { vInfo("vgId:%d, sync restore finished, not launch stream tasks since not leader", vgId); } + + taosWUnLockLatch(&pVnode->pTq->pStreamMeta->lock); } static void vnodeBecomeFollower(const SSyncFSM *pFsm) { @@ -578,7 +591,10 @@ static void vnodeBecomeFollower(const SSyncFSM *pFsm) { } taosThreadMutexUnlock(&pVnode->lock); - tqStopStreamTasks(pVnode->pTq); + if (pVnode->pTq) { + tqUpdateNodeStage(pVnode->pTq, false); + tqStopStreamTasks(pVnode->pTq); + } } static void vnodeBecomeLearner(const SSyncFSM *pFsm) { @@ -597,7 +613,7 @@ static void vnodeBecomeLearner(const SSyncFSM *pFsm) { static void vnodeBecomeLeader(const SSyncFSM *pFsm) { SVnode *pVnode = pFsm->data; if (pVnode->pTq) { - tqUpdateNodeStage(pVnode->pTq); + tqUpdateNodeStage(pVnode->pTq, true); } vDebug("vgId:%d, become leader", pVnode->config.vgId); } diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 55d9a46b11..9fa9a664b9 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -140,6 +140,11 @@ SStreamMeta* streamMetaOpen(const char* path, void* ahandle, FTaskExpand expandF goto _err; } + pMeta->pUpdateTaskList = taosHashInit(64, fp, false, HASH_NO_LOCK); + if (pMeta->pUpdateTaskList == NULL) { + goto _err; + } + // task list pMeta->pTaskList = taosArrayInit(4, sizeof(SStreamTaskId)); if (pMeta->pTaskList == NULL) { @@ -316,6 +321,7 @@ void streamMetaCloseImpl(void* arg) { taosHashCleanup(pMeta->pTasks); taosHashCleanup(pMeta->pTaskBackendUnique); + taosHashCleanup(pMeta->pUpdateTaskList); taosMemoryFree(pMeta->path); taosThreadMutexDestroy(&pMeta->backendMutex); @@ -758,9 +764,8 @@ int32_t tDecodeStreamHbMsg(SDecoder* pDecoder, SStreamHbMsg* pReq) { return 0; } -static bool readyToSendHb(SMetaHbInfo* pInfo) { - if ((++pInfo->tickCounter) >= META_HB_SEND_IDLE_COUNTER) { - // reset the counter +static bool enoughTimeDuration(SMetaHbInfo* pInfo) { + if ((++pInfo->tickCounter) >= META_HB_SEND_IDLE_COUNTER) { // reset the counter pInfo->tickCounter = 0; return true; } @@ -784,7 +789,14 @@ void metaHbToMnode(void* param, void* tmrId) { return; } - if (!readyToSendHb(&pMeta->hbInfo)) { + // not leader not send msg + if (!pMeta->leader) { + qInfo("vgId:%d follower not send hb to mnode", pMeta->vgId); + taosReleaseRef(streamMetaId, rid); + return; + } + + if (!enoughTimeDuration(&pMeta->hbInfo)) { taosTmrReset(metaHbToMnode, META_HB_CHECK_INTERVAL, param, streamEnv.timer, &pMeta->hbInfo.hbTmr); taosReleaseRef(streamMetaId, rid); return; @@ -907,10 +919,12 @@ void streamMetaNotifyClose(SStreamMeta* pMeta) { taosWUnLockLatch(&pMeta->lock); // wait for the stream meta hb function stopping - pMeta->hbInfo.stopFlag = STREAM_META_WILL_STOP; - while (pMeta->hbInfo.stopFlag != STREAM_META_OK_TO_STOP) { - taosMsleep(100); - qDebug("vgId:%d wait for meta to stop timer", pMeta->vgId); + if (pMeta->leader) { + pMeta->hbInfo.stopFlag = STREAM_META_WILL_STOP; + while (pMeta->hbInfo.stopFlag != STREAM_META_OK_TO_STOP) { + taosMsleep(100); + qDebug("vgId:%d wait for meta to stop timer", pMeta->vgId); + } } qDebug("vgId:%d start to check all tasks", vgId); @@ -924,3 +938,5 @@ void streamMetaNotifyClose(SStreamMeta* pMeta) { int64_t el = taosGetTimestampMs() - st; qDebug("vgId:%d all stream tasks are not in timer, continue close, elapsed time:%" PRId64 " ms", pMeta->vgId, el); } + +void streamMetaStartHb(SStreamMeta* pMeta) { metaHbToMnode(pMeta, NULL); } From 3d1f29ff21c33328d3c8db40f95ba1bdb33d882f Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 14 Sep 2023 19:33:05 +0800 Subject: [PATCH 009/158] fix(stream): fix memory leak. --- source/dnode/vnode/src/tq/tq.c | 64 ++++++++++------------------- source/libs/stream/src/streamExec.c | 10 ++--- source/libs/stream/src/streamMeta.c | 12 +++--- 3 files changed, 33 insertions(+), 53 deletions(-) diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index fa734096c9..c066aaab18 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -852,11 +852,21 @@ int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t ver) { pChkInfo->checkpointId, pChkInfo->checkpointVer, pChkInfo->nextProcessVer); } - tqInfo("vgId:%d expand stream task, s-task:%s, checkpointId:%" PRId64 " checkpointVer:%" PRId64 " nextProcessVer:%" PRId64 - " child id:%d, level:%d, status:%s fill-history:%d, trigger:%" PRId64 " ms", - vgId, pTask->id.idStr, pChkInfo->checkpointId, pChkInfo->checkpointVer, pChkInfo->nextProcessVer, - pTask->info.selfChildId, pTask->info.taskLevel, streamGetTaskStatusStr(pTask->status.taskStatus), - pTask->info.fillHistory, pTask->info.triggerParam); + if (pTask->info.fillHistory) { + tqInfo("vgId:%d expand stream task, s-task:%s, checkpointId:%" PRId64 " checkpointVer:%" PRId64 + " nextProcessVer:%" PRId64 + " child id:%d, level:%d, status:%s fill-history:%d, related stream task:0x%x trigger:%" PRId64 " ms", + vgId, pTask->id.idStr, pChkInfo->checkpointId, pChkInfo->checkpointVer, pChkInfo->nextProcessVer, + pTask->info.selfChildId, pTask->info.taskLevel, streamGetTaskStatusStr(pTask->status.taskStatus), + pTask->info.fillHistory, pTask->streamTaskId.taskId, pTask->info.triggerParam); + } else { + tqInfo("vgId:%d expand stream task, s-task:%s, checkpointId:%" PRId64 " checkpointVer:%" PRId64 + " nextProcessVer:%" PRId64 + " child id:%d, level:%d, status:%s fill-history:%d, related fill-task:0x%x trigger:%" PRId64 " ms", + vgId, pTask->id.idStr, pChkInfo->checkpointId, pChkInfo->checkpointVer, pChkInfo->nextProcessVer, + pTask->info.selfChildId, pTask->info.taskLevel, streamGetTaskStatusStr(pTask->status.taskStatus), + pTask->info.fillHistory, pTask->historyTaskId.taskId, pTask->info.triggerParam); + } return 0; } @@ -1181,44 +1191,6 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { return 0; } -// notify the downstream tasks to transfer executor state after handle all history blocks. -int32_t tqProcessTaskTransferStateReq(STQ* pTq, SRpcMsg* pMsg) { - char* pReq = POINTER_SHIFT(pMsg->pCont, sizeof(SMsgHead)); - int32_t len = pMsg->contLen - sizeof(SMsgHead); - - SStreamTransferReq req = {0}; - - SDecoder decoder; - tDecoderInit(&decoder, (uint8_t*)pReq, len); - int32_t code = tDecodeStreamScanHistoryFinishReq(&decoder, &req); - tDecoderClear(&decoder); - - tqDebug("vgId:%d start to process transfer state msg, from s-task:0x%x", pTq->pStreamMeta->vgId, - req.downstreamTaskId); - - SStreamTask* pTask = streamMetaAcquireTask(pTq->pStreamMeta, req.streamId, req.downstreamTaskId); - if (pTask == NULL) { - tqError("failed to find task:0x%x, it may have been dropped already. process transfer state failed", - req.downstreamTaskId); - return -1; - } - - int32_t remain = streamAlignTransferState(pTask); - if (remain > 0) { - tqDebug("s-task:%s receive upstream transfer state msg, remain:%d", pTask->id.idStr, remain); - streamMetaReleaseTask(pTq->pStreamMeta, pTask); - return 0; - } - - // transfer the ownership of executor state - tqDebug("s-task:%s all upstream tasks send transfer msg, open transfer state flag", pTask->id.idStr); - ASSERT(pTask->streamTaskId.taskId != 0 && pTask->info.fillHistory == 1); - - streamSchedExec(pTask); - streamMetaReleaseTask(pTq->pStreamMeta, pTask); - return 0; -} - // only the agg tasks and the sink tasks will receive this message from upstream tasks int32_t tqProcessTaskScanHistoryFinishReq(STQ* pTq, SRpcMsg* pMsg) { char* msg = POINTER_SHIFT(pMsg->pCont, sizeof(SMsgHead)); @@ -1592,6 +1564,10 @@ int32_t tqProcessStreamCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg) { int32_t code = 0; SStreamCheckpointSourceReq req = {0}; + if (!vnodeIsRoleLeader(pTq->pVnode)) { + tqDebug("vgId:%d not leader node, ignore checkpoint-source msg", vgId); + return TSDB_CODE_SUCCESS; + } SDecoder decoder; tDecoderInit(&decoder, (uint8_t*)msg, len); @@ -1801,6 +1777,8 @@ int32_t tqProcessTaskUpdateReq(STQ* pTq, SRpcMsg* pMsg) { vInfo("vgId:%d, restart all stream tasks", vgId); tqStartStreamTasks(pTq); tqCheckAndRunStreamTaskAsync(pTq); + } else { + vInfo("vgId:%d, follower node not start stream tasks", vgId); } pMeta->taskWillbeLaunched = 0; diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index 94dfc09314..9a45555d4a 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -358,18 +358,18 @@ int32_t streamDoTransferStateToStreamTask(SStreamTask* pTask) { streamTaskReleaseState(pTask); streamTaskReloadState(pStreamTask); - // 3. clear the link between fill-history task and stream task info - pStreamTask->historyTaskId.taskId = 0; - - // 4. resume the state of stream task, after this function, the stream task will run immidately. But it can not be + // 3. resume the state of stream task, after this function, the stream task will run immidately. But it can not be // pause, since the pause allowed attribute is not set yet. streamTaskResumeFromHalt(pStreamTask); qDebug("s-task:%s fill-history task set status to be dropping, save the state into disk", pTask->id.idStr); - // 5. free it and remove fill-history task from disk meta-store + // 4. free it and remove fill-history task from disk meta-store streamMetaUnregisterTask(pMeta, pTask->id.streamId, pTask->id.taskId); + // 5. clear the link between fill-history task and stream task info + pStreamTask->historyTaskId.taskId = 0; + // 6. save to disk taosWLockLatch(&pMeta->lock); streamMetaSaveTask(pMeta, pStreamTask); diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 9fa9a664b9..5084caad0c 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -207,9 +207,6 @@ _err: if (pMeta->pCheckpointDb) tdbTbClose(pMeta->pCheckpointDb); if (pMeta->db) tdbClose(pMeta->db); - // taosThreadMutexDestroy(&pMeta->backendMutex); - // taosThreadRwlockDestroy(&pMeta->lock); - taosMemoryFree(pMeta); qError("failed to open stream meta"); @@ -695,7 +692,6 @@ int32_t streamMetaLoadAllTasks(SStreamMeta* pMeta) { continue; } - streamTaskResetUpstreamStageInfo(pTask); if (taosHashPut(pMeta->pTasks, keys, sizeof(keys), &pTask, sizeof(void*)) < 0) { doClear(pKey, pVal, pCur, pRecycleList); tFreeStreamTask(pTask); @@ -708,6 +704,7 @@ int32_t streamMetaLoadAllTasks(SStreamMeta* pMeta) { ASSERT(pTask->status.downstreamReady == 0); } + qInfo("vgId:%d pause task num:%d", pMeta->vgId, pMeta->pauseTaskNum); tdbFree(pKey); @@ -939,4 +936,9 @@ void streamMetaNotifyClose(SStreamMeta* pMeta) { qDebug("vgId:%d all stream tasks are not in timer, continue close, elapsed time:%" PRId64 " ms", pMeta->vgId, el); } -void streamMetaStartHb(SStreamMeta* pMeta) { metaHbToMnode(pMeta, NULL); } +void streamMetaStartHb(SStreamMeta* pMeta) { + int64_t* pRid = taosMemoryMalloc(sizeof(int64_t)); + metaRefMgtAdd(pMeta->vgId, pRid); + *pRid = pMeta->rid; + metaHbToMnode(pRid, NULL); +} From 39e0c57323f0b69418c0038e816bffd10708118d Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 14 Sep 2023 22:44:16 +0800 Subject: [PATCH 010/158] fix(stream): set the correct guard flag. --- source/dnode/vnode/src/vnd/vnodeSync.c | 4 ++-- source/libs/stream/src/streamMeta.c | 7 ++++++- 2 files changed, 8 insertions(+), 3 deletions(-) diff --git a/source/dnode/vnode/src/vnd/vnodeSync.c b/source/dnode/vnode/src/vnd/vnodeSync.c index 9a4dfc8c11..42acdd2b40 100644 --- a/source/dnode/vnode/src/vnd/vnodeSync.c +++ b/source/dnode/vnode/src/vnd/vnodeSync.c @@ -551,13 +551,13 @@ static void vnodeRestoreFinish(const SSyncFSM *pFsm, const SyncIndex commitIdx) walApplyVer(pVnode->pWal, commitIdx); pVnode->restored = true; - if (!pVnode->pTq->pStreamMeta->taskWillbeLaunched) { + if (pVnode->pTq->pStreamMeta->taskWillbeLaunched) { vInfo("vgId:%d, sync restore finished, stream tasks will be launched by other thread", vgId); return; } taosWLockLatch(&pVnode->pTq->pStreamMeta->lock); - if (!pVnode->pTq->pStreamMeta->taskWillbeLaunched) { + if (pVnode->pTq->pStreamMeta->taskWillbeLaunched) { vInfo("vgId:%d, sync restore finished, stream tasks will be launched by other thread", vgId); taosWUnLockLatch(&pVnode->pTq->pStreamMeta->lock); return; diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 5084caad0c..fe76e0ad7a 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -794,11 +794,14 @@ void metaHbToMnode(void* param, void* tmrId) { } if (!enoughTimeDuration(&pMeta->hbInfo)) { + qInfo("vgId:%d not enough time, %d", pMeta->vgId, pMeta->hbInfo.tickCounter); taosTmrReset(metaHbToMnode, META_HB_CHECK_INTERVAL, param, streamEnv.timer, &pMeta->hbInfo.hbTmr); taosReleaseRef(streamMetaId, rid); return; } + qInfo("vgId:%d start hb", pMeta->vgId); + taosRLockLatch(&pMeta->lock); int32_t numOfTasks = streamMetaGetNumOfTasks(pMeta); @@ -820,7 +823,7 @@ void metaHbToMnode(void* param, void* tmrId) { STaskStatusEntry entry = {.streamId = pId->streamId, .taskId = pId->taskId, .status = (*pTask)->status.taskStatus}; taosArrayPush(hbMsg.pTaskStatus, &entry); - if (i == 0) { + if (!hasValEpset) { epsetAssign(&epset, &(*pTask)->info.mnodeEpset); hasValEpset = true; } @@ -866,6 +869,8 @@ void metaHbToMnode(void* param, void* tmrId) { qDebug("vgId:%d, build and send hb to mnode", pMeta->vgId); tmsgSendReq(&epset, &msg); + } else { + qError("vgId:%d no mnd epset", pMeta->vgId); } taosArrayDestroy(hbMsg.pTaskStatus); From 5f1146d4154c65510cf241c156cb9b4a56939cfa Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 14 Sep 2023 22:59:11 +0800 Subject: [PATCH 011/158] fix(stream): fix memory leak. --- source/dnode/mnode/impl/src/mndStream.c | 3 +++ source/dnode/vnode/src/tq/tq.c | 4 ++++ 2 files changed, 7 insertions(+) diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index 6d44d55d25..cbc8e1e099 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -1886,6 +1886,7 @@ static int32_t doBuildStreamTaskUpdateMsg(void **pBuf, int32_t *pLen, SVgroupCha tEncodeSize(tEncodeStreamTaskUpdateMsg, &req, blen, code); if (code < 0) { terrno = TSDB_CODE_OUT_OF_MEMORY; + taosArrayDestroy(req.pNodeList); return -1; } @@ -1894,6 +1895,7 @@ static int32_t doBuildStreamTaskUpdateMsg(void **pBuf, int32_t *pLen, SVgroupCha void *buf = taosMemoryMalloc(tlen); if (buf == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; + taosArrayDestroy(req.pNodeList); return -1; } @@ -1911,6 +1913,7 @@ static int32_t doBuildStreamTaskUpdateMsg(void **pBuf, int32_t *pLen, SVgroupCha *pBuf = buf; *pLen = tlen; + taosArrayDestroy(req.pNodeList); return TSDB_CODE_SUCCESS; } diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index c066aaab18..63dcce67d0 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1692,6 +1692,7 @@ int32_t tqProcessTaskUpdateReq(STQ* pTq, SRpcMsg* pMsg) { req.taskId); rsp.code = TSDB_CODE_SUCCESS; taosWUnLockLatch(&pMeta->lock); + taosArrayDestroy(req.pNodeList); return rsp.code; } @@ -1764,12 +1765,14 @@ int32_t tqProcessTaskUpdateReq(STQ* pTq, SRpcMsg* pMsg) { if (code != 0) { tqError("vgId:%d failed to reopen stream meta", vgId); taosWUnLockLatch(&pMeta->lock); + taosArrayDestroy(req.pNodeList); return -1; } if (streamMetaLoadAllTasks(pTq->pStreamMeta) < 0) { tqError("vgId:%d failed to load stream tasks", vgId); taosWUnLockLatch(&pMeta->lock); + taosArrayDestroy(req.pNodeList); return -1; } @@ -1786,5 +1789,6 @@ int32_t tqProcessTaskUpdateReq(STQ* pTq, SRpcMsg* pMsg) { } } + taosArrayDestroy(req.pNodeList); return rsp.code; } From c33e521aa198ee4d0adc104b0cdfa7c15b75bb8e Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 14 Sep 2023 23:17:23 +0800 Subject: [PATCH 012/158] fix(stream): update some logs. --- source/dnode/vnode/src/tq/tq.c | 2 +- source/libs/stream/src/streamMeta.c | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 63dcce67d0..f90281ee18 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -937,7 +937,7 @@ int32_t tqProcessStreamTaskCheckRsp(STQ* pTq, SRpcMsg* pMsg) { SStreamTask* pTask = streamMetaAcquireTask(pTq->pStreamMeta, rsp.streamId, rsp.upstreamTaskId); if (pTask == NULL) { - tqError("tq failed to locate the stream task:0x%" PRIx64 "-0x%x (vgId:%d), it may have been destroyed", + tqError("tq failed to locate the stream task:0x%" PRIx64 "-0x%x (vgId:%d), it may have been destroyed or stopped", rsp.streamId, rsp.upstreamTaskId, pTq->pStreamMeta->vgId); terrno = TSDB_CODE_STREAM_TASK_NOT_EXIST; return -1; diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index fe76e0ad7a..a64e268c12 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -794,7 +794,6 @@ void metaHbToMnode(void* param, void* tmrId) { } if (!enoughTimeDuration(&pMeta->hbInfo)) { - qInfo("vgId:%d not enough time, %d", pMeta->vgId, pMeta->hbInfo.tickCounter); taosTmrReset(metaHbToMnode, META_HB_CHECK_INTERVAL, param, streamEnv.timer, &pMeta->hbInfo.hbTmr); taosReleaseRef(streamMetaId, rid); return; From fe7864b976bbf8e748608617fd8ab1a2aaf2d2c8 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Fri, 15 Sep 2023 00:51:13 +0800 Subject: [PATCH 013/158] fix(stream): fix error in drop task. --- source/dnode/vnode/src/tq/tq.c | 7 ------- source/libs/stream/src/streamTask.c | 3 ++- 2 files changed, 2 insertions(+), 8 deletions(-) diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index f90281ee18..a93293d53a 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1343,14 +1343,7 @@ int32_t tqProcessTaskDispatchRsp(STQ* pTq, SRpcMsg* pMsg) { int32_t tqProcessTaskDropReq(STQ* pTq, int64_t sversion, char* msg, int32_t msgLen) { SVDropStreamTaskReq* pReq = (SVDropStreamTaskReq*)msg; tqDebug("vgId:%d receive msg to drop stream task:0x%x", TD_VID(pTq->pVnode), pReq->taskId); - SStreamTask* pTask = streamMetaAcquireTask(pTq->pStreamMeta, pReq->streamId, pReq->taskId); - if (pTask == NULL) { - tqError("vgId:%d failed to acquire s-task:0x%x when dropping it", pTq->pStreamMeta->vgId, pReq->taskId); - return 0; - } - streamMetaUnregisterTask(pTq->pStreamMeta, pReq->streamId, pReq->taskId); - streamMetaReleaseTask(pTq->pStreamMeta, pTask); return 0; } diff --git a/source/libs/stream/src/streamTask.c b/source/libs/stream/src/streamTask.c index af550f86cb..d42dee76f6 100644 --- a/source/libs/stream/src/streamTask.c +++ b/source/libs/stream/src/streamTask.c @@ -458,7 +458,8 @@ void streamTaskUpdateUpstreamInfo(SStreamTask* pTask, int32_t nodeId, const SEpS SStreamChildEpInfo* pInfo = taosArrayGetP(pTask->pUpstreamInfoList, i); if (pInfo->nodeId == nodeId) { epsetAssign(&pInfo->epSet, pEpSet); - qDebug("s-task:0x%x update the upstreamInfo, nodeId:%d newEpset:%s", pTask->id.taskId, nodeId, buf); + qDebug("s-task:0x%x update the upstreamInfo, nodeId:%d taskId:0x%x newEpset:%s", pTask->id.taskId, nodeId, + pInfo->taskId, buf); break; } } From e84eeee6b80573cf907ce4ec3a328f3779765d99 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Fri, 15 Sep 2023 09:25:22 +0800 Subject: [PATCH 014/158] refactor: do some internal refactor. --- include/libs/stream/tstream.h | 32 +++------ source/dnode/vnode/src/tq/tq.c | 45 +++++++------ source/dnode/vnode/src/tq/tqRead.c | 2 +- source/dnode/vnode/src/tq/tqSink.c | 6 +- source/dnode/vnode/src/tq/tqStreamTask.c | 8 +-- source/libs/stream/inc/streamInt.h | 11 ++- source/libs/stream/src/streamCheckpoint.c | 2 +- source/libs/stream/src/streamExec.c | 2 +- source/libs/stream/src/streamMeta.c | 81 +++++++++++++---------- source/libs/stream/src/streamQueue.c | 5 +- source/libs/stream/src/streamRecover.c | 6 +- source/libs/stream/src/streamTask.c | 43 ++++++++---- 12 files changed, 136 insertions(+), 107 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 932a6d951b..1fd2f2bc13 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -321,15 +321,13 @@ typedef struct { int64_t init; int64_t step1Start; int64_t step2Start; - int64_t sinkStart; -} STaskTimestamp; + int64_t execStart; + int32_t taskUpdateCount; + int64_t latestUpdateTs; +} STaskExecStatisInfo; -typedef struct STokenBucket { - int32_t capacity; // total capacity - int64_t fillTimestamp;// fill timestamp - int32_t numOfToken; // total available tokens - int32_t rate; // number of token per second -} STokenBucket; +typedef struct STokenBucket STokenBucket; +typedef struct SMetaHbInfo SMetaHbInfo; struct SStreamTask { int64_t ver; @@ -345,7 +343,7 @@ struct SStreamTask { SDataRange dataRange; SStreamTaskId historyTaskId; SStreamTaskId streamTaskId; - STaskTimestamp tsInfo; + STaskExecStatisInfo taskExecInfo; SArray* pReadyMsgList; // SArray TdThreadMutex lock; // secure the operation of set task status and puting data into inputQ SArray* pUpstreamInfoList; @@ -359,7 +357,7 @@ struct SStreamTask { STaskSinkFetch fetchSink; }; SSinkTaskRecorder sinkRecorder; - STokenBucket tokenBucket; + STokenBucket* pTokenBucket; void* launchTaskTimer; SMsgCb* pMsgCb; // msg handle @@ -381,19 +379,13 @@ struct SStreamTask { char reserve[256]; }; -typedef struct SMetaHbInfo { - tmr_h hbTmr; - int32_t stopFlag; - int32_t tickCounter; -} SMetaHbInfo; - // meta typedef struct SStreamMeta { char* path; TDB* db; TTB* pTaskDb; TTB* pCheckpointDb; - SHashObj* pTasks; + SHashObj* pTasksMap; SArray* pTaskList; // SArray void* ahandle; TXN* txn; @@ -403,15 +395,13 @@ typedef struct SStreamMeta { bool leader; int8_t taskWillbeLaunched; SRWLatch lock; -// TdThreadRwlock lock; int32_t walScanCounter; void* streamBackend; int64_t streamBackendRid; SHashObj* pTaskBackendUnique; TdThreadMutex backendMutex; - SMetaHbInfo hbInfo; - SHashObj* pUpdateTaskList; -// int32_t closedTask; + SMetaHbInfo* pHbInfo; + SHashObj* pUpdateTaskSet; int32_t totalTasks; // this value should be increased when a new task is added into the meta int32_t chkptNotReadyTasks; int64_t rid; diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index a93293d53a..6430fee6a7 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1006,8 +1006,8 @@ int32_t tqProcessTaskDeployReq(STQ* pTq, int64_t sversion, char* msg, int32_t ms bool restored = pTq->pVnode->restored; if (p != NULL && restored) { - p->tsInfo.init = taosGetTimestampMs(); - tqDebug("s-task:%s set the init ts:%"PRId64, p->id.idStr, p->tsInfo.init); + p->taskExecInfo.init = taosGetTimestampMs(); + tqDebug("s-task:%s set the init ts:%"PRId64, p->id.idStr, p->taskExecInfo.init); streamTaskCheckDownstream(p); } else if (!restored) { @@ -1045,14 +1045,14 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { const char* pStatus = streamGetTaskStatusStr(pTask->status.taskStatus); tqDebug("s-task:%s start scan-history stage(step 1), status:%s", id, pStatus); - if (pTask->tsInfo.step1Start == 0) { + if (pTask->taskExecInfo.step1Start == 0) { ASSERT(pTask->status.pauseAllowed == false); - pTask->tsInfo.step1Start = taosGetTimestampMs(); + pTask->taskExecInfo.step1Start = taosGetTimestampMs(); if (pTask->info.fillHistory == 1) { streamTaskEnablePause(pTask); } } else { - tqDebug("s-task:%s resume from paused, start ts:%" PRId64, pTask->id.idStr, pTask->tsInfo.step1Start); + tqDebug("s-task:%s resume from paused, start ts:%" PRId64, pTask->id.idStr, pTask->taskExecInfo.step1Start); } // we have to continue retrying to successfully execute the scan history task. @@ -1072,7 +1072,7 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { streamScanHistoryData(pTask); if (pTask->status.taskStatus == TASK_STATUS__PAUSE) { - double el = (taosGetTimestampMs() - pTask->tsInfo.step1Start) / 1000.0; + double el = (taosGetTimestampMs() - pTask->taskExecInfo.step1Start) / 1000.0; int8_t status = streamTaskSetSchedStatusInActive(pTask); tqDebug("s-task:%s is paused in the step1, elapsed time:%.2fs, sched-status:%d", pTask->id.idStr, el, status); streamMetaReleaseTask(pMeta, pTask); @@ -1080,7 +1080,7 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { } // the following procedure should be executed, no matter status is stop/pause or not - double el = (taosGetTimestampMs() - pTask->tsInfo.step1Start) / 1000.0; + double el = (taosGetTimestampMs() - pTask->taskExecInfo.step1Start) / 1000.0; tqDebug("s-task:%s scan-history stage(step 1) ended, elapsed time:%.2fs", id, el); if (pTask->info.fillHistory) { @@ -1128,7 +1128,7 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { done = streamHistoryTaskSetVerRangeStep2(pTask, latestVer); if (done) { - pTask->tsInfo.step2Start = taosGetTimestampMs(); + pTask->taskExecInfo.step2Start = taosGetTimestampMs(); qDebug("s-task:%s scan-history from WAL stage(step 2) ended, elapsed time:%.2fs", id, 0.0); streamTaskPutTranstateIntoInputQ(pTask); streamTryExec(pTask); // exec directly @@ -1140,7 +1140,7 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { pStreamTask->id.idStr); ASSERT(pTask->status.schedStatus == TASK_SCHED_STATUS__WAITING); - pTask->tsInfo.step2Start = taosGetTimestampMs(); + pTask->taskExecInfo.step2Start = taosGetTimestampMs(); streamSetParamForStreamScannerStep2(pTask, pRange, pWindow); int64_t dstVer = pTask->dataRange.range.minVer; @@ -1344,6 +1344,13 @@ int32_t tqProcessTaskDropReq(STQ* pTq, int64_t sversion, char* msg, int32_t msgL SVDropStreamTaskReq* pReq = (SVDropStreamTaskReq*)msg; tqDebug("vgId:%d receive msg to drop stream task:0x%x", TD_VID(pTq->pVnode), pReq->taskId); streamMetaUnregisterTask(pTq->pStreamMeta, pReq->streamId, pReq->taskId); + + // commit the update + taosWLockLatch(&pTq->pStreamMeta->lock); + if (streamMetaCommit(pTq->pStreamMeta) < 0) { + // persist to disk + } + taosWUnLockLatch(&pTq->pStreamMeta->lock); return 0; } @@ -1676,9 +1683,9 @@ int32_t tqProcessTaskUpdateReq(STQ* pTq, SRpcMsg* pMsg) { // update the nodeEpset when it exists taosWLockLatch(&pMeta->lock); - // when replay the WAL, we should update the task epset one again and again, the task may be in stop status. + // the task epset may be updated again and again, when replaying the WAL, the task may be in stop status. int64_t keys[2] = {req.streamId, req.taskId}; - SStreamTask** ppTask = (SStreamTask**)taosHashGet(pMeta->pTasks, keys, sizeof(keys)); + SStreamTask** ppTask = (SStreamTask**)taosHashGet(pMeta->pTasksMap, keys, sizeof(keys)); if (ppTask == NULL || *ppTask == NULL) { tqError("vgId:%d failed to acquire task:0x%x when handling update, it may have been dropped already", pMeta->vgId, @@ -1690,8 +1697,8 @@ int32_t tqProcessTaskUpdateReq(STQ* pTq, SRpcMsg* pMsg) { } SStreamTask* pTask = *ppTask; + tqDebug("s-task:%s receive nodeEp update msg from mnode", pTask->id.idStr); - tqDebug("s-task:%s receive task nodeEp update msg from mnode", pTask->id.idStr); streamTaskUpdateEpsetInfo(pTask, req.pNodeList); streamSetStatusNormal(pTask); @@ -1700,7 +1707,7 @@ int32_t tqProcessTaskUpdateReq(STQ* pTq, SRpcMsg* pMsg) { keys[0] = pTask->historyTaskId.streamId; keys[1] = pTask->historyTaskId.taskId; - ppHTask = (SStreamTask**)taosHashGet(pMeta->pTasks, keys, sizeof(keys)); + ppHTask = (SStreamTask**)taosHashGet(pMeta->pTasksMap, keys, sizeof(keys)); if (ppHTask == NULL || *ppHTask == NULL) { tqError("vgId:%d failed to acquire fill-history task:0x%x when handling update, it may have been dropped already", pMeta->vgId, req.taskId); @@ -1722,14 +1729,12 @@ int32_t tqProcessTaskUpdateReq(STQ* pTq, SRpcMsg* pMsg) { } streamTaskStop(pTask); + taosHashPut(pMeta->pUpdateTaskSet, &pTask->id, sizeof(pTask->id), NULL, 0); + if (ppHTask != NULL) { streamTaskStop(*ppHTask); - } - - taosHashPut(pMeta->pUpdateTaskList, &pTask->id, sizeof(pTask->id), NULL, 0); - if (ppHTask != NULL) { tqDebug("s-task:%s task nodeEp update completed, streamTask and related fill-history task closed", pTask->id.idStr); - taosHashPut(pMeta->pUpdateTaskList, &(*ppHTask)->id, sizeof(pTask->id), NULL, 0); + taosHashPut(pMeta->pUpdateTaskSet, &(*ppHTask)->id, sizeof(pTask->id), NULL, 0); } else { tqDebug("s-task:%s task nodeEp update completed, streamTask closed", pTask->id.idStr); } @@ -1738,14 +1743,14 @@ int32_t tqProcessTaskUpdateReq(STQ* pTq, SRpcMsg* pMsg) { // possibly only handle the stream task. int32_t numOfTasks = streamMetaGetNumOfTasks(pMeta); - int32_t updateTasks = taosHashGetSize(pMeta->pUpdateTaskList); + int32_t updateTasks = taosHashGetSize(pMeta->pUpdateTaskSet); if (updateTasks < numOfTasks) { pMeta->taskWillbeLaunched = 1; tqDebug("vgId:%d closed tasks:%d, unclosed:%d", vgId, updateTasks, (numOfTasks - updateTasks)); taosWUnLockLatch(&pMeta->lock); } else { - taosHashClear(pMeta->pUpdateTaskList); + taosHashClear(pMeta->pUpdateTaskSet); if (!pTq->pVnode->restored) { tqDebug("vgId:%d vnode restore not completed, not restart the tasks", vgId); diff --git a/source/dnode/vnode/src/tq/tqRead.c b/source/dnode/vnode/src/tq/tqRead.c index 39627a5f7b..916bd01cd0 100644 --- a/source/dnode/vnode/src/tq/tqRead.c +++ b/source/dnode/vnode/src/tq/tqRead.c @@ -1129,7 +1129,7 @@ int32_t tqUpdateTbUidList(STQ* pTq, const SArray* tbUidList, bool isAdd) { // update the table list handle for each stream scanner/wal reader taosWLockLatch(&pTq->pStreamMeta->lock); while (1) { - pIter = taosHashIterate(pTq->pStreamMeta->pTasks, pIter); + pIter = taosHashIterate(pTq->pStreamMeta->pTasksMap, pIter); if (pIter == NULL) { break; } diff --git a/source/dnode/vnode/src/tq/tqSink.c b/source/dnode/vnode/src/tq/tqSink.c index 0925573248..106a4cc9b0 100644 --- a/source/dnode/vnode/src/tq/tqSink.c +++ b/source/dnode/vnode/src/tq/tqSink.c @@ -274,7 +274,7 @@ int32_t doBuildAndSendSubmitMsg(SVnode* pVnode, SStreamTask* pTask, SSubmitReq2* if ((pTask->sinkRecorder.numOfSubmit % 5000) == 0) { SSinkTaskRecorder* pRec = &pTask->sinkRecorder; - double el = (taosGetTimestampMs() - pTask->tsInfo.sinkStart) / 1000.0; + double el = (taosGetTimestampMs() - pTask->taskExecInfo.execStart) / 1000.0; tqInfo("s-task:%s vgId:%d write %" PRId64 " blocks (%" PRId64 " rows) in %" PRId64 " submit into dst table, duration:%.2f Sec.", pTask->id.idStr, vgId, pRec->numOfBlocks, pRec->numOfRows, pRec->numOfSubmit, el); @@ -755,8 +755,8 @@ void tqSinkDataIntoDstTable(SStreamTask* pTask, void* vnode, void* data) { int32_t code = TSDB_CODE_SUCCESS; const char* id = pTask->id.idStr; - if (pTask->tsInfo.sinkStart == 0) { - pTask->tsInfo.sinkStart = taosGetTimestampMs(); + if (pTask->taskExecInfo.execStart == 0) { + pTask->taskExecInfo.execStart = taosGetTimestampMs(); } bool onlySubmitData = true; diff --git a/source/dnode/vnode/src/tq/tqStreamTask.c b/source/dnode/vnode/src/tq/tqStreamTask.c index 8c45aa4f8c..1e66988aab 100644 --- a/source/dnode/vnode/src/tq/tqStreamTask.c +++ b/source/dnode/vnode/src/tq/tqStreamTask.c @@ -94,8 +94,8 @@ int32_t tqCheckAndRunStreamTask(STQ* pTq) { continue; } - pTask->tsInfo.init = taosGetTimestampMs(); - tqDebug("s-task:%s set the init ts:%"PRId64, pTask->id.idStr, pTask->tsInfo.init); + pTask->taskExecInfo.init = taosGetTimestampMs(); + tqDebug("s-task:%s set the init ts:%"PRId64, pTask->id.idStr, pTask->taskExecInfo.init); streamSetStatusNormal(pTask); streamTaskCheckDownstream(pTask); @@ -241,7 +241,7 @@ int32_t tqStartStreamTasks(STQ* pTq) { SStreamTaskId* pTaskId = taosArrayGet(pMeta->pTaskList, i); int64_t key[2] = {pTaskId->streamId, pTaskId->taskId}; - SStreamTask** pTask = taosHashGet(pMeta->pTasks, key, sizeof(key)); + SStreamTask** pTask = taosHashGet(pMeta->pTasksMap, key, sizeof(key)); int8_t status = (*pTask)->status.taskStatus; if (status == TASK_STATUS__STOP && (*pTask)->info.fillHistory != 1) { @@ -307,7 +307,7 @@ void handleFillhistoryScanComplete(SStreamTask* pTask, int64_t ver) { ", not scan wal anymore, add transfer-state block into inputQ", id, ver, maxVer); - double el = (taosGetTimestampMs() - pTask->tsInfo.step2Start) / 1000.0; + double el = (taosGetTimestampMs() - pTask->taskExecInfo.step2Start) / 1000.0; qDebug("s-task:%s scan-history from WAL stage(step 2) ended, elapsed time:%.2fs", id, el); /*int32_t code = */streamTaskPutTranstateIntoInputQ(pTask); /*int32_t code = */streamSchedExec(pTask); diff --git a/source/libs/stream/inc/streamInt.h b/source/libs/stream/inc/streamInt.h index bbb7595e5a..fb11ec4ea4 100644 --- a/source/libs/stream/inc/streamInt.h +++ b/source/libs/stream/inc/streamInt.h @@ -29,17 +29,24 @@ extern "C" { #define ONE_MB_F (1048576.0) #define SIZE_IN_MB(_v) ((_v) / ONE_MB_F) -typedef struct { +typedef struct SStreamGlobalEnv { int8_t inited; void* timer; } SStreamGlobalEnv; -typedef struct { +typedef struct SStreamContinueExecInfo { SEpSet epset; int32_t taskId; SRpcMsg msg; } SStreamContinueExecInfo; +struct STokenBucket { + int32_t capacity; // total capacity + int64_t fillTimestamp;// fill timestamp + int32_t numOfToken; // total available tokens + int32_t rate; // number of token per second +}; + extern SStreamGlobalEnv streamEnv; extern int32_t streamBackendId; extern int32_t streamBackendCfWrapperId; diff --git a/source/libs/stream/src/streamCheckpoint.c b/source/libs/stream/src/streamCheckpoint.c index cc93d25fd5..cfbfdb5da4 100644 --- a/source/libs/stream/src/streamCheckpoint.c +++ b/source/libs/stream/src/streamCheckpoint.c @@ -270,7 +270,7 @@ int32_t streamSaveAllTaskStatus(SStreamMeta* pMeta, int64_t checkpointId) { keys[0] = pId->streamId; keys[1] = pId->taskId; - SStreamTask** ppTask = taosHashGet(pMeta->pTasks, keys, sizeof(keys)); + SStreamTask** ppTask = taosHashGet(pMeta->pTasksMap, keys, sizeof(keys)); if (ppTask == NULL) { continue; } diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index 9a45555d4a..3a34d941dd 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -201,7 +201,7 @@ int32_t streamScanHistoryData(SStreamTask* pTask) { while (!finished) { if (streamTaskShouldPause(&pTask->status)) { - double el = (taosGetTimestampMs() - pTask->tsInfo.step1Start) / 1000.0; + double el = (taosGetTimestampMs() - pTask->taskExecInfo.step1Start) / 1000.0; qDebug("s-task:%s paused from the scan-history task, elapsed time:%.2fsec", pTask->id.idStr, el); break; } diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index a64e268c12..8dd93d45fc 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -43,6 +43,12 @@ typedef struct { SHashObj* pTable; } SMetaRefMgt; +struct SMetaHbInfo { + tmr_h hbTmr; + int32_t stopFlag; + int32_t tickCounter; +}; + SMetaRefMgt gMetaRefMgt; void metaRefMgtInit(); @@ -135,13 +141,18 @@ SStreamMeta* streamMetaOpen(const char* path, void* ahandle, FTaskExpand expandF } _hash_fn_t fp = taosGetDefaultHashFunction(TSDB_DATA_TYPE_VARCHAR); - pMeta->pTasks = taosHashInit(64, fp, true, HASH_NO_LOCK); - if (pMeta->pTasks == NULL) { + pMeta->pTasksMap = taosHashInit(64, fp, true, HASH_NO_LOCK); + if (pMeta->pTasksMap == NULL) { goto _err; } - pMeta->pUpdateTaskList = taosHashInit(64, fp, false, HASH_NO_LOCK); - if (pMeta->pUpdateTaskList == NULL) { + pMeta->pUpdateTaskSet = taosHashInit(64, fp, false, HASH_NO_LOCK); + if (pMeta->pUpdateTaskSet == NULL) { + goto _err; + } + + pMeta->pHbInfo = taosMemoryCalloc(1, sizeof(SMetaHbInfo)); + if (pMeta->pHbInfo == NULL) { goto _err; } @@ -165,9 +176,9 @@ SStreamMeta* streamMetaOpen(const char* path, void* ahandle, FTaskExpand expandF metaRefMgtAdd(pMeta->vgId, pRid); - pMeta->hbInfo.hbTmr = taosTmrStart(metaHbToMnode, META_HB_CHECK_INTERVAL, pRid, streamEnv.timer); - pMeta->hbInfo.tickCounter = 0; - pMeta->hbInfo.stopFlag = 0; + pMeta->pHbInfo->hbTmr = taosTmrStart(metaHbToMnode, META_HB_CHECK_INTERVAL, pRid, streamEnv.timer); + pMeta->pHbInfo->tickCounter = 0; + pMeta->pHbInfo->stopFlag = 0; pMeta->pTaskBackendUnique = taosHashInit(64, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY), false, HASH_ENTRY_LOCK); @@ -201,11 +212,13 @@ SStreamMeta* streamMetaOpen(const char* path, void* ahandle, FTaskExpand expandF _err: taosMemoryFree(pMeta->path); - if (pMeta->pTasks) taosHashCleanup(pMeta->pTasks); + if (pMeta->pTasksMap) taosHashCleanup(pMeta->pTasksMap); if (pMeta->pTaskList) taosArrayDestroy(pMeta->pTaskList); if (pMeta->pTaskDb) tdbTbClose(pMeta->pTaskDb); if (pMeta->pCheckpointDb) tdbTbClose(pMeta->pCheckpointDb); if (pMeta->db) tdbClose(pMeta->db); + if (pMeta->pHbInfo) taosMemoryFreeClear(pMeta->pHbInfo); + if (pMeta->pUpdateTaskSet) taosHashCleanup(pMeta->pUpdateTaskSet); taosMemoryFree(pMeta); @@ -259,7 +272,7 @@ int32_t streamMetaReopen(SStreamMeta* pMeta) { void streamMetaClear(SStreamMeta* pMeta) { void* pIter = NULL; - while ((pIter = taosHashIterate(pMeta->pTasks, pIter)) != NULL) { + while ((pIter = taosHashIterate(pMeta->pTasksMap, pIter)) != NULL) { SStreamTask* p = *(SStreamTask**)pIter; // release the ref by timer @@ -275,7 +288,7 @@ void streamMetaClear(SStreamMeta* pMeta) { taosRemoveRef(streamBackendId, pMeta->streamBackendRid); - taosHashClear(pMeta->pTasks); + taosHashClear(pMeta->pTasksMap); taosHashClear(pMeta->pTaskBackendUnique); taosArrayClear(pMeta->pTaskList); @@ -316,9 +329,9 @@ void streamMetaCloseImpl(void* arg) { taosArrayDestroy(pMeta->chkpSaved); taosArrayDestroy(pMeta->chkpInUse); - taosHashCleanup(pMeta->pTasks); + taosHashCleanup(pMeta->pTasksMap); taosHashCleanup(pMeta->pTaskBackendUnique); - taosHashCleanup(pMeta->pUpdateTaskList); + taosHashCleanup(pMeta->pUpdateTaskSet); taosMemoryFree(pMeta->path); taosThreadMutexDestroy(&pMeta->backendMutex); @@ -380,7 +393,7 @@ int32_t streamMetaRegisterTask(SStreamMeta* pMeta, int64_t ver, SStreamTask* pTa *pAdded = false; int64_t keys[2] = {pTask->id.streamId, pTask->id.taskId}; - void* p = taosHashGet(pMeta->pTasks, keys, sizeof(keys)); + void* p = taosHashGet(pMeta->pTasksMap, keys, sizeof(keys)); if (p == NULL) { if (pMeta->expandFunc(pMeta->ahandle, pTask, ver) < 0) { tFreeStreamTask(pTask); @@ -402,14 +415,14 @@ int32_t streamMetaRegisterTask(SStreamMeta* pMeta, int64_t ver, SStreamTask* pTa return 0; } - taosHashPut(pMeta->pTasks, keys, sizeof(keys), &pTask, POINTER_BYTES); + taosHashPut(pMeta->pTasksMap, keys, sizeof(keys), &pTask, POINTER_BYTES); *pAdded = true; return 0; } int32_t streamMetaGetNumOfTasks(SStreamMeta* pMeta) { - size_t size = taosHashGetSize(pMeta->pTasks); - ASSERT(taosArrayGetSize(pMeta->pTaskList) == taosHashGetSize(pMeta->pTasks)); + size_t size = taosHashGetSize(pMeta->pTasksMap); + ASSERT(taosArrayGetSize(pMeta->pTaskList) == taosHashGetSize(pMeta->pTasksMap)); return (int32_t)size; } @@ -420,7 +433,7 @@ int32_t streamMetaGetNumOfStreamTasks(SStreamMeta* pMeta) { SStreamTaskId* pId = taosArrayGet(pMeta->pTaskList, i); int64_t keys[2] = {pId->streamId, pId->taskId}; - SStreamTask** p = taosHashGet(pMeta->pTasks, keys, sizeof(keys)); + SStreamTask** p = taosHashGet(pMeta->pTasksMap, keys, sizeof(keys)); if (p == NULL) { continue; } @@ -437,7 +450,7 @@ SStreamTask* streamMetaAcquireTask(SStreamMeta* pMeta, int64_t streamId, int32_t taosRLockLatch(&pMeta->lock); int64_t keys[2] = {streamId, taskId}; - SStreamTask** ppTask = (SStreamTask**)taosHashGet(pMeta->pTasks, keys, sizeof(keys)); + SStreamTask** ppTask = (SStreamTask**)taosHashGet(pMeta->pTasksMap, keys, sizeof(keys)); if (ppTask != NULL) { if (!streamTaskShouldStop(&(*ppTask)->status)) { int32_t ref = atomic_add_fetch_32(&(*ppTask)->refCnt, 1); @@ -481,7 +494,7 @@ int32_t streamMetaUnregisterTask(SStreamMeta* pMeta, int64_t streamId, int32_t t taosWLockLatch(&pMeta->lock); int64_t keys[2] = {streamId, taskId}; - SStreamTask** ppTask = (SStreamTask**)taosHashGet(pMeta->pTasks, keys, sizeof(keys)); + SStreamTask** ppTask = (SStreamTask**)taosHashGet(pMeta->pTasksMap, keys, sizeof(keys)); if (ppTask) { pTask = *ppTask; if (streamTaskShouldPause(&pTask->status)) { @@ -501,7 +514,7 @@ int32_t streamMetaUnregisterTask(SStreamMeta* pMeta, int64_t streamId, int32_t t while (1) { taosRLockLatch(&pMeta->lock); - ppTask = (SStreamTask**)taosHashGet(pMeta->pTasks, keys, sizeof(keys)); + ppTask = (SStreamTask**)taosHashGet(pMeta->pTasksMap, keys, sizeof(keys)); if (ppTask) { if ((*ppTask)->status.timerActive == 0) { @@ -520,9 +533,9 @@ int32_t streamMetaUnregisterTask(SStreamMeta* pMeta, int64_t streamId, int32_t t // let's do delete of stream task taosWLockLatch(&pMeta->lock); - ppTask = (SStreamTask**)taosHashGet(pMeta->pTasks, keys, sizeof(keys)); + ppTask = (SStreamTask**)taosHashGet(pMeta->pTasksMap, keys, sizeof(keys)); if (ppTask) { - taosHashRemove(pMeta->pTasks, keys, sizeof(keys)); + taosHashRemove(pMeta->pTasksMap, keys, sizeof(keys)); atomic_store_8(&pTask->status.taskStatus, TASK_STATUS__DROPPING); ASSERT(pTask->status.timerActive == 0); @@ -674,7 +687,7 @@ int32_t streamMetaLoadAllTasks(SStreamMeta* pMeta) { // do duplicate task check. int64_t keys[2] = {pTask->id.streamId, pTask->id.taskId}; - void* p = taosHashGet(pMeta->pTasks, keys, sizeof(keys)); + void* p = taosHashGet(pMeta->pTasksMap, keys, sizeof(keys)); if (p == NULL) { // pTask->chkInfo.checkpointVer may be 0, when a follower is become a leader // In this case, we try not to start fill-history task anymore. @@ -692,7 +705,7 @@ int32_t streamMetaLoadAllTasks(SStreamMeta* pMeta) { continue; } - if (taosHashPut(pMeta->pTasks, keys, sizeof(keys), &pTask, sizeof(void*)) < 0) { + if (taosHashPut(pMeta->pTasksMap, keys, sizeof(keys), &pTask, sizeof(void*)) < 0) { doClear(pKey, pVal, pCur, pRecycleList); tFreeStreamTask(pTask); return -1; @@ -779,8 +792,8 @@ void metaHbToMnode(void* param, void* tmrId) { } // need to stop, stop now - if (pMeta->hbInfo.stopFlag == STREAM_META_WILL_STOP) { - pMeta->hbInfo.stopFlag = STREAM_META_OK_TO_STOP; + if (pMeta->pHbInfo->stopFlag == STREAM_META_WILL_STOP) { + pMeta->pHbInfo->stopFlag = STREAM_META_OK_TO_STOP; qDebug("vgId:%d jump out of meta timer", pMeta->vgId); taosReleaseRef(streamMetaId, rid); return; @@ -793,8 +806,8 @@ void metaHbToMnode(void* param, void* tmrId) { return; } - if (!enoughTimeDuration(&pMeta->hbInfo)) { - taosTmrReset(metaHbToMnode, META_HB_CHECK_INTERVAL, param, streamEnv.timer, &pMeta->hbInfo.hbTmr); + if (!enoughTimeDuration(pMeta->pHbInfo)) { + taosTmrReset(metaHbToMnode, META_HB_CHECK_INTERVAL, param, streamEnv.timer, &pMeta->pHbInfo->hbTmr); taosReleaseRef(streamMetaId, rid); return; } @@ -813,7 +826,7 @@ void metaHbToMnode(void* param, void* tmrId) { for (int32_t i = 0; i < numOfTasks; ++i) { SStreamTaskId* pId = taosArrayGet(pMeta->pTaskList, i); int64_t keys[2] = {pId->streamId, pId->taskId}; - SStreamTask** pTask = taosHashGet(pMeta->pTasks, keys, sizeof(keys)); + SStreamTask** pTask = taosHashGet(pMeta->pTasksMap, keys, sizeof(keys)); if ((*pTask)->info.fillHistory == 1) { continue; @@ -873,7 +886,7 @@ void metaHbToMnode(void* param, void* tmrId) { } taosArrayDestroy(hbMsg.pTaskStatus); - taosTmrReset(metaHbToMnode, META_HB_CHECK_INTERVAL, param, streamEnv.timer, &pMeta->hbInfo.hbTmr); + taosTmrReset(metaHbToMnode, META_HB_CHECK_INTERVAL, param, streamEnv.timer, &pMeta->pHbInfo->hbTmr); taosReleaseRef(streamMetaId, rid); } @@ -884,7 +897,7 @@ static bool hasStreamTaskInTimer(SStreamMeta* pMeta) { void* pIter = NULL; while (1) { - pIter = taosHashIterate(pMeta->pTasks, pIter); + pIter = taosHashIterate(pMeta->pTasksMap, pIter); if (pIter == NULL) { break; } @@ -907,7 +920,7 @@ void streamMetaNotifyClose(SStreamMeta* pMeta) { void* pIter = NULL; while (1) { - pIter = taosHashIterate(pMeta->pTasks, pIter); + pIter = taosHashIterate(pMeta->pTasksMap, pIter); if (pIter == NULL) { break; } @@ -921,8 +934,8 @@ void streamMetaNotifyClose(SStreamMeta* pMeta) { // wait for the stream meta hb function stopping if (pMeta->leader) { - pMeta->hbInfo.stopFlag = STREAM_META_WILL_STOP; - while (pMeta->hbInfo.stopFlag != STREAM_META_OK_TO_STOP) { + pMeta->pHbInfo->stopFlag = STREAM_META_WILL_STOP; + while (pMeta->pHbInfo->stopFlag != STREAM_META_OK_TO_STOP) { taosMsleep(100); qDebug("vgId:%d wait for meta to stop timer", pMeta->vgId); } diff --git a/source/libs/stream/src/streamQueue.c b/source/libs/stream/src/streamQueue.c index a9d0c3b77e..6aaea2ce24 100644 --- a/source/libs/stream/src/streamQueue.c +++ b/source/libs/stream/src/streamQueue.c @@ -190,9 +190,8 @@ int32_t streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueItem** pInpu return TSDB_CODE_SUCCESS; } - STokenBucket* pBucket = &pTask->tokenBucket; - bool has = streamTaskHasAvailableToken(pBucket); - if (!has) { // no available token in th bucket, ignore this execution + STokenBucket* pBucket = pTask->pTokenBucket; + if (!streamTaskHasAvailableToken(pBucket)) { // no available token in th bucket, ignore this execution // qInfo("s-task:%s no available token for sink, capacity:%d, rate:%d token/sec, quit", pTask->id.idStr, // pBucket->capacity, pBucket->rate); return TSDB_CODE_SUCCESS; diff --git a/source/libs/stream/src/streamRecover.c b/source/libs/stream/src/streamRecover.c index 8b2a800576..db2e418171 100644 --- a/source/libs/stream/src/streamRecover.c +++ b/source/libs/stream/src/streamRecover.c @@ -40,7 +40,7 @@ static void streamTaskSetReady(SStreamTask* pTask, int32_t numOfReqs) { ASSERT(pTask->status.downstreamReady == 0); pTask->status.downstreamReady = 1; - int64_t el = (taosGetTimestampMs() - pTask->tsInfo.init); + int64_t el = (taosGetTimestampMs() - pTask->taskExecInfo.init); qDebug("s-task:%s all %d downstream ready, init completed, elapsed time:%"PRId64"ms, task status:%s", pTask->id.idStr, numOfReqs, el, streamGetTaskStatusStr(pTask->status.taskStatus)); } @@ -525,7 +525,7 @@ static void tryLaunchHistoryTask(void* param, void* tmrId) { taosWLockLatch(&pMeta->lock); int64_t keys[2] = {pInfo->streamId, pInfo->taskId}; - SStreamTask** ppTask = (SStreamTask**)taosHashGet(pMeta->pTasks, keys, sizeof(keys)); + SStreamTask** ppTask = (SStreamTask**)taosHashGet(pMeta->pTasksMap, keys, sizeof(keys)); if (ppTask) { ASSERT((*ppTask)->status.timerActive >= 1); @@ -590,7 +590,7 @@ int32_t streamLaunchFillHistoryTask(SStreamTask* pTask) { int64_t keys[2] = {pTask->historyTaskId.streamId, hTaskId}; // Set the execute conditions, including the query time window and the version range - SStreamTask** pHTask = taosHashGet(pMeta->pTasks, keys, sizeof(keys)); + SStreamTask** pHTask = taosHashGet(pMeta->pTasksMap, keys, sizeof(keys)); if (pHTask == NULL) { qWarn("s-task:%s vgId:%d failed to launch history task:0x%x, since it is not built yet", pTask->id.idStr, pMeta->vgId, hTaskId); diff --git a/source/libs/stream/src/streamTask.c b/source/libs/stream/src/streamTask.c index d42dee76f6..01318d89cd 100644 --- a/source/libs/stream/src/streamTask.c +++ b/source/libs/stream/src/streamTask.c @@ -355,6 +355,7 @@ void tFreeStreamTask(SStreamTask* pTask) { pTask->pUpstreamInfoList = NULL; } + taosMemoryFree(pTask->pTokenBucket); taosThreadMutexDestroy(&pTask->lock); taosMemoryFree(pTask); @@ -371,10 +372,10 @@ int32_t streamTaskInit(SStreamTask* pTask, SStreamMeta* pMeta, SMsgCb* pMsgCb, i if (pTask->inputInfo.queue == NULL || pTask->outputInfo.queue == NULL) { qError("s-task:%s failed to prepare the input/output queue, initialize task failed", pTask->id.idStr); - return -1; + return TSDB_CODE_OUT_OF_MEMORY; } - pTask->tsInfo.created = taosGetTimestampMs(); + pTask->taskExecInfo.created = taosGetTimestampMs(); pTask->inputInfo.status = TASK_INPUT_STATUS__NORMAL; pTask->outputInfo.status = TASK_OUTPUT_STATUS__NORMAL; pTask->pMeta = pMeta; @@ -384,19 +385,25 @@ int32_t streamTaskInit(SStreamTask* pTask, SStreamMeta* pMeta, SMsgCb* pMsgCb, i pTask->dataRange.range.minVer = ver; pTask->pMsgCb = pMsgCb; - streamTaskInitTokenBucket(&pTask->tokenBucket, 50, 50); - - TdThreadMutexAttr attr = {0}; - int ret = taosThreadMutexAttrInit(&attr); - if (ret != 0) { - qError("s-task:%s init mutex attr failed, code:%s", pTask->id.idStr, tstrerror(ret)); - return ret; + pTask->pTokenBucket = taosMemoryCalloc(1, sizeof(STokenBucket)); + if (pTask->pTokenBucket == NULL) { + qError("s-task:%s failed to prepare the tokenBucket, code:%s", pTask->id.idStr, tstrerror(TSDB_CODE_OUT_OF_MEMORY)); + return TSDB_CODE_OUT_OF_MEMORY; } - ret = taosThreadMutexAttrSetType(&attr, PTHREAD_MUTEX_RECURSIVE); - if (ret != 0) { - qError("s-task:%s set mutex attr recursive, code:%s", pTask->id.idStr, tstrerror(ret)); - return ret; + streamTaskInitTokenBucket(pTask->pTokenBucket, 50, 50); + + TdThreadMutexAttr attr = {0}; + int code = taosThreadMutexAttrInit(&attr); + if (code != 0) { + qError("s-task:%s init mutex attr failed, code:%s", pTask->id.idStr, tstrerror(code)); + return code; + } + + code = taosThreadMutexAttrSetType(&attr, PTHREAD_MUTEX_RECURSIVE); + if (code != 0) { + qError("s-task:%s set mutex attr recursive, code:%s", pTask->id.idStr, tstrerror(code)); + return code; } taosThreadMutexInit(&pTask->lock, &attr); @@ -517,7 +524,7 @@ int32_t streamTaskStop(SStreamTask* pTask) { taosMsleep(100); } - pTask->tsInfo.init = 0; + pTask->taskExecInfo.init = 0; int64_t el = taosGetTimestampMs() - st; qDebug("vgId:%d s-task:%s is closed in %" PRId64 " ms, and reset init ts", pMeta->vgId, pTask->id.idStr, el); return 0; @@ -547,10 +554,18 @@ int32_t doUpdateTaskEpset(SStreamTask* pTask, int32_t nodeId, SEpSet* pEpSet) { } int32_t streamTaskUpdateEpsetInfo(SStreamTask* pTask, SArray* pNodeList) { + STaskExecStatisInfo* p = &pTask->taskExecInfo; + qDebug("s-task:%s update task nodeEp epset, update count:%d, prevTs:%"PRId64, pTask->id.idStr, + p->taskUpdateCount + 1, p->latestUpdateTs); + + p->taskUpdateCount += 1; + p->latestUpdateTs = taosGetTimestampMs(); + for (int32_t i = 0; i < taosArrayGetSize(pNodeList); ++i) { SNodeUpdateInfo* pInfo = taosArrayGet(pNodeList, i); doUpdateTaskEpset(pTask, pInfo->nodeId, &pInfo->newEp); } + return 0; } From 98ef566eb4ea41830f4dd66b0039ad4ea3086fc3 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Fri, 15 Sep 2023 17:46:17 +0800 Subject: [PATCH 015/158] fix(stream): remove fill-history sink task. --- include/libs/stream/tstream.h | 6 ++-- source/dnode/vnode/src/tq/tq.c | 7 ++++- source/dnode/vnode/src/tq/tqSink.c | 6 ++-- source/libs/stream/src/streamExec.c | 22 +++++++------- source/libs/stream/src/streamMeta.c | 29 +++++++++++++----- source/libs/stream/src/streamTask.c | 46 +++++++++++++++++++++++++---- 6 files changed, 84 insertions(+), 32 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 1fd2f2bc13..c41834bd82 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -321,8 +321,8 @@ typedef struct { int64_t init; int64_t step1Start; int64_t step2Start; - int64_t execStart; - int32_t taskUpdateCount; + int64_t start; + int32_t updateCount; int64_t latestUpdateTs; } STaskExecStatisInfo; @@ -722,7 +722,7 @@ int32_t streamProcessCheckpointSourceReq(SStreamTask* pTask, SStreamCheckpointSo int32_t streamProcessCheckpointReadyMsg(SStreamTask* pTask); int32_t streamAlignTransferState(SStreamTask* pTask); - +int32_t streamBuildAndSendDropTaskMsg(SStreamTask* pTask, int32_t vgId, SStreamTaskId* pTaskId); int32_t streamAddCheckpointSourceRspMsg(SStreamCheckpointSourceReq* pReq, SRpcHandleInfo* pRpcInfo, SStreamTask* pTask, int8_t isSucceed); int32_t buildCheckpointSourceRsp(SStreamCheckpointSourceReq* pReq, SRpcHandleInfo* pRpcInfo, SRpcMsg* pMsg, diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 6430fee6a7..fc5300b8c7 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1097,7 +1097,8 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { tqDebug("s-task:%s fill-history task set status to be dropping", id); - streamMetaUnregisterTask(pMeta, pTask->id.streamId, pTask->id.taskId); +// streamMetaUnregisterTask(pMeta, pTask->id.streamId, pTask->id.taskId); + streamBuildAndSendDropTaskMsg(pTask, pMeta->vgId, &pTask->id); streamMetaReleaseTask(pMeta, pTask); return -1; } @@ -1347,10 +1348,14 @@ int32_t tqProcessTaskDropReq(STQ* pTq, int64_t sversion, char* msg, int32_t msgL // commit the update taosWLockLatch(&pTq->pStreamMeta->lock); + int32_t numOfTasks = streamMetaGetNumOfTasks(pTq->pStreamMeta); + tqDebug("vgId:%d task:0x%x dropped, remain tasks:%d", TD_VID(pTq->pVnode), pReq->taskId, numOfTasks); + if (streamMetaCommit(pTq->pStreamMeta) < 0) { // persist to disk } taosWUnLockLatch(&pTq->pStreamMeta->lock); + return 0; } diff --git a/source/dnode/vnode/src/tq/tqSink.c b/source/dnode/vnode/src/tq/tqSink.c index 106a4cc9b0..e0bae18545 100644 --- a/source/dnode/vnode/src/tq/tqSink.c +++ b/source/dnode/vnode/src/tq/tqSink.c @@ -274,7 +274,7 @@ int32_t doBuildAndSendSubmitMsg(SVnode* pVnode, SStreamTask* pTask, SSubmitReq2* if ((pTask->sinkRecorder.numOfSubmit % 5000) == 0) { SSinkTaskRecorder* pRec = &pTask->sinkRecorder; - double el = (taosGetTimestampMs() - pTask->taskExecInfo.execStart) / 1000.0; + double el = (taosGetTimestampMs() - pTask->taskExecInfo.start) / 1000.0; tqInfo("s-task:%s vgId:%d write %" PRId64 " blocks (%" PRId64 " rows) in %" PRId64 " submit into dst table, duration:%.2f Sec.", pTask->id.idStr, vgId, pRec->numOfBlocks, pRec->numOfRows, pRec->numOfSubmit, el); @@ -755,8 +755,8 @@ void tqSinkDataIntoDstTable(SStreamTask* pTask, void* vnode, void* data) { int32_t code = TSDB_CODE_SUCCESS; const char* id = pTask->id.idStr; - if (pTask->taskExecInfo.execStart == 0) { - pTask->taskExecInfo.execStart = taosGetTimestampMs(); + if (pTask->taskExecInfo.start == 0) { + pTask->taskExecInfo.start = taosGetTimestampMs(); } bool onlySubmitData = true; diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index 3a34d941dd..3b3dca7f5f 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -303,7 +303,8 @@ int32_t streamDoTransferStateToStreamTask(SStreamTask* pTask) { pTask->id.idStr, pTask->streamTaskId.taskId); // 1. free it and remove fill-history task from disk meta-store - streamMetaUnregisterTask(pMeta, pTask->id.streamId, pTask->id.taskId); +// streamMetaUnregisterTask(pMeta, pTask->id.streamId, pTask->id.taskId); + streamBuildAndSendDropTaskMsg(pStreamTask, pMeta->vgId, &pTask->id); // 2. save to disk taosWLockLatch(&pMeta->lock); @@ -365,7 +366,8 @@ int32_t streamDoTransferStateToStreamTask(SStreamTask* pTask) { qDebug("s-task:%s fill-history task set status to be dropping, save the state into disk", pTask->id.idStr); // 4. free it and remove fill-history task from disk meta-store - streamMetaUnregisterTask(pMeta, pTask->id.streamId, pTask->id.taskId); +// streamMetaUnregisterTask(pMeta, pTask->id.streamId, pTask->id.taskId); + streamBuildAndSendDropTaskMsg(pStreamTask, pMeta->vgId, &pTask->id); // 5. clear the link between fill-history task and stream task info pStreamTask->historyTaskId.taskId = 0; @@ -408,6 +410,8 @@ int32_t streamTransferStateToStreamTask(SStreamTask* pTask) { if (level == TASK_LEVEL__AGG || level == TASK_LEVEL__SOURCE) { // do transfer task operator states. code = streamDoTransferStateToStreamTask(pTask); + } else { // drop fill-history task + streamBuildAndSendDropTaskMsg(pTask, pTask->pMeta->vgId, &pTask->id); } return code; @@ -503,16 +507,12 @@ int32_t streamProcessTranstateBlock(SStreamTask* pTask, SStreamDataBlock* pBlock } } else { // non-dispatch task, do task state transfer directly streamFreeQitem((SStreamQueueItem*)pBlock); - if (level != TASK_LEVEL__SINK) { - qDebug("s-task:%s non-dispatch task, start to transfer state directly", id); - ASSERT(pTask->info.fillHistory == 1); - code = streamTransferStateToStreamTask(pTask); + qDebug("s-task:%s non-dispatch task, start to transfer state directly", id); + ASSERT(pTask->info.fillHistory == 1); + code = streamTransferStateToStreamTask(pTask); - if (code != TSDB_CODE_SUCCESS) { - /*int8_t status = */streamTaskSetSchedStatusInActive(pTask); - } - } else { - qDebug("s-task:%s sink task does not transfer state", id); + if (code != TSDB_CODE_SUCCESS) { + /*int8_t status = */ streamTaskSetSchedStatusInActive(pTask); } } diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 8dd93d45fc..6786f36c8e 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -47,6 +47,8 @@ struct SMetaHbInfo { tmr_h hbTmr; int32_t stopFlag; int32_t tickCounter; + int32_t hbCount; + int64_t hbStart; }; SMetaRefMgt gMetaRefMgt; @@ -333,6 +335,7 @@ void streamMetaCloseImpl(void* arg) { taosHashCleanup(pMeta->pTaskBackendUnique); taosHashCleanup(pMeta->pUpdateTaskSet); + taosMemoryFree(pMeta->pHbInfo); taosMemoryFree(pMeta->path); taosThreadMutexDestroy(&pMeta->backendMutex); @@ -785,7 +788,6 @@ static bool enoughTimeDuration(SMetaHbInfo* pInfo) { void metaHbToMnode(void* param, void* tmrId) { int64_t rid = *(int64_t*)param; - SStreamHbMsg hbMsg = {0}; SStreamMeta* pMeta = taosAcquireRef(streamMetaId, rid); if (pMeta == NULL) { return; @@ -803,31 +805,37 @@ void metaHbToMnode(void* param, void* tmrId) { if (!pMeta->leader) { qInfo("vgId:%d follower not send hb to mnode", pMeta->vgId); taosReleaseRef(streamMetaId, rid); + pMeta->pHbInfo->hbStart = 0; return; } + // set the hb start time + if (pMeta->pHbInfo->hbStart == 0) { + pMeta->pHbInfo->hbStart = taosGetTimestampMs(); + } + if (!enoughTimeDuration(pMeta->pHbInfo)) { taosTmrReset(metaHbToMnode, META_HB_CHECK_INTERVAL, param, streamEnv.timer, &pMeta->pHbInfo->hbTmr); taosReleaseRef(streamMetaId, rid); return; } - qInfo("vgId:%d start hb", pMeta->vgId); + qDebug("vgId:%d build stream task hb, leader:%d", pMeta->vgId, pMeta->leader); + SStreamHbMsg hbMsg = {0}; taosRLockLatch(&pMeta->lock); int32_t numOfTasks = streamMetaGetNumOfTasks(pMeta); SEpSet epset = {0}; bool hasValEpset = false; - hbMsg.vgId = pMeta->vgId; hbMsg.pTaskStatus = taosArrayInit(numOfTasks, sizeof(STaskStatusEntry)); for (int32_t i = 0; i < numOfTasks; ++i) { SStreamTaskId* pId = taosArrayGet(pMeta->pTaskList, i); - int64_t keys[2] = {pId->streamId, pId->taskId}; - SStreamTask** pTask = taosHashGet(pMeta->pTasksMap, keys, sizeof(keys)); + int64_t keys[2] = {pId->streamId, pId->taskId}; + SStreamTask** pTask = taosHashGet(pMeta->pTasksMap, keys, sizeof(keys)); if ((*pTask)->info.fillHistory == 1) { continue; } @@ -879,10 +887,13 @@ void metaHbToMnode(void* param, void* tmrId) { initRpcMsg(&msg, TDMT_MND_STREAM_HEARTBEAT, buf, tlen); msg.info.noResp = 1; - qDebug("vgId:%d, build and send hb to mnode", pMeta->vgId); + pMeta->pHbInfo->hbCount += 1; + + qDebug("vgId:%d, build and send hb to mnode, numOfTasks:%d total:%d", pMeta->vgId, hbMsg.numOfTasks, + pMeta->pHbInfo->hbCount); tmsgSendReq(&epset, &msg); } else { - qError("vgId:%d no mnd epset", pMeta->vgId); + qDebug("vgId:%d no tasks and no mnd epset, not send stream hb to mnode", pMeta->vgId); } taosArrayDestroy(hbMsg.pTaskStatus); @@ -915,7 +926,9 @@ static bool hasStreamTaskInTimer(SStreamMeta* pMeta) { void streamMetaNotifyClose(SStreamMeta* pMeta) { int32_t vgId = pMeta->vgId; - qDebug("vgId:%d notify all stream tasks that the vnode is closing", vgId); + qDebug("vgId:%d notify all stream tasks that the vnode is closing. isLeader:%d startHb%" PRId64 ", totalHb:%d", vgId, + pMeta->leader, pMeta->pHbInfo->hbStart, pMeta->pHbInfo->hbCount); + taosWLockLatch(&pMeta->lock); void* pIter = NULL; diff --git a/source/libs/stream/src/streamTask.c b/source/libs/stream/src/streamTask.c index 01318d89cd..23ace63d18 100644 --- a/source/libs/stream/src/streamTask.c +++ b/source/libs/stream/src/streamTask.c @@ -277,7 +277,20 @@ static void freeUpstreamItem(void* p) { void tFreeStreamTask(SStreamTask* pTask) { int32_t taskId = pTask->id.taskId; - qDebug("free s-task:0x%x, %p, state:%p", taskId, pTask, pTask->pState); + STaskExecStatisInfo* pStatis = &pTask->taskExecInfo; + + qDebug("start to free s-task:0x%x, %p, state:%p, status:%s", taskId, pTask, pTask->pState, + streamGetTaskStatusStr(pTask->status.taskStatus)); + + qDebug("s-task:0x%x exec info: create:%" PRId64 ", init:%" PRId64 ", start:%" PRId64 + ", updateCount:%d latestUpdate:%" PRId64 ", latestCheckPoint:%" PRId64 ", ver:%" PRId64 + " nextProcessVer:%" PRId64, + taskId, pStatis->created, pStatis->init, pStatis->start, pStatis->updateCount, pStatis->latestUpdateTs, + pTask->chkInfo.checkpointId, pTask->chkInfo.checkpointVer, pTask->chkInfo.nextProcessVer); + + if (pStatis->created == 0 || pStatis->init == 0 || pStatis->start == 0) { + int32_t k = 1; + } // remove the ref by timer while (pTask->status.timerActive > 0) { @@ -396,7 +409,7 @@ int32_t streamTaskInit(SStreamTask* pTask, SStreamMeta* pMeta, SMsgCb* pMsgCb, i TdThreadMutexAttr attr = {0}; int code = taosThreadMutexAttrInit(&attr); if (code != 0) { - qError("s-task:%s init mutex attr failed, code:%s", pTask->id.idStr, tstrerror(code)); + qError("s-task:%s initElapsed mutex attr failed, code:%s", pTask->id.idStr, tstrerror(code)); return code; } @@ -524,9 +537,8 @@ int32_t streamTaskStop(SStreamTask* pTask) { taosMsleep(100); } - pTask->taskExecInfo.init = 0; int64_t el = taosGetTimestampMs() - st; - qDebug("vgId:%d s-task:%s is closed in %" PRId64 " ms, and reset init ts", pMeta->vgId, pTask->id.idStr, el); + qDebug("vgId:%d s-task:%s is closed in %" PRId64 " ms", pMeta->vgId, pTask->id.idStr, el); return 0; } @@ -556,9 +568,9 @@ int32_t doUpdateTaskEpset(SStreamTask* pTask, int32_t nodeId, SEpSet* pEpSet) { int32_t streamTaskUpdateEpsetInfo(SStreamTask* pTask, SArray* pNodeList) { STaskExecStatisInfo* p = &pTask->taskExecInfo; qDebug("s-task:%s update task nodeEp epset, update count:%d, prevTs:%"PRId64, pTask->id.idStr, - p->taskUpdateCount + 1, p->latestUpdateTs); + p->updateCount + 1, p->latestUpdateTs); - p->taskUpdateCount += 1; + p->updateCount += 1; p->latestUpdateTs = taosGetTimestampMs(); for (int32_t i = 0; i < taosArrayGetSize(pNodeList); ++i) { @@ -615,3 +627,25 @@ int8_t streamTaskSetSchedStatusInActive(SStreamTask* pTask) { return status; } + +int32_t streamBuildAndSendDropTaskMsg(SStreamTask* pTask, int32_t vgId, SStreamTaskId* pTaskId) { + SVDropStreamTaskReq *pReq = rpcMallocCont(sizeof(SVDropStreamTaskReq)); + if (pReq == NULL) { + terrno = TSDB_CODE_OUT_OF_MEMORY; + return -1; + } + + pReq->head.vgId = vgId; + pReq->taskId = pTaskId->taskId; + pReq->streamId = pTaskId->streamId; + + SRpcMsg msg = {.msgType = TDMT_STREAM_TASK_DROP, .pCont = pReq, .contLen = sizeof(SVDropStreamTaskReq)}; + int32_t code = tmsgPutToQueue(pTask->pMsgCb, WRITE_QUEUE, &msg); + if (code != TSDB_CODE_SUCCESS) { + qError("vgId:%d failed to send drop task:0x%x msg, code:%s", vgId, pTaskId->taskId, tstrerror(code)); + return code; + } + + qDebug("vgId:%d build and send drop table:0x%x msg", vgId, pTaskId->taskId); + return code; +} From 7ccc273206c3a197986b6623de9c132d77723ee5 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sat, 16 Sep 2023 21:37:30 +0800 Subject: [PATCH 016/158] other: merge stream fix in main. --- include/libs/stream/tstream.h | 27 +++---- source/common/src/tglobal.c | 2 +- source/dnode/snode/src/snode.c | 18 +++-- source/dnode/vnode/src/tq/tq.c | 16 ++-- source/dnode/vnode/src/tq/tqStreamTask.c | 4 +- source/dnode/vnode/src/tq/tqStreamTaskSnap.c | 11 +-- source/libs/stream/inc/streamInt.h | 9 +++ source/libs/stream/src/streamCheckpoint.c | 7 +- source/libs/stream/src/streamExec.c | 4 +- source/libs/stream/src/streamMeta.c | 79 ++++++++++---------- source/libs/stream/src/streamRecover.c | 23 +++--- source/libs/stream/src/streamTask.c | 28 ++++--- 12 files changed, 112 insertions(+), 116 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index c41834bd82..4c359975ce 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -106,6 +106,7 @@ typedef struct { } SStreamQueueItem; typedef void FTbSink(SStreamTask* pTask, void* vnode, void* data); +typedef void FSmaSink(void* vnode, int64_t smaId, const SArray* data); typedef int32_t FTaskExpand(void* ahandle, SStreamTask* pTask, int64_t ver); typedef struct { @@ -154,8 +155,6 @@ typedef struct { int64_t size; } SStreamQueueRes; -void streamFreeQitem(SStreamQueueItem* data); - #if 0 bool streamQueueResEmpty(const SStreamQueueRes* pRes); int64_t streamQueueResSize(const SStreamQueueRes* pRes); @@ -185,12 +184,6 @@ typedef struct { int32_t streamInit(); void streamCleanUp(); -SStreamQueue* streamQueueOpen(int64_t cap); -void streamQueueClose(SStreamQueue* pQueue, int32_t taskId); -void streamQueueProcessSuccess(SStreamQueue* queue); -void streamQueueProcessFail(SStreamQueue* queue); -void* streamQueueNextItem(SStreamQueue* pQueue); - SStreamDataSubmit* streamDataSubmitNew(SPackedData* pData, int32_t type); void streamDataSubmitDestroy(SStreamDataSubmit* pDataSubmit); @@ -222,8 +215,6 @@ typedef struct { SSHashObj* pTblInfo; } STaskSinkTb; -typedef void FSmaSink(void* vnode, int64_t smaId, const SArray* data); - typedef struct { int64_t smaId; // following are not applicable to encoder and decoder @@ -244,10 +235,10 @@ typedef struct SStreamChildEpInfo { int64_t stage; // upstream task stage value, to denote if the upstream node has restart/replica changed/transfer } SStreamChildEpInfo; -typedef struct SStreamTaskKey { +typedef struct STaskId { int64_t streamId; int32_t taskId; -} SStreamTaskKey; +} STaskId; typedef struct SStreamTaskId { int64_t streamId; @@ -341,8 +332,8 @@ struct SStreamTask { SCheckpointInfo chkInfo; STaskExec exec; SDataRange dataRange; - SStreamTaskId historyTaskId; - SStreamTaskId streamTaskId; + STaskId historyTaskId; + STaskId streamTaskId; STaskExecStatisInfo taskExecInfo; SArray* pReadyMsgList; // SArray TdThreadMutex lock; // secure the operation of set task status and puting data into inputQ @@ -386,7 +377,7 @@ typedef struct SStreamMeta { TTB* pTaskDb; TTB* pCheckpointDb; SHashObj* pTasksMap; - SArray* pTaskList; // SArray + SArray* pTaskList; // SArray void* ahandle; TXN* txn; FTaskExpand* expandFunc; @@ -425,7 +416,7 @@ void tFreeStreamTask(SStreamTask* pTask); int32_t streamTaskInit(SStreamTask* pTask, SStreamMeta* pMeta, SMsgCb* pMsgCb, int64_t ver); int32_t tDecodeStreamTaskChkInfo(SDecoder* pDecoder, SCheckpointInfo* pChkpInfo); -int32_t tDecodeStreamTaskId(SDecoder* pDecoder, SStreamTaskId* pTaskId); +int32_t tDecodeStreamTaskId(SDecoder* pDecoder, STaskId* pTaskId); int32_t streamTaskPutDataIntoInputQ(SStreamTask* pTask, SStreamQueueItem* pItem); int32_t streamTaskPutDataIntoOutputQ(SStreamTask* pTask, SStreamDataBlock* pBlock); @@ -516,7 +507,7 @@ typedef struct { int32_t downstreamTaskId; int32_t upstreamNodeId; int32_t childId; -} SStreamScanHistoryFinishReq, SStreamTransferReq; +} SStreamScanHistoryFinishReq; int32_t tEncodeStreamScanHistoryFinishReq(SEncoder* pEncoder, const SStreamScanHistoryFinishReq* pReq); int32_t tDecodeStreamScanHistoryFinishReq(SDecoder* pDecoder, SStreamScanHistoryFinishReq* pReq); @@ -704,7 +695,7 @@ void streamMetaCleanup(); SStreamMeta* streamMetaOpen(const char* path, void* ahandle, FTaskExpand expandFunc, int32_t vgId, int64_t stage); void streamMetaClose(SStreamMeta* streamMeta); int32_t streamMetaSaveTask(SStreamMeta* pMeta, SStreamTask* pTask); // save to stream meta store -int32_t streamMetaRemoveTask(SStreamMeta* pMeta, int64_t* pKey); +int32_t streamMetaRemoveTask(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); diff --git a/source/common/src/tglobal.c b/source/common/src/tglobal.c index 1069c5830e..49ba20f8b0 100644 --- a/source/common/src/tglobal.c +++ b/source/common/src/tglobal.c @@ -241,7 +241,7 @@ int32_t tsTtlBatchDropNum = 10000; // number of tables dropped per batch // internal int32_t tsTransPullupInterval = 2; int32_t tsMqRebalanceInterval = 2; -int32_t tsStreamCheckpointTickInterval = 30; +int32_t tsStreamCheckpointTickInterval = 300; int32_t tsStreamNodeCheckInterval = 10; int32_t tsTtlUnit = 86400; int32_t tsTtlPushIntervalSec = 10; diff --git a/source/dnode/snode/src/snode.c b/source/dnode/snode/src/snode.c index 2b1885fb0e..ef9c1ebe2e 100644 --- a/source/dnode/snode/src/snode.c +++ b/source/dnode/snode/src/snode.c @@ -189,15 +189,17 @@ int32_t sndProcessTaskDeployReq(SSnode *pSnode, char *msg, int32_t msgLen) { int32_t sndProcessTaskDropReq(SSnode *pSnode, char *msg, int32_t msgLen) { SVDropStreamTaskReq *pReq = (SVDropStreamTaskReq *)msg; qDebug("snode:%d receive msg to drop stream task:0x%x", pSnode->pMeta->vgId, pReq->taskId); - - SStreamTask* pTask = streamMetaAcquireTask(pSnode->pMeta, pReq->streamId, pReq->taskId); - if (pTask == NULL) { - qError("vgId:%d failed to acquire s-task:0x%x when dropping it", pSnode->pMeta->vgId, pReq->taskId); - return 0; - } - streamMetaUnregisterTask(pSnode->pMeta, pReq->streamId, pReq->taskId); - streamMetaReleaseTask(pSnode->pMeta, pTask); + + // commit the update + taosWLockLatch(&pSnode->pMeta->lock); + int32_t numOfTasks = streamMetaGetNumOfTasks(pSnode->pMeta); + qDebug("vgId:%d task:0x%x dropped, remain tasks:%d", pSnode->pMeta->vgId, pReq->taskId, numOfTasks); + + if (streamMetaCommit(pSnode->pMeta) < 0) { + // persist to disk + } + taosWUnLockLatch(&pSnode->pMeta->lock); return 0; } diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index fc5300b8c7..867508bc5f 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -739,7 +739,8 @@ int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t ver) { SStreamTask* pStateTask = pTask; SStreamTask task = {0}; if (pTask->info.fillHistory) { - task.id = pTask->streamTaskId; + task.id.streamId = pTask->streamTaskId.streamId; + task.id.taskId = pTask->streamTaskId.taskId; task.pMeta = pTask->pMeta; pStateTask = &task; } @@ -773,7 +774,8 @@ int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t ver) { SStreamTask* pSateTask = pTask; SStreamTask task = {0}; if (pTask->info.fillHistory) { - task.id = pTask->streamTaskId; + task.id.streamId = pTask->streamTaskId.streamId; + task.id.taskId = pTask->streamTaskId.taskId; task.pMeta = pTask->pMeta; pSateTask = &task; } @@ -1689,9 +1691,8 @@ int32_t tqProcessTaskUpdateReq(STQ* pTq, SRpcMsg* pMsg) { taosWLockLatch(&pMeta->lock); // the task epset may be updated again and again, when replaying the WAL, the task may be in stop status. - int64_t keys[2] = {req.streamId, req.taskId}; - SStreamTask** ppTask = (SStreamTask**)taosHashGet(pMeta->pTasksMap, keys, sizeof(keys)); - + STaskId id = {.streamId = req.streamId, .taskId = req.taskId}; + SStreamTask** ppTask = (SStreamTask**)taosHashGet(pMeta->pTasksMap, &id, sizeof(id)); if (ppTask == NULL || *ppTask == NULL) { tqError("vgId:%d failed to acquire task:0x%x when handling update, it may have been dropped already", pMeta->vgId, req.taskId); @@ -1709,10 +1710,7 @@ int32_t tqProcessTaskUpdateReq(STQ* pTq, SRpcMsg* pMsg) { SStreamTask** ppHTask = NULL; if (pTask->historyTaskId.taskId != 0) { - keys[0] = pTask->historyTaskId.streamId; - keys[1] = pTask->historyTaskId.taskId; - - ppHTask = (SStreamTask**)taosHashGet(pMeta->pTasksMap, keys, sizeof(keys)); + ppHTask = (SStreamTask**)taosHashGet(pMeta->pTasksMap, &pTask->historyTaskId, sizeof(pTask->historyTaskId)); if (ppHTask == NULL || *ppHTask == NULL) { tqError("vgId:%d failed to acquire fill-history task:0x%x when handling update, it may have been dropped already", pMeta->vgId, req.taskId); diff --git a/source/dnode/vnode/src/tq/tqStreamTask.c b/source/dnode/vnode/src/tq/tqStreamTask.c index 1e66988aab..d82410e6ea 100644 --- a/source/dnode/vnode/src/tq/tqStreamTask.c +++ b/source/dnode/vnode/src/tq/tqStreamTask.c @@ -240,8 +240,8 @@ int32_t tqStartStreamTasks(STQ* pTq) { for (int32_t i = 0; i < numOfTasks; ++i) { SStreamTaskId* pTaskId = taosArrayGet(pMeta->pTaskList, i); - int64_t key[2] = {pTaskId->streamId, pTaskId->taskId}; - SStreamTask** pTask = taosHashGet(pMeta->pTasksMap, key, sizeof(key)); + STaskId id = {.streamId = pTaskId->streamId, .taskId = pTaskId->taskId}; + SStreamTask** pTask = taosHashGet(pMeta->pTasksMap, &id, sizeof(id)); int8_t status = (*pTask)->status.taskStatus; if (status == TASK_STATUS__STOP && (*pTask)->info.fillHistory != 1) { diff --git a/source/dnode/vnode/src/tq/tqStreamTaskSnap.c b/source/dnode/vnode/src/tq/tqStreamTaskSnap.c index 7b3f1aac6d..9dc918dcc7 100644 --- a/source/dnode/vnode/src/tq/tqStreamTaskSnap.c +++ b/source/dnode/vnode/src/tq/tqStreamTaskSnap.c @@ -228,25 +228,20 @@ _err: int32_t streamTaskSnapWrite(SStreamTaskWriter* pWriter, uint8_t* pData, uint32_t nData) { int32_t code = 0; STQ* pTq = pWriter->pTq; - STqHandle handle; SSnapDataHdr* pHdr = (SSnapDataHdr*)pData; if (pHdr->type == SNAP_DATA_STREAM_TASK) { - SStreamTaskId task = {0}; + STaskId taskId = {0}; SDecoder decoder; tDecoderInit(&decoder, (uint8_t*)pData + sizeof(SSnapDataHdr), nData - sizeof(SSnapDataHdr)); - - code = tDecodeStreamTaskId(&decoder, &task); + code = tDecodeStreamTaskId(&decoder, &taskId); if (code < 0) { tDecoderClear(&decoder); goto _err; } tDecoderClear(&decoder); // tdbTbInsert(TTB *pTb, const void *pKey, int keyLen, const void *pVal, int valLen, TXN *pTxn) - - taosWLockLatch(&pTq->pStreamMeta->lock); - int64_t key[2] = {task.streamId, task.taskId}; - + int64_t key[2] = {taskId.streamId, taskId.taskId}; taosWLockLatch(&pTq->pStreamMeta->lock); if (tdbTbUpsert(pTq->pStreamMeta->pTaskDb, key, sizeof(int64_t) << 1, (uint8_t*)pData + sizeof(SSnapDataHdr), nData - sizeof(SSnapDataHdr), pTq->pStreamMeta->txn) < 0) { diff --git a/source/libs/stream/inc/streamInt.h b/source/libs/stream/inc/streamInt.h index fb11ec4ea4..64df8e2f44 100644 --- a/source/libs/stream/inc/streamInt.h +++ b/source/libs/stream/inc/streamInt.h @@ -86,6 +86,15 @@ int32_t streamTransferStateToStreamTask(SStreamTask* pTask); int32_t streamTaskInitTokenBucket(STokenBucket* pBucket, int32_t cap, int32_t rate); +SStreamQueue* streamQueueOpen(int64_t cap); +void streamQueueClose(SStreamQueue* pQueue, int32_t taskId); +void streamQueueProcessSuccess(SStreamQueue* queue); +void streamQueueProcessFail(SStreamQueue* queue); +void* streamQueueNextItem(SStreamQueue* pQueue); +void streamFreeQitem(SStreamQueueItem* data); + +STaskId extractStreamTaskKey(const SStreamTask* pTask); + #ifdef __cplusplus } #endif diff --git a/source/libs/stream/src/streamCheckpoint.c b/source/libs/stream/src/streamCheckpoint.c index cfbfdb5da4..fce3526bee 100644 --- a/source/libs/stream/src/streamCheckpoint.c +++ b/source/libs/stream/src/streamCheckpoint.c @@ -266,11 +266,8 @@ int32_t streamSaveAllTaskStatus(SStreamMeta* pMeta, int64_t checkpointId) { int64_t keys[2]; for (int32_t i = 0; i < taosArrayGetSize(pMeta->pTaskList); ++i) { - SStreamTaskId* pId = taosArrayGet(pMeta->pTaskList, i); - keys[0] = pId->streamId; - keys[1] = pId->taskId; - - SStreamTask** ppTask = taosHashGet(pMeta->pTasksMap, keys, sizeof(keys)); + STaskId* pId = taosArrayGet(pMeta->pTaskList, i); + SStreamTask** ppTask = taosHashGet(pMeta->pTasksMap, pId, sizeof(*pId)); if (ppTask == NULL) { continue; } diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index 3b3dca7f5f..91c46c8ad9 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -300,10 +300,9 @@ int32_t streamDoTransferStateToStreamTask(SStreamTask* pTask) { qError( "s-task:%s failed to find related stream task:0x%x, it may have been destroyed or closed, destroy the related " "fill-history task", - pTask->id.idStr, pTask->streamTaskId.taskId); + pTask->id.idStr, (int32_t) pTask->streamTaskId.taskId); // 1. free it and remove fill-history task from disk meta-store -// streamMetaUnregisterTask(pMeta, pTask->id.streamId, pTask->id.taskId); streamBuildAndSendDropTaskMsg(pStreamTask, pMeta->vgId, &pTask->id); // 2. save to disk @@ -371,6 +370,7 @@ int32_t streamDoTransferStateToStreamTask(SStreamTask* pTask) { // 5. clear the link between fill-history task and stream task info pStreamTask->historyTaskId.taskId = 0; + pStreamTask->historyTaskId.streamId = 0; // 6. save to disk taosWLockLatch(&pMeta->lock); diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 6786f36c8e..d3e57433a4 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -36,7 +36,6 @@ static void metaHbToMnode(void* param, void* tmrId); static void streamMetaClear(SStreamMeta* pMeta); static int32_t streamMetaBegin(SStreamMeta* pMeta); static void streamMetaCloseImpl(void* arg); -static void extractStreamTaskKey(int64_t* pKey, const SStreamTask* pTask); typedef struct { TdThreadMutex mutex; @@ -362,10 +361,8 @@ int32_t streamMetaSaveTask(SStreamMeta* pMeta, SStreamTask* pTask) { tEncodeStreamTask(&encoder, pTask); tEncoderClear(&encoder); - int64_t key[2] = {0}; - extractStreamTaskKey(key, pTask); - - if (tdbTbUpsert(pMeta->pTaskDb, key, STREAM_TASK_KEY_LEN, buf, len, pMeta->txn) < 0) { + int64_t id[2] = {pTask->id.streamId, pTask->id.taskId}; + if (tdbTbUpsert(pMeta->pTaskDb, id, STREAM_TASK_KEY_LEN, buf, len, pMeta->txn) < 0) { qError("s-task:%s save to disk failed, code:%s", pTask->id.idStr, tstrerror(terrno)); return -1; } @@ -374,18 +371,14 @@ int32_t streamMetaSaveTask(SStreamMeta* pMeta, SStreamTask* pTask) { return 0; } -void extractStreamTaskKey(int64_t* pKey, const SStreamTask* pTask) { - pKey[0] = pTask->id.streamId; - pKey[1] = pTask->id.taskId; -} - -int32_t streamMetaRemoveTask(SStreamMeta* pMeta, int64_t* pKey) { - int32_t code = tdbTbDelete(pMeta->pTaskDb, pKey, STREAM_TASK_KEY_LEN, pMeta->txn); +int32_t streamMetaRemoveTask(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) { - qError("vgId:%d failed to remove task:0x%x from metastore, code:%s", pMeta->vgId, (int32_t)pKey[1], + qError("vgId:%d failed to remove task:0x%x from metastore, code:%s", pMeta->vgId, (int32_t) pTaskId->taskId, tstrerror(terrno)); } else { - qDebug("vgId:%d remove task:0x%x from metastore", pMeta->vgId, (int32_t)pKey[1]); + qDebug("vgId:%d remove task:0x%x from metastore", pMeta->vgId, (int32_t) pTaskId->taskId); } return code; @@ -395,8 +388,8 @@ int32_t streamMetaRemoveTask(SStreamMeta* pMeta, int64_t* pKey) { int32_t streamMetaRegisterTask(SStreamMeta* pMeta, int64_t ver, SStreamTask* pTask, bool* pAdded) { *pAdded = false; - int64_t keys[2] = {pTask->id.streamId, pTask->id.taskId}; - void* p = taosHashGet(pMeta->pTasksMap, keys, sizeof(keys)); + STaskId id = {.streamId = pTask->id.streamId, .taskId = pTask->id.taskId}; + void* p = taosHashGet(pMeta->pTasksMap, &id, sizeof(id)); if (p == NULL) { if (pMeta->expandFunc(pMeta->ahandle, pTask, ver) < 0) { tFreeStreamTask(pTask); @@ -418,7 +411,7 @@ int32_t streamMetaRegisterTask(SStreamMeta* pMeta, int64_t ver, SStreamTask* pTa return 0; } - taosHashPut(pMeta->pTasksMap, keys, sizeof(keys), &pTask, POINTER_BYTES); + taosHashPut(pMeta->pTasksMap, &id, sizeof(id), &pTask, POINTER_BYTES); *pAdded = true; return 0; } @@ -433,10 +426,8 @@ int32_t streamMetaGetNumOfStreamTasks(SStreamMeta* pMeta) { int32_t num = 0; size_t size = taosArrayGetSize(pMeta->pTaskList); for (int32_t i = 0; i < size; ++i) { - SStreamTaskId* pId = taosArrayGet(pMeta->pTaskList, i); - int64_t keys[2] = {pId->streamId, pId->taskId}; - - SStreamTask** p = taosHashGet(pMeta->pTasksMap, keys, sizeof(keys)); + STaskId* pId = taosArrayGet(pMeta->pTaskList, i); + SStreamTask** p = taosHashGet(pMeta->pTasksMap, pId, sizeof(*pId)); if (p == NULL) { continue; } @@ -452,8 +443,8 @@ int32_t streamMetaGetNumOfStreamTasks(SStreamMeta* pMeta) { SStreamTask* streamMetaAcquireTask(SStreamMeta* pMeta, int64_t streamId, int32_t taskId) { taosRLockLatch(&pMeta->lock); - int64_t keys[2] = {streamId, taskId}; - SStreamTask** ppTask = (SStreamTask**)taosHashGet(pMeta->pTasksMap, keys, sizeof(keys)); + STaskId id = {.streamId = streamId, .taskId = taskId}; + SStreamTask** ppTask = (SStreamTask**)taosHashGet(pMeta->pTasksMap, &id, sizeof(id)); if (ppTask != NULL) { if (!streamTaskShouldStop(&(*ppTask)->status)) { int32_t ref = atomic_add_fetch_32(&(*ppTask)->refCnt, 1); @@ -496,8 +487,8 @@ int32_t streamMetaUnregisterTask(SStreamMeta* pMeta, int64_t streamId, int32_t t // pre-delete operation taosWLockLatch(&pMeta->lock); - int64_t keys[2] = {streamId, taskId}; - SStreamTask** ppTask = (SStreamTask**)taosHashGet(pMeta->pTasksMap, keys, sizeof(keys)); + STaskId id = {.streamId = streamId, .taskId = taskId}; + SStreamTask** ppTask = (SStreamTask**)taosHashGet(pMeta->pTasksMap, &id, sizeof(id)); if (ppTask) { pTask = *ppTask; if (streamTaskShouldPause(&pTask->status)) { @@ -517,7 +508,7 @@ int32_t streamMetaUnregisterTask(SStreamMeta* pMeta, int64_t streamId, int32_t t while (1) { taosRLockLatch(&pMeta->lock); - ppTask = (SStreamTask**)taosHashGet(pMeta->pTasksMap, keys, sizeof(keys)); + ppTask = (SStreamTask**)taosHashGet(pMeta->pTasksMap, &id, sizeof(id)); if (ppTask) { if ((*ppTask)->status.timerActive == 0) { @@ -536,9 +527,9 @@ int32_t streamMetaUnregisterTask(SStreamMeta* pMeta, int64_t streamId, int32_t t // let's do delete of stream task taosWLockLatch(&pMeta->lock); - ppTask = (SStreamTask**)taosHashGet(pMeta->pTasksMap, keys, sizeof(keys)); + ppTask = (SStreamTask**)taosHashGet(pMeta->pTasksMap, &id, sizeof(id)); if (ppTask) { - taosHashRemove(pMeta->pTasksMap, keys, sizeof(keys)); + taosHashRemove(pMeta->pTasksMap, &id, sizeof(id)); atomic_store_8(&pTask->status.taskStatus, TASK_STATUS__DROPPING); ASSERT(pTask->status.timerActive == 0); @@ -551,7 +542,17 @@ int32_t streamMetaUnregisterTask(SStreamMeta* pMeta, int64_t streamId, int32_t t streamMetaReleaseTask(pMeta, pTask); } - streamMetaRemoveTask(pMeta, keys); + // it is an fill-history task, remove the related stream task's id that points to it + if ((*ppTask)->info.fillHistory == 1) { + STaskId id1 = {.streamId = (*ppTask)->streamTaskId.streamId, .taskId = (*ppTask)->streamTaskId.taskId}; + SStreamTask** ppStreamTask = (SStreamTask**)taosHashGet(pMeta->pTasksMap, &id1, sizeof(id1)); + if (ppStreamTask != NULL) { + (*ppStreamTask)->historyTaskId.taskId = 0; + (*ppStreamTask)->historyTaskId.streamId = 0; + } + } + + streamMetaRemoveTask(pMeta, &id); streamMetaReleaseTask(pMeta, pTask); } else { qDebug("vgId:%d failed to find the task:0x%x, it may have been dropped already", pMeta->vgId, taskId); @@ -652,7 +653,7 @@ int32_t streamMetaLoadAllTasks(SStreamMeta* pMeta) { void* pVal = NULL; int32_t vLen = 0; SDecoder decoder; - SArray* pRecycleList = taosArrayInit(4, STREAM_TASK_KEY_LEN); + SArray* pRecycleList = taosArrayInit(4, sizeof(STaskId)); tdbTbcMoveToFirst(pCur); while (tdbTbcNext(pCur, &pKey, &kLen, &pVal, &vLen) == 0) { @@ -679,18 +680,17 @@ int32_t streamMetaLoadAllTasks(SStreamMeta* pMeta) { int32_t taskId = pTask->id.taskId; tFreeStreamTask(pTask); - int64_t key[2] = {0}; - extractStreamTaskKey(key, pTask); + STaskId id = extractStreamTaskKey(pTask); - taosArrayPush(pRecycleList, key); + taosArrayPush(pRecycleList, &id); int32_t total = taosArrayGetSize(pRecycleList); qDebug("s-task:0x%x is already dropped, add into recycle list, total:%d", taskId, total); continue; } // do duplicate task check. - int64_t keys[2] = {pTask->id.streamId, pTask->id.taskId}; - void* p = taosHashGet(pMeta->pTasksMap, keys, sizeof(keys)); + STaskId id = {.streamId = pTask->id.streamId, .taskId = pTask->id.taskId}; + void* p = taosHashGet(pMeta->pTasksMap, &id, sizeof(id)); if (p == NULL) { // pTask->chkInfo.checkpointVer may be 0, when a follower is become a leader // In this case, we try not to start fill-history task anymore. @@ -708,7 +708,7 @@ int32_t streamMetaLoadAllTasks(SStreamMeta* pMeta) { continue; } - if (taosHashPut(pMeta->pTasksMap, keys, sizeof(keys), &pTask, sizeof(void*)) < 0) { + if (taosHashPut(pMeta->pTasksMap, &id, sizeof(id), &pTask, POINTER_BYTES) < 0) { doClear(pKey, pVal, pCur, pRecycleList); tFreeStreamTask(pTask); return -1; @@ -732,7 +732,7 @@ int32_t streamMetaLoadAllTasks(SStreamMeta* pMeta) { if (taosArrayGetSize(pRecycleList) > 0) { for (int32_t i = 0; i < taosArrayGetSize(pRecycleList); ++i) { - int64_t* pId = taosArrayGet(pRecycleList, i); + STaskId* pId = taosArrayGet(pRecycleList, i); streamMetaRemoveTask(pMeta, pId); } } @@ -832,10 +832,9 @@ void metaHbToMnode(void* param, void* tmrId) { hbMsg.pTaskStatus = taosArrayInit(numOfTasks, sizeof(STaskStatusEntry)); for (int32_t i = 0; i < numOfTasks; ++i) { - SStreamTaskId* pId = taosArrayGet(pMeta->pTaskList, i); + STaskId* pId = taosArrayGet(pMeta->pTaskList, i); - int64_t keys[2] = {pId->streamId, pId->taskId}; - SStreamTask** pTask = taosHashGet(pMeta->pTasksMap, keys, sizeof(keys)); + SStreamTask** pTask = taosHashGet(pMeta->pTasksMap, pId, sizeof(*pId)); if ((*pTask)->info.fillHistory == 1) { continue; } diff --git a/source/libs/stream/src/streamRecover.c b/source/libs/stream/src/streamRecover.c index db2e418171..7a318e2310 100644 --- a/source/libs/stream/src/streamRecover.c +++ b/source/libs/stream/src/streamRecover.c @@ -21,8 +21,7 @@ typedef struct SStreamTaskRetryInfo { SStreamMeta* pMeta; - int32_t taskId; - int64_t streamId; + STaskId id; } SStreamTaskRetryInfo; static int32_t streamSetParamForScanHistory(SStreamTask* pTask); @@ -520,12 +519,10 @@ static void tryLaunchHistoryTask(void* param, void* tmrId) { SStreamTaskRetryInfo* pInfo = param; SStreamMeta* pMeta = pInfo->pMeta; - qDebug("s-task:0x%x in timer to launch related history task", pInfo->taskId); + qDebug("s-task:0x%x in timer to launch related history task", (int32_t) pInfo->id.taskId); taosWLockLatch(&pMeta->lock); - int64_t keys[2] = {pInfo->streamId, pInfo->taskId}; - - SStreamTask** ppTask = (SStreamTask**)taosHashGet(pMeta->pTasksMap, keys, sizeof(keys)); + SStreamTask** ppTask = (SStreamTask**)taosHashGet(pMeta->pTasksMap, &pInfo->id, sizeof(pInfo->id)); if (ppTask) { ASSERT((*ppTask)->status.timerActive >= 1); @@ -541,7 +538,7 @@ static void tryLaunchHistoryTask(void* param, void* tmrId) { } taosWUnLockLatch(&pMeta->lock); - SStreamTask* pTask = streamMetaAcquireTask(pMeta, pInfo->streamId, pInfo->taskId); + SStreamTask* pTask = streamMetaAcquireTask(pMeta, pInfo->id.streamId, pInfo->id.taskId); if (pTask != NULL) { ASSERT(pTask->status.timerActive >= 1); @@ -552,7 +549,7 @@ static void tryLaunchHistoryTask(void* param, void* tmrId) { qWarn( "s-task:%s vgId:%d status:%s failed to launch history task:0x%x, since it may not be built, or may have been " "destroyed, or should stop", - pTask->id.idStr, pMeta->vgId, pStatus, pTask->historyTaskId.taskId); + pTask->id.idStr, pMeta->vgId, pStatus, (int32_t) pTask->historyTaskId.taskId); taosTmrReset(tryLaunchHistoryTask, 100, pInfo, streamEnv.timer, &pTask->launchTaskTimer); streamMetaReleaseTask(pMeta, pTask); @@ -568,7 +565,7 @@ static void tryLaunchHistoryTask(void* param, void* tmrId) { atomic_sub_fetch_8(&pTask->status.timerActive, 1); streamMetaReleaseTask(pMeta, pTask); } else { - qError("s-task:0x%x failed to load task, it may have been destroyed", pInfo->taskId); + qError("s-task:0x%x failed to load task, it may have been destroyed", (int32_t) pInfo->id.taskId); } taosMemoryFree(pInfo); @@ -587,17 +584,15 @@ int32_t streamLaunchFillHistoryTask(SStreamTask* pTask) { qDebug("s-task:%s start to launch related fill-history task:0x%" PRIx64 "-0x%x", pTask->id.idStr, pTask->historyTaskId.streamId, hTaskId); - int64_t keys[2] = {pTask->historyTaskId.streamId, hTaskId}; - // Set the execute conditions, including the query time window and the version range - SStreamTask** pHTask = taosHashGet(pMeta->pTasksMap, keys, sizeof(keys)); + SStreamTask** pHTask = taosHashGet(pMeta->pTasksMap, &pTask->historyTaskId, sizeof(pTask->historyTaskId)); if (pHTask == NULL) { qWarn("s-task:%s vgId:%d failed to launch history task:0x%x, since it is not built yet", pTask->id.idStr, pMeta->vgId, hTaskId); SStreamTaskRetryInfo* pInfo = taosMemoryCalloc(1, sizeof(SStreamTaskRetryInfo)); - pInfo->taskId = pTask->id.taskId; - pInfo->streamId = pTask->id.streamId; + pInfo->id.taskId = pTask->id.taskId; + pInfo->id.streamId = pTask->id.streamId; pInfo->pMeta = pTask->pMeta; if (pTask->launchTaskTimer == NULL) { diff --git a/source/libs/stream/src/streamTask.c b/source/libs/stream/src/streamTask.c index 23ace63d18..117c795a8d 100644 --- a/source/libs/stream/src/streamTask.c +++ b/source/libs/stream/src/streamTask.c @@ -251,7 +251,8 @@ int32_t tDecodeStreamTaskChkInfo(SDecoder* pDecoder, SCheckpointInfo* pChkpInfo) tEndDecode(pDecoder); return 0; } -int32_t tDecodeStreamTaskId(SDecoder* pDecoder, SStreamTaskId* pTaskId) { + +int32_t tDecodeStreamTaskId(SDecoder* pDecoder, STaskId* pTaskId) { int64_t ver; if (tStartDecode(pDecoder) < 0) return -1; if (tDecodeI64(pDecoder, &ver) < 0) return -1; @@ -478,8 +479,8 @@ void streamTaskUpdateUpstreamInfo(SStreamTask* pTask, int32_t nodeId, const SEpS SStreamChildEpInfo* pInfo = taosArrayGetP(pTask->pUpstreamInfoList, i); if (pInfo->nodeId == nodeId) { epsetAssign(&pInfo->epSet, pEpSet); - qDebug("s-task:0x%x update the upstreamInfo, nodeId:%d taskId:0x%x newEpset:%s", pTask->id.taskId, nodeId, - pInfo->taskId, buf); + qDebug("s-task:0x%x update the upstreamInfo taskId:0x%x(nodeId:%d) newEpset:%s", pTask->id.taskId, + pInfo->taskId, nodeId, buf); break; } } @@ -509,7 +510,8 @@ void streamTaskUpdateDownstreamInfo(SStreamTask* pTask, int32_t nodeId, const SE if (pVgInfo->vgId == nodeId) { epsetAssign(&pVgInfo->epSet, pEpSet); - qDebug("s-task:0x%x update the dispatch info, nodeId:%d newEpset:%s", pTask->id.taskId, nodeId, buf); + qDebug("s-task:0x%x update the dispatch info, task:0x%x(nodeId:%d) newEpset:%s", pTask->id.taskId, + pVgInfo->taskId, nodeId, buf); break; } } @@ -517,7 +519,8 @@ void streamTaskUpdateDownstreamInfo(SStreamTask* pTask, int32_t nodeId, const SE STaskDispatcherFixedEp* pDispatcher = &pTask->fixedEpDispatcher; if (pDispatcher->nodeId == nodeId) { epsetAssign(&pDispatcher->epSet, pEpSet); - qDebug("s-task:0x%x update the dispatch info, nodeId:%d newEpSet:%s", pTask->id.taskId, nodeId, buf); + qDebug("s-task:0x%x update the dispatch info, task:0x%x(nodeId:%d) newEpSet:%s", pTask->id.taskId, + pDispatcher->taskId, nodeId, buf); } } else { // do nothing @@ -567,17 +570,19 @@ int32_t doUpdateTaskEpset(SStreamTask* pTask, int32_t nodeId, SEpSet* pEpSet) { int32_t streamTaskUpdateEpsetInfo(SStreamTask* pTask, SArray* pNodeList) { STaskExecStatisInfo* p = &pTask->taskExecInfo; - qDebug("s-task:%s update task nodeEp epset, update count:%d, prevTs:%"PRId64, pTask->id.idStr, - p->updateCount + 1, p->latestUpdateTs); - p->updateCount += 1; + int32_t numOfNodes = taosArrayGetSize(pNodeList); + int64_t prevTs = p->latestUpdateTs; + p->latestUpdateTs = taosGetTimestampMs(); + p->updateCount += 1; + qDebug("s-task:%s update task nodeEp epset, updatedNodes:%d, updateCount:%d, prevTs:%" PRId64, pTask->id.idStr, + numOfNodes, p->updateCount, prevTs); for (int32_t i = 0; i < taosArrayGetSize(pNodeList); ++i) { SNodeUpdateInfo* pInfo = taosArrayGet(pNodeList, i); doUpdateTaskEpset(pTask, pInfo->nodeId, &pInfo->newEp); } - return 0; } @@ -649,3 +654,8 @@ int32_t streamBuildAndSendDropTaskMsg(SStreamTask* pTask, int32_t vgId, SStreamT qDebug("vgId:%d build and send drop table:0x%x msg", vgId, pTaskId->taskId); return code; } + +STaskId extractStreamTaskKey(const SStreamTask* pTask) { + STaskId id = {.streamId = pTask->id.streamId, .taskId = pTask->id.taskId}; + return id; +} \ No newline at end of file From 0459a4628c817fd06ac48e8ee4a105d40a618471 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sun, 17 Sep 2023 01:19:59 +0800 Subject: [PATCH 017/158] fix(stream): fix stream task id error. --- include/libs/stream/tstream.h | 9 ++--- source/dnode/mnode/impl/src/mndStream.c | 47 ++++++++--------------- source/dnode/vnode/src/inc/vnodeInt.h | 1 - source/dnode/vnode/src/tq/tq.c | 16 ++++---- source/dnode/vnode/src/tq/tqStreamTask.c | 2 +- source/libs/stream/src/streamCheckpoint.c | 7 +--- source/libs/stream/src/streamMeta.c | 36 ++++++++++------- source/libs/stream/src/streamRecover.c | 10 +++-- source/libs/stream/src/streamTask.c | 14 +++++-- 9 files changed, 71 insertions(+), 71 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 4c359975ce..2d70bb1e1c 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -237,7 +237,7 @@ typedef struct SStreamChildEpInfo { typedef struct STaskId { int64_t streamId; - int32_t taskId; + int64_t taskId; } STaskId; typedef struct SStreamTaskId { @@ -393,7 +393,8 @@ typedef struct SStreamMeta { TdThreadMutex backendMutex; SMetaHbInfo* pHbInfo; SHashObj* pUpdateTaskSet; - int32_t totalTasks; // this value should be increased when a new task is added into the meta + int32_t numOfStreamTasks; // this value should be increased when a new task is added into the meta + int32_t numOfPausedTasks; int32_t chkptNotReadyTasks; int64_t rid; @@ -402,7 +403,6 @@ typedef struct SStreamMeta { SArray* chkpInUse; int32_t chkpCap; SRWLatch chkpDirLock; - int32_t pauseTaskNum; } SStreamMeta; int32_t tEncodeStreamEpInfo(SEncoder* pEncoder, const SStreamChildEpInfo* pInfo); @@ -553,8 +553,7 @@ int32_t tEncodeStreamCheckpointReadyMsg(SEncoder* pEncoder, const SStreamCheckpo int32_t tDecodeStreamCheckpointReadyMsg(SDecoder* pDecoder, SStreamCheckpointReadyMsg* pRsp); typedef struct STaskStatusEntry { - int64_t streamId; - int32_t taskId; + STaskId id; int32_t status; } STaskStatusEntry; diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index cbc8e1e099..28d4716f0e 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -1194,7 +1194,7 @@ static int32_t mndProcessStreamDoCheckpoint(SRpcMsg *pReq) { STaskStatusEntry *p = taosArrayGet(execNodeList.pTaskList, i); if (p->status != TASK_STATUS__NORMAL) { mDebug("s-task:0x%" PRIx64 "-0x%x (nodeId:%d) status:%s not ready, create checkpoint msg not issued", - p->streamId, p->taskId, 0, streamGetTaskStatusStr(p->status)); + p->id.streamId, (int32_t)p->id.taskId, 0, streamGetTaskStatusStr(p->status)); ready = false; break; } @@ -1564,29 +1564,17 @@ static int32_t mndRetrieveStreamTask(SRpcMsg *pReq, SShowObj *pShow, SSDataBlock // status char status[20 + VARSTR_HEADER_SIZE] = {0}; - int8_t taskStatus = atomic_load_8(&pTask->status.taskStatus); - if (taskStatus == TASK_STATUS__NORMAL) { - memcpy(varDataVal(status), "normal", 6); - varDataSetLen(status, 6); - } else if (taskStatus == TASK_STATUS__DROPPING) { - memcpy(varDataVal(status), "dropping", 8); - varDataSetLen(status, 8); - } else if (taskStatus == TASK_STATUS__UNINIT) { - memcpy(varDataVal(status), "uninit", 6); - varDataSetLen(status, 4); - } else if (taskStatus == TASK_STATUS__STOP) { - memcpy(varDataVal(status), "stop", 4); - varDataSetLen(status, 4); - } else if (taskStatus == TASK_STATUS__SCAN_HISTORY) { - memcpy(varDataVal(status), "history", 7); - varDataSetLen(status, 7); - } else if (taskStatus == TASK_STATUS__HALT) { - memcpy(varDataVal(status), "halt", 4); - varDataSetLen(status, 4); - } else if (taskStatus == TASK_STATUS__PAUSE) { - memcpy(varDataVal(status), "pause", 5); - varDataSetLen(status, 5); + + STaskId id = {.streamId = pTask->id.streamId, .taskId = pTask->id.taskId}; + int32_t *index = taosHashGet(execNodeList.pTaskMap, &id, sizeof(id)); + if (index == NULL) { + continue; } + + STaskStatusEntry *pStatusEntry = taosArrayGet(execNodeList.pTaskList, *index); + const char* pStatus = streamGetTaskStatusStr(pStatusEntry->status); + STR_TO_VARSTR(status, pStatus); + pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); colDataSetVal(pColInfo, numOfRows, (const char *)&status, false); @@ -2269,16 +2257,16 @@ static void keepStreamTasksInBuf(SStreamObj *pStream, SStreamVnodeRevertIndex *p int32_t numOfTasks = taosArrayGetSize(pLevel); for (int32_t j = 0; j < numOfTasks; j++) { SStreamTask *pTask = taosArrayGetP(pLevel, j); - int64_t keys[2] = {pTask->id.streamId, pTask->id.taskId}; - void *p = taosHashGet(pExecNode->pTaskMap, keys, sizeof(keys)); + STaskId id = {.streamId = pTask->id.streamId, .taskId = pTask->id.taskId}; + void *p = taosHashGet(pExecNode->pTaskMap, &id, sizeof(id)); if (p == NULL) { STaskStatusEntry entry = { - .streamId = pTask->id.streamId, .taskId = pTask->id.taskId, .status = TASK_STATUS__STOP}; + .id.streamId = pTask->id.streamId, .id.taskId = pTask->id.taskId, .status = TASK_STATUS__STOP}; taosArrayPush(pExecNode->pTaskList, &entry); int32_t ordinal = taosArrayGetSize(pExecNode->pTaskList) - 1; - taosHashPut(pExecNode->pTaskMap, keys, sizeof(keys), &ordinal, sizeof(ordinal)); + taosHashPut(pExecNode->pTaskMap, &id, sizeof(id), &ordinal, sizeof(ordinal)); } } } @@ -2311,8 +2299,7 @@ int32_t mndProcessStreamHb(SRpcMsg *pReq) { for (int32_t i = 0; i < req.numOfTasks; ++i) { STaskStatusEntry *p = taosArrayGet(req.pTaskStatus, i); - int64_t k[2] = {p->streamId, p->taskId}; - int32_t *index = taosHashGet(execNodeList.pTaskMap, &k, sizeof(k)); + int32_t *index = taosHashGet(execNodeList.pTaskMap, &p->id, sizeof(p->id)); if (index == NULL) { continue; } @@ -2320,7 +2307,7 @@ int32_t mndProcessStreamHb(SRpcMsg *pReq) { STaskStatusEntry *pStatusEntry = taosArrayGet(execNodeList.pTaskList, *index); pStatusEntry->status = p->status; if (p->status != TASK_STATUS__NORMAL) { - mDebug("received s-task:0x%x not in ready status:%s", p->taskId, streamGetTaskStatusStr(p->status)); + mDebug("received s-task:0x%"PRIx64" not in ready status:%s", p->id.taskId, streamGetTaskStatusStr(p->status)); } } taosThreadMutexUnlock(&execNodeList.lock); diff --git a/source/dnode/vnode/src/inc/vnodeInt.h b/source/dnode/vnode/src/inc/vnodeInt.h index 536273c044..39f3d465f2 100644 --- a/source/dnode/vnode/src/inc/vnodeInt.h +++ b/source/dnode/vnode/src/inc/vnodeInt.h @@ -259,7 +259,6 @@ int32_t tqProcessTaskRetrieveRsp(STQ* pTq, SRpcMsg* pMsg); int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg); int32_t tqProcessTaskScanHistoryFinishReq(STQ* pTq, SRpcMsg* pMsg); int32_t tqProcessTaskScanHistoryFinishRsp(STQ* pTq, SRpcMsg* pMsg); -int32_t tqCheckLogInWal(STQ* pTq, int64_t version); // sma int32_t smaInit(); diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 867508bc5f..fe1ef1637f 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -860,14 +860,14 @@ int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t ver) { " child id:%d, level:%d, status:%s fill-history:%d, related stream task:0x%x trigger:%" PRId64 " ms", vgId, pTask->id.idStr, pChkInfo->checkpointId, pChkInfo->checkpointVer, pChkInfo->nextProcessVer, pTask->info.selfChildId, pTask->info.taskLevel, streamGetTaskStatusStr(pTask->status.taskStatus), - pTask->info.fillHistory, pTask->streamTaskId.taskId, pTask->info.triggerParam); + pTask->info.fillHistory, (int32_t)pTask->streamTaskId.taskId, pTask->info.triggerParam); } else { tqInfo("vgId:%d expand stream task, s-task:%s, checkpointId:%" PRId64 " checkpointVer:%" PRId64 " nextProcessVer:%" PRId64 " child id:%d, level:%d, status:%s fill-history:%d, related fill-task:0x%x trigger:%" PRId64 " ms", vgId, pTask->id.idStr, pChkInfo->checkpointId, pChkInfo->checkpointVer, pChkInfo->nextProcessVer, pTask->info.selfChildId, pTask->info.taskLevel, streamGetTaskStatusStr(pTask->status.taskStatus), - pTask->info.fillHistory, pTask->historyTaskId.taskId, pTask->info.triggerParam); + pTask->info.fillHistory, (int32_t)pTask->historyTaskId.taskId, pTask->info.triggerParam); } return 0; @@ -1094,7 +1094,7 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { pStreamTask = streamMetaAcquireTask(pMeta, pTask->streamTaskId.streamId, pTask->streamTaskId.taskId); if (pStreamTask == NULL) { // todo delete this task, if the related stream task is dropped - qError("failed to find s-task:0x%x, it may have been destroyed, drop fill-history task:%s", + qError("failed to find s-task:0x%"PRIx64", it may have been destroyed, drop fill-history task:%s", pTask->streamTaskId.taskId, pTask->id.idStr); tqDebug("s-task:%s fill-history task set status to be dropping", id); @@ -1380,7 +1380,8 @@ int32_t tqProcessTaskPauseReq(STQ* pTq, int64_t sversion, char* msg, int32_t msg if (pTask->historyTaskId.taskId != 0) { pHistoryTask = streamMetaAcquireTask(pMeta, pTask->historyTaskId.streamId, pTask->historyTaskId.taskId); if (pHistoryTask == NULL) { - tqError("vgId:%d process pause req, failed to acquire fill-history task:0x%x, it may have been dropped already", + tqError("vgId:%d process pause req, failed to acquire fill-history task:0x%" PRIx64 + ", it may have been dropped already", pMeta->vgId, pTask->historyTaskId.taskId); streamMetaReleaseTask(pMeta, pTask); @@ -1560,8 +1561,6 @@ FAIL: return -1; } -int32_t tqCheckLogInWal(STQ* pTq, int64_t sversion) { return sversion <= pTq->walLogLastVer; } - // todo error code cannot be return, since this is invoked by an mnode-launched transaction. int32_t tqProcessStreamCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg) { int32_t vgId = TD_VID(pTq->pVnode); @@ -1611,11 +1610,10 @@ int32_t tqProcessStreamCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg) { // set the initial value for generating check point // set the mgmt epset info according to the checkout source msg from mnode, todo update mgmt epset if needed if (pMeta->chkptNotReadyTasks == 0) { - pMeta->chkptNotReadyTasks = streamMetaGetNumOfStreamTasks(pMeta); - pMeta->totalTasks = pMeta->chkptNotReadyTasks; + pMeta->chkptNotReadyTasks = pMeta->numOfStreamTasks; } - total = taosArrayGetSize(pMeta->pTaskList); + total = pMeta->numOfStreamTasks; taosWUnLockLatch(&pMeta->lock); qDebug("s-task:%s (vgId:%d) level:%d receive checkpoint-source msg, chkpt:%" PRId64 ", total checkpoint req:%d", diff --git a/source/dnode/vnode/src/tq/tqStreamTask.c b/source/dnode/vnode/src/tq/tqStreamTask.c index d82410e6ea..3cba4567fe 100644 --- a/source/dnode/vnode/src/tq/tqStreamTask.c +++ b/source/dnode/vnode/src/tq/tqStreamTask.c @@ -166,7 +166,7 @@ int32_t tqScanWalAsync(STQ* pTq, bool ckPause) { return 0; } - int32_t numOfPauseTasks = pTq->pStreamMeta->pauseTaskNum; + int32_t numOfPauseTasks = pTq->pStreamMeta->numOfPausedTasks; if (ckPause && numOfTasks == numOfPauseTasks) { tqDebug("vgId:%d ignore all submit, all streams had been paused, reset the walScanCounter", vgId); diff --git a/source/libs/stream/src/streamCheckpoint.c b/source/libs/stream/src/streamCheckpoint.c index fce3526bee..a48f74ce86 100644 --- a/source/libs/stream/src/streamCheckpoint.c +++ b/source/libs/stream/src/streamCheckpoint.c @@ -182,8 +182,7 @@ int32_t streamProcessCheckpointBlock(SStreamTask* pTask, SStreamDataBlock* pBloc taosWLockLatch(&pMeta->lock); if (pMeta->chkptNotReadyTasks == 0) { - pMeta->chkptNotReadyTasks = streamMetaGetNumOfStreamTasks(pMeta); - pMeta->totalTasks = pMeta->chkptNotReadyTasks; + pMeta->chkptNotReadyTasks = pMeta->numOfStreamTasks; } taosWUnLockLatch(&pMeta->lock); @@ -315,15 +314,13 @@ int32_t streamTaskBuildCheckpoint(SStreamTask* pTask) { if (remain == 0) { // all tasks are ready qDebug("s-task:%s is ready for checkpoint", pTask->id.idStr); - pMeta->totalTasks = 0; - streamBackendDoCheckpoint(pMeta, pTask->checkpointingId); streamSaveAllTaskStatus(pMeta, pTask->checkpointingId); qDebug("vgId:%d vnode wide checkpoint completed, save all tasks status, checkpointId:%" PRId64, pMeta->vgId, pTask->checkpointingId); } else { qDebug("vgId:%d vnode wide tasks not reach checkpoint ready status, ready s-task:%s, not ready:%d/%d", pMeta->vgId, - pTask->id.idStr, remain, pMeta->totalTasks); + pTask->id.idStr, remain, pMeta->numOfStreamTasks); } // send check point response to upstream task diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index d3e57433a4..70c46bf2ed 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -205,8 +205,8 @@ SStreamMeta* streamMetaOpen(const char* path, void* ahandle, FTaskExpand expandF taosInitRWLatch(&pMeta->lock); taosThreadMutexInit(&pMeta->backendMutex, NULL); - pMeta->pauseTaskNum = 0; - + pMeta->numOfPausedTasks = 0; + pMeta->numOfStreamTasks = 0; qInfo("vgId:%d open stream meta successfully, latest checkpoint:%" PRId64 ", stage:%" PRId64, vgId, pMeta->chkpId, stage); return pMeta; @@ -412,6 +412,10 @@ int32_t streamMetaRegisterTask(SStreamMeta* pMeta, int64_t ver, SStreamTask* pTa } taosHashPut(pMeta->pTasksMap, &id, sizeof(id), &pTask, POINTER_BYTES); + if (pTask->info.fillHistory == 0) { + atomic_add_fetch_32(&pMeta->numOfStreamTasks, 1); + } + *pAdded = true; return 0; } @@ -492,7 +496,7 @@ int32_t streamMetaUnregisterTask(SStreamMeta* pMeta, int64_t streamId, int32_t t if (ppTask) { pTask = *ppTask; if (streamTaskShouldPause(&pTask->status)) { - int32_t num = atomic_sub_fetch_32(&pMeta->pauseTaskNum, 1); + int32_t num = atomic_sub_fetch_32(&pMeta->numOfPausedTasks, 1); qInfo("vgId:%d s-task:%s drop stream task. pause task num:%d", pMeta->vgId, pTask->id.idStr, num); } atomic_store_8(&pTask->status.taskStatus, TASK_STATUS__DROPPING); @@ -641,8 +645,8 @@ static void doClear(void* pKey, void* pVal, TBC* pCur, SArray* pRecycleList) { int32_t streamMetaLoadAllTasks(SStreamMeta* pMeta) { TBC* pCur = NULL; - qInfo("vgId:%d load stream tasks from meta files", pMeta->vgId); + if (tdbTbcOpen(pMeta->pTaskDb, &pCur, NULL) < 0) { qError("vgId:%d failed to open stream meta, code:%s", pMeta->vgId, tstrerror(terrno)); return -1; @@ -714,15 +718,17 @@ int32_t streamMetaLoadAllTasks(SStreamMeta* pMeta) { return -1; } + if (pTask->info.fillHistory == 0) { + atomic_add_fetch_32(&pMeta->numOfStreamTasks, 1); + } + if (streamTaskShouldPause(&pTask->status)) { - atomic_add_fetch_32(&pMeta->pauseTaskNum, 1); + atomic_add_fetch_32(&pMeta->numOfPausedTasks, 1); } ASSERT(pTask->status.downstreamReady == 0); } - qInfo("vgId:%d pause task num:%d", pMeta->vgId, pMeta->pauseTaskNum); - tdbFree(pKey); tdbFree(pVal); if (tdbTbcClose(pCur) < 0) { @@ -738,7 +744,8 @@ int32_t streamMetaLoadAllTasks(SStreamMeta* pMeta) { } int32_t numOfTasks = taosArrayGetSize(pMeta->pTaskList); - qDebug("vgId:%d load %d tasks into meta from disk completed", pMeta->vgId, numOfTasks); + qDebug("vgId:%d load %d tasks into meta from disk completed, streamTask:%d, paused:%d", pMeta->vgId, numOfTasks, + pMeta->numOfStreamTasks, pMeta->numOfPausedTasks); taosArrayDestroy(pRecycleList); return 0; } @@ -750,8 +757,8 @@ int32_t tEncodeStreamHbMsg(SEncoder* pEncoder, const SStreamHbMsg* pReq) { for (int32_t i = 0; i < pReq->numOfTasks; ++i) { STaskStatusEntry* ps = taosArrayGet(pReq->pTaskStatus, i); - if (tEncodeI64(pEncoder, ps->streamId) < 0) return -1; - if (tEncodeI32(pEncoder, ps->taskId) < 0) return -1; + if (tEncodeI64(pEncoder, ps->id.streamId) < 0) return -1; + if (tEncodeI32(pEncoder, ps->id.taskId) < 0) return -1; if (tEncodeI32(pEncoder, ps->status) < 0) return -1; } tEndEncode(pEncoder); @@ -766,8 +773,11 @@ int32_t tDecodeStreamHbMsg(SDecoder* pDecoder, SStreamHbMsg* pReq) { pReq->pTaskStatus = taosArrayInit(pReq->numOfTasks, sizeof(STaskStatusEntry)); for (int32_t i = 0; i < pReq->numOfTasks; ++i) { STaskStatusEntry hb = {0}; - if (tDecodeI64(pDecoder, &hb.streamId) < 0) return -1; - if (tDecodeI32(pDecoder, &hb.taskId) < 0) return -1; + if (tDecodeI64(pDecoder, &hb.id.streamId) < 0) return -1; + int32_t taskId = 0; + if (tDecodeI32(pDecoder, &taskId) < 0) return -1; + + hb.id.taskId = taskId; if (tDecodeI32(pDecoder, &hb.status) < 0) return -1; taosArrayPush(pReq->pTaskStatus, &hb); @@ -839,7 +849,7 @@ void metaHbToMnode(void* param, void* tmrId) { continue; } - STaskStatusEntry entry = {.streamId = pId->streamId, .taskId = pId->taskId, .status = (*pTask)->status.taskStatus}; + STaskStatusEntry entry = {.id = *pId, .status = (*pTask)->status.taskStatus}; taosArrayPush(hbMsg.pTaskStatus, &entry); if (!hasValEpset) { diff --git a/source/libs/stream/src/streamRecover.c b/source/libs/stream/src/streamRecover.c index 7a318e2310..d28ec85dd5 100644 --- a/source/libs/stream/src/streamRecover.c +++ b/source/libs/stream/src/streamRecover.c @@ -73,6 +73,7 @@ const char* streamGetTaskStatusStr(int32_t status) { case TASK_STATUS__CK: return "check-point"; case TASK_STATUS__DROPPING: return "dropping"; case TASK_STATUS__STOP: return "stop"; + case TASK_STATUS__UNINIT: return "uninitialized"; default:return ""; } } @@ -244,6 +245,7 @@ static void doProcessDownstreamReadyRsp(SStreamTask* pTask, int32_t numOfReqs) { ASSERT(pTask->historyTaskId.taskId == 0); } else { qDebug("s-task:%s downstream tasks are ready, now ready for data from wal, status:%s", id, str); + streamTaskEnablePause(pTask); } } @@ -818,7 +820,7 @@ void streamTaskPause(SStreamTask* pTask, SStreamMeta* pMeta) { } if(pTask->info.taskLevel == TASK_LEVEL__SINK) { - int32_t num = atomic_add_fetch_32(&pMeta->pauseTaskNum, 1); + int32_t num = atomic_add_fetch_32(&pMeta->numOfPausedTasks, 1); qInfo("vgId:%d s-task:%s pause stream sink task. pause task num:%d", pMeta->vgId, pTask->id.idStr, num); return; } @@ -852,7 +854,7 @@ void streamTaskPause(SStreamTask* pTask, SStreamMeta* pMeta) { atomic_store_8(&pTask->status.keepTaskStatus, pTask->status.taskStatus); atomic_store_8(&pTask->status.taskStatus, TASK_STATUS__PAUSE); - int32_t num = atomic_add_fetch_32(&pMeta->pauseTaskNum, 1); + int32_t num = atomic_add_fetch_32(&pMeta->numOfPausedTasks, 1); qInfo("vgId:%d s-task:%s pause stream task. pause task num:%d", pMeta->vgId, pTask->id.idStr, num); taosWUnLockLatch(&pMeta->lock); @@ -872,10 +874,10 @@ void streamTaskResume(SStreamTask* pTask, SStreamMeta* pMeta) { if (status == TASK_STATUS__PAUSE) { pTask->status.taskStatus = pTask->status.keepTaskStatus; pTask->status.keepTaskStatus = TASK_STATUS__NORMAL; - int32_t num = atomic_sub_fetch_32(&pMeta->pauseTaskNum, 1); + int32_t num = atomic_sub_fetch_32(&pMeta->numOfPausedTasks, 1); qInfo("vgId:%d s-task:%s resume from pause, status:%s. pause task num:%d", pMeta->vgId, pTask->id.idStr, streamGetTaskStatusStr(status), num); } else if (pTask->info.taskLevel == TASK_LEVEL__SINK) { - int32_t num = atomic_sub_fetch_32(&pMeta->pauseTaskNum, 1); + int32_t num = atomic_sub_fetch_32(&pMeta->numOfPausedTasks, 1); qInfo("vgId:%d s-task:%s sink task.resume from pause, status:%s. pause task num:%d", pMeta->vgId, pTask->id.idStr, streamGetTaskStatusStr(status), num); } else { qError("s-task:%s not in pause, failed to resume, status:%s", pTask->id.idStr, streamGetTaskStatusStr(status)); diff --git a/source/libs/stream/src/streamTask.c b/source/libs/stream/src/streamTask.c index 117c795a8d..4f320c3de0 100644 --- a/source/libs/stream/src/streamTask.c +++ b/source/libs/stream/src/streamTask.c @@ -165,9 +165,14 @@ int32_t tDecodeStreamTask(SDecoder* pDecoder, SStreamTask* pTask) { if (tDecodeI8(pDecoder, &pTask->info.fillHistory) < 0) return -1; if (tDecodeI64(pDecoder, &pTask->historyTaskId.streamId)) return -1; - if (tDecodeI32(pDecoder, &pTask->historyTaskId.taskId)) return -1; + + int32_t taskId = pTask->historyTaskId.taskId; + if (tDecodeI32(pDecoder, &taskId)) return -1; + if (tDecodeI64(pDecoder, &pTask->streamTaskId.streamId)) return -1; - if (tDecodeI32(pDecoder, &pTask->streamTaskId.taskId)) return -1; + + taskId = pTask->streamTaskId.taskId; + if (tDecodeI32(pDecoder, &taskId)) return -1; if (tDecodeU64(pDecoder, &pTask->dataRange.range.minVer)) return -1; if (tDecodeU64(pDecoder, &pTask->dataRange.range.maxVer)) return -1; @@ -259,8 +264,11 @@ int32_t tDecodeStreamTaskId(SDecoder* pDecoder, STaskId* pTaskId) { if (ver != SSTREAM_TASK_VER) return -1; if (tDecodeI64(pDecoder, &pTaskId->streamId) < 0) return -1; - if (tDecodeI32(pDecoder, &pTaskId->taskId) < 0) return -1; + int32_t taskId = 0; + if (tDecodeI32(pDecoder, &taskId) < 0) return -1; + + pTaskId->taskId = taskId; tEndDecode(pDecoder); return 0; } From 9d6199adf7f7639185ee9f053698732cf0a66faa Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sun, 17 Sep 2023 13:59:06 +0800 Subject: [PATCH 018/158] fix(stream): fix invalid read. --- source/libs/stream/src/streamMeta.c | 20 ++++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 70c46bf2ed..6041c06102 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -533,6 +533,16 @@ int32_t streamMetaUnregisterTask(SStreamMeta* pMeta, int64_t streamId, int32_t t taosWLockLatch(&pMeta->lock); ppTask = (SStreamTask**)taosHashGet(pMeta->pTasksMap, &id, sizeof(id)); if (ppTask) { + // it is an fill-history task, remove the related stream task's id that points to it + if ((*ppTask)->info.fillHistory == 1) { + STaskId streamTaskId = {.streamId = (*ppTask)->streamTaskId.streamId, .taskId = (*ppTask)->streamTaskId.taskId}; + SStreamTask** ppStreamTask = (SStreamTask**)taosHashGet(pMeta->pTasksMap, &streamTaskId, sizeof(streamTaskId)); + if (ppStreamTask != NULL) { + (*ppStreamTask)->historyTaskId.taskId = 0; + (*ppStreamTask)->historyTaskId.streamId = 0; + } + } + taosHashRemove(pMeta->pTasksMap, &id, sizeof(id)); atomic_store_8(&pTask->status.taskStatus, TASK_STATUS__DROPPING); @@ -546,16 +556,6 @@ int32_t streamMetaUnregisterTask(SStreamMeta* pMeta, int64_t streamId, int32_t t streamMetaReleaseTask(pMeta, pTask); } - // it is an fill-history task, remove the related stream task's id that points to it - if ((*ppTask)->info.fillHistory == 1) { - STaskId id1 = {.streamId = (*ppTask)->streamTaskId.streamId, .taskId = (*ppTask)->streamTaskId.taskId}; - SStreamTask** ppStreamTask = (SStreamTask**)taosHashGet(pMeta->pTasksMap, &id1, sizeof(id1)); - if (ppStreamTask != NULL) { - (*ppStreamTask)->historyTaskId.taskId = 0; - (*ppStreamTask)->historyTaskId.streamId = 0; - } - } - streamMetaRemoveTask(pMeta, &id); streamMetaReleaseTask(pMeta, pTask); } else { From 57bf1cadc4fd6c0ec73375e1d8eca2a465b468b4 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sun, 17 Sep 2023 18:07:26 +0800 Subject: [PATCH 019/158] fix(stream): fix error in decode stream task. --- source/dnode/mnode/impl/src/mndStream.c | 11 ----------- source/libs/stream/src/streamTask.c | 15 +++++++++------ 2 files changed, 9 insertions(+), 17 deletions(-) diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index 28d4716f0e..72d3ac5f95 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -799,17 +799,6 @@ static int32_t mndProcessCreateStreamReq(SRpcMsg *pReq) { } } - // pDb = mndAcquireDb(pMnode, streamObj.sourceDb); - // if (pDb->cfg.replications != 1) { - // mError("stream source db must have only 1 replica, but %s has %d", pDb->name, pDb->cfg.replications); - // terrno = TSDB_CODE_MND_MULTI_REPLICA_SOURCE_DB; - // mndReleaseDb(pMnode, pDb); - // pDb = NULL; - // goto _OVER; - // } - - // mndReleaseDb(pMnode, pDb); - STrans *pTrans = mndTransCreate(pMnode, TRN_POLICY_ROLLBACK, TRN_CONFLICT_DB_INSIDE, pReq, "create-stream"); if (pTrans == NULL) { mError("stream:%s, failed to create since %s", createStreamReq.name, terrstr()); diff --git a/source/libs/stream/src/streamTask.c b/source/libs/stream/src/streamTask.c index 4f320c3de0..ba8578f98e 100644 --- a/source/libs/stream/src/streamTask.c +++ b/source/libs/stream/src/streamTask.c @@ -97,9 +97,12 @@ int32_t tEncodeStreamTask(SEncoder* pEncoder, const SStreamTask* pTask) { if (tEncodeI8(pEncoder, pTask->info.fillHistory) < 0) return -1; if (tEncodeI64(pEncoder, pTask->historyTaskId.streamId)) return -1; - if (tEncodeI32(pEncoder, pTask->historyTaskId.taskId)) return -1; + int32_t taskId = pTask->historyTaskId.taskId; + if (tEncodeI32(pEncoder, taskId)) return -1; + if (tEncodeI64(pEncoder, pTask->streamTaskId.streamId)) return -1; - if (tEncodeI32(pEncoder, pTask->streamTaskId.taskId)) return -1; + taskId = pTask->streamTaskId.taskId; + if (tEncodeI32(pEncoder, taskId)) return -1; if (tEncodeU64(pEncoder, pTask->dataRange.range.minVer)) return -1; if (tEncodeU64(pEncoder, pTask->dataRange.range.maxVer)) return -1; @@ -141,6 +144,8 @@ int32_t tEncodeStreamTask(SEncoder* pEncoder, const SStreamTask* pTask) { } int32_t tDecodeStreamTask(SDecoder* pDecoder, SStreamTask* pTask) { + int32_t taskId = 0; + if (tStartDecode(pDecoder) < 0) return -1; if (tDecodeI64(pDecoder, &pTask->ver) < 0) return -1; if (pTask->ver != SSTREAM_TASK_VER) return -1; @@ -165,14 +170,12 @@ int32_t tDecodeStreamTask(SDecoder* pDecoder, SStreamTask* pTask) { if (tDecodeI8(pDecoder, &pTask->info.fillHistory) < 0) return -1; if (tDecodeI64(pDecoder, &pTask->historyTaskId.streamId)) return -1; - - int32_t taskId = pTask->historyTaskId.taskId; if (tDecodeI32(pDecoder, &taskId)) return -1; + pTask->historyTaskId.taskId = taskId; if (tDecodeI64(pDecoder, &pTask->streamTaskId.streamId)) return -1; - - taskId = pTask->streamTaskId.taskId; if (tDecodeI32(pDecoder, &taskId)) return -1; + pTask->streamTaskId.taskId = taskId; if (tDecodeU64(pDecoder, &pTask->dataRange.range.minVer)) return -1; if (tDecodeU64(pDecoder, &pTask->dataRange.range.maxVer)) return -1; From 636eccc1607d3059b9b76572b23cfefe70f580cb Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 18 Sep 2023 15:14:51 +0800 Subject: [PATCH 020/158] fix(stream): add null check --- include/libs/stream/tstream.h | 2 +- source/dnode/vnode/src/tq/tq.c | 3 +-- source/libs/stream/src/streamExec.c | 7 +++---- source/libs/stream/src/streamTask.c | 4 ++-- 4 files changed, 7 insertions(+), 9 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 2d70bb1e1c..a19ebd67b0 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -712,7 +712,7 @@ int32_t streamProcessCheckpointSourceReq(SStreamTask* pTask, SStreamCheckpointSo int32_t streamProcessCheckpointReadyMsg(SStreamTask* pTask); int32_t streamAlignTransferState(SStreamTask* pTask); -int32_t streamBuildAndSendDropTaskMsg(SStreamTask* pTask, int32_t vgId, SStreamTaskId* pTaskId); +int32_t streamBuildAndSendDropTaskMsg(SMsgCb* pMsgCb, int32_t vgId, SStreamTaskId* pTaskId); int32_t streamAddCheckpointSourceRspMsg(SStreamCheckpointSourceReq* pReq, SRpcHandleInfo* pRpcInfo, SStreamTask* pTask, int8_t isSucceed); int32_t buildCheckpointSourceRsp(SStreamCheckpointSourceReq* pReq, SRpcHandleInfo* pRpcInfo, SRpcMsg* pMsg, diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index fe1ef1637f..0340f1bb25 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1099,8 +1099,7 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { tqDebug("s-task:%s fill-history task set status to be dropping", id); -// streamMetaUnregisterTask(pMeta, pTask->id.streamId, pTask->id.taskId); - streamBuildAndSendDropTaskMsg(pTask, pMeta->vgId, &pTask->id); + streamBuildAndSendDropTaskMsg(pTask->pMsgCb, pMeta->vgId, &pTask->id); streamMetaReleaseTask(pMeta, pTask); return -1; } diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index 91c46c8ad9..969b547d71 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -303,7 +303,7 @@ int32_t streamDoTransferStateToStreamTask(SStreamTask* pTask) { pTask->id.idStr, (int32_t) pTask->streamTaskId.taskId); // 1. free it and remove fill-history task from disk meta-store - streamBuildAndSendDropTaskMsg(pStreamTask, pMeta->vgId, &pTask->id); + streamBuildAndSendDropTaskMsg(pTask->pMsgCb, pMeta->vgId, &pTask->id); // 2. save to disk taosWLockLatch(&pMeta->lock); @@ -365,8 +365,7 @@ int32_t streamDoTransferStateToStreamTask(SStreamTask* pTask) { qDebug("s-task:%s fill-history task set status to be dropping, save the state into disk", pTask->id.idStr); // 4. free it and remove fill-history task from disk meta-store -// streamMetaUnregisterTask(pMeta, pTask->id.streamId, pTask->id.taskId); - streamBuildAndSendDropTaskMsg(pStreamTask, pMeta->vgId, &pTask->id); + streamBuildAndSendDropTaskMsg(pTask->pMsgCb, pMeta->vgId, &pTask->id); // 5. clear the link between fill-history task and stream task info pStreamTask->historyTaskId.taskId = 0; @@ -411,7 +410,7 @@ int32_t streamTransferStateToStreamTask(SStreamTask* pTask) { if (level == TASK_LEVEL__AGG || level == TASK_LEVEL__SOURCE) { // do transfer task operator states. code = streamDoTransferStateToStreamTask(pTask); } else { // drop fill-history task - streamBuildAndSendDropTaskMsg(pTask, pTask->pMeta->vgId, &pTask->id); + streamBuildAndSendDropTaskMsg(pTask->pMsgCb, pTask->pMeta->vgId, &pTask->id); } return code; diff --git a/source/libs/stream/src/streamTask.c b/source/libs/stream/src/streamTask.c index ba8578f98e..d2e306fa01 100644 --- a/source/libs/stream/src/streamTask.c +++ b/source/libs/stream/src/streamTask.c @@ -644,7 +644,7 @@ int8_t streamTaskSetSchedStatusInActive(SStreamTask* pTask) { return status; } -int32_t streamBuildAndSendDropTaskMsg(SStreamTask* pTask, int32_t vgId, SStreamTaskId* pTaskId) { +int32_t streamBuildAndSendDropTaskMsg(SMsgCb* pMsgCb, int32_t vgId, SStreamTaskId* pTaskId) { SVDropStreamTaskReq *pReq = rpcMallocCont(sizeof(SVDropStreamTaskReq)); if (pReq == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; @@ -656,7 +656,7 @@ int32_t streamBuildAndSendDropTaskMsg(SStreamTask* pTask, int32_t vgId, SStreamT pReq->streamId = pTaskId->streamId; SRpcMsg msg = {.msgType = TDMT_STREAM_TASK_DROP, .pCont = pReq, .contLen = sizeof(SVDropStreamTaskReq)}; - int32_t code = tmsgPutToQueue(pTask->pMsgCb, WRITE_QUEUE, &msg); + int32_t code = tmsgPutToQueue(pMsgCb, WRITE_QUEUE, &msg); if (code != TSDB_CODE_SUCCESS) { qError("vgId:%d failed to send drop task:0x%x msg, code:%s", vgId, pTaskId->taskId, tstrerror(code)); return code; From 734f6488918db1e43f1c09ee1e378e3cb64c29a5 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 18 Sep 2023 18:25:22 +0800 Subject: [PATCH 021/158] fix(stream): set the output normal for stream dispatch when encountering the transfer state. --- 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 4d5234a68c..7003c570e9 100644 --- a/source/libs/stream/src/streamDispatch.c +++ b/source/libs/stream/src/streamDispatch.c @@ -990,6 +990,8 @@ int32_t streamProcessDispatchRsp(SStreamTask* pTask, SStreamDispatchRsp* pRsp, i if (code != TSDB_CODE_SUCCESS) { // todo: do nothing if error happens } + // now ready for next data output + atomic_store_8(&pTask->outputInfo.status, TASK_OUTPUT_STATUS__NORMAL); return TSDB_CODE_SUCCESS; } From 08b37dfc1310f6a7f242b4bf0885cd327124e8c8 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 18 Sep 2023 19:58:46 +0800 Subject: [PATCH 022/158] fix(stream): fix memory leak. --- source/common/src/tglobal.c | 2 +- source/libs/stream/src/streamQueue.c | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/source/common/src/tglobal.c b/source/common/src/tglobal.c index 49ba20f8b0..648bb486b1 100644 --- a/source/common/src/tglobal.c +++ b/source/common/src/tglobal.c @@ -241,7 +241,7 @@ int32_t tsTtlBatchDropNum = 10000; // number of tables dropped per batch // internal int32_t tsTransPullupInterval = 2; int32_t tsMqRebalanceInterval = 2; -int32_t tsStreamCheckpointTickInterval = 300; +int32_t tsStreamCheckpointTickInterval = 10; int32_t tsStreamNodeCheckInterval = 10; int32_t tsTtlUnit = 86400; int32_t tsTtlPushIntervalSec = 10; diff --git a/source/libs/stream/src/streamQueue.c b/source/libs/stream/src/streamQueue.c index 6aaea2ce24..d3d114d4aa 100644 --- a/source/libs/stream/src/streamQueue.c +++ b/source/libs/stream/src/streamQueue.c @@ -387,7 +387,7 @@ int32_t streamTaskPutDataIntoOutputQ(SStreamTask* pTask, SStreamDataBlock* pBloc qError("s-task:%s failed to put res into outputQ, outputQ items:%d, size:%.2fMiB code:%s, result lost", pTask->id.idStr, total + 1, size, tstrerror(code)); } else { - qInfo("s-task:%s data put into outputQ, outputQ items:%d, size:%.2fMiB", pTask->id.idStr, total, size); + qDebug("s-task:%s data put into outputQ, outputQ items:%d, size:%.2fMiB", pTask->id.idStr, total, size); } return TSDB_CODE_SUCCESS; From 38bf2d24e74739679a186834b2e79c87ab7a1af9 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 18 Sep 2023 20:25:24 +0800 Subject: [PATCH 023/158] fix(stream): remove stream in buf. --- source/dnode/mnode/impl/src/mndStream.c | 30 ++++++++++++++++++++----- 1 file changed, 25 insertions(+), 5 deletions(-) diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index 72d3ac5f95..eabf4d8e4a 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -83,6 +83,9 @@ static SVgroupChangeInfo mndFindChangedNodeInfo(SMnode *pMnode, const SArray *pP static int32_t mndPersistTransLog(SStreamObj *pStream, STrans *pTrans); static void initTransAction(STransAction *pAction, void *pCont, int32_t contLen, int32_t msgType, const SEpSet *pEpset); +static void removeStreamTasksInBuf(SStreamObj* pStream, SStreamVnodeRevertIndex* pExecNode); +static void keepStreamTasksInBuf(SStreamObj *pStream, SStreamVnodeRevertIndex *pExecNode); + int32_t mndInitStream(SMnode *pMnode) { SSdbTable table = { .sdbType = SDB_STREAM, @@ -1280,7 +1283,6 @@ static int32_t mndProcessDropStreamReq(SRpcMsg *pReq) { mndTransDrop(pTrans); return -1; } - // mndTransSetSerial(pTrans); // drop all tasks if (mndDropStreamTasks(pMnode, pTrans, pStream) < 0) { @@ -1304,13 +1306,13 @@ static int32_t mndProcessDropStreamReq(SRpcMsg *pReq) { return -1; } + removeStreamTasksInBuf(pStream, &execNodeList); + char detail[100] = {0}; sprintf(detail, "igNotExists:%d", dropReq.igNotExists); SName name = {0}; tNameFromString(&name, dropReq.name, T_NAME_ACCT | T_NAME_DB); - //reuse this function for stream - auditRecord(pReq, pMnode->clusterId, "dropStream", name.dbname, "", detail); sdbRelease(pMnode->pSdb, pStream); @@ -2238,7 +2240,7 @@ static int32_t mndProcessNodeCheck(SRpcMsg *pReq) { return 0; } -static void keepStreamTasksInBuf(SStreamObj *pStream, SStreamVnodeRevertIndex *pExecNode) { +void keepStreamTasksInBuf(SStreamObj *pStream, SStreamVnodeRevertIndex *pExecNode) { int32_t level = taosArrayGetSize(pStream->tasks); for (int32_t i = 0; i < level; i++) { SArray *pLevel = taosArrayGetP(pStream->tasks, i); @@ -2261,6 +2263,25 @@ static void keepStreamTasksInBuf(SStreamObj *pStream, SStreamVnodeRevertIndex *p } } +void removeStreamTasksInBuf(SStreamObj* pStream, SStreamVnodeRevertIndex* pExecNode) { + int32_t level = taosArrayGetSize(pStream->tasks); + for (int32_t i = 0; i < level; i++) { + SArray *pLevel = taosArrayGetP(pStream->tasks, i); + + int32_t numOfTasks = taosArrayGetSize(pLevel); + for (int32_t j = 0; j < numOfTasks; j++) { + SStreamTask *pTask = taosArrayGetP(pLevel, j); + + STaskId id = {.streamId = pTask->id.streamId, .taskId = pTask->id.taskId}; + void *p = taosHashGet(pExecNode->pTaskMap, &id, sizeof(id)); + if (p != NULL) { + taosArrayRemove(pExecNode->pTaskList, *(int32_t*)p); + taosHashRemove(pExecNode->pTaskMap, &id, sizeof(id)); + } + } + } +} + // todo: this process should be executed by the write queue worker of the mnode int32_t mndProcessStreamHb(SRpcMsg *pReq) { SMnode *pMnode = pReq->info.node; @@ -2277,7 +2298,6 @@ int32_t mndProcessStreamHb(SRpcMsg *pReq) { } tDecoderClear(&decoder); - // int64_t now = taosGetTimestampSec(); mTrace("receive stream-meta hb from vgId:%d, active numOfTasks:%d", req.vgId, req.numOfTasks); taosThreadMutexLock(&execNodeList.lock); From c3a3cf21fb1df695b78a29c6a9b865f166e8f319 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 18 Sep 2023 23:51:08 +0800 Subject: [PATCH 024/158] fix(stream): add some logs. --- source/libs/stream/src/streamMeta.c | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 6041c06102..2a4a8bbebc 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -645,10 +645,12 @@ static void doClear(void* pKey, void* pVal, TBC* pCur, SArray* pRecycleList) { int32_t streamMetaLoadAllTasks(SStreamMeta* pMeta) { TBC* pCur = NULL; - qInfo("vgId:%d load stream tasks from meta files", pMeta->vgId); + int32_t vgId = pMeta->vgId; + + qInfo("vgId:%d load stream tasks from meta files", vgId); if (tdbTbcOpen(pMeta->pTaskDb, &pCur, NULL) < 0) { - qError("vgId:%d failed to open stream meta, code:%s", pMeta->vgId, tstrerror(terrno)); + qError("vgId:%d failed to open stream meta, code:%s", vgId, tstrerror(terrno)); return -1; } @@ -663,6 +665,8 @@ int32_t streamMetaLoadAllTasks(SStreamMeta* pMeta) { while (tdbTbcNext(pCur, &pKey, &kLen, &pVal, &vLen) == 0) { SStreamTask* pTask = taosMemoryCalloc(1, sizeof(SStreamTask)); if (pTask == NULL) { + terrno = TSDB_CODE_OUT_OF_MEMORY; + qError("vgId:%d failed to load stream task from meta-files, code:%s", vgId, tstrerror(terrno)); doClear(pKey, pVal, pCur, pRecycleList); return -1; } @@ -673,9 +677,8 @@ int32_t streamMetaLoadAllTasks(SStreamMeta* pMeta) { doClear(pKey, pVal, pCur, pRecycleList); tFreeStreamTask(pTask); qError( - "stream read incompatible data, rm %s/vnode/vnode*/tq/stream if taosd cannot start, and rebuild stream " - "manually", - tsDataDir); + "vgId:%d stream read incompatible data, rm %s/vnode/vnode*/tq/stream if taosd cannot start, and rebuild stream " + "manually", vgId, tsDataDir); return -1; } tDecoderClear(&decoder); @@ -732,6 +735,7 @@ int32_t streamMetaLoadAllTasks(SStreamMeta* pMeta) { tdbFree(pKey); tdbFree(pVal); if (tdbTbcClose(pCur) < 0) { + qError("vgId:%d failed to close meta-file cursor", vgId); taosArrayDestroy(pRecycleList); return -1; } From e6fb9ee46c2852adcb6e3959d6b41a4b71b11b52 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 19 Sep 2023 00:52:35 +0800 Subject: [PATCH 025/158] fix(stream): reset task counter. --- 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 2a4a8bbebc..fa545943eb 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -295,6 +295,8 @@ void streamMetaClear(SStreamMeta* pMeta) { taosArrayClear(pMeta->pTaskList); taosArrayClear(pMeta->chkpSaved); taosArrayClear(pMeta->chkpInUse); + pMeta->numOfStreamTasks = 0; + pMeta->numOfPausedTasks = 0; } void streamMetaClose(SStreamMeta* pMeta) { @@ -748,6 +750,7 @@ int32_t streamMetaLoadAllTasks(SStreamMeta* pMeta) { } int32_t numOfTasks = taosArrayGetSize(pMeta->pTaskList); + ASSERT(pMeta->numOfStreamTasks <= numOfTasks); qDebug("vgId:%d load %d tasks into meta from disk completed, streamTask:%d, paused:%d", pMeta->vgId, numOfTasks, pMeta->numOfStreamTasks, pMeta->numOfPausedTasks); taosArrayDestroy(pRecycleList); From b95ad74c7f72bc4e371bd2996daf5cae6d9bef61 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 19 Sep 2023 09:43:14 +0800 Subject: [PATCH 026/158] fix(stream): not handle the check msg for follower tasks. --- include/libs/stream/tstream.h | 1 + source/dnode/vnode/src/tq/tq.c | 14 +++++++++++--- source/dnode/vnode/src/tq/tqStreamTask.c | 8 +------- source/libs/stream/src/streamMeta.c | 2 +- source/libs/stream/src/streamRecover.c | 19 ++++++++++++++++--- 5 files changed, 30 insertions(+), 14 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index a19ebd67b0..a55d188978 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -661,6 +661,7 @@ int32_t streamQueueGetNumOfItems(const SStreamQueue* pQueue); // common int32_t streamRestoreParam(SStreamTask* pTask); int32_t streamSetStatusNormal(SStreamTask* pTask); +int32_t streamSetStatusUnint(SStreamTask* pTask); const char* streamGetTaskStatusStr(int32_t status); void streamTaskPause(SStreamTask* pTask, SStreamMeta* pMeta); void streamTaskResume(SStreamTask* pTask, SStreamMeta* pMeta); diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 0340f1bb25..95637fad69 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -877,6 +877,7 @@ int32_t tqProcessStreamTaskCheckReq(STQ* pTq, SRpcMsg* pMsg) { char* msgStr = pMsg->pCont; char* msgBody = POINTER_SHIFT(msgStr, sizeof(SMsgHead)); int32_t msgLen = pMsg->contLen - sizeof(SMsgHead); + SStreamMeta* pMeta = pTq->pStreamMeta; SStreamTaskCheckReq req; SDecoder decoder; @@ -897,10 +898,17 @@ int32_t tqProcessStreamTaskCheckReq(STQ* pTq, SRpcMsg* pMsg) { .upstreamTaskId = req.upstreamTaskId, }; - SStreamTask* pTask = streamMetaAcquireTask(pTq->pStreamMeta, req.streamId, taskId); + // only the leader node handle the check request + if (!pMeta->leader) { + tqError("s-task:0x%x invalid check msg from upstream:0x%x(vgId:%d), vgId:%d is follower, not handle check msg", + taskId, req.upstreamTaskId, req.upstreamNodeId, pMeta->vgId); + return -1; + } + + SStreamTask* pTask = streamMetaAcquireTask(pMeta, req.streamId, taskId); if (pTask != NULL) { rsp.status = streamTaskCheckStatus(pTask, req.upstreamTaskId, req.upstreamNodeId, req.stage); - streamMetaReleaseTask(pTq->pStreamMeta, pTask); + streamMetaReleaseTask(pMeta, pTask); const char* pStatus = streamGetTaskStatusStr(pTask->status.taskStatus); tqDebug("s-task:%s status:%s, stage:%d recv task check req(reqId:0x%" PRIx64 ") task:0x%x (vgId:%d), ready:%d", @@ -912,7 +920,7 @@ int32_t tqProcessStreamTaskCheckReq(STQ* pTq, SRpcMsg* pMsg) { req.streamId, taskId, rsp.reqId, rsp.upstreamTaskId, rsp.upstreamNodeId, rsp.status); } - return streamSendCheckRsp(pTq->pStreamMeta, &req, &rsp, &pMsg->info, taskId); + return streamSendCheckRsp(pMeta, &req, &rsp, &pMsg->info, taskId); } int32_t tqProcessStreamTaskCheckRsp(STQ* pTq, SRpcMsg* pMsg) { diff --git a/source/dnode/vnode/src/tq/tqStreamTask.c b/source/dnode/vnode/src/tq/tqStreamTask.c index 3cba4567fe..3a5eeae561 100644 --- a/source/dnode/vnode/src/tq/tqStreamTask.c +++ b/source/dnode/vnode/src/tq/tqStreamTask.c @@ -95,7 +95,7 @@ int32_t tqCheckAndRunStreamTask(STQ* pTq) { } pTask->taskExecInfo.init = taosGetTimestampMs(); - tqDebug("s-task:%s set the init ts:%"PRId64, pTask->id.idStr, pTask->taskExecInfo.init); + tqDebug("s-task:%s start check downstream tasks, set the init ts:%"PRId64, pTask->id.idStr, pTask->taskExecInfo.init); streamSetStatusNormal(pTask); streamTaskCheckDownstream(pTask); @@ -111,12 +111,9 @@ int32_t tqCheckAndRunStreamTaskAsync(STQ* pTq) { int32_t vgId = TD_VID(pTq->pVnode); SStreamMeta* pMeta = pTq->pStreamMeta; -// taosWLockLatch(&pMeta->lock); - int32_t numOfTasks = taosArrayGetSize(pMeta->pTaskList); if (numOfTasks == 0) { tqDebug("vgId:%d no stream tasks existed to run", vgId); -// taosWUnLockLatch(&pMeta->lock); return 0; } @@ -124,7 +121,6 @@ int32_t tqCheckAndRunStreamTaskAsync(STQ* pTq) { if (pRunReq == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; tqError("vgId:%d failed to create msg to start wal scanning to launch stream tasks, code:%s", vgId, terrstr()); -// taosWUnLockLatch(&pMeta->lock); return -1; } @@ -135,8 +131,6 @@ int32_t tqCheckAndRunStreamTaskAsync(STQ* pTq) { SRpcMsg msg = {.msgType = TDMT_STREAM_TASK_RUN, .pCont = pRunReq, .contLen = sizeof(SStreamTaskRunReq)}; tmsgPutToQueue(&pTq->pVnode->msgCb, STREAM_QUEUE, &msg); -// taosWUnLockLatch(&pMeta->lock); - return 0; } diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index fa545943eb..66f05367c2 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -750,7 +750,7 @@ int32_t streamMetaLoadAllTasks(SStreamMeta* pMeta) { } int32_t numOfTasks = taosArrayGetSize(pMeta->pTaskList); - ASSERT(pMeta->numOfStreamTasks <= numOfTasks); + ASSERT(pMeta->numOfStreamTasks <= numOfTasks && pMeta->numOfPausedTasks <= numOfTasks); qDebug("vgId:%d load %d tasks into meta from disk completed, streamTask:%d, paused:%d", pMeta->vgId, numOfTasks, pMeta->numOfStreamTasks, pMeta->numOfPausedTasks); taosArrayDestroy(pRecycleList); diff --git a/source/libs/stream/src/streamRecover.c b/source/libs/stream/src/streamRecover.c index d28ec85dd5..2689e9ee70 100644 --- a/source/libs/stream/src/streamRecover.c +++ b/source/libs/stream/src/streamRecover.c @@ -205,21 +205,22 @@ int32_t streamTaskCheckStatus(SStreamTask* pTask, int32_t upstreamTaskId, int32_ SStreamChildEpInfo* pInfo = streamTaskGetUpstreamTaskEpInfo(pTask, upstreamTaskId); ASSERT(pInfo != NULL); + const char* id = pTask->id.idStr; if (stage == -1) { - qDebug("s-task:%s receive check msg from upstream task:0x%x, invalid stageId:%" PRId64 ", not ready", pTask->id.idStr, + qDebug("s-task:%s receive check msg from upstream task:0x%x, invalid stageId:%" PRId64 ", not ready", id, upstreamTaskId, stage); return 0; } if (pInfo->stage == -1) { pInfo->stage = stage; - qDebug("s-task:%s receive check msg from upstream task:0x%x, init stage value:%" PRId64, pTask->id.idStr, + qDebug("s-task:%s receive check msg from upstream task:0x%x for the time, init stage value:%" PRId64, id, upstreamTaskId, stage); } if (pInfo->stage < stage) { qError("s-task:%s receive msg from upstream task:0x%x(vgId:%d), new stage received:%" PRId64 ", prev:%" PRId64, - pTask->id.idStr, upstreamTaskId, vgId, stage, pInfo->stage); + id, upstreamTaskId, vgId, stage, pInfo->stage); } return ((pTask->status.downstreamReady == 1) && (pInfo->stage == stage))? 1:0; @@ -355,6 +356,18 @@ int32_t streamSetStatusNormal(SStreamTask* pTask) { } } +int32_t streamSetStatusUnint(SStreamTask* pTask) { + int32_t status = atomic_load_8(&pTask->status.taskStatus); + if (status == TASK_STATUS__DROPPING) { + qError("s-task:%s cannot be set uninit, since in dropping state", pTask->id.idStr); + return -1; + } else { + qDebug("s-task:%s set task status to be uninit, prev:%s", pTask->id.idStr, streamGetTaskStatusStr(status)); + atomic_store_8(&pTask->status.taskStatus, TASK_STATUS__UNINIT); + return 0; + } +} + // source int32_t streamSetParamForStreamScannerStep1(SStreamTask* pTask, SVersionRange *pVerRange, STimeWindow* pWindow) { return qStreamSourceScanParamForHistoryScanStep1(pTask->exec.pExecutor, pVerRange, pWindow); From 5c0b8ea804fbf84f934a1bc36dd5f2a081ea26c1 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 19 Sep 2023 09:53:33 +0800 Subject: [PATCH 027/158] enh(stream): log the checkpoint time. --- include/libs/stream/tstream.h | 1 + source/libs/stream/src/streamCheckpoint.c | 13 +++++++++---- 2 files changed, 10 insertions(+), 4 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index a55d188978..5329da2f17 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -247,6 +247,7 @@ typedef struct SStreamTaskId { } SStreamTaskId; typedef struct SCheckpointInfo { + int64_t startTs; int64_t checkpointId; int64_t checkpointVer; // latest checkpointId version int64_t nextProcessVer; // current offset in WAL, not serialize it diff --git a/source/libs/stream/src/streamCheckpoint.c b/source/libs/stream/src/streamCheckpoint.c index a48f74ce86..3f8b69785d 100644 --- a/source/libs/stream/src/streamCheckpoint.c +++ b/source/libs/stream/src/streamCheckpoint.c @@ -141,6 +141,7 @@ int32_t streamProcessCheckpointSourceReq(SStreamTask* pTask, SStreamCheckpointSo pTask->status.taskStatus = TASK_STATUS__CK; pTask->checkpointingId = pReq->checkpointId; pTask->checkpointNotReadyTasks = streamTaskGetNumOfDownstream(pTask); + pTask->chkInfo.startTs = taosGetTimestampMs(); // 2. let's dispatch checkpoint msg to downstream task directly and do nothing else. put the checkpoint block into // inputQ, to make sure all blocks with less version have been handled by this task already. @@ -312,15 +313,19 @@ int32_t streamTaskBuildCheckpoint(SStreamTask* pTask) { int32_t remain = atomic_sub_fetch_32(&pMeta->chkptNotReadyTasks, 1); ASSERT(remain >= 0); + double el = (taosGetTimestampMs() - pTask->chkInfo.startTs) / 1000.0; + if (remain == 0) { // all tasks are ready qDebug("s-task:%s is ready for checkpoint", pTask->id.idStr); streamBackendDoCheckpoint(pMeta, pTask->checkpointingId); streamSaveAllTaskStatus(pMeta, pTask->checkpointingId); - qDebug("vgId:%d vnode wide checkpoint completed, save all tasks status, checkpointId:%" PRId64, pMeta->vgId, - pTask->checkpointingId); + qDebug("vgId:%d vnode wide checkpoint completed, save all tasks status, elapsed time:%.2f Sec checkpointId:%" PRId64, pMeta->vgId, + el, pTask->checkpointingId); } else { - qDebug("vgId:%d vnode wide tasks not reach checkpoint ready status, ready s-task:%s, not ready:%d/%d", pMeta->vgId, - pTask->id.idStr, remain, pMeta->numOfStreamTasks); + qDebug( + "vgId:%d vnode wide tasks not reach checkpoint ready status, ready s-task:%s, elapsed time:%.2f Sec not " + "ready:%d/%d", + pMeta->vgId, pTask->id.idStr, el, remain, pMeta->numOfStreamTasks); } // send check point response to upstream task From ff3ea366e2ec6827d1b70a1696f32985aeb216d5 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 19 Sep 2023 10:33:58 +0800 Subject: [PATCH 028/158] fix(stream): keep the status entry in hash table, instead entry index. --- source/dnode/mnode/impl/src/mndStream.c | 46 ++++++++++++++++--------- 1 file changed, 29 insertions(+), 17 deletions(-) diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index eabf4d8e4a..68856e11f1 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -125,7 +125,7 @@ int32_t mndInitStream(SMnode *pMnode) { taosThreadMutexInit(&execNodeList.lock, NULL); execNodeList.pTaskMap = taosHashInit(64, taosGetDefaultHashFunction(TSDB_DATA_TYPE_VARCHAR), true, HASH_NO_LOCK); - execNodeList.pTaskList = taosArrayInit(4, sizeof(STaskStatusEntry)); + execNodeList.pTaskList = taosArrayInit(4, sizeof(STaskId)); return sdbSetTable(pMnode->pSdb, table); } @@ -1183,10 +1183,15 @@ static int32_t mndProcessStreamDoCheckpoint(SRpcMsg *pReq) { taosThreadMutexLock(&execNodeList.lock); for (int32_t i = 0; i < taosArrayGetSize(execNodeList.pTaskList); ++i) { - STaskStatusEntry *p = taosArrayGet(execNodeList.pTaskList, i); - if (p->status != TASK_STATUS__NORMAL) { + STaskId *p = taosArrayGet(execNodeList.pTaskList, i); + STaskStatusEntry* pEntry = taosHashGet(execNodeList.pTaskMap, p, sizeof(*p)); + if (pEntry == NULL) { + continue; + } + + if (pEntry->status != TASK_STATUS__NORMAL) { mDebug("s-task:0x%" PRIx64 "-0x%x (nodeId:%d) status:%s not ready, create checkpoint msg not issued", - p->id.streamId, (int32_t)p->id.taskId, 0, streamGetTaskStatusStr(p->status)); + pEntry->id.streamId, (int32_t)pEntry->id.taskId, 0, streamGetTaskStatusStr(pEntry->status)); ready = false; break; } @@ -1557,13 +1562,12 @@ static int32_t mndRetrieveStreamTask(SRpcMsg *pReq, SShowObj *pShow, SSDataBlock char status[20 + VARSTR_HEADER_SIZE] = {0}; STaskId id = {.streamId = pTask->id.streamId, .taskId = pTask->id.taskId}; - int32_t *index = taosHashGet(execNodeList.pTaskMap, &id, sizeof(id)); - if (index == NULL) { + STaskStatusEntry* pe = taosHashGet(execNodeList.pTaskMap, &id, sizeof(id)); + if (pe == NULL) { continue; } - STaskStatusEntry *pStatusEntry = taosArrayGet(execNodeList.pTaskList, *index); - const char* pStatus = streamGetTaskStatusStr(pStatusEntry->status); + const char* pStatus = streamGetTaskStatusStr(pe->status); STR_TO_VARSTR(status, pStatus); pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); @@ -2254,10 +2258,8 @@ void keepStreamTasksInBuf(SStreamObj *pStream, SStreamVnodeRevertIndex *pExecNod if (p == NULL) { STaskStatusEntry entry = { .id.streamId = pTask->id.streamId, .id.taskId = pTask->id.taskId, .status = TASK_STATUS__STOP}; - taosArrayPush(pExecNode->pTaskList, &entry); - - int32_t ordinal = taosArrayGetSize(pExecNode->pTaskList) - 1; - taosHashPut(pExecNode->pTaskMap, &id, sizeof(id), &ordinal, sizeof(ordinal)); + taosHashPut(pExecNode->pTaskMap, &id, sizeof(id), &entry, sizeof(entry)); + taosArrayPush(pExecNode->pTaskList, &id); } } } @@ -2275,11 +2277,21 @@ void removeStreamTasksInBuf(SStreamObj* pStream, SStreamVnodeRevertIndex* pExecN STaskId id = {.streamId = pTask->id.streamId, .taskId = pTask->id.taskId}; void *p = taosHashGet(pExecNode->pTaskMap, &id, sizeof(id)); if (p != NULL) { - taosArrayRemove(pExecNode->pTaskList, *(int32_t*)p); taosHashRemove(pExecNode->pTaskMap, &id, sizeof(id)); + + for(int32_t k = 0; k < taosArrayGetSize(pExecNode->pTaskList); ++k) { + STaskId* pId = taosArrayGet(pExecNode->pTaskList, k); + if (pId->taskId == id.taskId && pId->streamId == id.streamId) { + taosArrayRemove(pExecNode->pTaskList, k); + break; + } + } + } } } + + ASSERT(taosHashGetSize(pExecNode->pTaskMap) == taosArrayGetSize(pExecNode->pTaskList)); } // todo: this process should be executed by the write queue worker of the mnode @@ -2308,13 +2320,13 @@ int32_t mndProcessStreamHb(SRpcMsg *pReq) { for (int32_t i = 0; i < req.numOfTasks; ++i) { STaskStatusEntry *p = taosArrayGet(req.pTaskStatus, i); - int32_t *index = taosHashGet(execNodeList.pTaskMap, &p->id, sizeof(p->id)); - if (index == NULL) { + STaskStatusEntry* pEntry = taosHashGet(execNodeList.pTaskMap, &p->id, sizeof(p->id)); + if (pEntry == NULL) { + mError("s-task:0x%"PRIx64" not found in mnode task list", p->id.taskId); continue; } - STaskStatusEntry *pStatusEntry = taosArrayGet(execNodeList.pTaskList, *index); - pStatusEntry->status = p->status; + pEntry->status = p->status; if (p->status != TASK_STATUS__NORMAL) { mDebug("received s-task:0x%"PRIx64" not in ready status:%s", p->id.taskId, streamGetTaskStatusStr(p->status)); } From b38f9e02886caeeb919d5c2970e99788775f287d Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 19 Sep 2023 11:05:49 +0800 Subject: [PATCH 029/158] fix(stream): update logs. --- source/libs/stream/src/streamExec.c | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index 969b547d71..74b24fb4c3 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -525,6 +525,9 @@ int32_t streamProcessTranstateBlock(SStreamTask* pTask, SStreamDataBlock* pBlock int32_t streamExecForAll(SStreamTask* pTask) { const char* id = pTask->id.idStr; + // merge multiple input data if possible in the input queue. + qDebug("s-task:%s start to extract data block from inputQ", id); + while (1) { int32_t numOfBlocks = 0; SStreamQueueItem* pInput = NULL; @@ -533,9 +536,6 @@ int32_t streamExecForAll(SStreamTask* pTask) { break; } - // merge multiple input data if possible in the input queue. - qDebug("s-task:%s start to extract data block from inputQ", id); - /*int32_t code = */ streamTaskGetDataFromInputQ(pTask, &pInput, &numOfBlocks); if (pInput == NULL) { ASSERT(numOfBlocks == 0); From 053643be19a6789a2c4b58b7e67b6a303af069eb Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 19 Sep 2023 14:22:05 +0800 Subject: [PATCH 030/158] fix(stream): add timestamp. --- source/libs/stream/src/streamCheckpoint.c | 2 ++ 1 file changed, 2 insertions(+) diff --git a/source/libs/stream/src/streamCheckpoint.c b/source/libs/stream/src/streamCheckpoint.c index 3f8b69785d..f367ba932f 100644 --- a/source/libs/stream/src/streamCheckpoint.c +++ b/source/libs/stream/src/streamCheckpoint.c @@ -202,6 +202,8 @@ int32_t streamProcessCheckpointBlock(SStreamTask* pTask, SStreamDataBlock* pBloc } else if (taskLevel == TASK_LEVEL__SINK || taskLevel == TASK_LEVEL__AGG) { ASSERT(taosArrayGetSize(pTask->pUpstreamInfoList) > 0); + pTask->chkInfo.startTs = taosGetTimestampMs(); + // update the child Id for downstream tasks streamAddCheckpointReadyMsg(pTask, pBlock->srcTaskId, pTask->info.selfChildId, checkpointId); From ce85945583b982b9a67bbc99fca2735ff69bc14e Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 19 Sep 2023 16:49:04 +0800 Subject: [PATCH 031/158] fix(stream): set correct size for results generated by scan history stream tasks. --- source/libs/stream/src/streamData.c | 4 +++ source/libs/stream/src/streamExec.c | 38 +++++++++++++---------------- 2 files changed, 21 insertions(+), 21 deletions(-) diff --git a/source/libs/stream/src/streamData.c b/source/libs/stream/src/streamData.c index 00bf631d74..a108667f5d 100644 --- a/source/libs/stream/src/streamData.c +++ b/source/libs/stream/src/streamData.c @@ -65,6 +65,10 @@ SStreamDataBlock* createStreamBlockFromResults(SStreamQueueItem* pItem, SStreamT pStreamBlocks->type = STREAM_INPUT__DATA_BLOCK; pStreamBlocks->blocks = pRes; + if (pItem == NULL) { + return pStreamBlocks; + } + if (pItem->type == STREAM_INPUT__DATA_SUBMIT) { SStreamDataSubmit* pSubmit = (SStreamDataSubmit*)pItem; pStreamBlocks->sourceVer = pSubmit->ver; diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index 74b24fb4c3..d89817d236 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -16,9 +16,10 @@ #include "streamInt.h" // maximum allowed processed block batches. One block may include several submit blocks -#define MAX_STREAM_EXEC_BATCH_NUM 32 -#define MIN_STREAM_EXEC_BATCH_NUM 4 -#define STREAM_RESULT_DUMP_THRESHOLD 100 +#define MAX_STREAM_EXEC_BATCH_NUM 32 +#define MIN_STREAM_EXEC_BATCH_NUM 4 +#define STREAM_RESULT_DUMP_THRESHOLD 100 +#define STREAM_RESULT_DUMP_SIZE_THRESHOLD (1048576 * 1) static int32_t streamDoTransferStateToStreamTask(SStreamTask* pTask); @@ -75,7 +76,6 @@ static int32_t doDumpResult(SStreamTask* pTask, SStreamQueueItem* pItem, SArray* int32_t code = doOutputResultBlockImpl(pTask, pStreamBlocks); if (code != TSDB_CODE_SUCCESS) { // back pressure and record position - //code == TSDB_CODE_UTIL_QUEUE_OUT_OF_MEMORY destroyStreamDataBlock(pStreamBlocks); return code; } @@ -166,7 +166,7 @@ static int32_t streamTaskExecImpl(SStreamTask* pTask, SStreamQueueItem* pItem, i pTask->info.selfChildId, numOfBlocks, SIZE_IN_MB(size)); // current output should be dispatched to down stream nodes - if (numOfBlocks >= STREAM_RESULT_DUMP_THRESHOLD) { + if (numOfBlocks >= STREAM_RESULT_DUMP_THRESHOLD || size >= STREAM_RESULT_DUMP_SIZE_THRESHOLD) { ASSERT(numOfBlocks == taosArrayGetSize(pRes)); code = doDumpResult(pTask, pItem, pRes, size, totalSize, totalBlocks); if (code != TSDB_CODE_SUCCESS) { @@ -192,6 +192,7 @@ static int32_t streamTaskExecImpl(SStreamTask* pTask, SStreamQueueItem* pItem, i int32_t streamScanHistoryData(SStreamTask* pTask) { ASSERT(pTask->info.taskLevel == TASK_LEVEL__SOURCE); + int32_t size = 0; int32_t code = TSDB_CODE_SUCCESS; void* exec = pTask->exec.pExecutor; bool finished = false; @@ -244,29 +245,24 @@ int32_t streamScanHistoryData(SStreamTask* pTask) { block.info.childId = pTask->info.selfChildId; taosArrayPush(pRes, &block); - if ((++numOfBlocks) >= outputBatchSize) { - qDebug("s-task:%s scan exec numOfBlocks:%d, output limit:%d reached", pTask->id.idStr, numOfBlocks, outputBatchSize); + size += blockDataGetSize(output) + sizeof(SSDataBlock) + sizeof(SColumnInfoData) * blockDataGetNumOfCols(&block); + + if ((++numOfBlocks) >= outputBatchSize || size >= STREAM_RESULT_DUMP_SIZE_THRESHOLD) { + qDebug("s-task:%s scan exec numOfBlocks:%d, output num-limit:%d, size-limit:%d reached", pTask->id.idStr, numOfBlocks, + outputBatchSize, STREAM_RESULT_DUMP_SIZE_THRESHOLD); break; } } if (taosArrayGetSize(pRes) > 0) { - SStreamDataBlock* qRes = taosAllocateQitem(sizeof(SStreamDataBlock), DEF_QITEM, 0); - if (qRes == NULL) { - taosArrayDestroyEx(pRes, (FDelete)blockDataFreeRes); - terrno = TSDB_CODE_OUT_OF_MEMORY; - return -1; - } - - qRes->type = STREAM_INPUT__DATA_BLOCK; - qRes->blocks = pRes; - - code = doOutputResultBlockImpl(pTask, qRes); - if (code == TSDB_CODE_UTIL_QUEUE_OUT_OF_MEMORY) { - taosArrayDestroyEx(pRes, (FDelete)blockDataFreeRes); - taosFreeQitem(qRes); + SStreamDataBlock* pStreamBlocks = createStreamBlockFromResults(NULL, pTask, size, pRes); + code = doOutputResultBlockImpl(pTask, pStreamBlocks); + if (code != TSDB_CODE_SUCCESS) { + destroyStreamDataBlock(pStreamBlocks); return code; } + + size = 0; } else { taosArrayDestroy(pRes); } From 1a86ee651725a27204346a26c90ba327447be273 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 19 Sep 2023 17:40:26 +0800 Subject: [PATCH 032/158] refactor: update the log level. --- source/dnode/vnode/src/tq/tqSink.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/dnode/vnode/src/tq/tqSink.c b/source/dnode/vnode/src/tq/tqSink.c index e0bae18545..da7ac20600 100644 --- a/source/dnode/vnode/src/tq/tqSink.c +++ b/source/dnode/vnode/src/tq/tqSink.c @@ -537,7 +537,7 @@ int32_t doConvertRows(SSubmitTbData* pTableData, STSchema* pTSchema, SSDataBlock if (k == 0) { SColumnInfoData* pColData = taosArrayGet(pDataBlock->pDataBlock, dataIndex); void* colData = colDataGetData(pColData, j); - tqDebug("s-task:%s sink row %d, col %d ts %" PRId64, id, j, k, *(int64_t*)colData); + tqTrace("s-task:%s sink row %d, col %d ts %" PRId64, id, j, k, *(int64_t*)colData); } if (IS_SET_NULL(pCol)) { From 927f2d896c4a4a29465c17a4dd8f073e79a45c83 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 19 Sep 2023 17:45:28 +0800 Subject: [PATCH 033/158] fix(stream): limit the max scan times. --- source/dnode/vnode/src/tq/tqStreamTask.c | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/source/dnode/vnode/src/tq/tqStreamTask.c b/source/dnode/vnode/src/tq/tqStreamTask.c index 3a5eeae561..854478f41e 100644 --- a/source/dnode/vnode/src/tq/tqStreamTask.c +++ b/source/dnode/vnode/src/tq/tqStreamTask.c @@ -16,6 +16,8 @@ #include "tq.h" #include "vnd.h" +#define MAX_REPEAT_SCAN_THRESHOLD 3 + static int32_t doScanWalForAllTasks(SStreamMeta* pStreamMeta, bool* pScanIdle); static int32_t setWalReaderStartOffset(SStreamTask* pTask, int32_t vgId); static void handleFillhistoryScanComplete(SStreamTask* pTask, int64_t ver); @@ -153,6 +155,9 @@ int32_t tqScanWalAsync(STQ* pTq, bool ckPause) { } pMeta->walScanCounter += 1; + if (pMeta->walScanCounter > MAX_REPEAT_SCAN_THRESHOLD) { + pMeta->walScanCounter = MAX_REPEAT_SCAN_THRESHOLD; + } if (pMeta->walScanCounter > 1) { tqDebug("vgId:%d wal read task has been launched, remain scan times:%d", vgId, pMeta->walScanCounter); From e27111ecd19c7d9b9061e62005be661faffbe202 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 20 Sep 2023 10:35:01 +0800 Subject: [PATCH 034/158] enh(stream): add node stage check. --- include/libs/stream/tstream.h | 2 + source/dnode/mnode/impl/src/mndStream.c | 63 ++++++++++++++++++------- source/libs/stream/src/streamMeta.c | 21 +++++---- 3 files changed, 61 insertions(+), 25 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 5329da2f17..1dd016526f 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -556,6 +556,8 @@ int32_t tDecodeStreamCheckpointReadyMsg(SDecoder* pDecoder, SStreamCheckpointRea typedef struct STaskStatusEntry { STaskId id; int32_t status; + int32_t stage; + int32_t nodeId; } STaskStatusEntry; typedef struct SStreamHbMsg { diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index 68856e11f1..2cacd4af69 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -37,17 +37,18 @@ typedef struct SNodeEntry { int32_t nodeId; + bool stageUpdated; // the stage has been updated due to the leader/follower change or node reboot. SEpSet epset; // compare the epset to identify the vgroup tranferring between different dnodes. int64_t hbTimestamp; // second } SNodeEntry; -typedef struct SStreamVnodeRevertIndex { +typedef struct SStreamExecNodeInfo { SArray *pNodeEntryList; int64_t ts; // snapshot ts SHashObj *pTaskMap; SArray *pTaskList; TdThreadMutex lock; -} SStreamVnodeRevertIndex; +} SStreamExecNodeInfo; typedef struct SVgroupChangeInfo { SHashObj *pDBMap; @@ -55,7 +56,7 @@ typedef struct SVgroupChangeInfo { } SVgroupChangeInfo; static int32_t mndNodeCheckSentinel = 0; -static SStreamVnodeRevertIndex execNodeList; +static SStreamExecNodeInfo execNodeList; static int32_t mndStreamActionInsert(SSdb *pSdb, SStreamObj *pStream); static int32_t mndStreamActionDelete(SSdb *pSdb, SStreamObj *pStream); @@ -75,7 +76,6 @@ static int32_t mndBuildStreamCheckpointSourceReq2(void **pBuf, int32_t *pLen, in int64_t streamId, int32_t taskId); static int32_t mndProcessNodeCheck(SRpcMsg *pReq); static int32_t mndProcessNodeCheckReq(SRpcMsg *pMsg); -static void keepStreamTasksInBuf(SStreamObj *pStream, SStreamVnodeRevertIndex *pExecNode); static SArray *doExtractNodeListFromStream(SMnode *pMnode); static SArray *mndTakeVgroupSnapshot(SMnode *pMnode); @@ -83,8 +83,8 @@ static SVgroupChangeInfo mndFindChangedNodeInfo(SMnode *pMnode, const SArray *pP static int32_t mndPersistTransLog(SStreamObj *pStream, STrans *pTrans); static void initTransAction(STransAction *pAction, void *pCont, int32_t contLen, int32_t msgType, const SEpSet *pEpset); -static void removeStreamTasksInBuf(SStreamObj* pStream, SStreamVnodeRevertIndex* pExecNode); -static void keepStreamTasksInBuf(SStreamObj *pStream, SStreamVnodeRevertIndex *pExecNode); +static void removeStreamTasksInBuf(SStreamObj* pStream, SStreamExecNodeInfo * pExecNode); +static void keepStreamTasksInBuf(SStreamObj *pStream, SStreamExecNodeInfo *pExecNode); int32_t mndInitStream(SMnode *pMnode) { SSdbTable table = { @@ -1158,12 +1158,19 @@ static int32_t mndProcessStreamDoCheckpoint(SRpcMsg *pReq) { } if (taosArrayGetSize(execNodeList.pNodeEntryList) == 0) { - mDebug("end to do stream task node change checking, no vgroup exists, do nothing"); + mDebug("stream task node change checking done, no vgroups exist, do nothing"); execNodeList.ts = ts; - atomic_store_32(&mndNodeCheckSentinel, 0); return 0; } + for(int32_t i = 0; i < taosArrayGetSize(execNodeList.pNodeEntryList); ++i) { + SNodeEntry* pNodeEntry = taosArrayGet(execNodeList.pNodeEntryList, i); + if (pNodeEntry->stageUpdated) { + mDebug("stream task not ready due to node update detected, checkpoint not issued"); + return 0; + } + } + SArray *pNodeSnapshot = mndTakeVgroupSnapshot(pMnode); SVgroupChangeInfo changeInfo = mndFindChangedNodeInfo(pMnode, execNodeList.pNodeEntryList, pNodeSnapshot); @@ -1173,7 +1180,7 @@ static int32_t mndProcessStreamDoCheckpoint(SRpcMsg *pReq) { taosArrayDestroy(pNodeSnapshot); if (nodeUpdated) { - mDebug("stream task not ready due to node update, not generate checkpoint"); + mDebug("stream task not ready due to node update, checkpoint not issued"); return 0; } } @@ -1190,7 +1197,7 @@ static int32_t mndProcessStreamDoCheckpoint(SRpcMsg *pReq) { } if (pEntry->status != TASK_STATUS__NORMAL) { - mDebug("s-task:0x%" PRIx64 "-0x%x (nodeId:%d) status:%s not ready, create checkpoint msg not issued", + mDebug("s-task:0x%" PRIx64 "-0x%x (nodeId:%d) status:%s not ready, checkpoint msg not issued", pEntry->id.streamId, (int32_t)pEntry->id.taskId, 0, streamGetTaskStatusStr(pEntry->status)); ready = false; break; @@ -2028,7 +2035,7 @@ static SVgroupChangeInfo mndFindChangedNodeInfo(SMnode *pMnode, const SArray *pP SNodeEntry *pCurrent = taosArrayGet(pNodeList, j); if (pCurrent->nodeId == pPrevEntry->nodeId) { - if (isNodeEpsetChanged(&pPrevEntry->epset, &pCurrent->epset)) { + if (pPrevEntry->stageUpdated || isNodeEpsetChanged(&pPrevEntry->epset, &pCurrent->epset)) { const SEp *pPrevEp = GET_ACTIVE_EP(&pPrevEntry->epset); char buf[256] = {0}; @@ -2202,6 +2209,7 @@ static int32_t mndProcessNodeCheckReq(SRpcMsg *pMsg) { SArray *pNodeSnapshot = mndTakeVgroupSnapshot(pMnode); + taosThreadMutexLock(&execNodeList.lock); SVgroupChangeInfo changeInfo = mndFindChangedNodeInfo(pMnode, execNodeList.pNodeEntryList, pNodeSnapshot); if (taosArrayGetSize(changeInfo.pUpdateNodeList) > 0) { code = mndProcessVgroupChange(pMnode, &changeInfo); @@ -2218,6 +2226,7 @@ static int32_t mndProcessNodeCheckReq(SRpcMsg *pMsg) { taosArrayDestroy(pNodeSnapshot); } + taosThreadMutexUnlock(&execNodeList.lock); taosArrayDestroy(changeInfo.pUpdateNodeList); taosHashCleanup(changeInfo.pDBMap); @@ -2244,7 +2253,7 @@ static int32_t mndProcessNodeCheck(SRpcMsg *pReq) { return 0; } -void keepStreamTasksInBuf(SStreamObj *pStream, SStreamVnodeRevertIndex *pExecNode) { +void keepStreamTasksInBuf(SStreamObj *pStream, SStreamExecNodeInfo *pExecNode) { int32_t level = taosArrayGetSize(pStream->tasks); for (int32_t i = 0; i < level; i++) { SArray *pLevel = taosArrayGetP(pStream->tasks, i); @@ -2256,8 +2265,11 @@ void keepStreamTasksInBuf(SStreamObj *pStream, SStreamVnodeRevertIndex *pExecNod STaskId id = {.streamId = pTask->id.streamId, .taskId = pTask->id.taskId}; void *p = taosHashGet(pExecNode->pTaskMap, &id, sizeof(id)); if (p == NULL) { - STaskStatusEntry entry = { - .id.streamId = pTask->id.streamId, .id.taskId = pTask->id.taskId, .status = TASK_STATUS__STOP}; + STaskStatusEntry entry = {.id.streamId = pTask->id.streamId, + .id.taskId = pTask->id.taskId, + .stage = -1, + .nodeId = pTask->info.nodeId, + .status = TASK_STATUS__STOP}; taosHashPut(pExecNode->pTaskMap, &id, sizeof(id), &entry, sizeof(entry)); taosArrayPush(pExecNode->pTaskList, &id); } @@ -2265,7 +2277,7 @@ void keepStreamTasksInBuf(SStreamObj *pStream, SStreamVnodeRevertIndex *pExecNod } } -void removeStreamTasksInBuf(SStreamObj* pStream, SStreamVnodeRevertIndex* pExecNode) { +void removeStreamTasksInBuf(SStreamObj* pStream, SStreamExecNodeInfo * pExecNode) { int32_t level = taosArrayGetSize(pStream->tasks); for (int32_t i = 0; i < level; i++) { SArray *pLevel = taosArrayGetP(pStream->tasks, i); @@ -2296,9 +2308,8 @@ void removeStreamTasksInBuf(SStreamObj* pStream, SStreamVnodeRevertIndex* pExecN // todo: this process should be executed by the write queue worker of the mnode int32_t mndProcessStreamHb(SRpcMsg *pReq) { - SMnode *pMnode = pReq->info.node; + SMnode *pMnode = pReq->info.node; SStreamHbMsg req = {0}; - int32_t code = TSDB_CODE_SUCCESS; SDecoder decoder = {0}; tDecoderInit(&decoder, pReq->pCont, pReq->contLen); @@ -2326,11 +2337,29 @@ int32_t mndProcessStreamHb(SRpcMsg *pReq) { continue; } + if (p->stage != pEntry->stage && pEntry->stage != -1) { + int32_t numOfNodes = taosArrayGetSize(execNodeList.pNodeEntryList); + for(int32_t j = 0; j < numOfNodes; ++j) { + SNodeEntry* pNodeEntry = taosArrayGet(execNodeList.pNodeEntryList, j); + if (pNodeEntry->nodeId == pEntry->nodeId) { + mInfo("vgId:%d stage updated, from %d to %d, nodeUpdate should be trigger by s-task:0x%" PRIx64, + pEntry->nodeId, pEntry->stage, p->stage, pEntry->id.taskId); + + pNodeEntry->stageUpdated = true; + pEntry->stage = p->stage; + break; + } + } + } else { + pEntry->stage = p->stage; + } + pEntry->status = p->status; if (p->status != TASK_STATUS__NORMAL) { mDebug("received s-task:0x%"PRIx64" not in ready status:%s", p->id.taskId, streamGetTaskStatusStr(p->status)); } } + taosThreadMutexUnlock(&execNodeList.lock); taosArrayDestroy(req.pTaskStatus); diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 66f05367c2..5f60632a7c 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -767,6 +767,8 @@ int32_t tEncodeStreamHbMsg(SEncoder* pEncoder, const SStreamHbMsg* pReq) { if (tEncodeI64(pEncoder, ps->id.streamId) < 0) return -1; if (tEncodeI32(pEncoder, ps->id.taskId) < 0) return -1; if (tEncodeI32(pEncoder, ps->status) < 0) return -1; + if (tEncodeI32(pEncoder, ps->stage) < 0) return -1; + if (tEncodeI32(pEncoder, ps->nodeId) < 0) return -1; } tEndEncode(pEncoder); return pEncoder->pos; @@ -779,15 +781,17 @@ int32_t tDecodeStreamHbMsg(SDecoder* pDecoder, SStreamHbMsg* pReq) { pReq->pTaskStatus = taosArrayInit(pReq->numOfTasks, sizeof(STaskStatusEntry)); for (int32_t i = 0; i < pReq->numOfTasks; ++i) { - STaskStatusEntry hb = {0}; - if (tDecodeI64(pDecoder, &hb.id.streamId) < 0) return -1; - int32_t taskId = 0; + int32_t taskId = 0; + STaskStatusEntry entry = {0}; + + if (tDecodeI64(pDecoder, &entry.id.streamId) < 0) return -1; if (tDecodeI32(pDecoder, &taskId) < 0) return -1; + if (tDecodeI32(pDecoder, &entry.status) < 0) return -1; + if (tDecodeI32(pDecoder, &entry.stage) < 0) return -1; + if (tDecodeI32(pDecoder, &entry.nodeId) < 0) return -1; - hb.id.taskId = taskId; - if (tDecodeI32(pDecoder, &hb.status) < 0) return -1; - - taosArrayPush(pReq->pTaskStatus, &hb); + entry.id.taskId = taskId; + taosArrayPush(pReq->pTaskStatus, &entry); } tEndDecode(pDecoder); @@ -856,7 +860,8 @@ void metaHbToMnode(void* param, void* tmrId) { continue; } - STaskStatusEntry entry = {.id = *pId, .status = (*pTask)->status.taskStatus}; + STaskStatusEntry entry = { + .id = *pId, .status = (*pTask)->status.taskStatus, .nodeId = pMeta->vgId, .stage = pMeta->stage}; taosArrayPush(hbMsg.pTaskStatus, &entry); if (!hasValEpset) { From 8af76d28c8d890cd5011ab46e897e7d3168cae35 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 20 Sep 2023 10:49:18 +0800 Subject: [PATCH 035/158] log(stream): remove logs. --- source/dnode/vnode/src/vnd/vnodeSvr.c | 3 --- 1 file changed, 3 deletions(-) diff --git a/source/dnode/vnode/src/vnd/vnodeSvr.c b/source/dnode/vnode/src/vnd/vnodeSvr.c index 6ff33c4f6e..446da86eb0 100644 --- a/source/dnode/vnode/src/vnd/vnodeSvr.c +++ b/source/dnode/vnode/src/vnd/vnodeSvr.c @@ -1443,11 +1443,8 @@ static int32_t vnodeProcessSubmitReq(SVnode *pVnode, int64_t ver, void *pReq, in SColData *pColData = (SColData *)taosArrayGet(pSubmitTbData->aCol, 0); TSKEY *aKey = (TSKEY *)(pColData->pData); - vDebug("vgId:%d submit %d rows data, uid:%"PRId64, TD_VID(pVnode), pColData->nVal, pSubmitTbData->uid); for (int32_t iRow = 0; iRow < pColData->nVal; iRow++) { - vDebug("vgId:%d uid:%"PRId64" ts:%"PRId64, TD_VID(pVnode), pSubmitTbData->uid, aKey[iRow]); - if (aKey[iRow] < minKey || aKey[iRow] > maxKey || (iRow > 0 && aKey[iRow] <= aKey[iRow - 1])) { code = TSDB_CODE_INVALID_MSG; vError("vgId:%d %s failed since %s, version:%" PRId64, TD_VID(pVnode), __func__, tstrerror(terrno), ver); From 53237c34d144bec8b5b29630608873951c9d66f2 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 20 Sep 2023 13:15:50 +0800 Subject: [PATCH 036/158] enh(stream): add info in log. --- source/dnode/mnode/impl/src/mndStream.c | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index 2cacd4af69..05cc043802 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -2272,6 +2272,8 @@ void keepStreamTasksInBuf(SStreamObj *pStream, SStreamExecNodeInfo *pExecNode) { .status = TASK_STATUS__STOP}; taosHashPut(pExecNode->pTaskMap, &id, sizeof(id), &entry, sizeof(entry)); taosArrayPush(pExecNode->pTaskList, &id); + mInfo("s-task:0x%x add into task buffer, total:%d", (int32_t)entry.id.taskId, + (int32_t)taosArrayGetSize(pExecNode->pTaskList)); } } } @@ -2295,6 +2297,8 @@ void removeStreamTasksInBuf(SStreamObj* pStream, SStreamExecNodeInfo * pExecNode STaskId* pId = taosArrayGet(pExecNode->pTaskList, k); if (pId->taskId == id.taskId && pId->streamId == id.streamId) { taosArrayRemove(pExecNode->pTaskList, k); + mInfo("s-task:0x%x removed from buffer, remain:%d", (int32_t)id.taskId, + (int32_t)taosArrayGetSize(pExecNode->pTaskList)); break; } } From dff4bd3c8d18a890e3d48629adc559e191127fb1 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 20 Sep 2023 13:18:42 +0800 Subject: [PATCH 037/158] log(tsdb): remove some logs. --- source/dnode/vnode/src/vnd/vnodeSvr.c | 3 --- 1 file changed, 3 deletions(-) diff --git a/source/dnode/vnode/src/vnd/vnodeSvr.c b/source/dnode/vnode/src/vnd/vnodeSvr.c index 446da86eb0..016636a815 100644 --- a/source/dnode/vnode/src/vnd/vnodeSvr.c +++ b/source/dnode/vnode/src/vnd/vnodeSvr.c @@ -1455,10 +1455,7 @@ static int32_t vnodeProcessSubmitReq(SVnode *pVnode, int64_t ver, void *pReq, in } else { int32_t nRow = TARRAY_SIZE(pSubmitTbData->aRowP); SRow **aRow = (SRow **)TARRAY_DATA(pSubmitTbData->aRowP); - - vDebug("vgId:%d submit %d rows data, uid:%"PRId64, TD_VID(pVnode), nRow, pSubmitTbData->uid); for (int32_t iRow = 0; iRow < nRow; ++iRow) { - vDebug("vgId:%d uid:%"PRId64" ts:%"PRId64, TD_VID(pVnode), pSubmitTbData->uid, aRow[iRow]->ts); if (aRow[iRow]->ts < minKey || aRow[iRow]->ts > maxKey || (iRow > 0 && aRow[iRow]->ts <= aRow[iRow - 1]->ts)) { code = TSDB_CODE_INVALID_MSG; From a997fa5552cbebfec2cc543b5155cabe073d27df Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 20 Sep 2023 14:49:09 +0800 Subject: [PATCH 038/158] fix(stream): fix deadlock caused by streamTaskHalt function. --- source/dnode/vnode/src/tq/tq.c | 45 +++++++++++++++++++++++++++++----- 1 file changed, 39 insertions(+), 6 deletions(-) diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 95637fad69..758bf55666 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1126,12 +1126,45 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { // now we can stop the stream task execution int64_t latestVer = 0; - taosThreadMutexLock(&pStreamTask->lock); - streamTaskHalt(pStreamTask); - tqDebug("s-task:%s level:%d sched-status:%d is halt by fill-history task:%s", pStreamTask->id.idStr, - pStreamTask->info.taskLevel, pStreamTask->status.schedStatus, id); - latestVer = walReaderGetCurrentVer(pStreamTask->exec.pWalReader); - taosThreadMutexUnlock(&pStreamTask->lock); + while (1) { + taosThreadMutexLock(&pStreamTask->lock); + int8_t status = pTask->status.taskStatus; + if (status == TASK_STATUS__DROPPING || status == TASK_STATUS__STOP) { + // return; + // do nothing + } + + if (status == TASK_STATUS__HALT) { + // return; + // do nothing + } + + if (pTask->status.taskStatus == TASK_STATUS__CK) { + qDebug("s-task:%s status:%s during generating checkpoint, wait for 1sec and retry set status:halt", + pTask->id.idStr, streamGetTaskStatusStr(TASK_STATUS__CK)); + taosThreadMutexUnlock(&pStreamTask->lock); + taosMsleep(1000); + continue; + } + + // upgrade to halt status + if (status == TASK_STATUS__PAUSE) { + qDebug("s-task:%s upgrade status to %s from %s", pTask->id.idStr, streamGetTaskStatusStr(TASK_STATUS__HALT), + streamGetTaskStatusStr(TASK_STATUS__PAUSE)); + } else { + qDebug("s-task:%s halt task", pTask->id.idStr); + } + + pTask->status.keepTaskStatus = status; + pTask->status.taskStatus = TASK_STATUS__HALT; + + tqDebug("s-task:%s level:%d sched-status:%d is halt by fill-history task:%s", pStreamTask->id.idStr, + pStreamTask->info.taskLevel, pStreamTask->status.schedStatus, id); + latestVer = walReaderGetCurrentVer(pStreamTask->exec.pWalReader); + + taosThreadMutexUnlock(&pStreamTask->lock); + break; + } // if it's an source task, extract the last version in wal. pRange = &pTask->dataRange.range; From 29e1efb7571f3dc218cae4a7eaddb94dbeb2ba07 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 20 Sep 2023 14:50:43 +0800 Subject: [PATCH 039/158] fix(stream): fix obj. --- source/dnode/vnode/src/tq/tq.c | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 758bf55666..65fa9be5fd 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1128,7 +1128,7 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { while (1) { taosThreadMutexLock(&pStreamTask->lock); - int8_t status = pTask->status.taskStatus; + int8_t status = pStreamTask->status.taskStatus; if (status == TASK_STATUS__DROPPING || status == TASK_STATUS__STOP) { // return; // do nothing @@ -1139,9 +1139,9 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { // do nothing } - if (pTask->status.taskStatus == TASK_STATUS__CK) { + if (pStreamTask->status.taskStatus == TASK_STATUS__CK) { qDebug("s-task:%s status:%s during generating checkpoint, wait for 1sec and retry set status:halt", - pTask->id.idStr, streamGetTaskStatusStr(TASK_STATUS__CK)); + pStreamTask->id.idStr, streamGetTaskStatusStr(TASK_STATUS__CK)); taosThreadMutexUnlock(&pStreamTask->lock); taosMsleep(1000); continue; @@ -1149,14 +1149,14 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { // upgrade to halt status if (status == TASK_STATUS__PAUSE) { - qDebug("s-task:%s upgrade status to %s from %s", pTask->id.idStr, streamGetTaskStatusStr(TASK_STATUS__HALT), + qDebug("s-task:%s upgrade status to %s from %s", pStreamTask->id.idStr, streamGetTaskStatusStr(TASK_STATUS__HALT), streamGetTaskStatusStr(TASK_STATUS__PAUSE)); } else { - qDebug("s-task:%s halt task", pTask->id.idStr); + qDebug("s-task:%s halt task", pStreamTask->id.idStr); } - pTask->status.keepTaskStatus = status; - pTask->status.taskStatus = TASK_STATUS__HALT; + pStreamTask->status.keepTaskStatus = status; + pStreamTask->status.taskStatus = TASK_STATUS__HALT; tqDebug("s-task:%s level:%d sched-status:%d is halt by fill-history task:%s", pStreamTask->id.idStr, pStreamTask->info.taskLevel, pStreamTask->status.schedStatus, id); From d76e3ac718201840a4ef5eda7f18b51c1b25f0c6 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 20 Sep 2023 15:32:15 +0800 Subject: [PATCH 040/158] fix(stream): add task status check. --- source/dnode/vnode/src/tq/tq.c | 14 +++++++++++++- source/libs/stream/src/streamCheckpoint.c | 4 ---- 2 files changed, 13 insertions(+), 5 deletions(-) diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 65fa9be5fd..d7aaacca57 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1644,6 +1644,19 @@ int32_t tqProcessStreamCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg) { return TSDB_CODE_SUCCESS; } + taosThreadMutexLock(&pTask->lock); + if (pTask->status.taskStatus == TASK_STATUS__HALT) { + qError("s-task:%s not ready for checkpoint, since it is halt, ignore this checkpoint:%" PRId64 + ", set it failure", pTask->id.idStr, req.checkpointId); + streamMetaReleaseTask(pMeta, pTask); + + SRpcMsg rsp = {0}; + buildCheckpointSourceRsp(&req, &pMsg->info, &rsp, 0); + tmsgSendRsp(&rsp); // error occurs + } + streamProcessCheckpointSourceReq(pTask, &req); + taosThreadMutexUnlock(&pTask->lock); + int32_t total = 0; taosWLockLatch(&pMeta->lock); @@ -1666,7 +1679,6 @@ int32_t tqProcessStreamCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg) { // todo: when generating checkpoint, no new tasks are allowed to add into current Vnode // todo: when generating checkpoint, leader of mnode has transfer to other DNode? - streamProcessCheckpointSourceReq(pTask, &req); streamMetaReleaseTask(pMeta, pTask); return code; } diff --git a/source/libs/stream/src/streamCheckpoint.c b/source/libs/stream/src/streamCheckpoint.c index f367ba932f..0200abcf98 100644 --- a/source/libs/stream/src/streamCheckpoint.c +++ b/source/libs/stream/src/streamCheckpoint.c @@ -136,8 +136,6 @@ int32_t streamProcessCheckpointSourceReq(SStreamTask* pTask, SStreamCheckpointSo ASSERT(pTask->info.taskLevel == TASK_LEVEL__SOURCE); // 1. set task status to be prepared for check point, no data are allowed to put into inputQ. - taosThreadMutexLock(&pTask->lock); - pTask->status.taskStatus = TASK_STATUS__CK; pTask->checkpointingId = pReq->checkpointId; pTask->checkpointNotReadyTasks = streamTaskGetNumOfDownstream(pTask); @@ -146,8 +144,6 @@ int32_t streamProcessCheckpointSourceReq(SStreamTask* pTask, SStreamCheckpointSo // 2. let's dispatch checkpoint msg to downstream task directly and do nothing else. put the checkpoint block into // inputQ, to make sure all blocks with less version have been handled by this task already. int32_t code = appendCheckpointIntoInputQ(pTask, STREAM_INPUT__CHECKPOINT_TRIGGER); - taosThreadMutexUnlock(&pTask->lock); - return code; } From 36dcdad1a5c3f8412bb58eb65bf1416478f25efa Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 20 Sep 2023 15:38:49 +0800 Subject: [PATCH 041/158] fix(stream): return if failed. --- source/dnode/vnode/src/tq/tq.c | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index d7aaacca57..35b6d4ddc5 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1646,13 +1646,14 @@ int32_t tqProcessStreamCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg) { taosThreadMutexLock(&pTask->lock); if (pTask->status.taskStatus == TASK_STATUS__HALT) { - qError("s-task:%s not ready for checkpoint, since it is halt, ignore this checkpoint:%" PRId64 - ", set it failure", pTask->id.idStr, req.checkpointId); + qError("s-task:%s not ready for checkpoint, since it is halt, ignore this checkpoint:%" PRId64 ", set it failure", + pTask->id.idStr, req.checkpointId); streamMetaReleaseTask(pMeta, pTask); SRpcMsg rsp = {0}; buildCheckpointSourceRsp(&req, &pMsg->info, &rsp, 0); tmsgSendRsp(&rsp); // error occurs + return TSDB_CODE_SUCCESS; } streamProcessCheckpointSourceReq(pTask, &req); taosThreadMutexUnlock(&pTask->lock); From f5da458c77aaa668933e585202f2576241fc81e8 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 20 Sep 2023 15:45:15 +0800 Subject: [PATCH 042/158] fix(stream): unlock when returning. --- source/dnode/vnode/src/tq/tq.c | 1 + 1 file changed, 1 insertion(+) diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 35b6d4ddc5..7b203a37b8 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1653,6 +1653,7 @@ int32_t tqProcessStreamCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg) { SRpcMsg rsp = {0}; buildCheckpointSourceRsp(&req, &pMsg->info, &rsp, 0); tmsgSendRsp(&rsp); // error occurs + taosThreadMutexUnlock(&pTask->lock); return TSDB_CODE_SUCCESS; } streamProcessCheckpointSourceReq(pTask, &req); From 02610f554609a21100a2e3614ba1d998ffafca69 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 20 Sep 2023 15:47:09 +0800 Subject: [PATCH 043/158] fix(stream): avoid invalid read. --- source/dnode/vnode/src/tq/tq.c | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 7b203a37b8..3941b35db7 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1648,12 +1648,13 @@ int32_t tqProcessStreamCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg) { if (pTask->status.taskStatus == TASK_STATUS__HALT) { qError("s-task:%s not ready for checkpoint, since it is halt, ignore this checkpoint:%" PRId64 ", set it failure", pTask->id.idStr, req.checkpointId); + taosThreadMutexUnlock(&pTask->lock); + streamMetaReleaseTask(pMeta, pTask); SRpcMsg rsp = {0}; buildCheckpointSourceRsp(&req, &pMsg->info, &rsp, 0); tmsgSendRsp(&rsp); // error occurs - taosThreadMutexUnlock(&pTask->lock); return TSDB_CODE_SUCCESS; } streamProcessCheckpointSourceReq(pTask, &req); From c403a15f36d49841cd9d9ac948d4f7e1284fd6ea Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 20 Sep 2023 16:32:27 +0800 Subject: [PATCH 044/158] fix(stream): check the status before exec stream task when handling the dispatch requests. --- source/libs/stream/src/stream.c | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/source/libs/stream/src/stream.c b/source/libs/stream/src/stream.c index 5a7e14c629..676f3e1502 100644 --- a/source/libs/stream/src/stream.c +++ b/source/libs/stream/src/stream.c @@ -255,10 +255,14 @@ int32_t streamProcessDispatchMsg(SStreamTask* pTask, SStreamDispatchReq* pReq, S tDeleteStreamDispatchReq(pReq); - int8_t schedStatus = streamTaskSetSchedStatusWait(pTask); - if (schedStatus == TASK_SCHED_STATUS__INACTIVE) { - streamTryExec(pTask); + int8_t st = pTask->status.taskStatus; + if (st == TASK_STATUS__NORMAL || st == TASK_STATUS__SCAN_HISTORY || st == TASK_STATUS__CK) { + int8_t schedStatus = streamTaskSetSchedStatusWait(pTask); + if (schedStatus == TASK_SCHED_STATUS__INACTIVE) { + streamTryExec(pTask); + } } + return 0; } From 1ebafe4a1f6f74c3c95387ed2f6ff113cb51ebcc Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 20 Sep 2023 16:49:55 +0800 Subject: [PATCH 045/158] fix(stream): add more check. --- source/libs/stream/src/stream.c | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/source/libs/stream/src/stream.c b/source/libs/stream/src/stream.c index 676f3e1502..38f7a9e748 100644 --- a/source/libs/stream/src/stream.c +++ b/source/libs/stream/src/stream.c @@ -241,6 +241,11 @@ int32_t streamProcessDispatchMsg(SStreamTask* pTask, SStreamDispatchReq* pReq, S } } + int8_t st = pTask->status.taskStatus; + if (st == TASK_STATUS__HALT) { + status = TASK_INPUT_STATUS__BLOCKED; + } + { // do send response with the input status int32_t code = buildDispatchRsp(pTask, pReq, status, &pRsp->pCont); @@ -255,7 +260,6 @@ int32_t streamProcessDispatchMsg(SStreamTask* pTask, SStreamDispatchReq* pReq, S tDeleteStreamDispatchReq(pReq); - int8_t st = pTask->status.taskStatus; if (st == TASK_STATUS__NORMAL || st == TASK_STATUS__SCAN_HISTORY || st == TASK_STATUS__CK) { int8_t schedStatus = streamTaskSetSchedStatusWait(pTask); if (schedStatus == TASK_SCHED_STATUS__INACTIVE) { From d752fb1ed6a432dd1e021c40706ce4a2a0f7934d Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 20 Sep 2023 17:30:41 +0800 Subject: [PATCH 046/158] fix(stream): exec task in task queuee threads. --- source/libs/stream/src/stream.c | 8 +------- 1 file changed, 1 insertion(+), 7 deletions(-) diff --git a/source/libs/stream/src/stream.c b/source/libs/stream/src/stream.c index 38f7a9e748..f842ed2178 100644 --- a/source/libs/stream/src/stream.c +++ b/source/libs/stream/src/stream.c @@ -259,13 +259,7 @@ int32_t streamProcessDispatchMsg(SStreamTask* pTask, SStreamDispatchReq* pReq, S } tDeleteStreamDispatchReq(pReq); - - if (st == TASK_STATUS__NORMAL || st == TASK_STATUS__SCAN_HISTORY || st == TASK_STATUS__CK) { - int8_t schedStatus = streamTaskSetSchedStatusWait(pTask); - if (schedStatus == TASK_SCHED_STATUS__INACTIVE) { - streamTryExec(pTask); - } - } + streamSchedExec(pTask); return 0; } From 1f45bd82d27a7a698d449bdd40313e5bfd62e2f5 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 20 Sep 2023 18:18:31 +0800 Subject: [PATCH 047/158] other: merge stream fix in main. --- include/libs/stream/tstream.h | 4 + include/util/tlog.h | 1 + source/common/src/tglobal.c | 1 + source/dnode/mnode/impl/src/mndDnode.c | 2 +- source/dnode/vnode/src/inc/tq.h | 1 - source/dnode/vnode/src/tq/tqSink.c | 5 +- source/libs/stream/inc/streamInt.h | 10 +- source/libs/stream/src/stream.c | 29 ++-- source/libs/stream/src/streamBackendRocksdb.c | 163 +++++++++--------- source/libs/stream/src/streamCheckpoint.c | 28 +-- source/libs/stream/src/streamData.c | 2 +- source/libs/stream/src/streamDispatch.c | 72 ++++---- source/libs/stream/src/streamExec.c | 77 ++++----- source/libs/stream/src/streamMeta.c | 100 +++++------ source/libs/stream/src/streamQueue.c | 50 +++--- source/libs/stream/src/streamRecover.c | 129 +++++++------- source/libs/stream/src/streamSnapshot.c | 37 ++-- source/libs/stream/src/streamState.c | 16 +- source/libs/stream/src/streamTask.c | 38 ++-- source/util/src/tlog.c | 1 + 20 files changed, 394 insertions(+), 372 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 1dd016526f..eeb4d6d4f3 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -29,6 +29,9 @@ extern "C" { #ifndef _STREAM_H_ #define _STREAM_H_ +#define ONE_MB_F (1048576.0) +#define SIZE_IN_MB(_v) ((_v) / ONE_MB_F) + typedef struct SStreamTask SStreamTask; #define SSTREAM_TASK_VER 2 @@ -306,6 +309,7 @@ typedef struct SSinkTaskRecorder { int64_t numOfSubmit; int64_t numOfBlocks; int64_t numOfRows; + int64_t bytes; } SSinkTaskRecorder; typedef struct { diff --git a/include/util/tlog.h b/include/util/tlog.h index 5a421033c9..a6d146a79e 100644 --- a/include/util/tlog.h +++ b/include/util/tlog.h @@ -55,6 +55,7 @@ extern int32_t tmrDebugFlag; extern int32_t uDebugFlag; extern int32_t rpcDebugFlag; extern int32_t qDebugFlag; +extern int32_t stDebugFlag; extern int32_t wDebugFlag; extern int32_t sDebugFlag; extern int32_t tsdbDebugFlag; diff --git a/source/common/src/tglobal.c b/source/common/src/tglobal.c index 648bb486b1..80184784f3 100644 --- a/source/common/src/tglobal.c +++ b/source/common/src/tglobal.c @@ -404,6 +404,7 @@ static int32_t taosAddServerLogCfg(SConfig *pCfg) { if (cfgAddInt32(pCfg, "idxDebugFlag", idxDebugFlag, 0, 255, CFG_SCOPE_SERVER) != 0) return -1; if (cfgAddInt32(pCfg, "tdbDebugFlag", tdbDebugFlag, 0, 255, CFG_SCOPE_SERVER) != 0) return -1; if (cfgAddInt32(pCfg, "metaDebugFlag", metaDebugFlag, 0, 255, 0) != CFG_SCOPE_SERVER) return -1; + if (cfgAddInt32(pCfg, "stDebugFlag", stDebugFlag, 0, 255, CFG_SCOPE_CLIENT) != 0) return -1; return 0; } diff --git a/source/dnode/mnode/impl/src/mndDnode.c b/source/dnode/mnode/impl/src/mndDnode.c index a11bd2556e..94ec6c0cc3 100644 --- a/source/dnode/mnode/impl/src/mndDnode.c +++ b/source/dnode/mnode/impl/src/mndDnode.c @@ -1090,7 +1090,7 @@ static int32_t mndProcessConfigDnodeReq(SRpcMsg *pReq) { const char *options[] = { "debugFlag", "dDebugFlag", "vDebugFlag", "mDebugFlag", "wDebugFlag", "sDebugFlag", "tsdbDebugFlag", "tqDebugFlag", "fsDebugFlag", "udfDebugFlag", "smaDebugFlag", "idxDebugFlag", "tdbDebugFlag", "tmrDebugFlag", - "uDebugFlag", "smaDebugFlag", "rpcDebugFlag", "qDebugFlag", "metaDebugFlag", + "uDebugFlag", "smaDebugFlag", "rpcDebugFlag", "qDebugFlag", "metaDebugFlag", "stDebugFlag", }; int32_t optionSize = tListLen(options); diff --git a/source/dnode/vnode/src/inc/tq.h b/source/dnode/vnode/src/inc/tq.h index 4e73a481c8..7f91aed4eb 100644 --- a/source/dnode/vnode/src/inc/tq.h +++ b/source/dnode/vnode/src/inc/tq.h @@ -39,7 +39,6 @@ extern "C" { #define tqInfo(...) do { if (tqDebugFlag & DEBUG_INFO) { taosPrintLog("TQ ", DEBUG_INFO, 255, __VA_ARGS__); }} while(0) #define tqDebug(...) do { if (tqDebugFlag & DEBUG_DEBUG) { taosPrintLog("TQ ", DEBUG_DEBUG, tqDebugFlag, __VA_ARGS__); }} while(0) #define tqTrace(...) do { if (tqDebugFlag & DEBUG_TRACE) { taosPrintLog("TQ ", DEBUG_TRACE, tqDebugFlag, __VA_ARGS__); }} while(0) - // clang-format on typedef struct STqOffsetStore STqOffsetStore; diff --git a/source/dnode/vnode/src/tq/tqSink.c b/source/dnode/vnode/src/tq/tqSink.c index da7ac20600..f700294e79 100644 --- a/source/dnode/vnode/src/tq/tqSink.c +++ b/source/dnode/vnode/src/tq/tqSink.c @@ -276,8 +276,8 @@ int32_t doBuildAndSendSubmitMsg(SVnode* pVnode, SStreamTask* pTask, SSubmitReq2* SSinkTaskRecorder* pRec = &pTask->sinkRecorder; double el = (taosGetTimestampMs() - pTask->taskExecInfo.start) / 1000.0; tqInfo("s-task:%s vgId:%d write %" PRId64 " blocks (%" PRId64 " rows) in %" PRId64 - " submit into dst table, duration:%.2f Sec.", - pTask->id.idStr, vgId, pRec->numOfBlocks, pRec->numOfRows, pRec->numOfSubmit, el); + " submit into dst table, %.2fMiB duration:%.2f Sec.", + pTask->id.idStr, vgId, pRec->numOfBlocks, pRec->numOfRows, pRec->numOfSubmit, SIZE_IN_MB(pRec->bytes), el); } return TSDB_CODE_SUCCESS; @@ -868,6 +868,7 @@ void tqSinkDataIntoDstTable(SStreamTask* pTask, void* vnode, void* data) { } pTask->sinkRecorder.numOfRows += pDataBlock->info.rows; + pTask->sinkRecorder.bytes += pDataBlock->info.rowSize; } taosHashCleanup(pTableIndexMap); diff --git a/source/libs/stream/inc/streamInt.h b/source/libs/stream/inc/streamInt.h index 64df8e2f44..edf36d2a1c 100644 --- a/source/libs/stream/inc/streamInt.h +++ b/source/libs/stream/inc/streamInt.h @@ -26,8 +26,14 @@ extern "C" { #endif -#define ONE_MB_F (1048576.0) -#define SIZE_IN_MB(_v) ((_v) / ONE_MB_F) +// clang-format off +#define stFatal(...) do { if (stDebugFlag & DEBUG_FATAL) { taosPrintLog("STM FATAL ", DEBUG_FATAL, 255, __VA_ARGS__); }} while(0) +#define stError(...) do { if (stDebugFlag & DEBUG_ERROR) { taosPrintLog("STM ERROR ", DEBUG_ERROR, 255, __VA_ARGS__); }} while(0) +#define stWarn(...) do { if (stDebugFlag & DEBUG_WARN) { taosPrintLog("STM WARN ", DEBUG_WARN, 255, __VA_ARGS__); }} while(0) +#define stInfo(...) do { if (stDebugFlag & DEBUG_INFO) { taosPrintLog("STM ", DEBUG_INFO, 255, __VA_ARGS__); }} while(0) +#define stDebug(...) do { if (stDebugFlag & DEBUG_DEBUG) { taosPrintLog("STM ", DEBUG_DEBUG, tqDebugFlag, __VA_ARGS__); }} while(0) +#define stTrace(...) do { if (stDebugFlag & DEBUG_TRACE) { taosPrintLog("STM ", DEBUG_TRACE, tqDebugFlag, __VA_ARGS__); }} while(0) +// clang-format on typedef struct SStreamGlobalEnv { int8_t inited; diff --git a/source/libs/stream/src/stream.c b/source/libs/stream/src/stream.c index f842ed2178..c964d0b811 100644 --- a/source/libs/stream/src/stream.c +++ b/source/libs/stream/src/stream.c @@ -60,10 +60,10 @@ static void streamSchedByTimer(void* param, void* tmrId) { SStreamTask* pTask = (void*)param; int8_t status = atomic_load_8(&pTask->schedInfo.status); - qDebug("s-task:%s in scheduler, trigger status:%d, next:%dms", pTask->id.idStr, status, (int32_t)pTask->info.triggerParam); + stDebug("s-task:%s in scheduler, trigger status:%d, next:%dms", pTask->id.idStr, status, (int32_t)pTask->info.triggerParam); if (streamTaskShouldStop(&pTask->status) || streamTaskShouldPause(&pTask->status)) { - qDebug("s-task:%s jump out of schedTimer", pTask->id.idStr); + stDebug("s-task:%s jump out of schedTimer", pTask->id.idStr); return; } @@ -98,7 +98,7 @@ int32_t streamSetupScheduleTrigger(SStreamTask* pTask) { int32_t ref = atomic_add_fetch_32(&pTask->refCnt, 1); ASSERT(ref == 2 && pTask->schedInfo.pTimer == NULL); - qDebug("s-task:%s setup scheduler trigger, delay:%" PRId64 " ms", pTask->id.idStr, pTask->info.triggerParam); + stDebug("s-task:%s setup scheduler trigger, delay:%" PRId64 " ms", pTask->id.idStr, pTask->info.triggerParam); pTask->schedInfo.pTimer = taosTmrStart(streamSchedByTimer, (int32_t)pTask->info.triggerParam, pTask, streamEnv.timer); pTask->schedInfo.status = TASK_TRIGGER_STATUS__INACTIVE; @@ -114,7 +114,7 @@ int32_t streamSchedExec(SStreamTask* pTask) { if (pRunReq == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; /*int8_t status = */streamTaskSetSchedStatusInActive(pTask); - qError("failed to create msg to aunch s-task:%s, reason out of memory", pTask->id.idStr); + stError("failed to create msg to aunch s-task:%s, reason out of memory", pTask->id.idStr); return -1; } @@ -122,12 +122,12 @@ int32_t streamSchedExec(SStreamTask* pTask) { pRunReq->streamId = pTask->id.streamId; pRunReq->taskId = pTask->id.taskId; - qDebug("trigger to run s-task:%s", pTask->id.idStr); + stDebug("trigger to run s-task:%s", pTask->id.idStr); SRpcMsg msg = {.msgType = TDMT_STREAM_TASK_RUN, .pCont = pRunReq, .contLen = sizeof(SStreamTaskRunReq)}; tmsgPutToQueue(pTask->pMsgCb, STREAM_QUEUE, &msg); } else { - qDebug("s-task:%s not launch task since sched status:%d", pTask->id.idStr, pTask->status.schedStatus); + stDebug("s-task:%s not launch task since sched status:%d", pTask->id.idStr, pTask->status.schedStatus); } return 0; @@ -159,7 +159,7 @@ static int32_t streamTaskAppendInputBlocks(SStreamTask* pTask, const SStreamDisp if (pBlock == NULL) { streamTaskInputFail(pTask); status = TASK_INPUT_STATUS__FAILED; - qError("vgId:%d, s-task:%s failed to receive dispatch msg, reason: out of memory", pTask->pMeta->vgId, + stError("vgId:%d, s-task:%s failed to receive dispatch msg, reason: out of memory", pTask->pMeta->vgId, pTask->id.idStr); } else { if (pBlock->type == STREAM_INPUT__TRANS_STATE) { @@ -180,7 +180,7 @@ int32_t streamTaskEnqueueRetrieve(SStreamTask* pTask, SStreamRetrieveReq* pReq, // enqueue if (pData != NULL) { - qDebug("s-task:%s (child %d) recv retrieve req from task:0x%x(vgId:%d), reqId:0x%" PRIx64, pTask->id.idStr, + stDebug("s-task:%s (child %d) recv retrieve req from task:0x%x(vgId:%d), reqId:0x%" PRIx64, pTask->id.idStr, pTask->info.selfChildId, pReq->srcTaskId, pReq->srcNodeId, pReq->reqId); pData->type = STREAM_INPUT__DATA_RETRIEVE; @@ -211,7 +211,7 @@ int32_t streamTaskEnqueueRetrieve(SStreamTask* pTask, SStreamRetrieveReq* pReq, } int32_t streamProcessDispatchMsg(SStreamTask* pTask, SStreamDispatchReq* pReq, SRpcMsg* pRsp, bool exec) { - qDebug("s-task:%s receive dispatch msg from taskId:0x%x(vgId:%d), msgLen:%" PRId64, pTask->id.idStr, + stDebug("s-task:%s receive dispatch msg from taskId:0x%x(vgId:%d), msgLen:%" PRId64, pTask->id.idStr, pReq->upstreamTaskId, pReq->upstreamNodeId, pReq->totalLen); int32_t status = 0; @@ -220,21 +220,20 @@ int32_t streamProcessDispatchMsg(SStreamTask* pTask, SStreamDispatchReq* pReq, S // upstream task has restarted/leader-follower switch/transferred to other dnodes if (pReq->stage > pInfo->stage) { - qError("s-task:%s upstream task:0x%x (vgId:%d) has restart/leader-switch/vnode-transfer, prev stage:%" PRId64 + stError("s-task:%s upstream task:0x%x (vgId:%d) has restart/leader-switch/vnode-transfer, prev stage:%" PRId64 ", current:%" PRId64 " dispatch msg rejected", pTask->id.idStr, pReq->upstreamTaskId, pReq->upstreamNodeId, pInfo->stage, pReq->stage); status = TASK_INPUT_STATUS__BLOCKED; } else { if (!pInfo->dataAllowed) { - qWarn("s-task:%s data from task:0x%x is denied, since inputQ is closed for it", pTask->id.idStr, + stWarn("s-task:%s data from task:0x%x is denied, since inputQ is closed for it", pTask->id.idStr, pReq->upstreamTaskId); status = TASK_INPUT_STATUS__BLOCKED; } else { - // Current task has received the checkpoint req from the upstream task, from which the message should all be - // blocked + // This task has received the checkpoint req from the upstream task, from which all the messages should be blocked if (pReq->type == STREAM_INPUT__CHECKPOINT_TRIGGER) { streamTaskCloseUpstreamInput(pTask, pReq->upstreamTaskId); - qDebug("s-task:%s close inputQ for upstream:0x%x", pTask->id.idStr, pReq->upstreamTaskId); + stDebug("s-task:%s close inputQ for upstream:0x%x", pTask->id.idStr, pReq->upstreamTaskId); } status = streamTaskAppendInputBlocks(pTask, pReq); @@ -308,6 +307,6 @@ SStreamChildEpInfo* streamTaskGetUpstreamTaskEpInfo(SStreamTask* pTask, int32_t } } - qError("s-task:%s failed to find upstream task:0x%x", pTask->id.idStr, taskId); + stError("s-task:%s failed to find upstream task:0x%x", pTask->id.idStr, taskId); return NULL; } \ No newline at end of file diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index 8a80d74c63..db5d431e30 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -297,12 +297,16 @@ int32_t bkdMgtDumpTo(SBackendManager* bm, char* dname) { sprintf(dstDir, "%s%s%s", bm->path, TD_DIRSEP, dname); if (!taosDirExist(srcDir)) { - return 0; + stError("failed to dump srcDir %s, reason: not exist such dir", srcDir); + code = -1; + goto _ERROR; } code = taosMkDir(dstDir); if (code != 0) { - return code; + terrno = TAOS_SYSTEM_ERROR(errno); + stError("failed to mkdir srcDir %s, reason: %s", dstDir, terrstr()); + goto _ERROR; } // clear current file @@ -437,13 +441,13 @@ int32_t rebuildDirFromCheckpoint(const char* path, int64_t chkpId, char** dst) { taosMkDir(state); code = copyFiles(chkp, state); if (code != 0) { - qError("failed to restart stream backend from %s, reason: %s", chkp, tstrerror(TAOS_SYSTEM_ERROR(errno))); + stError("failed to restart stream backend from %s, reason: %s", chkp, tstrerror(TAOS_SYSTEM_ERROR(errno))); } else { - qInfo("start to restart stream backend at checkpoint path: %s", chkp); + stInfo("start to restart stream backend at checkpoint path: %s", chkp); } } else { - qError("failed to start stream backend at %s, reason: %s, restart from default state dir:%s", chkp, + stError("failed to start stream backend at %s, reason: %s, restart from default state dir:%s", chkp, tstrerror(TAOS_SYSTEM_ERROR(errno)), state); taosMkDir(state); } @@ -458,7 +462,7 @@ void* streamBackendInit(const char* streamPath, int64_t chkpId) { char* backendPath = NULL; int32_t code = rebuildDirFromCheckpoint(streamPath, chkpId, &backendPath); - qDebug("start to init stream backend at %s, checkpointid: %" PRId64 "", backendPath, chkpId); + stDebug("start to init stream backend at %s, checkpointid: %" PRId64 "", backendPath, chkpId); uint32_t dbMemLimit = nextPow2(tsMaxStreamBackendCache) << 20; SBackendWrapper* pHandle = taosMemoryCalloc(1, sizeof(SBackendWrapper)); @@ -502,7 +506,7 @@ void* streamBackendInit(const char* streamPath, int64_t chkpId) { taosMemoryFreeClear(err); pHandle->db = rocksdb_open(opts, backendPath, &err); if (err != NULL) { - qError("failed to open rocksdb, path:%s, reason:%s", backendPath, err); + stError("failed to open rocksdb, path:%s, reason:%s", backendPath, err); taosMemoryFreeClear(err); goto _EXIT; } @@ -519,7 +523,7 @@ void* streamBackendInit(const char* streamPath, int64_t chkpId) { if (cfs != NULL) { rocksdb_list_column_families_destroy(cfs, nCf); } - qDebug("succ to init stream backend at %s, backend:%p", backendPath, pHandle); + stDebug("succ to init stream backend at %s, backend:%p", backendPath, pHandle); taosMemoryFreeClear(backendPath); return (void*)pHandle; @@ -532,7 +536,7 @@ _EXIT: taosHashCleanup(pHandle->cfInst); tdListFree(pHandle->list); taosMemoryFree(pHandle); - qDebug("failed to init stream backend at %s", backendPath); + stDebug("failed to init stream backend at %s", backendPath); taosMemoryFree(backendPath); return NULL; } @@ -566,7 +570,7 @@ void streamBackendCleanup(void* arg) { taosThreadMutexDestroy(&pHandle->mutex); taosThreadMutexDestroy(&pHandle->cfMutex); - qDebug("destroy stream backend :%p", pHandle); + stDebug("destroy stream backend :%p", pHandle); taosMemoryFree(pHandle); return; } @@ -575,7 +579,7 @@ void streamBackendHandleCleanup(void* arg) { bool remove = wrapper->remove; taosThreadRwlockWrlock(&wrapper->rwLock); - qDebug("start to do-close backendwrapper %p, %s", wrapper, wrapper->idstr); + stDebug("start to do-close backendwrapper %p, %s", wrapper, wrapper->idstr); if (wrapper->rocksdb == NULL) { taosThreadRwlockUnlock(&wrapper->rwLock); return; @@ -588,7 +592,7 @@ void streamBackendHandleCleanup(void* arg) { for (int i = 0; i < cfLen; i++) { if (wrapper->pHandle[i] != NULL) rocksdb_drop_column_family(wrapper->rocksdb, wrapper->pHandle[i], &err); if (err != NULL) { - qError("failed to drop cf:%s_%s, reason:%s", wrapper->idstr, ginitDict[i].key, err); + stError("failed to drop cf:%s_%s, reason:%s", wrapper->idstr, ginitDict[i].key, err); taosMemoryFreeClear(err); } } @@ -599,7 +603,7 @@ void streamBackendHandleCleanup(void* arg) { for (int i = 0; i < cfLen; i++) { if (wrapper->pHandle[i] != NULL) rocksdb_flush_cf(wrapper->rocksdb, flushOpt, wrapper->pHandle[i], &err); if (err != NULL) { - qError("failed to flush cf:%s_%s, reason:%s", wrapper->idstr, ginitDict[i].key, err); + stError("failed to flush cf:%s_%s, reason:%s", wrapper->idstr, ginitDict[i].key, err); taosMemoryFreeClear(err); } } @@ -634,7 +638,7 @@ void streamBackendHandleCleanup(void* arg) { wrapper->rocksdb = NULL; taosReleaseRef(streamBackendId, wrapper->backendId); - qDebug("end to do-close backendwrapper %p, %s", wrapper, wrapper->idstr); + stDebug("end to do-close backendwrapper %p, %s", wrapper, wrapper->idstr); taosMemoryFree(wrapper); return; } @@ -810,14 +814,14 @@ int32_t chkpDoDbCheckpoint(rocksdb_t* db, char* path) { char* err = NULL; rocksdb_checkpoint_t* cp = rocksdb_checkpoint_object_create(db, &err); if (cp == NULL || err != NULL) { - qError("failed to do checkpoint at:%s, reason:%s", path, err); + stError("failed to do checkpoint at:%s, reason:%s", path, err); taosMemoryFreeClear(err); goto _ERROR; } rocksdb_checkpoint_create(cp, path, 64 << 20, &err); if (err != NULL) { - qError("failed to do checkpoint at:%s, reason:%s", path, err); + stError("failed to do checkpoint at:%s, reason:%s", path, err); taosMemoryFreeClear(err); } else { code = 0; @@ -835,7 +839,7 @@ int32_t chkpPreFlushDb(rocksdb_t* db, rocksdb_column_family_handle_t** cf, int32 rocksdb_flush_cfs(db, flushOpt, cf, nCf, &err); if (err != NULL) { - qError("failed to flush db before streamBackend clean up, reason:%s", err); + stError("failed to flush db before streamBackend clean up, reason:%s", err); taosMemoryFree(err); code = -1; } @@ -850,7 +854,7 @@ int32_t chkpPreCheckDir(char* path, int64_t chkpId, char** chkpDir, char** chkpI sprintf(pChkpDir, "%s%s%s", path, TD_DIRSEP, "checkpoints"); code = taosMulModeMkDir(pChkpDir, 0755, true); if (code != 0) { - qError("failed to prepare checkpoint dir, path:%s, reason:%s", path, tstrerror(code)); + stError("failed to prepare checkpoint dir, path:%s, reason:%s", path, tstrerror(code)); taosMemoryFree(pChkpDir); taosMemoryFree(pChkpIdDir); code = -1; @@ -859,7 +863,7 @@ int32_t chkpPreCheckDir(char* path, int64_t chkpId, char** chkpDir, char** chkpI sprintf(pChkpIdDir, "%s%scheckpoint%" PRId64, pChkpDir, TD_DIRSEP, chkpId); if (taosIsDir(pChkpIdDir)) { - qInfo("stream rm exist checkpoint%s", pChkpIdDir); + stInfo("stream rm exist checkpoint%s", pChkpIdDir); taosRemoveFile(pChkpIdDir); } *chkpDir = pChkpDir; @@ -883,19 +887,19 @@ int32_t streamBackendTriggerChkp(void* arg, char* dst) { goto _ERROR; } int32_t nCf = chkpGetAllDbCfHandle(pMeta, &ppCf, refs); - qDebug("stream backend:%p start to do checkpoint at:%s, cf num: %d ", pHandle, dst, nCf); + stDebug("stream backend:%p start to do checkpoint at:%s, cf num: %d ", pHandle, dst, nCf); code = chkpPreFlushDb(pHandle->db, ppCf, nCf); if (code == 0) { code = chkpDoDbCheckpoint(pHandle->db, dst); if (code != 0) { - qError("stream backend:%p failed to do checkpoint at:%s", pHandle, dst); + stError("stream backend:%p failed to do checkpoint at:%s", pHandle, dst); } else { - qDebug("stream backend:%p end to do checkpoint at:%s, time cost:%" PRId64 "ms", pHandle, dst, + stDebug("stream backend:%p end to do checkpoint at:%s, time cost:%" PRId64 "ms", pHandle, dst, taosGetTimestampMs() - st); } } else { - qError("stream backend:%p failed to flush db at:%s", pHandle, dst); + stError("stream backend:%p failed to flush db at:%s", pHandle, dst); } // release all ref to cfWrapper; @@ -957,19 +961,19 @@ int32_t streamBackendDoCheckpoint(void* arg, uint64_t checkpointId) { // Get all cf and acquire cfWrappter int32_t nCf = chkpGetAllDbCfHandle(pMeta, &ppCf, refs); - qDebug("stream backend:%p start to do checkpoint at:%s, cf num: %d ", pHandle, pChkpIdDir, nCf); + stDebug("stream backend:%p start to do checkpoint at:%s, cf num: %d ", pHandle, pChkpIdDir, nCf); code = chkpPreFlushDb(pHandle->db, ppCf, nCf); if (code == 0) { code = chkpDoDbCheckpoint(pHandle->db, pChkpIdDir); if (code != 0) { - qError("stream backend:%p failed to do checkpoint at:%s", pHandle, pChkpIdDir); + stError("stream backend:%p failed to do checkpoint at:%s", pHandle, pChkpIdDir); } else { - qDebug("stream backend:%p end to do checkpoint at:%s, time cost:%" PRId64 "ms", pHandle, pChkpIdDir, + stDebug("stream backend:%p end to do checkpoint at:%s, time cost:%" PRId64 "ms", pHandle, pChkpIdDir, taosGetTimestampMs() - st); } } else { - qError("stream backend:%p failed to flush db at:%s", pHandle, pChkpIdDir); + stError("stream backend:%p failed to flush db at:%s", pHandle, pChkpIdDir); } // release all ref to cfWrapper; for (int i = 0; i < taosArrayGetSize(refs); i++) { @@ -1375,7 +1379,7 @@ int32_t decodeValueFunc(void* value, int32_t vlen, int64_t* ttl, char** dest) { p = taosDecodeFixedI64(p, &key.unixTimestamp); p = taosDecodeFixedI32(p, &key.len); if (vlen != (sizeof(int64_t) + sizeof(int32_t) + key.len)) { - qError("vlen: %d, read len: %d", vlen, key.len); + stError("vlen: %d, read len: %d", vlen, key.len); goto _EXCEPT; } if (key.len != 0 && dest != NULL) p = taosDecodeBinary(p, (void**)dest, key.len); @@ -1483,7 +1487,11 @@ int32_t streamStateOpenBackendCf(void* backend, char* name, char** cfs, int32_t rocksdb_options_set_block_based_table_factory((rocksdb_options_t*)cfOpts[i], tableOpt); params[i].tableOpt = tableOpt; - int idx = streamStateGetCfIdx(NULL, funcname); + int idx = streamStateGetCfIdx(NULL, funcname); + if (idx < 0 || idx >= sizeof(ginitDict) / sizeof(ginitDict[0])) { + stError("failed to open cf"); + return -1; + } SCfInit* cfPara = &ginitDict[idx]; rocksdb_comparator_t* compare = @@ -1495,7 +1503,7 @@ int32_t streamStateOpenBackendCf(void* backend, char* name, char** cfs, int32_t rocksdb_t* db = rocksdb_open_column_families(handle->dbOpt, name, nCf, (const char* const*)cfs, (const rocksdb_options_t* const*)cfOpts, cfHandle, &err); if (err != NULL) { - qError("failed to open rocksdb cf, reason:%s", err); + stError("failed to open rocksdb cf, reason:%s", err); taosMemoryFree(err); taosMemoryFree(cfHandle); taosMemoryFree(pCompare); @@ -1504,7 +1512,7 @@ int32_t streamStateOpenBackendCf(void* backend, char* name, char** cfs, int32_t // fix other leak return -1; } else { - qDebug("succ to open rocksdb cf"); + stDebug("succ to open rocksdb cf"); } // close default cf if (((rocksdb_column_family_handle_t**)cfHandle)[0] != 0) { @@ -1615,7 +1623,7 @@ int streamStateOpenBackend(void* backend, SStreamState* pState) { int64_t id = taosAddRef(streamBackendCfWrapperId, pBackendCfWrapper); pState->pTdbState->backendCfWrapperId = id; pState->pTdbState->pBackendCfWrapper = pBackendCfWrapper; - qInfo("succ to open state %p on backendWrapper, %p, %s", pState, pBackendCfWrapper, pBackendCfWrapper->idstr); + stInfo("succ to open state %p on backendWrapper, %p, %s", pState, pBackendCfWrapper, pBackendCfWrapper->idstr); return 0; } taosThreadMutexUnlock(&handle->cfMutex); @@ -1667,7 +1675,7 @@ int streamStateOpenBackend(void* backend, SStreamState* pState) { int64_t id = taosAddRef(streamBackendCfWrapperId, pBackendCfWrapper); pState->pTdbState->backendCfWrapperId = id; pState->pTdbState->pBackendCfWrapper = pBackendCfWrapper; - qInfo("succ to open state %p on backendWrapper %p %s", pState, pBackendCfWrapper, pBackendCfWrapper->idstr); + stInfo("succ to open state %p on backendWrapper %p %s", pState, pBackendCfWrapper, pBackendCfWrapper->idstr); return 0; } @@ -1675,7 +1683,7 @@ void streamStateCloseBackend(SStreamState* pState, bool remove) { SBackendCfWrapper* wrapper = pState->pTdbState->pBackendCfWrapper; SBackendWrapper* pHandle = wrapper->pBackend; - qInfo("start to close state on backend: %p", pHandle); + stInfo("start to close state on backend: %p", pHandle); taosThreadMutexLock(&pHandle->cfMutex); RocksdbCfInst** ppInst = taosHashGet(pHandle->cfInst, wrapper->idstr, strlen(pState->pTdbState->idstr) + 1); @@ -1687,7 +1695,7 @@ void streamStateCloseBackend(SStreamState* pState, bool remove) { taosThreadMutexUnlock(&pHandle->cfMutex); char* status[] = {"close", "drop"}; - qInfo("start to %s state %p on backendWrapper %p %s", status[remove == false ? 0 : 1], pState, wrapper, + stInfo("start to %s state %p on backendWrapper %p %s", status[remove == false ? 0 : 1], pState, wrapper, wrapper->idstr); wrapper->remove |= remove; // update by other pState taosReleaseRef(streamBackendCfWrapperId, pState->pTdbState->backendCfWrapperId); @@ -1724,9 +1732,10 @@ int streamStateGetCfIdx(SStreamState* pState, const char* funcName) { cf = rocksdb_create_column_family(wrapper->rocksdb, wrapper->cfOpts[idx], buf, &err); if (err != NULL) { idx = -1; - qError("failed to to open cf, %p %s_%s, reason:%s", pState, wrapper->idstr, funcName, err); + stError("failed to open cf, %p %s_%s, reason:%s", pState, wrapper->idstr, funcName, err); taosMemoryFree(err); } else { + stDebug("succ to to open cf, %p %s_%s", pState, wrapper->idstr, funcName); wrapper->pHandle[idx] = cf; } taosThreadRwlockUnlock(&wrapper->rwLock); @@ -1769,7 +1778,7 @@ rocksdb_iterator_t* streamStateIterCreate(SStreamState* pState, const char* cfKe char* err = NULL; \ int i = streamStateGetCfIdx(pState, funcname); \ if (i < 0) { \ - qWarn("streamState failed to get cf name: %s", funcname); \ + stWarn("streamState failed to get cf name: %s", funcname); \ code = -1; \ break; \ } \ @@ -1784,11 +1793,11 @@ rocksdb_iterator_t* streamStateIterCreate(SStreamState* pState, const char* cfKe int32_t ttlVLen = ginitDict[i].enValueFunc((char*)value, vLen, 0, &ttlV); \ rocksdb_put_cf(db, opts, pHandle, (const char*)buf, klen, (const char*)ttlV, (size_t)ttlVLen, &err); \ if (err != NULL) { \ - qError("streamState str: %s failed to write to %s, err: %s", toString, funcname, err); \ + stError("streamState str: %s failed to write to %s, err: %s", toString, funcname, err); \ taosMemoryFree(err); \ code = -1; \ } else { \ - qTrace("streamState str:%s succ to write to %s, rowValLen:%d, ttlValLen:%d", toString, funcname, vLen, ttlVLen); \ + stTrace("streamState str:%s succ to write to %s, rowValLen:%d, ttlValLen:%d", toString, funcname, vLen, ttlVLen); \ } \ taosMemoryFree(ttlV); \ } while (0); @@ -1800,7 +1809,7 @@ rocksdb_iterator_t* streamStateIterCreate(SStreamState* pState, const char* cfKe char* err = NULL; \ int i = streamStateGetCfIdx(pState, funcname); \ if (i < 0) { \ - qWarn("streamState failed to get cf name: %s", funcname); \ + stWarn("streamState failed to get cf name: %s", funcname); \ code = -1; \ break; \ } \ @@ -1815,9 +1824,9 @@ rocksdb_iterator_t* streamStateIterCreate(SStreamState* pState, const char* cfKe char* val = rocksdb_get_cf(db, opts, pHandle, (const char*)buf, klen, (size_t*)&len, &err); \ if (val == NULL || len == 0) { \ if (err == NULL) { \ - qTrace("streamState str: %s failed to read from %s_%s, err: not exist", toString, wrapper->idstr, funcname); \ + stTrace("streamState str: %s failed to read from %s_%s, err: not exist", toString, wrapper->idstr, funcname); \ } else { \ - qError("streamState str: %s failed to read from %s_%s, err: %s", toString, wrapper->idstr, funcname, err); \ + stError("streamState str: %s failed to read from %s_%s, err: %s", toString, wrapper->idstr, funcname, err); \ taosMemoryFreeClear(err); \ } \ code = -1; \ @@ -1825,11 +1834,11 @@ rocksdb_iterator_t* streamStateIterCreate(SStreamState* pState, const char* cfKe char* p = NULL; \ int32_t tlen = ginitDict[i].deValueFunc(val, len, NULL, (char**)pVal); \ if (tlen <= 0) { \ - qError("streamState str: %s failed to read from %s_%s, err: already ttl ", toString, wrapper->idstr, \ + stError("streamState str: %s failed to read from %s_%s, err: already ttl ", toString, wrapper->idstr, \ funcname); \ code = -1; \ } else { \ - qTrace("streamState str: %s succ to read from %s_%s, valLen:%d", toString, wrapper->idstr, funcname, tlen); \ + stTrace("streamState str: %s succ to read from %s_%s, valLen:%d", toString, wrapper->idstr, funcname, tlen); \ } \ taosMemoryFree(val); \ if (vLen != NULL) *vLen = tlen; \ @@ -1843,7 +1852,7 @@ rocksdb_iterator_t* streamStateIterCreate(SStreamState* pState, const char* cfKe char* err = NULL; \ int i = streamStateGetCfIdx(pState, funcname); \ if (i < 0) { \ - qWarn("streamState failed to get cf name: %s_%s", pState->pTdbState->idstr, funcname); \ + stWarn("streamState failed to get cf name: %s_%s", pState->pTdbState->idstr, funcname); \ code = -1; \ break; \ } \ @@ -1856,11 +1865,11 @@ rocksdb_iterator_t* streamStateIterCreate(SStreamState* pState, const char* cfKe rocksdb_writeoptions_t* opts = wrapper->writeOpts; \ rocksdb_delete_cf(db, opts, pHandle, (const char*)buf, klen, &err); \ if (err != NULL) { \ - qError("streamState str: %s failed to del from %s_%s, err: %s", toString, wrapper->idstr, funcname, err); \ + stError("streamState str: %s failed to del from %s_%s, err: %s", toString, wrapper->idstr, funcname, err); \ taosMemoryFree(err); \ code = -1; \ } else { \ - qTrace("streamState str: %s succ to del from %s_%s", toString, wrapper->idstr, funcname); \ + stTrace("streamState str: %s succ to del from %s_%s", toString, wrapper->idstr, funcname); \ } \ } while (0); @@ -1885,7 +1894,7 @@ int32_t streamStateDel_rocksdb(SStreamState* pState, const SWinKey* key) { return code; } int32_t streamStateClear_rocksdb(SStreamState* pState) { - qDebug("streamStateClear_rocksdb"); + stDebug("streamStateClear_rocksdb"); SBackendCfWrapper* wrapper = pState->pTdbState->pBackendCfWrapper; @@ -1907,7 +1916,7 @@ int32_t streamStateClear_rocksdb(SStreamState* pState) { stateKeyToString(&sKey, toStringStart); stateKeyToString(&eKey, toStringEnd); - qWarn("failed to delete range cf(state) start: %s, end:%s, reason:%s", toStringStart, toStringEnd, err); + stWarn("failed to delete range cf(state) start: %s, end:%s, reason:%s", toStringStart, toStringEnd, err); taosMemoryFree(err); } else { rocksdb_compact_range_cf(wrapper->rocksdb, wrapper->pHandle[1], sKeyStr, sLen, eKeyStr, eLen); @@ -1924,7 +1933,7 @@ int32_t streamStateCurNext_rocksdb(SStreamState* pState, SStreamStateCur* pCur) return 0; } int32_t streamStateGetFirst_rocksdb(SStreamState* pState, SWinKey* key) { - qDebug("streamStateGetFirst_rocksdb"); + stDebug("streamStateGetFirst_rocksdb"); SWinKey tmp = {.ts = 0, .groupId = 0}; streamStatePut_rocksdb(pState, &tmp, NULL, 0); @@ -1936,7 +1945,7 @@ int32_t streamStateGetFirst_rocksdb(SStreamState* pState, SWinKey* key) { } int32_t streamStateGetGroupKVByCur_rocksdb(SStreamStateCur* pCur, SWinKey* pKey, const void** pVal, int32_t* pVLen) { - qDebug("streamStateGetGroupKVByCur_rocksdb"); + stDebug("streamStateGetGroupKVByCur_rocksdb"); if (!pCur) { return -1; } @@ -1953,7 +1962,7 @@ int32_t streamStateGetGroupKVByCur_rocksdb(SStreamStateCur* pCur, SWinKey* pKey, return -1; } int32_t streamStateAddIfNotExist_rocksdb(SStreamState* pState, const SWinKey* key, void** pVal, int32_t* pVLen) { - qDebug("streamStateAddIfNotExist_rocksdb"); + stDebug("streamStateAddIfNotExist_rocksdb"); int32_t size = *pVLen; if (streamStateGet_rocksdb(pState, key, pVal, pVLen) == 0) { return 0; @@ -1963,14 +1972,14 @@ int32_t streamStateAddIfNotExist_rocksdb(SStreamState* pState, const SWinKey* ke return 0; } int32_t streamStateCurPrev_rocksdb(SStreamState* pState, SStreamStateCur* pCur) { - qDebug("streamStateCurPrev_rocksdb"); + stDebug("streamStateCurPrev_rocksdb"); if (!pCur) return -1; rocksdb_iter_prev(pCur->iter); return 0; } int32_t streamStateGetKVByCur_rocksdb(SStreamStateCur* pCur, SWinKey* pKey, const void** pVal, int32_t* pVLen) { - qDebug("streamStateGetKVByCur_rocksdb"); + stDebug("streamStateGetKVByCur_rocksdb"); if (!pCur) return -1; SStateKey tkey; SStateKey* pKtmp = &tkey; @@ -1995,7 +2004,7 @@ int32_t streamStateGetKVByCur_rocksdb(SStreamStateCur* pCur, SWinKey* pKey, cons return -1; } SStreamStateCur* streamStateGetAndCheckCur_rocksdb(SStreamState* pState, SWinKey* key) { - qDebug("streamStateGetAndCheckCur_rocksdb"); + stDebug("streamStateGetAndCheckCur_rocksdb"); SStreamStateCur* pCur = streamStateFillGetCur_rocksdb(pState, key); if (pCur) { int32_t code = streamStateGetGroupKVByCur_rocksdb(pCur, key, NULL, 0); @@ -2006,7 +2015,7 @@ SStreamStateCur* streamStateGetAndCheckCur_rocksdb(SStreamState* pState, SWinKey } SStreamStateCur* streamStateSeekKeyNext_rocksdb(SStreamState* pState, const SWinKey* key) { - qDebug("streamStateSeekKeyNext_rocksdb"); + stDebug("streamStateSeekKeyNext_rocksdb"); SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); if (pCur == NULL) { return NULL; @@ -2045,7 +2054,7 @@ SStreamStateCur* streamStateSeekKeyNext_rocksdb(SStreamState* pState, const SWin } SStreamStateCur* streamStateSeekToLast_rocksdb(SStreamState* pState, const SWinKey* key) { - qDebug("streamStateGetCur_rocksdb"); + stDebug("streamStateGetCur_rocksdb"); int32_t code = 0; const SStateKey maxStateKey = {.key = {.groupId = UINT64_MAX, .ts = INT64_MAX}, .opNum = INT64_MAX}; @@ -2060,7 +2069,7 @@ SStreamStateCur* streamStateSeekToLast_rocksdb(SStreamState* pState, const SWinK { char tbuf[256] = {0}; stateKeyToString((void*)&maxStateKey, tbuf); - qDebug("seek to last:%s", tbuf); + stDebug("seek to last:%s", tbuf); } SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); @@ -2087,7 +2096,7 @@ SStreamStateCur* streamStateSeekToLast_rocksdb(SStreamState* pState, const SWinK } SStreamStateCur* streamStateGetCur_rocksdb(SStreamState* pState, const SWinKey* key) { - qDebug("streamStateGetCur_rocksdb"); + stDebug("streamStateGetCur_rocksdb"); SBackendCfWrapper* wrapper = pState->pTdbState->pBackendCfWrapper; SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); @@ -2144,7 +2153,7 @@ int32_t streamStateSessionPut_rocksdb(SStreamState* pState, const SSessionKey* k return code; } int32_t streamStateSessionGet_rocksdb(SStreamState* pState, SSessionKey* key, void** pVal, int32_t* pVLen) { - qDebug("streamStateSessionGet_rocksdb"); + stDebug("streamStateSessionGet_rocksdb"); int code = 0; SStreamStateCur* pCur = streamStateSessionSeekKeyCurrentNext_rocksdb(pState, key); SSessionKey resKey = *key; @@ -2176,7 +2185,7 @@ int32_t streamStateSessionDel_rocksdb(SStreamState* pState, const SSessionKey* k return code; } SStreamStateCur* streamStateSessionSeekKeyCurrentPrev_rocksdb(SStreamState* pState, const SSessionKey* key) { - qDebug("streamStateSessionSeekKeyCurrentPrev_rocksdb"); + stDebug("streamStateSessionSeekKeyCurrentPrev_rocksdb"); SBackendCfWrapper* wrapper = pState->pTdbState->pBackendCfWrapper; SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); @@ -2217,7 +2226,7 @@ SStreamStateCur* streamStateSessionSeekKeyCurrentPrev_rocksdb(SStreamState* pSta return pCur; } SStreamStateCur* streamStateSessionSeekKeyCurrentNext_rocksdb(SStreamState* pState, SSessionKey* key) { - qDebug("streamStateSessionSeekKeyCurrentNext_rocksdb"); + stDebug("streamStateSessionSeekKeyCurrentNext_rocksdb"); SBackendCfWrapper* wrapper = pState->pTdbState->pBackendCfWrapper; SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); if (pCur == NULL) { @@ -2255,7 +2264,7 @@ SStreamStateCur* streamStateSessionSeekKeyCurrentNext_rocksdb(SStreamState* pSta } SStreamStateCur* streamStateSessionSeekKeyNext_rocksdb(SStreamState* pState, const SSessionKey* key) { - qDebug("streamStateSessionSeekKeyNext_rocksdb"); + stDebug("streamStateSessionSeekKeyNext_rocksdb"); SBackendCfWrapper* wrapper = pState->pTdbState->pBackendCfWrapper; SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); if (pCur == NULL) { @@ -2294,7 +2303,7 @@ SStreamStateCur* streamStateSessionSeekKeyNext_rocksdb(SStreamState* pState, con return pCur; } int32_t streamStateSessionGetKVByCur_rocksdb(SStreamStateCur* pCur, SSessionKey* pKey, void** pVal, int32_t* pVLen) { - qDebug("streamStateSessionGetKVByCur_rocksdb"); + stDebug("streamStateSessionGetKVByCur_rocksdb"); if (!pCur) { return -1; } @@ -2357,7 +2366,7 @@ int32_t streamStateFillDel_rocksdb(SStreamState* pState, const SWinKey* key) { } SStreamStateCur* streamStateFillGetCur_rocksdb(SStreamState* pState, const SWinKey* key) { - qDebug("streamStateFillGetCur_rocksdb"); + stDebug("streamStateFillGetCur_rocksdb"); SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); SBackendCfWrapper* wrapper = pState->pTdbState->pBackendCfWrapper; @@ -2393,7 +2402,7 @@ SStreamStateCur* streamStateFillGetCur_rocksdb(SStreamState* pState, const SWinK return NULL; } int32_t streamStateFillGetKVByCur_rocksdb(SStreamStateCur* pCur, SWinKey* pKey, const void** pVal, int32_t* pVLen) { - qDebug("streamStateFillGetKVByCur_rocksdb"); + stDebug("streamStateFillGetKVByCur_rocksdb"); if (!pCur) { return -1; } @@ -2417,7 +2426,7 @@ int32_t streamStateFillGetKVByCur_rocksdb(SStreamStateCur* pCur, SWinKey* pKey, } SStreamStateCur* streamStateFillSeekKeyNext_rocksdb(SStreamState* pState, const SWinKey* key) { - qDebug("streamStateFillSeekKeyNext_rocksdb"); + stDebug("streamStateFillSeekKeyNext_rocksdb"); SBackendCfWrapper* wrapper = pState->pTdbState->pBackendCfWrapper; SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); if (!pCur) { @@ -2455,7 +2464,7 @@ SStreamStateCur* streamStateFillSeekKeyNext_rocksdb(SStreamState* pState, const return NULL; } SStreamStateCur* streamStateFillSeekKeyPrev_rocksdb(SStreamState* pState, const SWinKey* key) { - qDebug("streamStateFillSeekKeyPrev_rocksdb"); + stDebug("streamStateFillSeekKeyPrev_rocksdb"); SBackendCfWrapper* wrapper = pState->pTdbState->pBackendCfWrapper; SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); if (pCur == NULL) { @@ -2493,7 +2502,7 @@ SStreamStateCur* streamStateFillSeekKeyPrev_rocksdb(SStreamState* pState, const return NULL; } int32_t streamStateSessionGetKeyByRange_rocksdb(SStreamState* pState, const SSessionKey* key, SSessionKey* curKey) { - qDebug("streamStateSessionGetKeyByRange_rocksdb"); + stDebug("streamStateSessionGetKeyByRange_rocksdb"); SBackendCfWrapper* wrapper = pState->pTdbState->pBackendCfWrapper; SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); if (pCur == NULL) { @@ -2552,7 +2561,7 @@ int32_t streamStateSessionGetKeyByRange_rocksdb(SStreamState* pState, const SSes int32_t streamStateSessionAddIfNotExist_rocksdb(SStreamState* pState, SSessionKey* key, TSKEY gap, void** pVal, int32_t* pVLen) { - qDebug("streamStateSessionAddIfNotExist_rocksdb"); + stDebug("streamStateSessionAddIfNotExist_rocksdb"); // todo refactor int32_t res = 0; SSessionKey originKey = *key; @@ -2602,7 +2611,7 @@ _end: return res; } int32_t streamStateSessionClear_rocksdb(SStreamState* pState) { - qDebug("streamStateSessionClear_rocksdb"); + stDebug("streamStateSessionClear_rocksdb"); SSessionKey key = {.win.skey = 0, .win.ekey = 0, .groupId = 0}; SStreamStateCur* pCur = streamStateSessionSeekKeyCurrentNext_rocksdb(pState, &key); @@ -2628,7 +2637,7 @@ int32_t streamStateSessionClear_rocksdb(SStreamState* pState) { } int32_t streamStateStateAddIfNotExist_rocksdb(SStreamState* pState, SSessionKey* key, char* pKeyData, int32_t keyDataLen, state_key_cmpr_fn fn, void** pVal, int32_t* pVLen) { - qDebug("streamStateStateAddIfNotExist_rocksdb"); + stDebug("streamStateStateAddIfNotExist_rocksdb"); // todo refactor int32_t res = 0; SSessionKey tmpKey = *key; @@ -2821,7 +2830,7 @@ int32_t streamStatePutBatch(SStreamState* pState, const char* cfKeyName, rocksdb int i = streamStateGetCfIdx(pState, cfKeyName); if (i < 0) { - qError("streamState failed to put to cf name:%s", cfKeyName); + stError("streamState failed to put to cf name:%s", cfKeyName); return -1; } @@ -2838,7 +2847,7 @@ int32_t streamStatePutBatch(SStreamState* pState, const char* cfKeyName, rocksdb { char tbuf[256] = {0}; ginitDict[i].toStrFunc((void*)key, tbuf); - qDebug("streamState str: %s succ to write to %s_%s, len: %d", tbuf, wrapper->idstr, ginitDict[i].key, vlen); + stDebug("streamState str: %s succ to write to %s_%s, len: %d", tbuf, wrapper->idstr, ginitDict[i].key, vlen); } return 0; } @@ -2861,7 +2870,7 @@ int32_t streamStatePutBatchOptimize(SStreamState* pState, int32_t cfIdx, rocksdb { char tbuf[256] = {0}; ginitDict[cfIdx].toStrFunc((void*)key, tbuf); - qDebug("streamState str: %s succ to write to %s_%s", tbuf, wrapper->idstr, ginitDict[cfIdx].key); + stDebug("streamState str: %s succ to write to %s_%s", tbuf, wrapper->idstr, ginitDict[cfIdx].key); } return 0; } @@ -2870,11 +2879,11 @@ int32_t streamStatePutBatch_rocksdb(SStreamState* pState, void* pBatch) { SBackendCfWrapper* wrapper = pState->pTdbState->pBackendCfWrapper; rocksdb_write(wrapper->rocksdb, wrapper->writeOpts, (rocksdb_writebatch_t*)pBatch, &err); if (err != NULL) { - qError("streamState failed to write batch, err:%s", err); + stError("streamState failed to write batch, err:%s", err); taosMemoryFree(err); return -1; } else { - qDebug("write batch to backend:%p", wrapper->pBackend); + stDebug("write batch to backend:%p", wrapper->pBackend); } return 0; } diff --git a/source/libs/stream/src/streamCheckpoint.c b/source/libs/stream/src/streamCheckpoint.c index 0200abcf98..77b7d0e614 100644 --- a/source/libs/stream/src/streamCheckpoint.c +++ b/source/libs/stream/src/streamCheckpoint.c @@ -95,7 +95,7 @@ static int32_t streamAlignCheckpoint(SStreamTask* pTask) { int32_t num = taosArrayGetSize(pTask->pUpstreamInfoList); int64_t old = atomic_val_compare_exchange_32(&pTask->checkpointAlignCnt, 0, num); if (old == 0) { - qDebug("s-task:%s set initial align upstream num:%d", pTask->id.idStr, num); + stDebug("s-task:%s set initial align upstream num:%d", pTask->id.idStr, num); } return atomic_sub_fetch_32(&pTask->checkpointAlignCnt, 1); @@ -189,7 +189,7 @@ int32_t streamProcessCheckpointBlock(SStreamTask* pTask, SStreamDataBlock* pBloc int32_t taskLevel = pTask->info.taskLevel; if (taskLevel == TASK_LEVEL__SOURCE) { if (pTask->outputInfo.type == TASK_OUTPUT__FIXED_DISPATCH || pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { - qDebug("s-task:%s set childIdx:%d, and add checkpoint block into outputQ", id, pTask->info.selfChildId); + stDebug("s-task:%s set childIdx:%d, and add checkpoint block into outputQ", id, pTask->info.selfChildId); continueDispatchCheckpointBlock(pBlock, pTask); } else { // only one task exists, no need to dispatch downstream info streamProcessCheckpointReadyMsg(pTask); @@ -207,19 +207,19 @@ int32_t streamProcessCheckpointBlock(SStreamTask* pTask, SStreamDataBlock* pBloc int32_t notReady = streamAlignCheckpoint(pTask); int32_t num = taosArrayGetSize(pTask->pUpstreamInfoList); if (notReady > 0) { - qDebug("s-task:%s received checkpoint block, idx:%d, %d upstream tasks not send checkpoint info yet, total:%d", + stDebug("s-task:%s received checkpoint block, idx:%d, %d upstream tasks not send checkpoint info yet, total:%d", id, pTask->info.selfChildId, notReady, num); streamFreeQitem((SStreamQueueItem*)pBlock); return code; } if (taskLevel == TASK_LEVEL__SINK) { - qDebug("s-task:%s process checkpoint block, all %d upstreams sent checkpoint msgs, send ready msg to upstream", + stDebug("s-task:%s process checkpoint block, all %d upstreams sent checkpoint msgs, send ready msg to upstream", id, num); streamFreeQitem((SStreamQueueItem*)pBlock); streamTaskBuildCheckpoint(pTask); } else { - qDebug( + stDebug( "s-task:%s process checkpoint block, all %d upstreams sent checkpoint msgs, dispatch checkpoint msg " "downstream", id, num); @@ -248,12 +248,12 @@ int32_t streamProcessCheckpointReadyMsg(SStreamTask* pTask) { ASSERT(notReady >= 0); if (notReady == 0) { - qDebug("s-task:%s all downstream tasks have completed the checkpoint, start to do checkpoint for current task", + stDebug("s-task:%s all downstream tasks have completed the checkpoint, start to do checkpoint for current task", pTask->id.idStr); appendCheckpointIntoInputQ(pTask, STREAM_INPUT__CHECKPOINT); } else { int32_t total = streamTaskGetNumOfDownstream(pTask); - qDebug("s-task:%s %d/%d downstream tasks are not ready, wait", pTask->id.idStr, notReady, total); + stDebug("s-task:%s %d/%d downstream tasks are not ready, wait", pTask->id.idStr, notReady, total); } return 0; @@ -284,7 +284,7 @@ int32_t streamSaveAllTaskStatus(SStreamMeta* pMeta, int64_t checkpointId) { // save the task streamMetaSaveTask(pMeta, p); streamTaskOpenAllUpstreamInput(p); // open inputQ for all upstream tasks - qDebug("vgId:%d s-task:%s level:%d commit task status after checkpoint completed, checkpointId:%" PRId64 + stDebug("vgId:%d s-task:%s level:%d commit task status after checkpoint completed, checkpointId:%" PRId64 ", Ver(saved):%" PRId64 " currentVer:%" PRId64 ", status to be normal, prev:%s", pMeta->vgId, p->id.idStr, p->info.taskLevel, checkpointId, p->chkInfo.checkpointVer, p->chkInfo.nextProcessVer, streamGetTaskStatusStr(prev)); @@ -292,12 +292,12 @@ int32_t streamSaveAllTaskStatus(SStreamMeta* pMeta, int64_t checkpointId) { if (streamMetaCommit(pMeta) < 0) { taosWUnLockLatch(&pMeta->lock); - qError("vgId:%d failed to commit stream meta after do checkpoint, checkpointId:%" PRId64 ", since %s", pMeta->vgId, + stError("vgId:%d failed to commit stream meta after do checkpoint, checkpointId:%" PRId64 ", since %s", pMeta->vgId, checkpointId, terrstr()); return -1; } else { taosWUnLockLatch(&pMeta->lock); - qInfo("vgId:%d commit stream meta after do checkpoint, checkpointId:%" PRId64 " DONE", pMeta->vgId, checkpointId); + stInfo("vgId:%d commit stream meta after do checkpoint, checkpointId:%" PRId64 " DONE", pMeta->vgId, checkpointId); } return TSDB_CODE_SUCCESS; @@ -314,13 +314,13 @@ int32_t streamTaskBuildCheckpoint(SStreamTask* pTask) { double el = (taosGetTimestampMs() - pTask->chkInfo.startTs) / 1000.0; if (remain == 0) { // all tasks are ready - qDebug("s-task:%s is ready for checkpoint", pTask->id.idStr); + stDebug("s-task:%s is ready for checkpoint", pTask->id.idStr); streamBackendDoCheckpoint(pMeta, pTask->checkpointingId); streamSaveAllTaskStatus(pMeta, pTask->checkpointingId); - qDebug("vgId:%d vnode wide checkpoint completed, save all tasks status, elapsed time:%.2f Sec checkpointId:%" PRId64, pMeta->vgId, + stInfo("vgId:%d vnode wide checkpoint completed, save all tasks status, elapsed time:%.2f Sec checkpointId:%" PRId64, pMeta->vgId, el, pTask->checkpointingId); } else { - qDebug( + stInfo( "vgId:%d vnode wide tasks not reach checkpoint ready status, ready s-task:%s, elapsed time:%.2f Sec not " "ready:%d/%d", pMeta->vgId, pTask->id.idStr, el, remain, pMeta->numOfStreamTasks); @@ -335,7 +335,7 @@ int32_t streamTaskBuildCheckpoint(SStreamTask* pTask) { if (code != TSDB_CODE_SUCCESS) { // todo: let's retry send rsp to upstream/mnode - qError("s-task:%s failed to send checkpoint rsp to upstream, checkpointId:%" PRId64 ", code:%s", pTask->id.idStr, + stError("s-task:%s failed to send checkpoint rsp to upstream, checkpointId:%" PRId64 ", code:%s", pTask->id.idStr, pTask->checkpointingId, tstrerror(code)); } diff --git a/source/libs/stream/src/streamData.c b/source/libs/stream/src/streamData.c index a108667f5d..8cf9ea1bbc 100644 --- a/source/libs/stream/src/streamData.c +++ b/source/libs/stream/src/streamData.c @@ -186,7 +186,7 @@ SStreamQueueItem* streamMergeQueueItem(SStreamQueueItem* dst, SStreamQueueItem* taosFreeQitem(pElem); return (SStreamQueueItem*)pMerged; } else { - qDebug("block type:%s not merged with existed blocks list, type:%d", streamGetBlockTypeStr(pElem->type), dst->type); + stDebug("block type:%s not merged with existed blocks list, type:%d", streamGetBlockTypeStr(pElem->type), dst->type); return NULL; } } diff --git a/source/libs/stream/src/streamDispatch.c b/source/libs/stream/src/streamDispatch.c index 7003c570e9..7906f1d6f4 100644 --- a/source/libs/stream/src/streamDispatch.c +++ b/source/libs/stream/src/streamDispatch.c @@ -231,7 +231,7 @@ int32_t streamBroadcastToChildren(SStreamTask* pTask, const SSDataBlock* pBlock) } buf = NULL; - qDebug("s-task:%s (child %d) send retrieve req to task:0x%x (vgId:%d), reqId:0x%" PRIx64, pTask->id.idStr, + stDebug("s-task:%s (child %d) send retrieve req to task:0x%x (vgId:%d), reqId:0x%" PRIx64, pTask->id.idStr, pTask->info.selfChildId, pEpInfo->taskId, pEpInfo->nodeId, req.reqId); } code = 0; @@ -270,7 +270,7 @@ int32_t streamDispatchCheckMsg(SStreamTask* pTask, const SStreamTaskCheckReq* pR tEncoderClear(&encoder); initRpcMsg(&msg, TDMT_VND_STREAM_TASK_CHECK, buf, tlen + sizeof(SMsgHead)); - qDebug("s-task:%s (level:%d) send check msg to s-task:0x%" PRIx64 ":0x%x (vgId:%d)", pTask->id.idStr, + stDebug("s-task:%s (level:%d) send check msg to s-task:0x%" PRIx64 ":0x%x (vgId:%d)", pTask->id.idStr, pTask->info.taskLevel, pReq->streamId, pReq->downstreamTaskId, nodeId); tmsgSendReq(pEpSet, &msg); @@ -305,7 +305,7 @@ static int32_t doDispatchAllBlocks(SStreamTask* pTask, const SStreamDataBlock* p int32_t vgId = pTask->fixedEpDispatcher.nodeId; SEpSet* pEpSet = &pTask->fixedEpDispatcher.epSet; - qDebug("s-task:%s (child taskId:%d) fix-dispatch %d block(s) to s-task:0x%x (vgId:%d)", pTask->id.idStr, + stDebug("s-task:%s (child taskId:%d) fix-dispatch %d block(s) to s-task:0x%x (vgId:%d)", pTask->id.idStr, pTask->info.selfChildId, numOfBlocks, downstreamTaskId, vgId); code = doSendDispatchMsg(pTask, &req, vgId, pEpSet); @@ -357,13 +357,13 @@ static int32_t doDispatchAllBlocks(SStreamTask* pTask, const SStreamDataBlock* p } } - qDebug("s-task:%s (child taskId:%d) shuffle-dispatch blocks:%d to %d vgroups", pTask->id.idStr, + stDebug("s-task:%s (child taskId:%d) shuffle-dispatch blocks:%d to %d vgroups", pTask->id.idStr, pTask->info.selfChildId, numOfBlocks, vgSz); for (int32_t i = 0; i < vgSz; i++) { if (pReqs[i].blockNum > 0) { SVgroupInfo* pVgInfo = taosArrayGet(vgInfo, i); - qDebug("s-task:%s (child taskId:%d) shuffle-dispatch blocks:%d to vgId:%d", pTask->id.idStr, + stDebug("s-task:%s (child taskId:%d) shuffle-dispatch blocks:%d to vgId:%d", pTask->id.idStr, pTask->info.selfChildId, pReqs[i].blockNum, pVgInfo->vgId); code = doSendDispatchMsg(pTask, &pReqs[i], pVgInfo->vgId, &pVgInfo->epSet); @@ -392,7 +392,7 @@ static void doRetryDispatchData(void* param, void* tmrId) { if (streamTaskShouldStop(&pTask->status)) { int8_t ref = atomic_sub_fetch_8(&pTask->status.timerActive, 1); - qDebug("s-task:%s should stop, abort from timer, ref:%d", pTask->id.idStr, ref); + stDebug("s-task:%s should stop, abort from timer, ref:%d", pTask->id.idStr, ref); return; } @@ -401,7 +401,7 @@ static void doRetryDispatchData(void* param, void* tmrId) { int32_t code = doDispatchAllBlocks(pTask, pTask->msgInfo.pData); if (code != TSDB_CODE_SUCCESS) { if (!streamTaskShouldStop(&pTask->status)) { - qDebug("s-task:%s reset the waitRspCnt to be 0 before launch retry dispatch", pTask->id.idStr); + stDebug("s-task:%s reset the waitRspCnt to be 0 before launch retry dispatch", pTask->id.idStr); atomic_store_32(&pTask->shuffleDispatcher.waitingRspCnt, 0); if (streamTaskShouldPause(&pTask->status)) { streamRetryDispatchStreamBlock(pTask, DISPATCH_RETRY_INTERVAL_MS * 10); @@ -410,16 +410,16 @@ static void doRetryDispatchData(void* param, void* tmrId) { } } else { int32_t ref = atomic_sub_fetch_8(&pTask->status.timerActive, 1); - qDebug("s-task:%s should stop, abort from timer, ref:%d", pTask->id.idStr, ref); + stDebug("s-task:%s should stop, abort from timer, ref:%d", pTask->id.idStr, ref); } } else { int8_t ref = atomic_sub_fetch_8(&pTask->status.timerActive, 1); - qDebug("s-task:%s send success, jump out of timer, ref:%d", pTask->id.idStr, ref); + stDebug("s-task:%s send success, jump out of timer, ref:%d", pTask->id.idStr, ref); } } void streamRetryDispatchStreamBlock(SStreamTask* pTask, int64_t waitDuration) { - qWarn("s-task:%s dispatch data in %" PRId64 "ms, in timer", pTask->id.idStr, waitDuration); + stWarn("s-task:%s dispatch data in %" PRId64 "ms, in timer", pTask->id.idStr, waitDuration); if (pTask->launchTaskTimer != NULL) { taosTmrReset(doRetryDispatchData, waitDuration, pTask, streamEnv.timer, &pTask->launchTaskTimer); } else { @@ -501,24 +501,24 @@ int32_t streamDispatchStreamBlock(SStreamTask* pTask) { int32_t numOfElems = streamQueueGetNumOfItems(pTask->outputInfo.queue); if (numOfElems > 0) { double size = SIZE_IN_MB(taosQueueMemorySize(pTask->outputInfo.queue->pQueue)); - qDebug("s-task:%s start to dispatch intermediate block to downstream, elem in outputQ:%d, size:%.2fMiB", id, numOfElems, size); + stDebug("s-task:%s start to dispatch intermediate block to downstream, elem in outputQ:%d, size:%.2fMiB", id, numOfElems, size); } // to make sure only one dispatch is running int8_t old = atomic_val_compare_exchange_8(&pTask->outputInfo.status, TASK_OUTPUT_STATUS__NORMAL, TASK_OUTPUT_STATUS__WAIT); if (old != TASK_OUTPUT_STATUS__NORMAL) { - qDebug("s-task:%s wait for dispatch rsp, not dispatch now, output status:%d", id, old); + stDebug("s-task:%s wait for dispatch rsp, not dispatch now, output status:%d", id, old); return 0; } ASSERT(pTask->msgInfo.pData == NULL); - qDebug("s-task:%s start to dispatch msg, set output status:%d", id, pTask->outputInfo.status); + stDebug("s-task:%s start to dispatch msg, set output status:%d", id, pTask->outputInfo.status); SStreamDataBlock* pBlock = streamQueueNextItem(pTask->outputInfo.queue); if (pBlock == NULL) { atomic_store_8(&pTask->outputInfo.status, TASK_OUTPUT_STATUS__NORMAL); - qDebug("s-task:%s not dispatch since no elems in outputQ, output status:%d", id, pTask->outputInfo.status); + stDebug("s-task:%s not dispatch since no elems in outputQ, output status:%d", id, pTask->outputInfo.status); return 0; } @@ -534,7 +534,7 @@ int32_t streamDispatchStreamBlock(SStreamTask* pTask) { break; } - qDebug("s-task:%s failed to dispatch msg to downstream, code:%s, output status:%d, retry cnt:%d", id, + stDebug("s-task:%s failed to dispatch msg to downstream, code:%s, output status:%d, retry cnt:%d", id, tstrerror(terrno), pTask->outputInfo.status, retryCount); // todo deal with only partially success dispatch case @@ -548,7 +548,7 @@ int32_t streamDispatchStreamBlock(SStreamTask* pTask) { if (++retryCount > MAX_CONTINUE_RETRY_COUNT) { // add to timer to retry int8_t ref = atomic_add_fetch_8(&pTask->status.timerActive, 1); - qDebug("s-task:%s failed to dispatch msg to downstream for %d times, code:%s, add timer to retry in %dms, ref:%d", + stDebug("s-task:%s failed to dispatch msg to downstream for %d times, code:%s, add timer to retry in %dms, ref:%d", pTask->id.idStr, retryCount, tstrerror(terrno), DISPATCH_RETRY_INTERVAL_MS, ref); streamRetryDispatchStreamBlock(pTask, DISPATCH_RETRY_INTERVAL_MS); break; @@ -577,7 +577,7 @@ int32_t streamDispatchScanHistoryFinishMsg(SStreamTask* pTask) { int32_t numOfVgs = taosArrayGetSize(vgInfo); pTask->notReadyTasks = numOfVgs; - qDebug("s-task:%s send scan-history data complete msg to downstream (shuffle-dispatch) %d tasks, status:%s", pTask->id.idStr, + stDebug("s-task:%s send scan-history data complete msg to downstream (shuffle-dispatch) %d tasks, status:%s", pTask->id.idStr, numOfVgs, streamGetTaskStatusStr(pTask->status.taskStatus)); for (int32_t i = 0; i < numOfVgs; i++) { SVgroupInfo* pVgInfo = taosArrayGet(vgInfo, i); @@ -585,7 +585,7 @@ int32_t streamDispatchScanHistoryFinishMsg(SStreamTask* pTask) { doDispatchScanHistoryFinishMsg(pTask, &req, pVgInfo->vgId, &pVgInfo->epSet); } } else { - qDebug("s-task:%s no downstream tasks, invoke scan-history finish rsp directly", pTask->id.idStr); + stDebug("s-task:%s no downstream tasks, invoke scan-history finish rsp directly", pTask->id.idStr); streamProcessScanHistoryFinishRsp(pTask); } @@ -601,12 +601,12 @@ int32_t streamTaskSendCheckpointReadyMsg(SStreamTask* pTask) { SStreamChkptReadyInfo* pInfo = taosArrayGet(pTask->pReadyMsgList, i); tmsgSendReq(&pInfo->upstreamNodeEpset, &pInfo->msg); - qDebug("s-task:%s level:%d checkpoint ready msg sent to upstream:0x%x", pTask->id.idStr, pTask->info.taskLevel, + stDebug("s-task:%s level:%d checkpoint ready msg sent to upstream:0x%x", pTask->id.idStr, pTask->info.taskLevel, pInfo->upStreamTaskId); } taosArrayClear(pTask->pReadyMsgList); - qDebug("s-task:%s level:%d checkpoint ready msg sent to all %d upstreams", pTask->id.idStr, pTask->info.taskLevel, num); + stDebug("s-task:%s level:%d checkpoint ready msg sent to all %d upstreams", pTask->id.idStr, pTask->info.taskLevel, num); return TSDB_CODE_SUCCESS; } @@ -619,7 +619,7 @@ int32_t streamTaskSendCheckpointSourceRsp(SStreamTask* pTask) { tmsgSendRsp(&pInfo->msg); taosArrayClear(pTask->pReadyMsgList); - qDebug("s-task:%s level:%d source checkpoint completed msg sent to mnode", pTask->id.idStr, pTask->info.taskLevel); + stDebug("s-task:%s level:%d source checkpoint completed msg sent to mnode", pTask->id.idStr, pTask->info.taskLevel); return TSDB_CODE_SUCCESS; } @@ -691,7 +691,7 @@ int32_t doDispatchScanHistoryFinishMsg(SStreamTask* pTask, const SStreamScanHist tmsgSendReq(pEpSet, &msg); const char* pStatus = streamGetTaskStatusStr(pTask->status.taskStatus); - qDebug("s-task:%s status:%s dispatch scan-history finish msg to taskId:0x%x (vgId:%d)", pTask->id.idStr, pStatus, + stDebug("s-task:%s status:%s dispatch scan-history finish msg to taskId:0x%x (vgId:%d)", pTask->id.idStr, pStatus, pReq->downstreamTaskId, vgId); return 0; } @@ -725,7 +725,7 @@ int32_t doSendDispatchMsg(SStreamTask* pTask, const SStreamDispatchReq* pReq, in tEncoderClear(&encoder); initRpcMsg(&msg, pTask->msgInfo.msgType, buf, tlen + sizeof(SMsgHead)); - qDebug("s-task:%s dispatch msg to taskId:0x%x vgId:%d data msg", pTask->id.idStr, pReq->taskId, vgId); + stDebug("s-task:%s dispatch msg to taskId:0x%x vgId:%d data msg", pTask->id.idStr, pReq->taskId, vgId); return tmsgSendReq(pEpSet, &msg); @@ -786,7 +786,7 @@ int32_t streamAddCheckpointSourceRspMsg(SStreamCheckpointSourceReq* pReq, SRpcHa } taosArrayPush(pTask->pReadyMsgList, &info); - qDebug("s-task:%s add checkpoint source rsp msg, total:%d", pTask->id.idStr, (int32_t)taosArrayGetSize(pTask->pReadyMsgList)); + stDebug("s-task:%s add checkpoint source rsp msg, total:%d", pTask->id.idStr, (int32_t)taosArrayGetSize(pTask->pReadyMsgList)); return TSDB_CODE_SUCCESS; } @@ -836,7 +836,7 @@ int32_t streamAddCheckpointReadyMsg(SStreamTask* pTask, int32_t upstreamTaskId, initRpcMsg(&info.msg, TDMT_STREAM_TASK_CHECKPOINT_READY, buf, tlen + sizeof(SMsgHead)); info.msg.info.noResp = 1; // refactor later. - qDebug("s-task:%s (level:%d) prepare checkpoint ready msg to upstream s-task:0x%" PRIx64 ":0x%x (vgId:%d) idx:%d", + stDebug("s-task:%s (level:%d) prepare checkpoint ready msg to upstream s-task:0x%" PRIx64 ":0x%x (vgId:%d) idx:%d", pTask->id.idStr, pTask->info.taskLevel, req.streamId, req.upstreamTaskId, req.downstreamNodeId, index); if (pTask->pReadyMsgList == NULL) { @@ -924,7 +924,7 @@ int32_t streamAddEndScanHistoryMsg(SStreamTask* pTask, SRpcHandleInfo* pRpcInfo, taosThreadMutexUnlock(&pTask->lock); int32_t num = taosArrayGetSize(pTask->pRspMsgList); - qDebug("s-task:%s add scan history finish rsp msg for task:0x%x, total:%d", pTask->id.idStr, pReq->upstreamTaskId, + stDebug("s-task:%s add scan history finish rsp msg for task:0x%x, total:%d", pTask->id.idStr, pReq->upstreamTaskId, num); return TSDB_CODE_SUCCESS; } @@ -937,12 +937,12 @@ int32_t streamNotifyUpstreamContinue(SStreamTask* pTask) { SStreamContinueExecInfo* pInfo = taosArrayGet(pTask->pRspMsgList, i); tmsgSendRsp(&pInfo->msg); - qDebug("s-task:%s level:%d notify upstream:0x%x to continue process data in WAL", pTask->id.idStr, pTask->info.taskLevel, + stDebug("s-task:%s level:%d notify upstream:0x%x to continue process data in WAL", pTask->id.idStr, pTask->info.taskLevel, pInfo->taskId); } taosArrayClear(pTask->pRspMsgList); - qDebug("s-task:%s level:%d checkpoint ready msg sent to all %d upstreams", pTask->id.idStr, pTask->info.taskLevel, + stDebug("s-task:%s level:%d checkpoint ready msg sent to all %d upstreams", pTask->id.idStr, pTask->info.taskLevel, num); return 0; } @@ -957,9 +957,9 @@ int32_t streamProcessDispatchRsp(SStreamTask* pTask, SStreamDispatchRsp* pRsp, i // happened too fast. // todo handle the shuffle dispatch failure if (code == TSDB_CODE_STREAM_TASK_NOT_EXIST) { // destination task does not exist, not retry anymore - qWarn("s-task:%s failed to dispatch msg to task:0x%x, no retry, since it is destroyed already", id, pRsp->downstreamTaskId); + stWarn("s-task:%s failed to dispatch msg to task:0x%x, no retry, since it is destroyed already", id, pRsp->downstreamTaskId); } else { - qError("s-task:%s failed to dispatch msg to task:0x%x, code:%s, retry cnt:%d", id, pRsp->downstreamTaskId, + stError("s-task:%s failed to dispatch msg to task:0x%x, code:%s, retry cnt:%d", id, pRsp->downstreamTaskId, tstrerror(code), ++pTask->msgInfo.retryCount); int32_t ret = doDispatchAllBlocks(pTask, pTask->msgInfo.pData); if (ret != TSDB_CODE_SUCCESS) { @@ -969,13 +969,13 @@ int32_t streamProcessDispatchRsp(SStreamTask* pTask, SStreamDispatchRsp* pRsp, i return TSDB_CODE_SUCCESS; } - qDebug("s-task:%s recv dispatch rsp from 0x%x, downstream task input status:%d code:%d", id, pRsp->downstreamTaskId, + stDebug("s-task:%s recv dispatch rsp from 0x%x, downstream task input status:%d code:%d", id, pRsp->downstreamTaskId, pRsp->inputStatus, code); // there are other dispatch message not response yet if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { int32_t leftRsp = atomic_sub_fetch_32(&pTask->shuffleDispatcher.waitingRspCnt, 1); - qDebug("s-task:%s is shuffle, left waiting rsp %d", id, leftRsp); + stDebug("s-task:%s is shuffle, left waiting rsp %d", id, leftRsp); if (leftRsp > 0) { return 0; } @@ -984,7 +984,7 @@ int32_t streamProcessDispatchRsp(SStreamTask* pTask, SStreamDispatchRsp* pRsp, i // transtate msg has been sent to downstream successfully. let's transfer the fill-history task state SStreamDataBlock* p = pTask->msgInfo.pData; if (p->type == STREAM_INPUT__TRANS_STATE) { - qDebug("s-task:%s dispatch transtate msg to downstream successfully, start to transfer state", id); + stDebug("s-task:%s dispatch transtate msg to downstream successfully, start to transfer state", id); ASSERT(pTask->info.fillHistory == 1); code = streamTransferStateToStreamTask(pTask); if (code != TSDB_CODE_SUCCESS) { // todo: do nothing if error happens @@ -998,7 +998,7 @@ int32_t streamProcessDispatchRsp(SStreamTask* pTask, SStreamDispatchRsp* pRsp, i pTask->msgInfo.retryCount = 0; ASSERT(pTask->outputInfo.status == TASK_OUTPUT_STATUS__WAIT); - qDebug("s-task:%s output status is set to:%d", id, pTask->outputInfo.status); + stDebug("s-task:%s output status is set to:%d", id, pTask->outputInfo.status); // the input queue of the (down stream) task that receive the output data is full, // so the TASK_INPUT_STATUS_BLOCKED is rsp @@ -1012,7 +1012,7 @@ int32_t streamProcessDispatchRsp(SStreamTask* pTask, SStreamDispatchRsp* pRsp, i } int8_t ref = atomic_add_fetch_8(&pTask->status.timerActive, 1); - qError("s-task:%s inputQ of downstream task:0x%x is full, time:%" PRId64 + stError("s-task:%s inputQ of downstream task:0x%x is full, time:%" PRId64 " wait for %dms and retry dispatch data, total wait:%.2fSec ref:%d", id, pRsp->downstreamTaskId, pTask->msgInfo.blockingTs, DISPATCH_RETRY_INTERVAL_MS, el, ref); streamRetryDispatchStreamBlock(pTask, DISPATCH_RETRY_INTERVAL_MS); @@ -1023,7 +1023,7 @@ int32_t streamProcessDispatchRsp(SStreamTask* pTask, SStreamDispatchRsp* pRsp, i if (pTask->msgInfo.blockingTs != 0) { int64_t el = taosGetTimestampMs() - pTask->msgInfo.blockingTs; - qDebug("s-task:%s downstream task:0x%x resume to normal from inputQ blocking, blocking time:%" PRId64 "ms", id, + stDebug("s-task:%s downstream task:0x%x resume to normal from inputQ blocking, blocking time:%" PRId64 "ms", id, pRsp->downstreamTaskId, el); pTask->msgInfo.blockingTs = 0; diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index d89817d236..3109bf5967 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -66,12 +66,12 @@ static int32_t doDumpResult(SStreamTask* pTask, SStreamQueueItem* pItem, SArray* SStreamDataBlock* pStreamBlocks = createStreamBlockFromResults(pItem, pTask, size, pRes); if (pStreamBlocks == NULL) { - qError("s-task:%s failed to create result stream data block, code:%s", pTask->id.idStr, tstrerror(terrno)); + stError("s-task:%s failed to create result stream data block, code:%s", pTask->id.idStr, tstrerror(terrno)); taosArrayDestroyEx(pRes, (FDelete)blockDataFreeRes); return TSDB_CODE_OUT_OF_MEMORY; } - qDebug("s-task:%s dump stream result data blocks, num:%d, size:%.2fMiB", pTask->id.idStr, numOfBlocks, + stDebug("s-task:%s dump stream result data blocks, num:%d, size:%.2fMiB", pTask->id.idStr, numOfBlocks, SIZE_IN_MB(size)); int32_t code = doOutputResultBlockImpl(pTask, pStreamBlocks); @@ -109,7 +109,7 @@ static int32_t streamTaskExecImpl(SStreamTask* pTask, SStreamQueueItem* pItem, i } if (pTask->inputInfo.status == TASK_INPUT_STATUS__BLOCKED) { - qWarn("s-task:%s downstream task inputQ blocked, idle for 1sec and retry exec task", pTask->id.idStr); + stWarn("s-task:%s downstream task inputQ blocked, idle for 1sec and retry exec task", pTask->id.idStr); taosMsleep(1000); continue; } @@ -121,7 +121,7 @@ static int32_t streamTaskExecImpl(SStreamTask* pTask, SStreamQueueItem* pItem, i resetTaskInfo(pExecutor); } - qError("unexpected stream execution, s-task:%s since %s", pTask->id.idStr, tstrerror(code)); + stError("unexpected stream execution, s-task:%s since %s", pTask->id.idStr, tstrerror(code)); continue; } @@ -137,7 +137,7 @@ static int32_t streamTaskExecImpl(SStreamTask* pTask, SStreamQueueItem* pItem, i taosArrayPush(pRes, &block); numOfBlocks += 1; - qDebug("s-task:%s(child %d) retrieve process completed, reqId:0x%" PRIx64 " dump results", pTask->id.idStr, + stDebug("s-task:%s(child %d) retrieve process completed, reqId:0x%" PRIx64 " dump results", pTask->id.idStr, pTask->info.selfChildId, pRetrieveBlock->reqId); } @@ -162,7 +162,7 @@ static int32_t streamTaskExecImpl(SStreamTask* pTask, SStreamQueueItem* pItem, i taosArrayPush(pRes, &block); - qDebug("s-task:%s (child %d) executed and get %d result blocks, size:%.2fMiB", pTask->id.idStr, + stDebug("s-task:%s (child %d) executed and get %d result blocks, size:%.2fMiB", pTask->id.idStr, pTask->info.selfChildId, numOfBlocks, SIZE_IN_MB(size)); // current output should be dispatched to down stream nodes @@ -203,7 +203,7 @@ int32_t streamScanHistoryData(SStreamTask* pTask) { while (!finished) { if (streamTaskShouldPause(&pTask->status)) { double el = (taosGetTimestampMs() - pTask->taskExecInfo.step1Start) / 1000.0; - qDebug("s-task:%s paused from the scan-history task, elapsed time:%.2fsec", pTask->id.idStr, el); + stDebug("s-task:%s paused from the scan-history task, elapsed time:%.2fsec", pTask->id.idStr, el); break; } @@ -221,7 +221,7 @@ int32_t streamScanHistoryData(SStreamTask* pTask) { } if (pTask->inputInfo.status == TASK_INPUT_STATUS__BLOCKED) { - qDebug("s-task:%s inputQ is blocked, wait for 10sec and retry", pTask->id.idStr); + stDebug("s-task:%s inputQ is blocked, wait for 10sec and retry", pTask->id.idStr); taosMsleep(10000); continue; } @@ -230,7 +230,7 @@ int32_t streamScanHistoryData(SStreamTask* pTask) { uint64_t ts = 0; code = qExecTask(exec, &output, &ts); if (code != TSDB_CODE_TSC_QUERY_KILLED && code != TSDB_CODE_SUCCESS) { - qError("%s scan-history data error occurred code:%s, continue scan", pTask->id.idStr, tstrerror(code)); + stError("%s scan-history data error occurred code:%s, continue scan", pTask->id.idStr, tstrerror(code)); continue; } @@ -248,7 +248,7 @@ int32_t streamScanHistoryData(SStreamTask* pTask) { size += blockDataGetSize(output) + sizeof(SSDataBlock) + sizeof(SColumnInfoData) * blockDataGetNumOfCols(&block); if ((++numOfBlocks) >= outputBatchSize || size >= STREAM_RESULT_DUMP_SIZE_THRESHOLD) { - qDebug("s-task:%s scan exec numOfBlocks:%d, output num-limit:%d, size-limit:%d reached", pTask->id.idStr, numOfBlocks, + stDebug("s-task:%s scan exec numOfBlocks:%d, output num-limit:%d, size-limit:%d reached", pTask->id.idStr, numOfBlocks, outputBatchSize, STREAM_RESULT_DUMP_SIZE_THRESHOLD); break; } @@ -258,7 +258,6 @@ int32_t streamScanHistoryData(SStreamTask* pTask) { SStreamDataBlock* pStreamBlocks = createStreamBlockFromResults(NULL, pTask, size, pRes); code = doOutputResultBlockImpl(pTask, pStreamBlocks); if (code != TSDB_CODE_SUCCESS) { - destroyStreamDataBlock(pStreamBlocks); return code; } @@ -277,14 +276,14 @@ static void waitForTaskIdle(SStreamTask* pTask, SStreamTask* pStreamTask) { int64_t st = taosGetTimestampMs(); while (!streamTaskIsIdle(pStreamTask)) { - qDebug("s-task:%s level:%d wait for stream task:%s to be idle, check again in 100ms", id, pTask->info.taskLevel, + stDebug("s-task:%s level:%d wait for stream task:%s to be idle, check again in 100ms", id, pTask->info.taskLevel, pStreamTask->id.idStr); taosMsleep(100); } double el = (taosGetTimestampMs() - st) / 1000.0; if (el > 0) { - qDebug("s-task:%s wait for stream task:%s for %.2fs to be idle", id, pStreamTask->id.idStr, el); + stDebug("s-task:%s wait for stream task:%s for %.2fs to be idle", id, pStreamTask->id.idStr, el); } } @@ -293,7 +292,7 @@ int32_t streamDoTransferStateToStreamTask(SStreamTask* pTask) { SStreamTask* pStreamTask = streamMetaAcquireTask(pMeta, pTask->streamTaskId.streamId, pTask->streamTaskId.taskId); if (pStreamTask == NULL) { - qError( + stError( "s-task:%s failed to find related stream task:0x%x, it may have been destroyed or closed, destroy the related " "fill-history task", pTask->id.idStr, (int32_t) pTask->streamTaskId.taskId); @@ -309,7 +308,7 @@ int32_t streamDoTransferStateToStreamTask(SStreamTask* pTask) { taosWUnLockLatch(&pMeta->lock); return TSDB_CODE_STREAM_TASK_NOT_EXIST; } else { - qDebug("s-task:%s fill-history task end, update related stream task:%s info, transfer exec state", pTask->id.idStr, + stDebug("s-task:%s fill-history task end, update related stream task:%s info, transfer exec state", pTask->id.idStr, pStreamTask->id.idStr); } @@ -326,7 +325,7 @@ int32_t streamDoTransferStateToStreamTask(SStreamTask* pTask) { } else { ASSERT(status == TASK_STATUS__NORMAL); pStreamTask->status.taskStatus = TASK_STATUS__HALT; - qDebug("s-task:%s halt by related fill-history task:%s", pStreamTask->id.idStr, pTask->id.idStr); + stDebug("s-task:%s halt by related fill-history task:%s", pStreamTask->id.idStr, pTask->id.idStr); } // wait for the stream task to handle all in the inputQ, and to be idle @@ -338,12 +337,12 @@ int32_t streamDoTransferStateToStreamTask(SStreamTask* pTask) { // When a task is idle with halt status, all data in inputQ are consumed. if (pStreamTask->info.taskLevel == TASK_LEVEL__SOURCE) { // update the scan data range for source task. - qDebug("s-task:%s level:%d stream task window %" PRId64 " - %" PRId64 " update to %" PRId64 " - %" PRId64 + stDebug("s-task:%s level:%d stream task window %" PRId64 " - %" PRId64 " update to %" PRId64 " - %" PRId64 ", status:%s, sched-status:%d", pStreamTask->id.idStr, TASK_LEVEL__SOURCE, pTimeWindow->skey, pTimeWindow->ekey, INT64_MIN, pTimeWindow->ekey, streamGetTaskStatusStr(TASK_STATUS__NORMAL), pStreamTask->status.schedStatus); } else { - qDebug("s-task:%s no need to update time window for non-source task", pStreamTask->id.idStr); + stDebug("s-task:%s no need to update time window for non-source task", pStreamTask->id.idStr); } // 1. expand the query time window for stream task of WAL scanner @@ -358,7 +357,7 @@ int32_t streamDoTransferStateToStreamTask(SStreamTask* pTask) { // pause, since the pause allowed attribute is not set yet. streamTaskResumeFromHalt(pStreamTask); - qDebug("s-task:%s fill-history task set status to be dropping, save the state into disk", pTask->id.idStr); + stDebug("s-task:%s fill-history task set status to be dropping, save the state into disk", pTask->id.idStr); // 4. free it and remove fill-history task from disk meta-store streamBuildAndSendDropTaskMsg(pTask->pMsgCb, pMeta->vgId, &pTask->id); @@ -386,7 +385,7 @@ int32_t streamDoTransferStateToStreamTask(SStreamTask* pTask) { pItem->type = STREAM_INPUT__REF_DATA_BLOCK; pItem->pBlock = pDelBlock; int32_t code = streamTaskPutDataIntoInputQ(pStreamTask, (SStreamQueueItem*)pItem); - qDebug("s-task:%s append dummy delete block,res:%d", pStreamTask->id.idStr, code); + stDebug("s-task:%s append dummy delete block,res:%d", pStreamTask->id.idStr, code); } streamSchedExec(pStreamTask); @@ -425,7 +424,7 @@ static void doSetStreamInputBlock(SStreamTask* pTask, const void* pInput, int64_ ASSERT(pTask->info.taskLevel == TASK_LEVEL__SOURCE); const SStreamDataSubmit* pSubmit = (const SStreamDataSubmit*)pInput; qSetMultiStreamInput(pExecutor, &pSubmit->submit, 1, STREAM_INPUT__DATA_SUBMIT); - qDebug("s-task:%s set submit blocks as source block completed, %p %p len:%d ver:%" PRId64, id, pSubmit, + stDebug("s-task:%s set submit blocks as source block completed, %p %p len:%d ver:%" PRId64, id, pSubmit, pSubmit->submit.msgStr, pSubmit->submit.msgLen, pSubmit->submit.ver); ASSERT((*pVer) <= pSubmit->submit.ver); (*pVer) = pSubmit->submit.ver; @@ -435,7 +434,7 @@ static void doSetStreamInputBlock(SStreamTask* pTask, const void* pInput, int64_ SArray* pBlockList = pBlock->blocks; int32_t numOfBlocks = taosArrayGetSize(pBlockList); - qDebug("s-task:%s set sdata blocks as input num:%d, ver:%" PRId64, id, numOfBlocks, pBlock->sourceVer); + stDebug("s-task:%s set sdata blocks as input num:%d, ver:%" PRId64, id, numOfBlocks, pBlock->sourceVer); qSetMultiStreamInput(pExecutor, pBlockList->pData, numOfBlocks, STREAM_INPUT__DATA_BLOCK); } else if (pItem->type == STREAM_INPUT__MERGED_SUBMIT) { @@ -443,7 +442,7 @@ static void doSetStreamInputBlock(SStreamTask* pTask, const void* pInput, int64_ SArray* pBlockList = pMerged->submits; int32_t numOfBlocks = taosArrayGetSize(pBlockList); - qDebug("s-task:%s %p set (merged) submit blocks as a batch, numOfBlocks:%d, ver:%" PRId64, id, pTask, numOfBlocks, + stDebug("s-task:%s %p set (merged) submit blocks as a batch, numOfBlocks:%d, ver:%" PRId64, id, pTask, numOfBlocks, pMerged->ver); qSetMultiStreamInput(pExecutor, pBlockList->pData, numOfBlocks, STREAM_INPUT__MERGED_SUBMIT); ASSERT((*pVer) <= pMerged->ver); @@ -471,7 +470,7 @@ int32_t streamProcessTranstateBlock(SStreamTask* pTask, SStreamDataBlock* pBlock int32_t remain = streamAlignTransferState(pTask); if (remain > 0) { streamFreeQitem((SStreamQueueItem*)pBlock); - qDebug("s-task:%s receive upstream transfer state msg, remain:%d", id, remain); + stDebug("s-task:%s receive upstream transfer state msg, remain:%d", id, remain); return 0; } } @@ -482,9 +481,9 @@ int32_t streamProcessTranstateBlock(SStreamTask* pTask, SStreamDataBlock* pBlock // transfer the ownership of executor state if (type == TASK_OUTPUT__FIXED_DISPATCH || type == TASK_OUTPUT__SHUFFLE_DISPATCH) { if (level == TASK_LEVEL__SOURCE) { - qDebug("s-task:%s add transfer-state block into outputQ", id); + stDebug("s-task:%s add transfer-state block into outputQ", id); } else { - qDebug("s-task:%s all upstream tasks send transfer-state block, add transfer-state block into outputQ", id); + stDebug("s-task:%s all upstream tasks send transfer-state block, add transfer-state block into outputQ", id); ASSERT(pTask->streamTaskId.taskId != 0 && pTask->info.fillHistory == 1); } @@ -502,7 +501,7 @@ int32_t streamProcessTranstateBlock(SStreamTask* pTask, SStreamDataBlock* pBlock } } else { // non-dispatch task, do task state transfer directly streamFreeQitem((SStreamQueueItem*)pBlock); - qDebug("s-task:%s non-dispatch task, start to transfer state directly", id); + stDebug("s-task:%s non-dispatch task, start to transfer state directly", id); ASSERT(pTask->info.fillHistory == 1); code = streamTransferStateToStreamTask(pTask); @@ -522,13 +521,13 @@ int32_t streamExecForAll(SStreamTask* pTask) { const char* id = pTask->id.idStr; // merge multiple input data if possible in the input queue. - qDebug("s-task:%s start to extract data block from inputQ", id); + stDebug("s-task:%s start to extract data block from inputQ", id); while (1) { int32_t numOfBlocks = 0; SStreamQueueItem* pInput = NULL; if (streamTaskShouldStop(&pTask->status)) { - qDebug("s-task:%s stream task is stopped", id); + stDebug("s-task:%s stream task is stopped", id); break; } @@ -555,7 +554,7 @@ int32_t streamExecForAll(SStreamTask* pTask) { ASSERT(type == STREAM_INPUT__DATA_BLOCK || type == STREAM_INPUT__CHECKPOINT); if (type == STREAM_INPUT__DATA_BLOCK) { - qDebug("s-task:%s sink task start to sink %d blocks", id, numOfBlocks); + stDebug("s-task:%s sink task start to sink %d blocks", id, numOfBlocks); doOutputResultBlockImpl(pTask, (SStreamDataBlock*)pInput); continue; } @@ -564,7 +563,7 @@ int32_t streamExecForAll(SStreamTask* pTask) { int64_t st = taosGetTimestampMs(); const SStreamQueueItem* pItem = pInput; - qDebug("s-task:%s start to process batch of blocks, num:%d, type:%d", id, numOfBlocks, pItem->type); + stDebug("s-task:%s start to process batch of blocks, num:%d, type:%d", id, numOfBlocks, pItem->type); int64_t ver = pTask->chkInfo.checkpointVer; doSetStreamInputBlock(pTask, pInput, &ver, id); @@ -574,14 +573,14 @@ int32_t streamExecForAll(SStreamTask* pTask) { streamTaskExecImpl(pTask, pInput, &resSize, &totalBlocks); double el = (taosGetTimestampMs() - st) / 1000.0; - qDebug("s-task:%s batch of input blocks exec end, elapsed time:%.2fs, result size:%.2fMiB, numOfBlocks:%d", id, el, + stDebug("s-task:%s batch of input blocks exec end, elapsed time:%.2fs, result size:%.2fMiB, numOfBlocks:%d", id, el, SIZE_IN_MB(resSize), totalBlocks); // update the currentVer if processing the submit blocks. ASSERT(pTask->chkInfo.checkpointVer <= pTask->chkInfo.nextProcessVer && ver >= pTask->chkInfo.checkpointVer); if (ver != pTask->chkInfo.checkpointVer) { - qDebug("s-task:%s update checkpointVer(unsaved) from %" PRId64 " to %" PRId64 " , currentVer:%" PRId64, + stDebug("s-task:%s update checkpointVer(unsaved) from %" PRId64 " to %" PRId64 " , currentVer:%" PRId64, pTask->id.idStr, pTask->chkInfo.checkpointVer, ver, pTask->chkInfo.nextProcessVer); pTask->chkInfo.checkpointVer = ver; } @@ -591,7 +590,7 @@ int32_t streamExecForAll(SStreamTask* pTask) { // todo other thread may change the status // do nothing after sync executor state to storage backend, untill the vnode-level checkpoint is completed. if (type == STREAM_INPUT__CHECKPOINT) { - qDebug("s-task:%s checkpoint block received, set the status:%s", pTask->id.idStr, + stDebug("s-task:%s checkpoint block received, set the status:%s", pTask->id.idStr, streamGetTaskStatusStr(pTask->status.taskStatus)); streamTaskBuildCheckpoint(pTask); return 0; @@ -627,14 +626,14 @@ int32_t streamTryExec(SStreamTask* pTask) { atomic_store_8(&pTask->status.schedStatus, TASK_SCHED_STATUS__INACTIVE); taosThreadMutexUnlock(&pTask->lock); - qDebug("s-task:%s exec completed, status:%s, sched-status:%d", id, + stDebug("s-task:%s exec completed, status:%s, sched-status:%d", id, streamGetTaskStatusStr(pTask->status.taskStatus), pTask->status.schedStatus); return 0; } taosThreadMutexUnlock(&pTask->lock); } } else { - qDebug("s-task:%s already started to exec by other thread, status:%s, sched-status:%d", id, + stDebug("s-task:%s already started to exec by other thread, status:%s, sched-status:%d", id, streamGetTaskStatusStr(pTask->status.taskStatus), pTask->status.schedStatus); } @@ -642,7 +641,7 @@ int32_t streamTryExec(SStreamTask* pTask) { } int32_t streamTaskReleaseState(SStreamTask* pTask) { - qDebug("s-task:%s release exec state", pTask->id.idStr); + stDebug("s-task:%s release exec state", pTask->id.idStr); void* pExecutor = pTask->exec.pExecutor; if (pExecutor != NULL) { int32_t code = qStreamOperatorReleaseState(pExecutor); @@ -653,7 +652,7 @@ int32_t streamTaskReleaseState(SStreamTask* pTask) { } int32_t streamTaskReloadState(SStreamTask* pTask) { - qDebug("s-task:%s reload exec state", pTask->id.idStr); + stDebug("s-task:%s reload exec state", pTask->id.idStr); void* pExecutor = pTask->exec.pExecutor; if (pExecutor != NULL) { int32_t code = qStreamOperatorReloadState(pExecutor); @@ -667,7 +666,7 @@ int32_t streamAlignTransferState(SStreamTask* pTask) { int32_t numOfUpstream = taosArrayGetSize(pTask->pUpstreamInfoList); int32_t old = atomic_val_compare_exchange_32(&pTask->transferStateAlignCnt, 0, numOfUpstream); if (old == 0) { - qDebug("s-task:%s set the transfer state aligncnt %d", pTask->id.idStr, numOfUpstream); + stDebug("s-task:%s set the transfer state aligncnt %d", pTask->id.idStr, numOfUpstream); } return atomic_sub_fetch_32(&pTask->transferStateAlignCnt, 1); diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 5f60632a7c..7cb3fad292 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -115,7 +115,7 @@ SStreamMeta* streamMetaOpen(const char* path, void* ahandle, FTaskExpand expandF SStreamMeta* pMeta = taosMemoryCalloc(1, sizeof(SStreamMeta)); if (pMeta == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; - qError("vgId:%d failed to prepare stream meta, alloc size:%" PRIzu ", out of memory", vgId, sizeof(SStreamMeta)); + stError("vgId:%d failed to prepare stream meta, alloc size:%" PRIzu ", out of memory", vgId, sizeof(SStreamMeta)); return NULL; } @@ -194,8 +194,8 @@ SStreamMeta* streamMetaOpen(const char* path, void* ahandle, FTaskExpand expandF taosMsleep(2 * 1000); pMeta->streamBackend = streamBackendInit(pMeta->path, pMeta->chkpId); if (pMeta->streamBackend == NULL) { - qError("vgId:%d failed to init stream backend", pMeta->vgId); - qInfo("vgId:%d retry to init stream backend", pMeta->vgId); + stError("vgId:%d failed to init stream backend", pMeta->vgId); + stInfo("vgId:%d retry to init stream backend", pMeta->vgId); } } pMeta->streamBackendRid = taosAddRef(streamBackendId, pMeta->streamBackend); @@ -207,7 +207,7 @@ SStreamMeta* streamMetaOpen(const char* path, void* ahandle, FTaskExpand expandF pMeta->numOfPausedTasks = 0; pMeta->numOfStreamTasks = 0; - qInfo("vgId:%d open stream meta successfully, latest checkpoint:%" PRId64 ", stage:%" PRId64, vgId, pMeta->chkpId, + stInfo("vgId:%d open stream meta successfully, latest checkpoint:%" PRId64 ", stage:%" PRId64, vgId, pMeta->chkpId, stage); return pMeta; @@ -223,7 +223,7 @@ _err: taosMemoryFree(pMeta); - qError("failed to open stream meta"); + stError("failed to open stream meta"); return NULL; } @@ -246,7 +246,7 @@ int32_t streamMetaReopen(SStreamMeta* pMeta) { code = taosRenameFile(newPath, defaultPath); if (code != 0) { terrno = TAOS_SYSTEM_ERROR(code); - qError("vgId:%d failed to rename file, from %s to %s, code:%s", pMeta->vgId, newPath, defaultPath, + stError("vgId:%d failed to rename file, from %s to %s, code:%s", pMeta->vgId, newPath, defaultPath, tstrerror(terrno)); taosMemoryFree(defaultPath); @@ -260,8 +260,8 @@ int32_t streamMetaReopen(SStreamMeta* pMeta) { taosMsleep(2 * 1000); pMeta->streamBackend = streamBackendInit(pMeta->path, pMeta->chkpId); if (pMeta->streamBackend == NULL) { - qError("vgId:%d failed to init stream backend", pMeta->vgId); - qInfo("vgId:%d retry to init stream backend", pMeta->vgId); + stError("vgId:%d failed to init stream backend", pMeta->vgId); + stInfo("vgId:%d retry to init stream backend", pMeta->vgId); // return -1; } } @@ -278,7 +278,7 @@ void streamMetaClear(SStreamMeta* pMeta) { // release the ref by timer if (p->info.triggerParam != 0 && p->info.fillHistory == 0) { // one more ref in timer - qDebug("s-task:%s stop schedTimer, and (before) desc ref:%d", p->id.idStr, p->refCnt); + stDebug("s-task:%s stop schedTimer, and (before) desc ref:%d", p->id.idStr, p->refCnt); taosTmrStop(p->schedInfo.pTimer); p->info.triggerParam = 0; streamMetaReleaseTask(pMeta, p); @@ -300,7 +300,7 @@ void streamMetaClear(SStreamMeta* pMeta) { } void streamMetaClose(SStreamMeta* pMeta) { - qDebug("start to close stream meta"); + stDebug("start to close stream meta"); if (pMeta == NULL) { return; } @@ -316,7 +316,7 @@ void streamMetaClose(SStreamMeta* pMeta) { void streamMetaCloseImpl(void* arg) { SStreamMeta* pMeta = arg; - qDebug("start to do-close stream meta"); + stDebug("start to do-close stream meta"); if (pMeta == NULL) { return; } @@ -341,7 +341,7 @@ void streamMetaCloseImpl(void* arg) { taosThreadMutexDestroy(&pMeta->backendMutex); taosMemoryFree(pMeta); - qDebug("end to close stream meta"); + stDebug("end to close stream meta"); } int32_t streamMetaSaveTask(SStreamMeta* pMeta, SStreamTask* pTask) { @@ -365,7 +365,7 @@ int32_t streamMetaSaveTask(SStreamMeta* pMeta, SStreamTask* pTask) { int64_t id[2] = {pTask->id.streamId, pTask->id.taskId}; if (tdbTbUpsert(pMeta->pTaskDb, id, STREAM_TASK_KEY_LEN, buf, len, pMeta->txn) < 0) { - qError("s-task:%s save to disk failed, code:%s", pTask->id.idStr, tstrerror(terrno)); + stError("s-task:%s save to disk failed, code:%s", pTask->id.idStr, tstrerror(terrno)); return -1; } @@ -377,10 +377,10 @@ int32_t streamMetaRemoveTask(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) { - qError("vgId:%d failed to remove task:0x%x from metastore, code:%s", pMeta->vgId, (int32_t) pTaskId->taskId, + stError("vgId:%d failed to remove task:0x%x from metastore, code:%s", pMeta->vgId, (int32_t) pTaskId->taskId, tstrerror(terrno)); } else { - qDebug("vgId:%d remove task:0x%x from metastore", pMeta->vgId, (int32_t) pTaskId->taskId); + stDebug("vgId:%d remove task:0x%x from metastore", pMeta->vgId, (int32_t) pTaskId->taskId); } return code; @@ -455,7 +455,7 @@ SStreamTask* streamMetaAcquireTask(SStreamMeta* pMeta, int64_t streamId, int32_t if (!streamTaskShouldStop(&(*ppTask)->status)) { int32_t ref = atomic_add_fetch_32(&(*ppTask)->refCnt, 1); taosRUnLockLatch(&pMeta->lock); - qTrace("s-task:%s acquire task, ref:%d", (*ppTask)->id.idStr, ref); + stTrace("s-task:%s acquire task, ref:%d", (*ppTask)->id.idStr, ref); return *ppTask; } } @@ -467,13 +467,13 @@ SStreamTask* streamMetaAcquireTask(SStreamMeta* pMeta, int64_t streamId, int32_t void streamMetaReleaseTask(SStreamMeta* UNUSED_PARAM(pMeta), SStreamTask* pTask) { int32_t ref = atomic_sub_fetch_32(&pTask->refCnt, 1); if (ref > 0) { - qTrace("s-task:%s release task, ref:%d", pTask->id.idStr, ref); + stTrace("s-task:%s release task, ref:%d", pTask->id.idStr, ref); } else if (ref == 0) { ASSERT(streamTaskShouldStop(&pTask->status)); - qTrace("s-task:%s all refs are gone, free it", pTask->id.idStr); + stTrace("s-task:%s all refs are gone, free it", pTask->id.idStr); tFreeStreamTask(pTask); } else if (ref < 0) { - qError("task ref is invalid, ref:%d, %s", ref, pTask->id.idStr); + stError("task ref is invalid, ref:%d, %s", ref, pTask->id.idStr); } } @@ -499,17 +499,17 @@ int32_t streamMetaUnregisterTask(SStreamMeta* pMeta, int64_t streamId, int32_t t pTask = *ppTask; if (streamTaskShouldPause(&pTask->status)) { int32_t num = atomic_sub_fetch_32(&pMeta->numOfPausedTasks, 1); - qInfo("vgId:%d s-task:%s drop stream task. pause task num:%d", pMeta->vgId, pTask->id.idStr, num); + stInfo("vgId:%d s-task:%s drop stream task. pause task num:%d", pMeta->vgId, pTask->id.idStr, num); } atomic_store_8(&pTask->status.taskStatus, TASK_STATUS__DROPPING); } else { - qDebug("vgId:%d failed to find the task:0x%x, it may be dropped already", pMeta->vgId, taskId); + stDebug("vgId:%d failed to find the task:0x%x, it may be dropped already", pMeta->vgId, taskId); taosWUnLockLatch(&pMeta->lock); return 0; } taosWUnLockLatch(&pMeta->lock); - qDebug("s-task:0x%x set task status:%s and start to unregister it", taskId, + stDebug("s-task:0x%x set task status:%s and start to unregister it", taskId, streamGetTaskStatusStr(TASK_STATUS__DROPPING)); while (1) { @@ -523,7 +523,7 @@ int32_t streamMetaUnregisterTask(SStreamMeta* pMeta, int64_t streamId, int32_t t } taosMsleep(10); - qDebug("s-task:%s wait for quit from timer", (*ppTask)->id.idStr); + stDebug("s-task:%s wait for quit from timer", (*ppTask)->id.idStr); taosRUnLockLatch(&pMeta->lock); } else { taosRUnLockLatch(&pMeta->lock); @@ -552,7 +552,7 @@ int32_t streamMetaUnregisterTask(SStreamMeta* pMeta, int64_t streamId, int32_t t doRemoveIdFromList(pMeta, (int32_t)taosArrayGetSize(pMeta->pTaskList), &pTask->id); if (pTask->info.triggerParam != 0 && pTask->info.fillHistory == 0) { - qDebug("s-task:%s stop schedTimer, and (before) desc ref:%d", pTask->id.idStr, pTask->refCnt); + stDebug("s-task:%s stop schedTimer, and (before) desc ref:%d", pTask->id.idStr, pTask->refCnt); taosTmrStop(pTask->schedInfo.pTimer); pTask->info.triggerParam = 0; streamMetaReleaseTask(pMeta, pTask); @@ -561,7 +561,7 @@ int32_t streamMetaUnregisterTask(SStreamMeta* pMeta, int64_t streamId, int32_t t streamMetaRemoveTask(pMeta, &id); streamMetaReleaseTask(pMeta, pTask); } else { - qDebug("vgId:%d failed to find the task:0x%x, it may have been dropped already", pMeta->vgId, taskId); + stDebug("vgId:%d failed to find the task:0x%x, it may have been dropped already", pMeta->vgId, taskId); } taosWUnLockLatch(&pMeta->lock); @@ -582,18 +582,18 @@ int32_t streamMetaBegin(SStreamMeta* pMeta) { // todo add error log int32_t streamMetaCommit(SStreamMeta* pMeta) { if (tdbCommit(pMeta->db, pMeta->txn) < 0) { - qError("vgId:%d failed to commit stream meta", pMeta->vgId); + stError("vgId:%d failed to commit stream meta", pMeta->vgId); return -1; } if (tdbPostCommit(pMeta->db, pMeta->txn) < 0) { - qError("vgId:%d failed to do post-commit stream meta", pMeta->vgId); + stError("vgId:%d failed to do post-commit stream meta", pMeta->vgId); return -1; } if (tdbBegin(pMeta->db, &pMeta->txn, tdbDefaultMalloc, tdbDefaultFree, NULL, TDB_TXN_WRITE | TDB_TXN_READ_UNCOMMITTED) < 0) { - qError("vgId:%d failed to begin trans", pMeta->vgId); + stError("vgId:%d failed to begin trans", pMeta->vgId); return -1; } @@ -629,7 +629,7 @@ int64_t streamGetLatestCheckpointId(SStreamMeta* pMeta) { chkpId = TMAX(chkpId, info.checkpointId); } - qDebug("get max chkp id: %" PRId64 "", chkpId); + stDebug("get max chkp id: %" PRId64 "", chkpId); tdbFree(pKey); tdbFree(pVal); @@ -649,10 +649,10 @@ int32_t streamMetaLoadAllTasks(SStreamMeta* pMeta) { TBC* pCur = NULL; int32_t vgId = pMeta->vgId; - qInfo("vgId:%d load stream tasks from meta files", vgId); + stInfo("vgId:%d load stream tasks from meta files", vgId); if (tdbTbcOpen(pMeta->pTaskDb, &pCur, NULL) < 0) { - qError("vgId:%d failed to open stream meta, code:%s", vgId, tstrerror(terrno)); + stError("vgId:%d failed to open stream meta, code:%s", vgId, tstrerror(terrno)); return -1; } @@ -668,7 +668,7 @@ int32_t streamMetaLoadAllTasks(SStreamMeta* pMeta) { SStreamTask* pTask = taosMemoryCalloc(1, sizeof(SStreamTask)); if (pTask == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; - qError("vgId:%d failed to load stream task from meta-files, code:%s", vgId, tstrerror(terrno)); + stError("vgId:%d failed to load stream task from meta-files, code:%s", vgId, tstrerror(terrno)); doClear(pKey, pVal, pCur, pRecycleList); return -1; } @@ -678,7 +678,7 @@ int32_t streamMetaLoadAllTasks(SStreamMeta* pMeta) { tDecoderClear(&decoder); doClear(pKey, pVal, pCur, pRecycleList); tFreeStreamTask(pTask); - qError( + stError( "vgId:%d stream read incompatible data, rm %s/vnode/vnode*/tq/stream if taosd cannot start, and rebuild stream " "manually", vgId, tsDataDir); return -1; @@ -693,7 +693,7 @@ int32_t streamMetaLoadAllTasks(SStreamMeta* pMeta) { taosArrayPush(pRecycleList, &id); int32_t total = taosArrayGetSize(pRecycleList); - qDebug("s-task:0x%x is already dropped, add into recycle list, total:%d", taskId, total); + stDebug("s-task:0x%x is already dropped, add into recycle list, total:%d", taskId, total); continue; } @@ -737,7 +737,7 @@ int32_t streamMetaLoadAllTasks(SStreamMeta* pMeta) { tdbFree(pKey); tdbFree(pVal); if (tdbTbcClose(pCur) < 0) { - qError("vgId:%d failed to close meta-file cursor", vgId); + stError("vgId:%d failed to close meta-file cursor", vgId); taosArrayDestroy(pRecycleList); return -1; } @@ -751,7 +751,7 @@ int32_t streamMetaLoadAllTasks(SStreamMeta* pMeta) { int32_t numOfTasks = taosArrayGetSize(pMeta->pTaskList); ASSERT(pMeta->numOfStreamTasks <= numOfTasks && pMeta->numOfPausedTasks <= numOfTasks); - qDebug("vgId:%d load %d tasks into meta from disk completed, streamTask:%d, paused:%d", pMeta->vgId, numOfTasks, + stDebug("vgId:%d load %d tasks into meta from disk completed, streamTask:%d, paused:%d", pMeta->vgId, numOfTasks, pMeta->numOfStreamTasks, pMeta->numOfPausedTasks); taosArrayDestroy(pRecycleList); return 0; @@ -817,14 +817,14 @@ void metaHbToMnode(void* param, void* tmrId) { // need to stop, stop now if (pMeta->pHbInfo->stopFlag == STREAM_META_WILL_STOP) { pMeta->pHbInfo->stopFlag = STREAM_META_OK_TO_STOP; - qDebug("vgId:%d jump out of meta timer", pMeta->vgId); + stDebug("vgId:%d jump out of meta timer", pMeta->vgId); taosReleaseRef(streamMetaId, rid); return; } // not leader not send msg if (!pMeta->leader) { - qInfo("vgId:%d follower not send hb to mnode", pMeta->vgId); + stInfo("vgId:%d follower not send hb to mnode", pMeta->vgId); taosReleaseRef(streamMetaId, rid); pMeta->pHbInfo->hbStart = 0; return; @@ -841,7 +841,7 @@ void metaHbToMnode(void* param, void* tmrId) { return; } - qDebug("vgId:%d build stream task hb, leader:%d", pMeta->vgId, pMeta->leader); + stDebug("vgId:%d build stream task hb, leader:%d", pMeta->vgId, pMeta->leader); SStreamHbMsg hbMsg = {0}; taosRLockLatch(&pMeta->lock); @@ -879,7 +879,7 @@ void metaHbToMnode(void* param, void* tmrId) { tEncodeSize(tEncodeStreamHbMsg, &hbMsg, tlen, code); if (code < 0) { - qError("vgId:%d encode stream hb msg failed, code:%s", pMeta->vgId, tstrerror(code)); + stError("vgId:%d encode stream hb msg failed, code:%s", pMeta->vgId, tstrerror(code)); taosArrayDestroy(hbMsg.pTaskStatus); taosReleaseRef(streamMetaId, rid); return; @@ -887,7 +887,7 @@ void metaHbToMnode(void* param, void* tmrId) { void* buf = rpcMallocCont(tlen); if (buf == NULL) { - qError("vgId:%d encode stream hb msg failed, code:%s", pMeta->vgId, tstrerror(TSDB_CODE_OUT_OF_MEMORY)); + stError("vgId:%d encode stream hb msg failed, code:%s", pMeta->vgId, tstrerror(TSDB_CODE_OUT_OF_MEMORY)); taosArrayDestroy(hbMsg.pTaskStatus); taosReleaseRef(streamMetaId, rid); return; @@ -897,7 +897,7 @@ void metaHbToMnode(void* param, void* tmrId) { tEncoderInit(&encoder, buf, tlen); if ((code = tEncodeStreamHbMsg(&encoder, &hbMsg)) < 0) { rpcFreeCont(buf); - qError("vgId:%d encode stream hb msg failed, code:%s", pMeta->vgId, tstrerror(code)); + stError("vgId:%d encode stream hb msg failed, code:%s", pMeta->vgId, tstrerror(code)); taosArrayDestroy(hbMsg.pTaskStatus); taosReleaseRef(streamMetaId, rid); return; @@ -910,11 +910,11 @@ void metaHbToMnode(void* param, void* tmrId) { pMeta->pHbInfo->hbCount += 1; - qDebug("vgId:%d, build and send hb to mnode, numOfTasks:%d total:%d", pMeta->vgId, hbMsg.numOfTasks, + stDebug("vgId:%d, build and send hb to mnode, numOfTasks:%d total:%d", pMeta->vgId, hbMsg.numOfTasks, pMeta->pHbInfo->hbCount); tmsgSendReq(&epset, &msg); } else { - qDebug("vgId:%d no tasks and no mnd epset, not send stream hb to mnode", pMeta->vgId); + stDebug("vgId:%d no tasks and no mnd epset, not send stream hb to mnode", pMeta->vgId); } taosArrayDestroy(hbMsg.pTaskStatus); @@ -947,7 +947,7 @@ static bool hasStreamTaskInTimer(SStreamMeta* pMeta) { void streamMetaNotifyClose(SStreamMeta* pMeta) { int32_t vgId = pMeta->vgId; - qDebug("vgId:%d notify all stream tasks that the vnode is closing. isLeader:%d startHb%" PRId64 ", totalHb:%d", vgId, + stDebug("vgId:%d notify all stream tasks that the vnode is closing. isLeader:%d startHb%" PRId64 ", totalHb:%d", vgId, pMeta->leader, pMeta->pHbInfo->hbStart, pMeta->pHbInfo->hbCount); taosWLockLatch(&pMeta->lock); @@ -960,7 +960,7 @@ void streamMetaNotifyClose(SStreamMeta* pMeta) { } SStreamTask* pTask = *(SStreamTask**)pIter; - qDebug("vgId:%d s-task:%s set closing flag", vgId, pTask->id.idStr); + stDebug("vgId:%d s-task:%s set closing flag", vgId, pTask->id.idStr); streamTaskStop(pTask); } @@ -971,20 +971,20 @@ void streamMetaNotifyClose(SStreamMeta* pMeta) { pMeta->pHbInfo->stopFlag = STREAM_META_WILL_STOP; while (pMeta->pHbInfo->stopFlag != STREAM_META_OK_TO_STOP) { taosMsleep(100); - qDebug("vgId:%d wait for meta to stop timer", pMeta->vgId); + stDebug("vgId:%d wait for meta to stop timer", pMeta->vgId); } } - qDebug("vgId:%d start to check all tasks", vgId); + stDebug("vgId:%d start to check all tasks", vgId); int64_t st = taosGetTimestampMs(); while (hasStreamTaskInTimer(pMeta)) { - qDebug("vgId:%d some tasks in timer, wait for 100ms and recheck", pMeta->vgId); + stDebug("vgId:%d some tasks in timer, wait for 100ms and recheck", pMeta->vgId); taosMsleep(100); } int64_t el = taosGetTimestampMs() - st; - qDebug("vgId:%d all stream tasks are not in timer, continue close, elapsed time:%" PRId64 " ms", pMeta->vgId, el); + stDebug("vgId:%d all stream tasks are not in timer, continue close, elapsed time:%" PRId64 " ms", pMeta->vgId, el); } void streamMetaStartHb(SStreamMeta* pMeta) { diff --git a/source/libs/stream/src/streamQueue.c b/source/libs/stream/src/streamQueue.c index d3d114d4aa..901484377a 100644 --- a/source/libs/stream/src/streamQueue.c +++ b/source/libs/stream/src/streamQueue.c @@ -65,7 +65,7 @@ SStreamQueue* streamQueueOpen(int64_t cap) { } void streamQueueClose(SStreamQueue* pQueue, int32_t taskId) { - qDebug("s-task:0x%x free the queue:%p, items in queue:%d", taskId, pQueue->pQueue, taosQueueItemSize(pQueue->pQueue)); + stDebug("s-task:0x%x free the queue:%p, items in queue:%d", taskId, pQueue->pQueue, taosQueueItemSize(pQueue->pQueue)); streamQueueCleanup(pQueue); taosFreeQall(pQueue->qall); @@ -186,24 +186,24 @@ int32_t streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueItem** pInpu if (pTask->info.taskLevel == TASK_LEVEL__SINK) { // extract block from inputQ, one-by-one while (1) { if (streamTaskShouldPause(&pTask->status) || streamTaskShouldStop(&pTask->status)) { - qDebug("s-task:%s task should pause, extract input blocks:%d", pTask->id.idStr, *numOfBlocks); + stDebug("s-task:%s task should pause, extract input blocks:%d", pTask->id.idStr, *numOfBlocks); return TSDB_CODE_SUCCESS; } STokenBucket* pBucket = pTask->pTokenBucket; if (!streamTaskHasAvailableToken(pBucket)) { // no available token in th bucket, ignore this execution -// qInfo("s-task:%s no available token for sink, capacity:%d, rate:%d token/sec, quit", pTask->id.idStr, +// stInfo("s-task:%s no available token for sink, capacity:%d, rate:%d token/sec, quit", pTask->id.idStr, // pBucket->capacity, pBucket->rate); return TSDB_CODE_SUCCESS; } SStreamQueueItem* qItem = streamQueueNextItem(pTask->inputInfo.queue); if (qItem == NULL) { - qDebug("===stream===break batchSize:%d, %s", *numOfBlocks, id); + stDebug("===stream===break batchSize:%d, %s", *numOfBlocks, id); return TSDB_CODE_SUCCESS; } - qDebug("s-task:%s sink task handle block one-by-one, type:%d", id, qItem->type); + stDebug("s-task:%s sink task handle block one-by-one, type:%d", id, qItem->type); *numOfBlocks = 1; *pInput = qItem; @@ -213,7 +213,7 @@ int32_t streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueItem** pInpu while (1) { if (streamTaskShouldPause(&pTask->status) || streamTaskShouldStop(&pTask->status)) { - qDebug("s-task:%s task should pause, extract input blocks:%d", pTask->id.idStr, *numOfBlocks); + stDebug("s-task:%s task should pause, extract input blocks:%d", pTask->id.idStr, *numOfBlocks); return TSDB_CODE_SUCCESS; } @@ -221,11 +221,11 @@ int32_t streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueItem** pInpu if (qItem == NULL) { if (pTask->info.taskLevel == TASK_LEVEL__SOURCE && (++retryTimes) < MAX_RETRY_TIMES) { taosMsleep(10); - qDebug("===stream===try again batchSize:%d, retry:%d, %s", *numOfBlocks, retryTimes, id); + stDebug("===stream===try again batchSize:%d, retry:%d, %s", *numOfBlocks, retryTimes, id); continue; } - qDebug("===stream===break batchSize:%d, %s", *numOfBlocks, id); + stDebug("===stream===break batchSize:%d, %s", *numOfBlocks, id); return TSDB_CODE_SUCCESS; } @@ -235,14 +235,14 @@ int32_t streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueItem** pInpu const char* p = streamGetBlockTypeStr(qItem->type); if (*pInput == NULL) { - qDebug("s-task:%s %s msg extracted, start to process immediately", id, p); + stDebug("s-task:%s %s msg extracted, start to process immediately", id, p); *numOfBlocks = 1; *pInput = qItem; return TSDB_CODE_SUCCESS; } else { // previous existed blocks needs to be handle, before handle the checkpoint msg block - qDebug("s-task:%s %s msg extracted, handle previous blocks, numOfBlocks:%d", id, p, *numOfBlocks); + stDebug("s-task:%s %s msg extracted, handle previous blocks, numOfBlocks:%d", id, p, *numOfBlocks); streamQueueProcessFail(pTask->inputInfo.queue); return TSDB_CODE_SUCCESS; } @@ -255,7 +255,7 @@ int32_t streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueItem** pInpu void* newRet = streamMergeQueueItem(*pInput, qItem); if (newRet == NULL) { if (terrno != 0) { - qError("s-task:%s failed to merge blocks from inputQ, numOfBlocks:%d, code:%s", id, *numOfBlocks, + stError("s-task:%s failed to merge blocks from inputQ, numOfBlocks:%d, code:%s", id, *numOfBlocks, tstrerror(terrno)); } @@ -270,7 +270,7 @@ int32_t streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueItem** pInpu streamQueueProcessSuccess(pTask->inputInfo.queue); if (*numOfBlocks >= MAX_STREAM_EXEC_BATCH_NUM) { - qDebug("s-task:%s batch size limit:%d reached, start to process blocks", id, MAX_STREAM_EXEC_BATCH_NUM); + stDebug("s-task:%s batch size limit:%d reached, start to process blocks", id, MAX_STREAM_EXEC_BATCH_NUM); return TSDB_CODE_SUCCESS; } } @@ -286,7 +286,7 @@ int32_t streamTaskPutDataIntoInputQ(SStreamTask* pTask, SStreamQueueItem* pItem) SStreamDataSubmit* px = (SStreamDataSubmit*)pItem; if ((pTask->info.taskLevel == TASK_LEVEL__SOURCE) && streamQueueIsFull(pQueue, true)) { double size = SIZE_IN_MB(taosQueueMemorySize(pQueue)); - qTrace( + stTrace( "s-task:%s inputQ is full, capacity(size:%d num:%dMiB), current(blocks:%d, size:%.2fMiB) stop to push data", pTask->id.idStr, STREAM_TASK_QUEUE_CAPACITY, STREAM_TASK_INPUT_QUEUE_CAPACITY_IN_SIZE, total, size); streamDataSubmitDestroy(px); @@ -307,14 +307,14 @@ int32_t streamTaskPutDataIntoInputQ(SStreamTask* pTask, SStreamQueueItem* pItem) double size = SIZE_IN_MB(taosQueueMemorySize(pQueue)); // use the local variable to avoid the pItem be freed by other threads, since it has been put into queue already. - qDebug("s-task:%s submit enqueue msgLen:%d ver:%" PRId64 ", total in queue:%d, size:%.2fMiB", pTask->id.idStr, + stDebug("s-task:%s submit enqueue msgLen:%d ver:%" PRId64 ", total in queue:%d, size:%.2fMiB", pTask->id.idStr, msgLen, ver, total, size + SIZE_IN_MB(msgLen)); } else if (type == STREAM_INPUT__DATA_BLOCK || type == STREAM_INPUT__DATA_RETRIEVE || type == STREAM_INPUT__REF_DATA_BLOCK) { if (streamQueueIsFull(pQueue, true)) { double size = SIZE_IN_MB(taosQueueMemorySize(pQueue)); - qTrace("s-task:%s input queue is full, capacity:%d size:%d MiB, current(blocks:%d, size:%.2fMiB) abort", + stTrace("s-task:%s input queue is full, capacity:%d size:%d MiB, current(blocks:%d, size:%.2fMiB) abort", pTask->id.idStr, STREAM_TASK_QUEUE_CAPACITY, STREAM_TASK_INPUT_QUEUE_CAPACITY_IN_SIZE, total, size); destroyStreamDataBlock((SStreamDataBlock*)pItem); return -1; @@ -327,7 +327,7 @@ int32_t streamTaskPutDataIntoInputQ(SStreamTask* pTask, SStreamQueueItem* pItem) } double size = SIZE_IN_MB(taosQueueMemorySize(pQueue)); - qDebug("s-task:%s blockdata enqueue, total in queue:%d, size:%.2fMiB", pTask->id.idStr, total, size); + 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) { int32_t code = taosWriteQitem(pQueue, pItem); @@ -337,7 +337,7 @@ int32_t streamTaskPutDataIntoInputQ(SStreamTask* pTask, SStreamQueueItem* pItem) } double size = SIZE_IN_MB(taosQueueMemorySize(pQueue)); - qDebug("s-task:%s level:%d %s blockdata enqueue, total in queue:%d, size:%.2fMiB", pTask->id.idStr, + stDebug("s-task:%s level:%d %s blockdata enqueue, total in queue:%d, size:%.2fMiB", pTask->id.idStr, pTask->info.taskLevel, streamGetBlockTypeStr(type), total, size); } else if (type == STREAM_INPUT__GET_RES) { // use the default memory limit, refactor later. @@ -348,14 +348,14 @@ int32_t streamTaskPutDataIntoInputQ(SStreamTask* pTask, SStreamQueueItem* pItem) } double size = SIZE_IN_MB(taosQueueMemorySize(pQueue)); - qDebug("s-task:%s data res enqueue, current(blocks:%d, size:%.2fMiB)", pTask->id.idStr, total, size); + stDebug("s-task:%s data res enqueue, current(blocks:%d, size:%.2fMiB)", pTask->id.idStr, total, size); } else { ASSERT(0); } if (type != STREAM_INPUT__GET_RES && type != STREAM_INPUT__CHECKPOINT && pTask->info.triggerParam != 0) { atomic_val_compare_exchange_8(&pTask->schedInfo.status, TASK_TRIGGER_STATUS__INACTIVE, TASK_TRIGGER_STATUS__ACTIVE); - qDebug("s-task:%s new data arrived, active the trigger, triggerStatus:%d", pTask->id.idStr, pTask->schedInfo.status); + stDebug("s-task:%s new data arrived, active the trigger, triggerStatus:%d", pTask->id.idStr, pTask->schedInfo.status); } return 0; @@ -367,14 +367,14 @@ int32_t streamTaskPutDataIntoOutputQ(SStreamTask* pTask, SStreamDataBlock* pBloc while (streamQueueIsFull(pQueue, false)) { if (streamTaskShouldStop(&pTask->status)) { - qInfo("s-task:%s discard result block due to task stop", pTask->id.idStr); + stInfo("s-task:%s discard result block due to task stop", pTask->id.idStr); return TSDB_CODE_STREAM_EXEC_CANCELLED; } int32_t total = streamQueueGetNumOfItems(pTask->outputInfo.queue); double size = SIZE_IN_MB(taosQueueMemorySize(pQueue)); // let's wait for there are enough space to hold this result pBlock - qDebug("s-task:%s outputQ is full, wait for 500ms and retry, outputQ items:%d, size:%.2fMiB", pTask->id.idStr, + stDebug("s-task:%s outputQ is full, wait for 500ms and retry, outputQ items:%d, size:%.2fMiB", pTask->id.idStr, total, size); taosMsleep(500); } @@ -384,10 +384,10 @@ int32_t streamTaskPutDataIntoOutputQ(SStreamTask* pTask, SStreamDataBlock* pBloc int32_t total = streamQueueGetNumOfItems(pTask->outputInfo.queue); double size = SIZE_IN_MB(taosQueueMemorySize(pQueue)); if (code != 0) { - qError("s-task:%s failed to put res into outputQ, outputQ items:%d, size:%.2fMiB code:%s, result lost", + stError("s-task:%s failed to put res into outputQ, outputQ items:%d, size:%.2fMiB code:%s, result lost", pTask->id.idStr, total + 1, size, tstrerror(code)); } else { - qDebug("s-task:%s data put into outputQ, outputQ items:%d, size:%.2fMiB", pTask->id.idStr, total, size); + stDebug("s-task:%s data put into outputQ, outputQ items:%d, size:%.2fMiB", pTask->id.idStr, total, size); } return TSDB_CODE_SUCCESS; @@ -395,7 +395,7 @@ int32_t streamTaskPutDataIntoOutputQ(SStreamTask* pTask, SStreamDataBlock* pBloc int32_t streamTaskInitTokenBucket(STokenBucket* pBucket, int32_t cap, int32_t rate) { if (cap < 50 || rate < 50 || pBucket == NULL) { - qError("failed to init sink task bucket, cap:%d, rate:%d", cap, rate); + stError("failed to init sink task bucket, cap:%d, rate:%d", cap, rate); return TSDB_CODE_INVALID_PARA; } @@ -420,7 +420,7 @@ static void fillBucket(STokenBucket* pBucket) { } pBucket->fillTimestamp = now; - qDebug("new token available, current:%d, inc:%d ts:%"PRId64, pBucket->numOfToken, inc, now); + stDebug("new token available, current:%d, inc:%d ts:%"PRId64, pBucket->numOfToken, inc, now); } } diff --git a/source/libs/stream/src/streamRecover.c b/source/libs/stream/src/streamRecover.c index 2689e9ee70..83d8fe0b2a 100644 --- a/source/libs/stream/src/streamRecover.c +++ b/source/libs/stream/src/streamRecover.c @@ -31,7 +31,7 @@ static int32_t initScanHistoryReq(SStreamTask* pTask, SStreamScanHistoryReq* pRe static void streamTaskSetReady(SStreamTask* pTask, int32_t numOfReqs) { if (pTask->status.taskStatus == TASK_STATUS__SCAN_HISTORY && pTask->info.taskLevel != TASK_LEVEL__SOURCE) { pTask->numOfWaitingUpstream = taosArrayGetSize(pTask->pUpstreamInfoList); - qDebug("s-task:%s level:%d task wait for %d upstream tasks complete scan-history procedure, status:%s", + stDebug("s-task:%s level:%d task wait for %d upstream tasks complete scan-history procedure, status:%s", pTask->id.idStr, pTask->info.taskLevel, pTask->numOfWaitingUpstream, streamGetTaskStatusStr(pTask->status.taskStatus)); } @@ -40,7 +40,7 @@ static void streamTaskSetReady(SStreamTask* pTask, int32_t numOfReqs) { pTask->status.downstreamReady = 1; int64_t el = (taosGetTimestampMs() - pTask->taskExecInfo.init); - qDebug("s-task:%s all %d downstream ready, init completed, elapsed time:%"PRId64"ms, task status:%s", + stDebug("s-task:%s all %d downstream ready, init completed, elapsed time:%"PRId64"ms, task status:%s", pTask->id.idStr, numOfReqs, el, streamGetTaskStatusStr(pTask->status.taskStatus)); } @@ -95,7 +95,7 @@ int32_t streamTaskLaunchScanHistory(SStreamTask* pTask) { return doLaunchScanHistoryTask(pTask); } else { ASSERT(pTask->status.taskStatus == TASK_STATUS__NORMAL); - qDebug("s-task:%s no need to scan-history-data, status:%s, sched-status:%d, ver:%" PRId64, pTask->id.idStr, + stDebug("s-task:%s no need to scan-history-data, status:%s, sched-status:%d, ver:%" PRId64, pTask->id.idStr, streamGetTaskStatusStr(pTask->status.taskStatus), pTask->status.schedStatus, walReaderGetCurrentVer(pTask->exec.pWalReader)); } @@ -105,7 +105,7 @@ int32_t streamTaskLaunchScanHistory(SStreamTask* pTask) { streamTaskEnablePause(pTask); } } else if (pTask->info.taskLevel == TASK_LEVEL__SINK) { - qDebug("s-task:%s sink task do nothing to handle scan-history", pTask->id.idStr); + stDebug("s-task:%s sink task do nothing to handle scan-history", pTask->id.idStr); } return 0; } @@ -130,7 +130,7 @@ static int32_t doCheckDownstreamStatus(SStreamTask* pTask) { req.downstreamTaskId = pTask->fixedEpDispatcher.taskId; pTask->checkReqId = req.reqId; - qDebug("s-task:%s check single downstream task:0x%x(vgId:%d) ver:%" PRId64 "-%" PRId64 " window:%" PRId64 + stDebug("s-task:%s check single downstream task:0x%x(vgId:%d) ver:%" PRId64 "-%" PRId64 " window:%" PRId64 "-%" PRId64 ", stage:%"PRId64" req:0x%" PRIx64, pTask->id.idStr, req.downstreamTaskId, req.downstreamNodeId, pRange->range.minVer, pRange->range.maxVer, pWindow->skey, pWindow->ekey, req.stage, req.reqId); @@ -143,7 +143,7 @@ static int32_t doCheckDownstreamStatus(SStreamTask* pTask) { pTask->notReadyTasks = numOfVgs; pTask->checkReqIds = taosArrayInit(numOfVgs, sizeof(int64_t)); - qDebug("s-task:%s check %d downstream tasks, ver:%" PRId64 "-%" PRId64 " window:%" PRId64 "-%" PRId64, + stDebug("s-task:%s check %d downstream tasks, ver:%" PRId64 "-%" PRId64 " window:%" PRId64 "-%" PRId64, pTask->id.idStr, numOfVgs, pRange->range.minVer, pRange->range.maxVer, pWindow->skey, pWindow->ekey); for (int32_t i = 0; i < numOfVgs; i++) { @@ -152,12 +152,12 @@ static int32_t doCheckDownstreamStatus(SStreamTask* pTask) { taosArrayPush(pTask->checkReqIds, &req.reqId); req.downstreamNodeId = pVgInfo->vgId; req.downstreamTaskId = pVgInfo->taskId; - qDebug("s-task:%s (vgId:%d) check downstream task:0x%x (vgId:%d) (shuffle), idx:%d, stage:%" PRId64, + stDebug("s-task:%s (vgId:%d) check downstream task:0x%x (vgId:%d) (shuffle), idx:%d, stage:%" PRId64, pTask->id.idStr, pTask->info.nodeId, req.downstreamTaskId, req.downstreamNodeId, i, req.stage); streamDispatchCheckMsg(pTask, &req, pVgInfo->vgId, &pVgInfo->epSet); } } else { - qDebug("s-task:%s (vgId:%d) set downstream ready, since no downstream", pTask->id.idStr, pTask->info.nodeId); + stDebug("s-task:%s (vgId:%d) set downstream ready, since no downstream", pTask->id.idStr, pTask->info.nodeId); streamTaskSetReady(pTask, 0); streamTaskSetRangeStreamCalc(pTask); @@ -181,7 +181,7 @@ int32_t streamRecheckDownstream(SStreamTask* pTask, const SStreamTaskCheckRsp* p }; if (pTask->outputInfo.type == TASK_OUTPUT__FIXED_DISPATCH) { - qDebug("s-task:%s (vgId:%d) check downstream task:0x%x (vgId:%d) stage:%" PRId64 " (recheck)", pTask->id.idStr, + stDebug("s-task:%s (vgId:%d) check downstream task:0x%x (vgId:%d) stage:%" PRId64 " (recheck)", pTask->id.idStr, pTask->info.nodeId, req.downstreamTaskId, req.downstreamNodeId, req.stage); streamDispatchCheckMsg(pTask, &req, pRsp->downstreamNodeId, &pTask->fixedEpDispatcher.epSet); } else if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { @@ -191,7 +191,7 @@ int32_t streamRecheckDownstream(SStreamTask* pTask, const SStreamTaskCheckRsp* p for (int32_t i = 0; i < numOfVgs; i++) { SVgroupInfo* pVgInfo = taosArrayGet(vgInfo, i); if (pVgInfo->taskId == req.downstreamTaskId) { - qDebug("s-task:%s (vgId:%d) check downstream task:0x%x (vgId:%d) stage:%" PRId64 " (recheck)", pTask->id.idStr, + stDebug("s-task:%s (vgId:%d) check downstream task:0x%x (vgId:%d) stage:%" PRId64 " (recheck)", pTask->id.idStr, pTask->info.nodeId, req.downstreamTaskId, req.downstreamNodeId, req.stage); streamDispatchCheckMsg(pTask, &req, pRsp->downstreamNodeId, &pVgInfo->epSet); } @@ -207,19 +207,19 @@ int32_t streamTaskCheckStatus(SStreamTask* pTask, int32_t upstreamTaskId, int32_ const char* id = pTask->id.idStr; if (stage == -1) { - qDebug("s-task:%s receive check msg from upstream task:0x%x, invalid stageId:%" PRId64 ", not ready", id, + stDebug("s-task:%s receive check msg from upstream task:0x%x, invalid stageId:%" PRId64 ", not ready", id, upstreamTaskId, stage); return 0; } if (pInfo->stage == -1) { pInfo->stage = stage; - qDebug("s-task:%s receive check msg from upstream task:0x%x for the time, init stage value:%" PRId64, id, + stDebug("s-task:%s receive check msg from upstream task:0x%x for the time, init stage value:%" PRId64, id, upstreamTaskId, stage); } if (pInfo->stage < stage) { - qError("s-task:%s receive msg from upstream task:0x%x(vgId:%d), new stage received:%" PRId64 ", prev:%" PRId64, + stError("s-task:%s receive msg from upstream task:0x%x(vgId:%d), new stage received:%" PRId64 ", prev:%" PRId64, id, upstreamTaskId, vgId, stage, pInfo->stage); } @@ -237,15 +237,15 @@ static void doProcessDownstreamReadyRsp(SStreamTask* pTask, int32_t numOfReqs) { streamTaskSetRangeStreamCalc(pTask); if (status == TASK_STATUS__SCAN_HISTORY) { - qDebug("s-task:%s enter into scan-history data stage, status:%s", id, str); + stDebug("s-task:%s enter into scan-history data stage, status:%s", id, str); streamTaskLaunchScanHistory(pTask); } else { if (pTask->info.fillHistory == 1) { - qDebug("s-task:%s fill-history is set normal when start it, try to remove it,set it task to be dropping", id); + stDebug("s-task:%s fill-history is set normal when start it, try to remove it,set it task to be dropping", id); pTask->status.taskStatus = TASK_STATUS__DROPPING; ASSERT(pTask->historyTaskId.taskId == 0); } else { - qDebug("s-task:%s downstream tasks are ready, now ready for data from wal, status:%s", id, str); + stDebug("s-task:%s downstream tasks are ready, now ready for data from wal, status:%s", id, str); streamTaskEnablePause(pTask); } } @@ -286,7 +286,7 @@ int32_t streamProcessCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* pRs doProcessDownstreamReadyRsp(pTask, numOfReqs); } else { int32_t total = taosArrayGetSize(pTask->shuffleDispatcher.dbInfo.pVgroupInfos); - qDebug("s-task:%s (vgId:%d) recv check rsp from task:0x%x (vgId:%d) status:%d, total:%d not ready:%d", id, + stDebug("s-task:%s (vgId:%d) recv check rsp from task:0x%x (vgId:%d) status:%d, total:%d not ready:%d", id, pRsp->upstreamNodeId, pRsp->downstreamTaskId, pRsp->downstreamNodeId, pRsp->status, total, left); } } else { @@ -298,7 +298,7 @@ int32_t streamProcessCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* pRs doProcessDownstreamReadyRsp(pTask, 1); } } else { // not ready, wait for 100ms and retry - qDebug("s-task:%s downstream taskId:0x%x (vgId:%d) not ready, stage:%d, wait for 100ms and retry", id, + stDebug("s-task:%s downstream taskId:0x%x (vgId:%d) not ready, stage:%d, wait for 100ms and retry", id, pRsp->downstreamTaskId, pRsp->downstreamNodeId, pRsp->oldStage); taosMsleep(100); streamRecheckDownstream(pTask, pRsp); @@ -315,7 +315,7 @@ int32_t streamSendCheckRsp(const SStreamMeta* pMeta, const SStreamTaskCheckReq* tEncodeSize(tEncodeStreamTaskCheckRsp, pRsp, len, code); if (code < 0) { - qError("vgId:%d failed to encode task check rsp, s-task:0x%x", pMeta->vgId, taskId); + stError("vgId:%d failed to encode task check rsp, s-task:0x%x", pMeta->vgId, taskId); return -1; } @@ -335,22 +335,22 @@ int32_t streamSendCheckRsp(const SStreamMeta* pMeta, const SStreamTaskCheckReq* // common int32_t streamSetParamForScanHistory(SStreamTask* pTask) { - qDebug("s-task:%s set operator option for scan-history data", pTask->id.idStr); + stDebug("s-task:%s set operator option for scan-history data", pTask->id.idStr); return qSetStreamOperatorOptionForScanHistory(pTask->exec.pExecutor); } int32_t streamRestoreParam(SStreamTask* pTask) { - qDebug("s-task:%s restore operator param after scan-history", pTask->id.idStr); + stDebug("s-task:%s restore operator param after scan-history", pTask->id.idStr); return qRestoreStreamOperatorOption(pTask->exec.pExecutor); } int32_t streamSetStatusNormal(SStreamTask* pTask) { int32_t status = atomic_load_8(&pTask->status.taskStatus); if (status == TASK_STATUS__DROPPING) { - qError("s-task:%s cannot be set normal, since in dropping state", pTask->id.idStr); + stError("s-task:%s cannot be set normal, since in dropping state", pTask->id.idStr); return -1; } else { - qDebug("s-task:%s set task status to be normal, prev:%s", pTask->id.idStr, streamGetTaskStatusStr(status)); + stDebug("s-task:%s set task status to be normal, prev:%s", pTask->id.idStr, streamGetTaskStatusStr(status)); atomic_store_8(&pTask->status.taskStatus, TASK_STATUS__NORMAL); return 0; } @@ -359,10 +359,10 @@ int32_t streamSetStatusNormal(SStreamTask* pTask) { int32_t streamSetStatusUnint(SStreamTask* pTask) { int32_t status = atomic_load_8(&pTask->status.taskStatus); if (status == TASK_STATUS__DROPPING) { - qError("s-task:%s cannot be set uninit, since in dropping state", pTask->id.idStr); + stError("s-task:%s cannot be set uninit, since in dropping state", pTask->id.idStr); return -1; } else { - qDebug("s-task:%s set task status to be uninit, prev:%s", pTask->id.idStr, streamGetTaskStatusStr(status)); + stDebug("s-task:%s set task status to be uninit, prev:%s", pTask->id.idStr, streamGetTaskStatusStr(status)); atomic_store_8(&pTask->status.taskStatus, TASK_STATUS__UNINIT); return 0; } @@ -434,7 +434,7 @@ int32_t streamProcessScanHistoryFinishReq(SStreamTask* pTask, SStreamScanHistory ASSERT(taskLevel == TASK_LEVEL__AGG || taskLevel == TASK_LEVEL__SINK); if (pTask->status.taskStatus != TASK_STATUS__SCAN_HISTORY) { - qError("s-task:%s not in scan-history status, status:%s return upstream:0x%x scan-history finish directly", + stError("s-task:%s not in scan-history status, status:%s return upstream:0x%x scan-history finish directly", pTask->id.idStr, streamGetTaskStatusStr(pTask->status.taskStatus), pReq->upstreamTaskId); void* pBuf = NULL; @@ -445,7 +445,7 @@ int32_t streamProcessScanHistoryFinishReq(SStreamTask* pTask, SStreamScanHistory initRpcMsg(&msg, 0, pBuf, sizeof(SMsgHead) + len); tmsgSendRsp(&msg); - qDebug("s-task:%s level:%d notify upstream:0x%x(vgId:%d) to continue process data in WAL", pTask->id.idStr, + stDebug("s-task:%s level:%d notify upstream:0x%x(vgId:%d) to continue process data in WAL", pTask->id.idStr, pTask->info.taskLevel, pReq->upstreamTaskId, pReq->upstreamNodeId); return 0; } @@ -458,7 +458,7 @@ int32_t streamProcessScanHistoryFinishReq(SStreamTask* pTask, SStreamScanHistory if (left == 0) { int32_t numOfTasks = taosArrayGetSize(pTask->pUpstreamInfoList); - qDebug( + stDebug( "s-task:%s all %d upstream tasks finish scan-history data, set param for agg task for stream data and send " "rsp to all upstream tasks", pTask->id.idStr, numOfTasks); @@ -481,7 +481,7 @@ int32_t streamProcessScanHistoryFinishReq(SStreamTask* pTask, SStreamScanHistory } } } else { - qDebug("s-task:%s receive scan-history data finish msg from upstream:0x%x(index:%d), unfinished:%d", + stDebug("s-task:%s receive scan-history data finish msg from upstream:0x%x(index:%d), unfinished:%d", pTask->id.idStr, pReq->upstreamTaskId, pReq->childId, left); } @@ -518,12 +518,12 @@ static void checkFillhistoryTaskStatus(SStreamTask* pTask, SStreamTask* pHTask) pHTask->dataRange.range.maxVer = pTask->chkInfo.nextProcessVer - 1; if (pTask->info.taskLevel == TASK_LEVEL__SOURCE) { - qDebug("s-task:%s set the launch condition for fill-history s-task:%s, window:%" PRId64 " - %" PRId64 + stDebug("s-task:%s set the launch condition for fill-history s-task:%s, window:%" PRId64 " - %" PRId64 " ver range:%" PRId64 " - %" PRId64, pTask->id.idStr, pHTask->id.idStr, pHTask->dataRange.window.skey, pHTask->dataRange.window.ekey, pHTask->dataRange.range.minVer, pHTask->dataRange.range.maxVer); } else { - qDebug("s-task:%s no fill history condition for non-source task:%s", pTask->id.idStr, pHTask->id.idStr); + stDebug("s-task:%s no fill history condition for non-source task:%s", pTask->id.idStr, pHTask->id.idStr); } // check if downstream tasks have been ready @@ -534,7 +534,7 @@ static void tryLaunchHistoryTask(void* param, void* tmrId) { SStreamTaskRetryInfo* pInfo = param; SStreamMeta* pMeta = pInfo->pMeta; - qDebug("s-task:0x%x in timer to launch related history task", (int32_t) pInfo->id.taskId); + stDebug("s-task:0x%x in timer to launch related history task", (int32_t) pInfo->id.taskId); taosWLockLatch(&pMeta->lock); SStreamTask** ppTask = (SStreamTask**)taosHashGet(pMeta->pTasksMap, &pInfo->id, sizeof(pInfo->id)); @@ -543,7 +543,7 @@ static void tryLaunchHistoryTask(void* param, void* tmrId) { if (streamTaskShouldStop(&(*ppTask)->status)) { const char* pStatus = streamGetTaskStatusStr((*ppTask)->status.taskStatus); - qDebug("s-task:%s status:%s quit timer task", (*ppTask)->id.idStr, pStatus); + stDebug("s-task:%s status:%s quit timer task", (*ppTask)->id.idStr, pStatus); taosMemoryFree(pInfo); atomic_sub_fetch_8(&(*ppTask)->status.timerActive, 1); @@ -561,7 +561,7 @@ static void tryLaunchHistoryTask(void* param, void* tmrId) { SStreamTask* pHTask = streamMetaAcquireTask(pMeta, pTask->historyTaskId.streamId, pTask->historyTaskId.taskId); if (pHTask == NULL && (!streamTaskShouldStop(&pTask->status))) { const char* pStatus = streamGetTaskStatusStr(pTask->status.taskStatus); - qWarn( + stWarn( "s-task:%s vgId:%d status:%s failed to launch history task:0x%x, since it may not be built, or may have been " "destroyed, or should stop", pTask->id.idStr, pMeta->vgId, pStatus, (int32_t) pTask->historyTaskId.taskId); @@ -580,7 +580,7 @@ static void tryLaunchHistoryTask(void* param, void* tmrId) { atomic_sub_fetch_8(&pTask->status.timerActive, 1); streamMetaReleaseTask(pMeta, pTask); } else { - qError("s-task:0x%x failed to load task, it may have been destroyed", (int32_t) pInfo->id.taskId); + stError("s-task:0x%x failed to load task, it may have been destroyed", (int32_t) pInfo->id.taskId); } taosMemoryFree(pInfo); @@ -596,13 +596,13 @@ int32_t streamLaunchFillHistoryTask(SStreamTask* pTask) { } ASSERT(pTask->status.downstreamReady == 1); - qDebug("s-task:%s start to launch related fill-history task:0x%" PRIx64 "-0x%x", pTask->id.idStr, + stDebug("s-task:%s start to launch related fill-history task:0x%" PRIx64 "-0x%x", pTask->id.idStr, pTask->historyTaskId.streamId, hTaskId); // Set the execute conditions, including the query time window and the version range SStreamTask** pHTask = taosHashGet(pMeta->pTasksMap, &pTask->historyTaskId, sizeof(pTask->historyTaskId)); if (pHTask == NULL) { - qWarn("s-task:%s vgId:%d failed to launch history task:0x%x, since it is not built yet", pTask->id.idStr, + stWarn("s-task:%s vgId:%d failed to launch history task:0x%x, since it is not built yet", pTask->id.idStr, pMeta->vgId, hTaskId); SStreamTaskRetryInfo* pInfo = taosMemoryCalloc(1, sizeof(SStreamTaskRetryInfo)); @@ -618,11 +618,11 @@ int32_t streamLaunchFillHistoryTask(SStreamTask* pTask) { } else { int32_t ref = atomic_add_fetch_8(&pTask->status.timerActive, 1);// timer is active ASSERT(ref == 1); - qDebug("s-task:%s set timer active flag", pTask->id.idStr); + stDebug("s-task:%s set timer active flag", pTask->id.idStr); } } else { // timer exists ASSERT(pTask->status.timerActive == 1); - qDebug("s-task:%s set timer active flag, task timer not null", pTask->id.idStr); + stDebug("s-task:%s set timer active flag, task timer not null", pTask->id.idStr); taosTmrReset(tryLaunchHistoryTask, 100, pInfo, streamEnv.timer, &pTask->launchTaskTimer); } @@ -631,7 +631,7 @@ int32_t streamLaunchFillHistoryTask(SStreamTask* pTask) { } if ((*pHTask)->status.downstreamReady == 1) { - qDebug("s-task:%s fill-history task is ready, no need to check downstream", (*pHTask)->id.idStr); + stDebug("s-task:%s fill-history task is ready, no need to check downstream", (*pHTask)->id.idStr); } else { checkFillhistoryTaskStatus(pTask, *pHTask); } @@ -675,7 +675,7 @@ bool streamHistoryTaskSetVerRangeStep2(SStreamTask* pTask, int64_t latestVer) { if (nextStartVer > latestVer - 1) { // no input data yet. no need to execute the secondardy scan while stream task halt streamTaskFillHistoryFinished(pTask); - qDebug( + stDebug( "s-task:%s no need to perform secondary scan-history data(step 2), since no data ingest during step1 scan, " "related stream task currentVer:%" PRId64, pTask->id.idStr, latestVer); @@ -773,11 +773,11 @@ void streamTaskSetRangeStreamCalc(SStreamTask* pTask) { if (pTask->historyTaskId.taskId == 0) { SDataRange* pRange = &pTask->dataRange; if (pTask->info.fillHistory == 1) { - qDebug("s-task:%s fill-history task, time window:%" PRId64 "-%" PRId64 ", verRange:%" PRId64 + stDebug("s-task:%s fill-history task, time window:%" PRId64 "-%" PRId64 ", verRange:%" PRId64 "-%" PRId64, pTask->id.idStr, pRange->window.skey, pRange->window.ekey, pRange->range.minVer, pRange->range.maxVer); } else { - qDebug("s-task:%s no related fill-history task, stream time window:%" PRId64 "-%" PRId64 ", verRange:%" PRId64 + stDebug("s-task:%s no related fill-history task, stream time window:%" PRId64 "-%" PRId64 ", verRange:%" PRId64 "-%" PRId64, pTask->id.idStr, pRange->window.skey, pRange->window.ekey, pRange->range.minVer, pRange->range.maxVer); } @@ -798,7 +798,7 @@ void streamTaskSetRangeStreamCalc(SStreamTask* pTask) { pRange->range.minVer = 0; pRange->range.maxVer = ver; - qDebug("s-task:%s level:%d related fill-history task exists, update stream calc time window:%" PRId64 " - %" PRId64 + stDebug("s-task:%s level:%d related fill-history task exists, update stream calc time window:%" PRId64 " - %" PRId64 ", verRang:%" PRId64 " - %" PRId64, pTask->id.idStr, pTask->info.taskLevel, pRange->window.skey, pRange->window.ekey, pRange->range.minVer, pRange->range.maxVer); @@ -808,7 +808,7 @@ void streamTaskSetRangeStreamCalc(SStreamTask* pTask) { // only the downstream tasks are ready, set the task to be ready to work. void streamTaskCheckDownstream(SStreamTask* pTask) { if (pTask->info.fillHistory) { - qDebug("s-task:%s fill history task, wait for being launched", pTask->id.idStr); + stDebug("s-task:%s fill history task, wait for being launched", pTask->id.idStr); return; } @@ -822,36 +822,36 @@ void streamTaskPause(SStreamTask* pTask, SStreamMeta* pMeta) { int8_t status = pTask->status.taskStatus; if (status == TASK_STATUS__DROPPING) { - qDebug("vgId:%d s-task:%s task already dropped, do nothing", pMeta->vgId, pTask->id.idStr); + stDebug("vgId:%d s-task:%s task already dropped, do nothing", pMeta->vgId, pTask->id.idStr); return; } const char* str = streamGetTaskStatusStr(status); if (status == TASK_STATUS__STOP || status == TASK_STATUS__PAUSE) { - qDebug("vgId:%d s-task:%s task already stopped/paused, status:%s, do nothing", pMeta->vgId, pTask->id.idStr, str); + stDebug("vgId:%d s-task:%s task already stopped/paused, status:%s, do nothing", pMeta->vgId, pTask->id.idStr, str); return; } if(pTask->info.taskLevel == TASK_LEVEL__SINK) { int32_t num = atomic_add_fetch_32(&pMeta->numOfPausedTasks, 1); - qInfo("vgId:%d s-task:%s pause stream sink task. pause task num:%d", pMeta->vgId, pTask->id.idStr, num); + stInfo("vgId:%d s-task:%s pause stream sink task. pause task num:%d", pMeta->vgId, pTask->id.idStr, num); return; } while (!pTask->status.pauseAllowed || (pTask->status.taskStatus == TASK_STATUS__HALT)) { status = pTask->status.taskStatus; if (status == TASK_STATUS__DROPPING) { - qDebug("vgId:%d s-task:%s task already dropped, do nothing", pMeta->vgId, pTask->id.idStr); + stDebug("vgId:%d s-task:%s task already dropped, do nothing", pMeta->vgId, pTask->id.idStr); return; } if (status == TASK_STATUS__STOP || status == TASK_STATUS__PAUSE) { - qDebug("vgId:%d s-task:%s task already stopped/paused, status:%s, do nothing", pMeta->vgId, pTask->id.idStr, str); + stDebug("vgId:%d s-task:%s task already stopped/paused, status:%s, do nothing", pMeta->vgId, pTask->id.idStr, str); return; } const char* pStatus = streamGetTaskStatusStr(status); - qDebug("s-task:%s wait for the task can be paused, status:%s, vgId:%d", pTask->id.idStr, pStatus, pMeta->vgId); + stDebug("s-task:%s wait for the task can be paused, status:%s, vgId:%d", pTask->id.idStr, pStatus, pMeta->vgId); taosMsleep(100); } @@ -861,14 +861,14 @@ void streamTaskPause(SStreamTask* pTask, SStreamMeta* pMeta) { status = pTask->status.taskStatus; if (status == TASK_STATUS__DROPPING || status == TASK_STATUS__STOP) { taosWUnLockLatch(&pMeta->lock); - qDebug("vgId:%d s-task:%s task already dropped/stopped/paused, do nothing", pMeta->vgId, pTask->id.idStr); + stDebug("vgId:%d s-task:%s task already dropped/stopped/paused, do nothing", pMeta->vgId, pTask->id.idStr); return; } atomic_store_8(&pTask->status.keepTaskStatus, pTask->status.taskStatus); atomic_store_8(&pTask->status.taskStatus, TASK_STATUS__PAUSE); int32_t num = atomic_add_fetch_32(&pMeta->numOfPausedTasks, 1); - qInfo("vgId:%d s-task:%s pause stream task. pause task num:%d", pMeta->vgId, pTask->id.idStr, num); + stInfo("vgId:%d s-task:%s pause stream task. pause task num:%d", pMeta->vgId, pTask->id.idStr, num); taosWUnLockLatch(&pMeta->lock); // in case of fill-history task, stop the tsdb file scan operation. @@ -878,7 +878,7 @@ void streamTaskPause(SStreamTask* pTask, SStreamMeta* pMeta) { } int64_t el = taosGetTimestampMs() - st; - qDebug("vgId:%d s-task:%s set pause flag, prev:%s, pause elapsed time:%dms", pMeta->vgId, pTask->id.idStr, + stDebug("vgId:%d s-task:%s set pause flag, prev:%s, pause elapsed time:%dms", pMeta->vgId, pTask->id.idStr, streamGetTaskStatusStr(pTask->status.keepTaskStatus), (int32_t)el); } @@ -888,12 +888,12 @@ void streamTaskResume(SStreamTask* pTask, SStreamMeta* pMeta) { pTask->status.taskStatus = pTask->status.keepTaskStatus; pTask->status.keepTaskStatus = TASK_STATUS__NORMAL; int32_t num = atomic_sub_fetch_32(&pMeta->numOfPausedTasks, 1); - qInfo("vgId:%d s-task:%s resume from pause, status:%s. pause task num:%d", pMeta->vgId, pTask->id.idStr, streamGetTaskStatusStr(status), num); + stInfo("vgId:%d s-task:%s resume from pause, status:%s. pause task num:%d", pMeta->vgId, pTask->id.idStr, streamGetTaskStatusStr(status), num); } else if (pTask->info.taskLevel == TASK_LEVEL__SINK) { int32_t num = atomic_sub_fetch_32(&pMeta->numOfPausedTasks, 1); - qInfo("vgId:%d s-task:%s sink task.resume from pause, status:%s. pause task num:%d", pMeta->vgId, pTask->id.idStr, streamGetTaskStatusStr(status), num); + stInfo("vgId:%d s-task:%s sink task.resume from pause, status:%s. pause task num:%d", pMeta->vgId, pTask->id.idStr, streamGetTaskStatusStr(status), num); } else { - qError("s-task:%s not in pause, failed to resume, status:%s", pTask->id.idStr, streamGetTaskStatusStr(status)); + stError("s-task:%s not in pause, failed to resume, status:%s", pTask->id.idStr, streamGetTaskStatusStr(status)); } } @@ -902,19 +902,20 @@ void streamTaskDisablePause(SStreamTask* pTask) { // pre-condition check const char* id = pTask->id.idStr; while (pTask->status.taskStatus == TASK_STATUS__PAUSE) { - qDebug("s-task:%s already in pause, wait for pause being cancelled, and set pause disabled, recheck in 100ms", id); + stDebug("s-task:%s already in pause, wait for pause being cancelled, and set pause disabled, recheck in 100ms", id); taosMsleep(100); } - qDebug("s-task:%s disable task pause", id); + stDebug("s-task:%s disable task pause", id); pTask->status.pauseAllowed = 0; } void streamTaskEnablePause(SStreamTask* pTask) { - qDebug("s-task:%s enable task pause", pTask->id.idStr); + stDebug("s-task:%s enable task pause", pTask->id.idStr); pTask->status.pauseAllowed = 1; } +// fix: this function should be removed, it may cause deadlock. void streamTaskHalt(SStreamTask* pTask) { int8_t status = pTask->status.taskStatus; if (status == TASK_STATUS__DROPPING || status == TASK_STATUS__STOP) { @@ -934,10 +935,10 @@ void streamTaskHalt(SStreamTask* pTask) { // upgrade to halt status if (status == TASK_STATUS__PAUSE) { - qDebug("s-task:%s upgrade status to %s from %s", pTask->id.idStr, streamGetTaskStatusStr(TASK_STATUS__HALT), + stDebug("s-task:%s upgrade status to %s from %s", pTask->id.idStr, streamGetTaskStatusStr(TASK_STATUS__HALT), streamGetTaskStatusStr(TASK_STATUS__PAUSE)); } else { - qDebug("s-task:%s halt task", pTask->id.idStr); + stDebug("s-task:%s halt task", pTask->id.idStr); } pTask->status.keepTaskStatus = status; @@ -948,11 +949,11 @@ void streamTaskResumeFromHalt(SStreamTask* pTask) { const char* id = pTask->id.idStr; int8_t status = pTask->status.taskStatus; if (status != TASK_STATUS__HALT) { - qError("s-task:%s not in halt status, status:%s", id, streamGetTaskStatusStr(status)); + stError("s-task:%s not in halt status, status:%s", id, streamGetTaskStatusStr(status)); return; } pTask->status.taskStatus = pTask->status.keepTaskStatus; pTask->status.keepTaskStatus = TASK_STATUS__NORMAL; - qDebug("s-task:%s resume from halt, current status:%s", id, streamGetTaskStatusStr(pTask->status.taskStatus)); + stDebug("s-task:%s resume from halt, current status:%s", id, streamGetTaskStatusStr(pTask->status.taskStatus)); } diff --git a/source/libs/stream/src/streamSnapshot.c b/source/libs/stream/src/streamSnapshot.c index 8a4500dd86..2d13f300c9 100644 --- a/source/libs/stream/src/streamSnapshot.c +++ b/source/libs/stream/src/streamSnapshot.c @@ -18,6 +18,7 @@ #include "rocksdb/c.h" #include "streamBackendRocksdb.h" #include "tcommon.h" +#include "streamInt.h" enum SBackendFileType { ROCKSDB_OPTIONS_TYPE = 1, @@ -122,10 +123,10 @@ int32_t streamSnapHandleInit(SStreamSnapHandle* pHandle, char* path, int64_t chk chkpId); if (taosIsDir(tdir)) { validChkp = 1; - qInfo("%s start to read snap %s", STREAM_STATE_TRANSFER, tdir); + stInfo("%s start to read snap %s", STREAM_STATE_TRANSFER, tdir); streamBackendAddInUseChkp(pMeta, chkpId); } else { - qWarn("%s failed to read from %s, reason: dir not exist,retry to default state dir", STREAM_STATE_TRANSFER, tdir); + stWarn("%s failed to read from %s, reason: dir not exist,retry to default state dir", STREAM_STATE_TRANSFER, tdir); } } @@ -137,22 +138,22 @@ int32_t streamSnapHandleInit(SStreamSnapHandle* pHandle, char* path, int64_t chk taosMemoryFree(tdir); tdir = chkpdir; - qInfo("%s start to trigger checkpoint on %s", STREAM_STATE_TRANSFER, tdir); + stInfo("%s start to trigger checkpoint on %s", STREAM_STATE_TRANSFER, tdir); code = streamBackendTriggerChkp(pMeta, tdir); if (code != 0) { - qError("%s failed to trigger chekckpoint at %s", STREAM_STATE_TRANSFER, tdir); + stError("%s failed to trigger chekckpoint at %s", STREAM_STATE_TRANSFER, tdir); taosMemoryFree(tdir); return code; } chkpId = 0; } - qInfo("%s start to read dir: %s", STREAM_STATE_TRANSFER, tdir); + stInfo("%s start to read dir: %s", STREAM_STATE_TRANSFER, tdir); TdDirPtr pDir = taosOpenDir(tdir); if (NULL == pDir) { - qError("%s failed to open %s", STREAM_STATE_TRANSFER, tdir); + stError("%s failed to open %s", STREAM_STATE_TRANSFER, tdir); goto _err; } @@ -202,14 +203,14 @@ int32_t streamSnapHandleInit(SStreamSnapHandle* pHandle, char* path, int64_t chk } sprintf(buf + strlen(buf) - 1, "]"); - qInfo("%s get file list: %s", STREAM_STATE_TRANSFER, buf); + stInfo("%s get file list: %s", STREAM_STATE_TRANSFER, buf); taosMemoryFree(buf); } taosCloseDir(&pDir); if (pFile->pCurrent == NULL) { - qError("%s failed to open %s, reason: no valid file", STREAM_STATE_TRANSFER, tdir); + stError("%s failed to open %s, reason: no valid file", STREAM_STATE_TRANSFER, tdir); code = -1; tdir = NULL; goto _err; @@ -333,23 +334,23 @@ int32_t streamSnapRead(SStreamSnapReader* pReader, uint8_t** ppData, int64_t* si return 0; } else { pHandle->fd = streamOpenFile(pFile->path, item->name, TD_FILE_READ); - qDebug("%s open file %s, current offset:%" PRId64 ", size:% " PRId64 ", file no.%d", STREAM_STATE_TRANSFER, + stDebug("%s open file %s, current offset:%" PRId64 ", size:% " PRId64 ", file no.%d", STREAM_STATE_TRANSFER, item->name, (int64_t)pHandle->offset, item->size, pHandle->currFileIdx); } } - qDebug("%s start to read file %s, current offset:%" PRId64 ", size:%" PRId64 ", file no.%d", STREAM_STATE_TRANSFER, + stDebug("%s start to read file %s, current offset:%" PRId64 ", size:%" PRId64 ", file no.%d", STREAM_STATE_TRANSFER, item->name, (int64_t)pHandle->offset, item->size, pHandle->currFileIdx); uint8_t* buf = taosMemoryCalloc(1, sizeof(SStreamSnapBlockHdr) + kBlockSize); int64_t nread = taosPReadFile(pHandle->fd, buf + sizeof(SStreamSnapBlockHdr), kBlockSize, pHandle->offset); if (nread == -1) { code = TAOS_SYSTEM_ERROR(terrno); - qError("%s snap failed to read snap, file name:%s, type:%d,reason:%s", STREAM_STATE_TRANSFER, item->name, + stError("%s snap failed to read snap, file name:%s, type:%d,reason:%s", STREAM_STATE_TRANSFER, item->name, item->type, tstrerror(code)); return -1; } else if (nread > 0 && nread <= kBlockSize) { // left bytes less than kBlockSize - qDebug("%s read file %s, current offset:%" PRId64 ",size:% " PRId64 ", file no.%d", STREAM_STATE_TRANSFER, + stDebug("%s read file %s, current offset:%" PRId64 ",size:% " PRId64 ", file no.%d", STREAM_STATE_TRANSFER, item->name, (int64_t)pHandle->offset, item->size, pHandle->currFileIdx); pHandle->offset += nread; if (pHandle->offset >= item->size || nread < kBlockSize) { @@ -358,7 +359,7 @@ int32_t streamSnapRead(SStreamSnapReader* pReader, uint8_t** ppData, int64_t* si pHandle->currFileIdx += 1; } } else { - qDebug("%s no data read, close file no.%d, move to next file, open and read", STREAM_STATE_TRANSFER, + stDebug("%s no data read, close file no.%d, move to next file, open and read", STREAM_STATE_TRANSFER, pHandle->currFileIdx); taosCloseFile(&pHandle->fd); pHandle->offset = 0; @@ -376,7 +377,7 @@ int32_t streamSnapRead(SStreamSnapReader* pReader, uint8_t** ppData, int64_t* si nread = taosPReadFile(pHandle->fd, buf + sizeof(SStreamSnapBlockHdr), kBlockSize, pHandle->offset); pHandle->offset += nread; - qDebug("%s open file and read file %s, current offset:%" PRId64 ", size:% " PRId64 ", file no.%d", + stDebug("%s open file and read file %s, current offset:%" PRId64 ", size:% " PRId64 ", file no.%d", STREAM_STATE_TRANSFER, item->name, (int64_t)pHandle->offset, item->size, pHandle->currFileIdx); } @@ -432,7 +433,7 @@ int32_t streamSnapWrite(SStreamSnapWriter* pWriter, uint8_t* pData, uint32_t nDa pHandle->fd = streamOpenFile(pFile->path, pItem->name, TD_FILE_CREATE | TD_FILE_WRITE | TD_FILE_APPEND); if (pHandle->fd == NULL) { code = TAOS_SYSTEM_ERROR(terrno); - qError("%s failed to open file name:%s%s%s, reason:%s", STREAM_STATE_TRANSFER, pFile->path, TD_DIRSEP, pHdr->name, + stError("%s failed to open file name:%s%s%s, reason:%s", STREAM_STATE_TRANSFER, pFile->path, TD_DIRSEP, pHdr->name, tstrerror(code)); } } @@ -441,7 +442,7 @@ int32_t streamSnapWrite(SStreamSnapWriter* pWriter, uint8_t* pData, uint32_t nDa int64_t bytes = taosPWriteFile(pHandle->fd, pHdr->data, pHdr->size, pHandle->offset); if (bytes != pHdr->size) { code = TAOS_SYSTEM_ERROR(terrno); - qError("%s failed to write snap, file name:%s, reason:%s", STREAM_STATE_TRANSFER, pHdr->name, tstrerror(code)); + stError("%s failed to write snap, file name:%s, reason:%s", STREAM_STATE_TRANSFER, pHdr->name, tstrerror(code)); return code; } pHandle->offset += bytes; @@ -459,7 +460,7 @@ int32_t streamSnapWrite(SStreamSnapWriter* pWriter, uint8_t* pData, uint32_t nDa pHandle->fd = streamOpenFile(pFile->path, pItem->name, TD_FILE_CREATE | TD_FILE_WRITE | TD_FILE_APPEND); if (pHandle->fd == NULL) { code = TAOS_SYSTEM_ERROR(terrno); - qError("%s failed to open file name:%s%s%s, reason:%s", STREAM_STATE_TRANSFER, pFile->path, TD_DIRSEP, pHdr->name, + stError("%s failed to open file name:%s%s%s, reason:%s", STREAM_STATE_TRANSFER, pFile->path, TD_DIRSEP, pHdr->name, tstrerror(code)); } @@ -483,7 +484,7 @@ int32_t streamSnapWriterClose(SStreamSnapWriter* pWriter, int8_t rollback) { n += sprintf(buf + n, "%s %" PRId64 "]", item->name, item->size); } } - qDebug("%s snap get file list, %s", STREAM_STATE_TRANSFER, buf); + stDebug("%s snap get file list, %s", STREAM_STATE_TRANSFER, buf); taosMemoryFree(buf); } diff --git a/source/libs/stream/src/streamState.c b/source/libs/stream/src/streamState.c index 44c7b4f2e0..3dfe287ddc 100644 --- a/source/libs/stream/src/streamState.c +++ b/source/libs/stream/src/streamState.c @@ -91,7 +91,7 @@ int stateKeyCmpr(const void* pKey1, int kLen1, const void* pKey2, int kLen2) { } SStreamState* streamStateOpen(char* path, void* pTask, bool specPath, int32_t szPage, int32_t pages) { - qDebug("open stream state, %s", path); + stDebug("open stream state, %s", path); SStreamState* pState = taosMemoryCalloc(1, sizeof(SStreamState)); if (pState == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; @@ -139,7 +139,7 @@ SStreamState* streamStateOpen(char* path, void* pTask, bool specPath, int32_t sz pState->pTdbState->backendCfWrapperId = id; pState->pTdbState->pBackendCfWrapper = taosAcquireRef(streamBackendCfWrapperId, id); // already exist stream task for - qInfo("already exist stream-state for %s", pState->pTdbState->idstr); + stInfo("already exist stream-state for %s", pState->pTdbState->idstr); // taosAcquireRef(streamBackendId, pState->streamBackendRid); } taosThreadMutexUnlock(&pMeta->backendMutex); @@ -149,7 +149,7 @@ SStreamState* streamStateOpen(char* path, void* pTask, bool specPath, int32_t sz _hash_fn_t hashFn = taosGetDefaultHashFunction(TSDB_DATA_TYPE_BIGINT); pState->parNameMap = tSimpleHashInit(1024, hashFn); - qInfo("succ to open state %p on backend %p 0x%" PRIx64 "-%d", pState, pMeta->streamBackend, pState->streamId, + stInfo("succ to open state %p on backend %p 0x%" PRIx64 "-%d", pState, pMeta->streamBackend, pState->streamId, pState->taskId); return pState; @@ -468,7 +468,7 @@ int32_t streamStateAddIfNotExist(SStreamState* pState, const SWinKey* key, void* int32_t streamStateReleaseBuf(SStreamState* pState, const SWinKey* key, void* pVal) { // todo refactor - qDebug("streamStateReleaseBuf"); + stDebug("streamStateReleaseBuf"); if (!pVal) { return 0; } @@ -717,7 +717,7 @@ void streamStateFreeCur(SStreamStateCur* pCur) { if (!pCur) { return; } - qDebug("streamStateFreeCur"); + stDebug("streamStateFreeCur"); rocksdb_iter_destroy(pCur->iter); if (pCur->snapshot) rocksdb_release_snapshot(pCur->db, pCur->snapshot); rocksdb_readoptions_destroy(pCur->readOpt); @@ -736,7 +736,7 @@ void streamStateFreeVal(void* val) { int32_t streamStateSessionPut(SStreamState* pState, const SSessionKey* key, const void* value, int32_t vLen) { #ifdef USE_ROCKSDB - qDebug("===stream===save skey:%" PRId64 ", ekey:%" PRId64 ", groupId:%" PRIu64, key->win.skey, key->win.ekey, + stDebug("===stream===save skey:%" PRId64 ", ekey:%" PRId64 ", groupId:%" PRIu64, key->win.skey, key->win.ekey, key->groupId); return streamStateSessionPut_rocksdb(pState, key, value, vLen); #else @@ -771,7 +771,7 @@ int32_t streamStateSessionGet(SStreamState* pState, SSessionKey* key, void** pVa int32_t streamStateSessionDel(SStreamState* pState, const SSessionKey* key) { #ifdef USE_ROCKSDB - qDebug("===stream===delete skey:%" PRId64 ", ekey:%" PRId64 ", groupId:%" PRIu64, key->win.skey, key->win.ekey, + stDebug("===stream===delete skey:%" PRId64 ", ekey:%" PRId64 ", groupId:%" PRIu64, key->win.skey, key->win.ekey, key->groupId); return streamStateSessionDel_rocksdb(pState, key); #else @@ -1088,7 +1088,7 @@ _end: } int32_t streamStatePutParName(SStreamState* pState, int64_t groupId, const char tbname[TSDB_TABLE_NAME_LEN]) { - qDebug("try to write to cf parname"); + stDebug("try to write to cf parname"); #ifdef USE_ROCKSDB if (tSimpleHashGetSize(pState->parNameMap) > MAX_TABLE_NAME_NUM) { if (tSimpleHashGet(pState->parNameMap, &groupId, sizeof(int64_t)) == NULL) { diff --git a/source/libs/stream/src/streamTask.c b/source/libs/stream/src/streamTask.c index d2e306fa01..c328ff4bbc 100644 --- a/source/libs/stream/src/streamTask.c +++ b/source/libs/stream/src/streamTask.c @@ -291,10 +291,10 @@ void tFreeStreamTask(SStreamTask* pTask) { STaskExecStatisInfo* pStatis = &pTask->taskExecInfo; - qDebug("start to free s-task:0x%x, %p, state:%p, status:%s", taskId, pTask, pTask->pState, + stDebug("start to free s-task:0x%x, %p, state:%p, status:%s", taskId, pTask, pTask->pState, streamGetTaskStatusStr(pTask->status.taskStatus)); - qDebug("s-task:0x%x exec info: create:%" PRId64 ", init:%" PRId64 ", start:%" PRId64 + stDebug("s-task:0x%x exec info: create:%" PRId64 ", init:%" PRId64 ", start:%" PRId64 ", updateCount:%d latestUpdate:%" PRId64 ", latestCheckPoint:%" PRId64 ", ver:%" PRId64 " nextProcessVer:%" PRId64, taskId, pStatis->created, pStatis->init, pStatis->start, pStatis->updateCount, pStatis->latestUpdateTs, @@ -306,7 +306,7 @@ void tFreeStreamTask(SStreamTask* pTask) { // remove the ref by timer while (pTask->status.timerActive > 0) { - qDebug("s-task:%s wait for task stop timer activities", pTask->id.idStr); + stDebug("s-task:%s wait for task stop timer activities", pTask->id.idStr); taosMsleep(10); } @@ -352,7 +352,7 @@ void tFreeStreamTask(SStreamTask* pTask) { } if (pTask->pState) { - qDebug("s-task:0x%x start to free task state", taskId); + stDebug("s-task:0x%x start to free task state", taskId); streamStateClose(pTask->pState, status == TASK_STATUS__DROPPING); } @@ -384,7 +384,7 @@ void tFreeStreamTask(SStreamTask* pTask) { taosThreadMutexDestroy(&pTask->lock); taosMemoryFree(pTask); - qDebug("s-task:0x%x free task completed", taskId); + stDebug("s-task:0x%x free task completed", taskId); } int32_t streamTaskInit(SStreamTask* pTask, SStreamMeta* pMeta, SMsgCb* pMsgCb, int64_t ver) { @@ -396,7 +396,7 @@ int32_t streamTaskInit(SStreamTask* pTask, SStreamMeta* pMeta, SMsgCb* pMsgCb, i pTask->outputInfo.queue = streamQueueOpen(512 << 10); if (pTask->inputInfo.queue == NULL || pTask->outputInfo.queue == NULL) { - qError("s-task:%s failed to prepare the input/output queue, initialize task failed", pTask->id.idStr); + stError("s-task:%s failed to prepare the input/output queue, initialize task failed", pTask->id.idStr); return TSDB_CODE_OUT_OF_MEMORY; } @@ -412,7 +412,7 @@ int32_t streamTaskInit(SStreamTask* pTask, SStreamMeta* pMeta, SMsgCb* pMsgCb, i pTask->pTokenBucket = taosMemoryCalloc(1, sizeof(STokenBucket)); if (pTask->pTokenBucket == NULL) { - qError("s-task:%s failed to prepare the tokenBucket, code:%s", pTask->id.idStr, tstrerror(TSDB_CODE_OUT_OF_MEMORY)); + stError("s-task:%s failed to prepare the tokenBucket, code:%s", pTask->id.idStr, tstrerror(TSDB_CODE_OUT_OF_MEMORY)); return TSDB_CODE_OUT_OF_MEMORY; } @@ -421,13 +421,13 @@ int32_t streamTaskInit(SStreamTask* pTask, SStreamMeta* pMeta, SMsgCb* pMsgCb, i TdThreadMutexAttr attr = {0}; int code = taosThreadMutexAttrInit(&attr); if (code != 0) { - qError("s-task:%s initElapsed mutex attr failed, code:%s", pTask->id.idStr, tstrerror(code)); + stError("s-task:%s initElapsed mutex attr failed, code:%s", pTask->id.idStr, tstrerror(code)); return code; } code = taosThreadMutexAttrSetType(&attr, PTHREAD_MUTEX_RECURSIVE); if (code != 0) { - qError("s-task:%s set mutex attr recursive, code:%s", pTask->id.idStr, tstrerror(code)); + stError("s-task:%s set mutex attr recursive, code:%s", pTask->id.idStr, tstrerror(code)); return code; } @@ -490,7 +490,7 @@ void streamTaskUpdateUpstreamInfo(SStreamTask* pTask, int32_t nodeId, const SEpS SStreamChildEpInfo* pInfo = taosArrayGetP(pTask->pUpstreamInfoList, i); if (pInfo->nodeId == nodeId) { epsetAssign(&pInfo->epSet, pEpSet); - qDebug("s-task:0x%x update the upstreamInfo taskId:0x%x(nodeId:%d) newEpset:%s", pTask->id.taskId, + stDebug("s-task:0x%x update the upstreamInfo taskId:0x%x(nodeId:%d) newEpset:%s", pTask->id.taskId, pInfo->taskId, nodeId, buf); break; } @@ -521,7 +521,7 @@ void streamTaskUpdateDownstreamInfo(SStreamTask* pTask, int32_t nodeId, const SE if (pVgInfo->vgId == nodeId) { epsetAssign(&pVgInfo->epSet, pEpSet); - qDebug("s-task:0x%x update the dispatch info, task:0x%x(nodeId:%d) newEpset:%s", pTask->id.taskId, + stDebug("s-task:0x%x update the dispatch info, task:0x%x(nodeId:%d) newEpset:%s", pTask->id.taskId, pVgInfo->taskId, nodeId, buf); break; } @@ -530,7 +530,7 @@ void streamTaskUpdateDownstreamInfo(SStreamTask* pTask, int32_t nodeId, const SE STaskDispatcherFixedEp* pDispatcher = &pTask->fixedEpDispatcher; if (pDispatcher->nodeId == nodeId) { epsetAssign(&pDispatcher->epSet, pEpSet); - qDebug("s-task:0x%x update the dispatch info, task:0x%x(nodeId:%d) newEpSet:%s", pTask->id.taskId, + stDebug("s-task:0x%x update the dispatch info, task:0x%x(nodeId:%d) newEpSet:%s", pTask->id.taskId, pDispatcher->taskId, nodeId, buf); } } else { @@ -547,12 +547,12 @@ int32_t streamTaskStop(SStreamTask* pTask) { qKillTask(pTask->exec.pExecutor, TSDB_CODE_SUCCESS); while (/*pTask->status.schedStatus != TASK_SCHED_STATUS__INACTIVE */ !streamTaskIsIdle(pTask)) { - qDebug("s-task:%s level:%d wait for task to be idle, check again in 100ms", id, pTask->info.taskLevel); + stDebug("s-task:%s level:%d wait for task to be idle, check again in 100ms", id, pTask->info.taskLevel); taosMsleep(100); } int64_t el = taosGetTimestampMs() - st; - qDebug("vgId:%d s-task:%s is closed in %" PRId64 " ms", pMeta->vgId, pTask->id.idStr, el); + stDebug("vgId:%d s-task:%s is closed in %" PRId64 " ms", pMeta->vgId, pTask->id.idStr, el); return 0; } @@ -562,7 +562,7 @@ int32_t doUpdateTaskEpset(SStreamTask* pTask, int32_t nodeId, SEpSet* pEpSet) { if (pTask->info.nodeId == nodeId) { // execution task should be moved away epsetAssign(&pTask->info.epSet, pEpSet); EPSET_TO_STR(pEpSet, buf) - qDebug("s-task:0x%x (vgId:%d) self node epset is updated %s", pTask->id.taskId, nodeId, buf); + stDebug("s-task:0x%x (vgId:%d) self node epset is updated %s", pTask->id.taskId, nodeId, buf); } // check for the dispath info and the upstream task info @@ -587,7 +587,7 @@ int32_t streamTaskUpdateEpsetInfo(SStreamTask* pTask, SArray* pNodeList) { p->latestUpdateTs = taosGetTimestampMs(); p->updateCount += 1; - qDebug("s-task:%s update task nodeEp epset, updatedNodes:%d, updateCount:%d, prevTs:%" PRId64, pTask->id.idStr, + stDebug("s-task:%s update task nodeEp epset, updatedNodes:%d, updateCount:%d, prevTs:%" PRId64, pTask->id.idStr, numOfNodes, p->updateCount, prevTs); for (int32_t i = 0; i < taosArrayGetSize(pNodeList); ++i) { @@ -608,7 +608,7 @@ void streamTaskResetUpstreamStageInfo(SStreamTask* pTask) { pInfo->stage = -1; } - qDebug("s-task:%s reset all upstream tasks stage info", pTask->id.idStr); + stDebug("s-task:%s reset all upstream tasks stage info", pTask->id.idStr); } int8_t streamTaskSetSchedStatusWait(SStreamTask* pTask) { @@ -658,11 +658,11 @@ int32_t streamBuildAndSendDropTaskMsg(SMsgCb* pMsgCb, int32_t vgId, SStreamTaskI SRpcMsg msg = {.msgType = TDMT_STREAM_TASK_DROP, .pCont = pReq, .contLen = sizeof(SVDropStreamTaskReq)}; int32_t code = tmsgPutToQueue(pMsgCb, WRITE_QUEUE, &msg); if (code != TSDB_CODE_SUCCESS) { - qError("vgId:%d failed to send drop task:0x%x msg, code:%s", vgId, pTaskId->taskId, tstrerror(code)); + stError("vgId:%d failed to send drop task:0x%x msg, code:%s", vgId, pTaskId->taskId, tstrerror(code)); return code; } - qDebug("vgId:%d build and send drop table:0x%x msg", vgId, pTaskId->taskId); + stDebug("vgId:%d build and send drop table:0x%x msg", vgId, pTaskId->taskId); return code; } diff --git a/source/util/src/tlog.c b/source/util/src/tlog.c index a431b091ec..b858421e25 100644 --- a/source/util/src/tlog.c +++ b/source/util/src/tlog.c @@ -100,6 +100,7 @@ int32_t tmrDebugFlag = 131; int32_t uDebugFlag = 131; int32_t rpcDebugFlag = 131; int32_t qDebugFlag = 131; +int32_t stDebugFlag = 131; int32_t wDebugFlag = 131; int32_t sDebugFlag = 131; int32_t tsdbDebugFlag = 131; From a3d13d66340b2e2cabae357f73ed95245d880ae3 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 20 Sep 2023 19:19:04 +0800 Subject: [PATCH 048/158] fix(stream): remove invalid cached task info in buffer. --- source/dnode/mnode/impl/src/mndStream.c | 52 +++++++++++++++++++++++ source/libs/stream/src/streamCheckpoint.c | 1 + 2 files changed, 53 insertions(+) diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index 05cc043802..a2766843b8 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -2179,6 +2179,56 @@ static void doExtractTasksFromStream(SMnode *pMnode) { } } +static int32_t doRemoveFromTask(SStreamExecNodeInfo* pExecNode, STaskId* pRemovedId) { + void *p = taosHashGet(pExecNode->pTaskMap, pRemovedId, sizeof(*pRemovedId)); + + if (p != NULL) { + taosHashRemove(pExecNode->pTaskMap, pRemovedId, sizeof(*pRemovedId)); + + for(int32_t k = 0; k < taosArrayGetSize(pExecNode->pTaskList); ++k) { + STaskId* pId = taosArrayGet(pExecNode->pTaskList, k); + if (pId->taskId == pRemovedId->taskId && pId->streamId == pRemovedId->streamId) { + taosArrayRemove(pExecNode->pTaskList, k); + mInfo("s-task:0x%x removed from buffer, remain:%d", (int32_t) pRemovedId->taskId, + (int32_t)taosArrayGetSize(pExecNode->pTaskList)); + break; + } + } + } + return 0; +} + +static int32_t removeInvalidStreamTask(SArray *pNodeSnapshot) { + SArray* pRemoveTaskList = taosArrayInit(4, sizeof(STaskId)); + + int32_t numOfTask = taosArrayGetSize(execNodeList.pTaskList); + int32_t numOfVgroups = taosArrayGetSize(pNodeSnapshot); + for(int32_t i = 0; i < numOfTask; ++i) { + STaskId* pId = taosArrayGet(execNodeList.pTaskList, i); + STaskStatusEntry* pEntry = taosHashGet(execNodeList.pTaskMap, pId, sizeof(*pId)); + + bool existed = false; + for(int32_t j = 0; j < numOfVgroups; ++j) { + SNodeEntry* pNodeEntry = taosArrayGet(pNodeSnapshot, j); + if (pNodeEntry->nodeId == pEntry->nodeId) { + existed = true; + break; + } + } + + if (!existed) { + taosArrayPush(pRemoveTaskList, pId); + } + } + + for(int32_t i = 0; i < taosArrayGetSize(pRemoveTaskList); ++i) { + STaskId* pId = taosArrayGet(pRemoveTaskList, i); + doRemoveFromTask(&execNodeList, pId); + } + + return 0; +} + // this function runs by only one thread, so it is not multi-thread safe static int32_t mndProcessNodeCheckReq(SRpcMsg *pMsg) { int32_t code = 0; @@ -2210,6 +2260,8 @@ static int32_t mndProcessNodeCheckReq(SRpcMsg *pMsg) { SArray *pNodeSnapshot = mndTakeVgroupSnapshot(pMnode); taosThreadMutexLock(&execNodeList.lock); + removeInvalidStreamTask(pNodeSnapshot); + SVgroupChangeInfo changeInfo = mndFindChangedNodeInfo(pMnode, execNodeList.pNodeEntryList, pNodeSnapshot); if (taosArrayGetSize(changeInfo.pUpdateNodeList) > 0) { code = mndProcessVgroupChange(pMnode, &changeInfo); diff --git a/source/libs/stream/src/streamCheckpoint.c b/source/libs/stream/src/streamCheckpoint.c index 77b7d0e614..b3f749d7b9 100644 --- a/source/libs/stream/src/streamCheckpoint.c +++ b/source/libs/stream/src/streamCheckpoint.c @@ -155,6 +155,7 @@ static int32_t continueDispatchCheckpointBlock(SStreamDataBlock* pBlock, SStream if (code == 0) { streamDispatchStreamBlock(pTask); } else { + stError("s-task:%s failed to put checkpoint into outputQ, code:%s", pTask->id.idStr, tstrerror(code)); streamFreeQitem((SStreamQueueItem*)pBlock); } From 9a559b209a4502d19822ed6d32a901958f8b5dd0 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 20 Sep 2023 23:18:06 +0800 Subject: [PATCH 049/158] fix(stream): add logs. --- source/common/src/tglobal.c | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/source/common/src/tglobal.c b/source/common/src/tglobal.c index 80184784f3..85b1028bb7 100644 --- a/source/common/src/tglobal.c +++ b/source/common/src/tglobal.c @@ -847,6 +847,7 @@ static void taosSetServerLogCfg(SConfig *pCfg) { idxDebugFlag = cfgGetItem(pCfg, "idxDebugFlag")->i32; tdbDebugFlag = cfgGetItem(pCfg, "tdbDebugFlag")->i32; metaDebugFlag = cfgGetItem(pCfg, "metaDebugFlag")->i32; + stDebugFlag = cfgGetItem(pCfg, "stDebugFlag")->i32; } static int32_t taosSetSlowLogScope(char *pScope) { @@ -1675,12 +1676,12 @@ void taosCfgDynamicOptions(const char *option, const char *value) { const char *options[] = { "dDebugFlag", "vDebugFlag", "mDebugFlag", "wDebugFlag", "sDebugFlag", "tsdbDebugFlag", "tqDebugFlag", "fsDebugFlag", "udfDebugFlag", "smaDebugFlag", "idxDebugFlag", "tdbDebugFlag", "tmrDebugFlag", "uDebugFlag", - "smaDebugFlag", "rpcDebugFlag", "qDebugFlag", "metaDebugFlag", "jniDebugFlag", + "smaDebugFlag", "rpcDebugFlag", "qDebugFlag", "metaDebugFlag", "jniDebugFlag", "stDebugFlag", }; int32_t *optionVars[] = { &dDebugFlag, &vDebugFlag, &mDebugFlag, &wDebugFlag, &sDebugFlag, &tsdbDebugFlag, &tqDebugFlag, &fsDebugFlag, &udfDebugFlag, &smaDebugFlag, &idxDebugFlag, &tdbDebugFlag, &tmrDebugFlag, &uDebugFlag, - &smaDebugFlag, &rpcDebugFlag, &qDebugFlag, &metaDebugFlag, &jniDebugFlag, + &smaDebugFlag, &rpcDebugFlag, &qDebugFlag, &metaDebugFlag, &jniDebugFlag, &stDebugFlag, }; int32_t optionSize = tListLen(options); @@ -1733,6 +1734,7 @@ void taosSetAllDebugFlag(int32_t flag, bool rewrite) { taosSetDebugFlag(&idxDebugFlag, "idxDebugFlag", flag, rewrite); taosSetDebugFlag(&tdbDebugFlag, "tdbDebugFlag", flag, rewrite); taosSetDebugFlag(&metaDebugFlag, "metaDebugFlag", flag, rewrite); + taosSetDebugFlag(&stDebugFlag, "stDebugFlag", flag, rewrite); uInfo("all debug flag are set to %d", flag); } From e4ac83555495ae1d0bc22f17afb26dc83fce9c87 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 21 Sep 2023 01:19:58 +0800 Subject: [PATCH 050/158] fix(stream): disable token bucket. --- source/libs/stream/src/streamCheckpoint.c | 1 + source/libs/stream/src/streamDispatch.c | 2 +- source/libs/stream/src/streamExec.c | 2 +- source/libs/stream/src/streamQueue.c | 6 +++--- 4 files changed, 6 insertions(+), 5 deletions(-) diff --git a/source/libs/stream/src/streamCheckpoint.c b/source/libs/stream/src/streamCheckpoint.c index b3f749d7b9..ae4bc5366d 100644 --- a/source/libs/stream/src/streamCheckpoint.c +++ b/source/libs/stream/src/streamCheckpoint.c @@ -199,6 +199,7 @@ int32_t streamProcessCheckpointBlock(SStreamTask* pTask, SStreamDataBlock* pBloc } else if (taskLevel == TASK_LEVEL__SINK || taskLevel == TASK_LEVEL__AGG) { ASSERT(taosArrayGetSize(pTask->pUpstreamInfoList) > 0); + // todo: this may be not the first one pTask->chkInfo.startTs = taosGetTimestampMs(); // update the child Id for downstream tasks diff --git a/source/libs/stream/src/streamDispatch.c b/source/libs/stream/src/streamDispatch.c index 7906f1d6f4..8219f6ec85 100644 --- a/source/libs/stream/src/streamDispatch.c +++ b/source/libs/stream/src/streamDispatch.c @@ -942,7 +942,7 @@ int32_t streamNotifyUpstreamContinue(SStreamTask* pTask) { } taosArrayClear(pTask->pRspMsgList); - stDebug("s-task:%s level:%d checkpoint ready msg sent to all %d upstreams", pTask->id.idStr, pTask->info.taskLevel, + stDebug("s-task:%s level:%d continue process msg sent to all %d upstreams", pTask->id.idStr, pTask->info.taskLevel, num); return 0; } diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index 3109bf5967..57f8bd016b 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -580,7 +580,7 @@ int32_t streamExecForAll(SStreamTask* pTask) { ASSERT(pTask->chkInfo.checkpointVer <= pTask->chkInfo.nextProcessVer && ver >= pTask->chkInfo.checkpointVer); if (ver != pTask->chkInfo.checkpointVer) { - stDebug("s-task:%s update checkpointVer(unsaved) from %" PRId64 " to %" PRId64 " , currentVer:%" PRId64, + stDebug("s-task:%s update checkpointVer(unsaved) from %" PRId64 " to %" PRId64 ", nextProcessVer:%" PRId64, pTask->id.idStr, pTask->chkInfo.checkpointVer, ver, pTask->chkInfo.nextProcessVer); pTask->chkInfo.checkpointVer = ver; } diff --git a/source/libs/stream/src/streamQueue.c b/source/libs/stream/src/streamQueue.c index 901484377a..65a7d56923 100644 --- a/source/libs/stream/src/streamQueue.c +++ b/source/libs/stream/src/streamQueue.c @@ -191,11 +191,11 @@ int32_t streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueItem** pInpu } STokenBucket* pBucket = pTask->pTokenBucket; - if (!streamTaskHasAvailableToken(pBucket)) { // no available token in th bucket, ignore this execution +// if (!streamTaskHasAvailableToken(pBucket)) { // no available token in th bucket, ignore this execution // stInfo("s-task:%s no available token for sink, capacity:%d, rate:%d token/sec, quit", pTask->id.idStr, // pBucket->capacity, pBucket->rate); - return TSDB_CODE_SUCCESS; - } +// return TSDB_CODE_SUCCESS; +// } SStreamQueueItem* qItem = streamQueueNextItem(pTask->inputInfo.queue); if (qItem == NULL) { From ef9efc8a8e8899e87f8c75ae107d5d4417648ab5 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 21 Sep 2023 13:32:57 +0800 Subject: [PATCH 051/158] refactor: do refactor the sink task. --- include/libs/stream/tstream.h | 6 +- source/dnode/vnode/src/tq/tqSink.c | 9 +- source/libs/stream/inc/streamInt.h | 8 +- source/libs/stream/src/streamData.c | 21 +-- source/libs/stream/src/streamDispatch.c | 2 +- source/libs/stream/src/streamExec.c | 16 ++- source/libs/stream/src/streamQueue.c | 182 ++++++++++-------------- 7 files changed, 111 insertions(+), 133 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index eeb4d6d4f3..65c869433b 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -29,8 +29,10 @@ extern "C" { #ifndef _STREAM_H_ #define _STREAM_H_ -#define ONE_MB_F (1048576.0) -#define SIZE_IN_MB(_v) ((_v) / ONE_MB_F) +#define ONE_MiB_F (1048576.0) +#define ONE_KiB_F (1024.0) +#define SIZE_IN_MiB(_v) ((_v) / ONE_MiB_F) +#define SIZE_IN_KiB(_v) ((_v) / ONE_KiB_F) typedef struct SStreamTask SStreamTask; diff --git a/source/dnode/vnode/src/tq/tqSink.c b/source/dnode/vnode/src/tq/tqSink.c index f700294e79..0ca28277e7 100644 --- a/source/dnode/vnode/src/tq/tqSink.c +++ b/source/dnode/vnode/src/tq/tqSink.c @@ -270,14 +270,14 @@ int32_t doBuildAndSendSubmitMsg(SVnode* pVnode, SStreamTask* pTask, SSubmitReq2* tqError("s-task:%s failed to put into write-queue since %s", id, terrstr()); } - pTask->sinkRecorder.numOfSubmit += 1; + SSinkTaskRecorder* pRec = &pTask->sinkRecorder; - if ((pTask->sinkRecorder.numOfSubmit % 5000) == 0) { - SSinkTaskRecorder* pRec = &pTask->sinkRecorder; + pRec->numOfSubmit += 1; + if ((pRec->numOfSubmit % 5000) == 0) { double el = (taosGetTimestampMs() - pTask->taskExecInfo.start) / 1000.0; tqInfo("s-task:%s vgId:%d write %" PRId64 " blocks (%" PRId64 " rows) in %" PRId64 " submit into dst table, %.2fMiB duration:%.2f Sec.", - pTask->id.idStr, vgId, pRec->numOfBlocks, pRec->numOfRows, pRec->numOfSubmit, SIZE_IN_MB(pRec->bytes), el); + pTask->id.idStr, vgId, pRec->numOfBlocks, pRec->numOfRows, pRec->numOfSubmit, SIZE_IN_MiB(pRec->bytes), el); } return TSDB_CODE_SUCCESS; @@ -868,7 +868,6 @@ void tqSinkDataIntoDstTable(SStreamTask* pTask, void* vnode, void* data) { } pTask->sinkRecorder.numOfRows += pDataBlock->info.rows; - pTask->sinkRecorder.bytes += pDataBlock->info.rowSize; } taosHashCleanup(pTableIndexMap); diff --git a/source/libs/stream/inc/streamInt.h b/source/libs/stream/inc/streamInt.h index edf36d2a1c..40cadd3387 100644 --- a/source/libs/stream/inc/streamInt.h +++ b/source/libs/stream/inc/streamInt.h @@ -57,7 +57,6 @@ extern SStreamGlobalEnv streamEnv; extern int32_t streamBackendId; extern int32_t streamBackendCfWrapperId; -const char* streamGetBlockTypeStr(int32_t type); void streamRetryDispatchStreamBlock(SStreamTask* pTask, int64_t waitDuration); int32_t streamDispatchStreamBlock(SStreamTask* pTask); @@ -81,7 +80,12 @@ int32_t streamTaskSendCheckpointReadyMsg(SStreamTask* pTask); int32_t streamTaskSendCheckpointSourceRsp(SStreamTask* pTask); int32_t streamTaskGetNumOfDownstream(const SStreamTask* pTask); -int32_t streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueItem** pInput, int32_t* numOfBlocks); +int32_t streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueItem** pInput, int32_t* numOfBlocks); +int32_t streamQueueGetNumOfItemsInQueue(const SStreamQueue* pQueue); +int32_t streamQueueItemGetSize(const SStreamQueueItem* pItem); +void streamQueueItemIncSize(const SStreamQueueItem* pItem, int32_t size); +const char* streamQueueItemGetTypeStr(int32_t type); + SStreamQueueItem* streamMergeQueueItem(SStreamQueueItem* dst, SStreamQueueItem* pElem); int32_t streamTaskBuildScanhistoryRspMsg(SStreamTask* pTask, SStreamScanHistoryFinishReq* pReq, void** pBuffer, int32_t* pLen); diff --git a/source/libs/stream/src/streamData.c b/source/libs/stream/src/streamData.c index 8cf9ea1bbc..51487e5588 100644 --- a/source/libs/stream/src/streamData.c +++ b/source/libs/stream/src/streamData.c @@ -166,12 +166,16 @@ SStreamQueueItem* streamMergeQueueItem(SStreamQueueItem* dst, SStreamQueueItem* taosArrayAddAll(pBlock->blocks, pBlockSrc->blocks); taosArrayDestroy(pBlockSrc->blocks); taosFreeQitem(pElem); + + streamQueueItemIncSize(dst, streamQueueItemGetSize(pElem)); return dst; } else if (dst->type == STREAM_INPUT__MERGED_SUBMIT && pElem->type == STREAM_INPUT__DATA_SUBMIT) { SStreamMergedSubmit* pMerged = (SStreamMergedSubmit*)dst; SStreamDataSubmit* pBlockSrc = (SStreamDataSubmit*)pElem; streamMergeSubmit(pMerged, pBlockSrc); taosFreeQitem(pElem); + + streamQueueItemIncSize(dst, streamQueueItemGetSize(pElem)); return dst; } else if (dst->type == STREAM_INPUT__DATA_SUBMIT && pElem->type == STREAM_INPUT__DATA_SUBMIT) { SStreamMergedSubmit* pMerged = streamMergedSubmitNew(); @@ -184,9 +188,11 @@ SStreamQueueItem* streamMergeQueueItem(SStreamQueueItem* dst, SStreamQueueItem* streamMergeSubmit(pMerged, (SStreamDataSubmit*)pElem); taosFreeQitem(dst); taosFreeQitem(pElem); + + streamQueueItemIncSize((SStreamQueueItem*)pMerged, streamQueueItemGetSize(pElem)); return (SStreamQueueItem*)pMerged; } else { - stDebug("block type:%s not merged with existed blocks list, type:%d", streamGetBlockTypeStr(pElem->type), dst->type); + stDebug("block type:%s not merged with existed blocks list, type:%d", streamQueueItemGetTypeStr(pElem->type), dst->type); return NULL; } } @@ -227,16 +233,3 @@ void streamFreeQitem(SStreamQueueItem* data) { taosFreeQitem(pBlock); } } - -const char* streamGetBlockTypeStr(int32_t type) { - switch (type) { - case STREAM_INPUT__CHECKPOINT: - return "checkpoint"; - case STREAM_INPUT__CHECKPOINT_TRIGGER: - return "checkpoint-trigger"; - case STREAM_INPUT__TRANS_STATE: - return "trans-state"; - default: - return ""; - } -} \ No newline at end of file diff --git a/source/libs/stream/src/streamDispatch.c b/source/libs/stream/src/streamDispatch.c index 8219f6ec85..8baf411c83 100644 --- a/source/libs/stream/src/streamDispatch.c +++ b/source/libs/stream/src/streamDispatch.c @@ -500,7 +500,7 @@ int32_t streamDispatchStreamBlock(SStreamTask* pTask) { const char* id = pTask->id.idStr; int32_t numOfElems = streamQueueGetNumOfItems(pTask->outputInfo.queue); if (numOfElems > 0) { - double size = SIZE_IN_MB(taosQueueMemorySize(pTask->outputInfo.queue->pQueue)); + double size = SIZE_IN_MiB(taosQueueMemorySize(pTask->outputInfo.queue->pQueue)); stDebug("s-task:%s start to dispatch intermediate block to downstream, elem in outputQ:%d, size:%.2fMiB", id, numOfElems, size); } diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index 57f8bd016b..712b0fe610 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -72,7 +72,7 @@ static int32_t doDumpResult(SStreamTask* pTask, SStreamQueueItem* pItem, SArray* } stDebug("s-task:%s dump stream result data blocks, num:%d, size:%.2fMiB", pTask->id.idStr, numOfBlocks, - SIZE_IN_MB(size)); + SIZE_IN_MiB(size)); int32_t code = doOutputResultBlockImpl(pTask, pStreamBlocks); if (code != TSDB_CODE_SUCCESS) { // back pressure and record position @@ -163,7 +163,7 @@ static int32_t streamTaskExecImpl(SStreamTask* pTask, SStreamQueueItem* pItem, i taosArrayPush(pRes, &block); stDebug("s-task:%s (child %d) executed and get %d result blocks, size:%.2fMiB", pTask->id.idStr, - pTask->info.selfChildId, numOfBlocks, SIZE_IN_MB(size)); + pTask->info.selfChildId, numOfBlocks, SIZE_IN_MiB(size)); // current output should be dispatched to down stream nodes if (numOfBlocks >= STREAM_RESULT_DUMP_THRESHOLD || size >= STREAM_RESULT_DUMP_SIZE_THRESHOLD) { @@ -553,8 +553,12 @@ int32_t streamExecForAll(SStreamTask* pTask) { if (pTask->info.taskLevel == TASK_LEVEL__SINK) { ASSERT(type == STREAM_INPUT__DATA_BLOCK || type == STREAM_INPUT__CHECKPOINT); + // here only handle the data block sink operation if (type == STREAM_INPUT__DATA_BLOCK) { - stDebug("s-task:%s sink task start to sink %d blocks", id, numOfBlocks); + int32_t blockSize = streamQueueItemGetSize(pInput); + pTask->sinkRecorder.bytes += blockSize; + + stDebug("s-task:%s sink task start to sink %d blocks, size:%.2fKiB", id, numOfBlocks, SIZE_IN_KiB(blockSize)); doOutputResultBlockImpl(pTask, (SStreamDataBlock*)pInput); continue; } @@ -574,7 +578,7 @@ int32_t streamExecForAll(SStreamTask* pTask) { double el = (taosGetTimestampMs() - st) / 1000.0; stDebug("s-task:%s batch of input blocks exec end, elapsed time:%.2fs, result size:%.2fMiB, numOfBlocks:%d", id, el, - SIZE_IN_MB(resSize), totalBlocks); + SIZE_IN_MiB(resSize), totalBlocks); // update the currentVer if processing the submit blocks. ASSERT(pTask->chkInfo.checkpointVer <= pTask->chkInfo.nextProcessVer && ver >= pTask->chkInfo.checkpointVer); @@ -590,8 +594,8 @@ int32_t streamExecForAll(SStreamTask* pTask) { // todo other thread may change the status // do nothing after sync executor state to storage backend, untill the vnode-level checkpoint is completed. if (type == STREAM_INPUT__CHECKPOINT) { - stDebug("s-task:%s checkpoint block received, set the status:%s", pTask->id.idStr, - streamGetTaskStatusStr(pTask->status.taskStatus)); + stDebug("s-task:%s checkpoint block received, set status:%s", pTask->id.idStr, + streamGetTaskStatusStr(pTask->status.taskStatus)); streamTaskBuildCheckpoint(pTask); return 0; } diff --git a/source/libs/stream/src/streamQueue.c b/source/libs/stream/src/streamQueue.c index 65a7d56923..882c57383e 100644 --- a/source/libs/stream/src/streamQueue.c +++ b/source/libs/stream/src/streamQueue.c @@ -102,64 +102,6 @@ void streamQueueProcessFail(SStreamQueue* queue) { atomic_store_8(&queue->status, STREAM_QUEUE__FAILED); } -#if 0 -bool streamQueueResEmpty(const SStreamQueueRes* pRes) { - // - return true; -} -int64_t streamQueueResSize(const SStreamQueueRes* pRes) { return pRes->size; } -SStreamQueueNode* streamQueueResFront(SStreamQueueRes* pRes) { return pRes->head; } -SStreamQueueNode* streamQueueResPop(SStreamQueueRes* pRes) { - SStreamQueueNode* pRet = pRes->head; - pRes->head = pRes->head->next; - return pRet; -} - -void streamQueueResClear(SStreamQueueRes* pRes) { - while (pRes->head) { - SStreamQueueNode* pNode = pRes->head; - streamFreeQitem(pRes->head->item); - pRes->head = pNode; - } -} - -SStreamQueueRes streamQueueBuildRes(SStreamQueueNode* pTail) { - int64_t size = 0; - SStreamQueueNode* head = NULL; - - while (pTail) { - SStreamQueueNode* pTmp = pTail->next; - pTail->next = head; - head = pTail; - pTail = pTmp; - size++; - } - - return (SStreamQueueRes){.head = head, .size = size}; -} - -bool streamQueueHasTask(const SStreamQueue1* pQueue) { return atomic_load_ptr(pQueue->pHead); } -int32_t streamQueuePush(SStreamQueue1* pQueue, SStreamQueueItem* pItem) { - SStreamQueueNode* pNode = taosMemoryMalloc(sizeof(SStreamQueueNode)); - pNode->item = pItem; - SStreamQueueNode* pHead = atomic_load_ptr(pQueue->pHead); - while (1) { - pNode->next = pHead; - SStreamQueueNode* pOld = atomic_val_compare_exchange_ptr(pQueue->pHead, pHead, pNode); - if (pOld == pHead) { - break; - } - } - return 0; -} - -SStreamQueueRes streamQueueGetRes(SStreamQueue1* pQueue) { - SStreamQueueNode* pNode = atomic_exchange_ptr(pQueue->pHead, NULL); - if (pNode) return streamQueueBuildRes(pNode); - return (SStreamQueueRes){0}; -} -#endif - bool streamQueueIsFull(const STaosQueue* pQueue, bool inputQ) { bool isFull = taosQueueItemSize((STaosQueue*)pQueue) >= STREAM_TASK_QUEUE_CAPACITY; if (isFull) { @@ -167,7 +109,7 @@ bool streamQueueIsFull(const STaosQueue* pQueue, bool inputQ) { } int32_t threahold = (inputQ) ? STREAM_TASK_INPUT_QUEUE_CAPACITY_IN_SIZE : STREAM_TASK_OUTPUT_QUEUE_CAPACITY_IN_SIZE; - double size = SIZE_IN_MB(taosQueueMemorySize((STaosQueue*)pQueue)); + double size = SIZE_IN_MiB(taosQueueMemorySize((STaosQueue*)pQueue)); return (size >= threahold); } @@ -178,38 +120,74 @@ int32_t streamQueueGetNumOfItems(const SStreamQueue* pQueue) { return numOfItems1 + numOfItems2; } +int32_t streamQueueGetNumOfItemsInQueue(const SStreamQueue* pQueue) { + return taosQueueItemSize(pQueue->pQueue); +} + +int32_t streamQueueItemGetSize(const SStreamQueueItem* pItem) { + STaosQnode* p = (STaosQnode*)((char*) pItem - sizeof(STaosQnode)); + return p->dataSize; +} + +void streamQueueItemIncSize(const SStreamQueueItem* pItem, int32_t size) { + STaosQnode* p = (STaosQnode*)((char*) pItem - sizeof(STaosQnode)); + p->dataSize += size; +} + +const char* streamQueueItemGetTypeStr(int32_t type) { + switch (type) { + case STREAM_INPUT__CHECKPOINT: + return "checkpoint"; + case STREAM_INPUT__CHECKPOINT_TRIGGER: + return "checkpoint-trigger"; + case STREAM_INPUT__TRANS_STATE: + return "trans-state"; + default: + return "datablock"; + } +} + int32_t streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueItem** pInput, int32_t* numOfBlocks) { int32_t retryTimes = 0; int32_t MAX_RETRY_TIMES = 5; const char* id = pTask->id.idStr; + int32_t taskLevel = pTask->info.taskLevel; + *numOfBlocks = 0; - if (pTask->info.taskLevel == TASK_LEVEL__SINK) { // extract block from inputQ, one-by-one - while (1) { - if (streamTaskShouldPause(&pTask->status) || streamTaskShouldStop(&pTask->status)) { - stDebug("s-task:%s task should pause, extract input blocks:%d", pTask->id.idStr, *numOfBlocks); - return TSDB_CODE_SUCCESS; - } - - STokenBucket* pBucket = pTask->pTokenBucket; -// if (!streamTaskHasAvailableToken(pBucket)) { // no available token in th bucket, ignore this execution -// stInfo("s-task:%s no available token for sink, capacity:%d, rate:%d token/sec, quit", pTask->id.idStr, -// pBucket->capacity, pBucket->rate); -// return TSDB_CODE_SUCCESS; -// } - - SStreamQueueItem* qItem = streamQueueNextItem(pTask->inputInfo.queue); - if (qItem == NULL) { - stDebug("===stream===break batchSize:%d, %s", *numOfBlocks, id); - return TSDB_CODE_SUCCESS; - } - - stDebug("s-task:%s sink task handle block one-by-one, type:%d", id, qItem->type); - - *numOfBlocks = 1; - *pInput = qItem; - return TSDB_CODE_SUCCESS; - } - } + // if (pTask->info.taskLevel == TASK_LEVEL__SINK) { // extract block from inputQ, one-by-one + // while (1) { + // if (streamTaskShouldPause(&pTask->status) || streamTaskShouldStop(&pTask->status)) { + // stDebug("s-task:%s task should pause, extract input blocks:%d", pTask->id.idStr, *numOfBlocks); + // return TSDB_CODE_SUCCESS; + // } + // + // STokenBucket* pBucket = pTask->pTokenBucket; + // // if (!streamTaskHasAvailableToken(pBucket)) { // no available token in th bucket, ignore this + // execution + // // stInfo("s-task:%s no available token for sink, capacity:%d, rate:%d token/sec, quit", + // pTask->id.idStr, + // // pBucket->capacity, pBucket->rate); + // // return TSDB_CODE_SUCCESS; + // // } + // + // SStreamQueueItem* qItem = streamQueueNextItem(pTask->inputInfo.queue); + // if (qItem == NULL) { + // if (++retryTimes < MAX_RETRY_TIMES) { + // taosMsleep(10); + // continue; + // } + // + // return TSDB_CODE_SUCCESS; + // } + // + // stDebug("s-task:%s sink task handle block, type:%s", id, streamQueueItemGetTypeStr(qItem->type)); + // pTask->sinkRecorder.bytes += streamQueueItemGetSize(qItem); + // + // *numOfBlocks = 1; + // *pInput = qItem; + // return TSDB_CODE_SUCCESS; + // } + // } while (1) { if (streamTaskShouldPause(&pTask->status) || streamTaskShouldStop(&pTask->status)) { @@ -219,20 +197,18 @@ int32_t streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueItem** pInpu SStreamQueueItem* qItem = streamQueueNextItem(pTask->inputInfo.queue); if (qItem == NULL) { - if (pTask->info.taskLevel == TASK_LEVEL__SOURCE && (++retryTimes) < MAX_RETRY_TIMES) { + if ((taskLevel == TASK_LEVEL__SOURCE || taskLevel == TASK_LEVEL__SINK) && (++retryTimes) < MAX_RETRY_TIMES) { taosMsleep(10); - stDebug("===stream===try again batchSize:%d, retry:%d, %s", *numOfBlocks, retryTimes, id); continue; } - - stDebug("===stream===break batchSize:%d, %s", *numOfBlocks, id); return TSDB_CODE_SUCCESS; } // do not merge blocks for sink node and check point data block - if (qItem->type == STREAM_INPUT__CHECKPOINT || qItem->type == STREAM_INPUT__CHECKPOINT_TRIGGER || - qItem->type == STREAM_INPUT__TRANS_STATE) { - const char* p = streamGetBlockTypeStr(qItem->type); + int8_t type = qItem->type; + if (type == STREAM_INPUT__CHECKPOINT || type == STREAM_INPUT__CHECKPOINT_TRIGGER || + type == STREAM_INPUT__TRANS_STATE) { + const char* p = streamQueueItemGetTypeStr(qItem->type); if (*pInput == NULL) { stDebug("s-task:%s %s msg extracted, start to process immediately", id, p); @@ -285,7 +261,7 @@ int32_t streamTaskPutDataIntoInputQ(SStreamTask* pTask, SStreamQueueItem* pItem) if (type == STREAM_INPUT__DATA_SUBMIT) { SStreamDataSubmit* px = (SStreamDataSubmit*)pItem; if ((pTask->info.taskLevel == TASK_LEVEL__SOURCE) && streamQueueIsFull(pQueue, true)) { - double size = SIZE_IN_MB(taosQueueMemorySize(pQueue)); + double size = SIZE_IN_MiB(taosQueueMemorySize(pQueue)); stTrace( "s-task:%s inputQ is full, capacity(size:%d num:%dMiB), current(blocks:%d, size:%.2fMiB) stop to push data", pTask->id.idStr, STREAM_TASK_QUEUE_CAPACITY, STREAM_TASK_INPUT_QUEUE_CAPACITY_IN_SIZE, total, size); @@ -304,15 +280,15 @@ int32_t streamTaskPutDataIntoInputQ(SStreamTask* pTask, SStreamQueueItem* pItem) return code; } - double size = SIZE_IN_MB(taosQueueMemorySize(pQueue)); + double size = SIZE_IN_MiB(taosQueueMemorySize(pQueue)); // use the local variable to avoid the pItem be freed by other threads, since it has been put into queue already. stDebug("s-task:%s submit enqueue msgLen:%d ver:%" PRId64 ", total in queue:%d, size:%.2fMiB", pTask->id.idStr, - msgLen, ver, total, size + SIZE_IN_MB(msgLen)); + msgLen, ver, total, size + SIZE_IN_MiB(msgLen)); } else if (type == STREAM_INPUT__DATA_BLOCK || type == STREAM_INPUT__DATA_RETRIEVE || type == STREAM_INPUT__REF_DATA_BLOCK) { if (streamQueueIsFull(pQueue, true)) { - double size = SIZE_IN_MB(taosQueueMemorySize(pQueue)); + double size = SIZE_IN_MiB(taosQueueMemorySize(pQueue)); stTrace("s-task:%s input queue is full, capacity:%d size:%d MiB, current(blocks:%d, size:%.2fMiB) abort", pTask->id.idStr, STREAM_TASK_QUEUE_CAPACITY, STREAM_TASK_INPUT_QUEUE_CAPACITY_IN_SIZE, total, size); @@ -326,7 +302,7 @@ int32_t streamTaskPutDataIntoInputQ(SStreamTask* pTask, SStreamQueueItem* pItem) return code; } - double size = SIZE_IN_MB(taosQueueMemorySize(pQueue)); + double size = SIZE_IN_MiB(taosQueueMemorySize(pQueue)); 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) { @@ -336,9 +312,9 @@ int32_t streamTaskPutDataIntoInputQ(SStreamTask* pTask, SStreamQueueItem* pItem) return code; } - double size = SIZE_IN_MB(taosQueueMemorySize(pQueue)); + 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, streamGetBlockTypeStr(type), total, size); + 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); @@ -347,7 +323,7 @@ int32_t streamTaskPutDataIntoInputQ(SStreamTask* pTask, SStreamQueueItem* pItem) return code; } - double size = SIZE_IN_MB(taosQueueMemorySize(pQueue)); + double size = SIZE_IN_MiB(taosQueueMemorySize(pQueue)); stDebug("s-task:%s data res enqueue, current(blocks:%d, size:%.2fMiB)", pTask->id.idStr, total, size); } else { ASSERT(0); @@ -372,7 +348,7 @@ int32_t streamTaskPutDataIntoOutputQ(SStreamTask* pTask, SStreamDataBlock* pBloc } int32_t total = streamQueueGetNumOfItems(pTask->outputInfo.queue); - double size = SIZE_IN_MB(taosQueueMemorySize(pQueue)); + double size = SIZE_IN_MiB(taosQueueMemorySize(pQueue)); // let's wait for there are enough space to hold this result pBlock stDebug("s-task:%s outputQ is full, wait for 500ms and retry, outputQ items:%d, size:%.2fMiB", pTask->id.idStr, total, size); @@ -382,7 +358,7 @@ int32_t streamTaskPutDataIntoOutputQ(SStreamTask* pTask, SStreamDataBlock* pBloc int32_t code = taosWriteQitem(pQueue, pBlock); int32_t total = streamQueueGetNumOfItems(pTask->outputInfo.queue); - double size = SIZE_IN_MB(taosQueueMemorySize(pQueue)); + double size = SIZE_IN_MiB(taosQueueMemorySize(pQueue)); if (code != 0) { stError("s-task:%s failed to put res into outputQ, outputQ items:%d, size:%.2fMiB code:%s, result lost", pTask->id.idStr, total + 1, size, tstrerror(code)); From 118e078f4a81370c6b1c5321a95c8b4ec0c682d6 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 21 Sep 2023 14:01:41 +0800 Subject: [PATCH 052/158] log(stream): update log. --- source/dnode/vnode/src/tq/tq.c | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 3941b35db7..be35367528 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -900,7 +900,7 @@ int32_t tqProcessStreamTaskCheckReq(STQ* pTq, SRpcMsg* pMsg) { // only the leader node handle the check request if (!pMeta->leader) { - tqError("s-task:0x%x invalid check msg from upstream:0x%x(vgId:%d), vgId:%d is follower, not handle check msg", + tqError("s-task:0x%x invalid check msg from upstream:0x%x(vgId:%d), vgId:%d is follower, not handle check status msg", taskId, req.upstreamTaskId, req.upstreamNodeId, pMeta->vgId); return -1; } @@ -1611,7 +1611,7 @@ int32_t tqProcessStreamCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg) { SStreamCheckpointSourceReq req = {0}; if (!vnodeIsRoleLeader(pTq->pVnode)) { - tqDebug("vgId:%d not leader node, ignore checkpoint-source msg", vgId); + tqDebug("vgId:%d not leader, ignore checkpoint-source msg", vgId); return TSDB_CODE_SUCCESS; } @@ -1672,7 +1672,7 @@ int32_t tqProcessStreamCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg) { total = pMeta->numOfStreamTasks; taosWUnLockLatch(&pMeta->lock); - qDebug("s-task:%s (vgId:%d) level:%d receive checkpoint-source msg, chkpt:%" PRId64 ", total checkpoint req:%d", + qInfo("s-task:%s (vgId:%d) level:%d receive checkpoint-source msg chkpt:%" PRId64 ", total checkpoint reqs:%d", pTask->id.idStr, vgId, pTask->info.taskLevel, req.checkpointId, total); code = streamAddCheckpointSourceRspMsg(&req, &pMsg->info, pTask, 1); From 04405b84c51eaa8888d4ca808372a3e82b85267c Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 21 Sep 2023 14:03:14 +0800 Subject: [PATCH 053/158] fix(stream): update logs. --- source/dnode/vnode/src/tq/tqUtil.c | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/source/dnode/vnode/src/tq/tqUtil.c b/source/dnode/vnode/src/tq/tqUtil.c index f10f87b6b7..5b32269767 100644 --- a/source/dnode/vnode/src/tq/tqUtil.c +++ b/source/dnode/vnode/src/tq/tqUtil.c @@ -39,7 +39,8 @@ int32_t tqInitDataRsp(SMqDataRsp* pRsp, STqOffsetVal pOffset) { void tqUpdateNodeStage(STQ* pTq, bool isLeader) { SSyncState state = syncGetState(pTq->pVnode->sync); SStreamMeta* pMeta = pTq->pStreamMeta; - tqDebug("vgId:%d update the meta stage:%"PRId64", prev:%"PRId64" leader:%d", pMeta->vgId, state.term, pMeta->stage, isLeader); + tqInfo("vgId:%d update the meta stage:%"PRId64", prev:%"PRId64" leader:%d", pMeta->vgId, state.term, pMeta->stage, isLeader); + pMeta->stage = state.term; pMeta->leader = isLeader; if (isLeader) { From 7f4d9b6e4151d3405df45236604a3206a3845e2a Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 21 Sep 2023 15:53:21 +0800 Subject: [PATCH 054/158] fix(stream): wait for the task checkpoint before stop. --- source/dnode/vnode/src/tq/tqUtil.c | 9 +++++++-- source/dnode/vnode/src/vnd/vnodeSync.c | 2 +- source/libs/stream/src/streamTask.c | 17 ++++++++++++++++- 3 files changed, 24 insertions(+), 4 deletions(-) diff --git a/source/dnode/vnode/src/tq/tqUtil.c b/source/dnode/vnode/src/tq/tqUtil.c index 5b32269767..b87783cfd0 100644 --- a/source/dnode/vnode/src/tq/tqUtil.c +++ b/source/dnode/vnode/src/tq/tqUtil.c @@ -37,14 +37,19 @@ int32_t tqInitDataRsp(SMqDataRsp* pRsp, STqOffsetVal pOffset) { } void tqUpdateNodeStage(STQ* pTq, bool isLeader) { - SSyncState state = syncGetState(pTq->pVnode->sync); + SSyncState state = syncGetState(pTq->pVnode->sync); SStreamMeta* pMeta = pTq->pStreamMeta; - tqInfo("vgId:%d update the meta stage:%"PRId64", prev:%"PRId64" leader:%d", pMeta->vgId, state.term, pMeta->stage, isLeader); + int64_t stage = pMeta->stage; pMeta->stage = state.term; pMeta->leader = isLeader; if (isLeader) { + tqInfo("vgId:%d update meta stage:%" PRId64 ", prev:%" PRId64 " leader:%d, start to send Hb", pMeta->vgId, + state.term, stage, isLeader); streamMetaStartHb(pMeta); + } else { + tqInfo("vgId:%d update meta stage:%" PRId64 " prev:%" PRId64 " leader:%d", pMeta->vgId, state.term, stage, + isLeader); } } diff --git a/source/dnode/vnode/src/vnd/vnodeSync.c b/source/dnode/vnode/src/vnd/vnodeSync.c index 42acdd2b40..cf299dc79c 100644 --- a/source/dnode/vnode/src/vnd/vnodeSync.c +++ b/source/dnode/vnode/src/vnd/vnodeSync.c @@ -612,10 +612,10 @@ static void vnodeBecomeLearner(const SSyncFSM *pFsm) { static void vnodeBecomeLeader(const SSyncFSM *pFsm) { SVnode *pVnode = pFsm->data; + vDebug("vgId:%d, become leader", pVnode->config.vgId); if (pVnode->pTq) { tqUpdateNodeStage(pVnode->pTq, true); } - vDebug("vgId:%d, become leader", pVnode->config.vgId); } static bool vnodeApplyQueueEmpty(const SSyncFSM *pFsm) { diff --git a/source/libs/stream/src/streamTask.c b/source/libs/stream/src/streamTask.c index c328ff4bbc..777e93da47 100644 --- a/source/libs/stream/src/streamTask.c +++ b/source/libs/stream/src/streamTask.c @@ -543,7 +543,22 @@ int32_t streamTaskStop(SStreamTask* pTask) { int64_t st = taosGetTimestampMs(); const char* id = pTask->id.idStr; - pTask->status.taskStatus = TASK_STATUS__STOP; + // we should wait for the task complete the checkpoint operation before stop it, otherwise, the operation maybe blocked + // by the unfinished checkpoint operation, even if the leader has become the follower. + while(1) { + taosThreadMutexLock(&pTask->lock); + + if (pTask->status.taskStatus == TASK_STATUS__CK) { + stDebug("s-task:%s in checkpoint, wait for it completed for 500ms before stop task", pTask->id.idStr); + taosThreadMutexUnlock(&pTask->lock); + taosMsleep(500); + } else { + pTask->status.taskStatus = TASK_STATUS__STOP; + taosThreadMutexUnlock(&pTask->lock); + break; + } + } + qKillTask(pTask->exec.pExecutor, TSDB_CODE_SUCCESS); while (/*pTask->status.schedStatus != TASK_SCHED_STATUS__INACTIVE */ !streamTaskIsIdle(pTask)) { From 6d3a1340777e193a1f26bd3529a041ab6420cdfb Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 21 Sep 2023 17:22:09 +0800 Subject: [PATCH 055/158] fix(stream): ignore the dispatch failure, and set the correct rsp counter. --- source/libs/stream/src/streamCheckpoint.c | 2 +- source/libs/stream/src/streamDispatch.c | 77 +++++++++++++++-------- 2 files changed, 52 insertions(+), 27 deletions(-) diff --git a/source/libs/stream/src/streamCheckpoint.c b/source/libs/stream/src/streamCheckpoint.c index ae4bc5366d..8be41e8b39 100644 --- a/source/libs/stream/src/streamCheckpoint.c +++ b/source/libs/stream/src/streamCheckpoint.c @@ -264,7 +264,6 @@ int32_t streamProcessCheckpointReadyMsg(SStreamTask* pTask) { int32_t streamSaveAllTaskStatus(SStreamMeta* pMeta, int64_t checkpointId) { taosWLockLatch(&pMeta->lock); - int64_t keys[2]; for (int32_t i = 0; i < taosArrayGetSize(pMeta->pTaskList); ++i) { STaskId* pId = taosArrayGet(pMeta->pTaskList, i); SStreamTask** ppTask = taosHashGet(pMeta->pTasksMap, pId, sizeof(*pId)); @@ -305,6 +304,7 @@ int32_t streamSaveAllTaskStatus(SStreamMeta* pMeta, int64_t checkpointId) { return TSDB_CODE_SUCCESS; } +// todo: handle the case: during the checkpoint procedure, leader/follower changes happened. int32_t streamTaskBuildCheckpoint(SStreamTask* pTask) { int32_t code = 0; diff --git a/source/libs/stream/src/streamDispatch.c b/source/libs/stream/src/streamDispatch.c index 8baf411c83..5c29cb8ad2 100644 --- a/source/libs/stream/src/streamDispatch.c +++ b/source/libs/stream/src/streamDispatch.c @@ -947,20 +947,56 @@ int32_t streamNotifyUpstreamContinue(SStreamTask* pTask) { return 0; } +// this message has been sent successfully, let's try next one. +static int32_t handleDispatchSuccessRsp(SStreamTask* pTask, int32_t downstreamId) { + destroyStreamDataBlock(pTask->msgInfo.pData); + pTask->msgInfo.pData = NULL; + + if (pTask->msgInfo.blockingTs != 0) { + int64_t el = taosGetTimestampMs() - pTask->msgInfo.blockingTs; + stDebug("s-task:%s downstream task:0x%x resume to normal from inputQ blocking, blocking time:%" PRId64 "ms", + pTask->id.idStr, downstreamId, el); + pTask->msgInfo.blockingTs = 0; + + // put data into inputQ of current task is also allowed + pTask->inputInfo.status = TASK_INPUT_STATUS__NORMAL; + } + + // now ready for next data output + atomic_store_8(&pTask->outputInfo.status, TASK_OUTPUT_STATUS__NORMAL); + + // otherwise, continue dispatch the first block to down stream task in pipeline + streamDispatchStreamBlock(pTask); + return 0; +} + int32_t streamProcessDispatchRsp(SStreamTask* pTask, SStreamDispatchRsp* pRsp, int32_t code) { const char* id = pTask->id.idStr; if (code != TSDB_CODE_SUCCESS) { // dispatch message failed: network error, or node not available. - // in case of the input queue is full, the code will be TSDB_CODE_SUCCESS, the and pRsp>inputStatus will be set + // in case of the input queue is full, the code will be TSDB_CODE_SUCCESS, the and pRsp->inputStatus will be set // flag. here we need to retry dispatch this message to downstream task immediately. handle the case the failure // happened too fast. // todo handle the shuffle dispatch failure if (code == TSDB_CODE_STREAM_TASK_NOT_EXIST) { // destination task does not exist, not retry anymore - stWarn("s-task:%s failed to dispatch msg to task:0x%x, no retry, since it is destroyed already", id, pRsp->downstreamTaskId); + stError("s-task:%s failed to dispatch msg to task:0x%x(vgId:%d), no retry, since it is destroyed already", id, + pRsp->downstreamTaskId, pRsp->downstreamNodeId); + {// we should set the correct finish flag to make sure the shuffle dispatch will be executed completed. + if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { + int32_t left = atomic_sub_fetch_32(&pTask->shuffleDispatcher.waitingRspCnt, 1); + if (left > 0) { // do nothing + stError("s-task:%s add the shuffle dispatch counter to complete the dispatch process", id); + } else { + stError("s-task:%s the last rsp is failed, ignore it and continue, roll-back will discard this msg", id); + handleDispatchSuccessRsp(pTask, pRsp->downstreamTaskId); + } + } + } } else { - stError("s-task:%s failed to dispatch msg to task:0x%x, code:%s, retry cnt:%d", id, pRsp->downstreamTaskId, - tstrerror(code), ++pTask->msgInfo.retryCount); + stError("s-task:%s failed to dispatch msg to task:0x%x(vgId:%d), code:%s, retry cnt:%d", id, + pRsp->downstreamTaskId, pRsp->downstreamNodeId, tstrerror(code), ++pTask->msgInfo.retryCount); + int32_t ret = doDispatchAllBlocks(pTask, pTask->msgInfo.pData); if (ret != TSDB_CODE_SUCCESS) { } @@ -969,16 +1005,20 @@ int32_t streamProcessDispatchRsp(SStreamTask* pTask, SStreamDispatchRsp* pRsp, i return TSDB_CODE_SUCCESS; } - stDebug("s-task:%s recv dispatch rsp from 0x%x, downstream task input status:%d code:%d", id, pRsp->downstreamTaskId, - pRsp->inputStatus, code); - // there are other dispatch message not response yet if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { int32_t leftRsp = atomic_sub_fetch_32(&pTask->shuffleDispatcher.waitingRspCnt, 1); - stDebug("s-task:%s is shuffle, left waiting rsp %d", id, leftRsp); if (leftRsp > 0) { + stDebug("s-task:%s recv dispatch rsp from 0x%x(vgId:%d), downstream task input status:%d code:%d, waiting for %d rsp", id, pRsp->downstreamTaskId, + pRsp->downstreamNodeId, pRsp->inputStatus, code, leftRsp); return 0; + } else { + stDebug("s-task:%s recv dispatch rsp from 0x%x(vgId:%d), downstream task input status:%d code:%d, all rsp", id, + pRsp->downstreamTaskId, pRsp->downstreamNodeId, pRsp->inputStatus, code); } + } else { + stDebug("s-task:%s recv fix-dispatch rsp from 0x%x(vgId:%d), downstream task input status:%d code:%d", id, + pRsp->downstreamTaskId, pRsp->downstreamNodeId, pRsp->inputStatus, code); } // transtate msg has been sent to downstream successfully. let's transfer the fill-history task state @@ -986,6 +1026,7 @@ int32_t streamProcessDispatchRsp(SStreamTask* pTask, SStreamDispatchRsp* pRsp, i if (p->type == STREAM_INPUT__TRANS_STATE) { stDebug("s-task:%s dispatch transtate msg to downstream successfully, start to transfer state", id); ASSERT(pTask->info.fillHistory == 1); + code = streamTransferStateToStreamTask(pTask); if (code != TSDB_CODE_SUCCESS) { // todo: do nothing if error happens } @@ -1004,6 +1045,7 @@ int32_t streamProcessDispatchRsp(SStreamTask* pTask, SStreamDispatchRsp* pRsp, i // so the TASK_INPUT_STATUS_BLOCKED is rsp if (pRsp->inputStatus == TASK_INPUT_STATUS__BLOCKED) { pTask->inputInfo.status = TASK_INPUT_STATUS__BLOCKED; // block the input of current task, to push pressure to upstream + double el = 0; if (pTask->msgInfo.blockingTs == 0) { pTask->msgInfo.blockingTs = taosGetTimestampMs(); // record the blocking start time @@ -1018,24 +1060,7 @@ int32_t streamProcessDispatchRsp(SStreamTask* pTask, SStreamDispatchRsp* pRsp, i streamRetryDispatchStreamBlock(pTask, DISPATCH_RETRY_INTERVAL_MS); } else { // pipeline send data in output queue // this message has been sent successfully, let's try next one. - destroyStreamDataBlock(pTask->msgInfo.pData); - pTask->msgInfo.pData = NULL; - - if (pTask->msgInfo.blockingTs != 0) { - int64_t el = taosGetTimestampMs() - pTask->msgInfo.blockingTs; - stDebug("s-task:%s downstream task:0x%x resume to normal from inputQ blocking, blocking time:%" PRId64 "ms", id, - pRsp->downstreamTaskId, el); - pTask->msgInfo.blockingTs = 0; - - // put data into inputQ of current task is also allowed - pTask->inputInfo.status = TASK_INPUT_STATUS__NORMAL; - } - - // now ready for next data output - atomic_store_8(&pTask->outputInfo.status, TASK_OUTPUT_STATUS__NORMAL); - - // otherwise, continue dispatch the first block to down stream task in pipeline - streamDispatchStreamBlock(pTask); + handleDispatchSuccessRsp(pTask, pRsp->downstreamTaskId); } return 0; From bcecc2418a27eb5e9da159e9727341e13033b80b Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 21 Sep 2023 18:00:29 +0800 Subject: [PATCH 056/158] fix(stream): set the checkpoint interval to be 10min. --- source/common/src/tglobal.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/common/src/tglobal.c b/source/common/src/tglobal.c index 85b1028bb7..c6698fa2cb 100644 --- a/source/common/src/tglobal.c +++ b/source/common/src/tglobal.c @@ -241,7 +241,7 @@ int32_t tsTtlBatchDropNum = 10000; // number of tables dropped per batch // internal int32_t tsTransPullupInterval = 2; int32_t tsMqRebalanceInterval = 2; -int32_t tsStreamCheckpointTickInterval = 10; +int32_t tsStreamCheckpointTickInterval = 600; int32_t tsStreamNodeCheckInterval = 10; int32_t tsTtlUnit = 86400; int32_t tsTtlPushIntervalSec = 10; From 65cc4040f35fe8e823668ad5f209d39a3988e748 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 21 Sep 2023 19:14:34 +0800 Subject: [PATCH 057/158] fix(stream): update logs. --- source/libs/stream/src/streamCheckpoint.c | 10 ++++++---- source/libs/stream/src/streamDispatch.c | 2 +- 2 files changed, 7 insertions(+), 5 deletions(-) diff --git a/source/libs/stream/src/streamCheckpoint.c b/source/libs/stream/src/streamCheckpoint.c index 8be41e8b39..9dea265241 100644 --- a/source/libs/stream/src/streamCheckpoint.c +++ b/source/libs/stream/src/streamCheckpoint.c @@ -285,10 +285,12 @@ int32_t streamSaveAllTaskStatus(SStreamMeta* pMeta, int64_t checkpointId) { // save the task streamMetaSaveTask(pMeta, p); streamTaskOpenAllUpstreamInput(p); // open inputQ for all upstream tasks - stDebug("vgId:%d s-task:%s level:%d commit task status after checkpoint completed, checkpointId:%" PRId64 - ", Ver(saved):%" PRId64 " currentVer:%" PRId64 ", status to be normal, prev:%s", - pMeta->vgId, p->id.idStr, p->info.taskLevel, checkpointId, p->chkInfo.checkpointVer, p->chkInfo.nextProcessVer, - streamGetTaskStatusStr(prev)); + + stDebug( + "vgId:%d s-task:%s level:%d open upstream inputQ, commit task status after checkpoint completed, " + "checkpointId:%" PRId64 ", Ver(saved):%" PRId64 " currentVer:%" PRId64 ", status to be normal, prev:%s", + pMeta->vgId, p->id.idStr, p->info.taskLevel, checkpointId, p->chkInfo.checkpointVer, p->chkInfo.nextProcessVer, + streamGetTaskStatusStr(prev)); } if (streamMetaCommit(pMeta) < 0) { diff --git a/source/libs/stream/src/streamDispatch.c b/source/libs/stream/src/streamDispatch.c index 5c29cb8ad2..9af35eee88 100644 --- a/source/libs/stream/src/streamDispatch.c +++ b/source/libs/stream/src/streamDispatch.c @@ -837,7 +837,7 @@ int32_t streamAddCheckpointReadyMsg(SStreamTask* pTask, int32_t upstreamTaskId, info.msg.info.noResp = 1; // refactor later. stDebug("s-task:%s (level:%d) prepare checkpoint ready msg to upstream s-task:0x%" PRIx64 ":0x%x (vgId:%d) idx:%d", - pTask->id.idStr, pTask->info.taskLevel, req.streamId, req.upstreamTaskId, req.downstreamNodeId, index); + pTask->id.idStr, pTask->info.taskLevel, req.streamId, req.upstreamTaskId, req.upstreamNodeId, index); if (pTask->pReadyMsgList == NULL) { pTask->pReadyMsgList = taosArrayInit(4, sizeof(SStreamChkptReadyInfo)); From 7c61d7304c662deb9277d822368f8fce45675e01 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 21 Sep 2023 19:31:23 +0800 Subject: [PATCH 058/158] fix(stream): reset checkpoint to be 10 sec. --- source/common/src/tglobal.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/common/src/tglobal.c b/source/common/src/tglobal.c index c6698fa2cb..85b1028bb7 100644 --- a/source/common/src/tglobal.c +++ b/source/common/src/tglobal.c @@ -241,7 +241,7 @@ int32_t tsTtlBatchDropNum = 10000; // number of tables dropped per batch // internal int32_t tsTransPullupInterval = 2; int32_t tsMqRebalanceInterval = 2; -int32_t tsStreamCheckpointTickInterval = 600; +int32_t tsStreamCheckpointTickInterval = 10; int32_t tsStreamNodeCheckInterval = 10; int32_t tsTtlUnit = 86400; int32_t tsTtlPushIntervalSec = 10; From e5535fa1f53496f7d45ba0952a2dfc82ade9d3da Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 21 Sep 2023 23:15:18 +0800 Subject: [PATCH 059/158] fix(stream): handle dispatch checkpoint-trigger failure. --- source/dnode/vnode/src/tq/tq.c | 6 ++++ source/libs/stream/src/stream.c | 1 + source/libs/stream/src/streamCheckpoint.c | 6 ++-- source/libs/stream/src/streamDispatch.c | 34 ++++++++++++++--------- 4 files changed, 31 insertions(+), 16 deletions(-) diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index be35367528..12898847f7 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1552,6 +1552,12 @@ int32_t vnodeEnqueueStreamMsg(SVnode* pVnode, SRpcMsg* pMsg) { int32_t taskId = req.taskId; tqDebug("vgId:%d receive dispatch msg to s-task:0x%" PRIx64 "-0x%x", vgId, req.streamId, taskId); + // for test purpose +// if (req.type == STREAM_INPUT__CHECKPOINT_TRIGGER) { +// code = TSDB_CODE_STREAM_TASK_NOT_EXIST; +// goto FAIL; +// } + SStreamTask* pTask = streamMetaAcquireTask(pTq->pStreamMeta, req.streamId, taskId); if (pTask != NULL) { SRpcMsg rsp = {.info = pMsg->info, .code = 0}; diff --git a/source/libs/stream/src/stream.c b/source/libs/stream/src/stream.c index c964d0b811..6c12fbb822 100644 --- a/source/libs/stream/src/stream.c +++ b/source/libs/stream/src/stream.c @@ -240,6 +240,7 @@ int32_t streamProcessDispatchMsg(SStreamTask* pTask, SStreamDispatchReq* pReq, S } } + // disable the data from upstream tasks int8_t st = pTask->status.taskStatus; if (st == TASK_STATUS__HALT) { status = TASK_INPUT_STATUS__BLOCKED; diff --git a/source/libs/stream/src/streamCheckpoint.c b/source/libs/stream/src/streamCheckpoint.c index 9dea265241..4e0e667614 100644 --- a/source/libs/stream/src/streamCheckpoint.c +++ b/source/libs/stream/src/streamCheckpoint.c @@ -198,9 +198,9 @@ int32_t streamProcessCheckpointBlock(SStreamTask* pTask, SStreamDataBlock* pBloc } } else if (taskLevel == TASK_LEVEL__SINK || taskLevel == TASK_LEVEL__AGG) { ASSERT(taosArrayGetSize(pTask->pUpstreamInfoList) > 0); - - // todo: this may be not the first one - pTask->chkInfo.startTs = taosGetTimestampMs(); + if (pTask->chkInfo.startTs == 0) { + pTask->chkInfo.startTs = taosGetTimestampMs(); + } // update the child Id for downstream tasks streamAddCheckpointReadyMsg(pTask, pBlock->srcTaskId, pTask->info.selfChildId, checkpointId); diff --git a/source/libs/stream/src/streamDispatch.c b/source/libs/stream/src/streamDispatch.c index 9af35eee88..519271703b 100644 --- a/source/libs/stream/src/streamDispatch.c +++ b/source/libs/stream/src/streamDispatch.c @@ -373,6 +373,8 @@ static int32_t doDispatchAllBlocks(SStreamTask* pTask, const SStreamDataBlock* p } } + stDebug("s-task:%s complete shuffle-dispatch blocks to all %d vnodes", pTask->id.idStr, vgSz); + code = 0; FAIL_SHUFFLE_DISPATCH: @@ -976,22 +978,28 @@ int32_t streamProcessDispatchRsp(SStreamTask* pTask, SStreamDispatchRsp* pRsp, i if (code != TSDB_CODE_SUCCESS) { // dispatch message failed: network error, or node not available. // in case of the input queue is full, the code will be TSDB_CODE_SUCCESS, the and pRsp->inputStatus will be set - // flag. here we need to retry dispatch this message to downstream task immediately. handle the case the failure + // flag. Here we need to retry dispatch this message to downstream task immediately. handle the case the failure // happened too fast. // todo handle the shuffle dispatch failure - if (code == TSDB_CODE_STREAM_TASK_NOT_EXIST) { // destination task does not exist, not retry anymore + if (code == TSDB_CODE_STREAM_TASK_NOT_EXIST) { // destination task does not exist, not retry anymore stError("s-task:%s failed to dispatch msg to task:0x%x(vgId:%d), no retry, since it is destroyed already", id, - pRsp->downstreamTaskId, pRsp->downstreamNodeId); - {// we should set the correct finish flag to make sure the shuffle dispatch will be executed completed. - if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { - int32_t left = atomic_sub_fetch_32(&pTask->shuffleDispatcher.waitingRspCnt, 1); - if (left > 0) { // do nothing - stError("s-task:%s add the shuffle dispatch counter to complete the dispatch process", id); - } else { - stError("s-task:%s the last rsp is failed, ignore it and continue, roll-back will discard this msg", id); - handleDispatchSuccessRsp(pTask, pRsp->downstreamTaskId); - } - } + pRsp->downstreamTaskId, pRsp->downstreamNodeId); + + SStreamDataBlock* pMsgBlock = pTask->msgInfo.pData; + if (pMsgBlock->type == STREAM_INPUT__CHECKPOINT_TRIGGER) { + stError("s-task:%s checkpoint trigger send failed, continue do checkpoint ready process", id); + streamProcessCheckpointReadyMsg(pTask); + } + + // we should set the correct finish flag to make sure the shuffle dispatch will be executed completed. + if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { + int32_t left = atomic_sub_fetch_32(&pTask->shuffleDispatcher.waitingRspCnt, 1); + if (left > 0) { // do nothing + stError("s-task:%s add the shuffle dispatch counter to complete the dispatch process", id); + } else { + stError("s-task:%s the last rsp is failed, ignore it and continue, roll-back will discard this msg", id); + handleDispatchSuccessRsp(pTask, pRsp->downstreamTaskId); + } } } else { stError("s-task:%s failed to dispatch msg to task:0x%x(vgId:%d), code:%s, retry cnt:%d", id, From ab7c18f604d8b3f4a7df9eff6a0cf6797ede155a Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Fri, 22 Sep 2023 09:24:36 +0800 Subject: [PATCH 060/158] refactor: do some internal refactor. --- source/dnode/vnode/src/tq/tq.c | 2 ++ source/dnode/vnode/src/tq/tqStreamTask.c | 2 +- source/libs/stream/src/stream.c | 4 ++++ 3 files changed, 7 insertions(+), 1 deletion(-) diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 12898847f7..567c33b4e0 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1348,6 +1348,8 @@ int32_t tqProcessTaskDispatchReq(STQ* pTq, SRpcMsg* pMsg, bool exec) { tDecodeStreamDispatchReq(&decoder, &req); tDecoderClear(&decoder); + tqDebug("s-task:0x%x recv dispatch msg from 0x%x(vgId:%d)", req.taskId, req.upstreamTaskId, req.upstreamNodeId); + SStreamTask* pTask = streamMetaAcquireTask(pTq->pStreamMeta, req.streamId, req.taskId); if (pTask) { SRpcMsg rsp = {.info = pMsg->info, .code = 0}; diff --git a/source/dnode/vnode/src/tq/tqStreamTask.c b/source/dnode/vnode/src/tq/tqStreamTask.c index 854478f41e..8992d07879 100644 --- a/source/dnode/vnode/src/tq/tqStreamTask.c +++ b/source/dnode/vnode/src/tq/tqStreamTask.c @@ -355,7 +355,7 @@ int32_t doScanWalForAllTasks(SStreamMeta* pStreamMeta, bool* pScanIdle) { const char* pStatus = streamGetTaskStatusStr(status); if (status != TASK_STATUS__NORMAL) { - tqDebug("s-task:%s not ready for new submit block from wal, status:%s", pTask->id.idStr, pStatus); + tqTrace("s-task:%s not ready for new submit block from wal, status:%s", pTask->id.idStr, pStatus); streamMetaReleaseTask(pStreamMeta, pTask); continue; } diff --git a/source/libs/stream/src/stream.c b/source/libs/stream/src/stream.c index 6c12fbb822..2f3bf147d1 100644 --- a/source/libs/stream/src/stream.c +++ b/source/libs/stream/src/stream.c @@ -218,6 +218,10 @@ int32_t streamProcessDispatchMsg(SStreamTask* pTask, SStreamDispatchReq* pReq, S SStreamChildEpInfo* pInfo = streamTaskGetUpstreamTaskEpInfo(pTask, pReq->upstreamTaskId); ASSERT(pInfo != NULL); + if (!pTask->pMeta->leader) { + ASSERT(0); + } + // upstream task has restarted/leader-follower switch/transferred to other dnodes if (pReq->stage > pInfo->stage) { stError("s-task:%s upstream task:0x%x (vgId:%d) has restart/leader-switch/vnode-transfer, prev stage:%" PRId64 From 12687eb2878dc3bacd6064821adb2c13053001f4 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Fri, 22 Sep 2023 10:23:56 +0800 Subject: [PATCH 061/158] fix(stream): add check info detailed information. --- include/libs/stream/tstream.h | 6 ++++++ source/dnode/snode/src/snode.c | 2 +- source/dnode/vnode/src/tq/tq.c | 30 +++++++++++++------------- source/libs/stream/src/streamRecover.c | 30 ++++++++++++++++++++------ 4 files changed, 45 insertions(+), 23 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 65c869433b..d75cfde8d8 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -34,6 +34,12 @@ extern "C" { #define SIZE_IN_MiB(_v) ((_v) / ONE_MiB_F) #define SIZE_IN_KiB(_v) ((_v) / ONE_KiB_F) + +#define TASK_DOWNSTREAM_READY 0x0 +#define TASK_DOWNSTREAM_NOT_READY 0x1 +#define TASK_DOWNSTREAM_NOT_LEADER 0x2 +#define TASK_SELF_NEW_STAGE 0x3 + typedef struct SStreamTask SStreamTask; #define SSTREAM_TASK_VER 2 diff --git a/source/dnode/snode/src/snode.c b/source/dnode/snode/src/snode.c index ef9c1ebe2e..d6d751304e 100644 --- a/source/dnode/snode/src/snode.c +++ b/source/dnode/snode/src/snode.c @@ -361,7 +361,7 @@ int32_t sndProcessStreamTaskCheckReq(SSnode *pSnode, SRpcMsg *pMsg) { qDebug("s-task:%s status:%s, recv task check req(reqId:0x%" PRIx64 ") task:0x%x (vgId:%d), ready:%d", pTask->id.idStr, pStatus, rsp.reqId, rsp.upstreamTaskId, rsp.upstreamNodeId, rsp.status); } else { - rsp.status = 0; + rsp.status = TASK_DOWNSTREAM_NOT_READY; qDebug("recv task check(taskId:0x%x not built yet) req(reqId:0x%" PRIx64 ") from task:0x%x (vgId:%d), rsp status %d", taskId, rsp.reqId, rsp.upstreamTaskId, rsp.upstreamNodeId, rsp.status); } diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 567c33b4e0..327b242c93 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -902,22 +902,22 @@ int32_t tqProcessStreamTaskCheckReq(STQ* pTq, SRpcMsg* pMsg) { if (!pMeta->leader) { tqError("s-task:0x%x invalid check msg from upstream:0x%x(vgId:%d), vgId:%d is follower, not handle check status msg", taskId, req.upstreamTaskId, req.upstreamNodeId, pMeta->vgId); - return -1; - } - - SStreamTask* pTask = streamMetaAcquireTask(pMeta, req.streamId, taskId); - if (pTask != NULL) { - rsp.status = streamTaskCheckStatus(pTask, req.upstreamTaskId, req.upstreamNodeId, req.stage); - streamMetaReleaseTask(pMeta, pTask); - - const char* pStatus = streamGetTaskStatusStr(pTask->status.taskStatus); - tqDebug("s-task:%s status:%s, stage:%d recv task check req(reqId:0x%" PRIx64 ") task:0x%x (vgId:%d), ready:%d", - pTask->id.idStr, pStatus, rsp.oldStage, rsp.reqId, rsp.upstreamTaskId, rsp.upstreamNodeId, rsp.status); + rsp.status = TASK_DOWNSTREAM_NOT_LEADER; } else { - rsp.status = 0; - tqDebug("tq recv task check(taskId:0x%" PRIx64 "-0x%x not built yet) req(reqId:0x%" PRIx64 - ") from task:0x%x (vgId:%d), rsp status %d", - req.streamId, taskId, rsp.reqId, rsp.upstreamTaskId, rsp.upstreamNodeId, rsp.status); + SStreamTask* pTask = streamMetaAcquireTask(pMeta, req.streamId, taskId); + if (pTask != NULL) { + rsp.status = streamTaskCheckStatus(pTask, req.upstreamTaskId, req.upstreamNodeId, req.stage); + streamMetaReleaseTask(pMeta, pTask); + + const char* pStatus = streamGetTaskStatusStr(pTask->status.taskStatus); + tqDebug("s-task:%s status:%s, stage:%d recv task check req(reqId:0x%" PRIx64 ") task:0x%x (vgId:%d), ready:%d", + pTask->id.idStr, pStatus, rsp.oldStage, rsp.reqId, rsp.upstreamTaskId, rsp.upstreamNodeId, rsp.status); + } else { + rsp.status = TASK_DOWNSTREAM_NOT_READY; + tqDebug("tq recv task check(taskId:0x%" PRIx64 "-0x%x not built yet) req(reqId:0x%" PRIx64 + ") from task:0x%x (vgId:%d), rsp status %d", + req.streamId, taskId, rsp.reqId, rsp.upstreamTaskId, rsp.upstreamNodeId, rsp.status); + } } return streamSendCheckRsp(pMeta, &req, &rsp, &pMsg->info, taskId); diff --git a/source/libs/stream/src/streamRecover.c b/source/libs/stream/src/streamRecover.c index 83d8fe0b2a..b3838677ce 100644 --- a/source/libs/stream/src/streamRecover.c +++ b/source/libs/stream/src/streamRecover.c @@ -214,7 +214,7 @@ int32_t streamTaskCheckStatus(SStreamTask* pTask, int32_t upstreamTaskId, int32_ if (pInfo->stage == -1) { pInfo->stage = stage; - stDebug("s-task:%s receive check msg from upstream task:0x%x for the time, init stage value:%" PRId64, id, + stDebug("s-task:%s receive check msg from upstream task:0x%x first time, init stage value:%" PRId64, id, upstreamTaskId, stage); } @@ -223,7 +223,13 @@ int32_t streamTaskCheckStatus(SStreamTask* pTask, int32_t upstreamTaskId, int32_ id, upstreamTaskId, vgId, stage, pInfo->stage); } - return ((pTask->status.downstreamReady == 1) && (pInfo->stage == stage))? 1:0; + if (pTask->status.downstreamReady != 1) { + return TASK_DOWNSTREAM_NOT_READY; + } else if (pInfo->stage != stage) { + return TASK_SELF_NEW_STAGE; + } else { + return TASK_DOWNSTREAM_READY; + } } static void doProcessDownstreamReadyRsp(SStreamTask* pTask, int32_t numOfReqs) { @@ -259,7 +265,7 @@ int32_t streamProcessCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* pRs ASSERT(pTask->id.taskId == pRsp->upstreamTaskId); const char* id = pTask->id.idStr; - if (pRsp->status == 1) { + if (pRsp->status == TASK_DOWNSTREAM_READY) { if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { bool found = false; @@ -298,10 +304,20 @@ int32_t streamProcessCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* pRs doProcessDownstreamReadyRsp(pTask, 1); } } else { // not ready, wait for 100ms and retry - stDebug("s-task:%s downstream taskId:0x%x (vgId:%d) not ready, stage:%d, wait for 100ms and retry", id, - pRsp->downstreamTaskId, pRsp->downstreamNodeId, pRsp->oldStage); - taosMsleep(100); - streamRecheckDownstream(pTask, pRsp); + if (pRsp->status == TASK_DOWNSTREAM_NOT_LEADER) { + stError("s-task:%s downstream taskId:0x%x (vgId:%d) vnode-transfer/leader-change detected, roll-back needed not send check again", + id, pRsp->downstreamTaskId, pRsp->downstreamNodeId); + } else if (pRsp->status == TASK_SELF_NEW_STAGE) { + stError( + "s-task:%s vnode-transfer/leader-change/restart detected, old stage:%d, current stage:%d, roll-back needed " + "and not send check again", + id, pRsp->oldStage, (int32_t) pTask->pMeta->stage); + } else { + stDebug("s-task:%s downstream taskId:0x%x (vgId:%d) not ready, stage:%d, wait for 100ms and retry", id, + pRsp->downstreamTaskId, pRsp->downstreamNodeId, pRsp->oldStage); + taosMsleep(100); + streamRecheckDownstream(pTask, pRsp); + } } return 0; From 124e1ed1ab6031e80f0dd4661f2f5f3a8b21d083 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Fri, 22 Sep 2023 10:36:15 +0800 Subject: [PATCH 062/158] enh(stream): add stage info for streamt tasks. --- source/common/src/systable.c | 1 + source/dnode/mnode/impl/src/mndStream.c | 5 ++++- 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/source/common/src/systable.c b/source/common/src/systable.c index 66a498d15c..9ceaa067f1 100644 --- a/source/common/src/systable.c +++ b/source/common/src/systable.c @@ -163,6 +163,7 @@ static const SSysDbTableSchema streamTaskSchema[] = { {.name = "node_id", .bytes = 4, .type = TSDB_DATA_TYPE_INT, .sysInfo = false}, {.name = "level", .bytes = 20 + VARSTR_HEADER_SIZE, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = false}, {.name = "status", .bytes = 20 + VARSTR_HEADER_SIZE, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = false}, + {.name = "stage", .bytes = 4, .type = TSDB_DATA_TYPE_INT, .sysInfo = false}, }; static const SSysDbTableSchema userTblsSchema[] = { diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index a2766843b8..7d7987b72b 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -1566,7 +1566,7 @@ static int32_t mndRetrieveStreamTask(SRpcMsg *pReq, SShowObj *pShow, SSDataBlock colDataSetVal(pColInfo, numOfRows, (const char *)&level, false); // status - char status[20 + VARSTR_HEADER_SIZE] = {0}; + char status[20 + VARSTR_HEADER_SIZE] = {0}; STaskId id = {.streamId = pTask->id.streamId, .taskId = pTask->id.taskId}; STaskStatusEntry* pe = taosHashGet(execNodeList.pTaskMap, &id, sizeof(id)); @@ -1580,6 +1580,9 @@ static int32_t mndRetrieveStreamTask(SRpcMsg *pReq, SShowObj *pShow, SSDataBlock pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); colDataSetVal(pColInfo, numOfRows, (const char *)&status, false); + pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); + colDataSetVal(pColInfo, numOfRows, (const char *)&pe->stage, false); + numOfRows++; } } From 27e00b1f19f520119d855652c9d500418729dda4 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Fri, 22 Sep 2023 10:53:03 +0800 Subject: [PATCH 063/158] fix(stream): add some logs, and remove invalid assert. --- include/libs/stream/tstream.h | 2 +- source/common/src/systable.c | 4 ++-- source/dnode/snode/src/snode.c | 4 ++-- source/dnode/vnode/src/tq/tq.c | 4 ++-- source/libs/stream/src/stream.c | 4 ++-- 5 files changed, 9 insertions(+), 9 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index d75cfde8d8..bd2b51e9b2 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -636,7 +636,7 @@ void tDeleteStreamDispatchReq(SStreamDispatchReq* pReq); int32_t streamSetupScheduleTrigger(SStreamTask* pTask); int32_t streamProcessRunReq(SStreamTask* pTask); -int32_t streamProcessDispatchMsg(SStreamTask* pTask, SStreamDispatchReq* pReq, SRpcMsg* pMsg, bool exec); +int32_t streamProcessDispatchMsg(SStreamTask* pTask, SStreamDispatchReq* pReq, SRpcMsg* pMsg); int32_t streamProcessDispatchRsp(SStreamTask* pTask, SStreamDispatchRsp* pRsp, int32_t code); int32_t streamProcessRetrieveReq(SStreamTask* pTask, SStreamRetrieveReq* pReq, SRpcMsg* pMsg); diff --git a/source/common/src/systable.c b/source/common/src/systable.c index 9ceaa067f1..4118dfd484 100644 --- a/source/common/src/systable.c +++ b/source/common/src/systable.c @@ -158,8 +158,8 @@ static const SSysDbTableSchema streamSchema[] = { static const SSysDbTableSchema streamTaskSchema[] = { {.name = "stream_name", .bytes = SYSTABLE_SCH_DB_NAME_LEN, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = false}, - {.name = "task_id", .bytes = 32, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = false}, - {.name = "node_type", .bytes = SYSTABLE_SCH_DB_NAME_LEN, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = false}, + {.name = "task_id", .bytes = 16 + VARSTR_HEADER_SIZE, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = false}, + {.name = "node_type", .bytes = 20 + VARSTR_HEADER_SIZE, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = false}, {.name = "node_id", .bytes = 4, .type = TSDB_DATA_TYPE_INT, .sysInfo = false}, {.name = "level", .bytes = 20 + VARSTR_HEADER_SIZE, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = false}, {.name = "status", .bytes = 20 + VARSTR_HEADER_SIZE, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = false}, diff --git a/source/dnode/snode/src/snode.c b/source/dnode/snode/src/snode.c index d6d751304e..3c6518bd2f 100644 --- a/source/dnode/snode/src/snode.c +++ b/source/dnode/snode/src/snode.c @@ -41,7 +41,7 @@ void sndEnqueueStreamDispatch(SSnode *pSnode, SRpcMsg *pMsg) { .info = pMsg->info, .code = 0, }; - streamProcessDispatchMsg(pTask, &req, &rsp, false); + streamProcessDispatchMsg(pTask, &req, &rsp); streamMetaReleaseTask(pSnode->pMeta, pTask); rpcFreeCont(pMsg->pCont); taosFreeQitem(pMsg); @@ -228,7 +228,7 @@ int32_t sndProcessTaskDispatchReq(SSnode *pSnode, SRpcMsg *pMsg, bool exec) { SStreamTask *pTask = streamMetaAcquireTask(pSnode->pMeta, req.streamId, req.taskId); if (pTask) { SRpcMsg rsp = {.info = pMsg->info, .code = 0}; - streamProcessDispatchMsg(pTask, &req, &rsp, exec); + streamProcessDispatchMsg(pTask, &req, &rsp); streamMetaReleaseTask(pSnode->pMeta, pTask); return 0; } else { diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 327b242c93..629f5b5210 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1353,7 +1353,7 @@ int32_t tqProcessTaskDispatchReq(STQ* pTq, SRpcMsg* pMsg, bool exec) { SStreamTask* pTask = streamMetaAcquireTask(pTq->pStreamMeta, req.streamId, req.taskId); if (pTask) { SRpcMsg rsp = {.info = pMsg->info, .code = 0}; - streamProcessDispatchMsg(pTask, &req, &rsp, exec); + streamProcessDispatchMsg(pTask, &req, &rsp); streamMetaReleaseTask(pTq->pStreamMeta, pTask); return 0; } else { @@ -1563,7 +1563,7 @@ int32_t vnodeEnqueueStreamMsg(SVnode* pVnode, SRpcMsg* pMsg) { SStreamTask* pTask = streamMetaAcquireTask(pTq->pStreamMeta, req.streamId, taskId); if (pTask != NULL) { SRpcMsg rsp = {.info = pMsg->info, .code = 0}; - streamProcessDispatchMsg(pTask, &req, &rsp, false); + streamProcessDispatchMsg(pTask, &req, &rsp); streamMetaReleaseTask(pTq->pStreamMeta, pTask); rpcFreeCont(pMsg->pCont); taosFreeQitem(pMsg); diff --git a/source/libs/stream/src/stream.c b/source/libs/stream/src/stream.c index 2f3bf147d1..99d61c2348 100644 --- a/source/libs/stream/src/stream.c +++ b/source/libs/stream/src/stream.c @@ -210,7 +210,7 @@ int32_t streamTaskEnqueueRetrieve(SStreamTask* pTask, SStreamRetrieveReq* pReq, return status == TASK_INPUT_STATUS__NORMAL ? 0 : -1; } -int32_t streamProcessDispatchMsg(SStreamTask* pTask, SStreamDispatchReq* pReq, SRpcMsg* pRsp, bool exec) { +int32_t streamProcessDispatchMsg(SStreamTask* pTask, SStreamDispatchReq* pReq, SRpcMsg* pRsp) { stDebug("s-task:%s receive dispatch msg from taskId:0x%x(vgId:%d), msgLen:%" PRId64, pTask->id.idStr, pReq->upstreamTaskId, pReq->upstreamNodeId, pReq->totalLen); int32_t status = 0; @@ -219,7 +219,7 @@ int32_t streamProcessDispatchMsg(SStreamTask* pTask, SStreamDispatchReq* pReq, S ASSERT(pInfo != NULL); if (!pTask->pMeta->leader) { - ASSERT(0); + stError("s-task:%s task on follower received dispatch msgs, should discard it, not now", pTask->id.idStr); } // upstream task has restarted/leader-follower switch/transferred to other dnodes From 498519c94dfa9a1c08fae38a6d8ead7a72684ace Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Fri, 22 Sep 2023 11:45:46 +0800 Subject: [PATCH 064/158] fix(stream): remove invalid node. --- source/common/src/systable.c | 2 +- source/dnode/mnode/impl/src/mndStream.c | 17 +++++++++++++++++ 2 files changed, 18 insertions(+), 1 deletion(-) diff --git a/source/common/src/systable.c b/source/common/src/systable.c index 4118dfd484..7107f0e058 100644 --- a/source/common/src/systable.c +++ b/source/common/src/systable.c @@ -159,7 +159,7 @@ static const SSysDbTableSchema streamSchema[] = { static const SSysDbTableSchema streamTaskSchema[] = { {.name = "stream_name", .bytes = SYSTABLE_SCH_DB_NAME_LEN, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = false}, {.name = "task_id", .bytes = 16 + VARSTR_HEADER_SIZE, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = false}, - {.name = "node_type", .bytes = 20 + VARSTR_HEADER_SIZE, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = false}, + {.name = "node_type", .bytes = 10 + VARSTR_HEADER_SIZE, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = false}, {.name = "node_id", .bytes = 4, .type = TSDB_DATA_TYPE_INT, .sysInfo = false}, {.name = "level", .bytes = 20 + VARSTR_HEADER_SIZE, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = false}, {.name = "status", .bytes = 20 + VARSTR_HEADER_SIZE, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = false}, diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index 7d7987b72b..30e0791294 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -2229,6 +2229,23 @@ static int32_t removeInvalidStreamTask(SArray *pNodeSnapshot) { doRemoveFromTask(&execNodeList, pId); } + int32_t size = taosArrayGetSize(pNodeSnapshot); + SArray* pValidNodeEntryList = taosArrayInit(4, sizeof(SNodeEntry)); + for(int32_t i = 0; i < taosArrayGetSize(execNodeList.pNodeEntryList); ++i) { + SNodeEntry* pExisted = taosArrayGet(execNodeList.pNodeEntryList, i); + + for(int32_t j = 0; j < size; ++j) { + SNodeEntry* pEntry = taosArrayGet(pNodeSnapshot, j); + if (pEntry->nodeId == pExisted->nodeId) { + taosArrayPush(pValidNodeEntryList, pExisted); + break; + } + } + } + + execNodeList.pNodeEntryList = taosArrayDestroy(execNodeList.pNodeEntryList); + execNodeList.pNodeEntryList = pValidNodeEntryList; + return 0; } From 2b169769021eb30b90a69a14e8d62464e8dbfe81 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Fri, 22 Sep 2023 14:52:18 +0800 Subject: [PATCH 065/158] refactor(stream): refactor send msg. --- include/libs/stream/tstream.h | 10 +- source/dnode/vnode/src/tq/tq.c | 4 +- source/libs/stream/inc/streamInt.h | 2 + source/libs/stream/src/streamDispatch.c | 217 ++++++++++++++++-------- source/libs/stream/src/streamTask.c | 3 +- 5 files changed, 157 insertions(+), 79 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index bd2b51e9b2..b2927d839d 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -290,8 +290,11 @@ typedef struct SSTaskBasicInfo { int64_t triggerParam; // in msec } SSTaskBasicInfo; +typedef struct SStreamDispatchReq SStreamDispatchReq; + typedef struct SDispatchMsgInfo { - void* pData; // current dispatch data + SStreamDispatchReq* pData; // current dispatch data + int8_t dispatchMsgType; int16_t msgType; // dispatch msg type int32_t retryCount; // retry send data count int64_t blockingTs; // output blocking timestamp @@ -327,6 +330,7 @@ typedef struct { int64_t step2Start; int64_t start; int32_t updateCount; + int32_t dispatchCount; int64_t latestUpdateTs; } STaskExecStatisInfo; @@ -442,7 +446,7 @@ typedef struct { int32_t taskId; } SStreamTaskRunReq; -typedef struct { +struct SStreamDispatchReq { int32_t type; int64_t stage; // nodeId from upstream task int64_t streamId; @@ -455,7 +459,7 @@ typedef struct { int64_t totalLen; SArray* dataLen; // SArray SArray* data; // SArray -} SStreamDispatchReq; +}; typedef struct { int64_t streamId; diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 629f5b5210..55ce4b4b33 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -910,12 +910,12 @@ int32_t tqProcessStreamTaskCheckReq(STQ* pTq, SRpcMsg* pMsg) { streamMetaReleaseTask(pMeta, pTask); const char* pStatus = streamGetTaskStatusStr(pTask->status.taskStatus); - tqDebug("s-task:%s status:%s, stage:%d recv task check req(reqId:0x%" PRIx64 ") task:0x%x (vgId:%d), ready:%d", + tqDebug("s-task:%s status:%s, stage:%d recv task check req(reqId:0x%" PRIx64 ") task:0x%x (vgId:%d), check_status:%d", pTask->id.idStr, pStatus, rsp.oldStage, rsp.reqId, rsp.upstreamTaskId, rsp.upstreamNodeId, rsp.status); } else { rsp.status = TASK_DOWNSTREAM_NOT_READY; tqDebug("tq recv task check(taskId:0x%" PRIx64 "-0x%x not built yet) req(reqId:0x%" PRIx64 - ") from task:0x%x (vgId:%d), rsp status %d", + ") from task:0x%x (vgId:%d), rsp check_status %d", req.streamId, taskId, rsp.reqId, rsp.upstreamTaskId, rsp.upstreamNodeId, rsp.status); } } diff --git a/source/libs/stream/inc/streamInt.h b/source/libs/stream/inc/streamInt.h index 40cadd3387..10a7dc7be7 100644 --- a/source/libs/stream/inc/streamInt.h +++ b/source/libs/stream/inc/streamInt.h @@ -59,6 +59,8 @@ extern int32_t streamBackendCfWrapperId; void streamRetryDispatchStreamBlock(SStreamTask* pTask, int64_t waitDuration); int32_t streamDispatchStreamBlock(SStreamTask* pTask); +void destroyDispatchMsg(SStreamDispatchReq* pReq, int32_t numOfVgroups); +int32_t getNumOfDispatchBranch(SStreamTask* pTask); int32_t streamProcessCheckpointBlock(SStreamTask* pTask, SStreamDataBlock* pBlock); SStreamDataBlock* createStreamBlockFromDispatchMsg(const SStreamDispatchReq* pReq, int32_t blockType, int32_t srcVg); diff --git a/source/libs/stream/src/streamDispatch.c b/source/libs/stream/src/streamDispatch.c index 519271703b..2775a90abf 100644 --- a/source/libs/stream/src/streamDispatch.c +++ b/source/libs/stream/src/streamDispatch.c @@ -277,59 +277,66 @@ int32_t streamDispatchCheckMsg(SStreamTask* pTask, const SStreamTaskCheckReq* pR return 0; } -static int32_t doDispatchAllBlocks(SStreamTask* pTask, const SStreamDataBlock* pData) { +void destroyDispatchMsg(SStreamDispatchReq* pReq, int32_t numOfVgroups) { + for (int32_t i = 0; i < numOfVgroups; i++) { + taosArrayDestroyP(pReq[i].data, taosMemoryFree); + taosArrayDestroy(pReq[i].dataLen); + } + + taosMemoryFree(pReq); +} + +int32_t getNumOfDispatchBranch(SStreamTask* pTask) { + return (pTask->outputInfo.type == TASK_OUTPUT__FIXED_DISPATCH) + ? 1 + : taosArrayGetSize(pTask->shuffleDispatcher.dbInfo.pVgroupInfos); +} + +static int32_t doBuildDispatchMsg(SStreamTask* pTask, const SStreamDataBlock* pData) { int32_t code = 0; int32_t numOfBlocks = taosArrayGetSize(pData->blocks); - ASSERT(numOfBlocks != 0); + ASSERT(numOfBlocks != 0 && pTask->msgInfo.pData == NULL); + + pTask->msgInfo.dispatchMsgType = pData->type; if (pTask->outputInfo.type == TASK_OUTPUT__FIXED_DISPATCH) { - SStreamDispatchReq req = {0}; + SStreamDispatchReq* pReq = taosMemoryCalloc(1, sizeof(SStreamDispatchReq)); int32_t downstreamTaskId = pTask->fixedEpDispatcher.taskId; - code = tInitStreamDispatchReq(&req, pTask, pData->srcVgId, numOfBlocks, downstreamTaskId, pData->type); + code = tInitStreamDispatchReq(pReq, pTask, pData->srcVgId, numOfBlocks, downstreamTaskId, pData->type); if (code != TSDB_CODE_SUCCESS) { return code; } for (int32_t i = 0; i < numOfBlocks; i++) { SSDataBlock* pDataBlock = taosArrayGet(pData->blocks, i); - - code = streamAddBlockIntoDispatchMsg(pDataBlock, &req); + code = streamAddBlockIntoDispatchMsg(pDataBlock, pReq); if (code != TSDB_CODE_SUCCESS) { - taosArrayDestroyP(req.data, taosMemoryFree); - taosArrayDestroy(req.dataLen); + destroyDispatchMsg(pReq, 1); return code; } } - int32_t vgId = pTask->fixedEpDispatcher.nodeId; - SEpSet* pEpSet = &pTask->fixedEpDispatcher.epSet; - - stDebug("s-task:%s (child taskId:%d) fix-dispatch %d block(s) to s-task:0x%x (vgId:%d)", pTask->id.idStr, - pTask->info.selfChildId, numOfBlocks, downstreamTaskId, vgId); - - code = doSendDispatchMsg(pTask, &req, vgId, pEpSet); - taosArrayDestroyP(req.data, taosMemoryFree); - taosArrayDestroy(req.dataLen); - return code; + pTask->msgInfo.pData = pReq; } else if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { int32_t rspCnt = atomic_load_32(&pTask->shuffleDispatcher.waitingRspCnt); ASSERT(rspCnt == 0); SArray* vgInfo = pTask->shuffleDispatcher.dbInfo.pVgroupInfos; - int32_t vgSz = taosArrayGetSize(vgInfo); + int32_t numOfVgroups = taosArrayGetSize(vgInfo); - SStreamDispatchReq* pReqs = taosMemoryCalloc(vgSz, sizeof(SStreamDispatchReq)); + SStreamDispatchReq* pReqs = taosMemoryCalloc(numOfVgroups, sizeof(SStreamDispatchReq)); if (pReqs == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; return -1; } - for (int32_t i = 0; i < vgSz; i++) { + for (int32_t i = 0; i < numOfVgroups; i++) { SVgroupInfo* pVgInfo = taosArrayGet(vgInfo, i); code = tInitStreamDispatchReq(&pReqs[i], pTask, pData->srcVgId, 0, pVgInfo->taskId, pData->type); if (code != TSDB_CODE_SUCCESS) { - goto FAIL_SHUFFLE_DISPATCH; + destroyDispatchMsg(pReqs, numOfVgroups); + return code; } } @@ -338,52 +345,95 @@ static int32_t doDispatchAllBlocks(SStreamTask* pTask, const SStreamDataBlock* p // TODO: do not use broadcast if (pDataBlock->info.type == STREAM_DELETE_RESULT || pDataBlock->info.type == STREAM_CHECKPOINT || pDataBlock->info.type == STREAM_TRANS_STATE) { - for (int32_t j = 0; j < vgSz; j++) { - if (streamAddBlockIntoDispatchMsg(pDataBlock, &pReqs[j]) < 0) { - goto FAIL_SHUFFLE_DISPATCH; + for (int32_t j = 0; j < numOfVgroups; j++) { + code = streamAddBlockIntoDispatchMsg(pDataBlock, &pReqs[j]); + if (code != 0) { + destroyDispatchMsg(pReqs, numOfVgroups); + return code; } if (pReqs[j].blockNum == 0) { atomic_add_fetch_32(&pTask->shuffleDispatcher.waitingRspCnt, 1); } + pReqs[j].blockNum++; } continue; } - if (streamSearchAndAddBlock(pTask, pReqs, pDataBlock, vgSz, pDataBlock->info.id.groupId) < 0) { - goto FAIL_SHUFFLE_DISPATCH; + code = streamSearchAndAddBlock(pTask, pReqs, pDataBlock, numOfVgroups, pDataBlock->info.id.groupId); + if(code != 0) { + destroyDispatchMsg(pReqs, numOfVgroups); + return code; } } - stDebug("s-task:%s (child taskId:%d) shuffle-dispatch blocks:%d to %d vgroups", pTask->id.idStr, - pTask->info.selfChildId, numOfBlocks, vgSz); + pTask->msgInfo.pData = pReqs; +// *pDispatchReq = pReqs; - for (int32_t i = 0; i < vgSz; i++) { - if (pReqs[i].blockNum > 0) { +// stDebug("s-task:%s (child taskId:%d) shuffle-dispatch blocks:%d to %d vgroup(s), msgId:%d", pTask->id.idStr, +// pTask->info.selfChildId, numOfBlocks, numOfVgroups, msgId); +// +// for (int32_t i = 0; i < numOfVgroups; i++) { +// if (pReqs[i].blockNum > 0) { +// SVgroupInfo* pVgInfo = taosArrayGet(vgInfo, i); +// stDebug("s-task:%s (child taskId:%d) shuffle-dispatch blocks:%d to vgId:%d", pTask->id.idStr, +// pTask->info.selfChildId, pReqs[i].blockNum, pVgInfo->vgId); +// +// code = doSendDispatchMsg(pTask, &pReqs[i], pVgInfo->vgId, &pVgInfo->epSet); +// if (code < 0) { +// destroyDispatchMsg(pReqs, numOfVgroups); +// return code; +// } +// } +// } +// +// stDebug("s-task:%s complete shuffle-dispatch blocks to all %d vnodes, msgId:%d", pTask->id.idStr, numOfVgroups, msgId); +// code = 0; +// +// *pDispatchReq = pReqs; + } + + stDebug("s-task:%s build dispatch msg success, msgId:%d", pTask->id.idStr, pTask->taskExecInfo.dispatchCount); + return code; +} + +static int32_t sendDispatchMsg(SStreamTask* pTask, SStreamDispatchReq* pDispatchMsg) { + int32_t code = 0; + int32_t msgId = pTask->taskExecInfo.dispatchCount; + const char* id = pTask->id.idStr; + + if (pTask->outputInfo.type == TASK_OUTPUT__FIXED_DISPATCH) { + int32_t vgId = pTask->fixedEpDispatcher.nodeId; + SEpSet* pEpSet = &pTask->fixedEpDispatcher.epSet; + int32_t downstreamTaskId = pTask->fixedEpDispatcher.taskId; + + stDebug("s-task:%s (child taskId:%d) fix-dispatch %d block(s) to s-task:0x%x (vgId:%d), id:%d", id, + pTask->info.selfChildId, 1, downstreamTaskId, vgId, msgId); + + code = doSendDispatchMsg(pTask, pDispatchMsg, vgId, pEpSet); + } else { + SArray* vgInfo = pTask->shuffleDispatcher.dbInfo.pVgroupInfos; + int32_t numOfVgroups = taosArrayGetSize(vgInfo); + + stDebug("s-task:%s (child taskId:%d) start to shuffle-dispatch blocks to %d vgroup(s), msgId:%d", + id, pTask->info.selfChildId, numOfVgroups, msgId); + + for (int32_t i = 0; i < numOfVgroups; i++) { + if (pDispatchMsg[i].blockNum > 0) { SVgroupInfo* pVgInfo = taosArrayGet(vgInfo, i); stDebug("s-task:%s (child taskId:%d) shuffle-dispatch blocks:%d to vgId:%d", pTask->id.idStr, - pTask->info.selfChildId, pReqs[i].blockNum, pVgInfo->vgId); + pTask->info.selfChildId, pDispatchMsg[i].blockNum, pVgInfo->vgId); - code = doSendDispatchMsg(pTask, &pReqs[i], pVgInfo->vgId, &pVgInfo->epSet); + code = doSendDispatchMsg(pTask, &pDispatchMsg[i], pVgInfo->vgId, &pVgInfo->epSet); if (code < 0) { - goto FAIL_SHUFFLE_DISPATCH; + break; } } } - stDebug("s-task:%s complete shuffle-dispatch blocks to all %d vnodes", pTask->id.idStr, vgSz); - - code = 0; - - FAIL_SHUFFLE_DISPATCH: - for (int32_t i = 0; i < vgSz; i++) { - taosArrayDestroyP(pReqs[i].data, taosMemoryFree); - taosArrayDestroy(pReqs[i].dataLen); - } - - taosMemoryFree(pReqs); + stDebug("s-task:%s complete shuffle-dispatch blocks to all %d vnodes, msgId:%d", pTask->id.idStr, numOfVgroups, msgId); } return code; @@ -400,7 +450,7 @@ static void doRetryDispatchData(void* param, void* tmrId) { ASSERT(pTask->outputInfo.status == TASK_OUTPUT_STATUS__WAIT); - int32_t code = doDispatchAllBlocks(pTask, pTask->msgInfo.pData); + int32_t code = sendDispatchMsg(pTask, pTask->msgInfo.pData); if (code != TSDB_CODE_SUCCESS) { if (!streamTaskShouldStop(&pTask->status)) { stDebug("s-task:%s reset the waitRspCnt to be 0 before launch retry dispatch", pTask->id.idStr); @@ -524,25 +574,31 @@ int32_t streamDispatchStreamBlock(SStreamTask* pTask) { return 0; } - pTask->msgInfo.pData = pBlock; ASSERT(pBlock->type == STREAM_INPUT__DATA_BLOCK || pBlock->type == STREAM_INPUT__CHECKPOINT_TRIGGER || pBlock->type == STREAM_INPUT__TRANS_STATE); int32_t retryCount = 0; + pTask->taskExecInfo.dispatchCount += 1; + + int32_t code = doBuildDispatchMsg(pTask, pBlock); + if (code == 0) { + destroyStreamDataBlock(pBlock); + } else { // todo handle build dispatch msg failed + } while (1) { - int32_t code = doDispatchAllBlocks(pTask, pBlock); + code = sendDispatchMsg(pTask, pTask->msgInfo.pData); if (code == TSDB_CODE_SUCCESS) { break; } - stDebug("s-task:%s failed to dispatch msg to downstream, code:%s, output status:%d, retry cnt:%d", id, - tstrerror(terrno), pTask->outputInfo.status, retryCount); + stDebug("s-task:%s failed to dispatch msg:%d to downstream, code:%s, output status:%d, retry cnt:%d", id, + pTask->taskExecInfo.dispatchCount, tstrerror(terrno), pTask->outputInfo.status, retryCount); // todo deal with only partially success dispatch case atomic_store_32(&pTask->shuffleDispatcher.waitingRspCnt, 0); if (terrno == TSDB_CODE_APP_IS_STOPPING) { // in case of this error, do not retry anymore - destroyStreamDataBlock(pTask->msgInfo.pData); + destroyDispatchMsg(pTask->msgInfo.pData, getNumOfDispatchBranch(pTask)); pTask->msgInfo.pData = NULL; return code; } @@ -552,6 +608,7 @@ int32_t streamDispatchStreamBlock(SStreamTask* pTask) { stDebug("s-task:%s failed to dispatch msg to downstream for %d times, code:%s, add timer to retry in %dms, ref:%d", pTask->id.idStr, retryCount, tstrerror(terrno), DISPATCH_RETRY_INTERVAL_MS, ref); + streamRetryDispatchStreamBlock(pTask, DISPATCH_RETRY_INTERVAL_MS); break; } @@ -951,7 +1008,7 @@ int32_t streamNotifyUpstreamContinue(SStreamTask* pTask) { // this message has been sent successfully, let's try next one. static int32_t handleDispatchSuccessRsp(SStreamTask* pTask, int32_t downstreamId) { - destroyStreamDataBlock(pTask->msgInfo.pData); + destroyDispatchMsg(pTask->msgInfo.pData, getNumOfDispatchBranch(pTask)); pTask->msgInfo.pData = NULL; if (pTask->msgInfo.blockingTs != 0) { @@ -974,6 +1031,7 @@ static int32_t handleDispatchSuccessRsp(SStreamTask* pTask, int32_t downstreamId int32_t streamProcessDispatchRsp(SStreamTask* pTask, SStreamDispatchRsp* pRsp, int32_t code) { const char* id = pTask->id.idStr; + int32_t msgId = pTask->taskExecInfo.dispatchCount; if (code != TSDB_CODE_SUCCESS) { // dispatch message failed: network error, or node not available. @@ -982,14 +1040,14 @@ int32_t streamProcessDispatchRsp(SStreamTask* pTask, SStreamDispatchRsp* pRsp, i // happened too fast. // todo handle the shuffle dispatch failure if (code == TSDB_CODE_STREAM_TASK_NOT_EXIST) { // destination task does not exist, not retry anymore - stError("s-task:%s failed to dispatch msg to task:0x%x(vgId:%d), no retry, since it is destroyed already", id, - pRsp->downstreamTaskId, pRsp->downstreamNodeId); + stError("s-task:%s failed to dispatch msg to task:0x%x(vgId:%d), msgId:%d no retry, since task destroyed already", id, + pRsp->downstreamTaskId, pRsp->downstreamNodeId, msgId); - SStreamDataBlock* pMsgBlock = pTask->msgInfo.pData; - if (pMsgBlock->type == STREAM_INPUT__CHECKPOINT_TRIGGER) { - stError("s-task:%s checkpoint trigger send failed, continue do checkpoint ready process", id); - streamProcessCheckpointReadyMsg(pTask); - } +// SStreamDataBlock* pMsgBlock = pTask->msgInfo.pData; +// if (pMsgBlock->type == STREAM_INPUT__CHECKPOINT_TRIGGER) { +// stError("s-task:%s checkpoint trigger send failed, continue do checkpoint ready process", id); +// streamProcessCheckpointReadyMsg(pTask); +// } // we should set the correct finish flag to make sure the shuffle dispatch will be executed completed. if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { @@ -1002,11 +1060,23 @@ int32_t streamProcessDispatchRsp(SStreamTask* pTask, SStreamDispatchRsp* pRsp, i } } } else { - stError("s-task:%s failed to dispatch msg to task:0x%x(vgId:%d), code:%s, retry cnt:%d", id, - pRsp->downstreamTaskId, pRsp->downstreamNodeId, tstrerror(code), ++pTask->msgInfo.retryCount); + stError("s-task:%s failed to dispatch msgId:%d to task:0x%x(vgId:%d), code:%s, retry", id, msgId, + pRsp->downstreamTaskId, pRsp->downstreamNodeId, tstrerror(code)); + SStreamDispatchReq* pDispatchMsg = pTask->msgInfo.pData; - int32_t ret = doDispatchAllBlocks(pTask, pTask->msgInfo.pData); - if (ret != TSDB_CODE_SUCCESS) { + if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { + SArray* vgInfo = pTask->shuffleDispatcher.dbInfo.pVgroupInfos; + int32_t numOfVgroups = taosArrayGetSize(vgInfo); + for(int32_t i = 0; i < numOfVgroups; ++i) { + SVgroupInfo* pVgInfo = taosArrayGet(vgInfo, i); + if (pVgInfo->vgId == pRsp->downstreamNodeId) { + stDebug("s-task:%s (child taskId:%d) re-send blocks:%d to vgId:%d", pTask->id.idStr, + pTask->info.selfChildId, pDispatchMsg[i].blockNum, pVgInfo->vgId); + code = doSendDispatchMsg(pTask, &pDispatchMsg[i], pVgInfo->vgId, &pVgInfo->epSet); + } + } + } else { + sendDispatchMsg(pTask, pTask->msgInfo.pData); } } @@ -1017,22 +1087,23 @@ int32_t streamProcessDispatchRsp(SStreamTask* pTask, SStreamDispatchRsp* pRsp, i if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { int32_t leftRsp = atomic_sub_fetch_32(&pTask->shuffleDispatcher.waitingRspCnt, 1); if (leftRsp > 0) { - stDebug("s-task:%s recv dispatch rsp from 0x%x(vgId:%d), downstream task input status:%d code:%d, waiting for %d rsp", id, pRsp->downstreamTaskId, - pRsp->downstreamNodeId, pRsp->inputStatus, code, leftRsp); + stDebug( + "s-task:%s recv dispatch rsp, msgId:%d from 0x%x(vgId:%d), downstream task input status:%d code:%d, waiting for %d " + "rsp", + id, msgId, pRsp->downstreamTaskId, pRsp->downstreamNodeId, pRsp->inputStatus, code, leftRsp); return 0; } else { - stDebug("s-task:%s recv dispatch rsp from 0x%x(vgId:%d), downstream task input status:%d code:%d, all rsp", id, - pRsp->downstreamTaskId, pRsp->downstreamNodeId, pRsp->inputStatus, code); + stDebug("s-task:%s recv dispatch rsp, msgId:%d from 0x%x(vgId:%d), downstream task input status:%d code:%d, all rsp", id, + msgId, pRsp->downstreamTaskId, pRsp->downstreamNodeId, pRsp->inputStatus, code); } } else { - stDebug("s-task:%s recv fix-dispatch rsp from 0x%x(vgId:%d), downstream task input status:%d code:%d", id, - pRsp->downstreamTaskId, pRsp->downstreamNodeId, pRsp->inputStatus, code); + stDebug("s-task:%s recv fix-dispatch rsp, msgId:%d from 0x%x(vgId:%d), downstream task input status:%d code:%d", + id, msgId, pRsp->downstreamTaskId, pRsp->downstreamNodeId, pRsp->inputStatus, code); } // transtate msg has been sent to downstream successfully. let's transfer the fill-history task state - SStreamDataBlock* p = pTask->msgInfo.pData; - if (p->type == STREAM_INPUT__TRANS_STATE) { - stDebug("s-task:%s dispatch transtate msg to downstream successfully, start to transfer state", id); + if (pTask->msgInfo.dispatchMsgType == STREAM_INPUT__TRANS_STATE) { + stDebug("s-task:%s dispatch transtate msgId:%d to downstream successfully, start to transfer state", id, msgId); ASSERT(pTask->info.fillHistory == 1); code = streamTransferStateToStreamTask(pTask); @@ -1066,7 +1137,7 @@ int32_t streamProcessDispatchRsp(SStreamTask* pTask, SStreamDispatchRsp* pRsp, i " wait for %dms and retry dispatch data, total wait:%.2fSec ref:%d", id, pRsp->downstreamTaskId, pTask->msgInfo.blockingTs, DISPATCH_RETRY_INTERVAL_MS, el, ref); streamRetryDispatchStreamBlock(pTask, DISPATCH_RETRY_INTERVAL_MS); - } else { // pipeline send data in output queue + } else { // this message has been sent successfully, let's try next one. handleDispatchSuccessRsp(pTask, pRsp->downstreamTaskId); } diff --git a/source/libs/stream/src/streamTask.c b/source/libs/stream/src/streamTask.c index 777e93da47..8d651c43a0 100644 --- a/source/libs/stream/src/streamTask.c +++ b/source/libs/stream/src/streamTask.c @@ -358,8 +358,9 @@ void tFreeStreamTask(SStreamTask* pTask) { pTask->pReadyMsgList = taosArrayDestroy(pTask->pReadyMsgList); if (pTask->msgInfo.pData != NULL) { - destroyStreamDataBlock(pTask->msgInfo.pData); + destroyDispatchMsg(pTask->msgInfo.pData, getNumOfDispatchBranch(pTask)); pTask->msgInfo.pData = NULL; + pTask->msgInfo.dispatchMsgType = 0; } if (pTask->id.idStr != NULL) { From e0d19af00013d9b8b6394c83a08e14678aa1daa4 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Fri, 22 Sep 2023 15:57:55 +0800 Subject: [PATCH 066/158] fix(stream): refactor re-try dispatch msg for stream tasks. --- include/libs/stream/tstream.h | 3 +- source/libs/stream/src/streamDispatch.c | 233 ++++++++++++------------ source/libs/stream/src/streamTask.c | 2 + 3 files changed, 122 insertions(+), 116 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index b2927d839d..d7ad1ddf08 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -297,7 +297,8 @@ typedef struct SDispatchMsgInfo { int8_t dispatchMsgType; int16_t msgType; // dispatch msg type int32_t retryCount; // retry send data count - int64_t blockingTs; // output blocking timestamp + int64_t startTs; // output blocking timestamp + SArray* pRetryList; // current dispatch successfully completed node of downstream } SDispatchMsgInfo; typedef struct STaskOutputInfo { diff --git a/source/libs/stream/src/streamDispatch.c b/source/libs/stream/src/streamDispatch.c index 2775a90abf..a3eda3ceb2 100644 --- a/source/libs/stream/src/streamDispatch.c +++ b/source/libs/stream/src/streamDispatch.c @@ -370,29 +370,6 @@ static int32_t doBuildDispatchMsg(SStreamTask* pTask, const SStreamDataBlock* pD } pTask->msgInfo.pData = pReqs; -// *pDispatchReq = pReqs; - -// stDebug("s-task:%s (child taskId:%d) shuffle-dispatch blocks:%d to %d vgroup(s), msgId:%d", pTask->id.idStr, -// pTask->info.selfChildId, numOfBlocks, numOfVgroups, msgId); -// -// for (int32_t i = 0; i < numOfVgroups; i++) { -// if (pReqs[i].blockNum > 0) { -// SVgroupInfo* pVgInfo = taosArrayGet(vgInfo, i); -// stDebug("s-task:%s (child taskId:%d) shuffle-dispatch blocks:%d to vgId:%d", pTask->id.idStr, -// pTask->info.selfChildId, pReqs[i].blockNum, pVgInfo->vgId); -// -// code = doSendDispatchMsg(pTask, &pReqs[i], pVgInfo->vgId, &pVgInfo->epSet); -// if (code < 0) { -// destroyDispatchMsg(pReqs, numOfVgroups); -// return code; -// } -// } -// } -// -// stDebug("s-task:%s complete shuffle-dispatch blocks to all %d vnodes, msgId:%d", pTask->id.idStr, numOfVgroups, msgId); -// code = 0; -// -// *pDispatchReq = pReqs; } stDebug("s-task:%s build dispatch msg success, msgId:%d", pTask->id.idStr, pTask->taskExecInfo.dispatchCount); @@ -441,6 +418,8 @@ static int32_t sendDispatchMsg(SStreamTask* pTask, SStreamDispatchReq* pDispatch static void doRetryDispatchData(void* param, void* tmrId) { SStreamTask* pTask = param; + const char* id = pTask->id.idStr; + int32_t msgId = pTask->taskExecInfo.dispatchCount; if (streamTaskShouldStop(&pTask->status)) { int8_t ref = atomic_sub_fetch_8(&pTask->status.timerActive, 1); @@ -450,11 +429,53 @@ static void doRetryDispatchData(void* param, void* tmrId) { ASSERT(pTask->outputInfo.status == TASK_OUTPUT_STATUS__WAIT); - int32_t code = sendDispatchMsg(pTask, pTask->msgInfo.pData); + int32_t code = 0; + { + SStreamDispatchReq *pReq = pTask->msgInfo.pData; + + if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { + SArray* vgInfo = pTask->shuffleDispatcher.dbInfo.pVgroupInfos; + int32_t numOfVgroups = taosArrayGetSize(vgInfo); + + + int32_t numOfFailed = taosArrayGetSize(pTask->msgInfo.pRetryList); + stDebug("s-task:%s (child taskId:%d) re-try shuffle-dispatch blocks to %d vgroup(s), msgId:%d", + id, pTask->info.selfChildId, numOfFailed, msgId); + + for (int32_t i = 0; i < numOfFailed; i++) { + int32_t vgId = *(int32_t*) taosArrayGet(pTask->msgInfo.pRetryList, i); + + for(int32_t j = 0; j < numOfVgroups; ++j) { + SVgroupInfo* pVgInfo = taosArrayGet(vgInfo, j); + if (pVgInfo->vgId == vgId) { + stDebug("s-task:%s (child taskId:%d) shuffle-dispatch blocks:%d to vgId:%d", pTask->id.idStr, + pTask->info.selfChildId, pReq[j].blockNum, pVgInfo->vgId); + + code = doSendDispatchMsg(pTask, &pReq[j], pVgInfo->vgId, &pVgInfo->epSet); + if (code < 0) { + break; + } + } + } + } + + stDebug("s-task:%s complete re-try shuffle-dispatch blocks to all %d vnodes, msgId:%d", pTask->id.idStr, numOfFailed, msgId); + } else { + int32_t vgId = pTask->fixedEpDispatcher.nodeId; + SEpSet* pEpSet = &pTask->fixedEpDispatcher.epSet; + int32_t downstreamTaskId = pTask->fixedEpDispatcher.taskId; + + stDebug("s-task:%s (child taskId:%d) fix-dispatch %d block(s) to s-task:0x%x (vgId:%d), id:%d", id, + pTask->info.selfChildId, 1, downstreamTaskId, vgId, msgId); + + code = doSendDispatchMsg(pTask, pReq, vgId, pEpSet); + } + } + if (code != TSDB_CODE_SUCCESS) { if (!streamTaskShouldStop(&pTask->status)) { - stDebug("s-task:%s reset the waitRspCnt to be 0 before launch retry dispatch", pTask->id.idStr); - atomic_store_32(&pTask->shuffleDispatcher.waitingRspCnt, 0); +// stDebug("s-task:%s reset the waitRspCnt to be 0 before launch retry dispatch", pTask->id.idStr); +// atomic_store_32(&pTask->shuffleDispatcher.waitingRspCnt, 0); if (streamTaskShouldPause(&pTask->status)) { streamRetryDispatchStreamBlock(pTask, DISPATCH_RETRY_INTERVAL_MS * 10); } else { @@ -471,7 +492,9 @@ static void doRetryDispatchData(void* param, void* tmrId) { } void streamRetryDispatchStreamBlock(SStreamTask* pTask, int64_t waitDuration) { - stWarn("s-task:%s dispatch data in %" PRId64 "ms, in timer", pTask->id.idStr, waitDuration); + stWarn("s-task:%s retry send dispatch data in %" PRId64 "ms, in timer msgId:%d", pTask->id.idStr, waitDuration, + pTask->taskExecInfo.dispatchCount); + if (pTask->launchTaskTimer != NULL) { taosTmrReset(doRetryDispatchData, waitDuration, pTask, streamEnv.timer, &pTask->launchTaskTimer); } else { @@ -1011,12 +1034,14 @@ static int32_t handleDispatchSuccessRsp(SStreamTask* pTask, int32_t downstreamId destroyDispatchMsg(pTask->msgInfo.pData, getNumOfDispatchBranch(pTask)); pTask->msgInfo.pData = NULL; - if (pTask->msgInfo.blockingTs != 0) { - int64_t el = taosGetTimestampMs() - pTask->msgInfo.blockingTs; + if (pTask->msgInfo.startTs != 0) { + int64_t now = taosGetTimestampMs(); + + int64_t el = now - pTask->msgInfo.startTs; stDebug("s-task:%s downstream task:0x%x resume to normal from inputQ blocking, blocking time:%" PRId64 "ms", pTask->id.idStr, downstreamId, el); - pTask->msgInfo.blockingTs = 0; + pTask->msgInfo.startTs = now; // put data into inputQ of current task is also allowed pTask->inputInfo.status = TASK_INPUT_STATUS__NORMAL; } @@ -1033,12 +1058,28 @@ int32_t streamProcessDispatchRsp(SStreamTask* pTask, SStreamDispatchRsp* pRsp, i const char* id = pTask->id.idStr; int32_t msgId = pTask->taskExecInfo.dispatchCount; + int32_t leftRsp = 0; + + if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { + leftRsp = atomic_sub_fetch_32(&pTask->shuffleDispatcher.waitingRspCnt, 1); + if (leftRsp > 0) { + stDebug( "s-task:%s recv dispatch rsp, msgId:%d from 0x%x(vgId:%d), downstream task input status:%d code:%d, waiting for %d rsp", + id, msgId, pRsp->downstreamTaskId, pRsp->downstreamNodeId, pRsp->inputStatus, code, leftRsp); + } else { + stDebug( + "s-task:%s recv dispatch rsp, msgId:%d from 0x%x(vgId:%d), downstream task input status:%d code:%d, all rsp", + id, msgId, pRsp->downstreamTaskId, pRsp->downstreamNodeId, pRsp->inputStatus, code); + } + } else { + stDebug("s-task:%s recv fix-dispatch rsp, msgId:%d from 0x%x(vgId:%d), downstream task input status:%d code:%d", + id, msgId, pRsp->downstreamTaskId, pRsp->downstreamNodeId, pRsp->inputStatus, code); + } + if (code != TSDB_CODE_SUCCESS) { // dispatch message failed: network error, or node not available. // in case of the input queue is full, the code will be TSDB_CODE_SUCCESS, the and pRsp->inputStatus will be set // flag. Here we need to retry dispatch this message to downstream task immediately. handle the case the failure // happened too fast. - // todo handle the shuffle dispatch failure if (code == TSDB_CODE_STREAM_TASK_NOT_EXIST) { // destination task does not exist, not retry anymore stError("s-task:%s failed to dispatch msg to task:0x%x(vgId:%d), msgId:%d no retry, since task destroyed already", id, pRsp->downstreamTaskId, pRsp->downstreamNodeId, msgId); @@ -1048,98 +1089,60 @@ int32_t streamProcessDispatchRsp(SStreamTask* pTask, SStreamDispatchRsp* pRsp, i // stError("s-task:%s checkpoint trigger send failed, continue do checkpoint ready process", id); // streamProcessCheckpointReadyMsg(pTask); // } - - // we should set the correct finish flag to make sure the shuffle dispatch will be executed completed. - if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { - int32_t left = atomic_sub_fetch_32(&pTask->shuffleDispatcher.waitingRspCnt, 1); - if (left > 0) { // do nothing - stError("s-task:%s add the shuffle dispatch counter to complete the dispatch process", id); - } else { - stError("s-task:%s the last rsp is failed, ignore it and continue, roll-back will discard this msg", id); - handleDispatchSuccessRsp(pTask, pRsp->downstreamTaskId); - } - } } else { - stError("s-task:%s failed to dispatch msgId:%d to task:0x%x(vgId:%d), code:%s, retry", id, msgId, + stError("s-task:%s failed to dispatch msgId:%d to task:0x%x(vgId:%d), code:%s, add to retry list", id, msgId, pRsp->downstreamTaskId, pRsp->downstreamNodeId, tstrerror(code)); - SStreamDispatchReq* pDispatchMsg = pTask->msgInfo.pData; + taosArrayPush(pTask->msgInfo.pRetryList, &pRsp->downstreamNodeId); + } - if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { - SArray* vgInfo = pTask->shuffleDispatcher.dbInfo.pVgroupInfos; - int32_t numOfVgroups = taosArrayGetSize(vgInfo); - for(int32_t i = 0; i < numOfVgroups; ++i) { - SVgroupInfo* pVgInfo = taosArrayGet(vgInfo, i); - if (pVgInfo->vgId == pRsp->downstreamNodeId) { - stDebug("s-task:%s (child taskId:%d) re-send blocks:%d to vgId:%d", pTask->id.idStr, - pTask->info.selfChildId, pDispatchMsg[i].blockNum, pVgInfo->vgId); - code = doSendDispatchMsg(pTask, &pDispatchMsg[i], pVgInfo->vgId, &pVgInfo->epSet); - } - } - } else { - sendDispatchMsg(pTask, pTask->msgInfo.pData); + } else { // code == 0 + if (pRsp->inputStatus == TASK_INPUT_STATUS__BLOCKED) { + pTask->inputInfo.status = TASK_INPUT_STATUS__BLOCKED; + // block the input of current task, to push pressure to upstream + taosArrayPush(pTask->msgInfo.pRetryList, &pRsp->downstreamNodeId); + stError("s-task:%s inputQ of downstream task:0x%x(vgId:%d) is full, wait for %dms and retry dispatch data", id, + pRsp->downstreamTaskId, pRsp->downstreamNodeId, DISPATCH_RETRY_INTERVAL_MS); + } + + // transtate msg has been sent to downstream successfully. let's transfer the fill-history task state + if (pTask->msgInfo.dispatchMsgType == STREAM_INPUT__TRANS_STATE) { + stDebug("s-task:%s dispatch transtate msgId:%d to downstream successfully, start to transfer state", id, msgId); + ASSERT(pTask->info.fillHistory == 1); + + code = streamTransferStateToStreamTask(pTask); + if (code != TSDB_CODE_SUCCESS) { // todo: do nothing if error happens } - } - return TSDB_CODE_SUCCESS; + // now ready for next data output + atomic_store_8(&pTask->outputInfo.status, TASK_OUTPUT_STATUS__NORMAL); + return TSDB_CODE_SUCCESS; + } } - // there are other dispatch message not response yet - if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { - int32_t leftRsp = atomic_sub_fetch_32(&pTask->shuffleDispatcher.waitingRspCnt, 1); - if (leftRsp > 0) { - stDebug( - "s-task:%s recv dispatch rsp, msgId:%d from 0x%x(vgId:%d), downstream task input status:%d code:%d, waiting for %d " - "rsp", - id, msgId, pRsp->downstreamTaskId, pRsp->downstreamNodeId, pRsp->inputStatus, code, leftRsp); - return 0; - } else { - stDebug("s-task:%s recv dispatch rsp, msgId:%d from 0x%x(vgId:%d), downstream task input status:%d code:%d, all rsp", id, - msgId, pRsp->downstreamTaskId, pRsp->downstreamNodeId, pRsp->inputStatus, code); + ASSERT(leftRsp >= 0); + + // all msg rsp already, continue + if (leftRsp == 0) { + ASSERT(pTask->outputInfo.status == TASK_OUTPUT_STATUS__WAIT); + stDebug("s-task:%s output status is set to:%d", id, pTask->outputInfo.status); + + // we need to re-try send dispatch msg to downstream tasks + int32_t numOfFailed = taosArrayGetSize(pTask->msgInfo.pRetryList); + if (numOfFailed > 0) { + if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { + atomic_store_32(&pTask->shuffleDispatcher.waitingRspCnt, numOfFailed); + } + + pTask->msgInfo.retryCount++; + int8_t ref = atomic_add_fetch_8(&pTask->status.timerActive, 1); + stDebug("s-task:%s failed to dispatch msg to downstream for %d times, code:%s, add timer to retry in %dms, ref:%d", + pTask->id.idStr, pTask->msgInfo.retryCount, tstrerror(terrno), DISPATCH_RETRY_INTERVAL_MS, ref); + + streamRetryDispatchStreamBlock(pTask, DISPATCH_RETRY_INTERVAL_MS); + } else { // this message has been sent successfully, let's try next one. + pTask->msgInfo.retryCount = 0; + handleDispatchSuccessRsp(pTask, pRsp->downstreamTaskId); } - } else { - stDebug("s-task:%s recv fix-dispatch rsp, msgId:%d from 0x%x(vgId:%d), downstream task input status:%d code:%d", - id, msgId, pRsp->downstreamTaskId, pRsp->downstreamNodeId, pRsp->inputStatus, code); - } - - // transtate msg has been sent to downstream successfully. let's transfer the fill-history task state - if (pTask->msgInfo.dispatchMsgType == STREAM_INPUT__TRANS_STATE) { - stDebug("s-task:%s dispatch transtate msgId:%d to downstream successfully, start to transfer state", id, msgId); - ASSERT(pTask->info.fillHistory == 1); - - code = streamTransferStateToStreamTask(pTask); - if (code != TSDB_CODE_SUCCESS) { // todo: do nothing if error happens - } - - // now ready for next data output - atomic_store_8(&pTask->outputInfo.status, TASK_OUTPUT_STATUS__NORMAL); - return TSDB_CODE_SUCCESS; - } - - pTask->msgInfo.retryCount = 0; - ASSERT(pTask->outputInfo.status == TASK_OUTPUT_STATUS__WAIT); - - stDebug("s-task:%s output status is set to:%d", id, pTask->outputInfo.status); - - // the input queue of the (down stream) task that receive the output data is full, - // so the TASK_INPUT_STATUS_BLOCKED is rsp - if (pRsp->inputStatus == TASK_INPUT_STATUS__BLOCKED) { - pTask->inputInfo.status = TASK_INPUT_STATUS__BLOCKED; // block the input of current task, to push pressure to upstream - - double el = 0; - if (pTask->msgInfo.blockingTs == 0) { - pTask->msgInfo.blockingTs = taosGetTimestampMs(); // record the blocking start time - } else { - el = (taosGetTimestampMs() - pTask->msgInfo.blockingTs) / 1000.0; - } - - int8_t ref = atomic_add_fetch_8(&pTask->status.timerActive, 1); - stError("s-task:%s inputQ of downstream task:0x%x is full, time:%" PRId64 - " wait for %dms and retry dispatch data, total wait:%.2fSec ref:%d", - id, pRsp->downstreamTaskId, pTask->msgInfo.blockingTs, DISPATCH_RETRY_INTERVAL_MS, el, ref); - streamRetryDispatchStreamBlock(pTask, DISPATCH_RETRY_INTERVAL_MS); - } else { - // this message has been sent successfully, let's try next one. - handleDispatchSuccessRsp(pTask, pRsp->downstreamTaskId); } return 0; diff --git a/source/libs/stream/src/streamTask.c b/source/libs/stream/src/streamTask.c index 8d651c43a0..34d1f7a9c9 100644 --- a/source/libs/stream/src/streamTask.c +++ b/source/libs/stream/src/streamTask.c @@ -381,6 +381,7 @@ void tFreeStreamTask(SStreamTask* pTask) { pTask->pUpstreamInfoList = NULL; } + pTask->msgInfo.pRetryList = taosArrayDestroy(pTask->msgInfo.pRetryList); taosMemoryFree(pTask->pTokenBucket); taosThreadMutexDestroy(&pTask->lock); taosMemoryFree(pTask); @@ -410,6 +411,7 @@ int32_t streamTaskInit(SStreamTask* pTask, SStreamMeta* pMeta, SMsgCb* pMsgCb, i pTask->dataRange.range.maxVer = ver; pTask->dataRange.range.minVer = ver; pTask->pMsgCb = pMsgCb; + pTask->msgInfo.pRetryList = taosArrayInit(4, sizeof(int32_t)); pTask->pTokenBucket = taosMemoryCalloc(1, sizeof(STokenBucket)); if (pTask->pTokenBucket == NULL) { From 3726aa723eb350a94b3b517884febc4f13219f44 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Fri, 22 Sep 2023 16:05:12 +0800 Subject: [PATCH 067/158] fix(stream): fix invalid read. --- source/libs/stream/src/streamTask.c | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/source/libs/stream/src/streamTask.c b/source/libs/stream/src/streamTask.c index 34d1f7a9c9..56f97c565a 100644 --- a/source/libs/stream/src/streamTask.c +++ b/source/libs/stream/src/streamTask.c @@ -342,6 +342,13 @@ void tFreeStreamTask(SStreamTask* pTask) { walCloseReader(pTask->exec.pWalReader); } + pTask->pReadyMsgList = taosArrayDestroy(pTask->pReadyMsgList); + if (pTask->msgInfo.pData != NULL) { + destroyDispatchMsg(pTask->msgInfo.pData, getNumOfDispatchBranch(pTask)); + pTask->msgInfo.pData = NULL; + pTask->msgInfo.dispatchMsgType = 0; + } + if (pTask->outputInfo.type == TASK_OUTPUT__TABLE) { tDeleteSchemaWrapper(pTask->tbSink.pSchemaWrapper); taosMemoryFree(pTask->tbSink.pTSchema); @@ -356,13 +363,6 @@ void tFreeStreamTask(SStreamTask* pTask) { streamStateClose(pTask->pState, status == TASK_STATUS__DROPPING); } - pTask->pReadyMsgList = taosArrayDestroy(pTask->pReadyMsgList); - if (pTask->msgInfo.pData != NULL) { - destroyDispatchMsg(pTask->msgInfo.pData, getNumOfDispatchBranch(pTask)); - pTask->msgInfo.pData = NULL; - pTask->msgInfo.dispatchMsgType = 0; - } - if (pTask->id.idStr != NULL) { taosMemoryFree((void*)pTask->id.idStr); } From ed9e035095bc99badede3e543a11d8c6ee9f7922 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Fri, 22 Sep 2023 16:19:25 +0800 Subject: [PATCH 068/158] fix(stream): clear the failure vnode id list before send msg. --- source/libs/stream/src/streamDispatch.c | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/source/libs/stream/src/streamDispatch.c b/source/libs/stream/src/streamDispatch.c index a3eda3ceb2..e980b64247 100644 --- a/source/libs/stream/src/streamDispatch.c +++ b/source/libs/stream/src/streamDispatch.c @@ -431,6 +431,10 @@ static void doRetryDispatchData(void* param, void* tmrId) { int32_t code = 0; { + SArray* pList = taosArrayDup(pTask->msgInfo.pRetryList, NULL); + taosArrayDestroy(pTask->msgInfo.pRetryList); + pTask->msgInfo.pRetryList = NULL; + SStreamDispatchReq *pReq = pTask->msgInfo.pData; if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { @@ -438,12 +442,12 @@ static void doRetryDispatchData(void* param, void* tmrId) { int32_t numOfVgroups = taosArrayGetSize(vgInfo); - int32_t numOfFailed = taosArrayGetSize(pTask->msgInfo.pRetryList); + int32_t numOfFailed = taosArrayGetSize(pList); stDebug("s-task:%s (child taskId:%d) re-try shuffle-dispatch blocks to %d vgroup(s), msgId:%d", id, pTask->info.selfChildId, numOfFailed, msgId); for (int32_t i = 0; i < numOfFailed; i++) { - int32_t vgId = *(int32_t*) taosArrayGet(pTask->msgInfo.pRetryList, i); + int32_t vgId = *(int32_t*) taosArrayGet(pList, i); for(int32_t j = 0; j < numOfVgroups; ++j) { SVgroupInfo* pVgInfo = taosArrayGet(vgInfo, j); From f3bd5c411d7e2258456be4e153814aac67ebd24f Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Fri, 22 Sep 2023 16:44:55 +0800 Subject: [PATCH 069/158] fix(stream): remove the blocking flag after successfully retry sending msg. --- source/libs/stream/src/streamDispatch.c | 26 +++++++++++-------------- 1 file changed, 11 insertions(+), 15 deletions(-) diff --git a/source/libs/stream/src/streamDispatch.c b/source/libs/stream/src/streamDispatch.c index e980b64247..a66b85116f 100644 --- a/source/libs/stream/src/streamDispatch.c +++ b/source/libs/stream/src/streamDispatch.c @@ -496,8 +496,9 @@ static void doRetryDispatchData(void* param, void* tmrId) { } void streamRetryDispatchStreamBlock(SStreamTask* pTask, int64_t waitDuration) { - stWarn("s-task:%s retry send dispatch data in %" PRId64 "ms, in timer msgId:%d", pTask->id.idStr, waitDuration, - pTask->taskExecInfo.dispatchCount); + pTask->msgInfo.retryCount++; + stWarn("s-task:%s retry send dispatch data in %" PRId64 "ms, in timer msgId:%d, retryTimes:%d", pTask->id.idStr, waitDuration, + pTask->taskExecInfo.dispatchCount, pTask->msgInfo.retryCount); if (pTask->launchTaskTimer != NULL) { taosTmrReset(doRetryDispatchData, waitDuration, pTask, streamEnv.timer, &pTask->launchTaskTimer); @@ -606,6 +607,7 @@ int32_t streamDispatchStreamBlock(SStreamTask* pTask) { int32_t retryCount = 0; pTask->taskExecInfo.dispatchCount += 1; + pTask->msgInfo.startTs = taosGetTimestampMs(); int32_t code = doBuildDispatchMsg(pTask, pBlock); if (code == 0) { @@ -1038,17 +1040,12 @@ static int32_t handleDispatchSuccessRsp(SStreamTask* pTask, int32_t downstreamId destroyDispatchMsg(pTask->msgInfo.pData, getNumOfDispatchBranch(pTask)); pTask->msgInfo.pData = NULL; - if (pTask->msgInfo.startTs != 0) { - int64_t now = taosGetTimestampMs(); + int64_t el = taosGetTimestampMs() - pTask->msgInfo.startTs; + stDebug("s-task:%s downstream task:0x%x resume to normal from inputQ blocking, blocking time:%" PRId64 "ms", + pTask->id.idStr, downstreamId, el); - int64_t el = now - pTask->msgInfo.startTs; - stDebug("s-task:%s downstream task:0x%x resume to normal from inputQ blocking, blocking time:%" PRId64 "ms", - pTask->id.idStr, downstreamId, el); - - pTask->msgInfo.startTs = now; - // put data into inputQ of current task is also allowed - pTask->inputInfo.status = TASK_INPUT_STATUS__NORMAL; - } + // put data into inputQ of current task is also allowed + pTask->inputInfo.status = TASK_INPUT_STATUS__NORMAL; // now ready for next data output atomic_store_8(&pTask->outputInfo.status, TASK_OUTPUT_STATUS__NORMAL); @@ -1137,10 +1134,9 @@ int32_t streamProcessDispatchRsp(SStreamTask* pTask, SStreamDispatchRsp* pRsp, i atomic_store_32(&pTask->shuffleDispatcher.waitingRspCnt, numOfFailed); } - pTask->msgInfo.retryCount++; int8_t ref = atomic_add_fetch_8(&pTask->status.timerActive, 1); - stDebug("s-task:%s failed to dispatch msg to downstream for %d times, code:%s, add timer to retry in %dms, ref:%d", - pTask->id.idStr, pTask->msgInfo.retryCount, tstrerror(terrno), DISPATCH_RETRY_INTERVAL_MS, ref); + stDebug("s-task:%s failed to dispatch msg to downstream code:%s, add timer to retry in %dms, ref:%d", + pTask->id.idStr, tstrerror(terrno), DISPATCH_RETRY_INTERVAL_MS, ref); streamRetryDispatchStreamBlock(pTask, DISPATCH_RETRY_INTERVAL_MS); } else { // this message has been sent successfully, let's try next one. From 693254f60297efab5b5bffbbe75eb50d4c868b9c Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Fri, 22 Sep 2023 16:53:22 +0800 Subject: [PATCH 070/158] fix(stream): remove invalid free. --- source/libs/stream/src/streamDispatch.c | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/source/libs/stream/src/streamDispatch.c b/source/libs/stream/src/streamDispatch.c index a66b85116f..bf9c0d44db 100644 --- a/source/libs/stream/src/streamDispatch.c +++ b/source/libs/stream/src/streamDispatch.c @@ -432,8 +432,7 @@ static void doRetryDispatchData(void* param, void* tmrId) { int32_t code = 0; { SArray* pList = taosArrayDup(pTask->msgInfo.pRetryList, NULL); - taosArrayDestroy(pTask->msgInfo.pRetryList); - pTask->msgInfo.pRetryList = NULL; + taosArrayClear(pTask->msgInfo.pRetryList); SStreamDispatchReq *pReq = pTask->msgInfo.pData; From 192c47a7c257cccdc17f27db19cc86f4094d5ac0 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Fri, 22 Sep 2023 17:27:10 +0800 Subject: [PATCH 071/158] fix(stream): add some logs. --- source/libs/stream/src/streamDispatch.c | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/source/libs/stream/src/streamDispatch.c b/source/libs/stream/src/streamDispatch.c index bf9c0d44db..fe72ff05e6 100644 --- a/source/libs/stream/src/streamDispatch.c +++ b/source/libs/stream/src/streamDispatch.c @@ -1061,7 +1061,11 @@ int32_t streamProcessDispatchRsp(SStreamTask* pTask, SStreamDispatchRsp* pRsp, i int32_t leftRsp = 0; if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { + stDebug("s-task:%s waiting rsp:%d", id, pTask->shuffleDispatcher.waitingRspCnt); + leftRsp = atomic_sub_fetch_32(&pTask->shuffleDispatcher.waitingRspCnt, 1); + ASSERT(leftRsp >= 0); + if (leftRsp > 0) { stDebug( "s-task:%s recv dispatch rsp, msgId:%d from 0x%x(vgId:%d), downstream task input status:%d code:%d, waiting for %d rsp", id, msgId, pRsp->downstreamTaskId, pRsp->downstreamNodeId, pRsp->inputStatus, code, leftRsp); @@ -1131,6 +1135,7 @@ int32_t streamProcessDispatchRsp(SStreamTask* pTask, SStreamDispatchRsp* pRsp, i if (numOfFailed > 0) { if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { atomic_store_32(&pTask->shuffleDispatcher.waitingRspCnt, numOfFailed); + stDebug("s-task:%s waiting rsp set to be %d", id, pTask->shuffleDispatcher.waitingRspCnt); } int8_t ref = atomic_add_fetch_8(&pTask->status.timerActive, 1); From 5eee76df650c06c6de1d801b6af261aaa5b0101d Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Fri, 22 Sep 2023 18:13:18 +0800 Subject: [PATCH 072/158] fix(stream): add msg identifier check. --- include/libs/stream/tstream.h | 3 +++ source/dnode/vnode/src/tq/tq.c | 4 ++++ source/libs/stream/src/stream.c | 2 ++ source/libs/stream/src/streamDispatch.c | 16 ++++++++++++++++ 4 files changed, 25 insertions(+) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index d7ad1ddf08..f3630cb558 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -452,6 +452,7 @@ struct SStreamDispatchReq { int64_t stage; // nodeId from upstream task int64_t streamId; int32_t taskId; + int32_t msgId; // msg id to identify if the incoming msg from the same sender int32_t srcVgId; int32_t upstreamTaskId; int32_t upstreamChildId; @@ -468,7 +469,9 @@ typedef struct { int32_t upstreamTaskId; int32_t downstreamNodeId; int32_t downstreamTaskId; + int32_t msgId; int8_t inputStatus; + int64_t stage; } SStreamDispatchRsp; typedef struct { diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 55ce4b4b33..7315b8d91a 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1372,6 +1372,8 @@ int32_t tqProcessTaskDispatchRsp(STQ* pTq, SRpcMsg* pMsg) { pRsp->streamId = htobe64(pRsp->streamId); pRsp->downstreamTaskId = htonl(pRsp->downstreamTaskId); pRsp->downstreamNodeId = htonl(pRsp->downstreamNodeId); + pRsp->stage = htobe64(pRsp->stage); + pRsp->msgId = htonl(pRsp->msgId); SStreamTask* pTask = streamMetaAcquireTask(pTq->pStreamMeta, pRsp->streamId, pRsp->upstreamTaskId); if (pTask) { @@ -1597,6 +1599,8 @@ FAIL: pRsp->upstreamNodeId = htonl(req.upstreamNodeId); pRsp->downstreamNodeId = htonl(pVnode->config.vgId); pRsp->downstreamTaskId = htonl(req.taskId); + pRsp->msgId = htonl(req.msgId); + pRsp->stage = htobe64(req.stage); pRsp->inputStatus = TASK_OUTPUT_STATUS__NORMAL; int32_t len = sizeof(SMsgHead) + sizeof(SStreamDispatchRsp); diff --git a/source/libs/stream/src/stream.c b/source/libs/stream/src/stream.c index 99d61c2348..c176dfda88 100644 --- a/source/libs/stream/src/stream.c +++ b/source/libs/stream/src/stream.c @@ -142,6 +142,8 @@ static int32_t buildDispatchRsp(const SStreamTask* pTask, const SStreamDispatchR ((SMsgHead*)(*pBuf))->vgId = htonl(pReq->upstreamNodeId); SStreamDispatchRsp* pDispatchRsp = POINTER_SHIFT((*pBuf), sizeof(SMsgHead)); + pDispatchRsp->stage = htobe64(pReq->stage); + pDispatchRsp->msgId = htonl(pReq->msgId); pDispatchRsp->inputStatus = status; pDispatchRsp->streamId = htobe64(pReq->streamId); pDispatchRsp->upstreamNodeId = htonl(pReq->upstreamNodeId); diff --git a/source/libs/stream/src/streamDispatch.c b/source/libs/stream/src/streamDispatch.c index fe72ff05e6..e973cfd4dc 100644 --- a/source/libs/stream/src/streamDispatch.c +++ b/source/libs/stream/src/streamDispatch.c @@ -53,6 +53,7 @@ void initRpcMsg(SRpcMsg* pMsg, int32_t msgType, void* pCont, int32_t contLen) { int32_t tEncodeStreamDispatchReq(SEncoder* pEncoder, const SStreamDispatchReq* pReq) { if (tStartEncode(pEncoder) < 0) return -1; if (tEncodeI64(pEncoder, pReq->stage) < 0) return -1; + if (tEncodeI32(pEncoder, pReq->msgId) < 0) return -1; if (tEncodeI32(pEncoder, pReq->srcVgId) < 0) return -1; if (tEncodeI32(pEncoder, pReq->type) < 0) return -1; if (tEncodeI64(pEncoder, pReq->streamId) < 0) return -1; @@ -78,6 +79,7 @@ int32_t tEncodeStreamDispatchReq(SEncoder* pEncoder, const SStreamDispatchReq* p int32_t tDecodeStreamDispatchReq(SDecoder* pDecoder, SStreamDispatchReq* pReq) { if (tStartDecode(pDecoder) < 0) return -1; if (tDecodeI64(pDecoder, &pReq->stage) < 0) return -1; + if (tDecodeI32(pDecoder, &pReq->msgId) < 0) return -1; if (tDecodeI32(pDecoder, &pReq->srcVgId) < 0) return -1; if (tDecodeI32(pDecoder, &pReq->type) < 0) return -1; if (tDecodeI64(pDecoder, &pReq->streamId) < 0) return -1; @@ -112,6 +114,7 @@ static int32_t tInitStreamDispatchReq(SStreamDispatchReq* pReq, const SStreamTas pReq->streamId = pTask->id.streamId; pReq->srcVgId = vgId; pReq->stage = pTask->pMeta->stage; + pReq->msgId = pTask->taskExecInfo.dispatchCount; pReq->upstreamTaskId = pTask->id.taskId; pReq->upstreamChildId = pTask->info.selfChildId; pReq->upstreamNodeId = pTask->info.nodeId; @@ -1056,8 +1059,21 @@ static int32_t handleDispatchSuccessRsp(SStreamTask* pTask, int32_t downstreamId int32_t streamProcessDispatchRsp(SStreamTask* pTask, SStreamDispatchRsp* pRsp, int32_t code) { const char* id = pTask->id.idStr; + int32_t vgId = pTask->pMeta->vgId; int32_t msgId = pTask->taskExecInfo.dispatchCount; + if ((!pTask->pMeta->leader) || (pTask->status.downstreamReady != 1)) { + stError("s-task:%s vgId:%d is follower or task just re-launched, not handle the dispatch rsp, discard it", id, vgId); + return TSDB_CODE_STREAM_TASK_NOT_EXIST; + } + + if ((pRsp->msgId != msgId) || (pRsp->stage != pTask->pMeta->stage)) { + stError("s-task:%s vgId:%d not expect rsp, expected: msgId:%d, stage:%" PRId64 " actual msgId:%d, stage:%" PRId64 + " discard it", + id, vgId, msgId, pTask->pMeta->stage, pRsp->msgId, pRsp->stage); + return TSDB_CODE_INVALID_MSG; + } + int32_t leftRsp = 0; if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { From e49409a11e21702e3f5ac79f4baf2f276e217c82 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Fri, 22 Sep 2023 18:56:31 +0800 Subject: [PATCH 073/158] fix(stream): add some logs. --- source/libs/stream/src/stream.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/libs/stream/src/stream.c b/source/libs/stream/src/stream.c index c176dfda88..6babe66d0d 100644 --- a/source/libs/stream/src/stream.c +++ b/source/libs/stream/src/stream.c @@ -256,7 +256,7 @@ int32_t streamProcessDispatchMsg(SStreamTask* pTask, SStreamDispatchReq* pReq, S // do send response with the input status int32_t code = buildDispatchRsp(pTask, pReq, status, &pRsp->pCont); if (code != TSDB_CODE_SUCCESS) { - // todo handle failure + stError("s-task:%s failed to build dispatch rsp, msgId:%d, code:%s", pTask->id.idStr, pReq->msgId, tstrerror(code)); return code; } From 46b17aa9f1104c2a3cdb42efe3a7cd79df544dfb Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Fri, 22 Sep 2023 19:35:12 +0800 Subject: [PATCH 074/158] fix(stream): fix race condition. --- source/dnode/snode/src/snode.c | 5 +-- source/libs/stream/src/stream.c | 45 +++++++++++++------------ source/libs/stream/src/streamDispatch.c | 45 +++++++++++++------------ 3 files changed, 49 insertions(+), 46 deletions(-) diff --git a/source/dnode/snode/src/snode.c b/source/dnode/snode/src/snode.c index 3c6518bd2f..d6e575d97c 100644 --- a/source/dnode/snode/src/snode.c +++ b/source/dnode/snode/src/snode.c @@ -37,10 +37,7 @@ void sndEnqueueStreamDispatch(SSnode *pSnode, SRpcMsg *pMsg) { SStreamTask *pTask = streamMetaAcquireTask(pSnode->pMeta, req.streamId, req.taskId); if (pTask) { - SRpcMsg rsp = { - .info = pMsg->info, - .code = 0, - }; + SRpcMsg rsp = { .info = pMsg->info, .code = 0 }; streamProcessDispatchMsg(pTask, &req, &rsp); streamMetaReleaseTask(pSnode->pMeta, pTask); rpcFreeCont(pMsg->pCont); diff --git a/source/libs/stream/src/stream.c b/source/libs/stream/src/stream.c index 6babe66d0d..e1b9a2fb2f 100644 --- a/source/libs/stream/src/stream.c +++ b/source/libs/stream/src/stream.c @@ -213,36 +213,39 @@ int32_t streamTaskEnqueueRetrieve(SStreamTask* pTask, SStreamRetrieveReq* pReq, } int32_t streamProcessDispatchMsg(SStreamTask* pTask, SStreamDispatchReq* pReq, SRpcMsg* pRsp) { - stDebug("s-task:%s receive dispatch msg from taskId:0x%x(vgId:%d), msgLen:%" PRId64, pTask->id.idStr, - pReq->upstreamTaskId, pReq->upstreamNodeId, pReq->totalLen); - int32_t status = 0; + int32_t status = 0; + const char* id = pTask->id.idStr; + + stDebug("s-task:%s receive dispatch msg from taskId:0x%x(vgId:%d), msgLen:%" PRId64 ", msgId:%d", id, + pReq->upstreamTaskId, pReq->upstreamNodeId, pReq->totalLen, pReq->msgId); SStreamChildEpInfo* pInfo = streamTaskGetUpstreamTaskEpInfo(pTask, pReq->upstreamTaskId); ASSERT(pInfo != NULL); if (!pTask->pMeta->leader) { - stError("s-task:%s task on follower received dispatch msgs, should discard it, not now", pTask->id.idStr); - } - - // upstream task has restarted/leader-follower switch/transferred to other dnodes - if (pReq->stage > pInfo->stage) { - stError("s-task:%s upstream task:0x%x (vgId:%d) has restart/leader-switch/vnode-transfer, prev stage:%" PRId64 - ", current:%" PRId64 " dispatch msg rejected", - pTask->id.idStr, pReq->upstreamTaskId, pReq->upstreamNodeId, pInfo->stage, pReq->stage); + stError("s-task:%s task on follower received dispatch msgs, should discard it, not now", id); status = TASK_INPUT_STATUS__BLOCKED; } else { - if (!pInfo->dataAllowed) { - stWarn("s-task:%s data from task:0x%x is denied, since inputQ is closed for it", pTask->id.idStr, - pReq->upstreamTaskId); + if (pReq->stage > pInfo->stage) { + // upstream task has restarted/leader-follower switch/transferred to other dnodes + stError("s-task:%s upstream task:0x%x (vgId:%d) has restart/leader-switch/vnode-transfer, prev stage:%" PRId64 + ", current:%" PRId64 " dispatch msg rejected", + id, pReq->upstreamTaskId, pReq->upstreamNodeId, pInfo->stage, pReq->stage); status = TASK_INPUT_STATUS__BLOCKED; } else { - // This task has received the checkpoint req from the upstream task, from which all the messages should be blocked - if (pReq->type == STREAM_INPUT__CHECKPOINT_TRIGGER) { - streamTaskCloseUpstreamInput(pTask, pReq->upstreamTaskId); - stDebug("s-task:%s close inputQ for upstream:0x%x", pTask->id.idStr, pReq->upstreamTaskId); - } + if (!pInfo->dataAllowed) { + stWarn("s-task:%s data from task:0x%x is denied, since inputQ is closed for it", id, pReq->upstreamTaskId); + status = TASK_INPUT_STATUS__BLOCKED; + } else { + // This task has received the checkpoint req from the upstream task, from which all the messages should be + // blocked + if (pReq->type == STREAM_INPUT__CHECKPOINT_TRIGGER) { + streamTaskCloseUpstreamInput(pTask, pReq->upstreamTaskId); + stDebug("s-task:%s close inputQ for upstream:0x%x, msgId:%d", id, pReq->upstreamTaskId, pReq->msgId); + } - status = streamTaskAppendInputBlocks(pTask, pReq); + status = streamTaskAppendInputBlocks(pTask, pReq); + } } } @@ -256,7 +259,7 @@ int32_t streamProcessDispatchMsg(SStreamTask* pTask, SStreamDispatchReq* pReq, S // do send response with the input status int32_t code = buildDispatchRsp(pTask, pReq, status, &pRsp->pCont); if (code != TSDB_CODE_SUCCESS) { - stError("s-task:%s failed to build dispatch rsp, msgId:%d, code:%s", pTask->id.idStr, pReq->msgId, tstrerror(code)); + stError("s-task:%s failed to build dispatch rsp, msgId:%d, code:%s", id, pReq->msgId, tstrerror(code)); return code; } diff --git a/source/libs/stream/src/streamDispatch.c b/source/libs/stream/src/streamDispatch.c index e973cfd4dc..5977d31e0d 100644 --- a/source/libs/stream/src/streamDispatch.c +++ b/source/libs/stream/src/streamDispatch.c @@ -1074,27 +1074,6 @@ int32_t streamProcessDispatchRsp(SStreamTask* pTask, SStreamDispatchRsp* pRsp, i return TSDB_CODE_INVALID_MSG; } - int32_t leftRsp = 0; - - if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { - stDebug("s-task:%s waiting rsp:%d", id, pTask->shuffleDispatcher.waitingRspCnt); - - leftRsp = atomic_sub_fetch_32(&pTask->shuffleDispatcher.waitingRspCnt, 1); - ASSERT(leftRsp >= 0); - - if (leftRsp > 0) { - stDebug( "s-task:%s recv dispatch rsp, msgId:%d from 0x%x(vgId:%d), downstream task input status:%d code:%d, waiting for %d rsp", - id, msgId, pRsp->downstreamTaskId, pRsp->downstreamNodeId, pRsp->inputStatus, code, leftRsp); - } else { - stDebug( - "s-task:%s recv dispatch rsp, msgId:%d from 0x%x(vgId:%d), downstream task input status:%d code:%d, all rsp", - id, msgId, pRsp->downstreamTaskId, pRsp->downstreamNodeId, pRsp->inputStatus, code); - } - } else { - stDebug("s-task:%s recv fix-dispatch rsp, msgId:%d from 0x%x(vgId:%d), downstream task input status:%d code:%d", - id, msgId, pRsp->downstreamTaskId, pRsp->downstreamNodeId, pRsp->inputStatus, code); - } - if (code != TSDB_CODE_SUCCESS) { // dispatch message failed: network error, or node not available. // in case of the input queue is full, the code will be TSDB_CODE_SUCCESS, the and pRsp->inputStatus will be set @@ -1112,14 +1091,19 @@ int32_t streamProcessDispatchRsp(SStreamTask* pTask, SStreamDispatchRsp* pRsp, i } else { stError("s-task:%s failed to dispatch msgId:%d to task:0x%x(vgId:%d), code:%s, add to retry list", id, msgId, pRsp->downstreamTaskId, pRsp->downstreamNodeId, tstrerror(code)); + taosThreadMutexLock(&pTask->lock); taosArrayPush(pTask->msgInfo.pRetryList, &pRsp->downstreamNodeId); + taosThreadMutexUnlock(&pTask->lock); } } else { // code == 0 if (pRsp->inputStatus == TASK_INPUT_STATUS__BLOCKED) { pTask->inputInfo.status = TASK_INPUT_STATUS__BLOCKED; // block the input of current task, to push pressure to upstream + taosThreadMutexLock(&pTask->lock); taosArrayPush(pTask->msgInfo.pRetryList, &pRsp->downstreamNodeId); + taosThreadMutexUnlock(&pTask->lock); + stError("s-task:%s inputQ of downstream task:0x%x(vgId:%d) is full, wait for %dms and retry dispatch data", id, pRsp->downstreamTaskId, pRsp->downstreamNodeId, DISPATCH_RETRY_INTERVAL_MS); } @@ -1139,6 +1123,25 @@ int32_t streamProcessDispatchRsp(SStreamTask* pTask, SStreamDispatchRsp* pRsp, i } } + int32_t leftRsp = 0; + if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { + stDebug("s-task:%s waiting rsp:%d", id, pTask->shuffleDispatcher.waitingRspCnt); + leftRsp = atomic_sub_fetch_32(&pTask->shuffleDispatcher.waitingRspCnt, 1); + ASSERT(leftRsp >= 0); + + if (leftRsp > 0) { + stDebug( "s-task:%s recv dispatch rsp, msgId:%d from 0x%x(vgId:%d), downstream task input status:%d code:%d, waiting for %d rsp", + id, msgId, pRsp->downstreamTaskId, pRsp->downstreamNodeId, pRsp->inputStatus, code, leftRsp); + } else { + stDebug( + "s-task:%s recv dispatch rsp, msgId:%d from 0x%x(vgId:%d), downstream task input status:%d code:%d, all rsp", + id, msgId, pRsp->downstreamTaskId, pRsp->downstreamNodeId, pRsp->inputStatus, code); + } + } else { + stDebug("s-task:%s recv fix-dispatch rsp, msgId:%d from 0x%x(vgId:%d), downstream task input status:%d code:%d", + id, msgId, pRsp->downstreamTaskId, pRsp->downstreamNodeId, pRsp->inputStatus, code); + } + ASSERT(leftRsp >= 0); // all msg rsp already, continue From d1a6b8bd22a3b69a7e48159e46abb88cf98789bc Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Fri, 22 Sep 2023 22:50:21 +0800 Subject: [PATCH 075/158] refactor: do some internal refactor. --- source/dnode/vnode/src/tq/tq.c | 2 ++ source/libs/stream/src/stream.c | 2 ++ 2 files changed, 4 insertions(+) diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 7315b8d91a..677e71efd3 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1593,6 +1593,8 @@ FAIL: } pRspHead->vgId = htonl(req.upstreamNodeId); + ASSERT(pRspHead->vgId > 0); + SStreamDispatchRsp* pRsp = POINTER_SHIFT(pRspHead, sizeof(SMsgHead)); pRsp->streamId = htobe64(req.streamId); pRsp->upstreamTaskId = htonl(req.upstreamTaskId); diff --git a/source/libs/stream/src/stream.c b/source/libs/stream/src/stream.c index e1b9a2fb2f..2891c93e9b 100644 --- a/source/libs/stream/src/stream.c +++ b/source/libs/stream/src/stream.c @@ -140,6 +140,8 @@ static int32_t buildDispatchRsp(const SStreamTask* pTask, const SStreamDispatchR } ((SMsgHead*)(*pBuf))->vgId = htonl(pReq->upstreamNodeId); + ASSERT(((SMsgHead*)(*pBuf))->vgId > 0); + SStreamDispatchRsp* pDispatchRsp = POINTER_SHIFT((*pBuf), sizeof(SMsgHead)); pDispatchRsp->stage = htobe64(pReq->stage); From e671e872342bd5e9c93b91da0c96ade4624303df Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Fri, 22 Sep 2023 23:06:48 +0800 Subject: [PATCH 076/158] fix(stream): fix invalid assert. --- source/dnode/vnode/src/tq/tq.c | 2 +- source/libs/stream/src/stream.c | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 677e71efd3..5d8a778453 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1593,7 +1593,7 @@ FAIL: } pRspHead->vgId = htonl(req.upstreamNodeId); - ASSERT(pRspHead->vgId > 0); + ASSERT(pRspHead->vgId != 0); SStreamDispatchRsp* pRsp = POINTER_SHIFT(pRspHead, sizeof(SMsgHead)); pRsp->streamId = htobe64(req.streamId); diff --git a/source/libs/stream/src/stream.c b/source/libs/stream/src/stream.c index 2891c93e9b..888c9113d5 100644 --- a/source/libs/stream/src/stream.c +++ b/source/libs/stream/src/stream.c @@ -140,7 +140,7 @@ static int32_t buildDispatchRsp(const SStreamTask* pTask, const SStreamDispatchR } ((SMsgHead*)(*pBuf))->vgId = htonl(pReq->upstreamNodeId); - ASSERT(((SMsgHead*)(*pBuf))->vgId > 0); + ASSERT(((SMsgHead*)(*pBuf))->vgId != 0); SStreamDispatchRsp* pDispatchRsp = POINTER_SHIFT((*pBuf), sizeof(SMsgHead)); From b4bc4fac98ba230bf1577b21c2cec5297384f24c Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sat, 23 Sep 2023 11:31:14 +0800 Subject: [PATCH 077/158] fix(stream): disable auto rsp for checkpoint source. --- source/dnode/vnode/src/inc/vnodeInt.h | 2 +- source/dnode/vnode/src/tq/tq.c | 5 ++++- source/dnode/vnode/src/vnd/vnodeSvr.c | 2 +- source/libs/stream/src/streamDispatch.c | 1 - 4 files changed, 6 insertions(+), 4 deletions(-) diff --git a/source/dnode/vnode/src/inc/vnodeInt.h b/source/dnode/vnode/src/inc/vnodeInt.h index 39f3d465f2..a29d595ef7 100644 --- a/source/dnode/vnode/src/inc/vnodeInt.h +++ b/source/dnode/vnode/src/inc/vnodeInt.h @@ -224,7 +224,7 @@ int tqPushMsg(STQ*, tmsg_t msgType); int tqRegisterPushHandle(STQ* pTq, void* handle, SRpcMsg* pMsg); int tqUnregisterPushHandle(STQ* pTq, void* pHandle); int tqScanWalAsync(STQ* pTq, bool ckPause); -int32_t tqProcessStreamCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg); +int32_t tqProcessStreamCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg* pRsp); int32_t tqProcessStreamTaskCheckpointReadyMsg(STQ* pTq, SRpcMsg* pMsg); int32_t tqProcessTaskUpdateReq(STQ* pTq, SRpcMsg* pMsg); int32_t tqCheckAndRunStreamTaskAsync(STQ* pTq); diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 5d8a778453..3e89780c8f 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1616,13 +1616,16 @@ FAIL: } // todo error code cannot be return, since this is invoked by an mnode-launched transaction. -int32_t tqProcessStreamCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg) { +int32_t tqProcessStreamCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg* pRsp) { int32_t vgId = TD_VID(pTq->pVnode); SStreamMeta* pMeta = pTq->pStreamMeta; char* msg = POINTER_SHIFT(pMsg->pCont, sizeof(SMsgHead)); int32_t len = pMsg->contLen - sizeof(SMsgHead); int32_t code = 0; + // disable auto rsp to source + pRsp->info.handle = NULL; + SStreamCheckpointSourceReq req = {0}; if (!vnodeIsRoleLeader(pTq->pVnode)) { tqDebug("vgId:%d not leader, ignore checkpoint-source msg", vgId); diff --git a/source/dnode/vnode/src/vnd/vnodeSvr.c b/source/dnode/vnode/src/vnd/vnodeSvr.c index 016636a815..1e61a1f9dd 100644 --- a/source/dnode/vnode/src/vnd/vnodeSvr.c +++ b/source/dnode/vnode/src/vnd/vnodeSvr.c @@ -602,7 +602,7 @@ int32_t vnodeProcessWriteMsg(SVnode *pVnode, SRpcMsg *pMsg, int64_t ver, SRpcMsg vnodeProcessDropIndexReq(pVnode, ver, pReq, len, pRsp); break; case TDMT_VND_STREAM_CHECK_POINT_SOURCE: - tqProcessStreamCheckPointSourceReq(pVnode->pTq, pMsg); + tqProcessStreamCheckPointSourceReq(pVnode->pTq, pMsg, pRsp); break; case TDMT_VND_STREAM_TASK_UPDATE: tqProcessTaskUpdateReq(pVnode->pTq, pMsg); diff --git a/source/libs/stream/src/streamDispatch.c b/source/libs/stream/src/streamDispatch.c index 5977d31e0d..e98c115e7c 100644 --- a/source/libs/stream/src/streamDispatch.c +++ b/source/libs/stream/src/streamDispatch.c @@ -854,7 +854,6 @@ int32_t buildCheckpointSourceRsp(SStreamCheckpointSourceReq* pReq, SRpcHandleInf } ((SMsgHead*)pBuf)->vgId = htonl(pReq->mnodeId); - void* abuf = POINTER_SHIFT(pBuf, sizeof(SMsgHead)); tEncoderInit(&encoder, (uint8_t*)abuf, len); From c05a8de3e86f0ef124cfb4a0a46772055936467f Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sat, 23 Sep 2023 14:13:33 +0800 Subject: [PATCH 078/158] fix(stream): rsp when checkpoint source failed. --- include/libs/stream/tstream.h | 1 + source/dnode/vnode/src/tq/tq.c | 12 ++++++++++++ source/libs/stream/src/stream.c | 6 +++--- source/libs/stream/src/streamDispatch.c | 7 ++++++- 4 files changed, 22 insertions(+), 4 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index f3630cb558..f1c6e369cb 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -75,6 +75,7 @@ enum { TASK_INPUT_STATUS__NORMAL = 1, TASK_INPUT_STATUS__BLOCKED, TASK_INPUT_STATUS__FAILED, + TASK_INPUT_STATUS__REFUSED, }; enum { diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 3e89780c8f..ceb232b4af 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1629,6 +1629,9 @@ int32_t tqProcessStreamCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg* pRs SStreamCheckpointSourceReq req = {0}; if (!vnodeIsRoleLeader(pTq->pVnode)) { tqDebug("vgId:%d not leader, ignore checkpoint-source msg", vgId); + SRpcMsg rsp = {0}; + buildCheckpointSourceRsp(&req, &pMsg->info, &rsp, 0); + tmsgSendRsp(&rsp); // error occurs return TSDB_CODE_SUCCESS; } @@ -1638,6 +1641,9 @@ int32_t tqProcessStreamCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg* pRs code = TSDB_CODE_MSG_DECODE_ERROR; tDecoderClear(&decoder); tqError("vgId:%d failed to decode checkpoint-source msg, code:%s", vgId, tstrerror(code)); + SRpcMsg rsp = {0}; + buildCheckpointSourceRsp(&req, &pMsg->info, &rsp, 0); + tmsgSendRsp(&rsp); // error occurs return code; } tDecoderClear(&decoder); @@ -1646,6 +1652,9 @@ int32_t tqProcessStreamCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg* pRs if (pTask == NULL) { tqError("vgId:%d failed to find s-task:0x%x, ignore checkpoint msg. it may have been destroyed already", vgId, req.taskId); + SRpcMsg rsp = {0}; + buildCheckpointSourceRsp(&req, &pMsg->info, &rsp, 0); + tmsgSendRsp(&rsp); // error occurs return TSDB_CODE_SUCCESS; } @@ -1694,6 +1703,9 @@ int32_t tqProcessStreamCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg* pRs code = streamAddCheckpointSourceRspMsg(&req, &pMsg->info, pTask, 1); if (code != TSDB_CODE_SUCCESS) { + SRpcMsg rsp = {0}; + buildCheckpointSourceRsp(&req, &pMsg->info, &rsp, 0); + tmsgSendRsp(&rsp); // error occurs return code; } diff --git a/source/libs/stream/src/stream.c b/source/libs/stream/src/stream.c index 888c9113d5..dbcbfc0a94 100644 --- a/source/libs/stream/src/stream.c +++ b/source/libs/stream/src/stream.c @@ -225,15 +225,15 @@ int32_t streamProcessDispatchMsg(SStreamTask* pTask, SStreamDispatchReq* pReq, S ASSERT(pInfo != NULL); if (!pTask->pMeta->leader) { - stError("s-task:%s task on follower received dispatch msgs, should discard it, not now", id); - status = TASK_INPUT_STATUS__BLOCKED; + stError("s-task:%s task on follower received dispatch msgs, dispatch msg rejected", id); + status = TASK_INPUT_STATUS__REFUSED; } else { if (pReq->stage > pInfo->stage) { // upstream task has restarted/leader-follower switch/transferred to other dnodes stError("s-task:%s upstream task:0x%x (vgId:%d) has restart/leader-switch/vnode-transfer, prev stage:%" PRId64 ", current:%" PRId64 " dispatch msg rejected", id, pReq->upstreamTaskId, pReq->upstreamNodeId, pInfo->stage, pReq->stage); - status = TASK_INPUT_STATUS__BLOCKED; + status = TASK_INPUT_STATUS__REFUSED; } else { if (!pInfo->dataAllowed) { stWarn("s-task:%s data from task:0x%x is denied, since inputQ is closed for it", id, pReq->upstreamTaskId); diff --git a/source/libs/stream/src/streamDispatch.c b/source/libs/stream/src/streamDispatch.c index e98c115e7c..08be06c841 100644 --- a/source/libs/stream/src/streamDispatch.c +++ b/source/libs/stream/src/streamDispatch.c @@ -1046,7 +1046,9 @@ static int32_t handleDispatchSuccessRsp(SStreamTask* pTask, int32_t downstreamId pTask->id.idStr, downstreamId, el); // put data into inputQ of current task is also allowed - pTask->inputInfo.status = TASK_INPUT_STATUS__NORMAL; + if (pTask->inputInfo.status == TASK_INPUT_STATUS__BLOCKED) { + pTask->inputInfo.status = TASK_INPUT_STATUS__NORMAL; + } // now ready for next data output atomic_store_8(&pTask->outputInfo.status, TASK_OUTPUT_STATUS__NORMAL); @@ -1105,6 +1107,9 @@ int32_t streamProcessDispatchRsp(SStreamTask* pTask, SStreamDispatchRsp* pRsp, i stError("s-task:%s inputQ of downstream task:0x%x(vgId:%d) is full, wait for %dms and retry dispatch data", id, pRsp->downstreamTaskId, pRsp->downstreamNodeId, DISPATCH_RETRY_INTERVAL_MS); + } else if (pRsp->inputStatus == TASK_INPUT_STATUS__REFUSED) { + stError("s-task:%s downstream task:0x%x(vgId:%d) refused the dispatch msg, treat it as success", id, + pRsp->downstreamTaskId, pRsp->downstreamNodeId); } // transtate msg has been sent to downstream successfully. let's transfer the fill-history task state From 012ddadf25d64ea1e9173a1ace979f4f4a65a499 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sat, 23 Sep 2023 15:22:48 +0800 Subject: [PATCH 079/158] fix(stream): update logs. --- source/dnode/vnode/src/tq/tqSink.c | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/source/dnode/vnode/src/tq/tqSink.c b/source/dnode/vnode/src/tq/tqSink.c index 0ca28277e7..f3765f6484 100644 --- a/source/dnode/vnode/src/tq/tqSink.c +++ b/source/dnode/vnode/src/tq/tqSink.c @@ -327,7 +327,7 @@ int32_t doMergeExistedRows(SSubmitTbData* pExisted, const SSubmitTbData* pNew, c taosArrayDestroy(pExisted->aRowP); pExisted->aRowP = pFinal; - tqDebug("s-task:%s rows merged, final rows:%d, uid:%" PRId64 ", existed auto-create table:%d, new-block:%d", id, + tqTrace("s-task:%s rows merged, final rows:%d, uid:%" PRId64 ", existed auto-create table:%d, new-block:%d", id, (int32_t)taosArrayGetSize(pFinal), pExisted->uid, (pExisted->pCreateTbReq != NULL), (pNew->pCreateTbReq != NULL)); return TSDB_CODE_SUCCESS; } @@ -672,10 +672,10 @@ int32_t setDstTableDataUid(SVnode* pVnode, SStreamTask* pTask, SSDataBlock* pDat pTableData->uid = pTableSinkInfo->uid; if (pTableData->uid == 0) { - tqDebug("s-task:%s cached tableInfo uid is invalid, acquire it from meta", id); + tqTrace("s-task:%s cached tableInfo uid is invalid, acquire it from meta", id); return doWaitForDstTableCreated(pVnode, pTask, pTableSinkInfo, dstTableName, &pTableData->uid); } else { - tqDebug("s-task:%s set the dstTable uid from cache:%"PRId64, id, pTableData->uid); + tqTrace("s-task:%s set the dstTable uid from cache:%"PRId64, id, pTableData->uid); } } else { // The auto-create option will always set to be open for those submit messages, which arrive during the period @@ -740,7 +740,7 @@ int32_t setDstTableDataPayload(SStreamTask* pTask, int32_t blockIndex, SSDataBlo } taosArraySort(pTableData->aRowP, tsAscendingSortFn); - tqDebug("s-task:%s build submit msg for dstTable:%s, numOfRows:%d", id, dstTableName, numOfRows); + tqTrace("s-task:%s build submit msg for dstTable:%s, numOfRows:%d", id, dstTableName, numOfRows); return code; } From a6ffb845c8a8719a9ee45d1bc60533eae140ad36 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sat, 23 Sep 2023 17:57:09 +0800 Subject: [PATCH 080/158] fix(stream): fix invalid checkpoint start time. --- source/libs/stream/src/streamCheckpoint.c | 1 + 1 file changed, 1 insertion(+) diff --git a/source/libs/stream/src/streamCheckpoint.c b/source/libs/stream/src/streamCheckpoint.c index 4e0e667614..103302e75d 100644 --- a/source/libs/stream/src/streamCheckpoint.c +++ b/source/libs/stream/src/streamCheckpoint.c @@ -318,6 +318,7 @@ int32_t streamTaskBuildCheckpoint(SStreamTask* pTask) { double el = (taosGetTimestampMs() - pTask->chkInfo.startTs) / 1000.0; if (remain == 0) { // all tasks are ready + pTask->chkInfo.startTs = 0; // clear the recorded start time stDebug("s-task:%s is ready for checkpoint", pTask->id.idStr); streamBackendDoCheckpoint(pMeta, pTask->checkpointingId); streamSaveAllTaskStatus(pMeta, pTask->checkpointingId); From 3baddbe67aa1c9fceb88527d5d725021ff19dc8c Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sat, 23 Sep 2023 18:05:14 +0800 Subject: [PATCH 081/158] fix(stream): update the checkpoint record start time. --- source/libs/stream/src/streamCheckpoint.c | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/source/libs/stream/src/streamCheckpoint.c b/source/libs/stream/src/streamCheckpoint.c index 103302e75d..735136ba5b 100644 --- a/source/libs/stream/src/streamCheckpoint.c +++ b/source/libs/stream/src/streamCheckpoint.c @@ -306,7 +306,6 @@ int32_t streamSaveAllTaskStatus(SStreamMeta* pMeta, int64_t checkpointId) { return TSDB_CODE_SUCCESS; } -// todo: handle the case: during the checkpoint procedure, leader/follower changes happened. int32_t streamTaskBuildCheckpoint(SStreamTask* pTask) { int32_t code = 0; @@ -316,9 +315,9 @@ int32_t streamTaskBuildCheckpoint(SStreamTask* pTask) { ASSERT(remain >= 0); double el = (taosGetTimestampMs() - pTask->chkInfo.startTs) / 1000.0; + pTask->chkInfo.startTs = 0; // clear the recorded start time if (remain == 0) { // all tasks are ready - pTask->chkInfo.startTs = 0; // clear the recorded start time stDebug("s-task:%s is ready for checkpoint", pTask->id.idStr); streamBackendDoCheckpoint(pMeta, pTask->checkpointingId); streamSaveAllTaskStatus(pMeta, pTask->checkpointingId); From 52763ca2a31a4ee368e8b7a5def6dcc3f8ddddc2 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sun, 24 Sep 2023 01:26:51 +0800 Subject: [PATCH 082/158] refactor: do some internal refactor. --- include/libs/stream/tstream.h | 38 ++++---- source/dnode/vnode/src/tq/tq.c | 18 ++-- source/dnode/vnode/src/tq/tqSink.c | 14 +-- source/dnode/vnode/src/tq/tqStreamTask.c | 6 +- source/libs/stream/inc/streamInt.h | 12 ++- source/libs/stream/src/streamCheckpoint.c | 15 ++- source/libs/stream/src/streamDispatch.c | 58 +++++------ source/libs/stream/src/streamExec.c | 4 +- source/libs/stream/src/streamRecover.c | 113 +++++++++++++++------- source/libs/stream/src/streamTask.c | 58 +++++++---- 10 files changed, 205 insertions(+), 131 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index f1c6e369cb..20cf7bb110 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -209,7 +209,7 @@ typedef struct { int32_t taskId; int32_t nodeId; SEpSet epSet; -} STaskDispatcherFixedEp; +} STaskDispatcherFixed; typedef struct { char stbFullName[TSDB_TABLE_FNAME_LEN]; @@ -298,7 +298,7 @@ typedef struct SDispatchMsgInfo { int8_t dispatchMsgType; int16_t msgType; // dispatch msg type int32_t retryCount; // retry send data count - int64_t startTs; // output blocking timestamp + int64_t startTs; // dispatch start time, record total elapsed time for dispatch SArray* pRetryList; // current dispatch successfully completed node of downstream } SDispatchMsgInfo; @@ -318,24 +318,27 @@ typedef struct STaskSchedInfo { void* pTimer; } STaskSchedInfo; -typedef struct SSinkTaskRecorder { +typedef struct SSinkRecorder { int64_t numOfSubmit; int64_t numOfBlocks; int64_t numOfRows; int64_t bytes; -} SSinkTaskRecorder; +} SSinkRecorder; -typedef struct { - int64_t created; - int64_t init; - int64_t step1Start; - int64_t step2Start; - int64_t start; - int32_t updateCount; - int32_t dispatchCount; - int64_t latestUpdateTs; +typedef struct STaskExecStatisInfo { + int64_t created; + int64_t init; + int64_t step1Start; + int64_t step2Start; + int64_t start; + int32_t updateCount; + int32_t dispatch; + int64_t latestUpdateTs; + int32_t checkpoint; + SSinkRecorder sink; } STaskExecStatisInfo; +typedef struct STaskTimer STaskTimer; typedef struct STokenBucket STokenBucket; typedef struct SMetaHbInfo SMetaHbInfo; @@ -353,23 +356,22 @@ struct SStreamTask { SDataRange dataRange; STaskId historyTaskId; STaskId streamTaskId; - STaskExecStatisInfo taskExecInfo; + STaskExecStatisInfo execInfo; SArray* pReadyMsgList; // SArray TdThreadMutex lock; // secure the operation of set task status and puting data into inputQ SArray* pUpstreamInfoList; // output union { - STaskDispatcherFixedEp fixedEpDispatcher; + STaskDispatcherFixed fixedDispatcher; STaskDispatcherShuffle shuffleDispatcher; STaskSinkTb tbSink; STaskSinkSma smaSink; STaskSinkFetch fetchSink; }; - SSinkTaskRecorder sinkRecorder; - STokenBucket* pTokenBucket; - void* launchTaskTimer; + STokenBucket* pTokenBucket; + STaskTimer* pTimer; SMsgCb* pMsgCb; // msg handle SStreamState* pState; // state backend SArray* pRspMsgList; diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index ceb232b4af..9f678a5563 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1016,8 +1016,8 @@ int32_t tqProcessTaskDeployReq(STQ* pTq, int64_t sversion, char* msg, int32_t ms bool restored = pTq->pVnode->restored; if (p != NULL && restored) { - p->taskExecInfo.init = taosGetTimestampMs(); - tqDebug("s-task:%s set the init ts:%"PRId64, p->id.idStr, p->taskExecInfo.init); + p->execInfo.init = taosGetTimestampMs(); + tqDebug("s-task:%s set the init ts:%"PRId64, p->id.idStr, p->execInfo.init); streamTaskCheckDownstream(p); } else if (!restored) { @@ -1055,14 +1055,14 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { const char* pStatus = streamGetTaskStatusStr(pTask->status.taskStatus); tqDebug("s-task:%s start scan-history stage(step 1), status:%s", id, pStatus); - if (pTask->taskExecInfo.step1Start == 0) { + if (pTask->execInfo.step1Start == 0) { ASSERT(pTask->status.pauseAllowed == false); - pTask->taskExecInfo.step1Start = taosGetTimestampMs(); + pTask->execInfo.step1Start = taosGetTimestampMs(); if (pTask->info.fillHistory == 1) { streamTaskEnablePause(pTask); } } else { - tqDebug("s-task:%s resume from paused, start ts:%" PRId64, pTask->id.idStr, pTask->taskExecInfo.step1Start); + tqDebug("s-task:%s resume from paused, start ts:%" PRId64, pTask->id.idStr, pTask->execInfo.step1Start); } // we have to continue retrying to successfully execute the scan history task. @@ -1082,7 +1082,7 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { streamScanHistoryData(pTask); if (pTask->status.taskStatus == TASK_STATUS__PAUSE) { - double el = (taosGetTimestampMs() - pTask->taskExecInfo.step1Start) / 1000.0; + double el = (taosGetTimestampMs() - pTask->execInfo.step1Start) / 1000.0; int8_t status = streamTaskSetSchedStatusInActive(pTask); tqDebug("s-task:%s is paused in the step1, elapsed time:%.2fs, sched-status:%d", pTask->id.idStr, el, status); streamMetaReleaseTask(pMeta, pTask); @@ -1090,7 +1090,7 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { } // the following procedure should be executed, no matter status is stop/pause or not - double el = (taosGetTimestampMs() - pTask->taskExecInfo.step1Start) / 1000.0; + double el = (taosGetTimestampMs() - pTask->execInfo.step1Start) / 1000.0; tqDebug("s-task:%s scan-history stage(step 1) ended, elapsed time:%.2fs", id, el); if (pTask->info.fillHistory) { @@ -1171,7 +1171,7 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { done = streamHistoryTaskSetVerRangeStep2(pTask, latestVer); if (done) { - pTask->taskExecInfo.step2Start = taosGetTimestampMs(); + pTask->execInfo.step2Start = taosGetTimestampMs(); qDebug("s-task:%s scan-history from WAL stage(step 2) ended, elapsed time:%.2fs", id, 0.0); streamTaskPutTranstateIntoInputQ(pTask); streamTryExec(pTask); // exec directly @@ -1183,7 +1183,7 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { pStreamTask->id.idStr); ASSERT(pTask->status.schedStatus == TASK_SCHED_STATUS__WAITING); - pTask->taskExecInfo.step2Start = taosGetTimestampMs(); + pTask->execInfo.step2Start = taosGetTimestampMs(); streamSetParamForStreamScannerStep2(pTask, pRange, pWindow); int64_t dstVer = pTask->dataRange.range.minVer; diff --git a/source/dnode/vnode/src/tq/tqSink.c b/source/dnode/vnode/src/tq/tqSink.c index f3765f6484..81c3d3d07d 100644 --- a/source/dnode/vnode/src/tq/tqSink.c +++ b/source/dnode/vnode/src/tq/tqSink.c @@ -270,11 +270,11 @@ int32_t doBuildAndSendSubmitMsg(SVnode* pVnode, SStreamTask* pTask, SSubmitReq2* tqError("s-task:%s failed to put into write-queue since %s", id, terrstr()); } - SSinkTaskRecorder* pRec = &pTask->sinkRecorder; + SSinkRecorder* pRec = &pTask->execInfo.sink; pRec->numOfSubmit += 1; if ((pRec->numOfSubmit % 5000) == 0) { - double el = (taosGetTimestampMs() - pTask->taskExecInfo.start) / 1000.0; + double el = (taosGetTimestampMs() - pTask->execInfo.start) / 1000.0; tqInfo("s-task:%s vgId:%d write %" PRId64 " blocks (%" PRId64 " rows) in %" PRId64 " submit into dst table, %.2fMiB duration:%.2f Sec.", pTask->id.idStr, vgId, pRec->numOfBlocks, pRec->numOfRows, pRec->numOfSubmit, SIZE_IN_MiB(pRec->bytes), el); @@ -755,8 +755,8 @@ void tqSinkDataIntoDstTable(SStreamTask* pTask, void* vnode, void* data) { int32_t code = TSDB_CODE_SUCCESS; const char* id = pTask->id.idStr; - if (pTask->taskExecInfo.start == 0) { - pTask->taskExecInfo.start = taosGetTimestampMs(); + if (pTask->execInfo.start == 0) { + pTask->execInfo.start = taosGetTimestampMs(); } bool onlySubmitData = true; @@ -785,7 +785,7 @@ void tqSinkDataIntoDstTable(SStreamTask* pTask, void* vnode, void* data) { } else if (pDataBlock->info.type == STREAM_CHECKPOINT) { continue; } else { - pTask->sinkRecorder.numOfBlocks += 1; + pTask->execInfo.sink.numOfBlocks += 1; SSubmitReq2 submitReq = {.aSubmitTbData = taosArrayInit(1, sizeof(SSubmitTbData))}; if (submitReq.aSubmitTbData == NULL) { @@ -833,7 +833,7 @@ void tqSinkDataIntoDstTable(SStreamTask* pTask, void* vnode, void* data) { } hasSubmit = true; - pTask->sinkRecorder.numOfBlocks += 1; + pTask->execInfo.sink.numOfBlocks += 1; uint64_t groupId = pDataBlock->info.id.groupId; SSubmitTbData tbData = {.suid = suid, .uid = 0, .sver = pTSchema->version}; @@ -867,7 +867,7 @@ void tqSinkDataIntoDstTable(SStreamTask* pTask, void* vnode, void* data) { } } - pTask->sinkRecorder.numOfRows += pDataBlock->info.rows; + pTask->execInfo.sink.numOfRows += pDataBlock->info.rows; } taosHashCleanup(pTableIndexMap); diff --git a/source/dnode/vnode/src/tq/tqStreamTask.c b/source/dnode/vnode/src/tq/tqStreamTask.c index 8992d07879..b39132d675 100644 --- a/source/dnode/vnode/src/tq/tqStreamTask.c +++ b/source/dnode/vnode/src/tq/tqStreamTask.c @@ -96,8 +96,8 @@ int32_t tqCheckAndRunStreamTask(STQ* pTq) { continue; } - pTask->taskExecInfo.init = taosGetTimestampMs(); - tqDebug("s-task:%s start check downstream tasks, set the init ts:%"PRId64, pTask->id.idStr, pTask->taskExecInfo.init); + pTask->execInfo.init = taosGetTimestampMs(); + tqDebug("s-task:%s start check downstream tasks, set the init ts:%"PRId64, pTask->id.idStr, pTask->execInfo.init); streamSetStatusNormal(pTask); streamTaskCheckDownstream(pTask); @@ -306,7 +306,7 @@ void handleFillhistoryScanComplete(SStreamTask* pTask, int64_t ver) { ", not scan wal anymore, add transfer-state block into inputQ", id, ver, maxVer); - double el = (taosGetTimestampMs() - pTask->taskExecInfo.step2Start) / 1000.0; + double el = (taosGetTimestampMs() - pTask->execInfo.step2Start) / 1000.0; qDebug("s-task:%s scan-history from WAL stage(step 2) ended, elapsed time:%.2fs", id, el); /*int32_t code = */streamTaskPutTranstateIntoInputQ(pTask); /*int32_t code = */streamSchedExec(pTask); diff --git a/source/libs/stream/inc/streamInt.h b/source/libs/stream/inc/streamInt.h index 10a7dc7be7..6600d7dd04 100644 --- a/source/libs/stream/inc/streamInt.h +++ b/source/libs/stream/inc/streamInt.h @@ -26,6 +26,8 @@ extern "C" { #endif +#define CHECK_DOWNSTREAM_INTERVAL 100 + // clang-format off #define stFatal(...) do { if (stDebugFlag & DEBUG_FATAL) { taosPrintLog("STM FATAL ", DEBUG_FATAL, 255, __VA_ARGS__); }} while(0) #define stError(...) do { if (stDebugFlag & DEBUG_ERROR) { taosPrintLog("STM ERROR ", DEBUG_ERROR, 255, __VA_ARGS__); }} while(0) @@ -53,11 +55,17 @@ struct STokenBucket { int32_t rate; // number of token per second }; +struct STaskTimer { + void* hTaskLaunchTimer; + void* dispatchTimer; + void* checkTimer; +}; + extern SStreamGlobalEnv streamEnv; extern int32_t streamBackendId; extern int32_t streamBackendCfWrapperId; -void streamRetryDispatchStreamBlock(SStreamTask* pTask, int64_t waitDuration); +void streamRetryDispatchData(SStreamTask* pTask, int64_t waitDuration); int32_t streamDispatchStreamBlock(SStreamTask* pTask); void destroyDispatchMsg(SStreamDispatchReq* pReq, int32_t numOfVgroups); int32_t getNumOfDispatchBranch(SStreamTask* pTask); @@ -75,7 +83,7 @@ int32_t tEncodeStreamRetrieveReq(SEncoder* pEncoder, const SStreamRetrieveReq* p int32_t streamSaveAllTaskStatus(SStreamMeta* pMeta, int64_t checkpointId); int32_t streamTaskBuildCheckpoint(SStreamTask* pTask); -int32_t streamDispatchCheckMsg(SStreamTask* pTask, const SStreamTaskCheckReq* pReq, int32_t nodeId, SEpSet* pEpSet); +int32_t streamSendCheckMsg(SStreamTask* pTask, const SStreamTaskCheckReq* pReq, int32_t nodeId, SEpSet* pEpSet); int32_t streamAddCheckpointReadyMsg(SStreamTask* pTask, int32_t srcTaskId, int32_t index, int64_t checkpointId); int32_t streamTaskSendCheckpointReadyMsg(SStreamTask* pTask); diff --git a/source/libs/stream/src/streamCheckpoint.c b/source/libs/stream/src/streamCheckpoint.c index 735136ba5b..997fecbba9 100644 --- a/source/libs/stream/src/streamCheckpoint.c +++ b/source/libs/stream/src/streamCheckpoint.c @@ -141,6 +141,8 @@ int32_t streamProcessCheckpointSourceReq(SStreamTask* pTask, SStreamCheckpointSo pTask->checkpointNotReadyTasks = streamTaskGetNumOfDownstream(pTask); pTask->chkInfo.startTs = taosGetTimestampMs(); + pTask->execInfo.checkpoint += 1; + // 2. let's dispatch checkpoint msg to downstream task directly and do nothing else. put the checkpoint block into // inputQ, to make sure all blocks with less version have been handled by this task already. int32_t code = appendCheckpointIntoInputQ(pTask, STREAM_INPUT__CHECKPOINT_TRIGGER); @@ -200,6 +202,7 @@ int32_t streamProcessCheckpointBlock(SStreamTask* pTask, SStreamDataBlock* pBloc ASSERT(taosArrayGetSize(pTask->pUpstreamInfoList) > 0); if (pTask->chkInfo.startTs == 0) { pTask->chkInfo.startTs = taosGetTimestampMs(); + pTask->execInfo.checkpoint += 1; } // update the child Id for downstream tasks @@ -321,13 +324,15 @@ int32_t streamTaskBuildCheckpoint(SStreamTask* pTask) { stDebug("s-task:%s is ready for checkpoint", pTask->id.idStr); streamBackendDoCheckpoint(pMeta, pTask->checkpointingId); streamSaveAllTaskStatus(pMeta, pTask->checkpointingId); - stInfo("vgId:%d vnode wide checkpoint completed, save all tasks status, elapsed time:%.2f Sec checkpointId:%" PRId64, pMeta->vgId, - el, pTask->checkpointingId); + stInfo( + "vgId:%d vnode wide checkpoint completed, save all tasks status, last:%s, level:%d elapsed time:%.2f Sec " + "checkpointId:%" PRId64, + pMeta->vgId, pTask->id.idStr, pTask->info.taskLevel, el, pTask->checkpointingId); } else { stInfo( - "vgId:%d vnode wide tasks not reach checkpoint ready status, ready s-task:%s, elapsed time:%.2f Sec not " - "ready:%d/%d", - pMeta->vgId, pTask->id.idStr, el, remain, pMeta->numOfStreamTasks); + "vgId:%d vnode wide tasks not reach checkpoint ready status, ready s-task:%s, level:%d elapsed time:%.2f Sec " + "not ready:%d/%d", + pMeta->vgId, pTask->id.idStr, pTask->info.taskLevel, el, remain, pMeta->numOfStreamTasks); } // send check point response to upstream task diff --git a/source/libs/stream/src/streamDispatch.c b/source/libs/stream/src/streamDispatch.c index 08be06c841..bf7abc7457 100644 --- a/source/libs/stream/src/streamDispatch.c +++ b/source/libs/stream/src/streamDispatch.c @@ -114,7 +114,7 @@ static int32_t tInitStreamDispatchReq(SStreamDispatchReq* pReq, const SStreamTas pReq->streamId = pTask->id.streamId; pReq->srcVgId = vgId; pReq->stage = pTask->pMeta->stage; - pReq->msgId = pTask->taskExecInfo.dispatchCount; + pReq->msgId = pTask->execInfo.dispatch; pReq->upstreamTaskId = pTask->id.taskId; pReq->upstreamChildId = pTask->info.selfChildId; pReq->upstreamNodeId = pTask->info.nodeId; @@ -245,7 +245,7 @@ CLEAR: return code; } -int32_t streamDispatchCheckMsg(SStreamTask* pTask, const SStreamTaskCheckReq* pReq, int32_t nodeId, SEpSet* pEpSet) { +int32_t streamSendCheckMsg(SStreamTask* pTask, const SStreamTaskCheckReq* pReq, int32_t nodeId, SEpSet* pEpSet) { void* buf = NULL; int32_t code = -1; SRpcMsg msg = {0}; @@ -305,7 +305,7 @@ static int32_t doBuildDispatchMsg(SStreamTask* pTask, const SStreamDataBlock* pD if (pTask->outputInfo.type == TASK_OUTPUT__FIXED_DISPATCH) { SStreamDispatchReq* pReq = taosMemoryCalloc(1, sizeof(SStreamDispatchReq)); - int32_t downstreamTaskId = pTask->fixedEpDispatcher.taskId; + int32_t downstreamTaskId = pTask->fixedDispatcher.taskId; code = tInitStreamDispatchReq(pReq, pTask, pData->srcVgId, numOfBlocks, downstreamTaskId, pData->type); if (code != TSDB_CODE_SUCCESS) { return code; @@ -375,19 +375,19 @@ static int32_t doBuildDispatchMsg(SStreamTask* pTask, const SStreamDataBlock* pD pTask->msgInfo.pData = pReqs; } - stDebug("s-task:%s build dispatch msg success, msgId:%d", pTask->id.idStr, pTask->taskExecInfo.dispatchCount); + stDebug("s-task:%s build dispatch msg success, msgId:%d", pTask->id.idStr, pTask->execInfo.dispatch); return code; } static int32_t sendDispatchMsg(SStreamTask* pTask, SStreamDispatchReq* pDispatchMsg) { int32_t code = 0; - int32_t msgId = pTask->taskExecInfo.dispatchCount; + int32_t msgId = pTask->execInfo.dispatch; const char* id = pTask->id.idStr; if (pTask->outputInfo.type == TASK_OUTPUT__FIXED_DISPATCH) { - int32_t vgId = pTask->fixedEpDispatcher.nodeId; - SEpSet* pEpSet = &pTask->fixedEpDispatcher.epSet; - int32_t downstreamTaskId = pTask->fixedEpDispatcher.taskId; + int32_t vgId = pTask->fixedDispatcher.nodeId; + SEpSet* pEpSet = &pTask->fixedDispatcher.epSet; + int32_t downstreamTaskId = pTask->fixedDispatcher.taskId; stDebug("s-task:%s (child taskId:%d) fix-dispatch %d block(s) to s-task:0x%x (vgId:%d), id:%d", id, pTask->info.selfChildId, 1, downstreamTaskId, vgId, msgId); @@ -422,7 +422,7 @@ static int32_t sendDispatchMsg(SStreamTask* pTask, SStreamDispatchReq* pDispatch static void doRetryDispatchData(void* param, void* tmrId) { SStreamTask* pTask = param; const char* id = pTask->id.idStr; - int32_t msgId = pTask->taskExecInfo.dispatchCount; + int32_t msgId = pTask->execInfo.dispatch; if (streamTaskShouldStop(&pTask->status)) { int8_t ref = atomic_sub_fetch_8(&pTask->status.timerActive, 1); @@ -443,7 +443,6 @@ static void doRetryDispatchData(void* param, void* tmrId) { SArray* vgInfo = pTask->shuffleDispatcher.dbInfo.pVgroupInfos; int32_t numOfVgroups = taosArrayGetSize(vgInfo); - int32_t numOfFailed = taosArrayGetSize(pList); stDebug("s-task:%s (child taskId:%d) re-try shuffle-dispatch blocks to %d vgroup(s), msgId:%d", id, pTask->info.selfChildId, numOfFailed, msgId); @@ -467,9 +466,9 @@ static void doRetryDispatchData(void* param, void* tmrId) { stDebug("s-task:%s complete re-try shuffle-dispatch blocks to all %d vnodes, msgId:%d", pTask->id.idStr, numOfFailed, msgId); } else { - int32_t vgId = pTask->fixedEpDispatcher.nodeId; - SEpSet* pEpSet = &pTask->fixedEpDispatcher.epSet; - int32_t downstreamTaskId = pTask->fixedEpDispatcher.taskId; + int32_t vgId = pTask->fixedDispatcher.nodeId; + SEpSet* pEpSet = &pTask->fixedDispatcher.epSet; + int32_t downstreamTaskId = pTask->fixedDispatcher.taskId; stDebug("s-task:%s (child taskId:%d) fix-dispatch %d block(s) to s-task:0x%x (vgId:%d), id:%d", id, pTask->info.selfChildId, 1, downstreamTaskId, vgId, msgId); @@ -483,9 +482,9 @@ static void doRetryDispatchData(void* param, void* tmrId) { // stDebug("s-task:%s reset the waitRspCnt to be 0 before launch retry dispatch", pTask->id.idStr); // atomic_store_32(&pTask->shuffleDispatcher.waitingRspCnt, 0); if (streamTaskShouldPause(&pTask->status)) { - streamRetryDispatchStreamBlock(pTask, DISPATCH_RETRY_INTERVAL_MS * 10); + streamRetryDispatchData(pTask, DISPATCH_RETRY_INTERVAL_MS * 10); } else { - streamRetryDispatchStreamBlock(pTask, DISPATCH_RETRY_INTERVAL_MS); + streamRetryDispatchData(pTask, DISPATCH_RETRY_INTERVAL_MS); } } else { int32_t ref = atomic_sub_fetch_8(&pTask->status.timerActive, 1); @@ -497,15 +496,17 @@ static void doRetryDispatchData(void* param, void* tmrId) { } } -void streamRetryDispatchStreamBlock(SStreamTask* pTask, int64_t waitDuration) { +void streamRetryDispatchData(SStreamTask* pTask, int64_t waitDuration) { + STaskTimer* pTmr = pTask->pTimer; pTask->msgInfo.retryCount++; - stWarn("s-task:%s retry send dispatch data in %" PRId64 "ms, in timer msgId:%d, retryTimes:%d", pTask->id.idStr, waitDuration, - pTask->taskExecInfo.dispatchCount, pTask->msgInfo.retryCount); - if (pTask->launchTaskTimer != NULL) { - taosTmrReset(doRetryDispatchData, waitDuration, pTask, streamEnv.timer, &pTask->launchTaskTimer); + stWarn("s-task:%s retry send dispatch data in %" PRId64 "ms, in timer msgId:%d, retryTimes:%d", pTask->id.idStr, + waitDuration, pTask->execInfo.dispatch, pTask->msgInfo.retryCount); + + if (pTmr->dispatchTimer != NULL) { + taosTmrReset(doRetryDispatchData, waitDuration, pTask, streamEnv.timer, &pTmr->dispatchTimer); } else { - pTask->launchTaskTimer = taosTmrStart(doRetryDispatchData, waitDuration, pTask, streamEnv.timer); + pTmr->dispatchTimer = taosTmrStart(doRetryDispatchData, waitDuration, pTask, streamEnv.timer); } } @@ -608,7 +609,7 @@ int32_t streamDispatchStreamBlock(SStreamTask* pTask) { pBlock->type == STREAM_INPUT__TRANS_STATE); int32_t retryCount = 0; - pTask->taskExecInfo.dispatchCount += 1; + pTask->execInfo.dispatch += 1; pTask->msgInfo.startTs = taosGetTimestampMs(); int32_t code = doBuildDispatchMsg(pTask, pBlock); @@ -624,7 +625,7 @@ int32_t streamDispatchStreamBlock(SStreamTask* pTask) { } stDebug("s-task:%s failed to dispatch msg:%d to downstream, code:%s, output status:%d, retry cnt:%d", id, - pTask->taskExecInfo.dispatchCount, tstrerror(terrno), pTask->outputInfo.status, retryCount); + pTask->execInfo.dispatch, tstrerror(terrno), pTask->outputInfo.status, retryCount); // todo deal with only partially success dispatch case atomic_store_32(&pTask->shuffleDispatcher.waitingRspCnt, 0); @@ -636,11 +637,10 @@ int32_t streamDispatchStreamBlock(SStreamTask* pTask) { if (++retryCount > MAX_CONTINUE_RETRY_COUNT) { // add to timer to retry int8_t ref = atomic_add_fetch_8(&pTask->status.timerActive, 1); - stDebug("s-task:%s failed to dispatch msg to downstream for %d times, code:%s, add timer to retry in %dms, ref:%d", pTask->id.idStr, retryCount, tstrerror(terrno), DISPATCH_RETRY_INTERVAL_MS, ref); - streamRetryDispatchStreamBlock(pTask, DISPATCH_RETRY_INTERVAL_MS); + streamRetryDispatchData(pTask, DISPATCH_RETRY_INTERVAL_MS); break; } } @@ -659,9 +659,9 @@ int32_t streamDispatchScanHistoryFinishMsg(SStreamTask* pTask) { // serialize if (pTask->outputInfo.type == TASK_OUTPUT__FIXED_DISPATCH) { - req.downstreamTaskId = pTask->fixedEpDispatcher.taskId; + req.downstreamTaskId = pTask->fixedDispatcher.taskId; pTask->notReadyTasks = 1; - doDispatchScanHistoryFinishMsg(pTask, &req, pTask->fixedEpDispatcher.nodeId, &pTask->fixedEpDispatcher.epSet); + doDispatchScanHistoryFinishMsg(pTask, &req, pTask->fixedDispatcher.nodeId, &pTask->fixedDispatcher.epSet); } else if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { SArray* vgInfo = pTask->shuffleDispatcher.dbInfo.pVgroupInfos; int32_t numOfVgs = taosArrayGetSize(vgInfo); @@ -1061,7 +1061,7 @@ static int32_t handleDispatchSuccessRsp(SStreamTask* pTask, int32_t downstreamId int32_t streamProcessDispatchRsp(SStreamTask* pTask, SStreamDispatchRsp* pRsp, int32_t code) { const char* id = pTask->id.idStr; int32_t vgId = pTask->pMeta->vgId; - int32_t msgId = pTask->taskExecInfo.dispatchCount; + int32_t msgId = pTask->execInfo.dispatch; if ((!pTask->pMeta->leader) || (pTask->status.downstreamReady != 1)) { stError("s-task:%s vgId:%d is follower or task just re-launched, not handle the dispatch rsp, discard it", id, vgId); @@ -1165,7 +1165,7 @@ int32_t streamProcessDispatchRsp(SStreamTask* pTask, SStreamDispatchRsp* pRsp, i stDebug("s-task:%s failed to dispatch msg to downstream code:%s, add timer to retry in %dms, ref:%d", pTask->id.idStr, tstrerror(terrno), DISPATCH_RETRY_INTERVAL_MS, ref); - streamRetryDispatchStreamBlock(pTask, DISPATCH_RETRY_INTERVAL_MS); + streamRetryDispatchData(pTask, DISPATCH_RETRY_INTERVAL_MS); } else { // this message has been sent successfully, let's try next one. pTask->msgInfo.retryCount = 0; handleDispatchSuccessRsp(pTask, pRsp->downstreamTaskId); diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index 712b0fe610..a87bb00972 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -202,7 +202,7 @@ int32_t streamScanHistoryData(SStreamTask* pTask) { while (!finished) { if (streamTaskShouldPause(&pTask->status)) { - double el = (taosGetTimestampMs() - pTask->taskExecInfo.step1Start) / 1000.0; + double el = (taosGetTimestampMs() - pTask->execInfo.step1Start) / 1000.0; stDebug("s-task:%s paused from the scan-history task, elapsed time:%.2fsec", pTask->id.idStr, el); break; } @@ -556,7 +556,7 @@ int32_t streamExecForAll(SStreamTask* pTask) { // here only handle the data block sink operation if (type == STREAM_INPUT__DATA_BLOCK) { int32_t blockSize = streamQueueItemGetSize(pInput); - pTask->sinkRecorder.bytes += blockSize; + pTask->execInfo.sink.bytes += blockSize; stDebug("s-task:%s sink task start to sink %d blocks, size:%.2fKiB", id, numOfBlocks, SIZE_IN_KiB(blockSize)); doOutputResultBlockImpl(pTask, (SStreamDataBlock*)pInput); diff --git a/source/libs/stream/src/streamRecover.c b/source/libs/stream/src/streamRecover.c index b3838677ce..7bc9898fb0 100644 --- a/source/libs/stream/src/streamRecover.c +++ b/source/libs/stream/src/streamRecover.c @@ -13,16 +13,20 @@ * along with this program. If not, see . */ -#include #include "streamInt.h" #include "trpc.h" #include "ttimer.h" #include "wal.h" -typedef struct SStreamTaskRetryInfo { +typedef struct SLaunchHTaskInfo { SStreamMeta* pMeta; STaskId id; -} SStreamTaskRetryInfo; +} SLaunchHTaskInfo; + +typedef struct STaskRecheckInfo { + SStreamTask* pTask; + SStreamTaskCheckReq req; +} STaskRecheckInfo; static int32_t streamSetParamForScanHistory(SStreamTask* pTask); static void streamTaskSetRangeStreamCalc(SStreamTask* pTask); @@ -39,9 +43,10 @@ static void streamTaskSetReady(SStreamTask* pTask, int32_t numOfReqs) { ASSERT(pTask->status.downstreamReady == 0); pTask->status.downstreamReady = 1; - int64_t el = (taosGetTimestampMs() - pTask->taskExecInfo.init); - stDebug("s-task:%s all %d downstream ready, init completed, elapsed time:%"PRId64"ms, task status:%s", - pTask->id.idStr, numOfReqs, el, streamGetTaskStatusStr(pTask->status.taskStatus)); + pTask->execInfo.start = taosGetTimestampMs(); + int64_t el = (pTask->execInfo.start - pTask->execInfo.init); + stDebug("s-task:%s all %d downstream ready, init completed, elapsed time:%" PRId64 "ms, task status:%s", + pTask->id.idStr, numOfReqs, el, streamGetTaskStatusStr(pTask->status.taskStatus)); } int32_t streamStartScanHistoryAsync(SStreamTask* pTask, int8_t igUntreated) { @@ -126,8 +131,8 @@ static int32_t doCheckDownstreamStatus(SStreamTask* pTask) { // serialize streamProcessScanHistoryFinishRsp if (pTask->outputInfo.type == TASK_OUTPUT__FIXED_DISPATCH) { req.reqId = tGenIdPI64(); - req.downstreamNodeId = pTask->fixedEpDispatcher.nodeId; - req.downstreamTaskId = pTask->fixedEpDispatcher.taskId; + req.downstreamNodeId = pTask->fixedDispatcher.nodeId; + req.downstreamTaskId = pTask->fixedDispatcher.taskId; pTask->checkReqId = req.reqId; stDebug("s-task:%s check single downstream task:0x%x(vgId:%d) ver:%" PRId64 "-%" PRId64 " window:%" PRId64 @@ -135,7 +140,7 @@ static int32_t doCheckDownstreamStatus(SStreamTask* pTask) { pTask->id.idStr, req.downstreamTaskId, req.downstreamNodeId, pRange->range.minVer, pRange->range.maxVer, pWindow->skey, pWindow->ekey, req.stage, req.reqId); - streamDispatchCheckMsg(pTask, &req, pTask->fixedEpDispatcher.nodeId, &pTask->fixedEpDispatcher.epSet); + streamSendCheckMsg(pTask, &req, pTask->fixedDispatcher.nodeId, &pTask->fixedDispatcher.epSet); } else if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { SArray* vgInfo = pTask->shuffleDispatcher.dbInfo.pVgroupInfos; @@ -154,7 +159,7 @@ static int32_t doCheckDownstreamStatus(SStreamTask* pTask) { req.downstreamTaskId = pVgInfo->taskId; stDebug("s-task:%s (vgId:%d) check downstream task:0x%x (vgId:%d) (shuffle), idx:%d, stage:%" PRId64, pTask->id.idStr, pTask->info.nodeId, req.downstreamTaskId, req.downstreamNodeId, i, req.stage); - streamDispatchCheckMsg(pTask, &req, pVgInfo->vgId, &pVgInfo->epSet); + streamSendCheckMsg(pTask, &req, pVgInfo->vgId, &pVgInfo->epSet); } } else { stDebug("s-task:%s (vgId:%d) set downstream ready, since no downstream", pTask->id.idStr, pTask->info.nodeId); @@ -168,8 +173,15 @@ static int32_t doCheckDownstreamStatus(SStreamTask* pTask) { return 0; } -int32_t streamRecheckDownstream(SStreamTask* pTask, const SStreamTaskCheckRsp* pRsp) { - SStreamTaskCheckReq req = { +static STaskRecheckInfo* createRecheckInfo(SStreamTask* pTask, const SStreamTaskCheckRsp* pRsp) { + STaskRecheckInfo* pInfo = taosMemoryCalloc(1, sizeof(STaskRecheckInfo)); + if (pInfo == NULL) { + terrno = TSDB_CODE_OUT_OF_MEMORY; + return NULL; + } + + pInfo->pTask = pTask; + pInfo->req = (SStreamTaskCheckReq){ .reqId = pRsp->reqId, .streamId = pRsp->streamId, .upstreamTaskId = pRsp->upstreamTaskId, @@ -180,25 +192,41 @@ int32_t streamRecheckDownstream(SStreamTask* pTask, const SStreamTaskCheckRsp* p .stage = pTask->pMeta->stage, }; + return pInfo; +} + +static void destroyRecheckInfo(STaskRecheckInfo* pInfo) { + if (pInfo != NULL) { + taosMemoryFree(pInfo); + } +} + +static void recheckDownstreamTasks(void* param, void* tmrId) { + STaskRecheckInfo* pInfo = param; + SStreamTask* pTask = pInfo->pTask; + + SStreamTaskCheckReq* pReq = &pInfo->req; + if (pTask->outputInfo.type == TASK_OUTPUT__FIXED_DISPATCH) { stDebug("s-task:%s (vgId:%d) check downstream task:0x%x (vgId:%d) stage:%" PRId64 " (recheck)", pTask->id.idStr, - pTask->info.nodeId, req.downstreamTaskId, req.downstreamNodeId, req.stage); - streamDispatchCheckMsg(pTask, &req, pRsp->downstreamNodeId, &pTask->fixedEpDispatcher.epSet); + pTask->info.nodeId, pReq->downstreamTaskId, pReq->downstreamNodeId, pReq->stage); + streamSendCheckMsg(pTask, pReq, pReq->downstreamNodeId, &pTask->fixedDispatcher.epSet); } else if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { SArray* vgInfo = pTask->shuffleDispatcher.dbInfo.pVgroupInfos; int32_t numOfVgs = taosArrayGetSize(vgInfo); for (int32_t i = 0; i < numOfVgs; i++) { SVgroupInfo* pVgInfo = taosArrayGet(vgInfo, i); - if (pVgInfo->taskId == req.downstreamTaskId) { + if (pVgInfo->taskId == pReq->downstreamTaskId) { stDebug("s-task:%s (vgId:%d) check downstream task:0x%x (vgId:%d) stage:%" PRId64 " (recheck)", pTask->id.idStr, - pTask->info.nodeId, req.downstreamTaskId, req.downstreamNodeId, req.stage); - streamDispatchCheckMsg(pTask, &req, pRsp->downstreamNodeId, &pVgInfo->epSet); + pTask->info.nodeId, pReq->downstreamTaskId, pReq->downstreamNodeId, pReq->stage); + streamSendCheckMsg(pTask, pReq, pReq->downstreamNodeId, &pVgInfo->epSet); } } } - return 0; + destroyRecheckInfo(pInfo); + atomic_sub_fetch_8(&pTask->status.timerActive, 1); } int32_t streamTaskCheckStatus(SStreamTask* pTask, int32_t upstreamTaskId, int32_t vgId, int64_t stage) { @@ -265,6 +293,11 @@ int32_t streamProcessCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* pRs ASSERT(pTask->id.taskId == pRsp->upstreamTaskId); const char* id = pTask->id.idStr; + if (streamTaskShouldStop(&pTask->status)) { + stDebug("s-task:%s should stop, do not do check downstream again", id); + return TSDB_CODE_SUCCESS; + } + if (pRsp->status == TASK_DOWNSTREAM_READY) { if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { bool found = false; @@ -293,7 +326,7 @@ int32_t streamProcessCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* pRs } else { int32_t total = taosArrayGetSize(pTask->shuffleDispatcher.dbInfo.pVgroupInfos); stDebug("s-task:%s (vgId:%d) recv check rsp from task:0x%x (vgId:%d) status:%d, total:%d not ready:%d", id, - pRsp->upstreamNodeId, pRsp->downstreamTaskId, pRsp->downstreamNodeId, pRsp->status, total, left); + pRsp->upstreamNodeId, pRsp->downstreamTaskId, pRsp->downstreamNodeId, pRsp->status, total, left); } } else { ASSERT(pTask->outputInfo.type == TASK_OUTPUT__FIXED_DISPATCH); @@ -305,18 +338,28 @@ int32_t streamProcessCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* pRs } } else { // not ready, wait for 100ms and retry if (pRsp->status == TASK_DOWNSTREAM_NOT_LEADER) { - stError("s-task:%s downstream taskId:0x%x (vgId:%d) vnode-transfer/leader-change detected, roll-back needed not send check again", - id, pRsp->downstreamTaskId, pRsp->downstreamNodeId); + stError( + "s-task:%s downstream taskId:0x%x (vgId:%d) vnode-transfer/leader-change detected, not send check again, " + "roll-back needed", + id, pRsp->downstreamTaskId, pRsp->downstreamNodeId); } else if (pRsp->status == TASK_SELF_NEW_STAGE) { stError( - "s-task:%s vnode-transfer/leader-change/restart detected, old stage:%d, current stage:%d, roll-back needed " - "and not send check again", - id, pRsp->oldStage, (int32_t) pTask->pMeta->stage); + "s-task:%s vnode-transfer/leader-change/restart detected, old stage:%d, current stage:%d, not send check " + "again, roll-back needed", + id, pRsp->oldStage, (int32_t)pTask->pMeta->stage); } else { - stDebug("s-task:%s downstream taskId:0x%x (vgId:%d) not ready, stage:%d, wait for 100ms and retry", id, + stDebug("s-task:%s downstream taskId:0x%x (vgId:%d) not ready, stage:%d, retry in 100ms", id, pRsp->downstreamTaskId, pRsp->downstreamNodeId, pRsp->oldStage); - taosMsleep(100); - streamRecheckDownstream(pTask, pRsp); + + STaskTimer* pTmr = pTask->pTimer; + STaskRecheckInfo* pInfo = createRecheckInfo(pTask, pRsp); + + atomic_add_fetch_8(&pTask->status.timerActive, 1); + if (pTmr->checkTimer != NULL) { + taosTmrReset(recheckDownstreamTasks, CHECK_DOWNSTREAM_INTERVAL, pInfo, streamEnv.timer, &pTmr->checkTimer); + } else { + pTmr->checkTimer = taosTmrStart(recheckDownstreamTasks, CHECK_DOWNSTREAM_INTERVAL, pInfo, streamEnv.timer); + } } } @@ -547,8 +590,8 @@ static void checkFillhistoryTaskStatus(SStreamTask* pTask, SStreamTask* pHTask) } static void tryLaunchHistoryTask(void* param, void* tmrId) { - SStreamTaskRetryInfo* pInfo = param; - SStreamMeta* pMeta = pInfo->pMeta; + SLaunchHTaskInfo* pInfo = param; + SStreamMeta* pMeta = pInfo->pMeta; stDebug("s-task:0x%x in timer to launch related history task", (int32_t) pInfo->id.taskId); @@ -582,7 +625,7 @@ static void tryLaunchHistoryTask(void* param, void* tmrId) { "destroyed, or should stop", pTask->id.idStr, pMeta->vgId, pStatus, (int32_t) pTask->historyTaskId.taskId); - taosTmrReset(tryLaunchHistoryTask, 100, pInfo, streamEnv.timer, &pTask->launchTaskTimer); + taosTmrReset(tryLaunchHistoryTask, 100, pInfo, streamEnv.timer, &pTask->pTimer->hTaskLaunchTimer); streamMetaReleaseTask(pMeta, pTask); return; } @@ -621,14 +664,14 @@ int32_t streamLaunchFillHistoryTask(SStreamTask* pTask) { stWarn("s-task:%s vgId:%d failed to launch history task:0x%x, since it is not built yet", pTask->id.idStr, pMeta->vgId, hTaskId); - SStreamTaskRetryInfo* pInfo = taosMemoryCalloc(1, sizeof(SStreamTaskRetryInfo)); + SLaunchHTaskInfo* pInfo = taosMemoryCalloc(1, sizeof(SLaunchHTaskInfo)); pInfo->id.taskId = pTask->id.taskId; pInfo->id.streamId = pTask->id.streamId; pInfo->pMeta = pTask->pMeta; - if (pTask->launchTaskTimer == NULL) { - pTask->launchTaskTimer = taosTmrStart(tryLaunchHistoryTask, 100, pInfo, streamEnv.timer); - if (pTask->launchTaskTimer == NULL) { + if (pTask->pTimer->hTaskLaunchTimer == NULL) { + pTask->pTimer->hTaskLaunchTimer = taosTmrStart(tryLaunchHistoryTask, 100, pInfo, streamEnv.timer); + if (pTask->pTimer->hTaskLaunchTimer == NULL) { // todo failed to create timer taosMemoryFree(pInfo); } else { @@ -639,7 +682,7 @@ int32_t streamLaunchFillHistoryTask(SStreamTask* pTask) { } else { // timer exists ASSERT(pTask->status.timerActive == 1); stDebug("s-task:%s set timer active flag, task timer not null", pTask->id.idStr); - taosTmrReset(tryLaunchHistoryTask, 100, pInfo, streamEnv.timer, &pTask->launchTaskTimer); + taosTmrReset(tryLaunchHistoryTask, 100, pInfo, streamEnv.timer, &pTask->pTimer->hTaskLaunchTimer); } // try again in 100ms diff --git a/source/libs/stream/src/streamTask.c b/source/libs/stream/src/streamTask.c index 56f97c565a..30189ad185 100644 --- a/source/libs/stream/src/streamTask.c +++ b/source/libs/stream/src/streamTask.c @@ -129,9 +129,9 @@ int32_t tEncodeStreamTask(SEncoder* pEncoder, const SStreamTask* pTask) { } else if (pTask->outputInfo.type == TASK_OUTPUT__FETCH) { if (tEncodeI8(pEncoder, pTask->fetchSink.reserved) < 0) return -1; } else if (pTask->outputInfo.type == TASK_OUTPUT__FIXED_DISPATCH) { - if (tEncodeI32(pEncoder, pTask->fixedEpDispatcher.taskId) < 0) return -1; - if (tEncodeI32(pEncoder, pTask->fixedEpDispatcher.nodeId) < 0) return -1; - if (tEncodeSEpSet(pEncoder, &pTask->fixedEpDispatcher.epSet) < 0) return -1; + if (tEncodeI32(pEncoder, pTask->fixedDispatcher.taskId) < 0) return -1; + if (tEncodeI32(pEncoder, pTask->fixedDispatcher.nodeId) < 0) return -1; + if (tEncodeSEpSet(pEncoder, &pTask->fixedDispatcher.epSet) < 0) return -1; } else if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { if (tSerializeSUseDbRspImp(pEncoder, &pTask->shuffleDispatcher.dbInfo) < 0) return -1; if (tEncodeCStr(pEncoder, pTask->shuffleDispatcher.stbFullName) < 0) return -1; @@ -211,9 +211,9 @@ int32_t tDecodeStreamTask(SDecoder* pDecoder, SStreamTask* pTask) { } else if (pTask->outputInfo.type == TASK_OUTPUT__FETCH) { if (tDecodeI8(pDecoder, &pTask->fetchSink.reserved) < 0) return -1; } else if (pTask->outputInfo.type == TASK_OUTPUT__FIXED_DISPATCH) { - if (tDecodeI32(pDecoder, &pTask->fixedEpDispatcher.taskId) < 0) return -1; - if (tDecodeI32(pDecoder, &pTask->fixedEpDispatcher.nodeId) < 0) return -1; - if (tDecodeSEpSet(pDecoder, &pTask->fixedEpDispatcher.epSet) < 0) return -1; + if (tDecodeI32(pDecoder, &pTask->fixedDispatcher.taskId) < 0) return -1; + if (tDecodeI32(pDecoder, &pTask->fixedDispatcher.nodeId) < 0) return -1; + if (tDecodeSEpSet(pDecoder, &pTask->fixedDispatcher.epSet) < 0) return -1; } else if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { if (tDeserializeSUseDbRspImp(pDecoder, &pTask->shuffleDispatcher.dbInfo) < 0) return -1; if (tDecodeCStrTo(pDecoder, pTask->shuffleDispatcher.stbFullName) < 0) return -1; @@ -289,20 +289,17 @@ static void freeUpstreamItem(void* p) { void tFreeStreamTask(SStreamTask* pTask) { int32_t taskId = pTask->id.taskId; - STaskExecStatisInfo* pStatis = &pTask->taskExecInfo; + STaskExecStatisInfo* pStatis = &pTask->execInfo; stDebug("start to free s-task:0x%x, %p, state:%p, status:%s", taskId, pTask, pTask->pState, streamGetTaskStatusStr(pTask->status.taskStatus)); - stDebug("s-task:0x%x exec info: create:%" PRId64 ", init:%" PRId64 ", start:%" PRId64 + stDebug("s-task:0x%x task exec summary: create:%" PRId64 ", init:%" PRId64 ", start:%" PRId64 ", updateCount:%d latestUpdate:%" PRId64 ", latestCheckPoint:%" PRId64 ", ver:%" PRId64 - " nextProcessVer:%" PRId64, + " nextProcessVer:%" PRId64", checkpointCount:%d", taskId, pStatis->created, pStatis->init, pStatis->start, pStatis->updateCount, pStatis->latestUpdateTs, - pTask->chkInfo.checkpointId, pTask->chkInfo.checkpointVer, pTask->chkInfo.nextProcessVer); - - if (pStatis->created == 0 || pStatis->init == 0 || pStatis->start == 0) { - int32_t k = 1; - } + pTask->chkInfo.checkpointId, pTask->chkInfo.checkpointVer, pTask->chkInfo.nextProcessVer, + pStatis->checkpoint); // remove the ref by timer while (pTask->status.timerActive > 0) { @@ -315,9 +312,22 @@ void tFreeStreamTask(SStreamTask* pTask) { pTask->schedInfo.pTimer = NULL; } - if (pTask->launchTaskTimer != NULL) { - taosTmrStop(pTask->launchTaskTimer); - pTask->launchTaskTimer = NULL; + if (pTask->pTimer != NULL) { + if (pTask->pTimer->hTaskLaunchTimer != NULL) { + taosTmrStop(pTask->pTimer->hTaskLaunchTimer); + pTask->pTimer->hTaskLaunchTimer = NULL; + } + + if (pTask->pTimer->dispatchTimer != NULL) { + taosTmrStop(pTask->pTimer->dispatchTimer); + pTask->pTimer->dispatchTimer = NULL; + } + + if (pTask->pTimer->checkTimer != NULL) { + taosTmrStop(pTask->pTimer->checkTimer); + pTask->pTimer->checkTimer = NULL; + } + taosMemoryFreeClear(pTask->pTimer); } int32_t status = atomic_load_8((int8_t*)&(pTask->status.taskStatus)); @@ -402,7 +412,7 @@ int32_t streamTaskInit(SStreamTask* pTask, SStreamMeta* pMeta, SMsgCb* pMsgCb, i return TSDB_CODE_OUT_OF_MEMORY; } - pTask->taskExecInfo.created = taosGetTimestampMs(); + pTask->execInfo.created = taosGetTimestampMs(); pTask->inputInfo.status = TASK_INPUT_STATUS__NORMAL; pTask->outputInfo.status = TASK_OUTPUT_STATUS__NORMAL; pTask->pMeta = pMeta; @@ -419,6 +429,12 @@ int32_t streamTaskInit(SStreamTask* pTask, SStreamMeta* pMeta, SMsgCb* pMsgCb, i return TSDB_CODE_OUT_OF_MEMORY; } + pTask->pTimer = taosMemoryCalloc(1, sizeof(STaskTimer)); + if (pTask->pTimer == NULL) { + stError("s-task:%s failed to prepare the timer, code:%s", pTask->id.idStr, tstrerror(TSDB_CODE_OUT_OF_MEMORY)); + return TSDB_CODE_OUT_OF_MEMORY; + } + streamTaskInitTokenBucket(pTask->pTokenBucket, 50, 50); TdThreadMutexAttr attr = {0}; @@ -501,7 +517,7 @@ void streamTaskUpdateUpstreamInfo(SStreamTask* pTask, int32_t nodeId, const SEpS } void streamTaskSetFixedDownstreamInfo(SStreamTask* pTask, const SStreamTask* pDownstreamTask) { - STaskDispatcherFixedEp* pDispatcher = &pTask->fixedEpDispatcher; + STaskDispatcherFixed* pDispatcher = &pTask->fixedDispatcher; pDispatcher->taskId = pDownstreamTask->id.taskId; pDispatcher->nodeId = pDownstreamTask->info.nodeId; pDispatcher->epSet = pDownstreamTask->info.epSet; @@ -530,7 +546,7 @@ void streamTaskUpdateDownstreamInfo(SStreamTask* pTask, int32_t nodeId, const SE } } } else if (type == TASK_OUTPUT__FIXED_DISPATCH) { - STaskDispatcherFixedEp* pDispatcher = &pTask->fixedEpDispatcher; + STaskDispatcherFixed* pDispatcher = &pTask->fixedDispatcher; if (pDispatcher->nodeId == nodeId) { epsetAssign(&pDispatcher->epSet, pEpSet); stDebug("s-task:0x%x update the dispatch info, task:0x%x(nodeId:%d) newEpSet:%s", pTask->id.taskId, @@ -598,7 +614,7 @@ int32_t doUpdateTaskEpset(SStreamTask* pTask, int32_t nodeId, SEpSet* pEpSet) { } int32_t streamTaskUpdateEpsetInfo(SStreamTask* pTask, SArray* pNodeList) { - STaskExecStatisInfo* p = &pTask->taskExecInfo; + STaskExecStatisInfo* p = &pTask->execInfo; int32_t numOfNodes = taosArrayGetSize(pNodeList); int64_t prevTs = p->latestUpdateTs; From 477f57aa0e1c466a069fd39b52dfbe0610c748a6 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sun, 24 Sep 2023 01:55:59 +0800 Subject: [PATCH 083/158] refactor: do some internal refactor. --- source/libs/stream/src/streamRecover.c | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/source/libs/stream/src/streamRecover.c b/source/libs/stream/src/streamRecover.c index 7bc9898fb0..6b673ee1da 100644 --- a/source/libs/stream/src/streamRecover.c +++ b/source/libs/stream/src/streamRecover.c @@ -348,13 +348,13 @@ int32_t streamProcessCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* pRs "again, roll-back needed", id, pRsp->oldStage, (int32_t)pTask->pMeta->stage); } else { - stDebug("s-task:%s downstream taskId:0x%x (vgId:%d) not ready, stage:%d, retry in 100ms", id, - pRsp->downstreamTaskId, pRsp->downstreamNodeId, pRsp->oldStage); - STaskTimer* pTmr = pTask->pTimer; STaskRecheckInfo* pInfo = createRecheckInfo(pTask, pRsp); - atomic_add_fetch_8(&pTask->status.timerActive, 1); + int8_t ref = atomic_add_fetch_8(&pTask->status.timerActive, 1); + stDebug("s-task:%s downstream taskId:0x%x (vgId:%d) not ready, stage:%d, retry in 100ms, ref:%d ", id, + pRsp->downstreamTaskId, pRsp->downstreamNodeId, pRsp->oldStage, ref); + if (pTmr->checkTimer != NULL) { taosTmrReset(recheckDownstreamTasks, CHECK_DOWNSTREAM_INTERVAL, pInfo, streamEnv.timer, &pTmr->checkTimer); } else { @@ -677,7 +677,7 @@ int32_t streamLaunchFillHistoryTask(SStreamTask* pTask) { } else { int32_t ref = atomic_add_fetch_8(&pTask->status.timerActive, 1);// timer is active ASSERT(ref == 1); - stDebug("s-task:%s set timer active flag", pTask->id.idStr); + stDebug("s-task:%s set timer active flag, ref:%d", pTask->id.idStr, ref); } } else { // timer exists ASSERT(pTask->status.timerActive == 1); From 7336228c169952f9fea862996422068bdea4d1f2 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sun, 24 Sep 2023 01:56:52 +0800 Subject: [PATCH 084/158] refactor: do some internal refactor. --- source/libs/stream/src/streamRecover.c | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/source/libs/stream/src/streamRecover.c b/source/libs/stream/src/streamRecover.c index 6b673ee1da..72d05d0a74 100644 --- a/source/libs/stream/src/streamRecover.c +++ b/source/libs/stream/src/streamRecover.c @@ -226,7 +226,8 @@ static void recheckDownstreamTasks(void* param, void* tmrId) { } destroyRecheckInfo(pInfo); - atomic_sub_fetch_8(&pTask->status.timerActive, 1); + int8_t ref = atomic_sub_fetch_8(&pTask->status.timerActive, 1); + stDebug("s-task:%s complete send check in timer, ref:%d", pTask->id.idStr, ref); } int32_t streamTaskCheckStatus(SStreamTask* pTask, int32_t upstreamTaskId, int32_t vgId, int64_t stage) { From 7167c3c5de373224f787aac5c7d4db0359ba4caa Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sun, 24 Sep 2023 02:25:53 +0800 Subject: [PATCH 085/158] fix(stream): add new timer for each task. --- source/libs/stream/inc/streamInt.h | 1 - source/libs/stream/src/streamRecover.c | 12 +++++------- source/libs/stream/src/streamTask.c | 4 ---- 3 files changed, 5 insertions(+), 12 deletions(-) diff --git a/source/libs/stream/inc/streamInt.h b/source/libs/stream/inc/streamInt.h index 6600d7dd04..dbe868b54f 100644 --- a/source/libs/stream/inc/streamInt.h +++ b/source/libs/stream/inc/streamInt.h @@ -58,7 +58,6 @@ struct STokenBucket { struct STaskTimer { void* hTaskLaunchTimer; void* dispatchTimer; - void* checkTimer; }; extern SStreamGlobalEnv streamEnv; diff --git a/source/libs/stream/src/streamRecover.c b/source/libs/stream/src/streamRecover.c index 72d05d0a74..ff1728a6eb 100644 --- a/source/libs/stream/src/streamRecover.c +++ b/source/libs/stream/src/streamRecover.c @@ -24,8 +24,9 @@ typedef struct SLaunchHTaskInfo { } SLaunchHTaskInfo; typedef struct STaskRecheckInfo { - SStreamTask* pTask; + SStreamTask* pTask; SStreamTaskCheckReq req; + void* checkTimer; } STaskRecheckInfo; static int32_t streamSetParamForScanHistory(SStreamTask* pTask); @@ -197,6 +198,8 @@ static STaskRecheckInfo* createRecheckInfo(SStreamTask* pTask, const SStreamTask static void destroyRecheckInfo(STaskRecheckInfo* pInfo) { if (pInfo != NULL) { + taosTmrStop(pInfo->checkTimer); + pInfo->checkTimer = NULL; taosMemoryFree(pInfo); } } @@ -349,18 +352,13 @@ int32_t streamProcessCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* pRs "again, roll-back needed", id, pRsp->oldStage, (int32_t)pTask->pMeta->stage); } else { - STaskTimer* pTmr = pTask->pTimer; STaskRecheckInfo* pInfo = createRecheckInfo(pTask, pRsp); int8_t ref = atomic_add_fetch_8(&pTask->status.timerActive, 1); stDebug("s-task:%s downstream taskId:0x%x (vgId:%d) not ready, stage:%d, retry in 100ms, ref:%d ", id, pRsp->downstreamTaskId, pRsp->downstreamNodeId, pRsp->oldStage, ref); - if (pTmr->checkTimer != NULL) { - taosTmrReset(recheckDownstreamTasks, CHECK_DOWNSTREAM_INTERVAL, pInfo, streamEnv.timer, &pTmr->checkTimer); - } else { - pTmr->checkTimer = taosTmrStart(recheckDownstreamTasks, CHECK_DOWNSTREAM_INTERVAL, pInfo, streamEnv.timer); - } + taosTmrReset(recheckDownstreamTasks, CHECK_DOWNSTREAM_INTERVAL, pInfo, streamEnv.timer, &pInfo->checkTimer); } } diff --git a/source/libs/stream/src/streamTask.c b/source/libs/stream/src/streamTask.c index 30189ad185..4925eea262 100644 --- a/source/libs/stream/src/streamTask.c +++ b/source/libs/stream/src/streamTask.c @@ -323,10 +323,6 @@ void tFreeStreamTask(SStreamTask* pTask) { pTask->pTimer->dispatchTimer = NULL; } - if (pTask->pTimer->checkTimer != NULL) { - taosTmrStop(pTask->pTimer->checkTimer); - pTask->pTimer->checkTimer = NULL; - } taosMemoryFreeClear(pTask->pTimer); } From fc96ec6bae3906a97be6a860e81cdbd46f4a4f11 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sun, 24 Sep 2023 22:03:07 +0800 Subject: [PATCH 086/158] fix(stream): do not wait for the checkpoint finish before stop tasks. --- source/libs/stream/src/streamTask.c | 19 +++++-------------- 1 file changed, 5 insertions(+), 14 deletions(-) diff --git a/source/libs/stream/src/streamTask.c b/source/libs/stream/src/streamTask.c index 4925eea262..91e91fe3d8 100644 --- a/source/libs/stream/src/streamTask.c +++ b/source/libs/stream/src/streamTask.c @@ -558,21 +558,12 @@ int32_t streamTaskStop(SStreamTask* pTask) { int64_t st = taosGetTimestampMs(); const char* id = pTask->id.idStr; - // we should wait for the task complete the checkpoint operation before stop it, otherwise, the operation maybe blocked - // by the unfinished checkpoint operation, even if the leader has become the follower. - while(1) { - taosThreadMutexLock(&pTask->lock); - - if (pTask->status.taskStatus == TASK_STATUS__CK) { - stDebug("s-task:%s in checkpoint, wait for it completed for 500ms before stop task", pTask->id.idStr); - taosThreadMutexUnlock(&pTask->lock); - taosMsleep(500); - } else { - pTask->status.taskStatus = TASK_STATUS__STOP; - taosThreadMutexUnlock(&pTask->lock); - break; - } + taosThreadMutexLock(&pTask->lock); + if (pTask->status.taskStatus == TASK_STATUS__CK) { + stDebug("s-task:%s in checkpoint will be discarded since task is stopped", pTask->id.idStr); } + pTask->status.taskStatus = TASK_STATUS__STOP; + taosThreadMutexUnlock(&pTask->lock); qKillTask(pTask->exec.pExecutor, TSDB_CODE_SUCCESS); From 68b003c8043d652e261e09e4261b979834afabd9 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 25 Sep 2023 01:37:27 +0800 Subject: [PATCH 087/158] fix(stream): fix race condition. --- include/libs/stream/tstream.h | 2 +- source/dnode/vnode/src/tq/tq.c | 12 ++++++------ source/dnode/vnode/src/vnd/vnodeSync.c | 7 +------ source/libs/stream/src/streamRecover.c | 22 ++++++++++++++-------- 4 files changed, 22 insertions(+), 21 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 20cf7bb110..59bf050677 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -405,7 +405,7 @@ typedef struct SStreamMeta { int32_t vgId; int64_t stage; bool leader; - int8_t taskWillbeLaunched; + int8_t taskStartedByNodeUpdate; SRWLatch lock; int32_t walScanCounter; void* streamBackend; diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 9f678a5563..543289d621 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1830,15 +1830,16 @@ int32_t tqProcessTaskUpdateReq(STQ* pTq, SRpcMsg* pMsg) { int32_t numOfTasks = streamMetaGetNumOfTasks(pMeta); int32_t updateTasks = taosHashGetSize(pMeta->pUpdateTaskSet); if (updateTasks < numOfTasks) { - pMeta->taskWillbeLaunched = 1; - - tqDebug("vgId:%d closed tasks:%d, unclosed:%d", vgId, updateTasks, (numOfTasks - updateTasks)); + pMeta->taskStartedByNodeUpdate = 1; + tqDebug("vgId:%d closed tasks:%d, unclosed:%d, all tasks will be started when nodeEp update completed", vgId, + updateTasks, (numOfTasks - updateTasks)); taosWUnLockLatch(&pMeta->lock); } else { taosHashClear(pMeta->pUpdateTaskSet); if (!pTq->pVnode->restored) { - tqDebug("vgId:%d vnode restore not completed, not restart the tasks", vgId); + tqDebug("vgId:%d vnode restore not completed, not restart the tasks, clear the start after nodeUpdate flag", vgId); + pMeta->taskStartedByNodeUpdate = 0; taosWUnLockLatch(&pMeta->lock); } else { tqDebug("vgId:%d tasks are all updated and stopped, restart them", vgId); @@ -1860,14 +1861,13 @@ int32_t tqProcessTaskUpdateReq(STQ* pTq, SRpcMsg* pMsg) { } if (vnodeIsRoleLeader(pTq->pVnode) && !tsDisableStream) { - vInfo("vgId:%d, restart all stream tasks", vgId); + vInfo("vgId:%d restart all stream tasks", vgId); tqStartStreamTasks(pTq); tqCheckAndRunStreamTaskAsync(pTq); } else { vInfo("vgId:%d, follower node not start stream tasks", vgId); } - pMeta->taskWillbeLaunched = 0; taosWUnLockLatch(&pMeta->lock); } } diff --git a/source/dnode/vnode/src/vnd/vnodeSync.c b/source/dnode/vnode/src/vnd/vnodeSync.c index cf299dc79c..2d96bcfffd 100644 --- a/source/dnode/vnode/src/vnd/vnodeSync.c +++ b/source/dnode/vnode/src/vnd/vnodeSync.c @@ -551,13 +551,8 @@ static void vnodeRestoreFinish(const SSyncFSM *pFsm, const SyncIndex commitIdx) walApplyVer(pVnode->pWal, commitIdx); pVnode->restored = true; - if (pVnode->pTq->pStreamMeta->taskWillbeLaunched) { - vInfo("vgId:%d, sync restore finished, stream tasks will be launched by other thread", vgId); - return; - } - taosWLockLatch(&pVnode->pTq->pStreamMeta->lock); - if (pVnode->pTq->pStreamMeta->taskWillbeLaunched) { + if (pVnode->pTq->pStreamMeta->taskStartedByNodeUpdate) { vInfo("vgId:%d, sync restore finished, stream tasks will be launched by other thread", vgId); taosWUnLockLatch(&pVnode->pTq->pStreamMeta->lock); return; diff --git a/source/libs/stream/src/streamRecover.c b/source/libs/stream/src/streamRecover.c index ff1728a6eb..3ae50b5b2b 100644 --- a/source/libs/stream/src/streamRecover.c +++ b/source/libs/stream/src/streamRecover.c @@ -118,8 +118,8 @@ int32_t streamTaskLaunchScanHistory(SStreamTask* pTask) { // check status static int32_t doCheckDownstreamStatus(SStreamTask* pTask) { - SDataRange* pRange = &pTask->dataRange; - STimeWindow* pWindow = &pRange->window; + SDataRange* pRange = &pTask->dataRange; + STimeWindow* pWindow = &pRange->window; SStreamTaskCheckReq req = { .streamId = pTask->id.streamId, @@ -136,10 +136,10 @@ static int32_t doCheckDownstreamStatus(SStreamTask* pTask) { req.downstreamTaskId = pTask->fixedDispatcher.taskId; pTask->checkReqId = req.reqId; - stDebug("s-task:%s check single downstream task:0x%x(vgId:%d) ver:%" PRId64 "-%" PRId64 " window:%" PRId64 - "-%" PRId64 ", stage:%"PRId64" req:0x%" PRIx64, - pTask->id.idStr, req.downstreamTaskId, req.downstreamNodeId, pRange->range.minVer, pRange->range.maxVer, - pWindow->skey, pWindow->ekey, req.stage, req.reqId); + stDebug("s-task:%s stage:%" PRId64 " check single downstream task:0x%x(vgId:%d) ver:%" PRId64 "-%" PRId64 + " window:%" PRId64 "-%" PRId64 " req:0x%" PRIx64, + pTask->id.idStr, req.reqId, req.downstreamTaskId, req.downstreamNodeId, pRange->range.minVer, + pRange->range.maxVer, pWindow->skey, pWindow->ekey, req.reqId); streamSendCheckMsg(pTask, &req, pTask->fixedDispatcher.nodeId, &pTask->fixedDispatcher.epSet); } else if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { @@ -158,8 +158,8 @@ static int32_t doCheckDownstreamStatus(SStreamTask* pTask) { taosArrayPush(pTask->checkReqIds, &req.reqId); req.downstreamNodeId = pVgInfo->vgId; req.downstreamTaskId = pVgInfo->taskId; - stDebug("s-task:%s (vgId:%d) check downstream task:0x%x (vgId:%d) (shuffle), idx:%d, stage:%" PRId64, - pTask->id.idStr, pTask->info.nodeId, req.downstreamTaskId, req.downstreamNodeId, i, req.stage); + stDebug("s-task:%s (vgId:%d) stage:%" PRId64 "check downstream task:0x%x (vgId:%d) (shuffle), idx:%d", + pTask->id.idStr, pTask->info.nodeId, req.stage, req.downstreamTaskId, req.downstreamNodeId, i); streamSendCheckMsg(pTask, &req, pVgInfo->vgId, &pVgInfo->epSet); } } else { @@ -907,6 +907,12 @@ void streamTaskPause(SStreamTask* pTask, SStreamMeta* pMeta) { stDebug("vgId:%d s-task:%s task already stopped/paused, status:%s, do nothing", pMeta->vgId, pTask->id.idStr, str); return; } +// +// if (pTask->status.downstreamReady == 0) { +// ASSERT(pTask->execInfo.start == 0); +// stDebug("s-task:%s in check downstream procedure, abort and paused", pTask->id.idStr); +// break; +// } const char* pStatus = streamGetTaskStatusStr(status); stDebug("s-task:%s wait for the task can be paused, status:%s, vgId:%d", pTask->id.idStr, pStatus, pMeta->vgId); From f4f55fcbcee175aaf8a63ccbe9032a395517e2ff Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 25 Sep 2023 09:59:05 +0800 Subject: [PATCH 088/158] fix(stream): record the tasks started info, and do some internal refactor. --- include/libs/stream/tstream.h | 9 +++++- source/dnode/vnode/src/tq/tq.c | 4 +-- source/dnode/vnode/src/tq/tqStreamTask.c | 3 +- source/dnode/vnode/src/vnd/vnodeSync.c | 2 +- source/libs/stream/src/streamQueue.c | 35 ------------------------ source/libs/stream/src/streamRecover.c | 16 +++++++++++ 6 files changed, 29 insertions(+), 40 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 59bf050677..c4ab632837 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -391,6 +391,13 @@ struct SStreamTask { char reserve[256]; }; +typedef struct STaskStartInfo { + int64_t ts; + int32_t startedAfterNodeUpdate; + int32_t readyTasks; + int32_t elapsedTime; +} STaskStartInfo; + // meta typedef struct SStreamMeta { char* path; @@ -405,7 +412,7 @@ typedef struct SStreamMeta { int32_t vgId; int64_t stage; bool leader; - int8_t taskStartedByNodeUpdate; + STaskStartInfo startInfo; SRWLatch lock; int32_t walScanCounter; void* streamBackend; diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 543289d621..1e3a27d567 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1830,7 +1830,7 @@ int32_t tqProcessTaskUpdateReq(STQ* pTq, SRpcMsg* pMsg) { int32_t numOfTasks = streamMetaGetNumOfTasks(pMeta); int32_t updateTasks = taosHashGetSize(pMeta->pUpdateTaskSet); if (updateTasks < numOfTasks) { - pMeta->taskStartedByNodeUpdate = 1; + pMeta->startInfo.startedAfterNodeUpdate = 1; tqDebug("vgId:%d closed tasks:%d, unclosed:%d, all tasks will be started when nodeEp update completed", vgId, updateTasks, (numOfTasks - updateTasks)); taosWUnLockLatch(&pMeta->lock); @@ -1839,7 +1839,7 @@ int32_t tqProcessTaskUpdateReq(STQ* pTq, SRpcMsg* pMsg) { if (!pTq->pVnode->restored) { tqDebug("vgId:%d vnode restore not completed, not restart the tasks, clear the start after nodeUpdate flag", vgId); - pMeta->taskStartedByNodeUpdate = 0; + pMeta->startInfo.startedAfterNodeUpdate = 0; taosWUnLockLatch(&pMeta->lock); } else { tqDebug("vgId:%d tasks are all updated and stopped, restart them", vgId); diff --git a/source/dnode/vnode/src/tq/tqStreamTask.c b/source/dnode/vnode/src/tq/tqStreamTask.c index b39132d675..34cc74852f 100644 --- a/source/dnode/vnode/src/tq/tqStreamTask.c +++ b/source/dnode/vnode/src/tq/tqStreamTask.c @@ -231,11 +231,12 @@ int32_t tqStartStreamTasks(STQ* pTq) { int32_t numOfTasks = taosArrayGetSize(pMeta->pTaskList); tqDebug("vgId:%d start all %d stream task(s)", vgId, numOfTasks); - if (numOfTasks == 0) { return TSDB_CODE_SUCCESS; } + pMeta->startInfo.ts = taosGetTimestampMs(); + for (int32_t i = 0; i < numOfTasks; ++i) { SStreamTaskId* pTaskId = taosArrayGet(pMeta->pTaskList, i); diff --git a/source/dnode/vnode/src/vnd/vnodeSync.c b/source/dnode/vnode/src/vnd/vnodeSync.c index 2d96bcfffd..a6c743c87d 100644 --- a/source/dnode/vnode/src/vnd/vnodeSync.c +++ b/source/dnode/vnode/src/vnd/vnodeSync.c @@ -552,7 +552,7 @@ static void vnodeRestoreFinish(const SSyncFSM *pFsm, const SyncIndex commitIdx) pVnode->restored = true; taosWLockLatch(&pVnode->pTq->pStreamMeta->lock); - if (pVnode->pTq->pStreamMeta->taskStartedByNodeUpdate) { + if (pVnode->pTq->pStreamMeta->startInfo.startedAfterNodeUpdate) { vInfo("vgId:%d, sync restore finished, stream tasks will be launched by other thread", vgId); taosWUnLockLatch(&pVnode->pTq->pStreamMeta->lock); return; diff --git a/source/libs/stream/src/streamQueue.c b/source/libs/stream/src/streamQueue.c index 882c57383e..abf10487de 100644 --- a/source/libs/stream/src/streamQueue.c +++ b/source/libs/stream/src/streamQueue.c @@ -154,41 +154,6 @@ int32_t streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueItem** pInpu int32_t taskLevel = pTask->info.taskLevel; *numOfBlocks = 0; - // if (pTask->info.taskLevel == TASK_LEVEL__SINK) { // extract block from inputQ, one-by-one - // while (1) { - // if (streamTaskShouldPause(&pTask->status) || streamTaskShouldStop(&pTask->status)) { - // stDebug("s-task:%s task should pause, extract input blocks:%d", pTask->id.idStr, *numOfBlocks); - // return TSDB_CODE_SUCCESS; - // } - // - // STokenBucket* pBucket = pTask->pTokenBucket; - // // if (!streamTaskHasAvailableToken(pBucket)) { // no available token in th bucket, ignore this - // execution - // // stInfo("s-task:%s no available token for sink, capacity:%d, rate:%d token/sec, quit", - // pTask->id.idStr, - // // pBucket->capacity, pBucket->rate); - // // return TSDB_CODE_SUCCESS; - // // } - // - // SStreamQueueItem* qItem = streamQueueNextItem(pTask->inputInfo.queue); - // if (qItem == NULL) { - // if (++retryTimes < MAX_RETRY_TIMES) { - // taosMsleep(10); - // continue; - // } - // - // return TSDB_CODE_SUCCESS; - // } - // - // stDebug("s-task:%s sink task handle block, type:%s", id, streamQueueItemGetTypeStr(qItem->type)); - // pTask->sinkRecorder.bytes += streamQueueItemGetSize(qItem); - // - // *numOfBlocks = 1; - // *pInput = qItem; - // return TSDB_CODE_SUCCESS; - // } - // } - while (1) { if (streamTaskShouldPause(&pTask->status) || streamTaskShouldStop(&pTask->status)) { stDebug("s-task:%s task should pause, extract input blocks:%d", pTask->id.idStr, *numOfBlocks); diff --git a/source/libs/stream/src/streamRecover.c b/source/libs/stream/src/streamRecover.c index 3ae50b5b2b..d3a45a5e62 100644 --- a/source/libs/stream/src/streamRecover.c +++ b/source/libs/stream/src/streamRecover.c @@ -34,6 +34,9 @@ static void streamTaskSetRangeStreamCalc(SStreamTask* pTask); static int32_t initScanHistoryReq(SStreamTask* pTask, SStreamScanHistoryReq* pReq, int8_t igUntreated); static void streamTaskSetReady(SStreamTask* pTask, int32_t numOfReqs) { + SStreamMeta* pMeta = pTask->pMeta; + int32_t vgId = pMeta->vgId; + if (pTask->status.taskStatus == TASK_STATUS__SCAN_HISTORY && pTask->info.taskLevel != TASK_LEVEL__SOURCE) { pTask->numOfWaitingUpstream = taosArrayGetSize(pTask->pUpstreamInfoList); stDebug("s-task:%s level:%d task wait for %d upstream tasks complete scan-history procedure, status:%s", @@ -48,6 +51,19 @@ static void streamTaskSetReady(SStreamTask* pTask, int32_t numOfReqs) { int64_t el = (pTask->execInfo.start - pTask->execInfo.init); stDebug("s-task:%s all %d downstream ready, init completed, elapsed time:%" PRId64 "ms, task status:%s", pTask->id.idStr, numOfReqs, el, streamGetTaskStatusStr(pTask->status.taskStatus)); + + taosWLockLatch(&pMeta->lock); + pMeta->startInfo.readyTasks += 1; + int32_t numOfTotal = streamMetaGetNumOfTasks(pMeta); + if (pMeta->startInfo.readyTasks == numOfTotal) { + // reset value for next time start + pMeta->startInfo.readyTasks = 0; + pMeta->startInfo.startedAfterNodeUpdate = 0; + pMeta->startInfo.elapsedTime = pTask->execInfo.start - pMeta->startInfo.ts; + stDebug("vgId:%d all %d task(s) are started successfully, last ready task:%s level:%d, total elapsed time:%.2f sec", + vgId, numOfTotal, pTask->id.idStr, pTask->info.taskLevel, pMeta->startInfo.elapsedTime / 1000.0); + } + taosWUnLockLatch(&pMeta->lock); } int32_t streamStartScanHistoryAsync(SStreamTask* pTask, int8_t igUntreated) { From 4fe1382cd92b3ee3b16108e6b33148871be87d52 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 25 Sep 2023 10:40:30 +0800 Subject: [PATCH 089/158] fix(stream): adjust nodeUpdate check interval. --- source/common/src/tglobal.c | 2 +- source/dnode/mnode/impl/src/mndStream.c | 2 +- source/libs/stream/src/streamRecover.c | 13 +++++++------ 3 files changed, 9 insertions(+), 8 deletions(-) diff --git a/source/common/src/tglobal.c b/source/common/src/tglobal.c index 85b1028bb7..99de36af43 100644 --- a/source/common/src/tglobal.c +++ b/source/common/src/tglobal.c @@ -242,7 +242,7 @@ int32_t tsTtlBatchDropNum = 10000; // number of tables dropped per batch int32_t tsTransPullupInterval = 2; int32_t tsMqRebalanceInterval = 2; int32_t tsStreamCheckpointTickInterval = 10; -int32_t tsStreamNodeCheckInterval = 10; +int32_t tsStreamNodeCheckInterval = 30; int32_t tsTtlUnit = 86400; int32_t tsTtlPushIntervalSec = 10; int32_t tsTrimVDbIntervalSec = 60 * 60; // interval of trimming db in all vgroups diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index 30e0791294..43ce2423a1 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -2418,7 +2418,7 @@ int32_t mndProcessStreamHb(SRpcMsg *pReq) { for(int32_t j = 0; j < numOfNodes; ++j) { SNodeEntry* pNodeEntry = taosArrayGet(execNodeList.pNodeEntryList, j); if (pNodeEntry->nodeId == pEntry->nodeId) { - mInfo("vgId:%d stage updated, from %d to %d, nodeUpdate should be trigger by s-task:0x%" PRIx64, + mInfo("vgId:%d stage updated, from %d to %d, nodeUpdate trigger by s-task:0x%" PRIx64, pEntry->nodeId, pEntry->stage, p->stage, pEntry->id.taskId); pNodeEntry->stageUpdated = true; diff --git a/source/libs/stream/src/streamRecover.c b/source/libs/stream/src/streamRecover.c index d3a45a5e62..1bf4f2bf98 100644 --- a/source/libs/stream/src/streamRecover.c +++ b/source/libs/stream/src/streamRecover.c @@ -255,20 +255,21 @@ int32_t streamTaskCheckStatus(SStreamTask* pTask, int32_t upstreamTaskId, int32_ const char* id = pTask->id.idStr; if (stage == -1) { - stDebug("s-task:%s receive check msg from upstream task:0x%x, invalid stageId:%" PRId64 ", not ready", id, - upstreamTaskId, stage); + stDebug("s-task:%s receive check msg from upstream task:0x%x(vgId:%d), invalid stageId:%" PRId64 ", not ready", id, + upstreamTaskId, vgId, stage); return 0; } if (pInfo->stage == -1) { pInfo->stage = stage; - stDebug("s-task:%s receive check msg from upstream task:0x%x first time, init stage value:%" PRId64, id, - upstreamTaskId, stage); + stDebug("s-task:%s receive check msg from upstream task:0x%x(vgId:%d) first time, init stage value:%" PRId64, id, + upstreamTaskId, vgId, stage); } if (pInfo->stage < stage) { - stError("s-task:%s receive msg from upstream task:0x%x(vgId:%d), new stage received:%" PRId64 ", prev:%" PRId64, - id, upstreamTaskId, vgId, stage, pInfo->stage); + stError("s-task:%s receive check msg from upstream task:0x%x(vgId:%d), new stage received:%" PRId64 + ", prev:%" PRId64, + id, upstreamTaskId, vgId, stage, pInfo->stage); } if (pTask->status.downstreamReady != 1) { From 4af7812f0be9586129aa1c9808a1710ffe260c81 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 25 Sep 2023 15:58:24 +0800 Subject: [PATCH 090/158] fix(stream): fix the invalid counter for ready tasks in vnode, fix an invalid read. --- include/libs/stream/tstream.h | 8 ++++---- source/dnode/vnode/src/tq/tq.c | 2 +- source/dnode/vnode/src/tq/tqStreamTask.c | 4 ++-- source/libs/stream/src/streamData.c | 3 ++- source/libs/stream/src/streamMeta.c | 5 +++++ source/libs/stream/src/streamRecover.c | 18 +++++++++++------- 6 files changed, 25 insertions(+), 15 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index c4ab632837..ce4d7fa4b6 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -392,10 +392,10 @@ struct SStreamTask { }; typedef struct STaskStartInfo { - int64_t ts; - int32_t startedAfterNodeUpdate; - int32_t readyTasks; - int32_t elapsedTime; + int64_t ts; + int32_t startedAfterNodeUpdate; + SHashObj* pReadyTaskSet; // tasks that are all ready for running stream processing + int32_t elapsedTime; } STaskStartInfo; // meta diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 1e3a27d567..610db1b72f 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1861,7 +1861,7 @@ int32_t tqProcessTaskUpdateReq(STQ* pTq, SRpcMsg* pMsg) { } if (vnodeIsRoleLeader(pTq->pVnode) && !tsDisableStream) { - vInfo("vgId:%d restart all stream tasks", vgId); + vInfo("vgId:%d restart all stream tasks after all tasks being updated", vgId); tqStartStreamTasks(pTq); tqCheckAndRunStreamTaskAsync(pTq); } else { diff --git a/source/dnode/vnode/src/tq/tqStreamTask.c b/source/dnode/vnode/src/tq/tqStreamTask.c index 34cc74852f..c3ef52e96f 100644 --- a/source/dnode/vnode/src/tq/tqStreamTask.c +++ b/source/dnode/vnode/src/tq/tqStreamTask.c @@ -72,6 +72,8 @@ int32_t tqCheckAndRunStreamTask(STQ* pTq) { SArray* pTaskList = NULL; taosWLockLatch(&pMeta->lock); pTaskList = taosArrayDup(pMeta->pTaskList, NULL); + taosHashClear(pMeta->startInfo.pReadyTaskSet); + pMeta->startInfo.ts = taosGetTimestampMs(); taosWUnLockLatch(&pMeta->lock); // broadcast the check downstream tasks msg @@ -235,8 +237,6 @@ int32_t tqStartStreamTasks(STQ* pTq) { return TSDB_CODE_SUCCESS; } - pMeta->startInfo.ts = taosGetTimestampMs(); - for (int32_t i = 0; i < numOfTasks; ++i) { SStreamTaskId* pTaskId = taosArrayGet(pMeta->pTaskList, i); diff --git a/source/libs/stream/src/streamData.c b/source/libs/stream/src/streamData.c index 51487e5588..a0fee6be8f 100644 --- a/source/libs/stream/src/streamData.c +++ b/source/libs/stream/src/streamData.c @@ -184,12 +184,13 @@ SStreamQueueItem* streamMergeQueueItem(SStreamQueueItem* dst, SStreamQueueItem* return NULL; } + streamQueueItemIncSize((SStreamQueueItem*)pMerged, streamQueueItemGetSize(pElem)); + streamMergeSubmit(pMerged, (SStreamDataSubmit*)dst); streamMergeSubmit(pMerged, (SStreamDataSubmit*)pElem); taosFreeQitem(dst); taosFreeQitem(pElem); - streamQueueItemIncSize((SStreamQueueItem*)pMerged, streamQueueItemGetSize(pElem)); return (SStreamQueueItem*)pMerged; } else { stDebug("block type:%s not merged with existed blocks list, type:%d", streamQueueItemGetTypeStr(pElem->type), dst->type); diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 7cb3fad292..c0513d2e7e 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -152,6 +152,11 @@ SStreamMeta* streamMetaOpen(const char* path, void* ahandle, FTaskExpand expandF goto _err; } + pMeta->startInfo.pReadyTaskSet = taosHashInit(64, fp, false, HASH_NO_LOCK); + if (pMeta->startInfo.pReadyTaskSet == NULL) { + + } + pMeta->pHbInfo = taosMemoryCalloc(1, sizeof(SMetaHbInfo)); if (pMeta->pHbInfo == NULL) { goto _err; diff --git a/source/libs/stream/src/streamRecover.c b/source/libs/stream/src/streamRecover.c index 1bf4f2bf98..2a277f3fca 100644 --- a/source/libs/stream/src/streamRecover.c +++ b/source/libs/stream/src/streamRecover.c @@ -53,13 +53,17 @@ static void streamTaskSetReady(SStreamTask* pTask, int32_t numOfReqs) { pTask->id.idStr, numOfReqs, el, streamGetTaskStatusStr(pTask->status.taskStatus)); taosWLockLatch(&pMeta->lock); - pMeta->startInfo.readyTasks += 1; + + STaskId id = extractStreamTaskKey(pTask); + taosHashPut(pMeta->startInfo.pReadyTaskSet, &id, sizeof(id), NULL, 0); int32_t numOfTotal = streamMetaGetNumOfTasks(pMeta); - if (pMeta->startInfo.readyTasks == numOfTotal) { + + if (taosHashGetSize(pMeta->startInfo.pReadyTaskSet) == numOfTotal) { // reset value for next time start - pMeta->startInfo.readyTasks = 0; + taosHashClear(pMeta->startInfo.pReadyTaskSet); pMeta->startInfo.startedAfterNodeUpdate = 0; pMeta->startInfo.elapsedTime = pTask->execInfo.start - pMeta->startInfo.ts; + stDebug("vgId:%d all %d task(s) are started successfully, last ready task:%s level:%d, total elapsed time:%.2f sec", vgId, numOfTotal, pTask->id.idStr, pTask->info.taskLevel, pMeta->startInfo.elapsedTime / 1000.0); } @@ -174,7 +178,7 @@ static int32_t doCheckDownstreamStatus(SStreamTask* pTask) { taosArrayPush(pTask->checkReqIds, &req.reqId); req.downstreamNodeId = pVgInfo->vgId; req.downstreamTaskId = pVgInfo->taskId; - stDebug("s-task:%s (vgId:%d) stage:%" PRId64 "check downstream task:0x%x (vgId:%d) (shuffle), idx:%d", + stDebug("s-task:%s (vgId:%d) stage:%" PRId64 " check downstream task:0x%x (vgId:%d) (shuffle), idx:%d", pTask->id.idStr, pTask->info.nodeId, req.stage, req.downstreamTaskId, req.downstreamNodeId, i); streamSendCheckMsg(pTask, &req, pVgInfo->vgId, &pVgInfo->epSet); } @@ -272,10 +276,10 @@ int32_t streamTaskCheckStatus(SStreamTask* pTask, int32_t upstreamTaskId, int32_ id, upstreamTaskId, vgId, stage, pInfo->stage); } - if (pTask->status.downstreamReady != 1) { - return TASK_DOWNSTREAM_NOT_READY; - } else if (pInfo->stage != stage) { + if (pInfo->stage != stage) { return TASK_SELF_NEW_STAGE; + } else if (pTask->status.downstreamReady != 1) { + return TASK_DOWNSTREAM_NOT_READY; } else { return TASK_DOWNSTREAM_READY; } From a32b56f3811373b866e1a27b63a7c676fab814a0 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 25 Sep 2023 16:01:17 +0800 Subject: [PATCH 091/158] fix(stream): fix memory leak. --- source/client/test/clientTests.cpp | 2 +- source/libs/stream/src/streamMeta.c | 3 ++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/source/client/test/clientTests.cpp b/source/client/test/clientTests.cpp index a1c8690dfc..6481dfe5bf 100644 --- a/source/client/test/clientTests.cpp +++ b/source/client/test/clientTests.cpp @@ -832,7 +832,7 @@ TEST(clientCase, projection_query_tables) { for(int32_t i = 0; i < 1000000; ++i) { char t[512] = {0}; - sprintf(t, "insert into t1 values(%ld, %ld)", start + i, i); + sprintf(t, "insert into t1 values(now, %ld)", i); while(1) { void* p = taos_query(pConn, t); code = taos_errno(p); diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index c0513d2e7e..2760d31828 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -225,7 +225,7 @@ _err: if (pMeta->db) tdbClose(pMeta->db); if (pMeta->pHbInfo) taosMemoryFreeClear(pMeta->pHbInfo); if (pMeta->pUpdateTaskSet) taosHashCleanup(pMeta->pUpdateTaskSet); - + if (pMeta->startInfo.pReadyTaskSet) taosHashCleanup(pMeta->startInfo.pReadyTaskSet); taosMemoryFree(pMeta); stError("failed to open stream meta"); @@ -340,6 +340,7 @@ void streamMetaCloseImpl(void* arg) { taosHashCleanup(pMeta->pTasksMap); taosHashCleanup(pMeta->pTaskBackendUnique); taosHashCleanup(pMeta->pUpdateTaskSet); + taosHashCleanup(pMeta->startInfo.pReadyTaskSet); taosMemoryFree(pMeta->pHbInfo); taosMemoryFree(pMeta->path); From 65f66f0986f6e07dae0a09c76006c421e95ca597 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 25 Sep 2023 17:31:38 +0800 Subject: [PATCH 092/158] fix(stream): transfer state when all downstream rsp already. --- source/dnode/mnode/impl/src/mndStream.c | 1 + source/libs/stream/src/streamDispatch.c | 29 +++++++++++++------------ 2 files changed, 16 insertions(+), 14 deletions(-) diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index 43ce2423a1..57d14d000b 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -2246,6 +2246,7 @@ static int32_t removeInvalidStreamTask(SArray *pNodeSnapshot) { execNodeList.pNodeEntryList = taosArrayDestroy(execNodeList.pNodeEntryList); execNodeList.pNodeEntryList = pValidNodeEntryList; + taosArrayDestroy(pRemoveTaskList); return 0; } diff --git a/source/libs/stream/src/streamDispatch.c b/source/libs/stream/src/streamDispatch.c index bf7abc7457..fc3e17c826 100644 --- a/source/libs/stream/src/streamDispatch.c +++ b/source/libs/stream/src/streamDispatch.c @@ -1111,20 +1111,6 @@ int32_t streamProcessDispatchRsp(SStreamTask* pTask, SStreamDispatchRsp* pRsp, i stError("s-task:%s downstream task:0x%x(vgId:%d) refused the dispatch msg, treat it as success", id, pRsp->downstreamTaskId, pRsp->downstreamNodeId); } - - // transtate msg has been sent to downstream successfully. let's transfer the fill-history task state - if (pTask->msgInfo.dispatchMsgType == STREAM_INPUT__TRANS_STATE) { - stDebug("s-task:%s dispatch transtate msgId:%d to downstream successfully, start to transfer state", id, msgId); - ASSERT(pTask->info.fillHistory == 1); - - code = streamTransferStateToStreamTask(pTask); - if (code != TSDB_CODE_SUCCESS) { // todo: do nothing if error happens - } - - // now ready for next data output - atomic_store_8(&pTask->outputInfo.status, TASK_OUTPUT_STATUS__NORMAL); - return TSDB_CODE_SUCCESS; - } } int32_t leftRsp = 0; @@ -1168,6 +1154,21 @@ int32_t streamProcessDispatchRsp(SStreamTask* pTask, SStreamDispatchRsp* pRsp, i streamRetryDispatchData(pTask, DISPATCH_RETRY_INTERVAL_MS); } else { // this message has been sent successfully, let's try next one. pTask->msgInfo.retryCount = 0; + + // transtate msg has been sent to downstream successfully. let's transfer the fill-history task state + if (pTask->msgInfo.dispatchMsgType == STREAM_INPUT__TRANS_STATE) { + stDebug("s-task:%s dispatch transtate msgId:%d to downstream successfully, start to transfer state", id, msgId); + ASSERT(pTask->info.fillHistory == 1); + + code = streamTransferStateToStreamTask(pTask); + if (code != TSDB_CODE_SUCCESS) { // todo: do nothing if error happens + } + + // now ready for next data output + atomic_store_8(&pTask->outputInfo.status, TASK_OUTPUT_STATUS__NORMAL); + return TSDB_CODE_SUCCESS; + } + handleDispatchSuccessRsp(pTask, pRsp->downstreamTaskId); } } From 6a98b11beffc48cbcc175b89805877f30674a021 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 25 Sep 2023 23:10:33 +0800 Subject: [PATCH 093/158] fix(stream): fix invalid read and memory leak. --- source/dnode/vnode/src/tq/tqSink.c | 2 ++ source/libs/stream/src/streamData.c | 10 +++++----- 2 files changed, 7 insertions(+), 5 deletions(-) diff --git a/source/dnode/vnode/src/tq/tqSink.c b/source/dnode/vnode/src/tq/tqSink.c index 81c3d3d07d..9158702284 100644 --- a/source/dnode/vnode/src/tq/tqSink.c +++ b/source/dnode/vnode/src/tq/tqSink.c @@ -824,6 +824,8 @@ void tqSinkDataIntoDstTable(SStreamTask* pTask, void* vnode, void* data) { bool hasSubmit = false; for (int32_t i = 0; i < numOfBlocks; i++) { if (streamTaskShouldStop(&pTask->status)) { + taosHashCleanup(pTableIndexMap); + tDestroySubmitReq(&submitReq, TSDB_MSG_FLG_ENCODE); return; } diff --git a/source/libs/stream/src/streamData.c b/source/libs/stream/src/streamData.c index a0fee6be8f..80927b36b9 100644 --- a/source/libs/stream/src/streamData.c +++ b/source/libs/stream/src/streamData.c @@ -165,17 +165,17 @@ SStreamQueueItem* streamMergeQueueItem(SStreamQueueItem* dst, SStreamQueueItem* SStreamDataBlock* pBlockSrc = (SStreamDataBlock*)pElem; taosArrayAddAll(pBlock->blocks, pBlockSrc->blocks); taosArrayDestroy(pBlockSrc->blocks); - taosFreeQitem(pElem); - streamQueueItemIncSize(dst, streamQueueItemGetSize(pElem)); + + taosFreeQitem(pElem); return dst; } else if (dst->type == STREAM_INPUT__MERGED_SUBMIT && pElem->type == STREAM_INPUT__DATA_SUBMIT) { SStreamMergedSubmit* pMerged = (SStreamMergedSubmit*)dst; SStreamDataSubmit* pBlockSrc = (SStreamDataSubmit*)pElem; streamMergeSubmit(pMerged, pBlockSrc); - taosFreeQitem(pElem); - streamQueueItemIncSize(dst, streamQueueItemGetSize(pElem)); + + taosFreeQitem(pElem); return dst; } else if (dst->type == STREAM_INPUT__DATA_SUBMIT && pElem->type == STREAM_INPUT__DATA_SUBMIT) { SStreamMergedSubmit* pMerged = streamMergedSubmitNew(); @@ -188,9 +188,9 @@ SStreamQueueItem* streamMergeQueueItem(SStreamQueueItem* dst, SStreamQueueItem* streamMergeSubmit(pMerged, (SStreamDataSubmit*)dst); streamMergeSubmit(pMerged, (SStreamDataSubmit*)pElem); + taosFreeQitem(dst); taosFreeQitem(pElem); - return (SStreamQueueItem*)pMerged; } else { stDebug("block type:%s not merged with existed blocks list, type:%d", streamQueueItemGetTypeStr(pElem->type), dst->type); From feb09c8361406ff09771fd5ad0593a26c4ce2fae Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 26 Sep 2023 01:09:55 +0800 Subject: [PATCH 094/158] fix(stream): fix error in pause scan-history data. --- include/libs/stream/tstream.h | 1 - source/dnode/vnode/src/tq/tq.c | 17 ++++++-- source/libs/stream/src/streamRecover.c | 56 +++++++++++++------------- 3 files changed, 42 insertions(+), 32 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index ce4d7fa4b6..9420c5235d 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -698,7 +698,6 @@ int32_t streamSetStatusUnint(SStreamTask* pTask); const char* streamGetTaskStatusStr(int32_t status); void streamTaskPause(SStreamTask* pTask, SStreamMeta* pMeta); void streamTaskResume(SStreamTask* pTask, SStreamMeta* pMeta); -void streamTaskHalt(SStreamTask* pTask); void streamTaskResumeFromHalt(SStreamTask* pTask); void streamTaskDisablePause(SStreamTask* pTask); void streamTaskEnablePause(SStreamTask* pTask); diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 610db1b72f..7f0842736e 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1135,8 +1135,12 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { } if (status == TASK_STATUS__HALT) { - // return; - // do nothing +// tqDebug("s-task:%s level:%d sched-status:%d is halt by fill-history task:%s", pStreamTask->id.idStr, +// pStreamTask->info.taskLevel, pStreamTask->status.schedStatus, id); +// latestVer = walReaderGetCurrentVer(pStreamTask->exec.pWalReader); +// +// taosThreadMutexUnlock(&pStreamTask->lock); +// break; } if (pStreamTask->status.taskStatus == TASK_STATUS__CK) { @@ -1152,7 +1156,7 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { qDebug("s-task:%s upgrade status to %s from %s", pStreamTask->id.idStr, streamGetTaskStatusStr(TASK_STATUS__HALT), streamGetTaskStatusStr(TASK_STATUS__PAUSE)); } else { - qDebug("s-task:%s halt task", pStreamTask->id.idStr); + qDebug("s-task:%s halt task, prev status:%s", pStreamTask->id.idStr, streamGetTaskStatusStr(status)); } pStreamTask->status.keepTaskStatus = status; @@ -1174,6 +1178,13 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { pTask->execInfo.step2Start = taosGetTimestampMs(); qDebug("s-task:%s scan-history from WAL stage(step 2) ended, elapsed time:%.2fs", id, 0.0); streamTaskPutTranstateIntoInputQ(pTask); + + if (pTask->status.taskStatus == TASK_STATUS__PAUSE) { + pTask->status.keepTaskStatus = TASK_STATUS__NORMAL; + qDebug("s-task:%s prev status is %s, update the kept status to be:%s when after step 2", id, + streamGetTaskStatusStr(TASK_STATUS__PAUSE), streamGetTaskStatusStr(pTask->status.keepTaskStatus)); + } + streamTryExec(pTask); // exec directly } else { STimeWindow* pWindow = &pTask->dataRange.window; diff --git a/source/libs/stream/src/streamRecover.c b/source/libs/stream/src/streamRecover.c index 2a277f3fca..d910c7d785 100644 --- a/source/libs/stream/src/streamRecover.c +++ b/source/libs/stream/src/streamRecover.c @@ -1001,34 +1001,34 @@ void streamTaskEnablePause(SStreamTask* pTask) { } // fix: this function should be removed, it may cause deadlock. -void streamTaskHalt(SStreamTask* pTask) { - int8_t status = pTask->status.taskStatus; - if (status == TASK_STATUS__DROPPING || status == TASK_STATUS__STOP) { - return; - } - - if (status == TASK_STATUS__HALT) { - return; - } - - // wait for checkpoint completed - while(pTask->status.taskStatus == TASK_STATUS__CK) { - qDebug("s-task:%s status:%s during generating checkpoint, wait for 1sec and retry set status:halt", pTask->id.idStr, - streamGetTaskStatusStr(TASK_STATUS__CK)); - taosMsleep(1000); - } - - // upgrade to halt status - if (status == TASK_STATUS__PAUSE) { - stDebug("s-task:%s upgrade status to %s from %s", pTask->id.idStr, streamGetTaskStatusStr(TASK_STATUS__HALT), - streamGetTaskStatusStr(TASK_STATUS__PAUSE)); - } else { - stDebug("s-task:%s halt task", pTask->id.idStr); - } - - pTask->status.keepTaskStatus = status; - pTask->status.taskStatus = TASK_STATUS__HALT; -} +//void streamTaskHalt(SStreamTask* pTask) { +// int8_t status = pTask->status.taskStatus; +// if (status == TASK_STATUS__DROPPING || status == TASK_STATUS__STOP) { +// return; +// } +// +// if (status == TASK_STATUS__HALT) { +// return; +// } +// +// // wait for checkpoint completed +// while(pTask->status.taskStatus == TASK_STATUS__CK) { +// qDebug("s-task:%s status:%s during generating checkpoint, wait for 1sec and retry set status:halt", pTask->id.idStr, +// streamGetTaskStatusStr(TASK_STATUS__CK)); +// taosMsleep(1000); +// } +// +// // upgrade to halt status +// if (status == TASK_STATUS__PAUSE) { +// stDebug("s-task:%s upgrade status to %s from %s", pTask->id.idStr, streamGetTaskStatusStr(TASK_STATUS__HALT), +// streamGetTaskStatusStr(TASK_STATUS__PAUSE)); +// } else { +// stDebug("s-task:%s halt task", pTask->id.idStr); +// } +// +// pTask->status.keepTaskStatus = status; +// pTask->status.taskStatus = TASK_STATUS__HALT; +//} void streamTaskResumeFromHalt(SStreamTask* pTask) { const char* id = pTask->id.idStr; From 3912d80c02a5f3b4eab277fdd0e450d6805bc8c4 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 26 Sep 2023 09:32:57 +0800 Subject: [PATCH 095/158] fix(stream): do some internal refactor. --- include/libs/stream/tstream.h | 1 + source/common/src/tglobal.c | 2 +- source/dnode/snode/src/snode.c | 15 +++++++-------- source/libs/stream/src/streamDispatch.c | 6 ------ source/libs/stream/src/streamMeta.c | 5 +++++ tests/system-test/0-others/information_schema.py | 4 ++-- tests/system-test/2-query/db.py | 2 +- 7 files changed, 17 insertions(+), 18 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 9420c5235d..e093625ef1 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -739,6 +739,7 @@ int32_t streamMetaCommit(SStreamMeta* pMeta); int32_t streamMetaLoadAllTasks(SStreamMeta* pMeta); void streamMetaNotifyClose(SStreamMeta* pMeta); void streamMetaStartHb(SStreamMeta* pMeta); +void streamMetaInitForSnode(SStreamMeta* pMeta); // checkpoint int32_t streamProcessCheckpointSourceReq(SStreamTask* pTask, SStreamCheckpointSourceReq* pReq); diff --git a/source/common/src/tglobal.c b/source/common/src/tglobal.c index 99de36af43..1bbb969e7b 100644 --- a/source/common/src/tglobal.c +++ b/source/common/src/tglobal.c @@ -404,7 +404,7 @@ static int32_t taosAddServerLogCfg(SConfig *pCfg) { if (cfgAddInt32(pCfg, "idxDebugFlag", idxDebugFlag, 0, 255, CFG_SCOPE_SERVER) != 0) return -1; if (cfgAddInt32(pCfg, "tdbDebugFlag", tdbDebugFlag, 0, 255, CFG_SCOPE_SERVER) != 0) return -1; if (cfgAddInt32(pCfg, "metaDebugFlag", metaDebugFlag, 0, 255, 0) != CFG_SCOPE_SERVER) return -1; - if (cfgAddInt32(pCfg, "stDebugFlag", stDebugFlag, 0, 255, CFG_SCOPE_CLIENT) != 0) return -1; + if (cfgAddInt32(pCfg, "stDebugFlag", stDebugFlag, 0, 255, CFG_SCOPE_SERVER) != 0) return -1; return 0; } diff --git a/source/dnode/snode/src/snode.c b/source/dnode/snode/src/snode.c index d6e575d97c..dbbd68fa08 100644 --- a/source/dnode/snode/src/snode.c +++ b/source/dnode/snode/src/snode.c @@ -112,18 +112,16 @@ SSnode *sndOpen(const char *path, const SSnodeOpt *pOption) { terrno = TSDB_CODE_OUT_OF_MEMORY; goto FAIL; } - pSnode->msgCb = pOption->msgCb; + pSnode->msgCb = pOption->msgCb; pSnode->pMeta = streamMetaOpen(path, pSnode, (FTaskExpand *)sndExpandTask, SNODE_HANDLE, -1); if (pSnode->pMeta == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; goto FAIL; } - // todo fix it: send msg to mnode to rollback to an existed checkpoint, and broadcast the rollback msg to all other - // computing nodes. - pSnode->pMeta->stage = 0; - + // todo fix it: send msg to mnode to rollback to an existed checkpoint + streamMetaInitForSnode(pSnode->pMeta); return pSnode; FAIL: @@ -258,10 +256,11 @@ int32_t sndProcessTaskRetrieveReq(SSnode *pSnode, SRpcMsg *pMsg) { int32_t sndProcessTaskDispatchRsp(SSnode *pSnode, SRpcMsg *pMsg) { SStreamDispatchRsp *pRsp = POINTER_SHIFT(pMsg->pCont, sizeof(SMsgHead)); - int32_t taskId = htonl(pRsp->upstreamTaskId); - int64_t streamId = htobe64(pRsp->streamId); + pRsp->upstreamTaskId = htonl(pRsp->upstreamTaskId); + pRsp->streamId = htobe64(pRsp->streamId); + pRsp->msgId = htonl(pRsp->msgId); - SStreamTask *pTask = streamMetaAcquireTask(pSnode->pMeta, streamId, taskId); + SStreamTask *pTask = streamMetaAcquireTask(pSnode->pMeta, pRsp->streamId, pRsp->upstreamTaskId); if (pTask) { streamProcessDispatchRsp(pTask, pRsp, pMsg->code); streamMetaReleaseTask(pSnode->pMeta, pTask); diff --git a/source/libs/stream/src/streamDispatch.c b/source/libs/stream/src/streamDispatch.c index fc3e17c826..aa32fb6493 100644 --- a/source/libs/stream/src/streamDispatch.c +++ b/source/libs/stream/src/streamDispatch.c @@ -1083,12 +1083,6 @@ int32_t streamProcessDispatchRsp(SStreamTask* pTask, SStreamDispatchRsp* pRsp, i if (code == TSDB_CODE_STREAM_TASK_NOT_EXIST) { // destination task does not exist, not retry anymore stError("s-task:%s failed to dispatch msg to task:0x%x(vgId:%d), msgId:%d no retry, since task destroyed already", id, pRsp->downstreamTaskId, pRsp->downstreamNodeId, msgId); - -// SStreamDataBlock* pMsgBlock = pTask->msgInfo.pData; -// if (pMsgBlock->type == STREAM_INPUT__CHECKPOINT_TRIGGER) { -// stError("s-task:%s checkpoint trigger send failed, continue do checkpoint ready process", id); -// streamProcessCheckpointReadyMsg(pTask); -// } } else { stError("s-task:%s failed to dispatch msgId:%d to task:0x%x(vgId:%d), code:%s, add to retry list", id, msgId, pRsp->downstreamTaskId, pRsp->downstreamNodeId, tstrerror(code)); diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 2760d31828..5e25d911b0 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -999,3 +999,8 @@ void streamMetaStartHb(SStreamMeta* pMeta) { *pRid = pMeta->rid; metaHbToMnode(pRid, NULL); } + +void streamMetaInitForSnode(SStreamMeta* pMeta) { + pMeta->stage = 0; + pMeta->leader = true; +} \ No newline at end of file diff --git a/tests/system-test/0-others/information_schema.py b/tests/system-test/0-others/information_schema.py index baf0682fbb..99bd90b006 100644 --- a/tests/system-test/0-others/information_schema.py +++ b/tests/system-test/0-others/information_schema.py @@ -22,7 +22,7 @@ class TDTestCase: def init(self, conn, logSql, replicaVar=1): self.replicaVar = int(replicaVar) tdLog.debug("start to execute %s" % __file__) - tdSql.init(conn.cursor()) + tdSql.init(conn.cursor(), True) self.setsql = TDSetSql() self.dbname = 'db' self.stbname = 'stb' @@ -217,7 +217,7 @@ class TDTestCase: tdSql.checkEqual(20470,len(tdSql.queryResult)) tdSql.query("select * from information_schema.ins_columns where db_name ='information_schema'") - tdSql.checkEqual(193, len(tdSql.queryResult)) + tdSql.checkEqual(194, len(tdSql.queryResult)) tdSql.query("select * from information_schema.ins_columns where db_name ='performance_schema'") tdSql.checkEqual(54, len(tdSql.queryResult)) diff --git a/tests/system-test/2-query/db.py b/tests/system-test/2-query/db.py index 451fc0caf3..6870c59a0d 100644 --- a/tests/system-test/2-query/db.py +++ b/tests/system-test/2-query/db.py @@ -55,7 +55,7 @@ class TDTestCase: tdSql.checkData(0, 2, 0) tdSql.query("show dnode 1 variables like '%debugFlag'") - tdSql.checkRows(21) + tdSql.checkRows(22) tdSql.query("show dnode 1 variables like '____debugFlag'") tdSql.checkRows(2) From 0db357401df875a912f86aa2584cfd7910467fe6 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 26 Sep 2023 12:15:12 +0800 Subject: [PATCH 096/158] fix(stream): add new status for meta. --- include/libs/stream/tstream.h | 7 +++++- source/dnode/vnode/src/inc/vnodeInt.h | 6 ++--- source/dnode/vnode/src/tq/tq.c | 12 +++++----- source/dnode/vnode/src/tq/tqSink.c | 19 ++++++++-------- source/dnode/vnode/src/tq/tqUtil.c | 2 +- source/dnode/vnode/src/vnd/vnodeSvr.c | 6 ++--- source/libs/stream/src/stream.c | 2 +- source/libs/stream/src/streamDispatch.c | 16 ++++++------- source/libs/stream/src/streamMeta.c | 19 +++++++++------- source/libs/stream/src/streamRecover.c | 30 ------------------------- source/libs/stream/src/streamTask.c | 10 ++++----- 11 files changed, 54 insertions(+), 75 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index e093625ef1..a98ad5a4c2 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -40,6 +40,10 @@ extern "C" { #define TASK_DOWNSTREAM_NOT_LEADER 0x2 #define TASK_SELF_NEW_STAGE 0x3 +#define NODE_ROLE_UNINIT 0x1 +#define NODE_ROLE_LEADER 0x2 +#define NODE_ROLE_FOLLOWER 0x3 + typedef struct SStreamTask SStreamTask; #define SSTREAM_TASK_VER 2 @@ -411,7 +415,8 @@ typedef struct SStreamMeta { FTaskExpand* expandFunc; int32_t vgId; int64_t stage; - bool leader; +// bool leader; + int32_t role; STaskStartInfo startInfo; SRWLatch lock; int32_t walScanCounter; diff --git a/source/dnode/vnode/src/inc/vnodeInt.h b/source/dnode/vnode/src/inc/vnodeInt.h index a29d595ef7..3a62f52bdd 100644 --- a/source/dnode/vnode/src/inc/vnodeInt.h +++ b/source/dnode/vnode/src/inc/vnodeInt.h @@ -225,7 +225,7 @@ int tqRegisterPushHandle(STQ* pTq, void* handle, SRpcMsg* pMsg); int tqUnregisterPushHandle(STQ* pTq, void* pHandle); int tqScanWalAsync(STQ* pTq, bool ckPause); int32_t tqProcessStreamCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg* pRsp); -int32_t tqProcessStreamTaskCheckpointReadyMsg(STQ* pTq, SRpcMsg* pMsg); +int32_t tqProcessTaskCheckpointReadyMsg(STQ* pTq, SRpcMsg* pMsg); int32_t tqProcessTaskUpdateReq(STQ* pTq, SRpcMsg* pMsg); int32_t tqCheckAndRunStreamTaskAsync(STQ* pTq); @@ -249,8 +249,8 @@ int32_t tqProcessTaskDeployReq(STQ* pTq, int64_t version, char* msg, int32_t msg int32_t tqProcessTaskDropReq(STQ* pTq, int64_t version, char* msg, int32_t msgLen); int32_t tqProcessTaskPauseReq(STQ* pTq, int64_t version, char* msg, int32_t msgLen); int32_t tqProcessTaskResumeReq(STQ* pTq, int64_t version, char* msg, int32_t msgLen); -int32_t tqProcessStreamTaskCheckReq(STQ* pTq, SRpcMsg* pMsg); -int32_t tqProcessStreamTaskCheckRsp(STQ* pTq, SRpcMsg* pMsg); +int32_t tqProcessTaskCheckReq(STQ* pTq, SRpcMsg* pMsg); +int32_t tqProcessTaskCheckRsp(STQ* pTq, SRpcMsg* pMsg); int32_t tqProcessTaskRunReq(STQ* pTq, SRpcMsg* pMsg); int32_t tqProcessTaskDispatchReq(STQ* pTq, SRpcMsg* pMsg, bool exec); int32_t tqProcessTaskDispatchRsp(STQ* pTq, SRpcMsg* pMsg); diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 7f0842736e..3e060b4a38 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -873,7 +873,7 @@ int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t ver) { return 0; } -int32_t tqProcessStreamTaskCheckReq(STQ* pTq, SRpcMsg* pMsg) { +int32_t tqProcessTaskCheckReq(STQ* pTq, SRpcMsg* pMsg) { char* msgStr = pMsg->pCont; char* msgBody = POINTER_SHIFT(msgStr, sizeof(SMsgHead)); int32_t msgLen = pMsg->contLen - sizeof(SMsgHead); @@ -899,7 +899,7 @@ int32_t tqProcessStreamTaskCheckReq(STQ* pTq, SRpcMsg* pMsg) { }; // only the leader node handle the check request - if (!pMeta->leader) { + if (pMeta->role == NODE_ROLE_FOLLOWER) { tqError("s-task:0x%x invalid check msg from upstream:0x%x(vgId:%d), vgId:%d is follower, not handle check status msg", taskId, req.upstreamTaskId, req.upstreamNodeId, pMeta->vgId); rsp.status = TASK_DOWNSTREAM_NOT_LEADER; @@ -923,7 +923,7 @@ int32_t tqProcessStreamTaskCheckReq(STQ* pTq, SRpcMsg* pMsg) { return streamSendCheckRsp(pMeta, &req, &rsp, &pMsg->info, taskId); } -int32_t tqProcessStreamTaskCheckRsp(STQ* pTq, SRpcMsg* pMsg) { +int32_t tqProcessTaskCheckRsp(STQ* pTq, SRpcMsg* pMsg) { char* pReq = POINTER_SHIFT(pMsg->pCont, sizeof(SMsgHead)); int32_t len = pMsg->contLen - sizeof(SMsgHead); int32_t vgId = pTq->pStreamMeta->vgId; @@ -1727,7 +1727,7 @@ int32_t tqProcessStreamCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg* pRs } // downstream task has complete the stream task checkpoint procedure, let's start the handle the rsp by execute task -int32_t tqProcessStreamTaskCheckpointReadyMsg(STQ* pTq, SRpcMsg* pMsg) { +int32_t tqProcessTaskCheckpointReadyMsg(STQ* pTq, SRpcMsg* pMsg) { int32_t vgId = TD_VID(pTq->pVnode); SStreamMeta* pMeta = pTq->pStreamMeta; char* msg = POINTER_SHIFT(pMsg->pCont, sizeof(SMsgHead)); @@ -1840,8 +1840,10 @@ int32_t tqProcessTaskUpdateReq(STQ* pTq, SRpcMsg* pMsg) { // possibly only handle the stream task. int32_t numOfTasks = streamMetaGetNumOfTasks(pMeta); int32_t updateTasks = taosHashGetSize(pMeta->pUpdateTaskSet); + + pMeta->startInfo.startedAfterNodeUpdate = 1; + if (updateTasks < numOfTasks) { - pMeta->startInfo.startedAfterNodeUpdate = 1; tqDebug("vgId:%d closed tasks:%d, unclosed:%d, all tasks will be started when nodeEp update completed", vgId, updateTasks, (numOfTasks - updateTasks)); taosWUnLockLatch(&pMeta->lock); diff --git a/source/dnode/vnode/src/tq/tqSink.c b/source/dnode/vnode/src/tq/tqSink.c index 9158702284..a5958197bd 100644 --- a/source/dnode/vnode/src/tq/tqSink.c +++ b/source/dnode/vnode/src/tq/tqSink.c @@ -24,23 +24,22 @@ typedef struct STableSinkInfo { tstr name; } STableSinkInfo; +static int32_t tsAscendingSortFn(const void* p1, const void* p2); static int32_t setDstTableDataUid(SVnode* pVnode, SStreamTask* pTask, SSDataBlock* pDataBlock, char* stbFullName, SSubmitTbData* pTableData); static int32_t setDstTableDataPayload(SStreamTask* pTask, int32_t blockIndex, SSDataBlock* pDataBlock, SSubmitTbData* pTableData); static int32_t doBuildAndSendDeleteMsg(SVnode* pVnode, char* stbFullName, SSDataBlock* pDataBlock, SStreamTask* pTask, int64_t suid); -static int32_t tqBuildSubmitReq(SSubmitReq2* pSubmitReq, int32_t vgId, void** pMsg, int32_t* msgLen); -static int32_t tsAscendingSortFn(const void* p1, const void* p2); +static int32_t doBuildAndSendSubmitMsg(SVnode* pVnode, SStreamTask* pTask, SSubmitReq2* pReq, int32_t numOfBlocks); +static int32_t buildSubmitMsgImpl(SSubmitReq2* pSubmitReq, int32_t vgId, void** pMsg, int32_t* msgLen); static int32_t doConvertRows(SSubmitTbData* pTableData, STSchema* pTSchema, SSDataBlock* pDataBlock, const char* id); static int32_t doWaitForDstTableCreated(SVnode* pVnode, SStreamTask* pTask, STableSinkInfo* pTableSinkInfo, const char* dstTableName, int64_t* uid); static int32_t doPutIntoCache(SSHashObj* pSinkTableMap, STableSinkInfo* pTableSinkInfo, uint64_t groupId, const char* id); -static SVCreateTbReq* buildAutoCreateTableReq(char* stbFullName, int64_t suid, int32_t numOfCols, - SSDataBlock* pDataBlock); -static bool isValidDstChildTable(SMetaReader* pReader, int32_t vgId, const char* ctbName, int64_t suid); -static int32_t doMergeExistedRows(SSubmitTbData* pExisted, const SSubmitTbData* pNew, const char* id); -static int32_t doBuildAndSendSubmitMsg(SVnode* pVnode, SStreamTask* pTask, SSubmitReq2* pReq, int32_t numOfBlocks); +static int32_t doMergeExistedRows(SSubmitTbData* pExisted, const SSubmitTbData* pNew, const char* id); +static bool isValidDstChildTable(SMetaReader* pReader, int32_t vgId, const char* ctbName, int64_t suid); +static SVCreateTbReq* buildAutoCreateTableReq(char* stbFullName, int64_t suid, int32_t numOfCols, SSDataBlock* pDataBlock); int32_t tqBuildDeleteReq(const char* stbFullName, const SSDataBlock* pDataBlock, SBatchDeleteReq* deleteReq, const char* pIdStr) { @@ -255,7 +254,7 @@ int32_t doBuildAndSendSubmitMsg(SVnode* pVnode, SStreamTask* pTask, SSubmitReq2* void* pBuf = NULL; int32_t numOfFinalBlocks = taosArrayGetSize(pReq->aSubmitTbData); - int32_t code = tqBuildSubmitReq(pReq, vgId, &pBuf, &len); + int32_t code = buildSubmitMsgImpl(pReq, vgId, &pBuf, &len); if (code != TSDB_CODE_SUCCESS) { tqError("s-task:%s build submit msg failed, vgId:%d, code:%s", id, vgId, tstrerror(code)); return code; @@ -274,7 +273,7 @@ int32_t doBuildAndSendSubmitMsg(SVnode* pVnode, SStreamTask* pTask, SSubmitReq2* pRec->numOfSubmit += 1; if ((pRec->numOfSubmit % 5000) == 0) { - double el = (taosGetTimestampMs() - pTask->execInfo.start) / 1000.0; + double el = (taosGetTimestampMs() - pTask->execInfo.start) / 1000.0; tqInfo("s-task:%s vgId:%d write %" PRId64 " blocks (%" PRId64 " rows) in %" PRId64 " submit into dst table, %.2fMiB duration:%.2f Sec.", pTask->id.idStr, vgId, pRec->numOfBlocks, pRec->numOfRows, pRec->numOfSubmit, SIZE_IN_MiB(pRec->bytes), el); @@ -462,7 +461,7 @@ int32_t doPutIntoCache(SSHashObj* pSinkTableMap, STableSinkInfo* pTableSinkInfo, return code; } -int32_t tqBuildSubmitReq(SSubmitReq2* pSubmitReq, int32_t vgId, void** pMsg, int32_t* msgLen) { +int32_t buildSubmitMsgImpl(SSubmitReq2* pSubmitReq, int32_t vgId, void** pMsg, int32_t* msgLen) { int32_t code = 0; void* pBuf = NULL; *msgLen = 0; diff --git a/source/dnode/vnode/src/tq/tqUtil.c b/source/dnode/vnode/src/tq/tqUtil.c index b87783cfd0..46228a46a2 100644 --- a/source/dnode/vnode/src/tq/tqUtil.c +++ b/source/dnode/vnode/src/tq/tqUtil.c @@ -42,7 +42,7 @@ void tqUpdateNodeStage(STQ* pTq, bool isLeader) { int64_t stage = pMeta->stage; pMeta->stage = state.term; - pMeta->leader = isLeader; + pMeta->role = (isLeader)? NODE_ROLE_LEADER:NODE_ROLE_FOLLOWER; if (isLeader) { tqInfo("vgId:%d update meta stage:%" PRId64 ", prev:%" PRId64 " leader:%d, start to send Hb", pMeta->vgId, state.term, stage, isLeader); diff --git a/source/dnode/vnode/src/vnd/vnodeSvr.c b/source/dnode/vnode/src/vnd/vnodeSvr.c index 1e61a1f9dd..97f484849c 100644 --- a/source/dnode/vnode/src/vnd/vnodeSvr.c +++ b/source/dnode/vnode/src/vnd/vnodeSvr.c @@ -754,9 +754,9 @@ int32_t vnodeProcessStreamMsg(SVnode *pVnode, SRpcMsg *pMsg, SQueueInfo *pInfo) case TDMT_STREAM_TASK_DISPATCH_RSP: return tqProcessTaskDispatchRsp(pVnode->pTq, pMsg); case TDMT_VND_STREAM_TASK_CHECK: - return tqProcessStreamTaskCheckReq(pVnode->pTq, pMsg); + return tqProcessTaskCheckReq(pVnode->pTq, pMsg); case TDMT_VND_STREAM_TASK_CHECK_RSP: - return tqProcessStreamTaskCheckRsp(pVnode->pTq, pMsg); + return tqProcessTaskCheckRsp(pVnode->pTq, pMsg); case TDMT_STREAM_RETRIEVE: return tqProcessTaskRetrieveReq(pVnode->pTq, pMsg); case TDMT_STREAM_RETRIEVE_RSP: @@ -768,7 +768,7 @@ int32_t vnodeProcessStreamMsg(SVnode *pVnode, SRpcMsg *pMsg, SQueueInfo *pInfo) case TDMT_VND_STREAM_SCAN_HISTORY_FINISH_RSP: return tqProcessTaskScanHistoryFinishRsp(pVnode->pTq, pMsg); case TDMT_STREAM_TASK_CHECKPOINT_READY: - return tqProcessStreamTaskCheckpointReadyMsg(pVnode->pTq, pMsg); + return tqProcessTaskCheckpointReadyMsg(pVnode->pTq, pMsg); default: vError("unknown msg type:%d in stream queue", pMsg->msgType); return TSDB_CODE_APP_ERROR; diff --git a/source/libs/stream/src/stream.c b/source/libs/stream/src/stream.c index dbcbfc0a94..2abd3bac05 100644 --- a/source/libs/stream/src/stream.c +++ b/source/libs/stream/src/stream.c @@ -224,7 +224,7 @@ int32_t streamProcessDispatchMsg(SStreamTask* pTask, SStreamDispatchReq* pReq, S SStreamChildEpInfo* pInfo = streamTaskGetUpstreamTaskEpInfo(pTask, pReq->upstreamTaskId); ASSERT(pInfo != NULL); - if (!pTask->pMeta->leader) { + if (pTask->pMeta->role == NODE_ROLE_FOLLOWER) { stError("s-task:%s task on follower received dispatch msgs, dispatch msg rejected", id); status = TASK_INPUT_STATUS__REFUSED; } else { diff --git a/source/libs/stream/src/streamDispatch.c b/source/libs/stream/src/streamDispatch.c index aa32fb6493..bd5753cac3 100644 --- a/source/libs/stream/src/streamDispatch.c +++ b/source/libs/stream/src/streamDispatch.c @@ -1021,18 +1021,19 @@ int32_t streamAddEndScanHistoryMsg(SStreamTask* pTask, SRpcHandleInfo* pRpcInfo, int32_t streamNotifyUpstreamContinue(SStreamTask* pTask) { ASSERT(pTask->info.taskLevel == TASK_LEVEL__AGG || pTask->info.taskLevel == TASK_LEVEL__SINK); + const char* id = pTask->id.idStr; + int32_t level = pTask->info.taskLevel; + int32_t num = taosArrayGetSize(pTask->pRspMsgList); for (int32_t i = 0; i < num; ++i) { SStreamContinueExecInfo* pInfo = taosArrayGet(pTask->pRspMsgList, i); tmsgSendRsp(&pInfo->msg); - stDebug("s-task:%s level:%d notify upstream:0x%x to continue process data in WAL", pTask->id.idStr, pTask->info.taskLevel, - pInfo->taskId); + stDebug("s-task:%s level:%d notify upstream:0x%x continuing scan data in WAL", id, level, pInfo->taskId); } taosArrayClear(pTask->pRspMsgList); - stDebug("s-task:%s level:%d continue process msg sent to all %d upstreams", pTask->id.idStr, pTask->info.taskLevel, - num); + stDebug("s-task:%s level:%d continue process msg sent to all %d upstreams", id, level, num); return 0; } @@ -1063,7 +1064,7 @@ int32_t streamProcessDispatchRsp(SStreamTask* pTask, SStreamDispatchRsp* pRsp, i int32_t vgId = pTask->pMeta->vgId; int32_t msgId = pTask->execInfo.dispatch; - if ((!pTask->pMeta->leader) || (pTask->status.downstreamReady != 1)) { + if ((pTask->pMeta->role == NODE_ROLE_FOLLOWER) || (pTask->status.downstreamReady != 1)) { stError("s-task:%s vgId:%d is follower or task just re-launched, not handle the dispatch rsp, discard it", id, vgId); return TSDB_CODE_STREAM_TASK_NOT_EXIST; } @@ -1160,10 +1161,9 @@ int32_t streamProcessDispatchRsp(SStreamTask* pTask, SStreamDispatchRsp* pRsp, i // now ready for next data output atomic_store_8(&pTask->outputInfo.status, TASK_OUTPUT_STATUS__NORMAL); - return TSDB_CODE_SUCCESS; + } else { + handleDispatchSuccessRsp(pTask, pRsp->downstreamTaskId); } - - handleDispatchSuccessRsp(pTask, pRsp->downstreamTaskId); } } diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 5e25d911b0..092566fd84 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -196,7 +196,7 @@ SStreamMeta* streamMetaOpen(const char* path, void* ahandle, FTaskExpand expandF pMeta->chkpId = streamGetLatestCheckpointId(pMeta); pMeta->streamBackend = streamBackendInit(pMeta->path, pMeta->chkpId); while (pMeta->streamBackend == NULL) { - taosMsleep(2 * 1000); + taosMsleep(500); pMeta->streamBackend = streamBackendInit(pMeta->path, pMeta->chkpId); if (pMeta->streamBackend == NULL) { stError("vgId:%d failed to init stream backend", pMeta->vgId); @@ -205,6 +205,7 @@ SStreamMeta* streamMetaOpen(const char* path, void* ahandle, FTaskExpand expandF } pMeta->streamBackendRid = taosAddRef(streamBackendId, pMeta->streamBackend); + pMeta->role = NODE_ROLE_UNINIT; code = streamBackendLoadCheckpointInfo(pMeta); taosInitRWLatch(&pMeta->lock); @@ -237,6 +238,7 @@ int32_t streamMetaReopen(SStreamMeta* pMeta) { pMeta->streamBackendRid = -1; pMeta->streamBackend = NULL; + pMeta->role = NODE_ROLE_UNINIT; char* defaultPath = taosMemoryCalloc(1, strlen(pMeta->path) + 128); sprintf(defaultPath, "%s%s%s", pMeta->path, TD_DIRSEP, "state"); @@ -262,14 +264,14 @@ int32_t streamMetaReopen(SStreamMeta* pMeta) { pMeta->streamBackend = streamBackendInit(pMeta->path, pMeta->chkpId); while (pMeta->streamBackend == NULL) { - taosMsleep(2 * 1000); + taosMsleep(500); pMeta->streamBackend = streamBackendInit(pMeta->path, pMeta->chkpId); if (pMeta->streamBackend == NULL) { stError("vgId:%d failed to init stream backend", pMeta->vgId); stInfo("vgId:%d retry to init stream backend", pMeta->vgId); - // return -1; } } + pMeta->streamBackendRid = taosAddRef(streamBackendId, pMeta->streamBackend); streamBackendLoadCheckpointInfo(pMeta); @@ -346,6 +348,7 @@ void streamMetaCloseImpl(void* arg) { taosMemoryFree(pMeta->path); taosThreadMutexDestroy(&pMeta->backendMutex); + pMeta->role = NODE_ROLE_UNINIT; taosMemoryFree(pMeta); stDebug("end to close stream meta"); } @@ -829,7 +832,7 @@ void metaHbToMnode(void* param, void* tmrId) { } // not leader not send msg - if (!pMeta->leader) { + if (pMeta->role == NODE_ROLE_FOLLOWER) { stInfo("vgId:%d follower not send hb to mnode", pMeta->vgId); taosReleaseRef(streamMetaId, rid); pMeta->pHbInfo->hbStart = 0; @@ -847,7 +850,7 @@ void metaHbToMnode(void* param, void* tmrId) { return; } - stDebug("vgId:%d build stream task hb, leader:%d", pMeta->vgId, pMeta->leader); + stDebug("vgId:%d build stream task hb, leader:%d", pMeta->vgId, (pMeta->role == NODE_ROLE_LEADER)); SStreamHbMsg hbMsg = {0}; taosRLockLatch(&pMeta->lock); @@ -954,7 +957,7 @@ void streamMetaNotifyClose(SStreamMeta* pMeta) { int32_t vgId = pMeta->vgId; stDebug("vgId:%d notify all stream tasks that the vnode is closing. isLeader:%d startHb%" PRId64 ", totalHb:%d", vgId, - pMeta->leader, pMeta->pHbInfo->hbStart, pMeta->pHbInfo->hbCount); + (pMeta->role == NODE_ROLE_LEADER), pMeta->pHbInfo->hbStart, pMeta->pHbInfo->hbCount); taosWLockLatch(&pMeta->lock); @@ -973,7 +976,7 @@ void streamMetaNotifyClose(SStreamMeta* pMeta) { taosWUnLockLatch(&pMeta->lock); // wait for the stream meta hb function stopping - if (pMeta->leader) { + if (pMeta->role == NODE_ROLE_LEADER) { pMeta->pHbInfo->stopFlag = STREAM_META_WILL_STOP; while (pMeta->pHbInfo->stopFlag != STREAM_META_OK_TO_STOP) { taosMsleep(100); @@ -1002,5 +1005,5 @@ void streamMetaStartHb(SStreamMeta* pMeta) { void streamMetaInitForSnode(SStreamMeta* pMeta) { pMeta->stage = 0; - pMeta->leader = true; + pMeta->role = NODE_ROLE_LEADER; } \ No newline at end of file diff --git a/source/libs/stream/src/streamRecover.c b/source/libs/stream/src/streamRecover.c index d910c7d785..3ca81ea90b 100644 --- a/source/libs/stream/src/streamRecover.c +++ b/source/libs/stream/src/streamRecover.c @@ -1000,36 +1000,6 @@ void streamTaskEnablePause(SStreamTask* pTask) { pTask->status.pauseAllowed = 1; } -// fix: this function should be removed, it may cause deadlock. -//void streamTaskHalt(SStreamTask* pTask) { -// int8_t status = pTask->status.taskStatus; -// if (status == TASK_STATUS__DROPPING || status == TASK_STATUS__STOP) { -// return; -// } -// -// if (status == TASK_STATUS__HALT) { -// return; -// } -// -// // wait for checkpoint completed -// while(pTask->status.taskStatus == TASK_STATUS__CK) { -// qDebug("s-task:%s status:%s during generating checkpoint, wait for 1sec and retry set status:halt", pTask->id.idStr, -// streamGetTaskStatusStr(TASK_STATUS__CK)); -// taosMsleep(1000); -// } -// -// // upgrade to halt status -// if (status == TASK_STATUS__PAUSE) { -// stDebug("s-task:%s upgrade status to %s from %s", pTask->id.idStr, streamGetTaskStatusStr(TASK_STATUS__HALT), -// streamGetTaskStatusStr(TASK_STATUS__PAUSE)); -// } else { -// stDebug("s-task:%s halt task", pTask->id.idStr); -// } -// -// pTask->status.keepTaskStatus = status; -// pTask->status.taskStatus = TASK_STATUS__HALT; -//} - void streamTaskResumeFromHalt(SStreamTask* pTask) { const char* id = pTask->id.idStr; int8_t status = pTask->status.taskStatus; diff --git a/source/libs/stream/src/streamTask.c b/source/libs/stream/src/streamTask.c index 91e91fe3d8..e5088e9c69 100644 --- a/source/libs/stream/src/streamTask.c +++ b/source/libs/stream/src/streamTask.c @@ -554,26 +554,26 @@ void streamTaskUpdateDownstreamInfo(SStreamTask* pTask, int32_t nodeId, const SE } int32_t streamTaskStop(SStreamTask* pTask) { - SStreamMeta* pMeta = pTask->pMeta; + int32_t vgId = pTask->pMeta->vgId; int64_t st = taosGetTimestampMs(); const char* id = pTask->id.idStr; taosThreadMutexLock(&pTask->lock); if (pTask->status.taskStatus == TASK_STATUS__CK) { - stDebug("s-task:%s in checkpoint will be discarded since task is stopped", pTask->id.idStr); + stDebug("s-task:%s in checkpoint will be discarded since task is stopped", id); } pTask->status.taskStatus = TASK_STATUS__STOP; taosThreadMutexUnlock(&pTask->lock); qKillTask(pTask->exec.pExecutor, TSDB_CODE_SUCCESS); - while (/*pTask->status.schedStatus != TASK_SCHED_STATUS__INACTIVE */ !streamTaskIsIdle(pTask)) { - stDebug("s-task:%s level:%d wait for task to be idle, check again in 100ms", id, pTask->info.taskLevel); + stDebug("s-task:%s level:%d wait for task to be idle and then close, check again in 100ms", id, + pTask->info.taskLevel); taosMsleep(100); } int64_t el = taosGetTimestampMs() - st; - stDebug("vgId:%d s-task:%s is closed in %" PRId64 " ms", pMeta->vgId, pTask->id.idStr, el); + stDebug("vgId:%d s-task:%s is closed in %" PRId64 " ms", vgId, id, el); return 0; } From a248f3d088fa801ab7fcd43a6cbd702090251f84 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 26 Sep 2023 14:11:45 +0800 Subject: [PATCH 097/158] refactor: do some internal refactor. --- include/libs/stream/tstream.h | 10 ++++++---- source/dnode/vnode/src/tq/tqSink.c | 5 +++-- source/libs/stream/src/streamExec.c | 2 +- source/libs/stream/src/streamQueue.c | 6 ++++++ 4 files changed, 16 insertions(+), 7 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index a98ad5a4c2..29a75083ef 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -326,18 +326,21 @@ typedef struct SSinkRecorder { int64_t numOfSubmit; int64_t numOfBlocks; int64_t numOfRows; - int64_t bytes; + int64_t dataSize; } SSinkRecorder; typedef struct STaskExecStatisInfo { int64_t created; int64_t init; + int64_t start; int64_t step1Start; int64_t step2Start; - int64_t start; int32_t updateCount; - int32_t dispatch; int64_t latestUpdateTs; + int32_t processDataBlocks; + int64_t processDataSize; + int32_t dispatch; + int64_t dispatchDataSize; int32_t checkpoint; SSinkRecorder sink; } STaskExecStatisInfo; @@ -415,7 +418,6 @@ typedef struct SStreamMeta { FTaskExpand* expandFunc; int32_t vgId; int64_t stage; -// bool leader; int32_t role; STaskStartInfo startInfo; SRWLatch lock; diff --git a/source/dnode/vnode/src/tq/tqSink.c b/source/dnode/vnode/src/tq/tqSink.c index a5958197bd..8009eccb1b 100644 --- a/source/dnode/vnode/src/tq/tqSink.c +++ b/source/dnode/vnode/src/tq/tqSink.c @@ -272,11 +272,12 @@ int32_t doBuildAndSendSubmitMsg(SVnode* pVnode, SStreamTask* pTask, SSubmitReq2* SSinkRecorder* pRec = &pTask->execInfo.sink; pRec->numOfSubmit += 1; - if ((pRec->numOfSubmit % 5000) == 0) { + if ((pRec->numOfSubmit % 1000) == 0) { double el = (taosGetTimestampMs() - pTask->execInfo.start) / 1000.0; tqInfo("s-task:%s vgId:%d write %" PRId64 " blocks (%" PRId64 " rows) in %" PRId64 " submit into dst table, %.2fMiB duration:%.2f Sec.", - pTask->id.idStr, vgId, pRec->numOfBlocks, pRec->numOfRows, pRec->numOfSubmit, SIZE_IN_MiB(pRec->bytes), el); + pTask->id.idStr, vgId, pRec->numOfBlocks, pRec->numOfRows, pRec->numOfSubmit, SIZE_IN_MiB(pRec->dataSize), + el); } return TSDB_CODE_SUCCESS; diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index a87bb00972..e46b094f60 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -556,7 +556,7 @@ int32_t streamExecForAll(SStreamTask* pTask) { // here only handle the data block sink operation if (type == STREAM_INPUT__DATA_BLOCK) { int32_t blockSize = streamQueueItemGetSize(pInput); - pTask->execInfo.sink.bytes += blockSize; + pTask->execInfo.sink.dataSize += blockSize; stDebug("s-task:%s sink task start to sink %d blocks, size:%.2fKiB", id, numOfBlocks, SIZE_IN_KiB(blockSize)); doOutputResultBlockImpl(pTask, (SStreamDataBlock*)pInput); diff --git a/source/libs/stream/src/streamQueue.c b/source/libs/stream/src/streamQueue.c index abf10487de..61453cb54e 100644 --- a/source/libs/stream/src/streamQueue.c +++ b/source/libs/stream/src/streamQueue.c @@ -154,6 +154,12 @@ int32_t streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueItem** pInpu int32_t taskLevel = pTask->info.taskLevel; *numOfBlocks = 0; + // no available token in bucket for sink task, let's wait + if (taskLevel == TASK_LEVEL__SINK && (!streamTaskHasAvailableToken(pTask->pTokenBucket))) { + stDebug("s-task:%s no available token in bucket for sink data, wait", id); + return TSDB_CODE_SUCCESS; + } + while (1) { if (streamTaskShouldPause(&pTask->status) || streamTaskShouldStop(&pTask->status)) { stDebug("s-task:%s task should pause, extract input blocks:%d", pTask->id.idStr, *numOfBlocks); From 7f8da725bc061461593a723cb98b227e2d844164 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 26 Sep 2023 23:29:54 +0800 Subject: [PATCH 098/158] enh(stream): add quota limitation for sink task. --- source/libs/stream/inc/streamInt.h | 15 ++-- source/libs/stream/src/streamExec.c | 5 +- source/libs/stream/src/streamQueue.c | 115 ++++++++++++++++++++------- source/libs/stream/src/streamTask.c | 4 +- 4 files changed, 102 insertions(+), 37 deletions(-) diff --git a/source/libs/stream/inc/streamInt.h b/source/libs/stream/inc/streamInt.h index dbe868b54f..dc5cf497db 100644 --- a/source/libs/stream/inc/streamInt.h +++ b/source/libs/stream/inc/streamInt.h @@ -49,10 +49,13 @@ typedef struct SStreamContinueExecInfo { } SStreamContinueExecInfo; struct STokenBucket { - int32_t capacity; // total capacity - int64_t fillTimestamp;// fill timestamp - int32_t numOfToken; // total available tokens - int32_t rate; // number of token per second + int32_t numCapacity; // total capacity, available token per second + int32_t numOfToken; // total available tokens + int32_t numRate; // number of token per second + double bytesCapacity; // available capacity for maximum input size, KiloBytes per Second + double bytesRemain; // not consumed bytes per second + double bytesRate; // number of token per second + int64_t fillTimestamp; // fill timestamp }; struct STaskTimer { @@ -89,7 +92,7 @@ int32_t streamTaskSendCheckpointReadyMsg(SStreamTask* pTask); int32_t streamTaskSendCheckpointSourceRsp(SStreamTask* pTask); int32_t streamTaskGetNumOfDownstream(const SStreamTask* pTask); -int32_t streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueItem** pInput, int32_t* numOfBlocks); +int32_t streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueItem** pInput, int32_t* numOfBlocks, int32_t* blockSize); int32_t streamQueueGetNumOfItemsInQueue(const SStreamQueue* pQueue); int32_t streamQueueItemGetSize(const SStreamQueueItem* pItem); void streamQueueItemIncSize(const SStreamQueueItem* pItem, int32_t size); @@ -103,7 +106,7 @@ int32_t streamNotifyUpstreamContinue(SStreamTask* pTask); int32_t streamTaskFillHistoryFinished(SStreamTask* pTask); int32_t streamTransferStateToStreamTask(SStreamTask* pTask); -int32_t streamTaskInitTokenBucket(STokenBucket* pBucket, int32_t cap, int32_t rate); +int32_t streamTaskInitTokenBucket(STokenBucket* pBucket, int32_t numCap, int32_t numRate, int32_t bytesRate); SStreamQueue* streamQueueOpen(int64_t cap); void streamQueueClose(SStreamQueue* pQueue, int32_t taskId); diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index e46b094f60..a6f7ac27d4 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -524,6 +524,7 @@ int32_t streamExecForAll(SStreamTask* pTask) { stDebug("s-task:%s start to extract data block from inputQ", id); while (1) { + int32_t blockSize = 0; int32_t numOfBlocks = 0; SStreamQueueItem* pInput = NULL; if (streamTaskShouldStop(&pTask->status)) { @@ -531,7 +532,7 @@ int32_t streamExecForAll(SStreamTask* pTask) { break; } - /*int32_t code = */ streamTaskGetDataFromInputQ(pTask, &pInput, &numOfBlocks); + /*int32_t code = */ streamTaskGetDataFromInputQ(pTask, &pInput, &numOfBlocks, &blockSize); if (pInput == NULL) { ASSERT(numOfBlocks == 0); return 0; @@ -555,9 +556,7 @@ int32_t streamExecForAll(SStreamTask* pTask) { // here only handle the data block sink operation if (type == STREAM_INPUT__DATA_BLOCK) { - int32_t blockSize = streamQueueItemGetSize(pInput); pTask->execInfo.sink.dataSize += blockSize; - stDebug("s-task:%s sink task start to sink %d blocks, size:%.2fKiB", id, numOfBlocks, SIZE_IN_KiB(blockSize)); doOutputResultBlockImpl(pTask, (SStreamDataBlock*)pInput); continue; diff --git a/source/libs/stream/src/streamQueue.c b/source/libs/stream/src/streamQueue.c index 61453cb54e..708b0572a4 100644 --- a/source/libs/stream/src/streamQueue.c +++ b/source/libs/stream/src/streamQueue.c @@ -20,6 +20,7 @@ #define STREAM_TASK_QUEUE_CAPACITY 20480 #define STREAM_TASK_INPUT_QUEUE_CAPACITY_IN_SIZE (30) #define STREAM_TASK_OUTPUT_QUEUE_CAPACITY_IN_SIZE (50) +#define MAX_SMOOTH_BURST_RATIO 10 // 20 sec // todo refactor: // read data from input queue @@ -30,7 +31,9 @@ typedef struct SQueueReader { int32_t waitDuration; // maximum wait time to format several block into a batch to process, unit: ms } SQueueReader; -static bool streamTaskHasAvailableToken(STokenBucket* pBucket); +static bool streamTaskExtractAvailableToken(STokenBucket* pBucket); +static void streamTaskPutbackToken(STokenBucket* pBucket); +static void streamTaskConsumeQuota(STokenBucket* pBucket, int32_t bytes); static void streamQueueCleanup(SStreamQueue* pQueue) { void* qItem = NULL; @@ -147,15 +150,19 @@ const char* streamQueueItemGetTypeStr(int32_t type) { } } -int32_t streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueItem** pInput, int32_t* numOfBlocks) { +int32_t streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueItem** pInput, int32_t* numOfBlocks, + int32_t* blockSize) { int32_t retryTimes = 0; int32_t MAX_RETRY_TIMES = 5; const char* id = pTask->id.idStr; int32_t taskLevel = pTask->info.taskLevel; - *numOfBlocks = 0; - // no available token in bucket for sink task, let's wait - if (taskLevel == TASK_LEVEL__SINK && (!streamTaskHasAvailableToken(pTask->pTokenBucket))) { + *pInput = NULL; + *numOfBlocks = 0; + *blockSize = 0; + + // no available token in bucket for sink task, let's wait for a little bit + if (taskLevel == TASK_LEVEL__SINK && (!streamTaskExtractAvailableToken(pTask->pTokenBucket))) { stDebug("s-task:%s no available token in bucket for sink data, wait", id); return TSDB_CODE_SUCCESS; } @@ -172,6 +179,17 @@ int32_t streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueItem** pInpu taosMsleep(10); continue; } + + // restore the token to bucket + if (*numOfBlocks > 0) { + *blockSize = streamQueueItemGetSize(*pInput); + if (taskLevel == TASK_LEVEL__SINK) { + streamTaskConsumeQuota(pTask->pTokenBucket, *blockSize); + } + } else { + streamTaskPutbackToken(pTask->pTokenBucket); + } + return TSDB_CODE_SUCCESS; } @@ -179,17 +197,24 @@ int32_t streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueItem** pInpu int8_t type = qItem->type; if (type == STREAM_INPUT__CHECKPOINT || type == STREAM_INPUT__CHECKPOINT_TRIGGER || type == STREAM_INPUT__TRANS_STATE) { - const char* p = streamQueueItemGetTypeStr(qItem->type); + const char* p = streamQueueItemGetTypeStr(type); if (*pInput == NULL) { stDebug("s-task:%s %s msg extracted, start to process immediately", id, p); + // restore the token to bucket in case of checkpoint/trans-state msg + streamTaskPutbackToken(pTask->pTokenBucket); + *blockSize = 0; *numOfBlocks = 1; *pInput = qItem; return TSDB_CODE_SUCCESS; - } else { - // previous existed blocks needs to be handle, before handle the checkpoint msg block + } else { // previous existed blocks needs to be handle, 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->pTokenBucket, *blockSize); + } + streamQueueProcessFail(pTask->inputInfo.queue); return TSDB_CODE_SUCCESS; } @@ -198,7 +223,7 @@ int32_t streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueItem** pInpu ASSERT((*numOfBlocks) == 0); *pInput = qItem; } else { - // todo we need to sort the data block, instead of just appending into the array list. + // merge current block failed, let's handle the already merged blocks. void* newRet = streamMergeQueueItem(*pInput, qItem); if (newRet == NULL) { if (terrno != 0) { @@ -206,6 +231,11 @@ int32_t streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueItem** pInpu tstrerror(terrno)); } + *blockSize = streamQueueItemGetSize(*pInput); + if (taskLevel == TASK_LEVEL__SINK) { + streamTaskConsumeQuota(pTask->pTokenBucket, *blockSize); + } + streamQueueProcessFail(pTask->inputInfo.queue); return TSDB_CODE_SUCCESS; } @@ -218,6 +248,12 @@ int32_t streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueItem** pInpu 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); + + *blockSize = streamQueueItemGetSize(*pInput); + if (taskLevel == TASK_LEVEL__SINK) { + streamTaskConsumeQuota(pTask->pTokenBucket, *blockSize); + } + return TSDB_CODE_SUCCESS; } } @@ -340,43 +376,68 @@ int32_t streamTaskPutDataIntoOutputQ(SStreamTask* pTask, SStreamDataBlock* pBloc return TSDB_CODE_SUCCESS; } -int32_t streamTaskInitTokenBucket(STokenBucket* pBucket, int32_t cap, int32_t rate) { - if (cap < 50 || rate < 50 || pBucket == NULL) { - stError("failed to init sink task bucket, cap:%d, rate:%d", cap, rate); +int32_t streamTaskInitTokenBucket(STokenBucket* pBucket, int32_t numCap, int32_t numRate, int32_t bytesRate) { + if (numCap < 10 || numRate < 10 || pBucket == NULL) { + stError("failed to init sink task bucket, cap:%d, rate:%d", numCap, numRate); return TSDB_CODE_INVALID_PARA; } - pBucket->capacity = cap; - pBucket->rate = rate; - pBucket->numOfToken = cap; + pBucket->numCapacity = numCap; + pBucket->numOfToken = numCap; + pBucket->numRate = numRate; + + pBucket->bytesRate = bytesRate; + pBucket->bytesCapacity = bytesRate * MAX_SMOOTH_BURST_RATIO; + pBucket->bytesRemain = pBucket->bytesCapacity; + pBucket->fillTimestamp = taosGetTimestampMs(); return TSDB_CODE_SUCCESS; } -static void fillBucket(STokenBucket* pBucket) { +static void fillTokenBucket(STokenBucket* pBucket) { int64_t now = taosGetTimestampMs(); int64_t delta = now - pBucket->fillTimestamp; ASSERT(pBucket->numOfToken >= 0); - int32_t inc = (delta / 1000.0) * pBucket->rate; - if (inc > 0) { - if ((pBucket->numOfToken + inc) < pBucket->capacity) { - pBucket->numOfToken += inc; + int32_t incNum = (delta / 1000.0) * pBucket->numRate; + if (incNum > 0) { + if ((pBucket->numOfToken + incNum) < pBucket->numCapacity) { + pBucket->numOfToken += incNum; } else { - pBucket->numOfToken = pBucket->capacity; + pBucket->numOfToken = pBucket->numCapacity; } - pBucket->fillTimestamp = now; - stDebug("new token available, current:%d, inc:%d ts:%"PRId64, pBucket->numOfToken, inc, now); + stDebug("new token available, current:%d, inc:%d ts:%" PRId64, pBucket->numOfToken, incNum, now); + } + + // increase the new available quota as time goes on + double incSize = (delta / 1000.0) * pBucket->bytesRate; + if (incSize > 0) { + pBucket->bytesRemain = MIN(pBucket->bytesRemain + incSize, pBucket->bytesCapacity); + stDebug("new bytes token available, current:%.2fKiB, inc:%.2fKiB ts:%" PRId64, pBucket->bytesRemain, incSize, now); } } -bool streamTaskHasAvailableToken(STokenBucket* pBucket) { - fillBucket(pBucket); +bool streamTaskExtractAvailableToken(STokenBucket* pBucket) { + fillTokenBucket(pBucket); + if (pBucket->numOfToken > 0) { - --pBucket->numOfToken; - return true; + if (pBucket->bytesRemain > 0) { + pBucket->numOfToken -= 1; + return true; + } else { // no available size quota now + return false; + } } else { return false; } +} + +void streamTaskPutbackToken(STokenBucket* pBucket) { + pBucket->numOfToken = MIN(pBucket->numOfToken + 1, pBucket->numCapacity); +} + +// size in KB +void streamTaskConsumeQuota(STokenBucket* pBucket, int32_t bytes) { + pBucket->bytesRemain -= SIZE_IN_MiB(bytes); } \ No newline at end of file diff --git a/source/libs/stream/src/streamTask.c b/source/libs/stream/src/streamTask.c index e5088e9c69..ea7e89cf1b 100644 --- a/source/libs/stream/src/streamTask.c +++ b/source/libs/stream/src/streamTask.c @@ -431,7 +431,9 @@ int32_t streamTaskInit(SStreamTask* pTask, SStreamMeta* pMeta, SMsgCb* pMsgCb, i return TSDB_CODE_OUT_OF_MEMORY; } - streamTaskInitTokenBucket(pTask->pTokenBucket, 50, 50); + // 2MiB per second for sink task + // 50 times sink operator per second + streamTaskInitTokenBucket(pTask->pTokenBucket, 50, 50, 2); TdThreadMutexAttr attr = {0}; int code = taosThreadMutexAttrInit(&attr); From 2c0e54f4f70f50c5cb83c6e3b08d27a59843d8c2 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 26 Sep 2023 23:42:02 +0800 Subject: [PATCH 099/158] fix(stream): update the retry interval. --- source/libs/stream/src/streamMeta.c | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 092566fd84..dcb82e47f8 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -196,11 +196,10 @@ SStreamMeta* streamMetaOpen(const char* path, void* ahandle, FTaskExpand expandF pMeta->chkpId = streamGetLatestCheckpointId(pMeta); pMeta->streamBackend = streamBackendInit(pMeta->path, pMeta->chkpId); while (pMeta->streamBackend == NULL) { - taosMsleep(500); + taosMsleep(100); pMeta->streamBackend = streamBackendInit(pMeta->path, pMeta->chkpId); if (pMeta->streamBackend == NULL) { - stError("vgId:%d failed to init stream backend", pMeta->vgId); - stInfo("vgId:%d retry to init stream backend", pMeta->vgId); + stInfo("vgId:%d failed to init stream backend, retry in 100ms", pMeta->vgId); } } pMeta->streamBackendRid = taosAddRef(streamBackendId, pMeta->streamBackend); @@ -264,11 +263,10 @@ int32_t streamMetaReopen(SStreamMeta* pMeta) { pMeta->streamBackend = streamBackendInit(pMeta->path, pMeta->chkpId); while (pMeta->streamBackend == NULL) { - taosMsleep(500); + taosMsleep(100); pMeta->streamBackend = streamBackendInit(pMeta->path, pMeta->chkpId); if (pMeta->streamBackend == NULL) { - stError("vgId:%d failed to init stream backend", pMeta->vgId); - stInfo("vgId:%d retry to init stream backend", pMeta->vgId); + stInfo("vgId:%d failed to init stream backend, retry in 100ms", pMeta->vgId); } } From ecb3b44026fdc588e851c792075cc2b88b060005 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 27 Sep 2023 00:30:38 +0800 Subject: [PATCH 100/158] fix(stream): limit sink quota. --- source/libs/stream/src/streamQueue.c | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/source/libs/stream/src/streamQueue.c b/source/libs/stream/src/streamQueue.c index 708b0572a4..dc0eb949da 100644 --- a/source/libs/stream/src/streamQueue.c +++ b/source/libs/stream/src/streamQueue.c @@ -20,7 +20,7 @@ #define STREAM_TASK_QUEUE_CAPACITY 20480 #define STREAM_TASK_INPUT_QUEUE_CAPACITY_IN_SIZE (30) #define STREAM_TASK_OUTPUT_QUEUE_CAPACITY_IN_SIZE (50) -#define MAX_SMOOTH_BURST_RATIO 10 // 20 sec +#define MAX_SMOOTH_BURST_RATIO 5 // 20 sec // todo refactor: // read data from input queue @@ -401,20 +401,20 @@ static void fillTokenBucket(STokenBucket* pBucket) { int32_t incNum = (delta / 1000.0) * pBucket->numRate; if (incNum > 0) { - if ((pBucket->numOfToken + incNum) < pBucket->numCapacity) { - pBucket->numOfToken += incNum; - } else { - pBucket->numOfToken = pBucket->numCapacity; - } + pBucket->numOfToken = MIN(pBucket->numOfToken + incNum, pBucket->numCapacity); pBucket->fillTimestamp = now; - stDebug("new token available, current:%d, inc:%d ts:%" PRId64, pBucket->numOfToken, incNum, now); } // increase the new available quota as time goes on double incSize = (delta / 1000.0) * pBucket->bytesRate; if (incSize > 0) { pBucket->bytesRemain = MIN(pBucket->bytesRemain + incSize, pBucket->bytesCapacity); - stDebug("new bytes token available, current:%.2fKiB, inc:%.2fKiB ts:%" PRId64, pBucket->bytesRemain, incSize, now); + } + + if (incNum > 0) { + stDebug("new token and capacity available, current token:%d inc:%d, current quota:%.2fMiB inc:%.2fMiB, ts:%" PRId64 + " wait for %.2f Sec", + pBucket->numOfToken, incNum, pBucket->bytesRemain, incSize, now, delta / 1000.0); } } From f2848de12a091b28f2f206ab451d87d9d36b8e5d Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 27 Sep 2023 09:36:20 +0800 Subject: [PATCH 101/158] fix(stream): reduce the counter when remove stream tasks. --- source/libs/stream/src/streamDispatch.c | 7 ++++--- source/libs/stream/src/streamExec.c | 4 ++-- source/libs/stream/src/streamMeta.c | 2 ++ 3 files changed, 8 insertions(+), 5 deletions(-) diff --git a/source/libs/stream/src/streamDispatch.c b/source/libs/stream/src/streamDispatch.c index bd5753cac3..ea8c7c71ac 100644 --- a/source/libs/stream/src/streamDispatch.c +++ b/source/libs/stream/src/streamDispatch.c @@ -1043,12 +1043,14 @@ static int32_t handleDispatchSuccessRsp(SStreamTask* pTask, int32_t downstreamId pTask->msgInfo.pData = NULL; int64_t el = taosGetTimestampMs() - pTask->msgInfo.startTs; - stDebug("s-task:%s downstream task:0x%x resume to normal from inputQ blocking, blocking time:%" PRId64 "ms", - pTask->id.idStr, downstreamId, el); // put data into inputQ of current task is also allowed if (pTask->inputInfo.status == TASK_INPUT_STATUS__BLOCKED) { pTask->inputInfo.status = TASK_INPUT_STATUS__NORMAL; + stDebug("s-task:%s downstream task:0x%x resume to normal from inputQ blocking, blocking time:%" PRId64 "ms", + pTask->id.idStr, downstreamId, el); + } else { + stDebug("s-task:%s dispatch completed, elapsed time:%"PRId64"ms", pTask->id.idStr, el); } // now ready for next data output @@ -1110,7 +1112,6 @@ int32_t streamProcessDispatchRsp(SStreamTask* pTask, SStreamDispatchRsp* pRsp, i int32_t leftRsp = 0; if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { - stDebug("s-task:%s waiting rsp:%d", id, pTask->shuffleDispatcher.waitingRspCnt); leftRsp = atomic_sub_fetch_32(&pTask->shuffleDispatcher.waitingRspCnt, 1); ASSERT(leftRsp >= 0); diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index a6f7ac27d4..20c1495d49 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -248,8 +248,8 @@ int32_t streamScanHistoryData(SStreamTask* pTask) { size += blockDataGetSize(output) + sizeof(SSDataBlock) + sizeof(SColumnInfoData) * blockDataGetNumOfCols(&block); if ((++numOfBlocks) >= outputBatchSize || size >= STREAM_RESULT_DUMP_SIZE_THRESHOLD) { - stDebug("s-task:%s scan exec numOfBlocks:%d, output num-limit:%d, size-limit:%d reached", pTask->id.idStr, numOfBlocks, - outputBatchSize, STREAM_RESULT_DUMP_SIZE_THRESHOLD); + stDebug("s-task:%s scan exec numOfBlocks:%d, size:%d output num-limit:%d, size-limit:%d reached", + pTask->id.idStr, numOfBlocks, size, outputBatchSize, STREAM_RESULT_DUMP_SIZE_THRESHOLD); break; } } diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index dcb82e47f8..d37ee9840d 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -550,6 +550,8 @@ int32_t streamMetaUnregisterTask(SStreamMeta* pMeta, int64_t streamId, int32_t t (*ppStreamTask)->historyTaskId.taskId = 0; (*ppStreamTask)->historyTaskId.streamId = 0; } + } else { + atomic_sub_fetch_32(&pMeta->numOfStreamTasks, 1); } taosHashRemove(pMeta->pTasksMap, &id, sizeof(id)); From 38164435f940e2d1aa8ab737cd453bb15e21dc8b Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 27 Sep 2023 09:46:15 +0800 Subject: [PATCH 102/158] refactor: do some internal refactor. --- source/libs/stream/src/streamExec.c | 12 +++++------- 1 file changed, 5 insertions(+), 7 deletions(-) diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index 20c1495d49..7759de9590 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -192,11 +192,9 @@ static int32_t streamTaskExecImpl(SStreamTask* pTask, SStreamQueueItem* pItem, i int32_t streamScanHistoryData(SStreamTask* pTask) { ASSERT(pTask->info.taskLevel == TASK_LEVEL__SOURCE); - int32_t size = 0; int32_t code = TSDB_CODE_SUCCESS; void* exec = pTask->exec.pExecutor; bool finished = false; - int32_t outputBatchSize = 100; qSetStreamOpOpen(exec); @@ -213,6 +211,7 @@ int32_t streamScanHistoryData(SStreamTask* pTask) { return -1; } + int32_t size = 0; int32_t numOfBlocks = 0; while (1) { if (streamTaskShouldStop(&pTask->status)) { @@ -247,9 +246,10 @@ int32_t streamScanHistoryData(SStreamTask* pTask) { size += blockDataGetSize(output) + sizeof(SSDataBlock) + sizeof(SColumnInfoData) * blockDataGetNumOfCols(&block); - if ((++numOfBlocks) >= outputBatchSize || size >= STREAM_RESULT_DUMP_SIZE_THRESHOLD) { - stDebug("s-task:%s scan exec numOfBlocks:%d, size:%d output num-limit:%d, size-limit:%d reached", - pTask->id.idStr, numOfBlocks, size, outputBatchSize, STREAM_RESULT_DUMP_SIZE_THRESHOLD); + if ((++numOfBlocks) >= STREAM_RESULT_DUMP_THRESHOLD || size >= STREAM_RESULT_DUMP_SIZE_THRESHOLD) { + stDebug("s-task:%s scan exec numOfBlocks:%d, size:%.2fKiB output num-limit:%d, size-limit:%.2fKiB reached", + pTask->id.idStr, numOfBlocks, SIZE_IN_KiB(size), STREAM_RESULT_DUMP_THRESHOLD, + SIZE_IN_KiB(STREAM_RESULT_DUMP_SIZE_THRESHOLD)); break; } } @@ -260,8 +260,6 @@ int32_t streamScanHistoryData(SStreamTask* pTask) { if (code != TSDB_CODE_SUCCESS) { return code; } - - size = 0; } else { taosArrayDestroy(pRes); } From dfe8641c0c2dd6b6683ae396798d6a2d2ac82df4 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 27 Sep 2023 14:00:48 +0800 Subject: [PATCH 103/158] refactor: optimize the fill-history task launch policy, and do some other internal refactor. --- include/libs/stream/tstream.h | 27 +-- source/dnode/mnode/impl/src/mndScheduler.c | 4 +- source/dnode/vnode/src/tq/tq.c | 16 +- source/libs/stream/inc/streamInt.h | 24 ++- source/libs/stream/src/streamDispatch.c | 21 +-- source/libs/stream/src/streamExec.c | 6 +- source/libs/stream/src/streamMeta.c | 14 +- source/libs/stream/src/streamRecover.c | 198 ++++++++++++++------- source/libs/stream/src/streamTask.c | 62 ++++--- 9 files changed, 235 insertions(+), 137 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 29a75083ef..61d74b5809 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -270,13 +270,13 @@ typedef struct SCheckpointInfo { } SCheckpointInfo; typedef struct SStreamStatus { - int8_t taskStatus; - int8_t downstreamReady; // downstream tasks are all ready now, if this flag is set - int8_t schedStatus; - int8_t keepTaskStatus; - bool appendTranstateBlock; // has append the transfer state data block already, todo: remove it - int8_t timerActive; // timer is active - int8_t pauseAllowed; // allowed task status to be set to be paused + int8_t taskStatus; + int8_t downstreamReady; // downstream tasks are all ready now, if this flag is set + int8_t schedStatus; + int8_t keepTaskStatus; + bool appendTranstateBlock; // has append the transfer state data block already, todo: remove it + int8_t pauseAllowed; // allowed task status to be set to be paused + int32_t timerActive; // timer is active } SStreamStatus; typedef struct SDataRange { @@ -304,6 +304,7 @@ typedef struct SDispatchMsgInfo { int32_t retryCount; // retry send data count int64_t startTs; // dispatch start time, record total elapsed time for dispatch SArray* pRetryList; // current dispatch successfully completed node of downstream + void* pTimer; // used to dispatch data after a given time duration } SDispatchMsgInfo; typedef struct STaskOutputInfo { @@ -345,7 +346,14 @@ typedef struct STaskExecStatisInfo { SSinkRecorder sink; } STaskExecStatisInfo; -typedef struct STaskTimer STaskTimer; +typedef struct SHistoryTaskInfo { + STaskId id; + void* pTimer; + int32_t tickCount; + int32_t retryTimes; + int32_t waitInterval; +} SHistoryTaskInfo; + typedef struct STokenBucket STokenBucket; typedef struct SMetaHbInfo SMetaHbInfo; @@ -361,7 +369,7 @@ struct SStreamTask { SCheckpointInfo chkInfo; STaskExec exec; SDataRange dataRange; - STaskId historyTaskId; + SHistoryTaskInfo hTaskInfo; STaskId streamTaskId; STaskExecStatisInfo execInfo; SArray* pReadyMsgList; // SArray @@ -378,7 +386,6 @@ struct SStreamTask { }; STokenBucket* pTokenBucket; - STaskTimer* pTimer; SMsgCb* pMsgCb; // msg handle SStreamState* pState; // state backend SArray* pRspMsgList; diff --git a/source/dnode/mnode/impl/src/mndScheduler.c b/source/dnode/mnode/impl/src/mndScheduler.c index 1d7d391acf..62d5ff47e3 100644 --- a/source/dnode/mnode/impl/src/mndScheduler.c +++ b/source/dnode/mnode/impl/src/mndScheduler.c @@ -296,8 +296,8 @@ static void setHTasksId(SArray* pTaskList, const SArray* pHTaskList) { SStreamTask** pStreamTask = taosArrayGet(pTaskList, i); SStreamTask** pHTask = taosArrayGet(pHTaskList, i); - (*pStreamTask)->historyTaskId.taskId = (*pHTask)->id.taskId; - (*pStreamTask)->historyTaskId.streamId = (*pHTask)->id.streamId; + (*pStreamTask)->hTaskInfo.id.taskId = (*pHTask)->id.taskId; + (*pStreamTask)->hTaskInfo.id.streamId = (*pHTask)->id.streamId; (*pHTask)->streamTaskId.taskId = (*pStreamTask)->id.taskId; (*pHTask)->streamTaskId.streamId = (*pStreamTask)->id.streamId; diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 3e060b4a38..22f0f256ea 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -867,7 +867,7 @@ int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t ver) { " child id:%d, level:%d, status:%s fill-history:%d, related fill-task:0x%x trigger:%" PRId64 " ms", vgId, pTask->id.idStr, pChkInfo->checkpointId, pChkInfo->checkpointVer, pChkInfo->nextProcessVer, pTask->info.selfChildId, pTask->info.taskLevel, streamGetTaskStatusStr(pTask->status.taskStatus), - pTask->info.fillHistory, (int32_t)pTask->historyTaskId.taskId, pTask->info.triggerParam); + pTask->info.fillHistory, (int32_t)pTask->hTaskInfo.id.taskId, pTask->info.triggerParam); } return 0; @@ -1218,7 +1218,7 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { } else { STimeWindow* pWindow = &pTask->dataRange.window; - if (pTask->historyTaskId.taskId == 0) { + if (pTask->hTaskInfo.id.taskId == 0) { *pWindow = (STimeWindow){INT64_MIN, INT64_MAX}; tqDebug( "s-task:%s scan-history in stream time window completed, no related fill-history task, reset the time " @@ -1432,12 +1432,12 @@ int32_t tqProcessTaskPauseReq(STQ* pTq, int64_t sversion, char* msg, int32_t msg streamTaskPause(pTask, pMeta); SStreamTask* pHistoryTask = NULL; - if (pTask->historyTaskId.taskId != 0) { - pHistoryTask = streamMetaAcquireTask(pMeta, pTask->historyTaskId.streamId, pTask->historyTaskId.taskId); + if (pTask->hTaskInfo.id.taskId != 0) { + pHistoryTask = streamMetaAcquireTask(pMeta, pTask->hTaskInfo.id.streamId, pTask->hTaskInfo.id.taskId); if (pHistoryTask == NULL) { tqError("vgId:%d process pause req, failed to acquire fill-history task:0x%" PRIx64 ", it may have been dropped already", - pMeta->vgId, pTask->historyTaskId.taskId); + pMeta->vgId, pTask->hTaskInfo.id.taskId); streamMetaReleaseTask(pMeta, pTask); // since task is in [STOP|DROPPING] state, it is safe to assume the pause is active @@ -1506,7 +1506,7 @@ int32_t tqProcessTaskResumeReq(STQ* pTq, int64_t sversion, char* msg, int32_t ms } SStreamTask* pHistoryTask = - streamMetaAcquireTask(pTq->pStreamMeta, pTask->historyTaskId.streamId, pTask->historyTaskId.taskId); + streamMetaAcquireTask(pTq->pStreamMeta, pTask->hTaskInfo.id.streamId, pTask->hTaskInfo.id.taskId); if (pHistoryTask) { code = tqProcessTaskResumeImpl(pTq, pHistoryTask, sversion, pReq->igUntreated); } @@ -1802,8 +1802,8 @@ int32_t tqProcessTaskUpdateReq(STQ* pTq, SRpcMsg* pMsg) { streamSetStatusNormal(pTask); SStreamTask** ppHTask = NULL; - if (pTask->historyTaskId.taskId != 0) { - ppHTask = (SStreamTask**)taosHashGet(pMeta->pTasksMap, &pTask->historyTaskId, sizeof(pTask->historyTaskId)); + if (pTask->hTaskInfo.id.taskId != 0) { + ppHTask = (SStreamTask**)taosHashGet(pMeta->pTasksMap, &pTask->hTaskInfo.id, sizeof(pTask->hTaskInfo.id)); if (ppHTask == NULL || *ppHTask == NULL) { tqError("vgId:%d failed to acquire fill-history task:0x%x when handling update, it may have been dropped already", pMeta->vgId, req.taskId); diff --git a/source/libs/stream/inc/streamInt.h b/source/libs/stream/inc/streamInt.h index dc5cf497db..d6f6de3de8 100644 --- a/source/libs/stream/inc/streamInt.h +++ b/source/libs/stream/inc/streamInt.h @@ -26,7 +26,19 @@ extern "C" { #endif -#define CHECK_DOWNSTREAM_INTERVAL 100 +#define CHECK_DOWNSTREAM_INTERVAL 100 +#define LAUNCH_HTASK_INTERVAL 100 +#define WAIT_FOR_MINIMAL_INTERVAL 100.00 +#define MAX_RETRY_LAUNCH_HISTORY_TASK 20 +#define RETRY_LAUNCH_INTERVAL_INC_RATE 1.2 + +#define MAX_BLOCK_NAME_NUM 1024 +#define DISPATCH_RETRY_INTERVAL_MS 300 +#define MAX_CONTINUE_RETRY_COUNT 5 + +#define META_HB_CHECK_INTERVAL 200 +#define META_HB_SEND_IDLE_COUNTER 25 // send hb every 5 sec +#define STREAM_TASK_KEY_LEN ((sizeof(int64_t)) << 1) // clang-format off #define stFatal(...) do { if (stDebugFlag & DEBUG_FATAL) { taosPrintLog("STM FATAL ", DEBUG_FATAL, 255, __VA_ARGS__); }} while(0) @@ -58,11 +70,6 @@ struct STokenBucket { int64_t fillTimestamp; // fill timestamp }; -struct STaskTimer { - void* hTaskLaunchTimer; - void* dispatchTimer; -}; - extern SStreamGlobalEnv streamEnv; extern int32_t streamBackendId; extern int32_t streamBackendCfWrapperId; @@ -107,6 +114,9 @@ int32_t streamTaskFillHistoryFinished(SStreamTask* pTask); int32_t streamTransferStateToStreamTask(SStreamTask* pTask); int32_t streamTaskInitTokenBucket(STokenBucket* pBucket, int32_t numCap, int32_t numRate, int32_t bytesRate); +STaskId streamTaskExtractKey(const SStreamTask* pTask); +void streamTaskInitForLaunchHTask(SHistoryTaskInfo* pInfo); +void streamTaskSetRetryInfoForLaunch(SHistoryTaskInfo* pInfo); SStreamQueue* streamQueueOpen(int64_t cap); void streamQueueClose(SStreamQueue* pQueue, int32_t taskId); @@ -115,7 +125,7 @@ void streamQueueProcessFail(SStreamQueue* queue); void* streamQueueNextItem(SStreamQueue* pQueue); void streamFreeQitem(SStreamQueueItem* data); -STaskId extractStreamTaskKey(const SStreamTask* pTask); + #ifdef __cplusplus } diff --git a/source/libs/stream/src/streamDispatch.c b/source/libs/stream/src/streamDispatch.c index ea8c7c71ac..2d701d6bb0 100644 --- a/source/libs/stream/src/streamDispatch.c +++ b/source/libs/stream/src/streamDispatch.c @@ -18,10 +18,6 @@ #include "ttimer.h" #include "tmisce.h" -#define MAX_BLOCK_NAME_NUM 1024 -#define DISPATCH_RETRY_INTERVAL_MS 300 -#define MAX_CONTINUE_RETRY_COUNT 5 - typedef struct SBlockName { uint32_t hashValue; char parTbName[TSDB_TABLE_NAME_LEN]; @@ -425,7 +421,7 @@ static void doRetryDispatchData(void* param, void* tmrId) { int32_t msgId = pTask->execInfo.dispatch; if (streamTaskShouldStop(&pTask->status)) { - int8_t ref = atomic_sub_fetch_8(&pTask->status.timerActive, 1); + int8_t ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1); stDebug("s-task:%s should stop, abort from timer, ref:%d", pTask->id.idStr, ref); return; } @@ -487,26 +483,25 @@ static void doRetryDispatchData(void* param, void* tmrId) { streamRetryDispatchData(pTask, DISPATCH_RETRY_INTERVAL_MS); } } else { - int32_t ref = atomic_sub_fetch_8(&pTask->status.timerActive, 1); + int32_t ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1); stDebug("s-task:%s should stop, abort from timer, ref:%d", pTask->id.idStr, ref); } } else { - int8_t ref = atomic_sub_fetch_8(&pTask->status.timerActive, 1); + int8_t ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1); stDebug("s-task:%s send success, jump out of timer, ref:%d", pTask->id.idStr, ref); } } void streamRetryDispatchData(SStreamTask* pTask, int64_t waitDuration) { - STaskTimer* pTmr = pTask->pTimer; pTask->msgInfo.retryCount++; stWarn("s-task:%s retry send dispatch data in %" PRId64 "ms, in timer msgId:%d, retryTimes:%d", pTask->id.idStr, waitDuration, pTask->execInfo.dispatch, pTask->msgInfo.retryCount); - if (pTmr->dispatchTimer != NULL) { - taosTmrReset(doRetryDispatchData, waitDuration, pTask, streamEnv.timer, &pTmr->dispatchTimer); + if (pTask->msgInfo.pTimer != NULL) { + taosTmrReset(doRetryDispatchData, waitDuration, pTask, streamEnv.timer, &pTask->msgInfo.pTimer); } else { - pTmr->dispatchTimer = taosTmrStart(doRetryDispatchData, waitDuration, pTask, streamEnv.timer); + pTask->msgInfo.pTimer = taosTmrStart(doRetryDispatchData, waitDuration, pTask, streamEnv.timer); } } @@ -636,7 +631,7 @@ int32_t streamDispatchStreamBlock(SStreamTask* pTask) { } if (++retryCount > MAX_CONTINUE_RETRY_COUNT) { // add to timer to retry - int8_t ref = atomic_add_fetch_8(&pTask->status.timerActive, 1); + int8_t ref = atomic_add_fetch_32(&pTask->status.timerActive, 1); stDebug("s-task:%s failed to dispatch msg to downstream for %d times, code:%s, add timer to retry in %dms, ref:%d", pTask->id.idStr, retryCount, tstrerror(terrno), DISPATCH_RETRY_INTERVAL_MS, ref); @@ -1143,7 +1138,7 @@ int32_t streamProcessDispatchRsp(SStreamTask* pTask, SStreamDispatchRsp* pRsp, i stDebug("s-task:%s waiting rsp set to be %d", id, pTask->shuffleDispatcher.waitingRspCnt); } - int8_t ref = atomic_add_fetch_8(&pTask->status.timerActive, 1); + int8_t ref = atomic_add_fetch_32(&pTask->status.timerActive, 1); stDebug("s-task:%s failed to dispatch msg to downstream code:%s, add timer to retry in %dms, ref:%d", pTask->id.idStr, tstrerror(terrno), DISPATCH_RETRY_INTERVAL_MS, ref); diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index 7759de9590..b5ea82d347 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -310,7 +310,7 @@ int32_t streamDoTransferStateToStreamTask(SStreamTask* pTask) { pStreamTask->id.idStr); } - ASSERT(pStreamTask->historyTaskId.taskId == pTask->id.taskId && pTask->status.appendTranstateBlock == true); + ASSERT(pStreamTask->hTaskInfo.id.taskId == pTask->id.taskId && pTask->status.appendTranstateBlock == true); STimeWindow* pTimeWindow = &pStreamTask->dataRange.window; @@ -361,8 +361,8 @@ int32_t streamDoTransferStateToStreamTask(SStreamTask* pTask) { streamBuildAndSendDropTaskMsg(pTask->pMsgCb, pMeta->vgId, &pTask->id); // 5. clear the link between fill-history task and stream task info - pStreamTask->historyTaskId.taskId = 0; - pStreamTask->historyTaskId.streamId = 0; + pStreamTask->hTaskInfo.id.taskId = 0; + pStreamTask->hTaskInfo.id.streamId = 0; // 6. save to disk taosWLockLatch(&pMeta->lock); diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index d37ee9840d..da1acc6965 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -21,10 +21,6 @@ #include "tstream.h" #include "ttimer.h" -#define META_HB_CHECK_INTERVAL 200 -#define META_HB_SEND_IDLE_COUNTER 25 // send hb every 5 sec -#define STREAM_TASK_KEY_LEN ((sizeof(int64_t)) << 1) - static TdThreadOnce streamMetaModuleInit = PTHREAD_ONCE_INIT; int32_t streamBackendId = 0; @@ -547,8 +543,8 @@ int32_t streamMetaUnregisterTask(SStreamMeta* pMeta, int64_t streamId, int32_t t STaskId streamTaskId = {.streamId = (*ppTask)->streamTaskId.streamId, .taskId = (*ppTask)->streamTaskId.taskId}; SStreamTask** ppStreamTask = (SStreamTask**)taosHashGet(pMeta->pTasksMap, &streamTaskId, sizeof(streamTaskId)); if (ppStreamTask != NULL) { - (*ppStreamTask)->historyTaskId.taskId = 0; - (*ppStreamTask)->historyTaskId.streamId = 0; + (*ppStreamTask)->hTaskInfo.id.taskId = 0; + (*ppStreamTask)->hTaskInfo.id.streamId = 0; } } else { atomic_sub_fetch_32(&pMeta->numOfStreamTasks, 1); @@ -698,7 +694,7 @@ int32_t streamMetaLoadAllTasks(SStreamMeta* pMeta) { int32_t taskId = pTask->id.taskId; tFreeStreamTask(pTask); - STaskId id = extractStreamTaskKey(pTask); + STaskId id = streamTaskExtractKey(pTask); taosArrayPush(pRecycleList, &id); int32_t total = taosArrayGetSize(pRecycleList); @@ -807,7 +803,7 @@ int32_t tDecodeStreamHbMsg(SDecoder* pDecoder, SStreamHbMsg* pReq) { return 0; } -static bool enoughTimeDuration(SMetaHbInfo* pInfo) { +static bool waitForEnoughDuration(SMetaHbInfo* pInfo) { if ((++pInfo->tickCounter) >= META_HB_SEND_IDLE_COUNTER) { // reset the counter pInfo->tickCounter = 0; return true; @@ -844,7 +840,7 @@ void metaHbToMnode(void* param, void* tmrId) { pMeta->pHbInfo->hbStart = taosGetTimestampMs(); } - if (!enoughTimeDuration(pMeta->pHbInfo)) { + if (!waitForEnoughDuration(pMeta->pHbInfo)) { taosTmrReset(metaHbToMnode, META_HB_CHECK_INTERVAL, param, streamEnv.timer, &pMeta->pHbInfo->hbTmr); taosReleaseRef(streamMetaId, rid); return; diff --git a/source/libs/stream/src/streamRecover.c b/source/libs/stream/src/streamRecover.c index 3ca81ea90b..4018d86dd0 100644 --- a/source/libs/stream/src/streamRecover.c +++ b/source/libs/stream/src/streamRecover.c @@ -29,9 +29,12 @@ typedef struct STaskRecheckInfo { void* checkTimer; } STaskRecheckInfo; -static int32_t streamSetParamForScanHistory(SStreamTask* pTask); -static void streamTaskSetRangeStreamCalc(SStreamTask* pTask); -static int32_t initScanHistoryReq(SStreamTask* pTask, SStreamScanHistoryReq* pReq, int8_t igUntreated); +static int32_t streamSetParamForScanHistory(SStreamTask* pTask); +static void streamTaskSetRangeStreamCalc(SStreamTask* pTask); +static int32_t initScanHistoryReq(SStreamTask* pTask, SStreamScanHistoryReq* pReq, int8_t igUntreated); +static int32_t getNextRetryInterval(int32_t waitInterval); +static SLaunchHTaskInfo* createHTaskLaunchInfo(SStreamMeta* pMeta, int64_t streamId, int32_t taskId); +static void tryLaunchHistoryTask(void* param, void* tmrId); static void streamTaskSetReady(SStreamTask* pTask, int32_t numOfReqs) { SStreamMeta* pMeta = pTask->pMeta; @@ -54,7 +57,7 @@ static void streamTaskSetReady(SStreamTask* pTask, int32_t numOfReqs) { taosWLockLatch(&pMeta->lock); - STaskId id = extractStreamTaskKey(pTask); + STaskId id = streamTaskExtractKey(pTask); taosHashPut(pMeta->startInfo.pReadyTaskSet, &id, sizeof(id), NULL, 0); int32_t numOfTotal = streamMetaGetNumOfTasks(pMeta); @@ -90,20 +93,6 @@ int32_t streamStartScanHistoryAsync(SStreamTask* pTask, int8_t igUntreated) { return 0; } -const char* streamGetTaskStatusStr(int32_t status) { - switch(status) { - case TASK_STATUS__NORMAL: return "normal"; - case TASK_STATUS__SCAN_HISTORY: return "scan-history"; - case TASK_STATUS__HALT: return "halt"; - case TASK_STATUS__PAUSE: return "paused"; - case TASK_STATUS__CK: return "check-point"; - case TASK_STATUS__DROPPING: return "dropping"; - case TASK_STATUS__STOP: return "stop"; - case TASK_STATUS__UNINIT: return "uninitialized"; - default:return ""; - } -} - static int32_t doLaunchScanHistoryTask(SStreamTask* pTask) { SVersionRange* pRange = &pTask->dataRange.range; if (pTask->info.fillHistory) { @@ -249,7 +238,7 @@ static void recheckDownstreamTasks(void* param, void* tmrId) { } destroyRecheckInfo(pInfo); - int8_t ref = atomic_sub_fetch_8(&pTask->status.timerActive, 1); + int8_t ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1); stDebug("s-task:%s complete send check in timer, ref:%d", pTask->id.idStr, ref); } @@ -302,7 +291,7 @@ static void doProcessDownstreamReadyRsp(SStreamTask* pTask, int32_t numOfReqs) { if (pTask->info.fillHistory == 1) { stDebug("s-task:%s fill-history is set normal when start it, try to remove it,set it task to be dropping", id); pTask->status.taskStatus = TASK_STATUS__DROPPING; - ASSERT(pTask->historyTaskId.taskId == 0); + ASSERT(pTask->hTaskInfo.id.taskId == 0); } else { stDebug("s-task:%s downstream tasks are ready, now ready for data from wal, status:%s", id, str); streamTaskEnablePause(pTask); @@ -375,7 +364,7 @@ int32_t streamProcessCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* pRs } else { STaskRecheckInfo* pInfo = createRecheckInfo(pTask, pRsp); - int8_t ref = atomic_add_fetch_8(&pTask->status.timerActive, 1); + int8_t ref = atomic_add_fetch_32(&pTask->status.timerActive, 1); stDebug("s-task:%s downstream taskId:0x%x (vgId:%d) not ready, stage:%d, retry in 100ms, ref:%d ", id, pRsp->downstreamTaskId, pRsp->downstreamNodeId, pRsp->oldStage, ref); @@ -464,6 +453,10 @@ int32_t initScanHistoryReq(SStreamTask* pTask, SStreamScanHistoryReq* pReq, int8 return 0; } +int32_t getNextRetryInterval(int32_t waitInterval) { + return waitInterval * RETRY_LAUNCH_INTERVAL_INC_RATE; +} + int32_t streamTaskPutTranstateIntoInputQ(SStreamTask* pTask) { SStreamDataBlock* pTranstate = taosAllocateQitem(sizeof(SStreamDataBlock), DEF_QITEM, sizeof(SSDataBlock)); if (pTranstate == NULL) { @@ -609,12 +602,47 @@ static void checkFillhistoryTaskStatus(SStreamTask* pTask, SStreamTask* pHTask) doCheckDownstreamStatus(pHTask); } +static bool doLaunchHistoryTask(SStreamTask* pTask, SLaunchHTaskInfo* pInfo) { + SStreamMeta* pMeta = pTask->pMeta; + streamTaskSetRetryInfoForLaunch(&pTask->hTaskInfo); + + stDebug("s-task:%s try launch related fill-history task in timer, retry:%d", pTask->id.idStr, + pTask->hTaskInfo.retryTimes); + + ASSERT(pTask->status.timerActive >= 1); + + // abort the timer if intend to stop task + SStreamTask* pHTask = streamMetaAcquireTask(pMeta, pTask->hTaskInfo.id.streamId, pTask->hTaskInfo.id.taskId); + if (pHTask == NULL && (!streamTaskShouldStop(&pTask->status))) { + const char* p = streamGetTaskStatusStr(pTask->status.taskStatus); + stWarn( + "s-task:%s vgId:%d status:%s failed to launch history task:0x%x, since it may not be built, or may have " + "been destroyed, or should stop", + pTask->id.idStr, pMeta->vgId, streamGetTaskStatusStr(pTask->status.taskStatus), (int32_t)pTask->hTaskInfo.id.taskId); + + taosTmrReset(tryLaunchHistoryTask, LAUNCH_HTASK_INTERVAL, pInfo, streamEnv.timer, &pTask->hTaskInfo.pTimer); + streamMetaReleaseTask(pMeta, pTask); + return true; + } + + if (pHTask != NULL) { + checkFillhistoryTaskStatus(pTask, pHTask); + streamMetaReleaseTask(pMeta, pHTask); + } + + // not in timer anymore + int8_t ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1); + stDebug("s-task:0x%x fill-history task launch completed, retry times:%d, ref:%d", (int32_t)pInfo->id.taskId, + pTask->hTaskInfo.retryTimes, ref); + streamMetaReleaseTask(pMeta, pTask); + + return false; +} + static void tryLaunchHistoryTask(void* param, void* tmrId) { SLaunchHTaskInfo* pInfo = param; SStreamMeta* pMeta = pInfo->pMeta; - stDebug("s-task:0x%x in timer to launch related history task", (int32_t) pInfo->id.taskId); - taosWLockLatch(&pMeta->lock); SStreamTask** ppTask = (SStreamTask**)taosHashGet(pMeta->pTasksMap, &pInfo->id, sizeof(pInfo->id)); if (ppTask) { @@ -622,10 +650,12 @@ static void tryLaunchHistoryTask(void* param, void* tmrId) { if (streamTaskShouldStop(&(*ppTask)->status)) { const char* pStatus = streamGetTaskStatusStr((*ppTask)->status.taskStatus); - stDebug("s-task:%s status:%s quit timer task", (*ppTask)->id.idStr, pStatus); + + int32_t ref = atomic_sub_fetch_32(&(*ppTask)->status.timerActive, 1); + stDebug("s-task:%s status:%s should stop, quit launch fill-history task timer, retry:%d, ref:%d", + (*ppTask)->id.idStr, pStatus, (*ppTask)->hTaskInfo.retryTimes, ref); taosMemoryFree(pInfo); - atomic_sub_fetch_8(&(*ppTask)->status.timerActive, 1); taosWUnLockLatch(&pMeta->lock); return; } @@ -634,78 +664,120 @@ static void tryLaunchHistoryTask(void* param, void* tmrId) { SStreamTask* pTask = streamMetaAcquireTask(pMeta, pInfo->id.streamId, pInfo->id.taskId); if (pTask != NULL) { - ASSERT(pTask->status.timerActive >= 1); - // abort the timer if intend to stop task - SStreamTask* pHTask = streamMetaAcquireTask(pMeta, pTask->historyTaskId.streamId, pTask->historyTaskId.taskId); - if (pHTask == NULL && (!streamTaskShouldStop(&pTask->status))) { - const char* pStatus = streamGetTaskStatusStr(pTask->status.taskStatus); - stWarn( - "s-task:%s vgId:%d status:%s failed to launch history task:0x%x, since it may not be built, or may have been " - "destroyed, or should stop", - pTask->id.idStr, pMeta->vgId, pStatus, (int32_t) pTask->historyTaskId.taskId); + SHistoryTaskInfo* pHTaskInfo = &pTask->hTaskInfo; - taosTmrReset(tryLaunchHistoryTask, 100, pInfo, streamEnv.timer, &pTask->pTimer->hTaskLaunchTimer); + pHTaskInfo->tickCount -= 1; + if (pHTaskInfo->tickCount > 0) { + taosTmrReset(tryLaunchHistoryTask, LAUNCH_HTASK_INTERVAL, pInfo, streamEnv.timer, &pHTaskInfo->pTimer); streamMetaReleaseTask(pMeta, pTask); return; } - if (pHTask != NULL) { - checkFillhistoryTaskStatus(pTask, pHTask); - streamMetaReleaseTask(pMeta, pHTask); - } + if (pHTaskInfo->retryTimes > MAX_RETRY_LAUNCH_HISTORY_TASK) { + int8_t ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1); + taosMemoryFree(pInfo); + streamMetaReleaseTask(pMeta, pTask); - // not in timer anymore - atomic_sub_fetch_8(&pTask->status.timerActive, 1); - streamMetaReleaseTask(pMeta, pTask); + stError("s-task:%s max retry:%d reached, quit from retrying launch related fill-history task timer, ref:%d", + pTask->id.idStr, MAX_RETRY_LAUNCH_HISTORY_TASK, ref); + + pHTaskInfo->id.taskId = 0; + pHTaskInfo->id.streamId = 0; + } else { // not reach the limitation yet, let's continue retrying launch related fill-history task. + streamTaskSetRetryInfoForLaunch(pHTaskInfo); + ASSERT(pTask->status.timerActive >= 1); + + // abort the timer if intend to stop task + SStreamTask* pHTask = streamMetaAcquireTask(pMeta, pHTaskInfo->id.streamId, pHTaskInfo->id.taskId); + if (pHTask == NULL && (!streamTaskShouldStop(&pTask->status))) { + const char* p = streamGetTaskStatusStr(pTask->status.taskStatus); + int32_t hTaskId = pHTaskInfo->id.taskId; + stDebug( + "s-task:%s status:%s failed to launch fill-history task:0x%x, retry launch related fill-history task in " + "timer, retryCount:%d", + pTask->id.idStr, p, pHTaskInfo->retryTimes, hTaskId); + + taosTmrReset(tryLaunchHistoryTask, LAUNCH_HTASK_INTERVAL, pInfo, streamEnv.timer, &pHTaskInfo->pTimer); + streamMetaReleaseTask(pMeta, pTask); + return; + } + + if (pHTask != NULL) { + checkFillhistoryTaskStatus(pTask, pHTask); + streamMetaReleaseTask(pMeta, pHTask); + } + + // not in timer anymore + int8_t ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1); + stDebug("s-task:0x%x fill-history task launch completed, retry times:%d, ref:%d", (int32_t)pInfo->id.taskId, + pHTaskInfo->retryTimes, ref); + streamMetaReleaseTask(pMeta, pTask); + } } else { - stError("s-task:0x%x failed to load task, it may have been destroyed", (int32_t) pInfo->id.taskId); + stError("s-task:0x%x failed to load task, it may have been destroyed, not launch related fill-history task", + (int32_t)pInfo->id.taskId); } taosMemoryFree(pInfo); } -// todo fix the bug: 2. race condition +SLaunchHTaskInfo* createHTaskLaunchInfo(SStreamMeta* pMeta, int64_t streamId, int32_t taskId) { + SLaunchHTaskInfo* pInfo = taosMemoryCalloc(1, sizeof(SLaunchHTaskInfo)); + if (pInfo == NULL) { + terrno = TSDB_CODE_OUT_OF_MEMORY; + return NULL; + } + + pInfo->id.taskId = taskId; + pInfo->id.streamId = streamId; + pInfo->pMeta = pMeta; + return pInfo; +} + // an fill history task needs to be started. int32_t streamLaunchFillHistoryTask(SStreamTask* pTask) { SStreamMeta* pMeta = pTask->pMeta; - int32_t hTaskId = pTask->historyTaskId.taskId; + int32_t hTaskId = pTask->hTaskInfo.id.taskId; if (hTaskId == 0) { return TSDB_CODE_SUCCESS; } ASSERT(pTask->status.downstreamReady == 1); stDebug("s-task:%s start to launch related fill-history task:0x%" PRIx64 "-0x%x", pTask->id.idStr, - pTask->historyTaskId.streamId, hTaskId); + pTask->hTaskInfo.id.streamId, hTaskId); // Set the execute conditions, including the query time window and the version range - SStreamTask** pHTask = taosHashGet(pMeta->pTasksMap, &pTask->historyTaskId, sizeof(pTask->historyTaskId)); + SStreamTask** pHTask = taosHashGet(pMeta->pTasksMap, &pTask->hTaskInfo.id, sizeof(pTask->hTaskInfo.id)); if (pHTask == NULL) { - stWarn("s-task:%s vgId:%d failed to launch history task:0x%x, since it is not built yet", pTask->id.idStr, - pMeta->vgId, hTaskId); + stWarn("s-task:%s vgId:%d failed to launch history task:0x%x, since not built yet", pTask->id.idStr, pMeta->vgId, + hTaskId); - SLaunchHTaskInfo* pInfo = taosMemoryCalloc(1, sizeof(SLaunchHTaskInfo)); - pInfo->id.taskId = pTask->id.taskId; - pInfo->id.streamId = pTask->id.streamId; - pInfo->pMeta = pTask->pMeta; + SLaunchHTaskInfo* pInfo = createHTaskLaunchInfo(pTask->pMeta, pTask->id.streamId, pTask->id.taskId); + if (pInfo == NULL) { + stError("s-task:%s failed to launch related fill-history task, since Out Of Memory", pTask->id.idStr); + return terrno; + } - if (pTask->pTimer->hTaskLaunchTimer == NULL) { - pTask->pTimer->hTaskLaunchTimer = taosTmrStart(tryLaunchHistoryTask, 100, pInfo, streamEnv.timer); - if (pTask->pTimer->hTaskLaunchTimer == NULL) { - // todo failed to create timer + streamTaskInitForLaunchHTask(&pTask->hTaskInfo); + if (pTask->hTaskInfo.pTimer == NULL) { + int32_t ref = atomic_add_fetch_32(&pTask->status.timerActive, 1); + pTask->hTaskInfo.pTimer = taosTmrStart(tryLaunchHistoryTask, WAIT_FOR_MINIMAL_INTERVAL, pInfo, streamEnv.timer); + if (pTask->hTaskInfo.pTimer == NULL) { // todo failed to create timer + atomic_sub_fetch_32(&pTask->status.timerActive, 1); + stError("s-task:%s failed to start timer, related fill-history task not launched, ref:%d", pTask->id.idStr, + pTask->status.timerActive); taosMemoryFree(pInfo); } else { - int32_t ref = atomic_add_fetch_8(&pTask->status.timerActive, 1);// timer is active - ASSERT(ref == 1); + ASSERT(ref >= 1); stDebug("s-task:%s set timer active flag, ref:%d", pTask->id.idStr, ref); } } else { // timer exists - ASSERT(pTask->status.timerActive == 1); + ASSERT(pTask->status.timerActive >= 1); stDebug("s-task:%s set timer active flag, task timer not null", pTask->id.idStr); - taosTmrReset(tryLaunchHistoryTask, 100, pInfo, streamEnv.timer, &pTask->pTimer->hTaskLaunchTimer); + taosTmrReset(tryLaunchHistoryTask, WAIT_FOR_MINIMAL_INTERVAL, pInfo, streamEnv.timer, &pTask->hTaskInfo.pTimer); } - // try again in 100ms return TSDB_CODE_SUCCESS; } @@ -849,7 +921,7 @@ int32_t tDecodeStreamScanHistoryFinishReq(SDecoder* pDecoder, SStreamScanHistory } void streamTaskSetRangeStreamCalc(SStreamTask* pTask) { - if (pTask->historyTaskId.taskId == 0) { + if (pTask->hTaskInfo.id.taskId == 0) { SDataRange* pRange = &pTask->dataRange; if (pTask->info.fillHistory == 1) { stDebug("s-task:%s fill-history task, time window:%" PRId64 "-%" PRId64 ", verRange:%" PRId64 diff --git a/source/libs/stream/src/streamTask.c b/source/libs/stream/src/streamTask.c index ea7e89cf1b..e77ab16040 100644 --- a/source/libs/stream/src/streamTask.c +++ b/source/libs/stream/src/streamTask.c @@ -96,8 +96,8 @@ int32_t tEncodeStreamTask(SEncoder* pEncoder, const SStreamTask* pTask) { if (tEncodeI64(pEncoder, pTask->chkInfo.checkpointVer) < 0) return -1; if (tEncodeI8(pEncoder, pTask->info.fillHistory) < 0) return -1; - if (tEncodeI64(pEncoder, pTask->historyTaskId.streamId)) return -1; - int32_t taskId = pTask->historyTaskId.taskId; + if (tEncodeI64(pEncoder, pTask->hTaskInfo.id.streamId)) return -1; + int32_t taskId = pTask->hTaskInfo.id.taskId; if (tEncodeI32(pEncoder, taskId)) return -1; if (tEncodeI64(pEncoder, pTask->streamTaskId.streamId)) return -1; @@ -169,9 +169,9 @@ int32_t tDecodeStreamTask(SDecoder* pDecoder, SStreamTask* pTask) { if (tDecodeI64(pDecoder, &pTask->chkInfo.checkpointVer) < 0) return -1; if (tDecodeI8(pDecoder, &pTask->info.fillHistory) < 0) return -1; - if (tDecodeI64(pDecoder, &pTask->historyTaskId.streamId)) return -1; + if (tDecodeI64(pDecoder, &pTask->hTaskInfo.id.streamId)) return -1; if (tDecodeI32(pDecoder, &taskId)) return -1; - pTask->historyTaskId.taskId = taskId; + pTask->hTaskInfo.id.taskId = taskId; if (tDecodeI64(pDecoder, &pTask->streamTaskId.streamId)) return -1; if (tDecodeI32(pDecoder, &taskId)) return -1; @@ -312,18 +312,14 @@ void tFreeStreamTask(SStreamTask* pTask) { pTask->schedInfo.pTimer = NULL; } - if (pTask->pTimer != NULL) { - if (pTask->pTimer->hTaskLaunchTimer != NULL) { - taosTmrStop(pTask->pTimer->hTaskLaunchTimer); - pTask->pTimer->hTaskLaunchTimer = NULL; - } + if (pTask->hTaskInfo.pTimer != NULL) { + taosTmrStop(pTask->hTaskInfo.pTimer); + pTask->hTaskInfo.pTimer = NULL; + } - if (pTask->pTimer->dispatchTimer != NULL) { - taosTmrStop(pTask->pTimer->dispatchTimer); - pTask->pTimer->dispatchTimer = NULL; - } - - taosMemoryFreeClear(pTask->pTimer); + if (pTask->msgInfo.pTimer != NULL) { + taosTmrStop(pTask->msgInfo.pTimer); + pTask->msgInfo.pTimer = NULL; } int32_t status = atomic_load_8((int8_t*)&(pTask->status.taskStatus)); @@ -425,12 +421,6 @@ int32_t streamTaskInit(SStreamTask* pTask, SStreamMeta* pMeta, SMsgCb* pMsgCb, i return TSDB_CODE_OUT_OF_MEMORY; } - pTask->pTimer = taosMemoryCalloc(1, sizeof(STaskTimer)); - if (pTask->pTimer == NULL) { - stError("s-task:%s failed to prepare the timer, code:%s", pTask->id.idStr, tstrerror(TSDB_CODE_OUT_OF_MEMORY)); - return TSDB_CODE_OUT_OF_MEMORY; - } - // 2MiB per second for sink task // 50 times sink operator per second streamTaskInitTokenBucket(pTask->pTokenBucket, 50, 50, 2); @@ -689,7 +679,35 @@ int32_t streamBuildAndSendDropTaskMsg(SMsgCb* pMsgCb, int32_t vgId, SStreamTaskI return code; } -STaskId extractStreamTaskKey(const SStreamTask* pTask) { +STaskId streamTaskExtractKey(const SStreamTask* pTask) { STaskId id = {.streamId = pTask->id.streamId, .taskId = pTask->id.taskId}; return id; +} + +void streamTaskInitForLaunchHTask(SHistoryTaskInfo* pInfo) { + pInfo->waitInterval = LAUNCH_HTASK_INTERVAL; + pInfo->tickCount = ceil(LAUNCH_HTASK_INTERVAL / WAIT_FOR_MINIMAL_INTERVAL); + pInfo->retryTimes = 0; +} + +void streamTaskSetRetryInfoForLaunch(SHistoryTaskInfo* pInfo) { + ASSERT(pInfo->tickCount == 0); + + pInfo->waitInterval *= RETRY_LAUNCH_INTERVAL_INC_RATE; + pInfo->tickCount = ceil(pInfo->waitInterval / WAIT_FOR_MINIMAL_INTERVAL); + pInfo->retryTimes += 1; +} + +const char* streamGetTaskStatusStr(int32_t status) { + switch(status) { + case TASK_STATUS__NORMAL: return "normal"; + case TASK_STATUS__SCAN_HISTORY: return "scan-history"; + case TASK_STATUS__HALT: return "halt"; + case TASK_STATUS__PAUSE: return "paused"; + case TASK_STATUS__CK: return "check-point"; + case TASK_STATUS__DROPPING: return "dropping"; + case TASK_STATUS__STOP: return "stop"; + case TASK_STATUS__UNINIT: return "uninitialized"; + default:return ""; + } } \ No newline at end of file From 6b5770aab778624de406a011eac038cf5dc779dd Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 27 Sep 2023 14:03:57 +0800 Subject: [PATCH 104/158] log(stream): update the log. --- source/libs/stream/src/streamRecover.c | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/source/libs/stream/src/streamRecover.c b/source/libs/stream/src/streamRecover.c index 4018d86dd0..386580199f 100644 --- a/source/libs/stream/src/streamRecover.c +++ b/source/libs/stream/src/streamRecover.c @@ -695,8 +695,8 @@ static void tryLaunchHistoryTask(void* param, void* tmrId) { int32_t hTaskId = pHTaskInfo->id.taskId; stDebug( "s-task:%s status:%s failed to launch fill-history task:0x%x, retry launch related fill-history task in " - "timer, retryCount:%d", - pTask->id.idStr, p, pHTaskInfo->retryTimes, hTaskId); + "%dms, retryCount:%d", + pTask->id.idStr, p, hTaskId, pHTaskInfo->waitInterval, pHTaskInfo->retryTimes); taosTmrReset(tryLaunchHistoryTask, LAUNCH_HTASK_INTERVAL, pInfo, streamEnv.timer, &pHTaskInfo->pTimer); streamMetaReleaseTask(pMeta, pTask); From 6d2dbd549b2a8a6b7ce768d2eb20cd21f7542695 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 27 Sep 2023 14:05:14 +0800 Subject: [PATCH 105/158] fix(stream): avoid repeatly free obj. --- source/libs/stream/src/streamRecover.c | 1 - 1 file changed, 1 deletion(-) diff --git a/source/libs/stream/src/streamRecover.c b/source/libs/stream/src/streamRecover.c index 386580199f..3946506238 100644 --- a/source/libs/stream/src/streamRecover.c +++ b/source/libs/stream/src/streamRecover.c @@ -676,7 +676,6 @@ static void tryLaunchHistoryTask(void* param, void* tmrId) { if (pHTaskInfo->retryTimes > MAX_RETRY_LAUNCH_HISTORY_TASK) { int8_t ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1); - taosMemoryFree(pInfo); streamMetaReleaseTask(pMeta, pTask); stError("s-task:%s max retry:%d reached, quit from retrying launch related fill-history task timer, ref:%d", From d10b07d23d18aa0ec6e1c706f953ee7963a7c37b Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 27 Sep 2023 14:18:01 +0800 Subject: [PATCH 106/158] refactor: do some internal refactor. --- source/libs/stream/inc/streamInt.h | 2 +- source/libs/stream/src/streamRecover.c | 3 +-- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/source/libs/stream/inc/streamInt.h b/source/libs/stream/inc/streamInt.h index d6f6de3de8..1d8fde3a48 100644 --- a/source/libs/stream/inc/streamInt.h +++ b/source/libs/stream/inc/streamInt.h @@ -29,7 +29,7 @@ extern "C" { #define CHECK_DOWNSTREAM_INTERVAL 100 #define LAUNCH_HTASK_INTERVAL 100 #define WAIT_FOR_MINIMAL_INTERVAL 100.00 -#define MAX_RETRY_LAUNCH_HISTORY_TASK 20 +#define MAX_RETRY_LAUNCH_HISTORY_TASK 40 #define RETRY_LAUNCH_INTERVAL_INC_RATE 1.2 #define MAX_BLOCK_NAME_NUM 1024 diff --git a/source/libs/stream/src/streamRecover.c b/source/libs/stream/src/streamRecover.c index 3946506238..8b538db7fb 100644 --- a/source/libs/stream/src/streamRecover.c +++ b/source/libs/stream/src/streamRecover.c @@ -693,8 +693,7 @@ static void tryLaunchHistoryTask(void* param, void* tmrId) { const char* p = streamGetTaskStatusStr(pTask->status.taskStatus); int32_t hTaskId = pHTaskInfo->id.taskId; stDebug( - "s-task:%s status:%s failed to launch fill-history task:0x%x, retry launch related fill-history task in " - "%dms, retryCount:%d", + "s-task:%s status:%s failed to launch fill-history task:0x%x, retry launch %dms, retryCount:%d", pTask->id.idStr, p, hTaskId, pHTaskInfo->waitInterval, pHTaskInfo->retryTimes); taosTmrReset(tryLaunchHistoryTask, LAUNCH_HTASK_INTERVAL, pInfo, streamEnv.timer, &pHTaskInfo->pTimer); From 60a720efbbc97f4d039b75cb2e7a8b7c347bb456 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 27 Sep 2023 14:48:53 +0800 Subject: [PATCH 107/158] log: update logs. --- source/libs/stream/src/streamRecover.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/libs/stream/src/streamRecover.c b/source/libs/stream/src/streamRecover.c index 8b538db7fb..d074e115f7 100644 --- a/source/libs/stream/src/streamRecover.c +++ b/source/libs/stream/src/streamRecover.c @@ -693,7 +693,7 @@ static void tryLaunchHistoryTask(void* param, void* tmrId) { const char* p = streamGetTaskStatusStr(pTask->status.taskStatus); int32_t hTaskId = pHTaskInfo->id.taskId; stDebug( - "s-task:%s status:%s failed to launch fill-history task:0x%x, retry launch %dms, retryCount:%d", + "s-task:%s status:%s failed to launch fill-history task:0x%x, retry launch:%dms, retryCount:%d", pTask->id.idStr, p, hTaskId, pHTaskInfo->waitInterval, pHTaskInfo->retryTimes); taosTmrReset(tryLaunchHistoryTask, LAUNCH_HTASK_INTERVAL, pInfo, streamEnv.timer, &pHTaskInfo->pTimer); From b716f0d1fd56f4aa8535ac010aa1f5de454127e6 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 27 Sep 2023 15:26:46 +0800 Subject: [PATCH 108/158] fix(stream): fix dead lock. --- source/libs/stream/src/streamBackendRocksdb.c | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index db5d431e30..f538e3e427 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -783,7 +783,10 @@ int32_t chkpGetAllDbCfHandle(SStreamMeta* pMeta, rocksdb_column_family_handle_t* int64_t id = *(int64_t*)pIter; SBackendCfWrapper* wrapper = taosAcquireRef(streamBackendCfWrapperId, id); - if (wrapper == NULL) continue; + if (wrapper == NULL) { + pIter = taosHashIterate(pMeta->pTaskBackendUnique, pIter); + continue; + } taosThreadRwlockRdlock(&wrapper->rwLock); for (int i = 0; i < sizeof(ginitDict) / sizeof(ginitDict[0]); i++) { From 12eabb0c2ce7dffdecae3c40e13f9c3317270166 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 27 Sep 2023 15:37:12 +0800 Subject: [PATCH 109/158] log(stream): update logs. --- source/libs/stream/src/streamRecover.c | 46 ++------------------------ 1 file changed, 2 insertions(+), 44 deletions(-) diff --git a/source/libs/stream/src/streamRecover.c b/source/libs/stream/src/streamRecover.c index d074e115f7..03a5233023 100644 --- a/source/libs/stream/src/streamRecover.c +++ b/source/libs/stream/src/streamRecover.c @@ -32,7 +32,6 @@ typedef struct STaskRecheckInfo { static int32_t streamSetParamForScanHistory(SStreamTask* pTask); static void streamTaskSetRangeStreamCalc(SStreamTask* pTask); static int32_t initScanHistoryReq(SStreamTask* pTask, SStreamScanHistoryReq* pReq, int8_t igUntreated); -static int32_t getNextRetryInterval(int32_t waitInterval); static SLaunchHTaskInfo* createHTaskLaunchInfo(SStreamMeta* pMeta, int64_t streamId, int32_t taskId); static void tryLaunchHistoryTask(void* param, void* tmrId); @@ -453,10 +452,6 @@ int32_t initScanHistoryReq(SStreamTask* pTask, SStreamScanHistoryReq* pReq, int8 return 0; } -int32_t getNextRetryInterval(int32_t waitInterval) { - return waitInterval * RETRY_LAUNCH_INTERVAL_INC_RATE; -} - int32_t streamTaskPutTranstateIntoInputQ(SStreamTask* pTask) { SStreamDataBlock* pTranstate = taosAllocateQitem(sizeof(SStreamDataBlock), DEF_QITEM, sizeof(SSDataBlock)); if (pTranstate == NULL) { @@ -602,43 +597,6 @@ static void checkFillhistoryTaskStatus(SStreamTask* pTask, SStreamTask* pHTask) doCheckDownstreamStatus(pHTask); } -static bool doLaunchHistoryTask(SStreamTask* pTask, SLaunchHTaskInfo* pInfo) { - SStreamMeta* pMeta = pTask->pMeta; - streamTaskSetRetryInfoForLaunch(&pTask->hTaskInfo); - - stDebug("s-task:%s try launch related fill-history task in timer, retry:%d", pTask->id.idStr, - pTask->hTaskInfo.retryTimes); - - ASSERT(pTask->status.timerActive >= 1); - - // abort the timer if intend to stop task - SStreamTask* pHTask = streamMetaAcquireTask(pMeta, pTask->hTaskInfo.id.streamId, pTask->hTaskInfo.id.taskId); - if (pHTask == NULL && (!streamTaskShouldStop(&pTask->status))) { - const char* p = streamGetTaskStatusStr(pTask->status.taskStatus); - stWarn( - "s-task:%s vgId:%d status:%s failed to launch history task:0x%x, since it may not be built, or may have " - "been destroyed, or should stop", - pTask->id.idStr, pMeta->vgId, streamGetTaskStatusStr(pTask->status.taskStatus), (int32_t)pTask->hTaskInfo.id.taskId); - - taosTmrReset(tryLaunchHistoryTask, LAUNCH_HTASK_INTERVAL, pInfo, streamEnv.timer, &pTask->hTaskInfo.pTimer); - streamMetaReleaseTask(pMeta, pTask); - return true; - } - - if (pHTask != NULL) { - checkFillhistoryTaskStatus(pTask, pHTask); - streamMetaReleaseTask(pMeta, pHTask); - } - - // not in timer anymore - int8_t ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1); - stDebug("s-task:0x%x fill-history task launch completed, retry times:%d, ref:%d", (int32_t)pInfo->id.taskId, - pTask->hTaskInfo.retryTimes, ref); - streamMetaReleaseTask(pMeta, pTask); - - return false; -} - static void tryLaunchHistoryTask(void* param, void* tmrId) { SLaunchHTaskInfo* pInfo = param; SStreamMeta* pMeta = pInfo->pMeta; @@ -678,8 +636,8 @@ static void tryLaunchHistoryTask(void* param, void* tmrId) { int8_t ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1); streamMetaReleaseTask(pMeta, pTask); - stError("s-task:%s max retry:%d reached, quit from retrying launch related fill-history task timer, ref:%d", - pTask->id.idStr, MAX_RETRY_LAUNCH_HISTORY_TASK, ref); + stError("s-task:%s max retry:%d reached, quit from retrying launch related fill-history task:0x%x, ref:%d", + pTask->id.idStr, MAX_RETRY_LAUNCH_HISTORY_TASK, (int32_t)pHTaskInfo->id.taskId, ref); pHTaskInfo->id.taskId = 0; pHTaskInfo->id.streamId = 0; From 84772712409308b948b7902ef4aff0e4ded178a7 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 27 Sep 2023 17:15:49 +0800 Subject: [PATCH 110/158] fix(stream): fix dead lock. --- source/libs/stream/src/streamBackendRocksdb.c | 1 + source/libs/stream/src/streamCheckpoint.c | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index f538e3e427..6301ee7a91 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -959,6 +959,7 @@ int32_t streamBackendDoCheckpoint(void* arg, uint64_t checkpointId) { SBackendWrapper* pHandle = taosAcquireRef(streamBackendId, backendRid); if (pHandle == NULL || pHandle->db == NULL) { + stError("failed to acquire state-backend handle"); goto _ERROR; } diff --git a/source/libs/stream/src/streamCheckpoint.c b/source/libs/stream/src/streamCheckpoint.c index 997fecbba9..a87901eb47 100644 --- a/source/libs/stream/src/streamCheckpoint.c +++ b/source/libs/stream/src/streamCheckpoint.c @@ -321,7 +321,7 @@ int32_t streamTaskBuildCheckpoint(SStreamTask* pTask) { pTask->chkInfo.startTs = 0; // clear the recorded start time if (remain == 0) { // all tasks are ready - stDebug("s-task:%s is ready for checkpoint", pTask->id.idStr); + stDebug("s-task:%s all downstreams are ready, ready for do checkpoint", pTask->id.idStr); streamBackendDoCheckpoint(pMeta, pTask->checkpointingId); streamSaveAllTaskStatus(pMeta, pTask->checkpointingId); stInfo( From 0a2b70fd7f99aa9f65ef73c110ca98e5601f9756 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 28 Sep 2023 00:53:36 +0800 Subject: [PATCH 111/158] fix(stream): fix syntax error. --- source/libs/stream/src/streamQueue.c | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/source/libs/stream/src/streamQueue.c b/source/libs/stream/src/streamQueue.c index dc0eb949da..65400386b1 100644 --- a/source/libs/stream/src/streamQueue.c +++ b/source/libs/stream/src/streamQueue.c @@ -401,14 +401,14 @@ static void fillTokenBucket(STokenBucket* pBucket) { int32_t incNum = (delta / 1000.0) * pBucket->numRate; if (incNum > 0) { - pBucket->numOfToken = MIN(pBucket->numOfToken + incNum, pBucket->numCapacity); + pBucket->numOfToken = TMIN(pBucket->numOfToken + incNum, pBucket->numCapacity); pBucket->fillTimestamp = now; } // increase the new available quota as time goes on double incSize = (delta / 1000.0) * pBucket->bytesRate; if (incSize > 0) { - pBucket->bytesRemain = MIN(pBucket->bytesRemain + incSize, pBucket->bytesCapacity); + pBucket->bytesRemain = TMIN(pBucket->bytesRemain + incSize, pBucket->bytesCapacity); } if (incNum > 0) { @@ -434,7 +434,7 @@ bool streamTaskExtractAvailableToken(STokenBucket* pBucket) { } void streamTaskPutbackToken(STokenBucket* pBucket) { - pBucket->numOfToken = MIN(pBucket->numOfToken + 1, pBucket->numCapacity); + pBucket->numOfToken = TMIN(pBucket->numOfToken + 1, pBucket->numCapacity); } // size in KB From db897fb03a3e6059311859a937bcbff0d3115960 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 28 Sep 2023 09:25:23 +0800 Subject: [PATCH 112/158] fix(stream): opt perf. --- include/libs/stream/tstream.h | 2 +- source/dnode/vnode/src/tq/tq.c | 5 ++--- source/libs/stream/src/stream.c | 3 +-- source/libs/stream/src/streamTask.c | 21 +++++++++++++-------- 4 files changed, 17 insertions(+), 14 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 61d74b5809..82fefc7e1c 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -691,7 +691,7 @@ int32_t streamTaskLaunchScanHistory(SStreamTask* pTask); int32_t streamTaskCheckStatus(SStreamTask* pTask, int32_t upstreamTaskId, int32_t vgId, int64_t stage); int32_t streamTaskUpdateEpsetInfo(SStreamTask* pTask, SArray* pNodeList); void streamTaskResetUpstreamStageInfo(SStreamTask* pTask); -int8_t streamTaskSetSchedStatusWait(SStreamTask* pTask); +bool streamTaskSetSchedStatusWait(SStreamTask* pTask); int8_t streamTaskSetSchedStatusActive(SStreamTask* pTask); int8_t streamTaskSetSchedStatusInActive(SStreamTask* pTask); diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 22f0f256ea..3b11b5b764 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1066,12 +1066,11 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { } // we have to continue retrying to successfully execute the scan history task. - int8_t schedStatus = streamTaskSetSchedStatusWait(pTask); - if (schedStatus != TASK_SCHED_STATUS__INACTIVE) { + if (!streamTaskSetSchedStatusWait(pTask)) { tqError( "s-task:%s failed to start scan-history in first stream time window since already started, unexpected " "sched-status:%d", - id, schedStatus); + id, pTask->status.schedStatus); streamMetaReleaseTask(pMeta, pTask); return 0; } diff --git a/source/libs/stream/src/stream.c b/source/libs/stream/src/stream.c index 2abd3bac05..6f9a577a46 100644 --- a/source/libs/stream/src/stream.c +++ b/source/libs/stream/src/stream.c @@ -108,8 +108,7 @@ int32_t streamSetupScheduleTrigger(SStreamTask* pTask) { } int32_t streamSchedExec(SStreamTask* pTask) { - int8_t schedStatus = streamTaskSetSchedStatusWait(pTask); - if (schedStatus == TASK_SCHED_STATUS__INACTIVE) { + if (streamTaskSetSchedStatusWait(pTask)) { SStreamTaskRunReq* pRunReq = rpcMallocCont(sizeof(SStreamTaskRunReq)); if (pRunReq == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; diff --git a/source/libs/stream/src/streamTask.c b/source/libs/stream/src/streamTask.c index e77ab16040..c1ffcda8a5 100644 --- a/source/libs/stream/src/streamTask.c +++ b/source/libs/stream/src/streamTask.c @@ -624,15 +624,20 @@ void streamTaskResetUpstreamStageInfo(SStreamTask* pTask) { stDebug("s-task:%s reset all upstream tasks stage info", pTask->id.idStr); } -int8_t streamTaskSetSchedStatusWait(SStreamTask* pTask) { - taosThreadMutexLock(&pTask->lock); - int8_t status = pTask->status.schedStatus; - if (status == TASK_SCHED_STATUS__INACTIVE) { - pTask->status.schedStatus = TASK_SCHED_STATUS__WAITING; - } - taosThreadMutexUnlock(&pTask->lock); +bool streamTaskSetSchedStatusWait(SStreamTask* pTask) { + bool ret = false; - return status; + // double check + if (pTask->status.schedStatus == TASK_SCHED_STATUS__INACTIVE) { + taosThreadMutexLock(&pTask->lock); + if (pTask->status.schedStatus == TASK_SCHED_STATUS__INACTIVE) { + pTask->status.schedStatus = TASK_SCHED_STATUS__WAITING; + ret = true; + } + taosThreadMutexUnlock(&pTask->lock); + } + + return ret; } int8_t streamTaskSetSchedStatusActive(SStreamTask* pTask) { From c759d90bf69ba92d2d45631735fc9fab57ad7f3c Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 28 Sep 2023 11:23:03 +0800 Subject: [PATCH 113/158] enh(stream): optimize the stream status when creating without fill-history option. --- include/libs/stream/tstream.h | 4 +- source/dnode/mnode/impl/src/mndScheduler.c | 67 +++++++++++----------- source/dnode/mnode/impl/src/mndStream.c | 6 +- source/libs/stream/src/streamRecover.c | 20 ++++--- source/libs/stream/src/streamTask.c | 10 +++- 5 files changed, 58 insertions(+), 49 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 82fefc7e1c..88ba7f995a 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -450,8 +450,8 @@ typedef struct SStreamMeta { int32_t tEncodeStreamEpInfo(SEncoder* pEncoder, const SStreamChildEpInfo* pInfo); int32_t tDecodeStreamEpInfo(SDecoder* pDecoder, SStreamChildEpInfo* pInfo); -SStreamTask* tNewStreamTask(int64_t streamId, int8_t taskLevel, int8_t fillHistory, int64_t triggerParam, - SArray* pTaskList); +SStreamTask* tNewStreamTask(int64_t streamId, int8_t taskLevel, bool fillHistory, int64_t triggerParam, + SArray* pTaskList, bool hasFillhistory); int32_t tEncodeStreamTask(SEncoder* pEncoder, const SStreamTask* pTask); int32_t tDecodeStreamTask(SDecoder* pDecoder, SStreamTask* pTask); void tFreeStreamTask(SStreamTask* pTask); diff --git a/source/dnode/mnode/impl/src/mndScheduler.c b/source/dnode/mnode/impl/src/mndScheduler.c index 62d5ff47e3..a13c6371b1 100644 --- a/source/dnode/mnode/impl/src/mndScheduler.c +++ b/source/dnode/mnode/impl/src/mndScheduler.c @@ -27,8 +27,8 @@ #define SINK_NODE_LEVEL (0) extern bool tsDeployOnSnode; -static int32_t mndAddSinkTaskToStream(SStreamObj* pStream, SArray* pTaskList, SMnode* pMnode, int32_t vgId, - SVgObj* pVgroup, SEpSet* pEpset, int32_t fillHistory); +static int32_t doAddSinkTask(SStreamObj* pStream, SArray* pTaskList, SMnode* pMnode, int32_t vgId, SVgObj* pVgroup, + SEpSet* pEpset, bool isFillhistory); int32_t mndConvertRsmaTask(char** pDst, int32_t* pDstLen, const char* ast, int64_t uid, int8_t triggerType, int64_t watermark, int64_t deleteMark) { @@ -207,8 +207,7 @@ SVgObj* mndSchedFetchOneVg(SMnode* pMnode, int64_t dbUid) { } // create sink node for each vgroup. -int32_t mndAddShuffleSinkTasksToStream(SMnode* pMnode, SArray* pTaskList, SStreamObj* pStream, SEpSet* pEpset, - int32_t fillHistory) { +int32_t doAddShuffleSinkTask(SMnode* pMnode, SArray* pTaskList, SStreamObj* pStream, SEpSet* pEpset, bool fillHistory) { SSdb* pSdb = pMnode->pSdb; void* pIter = NULL; @@ -224,17 +223,17 @@ int32_t mndAddShuffleSinkTasksToStream(SMnode* pMnode, SArray* pTaskList, SStrea continue; } - mndAddSinkTaskToStream(pStream, pTaskList, pMnode, pVgroup->vgId, pVgroup, pEpset, fillHistory); + doAddSinkTask(pStream, pTaskList, pMnode, pVgroup->vgId, pVgroup, pEpset, fillHistory); sdbRelease(pSdb, pVgroup); } return 0; } -int32_t mndAddSinkTaskToStream(SStreamObj* pStream, SArray* pTaskList, SMnode* pMnode, int32_t vgId, SVgObj* pVgroup, - SEpSet* pEpset, int32_t fillHistory) { - int64_t uid = (fillHistory == 0)? pStream->uid:pStream->hTaskUid; - SStreamTask* pTask = tNewStreamTask(uid, TASK_LEVEL__SINK, fillHistory, 0, pTaskList); +int32_t doAddSinkTask(SStreamObj* pStream, SArray* pTaskList, SMnode* pMnode, int32_t vgId, SVgObj* pVgroup, + SEpSet* pEpset, bool isFillhistory) { + int64_t uid = (isFillhistory)? pStream->uid:pStream->hTaskUid; + SStreamTask* pTask = tNewStreamTask(uid, TASK_LEVEL__SINK, isFillhistory, 0, pTaskList, pStream->conf.fillHistory); if (pTask == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; return -1; @@ -248,17 +247,16 @@ int32_t mndAddSinkTaskToStream(SStreamObj* pStream, SArray* pTaskList, SMnode* p return 0; } -static int32_t addSourceStreamTask(SMnode* pMnode, SVgObj* pVgroup, SArray* pTaskList, SArray* pSinkTaskList, - SStreamObj* pStream, SSubplan* plan, uint64_t uid, SEpSet* pEpset, - int8_t fillHistory, bool hasExtraSink, int64_t firstWindowSkey) { - SStreamTask* pTask = tNewStreamTask(uid, TASK_LEVEL__SOURCE, fillHistory, pStream->conf.triggerParam, pTaskList); +static int32_t addSourceTask(SMnode* pMnode, SVgObj* pVgroup, SArray* pTaskList, SArray* pSinkTaskList, + SStreamObj* pStream, SSubplan* plan, uint64_t uid, SEpSet* pEpset, bool fillHistory, + bool hasExtraSink, int64_t firstWindowSkey, bool hasFillHistory) { + SStreamTask* pTask = + tNewStreamTask(uid, TASK_LEVEL__SOURCE, fillHistory, pStream->conf.triggerParam, pTaskList, hasFillHistory); if (pTask == NULL) { return terrno; } epsetAssign(&pTask->info.mnodeEpset, pEpset); - - // todo set the correct ts, which should be last key of queried table. STimeWindow* pWindow = &pTask->dataRange.window; pWindow->skey = INT64_MIN; @@ -345,8 +343,8 @@ static int32_t addSourceTasksForOneLevelStream(SMnode* pMnode, const SQueryPlan* // new stream task SArray** pSinkTaskList = taosArrayGet(pStream->tasks, SINK_NODE_LEVEL); - int32_t code = addSourceStreamTask(pMnode, pVgroup, pTaskList, *pSinkTaskList, pStream, plan, pStream->uid, pEpset, - 0, hasExtraSink, nextWindowSkey); + int32_t code = addSourceTask(pMnode, pVgroup, pTaskList, *pSinkTaskList, pStream, plan, pStream->uid, pEpset, + false, hasExtraSink, nextWindowSkey, pStream->conf.fillHistory); if (code != TSDB_CODE_SUCCESS) { sdbRelease(pSdb, pVgroup); return -1; @@ -354,8 +352,8 @@ static int32_t addSourceTasksForOneLevelStream(SMnode* pMnode, const SQueryPlan* if (pStream->conf.fillHistory) { SArray** pHSinkTaskList = taosArrayGet(pStream->pHTasksList, SINK_NODE_LEVEL); - code = addSourceStreamTask(pMnode, pVgroup, pHTaskList, *pHSinkTaskList, pStream, plan, pStream->hTaskUid, - pEpset, 1, hasExtraSink, nextWindowSkey); + code = addSourceTask(pMnode, pVgroup, pHTaskList, *pHSinkTaskList, pStream, plan, pStream->hTaskUid, + pEpset, true, hasExtraSink, nextWindowSkey, false); } sdbRelease(pSdb, pVgroup); @@ -371,10 +369,10 @@ static int32_t addSourceTasksForOneLevelStream(SMnode* pMnode, const SQueryPlan* return TSDB_CODE_SUCCESS; } -static int32_t doAddSourceTask(SArray* pTaskList, int8_t fillHistory, int64_t uid, SStreamTask* pDownstreamTask, +static int32_t doAddSourceTask(SArray* pTaskList, bool isFillhistory, int64_t uid, SStreamTask* pDownstreamTask, SMnode* pMnode, SSubplan* pPlan, SVgObj* pVgroup, SEpSet* pEpset, - int64_t nextWindowSkey) { - SStreamTask* pTask = tNewStreamTask(uid, TASK_LEVEL__SOURCE, fillHistory, 0, pTaskList); + int64_t nextWindowSkey, bool hasFillHistory) { + SStreamTask* pTask = tNewStreamTask(uid, TASK_LEVEL__SOURCE, isFillhistory, 0, pTaskList, hasFillHistory); if (pTask == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; return -1; @@ -400,8 +398,8 @@ static int32_t doAddSourceTask(SArray* pTaskList, int8_t fillHistory, int64_t ui } static int32_t doAddAggTask(uint64_t uid, SArray* pTaskList, SArray* pSinkNodeList, SMnode* pMnode, SStreamObj* pStream, - SEpSet* pEpset, int32_t fillHistory, SStreamTask** pAggTask) { - *pAggTask = tNewStreamTask(uid, TASK_LEVEL__AGG, fillHistory, pStream->conf.triggerParam, pTaskList); + SEpSet* pEpset, bool fillHistory, SStreamTask** pAggTask, bool hasFillhistory) { + *pAggTask = tNewStreamTask(uid, TASK_LEVEL__AGG, fillHistory, pStream->conf.triggerParam, pTaskList, hasFillhistory); if (*pAggTask == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; return -1; @@ -432,7 +430,8 @@ static int32_t addAggTask(SStreamObj* pStream, SMnode* pMnode, SQueryPlan* pPlan *pAggTask = NULL; SArray* pSinkNodeList = taosArrayGetP(pStream->tasks, SINK_NODE_LEVEL); - int32_t code = doAddAggTask(pStream->uid, pAggTaskList, pSinkNodeList, pMnode, pStream, pEpset, 0, pAggTask); + int32_t code = doAddAggTask(pStream->uid, pAggTaskList, pSinkNodeList, pMnode, pStream, pEpset, false, pAggTask, + pStream->conf.fillHistory); if (code != TSDB_CODE_SUCCESS) { return -1; } @@ -461,7 +460,7 @@ static int32_t addAggTask(SStreamObj* pStream, SMnode* pMnode, SQueryPlan* pPlan *pHAggTask = NULL; code = doAddAggTask(pStream->hTaskUid, pHAggTaskList, pHSinkNodeList, pMnode, pStream, pEpset, pStream->conf.fillHistory, - pHAggTask); + pHAggTask, false); if (code != TSDB_CODE_SUCCESS) { if (pSnode != NULL) { sdbRelease(pSdb, pSnode); @@ -520,8 +519,8 @@ static int32_t addSourceTasksForMultiLevelStream(SMnode* pMnode, SQueryPlan* pPl continue; } - int32_t code = - doAddSourceTask(pSourceTaskList, 0, pStream->uid, pDownstreamTask, pMnode, plan, pVgroup, pEpset, nextWindowSkey); + int32_t code = doAddSourceTask(pSourceTaskList, false, pStream->uid, pDownstreamTask, pMnode, plan, pVgroup, pEpset, + nextWindowSkey, pStream->conf.fillHistory); if (code != TSDB_CODE_SUCCESS) { sdbRelease(pSdb, pVgroup); terrno = code; @@ -529,8 +528,8 @@ static int32_t addSourceTasksForMultiLevelStream(SMnode* pMnode, SQueryPlan* pPl } if (pStream->conf.fillHistory) { - code = doAddSourceTask(pHSourceTaskList, 1, pStream->hTaskUid, pHDownstreamTask, pMnode, plan, pVgroup, - pEpset, nextWindowSkey); + code = doAddSourceTask(pHSourceTaskList, true, pStream->hTaskUid, pHDownstreamTask, pMnode, plan, pVgroup, pEpset, + nextWindowSkey, false); if (code != TSDB_CODE_SUCCESS) { sdbRelease(pSdb, pVgroup); return code; @@ -548,16 +547,16 @@ static int32_t addSourceTasksForMultiLevelStream(SMnode* pMnode, SQueryPlan* pPl } static int32_t addSinkTasks(SArray* pTasksList, SMnode* pMnode, SStreamObj* pStream, SArray** pCreatedTaskList, - SEpSet* pEpset, int32_t fillHistory) { + SEpSet* pEpset, bool fillHistory) { SArray* pSinkTaskList = addNewTaskList(pTasksList); if (pStream->fixedSinkVgId == 0) { - if (mndAddShuffleSinkTasksToStream(pMnode, pSinkTaskList, pStream, pEpset, fillHistory) < 0) { + if (doAddShuffleSinkTask(pMnode, pSinkTaskList, pStream, pEpset, fillHistory) < 0) { // TODO free return -1; } } else { - if (mndAddSinkTaskToStream(pStream, pSinkTaskList, pMnode, pStream->fixedSinkVgId, &pStream->fixedSinkVg, - pEpset, fillHistory) < 0) { + if (doAddSinkTask(pStream, pSinkTaskList, pMnode, pStream->fixedSinkVgId, &pStream->fixedSinkVg, pEpset, + fillHistory) < 0) { // TODO free return -1; } diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index 57d14d000b..8cbdeff19a 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -2320,8 +2320,9 @@ static int32_t mndProcessNodeCheck(SRpcMsg *pReq) { } SMStreamNodeCheckMsg *pMsg = rpcMallocCont(sizeof(SMStreamNodeCheckMsg)); - SRpcMsg rpcMsg = { - .msgType = TDMT_MND_STREAM_NODECHANGE_CHECK, .pCont = pMsg, .contLen = sizeof(SMStreamNodeCheckMsg)}; + + SRpcMsg rpcMsg = { + .msgType = TDMT_MND_STREAM_NODECHANGE_CHECK, .pCont = pMsg, .contLen = sizeof(SMStreamNodeCheckMsg)}; tmsgPutToQueue(&pMnode->msgCb, WRITE_QUEUE, &rpcMsg); return 0; } @@ -2383,7 +2384,6 @@ void removeStreamTasksInBuf(SStreamObj* pStream, SStreamExecNodeInfo * pExecNode ASSERT(taosHashGetSize(pExecNode->pTaskMap) == taosArrayGetSize(pExecNode->pTaskList)); } -// todo: this process should be executed by the write queue worker of the mnode int32_t mndProcessStreamHb(SRpcMsg *pReq) { SMnode *pMnode = pReq->info.node; SStreamHbMsg req = {0}; diff --git a/source/libs/stream/src/streamRecover.c b/source/libs/stream/src/streamRecover.c index 03a5233023..f31a16ec85 100644 --- a/source/libs/stream/src/streamRecover.c +++ b/source/libs/stream/src/streamRecover.c @@ -66,7 +66,7 @@ static void streamTaskSetReady(SStreamTask* pTask, int32_t numOfReqs) { pMeta->startInfo.startedAfterNodeUpdate = 0; pMeta->startInfo.elapsedTime = pTask->execInfo.start - pMeta->startInfo.ts; - stDebug("vgId:%d all %d task(s) are started successfully, last ready task:%s level:%d, total elapsed time:%.2f sec", + stDebug("vgId:%d all %d task(s) are started successfully, last ready task:%s level:%d, total elapsed time:%.2fs", vgId, numOfTotal, pTask->id.idStr, pTask->info.taskLevel, pMeta->startInfo.elapsedTime / 1000.0); } taosWUnLockLatch(&pMeta->lock); @@ -580,15 +580,21 @@ int32_t streamProcessScanHistoryFinishRsp(SStreamTask* pTask) { } static void checkFillhistoryTaskStatus(SStreamTask* pTask, SStreamTask* pHTask) { - pHTask->dataRange.range.minVer = 0; - // the query version range should be limited to the already processed data - pHTask->dataRange.range.maxVer = pTask->chkInfo.nextProcessVer - 1; + SDataRange* pRange = &pHTask->dataRange; + pRange->range.minVer = 0; + // the query version range should be limited to the already processed data + pRange->range.maxVer = pTask->chkInfo.nextProcessVer - 1; + if (pRange->range.maxVer < pRange->range.minVer) { + pRange->range.maxVer = pRange->range.minVer; + } + + pHTask->execInfo.init = taosGetTimestampMs(); if (pTask->info.taskLevel == TASK_LEVEL__SOURCE) { stDebug("s-task:%s set the launch condition for fill-history s-task:%s, window:%" PRId64 " - %" PRId64 - " ver range:%" PRId64 " - %" PRId64, - pTask->id.idStr, pHTask->id.idStr, pHTask->dataRange.window.skey, pHTask->dataRange.window.ekey, - pHTask->dataRange.range.minVer, pHTask->dataRange.range.maxVer); + " ver range:%" PRId64 " - %" PRId64", init:%"PRId64, + pTask->id.idStr, pHTask->id.idStr, pRange->window.skey, pRange->window.ekey, + pRange->range.minVer, pRange->range.maxVer, pHTask->execInfo.init); } else { stDebug("s-task:%s no fill history condition for non-source task:%s", pTask->id.idStr, pHTask->id.idStr); } diff --git a/source/libs/stream/src/streamTask.c b/source/libs/stream/src/streamTask.c index c1ffcda8a5..2907923d03 100644 --- a/source/libs/stream/src/streamTask.c +++ b/source/libs/stream/src/streamTask.c @@ -27,8 +27,8 @@ static int32_t addToTaskset(SArray* pArray, SStreamTask* pTask) { return 0; } -SStreamTask* tNewStreamTask(int64_t streamId, int8_t taskLevel, int8_t fillHistory, int64_t triggerParam, - SArray* pTaskList) { +SStreamTask* tNewStreamTask(int64_t streamId, int8_t taskLevel, bool fillHistory, int64_t triggerParam, + SArray* pTaskList, bool hasFillhistory) { SStreamTask* pTask = (SStreamTask*)taosMemoryCalloc(1, sizeof(SStreamTask)); if (pTask == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; @@ -46,10 +46,14 @@ SStreamTask* tNewStreamTask(int64_t streamId, int8_t taskLevel, int8_t fillHisto pTask->id.idStr = taosStrdup(buf); pTask->status.schedStatus = TASK_SCHED_STATUS__INACTIVE; - pTask->status.taskStatus = TASK_STATUS__SCAN_HISTORY; + pTask->status.taskStatus = (fillHistory || hasFillhistory)? TASK_STATUS__SCAN_HISTORY:TASK_STATUS__NORMAL; pTask->inputInfo.status = TASK_INPUT_STATUS__NORMAL; pTask->outputInfo.status = TASK_OUTPUT_STATUS__NORMAL; + if (fillHistory) { + ASSERT(hasFillhistory); + } + addToTaskset(pTaskList, pTask); return pTask; } From 6fbcf4b3d71e80e44b2f85b7030380880bbc0ab3 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 28 Sep 2023 11:42:19 +0800 Subject: [PATCH 114/158] fix(tsdb): replace destroy with reset. --- source/dnode/vnode/src/tsdb/tsdbRead2.c | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/source/dnode/vnode/src/tsdb/tsdbRead2.c b/source/dnode/vnode/src/tsdb/tsdbRead2.c index 9299264136..be77d4fd62 100644 --- a/source/dnode/vnode/src/tsdb/tsdbRead2.c +++ b/source/dnode/vnode/src/tsdb/tsdbRead2.c @@ -4605,14 +4605,14 @@ static SSDataBlock* doRetrieveDataBlock(STsdbReader* pReader) { code = doLoadFileBlockData(pReader, &pStatus->blockIter, &pStatus->fileBlockData, pBlockScanInfo->uid); if (code != TSDB_CODE_SUCCESS) { - tBlockDataDestroy(&pStatus->fileBlockData); + tBlockDataReset(&pStatus->fileBlockData); terrno = code; return NULL; } code = copyBlockDataToSDataBlock(pReader); if (code != TSDB_CODE_SUCCESS) { - tBlockDataDestroy(&pStatus->fileBlockData); + tBlockDataReset(&pStatus->fileBlockData); terrno = code; return NULL; } From bb4ba54f28ce80ebffbd49959ba9602823fd4881 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 28 Sep 2023 18:26:52 +0800 Subject: [PATCH 115/158] fix(stream): enable new time range for stream task. --- include/common/tmsg.h | 7 ++ include/common/tmsgdef.h | 1 + include/libs/stream/tstream.h | 2 + source/dnode/mgmt/mgmt_vnode/src/vmHandle.c | 1 + source/dnode/mnode/impl/src/mndScheduler.c | 2 +- source/dnode/snode/src/snode.c | 3 + source/dnode/vnode/src/inc/vnodeInt.h | 1 + source/dnode/vnode/src/tq/tq.c | 76 ++++++++++++++++----- source/dnode/vnode/src/vnd/vnodeSvr.c | 9 ++- source/libs/stream/src/streamMeta.c | 41 +++++------ source/libs/stream/src/streamRecover.c | 25 ++++--- source/libs/stream/src/streamTask.c | 58 ++++++++++++++++ 12 files changed, 176 insertions(+), 50 deletions(-) diff --git a/include/common/tmsg.h b/include/common/tmsg.h index bb843ced91..0c0d28e4b7 100644 --- a/include/common/tmsg.h +++ b/include/common/tmsg.h @@ -3040,6 +3040,13 @@ typedef struct { int32_t taskId; } SVDropStreamTaskReq; +typedef struct { + SMsgHead head; + int64_t streamId; + int32_t taskId; + int64_t dataVer; +} SVStreamTaskVerUpdateReq; + typedef struct { int8_t reserved; } SVDropStreamTaskRsp; diff --git a/include/common/tmsgdef.h b/include/common/tmsgdef.h index fb2c780724..279cf72f0b 100644 --- a/include/common/tmsgdef.h +++ b/include/common/tmsgdef.h @@ -308,6 +308,7 @@ enum { // WARN: new msg should be appended to segment tail TD_DEF_MSG_TYPE(TDMT_VND_STREAM_CHECK_POINT_SOURCE, "vnode-stream-checkpoint-source", NULL, NULL) TD_DEF_MSG_TYPE(TDMT_VND_STREAM_TASK_UPDATE, "vnode-stream-update", NULL, NULL) TD_DEF_MSG_TYPE(TDMT_VND_STREAM_TASK_CHECK, "vnode-stream-task-check", NULL, NULL) + TD_DEF_MSG_TYPE(TDMT_STREAM_TASK_VERUPDATE, "vnode-stream-ver-update", NULL, NULL) TD_DEF_MSG_TYPE(TDMT_VND_STREAM_MAX_MSG, "vnd-stream-max", NULL, NULL) TD_NEW_MSG_SEG(TDMT_VND_TMQ_MSG) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 88ba7f995a..40b2be72bc 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -723,6 +723,7 @@ int32_t streamTaskReleaseState(SStreamTask* pTask); int32_t streamTaskReloadState(SStreamTask* pTask); void streamTaskCloseUpstreamInput(SStreamTask* pTask, int32_t taskId); void streamTaskOpenAllUpstreamInput(SStreamTask* pTask); +int32_t streamTaskUpdateDataVer(SStreamTask* pTask, int64_t ver); // source level int32_t streamSetParamForStreamScannerStep1(SStreamTask* pTask, SVersionRange* pVerRange, STimeWindow* pWindow); @@ -761,6 +762,7 @@ int32_t streamProcessCheckpointReadyMsg(SStreamTask* pTask); int32_t streamAlignTransferState(SStreamTask* pTask); int32_t streamBuildAndSendDropTaskMsg(SMsgCb* pMsgCb, int32_t vgId, SStreamTaskId* pTaskId); +int32_t streamBuildAndSendVerUpdateMsg(SMsgCb* pMsgCb, int32_t vgId, SStreamTaskId* pTaskId, int64_t ver); int32_t streamAddCheckpointSourceRspMsg(SStreamCheckpointSourceReq* pReq, SRpcHandleInfo* pRpcInfo, SStreamTask* pTask, int8_t isSucceed); int32_t buildCheckpointSourceRsp(SStreamCheckpointSourceReq* pReq, SRpcHandleInfo* pRpcInfo, SRpcMsg* pMsg, diff --git a/source/dnode/mgmt/mgmt_vnode/src/vmHandle.c b/source/dnode/mgmt/mgmt_vnode/src/vmHandle.c index 0251b9b636..9d6b18c677 100644 --- a/source/dnode/mgmt/mgmt_vnode/src/vmHandle.c +++ b/source/dnode/mgmt/mgmt_vnode/src/vmHandle.c @@ -801,6 +801,7 @@ SArray *vmGetMsgHandles() { if (dmSetMgmtHandle(pArray, TDMT_VND_STREAM_CHECK_POINT_SOURCE, vmPutMsgToWriteQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_STREAM_TASK_CHECKPOINT_READY, vmPutMsgToStreamQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_VND_STREAM_TASK_UPDATE, vmPutMsgToWriteQueue, 0) == NULL) goto _OVER; + if (dmSetMgmtHandle(pArray, TDMT_STREAM_TASK_VERUPDATE, vmPutMsgToWriteQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_VND_ALTER_REPLICA, vmPutMsgToMgmtQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_VND_ALTER_CONFIG, vmPutMsgToWriteQueue, 0) == NULL) goto _OVER; diff --git a/source/dnode/mnode/impl/src/mndScheduler.c b/source/dnode/mnode/impl/src/mndScheduler.c index a13c6371b1..f2e783d64f 100644 --- a/source/dnode/mnode/impl/src/mndScheduler.c +++ b/source/dnode/mnode/impl/src/mndScheduler.c @@ -232,7 +232,7 @@ int32_t doAddShuffleSinkTask(SMnode* pMnode, SArray* pTaskList, SStreamObj* pStr int32_t doAddSinkTask(SStreamObj* pStream, SArray* pTaskList, SMnode* pMnode, int32_t vgId, SVgObj* pVgroup, SEpSet* pEpset, bool isFillhistory) { - int64_t uid = (isFillhistory)? pStream->uid:pStream->hTaskUid; + int64_t uid = (isFillhistory)? pStream->hTaskUid:pStream->uid; SStreamTask* pTask = tNewStreamTask(uid, TASK_LEVEL__SINK, isFillhistory, 0, pTaskList, pStream->conf.fillHistory); if (pTask == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; diff --git a/source/dnode/snode/src/snode.c b/source/dnode/snode/src/snode.c index dbbd68fa08..799c784f38 100644 --- a/source/dnode/snode/src/snode.c +++ b/source/dnode/snode/src/snode.c @@ -177,6 +177,9 @@ int32_t sndProcessTaskDeployReq(SSnode *pSnode, char *msg, int32_t msgLen) { qDebug("snode:%d s-task:%s is deployed on snode and add into meta, status:%s, numOfTasks:%d", SNODE_HANDLE, pTask->id.idStr, streamGetTaskStatusStr(pTask->status.taskStatus), numOfTasks); + // send msg to update the nextProcessedVer attribute for this task if it is a stream task + streamBuildAndSendVerUpdateMsg(pTask->pMsgCb, pSnode->pMeta->vgId, &pTask->id, 0); + streamTaskCheckDownstream(pTask); return 0; } diff --git a/source/dnode/vnode/src/inc/vnodeInt.h b/source/dnode/vnode/src/inc/vnodeInt.h index 3a62f52bdd..1e07c87cb2 100644 --- a/source/dnode/vnode/src/inc/vnodeInt.h +++ b/source/dnode/vnode/src/inc/vnodeInt.h @@ -227,6 +227,7 @@ int tqScanWalAsync(STQ* pTq, bool ckPause); int32_t tqProcessStreamCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg* pRsp); int32_t tqProcessTaskCheckpointReadyMsg(STQ* pTq, SRpcMsg* pMsg); int32_t tqProcessTaskUpdateReq(STQ* pTq, SRpcMsg* pMsg); +int32_t tqProcessTaskDataVerUpdateReq(STQ* pTq, char* pMsg, int32_t msgLen); int32_t tqCheckAndRunStreamTaskAsync(STQ* pTq); int tqCommit(STQ*); diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 3b11b5b764..b523faec7f 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -964,18 +964,17 @@ int32_t tqProcessTaskDeployReq(STQ* pTq, int64_t sversion, char* msg, int32_t ms if (tsDisableStream) { tqInfo("vgId:%d stream disabled, not deploy stream tasks", vgId); - return 0; + return code; } tqDebug("vgId:%d receive new stream task deploy msg, start to build stream task", vgId); // 1.deserialize msg and build task - SStreamTask* pTask = taosMemoryCalloc(1, sizeof(SStreamTask)); + int32_t size = sizeof(SStreamTask); + SStreamTask* pTask = taosMemoryCalloc(1, size); if (pTask == NULL) { - terrno = TSDB_CODE_OUT_OF_MEMORY; - tqError("vgId:%d failed to create stream task due to out of memory, alloc size:%d", vgId, - (int32_t)sizeof(SStreamTask)); - return -1; + tqError("vgId:%d failed to create stream task due to out of memory, alloc size:%d", vgId, size); + return TSDB_CODE_OUT_OF_MEMORY; } SDecoder decoder; @@ -983,9 +982,9 @@ int32_t tqProcessTaskDeployReq(STQ* pTq, int64_t sversion, char* msg, int32_t ms code = tDecodeStreamTask(&decoder, pTask); tDecoderClear(&decoder); - if (code < 0) { + if (code != TSDB_CODE_SUCCESS) { taosMemoryFree(pTask); - return -1; + return TSDB_CODE_INVALID_MSG; } SStreamMeta* pStreamMeta = pTq->pStreamMeta; @@ -1001,9 +1000,9 @@ int32_t tqProcessTaskDeployReq(STQ* pTq, int64_t sversion, char* msg, int32_t ms taosWUnLockLatch(&pStreamMeta->lock); if (code < 0) { - tqError("vgId:%d failed to add s-task:0x%x, total:%d, code:%s", vgId, taskId, numOfTasks, tstrerror(code)); + tqError("failed to add s-task:0x%x into vgId:%d meta, total:%d, code:%s", vgId, taskId, numOfTasks, tstrerror(code)); tFreeStreamTask(pTask); - return -1; + return code; } // added into meta store, pTask cannot be reference since it may have been destroyed by other threads already now if @@ -1012,6 +1011,16 @@ int32_t tqProcessTaskDeployReq(STQ* pTq, int64_t sversion, char* msg, int32_t ms // only handled in the leader node if (vnodeIsRoleLeader(pTq->pVnode)) { tqDebug("vgId:%d s-task:0x%x is deployed and add into meta, numOfTasks:%d", vgId, taskId, numOfTasks); +#if 0 + if (pTq->pVnode->restored) { + SStreamTask* p = streamMetaAcquireTask(pStreamMeta, streamId, taskId); + if (p != NULL) { + // send msg to update the nextProcessedVer attribute for this task if it is a stream task + streamBuildAndSendVerUpdateMsg(p->pMsgCb, vgId, &p->id, sversion); + streamMetaReleaseTask(pStreamMeta, p); + } + } +#endif SStreamTask* p = streamMetaAcquireTask(pStreamMeta, streamId, taskId); bool restored = pTq->pVnode->restored; @@ -1035,7 +1044,7 @@ int32_t tqProcessTaskDeployReq(STQ* pTq, int64_t sversion, char* msg, int32_t ms tFreeStreamTask(pTask); } - return 0; + return code; } int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { @@ -1498,14 +1507,15 @@ int32_t tqProcessTaskResumeImpl(STQ* pTq, SStreamTask* pTask, int64_t sversion, int32_t tqProcessTaskResumeReq(STQ* pTq, int64_t sversion, char* msg, int32_t msgLen) { SVResumeStreamTaskReq* pReq = (SVResumeStreamTaskReq*)msg; - SStreamTask* pTask = streamMetaAcquireTask(pTq->pStreamMeta, pReq->streamId, pReq->taskId); - int32_t code = tqProcessTaskResumeImpl(pTq, pTask, sversion, pReq->igUntreated); + + SStreamTask* pTask = streamMetaAcquireTask(pTq->pStreamMeta, pReq->streamId, pReq->taskId); + int32_t code = tqProcessTaskResumeImpl(pTq, pTask, sversion, pReq->igUntreated); if (code != 0) { return code; } - SStreamTask* pHistoryTask = - streamMetaAcquireTask(pTq->pStreamMeta, pTask->hTaskInfo.id.streamId, pTask->hTaskInfo.id.taskId); + STaskId* pHTaskId = &pTask->hTaskInfo.id; + SStreamTask* pHistoryTask = streamMetaAcquireTask(pTq->pStreamMeta, pHTaskId->streamId, pHTaskId->taskId); if (pHistoryTask) { code = tqProcessTaskResumeImpl(pTq, pHistoryTask, sversion, pReq->igUntreated); } @@ -1524,9 +1534,11 @@ int32_t tqProcessTaskRetrieveReq(STQ* pTq, SRpcMsg* pMsg) { tDecodeStreamRetrieveReq(&decoder, &req); tDecoderClear(&decoder); + int32_t vgId = pTq->pStreamMeta->vgId; SStreamTask* pTask = streamMetaAcquireTask(pTq->pStreamMeta, req.streamId, req.dstTaskId); if (pTask == NULL) { - // tDeleteStreamDispatchReq(&req); + tqError("vgId:%d process retrieve req, failed to acquire task:0x%x, it may have been dropped already", vgId, + req.dstTaskId); return -1; } @@ -1887,3 +1899,35 @@ int32_t tqProcessTaskUpdateReq(STQ* pTq, SRpcMsg* pMsg) { taosArrayDestroy(req.pNodeList); return rsp.code; } + +int32_t tqProcessTaskDataVerUpdateReq(STQ* pTq, char* pMsg, int32_t msgLen) { + SStreamMeta* pMeta = pTq->pStreamMeta; + int32_t vgId = pMeta->vgId; + + SVStreamTaskVerUpdateReq* pReq = (SVStreamTaskVerUpdateReq*) pMsg; + tqDebug("vgId:%d receive msg to update task dataVer, task:0x%x dataVer:%" PRId64, vgId, pReq->taskId, pReq->dataVer); + + SStreamTask* pTask = streamMetaAcquireTask(pMeta, pReq->streamId, pReq->taskId); + if (pTask == NULL) { + tqError("vgId:%d process dataVer msg, failed to find task:0x%x, it may have been destroyed", vgId, pReq->taskId); + return -1; + } + + // commit the dataVer update + streamTaskUpdateDataVer(pTask, pReq->dataVer); + + if (vnodeIsLeader(pTq->pVnode)) { + if (pTq->pVnode->restored) { + ASSERT(pTask->execInfo.init == 0); + + pTask->execInfo.init = taosGetTimestampMs(); + tqDebug("s-task:%s set the init ts:%" PRId64, pTask->id.idStr, pTask->execInfo.init); + streamTaskCheckDownstream(pTask); + } else { + tqWarn("s-task:%s not launched since vnode (vgId:%d) not ready", pTask->id.idStr, vgId); + } + } + + streamMetaReleaseTask(pMeta, pTask); + return 0; +} diff --git a/source/dnode/vnode/src/vnd/vnodeSvr.c b/source/dnode/vnode/src/vnd/vnodeSvr.c index 97f484849c..7a1e60f075 100644 --- a/source/dnode/vnode/src/vnd/vnodeSvr.c +++ b/source/dnode/vnode/src/vnd/vnodeSvr.c @@ -468,7 +468,6 @@ int32_t vnodeProcessWriteMsg(SVnode *pVnode, SRpcMsg *pMsg, int64_t ver, SRpcMsg void *ptr = NULL; void *pReq; int32_t len; - int32_t ret; if (ver <= pVnode->state.applied) { vError("vgId:%d, duplicate write request. ver: %" PRId64 ", applied: %" PRId64 "", TD_VID(pVnode), ver, @@ -561,7 +560,9 @@ int32_t vnodeProcessWriteMsg(SVnode *pVnode, SRpcMsg *pMsg, int64_t ver, SRpcMsg } break; case TDMT_STREAM_TASK_DEPLOY: { - if (tqProcessTaskDeployReq(pVnode->pTq, ver, pReq, len) < 0) { + int32_t code = tqProcessTaskDeployReq(pVnode->pTq, ver, pReq, len); + if (code != TSDB_CODE_SUCCESS) { + terrno = code; goto _err; } } break; @@ -582,13 +583,15 @@ int32_t vnodeProcessWriteMsg(SVnode *pVnode, SRpcMsg *pMsg, int64_t ver, SRpcMsg goto _err; } } break; + case TDMT_STREAM_TASK_VERUPDATE: + tqProcessTaskDataVerUpdateReq(pVnode->pTq, pMsg->pCont, pMsg->contLen); + break; case TDMT_VND_ALTER_CONFIRM: needCommit = pVnode->config.hashChange; if (vnodeProcessAlterConfirmReq(pVnode, ver, pReq, len, pRsp) < 0) { goto _err; } break; - case TDMT_VND_ALTER_CONFIG: vnodeProcessAlterConfigReq(pVnode, ver, pReq, len, pRsp); break; diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index da1acc6965..840eee98a8 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -394,28 +394,28 @@ int32_t streamMetaRegisterTask(SStreamMeta* pMeta, int64_t ver, SStreamTask* pTa *pAdded = false; STaskId id = {.streamId = pTask->id.streamId, .taskId = pTask->id.taskId}; - void* p = taosHashGet(pMeta->pTasksMap, &id, sizeof(id)); - if (p == NULL) { - if (pMeta->expandFunc(pMeta->ahandle, pTask, ver) < 0) { - tFreeStreamTask(pTask); - return -1; - } - - taosArrayPush(pMeta->pTaskList, &pTask->id); - - if (streamMetaSaveTask(pMeta, pTask) < 0) { - tFreeStreamTask(pTask); - return -1; - } - - if (streamMetaCommit(pMeta) < 0) { - tFreeStreamTask(pTask); - return -1; - } - } else { + void* p = taosHashGet(pMeta->pTasksMap, &id, sizeof(id)); + if (p != NULL) { return 0; } + if (pMeta->expandFunc(pMeta->ahandle, pTask, ver) < 0) { + tFreeStreamTask(pTask); + return -1; + } + + taosArrayPush(pMeta->pTaskList, &pTask->id); + +// if (streamMetaSaveTask(pMeta, pTask) < 0) { +// tFreeStreamTask(pTask); +// return -1; +// } +// +// if (streamMetaCommit(pMeta) < 0) { +// tFreeStreamTask(pTask); +// return -1; +// } + taosHashPut(pMeta->pTasksMap, &id, sizeof(id), &pTask, POINTER_BYTES); if (pTask->info.fillHistory == 0) { atomic_add_fetch_32(&pMeta->numOfStreamTasks, 1); @@ -716,6 +716,9 @@ int32_t streamMetaLoadAllTasks(SStreamMeta* pMeta) { taosArrayPush(pMeta->pTaskList, &pTask->id); } else { + stError("s-task:0x%x already added into table meta by replaying WAL, need check", pTask->id.taskId); + ASSERT(0); + tdbFree(pKey); tdbFree(pVal); taosMemoryFree(pTask); diff --git a/source/libs/stream/src/streamRecover.c b/source/libs/stream/src/streamRecover.c index f31a16ec85..3e491fa1bf 100644 --- a/source/libs/stream/src/streamRecover.c +++ b/source/libs/stream/src/streamRecover.c @@ -119,7 +119,9 @@ int32_t streamTaskLaunchScanHistory(SStreamTask* pTask) { streamTaskEnablePause(pTask); } } else if (pTask->info.taskLevel == TASK_LEVEL__SINK) { - stDebug("s-task:%s sink task do nothing to handle scan-history", pTask->id.idStr); + if (pTask->status.taskStatus == TASK_STATUS__SCAN_HISTORY) { + stDebug("s-task:%s sink task do nothing to handle scan-history", pTask->id.idStr); + } } return 0; } @@ -144,10 +146,10 @@ static int32_t doCheckDownstreamStatus(SStreamTask* pTask) { req.downstreamTaskId = pTask->fixedDispatcher.taskId; pTask->checkReqId = req.reqId; - stDebug("s-task:%s stage:%" PRId64 " check single downstream task:0x%x(vgId:%d) ver:%" PRId64 "-%" PRId64 + stDebug("s-task:%s (vgId:%d) stage:%" PRId64 " check single downstream task:0x%x(vgId:%d) ver:%" PRId64 "-%" PRId64 " window:%" PRId64 "-%" PRId64 " req:0x%" PRIx64, - pTask->id.idStr, req.reqId, req.downstreamTaskId, req.downstreamNodeId, pRange->range.minVer, - pRange->range.maxVer, pWindow->skey, pWindow->ekey, req.reqId); + pTask->id.idStr, pTask->info.nodeId, req.stage, req.downstreamTaskId, req.downstreamNodeId, + pRange->range.minVer, pRange->range.maxVer, pWindow->skey, pWindow->ekey, req.reqId); streamSendCheckMsg(pTask, &req, pTask->fixedDispatcher.nodeId, &pTask->fixedDispatcher.epSet); } else if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { @@ -583,6 +585,7 @@ static void checkFillhistoryTaskStatus(SStreamTask* pTask, SStreamTask* pHTask) SDataRange* pRange = &pHTask->dataRange; pRange->range.minVer = 0; + // todo remove this // the query version range should be limited to the already processed data pRange->range.maxVer = pTask->chkInfo.nextProcessVer - 1; if (pRange->range.maxVer < pRange->range.minVer) { @@ -725,7 +728,7 @@ int32_t streamLaunchFillHistoryTask(SStreamTask* pTask) { if (pTask->hTaskInfo.pTimer == NULL) { int32_t ref = atomic_add_fetch_32(&pTask->status.timerActive, 1); pTask->hTaskInfo.pTimer = taosTmrStart(tryLaunchHistoryTask, WAIT_FOR_MINIMAL_INTERVAL, pInfo, streamEnv.timer); - if (pTask->hTaskInfo.pTimer == NULL) { // todo failed to create timer + if (pTask->hTaskInfo.pTimer == NULL) { atomic_sub_fetch_32(&pTask->status.timerActive, 1); stError("s-task:%s failed to start timer, related fill-history task not launched, ref:%d", pTask->id.idStr, pTask->status.timerActive); @@ -883,20 +886,20 @@ int32_t tDecodeStreamScanHistoryFinishReq(SDecoder* pDecoder, SStreamScanHistory } void streamTaskSetRangeStreamCalc(SStreamTask* pTask) { + SDataRange* pRange = &pTask->dataRange; + if (pTask->hTaskInfo.id.taskId == 0) { - SDataRange* pRange = &pTask->dataRange; if (pTask->info.fillHistory == 1) { stDebug("s-task:%s fill-history task, time window:%" PRId64 "-%" PRId64 ", verRange:%" PRId64 "-%" PRId64, pTask->id.idStr, pRange->window.skey, pRange->window.ekey, pRange->range.minVer, pRange->range.maxVer); } else { - stDebug("s-task:%s no related fill-history task, stream time window:%" PRId64 "-%" PRId64 ", verRange:%" PRId64 - "-%" PRId64, - pTask->id.idStr, pRange->window.skey, pRange->window.ekey, pRange->range.minVer, pRange->range.maxVer); + stDebug( + "s-task:%s no related fill-history task, stream time window and verRange are not set. default stream time " + "window:%" PRId64 "-%" PRId64 ", verRange:%" PRId64 "-%" PRId64, + pTask->id.idStr, pRange->window.skey, pRange->window.ekey, pRange->range.minVer, pRange->range.maxVer); } } else { - SDataRange* pRange = &pTask->dataRange; - int64_t ekey = 0; if (pRange->window.ekey < INT64_MAX) { ekey = pRange->window.ekey + 1; diff --git a/source/libs/stream/src/streamTask.c b/source/libs/stream/src/streamTask.c index 2907923d03..8dffbec09f 100644 --- a/source/libs/stream/src/streamTask.c +++ b/source/libs/stream/src/streamTask.c @@ -413,6 +413,7 @@ int32_t streamTaskInit(SStreamTask* pTask, SStreamMeta* pMeta, SMsgCb* pMsgCb, i pTask->outputInfo.status = TASK_OUTPUT_STATUS__NORMAL; pTask->pMeta = pMeta; + pTask->chkInfo.checkpointVer = ver - 1; pTask->chkInfo.nextProcessVer = ver; pTask->dataRange.range.maxVer = ver; pTask->dataRange.range.minVer = ver; @@ -688,6 +689,63 @@ int32_t streamBuildAndSendDropTaskMsg(SMsgCb* pMsgCb, int32_t vgId, SStreamTaskI return code; } +int32_t streamBuildAndSendVerUpdateMsg(SMsgCb* pMsgCb, int32_t vgId, SStreamTaskId* pTaskId, int64_t ver) { + SVStreamTaskVerUpdateReq* pReq = rpcMallocCont(sizeof(SVStreamTaskVerUpdateReq)); + if (pReq == NULL) { + terrno = TSDB_CODE_OUT_OF_MEMORY; + return -1; + } + + pReq->head.vgId = vgId; + pReq->taskId = pTaskId->taskId; + pReq->streamId = pTaskId->streamId; + pReq->dataVer = ver; + + SRpcMsg msg = {.msgType = TDMT_STREAM_TASK_VERUPDATE, .pCont = pReq, .contLen = sizeof(SVStreamTaskVerUpdateReq)}; + int32_t code = tmsgPutToQueue(pMsgCb, WRITE_QUEUE, &msg); + if (code != TSDB_CODE_SUCCESS) { + stError("vgId:%d failed to send update task:0x%x dataVer msg, code:%s", vgId, pTaskId->taskId, tstrerror(code)); + return code; + } + + stDebug("vgId:%d build and send update table:0x%x dataVer:%"PRId64" msg", vgId, pTaskId->taskId, ver); + return code; +} + +int32_t streamTaskUpdateDataVer(SStreamTask* pTask, int64_t ver) { + SStreamMeta* pMeta = pTask->pMeta; + + // commit the dataVer update + int64_t prevVer = 0; + taosThreadMutexLock(&pTask->lock); + + if (pTask->chkInfo.checkpointId == 0) { + prevVer = pTask->chkInfo.nextProcessVer; + pTask->chkInfo.nextProcessVer = ver; + taosThreadMutexUnlock(&pTask->lock); + + taosWLockLatch(&pMeta->lock); + if (streamMetaSaveTask(pMeta, pTask) < 0) { +// return -1; + } + + if (streamMetaCommit(pMeta) < 0) { + // persist to disk + } + + stDebug("s-task:%s nextProcessedVer is update from %" PRId64 " to %" PRId64 " checkpointId:%" PRId64 + " checkpointVer:%" PRId64, + pTask->id.idStr, prevVer, ver, pTask->chkInfo.checkpointId, pTask->chkInfo.checkpointVer); + taosWUnLockLatch(&pMeta->lock); + } else { + stDebug("s-task:%s not update the dataVer, existed:%" PRId64 ", checkpointId:%" PRId64 " checkpointVer:%" PRId64, + pTask->id.idStr, pTask->chkInfo.nextProcessVer, pTask->chkInfo.checkpointId, pTask->chkInfo.checkpointVer); + taosThreadMutexUnlock(&pTask->lock); + } + + return TSDB_CODE_SUCCESS; +} + STaskId streamTaskExtractKey(const SStreamTask* pTask) { STaskId id = {.streamId = pTask->id.streamId, .taskId = pTask->id.taskId}; return id; From 83f84d92bc89a7365bcc42f94308a39d3421beb8 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 28 Sep 2023 22:44:13 +0800 Subject: [PATCH 116/158] refactor: do some internal refactor (discard the checkpoint source msg during restoring). --- source/dnode/vnode/src/inc/vnodeInt.h | 2 +- source/dnode/vnode/src/tq/tq.c | 14 +++++++++++--- source/dnode/vnode/src/vnd/vnodeSvr.c | 2 +- source/libs/stream/src/streamMeta.c | 3 +-- source/libs/stream/src/streamRecover.c | 19 +++++++------------ 5 files changed, 21 insertions(+), 19 deletions(-) diff --git a/source/dnode/vnode/src/inc/vnodeInt.h b/source/dnode/vnode/src/inc/vnodeInt.h index 1e07c87cb2..2fab7c087a 100644 --- a/source/dnode/vnode/src/inc/vnodeInt.h +++ b/source/dnode/vnode/src/inc/vnodeInt.h @@ -224,7 +224,7 @@ int tqPushMsg(STQ*, tmsg_t msgType); int tqRegisterPushHandle(STQ* pTq, void* handle, SRpcMsg* pMsg); int tqUnregisterPushHandle(STQ* pTq, void* pHandle); int tqScanWalAsync(STQ* pTq, bool ckPause); -int32_t tqProcessStreamCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg* pRsp); +int32_t tqProcessTaskCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg* pRsp); int32_t tqProcessTaskCheckpointReadyMsg(STQ* pTq, SRpcMsg* pMsg); int32_t tqProcessTaskUpdateReq(STQ* pTq, SRpcMsg* pMsg); int32_t tqProcessTaskDataVerUpdateReq(STQ* pTq, char* pMsg, int32_t msgLen); diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index b523faec7f..9780a1f046 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1638,7 +1638,7 @@ FAIL: } // todo error code cannot be return, since this is invoked by an mnode-launched transaction. -int32_t tqProcessStreamCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg* pRsp) { +int32_t tqProcessTaskCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg* pRsp) { int32_t vgId = TD_VID(pTq->pVnode); SStreamMeta* pMeta = pTq->pStreamMeta; char* msg = POINTER_SHIFT(pMsg->pCont, sizeof(SMsgHead)); @@ -1648,6 +1648,7 @@ int32_t tqProcessStreamCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg* pRs // disable auto rsp to source pRsp->info.handle = NULL; + // todo: add counter to make sure other tasks would not be trapped in checkpoint state SStreamCheckpointSourceReq req = {0}; if (!vnodeIsRoleLeader(pTq->pVnode)) { tqDebug("vgId:%d not leader, ignore checkpoint-source msg", vgId); @@ -1657,6 +1658,14 @@ int32_t tqProcessStreamCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg* pRs return TSDB_CODE_SUCCESS; } + if (!pTq->pVnode->restored) { + tqDebug("vgId:%d checkpoint-source msg received during restoring, ignore it", vgId); + SRpcMsg rsp = {0}; + buildCheckpointSourceRsp(&req, &pMsg->info, &rsp, 0); + tmsgSendRsp(&rsp); // error occurs + return TSDB_CODE_SUCCESS; + } + SDecoder decoder; tDecoderInit(&decoder, (uint8_t*)msg, len); if (tDecodeStreamCheckpointSourceReq(&decoder, &req) < 0) { @@ -1680,6 +1689,7 @@ int32_t tqProcessStreamCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg* pRs return TSDB_CODE_SUCCESS; } + // todo: handle the partial failure cases // downstream not ready, current the stream tasks are not all ready. Ignore this checkpoint req. if (pTask->status.downstreamReady != 1) { qError("s-task:%s not ready for checkpoint, since downstream not ready, ignore this checkpoint:%" PRId64 @@ -1731,8 +1741,6 @@ int32_t tqProcessStreamCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg* pRs return code; } - // todo: when generating checkpoint, no new tasks are allowed to add into current Vnode - // todo: when generating checkpoint, leader of mnode has transfer to other DNode? streamMetaReleaseTask(pMeta, pTask); return code; } diff --git a/source/dnode/vnode/src/vnd/vnodeSvr.c b/source/dnode/vnode/src/vnd/vnodeSvr.c index 7a1e60f075..5622568b7b 100644 --- a/source/dnode/vnode/src/vnd/vnodeSvr.c +++ b/source/dnode/vnode/src/vnd/vnodeSvr.c @@ -605,7 +605,7 @@ int32_t vnodeProcessWriteMsg(SVnode *pVnode, SRpcMsg *pMsg, int64_t ver, SRpcMsg vnodeProcessDropIndexReq(pVnode, ver, pReq, len, pRsp); break; case TDMT_VND_STREAM_CHECK_POINT_SOURCE: - tqProcessStreamCheckPointSourceReq(pVnode->pTq, pMsg, pRsp); + tqProcessTaskCheckPointSourceReq(pVnode->pTq, pMsg, pRsp); break; case TDMT_VND_STREAM_TASK_UPDATE: tqProcessTaskUpdateReq(pVnode->pTq, pMsg); diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 840eee98a8..5470da8360 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -716,9 +716,8 @@ int32_t streamMetaLoadAllTasks(SStreamMeta* pMeta) { taosArrayPush(pMeta->pTaskList, &pTask->id); } else { + // todo this should replace the existed object put by replay creating stream task msg from mnode stError("s-task:0x%x already added into table meta by replaying WAL, need check", pTask->id.taskId); - ASSERT(0); - tdbFree(pKey); tdbFree(pVal); taosMemoryFree(pTask); diff --git a/source/libs/stream/src/streamRecover.c b/source/libs/stream/src/streamRecover.c index 3e491fa1bf..67f4108270 100644 --- a/source/libs/stream/src/streamRecover.c +++ b/source/libs/stream/src/streamRecover.c @@ -583,16 +583,12 @@ int32_t streamProcessScanHistoryFinishRsp(SStreamTask* pTask) { static void checkFillhistoryTaskStatus(SStreamTask* pTask, SStreamTask* pHTask) { SDataRange* pRange = &pHTask->dataRange; - pRange->range.minVer = 0; - // todo remove this // the query version range should be limited to the already processed data + pRange->range.minVer = 0; pRange->range.maxVer = pTask->chkInfo.nextProcessVer - 1; - if (pRange->range.maxVer < pRange->range.minVer) { - pRange->range.maxVer = pRange->range.minVer; - } - pHTask->execInfo.init = taosGetTimestampMs(); + if (pTask->info.taskLevel == TASK_LEVEL__SOURCE) { stDebug("s-task:%s set the launch condition for fill-history s-task:%s, window:%" PRId64 " - %" PRId64 " ver range:%" PRId64 " - %" PRId64", init:%"PRId64, @@ -890,9 +886,8 @@ void streamTaskSetRangeStreamCalc(SStreamTask* pTask) { if (pTask->hTaskInfo.id.taskId == 0) { if (pTask->info.fillHistory == 1) { - stDebug("s-task:%s fill-history task, time window:%" PRId64 "-%" PRId64 ", verRange:%" PRId64 - "-%" PRId64, - pTask->id.idStr, pRange->window.skey, pRange->window.ekey, pRange->range.minVer, pRange->range.maxVer); + stDebug("s-task:%s fill-history task, time window:%" PRId64 "-%" PRId64 ", verRange:%" PRId64 "-%" PRId64, + pTask->id.idStr, pRange->window.skey, pRange->window.ekey, pRange->range.minVer, pRange->range.maxVer); } else { stDebug( "s-task:%s no related fill-history task, stream time window and verRange are not set. default stream time " @@ -915,9 +910,9 @@ void streamTaskSetRangeStreamCalc(SStreamTask* pTask) { pRange->range.maxVer = ver; stDebug("s-task:%s level:%d related fill-history task exists, update stream calc time window:%" PRId64 " - %" PRId64 - ", verRang:%" PRId64 " - %" PRId64, - pTask->id.idStr, pTask->info.taskLevel, pRange->window.skey, pRange->window.ekey, pRange->range.minVer, - pRange->range.maxVer); + ", verRang:%" PRId64 " - %" PRId64, + pTask->id.idStr, pTask->info.taskLevel, pRange->window.skey, pRange->window.ekey, pRange->range.minVer, + pRange->range.maxVer); } } From b0a4ed3217c8122b261704c990ecda5f1f187c49 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 28 Sep 2023 23:00:29 +0800 Subject: [PATCH 117/158] fix(stream): drop related fill-history task when dropping stream tasks. --- source/dnode/vnode/src/inc/vnodeInt.h | 2 +- source/dnode/vnode/src/tq/tq.c | 32 ++++++++++++++++++++------- source/dnode/vnode/src/vnd/vnodeSvr.c | 2 +- 3 files changed, 26 insertions(+), 10 deletions(-) diff --git a/source/dnode/vnode/src/inc/vnodeInt.h b/source/dnode/vnode/src/inc/vnodeInt.h index 2fab7c087a..2ffdf2fced 100644 --- a/source/dnode/vnode/src/inc/vnodeInt.h +++ b/source/dnode/vnode/src/inc/vnodeInt.h @@ -247,7 +247,7 @@ int32_t tqProcessVgCommittedInfoReq(STQ* pTq, SRpcMsg* pMsg); // tq-stream int32_t tqProcessTaskDeployReq(STQ* pTq, int64_t version, char* msg, int32_t msgLen); -int32_t tqProcessTaskDropReq(STQ* pTq, int64_t version, char* msg, int32_t msgLen); +int32_t tqProcessTaskDropReq(STQ* pTq, char* msg, int32_t msgLen); int32_t tqProcessTaskPauseReq(STQ* pTq, int64_t version, char* msg, int32_t msgLen); int32_t tqProcessTaskResumeReq(STQ* pTq, int64_t version, char* msg, int32_t msgLen); int32_t tqProcessTaskCheckReq(STQ* pTq, SRpcMsg* pMsg); diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 9780a1f046..455937b603 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1406,20 +1406,36 @@ int32_t tqProcessTaskDispatchRsp(STQ* pTq, SRpcMsg* pMsg) { } } -int32_t tqProcessTaskDropReq(STQ* pTq, int64_t sversion, char* msg, int32_t msgLen) { +int32_t tqProcessTaskDropReq(STQ* pTq, char* msg, int32_t msgLen) { SVDropStreamTaskReq* pReq = (SVDropStreamTaskReq*)msg; - tqDebug("vgId:%d receive msg to drop stream task:0x%x", TD_VID(pTq->pVnode), pReq->taskId); - streamMetaUnregisterTask(pTq->pStreamMeta, pReq->streamId, pReq->taskId); + + int32_t vgId = TD_VID(pTq->pVnode); + SStreamMeta* pMeta = pTq->pStreamMeta; + tqDebug("vgId:%d receive msg to drop s-task:0x%x", vgId, pReq->taskId); + + SStreamTask* pTask = streamMetaAcquireTask(pMeta, pReq->streamId, pReq->taskId); + if (pTask != NULL) { + // drop the related fill-history task firstly + if (pTask->hTaskInfo.id.taskId != 0) { + STaskId* pHTaskId = &pTask->hTaskInfo.id; + streamMetaUnregisterTask(pMeta, pHTaskId->streamId, pHTaskId->taskId); + tqDebug("vgId:%d drop fill-history task:0x%x dropped firstly", vgId, (int32_t)pHTaskId->taskId); + } + } + streamMetaReleaseTask(pMeta, pTask); + + // drop the stream task now + streamMetaUnregisterTask(pMeta, pReq->streamId, pReq->taskId); // commit the update - taosWLockLatch(&pTq->pStreamMeta->lock); - int32_t numOfTasks = streamMetaGetNumOfTasks(pTq->pStreamMeta); - tqDebug("vgId:%d task:0x%x dropped, remain tasks:%d", TD_VID(pTq->pVnode), pReq->taskId, numOfTasks); + taosWLockLatch(&pMeta->lock); + int32_t numOfTasks = streamMetaGetNumOfTasks(pMeta); + tqDebug("vgId:%d task:0x%x dropped, remain tasks:%d", vgId, pReq->taskId, numOfTasks); - if (streamMetaCommit(pTq->pStreamMeta) < 0) { + if (streamMetaCommit(pMeta) < 0) { // persist to disk } - taosWUnLockLatch(&pTq->pStreamMeta->lock); + taosWUnLockLatch(&pMeta->lock); return 0; } diff --git a/source/dnode/vnode/src/vnd/vnodeSvr.c b/source/dnode/vnode/src/vnd/vnodeSvr.c index 5622568b7b..f0cfe6cd0b 100644 --- a/source/dnode/vnode/src/vnd/vnodeSvr.c +++ b/source/dnode/vnode/src/vnd/vnodeSvr.c @@ -567,7 +567,7 @@ int32_t vnodeProcessWriteMsg(SVnode *pVnode, SRpcMsg *pMsg, int64_t ver, SRpcMsg } } break; case TDMT_STREAM_TASK_DROP: { - if (tqProcessTaskDropReq(pVnode->pTq, ver, pMsg->pCont, pMsg->contLen) < 0) { + if (tqProcessTaskDropReq(pVnode->pTq, pMsg->pCont, pMsg->contLen) < 0) { goto _err; } } break; From f9113ab7b07979e9321e1bfefa4643a00a1377bb Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 28 Sep 2023 23:26:46 +0800 Subject: [PATCH 118/158] fix(stream): fix invalid param. --- source/dnode/mnode/impl/src/mndScheduler.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/dnode/mnode/impl/src/mndScheduler.c b/source/dnode/mnode/impl/src/mndScheduler.c index f2e783d64f..3a4b36a346 100644 --- a/source/dnode/mnode/impl/src/mndScheduler.c +++ b/source/dnode/mnode/impl/src/mndScheduler.c @@ -460,7 +460,7 @@ static int32_t addAggTask(SStreamObj* pStream, SMnode* pMnode, SQueryPlan* pPlan *pHAggTask = NULL; code = doAddAggTask(pStream->hTaskUid, pHAggTaskList, pHSinkNodeList, pMnode, pStream, pEpset, pStream->conf.fillHistory, - pHAggTask, false); + pHAggTask, pStream->conf.fillHistory); if (code != TSDB_CODE_SUCCESS) { if (pSnode != NULL) { sdbRelease(pSdb, pSnode); From 19042cdea6d110d9fe998df8b9cb27980a3976b7 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 28 Sep 2023 23:31:07 +0800 Subject: [PATCH 119/158] fix(stream): fix invalid param. --- source/dnode/mnode/impl/src/mndScheduler.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/dnode/mnode/impl/src/mndScheduler.c b/source/dnode/mnode/impl/src/mndScheduler.c index 3a4b36a346..f152fc7c5d 100644 --- a/source/dnode/mnode/impl/src/mndScheduler.c +++ b/source/dnode/mnode/impl/src/mndScheduler.c @@ -529,7 +529,7 @@ static int32_t addSourceTasksForMultiLevelStream(SMnode* pMnode, SQueryPlan* pPl if (pStream->conf.fillHistory) { code = doAddSourceTask(pHSourceTaskList, true, pStream->hTaskUid, pHDownstreamTask, pMnode, plan, pVgroup, pEpset, - nextWindowSkey, false); + nextWindowSkey, pStream->conf.fillHistory); if (code != TSDB_CODE_SUCCESS) { sdbRelease(pSdb, pVgroup); return code; From 583ac1fc45feaf85819977861c44c9ea0fc27f6b Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sun, 1 Oct 2023 22:27:29 +0800 Subject: [PATCH 120/158] refactor: do some internal refactor. --- include/libs/stream/tstream.h | 24 ++++++----- source/common/src/systable.c | 4 +- source/dnode/mnode/impl/src/mndScheduler.c | 2 +- source/dnode/mnode/impl/src/mndStream.c | 28 ++++++++++++- source/dnode/snode/src/snode.c | 4 +- source/dnode/vnode/src/tq/tq.c | 8 ++-- source/dnode/vnode/src/tq/tqStreamTask.c | 4 +- source/libs/stream/inc/streamInt.h | 17 ++++++-- source/libs/stream/src/streamMeta.c | 48 ++++++++++++++++------ source/libs/stream/src/streamQueue.c | 25 +++++------ source/libs/stream/src/streamRecover.c | 16 ++++---- 11 files changed, 120 insertions(+), 60 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 40b2be72bc..b399459230 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -44,7 +44,8 @@ extern "C" { #define NODE_ROLE_LEADER 0x2 #define NODE_ROLE_FOLLOWER 0x3 -typedef struct SStreamTask SStreamTask; +typedef struct SStreamTask SStreamTask; +typedef struct SStreamQueue SStreamQueue; #define SSTREAM_TASK_VER 2 enum { @@ -190,13 +191,6 @@ int32_t streamQueuePush(SStreamQueue1* pQueue, SStreamQueueItem* pItem); SStreamQueueRes streamQueueGetRes(SStreamQueue1* pQueue); #endif -typedef struct { - STaosQueue* pQueue; - STaosQall* qall; - void* qItem; - int8_t status; -} SStreamQueue; - int32_t streamInit(); void streamCleanUp(); @@ -314,7 +308,7 @@ typedef struct STaskOutputInfo { } STaskOutputInfo; typedef struct STaskInputInfo { - int8_t status; + int8_t status; SStreamQueue* queue; } STaskInputInfo; @@ -406,7 +400,8 @@ struct SStreamTask { }; typedef struct STaskStartInfo { - int64_t ts; + int64_t startTs; + int64_t readyTs; int32_t startedAfterNodeUpdate; SHashObj* pReadyTaskSet; // tasks that are all ready for running stream processing int32_t elapsedTime; @@ -463,7 +458,7 @@ int32_t tDecodeStreamTaskId(SDecoder* pDecoder, STaskId* pTaskId); int32_t streamTaskPutDataIntoInputQ(SStreamTask* pTask, SStreamQueueItem* pItem); int32_t streamTaskPutDataIntoOutputQ(SStreamTask* pTask, SStreamDataBlock* pBlock); int32_t streamTaskPutTranstateIntoInputQ(SStreamTask* pTask); -bool streamQueueIsFull(const STaosQueue* pQueue, bool inputQ); +bool streamQueueIsFull(const SStreamQueue* pQueue, bool inputQ); typedef struct { SMsgHead head; @@ -602,6 +597,13 @@ typedef struct STaskStatusEntry { int32_t status; int32_t stage; int32_t nodeId; + int64_t verStart; // start version in WAL, only valid for source task + int64_t verEnd; // end version in WAL, only valid for source task + int64_t offset; // only valid for source task + double inputQUsed; // in MiB + double inputQCap; + double outputQUsed; // in MiB + double outputQCap; } STaskStatusEntry; typedef struct SStreamHbMsg { diff --git a/source/common/src/systable.c b/source/common/src/systable.c index 7107f0e058..e4e4f2ce99 100644 --- a/source/common/src/systable.c +++ b/source/common/src/systable.c @@ -161,9 +161,11 @@ static const SSysDbTableSchema streamTaskSchema[] = { {.name = "task_id", .bytes = 16 + VARSTR_HEADER_SIZE, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = false}, {.name = "node_type", .bytes = 10 + VARSTR_HEADER_SIZE, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = false}, {.name = "node_id", .bytes = 4, .type = TSDB_DATA_TYPE_INT, .sysInfo = false}, - {.name = "level", .bytes = 20 + VARSTR_HEADER_SIZE, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = false}, + {.name = "level", .bytes = 10 + VARSTR_HEADER_SIZE, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = false}, {.name = "status", .bytes = 20 + VARSTR_HEADER_SIZE, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = false}, {.name = "stage", .bytes = 4, .type = TSDB_DATA_TYPE_INT, .sysInfo = false}, + {.name = "in_queue", .bytes = 35, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = false}, + {.name = "out_queue", .bytes = 35, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = false}, }; static const SSysDbTableSchema userTblsSchema[] = { diff --git a/source/dnode/mnode/impl/src/mndScheduler.c b/source/dnode/mnode/impl/src/mndScheduler.c index f152fc7c5d..d598dc11d2 100644 --- a/source/dnode/mnode/impl/src/mndScheduler.c +++ b/source/dnode/mnode/impl/src/mndScheduler.c @@ -353,7 +353,7 @@ static int32_t addSourceTasksForOneLevelStream(SMnode* pMnode, const SQueryPlan* if (pStream->conf.fillHistory) { SArray** pHSinkTaskList = taosArrayGet(pStream->pHTasksList, SINK_NODE_LEVEL); code = addSourceTask(pMnode, pVgroup, pHTaskList, *pHSinkTaskList, pStream, plan, pStream->hTaskUid, - pEpset, true, hasExtraSink, nextWindowSkey, false); + pEpset, true, hasExtraSink, nextWindowSkey, true); } sdbRelease(pSdb, pVgroup); diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index 8cbdeff19a..11916bdb4f 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -1563,7 +1563,7 @@ static int32_t mndRetrieveStreamTask(SRpcMsg *pReq, SShowObj *pShow, SSDataBlock } pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); - colDataSetVal(pColInfo, numOfRows, (const char *)&level, false); + colDataSetVal(pColInfo, numOfRows, (const char *)level, false); // status char status[20 + VARSTR_HEADER_SIZE] = {0}; @@ -1577,12 +1577,31 @@ static int32_t mndRetrieveStreamTask(SRpcMsg *pReq, SShowObj *pShow, SSDataBlock const char* pStatus = streamGetTaskStatusStr(pe->status); STR_TO_VARSTR(status, pStatus); + // status pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); - colDataSetVal(pColInfo, numOfRows, (const char *)&status, false); + colDataSetVal(pColInfo, numOfRows, (const char *)status, false); + // stage pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); colDataSetVal(pColInfo, numOfRows, (const char *)&pe->stage, false); + // input queue + char vbuf[30] = {0}; + char buf[25] = {0}; + const char* queueInfoStr = "%.2fMiB (%.2f%, %.2fMiB)"; + sprintf(buf, queueInfoStr, pe->inputQUsed, pe->inputQUsed/pe->inputQCap, pe->inputQCap); + STR_TO_VARSTR(vbuf, buf); + + pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); + colDataSetVal(pColInfo, numOfRows, (const char*)vbuf, false); + + // output queue + sprintf(buf, queueInfoStr, pe->outputQUsed, pe->outputQUsed/pe->outputQCap, pe->outputQCap); + STR_TO_VARSTR(vbuf, buf); + + pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); + colDataSetVal(pColInfo, numOfRows, (const char*)vbuf, false); + numOfRows++; } } @@ -2429,6 +2448,11 @@ int32_t mndProcessStreamHb(SRpcMsg *pReq) { } } else { pEntry->stage = p->stage; + pEntry->inputQUsed = p->inputQUsed; + pEntry->inputQCap = p->inputQCap; + pEntry->outputQUsed = p->outputQUsed; + pEntry->outputQCap = p->outputQCap; + pEntry->offset = p->offset; } pEntry->status = p->status; diff --git a/source/dnode/snode/src/snode.c b/source/dnode/snode/src/snode.c index 799c784f38..c5fd202986 100644 --- a/source/dnode/snode/src/snode.c +++ b/source/dnode/snode/src/snode.c @@ -53,9 +53,9 @@ FAIL: taosFreeQitem(pMsg); } -int32_t sndExpandTask(SSnode *pSnode, SStreamTask *pTask, int64_t ver) { +int32_t sndExpandTask(SSnode *pSnode, SStreamTask *pTask, int64_t nextProcessVer) { ASSERT(pTask->info.taskLevel == TASK_LEVEL__AGG && taosArrayGetSize(pTask->pUpstreamInfoList) != 0); - int32_t code = streamTaskInit(pTask, pSnode->pMeta, &pSnode->msgCb, ver); + int32_t code = streamTaskInit(pTask, pSnode->pMeta, &pSnode->msgCb, nextProcessVer); if (code != TSDB_CODE_SUCCESS) { return code; } diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 455937b603..19bcf2a9d8 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -724,11 +724,11 @@ end: void freePtr(void* ptr) { taosMemoryFree(*(void**)ptr); } -int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t ver) { +int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t nextProcessVer) { int32_t vgId = TD_VID(pTq->pVnode); tqDebug("s-task:0x%x start to expand task", pTask->id.taskId); - int32_t code = streamTaskInit(pTask, pTq->pStreamMeta, &pTq->pVnode->msgCb, ver); + int32_t code = streamTaskInit(pTask, pTq->pStreamMeta, &pTq->pVnode->msgCb, nextProcessVer); if (code != TSDB_CODE_SUCCESS) { return code; } @@ -1421,8 +1421,8 @@ int32_t tqProcessTaskDropReq(STQ* pTq, char* msg, int32_t msgLen) { streamMetaUnregisterTask(pMeta, pHTaskId->streamId, pHTaskId->taskId); tqDebug("vgId:%d drop fill-history task:0x%x dropped firstly", vgId, (int32_t)pHTaskId->taskId); } + streamMetaReleaseTask(pMeta, pTask); } - streamMetaReleaseTask(pMeta, pTask); // drop the stream task now streamMetaUnregisterTask(pMeta, pReq->streamId, pReq->taskId); @@ -1510,7 +1510,7 @@ int32_t tqProcessTaskResumeImpl(STQ* pTq, SStreamTask* pTask, int64_t sversion, if (level == TASK_LEVEL__SOURCE && pTask->info.fillHistory && pTask->status.taskStatus == TASK_STATUS__SCAN_HISTORY) { streamStartScanHistoryAsync(pTask, igUntreated); - } else if (level == TASK_LEVEL__SOURCE && (taosQueueItemSize(pTask->inputInfo.queue->pQueue) == 0)) { + } else if (level == TASK_LEVEL__SOURCE && (streamQueueGetNumOfItems(pTask->inputInfo.queue) == 0)) { tqScanWalAsync(pTq, false); } else { streamSchedExec(pTask); diff --git a/source/dnode/vnode/src/tq/tqStreamTask.c b/source/dnode/vnode/src/tq/tqStreamTask.c index c3ef52e96f..069cc4cbbd 100644 --- a/source/dnode/vnode/src/tq/tqStreamTask.c +++ b/source/dnode/vnode/src/tq/tqStreamTask.c @@ -73,7 +73,7 @@ int32_t tqCheckAndRunStreamTask(STQ* pTq) { taosWLockLatch(&pMeta->lock); pTaskList = taosArrayDup(pMeta->pTaskList, NULL); taosHashClear(pMeta->startInfo.pReadyTaskSet); - pMeta->startInfo.ts = taosGetTimestampMs(); + pMeta->startInfo.startTs = taosGetTimestampMs(); taosWUnLockLatch(&pMeta->lock); // broadcast the check downstream tasks msg @@ -370,7 +370,7 @@ int32_t doScanWalForAllTasks(SStreamMeta* pStreamMeta, bool* pScanIdle) { continue; } - if (streamQueueIsFull(pTask->inputInfo.queue->pQueue, true)) { + if (streamQueueGetNumOfItems(pTask->inputInfo.queue)) { tqTrace("s-task:%s input queue is full, do nothing", pTask->id.idStr); streamMetaReleaseTask(pStreamMeta, pTask); continue; diff --git a/source/libs/stream/inc/streamInt.h b/source/libs/stream/inc/streamInt.h index 1d8fde3a48..fe4d73b566 100644 --- a/source/libs/stream/inc/streamInt.h +++ b/source/libs/stream/inc/streamInt.h @@ -40,6 +40,10 @@ extern "C" { #define META_HB_SEND_IDLE_COUNTER 25 // send hb every 5 sec #define STREAM_TASK_KEY_LEN ((sizeof(int64_t)) << 1) +#define STREAM_TASK_QUEUE_CAPACITY 20480 +#define STREAM_TASK_INPUT_QUEUE_CAPACITY_IN_SIZE (30) +#define STREAM_TASK_OUTPUT_QUEUE_CAPACITY_IN_SIZE (50) + // clang-format off #define stFatal(...) do { if (stDebugFlag & DEBUG_FATAL) { taosPrintLog("STM FATAL ", DEBUG_FATAL, 255, __VA_ARGS__); }} while(0) #define stError(...) do { if (stDebugFlag & DEBUG_ERROR) { taosPrintLog("STM ERROR ", DEBUG_ERROR, 255, __VA_ARGS__); }} while(0) @@ -70,6 +74,13 @@ struct STokenBucket { int64_t fillTimestamp; // fill timestamp }; +struct SStreamQueue { + STaosQueue* pQueue; + STaosQall* qall; + void* qItem; + int8_t status; +}; + extern SStreamGlobalEnv streamEnv; extern int32_t streamBackendId; extern int32_t streamBackendCfWrapperId; @@ -100,7 +111,6 @@ int32_t streamTaskSendCheckpointSourceRsp(SStreamTask* pTask); int32_t streamTaskGetNumOfDownstream(const SStreamTask* pTask); int32_t streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueItem** pInput, int32_t* numOfBlocks, int32_t* blockSize); -int32_t streamQueueGetNumOfItemsInQueue(const SStreamQueue* pQueue); int32_t streamQueueItemGetSize(const SStreamQueueItem* pItem); void streamQueueItemIncSize(const SStreamQueueItem* pItem, int32_t size); const char* streamQueueItemGetTypeStr(int32_t type); @@ -118,14 +128,15 @@ STaskId streamTaskExtractKey(const SStreamTask* pTask); void streamTaskInitForLaunchHTask(SHistoryTaskInfo* pInfo); void streamTaskSetRetryInfoForLaunch(SHistoryTaskInfo* pInfo); +void streamMetaResetStartInfo(STaskStartInfo* pMeta); + SStreamQueue* streamQueueOpen(int64_t cap); void streamQueueClose(SStreamQueue* pQueue, int32_t taskId); void streamQueueProcessSuccess(SStreamQueue* queue); void streamQueueProcessFail(SStreamQueue* queue); void* streamQueueNextItem(SStreamQueue* pQueue); void streamFreeQitem(SStreamQueueItem* data); - - +int32_t streamQueueGetItemSize(const SStreamQueue* pQueue); #ifdef __cplusplus } diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 5470da8360..0007b83fca 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -406,15 +406,15 @@ int32_t streamMetaRegisterTask(SStreamMeta* pMeta, int64_t ver, SStreamTask* pTa taosArrayPush(pMeta->pTaskList, &pTask->id); -// if (streamMetaSaveTask(pMeta, pTask) < 0) { -// tFreeStreamTask(pTask); -// return -1; -// } -// -// if (streamMetaCommit(pMeta) < 0) { -// tFreeStreamTask(pTask); -// return -1; -// } + if (streamMetaSaveTask(pMeta, pTask) < 0) { + tFreeStreamTask(pTask); + return -1; + } + + if (streamMetaCommit(pMeta) < 0) { + tFreeStreamTask(pTask); + return -1; + } taosHashPut(pMeta->pTasksMap, &id, sizeof(id), &pTask, POINTER_BYTES); if (pTask->info.fillHistory == 0) { @@ -706,9 +706,7 @@ int32_t streamMetaLoadAllTasks(SStreamMeta* pMeta) { STaskId id = {.streamId = pTask->id.streamId, .taskId = pTask->id.taskId}; void* p = taosHashGet(pMeta->pTasksMap, &id, sizeof(id)); if (p == NULL) { - // pTask->chkInfo.checkpointVer may be 0, when a follower is become a leader - // In this case, we try not to start fill-history task anymore. - if (pMeta->expandFunc(pMeta->ahandle, pTask, pTask->chkInfo.checkpointVer) < 0) { + if (pMeta->expandFunc(pMeta->ahandle, pTask, pTask->chkInfo.checkpointVer + 1) < 0) { doClear(pKey, pVal, pCur, pRecycleList); tFreeStreamTask(pTask); return -1; @@ -776,6 +774,10 @@ int32_t tEncodeStreamHbMsg(SEncoder* pEncoder, const SStreamHbMsg* pReq) { if (tEncodeI32(pEncoder, ps->status) < 0) return -1; if (tEncodeI32(pEncoder, ps->stage) < 0) return -1; if (tEncodeI32(pEncoder, ps->nodeId) < 0) return -1; + if (tEncodeDouble(pEncoder, ps->inputQUsed) < 0) return -1; + if (tEncodeDouble(pEncoder, ps->inputQCap) < 0) return -1; + if (tEncodeDouble(pEncoder, ps->outputQUsed) < 0) return -1; + if (tEncodeDouble(pEncoder, ps->outputQCap) < 0) return -1; } tEndEncode(pEncoder); return pEncoder->pos; @@ -796,6 +798,10 @@ int32_t tDecodeStreamHbMsg(SDecoder* pDecoder, SStreamHbMsg* pReq) { if (tDecodeI32(pDecoder, &entry.status) < 0) return -1; if (tDecodeI32(pDecoder, &entry.stage) < 0) return -1; if (tDecodeI32(pDecoder, &entry.nodeId) < 0) return -1; + if (tDecodeDouble(pDecoder, &entry.inputQUsed) < 0) return -1; + if (tDecodeDouble(pDecoder, &entry.inputQCap) < 0) return -1; + if (tDecodeDouble(pDecoder, &entry.outputQUsed) < 0) return -1; + if (tDecodeDouble(pDecoder, &entry.outputQCap) < 0) return -1; entry.id.taskId = taskId; taosArrayPush(pReq->pTaskStatus, &entry); @@ -863,12 +869,23 @@ void metaHbToMnode(void* param, void* tmrId) { STaskId* pId = taosArrayGet(pMeta->pTaskList, i); SStreamTask** pTask = taosHashGet(pMeta->pTasksMap, pId, sizeof(*pId)); + + // not report the status of fill-history task if ((*pTask)->info.fillHistory == 1) { continue; } STaskStatusEntry entry = { - .id = *pId, .status = (*pTask)->status.taskStatus, .nodeId = pMeta->vgId, .stage = pMeta->stage}; + .id = *pId, + .status = (*pTask)->status.taskStatus, + .nodeId = pMeta->vgId, + .stage = pMeta->stage, + .inputQUsed = SIZE_IN_MiB(streamQueueGetItemSize((*pTask)->inputInfo.queue)), + .inputQCap = STREAM_TASK_INPUT_QUEUE_CAPACITY_IN_SIZE, + .outputQUsed = SIZE_IN_MiB(streamQueueGetItemSize((*pTask)->outputInfo.queue)), + .outputQCap = STREAM_TASK_OUTPUT_QUEUE_CAPACITY_IN_SIZE, + }; + taosArrayPush(hbMsg.pTaskStatus, &entry); if (!hasValEpset) { @@ -1004,4 +1021,9 @@ void streamMetaStartHb(SStreamMeta* pMeta) { void streamMetaInitForSnode(SStreamMeta* pMeta) { pMeta->stage = 0; pMeta->role = NODE_ROLE_LEADER; +} + +void streamMetaResetStartInfo(STaskStartInfo* pStartInfo) { + taosHashClear(pStartInfo->pReadyTaskSet); + pStartInfo->startedAfterNodeUpdate = 0; } \ No newline at end of file diff --git a/source/libs/stream/src/streamQueue.c b/source/libs/stream/src/streamQueue.c index 65400386b1..8a7827b5a7 100644 --- a/source/libs/stream/src/streamQueue.c +++ b/source/libs/stream/src/streamQueue.c @@ -17,9 +17,6 @@ #define MAX_STREAM_EXEC_BATCH_NUM 32 #define MIN_STREAM_EXEC_BATCH_NUM 4 -#define STREAM_TASK_QUEUE_CAPACITY 20480 -#define STREAM_TASK_INPUT_QUEUE_CAPACITY_IN_SIZE (30) -#define STREAM_TASK_OUTPUT_QUEUE_CAPACITY_IN_SIZE (50) #define MAX_SMOOTH_BURST_RATIO 5 // 20 sec // todo refactor: @@ -105,15 +102,14 @@ void streamQueueProcessFail(SStreamQueue* queue) { atomic_store_8(&queue->status, STREAM_QUEUE__FAILED); } -bool streamQueueIsFull(const STaosQueue* pQueue, bool inputQ) { - bool isFull = taosQueueItemSize((STaosQueue*)pQueue) >= STREAM_TASK_QUEUE_CAPACITY; - if (isFull) { +bool streamQueueIsFull(const SStreamQueue* pQueue, bool inputQ) { + int32_t numOfItems = streamQueueGetNumOfItems(pQueue); + if (numOfItems >= STREAM_TASK_QUEUE_CAPACITY) { return true; } - int32_t threahold = (inputQ) ? STREAM_TASK_INPUT_QUEUE_CAPACITY_IN_SIZE : STREAM_TASK_OUTPUT_QUEUE_CAPACITY_IN_SIZE; - double size = SIZE_IN_MiB(taosQueueMemorySize((STaosQueue*)pQueue)); - return (size >= threahold); + int32_t threshold = (inputQ) ? STREAM_TASK_INPUT_QUEUE_CAPACITY_IN_SIZE : STREAM_TASK_OUTPUT_QUEUE_CAPACITY_IN_SIZE; + return (SIZE_IN_MiB(taosQueueMemorySize(pQueue->pQueue)) >= threshold); } int32_t streamQueueGetNumOfItems(const SStreamQueue* pQueue) { @@ -123,8 +119,9 @@ int32_t streamQueueGetNumOfItems(const SStreamQueue* pQueue) { return numOfItems1 + numOfItems2; } -int32_t streamQueueGetNumOfItemsInQueue(const SStreamQueue* pQueue) { - return taosQueueItemSize(pQueue->pQueue); +// todo: fix it: data in Qall is not included here +int32_t streamQueueGetItemSize(const SStreamQueue* pQueue) { + return taosQueueMemorySize(pQueue->pQueue); } int32_t streamQueueItemGetSize(const SStreamQueueItem* pItem) { @@ -267,7 +264,7 @@ int32_t streamTaskPutDataIntoInputQ(SStreamTask* pTask, SStreamQueueItem* pItem) if (type == STREAM_INPUT__DATA_SUBMIT) { SStreamDataSubmit* px = (SStreamDataSubmit*)pItem; - if ((pTask->info.taskLevel == TASK_LEVEL__SOURCE) && streamQueueIsFull(pQueue, true)) { + if ((pTask->info.taskLevel == TASK_LEVEL__SOURCE) && streamQueueIsFull(pTask->inputInfo.queue, true)) { double size = SIZE_IN_MiB(taosQueueMemorySize(pQueue)); stTrace( "s-task:%s inputQ is full, capacity(size:%d num:%dMiB), current(blocks:%d, size:%.2fMiB) stop to push data", @@ -294,7 +291,7 @@ int32_t streamTaskPutDataIntoInputQ(SStreamTask* pTask, SStreamQueueItem* pItem) msgLen, ver, total, size + SIZE_IN_MiB(msgLen)); } else if (type == STREAM_INPUT__DATA_BLOCK || type == STREAM_INPUT__DATA_RETRIEVE || type == STREAM_INPUT__REF_DATA_BLOCK) { - if (streamQueueIsFull(pQueue, true)) { + if (streamQueueIsFull(pTask->inputInfo.queue, true)) { double size = SIZE_IN_MiB(taosQueueMemorySize(pQueue)); stTrace("s-task:%s input queue is full, capacity:%d size:%d MiB, current(blocks:%d, size:%.2fMiB) abort", @@ -348,7 +345,7 @@ int32_t streamTaskPutDataIntoInputQ(SStreamTask* pTask, SStreamQueueItem* pItem) int32_t streamTaskPutDataIntoOutputQ(SStreamTask* pTask, SStreamDataBlock* pBlock) { STaosQueue* pQueue = pTask->outputInfo.queue->pQueue; - while (streamQueueIsFull(pQueue, false)) { + while (streamQueueIsFull(pTask->inputInfo.queue, false)) { if (streamTaskShouldStop(&pTask->status)) { stInfo("s-task:%s discard result block due to task stop", pTask->id.idStr); return TSDB_CODE_STREAM_EXEC_CANCELLED; diff --git a/source/libs/stream/src/streamRecover.c b/source/libs/stream/src/streamRecover.c index 67f4108270..85e57339e0 100644 --- a/source/libs/stream/src/streamRecover.c +++ b/source/libs/stream/src/streamRecover.c @@ -58,16 +58,18 @@ static void streamTaskSetReady(SStreamTask* pTask, int32_t numOfReqs) { STaskId id = streamTaskExtractKey(pTask); taosHashPut(pMeta->startInfo.pReadyTaskSet, &id, sizeof(id), NULL, 0); + int32_t numOfTotal = streamMetaGetNumOfTasks(pMeta); - if (taosHashGetSize(pMeta->startInfo.pReadyTaskSet) == numOfTotal) { - // reset value for next time start - taosHashClear(pMeta->startInfo.pReadyTaskSet); - pMeta->startInfo.startedAfterNodeUpdate = 0; - pMeta->startInfo.elapsedTime = pTask->execInfo.start - pMeta->startInfo.ts; + STaskStartInfo* pStartInfo = &pMeta->startInfo; + pStartInfo->readyTs = pTask->execInfo.start; + pStartInfo->elapsedTime = pStartInfo->readyTs - pStartInfo->startTs; + streamMetaResetStartInfo(pStartInfo); - stDebug("vgId:%d all %d task(s) are started successfully, last ready task:%s level:%d, total elapsed time:%.2fs", - vgId, numOfTotal, pTask->id.idStr, pTask->info.taskLevel, pMeta->startInfo.elapsedTime / 1000.0); + stDebug("vgId:%d all %d task(s) are started successfully, last ready task:%s level:%d, startTs:%" PRId64 + ", readyTs:%" PRId64 " total elapsed time:%.2fs", + vgId, numOfTotal, pTask->id.idStr, pTask->info.taskLevel, pStartInfo->startTs, pStartInfo->readyTs, + pStartInfo->elapsedTime / 1000.0); } taosWUnLockLatch(&pMeta->lock); } From d5abb21335e83ef0d914f5f38111362626339eab Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 3 Oct 2023 00:38:24 +0800 Subject: [PATCH 121/158] fix(stream): fix invalid check. --- source/dnode/vnode/src/tq/tqStreamTask.c | 2 +- source/libs/stream/src/streamDispatch.c | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/source/dnode/vnode/src/tq/tqStreamTask.c b/source/dnode/vnode/src/tq/tqStreamTask.c index 069cc4cbbd..2040f8e323 100644 --- a/source/dnode/vnode/src/tq/tqStreamTask.c +++ b/source/dnode/vnode/src/tq/tqStreamTask.c @@ -370,7 +370,7 @@ int32_t doScanWalForAllTasks(SStreamMeta* pStreamMeta, bool* pScanIdle) { continue; } - if (streamQueueGetNumOfItems(pTask->inputInfo.queue)) { + if (streamQueueIsFull(pTask->inputInfo.queue, true)) { tqTrace("s-task:%s input queue is full, do nothing", pTask->id.idStr); streamMetaReleaseTask(pStreamMeta, pTask); continue; diff --git a/source/libs/stream/src/streamDispatch.c b/source/libs/stream/src/streamDispatch.c index 2d701d6bb0..5b76354dff 100644 --- a/source/libs/stream/src/streamDispatch.c +++ b/source/libs/stream/src/streamDispatch.c @@ -1128,7 +1128,6 @@ int32_t streamProcessDispatchRsp(SStreamTask* pTask, SStreamDispatchRsp* pRsp, i // all msg rsp already, continue if (leftRsp == 0) { ASSERT(pTask->outputInfo.status == TASK_OUTPUT_STATUS__WAIT); - stDebug("s-task:%s output status is set to:%d", id, pTask->outputInfo.status); // we need to re-try send dispatch msg to downstream tasks int32_t numOfFailed = taosArrayGetSize(pTask->msgInfo.pRetryList); From c295ec322a2b9e8bfffe4963a91918760304b85e Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 3 Oct 2023 01:52:03 +0800 Subject: [PATCH 122/158] refactor: do some internal refactor. --- include/libs/stream/tstream.h | 4 ++-- source/common/src/systable.c | 6 +++--- source/dnode/mnode/impl/src/mndStream.c | 10 +++++----- source/libs/stream/src/streamMeta.c | 13 +++++++------ 4 files changed, 17 insertions(+), 16 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index b399459230..a9da7d5ef8 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -601,9 +601,9 @@ typedef struct STaskStatusEntry { int64_t verEnd; // end version in WAL, only valid for source task int64_t offset; // only valid for source task double inputQUsed; // in MiB - double inputQCap; + double inputRate; double outputQUsed; // in MiB - double outputQCap; + double outputRate; } STaskStatusEntry; typedef struct SStreamHbMsg { diff --git a/source/common/src/systable.c b/source/common/src/systable.c index e4e4f2ce99..cea1c559cf 100644 --- a/source/common/src/systable.c +++ b/source/common/src/systable.c @@ -162,10 +162,10 @@ static const SSysDbTableSchema streamTaskSchema[] = { {.name = "node_type", .bytes = 10 + VARSTR_HEADER_SIZE, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = false}, {.name = "node_id", .bytes = 4, .type = TSDB_DATA_TYPE_INT, .sysInfo = false}, {.name = "level", .bytes = 10 + VARSTR_HEADER_SIZE, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = false}, - {.name = "status", .bytes = 20 + VARSTR_HEADER_SIZE, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = false}, + {.name = "status", .bytes = 15 + VARSTR_HEADER_SIZE, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = false}, {.name = "stage", .bytes = 4, .type = TSDB_DATA_TYPE_INT, .sysInfo = false}, - {.name = "in_queue", .bytes = 35, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = false}, - {.name = "out_queue", .bytes = 35, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = false}, + {.name = "in_queue", .bytes = 25, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = false}, + {.name = "out_queue", .bytes = 25, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = false}, }; static const SSysDbTableSchema userTblsSchema[] = { diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index 11916bdb4f..ee54252813 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -1588,15 +1588,15 @@ static int32_t mndRetrieveStreamTask(SRpcMsg *pReq, SShowObj *pShow, SSDataBlock // input queue char vbuf[30] = {0}; char buf[25] = {0}; - const char* queueInfoStr = "%.2fMiB (%.2f%, %.2fMiB)"; - sprintf(buf, queueInfoStr, pe->inputQUsed, pe->inputQUsed/pe->inputQCap, pe->inputQCap); + const char* queueInfoStr = "%.2fMiB (%.2f%)"; + sprintf(buf, queueInfoStr, pe->inputQUsed, pe->inputRate); STR_TO_VARSTR(vbuf, buf); pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); colDataSetVal(pColInfo, numOfRows, (const char*)vbuf, false); // output queue - sprintf(buf, queueInfoStr, pe->outputQUsed, pe->outputQUsed/pe->outputQCap, pe->outputQCap); + sprintf(buf, queueInfoStr, pe->outputQUsed, pe->outputRate); STR_TO_VARSTR(vbuf, buf); pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); @@ -2449,9 +2449,9 @@ int32_t mndProcessStreamHb(SRpcMsg *pReq) { } else { pEntry->stage = p->stage; pEntry->inputQUsed = p->inputQUsed; - pEntry->inputQCap = p->inputQCap; + pEntry->inputRate = p->inputRate; pEntry->outputQUsed = p->outputQUsed; - pEntry->outputQCap = p->outputQCap; + pEntry->outputRate = p->outputRate; pEntry->offset = p->offset; } diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 0007b83fca..725bcb7401 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -775,9 +775,9 @@ int32_t tEncodeStreamHbMsg(SEncoder* pEncoder, const SStreamHbMsg* pReq) { if (tEncodeI32(pEncoder, ps->stage) < 0) return -1; if (tEncodeI32(pEncoder, ps->nodeId) < 0) return -1; if (tEncodeDouble(pEncoder, ps->inputQUsed) < 0) return -1; - if (tEncodeDouble(pEncoder, ps->inputQCap) < 0) return -1; + if (tEncodeDouble(pEncoder, ps->inputRate) < 0) return -1; if (tEncodeDouble(pEncoder, ps->outputQUsed) < 0) return -1; - if (tEncodeDouble(pEncoder, ps->outputQCap) < 0) return -1; + if (tEncodeDouble(pEncoder, ps->outputRate) < 0) return -1; } tEndEncode(pEncoder); return pEncoder->pos; @@ -799,9 +799,9 @@ int32_t tDecodeStreamHbMsg(SDecoder* pDecoder, SStreamHbMsg* pReq) { if (tDecodeI32(pDecoder, &entry.stage) < 0) return -1; if (tDecodeI32(pDecoder, &entry.nodeId) < 0) return -1; if (tDecodeDouble(pDecoder, &entry.inputQUsed) < 0) return -1; - if (tDecodeDouble(pDecoder, &entry.inputQCap) < 0) return -1; + if (tDecodeDouble(pDecoder, &entry.inputRate) < 0) return -1; if (tDecodeDouble(pDecoder, &entry.outputQUsed) < 0) return -1; - if (tDecodeDouble(pDecoder, &entry.outputQCap) < 0) return -1; + if (tDecodeDouble(pDecoder, &entry.outputRate) < 0) return -1; entry.id.taskId = taskId; taosArrayPush(pReq->pTaskStatus, &entry); @@ -881,11 +881,12 @@ void metaHbToMnode(void* param, void* tmrId) { .nodeId = pMeta->vgId, .stage = pMeta->stage, .inputQUsed = SIZE_IN_MiB(streamQueueGetItemSize((*pTask)->inputInfo.queue)), - .inputQCap = STREAM_TASK_INPUT_QUEUE_CAPACITY_IN_SIZE, .outputQUsed = SIZE_IN_MiB(streamQueueGetItemSize((*pTask)->outputInfo.queue)), - .outputQCap = STREAM_TASK_OUTPUT_QUEUE_CAPACITY_IN_SIZE, }; + entry.inputRate = entry.inputQUsed*100.0/STREAM_TASK_INPUT_QUEUE_CAPACITY_IN_SIZE; + entry.outputRate = entry.outputQUsed*100.0/STREAM_TASK_OUTPUT_QUEUE_CAPACITY_IN_SIZE; + taosArrayPush(hbMsg.pTaskStatus, &entry); if (!hasValEpset) { From 0ab5d4b342e282c3a7b8b813f2d6acc2a2b8caf6 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 3 Oct 2023 02:37:52 +0800 Subject: [PATCH 123/158] refactor: do some internal refactor. --- source/common/src/systable.c | 5 +++-- source/dnode/mnode/impl/src/mndStream.c | 14 ++++++++++++-- source/libs/stream/src/streamMeta.c | 9 +++++++++ 3 files changed, 24 insertions(+), 4 deletions(-) diff --git a/source/common/src/systable.c b/source/common/src/systable.c index cea1c559cf..19e8945cdf 100644 --- a/source/common/src/systable.c +++ b/source/common/src/systable.c @@ -164,8 +164,9 @@ static const SSysDbTableSchema streamTaskSchema[] = { {.name = "level", .bytes = 10 + VARSTR_HEADER_SIZE, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = false}, {.name = "status", .bytes = 15 + VARSTR_HEADER_SIZE, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = false}, {.name = "stage", .bytes = 4, .type = TSDB_DATA_TYPE_INT, .sysInfo = false}, - {.name = "in_queue", .bytes = 25, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = false}, - {.name = "out_queue", .bytes = 25, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = false}, + {.name = "in_queue", .bytes = 20, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = false}, +// {.name = "out_queue", .bytes = 20, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = false}, + {.name = "offset", .bytes = 25, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = false}, }; static const SSysDbTableSchema userTblsSchema[] = { diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index ee54252813..460308f97d 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -1588,7 +1588,7 @@ static int32_t mndRetrieveStreamTask(SRpcMsg *pReq, SShowObj *pShow, SSDataBlock // input queue char vbuf[30] = {0}; char buf[25] = {0}; - const char* queueInfoStr = "%.2fMiB (%.2f%)"; + const char* queueInfoStr = "%5.2fMiB(%5.2f%)"; sprintf(buf, queueInfoStr, pe->inputQUsed, pe->inputRate); STR_TO_VARSTR(vbuf, buf); @@ -1596,7 +1596,15 @@ static int32_t mndRetrieveStreamTask(SRpcMsg *pReq, SShowObj *pShow, SSDataBlock colDataSetVal(pColInfo, numOfRows, (const char*)vbuf, false); // output queue - sprintf(buf, queueInfoStr, pe->outputQUsed, pe->outputRate); +// sprintf(buf, queueInfoStr, pe->outputQUsed, pe->outputRate); +// STR_TO_VARSTR(vbuf, buf); + +// pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); +// colDataSetVal(pColInfo, numOfRows, (const char*)vbuf, false); + + // offset version info + const char* offsetStr = "%"PRId64"[%"PRId64",%"PRId64"]"; + sprintf(buf, offsetStr, pe->offset, pe->verStart, pe->verEnd); STR_TO_VARSTR(vbuf, buf); pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); @@ -2453,6 +2461,8 @@ int32_t mndProcessStreamHb(SRpcMsg *pReq) { pEntry->outputQUsed = p->outputQUsed; pEntry->outputRate = p->outputRate; pEntry->offset = p->offset; + pEntry->verStart = p->verStart; + pEntry->verEnd = p->verEnd; } pEntry->status = p->status; diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 725bcb7401..ca5c0045ec 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -20,6 +20,7 @@ #include "tref.h" #include "tstream.h" #include "ttimer.h" +#include "wal.h" static TdThreadOnce streamMetaModuleInit = PTHREAD_ONCE_INIT; @@ -778,6 +779,9 @@ int32_t tEncodeStreamHbMsg(SEncoder* pEncoder, const SStreamHbMsg* pReq) { if (tEncodeDouble(pEncoder, ps->inputRate) < 0) return -1; if (tEncodeDouble(pEncoder, ps->outputQUsed) < 0) return -1; if (tEncodeDouble(pEncoder, ps->outputRate) < 0) return -1; + if (tEncodeI64(pEncoder, ps->offset) < 0) return -1; + if (tEncodeI64(pEncoder, ps->verStart) < 0) return -1; + if (tEncodeI64(pEncoder, ps->verEnd) < 0) return -1; } tEndEncode(pEncoder); return pEncoder->pos; @@ -802,6 +806,9 @@ int32_t tDecodeStreamHbMsg(SDecoder* pDecoder, SStreamHbMsg* pReq) { if (tDecodeDouble(pDecoder, &entry.inputRate) < 0) return -1; if (tDecodeDouble(pDecoder, &entry.outputQUsed) < 0) return -1; if (tDecodeDouble(pDecoder, &entry.outputRate) < 0) return -1; + if (tDecodeI64(pDecoder, &entry.offset) < 0) return -1; + if (tDecodeI64(pDecoder, &entry.verStart) < 0) return -1; + if (tDecodeI64(pDecoder, &entry.verEnd) < 0) return -1; entry.id.taskId = taskId; taosArrayPush(pReq->pTaskStatus, &entry); @@ -886,6 +893,8 @@ void metaHbToMnode(void* param, void* tmrId) { entry.inputRate = entry.inputQUsed*100.0/STREAM_TASK_INPUT_QUEUE_CAPACITY_IN_SIZE; entry.outputRate = entry.outputQUsed*100.0/STREAM_TASK_OUTPUT_QUEUE_CAPACITY_IN_SIZE; + entry.offset = walReaderGetCurrentVer((*pTask)->exec.pWalReader); + walReaderValidVersionRange((*pTask)->exec.pWalReader, &entry.verStart, &entry.verEnd); taosArrayPush(hbMsg.pTaskStatus, &entry); From 7682c033ad63044bb355f15096b8cbc04d37145c Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 3 Oct 2023 02:39:57 +0800 Subject: [PATCH 124/158] fix(stream): add null check. --- source/libs/stream/src/streamMeta.c | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index ca5c0045ec..aaffd9cfba 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -893,8 +893,11 @@ void metaHbToMnode(void* param, void* tmrId) { entry.inputRate = entry.inputQUsed*100.0/STREAM_TASK_INPUT_QUEUE_CAPACITY_IN_SIZE; entry.outputRate = entry.outputQUsed*100.0/STREAM_TASK_OUTPUT_QUEUE_CAPACITY_IN_SIZE; - entry.offset = walReaderGetCurrentVer((*pTask)->exec.pWalReader); - walReaderValidVersionRange((*pTask)->exec.pWalReader, &entry.verStart, &entry.verEnd); + + if ((*pTask)->exec.pWalReader != NULL) { + entry.offset = walReaderGetCurrentVer((*pTask)->exec.pWalReader); + walReaderValidVersionRange((*pTask)->exec.pWalReader, &entry.verStart, &entry.verEnd); + } taosArrayPush(hbMsg.pTaskStatus, &entry); From 932d93ab352bf341be77ef6518845d5107d8ff77 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 3 Oct 2023 15:42:00 +0800 Subject: [PATCH 125/158] refactor: wait for more time for future data --- source/dnode/mnode/impl/src/mndStream.c | 2 +- source/libs/stream/src/streamExec.c | 1 - source/libs/stream/src/streamQueue.c | 3 +-- 3 files changed, 2 insertions(+), 4 deletions(-) diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index 460308f97d..69c588d1bf 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -1588,7 +1588,7 @@ static int32_t mndRetrieveStreamTask(SRpcMsg *pReq, SShowObj *pShow, SSDataBlock // input queue char vbuf[30] = {0}; char buf[25] = {0}; - const char* queueInfoStr = "%5.2fMiB(%5.2f%)"; + const char* queueInfoStr = "%4.2fMiB(%5.2f%)"; sprintf(buf, queueInfoStr, pe->inputQUsed, pe->inputRate); STR_TO_VARSTR(vbuf, buf); diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index b5ea82d347..dbceb83803 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -17,7 +17,6 @@ // maximum allowed processed block batches. One block may include several submit blocks #define MAX_STREAM_EXEC_BATCH_NUM 32 -#define MIN_STREAM_EXEC_BATCH_NUM 4 #define STREAM_RESULT_DUMP_THRESHOLD 100 #define STREAM_RESULT_DUMP_SIZE_THRESHOLD (1048576 * 1) diff --git a/source/libs/stream/src/streamQueue.c b/source/libs/stream/src/streamQueue.c index 8a7827b5a7..7f7c039423 100644 --- a/source/libs/stream/src/streamQueue.c +++ b/source/libs/stream/src/streamQueue.c @@ -16,7 +16,6 @@ #include "streamInt.h" #define MAX_STREAM_EXEC_BATCH_NUM 32 -#define MIN_STREAM_EXEC_BATCH_NUM 4 #define MAX_SMOOTH_BURST_RATIO 5 // 20 sec // todo refactor: @@ -173,7 +172,7 @@ int32_t streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueItem** pInpu SStreamQueueItem* qItem = streamQueueNextItem(pTask->inputInfo.queue); if (qItem == NULL) { if ((taskLevel == TASK_LEVEL__SOURCE || taskLevel == TASK_LEVEL__SINK) && (++retryTimes) < MAX_RETRY_TIMES) { - taosMsleep(10); + taosMsleep(40); continue; } From d2160efa05cf62c516d55034a379ca9a9136000b Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 3 Oct 2023 20:57:19 +0800 Subject: [PATCH 126/158] refactor: wait for 100ms before scan wal again. --- source/dnode/vnode/src/tq/tqStreamTask.c | 8 ++++---- source/libs/stream/src/streamQueue.c | 5 +++-- 2 files changed, 7 insertions(+), 6 deletions(-) diff --git a/source/dnode/vnode/src/tq/tqStreamTask.c b/source/dnode/vnode/src/tq/tqStreamTask.c index 2040f8e323..6448e9d2f7 100644 --- a/source/dnode/vnode/src/tq/tqStreamTask.c +++ b/source/dnode/vnode/src/tq/tqStreamTask.c @@ -17,6 +17,7 @@ #include "vnd.h" #define MAX_REPEAT_SCAN_THRESHOLD 3 +#define SCAN_WAL_IDLE_DURATION 100 static int32_t doScanWalForAllTasks(SStreamMeta* pStreamMeta, bool* pScanIdle); static int32_t setWalReaderStartOffset(SStreamTask* pTask, int32_t vgId); @@ -36,12 +37,10 @@ int32_t tqScanWal(STQ* pTq) { bool shouldIdle = true; doScanWalForAllTasks(pTq->pStreamMeta, &shouldIdle); - int32_t times = 0; - if (shouldIdle) { taosWLockLatch(&pMeta->lock); - times = (--pMeta->walScanCounter); + int32_t times = (--pMeta->walScanCounter); ASSERT(pMeta->walScanCounter >= 0); if (pMeta->walScanCounter <= 0) { @@ -50,7 +49,8 @@ int32_t tqScanWal(STQ* pTq) { } taosWUnLockLatch(&pMeta->lock); - tqDebug("vgId:%d scan wal for stream tasks for %d times", vgId, times); + tqDebug("vgId:%d scan wal for stream tasks for %d times in %dms", vgId, times, SCAN_WAL_IDLE_DURATION); + taosMsleep(SCAN_WAL_IDLE_DURATION); } } diff --git a/source/libs/stream/src/streamQueue.c b/source/libs/stream/src/streamQueue.c index 7f7c039423..887220f840 100644 --- a/source/libs/stream/src/streamQueue.c +++ b/source/libs/stream/src/streamQueue.c @@ -16,7 +16,8 @@ #include "streamInt.h" #define MAX_STREAM_EXEC_BATCH_NUM 32 -#define MAX_SMOOTH_BURST_RATIO 5 // 20 sec +#define MAX_SMOOTH_BURST_RATIO 5 // 20 sec +#define WAIT_FOR_DURATION 40 // todo refactor: // read data from input queue @@ -172,7 +173,7 @@ int32_t streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueItem** pInpu SStreamQueueItem* qItem = streamQueueNextItem(pTask->inputInfo.queue); if (qItem == NULL) { if ((taskLevel == TASK_LEVEL__SOURCE || taskLevel == TASK_LEVEL__SINK) && (++retryTimes) < MAX_RETRY_TIMES) { - taosMsleep(40); + taosMsleep(WAIT_FOR_DURATION); continue; } From cf14772dd9cb74716d8a56b177a4d276e56fff6e Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 4 Oct 2023 22:06:46 +0800 Subject: [PATCH 127/158] enh(stream): optimize the scan wal perf --- include/libs/stream/tstream.h | 3 +- source/dnode/vnode/src/tq/tqStreamTask.c | 154 +++++++++++++---------- source/libs/stream/inc/streamInt.h | 3 +- source/libs/stream/src/streamDispatch.c | 2 +- source/libs/stream/src/streamMeta.c | 4 +- source/libs/stream/src/streamQueue.c | 23 ++-- 6 files changed, 112 insertions(+), 77 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index a9da7d5ef8..bdc2c0a5ec 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -458,7 +458,7 @@ int32_t tDecodeStreamTaskId(SDecoder* pDecoder, STaskId* pTaskId); int32_t streamTaskPutDataIntoInputQ(SStreamTask* pTask, SStreamQueueItem* pItem); int32_t streamTaskPutDataIntoOutputQ(SStreamTask* pTask, SStreamDataBlock* pBlock); int32_t streamTaskPutTranstateIntoInputQ(SStreamTask* pTask); -bool streamQueueIsFull(const SStreamQueue* pQueue, bool inputQ); +bool streamQueueIsFull(const SStreamQueue* pQueue); typedef struct { SMsgHead head; @@ -706,6 +706,7 @@ int32_t streamTaskScanHistoryDataComplete(SStreamTask* pTask); int32_t streamStartScanHistoryAsync(SStreamTask* pTask, int8_t igUntreated); bool streamHistoryTaskSetVerRangeStep2(SStreamTask* pTask, int64_t latestVer); int32_t streamQueueGetNumOfItems(const SStreamQueue* pQueue); +int32_t streamQueueGetAvailableSpace(const SStreamQueue* pQueue, int32_t* availNum, double* availSize); // common int32_t streamRestoreParam(SStreamTask* pTask); diff --git a/source/dnode/vnode/src/tq/tqStreamTask.c b/source/dnode/vnode/src/tq/tqStreamTask.c index 6448e9d2f7..541d52d5b5 100644 --- a/source/dnode/vnode/src/tq/tqStreamTask.c +++ b/source/dnode/vnode/src/tq/tqStreamTask.c @@ -21,7 +21,7 @@ static int32_t doScanWalForAllTasks(SStreamMeta* pStreamMeta, bool* pScanIdle); static int32_t setWalReaderStartOffset(SStreamTask* pTask, int32_t vgId); -static void handleFillhistoryScanComplete(SStreamTask* pTask, int64_t ver); +static bool handleFillhistoryScanComplete(SStreamTask* pTask, int64_t ver); // extract data blocks(submit/delete) from WAL, and add them into the input queue for all the sources tasks. int32_t tqScanWal(STQ* pTq) { @@ -297,7 +297,7 @@ int32_t setWalReaderStartOffset(SStreamTask* pTask, int32_t vgId) { } // todo handle memory error -void handleFillhistoryScanComplete(SStreamTask* pTask, int64_t ver) { +bool handleFillhistoryScanComplete(SStreamTask* pTask, int64_t ver) { const char* id = pTask->id.idStr; int64_t maxVer = pTask->dataRange.range.maxVer; @@ -310,12 +310,94 @@ void handleFillhistoryScanComplete(SStreamTask* pTask, int64_t ver) { double el = (taosGetTimestampMs() - pTask->execInfo.step2Start) / 1000.0; qDebug("s-task:%s scan-history from WAL stage(step 2) ended, elapsed time:%.2fs", id, el); /*int32_t code = */streamTaskPutTranstateIntoInputQ(pTask); - /*int32_t code = */streamSchedExec(pTask); +// /*int32_t code = */streamSchedExec(pTask); + return true; } else { qWarn("s-task:%s fill-history scan WAL, nextProcessVer:%" PRId64 " out of the maximum ver:%" PRId64 ", not scan wal", id, ver, maxVer); } } + + return false; +} + +static bool taskReadyForDataFromWal(SStreamTask* pTask) { + // non-source or fill-history tasks don't need to response the WAL scan action. + if ((pTask->info.taskLevel != TASK_LEVEL__SOURCE) || (pTask->status.downstreamReady == 0)) { + return false; + } + + // not in ready state, do not handle the data from wal + int32_t status = pTask->status.taskStatus; + if (status != TASK_STATUS__NORMAL) { + tqTrace("s-task:%s not ready for submit block in wal, status:%s", pTask->id.idStr, streamGetTaskStatusStr(status)); + return false; + } + + // fill-history task has entered into the last phase, no need to anything + if ((pTask->info.fillHistory == 1) && pTask->status.appendTranstateBlock) { + ASSERT(status == TASK_STATUS__NORMAL); + // the maximum version of data in the WAL has reached already, the step2 is done + tqDebug("s-task:%s fill-history reach the maximum ver:%" PRId64 ", not scan wal anymore", pTask->id.idStr, + pTask->dataRange.range.maxVer); + return false; + } + + // check if input queue is full or not + if (streamQueueIsFull(pTask->inputInfo.queue)) { + tqTrace("s-task:%s input queue is full, do nothing", pTask->id.idStr); + return false; + } + + // the input queue of downstream task is full, so the output is blocked, stopped for a while + if (pTask->inputInfo.status == TASK_INPUT_STATUS__BLOCKED) { + tqDebug("s-task:%s inputQ is blocked, do nothing", pTask->id.idStr); + return false; + } + + return true; +} + +static bool doPutDataIntoInputQFromWal(SStreamTask* pTask, int64_t maxVer, int32_t* numOfItems) { + const char* id = pTask->id.idStr; + int32_t numOfNewItems = 0; + + while(1) { + if ((pTask->info.fillHistory == 1) && pTask->status.appendTranstateBlock) { + *numOfItems += numOfNewItems; + return numOfNewItems > 0; + } + + SStreamQueueItem* pItem = NULL; + int32_t code = extractMsgFromWal(pTask->exec.pWalReader, (void**)&pItem, maxVer, id); + + if ((code != TSDB_CODE_SUCCESS || pItem == NULL)/* && (numOfItems + numOfNewItems == 0)*/) { // failed, continue +// handleFillhistoryScanComplete(pTask, walReaderGetCurrentVer(pTask->exec.pWalReader)); +// streamMetaReleaseTask(pMeta, pTask); +// taosThreadMutexUnlock(&pTask->lock); + break; + } + + if (pItem != NULL) { + code = streamTaskPutDataIntoInputQ(pTask, pItem); + if (code == TSDB_CODE_SUCCESS) { + numOfNewItems += 1; + int64_t ver = walReaderGetCurrentVer(pTask->exec.pWalReader); + pTask->chkInfo.nextProcessVer = ver; + tqDebug("s-task:%s set the ver:%" PRId64 " from WALReader after extract block from WAL", id, ver); + + bool itemInFillhistory = handleFillhistoryScanComplete(pTask, ver); + if (itemInFillhistory) { + break; + } + } else { + tqError("s-task:%s append input queue failed, code: too many items, ver:%" PRId64, id, pTask->chkInfo.nextProcessVer); + } + } + } + + *numOfItems += numOfNewItems; + return numOfNewItems > 0; } int32_t doScanWalForAllTasks(SStreamMeta* pStreamMeta, bool* pScanIdle) { @@ -340,45 +422,13 @@ int32_t doScanWalForAllTasks(SStreamMeta* pStreamMeta, bool* pScanIdle) { numOfTasks = taosArrayGetSize(pTaskList); for (int32_t i = 0; i < numOfTasks; ++i) { - SStreamTaskId* pTaskId = taosArrayGet(pTaskList, i); + STaskId* pTaskId = taosArrayGet(pTaskList, i); SStreamTask* pTask = streamMetaAcquireTask(pStreamMeta, pTaskId->streamId, pTaskId->taskId); if (pTask == NULL) { continue; } - int32_t status = pTask->status.taskStatus; - - // non-source or fill-history tasks don't need to response the WAL scan action. - if ((pTask->info.taskLevel != TASK_LEVEL__SOURCE) || (pTask->status.downstreamReady == 0)) { - streamMetaReleaseTask(pStreamMeta, pTask); - continue; - } - - const char* pStatus = streamGetTaskStatusStr(status); - if (status != TASK_STATUS__NORMAL) { - tqTrace("s-task:%s not ready for new submit block from wal, status:%s", pTask->id.idStr, pStatus); - streamMetaReleaseTask(pStreamMeta, pTask); - continue; - } - - if ((pTask->info.fillHistory == 1) && pTask->status.appendTranstateBlock) { - ASSERT(status == TASK_STATUS__NORMAL); - // the maximum version of data in the WAL has reached already, the step2 is done - tqDebug("s-task:%s fill-history reach the maximum ver:%" PRId64 ", not scan wal anymore", pTask->id.idStr, - pTask->dataRange.range.maxVer); - streamMetaReleaseTask(pStreamMeta, pTask); - continue; - } - - if (streamQueueIsFull(pTask->inputInfo.queue, true)) { - tqTrace("s-task:%s input queue is full, do nothing", pTask->id.idStr); - streamMetaReleaseTask(pStreamMeta, pTask); - continue; - } - - // downstream task has blocked the output, stopped for a while - if (pTask->inputInfo.status == TASK_INPUT_STATUS__BLOCKED) { - tqDebug("s-task:%s inputQ is blocked, do nothing", pTask->id.idStr); + if (!taskReadyForDataFromWal(pTask)) { streamMetaReleaseTask(pStreamMeta, pTask); continue; } @@ -397,7 +447,7 @@ int32_t doScanWalForAllTasks(SStreamMeta* pStreamMeta, bool* pScanIdle) { taosThreadMutexLock(&pTask->lock); - pStatus = streamGetTaskStatusStr(pTask->status.taskStatus); + const char* pStatus = streamGetTaskStatusStr(pTask->status.taskStatus); if (pTask->status.taskStatus != TASK_STATUS__NORMAL) { tqDebug("s-task:%s not ready for submit block from wal, status:%s", pTask->id.idStr, pStatus); taosThreadMutexUnlock(&pTask->lock); @@ -405,33 +455,11 @@ int32_t doScanWalForAllTasks(SStreamMeta* pStreamMeta, bool* pScanIdle) { continue; } - SStreamQueueItem* pItem = NULL; - code = extractMsgFromWal(pTask->exec.pWalReader, (void**)&pItem, maxVer, pTask->id.idStr); - - if ((code != TSDB_CODE_SUCCESS || pItem == NULL) && (numOfItems == 0)) { // failed, continue - handleFillhistoryScanComplete(pTask, walReaderGetCurrentVer(pTask->exec.pWalReader)); - streamMetaReleaseTask(pStreamMeta, pTask); - taosThreadMutexUnlock(&pTask->lock); - continue; - } - - if (pItem != NULL) { - noDataInWal = false; - code = streamTaskPutDataIntoInputQ(pTask, pItem); - if (code == TSDB_CODE_SUCCESS) { - int64_t ver = walReaderGetCurrentVer(pTask->exec.pWalReader); - pTask->chkInfo.nextProcessVer = ver; - handleFillhistoryScanComplete(pTask, ver); - tqDebug("s-task:%s set the ver:%" PRId64 " from WALReader after extract block from WAL", pTask->id.idStr, ver); - } else { - tqError("s-task:%s append input queue failed, too many in inputQ, ver:%" PRId64, pTask->id.idStr, - pTask->chkInfo.nextProcessVer); - } - } - + bool hasNewData = doPutDataIntoInputQFromWal(pTask, maxVer, &numOfItems); taosThreadMutexUnlock(&pTask->lock); - if ((code == TSDB_CODE_SUCCESS) || (numOfItems > 0)) { + if (/*(code == TSDB_CODE_SUCCESS) || */(numOfItems > 0) || hasNewData) { + noDataInWal = false; code = streamSchedExec(pTask); if (code != TSDB_CODE_SUCCESS) { streamMetaReleaseTask(pStreamMeta, pTask); diff --git a/source/libs/stream/inc/streamInt.h b/source/libs/stream/inc/streamInt.h index fe4d73b566..2912c2954d 100644 --- a/source/libs/stream/inc/streamInt.h +++ b/source/libs/stream/inc/streamInt.h @@ -41,8 +41,7 @@ extern "C" { #define STREAM_TASK_KEY_LEN ((sizeof(int64_t)) << 1) #define STREAM_TASK_QUEUE_CAPACITY 20480 -#define STREAM_TASK_INPUT_QUEUE_CAPACITY_IN_SIZE (30) -#define STREAM_TASK_OUTPUT_QUEUE_CAPACITY_IN_SIZE (50) +#define STREAM_TASK_QUEUE_CAPACITY_IN_SIZE (30) // clang-format off #define stFatal(...) do { if (stDebugFlag & DEBUG_FATAL) { taosPrintLog("STM FATAL ", DEBUG_FATAL, 255, __VA_ARGS__); }} while(0) diff --git a/source/libs/stream/src/streamDispatch.c b/source/libs/stream/src/streamDispatch.c index 5b76354dff..7b23366c53 100644 --- a/source/libs/stream/src/streamDispatch.c +++ b/source/libs/stream/src/streamDispatch.c @@ -603,7 +603,6 @@ int32_t streamDispatchStreamBlock(SStreamTask* pTask) { ASSERT(pBlock->type == STREAM_INPUT__DATA_BLOCK || pBlock->type == STREAM_INPUT__CHECKPOINT_TRIGGER || pBlock->type == STREAM_INPUT__TRANS_STATE); - int32_t retryCount = 0; pTask->execInfo.dispatch += 1; pTask->msgInfo.startTs = taosGetTimestampMs(); @@ -613,6 +612,7 @@ int32_t streamDispatchStreamBlock(SStreamTask* pTask) { } else { // todo handle build dispatch msg failed } + int32_t retryCount = 0; while (1) { code = sendDispatchMsg(pTask, pTask->msgInfo.pData); if (code == TSDB_CODE_SUCCESS) { diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index aaffd9cfba..2d8b46f628 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -891,8 +891,8 @@ void metaHbToMnode(void* param, void* tmrId) { .outputQUsed = SIZE_IN_MiB(streamQueueGetItemSize((*pTask)->outputInfo.queue)), }; - entry.inputRate = entry.inputQUsed*100.0/STREAM_TASK_INPUT_QUEUE_CAPACITY_IN_SIZE; - entry.outputRate = entry.outputQUsed*100.0/STREAM_TASK_OUTPUT_QUEUE_CAPACITY_IN_SIZE; + entry.inputRate = entry.inputQUsed*100.0/STREAM_TASK_QUEUE_CAPACITY_IN_SIZE; + entry.outputRate = entry.outputQUsed*100.0/STREAM_TASK_QUEUE_CAPACITY_IN_SIZE; if ((*pTask)->exec.pWalReader != NULL) { entry.offset = walReaderGetCurrentVer((*pTask)->exec.pWalReader); diff --git a/source/libs/stream/src/streamQueue.c b/source/libs/stream/src/streamQueue.c index 887220f840..ed2dd42bcb 100644 --- a/source/libs/stream/src/streamQueue.c +++ b/source/libs/stream/src/streamQueue.c @@ -102,14 +102,13 @@ void streamQueueProcessFail(SStreamQueue* queue) { atomic_store_8(&queue->status, STREAM_QUEUE__FAILED); } -bool streamQueueIsFull(const SStreamQueue* pQueue, bool inputQ) { +bool streamQueueIsFull(const SStreamQueue* pQueue) { int32_t numOfItems = streamQueueGetNumOfItems(pQueue); if (numOfItems >= STREAM_TASK_QUEUE_CAPACITY) { return true; } - int32_t threshold = (inputQ) ? STREAM_TASK_INPUT_QUEUE_CAPACITY_IN_SIZE : STREAM_TASK_OUTPUT_QUEUE_CAPACITY_IN_SIZE; - return (SIZE_IN_MiB(taosQueueMemorySize(pQueue->pQueue)) >= threshold); + return (SIZE_IN_MiB(taosQueueMemorySize(pQueue->pQueue)) >= STREAM_TASK_QUEUE_CAPACITY_IN_SIZE); } int32_t streamQueueGetNumOfItems(const SStreamQueue* pQueue) { @@ -119,6 +118,14 @@ int32_t streamQueueGetNumOfItems(const SStreamQueue* pQueue) { return numOfItems1 + numOfItems2; } +int32_t streamQueueGetAvailableSpace(const SStreamQueue* pQueue, int32_t* availNum, double* availSize) { + int32_t num = streamQueueGetNumOfItems(pQueue); + *availNum = STREAM_TASK_QUEUE_CAPACITY - num; + + *availSize = STREAM_TASK_QUEUE_CAPACITY_IN_SIZE - taosQueueMemorySize(pQueue->pQueue); + return 0; +} + // todo: fix it: data in Qall is not included here int32_t streamQueueGetItemSize(const SStreamQueue* pQueue) { return taosQueueMemorySize(pQueue->pQueue); @@ -264,11 +271,11 @@ int32_t streamTaskPutDataIntoInputQ(SStreamTask* pTask, SStreamQueueItem* pItem) if (type == STREAM_INPUT__DATA_SUBMIT) { SStreamDataSubmit* px = (SStreamDataSubmit*)pItem; - if ((pTask->info.taskLevel == TASK_LEVEL__SOURCE) && streamQueueIsFull(pTask->inputInfo.queue, true)) { + if ((pTask->info.taskLevel == TASK_LEVEL__SOURCE) && streamQueueIsFull(pTask->inputInfo.queue)) { double size = SIZE_IN_MiB(taosQueueMemorySize(pQueue)); stTrace( "s-task:%s inputQ is full, capacity(size:%d num:%dMiB), current(blocks:%d, size:%.2fMiB) stop to push data", - pTask->id.idStr, STREAM_TASK_QUEUE_CAPACITY, STREAM_TASK_INPUT_QUEUE_CAPACITY_IN_SIZE, total, size); + pTask->id.idStr, STREAM_TASK_QUEUE_CAPACITY, STREAM_TASK_QUEUE_CAPACITY_IN_SIZE, total, size); streamDataSubmitDestroy(px); taosFreeQitem(pItem); return -1; @@ -291,11 +298,11 @@ int32_t streamTaskPutDataIntoInputQ(SStreamTask* pTask, SStreamQueueItem* pItem) msgLen, ver, total, size + SIZE_IN_MiB(msgLen)); } else if (type == STREAM_INPUT__DATA_BLOCK || type == STREAM_INPUT__DATA_RETRIEVE || type == STREAM_INPUT__REF_DATA_BLOCK) { - if (streamQueueIsFull(pTask->inputInfo.queue, true)) { + if (streamQueueIsFull(pTask->inputInfo.queue)) { double size = SIZE_IN_MiB(taosQueueMemorySize(pQueue)); stTrace("s-task:%s input queue is full, capacity:%d size:%d MiB, current(blocks:%d, size:%.2fMiB) abort", - pTask->id.idStr, STREAM_TASK_QUEUE_CAPACITY, STREAM_TASK_INPUT_QUEUE_CAPACITY_IN_SIZE, total, size); + pTask->id.idStr, STREAM_TASK_QUEUE_CAPACITY, STREAM_TASK_QUEUE_CAPACITY_IN_SIZE, total, size); destroyStreamDataBlock((SStreamDataBlock*)pItem); return -1; } @@ -345,7 +352,7 @@ int32_t streamTaskPutDataIntoInputQ(SStreamTask* pTask, SStreamQueueItem* pItem) int32_t streamTaskPutDataIntoOutputQ(SStreamTask* pTask, SStreamDataBlock* pBlock) { STaosQueue* pQueue = pTask->outputInfo.queue->pQueue; - while (streamQueueIsFull(pTask->inputInfo.queue, false)) { + while (streamQueueIsFull(pTask->inputInfo.queue)) { if (streamTaskShouldStop(&pTask->status)) { stInfo("s-task:%s discard result block due to task stop", pTask->id.idStr); return TSDB_CODE_STREAM_EXEC_CANCELLED; From 65daf3871568f9a45a6d4070a55841db527c826d Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 4 Oct 2023 23:33:03 +0800 Subject: [PATCH 128/158] refactor(stream): idle for 100ms when starting next round scan wal. --- source/dnode/vnode/src/tq/tqStreamTask.c | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/source/dnode/vnode/src/tq/tqStreamTask.c b/source/dnode/vnode/src/tq/tqStreamTask.c index 541d52d5b5..3685435e34 100644 --- a/source/dnode/vnode/src/tq/tqStreamTask.c +++ b/source/dnode/vnode/src/tq/tqStreamTask.c @@ -50,8 +50,9 @@ int32_t tqScanWal(STQ* pTq) { taosWUnLockLatch(&pMeta->lock); tqDebug("vgId:%d scan wal for stream tasks for %d times in %dms", vgId, times, SCAN_WAL_IDLE_DURATION); - taosMsleep(SCAN_WAL_IDLE_DURATION); } + + taosMsleep(SCAN_WAL_IDLE_DURATION); } int64_t el = (taosGetTimestampMs() - st); From b3b9e89d30457e0df45314f1a8c4c02c57c07add Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 5 Oct 2023 00:09:47 +0800 Subject: [PATCH 129/158] fix(stream): fix the invalid check. --- 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 ed2dd42bcb..7f12d471bc 100644 --- a/source/libs/stream/src/streamQueue.c +++ b/source/libs/stream/src/streamQueue.c @@ -352,7 +352,7 @@ int32_t streamTaskPutDataIntoInputQ(SStreamTask* pTask, SStreamQueueItem* pItem) int32_t streamTaskPutDataIntoOutputQ(SStreamTask* pTask, SStreamDataBlock* pBlock) { STaosQueue* pQueue = pTask->outputInfo.queue->pQueue; - while (streamQueueIsFull(pTask->inputInfo.queue)) { + while (streamQueueIsFull(pTask->outputInfo.queue)) { if (streamTaskShouldStop(&pTask->status)) { stInfo("s-task:%s discard result block due to task stop", pTask->id.idStr); return TSDB_CODE_STREAM_EXEC_CANCELLED; From 28008f8263869dbac63609c761a24a96588592b0 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sat, 7 Oct 2023 01:14:00 +0800 Subject: [PATCH 130/158] enh(stream): add more info for sink task. --- include/libs/stream/tstream.h | 4 ++-- source/dnode/mnode/impl/src/mndStream.c | 18 +++++++++++----- source/dnode/vnode/src/tq/tqSink.c | 26 ++++++++++++------------ source/dnode/vnode/src/tq/tqStreamTask.c | 7 ++----- source/libs/stream/inc/streamInt.h | 4 ++-- source/libs/stream/src/streamMeta.c | 17 ++++++++++------ source/libs/stream/src/streamQueue.c | 2 +- 7 files changed, 44 insertions(+), 34 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index bdc2c0a5ec..a67199a7d6 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -602,8 +602,8 @@ typedef struct STaskStatusEntry { int64_t offset; // only valid for source task double inputQUsed; // in MiB double inputRate; - double outputQUsed; // in MiB - double outputRate; + double sinkQuota; // existed quota size for sink task + double sinkDataSize; // sink to dest data size } STaskStatusEntry; typedef struct SStreamHbMsg { diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index 69c588d1bf..698b11cb7f 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -1602,9 +1602,15 @@ static int32_t mndRetrieveStreamTask(SRpcMsg *pReq, SShowObj *pShow, SSDataBlock // pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); // colDataSetVal(pColInfo, numOfRows, (const char*)vbuf, false); - // offset version info - const char* offsetStr = "%"PRId64"[%"PRId64",%"PRId64"]"; - sprintf(buf, offsetStr, pe->offset, pe->verStart, pe->verEnd); + if (pTask->info.taskLevel == TASK_LEVEL__SINK) { + const char* sinkStr = "Quota:%2.fMiB, SinkData:%.2fMiB"; + sprintf(buf, sinkStr, pe->sinkQuota, pe->sinkDataSize); + } else if (pTask->info.taskLevel == TASK_LEVEL__SOURCE) { + // offset version info + const char *offsetStr = "%" PRId64 " [%" PRId64 ", %" PRId64 "]"; + sprintf(buf, offsetStr, pe->offset, pe->verStart, pe->verEnd); + } + STR_TO_VARSTR(vbuf, buf); pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); @@ -2458,11 +2464,13 @@ int32_t mndProcessStreamHb(SRpcMsg *pReq) { pEntry->stage = p->stage; pEntry->inputQUsed = p->inputQUsed; pEntry->inputRate = p->inputRate; - pEntry->outputQUsed = p->outputQUsed; - pEntry->outputRate = p->outputRate; +// pEntry->outputQUsed = p->outputQUsed; +// pEntry->outputRate = p->outputRate; pEntry->offset = p->offset; pEntry->verStart = p->verStart; pEntry->verEnd = p->verEnd; + pEntry->sinkQuota = p->sinkQuota; + pEntry->sinkDataSize = p->sinkDataSize; } pEntry->status = p->status; diff --git a/source/dnode/vnode/src/tq/tqSink.c b/source/dnode/vnode/src/tq/tqSink.c index 8009eccb1b..23b5aff7fa 100644 --- a/source/dnode/vnode/src/tq/tqSink.c +++ b/source/dnode/vnode/src/tq/tqSink.c @@ -24,6 +24,7 @@ typedef struct STableSinkInfo { tstr name; } STableSinkInfo; +static bool hasOnlySubmitData(const SArray* pBlocks, int32_t numOfBlocks); static int32_t tsAscendingSortFn(const void* p1, const void* p2); static int32_t setDstTableDataUid(SVnode* pVnode, SStreamTask* pTask, SSDataBlock* pDataBlock, char* stbFullName, SSubmitTbData* pTableData); @@ -744,6 +745,17 @@ int32_t setDstTableDataPayload(SStreamTask* pTask, int32_t blockIndex, SSDataBlo return code; } +bool hasOnlySubmitData(const SArray* pBlocks, int32_t numOfBlocks) { + for(int32_t i = 0; i < numOfBlocks; ++i) { + SSDataBlock* p = taosArrayGet(pBlocks, i); + if (p->info.type == STREAM_DELETE_RESULT || p->info.type == STREAM_CREATE_CHILD_TABLE) { + return false; + } + } + + return true; +} + void tqSinkDataIntoDstTable(SStreamTask* pTask, void* vnode, void* data) { const SArray* pBlocks = (const SArray*)data; SVnode* pVnode = (SVnode*)vnode; @@ -755,19 +767,7 @@ void tqSinkDataIntoDstTable(SStreamTask* pTask, void* vnode, void* data) { int32_t code = TSDB_CODE_SUCCESS; const char* id = pTask->id.idStr; - if (pTask->execInfo.start == 0) { - pTask->execInfo.start = taosGetTimestampMs(); - } - - bool onlySubmitData = true; - for(int32_t i = 0; i < numOfBlocks; ++i) { - SSDataBlock* p = taosArrayGet(pBlocks, i); - if (p->info.type == STREAM_DELETE_RESULT || p->info.type == STREAM_CREATE_CHILD_TABLE) { - onlySubmitData = false; - break; - } - } - + bool onlySubmitData = hasOnlySubmitData(pBlocks, numOfBlocks); if (!onlySubmitData) { tqDebug("vgId:%d, s-task:%s write %d stream resBlock(s) into table, has delete block, submit one-by-one", vgId, id, numOfBlocks); diff --git a/source/dnode/vnode/src/tq/tqStreamTask.c b/source/dnode/vnode/src/tq/tqStreamTask.c index 3685435e34..8bee7d80a2 100644 --- a/source/dnode/vnode/src/tq/tqStreamTask.c +++ b/source/dnode/vnode/src/tq/tqStreamTask.c @@ -372,10 +372,7 @@ static bool doPutDataIntoInputQFromWal(SStreamTask* pTask, int64_t maxVer, int32 SStreamQueueItem* pItem = NULL; int32_t code = extractMsgFromWal(pTask->exec.pWalReader, (void**)&pItem, maxVer, id); - if ((code != TSDB_CODE_SUCCESS || pItem == NULL)/* && (numOfItems + numOfNewItems == 0)*/) { // failed, continue -// handleFillhistoryScanComplete(pTask, walReaderGetCurrentVer(pTask->exec.pWalReader)); -// streamMetaReleaseTask(pMeta, pTask); -// taosThreadMutexUnlock(&pTask->lock); + if (code != TSDB_CODE_SUCCESS || pItem == NULL) { // failed, continue break; } @@ -459,7 +456,7 @@ int32_t doScanWalForAllTasks(SStreamMeta* pStreamMeta, bool* pScanIdle) { bool hasNewData = doPutDataIntoInputQFromWal(pTask, maxVer, &numOfItems); taosThreadMutexUnlock(&pTask->lock); - if (/*(code == TSDB_CODE_SUCCESS) || */(numOfItems > 0) || hasNewData) { + if ((numOfItems > 0) || hasNewData) { noDataInWal = false; code = streamSchedExec(pTask); if (code != TSDB_CODE_SUCCESS) { diff --git a/source/libs/stream/inc/streamInt.h b/source/libs/stream/inc/streamInt.h index 2912c2954d..3ee88aaa1b 100644 --- a/source/libs/stream/inc/streamInt.h +++ b/source/libs/stream/inc/streamInt.h @@ -40,8 +40,8 @@ extern "C" { #define META_HB_SEND_IDLE_COUNTER 25 // send hb every 5 sec #define STREAM_TASK_KEY_LEN ((sizeof(int64_t)) << 1) -#define STREAM_TASK_QUEUE_CAPACITY 20480 -#define STREAM_TASK_QUEUE_CAPACITY_IN_SIZE (30) +#define STREAM_TASK_QUEUE_CAPACITY 20480 +#define STREAM_TASK_QUEUE_CAPACITY_IN_SIZE (30) // clang-format off #define stFatal(...) do { if (stDebugFlag & DEBUG_FATAL) { taosPrintLog("STM FATAL ", DEBUG_FATAL, 255, __VA_ARGS__); }} while(0) diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 2d8b46f628..f2f8bc15a8 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -777,8 +777,8 @@ int32_t tEncodeStreamHbMsg(SEncoder* pEncoder, const SStreamHbMsg* pReq) { if (tEncodeI32(pEncoder, ps->nodeId) < 0) return -1; if (tEncodeDouble(pEncoder, ps->inputQUsed) < 0) return -1; if (tEncodeDouble(pEncoder, ps->inputRate) < 0) return -1; - if (tEncodeDouble(pEncoder, ps->outputQUsed) < 0) return -1; - if (tEncodeDouble(pEncoder, ps->outputRate) < 0) return -1; + if (tEncodeDouble(pEncoder, ps->sinkQuota) < 0) return -1; + if (tEncodeDouble(pEncoder, ps->sinkDataSize) < 0) return -1; if (tEncodeI64(pEncoder, ps->offset) < 0) return -1; if (tEncodeI64(pEncoder, ps->verStart) < 0) return -1; if (tEncodeI64(pEncoder, ps->verEnd) < 0) return -1; @@ -804,8 +804,8 @@ int32_t tDecodeStreamHbMsg(SDecoder* pDecoder, SStreamHbMsg* pReq) { if (tDecodeI32(pDecoder, &entry.nodeId) < 0) return -1; if (tDecodeDouble(pDecoder, &entry.inputQUsed) < 0) return -1; if (tDecodeDouble(pDecoder, &entry.inputRate) < 0) return -1; - if (tDecodeDouble(pDecoder, &entry.outputQUsed) < 0) return -1; - if (tDecodeDouble(pDecoder, &entry.outputRate) < 0) return -1; + if (tDecodeDouble(pDecoder, &entry.sinkQuota) < 0) return -1; + if (tDecodeDouble(pDecoder, &entry.sinkDataSize) < 0) return -1; if (tDecodeI64(pDecoder, &entry.offset) < 0) return -1; if (tDecodeI64(pDecoder, &entry.verStart) < 0) return -1; if (tDecodeI64(pDecoder, &entry.verEnd) < 0) return -1; @@ -888,11 +888,16 @@ void metaHbToMnode(void* param, void* tmrId) { .nodeId = pMeta->vgId, .stage = pMeta->stage, .inputQUsed = SIZE_IN_MiB(streamQueueGetItemSize((*pTask)->inputInfo.queue)), - .outputQUsed = SIZE_IN_MiB(streamQueueGetItemSize((*pTask)->outputInfo.queue)), +// .outputQUsed = SIZE_IN_MiB(streamQueueGetItemSize((*pTask)->outputInfo.queue)), }; entry.inputRate = entry.inputQUsed*100.0/STREAM_TASK_QUEUE_CAPACITY_IN_SIZE; - entry.outputRate = entry.outputQUsed*100.0/STREAM_TASK_QUEUE_CAPACITY_IN_SIZE; + if ((*pTask)->info.taskLevel == TASK_LEVEL__SINK) { + entry.sinkQuota = (*pTask)->pTokenBucket->bytesRate; + entry.sinkDataSize = SIZE_IN_MiB((*pTask)->execInfo.sink.dataSize); + } + +// entry.outputRate = entry.outputQUsed*100.0/STREAM_TASK_QUEUE_CAPACITY_IN_SIZE; if ((*pTask)->exec.pWalReader != NULL) { entry.offset = walReaderGetCurrentVer((*pTask)->exec.pWalReader); diff --git a/source/libs/stream/src/streamQueue.c b/source/libs/stream/src/streamQueue.c index 7f12d471bc..2975d1f0f3 100644 --- a/source/libs/stream/src/streamQueue.c +++ b/source/libs/stream/src/streamQueue.c @@ -16,7 +16,7 @@ #include "streamInt.h" #define MAX_STREAM_EXEC_BATCH_NUM 32 -#define MAX_SMOOTH_BURST_RATIO 5 // 20 sec +#define MAX_SMOOTH_BURST_RATIO 5 // 5 sec #define WAIT_FOR_DURATION 40 // todo refactor: From f0c5b5a9705ff2b2481c1a0c8ba349e53a4a0875 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sat, 7 Oct 2023 09:04:38 +0800 Subject: [PATCH 131/158] refactor(stream): revised the display info. --- source/dnode/mnode/impl/src/mndStream.c | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index 698b11cb7f..6bc0d46475 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -1603,10 +1603,10 @@ static int32_t mndRetrieveStreamTask(SRpcMsg *pReq, SShowObj *pShow, SSDataBlock // colDataSetVal(pColInfo, numOfRows, (const char*)vbuf, false); if (pTask->info.taskLevel == TASK_LEVEL__SINK) { - const char* sinkStr = "Quota:%2.fMiB, SinkData:%.2fMiB"; + const char* sinkStr = "SinkData:%.2fMiB"; sprintf(buf, sinkStr, pe->sinkQuota, pe->sinkDataSize); } else if (pTask->info.taskLevel == TASK_LEVEL__SOURCE) { - // offset version info + // offset info const char *offsetStr = "%" PRId64 " [%" PRId64 ", %" PRId64 "]"; sprintf(buf, offsetStr, pe->offset, pe->verStart, pe->verEnd); } From c5997bee5b394774b5570570d35d6669e636de11 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sat, 7 Oct 2023 09:07:44 +0800 Subject: [PATCH 132/158] fix(stream): update the current processing version. --- source/libs/stream/src/streamMeta.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index f2f8bc15a8..358a75b4d9 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -900,7 +900,7 @@ void metaHbToMnode(void* param, void* tmrId) { // entry.outputRate = entry.outputQUsed*100.0/STREAM_TASK_QUEUE_CAPACITY_IN_SIZE; if ((*pTask)->exec.pWalReader != NULL) { - entry.offset = walReaderGetCurrentVer((*pTask)->exec.pWalReader); + entry.offset = (*pTask)->chkInfo.nextProcessVer; walReaderValidVersionRange((*pTask)->exec.pWalReader, &entry.verStart, &entry.verEnd); } From 11d8c8da39b1b8c625129d02632961c4d007d75d Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sat, 7 Oct 2023 09:22:25 +0800 Subject: [PATCH 133/158] fix(stream): update the sink data. --- source/common/src/systable.c | 2 +- source/dnode/mnode/impl/src/mndStream.c | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/source/common/src/systable.c b/source/common/src/systable.c index 19e8945cdf..cc3f8d20b6 100644 --- a/source/common/src/systable.c +++ b/source/common/src/systable.c @@ -166,7 +166,7 @@ static const SSysDbTableSchema streamTaskSchema[] = { {.name = "stage", .bytes = 4, .type = TSDB_DATA_TYPE_INT, .sysInfo = false}, {.name = "in_queue", .bytes = 20, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = false}, // {.name = "out_queue", .bytes = 20, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = false}, - {.name = "offset", .bytes = 25, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = false}, + {.name = "info", .bytes = 25, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = false}, }; static const SSysDbTableSchema userTblsSchema[] = { diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index 6bc0d46475..e821e952e0 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -1588,7 +1588,7 @@ static int32_t mndRetrieveStreamTask(SRpcMsg *pReq, SShowObj *pShow, SSDataBlock // input queue char vbuf[30] = {0}; char buf[25] = {0}; - const char* queueInfoStr = "%4.2fMiB(%5.2f%)"; + const char* queueInfoStr = "%4.2fMiB (%5.2f%)"; sprintf(buf, queueInfoStr, pe->inputQUsed, pe->inputRate); STR_TO_VARSTR(vbuf, buf); @@ -1604,7 +1604,7 @@ static int32_t mndRetrieveStreamTask(SRpcMsg *pReq, SShowObj *pShow, SSDataBlock if (pTask->info.taskLevel == TASK_LEVEL__SINK) { const char* sinkStr = "SinkData:%.2fMiB"; - sprintf(buf, sinkStr, pe->sinkQuota, pe->sinkDataSize); + sprintf(buf, sinkStr, pe->sinkDataSize); } else if (pTask->info.taskLevel == TASK_LEVEL__SOURCE) { // offset info const char *offsetStr = "%" PRId64 " [%" PRId64 ", %" PRId64 "]"; From 9d5a3b8d781982b19176a231d87bcd1b977bbac1 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sat, 7 Oct 2023 09:56:03 +0800 Subject: [PATCH 134/158] other: merge stream fix. --- source/common/src/tglobal.c | 6 ++---- source/dnode/mnode/impl/src/mndStream.c | 2 +- source/libs/stream/src/streamMeta.c | 3 --- 3 files changed, 3 insertions(+), 8 deletions(-) diff --git a/source/common/src/tglobal.c b/source/common/src/tglobal.c index 1bbb969e7b..6e5eb478c3 100644 --- a/source/common/src/tglobal.c +++ b/source/common/src/tglobal.c @@ -241,7 +241,7 @@ int32_t tsTtlBatchDropNum = 10000; // number of tables dropped per batch // internal int32_t tsTransPullupInterval = 2; int32_t tsMqRebalanceInterval = 2; -int32_t tsStreamCheckpointTickInterval = 10; +int32_t tsStreamCheckpointTickInterval = 300; int32_t tsStreamNodeCheckInterval = 30; int32_t tsTtlUnit = 86400; int32_t tsTtlPushIntervalSec = 10; @@ -264,8 +264,6 @@ char tsS3BucketName[TSDB_FQDN_LEN] = ""; char tsS3AppId[TSDB_FQDN_LEN] = ""; int8_t tsS3Enabled = false; -int32_t tsCheckpointInterval = 20; - #ifndef _STORAGE int32_t taosSetTfsCfg(SConfig *pCfg) { SConfigItem *pItem = cfgGetItem(pCfg, "dataDir"); @@ -642,7 +640,7 @@ static int32_t taosAddServerCfg(SConfig *pCfg) { if (cfgAddBool(pCfg, "disableStream", tsDisableStream, CFG_SCOPE_SERVER) != 0) return -1; if (cfgAddInt64(pCfg, "streamBufferSize", tsStreamBufferSize, 0, INT64_MAX, CFG_SCOPE_SERVER) != 0) return -1; - if (cfgAddInt64(pCfg, "checkpointInterval", tsCheckpointInterval, 0, INT64_MAX, CFG_SCOPE_SERVER) != 0) return -1; + if (cfgAddInt64(pCfg, "checkpointInterval", tsStreamCheckpointTickInterval, 60, 1200, CFG_SCOPE_SERVER) != 0) return -1; if (cfgAddInt32(pCfg, "cacheLazyLoadThreshold", tsCacheLazyLoadThreshold, 0, 100000, CFG_SCOPE_SERVER) != 0) return -1; diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index e821e952e0..82ae6a846c 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -1603,7 +1603,7 @@ static int32_t mndRetrieveStreamTask(SRpcMsg *pReq, SShowObj *pShow, SSDataBlock // colDataSetVal(pColInfo, numOfRows, (const char*)vbuf, false); if (pTask->info.taskLevel == TASK_LEVEL__SINK) { - const char* sinkStr = "SinkData:%.2fMiB"; + const char* sinkStr = "%.2fMiB"; sprintf(buf, sinkStr, pe->sinkDataSize); } else if (pTask->info.taskLevel == TASK_LEVEL__SOURCE) { // offset info diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 358a75b4d9..2583e6427f 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -888,7 +888,6 @@ void metaHbToMnode(void* param, void* tmrId) { .nodeId = pMeta->vgId, .stage = pMeta->stage, .inputQUsed = SIZE_IN_MiB(streamQueueGetItemSize((*pTask)->inputInfo.queue)), -// .outputQUsed = SIZE_IN_MiB(streamQueueGetItemSize((*pTask)->outputInfo.queue)), }; entry.inputRate = entry.inputQUsed*100.0/STREAM_TASK_QUEUE_CAPACITY_IN_SIZE; @@ -897,8 +896,6 @@ void metaHbToMnode(void* param, void* tmrId) { entry.sinkDataSize = SIZE_IN_MiB((*pTask)->execInfo.sink.dataSize); } -// entry.outputRate = entry.outputQUsed*100.0/STREAM_TASK_QUEUE_CAPACITY_IN_SIZE; - if ((*pTask)->exec.pWalReader != NULL) { entry.offset = (*pTask)->chkInfo.nextProcessVer; walReaderValidVersionRange((*pTask)->exec.pWalReader, &entry.verStart, &entry.verEnd); From 59e284c332648aed74db7c8ebda6ac724a405dea Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sat, 7 Oct 2023 14:54:11 +0800 Subject: [PATCH 135/158] fix(stream): quit from loop when input queue is full. --- source/dnode/vnode/src/tq/tqStreamTask.c | 2 +- source/libs/stream/inc/streamInt.h | 8 ++++---- source/libs/stream/src/streamMeta.c | 2 +- source/libs/stream/src/streamQueue.c | 22 +++++++++++----------- 4 files changed, 17 insertions(+), 17 deletions(-) diff --git a/source/dnode/vnode/src/tq/tqStreamTask.c b/source/dnode/vnode/src/tq/tqStreamTask.c index 8bee7d80a2..3dbb957151 100644 --- a/source/dnode/vnode/src/tq/tqStreamTask.c +++ b/source/dnode/vnode/src/tq/tqStreamTask.c @@ -311,7 +311,6 @@ bool handleFillhistoryScanComplete(SStreamTask* pTask, int64_t ver) { double el = (taosGetTimestampMs() - pTask->execInfo.step2Start) / 1000.0; qDebug("s-task:%s scan-history from WAL stage(step 2) ended, elapsed time:%.2fs", id, el); /*int32_t code = */streamTaskPutTranstateIntoInputQ(pTask); -// /*int32_t code = */streamSchedExec(pTask); return true; } else { qWarn("s-task:%s fill-history scan WAL, nextProcessVer:%" PRId64 " out of the maximum ver:%" PRId64 ", not scan wal", @@ -390,6 +389,7 @@ static bool doPutDataIntoInputQFromWal(SStreamTask* pTask, int64_t maxVer, int32 } } else { tqError("s-task:%s append input queue failed, code: too many items, ver:%" PRId64, id, pTask->chkInfo.nextProcessVer); + break; } } } diff --git a/source/libs/stream/inc/streamInt.h b/source/libs/stream/inc/streamInt.h index 3ee88aaa1b..4cd8319a07 100644 --- a/source/libs/stream/inc/streamInt.h +++ b/source/libs/stream/inc/streamInt.h @@ -67,9 +67,9 @@ struct STokenBucket { int32_t numCapacity; // total capacity, available token per second int32_t numOfToken; // total available tokens int32_t numRate; // number of token per second - double bytesCapacity; // available capacity for maximum input size, KiloBytes per Second - double bytesRemain; // not consumed bytes per second - double bytesRate; // number of token per second + double quotaCapacity; // available capacity for maximum input size, KiloBytes per Second + double quotaRemain; // not consumed bytes per second + double quotaRate; // number of token per second int64_t fillTimestamp; // fill timestamp }; @@ -122,7 +122,7 @@ int32_t streamNotifyUpstreamContinue(SStreamTask* pTask); int32_t streamTaskFillHistoryFinished(SStreamTask* pTask); int32_t streamTransferStateToStreamTask(SStreamTask* pTask); -int32_t streamTaskInitTokenBucket(STokenBucket* pBucket, int32_t numCap, int32_t numRate, int32_t bytesRate); +int32_t streamTaskInitTokenBucket(STokenBucket* pBucket, int32_t numCap, int32_t numRate, int32_t quotaRate); STaskId streamTaskExtractKey(const SStreamTask* pTask); void streamTaskInitForLaunchHTask(SHistoryTaskInfo* pInfo); void streamTaskSetRetryInfoForLaunch(SHistoryTaskInfo* pInfo); diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 2583e6427f..85ce7289ba 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -892,7 +892,7 @@ void metaHbToMnode(void* param, void* tmrId) { entry.inputRate = entry.inputQUsed*100.0/STREAM_TASK_QUEUE_CAPACITY_IN_SIZE; if ((*pTask)->info.taskLevel == TASK_LEVEL__SINK) { - entry.sinkQuota = (*pTask)->pTokenBucket->bytesRate; + entry.sinkQuota = (*pTask)->pTokenBucket->quotaRate; entry.sinkDataSize = SIZE_IN_MiB((*pTask)->execInfo.sink.dataSize); } diff --git a/source/libs/stream/src/streamQueue.c b/source/libs/stream/src/streamQueue.c index 2975d1f0f3..8b595c2593 100644 --- a/source/libs/stream/src/streamQueue.c +++ b/source/libs/stream/src/streamQueue.c @@ -380,7 +380,7 @@ int32_t streamTaskPutDataIntoOutputQ(SStreamTask* pTask, SStreamDataBlock* pBloc return TSDB_CODE_SUCCESS; } -int32_t streamTaskInitTokenBucket(STokenBucket* pBucket, int32_t numCap, int32_t numRate, int32_t bytesRate) { +int32_t streamTaskInitTokenBucket(STokenBucket* pBucket, int32_t numCap, int32_t numRate, int32_t quotaRate) { if (numCap < 10 || numRate < 10 || pBucket == NULL) { stError("failed to init sink task bucket, cap:%d, rate:%d", numCap, numRate); return TSDB_CODE_INVALID_PARA; @@ -390,15 +390,15 @@ int32_t streamTaskInitTokenBucket(STokenBucket* pBucket, int32_t numCap, int32_t pBucket->numOfToken = numCap; pBucket->numRate = numRate; - pBucket->bytesRate = bytesRate; - pBucket->bytesCapacity = bytesRate * MAX_SMOOTH_BURST_RATIO; - pBucket->bytesRemain = pBucket->bytesCapacity; + pBucket->quotaRate = quotaRate; + pBucket->quotaCapacity = quotaRate * MAX_SMOOTH_BURST_RATIO; + pBucket->quotaRemain = pBucket->quotaCapacity; pBucket->fillTimestamp = taosGetTimestampMs(); return TSDB_CODE_SUCCESS; } -static void fillTokenBucket(STokenBucket* pBucket) { +static void fillTokenBucket(STokenBucket* pBucket, const char* id) { int64_t now = taosGetTimestampMs(); int64_t delta = now - pBucket->fillTimestamp; ASSERT(pBucket->numOfToken >= 0); @@ -410,15 +410,15 @@ static void fillTokenBucket(STokenBucket* pBucket) { } // increase the new available quota as time goes on - double incSize = (delta / 1000.0) * pBucket->bytesRate; + double incSize = (delta / 1000.0) * pBucket->quotaRate; if (incSize > 0) { - pBucket->bytesRemain = TMIN(pBucket->bytesRemain + incSize, pBucket->bytesCapacity); + pBucket->quotaRemain = TMIN(pBucket->quotaRemain + incSize, pBucket->quotaCapacity); } if (incNum > 0) { stDebug("new token and capacity available, current token:%d inc:%d, current quota:%.2fMiB inc:%.2fMiB, ts:%" PRId64 - " wait for %.2f Sec", - pBucket->numOfToken, incNum, pBucket->bytesRemain, incSize, now, delta / 1000.0); + " idle for %.2f Sec, %s", + pBucket->numOfToken, incNum, pBucket->quotaRemain, incSize, now, delta / 1000.0, id); } } @@ -426,7 +426,7 @@ bool streamTaskExtractAvailableToken(STokenBucket* pBucket) { fillTokenBucket(pBucket); if (pBucket->numOfToken > 0) { - if (pBucket->bytesRemain > 0) { + if (pBucket->quotaRemain > 0) { pBucket->numOfToken -= 1; return true; } else { // no available size quota now @@ -443,5 +443,5 @@ void streamTaskPutbackToken(STokenBucket* pBucket) { // size in KB void streamTaskConsumeQuota(STokenBucket* pBucket, int32_t bytes) { - pBucket->bytesRemain -= SIZE_IN_MiB(bytes); + pBucket->quotaRemain -= SIZE_IN_MiB(bytes); } \ No newline at end of file From e95aea84aeeb4044dd2ef7e3364c5eecec0d5617 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sat, 7 Oct 2023 14:56:09 +0800 Subject: [PATCH 136/158] fix(stream): fix syntax error. --- source/libs/stream/src/streamQueue.c | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/source/libs/stream/src/streamQueue.c b/source/libs/stream/src/streamQueue.c index 8b595c2593..ce90f29451 100644 --- a/source/libs/stream/src/streamQueue.c +++ b/source/libs/stream/src/streamQueue.c @@ -28,7 +28,7 @@ typedef struct SQueueReader { int32_t waitDuration; // maximum wait time to format several block into a batch to process, unit: ms } SQueueReader; -static bool streamTaskExtractAvailableToken(STokenBucket* pBucket); +static bool streamTaskExtractAvailableToken(STokenBucket* pBucket, const char* id); static void streamTaskPutbackToken(STokenBucket* pBucket); static void streamTaskConsumeQuota(STokenBucket* pBucket, int32_t bytes); @@ -166,7 +166,7 @@ int32_t streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueItem** pInpu *blockSize = 0; // no available token in bucket for sink task, let's wait for a little bit - if (taskLevel == TASK_LEVEL__SINK && (!streamTaskExtractAvailableToken(pTask->pTokenBucket))) { + if (taskLevel == TASK_LEVEL__SINK && (!streamTaskExtractAvailableToken(pTask->pTokenBucket, pTask->id.idStr))) { stDebug("s-task:%s no available token in bucket for sink data, wait", id); return TSDB_CODE_SUCCESS; } @@ -422,8 +422,8 @@ static void fillTokenBucket(STokenBucket* pBucket, const char* id) { } } -bool streamTaskExtractAvailableToken(STokenBucket* pBucket) { - fillTokenBucket(pBucket); +bool streamTaskExtractAvailableToken(STokenBucket* pBucket, const char* id) { + fillTokenBucket(pBucket, id); if (pBucket->numOfToken > 0) { if (pBucket->quotaRemain > 0) { From 940d6a3cb4473a409c012515c0f3739a23ecf96e Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sat, 7 Oct 2023 19:16:25 +0800 Subject: [PATCH 137/158] fix(stream): check items in queue and qall, stead of only the queue items. --- source/libs/stream/src/streamExec.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index dbceb83803..2302e00bb3 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -621,7 +621,7 @@ int32_t streamTryExec(SStreamTask* pTask) { } taosThreadMutexLock(&pTask->lock); - if (taosQueueEmpty(pTask->inputInfo.queue->pQueue) || streamTaskShouldStop(&pTask->status) || + if ((streamQueueGetNumOfItems(pTask->inputInfo.queue) > 0) || streamTaskShouldStop(&pTask->status) || streamTaskShouldPause(&pTask->status)) { atomic_store_8(&pTask->status.schedStatus, TASK_SCHED_STATUS__INACTIVE); taosThreadMutexUnlock(&pTask->lock); From af2c0c95e92317dd80fe2187b7d53e9d2bb25fbf Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sat, 7 Oct 2023 19:16:55 +0800 Subject: [PATCH 138/158] refactor: do some internal refactor. --- include/libs/stream/tstream.h | 68 +++++++++------- source/dnode/mnode/impl/src/mndScheduler.c | 22 ++--- source/dnode/snode/src/snode.c | 6 +- source/dnode/vnode/src/tq/tq.c | 26 +++--- source/dnode/vnode/src/tq/tqSink.c | 24 +++--- source/libs/stream/src/stream.c | 22 +++-- source/libs/stream/src/streamCheckpoint.c | 10 +-- source/libs/stream/src/streamDispatch.c | 94 +++++++++++----------- source/libs/stream/src/streamExec.c | 12 +-- source/libs/stream/src/streamMeta.c | 2 +- source/libs/stream/src/streamQueue.c | 40 ++++++--- source/libs/stream/src/streamRecover.c | 26 +++--- 12 files changed, 188 insertions(+), 164 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index a67199a7d6..5c5a2e6adb 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -290,6 +290,8 @@ typedef struct SSTaskBasicInfo { } SSTaskBasicInfo; typedef struct SStreamDispatchReq SStreamDispatchReq; +typedef struct STokenBucket STokenBucket; +typedef struct SMetaHbInfo SMetaHbInfo; typedef struct SDispatchMsgInfo { SStreamDispatchReq* pData; // current dispatch data @@ -301,11 +303,10 @@ typedef struct SDispatchMsgInfo { void* pTimer; // used to dispatch data after a given time duration } SDispatchMsgInfo; -typedef struct STaskOutputInfo { - int8_t type; +typedef struct STaskOutputQueue { int8_t status; SStreamQueue* queue; -} STaskOutputInfo; +} STaskOutputQueue; typedef struct STaskInputInfo { int8_t status; @@ -348,29 +349,7 @@ typedef struct SHistoryTaskInfo { int32_t waitInterval; } SHistoryTaskInfo; -typedef struct STokenBucket STokenBucket; -typedef struct SMetaHbInfo SMetaHbInfo; - -struct SStreamTask { - int64_t ver; - SStreamTaskId id; - SSTaskBasicInfo info; - STaskOutputInfo outputInfo; - STaskInputInfo inputInfo; - STaskSchedInfo schedInfo; - SDispatchMsgInfo msgInfo; - SStreamStatus status; - SCheckpointInfo chkInfo; - STaskExec exec; - SDataRange dataRange; - SHistoryTaskInfo hTaskInfo; - STaskId streamTaskId; - STaskExecStatisInfo execInfo; - SArray* pReadyMsgList; // SArray - TdThreadMutex lock; // secure the operation of set task status and puting data into inputQ - SArray* pUpstreamInfoList; - - // output +typedef struct STaskOutputInfo { union { STaskDispatcherFixed fixedDispatcher; STaskDispatcherShuffle shuffleDispatcher; @@ -379,11 +358,38 @@ struct SStreamTask { STaskSinkFetch fetchSink; }; + void* pTimer; // timer for launch sink tasks + int8_t type; STokenBucket* pTokenBucket; - SMsgCb* pMsgCb; // msg handle - SStreamState* pState; // state backend - SArray* pRspMsgList; +} STaskOutputInfo; +typedef struct SUpstreamInfo { + SArray* pList; + int32_t numOfClosed; +} SUpstreamInfo; + +struct SStreamTask { + int64_t ver; + SStreamTaskId id; + SSTaskBasicInfo info; + STaskOutputQueue outputq; + STaskInputInfo inputInfo; + STaskSchedInfo schedInfo; // todo remove it + STaskOutputInfo outputInfo; + SDispatchMsgInfo msgInfo; + SStreamStatus status; + SCheckpointInfo chkInfo; + STaskExec exec; + SDataRange dataRange; + SHistoryTaskInfo hTaskInfo; + STaskId streamTaskId; + STaskExecStatisInfo execInfo; + SArray* pReadyMsgList; // SArray + TdThreadMutex lock; // secure the operation of set task status and puting data into inputQ + SMsgCb* pMsgCb; // msg handle + SStreamState* pState; // state backend + SArray* pRspMsgList; + SUpstreamInfo upstreamInfo; // the followings attributes don't be serialized int32_t notReadyTasks; int32_t numOfWaitingUpstream; @@ -669,7 +675,6 @@ void tDeleteStreamDispatchReq(SStreamDispatchReq* pReq); int32_t streamSetupScheduleTrigger(SStreamTask* pTask); -int32_t streamProcessRunReq(SStreamTask* pTask); int32_t streamProcessDispatchMsg(SStreamTask* pTask, SStreamDispatchReq* pReq, SRpcMsg* pMsg); int32_t streamProcessDispatchRsp(SStreamTask* pTask, SStreamDispatchRsp* pRsp, int32_t code); @@ -677,7 +682,7 @@ int32_t streamProcessRetrieveReq(SStreamTask* pTask, SStreamRetrieve SStreamChildEpInfo* streamTaskGetUpstreamTaskEpInfo(SStreamTask* pTask, int32_t taskId); void streamTaskInputFail(SStreamTask* pTask); -int32_t streamTryExec(SStreamTask* pTask); +int32_t streamExecTask(SStreamTask* pTask); int32_t streamSchedExec(SStreamTask* pTask); bool streamTaskShouldStop(const SStreamStatus* pStatus); bool streamTaskShouldPause(const SStreamStatus* pStatus); @@ -693,6 +698,7 @@ int32_t streamTaskLaunchScanHistory(SStreamTask* pTask); int32_t streamTaskCheckStatus(SStreamTask* pTask, int32_t upstreamTaskId, int32_t vgId, int64_t stage); int32_t streamTaskUpdateEpsetInfo(SStreamTask* pTask, SArray* pNodeList); void streamTaskResetUpstreamStageInfo(SStreamTask* pTask); +bool streamTaskAllUpstreamClosed(SStreamTask* pTask); bool streamTaskSetSchedStatusWait(SStreamTask* pTask); int8_t streamTaskSetSchedStatusActive(SStreamTask* pTask); int8_t streamTaskSetSchedStatusInActive(SStreamTask* pTask); diff --git a/source/dnode/mnode/impl/src/mndScheduler.c b/source/dnode/mnode/impl/src/mndScheduler.c index d598dc11d2..2931f6be6b 100644 --- a/source/dnode/mnode/impl/src/mndScheduler.c +++ b/source/dnode/mnode/impl/src/mndScheduler.c @@ -87,15 +87,17 @@ END: } int32_t mndSetSinkTaskInfo(SStreamObj* pStream, SStreamTask* pTask) { + STaskOutputInfo* pInfo = &pTask->outputInfo; + if (pStream->smaId != 0) { - pTask->outputInfo.type = TASK_OUTPUT__SMA; - pTask->smaSink.smaId = pStream->smaId; + pInfo->type = TASK_OUTPUT__SMA; + pInfo->smaSink.smaId = pStream->smaId; } else { - pTask->outputInfo.type = TASK_OUTPUT__TABLE; - pTask->tbSink.stbUid = pStream->targetStbUid; - memcpy(pTask->tbSink.stbFullName, pStream->targetSTbName, TSDB_TABLE_FNAME_LEN); - pTask->tbSink.pSchemaWrapper = tCloneSSchemaWrapper(&pStream->outputSchema); - if (pTask->tbSink.pSchemaWrapper == NULL) { + pInfo->type = TASK_OUTPUT__TABLE; + pInfo->tbSink.stbUid = pStream->targetStbUid; + memcpy(pInfo->tbSink.stbFullName, pStream->targetSTbName, TSDB_TABLE_FNAME_LEN); + pInfo->tbSink.pSchemaWrapper = tCloneSSchemaWrapper(&pStream->outputSchema); + if (pInfo->tbSink.pSchemaWrapper == NULL) { return TSDB_CODE_OUT_OF_MEMORY; } } @@ -113,7 +115,7 @@ int32_t mndAddDispatcherForInternalTask(SMnode* pMnode, SStreamObj* pStream, SAr isShuffle = true; pTask->outputInfo.type = TASK_OUTPUT__SHUFFLE_DISPATCH; pTask->msgInfo.msgType = TDMT_STREAM_TASK_DISPATCH; - if (mndExtractDbInfo(pMnode, pDb, &pTask->shuffleDispatcher.dbInfo, NULL) < 0) { + if (mndExtractDbInfo(pMnode, pDb, &pTask->outputInfo.shuffleDispatcher.dbInfo, NULL) < 0) { return -1; } } @@ -124,8 +126,8 @@ int32_t mndAddDispatcherForInternalTask(SMnode* pMnode, SStreamObj* pStream, SAr int32_t numOfSinkNodes = taosArrayGetSize(pSinkNodeList); if (isShuffle) { - memcpy(pTask->shuffleDispatcher.stbFullName, pStream->targetSTbName, TSDB_TABLE_FNAME_LEN); - SArray* pVgs = pTask->shuffleDispatcher.dbInfo.pVgroupInfos; + memcpy(pTask->outputInfo.shuffleDispatcher.stbFullName, pStream->targetSTbName, TSDB_TABLE_FNAME_LEN); + SArray* pVgs = pTask->outputInfo.shuffleDispatcher.dbInfo.pVgroupInfos; int32_t numOfVgroups = taosArrayGetSize(pVgs); for (int32_t i = 0; i < numOfVgroups; i++) { diff --git a/source/dnode/snode/src/snode.c b/source/dnode/snode/src/snode.c index c5fd202986..7fb0b6b40a 100644 --- a/source/dnode/snode/src/snode.c +++ b/source/dnode/snode/src/snode.c @@ -54,7 +54,7 @@ FAIL: } int32_t sndExpandTask(SSnode *pSnode, SStreamTask *pTask, int64_t nextProcessVer) { - ASSERT(pTask->info.taskLevel == TASK_LEVEL__AGG && taosArrayGetSize(pTask->pUpstreamInfoList) != 0); + ASSERT(pTask->info.taskLevel == TASK_LEVEL__AGG && taosArrayGetSize(pTask->upstreamInfo.pList) != 0); int32_t code = streamTaskInit(pTask, pSnode->pMeta, &pSnode->msgCb, nextProcessVer); if (code != TSDB_CODE_SUCCESS) { return code; @@ -70,7 +70,7 @@ int32_t sndExpandTask(SSnode *pSnode, SStreamTask *pTask, int64_t nextProcessVer qDebug("s-task:%s state:%p", pTask->id.idStr, pTask->pState); } - int32_t numOfChildEp = taosArrayGetSize(pTask->pUpstreamInfoList); + int32_t numOfChildEp = taosArrayGetSize(pTask->upstreamInfo.pList); SReadHandle handle = { .vnode = NULL, .numOfVgroups = numOfChildEp, .pStateBackend = pTask->pState, .fillHistory = pTask->info.fillHistory }; initStreamStateAPI(&handle.api); @@ -206,7 +206,7 @@ int32_t sndProcessTaskRunReq(SSnode *pSnode, SRpcMsg *pMsg) { SStreamTask *pTask = streamMetaAcquireTask(pSnode->pMeta, pReq->streamId, pReq->taskId); if (pTask) { - streamProcessRunReq(pTask); + streamExecTask(pTask); streamMetaReleaseTask(pSnode->pMeta, pTask); return 0; } else { diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 19bcf2a9d8..2be48b3c20 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -788,7 +788,7 @@ int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t nextProcessVer) { tqDebug("s-task:%s state:%p", pTask->id.idStr, pTask->pState); } - int32_t numOfVgroups = (int32_t)taosArrayGetSize(pTask->pUpstreamInfoList); + int32_t numOfVgroups = (int32_t)taosArrayGetSize(pTask->upstreamInfo.pList); SReadHandle handle = { .checkpointId = pTask->chkInfo.checkpointId, .vnode = NULL, @@ -809,27 +809,27 @@ int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t nextProcessVer) { // sink if (pTask->outputInfo.type == TASK_OUTPUT__SMA) { - pTask->smaSink.vnode = pTq->pVnode; - pTask->smaSink.smaSink = smaHandleRes; + pTask->outputInfo.smaSink.vnode = pTq->pVnode; + pTask->outputInfo.smaSink.smaSink = smaHandleRes; } else if (pTask->outputInfo.type == TASK_OUTPUT__TABLE) { - pTask->tbSink.vnode = pTq->pVnode; - pTask->tbSink.tbSinkFunc = tqSinkDataIntoDstTable; + pTask->outputInfo.tbSink.vnode = pTq->pVnode; + pTask->outputInfo.tbSink.tbSinkFunc = tqSinkDataIntoDstTable; int32_t ver1 = 1; SMetaInfo info = {0}; - code = metaGetInfo(pTq->pVnode->pMeta, pTask->tbSink.stbUid, &info, NULL); + code = metaGetInfo(pTq->pVnode->pMeta, pTask->outputInfo.tbSink.stbUid, &info, NULL); if (code == TSDB_CODE_SUCCESS) { ver1 = info.skmVer; } - SSchemaWrapper* pschemaWrapper = pTask->tbSink.pSchemaWrapper; - pTask->tbSink.pTSchema = tBuildTSchema(pschemaWrapper->pSchema, pschemaWrapper->nCols, ver1); - if (pTask->tbSink.pTSchema == NULL) { + SSchemaWrapper* pschemaWrapper = pTask->outputInfo.tbSink.pSchemaWrapper; + pTask->outputInfo.tbSink.pTSchema = tBuildTSchema(pschemaWrapper->pSchema, pschemaWrapper->nCols, ver1); + if (pTask->outputInfo.tbSink.pTSchema == NULL) { return -1; } - pTask->tbSink.pTblInfo = tSimpleHashInit(10240, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BIGINT)); - tSimpleHashSetFreeFp(pTask->tbSink.pTblInfo, freePtr); + pTask->outputInfo.tbSink.pTblInfo = tSimpleHashInit(10240, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BIGINT)); + tSimpleHashSetFreeFp(pTask->outputInfo.tbSink.pTblInfo, freePtr); } if (pTask->info.taskLevel == TASK_LEVEL__SOURCE) { @@ -1193,7 +1193,7 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { streamGetTaskStatusStr(TASK_STATUS__PAUSE), streamGetTaskStatusStr(pTask->status.keepTaskStatus)); } - streamTryExec(pTask); // exec directly + streamExecTask(pTask); // exec directly } else { STimeWindow* pWindow = &pTask->dataRange.window; tqDebug("s-task:%s level:%d verRange:%" PRId64 " - %" PRId64 " window:%" PRId64 "-%" PRId64 @@ -1338,7 +1338,7 @@ int32_t tqProcessTaskRunReq(STQ* pTq, SRpcMsg* pMsg) { if (st == TASK_STATUS__NORMAL || st == TASK_STATUS__SCAN_HISTORY || st == TASK_STATUS__CK) { tqDebug("vgId:%d s-task:%s start to process block from inputQ, next checked ver:%" PRId64, vgId, pTask->id.idStr, pTask->chkInfo.nextProcessVer); - streamProcessRunReq(pTask); + streamExecTask(pTask); } else { int8_t status = streamTaskSetSchedStatusInActive(pTask); tqDebug("vgId:%d s-task:%s ignore run req since not in ready state, status:%s, sched-status:%d", vgId, diff --git a/source/dnode/vnode/src/tq/tqSink.c b/source/dnode/vnode/src/tq/tqSink.c index 23b5aff7fa..7d1c754005 100644 --- a/source/dnode/vnode/src/tq/tqSink.c +++ b/source/dnode/vnode/src/tq/tqSink.c @@ -142,7 +142,7 @@ static int32_t doBuildAndSendCreateTableMsg(SVnode* pVnode, char* stbFullName, S int64_t suid) { tqDebug("s-task:%s build create table msg", pTask->id.idStr); - STSchema* pTSchema = pTask->tbSink.pTSchema; + STSchema* pTSchema = pTask->outputInfo.tbSink.pTSchema; int32_t rows = pDataBlock->info.rows; SArray* tagArray = NULL; int32_t code = 0; @@ -588,7 +588,7 @@ int32_t doConvertRows(SSubmitTbData* pTableData, STSchema* pTSchema, SSDataBlock int32_t doWaitForDstTableCreated(SVnode* pVnode, SStreamTask* pTask, STableSinkInfo* pTableSinkInfo, const char* dstTableName, int64_t* uid) { int32_t vgId = TD_VID(pVnode); - int64_t suid = pTask->tbSink.stbUid; + int64_t suid = pTask->outputInfo.tbSink.stbUid; const char* id = pTask->id.idStr; while (pTableSinkInfo->uid == 0) { @@ -631,12 +631,12 @@ int32_t setDstTableDataUid(SVnode* pVnode, SStreamTask* pTask, SSDataBlock* pDat char* dstTableName = pDataBlock->info.parTbName; int32_t numOfRows = pDataBlock->info.rows; const char* id = pTask->id.idStr; - int64_t suid = pTask->tbSink.stbUid; - STSchema* pTSchema = pTask->tbSink.pTSchema; + int64_t suid = pTask->outputInfo.tbSink.stbUid; + STSchema* pTSchema = pTask->outputInfo.tbSink.pTSchema; int32_t vgId = TD_VID(pVnode); STableSinkInfo* pTableSinkInfo = NULL; - bool alreadyCached = tqGetTableInfo(pTask->tbSink.pTblInfo, groupId, &pTableSinkInfo); + bool alreadyCached = tqGetTableInfo(pTask->outputInfo.tbSink.pTblInfo, groupId, &pTableSinkInfo); if (alreadyCached) { if (dstTableName[0] == 0) { // data block does not set the destination table name @@ -702,7 +702,7 @@ int32_t setDstTableDataUid(SVnode* pVnode, SStreamTask* pTask, SSDataBlock* pDat } pTableSinkInfo->uid = 0; - doPutIntoCache(pTask->tbSink.pTblInfo, pTableSinkInfo, groupId, id); + doPutIntoCache(pTask->outputInfo.tbSink.pTblInfo, pTableSinkInfo, groupId, id); } else { bool isValid = isValidDstChildTable(&mr, vgId, dstTableName, suid); if (!isValid) { @@ -716,7 +716,7 @@ int32_t setDstTableDataUid(SVnode* pVnode, SStreamTask* pTask, SSDataBlock* pDat pTableSinkInfo->uid = mr.me.uid; metaReaderClear(&mr); - doPutIntoCache(pTask->tbSink.pTblInfo, pTableSinkInfo, groupId, id); + doPutIntoCache(pTask->outputInfo.tbSink.pTblInfo, pTableSinkInfo, groupId, id); } } } @@ -730,11 +730,11 @@ int32_t setDstTableDataPayload(SStreamTask* pTask, int32_t blockIndex, SSDataBlo const char* id = pTask->id.idStr; tqDebug("s-task:%s sink data pipeline, build submit msg from %dth resBlock, including %d rows, dst suid:%" PRId64, - id, blockIndex + 1, numOfRows, pTask->tbSink.stbUid); + id, blockIndex + 1, numOfRows, pTask->outputInfo.tbSink.stbUid); char* dstTableName = pDataBlock->info.parTbName; // convert all rows - int32_t code = doConvertRows(pTableData, pTask->tbSink.pTSchema, pDataBlock, id); + int32_t code = doConvertRows(pTableData, pTask->outputInfo.tbSink.pTSchema, pDataBlock, id); if (code != TSDB_CODE_SUCCESS) { tqError("s-task:%s failed to convert rows from result block, code:%s", id, tstrerror(terrno)); return code; @@ -759,9 +759,9 @@ bool hasOnlySubmitData(const SArray* pBlocks, int32_t numOfBlocks) { void tqSinkDataIntoDstTable(SStreamTask* pTask, void* vnode, void* data) { const SArray* pBlocks = (const SArray*)data; SVnode* pVnode = (SVnode*)vnode; - int64_t suid = pTask->tbSink.stbUid; - char* stbFullName = pTask->tbSink.stbFullName; - STSchema* pTSchema = pTask->tbSink.pTSchema; + int64_t suid = pTask->outputInfo.tbSink.stbUid; + char* stbFullName = pTask->outputInfo.tbSink.stbFullName; + STSchema* pTSchema = pTask->outputInfo.tbSink.pTSchema; int32_t vgId = TD_VID(pVnode); int32_t numOfBlocks = taosArrayGetSize(pBlocks); int32_t code = TSDB_CODE_SUCCESS; diff --git a/source/libs/stream/src/stream.c b/source/libs/stream/src/stream.c index 6f9a577a46..97316dba07 100644 --- a/source/libs/stream/src/stream.c +++ b/source/libs/stream/src/stream.c @@ -126,7 +126,7 @@ int32_t streamSchedExec(SStreamTask* pTask) { SRpcMsg msg = {.msgType = TDMT_STREAM_TASK_RUN, .pCont = pRunReq, .contLen = sizeof(SStreamTaskRunReq)}; tmsgPutToQueue(pTask->pMsgCb, STREAM_QUEUE, &msg); } else { - stDebug("s-task:%s not launch task since sched status:%d", pTask->id.idStr, pTask->status.schedStatus); + stTrace("s-task:%s not launch task since sched status:%d", pTask->id.idStr, pTask->status.schedStatus); } return 0; @@ -239,8 +239,9 @@ int32_t streamProcessDispatchMsg(SStreamTask* pTask, SStreamDispatchReq* pReq, S status = TASK_INPUT_STATUS__BLOCKED; } else { // This task has received the checkpoint req from the upstream task, from which all the messages should be - // blocked + // blocked. Note that there is no race condition here. if (pReq->type == STREAM_INPUT__CHECKPOINT_TRIGGER) { + atomic_add_fetch_32(&pTask->upstreamInfo.numOfClosed, 1); streamTaskCloseUpstreamInput(pTask, pReq->upstreamTaskId); stDebug("s-task:%s close inputQ for upstream:0x%x, msgId:%d", id, pReq->upstreamTaskId, pReq->msgId); } @@ -274,13 +275,6 @@ int32_t streamProcessDispatchMsg(SStreamTask* pTask, SStreamDispatchReq* pReq, S return 0; } -int32_t streamProcessRunReq(SStreamTask* pTask) { - if (streamTryExec(pTask) < 0) { - return -1; - } - return 0; -} - int32_t streamProcessRetrieveReq(SStreamTask* pTask, SStreamRetrieveReq* pReq, SRpcMsg* pRsp) { streamTaskEnqueueRetrieve(pTask, pReq, pRsp); ASSERT(pTask->info.taskLevel != TASK_LEVEL__SINK); @@ -291,15 +285,17 @@ int32_t streamProcessRetrieveReq(SStreamTask* pTask, SStreamRetrieveReq* pReq, S void streamTaskInputFail(SStreamTask* pTask) { atomic_store_8(&pTask->inputInfo.status, TASK_INPUT_STATUS__FAILED); } void streamTaskOpenAllUpstreamInput(SStreamTask* pTask) { - int32_t num = taosArrayGetSize(pTask->pUpstreamInfoList); + int32_t num = taosArrayGetSize(pTask->upstreamInfo.pList); if (num == 0) { return; } for (int32_t i = 0; i < num; ++i) { - SStreamChildEpInfo* pInfo = taosArrayGetP(pTask->pUpstreamInfoList, i); + SStreamChildEpInfo* pInfo = taosArrayGetP(pTask->upstreamInfo.pList, i); pInfo->dataAllowed = true; } + + pTask->upstreamInfo.numOfClosed = 0; } void streamTaskCloseUpstreamInput(SStreamTask* pTask, int32_t taskId) { @@ -310,9 +306,9 @@ void streamTaskCloseUpstreamInput(SStreamTask* pTask, int32_t taskId) { } SStreamChildEpInfo* streamTaskGetUpstreamTaskEpInfo(SStreamTask* pTask, int32_t taskId) { - int32_t num = taosArrayGetSize(pTask->pUpstreamInfoList); + int32_t num = taosArrayGetSize(pTask->upstreamInfo.pList); for (int32_t i = 0; i < num; ++i) { - SStreamChildEpInfo* pInfo = taosArrayGetP(pTask->pUpstreamInfoList, i); + SStreamChildEpInfo* pInfo = taosArrayGetP(pTask->upstreamInfo.pList, i); if (pInfo->taskId == taskId) { return pInfo; } diff --git a/source/libs/stream/src/streamCheckpoint.c b/source/libs/stream/src/streamCheckpoint.c index a87901eb47..6924d99585 100644 --- a/source/libs/stream/src/streamCheckpoint.c +++ b/source/libs/stream/src/streamCheckpoint.c @@ -92,7 +92,7 @@ int32_t tDecodeStreamCheckpointReadyMsg(SDecoder* pDecoder, SStreamCheckpointRea } static int32_t streamAlignCheckpoint(SStreamTask* pTask) { - int32_t num = taosArrayGetSize(pTask->pUpstreamInfoList); + int32_t num = taosArrayGetSize(pTask->upstreamInfo.pList); int64_t old = atomic_val_compare_exchange_32(&pTask->checkpointAlignCnt, 0, num); if (old == 0) { stDebug("s-task:%s set initial align upstream num:%d", pTask->id.idStr, num); @@ -153,7 +153,7 @@ static int32_t continueDispatchCheckpointBlock(SStreamDataBlock* pBlock, SStream pBlock->srcTaskId = pTask->id.taskId; pBlock->srcVgId = pTask->pMeta->vgId; - int32_t code = taosWriteQitem(pTask->outputInfo.queue->pQueue, pBlock); + int32_t code = taosWriteQitem(pTask->outputq.queue->pQueue, pBlock); if (code == 0) { streamDispatchStreamBlock(pTask); } else { @@ -192,14 +192,14 @@ int32_t streamProcessCheckpointBlock(SStreamTask* pTask, SStreamDataBlock* pBloc int32_t taskLevel = pTask->info.taskLevel; if (taskLevel == TASK_LEVEL__SOURCE) { if (pTask->outputInfo.type == TASK_OUTPUT__FIXED_DISPATCH || pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { - stDebug("s-task:%s set childIdx:%d, and add checkpoint block into outputQ", id, pTask->info.selfChildId); + stDebug("s-task:%s set childIdx:%d, and add checkpoint-trigger block into outputQ", id, pTask->info.selfChildId); continueDispatchCheckpointBlock(pBlock, pTask); } else { // only one task exists, no need to dispatch downstream info streamProcessCheckpointReadyMsg(pTask); streamFreeQitem((SStreamQueueItem*)pBlock); } } else if (taskLevel == TASK_LEVEL__SINK || taskLevel == TASK_LEVEL__AGG) { - ASSERT(taosArrayGetSize(pTask->pUpstreamInfoList) > 0); + ASSERT(taosArrayGetSize(pTask->upstreamInfo.pList) > 0); if (pTask->chkInfo.startTs == 0) { pTask->chkInfo.startTs = taosGetTimestampMs(); pTask->execInfo.checkpoint += 1; @@ -210,7 +210,7 @@ int32_t streamProcessCheckpointBlock(SStreamTask* pTask, SStreamDataBlock* pBloc // there are still some upstream tasks not send checkpoint request, do nothing and wait for then int32_t notReady = streamAlignCheckpoint(pTask); - int32_t num = taosArrayGetSize(pTask->pUpstreamInfoList); + int32_t num = taosArrayGetSize(pTask->upstreamInfo.pList); if (notReady > 0) { stDebug("s-task:%s received checkpoint block, idx:%d, %d upstream tasks not send checkpoint info yet, total:%d", id, pTask->info.selfChildId, notReady, num); diff --git a/source/libs/stream/src/streamDispatch.c b/source/libs/stream/src/streamDispatch.c index 7b23366c53..a7a06dd884 100644 --- a/source/libs/stream/src/streamDispatch.c +++ b/source/libs/stream/src/streamDispatch.c @@ -195,11 +195,11 @@ int32_t streamBroadcastToChildren(SStreamTask* pTask, const SSDataBlock* pBlock) .retrieveLen = dataStrLen, }; - int32_t sz = taosArrayGetSize(pTask->pUpstreamInfoList); + int32_t sz = taosArrayGetSize(pTask->upstreamInfo.pList); ASSERT(sz > 0); for (int32_t i = 0; i < sz; i++) { req.reqId = tGenIdPI64(); - SStreamChildEpInfo* pEpInfo = taosArrayGetP(pTask->pUpstreamInfoList, i); + SStreamChildEpInfo* pEpInfo = taosArrayGetP(pTask->upstreamInfo.pList, i); req.dstNodeId = pEpInfo->nodeId; req.dstTaskId = pEpInfo->taskId; int32_t len; @@ -288,7 +288,7 @@ void destroyDispatchMsg(SStreamDispatchReq* pReq, int32_t numOfVgroups) { int32_t getNumOfDispatchBranch(SStreamTask* pTask) { return (pTask->outputInfo.type == TASK_OUTPUT__FIXED_DISPATCH) ? 1 - : taosArrayGetSize(pTask->shuffleDispatcher.dbInfo.pVgroupInfos); + : taosArrayGetSize(pTask->outputInfo.shuffleDispatcher.dbInfo.pVgroupInfos); } static int32_t doBuildDispatchMsg(SStreamTask* pTask, const SStreamDataBlock* pData) { @@ -301,7 +301,7 @@ static int32_t doBuildDispatchMsg(SStreamTask* pTask, const SStreamDataBlock* pD if (pTask->outputInfo.type == TASK_OUTPUT__FIXED_DISPATCH) { SStreamDispatchReq* pReq = taosMemoryCalloc(1, sizeof(SStreamDispatchReq)); - int32_t downstreamTaskId = pTask->fixedDispatcher.taskId; + int32_t downstreamTaskId = pTask->outputInfo.fixedDispatcher.taskId; code = tInitStreamDispatchReq(pReq, pTask, pData->srcVgId, numOfBlocks, downstreamTaskId, pData->type); if (code != TSDB_CODE_SUCCESS) { return code; @@ -318,10 +318,10 @@ static int32_t doBuildDispatchMsg(SStreamTask* pTask, const SStreamDataBlock* pD pTask->msgInfo.pData = pReq; } else if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { - int32_t rspCnt = atomic_load_32(&pTask->shuffleDispatcher.waitingRspCnt); + int32_t rspCnt = atomic_load_32(&pTask->outputInfo.shuffleDispatcher.waitingRspCnt); ASSERT(rspCnt == 0); - SArray* vgInfo = pTask->shuffleDispatcher.dbInfo.pVgroupInfos; + SArray* vgInfo = pTask->outputInfo.shuffleDispatcher.dbInfo.pVgroupInfos; int32_t numOfVgroups = taosArrayGetSize(vgInfo); SStreamDispatchReq* pReqs = taosMemoryCalloc(numOfVgroups, sizeof(SStreamDispatchReq)); @@ -352,7 +352,7 @@ static int32_t doBuildDispatchMsg(SStreamTask* pTask, const SStreamDataBlock* pD } if (pReqs[j].blockNum == 0) { - atomic_add_fetch_32(&pTask->shuffleDispatcher.waitingRspCnt, 1); + atomic_add_fetch_32(&pTask->outputInfo.shuffleDispatcher.waitingRspCnt, 1); } pReqs[j].blockNum++; @@ -381,16 +381,16 @@ static int32_t sendDispatchMsg(SStreamTask* pTask, SStreamDispatchReq* pDispatch const char* id = pTask->id.idStr; if (pTask->outputInfo.type == TASK_OUTPUT__FIXED_DISPATCH) { - int32_t vgId = pTask->fixedDispatcher.nodeId; - SEpSet* pEpSet = &pTask->fixedDispatcher.epSet; - int32_t downstreamTaskId = pTask->fixedDispatcher.taskId; + int32_t vgId = pTask->outputInfo.fixedDispatcher.nodeId; + SEpSet* pEpSet = &pTask->outputInfo.fixedDispatcher.epSet; + int32_t downstreamTaskId = pTask->outputInfo.fixedDispatcher.taskId; stDebug("s-task:%s (child taskId:%d) fix-dispatch %d block(s) to s-task:0x%x (vgId:%d), id:%d", id, pTask->info.selfChildId, 1, downstreamTaskId, vgId, msgId); code = doSendDispatchMsg(pTask, pDispatchMsg, vgId, pEpSet); } else { - SArray* vgInfo = pTask->shuffleDispatcher.dbInfo.pVgroupInfos; + SArray* vgInfo = pTask->outputInfo.shuffleDispatcher.dbInfo.pVgroupInfos; int32_t numOfVgroups = taosArrayGetSize(vgInfo); stDebug("s-task:%s (child taskId:%d) start to shuffle-dispatch blocks to %d vgroup(s), msgId:%d", @@ -421,12 +421,12 @@ static void doRetryDispatchData(void* param, void* tmrId) { int32_t msgId = pTask->execInfo.dispatch; if (streamTaskShouldStop(&pTask->status)) { - int8_t ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1); + int32_t ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1); stDebug("s-task:%s should stop, abort from timer, ref:%d", pTask->id.idStr, ref); return; } - ASSERT(pTask->outputInfo.status == TASK_OUTPUT_STATUS__WAIT); + ASSERT(pTask->outputq.status == TASK_OUTPUT_STATUS__WAIT); int32_t code = 0; { @@ -436,7 +436,7 @@ static void doRetryDispatchData(void* param, void* tmrId) { SStreamDispatchReq *pReq = pTask->msgInfo.pData; if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { - SArray* vgInfo = pTask->shuffleDispatcher.dbInfo.pVgroupInfos; + SArray* vgInfo = pTask->outputInfo.shuffleDispatcher.dbInfo.pVgroupInfos; int32_t numOfVgroups = taosArrayGetSize(vgInfo); int32_t numOfFailed = taosArrayGetSize(pList); @@ -462,9 +462,9 @@ static void doRetryDispatchData(void* param, void* tmrId) { stDebug("s-task:%s complete re-try shuffle-dispatch blocks to all %d vnodes, msgId:%d", pTask->id.idStr, numOfFailed, msgId); } else { - int32_t vgId = pTask->fixedDispatcher.nodeId; - SEpSet* pEpSet = &pTask->fixedDispatcher.epSet; - int32_t downstreamTaskId = pTask->fixedDispatcher.taskId; + int32_t vgId = pTask->outputInfo.fixedDispatcher.nodeId; + SEpSet* pEpSet = &pTask->outputInfo.fixedDispatcher.epSet; + int32_t downstreamTaskId = pTask->outputInfo.fixedDispatcher.taskId; stDebug("s-task:%s (child taskId:%d) fix-dispatch %d block(s) to s-task:0x%x (vgId:%d), id:%d", id, pTask->info.selfChildId, 1, downstreamTaskId, vgId, msgId); @@ -476,7 +476,7 @@ static void doRetryDispatchData(void* param, void* tmrId) { if (code != TSDB_CODE_SUCCESS) { if (!streamTaskShouldStop(&pTask->status)) { // stDebug("s-task:%s reset the waitRspCnt to be 0 before launch retry dispatch", pTask->id.idStr); -// atomic_store_32(&pTask->shuffleDispatcher.waitingRspCnt, 0); +// atomic_store_32(&pTask->outputInfo.shuffleDispatcher.waitingRspCnt, 0); if (streamTaskShouldPause(&pTask->status)) { streamRetryDispatchData(pTask, DISPATCH_RETRY_INTERVAL_MS * 10); } else { @@ -487,7 +487,7 @@ static void doRetryDispatchData(void* param, void* tmrId) { stDebug("s-task:%s should stop, abort from timer, ref:%d", pTask->id.idStr, ref); } } else { - int8_t ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1); + int32_t ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1); stDebug("s-task:%s send success, jump out of timer, ref:%d", pTask->id.idStr, ref); } } @@ -508,7 +508,7 @@ void streamRetryDispatchData(SStreamTask* pTask, int64_t waitDuration) { int32_t streamSearchAndAddBlock(SStreamTask* pTask, SStreamDispatchReq* pReqs, SSDataBlock* pDataBlock, int32_t vgSz, int64_t groupId) { uint32_t hashValue = 0; - SArray* vgInfo = pTask->shuffleDispatcher.dbInfo.pVgroupInfos; + SArray* vgInfo = pTask->outputInfo.shuffleDispatcher.dbInfo.pVgroupInfos; if (pTask->pNameMap == NULL) { pTask->pNameMap = tSimpleHashInit(1024, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BIGINT)); } @@ -528,14 +528,14 @@ int32_t streamSearchAndAddBlock(SStreamTask* pTask, SStreamDispatchReq* pReqs, S } if (pDataBlock->info.parTbName[0]) { - snprintf(ctbName, TSDB_TABLE_NAME_LEN, "%s.%s", pTask->shuffleDispatcher.dbInfo.db, pDataBlock->info.parTbName); + snprintf(ctbName, TSDB_TABLE_NAME_LEN, "%s.%s", pTask->outputInfo.shuffleDispatcher.dbInfo.db, pDataBlock->info.parTbName); } else { - buildCtbNameByGroupIdImpl(pTask->shuffleDispatcher.stbFullName, groupId, pDataBlock->info.parTbName); - snprintf(ctbName, TSDB_TABLE_NAME_LEN, "%s.%s", pTask->shuffleDispatcher.dbInfo.db, pDataBlock->info.parTbName); + buildCtbNameByGroupIdImpl(pTask->outputInfo.shuffleDispatcher.stbFullName, groupId, pDataBlock->info.parTbName); + snprintf(ctbName, TSDB_TABLE_NAME_LEN, "%s.%s", pTask->outputInfo.shuffleDispatcher.dbInfo.db, pDataBlock->info.parTbName); } /*uint32_t hashValue = MurmurHash3_32(ctbName, strlen(ctbName));*/ - SUseDbRsp* pDbInfo = &pTask->shuffleDispatcher.dbInfo; + SUseDbRsp* pDbInfo = &pTask->outputInfo.shuffleDispatcher.dbInfo; hashValue = taosGetTbHashVal(ctbName, strlen(ctbName), pDbInfo->hashMethod, pDbInfo->hashPrefix, pDbInfo->hashSuffix); taosMemoryFree(ctbName); @@ -560,7 +560,7 @@ int32_t streamSearchAndAddBlock(SStreamTask* pTask, SStreamDispatchReq* pReqs, S } if (pReqs[j].blockNum == 0) { - atomic_add_fetch_32(&pTask->shuffleDispatcher.waitingRspCnt, 1); + atomic_add_fetch_32(&pTask->outputInfo.shuffleDispatcher.waitingRspCnt, 1); } pReqs[j].blockNum++; @@ -576,27 +576,27 @@ int32_t streamDispatchStreamBlock(SStreamTask* pTask) { ASSERT((pTask->outputInfo.type == TASK_OUTPUT__FIXED_DISPATCH || pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH)); const char* id = pTask->id.idStr; - int32_t numOfElems = streamQueueGetNumOfItems(pTask->outputInfo.queue); + int32_t numOfElems = streamQueueGetNumOfItems(pTask->outputq.queue); if (numOfElems > 0) { - double size = SIZE_IN_MiB(taosQueueMemorySize(pTask->outputInfo.queue->pQueue)); + double size = SIZE_IN_MiB(taosQueueMemorySize(pTask->outputq.queue->pQueue)); stDebug("s-task:%s start to dispatch intermediate block to downstream, elem in outputQ:%d, size:%.2fMiB", id, numOfElems, size); } // to make sure only one dispatch is running int8_t old = - atomic_val_compare_exchange_8(&pTask->outputInfo.status, TASK_OUTPUT_STATUS__NORMAL, TASK_OUTPUT_STATUS__WAIT); + atomic_val_compare_exchange_8(&pTask->outputq.status, TASK_OUTPUT_STATUS__NORMAL, TASK_OUTPUT_STATUS__WAIT); if (old != TASK_OUTPUT_STATUS__NORMAL) { stDebug("s-task:%s wait for dispatch rsp, not dispatch now, output status:%d", id, old); return 0; } ASSERT(pTask->msgInfo.pData == NULL); - stDebug("s-task:%s start to dispatch msg, set output status:%d", id, pTask->outputInfo.status); + stDebug("s-task:%s start to dispatch msg, set output status:%d", id, pTask->outputq.status); - SStreamDataBlock* pBlock = streamQueueNextItem(pTask->outputInfo.queue); + SStreamDataBlock* pBlock = streamQueueNextItem(pTask->outputq.queue); if (pBlock == NULL) { - atomic_store_8(&pTask->outputInfo.status, TASK_OUTPUT_STATUS__NORMAL); - stDebug("s-task:%s not dispatch since no elems in outputQ, output status:%d", id, pTask->outputInfo.status); + atomic_store_8(&pTask->outputq.status, TASK_OUTPUT_STATUS__NORMAL); + stDebug("s-task:%s not dispatch since no elems in outputQ, output status:%d", id, pTask->outputq.status); return 0; } @@ -620,10 +620,10 @@ int32_t streamDispatchStreamBlock(SStreamTask* pTask) { } stDebug("s-task:%s failed to dispatch msg:%d to downstream, code:%s, output status:%d, retry cnt:%d", id, - pTask->execInfo.dispatch, tstrerror(terrno), pTask->outputInfo.status, retryCount); + pTask->execInfo.dispatch, tstrerror(terrno), pTask->outputq.status, retryCount); // todo deal with only partially success dispatch case - atomic_store_32(&pTask->shuffleDispatcher.waitingRspCnt, 0); + atomic_store_32(&pTask->outputInfo.shuffleDispatcher.waitingRspCnt, 0); if (terrno == TSDB_CODE_APP_IS_STOPPING) { // in case of this error, do not retry anymore destroyDispatchMsg(pTask->msgInfo.pData, getNumOfDispatchBranch(pTask)); pTask->msgInfo.pData = NULL; @@ -631,7 +631,7 @@ int32_t streamDispatchStreamBlock(SStreamTask* pTask) { } if (++retryCount > MAX_CONTINUE_RETRY_COUNT) { // add to timer to retry - int8_t ref = atomic_add_fetch_32(&pTask->status.timerActive, 1); + int32_t ref = atomic_add_fetch_32(&pTask->status.timerActive, 1); stDebug("s-task:%s failed to dispatch msg to downstream for %d times, code:%s, add timer to retry in %dms, ref:%d", pTask->id.idStr, retryCount, tstrerror(terrno), DISPATCH_RETRY_INTERVAL_MS, ref); @@ -654,11 +654,11 @@ int32_t streamDispatchScanHistoryFinishMsg(SStreamTask* pTask) { // serialize if (pTask->outputInfo.type == TASK_OUTPUT__FIXED_DISPATCH) { - req.downstreamTaskId = pTask->fixedDispatcher.taskId; + req.downstreamTaskId = pTask->outputInfo.fixedDispatcher.taskId; pTask->notReadyTasks = 1; - doDispatchScanHistoryFinishMsg(pTask, &req, pTask->fixedDispatcher.nodeId, &pTask->fixedDispatcher.epSet); + doDispatchScanHistoryFinishMsg(pTask, &req, pTask->outputInfo.fixedDispatcher.nodeId, &pTask->outputInfo.fixedDispatcher.epSet); } else if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { - SArray* vgInfo = pTask->shuffleDispatcher.dbInfo.pVgroupInfos; + SArray* vgInfo = pTask->outputInfo.shuffleDispatcher.dbInfo.pVgroupInfos; int32_t numOfVgs = taosArrayGetSize(vgInfo); pTask->notReadyTasks = numOfVgs; @@ -680,7 +680,7 @@ int32_t streamDispatchScanHistoryFinishMsg(SStreamTask* pTask) { // this function is usually invoked by sink/agg task int32_t streamTaskSendCheckpointReadyMsg(SStreamTask* pTask) { int32_t num = taosArrayGetSize(pTask->pReadyMsgList); - ASSERT(taosArrayGetSize(pTask->pUpstreamInfoList) == num); + ASSERT(taosArrayGetSize(pTask->upstreamInfo.pList) == num); for (int32_t i = 0; i < num; ++i) { SStreamChkptReadyInfo* pInfo = taosArrayGet(pTask->pReadyMsgList, i); @@ -1049,7 +1049,7 @@ static int32_t handleDispatchSuccessRsp(SStreamTask* pTask, int32_t downstreamId } // now ready for next data output - atomic_store_8(&pTask->outputInfo.status, TASK_OUTPUT_STATUS__NORMAL); + atomic_store_8(&pTask->outputq.status, TASK_OUTPUT_STATUS__NORMAL); // otherwise, continue dispatch the first block to down stream task in pipeline streamDispatchStreamBlock(pTask); @@ -1061,11 +1061,13 @@ int32_t streamProcessDispatchRsp(SStreamTask* pTask, SStreamDispatchRsp* pRsp, i int32_t vgId = pTask->pMeta->vgId; int32_t msgId = pTask->execInfo.dispatch; + // follower not handle the dispatch rsp if ((pTask->pMeta->role == NODE_ROLE_FOLLOWER) || (pTask->status.downstreamReady != 1)) { stError("s-task:%s vgId:%d is follower or task just re-launched, not handle the dispatch rsp, discard it", id, vgId); return TSDB_CODE_STREAM_TASK_NOT_EXIST; } + // discard invalid dispatch rsp msg if ((pRsp->msgId != msgId) || (pRsp->stage != pTask->pMeta->stage)) { stError("s-task:%s vgId:%d not expect rsp, expected: msgId:%d, stage:%" PRId64 " actual msgId:%d, stage:%" PRId64 " discard it", @@ -1107,7 +1109,7 @@ int32_t streamProcessDispatchRsp(SStreamTask* pTask, SStreamDispatchRsp* pRsp, i int32_t leftRsp = 0; if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { - leftRsp = atomic_sub_fetch_32(&pTask->shuffleDispatcher.waitingRspCnt, 1); + leftRsp = atomic_sub_fetch_32(&pTask->outputInfo.shuffleDispatcher.waitingRspCnt, 1); ASSERT(leftRsp >= 0); if (leftRsp > 0) { @@ -1127,17 +1129,17 @@ int32_t streamProcessDispatchRsp(SStreamTask* pTask, SStreamDispatchRsp* pRsp, i // all msg rsp already, continue if (leftRsp == 0) { - ASSERT(pTask->outputInfo.status == TASK_OUTPUT_STATUS__WAIT); + ASSERT(pTask->outputq.status == TASK_OUTPUT_STATUS__WAIT); // we need to re-try send dispatch msg to downstream tasks int32_t numOfFailed = taosArrayGetSize(pTask->msgInfo.pRetryList); if (numOfFailed > 0) { if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { - atomic_store_32(&pTask->shuffleDispatcher.waitingRspCnt, numOfFailed); - stDebug("s-task:%s waiting rsp set to be %d", id, pTask->shuffleDispatcher.waitingRspCnt); + atomic_store_32(&pTask->outputInfo.shuffleDispatcher.waitingRspCnt, numOfFailed); + stDebug("s-task:%s waiting rsp set to be %d", id, pTask->outputInfo.shuffleDispatcher.waitingRspCnt); } - int8_t ref = atomic_add_fetch_32(&pTask->status.timerActive, 1); + int32_t ref = atomic_add_fetch_32(&pTask->status.timerActive, 1); stDebug("s-task:%s failed to dispatch msg to downstream code:%s, add timer to retry in %dms, ref:%d", pTask->id.idStr, tstrerror(terrno), DISPATCH_RETRY_INTERVAL_MS, ref); @@ -1155,7 +1157,7 @@ int32_t streamProcessDispatchRsp(SStreamTask* pTask, SStreamDispatchRsp* pRsp, i } // now ready for next data output - atomic_store_8(&pTask->outputInfo.status, TASK_OUTPUT_STATUS__NORMAL); + atomic_store_8(&pTask->outputq.status, TASK_OUTPUT_STATUS__NORMAL); } else { handleDispatchSuccessRsp(pTask, pRsp->downstreamTaskId); } diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index 2302e00bb3..31ccbe50f6 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -36,10 +36,10 @@ static int32_t doOutputResultBlockImpl(SStreamTask* pTask, SStreamDataBlock* pBl int32_t code = 0; int32_t type = pTask->outputInfo.type; if (type == TASK_OUTPUT__TABLE) { - pTask->tbSink.tbSinkFunc(pTask, pTask->tbSink.vnode, pBlock->blocks); + pTask->outputInfo.tbSink.tbSinkFunc(pTask, pTask->outputInfo.tbSink.vnode, pBlock->blocks); destroyStreamDataBlock(pBlock); } else if (type == TASK_OUTPUT__SMA) { - pTask->smaSink.smaSink(pTask->smaSink.vnode, pTask->smaSink.smaId, pBlock->blocks); + pTask->outputInfo.smaSink.smaSink(pTask->outputInfo.smaSink.vnode, pTask->outputInfo.smaSink.smaId, pBlock->blocks); destroyStreamDataBlock(pBlock); } else { ASSERT(type == TASK_OUTPUT__FIXED_DISPATCH || type == TASK_OUTPUT__SHUFFLE_DISPATCH); @@ -487,7 +487,7 @@ int32_t streamProcessTranstateBlock(SStreamTask* pTask, SStreamDataBlock* pBlock // agg task should dispatch trans-state msg to sink task, to flush all data to sink task. if (level == TASK_LEVEL__AGG || level == TASK_LEVEL__SOURCE) { pBlock->srcVgId = pTask->pMeta->vgId; - code = taosWriteQitem(pTask->outputInfo.queue->pQueue, pBlock); + code = taosWriteQitem(pTask->outputq.queue->pQueue, pBlock); if (code == 0) { streamDispatchStreamBlock(pTask); } else { @@ -607,7 +607,7 @@ bool streamTaskIsIdle(const SStreamTask* pTask) { pTask->status.taskStatus == TASK_STATUS__DROPPING); } -int32_t streamTryExec(SStreamTask* pTask) { +int32_t streamExecTask(SStreamTask* pTask) { // this function may be executed by multi-threads, so status check is required. const char* id = pTask->id.idStr; @@ -615,7 +615,7 @@ int32_t streamTryExec(SStreamTask* pTask) { if (schedStatus == TASK_SCHED_STATUS__WAITING) { while (1) { int32_t code = streamExecForAll(pTask); - if (code < 0) { // todo this status shoudl be removed + if (code < 0) { // todo this status should be removed atomic_store_8(&pTask->status.schedStatus, TASK_SCHED_STATUS__FAILED); return -1; } @@ -663,7 +663,7 @@ int32_t streamTaskReloadState(SStreamTask* pTask) { } int32_t streamAlignTransferState(SStreamTask* pTask) { - int32_t numOfUpstream = taosArrayGetSize(pTask->pUpstreamInfoList); + int32_t numOfUpstream = taosArrayGetSize(pTask->upstreamInfo.pList); int32_t old = atomic_val_compare_exchange_32(&pTask->transferStateAlignCnt, 0, numOfUpstream); if (old == 0) { stDebug("s-task:%s set the transfer state aligncnt %d", pTask->id.idStr, numOfUpstream); diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 85ce7289ba..97d7507194 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -892,7 +892,7 @@ void metaHbToMnode(void* param, void* tmrId) { entry.inputRate = entry.inputQUsed*100.0/STREAM_TASK_QUEUE_CAPACITY_IN_SIZE; if ((*pTask)->info.taskLevel == TASK_LEVEL__SINK) { - entry.sinkQuota = (*pTask)->pTokenBucket->quotaRate; + entry.sinkQuota = (*pTask)->outputInfo.pTokenBucket->quotaRate; entry.sinkDataSize = SIZE_IN_MiB((*pTask)->execInfo.sink.dataSize); } diff --git a/source/libs/stream/src/streamQueue.c b/source/libs/stream/src/streamQueue.c index ce90f29451..676e85eadc 100644 --- a/source/libs/stream/src/streamQueue.c +++ b/source/libs/stream/src/streamQueue.c @@ -18,6 +18,7 @@ #define MAX_STREAM_EXEC_BATCH_NUM 32 #define MAX_SMOOTH_BURST_RATIO 5 // 5 sec #define WAIT_FOR_DURATION 40 +#define SINK_TASK_IDLE_DURATION 200 // 200 ms // todo refactor: // read data from input queue @@ -154,6 +155,10 @@ const char* streamQueueItemGetTypeStr(int32_t type) { } } +static void doLaunchSinkTask(void* param, void* tmrId) { + +} + int32_t streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueItem** pInput, int32_t* numOfBlocks, int32_t* blockSize) { int32_t retryTimes = 0; @@ -166,8 +171,21 @@ int32_t streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueItem** pInpu *blockSize = 0; // no available token in bucket for sink task, let's wait for a little bit - if (taskLevel == TASK_LEVEL__SINK && (!streamTaskExtractAvailableToken(pTask->pTokenBucket, pTask->id.idStr))) { + if (taskLevel == TASK_LEVEL__SINK && (!streamTaskExtractAvailableToken(pTask->outputInfo.pTokenBucket, pTask->id.idStr))) { stDebug("s-task:%s no available token in bucket for sink data, wait", id); + +// if (streamTaskAllUpstreamClosed(pTask)) { +// int32_t ref = atomic_add_fetch_32(&pTask->status.timerActive, 1); +// stDebug("s-task:%s try start task in %dms in tmr, since all upstream inputQ is closed, ref:%d", pTask->id.idStr, +// SINK_TASK_IDLE_DURATION, ref); +// +// if (pTask->outputInfo.pTimer == NULL) { +// pTask->outputInfo.pTimer = taosTmrStart(doLaunchSinkTask, SINK_TASK_IDLE_DURATION, pTask, streamEnv.timer); +// } else { +// taosTmrReset(doLaunchSinkTask, SINK_TASK_IDLE_DURATION, pTask, streamEnv.timer, &pTask->outputInfo.pTimer); +// } +// } + return TSDB_CODE_SUCCESS; } @@ -188,10 +206,10 @@ int32_t streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueItem** pInpu if (*numOfBlocks > 0) { *blockSize = streamQueueItemGetSize(*pInput); if (taskLevel == TASK_LEVEL__SINK) { - streamTaskConsumeQuota(pTask->pTokenBucket, *blockSize); + streamTaskConsumeQuota(pTask->outputInfo.pTokenBucket, *blockSize); } } else { - streamTaskPutbackToken(pTask->pTokenBucket); + streamTaskPutbackToken(pTask->outputInfo.pTokenBucket); } return TSDB_CODE_SUCCESS; @@ -207,7 +225,7 @@ int32_t streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueItem** pInpu stDebug("s-task:%s %s msg extracted, start to process immediately", id, p); // restore the token to bucket in case of checkpoint/trans-state msg - streamTaskPutbackToken(pTask->pTokenBucket); + streamTaskPutbackToken(pTask->outputInfo.pTokenBucket); *blockSize = 0; *numOfBlocks = 1; *pInput = qItem; @@ -216,7 +234,7 @@ int32_t streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueItem** pInpu 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->pTokenBucket, *blockSize); + streamTaskConsumeQuota(pTask->outputInfo.pTokenBucket, *blockSize); } streamQueueProcessFail(pTask->inputInfo.queue); @@ -237,7 +255,7 @@ int32_t streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueItem** pInpu *blockSize = streamQueueItemGetSize(*pInput); if (taskLevel == TASK_LEVEL__SINK) { - streamTaskConsumeQuota(pTask->pTokenBucket, *blockSize); + streamTaskConsumeQuota(pTask->outputInfo.pTokenBucket, *blockSize); } streamQueueProcessFail(pTask->inputInfo.queue); @@ -255,7 +273,7 @@ int32_t streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueItem** pInpu *blockSize = streamQueueItemGetSize(*pInput); if (taskLevel == TASK_LEVEL__SINK) { - streamTaskConsumeQuota(pTask->pTokenBucket, *blockSize); + streamTaskConsumeQuota(pTask->outputInfo.pTokenBucket, *blockSize); } return TSDB_CODE_SUCCESS; @@ -350,15 +368,15 @@ int32_t streamTaskPutDataIntoInputQ(SStreamTask* pTask, SStreamQueueItem* pItem) // the result should be put into the outputQ in any cases, otherwise, the result may be lost int32_t streamTaskPutDataIntoOutputQ(SStreamTask* pTask, SStreamDataBlock* pBlock) { - STaosQueue* pQueue = pTask->outputInfo.queue->pQueue; + STaosQueue* pQueue = pTask->outputq.queue->pQueue; - while (streamQueueIsFull(pTask->outputInfo.queue)) { + while (streamQueueIsFull(pTask->outputq.queue)) { if (streamTaskShouldStop(&pTask->status)) { stInfo("s-task:%s discard result block due to task stop", pTask->id.idStr); return TSDB_CODE_STREAM_EXEC_CANCELLED; } - int32_t total = streamQueueGetNumOfItems(pTask->outputInfo.queue); + int32_t total = streamQueueGetNumOfItems(pTask->outputq.queue); double size = SIZE_IN_MiB(taosQueueMemorySize(pQueue)); // let's wait for there are enough space to hold this result pBlock stDebug("s-task:%s outputQ is full, wait for 500ms and retry, outputQ items:%d, size:%.2fMiB", pTask->id.idStr, @@ -368,7 +386,7 @@ int32_t streamTaskPutDataIntoOutputQ(SStreamTask* pTask, SStreamDataBlock* pBloc int32_t code = taosWriteQitem(pQueue, pBlock); - int32_t total = streamQueueGetNumOfItems(pTask->outputInfo.queue); + int32_t total = streamQueueGetNumOfItems(pTask->outputq.queue); double size = SIZE_IN_MiB(taosQueueMemorySize(pQueue)); if (code != 0) { stError("s-task:%s failed to put res into outputQ, outputQ items:%d, size:%.2fMiB code:%s, result lost", diff --git a/source/libs/stream/src/streamRecover.c b/source/libs/stream/src/streamRecover.c index 85e57339e0..55ed555af6 100644 --- a/source/libs/stream/src/streamRecover.c +++ b/source/libs/stream/src/streamRecover.c @@ -40,7 +40,7 @@ static void streamTaskSetReady(SStreamTask* pTask, int32_t numOfReqs) { int32_t vgId = pMeta->vgId; if (pTask->status.taskStatus == TASK_STATUS__SCAN_HISTORY && pTask->info.taskLevel != TASK_LEVEL__SOURCE) { - pTask->numOfWaitingUpstream = taosArrayGetSize(pTask->pUpstreamInfoList); + pTask->numOfWaitingUpstream = taosArrayGetSize(pTask->upstreamInfo.pList); stDebug("s-task:%s level:%d task wait for %d upstream tasks complete scan-history procedure, status:%s", pTask->id.idStr, pTask->info.taskLevel, pTask->numOfWaitingUpstream, streamGetTaskStatusStr(pTask->status.taskStatus)); @@ -144,8 +144,8 @@ static int32_t doCheckDownstreamStatus(SStreamTask* pTask) { // serialize streamProcessScanHistoryFinishRsp if (pTask->outputInfo.type == TASK_OUTPUT__FIXED_DISPATCH) { req.reqId = tGenIdPI64(); - req.downstreamNodeId = pTask->fixedDispatcher.nodeId; - req.downstreamTaskId = pTask->fixedDispatcher.taskId; + req.downstreamNodeId = pTask->outputInfo.fixedDispatcher.nodeId; + req.downstreamTaskId = pTask->outputInfo.fixedDispatcher.taskId; pTask->checkReqId = req.reqId; stDebug("s-task:%s (vgId:%d) stage:%" PRId64 " check single downstream task:0x%x(vgId:%d) ver:%" PRId64 "-%" PRId64 @@ -153,9 +153,9 @@ static int32_t doCheckDownstreamStatus(SStreamTask* pTask) { pTask->id.idStr, pTask->info.nodeId, req.stage, req.downstreamTaskId, req.downstreamNodeId, pRange->range.minVer, pRange->range.maxVer, pWindow->skey, pWindow->ekey, req.reqId); - streamSendCheckMsg(pTask, &req, pTask->fixedDispatcher.nodeId, &pTask->fixedDispatcher.epSet); + streamSendCheckMsg(pTask, &req, pTask->outputInfo.fixedDispatcher.nodeId, &pTask->outputInfo.fixedDispatcher.epSet); } else if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { - SArray* vgInfo = pTask->shuffleDispatcher.dbInfo.pVgroupInfos; + SArray* vgInfo = pTask->outputInfo.shuffleDispatcher.dbInfo.pVgroupInfos; int32_t numOfVgs = taosArrayGetSize(vgInfo); pTask->notReadyTasks = numOfVgs; @@ -225,9 +225,9 @@ static void recheckDownstreamTasks(void* param, void* tmrId) { if (pTask->outputInfo.type == TASK_OUTPUT__FIXED_DISPATCH) { stDebug("s-task:%s (vgId:%d) check downstream task:0x%x (vgId:%d) stage:%" PRId64 " (recheck)", pTask->id.idStr, pTask->info.nodeId, pReq->downstreamTaskId, pReq->downstreamNodeId, pReq->stage); - streamSendCheckMsg(pTask, pReq, pReq->downstreamNodeId, &pTask->fixedDispatcher.epSet); + streamSendCheckMsg(pTask, pReq, pReq->downstreamNodeId, &pTask->outputInfo.fixedDispatcher.epSet); } else if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { - SArray* vgInfo = pTask->shuffleDispatcher.dbInfo.pVgroupInfos; + SArray* vgInfo = pTask->outputInfo.shuffleDispatcher.dbInfo.pVgroupInfos; int32_t numOfVgs = taosArrayGetSize(vgInfo); for (int32_t i = 0; i < numOfVgs; i++) { @@ -241,7 +241,7 @@ static void recheckDownstreamTasks(void* param, void* tmrId) { } destroyRecheckInfo(pInfo); - int8_t ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1); + int32_t ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1); stDebug("s-task:%s complete send check in timer, ref:%d", pTask->id.idStr, ref); } @@ -341,7 +341,7 @@ int32_t streamProcessCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* pRs doProcessDownstreamReadyRsp(pTask, numOfReqs); } else { - int32_t total = taosArrayGetSize(pTask->shuffleDispatcher.dbInfo.pVgroupInfos); + int32_t total = taosArrayGetSize(pTask->outputInfo.shuffleDispatcher.dbInfo.pVgroupInfos); stDebug("s-task:%s (vgId:%d) recv check rsp from task:0x%x (vgId:%d) status:%d, total:%d not ready:%d", id, pRsp->upstreamNodeId, pRsp->downstreamTaskId, pRsp->downstreamNodeId, pRsp->status, total, left); } @@ -367,7 +367,7 @@ int32_t streamProcessCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* pRs } else { STaskRecheckInfo* pInfo = createRecheckInfo(pTask, pRsp); - int8_t ref = atomic_add_fetch_32(&pTask->status.timerActive, 1); + int32_t ref = atomic_add_fetch_32(&pTask->status.timerActive, 1); stDebug("s-task:%s downstream taskId:0x%x (vgId:%d) not ready, stage:%d, retry in 100ms, ref:%d ", id, pRsp->downstreamTaskId, pRsp->downstreamNodeId, pRsp->oldStage, ref); @@ -528,7 +528,7 @@ int32_t streamProcessScanHistoryFinishReq(SStreamTask* pTask, SStreamScanHistory ASSERT(left >= 0); if (left == 0) { - int32_t numOfTasks = taosArrayGetSize(pTask->pUpstreamInfoList); + int32_t numOfTasks = taosArrayGetSize(pTask->upstreamInfo.pList); stDebug( "s-task:%s all %d upstream tasks finish scan-history data, set param for agg task for stream data and send " "rsp to all upstream tasks", @@ -640,7 +640,7 @@ static void tryLaunchHistoryTask(void* param, void* tmrId) { } if (pHTaskInfo->retryTimes > MAX_RETRY_LAUNCH_HISTORY_TASK) { - int8_t ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1); + int32_t ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1); streamMetaReleaseTask(pMeta, pTask); stError("s-task:%s max retry:%d reached, quit from retrying launch related fill-history task:0x%x, ref:%d", @@ -672,7 +672,7 @@ static void tryLaunchHistoryTask(void* param, void* tmrId) { } // not in timer anymore - int8_t ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1); + int32_t ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1); stDebug("s-task:0x%x fill-history task launch completed, retry times:%d, ref:%d", (int32_t)pInfo->id.taskId, pHTaskInfo->retryTimes, ref); streamMetaReleaseTask(pMeta, pTask); From 2936ac9b89447bad6a74a5cf71ae438b41da4e0d Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sat, 7 Oct 2023 19:18:26 +0800 Subject: [PATCH 139/158] fix(stream): add missing refactor. --- source/libs/stream/src/streamTask.c | 121 +++++++++++++++------------- 1 file changed, 66 insertions(+), 55 deletions(-) diff --git a/source/libs/stream/src/streamTask.c b/source/libs/stream/src/streamTask.c index 8dffbec09f..b6a60e28d7 100644 --- a/source/libs/stream/src/streamTask.c +++ b/source/libs/stream/src/streamTask.c @@ -20,6 +20,8 @@ #include "ttimer.h" #include "wal.h" +static void streamTaskDestroyUpstreamInfo(SUpstreamInfo* pUpstreamInfo); + static int32_t addToTaskset(SArray* pArray, SStreamTask* pTask) { int32_t childId = taosArrayGetSize(pArray); pTask->info.selfChildId = childId; @@ -48,7 +50,7 @@ SStreamTask* tNewStreamTask(int64_t streamId, int8_t taskLevel, bool fillHistory pTask->status.schedStatus = TASK_SCHED_STATUS__INACTIVE; pTask->status.taskStatus = (fillHistory || hasFillhistory)? TASK_STATUS__SCAN_HISTORY:TASK_STATUS__NORMAL; pTask->inputInfo.status = TASK_INPUT_STATUS__NORMAL; - pTask->outputInfo.status = TASK_OUTPUT_STATUS__NORMAL; + pTask->outputq.status = TASK_OUTPUT_STATUS__NORMAL; if (fillHistory) { ASSERT(hasFillhistory); @@ -113,10 +115,10 @@ int32_t tEncodeStreamTask(SEncoder* pEncoder, const SStreamTask* pTask) { if (tEncodeI64(pEncoder, pTask->dataRange.window.skey)) return -1; if (tEncodeI64(pEncoder, pTask->dataRange.window.ekey)) return -1; - int32_t epSz = taosArrayGetSize(pTask->pUpstreamInfoList); + int32_t epSz = taosArrayGetSize(pTask->upstreamInfo.pList); if (tEncodeI32(pEncoder, epSz) < 0) return -1; for (int32_t i = 0; i < epSz; i++) { - SStreamChildEpInfo* pInfo = taosArrayGetP(pTask->pUpstreamInfoList, i); + SStreamChildEpInfo* pInfo = taosArrayGetP(pTask->upstreamInfo.pList, i); if (tEncodeStreamEpInfo(pEncoder, pInfo) < 0) return -1; } @@ -125,20 +127,20 @@ int32_t tEncodeStreamTask(SEncoder* pEncoder, const SStreamTask* pTask) { } if (pTask->outputInfo.type == TASK_OUTPUT__TABLE) { - if (tEncodeI64(pEncoder, pTask->tbSink.stbUid) < 0) return -1; - if (tEncodeCStr(pEncoder, pTask->tbSink.stbFullName) < 0) return -1; - if (tEncodeSSchemaWrapper(pEncoder, pTask->tbSink.pSchemaWrapper) < 0) return -1; + if (tEncodeI64(pEncoder, pTask->outputInfo.tbSink.stbUid) < 0) return -1; + if (tEncodeCStr(pEncoder, pTask->outputInfo.tbSink.stbFullName) < 0) return -1; + if (tEncodeSSchemaWrapper(pEncoder, pTask->outputInfo.tbSink.pSchemaWrapper) < 0) return -1; } else if (pTask->outputInfo.type == TASK_OUTPUT__SMA) { - if (tEncodeI64(pEncoder, pTask->smaSink.smaId) < 0) return -1; + if (tEncodeI64(pEncoder, pTask->outputInfo.smaSink.smaId) < 0) return -1; } else if (pTask->outputInfo.type == TASK_OUTPUT__FETCH) { - if (tEncodeI8(pEncoder, pTask->fetchSink.reserved) < 0) return -1; + if (tEncodeI8(pEncoder, pTask->outputInfo.fetchSink.reserved) < 0) return -1; } else if (pTask->outputInfo.type == TASK_OUTPUT__FIXED_DISPATCH) { - if (tEncodeI32(pEncoder, pTask->fixedDispatcher.taskId) < 0) return -1; - if (tEncodeI32(pEncoder, pTask->fixedDispatcher.nodeId) < 0) return -1; - if (tEncodeSEpSet(pEncoder, &pTask->fixedDispatcher.epSet) < 0) return -1; + if (tEncodeI32(pEncoder, pTask->outputInfo.fixedDispatcher.taskId) < 0) return -1; + if (tEncodeI32(pEncoder, pTask->outputInfo.fixedDispatcher.nodeId) < 0) return -1; + if (tEncodeSEpSet(pEncoder, &pTask->outputInfo.fixedDispatcher.epSet) < 0) return -1; } else if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { - if (tSerializeSUseDbRspImp(pEncoder, &pTask->shuffleDispatcher.dbInfo) < 0) return -1; - if (tEncodeCStr(pEncoder, pTask->shuffleDispatcher.stbFullName) < 0) return -1; + if (tSerializeSUseDbRspImp(pEncoder, &pTask->outputInfo.shuffleDispatcher.dbInfo) < 0) return -1; + if (tEncodeCStr(pEncoder, pTask->outputInfo.shuffleDispatcher.stbFullName) < 0) return -1; } if (tEncodeI64(pEncoder, pTask->info.triggerParam) < 0) return -1; if (tEncodeCStrWithLen(pEncoder, pTask->reserve, sizeof(pTask->reserve) - 1) < 0) return -1; @@ -189,7 +191,7 @@ int32_t tDecodeStreamTask(SDecoder* pDecoder, SStreamTask* pTask) { int32_t epSz = -1; if (tDecodeI32(pDecoder, &epSz) < 0) return -1; - pTask->pUpstreamInfoList = taosArrayInit(epSz, POINTER_BYTES); + pTask->upstreamInfo.pList = taosArrayInit(epSz, POINTER_BYTES); for (int32_t i = 0; i < epSz; i++) { SStreamChildEpInfo* pInfo = taosMemoryCalloc(1, sizeof(SStreamChildEpInfo)); if (pInfo == NULL) return -1; @@ -197,7 +199,7 @@ int32_t tDecodeStreamTask(SDecoder* pDecoder, SStreamTask* pTask) { taosMemoryFreeClear(pInfo); return -1; } - taosArrayPush(pTask->pUpstreamInfoList, &pInfo); + taosArrayPush(pTask->upstreamInfo.pList, &pInfo); } if (pTask->info.taskLevel != TASK_LEVEL__SINK) { @@ -205,22 +207,22 @@ int32_t tDecodeStreamTask(SDecoder* pDecoder, SStreamTask* pTask) { } if (pTask->outputInfo.type == TASK_OUTPUT__TABLE) { - if (tDecodeI64(pDecoder, &pTask->tbSink.stbUid) < 0) return -1; - if (tDecodeCStrTo(pDecoder, pTask->tbSink.stbFullName) < 0) return -1; - pTask->tbSink.pSchemaWrapper = taosMemoryCalloc(1, sizeof(SSchemaWrapper)); - if (pTask->tbSink.pSchemaWrapper == NULL) return -1; - if (tDecodeSSchemaWrapper(pDecoder, pTask->tbSink.pSchemaWrapper) < 0) return -1; + if (tDecodeI64(pDecoder, &pTask->outputInfo.tbSink.stbUid) < 0) return -1; + if (tDecodeCStrTo(pDecoder, pTask->outputInfo.tbSink.stbFullName) < 0) return -1; + pTask->outputInfo.tbSink.pSchemaWrapper = taosMemoryCalloc(1, sizeof(SSchemaWrapper)); + if (pTask->outputInfo.tbSink.pSchemaWrapper == NULL) return -1; + if (tDecodeSSchemaWrapper(pDecoder, pTask->outputInfo.tbSink.pSchemaWrapper) < 0) return -1; } else if (pTask->outputInfo.type == TASK_OUTPUT__SMA) { - if (tDecodeI64(pDecoder, &pTask->smaSink.smaId) < 0) return -1; + if (tDecodeI64(pDecoder, &pTask->outputInfo.smaSink.smaId) < 0) return -1; } else if (pTask->outputInfo.type == TASK_OUTPUT__FETCH) { - if (tDecodeI8(pDecoder, &pTask->fetchSink.reserved) < 0) return -1; + if (tDecodeI8(pDecoder, &pTask->outputInfo.fetchSink.reserved) < 0) return -1; } else if (pTask->outputInfo.type == TASK_OUTPUT__FIXED_DISPATCH) { - if (tDecodeI32(pDecoder, &pTask->fixedDispatcher.taskId) < 0) return -1; - if (tDecodeI32(pDecoder, &pTask->fixedDispatcher.nodeId) < 0) return -1; - if (tDecodeSEpSet(pDecoder, &pTask->fixedDispatcher.epSet) < 0) return -1; + if (tDecodeI32(pDecoder, &pTask->outputInfo.fixedDispatcher.taskId) < 0) return -1; + if (tDecodeI32(pDecoder, &pTask->outputInfo.fixedDispatcher.nodeId) < 0) return -1; + if (tDecodeSEpSet(pDecoder, &pTask->outputInfo.fixedDispatcher.epSet) < 0) return -1; } else if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { - if (tDeserializeSUseDbRspImp(pDecoder, &pTask->shuffleDispatcher.dbInfo) < 0) return -1; - if (tDecodeCStrTo(pDecoder, pTask->shuffleDispatcher.stbFullName) < 0) return -1; + if (tDeserializeSUseDbRspImp(pDecoder, &pTask->outputInfo.shuffleDispatcher.dbInfo) < 0) return -1; + if (tDecodeCStrTo(pDecoder, pTask->outputInfo.shuffleDispatcher.stbFullName) < 0) return -1; } if (tDecodeI64(pDecoder, &pTask->info.triggerParam) < 0) return -1; if (tDecodeCStrTo(pDecoder, pTask->reserve) < 0) return -1; @@ -331,8 +333,8 @@ void tFreeStreamTask(SStreamTask* pTask) { streamQueueClose(pTask->inputInfo.queue, pTask->id.taskId); } - if (pTask->outputInfo.queue) { - streamQueueClose(pTask->outputInfo.queue, pTask->id.taskId); + if (pTask->outputq.queue) { + streamQueueClose(pTask->outputq.queue, pTask->id.taskId); } if (pTask->exec.qmsg) { @@ -356,11 +358,11 @@ void tFreeStreamTask(SStreamTask* pTask) { } if (pTask->outputInfo.type == TASK_OUTPUT__TABLE) { - tDeleteSchemaWrapper(pTask->tbSink.pSchemaWrapper); - taosMemoryFree(pTask->tbSink.pTSchema); - tSimpleHashCleanup(pTask->tbSink.pTblInfo); + tDeleteSchemaWrapper(pTask->outputInfo.tbSink.pSchemaWrapper); + taosMemoryFree(pTask->outputInfo.tbSink.pTSchema); + tSimpleHashCleanup(pTask->outputInfo.tbSink.pTblInfo); } else if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { - taosArrayDestroy(pTask->shuffleDispatcher.dbInfo.pVgroupInfos); + taosArrayDestroy(pTask->outputInfo.shuffleDispatcher.dbInfo.pVgroupInfos); pTask->checkReqIds = taosArrayDestroy(pTask->checkReqIds); } @@ -382,13 +384,10 @@ void tFreeStreamTask(SStreamTask* pTask) { pTask->pRspMsgList = NULL; } - if (pTask->pUpstreamInfoList != NULL) { - taosArrayDestroyEx(pTask->pUpstreamInfoList, freeUpstreamItem); - pTask->pUpstreamInfoList = NULL; - } + streamTaskDestroyUpstreamInfo(&pTask->upstreamInfo); pTask->msgInfo.pRetryList = taosArrayDestroy(pTask->msgInfo.pRetryList); - taosMemoryFree(pTask->pTokenBucket); + taosMemoryFree(pTask->outputInfo.pTokenBucket); taosThreadMutexDestroy(&pTask->lock); taosMemoryFree(pTask); @@ -401,16 +400,16 @@ int32_t streamTaskInit(SStreamTask* pTask, SStreamMeta* pMeta, SMsgCb* pMsgCb, i pTask->status.schedStatus = TASK_SCHED_STATUS__INACTIVE; pTask->status.timerActive = 0; pTask->inputInfo.queue = streamQueueOpen(512 << 10); - pTask->outputInfo.queue = streamQueueOpen(512 << 10); + pTask->outputq.queue = streamQueueOpen(512 << 10); - if (pTask->inputInfo.queue == NULL || pTask->outputInfo.queue == NULL) { + if (pTask->inputInfo.queue == NULL || pTask->outputq.queue == NULL) { stError("s-task:%s failed to prepare the input/output queue, initialize task failed", pTask->id.idStr); return TSDB_CODE_OUT_OF_MEMORY; } pTask->execInfo.created = taosGetTimestampMs(); pTask->inputInfo.status = TASK_INPUT_STATUS__NORMAL; - pTask->outputInfo.status = TASK_OUTPUT_STATUS__NORMAL; + pTask->outputq.status = TASK_OUTPUT_STATUS__NORMAL; pTask->pMeta = pMeta; pTask->chkInfo.checkpointVer = ver - 1; @@ -420,15 +419,15 @@ int32_t streamTaskInit(SStreamTask* pTask, SStreamMeta* pMeta, SMsgCb* pMsgCb, i pTask->pMsgCb = pMsgCb; pTask->msgInfo.pRetryList = taosArrayInit(4, sizeof(int32_t)); - pTask->pTokenBucket = taosMemoryCalloc(1, sizeof(STokenBucket)); - if (pTask->pTokenBucket == NULL) { + pTask->outputInfo.pTokenBucket = taosMemoryCalloc(1, sizeof(STokenBucket)); + if (pTask->outputInfo.pTokenBucket == NULL) { stError("s-task:%s failed to prepare the tokenBucket, code:%s", pTask->id.idStr, tstrerror(TSDB_CODE_OUT_OF_MEMORY)); return TSDB_CODE_OUT_OF_MEMORY; } // 2MiB per second for sink task // 50 times sink operator per second - streamTaskInitTokenBucket(pTask->pTokenBucket, 50, 50, 2); + streamTaskInitTokenBucket(pTask->outputInfo.pTokenBucket, 50, 50, 2); TdThreadMutexAttr attr = {0}; int code = taosThreadMutexAttrInit(&attr); @@ -457,7 +456,7 @@ int32_t streamTaskGetNumOfDownstream(const SStreamTask* pTask) { if (type == TASK_OUTPUT__FIXED_DISPATCH || type == TASK_OUTPUT__TABLE) { return 1; } else { - SArray* vgInfo = pTask->shuffleDispatcher.dbInfo.pVgroupInfos; + SArray* vgInfo = pTask->outputInfo.shuffleDispatcher.dbInfo.pVgroupInfos; return taosArrayGetSize(vgInfo); } } @@ -485,11 +484,11 @@ int32_t streamTaskSetUpstreamInfo(SStreamTask* pTask, const SStreamTask* pUpstre return TSDB_CODE_OUT_OF_MEMORY; } - if (pTask->pUpstreamInfoList == NULL) { - pTask->pUpstreamInfoList = taosArrayInit(4, POINTER_BYTES); + if (pTask->upstreamInfo.pList == NULL) { + pTask->upstreamInfo.pList = taosArrayInit(4, POINTER_BYTES); } - taosArrayPush(pTask->pUpstreamInfoList, &pEpInfo); + taosArrayPush(pTask->upstreamInfo.pList, &pEpInfo); return TSDB_CODE_SUCCESS; } @@ -497,9 +496,9 @@ void streamTaskUpdateUpstreamInfo(SStreamTask* pTask, int32_t nodeId, const SEpS char buf[512] = {0}; EPSET_TO_STR(pEpSet, buf); - int32_t numOfUpstream = taosArrayGetSize(pTask->pUpstreamInfoList); + int32_t numOfUpstream = taosArrayGetSize(pTask->upstreamInfo.pList); for (int32_t i = 0; i < numOfUpstream; ++i) { - SStreamChildEpInfo* pInfo = taosArrayGetP(pTask->pUpstreamInfoList, i); + SStreamChildEpInfo* pInfo = taosArrayGetP(pTask->upstreamInfo.pList, i); if (pInfo->nodeId == nodeId) { epsetAssign(&pInfo->epSet, pEpSet); stDebug("s-task:0x%x update the upstreamInfo taskId:0x%x(nodeId:%d) newEpset:%s", pTask->id.taskId, @@ -509,8 +508,16 @@ void streamTaskUpdateUpstreamInfo(SStreamTask* pTask, int32_t nodeId, const SEpS } } +void streamTaskDestroyUpstreamInfo(SUpstreamInfo* pUpstreamInfo) { + if (pUpstreamInfo->pList != NULL) { + taosArrayDestroyEx(pUpstreamInfo->pList, freeUpstreamItem); + pUpstreamInfo->numOfClosed = 0; + pUpstreamInfo->pList = NULL; + } +} + void streamTaskSetFixedDownstreamInfo(SStreamTask* pTask, const SStreamTask* pDownstreamTask) { - STaskDispatcherFixed* pDispatcher = &pTask->fixedDispatcher; + STaskDispatcherFixed* pDispatcher = &pTask->outputInfo.fixedDispatcher; pDispatcher->taskId = pDownstreamTask->id.taskId; pDispatcher->nodeId = pDownstreamTask->info.nodeId; pDispatcher->epSet = pDownstreamTask->info.epSet; @@ -525,7 +532,7 @@ void streamTaskUpdateDownstreamInfo(SStreamTask* pTask, int32_t nodeId, const SE int8_t type = pTask->outputInfo.type; if (type == TASK_OUTPUT__SHUFFLE_DISPATCH) { - SArray* pVgs = pTask->shuffleDispatcher.dbInfo.pVgroupInfos; + SArray* pVgs = pTask->outputInfo.shuffleDispatcher.dbInfo.pVgroupInfos; int32_t numOfVgroups = taosArrayGetSize(pVgs); for (int32_t i = 0; i < numOfVgroups; i++) { @@ -539,7 +546,7 @@ void streamTaskUpdateDownstreamInfo(SStreamTask* pTask, int32_t nodeId, const SE } } } else if (type == TASK_OUTPUT__FIXED_DISPATCH) { - STaskDispatcherFixed* pDispatcher = &pTask->fixedDispatcher; + STaskDispatcherFixed* pDispatcher = &pTask->outputInfo.fixedDispatcher; if (pDispatcher->nodeId == nodeId) { epsetAssign(&pDispatcher->epSet, pEpSet); stDebug("s-task:0x%x update the dispatch info, task:0x%x(nodeId:%d) newEpSet:%s", pTask->id.taskId, @@ -620,15 +627,19 @@ void streamTaskResetUpstreamStageInfo(SStreamTask* pTask) { return; } - int32_t size = taosArrayGetSize(pTask->pUpstreamInfoList); + int32_t size = taosArrayGetSize(pTask->upstreamInfo.pList); for (int32_t i = 0; i < size; ++i) { - SStreamChildEpInfo* pInfo = taosArrayGetP(pTask->pUpstreamInfoList, i); + SStreamChildEpInfo* pInfo = taosArrayGetP(pTask->upstreamInfo.pList, i); pInfo->stage = -1; } stDebug("s-task:%s reset all upstream tasks stage info", pTask->id.idStr); } +bool streamTaskAllUpstreamClosed(SStreamTask* pTask) { + return pTask->upstreamInfo.numOfClosed == taosArrayGetSize(pTask->upstreamInfo.pList); +} + bool streamTaskSetSchedStatusWait(SStreamTask* pTask) { bool ret = false; From da833ccb30f6702548516aa62b83ad0c5d6ce73a Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sat, 7 Oct 2023 19:20:48 +0800 Subject: [PATCH 140/158] refactor: inc the dump batch. --- source/libs/stream/src/streamExec.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index 31ccbe50f6..acc2e49b83 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -17,7 +17,7 @@ // maximum allowed processed block batches. One block may include several submit blocks #define MAX_STREAM_EXEC_BATCH_NUM 32 -#define STREAM_RESULT_DUMP_THRESHOLD 100 +#define STREAM_RESULT_DUMP_THRESHOLD 300 #define STREAM_RESULT_DUMP_SIZE_THRESHOLD (1048576 * 1) static int32_t streamDoTransferStateToStreamTask(SStreamTask* pTask); From 47804748774f83f206430af8145da32476913c74 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sat, 7 Oct 2023 19:25:40 +0800 Subject: [PATCH 141/158] fix(stream): continue --- source/libs/stream/src/streamExec.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index acc2e49b83..12b51e6c93 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -621,7 +621,7 @@ int32_t streamExecTask(SStreamTask* pTask) { } taosThreadMutexLock(&pTask->lock); - if ((streamQueueGetNumOfItems(pTask->inputInfo.queue) > 0) || streamTaskShouldStop(&pTask->status) || + if ((streamQueueGetNumOfItems(pTask->inputInfo.queue) == 0) || streamTaskShouldStop(&pTask->status) || streamTaskShouldPause(&pTask->status)) { atomic_store_8(&pTask->status.schedStatus, TASK_SCHED_STATUS__INACTIVE); taosThreadMutexUnlock(&pTask->lock); From 9af8ce21ed31e6caf55955989dff4be200bc2fee Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sat, 7 Oct 2023 19:41:49 +0800 Subject: [PATCH 142/158] fix(stream): wait for a while when no available token. --- source/libs/stream/src/streamQueue.c | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/source/libs/stream/src/streamQueue.c b/source/libs/stream/src/streamQueue.c index 676e85eadc..7a1ac052a6 100644 --- a/source/libs/stream/src/streamQueue.c +++ b/source/libs/stream/src/streamQueue.c @@ -155,10 +155,6 @@ const char* streamQueueItemGetTypeStr(int32_t type) { } } -static void doLaunchSinkTask(void* param, void* tmrId) { - -} - int32_t streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueItem** pInput, int32_t* numOfBlocks, int32_t* blockSize) { int32_t retryTimes = 0; @@ -172,7 +168,8 @@ int32_t streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueItem** pInpu // no available token in bucket for sink task, let's wait for a little bit if (taskLevel == TASK_LEVEL__SINK && (!streamTaskExtractAvailableToken(pTask->outputInfo.pTokenBucket, pTask->id.idStr))) { - stDebug("s-task:%s no available token in bucket for sink data, wait", id); + stDebug("s-task:%s no available token in bucket for sink data, wait for 50ms", id); + taosMsleep(50); // if (streamTaskAllUpstreamClosed(pTask)) { // int32_t ref = atomic_add_fetch_32(&pTask->status.timerActive, 1); @@ -433,7 +430,7 @@ static void fillTokenBucket(STokenBucket* pBucket, const char* id) { pBucket->quotaRemain = TMIN(pBucket->quotaRemain + incSize, pBucket->quotaCapacity); } - if (incNum > 0) { + if (incNum > 0 || incSize > 0) { stDebug("new token and capacity available, current token:%d inc:%d, current quota:%.2fMiB inc:%.2fMiB, ts:%" PRId64 " idle for %.2f Sec, %s", pBucket->numOfToken, incNum, pBucket->quotaRemain, incSize, now, delta / 1000.0, id); From 9a7167aabaf5fa8440ba329b7d80a4d71eccfa06 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sat, 7 Oct 2023 19:47:49 +0800 Subject: [PATCH 143/158] fix(stream): sleep for 10ms --- include/libs/stream/tstream.h | 2 +- source/libs/stream/src/streamQueue.c | 15 +-------------- 2 files changed, 2 insertions(+), 15 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 5c5a2e6adb..17c3fbf9c6 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -358,7 +358,7 @@ typedef struct STaskOutputInfo { STaskSinkFetch fetchSink; }; - void* pTimer; // timer for launch sink tasks +// void* pTimer; // timer for launch sink tasks int8_t type; STokenBucket* pTokenBucket; } STaskOutputInfo; diff --git a/source/libs/stream/src/streamQueue.c b/source/libs/stream/src/streamQueue.c index 7a1ac052a6..776b69848c 100644 --- a/source/libs/stream/src/streamQueue.c +++ b/source/libs/stream/src/streamQueue.c @@ -169,20 +169,7 @@ int32_t streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueItem** pInpu // no available token in bucket for sink task, let's wait for a little bit if (taskLevel == TASK_LEVEL__SINK && (!streamTaskExtractAvailableToken(pTask->outputInfo.pTokenBucket, pTask->id.idStr))) { stDebug("s-task:%s no available token in bucket for sink data, wait for 50ms", id); - taosMsleep(50); - -// if (streamTaskAllUpstreamClosed(pTask)) { -// int32_t ref = atomic_add_fetch_32(&pTask->status.timerActive, 1); -// stDebug("s-task:%s try start task in %dms in tmr, since all upstream inputQ is closed, ref:%d", pTask->id.idStr, -// SINK_TASK_IDLE_DURATION, ref); -// -// if (pTask->outputInfo.pTimer == NULL) { -// pTask->outputInfo.pTimer = taosTmrStart(doLaunchSinkTask, SINK_TASK_IDLE_DURATION, pTask, streamEnv.timer); -// } else { -// taosTmrReset(doLaunchSinkTask, SINK_TASK_IDLE_DURATION, pTask, streamEnv.timer, &pTask->outputInfo.pTimer); -// } -// } - + taosMsleep(10); return TSDB_CODE_SUCCESS; } From 64bde93f859ab6abe0731c18cdb4c3ca1728d10e Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 9 Oct 2023 10:07:19 +0800 Subject: [PATCH 144/158] fix(stream): not idle for sink task when no tokens are available. --- source/libs/stream/src/streamQueue.c | 1 - 1 file changed, 1 deletion(-) diff --git a/source/libs/stream/src/streamQueue.c b/source/libs/stream/src/streamQueue.c index 776b69848c..ae285046ef 100644 --- a/source/libs/stream/src/streamQueue.c +++ b/source/libs/stream/src/streamQueue.c @@ -169,7 +169,6 @@ int32_t streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueItem** pInpu // no available token in bucket for sink task, let's wait for a little bit if (taskLevel == TASK_LEVEL__SINK && (!streamTaskExtractAvailableToken(pTask->outputInfo.pTokenBucket, pTask->id.idStr))) { stDebug("s-task:%s no available token in bucket for sink data, wait for 50ms", id); - taosMsleep(10); return TSDB_CODE_SUCCESS; } From c5ee299d01f9d85a665d25c6e76ed9d588e954e4 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 9 Oct 2023 10:08:48 +0800 Subject: [PATCH 145/158] fix(stream): fix error in fill-history process. --- source/dnode/vnode/src/tq/tqStreamTask.c | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/source/dnode/vnode/src/tq/tqStreamTask.c b/source/dnode/vnode/src/tq/tqStreamTask.c index 3dbb957151..d8577453ba 100644 --- a/source/dnode/vnode/src/tq/tqStreamTask.c +++ b/source/dnode/vnode/src/tq/tqStreamTask.c @@ -370,8 +370,12 @@ static bool doPutDataIntoInputQFromWal(SStreamTask* pTask, int64_t maxVer, int32 SStreamQueueItem* pItem = NULL; int32_t code = extractMsgFromWal(pTask->exec.pWalReader, (void**)&pItem, maxVer, id); - if (code != TSDB_CODE_SUCCESS || pItem == NULL) { // failed, continue + int64_t currentVer = walReaderGetCurrentVer(pTask->exec.pWalReader); + bool itemInFillhistory = handleFillhistoryScanComplete(pTask, currentVer); + if (itemInFillhistory) { + numOfNewItems += 1; + } break; } From 21fb50269078b112267983c3f81df2a091634270 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 9 Oct 2023 10:13:53 +0800 Subject: [PATCH 146/158] enh(stream): add trans to reset task status to avoid being frozen in checkpoint status due to doing checkpoint failure of partial tasks. --- include/common/tmsg.h | 2 +- include/common/tmsgdef.h | 3 +- include/libs/stream/tstream.h | 23 +- source/dnode/mgmt/mgmt_mnode/src/mmHandle.c | 1 + source/dnode/mgmt/mgmt_vnode/src/vmHandle.c | 3 +- source/dnode/mnode/impl/src/mndStream.c | 303 ++++++++++++++------ source/dnode/mnode/impl/src/mndTrans.c | 1 - source/dnode/snode/src/snode.c | 3 - source/dnode/vnode/src/inc/vnodeInt.h | 2 +- source/dnode/vnode/src/tq/tq.c | 51 ++-- source/dnode/vnode/src/tq/tqRead.c | 2 +- source/dnode/vnode/src/vnd/vnodeSvr.c | 8 +- source/libs/executor/src/scanoperator.c | 3 +- source/libs/stream/src/streamCheckpoint.c | 18 +- source/libs/stream/src/streamMeta.c | 15 +- source/libs/stream/src/streamRecover.c | 1 - source/libs/stream/src/streamTask.c | 78 ++--- 17 files changed, 309 insertions(+), 208 deletions(-) diff --git a/include/common/tmsg.h b/include/common/tmsg.h index 0c0d28e4b7..a04018c947 100644 --- a/include/common/tmsg.h +++ b/include/common/tmsg.h @@ -3237,7 +3237,7 @@ typedef struct { SMsgHead head; int64_t streamId; int32_t taskId; -} SVPauseStreamTaskReq; +} SVPauseStreamTaskReq, SVResetStreamTaskReq; typedef struct { int8_t reserved; diff --git a/include/common/tmsgdef.h b/include/common/tmsgdef.h index 279cf72f0b..c97b8398dc 100644 --- a/include/common/tmsgdef.h +++ b/include/common/tmsgdef.h @@ -302,13 +302,12 @@ enum { // WARN: new msg should be appended to segment tail TD_DEF_MSG_TYPE(TDMT_SYNC_FORCE_FOLLOWER, "sync-force-become-follower", NULL, NULL) TD_NEW_MSG_SEG(TDMT_VND_STREAM_MSG) -// TD_DEF_MSG_TYPE(TDMT_VND_STREAM_TRIGGER, "vnode-stream-trigger", NULL, NULL) TD_DEF_MSG_TYPE(TDMT_VND_STREAM_SCAN_HISTORY, "vnode-stream-scan-history", NULL, NULL) TD_DEF_MSG_TYPE(TDMT_VND_STREAM_SCAN_HISTORY_FINISH, "vnode-stream-scan-history-finish", NULL, NULL) TD_DEF_MSG_TYPE(TDMT_VND_STREAM_CHECK_POINT_SOURCE, "vnode-stream-checkpoint-source", NULL, NULL) TD_DEF_MSG_TYPE(TDMT_VND_STREAM_TASK_UPDATE, "vnode-stream-update", NULL, NULL) + TD_DEF_MSG_TYPE(TDMT_VND_STREAM_TASK_RESET, "vnode-stream-reset", NULL, NULL) TD_DEF_MSG_TYPE(TDMT_VND_STREAM_TASK_CHECK, "vnode-stream-task-check", NULL, NULL) - TD_DEF_MSG_TYPE(TDMT_STREAM_TASK_VERUPDATE, "vnode-stream-ver-update", NULL, NULL) TD_DEF_MSG_TYPE(TDMT_VND_STREAM_MAX_MSG, "vnd-stream-max", NULL, NULL) TD_NEW_MSG_SEG(TDMT_VND_TMQ_MSG) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 17c3fbf9c6..8367c47464 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -259,8 +259,9 @@ typedef struct SStreamTaskId { typedef struct SCheckpointInfo { int64_t startTs; int64_t checkpointId; - int64_t checkpointVer; // latest checkpointId version + int64_t checkpointVer; // latest checkpointId version int64_t nextProcessVer; // current offset in WAL, not serialize it + int64_t failedId; // record the latest failed checkpoint id } SCheckpointInfo; typedef struct SStreamStatus { @@ -603,13 +604,15 @@ typedef struct STaskStatusEntry { int32_t status; int32_t stage; int32_t nodeId; - int64_t verStart; // start version in WAL, only valid for source task - int64_t verEnd; // end version in WAL, only valid for source task - int64_t offset; // only valid for source task - double inputQUsed; // in MiB + int64_t verStart; // start version in WAL, only valid for source task + int64_t verEnd; // end version in WAL, only valid for source task + int64_t processedVer; // only valid for source task + int64_t activeCheckpointId; // current active checkpoint id + bool checkpointFailed; // denote if the checkpoint is failed or not + double inputQUsed; // in MiB double inputRate; - double sinkQuota; // existed quota size for sink task - double sinkDataSize; // sink to dest data size + double sinkQuota; // existed quota size for sink task + double sinkDataSize; // sink to dest data size } STaskStatusEntry; typedef struct SStreamHbMsg { @@ -732,7 +735,9 @@ int32_t streamTaskReleaseState(SStreamTask* pTask); int32_t streamTaskReloadState(SStreamTask* pTask); void streamTaskCloseUpstreamInput(SStreamTask* pTask, int32_t taskId); void streamTaskOpenAllUpstreamInput(SStreamTask* pTask); -int32_t streamTaskUpdateDataVer(SStreamTask* pTask, int64_t ver); + +void streamTaskStatusInit(STaskStatusEntry* pEntry, const SStreamTask* pTask); +void streamTaskStatusCopy(STaskStatusEntry* pDst, const STaskStatusEntry* pSrc); // source level int32_t streamSetParamForStreamScannerStep1(SStreamTask* pTask, SVersionRange* pVerRange, STimeWindow* pWindow); @@ -768,10 +773,10 @@ void streamMetaInitForSnode(SStreamMeta* pMeta); // checkpoint int32_t streamProcessCheckpointSourceReq(SStreamTask* pTask, SStreamCheckpointSourceReq* pReq); int32_t streamProcessCheckpointReadyMsg(SStreamTask* pTask); +void streamTaskClearCheckInfo(SStreamTask* pTask); int32_t streamAlignTransferState(SStreamTask* pTask); int32_t streamBuildAndSendDropTaskMsg(SMsgCb* pMsgCb, int32_t vgId, SStreamTaskId* pTaskId); -int32_t streamBuildAndSendVerUpdateMsg(SMsgCb* pMsgCb, int32_t vgId, SStreamTaskId* pTaskId, int64_t ver); int32_t streamAddCheckpointSourceRspMsg(SStreamCheckpointSourceReq* pReq, SRpcHandleInfo* pRpcInfo, SStreamTask* pTask, int8_t isSucceed); int32_t buildCheckpointSourceRsp(SStreamCheckpointSourceReq* pReq, SRpcHandleInfo* pRpcInfo, SRpcMsg* pMsg, diff --git a/source/dnode/mgmt/mgmt_mnode/src/mmHandle.c b/source/dnode/mgmt/mgmt_mnode/src/mmHandle.c index 4c43326959..5059f25ca5 100644 --- a/source/dnode/mgmt/mgmt_mnode/src/mmHandle.c +++ b/source/dnode/mgmt/mgmt_mnode/src/mmHandle.c @@ -209,6 +209,7 @@ SArray *mmGetMsgHandles() { if (dmSetMgmtHandle(pArray, TDMT_STREAM_TASK_STOP_RSP, mmPutMsgToWriteQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_VND_STREAM_CHECK_POINT_SOURCE_RSP, mmPutMsgToWriteQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_VND_STREAM_TASK_UPDATE_RSP, mmPutMsgToWriteQueue, 0) == NULL) goto _OVER; + if (dmSetMgmtHandle(pArray, TDMT_VND_STREAM_TASK_RESET_RSP, mmPutMsgToWriteQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_MND_STREAM_HEARTBEAT, mmPutMsgToReadQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_VND_ALTER_CONFIG_RSP, mmPutMsgToWriteQueue, 0) == NULL) goto _OVER; diff --git a/source/dnode/mgmt/mgmt_vnode/src/vmHandle.c b/source/dnode/mgmt/mgmt_vnode/src/vmHandle.c index 9d6b18c677..130367f5a8 100644 --- a/source/dnode/mgmt/mgmt_vnode/src/vmHandle.c +++ b/source/dnode/mgmt/mgmt_vnode/src/vmHandle.c @@ -794,14 +794,13 @@ SArray *vmGetMsgHandles() { if (dmSetMgmtHandle(pArray, TDMT_VND_STREAM_SCAN_HISTORY_FINISH_RSP, vmPutMsgToStreamQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_VND_STREAM_TASK_CHECK, vmPutMsgToStreamQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_VND_STREAM_TASK_CHECK_RSP, vmPutMsgToStreamQueue, 0) == NULL) goto _OVER; -// if (dmSetMgmtHandle(pArray, TDMT_VND_STREAM_TRIGGER, vmPutMsgToStreamQueue, 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, vmPutMsgToStreamQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_VND_STREAM_TASK_UPDATE, vmPutMsgToWriteQueue, 0) == NULL) goto _OVER; - if (dmSetMgmtHandle(pArray, TDMT_STREAM_TASK_VERUPDATE, 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/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index 82ae6a846c..8ba8f613b1 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -44,7 +44,8 @@ typedef struct SNodeEntry { typedef struct SStreamExecNodeInfo { SArray *pNodeEntryList; - int64_t ts; // snapshot ts + int64_t ts; // snapshot ts + int64_t activeCheckpoint; // active check point id SHashObj *pTaskMap; SArray *pTaskList; TdThreadMutex lock; @@ -77,14 +78,18 @@ static int32_t mndBuildStreamCheckpointSourceReq2(void **pBuf, int32_t *pLen, in static int32_t mndProcessNodeCheck(SRpcMsg *pReq); static int32_t mndProcessNodeCheckReq(SRpcMsg *pMsg); -static SArray *doExtractNodeListFromStream(SMnode *pMnode); +static SArray *extractNodeListFromStream(SMnode *pMnode); static SArray *mndTakeVgroupSnapshot(SMnode *pMnode); static SVgroupChangeInfo mndFindChangedNodeInfo(SMnode *pMnode, const SArray *pPrevNodeList, const SArray *pNodeList); -static int32_t mndPersistTransLog(SStreamObj *pStream, STrans *pTrans); + +static STrans *doCreateTrans(SMnode *pMnode, SStreamObj *pStream, const char *name); +static int32_t mndPersistTransLog(SStreamObj *pStream, STrans *pTrans); static void initTransAction(STransAction *pAction, void *pCont, int32_t contLen, int32_t msgType, const SEpSet *pEpset); +static int32_t createStreamUpdateTrans(SMnode *pMnode, SStreamObj *pStream, SVgroupChangeInfo *pInfo); static void removeStreamTasksInBuf(SStreamObj* pStream, SStreamExecNodeInfo * pExecNode); static void keepStreamTasksInBuf(SStreamObj *pStream, SStreamExecNodeInfo *pExecNode); +static int32_t removeExpirednodeEntryAndTask(SArray *pNodeSnapshot); int32_t mndInitStream(SMnode *pMnode) { SSdbTable table = { @@ -107,6 +112,7 @@ int32_t mndInitStream(SMnode *pMnode) { mndSetMsgHandle(pMnode, TDMT_STREAM_TASK_RESUME_RSP, mndTransProcessRsp); mndSetMsgHandle(pMnode, TDMT_STREAM_TASK_STOP_RSP, mndTransProcessRsp); mndSetMsgHandle(pMnode, TDMT_VND_STREAM_TASK_UPDATE_RSP, mndTransProcessRsp); + mndSetMsgHandle(pMnode, TDMT_VND_STREAM_TASK_RESET_RSP, mndTransProcessRsp); mndSetMsgHandle(pMnode, TDMT_VND_STREAM_CHECK_POINT_SOURCE_RSP, mndTransProcessRsp); mndSetMsgHandle(pMnode, TDMT_MND_STREAM_CHECKPOINT_TIMER, mndProcessStreamCheckpointTmr); @@ -579,21 +585,6 @@ int32_t mndPersistDropStreamLog(SMnode *pMnode, STrans *pTrans, SStreamObj *pStr return 0; } -static int32_t mndSetStreamRecover(SMnode *pMnode, STrans *pTrans, const SStreamObj *pStream) { - SStreamObj streamObj = {0}; - memcpy(streamObj.name, pStream->name, TSDB_STREAM_FNAME_LEN); - streamObj.status = STREAM_STATUS__RECOVER; - - SSdbRaw *pCommitRaw = mndStreamActionEncode(&streamObj); - if (pCommitRaw == NULL) return -1; - if (mndTransAppendCommitlog(pTrans, pCommitRaw) != 0) { - mError("stream trans:%d, failed to append commit log since %s", pTrans->id, terrstr()); - return -1; - } - (void)sdbSetRawStatus(pCommitRaw, SDB_STATUS_READY); - return 0; -} - static int32_t mndCreateStbForStream(SMnode *pMnode, STrans *pTrans, const SStreamObj *pStream, const char *user) { SStbObj *pStb = NULL; SDbObj *pDb = NULL; @@ -1154,7 +1145,7 @@ static int32_t mndProcessStreamDoCheckpoint(SRpcMsg *pReq) { execNodeList.pNodeEntryList = taosArrayDestroy(execNodeList.pNodeEntryList); } - execNodeList.pNodeEntryList = doExtractNodeListFromStream(pMnode); + execNodeList.pNodeEntryList = extractNodeListFromStream(pMnode); } if (taosArrayGetSize(execNodeList.pNodeEntryList) == 0) { @@ -1608,7 +1599,7 @@ static int32_t mndRetrieveStreamTask(SRpcMsg *pReq, SShowObj *pShow, SSDataBlock } else if (pTask->info.taskLevel == TASK_LEVEL__SOURCE) { // offset info const char *offsetStr = "%" PRId64 " [%" PRId64 ", %" PRId64 "]"; - sprintf(buf, offsetStr, pe->offset, pe->verStart, pe->verEnd); + sprintf(buf, offsetStr, pe->processedVer, pe->verStart, pe->verEnd); } STR_TO_VARSTR(vbuf, buf); @@ -1657,7 +1648,9 @@ static int32_t mndPauseStreamTask(STrans *pTrans, SStreamTask *pTask) { return 0; } -int32_t mndPauseAllStreamTaskImpl(STrans *pTrans, SArray *tasks) { +int32_t mndPauseAllStreamTasks(STrans *pTrans, SStreamObj *pStream) { + SArray* tasks = pStream->tasks; + int32_t size = taosArrayGetSize(tasks); for (int32_t i = 0; i < size; i++) { SArray *pTasks = taosArrayGetP(tasks, i); @@ -1677,16 +1670,6 @@ int32_t mndPauseAllStreamTaskImpl(STrans *pTrans, SArray *tasks) { return 0; } -int32_t mndPauseAllStreamTasks(STrans *pTrans, SStreamObj *pStream) { - int32_t code = mndPauseAllStreamTaskImpl(pTrans, pStream->tasks); - if (code != 0) { - return code; - } - // pStream->pHTasksList is null - // code = mndPauseAllStreamTaskImpl(pTrans, pStream->pHTasksList); - return code; -} - static int32_t mndPersistStreamLog(STrans *pTrans, const SStreamObj *pStream, int8_t status) { SStreamObj streamObj = {0}; memcpy(streamObj.name, pStream->name, TSDB_STREAM_FNAME_LEN); @@ -1740,6 +1723,7 @@ static int32_t mndProcessPauseStreamReq(SRpcMsg *pReq) { sdbRelease(pMnode->pSdb, pStream); return -1; } + mInfo("trans:%d, used to pause stream:%s", pTrans->id, pauseReq.name); mndTransSetDbName(pTrans, pStream->sourceDb, pStream->targetDb); @@ -1751,7 +1735,7 @@ static int32_t mndProcessPauseStreamReq(SRpcMsg *pReq) { // pause all tasks if (mndPauseAllStreamTasks(pTrans, pStream) < 0) { - mError("stream:%s, failed to drop task since %s", pauseReq.name, terrstr()); + mError("stream:%s, failed to pause task since %s", pauseReq.name, terrstr()); sdbRelease(pMnode->pSdb, pStream); mndTransDrop(pTrans); return -1; @@ -1978,20 +1962,9 @@ void initTransAction(STransAction *pAction, void *pCont, int32_t contLen, int32_ // todo extract method: traverse stream tasks // build trans to update the epset static int32_t createStreamUpdateTrans(SMnode *pMnode, SStreamObj *pStream, SVgroupChangeInfo *pInfo) { - STrans *pTrans = mndTransCreate(pMnode, TRN_POLICY_RETRY, TRN_CONFLICT_DB, NULL, "stream-task-update"); + STrans* pTrans = doCreateTrans(pMnode, pStream, "stream-task-update"); if (pTrans == NULL) { - mError("failed to build stream task DAG update, reason: %s", tstrerror(TSDB_CODE_OUT_OF_MEMORY)); - return -1; - } - - mDebug("start to build stream:0x%" PRIx64 " task DAG update", pStream->uid); - - mndTransSetDbName(pTrans, pStream->sourceDb, pStream->targetDb); - if (mndTransCheckConflict(pMnode, pTrans) != 0) { - mError("failed to build stream:0x%" PRIx64 " task DAG update, code:%s", pStream->uid, - tstrerror(TSDB_CODE_MND_TRANS_CONFLICT)); - mndTransDrop(pTrans); - return -1; + return terrno; } taosWLockLatch(&pStream->lock); @@ -2152,7 +2125,7 @@ static int32_t mndProcessVgroupChange(SMnode *pMnode, SVgroupChangeInfo *pChange return 0; } -static SArray *doExtractNodeListFromStream(SMnode *pMnode) { +static SArray *extractNodeListFromStream(SMnode *pMnode) { SSdb *pSdb = pMnode->pSdb; SStreamObj *pStream = NULL; void *pIter = NULL; @@ -2173,11 +2146,9 @@ static SArray *doExtractNodeListFromStream(SMnode *pMnode) { int32_t numOfTasks = taosArrayGetSize(pLevel); for (int32_t k = 0; k < numOfTasks; ++k) { SStreamTask *pTask = taosArrayGetP(pLevel, k); - SNodeEntry entry = {0}; - epsetAssign(&entry.epset, &pTask->info.epSet); - entry.nodeId = pTask->info.nodeId; - entry.hbTimestamp = -1; + SNodeEntry entry = {.hbTimestamp = -1, .nodeId = pTask->info.nodeId}; + epsetAssign(&entry.epset, &pTask->info.epSet); taosHashPut(pHash, &entry.nodeId, sizeof(entry.nodeId), &entry, sizeof(entry)); } } @@ -2234,24 +2205,28 @@ static int32_t doRemoveFromTask(SStreamExecNodeInfo* pExecNode, STaskId* pRemove return 0; } -static int32_t removeInvalidStreamTask(SArray *pNodeSnapshot) { +static bool taskNodeExists(SArray* pList, int32_t nodeId) { + size_t num = taosArrayGetSize(pList); + + for(int32_t i = 0; i < num; ++i) { + SNodeEntry* pEntry = taosArrayGet(pList, i); + if (pEntry->nodeId == nodeId) { + return true; + } + } + + return false; +} + +int32_t removeExpirednodeEntryAndTask(SArray *pNodeSnapshot) { SArray* pRemoveTaskList = taosArrayInit(4, sizeof(STaskId)); int32_t numOfTask = taosArrayGetSize(execNodeList.pTaskList); - int32_t numOfVgroups = taosArrayGetSize(pNodeSnapshot); for(int32_t i = 0; i < numOfTask; ++i) { STaskId* pId = taosArrayGet(execNodeList.pTaskList, i); STaskStatusEntry* pEntry = taosHashGet(execNodeList.pTaskMap, pId, sizeof(*pId)); - bool existed = false; - for(int32_t j = 0; j < numOfVgroups; ++j) { - SNodeEntry* pNodeEntry = taosArrayGet(pNodeSnapshot, j); - if (pNodeEntry->nodeId == pEntry->nodeId) { - existed = true; - break; - } - } - + bool existed = taskNodeExists(pNodeSnapshot, pEntry->nodeId); if (!existed) { taosArrayPush(pRemoveTaskList, pId); } @@ -2262,15 +2237,18 @@ static int32_t removeInvalidStreamTask(SArray *pNodeSnapshot) { doRemoveFromTask(&execNodeList, pId); } + mDebug("remove invalid stream tasks:%d, remain:%d", (int32_t)taosArrayGetSize(pRemoveTaskList), + (int32_t) taosArrayGetSize(execNodeList.pTaskList)); + int32_t size = taosArrayGetSize(pNodeSnapshot); SArray* pValidNodeEntryList = taosArrayInit(4, sizeof(SNodeEntry)); for(int32_t i = 0; i < taosArrayGetSize(execNodeList.pNodeEntryList); ++i) { - SNodeEntry* pExisted = taosArrayGet(execNodeList.pNodeEntryList, i); + SNodeEntry* p = taosArrayGet(execNodeList.pNodeEntryList, i); for(int32_t j = 0; j < size; ++j) { SNodeEntry* pEntry = taosArrayGet(pNodeSnapshot, j); - if (pEntry->nodeId == pExisted->nodeId) { - taosArrayPush(pValidNodeEntryList, pExisted); + if (pEntry->nodeId == p->nodeId) { + taosArrayPush(pValidNodeEntryList, p); break; } } @@ -2301,7 +2279,7 @@ static int32_t mndProcessNodeCheckReq(SRpcMsg *pMsg) { execNodeList.pNodeEntryList = taosArrayDestroy(execNodeList.pNodeEntryList); } - execNodeList.pNodeEntryList = doExtractNodeListFromStream(pMnode); + execNodeList.pNodeEntryList = extractNodeListFromStream(pMnode); } if (taosArrayGetSize(execNodeList.pNodeEntryList) == 0) { @@ -2314,7 +2292,7 @@ static int32_t mndProcessNodeCheckReq(SRpcMsg *pMsg) { SArray *pNodeSnapshot = mndTakeVgroupSnapshot(pMnode); taosThreadMutexLock(&execNodeList.lock); - removeInvalidStreamTask(pNodeSnapshot); + removeExpirednodeEntryAndTask(pNodeSnapshot); SVgroupChangeInfo changeInfo = mndFindChangedNodeInfo(pMnode, execNodeList.pNodeEntryList, pNodeSnapshot); if (taosArrayGetSize(changeInfo.pUpdateNodeList) > 0) { @@ -2342,9 +2320,13 @@ static int32_t mndProcessNodeCheckReq(SRpcMsg *pMsg) { } typedef struct SMStreamNodeCheckMsg { - int8_t holder; // // to fix windows compile error, define place holder + int8_t placeHolder; // // to fix windows compile error, define place holder } SMStreamNodeCheckMsg; +typedef struct SMStreamTaskResetMsg { + int8_t placeHolder; +} SMStreamTaskResetMsg; + static int32_t mndProcessNodeCheck(SRpcMsg *pReq) { SMnode *pMnode = pReq->info.node; SSdb *pSdb = pMnode->pSdb; @@ -2362,6 +2344,7 @@ static int32_t mndProcessNodeCheck(SRpcMsg *pReq) { void keepStreamTasksInBuf(SStreamObj *pStream, SStreamExecNodeInfo *pExecNode) { int32_t level = taosArrayGetSize(pStream->tasks); + for (int32_t i = 0; i < level; i++) { SArray *pLevel = taosArrayGetP(pStream->tasks, i); @@ -2372,11 +2355,9 @@ void keepStreamTasksInBuf(SStreamObj *pStream, SStreamExecNodeInfo *pExecNode) { STaskId id = {.streamId = pTask->id.streamId, .taskId = pTask->id.taskId}; void *p = taosHashGet(pExecNode->pTaskMap, &id, sizeof(id)); if (p == NULL) { - STaskStatusEntry entry = {.id.streamId = pTask->id.streamId, - .id.taskId = pTask->id.taskId, - .stage = -1, - .nodeId = pTask->info.nodeId, - .status = TASK_STATUS__STOP}; + STaskStatusEntry entry = {0}; + streamTaskStatusInit(&entry, pTask); + taosHashPut(pExecNode->pTaskMap, &id, sizeof(id), &entry, sizeof(entry)); taosArrayPush(pExecNode->pTaskList, &id); mInfo("s-task:0x%x add into task buffer, total:%d", (int32_t)entry.id.taskId, @@ -2417,10 +2398,150 @@ void removeStreamTasksInBuf(SStreamObj* pStream, SStreamExecNodeInfo * pExecNode ASSERT(taosHashGetSize(pExecNode->pTaskMap) == taosArrayGetSize(pExecNode->pTaskList)); } +static STrans* doCreateTrans(SMnode* pMnode, SStreamObj* pStream, const char* name) { + STrans *pTrans = mndTransCreate(pMnode, TRN_POLICY_RETRY, TRN_CONFLICT_DB_INSIDE, NULL, name); + if (pTrans == NULL) { + mError("failed to build trans:%s, reason: %s", name, tstrerror(TSDB_CODE_OUT_OF_MEMORY)); + terrno = TSDB_CODE_OUT_OF_MEMORY; + return NULL; + } + + mDebug("start to build stream:0x%" PRIx64 " task DAG update", pStream->uid); + + mndTransSetDbName(pTrans, pStream->sourceDb, pStream->targetDb); + if (mndTransCheckConflict(pMnode, pTrans) != 0) { + terrno = TSDB_CODE_MND_TRANS_CONFLICT; + mError("failed to build trans:%s for stream:0x%" PRIx64 " code:%s", name, pStream->uid, tstrerror(terrno)); + mndTransDrop(pTrans); + return NULL; + } + + terrno = 0; + return pTrans; +} + +int32_t createStreamResetStatusTrans(SMnode* pMnode, SStreamObj* pStream) { + STrans *pTrans = doCreateTrans(pMnode, pStream, "stream-task-reset"); + if (pTrans == NULL) { + return terrno; + } + + taosWLockLatch(&pStream->lock); + int32_t numOfLevels = taosArrayGetSize(pStream->tasks); + + for (int32_t j = 0; j < numOfLevels; ++j) { + SArray *pLevel = taosArrayGetP(pStream->tasks, j); + + int32_t numOfTasks = taosArrayGetSize(pLevel); + for (int32_t k = 0; k < numOfTasks; ++k) { + SStreamTask *pTask = taosArrayGetP(pLevel, k); + + // todo extract method, with pause stream task + SVResetStreamTaskReq* pReq = taosMemoryCalloc(1, sizeof(SVResetStreamTaskReq)); + if (pReq == NULL) { + terrno = TSDB_CODE_OUT_OF_MEMORY; + mError("failed to malloc in reset stream, size:%" PRIzu ", code:%s", sizeof(SVResetStreamTaskReq), + tstrerror(TSDB_CODE_OUT_OF_MEMORY)); + return terrno; + } + + pReq->head.vgId = htonl(pTask->info.nodeId); + pReq->taskId = pTask->id.taskId; + pReq->streamId = pTask->id.streamId; + + STransAction action = {0}; + initTransAction(&action, pReq, sizeof(SVResetStreamTaskReq), TDMT_VND_STREAM_TASK_RESET, &pTask->info.epSet); + if (mndTransAppendRedoAction(pTrans, &action) != 0) { + taosMemoryFree(pReq); + taosWUnLockLatch(&pStream->lock); + mndTransDrop(pTrans); + return terrno; + } + } + } + + taosWUnLockLatch(&pStream->lock); + + int32_t code = mndPersistTransLog(pStream, pTrans); + if (code != TSDB_CODE_SUCCESS) { + sdbRelease(pMnode->pSdb, pStream); + return -1; + } + + if (mndTransPrepare(pMnode, pTrans) != 0) { + mError("trans:%d, failed to prepare update stream trans since %s", pTrans->id, terrstr()); + sdbRelease(pMnode->pSdb, pStream); + mndTransDrop(pTrans); + return -1; + } + + sdbRelease(pMnode->pSdb, pStream); + mndTransDrop(pTrans); + + return TSDB_CODE_ACTION_IN_PROGRESS; +} + +int32_t mndResetFromCheckpoint(SMnode* pMnode) { + // find the checkpoint trans id + int32_t transId = 0; + + { + SSdb *pSdb = pMnode->pSdb; + STrans *pTrans = NULL; + void* pIter = NULL; + while (1) { + pIter = sdbFetch(pSdb, SDB_TRANS, pIter, (void **)&pTrans); + if (pIter == NULL) { + break; + } + + if (strncmp(pTrans->opername, MND_STREAM_CHECKPOINT_NAME, tListLen(pTrans->opername) - 1) == 0) { + transId = pTrans->id; + sdbRelease(pSdb, pTrans); + sdbCancelFetch(pSdb, pIter); + break; + } + + sdbRelease(pSdb, pTrans); + } + } + + if (transId == 0) { + mError("failed to find the checkpoint trans, reset not executed"); + return TSDB_CODE_SUCCESS; + } + + STrans* pTrans = mndAcquireTrans(pMnode, transId); + mndKillTrans(pMnode, pTrans); + + // set all tasks status to be normal, refactor later to be stream level, instead of vnode level. + SSdb *pSdb = pMnode->pSdb; + SStreamObj *pStream = NULL; + void *pIter = NULL; + while (1) { + pIter = sdbFetch(pSdb, SDB_STREAM, pIter, (void **)&pStream); + if (pIter == NULL) { + break; + } + + mDebug("stream:%s (0x%" PRIx64 ") reset checkpoint procedure, create reset trans", pStream->name, pStream->uid); + int32_t code = createStreamResetStatusTrans(pMnode, pStream); + if (code != TSDB_CODE_SUCCESS) { + sdbCancelFetch(pSdb, pIter); + return code; + } + } + + return 0; +} + int32_t mndProcessStreamHb(SRpcMsg *pReq) { SMnode *pMnode = pReq->info.node; SStreamHbMsg req = {0}; + bool checkpointFailed = false; + int64_t activeCheckpointId = 0; + SDecoder decoder = {0}; tDecoderInit(&decoder, pReq->pCont, pReq->contLen); @@ -2441,9 +2562,9 @@ int32_t mndProcessStreamHb(SRpcMsg *pReq) { for (int32_t i = 0; i < req.numOfTasks; ++i) { STaskStatusEntry *p = taosArrayGet(req.pTaskStatus, i); - STaskStatusEntry* pEntry = taosHashGet(execNodeList.pTaskMap, &p->id, sizeof(p->id)); + STaskStatusEntry *pEntry = taosHashGet(execNodeList.pTaskMap, &p->id, sizeof(p->id)); if (pEntry == NULL) { - mError("s-task:0x%"PRIx64" not found in mnode task list", p->id.taskId); + mError("s-task:0x%" PRIx64 " not found in mnode task list", p->id.taskId); continue; } @@ -2461,16 +2582,18 @@ int32_t mndProcessStreamHb(SRpcMsg *pReq) { } } } else { - pEntry->stage = p->stage; - pEntry->inputQUsed = p->inputQUsed; - pEntry->inputRate = p->inputRate; -// pEntry->outputQUsed = p->outputQUsed; -// pEntry->outputRate = p->outputRate; - pEntry->offset = p->offset; - pEntry->verStart = p->verStart; - pEntry->verEnd = p->verEnd; - pEntry->sinkQuota = p->sinkQuota; - pEntry->sinkDataSize = p->sinkDataSize; + streamTaskStatusCopy(pEntry, p); + if (p->activeCheckpointId != 0) { + if (activeCheckpointId != 0) { + ASSERT(activeCheckpointId == p->activeCheckpointId); + } else { + activeCheckpointId = p->activeCheckpointId; + } + + if (p->checkpointFailed) { + checkpointFailed = p->checkpointFailed; + } + } } pEntry->status = p->status; @@ -2479,6 +2602,18 @@ int32_t mndProcessStreamHb(SRpcMsg *pReq) { } } + // current checkpoint is failed, rollback from the checkpoint trans + // kill the checkpoint trans and then set all tasks status to be normal + if (checkpointFailed && activeCheckpointId != 0) { + if (execNodeList.activeCheckpoint != activeCheckpointId) { + mInfo("checkpointId:%"PRId64" failed, issue task-reset trans to reset all tasks status", activeCheckpointId); + execNodeList.activeCheckpoint = activeCheckpointId; + mndResetFromCheckpoint(pMnode); + } else { + mDebug("checkpoint:%"PRId64" reset has issued already, ignore it", activeCheckpointId); + } + } + taosThreadMutexUnlock(&execNodeList.lock); taosArrayDestroy(req.pTaskStatus); diff --git a/source/dnode/mnode/impl/src/mndTrans.c b/source/dnode/mnode/impl/src/mndTrans.c index 5d150b731c..7a526016cc 100644 --- a/source/dnode/mnode/impl/src/mndTrans.c +++ b/source/dnode/mnode/impl/src/mndTrans.c @@ -1695,7 +1695,6 @@ static int32_t mndRetrieveTrans(SRpcMsg *pReq, SShowObj *pShow, SSDataBlock *pBl int32_t numOfRows = 0; STrans *pTrans = NULL; int32_t cols = 0; - char *pWrite; while (numOfRows < rows) { pShow->pIter = sdbFetch(pSdb, SDB_TRANS, pShow->pIter, (void **)&pTrans); diff --git a/source/dnode/snode/src/snode.c b/source/dnode/snode/src/snode.c index 7fb0b6b40a..6451dba2da 100644 --- a/source/dnode/snode/src/snode.c +++ b/source/dnode/snode/src/snode.c @@ -177,9 +177,6 @@ int32_t sndProcessTaskDeployReq(SSnode *pSnode, char *msg, int32_t msgLen) { qDebug("snode:%d s-task:%s is deployed on snode and add into meta, status:%s, numOfTasks:%d", SNODE_HANDLE, pTask->id.idStr, streamGetTaskStatusStr(pTask->status.taskStatus), numOfTasks); - // send msg to update the nextProcessedVer attribute for this task if it is a stream task - streamBuildAndSendVerUpdateMsg(pTask->pMsgCb, pSnode->pMeta->vgId, &pTask->id, 0); - streamTaskCheckDownstream(pTask); return 0; } diff --git a/source/dnode/vnode/src/inc/vnodeInt.h b/source/dnode/vnode/src/inc/vnodeInt.h index 2ffdf2fced..e1b75db723 100644 --- a/source/dnode/vnode/src/inc/vnodeInt.h +++ b/source/dnode/vnode/src/inc/vnodeInt.h @@ -227,7 +227,7 @@ int tqScanWalAsync(STQ* pTq, bool ckPause); int32_t tqProcessTaskCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg* pRsp); int32_t tqProcessTaskCheckpointReadyMsg(STQ* pTq, SRpcMsg* pMsg); int32_t tqProcessTaskUpdateReq(STQ* pTq, SRpcMsg* pMsg); -int32_t tqProcessTaskDataVerUpdateReq(STQ* pTq, char* pMsg, int32_t msgLen); +int32_t tqProcessTaskResetReq(STQ* pTq, SRpcMsg* pMsg); int32_t tqCheckAndRunStreamTaskAsync(STQ* pTq); int tqCommit(STQ*); diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 2be48b3c20..326c3eec8e 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1011,16 +1011,6 @@ int32_t tqProcessTaskDeployReq(STQ* pTq, int64_t sversion, char* msg, int32_t ms // only handled in the leader node if (vnodeIsRoleLeader(pTq->pVnode)) { tqDebug("vgId:%d s-task:0x%x is deployed and add into meta, numOfTasks:%d", vgId, taskId, numOfTasks); -#if 0 - if (pTq->pVnode->restored) { - SStreamTask* p = streamMetaAcquireTask(pStreamMeta, streamId, taskId); - if (p != NULL) { - // send msg to update the nextProcessedVer attribute for this task if it is a stream task - streamBuildAndSendVerUpdateMsg(p->pMsgCb, vgId, &p->id, sversion); - streamMetaReleaseTask(pStreamMeta, p); - } - } -#endif SStreamTask* p = streamMetaAcquireTask(pStreamMeta, streamId, taskId); bool restored = pTq->pVnode->restored; @@ -1661,7 +1651,7 @@ int32_t tqProcessTaskCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg* pRsp) int32_t len = pMsg->contLen - sizeof(SMsgHead); int32_t code = 0; - // disable auto rsp to source + // disable auto rsp to mnode pRsp->info.handle = NULL; // todo: add counter to make sure other tasks would not be trapped in checkpoint state @@ -1705,9 +1695,11 @@ int32_t tqProcessTaskCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg* pRsp) return TSDB_CODE_SUCCESS; } - // todo: handle the partial failure cases // downstream not ready, current the stream tasks are not all ready. Ignore this checkpoint req. if (pTask->status.downstreamReady != 1) { + pTask->chkInfo.failedId = req.checkpointId; // record the latest failed checkpoint id + pTask->checkpointingId = req.checkpointId; + qError("s-task:%s not ready for checkpoint, since downstream not ready, ignore this checkpoint:%" PRId64 ", set it failure", pTask->id.idStr, req.checkpointId); streamMetaReleaseTask(pMeta, pTask); @@ -1924,34 +1916,25 @@ int32_t tqProcessTaskUpdateReq(STQ* pTq, SRpcMsg* pMsg) { return rsp.code; } -int32_t tqProcessTaskDataVerUpdateReq(STQ* pTq, char* pMsg, int32_t msgLen) { +int32_t tqProcessTaskResetReq(STQ* pTq, SRpcMsg* pMsg) { + SVPauseStreamTaskReq* pReq = (SVPauseStreamTaskReq*) pMsg->pCont; + SStreamMeta* pMeta = pTq->pStreamMeta; - int32_t vgId = pMeta->vgId; - - SVStreamTaskVerUpdateReq* pReq = (SVStreamTaskVerUpdateReq*) pMsg; - tqDebug("vgId:%d receive msg to update task dataVer, task:0x%x dataVer:%" PRId64, vgId, pReq->taskId, pReq->dataVer); - SStreamTask* pTask = streamMetaAcquireTask(pMeta, pReq->streamId, pReq->taskId); if (pTask == NULL) { - tqError("vgId:%d process dataVer msg, failed to find task:0x%x, it may have been destroyed", vgId, pReq->taskId); - return -1; + tqError("vgId:%d process task-reset req, failed to acquire task:0x%x, it may have been dropped already", pMeta->vgId, + pReq->taskId); + return TSDB_CODE_SUCCESS; } - // commit the dataVer update - streamTaskUpdateDataVer(pTask, pReq->dataVer); + tqDebug("s-task:%s receive task-reset msg from mnode, reset status and ready for data processing", pTask->id.idStr); - if (vnodeIsLeader(pTq->pVnode)) { - if (pTq->pVnode->restored) { - ASSERT(pTask->execInfo.init == 0); - - pTask->execInfo.init = taosGetTimestampMs(); - tqDebug("s-task:%s set the init ts:%" PRId64, pTask->id.idStr, pTask->execInfo.init); - streamTaskCheckDownstream(pTask); - } else { - tqWarn("s-task:%s not launched since vnode (vgId:%d) not ready", pTask->id.idStr, vgId); - } + // clear flag set during do checkpoint, and open inputQ for all upstream tasks + if (pTask->status.taskStatus == TASK_STATUS__CK) { + streamTaskClearCheckInfo(pTask); + streamSetStatusNormal(pTask); } streamMetaReleaseTask(pMeta, pTask); - return 0; -} + return TSDB_CODE_SUCCESS; +} \ No newline at end of file diff --git a/source/dnode/vnode/src/tq/tqRead.c b/source/dnode/vnode/src/tq/tqRead.c index 916bd01cd0..252d3ac564 100644 --- a/source/dnode/vnode/src/tq/tqRead.c +++ b/source/dnode/vnode/src/tq/tqRead.c @@ -308,7 +308,7 @@ int32_t extractMsgFromWal(SWalReader* pReader, void** pItem, int64_t maxVer, con SWalCont* pCont = &pReader->pHead->head; int64_t ver = pCont->version; if (ver > maxVer) { - tqDebug("maxVer in WAL:%" PRId64 " reached current:%" PRId64 ", do not scan wal anymore, %s", maxVer, ver, id); + tqDebug("maxVer in WAL:%" PRId64 " reached, current:%" PRId64 ", do not scan wal anymore, %s", maxVer, ver, id); return TSDB_CODE_SUCCESS; } diff --git a/source/dnode/vnode/src/vnd/vnodeSvr.c b/source/dnode/vnode/src/vnd/vnodeSvr.c index f0cfe6cd0b..3c9b9bdb82 100644 --- a/source/dnode/vnode/src/vnd/vnodeSvr.c +++ b/source/dnode/vnode/src/vnd/vnodeSvr.c @@ -583,9 +583,11 @@ int32_t vnodeProcessWriteMsg(SVnode *pVnode, SRpcMsg *pMsg, int64_t ver, SRpcMsg goto _err; } } break; - case TDMT_STREAM_TASK_VERUPDATE: - tqProcessTaskDataVerUpdateReq(pVnode->pTq, pMsg->pCont, pMsg->contLen); - break; + case TDMT_VND_STREAM_TASK_RESET: { + if (pVnode->restored/* && vnodeIsLeader(pVnode)*/) { + tqProcessTaskResetReq(pVnode->pTq, pMsg); + } + } break; case TDMT_VND_ALTER_CONFIRM: needCommit = pVnode->config.hashChange; if (vnodeProcessAlterConfirmReq(pVnode, ver, pReq, len, pRsp) < 0) { diff --git a/source/libs/executor/src/scanoperator.c b/source/libs/executor/src/scanoperator.c index 474128007a..d40da4d234 100644 --- a/source/libs/executor/src/scanoperator.c +++ b/source/libs/executor/src/scanoperator.c @@ -1317,8 +1317,9 @@ static STimeWindow getSlidingWindow(TSKEY* startTsCol, TSKEY* endTsCol, uint64_t } static SSDataBlock* doRangeScan(SStreamScanInfo* pInfo, SSDataBlock* pSDB, int32_t tsColIndex, int32_t* pRowIndex) { - qInfo("do stream range scan. windows index:%d", *pRowIndex); + qDebug("do stream range scan. windows index:%d", *pRowIndex); bool prepareRes = true; + while (1) { SSDataBlock* pResult = NULL; pResult = doTableScan(pInfo->pTableScanOp); diff --git a/source/libs/stream/src/streamCheckpoint.c b/source/libs/stream/src/streamCheckpoint.c index 6924d99585..a26d667421 100644 --- a/source/libs/stream/src/streamCheckpoint.c +++ b/source/libs/stream/src/streamCheckpoint.c @@ -143,8 +143,8 @@ int32_t streamProcessCheckpointSourceReq(SStreamTask* pTask, SStreamCheckpointSo pTask->execInfo.checkpoint += 1; - // 2. let's dispatch checkpoint msg to downstream task directly and do nothing else. put the checkpoint block into - // inputQ, to make sure all blocks with less version have been handled by this task already. + // 2. Put the checkpoint block into inputQ, to make sure all blocks with less version have been handled by this task + // already. int32_t code = appendCheckpointIntoInputQ(pTask, STREAM_INPUT__CHECKPOINT_TRIGGER); return code; } @@ -264,6 +264,16 @@ int32_t streamProcessCheckpointReadyMsg(SStreamTask* pTask) { return 0; } +void streamTaskClearCheckInfo(SStreamTask* pTask) { + pTask->checkpointingId = 0; // clear the checkpoint id + pTask->chkInfo.failedId = 0; + pTask->chkInfo.startTs = 0; // clear the recorded start time + pTask->checkpointNotReadyTasks = 0; + pTask->checkpointAlignCnt = 0; + taosArrayClear(pTask->pReadyMsgList); + streamTaskOpenAllUpstreamInput(pTask); // open inputQ for all upstream tasks +} + int32_t streamSaveAllTaskStatus(SStreamMeta* pMeta, int64_t checkpointId) { taosWLockLatch(&pMeta->lock); @@ -283,11 +293,11 @@ int32_t streamSaveAllTaskStatus(SStreamMeta* pMeta, int64_t checkpointId) { ASSERT(p->chkInfo.checkpointId < p->checkpointingId && p->checkpointingId == checkpointId); p->chkInfo.checkpointId = p->checkpointingId; + streamTaskClearCheckInfo(p); streamSetStatusNormal(p); // save the task streamMetaSaveTask(pMeta, p); - streamTaskOpenAllUpstreamInput(p); // open inputQ for all upstream tasks stDebug( "vgId:%d s-task:%s level:%d open upstream inputQ, commit task status after checkpoint completed, " @@ -318,8 +328,6 @@ int32_t streamTaskBuildCheckpoint(SStreamTask* pTask) { ASSERT(remain >= 0); double el = (taosGetTimestampMs() - pTask->chkInfo.startTs) / 1000.0; - pTask->chkInfo.startTs = 0; // clear the recorded start time - if (remain == 0) { // all tasks are ready stDebug("s-task:%s all downstreams are ready, ready for do checkpoint", pTask->id.idStr); streamBackendDoCheckpoint(pMeta, pTask->checkpointingId); diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 97d7507194..c4aefa40ea 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -779,9 +779,11 @@ int32_t tEncodeStreamHbMsg(SEncoder* pEncoder, const SStreamHbMsg* pReq) { if (tEncodeDouble(pEncoder, ps->inputRate) < 0) return -1; if (tEncodeDouble(pEncoder, ps->sinkQuota) < 0) return -1; if (tEncodeDouble(pEncoder, ps->sinkDataSize) < 0) return -1; - if (tEncodeI64(pEncoder, ps->offset) < 0) return -1; + if (tEncodeI64(pEncoder, ps->processedVer) < 0) return -1; if (tEncodeI64(pEncoder, ps->verStart) < 0) return -1; if (tEncodeI64(pEncoder, ps->verEnd) < 0) return -1; + if (tEncodeI64(pEncoder, ps->activeCheckpointId) < 0) return -1; + if (tEncodeI8(pEncoder, ps->checkpointFailed) < 0) return -1; } tEndEncode(pEncoder); return pEncoder->pos; @@ -806,9 +808,11 @@ int32_t tDecodeStreamHbMsg(SDecoder* pDecoder, SStreamHbMsg* pReq) { if (tDecodeDouble(pDecoder, &entry.inputRate) < 0) return -1; if (tDecodeDouble(pDecoder, &entry.sinkQuota) < 0) return -1; if (tDecodeDouble(pDecoder, &entry.sinkDataSize) < 0) return -1; - if (tDecodeI64(pDecoder, &entry.offset) < 0) return -1; + if (tDecodeI64(pDecoder, &entry.processedVer) < 0) return -1; if (tDecodeI64(pDecoder, &entry.verStart) < 0) return -1; if (tDecodeI64(pDecoder, &entry.verEnd) < 0) return -1; + if (tDecodeI64(pDecoder, &entry.activeCheckpointId) < 0) return -1; + if (tDecodeI8(pDecoder, (int8_t*)&entry.checkpointFailed) < 0) return -1; entry.id.taskId = taskId; taosArrayPush(pReq->pTaskStatus, &entry); @@ -896,8 +900,13 @@ void metaHbToMnode(void* param, void* tmrId) { entry.sinkDataSize = SIZE_IN_MiB((*pTask)->execInfo.sink.dataSize); } + if ((*pTask)->checkpointingId != 0) { + entry.checkpointFailed = ((*pTask)->chkInfo.failedId >= (*pTask)->checkpointingId); + entry.activeCheckpointId = (*pTask)->checkpointingId; + } + if ((*pTask)->exec.pWalReader != NULL) { - entry.offset = (*pTask)->chkInfo.nextProcessVer; + entry.processedVer = (*pTask)->chkInfo.nextProcessVer - 1; walReaderValidVersionRange((*pTask)->exec.pWalReader, &entry.verStart, &entry.verEnd); } diff --git a/source/libs/stream/src/streamRecover.c b/source/libs/stream/src/streamRecover.c index 55ed555af6..43c1b84fa7 100644 --- a/source/libs/stream/src/streamRecover.c +++ b/source/libs/stream/src/streamRecover.c @@ -305,7 +305,6 @@ static void doProcessDownstreamReadyRsp(SStreamTask* pTask, int32_t numOfReqs) { streamLaunchFillHistoryTask(pTask); } -// todo handle error int32_t streamProcessCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* pRsp) { ASSERT(pTask->id.taskId == pRsp->upstreamTaskId); const char* id = pTask->id.idStr; diff --git a/source/libs/stream/src/streamTask.c b/source/libs/stream/src/streamTask.c index b6a60e28d7..37af1ce64f 100644 --- a/source/libs/stream/src/streamTask.c +++ b/source/libs/stream/src/streamTask.c @@ -700,63 +700,6 @@ int32_t streamBuildAndSendDropTaskMsg(SMsgCb* pMsgCb, int32_t vgId, SStreamTaskI return code; } -int32_t streamBuildAndSendVerUpdateMsg(SMsgCb* pMsgCb, int32_t vgId, SStreamTaskId* pTaskId, int64_t ver) { - SVStreamTaskVerUpdateReq* pReq = rpcMallocCont(sizeof(SVStreamTaskVerUpdateReq)); - if (pReq == NULL) { - terrno = TSDB_CODE_OUT_OF_MEMORY; - return -1; - } - - pReq->head.vgId = vgId; - pReq->taskId = pTaskId->taskId; - pReq->streamId = pTaskId->streamId; - pReq->dataVer = ver; - - SRpcMsg msg = {.msgType = TDMT_STREAM_TASK_VERUPDATE, .pCont = pReq, .contLen = sizeof(SVStreamTaskVerUpdateReq)}; - int32_t code = tmsgPutToQueue(pMsgCb, WRITE_QUEUE, &msg); - if (code != TSDB_CODE_SUCCESS) { - stError("vgId:%d failed to send update task:0x%x dataVer msg, code:%s", vgId, pTaskId->taskId, tstrerror(code)); - return code; - } - - stDebug("vgId:%d build and send update table:0x%x dataVer:%"PRId64" msg", vgId, pTaskId->taskId, ver); - return code; -} - -int32_t streamTaskUpdateDataVer(SStreamTask* pTask, int64_t ver) { - SStreamMeta* pMeta = pTask->pMeta; - - // commit the dataVer update - int64_t prevVer = 0; - taosThreadMutexLock(&pTask->lock); - - if (pTask->chkInfo.checkpointId == 0) { - prevVer = pTask->chkInfo.nextProcessVer; - pTask->chkInfo.nextProcessVer = ver; - taosThreadMutexUnlock(&pTask->lock); - - taosWLockLatch(&pMeta->lock); - if (streamMetaSaveTask(pMeta, pTask) < 0) { -// return -1; - } - - if (streamMetaCommit(pMeta) < 0) { - // persist to disk - } - - stDebug("s-task:%s nextProcessedVer is update from %" PRId64 " to %" PRId64 " checkpointId:%" PRId64 - " checkpointVer:%" PRId64, - pTask->id.idStr, prevVer, ver, pTask->chkInfo.checkpointId, pTask->chkInfo.checkpointVer); - taosWUnLockLatch(&pMeta->lock); - } else { - stDebug("s-task:%s not update the dataVer, existed:%" PRId64 ", checkpointId:%" PRId64 " checkpointVer:%" PRId64, - pTask->id.idStr, pTask->chkInfo.nextProcessVer, pTask->chkInfo.checkpointId, pTask->chkInfo.checkpointVer); - taosThreadMutexUnlock(&pTask->lock); - } - - return TSDB_CODE_SUCCESS; -} - STaskId streamTaskExtractKey(const SStreamTask* pTask) { STaskId id = {.streamId = pTask->id.streamId, .taskId = pTask->id.taskId}; return id; @@ -788,4 +731,25 @@ const char* streamGetTaskStatusStr(int32_t status) { case TASK_STATUS__UNINIT: return "uninitialized"; default:return ""; } +} + +void streamTaskStatusInit(STaskStatusEntry* pEntry, const SStreamTask* pTask) { + pEntry->id.streamId = pTask->id.streamId; + pEntry->id.taskId = pTask->id.taskId; + pEntry->stage = -1; + pEntry->nodeId = pTask->info.nodeId; + pEntry->status = TASK_STATUS__STOP; +} + +void streamTaskStatusCopy(STaskStatusEntry* pDst, const STaskStatusEntry* pSrc) { + pDst->stage = pSrc->stage; + pDst->inputQUsed = pSrc->inputQUsed; + pDst->inputRate = pSrc->inputRate; + pDst->processedVer = pSrc->processedVer; + pDst->verStart = pSrc->verStart; + pDst->verEnd = pSrc->verEnd; + pDst->sinkQuota = pSrc->sinkQuota; + pDst->sinkDataSize = pSrc->sinkDataSize; + pDst->activeCheckpointId = pSrc->activeCheckpointId; + pDst->checkpointFailed = pSrc->checkpointFailed; } \ No newline at end of file From 847baa8b71b06c6bf672e0a1c372424ce411704e Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 9 Oct 2023 14:48:11 +0800 Subject: [PATCH 147/158] fix(stream): fix the pause failure bug. --- include/libs/stream/tstream.h | 2 +- source/libs/stream/src/streamRecover.c | 30 ++++++++++++++++++-------- tests/script/sh/deploy.sh | 1 + 3 files changed, 23 insertions(+), 10 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 8367c47464..fe669b7fac 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -697,7 +697,7 @@ char* createStreamTaskIdStr(int64_t streamId, int32_t taskId); // recover and fill history void streamTaskCheckDownstream(SStreamTask* pTask); -int32_t streamTaskLaunchScanHistory(SStreamTask* pTask); +int32_t streamTaskStartScanHistory(SStreamTask* pTask); int32_t streamTaskCheckStatus(SStreamTask* pTask, int32_t upstreamTaskId, int32_t vgId, int64_t stage); int32_t streamTaskUpdateEpsetInfo(SStreamTask* pTask, SArray* pNodeList); void streamTaskResetUpstreamStageInfo(SStreamTask* pTask); diff --git a/source/libs/stream/src/streamRecover.c b/source/libs/stream/src/streamRecover.c index 43c1b84fa7..6aacb5d2bb 100644 --- a/source/libs/stream/src/streamRecover.c +++ b/source/libs/stream/src/streamRecover.c @@ -60,10 +60,17 @@ static void streamTaskSetReady(SStreamTask* pTask, int32_t numOfReqs) { taosHashPut(pMeta->startInfo.pReadyTaskSet, &id, sizeof(id), NULL, 0); int32_t numOfTotal = streamMetaGetNumOfTasks(pMeta); + if (taosHashGetSize(pMeta->startInfo.pReadyTaskSet) == numOfTotal) { STaskStartInfo* pStartInfo = &pMeta->startInfo; pStartInfo->readyTs = pTask->execInfo.start; - pStartInfo->elapsedTime = pStartInfo->readyTs - pStartInfo->startTs; + + if (pStartInfo->startTs != 0) { + pStartInfo->elapsedTime = pStartInfo->readyTs - pStartInfo->startTs; + } else { + pStartInfo->elapsedTime = 0; + } + streamMetaResetStartInfo(pStartInfo); stDebug("vgId:%d all %d task(s) are started successfully, last ready task:%s level:%d, startTs:%" PRId64 @@ -71,6 +78,7 @@ static void streamTaskSetReady(SStreamTask* pTask, int32_t numOfReqs) { vgId, numOfTotal, pTask->id.idStr, pTask->info.taskLevel, pStartInfo->startTs, pStartInfo->readyTs, pStartInfo->elapsedTime / 1000.0); } + taosWUnLockLatch(&pMeta->lock); } @@ -94,7 +102,7 @@ int32_t streamStartScanHistoryAsync(SStreamTask* pTask, int8_t igUntreated) { return 0; } -static int32_t doLaunchScanHistoryTask(SStreamTask* pTask) { +static int32_t doStartScanHistoryTask(SStreamTask* pTask) { SVersionRange* pRange = &pTask->dataRange.range; if (pTask->info.fillHistory) { streamSetParamForScanHistory(pTask); @@ -105,15 +113,18 @@ static int32_t doLaunchScanHistoryTask(SStreamTask* pTask) { return code; } -int32_t streamTaskLaunchScanHistory(SStreamTask* pTask) { +int32_t streamTaskStartScanHistory(SStreamTask* pTask) { + ASSERT(pTask->status.downstreamReady == 1); + if (pTask->info.taskLevel == TASK_LEVEL__SOURCE) { if (pTask->status.taskStatus == TASK_STATUS__SCAN_HISTORY) { - return doLaunchScanHistoryTask(pTask); + return doStartScanHistoryTask(pTask); } else { ASSERT(pTask->status.taskStatus == TASK_STATUS__NORMAL); - stDebug("s-task:%s no need to scan-history-data, status:%s, sched-status:%d, ver:%" PRId64, pTask->id.idStr, - streamGetTaskStatusStr(pTask->status.taskStatus), pTask->status.schedStatus, - walReaderGetCurrentVer(pTask->exec.pWalReader)); + stDebug("s-task:%s no need to scan-history data, status:%s, sched-status:%d, ver:%" PRId64, pTask->id.idStr, + streamGetTaskStatusStr(pTask->status.taskStatus), pTask->status.schedStatus, + walReaderGetCurrentVer(pTask->exec.pWalReader)); + streamTaskEnablePause(pTask); } } else if (pTask->info.taskLevel == TASK_LEVEL__AGG) { if (pTask->info.fillHistory) { @@ -179,7 +190,7 @@ static int32_t doCheckDownstreamStatus(SStreamTask* pTask) { streamTaskSetReady(pTask, 0); streamTaskSetRangeStreamCalc(pTask); - streamTaskLaunchScanHistory(pTask); + streamTaskStartScanHistory(pTask); streamLaunchFillHistoryTask(pTask); } @@ -289,8 +300,9 @@ static void doProcessDownstreamReadyRsp(SStreamTask* pTask, int32_t numOfReqs) { if (status == TASK_STATUS__SCAN_HISTORY) { stDebug("s-task:%s enter into scan-history data stage, status:%s", id, str); - streamTaskLaunchScanHistory(pTask); + streamTaskStartScanHistory(pTask); } else { + // fill-history tasks are not allowed to reach here. if (pTask->info.fillHistory == 1) { stDebug("s-task:%s fill-history is set normal when start it, try to remove it,set it task to be dropping", id); pTask->status.taskStatus = TASK_STATUS__DROPPING; diff --git a/tests/script/sh/deploy.sh b/tests/script/sh/deploy.sh index 7da8da09bf..3b3d275a07 100755 --- a/tests/script/sh/deploy.sh +++ b/tests/script/sh/deploy.sh @@ -137,6 +137,7 @@ echo "idxDebugFlag 143" >> $TAOS_CFG echo "udfDebugFlag 143" >> $TAOS_CFG echo "smaDebugFlag 143" >> $TAOS_CFG echo "metaDebugFlag 143" >> $TAOS_CFG +echo "stDebugFlag 143" >> $TAOS_CFG echo "numOfLogLines 20000000" >> $TAOS_CFG echo "asyncLog 0" >> $TAOS_CFG echo "locale en_US.UTF-8" >> $TAOS_CFG From af4618f9587cde53c890c0e3629a1548e3ac3676 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 9 Oct 2023 15:26:42 +0800 Subject: [PATCH 148/158] fix(stream): remove invalid clear of rspmsg list. --- source/dnode/vnode/src/tq/tq.c | 1 + source/libs/stream/src/streamCheckpoint.c | 1 - 2 files changed, 1 insertion(+), 1 deletion(-) diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 326c3eec8e..285f4df7b4 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1932,6 +1932,7 @@ int32_t tqProcessTaskResetReq(STQ* pTq, SRpcMsg* pMsg) { // clear flag set during do checkpoint, and open inputQ for all upstream tasks if (pTask->status.taskStatus == TASK_STATUS__CK) { streamTaskClearCheckInfo(pTask); + taosArrayClear(pTask->pReadyMsgList); streamSetStatusNormal(pTask); } diff --git a/source/libs/stream/src/streamCheckpoint.c b/source/libs/stream/src/streamCheckpoint.c index a26d667421..2cde368195 100644 --- a/source/libs/stream/src/streamCheckpoint.c +++ b/source/libs/stream/src/streamCheckpoint.c @@ -270,7 +270,6 @@ void streamTaskClearCheckInfo(SStreamTask* pTask) { pTask->chkInfo.startTs = 0; // clear the recorded start time pTask->checkpointNotReadyTasks = 0; pTask->checkpointAlignCnt = 0; - taosArrayClear(pTask->pReadyMsgList); streamTaskOpenAllUpstreamInput(pTask); // open inputQ for all upstream tasks } From 7d39164133046d5035288e13e225355d5a52c47c Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 9 Oct 2023 16:33:28 +0800 Subject: [PATCH 149/158] fix(stream): disable retrieving results during checkpoint procedure --- source/libs/stream/src/stream.c | 41 ++++++++++++++++++--------------- 1 file changed, 23 insertions(+), 18 deletions(-) diff --git a/source/libs/stream/src/stream.c b/source/libs/stream/src/stream.c index 97316dba07..b7d57b1728 100644 --- a/source/libs/stream/src/stream.c +++ b/source/libs/stream/src/stream.c @@ -67,27 +67,32 @@ static void streamSchedByTimer(void* param, void* tmrId) { return; } - if (status == TASK_TRIGGER_STATUS__ACTIVE) { - SStreamTrigger* pTrigger = taosAllocateQitem(sizeof(SStreamTrigger), DEF_QITEM, 0); - if (pTrigger == NULL) { - return; - } + if (pTask->status.taskStatus == TASK_STATUS__CK) { + stDebug("s-task:%s in checkpoint procedure, not retrieve result, next:%dms", pTask->id.idStr, (int32_t) pTask->info.triggerParam); + } else { + if (status == TASK_TRIGGER_STATUS__ACTIVE) { + SStreamTrigger* pTrigger = taosAllocateQitem(sizeof(SStreamTrigger), DEF_QITEM, 0); + if (pTrigger == NULL) { + return; + } - pTrigger->type = STREAM_INPUT__GET_RES; - pTrigger->pBlock = taosMemoryCalloc(1, sizeof(SSDataBlock)); - if (pTrigger->pBlock == NULL) { - taosFreeQitem(pTrigger); - return; - } + pTrigger->type = STREAM_INPUT__GET_RES; + pTrigger->pBlock = taosMemoryCalloc(1, sizeof(SSDataBlock)); + if (pTrigger->pBlock == NULL) { + taosFreeQitem(pTrigger); + return; + } - atomic_store_8(&pTask->schedInfo.status, TASK_TRIGGER_STATUS__INACTIVE); - pTrigger->pBlock->info.type = STREAM_GET_ALL; - if (streamTaskPutDataIntoInputQ(pTask, (SStreamQueueItem*)pTrigger) < 0) { - taosTmrReset(streamSchedByTimer, (int32_t)pTask->info.triggerParam, pTask, streamEnv.timer, &pTask->schedInfo.pTimer); - return; - } + atomic_store_8(&pTask->schedInfo.status, TASK_TRIGGER_STATUS__INACTIVE); + pTrigger->pBlock->info.type = STREAM_GET_ALL; + if (streamTaskPutDataIntoInputQ(pTask, (SStreamQueueItem*)pTrigger) < 0) { + taosTmrReset(streamSchedByTimer, (int32_t)pTask->info.triggerParam, pTask, streamEnv.timer, + &pTask->schedInfo.pTimer); + return; + } - streamSchedExec(pTask); + streamSchedExec(pTask); + } } taosTmrReset(streamSchedByTimer, (int32_t)pTask->info.triggerParam, pTask, streamEnv.timer, &pTask->schedInfo.pTimer); From a25d94ab6f76e806262ceae4c8ced8570051c27a Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 9 Oct 2023 16:40:05 +0800 Subject: [PATCH 150/158] fix(stream): retry when error occurs during timer. --- source/libs/stream/src/stream.c | 24 +++++++++++++++++------- 1 file changed, 17 insertions(+), 7 deletions(-) diff --git a/source/libs/stream/src/stream.c b/source/libs/stream/src/stream.c index b7d57b1728..03ba796b2c 100644 --- a/source/libs/stream/src/stream.c +++ b/source/libs/stream/src/stream.c @@ -58,21 +58,26 @@ char* createStreamTaskIdStr(int64_t streamId, int32_t taskId) { static void streamSchedByTimer(void* param, void* tmrId) { SStreamTask* pTask = (void*)param; + const char* id = pTask->id.idStr; + int32_t nextTrigger = (int32_t)pTask->info.triggerParam; int8_t status = atomic_load_8(&pTask->schedInfo.status); - stDebug("s-task:%s in scheduler, trigger status:%d, next:%dms", pTask->id.idStr, status, (int32_t)pTask->info.triggerParam); + stDebug("s-task:%s in scheduler, trigger status:%d, next:%dms", id, status, nextTrigger); if (streamTaskShouldStop(&pTask->status) || streamTaskShouldPause(&pTask->status)) { - stDebug("s-task:%s jump out of schedTimer", pTask->id.idStr); + stDebug("s-task:%s jump out of schedTimer", id); return; } if (pTask->status.taskStatus == TASK_STATUS__CK) { - stDebug("s-task:%s in checkpoint procedure, not retrieve result, next:%dms", pTask->id.idStr, (int32_t) pTask->info.triggerParam); + stDebug("s-task:%s in checkpoint procedure, not retrieve result, next:%dms", id, nextTrigger); } else { if (status == TASK_TRIGGER_STATUS__ACTIVE) { SStreamTrigger* pTrigger = taosAllocateQitem(sizeof(SStreamTrigger), DEF_QITEM, 0); if (pTrigger == NULL) { + stError("s-task:%s failed to prepare retrieve data trigger, code:%s, try again in %dms", id, "out of memory", + nextTrigger); + taosTmrReset(streamSchedByTimer, nextTrigger, pTask, streamEnv.timer, &pTask->schedInfo.pTimer); return; } @@ -80,14 +85,19 @@ static void streamSchedByTimer(void* param, void* tmrId) { pTrigger->pBlock = taosMemoryCalloc(1, sizeof(SSDataBlock)); if (pTrigger->pBlock == NULL) { taosFreeQitem(pTrigger); + + stError("s-task:%s failed to prepare retrieve data trigger, code:%s, try again in %dms", id, "out of memory", + nextTrigger); + taosTmrReset(streamSchedByTimer, nextTrigger, pTask, streamEnv.timer, &pTask->schedInfo.pTimer); return; } atomic_store_8(&pTask->schedInfo.status, TASK_TRIGGER_STATUS__INACTIVE); pTrigger->pBlock->info.type = STREAM_GET_ALL; - if (streamTaskPutDataIntoInputQ(pTask, (SStreamQueueItem*)pTrigger) < 0) { - taosTmrReset(streamSchedByTimer, (int32_t)pTask->info.triggerParam, pTask, streamEnv.timer, - &pTask->schedInfo.pTimer); + + int32_t code = streamTaskPutDataIntoInputQ(pTask, (SStreamQueueItem*)pTrigger); + if (code != TSDB_CODE_SUCCESS) { + taosTmrReset(streamSchedByTimer, nextTrigger, pTask, streamEnv.timer, &pTask->schedInfo.pTimer); return; } @@ -95,7 +105,7 @@ static void streamSchedByTimer(void* param, void* tmrId) { } } - taosTmrReset(streamSchedByTimer, (int32_t)pTask->info.triggerParam, pTask, streamEnv.timer, &pTask->schedInfo.pTimer); + taosTmrReset(streamSchedByTimer, nextTrigger, pTask, streamEnv.timer, &pTask->schedInfo.pTimer); } int32_t streamSetupScheduleTrigger(SStreamTask* pTask) { From 776897cdc0240757b4834a03f4f6bfab62db1bb6 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 9 Oct 2023 19:36:59 +0800 Subject: [PATCH 151/158] test: update the test cases. --- tests/script/tsim/stream/distributeInterval0.sim | 1 + .../tsim/stream/partitionbyColumnSession.sim | 2 ++ tests/script/tsim/stream/udTableAndTag1.sim | 10 ++++++++-- tests/script/tsim/stream/udTableAndTag2.sim | 15 ++++++++++----- tests/system-test/0-others/information_schema.py | 2 +- .../system-test/8-stream/at_once_interval_ext.py | 2 ++ 6 files changed, 24 insertions(+), 8 deletions(-) diff --git a/tests/script/tsim/stream/distributeInterval0.sim b/tests/script/tsim/stream/distributeInterval0.sim index 5bb03c8cbf..a4e7941c28 100644 --- a/tests/script/tsim/stream/distributeInterval0.sim +++ b/tests/script/tsim/stream/distributeInterval0.sim @@ -439,6 +439,7 @@ sql create table ts1 using st tags(1,1,1); sql create table ts2 using st tags(2,2,2); sql create stream stream_t2 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 watermark 20s into streamtST1 as select _wstart, count(*) c1, count(a) c2 , sum(a) c3 , max(b) c5, min(c) c6 from st interval(10s) ; +sleep 2000 sql insert into ts1 values(1648791211000,1,2,3); sql insert into ts1 values(1648791222001,2,2,3); sql insert into ts2 values(1648791211000,1,2,3); diff --git a/tests/script/tsim/stream/partitionbyColumnSession.sim b/tests/script/tsim/stream/partitionbyColumnSession.sim index 035fe1d232..a22e36e499 100644 --- a/tests/script/tsim/stream/partitionbyColumnSession.sim +++ b/tests/script/tsim/stream/partitionbyColumnSession.sim @@ -284,6 +284,7 @@ sql create table t1 using st tags(1,1,1); sql create table t2 using st tags(2,2,2); sql create stream streams2 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into test.streamt2 as select _wstart c1, count(*) c2, max(a) c3 from st partition by a session(ts, 5s); +sleep 1000 sql insert into t1 values(1648791213000,NULL,NULL,NULL,NULL); sql insert into t1 values(1648791213000,NULL,NULL,NULL,NULL); sql insert into t2 values(1648791213000,NULL,NULL,NULL,NULL); @@ -480,6 +481,7 @@ sql create table t3 using st tags(2,2,2); sql create table t4 using st tags(2,2,2); sql create stream streams4 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into test.streamt4 as select _wstart c1, count(*) c2, max(a) c3 from st partition by a session(ts, 5s); +sleep 2000 sql insert into t1 values(1648791213000,2,2,3,1.0); sql insert into t2 values(1648791213000,2,2,3,1.0); sql insert into t3 values(1648791213000,2,2,3,1.0); diff --git a/tests/script/tsim/stream/udTableAndTag1.sim b/tests/script/tsim/stream/udTableAndTag1.sim index 091615d0f3..e9dfbaabcf 100644 --- a/tests/script/tsim/stream/udTableAndTag1.sim +++ b/tests/script/tsim/stream/udTableAndTag1.sim @@ -22,6 +22,8 @@ sql create table t2 using st tags(2,2,2); #sql_error create stream streams1 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into result.streamt SUBTABLE("aaa") as select _wstart, count(*) c1 from st interval(10s); sql create stream streams1 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into result.streamt SUBTABLE( concat("aaa-", cast(a as varchar(10) ) ) ) as select _wstart, count(*) c1 from st partition by a interval(10s); +sleep 2000 + print ===== insert into 1 sql insert into t1 values(1648791213000,1,2,3); sql insert into t2 values(1648791213000,2,2,3); @@ -88,11 +90,12 @@ sql create table t1 using st tags(1,1,1); sql create table t2 using st tags(2,2,2); sql create stream streams2 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into result2.streamt2 TAGS(cc varchar(100)) as select _wstart, count(*) c1 from st partition by concat("col-", cast(a as varchar(10) ) ) as cc interval(10s); +sleep 2000 + print ===== insert into 2 sql insert into t1 values(1648791213000,1,2,3); sql insert into t2 values(1648791213000,2,2,3); - $loop_count = 0 loop2: @@ -172,6 +175,8 @@ sql create table t1 using st tags(1,1,1); sql create table t2 using st tags(2,2,2); sql create stream streams3 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into result3.streamt3 TAGS(dd varchar(100)) SUBTABLE(concat("tbn-", cast(a as varchar(10) ) ) ) as select _wstart, count(*) c1 from st partition by concat("col-", cast(a as varchar(10) ) ) as dd, a interval(10s); +sleep 2000 + print ===== insert into 3 sql insert into t1 values(1648791213000,1,2,3); sql insert into t2 values(1648791213000,2,2,3); @@ -284,8 +289,9 @@ sql create table t2 using st tags(2,2,2); sql create table t3 using st tags(3,3,3); sql create stream streams4 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into result4.streamt4 TAGS(dd varchar(100)) SUBTABLE(concat("tbn-", dd)) as select _wstart, count(*) c1 from st partition by concat("t", cast(a as varchar(10) ) ) as dd interval(10s); -sql insert into t1 values(1648791213000,1,1,1) t2 values(1648791213000,2,2,2) t3 values(1648791213000,3,3,3); +sleep 2000 +sql insert into t1 values(1648791213000,1,1,1) t2 values(1648791213000,2,2,2) t3 values(1648791213000,3,3,3); $loop_count = 0 loop7: diff --git a/tests/script/tsim/stream/udTableAndTag2.sim b/tests/script/tsim/stream/udTableAndTag2.sim index 9ad985c681..973c55b9ef 100644 --- a/tests/script/tsim/stream/udTableAndTag2.sim +++ b/tests/script/tsim/stream/udTableAndTag2.sim @@ -21,6 +21,8 @@ sql create table t1 using st tags(1,1,1); sql create table t2 using st tags(2,2,2); sql create stream streams1 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into result.streamt SUBTABLE("aaa") as select _wstart, count(*) c1 from st interval(10s); +sleep 2000 + print ===== insert into 1 sql insert into t1 values(1648791213000,1,2,3); sql insert into t2 values(1648791213000,2,2,3); @@ -94,11 +96,12 @@ sql create table t1 using st tags(1,1,1); sql create table t2 using st tags(2,2,2); sql create stream streams2 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into result2.streamt2 TAGS(cc varchar(100)) as select _wstart, count(*) c1 from st interval(10s); +sleep 2000 + print ===== insert into 2 sql insert into t1 values(1648791213000,1,2,3); sql insert into t2 values(1648791213000,2,2,3); - $loop_count = 0 loop2: @@ -186,21 +189,20 @@ print ===== step4 print ===== column name + table name sql create database result3 vgroups 1; - sql create database test3 vgroups 4; sql use test3; - sql create stable st(ts timestamp,a int,b int,c int) tags(ta int,tb int,tc int); sql create table t1 using st tags(1,1,1); sql create table t2 using st tags(2,2,2); sql create stream streams3 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into result3.streamt3 TAGS(dd varchar(100)) SUBTABLE(concat("tbn-", "1") ) as select _wstart, count(*) c1 from st interval(10s); +sleep 2000 + print ===== insert into 3 sql insert into t1 values(1648791213000,1,2,3); sql insert into t2 values(1648791213000,2,2,3); - $loop_count = 0 loop4: @@ -306,8 +308,9 @@ sql create table t2 using st tags(2,2,2); sql create table t3 using st tags(3,3,3); sql create stream streams4 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into result4.streamt4 TAGS(dd varchar(100)) SUBTABLE(concat("tbn-", "1")) as select _wstart, count(*) c1 from st interval(10s); -sql insert into t1 values(1648791213000,1,1,1) t2 values(1648791213000,2,2,2) t3 values(1648791213000,3,3,3); +sleep 2000 +sql insert into t1 values(1648791213000,1,1,1) t2 values(1648791213000,2,2,2) t3 values(1648791213000,3,3,3); $loop_count = 0 loop7: @@ -379,6 +382,8 @@ sql create stream streams51 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 in sql create stream streams52 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into result5.streamt52 TAGS(cc varchar(100)) as select _wstart, count(*) c1 from st interval(10s); sql create stream streams53 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into result5.streamt53 TAGS(dd varchar(100)) SUBTABLE(concat("aaa-", "1") ) as select _wstart, count(*) c1 from st interval(10s); +sleep 2000 + sql insert into t1 values(1648791213000,1,2,3); sql insert into t2 values(1648791213000,2,2,3); diff --git a/tests/system-test/0-others/information_schema.py b/tests/system-test/0-others/information_schema.py index 99bd90b006..4bb4e1c05e 100644 --- a/tests/system-test/0-others/information_schema.py +++ b/tests/system-test/0-others/information_schema.py @@ -217,7 +217,7 @@ class TDTestCase: tdSql.checkEqual(20470,len(tdSql.queryResult)) tdSql.query("select * from information_schema.ins_columns where db_name ='information_schema'") - tdSql.checkEqual(194, len(tdSql.queryResult)) + tdSql.checkEqual(195, len(tdSql.queryResult)) tdSql.query("select * from information_schema.ins_columns where db_name ='performance_schema'") tdSql.checkEqual(54, len(tdSql.queryResult)) diff --git a/tests/system-test/8-stream/at_once_interval_ext.py b/tests/system-test/8-stream/at_once_interval_ext.py index 8674e7f853..e1dc057448 100644 --- a/tests/system-test/8-stream/at_once_interval_ext.py +++ b/tests/system-test/8-stream/at_once_interval_ext.py @@ -92,6 +92,8 @@ class TDTestCase: else: range_count = self.tdCom.range_count + time.sleep(1) + for i in range(range_count): latency = 0 tag_value_list = list() From 21457424a0b1e9dbff8a31484acaed6e1d003b97 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 9 Oct 2023 22:52:00 +0800 Subject: [PATCH 152/158] test: update the test cases. --- tests/script/tsim/stream/deleteSession.sim | 5 +++++ tests/script/tsim/stream/udTableAndTag0.sim | 14 ++++++++++---- tests/system-test/0-others/information_schema.py | 2 +- .../8-stream/window_close_session_ext.py | 3 +++ 4 files changed, 19 insertions(+), 5 deletions(-) diff --git a/tests/script/tsim/stream/deleteSession.sim b/tests/script/tsim/stream/deleteSession.sim index f2694e79c7..18ff56ee3b 100644 --- a/tests/script/tsim/stream/deleteSession.sim +++ b/tests/script/tsim/stream/deleteSession.sim @@ -18,6 +18,8 @@ sql use test; sql create table t1(ts timestamp, a int, b int , c int, d double); sql create stream streams0 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt as select _wstart c1, count(*) c2, max(a) c3 from t1 session(ts, 5s); +sleep 2000 + sql insert into t1 values(1648791213000,NULL,NULL,NULL,NULL); sleep 1000 sql delete from t1 where ts = 1648791213000; @@ -193,6 +195,7 @@ sql create table t1 using st tags(1,1,1); sql create table t2 using st tags(2,2,2); sql create stream streams2 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into test.streamt2 as select _wstart c1, count(*) c2, max(a) c3 from st session(ts,5s); +sleep 2000 sql insert into t1 values(1648791213000,NULL,NULL,NULL,NULL); sql insert into t2 values(1648791213000,NULL,NULL,NULL,NULL); @@ -423,6 +426,7 @@ sql create stable st(ts timestamp, a int, b int, c int, d double) tags(ta int,tb sql create table t1 using st tags(1,1,1); sql create table t2 using st tags(2,2,2); sql create stream streams3 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into test.streamt3 as select _wstart c1, count(*) c2, max(a) c3 from st session(ts,5s); +sleep 2000 sql insert into t1 values(1648791210000,1,1,1,NULL); sql insert into t1 values(1648791210001,2,2,2,NULL); @@ -534,6 +538,7 @@ sql create table t2 using st tags(2,2,2); print create stream streams4 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt4 as select _wstart, count(*) c1 from st partition by tbname session(ts, 2s); sql create stream streams4 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt4 as select _wstart, count(*) c1 from st partition by tbname session(ts, 2s); +sleep 1000 sql insert into t1 values(1648791210000,1,2,3); sql insert into t1 values(1648791220000,2,2,3); diff --git a/tests/script/tsim/stream/udTableAndTag0.sim b/tests/script/tsim/stream/udTableAndTag0.sim index 3fe17dbfe8..c81927abcb 100644 --- a/tests/script/tsim/stream/udTableAndTag0.sim +++ b/tests/script/tsim/stream/udTableAndTag0.sim @@ -22,6 +22,8 @@ sql create table t2 using st tags(2,2,2); #sql_error create stream streams1 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into result.streamt SUBTABLE("aaa") as select _wstart, count(*) c1 from st interval(10s); sql create stream streams1 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into result.streamt SUBTABLE(concat("aaa-", tbname)) as select _wstart, count(*) c1 from st partition by tbname interval(10s); +sleep 1000 + sql insert into t1 values(1648791213000,1,2,3); sql insert into t2 values(1648791213000,1,2,3); @@ -89,10 +91,11 @@ sql create table t1 using st tags(1,1,1); sql create table t2 using st tags(2,2,2); sql create stream streams2 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into result2.streamt2 TAGS(cc varchar(100)) as select _wstart, count(*) c1 from st partition by concat("tag-", tbname) as cc interval(10s); +sleep 1000 + sql insert into t1 values(1648791213000,1,2,3); sql insert into t2 values(1648791213000,1,2,3); - $loop_count = 0 loop2: @@ -174,10 +177,11 @@ sql create table t1 using st tags(1,1,1); sql create table t2 using st tags(2,2,2); sql create stream streams3 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into result3.streamt3 TAGS(dd varchar(100)) SUBTABLE(concat("tbn-", tbname)) as select _wstart, count(*) c1 from st partition by concat("tag-", tbname) as dd, tbname interval(10s); +sleep 1000 + sql insert into t1 values(1648791213000,1,2,3); sql insert into t2 values(1648791213000,1,2,3); - $loop_count = 0 loop4: @@ -286,8 +290,9 @@ sql create table t2 using st tags(2,2,2); sql create table t3 using st tags(3,3,3); sql create stream streams4 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into result4.streamt4 TAGS(dd varchar(100)) SUBTABLE(concat("tbn-", tbname)) as select _wstart, count(*) c1 from st partition by concat("tag-", tbname) as dd, tbname interval(10s); -sql insert into t1 values(1648791213000,1,1,1) t2 values(1648791213000,2,2,2) t3 values(1648791213000,3,3,3); +sleep 1000 +sql insert into t1 values(1648791213000,1,1,1) t2 values(1648791213000,2,2,2) t3 values(1648791213000,3,3,3); $loop_count = 0 loop7: @@ -405,8 +410,9 @@ sql create table t2 using st tags("2",2,2); sql create table t3 using st tags("3",3,3); sql create stream streams6 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into result6.streamt6 TAGS(dd int) as select _wstart, count(*) c1 from st partition by concat(ta, "0") as dd, tbname interval(10s); -sql insert into t1 values(1648791213000,1,1,1) t2 values(1648791213000,2,2,2) t3 values(1648791213000,3,3,3); +sleep 1000 +sql insert into t1 values(1648791213000,1,1,1) t2 values(1648791213000,2,2,2) t3 values(1648791213000,3,3,3); $loop_count = 0 loop9: diff --git a/tests/system-test/0-others/information_schema.py b/tests/system-test/0-others/information_schema.py index 4bb4e1c05e..0e3ea9f695 100644 --- a/tests/system-test/0-others/information_schema.py +++ b/tests/system-test/0-others/information_schema.py @@ -217,7 +217,7 @@ class TDTestCase: tdSql.checkEqual(20470,len(tdSql.queryResult)) tdSql.query("select * from information_schema.ins_columns where db_name ='information_schema'") - tdSql.checkEqual(195, len(tdSql.queryResult)) + tdSql.checkEqual(196, len(tdSql.queryResult)) tdSql.query("select * from information_schema.ins_columns where db_name ='performance_schema'") tdSql.checkEqual(54, len(tdSql.queryResult)) diff --git a/tests/system-test/8-stream/window_close_session_ext.py b/tests/system-test/8-stream/window_close_session_ext.py index 0fc041e965..62e00b064a 100644 --- a/tests/system-test/8-stream/window_close_session_ext.py +++ b/tests/system-test/8-stream/window_close_session_ext.py @@ -43,6 +43,9 @@ class TDTestCase: watermark_value = None # create stb/ctb/tb stream self.tdCom.create_stream(stream_name=f'{self.stb_name}{self.tdCom.stream_suffix}', des_table=self.tdCom.ext_stb_stream_des_table, source_sql=f'select _wstart AS wstart, {partitial_tb_source_str} from {self.stb_name} session(ts, {self.tdCom.dataDict["session"]}s)', trigger_mode="window_close", watermark=watermark_value, subtable_value=stb_subtable_value, fill_history_value=fill_history_value, stb_field_name_value=stb_field_name_value, tag_value=tag_value, use_exist_stb=use_exist_stb) + + time.sleep(1) + for i in range(self.tdCom.range_count): if i == 0: window_close_ts = self.tdCom.cal_watermark_window_close_session_endts(self.tdCom.date_time, self.tdCom.dataDict['watermark'], self.tdCom.dataDict['session']) From 05dfadd8f85fc865b982958f875a30354f9efe5c Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 10 Oct 2023 14:00:30 +0800 Subject: [PATCH 153/158] fix(stream): set initial version for fill-history task. disable reentry of scan-history task function. --- include/libs/stream/tstream.h | 5 +- source/dnode/vnode/src/tq/tq.c | 75 ++++++++++++------- source/libs/stream/src/streamMeta.c | 7 +- source/libs/stream/src/streamRecover.c | 16 ++-- .../system-test/8-stream/pause_resume_test.py | 6 ++ 5 files changed, 72 insertions(+), 37 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index fe669b7fac..629efa00b3 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -272,6 +272,7 @@ typedef struct SStreamStatus { bool appendTranstateBlock; // has append the transfer state data block already, todo: remove it int8_t pauseAllowed; // allowed task status to be set to be paused int32_t timerActive; // timer is active + int32_t inScanHistorySentinel; } SStreamStatus; typedef struct SDataRange { @@ -358,8 +359,6 @@ typedef struct STaskOutputInfo { STaskSinkSma smaSink; STaskSinkFetch fetchSink; }; - -// void* pTimer; // timer for launch sink tasks int8_t type; STokenBucket* pTokenBucket; } STaskOutputInfo; @@ -375,7 +374,7 @@ struct SStreamTask { SSTaskBasicInfo info; STaskOutputQueue outputq; STaskInputInfo inputInfo; - STaskSchedInfo schedInfo; // todo remove it + STaskSchedInfo schedInfo; STaskOutputInfo outputInfo; SDispatchMsgInfo msgInfo; SStreamStatus status; diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 285f4df7b4..5ec83d868f 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1037,14 +1037,15 @@ int32_t tqProcessTaskDeployReq(STQ* pTq, int64_t sversion, char* msg, int32_t ms return code; } +// this function should be executed by only one thread int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { SStreamScanHistoryReq* pReq = (SStreamScanHistoryReq*)pMsg->pCont; SStreamMeta* pMeta = pTq->pStreamMeta; + int32_t code = TSDB_CODE_SUCCESS; - int32_t code = TSDB_CODE_SUCCESS; SStreamTask* pTask = streamMetaAcquireTask(pMeta, pReq->streamId, pReq->taskId); if (pTask == NULL) { - tqError("vgId:%d failed to acquire stream task:0x%x during stream recover, task may have been destroyed", + tqError("vgId:%d failed to acquire stream task:0x%x during scan history data, task may have been destroyed", pMeta->vgId, pReq->taskId); return -1; } @@ -1052,16 +1053,38 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { // do recovery step1 const char* id = pTask->id.idStr; const char* pStatus = streamGetTaskStatusStr(pTask->status.taskStatus); - tqDebug("s-task:%s start scan-history stage(step 1), status:%s", id, pStatus); + + // avoid multi-thread exec + while(1) { + int32_t sentinel = atomic_val_compare_exchange_32(&pTask->status.inScanHistorySentinel, 0, 1); + if (sentinel != 0) { + tqDebug("s-task:%s already in scan-history func, wait for 100ms, and try again", id); + taosMsleep(100); + } else { + break; + } + } if (pTask->execInfo.step1Start == 0) { ASSERT(pTask->status.pauseAllowed == false); - pTask->execInfo.step1Start = taosGetTimestampMs(); + int64_t ts = taosGetTimestampMs(); + + pTask->execInfo.step1Start = ts; + tqDebug("s-task:%s start scan-history stage(step 1), status:%s, step1 startTs:%" PRId64, id, pStatus, ts); + + // NOTE: in case of stream task, scan-history data in wal is not allowed to pause if (pTask->info.fillHistory == 1) { streamTaskEnablePause(pTask); } } else { - tqDebug("s-task:%s resume from paused, start ts:%" PRId64, pTask->id.idStr, pTask->execInfo.step1Start); + if (pTask->execInfo.step2Start == 0) { + tqDebug("s-task:%s resume from paused, original step1 startTs:%" PRId64, id, pTask->execInfo.step1Start); + } else { + tqDebug("s-task:%s already in step2, no need to scan-history data, step2 starTs:%"PRId64, id, pTask->execInfo.step2Start); + atomic_store_32(&pTask->status.inScanHistorySentinel, 0); + streamMetaReleaseTask(pMeta, pTask); + return 0; + } } // we have to continue retrying to successfully execute the scan history task. @@ -1070,6 +1093,7 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { "s-task:%s failed to start scan-history in first stream time window since already started, unexpected " "sched-status:%d", id, pTask->status.schedStatus); + atomic_store_32(&pTask->status.inScanHistorySentinel, 0); streamMetaReleaseTask(pMeta, pTask); return 0; } @@ -1079,16 +1103,18 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { } streamScanHistoryData(pTask); + + double el = (taosGetTimestampMs() - pTask->execInfo.step1Start) / 1000.0; if (pTask->status.taskStatus == TASK_STATUS__PAUSE) { - double el = (taosGetTimestampMs() - pTask->execInfo.step1Start) / 1000.0; int8_t status = streamTaskSetSchedStatusInActive(pTask); tqDebug("s-task:%s is paused in the step1, elapsed time:%.2fs, sched-status:%d", pTask->id.idStr, el, status); + + atomic_store_32(&pTask->status.inScanHistorySentinel, 0); streamMetaReleaseTask(pMeta, pTask); return 0; } // the following procedure should be executed, no matter status is stop/pause or not - double el = (taosGetTimestampMs() - pTask->execInfo.step1Start) / 1000.0; tqDebug("s-task:%s scan-history stage(step 1) ended, elapsed time:%.2fs", id, el); if (pTask->info.fillHistory) { @@ -1106,6 +1132,8 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { tqDebug("s-task:%s fill-history task set status to be dropping", id); streamBuildAndSendDropTaskMsg(pTask->pMsgCb, pMeta->vgId, &pTask->id); + + atomic_store_32(&pTask->status.inScanHistorySentinel, 0); streamMetaReleaseTask(pMeta, pTask); return -1; } @@ -1122,14 +1150,13 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { } // now we can stop the stream task execution - int64_t latestVer = 0; + int64_t nextProcessedVer = 0; while (1) { taosThreadMutexLock(&pStreamTask->lock); int8_t status = pStreamTask->status.taskStatus; if (status == TASK_STATUS__DROPPING || status == TASK_STATUS__STOP) { - // return; - // do nothing + // return; do nothing } if (status == TASK_STATUS__HALT) { @@ -1160,9 +1187,11 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { pStreamTask->status.keepTaskStatus = status; pStreamTask->status.taskStatus = TASK_STATUS__HALT; - tqDebug("s-task:%s level:%d sched-status:%d is halt by fill-history task:%s", pStreamTask->id.idStr, - pStreamTask->info.taskLevel, pStreamTask->status.schedStatus, id); - latestVer = walReaderGetCurrentVer(pStreamTask->exec.pWalReader); + nextProcessedVer = walReaderGetCurrentVer(pStreamTask->exec.pWalReader); + + tqDebug("s-task:%s level:%d nextProcessedVer:%" PRId64 ", sched-status:%d is halt by fill-history task:%s", + pStreamTask->id.idStr, pStreamTask->info.taskLevel, nextProcessedVer, pStreamTask->status.schedStatus, + id); taosThreadMutexUnlock(&pStreamTask->lock); break; @@ -1170,10 +1199,10 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { // if it's an source task, extract the last version in wal. pRange = &pTask->dataRange.range; - done = streamHistoryTaskSetVerRangeStep2(pTask, latestVer); + done = streamHistoryTaskSetVerRangeStep2(pTask, nextProcessedVer); + pTask->execInfo.step2Start = taosGetTimestampMs(); if (done) { - pTask->execInfo.step2Start = taosGetTimestampMs(); qDebug("s-task:%s scan-history from WAL stage(step 2) ended, elapsed time:%.2fs", id, 0.0); streamTaskPutTranstateIntoInputQ(pTask); @@ -1192,27 +1221,26 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { pStreamTask->id.idStr); ASSERT(pTask->status.schedStatus == TASK_SCHED_STATUS__WAITING); - pTask->execInfo.step2Start = taosGetTimestampMs(); streamSetParamForStreamScannerStep2(pTask, pRange, pWindow); int64_t dstVer = pTask->dataRange.range.minVer; pTask->chkInfo.nextProcessVer = dstVer; + walReaderSetSkipToVersion(pTask->exec.pWalReader, dstVer); tqDebug("s-task:%s wal reader start scan WAL verRange:%" PRId64 "-%" PRId64 ", set sched-status:%d", id, dstVer, pTask->dataRange.range.maxVer, TASK_SCHED_STATUS__INACTIVE); /*int8_t status = */streamTaskSetSchedStatusInActive(pTask); - // set the fill-history task to be normal + // the fill-history task starts to process data in wal, let's set it status to be normal now if (pTask->info.fillHistory == 1 && !streamTaskShouldStop(&pTask->status)) { streamSetStatusNormal(pTask); } tqScanWalAsync(pTq, false); } - - streamMetaReleaseTask(pMeta, pTask); streamMetaReleaseTask(pMeta, pStreamTask); + } else { STimeWindow* pWindow = &pTask->dataRange.window; @@ -1233,14 +1261,11 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { } code = streamTaskScanHistoryDataComplete(pTask); - streamMetaReleaseTask(pMeta, pTask); - - // when all source task complete to scan history data in stream time window, they are allowed to handle stream data - // at the same time. - return code; } - return 0; + atomic_store_32(&pTask->status.inScanHistorySentinel, 0); + streamMetaReleaseTask(pMeta, pTask); + return code; } // only the agg tasks and the sink tasks will receive this message from upstream tasks diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index c4aefa40ea..e49bf74c7d 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -394,12 +394,17 @@ int32_t streamMetaRemoveTask(SStreamMeta* pMeta, STaskId* pTaskId) { int32_t streamMetaRegisterTask(SStreamMeta* pMeta, int64_t ver, SStreamTask* pTask, bool* pAdded) { *pAdded = false; - STaskId id = {.streamId = pTask->id.streamId, .taskId = pTask->id.taskId}; + STaskId id = streamTaskExtractKey(pTask); void* p = taosHashGet(pMeta->pTasksMap, &id, sizeof(id)); if (p != NULL) { return 0; } + if (pTask->info.fillHistory == 1) { + stDebug("s-task:0x%x initial nextProcessVer is set to 1 for fill-history task", pTask->id.taskId); + ver = 1; + } + if (pMeta->expandFunc(pMeta->ahandle, pTask, ver) < 0) { tFreeStreamTask(pTask); return -1; diff --git a/source/libs/stream/src/streamRecover.c b/source/libs/stream/src/streamRecover.c index 6aacb5d2bb..cd15595411 100644 --- a/source/libs/stream/src/streamRecover.c +++ b/source/libs/stream/src/streamRecover.c @@ -792,24 +792,24 @@ int32_t streamTaskFillHistoryFinished(SStreamTask* pTask) { return qStreamInfoResetTimewindowFilter(exec); } -bool streamHistoryTaskSetVerRangeStep2(SStreamTask* pTask, int64_t latestVer) { +bool streamHistoryTaskSetVerRangeStep2(SStreamTask* pTask, int64_t nextProcessVer) { SVersionRange* pRange = &pTask->dataRange.range; - ASSERT(latestVer >= pRange->maxVer); + ASSERT(nextProcessVer >= pRange->maxVer); - int64_t nextStartVer = pRange->maxVer + 1; - if (nextStartVer > latestVer - 1) { - // no input data yet. no need to execute the secondardy scan while stream task halt + int64_t walScanStartVer = pRange->maxVer + 1; + if (walScanStartVer > nextProcessVer - 1) { + // no input data yet. no need to execute the secondary scan while stream task halt streamTaskFillHistoryFinished(pTask); stDebug( "s-task:%s no need to perform secondary scan-history data(step 2), since no data ingest during step1 scan, " "related stream task currentVer:%" PRId64, - pTask->id.idStr, latestVer); + pTask->id.idStr, nextProcessVer); return true; } else { // 2. do secondary scan of the history data, the time window remain, and the version range is updated to // [pTask->dataRange.range.maxVer, ver1] - pRange->minVer = nextStartVer; - pRange->maxVer = latestVer - 1; + pRange->minVer = walScanStartVer; + pRange->maxVer = nextProcessVer - 1; return false; } } diff --git a/tests/system-test/8-stream/pause_resume_test.py b/tests/system-test/8-stream/pause_resume_test.py index 421f499a3d..484383f1ce 100644 --- a/tests/system-test/8-stream/pause_resume_test.py +++ b/tests/system-test/8-stream/pause_resume_test.py @@ -56,6 +56,9 @@ class TDTestCase: self.tdCom.create_stream(stream_name=f'{self.stb_name}{self.tdCom.stream_suffix}', des_table=self.stb_stream_des_table, source_sql=f'select _wstart AS wstart, {self.tdCom.stb_source_select_str} from {self.stb_name} {partition_elm} interval({self.tdCom.dataDict["interval"]}s)', trigger_mode="at_once", subtable_value=stb_subtable_value, fill_history_value=fill_history_value) self.tdCom.create_stream(stream_name=f'{self.ctb_name}{self.tdCom.stream_suffix}', des_table=self.tdCom.ctb_stream_des_table, source_sql=f'select _wstart AS wstart, {self.tdCom.stb_source_select_str} from {self.ctb_name} {partition_elm} interval({self.tdCom.dataDict["interval"]}s)', trigger_mode="at_once", subtable_value=ctb_subtable_value, fill_history_value=fill_history_value) self.tdCom.create_stream(stream_name=f'{self.tb_name}{self.tdCom.stream_suffix}', des_table=self.tdCom.tb_stream_des_table, source_sql=f'select _wstart AS wstart, {self.tdCom.tb_source_select_str} from {self.tb_name} {partition_elm} interval({self.tdCom.dataDict["interval"]}s)', trigger_mode="at_once", subtable_value=tb_subtable_value, fill_history_value=fill_history_value) + + time.sleep(1) + for i in range(range_count): ts_value = str(date_time+self.tdCom.dataDict["interval"])+f'+{i*10}s' ts_cast_delete_value = self.tdCom.time_cast(ts_value) @@ -75,6 +78,9 @@ class TDTestCase: partition_elm = f'partition by {partition}' else: partition_elm = "" + + time.sleep(1) + # if i == int(range_count/2): if i > 2 and i % 3 == 0: for stream_name in [f'{self.stb_name}{self.tdCom.stream_suffix}', f'{self.ctb_name}{self.tdCom.stream_suffix}', f'{self.tb_name}{self.tdCom.stream_suffix}']: From fb2172933a059c9f43281174bf2467ceda769359 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 16 Oct 2023 09:49:51 +0800 Subject: [PATCH 154/158] fix(tsdb): enable pin stt datablock. --- source/dnode/vnode/src/inc/tsdb.h | 37 ++++--- source/dnode/vnode/src/tsdb/tsdbMergeTree.c | 109 ++++++++++++++++---- source/dnode/vnode/src/tsdb/tsdbRead2.c | 11 ++ source/dnode/vnode/src/tsdb/tsdbReadUtil.h | 1 - 4 files changed, 120 insertions(+), 38 deletions(-) diff --git a/source/dnode/vnode/src/inc/tsdb.h b/source/dnode/vnode/src/inc/tsdb.h index 826d8bb6e8..d3331d1c17 100644 --- a/source/dnode/vnode/src/inc/tsdb.h +++ b/source/dnode/vnode/src/inc/tsdb.h @@ -722,20 +722,25 @@ typedef struct SSttBlockLoadCostInfo { double statisElapsedTime; } SSttBlockLoadCostInfo; +typedef struct SBlockDataInfo { + SBlockData data; + bool pin; + int32_t sttBlockIndex; +} SBlockDataInfo; + typedef struct SSttBlockLoadInfo { - SBlockData blockData[2]; // buffered block data - int32_t statisBlockIndex; // buffered statistics block index - void *statisBlock; // buffered statistics block data - void *pSttStatisBlkArray; - SArray *aSttBlk; - int32_t blockIndex[2]; // to denote the loaded block in the corresponding position. - int32_t currentLoadBlockIndex; - STSchema *pSchema; - int16_t *colIds; - int32_t numOfCols; - bool checkRemainingRow; // todo: no assign value? - bool isLast; - bool sttBlockLoaded; + SBlockDataInfo blockData[2]; // buffered block data + int32_t statisBlockIndex; // buffered statistics block index + void *statisBlock; // buffered statistics block data + void *pSttStatisBlkArray; + SArray *aSttBlk; + int32_t currentLoadBlockIndex; + STSchema *pSchema; + int16_t *colIds; + int32_t numOfCols; + bool checkRemainingRow; // todo: no assign value? + bool isLast; + bool sttBlockLoaded; SSttBlockLoadCostInfo cost; } SSttBlockLoadInfo; @@ -805,9 +810,6 @@ typedef struct SLDataIter { } SLDataIter; #define tMergeTreeGetRow(_t) (&((_t)->pIter->rInfo.row)) -int32_t tMergeTreeOpen(SMergeTree *pMTree, int8_t backward, SDataFReader *pFReader, uint64_t suid, uint64_t uid, - STimeWindow *pTimeWindow, SVersionRange *pVerRange, SSttBlockLoadInfo *pBlockLoadInfo, - bool destroyLoadInfo, const char *idStr, bool strictTimeRange, SLDataIter *pLDataIter); struct SSttFileReader; typedef int32_t (*_load_tomb_fn)(STsdbReader *pReader, struct SSttFileReader *pSttFileReader, @@ -830,10 +832,13 @@ typedef struct { void *pReader; void *idstr; } SMergeTreeConf; + int32_t tMergeTreeOpen2(SMergeTree *pMTree, SMergeTreeConf *pConf); void tMergeTreeAddIter(SMergeTree *pMTree, SLDataIter *pIter); bool tMergeTreeNext(SMergeTree *pMTree); +void tMergeTreePinSttBlock(SMergeTree* pMTree); +void tMergeTreeUnpinSttBlock(SMergeTree* pMTree); bool tMergeTreeIgnoreEarlierTs(SMergeTree *pMTree); void tMergeTreeClose(SMergeTree *pMTree); diff --git a/source/dnode/vnode/src/tsdb/tsdbMergeTree.c b/source/dnode/vnode/src/tsdb/tsdbMergeTree.c index 260f4d8b2d..8dbbb4a6f0 100644 --- a/source/dnode/vnode/src/tsdb/tsdbMergeTree.c +++ b/source/dnode/vnode/src/tsdb/tsdbMergeTree.c @@ -29,16 +29,17 @@ SSttBlockLoadInfo *tCreateOneLastBlockLoadInfo(STSchema *pSchema, int16_t *colLi return NULL; } - pLoadInfo->blockIndex[0] = -1; - pLoadInfo->blockIndex[1] = -1; + pLoadInfo->blockData[0].sttBlockIndex = -1; + pLoadInfo->blockData[1].sttBlockIndex = -1; + pLoadInfo->currentLoadBlockIndex = 1; - int32_t code = tBlockDataCreate(&pLoadInfo->blockData[0]); + int32_t code = tBlockDataCreate(&pLoadInfo->blockData[0].data); if (code) { terrno = code; } - code = tBlockDataCreate(&pLoadInfo->blockData[1]); + code = tBlockDataCreate(&pLoadInfo->blockData[1].data); if (code) { terrno = code; } @@ -66,11 +67,16 @@ void *destroyLastBlockLoadInfo(SSttBlockLoadInfo *pLoadInfo) { } pLoadInfo->currentLoadBlockIndex = 1; - pLoadInfo->blockIndex[0] = -1; - pLoadInfo->blockIndex[1] = -1; - tBlockDataDestroy(&pLoadInfo->blockData[0]); - tBlockDataDestroy(&pLoadInfo->blockData[1]); + SBlockDataInfo* pInfo = &pLoadInfo->blockData[0]; + tBlockDataDestroy(&pInfo->data); + pInfo->sttBlockIndex = -1; + pInfo->pin = false; + + pInfo = &pLoadInfo->blockData[1]; + tBlockDataDestroy(&pInfo->data); + pInfo->sttBlockIndex = -1; + pInfo->pin = false; taosArrayDestroy(pLoadInfo->aSttBlk); taosMemoryFree(pLoadInfo); @@ -109,37 +115,46 @@ void *destroySttBlockReader(SArray *pLDataIterArray, SSttBlockLoadCostInfo* pLoa return NULL; } +// choose the unpinned slot to load next data block +static void updateBlockLoadSlot(SSttBlockLoadInfo* pLoadInfo) { + int32_t nextSlotIndex = pLoadInfo->currentLoadBlockIndex ^ 1; + if (pLoadInfo->blockData[nextSlotIndex].pin) { + nextSlotIndex = nextSlotIndex ^ 1; + } + + pLoadInfo->currentLoadBlockIndex = nextSlotIndex; +} + static SBlockData *loadLastBlock(SLDataIter *pIter, const char *idStr) { int32_t code = 0; SSttBlockLoadInfo *pInfo = pIter->pBlockLoadInfo; - if (pInfo->blockIndex[0] == pIter->iSttBlk) { + if (pInfo->blockData[0].sttBlockIndex == pIter->iSttBlk) { if (pInfo->currentLoadBlockIndex != 0) { tsdbDebug("current load index is set to 0, block index:%d, file index:%d, due to uid:%" PRIu64 ", load data, %s", pIter->iSttBlk, pIter->iStt, pIter->uid, idStr); pInfo->currentLoadBlockIndex = 0; } - return &pInfo->blockData[0]; + return &pInfo->blockData[0].data; } - if (pInfo->blockIndex[1] == pIter->iSttBlk) { + if (pInfo->blockData[1].sttBlockIndex == pIter->iSttBlk) { if (pInfo->currentLoadBlockIndex != 1) { tsdbDebug("current load index is set to 1, block index:%d, file index:%d, due to uid:%" PRIu64 ", load data, %s", pIter->iSttBlk, pIter->iStt, pIter->uid, idStr); pInfo->currentLoadBlockIndex = 1; } - return &pInfo->blockData[1]; + return &pInfo->blockData[1].data; } if (pIter->pSttBlk == NULL || pInfo->pSchema == NULL) { return NULL; } - // current block not loaded yet - pInfo->currentLoadBlockIndex ^= 1; + updateBlockLoadSlot(pInfo); int64_t st = taosGetTimestampUs(); - SBlockData *pBlock = &pInfo->blockData[pInfo->currentLoadBlockIndex]; + SBlockData *pBlock = &pInfo->blockData[pInfo->currentLoadBlockIndex].data; code = tsdbSttFileReadBlockDataByColumn(pIter->pReader, pIter->pSttBlk, pBlock, pInfo->pSchema, &pInfo->colIds[1], pInfo->numOfCols - 1); if (code != TSDB_CODE_SUCCESS) { @@ -155,12 +170,12 @@ static SBlockData *loadLastBlock(SLDataIter *pIter, const char *idStr) { pInfo->cost.loadBlocks, pIter->uid, pIter->iStt, pIter->iSttBlk, pInfo->currentLoadBlockIndex, pBlock->nRow, pBlock, el, idStr); - pInfo->blockIndex[pInfo->currentLoadBlockIndex] = pIter->iSttBlk; - pIter->iRow = (pIter->backward) ? pInfo->blockData[pInfo->currentLoadBlockIndex].nRow : -1; + pInfo->blockData[pInfo->currentLoadBlockIndex].sttBlockIndex = pIter->iSttBlk; + pIter->iRow = (pIter->backward) ? pInfo->blockData[pInfo->currentLoadBlockIndex].data.nRow : -1; - tsdbDebug("last block index list:%d, %d, rowIndex:%d %s", pInfo->blockIndex[0], pInfo->blockIndex[1], pIter->iRow, - idStr); - return &pInfo->blockData[pInfo->currentLoadBlockIndex]; + tsdbDebug("last block index list:%d, %d, rowIndex:%d %s", pInfo->blockData[0].sttBlockIndex, + pInfo->blockData[1].sttBlockIndex, pIter->iRow, idStr); + return &pInfo->blockData[pInfo->currentLoadBlockIndex].data; _exit: if (code != TSDB_CODE_SUCCESS) { @@ -816,8 +831,60 @@ void tMergeTreeAddIter(SMergeTree *pMTree, SLDataIter *pIter) { tRBTreePut(&pMTr bool tMergeTreeIgnoreEarlierTs(SMergeTree *pMTree) { return pMTree->ignoreEarlierTs; } +static void tLDataIterPinSttBlock(SLDataIter* pIter, const char* id) { + SSttBlockLoadInfo* pInfo = pIter->pBlockLoadInfo; + + if (pInfo->blockData[0].sttBlockIndex == pIter->iSttBlk) { + pInfo->blockData[0].pin = true; + ASSERT(!pInfo->blockData[1].pin); + return; + } + + if (pInfo->blockData[1].sttBlockIndex == pIter->iSttBlk) { + pInfo->blockData[1].pin = true; + ASSERT(!pInfo->blockData[0].pin); + return; + } + + tsdbError("failed to pin any stt block, sttBlock:%d", pIter->iSttBlk, id); +} + +static void tLDataIterUnpinSttBlock(SLDataIter* pIter, const char* id) { + SSttBlockLoadInfo* pInfo = pIter->pBlockLoadInfo; + if (pInfo->blockData[0].pin) { + ASSERT(!pInfo->blockData[1].pin); + pInfo->blockData[0].pin = false; + return; + } + + if (pInfo->blockData[1].pin) { + ASSERT(!pInfo->blockData[0].pin); + pInfo->blockData[1].pin = false; + return; + } + + tsdbError("failed to unpin any stt block, sttBlock:%d", pIter->iSttBlk, id); +} + +void tMergeTreePinSttBlock(SMergeTree *pMTree) { + if (pMTree->pIter == NULL) { + return; + } + + SLDataIter* pIter = pMTree->pIter; + tLDataIterPinSttBlock(pIter, pMTree->idStr); +} + +void tMergeTreeUnpinSttBlock(SMergeTree *pMTree) { + if (pMTree->pIter == NULL) { + return; + } + + SLDataIter* pIter = pMTree->pIter; + tLDataIterUnpinSttBlock(pIter, pMTree->idStr); +} + bool tMergeTreeNext(SMergeTree *pMTree) { - int32_t code = TSDB_CODE_SUCCESS; if (pMTree->pIter) { SLDataIter *pIter = pMTree->pIter; diff --git a/source/dnode/vnode/src/tsdb/tsdbRead2.c b/source/dnode/vnode/src/tsdb/tsdbRead2.c index 868529e4dd..db1de7742c 100644 --- a/source/dnode/vnode/src/tsdb/tsdbRead2.c +++ b/source/dnode/vnode/src/tsdb/tsdbRead2.c @@ -1420,6 +1420,14 @@ static bool nextRowFromLastBlocks(SLastBlockReader* pLastBlockReader, STableBloc } } +static void doPinSttBlock(SLastBlockReader* pLastBlockReader) { + tMergeTreePinSttBlock(&pLastBlockReader->mergeTree); +} + +static void doUnpinSttBlock(SLastBlockReader* pLastBlockReader) { + tMergeTreeUnpinSttBlock(&pLastBlockReader->mergeTree); +} + static bool tryCopyDistinctRowFromSttBlock(TSDBROW* fRow, SLastBlockReader* pLastBlockReader, STableBlockScanInfo* pScanInfo, int64_t ts, STsdbReader* pReader, bool* copied) { @@ -1427,7 +1435,10 @@ static bool tryCopyDistinctRowFromSttBlock(TSDBROW* fRow, SLastBlockReader* pLas *copied = false; + // avoid the fetch next row replace the referenced stt block in buffer + doPinSttBlock(pLastBlockReader); bool hasVal = nextRowFromLastBlocks(pLastBlockReader, pScanInfo, &pReader->info.verRange); + doUnpinSttBlock(pLastBlockReader); if (hasVal) { int64_t next1 = getCurrentKeyInLastBlock(pLastBlockReader); if (next1 != ts) { diff --git a/source/dnode/vnode/src/tsdb/tsdbReadUtil.h b/source/dnode/vnode/src/tsdb/tsdbReadUtil.h index 7cda8c71e2..ea8efe37ee 100644 --- a/source/dnode/vnode/src/tsdb/tsdbReadUtil.h +++ b/source/dnode/vnode/src/tsdb/tsdbReadUtil.h @@ -146,7 +146,6 @@ typedef struct SLastBlockReader { int32_t order; uint64_t uid; SMergeTree mergeTree; - SSttBlockLoadInfo* pInfo; int64_t currentKey; } SLastBlockReader; From 5fe7c8de24617573016c8b0a8c497aa20f09fb50 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 16 Oct 2023 10:16:39 +0800 Subject: [PATCH 155/158] fix(stream): fix syntax error. --- source/dnode/vnode/src/tsdb/tsdbMergeTree.c | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/source/dnode/vnode/src/tsdb/tsdbMergeTree.c b/source/dnode/vnode/src/tsdb/tsdbMergeTree.c index 8dbbb4a6f0..4cf20a617d 100644 --- a/source/dnode/vnode/src/tsdb/tsdbMergeTree.c +++ b/source/dnode/vnode/src/tsdb/tsdbMergeTree.c @@ -846,7 +846,7 @@ static void tLDataIterPinSttBlock(SLDataIter* pIter, const char* id) { return; } - tsdbError("failed to pin any stt block, sttBlock:%d", pIter->iSttBlk, id); + tsdbError("failed to pin any stt block, sttBlock:%d, %s", pIter->iSttBlk, id); } static void tLDataIterUnpinSttBlock(SLDataIter* pIter, const char* id) { @@ -863,7 +863,7 @@ static void tLDataIterUnpinSttBlock(SLDataIter* pIter, const char* id) { return; } - tsdbError("failed to unpin any stt block, sttBlock:%d", pIter->iSttBlk, id); + tsdbError("failed to unpin any stt block, sttBlock:%d, %s", pIter->iSttBlk, id); } void tMergeTreePinSttBlock(SMergeTree *pMTree) { From 6b86d050f211ce8dac31d596096f27d844030ac7 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 16 Oct 2023 11:48:57 +0800 Subject: [PATCH 156/158] fix(tsdb): record the pinned block iter. --- source/dnode/vnode/src/inc/tsdb.h | 13 +++++------ source/dnode/vnode/src/tsdb/tsdbMergeTree.c | 25 ++++++++++++--------- 2 files changed, 21 insertions(+), 17 deletions(-) diff --git a/source/dnode/vnode/src/inc/tsdb.h b/source/dnode/vnode/src/inc/tsdb.h index d3331d1c17..f332d46fbd 100644 --- a/source/dnode/vnode/src/inc/tsdb.h +++ b/source/dnode/vnode/src/inc/tsdb.h @@ -746,13 +746,12 @@ typedef struct SSttBlockLoadInfo { } SSttBlockLoadInfo; typedef struct SMergeTree { - int8_t backward; - SRBTree rbt; - SLDataIter *pIter; - bool destroyLoadInfo; - SSttBlockLoadInfo *pLoadInfo; - const char *idStr; - bool ignoreEarlierTs; + int8_t backward; + SRBTree rbt; + SLDataIter *pIter; + SLDataIter *pPinnedBlockIter; + const char *idStr; + bool ignoreEarlierTs; } SMergeTree; typedef struct { diff --git a/source/dnode/vnode/src/tsdb/tsdbMergeTree.c b/source/dnode/vnode/src/tsdb/tsdbMergeTree.c index 4cf20a617d..755c45ed35 100644 --- a/source/dnode/vnode/src/tsdb/tsdbMergeTree.c +++ b/source/dnode/vnode/src/tsdb/tsdbMergeTree.c @@ -837,16 +837,18 @@ static void tLDataIterPinSttBlock(SLDataIter* pIter, const char* id) { if (pInfo->blockData[0].sttBlockIndex == pIter->iSttBlk) { pInfo->blockData[0].pin = true; ASSERT(!pInfo->blockData[1].pin); + tsdbDebug("pin stt-block, blockIndex:%d, stt-fileVer:%" PRId64 " %s", pIter->iSttBlk, pIter->cid, id); return; } if (pInfo->blockData[1].sttBlockIndex == pIter->iSttBlk) { pInfo->blockData[1].pin = true; ASSERT(!pInfo->blockData[0].pin); + tsdbDebug("pin stt-block, blockIndex:%d, stt-fileVer:%"PRId64" %s", pIter->iSttBlk, pIter->cid, id); return; } - tsdbError("failed to pin any stt block, sttBlock:%d, %s", pIter->iSttBlk, id); + tsdbError("failed to pin any stt block, sttBlock:%d stt-fileVer:%"PRId64" %s", pIter->iSttBlk, pIter->cid, id); } static void tLDataIterUnpinSttBlock(SLDataIter* pIter, const char* id) { @@ -854,33 +856,39 @@ static void tLDataIterUnpinSttBlock(SLDataIter* pIter, const char* id) { if (pInfo->blockData[0].pin) { ASSERT(!pInfo->blockData[1].pin); pInfo->blockData[0].pin = false; + tsdbDebug("unpin stt-block, blockIndex:%d, stt-fileVer:%"PRId64" %s", pInfo->blockData[1].sttBlockIndex, + pIter->cid, id); return; } if (pInfo->blockData[1].pin) { ASSERT(!pInfo->blockData[0].pin); pInfo->blockData[1].pin = false; + tsdbDebug("pin stt-block, blockIndex:%d, stt-fileVer:%" PRId64 " %s", pInfo->blockData[1].sttBlockIndex, pIter->cid, + id); return; } - tsdbError("failed to unpin any stt block, sttBlock:%d, %s", pIter->iSttBlk, id); + tsdbError("failed to unpin any stt block, sttBlock:%d stt-fileVer:%" PRId64 " %s", pIter->iSttBlk, pIter->cid, id); } void tMergeTreePinSttBlock(SMergeTree *pMTree) { if (pMTree->pIter == NULL) { - return; + return; } - SLDataIter* pIter = pMTree->pIter; + SLDataIter *pIter = pMTree->pIter; + pMTree->pPinnedBlockIter = pIter; tLDataIterPinSttBlock(pIter, pMTree->idStr); } void tMergeTreeUnpinSttBlock(SMergeTree *pMTree) { - if (pMTree->pIter == NULL) { + if (pMTree->pPinnedBlockIter == NULL) { return; } - SLDataIter* pIter = pMTree->pIter; + SLDataIter* pIter = pMTree->pPinnedBlockIter; + pMTree->pPinnedBlockIter = NULL; tLDataIterUnpinSttBlock(pIter, pMTree->idStr); } @@ -918,8 +926,5 @@ bool tMergeTreeNext(SMergeTree *pMTree) { void tMergeTreeClose(SMergeTree *pMTree) { pMTree->pIter = NULL; - if (pMTree->destroyLoadInfo) { - pMTree->pLoadInfo = destroyLastBlockLoadInfo(pMTree->pLoadInfo); - pMTree->destroyLoadInfo = false; - } + pMTree->pPinnedBlockIter = NULL; } From fa6ba6faf8799473c7fe5ca17d956a81c1c5fd8a Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 16 Oct 2023 11:55:57 +0800 Subject: [PATCH 157/158] fix(tdsb): adjust the log level. --- source/dnode/vnode/src/tsdb/tsdbMergeTree.c | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/source/dnode/vnode/src/tsdb/tsdbMergeTree.c b/source/dnode/vnode/src/tsdb/tsdbMergeTree.c index 755c45ed35..c53caeff41 100644 --- a/source/dnode/vnode/src/tsdb/tsdbMergeTree.c +++ b/source/dnode/vnode/src/tsdb/tsdbMergeTree.c @@ -856,16 +856,14 @@ static void tLDataIterUnpinSttBlock(SLDataIter* pIter, const char* id) { if (pInfo->blockData[0].pin) { ASSERT(!pInfo->blockData[1].pin); pInfo->blockData[0].pin = false; - tsdbDebug("unpin stt-block, blockIndex:%d, stt-fileVer:%"PRId64" %s", pInfo->blockData[1].sttBlockIndex, - pIter->cid, id); + tsdbTrace("unpin stt-block:%d, stt-fileVer:%" PRId64 " %s", pInfo->blockData[0].sttBlockIndex, pIter->cid, id); return; } if (pInfo->blockData[1].pin) { ASSERT(!pInfo->blockData[0].pin); pInfo->blockData[1].pin = false; - tsdbDebug("pin stt-block, blockIndex:%d, stt-fileVer:%" PRId64 " %s", pInfo->blockData[1].sttBlockIndex, pIter->cid, - id); + tsdbTrace("unpin stt-block:%d, stt-fileVer:%" PRId64 " %s", pInfo->blockData[1].sttBlockIndex, pIter->cid, id); return; } From 33018a0bdb7f7c93549ce56855540870060fe9a7 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 16 Oct 2023 14:07:09 +0800 Subject: [PATCH 158/158] fix(tsdb): fix syntax error. --- source/dnode/vnode/src/inc/tsdb.h | 2 +- source/dnode/vnode/src/tsdb/tsdbMergeTree.c | 45 ++++++++++++--------- 2 files changed, 28 insertions(+), 19 deletions(-) diff --git a/source/dnode/vnode/src/inc/tsdb.h b/source/dnode/vnode/src/inc/tsdb.h index f332d46fbd..e83f47f7b6 100644 --- a/source/dnode/vnode/src/inc/tsdb.h +++ b/source/dnode/vnode/src/inc/tsdb.h @@ -795,7 +795,7 @@ struct SDiskDataBuilder { typedef struct SLDataIter { SRBTreeNode node; SSttBlk *pSttBlk; - int32_t iStt; // for debug purpose + int64_t cid; // for debug purpose int8_t backward; int32_t iSttBlk; int32_t iRow; diff --git a/source/dnode/vnode/src/tsdb/tsdbMergeTree.c b/source/dnode/vnode/src/tsdb/tsdbMergeTree.c index c53caeff41..0e15107606 100644 --- a/source/dnode/vnode/src/tsdb/tsdbMergeTree.c +++ b/source/dnode/vnode/src/tsdb/tsdbMergeTree.c @@ -131,8 +131,9 @@ static SBlockData *loadLastBlock(SLDataIter *pIter, const char *idStr) { SSttBlockLoadInfo *pInfo = pIter->pBlockLoadInfo; if (pInfo->blockData[0].sttBlockIndex == pIter->iSttBlk) { if (pInfo->currentLoadBlockIndex != 0) { - tsdbDebug("current load index is set to 0, block index:%d, file index:%d, due to uid:%" PRIu64 ", load data, %s", - pIter->iSttBlk, pIter->iStt, pIter->uid, idStr); + tsdbDebug("current load index is set to 0, block index:%d, fileVer:%" PRId64 ", due to uid:%" PRIu64 + ", load data, %s", + pIter->iSttBlk, pIter->cid, pIter->uid, idStr); pInfo->currentLoadBlockIndex = 0; } return &pInfo->blockData[0].data; @@ -140,8 +141,9 @@ static SBlockData *loadLastBlock(SLDataIter *pIter, const char *idStr) { if (pInfo->blockData[1].sttBlockIndex == pIter->iSttBlk) { if (pInfo->currentLoadBlockIndex != 1) { - tsdbDebug("current load index is set to 1, block index:%d, file index:%d, due to uid:%" PRIu64 ", load data, %s", - pIter->iSttBlk, pIter->iStt, pIter->uid, idStr); + tsdbDebug("current load index is set to 1, block index:%d, fileVer:%" PRId64 ", due to uid:%" PRIu64 + ", load data, %s", + pIter->iSttBlk, pIter->cid, pIter->uid, idStr); pInfo->currentLoadBlockIndex = 1; } return &pInfo->blockData[1].data; @@ -165,10 +167,12 @@ static SBlockData *loadLastBlock(SLDataIter *pIter, const char *idStr) { pInfo->cost.blockElapsedTime += el; pInfo->cost.loadBlocks += 1; - tsdbDebug("read last block, total load:%"PRId64", trigger by uid:%" PRIu64 - ", last file index:%d, last block index:%d, entry:%d, rows:%d, %p, elapsed time:%.2f ms, %s", - pInfo->cost.loadBlocks, pIter->uid, pIter->iStt, pIter->iSttBlk, pInfo->currentLoadBlockIndex, pBlock->nRow, - pBlock, el, idStr); + tsdbDebug("read last block, total load:%" PRId64 ", trigger by uid:%" PRIu64 ", stt-fileVer:%" PRId64 + ", last block index:%d, entry:%d, rows:%d, uidRange:%" PRId64 "-%" PRId64 " tsRange:%" PRId64 "-%" PRId64 + " %p, elapsed time:%.2f ms, %s", + pInfo->cost.loadBlocks, pIter->uid, pIter->cid, pIter->iSttBlk, pInfo->currentLoadBlockIndex, pBlock->nRow, + pIter->pSttBlk->minUid, pIter->pSttBlk->maxUid, pIter->pSttBlk->minKey, pIter->pSttBlk->maxKey, pBlock, el, + idStr); pInfo->blockData[pInfo->currentLoadBlockIndex].sttBlockIndex = pIter->iSttBlk; pIter->iRow = (pIter->backward) ? pInfo->blockData[pInfo->currentLoadBlockIndex].data.nRow : -1; @@ -177,7 +181,7 @@ static SBlockData *loadLastBlock(SLDataIter *pIter, const char *idStr) { pInfo->blockData[1].sttBlockIndex, pIter->iRow, idStr); return &pInfo->blockData[pInfo->currentLoadBlockIndex].data; -_exit: + _exit: if (code != TSDB_CODE_SUCCESS) { terrno = code; } @@ -439,14 +443,14 @@ static int32_t doLoadSttFilesBlk(SSttBlockLoadInfo *pBlockLoadInfo, SLDataIter * return code; } -int32_t tLDataIterOpen2(SLDataIter *pIter, SSttFileReader *pSttFileReader, int32_t iStt, int8_t backward, +int32_t tLDataIterOpen2(SLDataIter *pIter, SSttFileReader *pSttFileReader, int32_t cid, int8_t backward, uint64_t suid, uint64_t uid, STimeWindow *pTimeWindow, SVersionRange *pRange, SSttBlockLoadInfo *pBlockLoadInfo, const char *idStr, bool strictTimeRange, _load_tomb_fn loadTombFn, void *pReader1) { int32_t code = TSDB_CODE_SUCCESS; pIter->uid = uid; - pIter->iStt = iStt; + pIter->cid = cid; pIter->backward = backward; pIter->verRange.minVer = pRange->minVer; pIter->verRange.maxVer = pRange->maxVer; @@ -553,12 +557,15 @@ void tLDataIterNextBlock(SLDataIter *pIter, const char *idStr) { pIter->pSttBlk = NULL; if (index != -1) { + SSttBlk *p = taosArrayGet(pIter->pBlockLoadInfo->aSttBlk, index); + pIter->iSttBlk = index; pIter->pSttBlk = (SSttBlk *)taosArrayGet(pIter->pBlockLoadInfo->aSttBlk, pIter->iSttBlk); - tsdbDebug("try next last file block:%d from stt fileIdx:%d, trigger by uid:%" PRIu64 ", file index:%d, %s", - pIter->iSttBlk, oldIndex, pIter->uid, pIter->iStt, idStr); + tsdbDebug("try next stt-file block:%d from %d, trigger by uid:%" PRIu64 ", stt-fileVer:%" PRId64 + ", uidRange:%" PRId64 "-%" PRId64 " %s", + pIter->iSttBlk, oldIndex, pIter->uid, pIter->cid, p->minUid, p->maxUid, idStr); } else { - tsdbDebug("no more last block qualified, uid:%" PRIu64 ", file index:%d, %s", pIter->uid, oldIndex, idStr); + tsdbDebug("no more last block qualified, uid:%" PRIu64 ", stt-file block:%d, %s", pIter->uid, oldIndex, idStr); } } @@ -692,7 +699,7 @@ bool tLDataIterNextRow(SLDataIter *pIter, const char *idStr) { pIter->rInfo.uid = pBlockData->uid; pIter->rInfo.row = tsdbRowFromBlockData(pBlockData, pIter->iRow); -_exit: + _exit: return (terrno == TSDB_CODE_SUCCESS) && (pIter->pSttBlk != NULL) && (pBlockData != NULL); } @@ -802,13 +809,15 @@ int32_t tMergeTreeOpen2(SMergeTree *pMTree, SMergeTreeConf *pConf) { } memset(pIter, 0, sizeof(SLDataIter)); - code = tLDataIterOpen2(pIter, pSttFileReader, i, pMTree->backward, pConf->suid, pConf->uid, &pConf->timewindow, + + int64_t cid = pSttLevel->fobjArr->data[i]->f->cid; + code = tLDataIterOpen2(pIter, pSttFileReader, cid, pMTree->backward, pConf->suid, pConf->uid, &pConf->timewindow, &pConf->verRange, pLoadInfo, pMTree->idStr, pConf->strictTimeRange, pConf->loadTombFn, pConf->pReader); if (code != TSDB_CODE_SUCCESS) { goto _end; } - + bool hasVal = tLDataIterNextRow(pIter, pMTree->idStr); if (hasVal) { tMergeTreeAddIter(pMTree, pIter); @@ -822,7 +831,7 @@ int32_t tMergeTreeOpen2(SMergeTree *pMTree, SMergeTreeConf *pConf) { return code; -_end: + _end: tMergeTreeClose(pMTree); return code; }