From 284ef7d0857d9420c447cffc9724ece4cf293fb6 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 3 Jan 2024 14:44:04 +0800 Subject: [PATCH 001/102] fix(stream): remove the related checkpoint trans when trying to drop stream. --- source/dnode/mnode/impl/inc/mndStream.h | 3 +- source/dnode/mnode/impl/src/mndStream.c | 48 ++++++++++++-------- source/dnode/mnode/impl/src/mndStreamTrans.c | 26 ++++++++++- 3 files changed, 57 insertions(+), 20 deletions(-) diff --git a/source/dnode/mnode/impl/inc/mndStream.h b/source/dnode/mnode/impl/inc/mndStream.h index 4d06b16297..7b71ad873b 100644 --- a/source/dnode/mnode/impl/inc/mndStream.h +++ b/source/dnode/mnode/impl/inc/mndStream.h @@ -71,7 +71,8 @@ int32_t mndPersistStream(SMnode *pMnode, STrans *pTrans, SStreamObj *pStream int32_t mndStreamRegisterTrans(STrans* pTrans, const char* pTransName, int64_t streamUid); int32_t mndAddtoCheckpointWaitingList(SStreamObj *pStream, int64_t checkpointId); -bool streamTransConflictOtherTrans(SMnode *pMnode, int64_t streamUid, const char *pTransName, bool lock); +bool mndStreamTransConflictCheck(SMnode *pMnode, int64_t streamUid, const char *pTransName, bool lock); +int32_t mndStreamGetRelCheckpointTrans(SMnode *pMnode, int64_t streamUid); // for sma // TODO refactor diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index a656137770..3c700665d3 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -80,7 +80,9 @@ static int32_t createStreamUpdateTrans(SStreamObj *pStream, SVgroupChangeInfo *p static void removeStreamTasksInBuf(SStreamObj *pStream, SStreamExecInfo *pExecNode); static void saveStreamTasksInfo(SStreamObj *pStream, SStreamExecInfo *pExecNode); static int32_t removeExpirednodeEntryAndTask(SArray *pNodeSnapshot); -static int32_t killActiveCheckpointTrans(SMnode *pMnode, const char *pDbName, size_t len); +static int32_t doKillCheckpointTrans(SMnode *pMnode, const char *pDbName, size_t len); +static void killCheckpointTransImpl(SMnode *pMnode, int32_t transId, const char *pDbName); + static int32_t setNodeEpsetExpiredFlag(const SArray *pNodeList); static void freeCheckpointCandEntry(void *); @@ -1047,7 +1049,7 @@ static int32_t mndProcessStreamCheckpointTrans(SMnode *pMnode, SStreamObj *pStre return -1; } - bool conflict = streamTransConflictOtherTrans(pMnode, pStream->uid, MND_STREAM_CHECKPOINT_NAME, true); + bool conflict = mndStreamTransConflictCheck(pMnode, pStream->uid, MND_STREAM_CHECKPOINT_NAME, true); if (conflict) { mndAddtoCheckpointWaitingList(pStream, checkpointId); mWarn("checkpoint conflict with other trans in %s, ignore the checkpoint for stream:%s %" PRIx64, pStream->sourceDb, @@ -1344,6 +1346,7 @@ static int32_t mndProcessStreamCheckpointInCandid(SRpcMsg *pReq) { if (ps == NULL) { continue; } + mDebug("start to launch checkpoint for stream:%s %" PRIx64 " in candidate list", pEntry->pName, pEntry->streamId); code = mndProcessStreamCheckpointTrans(pMnode, ps, pEntry->checkpointId); @@ -1399,7 +1402,7 @@ static int32_t mndProcessDropStreamReq(SRpcMsg *pReq) { } // check if it is conflict with other trans in both sourceDb and targetDb. - bool conflict = streamTransConflictOtherTrans(pMnode, pStream->uid, MND_STREAM_DROP_NAME, true); + bool conflict = mndStreamTransConflictCheck(pMnode, pStream->uid, MND_STREAM_DROP_NAME, true); if (conflict) { sdbRelease(pMnode->pSdb, pStream); tFreeMDropStreamReq(&dropReq); @@ -1451,6 +1454,12 @@ static int32_t mndProcessDropStreamReq(SRpcMsg *pReq) { return -1; } + // kill the related checkpoint trans + int32_t transId = mndStreamGetRelCheckpointTrans(pMnode, pStream->uid); + if (transId != 0) { + killCheckpointTransImpl(pMnode, transId, pStream->sourceDb); + } + removeStreamTasksInBuf(pStream, &execInfo); SName name = {0}; @@ -1894,7 +1903,7 @@ static int32_t mndProcessPauseStreamReq(SRpcMsg *pReq) { } // check if it is conflict with other trans in both sourceDb and targetDb. - bool conflict = streamTransConflictOtherTrans(pMnode, pStream->uid, MND_STREAM_PAUSE_NAME, true); + bool conflict = mndStreamTransConflictCheck(pMnode, pStream->uid, MND_STREAM_PAUSE_NAME, true); if (conflict) { sdbRelease(pMnode->pSdb, pStream); return -1; @@ -2037,7 +2046,7 @@ static int32_t mndProcessResumeStreamReq(SRpcMsg *pReq) { } // check if it is conflict with other trans in both sourceDb and targetDb. - bool conflict = streamTransConflictOtherTrans(pMnode, pStream->uid, MND_STREAM_RESUME_NAME, true); + bool conflict = mndStreamTransConflictCheck(pMnode, pStream->uid, MND_STREAM_RESUME_NAME, true); if (conflict) { sdbRelease(pMnode->pSdb, pStream); return -1; @@ -2343,7 +2352,7 @@ static int32_t mndProcessVgroupChange(SMnode *pMnode, SVgroupChangeInfo *pChange break; } - bool conflict = streamTransConflictOtherTrans(pMnode, pStream->uid, MND_STREAM_TASK_UPDATE_NAME, false); + bool conflict = mndStreamTransConflictCheck(pMnode, pStream->uid, MND_STREAM_TASK_UPDATE_NAME, false); sdbRelease(pSdb, pStream); if (conflict) { @@ -2567,7 +2576,7 @@ static void killAllCheckpointTrans(SMnode *pMnode, SVgroupChangeInfo *pChangeInf size_t len = 0; void * pKey = taosHashGetKey(pDb, &len); - killActiveCheckpointTrans(pMnode, pKey, len); + doKillCheckpointTrans(pMnode, pKey, len); } } @@ -2812,7 +2821,16 @@ int32_t createStreamResetStatusTrans(SMnode *pMnode, SStreamObj *pStream) { return TSDB_CODE_ACTION_IN_PROGRESS; } -int32_t killActiveCheckpointTrans(SMnode *pMnode, const char *pDBName, size_t len) { +void killCheckpointTransImpl(SMnode* pMnode, int32_t transId, const char* pDbName) { + STrans *pTrans = mndAcquireTrans(pMnode, transId); + if (pTrans != NULL) { + mInfo("kill checkpoint transId:%d in Db:%s", transId, pDbName); + mndKillTrans(pMnode, pTrans); + mndReleaseTrans(pMnode, pTrans); + } +} + +int32_t doKillCheckpointTrans(SMnode *pMnode, const char *pDBName, size_t len) { // data in the hash table will be removed automatically, no need to remove it here. SStreamTransInfo *pTransInfo = taosHashGet(execInfo.transMgmt.pDBTrans, pDBName, len); if (pTransInfo == NULL) { @@ -2825,15 +2843,9 @@ int32_t killActiveCheckpointTrans(SMnode *pMnode, const char *pDBName, size_t le return TSDB_CODE_SUCCESS; } - STrans *pTrans = mndAcquireTrans(pMnode, pTransInfo->transId); - if (pTrans != NULL) { - char* pDupDBName = strndup(pDBName, len); - mInfo("kill checkpoint transId:%d in Db:%s", pTransInfo->transId, pDupDBName); - taosMemoryFree(pDupDBName); - - mndKillTrans(pMnode, pTrans); - mndReleaseTrans(pMnode, pTrans); - } + char* pDupDBName = strndup(pDBName, len); + killCheckpointTransImpl(pMnode, pTransInfo->transId, pDupDBName); + taosMemoryFree(pDupDBName); return TSDB_CODE_SUCCESS; } @@ -2855,7 +2867,7 @@ static int32_t mndResetStatusFromCheckpoint(SMnode *pMnode, int64_t streamId, in code = TSDB_CODE_STREAM_TASK_NOT_EXIST; mError("failed to acquire the streamObj:0x%" PRIx64 " to reset checkpoint, may have been dropped", pStream->uid); } else { - bool conflict = streamTransConflictOtherTrans(pMnode, pStream->uid, MND_STREAM_TASK_RESET_NAME, false); + bool conflict = mndStreamTransConflictCheck(pMnode, pStream->uid, MND_STREAM_TASK_RESET_NAME, false); if (conflict) { mError("stream:%s other trans exists in DB:%s, dstTable:%s failed to start reset-status trans", pStream->name, pStream->sourceDb, pStream->targetSTbName); diff --git a/source/dnode/mnode/impl/src/mndStreamTrans.c b/source/dnode/mnode/impl/src/mndStreamTrans.c index 8f94843500..f5047acc49 100644 --- a/source/dnode/mnode/impl/src/mndStreamTrans.c +++ b/source/dnode/mnode/impl/src/mndStreamTrans.c @@ -65,7 +65,7 @@ int32_t clearFinishedTrans(SMnode* pMnode) { return 0; } -bool streamTransConflictOtherTrans(SMnode* pMnode, int64_t streamUid, const char* pTransName, bool lock) { +bool mndStreamTransConflictCheck(SMnode* pMnode, int64_t streamUid, const char* pTransName, bool lock) { if (lock) { taosThreadMutexLock(&execInfo.lock); } @@ -113,6 +113,30 @@ bool streamTransConflictOtherTrans(SMnode* pMnode, int64_t streamUid, const char return false; } +int32_t mndStreamGetRelCheckpointTrans(SMnode* pMnode, int64_t streamUid) { + taosThreadMutexLock(&execInfo.lock); + int32_t num = taosHashGetSize(execInfo.transMgmt.pDBTrans); + if (num <= 0) { + taosThreadMutexUnlock(&execInfo.lock); + return 0; + } + + clearFinishedTrans(pMnode); + SStreamTransInfo* pEntry = taosHashGet(execInfo.transMgmt.pDBTrans, &streamUid, sizeof(streamUid)); + if (pEntry != NULL) { + SStreamTransInfo tInfo = *pEntry; + taosThreadMutexUnlock(&execInfo.lock); + + if (strcmp(tInfo.name, MND_STREAM_CHECKPOINT_NAME) == 0) { + return tInfo.transId; + } + } else { + taosThreadMutexUnlock(&execInfo.lock); + } + + return 0; +} + int32_t mndAddtoCheckpointWaitingList(SStreamObj* pStream, int64_t checkpointId) { SCheckpointCandEntry* pEntry = taosHashGet(execInfo.transMgmt.pWaitingList, &pStream->uid, sizeof(pStream->uid)); if (pEntry == NULL) { From 185fa75376c6fc9de82c02d9b4da50fbd49be1c7 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 3 Jan 2024 17:59:42 +0800 Subject: [PATCH 002/102] enh(stream): add task into timer to be idle during exec when the corresponding output queue is full. --- include/libs/stream/tstream.h | 23 ++-- source/dnode/mnode/impl/src/mndStream.c | 4 +- source/dnode/vnode/src/tq/tq.c | 2 +- source/dnode/vnode/src/tq/tqStreamTask.c | 6 +- source/dnode/vnode/src/tqCommon/tqCommon.c | 24 +++-- source/libs/stream/src/stream.c | 12 +-- source/libs/stream/src/streamExec.c | 118 +++++++++++++++++---- source/libs/stream/src/streamMeta.c | 4 +- source/libs/stream/src/streamQueue.c | 11 +- source/libs/stream/src/streamTask.c | 6 +- 10 files changed, 158 insertions(+), 52 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index a6dac7f5ba..79fc2a87a7 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -50,10 +50,11 @@ extern "C" { (_t)->hTaskInfo.id.streamId = 0; \ } while (0) -#define STREAM_EXEC_EXTRACT_DATA_IN_WAL_ID (-1) -#define STREAM_EXEC_START_ALL_TASKS_ID (-2) -#define STREAM_EXEC_RESTART_ALL_TASKS_ID (-3) -#define STREAM_EXEC_STOP_ALL_TASKS_ID (-4) +#define STREAM_EXEC_T_EXTRACT_WAL_DATA (-1) +#define STREAM_EXEC_T_START_ALL_TASKS (-2) +#define STREAM_EXEC_T_RESTART_ALL_TASKS (-3) +#define STREAM_EXEC_T_STOP_ALL_TASKS (-4) +#define STREAM_EXEC_T_RESUME_TASK (-5) typedef struct SStreamTask SStreamTask; typedef struct SStreamQueue SStreamQueue; @@ -81,14 +82,12 @@ typedef enum ETaskStatus { TASK_STATUS__HALT, // pause, but not be manipulated by user command TASK_STATUS__PAUSE, // pause TASK_STATUS__CK, // stream task is in checkpoint status, no data are allowed to put into inputQ anymore -// TASK_STATUS__STREAM_SCAN_HISTORY, } ETaskStatus; enum { TASK_SCHED_STATUS__INACTIVE = 1, TASK_SCHED_STATUS__WAITING, TASK_SCHED_STATUS__ACTIVE, - TASK_SCHED_STATUS__FAILED, TASK_SCHED_STATUS__DROPPING, }; @@ -322,10 +321,11 @@ 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 + int32_t schedIdleTime; // idle time before invoke again + int64_t lastExecTs; // last exec time stamp + int8_t statusBackup; + bool appendTranstateBlock; // has append the transfer state data block already int32_t timerActive; // timer is active - int8_t allowedAddInTimer; // allowed to add into timer int32_t inScanHistorySentinel; } SStreamStatus; @@ -366,7 +366,8 @@ typedef struct STaskQueue { typedef struct STaskSchedInfo { int8_t status; - void* pTimer; + tmr_h pDelayTimer; + tmr_h pIdleTimer; } STaskSchedInfo; typedef struct SSinkRecorder { @@ -541,6 +542,7 @@ typedef struct { SMsgHead head; int64_t streamId; int32_t taskId; + int32_t reqType; } SStreamTaskRunReq; struct SStreamDispatchReq { @@ -764,6 +766,7 @@ SStreamChildEpInfo* streamTaskGetUpstreamTaskEpInfo(SStreamTask* pTask, int32_t void streamTaskInputFail(SStreamTask* pTask); int32_t streamExecTask(SStreamTask* pTask); +int32_t streamResumeTask(SStreamTask* pTask); int32_t streamSchedExec(SStreamTask* pTask); bool streamTaskShouldStop(const SStreamTask* pStatus); bool streamTaskShouldPause(const SStreamTask* pStatus); diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index 3c700665d3..777a2a70b2 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -1847,7 +1847,7 @@ int32_t mndPauseAllStreamTasks(SMnode *pMnode, STrans *pTrans, SStreamObj *pStre } if (atomic_load_8(&pTask->status.taskStatus) != TASK_STATUS__PAUSE) { - atomic_store_8(&pTask->status.keepTaskStatus, pTask->status.taskStatus); + atomic_store_8(&pTask->status.statusBackup, pTask->status.taskStatus); atomic_store_8(&pTask->status.taskStatus, TASK_STATUS__PAUSE); } } @@ -2005,7 +2005,7 @@ int32_t mndResumeAllStreamTasks(STrans *pTrans, SMnode *pMnode, SStreamObj *pStr } if (atomic_load_8(&pTask->status.taskStatus) == TASK_STATUS__PAUSE) { - atomic_store_8(&pTask->status.taskStatus, pTask->status.keepTaskStatus); + atomic_store_8(&pTask->status.taskStatus, pTask->status.statusBackup); } } } diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 65cc56c0b4..2533c721ac 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1068,7 +1068,7 @@ int32_t tqProcessTaskRunReq(STQ* pTq, SRpcMsg* pMsg) { int32_t taskId = pReq->taskId; - if (taskId == STREAM_EXEC_EXTRACT_DATA_IN_WAL_ID) { // all tasks are extracted submit data from the wal + if (taskId == STREAM_EXEC_T_EXTRACT_WAL_DATA) { // all tasks are extracted submit data from the wal tqScanWal(pTq); return 0; } diff --git a/source/dnode/vnode/src/tq/tqStreamTask.c b/source/dnode/vnode/src/tq/tqStreamTask.c index aba3b0945f..b11e231c62 100644 --- a/source/dnode/vnode/src/tq/tqStreamTask.c +++ b/source/dnode/vnode/src/tq/tqStreamTask.c @@ -114,7 +114,8 @@ int32_t tqScanWalAsync(STQ* pTq, bool ckPause) { pRunReq->head.vgId = vgId; pRunReq->streamId = 0; - pRunReq->taskId = STREAM_EXEC_EXTRACT_DATA_IN_WAL_ID; + pRunReq->taskId = 0; + pRunReq->reqType = STREAM_EXEC_T_EXTRACT_WAL_DATA; SRpcMsg msg = {.msgType = TDMT_STREAM_TASK_RUN, .pCont = pRunReq, .contLen = sizeof(SStreamTaskRunReq)}; tmsgPutToQueue(&pTq->pVnode->msgCb, STREAM_QUEUE, &msg); @@ -138,7 +139,8 @@ int32_t tqStopStreamTasksAsync(STQ* pTq) { pRunReq->head.vgId = vgId; pRunReq->streamId = 0; - pRunReq->taskId = STREAM_EXEC_STOP_ALL_TASKS_ID; + pRunReq->taskId = 0; + pRunReq->reqType = STREAM_EXEC_T_STOP_ALL_TASKS; SRpcMsg msg = {.msgType = TDMT_STREAM_TASK_RUN, .pCont = pRunReq, .contLen = sizeof(SStreamTaskRunReq)}; tmsgPutToQueue(&pTq->pVnode->msgCb, STREAM_QUEUE, &msg); diff --git a/source/dnode/vnode/src/tqCommon/tqCommon.c b/source/dnode/vnode/src/tqCommon/tqCommon.c index a3d860fd78..b869d67bed 100644 --- a/source/dnode/vnode/src/tqCommon/tqCommon.c +++ b/source/dnode/vnode/src/tqCommon/tqCommon.c @@ -42,7 +42,8 @@ int32_t tqStreamTaskStartAsync(SStreamMeta* pMeta, SMsgCb* cb, bool restart) { tqDebug("vgId:%d start all %d stream task(s) async", vgId, numOfTasks); pRunReq->head.vgId = vgId; pRunReq->streamId = 0; - pRunReq->taskId = restart ? STREAM_EXEC_RESTART_ALL_TASKS_ID : STREAM_EXEC_START_ALL_TASKS_ID; + pRunReq->taskId = 0; + pRunReq->reqType = restart ? STREAM_EXEC_T_RESTART_ALL_TASKS : STREAM_EXEC_T_START_ALL_TASKS; SRpcMsg msg = {.msgType = TDMT_STREAM_TASK_RUN, .pCont = pRunReq, .contLen = sizeof(SStreamTaskRunReq)}; tmsgPutToQueue(cb, STREAM_QUEUE, &msg); @@ -675,21 +676,30 @@ static int32_t restartStreamTasks(SStreamMeta* pMeta, bool isLeader) { int32_t tqStreamTaskProcessRunReq(SStreamMeta* pMeta, SRpcMsg* pMsg, bool isLeader) { SStreamTaskRunReq* pReq = pMsg->pCont; - int32_t taskId = pReq->taskId; + int32_t type = pReq->reqType; int32_t vgId = pMeta->vgId; - if (taskId == STREAM_EXEC_START_ALL_TASKS_ID) { + if (type == STREAM_EXEC_T_START_ALL_TASKS) { streamMetaStartAllTasks(pMeta); return 0; - } else if (taskId == STREAM_EXEC_RESTART_ALL_TASKS_ID) { + } else if (type == STREAM_EXEC_T_RESTART_ALL_TASKS) { restartStreamTasks(pMeta, isLeader); return 0; - } else if (taskId == STREAM_EXEC_STOP_ALL_TASKS_ID) { + } else if (type == STREAM_EXEC_T_STOP_ALL_TASKS) { streamMetaStopAllTasks(pMeta); return 0; + } else if (type == STREAM_EXEC_T_RESUME_TASK) { + SStreamTask* pTask = streamMetaAcquireTask(pMeta, pReq->streamId, pReq->taskId); + if (pTask != NULL) { + ASSERT(streamTaskReadyToRun(pTask, NULL)); + streamResumeTask(pTask); + } + + streamMetaReleaseTask(pMeta, pTask); + return 0; } - SStreamTask* pTask = streamMetaAcquireTask(pMeta, pReq->streamId, taskId); + SStreamTask* pTask = streamMetaAcquireTask(pMeta, pReq->streamId, pReq->taskId); if (pTask != NULL) { // even in halt status, the data in inputQ must be processed char* p = NULL; if (streamTaskReadyToRun(pTask, &p)) { @@ -706,7 +716,7 @@ int32_t tqStreamTaskProcessRunReq(SStreamMeta* pMeta, SRpcMsg* pMsg, bool isLead return 0; } else { // NOTE: pTask->status.schedStatus is not updated since it is not be handled by the run exec. // todo add one function to handle this - tqError("vgId:%d failed to found s-task, taskId:0x%x may have been dropped", vgId, taskId); + tqError("vgId:%d failed to found s-task, taskId:0x%x may have been dropped", vgId, pReq->taskId); return -1; } } diff --git a/source/libs/stream/src/stream.c b/source/libs/stream/src/stream.c index 1bef42bf14..f383f0f31d 100644 --- a/source/libs/stream/src/stream.c +++ b/source/libs/stream/src/stream.c @@ -57,7 +57,7 @@ static void streamSchedByTimer(void* param, void* tmrId) { 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, streamTimer, &pTask->schedInfo.pTimer); + taosTmrReset(streamSchedByTimer, nextTrigger, pTask, streamTimer, &pTask->schedInfo.pDelayTimer); return; } @@ -68,7 +68,7 @@ static void streamSchedByTimer(void* param, void* tmrId) { 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, streamTimer, &pTask->schedInfo.pTimer); + taosTmrReset(streamSchedByTimer, nextTrigger, pTask, streamTimer, &pTask->schedInfo.pDelayTimer); return; } @@ -77,7 +77,7 @@ static void streamSchedByTimer(void* param, void* tmrId) { int32_t code = streamTaskPutDataIntoInputQ(pTask, (SStreamQueueItem*)pTrigger); if (code != TSDB_CODE_SUCCESS) { - taosTmrReset(streamSchedByTimer, nextTrigger, pTask, streamTimer, &pTask->schedInfo.pTimer); + taosTmrReset(streamSchedByTimer, nextTrigger, pTask, streamTimer, &pTask->schedInfo.pDelayTimer); return; } @@ -85,17 +85,17 @@ static void streamSchedByTimer(void* param, void* tmrId) { } } - taosTmrReset(streamSchedByTimer, nextTrigger, pTask, streamTimer, &pTask->schedInfo.pTimer); + taosTmrReset(streamSchedByTimer, nextTrigger, pTask, streamTimer, &pTask->schedInfo.pDelayTimer); } int32_t streamSetupScheduleTrigger(SStreamTask* pTask) { if (pTask->info.triggerParam != 0 && pTask->info.fillHistory == 0) { int32_t ref = atomic_add_fetch_32(&pTask->refCnt, 1); - ASSERT(ref == 2 && pTask->schedInfo.pTimer == NULL); + ASSERT(ref == 2 && pTask->schedInfo.pDelayTimer == NULL); 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, streamTimer); + pTask->schedInfo.pDelayTimer = taosTmrStart(streamSchedByTimer, (int32_t)pTask->info.triggerParam, pTask, streamTimer); pTask->schedInfo.status = TASK_TRIGGER_STATUS__INACTIVE; } diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index b9839dfc0c..70b7ec3309 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -549,6 +549,13 @@ int32_t streamProcessTranstateBlock(SStreamTask* pTask, SStreamDataBlock* pBlock return code; } +static void setTaskSchedInfo(SStreamTask* pTask, int32_t idleTime) { + SStreamStatus* pStatus = &pTask->status; + + pStatus->schedIdleTime = idleTime; + pStatus->lastExecTs = taosGetTimestampMs(); +} + /** * todo: the batch of blocks should be tuned dynamic, according to the total elapsed time of each batch of blocks, the * appropriate batch of blocks should be handled in 5 to 10 sec. @@ -568,6 +575,12 @@ int32_t doStreamExecTask(SStreamTask* pTask) { break; } + if (streamQueueIsFull(pTask->outputq.queue)) { + stWarn("s-task:%s outputQ is full, idle for 500ms and retry", id); + setTaskSchedInfo(pTask, 500); + break; + } + /*int32_t code = */ streamTaskGetDataFromInputQ(pTask, &pInput, &numOfBlocks, &blockSize); if (pInput == NULL) { ASSERT(numOfBlocks == 0); @@ -582,7 +595,7 @@ int32_t doStreamExecTask(SStreamTask* pTask) { continue; } - if (pInput->type == STREAM_INPUT__TRANS_STATE) { + if (type == STREAM_INPUT__TRANS_STATE) { streamProcessTranstateBlock(pTask, (SStreamDataBlock*)pInput); continue; } @@ -671,27 +684,85 @@ bool streamTaskIsIdle(const SStreamTask* pTask) { } bool streamTaskReadyToRun(const SStreamTask* pTask, char** pStatus) { - ETaskStatus st = streamTaskGetStatus(pTask, NULL); - return (st == TASK_STATUS__READY || st == TASK_STATUS__SCAN_HISTORY/* || st == TASK_STATUS__STREAM_SCAN_HISTORY*/ || - st == TASK_STATUS__CK); + ETaskStatus st = streamTaskGetStatus(pTask, pStatus); + return (st == TASK_STATUS__READY || st == TASK_STATUS__SCAN_HISTORY || st == TASK_STATUS__CK); } -int32_t streamExecTask(SStreamTask* pTask) { - // this function may be executed by multi-threads, so status check is required. +static void doStreamExecTaskHelper(void* param, void* tmrId) { + SStreamTask* pTask = (SStreamTask*)param; + + char* p = NULL; + ETaskStatus status = streamTaskGetStatus(pTask, &p); + if (status == TASK_STATUS__DROPPING || status == TASK_STATUS__STOP) { + streamTaskSetSchedStatusInactive(pTask); + + int32_t ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1); + stDebug("s-task:%s status:%s not resume task, ref:%d", pTask->id.idStr, p, ref); + + streamMetaReleaseTask(pTask->pMeta, pTask); + return; + } + + // task resume running + SStreamTaskRunReq* pRunReq = rpcMallocCont(sizeof(SStreamTaskRunReq)); + if (pRunReq == NULL) { + terrno = TSDB_CODE_OUT_OF_MEMORY; + /*int8_t status = */streamTaskSetSchedStatusInactive(pTask); + + int32_t ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1); + stError("failed to create msg to resume s-task:%s, reason out of memory, ref:%d", pTask->id.idStr, ref); + + streamMetaReleaseTask(pTask->pMeta, pTask); + return; + } + + pRunReq->head.vgId = pTask->info.nodeId; + pRunReq->streamId = pTask->id.streamId; + pRunReq->taskId = pTask->id.taskId; + pRunReq->reqType = STREAM_EXEC_T_RESUME_TASK; + + int32_t ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1); + stDebug("trigger to resume s-task:%s after being idled for %dms, ref:%d", pTask->id.idStr, pTask->status.schedIdleTime, ref); + + SRpcMsg msg = {.msgType = TDMT_STREAM_TASK_RUN, .pCont = pRunReq, .contLen = sizeof(SStreamTaskRunReq)}; + tmsgPutToQueue(pTask->pMsgCb, STREAM_QUEUE, &msg); + + // release the task ref count + streamMetaReleaseTask(pTask->pMeta, pTask); +} + +static int32_t schedTaskInFuture(SStreamTask* pTask) { + int32_t ref = atomic_add_fetch_32(&pTask->status.timerActive, 1); + stDebug("s-task:%s task should idle, add into timer to retry in %dms, ref:%d", + pTask->id.idStr, DISPATCH_RETRY_INTERVAL_MS, ref); + + // add one ref count for task + SStreamTask* pAddRefTask = streamMetaAcquireTask(pTask->pMeta, pTask->id.streamId, pTask->id.taskId); + + if (pTask->schedInfo.pIdleTimer == NULL) { + pTask->schedInfo.pIdleTimer = taosTmrStart(doStreamExecTaskHelper, pTask->status.schedIdleTime, pTask, streamTimer); + } else { + taosTmrReset(doStreamExecTaskHelper, pTask->status.schedIdleTime, pTask, streamTimer, &pTask->schedInfo.pIdleTimer); + } + + return TSDB_CODE_SUCCESS; +} + +int32_t streamResumeTask(SStreamTask* pTask) { + ASSERT(pTask->status.schedStatus == 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 = doStreamExecTask(pTask); - if (code < 0) { // todo this status should be removed - atomic_store_8(&pTask->status.schedStatus, TASK_SCHED_STATUS__FAILED); - return -1; - } + while (1) { + /*int32_t code = */doStreamExecTask(pTask); + taosThreadMutexLock(&pTask->lock); - taosThreadMutexLock(&pTask->lock); - if ((streamQueueGetNumOfItems(pTask->inputq.queue) == 0) || streamTaskShouldStop(pTask) || - streamTaskShouldPause(pTask)) { + // check if this task needs to be idle for a while + if (pTask->status.schedIdleTime > 0) { + stDebug("s-task:%s idled, and will be invoked in %dms", id, pTask->status.schedIdleTime); + schedTaskInFuture(pTask); + } else { + int32_t numOfItems = streamQueueGetNumOfItems(pTask->inputq.queue); + if ((numOfItems == 0) || streamTaskShouldStop(pTask) || streamTaskShouldPause(pTask)) { atomic_store_8(&pTask->status.schedStatus, TASK_SCHED_STATUS__INACTIVE); taosThreadMutexUnlock(&pTask->lock); @@ -700,8 +771,19 @@ int32_t streamExecTask(SStreamTask* pTask) { stDebug("s-task:%s exec completed, status:%s, sched-status:%d", id, p, pTask->status.schedStatus); return 0; } - taosThreadMutexUnlock(&pTask->lock); } + + taosThreadMutexUnlock(&pTask->lock); + } +} + +int32_t streamExecTask(SStreamTask* pTask) { + // this function may be executed by multi-threads, so status check is required. + const char* id = pTask->id.idStr; + + int8_t schedStatus = streamTaskSetSchedStatusActive(pTask); + if (schedStatus == TASK_SCHED_STATUS__WAITING) { + streamResumeTask(pTask); } else { char* p = NULL; streamTaskGetStatus(pTask, &p); diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 9c94799237..dc2e3bd651 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -452,7 +452,7 @@ void streamMetaClear(SStreamMeta* pMeta) { // release the ref by timer if (p->info.triggerParam != 0 && p->info.fillHistory == 0) { // one more ref in timer stDebug("s-task:%s stop schedTimer, and (before) desc ref:%d", p->id.idStr, p->refCnt); - taosTmrStop(p->schedInfo.pTimer); + taosTmrStop(p->schedInfo.pDelayTimer); p->info.triggerParam = 0; streamMetaReleaseTask(pMeta, p); } @@ -730,7 +730,7 @@ int32_t streamMetaUnregisterTask(SStreamMeta* pMeta, int64_t streamId, int32_t t if (pTask->info.triggerParam != 0 && pTask->info.fillHistory == 0) { stDebug("s-task:%s stop schedTimer, and (before) desc ref:%d", pTask->id.idStr, pTask->refCnt); - taosTmrStop(pTask->schedInfo.pTimer); + taosTmrStop(pTask->schedInfo.pDelayTimer); pTask->info.triggerParam = 0; streamMetaReleaseTask(pMeta, pTask); } diff --git a/source/libs/stream/src/streamQueue.c b/source/libs/stream/src/streamQueue.c index d1610362f9..459efb838c 100644 --- a/source/libs/stream/src/streamQueue.c +++ b/source/libs/stream/src/streamQueue.c @@ -343,6 +343,7 @@ int32_t streamTaskPutDataIntoInputQ(SStreamTask* pTask, SStreamQueueItem* pItem) int32_t streamTaskPutDataIntoOutputQ(SStreamTask* pTask, SStreamDataBlock* pBlock) { STaosQueue* pQueue = pTask->outputq.queue->pQueue; +#if 0 // wait for the output queue is available for new data to dispatch while (streamQueueIsFull(pTask->outputq.queue)) { if (streamTaskShouldStop(pTask)) { @@ -358,6 +359,7 @@ int32_t streamTaskPutDataIntoOutputQ(SStreamTask* pTask, SStreamDataBlock* pBloc taosMsleep(OUTPUT_QUEUE_FULL_WAIT_DURATION); } +#endif int32_t code = taosWriteQitem(pQueue, pBlock); @@ -367,7 +369,14 @@ int32_t streamTaskPutDataIntoOutputQ(SStreamTask* pTask, SStreamDataBlock* pBloc 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 { - stDebug("s-task:%s data put into outputQ, outputQ items:%d, size:%.2fMiB", pTask->id.idStr, total, size); + if (streamQueueIsFull(pTask->outputq.queue)) { + stWarn( + "s-task:%s outputQ is full(outputQ items:%d, size:%.2fMiB), set the output status BLOCKING, wait for 500ms " + "after handle this batch of blocks", + pTask->id.idStr, total, size); + } else { + stDebug("s-task:%s data put into outputQ, outputQ items:%d, size:%.2fMiB", pTask->id.idStr, total, size); + } } return TSDB_CODE_SUCCESS; diff --git a/source/libs/stream/src/streamTask.c b/source/libs/stream/src/streamTask.c index 4d343a484d..32fd5cbb39 100644 --- a/source/libs/stream/src/streamTask.c +++ b/source/libs/stream/src/streamTask.c @@ -328,9 +328,9 @@ void tFreeStreamTask(SStreamTask* pTask) { taosMsleep(100); } - if (pTask->schedInfo.pTimer != NULL) { - taosTmrStop(pTask->schedInfo.pTimer); - pTask->schedInfo.pTimer = NULL; + if (pTask->schedInfo.pDelayTimer != NULL) { + taosTmrStop(pTask->schedInfo.pDelayTimer); + pTask->schedInfo.pDelayTimer = NULL; } if (pTask->hTaskInfo.pTimer != NULL) { From 4138692cea039e3e9abb7b4bb337987a365669ce Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 3 Jan 2024 18:16:18 +0800 Subject: [PATCH 003/102] fix(stream): scan wal data files. --- source/dnode/vnode/src/tq/tq.c | 8 ++++---- source/libs/stream/src/streamExec.c | 1 + 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 2533c721ac..925671ae04 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1066,15 +1066,15 @@ int32_t tqProcessTaskScanHistoryFinishRsp(STQ* pTq, SRpcMsg* pMsg) { int32_t tqProcessTaskRunReq(STQ* pTq, SRpcMsg* pMsg) { SStreamTaskRunReq* pReq = pMsg->pCont; - int32_t taskId = pReq->taskId; - - if (taskId == STREAM_EXEC_T_EXTRACT_WAL_DATA) { // all tasks are extracted submit data from the wal + if (pReq->reqType == STREAM_EXEC_T_EXTRACT_WAL_DATA) { // all tasks are extracted submit data from the wal tqScanWal(pTq); return 0; } int32_t code = tqStreamTaskProcessRunReq(pTq->pStreamMeta, pMsg, vnodeIsRoleLeader(pTq->pVnode)); - if(code == 0 && taskId > 0){ + + // let's continue scan data in the wal files + if(code == 0 && pReq->reqType > 0){ tqScanWalAsync(pTq, false); } diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index 70b7ec3309..e73c5ffddb 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -728,6 +728,7 @@ static void doStreamExecTaskHelper(void* param, void* tmrId) { tmsgPutToQueue(pTask->pMsgCb, STREAM_QUEUE, &msg); // release the task ref count + pTask->status.schedIdleTime = 0; // clear the idle time streamMetaReleaseTask(pTask->pMeta, pTask); } From 4399867e14eea095966dd3a3d31a0d342ca90ed1 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 3 Jan 2024 18:28:23 +0800 Subject: [PATCH 004/102] fix(stream): fix the not running bug. --- source/dnode/vnode/src/tq/tq.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 925671ae04..851d3f967b 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1074,7 +1074,7 @@ int32_t tqProcessTaskRunReq(STQ* pTq, SRpcMsg* pMsg) { int32_t code = tqStreamTaskProcessRunReq(pTq->pStreamMeta, pMsg, vnodeIsRoleLeader(pTq->pVnode)); // let's continue scan data in the wal files - if(code == 0 && pReq->reqType > 0){ + if(code == 0 && pReq->reqType >= 0){ tqScanWalAsync(pTq, false); } From b06ccc0ca75daa1b35fdec47cc6926c6d61c3101 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 3 Jan 2024 19:18:06 +0800 Subject: [PATCH 005/102] fix(stream): cleanup the checkpoint trans & task info in execInfo --- source/dnode/mnode/impl/src/mndStream.c | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index 777a2a70b2..b57ca009d3 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -1500,6 +1500,16 @@ int32_t mndDropStreamByDb(SMnode *pMnode, STrans *pTrans, SDbObj *pDb) { return -1; } #endif + + // kill the related checkpoint trans + int32_t transId = mndStreamGetRelCheckpointTrans(pMnode, pStream->uid); + if (transId != 0) { + killCheckpointTransImpl(pMnode, transId, pStream->sourceDb); + } + + // drop the stream obj in execInfo + removeStreamTasksInBuf(pStream, &execInfo); + if (mndPersistDropStreamLog(pMnode, pTrans, pStream) < 0) { sdbRelease(pSdb, pStream); sdbCancelFetch(pSdb, pIter); From 3fea717373a0cf0ec228cf1abab9651922fad68f Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 3 Jan 2024 19:28:49 +0800 Subject: [PATCH 006/102] fix(stream): discard the orphan tasks. --- source/client/test/clientTests.cpp | 24 +++++++++++----------- source/dnode/mnode/impl/src/mndStream.c | 27 +++++++++++++++---------- 2 files changed, 28 insertions(+), 23 deletions(-) diff --git a/source/client/test/clientTests.cpp b/source/client/test/clientTests.cpp index e78783cf3c..e6519a436e 100644 --- a/source/client/test/clientTests.cpp +++ b/source/client/test/clientTests.cpp @@ -939,8 +939,8 @@ TEST(clientCase, agg_query_tables) { } taos_free_result(pRes); - int64_t st = 1685959293000; - for (int32_t i = 0; i < 10000000; ++i) { + int64_t st = 1685959293299; + for (int32_t i = 0; i < 5; ++i) { char s[256] = {0}; while (1) { @@ -954,16 +954,16 @@ TEST(clientCase, agg_query_tables) { } } - while (1) { - sprintf(s, "insert into t2 values(%ld, %d)", st + i, i); - pRes = taos_query(pConn, s); - int32_t ret = taos_errno(pRes); - - taos_free_result(pRes); - if (ret == 0) { - break; - } - } +// while (1) { +// sprintf(s, "insert into t2 values(%ld, %d)", st + i, i); +// pRes = taos_query(pConn, s); +// int32_t ret = taos_errno(pRes); +// +// taos_free_result(pRes); +// if (ret == 0) { +// break; +// } +// } } // pRes = taos_query(pConn, "show table distributed tup"); diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index b57ca009d3..146f9f6fc4 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -1642,10 +1642,18 @@ static void mndCancelGetNextStream(SMnode *pMnode, void *pIter) { sdbCancelFetch(pSdb, pIter); } -static void setTaskAttrInResBlock(SStreamObj *pStream, SStreamTask *pTask, SSDataBlock *pBlock, int32_t numOfRows) { +static int32_t setTaskAttrInResBlock(SStreamObj *pStream, SStreamTask *pTask, SSDataBlock *pBlock, int32_t numOfRows) { SColumnInfoData *pColInfo; int32_t cols = 0; + STaskId id = {.streamId = pTask->id.streamId, .taskId = pTask->id.taskId}; + + STaskStatusEntry *pe = taosHashGet(execInfo.pTaskMap, &id, sizeof(id)); + if (pe == NULL) { + mError("task:0x%" PRIx64 " not exists in vnode, no valid status/stage info", id.taskId); + return -1; + } + // stream name char streamName[TSDB_TABLE_NAME_LEN + VARSTR_HEADER_SIZE] = {0}; STR_WITH_MAXSIZE_TO_VARSTR(streamName, mndGetDbStr(pStream->name), sizeof(streamName)); @@ -1696,14 +1704,7 @@ static void setTaskAttrInResBlock(SStreamObj *pStream, SStreamTask *pTask, SSDat colDataSetVal(pColInfo, numOfRows, (const char *)level, false); // status - char status[20 + VARSTR_HEADER_SIZE] = {0}; - STaskId id = {.streamId = pTask->id.streamId, .taskId = pTask->id.taskId}; - - STaskStatusEntry *pe = taosHashGet(execInfo.pTaskMap, &id, sizeof(id)); - if (pe == NULL) { - mError("task:0x%" PRIx64 " not exists in vnode, no valid status/stage info", id.taskId); - return; - } + char status[20 + VARSTR_HEADER_SIZE] = {0}; const char *pStatus = streamTaskGetStatusStr(pe->status); STR_TO_VARSTR(status, pStatus); @@ -1746,6 +1747,8 @@ static void setTaskAttrInResBlock(SStreamObj *pStream, SStreamTask *pTask, SSDat pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); colDataSetVal(pColInfo, numOfRows, (const char *)vbuf, false); + + return TSDB_CODE_SUCCESS; } static int32_t getNumOfTasks(SArray *pTaskList) { @@ -1787,8 +1790,10 @@ static int32_t mndRetrieveStreamTask(SRpcMsg *pReq, SShowObj *pShow, SSDataBlock int32_t numOfLevels = taosArrayGetSize(pLevel); for (int32_t j = 0; j < numOfLevels; j++) { SStreamTask *pTask = taosArrayGetP(pLevel, j); - setTaskAttrInResBlock(pStream, pTask, pBlock, numOfRows); - numOfRows++; + int32_t code = setTaskAttrInResBlock(pStream, pTask, pBlock, numOfRows); + if (code == TSDB_CODE_SUCCESS) { + numOfRows++; + } } } From 74df121cf5e36a1b364030eb38dc5e2e60f3c678 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 3 Jan 2024 22:29:46 +0800 Subject: [PATCH 007/102] fix(stream): fix the deadlock. --- source/libs/stream/src/streamExec.c | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index e73c5ffddb..d68288222c 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -734,8 +734,8 @@ static void doStreamExecTaskHelper(void* param, void* tmrId) { static int32_t schedTaskInFuture(SStreamTask* pTask) { int32_t ref = atomic_add_fetch_32(&pTask->status.timerActive, 1); - stDebug("s-task:%s task should idle, add into timer to retry in %dms, ref:%d", - pTask->id.idStr, DISPATCH_RETRY_INTERVAL_MS, ref); + stDebug("s-task:%s task should idle, add into timer to retry in %dms, ref:%d", pTask->id.idStr, + pTask->status.schedIdleTime, ref); // add one ref count for task SStreamTask* pAddRefTask = streamMetaAcquireTask(pTask->pMeta, pTask->id.streamId, pTask->id.taskId); @@ -761,6 +761,9 @@ int32_t streamResumeTask(SStreamTask* pTask) { if (pTask->status.schedIdleTime > 0) { stDebug("s-task:%s idled, and will be invoked in %dms", id, pTask->status.schedIdleTime); schedTaskInFuture(pTask); + + taosThreadMutexUnlock(&pTask->lock); + return 0; } else { int32_t numOfItems = streamQueueGetNumOfItems(pTask->inputq.queue); if ((numOfItems == 0) || streamTaskShouldStop(pTask) || streamTaskShouldPause(pTask)) { From 1e3aad88831a7d2e34a725df9d3ba5739a36e9d2 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 3 Jan 2024 23:57:33 +0800 Subject: [PATCH 008/102] refactor: do some internal refactor. --- source/dnode/vnode/src/tq/tqStreamTask.c | 26 +++++++++++------------- 1 file changed, 12 insertions(+), 14 deletions(-) diff --git a/source/dnode/vnode/src/tq/tqStreamTask.c b/source/dnode/vnode/src/tq/tqStreamTask.c index b11e231c62..38b240b47e 100644 --- a/source/dnode/vnode/src/tq/tqStreamTask.c +++ b/source/dnode/vnode/src/tq/tqStreamTask.c @@ -32,26 +32,24 @@ int32_t tqScanWal(STQ* pTq) { int64_t st = taosGetTimestampMs(); while (1) { - int32_t scan = pMeta->walScanCounter; - tqDebug("vgId:%d continue check if data in wal are available, walScanCounter:%d", vgId, scan); + tqDebug("vgId:%d continue check if data in wal are available, walScanCounter:%d", vgId, pMeta->walScanCounter); // check all tasks bool shouldIdle = true; - doScanWalForAllTasks(pTq->pStreamMeta, &shouldIdle); + doScanWalForAllTasks(pMeta, &shouldIdle); -// if (shouldIdle) { - streamMetaWLock(pMeta); - int32_t times = (--pMeta->walScanCounter); - ASSERT(pMeta->walScanCounter >= 0); - streamMetaWUnLock(pMeta); + streamMetaWLock(pMeta); + int32_t times = (--pMeta->walScanCounter); + ASSERT(pMeta->walScanCounter >= 0); + streamMetaWUnLock(pMeta); - if (times <= 0) { - break; - } else { - tqDebug("vgId:%d scan wal for stream tasks for %d times in %dms", vgId, times, SCAN_WAL_IDLE_DURATION); - } -// } + if (times > 0) { + tqDebug("vgId:%d scan wal for stream tasks for %d times in %dms", vgId, times, SCAN_WAL_IDLE_DURATION); + } else { // times <= 0 + break; + } + // todo: remove the sleep taosMsleep(SCAN_WAL_IDLE_DURATION); } From c49425200271b920024277d9d532ef5f9defa005 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 4 Jan 2024 17:44:05 +0800 Subject: [PATCH 009/102] fix(stream): record the related fill-history task failure. --- source/dnode/vnode/src/tqCommon/tqCommon.c | 21 +++++++++++++++++++-- source/libs/stream/src/streamMeta.c | 11 ++++++++--- source/libs/stream/src/streamStart.c | 16 ++++------------ 3 files changed, 31 insertions(+), 17 deletions(-) diff --git a/source/dnode/vnode/src/tqCommon/tqCommon.c b/source/dnode/vnode/src/tqCommon/tqCommon.c index b869d67bed..95bc4f4131 100644 --- a/source/dnode/vnode/src/tqCommon/tqCommon.c +++ b/source/dnode/vnode/src/tqCommon/tqCommon.c @@ -438,19 +438,36 @@ int32_t tqStreamTaskProcessCheckRsp(SStreamMeta* pMeta, SRpcMsg* pMsg, bool isLe tqDebug("tq task:0x%x (vgId:%d) recv check rsp(reqId:0x%" PRIx64 ") from 0x%x (vgId:%d) status %d", rsp.upstreamTaskId, rsp.upstreamNodeId, rsp.reqId, rsp.downstreamTaskId, rsp.downstreamNodeId, rsp.status); + int64_t initTs = 0; + int64_t now = taosGetTimestampMs(); if (!isLeader) { - streamMetaUpdateTaskDownstreamStatus(pMeta, rsp.streamId, rsp.upstreamTaskId, 0, taosGetTimestampMs(), false); + SStreamTask* pTask = streamMetaAcquireTask(pMeta, rsp.streamId, rsp.upstreamTaskId); + if (pTask != NULL) { + initTs = pTask->execInfo.init; + + if (HAS_RELATED_FILLHISTORY_TASK(pTask)) { + STaskId* pId = &pTask->hTaskInfo.id; + streamMetaUpdateTaskDownstreamStatus(pMeta, pId->streamId, pId->taskId, initTs, now, false); + } + + streamMetaReleaseTask(pMeta, pTask); + } + + streamMetaUpdateTaskDownstreamStatus(pMeta, rsp.streamId, rsp.upstreamTaskId, initTs, now, false); tqError("vgId:%d not leader, task:0x%x not handle the check rsp, downstream:0x%x (vgId:%d)", vgId, rsp.upstreamTaskId, rsp.downstreamTaskId, rsp.downstreamNodeId); + return code; } SStreamTask* pTask = streamMetaAcquireTask(pMeta, rsp.streamId, rsp.upstreamTaskId); if (pTask == NULL) { - streamMetaUpdateTaskDownstreamStatus(pMeta, rsp.streamId, rsp.upstreamTaskId, 0, taosGetTimestampMs(), false); + streamMetaUpdateTaskDownstreamStatus(pMeta, rsp.streamId, rsp.upstreamTaskId, 0, now, false); 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, vgId); terrno = TSDB_CODE_STREAM_TASK_NOT_EXIST; + + // failed to find the related fill-history task return -1; } diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index dc2e3bd651..8017a88dea 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -1467,6 +1467,8 @@ int32_t streamMetaStartAllTasks(SStreamMeta* pMeta) { continue; } + // todo: may be we should find the related fill-history task and set it failed. + // fill-history task can only be launched by related stream tasks. STaskExecStatisInfo* pInfo = &pTask->execInfo; if (pTask->info.fillHistory == 1) { @@ -1486,13 +1488,16 @@ int32_t streamMetaStartAllTasks(SStreamMeta* pMeta) { continue; } - EStreamTaskEvent event = /*(HAS_RELATED_FILLHISTORY_TASK(pTask)) ? TASK_EVENT_INIT_STREAM_SCANHIST : */TASK_EVENT_INIT; - int32_t ret = streamTaskHandleEvent(pTask->status.pSM, event); + int32_t ret = streamTaskHandleEvent(pTask->status.pSM, TASK_EVENT_INIT); if (ret != TSDB_CODE_SUCCESS) { - stError("vgId:%d failed to handle event:%d", pMeta->vgId, event); + stError("vgId:%d failed to handle event:%d", pMeta->vgId, TASK_EVENT_INIT); code = ret; streamMetaUpdateTaskDownstreamStatus(pMeta, pTaskId->streamId, pTaskId->taskId, pInfo->init, pInfo->start, false); + if (HAS_RELATED_FILLHISTORY_TASK(pTask)) { + STaskId* pId = &pTask->hTaskInfo.id; + streamMetaUpdateTaskDownstreamStatus(pMeta, pId->streamId, pId->taskId, pInfo->init, pInfo->start, false); + } } streamMetaReleaseTask(pMeta, pTask); diff --git a/source/libs/stream/src/streamStart.c b/source/libs/stream/src/streamStart.c index 70f83e27a5..945d7844af 100644 --- a/source/libs/stream/src/streamStart.c +++ b/source/libs/stream/src/streamStart.c @@ -500,22 +500,14 @@ int32_t streamProcessCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* pRs addIntoNodeUpdateList(pTask, pRsp->downstreamNodeId); } - streamMetaUpdateTaskDownstreamStatus(pTask->pMeta, pTask->id.streamId, pTask->id.taskId, pTask->execInfo.init, - taosGetTimestampMs(), false); + int32_t startTs = pTask->execInfo.init; + int64_t now = taosGetTimestampMs(); + streamMetaUpdateTaskDownstreamStatus(pTask->pMeta, pTask->id.streamId, pTask->id.taskId, startTs, now, false); // automatically set the related fill-history task to be failed. if (HAS_RELATED_FILLHISTORY_TASK(pTask)) { STaskId* pId = &pTask->hTaskInfo.id; - int64_t current = taosGetTimestampMs(); - - SStreamTask* pHTask = streamMetaAcquireTask(pTask->pMeta, pId->streamId, pId->taskId); - if (pHTask != NULL) { - streamMetaUpdateTaskDownstreamStatus(pTask->pMeta, pId->streamId, pId->taskId, pHTask->execInfo.init, current, - false); - streamMetaReleaseTask(pTask->pMeta, pHTask); - } else { - streamMetaUpdateTaskDownstreamStatus(pTask->pMeta, pId->streamId, pId->taskId, 0, current, false); - } + streamMetaUpdateTaskDownstreamStatus(pTask->pMeta, pId->streamId, pId->taskId, startTs, now, false); } } else { // TASK_DOWNSTREAM_NOT_READY, let's retry in 100ms STaskRecheckInfo* pInfo = createRecheckInfo(pTask, pRsp); From 6d37e596ec466f26c84b7ccd481adf24b6c35897 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 4 Jan 2024 18:47:49 +0800 Subject: [PATCH 010/102] refactor: do some internal refactor. --- source/dnode/snode/src/snode.c | 4 +- source/dnode/vnode/src/tq/tq.c | 29 +++------ source/dnode/vnode/src/tq/tqStreamTask.c | 16 +++-- source/dnode/vnode/src/tqCommon/tqCommon.c | 10 +-- source/libs/stream/inc/streamsm.h | 5 -- source/libs/stream/src/stream.c | 4 +- source/libs/stream/src/streamCheckpoint.c | 7 +-- source/libs/stream/src/streamDispatch.c | 11 ++-- source/libs/stream/src/streamExec.c | 51 +++++++-------- source/libs/stream/src/streamMeta.c | 10 +-- source/libs/stream/src/streamStart.c | 72 ++++++++++------------ source/libs/stream/src/streamTask.c | 4 +- source/libs/stream/src/streamTaskSm.c | 13 ++-- 13 files changed, 103 insertions(+), 133 deletions(-) diff --git a/source/dnode/snode/src/snode.c b/source/dnode/snode/src/snode.c index 33eb9cd3ed..837c55c219 100644 --- a/source/dnode/snode/src/snode.c +++ b/source/dnode/snode/src/snode.c @@ -91,9 +91,7 @@ int32_t sndExpandTask(SSnode *pSnode, SStreamTask *pTask, int64_t nextProcessVer pTask->id.idStr, pChkInfo->checkpointId, pChkInfo->checkpointVer, pChkInfo->nextProcessVer); } - char *p = NULL; - streamTaskGetStatus(pTask, &p); - + char* p = streamTaskGetStatus(pTask)->name; if (pTask->info.fillHistory) { sndInfo("vgId:%d expand stream task, s-task:%s, checkpointId:%" PRId64 " checkpointVer:%" PRId64 " nextProcessVer:%" PRId64 diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 851d3f967b..203c34f1ce 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -842,8 +842,7 @@ int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t nextProcessVer) { pChkInfo->checkpointId, pChkInfo->checkpointVer, pChkInfo->nextProcessVer); } - char* p = NULL; - streamTaskGetStatus(pTask, &p); + char* p = streamTaskGetStatus(pTask)->name; if (pTask->info.fillHistory) { tqInfo("vgId:%d expand stream task, s-task:%s, checkpointId:%" PRId64 " checkpointVer:%" PRId64 @@ -932,8 +931,7 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { // do recovery step1 const char* id = pTask->id.idStr; - char* pStatus = NULL; - streamTaskGetStatus(pTask, &pStatus); + char* pStatus = streamTaskGetStatus(pTask)->name; // avoid multi-thread exec while (1) { @@ -990,15 +988,15 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { if (retInfo.ret == TASK_SCANHISTORY_REXEC) { streamReExecScanHistoryFuture(pTask, retInfo.idleTime); } else { - char* p = NULL; - ETaskStatus s = streamTaskGetStatus(pTask, &p); + SStreamTaskState* p = streamTaskGetStatus(pTask); + ETaskStatus s = p->state; if (s == TASK_STATUS__PAUSE) { tqDebug("s-task:%s is paused in the step1, elapsed time:%.2fs total:%.2fs, sched-status:%d", pTask->id.idStr, el, pTask->execInfo.step1El, status); } else if (s == TASK_STATUS__STOP || s == TASK_STATUS__DROPPING) { - tqDebug("s-task:%s status:%p not continue scan-history data, total elapsed time:%.2fs quit", pTask->id.idStr, p, - pTask->execInfo.step1El); + tqDebug("s-task:%s status:%p not continue scan-history data, total elapsed time:%.2fs quit", pTask->id.idStr, + p->name, pTask->execInfo.step1El); } } @@ -1038,15 +1036,6 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { streamMetaReleaseTask(pMeta, pStreamTask); } else { ASSERT(0); -// STimeWindow* pWindow = &pTask->dataRange.window; -// ASSERT(HAS_RELATED_FILLHISTORY_TASK(pTask) || streamTaskShouldStop(pTask)); -// -// // Not update the fill-history time window until the state transfer is completed. -// tqDebug("s-task:%s scan-history in stream time window completed, start to handle data from WAL, startVer:%" PRId64 -// ", window:%" PRId64 " - %" PRId64, -// id, pTask->chkInfo.nextProcessVer, pWindow->skey, pWindow->ekey); -// -// code = streamTaskScanHistoryDataComplete(pTask); } atomic_store_32(&pTask->status.inScanHistorySentinel, 0); @@ -1138,7 +1127,7 @@ int32_t tqProcessTaskResumeImpl(STQ* pTq, SStreamTask* pTask, int64_t sversion, } streamTaskResume(pTask); - ETaskStatus status = streamTaskGetStatus(pTask, NULL); + ETaskStatus status = streamTaskGetStatus(pTask)->state; int32_t level = pTask->info.taskLevel; if (level == TASK_LEVEL__SINK) { @@ -1275,7 +1264,7 @@ int32_t tqProcessTaskCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg* pRsp) // todo save the checkpoint failed info taosThreadMutexLock(&pTask->lock); - ETaskStatus status = streamTaskGetStatus(pTask, NULL); + ETaskStatus status = streamTaskGetStatus(pTask)->state; if (status == TASK_STATUS__HALT || status == TASK_STATUS__PAUSE) { tqError("s-task:%s not ready for checkpoint, since it is halt, ignore this checkpoint:%" PRId64 ", set it failure", @@ -1362,7 +1351,7 @@ int32_t tqProcessTaskDropHTask(STQ* pTq, SRpcMsg* pMsg) { } taosThreadMutexLock(&pTask->lock); - ETaskStatus status = streamTaskGetStatus(pTask, NULL); + ETaskStatus status = streamTaskGetStatus(pTask)->state; // if (status == TASK_STATUS__STREAM_SCAN_HISTORY) { // streamTaskHandleEvent(pTask->status.pSM, TASK_EVENT_SCANHIST_DONE); // } diff --git a/source/dnode/vnode/src/tq/tqStreamTask.c b/source/dnode/vnode/src/tq/tqStreamTask.c index 38b240b47e..12f540b9cf 100644 --- a/source/dnode/vnode/src/tq/tqStreamTask.c +++ b/source/dnode/vnode/src/tq/tqStreamTask.c @@ -220,16 +220,15 @@ bool taskReadyForDataFromWal(SStreamTask* pTask) { } // not in ready state, do not handle the data from wal - char* p = NULL; - int32_t status = streamTaskGetStatus(pTask, &p); - if (streamTaskGetStatus(pTask, &p) != TASK_STATUS__READY) { - tqTrace("s-task:%s not ready for submit block in wal, status:%s", pTask->id.idStr, p); + SStreamTaskState* pState = streamTaskGetStatus(pTask); + if (pState->state != TASK_STATUS__READY) { + tqTrace("s-task:%s not ready for submit block in wal, status:%s", pTask->id.idStr, pState->name); 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__READY); + ASSERT(pState->state == TASK_STATUS__READY); // 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); @@ -342,10 +341,9 @@ int32_t doScanWalForAllTasks(SStreamMeta* pStreamMeta, bool* pScanIdle) { taosThreadMutexLock(&pTask->lock); - char* p = NULL; - ETaskStatus status = streamTaskGetStatus(pTask, &p); - if (status != TASK_STATUS__READY) { - tqDebug("s-task:%s not ready for submit block from wal, status:%s", pTask->id.idStr, p); + SStreamTaskState* pState = streamTaskGetStatus(pTask); + if (pState->state != TASK_STATUS__READY) { + tqDebug("s-task:%s not ready for submit block from wal, status:%s", pTask->id.idStr, pState->name); taosThreadMutexUnlock(&pTask->lock); streamMetaReleaseTask(pStreamMeta, pTask); continue; diff --git a/source/dnode/vnode/src/tqCommon/tqCommon.c b/source/dnode/vnode/src/tqCommon/tqCommon.c index 95bc4f4131..6a47cfc3eb 100644 --- a/source/dnode/vnode/src/tqCommon/tqCommon.c +++ b/source/dnode/vnode/src/tqCommon/tqCommon.c @@ -400,11 +400,11 @@ int32_t tqStreamTaskProcessCheckReq(SStreamMeta* pMeta, SRpcMsg* pMsg) { rsp.status = streamTaskCheckStatus(pTask, req.upstreamTaskId, req.upstreamNodeId, req.stage, &rsp.oldStage); streamMetaReleaseTask(pMeta, pTask); - char* p = NULL; - streamTaskGetStatus(pTask, &p); + SStreamTaskState* pState = streamTaskGetStatus(pTask); tqDebug("s-task:%s status:%s, stage:%" PRId64 " recv task check req(reqId:0x%" PRIx64 ") task:0x%x (vgId:%d), check_status:%d", - pTask->id.idStr, p, rsp.oldStage, rsp.reqId, rsp.upstreamTaskId, rsp.upstreamNodeId, rsp.status); + pTask->id.idStr, pState->name, 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 @@ -706,9 +706,11 @@ int32_t tqStreamTaskProcessRunReq(SStreamMeta* pMeta, SRpcMsg* pMsg, bool isLead streamMetaStopAllTasks(pMeta); return 0; } else if (type == STREAM_EXEC_T_RESUME_TASK) { + // task resume to run after idle for a while SStreamTask* pTask = streamMetaAcquireTask(pMeta, pReq->streamId, pReq->taskId); if (pTask != NULL) { ASSERT(streamTaskReadyToRun(pTask, NULL)); + tqDebug("s-task:%s task resume to run after idle for a while", pTask->id.idStr); streamResumeTask(pTask); } @@ -772,7 +774,7 @@ int32_t tqStreamTaskProcessTaskResetReq(SStreamMeta* pMeta, SRpcMsg* pMsg) { tqDebug("s-task:%s receive task-reset msg from mnode, reset status and ready for data processing", pTask->id.idStr); // clear flag set during do checkpoint, and open inputQ for all upstream tasks - if (streamTaskGetStatus(pTask, NULL) == TASK_STATUS__CK) { + if (streamTaskGetStatus(pTask)->state == TASK_STATUS__CK) { streamTaskClearCheckInfo(pTask, true); streamTaskSetStatusReady(pTask); } diff --git a/source/libs/stream/inc/streamsm.h b/source/libs/stream/inc/streamsm.h index ea0522bd5a..abdafc0240 100644 --- a/source/libs/stream/inc/streamsm.h +++ b/source/libs/stream/inc/streamsm.h @@ -23,11 +23,6 @@ extern "C" { #endif // moore finite state machine for stream task -typedef struct SStreamTaskState { - ETaskStatus state; - char* name; -} SStreamTaskState; - typedef int32_t (*__state_trans_fn)(SStreamTask*); typedef int32_t (*__state_trans_succ_fn)(SStreamTask*); diff --git a/source/libs/stream/src/stream.c b/source/libs/stream/src/stream.c index f383f0f31d..10b1b3fbbe 100644 --- a/source/libs/stream/src/stream.c +++ b/source/libs/stream/src/stream.c @@ -49,7 +49,7 @@ static void streamSchedByTimer(void* param, void* tmrId) { return; } - if (streamTaskGetStatus(pTask, NULL) == TASK_STATUS__CK) { + if (streamTaskGetStatus(pTask)->state == TASK_STATUS__CK) { stDebug("s-task:%s in checkpoint procedure, not retrieve result, next:%dms", id, nextTrigger); } else { if (status == TASK_TRIGGER_STATUS__ACTIVE) { @@ -247,7 +247,7 @@ int32_t streamProcessDispatchMsg(SStreamTask* pTask, SStreamDispatchReq* pReq, S } // disable the data from upstream tasks - if (streamTaskGetStatus(pTask, NULL) == TASK_STATUS__HALT) { + if (streamTaskGetStatus(pTask)->state == 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 b54adb0f96..c75185b0ea 100644 --- a/source/libs/stream/src/streamCheckpoint.c +++ b/source/libs/stream/src/streamCheckpoint.c @@ -187,7 +187,7 @@ int32_t streamProcessCheckpointBlock(SStreamTask* pTask, SStreamDataBlock* pBloc int32_t code = TSDB_CODE_SUCCESS; // set task status - if (streamTaskGetStatus(pTask, NULL) != TASK_STATUS__CK) { + if (streamTaskGetStatus(pTask)->state != TASK_STATUS__CK) { pTask->chkInfo.checkpointingId = checkpointId; code = streamTaskHandleEvent(pTask->status.pSM, TASK_EVENT_GEN_CHECKPOINT); if (code != TSDB_CODE_SUCCESS) { @@ -309,8 +309,7 @@ int32_t streamSaveTaskCheckpointInfo(SStreamTask* p, int64_t checkpointId) { p->chkInfo.checkpointVer = p->chkInfo.processedVer; streamTaskClearCheckInfo(p, false); - char* str = NULL; - streamTaskGetStatus(p, &str); + SStreamTaskState* pState = streamTaskGetStatus(p); code = streamTaskHandleEvent(p->status.pSM, TASK_EVENT_CHECKPOINT_DONE); taosThreadMutexUnlock(&p->lock); @@ -322,7 +321,7 @@ int32_t streamSaveTaskCheckpointInfo(SStreamTask* p, int64_t checkpointId) { stDebug("vgId:%d s-task:%s level:%d open upstream inputQ, save status after checkpoint, checkpointId:%" PRId64 ", Ver(saved):%" PRId64 " currentVer:%" PRId64 ", status: normal, prev:%s", - vgId, id, p->info.taskLevel, checkpointId, p->chkInfo.checkpointVer, p->chkInfo.nextProcessVer, str); + vgId, id, p->info.taskLevel, checkpointId, p->chkInfo.checkpointVer, p->chkInfo.nextProcessVer, pState->name); // save the task if not sink task if (p->info.taskLevel != TASK_LEVEL__SINK) { diff --git a/source/libs/stream/src/streamDispatch.c b/source/libs/stream/src/streamDispatch.c index 00a8940b6a..a6d21e2625 100644 --- a/source/libs/stream/src/streamDispatch.c +++ b/source/libs/stream/src/streamDispatch.c @@ -696,10 +696,9 @@ int32_t streamDispatchScanHistoryFinishMsg(SStreamTask* pTask) { int32_t numOfVgs = taosArrayGetSize(vgInfo); pTask->notReadyTasks = numOfVgs; - char* p = NULL; - streamTaskGetStatus(pTask, &p); + SStreamTaskState* pState = streamTaskGetStatus(pTask); stDebug("s-task:%s send scan-history data complete msg to downstream (shuffle-dispatch) %d tasks, status:%s", - pTask->id.idStr, numOfVgs, p); + pTask->id.idStr, numOfVgs, pState->name); for (int32_t i = 0; i < numOfVgs; i++) { SVgroupInfo* pVgInfo = taosArrayGet(vgInfo, i); req.downstreamTaskId = pVgInfo->taskId; @@ -819,9 +818,9 @@ int32_t doDispatchScanHistoryFinishMsg(SStreamTask* pTask, const SStreamScanHist initRpcMsg(&msg, TDMT_VND_STREAM_SCAN_HISTORY_FINISH, buf, tlen + sizeof(SMsgHead)); tmsgSendReq(pEpSet, &msg); - char* p = NULL; - streamTaskGetStatus(pTask, &p); - stDebug("s-task:%s status:%s dispatch scan-history finish msg to taskId:0x%x (vgId:%d)", pTask->id.idStr, p, + + SStreamTaskState* pState = streamTaskGetStatus(pTask); + stDebug("s-task:%s status:%s dispatch scan-history finish msg to taskId:0x%x (vgId:%d)", pTask->id.idStr, pState->name, pReq->downstreamTaskId, vgId); return 0; } diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index d68288222c..d83d99f10f 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -24,12 +24,12 @@ static int32_t streamDoTransferStateToStreamTask(SStreamTask* pTask); bool streamTaskShouldStop(const SStreamTask* pTask) { - ETaskStatus s = streamTaskGetStatus(pTask, NULL); - return (s == TASK_STATUS__STOP) || (s == TASK_STATUS__DROPPING); + SStreamTaskState* pState = streamTaskGetStatus(pTask); + return (pState->state == TASK_STATUS__STOP) || (pState->state == TASK_STATUS__DROPPING); } bool streamTaskShouldPause(const SStreamTask* pTask) { - return (streamTaskGetStatus(pTask, NULL) == TASK_STATUS__PAUSE); + return (streamTaskGetStatus(pTask)->state == TASK_STATUS__PAUSE); } static int32_t doOutputResultBlockImpl(SStreamTask* pTask, SStreamDataBlock* pBlock) { @@ -344,11 +344,14 @@ int32_t streamDoTransferStateToStreamTask(SStreamTask* pTask) { streamMetaWUnLock(pMeta); return TSDB_CODE_STREAM_TASK_NOT_EXIST; } else { - stDebug("s-task:%s fill-history task end, update related stream task:%s info, transfer exec state", id, - pStreamTask->id.idStr); + double el = (taosGetTimestampMs() - pTask->execInfo.step2Start) / 1000.; + stDebug( + "s-task:%s fill-history task end, scal wal elapsed time:%.2fSec,update related stream task:%s info, transfer " + "exec state", + id, el, pStreamTask->id.idStr); } - ETaskStatus status = streamTaskGetStatus(pStreamTask, NULL); + ETaskStatus status = streamTaskGetStatus(pStreamTask)->state; STimeWindow* pTimeWindow = &pStreamTask->dataRange.window; // It must be halted for a source stream task, since when the related scan-history-data task start scan the history @@ -374,8 +377,10 @@ int32_t streamDoTransferStateToStreamTask(SStreamTask* pTask) { // In case of sink tasks, no need to halt them. // In case of source tasks and agg tasks, we should HALT them, and wait for them to be idle. And then, it's safe to // start the task state transfer procedure. - char* p = NULL; - status = streamTaskGetStatus(pStreamTask, &p); +// char* p = NULL; + SStreamTaskState* pState = streamTaskGetStatus(pStreamTask); + status = pState->state; + char* p = pState->name; if (status == TASK_STATUS__STOP || status == TASK_STATUS__DROPPING) { stError("s-task:%s failed to transfer state from fill-history task:%s, status:%s", id, pStreamTask->id.idStr, p); streamMetaReleaseTask(pMeta, pStreamTask); @@ -409,7 +414,7 @@ int32_t streamDoTransferStateToStreamTask(SStreamTask* pTask) { streamBuildAndSendDropTaskMsg(pTask->pMsgCb, pMeta->vgId, &pTask->id); // 5. save to disk - pStreamTask->status.taskStatus = streamTaskGetStatus(pStreamTask, NULL); + pStreamTask->status.taskStatus = streamTaskGetStatus(pStreamTask)->state; // 6. add empty delete block if ((pStreamTask->info.taskLevel == TASK_LEVEL__SOURCE) && taosQueueEmpty(pStreamTask->inputq.queue->pQueue)) { @@ -570,7 +575,7 @@ int32_t doStreamExecTask(SStreamTask* pTask) { int32_t blockSize = 0; int32_t numOfBlocks = 0; SStreamQueueItem* pInput = NULL; - if (streamTaskShouldStop(pTask) || (streamTaskGetStatus(pTask, NULL) == TASK_STATUS__UNINIT)) { + if (streamTaskShouldStop(pTask) || (streamTaskGetStatus(pTask)->state == TASK_STATUS__UNINIT)) { stDebug("s-task:%s stream task is stopped", id); break; } @@ -647,10 +652,9 @@ int32_t doStreamExecTask(SStreamTask* pTask) { if (type == STREAM_INPUT__CHECKPOINT) { // todo add lock - char* p = NULL; - ETaskStatus s = streamTaskGetStatus(pTask, &p); - if (s == TASK_STATUS__CK) { - stDebug("s-task:%s checkpoint block received, set status:%s", pTask->id.idStr, p); + SStreamTaskState* pState = streamTaskGetStatus(pTask); + if (pState->state == TASK_STATUS__CK) { + stDebug("s-task:%s checkpoint block received, set status:%s", pTask->id.idStr, pState->name); streamTaskBuildCheckpoint(pTask); } else { // todo refactor @@ -678,26 +682,27 @@ int32_t doStreamExecTask(SStreamTask* pTask) { // the task may be set dropping/stopping, while it is still in the task queue, therefore, the sched-status can not // be updated by tryExec function, therefore, the schedStatus will always be the TASK_SCHED_STATUS__WAITING. bool streamTaskIsIdle(const SStreamTask* pTask) { - ETaskStatus status = streamTaskGetStatus(pTask, NULL); + ETaskStatus status = streamTaskGetStatus(pTask)->state; return (pTask->status.schedStatus == TASK_SCHED_STATUS__INACTIVE || status == TASK_STATUS__STOP || status == TASK_STATUS__DROPPING); } bool streamTaskReadyToRun(const SStreamTask* pTask, char** pStatus) { - ETaskStatus st = streamTaskGetStatus(pTask, pStatus); + SStreamTaskState* pState = streamTaskGetStatus(pTask); + ETaskStatus st = pState->state; + *pStatus = pState->name; return (st == TASK_STATUS__READY || st == TASK_STATUS__SCAN_HISTORY || st == TASK_STATUS__CK); } static void doStreamExecTaskHelper(void* param, void* tmrId) { SStreamTask* pTask = (SStreamTask*)param; - char* p = NULL; - ETaskStatus status = streamTaskGetStatus(pTask, &p); - if (status == TASK_STATUS__DROPPING || status == TASK_STATUS__STOP) { + SStreamTaskState* p = streamTaskGetStatus(pTask); + if (p->state == TASK_STATUS__DROPPING || p->state == TASK_STATUS__STOP) { streamTaskSetSchedStatusInactive(pTask); int32_t ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1); - stDebug("s-task:%s status:%s not resume task, ref:%d", pTask->id.idStr, p, ref); + stDebug("s-task:%s status:%s not resume task, ref:%d", pTask->id.idStr, p->name, ref); streamMetaReleaseTask(pTask->pMeta, pTask); return; @@ -770,8 +775,7 @@ int32_t streamResumeTask(SStreamTask* pTask) { atomic_store_8(&pTask->status.schedStatus, TASK_SCHED_STATUS__INACTIVE); taosThreadMutexUnlock(&pTask->lock); - char* p = NULL; - streamTaskGetStatus(pTask, &p); + char* p = streamTaskGetStatus(pTask)->name; stDebug("s-task:%s exec completed, status:%s, sched-status:%d", id, p, pTask->status.schedStatus); return 0; } @@ -789,8 +793,7 @@ int32_t streamExecTask(SStreamTask* pTask) { if (schedStatus == TASK_SCHED_STATUS__WAITING) { streamResumeTask(pTask); } else { - char* p = NULL; - streamTaskGetStatus(pTask, &p); + char* p = streamTaskGetStatus(pTask)->name; stDebug("s-task:%s already started to exec by other thread, status:%s, sched-status:%d", id, p, pTask->status.schedStatus); } diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 8017a88dea..7ea42e62c1 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -1093,7 +1093,7 @@ static int32_t metaHeartbeatToMnodeImpl(SStreamMeta* pMeta) { STaskStatusEntry entry = { .id = *pId, - .status = streamTaskGetStatus(*pTask, NULL), + .status = streamTaskGetStatus(*pTask)->state, .nodeId = hbMsg.vgId, .stage = pMeta->stage, .inputQUsed = SIZE_IN_MiB(streamQueueGetItemSize((*pTask)->inputq.queue)), @@ -1362,13 +1362,13 @@ SArray* streamMetaSendMsgBeforeCloseTasks(SStreamMeta* pMeta) { } taosThreadMutexLock(&pTask->lock); - char* p = NULL; - ETaskStatus s = streamTaskGetStatus(pTask, &p); - if (s == TASK_STATUS__CK) { + + SStreamTaskState* pState = streamTaskGetStatus(pTask); + if (pState->state == TASK_STATUS__CK) { streamTaskSetCheckpointFailedId(pTask); stDebug("s-task:%s mark the checkpoint:%"PRId64" failed", pTask->id.idStr, pTask->chkInfo.checkpointingId); } else { - stDebug("s-task:%s status:%s not reset the checkpoint", pTask->id.idStr, p); + stDebug("s-task:%s status:%s not reset the checkpoint", pTask->id.idStr, pState->state); } taosThreadMutexUnlock(&pTask->lock); diff --git a/source/libs/stream/src/streamStart.c b/source/libs/stream/src/streamStart.c index 945d7844af..45a571e93c 100644 --- a/source/libs/stream/src/streamStart.c +++ b/source/libs/stream/src/streamStart.c @@ -48,15 +48,13 @@ static void tryLaunchHistoryTask(void* param, void* tmrId); static void doProcessDownstreamReadyRsp(SStreamTask* pTask); int32_t streamTaskSetReady(SStreamTask* pTask) { - char* p = NULL; - int32_t numOfDowns = streamTaskGetNumOfDownstream(pTask); - ETaskStatus status = streamTaskGetStatus(pTask, &p); + int32_t numOfDowns = streamTaskGetNumOfDownstream(pTask); + SStreamTaskState* p = streamTaskGetStatus(pTask); - if ((status == TASK_STATUS__SCAN_HISTORY/* || status == TASK_STATUS__STREAM_SCAN_HISTORY*/) && - pTask->info.taskLevel != TASK_LEVEL__SOURCE) { + if ((p->state == TASK_STATUS__SCAN_HISTORY) && pTask->info.taskLevel != TASK_LEVEL__SOURCE) { 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, p); + pTask->id.idStr, pTask->info.taskLevel, pTask->numOfWaitingUpstream, p->name); } ASSERT(pTask->status.downstreamReady == 0); @@ -93,12 +91,11 @@ static void doReExecScanhistory(void* param, void* tmrId) { SStreamTask* pTask = param; pTask->schedHistoryInfo.numOfTicks -= 1; - char* p = NULL; - ETaskStatus status = streamTaskGetStatus(pTask, &p); - if (status == TASK_STATUS__DROPPING || status == TASK_STATUS__STOP) { + SStreamTaskState* p = streamTaskGetStatus(pTask); + if (p->state == TASK_STATUS__DROPPING || p->state == TASK_STATUS__STOP) { streamMetaReleaseTask(pTask->pMeta, pTask); int32_t ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1); - stDebug("s-task:%s status:%s not start scan-history again, ref:%d", pTask->id.idStr, p, ref); + stDebug("s-task:%s status:%s not start scan-history again, ref:%d", pTask->id.idStr, p->name, ref); return; } @@ -155,7 +152,7 @@ static int32_t doStartScanHistoryTask(SStreamTask* pTask) { int32_t streamTaskStartScanHistory(SStreamTask* pTask) { int32_t level = pTask->info.taskLevel; - ETaskStatus status = streamTaskGetStatus(pTask, NULL); + ETaskStatus status = streamTaskGetStatus(pTask)->state; ASSERT(pTask->status.downstreamReady == 1 && ((status == TASK_STATUS__SCAN_HISTORY)/* || (status == TASK_STATUS__STREAM_SCAN_HISTORY)*/)); @@ -315,7 +312,7 @@ int32_t streamTaskCheckStatus(SStreamTask* pTask, int32_t upstreamTaskId, int32_ id, upstreamTaskId, vgId, stage, pInfo->stage); // record the checkpoint failure id and sent to mnode taosThreadMutexLock(&pTask->lock); - ETaskStatus status = streamTaskGetStatus(pTask, NULL); + ETaskStatus status = streamTaskGetStatus(pTask)->state; if (status == TASK_STATUS__CK) { streamTaskSetCheckpointFailedId(pTask); } @@ -325,7 +322,7 @@ int32_t streamTaskCheckStatus(SStreamTask* pTask, int32_t upstreamTaskId, int32_ if (pInfo->stage != stage) { taosThreadMutexLock(&pTask->lock); - ETaskStatus status = streamTaskGetStatus(pTask, NULL); + ETaskStatus status = streamTaskGetStatus(pTask)->state; if (status == TASK_STATUS__CK) { streamTaskSetCheckpointFailedId(pTask); } @@ -346,9 +343,8 @@ int32_t streamTaskOnNormalTaskReady(SStreamTask* pTask) { streamTaskSetReady(pTask); streamTaskSetRangeStreamCalc(pTask); - char* p = NULL; - ETaskStatus status = streamTaskGetStatus(pTask, &p); - ASSERT(status == TASK_STATUS__READY); + SStreamTaskState* p = streamTaskGetStatus(pTask); + ASSERT(p->state == TASK_STATUS__READY); if (pTask->info.taskLevel == TASK_LEVEL__SOURCE) { int64_t startVer = walReaderGetCurrentVer(pTask->exec.pWalReader); @@ -372,15 +368,14 @@ int32_t streamTaskOnScanhistoryTaskReady(SStreamTask* pTask) { streamTaskSetReady(pTask); streamTaskSetRangeStreamCalc(pTask); - char* p = NULL; - ETaskStatus status = streamTaskGetStatus(pTask, &p); - ASSERT(status == TASK_STATUS__SCAN_HISTORY/* || status == TASK_STATUS__STREAM_SCAN_HISTORY*/); + SStreamTaskState* p = streamTaskGetStatus(pTask); + ASSERT(p->state == TASK_STATUS__SCAN_HISTORY); if (pTask->info.fillHistory == 1) { - stDebug("s-task:%s fill-history task enters into scan-history data stage, status:%s", id, p); + stDebug("s-task:%s fill-history task enters into scan-history data stage, status:%s", id, p->name); streamTaskStartScanHistory(pTask); } else { - stDebug("s-task:%s scan wal data, status:%s", id, p); + stDebug("s-task:%s scan wal data, status:%s", id, p->name); } // NOTE: there will be an deadlock if launch fill history here. @@ -624,12 +619,11 @@ int32_t streamProcessScanHistoryFinishReq(SStreamTask* pTask, SStreamScanHistory ASSERT(taskLevel == TASK_LEVEL__AGG || taskLevel == TASK_LEVEL__SINK); const char* id = pTask->id.idStr; - char* p = NULL; - ETaskStatus status = streamTaskGetStatus(pTask, &p); + SStreamTaskState* p = streamTaskGetStatus(pTask); - if (status != TASK_STATUS__SCAN_HISTORY /*&& status != TASK_STATUS__STREAM_SCAN_HISTORY*/) { - stError("s-task:%s not in scan-history status, status:%s return upstream:0x%x scan-history finish directly", id, p, - pReq->upstreamTaskId); + if (p->state != TASK_STATUS__SCAN_HISTORY) { + stError("s-task:%s not in scan-history status, status:%s return upstream:0x%x scan-history finish directly", id, + p->name, pReq->upstreamTaskId); void* pBuf = NULL; int32_t len = 0; @@ -682,7 +676,7 @@ int32_t streamProcessScanHistoryFinishReq(SStreamTask* pTask, SStreamScanHistory } int32_t streamProcessScanHistoryFinishRsp(SStreamTask* pTask) { - ETaskStatus status = streamTaskGetStatus(pTask, NULL); + ETaskStatus status = streamTaskGetStatus(pTask)->state; // task restart now, not handle the scan-history finish rsp if (status == TASK_STATUS__UNINIT) { @@ -741,8 +735,7 @@ static void tryLaunchHistoryTask(void* param, void* tmrId) { ASSERT((*ppTask)->status.timerActive >= 1); if (streamTaskShouldStop(*ppTask)) { - char* p = NULL; - streamTaskGetStatus((*ppTask), &p); + char* p = streamTaskGetStatus(*ppTask)->name; 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, p, (*ppTask)->hTaskInfo.retryTimes, ref); @@ -781,10 +774,9 @@ static void tryLaunchHistoryTask(void* param, void* tmrId) { // abort the timer if intend to stop task SStreamTask* pHTask = streamMetaAcquireTask(pMeta, pHTaskInfo->id.streamId, pHTaskInfo->id.taskId); if (pHTask == NULL && (!streamTaskShouldStop(pTask))) { - char* p = NULL; + char* p = streamTaskGetStatus(pTask)->name; int32_t hTaskId = pHTaskInfo->id.taskId; - streamTaskGetStatus(pTask, &p); stDebug("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); @@ -878,7 +870,7 @@ int32_t streamLaunchFillHistoryTask(SStreamTask* pTask) { } int32_t streamTaskScanHistoryDataComplete(SStreamTask* pTask) { - if (streamTaskGetStatus(pTask, NULL) == TASK_STATUS__DROPPING) { + if (streamTaskGetStatus(pTask)->state == TASK_STATUS__DROPPING) { return 0; } @@ -1058,23 +1050,21 @@ void streamTaskPause(SStreamTask* pTask, SStreamMeta* pMeta) { } void streamTaskResume(SStreamTask* pTask) { - char* p = NULL; - ETaskStatus status = streamTaskGetStatus(pTask, &p); + SStreamTaskState* p = streamTaskGetStatus(pTask); SStreamMeta* pMeta = pTask->pMeta; - if (status == TASK_STATUS__PAUSE || status == TASK_STATUS__HALT) { + if (p->state == TASK_STATUS__PAUSE || p->state == TASK_STATUS__HALT) { streamTaskRestoreStatus(pTask); - char* pNew = NULL; - streamTaskGetStatus(pTask, &pNew); - if (status == TASK_STATUS__PAUSE) { + char* pNew = streamTaskGetStatus(pTask)->name; + if (p->state == TASK_STATUS__PAUSE) { int32_t num = atomic_sub_fetch_32(&pMeta->numOfPausedTasks, 1); - stInfo("s-task:%s status:%s resume from %s, paused task(s):%d", pTask->id.idStr, pNew, p, num); + stInfo("s-task:%s status:%s resume from %s, paused task(s):%d", pTask->id.idStr, pNew, p->name, num); } else { - stInfo("s-task:%s status:%s resume from %s", pTask->id.idStr, pNew, p); + stInfo("s-task:%s status:%s resume from %s", pTask->id.idStr, pNew, p->name); } } else { - stDebug("s-task:%s status:%s not in pause/halt status, no need to resume", pTask->id.idStr, p); + stDebug("s-task:%s status:%s not in pause/halt status, no need to resume", pTask->id.idStr, p->name); } } diff --git a/source/libs/stream/src/streamTask.c b/source/libs/stream/src/streamTask.c index 32fd5cbb39..c14d355dc8 100644 --- a/source/libs/stream/src/streamTask.c +++ b/source/libs/stream/src/streamTask.c @@ -309,7 +309,9 @@ void tFreeStreamTask(SStreamTask* pTask) { ETaskStatus status1 = TASK_STATUS__UNINIT; taosThreadMutexLock(&pTask->lock); if (pTask->status.pSM != NULL) { - status1 = streamTaskGetStatus(pTask, &p); + SStreamTaskState* pStatus = streamTaskGetStatus(pTask); + p = pStatus->name; + status1 = pStatus->state; } taosThreadMutexUnlock(&pTask->lock); diff --git a/source/libs/stream/src/streamTaskSm.c b/source/libs/stream/src/streamTaskSm.c index d785932109..819cdbf374 100644 --- a/source/libs/stream/src/streamTaskSm.c +++ b/source/libs/stream/src/streamTaskSm.c @@ -65,8 +65,7 @@ static STaskStateTrans createStateTransform(ETaskStatus current, ETaskStatus nex static int32_t dummyFn(SStreamTask* UNUSED_PARAM(p)) { return TSDB_CODE_SUCCESS; } static int32_t attachEvent(SStreamTask* pTask, SAttachedEventInfo* pEvtInfo) { - char* p = NULL; - streamTaskGetStatus(pTask, &p); + char* p = streamTaskGetStatus(pTask)->name; stDebug("s-task:%s status:%s attach event:%s required status:%s, since not allowed to handle it", pTask->id.idStr, p, GET_EVT_NAME(pEvtInfo->event), StreamTaskStatusList[pEvtInfo->status].name); @@ -275,7 +274,7 @@ static int32_t doHandleEvent(SStreamTaskSM* pSM, EStreamTaskEvent event, STaskSt while (1) { // wait for the task to be here taosThreadMutexLock(&pTask->lock); - ETaskStatus s = streamTaskGetStatus(pTask, NULL); + ETaskStatus s = streamTaskGetStatus(pTask)->state; taosThreadMutexUnlock(&pTask->lock); if ((s == pTrans->next.state) && (pSM->prev.evt == pTrans->event)) {// this event has been handled already @@ -400,12 +399,8 @@ int32_t streamTaskOnHandleEventSuccess(SStreamTaskSM* pSM, EStreamTaskEvent even return TSDB_CODE_SUCCESS; } -ETaskStatus streamTaskGetStatus(const SStreamTask* pTask, char** pStr) { - SStreamTaskState s = pTask->status.pSM->current; // copy one obj in case of multi-thread environment - if (pStr != NULL) { - *pStr = s.name; - } - return s.state; +SStreamTaskState* streamTaskGetStatus(const SStreamTask* pTask) { + return &pTask->status.pSM->current; // copy one obj in case of multi-thread environment } const char* streamTaskGetStatusStr(ETaskStatus status) { From fa9df2a49591987e1c1dda5308945b0ecf91d6fa Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 4 Jan 2024 18:48:19 +0800 Subject: [PATCH 011/102] refactor: do some internal refactor. --- include/libs/stream/tstream.h | 15 ++++++++++----- 1 file changed, 10 insertions(+), 5 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 79fc2a87a7..47c5149795 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -730,6 +730,11 @@ typedef struct SStreamTaskNodeUpdateMsg { int32_t tEncodeStreamTaskUpdateMsg(SEncoder* pEncoder, const SStreamTaskNodeUpdateMsg* pMsg); int32_t tDecodeStreamTaskUpdateMsg(SDecoder* pDecoder, SStreamTaskNodeUpdateMsg* pMsg); +typedef struct SStreamTaskState { + ETaskStatus state; + char* name; +} SStreamTaskState; + typedef struct { int64_t streamId; int32_t downstreamTaskId; @@ -773,11 +778,11 @@ bool streamTaskShouldPause(const SStreamTask* pStatus); bool streamTaskIsIdle(const SStreamTask* pTask); bool streamTaskReadyToRun(const SStreamTask* pTask, char** pStatus); -char* createStreamTaskIdStr(int64_t streamId, int32_t taskId); -ETaskStatus streamTaskGetStatus(const SStreamTask* pTask, char** pStr); -const char* streamTaskGetStatusStr(ETaskStatus status); -void streamTaskResetStatus(SStreamTask* pTask); -void streamTaskSetStatusReady(SStreamTask* pTask); +char* createStreamTaskIdStr(int64_t streamId, int32_t taskId); +SStreamTaskState* streamTaskGetStatus(const SStreamTask* pTask); +const char* streamTaskGetStatusStr(ETaskStatus status); +void streamTaskResetStatus(SStreamTask* pTask); +void streamTaskSetStatusReady(SStreamTask* pTask); void initRpcMsg(SRpcMsg* pMsg, int32_t msgType, void* pCont, int32_t contLen); From 33f698e9265b2cbf041b575e0312cc152539bf9b Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 4 Jan 2024 19:09:45 +0800 Subject: [PATCH 012/102] fix(stream): remove invalid assert and add check for task status before launching related fill-history task. --- source/dnode/vnode/src/tq/tq.c | 2 +- source/libs/stream/src/streamStart.c | 27 +++++++++++++++++---------- source/libs/stream/src/streamTaskSm.c | 1 - 3 files changed, 18 insertions(+), 12 deletions(-) diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 203c34f1ce..8cd16bbe60 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1160,7 +1160,7 @@ int32_t tqProcessTaskResumeImpl(STQ* pTq, SStreamTask* pTask, int64_t sversion, } else if (status == TASK_STATUS__UNINIT) { // todo: fill-history task init ? if (pTask->info.fillHistory == 0) { - EStreamTaskEvent event = /*HAS_RELATED_FILLHISTORY_TASK(pTask) ? TASK_EVENT_INIT_STREAM_SCANHIST : */TASK_EVENT_INIT; + EStreamTaskEvent event = TASK_EVENT_INIT; streamTaskHandleEvent(pTask->status.pSM, event); } } diff --git a/source/libs/stream/src/streamStart.c b/source/libs/stream/src/streamStart.c index 45a571e93c..412f1a452c 100644 --- a/source/libs/stream/src/streamStart.c +++ b/source/libs/stream/src/streamStart.c @@ -388,12 +388,7 @@ int32_t streamTaskOnScanhistoryTaskReady(SStreamTask* pTask) { } void doProcessDownstreamReadyRsp(SStreamTask* pTask) { - EStreamTaskEvent event; - if (pTask->info.fillHistory == 0) { - event = /*HAS_RELATED_FILLHISTORY_TASK(pTask) ? TASK_EVENT_INIT_STREAM_SCANHIST : */TASK_EVENT_INIT; - } else { - event = TASK_EVENT_INIT_SCANHIST; - } + EStreamTaskEvent event = (pTask->info.fillHistory == 0) ? TASK_EVENT_INIT : TASK_EVENT_INIT_SCANHIST; streamTaskOnHandleEventSuccess(pTask->status.pSM, event); @@ -820,11 +815,23 @@ SLaunchHTaskInfo* createHTaskLaunchInfo(SStreamMeta* pMeta, int64_t streamId, in // an fill history task needs to be started. int32_t streamLaunchFillHistoryTask(SStreamTask* pTask) { SStreamMeta* pMeta = pTask->pMeta; - int32_t hTaskId = pTask->hTaskInfo.id.taskId; - ASSERT((hTaskId != 0) && (pTask->status.downstreamReady == 1)); - stDebug("s-task:%s start to launch related fill-history task:0x%" PRIx64 "-0x%x", pTask->id.idStr, - pTask->hTaskInfo.id.streamId, hTaskId); + int64_t streamId = pTask->hTaskInfo.id.streamId; + int32_t hTaskId = pTask->hTaskInfo.id.taskId; + ASSERT(hTaskId != 0); + + SStreamTaskState* pStatus = streamTaskGetStatus(pTask); + if (pStatus->state != TASK_STATUS__READY) { + STaskExecStatisInfo* pInfo = &pTask->execInfo; + stDebug("s-task:%s not launch related fill-history task:0x%" PRIx64 "-0x%x, status:%s", pTask->id.idStr, + pTask->hTaskInfo.id.streamId, hTaskId); + + streamMetaUpdateTaskDownstreamStatus(pMeta, streamId, hTaskId, pInfo->init, pInfo->start, false); + return -1;// todo set the correct error code + } else { + stDebug("s-task:%s start to launch related fill-history task:0x%" PRIx64 "-0x%x", pTask->id.idStr, + pTask->hTaskInfo.id.streamId, hTaskId); + } // Set the execute conditions, including the query time window and the version range SStreamTask** pHTask = taosHashGet(pMeta->pTasksMap, &pTask->hTaskInfo.id, sizeof(pTask->hTaskInfo.id)); diff --git a/source/libs/stream/src/streamTaskSm.c b/source/libs/stream/src/streamTaskSm.c index 819cdbf374..dcfe4ba848 100644 --- a/source/libs/stream/src/streamTaskSm.c +++ b/source/libs/stream/src/streamTaskSm.c @@ -38,7 +38,6 @@ SStreamEventInfo StreamTaskEventList[12] = { {.event = 0, .name = ""}, // dummy event, place holder {.event = TASK_EVENT_INIT, .name = "initialize"}, {.event = TASK_EVENT_INIT_SCANHIST, .name = "scan-history-init"}, -// {.event = TASK_EVENT_INIT_STREAM_SCANHIST, .name = "stream-scan-history-init"}, {.event = TASK_EVENT_SCANHIST_DONE, .name = "scan-history-completed"}, {.event = TASK_EVENT_STOP, .name = "stopping"}, {.event = TASK_EVENT_GEN_CHECKPOINT, .name = "checkpoint"}, From 6286e42f5c561aa6f8b42def445feee5ca505bec Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 4 Jan 2024 19:11:50 +0800 Subject: [PATCH 013/102] refactor: do some internal refactor. --- source/libs/stream/src/streamTaskSm.c | 16 ++-------------- 1 file changed, 2 insertions(+), 14 deletions(-) diff --git a/source/libs/stream/src/streamTaskSm.c b/source/libs/stream/src/streamTaskSm.c index dcfe4ba848..f0dcc75c4c 100644 --- a/source/libs/stream/src/streamTaskSm.c +++ b/source/libs/stream/src/streamTaskSm.c @@ -108,12 +108,12 @@ int32_t streamTaskKeepCurrentVerInWal(SStreamTask* pTask) { // todo check rsp code for handle Event:TASK_EVENT_SCANHIST_DONE static bool isInvalidStateTransfer(ETaskStatus state, const EStreamTaskEvent event) { - if (/*event == TASK_EVENT_INIT_STREAM_SCANHIST || */event == TASK_EVENT_INIT || event == TASK_EVENT_INIT_SCANHIST) { + if (event == TASK_EVENT_INIT || event == TASK_EVENT_INIT_SCANHIST) { return (state != TASK_STATUS__UNINIT); } if (event == TASK_EVENT_SCANHIST_DONE) { - return (state != TASK_STATUS__SCAN_HISTORY/* && state != TASK_STATUS__STREAM_SCAN_HISTORY*/); + return (state != TASK_STATUS__SCAN_HISTORY); } if (event == TASK_EVENT_GEN_CHECKPOINT) { @@ -476,14 +476,10 @@ void doInitStateTransferTable(void) { taosArrayPush(streamTaskSMTrans, &trans); trans = createStateTransform(TASK_STATUS__UNINIT, TASK_STATUS__SCAN_HISTORY, TASK_EVENT_INIT_SCANHIST, streamTaskInitStatus, streamTaskOnScanhistoryTaskReady, false, false); taosArrayPush(streamTaskSMTrans, &trans); -// trans = createStateTransform(TASK_STATUS__UNINIT, TASK_STATUS__STREAM_SCAN_HISTORY, TASK_EVENT_INIT_STREAM_SCANHIST, streamTaskInitStatus, streamTaskOnScanhistoryTaskReady, false, false); -// taosArrayPush(streamTaskSMTrans, &trans); // scan-history related event trans = createStateTransform(TASK_STATUS__SCAN_HISTORY, TASK_STATUS__READY, TASK_EVENT_SCANHIST_DONE, NULL, NULL, NULL, true); taosArrayPush(streamTaskSMTrans, &trans); -// trans = createStateTransform(TASK_STATUS__STREAM_SCAN_HISTORY, TASK_STATUS__READY, TASK_EVENT_SCANHIST_DONE, NULL, NULL, NULL, true); -// taosArrayPush(streamTaskSMTrans, &trans); // halt stream task, from other task status trans = createStateTransform(TASK_STATUS__READY, TASK_STATUS__HALT, TASK_EVENT_HALT, NULL, streamTaskKeepCurrentVerInWal, NULL, true); @@ -493,8 +489,6 @@ void doInitStateTransferTable(void) { SAttachedEventInfo info = {.status = TASK_STATUS__READY, .event = TASK_EVENT_HALT}; -// trans = createStateTransform(TASK_STATUS__STREAM_SCAN_HISTORY, TASK_STATUS__HALT, TASK_EVENT_HALT, NULL, streamTaskKeepCurrentVerInWal, &info, true); -// taosArrayPush(streamTaskSMTrans, &trans); trans = createStateTransform(TASK_STATUS__CK, TASK_STATUS__HALT, TASK_EVENT_HALT, NULL, streamTaskKeepCurrentVerInWal, &info, true); taosArrayPush(streamTaskSMTrans, &trans); trans = createStateTransform(TASK_STATUS__PAUSE, TASK_STATUS__HALT, TASK_EVENT_HALT, NULL, streamTaskKeepCurrentVerInWal, NULL, true); @@ -513,8 +507,6 @@ void doInitStateTransferTable(void) { taosArrayPush(streamTaskSMTrans, &trans); info = (SAttachedEventInfo){.status = TASK_STATUS__READY, .event = TASK_EVENT_PAUSE}; -// trans = createStateTransform(TASK_STATUS__STREAM_SCAN_HISTORY, TASK_STATUS__PAUSE, TASK_EVENT_PAUSE, NULL, NULL, &info, true); -// taosArrayPush(streamTaskSMTrans, &trans); trans = createStateTransform(TASK_STATUS__CK, TASK_STATUS__PAUSE, TASK_EVENT_PAUSE, NULL, NULL, &info, true); taosArrayPush(streamTaskSMTrans, &trans); trans = createStateTransform(TASK_STATUS__HALT, TASK_STATUS__PAUSE, TASK_EVENT_PAUSE, NULL, NULL, &info, true); @@ -548,8 +540,6 @@ void doInitStateTransferTable(void) { taosArrayPush(streamTaskSMTrans, &trans); trans = createStateTransform(TASK_STATUS__CK, TASK_STATUS__STOP, TASK_EVENT_STOP, NULL, NULL, NULL, true); taosArrayPush(streamTaskSMTrans, &trans); -// trans = createStateTransform(TASK_STATUS__STREAM_SCAN_HISTORY, TASK_STATUS__STOP, TASK_EVENT_STOP, NULL, NULL, NULL, true); -// taosArrayPush(streamTaskSMTrans, &trans); // dropping related event trans = createStateTransform(TASK_STATUS__READY, TASK_STATUS__DROPPING, TASK_EVENT_DROPPING, NULL, NULL, NULL, true); @@ -568,7 +558,5 @@ void doInitStateTransferTable(void) { taosArrayPush(streamTaskSMTrans, &trans); trans = createStateTransform(TASK_STATUS__CK, TASK_STATUS__DROPPING, TASK_EVENT_DROPPING, streamTaskSendTransSuccessMsg, NULL, NULL, true); taosArrayPush(streamTaskSMTrans, &trans); -// trans = createStateTransform(TASK_STATUS__STREAM_SCAN_HISTORY, TASK_STATUS__DROPPING, TASK_EVENT_DROPPING, NULL, NULL, NULL, true); -// taosArrayPush(streamTaskSMTrans, &trans); } //clang-format on \ No newline at end of file From 4df83e92cd2d48abdcae503fcf56033203f5b7c0 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 4 Jan 2024 19:14:24 +0800 Subject: [PATCH 014/102] fix(stream): fix syntax error. --- 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 7ea42e62c1..16a9fb3274 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -1368,7 +1368,7 @@ SArray* streamMetaSendMsgBeforeCloseTasks(SStreamMeta* pMeta) { streamTaskSetCheckpointFailedId(pTask); stDebug("s-task:%s mark the checkpoint:%"PRId64" failed", pTask->id.idStr, pTask->chkInfo.checkpointingId); } else { - stDebug("s-task:%s status:%s not reset the checkpoint", pTask->id.idStr, pState->state); + stDebug("s-task:%s status:%s not reset the checkpoint", pTask->id.idStr, pState->name); } taosThreadMutexUnlock(&pTask->lock); From 14e90eb16eb6c1b6ee1e7ab67562c861c3d07263 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 4 Jan 2024 19:15:51 +0800 Subject: [PATCH 015/102] fix(stream): fix syntax error. --- source/libs/stream/src/streamStart.c | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/source/libs/stream/src/streamStart.c b/source/libs/stream/src/streamStart.c index 412f1a452c..5cae438eb1 100644 --- a/source/libs/stream/src/streamStart.c +++ b/source/libs/stream/src/streamStart.c @@ -355,7 +355,7 @@ int32_t streamTaskOnNormalTaskReady(SStreamTask* pTask) { stDebug("s-task:%s no need to scan-history data, status:%s, sched-status:%d, ready for data from wal ver:%" PRId64, id, p, pTask->status.schedStatus, startVer); } else { - stDebug("s-task:%s level:%d status:%s sched-status:%d", id, pTask->info.taskLevel, p, pTask->status.schedStatus); + stDebug("s-task:%s level:%d status:%s sched-status:%d", id, pTask->info.taskLevel, p->name, pTask->status.schedStatus); } return TSDB_CODE_SUCCESS; @@ -824,7 +824,7 @@ int32_t streamLaunchFillHistoryTask(SStreamTask* pTask) { if (pStatus->state != TASK_STATUS__READY) { STaskExecStatisInfo* pInfo = &pTask->execInfo; stDebug("s-task:%s not launch related fill-history task:0x%" PRIx64 "-0x%x, status:%s", pTask->id.idStr, - pTask->hTaskInfo.id.streamId, hTaskId); + pTask->hTaskInfo.id.streamId, hTaskId, pStatus->name); streamMetaUpdateTaskDownstreamStatus(pMeta, streamId, hTaskId, pInfo->init, pInfo->start, false); return -1;// todo set the correct error code From 356b7093f50d80c6a94227764bfaed4bcb1092d9 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 4 Jan 2024 19:18:10 +0800 Subject: [PATCH 016/102] fix(stream): fix syntax error. --- source/libs/stream/src/streamStart.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/libs/stream/src/streamStart.c b/source/libs/stream/src/streamStart.c index 5cae438eb1..a40fe0a2c2 100644 --- a/source/libs/stream/src/streamStart.c +++ b/source/libs/stream/src/streamStart.c @@ -353,7 +353,7 @@ int32_t streamTaskOnNormalTaskReady(SStreamTask* pTask) { } stDebug("s-task:%s no need to scan-history data, status:%s, sched-status:%d, ready for data from wal ver:%" PRId64, - id, p, pTask->status.schedStatus, startVer); + id, p->name, pTask->status.schedStatus, startVer); } else { stDebug("s-task:%s level:%d status:%s sched-status:%d", id, pTask->info.taskLevel, p->name, pTask->status.schedStatus); } From 0c8d5e3586ecebb7618733b2f85eb133e96a98b7 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 4 Jan 2024 19:20:50 +0800 Subject: [PATCH 017/102] fix(stream): fix syntax error. --- source/libs/stream/src/streamStart.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/libs/stream/src/streamStart.c b/source/libs/stream/src/streamStart.c index a40fe0a2c2..0dcc94a530 100644 --- a/source/libs/stream/src/streamStart.c +++ b/source/libs/stream/src/streamStart.c @@ -63,7 +63,7 @@ int32_t streamTaskSetReady(SStreamTask* pTask) { 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, numOfDowns, el, p); + pTask->id.idStr, numOfDowns, el, p->name); return TSDB_CODE_SUCCESS; } From 1912de9b498ab510d84ee4e6ba7ac18243efda73 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Fri, 5 Jan 2024 14:49:33 +0800 Subject: [PATCH 018/102] enh(stream): scan wal by using timer to trigger it. --- include/libs/stream/tstream.h | 7 +- source/dnode/vnode/src/inc/tq.h | 1 + source/dnode/vnode/src/tq/tq.c | 26 +++++-- source/dnode/vnode/src/tq/tqStreamTask.c | 98 ++++++++++++++++++------ source/libs/stream/src/streamMeta.c | 2 +- 5 files changed, 99 insertions(+), 35 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 47c5149795..2998c6d7b1 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -482,6 +482,11 @@ typedef struct STaskUpdateInfo { int32_t transId; } STaskUpdateInfo; +typedef struct SScanWalInfo { + int32_t scanCounter; + tmr_h scanTimer; +} SScanWalInfo; + // meta typedef struct SStreamMeta { char* path; @@ -499,7 +504,7 @@ typedef struct SStreamMeta { bool sendMsgBeforeClosing; // send hb to mnode before close all tasks when switch to follower. STaskStartInfo startInfo; TdThreadRwlock lock; - int32_t walScanCounter; + SScanWalInfo scanInfo; void* streamBackend; int64_t streamBackendRid; SHashObj* pTaskDbUnique; diff --git a/source/dnode/vnode/src/inc/tq.h b/source/dnode/vnode/src/inc/tq.h index 0ef29fcb3a..e4c673f533 100644 --- a/source/dnode/vnode/src/inc/tq.h +++ b/source/dnode/vnode/src/inc/tq.h @@ -42,6 +42,7 @@ extern "C" { // clang-format on typedef struct STqOffsetStore STqOffsetStore; +extern void* tqTimer; #define IS_OFFSET_RESET_TYPE(_t) ((_t) < 0) diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 8cd16bbe60..33e9d97f8d 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -19,13 +19,25 @@ // 0: not init // 1: already inited -// 2: wait to be inited or cleaup +// 2: wait to be inited or cleanup static int32_t tqInitialize(STQ* pTq); static FORCE_INLINE bool tqIsHandleExec(STqHandle* pHandle) { return TMQ_HANDLE_STATUS_EXEC == pHandle->status; } static FORCE_INLINE void tqSetHandleExec(STqHandle* pHandle) { pHandle->status = TMQ_HANDLE_STATUS_EXEC; } static FORCE_INLINE void tqSetHandleIdle(STqHandle* pHandle) { pHandle->status = TMQ_HANDLE_STATUS_IDLE; } +int32_t tqTimerInit() { + tqTimer = taosTmrInit(100, 100, 1000, "TQ"); + if (tqTimer == NULL) { + return -1; + } + return 0; +} + +void tqTimerCleanUp() { + taosTmrCleanUp(tqTimer); +} + void tqDestroyTqHandle(void* data) { STqHandle* pData = (STqHandle*)data; qDestroyTask(pData->execHandle.task); @@ -106,6 +118,7 @@ int32_t tqInitialize(STQ* pTq) { return -1; } + tqTimerInit(); return 0; } @@ -136,6 +149,8 @@ void tqClose(STQ* pTq) { taosMemoryFree(pTq->path); tqMetaClose(pTq); streamMetaClose(pTq->pStreamMeta); + tqTimerCleanUp(); + qDebug("end to close tq"); taosMemoryFree(pTq); } @@ -1055,7 +1070,8 @@ int32_t tqProcessTaskScanHistoryFinishRsp(STQ* pTq, SRpcMsg* pMsg) { int32_t tqProcessTaskRunReq(STQ* pTq, SRpcMsg* pMsg) { SStreamTaskRunReq* pReq = pMsg->pCont; - if (pReq->reqType == STREAM_EXEC_T_EXTRACT_WAL_DATA) { // all tasks are extracted submit data from the wal + // extracted submit data from wal files for all tasks + if (pReq->reqType == STREAM_EXEC_T_EXTRACT_WAL_DATA) { tqScanWal(pTq); return 0; } @@ -1351,14 +1367,8 @@ int32_t tqProcessTaskDropHTask(STQ* pTq, SRpcMsg* pMsg) { } taosThreadMutexLock(&pTask->lock); - ETaskStatus status = streamTaskGetStatus(pTask)->state; -// if (status == TASK_STATUS__STREAM_SCAN_HISTORY) { -// streamTaskHandleEvent(pTask->status.pSM, TASK_EVENT_SCANHIST_DONE); -// } - SStreamTaskId id = {.streamId = pTask->hTaskInfo.id.streamId, .taskId = pTask->hTaskInfo.id.taskId}; streamBuildAndSendDropTaskMsg(pTask->pMsgCb, pMeta->vgId, &id); - taosThreadMutexUnlock(&pTask->lock); // clear the scheduler status diff --git a/source/dnode/vnode/src/tq/tqStreamTask.c b/source/dnode/vnode/src/tq/tqStreamTask.c index 12f540b9cf..95a4474460 100644 --- a/source/dnode/vnode/src/tq/tqStreamTask.c +++ b/source/dnode/vnode/src/tq/tqStreamTask.c @@ -19,11 +19,14 @@ #define MAX_REPEAT_SCAN_THRESHOLD 3 #define SCAN_WAL_IDLE_DURATION 100 +void* tqTimer = NULL; + static int32_t doScanWalForAllTasks(SStreamMeta* pStreamMeta, bool* pScanIdle); static int32_t setWalReaderStartOffset(SStreamTask* pTask, int32_t vgId); static bool handleFillhistoryScanComplete(SStreamTask* pTask, int64_t ver); static bool taskReadyForDataFromWal(SStreamTask* pTask); static bool doPutDataIntoInputQFromWal(SStreamTask* pTask, int64_t maxVer, int32_t* numOfItems); +static int32_t tqScanWalInFuture(STQ* pTq, int32_t numOfTasks, int32_t idleDuration); // extract data blocks(submit/delete) from WAL, and add them into the input queue for all the sources tasks. int32_t tqScanWal(STQ* pTq) { @@ -31,33 +34,78 @@ int32_t tqScanWal(STQ* pTq) { SStreamMeta* pMeta = pTq->pStreamMeta; int64_t st = taosGetTimestampMs(); - while (1) { - tqDebug("vgId:%d continue check if data in wal are available, walScanCounter:%d", vgId, pMeta->walScanCounter); + tqDebug("vgId:%d continue to check if data in wal are available, scanCounter:%d", vgId, pMeta->scanInfo.scanCounter); - // check all tasks - bool shouldIdle = true; - doScanWalForAllTasks(pMeta, &shouldIdle); + // check all tasks + int32_t numOfTasks = 0; + bool shouldIdle = true; + doScanWalForAllTasks(pMeta, &shouldIdle); - streamMetaWLock(pMeta); - int32_t times = (--pMeta->walScanCounter); - ASSERT(pMeta->walScanCounter >= 0); - streamMetaWUnLock(pMeta); + streamMetaWLock(pMeta); + int32_t times = (--pMeta->scanInfo.scanCounter); + ASSERT(pMeta->scanInfo.scanCounter >= 0); - if (times > 0) { - tqDebug("vgId:%d scan wal for stream tasks for %d times in %dms", vgId, times, SCAN_WAL_IDLE_DURATION); - } else { // times <= 0 - break; - } - - // todo: remove the sleep - taosMsleep(SCAN_WAL_IDLE_DURATION); - } + numOfTasks = taosArrayGetSize(pMeta->pTaskList); + streamMetaWUnLock(pMeta); int64_t el = (taosGetTimestampMs() - st); tqDebug("vgId:%d scan wal for stream tasks completed, elapsed time:%" PRId64 " ms", vgId, el); + + if (times > 0) { + tqDebug("vgId:%d scan wal for stream tasks for %d times in %dms", vgId, times, SCAN_WAL_IDLE_DURATION); + tqScanWalInFuture(pTq, numOfTasks, SCAN_WAL_IDLE_DURATION); + } return 0; } +typedef struct SBuildScanWalMsgParam { + STQ* pTq; + int32_t numOfTasks; +} SBuildScanWalMsgParam; + +static void doStartScanWal(void* param, void* tmrId) { + SBuildScanWalMsgParam* pParam = (SBuildScanWalMsgParam*) param; + + int32_t vgId = pParam->pTq->pStreamMeta->vgId; + + SStreamTaskRunReq* pRunReq = rpcMallocCont(sizeof(SStreamTaskRunReq)); + if (pRunReq == NULL) { + taosMemoryFree(pParam); + 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()); + return; + } + + tqDebug("vgId:%d create msg to start wal scan, numOfTasks:%d, vnd restored:%d", vgId, pParam->numOfTasks, + pParam->pTq->pVnode->restored); + + pRunReq->head.vgId = vgId; + pRunReq->streamId = 0; + pRunReq->taskId = 0; + pRunReq->reqType = STREAM_EXEC_T_EXTRACT_WAL_DATA; + + SRpcMsg msg = {.msgType = TDMT_STREAM_TASK_RUN, .pCont = pRunReq, .contLen = sizeof(SStreamTaskRunReq)}; + tmsgPutToQueue(&pParam->pTq->pVnode->msgCb, STREAM_QUEUE, &msg); + + taosMemoryFree(pParam); +} + +int32_t tqScanWalInFuture(STQ* pTq, int32_t numOfTasks, int32_t idleDuration) { + SStreamMeta* pMeta = pTq->pStreamMeta; + + SBuildScanWalMsgParam* pParam = taosMemoryMalloc(sizeof(SBuildScanWalMsgParam)); + + pParam->pTq = pTq; + pParam->numOfTasks = numOfTasks; + if (pMeta->scanInfo.scanTimer == NULL) { + pMeta->scanInfo.scanTimer = taosTmrStart(doStartScanWal, idleDuration, pParam, tqTimer); + } else { + taosTmrReset(doStartScanWal, idleDuration, pParam, tqTimer, &pMeta->scanInfo.scanTimer); + } + + return TSDB_CODE_SUCCESS; +} + int32_t tqScanWalAsync(STQ* pTq, bool ckPause) { int32_t vgId = TD_VID(pTq->pVnode); SStreamMeta* pMeta = pTq->pStreamMeta; @@ -78,23 +126,23 @@ int32_t tqScanWalAsync(STQ* pTq, bool ckPause) { return 0; } - pMeta->walScanCounter += 1; - if (pMeta->walScanCounter > MAX_REPEAT_SCAN_THRESHOLD) { - pMeta->walScanCounter = MAX_REPEAT_SCAN_THRESHOLD; + pMeta->scanInfo.scanCounter += 1; + if (pMeta->scanInfo.scanCounter > MAX_REPEAT_SCAN_THRESHOLD) { + pMeta->scanInfo.scanCounter = MAX_REPEAT_SCAN_THRESHOLD; } - if (pMeta->walScanCounter > 1) { - tqDebug("vgId:%d wal read task has been launched, remain scan times:%d", vgId, pMeta->walScanCounter); + if (pMeta->scanInfo.scanCounter > 1) { + tqDebug("vgId:%d wal read task has been launched, remain scan times:%d", vgId, pMeta->scanInfo.scanCounter); streamMetaWUnLock(pMeta); return 0; } - int32_t numOfPauseTasks = pTq->pStreamMeta->numOfPausedTasks; + int32_t numOfPauseTasks = pMeta->numOfPausedTasks; if (ckPause && numOfTasks == numOfPauseTasks) { tqDebug("vgId:%d ignore all submit, all streams had been paused, reset the walScanCounter", vgId); // reset the counter value, since we do not launch the scan wal operation. - pMeta->walScanCounter = 0; + pMeta->scanInfo.scanCounter = 0; streamMetaWUnLock(pMeta); return 0; } diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 16a9fb3274..bf97bee5f2 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -359,7 +359,7 @@ SStreamMeta* streamMetaOpen(const char* path, void* ahandle, FTaskExpand expandF goto _err; } - pMeta->walScanCounter = 0; + pMeta->scanInfo.scanCounter = 0; pMeta->vgId = vgId; pMeta->ahandle = ahandle; pMeta->expandFunc = expandFunc; From 0c4b91dc7233adeb662282c086f85bd70ec9001f Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Fri, 5 Jan 2024 17:55:39 +0800 Subject: [PATCH 019/102] enh(stream): start task async. --- include/dnode/vnode/tqCommon.h | 4 +- include/libs/stream/tstream.h | 8 +-- source/dnode/snode/src/snode.c | 2 +- source/dnode/vnode/src/tq/tq.c | 3 +- source/dnode/vnode/src/tqCommon/tqCommon.c | 57 +++++++++++++++++----- source/libs/stream/src/streamMeta.c | 36 ++++++++++++++ 6 files changed, 91 insertions(+), 19 deletions(-) diff --git a/include/dnode/vnode/tqCommon.h b/include/dnode/vnode/tqCommon.h index da2325f006..6c84c3ea09 100644 --- a/include/dnode/vnode/tqCommon.h +++ b/include/dnode/vnode/tqCommon.h @@ -18,6 +18,7 @@ // message process int32_t tqStreamTaskStartAsync(SStreamMeta* pMeta, SMsgCb* cb, bool restart); +int32_t tqStreamOneTaskStartAsync(SStreamMeta* pMeta, SMsgCb* cb, int64_t streamId, int32_t taskId); int32_t tqStreamTaskProcessUpdateReq(SStreamMeta* pMeta, SMsgCb* cb, SRpcMsg* pMsg, bool restored); int32_t tqStreamTaskProcessDispatchReq(SStreamMeta* pMeta, SRpcMsg* pMsg); int32_t tqStreamTaskProcessDispatchRsp(SStreamMeta* pMeta, SRpcMsg* pMsg); @@ -27,7 +28,8 @@ int32_t tqStreamTaskProcessScanHistoryFinishRsp(SStreamMeta* pMeta, SRpcMsg* pMs int32_t tqStreamTaskProcessCheckReq(SStreamMeta* pMeta, SRpcMsg* pMsg); int32_t tqStreamTaskProcessCheckRsp(SStreamMeta* pMeta, SRpcMsg* pMsg, bool isLeader); int32_t tqStreamTaskProcessCheckpointReadyMsg(SStreamMeta* pMeta, SRpcMsg* pMsg); -int32_t tqStreamTaskProcessDeployReq(SStreamMeta* pMeta, int64_t sversion, char* msg, int32_t msgLen, bool isLeader, bool restored); +int32_t tqStreamTaskProcessDeployReq(SStreamMeta* pMeta, SMsgCb* cb, int64_t sversion, char* msg, int32_t msgLen, + bool isLeader, bool restored); int32_t tqStreamTaskProcessDropReq(SStreamMeta* pMeta, char* msg, int32_t msgLen); int32_t tqStreamTaskProcessRunReq(SStreamMeta* pMeta, SRpcMsg* pMsg, bool isLeader); int32_t tqStreamTaskResetStatus(SStreamMeta* pMeta); diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 2998c6d7b1..8c9eeb6f3a 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -52,9 +52,10 @@ extern "C" { #define STREAM_EXEC_T_EXTRACT_WAL_DATA (-1) #define STREAM_EXEC_T_START_ALL_TASKS (-2) -#define STREAM_EXEC_T_RESTART_ALL_TASKS (-3) -#define STREAM_EXEC_T_STOP_ALL_TASKS (-4) -#define STREAM_EXEC_T_RESUME_TASK (-5) +#define STREAM_EXEC_T_START_ONE_TASK (-3) +#define STREAM_EXEC_T_RESTART_ALL_TASKS (-4) +#define STREAM_EXEC_T_STOP_ALL_TASKS (-5) +#define STREAM_EXEC_T_RESUME_TASK (-6) typedef struct SStreamTask SStreamTask; typedef struct SStreamQueue SStreamQueue; @@ -880,6 +881,7 @@ void streamMetaUpdateStageRole(SStreamMeta* pMeta, int64_t stage, bool i int32_t streamMetaLoadAllTasks(SStreamMeta* pMeta); int32_t streamMetaStartAllTasks(SStreamMeta* pMeta); int32_t streamMetaStopAllTasks(SStreamMeta* pMeta); +int32_t streamMetaStartOneTask(SStreamMeta* pMeta, int64_t streamId, int32_t taskId); // checkpoint int32_t streamProcessCheckpointSourceReq(SStreamTask* pTask, SStreamCheckpointSourceReq* pReq); diff --git a/source/dnode/snode/src/snode.c b/source/dnode/snode/src/snode.c index 837c55c219..4ad054be27 100644 --- a/source/dnode/snode/src/snode.c +++ b/source/dnode/snode/src/snode.c @@ -193,7 +193,7 @@ int32_t sndProcessWriteMsg(SSnode *pSnode, SRpcMsg *pMsg, SRpcMsg *pRsp) { case TDMT_STREAM_TASK_DEPLOY: { void * pReq = POINTER_SHIFT(pMsg->pCont, sizeof(SMsgHead)); int32_t len = pMsg->contLen - sizeof(SMsgHead); - return tqStreamTaskProcessDeployReq(pSnode->pMeta, -1, pReq, len, true, true); + return tqStreamTaskProcessDeployReq(pSnode->pMeta, &pSnode->msgCb, -1, pReq, len, true, true); } case TDMT_STREAM_TASK_DROP: diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 33e9d97f8d..862d7e1d66 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -887,7 +887,8 @@ int32_t tqProcessTaskCheckRsp(STQ* pTq, SRpcMsg* pMsg) { } int32_t tqProcessTaskDeployReq(STQ* pTq, int64_t sversion, char* msg, int32_t msgLen) { - return tqStreamTaskProcessDeployReq(pTq->pStreamMeta, sversion, msg, msgLen, vnodeIsRoleLeader(pTq->pVnode), pTq->pVnode->restored); + return tqStreamTaskProcessDeployReq(pTq->pStreamMeta, &pTq->pVnode->msgCb, sversion, msg, msgLen, + vnodeIsRoleLeader(pTq->pVnode), pTq->pVnode->restored); } static void doStartFillhistoryStep2(SStreamTask* pTask, SStreamTask* pStreamTask, STQ* pTq) { diff --git a/source/dnode/vnode/src/tqCommon/tqCommon.c b/source/dnode/vnode/src/tqCommon/tqCommon.c index 6a47cfc3eb..a73c436044 100644 --- a/source/dnode/vnode/src/tqCommon/tqCommon.c +++ b/source/dnode/vnode/src/tqCommon/tqCommon.c @@ -50,6 +50,33 @@ int32_t tqStreamTaskStartAsync(SStreamMeta* pMeta, SMsgCb* cb, bool restart) { return 0; } +int32_t tqStreamOneTaskStartAsync(SStreamMeta* pMeta, SMsgCb* cb, int64_t streamId, int32_t taskId) { + int32_t vgId = pMeta->vgId; + + int32_t numOfTasks = taosArrayGetSize(pMeta->pTaskList); + if (numOfTasks == 0) { + tqDebug("vgId:%d no stream tasks existed to run", vgId); + return 0; + } + + SStreamTaskRunReq* pRunReq = rpcMallocCont(sizeof(SStreamTaskRunReq)); + if (pRunReq == NULL) { + terrno = TSDB_CODE_OUT_OF_MEMORY; + tqError("vgId:%d failed to create msg to start task:0x%x, code:%s", vgId, taskId, terrstr()); + return -1; + } + + tqDebug("vgId:%d start task:0x%x async", vgId, taskId); + pRunReq->head.vgId = vgId; + pRunReq->streamId = streamId; + pRunReq->taskId = taskId; + pRunReq->reqType = STREAM_EXEC_T_START_ONE_TASK; + + SRpcMsg msg = {.msgType = TDMT_STREAM_TASK_RUN, .pCont = pRunReq, .contLen = sizeof(SStreamTaskRunReq)}; + tmsgPutToQueue(cb, STREAM_QUEUE, &msg); + return 0; +} + int32_t tqStreamTaskProcessUpdateReq(SStreamMeta* pMeta, SMsgCb* cb, SRpcMsg* pMsg, bool restored) { int32_t vgId = pMeta->vgId; char* msg = POINTER_SHIFT(pMsg->pCont, sizeof(SMsgHead)); @@ -507,8 +534,8 @@ int32_t tqStreamTaskProcessCheckpointReadyMsg(SStreamMeta* pMeta, SRpcMsg* pMsg) return code; } -int32_t tqStreamTaskProcessDeployReq(SStreamMeta* pMeta, int64_t sversion, char* msg, int32_t msgLen, bool isLeader, - bool restored) { +int32_t tqStreamTaskProcessDeployReq(SStreamMeta* pMeta, SMsgCb* cb, int64_t sversion, char* msg, int32_t msgLen, + bool isLeader, bool restored) { int32_t code = 0; int32_t vgId = pMeta->vgId; @@ -560,18 +587,19 @@ int32_t tqStreamTaskProcessDeployReq(SStreamMeta* pMeta, int64_t sversion, char* // only handled in the leader node if (isLeader) { tqDebug("vgId:%d s-task:0x%x is deployed and add into meta, numOfTasks:%d", vgId, taskId, numOfTasks); - SStreamTask* p = streamMetaAcquireTask(pMeta, streamId, taskId); - if (p != NULL && restored && p->info.fillHistory == 0) { - EStreamTaskEvent event = TASK_EVENT_INIT; - streamTaskHandleEvent(p->status.pSM, event); - } else if (!restored) { - tqWarn("s-task:%s not launched since vnode(vgId:%d) not ready", p->id.idStr, vgId); + if (restored) { + SStreamTask* p = streamMetaAcquireTask(pMeta, streamId, taskId); + if (p != NULL && (p->info.fillHistory == 0)) { + tqStreamOneTaskStartAsync(pMeta, cb, streamId, taskId); + } + if (p != NULL) { + streamMetaReleaseTask(pMeta, p); + } + } else { + tqWarn("s-task:0x%x not launched since vnode(vgId:%d) not ready", taskId, vgId); } - if (p != NULL) { - streamMetaReleaseTask(pMeta, p); - } } else { tqDebug("vgId:%d not leader, not launch stream task s-task:0x%x", vgId, taskId); } @@ -696,7 +724,10 @@ int32_t tqStreamTaskProcessRunReq(SStreamMeta* pMeta, SRpcMsg* pMsg, bool isLead int32_t type = pReq->reqType; int32_t vgId = pMeta->vgId; - if (type == STREAM_EXEC_T_START_ALL_TASKS) { + if (type == STREAM_EXEC_T_START_ONE_TASK) { + streamMetaStartOneTask(pMeta, pReq->streamId, pReq->taskId); + return 0; + } else if (type == STREAM_EXEC_T_START_ALL_TASKS) { streamMetaStartAllTasks(pMeta); return 0; } else if (type == STREAM_EXEC_T_RESTART_ALL_TASKS) { @@ -755,7 +786,7 @@ int32_t tqStartTaskCompleteCallback(SStreamMeta* pMeta) { restartStreamTasks(pMeta, (pMeta->role == NODE_ROLE_LEADER)); } else { streamMetaWUnLock(pMeta); - tqDebug("vgId:%d start all tasks completed", pMeta->vgId); + tqDebug("vgId:%d start all tasks completed in callbackFn", pMeta->vgId); } return TSDB_CODE_SUCCESS; diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index bf97bee5f2..b8c5cf8e3f 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -1506,4 +1506,40 @@ int32_t streamMetaStartAllTasks(SStreamMeta* pMeta) { stInfo("vgId:%d start tasks completed", pMeta->vgId); taosArrayDestroy(pTaskList); return code; +} + +int32_t streamMetaStartOneTask(SStreamMeta* pMeta, int64_t streamId, int32_t taskId) { + int32_t vgId = pMeta->vgId; + stInfo("vgId:%d start to task:0x%x by checking downstream status", vgId, taskId); + + SStreamTask* pTask = streamMetaAcquireTask(pMeta, streamId, taskId); + if (pTask == NULL) { + stError("vgId:%d failed to acquire task:0x%x during start tasks", pMeta->vgId, taskId); + streamMetaUpdateTaskDownstreamStatus(pMeta, streamId, taskId, 0, taosGetTimestampMs(), false); + return TSDB_CODE_STREAM_TASK_IVLD_STATUS; + } + + // todo: may be we should find the related fill-history task and set it failed. + + // fill-history task can only be launched by related stream tasks. + STaskExecStatisInfo* pInfo = &pTask->execInfo; + if (pTask->info.fillHistory == 1) { + streamMetaReleaseTask(pMeta, pTask); + return TSDB_CODE_SUCCESS; + } + + ASSERT(pTask->status.downstreamReady == 0); + + int32_t ret = streamTaskHandleEvent(pTask->status.pSM, TASK_EVENT_INIT); + if (ret != TSDB_CODE_SUCCESS) { + stError("vgId:%d failed to handle event:%d", pMeta->vgId, TASK_EVENT_INIT); + + streamMetaUpdateTaskDownstreamStatus(pMeta, streamId, taskId, pInfo->init, pInfo->start, false); + if (HAS_RELATED_FILLHISTORY_TASK(pTask)) { + STaskId* pId = &pTask->hTaskInfo.id; + streamMetaUpdateTaskDownstreamStatus(pMeta, pId->streamId, pId->taskId, pInfo->init, pInfo->start, false); + } + } + + return ret; } \ No newline at end of file From 33253cbb54475e08f1a4ff12abbc9d0a8e5734a6 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Fri, 5 Jan 2024 18:13:35 +0800 Subject: [PATCH 020/102] fix(stream): add null ptr check. --- source/libs/stream/src/streamExec.c | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index d83d99f10f..5c38b24a77 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -689,8 +689,12 @@ bool streamTaskIsIdle(const SStreamTask* pTask) { bool streamTaskReadyToRun(const SStreamTask* pTask, char** pStatus) { SStreamTaskState* pState = streamTaskGetStatus(pTask); + ETaskStatus st = pState->state; - *pStatus = pState->name; + if (pStatus != NULL) { + *pStatus = pState->name; + } + return (st == TASK_STATUS__READY || st == TASK_STATUS__SCAN_HISTORY || st == TASK_STATUS__CK); } From 2386f842fb0e2094c7d5c3bcc6f3ab676f8dcf38 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Fri, 5 Jan 2024 18:46:15 +0800 Subject: [PATCH 021/102] enh(stream): remove sleep to opt perf. --- source/dnode/vnode/src/tqCommon/tqCommon.c | 13 +++++++------ source/libs/stream/src/streamExec.c | 20 +++++++++++++------- 2 files changed, 20 insertions(+), 13 deletions(-) diff --git a/source/dnode/vnode/src/tqCommon/tqCommon.c b/source/dnode/vnode/src/tqCommon/tqCommon.c index a73c436044..ad6d11aaf2 100644 --- a/source/dnode/vnode/src/tqCommon/tqCommon.c +++ b/source/dnode/vnode/src/tqCommon/tqCommon.c @@ -736,16 +736,17 @@ int32_t tqStreamTaskProcessRunReq(SStreamMeta* pMeta, SRpcMsg* pMsg, bool isLead } else if (type == STREAM_EXEC_T_STOP_ALL_TASKS) { streamMetaStopAllTasks(pMeta); return 0; - } else if (type == STREAM_EXEC_T_RESUME_TASK) { - // task resume to run after idle for a while + } else if (type == STREAM_EXEC_T_RESUME_TASK) { // task resume to run after idle for a while SStreamTask* pTask = streamMetaAcquireTask(pMeta, pReq->streamId, pReq->taskId); if (pTask != NULL) { ASSERT(streamTaskReadyToRun(pTask, NULL)); - tqDebug("s-task:%s task resume to run after idle for a while", pTask->id.idStr); - streamResumeTask(pTask); - } + int64_t execTs = pTask->status.lastExecTs; + int32_t idle = taosGetTimestampMs() - execTs; + tqDebug("s-task:%s task resume to run after idle for:%dms from:%" PRId64, pTask->id.idStr, idle, execTs); - streamMetaReleaseTask(pMeta, pTask); + streamResumeTask(pTask); + streamMetaReleaseTask(pMeta, pTask); + } return 0; } diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index 5c38b24a77..7b740a8e7a 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -107,12 +107,6 @@ static int32_t streamTaskExecImpl(SStreamTask* pTask, SStreamQueueItem* pItem, i return 0; } - if (pTask->inputq.status == TASK_INPUT_STATUS__BLOCKED) { - stWarn("s-task:%s downstream task inputQ blocked, idle for 1sec and retry exec task", pTask->id.idStr); - taosMsleep(1000); - continue; - } - SSDataBlock* output = NULL; uint64_t ts = 0; if ((code = qExecTask(pExecutor, &output, &ts)) < 0) { @@ -561,6 +555,11 @@ static void setTaskSchedInfo(SStreamTask* pTask, int32_t idleTime) { pStatus->lastExecTs = taosGetTimestampMs(); } +static void clearTaskSchedInfo(SStreamTask* pTask) { + SStreamStatus* pStatus = &pTask->status; + pStatus->schedIdleTime = 0; +} + /** * todo: the batch of blocks should be tuned dynamic, according to the total elapsed time of each batch of blocks, the * appropriate batch of blocks should be handled in 5 to 10 sec. @@ -586,6 +585,12 @@ int32_t doStreamExecTask(SStreamTask* pTask) { break; } + if (pTask->inputq.status == TASK_INPUT_STATUS__BLOCKED) { + stWarn("s-task:%s downstream task inputQ blocked, idle for 1sec and retry", pTask->id.idStr); + setTaskSchedInfo(pTask, 1000); + continue; + } + /*int32_t code = */ streamTaskGetDataFromInputQ(pTask, &pInput, &numOfBlocks, &blockSize); if (pInput == NULL) { ASSERT(numOfBlocks == 0); @@ -737,7 +742,7 @@ static void doStreamExecTaskHelper(void* param, void* tmrId) { tmsgPutToQueue(pTask->pMsgCb, STREAM_QUEUE, &msg); // release the task ref count - pTask->status.schedIdleTime = 0; // clear the idle time + clearTaskSchedInfo(pTask); streamMetaReleaseTask(pTask->pMeta, pTask); } @@ -747,6 +752,7 @@ static int32_t schedTaskInFuture(SStreamTask* pTask) { pTask->status.schedIdleTime, ref); // add one ref count for task + // todo this may be failed, and add ref may be failed. SStreamTask* pAddRefTask = streamMetaAcquireTask(pTask->pMeta, pTask->id.streamId, pTask->id.taskId); if (pTask->schedInfo.pIdleTimer == NULL) { From 522e688387d37184636f2c74a4109ac5514b895b Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Fri, 5 Jan 2024 22:16:58 +0800 Subject: [PATCH 022/102] fix(stream): check if in restart procedure in call back. --- source/dnode/vnode/src/tqCommon/tqCommon.c | 29 +++++++++++++--------- source/libs/stream/src/streamQueue.c | 1 + 2 files changed, 18 insertions(+), 12 deletions(-) diff --git a/source/dnode/vnode/src/tqCommon/tqCommon.c b/source/dnode/vnode/src/tqCommon/tqCommon.c index ad6d11aaf2..16d073797b 100644 --- a/source/dnode/vnode/src/tqCommon/tqCommon.c +++ b/source/dnode/vnode/src/tqCommon/tqCommon.c @@ -774,22 +774,27 @@ int32_t tqStreamTaskProcessRunReq(SStreamMeta* pMeta, SRpcMsg* pMsg, bool isLead int32_t tqStartTaskCompleteCallback(SStreamMeta* pMeta) { STaskStartInfo* pStartInfo = &pMeta->startInfo; + int32_t vgId = pMeta->vgId; + streamMetaWLock(pMeta); + if (pStartInfo->taskStarting == 1) { + tqDebug("vgId:%d already in start tasks procedure in other thread, restartCounter:%d, do nothing", vgId, + pMeta->startInfo.restartCount); + } else { // not in starting procedure + if (pStartInfo->restartCount > 0) { + pStartInfo->restartCount -= 1; + tqDebug("vgId:%d role:%d need to restart all tasks again, restartCounter:%d", vgId, pMeta->role, + pStartInfo->restartCount); - if (pStartInfo->restartCount > 0) { - pStartInfo->restartCount -= 1; - - ASSERT(pStartInfo->taskStarting == 0); - tqDebug("vgId:%d role:%d need to restart all tasks again, restartCounter:%d", pMeta->vgId, pMeta->role, - pStartInfo->restartCount); - - streamMetaWUnLock(pMeta); - restartStreamTasks(pMeta, (pMeta->role == NODE_ROLE_LEADER)); - } else { - streamMetaWUnLock(pMeta); - tqDebug("vgId:%d start all tasks completed in callbackFn", pMeta->vgId); + streamMetaWUnLock(pMeta); + restartStreamTasks(pMeta, (pMeta->role == NODE_ROLE_LEADER)); + return TSDB_CODE_SUCCESS; + } else { + tqDebug("vgId:%d start all tasks completed in callbackFn", pMeta->vgId); + } } + streamMetaWUnLock(pMeta); return TSDB_CODE_SUCCESS; } diff --git a/source/libs/stream/src/streamQueue.c b/source/libs/stream/src/streamQueue.c index 459efb838c..479612d9d1 100644 --- a/source/libs/stream/src/streamQueue.c +++ b/source/libs/stream/src/streamQueue.c @@ -157,6 +157,7 @@ int32_t streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueItem** pInpu *numOfBlocks = 0; *blockSize = 0; + // todo remove it // 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 10ms", id); From 445605ed2205c37afafbfb99cb520d7856ff8cef Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Fri, 5 Jan 2024 22:35:56 +0800 Subject: [PATCH 023/102] refactor: do some internal refactor. --- source/dnode/vnode/src/tqCommon/tqCommon.c | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/source/dnode/vnode/src/tqCommon/tqCommon.c b/source/dnode/vnode/src/tqCommon/tqCommon.c index 16d073797b..6ceb0a95ba 100644 --- a/source/dnode/vnode/src/tqCommon/tqCommon.c +++ b/source/dnode/vnode/src/tqCommon/tqCommon.c @@ -134,16 +134,11 @@ int32_t tqStreamTaskProcessUpdateReq(SStreamMeta* pMeta, SMsgCb* cb, SRpcMsg* pM return rsp.code; } -// streamMetaWUnLock(pMeta); - // todo for test purpose // the following two functions should not be executed within the scope of meta lock to avoid deadlock streamTaskUpdateEpsetInfo(pTask, req.pNodeList); streamTaskResetStatus(pTask); - // continue after lock the meta again -// streamMetaWLock(pMeta); - SStreamTask** ppHTask = NULL; if (HAS_RELATED_FILLHISTORY_TASK(pTask)) { ppHTask = (SStreamTask**)taosHashGet(pMeta->pTasksMap, &pTask->hTaskInfo.id, sizeof(pTask->hTaskInfo.id)); @@ -157,6 +152,8 @@ int32_t tqStreamTaskProcessUpdateReq(SStreamMeta* pMeta, SMsgCb* cb, SRpcMsg* pM } } + tqDebug("s-task:%s start to save task", pTask->id.idStr); + { streamMetaSaveTask(pMeta, pTask); if (ppHTask != NULL) { @@ -168,6 +165,8 @@ int32_t tqStreamTaskProcessUpdateReq(SStreamMeta* pMeta, SMsgCb* cb, SRpcMsg* pM } } + tqDebug("s-task:%s start to stop task after save task", pTask->id.idStr); + streamTaskStop(pTask); // keep the already handled info From 23341e1bdff0f89b64374b6514fcbc286cda4da8 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Fri, 5 Jan 2024 22:58:43 +0800 Subject: [PATCH 024/102] refactor: opt perf by avoid save tasks during restoring --- source/dnode/vnode/src/tqCommon/tqCommon.c | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/source/dnode/vnode/src/tqCommon/tqCommon.c b/source/dnode/vnode/src/tqCommon/tqCommon.c index 6ceb0a95ba..6e83c2d0a9 100644 --- a/source/dnode/vnode/src/tqCommon/tqCommon.c +++ b/source/dnode/vnode/src/tqCommon/tqCommon.c @@ -152,9 +152,8 @@ int32_t tqStreamTaskProcessUpdateReq(SStreamMeta* pMeta, SMsgCb* cb, SRpcMsg* pM } } - tqDebug("s-task:%s start to save task", pTask->id.idStr); - - { + if (restored) { + tqDebug("s-task:%s vgId:%d start to save task", pTask->id.idStr, vgId); streamMetaSaveTask(pMeta, pTask); if (ppHTask != NULL) { streamMetaSaveTask(pMeta, *ppHTask); @@ -163,10 +162,11 @@ int32_t tqStreamTaskProcessUpdateReq(SStreamMeta* pMeta, SMsgCb* cb, SRpcMsg* pM if (streamMetaCommit(pMeta) < 0) { // persist to disk } + } else { + tqDebug("s-task:%s vgId:%d not save since restore not finish", pTask->id.idStr, vgId); } tqDebug("s-task:%s start to stop task after save task", pTask->id.idStr); - streamTaskStop(pTask); // keep the already handled info From cfda97beef8b4d99253c6af757b77c0ca159a702 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sat, 6 Jan 2024 15:27:48 +0800 Subject: [PATCH 025/102] refactor: disable commit temp --- source/dnode/vnode/src/tqCommon/tqCommon.c | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/source/dnode/vnode/src/tqCommon/tqCommon.c b/source/dnode/vnode/src/tqCommon/tqCommon.c index 6e83c2d0a9..3989acad2e 100644 --- a/source/dnode/vnode/src/tqCommon/tqCommon.c +++ b/source/dnode/vnode/src/tqCommon/tqCommon.c @@ -134,7 +134,6 @@ int32_t tqStreamTaskProcessUpdateReq(SStreamMeta* pMeta, SMsgCb* cb, SRpcMsg* pM return rsp.code; } - // todo for test purpose // the following two functions should not be executed within the scope of meta lock to avoid deadlock streamTaskUpdateEpsetInfo(pTask, req.pNodeList); streamTaskResetStatus(pTask); @@ -159,9 +158,11 @@ int32_t tqStreamTaskProcessUpdateReq(SStreamMeta* pMeta, SMsgCb* cb, SRpcMsg* pM streamMetaSaveTask(pMeta, *ppHTask); } +#if 0 if (streamMetaCommit(pMeta) < 0) { // persist to disk } +#endif } else { tqDebug("s-task:%s vgId:%d not save since restore not finish", pTask->id.idStr, vgId); } From 02f6c971fd390fc405a21f1e03f2d5e131489f9e Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sat, 6 Jan 2024 19:43:39 +0800 Subject: [PATCH 026/102] fix(stream): check for not ready task when rsp returned. --- include/libs/stream/tstream.h | 1 + source/dnode/vnode/src/tqCommon/tqCommon.c | 94 ++++++++++++++++++---- source/libs/stream/src/streamMeta.c | 1 + 3 files changed, 80 insertions(+), 16 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 8c9eeb6f3a..dbcdef57b0 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -56,6 +56,7 @@ extern "C" { #define STREAM_EXEC_T_RESTART_ALL_TASKS (-4) #define STREAM_EXEC_T_STOP_ALL_TASKS (-5) #define STREAM_EXEC_T_RESUME_TASK (-6) +#define STREAM_EXEC_T_UPDATE_TASK_EPSET (-7) typedef struct SStreamTask SStreamTask; typedef struct SStreamQueue SStreamQueue; diff --git a/source/dnode/vnode/src/tqCommon/tqCommon.c b/source/dnode/vnode/src/tqCommon/tqCommon.c index 3989acad2e..aed1091c84 100644 --- a/source/dnode/vnode/src/tqCommon/tqCommon.c +++ b/source/dnode/vnode/src/tqCommon/tqCommon.c @@ -77,6 +77,33 @@ int32_t tqStreamOneTaskStartAsync(SStreamMeta* pMeta, SMsgCb* cb, int64_t stream return 0; } +int32_t tqUpdateNodeEpsetAsync(SStreamMeta* pMeta, SMsgCb* cb, int64_t streamId, int32_t taskId) { + int32_t vgId = pMeta->vgId; + + int32_t numOfTasks = taosArrayGetSize(pMeta->pTaskList); + if (numOfTasks == 0) { + tqDebug("vgId:%d no stream tasks existed to run", vgId); + return 0; + } + + SStreamTaskRunReq* pRunReq = rpcMallocCont(sizeof(SStreamTaskRunReq)); + if (pRunReq == NULL) { + terrno = TSDB_CODE_OUT_OF_MEMORY; + tqError("vgId:%d failed to create msg to start task:0x%x, code:%s", vgId, taskId, terrstr()); + return -1; + } + + tqDebug("vgId:%d update s-task:0x%x nodeEpset async", vgId, taskId); + pRunReq->head.vgId = vgId; + pRunReq->streamId = streamId; + pRunReq->taskId = taskId; + pRunReq->reqType = STREAM_EXEC_T_UPDATE_TASK_EPSET; + + SRpcMsg msg = {.msgType = TDMT_STREAM_TASK_RUN, .pCont = pRunReq, .contLen = sizeof(SStreamTaskRunReq)}; + tmsgPutToQueue(cb, STREAM_QUEUE, &msg); + return 0; +} + int32_t tqStreamTaskProcessUpdateReq(SStreamMeta* pMeta, SMsgCb* cb, SRpcMsg* pMsg, bool restored) { int32_t vgId = pMeta->vgId; char* msg = POINTER_SHIFT(pMsg->pCont, sizeof(SMsgHead)); @@ -443,12 +470,22 @@ int32_t tqStreamTaskProcessCheckReq(SStreamMeta* pMeta, SRpcMsg* pMsg) { return streamSendCheckRsp(pMeta, &req, &rsp, &pMsg->info, taskId); } +static void setParam(SStreamTask* pTask, int64_t* initTs, bool* hasHTask, STaskId* pId) { + *initTs = pTask->execInfo.init; + + if (HAS_RELATED_FILLHISTORY_TASK(pTask)) { + *hasHTask = true; + pId->streamId = pTask->hTaskInfo.id.streamId; + pId->taskId = pTask->hTaskInfo.id.taskId; + } +} + int32_t tqStreamTaskProcessCheckRsp(SStreamMeta* pMeta, SRpcMsg* pMsg, bool isLeader) { char* pReq = POINTER_SHIFT(pMsg->pCont, sizeof(SMsgHead)); int32_t len = pMsg->contLen - sizeof(SMsgHead); int32_t vgId = pMeta->vgId; + int32_t code = TSDB_CODE_SUCCESS; - int32_t code; SStreamTaskCheckRsp rsp; SDecoder decoder; @@ -467,35 +504,60 @@ int32_t tqStreamTaskProcessCheckRsp(SStreamMeta* pMeta, SRpcMsg* pMsg, bool isLe int64_t initTs = 0; int64_t now = taosGetTimestampMs(); - if (!isLeader) { - SStreamTask* pTask = streamMetaAcquireTask(pMeta, rsp.streamId, rsp.upstreamTaskId); - if (pTask != NULL) { - initTs = pTask->execInfo.init; + STaskId id = {.streamId = rsp.streamId, .taskId = rsp.upstreamTaskId}; + STaskId fId = {0}; + bool hasHistoryTask = false; - if (HAS_RELATED_FILLHISTORY_TASK(pTask)) { - STaskId* pId = &pTask->hTaskInfo.id; - streamMetaUpdateTaskDownstreamStatus(pMeta, pId->streamId, pId->taskId, initTs, now, false); + // todo extract method + if (!isLeader) { + // this task may have been stopped, so acquire task may failed. Retrieve it directly from the task hash map. + streamMetaRLock(pMeta); + + SStreamTask** ppTask = taosHashGet(pMeta->pTasksMap, &id, sizeof(id)); + if (ppTask != NULL) { + setParam(*ppTask, &initTs, &hasHistoryTask, &fId); + streamMetaRUnLock(pMeta); + + if (hasHistoryTask) { + streamMetaUpdateTaskDownstreamStatus(pMeta, fId.streamId, fId.taskId, initTs, now, false); } - streamMetaReleaseTask(pMeta, pTask); + tqError("vgId:%d not leader, task:0x%x not handle the check rsp, downstream:0x%x (vgId:%d)", vgId, + rsp.upstreamTaskId, rsp.downstreamTaskId, rsp.downstreamNodeId); + } else { + streamMetaRUnLock(pMeta); + + 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, vgId); + code = terrno = TSDB_CODE_STREAM_TASK_NOT_EXIST; } streamMetaUpdateTaskDownstreamStatus(pMeta, rsp.streamId, rsp.upstreamTaskId, initTs, now, false); - tqError("vgId:%d not leader, task:0x%x not handle the check rsp, downstream:0x%x (vgId:%d)", vgId, - rsp.upstreamTaskId, rsp.downstreamTaskId, rsp.downstreamNodeId); - return code; } SStreamTask* pTask = streamMetaAcquireTask(pMeta, rsp.streamId, rsp.upstreamTaskId); if (pTask == NULL) { - streamMetaUpdateTaskDownstreamStatus(pMeta, rsp.streamId, rsp.upstreamTaskId, 0, now, false); + streamMetaRLock(pMeta); + + SStreamTask** ppTask = taosHashGet(pMeta->pTasksMap, &id, sizeof(id)); + if (ppTask != NULL) { + setParam(*ppTask, &initTs, &hasHistoryTask, &fId); + streamMetaRUnLock(pMeta); + + if (hasHistoryTask) { + streamMetaUpdateTaskDownstreamStatus(pMeta, fId.streamId, fId.taskId, initTs, now, false); + } + } else { + streamMetaRUnLock(pMeta); + } + + streamMetaUpdateTaskDownstreamStatus(pMeta, rsp.streamId, rsp.upstreamTaskId, initTs, now, false); 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, vgId); - terrno = TSDB_CODE_STREAM_TASK_NOT_EXIST; - // failed to find the related fill-history task - return -1; + code = terrno = TSDB_CODE_STREAM_TASK_NOT_EXIST; + return code; } code = streamProcessCheckRsp(pTask, &rsp); diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index b8c5cf8e3f..7e69610f80 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -1472,6 +1472,7 @@ int32_t streamMetaStartAllTasks(SStreamMeta* pMeta) { // fill-history task can only be launched by related stream tasks. STaskExecStatisInfo* pInfo = &pTask->execInfo; if (pTask->info.fillHistory == 1) { + stDebug("s-task:%s fill-history task wait related stream task start", pTask->id.idStr); streamMetaReleaseTask(pMeta, pTask); continue; } From 5e32aea1f96aaed454d3fd75844c6917a36d4405 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sun, 7 Jan 2024 00:27:18 +0800 Subject: [PATCH 027/102] fix(stream): jump out of loop when downstream is blocking. --- 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 7b740a8e7a..eb5a432235 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -588,7 +588,7 @@ int32_t doStreamExecTask(SStreamTask* pTask) { if (pTask->inputq.status == TASK_INPUT_STATUS__BLOCKED) { stWarn("s-task:%s downstream task inputQ blocked, idle for 1sec and retry", pTask->id.idStr); setTaskSchedInfo(pTask, 1000); - continue; + break; } /*int32_t code = */ streamTaskGetDataFromInputQ(pTask, &pInput, &numOfBlocks, &blockSize); From 29219ba9f47d343d3840cf04628963f71d2300a9 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sun, 7 Jan 2024 01:32:50 +0800 Subject: [PATCH 028/102] fix(stream): add task number check. --- include/dnode/vnode/tqCommon.h | 2 +- source/dnode/snode/src/snode.c | 3 ++- source/dnode/vnode/src/tqCommon/tqCommon.c | 14 ++++++++------ source/dnode/vnode/src/vnd/vnodeSync.c | 8 +++++--- 4 files changed, 16 insertions(+), 11 deletions(-) diff --git a/include/dnode/vnode/tqCommon.h b/include/dnode/vnode/tqCommon.h index 6c84c3ea09..7265d6f58f 100644 --- a/include/dnode/vnode/tqCommon.h +++ b/include/dnode/vnode/tqCommon.h @@ -32,7 +32,7 @@ int32_t tqStreamTaskProcessDeployReq(SStreamMeta* pMeta, SMsgCb* cb, int64_t sve bool isLeader, bool restored); int32_t tqStreamTaskProcessDropReq(SStreamMeta* pMeta, char* msg, int32_t msgLen); int32_t tqStreamTaskProcessRunReq(SStreamMeta* pMeta, SRpcMsg* pMsg, bool isLeader); -int32_t tqStreamTaskResetStatus(SStreamMeta* pMeta); +int32_t tqStreamTaskResetStatus(SStreamMeta* pMeta, int32_t* numOfTasks); int32_t tqStartTaskCompleteCallback(SStreamMeta* pMeta); int32_t tqStreamTaskProcessTaskResetReq(SStreamMeta* pMeta, SRpcMsg* pMsg); diff --git a/source/dnode/snode/src/snode.c b/source/dnode/snode/src/snode.c index 4ad054be27..613480f4bc 100644 --- a/source/dnode/snode/src/snode.c +++ b/source/dnode/snode/src/snode.c @@ -145,7 +145,8 @@ FAIL: } int32_t sndInit(SSnode *pSnode) { - tqStreamTaskResetStatus(pSnode->pMeta); + int32_t numOfTasks = 0; + tqStreamTaskResetStatus(pSnode->pMeta, &numOfTasks); streamMetaStartAllTasks(pSnode->pMeta); return 0; } diff --git a/source/dnode/vnode/src/tqCommon/tqCommon.c b/source/dnode/vnode/src/tqCommon/tqCommon.c index aed1091c84..6b1c7ccedb 100644 --- a/source/dnode/vnode/src/tqCommon/tqCommon.c +++ b/source/dnode/vnode/src/tqCommon/tqCommon.c @@ -706,16 +706,16 @@ int32_t tqStreamTaskProcessDropReq(SStreamMeta* pMeta, char* msg, int32_t msgLen return 0; } -int32_t tqStreamTaskResetStatus(SStreamMeta* pMeta) { +int32_t tqStreamTaskResetStatus(SStreamMeta* pMeta, int32_t* numOfTasks) { int32_t vgId = pMeta->vgId; - int32_t numOfTasks = taosArrayGetSize(pMeta->pTaskList); + *numOfTasks = taosArrayGetSize(pMeta->pTaskList); - tqDebug("vgId:%d reset all %d stream task(s) status to be uninit", vgId, numOfTasks); - if (numOfTasks == 0) { + tqDebug("vgId:%d reset all %d stream task(s) status to be uninit", vgId, *numOfTasks); + if (*numOfTasks == 0) { return TSDB_CODE_SUCCESS; } - for (int32_t i = 0; i < numOfTasks; ++i) { + for (int32_t i = 0; i < (*numOfTasks); ++i) { SStreamTaskId* pTaskId = taosArrayGet(pMeta->pTaskList, i); STaskId id = {.streamId = pTaskId->streamId, .taskId = pTaskId->taskId}; @@ -767,8 +767,10 @@ static int32_t restartStreamTasks(SStreamMeta* pMeta, bool isLeader) { } if (isLeader && !tsDisableStream) { - tqStreamTaskResetStatus(pMeta); + int32_t numOfTasks = 0; + tqStreamTaskResetStatus(pMeta, &numOfTasks); streamMetaWUnLock(pMeta); + streamMetaStartAllTasks(pMeta); } else { streamMetaResetStartInfo(&pMeta->startInfo); diff --git a/source/dnode/vnode/src/vnd/vnodeSync.c b/source/dnode/vnode/src/vnd/vnodeSync.c index 34ae6623a7..f0ef351184 100644 --- a/source/dnode/vnode/src/vnd/vnodeSync.c +++ b/source/dnode/vnode/src/vnd/vnodeSync.c @@ -570,10 +570,12 @@ static void vnodeRestoreFinish(const SSyncFSM *pFsm, const SyncIndex commitIdx) streamMetaWUnLock(pMeta); vInfo("vgId:%d, sync restore finished, not launch stream tasks, since stream tasks are disabled", vgId); } else { - vInfo("vgId:%d sync restore finished, start to launch stream tasks", pVnode->config.vgId); - tqStreamTaskResetStatus(pVnode->pTq->pStreamMeta); + vInfo("vgId:%d sync restore finished, start to launch stream task(s)", pVnode->config.vgId); - { + int32_t numOfTasks = 0; + tqStreamTaskResetStatus(pVnode->pTq->pStreamMeta, &numOfTasks); + + if (numOfTasks > 0) { if (pMeta->startInfo.taskStarting == 1) { pMeta->startInfo.restartCount += 1; tqDebug("vgId:%d in start tasks procedure, inc restartCounter by 1, remaining restart:%d", vgId, From a024fe960c2500f1f2519a1a183a78bb47dc934a Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sun, 7 Jan 2024 14:53:17 +0800 Subject: [PATCH 029/102] fix(stream): fix deadlock. --- source/dnode/vnode/src/vnd/vnodeSync.c | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/source/dnode/vnode/src/vnd/vnodeSync.c b/source/dnode/vnode/src/vnd/vnodeSync.c index f0ef351184..26d232b443 100644 --- a/source/dnode/vnode/src/vnd/vnodeSync.c +++ b/source/dnode/vnode/src/vnd/vnodeSync.c @@ -567,31 +567,30 @@ static void vnodeRestoreFinish(const SSyncFSM *pFsm, const SyncIndex commitIdx) if (vnodeIsRoleLeader(pVnode)) { // start to restore all stream tasks if (tsDisableStream) { - streamMetaWUnLock(pMeta); vInfo("vgId:%d, sync restore finished, not launch stream tasks, since stream tasks are disabled", vgId); } else { vInfo("vgId:%d sync restore finished, start to launch stream task(s)", pVnode->config.vgId); - int32_t numOfTasks = 0; - tqStreamTaskResetStatus(pVnode->pTq->pStreamMeta, &numOfTasks); + tqStreamTaskResetStatus(pMeta, &numOfTasks); if (numOfTasks > 0) { if (pMeta->startInfo.taskStarting == 1) { pMeta->startInfo.restartCount += 1; tqDebug("vgId:%d in start tasks procedure, inc restartCounter by 1, remaining restart:%d", vgId, pMeta->startInfo.restartCount); - streamMetaWUnLock(pMeta); } else { pMeta->startInfo.taskStarting = 1; streamMetaWUnLock(pMeta); tqStreamTaskStartAsync(pMeta, &pVnode->msgCb, false); + return; } } } } else { - streamMetaWUnLock(pMeta); vInfo("vgId:%d, sync restore finished, not launch stream tasks since not leader", vgId); } + + streamMetaWUnLock(pMeta); } static void vnodeBecomeFollower(const SSyncFSM *pFsm) { From bb9edde79ed8b7ab5768a73677aab1f2625fcac0 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 8 Jan 2024 11:33:54 +0800 Subject: [PATCH 030/102] fix(stream): add some logs. --- source/dnode/vnode/src/tqCommon/tqCommon.c | 15 +++++++++++---- source/libs/stream/src/streamQueue.c | 1 + 2 files changed, 12 insertions(+), 4 deletions(-) diff --git a/source/dnode/vnode/src/tqCommon/tqCommon.c b/source/dnode/vnode/src/tqCommon/tqCommon.c index 6b1c7ccedb..656ee86831 100644 --- a/source/dnode/vnode/src/tqCommon/tqCommon.c +++ b/source/dnode/vnode/src/tqCommon/tqCommon.c @@ -109,6 +109,7 @@ int32_t tqStreamTaskProcessUpdateReq(SStreamMeta* pMeta, SMsgCb* cb, SRpcMsg* pM char* msg = POINTER_SHIFT(pMsg->pCont, sizeof(SMsgHead)); int32_t len = pMsg->contLen - sizeof(SMsgHead); SRpcMsg rsp = {.info = pMsg->info, .code = TSDB_CODE_SUCCESS}; + int64_t st = taosGetTimestampMs(); SStreamTaskNodeUpdateMsg req = {0}; @@ -194,7 +195,7 @@ int32_t tqStreamTaskProcessUpdateReq(SStreamMeta* pMeta, SMsgCb* cb, SRpcMsg* pM tqDebug("s-task:%s vgId:%d not save since restore not finish", pTask->id.idStr, vgId); } - tqDebug("s-task:%s start to stop task after save task", pTask->id.idStr); + tqDebug("s-task:%s vgId:%d start to stop task after save task", pTask->id.idStr, vgId); streamTaskStop(pTask); // keep the already handled info @@ -202,10 +203,16 @@ int32_t tqStreamTaskProcessUpdateReq(SStreamMeta* pMeta, SMsgCb* cb, SRpcMsg* pM if (ppHTask != NULL) { streamTaskStop(*ppHTask); - tqDebug("s-task:%s task nodeEp update completed, streamTask and related fill-history task closed", pTask->id.idStr); + + int64_t now = taosGetTimestampMs(); + tqDebug("s-task:%s vgId:%d task nodeEp update completed, streamTask and related fill-history task closed, elapsed:%" PRId64 + " ms", + pTask->id.idStr, vgId, now-st); taosHashPut(pMeta->updateInfo.pTasks, &(*ppHTask)->id, sizeof(pTask->id), NULL, 0); } else { - tqDebug("s-task:%s task nodeEp update completed, streamTask closed", pTask->id.idStr); + int64_t now = taosGetTimestampMs(); + tqDebug("s-task:%s vgId:%d, task nodeEp update completed, streamTask closed, elapsed time:%" PRId64 "ms", pTask->id.idStr, + vgId, now - st); } rsp.code = 0; @@ -226,7 +233,7 @@ int32_t tqStreamTaskProcessUpdateReq(SStreamMeta* pMeta, SMsgCb* cb, SRpcMsg* pM pMeta->startInfo.tasksWillRestart = 0; streamMetaWUnLock(pMeta); } else { - tqDebug("vgId:%d all %d task(s) nodeEp updated and closed", vgId, numOfTasks); + tqDebug("vgId:%d all %d task(s) nodeEp updated and closed, transId:%d", vgId, numOfTasks, req.transId); #if 0 // for test purpose, to trigger the leader election taosMSleep(5000); diff --git a/source/libs/stream/src/streamQueue.c b/source/libs/stream/src/streamQueue.c index 479612d9d1..5f55285ab1 100644 --- a/source/libs/stream/src/streamQueue.c +++ b/source/libs/stream/src/streamQueue.c @@ -175,6 +175,7 @@ int32_t streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueItem** pInpu if (qItem == NULL) { if ((taskLevel == TASK_LEVEL__SOURCE || taskLevel == TASK_LEVEL__SINK) && (++retryTimes) < MAX_RETRY_TIMES) { taosMsleep(WAIT_FOR_DURATION); + // todo remove it continue; } From 275e2d6f9457aae002829125612f294a6e6d54f1 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 8 Jan 2024 11:34:47 +0800 Subject: [PATCH 031/102] refactor: do some internal refactor. --- source/dnode/vnode/src/tqCommon/tqCommon.c | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/source/dnode/vnode/src/tqCommon/tqCommon.c b/source/dnode/vnode/src/tqCommon/tqCommon.c index 656ee86831..9a3fdbe512 100644 --- a/source/dnode/vnode/src/tqCommon/tqCommon.c +++ b/source/dnode/vnode/src/tqCommon/tqCommon.c @@ -205,9 +205,8 @@ int32_t tqStreamTaskProcessUpdateReq(SStreamMeta* pMeta, SMsgCb* cb, SRpcMsg* pM streamTaskStop(*ppHTask); int64_t now = taosGetTimestampMs(); - tqDebug("s-task:%s vgId:%d task nodeEp update completed, streamTask and related fill-history task closed, elapsed:%" PRId64 - " ms", - pTask->id.idStr, vgId, now-st); + tqDebug("s-task:%s vgId:%d task nodeEp update completed, streamTask/fill-history closed, elapsed:%" PRId64 " ms", + pTask->id.idStr, vgId, now - st); taosHashPut(pMeta->updateInfo.pTasks, &(*ppHTask)->id, sizeof(pTask->id), NULL, 0); } else { int64_t now = taosGetTimestampMs(); From 3bcd038b7d87a18edf1bac625f34a444ddee14dc Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 8 Jan 2024 14:04:36 +0800 Subject: [PATCH 032/102] refactor: not reload info from disk when trying to restart stream tasks. --- source/dnode/vnode/src/tqCommon/tqCommon.c | 50 +++++++++++++--------- 1 file changed, 29 insertions(+), 21 deletions(-) diff --git a/source/dnode/vnode/src/tqCommon/tqCommon.c b/source/dnode/vnode/src/tqCommon/tqCommon.c index 9a3fdbe512..5467e3dadd 100644 --- a/source/dnode/vnode/src/tqCommon/tqCommon.c +++ b/source/dnode/vnode/src/tqCommon/tqCommon.c @@ -25,7 +25,6 @@ typedef struct STaskUpdateEntry { int32_t tqStreamTaskStartAsync(SStreamMeta* pMeta, SMsgCb* cb, bool restart) { int32_t vgId = pMeta->vgId; - int32_t numOfTasks = taosArrayGetSize(pMeta->pTaskList); if (numOfTasks == 0) { tqDebug("vgId:%d no stream tasks existed to run", vgId); @@ -131,7 +130,7 @@ int32_t tqStreamTaskProcessUpdateReq(SStreamMeta* pMeta, SMsgCb* cb, SRpcMsg* pM 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, + tqError("vgId:%d failed to acquire task:0x%x when handling update, it may have been dropped already", vgId, req.taskId); rsp.code = TSDB_CODE_SUCCESS; streamMetaWUnLock(pMeta); @@ -141,21 +140,22 @@ int32_t tqStreamTaskProcessUpdateReq(SStreamMeta* pMeta, SMsgCb* cb, SRpcMsg* pM } SStreamTask* pTask = *ppTask; + const char* idstr = pTask->id.idStr; if (pMeta->updateInfo.transId != req.transId) { pMeta->updateInfo.transId = req.transId; - tqInfo("s-task:%s receive new trans to update nodeEp msg from mnode, transId:%d", pTask->id.idStr, req.transId); + tqInfo("s-task:%s receive new trans to update nodeEp msg from mnode, transId:%d", idstr, req.transId); // info needs to be kept till the new trans to update the nodeEp arrived. taosHashClear(pMeta->updateInfo.pTasks); } else { - tqDebug("s-task:%s recv trans to update nodeEp from mnode, transId:%d", pTask->id.idStr, req.transId); + tqDebug("s-task:%s recv trans to update nodeEp from mnode, transId:%d", idstr, req.transId); } STaskUpdateEntry entry = {.streamId = req.streamId, .taskId = req.taskId, .transId = req.transId}; - void* exist = taosHashGet(pMeta->updateInfo.pTasks, &entry, sizeof(STaskUpdateEntry)); - if (exist != NULL) { - tqDebug("s-task:%s (vgId:%d) already update in trans:%d, discard the nodeEp update msg", pTask->id.idStr, vgId, - req.transId); + + void* pReqTask = taosHashGet(pMeta->updateInfo.pTasks, &entry, sizeof(STaskUpdateEntry)); + if (pReqTask != NULL) { + tqDebug("s-task:%s (vgId:%d) already update in trans:%d, discard the nodeEp update msg", idstr, vgId, req.transId); rsp.code = TSDB_CODE_SUCCESS; streamMetaWUnLock(pMeta); taosArrayDestroy(req.pNodeList); @@ -171,7 +171,7 @@ int32_t tqStreamTaskProcessUpdateReq(SStreamMeta* pMeta, SMsgCb* cb, SRpcMsg* pM 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); + vgId, req.taskId); CLEAR_RELATED_FILLHISTORY_TASK(pTask); } else { tqDebug("s-task:%s fill-history task update nodeEp along with stream task", (*ppHTask)->id.idStr); @@ -180,7 +180,7 @@ int32_t tqStreamTaskProcessUpdateReq(SStreamMeta* pMeta, SMsgCb* cb, SRpcMsg* pM } if (restored) { - tqDebug("s-task:%s vgId:%d start to save task", pTask->id.idStr, vgId); + tqDebug("s-task:%s vgId:%d start to save task", idstr, vgId); streamMetaSaveTask(pMeta, pTask); if (ppHTask != NULL) { streamMetaSaveTask(pMeta, *ppHTask); @@ -192,10 +192,10 @@ int32_t tqStreamTaskProcessUpdateReq(SStreamMeta* pMeta, SMsgCb* cb, SRpcMsg* pM } #endif } else { - tqDebug("s-task:%s vgId:%d not save since restore not finish", pTask->id.idStr, vgId); + tqDebug("s-task:%s vgId:%d not save since restore not finish", idstr, vgId); } - tqDebug("s-task:%s vgId:%d start to stop task after save task", pTask->id.idStr, vgId); + tqDebug("s-task:%s vgId:%d start to stop task after save task", idstr, vgId); streamTaskStop(pTask); // keep the already handled info @@ -206,11 +206,11 @@ int32_t tqStreamTaskProcessUpdateReq(SStreamMeta* pMeta, SMsgCb* cb, SRpcMsg* pM int64_t now = taosGetTimestampMs(); tqDebug("s-task:%s vgId:%d task nodeEp update completed, streamTask/fill-history closed, elapsed:%" PRId64 " ms", - pTask->id.idStr, vgId, now - st); + idstr, vgId, now - st); taosHashPut(pMeta->updateInfo.pTasks, &(*ppHTask)->id, sizeof(pTask->id), NULL, 0); } else { int64_t now = taosGetTimestampMs(); - tqDebug("s-task:%s vgId:%d, task nodeEp update completed, streamTask closed, elapsed time:%" PRId64 "ms", pTask->id.idStr, + tqDebug("s-task:%s vgId:%d, task nodeEp update completed, streamTask closed, elapsed time:%" PRId64 "ms", idstr, vgId, now - st); } @@ -237,6 +237,7 @@ int32_t tqStreamTaskProcessUpdateReq(SStreamMeta* pMeta, SMsgCb* cb, SRpcMsg* pM // for test purpose, to trigger the leader election taosMSleep(5000); #endif + tqStreamTaskStartAsync(pMeta, cb, true); streamMetaWUnLock(pMeta); } @@ -759,17 +760,24 @@ static int32_t restartStreamTasks(SStreamMeta* pMeta, bool isLeader) { } streamMetaWLock(pMeta); - streamMetaClear(pMeta); +// streamMetaClear(pMeta); int64_t el = taosGetTimestampMs() - st; tqInfo("vgId:%d close&reload state elapsed time:%.3fs", vgId, el / 1000.); - code = streamMetaLoadAllTasks(pMeta); - if (code != TSDB_CODE_SUCCESS) { - tqError("vgId:%d failed to load stream tasks, code:%s", vgId, tstrerror(terrno)); - streamMetaWUnLock(pMeta); - code = terrno; - return code; +// code = streamMetaLoadAllTasks(pMeta); +// if (code != TSDB_CODE_SUCCESS) { +// tqError("vgId:%d failed to load stream tasks, code:%s", vgId, tstrerror(terrno)); +// streamMetaWUnLock(pMeta); +// code = terrno; +// return code; +// } + + { + STaskStartInfo* pStartInfo = &pMeta->startInfo; + taosHashClear(pStartInfo->pReadyTaskSet); + taosHashClear(pStartInfo->pFailedTaskSet); + pStartInfo->readyTs = 0; } if (isLeader && !tsDisableStream) { From 70c1e66959872c0fd0c0b75bc032a2a3b1bb8862 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 8 Jan 2024 18:19:31 +0800 Subject: [PATCH 033/102] fix(stream): release task after launch stream tasks. --- 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 7e69610f80..6616fddaed 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -1517,6 +1517,8 @@ int32_t streamMetaStartOneTask(SStreamMeta* pMeta, int64_t streamId, int32_t tas if (pTask == NULL) { stError("vgId:%d failed to acquire task:0x%x during start tasks", pMeta->vgId, taskId); streamMetaUpdateTaskDownstreamStatus(pMeta, streamId, taskId, 0, taosGetTimestampMs(), false); + + streamMetaReleaseTask(pMeta, pTask); return TSDB_CODE_STREAM_TASK_IVLD_STATUS; } @@ -1542,5 +1544,6 @@ int32_t streamMetaStartOneTask(SStreamMeta* pMeta, int64_t streamId, int32_t tas } } + streamMetaReleaseTask(pMeta, pTask); return ret; } \ No newline at end of file From af9dc441b20e646882a8cbb4a23bfb34f9518f8f Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 8 Jan 2024 18:40:06 +0800 Subject: [PATCH 034/102] fix(stream): reset the saved value for fill-history task. --- include/libs/executor/executor.h | 1 + include/libs/stream/tstream.h | 1 + source/dnode/vnode/src/tqCommon/tqCommon.c | 13 ++---- source/libs/executor/src/executor.c | 51 ++++++++++++++++++++++ source/libs/stream/src/streamStart.c | 5 +++ 5 files changed, 62 insertions(+), 9 deletions(-) diff --git a/include/libs/executor/executor.h b/include/libs/executor/executor.h index f78b7a3126..be11d04ff8 100644 --- a/include/libs/executor/executor.h +++ b/include/libs/executor/executor.h @@ -210,6 +210,7 @@ void* qExtractReaderFromStreamScanner(void* scanner); int32_t qExtractStreamScanner(qTaskInfo_t tinfo, void** scanner); int32_t qSetStreamOperatorOptionForScanHistory(qTaskInfo_t tinfo); +int32_t qResetStreamOperatorOptionForScanHistory(qTaskInfo_t tinfo); int32_t qStreamSourceScanParamForHistoryScanStep1(qTaskInfo_t tinfo, SVersionRange *pVerRange, STimeWindow* pWindow); int32_t qStreamSourceScanParamForHistoryScanStep2(qTaskInfo_t tinfo, SVersionRange *pVerRange, STimeWindow* pWindow); int32_t qStreamRecoverFinish(qTaskInfo_t tinfo); diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index dbcdef57b0..d11a4ad23b 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -825,6 +825,7 @@ int32_t streamQueueGetNumOfItems(const SStreamQueue* pQueue); // common int32_t streamRestoreParam(SStreamTask* pTask); +int32_t streamResetParamForScanHistory(SStreamTask* pTask); void streamTaskPause(SStreamTask* pTask, SStreamMeta* pMeta); void streamTaskResume(SStreamTask* pTask); int32_t streamTaskSetUpstreamInfo(SStreamTask* pTask, const SStreamTask* pUpstreamTask); diff --git a/source/dnode/vnode/src/tqCommon/tqCommon.c b/source/dnode/vnode/src/tqCommon/tqCommon.c index 5467e3dadd..2e83cd5bc5 100644 --- a/source/dnode/vnode/src/tqCommon/tqCommon.c +++ b/source/dnode/vnode/src/tqCommon/tqCommon.c @@ -728,6 +728,10 @@ int32_t tqStreamTaskResetStatus(SStreamMeta* pMeta, int32_t* numOfTasks) { STaskId id = {.streamId = pTaskId->streamId, .taskId = pTaskId->taskId}; SStreamTask** pTask = taosHashGet(pMeta->pTasksMap, &id, sizeof(id)); streamTaskResetStatus(*pTask); + + if ((*pTask)->info.fillHistory == 1) { + streamResetParamForScanHistory(*pTask); + } } return 0; @@ -760,19 +764,10 @@ static int32_t restartStreamTasks(SStreamMeta* pMeta, bool isLeader) { } streamMetaWLock(pMeta); -// streamMetaClear(pMeta); int64_t el = taosGetTimestampMs() - st; tqInfo("vgId:%d close&reload state elapsed time:%.3fs", vgId, el / 1000.); -// code = streamMetaLoadAllTasks(pMeta); -// if (code != TSDB_CODE_SUCCESS) { -// tqError("vgId:%d failed to load stream tasks, code:%s", vgId, tstrerror(terrno)); -// streamMetaWUnLock(pMeta); -// code = terrno; -// return code; -// } - { STaskStartInfo* pStartInfo = &pMeta->startInfo; taosHashClear(pStartInfo->pReadyTaskSet); diff --git a/source/libs/executor/src/executor.c b/source/libs/executor/src/executor.c index 6cee79bff2..c29fae0cef 100644 --- a/source/libs/executor/src/executor.c +++ b/source/libs/executor/src/executor.c @@ -1023,6 +1023,57 @@ int32_t qSetStreamOperatorOptionForScanHistory(qTaskInfo_t tinfo) { return 0; } +int32_t qResetStreamOperatorOptionForScanHistory(qTaskInfo_t tinfo) { + SExecTaskInfo* pTaskInfo = (SExecTaskInfo*)tinfo; + SOperatorInfo* pOperator = pTaskInfo->pRoot; + + while (1) { + int32_t type = pOperator->operatorType; + if (type == QUERY_NODE_PHYSICAL_PLAN_STREAM_INTERVAL || type == QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_INTERVAL || + type == QUERY_NODE_PHYSICAL_PLAN_STREAM_FINAL_INTERVAL) { + SStreamIntervalOperatorInfo* pInfo = pOperator->info; + STimeWindowAggSupp* pSup = &pInfo->twAggSup; + + pSup->calTriggerSaved = 0; + pSup->deleteMarkSaved = 0; + qInfo("reset stream param for interval: %d, %" PRId64, pSup->calTrigger, pSup->deleteMark); + + } else if (type == QUERY_NODE_PHYSICAL_PLAN_STREAM_SESSION || + type == QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_SESSION || + type == QUERY_NODE_PHYSICAL_PLAN_STREAM_FINAL_SESSION) { + SStreamSessionAggOperatorInfo* pInfo = pOperator->info; + STimeWindowAggSupp* pSup = &pInfo->twAggSup; + + pSup->calTriggerSaved = 0; + pSup->deleteMarkSaved = 0; + qInfo("reset stream param for session: %d, %" PRId64, pSup->calTrigger, pSup->deleteMark); + + } else if (type == QUERY_NODE_PHYSICAL_PLAN_STREAM_STATE) { + SStreamStateAggOperatorInfo* pInfo = pOperator->info; + STimeWindowAggSupp* pSup = &pInfo->twAggSup; + + pSup->calTriggerSaved = 0; + pSup->deleteMarkSaved = 0; + qInfo("reset stream param for state: %d, %" PRId64, pSup->calTrigger, pSup->deleteMark); + + } else if (type == QUERY_NODE_PHYSICAL_PLAN_STREAM_EVENT) { + SStreamEventAggOperatorInfo* pInfo = pOperator->info; + STimeWindowAggSupp* pSup = &pInfo->twAggSup; + + pSup->calTriggerSaved = 0; + pSup->deleteMarkSaved = 0; + qInfo("save stream param for state: %d, %" PRId64, pSup->calTrigger, pSup->deleteMark); + } + + // iterate operator tree + if (pOperator->numOfDownstream != 1 || pOperator->pDownstream[0] == NULL) { + return 0; + } else { + pOperator = pOperator->pDownstream[0]; + } + } +} + int32_t qRestoreStreamOperatorOption(qTaskInfo_t tinfo) { SExecTaskInfo* pTaskInfo = (SExecTaskInfo*)tinfo; const char* id = GET_TASKID(pTaskInfo); diff --git a/source/libs/stream/src/streamStart.c b/source/libs/stream/src/streamStart.c index 0dcc94a530..0c2993e296 100644 --- a/source/libs/stream/src/streamStart.c +++ b/source/libs/stream/src/streamStart.c @@ -544,6 +544,11 @@ int32_t streamSetParamForScanHistory(SStreamTask* pTask) { return qSetStreamOperatorOptionForScanHistory(pTask->exec.pExecutor); } +int32_t streamResetParamForScanHistory(SStreamTask* pTask) { + stDebug("s-task:%s reset operator option for scan-history data", pTask->id.idStr); + return qResetStreamOperatorOptionForScanHistory(pTask->exec.pExecutor); +} + int32_t streamRestoreParam(SStreamTask* pTask) { stDebug("s-task:%s restore operator param after scan-history", pTask->id.idStr); return qRestoreStreamOperatorOption(pTask->exec.pExecutor); From bbaaad6f1f65666beb6a0a70d99b4b48b719b43a Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 8 Jan 2024 18:45:36 +0800 Subject: [PATCH 035/102] fix(stream): add null ptr check. --- source/libs/stream/src/streamStart.c | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/source/libs/stream/src/streamStart.c b/source/libs/stream/src/streamStart.c index 0c2993e296..52c3c87431 100644 --- a/source/libs/stream/src/streamStart.c +++ b/source/libs/stream/src/streamStart.c @@ -154,8 +154,7 @@ int32_t streamTaskStartScanHistory(SStreamTask* pTask) { int32_t level = pTask->info.taskLevel; ETaskStatus status = streamTaskGetStatus(pTask)->state; - ASSERT(pTask->status.downstreamReady == 1 && - ((status == TASK_STATUS__SCAN_HISTORY)/* || (status == TASK_STATUS__STREAM_SCAN_HISTORY)*/)); + ASSERT((pTask->status.downstreamReady == 1) && (status == TASK_STATUS__SCAN_HISTORY)); if (level == TASK_LEVEL__SOURCE) { return doStartScanHistoryTask(pTask); @@ -546,7 +545,11 @@ int32_t streamSetParamForScanHistory(SStreamTask* pTask) { int32_t streamResetParamForScanHistory(SStreamTask* pTask) { stDebug("s-task:%s reset operator option for scan-history data", pTask->id.idStr); - return qResetStreamOperatorOptionForScanHistory(pTask->exec.pExecutor); + if (pTask->exec.pExecutor != NULL) { + return qResetStreamOperatorOptionForScanHistory(pTask->exec.pExecutor); + } else { + return TSDB_CODE_SUCCESS; + } } int32_t streamRestoreParam(SStreamTask* pTask) { From f9d5c0d4039e8e00e685b8dd24f629c68f8268a7 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 9 Jan 2024 09:13:53 +0800 Subject: [PATCH 036/102] fix(stream): fix error in tq timer. --- source/dnode/mgmt/node_mgmt/src/dmEnv.c | 6 +----- source/dnode/mgmt/node_mgmt/src/dmMgmt.c | 5 ++++- source/dnode/vnode/src/inc/tq.h | 2 +- source/dnode/vnode/src/tq/tq.c | 14 +++++++------- source/libs/stream/inc/streamInt.h | 3 +++ source/libs/stream/src/stream.c | 5 +++++ source/libs/stream/src/streamMeta.c | 24 +++++++++++++----------- 7 files changed, 34 insertions(+), 25 deletions(-) diff --git a/source/dnode/mgmt/node_mgmt/src/dmEnv.c b/source/dnode/mgmt/node_mgmt/src/dmEnv.c index f9bba19fbb..d173c83e97 100644 --- a/source/dnode/mgmt/node_mgmt/src/dmEnv.c +++ b/source/dnode/mgmt/node_mgmt/src/dmEnv.c @@ -28,9 +28,6 @@ } \ } while (0) -extern int32_t streamTimerInit(); -extern void streamTimerCleanUp(); - static SDnode globalDnode = {0}; SDnode *dmInstance() { return &globalDnode; } @@ -169,7 +166,6 @@ int32_t dmInit() { #if defined(USE_S3) if (s3Begin() != 0) return -1; #endif - if (streamTimerInit() != 0) return -1; dInfo("dnode env is initialized"); return 0; @@ -196,10 +192,10 @@ void dmCleanup() { udfStopUdfd(); taosStopCacheRefreshWorker(); dmDiskClose(); + #if defined(USE_S3) s3End(); #endif - streamTimerCleanUp(); dInfo("dnode env is cleaned up"); diff --git a/source/dnode/mgmt/node_mgmt/src/dmMgmt.c b/source/dnode/mgmt/node_mgmt/src/dmMgmt.c index 84465640c0..070d78fd69 100644 --- a/source/dnode/mgmt/node_mgmt/src/dmMgmt.c +++ b/source/dnode/mgmt/node_mgmt/src/dmMgmt.c @@ -90,6 +90,7 @@ int32_t dmInitDnode(SDnode *pDnode) { goto _OVER; } #endif + indexInit(tsNumOfCommitThreads); streamMetaInit(); @@ -108,7 +109,9 @@ _OVER: } void dmCleanupDnode(SDnode *pDnode) { - if (pDnode == NULL) return; + if (pDnode == NULL) { + return; + } dmCleanupClient(pDnode); dmCleanupStatusClient(pDnode); diff --git a/source/dnode/vnode/src/inc/tq.h b/source/dnode/vnode/src/inc/tq.h index e4c673f533..435b12bc06 100644 --- a/source/dnode/vnode/src/inc/tq.h +++ b/source/dnode/vnode/src/inc/tq.h @@ -42,7 +42,6 @@ extern "C" { // clang-format on typedef struct STqOffsetStore STqOffsetStore; -extern void* tqTimer; #define IS_OFFSET_RESET_TYPE(_t) ((_t) < 0) @@ -108,6 +107,7 @@ struct STQ { TTB* pExecStore; TTB* pCheckStore; SStreamMeta* pStreamMeta; + void* tqTimer; }; int32_t tEncodeSTqHandle(SEncoder* pEncoder, const STqHandle* pHandle); diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 862d7e1d66..b3cb3c80f7 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -26,16 +26,16 @@ static FORCE_INLINE bool tqIsHandleExec(STqHandle* pHandle) { return TMQ_HANDLE_ static FORCE_INLINE void tqSetHandleExec(STqHandle* pHandle) { pHandle->status = TMQ_HANDLE_STATUS_EXEC; } static FORCE_INLINE void tqSetHandleIdle(STqHandle* pHandle) { pHandle->status = TMQ_HANDLE_STATUS_IDLE; } -int32_t tqTimerInit() { - tqTimer = taosTmrInit(100, 100, 1000, "TQ"); - if (tqTimer == NULL) { +static int32_t tqTimerInit(STQ* pTq) { + pTq->tqTimer = taosTmrInit(100, 100, 1000, "TQ"); + if (pTq->tqTimer == NULL) { return -1; } return 0; } -void tqTimerCleanUp() { - taosTmrCleanUp(tqTimer); +static void tqTimerCleanUp(STQ* pTq) { + taosTmrCleanUp(pTq->tqTimer); } void tqDestroyTqHandle(void* data) { @@ -118,7 +118,7 @@ int32_t tqInitialize(STQ* pTq) { return -1; } - tqTimerInit(); + tqTimerInit(pTq); return 0; } @@ -149,7 +149,7 @@ void tqClose(STQ* pTq) { taosMemoryFree(pTq->path); tqMetaClose(pTq); streamMetaClose(pTq->pStreamMeta); - tqTimerCleanUp(); + tqTimerCleanUp(pTq); qDebug("end to close tq"); taosMemoryFree(pTq); diff --git a/source/libs/stream/inc/streamInt.h b/source/libs/stream/inc/streamInt.h index 9b5134d449..1d1eda74f6 100644 --- a/source/libs/stream/inc/streamInt.h +++ b/source/libs/stream/inc/streamInt.h @@ -92,6 +92,9 @@ extern int32_t streamBackendId; extern int32_t streamBackendCfWrapperId; extern int32_t taskDbWrapperId; +int32_t streamTimerInit(); +void streamTimerCleanUp(); + void streamRetryDispatchData(SStreamTask* pTask, int64_t waitDuration); int32_t streamDispatchStreamBlock(SStreamTask* pTask); void destroyDispatchMsg(SStreamDispatchReq* pReq, int32_t numOfVgroups); diff --git a/source/libs/stream/src/stream.c b/source/libs/stream/src/stream.c index 10b1b3fbbe..7db4bec6a0 100644 --- a/source/libs/stream/src/stream.c +++ b/source/libs/stream/src/stream.c @@ -21,13 +21,18 @@ void* streamTimer = NULL; int32_t streamTimerInit() { streamTimer = taosTmrInit(1000, 100, 10000, "STREAM"); if (streamTimer == NULL) { + stError("init stream timer failed, code:%s", tstrerror(terrno)); return -1; } + + stInfo("init stream timer, %p", streamTimer); return 0; } void streamTimerCleanUp() { + stInfo("cleanup stream timer, %p", streamTimer); taosTmrCleanUp(streamTimer); + streamTimer = NULL; } char* createStreamTaskIdStr(int64_t streamId, int32_t taskId) { diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 6616fddaed..f687841016 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -23,7 +23,7 @@ #include "ttimer.h" #include "wal.h" -static TdThreadOnce streamMetaModuleInit = PTHREAD_ONCE_INIT; +TdThreadOnce streamMetaModuleInit = PTHREAD_ONCE_INIT; int32_t streamBackendId = 0; int32_t streamBackendCfWrapperId = 0; @@ -61,9 +61,10 @@ static void streamMetaEnvInit() { streamMetaId = taosOpenRef(64, streamMetaCloseImpl); metaRefMgtInit(); + streamTimerInit(); } -void streamMetaInit() { taosThreadOnce(&streamMetaModuleInit, streamMetaEnvInit); } +void streamMetaInit() { /*taosThreadOnce(&streamMetaModuleInit, streamMetaEnvInit);*/ streamMetaEnvInit();} void streamMetaCleanup() { taosCloseRef(streamBackendId); @@ -71,6 +72,7 @@ void streamMetaCleanup() { taosCloseRef(streamMetaId); metaRefMgtCleanup(); + streamTimerCleanUp(); } void metaRefMgtInit() { @@ -391,7 +393,7 @@ SStreamMeta* streamMetaOpen(const char* path, void* ahandle, FTaskExpand expandF memcpy(pRid, &pMeta->rid, sizeof(pMeta->rid)); metaRefMgtAdd(pMeta->vgId, pRid); - pMeta->pHbInfo->hbTmr = taosTmrStart(metaHbToMnode, META_HB_CHECK_INTERVAL, pRid, streamTimer); +// pMeta->pHbInfo->hbTmr = taosTmrStart(metaHbToMnode, META_HB_CHECK_INTERVAL, pRid, streamTimer); pMeta->pHbInfo->tickCounter = 0; pMeta->pHbInfo->stopFlag = 0; pMeta->qHandle = taosInitScheduler(32, 1, "stream-chkp", NULL); @@ -1202,7 +1204,7 @@ void metaHbToMnode(void* param, void* tmrId) { } if (!waitForEnoughDuration(pMeta->pHbInfo)) { - taosTmrReset(metaHbToMnode, META_HB_CHECK_INTERVAL, param, streamTimer, &pMeta->pHbInfo->hbTmr); +// taosTmrReset(metaHbToMnode, META_HB_CHECK_INTERVAL, param, streamTimer, &pMeta->pHbInfo->hbTmr); taosReleaseRef(streamMetaId, rid); return; } @@ -1212,7 +1214,7 @@ void metaHbToMnode(void* param, void* tmrId) { metaHeartbeatToMnodeImpl(pMeta); streamMetaRUnLock(pMeta); - taosTmrReset(metaHbToMnode, META_HB_CHECK_INTERVAL, param, streamTimer, &pMeta->pHbInfo->hbTmr); +// taosTmrReset(metaHbToMnode, META_HB_CHECK_INTERVAL, param, streamTimer, &pMeta->pHbInfo->hbTmr); taosReleaseRef(streamMetaId, rid); } @@ -1262,11 +1264,11 @@ void streamMetaNotifyClose(SStreamMeta* pMeta) { // wait for the stream meta hb function stopping if (pMeta->role == NODE_ROLE_LEADER) { - pMeta->pHbInfo->stopFlag = STREAM_META_WILL_STOP; - while (pMeta->pHbInfo->stopFlag != STREAM_META_OK_TO_STOP) { - taosMsleep(100); - stDebug("vgId:%d wait for meta to stop timer", pMeta->vgId); - } +// pMeta->pHbInfo->stopFlag = STREAM_META_WILL_STOP; +// while (pMeta->pHbInfo->stopFlag != STREAM_META_OK_TO_STOP) { +// taosMsleep(100); +// stDebug("vgId:%d wait for meta to stop timer", pMeta->vgId); +// } } stDebug("vgId:%d start to check all tasks", vgId); @@ -1285,7 +1287,7 @@ void streamMetaStartHb(SStreamMeta* pMeta) { int64_t* pRid = taosMemoryMalloc(sizeof(int64_t)); metaRefMgtAdd(pMeta->vgId, pRid); *pRid = pMeta->rid; - metaHbToMnode(pRid, NULL); +// metaHbToMnode(pRid, NULL); } void streamMetaResetStartInfo(STaskStartInfo* pStartInfo) { From 7979d4532bfd9ce3e0c6291e7bd80ad3eec20ed9 Mon Sep 17 00:00:00 2001 From: Benguang Zhao Date: Tue, 9 Jan 2024 16:49:27 +0800 Subject: [PATCH 037/102] enh: let max num of concurrent msgs of snap replication be configurable --- include/common/tglobal.h | 1 + source/common/src/tglobal.c | 5 +++++ source/libs/sync/src/syncSnapshot.c | 3 ++- 3 files changed, 8 insertions(+), 1 deletion(-) diff --git a/include/common/tglobal.h b/include/common/tglobal.h index f006779a48..8f8ddc632f 100644 --- a/include/common/tglobal.h +++ b/include/common/tglobal.h @@ -74,6 +74,7 @@ extern int64_t tsRpcQueueMemoryAllowed; extern int32_t tsElectInterval; extern int32_t tsHeartbeatInterval; extern int32_t tsHeartbeatTimeout; +extern int32_t tsSnapReplicationMsgNLimit; // vnode extern int64_t tsVndCommitMaxIntervalMs; diff --git a/source/common/src/tglobal.c b/source/common/src/tglobal.c index 79d21955d4..5228226235 100644 --- a/source/common/src/tglobal.c +++ b/source/common/src/tglobal.c @@ -72,6 +72,7 @@ int32_t tsPQSortMemThreshold = 16; // M int32_t tsElectInterval = 25 * 1000; int32_t tsHeartbeatInterval = 1000; int32_t tsHeartbeatTimeout = 20 * 1000; +int32_t tsSnapReplicationMsgNLimit = 128; // mnode int64_t tsMndSdbWriteDelta = 200; @@ -673,6 +674,9 @@ static int32_t taosAddServerCfg(SConfig *pCfg) { if (cfgAddInt32(pCfg, "syncHeartbeatTimeout", tsHeartbeatTimeout, 10, 1000 * 60 * 24 * 2, CFG_SCOPE_SERVER, CFG_DYN_NONE) != 0) return -1; + if (cfgAddInt32(pCfg, "syncSnapReplicationMsgNLimit", tsSnapReplicationMsgNLimit, 16, + (TSDB_SYNC_SNAP_BUFFER_SIZE >> 2), CFG_SCOPE_SERVER, CFG_DYN_NONE) != 0) + return -1; if (cfgAddInt64(pCfg, "mndSdbWriteDelta", tsMndSdbWriteDelta, 20, 10000, CFG_SCOPE_SERVER, CFG_DYN_ENT_SERVER) != 0) return -1; @@ -1183,6 +1187,7 @@ static int32_t taosSetServerCfg(SConfig *pCfg) { tsElectInterval = cfgGetItem(pCfg, "syncElectInterval")->i32; tsHeartbeatInterval = cfgGetItem(pCfg, "syncHeartbeatInterval")->i32; tsHeartbeatTimeout = cfgGetItem(pCfg, "syncHeartbeatTimeout")->i32; + tsSnapReplicationMsgNLimit = cfgGetItem(pCfg, "syncSnapReplicationMsgNLimit")->i32; tsMndSdbWriteDelta = cfgGetItem(pCfg, "mndSdbWriteDelta")->i64; tsMndLogRetention = cfgGetItem(pCfg, "mndLogRetention")->i64; diff --git a/source/libs/sync/src/syncSnapshot.c b/source/libs/sync/src/syncSnapshot.c index f0e457ef8d..564d28b0f4 100644 --- a/source/libs/sync/src/syncSnapshot.c +++ b/source/libs/sync/src/syncSnapshot.c @@ -22,6 +22,7 @@ #include "syncRaftStore.h" #include "syncReplication.h" #include "syncUtil.h" +#include "tglobal.h" static SyncIndex syncNodeGetSnapBeginIndex(SSyncNode *ths); @@ -1186,7 +1187,7 @@ static int32_t syncSnapBufferSend(SSyncSnapshotSender *pSender, SyncSnapshotRsp pSndBuf->start = ack + 1; } - while (pSender->seq != SYNC_SNAPSHOT_SEQ_END && pSender->seq - pSndBuf->start < (pSndBuf->size >> 2)) { + while (pSender->seq != SYNC_SNAPSHOT_SEQ_END && pSender->seq - pSndBuf->start < tsSnapReplicationMsgNLimit) { if (snapshotSend(pSender) != 0) { code = terrno; goto _out; From dc0c5539a0f8fc2478bef1ba874f439f2d94790b Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 10 Jan 2024 08:48:49 +0800 Subject: [PATCH 038/102] fix(stream): remove invalid timer controllers. --- source/dnode/vnode/src/tq/tqStreamTask.c | 6 ++---- source/libs/stream/src/streamMeta.c | 10 +++++----- source/libs/stream/src/streamStart.c | 7 ++++--- source/libs/wal/src/walRead.c | 12 ++++++------ 4 files changed, 17 insertions(+), 18 deletions(-) diff --git a/source/dnode/vnode/src/tq/tqStreamTask.c b/source/dnode/vnode/src/tq/tqStreamTask.c index 95a4474460..2a600d08ef 100644 --- a/source/dnode/vnode/src/tq/tqStreamTask.c +++ b/source/dnode/vnode/src/tq/tqStreamTask.c @@ -19,8 +19,6 @@ #define MAX_REPEAT_SCAN_THRESHOLD 3 #define SCAN_WAL_IDLE_DURATION 100 -void* tqTimer = NULL; - static int32_t doScanWalForAllTasks(SStreamMeta* pStreamMeta, bool* pScanIdle); static int32_t setWalReaderStartOffset(SStreamTask* pTask, int32_t vgId); static bool handleFillhistoryScanComplete(SStreamTask* pTask, int64_t ver); @@ -98,9 +96,9 @@ int32_t tqScanWalInFuture(STQ* pTq, int32_t numOfTasks, int32_t idleDuration) { pParam->pTq = pTq; pParam->numOfTasks = numOfTasks; if (pMeta->scanInfo.scanTimer == NULL) { - pMeta->scanInfo.scanTimer = taosTmrStart(doStartScanWal, idleDuration, pParam, tqTimer); + pMeta->scanInfo.scanTimer = taosTmrStart(doStartScanWal, idleDuration, pParam, pTq->tqTimer); } else { - taosTmrReset(doStartScanWal, idleDuration, pParam, tqTimer, &pMeta->scanInfo.scanTimer); + taosTmrReset(doStartScanWal, idleDuration, pParam, pTq->tqTimer, &pMeta->scanInfo.scanTimer); } return TSDB_CODE_SUCCESS; diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index f687841016..15256ececf 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -23,7 +23,7 @@ #include "ttimer.h" #include "wal.h" -TdThreadOnce streamMetaModuleInit = PTHREAD_ONCE_INIT; +static TdThreadOnce streamMetaModuleInit = PTHREAD_ONCE_INIT; int32_t streamBackendId = 0; int32_t streamBackendCfWrapperId = 0; @@ -64,7 +64,7 @@ static void streamMetaEnvInit() { streamTimerInit(); } -void streamMetaInit() { /*taosThreadOnce(&streamMetaModuleInit, streamMetaEnvInit);*/ streamMetaEnvInit();} +void streamMetaInit() { taosThreadOnce(&streamMetaModuleInit, streamMetaEnvInit);} void streamMetaCleanup() { taosCloseRef(streamBackendId); @@ -393,7 +393,7 @@ SStreamMeta* streamMetaOpen(const char* path, void* ahandle, FTaskExpand expandF memcpy(pRid, &pMeta->rid, sizeof(pMeta->rid)); metaRefMgtAdd(pMeta->vgId, pRid); -// pMeta->pHbInfo->hbTmr = taosTmrStart(metaHbToMnode, META_HB_CHECK_INTERVAL, pRid, streamTimer); + pMeta->pHbInfo->hbTmr = taosTmrStart(metaHbToMnode, META_HB_CHECK_INTERVAL, pRid, streamTimer); pMeta->pHbInfo->tickCounter = 0; pMeta->pHbInfo->stopFlag = 0; pMeta->qHandle = taosInitScheduler(32, 1, "stream-chkp", NULL); @@ -1204,7 +1204,7 @@ void metaHbToMnode(void* param, void* tmrId) { } if (!waitForEnoughDuration(pMeta->pHbInfo)) { -// taosTmrReset(metaHbToMnode, META_HB_CHECK_INTERVAL, param, streamTimer, &pMeta->pHbInfo->hbTmr); + taosTmrReset(metaHbToMnode, META_HB_CHECK_INTERVAL, param, streamTimer, &pMeta->pHbInfo->hbTmr); taosReleaseRef(streamMetaId, rid); return; } @@ -1214,7 +1214,7 @@ void metaHbToMnode(void* param, void* tmrId) { metaHeartbeatToMnodeImpl(pMeta); streamMetaRUnLock(pMeta); -// taosTmrReset(metaHbToMnode, META_HB_CHECK_INTERVAL, param, streamTimer, &pMeta->pHbInfo->hbTmr); + taosTmrReset(metaHbToMnode, META_HB_CHECK_INTERVAL, param, streamTimer, &pMeta->pHbInfo->hbTmr); taosReleaseRef(streamMetaId, rid); } diff --git a/source/libs/stream/src/streamStart.c b/source/libs/stream/src/streamStart.c index 52c3c87431..c852215d16 100644 --- a/source/libs/stream/src/streamStart.c +++ b/source/libs/stream/src/streamStart.c @@ -345,16 +345,17 @@ int32_t streamTaskOnNormalTaskReady(SStreamTask* pTask) { SStreamTaskState* p = streamTaskGetStatus(pTask); ASSERT(p->state == TASK_STATUS__READY); + int8_t schedStatus = pTask->status.schedStatus; if (pTask->info.taskLevel == TASK_LEVEL__SOURCE) { int64_t startVer = walReaderGetCurrentVer(pTask->exec.pWalReader); if (startVer == -1) { startVer = pTask->chkInfo.nextProcessVer; } - stDebug("s-task:%s no need to scan-history data, status:%s, sched-status:%d, ready for data from wal ver:%" PRId64, - id, p->name, pTask->status.schedStatus, startVer); + stDebug("s-task:%s status:%s, sched-status:%d, ready for data from wal ver:%" PRId64, id, p->name, schedStatus, + startVer); } else { - stDebug("s-task:%s level:%d status:%s sched-status:%d", id, pTask->info.taskLevel, p->name, pTask->status.schedStatus); + stDebug("s-task:%s level:%d status:%s sched-status:%d", id, pTask->info.taskLevel, p->name, schedStatus); } return TSDB_CODE_SUCCESS; diff --git a/source/libs/wal/src/walRead.c b/source/libs/wal/src/walRead.c index 1660a0ecf0..6748d161ae 100644 --- a/source/libs/wal/src/walRead.c +++ b/source/libs/wal/src/walRead.c @@ -305,8 +305,8 @@ int32_t walFetchHead(SWalReader *pRead, int64_t ver) { } int32_t walSkipFetchBody(SWalReader *pRead) { - wDebug("vgId:%d, skip fetch body %" PRId64 ", first ver:%" PRId64 ", commit ver:%" PRId64 ", last ver:%" PRId64 - ", applied ver:%" PRId64", 0x%"PRIx64, + wDebug("vgId:%d, skip fetch body:%" PRId64 ", first:%" PRId64 ", commit:%" PRId64 ", last:%" PRId64 + ", applied:%" PRId64 ", 0x%" PRIx64, pRead->pWal->cfg.vgId, pRead->pHead->head.version, pRead->pWal->vers.firstVer, pRead->pWal->vers.commitVer, pRead->pWal->vers.lastVer, pRead->pWal->vers.appliedVer, pRead->readerId); @@ -325,11 +325,11 @@ int32_t walFetchBody(SWalReader *pRead) { int64_t ver = pReadHead->version; int32_t vgId = pRead->pWal->cfg.vgId; int64_t id = pRead->readerId; + SWalVer *pVer = &pRead->pWal->vers; - wDebug("vgId:%d, fetch body %" PRId64 ", first ver:%" PRId64 ", commit ver:%" PRId64 ", last ver:%" PRId64 - ", applied ver:%" PRId64 ", 0x%" PRIx64, - vgId, ver, pRead->pWal->vers.firstVer, pRead->pWal->vers.commitVer, pRead->pWal->vers.lastVer, - pRead->pWal->vers.appliedVer, id); + wDebug("vgId:%d, fetch body:%" PRId64 ", first:%" PRId64 ", commit:%" PRId64 ", last:%" PRId64 ", applied:%" PRId64 + ", 0x%" PRIx64, + vgId, ver, pVer->firstVer, pVer->commitVer, pVer->lastVer, pVer->appliedVer, id); if (pRead->capacity < pReadHead->bodyLen) { SWalCkHead *ptr = (SWalCkHead *)taosMemoryRealloc(pRead->pHead, sizeof(SWalCkHead) + pReadHead->bodyLen); From 9dbddeed16ad722f56735f4f993dfe02f121b588 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Wed, 10 Jan 2024 10:03:38 +0800 Subject: [PATCH 039/102] opt msg on mnd --- source/common/src/tglobal.c | 137 +++++++++++++------------- source/dnode/mnode/impl/src/mndMain.c | 39 ++++---- 2 files changed, 86 insertions(+), 90 deletions(-) diff --git a/source/common/src/tglobal.c b/source/common/src/tglobal.c index 79d21955d4..76e1ab46b7 100644 --- a/source/common/src/tglobal.c +++ b/source/common/src/tglobal.c @@ -95,9 +95,9 @@ int32_t tsMonitorMaxLogs = 100; bool tsMonitorComp = false; // audit -bool tsEnableAudit = true; -bool tsEnableAuditCreateTable = true; -int32_t tsAuditInterval = 5000; +bool tsEnableAudit = true; +bool tsEnableAuditCreateTable = true; +int32_t tsAuditInterval = 5000; // telem #ifdef TD_ENTERPRISE @@ -252,7 +252,7 @@ int32_t tsCompactPullupInterval = 10; int32_t tsMqRebalanceInterval = 2; int32_t tsStreamCheckpointInterval = 60; float tsSinkDataRate = 2.0; -int32_t tsStreamNodeCheckInterval = 15; +int32_t tsStreamNodeCheckInterval = 16; int32_t tsTtlUnit = 86400; int32_t tsTtlPushIntervalSec = 10; int32_t tsTrimVDbIntervalSec = 60 * 60; // interval of trimming db in all vgroups @@ -282,7 +282,7 @@ int32_t tsS3BlockCacheSize = 16; // number of blocks int32_t tsS3PageCacheSize = 4096; // number of pages int32_t tsS3UploadDelaySec = 60 * 60 * 24; -bool tsExperimental = true; +bool tsExperimental = true; #ifndef _STORAGE int32_t taosSetTfsCfg(SConfig *pCfg) { @@ -691,8 +691,7 @@ static int32_t taosAddServerCfg(SConfig *pCfg) { if (cfgAddBool(pCfg, "audit", tsEnableAudit, CFG_SCOPE_SERVER, CFG_DYN_ENT_SERVER) != 0) return -1; if (cfgAddBool(pCfg, "auditCreateTable", tsEnableAuditCreateTable, CFG_SCOPE_SERVER, CFG_DYN_NONE) != 0) return -1; - if (cfgAddInt32(pCfg, "auditInterval", tsAuditInterval, 500, 200000, CFG_SCOPE_SERVER, CFG_DYN_NONE) != 0) - return -1; + if (cfgAddInt32(pCfg, "auditInterval", tsAuditInterval, 500, 200000, CFG_SCOPE_SERVER, CFG_DYN_NONE) != 0) return -1; if (cfgAddBool(pCfg, "crashReporting", tsEnableCrashReport, CFG_SCOPE_BOTH, CFG_DYN_NONE) != 0) return -1; if (cfgAddBool(pCfg, "telemetryReporting", tsEnableTelem, CFG_SCOPE_BOTH, CFG_DYN_ENT_SERVER) != 0) return -1; @@ -711,8 +710,8 @@ static int32_t taosAddServerCfg(SConfig *pCfg) { if (cfgAddInt32(pCfg, "transPullupInterval", tsTransPullupInterval, 1, 10000, CFG_SCOPE_SERVER, CFG_DYN_ENT_SERVER) != 0) return -1; - if (cfgAddInt32(pCfg, "compactPullupInterval", tsCompactPullupInterval, 1, 10000, CFG_SCOPE_SERVER, CFG_DYN_ENT_SERVER) != - 0) + if (cfgAddInt32(pCfg, "compactPullupInterval", tsCompactPullupInterval, 1, 10000, CFG_SCOPE_SERVER, + CFG_DYN_ENT_SERVER) != 0) return -1; if (cfgAddInt32(pCfg, "mqRebalanceInterval", tsMqRebalanceInterval, 1, 10000, CFG_SCOPE_SERVER, CFG_DYN_ENT_SERVER) != 0) @@ -1465,40 +1464,38 @@ static int32_t taosCfgDynamicOptionsForServer(SConfig *pCfg, char *name) { {"stDebugFlag", &stDebugFlag}, {"sndDebugFlag", &sndDebugFlag}, }; - static OptionNameAndVar options[] = { - {"audit", &tsEnableAudit}, - {"asynclog", &tsAsyncLog}, - {"disableStream", &tsDisableStream}, - {"enableWhiteList", &tsEnableWhiteList}, - {"telemetryReporting", &tsEnableTelem}, - {"monitor", &tsEnableMonitor}, + static OptionNameAndVar options[] = {{"audit", &tsEnableAudit}, + {"asynclog", &tsAsyncLog}, + {"disableStream", &tsDisableStream}, + {"enableWhiteList", &tsEnableWhiteList}, + {"telemetryReporting", &tsEnableTelem}, + {"monitor", &tsEnableMonitor}, - {"mndSdbWriteDelta", &tsMndSdbWriteDelta}, - {"minDiskFreeSize", &tsMinDiskFreeSize}, + {"mndSdbWriteDelta", &tsMndSdbWriteDelta}, + {"minDiskFreeSize", &tsMinDiskFreeSize}, - {"cacheLazyLoadThreshold", &tsCacheLazyLoadThreshold}, - {"checkpointInterval", &tsStreamCheckpointInterval}, - {"keepAliveIdle", &tsKeepAliveIdle}, - {"logKeepDays", &tsLogKeepDays}, - {"maxStreamBackendCache", &tsMaxStreamBackendCache}, - {"mqRebalanceInterval", &tsMqRebalanceInterval}, - {"numOfLogLines", &tsNumOfLogLines}, - {"queryRspPolicy", &tsQueryRspPolicy}, - {"timeseriesThreshold", &tsTimeSeriesThreshold}, - {"tmqMaxTopicNum", &tmqMaxTopicNum}, - {"transPullupInterval", &tsTransPullupInterval}, - {"compactPullupInterval", &tsCompactPullupInterval}, - {"trimVDbIntervalSec", &tsTrimVDbIntervalSec}, - {"ttlBatchDropNum", &tsTtlBatchDropNum}, - {"ttlFlushThreshold", &tsTtlFlushThreshold}, - {"ttlPushInterval", &tsTtlPushIntervalSec}, - //{"s3BlockSize", &tsS3BlockSize}, - {"s3BlockCacheSize", &tsS3BlockCacheSize}, - {"s3PageCacheSize", &tsS3PageCacheSize}, - {"s3UploadDelaySec", &tsS3UploadDelaySec}, - {"supportVnodes", &tsNumOfSupportVnodes}, - {"experimental", &tsExperimental} - }; + {"cacheLazyLoadThreshold", &tsCacheLazyLoadThreshold}, + {"checkpointInterval", &tsStreamCheckpointInterval}, + {"keepAliveIdle", &tsKeepAliveIdle}, + {"logKeepDays", &tsLogKeepDays}, + {"maxStreamBackendCache", &tsMaxStreamBackendCache}, + {"mqRebalanceInterval", &tsMqRebalanceInterval}, + {"numOfLogLines", &tsNumOfLogLines}, + {"queryRspPolicy", &tsQueryRspPolicy}, + {"timeseriesThreshold", &tsTimeSeriesThreshold}, + {"tmqMaxTopicNum", &tmqMaxTopicNum}, + {"transPullupInterval", &tsTransPullupInterval}, + {"compactPullupInterval", &tsCompactPullupInterval}, + {"trimVDbIntervalSec", &tsTrimVDbIntervalSec}, + {"ttlBatchDropNum", &tsTtlBatchDropNum}, + {"ttlFlushThreshold", &tsTtlFlushThreshold}, + {"ttlPushInterval", &tsTtlPushIntervalSec}, + //{"s3BlockSize", &tsS3BlockSize}, + {"s3BlockCacheSize", &tsS3BlockCacheSize}, + {"s3PageCacheSize", &tsS3PageCacheSize}, + {"s3UploadDelaySec", &tsS3UploadDelaySec}, + {"supportVnodes", &tsNumOfSupportVnodes}, + {"experimental", &tsExperimental}}; if (taosCfgSetOption(debugOptions, tListLen(debugOptions), pItem, true) != 0) { taosCfgSetOption(options, tListLen(options), pItem, false); @@ -1692,36 +1689,34 @@ static int32_t taosCfgDynamicOptionsForClient(SConfig *pCfg, char *name) { {"cDebugFlag", &cDebugFlag}, {"dDebugFlag", &dDebugFlag}, {"fsDebugFlag", &fsDebugFlag}, {"idxDebugFlag", &idxDebugFlag}, {"jniDebugFlag", &jniDebugFlag}, {"qDebugFlag", &qDebugFlag}, {"rpcDebugFlag", &rpcDebugFlag}, {"smaDebugFlag", &smaDebugFlag}, {"tmrDebugFlag", &tmrDebugFlag}, - {"uDebugFlag", &uDebugFlag}, {"simDebugFlag", &simDebugFlag}, + {"uDebugFlag", &uDebugFlag}, {"simDebugFlag", &simDebugFlag}, }; - static OptionNameAndVar options[] = { - {"asyncLog", &tsAsyncLog}, - {"assert", &tsAssert}, - {"compressMsgSize", &tsCompressMsgSize}, - {"countAlwaysReturnValue", &tsCountAlwaysReturnValue}, - {"crashReporting", &tsEnableCrashReport}, - {"enableCoreFile", &tsAsyncLog}, - {"enableQueryHb", &tsEnableQueryHb}, - {"keepColumnName", &tsKeepColumnName}, - {"keepAliveIdle", &tsKeepAliveIdle}, - {"logKeepDays", &tsLogKeepDays}, - {"maxInsertBatchRows", &tsMaxInsertBatchRows}, - {"maxRetryWaitTime", &tsMaxRetryWaitTime}, - {"minSlidingTime", &tsMinSlidingTime}, - {"minIntervalTime", &tsMinIntervalTime}, - {"numOfLogLines", &tsNumOfLogLines}, - {"querySmaOptimize", &tsQuerySmaOptimize}, - {"queryPolicy", &tsQueryPolicy}, - {"queryPlannerTrace", &tsQueryPlannerTrace}, - {"queryNodeChunkSize", &tsQueryNodeChunkSize}, - {"queryUseNodeAllocator", &tsQueryUseNodeAllocator}, - {"smlDot2Underline", &tsSmlDot2Underline}, - {"shellActivityTimer", &tsShellActivityTimer}, - {"slowLogThreshold", &tsSlowLogThreshold}, - {"useAdapter", &tsUseAdapter}, - {"experimental", &tsExperimental} - }; + static OptionNameAndVar options[] = {{"asyncLog", &tsAsyncLog}, + {"assert", &tsAssert}, + {"compressMsgSize", &tsCompressMsgSize}, + {"countAlwaysReturnValue", &tsCountAlwaysReturnValue}, + {"crashReporting", &tsEnableCrashReport}, + {"enableCoreFile", &tsAsyncLog}, + {"enableQueryHb", &tsEnableQueryHb}, + {"keepColumnName", &tsKeepColumnName}, + {"keepAliveIdle", &tsKeepAliveIdle}, + {"logKeepDays", &tsLogKeepDays}, + {"maxInsertBatchRows", &tsMaxInsertBatchRows}, + {"maxRetryWaitTime", &tsMaxRetryWaitTime}, + {"minSlidingTime", &tsMinSlidingTime}, + {"minIntervalTime", &tsMinIntervalTime}, + {"numOfLogLines", &tsNumOfLogLines}, + {"querySmaOptimize", &tsQuerySmaOptimize}, + {"queryPolicy", &tsQueryPolicy}, + {"queryPlannerTrace", &tsQueryPlannerTrace}, + {"queryNodeChunkSize", &tsQueryNodeChunkSize}, + {"queryUseNodeAllocator", &tsQueryUseNodeAllocator}, + {"smlDot2Underline", &tsSmlDot2Underline}, + {"shellActivityTimer", &tsShellActivityTimer}, + {"slowLogThreshold", &tsSlowLogThreshold}, + {"useAdapter", &tsUseAdapter}, + {"experimental", &tsExperimental}}; if (taosCfgSetOption(debugOptions, tListLen(debugOptions), pItem, true) != 0) { taosCfgSetOption(options, tListLen(options), pItem, false); @@ -1763,7 +1758,7 @@ static void taosCheckAndSetDebugFlag(int32_t *pFlagPtr, char *name, int32_t flag void taosSetAllDebugFlag(int32_t flag) { if (flag <= 0) return; - SArray *noNeedToSetVars = NULL; + SArray * noNeedToSetVars = NULL; SConfigItem *pItem = cfgGetItem(tsCfg, "debugFlag"); if (pItem != NULL) { pItem->i32 = flag; @@ -1798,7 +1793,7 @@ void taosSetAllDebugFlag(int32_t flag) { taosArrayClear(noNeedToSetVars); // reset array uInfo("all debug flag are set to %d", flag); - if (terrno == TSDB_CODE_CFG_NOT_FOUND) terrno = TSDB_CODE_SUCCESS; // ignore not exist + if (terrno == TSDB_CODE_CFG_NOT_FOUND) terrno = TSDB_CODE_SUCCESS; // ignore not exist } int8_t taosGranted() { return atomic_load_8(&tsGrant); } diff --git a/source/dnode/mnode/impl/src/mndMain.c b/source/dnode/mnode/impl/src/mndMain.c index c0fd93c65d..4bd0776ecd 100644 --- a/source/dnode/mnode/impl/src/mndMain.c +++ b/source/dnode/mnode/impl/src/mndMain.c @@ -107,7 +107,7 @@ static void *mndBuildCheckpointTickMsg(int32_t *pContLen, int64_t sec) { static void mndPullupTrans(SMnode *pMnode) { mTrace("pullup trans msg"); int32_t contLen = 0; - void *pReq = mndBuildTimerMsg(&contLen); + void * pReq = mndBuildTimerMsg(&contLen); if (pReq != NULL) { SRpcMsg rpcMsg = {.msgType = TDMT_MND_TRANS_TIMER, .pCont = pReq, .contLen = contLen}; tmsgPutToQueue(&pMnode->msgCb, WRITE_QUEUE, &rpcMsg); @@ -117,7 +117,7 @@ static void mndPullupTrans(SMnode *pMnode) { static void mndPullupCompacts(SMnode *pMnode) { mTrace("pullup compact timer msg"); int32_t contLen = 0; - void *pReq = mndBuildTimerMsg(&contLen); + void * pReq = mndBuildTimerMsg(&contLen); if (pReq != NULL) { SRpcMsg rpcMsg = {.msgType = TDMT_MND_COMPACT_TIMER, .pCont = pReq, .contLen = contLen}; tmsgPutToQueue(&pMnode->msgCb, WRITE_QUEUE, &rpcMsg); @@ -127,7 +127,7 @@ static void mndPullupCompacts(SMnode *pMnode) { static void mndPullupTtl(SMnode *pMnode) { mTrace("pullup ttl"); int32_t contLen = 0; - void *pReq = mndBuildTimerMsg(&contLen); + void * pReq = mndBuildTimerMsg(&contLen); SRpcMsg rpcMsg = {.msgType = TDMT_MND_TTL_TIMER, .pCont = pReq, .contLen = contLen}; tmsgPutToQueue(&pMnode->msgCb, WRITE_QUEUE, &rpcMsg); } @@ -135,14 +135,14 @@ static void mndPullupTtl(SMnode *pMnode) { static void mndPullupTrimDb(SMnode *pMnode) { mTrace("pullup trim"); int32_t contLen = 0; - void *pReq = mndBuildTimerMsg(&contLen); + void * pReq = mndBuildTimerMsg(&contLen); SRpcMsg rpcMsg = {.msgType = TDMT_MND_TRIM_DB_TIMER, .pCont = pReq, .contLen = contLen}; tmsgPutToQueue(&pMnode->msgCb, WRITE_QUEUE, &rpcMsg); } static void mndCalMqRebalance(SMnode *pMnode) { int32_t contLen = 0; - void *pReq = mndBuildTimerMsg(&contLen); + void * pReq = mndBuildTimerMsg(&contLen); if (pReq != NULL) { SRpcMsg rpcMsg = {.msgType = TDMT_MND_TMQ_TIMER, .pCont = pReq, .contLen = contLen}; tmsgPutToQueue(&pMnode->msgCb, READ_QUEUE, &rpcMsg); @@ -151,7 +151,7 @@ static void mndCalMqRebalance(SMnode *pMnode) { static void mndStreamCheckpointTick(SMnode *pMnode, int64_t sec) { int32_t contLen = 0; - void *pReq = mndBuildCheckpointTickMsg(&contLen, sec); + void * pReq = mndBuildCheckpointTickMsg(&contLen, sec); if (pReq != NULL) { SRpcMsg rpcMsg = {.msgType = TDMT_MND_STREAM_CHECKPOINT_TIMER, .pCont = pReq, .contLen = contLen}; tmsgPutToQueue(&pMnode->msgCb, READ_QUEUE, &rpcMsg); @@ -160,7 +160,7 @@ static void mndStreamCheckpointTick(SMnode *pMnode, int64_t sec) { static void mndStreamCheckpointRemain(SMnode *pMnode) { int32_t contLen = 0; - void *pReq = mndBuildCheckpointTickMsg(&contLen, 0); + void * pReq = mndBuildCheckpointTickMsg(&contLen, 0); if (pReq != NULL) { SRpcMsg rpcMsg = {.msgType = TDMT_MND_STREAM_CHECKPOINT_CANDIDITATE, .pCont = pReq, .contLen = contLen}; tmsgPutToQueue(&pMnode->msgCb, READ_QUEUE, &rpcMsg); @@ -169,7 +169,7 @@ static void mndStreamCheckpointRemain(SMnode *pMnode) { static void mndStreamCheckNode(SMnode *pMnode) { int32_t contLen = 0; - void *pReq = mndBuildTimerMsg(&contLen); + void * pReq = mndBuildTimerMsg(&contLen); if (pReq != NULL) { SRpcMsg rpcMsg = {.msgType = TDMT_MND_NODECHECK_TIMER, .pCont = pReq, .contLen = contLen}; tmsgPutToQueue(&pMnode->msgCb, READ_QUEUE, &rpcMsg); @@ -179,7 +179,7 @@ static void mndStreamCheckNode(SMnode *pMnode) { static void mndPullupTelem(SMnode *pMnode) { mTrace("pullup telem msg"); int32_t contLen = 0; - void *pReq = mndBuildTimerMsg(&contLen); + void * pReq = mndBuildTimerMsg(&contLen); if (pReq != NULL) { SRpcMsg rpcMsg = {.msgType = TDMT_MND_TELEM_TIMER, .pCont = pReq, .contLen = contLen}; tmsgPutToQueue(&pMnode->msgCb, READ_QUEUE, &rpcMsg); @@ -189,7 +189,7 @@ static void mndPullupTelem(SMnode *pMnode) { static void mndPullupGrant(SMnode *pMnode) { mTrace("pullup grant msg"); int32_t contLen = 0; - void *pReq = mndBuildTimerMsg(&contLen); + void * pReq = mndBuildTimerMsg(&contLen); if (pReq != NULL) { SRpcMsg rpcMsg = { .msgType = TDMT_MND_GRANT_HB_TIMER, .pCont = pReq, .contLen = contLen, .info.ahandle = (void *)0x9527}; @@ -200,7 +200,7 @@ static void mndPullupGrant(SMnode *pMnode) { static void mndIncreaseUpTime(SMnode *pMnode) { mTrace("increate uptime"); int32_t contLen = 0; - void *pReq = mndBuildTimerMsg(&contLen); + void * pReq = mndBuildTimerMsg(&contLen); if (pReq != NULL) { SRpcMsg rpcMsg = { .msgType = TDMT_MND_UPTIME_TIMER, .pCont = pReq, .contLen = contLen, .info.ahandle = (void *)0x9528}; @@ -254,7 +254,7 @@ static void mndCheckDnodeOffline(SMnode *pMnode) { mTrace("check dnode offline"); if (mndAcquireRpc(pMnode) != 0) return; - SSdb *pSdb = pMnode->pSdb; + SSdb * pSdb = pMnode->pSdb; int64_t curMs = taosGetTimestampMs(); void *pIter = NULL; @@ -299,14 +299,14 @@ static bool mnodeIsNotLeader(SMnode *pMnode) { } static int32_t minCronTime() { - int64_t min = INT64_MAX; + int32_t min = INT32_MAX; min = TMIN(min, tsTtlPushIntervalSec); min = TMIN(min, tsTrimVDbIntervalSec); min = TMIN(min, tsTransPullupInterval); min = TMIN(min, tsCompactPullupInterval); min = TMIN(min, tsMqRebalanceInterval); min = TMIN(min, tsStreamCheckpointInterval); - min = TMIN(min, 5); // checkpointRemain + min = TMIN(min, 6); // checkpointRemain min = TMIN(min, tsStreamNodeCheckInterval); int64_t telemInt = TMIN(60, (tsTelemInterval - 1)); @@ -386,7 +386,8 @@ static void *mndThreadFp(void *param) { int64_t minCron = minCronTime(); if (sec % minCron == 0 && mnodeIsNotLeader(pMnode)) { // not leader, do nothing - mTrace("timer not process since mnode is not leader, reason: %s", tstrerror(terrno)) terrno = 0; + mTrace("timer not process since mnode is not leader, reason: %s", tstrerror(terrno)); + terrno = 0; continue; } mndDoTimerPullupTask(pMnode, sec); @@ -700,7 +701,7 @@ void mndStop(SMnode *pMnode) { } int32_t mndProcessSyncMsg(SRpcMsg *pMsg) { - SMnode *pMnode = pMsg->info.node; + SMnode * pMnode = pMsg->info.node; SSyncMgmt *pMgmt = &pMnode->syncMgmt; const STraceId *trace = &pMsg->info.traceId; @@ -802,7 +803,7 @@ _OVER: } int32_t mndProcessRpcMsg(SRpcMsg *pMsg) { - SMnode *pMnode = pMsg->info.node; + SMnode * pMnode = pMsg->info.node; const STraceId *trace = &pMsg->info.traceId; MndMsgFp fp = pMnode->msgFp[TMSG_INDEX(pMsg->msgType)]; @@ -857,7 +858,7 @@ int32_t mndGetMonitorInfo(SMnode *pMnode, SMonClusterInfo *pClusterInfo, SMonVgr SMonStbInfo *pStbInfo, SMonGrantInfo *pGrantInfo) { if (mndAcquireRpc(pMnode) != 0) return -1; - SSdb *pSdb = pMnode->pSdb; + SSdb * pSdb = pMnode->pSdb; int64_t ms = taosGetTimestampMs(); pClusterInfo->dnodes = taosArrayInit(sdbGetSize(pSdb, SDB_DNODE), sizeof(SMonDnodeDesc)); @@ -941,7 +942,7 @@ int32_t mndGetMonitorInfo(SMnode *pMnode, SMonClusterInfo *pClusterInfo, SMonVgr pGrantInfo->timeseries_used += pVgroup->numOfTimeSeries; tstrncpy(desc.status, "unsynced", sizeof(desc.status)); for (int32_t i = 0; i < pVgroup->replica; ++i) { - SVnodeGid *pVgid = &pVgroup->vnodeGid[i]; + SVnodeGid * pVgid = &pVgroup->vnodeGid[i]; SMonVnodeDesc *pVnDesc = &desc.vnodes[i]; pVnDesc->dnode_id = pVgid->dnodeId; tstrncpy(pVnDesc->vnode_role, syncStr(pVgid->syncState), sizeof(pVnDesc->vnode_role)); From 485bad1ebd1fc90651b48b31687fb2f5533f8de1 Mon Sep 17 00:00:00 2001 From: Benguang Zhao Date: Wed, 10 Jan 2024 11:30:42 +0800 Subject: [PATCH 040/102] refact: use config syncSnapReplMaxWaitN instead --- include/common/tglobal.h | 2 +- source/common/src/tglobal.c | 6 +++--- source/libs/sync/src/syncSnapshot.c | 2 +- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/include/common/tglobal.h b/include/common/tglobal.h index 8f8ddc632f..f23bb4d51b 100644 --- a/include/common/tglobal.h +++ b/include/common/tglobal.h @@ -74,7 +74,7 @@ extern int64_t tsRpcQueueMemoryAllowed; extern int32_t tsElectInterval; extern int32_t tsHeartbeatInterval; extern int32_t tsHeartbeatTimeout; -extern int32_t tsSnapReplicationMsgNLimit; +extern int32_t tsSnapReplMaxWaitN; // vnode extern int64_t tsVndCommitMaxIntervalMs; diff --git a/source/common/src/tglobal.c b/source/common/src/tglobal.c index 5228226235..dd51c5267a 100644 --- a/source/common/src/tglobal.c +++ b/source/common/src/tglobal.c @@ -72,7 +72,7 @@ int32_t tsPQSortMemThreshold = 16; // M int32_t tsElectInterval = 25 * 1000; int32_t tsHeartbeatInterval = 1000; int32_t tsHeartbeatTimeout = 20 * 1000; -int32_t tsSnapReplicationMsgNLimit = 128; +int32_t tsSnapReplMaxWaitN = 128; // mnode int64_t tsMndSdbWriteDelta = 200; @@ -674,7 +674,7 @@ static int32_t taosAddServerCfg(SConfig *pCfg) { if (cfgAddInt32(pCfg, "syncHeartbeatTimeout", tsHeartbeatTimeout, 10, 1000 * 60 * 24 * 2, CFG_SCOPE_SERVER, CFG_DYN_NONE) != 0) return -1; - if (cfgAddInt32(pCfg, "syncSnapReplicationMsgNLimit", tsSnapReplicationMsgNLimit, 16, + if (cfgAddInt32(pCfg, "syncSnapReplMaxWaitN", tsSnapReplMaxWaitN, 16, (TSDB_SYNC_SNAP_BUFFER_SIZE >> 2), CFG_SCOPE_SERVER, CFG_DYN_NONE) != 0) return -1; @@ -1187,7 +1187,7 @@ static int32_t taosSetServerCfg(SConfig *pCfg) { tsElectInterval = cfgGetItem(pCfg, "syncElectInterval")->i32; tsHeartbeatInterval = cfgGetItem(pCfg, "syncHeartbeatInterval")->i32; tsHeartbeatTimeout = cfgGetItem(pCfg, "syncHeartbeatTimeout")->i32; - tsSnapReplicationMsgNLimit = cfgGetItem(pCfg, "syncSnapReplicationMsgNLimit")->i32; + tsSnapReplMaxWaitN = cfgGetItem(pCfg, "syncSnapReplMaxWaitN")->i32; tsMndSdbWriteDelta = cfgGetItem(pCfg, "mndSdbWriteDelta")->i64; tsMndLogRetention = cfgGetItem(pCfg, "mndLogRetention")->i64; diff --git a/source/libs/sync/src/syncSnapshot.c b/source/libs/sync/src/syncSnapshot.c index 564d28b0f4..578e6798e0 100644 --- a/source/libs/sync/src/syncSnapshot.c +++ b/source/libs/sync/src/syncSnapshot.c @@ -1187,7 +1187,7 @@ static int32_t syncSnapBufferSend(SSyncSnapshotSender *pSender, SyncSnapshotRsp pSndBuf->start = ack + 1; } - while (pSender->seq != SYNC_SNAPSHOT_SEQ_END && pSender->seq - pSndBuf->start < tsSnapReplicationMsgNLimit) { + while (pSender->seq != SYNC_SNAPSHOT_SEQ_END && pSender->seq - pSndBuf->start < tsSnapReplMaxWaitN) { if (snapshotSend(pSender) != 0) { code = terrno; goto _out; From e8aea4249816a7a2201439f7fa5828809fc2f8dc Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 9 Jan 2024 21:40:27 +0800 Subject: [PATCH 041/102] fix(tsdb): open reader during resume task for main reader after suspending reader. --- source/dnode/vnode/src/tsdb/tsdbRead2.c | 100 ++++++++++++++---------- 1 file changed, 59 insertions(+), 41 deletions(-) diff --git a/source/dnode/vnode/src/tsdb/tsdbRead2.c b/source/dnode/vnode/src/tsdb/tsdbRead2.c index 255d5d4105..3f11546cf9 100644 --- a/source/dnode/vnode/src/tsdb/tsdbRead2.c +++ b/source/dnode/vnode/src/tsdb/tsdbRead2.c @@ -676,6 +676,10 @@ static int32_t doCopyColVal(SColumnInfoData* pColInfoData, int32_t rowIndex, int } static SFileDataBlockInfo* getCurrentBlockInfo(SDataBlockIter* pBlockIter) { + if (pBlockIter->blockList == NULL) { + return NULL; + } + size_t num = TARRAY_SIZE(pBlockIter->blockList); if (num == 0) { ASSERT(pBlockIter->numOfBlocks == num); @@ -2054,30 +2058,34 @@ static int32_t initMemDataIterator(STableBlockScanInfo* pBlockScanInfo, STsdbRea return TSDB_CODE_SUCCESS; } -static bool isValidFileBlockRow(SBlockData* pBlockData, SFileBlockDumpInfo* pDumpInfo, - STableBlockScanInfo* pBlockScanInfo, STsdbReader* pReader) { +static bool isValidFileBlockRow(SBlockData* pBlockData, int32_t rowIndex, STableBlockScanInfo* pBlockScanInfo, bool asc, + STsdbReaderInfo* pInfo) { // it is an multi-table data block if (pBlockData->aUid != NULL) { - uint64_t uid = pBlockData->aUid[pDumpInfo->rowIndex]; + uint64_t uid = pBlockData->aUid[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->info.verRange.maxVer || ver < pReader->info.verRange.minVer) { + int64_t ver = pBlockData->aVersion[rowIndex]; + if (ver > pInfo->verRange.maxVer || ver < pInfo->verRange.minVer) { return false; } - int64_t ts = pBlockData->aTSKEY[pDumpInfo->rowIndex]; - if (ts > pReader->info.window.ekey || ts < pReader->info.window.skey) { + int64_t ts = pBlockData->aTSKEY[rowIndex]; + if (ts > pInfo->window.ekey || ts < pInfo->window.skey) { + return false; + } + + if ((asc && (ts <= pBlockScanInfo->lastProcKey)) || ((!asc) && (ts >= pBlockScanInfo->lastProcKey))) { return false; } if (pBlockScanInfo->delSkyline != NULL && TARRAY_SIZE(pBlockScanInfo->delSkyline) > 0) { bool dropped = hasBeenDropped(pBlockScanInfo->delSkyline, &pBlockScanInfo->fileDelIndex, ts, ver, - pReader->info.order, &pReader->info.verRange); + pInfo->order, &pInfo->verRange); if (dropped) { return false; } @@ -2424,7 +2432,7 @@ static int32_t buildComposedDataBlock(STsdbReader* pReader) { { while (pBlockData->nRow > 0 && pBlockData->uid == pBlockScanInfo->uid) { // find the first qualified row in data block - if (isValidFileBlockRow(pBlockData, pDumpInfo, pBlockScanInfo, pReader)) { + if (isValidFileBlockRow(pBlockData, pDumpInfo->rowIndex, pBlockScanInfo, asc, &pReader->info)) { hasBlockData = true; break; } @@ -4210,37 +4218,21 @@ void tsdbReaderClose2(STsdbReader* pReader) { taosMemoryFreeClear(pReader); } -int32_t tsdbReaderSuspend2(STsdbReader* pReader) { - // save reader's base state & reset top state to be reconstructed from base state - int32_t code = 0; - SReaderStatus* pStatus = &pReader->status; - STableBlockScanInfo* pBlockScanInfo = NULL; - - pReader->status.suspendInvoked = true; // record the suspend status +static int32_t doSuspendCurrentReader(STsdbReader* pCurrentReader) { + SReaderStatus* pStatus = &pCurrentReader->status; 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; - } + tsdbDataFileReaderClose(&pCurrentReader->pFileReader); - tsdbDataFileReaderClose(&pReader->pFileReader); - - SReadCostSummary* pCost = &pReader->cost; - pReader->status.pLDataIterArray = destroySttBlockReader(pReader->status.pLDataIterArray, &pCost->sttCost); - pReader->status.pLDataIterArray = taosArrayInit(4, POINTER_BYTES); + SReadCostSummary* pCost = &pCurrentReader->cost; + pStatus->pLDataIterArray = destroySttBlockReader(pStatus->pLDataIterArray, &pCost->sttCost); + pStatus->pLDataIterArray = taosArrayInit(4, POINTER_BYTES); } // resetDataBlockScanInfo excluding lastKey STableBlockScanInfo** p = NULL; - int32_t step = ASCENDING_TRAVERSE(pReader->info.order) ? 1 : -1; - + int32_t step = ASCENDING_TRAVERSE(pCurrentReader->info.order) ? 1 : -1; int32_t iter = 0; while ((p = tSimpleHashIterate(pStatus->pTableMap, p, &iter)) != NULL) { STableBlockScanInfo* pInfo = *(STableBlockScanInfo**)p; @@ -4251,6 +4243,26 @@ int32_t tsdbReaderSuspend2(STsdbReader* pReader) { pStatus->uidList.currentIndex = 0; initReaderStatus(pStatus); + return TSDB_CODE_SUCCESS; +} + +int32_t tsdbReaderSuspend2(STsdbReader* pReader) { + // save reader's base state & reset top state to be reconstructed from base state + int32_t code = 0; + pReader->status.suspendInvoked = true; // record the suspend status + + if (pReader->type == TIMEWINDOW_RANGE_EXTERNAL) { + if (pReader->step == EXTERNAL_ROWS_PREV) { + doSuspendCurrentReader(pReader->innerReader[0]); + } else if (pReader->step == EXTERNAL_ROWS_MAIN) { + doSuspendCurrentReader(pReader); + } else { + doSuspendCurrentReader(pReader->innerReader[1]); + } + } else { + doSuspendCurrentReader(pReader); + } + tsdbUntakeReadSnap2(pReader, pReader->pReadSnap, false); pReader->pReadSnap = NULL; if (pReader->bFilesetDelimited) { @@ -4259,16 +4271,16 @@ int32_t tsdbReaderSuspend2(STsdbReader* pReader) { } pReader->flag = READER_STATUS_SUSPEND; + if (pReader->type == TIMEWINDOW_RANGE_EXTERNAL) { + clearSharedPtr(pReader->innerReader[0]); + clearSharedPtr(pReader->innerReader[1]); + } + #if SUSPEND_RESUME_TEST tsem_post(&pReader->resumeAfterSuspend); #endif - 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); + tsdbDebug("reader: %p suspended in this query %s, step:%d", pReader, pReader->idStr, pReader->step); return code; } @@ -4299,8 +4311,7 @@ int32_t tsdbReaderResume2(STsdbReader* pReader) { int32_t code = 0; STableBlockScanInfo** pBlockScanInfo = pReader->status.pTableIter; - // restore reader's state - // task snapshot + // restore reader's state, task snapshot int32_t numOfTables = tSimpleHashGetSize(pReader->status.pTableMap); if (numOfTables > 0) { qTrace("tsdb/reader: %p, take snapshot", pReader); @@ -4325,7 +4336,14 @@ int32_t tsdbReaderResume2(STsdbReader* pReader) { pNextReader->resBlockInfo.capacity = 1; setSharedPtr(pNextReader, pReader); - code = doOpenReaderImpl(pPrevReader); + if (pReader->step == 0 || pReader->step == EXTERNAL_ROWS_PREV) { + code = doOpenReaderImpl(pPrevReader); + } else if (pReader->step == EXTERNAL_ROWS_MAIN) { + code = doOpenReaderImpl(pReader); + } else { + code = doOpenReaderImpl(pNextReader); + } + if (code != TSDB_CODE_SUCCESS) { return code; } From 2537adb72887dda2d0eb2bae6e015a2fcb75d6d3 Mon Sep 17 00:00:00 2001 From: Yihao Deng Date: Wed, 10 Jan 2024 06:00:44 +0000 Subject: [PATCH 042/102] opt msg on mnode --- source/common/src/tglobal.c | 8 +++---- source/dnode/mnode/impl/src/mndMain.c | 32 +++++++++++++-------------- 2 files changed, 20 insertions(+), 20 deletions(-) diff --git a/source/common/src/tglobal.c b/source/common/src/tglobal.c index 76e1ab46b7..4b32fa676e 100644 --- a/source/common/src/tglobal.c +++ b/source/common/src/tglobal.c @@ -108,7 +108,7 @@ bool tsEnableTelem = true; int32_t tsTelemInterval = 43200; char tsTelemServer[TSDB_FQDN_LEN] = "telemetry.tdengine.com"; uint16_t tsTelemPort = 80; -char * tsTelemUri = "/report"; +char *tsTelemUri = "/report"; #ifdef TD_ENTERPRISE bool tsEnableCrashReport = false; @@ -1376,7 +1376,7 @@ void taosCleanupCfg() { typedef struct { const char *optionName; - void * optionVar; + void *optionVar; } OptionNameAndVar; static int32_t taosCfgSetOption(OptionNameAndVar *pOptions, int32_t optionSize, SConfigItem *pItem, bool isDebugflag) { @@ -1389,7 +1389,7 @@ static int32_t taosCfgSetOption(OptionNameAndVar *pOptions, int32_t optionSize, switch (pItem->dtype) { case CFG_DTYPE_BOOL: { int32_t flag = pItem->i32; - bool * pVar = pOptions[d].optionVar; + bool *pVar = pOptions[d].optionVar; uInfo("%s set from %d to %d", optName, *pVar, flag); *pVar = flag; terrno = TSDB_CODE_SUCCESS; @@ -1758,7 +1758,7 @@ static void taosCheckAndSetDebugFlag(int32_t *pFlagPtr, char *name, int32_t flag void taosSetAllDebugFlag(int32_t flag) { if (flag <= 0) return; - SArray * noNeedToSetVars = NULL; + SArray *noNeedToSetVars = NULL; SConfigItem *pItem = cfgGetItem(tsCfg, "debugFlag"); if (pItem != NULL) { pItem->i32 = flag; diff --git a/source/dnode/mnode/impl/src/mndMain.c b/source/dnode/mnode/impl/src/mndMain.c index 4bd0776ecd..aa54de9ae7 100644 --- a/source/dnode/mnode/impl/src/mndMain.c +++ b/source/dnode/mnode/impl/src/mndMain.c @@ -107,7 +107,7 @@ static void *mndBuildCheckpointTickMsg(int32_t *pContLen, int64_t sec) { static void mndPullupTrans(SMnode *pMnode) { mTrace("pullup trans msg"); int32_t contLen = 0; - void * pReq = mndBuildTimerMsg(&contLen); + void *pReq = mndBuildTimerMsg(&contLen); if (pReq != NULL) { SRpcMsg rpcMsg = {.msgType = TDMT_MND_TRANS_TIMER, .pCont = pReq, .contLen = contLen}; tmsgPutToQueue(&pMnode->msgCb, WRITE_QUEUE, &rpcMsg); @@ -117,7 +117,7 @@ static void mndPullupTrans(SMnode *pMnode) { static void mndPullupCompacts(SMnode *pMnode) { mTrace("pullup compact timer msg"); int32_t contLen = 0; - void * pReq = mndBuildTimerMsg(&contLen); + void *pReq = mndBuildTimerMsg(&contLen); if (pReq != NULL) { SRpcMsg rpcMsg = {.msgType = TDMT_MND_COMPACT_TIMER, .pCont = pReq, .contLen = contLen}; tmsgPutToQueue(&pMnode->msgCb, WRITE_QUEUE, &rpcMsg); @@ -127,7 +127,7 @@ static void mndPullupCompacts(SMnode *pMnode) { static void mndPullupTtl(SMnode *pMnode) { mTrace("pullup ttl"); int32_t contLen = 0; - void * pReq = mndBuildTimerMsg(&contLen); + void *pReq = mndBuildTimerMsg(&contLen); SRpcMsg rpcMsg = {.msgType = TDMT_MND_TTL_TIMER, .pCont = pReq, .contLen = contLen}; tmsgPutToQueue(&pMnode->msgCb, WRITE_QUEUE, &rpcMsg); } @@ -135,14 +135,14 @@ static void mndPullupTtl(SMnode *pMnode) { static void mndPullupTrimDb(SMnode *pMnode) { mTrace("pullup trim"); int32_t contLen = 0; - void * pReq = mndBuildTimerMsg(&contLen); + void *pReq = mndBuildTimerMsg(&contLen); SRpcMsg rpcMsg = {.msgType = TDMT_MND_TRIM_DB_TIMER, .pCont = pReq, .contLen = contLen}; tmsgPutToQueue(&pMnode->msgCb, WRITE_QUEUE, &rpcMsg); } static void mndCalMqRebalance(SMnode *pMnode) { int32_t contLen = 0; - void * pReq = mndBuildTimerMsg(&contLen); + void *pReq = mndBuildTimerMsg(&contLen); if (pReq != NULL) { SRpcMsg rpcMsg = {.msgType = TDMT_MND_TMQ_TIMER, .pCont = pReq, .contLen = contLen}; tmsgPutToQueue(&pMnode->msgCb, READ_QUEUE, &rpcMsg); @@ -151,7 +151,7 @@ static void mndCalMqRebalance(SMnode *pMnode) { static void mndStreamCheckpointTick(SMnode *pMnode, int64_t sec) { int32_t contLen = 0; - void * pReq = mndBuildCheckpointTickMsg(&contLen, sec); + void *pReq = mndBuildCheckpointTickMsg(&contLen, sec); if (pReq != NULL) { SRpcMsg rpcMsg = {.msgType = TDMT_MND_STREAM_CHECKPOINT_TIMER, .pCont = pReq, .contLen = contLen}; tmsgPutToQueue(&pMnode->msgCb, READ_QUEUE, &rpcMsg); @@ -160,7 +160,7 @@ static void mndStreamCheckpointTick(SMnode *pMnode, int64_t sec) { static void mndStreamCheckpointRemain(SMnode *pMnode) { int32_t contLen = 0; - void * pReq = mndBuildCheckpointTickMsg(&contLen, 0); + void *pReq = mndBuildCheckpointTickMsg(&contLen, 0); if (pReq != NULL) { SRpcMsg rpcMsg = {.msgType = TDMT_MND_STREAM_CHECKPOINT_CANDIDITATE, .pCont = pReq, .contLen = contLen}; tmsgPutToQueue(&pMnode->msgCb, READ_QUEUE, &rpcMsg); @@ -169,7 +169,7 @@ static void mndStreamCheckpointRemain(SMnode *pMnode) { static void mndStreamCheckNode(SMnode *pMnode) { int32_t contLen = 0; - void * pReq = mndBuildTimerMsg(&contLen); + void *pReq = mndBuildTimerMsg(&contLen); if (pReq != NULL) { SRpcMsg rpcMsg = {.msgType = TDMT_MND_NODECHECK_TIMER, .pCont = pReq, .contLen = contLen}; tmsgPutToQueue(&pMnode->msgCb, READ_QUEUE, &rpcMsg); @@ -179,7 +179,7 @@ static void mndStreamCheckNode(SMnode *pMnode) { static void mndPullupTelem(SMnode *pMnode) { mTrace("pullup telem msg"); int32_t contLen = 0; - void * pReq = mndBuildTimerMsg(&contLen); + void *pReq = mndBuildTimerMsg(&contLen); if (pReq != NULL) { SRpcMsg rpcMsg = {.msgType = TDMT_MND_TELEM_TIMER, .pCont = pReq, .contLen = contLen}; tmsgPutToQueue(&pMnode->msgCb, READ_QUEUE, &rpcMsg); @@ -189,7 +189,7 @@ static void mndPullupTelem(SMnode *pMnode) { static void mndPullupGrant(SMnode *pMnode) { mTrace("pullup grant msg"); int32_t contLen = 0; - void * pReq = mndBuildTimerMsg(&contLen); + void *pReq = mndBuildTimerMsg(&contLen); if (pReq != NULL) { SRpcMsg rpcMsg = { .msgType = TDMT_MND_GRANT_HB_TIMER, .pCont = pReq, .contLen = contLen, .info.ahandle = (void *)0x9527}; @@ -200,7 +200,7 @@ static void mndPullupGrant(SMnode *pMnode) { static void mndIncreaseUpTime(SMnode *pMnode) { mTrace("increate uptime"); int32_t contLen = 0; - void * pReq = mndBuildTimerMsg(&contLen); + void *pReq = mndBuildTimerMsg(&contLen); if (pReq != NULL) { SRpcMsg rpcMsg = { .msgType = TDMT_MND_UPTIME_TIMER, .pCont = pReq, .contLen = contLen, .info.ahandle = (void *)0x9528}; @@ -254,7 +254,7 @@ static void mndCheckDnodeOffline(SMnode *pMnode) { mTrace("check dnode offline"); if (mndAcquireRpc(pMnode) != 0) return; - SSdb * pSdb = pMnode->pSdb; + SSdb *pSdb = pMnode->pSdb; int64_t curMs = taosGetTimestampMs(); void *pIter = NULL; @@ -701,7 +701,7 @@ void mndStop(SMnode *pMnode) { } int32_t mndProcessSyncMsg(SRpcMsg *pMsg) { - SMnode * pMnode = pMsg->info.node; + SMnode *pMnode = pMsg->info.node; SSyncMgmt *pMgmt = &pMnode->syncMgmt; const STraceId *trace = &pMsg->info.traceId; @@ -803,7 +803,7 @@ _OVER: } int32_t mndProcessRpcMsg(SRpcMsg *pMsg) { - SMnode * pMnode = pMsg->info.node; + SMnode *pMnode = pMsg->info.node; const STraceId *trace = &pMsg->info.traceId; MndMsgFp fp = pMnode->msgFp[TMSG_INDEX(pMsg->msgType)]; @@ -858,7 +858,7 @@ int32_t mndGetMonitorInfo(SMnode *pMnode, SMonClusterInfo *pClusterInfo, SMonVgr SMonStbInfo *pStbInfo, SMonGrantInfo *pGrantInfo) { if (mndAcquireRpc(pMnode) != 0) return -1; - SSdb * pSdb = pMnode->pSdb; + SSdb *pSdb = pMnode->pSdb; int64_t ms = taosGetTimestampMs(); pClusterInfo->dnodes = taosArrayInit(sdbGetSize(pSdb, SDB_DNODE), sizeof(SMonDnodeDesc)); @@ -942,7 +942,7 @@ int32_t mndGetMonitorInfo(SMnode *pMnode, SMonClusterInfo *pClusterInfo, SMonVgr pGrantInfo->timeseries_used += pVgroup->numOfTimeSeries; tstrncpy(desc.status, "unsynced", sizeof(desc.status)); for (int32_t i = 0; i < pVgroup->replica; ++i) { - SVnodeGid * pVgid = &pVgroup->vnodeGid[i]; + SVnodeGid *pVgid = &pVgroup->vnodeGid[i]; SMonVnodeDesc *pVnDesc = &desc.vnodes[i]; pVnDesc->dnode_id = pVgid->dnodeId; tstrncpy(pVnDesc->vnode_role, syncStr(pVgid->syncState), sizeof(pVnDesc->vnode_role)); From 3f0236ec171801f3ca09f55d0746611a6a50d139 Mon Sep 17 00:00:00 2001 From: Yihao Deng Date: Wed, 10 Jan 2024 06:18:39 +0000 Subject: [PATCH 043/102] opt msg on mnode --- source/dnode/mgmt/mgmt_dnode/src/dmWorker.c | 29 +++++++++++---------- 1 file changed, 15 insertions(+), 14 deletions(-) diff --git a/source/dnode/mgmt/mgmt_dnode/src/dmWorker.c b/source/dnode/mgmt/mgmt_dnode/src/dmWorker.c index b79b3f9e11..7ca19d7725 100644 --- a/source/dnode/mgmt/mgmt_dnode/src/dmWorker.c +++ b/source/dnode/mgmt/mgmt_dnode/src/dmWorker.c @@ -47,28 +47,29 @@ static void *dmStatusThreadFp(void *param) { } SDmNotifyHandle dmNotifyHdl = {.state = 0}; -static void *dmNotifyThreadFp(void *param) { - SDnodeMgmt *pMgmt = param; - setThreadName("dnode-notify"); - if (tsem_init(&dmNotifyHdl.sem, 0, 0) != 0) { - return NULL; +static void *dmNotifyThreadFp(void *param) { + SDnodeMgmt *pMgmt = param; + setThreadName("dnode-notify"); + + if (tsem_init(&dmNotifyHdl.sem, 0, 0) != 0) { + return NULL; } - bool wait = true; - while (1) { - if (pMgmt->pData->dropped || pMgmt->pData->stopped) break; + bool wait = true; + while (1) { + if (pMgmt->pData->dropped || pMgmt->pData->stopped) break; if (wait) tsem_wait(&dmNotifyHdl.sem); atomic_store_8(&dmNotifyHdl.state, 1); - dmSendNotifyReq(pMgmt); - if (1 == atomic_val_compare_exchange_8(&dmNotifyHdl.state, 1, 0)) { - wait = true; - continue; + dmSendNotifyReq(pMgmt); + if (1 == atomic_val_compare_exchange_8(&dmNotifyHdl.state, 1, 0)) { + wait = true; + continue; } - wait = false; + wait = false; } - return NULL; + return NULL; } static void *dmMonitorThreadFp(void *param) { From ba2ba0a8cbb292fd69b8011e45efe5d13c818f88 Mon Sep 17 00:00:00 2001 From: slzhou Date: Wed, 10 Jan 2024 16:19:20 +0800 Subject: [PATCH 044/102] fix: time window computation error --- source/common/src/ttime.c | 13 ++--- source/libs/executor/test/timewindowTest.cpp | 23 ++++++++ tests/parallel_test/cases.task | 1 + tests/system-test/2-query/count_interval.py | 60 ++++++++++++++++++++ 4 files changed, 90 insertions(+), 7 deletions(-) create mode 100644 tests/system-test/2-query/count_interval.py diff --git a/source/common/src/ttime.c b/source/common/src/ttime.c index 023a425df2..376cfce255 100644 --- a/source/common/src/ttime.c +++ b/source/common/src/ttime.c @@ -872,18 +872,17 @@ int64_t taosTimeTruncate(int64_t ts, const SInterval* pInterval) { ASSERT(pInterval->offset >= 0); if (pInterval->offset > 0) { - start = taosTimeAdd(start, pInterval->offset, pInterval->offsetUnit, precision); - // try to move current window to the left-hande-side, due to the offset effect. int64_t end = taosTimeAdd(start, pInterval->interval, pInterval->intervalUnit, precision) - 1; - int64_t newe = end; + int64_t slidingEnd = end; while (newe >= ts) { - end = newe; - newe = taosTimeAdd(newe, -pInterval->sliding, pInterval->slidingUnit, precision); + end = slidingEnd; + slidingEnd = taosTimeAdd(slidingEnd, -pInterval->sliding, pInterval->slidingUnit, precision); + newe = taosTimeAdd(slidingEnd, pInterval->offset, pInterval->offsetUnit, precision); } - - start = taosTimeAdd(end, -pInterval->interval, pInterval->intervalUnit, precision) + 1; + int64_t slidingStart = taosTimeAdd(end, -pInterval->interval, pInterval->intervalUnit, precision) + 1; + start = taosTimeAdd(slidingStart, pInterval->offset, pInterval->offsetUnit, precision); } return start; diff --git a/source/libs/executor/test/timewindowTest.cpp b/source/libs/executor/test/timewindowTest.cpp index 2894c66587..7a83b64fac 100644 --- a/source/libs/executor/test/timewindowTest.cpp +++ b/source/libs/executor/test/timewindowTest.cpp @@ -158,4 +158,27 @@ TEST(testCase, timewindow_gen) { } +TEST(testCase, timewindow_natural) { + osSetTimezone("CST"); + + int32_t precision = TSDB_TIME_PRECISION_MILLI; + + SInterval interval2 = createInterval(17, 17, 13392000000, 'n', 'n', 0, precision); + int64_t key1 = 1633446027072; + STimeWindow w1 = {0}; + getInitialStartTimeWindow(&interval2, key1, &w1, true); + printTimeWindow(&w1, precision, key1); + STimeWindow w3 = getAlignQueryTimeWindow(&interval2, key1); + printf("%ld win %ld, %ld\n", key1, w3.skey, w3.ekey); + + int64_t key2 = 1648758398208; + STimeWindow w2 = {0}; + getInitialStartTimeWindow(&interval2, key2, &w2, true); + printTimeWindow(&w2, precision, key2); + STimeWindow w4 = getAlignQueryTimeWindow(&interval2, key2); + printf("%ld win %ld, %ld\n", key2, w3.skey, w3.ekey); + + ASSERT_EQ(w3.skey, w4.skey); + ASSERT_EQ(w3.ekey, w4.ekey); +} #pragma GCC diagnostic pop \ No newline at end of file diff --git a/tests/parallel_test/cases.task b/tests/parallel_test/cases.task index 3db6eccffc..752234bbf5 100644 --- a/tests/parallel_test/cases.task +++ b/tests/parallel_test/cases.task @@ -36,6 +36,7 @@ #,,n,system-test,python3 ./test.py -f 8-stream/snode_restart_with_checkpoint.py -N 4 ,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/tbname_vgroup.py +,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/count_interval.py ,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/compact-col.py ,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/stbJoin.py ,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/stbJoin.py -Q 2 diff --git a/tests/system-test/2-query/count_interval.py b/tests/system-test/2-query/count_interval.py new file mode 100644 index 0000000000..b37cc1db22 --- /dev/null +++ b/tests/system-test/2-query/count_interval.py @@ -0,0 +1,60 @@ +################################################################### +# Copyright (c) 2016 by TAOS Technologies, Inc. +# All rights reserved. +# +# This file is proprietary and confidential to TAOS Technologies. +# No part of this file may be reproduced, stored, transmitted, +# disclosed or used in any form or by any means other than as +# expressly provided by the written permission from Jianhui Tao +# +################################################################### + +# -*- coding: utf-8 -*- + +import sys +import taos +from util.log import tdLog +from util.cases import tdCases +from util.sql import tdSql +from util.dnodes import tdDnodes +import random + + +class TDTestCase: + def init(self, conn, logSql, replicaVar=1): + tdLog.debug("start to execute %s" % __file__) + tdSql.init(conn.cursor(), logSql) + + def restartTaosd(self, index=1, dbname="db"): + tdDnodes.stop(index) + tdDnodes.startWithoutSleep(index) + tdSql.execute(f"use d") + + def run(self): + tdSql.execute("drop database if exists d"); + tdSql.execute("create database d"); + tdSql.execute("use d"); + tdSql.execute("create table st(ts timestamp, f int) tags (t int)") + + for i in range(-2048, 2047): + ts = 1626624000000 + i; + tdSql.execute(f"insert into ct1 using st tags(1) values({ts}, {i})") + + tdSql.execute("flush database d") + for i in range(1638): + ts = 1648758398208 + i + tdSql.execute(f"insert into ct1 using st tags(1) values({ts}, {i})") + tdSql.execute("insert into ct1 using st tags(1) values(1648970742528, 1638)") + tdSql.execute("flush database d") + + tdSql.query("select count(ts) from ct1 interval(17n, 5n)") + self.restartTaosd() + tdSql.query("select count(ts) from ct1 interval(17n, 5n)") + + def stop(self): + tdSql.close() + tdLog.success("%s successfully executed" % __file__) + + +tdCases.addWindows(__file__, TDTestCase()) +tdCases.addLinux(__file__, TDTestCase()) \ No newline at end of file From 27b63b721fb40e96a74f920a50edbf9040d5d5bf Mon Sep 17 00:00:00 2001 From: wangmm0220 Date: Wed, 10 Jan 2024 17:36:51 +0800 Subject: [PATCH 045/102] fix:decode mqDataMsg error for the message with old version --- source/common/src/tmsg.c | 24 +++++++++++++++++++----- 1 file changed, 19 insertions(+), 5 deletions(-) diff --git a/source/common/src/tmsg.c b/source/common/src/tmsg.c index 3ff814dda9..c9e2908e8a 100644 --- a/source/common/src/tmsg.c +++ b/source/common/src/tmsg.c @@ -8334,7 +8334,7 @@ int32_t tDecodeMqMetaRsp(SDecoder *pDecoder, SMqMetaRsp *pRsp) { return 0; } -int32_t tEncodeMqDataRsp(SEncoder *pEncoder, const SMqDataRsp *pRsp) { +int32_t tEncodeMqDataRspCommon(SEncoder *pEncoder, const SMqDataRsp *pRsp) { if (tEncodeSTqOffsetVal(pEncoder, &pRsp->reqOffset) < 0) return -1; if (tEncodeSTqOffsetVal(pEncoder, &pRsp->rspOffset) < 0) return -1; if (tEncodeI32(pEncoder, pRsp->blockNum) < 0) return -1; @@ -8356,11 +8356,16 @@ int32_t tEncodeMqDataRsp(SEncoder *pEncoder, const SMqDataRsp *pRsp) { } } } + return 0; +} + +int32_t tEncodeMqDataRsp(SEncoder *pEncoder, const SMqDataRsp *pRsp) { + if (tEncodeMqDataRspCommon(pEncoder, pRsp) < 0) return -1; if (tEncodeI64(pEncoder, pRsp->sleepTime) < 0) return -1; return 0; } -int32_t tDecodeMqDataRsp(SDecoder *pDecoder, SMqDataRsp *pRsp) { +int32_t tDecodeMqDataRspCommon(SDecoder *pDecoder, SMqDataRsp *pRsp) { if (tDecodeSTqOffsetVal(pDecoder, &pRsp->reqOffset) < 0) return -1; if (tDecodeSTqOffsetVal(pDecoder, &pRsp->rspOffset) < 0) return -1; if (tDecodeI32(pDecoder, &pRsp->blockNum) < 0) return -1; @@ -8402,7 +8407,15 @@ int32_t tDecodeMqDataRsp(SDecoder *pDecoder, SMqDataRsp *pRsp) { } } } - if (tDecodeI64(pDecoder, &pRsp->sleepTime) < 0) return -1; + + return 0; +} + +int32_t tDecodeMqDataRsp(SDecoder *pDecoder, SMqDataRsp *pRsp) { + if (tDecodeMqDataRspCommon(pDecoder, pRsp) < 0) return -1; + if (!tDecodeIsEnd(pDecoder)) { + if (tDecodeI64(pDecoder, &pRsp->sleepTime) < 0) return -1; + } return 0; } @@ -8418,7 +8431,7 @@ void tDeleteMqDataRsp(SMqDataRsp *pRsp) { } int32_t tEncodeSTaosxRsp(SEncoder *pEncoder, const STaosxRsp *pRsp) { - if (tEncodeMqDataRsp(pEncoder, (const SMqDataRsp *)pRsp) < 0) return -1; + if (tEncodeMqDataRspCommon(pEncoder, (const SMqDataRsp *)pRsp) < 0) return -1; if (tEncodeI32(pEncoder, pRsp->createTableNum) < 0) return -1; if (pRsp->createTableNum) { @@ -8432,7 +8445,7 @@ int32_t tEncodeSTaosxRsp(SEncoder *pEncoder, const STaosxRsp *pRsp) { } int32_t tDecodeSTaosxRsp(SDecoder *pDecoder, STaosxRsp *pRsp) { - if (tDecodeMqDataRsp(pDecoder, (SMqDataRsp *)pRsp) < 0) return -1; + if (tDecodeMqDataRspCommon(pDecoder, (SMqDataRsp*)pRsp) < 0) return -1; if (tDecodeI32(pDecoder, &pRsp->createTableNum) < 0) return -1; if (pRsp->createTableNum) { @@ -8447,6 +8460,7 @@ int32_t tDecodeSTaosxRsp(SDecoder *pDecoder, STaosxRsp *pRsp) { taosArrayPush(pRsp->createTableReq, &pCreate); } } + return 0; } From b5e5167b8fb845ffc0bd266e457c4abda533222e Mon Sep 17 00:00:00 2001 From: Yihao Deng Date: Wed, 10 Jan 2024 09:57:08 +0000 Subject: [PATCH 046/102] split sync/status channel --- include/common/tmsgcb.h | 4 + source/dnode/mgmt/mgmt_dnode/src/dmHandle.c | 2 +- source/dnode/mgmt/node_mgmt/inc/dmMgmt.h | 4 + source/dnode/mgmt/node_mgmt/src/dmMgmt.c | 5 + source/dnode/mgmt/node_mgmt/src/dmTransport.c | 93 +++++++++++++++++-- source/dnode/mnode/impl/src/mndDump.c | 9 +- source/dnode/mnode/impl/test/trans/trans2.cpp | 1 + source/dnode/vnode/src/vnd/vnodeSync.c | 12 +-- source/libs/transport/src/tmsgcb.c | 8 ++ 9 files changed, 119 insertions(+), 19 deletions(-) diff --git a/include/common/tmsgcb.h b/include/common/tmsgcb.h index 311bffb7da..dd1e54318e 100644 --- a/include/common/tmsgcb.h +++ b/include/common/tmsgcb.h @@ -53,9 +53,12 @@ typedef struct { void* mgmt; void* clientRpc; void* serverRpc; + void* statusRpc; + void* syncRpc; PutToQueueFp putToQueueFp; GetQueueSizeFp qsizeFp; SendReqFp sendReqFp; + SendReqFp sendSyncReqFp; SendRspFp sendRspFp; RegisterBrokenLinkArgFp registerBrokenLinkArgFp; ReleaseHandleFp releaseHandleFp; @@ -67,6 +70,7 @@ void tmsgSetDefault(const SMsgCb* msgcb); int32_t tmsgPutToQueue(const SMsgCb* msgcb, EQueueType qtype, SRpcMsg* pMsg); int32_t tmsgGetQueueSize(const SMsgCb* msgcb, int32_t vgId, EQueueType qtype); int32_t tmsgSendReq(const SEpSet* epSet, SRpcMsg* pMsg); +int32_t tmsgSendSyncReq(const SEpSet* epSet, SRpcMsg* pMsg); void tmsgSendRsp(SRpcMsg* pMsg); void tmsgRegisterBrokenLinkArg(SRpcMsg* pMsg); void tmsgReleaseHandle(SRpcHandleInfo* pHandle, int8_t type); diff --git a/source/dnode/mgmt/mgmt_dnode/src/dmHandle.c b/source/dnode/mgmt/mgmt_dnode/src/dmHandle.c index a95ec42bd0..545a5fc870 100644 --- a/source/dnode/mgmt/mgmt_dnode/src/dmHandle.c +++ b/source/dnode/mgmt/mgmt_dnode/src/dmHandle.c @@ -161,7 +161,7 @@ void dmSendStatusReq(SDnodeMgmt *pMgmt) { SEpSet epSet = {0}; int8_t epUpdated = 0; dmGetMnodeEpSet(pMgmt->pData, &epSet); - rpcSendRecvWithTimeout(pMgmt->msgCb.clientRpc, &epSet, &rpcMsg, &rpcRsp, &epUpdated, 5000); + rpcSendRecvWithTimeout(pMgmt->msgCb.statusRpc, &epSet, &rpcMsg, &rpcRsp, &epUpdated, tsStatusInterval * 10); if (rpcRsp.code != 0) { dmRotateMnodeEpSet(pMgmt->pData); char tbuf[512]; diff --git a/source/dnode/mgmt/node_mgmt/inc/dmMgmt.h b/source/dnode/mgmt/node_mgmt/inc/dmMgmt.h index c646bb4bdd..cb3395dcc2 100644 --- a/source/dnode/mgmt/node_mgmt/inc/dmMgmt.h +++ b/source/dnode/mgmt/node_mgmt/inc/dmMgmt.h @@ -48,6 +48,8 @@ typedef struct { typedef struct { void *serverRpc; void *clientRpc; + void *statusRpc; + void *syncRpc; SDnodeHandle msgHandles[TDMT_MAX]; } SDnodeTrans; @@ -136,8 +138,10 @@ int32_t dmInitServer(SDnode *pDnode); void dmCleanupServer(SDnode *pDnode); int32_t dmInitClient(SDnode *pDnode); int32_t dmInitStatusClient(SDnode *pDnode); +int32_t dmInitSyncClient(SDnode *pDnode); void dmCleanupClient(SDnode *pDnode); void dmCleanupStatusClient(SDnode *pDnode); +void dmCleanupSyncClient(SDnode *pDnode); SMsgCb dmGetMsgcb(SDnode *pDnode); #ifdef TD_MODULE_OPTIMIZE int32_t dmInitMsgHandle(SDnode *pDnode, SMgmtWrapper *wrappers); diff --git a/source/dnode/mgmt/node_mgmt/src/dmMgmt.c b/source/dnode/mgmt/node_mgmt/src/dmMgmt.c index 84465640c0..63122171b1 100644 --- a/source/dnode/mgmt/node_mgmt/src/dmMgmt.c +++ b/source/dnode/mgmt/node_mgmt/src/dmMgmt.c @@ -93,6 +93,9 @@ int32_t dmInitDnode(SDnode *pDnode) { indexInit(tsNumOfCommitThreads); streamMetaInit(); + dmInitStatusClient(pDnode); + dmInitSyncClient(pDnode); + dmReportStartup("dnode-transport", "initialized"); dDebug("dnode is created, ptr:%p", pDnode); code = 0; @@ -112,7 +115,9 @@ void dmCleanupDnode(SDnode *pDnode) { dmCleanupClient(pDnode); dmCleanupStatusClient(pDnode); + dmCleanupSyncClient(pDnode); dmCleanupServer(pDnode); + dmClearVars(pDnode); rpcCleanup(); streamMetaCleanup(); diff --git a/source/dnode/mgmt/node_mgmt/src/dmTransport.c b/source/dnode/mgmt/node_mgmt/src/dmTransport.c index 1ea61f0e93..512f090ab0 100644 --- a/source/dnode/mgmt/node_mgmt/src/dmTransport.c +++ b/source/dnode/mgmt/node_mgmt/src/dmTransport.c @@ -322,6 +322,23 @@ static inline int32_t dmSendReq(const SEpSet *pEpSet, SRpcMsg *pMsg) { return 0; } } +static inline int32_t dmSendSyncReq(const SEpSet *pEpSet, SRpcMsg *pMsg) { + SDnode *pDnode = dmInstance(); + if (pDnode->status != DND_STAT_RUNNING && pMsg->msgType < TDMT_SYNC_MSG) { + rpcFreeCont(pMsg->pCont); + pMsg->pCont = NULL; + if (pDnode->status == DND_STAT_INIT) { + terrno = TSDB_CODE_APP_IS_STARTING; + } else { + terrno = TSDB_CODE_APP_IS_STOPPING; + } + dError("failed to send rpc msg:%s since %s, handle:%p", TMSG_INFO(pMsg->msgType), terrstr(), pMsg->info.handle); + return -1; + } else { + rpcSendRequest(pDnode->trans.syncRpc, pEpSet, pMsg, NULL); + return 0; + } +} static inline void dmRegisterBrokenLinkArg(SRpcMsg *pMsg) { rpcRegisterBrokenLinkArg(pMsg); } @@ -421,16 +438,61 @@ int32_t dmInitStatusClient(SDnode *pDnode) { rpcInit.timeToGetConn = tsTimeToGetAvailableConn; taosVersionStrToInt(version, &(rpcInit.compatibilityVer)); - // pTrans->statusClientRpc = rpcOpen(&rpcInit); - // if (pTrans->statusClientRpc == NULL) { - // dError("failed to init dnode rpc status client"); - // return -1; - // } + pTrans->statusRpc = rpcOpen(&rpcInit); + if (pTrans->statusRpc == NULL) { + dError("failed to init dnode rpc status client"); + return -1; + } dDebug("dnode rpc status client is initialized"); return 0; } +int32_t dmInitSyncClient(SDnode *pDnode) { + SDnodeTrans *pTrans = &pDnode->trans; + + SRpcInit rpcInit = {0}; + rpcInit.label = "DND-SYNC"; + rpcInit.numOfThreads = tsNumOfRpcThreads; + rpcInit.cfp = (RpcCfp)dmProcessRpcMsg; + rpcInit.sessions = 1024; + rpcInit.connType = TAOS_CONN_CLIENT; + rpcInit.user = TSDB_DEFAULT_USER; + rpcInit.idleTime = tsShellActivityTimer * 1000; + rpcInit.parent = pDnode; + rpcInit.rfp = rpcRfp; + rpcInit.compressSize = tsCompressMsgSize; + + rpcInit.retryMinInterval = tsRedirectPeriod; + rpcInit.retryStepFactor = tsRedirectFactor; + rpcInit.retryMaxInterval = tsRedirectMaxPeriod; + rpcInit.retryMaxTimeout = tsMaxRetryWaitTime; + + rpcInit.failFastInterval = 5000; // interval threshold(ms) + rpcInit.failFastThreshold = 3; // failed threshold + rpcInit.ffp = dmFailFastFp; + + int32_t connLimitNum = 100; + connLimitNum = TMAX(connLimitNum, 10); + connLimitNum = TMIN(connLimitNum, 500); + + rpcInit.connLimitNum = connLimitNum; + rpcInit.connLimitLock = 1; + rpcInit.supportBatch = 1; + rpcInit.batchSize = 8 * 1024; + rpcInit.timeToGetConn = tsTimeToGetAvailableConn; + taosVersionStrToInt(version, &(rpcInit.compatibilityVer)); + + pTrans->syncRpc = rpcOpen(&rpcInit); + if (pTrans->syncRpc == NULL) { + dError("failed to init dnode rpc sync client"); + return -1; + } + + dDebug("dnode rpc sync client is initialized"); + return 0; +} + void dmCleanupClient(SDnode *pDnode) { SDnodeTrans *pTrans = &pDnode->trans; if (pTrans->clientRpc) { @@ -441,11 +503,19 @@ void dmCleanupClient(SDnode *pDnode) { } void dmCleanupStatusClient(SDnode *pDnode) { SDnodeTrans *pTrans = &pDnode->trans; - // if (pTrans->statusClientRpc) { - // rpcClose(pTrans->statusClientRpc); - // pTrans->statusClientRpc = NULL; - // dDebug("dnode rpc status client is closed"); - // } + if (pTrans->statusRpc) { + rpcClose(pTrans->statusRpc); + pTrans->statusRpc = NULL; + dDebug("dnode rpc status client is closed"); + } +} +void dmCleanupSyncClient(SDnode *pDnode) { + SDnodeTrans *pTrans = &pDnode->trans; + if (pTrans->syncRpc) { + rpcClose(pTrans->syncRpc); + pTrans->syncRpc = NULL; + dDebug("dnode rpc sync client is closed"); + } } int32_t dmInitServer(SDnode *pDnode) { @@ -486,7 +556,10 @@ SMsgCb dmGetMsgcb(SDnode *pDnode) { SMsgCb msgCb = { .clientRpc = pDnode->trans.clientRpc, .serverRpc = pDnode->trans.serverRpc, + .statusRpc = pDnode->trans.statusRpc, + .syncRpc = pDnode->trans.syncRpc, .sendReqFp = dmSendReq, + .sendSyncReqFp = dmSendSyncReq, .sendRspFp = dmSendRsp, .registerBrokenLinkArgFp = dmRegisterBrokenLinkArg, .releaseHandleFp = dmReleaseHandle, diff --git a/source/dnode/mnode/impl/src/mndDump.c b/source/dnode/mnode/impl/src/mndDump.c index 5efebbc16e..c68b11d184 100644 --- a/source/dnode/mnode/impl/src/mndDump.c +++ b/source/dnode/mnode/impl/src/mndDump.c @@ -32,6 +32,10 @@ int32_t sendReq(const SEpSet *pEpSet, SRpcMsg *pMsg) { terrno = TSDB_CODE_INVALID_PTR; return -1; } +int32_t sendSyncReq(const SEpSet *pEpSet, SRpcMsg *pMsg) { + terrno = TSDB_CODE_INVALID_PTR; + return -1; +} char *i642str(int64_t val) { static char str[24] = {0}; @@ -568,6 +572,7 @@ void mndDumpSdb() { SMsgCb msgCb = {0}; msgCb.reportStartupFp = reportStartup; msgCb.sendReqFp = sendReq; + msgCb.sendSyncReqFp = sendSyncReq; msgCb.sendRspFp = sendRsp; msgCb.mgmt = (SMgmtWrapper *)(&msgCb); // hack tmsgSetDefault(&msgCb); @@ -590,7 +595,7 @@ void mndDumpSdb() { dumpTopic(pSdb, json); dumpConsumer(pSdb, json); dumpSubscribe(pSdb, json); -// dumpOffset(pSdb, json); + // dumpOffset(pSdb, json); dumpStream(pSdb, json); dumpAcct(pSdb, json); dumpAuth(pSdb, json); @@ -605,7 +610,7 @@ void mndDumpSdb() { char *pCont = tjsonToString(json); int32_t contLen = strlen(pCont); char file[] = "sdb.json"; - TdFilePtr pFile = taosOpenFile(file, TD_FILE_CREATE | TD_FILE_WRITE | TD_FILE_TRUNC| TD_FILE_WRITE_THROUGH); + TdFilePtr pFile = taosOpenFile(file, TD_FILE_CREATE | TD_FILE_WRITE | TD_FILE_TRUNC | TD_FILE_WRITE_THROUGH); if (pFile == NULL) { terrno = TAOS_SYSTEM_ERROR(errno); mError("failed to write %s since %s", file, terrstr()); diff --git a/source/dnode/mnode/impl/test/trans/trans2.cpp b/source/dnode/mnode/impl/test/trans/trans2.cpp index 2d03631a37..4d0d53ced0 100644 --- a/source/dnode/mnode/impl/test/trans/trans2.cpp +++ b/source/dnode/mnode/impl/test/trans/trans2.cpp @@ -61,6 +61,7 @@ class MndTestTrans2 : public ::testing::Test { static SMsgCb msgCb = {0}; msgCb.reportStartupFp = reportStartup; msgCb.sendReqFp = sendReq; + msgCb.sendSyncReqFp = sendSyncReq; msgCb.sendRspFp = sendRsp; msgCb.queueFps[SYNC_QUEUE] = putToQueue; msgCb.queueFps[WRITE_QUEUE] = putToQueue; diff --git a/source/dnode/vnode/src/vnd/vnodeSync.c b/source/dnode/vnode/src/vnd/vnodeSync.c index 82baaa652b..09becffdaa 100644 --- a/source/dnode/vnode/src/vnd/vnodeSync.c +++ b/source/dnode/vnode/src/vnd/vnodeSync.c @@ -14,11 +14,11 @@ */ #define _DEFAULT_SOURCE -#include "tq.h" #include "sync.h" +#include "tq.h" +#include "tqCommon.h" #include "tsdb.h" #include "vnd.h" -#include "tqCommon.h" #define BATCH_ENABLE 0 @@ -411,7 +411,7 @@ static int32_t vnodeSyncEqMsg(const SMsgCb *msgcb, SRpcMsg *pMsg) { } static int32_t vnodeSyncSendMsg(const SEpSet *pEpSet, SRpcMsg *pMsg) { - int32_t code = tmsgSendReq(pEpSet, pMsg); + int32_t code = tmsgSendSyncReq(pEpSet, pMsg); if (code != 0) { rpcFreeCont(pMsg->pCont); pMsg->pCont = NULL; @@ -477,8 +477,8 @@ static void vnodeSyncRollBackMsg(const SSyncFSM *pFsm, SRpcMsg *pMsg, SFsmCbMeta } static int32_t vnodeSnapshotStartRead(const SSyncFSM *pFsm, void *pParam, void **ppReader) { - SVnode *pVnode = pFsm->data; - int32_t code = vnodeSnapReaderOpen(pVnode, (SSnapshotParam *)pParam, (SVSnapReader **)ppReader); + SVnode *pVnode = pFsm->data; + int32_t code = vnodeSnapReaderOpen(pVnode, (SSnapshotParam *)pParam, (SVSnapReader **)ppReader); return code; } @@ -555,7 +555,7 @@ static void vnodeRestoreFinish(const SSyncFSM *pFsm, const SyncIndex commitIdx) walApplyVer(pVnode->pWal, commitIdx); pVnode->restored = true; - SStreamMeta* pMeta = pVnode->pTq->pStreamMeta; + SStreamMeta *pMeta = pVnode->pTq->pStreamMeta; streamMetaWLock(pMeta); if (pMeta->startInfo.tasksWillRestart) { diff --git a/source/libs/transport/src/tmsgcb.c b/source/libs/transport/src/tmsgcb.c index 1b1fa8cc1c..e44328c683 100644 --- a/source/libs/transport/src/tmsgcb.c +++ b/source/libs/transport/src/tmsgcb.c @@ -48,6 +48,14 @@ int32_t tmsgSendReq(const SEpSet* epSet, SRpcMsg* pMsg) { } return code; } +int32_t tmsgSendSyncReq(const SEpSet* epSet, SRpcMsg* pMsg) { + int32_t code = (*defaultMsgCb.sendSyncReqFp)(epSet, pMsg); + if (code != 0) { + rpcFreeCont(pMsg->pCont); + pMsg->pCont = NULL; + } + return code; +} void tmsgSendRsp(SRpcMsg* pMsg) { #if 1 From df69a89da202a60df6915c2653e9af00f74aa6a7 Mon Sep 17 00:00:00 2001 From: wangmm0220 Date: Wed, 10 Jan 2024 18:46:10 +0800 Subject: [PATCH 047/102] fix:[TD-28185]add pause & resume logic for snode --- include/dnode/vnode/tqCommon.h | 2 + source/dnode/snode/src/snode.c | 6 +- source/dnode/vnode/src/tq/tq.c | 101 +------------------- source/dnode/vnode/src/tqCommon/tqCommon.c | 105 +++++++++++++++++++++ source/libs/stream/src/streamMeta.c | 8 +- 5 files changed, 118 insertions(+), 104 deletions(-) diff --git a/include/dnode/vnode/tqCommon.h b/include/dnode/vnode/tqCommon.h index da2325f006..82c4102369 100644 --- a/include/dnode/vnode/tqCommon.h +++ b/include/dnode/vnode/tqCommon.h @@ -33,5 +33,7 @@ int32_t tqStreamTaskProcessRunReq(SStreamMeta* pMeta, SRpcMsg* pMsg, bool isLead int32_t tqStreamTaskResetStatus(SStreamMeta* pMeta); int32_t tqStartTaskCompleteCallback(SStreamMeta* pMeta); int32_t tqStreamTaskProcessTaskResetReq(SStreamMeta* pMeta, SRpcMsg* pMsg); +int32_t tqStreamTaskProcessTaskPauseReq(SStreamMeta* pMeta, char* pMsg); +int32_t tqStreamTaskProcessTaskResumeReq(void* handle, int64_t sversion, char* pMsg, bool fromVnode); #endif // TDENGINE_TQ_COMMON_H diff --git a/source/dnode/snode/src/snode.c b/source/dnode/snode/src/snode.c index 33eb9cd3ed..9177559f61 100644 --- a/source/dnode/snode/src/snode.c +++ b/source/dnode/snode/src/snode.c @@ -195,7 +195,7 @@ int32_t sndProcessWriteMsg(SSnode *pSnode, SRpcMsg *pMsg, SRpcMsg *pRsp) { case TDMT_STREAM_TASK_DEPLOY: { void * pReq = POINTER_SHIFT(pMsg->pCont, sizeof(SMsgHead)); int32_t len = pMsg->contLen - sizeof(SMsgHead); - return tqStreamTaskProcessDeployReq(pSnode->pMeta, -1, pReq, len, true, true); + return tqStreamTaskProcessDeployReq(pSnode->pMeta, pMsg->info.conn.applyIndex, pReq, len, true, true); } case TDMT_STREAM_TASK_DROP: @@ -204,6 +204,10 @@ int32_t sndProcessWriteMsg(SSnode *pSnode, SRpcMsg *pMsg, SRpcMsg *pRsp) { return tqStreamTaskProcessUpdateReq(pSnode->pMeta, &pSnode->msgCb, pMsg, true); case TDMT_VND_STREAM_TASK_RESET: return tqStreamTaskProcessTaskResetReq(pSnode->pMeta, pMsg); + case TDMT_STREAM_TASK_PAUSE: + return tqStreamTaskProcessTaskPauseReq(pSnode->pMeta, pMsg->pCont); + case TDMT_STREAM_TASK_RESUME: + return tqStreamTaskProcessTaskResumeReq(pSnode->pMeta, pMsg->info.conn.applyIndex, pMsg->pCont, false); default: ASSERT(0); } diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 65cc56c0b4..275bbffe25 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1094,108 +1094,11 @@ int32_t tqProcessTaskDropReq(STQ* pTq, char* msg, int32_t msgLen) { } int32_t tqProcessTaskPauseReq(STQ* pTq, int64_t sversion, char* msg, int32_t msgLen) { - SVPauseStreamTaskReq* pReq = (SVPauseStreamTaskReq*)msg; - - SStreamMeta* pMeta = pTq->pStreamMeta; - SStreamTask* pTask = streamMetaAcquireTask(pMeta, pReq->streamId, pReq->taskId); - if (pTask == NULL) { - tqError("vgId:%d process pause req, failed to acquire task:0x%x, it may have been dropped already", pMeta->vgId, - pReq->taskId); - // since task is in [STOP|DROPPING] state, it is safe to assume the pause is active - return TSDB_CODE_SUCCESS; - } - - tqDebug("s-task:%s receive pause msg from mnode", pTask->id.idStr); - streamTaskPause(pTask, pMeta); - - SStreamTask* pHistoryTask = NULL; - if (HAS_RELATED_FILLHISTORY_TASK(pTask)) { - 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->hTaskInfo.id.taskId); - streamMetaReleaseTask(pMeta, pTask); - - // since task is in [STOP|DROPPING] state, it is safe to assume the pause is active - return TSDB_CODE_SUCCESS; - } - - tqDebug("s-task:%s fill-history task handle paused along with related stream task", pHistoryTask->id.idStr); - - streamTaskPause(pHistoryTask, pMeta); - streamMetaReleaseTask(pMeta, pHistoryTask); - } - - streamMetaReleaseTask(pMeta, pTask); - return TSDB_CODE_SUCCESS; -} - -int32_t tqProcessTaskResumeImpl(STQ* pTq, SStreamTask* pTask, int64_t sversion, int8_t igUntreated) { - int32_t vgId = pTq->pStreamMeta->vgId; - if (pTask == NULL) { - return -1; - } - - streamTaskResume(pTask); - ETaskStatus status = streamTaskGetStatus(pTask, NULL); - - int32_t level = pTask->info.taskLevel; - if (level == TASK_LEVEL__SINK) { - if (status == TASK_STATUS__UNINIT) { - } - streamMetaReleaseTask(pTq->pStreamMeta, pTask); - return 0; - } - - if (status == TASK_STATUS__READY || status == TASK_STATUS__SCAN_HISTORY || status == TASK_STATUS__CK) { - // no lock needs to secure the access of the version - if (igUntreated && level == TASK_LEVEL__SOURCE && !pTask->info.fillHistory) { - // discard all the data when the stream task is suspended. - walReaderSetSkipToVersion(pTask->exec.pWalReader, sversion); - tqDebug("vgId:%d s-task:%s resume to exec, prev paused version:%" PRId64 ", start from vnode ver:%" PRId64 - ", schedStatus:%d", - vgId, pTask->id.idStr, pTask->chkInfo.nextProcessVer, sversion, pTask->status.schedStatus); - } else { // from the previous paused version and go on - tqDebug("vgId:%d s-task:%s resume to exec, from paused ver:%" PRId64 ", vnode ver:%" PRId64 ", schedStatus:%d", - vgId, pTask->id.idStr, pTask->chkInfo.nextProcessVer, sversion, pTask->status.schedStatus); - } - - if (level == TASK_LEVEL__SOURCE && pTask->info.fillHistory && status == TASK_STATUS__SCAN_HISTORY) { - streamStartScanHistoryAsync(pTask, igUntreated); - } else if (level == TASK_LEVEL__SOURCE && (streamQueueGetNumOfItems(pTask->inputq.queue) == 0)) { - tqScanWalAsync(pTq, false); - } else { - streamSchedExec(pTask); - } - } else if (status == TASK_STATUS__UNINIT) { - // todo: fill-history task init ? - if (pTask->info.fillHistory == 0) { - EStreamTaskEvent event = /*HAS_RELATED_FILLHISTORY_TASK(pTask) ? TASK_EVENT_INIT_STREAM_SCANHIST : */TASK_EVENT_INIT; - streamTaskHandleEvent(pTask->status.pSM, event); - } - } - - streamMetaReleaseTask(pTq->pStreamMeta, pTask); - return 0; + return tqStreamTaskProcessTaskPauseReq(pTq->pStreamMeta, msg); } 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); - if (code != 0) { - return code; - } - - STaskId* pHTaskId = &pTask->hTaskInfo.id; - SStreamTask* pHistoryTask = streamMetaAcquireTask(pTq->pStreamMeta, pHTaskId->streamId, pHTaskId->taskId); - if (pHistoryTask) { - code = tqProcessTaskResumeImpl(pTq, pHistoryTask, sversion, pReq->igUntreated); - } - - return code; + return tqStreamTaskProcessTaskResumeReq(pTq, sversion, msg, true); } int32_t tqProcessTaskRetrieveReq(STQ* pTq, SRpcMsg* pMsg) { diff --git a/source/dnode/vnode/src/tqCommon/tqCommon.c b/source/dnode/vnode/src/tqCommon/tqCommon.c index a3d860fd78..70bfa00609 100644 --- a/source/dnode/vnode/src/tqCommon/tqCommon.c +++ b/source/dnode/vnode/src/tqCommon/tqCommon.c @@ -752,4 +752,109 @@ int32_t tqStreamTaskProcessTaskResetReq(SStreamMeta* pMeta, SRpcMsg* pMsg) { streamMetaReleaseTask(pMeta, pTask); return TSDB_CODE_SUCCESS; +} + +int32_t tqStreamTaskProcessTaskPauseReq(SStreamMeta* pMeta, char* pMsg){ + SVPauseStreamTaskReq* pReq = (SVPauseStreamTaskReq*)pMsg; + + SStreamTask* pTask = streamMetaAcquireTask(pMeta, pReq->streamId, pReq->taskId); + if (pTask == NULL) { + tqError("vgId:%d process pause req, failed to acquire task:0x%x, it may have been dropped already", pMeta->vgId, + pReq->taskId); + // since task is in [STOP|DROPPING] state, it is safe to assume the pause is active + return TSDB_CODE_SUCCESS; + } + + tqDebug("s-task:%s receive pause msg from mnode", pTask->id.idStr); + streamTaskPause(pTask, pMeta); + + SStreamTask* pHistoryTask = NULL; + if (HAS_RELATED_FILLHISTORY_TASK(pTask)) { + 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->hTaskInfo.id.taskId); + streamMetaReleaseTask(pMeta, pTask); + + // since task is in [STOP|DROPPING] state, it is safe to assume the pause is active + return TSDB_CODE_SUCCESS; + } + + tqDebug("s-task:%s fill-history task handle paused along with related stream task", pHistoryTask->id.idStr); + + streamTaskPause(pHistoryTask, pMeta); + streamMetaReleaseTask(pMeta, pHistoryTask); + } + + streamMetaReleaseTask(pMeta, pTask); + return TSDB_CODE_SUCCESS; +} + +static int32_t tqProcessTaskResumeImpl(void* handle, SStreamTask* pTask, int64_t sversion, int8_t igUntreated, bool fromVnode) { + SStreamMeta *pMeta = fromVnode ? ((STQ*)handle)->pStreamMeta : handle; + int32_t vgId = pMeta->vgId; + if (pTask == NULL) { + return -1; + } + + streamTaskResume(pTask); + ETaskStatus status = streamTaskGetStatus(pTask, NULL); + + int32_t level = pTask->info.taskLevel; + if (level == TASK_LEVEL__SINK) { + if (status == TASK_STATUS__UNINIT) { + } + streamMetaReleaseTask(pMeta, pTask); + return 0; + } + + if (status == TASK_STATUS__READY || status == TASK_STATUS__SCAN_HISTORY || status == TASK_STATUS__CK) { + // no lock needs to secure the access of the version + if (igUntreated && level == TASK_LEVEL__SOURCE && !pTask->info.fillHistory) { + // discard all the data when the stream task is suspended. + walReaderSetSkipToVersion(pTask->exec.pWalReader, sversion); + tqDebug("vgId:%d s-task:%s resume to exec, prev paused version:%" PRId64 ", start from vnode ver:%" PRId64 + ", schedStatus:%d", + vgId, pTask->id.idStr, pTask->chkInfo.nextProcessVer, sversion, pTask->status.schedStatus); + } else { // from the previous paused version and go on + tqDebug("vgId:%d s-task:%s resume to exec, from paused ver:%" PRId64 ", vnode ver:%" PRId64 ", schedStatus:%d", + vgId, pTask->id.idStr, pTask->chkInfo.nextProcessVer, sversion, pTask->status.schedStatus); + } + + if (level == TASK_LEVEL__SOURCE && pTask->info.fillHistory && status == TASK_STATUS__SCAN_HISTORY) { + streamStartScanHistoryAsync(pTask, igUntreated); + } else if (level == TASK_LEVEL__SOURCE && (streamQueueGetNumOfItems(pTask->inputq.queue) == 0)) { + tqScanWalAsync((STQ*)handle, false); + } else { + streamSchedExec(pTask); + } + } else if (status == TASK_STATUS__UNINIT) { + // todo: fill-history task init ? + if (pTask->info.fillHistory == 0) { + EStreamTaskEvent event = /*HAS_RELATED_FILLHISTORY_TASK(pTask) ? TASK_EVENT_INIT_STREAM_SCANHIST : */TASK_EVENT_INIT; + streamTaskHandleEvent(pTask->status.pSM, event); + } + } + + streamMetaReleaseTask(pMeta, pTask); + return 0; +} + +int32_t tqStreamTaskProcessTaskResumeReq(void* handle, int64_t sversion, char* msg, bool fromVnode){ + SVResumeStreamTaskReq* pReq = (SVResumeStreamTaskReq*)msg; + SStreamMeta *pMeta = fromVnode ? ((STQ*)handle)->pStreamMeta : handle; + SStreamTask* pTask = streamMetaAcquireTask(pMeta, pReq->streamId, pReq->taskId); + int32_t code = tqProcessTaskResumeImpl(handle, pTask, sversion, pReq->igUntreated, fromVnode); + if (code != 0) { + return code; + } + + STaskId* pHTaskId = &pTask->hTaskInfo.id; + SStreamTask* pHistoryTask = streamMetaAcquireTask(pMeta, pHTaskId->streamId, pHTaskId->taskId); + if (pHistoryTask) { + code = tqProcessTaskResumeImpl(pMeta, pHistoryTask, sversion, pReq->igUntreated, fromVnode); + } + + return code; } \ No newline at end of file diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 7ea80ad8d8..4e30dc2555 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -377,16 +377,16 @@ SStreamMeta* streamMetaOpen(const char* path, void* ahandle, FTaskExpand expandF pMeta->rid = taosAddRef(streamMetaId, pMeta); // set the attribute when running on Linux OS -#if defined LINUX TdThreadRwlockAttr attr; taosThreadRwlockAttrInit(&attr); +#ifdef LINUX pthread_rwlockattr_setkind_np(&attr, PTHREAD_RWLOCK_PREFER_WRITER_NONRECURSIVE_NP); - taosThreadRwlockInit(&pMeta->lock, &attr); - - taosThreadRwlockAttrDestroy(&attr); #endif + taosThreadRwlockInit(&pMeta->lock, &attr); + taosThreadRwlockAttrDestroy(&attr); + int64_t* pRid = taosMemoryMalloc(sizeof(int64_t)); memcpy(pRid, &pMeta->rid, sizeof(pMeta->rid)); metaRefMgtAdd(pMeta->vgId, pRid); From dcbe963d326972ed1c29f63790abd3db09e9c99f Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 10 Jan 2024 19:19:53 +0800 Subject: [PATCH 048/102] fix(stream): keep the previous status in stack object. --- source/libs/stream/src/streamStart.c | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/source/libs/stream/src/streamStart.c b/source/libs/stream/src/streamStart.c index c852215d16..d39528312d 100644 --- a/source/libs/stream/src/streamStart.c +++ b/source/libs/stream/src/streamStart.c @@ -1066,21 +1066,21 @@ void streamTaskPause(SStreamTask* pTask, SStreamMeta* pMeta) { } void streamTaskResume(SStreamTask* pTask) { - SStreamTaskState* p = streamTaskGetStatus(pTask); + SStreamTaskState prevState = *streamTaskGetStatus(pTask); SStreamMeta* pMeta = pTask->pMeta; - if (p->state == TASK_STATUS__PAUSE || p->state == TASK_STATUS__HALT) { + if (prevState.state == TASK_STATUS__PAUSE || prevState.state == TASK_STATUS__HALT) { streamTaskRestoreStatus(pTask); char* pNew = streamTaskGetStatus(pTask)->name; - if (p->state == TASK_STATUS__PAUSE) { + if (prevState.state == TASK_STATUS__PAUSE) { int32_t num = atomic_sub_fetch_32(&pMeta->numOfPausedTasks, 1); - stInfo("s-task:%s status:%s resume from %s, paused task(s):%d", pTask->id.idStr, pNew, p->name, num); + stInfo("s-task:%s status:%s resume from %s, paused task(s):%d", pTask->id.idStr, pNew, prevState.name, num); } else { - stInfo("s-task:%s status:%s resume from %s", pTask->id.idStr, pNew, p->name); + stInfo("s-task:%s status:%s resume from %s", pTask->id.idStr, pNew, prevState.name); } } else { - stDebug("s-task:%s status:%s not in pause/halt status, no need to resume", pTask->id.idStr, p->name); + stDebug("s-task:%s status:%s not in pause/halt status, no need to resume", pTask->id.idStr, prevState.name); } } From fb42c6afd5a06d7a748afa69df6e9436a31528f6 Mon Sep 17 00:00:00 2001 From: Alex Duan <417921451@qq.com> Date: Wed, 10 Jan 2024 19:59:44 +0800 Subject: [PATCH 049/102] coverage: update coverage script file and BlockSMA trigger is 1 --- tests/script/coverage_test.sh | 363 +++++++++++--------------- tests/system-test/2-query/blockSMA.py | 2 +- 2 files changed, 160 insertions(+), 205 deletions(-) mode change 100755 => 100644 tests/script/coverage_test.sh diff --git a/tests/script/coverage_test.sh b/tests/script/coverage_test.sh old mode 100755 new mode 100644 index 535b6802f2..1b9e9753d3 --- a/tests/script/coverage_test.sh +++ b/tests/script/coverage_test.sh @@ -12,7 +12,6 @@ fi today=`date +"%Y%m%d"` TDENGINE_DIR=/root/TDengine JDBC_DIR=/root/taos-connector-jdbc -TAOSKEEPER_DIR=/root/taoskeeper TDENGINE_COVERAGE_REPORT=$TDENGINE_DIR/tests/coverage-report-$today.log # Color setting @@ -23,52 +22,52 @@ GREEN_UNDERLINE='\033[4;32m' NC='\033[0m' function buildTDengine() { - echo "check if TDengine need build" - cd $TDENGINE_DIR - git remote prune origin > /dev/null - git remote update > /dev/null - REMOTE_COMMIT=`git rev-parse --short remotes/origin/$branch` - LOCAL_COMMIT=`git rev-parse --short @` - echo " LOCAL: $LOCAL_COMMIT" - echo "REMOTE: $REMOTE_COMMIT" + echo "check if TDengine need build" + cd $TDENGINE_DIR + git remote prune origin > /dev/null + git remote update > /dev/null + REMOTE_COMMIT=`git rev-parse --short remotes/origin/$branch` + LOCAL_COMMIT=`git rev-parse --short @` + echo " LOCAL: $LOCAL_COMMIT" + echo "REMOTE: $REMOTE_COMMIT" - # reset counter - lcov -d . --zerocounters + # reset counter + lcov -d . --zerocounters - if [ "$LOCAL_COMMIT" == "$REMOTE_COMMIT" ]; then - echo "repo up-to-date" - else - echo "repo need to pull" - fi + if [ "$LOCAL_COMMIT" == "$REMOTE_COMMIT" ]; then + echo "repo up-to-date" + else + echo "repo need to pull" + fi - git reset --hard - git checkout -- . - git checkout $branch - git checkout -- . - git clean -dfx - git pull - - [ -d $TDENGINE_DIR/debug ] || mkdir $TDENGINE_DIR/debug - cd $TDENGINE_DIR/debug + git reset --hard + git checkout -- . + git checkout $branch + git checkout -- . + git clean -dfx + git pull + + [ -d $TDENGINE_DIR/debug ] || mkdir $TDENGINE_DIR/debug + cd $TDENGINE_DIR/debug echo "rebuild.." - LOCAL_COMMIT=`git rev-parse --short @` + LOCAL_COMMIT=`git rev-parse --short @` - rm -rf * - if [ "$branch" == "3.0" ]; then - echo "3.0 =============" - cmake -DCOVER=true -DBUILD_TEST=true -DBUILD_HTTP=false -DBUILD_TOOLS=true .. > /dev/null - else - cmake -DCOVER=true -DBUILD_TEST=true -DBUILD_TOOLS=true -DBUILD_HTTP=false .. > /dev/null - fi - make -j - make install + rm -rf * + if [ "$branch" == "3.0" ]; then + echo "3.0 =============" + cmake -DCOVER=true -DBUILD_TEST=true -DBUILD_HTTP=false -DBUILD_TOOLS=true -DBUILD_GEOS=true -DBUILD_CONTRIB=true .. + else + cmake -DCOVER=true -DBUILD_TOOLS=true -DBUILD_HTTP=false .. > /dev/null + fi + make -j + make install } function runCasesOneByOne () { - while read -r line; do - if [[ "$line" != "#"* ]]; then - cmd=`echo $line | cut -d',' -f 5` + while read -r line; do + if [[ "$line" != "#"* ]]; then + cmd=`echo $line | cut -d',' -f 5` if [[ "$2" == "sim" ]] && [[ $line == *"script"* ]]; then case=`echo $cmd | cut -d' ' -f 3` start_time=`date +%s` @@ -77,18 +76,7 @@ function runCasesOneByOne () { || echo -e "${RED}$case failed${NC}" | tee -a $TDENGINE_COVERAGE_REPORT end_time=`date +%s` echo execution time of $case was `expr $end_time - $start_time`s. | tee -a $TDENGINE_COVERAGE_REPORT - elif [[ "$2" == "system-test" ]] && [[ $line == *"system-test"* ]]; then - if [[ "$cmd" == *"pytest.sh"* ]]; then - cmd=`echo $cmd | cut -d' ' -f 2-20` - fi - case=`echo $cmd | cut -d' ' -f 4-20` - start_time=`date +%s` - date +%F\ %T | tee -a $TDENGINE_COVERAGE_REPORT && $cmd > /dev/null 2>&1 && \ - echo -e "${GREEN}$case success${NC}" | tee -a $TDENGINE_COVERAGE_REPORT || \ - echo -e "${RED}$case failed${NC}" | tee -a $TDENGINE_COVERAGE_REPORT - end_time=`date +%s` - echo execution time of $case was `expr $end_time - $start_time`s. | tee -a $TDENGINE_COVERAGE_REPORT - elif [[ "$2" == "develop-test" ]] && [[ $line == *"develop-test"* ]]; then + elif [$line == *"$2"* ]; then if [[ "$cmd" == *"pytest.sh"* ]]; then cmd=`echo $cmd | cut -d' ' -f 2-20` fi @@ -101,209 +89,176 @@ function runCasesOneByOne () { echo execution time of $case was `expr $end_time - $start_time`s. | tee -a $TDENGINE_COVERAGE_REPORT fi fi - done < $1 + done < $1 } function runUnitTest() { - echo "=== Run unit test case ===" - echo " $TDENGINE_DIR/debug" - cd $TDENGINE_DIR/debug - ctest -j12 - - echo " $TDENGINE_DIR/tests/script/api" - cd $TDENGINE_DIR/tests/script/api - make clean && make - - stopTaosd - stopTaosadapter - - nohup taosd -c /etc/taos >> /dev/null 2>&1 & - ./batchprepare 127.0.0.1 - ./dbTableRoute 127.0.0.1 - ./stopquery 127.0.0.1 demo t1 - - echo "3.0 unit test done" + echo "=== Run unit test case ===" + echo " $TDENGINE_DIR/debug" + cd $TDENGINE_DIR/debug + ctest -j12 + echo "3.0 unit test done" } function runSimCases() { - echo "=== Run sim cases ===" - - cd $TDENGINE_DIR/tests/script - runCasesOneByOne ../parallel_test/cases.task sim - - totalSuccess=`grep 'sim success' $TDENGINE_COVERAGE_REPORT | wc -l` - if [ "$totalSuccess" -gt "0" ]; then - echo "### Total $totalSuccess SIM test case(s) succeed! ###" | tee -a $TDENGINE_COVERAGE_REPORT - fi + echo "=== Run sim cases ===" - totalFailed=`grep 'sim failed\|fault' $TDENGINE_COVERAGE_REPORT | wc -l` - if [ "$totalFailed" -ne "0" ]; then - echo "### Total $totalFailed SIM test case(s) failed! ###" | tee -a $TDENGINE_COVERAGE_REPORT - fi + cd $TDENGINE_DIR/tests/script + runCasesOneByOne $TDENGINE_DIR/tests/parallel_test/cases.task sim + + totalSuccess=`grep 'sim success' $TDENGINE_COVERAGE_REPORT | wc -l` + if [ "$totalSuccess" -gt "0" ]; then + echo "### Total $totalSuccess SIM test case(s) succeed! ###" | tee -a $TDENGINE_COVERAGE_REPORT + fi + + totalFailed=`grep 'sim failed\|fault' $TDENGINE_COVERAGE_REPORT | wc -l` + if [ "$totalFailed" -ne "0" ]; then + echo "### Total $totalFailed SIM test case(s) failed! ###" | tee -a $TDENGINE_COVERAGE_REPORT + fi } function runPythonCases() { - echo "=== Run python cases ===" + echo "=== Run python cases ===" - cd $TDENGINE_DIR/tests/parallel_test - sed -i '/compatibility.py/d' cases.task - - cd $TDENGINE_DIR/tests/system-test - runCasesOneByOne ../parallel_test/cases.task system-test + cd $TDENGINE_DIR/tests/parallel_test + sed -i '/compatibility.py/d' cases.task - cd $TDENGINE_DIR/tests/develop-test - runCasesOneByOne ../parallel_test/cases.task develop-test + # army + cd $TDENGINE_DIR/tests/army + runCasesOneByOne ../parallel_test/cases.task army - totalSuccess=`grep 'py success' $TDENGINE_COVERAGE_REPORT | wc -l` - if [ "$totalSuccess" -gt "0" ]; then - echo "### Total $totalSuccess python test case(s) succeed! ###" | tee -a $TDENGINE_COVERAGE_REPORT - fi + # system-test + cd $TDENGINE_DIR/tests/system-test + runCasesOneByOne ../parallel_test/cases.task system-test - totalFailed=`grep 'py failed\|fault' $TDENGINE_COVERAGE_REPORT | wc -l` - if [ "$totalFailed" -ne "0" ]; then - echo "### Total $totalFailed python test case(s) failed! ###" | tee -a $TDENGINE_COVERAGE_REPORT - fi + # develop-test + cd $TDENGINE_DIR/tests/develop-test + runCasesOneByOne ../parallel_test/cases.task develop-test + + totalSuccess=`grep 'py success' $TDENGINE_COVERAGE_REPORT | wc -l` + if [ "$totalSuccess" -gt "0" ]; then + echo "### Total $totalSuccess python test case(s) succeed! ###" | tee -a $TDENGINE_COVERAGE_REPORT + fi + + totalFailed=`grep 'py failed\|fault' $TDENGINE_COVERAGE_REPORT | wc -l` + if [ "$totalFailed" -ne "0" ]; then + echo "### Total $totalFailed python test case(s) failed! ###" | tee -a $TDENGINE_COVERAGE_REPORT + fi } function runJDBCCases() { - echo "=== Run JDBC cases ===" + echo "=== Run JDBC cases ===" - cd $JDBC_DIR - git checkout -- . - git reset --hard HEAD - git checkout main - git pull + cd $JDBC_DIR + git checkout -- . + git reset --hard HEAD + git checkout main + git pull - stopTaosd - stopTaosadapter + stopTaosd + stopTaosadapter - nohup taosd -c /etc/taos >> /dev/null 2>&1 & - nohup taosadapter >> /dev/null 2>&1 & + nohup $TDENGINE_DIR/debug/build/bin/taosd -c /etc/taos >> /dev/null 2>&1 & + nohup taosadapter >> /dev/null 2>&1 & - mvn clean test > result.txt 2>&1 - summary=`grep "Tests run:" result.txt | tail -n 1` - echo -e "### JDBC test result: $summary ###" | tee -a $TDENGINE_COVERAGE_REPORT -} - -function runTaosKeeperCases() { - echo "=== Run taoskeeper cases ===" - - cd $TAOSKEEPER_DIR - git checkout -- . - git reset --hard HEAD - git checkout master - git pull - - stopTaosd - stopTaosadapter - - taosd -c /etc/taos >> /dev/null 2>&1 & - taosadapter >> /dev/null 2>&1 & - - go mod tidy && go test -v ./... + mvn clean test > result.txt 2>&1 + summary=`grep "Tests run:" result.txt | tail -n 1` + echo -e "### JDBC test result: $summary ###" | tee -a $TDENGINE_COVERAGE_REPORT } function runTest() { - echo "run Test" - - cd $TDENGINE_DIR - [ -d sim ] && rm -rf sim + echo "run Test" + + cd $TDENGINE_DIR + [ -d sim ] && rm -rf sim [ -f $TDENGINE_COVERAGE_REPORT ] && rm $TDENGINE_COVERAGE_REPORT - runUnitTest - runSimCases - runPythonCases - runJDBCCases - runTaosKeeperCases - - stopTaosd - cd $TDENGINE_DIR/tests/script - find . -name '*.sql' | xargs rm -f + runUnitTest + runSimCases + runPythonCases + runJDBCCases - cd $TDENGINE_DIR/tests/pytest - find . -name '*.sql' | xargs rm -f + stopTaosd + cd $TDENGINE_DIR/tests/script + find . -name '*.sql' | xargs rm -f + + cd $TDENGINE_DIR/tests/pytest + find . -name '*.sql' | xargs rm -f } function lcovFunc { - echo "collect data by lcov" - cd $TDENGINE_DIR + echo "collect data by lcov" + cd $TDENGINE_DIR - # collect data - lcov -d . --capture --rc lcov_branch_coverage=1 --rc genhtml_branch_coverage=1 --no-external -b $TDENGINE_DIR -o coverage.info + # collect data + lcov --ignore-errors negative --ignore-errors mismatch -d . --capture --rc lcov_branch_coverage=1 --rc branch_coverage=1 --no-external -b $TDENGINE_DIR -o coverage.info - # remove exclude paths - if [ "$branch" == "main" ] ; then - lcov --remove coverage.info \ - '*/contrib/*' '*/tests/*' '*/test/*' '*/tools/*' '*/libs/sync/*'\ - '*/AccessBridgeCalls.c' '*/ttszip.c' '*/dataInserter.c' '*/tlinearhash.c' '*/tsimplehash.c' '*/tsdbDiskData.c'\ - '*/texpr.c' '*/runUdf.c' '*/schDbg.c' '*/syncIO.c' '*/tdbOs.c' '*/pushServer.c' '*/osLz4.c'\ - '*/tbase64.c' '*/tbuffer.c' '*/tdes.c' '*/texception.c' '*/tidpool.c' '*/tmempool.c'\ - '*/clientJniConnector.c' '*/clientTmqConnector.c' '*/version.c' '*/shellAuto.c' '*/shellTire.c'\ - '*/tthread.c' '*/tversion.c' '*/ctgDbg.c' '*/schDbg.c' '*/qwDbg.c' '*/tencode.h' '*/catalog.c'\ - '*/tqSnapshot.c' '*/tsdbSnapshot.c''*/metaSnapshot.c' '*/smaSnapshot.c' '*/tqOffsetSnapshot.c'\ - '*/vnodeSnapshot.c' '*/metaSnapshot.c' '*/tsdbSnapshot.c' '*/mndGrant.c' '*/mndSnode.c' '*/streamRecover.c'\ - '*/osAtomic.c' '*/osDir.c' '*/osFile.c' '*/osMath.c' '*/osSignal.c' '*/osSleep.c' '*/osString.c' '*/osSystem.c'\ - '*/osThread.c' '*/osTime.c' '*/osTimezone.c' \ - --rc lcov_branch_coverage=1 -o coverage.info - else - lcov --remove coverage.info \ - '*/tests/*' '*/test/*' '*/deps/*' '*/plugins/*' '*/taosdef.h' '*/ttype.h' '*/tarithoperator.c' '*/TSDBJNIConnector.c' '*/taosdemo.c' '*/clientJniConnector.c'\ - --rc lcov_branch_coverage=1 -o coverage.info - fi + # remove exclude paths + lcov --remove coverage.info \ + '*/contrib/*' '*/tests/*' '*/test/*' '*/packaging/*' '*/taos-tools/*' '*/taosadapter/*' '*/TSZ/*' \ + '*/AccessBridgeCalls.c' '*/ttszip.c' '*/dataInserter.c' '*/tlinearhash.c' '*/tsimplehash.c' '*/tsdbDiskData.c'\ + '*/texpr.c' '*/runUdf.c' '*/schDbg.c' '*/syncIO.c' '*/tdbOs.c' '*/pushServer.c' '*/osLz4.c'\ + '*/tbase64.c' '*/tbuffer.c' '*/tdes.c' '*/texception.c' '*/examples/*' '*/tidpool.c' '*/tmempool.c'\ + '*/clientJniConnector.c' '*/clientTmqConnector.c' '*/version.c'\ + '*/tthread.c' '*/tversion.c' '*/ctgDbg.c' '*/schDbg.c' '*/qwDbg.c' '*/tencode.h' \ + '*/shellAuto.c' '*/shellTire.c' '*/shellCommand.c'\ + --rc branch_coverage=1 -o coverage.info + # generate result + echo "generate result" + lcov -l --rc branch_coverage=1 coverage.info | tee -a $TDENGINE_COVERAGE_REPORT - # generate result - echo "generate result" - lcov -l --rc lcov_branch_coverage=1 coverage.info | tee -a $TDENGINE_COVERAGE_REPORT + sed -i 's/\/root\/TDengine\/sql.c/\/root\/TDengine\/source\/libs\/parser\/inc\/sql.c/g' coverage.info + sed -i 's/\/root\/TDengine\/sql.y/\/root\/TDengine\/source\/libs\/parser\/inc\/sql.y/g' coverage.info - # push result to coveralls.io - echo "push result to coveralls.io" - /usr/local/bin/coveralls-lcov coverage.info -t o7uY02qEAgKyJHrkxLGiCOTfL3IGQR2zm | tee -a $TDENGINE_COVERAGE_REPORT + # push result to coveralls.io + echo "push result to coveralls.io" + /usr/local/bin/coveralls-lcov coverage.info -t o7uY02qEAgKyJHrkxLGiCOTfL3IGQR2zm | tee -a $TDENGINE_COVERAGE_REPORT + + #/root/pxiao/checkCoverageFile.sh -s $TDENGINE_DIR/source -f $TDENGINE_COVERAGE_REPORT + #cat /root/pxiao/fileListNoCoverage.log | tee -a $TDENGINE_COVERAGE_REPORT + cat $TDENGINE_COVERAGE_REPORT | grep "| 0.0%" | awk -F "%" '{print $1}' | awk -F "|" '{if($2==0.0)print $1}' | tee -a $TDENGINE_COVERAGE_REPORT - #/root/pxiao/checkCoverageFile.sh -s $TDENGINE_DIR/source -f $TDENGINE_COVERAGE_REPORT - #cat /root/pxiao/fileListNoCoverage.log | tee -a $TDENGINE_COVERAGE_REPORT - cat $TDENGINE_COVERAGE_REPORT | grep "| 0.0%" | awk -F "%" '{print $1}' | awk -F "|" '{if($2==0.0)print $1}' | tee -a $TDENGINE_COVERAGE_REPORT - } function sendReport { - echo "send report" - receiver="develop@taosdata.com" - mimebody="MIME-Version: 1.0\nContent-Type: text/html; charset=utf-8\n" + echo "send report" + receiver="develop@taosdata.com" + mimebody="MIME-Version: 1.0\nContent-Type: text/html; charset=utf-8\n" - cd $TDENGINE_DIR + cd $TDENGINE_DIR - sed -i 's/\x1b\[[0-9;]*m//g' $TDENGINE_COVERAGE_REPORT - BODY_CONTENT=`cat $TDENGINE_COVERAGE_REPORT` - echo -e "from: \nto: ${receiver}\nsubject: Coverage test report ${branch} ${today}, commit ID: ${LOCAL_COMMIT}\n\n${today}:\n${BODY_CONTENT}" | \ - (cat - && uuencode $TDENGINE_COVERAGE_REPORT coverage-report-$today.log) | \ - /usr/sbin/ssmtp "${receiver}" && echo "Report Sent!" + sed -i 's/\x1b\[[0-9;]*m//g' $TDENGINE_COVERAGE_REPORT + BODY_CONTENT=`cat $TDENGINE_COVERAGE_REPORT` + echo -e "from: \nto: ${receiver}\nsubject: Coverage test report ${branch} ${today}, commit ID: ${LOCAL_COMMIT}\n\n${today}:\n${BODY_CONTENT}" | \ + (cat - && uuencode $TDENGINE_COVERAGE_REPORT coverage-report-$today.log) | \ + /usr/sbin/ssmtp "${receiver}" && echo "Report Sent!" } function stopTaosd { - echo "Stop taosd start" + echo "Stop taosd start" systemctl stop taosd - PID=`ps -ef|grep -w taosd | grep -v grep | awk '{print $2}'` - while [ -n "$PID" ] - do + PID=`ps -ef|grep -w taosd | grep -v grep | awk '{print $2}'` + while [ -n "$PID" ] + do pkill -TERM -x taosd sleep 1 - PID=`ps -ef|grep -w taosd | grep -v grep | awk '{print $2}'` - done - echo "Stop tasod end" + PID=`ps -ef|grep -w taosd | grep -v grep | awk '{print $2}'` + done + echo "Stop tasod end" } function stopTaosadapter { - echo "Stop taosadapter" - systemctl stop taosadapter.service - PID=`ps -ef|grep -w taosadapter | grep -v grep | awk '{print $2}'` - while [ -n "$PID" ] - do - pkill -TERM -x taosadapter - sleep 1 - PID=`ps -ef|grep -w taosd | grep -v grep | awk '{print $2}'` - done - echo "Stop tasoadapter end" + echo "Stop taosadapter" + systemctl stop taosadapter.service + PID=`ps -ef|grep -w taosadapter | grep -v grep | awk '{print $2}'` + while [ -n "$PID" ] + do + pkill -TERM -x taosadapter + sleep 1 + PID=`ps -ef|grep -w taosd | grep -v grep | awk '{print $2}'` + done + echo "Stop tasoadapter end" } @@ -318,7 +273,7 @@ buildTDengine runTest lcovFunc -sendReport +#sendReport stopTaosd date >> $WORK_DIR/cron.log diff --git a/tests/system-test/2-query/blockSMA.py b/tests/system-test/2-query/blockSMA.py index 85c0189e27..605542cb3c 100644 --- a/tests/system-test/2-query/blockSMA.py +++ b/tests/system-test/2-query/blockSMA.py @@ -16,7 +16,7 @@ class TDTestCase: def run(self): dbname = "db" - tdSql.prepare() + tdSql.prepare(dbname=dbname, drop=True, stt_trigger=1) tdSql.execute(f'''create table {dbname}.ntb(ts timestamp, col1 tinyint, col2 smallint, col3 int, col4 bigint, col5 float, col6 double, col7 bool, col8 binary(20), col9 nchar(20), col11 tinyint unsigned, col12 smallint unsigned, col13 int unsigned, col14 bigint unsigned)''') From 216830f8b4e35254e707fbf23189e77d4ee05f83 Mon Sep 17 00:00:00 2001 From: wangmm0220 Date: Wed, 10 Jan 2024 20:33:13 +0800 Subject: [PATCH 050/102] fix:[TD-28185]add pause & resume logic for snode --- source/dnode/vnode/src/tqCommon/tqCommon.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/dnode/vnode/src/tqCommon/tqCommon.c b/source/dnode/vnode/src/tqCommon/tqCommon.c index 70bfa00609..399788e731 100644 --- a/source/dnode/vnode/src/tqCommon/tqCommon.c +++ b/source/dnode/vnode/src/tqCommon/tqCommon.c @@ -853,7 +853,7 @@ int32_t tqStreamTaskProcessTaskResumeReq(void* handle, int64_t sversion, char* m STaskId* pHTaskId = &pTask->hTaskInfo.id; SStreamTask* pHistoryTask = streamMetaAcquireTask(pMeta, pHTaskId->streamId, pHTaskId->taskId); if (pHistoryTask) { - code = tqProcessTaskResumeImpl(pMeta, pHistoryTask, sversion, pReq->igUntreated, fromVnode); + code = tqProcessTaskResumeImpl(handle, pHistoryTask, sversion, pReq->igUntreated, fromVnode); } return code; From b4b44009587f42a0eb61fc50c507e9d450fda76b Mon Sep 17 00:00:00 2001 From: slzhou Date: Wed, 10 Jan 2024 21:36:09 +0800 Subject: [PATCH 051/102] fix: recalcuate window ekey --- source/libs/executor/src/executorInt.c | 8 +++++++- source/libs/executor/test/timewindowTest.cpp | 5 +++++ 2 files changed, 12 insertions(+), 1 deletion(-) diff --git a/source/libs/executor/src/executorInt.c b/source/libs/executor/src/executorInt.c index 95d26fdd0e..cb4988c31f 100644 --- a/source/libs/executor/src/executorInt.c +++ b/source/libs/executor/src/executorInt.c @@ -449,7 +449,13 @@ STimeWindow getAlignQueryTimeWindow(const SInterval* pInterval, int64_t key) { * if the realSkey > INT64_MAX - pInterval->interval, the query duration between * realSkey and realEkey must be less than one interval.Therefore, no need to adjust the query ranges. */ - win.ekey = taosTimeAdd(win.skey, pInterval->interval, pInterval->intervalUnit, pInterval->precision) - 1; + if (pInterval->offset > 0) { + int64_t slideStart = taosTimeAdd(win.skey, -1 * pInterval->offset, pInterval->offsetUnit, pInterval->precision); + int64_t slideEnd = taosTimeAdd(slideStart, pInterval->interval, pInterval->intervalUnit, pInterval->precision) - 1; + win.ekey = taosTimeAdd(slideEnd, pInterval->offset, pInterval->offsetUnit, pInterval->precision); + } else { + win.ekey = taosTimeAdd(win.skey, pInterval->interval, pInterval->intervalUnit, pInterval->precision) - 1; + } if (win.ekey < win.skey) { win.ekey = INT64_MAX; } diff --git a/source/libs/executor/test/timewindowTest.cpp b/source/libs/executor/test/timewindowTest.cpp index 7a83b64fac..3639bf15e1 100644 --- a/source/libs/executor/test/timewindowTest.cpp +++ b/source/libs/executor/test/timewindowTest.cpp @@ -164,6 +164,11 @@ TEST(testCase, timewindow_natural) { int32_t precision = TSDB_TIME_PRECISION_MILLI; SInterval interval2 = createInterval(17, 17, 13392000000, 'n', 'n', 0, precision); + int64_t key = 1648970865984; + STimeWindow w0 = getAlignQueryTimeWindow(&interval2, key); + printTimeWindow(&w0, precision, key); + ASSERT_GE(w0.ekey, key); + int64_t key1 = 1633446027072; STimeWindow w1 = {0}; getInitialStartTimeWindow(&interval2, key1, &w1, true); From ab137a56f0107734a766023bd91eebfabba20659 Mon Sep 17 00:00:00 2001 From: slzhou Date: Thu, 11 Jan 2024 08:31:16 +0800 Subject: [PATCH 052/102] fix: more interval end calculation with slide-start/slide-end as anchor --- include/common/ttime.h | 1 + source/common/src/ttime.c | 12 ++++++++++++ source/libs/executor/src/executil.c | 2 +- source/libs/executor/src/executorInt.c | 8 +------- source/libs/executor/src/timewindowoperator.c | 4 ++-- 5 files changed, 17 insertions(+), 10 deletions(-) diff --git a/include/common/ttime.h b/include/common/ttime.h index ed4d1a9290..4db25d2f71 100644 --- a/include/common/ttime.h +++ b/include/common/ttime.h @@ -75,6 +75,7 @@ static FORCE_INLINE int64_t taosGetTimestampToday(int32_t precision) { int64_t taosTimeAdd(int64_t t, int64_t duration, char unit, int32_t precision); int64_t taosTimeTruncate(int64_t ts, const SInterval* pInterval); +int64_t taosTimeGetIntervalEnd(int64_t ts, const SInterval* pInterval); int32_t taosTimeCountIntervalForFill(int64_t skey, int64_t ekey, int64_t interval, char unit, int32_t precision, int32_t order); int32_t parseAbsoluteDuration(const char* token, int32_t tokenlen, int64_t* ts, char* unit, int32_t timePrecision); diff --git a/source/common/src/ttime.c b/source/common/src/ttime.c index 376cfce255..f683baee7c 100644 --- a/source/common/src/ttime.c +++ b/source/common/src/ttime.c @@ -888,6 +888,18 @@ int64_t taosTimeTruncate(int64_t ts, const SInterval* pInterval) { return start; } +// used together with taosTimeTruncate. when offset is great than zero, slide-start/slide-end is the anchor point +int64_t taosTimeGetIntervalEnd(int64_t intervalStart, const SInterval* pInterval) { + if (pInterval->offset > 0) { + int64_t slideStart = taosTimeAdd(intervalStart, -1 * pInterval->offset, pInterval->offsetUnit, pInterval->precision); + int64_t end = taosTimeAdd(slideStart, pInterval->interval, pInterval->intervalUnit, pInterval->precision) - 1; + int64_t result = taosTimeAdd(end, pInterval->offset, pInterval->offsetUnit, pInterval->precision); + return result; + } else { + int64_t result = taosTimeAdd(intervalStart, pInterval->interval, pInterval->intervalUnit, pInterval->precision) - 1; + return result; + } +} // internal function, when program is paused in debugger, // one can call this function from debugger to print a // timestamp as human readable string, for example (gdb): diff --git a/source/libs/executor/src/executil.c b/source/libs/executor/src/executil.c index b713b9b112..019483a015 100644 --- a/source/libs/executor/src/executil.c +++ b/source/libs/executor/src/executil.c @@ -1838,7 +1838,7 @@ static STimeWindow doCalculateTimeWindow(int64_t ts, SInterval* pInterval) { STimeWindow w = {0}; w.skey = taosTimeTruncate(ts, pInterval); - w.ekey = taosTimeAdd(w.skey, pInterval->interval, pInterval->intervalUnit, pInterval->precision) - 1; + w.ekey = taosTimeGetIntervalEnd(w.skey, pInterval); return w; } diff --git a/source/libs/executor/src/executorInt.c b/source/libs/executor/src/executorInt.c index cb4988c31f..ff4d3d0d27 100644 --- a/source/libs/executor/src/executorInt.c +++ b/source/libs/executor/src/executorInt.c @@ -449,13 +449,7 @@ STimeWindow getAlignQueryTimeWindow(const SInterval* pInterval, int64_t key) { * if the realSkey > INT64_MAX - pInterval->interval, the query duration between * realSkey and realEkey must be less than one interval.Therefore, no need to adjust the query ranges. */ - if (pInterval->offset > 0) { - int64_t slideStart = taosTimeAdd(win.skey, -1 * pInterval->offset, pInterval->offsetUnit, pInterval->precision); - int64_t slideEnd = taosTimeAdd(slideStart, pInterval->interval, pInterval->intervalUnit, pInterval->precision) - 1; - win.ekey = taosTimeAdd(slideEnd, pInterval->offset, pInterval->offsetUnit, pInterval->precision); - } else { - win.ekey = taosTimeAdd(win.skey, pInterval->interval, pInterval->intervalUnit, pInterval->precision) - 1; - } + win.ekey = taosTimeGetIntervalEnd(win.skey, pInterval); if (win.ekey < win.skey) { win.ekey = INT64_MAX; } diff --git a/source/libs/executor/src/timewindowoperator.c b/source/libs/executor/src/timewindowoperator.c index afe1921d30..c459e4d8b9 100644 --- a/source/libs/executor/src/timewindowoperator.c +++ b/source/libs/executor/src/timewindowoperator.c @@ -450,7 +450,7 @@ int32_t getNextQualifiedWindow(SInterval* pInterval, STimeWindow* pNext, SDataBl TSKEY next = primaryKeys[startPos]; if (pInterval->intervalUnit == 'n' || pInterval->intervalUnit == 'y') { pNext->skey = taosTimeTruncate(next, pInterval); - pNext->ekey = taosTimeAdd(pNext->skey, pInterval->interval, pInterval->intervalUnit, precision) - 1; + pNext->ekey = taosTimeGetIntervalEnd(pNext->skey, pInterval); } else { pNext->ekey += ((next - pNext->ekey + pInterval->sliding - 1) / pInterval->sliding) * pInterval->sliding; pNext->skey = pNext->ekey - pInterval->interval + 1; @@ -459,7 +459,7 @@ int32_t getNextQualifiedWindow(SInterval* pInterval, STimeWindow* pNext, SDataBl TSKEY next = primaryKeys[startPos]; if (pInterval->intervalUnit == 'n' || pInterval->intervalUnit == 'y') { pNext->skey = taosTimeTruncate(next, pInterval); - pNext->ekey = taosTimeAdd(pNext->skey, pInterval->interval, pInterval->intervalUnit, precision) - 1; + pNext->ekey = taosTimeGetIntervalEnd(pNext->skey, pInterval); } else { pNext->skey -= ((pNext->skey - next + pInterval->sliding - 1) / pInterval->sliding) * pInterval->sliding; pNext->ekey = pNext->skey + pInterval->interval - 1; From 9294c8d4eea9a6344a92669191b29459b1ecb4e6 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 11 Jan 2024 09:20:34 +0800 Subject: [PATCH 053/102] fix(stream): enable the hb to mnode for stream meta when it becomes the leader. --- 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 3522feb244..30a40ffc9f 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -1287,7 +1287,7 @@ void streamMetaStartHb(SStreamMeta* pMeta) { int64_t* pRid = taosMemoryMalloc(sizeof(int64_t)); metaRefMgtAdd(pMeta->vgId, pRid); *pRid = pMeta->rid; -// metaHbToMnode(pRid, NULL); + metaHbToMnode(pRid, NULL); } void streamMetaResetStartInfo(STaskStartInfo* pStartInfo) { From 3b36dbbe189f625896c6b4e13a0badd9ace42afb Mon Sep 17 00:00:00 2001 From: 54liuyao <54liuyao> Date: Thu, 11 Jan 2024 09:40:54 +0800 Subject: [PATCH 054/102] opt recover buff --- source/libs/stream/src/tstreamFileState.c | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/source/libs/stream/src/tstreamFileState.c b/source/libs/stream/src/tstreamFileState.c index d31e4cbfcd..aac78cf03b 100644 --- a/source/libs/stream/src/tstreamFileState.c +++ b/source/libs/stream/src/tstreamFileState.c @@ -26,6 +26,7 @@ #define FLUSH_NUM 4 #define DEFAULT_MAX_STREAM_BUFFER_SIZE (128 * 1024 * 1024) #define MIN_NUM_OF_ROW_BUFF 10240 +#define MIN_NUM_OF_RECOVER_ROW_BUFF 128 #define TASK_KEY "streamFileState" #define STREAM_STATE_INFO_NAME "StreamStateCheckPoint" @@ -660,7 +661,7 @@ int32_t recoverSesssion(SStreamFileState* pFileState, int64_t ckId) { if (pCur == NULL) { return -1; } - int32_t recoverNum = TMIN(MIN_NUM_OF_ROW_BUFF, pFileState->maxRowCount); + int32_t recoverNum = TMIN(MIN_NUM_OF_RECOVER_ROW_BUFF, pFileState->maxRowCount); while (code == TSDB_CODE_SUCCESS) { if (pFileState->curRowCount >= recoverNum) { break; @@ -694,7 +695,7 @@ int32_t recoverSnapshot(SStreamFileState* pFileState, int64_t ckId) { if (pCur == NULL) { return -1; } - int32_t recoverNum = TMIN(MIN_NUM_OF_ROW_BUFF, pFileState->maxRowCount); + int32_t recoverNum = TMIN(MIN_NUM_OF_RECOVER_ROW_BUFF, pFileState->maxRowCount); while (code == TSDB_CODE_SUCCESS) { if (pFileState->curRowCount >= recoverNum) { break; From 1c562bd535215f8292f993c8253501302ec84cee Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 11 Jan 2024 09:41:40 +0800 Subject: [PATCH 055/102] refactor: enable the restart of stream tasks. --- source/dnode/vnode/src/tqCommon/tqCommon.c | 15 ++++++++++++--- source/libs/stream/src/streamMeta.c | 1 + 2 files changed, 13 insertions(+), 3 deletions(-) diff --git a/source/dnode/vnode/src/tqCommon/tqCommon.c b/source/dnode/vnode/src/tqCommon/tqCommon.c index 2e83cd5bc5..b7b3893dfe 100644 --- a/source/dnode/vnode/src/tqCommon/tqCommon.c +++ b/source/dnode/vnode/src/tqCommon/tqCommon.c @@ -729,9 +729,9 @@ int32_t tqStreamTaskResetStatus(SStreamMeta* pMeta, int32_t* numOfTasks) { SStreamTask** pTask = taosHashGet(pMeta->pTasksMap, &id, sizeof(id)); streamTaskResetStatus(*pTask); - if ((*pTask)->info.fillHistory == 1) { - streamResetParamForScanHistory(*pTask); - } +// if ((*pTask)->info.fillHistory == 1) { +// streamResetParamForScanHistory(*pTask); +// } } return 0; @@ -764,10 +764,19 @@ static int32_t restartStreamTasks(SStreamMeta* pMeta, bool isLeader) { } streamMetaWLock(pMeta); + streamMetaClear(pMeta); int64_t el = taosGetTimestampMs() - st; tqInfo("vgId:%d close&reload state elapsed time:%.3fs", vgId, el / 1000.); + code = streamMetaLoadAllTasks(pMeta); + if (code != TSDB_CODE_SUCCESS) { + tqError("vgId:%d failed to load stream tasks, code:%s", vgId, tstrerror(terrno)); + streamMetaWUnLock(pMeta); + code = terrno; + return code; + } + { STaskStartInfo* pStartInfo = &pMeta->startInfo; taosHashClear(pStartInfo->pReadyTaskSet); diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 30a40ffc9f..6bb90a9c25 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -469,6 +469,7 @@ void streamMetaClear(SStreamMeta* pMeta) { taosArrayClear(pMeta->pTaskList); taosArrayClear(pMeta->chkpSaved); taosArrayClear(pMeta->chkpInUse); + pMeta->numOfStreamTasks = 0; pMeta->numOfPausedTasks = 0; From f6c06491084fa1ce5a24fe2d28f7abcfbb0ad2da Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 11 Jan 2024 09:53:19 +0800 Subject: [PATCH 056/102] fix(stream): fix syntax error. --- source/dnode/snode/src/snode.c | 2 +- source/dnode/vnode/src/tqCommon/tqCommon.c | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/source/dnode/snode/src/snode.c b/source/dnode/snode/src/snode.c index 9112d9bda7..4815dfe65c 100644 --- a/source/dnode/snode/src/snode.c +++ b/source/dnode/snode/src/snode.c @@ -194,7 +194,7 @@ int32_t sndProcessWriteMsg(SSnode *pSnode, SRpcMsg *pMsg, SRpcMsg *pRsp) { case TDMT_STREAM_TASK_DEPLOY: { void * pReq = POINTER_SHIFT(pMsg->pCont, sizeof(SMsgHead)); int32_t len = pMsg->contLen - sizeof(SMsgHead); - return tqStreamTaskProcessDeployReq(pSnode->pMeta, pMsg->info.conn.applyIndex, pReq, len, true, true); + return tqStreamTaskProcessDeployReq(pSnode->pMeta, &pSnode->msgCb,pMsg->info.conn.applyIndex, pReq, len, true, true); } case TDMT_STREAM_TASK_DROP: diff --git a/source/dnode/vnode/src/tqCommon/tqCommon.c b/source/dnode/vnode/src/tqCommon/tqCommon.c index a745f7996e..da1c2ecf16 100644 --- a/source/dnode/vnode/src/tqCommon/tqCommon.c +++ b/source/dnode/vnode/src/tqCommon/tqCommon.c @@ -947,7 +947,7 @@ static int32_t tqProcessTaskResumeImpl(void* handle, SStreamTask* pTask, int64_t } streamTaskResume(pTask); - ETaskStatus status = streamTaskGetStatus(pTask, NULL); + ETaskStatus status = streamTaskGetStatus(pTask)->state; int32_t level = pTask->info.taskLevel; if (level == TASK_LEVEL__SINK) { From 686108050ba6bcde7946423ff1d6f9246dd9715f Mon Sep 17 00:00:00 2001 From: wangjiaming0909 <604227650@qq.com> Date: Wed, 10 Jan 2024 18:44:04 +0800 Subject: [PATCH 057/102] fix: table merge scan return disordered rows --- source/dnode/vnode/src/tsdb/tsdbRead2.c | 15 ++++++++++++++- source/dnode/vnode/src/tsdb/tsdbReadUtil.h | 1 + 2 files changed, 15 insertions(+), 1 deletion(-) diff --git a/source/dnode/vnode/src/tsdb/tsdbRead2.c b/source/dnode/vnode/src/tsdb/tsdbRead2.c index b57d2dfb45..256ef10c5e 100644 --- a/source/dnode/vnode/src/tsdb/tsdbRead2.c +++ b/source/dnode/vnode/src/tsdb/tsdbRead2.c @@ -73,6 +73,7 @@ static int32_t getInitialDelIndex(const SArray* pDelSkyline, int32_t order static void resetTableListIndex(SReaderStatus* pStatus); static void getMemTableTimeRange(STsdbReader* pReader, int64_t* pMaxKey, int64_t* pMinKey); static void updateComposedBlockInfo(STsdbReader* pReader, double el, STableBlockScanInfo* pBlockScanInfo); +static int32_t buildFromPreFilesetBuffer(STsdbReader* pReader); static bool outOfTimeWindow(int64_t ts, STimeWindow* pWindow) { return (ts > pWindow->ekey) || (ts < pWindow->skey); } @@ -3095,6 +3096,17 @@ static ERetrieveType doReadDataFromSttFiles(STsdbReader* pReader) { return TSDB_READ_RETURN; } + if (pReader->status.bProcMemPreFileset) { + code = buildFromPreFilesetBuffer(pReader); + if (code != TSDB_CODE_SUCCESS) { + return code; + } + if (pResBlock->info.rows > 0) { + pReader->status.processingMemPreFileSet = true; + 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 @@ -4342,6 +4354,7 @@ static int32_t buildFromPreFilesetBuffer(STsdbReader* pReader) { } else { tsdbDebug("finished pre-fileset %d buffer processing. %s", fid, pReader->idStr); pStatus->bProcMemPreFileset = false; + pStatus->processingMemPreFileSet = false; if (pReader->notifyFn) { STsdReaderNotifyInfo info = {0}; info.duration.filesetId = fid; @@ -4374,7 +4387,7 @@ static int32_t doTsdbNextDataBlockFilesetDelimited(STsdbReader* pReader) { pStatus->bProcMemFirstFileset, pReader->idStr); if (pStatus->bProcMemPreFileset) { if (pBlock->info.rows > 0) { - if (pReader->notifyFn) { + if (pReader->notifyFn && !pReader->status.processingMemPreFileSet) { int32_t fid = pReader->status.pCurrentFileset->fid; STsdReaderNotifyInfo info = {0}; info.duration.filesetId = fid; diff --git a/source/dnode/vnode/src/tsdb/tsdbReadUtil.h b/source/dnode/vnode/src/tsdb/tsdbReadUtil.h index 39e65f22b1..2157c7a423 100644 --- a/source/dnode/vnode/src/tsdb/tsdbReadUtil.h +++ b/source/dnode/vnode/src/tsdb/tsdbReadUtil.h @@ -219,6 +219,7 @@ typedef struct SReaderStatus { int64_t prevFilesetStartKey; int64_t prevFilesetEndKey; bool bProcMemFirstFileset; + bool processingMemPreFileSet; STableUidList procMemUidList; STableBlockScanInfo** pProcMemTableIter; } SReaderStatus; From df235b0adf1918842740496c0d54e6546907d413 Mon Sep 17 00:00:00 2001 From: Yihao Deng Date: Thu, 11 Jan 2024 03:22:34 +0000 Subject: [PATCH 058/102] split sync/status channel --- source/dnode/mgmt/mgmt_dnode/src/dmHandle.c | 2 +- source/libs/transport/src/transCli.c | 1 + source/libs/transport/src/transComm.c | 2 +- 3 files changed, 3 insertions(+), 2 deletions(-) diff --git a/source/dnode/mgmt/mgmt_dnode/src/dmHandle.c b/source/dnode/mgmt/mgmt_dnode/src/dmHandle.c index 545a5fc870..d83ee541df 100644 --- a/source/dnode/mgmt/mgmt_dnode/src/dmHandle.c +++ b/source/dnode/mgmt/mgmt_dnode/src/dmHandle.c @@ -161,7 +161,7 @@ void dmSendStatusReq(SDnodeMgmt *pMgmt) { SEpSet epSet = {0}; int8_t epUpdated = 0; dmGetMnodeEpSet(pMgmt->pData, &epSet); - rpcSendRecvWithTimeout(pMgmt->msgCb.statusRpc, &epSet, &rpcMsg, &rpcRsp, &epUpdated, tsStatusInterval * 10); + rpcSendRecvWithTimeout(pMgmt->msgCb.statusRpc, &epSet, &rpcMsg, &rpcRsp, &epUpdated, tsStatusInterval * 10 * 1000); if (rpcRsp.code != 0) { dmRotateMnodeEpSet(pMgmt->pData); char tbuf[512]; diff --git a/source/libs/transport/src/transCli.c b/source/libs/transport/src/transCli.c index 4b3000b47e..1ccf708703 100644 --- a/source/libs/transport/src/transCli.c +++ b/source/libs/transport/src/transCli.c @@ -2701,6 +2701,7 @@ int transSendRecvWithTimeout(void* shandle, SEpSet* pEpSet, STransMsg* pReq, STr ret = TSDB_CODE_TIMEOUT_ERROR; } else { memcpy(pRsp, pSyncMsg->pRsp, sizeof(STransMsg)); + pSyncMsg->pRsp->pCont = NULL; if (pSyncMsg->hasEpSet == 1) { epsetAssign(pEpSet, &pSyncMsg->epSet); *epUpdated = 1; diff --git a/source/libs/transport/src/transComm.c b/source/libs/transport/src/transComm.c index 759a4d79db..b1fb9a2450 100644 --- a/source/libs/transport/src/transComm.c +++ b/source/libs/transport/src/transComm.c @@ -667,7 +667,7 @@ void transDestroySyncMsg(void* msg) { STransSyncMsg* pSyncMsg = msg; tsem_destroy(pSyncMsg->pSem); taosMemoryFree(pSyncMsg->pSem); - + transFreeMsg(pSyncMsg->pRsp->pCont); taosMemoryFree(pSyncMsg->pRsp); taosMemoryFree(pSyncMsg); } From 58694c67dd73b187c6972d11c686997e2308f1f8 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 11 Jan 2024 11:50:52 +0800 Subject: [PATCH 059/102] refactor: do some internal refactor. --- include/libs/stream/tstream.h | 18 +++++++++--------- include/util/tworker.h | 5 ++--- source/common/src/systable.c | 2 +- source/common/src/tglobal.c | 4 ++-- source/dnode/mgmt/mgmt_vnode/src/vmWorker.c | 8 ++------ source/dnode/mnode/impl/src/mndStream.c | 2 +- source/libs/stream/src/streamMeta.c | 2 +- 7 files changed, 18 insertions(+), 23 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index d11a4ad23b..9ea655c15c 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -685,18 +685,18 @@ typedef struct STaskStatusEntry { int32_t statusLastDuration; // to record the last duration of current status int64_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 processedVer; // only valid for source task + 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 - int32_t chkpointTransId; // checkpoint trans id - bool checkpointFailed; // denote if the checkpoint is failed or not - bool inputQChanging; // inputQ is changing or not + int32_t chkpointTransId; // checkpoint trans id + bool checkpointFailed; // denote if the checkpoint is failed or not + bool inputQChanging; // inputQ is changing or not int64_t inputQUnchangeCounter; - double inputQUsed; // in MiB + double inputQUsed; // in MiB double inputRate; - double sinkQuota; // existed quota size for sink task - double sinkDataSize; // sink to dst data size + double sinkQuota; // existed quota size for sink task + double sinkDataSize; // sink to dst data size } STaskStatusEntry; typedef struct SStreamHbMsg { diff --git a/include/util/tworker.h b/include/util/tworker.h index 8508adf052..f39540d24b 100644 --- a/include/util/tworker.h +++ b/include/util/tworker.h @@ -23,7 +23,6 @@ extern "C" { #endif -typedef struct SQWorkerPool SQWorkerPool; typedef struct SWWorkerPool SWWorkerPool; typedef struct SQueueWorker { @@ -60,14 +59,14 @@ typedef struct SWWorker { SWWorkerPool *pool; } SWWorker; -typedef struct SWWorkerPool { +struct SWWorkerPool { int32_t max; // max number of workers int32_t num; int32_t nextId; // from 0 to max-1, cyclic const char *name; SWWorker *workers; TdThreadMutex mutex; -} SWWorkerPool; +}; int32_t tQWorkerInit(SQWorkerPool *pool); void tQWorkerCleanup(SQWorkerPool *pool); diff --git a/source/common/src/systable.c b/source/common/src/systable.c index 2e52c77080..75a54a0cd5 100644 --- a/source/common/src/systable.c +++ b/source/common/src/systable.c @@ -166,7 +166,7 @@ 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 = 15 + VARSTR_HEADER_SIZE, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = false}, + {.name = "status", .bytes = 12 + VARSTR_HEADER_SIZE, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = false}, {.name = "stage", .bytes = 8, .type = TSDB_DATA_TYPE_BIGINT, .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}, diff --git a/source/common/src/tglobal.c b/source/common/src/tglobal.c index 79d21955d4..b72c4fe077 100644 --- a/source/common/src/tglobal.c +++ b/source/common/src/tglobal.c @@ -58,7 +58,7 @@ int32_t tsNumOfMnodeQueryThreads = 4; int32_t tsNumOfMnodeFetchThreads = 1; int32_t tsNumOfMnodeReadThreads = 1; int32_t tsNumOfVnodeQueryThreads = 4; -float tsRatioOfVnodeStreamThreads = 4.0; +float tsRatioOfVnodeStreamThreads = 1.0; int32_t tsNumOfVnodeFetchThreads = 4; int32_t tsNumOfVnodeRsmaThreads = 2; int32_t tsNumOfQnodeQueryThreads = 4; @@ -621,7 +621,7 @@ static int32_t taosAddServerCfg(SConfig *pCfg) { 0) return -1; - if (cfgAddFloat(pCfg, "ratioOfVnodeStreamThreads", tsRatioOfVnodeStreamThreads, 0.01, 100, CFG_SCOPE_SERVER, + if (cfgAddFloat(pCfg, "ratioOfVnodeStreamThreads", tsRatioOfVnodeStreamThreads, 0.01, 10, CFG_SCOPE_SERVER, CFG_DYN_NONE) != 0) return -1; diff --git a/source/dnode/mgmt/mgmt_vnode/src/vmWorker.c b/source/dnode/mgmt/mgmt_vnode/src/vmWorker.c index 9a792a2774..8b80527447 100644 --- a/source/dnode/mgmt/mgmt_vnode/src/vmWorker.c +++ b/source/dnode/mgmt/mgmt_vnode/src/vmWorker.c @@ -407,12 +407,8 @@ int32_t vmStartWorker(SVnodeMgmt *pMgmt) { if (tWWorkerInit(pFPool) != 0) return -1; SSingleWorkerCfg mgmtCfg = { - .min = 1, - .max = 1, - .name = "vnode-mgmt", - .fp = (FItem)vmProcessMgmtQueue, - .param = pMgmt, - }; + .min = 1, .max = 1, .name = "vnode-mgmt", .fp = (FItem)vmProcessMgmtQueue, .param = pMgmt}; + if (tSingleWorkerInit(&pMgmt->mgmtWorker, &mgmtCfg) != 0) return -1; dDebug("vnode workers are initialized"); diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index 146f9f6fc4..95150b9d6e 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -1728,7 +1728,7 @@ static int32_t setTaskAttrInResBlock(SStreamObj *pStream, SStreamTask *pTask, SS 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++); diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index de435c63a3..04d34b0945 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -1102,7 +1102,7 @@ static int32_t metaHeartbeatToMnodeImpl(SStreamMeta* pMeta) { .inputQUsed = SIZE_IN_MiB(streamQueueGetItemSize((*pTask)->inputq.queue)), }; - entry.inputRate = entry.inputQUsed * 100.0 / STREAM_TASK_QUEUE_CAPACITY_IN_SIZE; + entry.inputRate = entry.inputQUsed * 100.0 / (2*STREAM_TASK_QUEUE_CAPACITY_IN_SIZE); if ((*pTask)->info.taskLevel == TASK_LEVEL__SINK) { entry.sinkQuota = (*pTask)->outputInfo.pTokenBucket->quotaRate; entry.sinkDataSize = SIZE_IN_MiB((*pTask)->execInfo.sink.dataSize); From 5f7ce21530cfc7535dc8fc80aedb03a27d60235b Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 11 Jan 2024 11:59:29 +0800 Subject: [PATCH 060/102] refactor: do some internal refactor. --- source/dnode/mnode/impl/inc/mndStream.h | 2 +- source/dnode/mnode/impl/src/mndStream.c | 21 ++++++++++---------- source/dnode/mnode/impl/src/mndStreamTrans.c | 4 ++-- 3 files changed, 13 insertions(+), 14 deletions(-) diff --git a/source/dnode/mnode/impl/inc/mndStream.h b/source/dnode/mnode/impl/inc/mndStream.h index 7b71ad873b..58a4c92d3e 100644 --- a/source/dnode/mnode/impl/inc/mndStream.h +++ b/source/dnode/mnode/impl/inc/mndStream.h @@ -72,7 +72,7 @@ int32_t mndPersistStream(SMnode *pMnode, STrans *pTrans, SStreamObj *pStream int32_t mndStreamRegisterTrans(STrans* pTrans, const char* pTransName, int64_t streamUid); int32_t mndAddtoCheckpointWaitingList(SStreamObj *pStream, int64_t checkpointId); bool mndStreamTransConflictCheck(SMnode *pMnode, int64_t streamUid, const char *pTransName, bool lock); -int32_t mndStreamGetRelCheckpointTrans(SMnode *pMnode, int64_t streamUid); +int32_t mndStreamGetRelTrans(SMnode *pMnode, int64_t streamUid); // for sma // TODO refactor diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index 95150b9d6e..a69543a6d6 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -81,7 +81,7 @@ static void removeStreamTasksInBuf(SStreamObj *pStream, SStreamExecInfo *pExe static void saveStreamTasksInfo(SStreamObj *pStream, SStreamExecInfo *pExecNode); static int32_t removeExpirednodeEntryAndTask(SArray *pNodeSnapshot); static int32_t doKillCheckpointTrans(SMnode *pMnode, const char *pDbName, size_t len); -static void killCheckpointTransImpl(SMnode *pMnode, int32_t transId, const char *pDbName); +static void killTransImpl(SMnode *pMnode, int32_t transId, const char *pDbName); static int32_t setNodeEpsetExpiredFlag(const SArray *pNodeList); static void freeCheckpointCandEntry(void *); @@ -95,9 +95,6 @@ static int32_t mndStreamSeqActionInsert(SSdb *pSdb, SStreamSeq *pStream); static int32_t mndStreamSeqActionDelete(SSdb *pSdb, SStreamSeq *pStream); static int32_t mndStreamSeqActionUpdate(SSdb *pSdb, SStreamSeq *pOldStream, SStreamSeq *pNewStream); -static SSdbRaw *mndStreamActionEncode(SStreamObj *pStream); -static SSdbRow *mndStreamActionDecode(SSdbRaw *pRaw); - int32_t mndInitStream(SMnode *pMnode) { SSdbTable table = { .sdbType = SDB_STREAM, @@ -1455,9 +1452,10 @@ static int32_t mndProcessDropStreamReq(SRpcMsg *pReq) { } // kill the related checkpoint trans - int32_t transId = mndStreamGetRelCheckpointTrans(pMnode, pStream->uid); + int32_t transId = mndStreamGetRelTrans(pMnode, pStream->uid); if (transId != 0) { - killCheckpointTransImpl(pMnode, transId, pStream->sourceDb); + mDebug("drop active related transId:%d due to stream:%s dropped", transId, pStream->name); + killTransImpl(pMnode, transId, pStream->sourceDb); } removeStreamTasksInBuf(pStream, &execInfo); @@ -1502,9 +1500,10 @@ int32_t mndDropStreamByDb(SMnode *pMnode, STrans *pTrans, SDbObj *pDb) { #endif // kill the related checkpoint trans - int32_t transId = mndStreamGetRelCheckpointTrans(pMnode, pStream->uid); + int32_t transId = mndStreamGetRelTrans(pMnode, pStream->uid); if (transId != 0) { - killCheckpointTransImpl(pMnode, transId, pStream->sourceDb); + mDebug("drop active related transId:%d due to stream:%s dropped", transId, pStream->name); + killTransImpl(pMnode, transId, pStream->sourceDb); } // drop the stream obj in execInfo @@ -2836,10 +2835,10 @@ int32_t createStreamResetStatusTrans(SMnode *pMnode, SStreamObj *pStream) { return TSDB_CODE_ACTION_IN_PROGRESS; } -void killCheckpointTransImpl(SMnode* pMnode, int32_t transId, const char* pDbName) { +void killTransImpl(SMnode* pMnode, int32_t transId, const char* pDbName) { STrans *pTrans = mndAcquireTrans(pMnode, transId); if (pTrans != NULL) { - mInfo("kill checkpoint transId:%d in Db:%s", transId, pDbName); + mInfo("kill active transId:%d in Db:%s", transId, pDbName); mndKillTrans(pMnode, pTrans); mndReleaseTrans(pMnode, pTrans); } @@ -2859,7 +2858,7 @@ int32_t doKillCheckpointTrans(SMnode *pMnode, const char *pDBName, size_t len) { } char* pDupDBName = strndup(pDBName, len); - killCheckpointTransImpl(pMnode, pTransInfo->transId, pDupDBName); + killTransImpl(pMnode, pTransInfo->transId, pDupDBName); taosMemoryFree(pDupDBName); return TSDB_CODE_SUCCESS; diff --git a/source/dnode/mnode/impl/src/mndStreamTrans.c b/source/dnode/mnode/impl/src/mndStreamTrans.c index f5047acc49..bfea4349b0 100644 --- a/source/dnode/mnode/impl/src/mndStreamTrans.c +++ b/source/dnode/mnode/impl/src/mndStreamTrans.c @@ -113,7 +113,7 @@ bool mndStreamTransConflictCheck(SMnode* pMnode, int64_t streamUid, const char* return false; } -int32_t mndStreamGetRelCheckpointTrans(SMnode* pMnode, int64_t streamUid) { +int32_t mndStreamGetRelTrans(SMnode* pMnode, int64_t streamUid) { taosThreadMutexLock(&execInfo.lock); int32_t num = taosHashGetSize(execInfo.transMgmt.pDBTrans); if (num <= 0) { @@ -127,7 +127,7 @@ int32_t mndStreamGetRelCheckpointTrans(SMnode* pMnode, int64_t streamUid) { SStreamTransInfo tInfo = *pEntry; taosThreadMutexUnlock(&execInfo.lock); - if (strcmp(tInfo.name, MND_STREAM_CHECKPOINT_NAME) == 0) { + if (strcmp(tInfo.name, MND_STREAM_CHECKPOINT_NAME) == 0 || strcmp(tInfo.name, MND_STREAM_TASK_UPDATE_NAME) == 0) { return tInfo.transId; } } else { From 44146caa9a73c947c69a215d640dbcd8bd671d09 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Thu, 11 Jan 2024 13:46:54 +0800 Subject: [PATCH 061/102] opt msg on mnd --- source/dnode/mgmt/mgmt_dnode/src/dmHandle.c | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/source/dnode/mgmt/mgmt_dnode/src/dmHandle.c b/source/dnode/mgmt/mgmt_dnode/src/dmHandle.c index d83ee541df..14853009e0 100644 --- a/source/dnode/mgmt/mgmt_dnode/src/dmHandle.c +++ b/source/dnode/mgmt/mgmt_dnode/src/dmHandle.c @@ -45,7 +45,7 @@ static void dmMayShouldUpdateIpWhiteList(SDnodeMgmt *pMgmt, int64_t ver) { SRetrieveIpWhiteReq req = {.ipWhiteVer = oldVer}; int32_t contLen = tSerializeRetrieveIpWhite(NULL, 0, &req); - void *pHead = rpcMallocCont(contLen); + void * pHead = rpcMallocCont(contLen); tSerializeRetrieveIpWhite(pHead, contLen, &req); SRpcMsg rpcMsg = {.pCont = pHead, @@ -144,7 +144,7 @@ void dmSendStatusReq(SDnodeMgmt *pMgmt) { req.ipWhiteVer = pMgmt->pData->ipWhiteVer; int32_t contLen = tSerializeSStatusReq(NULL, 0, &req); - void *pHead = rpcMallocCont(contLen); + void * pHead = rpcMallocCont(contLen); tSerializeSStatusReq(pHead, contLen, &req); tFreeSStatusReq(&req); @@ -161,7 +161,7 @@ void dmSendStatusReq(SDnodeMgmt *pMgmt) { SEpSet epSet = {0}; int8_t epUpdated = 0; dmGetMnodeEpSet(pMgmt->pData, &epSet); - rpcSendRecvWithTimeout(pMgmt->msgCb.statusRpc, &epSet, &rpcMsg, &rpcRsp, &epUpdated, tsStatusInterval * 10 * 1000); + rpcSendRecvWithTimeout(pMgmt->msgCb.statusRpc, &epSet, &rpcMsg, &rpcRsp, &epUpdated, tsStatusInterval * 5 * 1000); if (rpcRsp.code != 0) { dmRotateMnodeEpSet(pMgmt->pData); char tbuf[512]; @@ -189,7 +189,7 @@ void dmSendNotifyReq(SDnodeMgmt *pMgmt) { req.pVloads = vinfo.pVloads; int32_t contLen = tSerializeSNotifyReq(NULL, 0, &req); - void *pHead = rpcMallocCont(contLen); + void * pHead = rpcMallocCont(contLen); tSerializeSNotifyReq(pHead, contLen, &req); tFreeSNotifyReq(&req); @@ -284,7 +284,7 @@ int32_t dmProcessServerRunStatus(SDnodeMgmt *pMgmt, SRpcMsg *pMsg) { } SSDataBlock *dmBuildVariablesBlock(void) { - SSDataBlock *pBlock = taosMemoryCalloc(1, sizeof(SSDataBlock)); + SSDataBlock * pBlock = taosMemoryCalloc(1, sizeof(SSDataBlock)); size_t size = 0; const SSysTableMeta *pMeta = NULL; getInfosDbMeta(&pMeta, &size); From 423fa762074105df4bc6d03d4e8a6e91646c160f Mon Sep 17 00:00:00 2001 From: Yihao Deng Date: Thu, 11 Jan 2024 06:01:06 +0000 Subject: [PATCH 062/102] split sync/status channel --- source/dnode/mgmt/node_mgmt/src/dmTransport.c | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/source/dnode/mgmt/node_mgmt/src/dmTransport.c b/source/dnode/mgmt/node_mgmt/src/dmTransport.c index 512f090ab0..4f72ac24b4 100644 --- a/source/dnode/mgmt/node_mgmt/src/dmTransport.c +++ b/source/dnode/mgmt/node_mgmt/src/dmTransport.c @@ -364,7 +364,7 @@ int32_t dmInitClient(SDnode *pDnode) { SRpcInit rpcInit = {0}; rpcInit.label = "DND-C"; - rpcInit.numOfThreads = tsNumOfRpcThreads; + rpcInit.numOfThreads = tsNumOfRpcThreads / 2; rpcInit.cfp = (RpcCfp)dmProcessRpcMsg; rpcInit.sessions = 1024; rpcInit.connType = TAOS_CONN_CLIENT; @@ -383,7 +383,7 @@ int32_t dmInitClient(SDnode *pDnode) { rpcInit.failFastThreshold = 3; // failed threshold rpcInit.ffp = dmFailFastFp; - int32_t connLimitNum = tsNumOfRpcSessions / (tsNumOfRpcThreads * 3); + int32_t connLimitNum = tsNumOfRpcSessions / (tsNumOfRpcThreads * 3) / 2; connLimitNum = TMAX(connLimitNum, 10); connLimitNum = TMIN(connLimitNum, 500); @@ -453,7 +453,7 @@ int32_t dmInitSyncClient(SDnode *pDnode) { SRpcInit rpcInit = {0}; rpcInit.label = "DND-SYNC"; - rpcInit.numOfThreads = tsNumOfRpcThreads; + rpcInit.numOfThreads = tsNumOfRpcThreads / 2; rpcInit.cfp = (RpcCfp)dmProcessRpcMsg; rpcInit.sessions = 1024; rpcInit.connType = TAOS_CONN_CLIENT; @@ -472,7 +472,7 @@ int32_t dmInitSyncClient(SDnode *pDnode) { rpcInit.failFastThreshold = 3; // failed threshold rpcInit.ffp = dmFailFastFp; - int32_t connLimitNum = 100; + int32_t connLimitNum = tsNumOfRpcSessions / (tsNumOfRpcThreads * 3) / 2; connLimitNum = TMAX(connLimitNum, 10); connLimitNum = TMIN(connLimitNum, 500); From 7d46458bb70c38a5f5739e2ff6a5e510e8d4aa1f Mon Sep 17 00:00:00 2001 From: slzhou Date: Thu, 11 Jan 2024 14:50:14 +0800 Subject: [PATCH 063/102] fix: use slide start to calculate previous slide start --- source/common/src/ttime.c | 17 ++++++++--------- 1 file changed, 8 insertions(+), 9 deletions(-) diff --git a/source/common/src/ttime.c b/source/common/src/ttime.c index f683baee7c..bbda31efb1 100644 --- a/source/common/src/ttime.c +++ b/source/common/src/ttime.c @@ -873,16 +873,15 @@ int64_t taosTimeTruncate(int64_t ts, const SInterval* pInterval) { if (pInterval->offset > 0) { // try to move current window to the left-hande-side, due to the offset effect. - int64_t end = taosTimeAdd(start, pInterval->interval, pInterval->intervalUnit, precision) - 1; - int64_t newe = end; - int64_t slidingEnd = end; + int64_t newe = taosTimeAdd(start, pInterval->interval, pInterval->intervalUnit, precision) - 1; + int64_t slidingStart = start; while (newe >= ts) { - end = slidingEnd; - slidingEnd = taosTimeAdd(slidingEnd, -pInterval->sliding, pInterval->slidingUnit, precision); + start = slidingStart; + slidingStart = taosTimeAdd(slidingStart, -pInterval->sliding, pInterval->slidingUnit, precision); + int64_t slidingEnd = taosTimeAdd(slidingStart, pInterval->interval, pInterval->intervalUnit, precision) - 1; newe = taosTimeAdd(slidingEnd, pInterval->offset, pInterval->offsetUnit, precision); } - int64_t slidingStart = taosTimeAdd(end, -pInterval->interval, pInterval->intervalUnit, precision) + 1; - start = taosTimeAdd(slidingStart, pInterval->offset, pInterval->offsetUnit, precision); + start = taosTimeAdd(start, pInterval->offset, pInterval->offsetUnit, precision); } return start; @@ -892,8 +891,8 @@ int64_t taosTimeTruncate(int64_t ts, const SInterval* pInterval) { int64_t taosTimeGetIntervalEnd(int64_t intervalStart, const SInterval* pInterval) { if (pInterval->offset > 0) { int64_t slideStart = taosTimeAdd(intervalStart, -1 * pInterval->offset, pInterval->offsetUnit, pInterval->precision); - int64_t end = taosTimeAdd(slideStart, pInterval->interval, pInterval->intervalUnit, pInterval->precision) - 1; - int64_t result = taosTimeAdd(end, pInterval->offset, pInterval->offsetUnit, pInterval->precision); + int64_t slideEnd = taosTimeAdd(slideStart, pInterval->interval, pInterval->intervalUnit, pInterval->precision) - 1; + int64_t result = taosTimeAdd(slideEnd, pInterval->offset, pInterval->offsetUnit, pInterval->precision); return result; } else { int64_t result = taosTimeAdd(intervalStart, pInterval->interval, pInterval->intervalUnit, pInterval->precision) - 1; From d4fd544c745a3f46ef26ca7d9557945a8010d375 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 11 Jan 2024 15:18:11 +0800 Subject: [PATCH 064/102] refactor: remove sleep. --- source/libs/stream/src/streamExec.c | 49 ++++++++++--------- source/libs/stream/src/streamQueue.c | 33 +++---------- .../8-stream/at_once_state_window.py | 3 ++ 3 files changed, 35 insertions(+), 50 deletions(-) diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index eb5a432235..18f7ed061a 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -548,17 +548,9 @@ int32_t streamProcessTranstateBlock(SStreamTask* pTask, SStreamDataBlock* pBlock return code; } -static void setTaskSchedInfo(SStreamTask* pTask, int32_t idleTime) { - SStreamStatus* pStatus = &pTask->status; - - pStatus->schedIdleTime = idleTime; - pStatus->lastExecTs = taosGetTimestampMs(); -} - -static void clearTaskSchedInfo(SStreamTask* pTask) { - SStreamStatus* pStatus = &pTask->status; - pStatus->schedIdleTime = 0; -} +static void setTaskSchedInfo(SStreamTask* pTask, int32_t idleTime) { pTask->status.schedIdleTime = idleTime; } +static void clearTaskSchedInfo(SStreamTask* pTask) { pTask->status.schedIdleTime = 0; } +static void setLastExecTs(SStreamTask* pTask, int64_t ts) { pTask->status.lastExecTs = ts; } /** * todo: the batch of blocks should be tuned dynamic, according to the total elapsed time of each batch of blocks, the @@ -574,21 +566,28 @@ int32_t doStreamExecTask(SStreamTask* pTask) { int32_t blockSize = 0; int32_t numOfBlocks = 0; SStreamQueueItem* pInput = NULL; + if (streamTaskShouldStop(pTask) || (streamTaskGetStatus(pTask)->state == TASK_STATUS__UNINIT)) { stDebug("s-task:%s stream task is stopped", id); - break; + return 0; } if (streamQueueIsFull(pTask->outputq.queue)) { stWarn("s-task:%s outputQ is full, idle for 500ms and retry", id); setTaskSchedInfo(pTask, 500); - break; + return 0; } if (pTask->inputq.status == TASK_INPUT_STATUS__BLOCKED) { - stWarn("s-task:%s downstream task inputQ blocked, idle for 1sec and retry", pTask->id.idStr); + stWarn("s-task:%s downstream task inputQ blocked, idle for 1sec and retry", id); setTaskSchedInfo(pTask, 1000); - break; + return 0; + } + + if (taosGetTimestampMs() - pTask->status.lastExecTs < 50) { + stDebug("s-task:%s invoke with high frequency, idle and retry exec in 50ms", id); + setTaskSchedInfo(pTask, 50); + return 0; } /*int32_t code = */ streamTaskGetDataFromInputQ(pTask, &pInput, &numOfBlocks, &blockSize); @@ -597,9 +596,8 @@ int32_t doStreamExecTask(SStreamTask* pTask) { return 0; } - int32_t type = pInput->type; - // dispatch checkpoint msg to all downstream tasks + int32_t type = pInput->type; if (type == STREAM_INPUT__CHECKPOINT_TRIGGER) { streamProcessCheckpointBlock(pTask, (SStreamDataBlock*)pInput); continue; @@ -646,7 +644,7 @@ int32_t doStreamExecTask(SStreamTask* pTask) { if (ver != pInfo->processedVer) { stDebug("s-task:%s update processedVer(unsaved) from %" PRId64 " to %" PRId64 " nextProcessVer:%" PRId64 " ckpt:%" PRId64, - pTask->id.idStr, pInfo->processedVer, ver, pInfo->nextProcessVer, pInfo->checkpointVer); + id, pInfo->processedVer, ver, pInfo->nextProcessVer, pInfo->checkpointVer); pInfo->processedVer = ver; } @@ -659,7 +657,7 @@ int32_t doStreamExecTask(SStreamTask* pTask) { // todo add lock SStreamTaskState* pState = streamTaskGetStatus(pTask); if (pState->state == TASK_STATUS__CK) { - stDebug("s-task:%s checkpoint block received, set status:%s", pTask->id.idStr, pState->name); + stDebug("s-task:%s checkpoint block received, set status:%s", id, pState->name); streamTaskBuildCheckpoint(pTask); } else { // todo refactor @@ -672,8 +670,8 @@ int32_t doStreamExecTask(SStreamTask* pTask) { if (code != TSDB_CODE_SUCCESS) { // todo: let's retry send rsp to upstream/mnode - stError("s-task:%s failed to send checkpoint rsp to upstream, checkpointId:%d, code:%s", pTask->id.idStr, - 0, tstrerror(code)); + stError("s-task:%s failed to send checkpoint rsp to upstream, checkpointId:%d, code:%s", id, 0, + tstrerror(code)); } } @@ -774,19 +772,24 @@ int32_t streamResumeTask(SStreamTask* pTask) { // check if this task needs to be idle for a while if (pTask->status.schedIdleTime > 0) { - stDebug("s-task:%s idled, and will be invoked in %dms", id, pTask->status.schedIdleTime); schedTaskInFuture(pTask); taosThreadMutexUnlock(&pTask->lock); + setLastExecTs(pTask, taosGetTimestampMs()); return 0; } else { int32_t numOfItems = streamQueueGetNumOfItems(pTask->inputq.queue); + if ((numOfItems == 0) || streamTaskShouldStop(pTask) || streamTaskShouldPause(pTask)) { atomic_store_8(&pTask->status.schedStatus, TASK_SCHED_STATUS__INACTIVE); taosThreadMutexUnlock(&pTask->lock); + setLastExecTs(pTask, taosGetTimestampMs()); + char* p = streamTaskGetStatus(pTask)->name; - stDebug("s-task:%s exec completed, status:%s, sched-status:%d", id, p, pTask->status.schedStatus); + stDebug("s-task:%s exec completed, status:%s, sched-status:%d, lastExecTs:%" PRId64, id, p, + pTask->status.schedStatus, pTask->status.lastExecTs); + return 0; } } diff --git a/source/libs/stream/src/streamQueue.c b/source/libs/stream/src/streamQueue.c index 5f55285ab1..d3287a6b96 100644 --- a/source/libs/stream/src/streamQueue.c +++ b/source/libs/stream/src/streamQueue.c @@ -17,8 +17,7 @@ #define MAX_STREAM_EXEC_BATCH_NUM 32 #define MAX_SMOOTH_BURST_RATIO 5 // 5 sec -#define WAIT_FOR_DURATION 40 -#define OUTPUT_QUEUE_FULL_WAIT_DURATION 500 // 500 ms +#define WAIT_FOR_DURATION 10 // todo refactor: // read data from input queue @@ -161,7 +160,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 10ms", id); - taosMsleep(10); + taosMsleep(WAIT_FOR_DURATION); return TSDB_CODE_SUCCESS; } @@ -173,11 +172,10 @@ int32_t streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueItem** pInpu SStreamQueueItem* qItem = streamQueueNextItem(pTask->inputq.queue); if (qItem == NULL) { - if ((taskLevel == TASK_LEVEL__SOURCE || taskLevel == TASK_LEVEL__SINK) && (++retryTimes) < MAX_RETRY_TIMES) { - taosMsleep(WAIT_FOR_DURATION); - // todo remove it - continue; - } +// if ((taskLevel == TASK_LEVEL__SOURCE || taskLevel == TASK_LEVEL__SINK) && (++retryTimes) < MAX_RETRY_TIMES) { +// taosMsleep(WAIT_FOR_DURATION); +// continue; +// } // restore the token to bucket if (*numOfBlocks > 0) { @@ -344,25 +342,6 @@ int32_t streamTaskPutDataIntoInputQ(SStreamTask* pTask, SStreamQueueItem* pItem) // the result should be put into the outputQ in any cases, the result may be lost otherwise. int32_t streamTaskPutDataIntoOutputQ(SStreamTask* pTask, SStreamDataBlock* pBlock) { STaosQueue* pQueue = pTask->outputq.queue->pQueue; - -#if 0 - // wait for the output queue is available for new data to dispatch - while (streamQueueIsFull(pTask->outputq.queue)) { - if (streamTaskShouldStop(pTask)) { - 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->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 %dms and retry, outputQ items:%d, size:%.2fMiB", pTask->id.idStr, - OUTPUT_QUEUE_FULL_WAIT_DURATION, total, size); - - taosMsleep(OUTPUT_QUEUE_FULL_WAIT_DURATION); - } -#endif - int32_t code = taosWriteQitem(pQueue, pBlock); int32_t total = streamQueueGetNumOfItems(pTask->outputq.queue); diff --git a/tests/system-test/8-stream/at_once_state_window.py b/tests/system-test/8-stream/at_once_state_window.py index fa9f4ddd78..60a4f4e890 100644 --- a/tests/system-test/8-stream/at_once_state_window.py +++ b/tests/system-test/8-stream/at_once_state_window.py @@ -59,6 +59,9 @@ class TDTestCase: self.tdCom.create_stream(stream_name=f'{self.tb_name}{self.tdCom.stream_suffix}', des_table=self.tb_stream_des_table, source_sql=f'select _wstart AS wstart, {self.tdCom.tb_source_select_str} from {self.tb_name} partition by {partition} {partition_elm_alias} state_window({stream_state_window})', trigger_mode="at_once", subtable_value=tb_subtable_value, fill_history_value=fill_history_value) range_times = self.tdCom.range_count state_window_max = self.tdCom.dataDict['state_window_max'] + + time.sleep(2) + for i in range(range_times): state_window_value = random.randint(int((i)*state_window_max/range_times), int((i+1)*state_window_max/range_times)) for i in range(2, range_times+3): From 5e1a7c694b790f2ece43112dd9127935e4d7079c Mon Sep 17 00:00:00 2001 From: Yihao Deng Date: Thu, 11 Jan 2024 07:23:46 +0000 Subject: [PATCH 065/102] split sync/status channel --- include/util/tdef.h | 34 +++++++++---------- source/dnode/mgmt/node_mgmt/src/dmTransport.c | 6 ++-- source/libs/transport/src/transCli.c | 7 +++- 3 files changed, 26 insertions(+), 21 deletions(-) diff --git a/include/util/tdef.h b/include/util/tdef.h index 875a6f5738..aee20514ad 100644 --- a/include/util/tdef.h +++ b/include/util/tdef.h @@ -109,12 +109,12 @@ extern const int32_t TYPE_BYTES[21]; #define TSDB_INS_USER_STABLES_DBNAME_COLID 2 static const int64_t TICK_PER_SECOND[] = { - 1000LL, // MILLISECOND - 1000000LL, // MICROSECOND - 1000000000LL, // NANOSECOND - 0LL, // HOUR - 0LL, // MINUTE - 1LL // SECOND + 1000LL, // MILLISECOND + 1000000LL, // MICROSECOND + 1000000000LL, // NANOSECOND + 0LL, // HOUR + 0LL, // MINUTE + 1LL // SECOND }; #define TSDB_TICK_PER_SECOND(precision) \ @@ -239,8 +239,8 @@ typedef enum ELogicConditionType { #define TSDB_MAX_SQL_SHOW_LEN 1024 #define TSDB_MAX_ALLOWED_SQL_LEN (1 * 1024 * 1024u) // sql length should be less than 1mb -#define TSDB_VIEW_NAME_LEN 193 -#define TSDB_VIEW_FNAME_LEN (TSDB_DB_FNAME_LEN + TSDB_VIEW_NAME_LEN + TSDB_NAME_DELIMITER_LEN) +#define TSDB_VIEW_NAME_LEN 193 +#define TSDB_VIEW_FNAME_LEN (TSDB_DB_FNAME_LEN + TSDB_VIEW_NAME_LEN + TSDB_NAME_DELIMITER_LEN) #define TSDB_APP_NAME_LEN TSDB_UNI_LEN #define TSDB_TB_COMMENT_LEN 1025 @@ -260,7 +260,7 @@ typedef enum ELogicConditionType { #define TSDB_PASSWORD_LEN 32 #define TSDB_USET_PASSWORD_LEN 129 #define TSDB_VERSION_LEN 32 -#define TSDB_LABEL_LEN 12 +#define TSDB_LABEL_LEN 16 #define TSDB_JOB_STATUS_LEN 32 #define TSDB_CLUSTER_ID_LEN 40 @@ -288,7 +288,7 @@ typedef enum ELogicConditionType { #define TSDB_ACTIVE_KEY_LEN 109 #define TSDB_CONN_ACTIVE_KEY_LEN 255 -#define TSDB_DEFAULT_PKT_SIZE 65480 // same as RPC_MAX_UDP_SIZE +#define TSDB_DEFAULT_PKT_SIZE 65480 // same as RPC_MAX_UDP_SIZE #define TSDB_SNAP_DATA_PAYLOAD_SIZE (1 * 1024 * 1024) #define TSDB_PAYLOAD_SIZE TSDB_DEFAULT_PKT_SIZE @@ -397,13 +397,13 @@ typedef enum ELogicConditionType { #define TSDB_MAX_STT_TRIGGER 1 #define TSDB_DEFAULT_SST_TRIGGER 1 #endif -#define TSDB_STT_TRIGGER_ARRAY_SIZE 16 // maximum of TSDB_MAX_STT_TRIGGER of TD_ENTERPRISE and TD_COMMUNITY -#define TSDB_MIN_HASH_PREFIX (2 - TSDB_TABLE_NAME_LEN) -#define TSDB_MAX_HASH_PREFIX (TSDB_TABLE_NAME_LEN - 2) -#define TSDB_DEFAULT_HASH_PREFIX 0 -#define TSDB_MIN_HASH_SUFFIX (2 - TSDB_TABLE_NAME_LEN) -#define TSDB_MAX_HASH_SUFFIX (TSDB_TABLE_NAME_LEN - 2) -#define TSDB_DEFAULT_HASH_SUFFIX 0 +#define TSDB_STT_TRIGGER_ARRAY_SIZE 16 // maximum of TSDB_MAX_STT_TRIGGER of TD_ENTERPRISE and TD_COMMUNITY +#define TSDB_MIN_HASH_PREFIX (2 - TSDB_TABLE_NAME_LEN) +#define TSDB_MAX_HASH_PREFIX (TSDB_TABLE_NAME_LEN - 2) +#define TSDB_DEFAULT_HASH_PREFIX 0 +#define TSDB_MIN_HASH_SUFFIX (2 - TSDB_TABLE_NAME_LEN) +#define TSDB_MAX_HASH_SUFFIX (TSDB_TABLE_NAME_LEN - 2) +#define TSDB_DEFAULT_HASH_SUFFIX 0 #define TSDB_DB_MIN_WAL_RETENTION_PERIOD -1 #define TSDB_REP_DEF_DB_WAL_RET_PERIOD 3600 diff --git a/source/dnode/mgmt/node_mgmt/src/dmTransport.c b/source/dnode/mgmt/node_mgmt/src/dmTransport.c index 4f72ac24b4..ceafdbd2e2 100644 --- a/source/dnode/mgmt/node_mgmt/src/dmTransport.c +++ b/source/dnode/mgmt/node_mgmt/src/dmTransport.c @@ -363,7 +363,7 @@ int32_t dmInitClient(SDnode *pDnode) { SDnodeTrans *pTrans = &pDnode->trans; SRpcInit rpcInit = {0}; - rpcInit.label = "DND-C"; + rpcInit.label = "DND-CLI"; rpcInit.numOfThreads = tsNumOfRpcThreads / 2; rpcInit.cfp = (RpcCfp)dmProcessRpcMsg; rpcInit.sessions = 1024; @@ -407,7 +407,7 @@ int32_t dmInitStatusClient(SDnode *pDnode) { SDnodeTrans *pTrans = &pDnode->trans; SRpcInit rpcInit = {0}; - rpcInit.label = "DND-STATUS"; + rpcInit.label = "DND-STATUS-CLI"; rpcInit.numOfThreads = 1; rpcInit.cfp = (RpcCfp)dmProcessRpcMsg; rpcInit.sessions = 1024; @@ -452,7 +452,7 @@ int32_t dmInitSyncClient(SDnode *pDnode) { SDnodeTrans *pTrans = &pDnode->trans; SRpcInit rpcInit = {0}; - rpcInit.label = "DND-SYNC"; + rpcInit.label = "DND-SYNC-CLI"; rpcInit.numOfThreads = tsNumOfRpcThreads / 2; rpcInit.cfp = (RpcCfp)dmProcessRpcMsg; rpcInit.sessions = 1024; diff --git a/source/libs/transport/src/transCli.c b/source/libs/transport/src/transCli.c index 1ccf708703..8c65a3ac1b 100644 --- a/source/libs/transport/src/transCli.c +++ b/source/libs/transport/src/transCli.c @@ -1907,7 +1907,12 @@ bool cliRecvReleaseReq(SCliConn* conn, STransMsgHead* pHead) { static void* cliWorkThread(void* arg) { SCliThrd* pThrd = (SCliThrd*)arg; pThrd->pid = taosGetSelfPthreadId(); - setThreadName("trans-cli-work"); + + char threadName[TSDB_LABEL_LEN] = {0}; + STrans* pInst = pThrd->pTransInst; + strntolower(threadName, pInst->label, sizeof(threadName)); + setThreadName(threadName); + uv_run(pThrd->loop, UV_RUN_DEFAULT); tDebug("thread quit-thread:%08" PRId64, pThrd->pid); From 5a1510aa2fd7769d6916cac544455d20268fc4d2 Mon Sep 17 00:00:00 2001 From: Yihao Deng Date: Thu, 11 Jan 2024 07:30:03 +0000 Subject: [PATCH 066/102] split sync/status channel --- source/dnode/mgmt/node_mgmt/src/dmTransport.c | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/source/dnode/mgmt/node_mgmt/src/dmTransport.c b/source/dnode/mgmt/node_mgmt/src/dmTransport.c index ceafdbd2e2..d79e9ff8ef 100644 --- a/source/dnode/mgmt/node_mgmt/src/dmTransport.c +++ b/source/dnode/mgmt/node_mgmt/src/dmTransport.c @@ -407,7 +407,7 @@ int32_t dmInitStatusClient(SDnode *pDnode) { SDnodeTrans *pTrans = &pDnode->trans; SRpcInit rpcInit = {0}; - rpcInit.label = "DND-STATUS-CLI"; + rpcInit.label = "DNODE-STA-CLI"; rpcInit.numOfThreads = 1; rpcInit.cfp = (RpcCfp)dmProcessRpcMsg; rpcInit.sessions = 1024; @@ -452,7 +452,7 @@ int32_t dmInitSyncClient(SDnode *pDnode) { SDnodeTrans *pTrans = &pDnode->trans; SRpcInit rpcInit = {0}; - rpcInit.label = "DND-SYNC-CLI"; + rpcInit.label = "DNODE-SYNC-CLI"; rpcInit.numOfThreads = tsNumOfRpcThreads / 2; rpcInit.cfp = (RpcCfp)dmProcessRpcMsg; rpcInit.sessions = 1024; From b0f670947452c47d6560a540ebdaf7edcc1a3222 Mon Sep 17 00:00:00 2001 From: Yihao Deng Date: Thu, 11 Jan 2024 07:30:39 +0000 Subject: [PATCH 067/102] split sync/status channel --- source/dnode/mgmt/node_mgmt/src/dmTransport.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/dnode/mgmt/node_mgmt/src/dmTransport.c b/source/dnode/mgmt/node_mgmt/src/dmTransport.c index d79e9ff8ef..3b7ecce77c 100644 --- a/source/dnode/mgmt/node_mgmt/src/dmTransport.c +++ b/source/dnode/mgmt/node_mgmt/src/dmTransport.c @@ -363,7 +363,7 @@ int32_t dmInitClient(SDnode *pDnode) { SDnodeTrans *pTrans = &pDnode->trans; SRpcInit rpcInit = {0}; - rpcInit.label = "DND-CLI"; + rpcInit.label = "DNODE-CLI"; rpcInit.numOfThreads = tsNumOfRpcThreads / 2; rpcInit.cfp = (RpcCfp)dmProcessRpcMsg; rpcInit.sessions = 1024; From 3cea4fd2c79e8f1727acf837228219cf9a8985f3 Mon Sep 17 00:00:00 2001 From: slzhou Date: Thu, 11 Jan 2024 16:41:11 +0800 Subject: [PATCH 068/102] fix: getNextTimeWindow use new time algorithm --- source/common/src/ttime.c | 12 +++++++++- source/libs/executor/src/executil.c | 34 +++++++++-------------------- 2 files changed, 21 insertions(+), 25 deletions(-) diff --git a/source/common/src/ttime.c b/source/common/src/ttime.c index bbda31efb1..775cff6670 100644 --- a/source/common/src/ttime.c +++ b/source/common/src/ttime.c @@ -683,6 +683,10 @@ int32_t parseNatualDuration(const char* token, int32_t tokenLen, int64_t* durati return getDuration(*duration, *unit, duration, timePrecision); } +static bool taosIsLeapYear(int32_t year) { + return (year % 4 == 0 && (year % 100 != 0 || year % 400 == 0)); +} + int64_t taosTimeAdd(int64_t t, int64_t duration, char unit, int32_t precision) { if (duration == 0) { return t; @@ -702,7 +706,13 @@ int64_t taosTimeAdd(int64_t t, int64_t duration, char unit, int32_t precision) { int32_t mon = tm.tm_year * 12 + tm.tm_mon + (int32_t)numOfMonth; tm.tm_year = mon / 12; tm.tm_mon = mon % 12; - + int daysOfMonth[] = {31, 28, 31, 30, 31, 30, 31, 31, 30, 31, 30, 31}; + if (taosIsLeapYear(1900 + tm.tm_year)) { + daysOfMonth[1] = 29; + } + if (tm.tm_mday > daysOfMonth[tm.tm_mon]) { + tm.tm_mday = daysOfMonth[tm.tm_mon]; + } return (int64_t)(taosMktime(&tm) * TSDB_TICK_PER_SECOND(precision) + fraction); } diff --git a/source/libs/executor/src/executil.c b/source/libs/executor/src/executil.c index 019483a015..efab09f02b 100644 --- a/source/libs/executor/src/executil.c +++ b/source/libs/executor/src/executil.c @@ -1887,31 +1887,17 @@ STimeWindow getActiveTimeWindow(SDiskbasedBuf* pBuf, SResultRowInfo* pResultRowI } void getNextTimeWindow(const SInterval* pInterval, STimeWindow* tw, int32_t order) { + int64_t slidingStart = 0; + if (pInterval->offset > 0) { + slidingStart = taosTimeAdd(tw->skey, -1 * pInterval->offset, pInterval->offsetUnit, pInterval->precision); + } else { + slidingStart = tw->skey; + } int32_t factor = GET_FORWARD_DIRECTION_FACTOR(order); - if (!IS_CALENDAR_TIME_DURATION(pInterval->slidingUnit)) { - tw->skey += pInterval->sliding * factor; - tw->ekey = taosTimeAdd(tw->skey, pInterval->interval, pInterval->intervalUnit, pInterval->precision) - 1; - return; - } - - // convert key to second - int64_t key = convertTimePrecision(tw->skey, pInterval->precision, TSDB_TIME_PRECISION_MILLI) / 1000; - - int64_t duration = pInterval->sliding; - if (pInterval->slidingUnit == 'y') { - duration *= 12; - } - - struct tm tm; - time_t t = (time_t)key; - taosLocalTime(&t, &tm, NULL); - - int mon = (int)(tm.tm_year * 12 + tm.tm_mon + duration * factor); - tm.tm_year = mon / 12; - tm.tm_mon = mon % 12; - tw->skey = convertTimePrecision((int64_t)taosMktime(&tm) * 1000LL, TSDB_TIME_PRECISION_MILLI, pInterval->precision); - - tw->ekey = taosTimeAdd(tw->skey, pInterval->interval, pInterval->intervalUnit, pInterval->precision) - 1; + slidingStart = taosTimeAdd(slidingStart, factor * pInterval->sliding, pInterval->slidingUnit, pInterval->precision); + tw->skey = taosTimeAdd(slidingStart, pInterval->offset, pInterval->offsetUnit, pInterval->precision); + int64_t slidingEnd = taosTimeAdd(slidingStart, pInterval->interval, pInterval->intervalUnit, pInterval->precision) - 1; + tw->ekey = taosTimeAdd(slidingEnd, pInterval->offset, pInterval->offsetUnit, pInterval->precision); } bool hasLimitOffsetInfo(SLimitInfo* pLimitInfo) { From 9703018f561fedc209e23ef7829187a37cd8556c Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 11 Jan 2024 16:47:51 +0800 Subject: [PATCH 069/102] fix(stream): add checkpoint status check. --- source/dnode/mnode/impl/src/mndStream.c | 13 -------- source/libs/stream/src/streamCheckpoint.c | 40 ++++++++++++++--------- source/libs/stream/src/streamQueue.c | 5 --- 3 files changed, 25 insertions(+), 33 deletions(-) diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index a69543a6d6..f4ece1a141 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -3116,19 +3116,6 @@ int32_t mndProcessStreamHb(SRpcMsg *pReq) { if (p->status != TASK_STATUS__READY) { mDebug("received s-task:0x%" PRIx64 " not in ready status:%s", p->id.taskId, streamTaskGetStatusStr(p->status)); - -// if (p->status == TASK_STATUS__STREAM_SCAN_HISTORY) { -// bool drop = needDropRelatedFillhistoryTask(pTaskEntry, &execInfo); -// if (drop) { -// SStreamObj *pStreamObj = mndGetStreamObj(pMnode, pTaskEntry->id.streamId); -// if (pStreamObj == NULL) { -// mError("failed to acquire the streamObj:0x%" PRIx64 " it may have been dropped", pStreamObj->uid); -// } else { -// mndDropRelatedFillhistoryTask(pMnode, pTaskEntry, pStreamObj); -// mndReleaseStream(pMnode, pStreamObj); -// } -// } -// } } } diff --git a/source/libs/stream/src/streamCheckpoint.c b/source/libs/stream/src/streamCheckpoint.c index c75185b0ea..1c9361fa61 100644 --- a/source/libs/stream/src/streamCheckpoint.c +++ b/source/libs/stream/src/streamCheckpoint.c @@ -288,10 +288,11 @@ void streamTaskClearCheckInfo(SStreamTask* pTask, bool clearChkpReadyMsg) { } int32_t streamSaveTaskCheckpointInfo(SStreamTask* p, int64_t checkpointId) { - SStreamMeta* pMeta = p->pMeta; - int32_t vgId = pMeta->vgId; - const char* id = p->id.idStr; - int32_t code = 0; + SStreamMeta* pMeta = p->pMeta; + int32_t vgId = pMeta->vgId; + const char* id = p->id.idStr; + int32_t code = 0; + SCheckpointInfo* pCKInfo = &p->chkInfo; if (p->info.fillHistory == 1) { return code; @@ -303,25 +304,33 @@ int32_t streamSaveTaskCheckpointInfo(SStreamTask* p, int64_t checkpointId) { taosThreadMutexLock(&p->lock); - ASSERT(p->chkInfo.checkpointId <= p->chkInfo.checkpointingId && p->chkInfo.checkpointingId == checkpointId && - p->chkInfo.checkpointVer <= p->chkInfo.processedVer); - p->chkInfo.checkpointId = p->chkInfo.checkpointingId; - p->chkInfo.checkpointVer = p->chkInfo.processedVer; + SStreamTaskState* pStatus = streamTaskGetStatus(p); + if (pStatus->state == TASK_STATUS__CK) { + ASSERT(pCKInfo->checkpointId <= pCKInfo->checkpointingId && pCKInfo->checkpointingId == checkpointId && + pCKInfo->checkpointVer <= pCKInfo->processedVer); - streamTaskClearCheckInfo(p, false); - SStreamTaskState* pState = streamTaskGetStatus(p); + pCKInfo->checkpointId = pCKInfo->checkpointingId; + pCKInfo->checkpointVer = pCKInfo->processedVer; - code = streamTaskHandleEvent(p->status.pSM, TASK_EVENT_CHECKPOINT_DONE); - taosThreadMutexUnlock(&p->lock); + streamTaskClearCheckInfo(p, false); + code = streamTaskHandleEvent(p->status.pSM, TASK_EVENT_CHECKPOINT_DONE); + taosThreadMutexUnlock(&p->lock); + } else { + stDebug("s-task:%s vgId:%d status:%s not keep the checkpoint metaInfo, checkpoint:%" PRId64 " failed", id, vgId, + pStatus->name, pCKInfo->checkpointingId); + taosThreadMutexUnlock(&p->lock); + + return TSDB_CODE_STREAM_TASK_IVLD_STATUS; + } if (code != TSDB_CODE_SUCCESS) { stDebug("s-task:%s vgId:%d handle event:checkpoint-done failed", id, vgId); - return -1; + return code; } stDebug("vgId:%d s-task:%s level:%d open upstream inputQ, save status after checkpoint, checkpointId:%" PRId64 ", Ver(saved):%" PRId64 " currentVer:%" PRId64 ", status: normal, prev:%s", - vgId, id, p->info.taskLevel, checkpointId, p->chkInfo.checkpointVer, p->chkInfo.nextProcessVer, pState->name); + vgId, id, p->info.taskLevel, checkpointId, pCKInfo->checkpointVer, pCKInfo->nextProcessVer, pStatus->name); // save the task if not sink task if (p->info.taskLevel != TASK_LEVEL__SINK) { @@ -343,6 +352,7 @@ int32_t streamSaveTaskCheckpointInfo(SStreamTask* p, int64_t checkpointId) { streamMetaWUnLock(pMeta); } + return code; } @@ -474,7 +484,7 @@ int32_t streamTaskBuildCheckpoint(SStreamTask* pTask) { code = streamSaveTaskCheckpointInfo(pTask, ckId); if (code != TSDB_CODE_SUCCESS) { stError("s-task:%s commit taskInfo failed, checkpoint:%" PRId64 " failed, code:%s", pTask->id.idStr, ckId, - tstrerror(terrno)); + tstrerror(code)); } else { code = streamTaskUploadChkp(pTask, ckId, (char*)pTask->id.idStr); if (code != 0) { diff --git a/source/libs/stream/src/streamQueue.c b/source/libs/stream/src/streamQueue.c index d3287a6b96..8b5b2da061 100644 --- a/source/libs/stream/src/streamQueue.c +++ b/source/libs/stream/src/streamQueue.c @@ -160,7 +160,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 10ms", id); - taosMsleep(WAIT_FOR_DURATION); return TSDB_CODE_SUCCESS; } @@ -172,10 +171,6 @@ int32_t streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueItem** pInpu SStreamQueueItem* qItem = streamQueueNextItem(pTask->inputq.queue); if (qItem == NULL) { -// if ((taskLevel == TASK_LEVEL__SOURCE || taskLevel == TASK_LEVEL__SINK) && (++retryTimes) < MAX_RETRY_TIMES) { -// taosMsleep(WAIT_FOR_DURATION); -// continue; -// } // restore the token to bucket if (*numOfBlocks > 0) { From d3cfa9774040391e22a614571f3c0dee0858afe9 Mon Sep 17 00:00:00 2001 From: Yihao Deng Date: Thu, 11 Jan 2024 09:41:02 +0000 Subject: [PATCH 070/102] fix invalid read --- source/libs/transport/src/transCli.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/libs/transport/src/transCli.c b/source/libs/transport/src/transCli.c index 8c65a3ac1b..246605694b 100644 --- a/source/libs/transport/src/transCli.c +++ b/source/libs/transport/src/transCli.c @@ -1910,7 +1910,7 @@ static void* cliWorkThread(void* arg) { char threadName[TSDB_LABEL_LEN] = {0}; STrans* pInst = pThrd->pTransInst; - strntolower(threadName, pInst->label, sizeof(threadName)); + strtolower(threadName, pInst->label); setThreadName(threadName); uv_run(pThrd->loop, UV_RUN_DEFAULT); From e13ee9a37fe6aad16a3ed8a9a133fe2bb5899704 Mon Sep 17 00:00:00 2001 From: Alex Duan <417921451@qq.com> Date: Thu, 11 Jan 2024 17:46:50 +0800 Subject: [PATCH 071/102] coverage : modify coverage_test.sh to 2.0 --- tests/script/coverage_test.sh | 49 ++++++++++++++++++++++------------- 1 file changed, 31 insertions(+), 18 deletions(-) diff --git a/tests/script/coverage_test.sh b/tests/script/coverage_test.sh index 1b9e9753d3..136530e451 100644 --- a/tests/script/coverage_test.sh +++ b/tests/script/coverage_test.sh @@ -9,8 +9,14 @@ else exit 1 fi +# work main path +TDENGINE_DIR=/root/TDinternal/community +if [ x$2 != x ];then + TDENGINE_DIR=$2 +fi + +echo "TDENGINE_DIR=$TDENGINE_DIR" today=`date +"%Y%m%d"` -TDENGINE_DIR=/root/TDengine JDBC_DIR=/root/taos-connector-jdbc TDENGINE_COVERAGE_REPORT=$TDENGINE_DIR/tests/coverage-report-$today.log @@ -23,7 +29,16 @@ NC='\033[0m' function buildTDengine() { echo "check if TDengine need build" + + # pull parent code + cd "$TDENGINE_DIR/../" + echo "git pull parent code..." + git remote prune origin > /dev/null + git remote update > /dev/null + + # pull tdengine code cd $TDENGINE_DIR + echo "git pull tdengine code..." git remote prune origin > /dev/null git remote update > /dev/null REMOTE_COMMIT=`git rev-parse --short remotes/origin/$branch` @@ -54,13 +69,11 @@ function buildTDengine() { LOCAL_COMMIT=`git rev-parse --short @` rm -rf * - if [ "$branch" == "3.0" ]; then - echo "3.0 =============" - cmake -DCOVER=true -DBUILD_TEST=true -DBUILD_HTTP=false -DBUILD_TOOLS=true -DBUILD_GEOS=true -DBUILD_CONTRIB=true .. - else - cmake -DCOVER=true -DBUILD_TOOLS=true -DBUILD_HTTP=false .. > /dev/null - fi - make -j + makecmd="cmake -DCOVER=true -DBUILD_TEST=true -DBUILD_HTTP=false -DBUILD_TOOLS=true -DBUILD_GEOS=true -DBUILD_CONTRIB=true ../../" + echo "$makecmd" + $makecmd + + make -j 8 make install } @@ -71,18 +84,18 @@ function runCasesOneByOne () { if [[ "$2" == "sim" ]] && [[ $line == *"script"* ]]; then case=`echo $cmd | cut -d' ' -f 3` start_time=`date +%s` - date +%F\ %T | tee -a $TDENGINE_COVERAGE_REPORT && $cmd > /dev/null 2>&1 && \ + date +%F\ %T | tee -a $TDENGINE_COVERAGE_REPORT && timeout 30m $cmd > /dev/null 2>&1 && \ echo -e "${GREEN}$case success${NC}" | tee -a $TDENGINE_COVERAGE_REPORT \ || echo -e "${RED}$case failed${NC}" | tee -a $TDENGINE_COVERAGE_REPORT end_time=`date +%s` echo execution time of $case was `expr $end_time - $start_time`s. | tee -a $TDENGINE_COVERAGE_REPORT - elif [$line == *"$2"* ]; then + elif [[ "$line" == *"$2"* ]]; then if [[ "$cmd" == *"pytest.sh"* ]]; then cmd=`echo $cmd | cut -d' ' -f 2-20` fi case=`echo $cmd | cut -d' ' -f 4-20` start_time=`date +%s` - date +%F\ %T | tee -a $TDENGINE_COVERAGE_REPORT && $cmd > /dev/null 2>&1 && \ + date +%F\ %T | tee -a $TDENGINE_COVERAGE_REPORT && timeout 30m $cmd > /dev/null 2>&1 && \ echo -e "${GREEN}$case success${NC}" | tee -a $TDENGINE_COVERAGE_REPORT || \ echo -e "${RED}$case failed${NC}" | tee -a $TDENGINE_COVERAGE_REPORT end_time=`date +%s` @@ -178,7 +191,7 @@ function runTest() { runPythonCases runJDBCCases - stopTaosd + stopTaosd cd $TDENGINE_DIR/tests/script find . -name '*.sql' | xargs rm -f @@ -191,10 +204,10 @@ function lcovFunc { cd $TDENGINE_DIR # collect data - lcov --ignore-errors negative --ignore-errors mismatch -d . --capture --rc lcov_branch_coverage=1 --rc branch_coverage=1 --no-external -b $TDENGINE_DIR -o coverage.info + lcov --capture -d $TDENGINE_DIR -b $TDENGINE_DIR -o coverage.info --ignore-errors negative,mismatch,inconsistent,source --branch-coverage --function-coverage --no-external # remove exclude paths - lcov --remove coverage.info \ + lcov --remove coverage1.info \ '*/contrib/*' '*/tests/*' '*/test/*' '*/packaging/*' '*/taos-tools/*' '*/taosadapter/*' '*/TSZ/*' \ '*/AccessBridgeCalls.c' '*/ttszip.c' '*/dataInserter.c' '*/tlinearhash.c' '*/tsimplehash.c' '*/tsdbDiskData.c'\ '*/texpr.c' '*/runUdf.c' '*/schDbg.c' '*/syncIO.c' '*/tdbOs.c' '*/pushServer.c' '*/osLz4.c'\ @@ -202,23 +215,23 @@ function lcovFunc { '*/clientJniConnector.c' '*/clientTmqConnector.c' '*/version.c'\ '*/tthread.c' '*/tversion.c' '*/ctgDbg.c' '*/schDbg.c' '*/qwDbg.c' '*/tencode.h' \ '*/shellAuto.c' '*/shellTire.c' '*/shellCommand.c'\ - --rc branch_coverage=1 -o coverage.info + '*/sql.c' '*/sql.y'\ + --branch-coverage --function-coverage -o coverage.info # generate result echo "generate result" - lcov -l --rc branch_coverage=1 coverage.info | tee -a $TDENGINE_COVERAGE_REPORT + lcov -l --branch-coverage --function-coverage coverage.info | tee -a $TDENGINE_COVERAGE_REPORT sed -i 's/\/root\/TDengine\/sql.c/\/root\/TDengine\/source\/libs\/parser\/inc\/sql.c/g' coverage.info sed -i 's/\/root\/TDengine\/sql.y/\/root\/TDengine\/source\/libs\/parser\/inc\/sql.y/g' coverage.info # push result to coveralls.io echo "push result to coveralls.io" - /usr/local/bin/coveralls-lcov coverage.info -t o7uY02qEAgKyJHrkxLGiCOTfL3IGQR2zm | tee -a $TDENGINE_COVERAGE_REPORT + /usr/local/bin/coveralls-lcov coverage.info -b 3.0 -t o7uY02qEAgKyJHrkxLGiCOTfL3IGQR2zm | tee -a $TDENGINE_COVERAGE_REPORT #/root/pxiao/checkCoverageFile.sh -s $TDENGINE_DIR/source -f $TDENGINE_COVERAGE_REPORT #cat /root/pxiao/fileListNoCoverage.log | tee -a $TDENGINE_COVERAGE_REPORT cat $TDENGINE_COVERAGE_REPORT | grep "| 0.0%" | awk -F "%" '{print $1}' | awk -F "|" '{if($2==0.0)print $1}' | tee -a $TDENGINE_COVERAGE_REPORT - } function sendReport { From d4900919798b5ea4f9f5bff59e58a479cd020dca Mon Sep 17 00:00:00 2001 From: Alex Duan <417921451@qq.com> Date: Thu, 11 Jan 2024 18:10:29 +0800 Subject: [PATCH 072/102] fix: tweak timeout from 30m to 20m --- tests/script/coverage_test.sh | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/tests/script/coverage_test.sh b/tests/script/coverage_test.sh index 136530e451..2a93fe69ef 100644 --- a/tests/script/coverage_test.sh +++ b/tests/script/coverage_test.sh @@ -73,8 +73,7 @@ function buildTDengine() { echo "$makecmd" $makecmd - make -j 8 - make install + make -j 8 install } function runCasesOneByOne () { @@ -84,7 +83,7 @@ function runCasesOneByOne () { if [[ "$2" == "sim" ]] && [[ $line == *"script"* ]]; then case=`echo $cmd | cut -d' ' -f 3` start_time=`date +%s` - date +%F\ %T | tee -a $TDENGINE_COVERAGE_REPORT && timeout 30m $cmd > /dev/null 2>&1 && \ + date +%F\ %T | tee -a $TDENGINE_COVERAGE_REPORT && timeout 20m $cmd > /dev/null 2>&1 && \ echo -e "${GREEN}$case success${NC}" | tee -a $TDENGINE_COVERAGE_REPORT \ || echo -e "${RED}$case failed${NC}" | tee -a $TDENGINE_COVERAGE_REPORT end_time=`date +%s` @@ -95,7 +94,7 @@ function runCasesOneByOne () { fi case=`echo $cmd | cut -d' ' -f 4-20` start_time=`date +%s` - date +%F\ %T | tee -a $TDENGINE_COVERAGE_REPORT && timeout 30m $cmd > /dev/null 2>&1 && \ + date +%F\ %T | tee -a $TDENGINE_COVERAGE_REPORT && timeout 20m $cmd > /dev/null 2>&1 && \ echo -e "${GREEN}$case success${NC}" | tee -a $TDENGINE_COVERAGE_REPORT || \ echo -e "${RED}$case failed${NC}" | tee -a $TDENGINE_COVERAGE_REPORT end_time=`date +%s` @@ -207,7 +206,7 @@ function lcovFunc { lcov --capture -d $TDENGINE_DIR -b $TDENGINE_DIR -o coverage.info --ignore-errors negative,mismatch,inconsistent,source --branch-coverage --function-coverage --no-external # remove exclude paths - lcov --remove coverage1.info \ + lcov --remove coverage.info \ '*/contrib/*' '*/tests/*' '*/test/*' '*/packaging/*' '*/taos-tools/*' '*/taosadapter/*' '*/TSZ/*' \ '*/AccessBridgeCalls.c' '*/ttszip.c' '*/dataInserter.c' '*/tlinearhash.c' '*/tsimplehash.c' '*/tsdbDiskData.c'\ '*/texpr.c' '*/runUdf.c' '*/schDbg.c' '*/syncIO.c' '*/tdbOs.c' '*/pushServer.c' '*/osLz4.c'\ From 5e4d8ed26358ab8112a6e00e14c0b054d073bc76 Mon Sep 17 00:00:00 2001 From: Alex Duan <417921451@qq.com> Date: Thu, 11 Jan 2024 18:38:16 +0800 Subject: [PATCH 073/102] fix: change 3.0 to dynamic variant name --- tests/script/coverage_test.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/script/coverage_test.sh b/tests/script/coverage_test.sh index 2a93fe69ef..c5e0c31f83 100644 --- a/tests/script/coverage_test.sh +++ b/tests/script/coverage_test.sh @@ -226,7 +226,7 @@ function lcovFunc { # push result to coveralls.io echo "push result to coveralls.io" - /usr/local/bin/coveralls-lcov coverage.info -b 3.0 -t o7uY02qEAgKyJHrkxLGiCOTfL3IGQR2zm | tee -a $TDENGINE_COVERAGE_REPORT + /usr/local/bin/coveralls-lcov coverage.info -b $branch -t o7uY02qEAgKyJHrkxLGiCOTfL3IGQR2zm | tee -a $TDENGINE_COVERAGE_REPORT #/root/pxiao/checkCoverageFile.sh -s $TDENGINE_DIR/source -f $TDENGINE_COVERAGE_REPORT #cat /root/pxiao/fileListNoCoverage.log | tee -a $TDENGINE_COVERAGE_REPORT From d3659db2fff9a97f0535c9b081dc1e760a294cb7 Mon Sep 17 00:00:00 2001 From: wangmm0220 Date: Thu, 11 Jan 2024 19:22:54 +0800 Subject: [PATCH 074/102] fix:[TD-28202]move tq timer to write thread in mnode --- source/client/src/clientTmq.c | 4 ++-- source/dnode/mnode/impl/src/mndMain.c | 2 +- source/dnode/mnode/impl/src/mndSubscribe.c | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/source/client/src/clientTmq.c b/source/client/src/clientTmq.c index 15c8903978..28e269ee10 100644 --- a/source/client/src/clientTmq.c +++ b/source/client/src/clientTmq.c @@ -1968,7 +1968,7 @@ TAOS_RES* tmq_consumer_poll(tmq_t* tmq, int64_t timeout) { void* rspObj = NULL; int64_t startTime = taosGetTimestampMs(); - tscInfo("consumer:0x%" PRIx64 " start to poll at %" PRId64 ", timeout:%" PRId64, tmq->consumerId, startTime, + tscDebug("consumer:0x%" PRIx64 " start to poll at %" PRId64 ", timeout:%" PRId64, tmq->consumerId, startTime, timeout); // in no topic status, delayed task also need to be processed @@ -2015,7 +2015,7 @@ TAOS_RES* tmq_consumer_poll(tmq_t* tmq, int64_t timeout) { int64_t currentTime = taosGetTimestampMs(); int64_t elapsedTime = currentTime - startTime; if (elapsedTime > timeout) { - tscInfo("consumer:0x%" PRIx64 " (epoch %d) timeout, no rsp, start time %" PRId64 ", current time %" PRId64, + tscDebug("consumer:0x%" PRIx64 " (epoch %d) timeout, no rsp, start time %" PRId64 ", current time %" PRId64, tmq->consumerId, tmq->epoch, startTime, currentTime); return NULL; } diff --git a/source/dnode/mnode/impl/src/mndMain.c b/source/dnode/mnode/impl/src/mndMain.c index c0fd93c65d..acc65c0650 100644 --- a/source/dnode/mnode/impl/src/mndMain.c +++ b/source/dnode/mnode/impl/src/mndMain.c @@ -145,7 +145,7 @@ static void mndCalMqRebalance(SMnode *pMnode) { void *pReq = mndBuildTimerMsg(&contLen); if (pReq != NULL) { SRpcMsg rpcMsg = {.msgType = TDMT_MND_TMQ_TIMER, .pCont = pReq, .contLen = contLen}; - tmsgPutToQueue(&pMnode->msgCb, READ_QUEUE, &rpcMsg); + tmsgPutToQueue(&pMnode->msgCb, WRITE_QUEUE, &rpcMsg); } } diff --git a/source/dnode/mnode/impl/src/mndSubscribe.c b/source/dnode/mnode/impl/src/mndSubscribe.c index 62b671a12f..0909003201 100644 --- a/source/dnode/mnode/impl/src/mndSubscribe.c +++ b/source/dnode/mnode/impl/src/mndSubscribe.c @@ -747,7 +747,7 @@ static int32_t mndCheckConsumer(SRpcMsg *pMsg, SHashObj* rebSubHash) { int32_t hbStatus = atomic_add_fetch_32(&pConsumer->hbStatus, 1); int32_t status = atomic_load_32(&pConsumer->status); - mInfo("check for consumer:0x%" PRIx64 " status:%d(%s), sub-time:%" PRId64 ", createTime:%" PRId64 ", hbstatus:%d", + mDebug("check for consumer:0x%" PRIx64 " status:%d(%s), sub-time:%" PRId64 ", createTime:%" PRId64 ", hbstatus:%d", pConsumer->consumerId, status, mndConsumerStatusName(status), pConsumer->subscribeTime, pConsumer->createTime, hbStatus); From aa410a6deb5b9d71e30f3dd9dd2c82a884e65595 Mon Sep 17 00:00:00 2001 From: Minglei Jin Date: Fri, 12 Jan 2024 09:12:56 +0800 Subject: [PATCH 075/102] fix(tsdb/cache): clear fs state before next open --- source/dnode/vnode/src/tsdb/tsdbCache.c | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/source/dnode/vnode/src/tsdb/tsdbCache.c b/source/dnode/vnode/src/tsdb/tsdbCache.c index ab504acea7..9b29329cf3 100644 --- a/source/dnode/vnode/src/tsdb/tsdbCache.c +++ b/source/dnode/vnode/src/tsdb/tsdbCache.c @@ -1990,9 +1990,9 @@ static int32_t getNextRowFromFS(void *iter, TSDBROW **ppRow, bool *pIgnoreEarlie if (SFSNEXTROW_FILESET == state->state) { _next_fileset: - if (--state->iFileSet < 0) { - clearLastFileSet(state); + clearLastFileSet(state); + if (--state->iFileSet < 0) { *ppRow = NULL; return code; } else { @@ -2862,7 +2862,9 @@ static int32_t mergeLastCid(tb_uid_t uid, STsdb *pTsdb, SArray **ppLastArray, SC taosArraySet(pColArray, iCol, &lastCol); int32_t aColIndex = taosArraySearchIdx(aColArray, &lastCol.colVal.cid, compareInt16Val, TD_EQ); - taosArrayRemove(aColArray, aColIndex); + if (aColIndex >= 0) { + taosArrayRemove(aColArray, aColIndex); + } } else if (!COL_VAL_IS_VALUE(tColVal) && !COL_VAL_IS_VALUE(pColVal) && !setNoneCol) { noneCol = iCol; setNoneCol = true; From a9fe3ecaa90606efde37d86e0aded1c24c0d9467 Mon Sep 17 00:00:00 2001 From: zk66214 Date: Fri, 12 Jan 2024 10:21:13 +0800 Subject: [PATCH 076/102] fix characters problem --- .../system-test/1-insert/insert_timestamp.py | 42 +++++++------------ 1 file changed, 14 insertions(+), 28 deletions(-) diff --git a/tests/system-test/1-insert/insert_timestamp.py b/tests/system-test/1-insert/insert_timestamp.py index bc3b1c0275..310a27dae0 100644 --- a/tests/system-test/1-insert/insert_timestamp.py +++ b/tests/system-test/1-insert/insert_timestamp.py @@ -13,25 +13,13 @@ class TDTestCase: tdSql.init(conn.cursor(), True) def run(self): - """ - timestamp输入插入规则: - 对于插入的字段类型为timestamp类型的字段,只允许这么几种情况: - timestamp - timestamp +/- interval - interval + timestamp - timestamp可以是字符串譬如:"2023-12-05 00:00:00.000", 也可以是int型, 譬如:1701619200000 - interval支持:b, u, a, s, m, h, d, w 不支持n, y,譬如:1h, 2d - - 仅支持2元表达式,譬如:timestamp + 2h, 不支持2元以上表达,譬如timestamp + 2h + 1d - """ - tdSql.execute("create database test_insert_timestamp PRECISION 'ns';") tdSql.execute("use test_insert_timestamp;") tdSql.execute("create stable st(ts timestamp, c1 int) tags(id int);") tdSql.execute("create table test_t using st tags(1);") expectErrInfo = "syntax error" - # 异常场景:timestamp + timestamp + # abnormal scenario: timestamp + timestamp tdSql.error("insert into test_t values(now + today(), 1 );", expectErrInfo=expectErrInfo, fullMatched=False) tdSql.error("insert into test_t values(now - today(), 1 );", expectErrInfo=expectErrInfo, fullMatched=False) tdSql.error("insert into test_t values(today() + now(), 1 ); ", expectErrInfo=expectErrInfo, fullMatched=False) @@ -40,24 +28,24 @@ class TDTestCase: tdSql.error("insert into test_t values('2023-11-28 00:00:00.000' + 1701111600000, 1 ); ", expectErrInfo=expectErrInfo, fullMatched=False) tdSql.error("insert into test_t values(1701111500000 + 1701111600000, 1 ); ", expectErrInfo=expectErrInfo, fullMatched=False) - # 异常场景:timestamp + interval + interval + # abnormal scenario: timestamp + interval + interval tdSql.error("insert into test_t values(today() + 1d + 1s, 1);", expectErrInfo=expectErrInfo, fullMatched=False) - # 异常场景:interval - timestamp + # abnormal scenario: interval - timestamp tdSql.error("insert into test_t values(2h - now(), 1 ); ", expectErrInfo=expectErrInfo, fullMatched=False) tdSql.error("insert into test_t values(2h - today(), 1 ); ", expectErrInfo=expectErrInfo, fullMatched=False) - # 异常场景:interval + interval + # abnormal scenario: interval + interval tdSql.error("insert into test_t values(2h - 1h, 1 ); ", expectErrInfo=expectErrInfo, fullMatched=False) tdSql.error("insert into test_t values(2h + 1h, 1 ); ", expectErrInfo=expectErrInfo, fullMatched=False) - # 异常场景:非法interval类型n + # abnormal scenario: non-support datatype - n tdSql.error("insert into test_t values(today() + 2n, 7); ", expectErrInfo=expectErrInfo, fullMatched=False) - # 异常场景:非法interval类型y + # abnormal scenario: non-support datatype - y tdSql.error("insert into test_t values(today() - 2y, 8);", expectErrInfo=expectErrInfo, fullMatched=False) - # 异常场景:数据类型不对 + # abnormal scenario: non-support datatype tdSql.error("insert into test_t values('a1701619200000', 8);", expectErrInfo=expectErrInfo, fullMatched=False) tdSql.error("insert into test_t values('ss2023-12-05 00:00:00.000' + '1701619200000', 1);", expectErrInfo=expectErrInfo, fullMatched=False) tdSql.error("insert into test_t values(123456, 1);", expectErrInfo="Timestamp data out of range") @@ -66,31 +54,31 @@ class TDTestCase: tdSql.error("insert into test_t values(None, 1);", expectErrInfo=expectErrInfo, fullMatched=False) tdSql.error("insert into test_t values(null, 1);", expectErrInfo=expectErrInfo, fullMatched=False) - # 异常场景:格式不对 + # abnormal scenario: incorrect format tdSql.error("insert into test_t values('2023-122-05 00:00:00.000' + '1701619200000', 1);", expectErrInfo=expectErrInfo, fullMatched=False) tdSql.error("insert into test_t values('2023-12--05 00:00:00.000' + '1701619200000', 1);", expectErrInfo=expectErrInfo, fullMatched=False) tdSql.error("insert into test_t values('12/12/2023' + 10a, 1);", expectErrInfo=expectErrInfo, fullMatched=False) tdSql.error("insert into test_t values(1701619200000111, 1);", expectErrInfo="Timestamp data out of range", fullMatched=False) - # 正常场景:timestamp + interval + # normal scenario:timestamp + interval tdSql.execute("insert into test_t values(today() + 2b, 1);") tdSql.execute("insert into test_t values(1701619200000000000 + 2u, 2);") tdSql.execute("insert into test_t values(today + 2a, 3);") tdSql.execute("insert into test_t values('2023-12-05 23:59:59.999' + 2a, 4);") tdSql.execute("insert into test_t values(1701921599000000000 + 3a, 5);") - # 正常场景:timestamp - interval + # normal scenario:timestamp - interval tdSql.execute("insert into test_t values(today() - 2s, 6);") tdSql.execute("insert into test_t values(now() - 2m, 7);") tdSql.execute("insert into test_t values(today - 2h, 8);") tdSql.execute("insert into test_t values('2023-12-05 00:00:00.000000000' - 2a, 9);") tdSql.execute("insert into test_t values(1701669000000000000 - 2a, 10);") - # 正常场景:interval + timestamp + # normal scenario:interval + timestamp tdSql.execute("insert into test_t values(2d + now, 11);") tdSql.execute("insert into test_t values(2w + today, 12);") - # 正常场景:timestamp + # normal scenario:timestamp tdSql.execute("insert into test_t values('2023-12-05 00:00:00.000', 13);") tdSql.execute("insert into test_t values(1701629100000000000, 14);") tdSql.execute("insert into test_t values(now() + 2s, 15);") @@ -102,7 +90,7 @@ class TDTestCase: tdSql.execute("insert into test_t values(1701619200000000000, -5);") tdSql.execute("insert into test_t values('2023-12-05 12:12:12' + 10a, 19);") - # 验证数据 + # data verification tdSql.query(f'select ts,c1 from test_t order by c1;') tdSql.checkRows(22) tdSql.checkEqual(tdSql.queryResult[0][0], 1699977600000000000) # c1=-15 @@ -133,12 +121,10 @@ class TDTestCase: tdSql.execute("drop database if exists test_insert_timestamp;") def __convert_ts_to_date(self, ts: int) -> str: - # 创建datetime对象并进行转换 dt_object = datetime.datetime.fromtimestamp(ts / 1e9) - # 格式化日期字符串 formatted_date = dt_object.strftime('%Y-%m-%d') - # print("转换后的日期为:", formatted_date) + return formatted_date def __get_today_ts(self) -> int: From b36cc972367fa9a3520ddf3bc266ec76801bee8c Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Fri, 12 Jan 2024 10:44:49 +0800 Subject: [PATCH 077/102] fix(stream): close the inputQ of the related stream task when the fill-history task recv the trans-state msg from upstream. --- include/libs/stream/tstream.h | 1 + source/dnode/vnode/src/tq/tq.c | 1 - source/libs/executor/src/executor.c | 6 ++-- source/libs/stream/src/stream.c | 30 +++++++++++++++---- source/libs/stream/src/streamDispatch.c | 3 ++ source/libs/stream/src/streamExec.c | 39 +++++++++++++++++-------- source/libs/stream/src/streamQueue.c | 2 -- source/libs/stream/src/streamStart.c | 14 +++++++-- 8 files changed, 69 insertions(+), 27 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 9ea655c15c..a115783b70 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -562,6 +562,7 @@ struct SStreamDispatchReq { int32_t upstreamTaskId; int32_t upstreamChildId; int32_t upstreamNodeId; + int32_t upstreamRelTaskId; int32_t blockNum; int64_t totalLen; SArray* dataLen; // SArray diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 10420ec463..a53b322753 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -963,7 +963,6 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { // let's decide which step should be executed now if (pTask->execInfo.step1Start == 0) { 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); } else { diff --git a/source/libs/executor/src/executor.c b/source/libs/executor/src/executor.c index 6b38457d70..d27daad550 100644 --- a/source/libs/executor/src/executor.c +++ b/source/libs/executor/src/executor.c @@ -922,8 +922,8 @@ int32_t qStreamSourceScanParamForHistoryScanStep2(qTaskInfo_t tinfo, SVersionRan pStreamInfo->fillHistoryWindow = *pWindow; pStreamInfo->recoverStep = STREAM_RECOVER_STEP__PREPARE2; - qDebug("%s step 2. set param for stream scanner for scan-history data, verRange:%" PRId64 " - %" PRId64 - ", window:%" PRId64 " - %" PRId64, + qDebug("%s step 2. set param for stream scanner scan wal, verRange:%" PRId64 " - %" PRId64 ", window:%" PRId64 + " - %" PRId64, GET_TASKID(pTaskInfo), pStreamInfo->fillHistoryVer.minVer, pStreamInfo->fillHistoryVer.maxVer, pWindow->skey, pWindow->ekey); return 0; @@ -1129,7 +1129,7 @@ int32_t qStreamInfoResetTimewindowFilter(qTaskInfo_t tinfo) { SExecTaskInfo* pTaskInfo = (SExecTaskInfo*)tinfo; STimeWindow* pWindow = &pTaskInfo->streamInfo.fillHistoryWindow; - qDebug("%s remove scan-history filter window:%" PRId64 "-%" PRId64 ", set new window:%" PRId64 "-%" PRId64, + qDebug("%s remove timeWindow filter:%" PRId64 "-%" PRId64 ", set new window:%" PRId64 "-%" PRId64, GET_TASKID(pTaskInfo), pWindow->skey, pWindow->ekey, INT64_MIN, INT64_MAX); pWindow->skey = INT64_MIN; diff --git a/source/libs/stream/src/stream.c b/source/libs/stream/src/stream.c index 7db4bec6a0..7d507a1394 100644 --- a/source/libs/stream/src/stream.c +++ b/source/libs/stream/src/stream.c @@ -214,8 +214,9 @@ int32_t streamTaskEnqueueRetrieve(SStreamTask* pTask, SStreamRetrieveReq* pReq, } int32_t streamProcessDispatchMsg(SStreamTask* pTask, SStreamDispatchReq* pReq, SRpcMsg* pRsp) { - int32_t status = 0; - const char* id = pTask->id.idStr; + int32_t status = 0; + SStreamMeta* pMeta = pTask->pMeta; + 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); @@ -223,7 +224,7 @@ int32_t streamProcessDispatchMsg(SStreamTask* pTask, SStreamDispatchReq* pReq, S SStreamChildEpInfo* pInfo = streamTaskGetUpstreamTaskEpInfo(pTask, pReq->upstreamTaskId); ASSERT(pInfo != NULL); - if (pTask->pMeta->role == NODE_ROLE_FOLLOWER) { + if (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 { @@ -244,6 +245,22 @@ int32_t streamProcessDispatchMsg(SStreamTask* pTask, SStreamDispatchReq* pReq, S 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); + } else if (pReq->type == STREAM_INPUT__TRANS_STATE) { + atomic_add_fetch_32(&pTask->upstreamInfo.numOfClosed, 1); + streamTaskCloseUpstreamInput(pTask, pReq->upstreamTaskId); + + // disable the related stream task here to avoid it to receive the newly arrived data after the transfer-state + STaskId* pRelTaskId = &pTask->streamTaskId; + SStreamTask* pStreamTask = streamMetaAcquireTask(pMeta, pRelTaskId->streamId, pRelTaskId->taskId); + if (pStreamTask != NULL) { + atomic_add_fetch_32(&pStreamTask->upstreamInfo.numOfClosed, 1); + streamTaskCloseUpstreamInput(pStreamTask, pReq->upstreamRelTaskId); + streamMetaReleaseTask(pMeta, pStreamTask); + } + + stDebug("s-task:%s close inputQ for upstream:0x%x since trans-state msgId:%d recv, rel stream-task:0x%" PRIx64 + " close inputQ for upstream:0x%x", + id, pReq->upstreamTaskId, pReq->msgId, pTask->streamTaskId.taskId, pReq->upstreamRelTaskId); } status = streamTaskAppendInputBlocks(pTask, pReq); @@ -252,9 +269,9 @@ int32_t streamProcessDispatchMsg(SStreamTask* pTask, SStreamDispatchReq* pReq, S } // disable the data from upstream tasks - if (streamTaskGetStatus(pTask)->state == TASK_STATUS__HALT) { - status = TASK_INPUT_STATUS__BLOCKED; - } +// if (streamTaskGetStatus(pTask)->state == TASK_STATUS__HALT) { +// status = TASK_INPUT_STATUS__BLOCKED; +// } { // do send response with the input status @@ -295,6 +312,7 @@ void streamTaskOpenAllUpstreamInput(SStreamTask* pTask) { } pTask->upstreamInfo.numOfClosed = 0; + stDebug("s-task:%s opening up inputQ from upstream tasks", pTask->id.idStr); } void streamTaskCloseUpstreamInput(SStreamTask* pTask, int32_t taskId) { diff --git a/source/libs/stream/src/streamDispatch.c b/source/libs/stream/src/streamDispatch.c index a6d21e2625..e2c8704b1c 100644 --- a/source/libs/stream/src/streamDispatch.c +++ b/source/libs/stream/src/streamDispatch.c @@ -58,6 +58,7 @@ int32_t tEncodeStreamDispatchReq(SEncoder* pEncoder, const SStreamDispatchReq* p if (tEncodeI32(pEncoder, pReq->upstreamTaskId) < 0) return -1; if (tEncodeI32(pEncoder, pReq->upstreamChildId) < 0) return -1; if (tEncodeI32(pEncoder, pReq->upstreamNodeId) < 0) return -1; + if (tEncodeI32(pEncoder, pReq->upstreamRelTaskId) < 0) return -1; if (tEncodeI32(pEncoder, pReq->blockNum) < 0) return -1; if (tEncodeI64(pEncoder, pReq->totalLen) < 0) return -1; ASSERT(taosArrayGetSize(pReq->data) == pReq->blockNum); @@ -84,6 +85,7 @@ int32_t tDecodeStreamDispatchReq(SDecoder* pDecoder, SStreamDispatchReq* pReq) { if (tDecodeI32(pDecoder, &pReq->upstreamTaskId) < 0) return -1; if (tDecodeI32(pDecoder, &pReq->upstreamChildId) < 0) return -1; if (tDecodeI32(pDecoder, &pReq->upstreamNodeId) < 0) return -1; + if (tDecodeI32(pDecoder, &pReq->upstreamRelTaskId) < 0) return -1; if (tDecodeI32(pDecoder, &pReq->blockNum) < 0) return -1; if (tDecodeI64(pDecoder, &pReq->totalLen) < 0) return -1; @@ -114,6 +116,7 @@ static int32_t tInitStreamDispatchReq(SStreamDispatchReq* pReq, const SStreamTas pReq->upstreamTaskId = pTask->id.taskId; pReq->upstreamChildId = pTask->info.selfChildId; pReq->upstreamNodeId = pTask->info.nodeId; + pReq->upstreamRelTaskId = pTask->streamTaskId.taskId; pReq->blockNum = numOfBlocks; pReq->taskId = dstTaskId; pReq->type = type; diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index 18f7ed061a..c69fe2a79c 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -371,7 +371,6 @@ int32_t streamDoTransferStateToStreamTask(SStreamTask* pTask) { // In case of sink tasks, no need to halt them. // In case of source tasks and agg tasks, we should HALT them, and wait for them to be idle. And then, it's safe to // start the task state transfer procedure. -// char* p = NULL; SStreamTaskState* pState = streamTaskGetStatus(pStreamTask); status = pState->state; char* p = pState->name; @@ -392,8 +391,12 @@ int32_t streamDoTransferStateToStreamTask(SStreamTask* pTask) { } // 1. expand the query time window for stream task of WAL scanner - pTimeWindow->skey = INT64_MIN; - qStreamInfoResetTimewindowFilter(pStreamTask->exec.pExecutor); + if (pStreamTask->info.taskLevel == TASK_LEVEL__SOURCE) { + pTimeWindow->skey = INT64_MIN; + qStreamInfoResetTimewindowFilter(pStreamTask->exec.pExecutor); + } else { + stDebug("s-task:%s non-source task no need to reset filter window", pStreamTask->id.idStr); + } // 2. transfer the ownership of executor state streamTaskReleaseState(pTask); @@ -407,10 +410,13 @@ int32_t streamDoTransferStateToStreamTask(SStreamTask* pTask) { // 4. free it and remove fill-history task from disk meta-store streamBuildAndSendDropTaskMsg(pTask->pMsgCb, pMeta->vgId, &pTask->id); - // 5. save to disk + // 5. assign the status to the value that will be kept in disk pStreamTask->status.taskStatus = streamTaskGetStatus(pStreamTask)->state; - // 6. add empty delete block + // 6. open the inputQ for all upstream tasks + streamTaskOpenAllUpstreamInput(pStreamTask); + + // 7. add empty delete block if ((pStreamTask->info.taskLevel == TASK_LEVEL__SOURCE) && taosQueueEmpty(pStreamTask->inputq.queue->pQueue)) { SStreamRefDataBlock* pItem = taosAllocateQitem(sizeof(SStreamRefDataBlock), DEF_QITEM, 0); @@ -430,6 +436,8 @@ int32_t streamDoTransferStateToStreamTask(SStreamTask* pTask) { int32_t streamTransferStateToStreamTask(SStreamTask* pTask) { int32_t code = TSDB_CODE_SUCCESS; + SStreamMeta* pMeta = pTask->pMeta; + ASSERT(pTask->status.appendTranstateBlock == 1); int32_t level = pTask->info.taskLevel; @@ -439,8 +447,14 @@ 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->pMsgCb, pTask->pMeta->vgId, &pTask->id); + } else { // drop fill-history task and open inputQ of sink task + SStreamTask* pStreamTask = streamMetaAcquireTask(pMeta, pTask->streamTaskId.streamId, pTask->streamTaskId.taskId); + if (pStreamTask != NULL) { + streamTaskOpenAllUpstreamInput(pStreamTask); + streamMetaReleaseTask(pMeta, pStreamTask); + } + + streamBuildAndSendDropTaskMsg(pTask->pMsgCb, pMeta->vgId, &pTask->id); } return code; @@ -496,16 +510,17 @@ static void doSetStreamInputBlock(SStreamTask* pTask, const void* pInput, int64_ } } -int32_t streamProcessTranstateBlock(SStreamTask* pTask, SStreamDataBlock* pBlock) { +int32_t streamProcessTransstateBlock(SStreamTask* pTask, SStreamDataBlock* pBlock) { const char* id = pTask->id.idStr; int32_t code = TSDB_CODE_SUCCESS; + int32_t level = pTask->info.taskLevel; - int32_t level = pTask->info.taskLevel; if (level == TASK_LEVEL__AGG || level == TASK_LEVEL__SINK) { int32_t remain = streamAlignTransferState(pTask); + if (remain > 0) { streamFreeQitem((SStreamQueueItem*)pBlock); - stDebug("s-task:%s receive upstream transfer state msg, remain:%d", id, remain); + stDebug("s-task:%s receive upstream trans-state msg, not sent remain:%d", id, remain); return 0; } } @@ -536,7 +551,7 @@ int32_t streamProcessTranstateBlock(SStreamTask* pTask, SStreamDataBlock* pBlock } } else { // non-dispatch task, do task state transfer directly streamFreeQitem((SStreamQueueItem*)pBlock); - stDebug("s-task:%s non-dispatch task, level:%d start to transfer state directly", id, pTask->info.taskLevel); + stDebug("s-task:%s non-dispatch task, level:%d start to transfer state directly", id, level); ASSERT(pTask->info.fillHistory == 1); code = streamTransferStateToStreamTask(pTask); @@ -604,7 +619,7 @@ int32_t doStreamExecTask(SStreamTask* pTask) { } if (type == STREAM_INPUT__TRANS_STATE) { - streamProcessTranstateBlock(pTask, (SStreamDataBlock*)pInput); + streamProcessTransstateBlock(pTask, (SStreamDataBlock*)pInput); continue; } diff --git a/source/libs/stream/src/streamQueue.c b/source/libs/stream/src/streamQueue.c index 8b5b2da061..90823cd937 100644 --- a/source/libs/stream/src/streamQueue.c +++ b/source/libs/stream/src/streamQueue.c @@ -147,8 +147,6 @@ const char* streamQueueItemGetTypeStr(int32_t type) { 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; diff --git a/source/libs/stream/src/streamStart.c b/source/libs/stream/src/streamStart.c index d39528312d..24909a9776 100644 --- a/source/libs/stream/src/streamStart.c +++ b/source/libs/stream/src/streamStart.c @@ -1029,6 +1029,11 @@ void streamTaskSetRangeStreamCalc(SStreamTask* pTask) { pTask->id.idStr, pRange->window.skey, pRange->window.ekey, pRange->range.minVer, pRange->range.maxVer); } } else { + ASSERT(pTask->info.fillHistory == 0); + if (pTask->info.taskLevel >= TASK_LEVEL__AGG) { + return; + } + int64_t ekey = 0; if (pRange->window.ekey < INT64_MAX) { ekey = pRange->window.ekey + 1; @@ -1043,10 +1048,13 @@ void streamTaskSetRangeStreamCalc(SStreamTask* pTask) { pRange->range.minVer = 0; pRange->range.maxVer = ver; - stDebug("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, set stream task timeWindow:%" PRId64 " - %" PRId64 ", verRang:%" PRId64 " - %" PRId64, - pTask->id.idStr, pTask->info.taskLevel, pRange->window.skey, pRange->window.ekey, pRange->range.minVer, - pRange->range.maxVer); + pTask->id.idStr, pTask->info.taskLevel, pRange->window.skey, pRange->window.ekey, ver, INT64_MAX); + + SVersionRange verRange = {.minVer = ver, .maxVer = INT64_MAX}; + STimeWindow win = pRange->window; + streamSetParamForStreamScannerStep2(pTask, &verRange, &win); } } From 0162fb9e76a879a48dadc71ed1b3be94ceb2ec85 Mon Sep 17 00:00:00 2001 From: slzhou Date: Fri, 12 Jan 2024 10:46:03 +0800 Subject: [PATCH 078/102] set the timelineresmode to subquery timeline resmode --- source/libs/parser/src/parTranslater.c | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/source/libs/parser/src/parTranslater.c b/source/libs/parser/src/parTranslater.c index ea1a5f7d1d..e407cc1e81 100644 --- a/source/libs/parser/src/parTranslater.c +++ b/source/libs/parser/src/parTranslater.c @@ -3158,7 +3158,11 @@ int32_t translateTable(STranslateContext* pCxt, SNode** pTable) { ((SSelectStmt*)pTempTable->pSubquery)->isEmptyResult && isSelectStmt(pCxt->pCurrStmt)) { ((SSelectStmt*)pCxt->pCurrStmt)->isEmptyResult = true; } - + if (QUERY_NODE_SELECT_STMT == nodeType(pTempTable->pSubquery) && isSelectStmt(pCxt->pCurrStmt)) { + SSelectStmt* pSubStmt = (SSelectStmt*)pTempTable->pSubquery; + SSelectStmt* pCurrSmt = (SSelectStmt*)(pCxt->pCurrStmt); + pCurrSmt->timeLineResMode = pSubStmt->timeLineResMode; + } pTempTable->table.precision = getStmtPrecision(pTempTable->pSubquery); pTempTable->table.singleTable = stmtIsSingleTable(pTempTable->pSubquery); code = addNamespace(pCxt, pTempTable); From cf941f74884288b783139e5dc14e8ef2f4a5d391 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Fri, 12 Jan 2024 11:12:23 +0800 Subject: [PATCH 079/102] fix(stream): check status when resume to run. --- source/dnode/vnode/src/tqCommon/tqCommon.c | 20 ++++++++++++++------ 1 file changed, 14 insertions(+), 6 deletions(-) diff --git a/source/dnode/vnode/src/tqCommon/tqCommon.c b/source/dnode/vnode/src/tqCommon/tqCommon.c index da1c2ecf16..7883c858f0 100644 --- a/source/dnode/vnode/src/tqCommon/tqCommon.c +++ b/source/dnode/vnode/src/tqCommon/tqCommon.c @@ -820,15 +820,23 @@ int32_t tqStreamTaskProcessRunReq(SStreamMeta* pMeta, SRpcMsg* pMsg, bool isLead return 0; } else if (type == STREAM_EXEC_T_RESUME_TASK) { // task resume to run after idle for a while SStreamTask* pTask = streamMetaAcquireTask(pMeta, pReq->streamId, pReq->taskId); - if (pTask != NULL) { - ASSERT(streamTaskReadyToRun(pTask, NULL)); - int64_t execTs = pTask->status.lastExecTs; - int32_t idle = taosGetTimestampMs() - execTs; - tqDebug("s-task:%s task resume to run after idle for:%dms from:%" PRId64, pTask->id.idStr, idle, execTs); - streamResumeTask(pTask); + if (pTask != NULL) { + char* pStatus = NULL; + if (streamTaskReadyToRun(pTask, &pStatus)) { + int64_t execTs = pTask->status.lastExecTs; + int32_t idle = taosGetTimestampMs() - execTs; + tqDebug("s-task:%s task resume to run after idle for:%dms from:%" PRId64, pTask->id.idStr, idle, execTs); + + streamResumeTask(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, + pTask->id.idStr, pStatus, status); + } streamMetaReleaseTask(pMeta, pTask); } + return 0; } From 5fa8f85c425e03a2f86eb80f76b1b21fce50ad78 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Fri, 12 Jan 2024 11:28:41 +0800 Subject: [PATCH 080/102] fix(stream): remove invalid release. --- source/libs/stream/src/streamMeta.c | 2 -- 1 file changed, 2 deletions(-) diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 04d34b0945..68348f462f 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -1520,8 +1520,6 @@ int32_t streamMetaStartOneTask(SStreamMeta* pMeta, int64_t streamId, int32_t tas if (pTask == NULL) { stError("vgId:%d failed to acquire task:0x%x during start tasks", pMeta->vgId, taskId); streamMetaUpdateTaskDownstreamStatus(pMeta, streamId, taskId, 0, taosGetTimestampMs(), false); - - streamMetaReleaseTask(pMeta, pTask); return TSDB_CODE_STREAM_TASK_IVLD_STATUS; } From 52d8811bdca4a7622c1ee74417f9ffd7df5a2aec Mon Sep 17 00:00:00 2001 From: 54liuyao <54liuyao> Date: Fri, 12 Jan 2024 13:49:32 +0800 Subject: [PATCH 081/102] code coverage --- include/libs/function/functionMgt.h | 4 ++ .../executor/src/streamtimewindowoperator.c | 6 +++ source/libs/executor/src/timewindowoperator.c | 10 ---- source/libs/function/inc/builtins.h | 2 + source/libs/function/inc/builtinsimpl.h | 16 +++++- source/libs/function/src/builtins.c | 50 +++++++++++++++++++ source/libs/function/src/builtinsimpl.c | 11 ++-- .../libs/function/src/detail/tavgfunction.c | 2 + source/libs/function/src/functionMgt.c | 2 + 9 files changed, 87 insertions(+), 16 deletions(-) diff --git a/include/libs/function/functionMgt.h b/include/libs/function/functionMgt.h index 865f1b2295..a0b5d938e3 100644 --- a/include/libs/function/functionMgt.h +++ b/include/libs/function/functionMgt.h @@ -259,9 +259,13 @@ EFuncDataRequired fmFuncDynDataRequired(int32_t funcId, void* pRes, STimeWindow* int32_t fmGetFuncExecFuncs(int32_t funcId, SFuncExecFuncs* pFpSet); int32_t fmGetScalarFuncExecFuncs(int32_t funcId, SScalarFuncExecFuncs* pFpSet); int32_t fmGetUdafExecFuncs(int32_t funcId, SFuncExecFuncs* pFpSet); + +#ifdef BUILD_NO_CALL int32_t fmSetInvertFunc(int32_t funcId, SFuncExecFuncs* pFpSet); int32_t fmSetNormalFunc(int32_t funcId, SFuncExecFuncs* pFpSet); bool fmIsInvertible(int32_t funcId); +#endif + char* fmGetFuncName(int32_t funcId); #ifdef __cplusplus diff --git a/source/libs/executor/src/streamtimewindowoperator.c b/source/libs/executor/src/streamtimewindowoperator.c index b2d0f25466..22339aa384 100644 --- a/source/libs/executor/src/streamtimewindowoperator.c +++ b/source/libs/executor/src/streamtimewindowoperator.c @@ -427,6 +427,7 @@ void destroyStreamFinalIntervalOperatorInfo(void* param) { taosMemoryFreeClear(param); } +#ifdef BUILD_NO_CALL static bool allInvertible(SqlFunctionCtx* pFCtx, int32_t numOfCols) { for (int32_t i = 0; i < numOfCols; i++) { if (fmIsUserDefinedFunc(pFCtx[i].functionId) || !fmIsInvertible(pFCtx[i].functionId)) { @@ -435,6 +436,7 @@ static bool allInvertible(SqlFunctionCtx* pFCtx, int32_t numOfCols) { } return true; } +#endif void reloadFromDownStream(SOperatorInfo* downstream, SStreamIntervalOperatorInfo* pInfo) { SStateStore* pAPI = &downstream->pTaskInfo->storageAPI.stateStore; @@ -3846,6 +3848,7 @@ _error: return NULL; } +#ifdef BUILD_NO_CALL static void setInverFunction(SqlFunctionCtx* pCtx, int32_t num, EStreamType type) { for (int i = 0; i < num; i++) { if (type == STREAM_INVERT) { @@ -3855,6 +3858,7 @@ static void setInverFunction(SqlFunctionCtx* pCtx, int32_t num, EStreamType type } } } +#endif static SSDataBlock* doStreamIntervalAgg(SOperatorInfo* pOperator) { SStreamIntervalOperatorInfo* pInfo = pOperator->info; @@ -3947,9 +3951,11 @@ static SSDataBlock* doStreamIntervalAgg(SOperatorInfo* pOperator) { // caller. Note that all the time window are not close till now. // the pDataBlock are always the same one, no need to call this again setInputDataBlock(pSup, pBlock, TSDB_ORDER_ASC, MAIN_SCAN, true); +#ifdef BUILD_NO_CALL if (pInfo->invertible) { setInverFunction(pSup->pCtx, pOperator->exprSupp.numOfExprs, pBlock->info.type); } +#endif doStreamIntervalAggImpl(pOperator, pBlock, pBlock->info.id.groupId, pInfo->pUpdatedMap); pInfo->twAggSup.maxTs = TMAX(pInfo->twAggSup.maxTs, pBlock->info.window.ekey); diff --git a/source/libs/executor/src/timewindowoperator.c b/source/libs/executor/src/timewindowoperator.c index c459e4d8b9..b86253a8d1 100644 --- a/source/libs/executor/src/timewindowoperator.c +++ b/source/libs/executor/src/timewindowoperator.c @@ -1079,16 +1079,6 @@ static SSDataBlock* doBuildIntervalResult(SOperatorInfo* pOperator) { return (rows == 0) ? NULL : pBlock; } -static void setInverFunction(SqlFunctionCtx* pCtx, int32_t num, EStreamType type) { - for (int i = 0; i < num; i++) { - if (type == STREAM_INVERT) { - fmSetInvertFunc(pCtx[i].functionId, &(pCtx[i].fpSet)); - } else if (type == STREAM_NORMAL) { - fmSetNormalFunc(pCtx[i].functionId, &(pCtx[i].fpSet)); - } - } -} - static void doClearWindowImpl(SResultRowPosition* p1, SDiskbasedBuf* pResultBuf, SExprSupp* pSup, int32_t numOfOutput) { SResultRow* pResult = getResultRowByPos(pResultBuf, p1, false); if (NULL == pResult) { diff --git a/source/libs/function/inc/builtins.h b/source/libs/function/inc/builtins.h index e7fcc38818..6181a9b929 100644 --- a/source/libs/function/inc/builtins.h +++ b/source/libs/function/inc/builtins.h @@ -40,7 +40,9 @@ typedef struct SBuiltinFuncDefinition { FExecProcess processFunc; FScalarExecProcess sprocessFunc; FExecFinalize finalizeFunc; +#ifdef BUILD_NO_CALL FExecProcess invertFunc; +#endif FExecCombine combineFunc; const char* pPartialFunc; const char* pMergeFunc; diff --git a/source/libs/function/inc/builtinsimpl.h b/source/libs/function/inc/builtinsimpl.h index d2f19ed2eb..ba7bf72aea 100644 --- a/source/libs/function/inc/builtinsimpl.h +++ b/source/libs/function/inc/builtinsimpl.h @@ -59,12 +59,19 @@ int32_t combineFunction(SqlFunctionCtx* pDestCtx, SqlFunctionCtx* pSourceCtx); EFuncDataRequired countDataRequired(SFunctionNode* pFunc, STimeWindow* pTimeWindow); bool getCountFuncEnv(struct SFunctionNode* pFunc, SFuncExecEnv* pEnv); int32_t countFunction(SqlFunctionCtx* pCtx); + +#ifdef BUILD_NO_CALL int32_t countInvertFunction(SqlFunctionCtx* pCtx); +#endif EFuncDataRequired statisDataRequired(SFunctionNode* pFunc, STimeWindow* pTimeWindow); bool getSumFuncEnv(struct SFunctionNode* pFunc, SFuncExecEnv* pEnv); int32_t sumFunction(SqlFunctionCtx* pCtx); + +#ifdef BUILD_NO_CALL int32_t sumInvertFunction(SqlFunctionCtx* pCtx); +#endif + int32_t sumCombine(SqlFunctionCtx* pDestCtx, SqlFunctionCtx* pSourceCtx); bool minmaxFunctionSetup(SqlFunctionCtx* pCtx, SResultRowEntryInfo* pResultInfo); @@ -81,7 +88,11 @@ int32_t avgFunction(SqlFunctionCtx* pCtx); int32_t avgFunctionMerge(SqlFunctionCtx* pCtx); int32_t avgFinalize(SqlFunctionCtx* pCtx, SSDataBlock* pBlock); int32_t avgPartialFinalize(SqlFunctionCtx* pCtx, SSDataBlock* pBlock); + +#ifdef BUILD_NO_CALL int32_t avgInvertFunction(SqlFunctionCtx* pCtx); +#endif + int32_t avgCombine(SqlFunctionCtx* pDestCtx, SqlFunctionCtx* pSourceCtx); int32_t getAvgInfoSize(); @@ -91,7 +102,11 @@ int32_t stddevFunction(SqlFunctionCtx* pCtx); int32_t stddevFunctionMerge(SqlFunctionCtx* pCtx); int32_t stddevFinalize(SqlFunctionCtx* pCtx, SSDataBlock* pBlock); int32_t stddevPartialFinalize(SqlFunctionCtx* pCtx, SSDataBlock* pBlock); + +#ifdef BUILD_NO_CALL int32_t stddevInvertFunction(SqlFunctionCtx* pCtx); +#endif + int32_t stddevCombine(SqlFunctionCtx* pDestCtx, SqlFunctionCtx* pSourceCtx); int32_t getStddevInfoSize(); @@ -99,7 +114,6 @@ bool getLeastSQRFuncEnv(struct SFunctionNode* pFunc, SFuncExecEnv* pEnv); bool leastSQRFunctionSetup(SqlFunctionCtx* pCtx, SResultRowEntryInfo* pResultInfo); int32_t leastSQRFunction(SqlFunctionCtx* pCtx); int32_t leastSQRFinalize(SqlFunctionCtx* pCtx, SSDataBlock* pBlock); -int32_t leastSQRInvertFunction(SqlFunctionCtx* pCtx); int32_t leastSQRCombine(SqlFunctionCtx* pDestCtx, SqlFunctionCtx* pSourceCtx); bool getPercentileFuncEnv(struct SFunctionNode* pFunc, SFuncExecEnv* pEnv); diff --git a/source/libs/function/src/builtins.c b/source/libs/function/src/builtins.c index 98fda024fa..61149f593a 100644 --- a/source/libs/function/src/builtins.c +++ b/source/libs/function/src/builtins.c @@ -2368,7 +2368,9 @@ const SBuiltinFuncDefinition funcMgtBuiltins[] = { .processFunc = countFunction, .sprocessFunc = countScalarFunction, .finalizeFunc = functionFinalize, +#ifdef BUILD_NO_CALL .invertFunc = countInvertFunction, +#endif .combineFunc = combineFunction, .pPartialFunc = "count", .pMergeFunc = "sum" @@ -2384,7 +2386,9 @@ const SBuiltinFuncDefinition funcMgtBuiltins[] = { .processFunc = sumFunction, .sprocessFunc = sumScalarFunction, .finalizeFunc = functionFinalize, +#ifdef BUILD_NO_CALL .invertFunc = sumInvertFunction, +#endif .combineFunc = sumCombine, .pPartialFunc = "sum", .pMergeFunc = "sum" @@ -2429,7 +2433,9 @@ const SBuiltinFuncDefinition funcMgtBuiltins[] = { .processFunc = stddevFunction, .sprocessFunc = stddevScalarFunction, .finalizeFunc = stddevFinalize, +#ifdef BUILD_NO_CALL .invertFunc = stddevInvertFunction, +#endif .combineFunc = stddevCombine, .pPartialFunc = "_stddev_partial", .pMergeFunc = "_stddev_merge" @@ -2443,7 +2449,9 @@ const SBuiltinFuncDefinition funcMgtBuiltins[] = { .initFunc = stddevFunctionSetup, .processFunc = stddevFunction, .finalizeFunc = stddevPartialFinalize, +#ifdef BUILD_NO_CALL .invertFunc = stddevInvertFunction, +#endif .combineFunc = stddevCombine, }, { @@ -2455,7 +2463,9 @@ const SBuiltinFuncDefinition funcMgtBuiltins[] = { .initFunc = stddevFunctionSetup, .processFunc = stddevFunctionMerge, .finalizeFunc = stddevFinalize, +#ifdef BUILD_NO_CALL .invertFunc = stddevInvertFunction, +#endif .combineFunc = stddevCombine, }, { @@ -2468,7 +2478,9 @@ const SBuiltinFuncDefinition funcMgtBuiltins[] = { .processFunc = leastSQRFunction, .sprocessFunc = leastSQRScalarFunction, .finalizeFunc = leastSQRFinalize, +#ifdef BUILD_NO_CALL .invertFunc = NULL, +#endif .combineFunc = leastSQRCombine, }, { @@ -2482,7 +2494,9 @@ const SBuiltinFuncDefinition funcMgtBuiltins[] = { .processFunc = avgFunction, .sprocessFunc = avgScalarFunction, .finalizeFunc = avgFinalize, +#ifdef BUILD_NO_CALL .invertFunc = avgInvertFunction, +#endif .combineFunc = avgCombine, .pPartialFunc = "_avg_partial", .pMergeFunc = "_avg_merge" @@ -2497,7 +2511,9 @@ const SBuiltinFuncDefinition funcMgtBuiltins[] = { .initFunc = avgFunctionSetup, .processFunc = avgFunction, .finalizeFunc = avgPartialFinalize, +#ifdef BUILD_NO_CALL .invertFunc = avgInvertFunction, +#endif .combineFunc = avgCombine, }, { @@ -2509,7 +2525,9 @@ const SBuiltinFuncDefinition funcMgtBuiltins[] = { .initFunc = avgFunctionSetup, .processFunc = avgFunctionMerge, .finalizeFunc = avgFinalize, +#ifdef BUILD_NO_CALL .invertFunc = avgInvertFunction, +#endif .combineFunc = avgCombine, }, { @@ -2523,7 +2541,9 @@ const SBuiltinFuncDefinition funcMgtBuiltins[] = { .processFunc = percentileFunction, .sprocessFunc = percentileScalarFunction, .finalizeFunc = percentileFinalize, +#ifdef BUILD_NO_CALL .invertFunc = NULL, +#endif .combineFunc = NULL, }, { @@ -2536,7 +2556,9 @@ const SBuiltinFuncDefinition funcMgtBuiltins[] = { .processFunc = apercentileFunction, .sprocessFunc = apercentileScalarFunction, .finalizeFunc = apercentileFinalize, +#ifdef BUILD_NO_CALL .invertFunc = NULL, +#endif .combineFunc = apercentileCombine, .pPartialFunc = "_apercentile_partial", .pMergeFunc = "_apercentile_merge", @@ -2551,7 +2573,9 @@ const SBuiltinFuncDefinition funcMgtBuiltins[] = { .initFunc = apercentileFunctionSetup, .processFunc = apercentileFunction, .finalizeFunc = apercentilePartialFinalize, +#ifdef BUILD_NO_CALL .invertFunc = NULL, +#endif .combineFunc = apercentileCombine, }, { @@ -2563,7 +2587,9 @@ const SBuiltinFuncDefinition funcMgtBuiltins[] = { .initFunc = apercentileFunctionSetup, .processFunc = apercentileFunctionMerge, .finalizeFunc = apercentileFinalize, +#ifdef BUILD_NO_CALL .invertFunc = NULL, +#endif .combineFunc = apercentileCombine, }, { @@ -2609,7 +2635,9 @@ const SBuiltinFuncDefinition funcMgtBuiltins[] = { .processFunc = spreadFunction, .sprocessFunc = spreadScalarFunction, .finalizeFunc = spreadFinalize, +#ifdef BUILD_NO_CALL .invertFunc = NULL, +#endif .combineFunc = spreadCombine, .pPartialFunc = "_spread_partial", .pMergeFunc = "_spread_merge" @@ -2624,7 +2652,9 @@ const SBuiltinFuncDefinition funcMgtBuiltins[] = { .initFunc = spreadFunctionSetup, .processFunc = spreadFunction, .finalizeFunc = spreadPartialFinalize, +#ifdef BUILD_NO_CALL .invertFunc = NULL, +#endif .combineFunc = spreadCombine, }, { @@ -2637,7 +2667,9 @@ const SBuiltinFuncDefinition funcMgtBuiltins[] = { .initFunc = spreadFunctionSetup, .processFunc = spreadFunctionMerge, .finalizeFunc = spreadFinalize, +#ifdef BUILD_NO_CALL .invertFunc = NULL, +#endif .combineFunc = spreadCombine, }, { @@ -2651,7 +2683,9 @@ const SBuiltinFuncDefinition funcMgtBuiltins[] = { .initFunc = elapsedFunctionSetup, .processFunc = elapsedFunction, .finalizeFunc = elapsedFinalize, +#ifdef BUILD_NO_CALL .invertFunc = NULL, +#endif .combineFunc = elapsedCombine, }, { @@ -2664,7 +2698,9 @@ const SBuiltinFuncDefinition funcMgtBuiltins[] = { .initFunc = elapsedFunctionSetup, .processFunc = elapsedFunction, .finalizeFunc = elapsedPartialFinalize, +#ifdef BUILD_NO_CALL .invertFunc = NULL, +#endif .combineFunc = elapsedCombine, }, { @@ -2677,7 +2713,9 @@ const SBuiltinFuncDefinition funcMgtBuiltins[] = { .initFunc = elapsedFunctionSetup, .processFunc = elapsedFunctionMerge, .finalizeFunc = elapsedFinalize, + #ifdef BUILD_NO_CALL .invertFunc = NULL, + #endif .combineFunc = elapsedCombine, }, { @@ -2907,7 +2945,9 @@ const SBuiltinFuncDefinition funcMgtBuiltins[] = { .processFunc = histogramFunction, .sprocessFunc = histogramScalarFunction, .finalizeFunc = histogramFinalize, +#ifdef BUILD_NO_CALL .invertFunc = NULL, +#endif .combineFunc = histogramCombine, .pPartialFunc = "_histogram_partial", .pMergeFunc = "_histogram_merge", @@ -2921,7 +2961,9 @@ const SBuiltinFuncDefinition funcMgtBuiltins[] = { .initFunc = histogramFunctionSetup, .processFunc = histogramFunctionPartial, .finalizeFunc = histogramPartialFinalize, +#ifdef BUILD_NO_CALL .invertFunc = NULL, +#endif .combineFunc = histogramCombine, }, { @@ -2933,7 +2975,9 @@ const SBuiltinFuncDefinition funcMgtBuiltins[] = { .initFunc = functionSetup, .processFunc = histogramFunctionMerge, .finalizeFunc = histogramFinalize, +#ifdef BUILD_NO_CALL .invertFunc = NULL, +#endif .combineFunc = histogramCombine, }, { @@ -2946,7 +2990,9 @@ const SBuiltinFuncDefinition funcMgtBuiltins[] = { .processFunc = hllFunction, .sprocessFunc = hllScalarFunction, .finalizeFunc = hllFinalize, +#ifdef BUILD_NO_CALL .invertFunc = NULL, +#endif .combineFunc = hllCombine, .pPartialFunc = "_hyperloglog_partial", .pMergeFunc = "_hyperloglog_merge" @@ -2960,7 +3006,9 @@ const SBuiltinFuncDefinition funcMgtBuiltins[] = { .initFunc = functionSetup, .processFunc = hllFunction, .finalizeFunc = hllPartialFinalize, +#ifdef BUILD_NO_CALL .invertFunc = NULL, +#endif .combineFunc = hllCombine, }, { @@ -2972,7 +3020,9 @@ const SBuiltinFuncDefinition funcMgtBuiltins[] = { .initFunc = functionSetup, .processFunc = hllFunctionMerge, .finalizeFunc = hllFinalize, +#ifdef BUILD_NO_CALL .invertFunc = NULL, +#endif .combineFunc = hllCombine, }, { diff --git a/source/libs/function/src/builtinsimpl.c b/source/libs/function/src/builtinsimpl.c index 8a2e118fe7..390190e8db 100644 --- a/source/libs/function/src/builtinsimpl.c +++ b/source/libs/function/src/builtinsimpl.c @@ -549,6 +549,7 @@ int32_t countFunction(SqlFunctionCtx* pCtx) { return TSDB_CODE_SUCCESS; } +#ifdef BUILD_NO_CALL int32_t countInvertFunction(SqlFunctionCtx* pCtx) { int64_t numOfElem = getNumOfElems(pCtx); @@ -559,6 +560,7 @@ int32_t countInvertFunction(SqlFunctionCtx* pCtx) { SET_VAL(pResInfo, *((int64_t*)buf), 1); return TSDB_CODE_SUCCESS; } +#endif int32_t combineFunction(SqlFunctionCtx* pDestCtx, SqlFunctionCtx* pSourceCtx) { SResultRowEntryInfo* pDResInfo = GET_RES_INFO(pDestCtx); @@ -642,6 +644,7 @@ _sum_over: return TSDB_CODE_SUCCESS; } +#ifdef BUILD_NO_CALL int32_t sumInvertFunction(SqlFunctionCtx* pCtx) { int32_t numOfElem = 0; @@ -699,6 +702,7 @@ int32_t sumInvertFunction(SqlFunctionCtx* pCtx) { SET_VAL(GET_RES_INFO(pCtx), numOfElem, 1); return TSDB_CODE_SUCCESS; } +#endif int32_t sumCombine(SqlFunctionCtx* pDestCtx, SqlFunctionCtx* pSourceCtx) { SResultRowEntryInfo* pDResInfo = GET_RES_INFO(pDestCtx); @@ -1230,6 +1234,7 @@ int32_t stddevFunctionMerge(SqlFunctionCtx* pCtx) { return TSDB_CODE_SUCCESS; } +#ifdef BUILD_NO_CALL int32_t stddevInvertFunction(SqlFunctionCtx* pCtx) { int32_t numOfElem = 0; @@ -1294,6 +1299,7 @@ int32_t stddevInvertFunction(SqlFunctionCtx* pCtx) { SET_VAL(GET_RES_INFO(pCtx), numOfElem, 1); return TSDB_CODE_SUCCESS; } +#endif int32_t stddevFinalize(SqlFunctionCtx* pCtx, SSDataBlock* pBlock) { SInputColumnInfoData* pInput = &pCtx->input; @@ -1578,11 +1584,6 @@ int32_t leastSQRFinalize(SqlFunctionCtx* pCtx, SSDataBlock* pBlock) { return pResInfo->numOfRes; } -int32_t leastSQRInvertFunction(SqlFunctionCtx* pCtx) { - // TODO - return TSDB_CODE_SUCCESS; -} - int32_t leastSQRCombine(SqlFunctionCtx* pDestCtx, SqlFunctionCtx* pSourceCtx) { SResultRowEntryInfo* pDResInfo = GET_RES_INFO(pDestCtx); SLeastSQRInfo* pDBuf = GET_ROWCELL_INTERBUF(pDResInfo); diff --git a/source/libs/function/src/detail/tavgfunction.c b/source/libs/function/src/detail/tavgfunction.c index e626c937da..6bcbd1c3a7 100644 --- a/source/libs/function/src/detail/tavgfunction.c +++ b/source/libs/function/src/detail/tavgfunction.c @@ -724,6 +724,7 @@ int32_t avgFunctionMerge(SqlFunctionCtx* pCtx) { return TSDB_CODE_SUCCESS; } +#ifdef BUILD_NO_CALL int32_t avgInvertFunction(SqlFunctionCtx* pCtx) { int32_t numOfElem = 0; @@ -786,6 +787,7 @@ int32_t avgInvertFunction(SqlFunctionCtx* pCtx) { SET_VAL(GET_RES_INFO(pCtx), numOfElem, 1); return TSDB_CODE_SUCCESS; } +#endif int32_t avgCombine(SqlFunctionCtx* pDestCtx, SqlFunctionCtx* pSourceCtx) { SResultRowEntryInfo* pDResInfo = GET_RES_INFO(pDestCtx); diff --git a/source/libs/function/src/functionMgt.c b/source/libs/function/src/functionMgt.c index 036e4238d4..7bb863839a 100644 --- a/source/libs/function/src/functionMgt.c +++ b/source/libs/function/src/functionMgt.c @@ -281,6 +281,7 @@ void fmFuncMgtDestroy() { } } +#ifdef BUILD_NO_CALL int32_t fmSetInvertFunc(int32_t funcId, SFuncExecFuncs* pFpSet) { if (fmIsUserDefinedFunc(funcId) || funcId < 0 || funcId >= funcMgtBuiltinsNum) { return TSDB_CODE_FAILED; @@ -314,6 +315,7 @@ bool fmIsInvertible(int32_t funcId) { } return res; } +#endif // function has same input/output type bool fmIsSameInOutType(int32_t funcId) { From bc7434e2b5b4ddc539d194e67a2fd1da272061c4 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Fri, 12 Jan 2024 14:49:16 +0800 Subject: [PATCH 082/102] fix(stream): set error code for return. --- source/dnode/mnode/impl/src/mndStreamTrans.c | 2 ++ 1 file changed, 2 insertions(+) diff --git a/source/dnode/mnode/impl/src/mndStreamTrans.c b/source/dnode/mnode/impl/src/mndStreamTrans.c index bfea4349b0..847f55a8fd 100644 --- a/source/dnode/mnode/impl/src/mndStreamTrans.c +++ b/source/dnode/mnode/impl/src/mndStreamTrans.c @@ -92,6 +92,7 @@ bool mndStreamTransConflictCheck(SMnode* pMnode, int64_t streamUid, const char* if (strcmp(pTransName, MND_STREAM_DROP_NAME) != 0) { mWarn("conflict with other transId:%d streamUid:0x%" PRIx64 ", trans:%s", tInfo.transId, tInfo.streamUid, tInfo.name); + terrno = TSDB_CODE_MND_TRANS_CONFLICT; return true; } else { mDebug("not conflict with checkpoint trans, name:%s, continue create trans", pTransName); @@ -100,6 +101,7 @@ bool mndStreamTransConflictCheck(SMnode* pMnode, int64_t streamUid, const char* (strcmp(tInfo.name, MND_STREAM_TASK_RESET_NAME) == 0)) { mWarn("conflict with other transId:%d streamUid:0x%" PRIx64 ", trans:%s", tInfo.transId, tInfo.streamUid, tInfo.name); + terrno = TSDB_CODE_MND_TRANS_CONFLICT; return true; } } else { From bc1eacf626e601004f0fe0e62660686a69032942 Mon Sep 17 00:00:00 2001 From: wangjiaming0909 <604227650@qq.com> Date: Thu, 11 Jan 2024 15:48:07 +0800 Subject: [PATCH 083/102] add PARTITION_BEFORE_GROUP hint to use partition node before agg --- docs/en/12-taos-sql/06-select.md | 4 +++- docs/zh/12-taos-sql/06-select.md | 4 +++- include/common/ttokendef.h | 1 + include/libs/nodes/querynodes.h | 1 + source/libs/parser/src/parAstCreater.c | 24 +++++++++++++++++++ source/libs/parser/src/parTokenizer.c | 1 + source/libs/planner/inc/planInt.h | 1 + source/libs/planner/src/planOptimizer.c | 3 ++- source/libs/planner/src/planUtil.c | 13 ++++++++++ tests/system-test/2-query/partition_by_col.py | 10 ++++++++ 10 files changed, 59 insertions(+), 3 deletions(-) diff --git a/docs/en/12-taos-sql/06-select.md b/docs/en/12-taos-sql/06-select.md index 2c94787440..1fc6fb7e67 100755 --- a/docs/en/12-taos-sql/06-select.md +++ b/docs/en/12-taos-sql/06-select.md @@ -91,13 +91,15 @@ The list of currently supported Hints is as follows: | :-----------: | -------------- | -------------------------- | -----------------------------------| | BATCH_SCAN | None | Batch table scan | JOIN statment for stable | | NO_BATCH_SCAN | None | Sequential table scan | JOIN statment for stable | -| SORT_FOR_GROUP| None | Use sort for partition | With normal column in partition by list | +| SORT_FOR_GROUP| None | Use sort for partition, conflict with PARTITION_FIRST | With normal column in partition by list | +| PARTITION_FIRST| None | Use Partition before aggregate, conflict with SORT_FOR_GROUP | With normal column in partition by list | For example: ```sql SELECT /*+ BATCH_SCAN() */ a.ts FROM stable1 a, stable2 b where a.tag0 = b.tag0 and a.ts = b.ts; SELECT /*+ SORT_FOR_GROUP() */ count(*), c1 FROM stable1 PARTITION BY c1; +SELECT /*+ PARTITION_FIRST() */ count(*), c1 FROM stable1 PARTITION BY c1; ``` ## Lists diff --git a/docs/zh/12-taos-sql/06-select.md b/docs/zh/12-taos-sql/06-select.md index 3c6e4f3bcf..2a7dff6f7d 100755 --- a/docs/zh/12-taos-sql/06-select.md +++ b/docs/zh/12-taos-sql/06-select.md @@ -91,13 +91,15 @@ Hints 是用户控制单个语句查询优化的一种手段,当 Hint 不适 | :-----------: | -------------- | -------------------------- | -----------------------------| | BATCH_SCAN | 无 | 采用批量读表的方式 | 超级表 JOIN 语句 | | NO_BATCH_SCAN | 无 | 采用顺序读表的方式 | 超级表 JOIN 语句 | -| SORT_FOR_GROUP| 无 | 采用sort方式进行分组 | partition by 列表有普通列时 | +| SORT_FOR_GROUP| 无 | 采用sort方式进行分组, 与PARTITION_FIRST冲突 | partition by 列表有普通列时 | +| PARTITION_FIRST| 无 | 在聚合之前使用PARTITION计算分组, 与SORT_FOR_GROUP冲突 | partition by 列表有普通列时 | 举例: ```sql SELECT /*+ BATCH_SCAN() */ a.ts FROM stable1 a, stable2 b where a.tag0 = b.tag0 and a.ts = b.ts; SELECT /*+ SORT_FOR_GROUP() */ count(*), c1 FROM stable1 PARTITION BY c1; +SELECT /*+ PARTITION_FIRST() */ count(*), c1 FROM stable1 PARTITION BY c1; ``` ## 列表 diff --git a/include/common/ttokendef.h b/include/common/ttokendef.h index bdee3934fe..445fe0737b 100644 --- a/include/common/ttokendef.h +++ b/include/common/ttokendef.h @@ -373,6 +373,7 @@ #define TK_BATCH_SCAN 606 #define TK_NO_BATCH_SCAN 607 #define TK_SORT_FOR_GROUP 608 +#define TK_PARTITION_FIRST 609 #define TK_NK_NIL 65535 diff --git a/include/libs/nodes/querynodes.h b/include/libs/nodes/querynodes.h index fdf598153f..331679a3d3 100644 --- a/include/libs/nodes/querynodes.h +++ b/include/libs/nodes/querynodes.h @@ -126,6 +126,7 @@ typedef enum EHintOption { HINT_NO_BATCH_SCAN = 1, HINT_BATCH_SCAN, HINT_SORT_FOR_GROUP, + HINT_PARTITION_FIRST, } EHintOption; typedef struct SHintNode { diff --git a/source/libs/parser/src/parAstCreater.c b/source/libs/parser/src/parAstCreater.c index eb866c99aa..8e89ae1f53 100644 --- a/source/libs/parser/src/parAstCreater.c +++ b/source/libs/parser/src/parAstCreater.c @@ -371,6 +371,18 @@ SNode* createValueNode(SAstCreateContext* pCxt, int32_t dataType, const SToken* return (SNode*)val; } +static bool hasHint(SNodeList* pHintList, EHintOption hint) { + if (!pHintList) return false; + SNode* pNode; + FOREACH(pNode, pHintList) { + SHintNode* pHint = (SHintNode*)pNode; + if (pHint->option == hint) { + return true; + } + } + return false; +} + bool addHintNodeToList(SAstCreateContext* pCxt, SNodeList** ppHintList, EHintOption opt, SToken* paramList, int32_t paramNum) { void* value = NULL; @@ -384,6 +396,10 @@ bool addHintNodeToList(SAstCreateContext* pCxt, SNodeList** ppHintList, EHintOpt } case HINT_SORT_FOR_GROUP: if (paramNum > 0) return true; + if (hasHint(*ppHintList, HINT_PARTITION_FIRST)) return true; + break; + case HINT_PARTITION_FIRST: + if (paramNum > 0 || hasHint(*ppHintList, HINT_SORT_FOR_GROUP)) return true; break; default: return true; @@ -455,6 +471,14 @@ SNodeList* createHintNodeList(SAstCreateContext* pCxt, const SToken* pLiteral) { } opt = HINT_SORT_FOR_GROUP; break; + case TK_PARTITION_FIRST: + lastComma = false; + if (0 != opt || inParamList) { + quit = true; + break; + } + opt = HINT_PARTITION_FIRST; + break; case TK_NK_LP: lastComma = false; if (0 == opt || inParamList) { diff --git a/source/libs/parser/src/parTokenizer.c b/source/libs/parser/src/parTokenizer.c index 03a5317cd3..072892fe7f 100644 --- a/source/libs/parser/src/parTokenizer.c +++ b/source/libs/parser/src/parTokenizer.c @@ -170,6 +170,7 @@ static SKeyword keywordTable[] = { {"PAGES", TK_PAGES}, {"PAGESIZE", TK_PAGESIZE}, {"PARTITION", TK_PARTITION}, + {"PARTITION_FIRST", TK_PARTITION_FIRST}, {"PASS", TK_PASS}, {"PORT", TK_PORT}, {"PPS", TK_PPS}, diff --git a/source/libs/planner/inc/planInt.h b/source/libs/planner/inc/planInt.h index e2a4ded5a9..d4074e1373 100644 --- a/source/libs/planner/inc/planInt.h +++ b/source/libs/planner/inc/planInt.h @@ -47,6 +47,7 @@ int32_t validateQueryPlan(SPlanContext* pCxt, SQueryPlan* pPlan); bool getBatchScanOptionFromHint(SNodeList* pList); bool getSortForGroupOptHint(SNodeList* pList); +bool getOptHint(SNodeList* pList, EHintOption hint); SLogicNode* getLogicNodeRootNode(SLogicNode* pCurr); int32_t collectTableAliasFromNodes(SNode* pNode, SSHashObj** ppRes); bool isPartTableAgg(SAggLogicNode* pAgg); diff --git a/source/libs/planner/src/planOptimizer.c b/source/libs/planner/src/planOptimizer.c index 035377d22e..730dcd9352 100644 --- a/source/libs/planner/src/planOptimizer.c +++ b/source/libs/planner/src/planOptimizer.c @@ -3997,7 +3997,8 @@ static int32_t partitionColsOpt(SOptimizeContext* pCxt, SLogicSubplan* pLogicSub } } return code; - } else if (pNode->node.pParent && nodeType(pNode->node.pParent) == QUERY_NODE_LOGIC_PLAN_AGG) { + } else if (pNode->node.pParent && nodeType(pNode->node.pParent) == QUERY_NODE_LOGIC_PLAN_AGG && + !getOptHint(pRootNode->pHint, HINT_PARTITION_FIRST)) { // Check if we can delete partition node SAggLogicNode* pAgg = (SAggLogicNode*)pNode->node.pParent; FOREACH(node, pNode->pPartitionKeys) { diff --git a/source/libs/planner/src/planUtil.c b/source/libs/planner/src/planUtil.c index 2da270e42d..aeef3f2487 100644 --- a/source/libs/planner/src/planUtil.c +++ b/source/libs/planner/src/planUtil.c @@ -432,6 +432,7 @@ bool getBatchScanOptionFromHint(SNodeList* pList) { } bool getSortForGroupOptHint(SNodeList* pList) { + if (!pList) return false; SNode* pNode; FOREACH(pNode, pList) { SHintNode* pHint = (SHintNode*)pNode; @@ -442,6 +443,18 @@ bool getSortForGroupOptHint(SNodeList* pList) { return false; } +bool getOptHint(SNodeList* pList, EHintOption hint) { + if (!pList) return false; + SNode* pNode; + FOREACH(pNode, pList) { + SHintNode* pHint = (SHintNode*)pNode; + if (pHint->option == hint) { + return true; + } + } + return false; +} + int32_t collectTableAliasFromNodes(SNode* pNode, SSHashObj** ppRes) { int32_t code = TSDB_CODE_SUCCESS; SLogicNode* pCurr = (SLogicNode*)pNode; diff --git a/tests/system-test/2-query/partition_by_col.py b/tests/system-test/2-query/partition_by_col.py index feb8693e3e..549e2738be 100644 --- a/tests/system-test/2-query/partition_by_col.py +++ b/tests/system-test/2-query/partition_by_col.py @@ -169,6 +169,16 @@ class TDTestCase: self.check_explain_res_has_row("Partition on", self.explain_sql(sql)) self.check_explain_res_has_row("Sort", self.explain_sql(sql_hint)) + sql = 'select count(*), c1 from meters partition by c1' + sql_hint = 'select /*+ sort_for_group() partition_first()*/ count(*), c1 from meters partition by c1' + self.check_explain_res_has_row("Sort", self.explain_sql(sql_hint)) + sql_hint = 'select /*+ partition_first()*/ count(*), c1 from meters partition by c1' + self.check_explain_res_has_row("Partition on", self.explain_sql(sql_hint)) + sql_hint = 'select /*+ partition_first() sort_for_group()*/ count(*), c1 from meters partition by c1' + self.check_explain_res_has_row("Partition on", self.explain_sql(sql_hint)) + sql_hint = 'select /*+ sort_for_group() partition_first()*/ count(*), c1 from meters partition by c1' + self.check_explain_res_has_row("Sort", self.explain_sql(sql_hint)) + def add_order_by(self, sql: str, order_by: str, select_list: str = "*") -> str: return "select %s from (%s)t order by %s" % (select_list, sql, order_by) From 00e2bdec23838ae8b3eeb8e0dd537fa5c3e849e5 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Fri, 12 Jan 2024 19:08:46 +0800 Subject: [PATCH 084/102] refactor: do some internal refactor. --- include/libs/stream/tstream.h | 2 +- source/dnode/vnode/src/tqCommon/tqCommon.c | 8 +- source/libs/stream/src/streamMeta.c | 14 +- source/libs/stream/src/streamStart.c | 166 ++++++++++++++------- 4 files changed, 120 insertions(+), 70 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index a115783b70..04694b05fd 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -872,7 +872,7 @@ int64_t streamMetaGetLatestCheckpointId(SStreamMeta* pMeta); void streamMetaNotifyClose(SStreamMeta* pMeta); void streamMetaStartHb(SStreamMeta* pMeta); bool streamMetaTaskInTimer(SStreamMeta* pMeta); -int32_t streamMetaUpdateTaskDownstreamStatus(SStreamMeta* pMeta, int64_t streamId, int32_t taskId, int64_t startTs, +int32_t streamMetaAddTaskLaunchResult(SStreamMeta* pMeta, int64_t streamId, int32_t taskId, int64_t startTs, int64_t endTs, bool ready); void streamMetaRLock(SStreamMeta* pMeta); void streamMetaRUnLock(SStreamMeta* pMeta); diff --git a/source/dnode/vnode/src/tqCommon/tqCommon.c b/source/dnode/vnode/src/tqCommon/tqCommon.c index 7883c858f0..5a92677462 100644 --- a/source/dnode/vnode/src/tqCommon/tqCommon.c +++ b/source/dnode/vnode/src/tqCommon/tqCommon.c @@ -526,7 +526,7 @@ int32_t tqStreamTaskProcessCheckRsp(SStreamMeta* pMeta, SRpcMsg* pMsg, bool isLe streamMetaRUnLock(pMeta); if (hasHistoryTask) { - streamMetaUpdateTaskDownstreamStatus(pMeta, fId.streamId, fId.taskId, initTs, now, false); + streamMetaAddTaskLaunchResult(pMeta, fId.streamId, fId.taskId, initTs, now, false); } tqError("vgId:%d not leader, task:0x%x not handle the check rsp, downstream:0x%x (vgId:%d)", vgId, @@ -539,7 +539,7 @@ int32_t tqStreamTaskProcessCheckRsp(SStreamMeta* pMeta, SRpcMsg* pMsg, bool isLe code = terrno = TSDB_CODE_STREAM_TASK_NOT_EXIST; } - streamMetaUpdateTaskDownstreamStatus(pMeta, rsp.streamId, rsp.upstreamTaskId, initTs, now, false); + streamMetaAddTaskLaunchResult(pMeta, rsp.streamId, rsp.upstreamTaskId, initTs, now, false); return code; } @@ -553,13 +553,13 @@ int32_t tqStreamTaskProcessCheckRsp(SStreamMeta* pMeta, SRpcMsg* pMsg, bool isLe streamMetaRUnLock(pMeta); if (hasHistoryTask) { - streamMetaUpdateTaskDownstreamStatus(pMeta, fId.streamId, fId.taskId, initTs, now, false); + streamMetaAddTaskLaunchResult(pMeta, fId.streamId, fId.taskId, initTs, now, false); } } else { streamMetaRUnLock(pMeta); } - streamMetaUpdateTaskDownstreamStatus(pMeta, rsp.streamId, rsp.upstreamTaskId, initTs, now, false); + streamMetaAddTaskLaunchResult(pMeta, rsp.streamId, rsp.upstreamTaskId, initTs, now, false); 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, vgId); diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 68348f462f..ee2b70eebe 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -1466,7 +1466,7 @@ int32_t streamMetaStartAllTasks(SStreamMeta* pMeta) { SStreamTask* pTask = streamMetaAcquireTask(pMeta, pTaskId->streamId, pTaskId->taskId); if (pTask == NULL) { stError("vgId:%d failed to acquire task:0x%x during start tasks", pMeta->vgId, pTaskId->taskId); - streamMetaUpdateTaskDownstreamStatus(pMeta, pTaskId->streamId, pTaskId->taskId, 0, now, false); + streamMetaAddTaskLaunchResult(pMeta, pTaskId->streamId, pTaskId->taskId, 0, now, false); continue; } @@ -1487,7 +1487,7 @@ int32_t streamMetaStartAllTasks(SStreamMeta* pMeta) { streamLaunchFillHistoryTask(pTask); } - streamMetaUpdateTaskDownstreamStatus(pMeta, pTaskId->streamId, pTaskId->taskId, pInfo->init, pInfo->start, true); + streamMetaAddTaskLaunchResult(pMeta, pTaskId->streamId, pTaskId->taskId, pInfo->init, pInfo->start, true); streamMetaReleaseTask(pMeta, pTask); continue; } @@ -1497,10 +1497,10 @@ int32_t streamMetaStartAllTasks(SStreamMeta* pMeta) { stError("vgId:%d failed to handle event:%d", pMeta->vgId, TASK_EVENT_INIT); code = ret; - streamMetaUpdateTaskDownstreamStatus(pMeta, pTaskId->streamId, pTaskId->taskId, pInfo->init, pInfo->start, false); + streamMetaAddTaskLaunchResult(pMeta, pTaskId->streamId, pTaskId->taskId, pInfo->init, pInfo->start, false); if (HAS_RELATED_FILLHISTORY_TASK(pTask)) { STaskId* pId = &pTask->hTaskInfo.id; - streamMetaUpdateTaskDownstreamStatus(pMeta, pId->streamId, pId->taskId, pInfo->init, pInfo->start, false); + streamMetaAddTaskLaunchResult(pMeta, pId->streamId, pId->taskId, pInfo->init, pInfo->start, false); } } @@ -1519,7 +1519,7 @@ int32_t streamMetaStartOneTask(SStreamMeta* pMeta, int64_t streamId, int32_t tas SStreamTask* pTask = streamMetaAcquireTask(pMeta, streamId, taskId); if (pTask == NULL) { stError("vgId:%d failed to acquire task:0x%x during start tasks", pMeta->vgId, taskId); - streamMetaUpdateTaskDownstreamStatus(pMeta, streamId, taskId, 0, taosGetTimestampMs(), false); + streamMetaAddTaskLaunchResult(pMeta, streamId, taskId, 0, taosGetTimestampMs(), false); return TSDB_CODE_STREAM_TASK_IVLD_STATUS; } @@ -1538,10 +1538,10 @@ int32_t streamMetaStartOneTask(SStreamMeta* pMeta, int64_t streamId, int32_t tas if (ret != TSDB_CODE_SUCCESS) { stError("vgId:%d failed to handle event:%d", pMeta->vgId, TASK_EVENT_INIT); - streamMetaUpdateTaskDownstreamStatus(pMeta, streamId, taskId, pInfo->init, pInfo->start, false); + streamMetaAddTaskLaunchResult(pMeta, streamId, taskId, pInfo->init, pInfo->start, false); if (HAS_RELATED_FILLHISTORY_TASK(pTask)) { STaskId* pId = &pTask->hTaskInfo.id; - streamMetaUpdateTaskDownstreamStatus(pMeta, pId->streamId, pId->taskId, pInfo->init, pInfo->start, false); + streamMetaAddTaskLaunchResult(pMeta, pId->streamId, pId->taskId, pInfo->init, pInfo->start, false); } } diff --git a/source/libs/stream/src/streamStart.c b/source/libs/stream/src/streamStart.c index 24909a9776..27ed6af402 100644 --- a/source/libs/stream/src/streamStart.c +++ b/source/libs/stream/src/streamStart.c @@ -26,6 +26,7 @@ typedef struct SLaunchHTaskInfo { SStreamMeta* pMeta; STaskId id; + STaskId hTaskId; } SLaunchHTaskInfo; typedef struct STaskRecheckInfo { @@ -43,7 +44,8 @@ typedef struct STaskInitTs { static int32_t streamSetParamForScanHistory(SStreamTask* pTask); static void streamTaskSetRangeStreamCalc(SStreamTask* pTask); static int32_t initScanHistoryReq(SStreamTask* pTask, SStreamScanHistoryReq* pReq, int8_t igUntreated); -static SLaunchHTaskInfo* createHTaskLaunchInfo(SStreamMeta* pMeta, int64_t streamId, int32_t taskId); +static SLaunchHTaskInfo* createHTaskLaunchInfo(SStreamMeta* pMeta, int64_t streamId, int32_t taskId, int64_t hStreamId, + int32_t hTaskId); static void tryLaunchHistoryTask(void* param, void* tmrId); static void doProcessDownstreamReadyRsp(SStreamTask* pTask); @@ -394,7 +396,7 @@ void doProcessDownstreamReadyRsp(SStreamTask* pTask) { int64_t initTs = pTask->execInfo.init; int64_t startTs = pTask->execInfo.start; - streamMetaUpdateTaskDownstreamStatus(pTask->pMeta, pTask->id.streamId, pTask->id.taskId, initTs, startTs, true); + streamMetaAddTaskLaunchResult(pTask->pMeta, pTask->id.streamId, pTask->id.taskId, initTs, startTs, true); // start the related fill-history task, when current task is ready // not invoke in success callback due to the deadlock. @@ -492,12 +494,12 @@ int32_t streamProcessCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* pRs int32_t startTs = pTask->execInfo.init; int64_t now = taosGetTimestampMs(); - streamMetaUpdateTaskDownstreamStatus(pTask->pMeta, pTask->id.streamId, pTask->id.taskId, startTs, now, false); + streamMetaAddTaskLaunchResult(pTask->pMeta, pTask->id.streamId, pTask->id.taskId, startTs, now, false); // automatically set the related fill-history task to be failed. if (HAS_RELATED_FILLHISTORY_TASK(pTask)) { STaskId* pId = &pTask->hTaskInfo.id; - streamMetaUpdateTaskDownstreamStatus(pTask->pMeta, pId->streamId, pId->taskId, startTs, now, false); + streamMetaAddTaskLaunchResult(pTask->pMeta, pId->streamId, pId->taskId, startTs, now, false); } } else { // TASK_DOWNSTREAM_NOT_READY, let's retry in 100ms STaskRecheckInfo* pInfo = createRecheckInfo(pTask, pRsp); @@ -731,21 +733,27 @@ static void checkFillhistoryTaskStatus(SStreamTask* pTask, SStreamTask* pHTask) static void tryLaunchHistoryTask(void* param, void* tmrId) { SLaunchHTaskInfo* pInfo = param; SStreamMeta* pMeta = pInfo->pMeta; + int64_t now = taosGetTimestampMs(); streamMetaWLock(pMeta); SStreamTask** ppTask = (SStreamTask**)taosHashGet(pMeta->pTasksMap, &pInfo->id, sizeof(pInfo->id)); - if (ppTask) { + if (ppTask != NULL) { ASSERT((*ppTask)->status.timerActive >= 1); if (streamTaskShouldStop(*ppTask)) { char* p = streamTaskGetStatus(*ppTask)->name; + 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, p, (*ppTask)->hTaskInfo.retryTimes, ref); + streamMetaWUnLock(pMeta); + + // record the related fill-history task failed + STaskId* pHTaskId = &(*ppTask)->hTaskInfo.id; + streamMetaAddTaskLaunchResult(pMeta, pHTaskId->streamId, pHTaskId->taskId, 0, now, false); taosMemoryFree(pInfo); - streamMetaWUnLock(pMeta); return; } } @@ -764,8 +772,9 @@ static void tryLaunchHistoryTask(void* param, void* tmrId) { if (pHTaskInfo->retryTimes > MAX_RETRY_LAUNCH_HISTORY_TASK) { int32_t ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1); - streamMetaReleaseTask(pMeta, pTask); + streamMetaAddTaskLaunchResult(pMeta, pInfo->id.streamId, pInfo->id.taskId, 0, now, false); + streamMetaReleaseTask(pMeta, pTask); 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); @@ -798,91 +807,132 @@ static void tryLaunchHistoryTask(void* param, void* tmrId) { 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); } } else { - stError("s-task:0x%x failed to load task, it may have been destroyed, not launch related fill-history task", + streamMetaAddTaskLaunchResult(pMeta, pInfo->id.streamId, pInfo->id.taskId, 0, now, false); + + int32_t ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1); + stError("s-task:0x%x failed to load fill-history task, it may have been destroyed, not launch fill-history task", (int32_t)pInfo->id.taskId); } taosMemoryFree(pInfo); } -SLaunchHTaskInfo* createHTaskLaunchInfo(SStreamMeta* pMeta, int64_t streamId, int32_t taskId) { +SLaunchHTaskInfo* createHTaskLaunchInfo(SStreamMeta* pMeta, int64_t streamId, int32_t taskId, int64_t hStreamId, + int32_t hTaskId) { 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->id.taskId = taskId; + + pInfo->hTaskId.streamId = hStreamId; + pInfo->hTaskId.taskId = hTaskId; + pInfo->pMeta = pMeta; return pInfo; } -// an fill history task needs to be started. -int32_t streamLaunchFillHistoryTask(SStreamTask* pTask) { - SStreamMeta* pMeta = pTask->pMeta; - - int64_t streamId = pTask->hTaskInfo.id.streamId; - int32_t hTaskId = pTask->hTaskInfo.id.taskId; +static int32_t handleNotBuiltFillHistoryTask(SStreamTask* pTask) { + SStreamMeta* pMeta = pTask->pMeta; + STaskExecStatisInfo* pExecInfo = &pTask->execInfo; + const char* idStr = pTask->id.idStr; + int64_t hStreamId = pTask->hTaskInfo.id.streamId; + int32_t hTaskId = pTask->hTaskInfo.id.taskId; ASSERT(hTaskId != 0); - SStreamTaskState* pStatus = streamTaskGetStatus(pTask); - if (pStatus->state != TASK_STATUS__READY) { - STaskExecStatisInfo* pInfo = &pTask->execInfo; - stDebug("s-task:%s not launch related fill-history task:0x%" PRIx64 "-0x%x, status:%s", pTask->id.idStr, - pTask->hTaskInfo.id.streamId, hTaskId, pStatus->name); + stWarn("s-task:%s vgId:%d failed to launch history task:0x%x, since not built yet", idStr, pMeta->vgId, hTaskId); - streamMetaUpdateTaskDownstreamStatus(pMeta, streamId, hTaskId, pInfo->init, pInfo->start, false); - return -1;// todo set the correct error code - } else { - stDebug("s-task:%s start to launch related fill-history task:0x%" PRIx64 "-0x%x", pTask->id.idStr, - pTask->hTaskInfo.id.streamId, hTaskId); + SLaunchHTaskInfo* pInfo = createHTaskLaunchInfo(pMeta, pTask->id.streamId, pTask->id.taskId, hStreamId, hTaskId); + if (pInfo == NULL) { + stError("s-task:%s failed to launch related fill-history task, since Out Of Memory", idStr); + + streamMetaAddTaskLaunchResult(pMeta, hStreamId, hTaskId, pExecInfo->init, pExecInfo->start, false); + return terrno; } - // Set the execute conditions, including the query time window and the version range - 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 not built yet", pTask->id.idStr, pMeta->vgId, - hTaskId); + // set the launch time info + streamTaskInitForLaunchHTask(&pTask->hTaskInfo); - 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); + // check for the timer + 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, streamTimer); + + if (pTask->hTaskInfo.pTimer == NULL) { + ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1); + stError("s-task:%s failed to start timer, related fill-history task not launched, ref:%d", idStr, + pTask->status.timerActive); + + taosMemoryFree(pInfo); + streamMetaAddTaskLaunchResult(pMeta, hStreamId, hTaskId, pExecInfo->init, pExecInfo->start, false); return terrno; } - 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, streamTimer); - 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); - taosMemoryFree(pInfo); - } else { - ASSERT(ref >= 1); - stDebug("s-task:%s set timer active flag, ref:%d", pTask->id.idStr, ref); + ASSERT(ref >= 1); + + stDebug("s-task:%s set timer active flag, ref:%d", idStr, ref); + } else { // timer exists + ASSERT(pTask->status.timerActive >= 1); + stDebug("s-task:%s set timer active flag, task timer not null", idStr); + taosTmrReset(tryLaunchHistoryTask, WAIT_FOR_MINIMAL_INTERVAL, pInfo, streamTimer, &pTask->hTaskInfo.pTimer); + } + + return TSDB_CODE_SUCCESS; +} + +// an fill history task needs to be started. +int32_t streamLaunchFillHistoryTask(SStreamTask* pTask) { + SStreamMeta* pMeta = pTask->pMeta; + STaskExecStatisInfo* pExecInfo = &pTask->execInfo; + const char* idStr = pTask->id.idStr; + int64_t hStreamId = pTask->hTaskInfo.id.streamId; + int32_t hTaskId = pTask->hTaskInfo.id.taskId; + ASSERT(hTaskId != 0); + + // check stream task status in the first place. + SStreamTaskState* pStatus = streamTaskGetStatus(pTask); + if (pStatus->state != TASK_STATUS__READY) { + stDebug("s-task:%s not launch related fill-history task:0x%" PRIx64 "-0x%x, status:%s", idStr, hStreamId, hTaskId, + pStatus->name); + + streamMetaAddTaskLaunchResult(pMeta, hStreamId, hTaskId, pExecInfo->init, pExecInfo->start, false); + return -1; // todo set the correct error code + } + + stDebug("s-task:%s start to launch related fill-history task:0x%" PRIx64 "-0x%x", idStr, hStreamId, hTaskId); + + // Set the execute conditions, including the query time window and the version range + streamMetaRLock(pMeta); + SStreamTask** pHTask = taosHashGet(pMeta->pTasksMap, &pTask->hTaskInfo.id, sizeof(pTask->hTaskInfo.id)); + streamMetaRUnLock(pMeta); + + if (pHTask != NULL) { + SStreamTask* pHisTask = streamMetaAcquireTask(pMeta, hStreamId, hTaskId); + if (pHisTask == NULL) { + stDebug("s-task:%s failed acquire and start fill-history task, it may have been dropped", idStr); + streamMetaAddTaskLaunchResult(pMeta, hStreamId, hTaskId, pExecInfo->init, pExecInfo->start, false); + } else { + if (pHisTask->status.downstreamReady == 1) { // it's ready now, do nothing + stDebug("s-task:%s fill-history task is ready, no need to check downstream", pHisTask->id.idStr); + streamMetaAddTaskLaunchResult(pMeta, hStreamId, hTaskId, pExecInfo->init, pExecInfo->start, true); + } else { // exist, but not ready, continue check downstream task status + checkFillhistoryTaskStatus(pTask, pHisTask); } - } 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, WAIT_FOR_MINIMAL_INTERVAL, pInfo, streamTimer, &pTask->hTaskInfo.pTimer); + + streamMetaReleaseTask(pMeta, pHisTask); } return TSDB_CODE_SUCCESS; } - if ((*pHTask)->status.downstreamReady == 1) { - stDebug("s-task:%s fill-history task is ready, no need to check downstream", (*pHTask)->id.idStr); - } else { - checkFillhistoryTaskStatus(pTask, *pHTask); - } - - return TSDB_CODE_SUCCESS; + return handleNotBuiltFillHistoryTask(pTask); } int32_t streamTaskScanHistoryDataComplete(SStreamTask* pTask) { @@ -1114,7 +1164,7 @@ static void displayStatusInfo(SStreamMeta* pMeta, SHashObj* pTaskSet, bool succ) } } -int32_t streamMetaUpdateTaskDownstreamStatus(SStreamMeta* pMeta, int64_t streamId, int32_t taskId, int64_t startTs, +int32_t streamMetaAddTaskLaunchResult(SStreamMeta* pMeta, int64_t streamId, int32_t taskId, int64_t startTs, int64_t endTs, bool ready) { STaskStartInfo* pStartInfo = &pMeta->startInfo; STaskId id = {.streamId = streamId, .taskId = taskId}; From 44208925f9deb2651f0aec2828561522f5c52cab Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sat, 13 Jan 2024 00:10:47 +0800 Subject: [PATCH 085/102] refactor: do some internal refactor. --- source/libs/stream/src/streamStart.c | 111 +++++++++++++++++---------- 1 file changed, 69 insertions(+), 42 deletions(-) diff --git a/source/libs/stream/src/streamStart.c b/source/libs/stream/src/streamStart.c index 27ed6af402..c8879e59c7 100644 --- a/source/libs/stream/src/streamStart.c +++ b/source/libs/stream/src/streamStart.c @@ -730,6 +730,42 @@ static void checkFillhistoryTaskStatus(SStreamTask* pTask, SStreamTask* pHTask) streamTaskHandleEvent(pHTask->status.pSM, TASK_EVENT_INIT_SCANHIST); } +static void noRetryLaunchFillHistoryTask(SStreamTask* pTask, SLaunchHTaskInfo* pInfo, int64_t now) { + SStreamMeta* pMeta = pTask->pMeta; + SHistoryTaskInfo* pHTaskInfo = &pTask->hTaskInfo; + + int32_t ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1); + streamMetaAddTaskLaunchResult(pMeta, pInfo->hTaskId.streamId, pInfo->hTaskId.taskId, 0, now, false); + + 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; +} + +static void doRetryLaunchFillHistoryTask(SStreamTask* pTask, SLaunchHTaskInfo* pInfo, int64_t now) { + SStreamMeta* pMeta = pTask->pMeta; + SHistoryTaskInfo* pHTaskInfo = &pTask->hTaskInfo; + + if (streamTaskShouldStop(pTask)) { // record the failure + int32_t ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1); + stDebug("s-task:0x%" PRIx64 " stopped, not launch rel history task:0x%" PRIx64 ", ref:%d", pInfo->id.taskId, + pInfo->hTaskId.taskId, ref); + + streamMetaAddTaskLaunchResult(pMeta, pInfo->hTaskId.streamId, pInfo->hTaskId.taskId, 0, now, false); + taosMemoryFree(pInfo); + } else { + char* p = streamTaskGetStatus(pTask)->name; + 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", + pTask->id.idStr, p, hTaskId, pHTaskInfo->waitInterval, pHTaskInfo->retryTimes); + + taosTmrReset(tryLaunchHistoryTask, LAUNCH_HTASK_INTERVAL, pInfo, streamTimer, &pHTaskInfo->pTimer); + } +} + static void tryLaunchHistoryTask(void* param, void* tmrId) { SLaunchHTaskInfo* pInfo = param; SStreamMeta* pMeta = pInfo->pMeta; @@ -738,28 +774,35 @@ static void tryLaunchHistoryTask(void* param, void* tmrId) { streamMetaWLock(pMeta); SStreamTask** ppTask = (SStreamTask**)taosHashGet(pMeta->pTasksMap, &pInfo->id, sizeof(pInfo->id)); - if (ppTask != NULL) { + if (ppTask == NULL || *ppTask == NULL) { + stError("s-task:0x%x and rel fill-history task:0x%" PRIx64 " all have been destroyed, not launch", + (int32_t)pInfo->id.taskId, pInfo->hTaskId.taskId); + streamMetaWUnLock(pMeta); + + // already dropped, no need to set the failure info into the stream task meta. + taosMemoryFree(pInfo); + return; + } + + if (streamTaskShouldStop(*ppTask)) { ASSERT((*ppTask)->status.timerActive >= 1); - if (streamTaskShouldStop(*ppTask)) { - char* p = streamTaskGetStatus(*ppTask)->name; + char* p = streamTaskGetStatus(*ppTask)->name; + 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, p, (*ppTask)->hTaskInfo.retryTimes, ref); - 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, p, (*ppTask)->hTaskInfo.retryTimes, ref); - streamMetaWUnLock(pMeta); + streamMetaWUnLock(pMeta); - // record the related fill-history task failed - STaskId* pHTaskId = &(*ppTask)->hTaskInfo.id; - streamMetaAddTaskLaunchResult(pMeta, pHTaskId->streamId, pHTaskId->taskId, 0, now, false); - - taosMemoryFree(pInfo); - return; - } + // record the related fill-history task failed + streamMetaAddTaskLaunchResult(pMeta, pInfo->hTaskId.streamId, pInfo->hTaskId.taskId, 0, now, false); + taosMemoryFree(pInfo); + return; } + + SStreamTask* pTask = streamMetaAcquireTaskNoLock(pMeta, pInfo->id.streamId, pInfo->id.taskId); streamMetaWUnLock(pMeta); - SStreamTask* pTask = streamMetaAcquireTask(pMeta, pInfo->id.streamId, pInfo->id.taskId); if (pTask != NULL) { SHistoryTaskInfo* pHTaskInfo = &pTask->hTaskInfo; @@ -771,34 +814,18 @@ static void tryLaunchHistoryTask(void* param, void* tmrId) { } if (pHTaskInfo->retryTimes > MAX_RETRY_LAUNCH_HISTORY_TASK) { - int32_t ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1); - streamMetaAddTaskLaunchResult(pMeta, pInfo->id.streamId, pInfo->id.taskId, 0, now, false); - - streamMetaReleaseTask(pMeta, pTask); - 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; + noRetryLaunchFillHistoryTask(pTask, pInfo, now); } 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))) { - char* p = streamTaskGetStatus(pTask)->name; - 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", - pTask->id.idStr, p, hTaskId, pHTaskInfo->waitInterval, pHTaskInfo->retryTimes); - - taosTmrReset(tryLaunchHistoryTask, LAUNCH_HTASK_INTERVAL, pInfo, streamTimer, &pHTaskInfo->pTimer); + if (pHTask == NULL) { + doRetryLaunchFillHistoryTask(pTask, pInfo, now); streamMetaReleaseTask(pMeta, pTask); return; - } - - if (pHTask != NULL) { + } else { checkFillhistoryTaskStatus(pTask, pHTask); streamMetaReleaseTask(pMeta, pHTask); } @@ -807,15 +834,15 @@ static void tryLaunchHistoryTask(void* param, void* tmrId) { 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); } - } else { - streamMetaAddTaskLaunchResult(pMeta, pInfo->id.streamId, pInfo->id.taskId, 0, now, false); - int32_t ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1); - stError("s-task:0x%x failed to load fill-history task, it may have been destroyed, not launch fill-history task", - (int32_t)pInfo->id.taskId); + streamMetaReleaseTask(pMeta, pTask); + } else { + streamMetaAddTaskLaunchResult(pMeta, pInfo->hTaskId.streamId, pInfo->hTaskId.taskId, 0, now, false); + + int32_t ref = atomic_sub_fetch_32(&(*ppTask)->status.timerActive, 1); + stError("s-task:0x%x rel fill-history task:0x%" PRIx64 " may have been destroyed, not launch, ref:%d", + (int32_t)pInfo->id.taskId, pInfo->hTaskId.taskId, ref); } taosMemoryFree(pInfo); From 98758862cf42d403b9f8bcc29729fc594d9acbcf Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sat, 13 Jan 2024 00:17:29 +0800 Subject: [PATCH 086/102] refactor: do some internal refactor. --- source/libs/stream/src/streamStart.c | 13 ++++++------- 1 file changed, 6 insertions(+), 7 deletions(-) diff --git a/source/libs/stream/src/streamStart.c b/source/libs/stream/src/streamStart.c index c8879e59c7..53c40e5b5c 100644 --- a/source/libs/stream/src/streamStart.c +++ b/source/libs/stream/src/streamStart.c @@ -866,7 +866,7 @@ SLaunchHTaskInfo* createHTaskLaunchInfo(SStreamMeta* pMeta, int64_t streamId, in return pInfo; } -static int32_t handleNotBuiltFillHistoryTask(SStreamTask* pTask) { +static int32_t launchNotBuiltFillHistoryTask(SStreamTask* pTask) { SStreamMeta* pMeta = pTask->pMeta; STaskExecStatisInfo* pExecInfo = &pTask->execInfo; const char* idStr = pTask->id.idStr; @@ -894,8 +894,7 @@ static int32_t handleNotBuiltFillHistoryTask(SStreamTask* pTask) { if (pTask->hTaskInfo.pTimer == NULL) { ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1); - stError("s-task:%s failed to start timer, related fill-history task not launched, ref:%d", idStr, - pTask->status.timerActive); + stError("s-task:%s failed to start timer, related fill-history task not launched, ref:%d", idStr, ref); taosMemoryFree(pInfo); streamMetaAddTaskLaunchResult(pMeta, hStreamId, hTaskId, pExecInfo->init, pExecInfo->start, false); @@ -940,10 +939,10 @@ int32_t streamLaunchFillHistoryTask(SStreamTask* pTask) { SStreamTask** pHTask = taosHashGet(pMeta->pTasksMap, &pTask->hTaskInfo.id, sizeof(pTask->hTaskInfo.id)); streamMetaRUnLock(pMeta); - if (pHTask != NULL) { + if (pHTask != NULL) { // it is already added into stream meta store. SStreamTask* pHisTask = streamMetaAcquireTask(pMeta, hStreamId, hTaskId); if (pHisTask == NULL) { - stDebug("s-task:%s failed acquire and start fill-history task, it may have been dropped", idStr); + stDebug("s-task:%s failed acquire and start fill-history task, it may have been dropped/stopped", idStr); streamMetaAddTaskLaunchResult(pMeta, hStreamId, hTaskId, pExecInfo->init, pExecInfo->start, false); } else { if (pHisTask->status.downstreamReady == 1) { // it's ready now, do nothing @@ -957,9 +956,9 @@ int32_t streamLaunchFillHistoryTask(SStreamTask* pTask) { } return TSDB_CODE_SUCCESS; + } else { + return launchNotBuiltFillHistoryTask(pTask); } - - return handleNotBuiltFillHistoryTask(pTask); } int32_t streamTaskScanHistoryDataComplete(SStreamTask* pTask) { From dc1a0821a7c06495e1dc91db5f1b095c8623cad3 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sat, 13 Jan 2024 00:19:46 +0800 Subject: [PATCH 087/102] refactor: do some internal refactor. --- source/libs/stream/src/streamStart.c | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/source/libs/stream/src/streamStart.c b/source/libs/stream/src/streamStart.c index 53c40e5b5c..84f1dbb4d7 100644 --- a/source/libs/stream/src/streamStart.c +++ b/source/libs/stream/src/streamStart.c @@ -828,12 +828,12 @@ static void tryLaunchHistoryTask(void* param, void* tmrId) { } else { checkFillhistoryTaskStatus(pTask, pHTask); streamMetaReleaseTask(pMeta, pHTask); - } - // not in timer anymore - 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); + // not in timer anymore + 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 c7518e0913b33820b173f542a8548f5e3f1a1921 Mon Sep 17 00:00:00 2001 From: Alex Duan <417921451@qq.com> Date: Sat, 13 Jan 2024 12:26:01 +0800 Subject: [PATCH 088/102] dnodes.py split dnode.py simClient.py --- tests/army/community/cmdline/taosd.py | 69 +++ tests/army/frame/caseBase.py | 5 + tests/army/frame/eos.py | 10 +- tests/army/frame/etool.py | 9 + tests/army/frame/server/dnode.py | 591 +++++++++++++++++++++++ tests/army/frame/server/dnodes.py | 656 +------------------------- tests/army/frame/server/simClient.py | 116 +++++ tests/army/frame/srvCtl.py | 9 + 8 files changed, 812 insertions(+), 653 deletions(-) create mode 100644 tests/army/community/cmdline/taosd.py create mode 100644 tests/army/frame/server/dnode.py create mode 100644 tests/army/frame/server/simClient.py diff --git a/tests/army/community/cmdline/taosd.py b/tests/army/community/cmdline/taosd.py new file mode 100644 index 0000000000..c361732dca --- /dev/null +++ b/tests/army/community/cmdline/taosd.py @@ -0,0 +1,69 @@ +################################################################### +# Copyright (c) 2016 by TAOS Technologies, Inc. +# All rights reserved. +# +# This file is proprietary and confidential to TAOS Technologies. +# No part of this file may be reproduced, stored, transmitted, +# disclosed or used in any form or by any means other than as +# expressly provided by the written permission from Jianhui Tao +# +################################################################### + +# -*- coding: utf-8 -*- + +import sys +import time +import random + +import taos +import frame + +from frame.log import * +from frame.sql import * +from frame.cases import * +from frame.caseBase import * +from frame.srvCtl import * +from frame import * + + +class TDTestCase(TBase): + def insertData(self): + tdLog.info(f"insert data.") + + tdSql.execute(f"use {self.db}") + # set insert data information + self.childtable_count = 10 + self.insert_rows = 10000 + self.timestamp_step = 1000 + + # taosBenchmark run + etool.runBenchmark(command = f"-d {self.db} -t {self.childtable_count} -n {self.insert_rows} -y") + + + def doAction(self): + tdLog.info(f"do action.") + + # dump out sdb + sdb = epath.binFile("sdb.json") + eos.delFile(sdb) + + cfg = sc.dnodeCfgPath(1) + etool.runBinFile("taosd", f"-s -c {cfg}") + self.checkFileExist(sdb) + + + # run + def run(self): + tdLog.debug(f"start to excute {__file__}") + + # insert data + self.insertData() + + # do action + self.doAction() + + tdLog.success(f"{__file__} successfully executed") + + +tdCases.addLinux(__file__, TDTestCase()) +tdCases.addWindows(__file__, TDTestCase()) diff --git a/tests/army/frame/caseBase.py b/tests/army/frame/caseBase.py index 8d2c1e6d18..535b7aeeb4 100644 --- a/tests/army/frame/caseBase.py +++ b/tests/army/frame/caseBase.py @@ -188,3 +188,8 @@ class TBase: time.sleep(interval) return False + + # check file exist + def checkFileExist(self, pathFile): + if os.path.exists(pathFile) == False: + tdLog.exit(f"file not exist {pathFile}") \ No newline at end of file diff --git a/tests/army/frame/eos.py b/tests/army/frame/eos.py index 1c00f94380..d0b37d1d10 100644 --- a/tests/army/frame/eos.py +++ b/tests/army/frame/eos.py @@ -73,6 +73,14 @@ def run(command): # return list after run def runRetList(command): - lines = [] output,error = run(command) return output.splitlines() + + + +# +# file +# + +def delFile(file): + return exe(f"rm -rf {file}") \ No newline at end of file diff --git a/tests/army/frame/etool.py b/tests/army/frame/etool.py index 35c390dc1a..d0c51ca39e 100644 --- a/tests/army/frame/etool.py +++ b/tests/army/frame/etool.py @@ -44,3 +44,12 @@ def runBenchmark(command = "", json = "") : # get current directory file name def curFile(fullPath, filename): return os.path.dirname(fullPath) + "/" + filename + + +# run build/bin file +def runBinFile(fname, command): + binFile = frame.epath.binFile(fname) + if frame.eos.isWin(): + binFile += ".exe" + + frame.eos.runRetList(f"{binFile} {command}") \ No newline at end of file diff --git a/tests/army/frame/server/dnode.py b/tests/army/frame/server/dnode.py new file mode 100644 index 0000000000..6331939636 --- /dev/null +++ b/tests/army/frame/server/dnode.py @@ -0,0 +1,591 @@ +################################################################### +# Copyright (c) 2016 by TAOS Technologies, Inc. +# All rights reserved. +# +# This file is proprietary and confidential to TAOS Technologies. +# No part of this file may be reproduced, stored, transmitted, +# disclosed or used in any form or by any means other than as +# expressly provided by the written permission from Jianhui Tao +# +################################################################### + +# -*- coding: utf-8 -*- + +import sys +import os +import os.path +import platform +import distro +import subprocess +from time import sleep +import base64 +import json +import copy +from fabric2 import Connection +from shutil import which + +# self +from frame.log import * + + +class TDDnode: + def __init__(self, index=1, level=1, disk=1): + self.index = index + self.level = level + self.disk = disk + self.dataDir = [] + self.running = 0 + self.deployed = 0 + self.testCluster = False + self.valgrind = 0 + self.asan = False + self.remoteIP = "" + self.cfgDict = { + "fqdn": "localhost", + "monitor": "0", + "maxShellConns": "30000", + "locale": "en_US.UTF-8", + "charset": "UTF-8", + "asyncLog": "0", + "mDebugFlag": "143", + "dDebugFlag": "143", + "vDebugFlag": "143", + "tqDebugFlag": "143", + "cDebugFlag": "143", + "stDebugFlag": "143", + "smaDebugFlag": "143", + "jniDebugFlag": "143", + "qDebugFlag": "143", + "rpcDebugFlag": "143", + "tmrDebugFlag": "131", + "uDebugFlag": "135", + "sDebugFlag": "135", + "wDebugFlag": "135", + "numOfLogLines": "100000000", + "statusInterval": "1", + "enableQueryHb": "1", + "supportVnodes": "1024", + "telemetryReporting": "0" + } + + def init(self, path, remoteIP = ""): + self.path = path + self.remoteIP = remoteIP + if (not self.remoteIP == ""): + try: + self.config = eval(self.remoteIP) + self.remote_conn = Connection(host=self.config["host"], port=self.config["port"], user=self.config["user"], connect_kwargs={'password':self.config["password"]}) + except Exception as r: + print(r) + + def setTestCluster(self, value): + self.testCluster = value + + def setValgrind(self, value): + self.valgrind = value + + def setAsan(self, value): + self.asan = value + if value: + selfPath = os.path.dirname(os.path.realpath(__file__)) + if ("community" in selfPath): + self.execPath = os.path.abspath(self.path + "/community/tests/script/sh/exec.sh") + else: + self.execPath = os.path.abspath(self.path + "/tests/script/sh/exec.sh") + + def getDataSize(self): + totalSize = 0 + + if (self.deployed == 1): + for dirpath, dirnames, filenames in os.walk(self.dataDir): + for f in filenames: + fp = os.path.join(dirpath, f) + + if not os.path.islink(fp): + totalSize = totalSize + os.path.getsize(fp) + + return totalSize + + def addExtraCfg(self, option, value): + self.cfgDict.update({option: value}) + + def remoteExec(self, updateCfgDict, execCmd): + valgrindStr = '' + if (self.valgrind==1): + valgrindStr = '-g' + remoteCfgDict = copy.deepcopy(updateCfgDict) + if ("logDir" in remoteCfgDict): + del remoteCfgDict["logDir"] + if ("dataDir" in remoteCfgDict): + del remoteCfgDict["dataDir"] + if ("cfgDir" in remoteCfgDict): + del remoteCfgDict["cfgDir"] + remoteCfgDictStr = base64.b64encode(json.dumps(remoteCfgDict).encode()).decode() + execCmdStr = base64.b64encode(execCmd.encode()).decode() + with self.remote_conn.cd((self.config["path"]+sys.path[0].replace(self.path, '')).replace('\\','/')): + self.remote_conn.run("python3 ./test.py %s -d %s -e %s"%(valgrindStr,remoteCfgDictStr,execCmdStr)) + + def deploy(self, *updatecfgDict): + # logDir + self.logDir = os.path.join(self.path,"sim","dnode%d" % self.index, "log") + # dataDir + simPath = os.path.join(self.path, "sim", "dnode%d" % self.index) + primary = 1 + if self.level == 1 and self.disk == 1: + eDir = os.path.join(simPath, "data") + self.dataDir.append(eDir) + else: + for i in range(self.level): + for j in range(self.disk): + eDir = os.path.join(simPath, f"data{i}{j}") + self.dataDir.append(f"{eDir} {i} {primary}") + if primary == 1: + primary = 0 + + # taos.cfg + self.cfgDir = os.path.join(self.path,"sim","dnode%d" % self.index, "cfg") + self.cfgPath = os.path.join(self.cfgDir, "taos.cfg") + + for eDir in self.dataDir: + cmd = "rm -rf " + eDir + if os.system(cmd) != 0: + tdLog.exit(cmd) + + cmd = "rm -rf " + self.logDir + if os.system(cmd) != 0: + tdLog.exit(cmd) + + cmd = "rm -rf " + self.cfgDir + if os.system(cmd) != 0: + tdLog.exit(cmd) + + # cmd = "mkdir -p " + self.dataDir + # if os.system(cmd) != 0: + # tdLog.exit(cmd) + for eDir in self.dataDir: + os.makedirs(eDir.split(' ')[0]) + + # cmd = "mkdir -p " + self.logDir + # if os.system(cmd) != 0: + # tdLog.exit(cmd) + os.makedirs(self.logDir) + + # cmd = "mkdir -p " + self.cfgDir + # if os.system(cmd) != 0: + # tdLog.exit(cmd) + os.makedirs(self.cfgDir) + + cmd = "touch " + self.cfgPath + if os.system(cmd) != 0: + tdLog.exit(cmd) + + if self.testCluster: + self.startIP() + + if self.testCluster: + self.cfg("masterIp", "192.168.0.1") + self.cfg("secondIp", "192.168.0.2") + self.cfg("publicIp", "192.168.0.%d" % (self.index)) + self.cfg("internalIp", "192.168.0.%d" % (self.index)) + self.cfg("privateIp", "192.168.0.%d" % (self.index)) + self.cfgDict["dataDir"] = self.dataDir + self.cfgDict["logDir"] = self.logDir + # self.cfg("dataDir",self.dataDir) + # self.cfg("logDir",self.logDir) + # print(updatecfgDict) + isFirstDir = 1 + if bool(updatecfgDict) and updatecfgDict[0] and updatecfgDict[0][0]: + for key, value in updatecfgDict[0][0].items(): + if key == "clientCfg" and self.remoteIP == "" and not platform.system().lower() == 'windows': + continue + if value == 'dataDir': + if isFirstDir: + self.cfgDict.pop('dataDir') + self.cfg(value, key) + isFirstDir = 0 + else: + self.cfg(value, key) + else: + self.addExtraCfg(key, value) + if (self.remoteIP == ""): + for key, value in self.cfgDict.items(): + if type(value) == list: + for v in value: + self.cfg(key, v) + else: + self.cfg(key, value) + else: + self.remoteExec(self.cfgDict, "tdDnodes.deploy(%d,updateCfgDict)"%self.index) + + self.deployed = 1 + tdLog.debug( + "dnode:%d is deployed and configured by %s" % + (self.index, self.cfgPath)) + + def getPath(self, tool="taosd"): + selfPath = os.path.dirname(os.path.realpath(__file__)) + + if ("community" in selfPath): + projPath = selfPath[:selfPath.find("community")] + else: + projPath = selfPath[:selfPath.find("tests")] + + paths = [] + for root, dirs, files in os.walk(projPath): + if ((tool) in files or ("%s.exe"%tool) in files): + rootRealPath = os.path.dirname(os.path.realpath(root)) + if ("packaging" not in rootRealPath): + paths.append(os.path.join(root, tool)) + break + if (len(paths) == 0): + return "" + return paths[0] + + def starttaosd(self): + binPath = self.getPath() + + if (binPath == ""): + tdLog.exit("taosd not found!") + else: + tdLog.info("taosd found: %s" % binPath) + + if self.deployed == 0: + tdLog.exit("dnode:%d is not deployed" % (self.index)) + + if self.valgrind == 0: + if platform.system().lower() == 'windows': + cmd = "mintty -h never %s -c %s" % ( + binPath, self.cfgDir) + else: + if self.asan: + asanDir = "%s/sim/asan/dnode%d.asan" % ( + self.path, self.index) + cmd = "nohup %s -c %s > /dev/null 2> %s & " % ( + binPath, self.cfgDir, asanDir) + else: + cmd = "nohup %s -c %s > /dev/null 2>&1 & " % ( + binPath, self.cfgDir) + else: + valgrindCmdline = "valgrind --log-file=\"%s/../log/valgrind.log\" --tool=memcheck --leak-check=full --show-reachable=no --track-origins=yes --show-leak-kinds=all -v --workaround-gcc296-bugs=yes"%self.cfgDir + + if platform.system().lower() == 'windows': + cmd = "mintty -h never %s %s -c %s" % ( + valgrindCmdline, binPath, self.cfgDir) + else: + cmd = "nohup %s %s -c %s 2>&1 & " % ( + valgrindCmdline, binPath, self.cfgDir) + + print(cmd) + + if (not self.remoteIP == ""): + self.remoteExec(self.cfgDict, "tdDnodes.dnodes[%d].deployed=1\ntdDnodes.dnodes[%d].logDir=\"%%s/sim/dnode%%d/log\"%%(tdDnodes.dnodes[%d].path,%d)\ntdDnodes.dnodes[%d].cfgDir=\"%%s/sim/dnode%%d/cfg\"%%(tdDnodes.dnodes[%d].path,%d)\ntdDnodes.start(%d)"%(self.index-1,self.index-1,self.index-1,self.index,self.index-1,self.index-1,self.index,self.index)) + self.running = 1 + else: + if os.system(cmd) != 0: + tdLog.exit(cmd) + self.running = 1 + tdLog.debug("dnode:%d is running with %s " % (self.index, cmd)) + if self.valgrind == 0: + time.sleep(0.1) + key1 = 'from offline to online' + bkey1 = bytes(key1, encoding="utf8") + key2= 'TDengine initialized successfully' + bkey2 = bytes(key2, encoding="utf8") + logFile = self.logDir + "/taosdlog.0" + i = 0 + # while not os.path.exists(logFile): + # sleep(0.1) + # i += 1 + # if i > 10: + # break + # tailCmdStr = 'tail -f ' + # if platform.system().lower() == 'windows': + # tailCmdStr = 'tail -n +0 -f ' + # popen = subprocess.Popen( + # tailCmdStr + logFile, + # stdout=subprocess.PIPE, + # stderr=subprocess.PIPE, + # shell=True) + # pid = popen.pid + # # print('Popen.pid:' + str(pid)) + # timeout = time.time() + 60 * 2 + # while True: + # line = popen.stdout.readline().strip() + # print(line) + # if bkey1 in line: + # popen.kill() + # break + # elif bkey2 in line: + # popen.kill() + # break + # if time.time() > timeout: + # print(time.time(),timeout) + # tdLog.exit('wait too long for taosd start') + tdLog.debug("the dnode:%d has been started." % (self.index)) + else: + tdLog.debug( + "wait 10 seconds for the dnode:%d to start." % + (self.index)) + time.sleep(10) + + def start(self): + binPath = self.getPath() + + if (binPath == ""): + tdLog.exit("taosd not found!") + else: + tdLog.info("taosd found: %s" % binPath) + + if self.deployed == 0: + tdLog.exit("dnode:%d is not deployed" % (self.index)) + + if self.valgrind == 0: + if platform.system().lower() == 'windows': + cmd = "mintty -h never %s -c %s" % ( + binPath, self.cfgDir) + else: + if self.asan: + asanDir = "%s/sim/asan/dnode%d.asan" % ( + self.path, self.index) + cmd = "nohup %s -c %s > /dev/null 2> %s & " % ( + binPath, self.cfgDir, asanDir) + else: + cmd = "nohup %s -c %s > /dev/null 2>&1 & " % ( + binPath, self.cfgDir) + else: + valgrindCmdline = "valgrind --log-file=\"%s/../log/valgrind.log\" --tool=memcheck --leak-check=full --show-reachable=no --track-origins=yes --show-leak-kinds=all -v --workaround-gcc296-bugs=yes"%self.cfgDir + + if platform.system().lower() == 'windows': + cmd = "mintty -h never %s %s -c %s" % ( + valgrindCmdline, binPath, self.cfgDir) + else: + cmd = "nohup %s %s -c %s 2>&1 & " % ( + valgrindCmdline, binPath, self.cfgDir) + + print(cmd) + + if (not self.remoteIP == ""): + self.remoteExec(self.cfgDict, "tdDnodes.dnodes[%d].deployed=1\ntdDnodes.dnodes[%d].logDir=\"%%s/sim/dnode%%d/log\"%%(tdDnodes.dnodes[%d].path,%d)\ntdDnodes.dnodes[%d].cfgDir=\"%%s/sim/dnode%%d/cfg\"%%(tdDnodes.dnodes[%d].path,%d)\ntdDnodes.start(%d)"%(self.index-1,self.index-1,self.index-1,self.index,self.index-1,self.index-1,self.index,self.index)) + self.running = 1 + else: + os.system("rm -rf %s/taosdlog.0"%self.logDir) + if os.system(cmd) != 0: + tdLog.exit(cmd) + self.running = 1 + tdLog.debug("dnode:%d is running with %s " % (self.index, cmd)) + if self.valgrind == 0: + time.sleep(0.1) + key = 'from offline to online' + bkey = bytes(key, encoding="utf8") + logFile = self.logDir + "/taosdlog.0" + i = 0 + while not os.path.exists(logFile): + sleep(0.1) + i += 1 + if i > 50: + break + with open(logFile) as f: + timeout = time.time() + 10 * 2 + while True: + line = f.readline().encode('utf-8') + if bkey in line: + break + if time.time() > timeout: + tdLog.exit('wait too long for taosd start') + tdLog.debug("the dnode:%d has been started." % (self.index)) + else: + tdLog.debug( + "wait 10 seconds for the dnode:%d to start." % + (self.index)) + time.sleep(10) + + def startWithoutSleep(self): + binPath = self.getPath() + + if (binPath == ""): + tdLog.exit("taosd not found!") + else: + tdLog.info("taosd found: %s" % binPath) + + if self.deployed == 0: + tdLog.exit("dnode:%d is not deployed" % (self.index)) + + if self.valgrind == 0: + if platform.system().lower() == 'windows': + cmd = "mintty -h never %s -c %s" % (binPath, self.cfgDir) + else: + if self.asan: + asanDir = "%s/sim/asan/dnode%d.asan" % ( + self.path, self.index) + cmd = "nohup %s -c %s > /dev/null 2> %s & " % ( + binPath, self.cfgDir, asanDir) + else: + cmd = "nohup %s -c %s > /dev/null 2>&1 & " % ( + binPath, self.cfgDir) + else: + valgrindCmdline = "valgrind --log-file=\"%s/../log/valgrind.log\" --tool=memcheck --leak-check=full --show-reachable=no --track-origins=yes --show-leak-kinds=all -v --workaround-gcc296-bugs=yes"%self.cfgDir + if platform.system().lower() == 'windows': + cmd = "mintty -h never %s %s -c %s" % ( + valgrindCmdline, binPath, self.cfgDir) + else: + cmd = "nohup %s %s -c %s 2>&1 & " % ( + valgrindCmdline, binPath, self.cfgDir) + print(cmd) + + if (self.remoteIP == ""): + if os.system(cmd) != 0: + tdLog.exit(cmd) + else: + self.remoteExec(self.cfgDict, "tdDnodes.dnodes[%d].deployed=1\ntdDnodes.dnodes[%d].logDir=\"%%s/sim/dnode%%d/log\"%%(tdDnodes.dnodes[%d].path,%d)\ntdDnodes.dnodes[%d].cfgDir=\"%%s/sim/dnode%%d/cfg\"%%(tdDnodes.dnodes[%d].path,%d)\ntdDnodes.startWithoutSleep(%d)"%(self.index-1,self.index-1,self.index-1,self.index,self.index-1,self.index-1,self.index,self.index)) + + self.running = 1 + tdLog.debug("dnode:%d is running with %s " % (self.index, cmd)) + + def stop(self): + if self.asan: + stopCmd = "%s -s stop -n dnode%d" % (self.execPath, self.index) + tdLog.info("execute script: " + stopCmd) + os.system(stopCmd) + return + + if (not self.remoteIP == ""): + self.remoteExec(self.cfgDict, "tdDnodes.dnodes[%d].running=1\ntdDnodes.dnodes[%d].stop()"%(self.index-1,self.index-1)) + tdLog.info("stop dnode%d"%self.index) + return + if self.valgrind == 0: + toBeKilled = "taosd" + else: + toBeKilled = "valgrind.bin" + + if self.running != 0: + psCmd = "ps -ef|grep -w %s| grep -v grep | awk '{print $2}' | xargs" % toBeKilled + processID = subprocess.check_output( + psCmd, shell=True).decode("utf-8").strip() + + onlyKillOnceWindows = 0 + while(processID): + if not platform.system().lower() == 'windows' or (onlyKillOnceWindows == 0 and platform.system().lower() == 'windows'): + killCmd = "kill -INT %s > /dev/null 2>&1" % processID + if platform.system().lower() == 'windows': + killCmd = "kill -INT %s > nul 2>&1" % processID + os.system(killCmd) + onlyKillOnceWindows = 1 + time.sleep(1) + processID = subprocess.check_output( + psCmd, shell=True).decode("utf-8").strip() + if not platform.system().lower() == 'windows': + for port in range(6030, 6041): + fuserCmd = "fuser -k -n tcp %d > /dev/null" % port + os.system(fuserCmd) + if self.valgrind: + time.sleep(2) + + self.running = 0 + tdLog.debug("dnode:%d is stopped by kill -INT" % (self.index)) + + + def stoptaosd(self): + tdLog.debug("start to stop taosd on dnode: %d "% (self.index)) + # print(self.asan,self.running,self.remoteIP,self.valgrind) + if self.asan: + stopCmd = "%s -s stop -n dnode%d" % (self.execPath, self.index) + tdLog.info("execute script: " + stopCmd) + os.system(stopCmd) + return + + if (not self.remoteIP == ""): + self.remoteExec(self.cfgDict, "tdDnodes.dnodes[%d].running=1\ntdDnodes.dnodes[%d].stop()"%(self.index-1,self.index-1)) + tdLog.info("stop dnode%d"%self.index) + return + if self.valgrind == 0: + toBeKilled = "taosd" + else: + toBeKilled = "valgrind.bin" + + if self.running != 0: + if platform.system().lower() == 'windows': + psCmd = "for /f %%a in ('wmic process where \"name='taosd.exe' and CommandLine like '%%dnode%d%%'\" get processId ^| xargs echo ^| awk ^'{print $2}^' ^&^& echo aa') do @(ps | grep %%a | awk '{print $1}' | xargs)" % (self.index) + else: + psCmd = "ps -ef|grep -w %s| grep dnode%d|grep -v grep | awk '{print $2}' | xargs" % (toBeKilled,self.index) + processID = subprocess.check_output( + psCmd, shell=True).decode("utf-8").strip() + + onlyKillOnceWindows = 0 + while(processID): + if not platform.system().lower() == 'windows' or (onlyKillOnceWindows == 0 and platform.system().lower() == 'windows'): + killCmd = "kill -INT %s > /dev/null 2>&1" % processID + if platform.system().lower() == 'windows': + killCmd = "kill -INT %s > nul 2>&1" % processID + os.system(killCmd) + onlyKillOnceWindows = 1 + time.sleep(1) + processID = subprocess.check_output( + psCmd, shell=True).decode("utf-8").strip() + if self.valgrind: + time.sleep(2) + + self.running = 0 + tdLog.debug("dnode:%d is stopped by kill -INT" % (self.index)) + + def forcestop(self): + if self.asan: + stopCmd = "%s -s stop -n dnode%d -x SIGKILL" + \ + (self.execPath, self.index) + tdLog.info("execute script: " + stopCmd) + os.system(stopCmd) + return + + if (not self.remoteIP == ""): + self.remoteExec(self.cfgDict, "tdDnodes.dnodes[%d].running=1\ntdDnodes.dnodes[%d].forcestop()"%(self.index-1,self.index-1)) + return + if self.valgrind == 0: + toBeKilled = "taosd" + else: + toBeKilled = "valgrind.bin" + + if self.running != 0: + psCmd = "ps -ef|grep -w %s| grep -v grep | awk '{print $2}' | xargs" % toBeKilled + processID = subprocess.check_output( + psCmd, shell=True).decode("utf-8").strip() + + onlyKillOnceWindows = 0 + while(processID): + if not platform.system().lower() == 'windows' or (onlyKillOnceWindows == 0 and platform.system().lower() == 'windows'): + killCmd = "kill -KILL %s > /dev/null 2>&1" % processID + os.system(killCmd) + onlyKillOnceWindows = 1 + time.sleep(1) + processID = subprocess.check_output( + psCmd, shell=True).decode("utf-8").strip() + for port in range(6030, 6041): + fuserCmd = "fuser -k -n tcp %d" % port + os.system(fuserCmd) + if self.valgrind: + time.sleep(2) + + self.running = 0 + tdLog.debug("dnode:%d is stopped by kill -KILL" % (self.index)) + + def startIP(self): + cmd = "sudo ifconfig lo:%d 192.168.0.%d up" % (self.index, self.index) + if os.system(cmd) != 0: + tdLog.exit(cmd) + + def stopIP(self): + cmd = "sudo ifconfig lo:%d 192.168.0.%d down" % ( + self.index, self.index) + if os.system(cmd) != 0: + tdLog.exit(cmd) + + def cfg(self, option, value): + cmd = "echo %s %s >> %s" % (option, value, self.cfgPath) + if os.system(cmd) != 0: + tdLog.exit(cmd) + + def getDnodeRootDir(self, index): + dnodeRootDir = os.path.join(self.path,"sim","psim","dnode%d" % index) + return dnodeRootDir + + def getDnodesRootDir(self): + dnodesRootDir = os.path.join(self.path,"sim","psim") + return dnodesRootDir \ No newline at end of file diff --git a/tests/army/frame/server/dnodes.py b/tests/army/frame/server/dnodes.py index 0d40b665dd..a22e1cbff7 100644 --- a/tests/army/frame/server/dnodes.py +++ b/tests/army/frame/server/dnodes.py @@ -22,660 +22,12 @@ import base64 import json import copy from fabric2 import Connection -from frame.log import * from shutil import which - -class TDSimClient: - def __init__(self, path): - self.testCluster = False - self.path = path - self.cfgDict = { - "fqdn": "localhost", - "numOfLogLines": "100000000", - "locale": "en_US.UTF-8", - "charset": "UTF-8", - "asyncLog": "0", - "rpcDebugFlag": "135", - "tmrDebugFlag": "131", - "cDebugFlag": "135", - "uDebugFlag": "135", - "jniDebugFlag": "135", - "qDebugFlag": "135", - "supportVnodes": "1024", - "enableQueryHb": "1", - "telemetryReporting": "0", - "tqDebugflag": "135", - "wDebugflag":"135", - } - - def getLogDir(self): - self.logDir = os.path.join(self.path,"sim","psim","log") - return self.logDir - - def getCfgDir(self): - self.cfgDir = os.path.join(self.path,"sim","psim","cfg") - return self.cfgDir - - def setTestCluster(self, value): - self.testCluster = value - - def addExtraCfg(self, option, value): - self.cfgDict.update({option: value}) - - def cfg(self, option, value): - cmd = "echo %s %s >> %s" % (option, value, self.cfgPath) - if os.system(cmd) != 0: - tdLog.exit(cmd) - - def deploy(self, *updatecfgDict): - self.logDir = os.path.join(self.path,"sim","psim","log") - self.cfgDir = os.path.join(self.path,"sim","psim","cfg") - self.cfgPath = os.path.join(self.path,"sim","psim","cfg","taos.cfg") - - cmd = "rm -rf " + self.logDir - if os.system(cmd) != 0: - tdLog.exit(cmd) - - # cmd = "mkdir -p " + self.logDir - # if os.system(cmd) != 0: - # tdLog.exit(cmd) - os.makedirs(self.logDir) - - cmd = "rm -rf " + self.cfgDir - if os.system(cmd) != 0: - tdLog.exit(cmd) - - # cmd = "mkdir -p " + self.cfgDir - # if os.system(cmd) != 0: - # tdLog.exit(cmd) - os.makedirs(self.cfgDir) - - cmd = "touch " + self.cfgPath - if os.system(cmd) != 0: - tdLog.exit(cmd) - - if self.testCluster: - self.cfg("masterIp", "192.168.0.1") - self.cfg("secondIp", "192.168.0.2") - self.cfg("logDir", self.logDir) - - for key, value in self.cfgDict.items(): - self.cfg(key, value) - - try: - if bool(updatecfgDict) and updatecfgDict[0] and updatecfgDict[0][0]: - clientCfg = dict (updatecfgDict[0][0].get('clientCfg')) - for key, value in clientCfg.items(): - self.cfg(key, value) - except Exception: - pass - - tdLog.debug("psim is deployed and configured by %s" % (self.cfgPath)) - - -class TDDnode: - def __init__(self, index=1, level=1, disk=1): - self.index = index - self.level = level - self.disk = disk - self.dataDir = [] - self.running = 0 - self.deployed = 0 - self.testCluster = False - self.valgrind = 0 - self.asan = False - self.remoteIP = "" - self.cfgDict = { - "fqdn": "localhost", - "monitor": "0", - "maxShellConns": "30000", - "locale": "en_US.UTF-8", - "charset": "UTF-8", - "asyncLog": "0", - "mDebugFlag": "143", - "dDebugFlag": "143", - "vDebugFlag": "143", - "tqDebugFlag": "143", - "cDebugFlag": "143", - "stDebugFlag": "143", - "smaDebugFlag": "143", - "jniDebugFlag": "143", - "qDebugFlag": "143", - "rpcDebugFlag": "143", - "tmrDebugFlag": "131", - "uDebugFlag": "135", - "sDebugFlag": "135", - "wDebugFlag": "135", - "numOfLogLines": "100000000", - "statusInterval": "1", - "enableQueryHb": "1", - "supportVnodes": "1024", - "telemetryReporting": "0" - } - - def init(self, path, remoteIP = ""): - self.path = path - self.remoteIP = remoteIP - if (not self.remoteIP == ""): - try: - self.config = eval(self.remoteIP) - self.remote_conn = Connection(host=self.config["host"], port=self.config["port"], user=self.config["user"], connect_kwargs={'password':self.config["password"]}) - except Exception as r: - print(r) - - def setTestCluster(self, value): - self.testCluster = value - - def setValgrind(self, value): - self.valgrind = value - - def setAsan(self, value): - self.asan = value - if value: - selfPath = os.path.dirname(os.path.realpath(__file__)) - if ("community" in selfPath): - self.execPath = os.path.abspath(self.path + "/community/tests/script/sh/exec.sh") - else: - self.execPath = os.path.abspath(self.path + "/tests/script/sh/exec.sh") - - def getDataSize(self): - totalSize = 0 - - if (self.deployed == 1): - for dirpath, dirnames, filenames in os.walk(self.dataDir): - for f in filenames: - fp = os.path.join(dirpath, f) - - if not os.path.islink(fp): - totalSize = totalSize + os.path.getsize(fp) - - return totalSize - - def addExtraCfg(self, option, value): - self.cfgDict.update({option: value}) - - def remoteExec(self, updateCfgDict, execCmd): - valgrindStr = '' - if (self.valgrind==1): - valgrindStr = '-g' - remoteCfgDict = copy.deepcopy(updateCfgDict) - if ("logDir" in remoteCfgDict): - del remoteCfgDict["logDir"] - if ("dataDir" in remoteCfgDict): - del remoteCfgDict["dataDir"] - if ("cfgDir" in remoteCfgDict): - del remoteCfgDict["cfgDir"] - remoteCfgDictStr = base64.b64encode(json.dumps(remoteCfgDict).encode()).decode() - execCmdStr = base64.b64encode(execCmd.encode()).decode() - with self.remote_conn.cd((self.config["path"]+sys.path[0].replace(self.path, '')).replace('\\','/')): - self.remote_conn.run("python3 ./test.py %s -d %s -e %s"%(valgrindStr,remoteCfgDictStr,execCmdStr)) - - def deploy(self, *updatecfgDict): - # logDir - self.logDir = os.path.join(self.path,"sim","dnode%d" % self.index, "log") - # dataDir - simPath = os.path.join(self.path, "sim", "dnode%d" % self.index) - primary = 1 - if self.level == 1 and self.disk == 1: - eDir = os.path.join(simPath, "data") - self.dataDir.append(eDir) - else: - for i in range(self.level): - for j in range(self.disk): - eDir = os.path.join(simPath, f"data{i}{j}") - self.dataDir.append(f"{eDir} {i} {primary}") - if primary == 1: - primary = 0 - - # taos.cfg - self.cfgDir = os.path.join(self.path,"sim","dnode%d" % self.index, "cfg") - self.cfgPath = os.path.join(self.path,"sim","dnode%d" % self.index, "cfg","taos.cfg") - - for eDir in self.dataDir: - cmd = "rm -rf " + eDir - if os.system(cmd) != 0: - tdLog.exit(cmd) - - cmd = "rm -rf " + self.logDir - if os.system(cmd) != 0: - tdLog.exit(cmd) - - cmd = "rm -rf " + self.cfgDir - if os.system(cmd) != 0: - tdLog.exit(cmd) - - # cmd = "mkdir -p " + self.dataDir - # if os.system(cmd) != 0: - # tdLog.exit(cmd) - for eDir in self.dataDir: - os.makedirs(eDir.split(' ')[0]) - - # cmd = "mkdir -p " + self.logDir - # if os.system(cmd) != 0: - # tdLog.exit(cmd) - os.makedirs(self.logDir) - - # cmd = "mkdir -p " + self.cfgDir - # if os.system(cmd) != 0: - # tdLog.exit(cmd) - os.makedirs(self.cfgDir) - - cmd = "touch " + self.cfgPath - if os.system(cmd) != 0: - tdLog.exit(cmd) - - if self.testCluster: - self.startIP() - - if self.testCluster: - self.cfg("masterIp", "192.168.0.1") - self.cfg("secondIp", "192.168.0.2") - self.cfg("publicIp", "192.168.0.%d" % (self.index)) - self.cfg("internalIp", "192.168.0.%d" % (self.index)) - self.cfg("privateIp", "192.168.0.%d" % (self.index)) - self.cfgDict["dataDir"] = self.dataDir - self.cfgDict["logDir"] = self.logDir - # self.cfg("dataDir",self.dataDir) - # self.cfg("logDir",self.logDir) - # print(updatecfgDict) - isFirstDir = 1 - if bool(updatecfgDict) and updatecfgDict[0] and updatecfgDict[0][0]: - for key, value in updatecfgDict[0][0].items(): - if key == "clientCfg" and self.remoteIP == "" and not platform.system().lower() == 'windows': - continue - if value == 'dataDir': - if isFirstDir: - self.cfgDict.pop('dataDir') - self.cfg(value, key) - isFirstDir = 0 - else: - self.cfg(value, key) - else: - self.addExtraCfg(key, value) - if (self.remoteIP == ""): - for key, value in self.cfgDict.items(): - if type(value) == list: - for v in value: - self.cfg(key, v) - else: - self.cfg(key, value) - else: - self.remoteExec(self.cfgDict, "tdDnodes.deploy(%d,updateCfgDict)"%self.index) - - self.deployed = 1 - tdLog.debug( - "dnode:%d is deployed and configured by %s" % - (self.index, self.cfgPath)) - - def getPath(self, tool="taosd"): - selfPath = os.path.dirname(os.path.realpath(__file__)) - - if ("community" in selfPath): - projPath = selfPath[:selfPath.find("community")] - else: - projPath = selfPath[:selfPath.find("tests")] - - paths = [] - for root, dirs, files in os.walk(projPath): - if ((tool) in files or ("%s.exe"%tool) in files): - rootRealPath = os.path.dirname(os.path.realpath(root)) - if ("packaging" not in rootRealPath): - paths.append(os.path.join(root, tool)) - break - if (len(paths) == 0): - return "" - return paths[0] - - def starttaosd(self): - binPath = self.getPath() - - if (binPath == ""): - tdLog.exit("taosd not found!") - else: - tdLog.info("taosd found: %s" % binPath) - - if self.deployed == 0: - tdLog.exit("dnode:%d is not deployed" % (self.index)) - - if self.valgrind == 0: - if platform.system().lower() == 'windows': - cmd = "mintty -h never %s -c %s" % ( - binPath, self.cfgDir) - else: - if self.asan: - asanDir = "%s/sim/asan/dnode%d.asan" % ( - self.path, self.index) - cmd = "nohup %s -c %s > /dev/null 2> %s & " % ( - binPath, self.cfgDir, asanDir) - else: - cmd = "nohup %s -c %s > /dev/null 2>&1 & " % ( - binPath, self.cfgDir) - else: - valgrindCmdline = "valgrind --log-file=\"%s/../log/valgrind.log\" --tool=memcheck --leak-check=full --show-reachable=no --track-origins=yes --show-leak-kinds=all -v --workaround-gcc296-bugs=yes"%self.cfgDir - - if platform.system().lower() == 'windows': - cmd = "mintty -h never %s %s -c %s" % ( - valgrindCmdline, binPath, self.cfgDir) - else: - cmd = "nohup %s %s -c %s 2>&1 & " % ( - valgrindCmdline, binPath, self.cfgDir) - - print(cmd) - - if (not self.remoteIP == ""): - self.remoteExec(self.cfgDict, "tdDnodes.dnodes[%d].deployed=1\ntdDnodes.dnodes[%d].logDir=\"%%s/sim/dnode%%d/log\"%%(tdDnodes.dnodes[%d].path,%d)\ntdDnodes.dnodes[%d].cfgDir=\"%%s/sim/dnode%%d/cfg\"%%(tdDnodes.dnodes[%d].path,%d)\ntdDnodes.start(%d)"%(self.index-1,self.index-1,self.index-1,self.index,self.index-1,self.index-1,self.index,self.index)) - self.running = 1 - else: - if os.system(cmd) != 0: - tdLog.exit(cmd) - self.running = 1 - tdLog.debug("dnode:%d is running with %s " % (self.index, cmd)) - if self.valgrind == 0: - time.sleep(0.1) - key1 = 'from offline to online' - bkey1 = bytes(key1, encoding="utf8") - key2= 'TDengine initialized successfully' - bkey2 = bytes(key2, encoding="utf8") - logFile = self.logDir + "/taosdlog.0" - i = 0 - # while not os.path.exists(logFile): - # sleep(0.1) - # i += 1 - # if i > 10: - # break - # tailCmdStr = 'tail -f ' - # if platform.system().lower() == 'windows': - # tailCmdStr = 'tail -n +0 -f ' - # popen = subprocess.Popen( - # tailCmdStr + logFile, - # stdout=subprocess.PIPE, - # stderr=subprocess.PIPE, - # shell=True) - # pid = popen.pid - # # print('Popen.pid:' + str(pid)) - # timeout = time.time() + 60 * 2 - # while True: - # line = popen.stdout.readline().strip() - # print(line) - # if bkey1 in line: - # popen.kill() - # break - # elif bkey2 in line: - # popen.kill() - # break - # if time.time() > timeout: - # print(time.time(),timeout) - # tdLog.exit('wait too long for taosd start') - tdLog.debug("the dnode:%d has been started." % (self.index)) - else: - tdLog.debug( - "wait 10 seconds for the dnode:%d to start." % - (self.index)) - time.sleep(10) - - def start(self): - binPath = self.getPath() - - if (binPath == ""): - tdLog.exit("taosd not found!") - else: - tdLog.info("taosd found: %s" % binPath) - - if self.deployed == 0: - tdLog.exit("dnode:%d is not deployed" % (self.index)) - - if self.valgrind == 0: - if platform.system().lower() == 'windows': - cmd = "mintty -h never %s -c %s" % ( - binPath, self.cfgDir) - else: - if self.asan: - asanDir = "%s/sim/asan/dnode%d.asan" % ( - self.path, self.index) - cmd = "nohup %s -c %s > /dev/null 2> %s & " % ( - binPath, self.cfgDir, asanDir) - else: - cmd = "nohup %s -c %s > /dev/null 2>&1 & " % ( - binPath, self.cfgDir) - else: - valgrindCmdline = "valgrind --log-file=\"%s/../log/valgrind.log\" --tool=memcheck --leak-check=full --show-reachable=no --track-origins=yes --show-leak-kinds=all -v --workaround-gcc296-bugs=yes"%self.cfgDir - - if platform.system().lower() == 'windows': - cmd = "mintty -h never %s %s -c %s" % ( - valgrindCmdline, binPath, self.cfgDir) - else: - cmd = "nohup %s %s -c %s 2>&1 & " % ( - valgrindCmdline, binPath, self.cfgDir) - - print(cmd) - - if (not self.remoteIP == ""): - self.remoteExec(self.cfgDict, "tdDnodes.dnodes[%d].deployed=1\ntdDnodes.dnodes[%d].logDir=\"%%s/sim/dnode%%d/log\"%%(tdDnodes.dnodes[%d].path,%d)\ntdDnodes.dnodes[%d].cfgDir=\"%%s/sim/dnode%%d/cfg\"%%(tdDnodes.dnodes[%d].path,%d)\ntdDnodes.start(%d)"%(self.index-1,self.index-1,self.index-1,self.index,self.index-1,self.index-1,self.index,self.index)) - self.running = 1 - else: - os.system("rm -rf %s/taosdlog.0"%self.logDir) - if os.system(cmd) != 0: - tdLog.exit(cmd) - self.running = 1 - tdLog.debug("dnode:%d is running with %s " % (self.index, cmd)) - if self.valgrind == 0: - time.sleep(0.1) - key = 'from offline to online' - bkey = bytes(key, encoding="utf8") - logFile = self.logDir + "/taosdlog.0" - i = 0 - while not os.path.exists(logFile): - sleep(0.1) - i += 1 - if i > 50: - break - with open(logFile) as f: - timeout = time.time() + 10 * 2 - while True: - line = f.readline().encode('utf-8') - if bkey in line: - break - if time.time() > timeout: - tdLog.exit('wait too long for taosd start') - tdLog.debug("the dnode:%d has been started." % (self.index)) - else: - tdLog.debug( - "wait 10 seconds for the dnode:%d to start." % - (self.index)) - time.sleep(10) - - def startWithoutSleep(self): - binPath = self.getPath() - - if (binPath == ""): - tdLog.exit("taosd not found!") - else: - tdLog.info("taosd found: %s" % binPath) - - if self.deployed == 0: - tdLog.exit("dnode:%d is not deployed" % (self.index)) - - if self.valgrind == 0: - if platform.system().lower() == 'windows': - cmd = "mintty -h never %s -c %s" % (binPath, self.cfgDir) - else: - if self.asan: - asanDir = "%s/sim/asan/dnode%d.asan" % ( - self.path, self.index) - cmd = "nohup %s -c %s > /dev/null 2> %s & " % ( - binPath, self.cfgDir, asanDir) - else: - cmd = "nohup %s -c %s > /dev/null 2>&1 & " % ( - binPath, self.cfgDir) - else: - valgrindCmdline = "valgrind --log-file=\"%s/../log/valgrind.log\" --tool=memcheck --leak-check=full --show-reachable=no --track-origins=yes --show-leak-kinds=all -v --workaround-gcc296-bugs=yes"%self.cfgDir - if platform.system().lower() == 'windows': - cmd = "mintty -h never %s %s -c %s" % ( - valgrindCmdline, binPath, self.cfgDir) - else: - cmd = "nohup %s %s -c %s 2>&1 & " % ( - valgrindCmdline, binPath, self.cfgDir) - print(cmd) - - if (self.remoteIP == ""): - if os.system(cmd) != 0: - tdLog.exit(cmd) - else: - self.remoteExec(self.cfgDict, "tdDnodes.dnodes[%d].deployed=1\ntdDnodes.dnodes[%d].logDir=\"%%s/sim/dnode%%d/log\"%%(tdDnodes.dnodes[%d].path,%d)\ntdDnodes.dnodes[%d].cfgDir=\"%%s/sim/dnode%%d/cfg\"%%(tdDnodes.dnodes[%d].path,%d)\ntdDnodes.startWithoutSleep(%d)"%(self.index-1,self.index-1,self.index-1,self.index,self.index-1,self.index-1,self.index,self.index)) - - self.running = 1 - tdLog.debug("dnode:%d is running with %s " % (self.index, cmd)) - - def stop(self): - if self.asan: - stopCmd = "%s -s stop -n dnode%d" % (self.execPath, self.index) - tdLog.info("execute script: " + stopCmd) - os.system(stopCmd) - return - - if (not self.remoteIP == ""): - self.remoteExec(self.cfgDict, "tdDnodes.dnodes[%d].running=1\ntdDnodes.dnodes[%d].stop()"%(self.index-1,self.index-1)) - tdLog.info("stop dnode%d"%self.index) - return - if self.valgrind == 0: - toBeKilled = "taosd" - else: - toBeKilled = "valgrind.bin" - - if self.running != 0: - psCmd = "ps -ef|grep -w %s| grep -v grep | awk '{print $2}' | xargs" % toBeKilled - processID = subprocess.check_output( - psCmd, shell=True).decode("utf-8").strip() - - onlyKillOnceWindows = 0 - while(processID): - if not platform.system().lower() == 'windows' or (onlyKillOnceWindows == 0 and platform.system().lower() == 'windows'): - killCmd = "kill -INT %s > /dev/null 2>&1" % processID - if platform.system().lower() == 'windows': - killCmd = "kill -INT %s > nul 2>&1" % processID - os.system(killCmd) - onlyKillOnceWindows = 1 - time.sleep(1) - processID = subprocess.check_output( - psCmd, shell=True).decode("utf-8").strip() - if not platform.system().lower() == 'windows': - for port in range(6030, 6041): - fuserCmd = "fuser -k -n tcp %d > /dev/null" % port - os.system(fuserCmd) - if self.valgrind: - time.sleep(2) - - self.running = 0 - tdLog.debug("dnode:%d is stopped by kill -INT" % (self.index)) - - - def stoptaosd(self): - tdLog.debug("start to stop taosd on dnode: %d "% (self.index)) - # print(self.asan,self.running,self.remoteIP,self.valgrind) - if self.asan: - stopCmd = "%s -s stop -n dnode%d" % (self.execPath, self.index) - tdLog.info("execute script: " + stopCmd) - os.system(stopCmd) - return - - if (not self.remoteIP == ""): - self.remoteExec(self.cfgDict, "tdDnodes.dnodes[%d].running=1\ntdDnodes.dnodes[%d].stop()"%(self.index-1,self.index-1)) - tdLog.info("stop dnode%d"%self.index) - return - if self.valgrind == 0: - toBeKilled = "taosd" - else: - toBeKilled = "valgrind.bin" - - if self.running != 0: - if platform.system().lower() == 'windows': - psCmd = "for /f %%a in ('wmic process where \"name='taosd.exe' and CommandLine like '%%dnode%d%%'\" get processId ^| xargs echo ^| awk ^'{print $2}^' ^&^& echo aa') do @(ps | grep %%a | awk '{print $1}' | xargs)" % (self.index) - else: - psCmd = "ps -ef|grep -w %s| grep dnode%d|grep -v grep | awk '{print $2}' | xargs" % (toBeKilled,self.index) - processID = subprocess.check_output( - psCmd, shell=True).decode("utf-8").strip() - - onlyKillOnceWindows = 0 - while(processID): - if not platform.system().lower() == 'windows' or (onlyKillOnceWindows == 0 and platform.system().lower() == 'windows'): - killCmd = "kill -INT %s > /dev/null 2>&1" % processID - if platform.system().lower() == 'windows': - killCmd = "kill -INT %s > nul 2>&1" % processID - os.system(killCmd) - onlyKillOnceWindows = 1 - time.sleep(1) - processID = subprocess.check_output( - psCmd, shell=True).decode("utf-8").strip() - if self.valgrind: - time.sleep(2) - - self.running = 0 - tdLog.debug("dnode:%d is stopped by kill -INT" % (self.index)) - - def forcestop(self): - if self.asan: - stopCmd = "%s -s stop -n dnode%d -x SIGKILL" + \ - (self.execPath, self.index) - tdLog.info("execute script: " + stopCmd) - os.system(stopCmd) - return - - if (not self.remoteIP == ""): - self.remoteExec(self.cfgDict, "tdDnodes.dnodes[%d].running=1\ntdDnodes.dnodes[%d].forcestop()"%(self.index-1,self.index-1)) - return - if self.valgrind == 0: - toBeKilled = "taosd" - else: - toBeKilled = "valgrind.bin" - - if self.running != 0: - psCmd = "ps -ef|grep -w %s| grep -v grep | awk '{print $2}' | xargs" % toBeKilled - processID = subprocess.check_output( - psCmd, shell=True).decode("utf-8").strip() - - onlyKillOnceWindows = 0 - while(processID): - if not platform.system().lower() == 'windows' or (onlyKillOnceWindows == 0 and platform.system().lower() == 'windows'): - killCmd = "kill -KILL %s > /dev/null 2>&1" % processID - os.system(killCmd) - onlyKillOnceWindows = 1 - time.sleep(1) - processID = subprocess.check_output( - psCmd, shell=True).decode("utf-8").strip() - for port in range(6030, 6041): - fuserCmd = "fuser -k -n tcp %d" % port - os.system(fuserCmd) - if self.valgrind: - time.sleep(2) - - self.running = 0 - tdLog.debug("dnode:%d is stopped by kill -KILL" % (self.index)) - - def startIP(self): - cmd = "sudo ifconfig lo:%d 192.168.0.%d up" % (self.index, self.index) - if os.system(cmd) != 0: - tdLog.exit(cmd) - - def stopIP(self): - cmd = "sudo ifconfig lo:%d 192.168.0.%d down" % ( - self.index, self.index) - if os.system(cmd) != 0: - tdLog.exit(cmd) - - def cfg(self, option, value): - cmd = "echo %s %s >> %s" % (option, value, self.cfgPath) - if os.system(cmd) != 0: - tdLog.exit(cmd) - - def getDnodeRootDir(self, index): - dnodeRootDir = os.path.join(self.path,"sim","psim","dnode%d" % index) - return dnodeRootDir - - def getDnodesRootDir(self): - dnodesRootDir = os.path.join(self.path,"sim","psim") - return dnodesRootDir - +# self +from frame.log import * +from frame.server.dnode import * +from frame.server.simClient import * class TDDnodes: def __init__(self): diff --git a/tests/army/frame/server/simClient.py b/tests/army/frame/server/simClient.py new file mode 100644 index 0000000000..a4b719f798 --- /dev/null +++ b/tests/army/frame/server/simClient.py @@ -0,0 +1,116 @@ +################################################################### +# Copyright (c) 2016 by TAOS Technologies, Inc. +# All rights reserved. +# +# This file is proprietary and confidential to TAOS Technologies. +# No part of this file may be reproduced, stored, transmitted, +# disclosed or used in any form or by any means other than as +# expressly provided by the written permission from Jianhui Tao +# +################################################################### + +# -*- coding: utf-8 -*- + +import sys +import os +import os.path +import platform +import distro +import subprocess +from time import sleep +import base64 +import json +import copy +from fabric2 import Connection +from shutil import which + +# self +from frame.log import * + + +class TDSimClient: + def __init__(self, path): + self.testCluster = False + self.path = path + self.cfgDict = { + "fqdn": "localhost", + "numOfLogLines": "100000000", + "locale": "en_US.UTF-8", + "charset": "UTF-8", + "asyncLog": "0", + "rpcDebugFlag": "135", + "tmrDebugFlag": "131", + "cDebugFlag": "135", + "uDebugFlag": "135", + "jniDebugFlag": "135", + "qDebugFlag": "135", + "supportVnodes": "1024", + "enableQueryHb": "1", + "telemetryReporting": "0", + "tqDebugflag": "135", + "wDebugflag":"135", + } + + def getLogDir(self): + self.logDir = os.path.join(self.path,"sim","psim","log") + return self.logDir + + def getCfgDir(self): + self.cfgDir = os.path.join(self.path,"sim","psim","cfg") + return self.cfgDir + + def setTestCluster(self, value): + self.testCluster = value + + def addExtraCfg(self, option, value): + self.cfgDict.update({option: value}) + + def cfg(self, option, value): + cmd = "echo %s %s >> %s" % (option, value, self.cfgPath) + if os.system(cmd) != 0: + tdLog.exit(cmd) + + def deploy(self, *updatecfgDict): + self.logDir = os.path.join(self.path,"sim","psim","log") + self.cfgDir = os.path.join(self.path,"sim","psim","cfg") + self.cfgPath = os.path.join(self.path,"sim","psim","cfg","taos.cfg") + + cmd = "rm -rf " + self.logDir + if os.system(cmd) != 0: + tdLog.exit(cmd) + + # cmd = "mkdir -p " + self.logDir + # if os.system(cmd) != 0: + # tdLog.exit(cmd) + os.makedirs(self.logDir) + + cmd = "rm -rf " + self.cfgDir + if os.system(cmd) != 0: + tdLog.exit(cmd) + + # cmd = "mkdir -p " + self.cfgDir + # if os.system(cmd) != 0: + # tdLog.exit(cmd) + os.makedirs(self.cfgDir) + + cmd = "touch " + self.cfgPath + if os.system(cmd) != 0: + tdLog.exit(cmd) + + if self.testCluster: + self.cfg("masterIp", "192.168.0.1") + self.cfg("secondIp", "192.168.0.2") + self.cfg("logDir", self.logDir) + + for key, value in self.cfgDict.items(): + self.cfg(key, value) + + try: + if bool(updatecfgDict) and updatecfgDict[0] and updatecfgDict[0][0]: + clientCfg = dict (updatecfgDict[0][0].get('clientCfg')) + for key, value in clientCfg.items(): + self.cfg(key, value) + except Exception: + pass + + tdLog.debug("psim is deployed and configured by %s" % (self.cfgPath)) \ No newline at end of file diff --git a/tests/army/frame/srvCtl.py b/tests/army/frame/srvCtl.py index 155c83094d..e708656b70 100644 --- a/tests/army/frame/srvCtl.py +++ b/tests/army/frame/srvCtl.py @@ -16,8 +16,10 @@ import os import time import datetime +from frame.server.dnode import * from frame.server.dnodes import * + class srvCtl: def __init__(self): # record server information @@ -38,7 +40,14 @@ class srvCtl: def dnodeDataFiles(self, idx): files = [] return files + + # + # get dnodes information + # + # taos.cfg position + def dnodeCfgPath(self, idx): + return tdDnodes.tdDnodes[idx-1].cfgPath sc = srvCtl() \ No newline at end of file From 782607fe68dc5c591c93722e1d5e6c0b175cc977 Mon Sep 17 00:00:00 2001 From: Alex Duan <417921451@qq.com> Date: Sat, 13 Jan 2024 13:31:40 +0800 Subject: [PATCH 089/102] add case fullopt.py --- tests/army/community/cmdline/{taosd.py => fullopt.py} | 5 ++--- tests/army/frame/srvCtl.py | 2 +- tests/parallel_test/cases.task | 1 + 3 files changed, 4 insertions(+), 4 deletions(-) rename tests/army/community/cmdline/{taosd.py => fullopt.py} (94%) diff --git a/tests/army/community/cmdline/taosd.py b/tests/army/community/cmdline/fullopt.py similarity index 94% rename from tests/army/community/cmdline/taosd.py rename to tests/army/community/cmdline/fullopt.py index c361732dca..8a69ae11c5 100644 --- a/tests/army/community/cmdline/taosd.py +++ b/tests/army/community/cmdline/fullopt.py @@ -16,7 +16,7 @@ import time import random import taos -import frame +import frame.etool from frame.log import * from frame.sql import * @@ -30,7 +30,6 @@ class TDTestCase(TBase): def insertData(self): tdLog.info(f"insert data.") - tdSql.execute(f"use {self.db}") # set insert data information self.childtable_count = 10 self.insert_rows = 10000 @@ -44,7 +43,7 @@ class TDTestCase(TBase): tdLog.info(f"do action.") # dump out sdb - sdb = epath.binFile("sdb.json") + sdb = "./sdb.json" eos.delFile(sdb) cfg = sc.dnodeCfgPath(1) diff --git a/tests/army/frame/srvCtl.py b/tests/army/frame/srvCtl.py index e708656b70..6c97d6bbc4 100644 --- a/tests/army/frame/srvCtl.py +++ b/tests/army/frame/srvCtl.py @@ -47,7 +47,7 @@ class srvCtl: # taos.cfg position def dnodeCfgPath(self, idx): - return tdDnodes.tdDnodes[idx-1].cfgPath + return tdDnodes.dnodes[idx-1].cfgPath sc = srvCtl() \ No newline at end of file diff --git a/tests/parallel_test/cases.task b/tests/parallel_test/cases.task index f19a0bda46..a3f35cbef0 100644 --- a/tests/parallel_test/cases.task +++ b/tests/parallel_test/cases.task @@ -11,6 +11,7 @@ ,,y,army,./pytest.sh python3 ./test.py -f enterprise/multi-level/mlevel_basic.py -N 3 -L 3 -D 2 ,,y,army,./pytest.sh python3 ./test.py -f enterprise/s3/s3_basic.py -L 3 -D 1 ,,y,army,./pytest.sh python3 ./test.py -f community/cluster/snapshot.py -N 3 -L 3 -D 2 +,,y,army,./pytest.sh python3 ./test.py -f community/cmdline/fullopt.py # From 7b29c97f27cadddfa7c3ebcec4eb3fca13ef7ea3 Mon Sep 17 00:00:00 2001 From: Alex Duan <417921451@qq.com> Date: Sat, 13 Jan 2024 14:50:17 +0800 Subject: [PATCH 090/102] feat: taosd full options do --- tests/army/community/cmdline/fullopt.py | 47 ++++++++++++++++++++----- tests/army/frame/caseBase.py | 10 ++++-- tests/army/frame/srvCtl.py | 13 +++++++ 3 files changed, 60 insertions(+), 10 deletions(-) diff --git a/tests/army/community/cmdline/fullopt.py b/tests/army/community/cmdline/fullopt.py index 8a69ae11c5..e0a0a51bfc 100644 --- a/tests/army/community/cmdline/fullopt.py +++ b/tests/army/community/cmdline/fullopt.py @@ -36,20 +36,48 @@ class TDTestCase(TBase): self.timestamp_step = 1000 # taosBenchmark run - etool.runBenchmark(command = f"-d {self.db} -t {self.childtable_count} -n {self.insert_rows} -y") + etool.runBenchmark(command = f"-d {self.db} -t {self.childtable_count} -n {self.insert_rows} -v 2 -y") - def doAction(self): - tdLog.info(f"do action.") + def doTaosd(self): + tdLog.info(f"check taosd command options...") + idx = 1 # dnode1 + cfg = sc.dnodeCfgPath(idx) - # dump out sdb + # -s sdb = "./sdb.json" eos.delFile(sdb) - - cfg = sc.dnodeCfgPath(1) etool.runBinFile("taosd", f"-s -c {cfg}") self.checkFileExist(sdb) - + + # -C + etool.runBinFile("taosd", "-C") + # -k + rets = etool.runBinFile("taosd", "-C") + self.checkListNotEmpty(rets) + # -V + rets = etool.runBinFile("taosd", "-V") + self.checkListNotEmpty(rets) + # --help + rets = etool.runBinFile("taosd", "--help") + self.checkListNotEmpty(rets) + + # except input + etool.runBinFile("taosd", "-c") + etool.runBinFile("taosd", "-e") + + # stop taosd + sc.dnodeStop(idx) + # other + etool.runBinFile("taosd", f"-dm -c {cfg}") + sc.dnodeStop(idx) + etool.runBinFile("taosd", "-a http://192.168.1.10") + etool.runBinFile("taosd", f"-E abc -c {cfg}") + etool.runBinFile("taosd", f"-e abc -c {cfg}") + + def doTaos(self): + tdLog.info(f"check taos command options...") + # run def run(self): @@ -59,7 +87,10 @@ class TDTestCase(TBase): self.insertData() # do action - self.doAction() + self.doTaosd() + + # do taos + self.doTaos() tdLog.success(f"{__file__} successfully executed") diff --git a/tests/army/frame/caseBase.py b/tests/army/frame/caseBase.py index 535b7aeeb4..c9f3aa1880 100644 --- a/tests/army/frame/caseBase.py +++ b/tests/army/frame/caseBase.py @@ -189,7 +189,13 @@ class TBase: return False - # check file exist + # check file exist def checkFileExist(self, pathFile): if os.path.exists(pathFile) == False: - tdLog.exit(f"file not exist {pathFile}") \ No newline at end of file + tdLog.exit(f"file not exist {pathFile}") + + # check list not exist + def checkListNotEmpty(self, lists, tips=""): + if len(lists) == 0: + tdLog.exit(f"list is empty {tips}") + diff --git a/tests/army/frame/srvCtl.py b/tests/army/frame/srvCtl.py index 6c97d6bbc4..c01ea33578 100644 --- a/tests/army/frame/srvCtl.py +++ b/tests/army/frame/srvCtl.py @@ -28,6 +28,19 @@ class srvCtl: self.mLevel = 0 self.mLevelDisk = 0 + # + # control server + # + + # start + def dnodeStart(self, idx): + return tdDnodes.starttaosd(idx) + + # stop + def dnodeStop(self, idx): + return tdDnodes.stoptaosd(idx) + + # # about path # From 34878b4f0d2a52c0de50e5fe00a17c17676b43ec Mon Sep 17 00:00:00 2001 From: Alex Duan <417921451@qq.com> Date: Sat, 13 Jan 2024 15:10:48 +0800 Subject: [PATCH 091/102] tweak run and exe method --- tests/army/community/cluster/snapshot.py | 2 +- tests/army/community/cmdline/fullopt.py | 20 +++++++++++-------- .../enterprise/multi-level/mlevel_basic.py | 2 +- tests/army/enterprise/s3/s3_basic.py | 2 +- tests/army/frame/eos.py | 8 ++++---- tests/army/frame/etool.py | 12 +++++++++-- 6 files changed, 29 insertions(+), 17 deletions(-) diff --git a/tests/army/community/cluster/snapshot.py b/tests/army/community/cluster/snapshot.py index b177ba3b29..92ecc00726 100644 --- a/tests/army/community/cluster/snapshot.py +++ b/tests/army/community/cluster/snapshot.py @@ -34,7 +34,7 @@ class TDTestCase(TBase): tdLog.info(f"insert data.") # taosBenchmark run jfile = etool.curFile(__file__, "snapshot.json") - etool.runBenchmark(json=jfile) + etool.benchMark(json=jfile) tdSql.execute(f"use {self.db}") # set insert data information diff --git a/tests/army/community/cmdline/fullopt.py b/tests/army/community/cmdline/fullopt.py index e0a0a51bfc..29711c7e0a 100644 --- a/tests/army/community/cmdline/fullopt.py +++ b/tests/army/community/cmdline/fullopt.py @@ -36,8 +36,11 @@ class TDTestCase(TBase): self.timestamp_step = 1000 # taosBenchmark run - etool.runBenchmark(command = f"-d {self.db} -t {self.childtable_count} -n {self.insert_rows} -v 2 -y") - + etool.benchMark(command = f"-d {self.db} -t {self.childtable_count} -n {self.insert_rows} -v 2 -y") + + def doTaos(self): + tdLog.info(f"check taos command options...") + def doTaosd(self): tdLog.info(f"check taosd command options...") @@ -72,11 +75,11 @@ class TDTestCase(TBase): etool.runBinFile("taosd", f"-dm -c {cfg}") sc.dnodeStop(idx) etool.runBinFile("taosd", "-a http://192.168.1.10") - etool.runBinFile("taosd", f"-E abc -c {cfg}") - etool.runBinFile("taosd", f"-e abc -c {cfg}") - def doTaos(self): - tdLog.info(f"check taos command options...") + #exe + etool.exeBinFile("taosd", f"-E abc -c {cfg}") + sc.dnodeStop(idx) + etool.exeBinFile("taosd", f"-e def -c {cfg}") # run @@ -86,11 +89,12 @@ class TDTestCase(TBase): # insert data self.insertData() + # do taos + self.doTaos() + # do action self.doTaosd() - # do taos - self.doTaos() tdLog.success(f"{__file__} successfully executed") diff --git a/tests/army/enterprise/multi-level/mlevel_basic.py b/tests/army/enterprise/multi-level/mlevel_basic.py index 3bec2bfb72..081a1bb9bd 100644 --- a/tests/army/enterprise/multi-level/mlevel_basic.py +++ b/tests/army/enterprise/multi-level/mlevel_basic.py @@ -32,7 +32,7 @@ class TDTestCase(TBase): tdLog.info(f"insert data.") # taosBenchmark run json = etool.curFile(__file__, "mlevel_basic.json") - etool.runBenchmark(json=json) + etool.benchMark(json=json) tdSql.execute(f"use {self.db}") # set insert data information diff --git a/tests/army/enterprise/s3/s3_basic.py b/tests/army/enterprise/s3/s3_basic.py index 8e7c1f2d35..f22d482502 100644 --- a/tests/army/enterprise/s3/s3_basic.py +++ b/tests/army/enterprise/s3/s3_basic.py @@ -54,7 +54,7 @@ class TDTestCase(TBase): tdLog.info(f"insert data.") # taosBenchmark run json = etool.curFile(__file__, "s3_basic.json") - etool.runBenchmark(json=json) + etool.benchMark(json=json) tdSql.execute(f"use {self.db}") # set insert data information diff --git a/tests/army/frame/eos.py b/tests/army/frame/eos.py index d0b37d1d10..defbaf4975 100644 --- a/tests/army/frame/eos.py +++ b/tests/army/frame/eos.py @@ -61,9 +61,9 @@ def exeNoWait(file): # run return output and error -def run(command): +def run(command, timeout = 10): process = subprocess.Popen(command, shell=True, stdout=subprocess.PIPE, stderr=subprocess.PIPE) - process.wait(3) + process.wait(timeout) output = process.stdout.read().decode(encoding="gbk") error = process.stderr.read().decode(encoding="gbk") @@ -72,8 +72,8 @@ def run(command): # return list after run -def runRetList(command): - output,error = run(command) +def runRetList(command, timeout=10): + output,error = run(command, timeout) return output.splitlines() diff --git a/tests/army/frame/etool.py b/tests/army/frame/etool.py index d0c51ca39e..6ab827d028 100644 --- a/tests/army/frame/etool.py +++ b/tests/army/frame/etool.py @@ -24,7 +24,7 @@ import frame.eos from frame.log import * # run taosBenchmark with command or json file mode -def runBenchmark(command = "", json = "") : +def benchMark(command = "", json = "") : # get taosBenchmark path bmFile = frame.epath.binFile("taosBenchmark") if frame.eos.isWin(): @@ -52,4 +52,12 @@ def runBinFile(fname, command): if frame.eos.isWin(): binFile += ".exe" - frame.eos.runRetList(f"{binFile} {command}") \ No newline at end of file + frame.eos.runRetList(f"{binFile} {command}") + +# exe build/bin file +def exeBinFile(fname, command): + binFile = frame.epath.binFile(fname) + if frame.eos.isWin(): + binFile += ".exe" + + frame.eos.exe(f"{binFile} {command}") \ No newline at end of file From 09f68ce80c8e705975fc0999ec7754e3dcd30373 Mon Sep 17 00:00:00 2001 From: Alex Duan <417921451@qq.com> Date: Sat, 13 Jan 2024 15:12:57 +0800 Subject: [PATCH 092/102] tweak run and exe method --- tests/army/frame/etool.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/army/frame/etool.py b/tests/army/frame/etool.py index 6ab827d028..2c22095ba7 100644 --- a/tests/army/frame/etool.py +++ b/tests/army/frame/etool.py @@ -60,4 +60,4 @@ def exeBinFile(fname, command): if frame.eos.isWin(): binFile += ".exe" - frame.eos.exe(f"{binFile} {command}") \ No newline at end of file + return frame.eos.exe(f"{binFile} {command}") \ No newline at end of file From 90b69d86eadc30f852a02f592bef280e0c7cf9a3 Mon Sep 17 00:00:00 2001 From: Alex Duan <417921451@qq.com> Date: Sat, 13 Jan 2024 15:31:32 +0800 Subject: [PATCH 093/102] fix: output list fixed --- tests/army/community/cmdline/fullopt.py | 8 ++++---- tests/army/frame/eos.py | 2 -- tests/army/frame/etool.py | 2 +- 3 files changed, 5 insertions(+), 7 deletions(-) diff --git a/tests/army/community/cmdline/fullopt.py b/tests/army/community/cmdline/fullopt.py index 29711c7e0a..dfaee22c2f 100644 --- a/tests/army/community/cmdline/fullopt.py +++ b/tests/army/community/cmdline/fullopt.py @@ -66,15 +66,15 @@ class TDTestCase(TBase): self.checkListNotEmpty(rets) # except input - etool.runBinFile("taosd", "-c") - etool.runBinFile("taosd", "-e") + etool.exeBinFile("taosd", "-c") + etool.exeBinFile("taosd", "-e") # stop taosd sc.dnodeStop(idx) # other - etool.runBinFile("taosd", f"-dm -c {cfg}") + etool.exeBinFile("taosd", f"-dm -c {cfg}") sc.dnodeStop(idx) - etool.runBinFile("taosd", "-a http://192.168.1.10") + etool.exeBinFile("taosd", "-a http://192.168.1.10") #exe etool.exeBinFile("taosd", f"-E abc -c {cfg}") diff --git a/tests/army/frame/eos.py b/tests/army/frame/eos.py index defbaf4975..bd84927c78 100644 --- a/tests/army/frame/eos.py +++ b/tests/army/frame/eos.py @@ -76,8 +76,6 @@ def runRetList(command, timeout=10): output,error = run(command, timeout) return output.splitlines() - - # # file # diff --git a/tests/army/frame/etool.py b/tests/army/frame/etool.py index 2c22095ba7..d57932b459 100644 --- a/tests/army/frame/etool.py +++ b/tests/army/frame/etool.py @@ -52,7 +52,7 @@ def runBinFile(fname, command): if frame.eos.isWin(): binFile += ".exe" - frame.eos.runRetList(f"{binFile} {command}") + return frame.eos.runRetList(f"{binFile} {command}") # exe build/bin file def exeBinFile(fname, command): From 5e47bc8e3a08e54b578c47a7dc8b672af1715700 Mon Sep 17 00:00:00 2001 From: Alex Duan <417921451@qq.com> Date: Sat, 13 Jan 2024 15:45:21 +0800 Subject: [PATCH 094/102] fix: add exeNoWait --- tests/army/community/cmdline/fullopt.py | 7 +++---- tests/army/frame/eos.py | 7 +++++-- tests/army/frame/etool.py | 7 +++++-- 3 files changed, 13 insertions(+), 8 deletions(-) diff --git a/tests/army/community/cmdline/fullopt.py b/tests/army/community/cmdline/fullopt.py index dfaee22c2f..f6baf5b004 100644 --- a/tests/army/community/cmdline/fullopt.py +++ b/tests/army/community/cmdline/fullopt.py @@ -72,15 +72,14 @@ class TDTestCase(TBase): # stop taosd sc.dnodeStop(idx) # other - etool.exeBinFile("taosd", f"-dm -c {cfg}") + etool.exeBinFile("taosd", f"-dm -c {cfg}", False) sc.dnodeStop(idx) etool.exeBinFile("taosd", "-a http://192.168.1.10") #exe - etool.exeBinFile("taosd", f"-E abc -c {cfg}") + etool.exeBinFile("taosd", f"-E abc -c {cfg}", False) sc.dnodeStop(idx) - etool.exeBinFile("taosd", f"-e def -c {cfg}") - + etool.exeBinFile("taosd", f"-e def -c {cfg}", False) # run def run(self): diff --git a/tests/army/frame/eos.py b/tests/army/frame/eos.py index bd84927c78..6bd8fd3aab 100644 --- a/tests/army/frame/eos.py +++ b/tests/army/frame/eos.py @@ -57,8 +57,11 @@ def exe(file): # execute file and return immediately def exeNoWait(file): - print("exe no wait") - + if isWin(): + cmd = f"mintty -h never {file}" + else: + cmd = f"nohup {file} > /dev/null 2>&1 & " + return exe(file) # run return output and error def run(command, timeout = 10): diff --git a/tests/army/frame/etool.py b/tests/army/frame/etool.py index d57932b459..c06afcd76e 100644 --- a/tests/army/frame/etool.py +++ b/tests/army/frame/etool.py @@ -55,9 +55,12 @@ def runBinFile(fname, command): return frame.eos.runRetList(f"{binFile} {command}") # exe build/bin file -def exeBinFile(fname, command): +def exeBinFile(fname, command, wait=True): binFile = frame.epath.binFile(fname) if frame.eos.isWin(): binFile += ".exe" - return frame.eos.exe(f"{binFile} {command}") \ No newline at end of file + if wait: + return frame.eos.exe(f"{binFile} {command}") + else: + return frame.eos.exeNoWait(f"{binFile} {command}") \ No newline at end of file From cc909d7d30f493bf473158d16718a00111635025 Mon Sep 17 00:00:00 2001 From: Alex Duan <417921451@qq.com> Date: Sat, 13 Jan 2024 17:07:09 +0800 Subject: [PATCH 095/102] fix: add taos command line test --- tests/army/community/cmdline/fullopt.py | 40 +++++++++++++++++++++++++ tests/army/frame/eos.py | 2 +- tests/parallel_test/cases.task | 2 +- 3 files changed, 42 insertions(+), 2 deletions(-) diff --git a/tests/army/community/cmdline/fullopt.py b/tests/army/community/cmdline/fullopt.py index f6baf5b004..70be4eb920 100644 --- a/tests/army/community/cmdline/fullopt.py +++ b/tests/army/community/cmdline/fullopt.py @@ -40,6 +40,40 @@ class TDTestCase(TBase): def doTaos(self): tdLog.info(f"check taos command options...") + # help + rets = etool.runBinFile("taos", "--help") + self.checkListNotEmpty(rets) + # b r w s + sql = f"select * from {self.db}.{self.stb} limit 10" + rets = etool.runBinFile("taos", f'-B -r -w -s "{sql}" ') + self.checkListNotEmpty(rets) + # -C + rets = etool.runBinFile("taos", "-C") + self.checkListNotEmpty(rets) + # -t + rets = etool.runBinFile("taos", "-t") + self.checkListNotEmpty(rets) + # -v + rets = etool.runBinFile("taos", "-V") + self.checkListNotEmpty(rets) + # -? + rets = etool.runBinFile("taos", "-?") + self.checkListNotEmpty(rets) + + # TSDB_FQDN_LEN = 128 + lname = "testhostnamelength" + lname.rjust(130, 'a') + + # invalid + sql = f"show vgroups;" + etool.exeBinFile("taos", f'-h {lname} -s "{sql}" ', wait=False) + etool.exeBinFile("taos", f'-u {lname} -s "{sql}" ', wait=False) + etool.exeBinFile("taos", f'-d {lname} -s "{sql}" ', wait=False) + etool.exeBinFile("taos", f'-a {lname} -s "{sql}" ', wait=False) + etool.exeBinFile("taos", f'-p{lname} -s "{sql}" ', wait=False) + + etool.exeBinFile("taos", f'-N 200 -l 2048 -s "{sql}" ', wait=False) + etool.exeBinFile("taos", f'-n server', wait=False) def doTaosd(self): @@ -81,6 +115,10 @@ class TDTestCase(TBase): sc.dnodeStop(idx) etool.exeBinFile("taosd", f"-e def -c {cfg}", False) + # stop taosd test taos as server + sc.dnodeStop(idx) + etool.exeBinFile("taos", f'-n server', wait=False) + # run def run(self): tdLog.debug(f"start to excute {__file__}") @@ -94,6 +132,8 @@ class TDTestCase(TBase): # do action self.doTaosd() + + tdLog.success(f"{__file__} successfully executed") diff --git a/tests/army/frame/eos.py b/tests/army/frame/eos.py index 6bd8fd3aab..802b62e052 100644 --- a/tests/army/frame/eos.py +++ b/tests/army/frame/eos.py @@ -61,7 +61,7 @@ def exeNoWait(file): cmd = f"mintty -h never {file}" else: cmd = f"nohup {file} > /dev/null 2>&1 & " - return exe(file) + return exe(cmd) # run return output and error def run(command, timeout = 10): diff --git a/tests/parallel_test/cases.task b/tests/parallel_test/cases.task index a3f35cbef0..119e0ed334 100644 --- a/tests/parallel_test/cases.task +++ b/tests/parallel_test/cases.task @@ -11,7 +11,7 @@ ,,y,army,./pytest.sh python3 ./test.py -f enterprise/multi-level/mlevel_basic.py -N 3 -L 3 -D 2 ,,y,army,./pytest.sh python3 ./test.py -f enterprise/s3/s3_basic.py -L 3 -D 1 ,,y,army,./pytest.sh python3 ./test.py -f community/cluster/snapshot.py -N 3 -L 3 -D 2 -,,y,army,./pytest.sh python3 ./test.py -f community/cmdline/fullopt.py +,,n,army,./pytest.sh python3 ./test.py -f community/cmdline/fullopt.py # From bdcd4732cc1734f16aad06375c5644e6c5e92e4b Mon Sep 17 00:00:00 2001 From: Alex Duan <417921451@qq.com> Date: Sat, 13 Jan 2024 17:21:48 +0800 Subject: [PATCH 096/102] fix: fullopt.py test ok --- tests/army/community/cmdline/fullopt.py | 6 ++++-- tests/army/frame/etool.py | 16 ++++++++++++---- 2 files changed, 16 insertions(+), 6 deletions(-) diff --git a/tests/army/community/cmdline/fullopt.py b/tests/army/community/cmdline/fullopt.py index 70be4eb920..22667149ae 100644 --- a/tests/army/community/cmdline/fullopt.py +++ b/tests/army/community/cmdline/fullopt.py @@ -45,7 +45,7 @@ class TDTestCase(TBase): self.checkListNotEmpty(rets) # b r w s sql = f"select * from {self.db}.{self.stb} limit 10" - rets = etool.runBinFile("taos", f'-B -r -w -s "{sql}" ') + rets = etool.runBinFile("taos", f'-B -r -w 100 -s "{sql}" ') self.checkListNotEmpty(rets) # -C rets = etool.runBinFile("taos", "-C") @@ -64,14 +64,16 @@ class TDTestCase(TBase): lname = "testhostnamelength" lname.rjust(130, 'a') - # invalid + # except test sql = f"show vgroups;" etool.exeBinFile("taos", f'-h {lname} -s "{sql}" ', wait=False) etool.exeBinFile("taos", f'-u {lname} -s "{sql}" ', wait=False) etool.exeBinFile("taos", f'-d {lname} -s "{sql}" ', wait=False) etool.exeBinFile("taos", f'-a {lname} -s "{sql}" ', wait=False) etool.exeBinFile("taos", f'-p{lname} -s "{sql}" ', wait=False) + etool.exeBinFile("taos", f'-w -s "{sql}" ', wait=False) + # others etool.exeBinFile("taos", f'-N 200 -l 2048 -s "{sql}" ', wait=False) etool.exeBinFile("taos", f'-n server', wait=False) diff --git a/tests/army/frame/etool.py b/tests/army/frame/etool.py index c06afcd76e..0650d4a0e2 100644 --- a/tests/army/frame/etool.py +++ b/tests/army/frame/etool.py @@ -47,20 +47,28 @@ def curFile(fullPath, filename): # run build/bin file -def runBinFile(fname, command): +def runBinFile(fname, command, show=True): binFile = frame.epath.binFile(fname) if frame.eos.isWin(): binFile += ".exe" - return frame.eos.runRetList(f"{binFile} {command}") + cmd = f"{binFile} {command}" + if show: + tdLog.info(cmd) + return frame.eos.runRetList(cmd) # exe build/bin file -def exeBinFile(fname, command, wait=True): +def exeBinFile(fname, command, wait=True, show=True): binFile = frame.epath.binFile(fname) if frame.eos.isWin(): binFile += ".exe" + cmd = f"{binFile} {command}" if wait: + if show: + tdLog.info("wait exe:" + cmd) return frame.eos.exe(f"{binFile} {command}") else: - return frame.eos.exeNoWait(f"{binFile} {command}") \ No newline at end of file + if show: + tdLog.info("no wait exe:" + cmd) + return frame.eos.exeNoWait(cmd) \ No newline at end of file From 9b94b7bf002ec31d1bf0060026379a37e77da557 Mon Sep 17 00:00:00 2001 From: Alex Duan <417921451@qq.com> Date: Sat, 13 Jan 2024 21:24:13 +0800 Subject: [PATCH 097/102] fix: remove asan check fullopt.py --- tests/parallel_test/cases.task | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/parallel_test/cases.task b/tests/parallel_test/cases.task index 119e0ed334..63745f75ab 100644 --- a/tests/parallel_test/cases.task +++ b/tests/parallel_test/cases.task @@ -11,7 +11,7 @@ ,,y,army,./pytest.sh python3 ./test.py -f enterprise/multi-level/mlevel_basic.py -N 3 -L 3 -D 2 ,,y,army,./pytest.sh python3 ./test.py -f enterprise/s3/s3_basic.py -L 3 -D 1 ,,y,army,./pytest.sh python3 ./test.py -f community/cluster/snapshot.py -N 3 -L 3 -D 2 -,,n,army,./pytest.sh python3 ./test.py -f community/cmdline/fullopt.py +,,n,army,python3 ./test.py -f community/cmdline/fullopt.py # From 8a2760e4d00c81adbaf7e42a187d2bc9a6fe713a Mon Sep 17 00:00:00 2001 From: Alex Duan <417921451@qq.com> Date: Sun, 14 Jan 2024 15:39:10 +0800 Subject: [PATCH 098/102] feat: s3 support stream check --- tests/army/enterprise/s3/s3_basic.py | 30 +++++++++++++++++++++++++++- tests/army/frame/caseBase.py | 3 +++ tests/army/frame/sql.py | 5 +++++ 3 files changed, 37 insertions(+), 1 deletion(-) diff --git a/tests/army/enterprise/s3/s3_basic.py b/tests/army/enterprise/s3/s3_basic.py index f22d482502..39ab64e85d 100644 --- a/tests/army/enterprise/s3/s3_basic.py +++ b/tests/army/enterprise/s3/s3_basic.py @@ -57,13 +57,18 @@ class TDTestCase(TBase): etool.benchMark(json=json) tdSql.execute(f"use {self.db}") - # set insert data information + # come from s3_basic.json self.childtable_count = 4 self.insert_rows = 1000000 self.timestamp_step = 1000 + def createStream(self, sname): + sql = f"create stream {sname} fill_history 1 into stm1 as select count(*) from {self.db}.{self.stb} interval(10s);" + tdSql.execute(sql) + def doAction(self): tdLog.info(f"do action.") + self.flushDb() self.compactDb() @@ -80,16 +85,33 @@ class TDTestCase(TBase): time.sleep(5) self.trimDb(True) loop += 1 + tdLog.info(f"loop={loop} wait 5s...") + + def checkStreamCorrect(self): + sql = f"select count(*) from {self.db}.stm1" + count = 0 + for i in range(30): + tdSql.query(sql) + count = tdSql.getData(0, 0) + if count == 100000 or count == 100001: + return True + time.sleep(1) + + tdLog.exit(f"stream count is not expect . expect = 100000 or 100001 real={count} . sql={sql}") # run def run(self): tdLog.debug(f"start to excute {__file__}") + self.sname = "stream1" if eos.isArm64Cpu(): tdLog.success(f"{__file__} arm64 ignore executed") else: # insert data self.insertData() + # creat stream + self.createStream(self.sname) + # check insert data correct self.checkInsertCorrect() @@ -105,6 +127,12 @@ class TDTestCase(TBase): # check insert correct again self.checkInsertCorrect() + # check stream correct and drop stream + self.checkStreamCorrect() + + # drop stream + self.dropStream(self.sname) + # drop database and free s3 file self.dropDb() diff --git a/tests/army/frame/caseBase.py b/tests/army/frame/caseBase.py index c9f3aa1880..ec6b36aa1b 100644 --- a/tests/army/frame/caseBase.py +++ b/tests/army/frame/caseBase.py @@ -72,6 +72,9 @@ class TBase: def dropDb(self, show = False): tdSql.execute(f"drop database {self.db}", show = show) + def dropStream(self, sname, show = False): + tdSql.execute(f"drop stream {sname}", show = show) + def splitVGroups(self): vgids = self.getVGroup(self.db) selid = random.choice(vgids) diff --git a/tests/army/frame/sql.py b/tests/army/frame/sql.py index 2e14f0c2f0..eafae9be2d 100644 --- a/tests/army/frame/sql.py +++ b/tests/army/frame/sql.py @@ -482,6 +482,11 @@ class TDSql: time.sleep(1) pass + # execute many sql + def executes(self, sqls, queryTimes=30, show=False): + for sql in sqls: + self.execute(sql, queryTimes, show) + def checkAffectedRows(self, expectAffectedRows): if self.affectedRows != expectAffectedRows: caller = inspect.getframeinfo(inspect.stack()[1][0]) From 3573eddd8709e50998a5b260ab612a59fec410b1 Mon Sep 17 00:00:00 2001 From: Alex Duan <417921451@qq.com> Date: Sun, 14 Jan 2024 16:42:59 +0800 Subject: [PATCH 099/102] fix: change delay 120s to wait stream result --- tests/army/enterprise/s3/s3_basic.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/army/enterprise/s3/s3_basic.py b/tests/army/enterprise/s3/s3_basic.py index 39ab64e85d..976ad85747 100644 --- a/tests/army/enterprise/s3/s3_basic.py +++ b/tests/army/enterprise/s3/s3_basic.py @@ -90,7 +90,7 @@ class TDTestCase(TBase): def checkStreamCorrect(self): sql = f"select count(*) from {self.db}.stm1" count = 0 - for i in range(30): + for i in range(120): tdSql.query(sql) count = tdSql.getData(0, 0) if count == 100000 or count == 100001: From ce855bc493dbf27fd07ffaf5a20da5f24d2ad0f0 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 15 Jan 2024 09:21:31 +0800 Subject: [PATCH 100/102] refactor: do some internal refactor. --- source/libs/stream/src/streamCheckpoint.c | 47 +++++++++++------------ 1 file changed, 22 insertions(+), 25 deletions(-) diff --git a/source/libs/stream/src/streamCheckpoint.c b/source/libs/stream/src/streamCheckpoint.c index 1c9361fa61..fb9d111124 100644 --- a/source/libs/stream/src/streamCheckpoint.c +++ b/source/libs/stream/src/streamCheckpoint.c @@ -294,11 +294,8 @@ int32_t streamSaveTaskCheckpointInfo(SStreamTask* p, int64_t checkpointId) { int32_t code = 0; SCheckpointInfo* pCKInfo = &p->chkInfo; - if (p->info.fillHistory == 1) { - return code; - } - - if (p->info.taskLevel > TASK_LEVEL__SINK) { + // fill-history task, rsma task, and sink task will not generate the checkpoint + if ((p->info.fillHistory == 1) || (p->info.taskLevel >= TASK_LEVEL__SINK)) { return code; } @@ -333,7 +330,7 @@ int32_t streamSaveTaskCheckpointInfo(SStreamTask* p, int64_t checkpointId) { vgId, id, p->info.taskLevel, checkpointId, pCKInfo->checkpointVer, pCKInfo->nextProcessVer, pStatus->name); // save the task if not sink task - if (p->info.taskLevel != TASK_LEVEL__SINK) { + if (p->info.taskLevel < TASK_LEVEL__SINK) { streamMetaWLock(pMeta); code = streamMetaSaveTask(pMeta, p); @@ -451,16 +448,17 @@ int32_t streamTaskUploadChkp(SStreamTask* pTask, int64_t chkpId, char* taskId) { return streamMetaAsyncExec(pTask->pMeta, doUploadChkp, arg, NULL); } int32_t streamTaskBuildCheckpoint(SStreamTask* pTask) { - int32_t code = TSDB_CODE_SUCCESS; - int64_t startTs = pTask->chkInfo.startTs; - int64_t ckId = pTask->chkInfo.checkpointingId; + int32_t code = TSDB_CODE_SUCCESS; + int64_t startTs = pTask->chkInfo.startTs; + int64_t ckId = pTask->chkInfo.checkpointingId; + const char* id = pTask->id.idStr; // sink task do not need to save the status, and generated the checkpoint if (pTask->info.taskLevel != TASK_LEVEL__SINK) { - stDebug("s-task:%s level:%d start gen checkpoint", pTask->id.idStr, pTask->info.taskLevel); + stDebug("s-task:%s level:%d start gen checkpoint", id, pTask->info.taskLevel); code = streamBackendDoCheckpoint(pTask->pBackend, ckId); if (code != TSDB_CODE_SUCCESS) { - stError("s-task:%s gen checkpoint:%" PRId64 " failed, code:%s", pTask->id.idStr, ckId, tstrerror(terrno)); + stError("s-task:%s gen checkpoint:%" PRId64 " failed, code:%s", id, ckId, tstrerror(terrno)); } } @@ -474,39 +472,38 @@ int32_t streamTaskBuildCheckpoint(SStreamTask* pTask) { if (code != TSDB_CODE_SUCCESS) { // todo: let's retry send rsp to upstream/mnode - stError("s-task:%s failed to send checkpoint rsp to upstream, checkpointId:%" PRId64 ", code:%s", pTask->id.idStr, - ckId, tstrerror(code)); + stError("s-task:%s failed to send checkpoint rsp to upstream, checkpointId:%" PRId64 ", code:%s", id, ckId, + tstrerror(code)); } } // clear the checkpoint info, and commit the newest checkpoint info if all works are done successfully if (code == TSDB_CODE_SUCCESS) { code = streamSaveTaskCheckpointInfo(pTask, ckId); - if (code != TSDB_CODE_SUCCESS) { - stError("s-task:%s commit taskInfo failed, checkpoint:%" PRId64 " failed, code:%s", pTask->id.idStr, ckId, - tstrerror(code)); - } else { - code = streamTaskUploadChkp(pTask, ckId, (char*)pTask->id.idStr); - if (code != 0) { - stError("s-task:%s failed to upload checkpoint:%" PRId64 " failed", pTask->id.idStr, ckId); + if (code == TSDB_CODE_SUCCESS) { + code = streamTaskUploadChkp(pTask, ckId, (char*)id); + if (code != TSDB_CODE_SUCCESS) { + stError("s-task:%s failed to upload checkpoint:%" PRId64 " failed", id, ckId); } + } else { + stError("s-task:%s commit taskInfo failed, checkpoint:%" PRId64 " failed, code:%s", id, ckId, tstrerror(code)); } } - if (code != TSDB_CODE_SUCCESS) { // clear the checkpoint info if failed + // clear the checkpoint info if failed + if (code != TSDB_CODE_SUCCESS) { taosThreadMutexLock(&pTask->lock); streamTaskClearCheckInfo(pTask, false); code = streamTaskHandleEvent(pTask->status.pSM, TASK_EVENT_CHECKPOINT_DONE); taosThreadMutexUnlock(&pTask->lock); streamTaskSetCheckpointFailedId(pTask); - stDebug("s-task:%s clear checkpoint flag since gen checkpoint failed, checkpointId:%" PRId64, pTask->id.idStr, - ckId); + stDebug("s-task:%s clear checkpoint flag since gen checkpoint failed, checkpointId:%" PRId64, id, ckId); } double el = (taosGetTimestampMs() - startTs) / 1000.0; - stInfo("s-task:%s vgId:%d level:%d, checkpointId:%" PRId64 " ver:%" PRId64 " elapsed time:%.2f Sec, %s ", - pTask->id.idStr, pTask->pMeta->vgId, pTask->info.taskLevel, ckId, pTask->chkInfo.checkpointVer, el, + stInfo("s-task:%s vgId:%d level:%d, checkpointId:%" PRId64 " ver:%" PRId64 " elapsed time:%.2f Sec, %s ", id, + pTask->pMeta->vgId, pTask->info.taskLevel, ckId, pTask->chkInfo.checkpointVer, el, (code == TSDB_CODE_SUCCESS) ? "succ" : "failed"); return code; From e058653e1d1b1820eaedeec254699c4502a61a6f Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 15 Jan 2024 09:45:04 +0800 Subject: [PATCH 101/102] fix(stream): adjust test cases. --- tests/system-test/8-stream/at_once_session.py | 4 ++++ tests/system-test/8-stream/max_delay_interval_ext.py | 2 ++ 2 files changed, 6 insertions(+) diff --git a/tests/system-test/8-stream/at_once_session.py b/tests/system-test/8-stream/at_once_session.py index 9a253a187f..3462561c4e 100644 --- a/tests/system-test/8-stream/at_once_session.py +++ b/tests/system-test/8-stream/at_once_session.py @@ -59,10 +59,14 @@ class TDTestCase: ctb_subtable_value = f'concat(concat("{self.ctb_name}_{self.tdCom.subtable_prefix}", cast(cast({partition_elm_alias} as bigint) as varchar(20))), "{self.tdCom.subtable_suffix}")' if self.tdCom.subtable else None tb_subtable_value = f'concat(concat("{self.tb_name}_{self.tdCom.subtable_prefix}", cast(cast({partition_elm_alias} as bigint) as varchar(20))), "{self.tdCom.subtable_suffix}")' if self.tdCom.subtable else None + time.sleep(1) # create stb/ctb/tb stream self.tdCom.create_stream(stream_name=f'{self.ctb_name}{self.tdCom.stream_suffix}', des_table=self.ctb_stream_des_table, source_sql=f'select _wstart AS wstart, _wend AS wend, {self.tdCom.stb_source_select_str} from {self.ctb_name} partition by {partition} {partition_elm_alias} session(ts, {self.tdCom.dataDict["session"]}s)', trigger_mode="at_once", ignore_expired=ignore_expired, ignore_update=ignore_update, 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.tb_stream_des_table, source_sql=f'select _wstart AS wstart, _wend AS wend, {self.tdCom.tb_source_select_str} from {self.tb_name} partition by {partition} {partition_elm_alias} session(ts, {self.tdCom.dataDict["session"]}s)', trigger_mode="at_once", ignore_expired=ignore_expired, ignore_update=ignore_update, subtable_value=tb_subtable_value, fill_history_value=fill_history_value) + + time.sleep(1) + for i in range(self.tdCom.range_count): ctb_name = self.tdCom.get_long_name() self.tdCom.screate_ctable(stbname=self.stb_name, ctbname=ctb_name) diff --git a/tests/system-test/8-stream/max_delay_interval_ext.py b/tests/system-test/8-stream/max_delay_interval_ext.py index 653fcd997c..6536309a25 100644 --- a/tests/system-test/8-stream/max_delay_interval_ext.py +++ b/tests/system-test/8-stream/max_delay_interval_ext.py @@ -50,6 +50,8 @@ class TDTestCase: # 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, subtable_value=stb_subtable_value, source_sql=f'select _wstart AS wstart, {partitial_tb_source_str} from {self.stb_name} interval({self.tdCom.dataDict["interval"]}s)', trigger_mode="max_delay", watermark=watermark_value, max_delay=max_delay_value, fill_value=fill_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) + init_num = 0 for i in range(self.tdCom.range_count): if i == 0: From d46b5e6c9a6be8676eeb48bb92c7abc72673869b Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 15 Jan 2024 09:53:15 +0800 Subject: [PATCH 102/102] fix(stream): enable sink tasks handle the checkpoint. --- source/libs/stream/src/streamCheckpoint.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/libs/stream/src/streamCheckpoint.c b/source/libs/stream/src/streamCheckpoint.c index fb9d111124..cb3f7a3504 100644 --- a/source/libs/stream/src/streamCheckpoint.c +++ b/source/libs/stream/src/streamCheckpoint.c @@ -295,7 +295,7 @@ int32_t streamSaveTaskCheckpointInfo(SStreamTask* p, int64_t checkpointId) { SCheckpointInfo* pCKInfo = &p->chkInfo; // fill-history task, rsma task, and sink task will not generate the checkpoint - if ((p->info.fillHistory == 1) || (p->info.taskLevel >= TASK_LEVEL__SINK)) { + if ((p->info.fillHistory == 1) || (p->info.taskLevel > TASK_LEVEL__SINK)) { return code; }