diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 5329da2f17..a98ad5a4c2 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -29,6 +29,21 @@ extern "C" { #ifndef _STREAM_H_ #define _STREAM_H_ +#define ONE_MiB_F (1048576.0) +#define ONE_KiB_F (1024.0) +#define SIZE_IN_MiB(_v) ((_v) / ONE_MiB_F) +#define SIZE_IN_KiB(_v) ((_v) / ONE_KiB_F) + + +#define TASK_DOWNSTREAM_READY 0x0 +#define TASK_DOWNSTREAM_NOT_READY 0x1 +#define TASK_DOWNSTREAM_NOT_LEADER 0x2 +#define TASK_SELF_NEW_STAGE 0x3 + +#define NODE_ROLE_UNINIT 0x1 +#define NODE_ROLE_LEADER 0x2 +#define NODE_ROLE_FOLLOWER 0x3 + typedef struct SStreamTask SStreamTask; #define SSTREAM_TASK_VER 2 @@ -64,6 +79,7 @@ enum { TASK_INPUT_STATUS__NORMAL = 1, TASK_INPUT_STATUS__BLOCKED, TASK_INPUT_STATUS__FAILED, + TASK_INPUT_STATUS__REFUSED, }; enum { @@ -197,7 +213,7 @@ typedef struct { int32_t taskId; int32_t nodeId; SEpSet epSet; -} STaskDispatcherFixedEp; +} STaskDispatcherFixed; typedef struct { char stbFullName[TSDB_TABLE_FNAME_LEN]; @@ -279,11 +295,15 @@ typedef struct SSTaskBasicInfo { int64_t triggerParam; // in msec } SSTaskBasicInfo; +typedef struct SStreamDispatchReq SStreamDispatchReq; + typedef struct SDispatchMsgInfo { - void* pData; // current dispatch data + SStreamDispatchReq* pData; // current dispatch data + int8_t dispatchMsgType; int16_t msgType; // dispatch msg type int32_t retryCount; // retry send data count - int64_t blockingTs; // output blocking timestamp + int64_t startTs; // dispatch start time, record total elapsed time for dispatch + SArray* pRetryList; // current dispatch successfully completed node of downstream } SDispatchMsgInfo; typedef struct STaskOutputInfo { @@ -302,22 +322,27 @@ typedef struct STaskSchedInfo { void* pTimer; } STaskSchedInfo; -typedef struct SSinkTaskRecorder { +typedef struct SSinkRecorder { int64_t numOfSubmit; int64_t numOfBlocks; int64_t numOfRows; -} SSinkTaskRecorder; + int64_t bytes; +} SSinkRecorder; -typedef struct { - int64_t created; - int64_t init; - int64_t step1Start; - int64_t step2Start; - int64_t start; - int32_t updateCount; - int64_t latestUpdateTs; +typedef struct STaskExecStatisInfo { + int64_t created; + int64_t init; + int64_t step1Start; + int64_t step2Start; + int64_t start; + int32_t updateCount; + int32_t dispatch; + int64_t latestUpdateTs; + int32_t checkpoint; + SSinkRecorder sink; } STaskExecStatisInfo; +typedef struct STaskTimer STaskTimer; typedef struct STokenBucket STokenBucket; typedef struct SMetaHbInfo SMetaHbInfo; @@ -335,23 +360,22 @@ struct SStreamTask { SDataRange dataRange; STaskId historyTaskId; STaskId streamTaskId; - STaskExecStatisInfo taskExecInfo; + STaskExecStatisInfo execInfo; SArray* pReadyMsgList; // SArray TdThreadMutex lock; // secure the operation of set task status and puting data into inputQ SArray* pUpstreamInfoList; // output union { - STaskDispatcherFixedEp fixedEpDispatcher; + STaskDispatcherFixed fixedDispatcher; STaskDispatcherShuffle shuffleDispatcher; STaskSinkTb tbSink; STaskSinkSma smaSink; STaskSinkFetch fetchSink; }; - SSinkTaskRecorder sinkRecorder; - STokenBucket* pTokenBucket; - void* launchTaskTimer; + STokenBucket* pTokenBucket; + STaskTimer* pTimer; SMsgCb* pMsgCb; // msg handle SStreamState* pState; // state backend SArray* pRspMsgList; @@ -371,6 +395,13 @@ struct SStreamTask { char reserve[256]; }; +typedef struct STaskStartInfo { + int64_t ts; + int32_t startedAfterNodeUpdate; + SHashObj* pReadyTaskSet; // tasks that are all ready for running stream processing + int32_t elapsedTime; +} STaskStartInfo; + // meta typedef struct SStreamMeta { char* path; @@ -384,8 +415,9 @@ typedef struct SStreamMeta { FTaskExpand* expandFunc; int32_t vgId; int64_t stage; - bool leader; - int8_t taskWillbeLaunched; +// bool leader; + int32_t role; + STaskStartInfo startInfo; SRWLatch lock; int32_t walScanCounter; void* streamBackend; @@ -430,11 +462,12 @@ typedef struct { int32_t taskId; } SStreamTaskRunReq; -typedef struct { +struct SStreamDispatchReq { int32_t type; int64_t stage; // nodeId from upstream task int64_t streamId; int32_t taskId; + int32_t msgId; // msg id to identify if the incoming msg from the same sender int32_t srcVgId; int32_t upstreamTaskId; int32_t upstreamChildId; @@ -443,7 +476,7 @@ typedef struct { int64_t totalLen; SArray* dataLen; // SArray SArray* data; // SArray -} SStreamDispatchReq; +}; typedef struct { int64_t streamId; @@ -451,7 +484,9 @@ typedef struct { int32_t upstreamTaskId; int32_t downstreamNodeId; int32_t downstreamTaskId; + int32_t msgId; int8_t inputStatus; + int64_t stage; } SStreamDispatchRsp; typedef struct { @@ -556,6 +591,8 @@ int32_t tDecodeStreamCheckpointReadyMsg(SDecoder* pDecoder, SStreamCheckpointRea typedef struct STaskStatusEntry { STaskId id; int32_t status; + int32_t stage; + int32_t nodeId; } STaskStatusEntry; typedef struct SStreamHbMsg { @@ -622,7 +659,7 @@ void tDeleteStreamDispatchReq(SStreamDispatchReq* pReq); int32_t streamSetupScheduleTrigger(SStreamTask* pTask); int32_t streamProcessRunReq(SStreamTask* pTask); -int32_t streamProcessDispatchMsg(SStreamTask* pTask, SStreamDispatchReq* pReq, SRpcMsg* pMsg, bool exec); +int32_t streamProcessDispatchMsg(SStreamTask* pTask, SStreamDispatchReq* pReq, SRpcMsg* pMsg); int32_t streamProcessDispatchRsp(SStreamTask* pTask, SStreamDispatchRsp* pRsp, int32_t code); int32_t streamProcessRetrieveReq(SStreamTask* pTask, SStreamRetrieveReq* pReq, SRpcMsg* pMsg); @@ -666,7 +703,6 @@ int32_t streamSetStatusUnint(SStreamTask* pTask); const char* streamGetTaskStatusStr(int32_t status); void streamTaskPause(SStreamTask* pTask, SStreamMeta* pMeta); void streamTaskResume(SStreamTask* pTask, SStreamMeta* pMeta); -void streamTaskHalt(SStreamTask* pTask); void streamTaskResumeFromHalt(SStreamTask* pTask); void streamTaskDisablePause(SStreamTask* pTask); void streamTaskEnablePause(SStreamTask* pTask); @@ -708,6 +744,7 @@ int32_t streamMetaCommit(SStreamMeta* pMeta); int32_t streamMetaLoadAllTasks(SStreamMeta* pMeta); void streamMetaNotifyClose(SStreamMeta* pMeta); void streamMetaStartHb(SStreamMeta* pMeta); +void streamMetaInitForSnode(SStreamMeta* pMeta); // checkpoint int32_t streamProcessCheckpointSourceReq(SStreamTask* pTask, SStreamCheckpointSourceReq* pReq); diff --git a/include/util/tlog.h b/include/util/tlog.h index 5a421033c9..a6d146a79e 100644 --- a/include/util/tlog.h +++ b/include/util/tlog.h @@ -55,6 +55,7 @@ extern int32_t tmrDebugFlag; extern int32_t uDebugFlag; extern int32_t rpcDebugFlag; extern int32_t qDebugFlag; +extern int32_t stDebugFlag; extern int32_t wDebugFlag; extern int32_t sDebugFlag; extern int32_t tsdbDebugFlag; diff --git a/source/client/test/clientTests.cpp b/source/client/test/clientTests.cpp index a1c8690dfc..6481dfe5bf 100644 --- a/source/client/test/clientTests.cpp +++ b/source/client/test/clientTests.cpp @@ -832,7 +832,7 @@ TEST(clientCase, projection_query_tables) { for(int32_t i = 0; i < 1000000; ++i) { char t[512] = {0}; - sprintf(t, "insert into t1 values(%ld, %ld)", start + i, i); + sprintf(t, "insert into t1 values(now, %ld)", i); while(1) { void* p = taos_query(pConn, t); code = taos_errno(p); diff --git a/source/common/src/systable.c b/source/common/src/systable.c index 66a498d15c..7107f0e058 100644 --- a/source/common/src/systable.c +++ b/source/common/src/systable.c @@ -158,11 +158,12 @@ static const SSysDbTableSchema streamSchema[] = { static const SSysDbTableSchema streamTaskSchema[] = { {.name = "stream_name", .bytes = SYSTABLE_SCH_DB_NAME_LEN, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = false}, - {.name = "task_id", .bytes = 32, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = false}, - {.name = "node_type", .bytes = SYSTABLE_SCH_DB_NAME_LEN, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = false}, + {.name = "task_id", .bytes = 16 + VARSTR_HEADER_SIZE, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = false}, + {.name = "node_type", .bytes = 10 + VARSTR_HEADER_SIZE, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = false}, {.name = "node_id", .bytes = 4, .type = TSDB_DATA_TYPE_INT, .sysInfo = false}, {.name = "level", .bytes = 20 + VARSTR_HEADER_SIZE, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = false}, {.name = "status", .bytes = 20 + VARSTR_HEADER_SIZE, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = false}, + {.name = "stage", .bytes = 4, .type = TSDB_DATA_TYPE_INT, .sysInfo = false}, }; static const SSysDbTableSchema userTblsSchema[] = { diff --git a/source/common/src/tglobal.c b/source/common/src/tglobal.c index 62d14a5cc6..ef35400cc0 100644 --- a/source/common/src/tglobal.c +++ b/source/common/src/tglobal.c @@ -241,7 +241,7 @@ int32_t tsTtlBatchDropNum = 10000; // number of tables dropped per batch int32_t tsTransPullupInterval = 2; int32_t tsMqRebalanceInterval = 2; int32_t tsStreamCheckpointTickInterval = 600; -int32_t tsStreamNodeCheckInterval = 10; +int32_t tsStreamNodeCheckInterval = 30; int32_t tsTtlUnit = 86400; int32_t tsTtlPushIntervalSec = 10; int32_t tsTrimVDbIntervalSec = 60 * 60; // interval of trimming db in all vgroups @@ -403,6 +403,7 @@ static int32_t taosAddServerLogCfg(SConfig *pCfg) { if (cfgAddInt32(pCfg, "idxDebugFlag", idxDebugFlag, 0, 255, CFG_SCOPE_SERVER) != 0) return -1; if (cfgAddInt32(pCfg, "tdbDebugFlag", tdbDebugFlag, 0, 255, CFG_SCOPE_SERVER) != 0) return -1; if (cfgAddInt32(pCfg, "metaDebugFlag", metaDebugFlag, 0, 255, 0) != CFG_SCOPE_SERVER) return -1; + if (cfgAddInt32(pCfg, "stDebugFlag", stDebugFlag, 0, 255, CFG_SCOPE_SERVER) != 0) return -1; return 0; } @@ -844,6 +845,7 @@ static void taosSetServerLogCfg(SConfig *pCfg) { idxDebugFlag = cfgGetItem(pCfg, "idxDebugFlag")->i32; tdbDebugFlag = cfgGetItem(pCfg, "tdbDebugFlag")->i32; metaDebugFlag = cfgGetItem(pCfg, "metaDebugFlag")->i32; + stDebugFlag = cfgGetItem(pCfg, "stDebugFlag")->i32; } static int32_t taosSetSlowLogScope(char *pScope) { @@ -1671,12 +1673,12 @@ void taosCfgDynamicOptions(const char *option, const char *value) { const char *options[] = { "dDebugFlag", "vDebugFlag", "mDebugFlag", "wDebugFlag", "sDebugFlag", "tsdbDebugFlag", "tqDebugFlag", "fsDebugFlag", "udfDebugFlag", "smaDebugFlag", "idxDebugFlag", "tdbDebugFlag", "tmrDebugFlag", "uDebugFlag", - "smaDebugFlag", "rpcDebugFlag", "qDebugFlag", "metaDebugFlag", "jniDebugFlag", + "smaDebugFlag", "rpcDebugFlag", "qDebugFlag", "metaDebugFlag", "jniDebugFlag", "stDebugFlag", }; int32_t *optionVars[] = { &dDebugFlag, &vDebugFlag, &mDebugFlag, &wDebugFlag, &sDebugFlag, &tsdbDebugFlag, &tqDebugFlag, &fsDebugFlag, &udfDebugFlag, &smaDebugFlag, &idxDebugFlag, &tdbDebugFlag, &tmrDebugFlag, &uDebugFlag, - &smaDebugFlag, &rpcDebugFlag, &qDebugFlag, &metaDebugFlag, &jniDebugFlag, + &smaDebugFlag, &rpcDebugFlag, &qDebugFlag, &metaDebugFlag, &jniDebugFlag, &stDebugFlag, }; int32_t optionSize = tListLen(options); @@ -1729,6 +1731,7 @@ void taosSetAllDebugFlag(int32_t flag, bool rewrite) { taosSetDebugFlag(&idxDebugFlag, "idxDebugFlag", flag, rewrite); taosSetDebugFlag(&tdbDebugFlag, "tdbDebugFlag", flag, rewrite); taosSetDebugFlag(&metaDebugFlag, "metaDebugFlag", flag, rewrite); + taosSetDebugFlag(&stDebugFlag, "stDebugFlag", flag, rewrite); uInfo("all debug flag are set to %d", flag); } diff --git a/source/dnode/mnode/impl/src/mndDnode.c b/source/dnode/mnode/impl/src/mndDnode.c index 5e2d520f7a..4871d88a57 100644 --- a/source/dnode/mnode/impl/src/mndDnode.c +++ b/source/dnode/mnode/impl/src/mndDnode.c @@ -1086,7 +1086,7 @@ static int32_t mndProcessConfigDnodeReq(SRpcMsg *pReq) { const char *options[] = { "debugFlag", "dDebugFlag", "vDebugFlag", "mDebugFlag", "wDebugFlag", "sDebugFlag", "tsdbDebugFlag", "tqDebugFlag", "fsDebugFlag", "udfDebugFlag", "smaDebugFlag", "idxDebugFlag", "tdbDebugFlag", "tmrDebugFlag", - "uDebugFlag", "smaDebugFlag", "rpcDebugFlag", "qDebugFlag", "metaDebugFlag", + "uDebugFlag", "smaDebugFlag", "rpcDebugFlag", "qDebugFlag", "metaDebugFlag", "stDebugFlag", }; int32_t optionSize = tListLen(options); diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index dd3a3ba500..dbd4362466 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -37,17 +37,18 @@ typedef struct SNodeEntry { int32_t nodeId; + bool stageUpdated; // the stage has been updated due to the leader/follower change or node reboot. SEpSet epset; // compare the epset to identify the vgroup tranferring between different dnodes. int64_t hbTimestamp; // second } SNodeEntry; -typedef struct SStreamVnodeRevertIndex { +typedef struct SStreamExecNodeInfo { SArray *pNodeEntryList; int64_t ts; // snapshot ts SHashObj *pTaskMap; SArray *pTaskList; TdThreadMutex lock; -} SStreamVnodeRevertIndex; +} SStreamExecNodeInfo; typedef struct SVgroupChangeInfo { SHashObj *pDBMap; @@ -55,7 +56,7 @@ typedef struct SVgroupChangeInfo { } SVgroupChangeInfo; static int32_t mndNodeCheckSentinel = 0; -static SStreamVnodeRevertIndex execNodeList; +static SStreamExecNodeInfo execNodeList; static int32_t mndStreamActionInsert(SSdb *pSdb, SStreamObj *pStream); static int32_t mndStreamActionDelete(SSdb *pSdb, SStreamObj *pStream); @@ -75,7 +76,6 @@ static int32_t mndBuildStreamCheckpointSourceReq2(void **pBuf, int32_t *pLen, in int64_t streamId, int32_t taskId); static int32_t mndProcessNodeCheck(SRpcMsg *pReq); static int32_t mndProcessNodeCheckReq(SRpcMsg *pMsg); -static void keepStreamTasksInBuf(SStreamObj *pStream, SStreamVnodeRevertIndex *pExecNode); static SArray *doExtractNodeListFromStream(SMnode *pMnode); static SArray *mndTakeVgroupSnapshot(SMnode *pMnode); @@ -83,8 +83,8 @@ static SVgroupChangeInfo mndFindChangedNodeInfo(SMnode *pMnode, const SArray *pP static int32_t mndPersistTransLog(SStreamObj *pStream, STrans *pTrans); static void initTransAction(STransAction *pAction, void *pCont, int32_t contLen, int32_t msgType, const SEpSet *pEpset); -static void removeStreamTasksInBuf(SStreamObj* pStream, SStreamVnodeRevertIndex* pExecNode); -static void keepStreamTasksInBuf(SStreamObj *pStream, SStreamVnodeRevertIndex *pExecNode); +static void removeStreamTasksInBuf(SStreamObj* pStream, SStreamExecNodeInfo * pExecNode); +static void keepStreamTasksInBuf(SStreamObj *pStream, SStreamExecNodeInfo *pExecNode); int32_t mndInitStream(SMnode *pMnode) { SSdbTable table = { @@ -1151,12 +1151,19 @@ static int32_t mndProcessStreamDoCheckpoint(SRpcMsg *pReq) { } if (taosArrayGetSize(execNodeList.pNodeEntryList) == 0) { - mDebug("end to do stream task node change checking, no vgroup exists, do nothing"); + mDebug("stream task node change checking done, no vgroups exist, do nothing"); execNodeList.ts = ts; - atomic_store_32(&mndNodeCheckSentinel, 0); return 0; } + for(int32_t i = 0; i < taosArrayGetSize(execNodeList.pNodeEntryList); ++i) { + SNodeEntry* pNodeEntry = taosArrayGet(execNodeList.pNodeEntryList, i); + if (pNodeEntry->stageUpdated) { + mDebug("stream task not ready due to node update detected, checkpoint not issued"); + return 0; + } + } + SArray *pNodeSnapshot = mndTakeVgroupSnapshot(pMnode); SVgroupChangeInfo changeInfo = mndFindChangedNodeInfo(pMnode, execNodeList.pNodeEntryList, pNodeSnapshot); @@ -1166,7 +1173,7 @@ static int32_t mndProcessStreamDoCheckpoint(SRpcMsg *pReq) { taosArrayDestroy(pNodeSnapshot); if (nodeUpdated) { - mDebug("stream task not ready due to node update, not generate checkpoint"); + mDebug("stream task not ready due to node update, checkpoint not issued"); return 0; } } @@ -1183,7 +1190,7 @@ static int32_t mndProcessStreamDoCheckpoint(SRpcMsg *pReq) { } if (pEntry->status != TASK_STATUS__NORMAL) { - mDebug("s-task:0x%" PRIx64 "-0x%x (nodeId:%d) status:%s not ready, create checkpoint msg not issued", + mDebug("s-task:0x%" PRIx64 "-0x%x (nodeId:%d) status:%s not ready, checkpoint msg not issued", pEntry->id.streamId, (int32_t)pEntry->id.taskId, 0, streamGetTaskStatusStr(pEntry->status)); ready = false; break; @@ -1560,7 +1567,7 @@ static int32_t mndRetrieveStreamTask(SRpcMsg *pReq, SShowObj *pShow, SSDataBlock colDataSetVal(pColInfo, numOfRows, (const char *)&level, false); // status - char status[20 + VARSTR_HEADER_SIZE] = {0}; + char status[20 + VARSTR_HEADER_SIZE] = {0}; STaskId id = {.streamId = pTask->id.streamId, .taskId = pTask->id.taskId}; STaskStatusEntry* pe = taosHashGet(execNodeList.pTaskMap, &id, sizeof(id)); @@ -1574,6 +1581,9 @@ static int32_t mndRetrieveStreamTask(SRpcMsg *pReq, SShowObj *pShow, SSDataBlock pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); colDataSetVal(pColInfo, numOfRows, (const char *)&status, false); + pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); + colDataSetVal(pColInfo, numOfRows, (const char *)&pe->stage, false); + numOfRows++; } } @@ -2029,7 +2039,7 @@ static SVgroupChangeInfo mndFindChangedNodeInfo(SMnode *pMnode, const SArray *pP SNodeEntry *pCurrent = taosArrayGet(pNodeList, j); if (pCurrent->nodeId == pPrevEntry->nodeId) { - if (isNodeEpsetChanged(&pPrevEntry->epset, &pCurrent->epset)) { + if (pPrevEntry->stageUpdated || isNodeEpsetChanged(&pPrevEntry->epset, &pCurrent->epset)) { const SEp *pPrevEp = GET_ACTIVE_EP(&pPrevEntry->epset); char buf[256] = {0}; @@ -2173,6 +2183,74 @@ static void doExtractTasksFromStream(SMnode *pMnode) { } } +static int32_t doRemoveFromTask(SStreamExecNodeInfo* pExecNode, STaskId* pRemovedId) { + void *p = taosHashGet(pExecNode->pTaskMap, pRemovedId, sizeof(*pRemovedId)); + + if (p != NULL) { + taosHashRemove(pExecNode->pTaskMap, pRemovedId, sizeof(*pRemovedId)); + + for(int32_t k = 0; k < taosArrayGetSize(pExecNode->pTaskList); ++k) { + STaskId* pId = taosArrayGet(pExecNode->pTaskList, k); + if (pId->taskId == pRemovedId->taskId && pId->streamId == pRemovedId->streamId) { + taosArrayRemove(pExecNode->pTaskList, k); + mInfo("s-task:0x%x removed from buffer, remain:%d", (int32_t) pRemovedId->taskId, + (int32_t)taosArrayGetSize(pExecNode->pTaskList)); + break; + } + } + } + return 0; +} + +static int32_t removeInvalidStreamTask(SArray *pNodeSnapshot) { + SArray* pRemoveTaskList = taosArrayInit(4, sizeof(STaskId)); + + int32_t numOfTask = taosArrayGetSize(execNodeList.pTaskList); + int32_t numOfVgroups = taosArrayGetSize(pNodeSnapshot); + for(int32_t i = 0; i < numOfTask; ++i) { + STaskId* pId = taosArrayGet(execNodeList.pTaskList, i); + STaskStatusEntry* pEntry = taosHashGet(execNodeList.pTaskMap, pId, sizeof(*pId)); + + bool existed = false; + for(int32_t j = 0; j < numOfVgroups; ++j) { + SNodeEntry* pNodeEntry = taosArrayGet(pNodeSnapshot, j); + if (pNodeEntry->nodeId == pEntry->nodeId) { + existed = true; + break; + } + } + + if (!existed) { + taosArrayPush(pRemoveTaskList, pId); + } + } + + for(int32_t i = 0; i < taosArrayGetSize(pRemoveTaskList); ++i) { + STaskId* pId = taosArrayGet(pRemoveTaskList, i); + doRemoveFromTask(&execNodeList, pId); + } + + int32_t size = taosArrayGetSize(pNodeSnapshot); + SArray* pValidNodeEntryList = taosArrayInit(4, sizeof(SNodeEntry)); + for(int32_t i = 0; i < taosArrayGetSize(execNodeList.pNodeEntryList); ++i) { + SNodeEntry* pExisted = taosArrayGet(execNodeList.pNodeEntryList, i); + + for(int32_t j = 0; j < size; ++j) { + SNodeEntry* pEntry = taosArrayGet(pNodeSnapshot, j); + if (pEntry->nodeId == pExisted->nodeId) { + taosArrayPush(pValidNodeEntryList, pExisted); + break; + } + } + } + + execNodeList.pNodeEntryList = taosArrayDestroy(execNodeList.pNodeEntryList); + execNodeList.pNodeEntryList = pValidNodeEntryList; + + taosArrayDestroy(pRemoveTaskList); + return 0; +} + // this function runs by only one thread, so it is not multi-thread safe static int32_t mndProcessNodeCheckReq(SRpcMsg *pMsg) { int32_t code = 0; @@ -2203,6 +2281,9 @@ static int32_t mndProcessNodeCheckReq(SRpcMsg *pMsg) { SArray *pNodeSnapshot = mndTakeVgroupSnapshot(pMnode); + taosThreadMutexLock(&execNodeList.lock); + removeInvalidStreamTask(pNodeSnapshot); + SVgroupChangeInfo changeInfo = mndFindChangedNodeInfo(pMnode, execNodeList.pNodeEntryList, pNodeSnapshot); if (taosArrayGetSize(changeInfo.pUpdateNodeList) > 0) { code = mndProcessVgroupChange(pMnode, &changeInfo); @@ -2219,6 +2300,7 @@ static int32_t mndProcessNodeCheckReq(SRpcMsg *pMsg) { taosArrayDestroy(pNodeSnapshot); } + taosThreadMutexUnlock(&execNodeList.lock); taosArrayDestroy(changeInfo.pUpdateNodeList); taosHashCleanup(changeInfo.pDBMap); @@ -2245,7 +2327,7 @@ static int32_t mndProcessNodeCheck(SRpcMsg *pReq) { return 0; } -void keepStreamTasksInBuf(SStreamObj *pStream, SStreamVnodeRevertIndex *pExecNode) { +void keepStreamTasksInBuf(SStreamObj *pStream, SStreamExecNodeInfo *pExecNode) { int32_t level = taosArrayGetSize(pStream->tasks); for (int32_t i = 0; i < level; i++) { SArray *pLevel = taosArrayGetP(pStream->tasks, i); @@ -2257,16 +2339,21 @@ void keepStreamTasksInBuf(SStreamObj *pStream, SStreamVnodeRevertIndex *pExecNod STaskId id = {.streamId = pTask->id.streamId, .taskId = pTask->id.taskId}; void *p = taosHashGet(pExecNode->pTaskMap, &id, sizeof(id)); if (p == NULL) { - STaskStatusEntry entry = { - .id.streamId = pTask->id.streamId, .id.taskId = pTask->id.taskId, .status = TASK_STATUS__STOP}; + STaskStatusEntry entry = {.id.streamId = pTask->id.streamId, + .id.taskId = pTask->id.taskId, + .stage = -1, + .nodeId = pTask->info.nodeId, + .status = TASK_STATUS__STOP}; taosHashPut(pExecNode->pTaskMap, &id, sizeof(id), &entry, sizeof(entry)); taosArrayPush(pExecNode->pTaskList, &id); + mInfo("s-task:0x%x add into task buffer, total:%d", (int32_t)entry.id.taskId, + (int32_t)taosArrayGetSize(pExecNode->pTaskList)); } } } } -void removeStreamTasksInBuf(SStreamObj* pStream, SStreamVnodeRevertIndex* pExecNode) { +void removeStreamTasksInBuf(SStreamObj* pStream, SStreamExecNodeInfo * pExecNode) { int32_t level = taosArrayGetSize(pStream->tasks); for (int32_t i = 0; i < level; i++) { SArray *pLevel = taosArrayGetP(pStream->tasks, i); @@ -2284,6 +2371,8 @@ void removeStreamTasksInBuf(SStreamObj* pStream, SStreamVnodeRevertIndex* pExecN STaskId* pId = taosArrayGet(pExecNode->pTaskList, k); if (pId->taskId == id.taskId && pId->streamId == id.streamId) { taosArrayRemove(pExecNode->pTaskList, k); + mInfo("s-task:0x%x removed from buffer, remain:%d", (int32_t)id.taskId, + (int32_t)taosArrayGetSize(pExecNode->pTaskList)); break; } } @@ -2297,9 +2386,8 @@ void removeStreamTasksInBuf(SStreamObj* pStream, SStreamVnodeRevertIndex* pExecN // todo: this process should be executed by the write queue worker of the mnode int32_t mndProcessStreamHb(SRpcMsg *pReq) { - SMnode *pMnode = pReq->info.node; + SMnode *pMnode = pReq->info.node; SStreamHbMsg req = {0}; - int32_t code = TSDB_CODE_SUCCESS; SDecoder decoder = {0}; tDecoderInit(&decoder, pReq->pCont, pReq->contLen); @@ -2327,11 +2415,29 @@ int32_t mndProcessStreamHb(SRpcMsg *pReq) { continue; } + if (p->stage != pEntry->stage && pEntry->stage != -1) { + int32_t numOfNodes = taosArrayGetSize(execNodeList.pNodeEntryList); + for(int32_t j = 0; j < numOfNodes; ++j) { + SNodeEntry* pNodeEntry = taosArrayGet(execNodeList.pNodeEntryList, j); + if (pNodeEntry->nodeId == pEntry->nodeId) { + mInfo("vgId:%d stage updated, from %d to %d, nodeUpdate trigger by s-task:0x%" PRIx64, + pEntry->nodeId, pEntry->stage, p->stage, pEntry->id.taskId); + + pNodeEntry->stageUpdated = true; + pEntry->stage = p->stage; + break; + } + } + } else { + pEntry->stage = p->stage; + } + pEntry->status = p->status; if (p->status != TASK_STATUS__NORMAL) { mDebug("received s-task:0x%"PRIx64" not in ready status:%s", p->id.taskId, streamGetTaskStatusStr(p->status)); } } + taosThreadMutexUnlock(&execNodeList.lock); taosArrayDestroy(req.pTaskStatus); diff --git a/source/dnode/snode/src/snode.c b/source/dnode/snode/src/snode.c index ef9c1ebe2e..dbbd68fa08 100644 --- a/source/dnode/snode/src/snode.c +++ b/source/dnode/snode/src/snode.c @@ -37,11 +37,8 @@ void sndEnqueueStreamDispatch(SSnode *pSnode, SRpcMsg *pMsg) { SStreamTask *pTask = streamMetaAcquireTask(pSnode->pMeta, req.streamId, req.taskId); if (pTask) { - SRpcMsg rsp = { - .info = pMsg->info, - .code = 0, - }; - streamProcessDispatchMsg(pTask, &req, &rsp, false); + SRpcMsg rsp = { .info = pMsg->info, .code = 0 }; + streamProcessDispatchMsg(pTask, &req, &rsp); streamMetaReleaseTask(pSnode->pMeta, pTask); rpcFreeCont(pMsg->pCont); taosFreeQitem(pMsg); @@ -115,18 +112,16 @@ SSnode *sndOpen(const char *path, const SSnodeOpt *pOption) { terrno = TSDB_CODE_OUT_OF_MEMORY; goto FAIL; } - pSnode->msgCb = pOption->msgCb; + pSnode->msgCb = pOption->msgCb; pSnode->pMeta = streamMetaOpen(path, pSnode, (FTaskExpand *)sndExpandTask, SNODE_HANDLE, -1); if (pSnode->pMeta == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; goto FAIL; } - // todo fix it: send msg to mnode to rollback to an existed checkpoint, and broadcast the rollback msg to all other - // computing nodes. - pSnode->pMeta->stage = 0; - + // todo fix it: send msg to mnode to rollback to an existed checkpoint + streamMetaInitForSnode(pSnode->pMeta); return pSnode; FAIL: @@ -228,7 +223,7 @@ int32_t sndProcessTaskDispatchReq(SSnode *pSnode, SRpcMsg *pMsg, bool exec) { SStreamTask *pTask = streamMetaAcquireTask(pSnode->pMeta, req.streamId, req.taskId); if (pTask) { SRpcMsg rsp = {.info = pMsg->info, .code = 0}; - streamProcessDispatchMsg(pTask, &req, &rsp, exec); + streamProcessDispatchMsg(pTask, &req, &rsp); streamMetaReleaseTask(pSnode->pMeta, pTask); return 0; } else { @@ -261,10 +256,11 @@ int32_t sndProcessTaskRetrieveReq(SSnode *pSnode, SRpcMsg *pMsg) { int32_t sndProcessTaskDispatchRsp(SSnode *pSnode, SRpcMsg *pMsg) { SStreamDispatchRsp *pRsp = POINTER_SHIFT(pMsg->pCont, sizeof(SMsgHead)); - int32_t taskId = htonl(pRsp->upstreamTaskId); - int64_t streamId = htobe64(pRsp->streamId); + pRsp->upstreamTaskId = htonl(pRsp->upstreamTaskId); + pRsp->streamId = htobe64(pRsp->streamId); + pRsp->msgId = htonl(pRsp->msgId); - SStreamTask *pTask = streamMetaAcquireTask(pSnode->pMeta, streamId, taskId); + SStreamTask *pTask = streamMetaAcquireTask(pSnode->pMeta, pRsp->streamId, pRsp->upstreamTaskId); if (pTask) { streamProcessDispatchRsp(pTask, pRsp, pMsg->code); streamMetaReleaseTask(pSnode->pMeta, pTask); @@ -361,7 +357,7 @@ int32_t sndProcessStreamTaskCheckReq(SSnode *pSnode, SRpcMsg *pMsg) { qDebug("s-task:%s status:%s, recv task check req(reqId:0x%" PRIx64 ") task:0x%x (vgId:%d), ready:%d", pTask->id.idStr, pStatus, rsp.reqId, rsp.upstreamTaskId, rsp.upstreamNodeId, rsp.status); } else { - rsp.status = 0; + rsp.status = TASK_DOWNSTREAM_NOT_READY; qDebug("recv task check(taskId:0x%x not built yet) req(reqId:0x%" PRIx64 ") from task:0x%x (vgId:%d), rsp status %d", taskId, rsp.reqId, rsp.upstreamTaskId, rsp.upstreamNodeId, rsp.status); } diff --git a/source/dnode/vnode/src/inc/tq.h b/source/dnode/vnode/src/inc/tq.h index 4e73a481c8..7f91aed4eb 100644 --- a/source/dnode/vnode/src/inc/tq.h +++ b/source/dnode/vnode/src/inc/tq.h @@ -39,7 +39,6 @@ extern "C" { #define tqInfo(...) do { if (tqDebugFlag & DEBUG_INFO) { taosPrintLog("TQ ", DEBUG_INFO, 255, __VA_ARGS__); }} while(0) #define tqDebug(...) do { if (tqDebugFlag & DEBUG_DEBUG) { taosPrintLog("TQ ", DEBUG_DEBUG, tqDebugFlag, __VA_ARGS__); }} while(0) #define tqTrace(...) do { if (tqDebugFlag & DEBUG_TRACE) { taosPrintLog("TQ ", DEBUG_TRACE, tqDebugFlag, __VA_ARGS__); }} while(0) - // clang-format on typedef struct STqOffsetStore STqOffsetStore; diff --git a/source/dnode/vnode/src/inc/vnodeInt.h b/source/dnode/vnode/src/inc/vnodeInt.h index 39f3d465f2..3a62f52bdd 100644 --- a/source/dnode/vnode/src/inc/vnodeInt.h +++ b/source/dnode/vnode/src/inc/vnodeInt.h @@ -224,8 +224,8 @@ int tqPushMsg(STQ*, tmsg_t msgType); int tqRegisterPushHandle(STQ* pTq, void* handle, SRpcMsg* pMsg); int tqUnregisterPushHandle(STQ* pTq, void* pHandle); int tqScanWalAsync(STQ* pTq, bool ckPause); -int32_t tqProcessStreamCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg); -int32_t tqProcessStreamTaskCheckpointReadyMsg(STQ* pTq, SRpcMsg* pMsg); +int32_t tqProcessStreamCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg* pRsp); +int32_t tqProcessTaskCheckpointReadyMsg(STQ* pTq, SRpcMsg* pMsg); int32_t tqProcessTaskUpdateReq(STQ* pTq, SRpcMsg* pMsg); int32_t tqCheckAndRunStreamTaskAsync(STQ* pTq); @@ -249,8 +249,8 @@ int32_t tqProcessTaskDeployReq(STQ* pTq, int64_t version, char* msg, int32_t msg int32_t tqProcessTaskDropReq(STQ* pTq, int64_t version, char* msg, int32_t msgLen); int32_t tqProcessTaskPauseReq(STQ* pTq, int64_t version, char* msg, int32_t msgLen); int32_t tqProcessTaskResumeReq(STQ* pTq, int64_t version, char* msg, int32_t msgLen); -int32_t tqProcessStreamTaskCheckReq(STQ* pTq, SRpcMsg* pMsg); -int32_t tqProcessStreamTaskCheckRsp(STQ* pTq, SRpcMsg* pMsg); +int32_t tqProcessTaskCheckReq(STQ* pTq, SRpcMsg* pMsg); +int32_t tqProcessTaskCheckRsp(STQ* pTq, SRpcMsg* pMsg); int32_t tqProcessTaskRunReq(STQ* pTq, SRpcMsg* pMsg); int32_t tqProcessTaskDispatchReq(STQ* pTq, SRpcMsg* pMsg, bool exec); int32_t tqProcessTaskDispatchRsp(STQ* pTq, SRpcMsg* pMsg); diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index a516224413..dada1695ef 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -882,7 +882,7 @@ int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t ver) { return 0; } -int32_t tqProcessStreamTaskCheckReq(STQ* pTq, SRpcMsg* pMsg) { +int32_t tqProcessTaskCheckReq(STQ* pTq, SRpcMsg* pMsg) { char* msgStr = pMsg->pCont; char* msgBody = POINTER_SHIFT(msgStr, sizeof(SMsgHead)); int32_t msgLen = pMsg->contLen - sizeof(SMsgHead); @@ -908,31 +908,31 @@ int32_t tqProcessStreamTaskCheckReq(STQ* pTq, SRpcMsg* pMsg) { }; // only the leader node handle the check request - if (!pMeta->leader) { - tqError("s-task:0x%x invalid check msg from upstream:0x%x(vgId:%d), vgId:%d is follower, not handle check msg", + if (pMeta->role == NODE_ROLE_FOLLOWER) { + tqError("s-task:0x%x invalid check msg from upstream:0x%x(vgId:%d), vgId:%d is follower, not handle check status msg", taskId, req.upstreamTaskId, req.upstreamNodeId, pMeta->vgId); - return -1; - } - - SStreamTask* pTask = streamMetaAcquireTask(pMeta, req.streamId, taskId); - if (pTask != NULL) { - rsp.status = streamTaskCheckStatus(pTask, req.upstreamTaskId, req.upstreamNodeId, req.stage); - streamMetaReleaseTask(pMeta, pTask); - - const char* pStatus = streamGetTaskStatusStr(pTask->status.taskStatus); - tqDebug("s-task:%s status:%s, stage:%d recv task check req(reqId:0x%" PRIx64 ") task:0x%x (vgId:%d), ready:%d", - pTask->id.idStr, pStatus, rsp.oldStage, rsp.reqId, rsp.upstreamTaskId, rsp.upstreamNodeId, rsp.status); + rsp.status = TASK_DOWNSTREAM_NOT_LEADER; } else { - rsp.status = 0; - tqDebug("tq recv task check(taskId:0x%" PRIx64 "-0x%x not built yet) req(reqId:0x%" PRIx64 - ") from task:0x%x (vgId:%d), rsp status %d", - req.streamId, taskId, rsp.reqId, rsp.upstreamTaskId, rsp.upstreamNodeId, rsp.status); + SStreamTask* pTask = streamMetaAcquireTask(pMeta, req.streamId, taskId); + if (pTask != NULL) { + rsp.status = streamTaskCheckStatus(pTask, req.upstreamTaskId, req.upstreamNodeId, req.stage); + streamMetaReleaseTask(pMeta, pTask); + + const char* pStatus = streamGetTaskStatusStr(pTask->status.taskStatus); + tqDebug("s-task:%s status:%s, stage:%d recv task check req(reqId:0x%" PRIx64 ") task:0x%x (vgId:%d), check_status:%d", + pTask->id.idStr, pStatus, rsp.oldStage, rsp.reqId, rsp.upstreamTaskId, rsp.upstreamNodeId, rsp.status); + } else { + rsp.status = TASK_DOWNSTREAM_NOT_READY; + tqDebug("tq recv task check(taskId:0x%" PRIx64 "-0x%x not built yet) req(reqId:0x%" PRIx64 + ") from task:0x%x (vgId:%d), rsp check_status %d", + req.streamId, taskId, rsp.reqId, rsp.upstreamTaskId, rsp.upstreamNodeId, rsp.status); + } } return streamSendCheckRsp(pMeta, &req, &rsp, &pMsg->info, taskId); } -int32_t tqProcessStreamTaskCheckRsp(STQ* pTq, SRpcMsg* pMsg) { +int32_t tqProcessTaskCheckRsp(STQ* pTq, SRpcMsg* pMsg) { char* pReq = POINTER_SHIFT(pMsg->pCont, sizeof(SMsgHead)); int32_t len = pMsg->contLen - sizeof(SMsgHead); int32_t vgId = pTq->pStreamMeta->vgId; @@ -1025,8 +1025,8 @@ int32_t tqProcessTaskDeployReq(STQ* pTq, int64_t sversion, char* msg, int32_t ms bool restored = pTq->pVnode->restored; if (p != NULL && restored) { - p->taskExecInfo.init = taosGetTimestampMs(); - tqDebug("s-task:%s set the init ts:%"PRId64, p->id.idStr, p->taskExecInfo.init); + p->execInfo.init = taosGetTimestampMs(); + tqDebug("s-task:%s set the init ts:%"PRId64, p->id.idStr, p->execInfo.init); streamTaskCheckDownstream(p); } else if (!restored) { @@ -1064,14 +1064,14 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { const char* pStatus = streamGetTaskStatusStr(pTask->status.taskStatus); tqDebug("s-task:%s start scan-history stage(step 1), status:%s", id, pStatus); - if (pTask->taskExecInfo.step1Start == 0) { + if (pTask->execInfo.step1Start == 0) { ASSERT(pTask->status.pauseAllowed == false); - pTask->taskExecInfo.step1Start = taosGetTimestampMs(); + pTask->execInfo.step1Start = taosGetTimestampMs(); if (pTask->info.fillHistory == 1) { streamTaskEnablePause(pTask); } } else { - tqDebug("s-task:%s resume from paused, start ts:%" PRId64, pTask->id.idStr, pTask->taskExecInfo.step1Start); + tqDebug("s-task:%s resume from paused, start ts:%" PRId64, pTask->id.idStr, pTask->execInfo.step1Start); } // we have to continue retrying to successfully execute the scan history task. @@ -1091,7 +1091,7 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { streamScanHistoryData(pTask); if (pTask->status.taskStatus == TASK_STATUS__PAUSE) { - double el = (taosGetTimestampMs() - pTask->taskExecInfo.step1Start) / 1000.0; + double el = (taosGetTimestampMs() - pTask->execInfo.step1Start) / 1000.0; int8_t status = streamTaskSetSchedStatusInActive(pTask); tqDebug("s-task:%s is paused in the step1, elapsed time:%.2fs, sched-status:%d", pTask->id.idStr, el, status); streamMetaReleaseTask(pMeta, pTask); @@ -1099,7 +1099,7 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { } // the following procedure should be executed, no matter status is stop/pause or not - double el = (taosGetTimestampMs() - pTask->taskExecInfo.step1Start) / 1000.0; + double el = (taosGetTimestampMs() - pTask->execInfo.step1Start) / 1000.0; tqDebug("s-task:%s scan-history stage(step 1) ended, elapsed time:%.2fs", id, el); if (pTask->info.fillHistory) { @@ -1135,21 +1135,65 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { // now we can stop the stream task execution int64_t latestVer = 0; - taosThreadMutexLock(&pStreamTask->lock); - streamTaskHalt(pStreamTask); - tqDebug("s-task:%s level:%d sched-status:%d is halt by fill-history task:%s", pStreamTask->id.idStr, - pStreamTask->info.taskLevel, pStreamTask->status.schedStatus, id); - latestVer = walReaderGetCurrentVer(pStreamTask->exec.pWalReader); - taosThreadMutexUnlock(&pStreamTask->lock); + while (1) { + taosThreadMutexLock(&pStreamTask->lock); + int8_t status = pStreamTask->status.taskStatus; + if (status == TASK_STATUS__DROPPING || status == TASK_STATUS__STOP) { + // return; + // do nothing + } + + if (status == TASK_STATUS__HALT) { +// tqDebug("s-task:%s level:%d sched-status:%d is halt by fill-history task:%s", pStreamTask->id.idStr, +// pStreamTask->info.taskLevel, pStreamTask->status.schedStatus, id); +// latestVer = walReaderGetCurrentVer(pStreamTask->exec.pWalReader); +// +// taosThreadMutexUnlock(&pStreamTask->lock); +// break; + } + + if (pStreamTask->status.taskStatus == TASK_STATUS__CK) { + qDebug("s-task:%s status:%s during generating checkpoint, wait for 1sec and retry set status:halt", + pStreamTask->id.idStr, streamGetTaskStatusStr(TASK_STATUS__CK)); + taosThreadMutexUnlock(&pStreamTask->lock); + taosMsleep(1000); + continue; + } + + // upgrade to halt status + if (status == TASK_STATUS__PAUSE) { + qDebug("s-task:%s upgrade status to %s from %s", pStreamTask->id.idStr, streamGetTaskStatusStr(TASK_STATUS__HALT), + streamGetTaskStatusStr(TASK_STATUS__PAUSE)); + } else { + qDebug("s-task:%s halt task, prev status:%s", pStreamTask->id.idStr, streamGetTaskStatusStr(status)); + } + + pStreamTask->status.keepTaskStatus = status; + pStreamTask->status.taskStatus = TASK_STATUS__HALT; + + tqDebug("s-task:%s level:%d sched-status:%d is halt by fill-history task:%s", pStreamTask->id.idStr, + pStreamTask->info.taskLevel, pStreamTask->status.schedStatus, id); + latestVer = walReaderGetCurrentVer(pStreamTask->exec.pWalReader); + + taosThreadMutexUnlock(&pStreamTask->lock); + break; + } // if it's an source task, extract the last version in wal. pRange = &pTask->dataRange.range; done = streamHistoryTaskSetVerRangeStep2(pTask, latestVer); if (done) { - pTask->taskExecInfo.step2Start = taosGetTimestampMs(); + pTask->execInfo.step2Start = taosGetTimestampMs(); qDebug("s-task:%s scan-history from WAL stage(step 2) ended, elapsed time:%.2fs", id, 0.0); streamTaskPutTranstateIntoInputQ(pTask); + + if (pTask->status.taskStatus == TASK_STATUS__PAUSE) { + pTask->status.keepTaskStatus = TASK_STATUS__NORMAL; + qDebug("s-task:%s prev status is %s, update the kept status to be:%s when after step 2", id, + streamGetTaskStatusStr(TASK_STATUS__PAUSE), streamGetTaskStatusStr(pTask->status.keepTaskStatus)); + } + streamTryExec(pTask); // exec directly } else { STimeWindow* pWindow = &pTask->dataRange.window; @@ -1159,7 +1203,7 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { pStreamTask->id.idStr); ASSERT(pTask->status.schedStatus == TASK_SCHED_STATUS__WAITING); - pTask->taskExecInfo.step2Start = taosGetTimestampMs(); + pTask->execInfo.step2Start = taosGetTimestampMs(); streamSetParamForStreamScannerStep2(pTask, pRange, pWindow); int64_t dstVer = pTask->dataRange.range.minVer; @@ -1324,10 +1368,12 @@ int32_t tqProcessTaskDispatchReq(STQ* pTq, SRpcMsg* pMsg, bool exec) { tDecodeStreamDispatchReq(&decoder, &req); tDecoderClear(&decoder); + tqDebug("s-task:0x%x recv dispatch msg from 0x%x(vgId:%d)", req.taskId, req.upstreamTaskId, req.upstreamNodeId); + SStreamTask* pTask = streamMetaAcquireTask(pTq->pStreamMeta, req.streamId, req.taskId); if (pTask) { SRpcMsg rsp = {.info = pMsg->info, .code = 0}; - streamProcessDispatchMsg(pTask, &req, &rsp, exec); + streamProcessDispatchMsg(pTask, &req, &rsp); streamMetaReleaseTask(pTq->pStreamMeta, pTask); return 0; } else { @@ -1346,6 +1392,8 @@ int32_t tqProcessTaskDispatchRsp(STQ* pTq, SRpcMsg* pMsg) { pRsp->streamId = htobe64(pRsp->streamId); pRsp->downstreamTaskId = htonl(pRsp->downstreamTaskId); pRsp->downstreamNodeId = htonl(pRsp->downstreamNodeId); + pRsp->stage = htobe64(pRsp->stage); + pRsp->msgId = htonl(pRsp->msgId); SStreamTask* pTask = streamMetaAcquireTask(pTq->pStreamMeta, pRsp->streamId, pRsp->upstreamTaskId); if (pTask) { @@ -1528,10 +1576,16 @@ int32_t vnodeEnqueueStreamMsg(SVnode* pVnode, SRpcMsg* pMsg) { int32_t taskId = req.taskId; tqDebug("vgId:%d receive dispatch msg to s-task:0x%" PRIx64 "-0x%x", vgId, req.streamId, taskId); + // for test purpose +// if (req.type == STREAM_INPUT__CHECKPOINT_TRIGGER) { +// code = TSDB_CODE_STREAM_TASK_NOT_EXIST; +// goto FAIL; +// } + SStreamTask* pTask = streamMetaAcquireTask(pTq->pStreamMeta, req.streamId, taskId); if (pTask != NULL) { SRpcMsg rsp = {.info = pMsg->info, .code = 0}; - streamProcessDispatchMsg(pTask, &req, &rsp, false); + streamProcessDispatchMsg(pTask, &req, &rsp); streamMetaReleaseTask(pTq->pStreamMeta, pTask); rpcFreeCont(pMsg->pCont); taosFreeQitem(pMsg); @@ -1559,12 +1613,16 @@ FAIL: } pRspHead->vgId = htonl(req.upstreamNodeId); + ASSERT(pRspHead->vgId != 0); + SStreamDispatchRsp* pRsp = POINTER_SHIFT(pRspHead, sizeof(SMsgHead)); pRsp->streamId = htobe64(req.streamId); pRsp->upstreamTaskId = htonl(req.upstreamTaskId); pRsp->upstreamNodeId = htonl(req.upstreamNodeId); pRsp->downstreamNodeId = htonl(pVnode->config.vgId); pRsp->downstreamTaskId = htonl(req.taskId); + pRsp->msgId = htonl(req.msgId); + pRsp->stage = htobe64(req.stage); pRsp->inputStatus = TASK_OUTPUT_STATUS__NORMAL; int32_t len = sizeof(SMsgHead) + sizeof(SStreamDispatchRsp); @@ -1578,16 +1636,22 @@ FAIL: } // todo error code cannot be return, since this is invoked by an mnode-launched transaction. -int32_t tqProcessStreamCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg) { +int32_t tqProcessStreamCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg* pRsp) { int32_t vgId = TD_VID(pTq->pVnode); SStreamMeta* pMeta = pTq->pStreamMeta; char* msg = POINTER_SHIFT(pMsg->pCont, sizeof(SMsgHead)); int32_t len = pMsg->contLen - sizeof(SMsgHead); int32_t code = 0; + // disable auto rsp to source + pRsp->info.handle = NULL; + SStreamCheckpointSourceReq req = {0}; if (!vnodeIsRoleLeader(pTq->pVnode)) { - tqDebug("vgId:%d not leader node, ignore checkpoint-source msg", vgId); + tqDebug("vgId:%d not leader, ignore checkpoint-source msg", vgId); + SRpcMsg rsp = {0}; + buildCheckpointSourceRsp(&req, &pMsg->info, &rsp, 0); + tmsgSendRsp(&rsp); // error occurs return TSDB_CODE_SUCCESS; } @@ -1597,6 +1661,9 @@ int32_t tqProcessStreamCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg) { code = TSDB_CODE_MSG_DECODE_ERROR; tDecoderClear(&decoder); tqError("vgId:%d failed to decode checkpoint-source msg, code:%s", vgId, tstrerror(code)); + SRpcMsg rsp = {0}; + buildCheckpointSourceRsp(&req, &pMsg->info, &rsp, 0); + tmsgSendRsp(&rsp); // error occurs return code; } tDecoderClear(&decoder); @@ -1605,6 +1672,9 @@ int32_t tqProcessStreamCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg) { if (pTask == NULL) { tqError("vgId:%d failed to find s-task:0x%x, ignore checkpoint msg. it may have been destroyed already", vgId, req.taskId); + SRpcMsg rsp = {0}; + buildCheckpointSourceRsp(&req, &pMsg->info, &rsp, 0); + tmsgSendRsp(&rsp); // error occurs return TSDB_CODE_SUCCESS; } @@ -1620,6 +1690,22 @@ int32_t tqProcessStreamCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg) { return TSDB_CODE_SUCCESS; } + taosThreadMutexLock(&pTask->lock); + if (pTask->status.taskStatus == TASK_STATUS__HALT) { + qError("s-task:%s not ready for checkpoint, since it is halt, ignore this checkpoint:%" PRId64 ", set it failure", + pTask->id.idStr, req.checkpointId); + taosThreadMutexUnlock(&pTask->lock); + + streamMetaReleaseTask(pMeta, pTask); + + SRpcMsg rsp = {0}; + buildCheckpointSourceRsp(&req, &pMsg->info, &rsp, 0); + tmsgSendRsp(&rsp); // error occurs + return TSDB_CODE_SUCCESS; + } + streamProcessCheckpointSourceReq(pTask, &req); + taosThreadMutexUnlock(&pTask->lock); + int32_t total = 0; taosWLockLatch(&pMeta->lock); @@ -1632,23 +1718,25 @@ int32_t tqProcessStreamCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg) { total = pMeta->numOfStreamTasks; taosWUnLockLatch(&pMeta->lock); - qDebug("s-task:%s (vgId:%d) level:%d receive checkpoint-source msg, chkpt:%" PRId64 ", total checkpoint req:%d", + qInfo("s-task:%s (vgId:%d) level:%d receive checkpoint-source msg chkpt:%" PRId64 ", total checkpoint reqs:%d", pTask->id.idStr, vgId, pTask->info.taskLevel, req.checkpointId, total); code = streamAddCheckpointSourceRspMsg(&req, &pMsg->info, pTask, 1); if (code != TSDB_CODE_SUCCESS) { + SRpcMsg rsp = {0}; + buildCheckpointSourceRsp(&req, &pMsg->info, &rsp, 0); + tmsgSendRsp(&rsp); // error occurs return code; } // todo: when generating checkpoint, no new tasks are allowed to add into current Vnode // todo: when generating checkpoint, leader of mnode has transfer to other DNode? - streamProcessCheckpointSourceReq(pTask, &req); streamMetaReleaseTask(pMeta, pTask); return code; } // downstream task has complete the stream task checkpoint procedure, let's start the handle the rsp by execute task -int32_t tqProcessStreamTaskCheckpointReadyMsg(STQ* pTq, SRpcMsg* pMsg) { +int32_t tqProcessTaskCheckpointReadyMsg(STQ* pTq, SRpcMsg* pMsg) { int32_t vgId = TD_VID(pTq->pVnode); SStreamMeta* pMeta = pTq->pStreamMeta; char* msg = POINTER_SHIFT(pMsg->pCont, sizeof(SMsgHead)); @@ -1760,16 +1848,19 @@ int32_t tqProcessTaskUpdateReq(STQ* pTq, SRpcMsg* pMsg) { // possibly only handle the stream task. int32_t numOfTasks = streamMetaGetNumOfTasks(pMeta); int32_t updateTasks = taosHashGetSize(pMeta->pUpdateTaskSet); - if (updateTasks < numOfTasks) { - pMeta->taskWillbeLaunched = 1; - tqDebug("vgId:%d closed tasks:%d, unclosed:%d", vgId, updateTasks, (numOfTasks - updateTasks)); + pMeta->startInfo.startedAfterNodeUpdate = 1; + + if (updateTasks < numOfTasks) { + tqDebug("vgId:%d closed tasks:%d, unclosed:%d, all tasks will be started when nodeEp update completed", vgId, + updateTasks, (numOfTasks - updateTasks)); taosWUnLockLatch(&pMeta->lock); } else { taosHashClear(pMeta->pUpdateTaskSet); if (!pTq->pVnode->restored) { - tqDebug("vgId:%d vnode restore not completed, not restart the tasks", vgId); + tqDebug("vgId:%d vnode restore not completed, not restart the tasks, clear the start after nodeUpdate flag", vgId); + pMeta->startInfo.startedAfterNodeUpdate = 0; taosWUnLockLatch(&pMeta->lock); } else { tqDebug("vgId:%d tasks are all updated and stopped, restart them", vgId); @@ -1791,14 +1882,13 @@ int32_t tqProcessTaskUpdateReq(STQ* pTq, SRpcMsg* pMsg) { } if (vnodeIsRoleLeader(pTq->pVnode) && !tsDisableStream) { - vInfo("vgId:%d, restart all stream tasks", vgId); + vInfo("vgId:%d restart all stream tasks after all tasks being updated", vgId); tqStartStreamTasks(pTq); tqCheckAndRunStreamTaskAsync(pTq); } else { vInfo("vgId:%d, follower node not start stream tasks", vgId); } - pMeta->taskWillbeLaunched = 0; taosWUnLockLatch(&pMeta->lock); } } diff --git a/source/dnode/vnode/src/tq/tqSink.c b/source/dnode/vnode/src/tq/tqSink.c index da7ac20600..a5958197bd 100644 --- a/source/dnode/vnode/src/tq/tqSink.c +++ b/source/dnode/vnode/src/tq/tqSink.c @@ -24,23 +24,22 @@ typedef struct STableSinkInfo { tstr name; } STableSinkInfo; +static int32_t tsAscendingSortFn(const void* p1, const void* p2); static int32_t setDstTableDataUid(SVnode* pVnode, SStreamTask* pTask, SSDataBlock* pDataBlock, char* stbFullName, SSubmitTbData* pTableData); static int32_t setDstTableDataPayload(SStreamTask* pTask, int32_t blockIndex, SSDataBlock* pDataBlock, SSubmitTbData* pTableData); static int32_t doBuildAndSendDeleteMsg(SVnode* pVnode, char* stbFullName, SSDataBlock* pDataBlock, SStreamTask* pTask, int64_t suid); -static int32_t tqBuildSubmitReq(SSubmitReq2* pSubmitReq, int32_t vgId, void** pMsg, int32_t* msgLen); -static int32_t tsAscendingSortFn(const void* p1, const void* p2); +static int32_t doBuildAndSendSubmitMsg(SVnode* pVnode, SStreamTask* pTask, SSubmitReq2* pReq, int32_t numOfBlocks); +static int32_t buildSubmitMsgImpl(SSubmitReq2* pSubmitReq, int32_t vgId, void** pMsg, int32_t* msgLen); static int32_t doConvertRows(SSubmitTbData* pTableData, STSchema* pTSchema, SSDataBlock* pDataBlock, const char* id); static int32_t doWaitForDstTableCreated(SVnode* pVnode, SStreamTask* pTask, STableSinkInfo* pTableSinkInfo, const char* dstTableName, int64_t* uid); static int32_t doPutIntoCache(SSHashObj* pSinkTableMap, STableSinkInfo* pTableSinkInfo, uint64_t groupId, const char* id); -static SVCreateTbReq* buildAutoCreateTableReq(char* stbFullName, int64_t suid, int32_t numOfCols, - SSDataBlock* pDataBlock); -static bool isValidDstChildTable(SMetaReader* pReader, int32_t vgId, const char* ctbName, int64_t suid); -static int32_t doMergeExistedRows(SSubmitTbData* pExisted, const SSubmitTbData* pNew, const char* id); -static int32_t doBuildAndSendSubmitMsg(SVnode* pVnode, SStreamTask* pTask, SSubmitReq2* pReq, int32_t numOfBlocks); +static int32_t doMergeExistedRows(SSubmitTbData* pExisted, const SSubmitTbData* pNew, const char* id); +static bool isValidDstChildTable(SMetaReader* pReader, int32_t vgId, const char* ctbName, int64_t suid); +static SVCreateTbReq* buildAutoCreateTableReq(char* stbFullName, int64_t suid, int32_t numOfCols, SSDataBlock* pDataBlock); int32_t tqBuildDeleteReq(const char* stbFullName, const SSDataBlock* pDataBlock, SBatchDeleteReq* deleteReq, const char* pIdStr) { @@ -255,7 +254,7 @@ int32_t doBuildAndSendSubmitMsg(SVnode* pVnode, SStreamTask* pTask, SSubmitReq2* void* pBuf = NULL; int32_t numOfFinalBlocks = taosArrayGetSize(pReq->aSubmitTbData); - int32_t code = tqBuildSubmitReq(pReq, vgId, &pBuf, &len); + int32_t code = buildSubmitMsgImpl(pReq, vgId, &pBuf, &len); if (code != TSDB_CODE_SUCCESS) { tqError("s-task:%s build submit msg failed, vgId:%d, code:%s", id, vgId, tstrerror(code)); return code; @@ -270,14 +269,14 @@ int32_t doBuildAndSendSubmitMsg(SVnode* pVnode, SStreamTask* pTask, SSubmitReq2* tqError("s-task:%s failed to put into write-queue since %s", id, terrstr()); } - pTask->sinkRecorder.numOfSubmit += 1; + SSinkRecorder* pRec = &pTask->execInfo.sink; - if ((pTask->sinkRecorder.numOfSubmit % 5000) == 0) { - SSinkTaskRecorder* pRec = &pTask->sinkRecorder; - double el = (taosGetTimestampMs() - pTask->taskExecInfo.start) / 1000.0; + pRec->numOfSubmit += 1; + if ((pRec->numOfSubmit % 5000) == 0) { + double el = (taosGetTimestampMs() - pTask->execInfo.start) / 1000.0; tqInfo("s-task:%s vgId:%d write %" PRId64 " blocks (%" PRId64 " rows) in %" PRId64 - " submit into dst table, duration:%.2f Sec.", - pTask->id.idStr, vgId, pRec->numOfBlocks, pRec->numOfRows, pRec->numOfSubmit, el); + " submit into dst table, %.2fMiB duration:%.2f Sec.", + pTask->id.idStr, vgId, pRec->numOfBlocks, pRec->numOfRows, pRec->numOfSubmit, SIZE_IN_MiB(pRec->bytes), el); } return TSDB_CODE_SUCCESS; @@ -327,7 +326,7 @@ int32_t doMergeExistedRows(SSubmitTbData* pExisted, const SSubmitTbData* pNew, c taosArrayDestroy(pExisted->aRowP); pExisted->aRowP = pFinal; - tqDebug("s-task:%s rows merged, final rows:%d, uid:%" PRId64 ", existed auto-create table:%d, new-block:%d", id, + tqTrace("s-task:%s rows merged, final rows:%d, uid:%" PRId64 ", existed auto-create table:%d, new-block:%d", id, (int32_t)taosArrayGetSize(pFinal), pExisted->uid, (pExisted->pCreateTbReq != NULL), (pNew->pCreateTbReq != NULL)); return TSDB_CODE_SUCCESS; } @@ -462,7 +461,7 @@ int32_t doPutIntoCache(SSHashObj* pSinkTableMap, STableSinkInfo* pTableSinkInfo, return code; } -int32_t tqBuildSubmitReq(SSubmitReq2* pSubmitReq, int32_t vgId, void** pMsg, int32_t* msgLen) { +int32_t buildSubmitMsgImpl(SSubmitReq2* pSubmitReq, int32_t vgId, void** pMsg, int32_t* msgLen) { int32_t code = 0; void* pBuf = NULL; *msgLen = 0; @@ -672,10 +671,10 @@ int32_t setDstTableDataUid(SVnode* pVnode, SStreamTask* pTask, SSDataBlock* pDat pTableData->uid = pTableSinkInfo->uid; if (pTableData->uid == 0) { - tqDebug("s-task:%s cached tableInfo uid is invalid, acquire it from meta", id); + tqTrace("s-task:%s cached tableInfo uid is invalid, acquire it from meta", id); return doWaitForDstTableCreated(pVnode, pTask, pTableSinkInfo, dstTableName, &pTableData->uid); } else { - tqDebug("s-task:%s set the dstTable uid from cache:%"PRId64, id, pTableData->uid); + tqTrace("s-task:%s set the dstTable uid from cache:%"PRId64, id, pTableData->uid); } } else { // The auto-create option will always set to be open for those submit messages, which arrive during the period @@ -740,7 +739,7 @@ int32_t setDstTableDataPayload(SStreamTask* pTask, int32_t blockIndex, SSDataBlo } taosArraySort(pTableData->aRowP, tsAscendingSortFn); - tqDebug("s-task:%s build submit msg for dstTable:%s, numOfRows:%d", id, dstTableName, numOfRows); + tqTrace("s-task:%s build submit msg for dstTable:%s, numOfRows:%d", id, dstTableName, numOfRows); return code; } @@ -755,8 +754,8 @@ void tqSinkDataIntoDstTable(SStreamTask* pTask, void* vnode, void* data) { int32_t code = TSDB_CODE_SUCCESS; const char* id = pTask->id.idStr; - if (pTask->taskExecInfo.start == 0) { - pTask->taskExecInfo.start = taosGetTimestampMs(); + if (pTask->execInfo.start == 0) { + pTask->execInfo.start = taosGetTimestampMs(); } bool onlySubmitData = true; @@ -785,7 +784,7 @@ void tqSinkDataIntoDstTable(SStreamTask* pTask, void* vnode, void* data) { } else if (pDataBlock->info.type == STREAM_CHECKPOINT) { continue; } else { - pTask->sinkRecorder.numOfBlocks += 1; + pTask->execInfo.sink.numOfBlocks += 1; SSubmitReq2 submitReq = {.aSubmitTbData = taosArrayInit(1, sizeof(SSubmitTbData))}; if (submitReq.aSubmitTbData == NULL) { @@ -824,6 +823,8 @@ void tqSinkDataIntoDstTable(SStreamTask* pTask, void* vnode, void* data) { bool hasSubmit = false; for (int32_t i = 0; i < numOfBlocks; i++) { if (streamTaskShouldStop(&pTask->status)) { + taosHashCleanup(pTableIndexMap); + tDestroySubmitReq(&submitReq, TSDB_MSG_FLG_ENCODE); return; } @@ -833,7 +834,7 @@ void tqSinkDataIntoDstTable(SStreamTask* pTask, void* vnode, void* data) { } hasSubmit = true; - pTask->sinkRecorder.numOfBlocks += 1; + pTask->execInfo.sink.numOfBlocks += 1; uint64_t groupId = pDataBlock->info.id.groupId; SSubmitTbData tbData = {.suid = suid, .uid = 0, .sver = pTSchema->version}; @@ -867,7 +868,7 @@ void tqSinkDataIntoDstTable(SStreamTask* pTask, void* vnode, void* data) { } } - pTask->sinkRecorder.numOfRows += pDataBlock->info.rows; + pTask->execInfo.sink.numOfRows += pDataBlock->info.rows; } taosHashCleanup(pTableIndexMap); diff --git a/source/dnode/vnode/src/tq/tqStreamTask.c b/source/dnode/vnode/src/tq/tqStreamTask.c index 854478f41e..c3ef52e96f 100644 --- a/source/dnode/vnode/src/tq/tqStreamTask.c +++ b/source/dnode/vnode/src/tq/tqStreamTask.c @@ -72,6 +72,8 @@ int32_t tqCheckAndRunStreamTask(STQ* pTq) { SArray* pTaskList = NULL; taosWLockLatch(&pMeta->lock); pTaskList = taosArrayDup(pMeta->pTaskList, NULL); + taosHashClear(pMeta->startInfo.pReadyTaskSet); + pMeta->startInfo.ts = taosGetTimestampMs(); taosWUnLockLatch(&pMeta->lock); // broadcast the check downstream tasks msg @@ -96,8 +98,8 @@ int32_t tqCheckAndRunStreamTask(STQ* pTq) { continue; } - pTask->taskExecInfo.init = taosGetTimestampMs(); - tqDebug("s-task:%s start check downstream tasks, set the init ts:%"PRId64, pTask->id.idStr, pTask->taskExecInfo.init); + pTask->execInfo.init = taosGetTimestampMs(); + tqDebug("s-task:%s start check downstream tasks, set the init ts:%"PRId64, pTask->id.idStr, pTask->execInfo.init); streamSetStatusNormal(pTask); streamTaskCheckDownstream(pTask); @@ -231,7 +233,6 @@ int32_t tqStartStreamTasks(STQ* pTq) { int32_t numOfTasks = taosArrayGetSize(pMeta->pTaskList); tqDebug("vgId:%d start all %d stream task(s)", vgId, numOfTasks); - if (numOfTasks == 0) { return TSDB_CODE_SUCCESS; } @@ -306,7 +307,7 @@ void handleFillhistoryScanComplete(SStreamTask* pTask, int64_t ver) { ", not scan wal anymore, add transfer-state block into inputQ", id, ver, maxVer); - double el = (taosGetTimestampMs() - pTask->taskExecInfo.step2Start) / 1000.0; + double el = (taosGetTimestampMs() - pTask->execInfo.step2Start) / 1000.0; qDebug("s-task:%s scan-history from WAL stage(step 2) ended, elapsed time:%.2fs", id, el); /*int32_t code = */streamTaskPutTranstateIntoInputQ(pTask); /*int32_t code = */streamSchedExec(pTask); @@ -355,7 +356,7 @@ int32_t doScanWalForAllTasks(SStreamMeta* pStreamMeta, bool* pScanIdle) { const char* pStatus = streamGetTaskStatusStr(status); if (status != TASK_STATUS__NORMAL) { - tqDebug("s-task:%s not ready for new submit block from wal, status:%s", pTask->id.idStr, pStatus); + tqTrace("s-task:%s not ready for new submit block from wal, status:%s", pTask->id.idStr, pStatus); streamMetaReleaseTask(pStreamMeta, pTask); continue; } diff --git a/source/dnode/vnode/src/tq/tqUtil.c b/source/dnode/vnode/src/tq/tqUtil.c index 73e7f9d44a..b77424d4a5 100644 --- a/source/dnode/vnode/src/tq/tqUtil.c +++ b/source/dnode/vnode/src/tq/tqUtil.c @@ -37,13 +37,19 @@ int32_t tqInitDataRsp(SMqDataRsp* pRsp, STqOffsetVal pOffset) { } void tqUpdateNodeStage(STQ* pTq, bool isLeader) { - SSyncState state = syncGetState(pTq->pVnode->sync); + SSyncState state = syncGetState(pTq->pVnode->sync); SStreamMeta* pMeta = pTq->pStreamMeta; - tqDebug("vgId:%d update the meta stage:%"PRId64", prev:%"PRId64" leader:%d", pMeta->vgId, state.term, pMeta->stage, isLeader); + int64_t stage = pMeta->stage; + pMeta->stage = state.term; - pMeta->leader = isLeader; + pMeta->role = (isLeader)? NODE_ROLE_LEADER:NODE_ROLE_FOLLOWER; if (isLeader) { + tqInfo("vgId:%d update meta stage:%" PRId64 ", prev:%" PRId64 " leader:%d, start to send Hb", pMeta->vgId, + state.term, stage, isLeader); streamMetaStartHb(pMeta); + } else { + tqInfo("vgId:%d update meta stage:%" PRId64 " prev:%" PRId64 " leader:%d", pMeta->vgId, state.term, stage, + isLeader); } } diff --git a/source/dnode/vnode/src/vnd/vnodeSvr.c b/source/dnode/vnode/src/vnd/vnodeSvr.c index 31d66f9b85..2763111645 100644 --- a/source/dnode/vnode/src/vnd/vnodeSvr.c +++ b/source/dnode/vnode/src/vnd/vnodeSvr.c @@ -602,7 +602,7 @@ int32_t vnodeProcessWriteMsg(SVnode *pVnode, SRpcMsg *pMsg, int64_t ver, SRpcMsg vnodeProcessDropIndexReq(pVnode, ver, pReq, len, pRsp); break; case TDMT_VND_STREAM_CHECK_POINT_SOURCE: - tqProcessStreamCheckPointSourceReq(pVnode->pTq, pMsg); + tqProcessStreamCheckPointSourceReq(pVnode->pTq, pMsg, pRsp); break; case TDMT_VND_STREAM_TASK_UPDATE: tqProcessTaskUpdateReq(pVnode->pTq, pMsg); @@ -754,9 +754,9 @@ int32_t vnodeProcessStreamMsg(SVnode *pVnode, SRpcMsg *pMsg, SQueueInfo *pInfo) case TDMT_STREAM_TASK_DISPATCH_RSP: return tqProcessTaskDispatchRsp(pVnode->pTq, pMsg); case TDMT_VND_STREAM_TASK_CHECK: - return tqProcessStreamTaskCheckReq(pVnode->pTq, pMsg); + return tqProcessTaskCheckReq(pVnode->pTq, pMsg); case TDMT_VND_STREAM_TASK_CHECK_RSP: - return tqProcessStreamTaskCheckRsp(pVnode->pTq, pMsg); + return tqProcessTaskCheckRsp(pVnode->pTq, pMsg); case TDMT_STREAM_RETRIEVE: return tqProcessTaskRetrieveReq(pVnode->pTq, pMsg); case TDMT_STREAM_RETRIEVE_RSP: @@ -768,7 +768,7 @@ int32_t vnodeProcessStreamMsg(SVnode *pVnode, SRpcMsg *pMsg, SQueueInfo *pInfo) case TDMT_VND_STREAM_SCAN_HISTORY_FINISH_RSP: return tqProcessTaskScanHistoryFinishRsp(pVnode->pTq, pMsg); case TDMT_STREAM_TASK_CHECKPOINT_READY: - return tqProcessStreamTaskCheckpointReadyMsg(pVnode->pTq, pMsg); + return tqProcessTaskCheckpointReadyMsg(pVnode->pTq, pMsg); default: vError("unknown msg type:%d in stream queue", pMsg->msgType); return TSDB_CODE_APP_ERROR; @@ -1442,11 +1442,8 @@ static int32_t vnodeProcessSubmitReq(SVnode *pVnode, int64_t ver, void *pReq, in SColData *pColData = (SColData *)taosArrayGet(pSubmitTbData->aCol, 0); TSKEY *aKey = (TSKEY *)(pColData->pData); - vDebug("vgId:%d submit %d rows data, uid:%"PRId64, TD_VID(pVnode), pColData->nVal, pSubmitTbData->uid); for (int32_t iRow = 0; iRow < pColData->nVal; iRow++) { - vDebug("vgId:%d uid:%"PRId64" ts:%"PRId64, TD_VID(pVnode), pSubmitTbData->uid, aKey[iRow]); - if (aKey[iRow] < minKey || aKey[iRow] > maxKey || (iRow > 0 && aKey[iRow] <= aKey[iRow - 1])) { code = TSDB_CODE_INVALID_MSG; vError("vgId:%d %s failed since %s, version:%" PRId64, TD_VID(pVnode), __func__, tstrerror(terrno), ver); @@ -1457,10 +1454,7 @@ static int32_t vnodeProcessSubmitReq(SVnode *pVnode, int64_t ver, void *pReq, in } else { int32_t nRow = TARRAY_SIZE(pSubmitTbData->aRowP); SRow **aRow = (SRow **)TARRAY_DATA(pSubmitTbData->aRowP); - - vDebug("vgId:%d submit %d rows data, uid:%"PRId64, TD_VID(pVnode), nRow, pSubmitTbData->uid); for (int32_t iRow = 0; iRow < nRow; ++iRow) { - vDebug("vgId:%d uid:%"PRId64" ts:%"PRId64, TD_VID(pVnode), pSubmitTbData->uid, aRow[iRow]->ts); if (aRow[iRow]->ts < minKey || aRow[iRow]->ts > maxKey || (iRow > 0 && aRow[iRow]->ts <= aRow[iRow - 1]->ts)) { code = TSDB_CODE_INVALID_MSG; diff --git a/source/dnode/vnode/src/vnd/vnodeSync.c b/source/dnode/vnode/src/vnd/vnodeSync.c index 42acdd2b40..a6c743c87d 100644 --- a/source/dnode/vnode/src/vnd/vnodeSync.c +++ b/source/dnode/vnode/src/vnd/vnodeSync.c @@ -551,13 +551,8 @@ static void vnodeRestoreFinish(const SSyncFSM *pFsm, const SyncIndex commitIdx) walApplyVer(pVnode->pWal, commitIdx); pVnode->restored = true; - if (pVnode->pTq->pStreamMeta->taskWillbeLaunched) { - vInfo("vgId:%d, sync restore finished, stream tasks will be launched by other thread", vgId); - return; - } - taosWLockLatch(&pVnode->pTq->pStreamMeta->lock); - if (pVnode->pTq->pStreamMeta->taskWillbeLaunched) { + if (pVnode->pTq->pStreamMeta->startInfo.startedAfterNodeUpdate) { vInfo("vgId:%d, sync restore finished, stream tasks will be launched by other thread", vgId); taosWUnLockLatch(&pVnode->pTq->pStreamMeta->lock); return; @@ -612,10 +607,10 @@ static void vnodeBecomeLearner(const SSyncFSM *pFsm) { static void vnodeBecomeLeader(const SSyncFSM *pFsm) { SVnode *pVnode = pFsm->data; + vDebug("vgId:%d, become leader", pVnode->config.vgId); if (pVnode->pTq) { tqUpdateNodeStage(pVnode->pTq, true); } - vDebug("vgId:%d, become leader", pVnode->config.vgId); } static bool vnodeApplyQueueEmpty(const SSyncFSM *pFsm) { diff --git a/source/libs/stream/inc/streamInt.h b/source/libs/stream/inc/streamInt.h index 64df8e2f44..dbe868b54f 100644 --- a/source/libs/stream/inc/streamInt.h +++ b/source/libs/stream/inc/streamInt.h @@ -26,8 +26,16 @@ extern "C" { #endif -#define ONE_MB_F (1048576.0) -#define SIZE_IN_MB(_v) ((_v) / ONE_MB_F) +#define CHECK_DOWNSTREAM_INTERVAL 100 + +// clang-format off +#define stFatal(...) do { if (stDebugFlag & DEBUG_FATAL) { taosPrintLog("STM FATAL ", DEBUG_FATAL, 255, __VA_ARGS__); }} while(0) +#define stError(...) do { if (stDebugFlag & DEBUG_ERROR) { taosPrintLog("STM ERROR ", DEBUG_ERROR, 255, __VA_ARGS__); }} while(0) +#define stWarn(...) do { if (stDebugFlag & DEBUG_WARN) { taosPrintLog("STM WARN ", DEBUG_WARN, 255, __VA_ARGS__); }} while(0) +#define stInfo(...) do { if (stDebugFlag & DEBUG_INFO) { taosPrintLog("STM ", DEBUG_INFO, 255, __VA_ARGS__); }} while(0) +#define stDebug(...) do { if (stDebugFlag & DEBUG_DEBUG) { taosPrintLog("STM ", DEBUG_DEBUG, tqDebugFlag, __VA_ARGS__); }} while(0) +#define stTrace(...) do { if (stDebugFlag & DEBUG_TRACE) { taosPrintLog("STM ", DEBUG_TRACE, tqDebugFlag, __VA_ARGS__); }} while(0) +// clang-format on typedef struct SStreamGlobalEnv { int8_t inited; @@ -47,13 +55,19 @@ struct STokenBucket { int32_t rate; // number of token per second }; +struct STaskTimer { + void* hTaskLaunchTimer; + void* dispatchTimer; +}; + extern SStreamGlobalEnv streamEnv; extern int32_t streamBackendId; extern int32_t streamBackendCfWrapperId; -const char* streamGetBlockTypeStr(int32_t type); -void streamRetryDispatchStreamBlock(SStreamTask* pTask, int64_t waitDuration); +void streamRetryDispatchData(SStreamTask* pTask, int64_t waitDuration); int32_t streamDispatchStreamBlock(SStreamTask* pTask); +void destroyDispatchMsg(SStreamDispatchReq* pReq, int32_t numOfVgroups); +int32_t getNumOfDispatchBranch(SStreamTask* pTask); int32_t streamProcessCheckpointBlock(SStreamTask* pTask, SStreamDataBlock* pBlock); SStreamDataBlock* createStreamBlockFromDispatchMsg(const SStreamDispatchReq* pReq, int32_t blockType, int32_t srcVg); @@ -68,14 +82,19 @@ int32_t tEncodeStreamRetrieveReq(SEncoder* pEncoder, const SStreamRetrieveReq* p int32_t streamSaveAllTaskStatus(SStreamMeta* pMeta, int64_t checkpointId); int32_t streamTaskBuildCheckpoint(SStreamTask* pTask); -int32_t streamDispatchCheckMsg(SStreamTask* pTask, const SStreamTaskCheckReq* pReq, int32_t nodeId, SEpSet* pEpSet); +int32_t streamSendCheckMsg(SStreamTask* pTask, const SStreamTaskCheckReq* pReq, int32_t nodeId, SEpSet* pEpSet); int32_t streamAddCheckpointReadyMsg(SStreamTask* pTask, int32_t srcTaskId, int32_t index, int64_t checkpointId); int32_t streamTaskSendCheckpointReadyMsg(SStreamTask* pTask); int32_t streamTaskSendCheckpointSourceRsp(SStreamTask* pTask); int32_t streamTaskGetNumOfDownstream(const SStreamTask* pTask); -int32_t streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueItem** pInput, int32_t* numOfBlocks); +int32_t streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueItem** pInput, int32_t* numOfBlocks); +int32_t streamQueueGetNumOfItemsInQueue(const SStreamQueue* pQueue); +int32_t streamQueueItemGetSize(const SStreamQueueItem* pItem); +void streamQueueItemIncSize(const SStreamQueueItem* pItem, int32_t size); +const char* streamQueueItemGetTypeStr(int32_t type); + SStreamQueueItem* streamMergeQueueItem(SStreamQueueItem* dst, SStreamQueueItem* pElem); int32_t streamTaskBuildScanhistoryRspMsg(SStreamTask* pTask, SStreamScanHistoryFinishReq* pReq, void** pBuffer, int32_t* pLen); diff --git a/source/libs/stream/src/stream.c b/source/libs/stream/src/stream.c index 5a7e14c629..2abd3bac05 100644 --- a/source/libs/stream/src/stream.c +++ b/source/libs/stream/src/stream.c @@ -60,10 +60,10 @@ static void streamSchedByTimer(void* param, void* tmrId) { SStreamTask* pTask = (void*)param; int8_t status = atomic_load_8(&pTask->schedInfo.status); - qDebug("s-task:%s in scheduler, trigger status:%d, next:%dms", pTask->id.idStr, status, (int32_t)pTask->info.triggerParam); + stDebug("s-task:%s in scheduler, trigger status:%d, next:%dms", pTask->id.idStr, status, (int32_t)pTask->info.triggerParam); if (streamTaskShouldStop(&pTask->status) || streamTaskShouldPause(&pTask->status)) { - qDebug("s-task:%s jump out of schedTimer", pTask->id.idStr); + stDebug("s-task:%s jump out of schedTimer", pTask->id.idStr); return; } @@ -98,7 +98,7 @@ int32_t streamSetupScheduleTrigger(SStreamTask* pTask) { int32_t ref = atomic_add_fetch_32(&pTask->refCnt, 1); ASSERT(ref == 2 && pTask->schedInfo.pTimer == NULL); - qDebug("s-task:%s setup scheduler trigger, delay:%" PRId64 " ms", pTask->id.idStr, pTask->info.triggerParam); + stDebug("s-task:%s setup scheduler trigger, delay:%" PRId64 " ms", pTask->id.idStr, pTask->info.triggerParam); pTask->schedInfo.pTimer = taosTmrStart(streamSchedByTimer, (int32_t)pTask->info.triggerParam, pTask, streamEnv.timer); pTask->schedInfo.status = TASK_TRIGGER_STATUS__INACTIVE; @@ -114,7 +114,7 @@ int32_t streamSchedExec(SStreamTask* pTask) { if (pRunReq == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; /*int8_t status = */streamTaskSetSchedStatusInActive(pTask); - qError("failed to create msg to aunch s-task:%s, reason out of memory", pTask->id.idStr); + stError("failed to create msg to aunch s-task:%s, reason out of memory", pTask->id.idStr); return -1; } @@ -122,12 +122,12 @@ int32_t streamSchedExec(SStreamTask* pTask) { pRunReq->streamId = pTask->id.streamId; pRunReq->taskId = pTask->id.taskId; - qDebug("trigger to run s-task:%s", pTask->id.idStr); + stDebug("trigger to run s-task:%s", pTask->id.idStr); SRpcMsg msg = {.msgType = TDMT_STREAM_TASK_RUN, .pCont = pRunReq, .contLen = sizeof(SStreamTaskRunReq)}; tmsgPutToQueue(pTask->pMsgCb, STREAM_QUEUE, &msg); } else { - qDebug("s-task:%s not launch task since sched status:%d", pTask->id.idStr, pTask->status.schedStatus); + stDebug("s-task:%s not launch task since sched status:%d", pTask->id.idStr, pTask->status.schedStatus); } return 0; @@ -140,8 +140,12 @@ static int32_t buildDispatchRsp(const SStreamTask* pTask, const SStreamDispatchR } ((SMsgHead*)(*pBuf))->vgId = htonl(pReq->upstreamNodeId); + ASSERT(((SMsgHead*)(*pBuf))->vgId != 0); + SStreamDispatchRsp* pDispatchRsp = POINTER_SHIFT((*pBuf), sizeof(SMsgHead)); + pDispatchRsp->stage = htobe64(pReq->stage); + pDispatchRsp->msgId = htonl(pReq->msgId); pDispatchRsp->inputStatus = status; pDispatchRsp->streamId = htobe64(pReq->streamId); pDispatchRsp->upstreamNodeId = htonl(pReq->upstreamNodeId); @@ -159,7 +163,7 @@ static int32_t streamTaskAppendInputBlocks(SStreamTask* pTask, const SStreamDisp if (pBlock == NULL) { streamTaskInputFail(pTask); status = TASK_INPUT_STATUS__FAILED; - qError("vgId:%d, s-task:%s failed to receive dispatch msg, reason: out of memory", pTask->pMeta->vgId, + stError("vgId:%d, s-task:%s failed to receive dispatch msg, reason: out of memory", pTask->pMeta->vgId, pTask->id.idStr); } else { if (pBlock->type == STREAM_INPUT__TRANS_STATE) { @@ -180,7 +184,7 @@ int32_t streamTaskEnqueueRetrieve(SStreamTask* pTask, SStreamRetrieveReq* pReq, // enqueue if (pData != NULL) { - qDebug("s-task:%s (child %d) recv retrieve req from task:0x%x(vgId:%d), reqId:0x%" PRIx64, pTask->id.idStr, + stDebug("s-task:%s (child %d) recv retrieve req from task:0x%x(vgId:%d), reqId:0x%" PRIx64, pTask->id.idStr, pTask->info.selfChildId, pReq->srcTaskId, pReq->srcNodeId, pReq->reqId); pData->type = STREAM_INPUT__DATA_RETRIEVE; @@ -210,42 +214,54 @@ int32_t streamTaskEnqueueRetrieve(SStreamTask* pTask, SStreamRetrieveReq* pReq, return status == TASK_INPUT_STATUS__NORMAL ? 0 : -1; } -int32_t streamProcessDispatchMsg(SStreamTask* pTask, SStreamDispatchReq* pReq, SRpcMsg* pRsp, bool exec) { - qDebug("s-task:%s receive dispatch msg from taskId:0x%x(vgId:%d), msgLen:%" PRId64, pTask->id.idStr, - pReq->upstreamTaskId, pReq->upstreamNodeId, pReq->totalLen); - int32_t status = 0; +int32_t streamProcessDispatchMsg(SStreamTask* pTask, SStreamDispatchReq* pReq, SRpcMsg* pRsp) { + int32_t status = 0; + const char* id = pTask->id.idStr; + + stDebug("s-task:%s receive dispatch msg from taskId:0x%x(vgId:%d), msgLen:%" PRId64 ", msgId:%d", id, + pReq->upstreamTaskId, pReq->upstreamNodeId, pReq->totalLen, pReq->msgId); SStreamChildEpInfo* pInfo = streamTaskGetUpstreamTaskEpInfo(pTask, pReq->upstreamTaskId); ASSERT(pInfo != NULL); - // upstream task has restarted/leader-follower switch/transferred to other dnodes - if (pReq->stage > pInfo->stage) { - qError("s-task:%s upstream task:0x%x (vgId:%d) has restart/leader-switch/vnode-transfer, prev stage:%" PRId64 - ", current:%" PRId64 " dispatch msg rejected", - pTask->id.idStr, pReq->upstreamTaskId, pReq->upstreamNodeId, pInfo->stage, pReq->stage); - status = TASK_INPUT_STATUS__BLOCKED; + if (pTask->pMeta->role == NODE_ROLE_FOLLOWER) { + stError("s-task:%s task on follower received dispatch msgs, dispatch msg rejected", id); + status = TASK_INPUT_STATUS__REFUSED; } else { - if (!pInfo->dataAllowed) { - qWarn("s-task:%s data from task:0x%x is denied, since inputQ is closed for it", pTask->id.idStr, - pReq->upstreamTaskId); - status = TASK_INPUT_STATUS__BLOCKED; + if (pReq->stage > pInfo->stage) { + // upstream task has restarted/leader-follower switch/transferred to other dnodes + stError("s-task:%s upstream task:0x%x (vgId:%d) has restart/leader-switch/vnode-transfer, prev stage:%" PRId64 + ", current:%" PRId64 " dispatch msg rejected", + id, pReq->upstreamTaskId, pReq->upstreamNodeId, pInfo->stage, pReq->stage); + status = TASK_INPUT_STATUS__REFUSED; } else { - // Current task has received the checkpoint req from the upstream task, from which the message should all be - // blocked - if (pReq->type == STREAM_INPUT__CHECKPOINT_TRIGGER) { - streamTaskCloseUpstreamInput(pTask, pReq->upstreamTaskId); - qDebug("s-task:%s close inputQ for upstream:0x%x", pTask->id.idStr, pReq->upstreamTaskId); - } + if (!pInfo->dataAllowed) { + stWarn("s-task:%s data from task:0x%x is denied, since inputQ is closed for it", id, pReq->upstreamTaskId); + status = TASK_INPUT_STATUS__BLOCKED; + } else { + // This task has received the checkpoint req from the upstream task, from which all the messages should be + // blocked + if (pReq->type == STREAM_INPUT__CHECKPOINT_TRIGGER) { + streamTaskCloseUpstreamInput(pTask, pReq->upstreamTaskId); + stDebug("s-task:%s close inputQ for upstream:0x%x, msgId:%d", id, pReq->upstreamTaskId, pReq->msgId); + } - status = streamTaskAppendInputBlocks(pTask, pReq); + status = streamTaskAppendInputBlocks(pTask, pReq); + } } } + // disable the data from upstream tasks + int8_t st = pTask->status.taskStatus; + if (st == TASK_STATUS__HALT) { + status = TASK_INPUT_STATUS__BLOCKED; + } + { // do send response with the input status int32_t code = buildDispatchRsp(pTask, pReq, status, &pRsp->pCont); if (code != TSDB_CODE_SUCCESS) { - // todo handle failure + stError("s-task:%s failed to build dispatch rsp, msgId:%d, code:%s", id, pReq->msgId, tstrerror(code)); return code; } @@ -254,11 +270,8 @@ int32_t streamProcessDispatchMsg(SStreamTask* pTask, SStreamDispatchReq* pReq, S } tDeleteStreamDispatchReq(pReq); + streamSchedExec(pTask); - int8_t schedStatus = streamTaskSetSchedStatusWait(pTask); - if (schedStatus == TASK_SCHED_STATUS__INACTIVE) { - streamTryExec(pTask); - } return 0; } @@ -306,6 +319,6 @@ SStreamChildEpInfo* streamTaskGetUpstreamTaskEpInfo(SStreamTask* pTask, int32_t } } - qError("s-task:%s failed to find upstream task:0x%x", pTask->id.idStr, taskId); + stError("s-task:%s failed to find upstream task:0x%x", pTask->id.idStr, taskId); return NULL; } \ No newline at end of file diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index 7b3c99acfb..85494693c1 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -303,7 +303,7 @@ int32_t bkdMgtDumpTo(SBackendManager* bm, char* dname) { sprintf(dstDir, "%s%s%s", bm->path, TD_DIRSEP, dname); if (!taosDirExist(srcDir)) { - qError("failed to dump srcDir %s, reason: not exist such dir", srcDir); + stError("failed to dump srcDir %s, reason: not exist such dir", srcDir); code = -1; goto _ERROR; } @@ -311,7 +311,7 @@ int32_t bkdMgtDumpTo(SBackendManager* bm, char* dname) { code = taosMkDir(dstDir); if (code != 0) { terrno = TAOS_SYSTEM_ERROR(errno); - qError("failed to mkdir srcDir %s, reason: %s", dstDir, terrstr()); + stError("failed to mkdir srcDir %s, reason: %s", dstDir, terrstr()); goto _ERROR; } @@ -452,13 +452,13 @@ int32_t rebuildDirFromCheckpoint(const char* path, int64_t chkpId, char** dst) { taosMkDir(state); code = copyFiles(chkp, state); if (code != 0) { - qError("failed to restart stream backend from %s, reason: %s", chkp, tstrerror(TAOS_SYSTEM_ERROR(errno))); + stError("failed to restart stream backend from %s, reason: %s", chkp, tstrerror(TAOS_SYSTEM_ERROR(errno))); } else { - qInfo("start to restart stream backend at checkpoint path: %s", chkp); + stInfo("start to restart stream backend at checkpoint path: %s", chkp); } } else { - qError("failed to start stream backend at %s, reason: %s, restart from default state dir:%s", chkp, + stError("failed to start stream backend at %s, reason: %s, restart from default state dir:%s", chkp, tstrerror(TAOS_SYSTEM_ERROR(errno)), state); taosMkDir(state); } @@ -473,7 +473,7 @@ void* streamBackendInit(const char* streamPath, int64_t chkpId) { char* backendPath = NULL; int32_t code = rebuildDirFromCheckpoint(streamPath, chkpId, &backendPath); - qDebug("start to init stream backend at %s, checkpointid: %" PRId64 "", backendPath, chkpId); + stDebug("start to init stream backend at %s, checkpointid: %" PRId64 "", backendPath, chkpId); uint32_t dbMemLimit = nextPow2(tsMaxStreamBackendCache) << 20; SBackendWrapper* pHandle = taosMemoryCalloc(1, sizeof(SBackendWrapper)); @@ -517,7 +517,7 @@ void* streamBackendInit(const char* streamPath, int64_t chkpId) { taosMemoryFreeClear(err); pHandle->db = rocksdb_open(opts, backendPath, &err); if (err != NULL) { - qError("failed to open rocksdb, path:%s, reason:%s", backendPath, err); + stError("failed to open rocksdb, path:%s, reason:%s", backendPath, err); taosMemoryFreeClear(err); goto _EXIT; } @@ -534,7 +534,7 @@ void* streamBackendInit(const char* streamPath, int64_t chkpId) { if (cfs != NULL) { rocksdb_list_column_families_destroy(cfs, nCf); } - qDebug("succ to init stream backend at %s, backend:%p", backendPath, pHandle); + stDebug("succ to init stream backend at %s, backend:%p", backendPath, pHandle); taosMemoryFreeClear(backendPath); return (void*)pHandle; @@ -547,7 +547,7 @@ _EXIT: taosHashCleanup(pHandle->cfInst); tdListFree(pHandle->list); taosMemoryFree(pHandle); - qDebug("failed to init stream backend at %s", backendPath); + stDebug("failed to init stream backend at %s", backendPath); taosMemoryFree(backendPath); return NULL; } @@ -581,7 +581,7 @@ void streamBackendCleanup(void* arg) { taosThreadMutexDestroy(&pHandle->mutex); taosThreadMutexDestroy(&pHandle->cfMutex); - qDebug("destroy stream backend :%p", pHandle); + stDebug("destroy stream backend :%p", pHandle); taosMemoryFree(pHandle); return; } @@ -590,7 +590,7 @@ void streamBackendHandleCleanup(void* arg) { bool remove = wrapper->remove; taosThreadRwlockWrlock(&wrapper->rwLock); - qDebug("start to do-close backendwrapper %p, %s", wrapper, wrapper->idstr); + stDebug("start to do-close backendwrapper %p, %s", wrapper, wrapper->idstr); if (wrapper->rocksdb == NULL) { taosThreadRwlockUnlock(&wrapper->rwLock); return; @@ -603,7 +603,7 @@ void streamBackendHandleCleanup(void* arg) { for (int i = 0; i < cfLen; i++) { if (wrapper->pHandle[i] != NULL) rocksdb_drop_column_family(wrapper->rocksdb, wrapper->pHandle[i], &err); if (err != NULL) { - qError("failed to drop cf:%s_%s, reason:%s", wrapper->idstr, ginitDict[i].key, err); + stError("failed to drop cf:%s_%s, reason:%s", wrapper->idstr, ginitDict[i].key, err); taosMemoryFreeClear(err); } } @@ -614,7 +614,7 @@ void streamBackendHandleCleanup(void* arg) { for (int i = 0; i < cfLen; i++) { if (wrapper->pHandle[i] != NULL) rocksdb_flush_cf(wrapper->rocksdb, flushOpt, wrapper->pHandle[i], &err); if (err != NULL) { - qError("failed to flush cf:%s_%s, reason:%s", wrapper->idstr, ginitDict[i].key, err); + stError("failed to flush cf:%s_%s, reason:%s", wrapper->idstr, ginitDict[i].key, err); taosMemoryFreeClear(err); } } @@ -649,7 +649,7 @@ void streamBackendHandleCleanup(void* arg) { wrapper->rocksdb = NULL; taosReleaseRef(streamBackendId, wrapper->backendId); - qDebug("end to do-close backendwrapper %p, %s", wrapper, wrapper->idstr); + stDebug("end to do-close backendwrapper %p, %s", wrapper, wrapper->idstr); taosMemoryFree(wrapper); return; } @@ -825,14 +825,14 @@ int32_t chkpDoDbCheckpoint(rocksdb_t* db, char* path) { char* err = NULL; rocksdb_checkpoint_t* cp = rocksdb_checkpoint_object_create(db, &err); if (cp == NULL || err != NULL) { - qError("failed to do checkpoint at:%s, reason:%s", path, err); + stError("failed to do checkpoint at:%s, reason:%s", path, err); taosMemoryFreeClear(err); goto _ERROR; } rocksdb_checkpoint_create(cp, path, 64 << 20, &err); if (err != NULL) { - qError("failed to do checkpoint at:%s, reason:%s", path, err); + stError("failed to do checkpoint at:%s, reason:%s", path, err); taosMemoryFreeClear(err); } else { code = 0; @@ -850,7 +850,7 @@ int32_t chkpPreFlushDb(rocksdb_t* db, rocksdb_column_family_handle_t** cf, int32 rocksdb_flush_cfs(db, flushOpt, cf, nCf, &err); if (err != NULL) { - qError("failed to flush db before streamBackend clean up, reason:%s", err); + stError("failed to flush db before streamBackend clean up, reason:%s", err); taosMemoryFree(err); code = -1; } @@ -865,7 +865,7 @@ int32_t chkpPreCheckDir(char* path, int64_t chkpId, char** chkpDir, char** chkpI sprintf(pChkpDir, "%s%s%s", path, TD_DIRSEP, "checkpoints"); code = taosMulModeMkDir(pChkpDir, 0755, true); if (code != 0) { - qError("failed to prepare checkpoint dir, path:%s, reason:%s", path, tstrerror(code)); + stError("failed to prepare checkpoint dir, path:%s, reason:%s", path, tstrerror(code)); taosMemoryFree(pChkpDir); taosMemoryFree(pChkpIdDir); code = -1; @@ -874,7 +874,7 @@ int32_t chkpPreCheckDir(char* path, int64_t chkpId, char** chkpDir, char** chkpI sprintf(pChkpIdDir, "%s%scheckpoint%" PRId64, pChkpDir, TD_DIRSEP, chkpId); if (taosIsDir(pChkpIdDir)) { - qInfo("stream rm exist checkpoint%s", pChkpIdDir); + stInfo("stream rm exist checkpoint%s", pChkpIdDir); taosRemoveFile(pChkpIdDir); } *chkpDir = pChkpDir; @@ -898,19 +898,19 @@ int32_t streamBackendTriggerChkp(void* arg, char* dst) { goto _ERROR; } int32_t nCf = chkpGetAllDbCfHandle(pMeta, &ppCf, refs); - qDebug("stream backend:%p start to do checkpoint at:%s, cf num: %d ", pHandle, dst, nCf); + stDebug("stream backend:%p start to do checkpoint at:%s, cf num: %d ", pHandle, dst, nCf); code = chkpPreFlushDb(pHandle->db, ppCf, nCf); if (code == 0) { code = chkpDoDbCheckpoint(pHandle->db, dst); if (code != 0) { - qError("stream backend:%p failed to do checkpoint at:%s", pHandle, dst); + stError("stream backend:%p failed to do checkpoint at:%s", pHandle, dst); } else { - qDebug("stream backend:%p end to do checkpoint at:%s, time cost:%" PRId64 "ms", pHandle, dst, + stDebug("stream backend:%p end to do checkpoint at:%s, time cost:%" PRId64 "ms", pHandle, dst, taosGetTimestampMs() - st); } } else { - qError("stream backend:%p failed to flush db at:%s", pHandle, dst); + stError("stream backend:%p failed to flush db at:%s", pHandle, dst); } // release all ref to cfWrapper; @@ -972,19 +972,19 @@ int32_t streamBackendDoCheckpoint(void* arg, uint64_t checkpointId) { // Get all cf and acquire cfWrappter int32_t nCf = chkpGetAllDbCfHandle(pMeta, &ppCf, refs); - qDebug("stream backend:%p start to do checkpoint at:%s, cf num: %d ", pHandle, pChkpIdDir, nCf); + stDebug("stream backend:%p start to do checkpoint at:%s, cf num: %d ", pHandle, pChkpIdDir, nCf); code = chkpPreFlushDb(pHandle->db, ppCf, nCf); if (code == 0) { code = chkpDoDbCheckpoint(pHandle->db, pChkpIdDir); if (code != 0) { - qError("stream backend:%p failed to do checkpoint at:%s", pHandle, pChkpIdDir); + stError("stream backend:%p failed to do checkpoint at:%s", pHandle, pChkpIdDir); } else { - qDebug("stream backend:%p end to do checkpoint at:%s, time cost:%" PRId64 "ms", pHandle, pChkpIdDir, + stDebug("stream backend:%p end to do checkpoint at:%s, time cost:%" PRId64 "ms", pHandle, pChkpIdDir, taosGetTimestampMs() - st); } } else { - qError("stream backend:%p failed to flush db at:%s", pHandle, pChkpIdDir); + stError("stream backend:%p failed to flush db at:%s", pHandle, pChkpIdDir); } // release all ref to cfWrapper; for (int i = 0; i < taosArrayGetSize(refs); i++) { @@ -1390,7 +1390,7 @@ int32_t decodeValueFunc(void* value, int32_t vlen, int64_t* ttl, char** dest) { p = taosDecodeFixedI64(p, &key.unixTimestamp); p = taosDecodeFixedI32(p, &key.len); if (vlen != (sizeof(int64_t) + sizeof(int32_t) + key.len)) { - qError("vlen: %d, read len: %d", vlen, key.len); + stError("vlen: %d, read len: %d", vlen, key.len); goto _EXCEPT; } if (key.len != 0 && dest != NULL) p = taosDecodeBinary(p, (void**)dest, key.len); @@ -1500,7 +1500,7 @@ int32_t streamStateOpenBackendCf(void* backend, char* name, char** cfs, int32_t int idx = streamStateGetCfIdx(NULL, funcname); if (idx < 0 || idx >= sizeof(ginitDict) / sizeof(ginitDict[0])) { - qError("failed to open cf"); + stError("failed to open cf"); return -1; } SCfInit* cfPara = &ginitDict[idx]; @@ -1514,7 +1514,7 @@ int32_t streamStateOpenBackendCf(void* backend, char* name, char** cfs, int32_t rocksdb_t* db = rocksdb_open_column_families(handle->dbOpt, name, nCf, (const char* const*)cfs, (const rocksdb_options_t* const*)cfOpts, cfHandle, &err); if (err != NULL) { - qError("failed to open rocksdb cf, reason:%s", err); + stError("failed to open rocksdb cf, reason:%s", err); taosMemoryFree(err); taosMemoryFree(cfHandle); taosMemoryFree(pCompare); @@ -1523,7 +1523,7 @@ int32_t streamStateOpenBackendCf(void* backend, char* name, char** cfs, int32_t // fix other leak return -1; } else { - qDebug("succ to open rocksdb cf"); + stDebug("succ to open rocksdb cf"); } // close default cf if (((rocksdb_column_family_handle_t**)cfHandle)[0] != 0) { @@ -1634,7 +1634,7 @@ int streamStateOpenBackend(void* backend, SStreamState* pState) { int64_t id = taosAddRef(streamBackendCfWrapperId, pBackendCfWrapper); pState->pTdbState->backendCfWrapperId = id; pState->pTdbState->pBackendCfWrapper = pBackendCfWrapper; - qInfo("succ to open state %p on backendWrapper, %p, %s", pState, pBackendCfWrapper, pBackendCfWrapper->idstr); + stInfo("succ to open state %p on backendWrapper, %p, %s", pState, pBackendCfWrapper, pBackendCfWrapper->idstr); return 0; } taosThreadMutexUnlock(&handle->cfMutex); @@ -1686,7 +1686,7 @@ int streamStateOpenBackend(void* backend, SStreamState* pState) { int64_t id = taosAddRef(streamBackendCfWrapperId, pBackendCfWrapper); pState->pTdbState->backendCfWrapperId = id; pState->pTdbState->pBackendCfWrapper = pBackendCfWrapper; - qInfo("succ to open state %p on backendWrapper %p %s", pState, pBackendCfWrapper, pBackendCfWrapper->idstr); + stInfo("succ to open state %p on backendWrapper %p %s", pState, pBackendCfWrapper, pBackendCfWrapper->idstr); return 0; } @@ -1694,7 +1694,7 @@ void streamStateCloseBackend(SStreamState* pState, bool remove) { SBackendCfWrapper* wrapper = pState->pTdbState->pBackendCfWrapper; SBackendWrapper* pHandle = wrapper->pBackend; - qInfo("start to close state on backend: %p", pHandle); + stInfo("start to close state on backend: %p", pHandle); taosThreadMutexLock(&pHandle->cfMutex); RocksdbCfInst** ppInst = taosHashGet(pHandle->cfInst, wrapper->idstr, strlen(pState->pTdbState->idstr) + 1); @@ -1706,7 +1706,7 @@ void streamStateCloseBackend(SStreamState* pState, bool remove) { taosThreadMutexUnlock(&pHandle->cfMutex); char* status[] = {"close", "drop"}; - qInfo("start to %s state %p on backendWrapper %p %s", status[remove == false ? 0 : 1], pState, wrapper, + stInfo("start to %s state %p on backendWrapper %p %s", status[remove == false ? 0 : 1], pState, wrapper, wrapper->idstr); wrapper->remove |= remove; // update by other pState taosReleaseRef(streamBackendCfWrapperId, pState->pTdbState->backendCfWrapperId); @@ -1740,10 +1740,10 @@ int streamStateGetCfIdx(SStreamState* pState, const char* funcName) { cf = rocksdb_create_column_family(wrapper->rocksdb, wrapper->cfOpts[idx], buf, &err); if (err != NULL) { idx = -1; - qError("failed to open cf, %p %s_%s, reason:%s", pState, wrapper->idstr, funcName, err); + stError("failed to open cf, %p %s_%s, reason:%s", pState, wrapper->idstr, funcName, err); taosMemoryFree(err); } else { - qDebug("succ to to open cf, %p %s_%s", pState, wrapper->idstr, funcName); + stDebug("succ to to open cf, %p %s_%s", pState, wrapper->idstr, funcName); wrapper->pHandle[idx] = cf; } } @@ -1786,7 +1786,7 @@ rocksdb_iterator_t* streamStateIterCreate(SStreamState* pState, const char* cfKe char* err = NULL; \ int i = streamStateGetCfIdx(pState, funcname); \ if (i < 0) { \ - qWarn("streamState failed to get cf name: %s", funcname); \ + stWarn("streamState failed to get cf name: %s", funcname); \ code = -1; \ break; \ } \ @@ -1801,11 +1801,11 @@ rocksdb_iterator_t* streamStateIterCreate(SStreamState* pState, const char* cfKe int32_t ttlVLen = ginitDict[i].enValueFunc((char*)value, vLen, 0, &ttlV); \ rocksdb_put_cf(db, opts, pHandle, (const char*)buf, klen, (const char*)ttlV, (size_t)ttlVLen, &err); \ if (err != NULL) { \ - qError("streamState str: %s failed to write to %s, err: %s", toString, funcname, err); \ + stError("streamState str: %s failed to write to %s, err: %s", toString, funcname, err); \ taosMemoryFree(err); \ code = -1; \ } else { \ - qTrace("streamState str:%s succ to write to %s, rowValLen:%d, ttlValLen:%d", toString, funcname, vLen, ttlVLen); \ + stTrace("streamState str:%s succ to write to %s, rowValLen:%d, ttlValLen:%d", toString, funcname, vLen, ttlVLen); \ } \ taosMemoryFree(ttlV); \ } while (0); @@ -1817,7 +1817,7 @@ rocksdb_iterator_t* streamStateIterCreate(SStreamState* pState, const char* cfKe char* err = NULL; \ int i = streamStateGetCfIdx(pState, funcname); \ if (i < 0) { \ - qWarn("streamState failed to get cf name: %s", funcname); \ + stWarn("streamState failed to get cf name: %s", funcname); \ code = -1; \ break; \ } \ @@ -1832,9 +1832,9 @@ rocksdb_iterator_t* streamStateIterCreate(SStreamState* pState, const char* cfKe char* val = rocksdb_get_cf(db, opts, pHandle, (const char*)buf, klen, (size_t*)&len, &err); \ if (val == NULL || len == 0) { \ if (err == NULL) { \ - qTrace("streamState str: %s failed to read from %s_%s, err: not exist", toString, wrapper->idstr, funcname); \ + stTrace("streamState str: %s failed to read from %s_%s, err: not exist", toString, wrapper->idstr, funcname); \ } else { \ - qError("streamState str: %s failed to read from %s_%s, err: %s", toString, wrapper->idstr, funcname, err); \ + stError("streamState str: %s failed to read from %s_%s, err: %s", toString, wrapper->idstr, funcname, err); \ taosMemoryFreeClear(err); \ } \ code = -1; \ @@ -1842,11 +1842,11 @@ rocksdb_iterator_t* streamStateIterCreate(SStreamState* pState, const char* cfKe char* p = NULL; \ int32_t tlen = ginitDict[i].deValueFunc(val, len, NULL, (char**)pVal); \ if (tlen <= 0) { \ - qError("streamState str: %s failed to read from %s_%s, err: already ttl ", toString, wrapper->idstr, \ + stError("streamState str: %s failed to read from %s_%s, err: already ttl ", toString, wrapper->idstr, \ funcname); \ code = -1; \ } else { \ - qTrace("streamState str: %s succ to read from %s_%s, valLen:%d", toString, wrapper->idstr, funcname, tlen); \ + stTrace("streamState str: %s succ to read from %s_%s, valLen:%d", toString, wrapper->idstr, funcname, tlen); \ } \ taosMemoryFree(val); \ if (vLen != NULL) *vLen = tlen; \ @@ -1860,7 +1860,7 @@ rocksdb_iterator_t* streamStateIterCreate(SStreamState* pState, const char* cfKe char* err = NULL; \ int i = streamStateGetCfIdx(pState, funcname); \ if (i < 0) { \ - qWarn("streamState failed to get cf name: %s_%s", pState->pTdbState->idstr, funcname); \ + stWarn("streamState failed to get cf name: %s_%s", pState->pTdbState->idstr, funcname); \ code = -1; \ break; \ } \ @@ -1873,11 +1873,11 @@ rocksdb_iterator_t* streamStateIterCreate(SStreamState* pState, const char* cfKe rocksdb_writeoptions_t* opts = wrapper->writeOpts; \ rocksdb_delete_cf(db, opts, pHandle, (const char*)buf, klen, &err); \ if (err != NULL) { \ - qError("streamState str: %s failed to del from %s_%s, err: %s", toString, wrapper->idstr, funcname, err); \ + stError("streamState str: %s failed to del from %s_%s, err: %s", toString, wrapper->idstr, funcname, err); \ taosMemoryFree(err); \ code = -1; \ } else { \ - qTrace("streamState str: %s succ to del from %s_%s", toString, wrapper->idstr, funcname); \ + stTrace("streamState str: %s succ to del from %s_%s", toString, wrapper->idstr, funcname); \ } \ } while (0); @@ -1902,7 +1902,7 @@ int32_t streamStateDel_rocksdb(SStreamState* pState, const SWinKey* key) { return code; } int32_t streamStateClear_rocksdb(SStreamState* pState) { - qDebug("streamStateClear_rocksdb"); + stDebug("streamStateClear_rocksdb"); SBackendCfWrapper* wrapper = pState->pTdbState->pBackendCfWrapper; @@ -1924,7 +1924,7 @@ int32_t streamStateClear_rocksdb(SStreamState* pState) { stateKeyToString(&sKey, toStringStart); stateKeyToString(&eKey, toStringEnd); - qWarn("failed to delete range cf(state) start: %s, end:%s, reason:%s", toStringStart, toStringEnd, err); + stWarn("failed to delete range cf(state) start: %s, end:%s, reason:%s", toStringStart, toStringEnd, err); taosMemoryFree(err); } else { rocksdb_compact_range_cf(wrapper->rocksdb, wrapper->pHandle[1], sKeyStr, sLen, eKeyStr, eLen); @@ -1941,7 +1941,7 @@ int32_t streamStateCurNext_rocksdb(SStreamState* pState, SStreamStateCur* pCur) return 0; } int32_t streamStateGetFirst_rocksdb(SStreamState* pState, SWinKey* key) { - qDebug("streamStateGetFirst_rocksdb"); + stDebug("streamStateGetFirst_rocksdb"); SWinKey tmp = {.ts = 0, .groupId = 0}; streamStatePut_rocksdb(pState, &tmp, NULL, 0); @@ -1953,7 +1953,7 @@ int32_t streamStateGetFirst_rocksdb(SStreamState* pState, SWinKey* key) { } int32_t streamStateGetGroupKVByCur_rocksdb(SStreamStateCur* pCur, SWinKey* pKey, const void** pVal, int32_t* pVLen) { - qDebug("streamStateGetGroupKVByCur_rocksdb"); + stDebug("streamStateGetGroupKVByCur_rocksdb"); if (!pCur) { return -1; } @@ -1970,7 +1970,7 @@ int32_t streamStateGetGroupKVByCur_rocksdb(SStreamStateCur* pCur, SWinKey* pKey, return -1; } int32_t streamStateAddIfNotExist_rocksdb(SStreamState* pState, const SWinKey* key, void** pVal, int32_t* pVLen) { - qDebug("streamStateAddIfNotExist_rocksdb"); + stDebug("streamStateAddIfNotExist_rocksdb"); int32_t size = *pVLen; if (streamStateGet_rocksdb(pState, key, pVal, pVLen) == 0) { return 0; @@ -1980,14 +1980,14 @@ int32_t streamStateAddIfNotExist_rocksdb(SStreamState* pState, const SWinKey* ke return 0; } int32_t streamStateCurPrev_rocksdb(SStreamState* pState, SStreamStateCur* pCur) { - qDebug("streamStateCurPrev_rocksdb"); + stDebug("streamStateCurPrev_rocksdb"); if (!pCur) return -1; rocksdb_iter_prev(pCur->iter); return 0; } int32_t streamStateGetKVByCur_rocksdb(SStreamStateCur* pCur, SWinKey* pKey, const void** pVal, int32_t* pVLen) { - qDebug("streamStateGetKVByCur_rocksdb"); + stDebug("streamStateGetKVByCur_rocksdb"); if (!pCur) return -1; SStateKey tkey; SStateKey* pKtmp = &tkey; @@ -2012,7 +2012,7 @@ int32_t streamStateGetKVByCur_rocksdb(SStreamStateCur* pCur, SWinKey* pKey, cons return -1; } SStreamStateCur* streamStateGetAndCheckCur_rocksdb(SStreamState* pState, SWinKey* key) { - qDebug("streamStateGetAndCheckCur_rocksdb"); + stDebug("streamStateGetAndCheckCur_rocksdb"); SStreamStateCur* pCur = streamStateFillGetCur_rocksdb(pState, key); if (pCur) { int32_t code = streamStateGetGroupKVByCur_rocksdb(pCur, key, NULL, 0); @@ -2023,7 +2023,7 @@ SStreamStateCur* streamStateGetAndCheckCur_rocksdb(SStreamState* pState, SWinKey } SStreamStateCur* streamStateSeekKeyNext_rocksdb(SStreamState* pState, const SWinKey* key) { - qDebug("streamStateSeekKeyNext_rocksdb"); + stDebug("streamStateSeekKeyNext_rocksdb"); SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); if (pCur == NULL) { return NULL; @@ -2062,7 +2062,7 @@ SStreamStateCur* streamStateSeekKeyNext_rocksdb(SStreamState* pState, const SWin } SStreamStateCur* streamStateSeekToLast_rocksdb(SStreamState* pState, const SWinKey* key) { - qDebug("streamStateGetCur_rocksdb"); + stDebug("streamStateGetCur_rocksdb"); int32_t code = 0; const SStateKey maxStateKey = {.key = {.groupId = UINT64_MAX, .ts = INT64_MAX}, .opNum = INT64_MAX}; @@ -2077,7 +2077,7 @@ SStreamStateCur* streamStateSeekToLast_rocksdb(SStreamState* pState, const SWinK { char tbuf[256] = {0}; stateKeyToString((void*)&maxStateKey, tbuf); - qDebug("seek to last:%s", tbuf); + stDebug("seek to last:%s", tbuf); } SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); @@ -2104,7 +2104,7 @@ SStreamStateCur* streamStateSeekToLast_rocksdb(SStreamState* pState, const SWinK } SStreamStateCur* streamStateGetCur_rocksdb(SStreamState* pState, const SWinKey* key) { - qDebug("streamStateGetCur_rocksdb"); + stDebug("streamStateGetCur_rocksdb"); SBackendCfWrapper* wrapper = pState->pTdbState->pBackendCfWrapper; SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); @@ -2161,7 +2161,7 @@ int32_t streamStateSessionPut_rocksdb(SStreamState* pState, const SSessionKey* k return code; } int32_t streamStateSessionGet_rocksdb(SStreamState* pState, SSessionKey* key, void** pVal, int32_t* pVLen) { - qDebug("streamStateSessionGet_rocksdb"); + stDebug("streamStateSessionGet_rocksdb"); int code = 0; SStreamStateCur* pCur = streamStateSessionSeekKeyCurrentNext_rocksdb(pState, key); SSessionKey resKey = *key; @@ -2193,7 +2193,7 @@ int32_t streamStateSessionDel_rocksdb(SStreamState* pState, const SSessionKey* k return code; } SStreamStateCur* streamStateSessionSeekKeyCurrentPrev_rocksdb(SStreamState* pState, const SSessionKey* key) { - qDebug("streamStateSessionSeekKeyCurrentPrev_rocksdb"); + stDebug("streamStateSessionSeekKeyCurrentPrev_rocksdb"); SBackendCfWrapper* wrapper = pState->pTdbState->pBackendCfWrapper; SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); @@ -2234,7 +2234,7 @@ SStreamStateCur* streamStateSessionSeekKeyCurrentPrev_rocksdb(SStreamState* pSta return pCur; } SStreamStateCur* streamStateSessionSeekKeyCurrentNext_rocksdb(SStreamState* pState, SSessionKey* key) { - qDebug("streamStateSessionSeekKeyCurrentNext_rocksdb"); + stDebug("streamStateSessionSeekKeyCurrentNext_rocksdb"); SBackendCfWrapper* wrapper = pState->pTdbState->pBackendCfWrapper; SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); if (pCur == NULL) { @@ -2272,7 +2272,7 @@ SStreamStateCur* streamStateSessionSeekKeyCurrentNext_rocksdb(SStreamState* pSta } SStreamStateCur* streamStateSessionSeekKeyNext_rocksdb(SStreamState* pState, const SSessionKey* key) { - qDebug("streamStateSessionSeekKeyNext_rocksdb"); + stDebug("streamStateSessionSeekKeyNext_rocksdb"); SBackendCfWrapper* wrapper = pState->pTdbState->pBackendCfWrapper; SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); if (pCur == NULL) { @@ -2311,7 +2311,7 @@ SStreamStateCur* streamStateSessionSeekKeyNext_rocksdb(SStreamState* pState, con return pCur; } int32_t streamStateSessionGetKVByCur_rocksdb(SStreamStateCur* pCur, SSessionKey* pKey, void** pVal, int32_t* pVLen) { - qDebug("streamStateSessionGetKVByCur_rocksdb"); + stDebug("streamStateSessionGetKVByCur_rocksdb"); if (!pCur) { return -1; } @@ -2375,7 +2375,7 @@ int32_t streamStateFillDel_rocksdb(SStreamState* pState, const SWinKey* key) { } SStreamStateCur* streamStateFillGetCur_rocksdb(SStreamState* pState, const SWinKey* key) { - qDebug("streamStateFillGetCur_rocksdb"); + stDebug("streamStateFillGetCur_rocksdb"); SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); SBackendCfWrapper* wrapper = pState->pTdbState->pBackendCfWrapper; @@ -2411,7 +2411,7 @@ SStreamStateCur* streamStateFillGetCur_rocksdb(SStreamState* pState, const SWinK return NULL; } int32_t streamStateFillGetKVByCur_rocksdb(SStreamStateCur* pCur, SWinKey* pKey, const void** pVal, int32_t* pVLen) { - qDebug("streamStateFillGetKVByCur_rocksdb"); + stDebug("streamStateFillGetKVByCur_rocksdb"); if (!pCur) { return -1; } @@ -2435,7 +2435,7 @@ int32_t streamStateFillGetKVByCur_rocksdb(SStreamStateCur* pCur, SWinKey* pKey, } SStreamStateCur* streamStateFillSeekKeyNext_rocksdb(SStreamState* pState, const SWinKey* key) { - qDebug("streamStateFillSeekKeyNext_rocksdb"); + stDebug("streamStateFillSeekKeyNext_rocksdb"); SBackendCfWrapper* wrapper = pState->pTdbState->pBackendCfWrapper; SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); if (!pCur) { @@ -2473,7 +2473,7 @@ SStreamStateCur* streamStateFillSeekKeyNext_rocksdb(SStreamState* pState, const return NULL; } SStreamStateCur* streamStateFillSeekKeyPrev_rocksdb(SStreamState* pState, const SWinKey* key) { - qDebug("streamStateFillSeekKeyPrev_rocksdb"); + stDebug("streamStateFillSeekKeyPrev_rocksdb"); SBackendCfWrapper* wrapper = pState->pTdbState->pBackendCfWrapper; SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); if (pCur == NULL) { @@ -2511,7 +2511,7 @@ SStreamStateCur* streamStateFillSeekKeyPrev_rocksdb(SStreamState* pState, const return NULL; } int32_t streamStateSessionGetKeyByRange_rocksdb(SStreamState* pState, const SSessionKey* key, SSessionKey* curKey) { - qDebug("streamStateSessionGetKeyByRange_rocksdb"); + stDebug("streamStateSessionGetKeyByRange_rocksdb"); SBackendCfWrapper* wrapper = pState->pTdbState->pBackendCfWrapper; SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); if (pCur == NULL) { @@ -2570,7 +2570,7 @@ int32_t streamStateSessionGetKeyByRange_rocksdb(SStreamState* pState, const SSes int32_t streamStateSessionAddIfNotExist_rocksdb(SStreamState* pState, SSessionKey* key, TSKEY gap, void** pVal, int32_t* pVLen) { - qDebug("streamStateSessionAddIfNotExist_rocksdb"); + stDebug("streamStateSessionAddIfNotExist_rocksdb"); // todo refactor int32_t res = 0; SSessionKey originKey = *key; @@ -2620,7 +2620,7 @@ _end: return res; } int32_t streamStateSessionClear_rocksdb(SStreamState* pState) { - qDebug("streamStateSessionClear_rocksdb"); + stDebug("streamStateSessionClear_rocksdb"); SSessionKey key = {.win.skey = 0, .win.ekey = 0, .groupId = 0}; SStreamStateCur* pCur = streamStateSessionSeekKeyCurrentNext_rocksdb(pState, &key); @@ -2646,7 +2646,7 @@ int32_t streamStateSessionClear_rocksdb(SStreamState* pState) { } int32_t streamStateStateAddIfNotExist_rocksdb(SStreamState* pState, SSessionKey* key, char* pKeyData, int32_t keyDataLen, state_key_cmpr_fn fn, void** pVal, int32_t* pVLen) { - qDebug("streamStateStateAddIfNotExist_rocksdb"); + stDebug("streamStateStateAddIfNotExist_rocksdb"); // todo refactor int32_t res = 0; SSessionKey tmpKey = *key; @@ -2840,7 +2840,7 @@ int32_t streamStatePutBatch(SStreamState* pState, const char* cfKeyName, rocksdb int i = streamStateGetCfIdx(pState, cfKeyName); if (i < 0) { - qError("streamState failed to put to cf name:%s", cfKeyName); + stError("streamState failed to put to cf name:%s", cfKeyName); return -1; } @@ -2857,7 +2857,7 @@ int32_t streamStatePutBatch(SStreamState* pState, const char* cfKeyName, rocksdb { char tbuf[256] = {0}; ginitDict[i].toStrFunc((void*)key, tbuf); - qDebug("streamState str: %s succ to write to %s_%s, len: %d", tbuf, wrapper->idstr, ginitDict[i].key, vlen); + stDebug("streamState str: %s succ to write to %s_%s, len: %d", tbuf, wrapper->idstr, ginitDict[i].key, vlen); } return 0; } @@ -2880,7 +2880,7 @@ int32_t streamStatePutBatchOptimize(SStreamState* pState, int32_t cfIdx, rocksdb { char tbuf[256] = {0}; ginitDict[cfIdx].toStrFunc((void*)key, tbuf); - qDebug("streamState str: %s succ to write to %s_%s", tbuf, wrapper->idstr, ginitDict[cfIdx].key); + stDebug("streamState str: %s succ to write to %s_%s", tbuf, wrapper->idstr, ginitDict[cfIdx].key); } return 0; } @@ -2889,11 +2889,11 @@ int32_t streamStatePutBatch_rocksdb(SStreamState* pState, void* pBatch) { SBackendCfWrapper* wrapper = pState->pTdbState->pBackendCfWrapper; rocksdb_write(wrapper->rocksdb, wrapper->writeOpts, (rocksdb_writebatch_t*)pBatch, &err); if (err != NULL) { - qError("streamState failed to write batch, err:%s", err); + stError("streamState failed to write batch, err:%s", err); taosMemoryFree(err); return -1; } else { - qDebug("write batch to backend:%p", wrapper->pBackend); + stDebug("write batch to backend:%p", wrapper->pBackend); } return 0; } diff --git a/source/libs/stream/src/streamCheckpoint.c b/source/libs/stream/src/streamCheckpoint.c index f367ba932f..997fecbba9 100644 --- a/source/libs/stream/src/streamCheckpoint.c +++ b/source/libs/stream/src/streamCheckpoint.c @@ -95,7 +95,7 @@ static int32_t streamAlignCheckpoint(SStreamTask* pTask) { int32_t num = taosArrayGetSize(pTask->pUpstreamInfoList); int64_t old = atomic_val_compare_exchange_32(&pTask->checkpointAlignCnt, 0, num); if (old == 0) { - qDebug("s-task:%s set initial align upstream num:%d", pTask->id.idStr, num); + stDebug("s-task:%s set initial align upstream num:%d", pTask->id.idStr, num); } return atomic_sub_fetch_32(&pTask->checkpointAlignCnt, 1); @@ -136,18 +136,16 @@ int32_t streamProcessCheckpointSourceReq(SStreamTask* pTask, SStreamCheckpointSo ASSERT(pTask->info.taskLevel == TASK_LEVEL__SOURCE); // 1. set task status to be prepared for check point, no data are allowed to put into inputQ. - taosThreadMutexLock(&pTask->lock); - pTask->status.taskStatus = TASK_STATUS__CK; pTask->checkpointingId = pReq->checkpointId; pTask->checkpointNotReadyTasks = streamTaskGetNumOfDownstream(pTask); pTask->chkInfo.startTs = taosGetTimestampMs(); + pTask->execInfo.checkpoint += 1; + // 2. let's dispatch checkpoint msg to downstream task directly and do nothing else. put the checkpoint block into // inputQ, to make sure all blocks with less version have been handled by this task already. int32_t code = appendCheckpointIntoInputQ(pTask, STREAM_INPUT__CHECKPOINT_TRIGGER); - taosThreadMutexUnlock(&pTask->lock); - return code; } @@ -159,6 +157,7 @@ static int32_t continueDispatchCheckpointBlock(SStreamDataBlock* pBlock, SStream if (code == 0) { streamDispatchStreamBlock(pTask); } else { + stError("s-task:%s failed to put checkpoint into outputQ, code:%s", pTask->id.idStr, tstrerror(code)); streamFreeQitem((SStreamQueueItem*)pBlock); } @@ -193,7 +192,7 @@ int32_t streamProcessCheckpointBlock(SStreamTask* pTask, SStreamDataBlock* pBloc int32_t taskLevel = pTask->info.taskLevel; if (taskLevel == TASK_LEVEL__SOURCE) { if (pTask->outputInfo.type == TASK_OUTPUT__FIXED_DISPATCH || pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { - qDebug("s-task:%s set childIdx:%d, and add checkpoint block into outputQ", id, pTask->info.selfChildId); + stDebug("s-task:%s set childIdx:%d, and add checkpoint block into outputQ", id, pTask->info.selfChildId); continueDispatchCheckpointBlock(pBlock, pTask); } else { // only one task exists, no need to dispatch downstream info streamProcessCheckpointReadyMsg(pTask); @@ -201,8 +200,10 @@ int32_t streamProcessCheckpointBlock(SStreamTask* pTask, SStreamDataBlock* pBloc } } else if (taskLevel == TASK_LEVEL__SINK || taskLevel == TASK_LEVEL__AGG) { ASSERT(taosArrayGetSize(pTask->pUpstreamInfoList) > 0); - - pTask->chkInfo.startTs = taosGetTimestampMs(); + if (pTask->chkInfo.startTs == 0) { + pTask->chkInfo.startTs = taosGetTimestampMs(); + pTask->execInfo.checkpoint += 1; + } // update the child Id for downstream tasks streamAddCheckpointReadyMsg(pTask, pBlock->srcTaskId, pTask->info.selfChildId, checkpointId); @@ -211,19 +212,19 @@ int32_t streamProcessCheckpointBlock(SStreamTask* pTask, SStreamDataBlock* pBloc int32_t notReady = streamAlignCheckpoint(pTask); int32_t num = taosArrayGetSize(pTask->pUpstreamInfoList); if (notReady > 0) { - qDebug("s-task:%s received checkpoint block, idx:%d, %d upstream tasks not send checkpoint info yet, total:%d", + stDebug("s-task:%s received checkpoint block, idx:%d, %d upstream tasks not send checkpoint info yet, total:%d", id, pTask->info.selfChildId, notReady, num); streamFreeQitem((SStreamQueueItem*)pBlock); return code; } if (taskLevel == TASK_LEVEL__SINK) { - qDebug("s-task:%s process checkpoint block, all %d upstreams sent checkpoint msgs, send ready msg to upstream", + stDebug("s-task:%s process checkpoint block, all %d upstreams sent checkpoint msgs, send ready msg to upstream", id, num); streamFreeQitem((SStreamQueueItem*)pBlock); streamTaskBuildCheckpoint(pTask); } else { - qDebug( + stDebug( "s-task:%s process checkpoint block, all %d upstreams sent checkpoint msgs, dispatch checkpoint msg " "downstream", id, num); @@ -252,12 +253,12 @@ int32_t streamProcessCheckpointReadyMsg(SStreamTask* pTask) { ASSERT(notReady >= 0); if (notReady == 0) { - qDebug("s-task:%s all downstream tasks have completed the checkpoint, start to do checkpoint for current task", + stDebug("s-task:%s all downstream tasks have completed the checkpoint, start to do checkpoint for current task", pTask->id.idStr); appendCheckpointIntoInputQ(pTask, STREAM_INPUT__CHECKPOINT); } else { int32_t total = streamTaskGetNumOfDownstream(pTask); - qDebug("s-task:%s %d/%d downstream tasks are not ready, wait", pTask->id.idStr, notReady, total); + stDebug("s-task:%s %d/%d downstream tasks are not ready, wait", pTask->id.idStr, notReady, total); } return 0; @@ -266,7 +267,6 @@ int32_t streamProcessCheckpointReadyMsg(SStreamTask* pTask) { int32_t streamSaveAllTaskStatus(SStreamMeta* pMeta, int64_t checkpointId) { taosWLockLatch(&pMeta->lock); - int64_t keys[2]; for (int32_t i = 0; i < taosArrayGetSize(pMeta->pTaskList); ++i) { STaskId* pId = taosArrayGet(pMeta->pTaskList, i); SStreamTask** ppTask = taosHashGet(pMeta->pTasksMap, pId, sizeof(*pId)); @@ -288,20 +288,22 @@ int32_t streamSaveAllTaskStatus(SStreamMeta* pMeta, int64_t checkpointId) { // save the task streamMetaSaveTask(pMeta, p); streamTaskOpenAllUpstreamInput(p); // open inputQ for all upstream tasks - qDebug("vgId:%d s-task:%s level:%d commit task status after checkpoint completed, checkpointId:%" PRId64 - ", Ver(saved):%" PRId64 " currentVer:%" PRId64 ", status to be normal, prev:%s", - pMeta->vgId, p->id.idStr, p->info.taskLevel, checkpointId, p->chkInfo.checkpointVer, p->chkInfo.nextProcessVer, - streamGetTaskStatusStr(prev)); + + stDebug( + "vgId:%d s-task:%s level:%d open upstream inputQ, commit task status after checkpoint completed, " + "checkpointId:%" PRId64 ", Ver(saved):%" PRId64 " currentVer:%" PRId64 ", status to be normal, prev:%s", + pMeta->vgId, p->id.idStr, p->info.taskLevel, checkpointId, p->chkInfo.checkpointVer, p->chkInfo.nextProcessVer, + streamGetTaskStatusStr(prev)); } if (streamMetaCommit(pMeta) < 0) { taosWUnLockLatch(&pMeta->lock); - qError("vgId:%d failed to commit stream meta after do checkpoint, checkpointId:%" PRId64 ", since %s", pMeta->vgId, + stError("vgId:%d failed to commit stream meta after do checkpoint, checkpointId:%" PRId64 ", since %s", pMeta->vgId, checkpointId, terrstr()); return -1; } else { taosWUnLockLatch(&pMeta->lock); - qInfo("vgId:%d commit stream meta after do checkpoint, checkpointId:%" PRId64 " DONE", pMeta->vgId, checkpointId); + stInfo("vgId:%d commit stream meta after do checkpoint, checkpointId:%" PRId64 " DONE", pMeta->vgId, checkpointId); } return TSDB_CODE_SUCCESS; @@ -316,18 +318,21 @@ int32_t streamTaskBuildCheckpoint(SStreamTask* pTask) { ASSERT(remain >= 0); double el = (taosGetTimestampMs() - pTask->chkInfo.startTs) / 1000.0; + pTask->chkInfo.startTs = 0; // clear the recorded start time if (remain == 0) { // all tasks are ready - qDebug("s-task:%s is ready for checkpoint", pTask->id.idStr); + stDebug("s-task:%s is ready for checkpoint", pTask->id.idStr); streamBackendDoCheckpoint(pMeta, pTask->checkpointingId); streamSaveAllTaskStatus(pMeta, pTask->checkpointingId); - qDebug("vgId:%d vnode wide checkpoint completed, save all tasks status, elapsed time:%.2f Sec checkpointId:%" PRId64, pMeta->vgId, - el, pTask->checkpointingId); + stInfo( + "vgId:%d vnode wide checkpoint completed, save all tasks status, last:%s, level:%d elapsed time:%.2f Sec " + "checkpointId:%" PRId64, + pMeta->vgId, pTask->id.idStr, pTask->info.taskLevel, el, pTask->checkpointingId); } else { - qDebug( - "vgId:%d vnode wide tasks not reach checkpoint ready status, ready s-task:%s, elapsed time:%.2f Sec not " - "ready:%d/%d", - pMeta->vgId, pTask->id.idStr, el, remain, pMeta->numOfStreamTasks); + stInfo( + "vgId:%d vnode wide tasks not reach checkpoint ready status, ready s-task:%s, level:%d elapsed time:%.2f Sec " + "not ready:%d/%d", + pMeta->vgId, pTask->id.idStr, pTask->info.taskLevel, el, remain, pMeta->numOfStreamTasks); } // send check point response to upstream task @@ -339,7 +344,7 @@ int32_t streamTaskBuildCheckpoint(SStreamTask* pTask) { if (code != TSDB_CODE_SUCCESS) { // todo: let's retry send rsp to upstream/mnode - qError("s-task:%s failed to send checkpoint rsp to upstream, checkpointId:%" PRId64 ", code:%s", pTask->id.idStr, + stError("s-task:%s failed to send checkpoint rsp to upstream, checkpointId:%" PRId64 ", code:%s", pTask->id.idStr, pTask->checkpointingId, tstrerror(code)); } diff --git a/source/libs/stream/src/streamData.c b/source/libs/stream/src/streamData.c index a108667f5d..80927b36b9 100644 --- a/source/libs/stream/src/streamData.c +++ b/source/libs/stream/src/streamData.c @@ -165,12 +165,16 @@ SStreamQueueItem* streamMergeQueueItem(SStreamQueueItem* dst, SStreamQueueItem* SStreamDataBlock* pBlockSrc = (SStreamDataBlock*)pElem; taosArrayAddAll(pBlock->blocks, pBlockSrc->blocks); taosArrayDestroy(pBlockSrc->blocks); + streamQueueItemIncSize(dst, streamQueueItemGetSize(pElem)); + taosFreeQitem(pElem); return dst; } else if (dst->type == STREAM_INPUT__MERGED_SUBMIT && pElem->type == STREAM_INPUT__DATA_SUBMIT) { SStreamMergedSubmit* pMerged = (SStreamMergedSubmit*)dst; SStreamDataSubmit* pBlockSrc = (SStreamDataSubmit*)pElem; streamMergeSubmit(pMerged, pBlockSrc); + streamQueueItemIncSize(dst, streamQueueItemGetSize(pElem)); + taosFreeQitem(pElem); return dst; } else if (dst->type == STREAM_INPUT__DATA_SUBMIT && pElem->type == STREAM_INPUT__DATA_SUBMIT) { @@ -180,13 +184,16 @@ SStreamQueueItem* streamMergeQueueItem(SStreamQueueItem* dst, SStreamQueueItem* return NULL; } + streamQueueItemIncSize((SStreamQueueItem*)pMerged, streamQueueItemGetSize(pElem)); + streamMergeSubmit(pMerged, (SStreamDataSubmit*)dst); streamMergeSubmit(pMerged, (SStreamDataSubmit*)pElem); + taosFreeQitem(dst); taosFreeQitem(pElem); return (SStreamQueueItem*)pMerged; } else { - qDebug("block type:%s not merged with existed blocks list, type:%d", streamGetBlockTypeStr(pElem->type), dst->type); + stDebug("block type:%s not merged with existed blocks list, type:%d", streamQueueItemGetTypeStr(pElem->type), dst->type); return NULL; } } @@ -227,16 +234,3 @@ void streamFreeQitem(SStreamQueueItem* data) { taosFreeQitem(pBlock); } } - -const char* streamGetBlockTypeStr(int32_t type) { - switch (type) { - case STREAM_INPUT__CHECKPOINT: - return "checkpoint"; - case STREAM_INPUT__CHECKPOINT_TRIGGER: - return "checkpoint-trigger"; - case STREAM_INPUT__TRANS_STATE: - return "trans-state"; - default: - return ""; - } -} \ No newline at end of file diff --git a/source/libs/stream/src/streamDispatch.c b/source/libs/stream/src/streamDispatch.c index 7003c570e9..bd5753cac3 100644 --- a/source/libs/stream/src/streamDispatch.c +++ b/source/libs/stream/src/streamDispatch.c @@ -53,6 +53,7 @@ void initRpcMsg(SRpcMsg* pMsg, int32_t msgType, void* pCont, int32_t contLen) { int32_t tEncodeStreamDispatchReq(SEncoder* pEncoder, const SStreamDispatchReq* pReq) { if (tStartEncode(pEncoder) < 0) return -1; if (tEncodeI64(pEncoder, pReq->stage) < 0) return -1; + if (tEncodeI32(pEncoder, pReq->msgId) < 0) return -1; if (tEncodeI32(pEncoder, pReq->srcVgId) < 0) return -1; if (tEncodeI32(pEncoder, pReq->type) < 0) return -1; if (tEncodeI64(pEncoder, pReq->streamId) < 0) return -1; @@ -78,6 +79,7 @@ int32_t tEncodeStreamDispatchReq(SEncoder* pEncoder, const SStreamDispatchReq* p int32_t tDecodeStreamDispatchReq(SDecoder* pDecoder, SStreamDispatchReq* pReq) { if (tStartDecode(pDecoder) < 0) return -1; if (tDecodeI64(pDecoder, &pReq->stage) < 0) return -1; + if (tDecodeI32(pDecoder, &pReq->msgId) < 0) return -1; if (tDecodeI32(pDecoder, &pReq->srcVgId) < 0) return -1; if (tDecodeI32(pDecoder, &pReq->type) < 0) return -1; if (tDecodeI64(pDecoder, &pReq->streamId) < 0) return -1; @@ -112,6 +114,7 @@ static int32_t tInitStreamDispatchReq(SStreamDispatchReq* pReq, const SStreamTas pReq->streamId = pTask->id.streamId; pReq->srcVgId = vgId; pReq->stage = pTask->pMeta->stage; + pReq->msgId = pTask->execInfo.dispatch; pReq->upstreamTaskId = pTask->id.taskId; pReq->upstreamChildId = pTask->info.selfChildId; pReq->upstreamNodeId = pTask->info.nodeId; @@ -231,7 +234,7 @@ int32_t streamBroadcastToChildren(SStreamTask* pTask, const SSDataBlock* pBlock) } buf = NULL; - qDebug("s-task:%s (child %d) send retrieve req to task:0x%x (vgId:%d), reqId:0x%" PRIx64, pTask->id.idStr, + stDebug("s-task:%s (child %d) send retrieve req to task:0x%x (vgId:%d), reqId:0x%" PRIx64, pTask->id.idStr, pTask->info.selfChildId, pEpInfo->taskId, pEpInfo->nodeId, req.reqId); } code = 0; @@ -242,7 +245,7 @@ CLEAR: return code; } -int32_t streamDispatchCheckMsg(SStreamTask* pTask, const SStreamTaskCheckReq* pReq, int32_t nodeId, SEpSet* pEpSet) { +int32_t streamSendCheckMsg(SStreamTask* pTask, const SStreamTaskCheckReq* pReq, int32_t nodeId, SEpSet* pEpSet) { void* buf = NULL; int32_t code = -1; SRpcMsg msg = {0}; @@ -270,66 +273,73 @@ int32_t streamDispatchCheckMsg(SStreamTask* pTask, const SStreamTaskCheckReq* pR tEncoderClear(&encoder); initRpcMsg(&msg, TDMT_VND_STREAM_TASK_CHECK, buf, tlen + sizeof(SMsgHead)); - qDebug("s-task:%s (level:%d) send check msg to s-task:0x%" PRIx64 ":0x%x (vgId:%d)", pTask->id.idStr, + stDebug("s-task:%s (level:%d) send check msg to s-task:0x%" PRIx64 ":0x%x (vgId:%d)", pTask->id.idStr, pTask->info.taskLevel, pReq->streamId, pReq->downstreamTaskId, nodeId); tmsgSendReq(pEpSet, &msg); return 0; } -static int32_t doDispatchAllBlocks(SStreamTask* pTask, const SStreamDataBlock* pData) { +void destroyDispatchMsg(SStreamDispatchReq* pReq, int32_t numOfVgroups) { + for (int32_t i = 0; i < numOfVgroups; i++) { + taosArrayDestroyP(pReq[i].data, taosMemoryFree); + taosArrayDestroy(pReq[i].dataLen); + } + + taosMemoryFree(pReq); +} + +int32_t getNumOfDispatchBranch(SStreamTask* pTask) { + return (pTask->outputInfo.type == TASK_OUTPUT__FIXED_DISPATCH) + ? 1 + : taosArrayGetSize(pTask->shuffleDispatcher.dbInfo.pVgroupInfos); +} + +static int32_t doBuildDispatchMsg(SStreamTask* pTask, const SStreamDataBlock* pData) { int32_t code = 0; int32_t numOfBlocks = taosArrayGetSize(pData->blocks); - ASSERT(numOfBlocks != 0); + ASSERT(numOfBlocks != 0 && pTask->msgInfo.pData == NULL); + + pTask->msgInfo.dispatchMsgType = pData->type; if (pTask->outputInfo.type == TASK_OUTPUT__FIXED_DISPATCH) { - SStreamDispatchReq req = {0}; + SStreamDispatchReq* pReq = taosMemoryCalloc(1, sizeof(SStreamDispatchReq)); - int32_t downstreamTaskId = pTask->fixedEpDispatcher.taskId; - code = tInitStreamDispatchReq(&req, pTask, pData->srcVgId, numOfBlocks, downstreamTaskId, pData->type); + int32_t downstreamTaskId = pTask->fixedDispatcher.taskId; + code = tInitStreamDispatchReq(pReq, pTask, pData->srcVgId, numOfBlocks, downstreamTaskId, pData->type); if (code != TSDB_CODE_SUCCESS) { return code; } for (int32_t i = 0; i < numOfBlocks; i++) { SSDataBlock* pDataBlock = taosArrayGet(pData->blocks, i); - - code = streamAddBlockIntoDispatchMsg(pDataBlock, &req); + code = streamAddBlockIntoDispatchMsg(pDataBlock, pReq); if (code != TSDB_CODE_SUCCESS) { - taosArrayDestroyP(req.data, taosMemoryFree); - taosArrayDestroy(req.dataLen); + destroyDispatchMsg(pReq, 1); return code; } } - int32_t vgId = pTask->fixedEpDispatcher.nodeId; - SEpSet* pEpSet = &pTask->fixedEpDispatcher.epSet; - - qDebug("s-task:%s (child taskId:%d) fix-dispatch %d block(s) to s-task:0x%x (vgId:%d)", pTask->id.idStr, - pTask->info.selfChildId, numOfBlocks, downstreamTaskId, vgId); - - code = doSendDispatchMsg(pTask, &req, vgId, pEpSet); - taosArrayDestroyP(req.data, taosMemoryFree); - taosArrayDestroy(req.dataLen); - return code; + pTask->msgInfo.pData = pReq; } else if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { int32_t rspCnt = atomic_load_32(&pTask->shuffleDispatcher.waitingRspCnt); ASSERT(rspCnt == 0); SArray* vgInfo = pTask->shuffleDispatcher.dbInfo.pVgroupInfos; - int32_t vgSz = taosArrayGetSize(vgInfo); + int32_t numOfVgroups = taosArrayGetSize(vgInfo); - SStreamDispatchReq* pReqs = taosMemoryCalloc(vgSz, sizeof(SStreamDispatchReq)); + SStreamDispatchReq* pReqs = taosMemoryCalloc(numOfVgroups, sizeof(SStreamDispatchReq)); if (pReqs == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; return -1; } - for (int32_t i = 0; i < vgSz; i++) { + for (int32_t i = 0; i < numOfVgroups; i++) { SVgroupInfo* pVgInfo = taosArrayGet(vgInfo, i); code = tInitStreamDispatchReq(&pReqs[i], pTask, pData->srcVgId, 0, pVgInfo->taskId, pData->type); if (code != TSDB_CODE_SUCCESS) { - goto FAIL_SHUFFLE_DISPATCH; + destroyDispatchMsg(pReqs, numOfVgroups); + return code; } } @@ -338,50 +348,72 @@ static int32_t doDispatchAllBlocks(SStreamTask* pTask, const SStreamDataBlock* p // TODO: do not use broadcast if (pDataBlock->info.type == STREAM_DELETE_RESULT || pDataBlock->info.type == STREAM_CHECKPOINT || pDataBlock->info.type == STREAM_TRANS_STATE) { - for (int32_t j = 0; j < vgSz; j++) { - if (streamAddBlockIntoDispatchMsg(pDataBlock, &pReqs[j]) < 0) { - goto FAIL_SHUFFLE_DISPATCH; + for (int32_t j = 0; j < numOfVgroups; j++) { + code = streamAddBlockIntoDispatchMsg(pDataBlock, &pReqs[j]); + if (code != 0) { + destroyDispatchMsg(pReqs, numOfVgroups); + return code; } if (pReqs[j].blockNum == 0) { atomic_add_fetch_32(&pTask->shuffleDispatcher.waitingRspCnt, 1); } + pReqs[j].blockNum++; } continue; } - if (streamSearchAndAddBlock(pTask, pReqs, pDataBlock, vgSz, pDataBlock->info.id.groupId) < 0) { - goto FAIL_SHUFFLE_DISPATCH; + code = streamSearchAndAddBlock(pTask, pReqs, pDataBlock, numOfVgroups, pDataBlock->info.id.groupId); + if(code != 0) { + destroyDispatchMsg(pReqs, numOfVgroups); + return code; } } - qDebug("s-task:%s (child taskId:%d) shuffle-dispatch blocks:%d to %d vgroups", pTask->id.idStr, - pTask->info.selfChildId, numOfBlocks, vgSz); + pTask->msgInfo.pData = pReqs; + } - for (int32_t i = 0; i < vgSz; i++) { - if (pReqs[i].blockNum > 0) { + stDebug("s-task:%s build dispatch msg success, msgId:%d", pTask->id.idStr, pTask->execInfo.dispatch); + return code; +} + +static int32_t sendDispatchMsg(SStreamTask* pTask, SStreamDispatchReq* pDispatchMsg) { + int32_t code = 0; + int32_t msgId = pTask->execInfo.dispatch; + const char* id = pTask->id.idStr; + + if (pTask->outputInfo.type == TASK_OUTPUT__FIXED_DISPATCH) { + int32_t vgId = pTask->fixedDispatcher.nodeId; + SEpSet* pEpSet = &pTask->fixedDispatcher.epSet; + int32_t downstreamTaskId = pTask->fixedDispatcher.taskId; + + stDebug("s-task:%s (child taskId:%d) fix-dispatch %d block(s) to s-task:0x%x (vgId:%d), id:%d", id, + pTask->info.selfChildId, 1, downstreamTaskId, vgId, msgId); + + code = doSendDispatchMsg(pTask, pDispatchMsg, vgId, pEpSet); + } else { + SArray* vgInfo = pTask->shuffleDispatcher.dbInfo.pVgroupInfos; + int32_t numOfVgroups = taosArrayGetSize(vgInfo); + + stDebug("s-task:%s (child taskId:%d) start to shuffle-dispatch blocks to %d vgroup(s), msgId:%d", + id, pTask->info.selfChildId, numOfVgroups, msgId); + + for (int32_t i = 0; i < numOfVgroups; i++) { + if (pDispatchMsg[i].blockNum > 0) { SVgroupInfo* pVgInfo = taosArrayGet(vgInfo, i); - qDebug("s-task:%s (child taskId:%d) shuffle-dispatch blocks:%d to vgId:%d", pTask->id.idStr, - pTask->info.selfChildId, pReqs[i].blockNum, pVgInfo->vgId); + stDebug("s-task:%s (child taskId:%d) shuffle-dispatch blocks:%d to vgId:%d", pTask->id.idStr, + pTask->info.selfChildId, pDispatchMsg[i].blockNum, pVgInfo->vgId); - code = doSendDispatchMsg(pTask, &pReqs[i], pVgInfo->vgId, &pVgInfo->epSet); + code = doSendDispatchMsg(pTask, &pDispatchMsg[i], pVgInfo->vgId, &pVgInfo->epSet); if (code < 0) { - goto FAIL_SHUFFLE_DISPATCH; + break; } } } - code = 0; - - FAIL_SHUFFLE_DISPATCH: - for (int32_t i = 0; i < vgSz; i++) { - taosArrayDestroyP(pReqs[i].data, taosMemoryFree); - taosArrayDestroy(pReqs[i].dataLen); - } - - taosMemoryFree(pReqs); + stDebug("s-task:%s complete shuffle-dispatch blocks to all %d vnodes, msgId:%d", pTask->id.idStr, numOfVgroups, msgId); } return code; @@ -389,41 +421,92 @@ static int32_t doDispatchAllBlocks(SStreamTask* pTask, const SStreamDataBlock* p static void doRetryDispatchData(void* param, void* tmrId) { SStreamTask* pTask = param; + const char* id = pTask->id.idStr; + int32_t msgId = pTask->execInfo.dispatch; if (streamTaskShouldStop(&pTask->status)) { int8_t ref = atomic_sub_fetch_8(&pTask->status.timerActive, 1); - qDebug("s-task:%s should stop, abort from timer, ref:%d", pTask->id.idStr, ref); + stDebug("s-task:%s should stop, abort from timer, ref:%d", pTask->id.idStr, ref); return; } ASSERT(pTask->outputInfo.status == TASK_OUTPUT_STATUS__WAIT); - int32_t code = doDispatchAllBlocks(pTask, pTask->msgInfo.pData); + int32_t code = 0; + { + SArray* pList = taosArrayDup(pTask->msgInfo.pRetryList, NULL); + taosArrayClear(pTask->msgInfo.pRetryList); + + SStreamDispatchReq *pReq = pTask->msgInfo.pData; + + if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { + SArray* vgInfo = pTask->shuffleDispatcher.dbInfo.pVgroupInfos; + int32_t numOfVgroups = taosArrayGetSize(vgInfo); + + int32_t numOfFailed = taosArrayGetSize(pList); + stDebug("s-task:%s (child taskId:%d) re-try shuffle-dispatch blocks to %d vgroup(s), msgId:%d", + id, pTask->info.selfChildId, numOfFailed, msgId); + + for (int32_t i = 0; i < numOfFailed; i++) { + int32_t vgId = *(int32_t*) taosArrayGet(pList, i); + + for(int32_t j = 0; j < numOfVgroups; ++j) { + SVgroupInfo* pVgInfo = taosArrayGet(vgInfo, j); + if (pVgInfo->vgId == vgId) { + stDebug("s-task:%s (child taskId:%d) shuffle-dispatch blocks:%d to vgId:%d", pTask->id.idStr, + pTask->info.selfChildId, pReq[j].blockNum, pVgInfo->vgId); + + code = doSendDispatchMsg(pTask, &pReq[j], pVgInfo->vgId, &pVgInfo->epSet); + if (code < 0) { + break; + } + } + } + } + + stDebug("s-task:%s complete re-try shuffle-dispatch blocks to all %d vnodes, msgId:%d", pTask->id.idStr, numOfFailed, msgId); + } else { + int32_t vgId = pTask->fixedDispatcher.nodeId; + SEpSet* pEpSet = &pTask->fixedDispatcher.epSet; + int32_t downstreamTaskId = pTask->fixedDispatcher.taskId; + + stDebug("s-task:%s (child taskId:%d) fix-dispatch %d block(s) to s-task:0x%x (vgId:%d), id:%d", id, + pTask->info.selfChildId, 1, downstreamTaskId, vgId, msgId); + + code = doSendDispatchMsg(pTask, pReq, vgId, pEpSet); + } + } + if (code != TSDB_CODE_SUCCESS) { if (!streamTaskShouldStop(&pTask->status)) { - qDebug("s-task:%s reset the waitRspCnt to be 0 before launch retry dispatch", pTask->id.idStr); - atomic_store_32(&pTask->shuffleDispatcher.waitingRspCnt, 0); +// stDebug("s-task:%s reset the waitRspCnt to be 0 before launch retry dispatch", pTask->id.idStr); +// atomic_store_32(&pTask->shuffleDispatcher.waitingRspCnt, 0); if (streamTaskShouldPause(&pTask->status)) { - streamRetryDispatchStreamBlock(pTask, DISPATCH_RETRY_INTERVAL_MS * 10); + streamRetryDispatchData(pTask, DISPATCH_RETRY_INTERVAL_MS * 10); } else { - streamRetryDispatchStreamBlock(pTask, DISPATCH_RETRY_INTERVAL_MS); + streamRetryDispatchData(pTask, DISPATCH_RETRY_INTERVAL_MS); } } else { int32_t ref = atomic_sub_fetch_8(&pTask->status.timerActive, 1); - qDebug("s-task:%s should stop, abort from timer, ref:%d", pTask->id.idStr, ref); + stDebug("s-task:%s should stop, abort from timer, ref:%d", pTask->id.idStr, ref); } } else { int8_t ref = atomic_sub_fetch_8(&pTask->status.timerActive, 1); - qDebug("s-task:%s send success, jump out of timer, ref:%d", pTask->id.idStr, ref); + stDebug("s-task:%s send success, jump out of timer, ref:%d", pTask->id.idStr, ref); } } -void streamRetryDispatchStreamBlock(SStreamTask* pTask, int64_t waitDuration) { - qWarn("s-task:%s dispatch data in %" PRId64 "ms, in timer", pTask->id.idStr, waitDuration); - if (pTask->launchTaskTimer != NULL) { - taosTmrReset(doRetryDispatchData, waitDuration, pTask, streamEnv.timer, &pTask->launchTaskTimer); +void streamRetryDispatchData(SStreamTask* pTask, int64_t waitDuration) { + STaskTimer* pTmr = pTask->pTimer; + pTask->msgInfo.retryCount++; + + stWarn("s-task:%s retry send dispatch data in %" PRId64 "ms, in timer msgId:%d, retryTimes:%d", pTask->id.idStr, + waitDuration, pTask->execInfo.dispatch, pTask->msgInfo.retryCount); + + if (pTmr->dispatchTimer != NULL) { + taosTmrReset(doRetryDispatchData, waitDuration, pTask, streamEnv.timer, &pTmr->dispatchTimer); } else { - pTask->launchTaskTimer = taosTmrStart(doRetryDispatchData, waitDuration, pTask, streamEnv.timer); + pTmr->dispatchTimer = taosTmrStart(doRetryDispatchData, waitDuration, pTask, streamEnv.timer); } } @@ -500,57 +583,64 @@ int32_t streamDispatchStreamBlock(SStreamTask* pTask) { const char* id = pTask->id.idStr; int32_t numOfElems = streamQueueGetNumOfItems(pTask->outputInfo.queue); if (numOfElems > 0) { - double size = SIZE_IN_MB(taosQueueMemorySize(pTask->outputInfo.queue->pQueue)); - qDebug("s-task:%s start to dispatch intermediate block to downstream, elem in outputQ:%d, size:%.2fMiB", id, numOfElems, size); + double size = SIZE_IN_MiB(taosQueueMemorySize(pTask->outputInfo.queue->pQueue)); + stDebug("s-task:%s start to dispatch intermediate block to downstream, elem in outputQ:%d, size:%.2fMiB", id, numOfElems, size); } // to make sure only one dispatch is running int8_t old = atomic_val_compare_exchange_8(&pTask->outputInfo.status, TASK_OUTPUT_STATUS__NORMAL, TASK_OUTPUT_STATUS__WAIT); if (old != TASK_OUTPUT_STATUS__NORMAL) { - qDebug("s-task:%s wait for dispatch rsp, not dispatch now, output status:%d", id, old); + stDebug("s-task:%s wait for dispatch rsp, not dispatch now, output status:%d", id, old); return 0; } ASSERT(pTask->msgInfo.pData == NULL); - qDebug("s-task:%s start to dispatch msg, set output status:%d", id, pTask->outputInfo.status); + stDebug("s-task:%s start to dispatch msg, set output status:%d", id, pTask->outputInfo.status); SStreamDataBlock* pBlock = streamQueueNextItem(pTask->outputInfo.queue); if (pBlock == NULL) { atomic_store_8(&pTask->outputInfo.status, TASK_OUTPUT_STATUS__NORMAL); - qDebug("s-task:%s not dispatch since no elems in outputQ, output status:%d", id, pTask->outputInfo.status); + stDebug("s-task:%s not dispatch since no elems in outputQ, output status:%d", id, pTask->outputInfo.status); return 0; } - pTask->msgInfo.pData = pBlock; ASSERT(pBlock->type == STREAM_INPUT__DATA_BLOCK || pBlock->type == STREAM_INPUT__CHECKPOINT_TRIGGER || pBlock->type == STREAM_INPUT__TRANS_STATE); int32_t retryCount = 0; + pTask->execInfo.dispatch += 1; + pTask->msgInfo.startTs = taosGetTimestampMs(); + + int32_t code = doBuildDispatchMsg(pTask, pBlock); + if (code == 0) { + destroyStreamDataBlock(pBlock); + } else { // todo handle build dispatch msg failed + } while (1) { - int32_t code = doDispatchAllBlocks(pTask, pBlock); + code = sendDispatchMsg(pTask, pTask->msgInfo.pData); if (code == TSDB_CODE_SUCCESS) { break; } - qDebug("s-task:%s failed to dispatch msg to downstream, code:%s, output status:%d, retry cnt:%d", id, - tstrerror(terrno), pTask->outputInfo.status, retryCount); + stDebug("s-task:%s failed to dispatch msg:%d to downstream, code:%s, output status:%d, retry cnt:%d", id, + pTask->execInfo.dispatch, tstrerror(terrno), pTask->outputInfo.status, retryCount); // todo deal with only partially success dispatch case atomic_store_32(&pTask->shuffleDispatcher.waitingRspCnt, 0); if (terrno == TSDB_CODE_APP_IS_STOPPING) { // in case of this error, do not retry anymore - destroyStreamDataBlock(pTask->msgInfo.pData); + destroyDispatchMsg(pTask->msgInfo.pData, getNumOfDispatchBranch(pTask)); pTask->msgInfo.pData = NULL; return code; } if (++retryCount > MAX_CONTINUE_RETRY_COUNT) { // add to timer to retry int8_t ref = atomic_add_fetch_8(&pTask->status.timerActive, 1); - - qDebug("s-task:%s failed to dispatch msg to downstream for %d times, code:%s, add timer to retry in %dms, ref:%d", + stDebug("s-task:%s failed to dispatch msg to downstream for %d times, code:%s, add timer to retry in %dms, ref:%d", pTask->id.idStr, retryCount, tstrerror(terrno), DISPATCH_RETRY_INTERVAL_MS, ref); - streamRetryDispatchStreamBlock(pTask, DISPATCH_RETRY_INTERVAL_MS); + + streamRetryDispatchData(pTask, DISPATCH_RETRY_INTERVAL_MS); break; } } @@ -569,15 +659,15 @@ int32_t streamDispatchScanHistoryFinishMsg(SStreamTask* pTask) { // serialize if (pTask->outputInfo.type == TASK_OUTPUT__FIXED_DISPATCH) { - req.downstreamTaskId = pTask->fixedEpDispatcher.taskId; + req.downstreamTaskId = pTask->fixedDispatcher.taskId; pTask->notReadyTasks = 1; - doDispatchScanHistoryFinishMsg(pTask, &req, pTask->fixedEpDispatcher.nodeId, &pTask->fixedEpDispatcher.epSet); + doDispatchScanHistoryFinishMsg(pTask, &req, pTask->fixedDispatcher.nodeId, &pTask->fixedDispatcher.epSet); } else if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { SArray* vgInfo = pTask->shuffleDispatcher.dbInfo.pVgroupInfos; int32_t numOfVgs = taosArrayGetSize(vgInfo); pTask->notReadyTasks = numOfVgs; - qDebug("s-task:%s send scan-history data complete msg to downstream (shuffle-dispatch) %d tasks, status:%s", pTask->id.idStr, + stDebug("s-task:%s send scan-history data complete msg to downstream (shuffle-dispatch) %d tasks, status:%s", pTask->id.idStr, numOfVgs, streamGetTaskStatusStr(pTask->status.taskStatus)); for (int32_t i = 0; i < numOfVgs; i++) { SVgroupInfo* pVgInfo = taosArrayGet(vgInfo, i); @@ -585,7 +675,7 @@ int32_t streamDispatchScanHistoryFinishMsg(SStreamTask* pTask) { doDispatchScanHistoryFinishMsg(pTask, &req, pVgInfo->vgId, &pVgInfo->epSet); } } else { - qDebug("s-task:%s no downstream tasks, invoke scan-history finish rsp directly", pTask->id.idStr); + stDebug("s-task:%s no downstream tasks, invoke scan-history finish rsp directly", pTask->id.idStr); streamProcessScanHistoryFinishRsp(pTask); } @@ -601,12 +691,12 @@ int32_t streamTaskSendCheckpointReadyMsg(SStreamTask* pTask) { SStreamChkptReadyInfo* pInfo = taosArrayGet(pTask->pReadyMsgList, i); tmsgSendReq(&pInfo->upstreamNodeEpset, &pInfo->msg); - qDebug("s-task:%s level:%d checkpoint ready msg sent to upstream:0x%x", pTask->id.idStr, pTask->info.taskLevel, + stDebug("s-task:%s level:%d checkpoint ready msg sent to upstream:0x%x", pTask->id.idStr, pTask->info.taskLevel, pInfo->upStreamTaskId); } taosArrayClear(pTask->pReadyMsgList); - qDebug("s-task:%s level:%d checkpoint ready msg sent to all %d upstreams", pTask->id.idStr, pTask->info.taskLevel, num); + stDebug("s-task:%s level:%d checkpoint ready msg sent to all %d upstreams", pTask->id.idStr, pTask->info.taskLevel, num); return TSDB_CODE_SUCCESS; } @@ -619,7 +709,7 @@ int32_t streamTaskSendCheckpointSourceRsp(SStreamTask* pTask) { tmsgSendRsp(&pInfo->msg); taosArrayClear(pTask->pReadyMsgList); - qDebug("s-task:%s level:%d source checkpoint completed msg sent to mnode", pTask->id.idStr, pTask->info.taskLevel); + stDebug("s-task:%s level:%d source checkpoint completed msg sent to mnode", pTask->id.idStr, pTask->info.taskLevel); return TSDB_CODE_SUCCESS; } @@ -691,7 +781,7 @@ int32_t doDispatchScanHistoryFinishMsg(SStreamTask* pTask, const SStreamScanHist tmsgSendReq(pEpSet, &msg); const char* pStatus = streamGetTaskStatusStr(pTask->status.taskStatus); - qDebug("s-task:%s status:%s dispatch scan-history finish msg to taskId:0x%x (vgId:%d)", pTask->id.idStr, pStatus, + stDebug("s-task:%s status:%s dispatch scan-history finish msg to taskId:0x%x (vgId:%d)", pTask->id.idStr, pStatus, pReq->downstreamTaskId, vgId); return 0; } @@ -725,7 +815,7 @@ int32_t doSendDispatchMsg(SStreamTask* pTask, const SStreamDispatchReq* pReq, in tEncoderClear(&encoder); initRpcMsg(&msg, pTask->msgInfo.msgType, buf, tlen + sizeof(SMsgHead)); - qDebug("s-task:%s dispatch msg to taskId:0x%x vgId:%d data msg", pTask->id.idStr, pReq->taskId, vgId); + stDebug("s-task:%s dispatch msg to taskId:0x%x vgId:%d data msg", pTask->id.idStr, pReq->taskId, vgId); return tmsgSendReq(pEpSet, &msg); @@ -764,7 +854,6 @@ int32_t buildCheckpointSourceRsp(SStreamCheckpointSourceReq* pReq, SRpcHandleInf } ((SMsgHead*)pBuf)->vgId = htonl(pReq->mnodeId); - void* abuf = POINTER_SHIFT(pBuf, sizeof(SMsgHead)); tEncoderInit(&encoder, (uint8_t*)abuf, len); @@ -786,7 +875,7 @@ int32_t streamAddCheckpointSourceRspMsg(SStreamCheckpointSourceReq* pReq, SRpcHa } taosArrayPush(pTask->pReadyMsgList, &info); - qDebug("s-task:%s add checkpoint source rsp msg, total:%d", pTask->id.idStr, (int32_t)taosArrayGetSize(pTask->pReadyMsgList)); + stDebug("s-task:%s add checkpoint source rsp msg, total:%d", pTask->id.idStr, (int32_t)taosArrayGetSize(pTask->pReadyMsgList)); return TSDB_CODE_SUCCESS; } @@ -836,8 +925,8 @@ int32_t streamAddCheckpointReadyMsg(SStreamTask* pTask, int32_t upstreamTaskId, initRpcMsg(&info.msg, TDMT_STREAM_TASK_CHECKPOINT_READY, buf, tlen + sizeof(SMsgHead)); info.msg.info.noResp = 1; // refactor later. - qDebug("s-task:%s (level:%d) prepare checkpoint ready msg to upstream s-task:0x%" PRIx64 ":0x%x (vgId:%d) idx:%d", - pTask->id.idStr, pTask->info.taskLevel, req.streamId, req.upstreamTaskId, req.downstreamNodeId, index); + stDebug("s-task:%s (level:%d) prepare checkpoint ready msg to upstream s-task:0x%" PRIx64 ":0x%x (vgId:%d) idx:%d", + pTask->id.idStr, pTask->info.taskLevel, req.streamId, req.upstreamTaskId, req.upstreamNodeId, index); if (pTask->pReadyMsgList == NULL) { pTask->pReadyMsgList = taosArrayInit(4, sizeof(SStreamChkptReadyInfo)); @@ -924,7 +1013,7 @@ int32_t streamAddEndScanHistoryMsg(SStreamTask* pTask, SRpcHandleInfo* pRpcInfo, taosThreadMutexUnlock(&pTask->lock); int32_t num = taosArrayGetSize(pTask->pRspMsgList); - qDebug("s-task:%s add scan history finish rsp msg for task:0x%x, total:%d", pTask->id.idStr, pReq->upstreamTaskId, + stDebug("s-task:%s add scan history finish rsp msg for task:0x%x, total:%d", pTask->id.idStr, pReq->upstreamTaskId, num); return TSDB_CODE_SUCCESS; } @@ -932,110 +1021,150 @@ int32_t streamAddEndScanHistoryMsg(SStreamTask* pTask, SRpcHandleInfo* pRpcInfo, int32_t streamNotifyUpstreamContinue(SStreamTask* pTask) { ASSERT(pTask->info.taskLevel == TASK_LEVEL__AGG || pTask->info.taskLevel == TASK_LEVEL__SINK); + const char* id = pTask->id.idStr; + int32_t level = pTask->info.taskLevel; + int32_t num = taosArrayGetSize(pTask->pRspMsgList); for (int32_t i = 0; i < num; ++i) { SStreamContinueExecInfo* pInfo = taosArrayGet(pTask->pRspMsgList, i); tmsgSendRsp(&pInfo->msg); - qDebug("s-task:%s level:%d notify upstream:0x%x to continue process data in WAL", pTask->id.idStr, pTask->info.taskLevel, - pInfo->taskId); + stDebug("s-task:%s level:%d notify upstream:0x%x continuing scan data in WAL", id, level, pInfo->taskId); } taosArrayClear(pTask->pRspMsgList); - qDebug("s-task:%s level:%d checkpoint ready msg sent to all %d upstreams", pTask->id.idStr, pTask->info.taskLevel, - num); + stDebug("s-task:%s level:%d continue process msg sent to all %d upstreams", id, level, num); + return 0; +} + +// this message has been sent successfully, let's try next one. +static int32_t handleDispatchSuccessRsp(SStreamTask* pTask, int32_t downstreamId) { + destroyDispatchMsg(pTask->msgInfo.pData, getNumOfDispatchBranch(pTask)); + pTask->msgInfo.pData = NULL; + + int64_t el = taosGetTimestampMs() - pTask->msgInfo.startTs; + stDebug("s-task:%s downstream task:0x%x resume to normal from inputQ blocking, blocking time:%" PRId64 "ms", + pTask->id.idStr, downstreamId, el); + + // put data into inputQ of current task is also allowed + if (pTask->inputInfo.status == TASK_INPUT_STATUS__BLOCKED) { + pTask->inputInfo.status = TASK_INPUT_STATUS__NORMAL; + } + + // now ready for next data output + atomic_store_8(&pTask->outputInfo.status, TASK_OUTPUT_STATUS__NORMAL); + + // otherwise, continue dispatch the first block to down stream task in pipeline + streamDispatchStreamBlock(pTask); return 0; } int32_t streamProcessDispatchRsp(SStreamTask* pTask, SStreamDispatchRsp* pRsp, int32_t code) { const char* id = pTask->id.idStr; + int32_t vgId = pTask->pMeta->vgId; + int32_t msgId = pTask->execInfo.dispatch; + + if ((pTask->pMeta->role == NODE_ROLE_FOLLOWER) || (pTask->status.downstreamReady != 1)) { + stError("s-task:%s vgId:%d is follower or task just re-launched, not handle the dispatch rsp, discard it", id, vgId); + return TSDB_CODE_STREAM_TASK_NOT_EXIST; + } + + if ((pRsp->msgId != msgId) || (pRsp->stage != pTask->pMeta->stage)) { + stError("s-task:%s vgId:%d not expect rsp, expected: msgId:%d, stage:%" PRId64 " actual msgId:%d, stage:%" PRId64 + " discard it", + id, vgId, msgId, pTask->pMeta->stage, pRsp->msgId, pRsp->stage); + return TSDB_CODE_INVALID_MSG; + } if (code != TSDB_CODE_SUCCESS) { // dispatch message failed: network error, or node not available. - // in case of the input queue is full, the code will be TSDB_CODE_SUCCESS, the and pRsp>inputStatus will be set - // flag. here we need to retry dispatch this message to downstream task immediately. handle the case the failure + // in case of the input queue is full, the code will be TSDB_CODE_SUCCESS, the and pRsp->inputStatus will be set + // flag. Here we need to retry dispatch this message to downstream task immediately. handle the case the failure // happened too fast. - // todo handle the shuffle dispatch failure - if (code == TSDB_CODE_STREAM_TASK_NOT_EXIST) { // destination task does not exist, not retry anymore - qWarn("s-task:%s failed to dispatch msg to task:0x%x, no retry, since it is destroyed already", id, pRsp->downstreamTaskId); + if (code == TSDB_CODE_STREAM_TASK_NOT_EXIST) { // destination task does not exist, not retry anymore + stError("s-task:%s failed to dispatch msg to task:0x%x(vgId:%d), msgId:%d no retry, since task destroyed already", id, + pRsp->downstreamTaskId, pRsp->downstreamNodeId, msgId); } else { - qError("s-task:%s failed to dispatch msg to task:0x%x, code:%s, retry cnt:%d", id, pRsp->downstreamTaskId, - tstrerror(code), ++pTask->msgInfo.retryCount); - int32_t ret = doDispatchAllBlocks(pTask, pTask->msgInfo.pData); - if (ret != TSDB_CODE_SUCCESS) { + stError("s-task:%s failed to dispatch msgId:%d to task:0x%x(vgId:%d), code:%s, add to retry list", id, msgId, + pRsp->downstreamTaskId, pRsp->downstreamNodeId, tstrerror(code)); + taosThreadMutexLock(&pTask->lock); + taosArrayPush(pTask->msgInfo.pRetryList, &pRsp->downstreamNodeId); + taosThreadMutexUnlock(&pTask->lock); + } + + } else { // code == 0 + if (pRsp->inputStatus == TASK_INPUT_STATUS__BLOCKED) { + pTask->inputInfo.status = TASK_INPUT_STATUS__BLOCKED; + // block the input of current task, to push pressure to upstream + taosThreadMutexLock(&pTask->lock); + taosArrayPush(pTask->msgInfo.pRetryList, &pRsp->downstreamNodeId); + taosThreadMutexUnlock(&pTask->lock); + + stError("s-task:%s inputQ of downstream task:0x%x(vgId:%d) is full, wait for %dms and retry dispatch data", id, + pRsp->downstreamTaskId, pRsp->downstreamNodeId, DISPATCH_RETRY_INTERVAL_MS); + } else if (pRsp->inputStatus == TASK_INPUT_STATUS__REFUSED) { + stError("s-task:%s downstream task:0x%x(vgId:%d) refused the dispatch msg, treat it as success", id, + pRsp->downstreamTaskId, pRsp->downstreamNodeId); + } + } + + int32_t leftRsp = 0; + if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { + stDebug("s-task:%s waiting rsp:%d", id, pTask->shuffleDispatcher.waitingRspCnt); + leftRsp = atomic_sub_fetch_32(&pTask->shuffleDispatcher.waitingRspCnt, 1); + ASSERT(leftRsp >= 0); + + if (leftRsp > 0) { + stDebug( "s-task:%s recv dispatch rsp, msgId:%d from 0x%x(vgId:%d), downstream task input status:%d code:%d, waiting for %d rsp", + id, msgId, pRsp->downstreamTaskId, pRsp->downstreamNodeId, pRsp->inputStatus, code, leftRsp); + } else { + stDebug( + "s-task:%s recv dispatch rsp, msgId:%d from 0x%x(vgId:%d), downstream task input status:%d code:%d, all rsp", + id, msgId, pRsp->downstreamTaskId, pRsp->downstreamNodeId, pRsp->inputStatus, code); + } + } else { + stDebug("s-task:%s recv fix-dispatch rsp, msgId:%d from 0x%x(vgId:%d), downstream task input status:%d code:%d", + id, msgId, pRsp->downstreamTaskId, pRsp->downstreamNodeId, pRsp->inputStatus, code); + } + + ASSERT(leftRsp >= 0); + + // all msg rsp already, continue + if (leftRsp == 0) { + ASSERT(pTask->outputInfo.status == TASK_OUTPUT_STATUS__WAIT); + stDebug("s-task:%s output status is set to:%d", id, pTask->outputInfo.status); + + // we need to re-try send dispatch msg to downstream tasks + int32_t numOfFailed = taosArrayGetSize(pTask->msgInfo.pRetryList); + if (numOfFailed > 0) { + if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { + atomic_store_32(&pTask->shuffleDispatcher.waitingRspCnt, numOfFailed); + stDebug("s-task:%s waiting rsp set to be %d", id, pTask->shuffleDispatcher.waitingRspCnt); + } + + int8_t ref = atomic_add_fetch_8(&pTask->status.timerActive, 1); + stDebug("s-task:%s failed to dispatch msg to downstream code:%s, add timer to retry in %dms, ref:%d", + pTask->id.idStr, tstrerror(terrno), DISPATCH_RETRY_INTERVAL_MS, ref); + + streamRetryDispatchData(pTask, DISPATCH_RETRY_INTERVAL_MS); + } else { // this message has been sent successfully, let's try next one. + pTask->msgInfo.retryCount = 0; + + // transtate msg has been sent to downstream successfully. let's transfer the fill-history task state + if (pTask->msgInfo.dispatchMsgType == STREAM_INPUT__TRANS_STATE) { + stDebug("s-task:%s dispatch transtate msgId:%d to downstream successfully, start to transfer state", id, msgId); + ASSERT(pTask->info.fillHistory == 1); + + code = streamTransferStateToStreamTask(pTask); + if (code != TSDB_CODE_SUCCESS) { // todo: do nothing if error happens + } + + // now ready for next data output + atomic_store_8(&pTask->outputInfo.status, TASK_OUTPUT_STATUS__NORMAL); + } else { + handleDispatchSuccessRsp(pTask, pRsp->downstreamTaskId); } } - - return TSDB_CODE_SUCCESS; - } - - qDebug("s-task:%s recv dispatch rsp from 0x%x, downstream task input status:%d code:%d", id, pRsp->downstreamTaskId, - pRsp->inputStatus, code); - - // there are other dispatch message not response yet - if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { - int32_t leftRsp = atomic_sub_fetch_32(&pTask->shuffleDispatcher.waitingRspCnt, 1); - qDebug("s-task:%s is shuffle, left waiting rsp %d", id, leftRsp); - if (leftRsp > 0) { - return 0; - } - } - - // transtate msg has been sent to downstream successfully. let's transfer the fill-history task state - SStreamDataBlock* p = pTask->msgInfo.pData; - if (p->type == STREAM_INPUT__TRANS_STATE) { - qDebug("s-task:%s dispatch transtate msg to downstream successfully, start to transfer state", id); - ASSERT(pTask->info.fillHistory == 1); - code = streamTransferStateToStreamTask(pTask); - if (code != TSDB_CODE_SUCCESS) { // todo: do nothing if error happens - } - - // now ready for next data output - atomic_store_8(&pTask->outputInfo.status, TASK_OUTPUT_STATUS__NORMAL); - return TSDB_CODE_SUCCESS; - } - - pTask->msgInfo.retryCount = 0; - ASSERT(pTask->outputInfo.status == TASK_OUTPUT_STATUS__WAIT); - - qDebug("s-task:%s output status is set to:%d", id, pTask->outputInfo.status); - - // the input queue of the (down stream) task that receive the output data is full, - // so the TASK_INPUT_STATUS_BLOCKED is rsp - if (pRsp->inputStatus == TASK_INPUT_STATUS__BLOCKED) { - pTask->inputInfo.status = TASK_INPUT_STATUS__BLOCKED; // block the input of current task, to push pressure to upstream - double el = 0; - if (pTask->msgInfo.blockingTs == 0) { - pTask->msgInfo.blockingTs = taosGetTimestampMs(); // record the blocking start time - } else { - el = (taosGetTimestampMs() - pTask->msgInfo.blockingTs) / 1000.0; - } - - int8_t ref = atomic_add_fetch_8(&pTask->status.timerActive, 1); - qError("s-task:%s inputQ of downstream task:0x%x is full, time:%" PRId64 - " wait for %dms and retry dispatch data, total wait:%.2fSec ref:%d", - id, pRsp->downstreamTaskId, pTask->msgInfo.blockingTs, DISPATCH_RETRY_INTERVAL_MS, el, ref); - streamRetryDispatchStreamBlock(pTask, DISPATCH_RETRY_INTERVAL_MS); - } else { // pipeline send data in output queue - // this message has been sent successfully, let's try next one. - destroyStreamDataBlock(pTask->msgInfo.pData); - pTask->msgInfo.pData = NULL; - - if (pTask->msgInfo.blockingTs != 0) { - int64_t el = taosGetTimestampMs() - pTask->msgInfo.blockingTs; - qDebug("s-task:%s downstream task:0x%x resume to normal from inputQ blocking, blocking time:%" PRId64 "ms", id, - pRsp->downstreamTaskId, el); - pTask->msgInfo.blockingTs = 0; - - // put data into inputQ of current task is also allowed - pTask->inputInfo.status = TASK_INPUT_STATUS__NORMAL; - } - - // now ready for next data output - atomic_store_8(&pTask->outputInfo.status, TASK_OUTPUT_STATUS__NORMAL); - - // otherwise, continue dispatch the first block to down stream task in pipeline - streamDispatchStreamBlock(pTask); } return 0; diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index d89817d236..a87bb00972 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -66,13 +66,13 @@ static int32_t doDumpResult(SStreamTask* pTask, SStreamQueueItem* pItem, SArray* SStreamDataBlock* pStreamBlocks = createStreamBlockFromResults(pItem, pTask, size, pRes); if (pStreamBlocks == NULL) { - qError("s-task:%s failed to create result stream data block, code:%s", pTask->id.idStr, tstrerror(terrno)); + stError("s-task:%s failed to create result stream data block, code:%s", pTask->id.idStr, tstrerror(terrno)); taosArrayDestroyEx(pRes, (FDelete)blockDataFreeRes); return TSDB_CODE_OUT_OF_MEMORY; } - qDebug("s-task:%s dump stream result data blocks, num:%d, size:%.2fMiB", pTask->id.idStr, numOfBlocks, - SIZE_IN_MB(size)); + stDebug("s-task:%s dump stream result data blocks, num:%d, size:%.2fMiB", pTask->id.idStr, numOfBlocks, + SIZE_IN_MiB(size)); int32_t code = doOutputResultBlockImpl(pTask, pStreamBlocks); if (code != TSDB_CODE_SUCCESS) { // back pressure and record position @@ -109,7 +109,7 @@ static int32_t streamTaskExecImpl(SStreamTask* pTask, SStreamQueueItem* pItem, i } if (pTask->inputInfo.status == TASK_INPUT_STATUS__BLOCKED) { - qWarn("s-task:%s downstream task inputQ blocked, idle for 1sec and retry exec task", pTask->id.idStr); + stWarn("s-task:%s downstream task inputQ blocked, idle for 1sec and retry exec task", pTask->id.idStr); taosMsleep(1000); continue; } @@ -121,7 +121,7 @@ static int32_t streamTaskExecImpl(SStreamTask* pTask, SStreamQueueItem* pItem, i resetTaskInfo(pExecutor); } - qError("unexpected stream execution, s-task:%s since %s", pTask->id.idStr, tstrerror(code)); + stError("unexpected stream execution, s-task:%s since %s", pTask->id.idStr, tstrerror(code)); continue; } @@ -137,7 +137,7 @@ static int32_t streamTaskExecImpl(SStreamTask* pTask, SStreamQueueItem* pItem, i taosArrayPush(pRes, &block); numOfBlocks += 1; - qDebug("s-task:%s(child %d) retrieve process completed, reqId:0x%" PRIx64 " dump results", pTask->id.idStr, + stDebug("s-task:%s(child %d) retrieve process completed, reqId:0x%" PRIx64 " dump results", pTask->id.idStr, pTask->info.selfChildId, pRetrieveBlock->reqId); } @@ -162,8 +162,8 @@ static int32_t streamTaskExecImpl(SStreamTask* pTask, SStreamQueueItem* pItem, i taosArrayPush(pRes, &block); - qDebug("s-task:%s (child %d) executed and get %d result blocks, size:%.2fMiB", pTask->id.idStr, - pTask->info.selfChildId, numOfBlocks, SIZE_IN_MB(size)); + stDebug("s-task:%s (child %d) executed and get %d result blocks, size:%.2fMiB", pTask->id.idStr, + pTask->info.selfChildId, numOfBlocks, SIZE_IN_MiB(size)); // current output should be dispatched to down stream nodes if (numOfBlocks >= STREAM_RESULT_DUMP_THRESHOLD || size >= STREAM_RESULT_DUMP_SIZE_THRESHOLD) { @@ -202,8 +202,8 @@ int32_t streamScanHistoryData(SStreamTask* pTask) { while (!finished) { if (streamTaskShouldPause(&pTask->status)) { - double el = (taosGetTimestampMs() - pTask->taskExecInfo.step1Start) / 1000.0; - qDebug("s-task:%s paused from the scan-history task, elapsed time:%.2fsec", pTask->id.idStr, el); + double el = (taosGetTimestampMs() - pTask->execInfo.step1Start) / 1000.0; + stDebug("s-task:%s paused from the scan-history task, elapsed time:%.2fsec", pTask->id.idStr, el); break; } @@ -221,7 +221,7 @@ int32_t streamScanHistoryData(SStreamTask* pTask) { } if (pTask->inputInfo.status == TASK_INPUT_STATUS__BLOCKED) { - qDebug("s-task:%s inputQ is blocked, wait for 10sec and retry", pTask->id.idStr); + stDebug("s-task:%s inputQ is blocked, wait for 10sec and retry", pTask->id.idStr); taosMsleep(10000); continue; } @@ -230,7 +230,7 @@ int32_t streamScanHistoryData(SStreamTask* pTask) { uint64_t ts = 0; code = qExecTask(exec, &output, &ts); if (code != TSDB_CODE_TSC_QUERY_KILLED && code != TSDB_CODE_SUCCESS) { - qError("%s scan-history data error occurred code:%s, continue scan", pTask->id.idStr, tstrerror(code)); + stError("%s scan-history data error occurred code:%s, continue scan", pTask->id.idStr, tstrerror(code)); continue; } @@ -248,7 +248,7 @@ int32_t streamScanHistoryData(SStreamTask* pTask) { size += blockDataGetSize(output) + sizeof(SSDataBlock) + sizeof(SColumnInfoData) * blockDataGetNumOfCols(&block); if ((++numOfBlocks) >= outputBatchSize || size >= STREAM_RESULT_DUMP_SIZE_THRESHOLD) { - qDebug("s-task:%s scan exec numOfBlocks:%d, output num-limit:%d, size-limit:%d reached", pTask->id.idStr, numOfBlocks, + stDebug("s-task:%s scan exec numOfBlocks:%d, output num-limit:%d, size-limit:%d reached", pTask->id.idStr, numOfBlocks, outputBatchSize, STREAM_RESULT_DUMP_SIZE_THRESHOLD); break; } @@ -258,7 +258,6 @@ int32_t streamScanHistoryData(SStreamTask* pTask) { SStreamDataBlock* pStreamBlocks = createStreamBlockFromResults(NULL, pTask, size, pRes); code = doOutputResultBlockImpl(pTask, pStreamBlocks); if (code != TSDB_CODE_SUCCESS) { - destroyStreamDataBlock(pStreamBlocks); return code; } @@ -277,14 +276,14 @@ static void waitForTaskIdle(SStreamTask* pTask, SStreamTask* pStreamTask) { int64_t st = taosGetTimestampMs(); while (!streamTaskIsIdle(pStreamTask)) { - qDebug("s-task:%s level:%d wait for stream task:%s to be idle, check again in 100ms", id, pTask->info.taskLevel, + stDebug("s-task:%s level:%d wait for stream task:%s to be idle, check again in 100ms", id, pTask->info.taskLevel, pStreamTask->id.idStr); taosMsleep(100); } double el = (taosGetTimestampMs() - st) / 1000.0; if (el > 0) { - qDebug("s-task:%s wait for stream task:%s for %.2fs to be idle", id, pStreamTask->id.idStr, el); + stDebug("s-task:%s wait for stream task:%s for %.2fs to be idle", id, pStreamTask->id.idStr, el); } } @@ -293,7 +292,7 @@ int32_t streamDoTransferStateToStreamTask(SStreamTask* pTask) { SStreamTask* pStreamTask = streamMetaAcquireTask(pMeta, pTask->streamTaskId.streamId, pTask->streamTaskId.taskId); if (pStreamTask == NULL) { - qError( + stError( "s-task:%s failed to find related stream task:0x%x, it may have been destroyed or closed, destroy the related " "fill-history task", pTask->id.idStr, (int32_t) pTask->streamTaskId.taskId); @@ -309,7 +308,7 @@ int32_t streamDoTransferStateToStreamTask(SStreamTask* pTask) { taosWUnLockLatch(&pMeta->lock); return TSDB_CODE_STREAM_TASK_NOT_EXIST; } else { - qDebug("s-task:%s fill-history task end, update related stream task:%s info, transfer exec state", pTask->id.idStr, + stDebug("s-task:%s fill-history task end, update related stream task:%s info, transfer exec state", pTask->id.idStr, pStreamTask->id.idStr); } @@ -326,7 +325,7 @@ int32_t streamDoTransferStateToStreamTask(SStreamTask* pTask) { } else { ASSERT(status == TASK_STATUS__NORMAL); pStreamTask->status.taskStatus = TASK_STATUS__HALT; - qDebug("s-task:%s halt by related fill-history task:%s", pStreamTask->id.idStr, pTask->id.idStr); + stDebug("s-task:%s halt by related fill-history task:%s", pStreamTask->id.idStr, pTask->id.idStr); } // wait for the stream task to handle all in the inputQ, and to be idle @@ -338,12 +337,12 @@ int32_t streamDoTransferStateToStreamTask(SStreamTask* pTask) { // When a task is idle with halt status, all data in inputQ are consumed. if (pStreamTask->info.taskLevel == TASK_LEVEL__SOURCE) { // update the scan data range for source task. - qDebug("s-task:%s level:%d stream task window %" PRId64 " - %" PRId64 " update to %" PRId64 " - %" PRId64 + stDebug("s-task:%s level:%d stream task window %" PRId64 " - %" PRId64 " update to %" PRId64 " - %" PRId64 ", status:%s, sched-status:%d", pStreamTask->id.idStr, TASK_LEVEL__SOURCE, pTimeWindow->skey, pTimeWindow->ekey, INT64_MIN, pTimeWindow->ekey, streamGetTaskStatusStr(TASK_STATUS__NORMAL), pStreamTask->status.schedStatus); } else { - qDebug("s-task:%s no need to update time window for non-source task", pStreamTask->id.idStr); + stDebug("s-task:%s no need to update time window for non-source task", pStreamTask->id.idStr); } // 1. expand the query time window for stream task of WAL scanner @@ -358,7 +357,7 @@ int32_t streamDoTransferStateToStreamTask(SStreamTask* pTask) { // pause, since the pause allowed attribute is not set yet. streamTaskResumeFromHalt(pStreamTask); - qDebug("s-task:%s fill-history task set status to be dropping, save the state into disk", pTask->id.idStr); + stDebug("s-task:%s fill-history task set status to be dropping, save the state into disk", pTask->id.idStr); // 4. free it and remove fill-history task from disk meta-store streamBuildAndSendDropTaskMsg(pTask->pMsgCb, pMeta->vgId, &pTask->id); @@ -386,7 +385,7 @@ int32_t streamDoTransferStateToStreamTask(SStreamTask* pTask) { pItem->type = STREAM_INPUT__REF_DATA_BLOCK; pItem->pBlock = pDelBlock; int32_t code = streamTaskPutDataIntoInputQ(pStreamTask, (SStreamQueueItem*)pItem); - qDebug("s-task:%s append dummy delete block,res:%d", pStreamTask->id.idStr, code); + stDebug("s-task:%s append dummy delete block,res:%d", pStreamTask->id.idStr, code); } streamSchedExec(pStreamTask); @@ -425,7 +424,7 @@ static void doSetStreamInputBlock(SStreamTask* pTask, const void* pInput, int64_ ASSERT(pTask->info.taskLevel == TASK_LEVEL__SOURCE); const SStreamDataSubmit* pSubmit = (const SStreamDataSubmit*)pInput; qSetMultiStreamInput(pExecutor, &pSubmit->submit, 1, STREAM_INPUT__DATA_SUBMIT); - qDebug("s-task:%s set submit blocks as source block completed, %p %p len:%d ver:%" PRId64, id, pSubmit, + stDebug("s-task:%s set submit blocks as source block completed, %p %p len:%d ver:%" PRId64, id, pSubmit, pSubmit->submit.msgStr, pSubmit->submit.msgLen, pSubmit->submit.ver); ASSERT((*pVer) <= pSubmit->submit.ver); (*pVer) = pSubmit->submit.ver; @@ -435,7 +434,7 @@ static void doSetStreamInputBlock(SStreamTask* pTask, const void* pInput, int64_ SArray* pBlockList = pBlock->blocks; int32_t numOfBlocks = taosArrayGetSize(pBlockList); - qDebug("s-task:%s set sdata blocks as input num:%d, ver:%" PRId64, id, numOfBlocks, pBlock->sourceVer); + stDebug("s-task:%s set sdata blocks as input num:%d, ver:%" PRId64, id, numOfBlocks, pBlock->sourceVer); qSetMultiStreamInput(pExecutor, pBlockList->pData, numOfBlocks, STREAM_INPUT__DATA_BLOCK); } else if (pItem->type == STREAM_INPUT__MERGED_SUBMIT) { @@ -443,7 +442,7 @@ static void doSetStreamInputBlock(SStreamTask* pTask, const void* pInput, int64_ SArray* pBlockList = pMerged->submits; int32_t numOfBlocks = taosArrayGetSize(pBlockList); - qDebug("s-task:%s %p set (merged) submit blocks as a batch, numOfBlocks:%d, ver:%" PRId64, id, pTask, numOfBlocks, + stDebug("s-task:%s %p set (merged) submit blocks as a batch, numOfBlocks:%d, ver:%" PRId64, id, pTask, numOfBlocks, pMerged->ver); qSetMultiStreamInput(pExecutor, pBlockList->pData, numOfBlocks, STREAM_INPUT__MERGED_SUBMIT); ASSERT((*pVer) <= pMerged->ver); @@ -471,7 +470,7 @@ int32_t streamProcessTranstateBlock(SStreamTask* pTask, SStreamDataBlock* pBlock int32_t remain = streamAlignTransferState(pTask); if (remain > 0) { streamFreeQitem((SStreamQueueItem*)pBlock); - qDebug("s-task:%s receive upstream transfer state msg, remain:%d", id, remain); + stDebug("s-task:%s receive upstream transfer state msg, remain:%d", id, remain); return 0; } } @@ -482,9 +481,9 @@ int32_t streamProcessTranstateBlock(SStreamTask* pTask, SStreamDataBlock* pBlock // transfer the ownership of executor state if (type == TASK_OUTPUT__FIXED_DISPATCH || type == TASK_OUTPUT__SHUFFLE_DISPATCH) { if (level == TASK_LEVEL__SOURCE) { - qDebug("s-task:%s add transfer-state block into outputQ", id); + stDebug("s-task:%s add transfer-state block into outputQ", id); } else { - qDebug("s-task:%s all upstream tasks send transfer-state block, add transfer-state block into outputQ", id); + stDebug("s-task:%s all upstream tasks send transfer-state block, add transfer-state block into outputQ", id); ASSERT(pTask->streamTaskId.taskId != 0 && pTask->info.fillHistory == 1); } @@ -502,7 +501,7 @@ int32_t streamProcessTranstateBlock(SStreamTask* pTask, SStreamDataBlock* pBlock } } else { // non-dispatch task, do task state transfer directly streamFreeQitem((SStreamQueueItem*)pBlock); - qDebug("s-task:%s non-dispatch task, start to transfer state directly", id); + stDebug("s-task:%s non-dispatch task, start to transfer state directly", id); ASSERT(pTask->info.fillHistory == 1); code = streamTransferStateToStreamTask(pTask); @@ -522,13 +521,13 @@ int32_t streamExecForAll(SStreamTask* pTask) { const char* id = pTask->id.idStr; // merge multiple input data if possible in the input queue. - qDebug("s-task:%s start to extract data block from inputQ", id); + stDebug("s-task:%s start to extract data block from inputQ", id); while (1) { int32_t numOfBlocks = 0; SStreamQueueItem* pInput = NULL; if (streamTaskShouldStop(&pTask->status)) { - qDebug("s-task:%s stream task is stopped", id); + stDebug("s-task:%s stream task is stopped", id); break; } @@ -554,8 +553,12 @@ int32_t streamExecForAll(SStreamTask* pTask) { if (pTask->info.taskLevel == TASK_LEVEL__SINK) { ASSERT(type == STREAM_INPUT__DATA_BLOCK || type == STREAM_INPUT__CHECKPOINT); + // here only handle the data block sink operation if (type == STREAM_INPUT__DATA_BLOCK) { - qDebug("s-task:%s sink task start to sink %d blocks", id, numOfBlocks); + int32_t blockSize = streamQueueItemGetSize(pInput); + pTask->execInfo.sink.bytes += blockSize; + + stDebug("s-task:%s sink task start to sink %d blocks, size:%.2fKiB", id, numOfBlocks, SIZE_IN_KiB(blockSize)); doOutputResultBlockImpl(pTask, (SStreamDataBlock*)pInput); continue; } @@ -564,7 +567,7 @@ int32_t streamExecForAll(SStreamTask* pTask) { int64_t st = taosGetTimestampMs(); const SStreamQueueItem* pItem = pInput; - qDebug("s-task:%s start to process batch of blocks, num:%d, type:%d", id, numOfBlocks, pItem->type); + stDebug("s-task:%s start to process batch of blocks, num:%d, type:%d", id, numOfBlocks, pItem->type); int64_t ver = pTask->chkInfo.checkpointVer; doSetStreamInputBlock(pTask, pInput, &ver, id); @@ -574,14 +577,14 @@ int32_t streamExecForAll(SStreamTask* pTask) { streamTaskExecImpl(pTask, pInput, &resSize, &totalBlocks); double el = (taosGetTimestampMs() - st) / 1000.0; - qDebug("s-task:%s batch of input blocks exec end, elapsed time:%.2fs, result size:%.2fMiB, numOfBlocks:%d", id, el, - SIZE_IN_MB(resSize), totalBlocks); + stDebug("s-task:%s batch of input blocks exec end, elapsed time:%.2fs, result size:%.2fMiB, numOfBlocks:%d", id, el, + SIZE_IN_MiB(resSize), totalBlocks); // update the currentVer if processing the submit blocks. ASSERT(pTask->chkInfo.checkpointVer <= pTask->chkInfo.nextProcessVer && ver >= pTask->chkInfo.checkpointVer); if (ver != pTask->chkInfo.checkpointVer) { - qDebug("s-task:%s update checkpointVer(unsaved) from %" PRId64 " to %" PRId64 " , currentVer:%" PRId64, + stDebug("s-task:%s update checkpointVer(unsaved) from %" PRId64 " to %" PRId64 ", nextProcessVer:%" PRId64, pTask->id.idStr, pTask->chkInfo.checkpointVer, ver, pTask->chkInfo.nextProcessVer); pTask->chkInfo.checkpointVer = ver; } @@ -591,8 +594,8 @@ int32_t streamExecForAll(SStreamTask* pTask) { // todo other thread may change the status // do nothing after sync executor state to storage backend, untill the vnode-level checkpoint is completed. if (type == STREAM_INPUT__CHECKPOINT) { - qDebug("s-task:%s checkpoint block received, set the status:%s", pTask->id.idStr, - streamGetTaskStatusStr(pTask->status.taskStatus)); + stDebug("s-task:%s checkpoint block received, set status:%s", pTask->id.idStr, + streamGetTaskStatusStr(pTask->status.taskStatus)); streamTaskBuildCheckpoint(pTask); return 0; } @@ -627,14 +630,14 @@ int32_t streamTryExec(SStreamTask* pTask) { atomic_store_8(&pTask->status.schedStatus, TASK_SCHED_STATUS__INACTIVE); taosThreadMutexUnlock(&pTask->lock); - qDebug("s-task:%s exec completed, status:%s, sched-status:%d", id, + stDebug("s-task:%s exec completed, status:%s, sched-status:%d", id, streamGetTaskStatusStr(pTask->status.taskStatus), pTask->status.schedStatus); return 0; } taosThreadMutexUnlock(&pTask->lock); } } else { - qDebug("s-task:%s already started to exec by other thread, status:%s, sched-status:%d", id, + stDebug("s-task:%s already started to exec by other thread, status:%s, sched-status:%d", id, streamGetTaskStatusStr(pTask->status.taskStatus), pTask->status.schedStatus); } @@ -642,7 +645,7 @@ int32_t streamTryExec(SStreamTask* pTask) { } int32_t streamTaskReleaseState(SStreamTask* pTask) { - qDebug("s-task:%s release exec state", pTask->id.idStr); + stDebug("s-task:%s release exec state", pTask->id.idStr); void* pExecutor = pTask->exec.pExecutor; if (pExecutor != NULL) { int32_t code = qStreamOperatorReleaseState(pExecutor); @@ -653,7 +656,7 @@ int32_t streamTaskReleaseState(SStreamTask* pTask) { } int32_t streamTaskReloadState(SStreamTask* pTask) { - qDebug("s-task:%s reload exec state", pTask->id.idStr); + stDebug("s-task:%s reload exec state", pTask->id.idStr); void* pExecutor = pTask->exec.pExecutor; if (pExecutor != NULL) { int32_t code = qStreamOperatorReloadState(pExecutor); @@ -667,7 +670,7 @@ int32_t streamAlignTransferState(SStreamTask* pTask) { int32_t numOfUpstream = taosArrayGetSize(pTask->pUpstreamInfoList); int32_t old = atomic_val_compare_exchange_32(&pTask->transferStateAlignCnt, 0, numOfUpstream); if (old == 0) { - qDebug("s-task:%s set the transfer state aligncnt %d", pTask->id.idStr, numOfUpstream); + stDebug("s-task:%s set the transfer state aligncnt %d", pTask->id.idStr, numOfUpstream); } return atomic_sub_fetch_32(&pTask->transferStateAlignCnt, 1); diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 93f4b7c4dd..e98a292d37 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -115,7 +115,7 @@ SStreamMeta* streamMetaOpen(const char* path, void* ahandle, FTaskExpand expandF SStreamMeta* pMeta = taosMemoryCalloc(1, sizeof(SStreamMeta)); if (pMeta == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; - qError("vgId:%d failed to prepare stream meta, alloc size:%" PRIzu ", out of memory", vgId, sizeof(SStreamMeta)); + stError("vgId:%d failed to prepare stream meta, alloc size:%" PRIzu ", out of memory", vgId, sizeof(SStreamMeta)); return NULL; } @@ -151,6 +151,11 @@ SStreamMeta* streamMetaOpen(const char* path, void* ahandle, FTaskExpand expandF goto _err; } + pMeta->startInfo.pReadyTaskSet = taosHashInit(64, fp, false, HASH_NO_LOCK); + if (pMeta->startInfo.pReadyTaskSet == NULL) { + + } + pMeta->pHbInfo = taosMemoryCalloc(1, sizeof(SMetaHbInfo)); if (pMeta->pHbInfo == NULL) { goto _err; @@ -190,15 +195,16 @@ SStreamMeta* streamMetaOpen(const char* path, void* ahandle, FTaskExpand expandF pMeta->chkpId = streamGetLatestCheckpointId(pMeta); pMeta->streamBackend = streamBackendInit(pMeta->path, pMeta->chkpId); while (pMeta->streamBackend == NULL) { - taosMsleep(2 * 1000); + taosMsleep(500); pMeta->streamBackend = streamBackendInit(pMeta->path, pMeta->chkpId); if (pMeta->streamBackend == NULL) { - qError("vgId:%d failed to init stream backend", pMeta->vgId); - qInfo("vgId:%d retry to init stream backend", pMeta->vgId); + stError("vgId:%d failed to init stream backend", pMeta->vgId); + stInfo("vgId:%d retry to init stream backend", pMeta->vgId); } } pMeta->streamBackendRid = taosAddRef(streamBackendId, pMeta->streamBackend); + pMeta->role = NODE_ROLE_UNINIT; code = streamBackendLoadCheckpointInfo(pMeta); taosInitRWLatch(&pMeta->lock); @@ -206,7 +212,7 @@ SStreamMeta* streamMetaOpen(const char* path, void* ahandle, FTaskExpand expandF pMeta->numOfPausedTasks = 0; pMeta->numOfStreamTasks = 0; - qInfo("vgId:%d open stream meta successfully, latest checkpoint:%" PRId64 ", stage:%" PRId64, vgId, pMeta->chkpId, + stInfo("vgId:%d open stream meta successfully, latest checkpoint:%" PRId64 ", stage:%" PRId64, vgId, pMeta->chkpId, stage); return pMeta; @@ -219,10 +225,10 @@ _err: if (pMeta->db) tdbClose(pMeta->db); if (pMeta->pHbInfo) taosMemoryFreeClear(pMeta->pHbInfo); if (pMeta->pUpdateTaskSet) taosHashCleanup(pMeta->pUpdateTaskSet); - + if (pMeta->startInfo.pReadyTaskSet) taosHashCleanup(pMeta->startInfo.pReadyTaskSet); taosMemoryFree(pMeta); - qError("failed to open stream meta"); + stError("failed to open stream meta"); return NULL; } @@ -231,6 +237,7 @@ int32_t streamMetaReopen(SStreamMeta* pMeta) { pMeta->streamBackendRid = -1; pMeta->streamBackend = NULL; + pMeta->role = NODE_ROLE_UNINIT; char* defaultPath = taosMemoryCalloc(1, strlen(pMeta->path) + 128); sprintf(defaultPath, "%s%s%s", pMeta->path, TD_DIRSEP, "state"); @@ -245,7 +252,7 @@ int32_t streamMetaReopen(SStreamMeta* pMeta) { code = taosRenameFile(newPath, defaultPath); if (code != 0) { terrno = TAOS_SYSTEM_ERROR(code); - qError("vgId:%d failed to rename file, from %s to %s, code:%s", pMeta->vgId, newPath, defaultPath, + stError("vgId:%d failed to rename file, from %s to %s, code:%s", pMeta->vgId, newPath, defaultPath, tstrerror(terrno)); taosMemoryFree(defaultPath); @@ -256,14 +263,14 @@ int32_t streamMetaReopen(SStreamMeta* pMeta) { pMeta->streamBackend = streamBackendInit(pMeta->path, pMeta->chkpId); while (pMeta->streamBackend == NULL) { - taosMsleep(2 * 1000); + taosMsleep(500); pMeta->streamBackend = streamBackendInit(pMeta->path, pMeta->chkpId); if (pMeta->streamBackend == NULL) { - qError("vgId:%d failed to init stream backend", pMeta->vgId); - qInfo("vgId:%d retry to init stream backend", pMeta->vgId); - // return -1; + stError("vgId:%d failed to init stream backend", pMeta->vgId); + stInfo("vgId:%d retry to init stream backend", pMeta->vgId); } } + pMeta->streamBackendRid = taosAddRef(streamBackendId, pMeta->streamBackend); streamBackendLoadCheckpointInfo(pMeta); @@ -277,7 +284,7 @@ void streamMetaClear(SStreamMeta* pMeta) { // release the ref by timer if (p->info.triggerParam != 0 && p->info.fillHistory == 0) { // one more ref in timer - qDebug("s-task:%s stop schedTimer, and (before) desc ref:%d", p->id.idStr, p->refCnt); + stDebug("s-task:%s stop schedTimer, and (before) desc ref:%d", p->id.idStr, p->refCnt); taosTmrStop(p->schedInfo.pTimer); p->info.triggerParam = 0; streamMetaReleaseTask(pMeta, p); @@ -299,7 +306,7 @@ void streamMetaClear(SStreamMeta* pMeta) { } void streamMetaClose(SStreamMeta* pMeta) { - qDebug("start to close stream meta"); + stDebug("start to close stream meta"); if (pMeta == NULL) { return; } @@ -315,7 +322,7 @@ void streamMetaClose(SStreamMeta* pMeta) { void streamMetaCloseImpl(void* arg) { SStreamMeta* pMeta = arg; - qDebug("start to do-close stream meta"); + stDebug("start to do-close stream meta"); if (pMeta == NULL) { return; } @@ -334,13 +341,15 @@ void streamMetaCloseImpl(void* arg) { taosHashCleanup(pMeta->pTasksMap); taosHashCleanup(pMeta->pTaskBackendUnique); taosHashCleanup(pMeta->pUpdateTaskSet); + taosHashCleanup(pMeta->startInfo.pReadyTaskSet); taosMemoryFree(pMeta->pHbInfo); taosMemoryFree(pMeta->path); taosThreadMutexDestroy(&pMeta->backendMutex); + pMeta->role = NODE_ROLE_UNINIT; taosMemoryFree(pMeta); - qDebug("end to close stream meta"); + stDebug("end to close stream meta"); } int32_t streamMetaSaveTask(SStreamMeta* pMeta, SStreamTask* pTask) { @@ -364,7 +373,7 @@ int32_t streamMetaSaveTask(SStreamMeta* pMeta, SStreamTask* pTask) { int64_t id[2] = {pTask->id.streamId, pTask->id.taskId}; if (tdbTbUpsert(pMeta->pTaskDb, id, STREAM_TASK_KEY_LEN, buf, len, pMeta->txn) < 0) { - qError("s-task:%s save to disk failed, code:%s", pTask->id.idStr, tstrerror(terrno)); + stError("s-task:%s save to disk failed, code:%s", pTask->id.idStr, tstrerror(terrno)); return -1; } @@ -376,10 +385,10 @@ int32_t streamMetaRemoveTask(SStreamMeta* pMeta, STaskId* pTaskId) { int64_t key[2] = {pTaskId->streamId, pTaskId->taskId}; int32_t code = tdbTbDelete(pMeta->pTaskDb, key, STREAM_TASK_KEY_LEN, pMeta->txn); if (code != 0) { - qError("vgId:%d failed to remove task:0x%x from metastore, code:%s", pMeta->vgId, (int32_t) pTaskId->taskId, + stError("vgId:%d failed to remove task:0x%x from metastore, code:%s", pMeta->vgId, (int32_t) pTaskId->taskId, tstrerror(terrno)); } else { - qDebug("vgId:%d remove task:0x%x from metastore", pMeta->vgId, (int32_t) pTaskId->taskId); + stDebug("vgId:%d remove task:0x%x from metastore", pMeta->vgId, (int32_t) pTaskId->taskId); } return code; @@ -454,7 +463,7 @@ SStreamTask* streamMetaAcquireTask(SStreamMeta* pMeta, int64_t streamId, int32_t if (!streamTaskShouldStop(&(*ppTask)->status)) { int32_t ref = atomic_add_fetch_32(&(*ppTask)->refCnt, 1); taosRUnLockLatch(&pMeta->lock); - qTrace("s-task:%s acquire task, ref:%d", (*ppTask)->id.idStr, ref); + stTrace("s-task:%s acquire task, ref:%d", (*ppTask)->id.idStr, ref); return *ppTask; } } @@ -466,13 +475,13 @@ SStreamTask* streamMetaAcquireTask(SStreamMeta* pMeta, int64_t streamId, int32_t void streamMetaReleaseTask(SStreamMeta* UNUSED_PARAM(pMeta), SStreamTask* pTask) { int32_t ref = atomic_sub_fetch_32(&pTask->refCnt, 1); if (ref > 0) { - qTrace("s-task:%s release task, ref:%d", pTask->id.idStr, ref); + stTrace("s-task:%s release task, ref:%d", pTask->id.idStr, ref); } else if (ref == 0) { ASSERT(streamTaskShouldStop(&pTask->status)); - qTrace("s-task:%s all refs are gone, free it", pTask->id.idStr); + stTrace("s-task:%s all refs are gone, free it", pTask->id.idStr); tFreeStreamTask(pTask); } else if (ref < 0) { - qError("task ref is invalid, ref:%d, %s", ref, pTask->id.idStr); + stError("task ref is invalid, ref:%d, %s", ref, pTask->id.idStr); } } @@ -498,17 +507,17 @@ int32_t streamMetaUnregisterTask(SStreamMeta* pMeta, int64_t streamId, int32_t t pTask = *ppTask; if (streamTaskShouldPause(&pTask->status)) { int32_t num = atomic_sub_fetch_32(&pMeta->numOfPausedTasks, 1); - qInfo("vgId:%d s-task:%s drop stream task. pause task num:%d", pMeta->vgId, pTask->id.idStr, num); + stInfo("vgId:%d s-task:%s drop stream task. pause task num:%d", pMeta->vgId, pTask->id.idStr, num); } atomic_store_8(&pTask->status.taskStatus, TASK_STATUS__DROPPING); } else { - qDebug("vgId:%d failed to find the task:0x%x, it may be dropped already", pMeta->vgId, taskId); + stDebug("vgId:%d failed to find the task:0x%x, it may be dropped already", pMeta->vgId, taskId); taosWUnLockLatch(&pMeta->lock); return 0; } taosWUnLockLatch(&pMeta->lock); - qDebug("s-task:0x%x set task status:%s and start to unregister it", taskId, + stDebug("s-task:0x%x set task status:%s and start to unregister it", taskId, streamGetTaskStatusStr(TASK_STATUS__DROPPING)); while (1) { @@ -522,7 +531,7 @@ int32_t streamMetaUnregisterTask(SStreamMeta* pMeta, int64_t streamId, int32_t t } taosMsleep(10); - qDebug("s-task:%s wait for quit from timer", (*ppTask)->id.idStr); + stDebug("s-task:%s wait for quit from timer", (*ppTask)->id.idStr); taosRUnLockLatch(&pMeta->lock); } else { taosRUnLockLatch(&pMeta->lock); @@ -551,7 +560,7 @@ int32_t streamMetaUnregisterTask(SStreamMeta* pMeta, int64_t streamId, int32_t t doRemoveIdFromList(pMeta, (int32_t)taosArrayGetSize(pMeta->pTaskList), &pTask->id); if (pTask->info.triggerParam != 0 && pTask->info.fillHistory == 0) { - qDebug("s-task:%s stop schedTimer, and (before) desc ref:%d", pTask->id.idStr, pTask->refCnt); + stDebug("s-task:%s stop schedTimer, and (before) desc ref:%d", pTask->id.idStr, pTask->refCnt); taosTmrStop(pTask->schedInfo.pTimer); pTask->info.triggerParam = 0; streamMetaReleaseTask(pMeta, pTask); @@ -560,7 +569,7 @@ int32_t streamMetaUnregisterTask(SStreamMeta* pMeta, int64_t streamId, int32_t t streamMetaRemoveTask(pMeta, &id); streamMetaReleaseTask(pMeta, pTask); } else { - qDebug("vgId:%d failed to find the task:0x%x, it may have been dropped already", pMeta->vgId, taskId); + stDebug("vgId:%d failed to find the task:0x%x, it may have been dropped already", pMeta->vgId, taskId); } taosWUnLockLatch(&pMeta->lock); @@ -581,18 +590,18 @@ int32_t streamMetaBegin(SStreamMeta* pMeta) { // todo add error log int32_t streamMetaCommit(SStreamMeta* pMeta) { if (tdbCommit(pMeta->db, pMeta->txn) < 0) { - qError("vgId:%d failed to commit stream meta", pMeta->vgId); + stError("vgId:%d failed to commit stream meta", pMeta->vgId); return -1; } if (tdbPostCommit(pMeta->db, pMeta->txn) < 0) { - qError("vgId:%d failed to do post-commit stream meta", pMeta->vgId); + stError("vgId:%d failed to do post-commit stream meta", pMeta->vgId); return -1; } if (tdbBegin(pMeta->db, &pMeta->txn, tdbDefaultMalloc, tdbDefaultFree, NULL, TDB_TXN_WRITE | TDB_TXN_READ_UNCOMMITTED) < 0) { - qError("vgId:%d failed to begin trans", pMeta->vgId); + stError("vgId:%d failed to begin trans", pMeta->vgId); return -1; } @@ -628,7 +637,7 @@ int64_t streamGetLatestCheckpointId(SStreamMeta* pMeta) { chkpId = TMAX(chkpId, info.checkpointId); } - qDebug("get max chkp id: %" PRId64 "", chkpId); + stDebug("get max chkp id: %" PRId64 "", chkpId); tdbFree(pKey); tdbFree(pVal); @@ -648,10 +657,10 @@ int32_t streamMetaLoadAllTasks(SStreamMeta* pMeta) { TBC* pCur = NULL; int32_t vgId = pMeta->vgId; - qInfo("vgId:%d load stream tasks from meta files", vgId); + stInfo("vgId:%d load stream tasks from meta files", vgId); if (tdbTbcOpen(pMeta->pTaskDb, &pCur, NULL) < 0) { - qError("vgId:%d failed to open stream meta, code:%s", vgId, tstrerror(terrno)); + stError("vgId:%d failed to open stream meta, code:%s", vgId, tstrerror(terrno)); return -1; } @@ -667,7 +676,7 @@ int32_t streamMetaLoadAllTasks(SStreamMeta* pMeta) { SStreamTask* pTask = taosMemoryCalloc(1, sizeof(SStreamTask)); if (pTask == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; - qError("vgId:%d failed to load stream task from meta-files, code:%s", vgId, tstrerror(terrno)); + stError("vgId:%d failed to load stream task from meta-files, code:%s", vgId, tstrerror(terrno)); doClear(pKey, pVal, pCur, pRecycleList); return -1; } @@ -677,7 +686,7 @@ int32_t streamMetaLoadAllTasks(SStreamMeta* pMeta) { tDecoderClear(&decoder); doClear(pKey, pVal, pCur, pRecycleList); tFreeStreamTask(pTask); - qError( + stError( "vgId:%d stream read incompatible data, rm %s/vnode/vnode*/tq/stream if taosd cannot start, and rebuild stream " "manually", vgId, tsDataDir); return -1; @@ -692,7 +701,7 @@ int32_t streamMetaLoadAllTasks(SStreamMeta* pMeta) { taosArrayPush(pRecycleList, &id); int32_t total = taosArrayGetSize(pRecycleList); - qDebug("s-task:0x%x is already dropped, add into recycle list, total:%d", taskId, total); + stDebug("s-task:0x%x is already dropped, add into recycle list, total:%d", taskId, total); continue; } @@ -736,7 +745,7 @@ int32_t streamMetaLoadAllTasks(SStreamMeta* pMeta) { tdbFree(pKey); tdbFree(pVal); if (tdbTbcClose(pCur) < 0) { - qError("vgId:%d failed to close meta-file cursor", vgId); + stError("vgId:%d failed to close meta-file cursor", vgId); taosArrayDestroy(pRecycleList); return -1; } @@ -750,7 +759,7 @@ int32_t streamMetaLoadAllTasks(SStreamMeta* pMeta) { int32_t numOfTasks = taosArrayGetSize(pMeta->pTaskList); ASSERT(pMeta->numOfStreamTasks <= numOfTasks && pMeta->numOfPausedTasks <= numOfTasks); - qDebug("vgId:%d load %d tasks into meta from disk completed, streamTask:%d, paused:%d", pMeta->vgId, numOfTasks, + stDebug("vgId:%d load %d tasks into meta from disk completed, streamTask:%d, paused:%d", pMeta->vgId, numOfTasks, pMeta->numOfStreamTasks, pMeta->numOfPausedTasks); taosArrayDestroy(pRecycleList); return 0; @@ -766,6 +775,8 @@ int32_t tEncodeStreamHbMsg(SEncoder* pEncoder, const SStreamHbMsg* pReq) { if (tEncodeI64(pEncoder, ps->id.streamId) < 0) return -1; if (tEncodeI32(pEncoder, ps->id.taskId) < 0) return -1; if (tEncodeI32(pEncoder, ps->status) < 0) return -1; + if (tEncodeI32(pEncoder, ps->stage) < 0) return -1; + if (tEncodeI32(pEncoder, ps->nodeId) < 0) return -1; } tEndEncode(pEncoder); return pEncoder->pos; @@ -778,15 +789,17 @@ int32_t tDecodeStreamHbMsg(SDecoder* pDecoder, SStreamHbMsg* pReq) { pReq->pTaskStatus = taosArrayInit(pReq->numOfTasks, sizeof(STaskStatusEntry)); for (int32_t i = 0; i < pReq->numOfTasks; ++i) { - STaskStatusEntry hb = {0}; - if (tDecodeI64(pDecoder, &hb.id.streamId) < 0) return -1; - int32_t taskId = 0; + int32_t taskId = 0; + STaskStatusEntry entry = {0}; + + if (tDecodeI64(pDecoder, &entry.id.streamId) < 0) return -1; if (tDecodeI32(pDecoder, &taskId) < 0) return -1; + if (tDecodeI32(pDecoder, &entry.status) < 0) return -1; + if (tDecodeI32(pDecoder, &entry.stage) < 0) return -1; + if (tDecodeI32(pDecoder, &entry.nodeId) < 0) return -1; - hb.id.taskId = taskId; - if (tDecodeI32(pDecoder, &hb.status) < 0) return -1; - - taosArrayPush(pReq->pTaskStatus, &hb); + entry.id.taskId = taskId; + taosArrayPush(pReq->pTaskStatus, &entry); } tEndDecode(pDecoder); @@ -812,14 +825,14 @@ void metaHbToMnode(void* param, void* tmrId) { // need to stop, stop now if (pMeta->pHbInfo->stopFlag == STREAM_META_WILL_STOP) { pMeta->pHbInfo->stopFlag = STREAM_META_OK_TO_STOP; - qDebug("vgId:%d jump out of meta timer", pMeta->vgId); + stDebug("vgId:%d jump out of meta timer", pMeta->vgId); taosReleaseRef(streamMetaId, rid); return; } // not leader not send msg - if (!pMeta->leader) { - qInfo("vgId:%d follower not send hb to mnode", pMeta->vgId); + if (pMeta->role == NODE_ROLE_FOLLOWER) { + stInfo("vgId:%d follower not send hb to mnode", pMeta->vgId); taosReleaseRef(streamMetaId, rid); pMeta->pHbInfo->hbStart = 0; return; @@ -836,7 +849,7 @@ void metaHbToMnode(void* param, void* tmrId) { return; } - qDebug("vgId:%d build stream task hb, leader:%d", pMeta->vgId, pMeta->leader); + stDebug("vgId:%d build stream task hb, leader:%d", pMeta->vgId, (pMeta->role == NODE_ROLE_LEADER)); SStreamHbMsg hbMsg = {0}; taosRLockLatch(&pMeta->lock); @@ -855,7 +868,8 @@ void metaHbToMnode(void* param, void* tmrId) { continue; } - STaskStatusEntry entry = {.id = *pId, .status = (*pTask)->status.taskStatus}; + STaskStatusEntry entry = { + .id = *pId, .status = (*pTask)->status.taskStatus, .nodeId = pMeta->vgId, .stage = pMeta->stage}; taosArrayPush(hbMsg.pTaskStatus, &entry); if (!hasValEpset) { @@ -873,7 +887,7 @@ void metaHbToMnode(void* param, void* tmrId) { tEncodeSize(tEncodeStreamHbMsg, &hbMsg, tlen, code); if (code < 0) { - qError("vgId:%d encode stream hb msg failed, code:%s", pMeta->vgId, tstrerror(code)); + stError("vgId:%d encode stream hb msg failed, code:%s", pMeta->vgId, tstrerror(code)); taosArrayDestroy(hbMsg.pTaskStatus); taosReleaseRef(streamMetaId, rid); return; @@ -881,7 +895,7 @@ void metaHbToMnode(void* param, void* tmrId) { void* buf = rpcMallocCont(tlen); if (buf == NULL) { - qError("vgId:%d encode stream hb msg failed, code:%s", pMeta->vgId, tstrerror(TSDB_CODE_OUT_OF_MEMORY)); + stError("vgId:%d encode stream hb msg failed, code:%s", pMeta->vgId, tstrerror(TSDB_CODE_OUT_OF_MEMORY)); taosArrayDestroy(hbMsg.pTaskStatus); taosReleaseRef(streamMetaId, rid); return; @@ -891,7 +905,7 @@ void metaHbToMnode(void* param, void* tmrId) { tEncoderInit(&encoder, buf, tlen); if ((code = tEncodeStreamHbMsg(&encoder, &hbMsg)) < 0) { rpcFreeCont(buf); - qError("vgId:%d encode stream hb msg failed, code:%s", pMeta->vgId, tstrerror(code)); + stError("vgId:%d encode stream hb msg failed, code:%s", pMeta->vgId, tstrerror(code)); taosArrayDestroy(hbMsg.pTaskStatus); taosReleaseRef(streamMetaId, rid); return; @@ -904,11 +918,11 @@ void metaHbToMnode(void* param, void* tmrId) { pMeta->pHbInfo->hbCount += 1; - qDebug("vgId:%d, build and send hb to mnode, numOfTasks:%d total:%d", pMeta->vgId, hbMsg.numOfTasks, + stDebug("vgId:%d, build and send hb to mnode, numOfTasks:%d total:%d", pMeta->vgId, hbMsg.numOfTasks, pMeta->pHbInfo->hbCount); tmsgSendReq(&epset, &msg); } else { - qDebug("vgId:%d no tasks and no mnd epset, not send stream hb to mnode", pMeta->vgId); + stDebug("vgId:%d no tasks and no mnd epset, not send stream hb to mnode", pMeta->vgId); } taosArrayDestroy(hbMsg.pTaskStatus); @@ -941,8 +955,8 @@ static bool hasStreamTaskInTimer(SStreamMeta* pMeta) { void streamMetaNotifyClose(SStreamMeta* pMeta) { int32_t vgId = pMeta->vgId; - qDebug("vgId:%d notify all stream tasks that the vnode is closing. isLeader:%d startHb%" PRId64 ", totalHb:%d", vgId, - pMeta->leader, pMeta->pHbInfo->hbStart, pMeta->pHbInfo->hbCount); + stDebug("vgId:%d notify all stream tasks that the vnode is closing. isLeader:%d startHb%" PRId64 ", totalHb:%d", vgId, + (pMeta->role == NODE_ROLE_LEADER), pMeta->pHbInfo->hbStart, pMeta->pHbInfo->hbCount); taosWLockLatch(&pMeta->lock); @@ -954,31 +968,31 @@ void streamMetaNotifyClose(SStreamMeta* pMeta) { } SStreamTask* pTask = *(SStreamTask**)pIter; - qDebug("vgId:%d s-task:%s set closing flag", vgId, pTask->id.idStr); + stDebug("vgId:%d s-task:%s set closing flag", vgId, pTask->id.idStr); streamTaskStop(pTask); } taosWUnLockLatch(&pMeta->lock); // wait for the stream meta hb function stopping - if (pMeta->leader) { + if (pMeta->role == NODE_ROLE_LEADER) { pMeta->pHbInfo->stopFlag = STREAM_META_WILL_STOP; while (pMeta->pHbInfo->stopFlag != STREAM_META_OK_TO_STOP) { taosMsleep(100); - qDebug("vgId:%d wait for meta to stop timer", pMeta->vgId); + stDebug("vgId:%d wait for meta to stop timer", pMeta->vgId); } } - qDebug("vgId:%d start to check all tasks", vgId); + stDebug("vgId:%d start to check all tasks", vgId); int64_t st = taosGetTimestampMs(); while (hasStreamTaskInTimer(pMeta)) { - qDebug("vgId:%d some tasks in timer, wait for 100ms and recheck", pMeta->vgId); + stDebug("vgId:%d some tasks in timer, wait for 100ms and recheck", pMeta->vgId); taosMsleep(100); } int64_t el = taosGetTimestampMs() - st; - qDebug("vgId:%d all stream tasks are not in timer, continue close, elapsed time:%" PRId64 " ms", pMeta->vgId, el); + stDebug("vgId:%d all stream tasks are not in timer, continue close, elapsed time:%" PRId64 " ms", pMeta->vgId, el); } void streamMetaStartHb(SStreamMeta* pMeta) { @@ -987,3 +1001,8 @@ void streamMetaStartHb(SStreamMeta* pMeta) { *pRid = pMeta->rid; metaHbToMnode(pRid, NULL); } + +void streamMetaInitForSnode(SStreamMeta* pMeta) { + pMeta->stage = 0; + pMeta->role = NODE_ROLE_LEADER; +} \ No newline at end of file diff --git a/source/libs/stream/src/streamQueue.c b/source/libs/stream/src/streamQueue.c index d3d114d4aa..abf10487de 100644 --- a/source/libs/stream/src/streamQueue.c +++ b/source/libs/stream/src/streamQueue.c @@ -65,7 +65,7 @@ SStreamQueue* streamQueueOpen(int64_t cap) { } void streamQueueClose(SStreamQueue* pQueue, int32_t taskId) { - qDebug("s-task:0x%x free the queue:%p, items in queue:%d", taskId, pQueue->pQueue, taosQueueItemSize(pQueue->pQueue)); + stDebug("s-task:0x%x free the queue:%p, items in queue:%d", taskId, pQueue->pQueue, taosQueueItemSize(pQueue->pQueue)); streamQueueCleanup(pQueue); taosFreeQall(pQueue->qall); @@ -102,64 +102,6 @@ void streamQueueProcessFail(SStreamQueue* queue) { atomic_store_8(&queue->status, STREAM_QUEUE__FAILED); } -#if 0 -bool streamQueueResEmpty(const SStreamQueueRes* pRes) { - // - return true; -} -int64_t streamQueueResSize(const SStreamQueueRes* pRes) { return pRes->size; } -SStreamQueueNode* streamQueueResFront(SStreamQueueRes* pRes) { return pRes->head; } -SStreamQueueNode* streamQueueResPop(SStreamQueueRes* pRes) { - SStreamQueueNode* pRet = pRes->head; - pRes->head = pRes->head->next; - return pRet; -} - -void streamQueueResClear(SStreamQueueRes* pRes) { - while (pRes->head) { - SStreamQueueNode* pNode = pRes->head; - streamFreeQitem(pRes->head->item); - pRes->head = pNode; - } -} - -SStreamQueueRes streamQueueBuildRes(SStreamQueueNode* pTail) { - int64_t size = 0; - SStreamQueueNode* head = NULL; - - while (pTail) { - SStreamQueueNode* pTmp = pTail->next; - pTail->next = head; - head = pTail; - pTail = pTmp; - size++; - } - - return (SStreamQueueRes){.head = head, .size = size}; -} - -bool streamQueueHasTask(const SStreamQueue1* pQueue) { return atomic_load_ptr(pQueue->pHead); } -int32_t streamQueuePush(SStreamQueue1* pQueue, SStreamQueueItem* pItem) { - SStreamQueueNode* pNode = taosMemoryMalloc(sizeof(SStreamQueueNode)); - pNode->item = pItem; - SStreamQueueNode* pHead = atomic_load_ptr(pQueue->pHead); - while (1) { - pNode->next = pHead; - SStreamQueueNode* pOld = atomic_val_compare_exchange_ptr(pQueue->pHead, pHead, pNode); - if (pOld == pHead) { - break; - } - } - return 0; -} - -SStreamQueueRes streamQueueGetRes(SStreamQueue1* pQueue) { - SStreamQueueNode* pNode = atomic_exchange_ptr(pQueue->pHead, NULL); - if (pNode) return streamQueueBuildRes(pNode); - return (SStreamQueueRes){0}; -} -#endif - bool streamQueueIsFull(const STaosQueue* pQueue, bool inputQ) { bool isFull = taosQueueItemSize((STaosQueue*)pQueue) >= STREAM_TASK_QUEUE_CAPACITY; if (isFull) { @@ -167,7 +109,7 @@ bool streamQueueIsFull(const STaosQueue* pQueue, bool inputQ) { } int32_t threahold = (inputQ) ? STREAM_TASK_INPUT_QUEUE_CAPACITY_IN_SIZE : STREAM_TASK_OUTPUT_QUEUE_CAPACITY_IN_SIZE; - double size = SIZE_IN_MB(taosQueueMemorySize((STaosQueue*)pQueue)); + double size = SIZE_IN_MiB(taosQueueMemorySize((STaosQueue*)pQueue)); return (size >= threahold); } @@ -178,71 +120,70 @@ int32_t streamQueueGetNumOfItems(const SStreamQueue* pQueue) { return numOfItems1 + numOfItems2; } +int32_t streamQueueGetNumOfItemsInQueue(const SStreamQueue* pQueue) { + return taosQueueItemSize(pQueue->pQueue); +} + +int32_t streamQueueItemGetSize(const SStreamQueueItem* pItem) { + STaosQnode* p = (STaosQnode*)((char*) pItem - sizeof(STaosQnode)); + return p->dataSize; +} + +void streamQueueItemIncSize(const SStreamQueueItem* pItem, int32_t size) { + STaosQnode* p = (STaosQnode*)((char*) pItem - sizeof(STaosQnode)); + p->dataSize += size; +} + +const char* streamQueueItemGetTypeStr(int32_t type) { + switch (type) { + case STREAM_INPUT__CHECKPOINT: + return "checkpoint"; + case STREAM_INPUT__CHECKPOINT_TRIGGER: + return "checkpoint-trigger"; + case STREAM_INPUT__TRANS_STATE: + return "trans-state"; + default: + return "datablock"; + } +} + int32_t streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueItem** pInput, int32_t* numOfBlocks) { int32_t retryTimes = 0; int32_t MAX_RETRY_TIMES = 5; const char* id = pTask->id.idStr; - - if (pTask->info.taskLevel == TASK_LEVEL__SINK) { // extract block from inputQ, one-by-one - while (1) { - if (streamTaskShouldPause(&pTask->status) || streamTaskShouldStop(&pTask->status)) { - qDebug("s-task:%s task should pause, extract input blocks:%d", pTask->id.idStr, *numOfBlocks); - return TSDB_CODE_SUCCESS; - } - - STokenBucket* pBucket = pTask->pTokenBucket; - if (!streamTaskHasAvailableToken(pBucket)) { // no available token in th bucket, ignore this execution -// qInfo("s-task:%s no available token for sink, capacity:%d, rate:%d token/sec, quit", pTask->id.idStr, -// pBucket->capacity, pBucket->rate); - return TSDB_CODE_SUCCESS; - } - - SStreamQueueItem* qItem = streamQueueNextItem(pTask->inputInfo.queue); - if (qItem == NULL) { - qDebug("===stream===break batchSize:%d, %s", *numOfBlocks, id); - return TSDB_CODE_SUCCESS; - } - - qDebug("s-task:%s sink task handle block one-by-one, type:%d", id, qItem->type); - - *numOfBlocks = 1; - *pInput = qItem; - return TSDB_CODE_SUCCESS; - } - } + int32_t taskLevel = pTask->info.taskLevel; + *numOfBlocks = 0; while (1) { if (streamTaskShouldPause(&pTask->status) || streamTaskShouldStop(&pTask->status)) { - qDebug("s-task:%s task should pause, extract input blocks:%d", pTask->id.idStr, *numOfBlocks); + stDebug("s-task:%s task should pause, extract input blocks:%d", pTask->id.idStr, *numOfBlocks); return TSDB_CODE_SUCCESS; } SStreamQueueItem* qItem = streamQueueNextItem(pTask->inputInfo.queue); if (qItem == NULL) { - if (pTask->info.taskLevel == TASK_LEVEL__SOURCE && (++retryTimes) < MAX_RETRY_TIMES) { + if ((taskLevel == TASK_LEVEL__SOURCE || taskLevel == TASK_LEVEL__SINK) && (++retryTimes) < MAX_RETRY_TIMES) { taosMsleep(10); - qDebug("===stream===try again batchSize:%d, retry:%d, %s", *numOfBlocks, retryTimes, id); continue; } - - qDebug("===stream===break batchSize:%d, %s", *numOfBlocks, id); return TSDB_CODE_SUCCESS; } // do not merge blocks for sink node and check point data block - if (qItem->type == STREAM_INPUT__CHECKPOINT || qItem->type == STREAM_INPUT__CHECKPOINT_TRIGGER || - qItem->type == STREAM_INPUT__TRANS_STATE) { - const char* p = streamGetBlockTypeStr(qItem->type); + int8_t type = qItem->type; + if (type == STREAM_INPUT__CHECKPOINT || type == STREAM_INPUT__CHECKPOINT_TRIGGER || + type == STREAM_INPUT__TRANS_STATE) { + const char* p = streamQueueItemGetTypeStr(qItem->type); if (*pInput == NULL) { - qDebug("s-task:%s %s msg extracted, start to process immediately", id, p); + stDebug("s-task:%s %s msg extracted, start to process immediately", id, p); *numOfBlocks = 1; *pInput = qItem; return TSDB_CODE_SUCCESS; } else { // previous existed blocks needs to be handle, before handle the checkpoint msg block - qDebug("s-task:%s %s msg extracted, handle previous blocks, numOfBlocks:%d", id, p, *numOfBlocks); + stDebug("s-task:%s %s msg extracted, handle previous blocks, numOfBlocks:%d", id, p, *numOfBlocks); streamQueueProcessFail(pTask->inputInfo.queue); return TSDB_CODE_SUCCESS; } @@ -255,7 +196,7 @@ int32_t streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueItem** pInpu void* newRet = streamMergeQueueItem(*pInput, qItem); if (newRet == NULL) { if (terrno != 0) { - qError("s-task:%s failed to merge blocks from inputQ, numOfBlocks:%d, code:%s", id, *numOfBlocks, + stError("s-task:%s failed to merge blocks from inputQ, numOfBlocks:%d, code:%s", id, *numOfBlocks, tstrerror(terrno)); } @@ -270,7 +211,7 @@ int32_t streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueItem** pInpu streamQueueProcessSuccess(pTask->inputInfo.queue); if (*numOfBlocks >= MAX_STREAM_EXEC_BATCH_NUM) { - qDebug("s-task:%s batch size limit:%d reached, start to process blocks", id, MAX_STREAM_EXEC_BATCH_NUM); + stDebug("s-task:%s batch size limit:%d reached, start to process blocks", id, MAX_STREAM_EXEC_BATCH_NUM); return TSDB_CODE_SUCCESS; } } @@ -285,8 +226,8 @@ int32_t streamTaskPutDataIntoInputQ(SStreamTask* pTask, SStreamQueueItem* pItem) if (type == STREAM_INPUT__DATA_SUBMIT) { SStreamDataSubmit* px = (SStreamDataSubmit*)pItem; if ((pTask->info.taskLevel == TASK_LEVEL__SOURCE) && streamQueueIsFull(pQueue, true)) { - double size = SIZE_IN_MB(taosQueueMemorySize(pQueue)); - qTrace( + double size = SIZE_IN_MiB(taosQueueMemorySize(pQueue)); + stTrace( "s-task:%s inputQ is full, capacity(size:%d num:%dMiB), current(blocks:%d, size:%.2fMiB) stop to push data", pTask->id.idStr, STREAM_TASK_QUEUE_CAPACITY, STREAM_TASK_INPUT_QUEUE_CAPACITY_IN_SIZE, total, size); streamDataSubmitDestroy(px); @@ -304,17 +245,17 @@ int32_t streamTaskPutDataIntoInputQ(SStreamTask* pTask, SStreamQueueItem* pItem) return code; } - double size = SIZE_IN_MB(taosQueueMemorySize(pQueue)); + double size = SIZE_IN_MiB(taosQueueMemorySize(pQueue)); // use the local variable to avoid the pItem be freed by other threads, since it has been put into queue already. - qDebug("s-task:%s submit enqueue msgLen:%d ver:%" PRId64 ", total in queue:%d, size:%.2fMiB", pTask->id.idStr, - msgLen, ver, total, size + SIZE_IN_MB(msgLen)); + stDebug("s-task:%s submit enqueue msgLen:%d ver:%" PRId64 ", total in queue:%d, size:%.2fMiB", pTask->id.idStr, + msgLen, ver, total, size + SIZE_IN_MiB(msgLen)); } else if (type == STREAM_INPUT__DATA_BLOCK || type == STREAM_INPUT__DATA_RETRIEVE || type == STREAM_INPUT__REF_DATA_BLOCK) { if (streamQueueIsFull(pQueue, true)) { - double size = SIZE_IN_MB(taosQueueMemorySize(pQueue)); + double size = SIZE_IN_MiB(taosQueueMemorySize(pQueue)); - qTrace("s-task:%s input queue is full, capacity:%d size:%d MiB, current(blocks:%d, size:%.2fMiB) abort", + stTrace("s-task:%s input queue is full, capacity:%d size:%d MiB, current(blocks:%d, size:%.2fMiB) abort", pTask->id.idStr, STREAM_TASK_QUEUE_CAPACITY, STREAM_TASK_INPUT_QUEUE_CAPACITY_IN_SIZE, total, size); destroyStreamDataBlock((SStreamDataBlock*)pItem); return -1; @@ -326,8 +267,8 @@ int32_t streamTaskPutDataIntoInputQ(SStreamTask* pTask, SStreamQueueItem* pItem) return code; } - double size = SIZE_IN_MB(taosQueueMemorySize(pQueue)); - qDebug("s-task:%s blockdata enqueue, total in queue:%d, size:%.2fMiB", pTask->id.idStr, total, size); + double size = SIZE_IN_MiB(taosQueueMemorySize(pQueue)); + stDebug("s-task:%s blockdata enqueue, total in queue:%d, size:%.2fMiB", pTask->id.idStr, total, size); } else if (type == STREAM_INPUT__CHECKPOINT || type == STREAM_INPUT__CHECKPOINT_TRIGGER || type == STREAM_INPUT__TRANS_STATE) { int32_t code = taosWriteQitem(pQueue, pItem); @@ -336,9 +277,9 @@ int32_t streamTaskPutDataIntoInputQ(SStreamTask* pTask, SStreamQueueItem* pItem) return code; } - double size = SIZE_IN_MB(taosQueueMemorySize(pQueue)); - qDebug("s-task:%s level:%d %s blockdata enqueue, total in queue:%d, size:%.2fMiB", pTask->id.idStr, - pTask->info.taskLevel, streamGetBlockTypeStr(type), total, size); + double size = SIZE_IN_MiB(taosQueueMemorySize(pQueue)); + stDebug("s-task:%s level:%d %s blockdata enqueue, total in queue:%d, size:%.2fMiB", pTask->id.idStr, + pTask->info.taskLevel, streamQueueItemGetTypeStr(type), total, size); } else if (type == STREAM_INPUT__GET_RES) { // use the default memory limit, refactor later. int32_t code = taosWriteQitem(pQueue, pItem); @@ -347,15 +288,15 @@ int32_t streamTaskPutDataIntoInputQ(SStreamTask* pTask, SStreamQueueItem* pItem) return code; } - double size = SIZE_IN_MB(taosQueueMemorySize(pQueue)); - qDebug("s-task:%s data res enqueue, current(blocks:%d, size:%.2fMiB)", pTask->id.idStr, total, size); + double size = SIZE_IN_MiB(taosQueueMemorySize(pQueue)); + stDebug("s-task:%s data res enqueue, current(blocks:%d, size:%.2fMiB)", pTask->id.idStr, total, size); } else { ASSERT(0); } if (type != STREAM_INPUT__GET_RES && type != STREAM_INPUT__CHECKPOINT && pTask->info.triggerParam != 0) { atomic_val_compare_exchange_8(&pTask->schedInfo.status, TASK_TRIGGER_STATUS__INACTIVE, TASK_TRIGGER_STATUS__ACTIVE); - qDebug("s-task:%s new data arrived, active the trigger, triggerStatus:%d", pTask->id.idStr, pTask->schedInfo.status); + stDebug("s-task:%s new data arrived, active the trigger, triggerStatus:%d", pTask->id.idStr, pTask->schedInfo.status); } return 0; @@ -367,14 +308,14 @@ int32_t streamTaskPutDataIntoOutputQ(SStreamTask* pTask, SStreamDataBlock* pBloc while (streamQueueIsFull(pQueue, false)) { if (streamTaskShouldStop(&pTask->status)) { - qInfo("s-task:%s discard result block due to task stop", pTask->id.idStr); + stInfo("s-task:%s discard result block due to task stop", pTask->id.idStr); return TSDB_CODE_STREAM_EXEC_CANCELLED; } int32_t total = streamQueueGetNumOfItems(pTask->outputInfo.queue); - double size = SIZE_IN_MB(taosQueueMemorySize(pQueue)); + double size = SIZE_IN_MiB(taosQueueMemorySize(pQueue)); // let's wait for there are enough space to hold this result pBlock - qDebug("s-task:%s outputQ is full, wait for 500ms and retry, outputQ items:%d, size:%.2fMiB", pTask->id.idStr, + stDebug("s-task:%s outputQ is full, wait for 500ms and retry, outputQ items:%d, size:%.2fMiB", pTask->id.idStr, total, size); taosMsleep(500); } @@ -382,12 +323,12 @@ int32_t streamTaskPutDataIntoOutputQ(SStreamTask* pTask, SStreamDataBlock* pBloc int32_t code = taosWriteQitem(pQueue, pBlock); int32_t total = streamQueueGetNumOfItems(pTask->outputInfo.queue); - double size = SIZE_IN_MB(taosQueueMemorySize(pQueue)); + double size = SIZE_IN_MiB(taosQueueMemorySize(pQueue)); if (code != 0) { - qError("s-task:%s failed to put res into outputQ, outputQ items:%d, size:%.2fMiB code:%s, result lost", + stError("s-task:%s failed to put res into outputQ, outputQ items:%d, size:%.2fMiB code:%s, result lost", pTask->id.idStr, total + 1, size, tstrerror(code)); } else { - qDebug("s-task:%s data put into outputQ, outputQ items:%d, size:%.2fMiB", pTask->id.idStr, total, size); + stDebug("s-task:%s data put into outputQ, outputQ items:%d, size:%.2fMiB", pTask->id.idStr, total, size); } return TSDB_CODE_SUCCESS; @@ -395,7 +336,7 @@ int32_t streamTaskPutDataIntoOutputQ(SStreamTask* pTask, SStreamDataBlock* pBloc int32_t streamTaskInitTokenBucket(STokenBucket* pBucket, int32_t cap, int32_t rate) { if (cap < 50 || rate < 50 || pBucket == NULL) { - qError("failed to init sink task bucket, cap:%d, rate:%d", cap, rate); + stError("failed to init sink task bucket, cap:%d, rate:%d", cap, rate); return TSDB_CODE_INVALID_PARA; } @@ -420,7 +361,7 @@ static void fillBucket(STokenBucket* pBucket) { } pBucket->fillTimestamp = now; - qDebug("new token available, current:%d, inc:%d ts:%"PRId64, pBucket->numOfToken, inc, now); + stDebug("new token available, current:%d, inc:%d ts:%"PRId64, pBucket->numOfToken, inc, now); } } diff --git a/source/libs/stream/src/streamRecover.c b/source/libs/stream/src/streamRecover.c index 2689e9ee70..3ca81ea90b 100644 --- a/source/libs/stream/src/streamRecover.c +++ b/source/libs/stream/src/streamRecover.c @@ -13,25 +13,33 @@ * along with this program. If not, see . */ -#include #include "streamInt.h" #include "trpc.h" #include "ttimer.h" #include "wal.h" -typedef struct SStreamTaskRetryInfo { +typedef struct SLaunchHTaskInfo { SStreamMeta* pMeta; STaskId id; -} SStreamTaskRetryInfo; +} SLaunchHTaskInfo; + +typedef struct STaskRecheckInfo { + SStreamTask* pTask; + SStreamTaskCheckReq req; + void* checkTimer; +} STaskRecheckInfo; static int32_t streamSetParamForScanHistory(SStreamTask* pTask); static void streamTaskSetRangeStreamCalc(SStreamTask* pTask); static int32_t initScanHistoryReq(SStreamTask* pTask, SStreamScanHistoryReq* pReq, int8_t igUntreated); static void streamTaskSetReady(SStreamTask* pTask, int32_t numOfReqs) { + SStreamMeta* pMeta = pTask->pMeta; + int32_t vgId = pMeta->vgId; + if (pTask->status.taskStatus == TASK_STATUS__SCAN_HISTORY && pTask->info.taskLevel != TASK_LEVEL__SOURCE) { pTask->numOfWaitingUpstream = taosArrayGetSize(pTask->pUpstreamInfoList); - qDebug("s-task:%s level:%d task wait for %d upstream tasks complete scan-history procedure, status:%s", + stDebug("s-task:%s level:%d task wait for %d upstream tasks complete scan-history procedure, status:%s", pTask->id.idStr, pTask->info.taskLevel, pTask->numOfWaitingUpstream, streamGetTaskStatusStr(pTask->status.taskStatus)); } @@ -39,9 +47,27 @@ static void streamTaskSetReady(SStreamTask* pTask, int32_t numOfReqs) { ASSERT(pTask->status.downstreamReady == 0); pTask->status.downstreamReady = 1; - int64_t el = (taosGetTimestampMs() - pTask->taskExecInfo.init); - qDebug("s-task:%s all %d downstream ready, init completed, elapsed time:%"PRId64"ms, task status:%s", - pTask->id.idStr, numOfReqs, el, streamGetTaskStatusStr(pTask->status.taskStatus)); + pTask->execInfo.start = taosGetTimestampMs(); + int64_t el = (pTask->execInfo.start - pTask->execInfo.init); + stDebug("s-task:%s all %d downstream ready, init completed, elapsed time:%" PRId64 "ms, task status:%s", + pTask->id.idStr, numOfReqs, el, streamGetTaskStatusStr(pTask->status.taskStatus)); + + taosWLockLatch(&pMeta->lock); + + STaskId id = extractStreamTaskKey(pTask); + taosHashPut(pMeta->startInfo.pReadyTaskSet, &id, sizeof(id), NULL, 0); + int32_t numOfTotal = streamMetaGetNumOfTasks(pMeta); + + if (taosHashGetSize(pMeta->startInfo.pReadyTaskSet) == numOfTotal) { + // reset value for next time start + taosHashClear(pMeta->startInfo.pReadyTaskSet); + pMeta->startInfo.startedAfterNodeUpdate = 0; + pMeta->startInfo.elapsedTime = pTask->execInfo.start - pMeta->startInfo.ts; + + stDebug("vgId:%d all %d task(s) are started successfully, last ready task:%s level:%d, total elapsed time:%.2f sec", + vgId, numOfTotal, pTask->id.idStr, pTask->info.taskLevel, pMeta->startInfo.elapsedTime / 1000.0); + } + taosWUnLockLatch(&pMeta->lock); } int32_t streamStartScanHistoryAsync(SStreamTask* pTask, int8_t igUntreated) { @@ -95,7 +121,7 @@ int32_t streamTaskLaunchScanHistory(SStreamTask* pTask) { return doLaunchScanHistoryTask(pTask); } else { ASSERT(pTask->status.taskStatus == TASK_STATUS__NORMAL); - qDebug("s-task:%s no need to scan-history-data, status:%s, sched-status:%d, ver:%" PRId64, pTask->id.idStr, + stDebug("s-task:%s no need to scan-history-data, status:%s, sched-status:%d, ver:%" PRId64, pTask->id.idStr, streamGetTaskStatusStr(pTask->status.taskStatus), pTask->status.schedStatus, walReaderGetCurrentVer(pTask->exec.pWalReader)); } @@ -105,15 +131,15 @@ int32_t streamTaskLaunchScanHistory(SStreamTask* pTask) { streamTaskEnablePause(pTask); } } else if (pTask->info.taskLevel == TASK_LEVEL__SINK) { - qDebug("s-task:%s sink task do nothing to handle scan-history", pTask->id.idStr); + stDebug("s-task:%s sink task do nothing to handle scan-history", pTask->id.idStr); } return 0; } // check status static int32_t doCheckDownstreamStatus(SStreamTask* pTask) { - SDataRange* pRange = &pTask->dataRange; - STimeWindow* pWindow = &pRange->window; + SDataRange* pRange = &pTask->dataRange; + STimeWindow* pWindow = &pRange->window; SStreamTaskCheckReq req = { .streamId = pTask->id.streamId, @@ -126,16 +152,16 @@ static int32_t doCheckDownstreamStatus(SStreamTask* pTask) { // serialize streamProcessScanHistoryFinishRsp if (pTask->outputInfo.type == TASK_OUTPUT__FIXED_DISPATCH) { req.reqId = tGenIdPI64(); - req.downstreamNodeId = pTask->fixedEpDispatcher.nodeId; - req.downstreamTaskId = pTask->fixedEpDispatcher.taskId; + req.downstreamNodeId = pTask->fixedDispatcher.nodeId; + req.downstreamTaskId = pTask->fixedDispatcher.taskId; pTask->checkReqId = req.reqId; - qDebug("s-task:%s check single downstream task:0x%x(vgId:%d) ver:%" PRId64 "-%" PRId64 " window:%" PRId64 - "-%" PRId64 ", stage:%"PRId64" req:0x%" PRIx64, - pTask->id.idStr, req.downstreamTaskId, req.downstreamNodeId, pRange->range.minVer, pRange->range.maxVer, - pWindow->skey, pWindow->ekey, req.stage, req.reqId); + stDebug("s-task:%s stage:%" PRId64 " check single downstream task:0x%x(vgId:%d) ver:%" PRId64 "-%" PRId64 + " window:%" PRId64 "-%" PRId64 " req:0x%" PRIx64, + pTask->id.idStr, req.reqId, req.downstreamTaskId, req.downstreamNodeId, pRange->range.minVer, + pRange->range.maxVer, pWindow->skey, pWindow->ekey, req.reqId); - streamDispatchCheckMsg(pTask, &req, pTask->fixedEpDispatcher.nodeId, &pTask->fixedEpDispatcher.epSet); + streamSendCheckMsg(pTask, &req, pTask->fixedDispatcher.nodeId, &pTask->fixedDispatcher.epSet); } else if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { SArray* vgInfo = pTask->shuffleDispatcher.dbInfo.pVgroupInfos; @@ -143,7 +169,7 @@ static int32_t doCheckDownstreamStatus(SStreamTask* pTask) { pTask->notReadyTasks = numOfVgs; pTask->checkReqIds = taosArrayInit(numOfVgs, sizeof(int64_t)); - qDebug("s-task:%s check %d downstream tasks, ver:%" PRId64 "-%" PRId64 " window:%" PRId64 "-%" PRId64, + stDebug("s-task:%s check %d downstream tasks, ver:%" PRId64 "-%" PRId64 " window:%" PRId64 "-%" PRId64, pTask->id.idStr, numOfVgs, pRange->range.minVer, pRange->range.maxVer, pWindow->skey, pWindow->ekey); for (int32_t i = 0; i < numOfVgs; i++) { @@ -152,12 +178,12 @@ static int32_t doCheckDownstreamStatus(SStreamTask* pTask) { taosArrayPush(pTask->checkReqIds, &req.reqId); req.downstreamNodeId = pVgInfo->vgId; req.downstreamTaskId = pVgInfo->taskId; - qDebug("s-task:%s (vgId:%d) check downstream task:0x%x (vgId:%d) (shuffle), idx:%d, stage:%" PRId64, - pTask->id.idStr, pTask->info.nodeId, req.downstreamTaskId, req.downstreamNodeId, i, req.stage); - streamDispatchCheckMsg(pTask, &req, pVgInfo->vgId, &pVgInfo->epSet); + stDebug("s-task:%s (vgId:%d) stage:%" PRId64 " check downstream task:0x%x (vgId:%d) (shuffle), idx:%d", + pTask->id.idStr, pTask->info.nodeId, req.stage, req.downstreamTaskId, req.downstreamNodeId, i); + streamSendCheckMsg(pTask, &req, pVgInfo->vgId, &pVgInfo->epSet); } } else { - qDebug("s-task:%s (vgId:%d) set downstream ready, since no downstream", pTask->id.idStr, pTask->info.nodeId); + stDebug("s-task:%s (vgId:%d) set downstream ready, since no downstream", pTask->id.idStr, pTask->info.nodeId); streamTaskSetReady(pTask, 0); streamTaskSetRangeStreamCalc(pTask); @@ -168,8 +194,15 @@ static int32_t doCheckDownstreamStatus(SStreamTask* pTask) { return 0; } -int32_t streamRecheckDownstream(SStreamTask* pTask, const SStreamTaskCheckRsp* pRsp) { - SStreamTaskCheckReq req = { +static STaskRecheckInfo* createRecheckInfo(SStreamTask* pTask, const SStreamTaskCheckRsp* pRsp) { + STaskRecheckInfo* pInfo = taosMemoryCalloc(1, sizeof(STaskRecheckInfo)); + if (pInfo == NULL) { + terrno = TSDB_CODE_OUT_OF_MEMORY; + return NULL; + } + + pInfo->pTask = pTask; + pInfo->req = (SStreamTaskCheckReq){ .reqId = pRsp->reqId, .streamId = pRsp->streamId, .upstreamTaskId = pRsp->upstreamTaskId, @@ -180,25 +213,44 @@ int32_t streamRecheckDownstream(SStreamTask* pTask, const SStreamTaskCheckRsp* p .stage = pTask->pMeta->stage, }; + return pInfo; +} + +static void destroyRecheckInfo(STaskRecheckInfo* pInfo) { + if (pInfo != NULL) { + taosTmrStop(pInfo->checkTimer); + pInfo->checkTimer = NULL; + taosMemoryFree(pInfo); + } +} + +static void recheckDownstreamTasks(void* param, void* tmrId) { + STaskRecheckInfo* pInfo = param; + SStreamTask* pTask = pInfo->pTask; + + SStreamTaskCheckReq* pReq = &pInfo->req; + if (pTask->outputInfo.type == TASK_OUTPUT__FIXED_DISPATCH) { - qDebug("s-task:%s (vgId:%d) check downstream task:0x%x (vgId:%d) stage:%" PRId64 " (recheck)", pTask->id.idStr, - pTask->info.nodeId, req.downstreamTaskId, req.downstreamNodeId, req.stage); - streamDispatchCheckMsg(pTask, &req, pRsp->downstreamNodeId, &pTask->fixedEpDispatcher.epSet); + stDebug("s-task:%s (vgId:%d) check downstream task:0x%x (vgId:%d) stage:%" PRId64 " (recheck)", pTask->id.idStr, + pTask->info.nodeId, pReq->downstreamTaskId, pReq->downstreamNodeId, pReq->stage); + streamSendCheckMsg(pTask, pReq, pReq->downstreamNodeId, &pTask->fixedDispatcher.epSet); } else if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { SArray* vgInfo = pTask->shuffleDispatcher.dbInfo.pVgroupInfos; int32_t numOfVgs = taosArrayGetSize(vgInfo); for (int32_t i = 0; i < numOfVgs; i++) { SVgroupInfo* pVgInfo = taosArrayGet(vgInfo, i); - if (pVgInfo->taskId == req.downstreamTaskId) { - qDebug("s-task:%s (vgId:%d) check downstream task:0x%x (vgId:%d) stage:%" PRId64 " (recheck)", pTask->id.idStr, - pTask->info.nodeId, req.downstreamTaskId, req.downstreamNodeId, req.stage); - streamDispatchCheckMsg(pTask, &req, pRsp->downstreamNodeId, &pVgInfo->epSet); + if (pVgInfo->taskId == pReq->downstreamTaskId) { + stDebug("s-task:%s (vgId:%d) check downstream task:0x%x (vgId:%d) stage:%" PRId64 " (recheck)", pTask->id.idStr, + pTask->info.nodeId, pReq->downstreamTaskId, pReq->downstreamNodeId, pReq->stage); + streamSendCheckMsg(pTask, pReq, pReq->downstreamNodeId, &pVgInfo->epSet); } } } - return 0; + destroyRecheckInfo(pInfo); + int8_t ref = atomic_sub_fetch_8(&pTask->status.timerActive, 1); + stDebug("s-task:%s complete send check in timer, ref:%d", pTask->id.idStr, ref); } int32_t streamTaskCheckStatus(SStreamTask* pTask, int32_t upstreamTaskId, int32_t vgId, int64_t stage) { @@ -207,23 +259,30 @@ int32_t streamTaskCheckStatus(SStreamTask* pTask, int32_t upstreamTaskId, int32_ const char* id = pTask->id.idStr; if (stage == -1) { - qDebug("s-task:%s receive check msg from upstream task:0x%x, invalid stageId:%" PRId64 ", not ready", id, - upstreamTaskId, stage); + stDebug("s-task:%s receive check msg from upstream task:0x%x(vgId:%d), invalid stageId:%" PRId64 ", not ready", id, + upstreamTaskId, vgId, stage); return 0; } if (pInfo->stage == -1) { pInfo->stage = stage; - qDebug("s-task:%s receive check msg from upstream task:0x%x for the time, init stage value:%" PRId64, id, - upstreamTaskId, stage); + stDebug("s-task:%s receive check msg from upstream task:0x%x(vgId:%d) first time, init stage value:%" PRId64, id, + upstreamTaskId, vgId, stage); } if (pInfo->stage < stage) { - qError("s-task:%s receive msg from upstream task:0x%x(vgId:%d), new stage received:%" PRId64 ", prev:%" PRId64, - id, upstreamTaskId, vgId, stage, pInfo->stage); + stError("s-task:%s receive check msg from upstream task:0x%x(vgId:%d), new stage received:%" PRId64 + ", prev:%" PRId64, + id, upstreamTaskId, vgId, stage, pInfo->stage); } - return ((pTask->status.downstreamReady == 1) && (pInfo->stage == stage))? 1:0; + if (pInfo->stage != stage) { + return TASK_SELF_NEW_STAGE; + } else if (pTask->status.downstreamReady != 1) { + return TASK_DOWNSTREAM_NOT_READY; + } else { + return TASK_DOWNSTREAM_READY; + } } static void doProcessDownstreamReadyRsp(SStreamTask* pTask, int32_t numOfReqs) { @@ -237,15 +296,15 @@ static void doProcessDownstreamReadyRsp(SStreamTask* pTask, int32_t numOfReqs) { streamTaskSetRangeStreamCalc(pTask); if (status == TASK_STATUS__SCAN_HISTORY) { - qDebug("s-task:%s enter into scan-history data stage, status:%s", id, str); + stDebug("s-task:%s enter into scan-history data stage, status:%s", id, str); streamTaskLaunchScanHistory(pTask); } else { if (pTask->info.fillHistory == 1) { - qDebug("s-task:%s fill-history is set normal when start it, try to remove it,set it task to be dropping", id); + stDebug("s-task:%s fill-history is set normal when start it, try to remove it,set it task to be dropping", id); pTask->status.taskStatus = TASK_STATUS__DROPPING; ASSERT(pTask->historyTaskId.taskId == 0); } else { - qDebug("s-task:%s downstream tasks are ready, now ready for data from wal, status:%s", id, str); + stDebug("s-task:%s downstream tasks are ready, now ready for data from wal, status:%s", id, str); streamTaskEnablePause(pTask); } } @@ -259,7 +318,12 @@ int32_t streamProcessCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* pRs ASSERT(pTask->id.taskId == pRsp->upstreamTaskId); const char* id = pTask->id.idStr; - if (pRsp->status == 1) { + if (streamTaskShouldStop(&pTask->status)) { + stDebug("s-task:%s should stop, do not do check downstream again", id); + return TSDB_CODE_SUCCESS; + } + + if (pRsp->status == TASK_DOWNSTREAM_READY) { if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { bool found = false; @@ -286,8 +350,8 @@ int32_t streamProcessCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* pRs doProcessDownstreamReadyRsp(pTask, numOfReqs); } else { int32_t total = taosArrayGetSize(pTask->shuffleDispatcher.dbInfo.pVgroupInfos); - qDebug("s-task:%s (vgId:%d) recv check rsp from task:0x%x (vgId:%d) status:%d, total:%d not ready:%d", id, - pRsp->upstreamNodeId, pRsp->downstreamTaskId, pRsp->downstreamNodeId, pRsp->status, total, left); + stDebug("s-task:%s (vgId:%d) recv check rsp from task:0x%x (vgId:%d) status:%d, total:%d not ready:%d", id, + pRsp->upstreamNodeId, pRsp->downstreamTaskId, pRsp->downstreamNodeId, pRsp->status, total, left); } } else { ASSERT(pTask->outputInfo.type == TASK_OUTPUT__FIXED_DISPATCH); @@ -298,10 +362,25 @@ int32_t streamProcessCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* pRs doProcessDownstreamReadyRsp(pTask, 1); } } else { // not ready, wait for 100ms and retry - qDebug("s-task:%s downstream taskId:0x%x (vgId:%d) not ready, stage:%d, wait for 100ms and retry", id, - pRsp->downstreamTaskId, pRsp->downstreamNodeId, pRsp->oldStage); - taosMsleep(100); - streamRecheckDownstream(pTask, pRsp); + if (pRsp->status == TASK_DOWNSTREAM_NOT_LEADER) { + stError( + "s-task:%s downstream taskId:0x%x (vgId:%d) vnode-transfer/leader-change detected, not send check again, " + "roll-back needed", + id, pRsp->downstreamTaskId, pRsp->downstreamNodeId); + } else if (pRsp->status == TASK_SELF_NEW_STAGE) { + stError( + "s-task:%s vnode-transfer/leader-change/restart detected, old stage:%d, current stage:%d, not send check " + "again, roll-back needed", + id, pRsp->oldStage, (int32_t)pTask->pMeta->stage); + } else { + STaskRecheckInfo* pInfo = createRecheckInfo(pTask, pRsp); + + int8_t ref = atomic_add_fetch_8(&pTask->status.timerActive, 1); + stDebug("s-task:%s downstream taskId:0x%x (vgId:%d) not ready, stage:%d, retry in 100ms, ref:%d ", id, + pRsp->downstreamTaskId, pRsp->downstreamNodeId, pRsp->oldStage, ref); + + taosTmrReset(recheckDownstreamTasks, CHECK_DOWNSTREAM_INTERVAL, pInfo, streamEnv.timer, &pInfo->checkTimer); + } } return 0; @@ -315,7 +394,7 @@ int32_t streamSendCheckRsp(const SStreamMeta* pMeta, const SStreamTaskCheckReq* tEncodeSize(tEncodeStreamTaskCheckRsp, pRsp, len, code); if (code < 0) { - qError("vgId:%d failed to encode task check rsp, s-task:0x%x", pMeta->vgId, taskId); + stError("vgId:%d failed to encode task check rsp, s-task:0x%x", pMeta->vgId, taskId); return -1; } @@ -335,22 +414,22 @@ int32_t streamSendCheckRsp(const SStreamMeta* pMeta, const SStreamTaskCheckReq* // common int32_t streamSetParamForScanHistory(SStreamTask* pTask) { - qDebug("s-task:%s set operator option for scan-history data", pTask->id.idStr); + stDebug("s-task:%s set operator option for scan-history data", pTask->id.idStr); return qSetStreamOperatorOptionForScanHistory(pTask->exec.pExecutor); } int32_t streamRestoreParam(SStreamTask* pTask) { - qDebug("s-task:%s restore operator param after scan-history", pTask->id.idStr); + stDebug("s-task:%s restore operator param after scan-history", pTask->id.idStr); return qRestoreStreamOperatorOption(pTask->exec.pExecutor); } int32_t streamSetStatusNormal(SStreamTask* pTask) { int32_t status = atomic_load_8(&pTask->status.taskStatus); if (status == TASK_STATUS__DROPPING) { - qError("s-task:%s cannot be set normal, since in dropping state", pTask->id.idStr); + stError("s-task:%s cannot be set normal, since in dropping state", pTask->id.idStr); return -1; } else { - qDebug("s-task:%s set task status to be normal, prev:%s", pTask->id.idStr, streamGetTaskStatusStr(status)); + stDebug("s-task:%s set task status to be normal, prev:%s", pTask->id.idStr, streamGetTaskStatusStr(status)); atomic_store_8(&pTask->status.taskStatus, TASK_STATUS__NORMAL); return 0; } @@ -359,10 +438,10 @@ int32_t streamSetStatusNormal(SStreamTask* pTask) { int32_t streamSetStatusUnint(SStreamTask* pTask) { int32_t status = atomic_load_8(&pTask->status.taskStatus); if (status == TASK_STATUS__DROPPING) { - qError("s-task:%s cannot be set uninit, since in dropping state", pTask->id.idStr); + stError("s-task:%s cannot be set uninit, since in dropping state", pTask->id.idStr); return -1; } else { - qDebug("s-task:%s set task status to be uninit, prev:%s", pTask->id.idStr, streamGetTaskStatusStr(status)); + stDebug("s-task:%s set task status to be uninit, prev:%s", pTask->id.idStr, streamGetTaskStatusStr(status)); atomic_store_8(&pTask->status.taskStatus, TASK_STATUS__UNINIT); return 0; } @@ -434,7 +513,7 @@ int32_t streamProcessScanHistoryFinishReq(SStreamTask* pTask, SStreamScanHistory ASSERT(taskLevel == TASK_LEVEL__AGG || taskLevel == TASK_LEVEL__SINK); if (pTask->status.taskStatus != TASK_STATUS__SCAN_HISTORY) { - qError("s-task:%s not in scan-history status, status:%s return upstream:0x%x scan-history finish directly", + stError("s-task:%s not in scan-history status, status:%s return upstream:0x%x scan-history finish directly", pTask->id.idStr, streamGetTaskStatusStr(pTask->status.taskStatus), pReq->upstreamTaskId); void* pBuf = NULL; @@ -445,7 +524,7 @@ int32_t streamProcessScanHistoryFinishReq(SStreamTask* pTask, SStreamScanHistory initRpcMsg(&msg, 0, pBuf, sizeof(SMsgHead) + len); tmsgSendRsp(&msg); - qDebug("s-task:%s level:%d notify upstream:0x%x(vgId:%d) to continue process data in WAL", pTask->id.idStr, + stDebug("s-task:%s level:%d notify upstream:0x%x(vgId:%d) to continue process data in WAL", pTask->id.idStr, pTask->info.taskLevel, pReq->upstreamTaskId, pReq->upstreamNodeId); return 0; } @@ -458,7 +537,7 @@ int32_t streamProcessScanHistoryFinishReq(SStreamTask* pTask, SStreamScanHistory if (left == 0) { int32_t numOfTasks = taosArrayGetSize(pTask->pUpstreamInfoList); - qDebug( + stDebug( "s-task:%s all %d upstream tasks finish scan-history data, set param for agg task for stream data and send " "rsp to all upstream tasks", pTask->id.idStr, numOfTasks); @@ -481,7 +560,7 @@ int32_t streamProcessScanHistoryFinishReq(SStreamTask* pTask, SStreamScanHistory } } } else { - qDebug("s-task:%s receive scan-history data finish msg from upstream:0x%x(index:%d), unfinished:%d", + stDebug("s-task:%s receive scan-history data finish msg from upstream:0x%x(index:%d), unfinished:%d", pTask->id.idStr, pReq->upstreamTaskId, pReq->childId, left); } @@ -518,12 +597,12 @@ static void checkFillhistoryTaskStatus(SStreamTask* pTask, SStreamTask* pHTask) pHTask->dataRange.range.maxVer = pTask->chkInfo.nextProcessVer - 1; if (pTask->info.taskLevel == TASK_LEVEL__SOURCE) { - qDebug("s-task:%s set the launch condition for fill-history s-task:%s, window:%" PRId64 " - %" PRId64 + stDebug("s-task:%s set the launch condition for fill-history s-task:%s, window:%" PRId64 " - %" PRId64 " ver range:%" PRId64 " - %" PRId64, pTask->id.idStr, pHTask->id.idStr, pHTask->dataRange.window.skey, pHTask->dataRange.window.ekey, pHTask->dataRange.range.minVer, pHTask->dataRange.range.maxVer); } else { - qDebug("s-task:%s no fill history condition for non-source task:%s", pTask->id.idStr, pHTask->id.idStr); + stDebug("s-task:%s no fill history condition for non-source task:%s", pTask->id.idStr, pHTask->id.idStr); } // check if downstream tasks have been ready @@ -531,10 +610,10 @@ static void checkFillhistoryTaskStatus(SStreamTask* pTask, SStreamTask* pHTask) } static void tryLaunchHistoryTask(void* param, void* tmrId) { - SStreamTaskRetryInfo* pInfo = param; - SStreamMeta* pMeta = pInfo->pMeta; + SLaunchHTaskInfo* pInfo = param; + SStreamMeta* pMeta = pInfo->pMeta; - qDebug("s-task:0x%x in timer to launch related history task", (int32_t) pInfo->id.taskId); + stDebug("s-task:0x%x in timer to launch related history task", (int32_t) pInfo->id.taskId); taosWLockLatch(&pMeta->lock); SStreamTask** ppTask = (SStreamTask**)taosHashGet(pMeta->pTasksMap, &pInfo->id, sizeof(pInfo->id)); @@ -543,7 +622,7 @@ static void tryLaunchHistoryTask(void* param, void* tmrId) { if (streamTaskShouldStop(&(*ppTask)->status)) { const char* pStatus = streamGetTaskStatusStr((*ppTask)->status.taskStatus); - qDebug("s-task:%s status:%s quit timer task", (*ppTask)->id.idStr, pStatus); + stDebug("s-task:%s status:%s quit timer task", (*ppTask)->id.idStr, pStatus); taosMemoryFree(pInfo); atomic_sub_fetch_8(&(*ppTask)->status.timerActive, 1); @@ -561,12 +640,12 @@ static void tryLaunchHistoryTask(void* param, void* tmrId) { SStreamTask* pHTask = streamMetaAcquireTask(pMeta, pTask->historyTaskId.streamId, pTask->historyTaskId.taskId); if (pHTask == NULL && (!streamTaskShouldStop(&pTask->status))) { const char* pStatus = streamGetTaskStatusStr(pTask->status.taskStatus); - qWarn( + stWarn( "s-task:%s vgId:%d status:%s failed to launch history task:0x%x, since it may not be built, or may have been " "destroyed, or should stop", pTask->id.idStr, pMeta->vgId, pStatus, (int32_t) pTask->historyTaskId.taskId); - taosTmrReset(tryLaunchHistoryTask, 100, pInfo, streamEnv.timer, &pTask->launchTaskTimer); + taosTmrReset(tryLaunchHistoryTask, 100, pInfo, streamEnv.timer, &pTask->pTimer->hTaskLaunchTimer); streamMetaReleaseTask(pMeta, pTask); return; } @@ -580,7 +659,7 @@ static void tryLaunchHistoryTask(void* param, void* tmrId) { atomic_sub_fetch_8(&pTask->status.timerActive, 1); streamMetaReleaseTask(pMeta, pTask); } else { - qError("s-task:0x%x failed to load task, it may have been destroyed", (int32_t) pInfo->id.taskId); + stError("s-task:0x%x failed to load task, it may have been destroyed", (int32_t) pInfo->id.taskId); } taosMemoryFree(pInfo); @@ -596,34 +675,34 @@ int32_t streamLaunchFillHistoryTask(SStreamTask* pTask) { } ASSERT(pTask->status.downstreamReady == 1); - qDebug("s-task:%s start to launch related fill-history task:0x%" PRIx64 "-0x%x", pTask->id.idStr, + stDebug("s-task:%s start to launch related fill-history task:0x%" PRIx64 "-0x%x", pTask->id.idStr, pTask->historyTaskId.streamId, hTaskId); // Set the execute conditions, including the query time window and the version range SStreamTask** pHTask = taosHashGet(pMeta->pTasksMap, &pTask->historyTaskId, sizeof(pTask->historyTaskId)); if (pHTask == NULL) { - qWarn("s-task:%s vgId:%d failed to launch history task:0x%x, since it is not built yet", pTask->id.idStr, + stWarn("s-task:%s vgId:%d failed to launch history task:0x%x, since it is not built yet", pTask->id.idStr, pMeta->vgId, hTaskId); - SStreamTaskRetryInfo* pInfo = taosMemoryCalloc(1, sizeof(SStreamTaskRetryInfo)); + SLaunchHTaskInfo* pInfo = taosMemoryCalloc(1, sizeof(SLaunchHTaskInfo)); pInfo->id.taskId = pTask->id.taskId; pInfo->id.streamId = pTask->id.streamId; pInfo->pMeta = pTask->pMeta; - if (pTask->launchTaskTimer == NULL) { - pTask->launchTaskTimer = taosTmrStart(tryLaunchHistoryTask, 100, pInfo, streamEnv.timer); - if (pTask->launchTaskTimer == NULL) { + if (pTask->pTimer->hTaskLaunchTimer == NULL) { + pTask->pTimer->hTaskLaunchTimer = taosTmrStart(tryLaunchHistoryTask, 100, pInfo, streamEnv.timer); + if (pTask->pTimer->hTaskLaunchTimer == NULL) { // todo failed to create timer taosMemoryFree(pInfo); } else { int32_t ref = atomic_add_fetch_8(&pTask->status.timerActive, 1);// timer is active ASSERT(ref == 1); - qDebug("s-task:%s set timer active flag", pTask->id.idStr); + stDebug("s-task:%s set timer active flag, ref:%d", pTask->id.idStr, ref); } } else { // timer exists ASSERT(pTask->status.timerActive == 1); - qDebug("s-task:%s set timer active flag, task timer not null", pTask->id.idStr); - taosTmrReset(tryLaunchHistoryTask, 100, pInfo, streamEnv.timer, &pTask->launchTaskTimer); + stDebug("s-task:%s set timer active flag, task timer not null", pTask->id.idStr); + taosTmrReset(tryLaunchHistoryTask, 100, pInfo, streamEnv.timer, &pTask->pTimer->hTaskLaunchTimer); } // try again in 100ms @@ -631,7 +710,7 @@ int32_t streamLaunchFillHistoryTask(SStreamTask* pTask) { } if ((*pHTask)->status.downstreamReady == 1) { - qDebug("s-task:%s fill-history task is ready, no need to check downstream", (*pHTask)->id.idStr); + stDebug("s-task:%s fill-history task is ready, no need to check downstream", (*pHTask)->id.idStr); } else { checkFillhistoryTaskStatus(pTask, *pHTask); } @@ -675,7 +754,7 @@ bool streamHistoryTaskSetVerRangeStep2(SStreamTask* pTask, int64_t latestVer) { if (nextStartVer > latestVer - 1) { // no input data yet. no need to execute the secondardy scan while stream task halt streamTaskFillHistoryFinished(pTask); - qDebug( + stDebug( "s-task:%s no need to perform secondary scan-history data(step 2), since no data ingest during step1 scan, " "related stream task currentVer:%" PRId64, pTask->id.idStr, latestVer); @@ -773,11 +852,11 @@ void streamTaskSetRangeStreamCalc(SStreamTask* pTask) { if (pTask->historyTaskId.taskId == 0) { SDataRange* pRange = &pTask->dataRange; if (pTask->info.fillHistory == 1) { - qDebug("s-task:%s fill-history task, time window:%" PRId64 "-%" PRId64 ", verRange:%" PRId64 + stDebug("s-task:%s fill-history task, time window:%" PRId64 "-%" PRId64 ", verRange:%" PRId64 "-%" PRId64, pTask->id.idStr, pRange->window.skey, pRange->window.ekey, pRange->range.minVer, pRange->range.maxVer); } else { - qDebug("s-task:%s no related fill-history task, stream time window:%" PRId64 "-%" PRId64 ", verRange:%" PRId64 + stDebug("s-task:%s no related fill-history task, stream time window:%" PRId64 "-%" PRId64 ", verRange:%" PRId64 "-%" PRId64, pTask->id.idStr, pRange->window.skey, pRange->window.ekey, pRange->range.minVer, pRange->range.maxVer); } @@ -798,7 +877,7 @@ void streamTaskSetRangeStreamCalc(SStreamTask* pTask) { pRange->range.minVer = 0; pRange->range.maxVer = ver; - qDebug("s-task:%s level:%d related fill-history task exists, update stream calc time window:%" PRId64 " - %" PRId64 + stDebug("s-task:%s level:%d related fill-history task exists, update stream calc time window:%" PRId64 " - %" PRId64 ", verRang:%" PRId64 " - %" PRId64, pTask->id.idStr, pTask->info.taskLevel, pRange->window.skey, pRange->window.ekey, pRange->range.minVer, pRange->range.maxVer); @@ -808,7 +887,7 @@ void streamTaskSetRangeStreamCalc(SStreamTask* pTask) { // only the downstream tasks are ready, set the task to be ready to work. void streamTaskCheckDownstream(SStreamTask* pTask) { if (pTask->info.fillHistory) { - qDebug("s-task:%s fill history task, wait for being launched", pTask->id.idStr); + stDebug("s-task:%s fill history task, wait for being launched", pTask->id.idStr); return; } @@ -822,36 +901,42 @@ void streamTaskPause(SStreamTask* pTask, SStreamMeta* pMeta) { int8_t status = pTask->status.taskStatus; if (status == TASK_STATUS__DROPPING) { - qDebug("vgId:%d s-task:%s task already dropped, do nothing", pMeta->vgId, pTask->id.idStr); + stDebug("vgId:%d s-task:%s task already dropped, do nothing", pMeta->vgId, pTask->id.idStr); return; } const char* str = streamGetTaskStatusStr(status); if (status == TASK_STATUS__STOP || status == TASK_STATUS__PAUSE) { - qDebug("vgId:%d s-task:%s task already stopped/paused, status:%s, do nothing", pMeta->vgId, pTask->id.idStr, str); + stDebug("vgId:%d s-task:%s task already stopped/paused, status:%s, do nothing", pMeta->vgId, pTask->id.idStr, str); return; } if(pTask->info.taskLevel == TASK_LEVEL__SINK) { int32_t num = atomic_add_fetch_32(&pMeta->numOfPausedTasks, 1); - qInfo("vgId:%d s-task:%s pause stream sink task. pause task num:%d", pMeta->vgId, pTask->id.idStr, num); + stInfo("vgId:%d s-task:%s pause stream sink task. pause task num:%d", pMeta->vgId, pTask->id.idStr, num); return; } while (!pTask->status.pauseAllowed || (pTask->status.taskStatus == TASK_STATUS__HALT)) { status = pTask->status.taskStatus; if (status == TASK_STATUS__DROPPING) { - qDebug("vgId:%d s-task:%s task already dropped, do nothing", pMeta->vgId, pTask->id.idStr); + stDebug("vgId:%d s-task:%s task already dropped, do nothing", pMeta->vgId, pTask->id.idStr); return; } if (status == TASK_STATUS__STOP || status == TASK_STATUS__PAUSE) { - qDebug("vgId:%d s-task:%s task already stopped/paused, status:%s, do nothing", pMeta->vgId, pTask->id.idStr, str); + stDebug("vgId:%d s-task:%s task already stopped/paused, status:%s, do nothing", pMeta->vgId, pTask->id.idStr, str); return; } +// +// if (pTask->status.downstreamReady == 0) { +// ASSERT(pTask->execInfo.start == 0); +// stDebug("s-task:%s in check downstream procedure, abort and paused", pTask->id.idStr); +// break; +// } const char* pStatus = streamGetTaskStatusStr(status); - qDebug("s-task:%s wait for the task can be paused, status:%s, vgId:%d", pTask->id.idStr, pStatus, pMeta->vgId); + stDebug("s-task:%s wait for the task can be paused, status:%s, vgId:%d", pTask->id.idStr, pStatus, pMeta->vgId); taosMsleep(100); } @@ -861,14 +946,14 @@ void streamTaskPause(SStreamTask* pTask, SStreamMeta* pMeta) { status = pTask->status.taskStatus; if (status == TASK_STATUS__DROPPING || status == TASK_STATUS__STOP) { taosWUnLockLatch(&pMeta->lock); - qDebug("vgId:%d s-task:%s task already dropped/stopped/paused, do nothing", pMeta->vgId, pTask->id.idStr); + stDebug("vgId:%d s-task:%s task already dropped/stopped/paused, do nothing", pMeta->vgId, pTask->id.idStr); return; } atomic_store_8(&pTask->status.keepTaskStatus, pTask->status.taskStatus); atomic_store_8(&pTask->status.taskStatus, TASK_STATUS__PAUSE); int32_t num = atomic_add_fetch_32(&pMeta->numOfPausedTasks, 1); - qInfo("vgId:%d s-task:%s pause stream task. pause task num:%d", pMeta->vgId, pTask->id.idStr, num); + stInfo("vgId:%d s-task:%s pause stream task. pause task num:%d", pMeta->vgId, pTask->id.idStr, num); taosWUnLockLatch(&pMeta->lock); // in case of fill-history task, stop the tsdb file scan operation. @@ -878,7 +963,7 @@ void streamTaskPause(SStreamTask* pTask, SStreamMeta* pMeta) { } int64_t el = taosGetTimestampMs() - st; - qDebug("vgId:%d s-task:%s set pause flag, prev:%s, pause elapsed time:%dms", pMeta->vgId, pTask->id.idStr, + stDebug("vgId:%d s-task:%s set pause flag, prev:%s, pause elapsed time:%dms", pMeta->vgId, pTask->id.idStr, streamGetTaskStatusStr(pTask->status.keepTaskStatus), (int32_t)el); } @@ -888,12 +973,12 @@ void streamTaskResume(SStreamTask* pTask, SStreamMeta* pMeta) { pTask->status.taskStatus = pTask->status.keepTaskStatus; pTask->status.keepTaskStatus = TASK_STATUS__NORMAL; int32_t num = atomic_sub_fetch_32(&pMeta->numOfPausedTasks, 1); - qInfo("vgId:%d s-task:%s resume from pause, status:%s. pause task num:%d", pMeta->vgId, pTask->id.idStr, streamGetTaskStatusStr(status), num); + stInfo("vgId:%d s-task:%s resume from pause, status:%s. pause task num:%d", pMeta->vgId, pTask->id.idStr, streamGetTaskStatusStr(status), num); } else if (pTask->info.taskLevel == TASK_LEVEL__SINK) { int32_t num = atomic_sub_fetch_32(&pMeta->numOfPausedTasks, 1); - qInfo("vgId:%d s-task:%s sink task.resume from pause, status:%s. pause task num:%d", pMeta->vgId, pTask->id.idStr, streamGetTaskStatusStr(status), num); + stInfo("vgId:%d s-task:%s sink task.resume from pause, status:%s. pause task num:%d", pMeta->vgId, pTask->id.idStr, streamGetTaskStatusStr(status), num); } else { - qError("s-task:%s not in pause, failed to resume, status:%s", pTask->id.idStr, streamGetTaskStatusStr(status)); + stError("s-task:%s not in pause, failed to resume, status:%s", pTask->id.idStr, streamGetTaskStatusStr(status)); } } @@ -902,57 +987,28 @@ void streamTaskDisablePause(SStreamTask* pTask) { // pre-condition check const char* id = pTask->id.idStr; while (pTask->status.taskStatus == TASK_STATUS__PAUSE) { - qDebug("s-task:%s already in pause, wait for pause being cancelled, and set pause disabled, recheck in 100ms", id); + stDebug("s-task:%s already in pause, wait for pause being cancelled, and set pause disabled, recheck in 100ms", id); taosMsleep(100); } - qDebug("s-task:%s disable task pause", id); + stDebug("s-task:%s disable task pause", id); pTask->status.pauseAllowed = 0; } void streamTaskEnablePause(SStreamTask* pTask) { - qDebug("s-task:%s enable task pause", pTask->id.idStr); + stDebug("s-task:%s enable task pause", pTask->id.idStr); pTask->status.pauseAllowed = 1; } -void streamTaskHalt(SStreamTask* pTask) { - int8_t status = pTask->status.taskStatus; - if (status == TASK_STATUS__DROPPING || status == TASK_STATUS__STOP) { - return; - } - - if (status == TASK_STATUS__HALT) { - return; - } - - // wait for checkpoint completed - while(pTask->status.taskStatus == TASK_STATUS__CK) { - qDebug("s-task:%s status:%s during generating checkpoint, wait for 1sec and retry set status:halt", pTask->id.idStr, - streamGetTaskStatusStr(TASK_STATUS__CK)); - taosMsleep(1000); - } - - // upgrade to halt status - if (status == TASK_STATUS__PAUSE) { - qDebug("s-task:%s upgrade status to %s from %s", pTask->id.idStr, streamGetTaskStatusStr(TASK_STATUS__HALT), - streamGetTaskStatusStr(TASK_STATUS__PAUSE)); - } else { - qDebug("s-task:%s halt task", pTask->id.idStr); - } - - pTask->status.keepTaskStatus = status; - pTask->status.taskStatus = TASK_STATUS__HALT; -} - void streamTaskResumeFromHalt(SStreamTask* pTask) { const char* id = pTask->id.idStr; int8_t status = pTask->status.taskStatus; if (status != TASK_STATUS__HALT) { - qError("s-task:%s not in halt status, status:%s", id, streamGetTaskStatusStr(status)); + stError("s-task:%s not in halt status, status:%s", id, streamGetTaskStatusStr(status)); return; } pTask->status.taskStatus = pTask->status.keepTaskStatus; pTask->status.keepTaskStatus = TASK_STATUS__NORMAL; - qDebug("s-task:%s resume from halt, current status:%s", id, streamGetTaskStatusStr(pTask->status.taskStatus)); + stDebug("s-task:%s resume from halt, current status:%s", id, streamGetTaskStatusStr(pTask->status.taskStatus)); } diff --git a/source/libs/stream/src/streamSnapshot.c b/source/libs/stream/src/streamSnapshot.c index feb127e313..d988d242c8 100644 --- a/source/libs/stream/src/streamSnapshot.c +++ b/source/libs/stream/src/streamSnapshot.c @@ -18,6 +18,7 @@ #include "rocksdb/c.h" #include "streamBackendRocksdb.h" #include "tcommon.h" +#include "streamInt.h" enum SBackendFileType { ROCKSDB_OPTIONS_TYPE = 1, @@ -122,10 +123,10 @@ int32_t streamSnapHandleInit(SStreamSnapHandle* pHandle, char* path, int64_t chk chkpId); if (taosIsDir(tdir)) { validChkp = 1; - qInfo("%s start to read snap %s", STREAM_STATE_TRANSFER, tdir); + stInfo("%s start to read snap %s", STREAM_STATE_TRANSFER, tdir); streamBackendAddInUseChkp(pMeta, chkpId); } else { - qWarn("%s failed to read from %s, reason: dir not exist,retry to default state dir", STREAM_STATE_TRANSFER, tdir); + stWarn("%s failed to read from %s, reason: dir not exist,retry to default state dir", STREAM_STATE_TRANSFER, tdir); } } @@ -137,22 +138,22 @@ int32_t streamSnapHandleInit(SStreamSnapHandle* pHandle, char* path, int64_t chk taosMemoryFree(tdir); tdir = chkpdir; - qInfo("%s start to trigger checkpoint on %s", STREAM_STATE_TRANSFER, tdir); + stInfo("%s start to trigger checkpoint on %s", STREAM_STATE_TRANSFER, tdir); code = streamBackendTriggerChkp(pMeta, tdir); if (code != 0) { - qError("%s failed to trigger chekckpoint at %s", STREAM_STATE_TRANSFER, tdir); + stError("%s failed to trigger chekckpoint at %s", STREAM_STATE_TRANSFER, tdir); taosMemoryFree(tdir); return code; } chkpId = 0; } - qInfo("%s start to read dir: %s", STREAM_STATE_TRANSFER, tdir); + stInfo("%s start to read dir: %s", STREAM_STATE_TRANSFER, tdir); TdDirPtr pDir = taosOpenDir(tdir); if (NULL == pDir) { - qError("%s failed to open %s", STREAM_STATE_TRANSFER, tdir); + stError("%s failed to open %s", STREAM_STATE_TRANSFER, tdir); goto _err; } @@ -202,14 +203,14 @@ int32_t streamSnapHandleInit(SStreamSnapHandle* pHandle, char* path, int64_t chk } sprintf(buf + strlen(buf) - 1, "]"); - qInfo("%s get file list: %s", STREAM_STATE_TRANSFER, buf); + stInfo("%s get file list: %s", STREAM_STATE_TRANSFER, buf); taosMemoryFree(buf); } taosCloseDir(&pDir); if (pFile->pCurrent == NULL) { - qError("%s failed to open %s, reason: no valid file", STREAM_STATE_TRANSFER, tdir); + stError("%s failed to open %s, reason: no valid file", STREAM_STATE_TRANSFER, tdir); code = -1; tdir = NULL; goto _err; @@ -333,24 +334,24 @@ int32_t streamSnapRead(SStreamSnapReader* pReader, uint8_t** ppData, int64_t* si return 0; } else { pHandle->fd = streamOpenFile(pFile->path, item->name, TD_FILE_READ); - qDebug("%s open file %s, current offset:%" PRId64 ", size:% " PRId64 ", file no.%d", STREAM_STATE_TRANSFER, + stDebug("%s open file %s, current offset:%" PRId64 ", size:% " PRId64 ", file no.%d", STREAM_STATE_TRANSFER, item->name, (int64_t)pHandle->offset, item->size, pHandle->currFileIdx); } } - qDebug("%s start to read file %s, current offset:%" PRId64 ", size:%" PRId64 ", file no.%d", STREAM_STATE_TRANSFER, + stDebug("%s start to read file %s, current offset:%" PRId64 ", size:%" PRId64 ", file no.%d", STREAM_STATE_TRANSFER, item->name, (int64_t)pHandle->offset, item->size, pHandle->currFileIdx); uint8_t* buf = taosMemoryCalloc(1, sizeof(SStreamSnapBlockHdr) + kBlockSize); int64_t nread = taosPReadFile(pHandle->fd, buf + sizeof(SStreamSnapBlockHdr), kBlockSize, pHandle->offset); if (nread == -1) { taosMemoryFree(buf); code = TAOS_SYSTEM_ERROR(terrno); - qError("%s snap failed to read snap, file name:%s, type:%d,reason:%s", STREAM_STATE_TRANSFER, item->name, + stError("%s snap failed to read snap, file name:%s, type:%d,reason:%s", STREAM_STATE_TRANSFER, item->name, item->type, tstrerror(code)); return -1; } else if (nread > 0 && nread <= kBlockSize) { // left bytes less than kBlockSize - qDebug("%s read file %s, current offset:%" PRId64 ",size:% " PRId64 ", file no.%d", STREAM_STATE_TRANSFER, + stDebug("%s read file %s, current offset:%" PRId64 ",size:% " PRId64 ", file no.%d", STREAM_STATE_TRANSFER, item->name, (int64_t)pHandle->offset, item->size, pHandle->currFileIdx); pHandle->offset += nread; if (pHandle->offset >= item->size || nread < kBlockSize) { @@ -359,7 +360,7 @@ int32_t streamSnapRead(SStreamSnapReader* pReader, uint8_t** ppData, int64_t* si pHandle->currFileIdx += 1; } } else { - qDebug("%s no data read, close file no.%d, move to next file, open and read", STREAM_STATE_TRANSFER, + stDebug("%s no data read, close file no.%d, move to next file, open and read", STREAM_STATE_TRANSFER, pHandle->currFileIdx); taosCloseFile(&pHandle->fd); pHandle->offset = 0; @@ -377,7 +378,7 @@ int32_t streamSnapRead(SStreamSnapReader* pReader, uint8_t** ppData, int64_t* si nread = taosPReadFile(pHandle->fd, buf + sizeof(SStreamSnapBlockHdr), kBlockSize, pHandle->offset); pHandle->offset += nread; - qDebug("%s open file and read file %s, current offset:%" PRId64 ", size:% " PRId64 ", file no.%d", + stDebug("%s open file and read file %s, current offset:%" PRId64 ", size:% " PRId64 ", file no.%d", STREAM_STATE_TRANSFER, item->name, (int64_t)pHandle->offset, item->size, pHandle->currFileIdx); } @@ -433,7 +434,7 @@ int32_t streamSnapWrite(SStreamSnapWriter* pWriter, uint8_t* pData, uint32_t nDa pHandle->fd = streamOpenFile(pFile->path, pItem->name, TD_FILE_CREATE | TD_FILE_WRITE | TD_FILE_APPEND); if (pHandle->fd == NULL) { code = TAOS_SYSTEM_ERROR(terrno); - qError("%s failed to open file name:%s%s%s, reason:%s", STREAM_STATE_TRANSFER, pFile->path, TD_DIRSEP, pHdr->name, + stError("%s failed to open file name:%s%s%s, reason:%s", STREAM_STATE_TRANSFER, pFile->path, TD_DIRSEP, pHdr->name, tstrerror(code)); } } @@ -442,7 +443,7 @@ int32_t streamSnapWrite(SStreamSnapWriter* pWriter, uint8_t* pData, uint32_t nDa int64_t bytes = taosPWriteFile(pHandle->fd, pHdr->data, pHdr->size, pHandle->offset); if (bytes != pHdr->size) { code = TAOS_SYSTEM_ERROR(terrno); - qError("%s failed to write snap, file name:%s, reason:%s", STREAM_STATE_TRANSFER, pHdr->name, tstrerror(code)); + stError("%s failed to write snap, file name:%s, reason:%s", STREAM_STATE_TRANSFER, pHdr->name, tstrerror(code)); return code; } pHandle->offset += bytes; @@ -460,7 +461,7 @@ int32_t streamSnapWrite(SStreamSnapWriter* pWriter, uint8_t* pData, uint32_t nDa pHandle->fd = streamOpenFile(pFile->path, pItem->name, TD_FILE_CREATE | TD_FILE_WRITE | TD_FILE_APPEND); if (pHandle->fd == NULL) { code = TAOS_SYSTEM_ERROR(terrno); - qError("%s failed to open file name:%s%s%s, reason:%s", STREAM_STATE_TRANSFER, pFile->path, TD_DIRSEP, pHdr->name, + stError("%s failed to open file name:%s%s%s, reason:%s", STREAM_STATE_TRANSFER, pFile->path, TD_DIRSEP, pHdr->name, tstrerror(code)); } @@ -484,7 +485,7 @@ int32_t streamSnapWriterClose(SStreamSnapWriter* pWriter, int8_t rollback) { n += sprintf(buf + n, "%s %" PRId64 "]", item->name, item->size); } } - qDebug("%s snap get file list, %s", STREAM_STATE_TRANSFER, buf); + stDebug("%s snap get file list, %s", STREAM_STATE_TRANSFER, buf); taosMemoryFree(buf); } diff --git a/source/libs/stream/src/streamState.c b/source/libs/stream/src/streamState.c index 83aed42fe2..075414b876 100644 --- a/source/libs/stream/src/streamState.c +++ b/source/libs/stream/src/streamState.c @@ -91,7 +91,7 @@ int stateKeyCmpr(const void* pKey1, int kLen1, const void* pKey2, int kLen2) { } SStreamState* streamStateOpen(char* path, void* pTask, bool specPath, int32_t szPage, int32_t pages) { - qDebug("open stream state, %s", path); + stDebug("open stream state, %s", path); SStreamState* pState = taosMemoryCalloc(1, sizeof(SStreamState)); if (pState == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; @@ -139,7 +139,7 @@ SStreamState* streamStateOpen(char* path, void* pTask, bool specPath, int32_t sz pState->pTdbState->backendCfWrapperId = id; pState->pTdbState->pBackendCfWrapper = taosAcquireRef(streamBackendCfWrapperId, id); // already exist stream task for - qInfo("already exist stream-state for %s", pState->pTdbState->idstr); + stInfo("already exist stream-state for %s", pState->pTdbState->idstr); // taosAcquireRef(streamBackendId, pState->streamBackendRid); } taosThreadMutexUnlock(&pMeta->backendMutex); @@ -149,7 +149,7 @@ SStreamState* streamStateOpen(char* path, void* pTask, bool specPath, int32_t sz _hash_fn_t hashFn = taosGetDefaultHashFunction(TSDB_DATA_TYPE_BIGINT); pState->parNameMap = tSimpleHashInit(1024, hashFn); - qInfo("succ to open state %p on backend %p 0x%" PRIx64 "-%d", pState, pMeta->streamBackend, pState->streamId, + stInfo("succ to open state %p on backend %p 0x%" PRIx64 "-%d", pState, pMeta->streamBackend, pState->streamId, pState->taskId); return pState; @@ -468,7 +468,7 @@ int32_t streamStateAddIfNotExist(SStreamState* pState, const SWinKey* key, void* int32_t streamStateReleaseBuf(SStreamState* pState, const SWinKey* key, void* pVal) { // todo refactor - qDebug("streamStateReleaseBuf"); + stDebug("streamStateReleaseBuf"); if (!pVal) { return 0; } @@ -717,7 +717,7 @@ void streamStateFreeCur(SStreamStateCur* pCur) { if (!pCur) { return; } - qDebug("streamStateFreeCur"); + stDebug("streamStateFreeCur"); rocksdb_iter_destroy(pCur->iter); if (pCur->snapshot) rocksdb_release_snapshot(pCur->db, pCur->snapshot); rocksdb_readoptions_destroy(pCur->readOpt); @@ -736,7 +736,7 @@ void streamStateFreeVal(void* val) { int32_t streamStateSessionPut(SStreamState* pState, const SSessionKey* key, const void* value, int32_t vLen) { #ifdef USE_ROCKSDB - qDebug("===stream===save skey:%" PRId64 ", ekey:%" PRId64 ", groupId:%" PRIu64, key->win.skey, key->win.ekey, + stDebug("===stream===save skey:%" PRId64 ", ekey:%" PRId64 ", groupId:%" PRIu64, key->win.skey, key->win.ekey, key->groupId); return streamStateSessionPut_rocksdb(pState, key, value, vLen); #else @@ -771,7 +771,7 @@ int32_t streamStateSessionGet(SStreamState* pState, SSessionKey* key, void** pVa int32_t streamStateSessionDel(SStreamState* pState, const SSessionKey* key) { #ifdef USE_ROCKSDB - qDebug("===stream===delete skey:%" PRId64 ", ekey:%" PRId64 ", groupId:%" PRIu64, key->win.skey, key->win.ekey, + stDebug("===stream===delete skey:%" PRId64 ", ekey:%" PRId64 ", groupId:%" PRIu64, key->win.skey, key->win.ekey, key->groupId); return streamStateSessionDel_rocksdb(pState, key); #else @@ -1088,7 +1088,7 @@ _end: } int32_t streamStatePutParName(SStreamState* pState, int64_t groupId, const char tbname[TSDB_TABLE_NAME_LEN]) { - qDebug("try to write to cf parname"); + stDebug("try to write to cf parname"); #ifdef USE_ROCKSDB if (tSimpleHashGetSize(pState->parNameMap) > MAX_TABLE_NAME_NUM) { if (tSimpleHashGet(pState->parNameMap, &groupId, sizeof(int64_t)) == NULL) { diff --git a/source/libs/stream/src/streamTask.c b/source/libs/stream/src/streamTask.c index d2e306fa01..e5088e9c69 100644 --- a/source/libs/stream/src/streamTask.c +++ b/source/libs/stream/src/streamTask.c @@ -129,9 +129,9 @@ int32_t tEncodeStreamTask(SEncoder* pEncoder, const SStreamTask* pTask) { } else if (pTask->outputInfo.type == TASK_OUTPUT__FETCH) { if (tEncodeI8(pEncoder, pTask->fetchSink.reserved) < 0) return -1; } else if (pTask->outputInfo.type == TASK_OUTPUT__FIXED_DISPATCH) { - if (tEncodeI32(pEncoder, pTask->fixedEpDispatcher.taskId) < 0) return -1; - if (tEncodeI32(pEncoder, pTask->fixedEpDispatcher.nodeId) < 0) return -1; - if (tEncodeSEpSet(pEncoder, &pTask->fixedEpDispatcher.epSet) < 0) return -1; + if (tEncodeI32(pEncoder, pTask->fixedDispatcher.taskId) < 0) return -1; + if (tEncodeI32(pEncoder, pTask->fixedDispatcher.nodeId) < 0) return -1; + if (tEncodeSEpSet(pEncoder, &pTask->fixedDispatcher.epSet) < 0) return -1; } else if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { if (tSerializeSUseDbRspImp(pEncoder, &pTask->shuffleDispatcher.dbInfo) < 0) return -1; if (tEncodeCStr(pEncoder, pTask->shuffleDispatcher.stbFullName) < 0) return -1; @@ -211,9 +211,9 @@ int32_t tDecodeStreamTask(SDecoder* pDecoder, SStreamTask* pTask) { } else if (pTask->outputInfo.type == TASK_OUTPUT__FETCH) { if (tDecodeI8(pDecoder, &pTask->fetchSink.reserved) < 0) return -1; } else if (pTask->outputInfo.type == TASK_OUTPUT__FIXED_DISPATCH) { - if (tDecodeI32(pDecoder, &pTask->fixedEpDispatcher.taskId) < 0) return -1; - if (tDecodeI32(pDecoder, &pTask->fixedEpDispatcher.nodeId) < 0) return -1; - if (tDecodeSEpSet(pDecoder, &pTask->fixedEpDispatcher.epSet) < 0) return -1; + if (tDecodeI32(pDecoder, &pTask->fixedDispatcher.taskId) < 0) return -1; + if (tDecodeI32(pDecoder, &pTask->fixedDispatcher.nodeId) < 0) return -1; + if (tDecodeSEpSet(pDecoder, &pTask->fixedDispatcher.epSet) < 0) return -1; } else if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { if (tDeserializeSUseDbRspImp(pDecoder, &pTask->shuffleDispatcher.dbInfo) < 0) return -1; if (tDecodeCStrTo(pDecoder, pTask->shuffleDispatcher.stbFullName) < 0) return -1; @@ -289,24 +289,21 @@ static void freeUpstreamItem(void* p) { void tFreeStreamTask(SStreamTask* pTask) { int32_t taskId = pTask->id.taskId; - STaskExecStatisInfo* pStatis = &pTask->taskExecInfo; + STaskExecStatisInfo* pStatis = &pTask->execInfo; - qDebug("start to free s-task:0x%x, %p, state:%p, status:%s", taskId, pTask, pTask->pState, + stDebug("start to free s-task:0x%x, %p, state:%p, status:%s", taskId, pTask, pTask->pState, streamGetTaskStatusStr(pTask->status.taskStatus)); - qDebug("s-task:0x%x exec info: create:%" PRId64 ", init:%" PRId64 ", start:%" PRId64 + stDebug("s-task:0x%x task exec summary: create:%" PRId64 ", init:%" PRId64 ", start:%" PRId64 ", updateCount:%d latestUpdate:%" PRId64 ", latestCheckPoint:%" PRId64 ", ver:%" PRId64 - " nextProcessVer:%" PRId64, + " nextProcessVer:%" PRId64", checkpointCount:%d", taskId, pStatis->created, pStatis->init, pStatis->start, pStatis->updateCount, pStatis->latestUpdateTs, - pTask->chkInfo.checkpointId, pTask->chkInfo.checkpointVer, pTask->chkInfo.nextProcessVer); - - if (pStatis->created == 0 || pStatis->init == 0 || pStatis->start == 0) { - int32_t k = 1; - } + pTask->chkInfo.checkpointId, pTask->chkInfo.checkpointVer, pTask->chkInfo.nextProcessVer, + pStatis->checkpoint); // remove the ref by timer while (pTask->status.timerActive > 0) { - qDebug("s-task:%s wait for task stop timer activities", pTask->id.idStr); + stDebug("s-task:%s wait for task stop timer activities", pTask->id.idStr); taosMsleep(10); } @@ -315,9 +312,18 @@ void tFreeStreamTask(SStreamTask* pTask) { pTask->schedInfo.pTimer = NULL; } - if (pTask->launchTaskTimer != NULL) { - taosTmrStop(pTask->launchTaskTimer); - pTask->launchTaskTimer = NULL; + if (pTask->pTimer != NULL) { + if (pTask->pTimer->hTaskLaunchTimer != NULL) { + taosTmrStop(pTask->pTimer->hTaskLaunchTimer); + pTask->pTimer->hTaskLaunchTimer = NULL; + } + + if (pTask->pTimer->dispatchTimer != NULL) { + taosTmrStop(pTask->pTimer->dispatchTimer); + pTask->pTimer->dispatchTimer = NULL; + } + + taosMemoryFreeClear(pTask->pTimer); } int32_t status = atomic_load_8((int8_t*)&(pTask->status.taskStatus)); @@ -342,6 +348,13 @@ void tFreeStreamTask(SStreamTask* pTask) { walCloseReader(pTask->exec.pWalReader); } + pTask->pReadyMsgList = taosArrayDestroy(pTask->pReadyMsgList); + if (pTask->msgInfo.pData != NULL) { + destroyDispatchMsg(pTask->msgInfo.pData, getNumOfDispatchBranch(pTask)); + pTask->msgInfo.pData = NULL; + pTask->msgInfo.dispatchMsgType = 0; + } + if (pTask->outputInfo.type == TASK_OUTPUT__TABLE) { tDeleteSchemaWrapper(pTask->tbSink.pSchemaWrapper); taosMemoryFree(pTask->tbSink.pTSchema); @@ -352,16 +365,10 @@ void tFreeStreamTask(SStreamTask* pTask) { } if (pTask->pState) { - qDebug("s-task:0x%x start to free task state", taskId); + stDebug("s-task:0x%x start to free task state", taskId); streamStateClose(pTask->pState, status == TASK_STATUS__DROPPING); } - pTask->pReadyMsgList = taosArrayDestroy(pTask->pReadyMsgList); - if (pTask->msgInfo.pData != NULL) { - destroyStreamDataBlock(pTask->msgInfo.pData); - pTask->msgInfo.pData = NULL; - } - if (pTask->id.idStr != NULL) { taosMemoryFree((void*)pTask->id.idStr); } @@ -380,11 +387,12 @@ void tFreeStreamTask(SStreamTask* pTask) { pTask->pUpstreamInfoList = NULL; } + pTask->msgInfo.pRetryList = taosArrayDestroy(pTask->msgInfo.pRetryList); taosMemoryFree(pTask->pTokenBucket); taosThreadMutexDestroy(&pTask->lock); taosMemoryFree(pTask); - qDebug("s-task:0x%x free task completed", taskId); + stDebug("s-task:0x%x free task completed", taskId); } int32_t streamTaskInit(SStreamTask* pTask, SStreamMeta* pMeta, SMsgCb* pMsgCb, int64_t ver) { @@ -396,11 +404,11 @@ int32_t streamTaskInit(SStreamTask* pTask, SStreamMeta* pMeta, SMsgCb* pMsgCb, i pTask->outputInfo.queue = streamQueueOpen(512 << 10); if (pTask->inputInfo.queue == NULL || pTask->outputInfo.queue == NULL) { - qError("s-task:%s failed to prepare the input/output queue, initialize task failed", pTask->id.idStr); + stError("s-task:%s failed to prepare the input/output queue, initialize task failed", pTask->id.idStr); return TSDB_CODE_OUT_OF_MEMORY; } - pTask->taskExecInfo.created = taosGetTimestampMs(); + pTask->execInfo.created = taosGetTimestampMs(); pTask->inputInfo.status = TASK_INPUT_STATUS__NORMAL; pTask->outputInfo.status = TASK_OUTPUT_STATUS__NORMAL; pTask->pMeta = pMeta; @@ -409,10 +417,17 @@ int32_t streamTaskInit(SStreamTask* pTask, SStreamMeta* pMeta, SMsgCb* pMsgCb, i pTask->dataRange.range.maxVer = ver; pTask->dataRange.range.minVer = ver; pTask->pMsgCb = pMsgCb; + pTask->msgInfo.pRetryList = taosArrayInit(4, sizeof(int32_t)); pTask->pTokenBucket = taosMemoryCalloc(1, sizeof(STokenBucket)); if (pTask->pTokenBucket == NULL) { - qError("s-task:%s failed to prepare the tokenBucket, code:%s", pTask->id.idStr, tstrerror(TSDB_CODE_OUT_OF_MEMORY)); + stError("s-task:%s failed to prepare the tokenBucket, code:%s", pTask->id.idStr, tstrerror(TSDB_CODE_OUT_OF_MEMORY)); + return TSDB_CODE_OUT_OF_MEMORY; + } + + pTask->pTimer = taosMemoryCalloc(1, sizeof(STaskTimer)); + if (pTask->pTimer == NULL) { + stError("s-task:%s failed to prepare the timer, code:%s", pTask->id.idStr, tstrerror(TSDB_CODE_OUT_OF_MEMORY)); return TSDB_CODE_OUT_OF_MEMORY; } @@ -421,13 +436,13 @@ int32_t streamTaskInit(SStreamTask* pTask, SStreamMeta* pMeta, SMsgCb* pMsgCb, i TdThreadMutexAttr attr = {0}; int code = taosThreadMutexAttrInit(&attr); if (code != 0) { - qError("s-task:%s initElapsed mutex attr failed, code:%s", pTask->id.idStr, tstrerror(code)); + stError("s-task:%s initElapsed mutex attr failed, code:%s", pTask->id.idStr, tstrerror(code)); return code; } code = taosThreadMutexAttrSetType(&attr, PTHREAD_MUTEX_RECURSIVE); if (code != 0) { - qError("s-task:%s set mutex attr recursive, code:%s", pTask->id.idStr, tstrerror(code)); + stError("s-task:%s set mutex attr recursive, code:%s", pTask->id.idStr, tstrerror(code)); return code; } @@ -490,7 +505,7 @@ void streamTaskUpdateUpstreamInfo(SStreamTask* pTask, int32_t nodeId, const SEpS SStreamChildEpInfo* pInfo = taosArrayGetP(pTask->pUpstreamInfoList, i); if (pInfo->nodeId == nodeId) { epsetAssign(&pInfo->epSet, pEpSet); - qDebug("s-task:0x%x update the upstreamInfo taskId:0x%x(nodeId:%d) newEpset:%s", pTask->id.taskId, + stDebug("s-task:0x%x update the upstreamInfo taskId:0x%x(nodeId:%d) newEpset:%s", pTask->id.taskId, pInfo->taskId, nodeId, buf); break; } @@ -498,7 +513,7 @@ void streamTaskUpdateUpstreamInfo(SStreamTask* pTask, int32_t nodeId, const SEpS } void streamTaskSetFixedDownstreamInfo(SStreamTask* pTask, const SStreamTask* pDownstreamTask) { - STaskDispatcherFixedEp* pDispatcher = &pTask->fixedEpDispatcher; + STaskDispatcherFixed* pDispatcher = &pTask->fixedDispatcher; pDispatcher->taskId = pDownstreamTask->id.taskId; pDispatcher->nodeId = pDownstreamTask->info.nodeId; pDispatcher->epSet = pDownstreamTask->info.epSet; @@ -521,16 +536,16 @@ void streamTaskUpdateDownstreamInfo(SStreamTask* pTask, int32_t nodeId, const SE if (pVgInfo->vgId == nodeId) { epsetAssign(&pVgInfo->epSet, pEpSet); - qDebug("s-task:0x%x update the dispatch info, task:0x%x(nodeId:%d) newEpset:%s", pTask->id.taskId, + stDebug("s-task:0x%x update the dispatch info, task:0x%x(nodeId:%d) newEpset:%s", pTask->id.taskId, pVgInfo->taskId, nodeId, buf); break; } } } else if (type == TASK_OUTPUT__FIXED_DISPATCH) { - STaskDispatcherFixedEp* pDispatcher = &pTask->fixedEpDispatcher; + STaskDispatcherFixed* pDispatcher = &pTask->fixedDispatcher; if (pDispatcher->nodeId == nodeId) { epsetAssign(&pDispatcher->epSet, pEpSet); - qDebug("s-task:0x%x update the dispatch info, task:0x%x(nodeId:%d) newEpSet:%s", pTask->id.taskId, + stDebug("s-task:0x%x update the dispatch info, task:0x%x(nodeId:%d) newEpSet:%s", pTask->id.taskId, pDispatcher->taskId, nodeId, buf); } } else { @@ -539,20 +554,26 @@ void streamTaskUpdateDownstreamInfo(SStreamTask* pTask, int32_t nodeId, const SE } int32_t streamTaskStop(SStreamTask* pTask) { - SStreamMeta* pMeta = pTask->pMeta; + int32_t vgId = pTask->pMeta->vgId; int64_t st = taosGetTimestampMs(); const char* id = pTask->id.idStr; + taosThreadMutexLock(&pTask->lock); + if (pTask->status.taskStatus == TASK_STATUS__CK) { + stDebug("s-task:%s in checkpoint will be discarded since task is stopped", id); + } pTask->status.taskStatus = TASK_STATUS__STOP; - qKillTask(pTask->exec.pExecutor, TSDB_CODE_SUCCESS); + taosThreadMutexUnlock(&pTask->lock); + qKillTask(pTask->exec.pExecutor, TSDB_CODE_SUCCESS); while (/*pTask->status.schedStatus != TASK_SCHED_STATUS__INACTIVE */ !streamTaskIsIdle(pTask)) { - qDebug("s-task:%s level:%d wait for task to be idle, check again in 100ms", id, pTask->info.taskLevel); + stDebug("s-task:%s level:%d wait for task to be idle and then close, check again in 100ms", id, + pTask->info.taskLevel); taosMsleep(100); } int64_t el = taosGetTimestampMs() - st; - qDebug("vgId:%d s-task:%s is closed in %" PRId64 " ms", pMeta->vgId, pTask->id.idStr, el); + stDebug("vgId:%d s-task:%s is closed in %" PRId64 " ms", vgId, id, el); return 0; } @@ -562,7 +583,7 @@ int32_t doUpdateTaskEpset(SStreamTask* pTask, int32_t nodeId, SEpSet* pEpSet) { if (pTask->info.nodeId == nodeId) { // execution task should be moved away epsetAssign(&pTask->info.epSet, pEpSet); EPSET_TO_STR(pEpSet, buf) - qDebug("s-task:0x%x (vgId:%d) self node epset is updated %s", pTask->id.taskId, nodeId, buf); + stDebug("s-task:0x%x (vgId:%d) self node epset is updated %s", pTask->id.taskId, nodeId, buf); } // check for the dispath info and the upstream task info @@ -580,14 +601,14 @@ int32_t doUpdateTaskEpset(SStreamTask* pTask, int32_t nodeId, SEpSet* pEpSet) { } int32_t streamTaskUpdateEpsetInfo(SStreamTask* pTask, SArray* pNodeList) { - STaskExecStatisInfo* p = &pTask->taskExecInfo; + STaskExecStatisInfo* p = &pTask->execInfo; int32_t numOfNodes = taosArrayGetSize(pNodeList); int64_t prevTs = p->latestUpdateTs; p->latestUpdateTs = taosGetTimestampMs(); p->updateCount += 1; - qDebug("s-task:%s update task nodeEp epset, updatedNodes:%d, updateCount:%d, prevTs:%" PRId64, pTask->id.idStr, + stDebug("s-task:%s update task nodeEp epset, updatedNodes:%d, updateCount:%d, prevTs:%" PRId64, pTask->id.idStr, numOfNodes, p->updateCount, prevTs); for (int32_t i = 0; i < taosArrayGetSize(pNodeList); ++i) { @@ -608,7 +629,7 @@ void streamTaskResetUpstreamStageInfo(SStreamTask* pTask) { pInfo->stage = -1; } - qDebug("s-task:%s reset all upstream tasks stage info", pTask->id.idStr); + stDebug("s-task:%s reset all upstream tasks stage info", pTask->id.idStr); } int8_t streamTaskSetSchedStatusWait(SStreamTask* pTask) { @@ -658,11 +679,11 @@ int32_t streamBuildAndSendDropTaskMsg(SMsgCb* pMsgCb, int32_t vgId, SStreamTaskI SRpcMsg msg = {.msgType = TDMT_STREAM_TASK_DROP, .pCont = pReq, .contLen = sizeof(SVDropStreamTaskReq)}; int32_t code = tmsgPutToQueue(pMsgCb, WRITE_QUEUE, &msg); if (code != TSDB_CODE_SUCCESS) { - qError("vgId:%d failed to send drop task:0x%x msg, code:%s", vgId, pTaskId->taskId, tstrerror(code)); + stError("vgId:%d failed to send drop task:0x%x msg, code:%s", vgId, pTaskId->taskId, tstrerror(code)); return code; } - qDebug("vgId:%d build and send drop table:0x%x msg", vgId, pTaskId->taskId); + stDebug("vgId:%d build and send drop table:0x%x msg", vgId, pTaskId->taskId); return code; } diff --git a/source/libs/transport/src/transSvr.c b/source/libs/transport/src/transSvr.c index 3117bbf00e..2007933497 100644 --- a/source/libs/transport/src/transSvr.c +++ b/source/libs/transport/src/transSvr.c @@ -1299,7 +1299,10 @@ int transSendResponse(const STransMsg* msg) { return 0; } SExHandle* exh = msg->info.handle; - int64_t refId = msg->info.refId; + if (exh == NULL) { + return 0; + } + int64_t refId = msg->info.refId; ASYNC_CHECK_HANDLE(exh, refId); STransMsg tmsg = *msg; diff --git a/source/util/src/tlog.c b/source/util/src/tlog.c index a431b091ec..b858421e25 100644 --- a/source/util/src/tlog.c +++ b/source/util/src/tlog.c @@ -100,6 +100,7 @@ int32_t tmrDebugFlag = 131; int32_t uDebugFlag = 131; int32_t rpcDebugFlag = 131; int32_t qDebugFlag = 131; +int32_t stDebugFlag = 131; int32_t wDebugFlag = 131; int32_t sDebugFlag = 131; int32_t tsdbDebugFlag = 131; diff --git a/tests/system-test/0-others/information_schema.py b/tests/system-test/0-others/information_schema.py index baf0682fbb..99bd90b006 100644 --- a/tests/system-test/0-others/information_schema.py +++ b/tests/system-test/0-others/information_schema.py @@ -22,7 +22,7 @@ class TDTestCase: def init(self, conn, logSql, replicaVar=1): self.replicaVar = int(replicaVar) tdLog.debug("start to execute %s" % __file__) - tdSql.init(conn.cursor()) + tdSql.init(conn.cursor(), True) self.setsql = TDSetSql() self.dbname = 'db' self.stbname = 'stb' @@ -217,7 +217,7 @@ class TDTestCase: tdSql.checkEqual(20470,len(tdSql.queryResult)) tdSql.query("select * from information_schema.ins_columns where db_name ='information_schema'") - tdSql.checkEqual(193, len(tdSql.queryResult)) + tdSql.checkEqual(194, len(tdSql.queryResult)) tdSql.query("select * from information_schema.ins_columns where db_name ='performance_schema'") tdSql.checkEqual(54, len(tdSql.queryResult)) diff --git a/tests/system-test/2-query/db.py b/tests/system-test/2-query/db.py index 451fc0caf3..6870c59a0d 100644 --- a/tests/system-test/2-query/db.py +++ b/tests/system-test/2-query/db.py @@ -55,7 +55,7 @@ class TDTestCase: tdSql.checkData(0, 2, 0) tdSql.query("show dnode 1 variables like '%debugFlag'") - tdSql.checkRows(21) + tdSql.checkRows(22) tdSql.query("show dnode 1 variables like '____debugFlag'") tdSql.checkRows(2)