diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index fd9b5667d7..5327428f5b 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -59,8 +59,6 @@ extern "C" { #define STREAM_EXEC_T_STOP_ALL_TASKS (-5) #define STREAM_EXEC_T_RESUME_TASK (-6) #define STREAM_EXEC_T_ADD_FAILED_TASK (-7) -// the load and start stream task should be executed after snode has started successfully, since the load of stream -// tasks may incur the download of checkpoint data from remote, which may consume significant network and CPU resources. typedef struct SStreamTask SStreamTask; typedef struct SStreamQueue SStreamQueue; @@ -190,8 +188,8 @@ typedef struct { SSDataBlock* pBlock; } SStreamRefDataBlock; -SStreamDataSubmit* streamDataSubmitNew(SPackedData* pData, int32_t type); -void streamDataSubmitDestroy(SStreamDataSubmit* pDataSubmit); +int32_t streamDataSubmitNew(SPackedData* pData, int32_t type, SStreamDataSubmit** pSubmit); +void streamDataSubmitDestroy(SStreamDataSubmit* pDataSubmit); typedef struct { char* qmsg; @@ -530,13 +528,13 @@ typedef struct STaskUpdateEntry { typedef int32_t (*__state_trans_user_fn)(SStreamTask*, void* param); -SStreamTask* tNewStreamTask(int64_t streamId, int8_t taskLevel, SEpSet* pEpset, bool fillHistory, int64_t triggerParam, - SArray* pTaskList, bool hasFillhistory, int8_t subtableWithoutMd5); -void tFreeStreamTask(SStreamTask* pTask); -int32_t tEncodeStreamTask(SEncoder* pEncoder, const SStreamTask* pTask); -int32_t tDecodeStreamTask(SDecoder* pDecoder, SStreamTask* pTask); -int32_t streamTaskInit(SStreamTask* pTask, SStreamMeta* pMeta, SMsgCb* pMsgCb, int64_t ver); -void streamFreeTaskState(SStreamTask* pTask, ETaskStatus status); +int32_t tNewStreamTask(int64_t streamId, int8_t taskLevel, SEpSet* pEpset, bool fillHistory, int64_t triggerParam, + SArray* pTaskList, bool hasFillhistory, int8_t subtableWithoutMd5, SStreamTask** pTask); +void tFreeStreamTask(SStreamTask* pTask); +int32_t tEncodeStreamTask(SEncoder* pEncoder, const SStreamTask* pTask); +int32_t tDecodeStreamTask(SDecoder* pDecoder, SStreamTask* pTask); +int32_t streamTaskInit(SStreamTask* pTask, SStreamMeta* pMeta, SMsgCb* pMsgCb, int64_t ver); +void streamFreeTaskState(SStreamTask* pTask, ETaskStatus status); int32_t tDecodeStreamTaskChkInfo(SDecoder* pDecoder, SCheckpointInfo* pChkpInfo); int32_t tDecodeStreamTaskId(SDecoder* pDecoder, STaskId* pTaskId); @@ -631,8 +629,10 @@ int32_t streamSetupScheduleTrigger(SStreamTask* pTask); int32_t streamProcessDispatchMsg(SStreamTask* pTask, SStreamDispatchReq* pReq, SRpcMsg* pMsg); int32_t streamProcessDispatchRsp(SStreamTask* pTask, SStreamDispatchRsp* pRsp, int32_t code); -SStreamUpstreamEpInfo* streamTaskGetUpstreamTaskEpInfo(SStreamTask* pTask, int32_t taskId); -SEpSet* streamTaskGetDownstreamEpInfo(SStreamTask* pTask, int32_t taskId); +int32_t streamTaskGetUpstreamTaskEpInfo(SStreamTask* pTask, int32_t taskId, SStreamUpstreamEpInfo** pEpInfo); +#if 0 +SEpSet* streamTaskGetDownstreamEpInfo(SStreamTask* pTask, int32_t taskId); +#endif void streamTaskInputFail(SStreamTask* pTask); @@ -641,8 +641,8 @@ bool streamTaskShouldPause(const SStreamTask* pStatus); bool streamTaskIsIdle(const SStreamTask* pTask); bool streamTaskReadyToRun(const SStreamTask* pTask, char** pStatus); -char* createStreamTaskIdStr(int64_t streamId, int32_t taskId); -SStreamTaskState* streamTaskGetStatus(const SStreamTask* pTask); +int32_t createStreamTaskIdStr(int64_t streamId, int32_t taskId, const char** pId); +SStreamTaskState streamTaskGetStatus(const SStreamTask* pTask); const char* streamTaskGetStatusStr(ETaskStatus status); void streamTaskResetStatus(SStreamTask* pTask); void streamTaskSetStatusReady(SStreamTask* pTask); @@ -724,43 +724,43 @@ int32_t streamSetParamForStreamScannerStep2(SStreamTask* pTask, SVersionRange* p SScanhistoryDataInfo streamScanHistoryData(SStreamTask* pTask, int64_t st); // stream task meta -void streamMetaInit(); -void streamMetaCleanup(); -SStreamMeta* streamMetaOpen(const char* path, void* ahandle, FTaskBuild expandFunc, FTaskExpand expandTaskFn, - int32_t vgId, int64_t stage, startComplete_fn_t fn); -void streamMetaClose(SStreamMeta* streamMeta); -int32_t streamMetaSaveTask(SStreamMeta* pMeta, SStreamTask* pTask); // save to stream meta store -int32_t streamMetaRemoveTask(SStreamMeta* pMeta, STaskId* pKey); -int32_t streamMetaRegisterTask(SStreamMeta* pMeta, int64_t ver, SStreamTask* pTask, bool* pAdded); -int32_t streamMetaUnregisterTask(SStreamMeta* pMeta, int64_t streamId, int32_t taskId); -int32_t streamMetaGetNumOfTasks(SStreamMeta* pMeta); -SStreamTask* streamMetaAcquireTaskNoLock(SStreamMeta* pMeta, int64_t streamId, int32_t taskId); -SStreamTask* streamMetaAcquireTask(SStreamMeta* pMeta, int64_t streamId, int32_t taskId); -void streamMetaReleaseTask(SStreamMeta* pMeta, SStreamTask* pTask); -SStreamTask* streamMetaAcquireOneTask(SStreamTask* pTask); -void streamMetaClear(SStreamMeta* pMeta); -void streamMetaInitBackend(SStreamMeta* pMeta); -int32_t streamMetaCommit(SStreamMeta* pMeta); -int64_t streamMetaGetLatestCheckpointId(SStreamMeta* pMeta); -void streamMetaNotifyClose(SStreamMeta* pMeta); -void streamMetaStartHb(SStreamMeta* pMeta); -bool streamMetaTaskInTimer(SStreamMeta* pMeta); -int32_t streamMetaAddTaskLaunchResult(SStreamMeta* pMeta, int64_t streamId, int32_t taskId, int64_t startTs, - int64_t endTs, bool ready); -int32_t streamMetaResetTaskStatus(SStreamMeta* pMeta); -int32_t streamMetaAddFailedTask(SStreamMeta* pMeta, int64_t streamId, int32_t taskId); -void streamMetaAddFailedTaskSelf(SStreamTask* pTask, int64_t failedTs); -void streamMetaAddIntoUpdateTaskList(SStreamMeta* pMeta, SStreamTask* pTask, SStreamTask* pHTask, int32_t transId, - int64_t startTs); -void streamMetaClearUpdateTaskList(SStreamMeta* pMeta); -void streamMetaInitUpdateTaskList(SStreamMeta* pMeta, int32_t transId); +void streamMetaInit(); +void streamMetaCleanup(); +int32_t streamMetaOpen(const char* path, void* ahandle, FTaskBuild expandFunc, FTaskExpand expandTaskFn, int32_t vgId, + int64_t stage, startComplete_fn_t fn, SStreamMeta** pMeta); +void streamMetaClose(SStreamMeta* streamMeta); +int32_t streamMetaSaveTask(SStreamMeta* pMeta, SStreamTask* pTask); // save to stream meta store +int32_t streamMetaRemoveTask(SStreamMeta* pMeta, STaskId* pKey); +int32_t streamMetaRegisterTask(SStreamMeta* pMeta, int64_t ver, SStreamTask* pTask, bool* pAdded); +int32_t streamMetaUnregisterTask(SStreamMeta* pMeta, int64_t streamId, int32_t taskId); +int32_t streamMetaGetNumOfTasks(SStreamMeta* pMeta); +int32_t streamMetaAcquireTaskNoLock(SStreamMeta* pMeta, int64_t streamId, int32_t taskId, SStreamTask** pTask); +int32_t streamMetaAcquireTask(SStreamMeta* pMeta, int64_t streamId, int32_t taskId, SStreamTask** pTask); +void streamMetaReleaseTask(SStreamMeta* pMeta, SStreamTask* pTask); +void streamMetaAcquireOneTask(SStreamTask* pTask); +void streamMetaClear(SStreamMeta* pMeta); +void streamMetaInitBackend(SStreamMeta* pMeta); +int32_t streamMetaCommit(SStreamMeta* pMeta); +int64_t streamMetaGetLatestCheckpointId(SStreamMeta* pMeta); +void streamMetaNotifyClose(SStreamMeta* pMeta); +void streamMetaStartHb(SStreamMeta* pMeta); +bool streamMetaTaskInTimer(SStreamMeta* pMeta); +int32_t streamMetaAddTaskLaunchResult(SStreamMeta* pMeta, int64_t streamId, int32_t taskId, int64_t startTs, + int64_t endTs, bool ready); +int32_t streamMetaResetTaskStatus(SStreamMeta* pMeta); +int32_t streamMetaAddFailedTask(SStreamMeta* pMeta, int64_t streamId, int32_t taskId); +void streamMetaAddFailedTaskSelf(SStreamTask* pTask, int64_t failedTs); +void streamMetaAddIntoUpdateTaskList(SStreamMeta* pMeta, SStreamTask* pTask, SStreamTask* pHTask, int32_t transId, + int64_t startTs); +void streamMetaClearUpdateTaskList(SStreamMeta* pMeta); +void streamMetaInitUpdateTaskList(SStreamMeta* pMeta, int32_t transId); void streamMetaRLock(SStreamMeta* pMeta); void streamMetaRUnLock(SStreamMeta* pMeta); void streamMetaWLock(SStreamMeta* pMeta); void streamMetaWUnLock(SStreamMeta* pMeta); void streamMetaResetStartInfo(STaskStartInfo* pMeta, int32_t vgId); -SArray* streamMetaSendMsgBeforeCloseTasks(SStreamMeta* pMeta); +int32_t streamMetaSendMsgBeforeCloseTasks(SStreamMeta* pMeta, SArray** pTaskList); void streamMetaUpdateStageRole(SStreamMeta* pMeta, int64_t stage, bool isLeader); void streamMetaLoadAllTasks(SStreamMeta* pMeta); int32_t streamMetaStartAllTasks(SStreamMeta* pMeta); @@ -771,6 +771,8 @@ int32_t streamTaskSendRestoreChkptMsg(SStreamTask* pTask); // timer tmr_h streamTimerGetInstance(); +void streamTmrReset(TAOS_TMR_CALLBACK fp, int32_t mseconds, void* param, void* handle, tmr_h* pTmrId, int32_t vgId, + const char* pMsg); // checkpoint int32_t streamProcessCheckpointSourceReq(SStreamTask* pTask, SStreamCheckpointSourceReq* pReq); @@ -787,14 +789,14 @@ int32_t streamTaskBuildCheckpointSourceRsp(SStreamCheckpointSourceReq* pReq, SRp int32_t setCode); int32_t streamSendChkptReportMsg(SStreamTask* pTask, SCheckpointInfo* pCheckpointInfo, int8_t dropRelHTask); int32_t streamTaskUpdateTaskCheckpointInfo(SStreamTask* pTask, bool restored, SVUpdateCheckpointInfoReq* pReq); -SActiveCheckpointInfo* streamTaskCreateActiveChkptInfo(); +int32_t streamTaskCreateActiveChkptInfo(SActiveCheckpointInfo** pRes); // stream task state machine, and event handling -SStreamTaskSM* streamCreateStateMachine(SStreamTask* pTask); -void* streamDestroyStateMachine(SStreamTaskSM* pSM); -int32_t streamTaskHandleEvent(SStreamTaskSM* pSM, EStreamTaskEvent event); -int32_t streamTaskHandleEventAsync(SStreamTaskSM* pSM, EStreamTaskEvent event, __state_trans_user_fn callbackFn, - void* param); +int32_t streamCreateStateMachine(SStreamTask* pTask); +void streamDestroyStateMachine(SStreamTaskSM* pSM); +int32_t streamTaskHandleEvent(SStreamTaskSM* pSM, EStreamTaskEvent event); +int32_t streamTaskHandleEventAsync(SStreamTaskSM* pSM, EStreamTaskEvent event, __state_trans_user_fn callbackFn, + void* param); int32_t streamTaskOnHandleEventSuccess(SStreamTaskSM* pSM, EStreamTaskEvent event, __state_trans_user_fn callbackFn, void* param); int32_t streamTaskRestoreStatus(SStreamTask* pTask); @@ -805,7 +807,7 @@ int32_t streamTaskBroadcastRetrieveReq(SStreamTask* pTask, SStreamRetrieveReq* r void streamTaskSendRetrieveRsp(SStreamRetrieveReq* pReq, SRpcMsg* pRsp); int32_t streamProcessHeartbeatRsp(SStreamMeta* pMeta, SMStreamHbRspMsg* pRsp); -int32_t streamTaskSendPreparedCheckpointsourceRsp(SStreamTask* pTask); +int32_t streamTaskSendCheckpointsourceRsp(SStreamTask* pTask); #ifdef __cplusplus diff --git a/source/dnode/mnode/impl/src/mndScheduler.c b/source/dnode/mnode/impl/src/mndScheduler.c index fc2e3b4619..db8c08e2e3 100644 --- a/source/dnode/mnode/impl/src/mndScheduler.c +++ b/source/dnode/mnode/impl/src/mndScheduler.c @@ -242,11 +242,11 @@ static int32_t doAddSinkTask(SStreamObj* pStream, SMnode* pMnode, SVgObj* pVgrou int64_t uid = (isFillhistory) ? pStream->hTaskUid : pStream->uid; SArray** pTaskList = (isFillhistory) ? taosArrayGetLast(pStream->pHTasksList) : taosArrayGetLast(pStream->tasks); - SStreamTask* pTask = tNewStreamTask(uid, TASK_LEVEL__SINK, pEpset, isFillhistory, 0, *pTaskList, - pStream->conf.fillHistory, pStream->subTableWithoutMd5); - if (pTask == NULL) { - terrno = TSDB_CODE_OUT_OF_MEMORY; - return terrno; + SStreamTask* pTask = NULL; + int32_t code = tNewStreamTask(uid, TASK_LEVEL__SINK, pEpset, isFillhistory, 0, *pTaskList, pStream->conf.fillHistory, + pStream->subTableWithoutMd5, &pTask); + if (code != 0) { + return code; } mDebug("doAddSinkTask taskId:%s, vgId:%d, isFillHistory:%d", pTask->id.idStr, pVgroup->vgId, isFillhistory); @@ -353,18 +353,13 @@ static void haltInitialTaskStatus(SStreamTask* pTask, SSubplan* pPlan, bool isFi } } -static SStreamTask* buildSourceTask(SStreamObj* pStream, SEpSet* pEpset, bool isFillhistory, bool useTriggerParam) { +static int32_t buildSourceTask(SStreamObj* pStream, SEpSet* pEpset, bool isFillhistory, bool useTriggerParam, SStreamTask** pTask) { uint64_t uid = (isFillhistory) ? pStream->hTaskUid : pStream->uid; SArray** pTaskList = (isFillhistory) ? taosArrayGetLast(pStream->pHTasksList) : taosArrayGetLast(pStream->tasks); - SStreamTask* pTask = - tNewStreamTask(uid, TASK_LEVEL__SOURCE, pEpset, isFillhistory, useTriggerParam ? pStream->conf.triggerParam : 0, - *pTaskList, pStream->conf.fillHistory, pStream->subTableWithoutMd5); - if (pTask == NULL) { - return NULL; - } - - return pTask; + int32_t code = tNewStreamTask(uid, TASK_LEVEL__SOURCE, pEpset, isFillhistory, useTriggerParam ? pStream->conf.triggerParam : 0, + *pTaskList, pStream->conf.fillHistory, pStream->subTableWithoutMd5, pTask); + return code; } static void addNewTaskList(SStreamObj* pStream) { @@ -398,10 +393,10 @@ static void setHTasksId(SStreamObj* pStream) { static int32_t doAddSourceTask(SMnode* pMnode, SSubplan* plan, SStreamObj* pStream, SEpSet* pEpset, int64_t skey, SArray* pVerList, SVgObj* pVgroup, bool isFillhistory, bool useTriggerParam) { - SStreamTask* pTask = buildSourceTask(pStream, pEpset, isFillhistory, useTriggerParam); - if (pTask == NULL) { - terrno = TSDB_CODE_OUT_OF_MEMORY; - return terrno; + SStreamTask* pTask = NULL; + int32_t code = buildSourceTask(pStream, pEpset, isFillhistory, useTriggerParam, &pTask); + if (code != TSDB_CODE_SUCCESS) { + return code; } mDebug("doAddSourceTask taskId:%s, vgId:%d, isFillHistory:%d", pTask->id.idStr, pVgroup->vgId, isFillhistory); @@ -412,10 +407,9 @@ static int32_t doAddSourceTask(SMnode* pMnode, SSubplan* plan, SStreamObj* pStre streamTaskSetDataRange(pTask, skey, pVerList, pVgroup->vgId); - int32_t code = mndAssignStreamTaskToVgroup(pMnode, pTask, plan, pVgroup); - if (code != 0) { - terrno = code; - return terrno; + code = mndAssignStreamTaskToVgroup(pMnode, pTask, plan, pVgroup); + if (code != TSDB_CODE_SUCCESS) { + return code; } return TDB_CODE_SUCCESS; @@ -498,28 +492,29 @@ static int32_t addSourceTask(SMnode* pMnode, SSubplan* plan, SStreamObj* pStream return TSDB_CODE_SUCCESS; } -static SStreamTask* buildAggTask(SStreamObj* pStream, SEpSet* pEpset, bool isFillhistory, bool useTriggerParam) { +static int32_t buildAggTask(SStreamObj* pStream, SEpSet* pEpset, bool isFillhistory, bool useTriggerParam, + SStreamTask** pAggTask) { + *pAggTask = NULL; + uint64_t uid = (isFillhistory) ? pStream->hTaskUid : pStream->uid; SArray** pTaskList = (isFillhistory) ? taosArrayGetLast(pStream->pHTasksList) : taosArrayGetLast(pStream->tasks); - SStreamTask* pAggTask = + int32_t code = tNewStreamTask(uid, TASK_LEVEL__AGG, pEpset, isFillhistory, useTriggerParam ? pStream->conf.triggerParam : 0, - *pTaskList, pStream->conf.fillHistory, pStream->subTableWithoutMd5); - if (pAggTask == NULL) { - terrno = TSDB_CODE_OUT_OF_MEMORY; - return NULL; - } - - return pAggTask; + *pTaskList, pStream->conf.fillHistory, pStream->subTableWithoutMd5, pAggTask); + return code; } static int32_t doAddAggTask(SStreamObj* pStream, SMnode* pMnode, SSubplan* plan, SEpSet* pEpset, SVgObj* pVgroup, SSnodeObj* pSnode, bool isFillhistory, bool useTriggerParam) { int32_t code = 0; - SStreamTask* pTask = buildAggTask(pStream, pEpset, isFillhistory, useTriggerParam); - if (pTask == NULL) { - return terrno; + SStreamTask* pTask = NULL; + + code = buildAggTask(pStream, pEpset, isFillhistory, useTriggerParam, &pTask); + if (code != TSDB_CODE_SUCCESS) { + return code; } + if (pSnode != NULL) { code = mndAssignStreamTaskToSnode(pMnode, pTask, plan, pSnode); mDebug("doAddAggTask taskId:%s, snode id:%d, isFillHistory:%d", pTask->id.idStr, pSnode->id, isFillhistory); diff --git a/source/dnode/snode/src/snode.c b/source/dnode/snode/src/snode.c index cfa24b2430..da60d7d30e 100644 --- a/source/dnode/snode/src/snode.c +++ b/source/dnode/snode/src/snode.c @@ -41,7 +41,7 @@ int32_t sndBuildStreamTask(SSnode *pSnode, SStreamTask *pTask, int64_t nextProce SCheckpointInfo *pChkInfo = &pTask->chkInfo; tqSetRestoreVersionInfo(pTask); - char *p = streamTaskGetStatus(pTask)->name; + char *p = streamTaskGetStatus(pTask).name; if (pTask->info.fillHistory) { sndInfo("vgId:%d build stream task, s-task:%s, checkpointId:%" PRId64 " checkpointVer:%" PRId64 " nextProcessVer:%" PRId64 @@ -71,9 +71,9 @@ SSnode *sndOpen(const char *path, const SSnodeOpt *pOption) { startRsync(); pSnode->msgCb = pOption->msgCb; - pSnode->pMeta = streamMetaOpen(path, pSnode, (FTaskBuild *)sndBuildStreamTask, tqExpandStreamTask, SNODE_HANDLE, taosGetTimestampMs(), tqStartTaskCompleteCallback); - if (pSnode->pMeta == NULL) { - terrno = TSDB_CODE_OUT_OF_MEMORY; + int32_t code = streamMetaOpen(path, pSnode, (FTaskBuild *)sndBuildStreamTask, tqExpandStreamTask, SNODE_HANDLE, + taosGetTimestampMs(), tqStartTaskCompleteCallback, &pSnode->pMeta); + if (code != TSDB_CODE_SUCCESS) { goto FAIL; } diff --git a/source/dnode/vnode/src/sma/smaRollup.c b/source/dnode/vnode/src/sma/smaRollup.c index f68d83350b..5ab2a57954 100644 --- a/source/dnode/vnode/src/sma/smaRollup.c +++ b/source/dnode/vnode/src/sma/smaRollup.c @@ -297,8 +297,17 @@ static int32_t tdSetRSmaInfoItemParams(SSma *pSma, SRSmaParam *param, SRSmaStat sprintf(pStreamTask->exec.qmsg, "%s", RSMA_EXEC_TASK_FLAG); pStreamTask->chkInfo.checkpointId = streamMetaGetLatestCheckpointId(pStreamTask->pMeta); tdRSmaTaskInit(pStreamTask->pMeta, pItem, &pStreamTask->id); - pStreamTask->status.pSM = streamCreateStateMachine(pStreamTask); - pStreamTask->chkInfo.pActiveInfo = streamTaskCreateActiveChkptInfo(); + + int32_t code = streamCreateStateMachine(pStreamTask); + if (code != TSDB_CODE_SUCCESS) { + return code; + } + + code = streamTaskCreateActiveChkptInfo(&pStreamTask->chkInfo.pActiveInfo); + if (code != TSDB_CODE_SUCCESS) { + return code; + } + pStreamState = streamStateOpen(taskInfDir, pStreamTask, pStreamTask->id.streamId, pStreamTask->id.taskId); if (!pStreamState) { terrno = TSDB_CODE_RSMA_STREAM_STATE_OPEN; diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 7daf807c27..68de7018ab 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -90,10 +90,11 @@ STQ* tqOpen(const char* path, SVnode* pVnode) { int32_t tqInitialize(STQ* pTq) { int32_t vgId = TD_VID(pTq->pVnode); - pTq->pStreamMeta = - streamMetaOpen(pTq->path, pTq, tqBuildStreamTask, tqExpandStreamTask, vgId, -1, tqStartTaskCompleteCallback); - if (pTq->pStreamMeta == NULL) { - return -1; + + int32_t code = streamMetaOpen(pTq->path, pTq, tqBuildStreamTask, tqExpandStreamTask, vgId, -1, + tqStartTaskCompleteCallback, &pTq->pStreamMeta); + if (code != TSDB_CODE_SUCCESS) { + return code; } streamMetaLoadAllTasks(pTq->pStreamMeta); @@ -767,7 +768,7 @@ int32_t tqBuildStreamTask(void* pTqObj, SStreamTask* pTask, int64_t nextProcessV SCheckpointInfo* pChkInfo = &pTask->chkInfo; tqSetRestoreVersionInfo(pTask); - char* p = streamTaskGetStatus(pTask)->name; + char* p = streamTaskGetStatus(pTask).name; const char* pNext = streamTaskGetStatusStr(pTask->status.taskStatus); if (pTask->info.fillHistory) { @@ -856,7 +857,8 @@ int32_t handleStep2Async(SStreamTask* pStreamTask, void* param) { SStreamMeta* pMeta = pStreamTask->pMeta; STaskId hId = pStreamTask->hTaskInfo.id; - SStreamTask* pTask = streamMetaAcquireTask(pStreamTask->pMeta, hId.streamId, hId.taskId); + SStreamTask* pTask = NULL; + int32_t code = streamMetaAcquireTask(pStreamTask->pMeta, hId.streamId, hId.taskId, &pTask); if (pTask == NULL) { tqWarn("s-task:0x%x failed to acquired it to exec step 2, scan wal quit", (int32_t) hId.taskId); return TSDB_CODE_SUCCESS; @@ -874,7 +876,8 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { SStreamMeta* pMeta = pTq->pStreamMeta; int32_t code = TSDB_CODE_SUCCESS; - SStreamTask* pTask = streamMetaAcquireTask(pMeta, pReq->streamId, pReq->taskId); + SStreamTask* pTask = NULL; + code = streamMetaAcquireTask(pMeta, pReq->streamId, pReq->taskId, &pTask); if (pTask == NULL) { tqError("vgId:%d failed to acquire stream task:0x%x during scan history data, task may have been destroyed", pMeta->vgId, pReq->taskId); @@ -883,7 +886,7 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { // do recovery step1 const char* id = pTask->id.idStr; - char* pStatus = streamTaskGetStatus(pTask)->name; + char* pStatus = streamTaskGetStatus(pTask).name; // avoid multi-thread exec while (1) { @@ -939,15 +942,15 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { if (retInfo.ret == TASK_SCANHISTORY_REXEC) { streamExecScanHistoryInFuture(pTask, retInfo.idleTime); } else { - SStreamTaskState* p = streamTaskGetStatus(pTask); - ETaskStatus s = p->state; + SStreamTaskState p = streamTaskGetStatus(pTask); + ETaskStatus s = p.state; if (s == TASK_STATUS__PAUSE) { tqDebug("s-task:%s is paused in the step1, elapsed time:%.2fs total:%.2fs, sched-status:%d", pTask->id.idStr, el, pTask->execInfo.step1El, status); } else if (s == TASK_STATUS__STOP || s == TASK_STATUS__DROPPING) { tqDebug("s-task:%s status:%p not continue scan-history data, total elapsed time:%.2fs quit", pTask->id.idStr, - p->name, pTask->execInfo.step1El); + p.name, pTask->execInfo.step1El); } } @@ -961,7 +964,8 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { ASSERT(pTask->info.fillHistory == 1); // 1. get the related stream task - SStreamTask* pStreamTask = streamMetaAcquireTask(pMeta, pTask->streamTaskId.streamId, pTask->streamTaskId.taskId); + SStreamTask* pStreamTask = NULL; + code = streamMetaAcquireTask(pMeta, pTask->streamTaskId.streamId, pTask->streamTaskId.taskId, &pStreamTask); if (pStreamTask == NULL) { tqError("failed to find s-task:0x%" PRIx64 ", it may have been destroyed, drop related fill-history task:%s", pTask->streamTaskId.taskId, pTask->id.idStr); @@ -1121,7 +1125,8 @@ int32_t tqProcessTaskCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg* pRsp) return TSDB_CODE_SUCCESS; } - SStreamTask* pTask = streamMetaAcquireTask(pMeta, req.streamId, req.taskId); + SStreamTask* pTask = NULL; + code = streamMetaAcquireTask(pMeta, req.streamId, req.taskId, &pTask); if (pTask == NULL) { tqError("vgId:%d failed to find s-task:0x%x, ignore checkpoint msg. checkpointId:%" PRId64 " transId:%d it may have been destroyed", @@ -1147,7 +1152,7 @@ int32_t tqProcessTaskCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg* pRsp) // todo save the checkpoint failed info taosThreadMutexLock(&pTask->lock); - ETaskStatus status = streamTaskGetStatus(pTask)->state; + ETaskStatus status = streamTaskGetStatus(pTask).state; if (req.mndTrigger == 1) { if (status == TASK_STATUS__HALT || status == TASK_STATUS__PAUSE) { diff --git a/source/dnode/vnode/src/tq/tqRead.c b/source/dnode/vnode/src/tq/tqRead.c index d3be8fa666..b57af518b4 100644 --- a/source/dnode/vnode/src/tq/tqRead.c +++ b/source/dnode/vnode/src/tq/tqRead.c @@ -341,10 +341,8 @@ int32_t extractMsgFromWal(SWalReader* pReader, void** pItem, int64_t maxVer, con memcpy(data, pBody, len); SPackedData data1 = (SPackedData){.ver = ver, .msgLen = len, .msgStr = data}; - *pItem = (SStreamQueueItem*)streamDataSubmitNew(&data1, STREAM_INPUT__DATA_SUBMIT); - if (*pItem == NULL) { - code = TSDB_CODE_OUT_OF_MEMORY; - terrno = code; + code = streamDataSubmitNew(&data1, STREAM_INPUT__DATA_SUBMIT, (SStreamDataSubmit**)pItem); + if (code != 0) { tqError("%s failed to create data submit for stream since out of memory", id); return code; } diff --git a/source/dnode/vnode/src/tq/tqStreamTask.c b/source/dnode/vnode/src/tq/tqStreamTask.c index bcf17bf1e1..b4829d1dd8 100644 --- a/source/dnode/vnode/src/tq/tqStreamTask.c +++ b/source/dnode/vnode/src/tq/tqStreamTask.c @@ -226,15 +226,15 @@ bool taskReadyForDataFromWal(SStreamTask* pTask) { } // not in ready state, do not handle the data from wal - SStreamTaskState* pState = streamTaskGetStatus(pTask); - if (pState->state != TASK_STATUS__READY) { - tqTrace("s-task:%s not ready for submit block in wal, status:%s", pTask->id.idStr, pState->name); + SStreamTaskState pState = streamTaskGetStatus(pTask); + if (pState.state != TASK_STATUS__READY) { + tqTrace("s-task:%s not ready for submit block in wal, status:%s", pTask->id.idStr, pState.name); return false; } // fill-history task has entered into the last phase, no need to anything if ((pTask->info.fillHistory == 1) && pTask->status.appendTranstateBlock) { - ASSERT(pState->state == TASK_STATUS__READY); + ASSERT(pState.state == TASK_STATUS__READY); // the maximum version of data in the WAL has reached already, the step2 is done tqDebug("s-task:%s fill-history reach the maximum ver:%" PRId64 ", not scan wal anymore", pTask->id.idStr, pTask->dataRange.range.maxVer); @@ -324,7 +324,8 @@ int32_t doScanWalForAllTasks(SStreamMeta* pStreamMeta, bool* pScanIdle) { for (int32_t i = 0; i < numOfTasks; ++i) { STaskId* pTaskId = taosArrayGet(pTaskList, i); - SStreamTask* pTask = streamMetaAcquireTask(pStreamMeta, pTaskId->streamId, pTaskId->taskId); + SStreamTask* pTask = NULL; + int32_t code = streamMetaAcquireTask(pStreamMeta, pTaskId->streamId, pTaskId->taskId, &pTask); if (pTask == NULL) { continue; } @@ -337,7 +338,7 @@ int32_t doScanWalForAllTasks(SStreamMeta* pStreamMeta, bool* pScanIdle) { *pScanIdle = false; // seek the stored version and extract data from WAL - int32_t code = setWalReaderStartOffset(pTask, vgId); + code = setWalReaderStartOffset(pTask, vgId); if (code != TSDB_CODE_SUCCESS) { streamMetaReleaseTask(pStreamMeta, pTask); continue; @@ -348,9 +349,9 @@ int32_t doScanWalForAllTasks(SStreamMeta* pStreamMeta, bool* pScanIdle) { taosThreadMutexLock(&pTask->lock); - SStreamTaskState* pState = streamTaskGetStatus(pTask); - if (pState->state != TASK_STATUS__READY) { - tqDebug("s-task:%s not ready for submit block from wal, status:%s", pTask->id.idStr, pState->name); + SStreamTaskState pState = streamTaskGetStatus(pTask); + if (pState.state != TASK_STATUS__READY) { + tqDebug("s-task:%s not ready for submit block from wal, status:%s", pTask->id.idStr, pState.name); taosThreadMutexUnlock(&pTask->lock); streamMetaReleaseTask(pStreamMeta, pTask); continue; diff --git a/source/dnode/vnode/src/tqCommon/tqCommon.c b/source/dnode/vnode/src/tqCommon/tqCommon.c index 1f3c049211..3c6100a8f4 100644 --- a/source/dnode/vnode/src/tqCommon/tqCommon.c +++ b/source/dnode/vnode/src/tqCommon/tqCommon.c @@ -140,17 +140,19 @@ int32_t tqStreamTaskRestoreCheckpoint(SStreamMeta* pMeta, int64_t streamId, int3 } tqDebug("vgId:%d restore task:0x%" PRIx64 "-0x%x checkpointId", vgId, streamId, taskId); - SStreamTask* pTask = streamMetaAcquireTask(pMeta, streamId, taskId); + SStreamTask* pTask = NULL; + int32_t code = streamMetaAcquireTask(pMeta, streamId, taskId, &pTask); if (pTask == NULL) { tqError("failed to acquire task:0x%x when trying to restore checkpointId", taskId); return TSDB_CODE_STREAM_TASK_NOT_EXIST; } - int32_t code = streamTaskSendRestoreChkptMsg(pTask); + code = streamTaskSendRestoreChkptMsg(pTask); streamMetaReleaseTask(pMeta, pTask); return code; } +// this is to process request from transaction, always return true. int32_t tqStreamTaskProcessUpdateReq(SStreamMeta* pMeta, SMsgCb* cb, SRpcMsg* pMsg, bool restored) { int32_t vgId = pMeta->vgId; char* msg = POINTER_SHIFT(pMsg->pCont, sizeof(SMsgHead)); @@ -230,7 +232,7 @@ int32_t tqStreamTaskProcessUpdateReq(SStreamMeta* pMeta, SMsgCb* cb, SRpcMsg* pM updated = streamTaskUpdateEpsetInfo(pTask, req.pNodeList); // send the checkpoint-source-rsp for source task to end the checkpoint trans in mnode - streamTaskSendPreparedCheckpointsourceRsp(pTask); + (void) streamTaskSendCheckpointsourceRsp(pTask); streamTaskResetStatus(pTask); streamTaskStopMonitorCheckRsp(&pTask->taskCheckInfo, pTask->id.idStr); @@ -329,7 +331,8 @@ int32_t tqStreamTaskProcessDispatchReq(SStreamMeta* pMeta, SRpcMsg* pMsg) { tqDebug("s-task:0x%x recv dispatch msg from 0x%x(vgId:%d)", req.taskId, req.upstreamTaskId, req.upstreamNodeId); - SStreamTask* pTask = streamMetaAcquireTask(pMeta, req.streamId, req.taskId); + SStreamTask* pTask = NULL; + int32_t code = streamMetaAcquireTask(pMeta, req.streamId, req.taskId, &pTask); if (pTask) { SRpcMsg rsp = {.info = pMsg->info, .code = 0}; if (streamProcessDispatchMsg(pTask, &req, &rsp) != 0) { @@ -388,7 +391,8 @@ int32_t tqStreamTaskProcessDispatchRsp(SStreamMeta* pMeta, SRpcMsg* pMsg) { tqDebug("s-task:0x%x vgId:%d recv dispatch-rsp from 0x%x vgId:%d", pRsp->upstreamTaskId, pRsp->upstreamNodeId, pRsp->downstreamTaskId, pRsp->downstreamNodeId); - SStreamTask* pTask = streamMetaAcquireTask(pMeta, pRsp->streamId, pRsp->upstreamTaskId); + SStreamTask* pTask = NULL; + int32_t code = streamMetaAcquireTask(pMeta, pRsp->streamId, pRsp->upstreamTaskId, &pTask); if (pTask) { streamProcessDispatchRsp(pTask, pRsp, pMsg->code); streamMetaReleaseTask(pMeta, pTask); @@ -411,7 +415,8 @@ int32_t tqStreamTaskProcessRetrieveReq(SStreamMeta* pMeta, SRpcMsg* pMsg) { tDecodeStreamRetrieveReq(&decoder, &req); tDecoderClear(&decoder); - SStreamTask* pTask = streamMetaAcquireTask(pMeta, req.streamId, req.dstTaskId); + SStreamTask* pTask = NULL; + int32_t code = streamMetaAcquireTask(pMeta, req.streamId, req.dstTaskId, &pTask); if (pTask == NULL) { tqError("vgId:%d process retrieve req, failed to acquire task:0x%x, it may have been dropped already", pMeta->vgId, req.dstTaskId); @@ -419,7 +424,6 @@ int32_t tqStreamTaskProcessRetrieveReq(SStreamMeta* pMeta, SRpcMsg* pMsg) { return -1; } - int32_t code = 0; if (pTask->info.taskLevel == TASK_LEVEL__SOURCE) { code = streamProcessRetrieveReq(pTask, &req); } else { @@ -484,7 +488,8 @@ int32_t tqStreamTaskProcessCheckRsp(SStreamMeta* pMeta, SRpcMsg* pMsg, bool isLe return streamMetaAddFailedTask(pMeta, rsp.streamId, rsp.upstreamTaskId); } - SStreamTask* pTask = streamMetaAcquireTask(pMeta, rsp.streamId, rsp.upstreamTaskId); + SStreamTask* pTask = NULL; + code = streamMetaAcquireTask(pMeta, rsp.streamId, rsp.upstreamTaskId, &pTask); if (pTask == NULL) { return streamMetaAddFailedTask(pMeta, rsp.streamId, rsp.upstreamTaskId); } @@ -511,7 +516,8 @@ int32_t tqStreamTaskProcessCheckpointReadyMsg(SStreamMeta* pMeta, SRpcMsg* pMsg) } tDecoderClear(&decoder); - SStreamTask* pTask = streamMetaAcquireTask(pMeta, req.streamId, req.upstreamTaskId); + SStreamTask* pTask = NULL; + code = streamMetaAcquireTask(pMeta, req.streamId, req.upstreamTaskId, &pTask); if (pTask == NULL) { tqError("vgId:%d failed to find s-task:0x%x, it may have been destroyed already", vgId, req.downstreamTaskId); return TSDB_CODE_STREAM_TASK_NOT_EXIST; @@ -598,7 +604,8 @@ int32_t tqStreamTaskProcessDeployReq(SStreamMeta* pMeta, SMsgCb* cb, int64_t sve tqDebug("vgId:%d s-task:0x%x is deployed and add into meta, numOfTasks:%d", vgId, taskId, numOfTasks); if (restored) { - SStreamTask* p = streamMetaAcquireTask(pMeta, streamId, taskId); + SStreamTask* p = NULL; + code = streamMetaAcquireTask(pMeta, streamId, taskId, &p); if ((p != NULL) && (p->info.fillHistory == 0)) { tqStreamStartOneTaskAsync(pMeta, cb, streamId, taskId); } @@ -773,7 +780,8 @@ int32_t tqStreamTaskProcessRunReq(SStreamMeta* pMeta, SRpcMsg* pMsg, bool isLead int32_t code = streamMetaAddFailedTask(pMeta, pReq->streamId, pReq->taskId); return code; } else if (type == STREAM_EXEC_T_RESUME_TASK) { // task resume to run after idle for a while - SStreamTask* pTask = streamMetaAcquireTask(pMeta, pReq->streamId, pReq->taskId); + SStreamTask* pTask = NULL; + int32_t code = streamMetaAcquireTask(pMeta, pReq->streamId, pReq->taskId, &pTask); if (pTask != NULL) { char* pStatus = NULL; @@ -794,7 +802,8 @@ int32_t tqStreamTaskProcessRunReq(SStreamMeta* pMeta, SRpcMsg* pMsg, bool isLead return 0; } - SStreamTask* pTask = streamMetaAcquireTask(pMeta, pReq->streamId, pReq->taskId); + SStreamTask* pTask = NULL; + int32_t code = streamMetaAcquireTask(pMeta, pReq->streamId, pReq->taskId, &pTask); if (pTask != NULL) { // even in halt status, the data in inputQ must be processed char* p = NULL; if (streamTaskReadyToRun(pTask, &p)) { @@ -862,7 +871,8 @@ int32_t tqStartTaskCompleteCallback(SStreamMeta* pMeta) { int32_t tqStreamTaskProcessTaskResetReq(SStreamMeta* pMeta, char* pMsg) { SVPauseStreamTaskReq* pReq = (SVPauseStreamTaskReq*)pMsg; - SStreamTask* pTask = streamMetaAcquireTask(pMeta, pReq->streamId, pReq->taskId); + SStreamTask* pTask = NULL; + int32_t code = streamMetaAcquireTask(pMeta, pReq->streamId, pReq->taskId, &pTask); if (pTask == NULL) { tqError("vgId:%d process task-reset req, failed to acquire task:0x%x, it may have been dropped already", pMeta->vgId, pReq->taskId); @@ -875,8 +885,8 @@ int32_t tqStreamTaskProcessTaskResetReq(SStreamMeta* pMeta, char* pMsg) { streamTaskClearCheckInfo(pTask, true); // clear flag set during do checkpoint, and open inputQ for all upstream tasks - SStreamTaskState* pState = streamTaskGetStatus(pTask); - if (pState->state == TASK_STATUS__CK) { + SStreamTaskState pState = streamTaskGetStatus(pTask); + if (pState.state == TASK_STATUS__CK) { int32_t tranId = 0; int64_t activeChkId = 0; streamTaskGetActiveCheckpointInfo(pTask, &tranId, &activeChkId); @@ -885,13 +895,13 @@ int32_t tqStreamTaskProcessTaskResetReq(SStreamMeta* pMeta, char* pMsg) { pTask->id.idStr, activeChkId, tranId); streamTaskSetStatusReady(pTask); - } else if (pState->state == TASK_STATUS__UNINIT) { + } else if (pState.state == TASK_STATUS__UNINIT) { // tqDebug("s-task:%s start task by checking downstream tasks", pTask->id.idStr); // ASSERT(pTask->status.downstreamReady == 0); // tqStreamTaskRestoreCheckpoint(pMeta, pTask->id.streamId, pTask->id.taskId); - tqDebug("s-task:%s status:%s do nothing after receiving reset-task from mnode", pTask->id.idStr, pState->name); + tqDebug("s-task:%s status:%s do nothing after receiving reset-task from mnode", pTask->id.idStr, pState.name); } else { - tqDebug("s-task:%s status:%s do nothing after receiving reset-task from mnode", pTask->id.idStr, pState->name); + tqDebug("s-task:%s status:%s do nothing after receiving reset-task from mnode", pTask->id.idStr, pState.name); } taosThreadMutexUnlock(&pTask->lock); @@ -903,7 +913,8 @@ int32_t tqStreamTaskProcessTaskResetReq(SStreamMeta* pMeta, char* pMsg) { int32_t tqStreamTaskProcessRetrieveTriggerReq(SStreamMeta* pMeta, SRpcMsg* pMsg) { SRetrieveChkptTriggerReq* pReq = (SRetrieveChkptTriggerReq*)pMsg->pCont; - SStreamTask* pTask = streamMetaAcquireTask(pMeta, pReq->streamId, pReq->upstreamTaskId); + SStreamTask* pTask = NULL; + int32_t code = streamMetaAcquireTask(pMeta, pReq->streamId, pReq->upstreamTaskId, &pTask); if (pTask == NULL) { tqError("vgId:%d process retrieve checkpoint trigger, checkpointId:%" PRId64 " from s-task:0x%x, failed to acquire task:0x%x, it may have been dropped already", @@ -925,8 +936,8 @@ int32_t tqStreamTaskProcessRetrieveTriggerReq(SStreamMeta* pMeta, SRpcMsg* pMsg) return TSDB_CODE_SUCCESS; } - SStreamTaskState* pState = streamTaskGetStatus(pTask); - if (pState->state == TASK_STATUS__CK) { // recv the checkpoint-source/trigger already + SStreamTaskState pState = streamTaskGetStatus(pTask); + if (pState.state == TASK_STATUS__CK) { // recv the checkpoint-source/trigger already int32_t transId = 0; int64_t checkpointId = 0; @@ -955,7 +966,7 @@ int32_t tqStreamTaskProcessRetrieveTriggerReq(SStreamMeta* pMeta, SRpcMsg* pMsg) TSDB_CODE_ACTION_IN_PROGRESS); } } else { // upstream not recv the checkpoint-source/trigger till now - ASSERT(pState->state == TASK_STATUS__READY || pState->state == TASK_STATUS__HALT); + ASSERT(pState.state == TASK_STATUS__READY || pState.state == TASK_STATUS__HALT); tqWarn( "s-task:%s not recv checkpoint-source from mnode or checkpoint-trigger from upstream yet, wait for all " "upstream sending checkpoint-source/trigger", @@ -971,7 +982,8 @@ int32_t tqStreamTaskProcessRetrieveTriggerReq(SStreamMeta* pMeta, SRpcMsg* pMsg) int32_t tqStreamTaskProcessRetrieveTriggerRsp(SStreamMeta* pMeta, SRpcMsg* pMsg) { SCheckpointTriggerRsp* pRsp = POINTER_SHIFT(pMsg->pCont, sizeof(SMsgHead)); - SStreamTask* pTask = streamMetaAcquireTask(pMeta, pRsp->streamId, pRsp->taskId); + SStreamTask* pTask = NULL; + int32_t code = streamMetaAcquireTask(pMeta, pRsp->streamId, pRsp->taskId, &pTask); if (pTask == NULL) { tqError( "vgId:%d process retrieve checkpoint-trigger, failed to acquire task:0x%x, it may have been dropped already", @@ -990,7 +1002,8 @@ int32_t tqStreamTaskProcessRetrieveTriggerRsp(SStreamMeta* pMeta, SRpcMsg* pMsg) int32_t tqStreamTaskProcessTaskPauseReq(SStreamMeta* pMeta, char* pMsg) { SVPauseStreamTaskReq* pReq = (SVPauseStreamTaskReq*)pMsg; - SStreamTask* pTask = streamMetaAcquireTask(pMeta, pReq->streamId, pReq->taskId); + SStreamTask* pTask = NULL; + int32_t code = streamMetaAcquireTask(pMeta, pReq->streamId, pReq->taskId, &pTask); if (pTask == NULL) { tqError("vgId:%d process pause req, failed to acquire task:0x%x, it may have been dropped already", pMeta->vgId, pReq->taskId); @@ -1003,7 +1016,8 @@ int32_t tqStreamTaskProcessTaskPauseReq(SStreamMeta* pMeta, char* pMsg) { SStreamTask* pHistoryTask = NULL; if (HAS_RELATED_FILLHISTORY_TASK(pTask)) { - pHistoryTask = streamMetaAcquireTask(pMeta, pTask->hTaskInfo.id.streamId, pTask->hTaskInfo.id.taskId); + pHistoryTask = NULL; + code = streamMetaAcquireTask(pMeta, pTask->hTaskInfo.id.streamId, pTask->hTaskInfo.id.taskId, &pHistoryTask); if (pHistoryTask == NULL) { tqError("vgId:%d process pause req, failed to acquire fill-history task:0x%" PRIx64 ", it may have been dropped already", @@ -1033,7 +1047,7 @@ static int32_t tqProcessTaskResumeImpl(void* handle, SStreamTask* pTask, int64_t } streamTaskResume(pTask); - ETaskStatus status = streamTaskGetStatus(pTask)->state; + ETaskStatus status = streamTaskGetStatus(pTask).state; int32_t level = pTask->info.taskLevel; if (status == TASK_STATUS__READY || status == TASK_STATUS__SCAN_HISTORY || status == TASK_STATUS__CK) { @@ -1070,28 +1084,30 @@ int32_t tqStreamTaskProcessTaskResumeReq(void* handle, int64_t sversion, char* m SStreamMeta* pMeta = fromVnode ? ((STQ*)handle)->pStreamMeta : handle; - SStreamTask* pTask = streamMetaAcquireTask(pMeta, pReq->streamId, pReq->taskId); + SStreamTask* pTask = NULL; + int32_t code = streamMetaAcquireTask(pMeta, pReq->streamId, pReq->taskId, &pTask); if (pTask == NULL) { tqError("s-task:0x%x failed to acquire task to resume, it may have been dropped or stopped", pReq->taskId); return TSDB_CODE_STREAM_TASK_IVLD_STATUS; } taosThreadMutexLock(&pTask->lock); - SStreamTaskState* pState = streamTaskGetStatus(pTask); - tqDebug("s-task:%s start to resume from paused, current status:%s", pTask->id.idStr, pState->name); + SStreamTaskState pState = streamTaskGetStatus(pTask); + tqDebug("s-task:%s start to resume from paused, current status:%s", pTask->id.idStr, pState.name); taosThreadMutexUnlock(&pTask->lock); - int32_t code = tqProcessTaskResumeImpl(handle, pTask, sversion, pReq->igUntreated, fromVnode); + code = tqProcessTaskResumeImpl(handle, pTask, sversion, pReq->igUntreated, fromVnode); if (code != 0) { return code; } STaskId* pHTaskId = &pTask->hTaskInfo.id; - SStreamTask* pHTask = streamMetaAcquireTask(pMeta, pHTaskId->streamId, pHTaskId->taskId); + SStreamTask* pHTask = NULL; + code = streamMetaAcquireTask(pMeta, pHTaskId->streamId, pHTaskId->taskId, &pHTask); if (pHTask) { taosThreadMutexLock(&pHTask->lock); - SStreamTaskState* p = streamTaskGetStatus(pHTask); - tqDebug("s-task:%s related history task start to resume from paused, current status:%s", pHTask->id.idStr, p->name); + SStreamTaskState p = streamTaskGetStatus(pHTask); + tqDebug("s-task:%s related history task start to resume from paused, current status:%s", pHTask->id.idStr, p.name); taosThreadMutexUnlock(&pHTask->lock); code = tqProcessTaskResumeImpl(handle, pHTask, sversion, pReq->igUntreated, fromVnode); @@ -1121,7 +1137,8 @@ int32_t tqStreamProcessConsensusChkptRsp2(SStreamMeta* pMeta, SRpcMsg* pMsg) { r int32_t tqStreamProcessCheckpointReadyRsp(SStreamMeta* pMeta, SRpcMsg* pMsg) { SMStreamCheckpointReadyRspMsg* pRsp = pMsg->pCont; - SStreamTask* pTask = streamMetaAcquireTask(pMeta, pRsp->streamId, pRsp->downstreamTaskId); + SStreamTask* pTask = NULL; + int32_t code = streamMetaAcquireTask(pMeta, pRsp->streamId, pRsp->downstreamTaskId, &pTask); if (pTask == NULL) { tqError("vgId:%d failed to acquire task:0x%x when handling checkpoint-ready msg, it may have been dropped", pRsp->downstreamNodeId, pRsp->downstreamTaskId); @@ -1154,7 +1171,8 @@ int32_t tqStreamTaskProcessConsenChkptIdReq(SStreamMeta* pMeta, SRpcMsg* pMsg) { tDecoderClear(&decoder); - SStreamTask* pTask = streamMetaAcquireTask(pMeta, req.streamId, req.taskId); + SStreamTask* pTask = NULL; + code = streamMetaAcquireTask(pMeta, req.streamId, req.taskId, &pTask); if (pTask == NULL) { tqError("vgId:%d process set consensus checkpointId req, failed to acquire task:0x%x, it may have been dropped already", pMeta->vgId, req.taskId); diff --git a/source/dnode/vnode/src/tsdb/tsdbRead2.c b/source/dnode/vnode/src/tsdb/tsdbRead2.c index cf2d23cfc8..2474666e93 100644 --- a/source/dnode/vnode/src/tsdb/tsdbRead2.c +++ b/source/dnode/vnode/src/tsdb/tsdbRead2.c @@ -48,7 +48,7 @@ typedef struct { static int32_t getCurrentBlockInfo(SDataBlockIter* pBlockIter, SFileDataBlockInfo** pInfo); static int32_t buildDataBlockFromBufImpl(STableBlockScanInfo* pBlockScanInfo, int64_t endKey, int32_t capacity, STsdbReader* pReader); -static TSDBROW* getValidMemRow(SIterInfo* pIter, const SArray* pDelList, STsdbReader* pReader); +static int32_t getValidMemRow(SIterInfo* pIter, const SArray* pDelList, STsdbReader* pReader, TSDBROW** pRow); static int32_t doMergeRowsInFileBlocks(SBlockData* pBlockData, STableBlockScanInfo* pScanInfo, SRowKey* pKey, STsdbReader* pReader); static int32_t doMergeRowsInSttBlock(SSttBlockReader* pSttBlockReader, STableBlockScanInfo* pScanInfo, @@ -296,8 +296,11 @@ static int32_t filesetIteratorNext(SFilesetIter* pIter, STsdbReader* pReader, bo pIter->pSttBlockReader->uid = 0; tMergeTreeClose(&pIter->pSttBlockReader->mergeTree); destroySttBlockReader(pReader->status.pLDataIterArray, &pCost->sttCost); - pReader->status.pLDataIterArray = NULL; + pReader->status.pLDataIterArray = taosArrayInit(4, POINTER_BYTES); + if (pReader->status.pLDataIterArray == NULL) { + return TSDB_CODE_OUT_OF_MEMORY; + } // check file the time range of coverage STimeWindow win = {0}; @@ -708,6 +711,10 @@ static int32_t loadFileBlockBrinInfo(STsdbReader* pReader, SArray* pIndexList, S while (1) { int32_t code = getNextBrinRecord(&iter, &pRecord); if (code != TSDB_CODE_SUCCESS) { + return code; + } + + if (pRecord == NULL) { break; } @@ -2025,9 +2032,11 @@ static int32_t doMergeMultiLevelRows(STsdbReader* pReader, STableBlockScanInfo* SFileBlockDumpInfo* pDumpInfo = &pReader->status.fBlockDumpInfo; SArray* pDelList = pBlockScanInfo->delSkyline; int32_t pkSrcSlot = pReader->suppInfo.pkSrcSlot; + TSDBROW* pRow = NULL; + TSDBROW* piRow = NULL; - TSDBROW* pRow = getValidMemRow(&pBlockScanInfo->iter, pDelList, pReader); - TSDBROW* piRow = getValidMemRow(&pBlockScanInfo->iiter, pDelList, pReader); + getValidMemRow(&pBlockScanInfo->iter, pDelList, pReader, &pRow); + getValidMemRow(&pBlockScanInfo->iiter, pDelList, pReader, &piRow); SRowKey* pSttKey = NULL; if (hasDataInSttBlock(pBlockScanInfo) && (!pBlockScanInfo->cleanSttBlocks)) { @@ -2201,9 +2210,10 @@ int32_t doInitMemDataIter(STsdbReader* pReader, STbData** pData, STableBlockScan static void doForwardDataIter(SRowKey* pKey, SIterInfo* pIter, STableBlockScanInfo* pBlockScanInfo, STsdbReader* pReader) { SRowKey rowKey = {0}; + TSDBROW* pRow = NULL; while (1) { - TSDBROW* pRow = getValidMemRow(pIter, pBlockScanInfo->delSkyline, pReader); + getValidMemRow(pIter, pBlockScanInfo->delSkyline, pReader, &pRow); if (!pIter->hasVal) { break; } @@ -2559,11 +2569,11 @@ static int32_t buildComposedDataBlockImpl(STsdbReader* pReader, STableBlockScanI } if (pBlockScanInfo->iter.hasVal) { - pRow = getValidMemRow(&pBlockScanInfo->iter, pBlockScanInfo->delSkyline, pReader); + getValidMemRow(&pBlockScanInfo->iter, pBlockScanInfo->delSkyline, pReader, &pRow); } if (pBlockScanInfo->iiter.hasVal) { - piRow = getValidMemRow(&pBlockScanInfo->iiter, pBlockScanInfo->delSkyline, pReader); + getValidMemRow(&pBlockScanInfo->iiter, pBlockScanInfo->delSkyline, pReader, &piRow); } // two levels of mem-table does contain the valid rows @@ -2810,13 +2820,16 @@ TSDBKEY getCurrentKeyInBuf(STableBlockScanInfo* pScanInfo, STsdbReader* pReader) TSDBKEY key = {.ts = TSKEY_INITIAL_VAL}, ikey = {.ts = TSKEY_INITIAL_VAL}; bool hasKey = false, hasIKey = false; - TSDBROW* pRow = getValidMemRow(&pScanInfo->iter, pScanInfo->delSkyline, pReader); + TSDBROW* pRow = NULL; + TSDBROW* pIRow = NULL; + + getValidMemRow(&pScanInfo->iter, pScanInfo->delSkyline, pReader, &pRow); if (pRow != NULL) { hasKey = true; key = TSDBROW_KEY(pRow); } - TSDBROW* pIRow = getValidMemRow(&pScanInfo->iiter, pScanInfo->delSkyline, pReader); + getValidMemRow(&pScanInfo->iiter, pScanInfo->delSkyline, pReader, &pIRow); if (pIRow != NULL) { hasIKey = true; ikey = TSDBROW_KEY(pIRow); @@ -3742,9 +3755,11 @@ bool hasBeenDropped(const SArray* pDelList, int32_t* index, int64_t key, int64_t return false; } -FORCE_INLINE TSDBROW* getValidMemRow(SIterInfo* pIter, const SArray* pDelList, STsdbReader* pReader) { +FORCE_INLINE int32_t getValidMemRow(SIterInfo* pIter, const SArray* pDelList, STsdbReader* pReader, TSDBROW** pRes) { + *pRes = NULL; + if (!pIter->hasVal) { - return NULL; + return TSDB_CODE_SUCCESS; } int32_t order = pReader->info.order; @@ -3754,18 +3769,20 @@ FORCE_INLINE TSDBROW* getValidMemRow(SIterInfo* pIter, const SArray* pDelList, S TSDBROW_INIT_KEY(pRow, key); if (outOfTimeWindow(key.ts, &pReader->info.window)) { pIter->hasVal = false; - return NULL; + return TSDB_CODE_SUCCESS; } // it is a valid data version if (key.version <= pReader->info.verRange.maxVer && key.version >= pReader->info.verRange.minVer) { if (pDelList == NULL || TARRAY_SIZE(pDelList) == 0) { - return pRow; + *pRes = pRow; + return TSDB_CODE_SUCCESS; } else { bool dropped = hasBeenDropped(pDelList, &pIter->index, key.ts, key.version, order, &pReader->info.verRange, pReader->suppInfo.numOfPks > 0); if (!dropped) { - return pRow; + *pRes = pRow; + return TSDB_CODE_SUCCESS; } } } @@ -3773,7 +3790,7 @@ FORCE_INLINE TSDBROW* getValidMemRow(SIterInfo* pIter, const SArray* pDelList, S while (1) { pIter->hasVal = tsdbTbDataIterNext(pIter->iter); if (!pIter->hasVal) { - return NULL; + return TSDB_CODE_SUCCESS; } pRow = tsdbTbDataIterGet(pIter->iter); @@ -3781,17 +3798,19 @@ FORCE_INLINE TSDBROW* getValidMemRow(SIterInfo* pIter, const SArray* pDelList, S TSDBROW_INIT_KEY(pRow, key); if (outOfTimeWindow(key.ts, &pReader->info.window)) { pIter->hasVal = false; - return NULL; + return TSDB_CODE_SUCCESS; } if (key.version <= pReader->info.verRange.maxVer && key.version >= pReader->info.verRange.minVer) { if (pDelList == NULL || TARRAY_SIZE(pDelList) == 0) { - return pRow; + *pRes = pRow; + return TSDB_CODE_SUCCESS; } else { bool dropped = hasBeenDropped(pDelList, &pIter->index, key.ts, key.version, order, &pReader->info.verRange, pReader->suppInfo.numOfPks > 0); if (!dropped) { - return pRow; + *pRes = pRow; + return TSDB_CODE_SUCCESS; } } } @@ -3809,7 +3828,8 @@ int32_t doMergeRowsInBuf(SIterInfo* pIter, uint64_t uid, SRowKey* pCurKey, SArra } // data exists but not valid - TSDBROW* pRow = getValidMemRow(pIter, pDelList, pReader); + TSDBROW* pRow = NULL; + getValidMemRow(pIter, pDelList, pReader, &pRow); if (pRow == NULL) { break; } @@ -3974,7 +3994,7 @@ int32_t doMergeMemTableMultiRows(TSDBROW* pRow, SRowKey* pKey, uint64_t uid, SIt *freeTSRow = false; return TSDB_CODE_SUCCESS; } else { // has next point in mem/imem - pNextRow = getValidMemRow(pIter, pDelList, pReader); + getValidMemRow(pIter, pDelList, pReader, &pNextRow); if (pNextRow == NULL) { *pResRow = current; *freeTSRow = false; @@ -4118,8 +4138,11 @@ int32_t doMergeMemIMemRows(TSDBROW* pRow, SRowKey* pRowKey, TSDBROW* piRow, SRow static int32_t tsdbGetNextRowInMem(STableBlockScanInfo* pBlockScanInfo, STsdbReader* pReader, TSDBROW* pResRow, int64_t endKey, bool* freeTSRow) { - TSDBROW* pRow = getValidMemRow(&pBlockScanInfo->iter, pBlockScanInfo->delSkyline, pReader); - TSDBROW* piRow = getValidMemRow(&pBlockScanInfo->iiter, pBlockScanInfo->delSkyline, pReader); + TSDBROW* pRow = NULL; + TSDBROW* piRow = NULL; + + getValidMemRow(&pBlockScanInfo->iter, pBlockScanInfo->delSkyline, pReader, &pRow); + getValidMemRow(&pBlockScanInfo->iiter, pBlockScanInfo->delSkyline, pReader, &piRow); SArray* pDelList = pBlockScanInfo->delSkyline; uint64_t uid = pBlockScanInfo->uid; diff --git a/source/dnode/vnode/src/tsdb/tsdbReadUtil.c b/source/dnode/vnode/src/tsdb/tsdbReadUtil.c index 84f235ab25..d7da6cbd9d 100644 --- a/source/dnode/vnode/src/tsdb/tsdbReadUtil.c +++ b/source/dnode/vnode/src/tsdb/tsdbReadUtil.c @@ -27,8 +27,12 @@ static bool overlapWithDelSkylineWithoutVer(STableBlockScanInfo* pBlockScanInfo, static int32_t initBlockScanInfoBuf(SBlockInfoBuf* pBuf, int32_t numOfTables) { int32_t num = numOfTables / pBuf->numPerBucket; int32_t remainder = numOfTables % pBuf->numPerBucket; + if (pBuf->pData == NULL) { pBuf->pData = taosArrayInit(num + 1, POINTER_BYTES); + if (pBuf->pData == NULL) { + return TSDB_CODE_OUT_OF_MEMORY; + } } for (int32_t i = 0; i < num; ++i) { @@ -37,7 +41,10 @@ static int32_t initBlockScanInfoBuf(SBlockInfoBuf* pBuf, int32_t numOfTables) { return TSDB_CODE_OUT_OF_MEMORY; } - taosArrayPush(pBuf->pData, &p); + void* px = taosArrayPush(pBuf->pData, &p); + if (px == NULL) { + return TSDB_CODE_OUT_OF_MEMORY; + } } if (remainder > 0) { @@ -45,7 +52,10 @@ static int32_t initBlockScanInfoBuf(SBlockInfoBuf* pBuf, int32_t numOfTables) { if (p == NULL) { return TSDB_CODE_OUT_OF_MEMORY; } - taosArrayPush(pBuf->pData, &p); + void* px = taosArrayPush(pBuf->pData, &p); + if (px == NULL) { + return TSDB_CODE_OUT_OF_MEMORY; + } } pBuf->numOfTables = numOfTables; @@ -86,7 +96,10 @@ int32_t ensureBlockScanInfoBuf(SBlockInfoBuf* pBuf, int32_t numOfTables) { return TSDB_CODE_OUT_OF_MEMORY; } - taosArrayPush(pBuf->pData, &p); + void* px = taosArrayPush(pBuf->pData, &p); + if (px == NULL) { + return TSDB_CODE_OUT_OF_MEMORY; + } } if (remainder > 0) { @@ -94,7 +107,10 @@ int32_t ensureBlockScanInfoBuf(SBlockInfoBuf* pBuf, int32_t numOfTables) { if (p == NULL) { return TSDB_CODE_OUT_OF_MEMORY; } - taosArrayPush(pBuf->pData, &p); + void* px = taosArrayPush(pBuf->pData, &p); + if (px == NULL) { + return TSDB_CODE_OUT_OF_MEMORY; + } } pBuf->numOfTables = numOfTables; @@ -214,7 +230,8 @@ void clearRowKey(SRowKey* pKey) { taosMemoryFreeClear(pKey->pks[0].pData); } -static void initLastProcKey(STableBlockScanInfo *pScanInfo, STsdbReader* pReader) { +static int32_t initLastProcKey(STableBlockScanInfo *pScanInfo, STsdbReader* pReader) { + int32_t code = 0; int32_t numOfPks = pReader->suppInfo.numOfPks; bool asc = ASCENDING_TRAVERSE(pReader->info.order); int8_t type = pReader->suppInfo.pk.type; @@ -225,18 +242,37 @@ static void initLastProcKey(STableBlockScanInfo *pScanInfo, STsdbReader* pReader int64_t skey = pReader->info.window.skey; int64_t ts = (skey > INT64_MIN) ? (skey - 1) : skey; - initRowKey(pRowKey, ts, numOfPks, type, bytes, asc); - initRowKey(&pScanInfo->sttKeyInfo.nextProcKey, skey, numOfPks, type, bytes, asc); + code = initRowKey(pRowKey, ts, numOfPks, type, bytes, asc); + if (code != TSDB_CODE_SUCCESS) { + return code; + } + + code = initRowKey(&pScanInfo->sttKeyInfo.nextProcKey, skey, numOfPks, type, bytes, asc); + if (code != TSDB_CODE_SUCCESS) { + return code; + } } else { int64_t ekey = pReader->info.window.ekey; int64_t ts = (ekey < INT64_MAX) ? (ekey + 1) : ekey; - initRowKey(pRowKey, ts, numOfPks, type, bytes, asc); - initRowKey(&pScanInfo->sttKeyInfo.nextProcKey, ekey, numOfPks, type, bytes, asc); + code = initRowKey(pRowKey, ts, numOfPks, type, bytes, asc); + if (code != TSDB_CODE_SUCCESS) { + return code; + } + + code = initRowKey(&pScanInfo->sttKeyInfo.nextProcKey, ekey, numOfPks, type, bytes, asc); + if (code != TSDB_CODE_SUCCESS) { + return code; + } } - initRowKey(&pScanInfo->sttRange.skey, INT64_MAX, numOfPks, type, bytes, asc); - initRowKey(&pScanInfo->sttRange.ekey, INT64_MIN, numOfPks, type, bytes, asc); + code = initRowKey(&pScanInfo->sttRange.skey, INT64_MAX, numOfPks, type, bytes, asc); + if (code != TSDB_CODE_SUCCESS) { + return code; + } + + code = initRowKey(&pScanInfo->sttRange.ekey, INT64_MIN, numOfPks, type, bytes, asc); + return code; } int32_t initTableBlockScanInfo(STableBlockScanInfo* pScanInfo, uint64_t uid, SSHashObj* pTableMap, @@ -248,10 +284,13 @@ int32_t initTableBlockScanInfo(STableBlockScanInfo* pScanInfo, uint64_t uid, SSH pScanInfo->cleanSttBlocks = false; pScanInfo->sttBlockReturned = false; - initLastProcKey(pScanInfo, pReader); + int32_t code = initLastProcKey(pScanInfo, pReader); + if (code != TSDB_CODE_SUCCESS) { + return code; + } pScanInfo->sttKeyInfo.status = STT_FILE_READER_UNINIT; - int32_t code = tSimpleHashPut(pTableMap, &pScanInfo->uid, sizeof(uint64_t), &pScanInfo, POINTER_BYTES); + code = tSimpleHashPut(pTableMap, &pScanInfo->uid, sizeof(uint64_t), &pScanInfo, POINTER_BYTES); if (code != TSDB_CODE_SUCCESS) { return code; } @@ -275,7 +314,10 @@ int32_t createDataBlockScanInfo(STsdbReader* pTsdbReader, SBlockInfoBuf* pBuf, c } int64_t st = taosGetTimestampUs(); - initBlockScanInfoBuf(pBuf, numOfTables); + code = initBlockScanInfoBuf(pBuf, numOfTables); + if (code != TSDB_CODE_SUCCESS) { + return code; + } pUidList->tableUidList = taosMemoryMalloc(numOfTables * sizeof(uint64_t)); if (pUidList->tableUidList == NULL) { @@ -419,29 +461,29 @@ int32_t getNextBrinRecord(SBrinRecordIter* pIter, SBrinRecord** pRecord) { if (pIter->blockIndex == -1 || (pIter->recordIndex + 1) >= pIter->block.numOfRecords) { pIter->blockIndex += 1; if (pIter->blockIndex >= taosArrayGetSize(pIter->pBrinBlockList)) { - return TSDB_CODE_FAILED; + return TSDB_CODE_SUCCESS; } pIter->pCurrentBlk = taosArrayGet(pIter->pBrinBlockList, pIter->blockIndex); - tBrinBlockClear(&pIter->block); + (void) tBrinBlockClear(&pIter->block); int32_t code = tsdbDataFileReadBrinBlock(pIter->pReader, pIter->pCurrentBlk, &pIter->block); if (code != TSDB_CODE_SUCCESS) { tsdbError("failed to read brinBlock from file, code:%s", tstrerror(code)); - return TSDB_CODE_FAILED; + return code; } pIter->recordIndex = -1; } pIter->recordIndex += 1; - tBrinBlockGet(&pIter->block, pIter->recordIndex, &pIter->record); + int32_t code = tBrinBlockGet(&pIter->block, pIter->recordIndex, &pIter->record); *pRecord = &pIter->record; - return TSDB_CODE_SUCCESS; + return code; } -void clearBrinBlockIter(SBrinRecordIter* pIter) { tBrinBlockDestroy(&pIter->block); } +void clearBrinBlockIter(SBrinRecordIter* pIter) { (void) tBrinBlockDestroy(&pIter->block); } // initialize the file block access order // sort the file blocks according to the offset of each data block in the files @@ -607,10 +649,17 @@ int32_t initBlockIterator(STsdbReader* pReader, SDataBlockIter* pBlockIter, int3 STableBlockScanInfo* pTableScanInfo = taosArrayGetP(pTableList, 0); for (int32_t i = 0; i < numOfBlocks; ++i) { STableDataBlockIdx tableDataBlockIdx = {.globalIndex = i}; - taosArrayPush(pTableScanInfo->pBlockIdxList, &tableDataBlockIdx); + void* px = taosArrayPush(pTableScanInfo->pBlockIdxList, &tableDataBlockIdx); + if (px == NULL) { + return TSDB_CODE_OUT_OF_MEMORY; + } + } + + void* p = taosArrayAddAll(pBlockIter->blockList, pTableScanInfo->pBlockList); + if (p == NULL) { + return TSDB_CODE_OUT_OF_MEMORY; } - taosArrayAddAll(pBlockIter->blockList, pTableScanInfo->pBlockList); taosArrayDestroy(pTableScanInfo->pBlockList); pTableScanInfo->pBlockList = NULL; @@ -640,11 +689,18 @@ int32_t initBlockIterator(STsdbReader* pReader, SDataBlockIter* pBlockIter, int3 int32_t index = sup.indexPerTable[pos]++; SFileDataBlockInfo* pBlockInfo = taosArrayGet(sup.pDataBlockInfo[pos][index].pInfo->pBlockList, index); - taosArrayPush(pBlockIter->blockList, pBlockInfo); + void* px = taosArrayPush(pBlockIter->blockList, pBlockInfo); + if (px == NULL) { + return TSDB_CODE_OUT_OF_MEMORY; + } STableBlockScanInfo* pTableScanInfo = sup.pDataBlockInfo[pos][index].pInfo; STableDataBlockIdx tableDataBlockIdx = {.globalIndex = numOfTotal}; - taosArrayPush(pTableScanInfo->pBlockIdxList, &tableDataBlockIdx); + + px = taosArrayPush(pTableScanInfo->pBlockIdxList, &tableDataBlockIdx); + if (px == NULL) { + return TSDB_CODE_OUT_OF_MEMORY; + } // set data block index overflow, in order to disable the offset comparator if (sup.indexPerTable[pos] >= sup.numOfBlocksPerTable[pos]) { @@ -752,7 +808,10 @@ static int32_t doCheckTombBlock(STombBlock* pBlock, STsdbReader* pReader, int32_ if (record.version <= pReader->info.verRange.maxVer) { SDelData delData = {.version = record.version, .sKey = record.skey, .eKey = record.ekey}; - taosArrayPush(pScanInfo->pFileDelData, &delData); + void* px = taosArrayPush(pScanInfo->pFileDelData, &delData); + if (px == NULL) { + return TSDB_CODE_OUT_OF_MEMORY; + } } } @@ -806,7 +865,7 @@ static int32_t doLoadTombDataFromTombBlk(const TTombBlkArray* pTombBlkArray, STs ETombBlkCheckEnum ret = 0; code = doCheckTombBlock(&block, pReader, numOfTables, &j, &ret); - tTombBlockDestroy(&block); + (void) tTombBlockDestroy(&block); if (code != TSDB_CODE_SUCCESS || ret == BLK_CHECK_QUIT) { return code; } @@ -858,7 +917,10 @@ int32_t loadMemTombData(SArray** ppMemDelData, STbData* pMemTbData, STbData* piM p = pMemTbData->pHead; while (p) { if (p->version <= ver) { - taosArrayPush(pMemDelData, p); + void* px = taosArrayPush(pMemDelData, p); + if (px == NULL) { + return TSDB_CODE_OUT_OF_MEMORY; + } } p = p->pNext; @@ -870,7 +932,10 @@ int32_t loadMemTombData(SArray** ppMemDelData, STbData* pMemTbData, STbData* piM p = piMemTbData->pHead; while (p) { if (p->version <= ver) { - taosArrayPush(pMemDelData, p); + void* px = taosArrayPush(pMemDelData, p); + if (px == NULL) { + return TSDB_CODE_OUT_OF_MEMORY; + } } p = p->pNext; } @@ -899,10 +964,13 @@ int32_t getNumOfRowsInSttBlock(SSttFileReader* pSttFileReader, SSttBlockLoadInfo SStatisBlk* p = &pStatisBlkArray->data[i]; STbStatisBlock* pStatisBlock = taosMemoryCalloc(1, sizeof(STbStatisBlock)); - tStatisBlockInit(pStatisBlock); + (void) tStatisBlockInit(pStatisBlock); int64_t st = taosGetTimestampMs(); - tsdbSttFileReadStatisBlock(pSttFileReader, p, pStatisBlock); + int32_t code = tsdbSttFileReadStatisBlock(pSttFileReader, p, pStatisBlock); + if (code != TSDB_CODE_SUCCESS) { + return 0; + } double el = (taosGetTimestampMs() - st) / 1000.0; pBlockLoadInfo->cost.loadStatisBlocks += 1; @@ -914,7 +982,7 @@ int32_t getNumOfRowsInSttBlock(SSttFileReader* pSttFileReader, SSttBlockLoadInfo } if (index >= pStatisBlock->numOfRecords) { - tStatisBlockDestroy(pStatisBlock); + (void) tStatisBlockDestroy(pStatisBlock); taosMemoryFreeClear(pStatisBlock); return num; } @@ -924,7 +992,7 @@ int32_t getNumOfRowsInSttBlock(SSttFileReader* pSttFileReader, SSttBlockLoadInfo while (i < TARRAY2_SIZE(pStatisBlkArray) && uidIndex < numOfTables) { p = &pStatisBlkArray->data[i]; if (p->minTbid.suid > suid) { - tStatisBlockDestroy(pStatisBlock); + (void) tStatisBlockDestroy(pStatisBlock); taosMemoryFreeClear(pStatisBlock); return num; } @@ -944,7 +1012,7 @@ int32_t getNumOfRowsInSttBlock(SSttFileReader* pSttFileReader, SSttBlockLoadInfo } } - tStatisBlockDestroy(pStatisBlock); + (void) tStatisBlockDestroy(pStatisBlock); taosMemoryFreeClear(pStatisBlock); return num; } @@ -956,19 +1024,22 @@ static void loadNextStatisticsBlock(SSttFileReader* pSttFileReader, STbStatisBlo (*i) += 1; (*j) = 0; if ((*i) < TARRAY2_SIZE(pStatisBlkArray)) { - tsdbSttFileReadStatisBlock(pSttFileReader, &pStatisBlkArray->data[(*i)], pStatisBlock); + (void) tsdbSttFileReadStatisBlock(pSttFileReader, &pStatisBlkArray->data[(*i)], pStatisBlock); } } } -void doAdjustValidDataIters(SArray* pLDIterList, int32_t numOfFileObj) { +int32_t doAdjustValidDataIters(SArray* pLDIterList, int32_t numOfFileObj) { int32_t size = taosArrayGetSize(pLDIterList); if (size < numOfFileObj) { int32_t inc = numOfFileObj - size; for (int32_t k = 0; k < inc; ++k) { SLDataIter* pIter = taosMemoryCalloc(1, sizeof(SLDataIter)); - taosArrayPush(pLDIterList, &pIter); + void* px = taosArrayPush(pLDIterList, &pIter); + if (px == NULL) { + return TSDB_CODE_OUT_OF_MEMORY; + } } } else if (size > numOfFileObj) { // remove unused LDataIter int32_t inc = size - numOfFileObj; @@ -978,21 +1049,30 @@ void doAdjustValidDataIters(SArray* pLDIterList, int32_t numOfFileObj) { destroyLDataIter(pIter); } } + + return TSDB_CODE_SUCCESS; } int32_t adjustSttDataIters(SArray* pSttFileBlockIterArray, STFileSet* pFileSet) { int32_t numOfLevels = pFileSet->lvlArr->size; + int32_t code = 0; // add the list/iter placeholder while (taosArrayGetSize(pSttFileBlockIterArray) < numOfLevels) { SArray* pList = taosArrayInit(4, POINTER_BYTES); - taosArrayPush(pSttFileBlockIterArray, &pList); + void* px = taosArrayPush(pSttFileBlockIterArray, &pList); + if (px == NULL) { + return TSDB_CODE_OUT_OF_MEMORY; + } } for (int32_t j = 0; j < numOfLevels; ++j) { SSttLvl* pSttLevel = pFileSet->lvlArr->data[j]; SArray* pList = taosArrayGetP(pSttFileBlockIterArray, j); - doAdjustValidDataIters(pList, TARRAY2_SIZE(pSttLevel->fobjArr)); + code = doAdjustValidDataIters(pList, TARRAY2_SIZE(pSttLevel->fobjArr)); + if (code != TSDB_CODE_SUCCESS) { + return code; + } } return TSDB_CODE_SUCCESS; @@ -1010,7 +1090,10 @@ int32_t tsdbGetRowsInSttFiles(STFileSet* pFileSet, SArray* pSttFileBlockIterArra } // add the list/iter placeholder - adjustSttDataIters(pSttFileBlockIterArray, pFileSet); + code = adjustSttDataIters(pSttFileBlockIterArray, pFileSet); + if (code != TSDB_CODE_SUCCESS) { + return numOfRows; + } for (int32_t j = 0; j < numOfLevels; ++j) { SSttLvl* pSttLevel = pFileSet->lvlArr->data[j]; diff --git a/source/dnode/vnode/src/tsdb/tsdbReadUtil.h b/source/dnode/vnode/src/tsdb/tsdbReadUtil.h index 45a2384b65..865e8e2d41 100644 --- a/source/dnode/vnode/src/tsdb/tsdbReadUtil.h +++ b/source/dnode/vnode/src/tsdb/tsdbReadUtil.h @@ -247,7 +247,6 @@ typedef struct SDataBlockIter { typedef struct SFileBlockDumpInfo { int32_t totalRows; int32_t rowIndex; -// STsdbRowKey lastKey; // this key should be removed bool allDumped; } SFileBlockDumpInfo; diff --git a/source/libs/stream/inc/streamInt.h b/source/libs/stream/inc/streamInt.h index 008d066717..fd248861e3 100644 --- a/source/libs/stream/inc/streamInt.h +++ b/source/libs/stream/inc/streamInt.h @@ -65,11 +65,6 @@ struct SActiveCheckpointInfo { tmr_h pSendReadyMsgTmr; }; -struct SConsensusCheckpoint { - int8_t inProcess; - -}; - typedef struct { int8_t type; SSDataBlock* pBlock; @@ -166,19 +161,19 @@ extern int32_t streamMetaId; int32_t streamTimerInit(); void streamTimerCleanUp(); - -void initRpcMsg(SRpcMsg* pMsg, int32_t msgType, void* pCont, int32_t contLen); +void initRpcMsg(SRpcMsg* pMsg, int32_t msgType, void* pCont, int32_t contLen); void streamStartMonitorDispatchData(SStreamTask* pTask, int64_t waitDuration); int32_t streamDispatchStreamBlock(SStreamTask* pTask); void destroyDispatchMsg(SStreamDispatchReq* pReq, int32_t numOfVgroups); void clearBufferedDispatchMsg(SStreamTask* pTask); -int32_t streamProcessCheckpointTriggerBlock(SStreamTask* pTask, SStreamDataBlock* pBlock); -SStreamDataBlock* createStreamBlockFromDispatchMsg(const SStreamDispatchReq* pReq, int32_t blockType, int32_t srcVg); -SStreamDataBlock* createStreamBlockFromResults(SStreamQueueItem* pItem, SStreamTask* pTask, int64_t resultSize, - SArray* pRes); -void destroyStreamDataBlock(SStreamDataBlock* pBlock); +int32_t streamProcessCheckpointTriggerBlock(SStreamTask* pTask, SStreamDataBlock* pBlock); +int32_t createStreamBlockFromDispatchMsg(const SStreamDispatchReq* pReq, int32_t blockType, int32_t srcVg, + SStreamDataBlock** pBlock); +int32_t createStreamBlockFromResults(SStreamQueueItem* pItem, SStreamTask* pTask, int64_t resultSize, SArray* pRes, + SStreamDataBlock** pBlock); +void destroyStreamDataBlock(SStreamDataBlock* pBlock); int32_t streamRetrieveReqToData(const SStreamRetrieveReq* pReq, SStreamDataBlock* pData, const char* idstr); int32_t streamBroadcastToUpTasks(SStreamTask* pTask, const SSDataBlock* pBlock); @@ -206,24 +201,24 @@ EExtractDataCode streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueIt int32_t streamQueueItemGetSize(const SStreamQueueItem* pItem); void streamQueueItemIncSize(const SStreamQueueItem* pItem, int32_t size); const char* streamQueueItemGetTypeStr(int32_t type); -SStreamQueueItem* streamQueueMergeQueueItem(SStreamQueueItem* dst, SStreamQueueItem* pElem); +int32_t streamQueueMergeQueueItem(SStreamQueueItem* dst, SStreamQueueItem* pElem, SStreamQueueItem** pRes); int32_t streamTransferStatePrepare(SStreamTask* pTask); -SStreamQueue* streamQueueOpen(int64_t cap); -void streamQueueClose(SStreamQueue* pQueue, int32_t taskId); -void streamQueueProcessSuccess(SStreamQueue* queue); -void streamQueueProcessFail(SStreamQueue* queue); -void* streamQueueNextItem(SStreamQueue* pQueue); -void streamFreeQitem(SStreamQueueItem* data); -int32_t streamQueueGetItemSize(const SStreamQueue* pQueue); +int32_t streamQueueOpen(int64_t cap, SStreamQueue** pQ); +void streamQueueClose(SStreamQueue* pQueue, int32_t taskId); +void streamQueueProcessSuccess(SStreamQueue* queue); +void streamQueueProcessFail(SStreamQueue* queue); +void* streamQueueNextItem(SStreamQueue* pQueue); +void streamFreeQitem(SStreamQueueItem* data); +int32_t streamQueueGetItemSize(const SStreamQueue* pQueue); -void streamMetaRemoveDB(void* arg, char* key); -void streamMetaHbToMnode(void* param, void* tmrId); -SMetaHbInfo* createMetaHbInfo(int64_t* pRid); -void* destroyMetaHbInfo(SMetaHbInfo* pInfo); -void streamMetaWaitForHbTmrQuit(SStreamMeta* pMeta); -void streamMetaGetHbSendInfo(SMetaHbInfo* pInfo, int64_t* pStartTs, int32_t* pSendCount); -int32_t streamMetaSendHbHelper(SStreamMeta* pMeta); +void streamMetaRemoveDB(void* arg, char* key); +void streamMetaHbToMnode(void* param, void* tmrId); +int32_t createMetaHbInfo(int64_t* pRid, SMetaHbInfo** pRes); +void destroyMetaHbInfo(SMetaHbInfo* pInfo); +void streamMetaWaitForHbTmrQuit(SStreamMeta* pMeta); +void streamMetaGetHbSendInfo(SMetaHbInfo* pInfo, int64_t* pStartTs, int32_t* pSendCount); +int32_t streamMetaSendHbHelper(SStreamMeta* pMeta); ECHECKPOINT_BACKUP_TYPE streamGetCheckpointBackupType(); diff --git a/source/libs/stream/src/streamCheckStatus.c b/source/libs/stream/src/streamCheckStatus.c index 3ea3e814ed..e588be0784 100644 --- a/source/libs/stream/src/streamCheckStatus.c +++ b/source/libs/stream/src/streamCheckStatus.c @@ -40,8 +40,11 @@ static SDownstreamStatusInfo* findCheckRspStatus(STaskCheckInfo* pInfo, int32_t int32_t streamTaskCheckStatus(SStreamTask* pTask, int32_t upstreamTaskId, int32_t vgId, int64_t stage, int64_t* oldStage) { - SStreamUpstreamEpInfo* pInfo = streamTaskGetUpstreamTaskEpInfo(pTask, upstreamTaskId); - ASSERT(pInfo != NULL); + SStreamUpstreamEpInfo* pInfo = NULL; + int32_t code = streamTaskGetUpstreamTaskEpInfo(pTask, upstreamTaskId, &pInfo); + if (code != TSDB_CODE_SUCCESS) { + return code; + } *oldStage = pInfo->stage; const char* id = pTask->id.idStr; @@ -63,7 +66,7 @@ int32_t streamTaskCheckStatus(SStreamTask* pTask, int32_t upstreamTaskId, int32_ id, upstreamTaskId, vgId, stage, pInfo->stage); // record the checkpoint failure id and sent to mnode taosThreadMutexLock(&pTask->lock); - ETaskStatus status = streamTaskGetStatus(pTask)->state; + ETaskStatus status = streamTaskGetStatus(pTask).state; if (status == TASK_STATUS__CK) { streamTaskSetFailedCheckpointId(pTask); } @@ -72,7 +75,7 @@ int32_t streamTaskCheckStatus(SStreamTask* pTask, int32_t upstreamTaskId, int32_ if (pInfo->stage != stage) { taosThreadMutexLock(&pTask->lock); - ETaskStatus status = streamTaskGetStatus(pTask)->state; + ETaskStatus status = streamTaskGetStatus(pTask).state; if (status == TASK_STATUS__CK) { streamTaskSetFailedCheckpointId(pTask); } @@ -166,15 +169,16 @@ void streamTaskProcessCheckMsg(SStreamMeta* pMeta, SStreamTaskCheckReq* pReq, SS taskId, pReq->upstreamTaskId, pReq->upstreamNodeId, pMeta->vgId); pRsp->status = TASK_DOWNSTREAM_NOT_LEADER; } else { - SStreamTask* pTask = streamMetaAcquireTask(pMeta, pReq->streamId, taskId); + SStreamTask* pTask = NULL; + int32_t code = streamMetaAcquireTask(pMeta, pReq->streamId, taskId, &pTask); if (pTask != NULL) { pRsp->status = streamTaskCheckStatus(pTask, pReq->upstreamTaskId, pReq->upstreamNodeId, pReq->stage, &pRsp->oldStage); - SStreamTaskState* pState = streamTaskGetStatus(pTask); + SStreamTaskState pState = streamTaskGetStatus(pTask); stDebug("s-task:%s status:%s, stage:%" PRId64 " recv task check req(reqId:0x%" PRIx64 ") task:0x%x (vgId:%d), check_status:%d", - pTask->id.idStr, pState->name, pRsp->oldStage, pRsp->reqId, pRsp->upstreamTaskId, pRsp->upstreamNodeId, + pTask->id.idStr, pState.name, pRsp->oldStage, pRsp->reqId, pRsp->upstreamTaskId, pRsp->upstreamNodeId, pRsp->status); streamMetaReleaseTask(pMeta, pTask); } else { @@ -647,12 +651,12 @@ int32_t addDownstreamFailedStatusResultAsync(SMsgCb* pMsgCb, int32_t vgId, int64 void rspMonitorFn(void* param, void* tmrId) { SStreamTask* pTask = param; SStreamMeta* pMeta = pTask->pMeta; - SStreamTaskState* pStat = streamTaskGetStatus(pTask); + SStreamTaskState pStat = streamTaskGetStatus(pTask); STaskCheckInfo* pInfo = &pTask->taskCheckInfo; int32_t vgId = pTask->pMeta->vgId; int64_t now = taosGetTimestampMs(); int64_t timeoutDuration = now - pInfo->timeoutStartTs; - ETaskStatus state = pStat->state; + ETaskStatus state = pStat.state; const char* id = pTask->id.idStr; int32_t numOfReady = 0; int32_t numOfFault = 0; @@ -665,7 +669,7 @@ void rspMonitorFn(void* param, void* tmrId) { if (state == TASK_STATUS__STOP) { int32_t ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1); - stDebug("s-task:%s status:%s vgId:%d quit from monitor check-rsp tmr, ref:%d", id, pStat->name, vgId, ref); + stDebug("s-task:%s status:%s vgId:%d quit from monitor check-rsp tmr, ref:%d", id, pStat.name, vgId, ref); streamTaskCompleteCheckRsp(pInfo, true, id); @@ -681,7 +685,7 @@ void rspMonitorFn(void* param, void* tmrId) { if (state == TASK_STATUS__DROPPING || state == TASK_STATUS__READY) { int32_t ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1); - stDebug("s-task:%s status:%s vgId:%d quit from monitor check-rsp tmr, ref:%d", id, pStat->name, vgId, ref); + stDebug("s-task:%s status:%s vgId:%d quit from monitor check-rsp tmr, ref:%d", id, pStat.name, vgId, ref); streamTaskCompleteCheckRsp(pInfo, true, id); streamMetaReleaseTask(pMeta, pTask); @@ -691,7 +695,7 @@ void rspMonitorFn(void* param, void* tmrId) { taosThreadMutexLock(&pInfo->checkInfoLock); if (pInfo->notReadyTasks == 0) { int32_t ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1); - stDebug("s-task:%s status:%s vgId:%d all downstream ready, quit from monitor rsp tmr, ref:%d", id, pStat->name, + stDebug("s-task:%s status:%s vgId:%d all downstream ready, quit from monitor rsp tmr, ref:%d", id, pStat.name, vgId, ref); streamTaskCompleteCheckRsp(pInfo, false, id); @@ -703,7 +707,7 @@ void rspMonitorFn(void* param, void* tmrId) { SArray* pNotReadyList = taosArrayInit(4, sizeof(int64_t)); SArray* pTimeoutList = taosArrayInit(4, sizeof(int64_t)); - if (pStat->state == TASK_STATUS__UNINIT) { + if (pStat.state == TASK_STATUS__UNINIT) { getCheckRspStatus(pInfo, timeoutDuration, &numOfReady, &numOfFault, &numOfNotRsp, pTimeoutList, pNotReadyList, id); numOfNotReady = (int32_t)taosArrayGetSize(pNotReadyList); @@ -716,7 +720,7 @@ void rspMonitorFn(void* param, void* tmrId) { stDebug( "s-task:%s status:%s vgId:%d all rsp. quit from monitor rsp tmr, since vnode-transfer/leader-change/restart " "detected, total:%d, notRsp:%d, notReady:%d, fault:%d, timeout:%d, ready:%d ref:%d", - id, pStat->name, vgId, total, numOfNotRsp, numOfNotReady, numOfFault, numOfTimeout, numOfReady, ref); + id, pStat.name, vgId, total, numOfNotRsp, numOfNotReady, numOfFault, numOfTimeout, numOfReady, ref); streamTaskCompleteCheckRsp(pInfo, false, id); taosThreadMutexUnlock(&pInfo->checkInfoLock); @@ -727,7 +731,7 @@ void rspMonitorFn(void* param, void* tmrId) { return; } } else { // unexpected status - stError("s-task:%s unexpected task status:%s during waiting for check rsp", id, pStat->name); + stError("s-task:%s unexpected task status:%s during waiting for check rsp", id, pStat.name); } // checking of downstream tasks has been stopped by other threads @@ -736,7 +740,7 @@ void rspMonitorFn(void* param, void* tmrId) { stDebug( "s-task:%s status:%s vgId:%d stopped by other threads to check downstream process, total:%d, notRsp:%d, " "notReady:%d, fault:%d, timeout:%d, ready:%d ref:%d", - id, pStat->name, vgId, total, numOfNotRsp, numOfNotReady, numOfFault, numOfTimeout, numOfReady, ref); + id, pStat.name, vgId, total, numOfNotRsp, numOfNotReady, numOfFault, numOfTimeout, numOfReady, ref); streamTaskCompleteCheckRsp(pInfo, false, id); taosThreadMutexUnlock(&pInfo->checkInfoLock); diff --git a/source/libs/stream/src/streamCheckpoint.c b/source/libs/stream/src/streamCheckpoint.c index 023783a2ae..8de923e900 100644 --- a/source/libs/stream/src/streamCheckpoint.c +++ b/source/libs/stream/src/streamCheckpoint.c @@ -29,14 +29,12 @@ static int32_t appendCheckpointIntoInputQ(SStreamTask* pTask, int32_t checkpoint static int32_t doSendRetrieveTriggerMsg(SStreamTask* pTask, SArray* pNotSendList); static void checkpointTriggerMonitorFn(void* param, void* tmrId); -SStreamDataBlock* createChkptTriggerBlock(SStreamTask* pTask, int32_t checkpointType, int64_t checkpointId, - int32_t transId, int32_t srcTaskId) { - SStreamDataBlock* pChkpoint; - +int32_t createChkptTriggerBlock(SStreamTask* pTask, int32_t checkpointType, int64_t checkpointId, int32_t transId, + int32_t srcTaskId, SStreamDataBlock** pRes) { + SStreamDataBlock* pChkpoint = NULL; int32_t code = taosAllocateQitem(sizeof(SStreamDataBlock), DEF_QITEM, sizeof(SSDataBlock), (void**)&pChkpoint); if (code) { - terrno = code; - return NULL; + return code; } pChkpoint->type = checkpointType; @@ -48,8 +46,7 @@ SStreamDataBlock* createChkptTriggerBlock(SStreamTask* pTask, int32_t checkpoint SSDataBlock* pBlock = taosMemoryCalloc(1, sizeof(SSDataBlock)); if (pBlock == NULL) { taosFreeQitem(pChkpoint); - terrno = TSDB_CODE_OUT_OF_MEMORY; - return NULL; + return TSDB_CODE_OUT_OF_MEMORY; } pBlock->info.type = STREAM_CHECKPOINT; @@ -62,28 +59,37 @@ SStreamDataBlock* createChkptTriggerBlock(SStreamTask* pTask, int32_t checkpoint if (pChkpoint->blocks == NULL) { taosMemoryFree(pBlock); taosFreeQitem(pChkpoint); - terrno = TSDB_CODE_OUT_OF_MEMORY; - return NULL; + return TSDB_CODE_OUT_OF_MEMORY; } - taosArrayPush(pChkpoint->blocks, pBlock); + void* p = taosArrayPush(pChkpoint->blocks, pBlock); + if (p == NULL) { + taosArrayDestroy(pChkpoint->blocks); + taosMemoryFree(pBlock); + taosFreeQitem(pChkpoint); + return TSDB_CODE_OUT_OF_MEMORY; + } + + *pRes = pChkpoint; taosMemoryFree(pBlock); - terrno = 0; - - return pChkpoint; + return TSDB_CODE_SUCCESS; } +// this message must be put into inputq successfully, continue retrying until it succeeds int32_t appendCheckpointIntoInputQ(SStreamTask* pTask, int32_t checkpointType, int64_t checkpointId, int32_t transId, int32_t srcTaskId) { - SStreamDataBlock* pCheckpoint = createChkptTriggerBlock(pTask, checkpointType, checkpointId, transId, srcTaskId); + SStreamDataBlock* pCheckpoint = NULL; + int32_t code = createChkptTriggerBlock(pTask, checkpointType, checkpointId, transId, srcTaskId, &pCheckpoint); + if (code != TSDB_CODE_SUCCESS) { + return code; + } if (streamTaskPutDataIntoInputQ(pTask, (SStreamQueueItem*)pCheckpoint) < 0) { return TSDB_CODE_OUT_OF_MEMORY; } - streamTrySchedExec(pTask); - return TSDB_CODE_SUCCESS; + return streamTrySchedExec(pTask); } int32_t streamProcessCheckpointSourceReq(SStreamTask* pTask, SStreamCheckpointSourceReq* pReq) { @@ -178,22 +184,26 @@ int32_t streamProcessCheckpointTriggerBlock(SStreamTask* pTask, SStreamDataBlock SActiveCheckpointInfo* pActiveInfo = pTask->chkInfo.pActiveInfo; - taosThreadMutexLock(&pTask->lock); + (void) taosThreadMutexLock(&pTask->lock); if (pTask->chkInfo.checkpointId > checkpointId) { stError("s-task:%s vgId:%d current checkpointId:%" PRId64 " recv expired checkpoint-trigger block, checkpointId:%" PRId64 " transId:%d, discard", id, vgId, pTask->chkInfo.checkpointId, checkpointId, transId); - taosThreadMutexUnlock(&pTask->lock); + code = taosThreadMutexUnlock(&pTask->lock); streamFreeQitem((SStreamQueueItem*)pBlock); - return TSDB_CODE_SUCCESS; + return code; } if (pTask->chkInfo.checkpointId == checkpointId) { { // send checkpoint-ready msg to upstream SRpcMsg msg = {0}; + SStreamUpstreamEpInfo* pInfo = NULL; + code = streamTaskGetUpstreamTaskEpInfo(pTask, pBlock->srcTaskId, &pInfo); + if (code != TSDB_CODE_SUCCESS) { + return code; + } - SStreamUpstreamEpInfo* pInfo = streamTaskGetUpstreamTaskEpInfo(pTask, pBlock->srcTaskId); initCheckpointReadyMsg(pTask, pInfo->nodeId, pBlock->srcTaskId, pInfo->childId, checkpointId, &msg); tmsgSendReq(&pInfo->epSet, &msg); } @@ -204,30 +214,30 @@ int32_t streamProcessCheckpointTriggerBlock(SStreamTask* pTask, SStreamDataBlock id, vgId, pBlock->srcTaskId); streamTaskOpenUpstreamInput(pTask, pBlock->srcTaskId); - taosThreadMutexUnlock(&pTask->lock); + code = taosThreadMutexUnlock(&pTask->lock); streamFreeQitem((SStreamQueueItem*)pBlock); - return TSDB_CODE_SUCCESS; + return code; } - if (streamTaskGetStatus(pTask)->state == TASK_STATUS__CK) { + if (streamTaskGetStatus(pTask).state == TASK_STATUS__CK) { if (pActiveInfo->activeId != checkpointId) { stError("s-task:%s vgId:%d active checkpointId:%" PRId64 ", recv invalid checkpoint-trigger checkpointId:%" PRId64 " discard", id, vgId, pActiveInfo->activeId, checkpointId); - taosThreadMutexUnlock(&pTask->lock); + code = taosThreadMutexUnlock(&pTask->lock); streamFreeQitem((SStreamQueueItem*)pBlock); - return TSDB_CODE_SUCCESS; + return code; } else { // checkpointId == pActiveInfo->activeId if (pActiveInfo->allUpstreamTriggerRecv == 1) { stDebug( "s-task:%s vgId:%d all upstream checkpoint-trigger recv, discard this checkpoint-trigger, " "checkpointId:%" PRId64 " transId:%d", id, vgId, checkpointId, transId); - taosThreadMutexUnlock(&pTask->lock); + code = taosThreadMutexUnlock(&pTask->lock); streamFreeQitem((SStreamQueueItem*)pBlock); - return TSDB_CODE_SUCCESS; + return code; } if (taskLevel == TASK_LEVEL__SINK || taskLevel == TASK_LEVEL__AGG) { @@ -240,23 +250,23 @@ int32_t streamProcessCheckpointTriggerBlock(SStreamTask* pTask, SStreamDataBlock ", prev recvTs:%" PRId64 " discard", pTask->id.idStr, p->upstreamTaskId, p->upstreamNodeId, p->checkpointId, p->recvTs); - taosThreadMutexUnlock(&pTask->lock); + code = taosThreadMutexUnlock(&pTask->lock); streamFreeQitem((SStreamQueueItem*)pBlock); - return TSDB_CODE_SUCCESS; + return code; } } } } } - taosThreadMutexUnlock(&pTask->lock); + (void) taosThreadMutexUnlock(&pTask->lock); stDebug("s-task:%s vgId:%d start to handle the checkpoint-trigger block, checkpointId:%" PRId64 " ver:%" PRId64 ", transId:%d current active checkpointId:%" PRId64, id, vgId, pTask->chkInfo.checkpointId, pTask->chkInfo.checkpointVer, transId, checkpointId); // set task status - if (streamTaskGetStatus(pTask)->state != TASK_STATUS__CK) { + if (streamTaskGetStatus(pTask).state != TASK_STATUS__CK) { pActiveInfo->activeId = checkpointId; pActiveInfo->transId = transId; @@ -345,9 +355,9 @@ int32_t streamProcessCheckpointReadyMsg(SStreamTask* pTask, int64_t checkpointId ASSERT(total > 0); // 1. not in checkpoint status now - SStreamTaskState* pStat = streamTaskGetStatus(pTask); - if (pStat->state != TASK_STATUS__CK) { - stError("s-task:%s status:%s discard checkpoint-ready msg from task:0x%x", id, pStat->name, downstreamTaskId); + SStreamTaskState pStat = streamTaskGetStatus(pTask); + if (pStat.state != TASK_STATUS__CK) { + stError("s-task:%s status:%s discard checkpoint-ready msg from task:0x%x", id, pStat.name, downstreamTaskId); return TSDB_CODE_STREAM_TASK_IVLD_STATUS; } @@ -355,11 +365,11 @@ int32_t streamProcessCheckpointReadyMsg(SStreamTask* pTask, int64_t checkpointId if (pTask->chkInfo.checkpointId > checkpointId || pInfo->activeId != checkpointId) { stError("s-task:%s status:%s checkpointId:%" PRId64 " new arrival checkpoint-ready msg (checkpointId:%" PRId64 ") from task:0x%x, expired and discard ", - id, pStat->name, pTask->chkInfo.checkpointId, checkpointId, downstreamTaskId); + id, pStat.name, pTask->chkInfo.checkpointId, checkpointId, downstreamTaskId); return -1; } - taosThreadMutexLock(&pInfo->lock); + (void) taosThreadMutexLock(&pInfo->lock); // only when all downstream tasks are send checkpoint rsp, we can start the checkpoint procedure for the agg task int32_t size = taosArrayGetSize(pInfo->pCheckpointReadyRecvList); @@ -386,7 +396,7 @@ int32_t streamProcessCheckpointReadyMsg(SStreamTask* pTask, int64_t checkpointId int32_t notReady = total - taosArrayGetSize(pInfo->pCheckpointReadyRecvList); int32_t transId = pInfo->transId; - taosThreadMutexUnlock(&pInfo->lock); + (void) taosThreadMutexUnlock(&pInfo->lock); if (notReady == 0) { stDebug("s-task:%s all downstream tasks have completed build checkpoint, do checkpoint for current task", id); @@ -401,7 +411,7 @@ int32_t streamTaskProcessCheckpointReadyRsp(SStreamTask* pTask, int32_t upstream int64_t now = taosGetTimestampMs(); int32_t numOfConfirmed = 0; - taosThreadMutexLock(&pInfo->lock); + (void) taosThreadMutexLock(&pInfo->lock); for (int32_t i = 0; i < taosArrayGetSize(pInfo->pReadyMsgList); ++i) { STaskCheckpointReadyInfo* pReadyInfo = taosArrayGet(pInfo->pReadyMsgList, i); if (pReadyInfo->upstreamTaskId == upstreamTaskId && pReadyInfo->checkpointId == checkpointId) { @@ -422,7 +432,7 @@ int32_t streamTaskProcessCheckpointReadyRsp(SStreamTask* pTask, int32_t upstream stDebug("s-task:%s send checkpoint-ready msg to %d upstream confirmed, checkpointId:%" PRId64, pTask->id.idStr, numOfConfirmed, checkpointId); - taosThreadMutexUnlock(&pInfo->lock); + (void) taosThreadMutexUnlock(&pInfo->lock); return TSDB_CODE_SUCCESS; } @@ -430,12 +440,12 @@ void streamTaskClearCheckInfo(SStreamTask* pTask, bool clearChkpReadyMsg) { pTask->chkInfo.startTs = 0; // clear the recorded start time streamTaskOpenAllUpstreamInput(pTask); // open inputQ for all upstream tasks - taosThreadMutexLock(&pTask->chkInfo.pActiveInfo->lock); + (void) taosThreadMutexLock(&pTask->chkInfo.pActiveInfo->lock); streamTaskClearActiveInfo(pTask->chkInfo.pActiveInfo); if (clearChkpReadyMsg) { streamClearChkptReadyMsg(pTask->chkInfo.pActiveInfo); } - taosThreadMutexUnlock(&pTask->chkInfo.pActiveInfo->lock); + (void) taosThreadMutexUnlock(&pTask->chkInfo.pActiveInfo->lock); } int32_t streamTaskUpdateTaskCheckpointInfo(SStreamTask* pTask, bool restored, SVUpdateCheckpointInfoReq* pReq) { @@ -445,7 +455,7 @@ int32_t streamTaskUpdateTaskCheckpointInfo(SStreamTask* pTask, bool restored, SV const char* id = pTask->id.idStr; SCheckpointInfo* pInfo = &pTask->chkInfo; - taosThreadMutexLock(&pTask->lock); + (void) taosThreadMutexLock(&pTask->lock); if (pReq->checkpointId <= pInfo->checkpointId) { stDebug("s-task:%s vgId:%d latest checkpointId:%" PRId64 " checkpointVer:%" PRId64 @@ -453,13 +463,13 @@ int32_t streamTaskUpdateTaskCheckpointInfo(SStreamTask* pTask, bool restored, SV " transId:%d ignored", id, vgId, pInfo->checkpointId, pInfo->checkpointVer, pReq->checkpointId, pReq->checkpointVer, pReq->transId); - taosThreadMutexUnlock(&pTask->lock); + (void) taosThreadMutexUnlock(&pTask->lock); { // destroy the related fill-history tasks // drop task should not in the meta-lock, and drop the related fill-history task now streamMetaWUnLock(pMeta); if (pReq->dropRelHTask) { - streamMetaUnregisterTask(pMeta, pReq->hStreamId, pReq->hTaskId); + (void) streamMetaUnregisterTask(pMeta, pReq->hStreamId, pReq->hTaskId); int32_t numOfTasks = streamMetaGetNumOfTasks(pMeta); stDebug("s-task:%s vgId:%d related fill-history task:0x%x dropped in update checkpointInfo, remain tasks:%d", id, vgId, pReq->taskId, numOfTasks); @@ -473,17 +483,17 @@ int32_t streamTaskUpdateTaskCheckpointInfo(SStreamTask* pTask, bool restored, SV return TSDB_CODE_SUCCESS; } - SStreamTaskState* pStatus = streamTaskGetStatus(pTask); + SStreamTaskState pStatus = streamTaskGetStatus(pTask); if (!restored) { // during restore procedure, do update checkpoint-info stDebug("s-task:%s vgId:%d status:%s update the checkpoint-info during restore, checkpointId:%" PRId64 "->%" PRId64 " checkpointVer:%" PRId64 "->%" PRId64 " checkpointTs:%" PRId64 "->%" PRId64, - id, vgId, pStatus->name, pInfo->checkpointId, pReq->checkpointId, pInfo->checkpointVer, pReq->checkpointVer, + id, vgId, pStatus.name, pInfo->checkpointId, pReq->checkpointId, pInfo->checkpointVer, pReq->checkpointVer, pInfo->checkpointTime, pReq->checkpointTs); } else { // not in restore status, must be in checkpoint status stDebug("s-task:%s vgId:%d status:%s start to update the checkpoint-info, checkpointId:%" PRId64 "->%" PRId64 " checkpointVer:%" PRId64 "->%" PRId64 " checkpointTs:%" PRId64 "->%" PRId64, - id, vgId, pStatus->name, pInfo->checkpointId, pReq->checkpointId, pInfo->checkpointVer, pReq->checkpointVer, + id, vgId, pStatus.name, pInfo->checkpointId, pReq->checkpointId, pInfo->checkpointVer, pReq->checkpointVer, pInfo->checkpointTime, pReq->checkpointTs); } @@ -496,11 +506,11 @@ int32_t streamTaskUpdateTaskCheckpointInfo(SStreamTask* pTask, bool restored, SV streamTaskClearCheckInfo(pTask, true); - if (pStatus->state == TASK_STATUS__CK) { + if (pStatus.state == TASK_STATUS__CK) { // todo handle error code = streamTaskHandleEvent(pTask->status.pSM, TASK_EVENT_CHECKPOINT_DONE); } else { - stDebug("s-task:0x%x vgId:%d not handle checkpoint-done event, status:%s", pReq->taskId, vgId, pStatus->name); + stDebug("s-task:0x%x vgId:%d not handle checkpoint-done event, status:%s", pReq->taskId, vgId, pStatus.name); } if (pReq->dropRelHTask) { @@ -510,7 +520,7 @@ int32_t streamTaskUpdateTaskCheckpointInfo(SStreamTask* pTask, bool restored, SV } stDebug("s-task:0x%x set the persistent status attr to be ready, prev:%s, status in sm:%s", pReq->taskId, - streamTaskGetStatusStr(pTask->status.taskStatus), streamTaskGetStatus(pTask)->name); + streamTaskGetStatusStr(pTask->status.taskStatus), streamTaskGetStatus(pTask).name); pTask->status.taskStatus = TASK_STATUS__READY; @@ -521,12 +531,12 @@ int32_t streamTaskUpdateTaskCheckpointInfo(SStreamTask* pTask, bool restored, SV return code; } - taosThreadMutexUnlock(&pTask->lock); + (void) taosThreadMutexUnlock(&pTask->lock); streamMetaWUnLock(pMeta); // drop task should not in the meta-lock, and drop the related fill-history task now if (pReq->dropRelHTask) { - streamMetaUnregisterTask(pMeta, pReq->hStreamId, pReq->hTaskId); + (void) streamMetaUnregisterTask(pMeta, pReq->hStreamId, pReq->hTaskId); int32_t numOfTasks = streamMetaGetNumOfTasks(pMeta); stDebug("s-task:%s vgId:%d related fill-history task:0x%x dropped, remain tasks:%d", id, vgId, (int32_t)pReq->hTaskId, numOfTasks); @@ -720,9 +730,9 @@ int32_t streamTaskBuildCheckpoint(SStreamTask* pTask) { code = streamSendChkptReportMsg(pTask, &pTask->chkInfo, dropRelHTask); } } else { // clear the checkpoint info if failed - taosThreadMutexLock(&pTask->lock); + (void) taosThreadMutexLock(&pTask->lock); streamTaskSetFailedCheckpointId(pTask); // set failed checkpoint id before clear the checkpoint info - taosThreadMutexUnlock(&pTask->lock); + (void) taosThreadMutexUnlock(&pTask->lock); code = streamTaskHandleEvent(pTask->status.pSM, TASK_EVENT_CHECKPOINT_DONE); stDebug("s-task:%s clear checkpoint flag since gen checkpoint failed, checkpointId:%" PRId64, id, ckId); @@ -760,13 +770,13 @@ void checkpointTriggerMonitorFn(void* param, void* tmrId) { pActiveInfo->checkCounter = 0; stDebug("s-task:%s vgId:%d checkpoint-trigger monitor in tmr, ts:%" PRId64, id, vgId, now); - taosThreadMutexLock(&pTask->lock); - SStreamTaskState* pState = streamTaskGetStatus(pTask); - if (pState->state != TASK_STATUS__CK) { + (void) taosThreadMutexLock(&pTask->lock); + SStreamTaskState pState = streamTaskGetStatus(pTask); + if (pState.state != TASK_STATUS__CK) { int32_t ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1); stDebug("s-task:%s vgId:%d not in checkpoint status, quit from monitor checkpoint-trigger, ref:%d", id, vgId, ref); - taosThreadMutexUnlock(&pTask->lock); + (void) taosThreadMutexUnlock(&pTask->lock); streamMetaReleaseTask(pTask->pMeta, pTask); return; } @@ -777,14 +787,14 @@ void checkpointTriggerMonitorFn(void* param, void* tmrId) { stDebug("s-task:%s vgId:%d all checkpoint-trigger recv, quit from monitor checkpoint-trigger, ref:%d", id, vgId, ref); - taosThreadMutexUnlock(&pTask->lock); + (void) taosThreadMutexUnlock(&pTask->lock); streamMetaReleaseTask(pTask->pMeta, pTask); return; } - taosThreadMutexUnlock(&pTask->lock); + (void) taosThreadMutexUnlock(&pTask->lock); - taosThreadMutexLock(&pActiveInfo->lock); + (void) taosThreadMutexLock(&pActiveInfo->lock); // send msg to retrieve checkpoint trigger msg SArray* pList = pTask->upstreamInfo.pList; @@ -817,7 +827,7 @@ void checkpointTriggerMonitorFn(void* param, void* tmrId) { // do send retrieve checkpoint trigger msg to upstream int32_t size = taosArrayGetSize(pNotSendList); doSendRetrieveTriggerMsg(pTask, pNotSendList); - taosThreadMutexUnlock(&pActiveInfo->lock); + (void) taosThreadMutexUnlock(&pActiveInfo->lock); // check every 100ms if (size > 0) { @@ -881,15 +891,15 @@ bool streamTaskAlreadySendTrigger(SStreamTask* pTask, int32_t downstreamNodeId) int64_t now = taosGetTimestampMs(); const char* id = pTask->id.idStr; SActiveCheckpointInfo* pInfo = pTask->chkInfo.pActiveInfo; - SStreamTaskState* pStatus = streamTaskGetStatus(pTask); + SStreamTaskState pStatus = streamTaskGetStatus(pTask); - if (pStatus->state != TASK_STATUS__CK) { + if (pStatus.state != TASK_STATUS__CK) { return false; } - taosThreadMutexLock(&pInfo->lock); + (void) taosThreadMutexLock(&pInfo->lock); if (!pInfo->dispatchTrigger) { - taosThreadMutexUnlock(&pInfo->lock); + (void) taosThreadMutexUnlock(&pInfo->lock); return false; } @@ -911,7 +921,7 @@ bool streamTaskAlreadySendTrigger(SStreamTask* pTask, int32_t downstreamNodeId) id, pSendInfo->sendTs, before, pInfo->activeId, pInfo->transId); } - taosThreadMutexUnlock(&pInfo->lock); + (void) taosThreadMutexUnlock(&pInfo->lock); return true; } @@ -930,11 +940,12 @@ void streamTaskGetTriggerRecvStatus(SStreamTask* pTask, int32_t* pRecved, int32_ } // record the dispatch checkpoint trigger info in the list +// memory insufficient may cause the stream computing stopped void streamTaskInitTriggerDispatchInfo(SStreamTask* pTask) { SActiveCheckpointInfo* pInfo = pTask->chkInfo.pActiveInfo; int64_t now = taosGetTimestampMs(); - taosThreadMutexLock(&pInfo->lock); + (void) taosThreadMutexLock(&pInfo->lock); // outputQ should be empty here ASSERT(streamQueueGetNumOfUnAccessedItems(pTask->outputq.queue) == 0); @@ -944,31 +955,37 @@ void streamTaskInitTriggerDispatchInfo(SStreamTask* pTask) { STaskDispatcherFixed* pDispatch = &pTask->outputInfo.fixedDispatcher; STaskTriggerSendInfo p = {.sendTs = now, .recved = false, .nodeId = pDispatch->nodeId, .taskId = pDispatch->taskId}; - taosArrayPush(pInfo->pDispatchTriggerList, &p); + void* px = taosArrayPush(pInfo->pDispatchTriggerList, &p); + if (px == NULL) { + // pause the stream task, if memory not enough + } } else { for (int32_t i = 0; i < streamTaskGetNumOfDownstream(pTask); ++i) { SVgroupInfo* pVgInfo = taosArrayGet(pTask->outputInfo.shuffleDispatcher.dbInfo.pVgroupInfos, i); STaskTriggerSendInfo p = {.sendTs = now, .recved = false, .nodeId = pVgInfo->vgId, .taskId = pVgInfo->taskId}; - taosArrayPush(pInfo->pDispatchTriggerList, &p); + void* px = taosArrayPush(pInfo->pDispatchTriggerList, &p); + if (px == NULL) { + // pause the stream task, if memory not enough + } } } - taosThreadMutexUnlock(&pInfo->lock); + (void) taosThreadMutexUnlock(&pInfo->lock); } int32_t streamTaskGetNumOfConfirmed(SStreamTask* pTask) { SActiveCheckpointInfo* pInfo = pTask->chkInfo.pActiveInfo; int32_t num = 0; - taosThreadMutexLock(&pInfo->lock); + (void) taosThreadMutexLock(&pInfo->lock); for (int32_t i = 0; i < taosArrayGetSize(pInfo->pDispatchTriggerList); ++i) { STaskTriggerSendInfo* p = taosArrayGet(pInfo->pDispatchTriggerList, i); if (p->recved) { num++; } } - taosThreadMutexUnlock(&pInfo->lock); + (void) taosThreadMutexUnlock(&pInfo->lock); return num; } @@ -976,7 +993,7 @@ void streamTaskSetTriggerDispatchConfirmed(SStreamTask* pTask, int32_t vgId) { SActiveCheckpointInfo* pInfo = pTask->chkInfo.pActiveInfo; int32_t taskId = 0; - taosThreadMutexLock(&pInfo->lock); + (void) taosThreadMutexLock(&pInfo->lock); for (int32_t i = 0; i < taosArrayGetSize(pInfo->pDispatchTriggerList); ++i) { STaskTriggerSendInfo* p = taosArrayGet(pInfo->pDispatchTriggerList, i); @@ -990,7 +1007,7 @@ void streamTaskSetTriggerDispatchConfirmed(SStreamTask* pTask, int32_t vgId) { } } - taosThreadMutexUnlock(&pInfo->lock); + (void) taosThreadMutexUnlock(&pInfo->lock); int32_t numOfConfirmed = streamTaskGetNumOfConfirmed(pTask); int32_t total = streamTaskGetNumOfDownstream(pTask); @@ -1047,7 +1064,8 @@ static int32_t uploadCheckpointToS3(const char* id, const char* path) { stDebug("[s3] upload checkpoint:%s", filename); } } - taosCloseDir(&pDir); + + (void) taosCloseDir(&pDir); return code; } @@ -1164,78 +1182,37 @@ int32_t deleteCheckpointFile(const char* id, const char* name) { } int32_t streamTaskSendRestoreChkptMsg(SStreamTask* pTask) { - int32_t code; - int32_t tlen = 0; - int32_t vgId = pTask->pMeta->vgId; - const char* id = pTask->id.idStr; - SCheckpointInfo* pInfo = &pTask->chkInfo; + const char* id = pTask->id.idStr; - taosThreadMutexLock(&pTask->lock); + (void) taosThreadMutexLock(&pTask->lock); if (pTask->status.sendConsensusChkptId == true) { stDebug("s-task:%s already start to consensus-checkpointId, not start again before it completed", id); - taosThreadMutexUnlock(&pTask->lock); + (void) taosThreadMutexUnlock(&pTask->lock); return TSDB_CODE_SUCCESS; } else { pTask->status.sendConsensusChkptId = true; } - taosThreadMutexUnlock(&pTask->lock); + (void) taosThreadMutexUnlock(&pTask->lock); ASSERT(pTask->pBackend == NULL); pTask->status.requireConsensusChkptId = true; -#if 0 - SRestoreCheckpointInfo req = { - .streamId = pTask->id.streamId, - .taskId = pTask->id.taskId, - .nodeId = vgId, - .checkpointId = pInfo->checkpointId, - .startTs = pTask->execInfo.created, - }; - tEncodeSize(tEncodeRestoreCheckpointInfo, &req, tlen, code); - if (code < 0) { - stError("s-task:%s vgId:%d encode stream task latest-checkpoint-id failed, code:%s", id, vgId, tstrerror(code)); - return TSDB_CODE_INVALID_MSG; - } - - void* buf = rpcMallocCont(tlen); - if (buf == NULL) { - stError("s-task:%s vgId:%d encode stream task latest-checkpoint-id msg failed, code:%s", id, vgId, - tstrerror(TSDB_CODE_OUT_OF_MEMORY)); - return TSDB_CODE_OUT_OF_MEMORY; - } - - SEncoder encoder; - tEncoderInit(&encoder, buf, tlen); - if ((code = tEncodeRestoreCheckpointInfo(&encoder, &req)) < 0) { - rpcFreeCont(buf); - stError("s-task:%s vgId:%d encode stream task latest-checkpoint-id msg failed, code:%s", id, vgId, tstrerror(code)); - return -1; - } - tEncoderClear(&encoder); - - SRpcMsg msg = {0}; - initRpcMsg(&msg, TDMT_MND_STREAM_REQ_CONSEN_CHKPT, buf, tlen); - stDebug("s-task:%s vgId:%d send latest checkpointId:%" PRId64 " to mnode to get the consensus checkpointId", id, vgId, - pInfo->checkpointId); - - tmsgSendReq(&pTask->info.mnodeEpset, &msg); -#endif return 0; } -int32_t streamTaskSendPreparedCheckpointsourceRsp(SStreamTask* pTask) { +int32_t streamTaskSendCheckpointsourceRsp(SStreamTask* pTask) { int32_t code = 0; if (pTask->info.taskLevel != TASK_LEVEL__SOURCE) { return code; } - taosThreadMutexLock(&pTask->lock); - SStreamTaskState* p = streamTaskGetStatus(pTask); - if (p->state == TASK_STATUS__CK) { + (void) taosThreadMutexLock(&pTask->lock); + SStreamTaskState p = streamTaskGetStatus(pTask); + if (p.state == TASK_STATUS__CK) { code = streamTaskSendCheckpointSourceRsp(pTask); } - taosThreadMutexUnlock(&pTask->lock); + (void) taosThreadMutexUnlock(&pTask->lock); return code; } \ No newline at end of file diff --git a/source/libs/stream/src/streamData.c b/source/libs/stream/src/streamData.c index 2994287aff..af4946cf81 100644 --- a/source/libs/stream/src/streamData.c +++ b/source/libs/stream/src/streamData.c @@ -15,13 +15,11 @@ #include "streamInt.h" -SStreamDataBlock* createStreamBlockFromDispatchMsg(const SStreamDispatchReq* pReq, int32_t blockType, int32_t srcVg) { - SStreamDataBlock* pData; - +int32_t createStreamBlockFromDispatchMsg(const SStreamDispatchReq* pReq, int32_t blockType, int32_t srcVg, SStreamDataBlock** pRes) { + SStreamDataBlock* pData = NULL; int32_t code = taosAllocateQitem(sizeof(SStreamDataBlock), DEF_QITEM, pReq->totalLen, (void**)&pData); if (code) { - terrno = code; - return NULL; + return terrno = code; } pData->type = blockType; @@ -32,7 +30,7 @@ SStreamDataBlock* createStreamBlockFromDispatchMsg(const SStreamDispatchReq* pRe SArray* pArray = taosArrayInit_s(sizeof(SSDataBlock), blockNum); if (pArray == NULL) { taosFreeQitem(pData); - return NULL; + return code; } ASSERT((pReq->blockNum == taosArrayGetSize(pReq->data)) && (pReq->blockNum == taosArrayGetSize(pReq->dataLen))); @@ -69,37 +67,36 @@ SStreamDataBlock* createStreamBlockFromDispatchMsg(const SStreamDispatchReq* pRe } pData->blocks = pArray; - return pData; + *pRes = pData; + + return code; } -SStreamDataBlock* createStreamBlockFromResults(SStreamQueueItem* pItem, SStreamTask* pTask, int64_t resultSize, - SArray* pRes) { - SStreamDataBlock* pStreamBlocks; - - int32_t code = taosAllocateQitem(sizeof(SStreamDataBlock), DEF_QITEM, resultSize, (void**)&pStreamBlocks); +int32_t createStreamBlockFromResults(SStreamQueueItem* pItem, SStreamTask* pTask, int64_t resultSize, SArray* pRes, + SStreamDataBlock** pBlock) { + int32_t code = taosAllocateQitem(sizeof(SStreamDataBlock), DEF_QITEM, resultSize, (void**)pBlock); if (code) { taosArrayClearEx(pRes, (FDelete)blockDataFreeRes); - terrno = code; - return NULL; + return terrno = code; } - pStreamBlocks->srcTaskId = pTask->id.taskId; - pStreamBlocks->type = STREAM_INPUT__DATA_BLOCK; - pStreamBlocks->blocks = pRes; + (*pBlock)->srcTaskId = pTask->id.taskId; + (*pBlock)->type = STREAM_INPUT__DATA_BLOCK; + (*pBlock)->blocks = pRes; if (pItem == NULL) { - return pStreamBlocks; + return code; } if (pItem->type == STREAM_INPUT__DATA_SUBMIT) { SStreamDataSubmit* pSubmit = (SStreamDataSubmit*)pItem; - pStreamBlocks->sourceVer = pSubmit->ver; + (*pBlock)->sourceVer = pSubmit->ver; } else if (pItem->type == STREAM_INPUT__MERGED_SUBMIT) { SStreamMergedSubmit* pMerged = (SStreamMergedSubmit*)pItem; - pStreamBlocks->sourceVer = pMerged->ver; + (*pBlock)->sourceVer = pMerged->ver; } - return pStreamBlocks; + return code; } void destroyStreamDataBlock(SStreamDataBlock* pBlock) { @@ -138,19 +135,19 @@ int32_t streamRetrieveReqToData(const SStreamRetrieveReq* pReq, SStreamDataBlock return TSDB_CODE_SUCCESS; } -SStreamDataSubmit* streamDataSubmitNew(SPackedData* pData, int32_t type) { - SStreamDataSubmit* pDataSubmit; +int32_t streamDataSubmitNew(SPackedData* pData, int32_t type, SStreamDataSubmit** pSubmit) { + SStreamDataSubmit* pDataSubmit = NULL; int32_t code = taosAllocateQitem(sizeof(SStreamDataSubmit), DEF_QITEM, pData->msgLen, (void**)&pDataSubmit); if (code) { - terrno = code; - return NULL; + return code; } pDataSubmit->ver = pData->ver; pDataSubmit->submit = *pData; pDataSubmit->type = type; - return pDataSubmit; + *pSubmit = pDataSubmit; + return TSDB_CODE_SUCCESS; } void streamDataSubmitDestroy(SStreamDataSubmit* pDataSubmit) { @@ -188,8 +185,8 @@ int32_t streamMergeSubmit(SStreamMergedSubmit* pMerged, SStreamDataSubmit* pSubm } // todo handle memory error -SStreamQueueItem* streamQueueMergeQueueItem(SStreamQueueItem* dst, SStreamQueueItem* pElem) { - terrno = 0; +int32_t streamQueueMergeQueueItem(SStreamQueueItem* dst, SStreamQueueItem* pElem, SStreamQueueItem** pRes) { + *pRes = NULL; if (dst->type == STREAM_INPUT__DATA_BLOCK && pElem->type == STREAM_INPUT__DATA_BLOCK) { SStreamDataBlock* pBlock = (SStreamDataBlock*)dst; @@ -199,7 +196,8 @@ SStreamQueueItem* streamQueueMergeQueueItem(SStreamQueueItem* dst, SStreamQueueI streamQueueItemIncSize(dst, streamQueueItemGetSize(pElem)); taosFreeQitem(pElem); - return dst; + *pRes = dst; + return TSDB_CODE_SUCCESS; } else if (dst->type == STREAM_INPUT__MERGED_SUBMIT && pElem->type == STREAM_INPUT__DATA_SUBMIT) { SStreamMergedSubmit* pMerged = (SStreamMergedSubmit*)dst; SStreamDataSubmit* pBlockSrc = (SStreamDataSubmit*)pElem; @@ -207,12 +205,13 @@ SStreamQueueItem* streamQueueMergeQueueItem(SStreamQueueItem* dst, SStreamQueueI streamQueueItemIncSize(dst, streamQueueItemGetSize(pElem)); taosFreeQitem(pElem); - return dst; + *pRes = dst; + *pRes = dst; + return TSDB_CODE_SUCCESS; } else if (dst->type == STREAM_INPUT__DATA_SUBMIT && pElem->type == STREAM_INPUT__DATA_SUBMIT) { SStreamMergedSubmit* pMerged = streamMergedSubmitNew(); if (pMerged == NULL) { - terrno = TSDB_CODE_OUT_OF_MEMORY; - return NULL; + return TSDB_CODE_OUT_OF_MEMORY; } streamQueueItemIncSize((SStreamQueueItem*)pMerged, streamQueueItemGetSize(pElem)); @@ -222,11 +221,13 @@ SStreamQueueItem* streamQueueMergeQueueItem(SStreamQueueItem* dst, SStreamQueueI taosFreeQitem(dst); taosFreeQitem(pElem); - return (SStreamQueueItem*)pMerged; + + *pRes = (SStreamQueueItem*)pMerged; + return TSDB_CODE_SUCCESS; } else { stDebug("block type:%s not merged with existed blocks list, type:%d", streamQueueItemGetTypeStr(pElem->type), dst->type); - return NULL; + return TSDB_CODE_FAILED; } } diff --git a/source/libs/stream/src/streamDispatch.c b/source/libs/stream/src/streamDispatch.c index 1959180a3e..9e4b6bc09d 100644 --- a/source/libs/stream/src/streamDispatch.c +++ b/source/libs/stream/src/streamDispatch.c @@ -799,11 +799,11 @@ static void checkpointReadyMsgSendMonitorFn(void* param, void* tmrId) { stDebug("s-task:%s in sending checkpoint-ready msg monitor timer", id); taosThreadMutexLock(&pTask->lock); - SStreamTaskState* pState = streamTaskGetStatus(pTask); - if (pState->state != TASK_STATUS__CK) { + SStreamTaskState pState = streamTaskGetStatus(pTask); + if (pState.state != TASK_STATUS__CK) { int32_t ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1); stDebug("s-task:%s vgId:%d status:%s not in checkpoint, quit from monitor checkpoint-ready send, ref:%d", id, vgId, - pState->name, ref); + pState.name, ref); taosThreadMutexUnlock(&pTask->lock); streamMetaReleaseTask(pTask->pMeta, pTask); return; @@ -1118,7 +1118,11 @@ int32_t streamAddCheckpointReadyMsg(SStreamTask* pTask, int32_t upstreamTaskId, return TSDB_CODE_SUCCESS; } - SStreamUpstreamEpInfo* pInfo = streamTaskGetUpstreamTaskEpInfo(pTask, upstreamTaskId); + SStreamUpstreamEpInfo* pInfo = NULL; + int32_t code = streamTaskGetUpstreamTaskEpInfo(pTask, upstreamTaskId, &pInfo); + if (code != TSDB_CODE_SUCCESS) { + return code; + } STaskCheckpointReadyInfo info = {0}; initCheckpointReadyInfo(&info, pInfo->nodeId, pInfo->taskId, pInfo->childId, &pInfo->epSet, checkpointId); @@ -1313,7 +1317,7 @@ int32_t streamProcessDispatchRsp(SStreamTask* pTask, SStreamDispatchRsp* pRsp, i if (delayDispatch) { taosThreadMutexLock(&pTask->lock); // we only set the dispatch msg info for current checkpoint trans - if (streamTaskGetStatus(pTask)->state == TASK_STATUS__CK && + if (streamTaskGetStatus(pTask).state == TASK_STATUS__CK && pTask->chkInfo.pActiveInfo->activeId == pMsgInfo->checkpointId) { ASSERT(pTask->chkInfo.pActiveInfo->transId == pMsgInfo->transId); stDebug("s-task:%s checkpoint-trigger msg to 0x%x rsp for checkpointId:%" PRId64 " transId:%d confirmed", @@ -1404,8 +1408,10 @@ static int32_t buildDispatchRsp(const SStreamTask* pTask, const SStreamDispatchR static int32_t streamTaskAppendInputBlocks(SStreamTask* pTask, const SStreamDispatchReq* pReq) { int8_t status = 0; - SStreamDataBlock* pBlock = createStreamBlockFromDispatchMsg(pReq, pReq->type, pReq->srcVgId); - if (pBlock == NULL) { + SStreamDataBlock* pBlock = NULL; + + int32_t code = createStreamBlockFromDispatchMsg(pReq, pReq->type, pReq->srcVgId, &pBlock); + if (code) { streamTaskInputFail(pTask); status = TASK_INPUT_STATUS__FAILED; stError("vgId:%d, s-task:%s failed to receive dispatch msg, reason: out of memory", pTask->pMeta->vgId, @@ -1415,7 +1421,7 @@ static int32_t streamTaskAppendInputBlocks(SStreamTask* pTask, const SStreamDisp pTask->status.appendTranstateBlock = true; } - int32_t code = streamTaskPutDataIntoInputQ(pTask, (SStreamQueueItem*)pBlock); + code = streamTaskPutDataIntoInputQ(pTask, (SStreamQueueItem*)pBlock); // input queue is full, upstream is blocked now status = (code == TSDB_CODE_SUCCESS) ? TASK_INPUT_STATUS__NORMAL : TASK_INPUT_STATUS__BLOCKED; } @@ -1431,8 +1437,11 @@ int32_t streamProcessDispatchMsg(SStreamTask* pTask, SStreamDispatchReq* pReq, S 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); - SStreamUpstreamEpInfo* pInfo = streamTaskGetUpstreamTaskEpInfo(pTask, pReq->upstreamTaskId); - ASSERT(pInfo != NULL); + SStreamUpstreamEpInfo* pInfo = NULL; + int32_t code = streamTaskGetUpstreamTaskEpInfo(pTask, pReq->upstreamTaskId, &pInfo); + if (code != TSDB_CODE_SUCCESS) { + return code; + } if (pMeta->role == NODE_ROLE_FOLLOWER) { stError("s-task:%s task on follower received dispatch msgs, dispatch msg rejected", id); @@ -1465,10 +1474,9 @@ int32_t streamProcessDispatchMsg(SStreamTask* pTask, SStreamDispatchReq* pReq, S { // do send response with the input status - int32_t code = buildDispatchRsp(pTask, pReq, status, &pRsp->pCont); + code = buildDispatchRsp(pTask, pReq, status, &pRsp->pCont); if (code != TSDB_CODE_SUCCESS) { stError("s-task:%s failed to build dispatch rsp, msgId:%d, code:%s", id, pReq->msgId, tstrerror(code)); - terrno = code; return code; } @@ -1477,5 +1485,5 @@ int32_t streamProcessDispatchMsg(SStreamTask* pTask, SStreamDispatchReq* pReq, S } streamTrySchedExec(pTask); - return 0; + return code; } diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index b0915640cc..238db27d60 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -27,12 +27,12 @@ static int32_t streamTransferStateDoPrepare(SStreamTask* pTask); static int32_t streamTaskExecImpl(SStreamTask* pTask, SStreamQueueItem* pItem, int64_t* totalSize, int32_t* totalBlocks); bool streamTaskShouldStop(const SStreamTask* pTask) { - SStreamTaskState* pState = streamTaskGetStatus(pTask); - return (pState->state == TASK_STATUS__STOP) || (pState->state == TASK_STATUS__DROPPING); + SStreamTaskState pState = streamTaskGetStatus(pTask); + return (pState.state == TASK_STATUS__STOP) || (pState.state == TASK_STATUS__DROPPING); } bool streamTaskShouldPause(const SStreamTask* pTask) { - return (streamTaskGetStatus(pTask)->state == TASK_STATUS__PAUSE); + return (streamTaskGetStatus(pTask).state == TASK_STATUS__PAUSE); } static int32_t doOutputResultBlockImpl(SStreamTask* pTask, SStreamDataBlock* pBlock) { @@ -67,8 +67,10 @@ static int32_t doDumpResult(SStreamTask* pTask, SStreamQueueItem* pItem, SArray* return TSDB_CODE_SUCCESS; } - SStreamDataBlock* pStreamBlocks = createStreamBlockFromResults(pItem, pTask, size, pRes); - if (pStreamBlocks == NULL) { + SStreamDataBlock* pStreamBlocks = NULL; + + int32_t code = createStreamBlockFromResults(pItem, pTask, size, pRes, &pStreamBlocks); + if (code) { 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; @@ -77,7 +79,7 @@ static int32_t doDumpResult(SStreamTask* pTask, SStreamQueueItem* pItem, SArray* stDebug("s-task:%s dump stream result data blocks, num:%d, size:%.2fMiB", pTask->id.idStr, numOfBlocks, SIZE_IN_MiB(size)); - int32_t code = doOutputResultBlockImpl(pTask, pStreamBlocks); + code = doOutputResultBlockImpl(pTask, pStreamBlocks); if (code != TSDB_CODE_SUCCESS) { // back pressure and record position return code; } @@ -187,7 +189,12 @@ int32_t streamTaskExecImpl(SStreamTask* pTask, SStreamQueueItem* pItem, int64_t* static int32_t handleSanhistoryResultBlocks(SStreamTask* pTask, SArray* pRes, int32_t size) { int32_t code = TSDB_CODE_SUCCESS; if (taosArrayGetSize(pRes) > 0) { - SStreamDataBlock* pStreamBlocks = createStreamBlockFromResults(NULL, pTask, size, pRes); + SStreamDataBlock* pStreamBlocks = NULL; + code = createStreamBlockFromResults(NULL, pTask, size, pRes, &pStreamBlocks); + if (code) { + return code; + } + code = doOutputResultBlockImpl(pTask, pStreamBlocks); if (code != TSDB_CODE_SUCCESS) { // should not have error code stError("s-task:%s dump fill-history results failed, code:%s", pTask->id.idStr, tstrerror(code)); @@ -314,7 +321,8 @@ int32_t streamTransferStateDoPrepare(SStreamTask* pTask) { SStreamMeta* pMeta = pTask->pMeta; const char* id = pTask->id.idStr; - SStreamTask* pStreamTask = streamMetaAcquireTask(pMeta, pTask->streamTaskId.streamId, pTask->streamTaskId.taskId); + SStreamTask* pStreamTask = NULL; + int32_t code = streamMetaAcquireTask(pMeta, pTask->streamTaskId.streamId, pTask->streamTaskId.taskId, &pStreamTask); if (pStreamTask == NULL) { stError( "s-task:%s failed to find related stream task:0x%x, it may have been destroyed or closed, destroy the related " @@ -336,10 +344,10 @@ int32_t streamTransferStateDoPrepare(SStreamTask* pTask) { stDebug( "s-task:%s fill-history task end, status:%s, scan wal elapsed time:%.2fSec, update related stream task:%s " "info, prepare transfer exec state", - id, streamTaskGetStatus(pTask)->name, el, pStreamTask->id.idStr); + id, streamTaskGetStatus(pTask).name, el, pStreamTask->id.idStr); } - ETaskStatus status = streamTaskGetStatus(pStreamTask)->state; + ETaskStatus status = streamTaskGetStatus(pStreamTask).state; STimeWindow* pTimeWindow = &pStreamTask->dataRange.window; // It must be halted for a source stream task, since when the related scan-history-data task start scan the history @@ -349,7 +357,7 @@ int32_t streamTransferStateDoPrepare(SStreamTask* pTask) { } else { ASSERT(status == TASK_STATUS__READY || status == TASK_STATUS__PAUSE || status == TASK_STATUS__DROPPING || status == TASK_STATUS__STOP); - int32_t code = streamTaskHandleEvent(pStreamTask->status.pSM, TASK_EVENT_HALT); + code = streamTaskHandleEvent(pStreamTask->status.pSM, TASK_EVENT_HALT); if (code != TSDB_CODE_SUCCESS) { stError("s-task:%s halt stream task:%s failed, code:%s not transfer state to stream task", id, pStreamTask->id.idStr, tstrerror(code)); @@ -363,9 +371,9 @@ int32_t streamTransferStateDoPrepare(SStreamTask* pTask) { // In case of sink tasks, no need to halt them. // In case of source tasks and agg tasks, we should HALT them, and wait for them to be idle. And then, it's safe to // start the task state transfer procedure. - SStreamTaskState* pState = streamTaskGetStatus(pStreamTask); - status = pState->state; - char* p = pState->name; + SStreamTaskState pState = streamTaskGetStatus(pStreamTask); + status = pState.state; + char* p = pState.name; if (status == TASK_STATUS__STOP || status == TASK_STATUS__DROPPING) { stError("s-task:%s failed to transfer state from fill-history task:%s, status:%s", id, pStreamTask->id.idStr, p); streamMetaReleaseTask(pMeta, pStreamTask); @@ -390,7 +398,7 @@ int32_t streamTransferStateDoPrepare(SStreamTask* pTask) { streamTaskSendCheckpointReq(pStreamTask); // 3. assign the status to the value that will be kept in disk - pStreamTask->status.taskStatus = streamTaskGetStatus(pStreamTask)->state; + pStreamTask->status.taskStatus = streamTaskGetStatus(pStreamTask).state; // 4. open the inputQ for all upstream tasks streamTaskOpenAllUpstreamInput(pStreamTask); @@ -416,7 +424,8 @@ int32_t streamTransferStatePrepare(SStreamTask* pTask) { code = streamTransferStateDoPrepare(pTask); } else { // no state transfer for sink tasks, and drop fill-history task, followed by opening inputQ of sink task. - SStreamTask* pStreamTask = streamMetaAcquireTask(pMeta, pTask->streamTaskId.streamId, pTask->streamTaskId.taskId); + SStreamTask* pStreamTask = NULL; + code = streamMetaAcquireTask(pMeta, pTask->streamTaskId.streamId, pTask->streamTaskId.taskId, &pStreamTask); if (pStreamTask != NULL) { // halt the related stream sink task code = streamTaskHandleEventAsync(pStreamTask->status.pSM, TASK_EVENT_HALT, haltCallback, NULL); @@ -590,12 +599,13 @@ void flushStateDataInExecutor(SStreamTask* pTask, SStreamQueueItem* pCheckpointB ASSERT(HAS_RELATED_FILLHISTORY_TASK(pTask)); STaskId* pHTaskId = &pTask->hTaskInfo.id; - SStreamTask* pHTask = streamMetaAcquireTask(pTask->pMeta, pHTaskId->streamId, pHTaskId->taskId); + SStreamTask* pHTask = NULL; + int32_t code = streamMetaAcquireTask(pTask->pMeta, pHTaskId->streamId, pHTaskId->taskId, &pHTask); if (pHTask != NULL) { streamTaskReleaseState(pHTask); streamTaskReloadState(pTask); stDebug("s-task:%s transfer state from fill-history task:%s, status:%s completed", id, pHTask->id.idStr, - streamTaskGetStatus(pHTask)->name); + streamTaskGetStatus(pHTask).name); streamMetaReleaseTask(pTask->pMeta, pHTask); } else { @@ -625,7 +635,7 @@ static int32_t doStreamExecTask(SStreamTask* pTask) { int32_t numOfBlocks = 0; SStreamQueueItem* pInput = NULL; - if (streamTaskShouldStop(pTask) || (streamTaskGetStatus(pTask)->state == TASK_STATUS__UNINIT)) { + if (streamTaskShouldStop(pTask) || (streamTaskGetStatus(pTask).state == TASK_STATUS__UNINIT)) { stDebug("s-task:%s stream task is stopped", id); return 0; } @@ -703,9 +713,9 @@ static int32_t doStreamExecTask(SStreamTask* pTask) { } else { // todo other thread may change the status // do nothing after sync executor state to storage backend, untill the vnode-level checkpoint is completed. taosThreadMutexLock(&pTask->lock); - SStreamTaskState* pState = streamTaskGetStatus(pTask); - if (pState->state == TASK_STATUS__CK) { - stDebug("s-task:%s checkpoint block received, set status:%s", id, pState->name); + SStreamTaskState pState = streamTaskGetStatus(pTask); + if (pState.state == TASK_STATUS__CK) { + stDebug("s-task:%s checkpoint block received, set status:%s", id, pState.name); streamTaskBuildCheckpoint(pTask); } else { // todo refactor int32_t code = 0; @@ -732,17 +742,17 @@ static int32_t doStreamExecTask(SStreamTask* pTask) { // the task may be set dropping/stopping, while it is still in the task queue, therefore, the sched-status can not // be updated by tryExec function, therefore, the schedStatus will always be the TASK_SCHED_STATUS__WAITING. bool streamTaskIsIdle(const SStreamTask* pTask) { - ETaskStatus status = streamTaskGetStatus(pTask)->state; + ETaskStatus status = streamTaskGetStatus(pTask).state; return (pTask->status.schedStatus == TASK_SCHED_STATUS__INACTIVE || status == TASK_STATUS__STOP || status == TASK_STATUS__DROPPING); } bool streamTaskReadyToRun(const SStreamTask* pTask, char** pStatus) { - SStreamTaskState* pState = streamTaskGetStatus(pTask); + SStreamTaskState pState = streamTaskGetStatus(pTask); - ETaskStatus st = pState->state; + ETaskStatus st = pState.state; if (pStatus != NULL) { - *pStatus = pState->name; + *pStatus = pState.name; } // pause & halt will still run for sink tasks. @@ -773,7 +783,7 @@ int32_t streamResumeTask(SStreamTask* pTask) { setLastExecTs(pTask, taosGetTimestampMs()); - char* p = streamTaskGetStatus(pTask)->name; + char* p = streamTaskGetStatus(pTask).name; stDebug("s-task:%s exec completed, status:%s, sched-status:%d, lastExecTs:%" PRId64, id, p, pTask->status.schedStatus, pTask->status.lastExecTs); @@ -801,7 +811,7 @@ int32_t streamExecTask(SStreamTask* pTask) { if (schedStatus == TASK_SCHED_STATUS__WAITING) { streamResumeTask(pTask); } else { - char* p = streamTaskGetStatus(pTask)->name; + char* p = streamTaskGetStatus(pTask).name; stDebug("s-task:%s already started to exec by other thread, status:%s, sched-status:%d", id, p, pTask->status.schedStatus); } diff --git a/source/libs/stream/src/streamHb.c b/source/libs/stream/src/streamHb.c index 16cb23de10..4aaaed615c 100644 --- a/source/libs/stream/src/streamHb.c +++ b/source/libs/stream/src/streamHb.c @@ -54,7 +54,7 @@ static bool existInHbMsg(SStreamHbMsg* pMsg, SDownstreamTaskEpset* pTaskEpset) { static void addUpdateNodeIntoHbMsg(SStreamTask* pTask, SStreamHbMsg* pMsg) { SStreamMeta* pMeta = pTask->pMeta; - taosThreadMutexLock(&pTask->lock); + (void) taosThreadMutexLock(&pTask->lock); int32_t num = taosArrayGetSize(pTask->outputInfo.pNodeEpsetUpdateList); for (int j = 0; j < num; ++j) { @@ -62,14 +62,18 @@ static void addUpdateNodeIntoHbMsg(SStreamTask* pTask, SStreamHbMsg* pMsg) { bool exist = existInHbMsg(pMsg, pTaskEpset); if (!exist) { - taosArrayPush(pMsg->pUpdateNodes, &pTaskEpset->nodeId); + void* p = taosArrayPush(pMsg->pUpdateNodes, &pTaskEpset->nodeId); + if (p == NULL) { + stError("failed to set the updateNode info in hbMsg, vgId:%d", pMeta->vgId); + } + stDebug("vgId:%d nodeId:%d added into hbMsg update list, total:%d", pMeta->vgId, pTaskEpset->nodeId, (int32_t)taosArrayGetSize(pMsg->pUpdateNodes)); } } taosArrayClear(pTask->outputInfo.pNodeEpsetUpdateList); - taosThreadMutexUnlock(&pTask->lock); + (void) taosThreadMutexUnlock(&pTask->lock); } static int32_t doSendHbMsgInfo(SStreamHbMsg* pMsg, SStreamMeta* pMeta, SEpSet* pEpset) { @@ -101,9 +105,7 @@ static int32_t doSendHbMsgInfo(SStreamHbMsg* pMsg, SStreamMeta* pMeta, SEpSet* p SRpcMsg msg = {0}; initRpcMsg(&msg, TDMT_MND_STREAM_HEARTBEAT, buf, tlen); - tmsgSendReq(pEpset, &msg); - - return TSDB_CODE_SUCCESS; + return tmsgSendReq(pEpset, &msg); } // NOTE: this task should be executed within the SStreamMeta lock region. @@ -112,6 +114,7 @@ int32_t streamMetaSendHbHelper(SStreamMeta* pMeta) { bool hasMnodeEpset = false; int32_t numOfTasks = streamMetaGetNumOfTasks(pMeta); SMetaHbInfo* pInfo = pMeta->pHbInfo; + int32_t code = 0; // not recv the hb msg rsp yet, send current hb msg again if (pInfo->msgSendTs > 0) { @@ -135,8 +138,7 @@ int32_t streamMetaSendHbHelper(SStreamMeta* pMeta) { } pInfo->msgSendTs = taosGetTimestampMs(); - doSendHbMsgInfo(&pInfo->hbMsg, pMeta, &epset); - return TSDB_CODE_SUCCESS; + return doSendHbMsgInfo(&pInfo->hbMsg, pMeta, &epset); } SStreamHbMsg* pMsg = &pInfo->hbMsg; @@ -168,9 +170,9 @@ int32_t streamMetaSendHbHelper(SStreamMeta* pMeta) { continue; } - taosThreadMutexLock(&(*pTask)->lock); + (void) taosThreadMutexLock(&(*pTask)->lock); STaskStatusEntry entry = streamTaskGetStatusEntry(*pTask); - taosThreadMutexUnlock(&(*pTask)->lock); + (void) taosThreadMutexUnlock(&(*pTask)->lock); entry.inputRate = entry.inputQUsed * 100.0 / (2 * STREAM_TASK_QUEUE_CAPACITY_IN_SIZE); if ((*pTask)->info.taskLevel == TASK_LEVEL__SINK) { @@ -188,9 +190,9 @@ int32_t streamMetaSendHbHelper(SStreamMeta* pMeta) { stInfo("s-task:%s set kill checkpoint trans in hbMsg, transId:%d, clear the active checkpointInfo", (*pTask)->id.idStr, p->transId); - taosThreadMutexLock(&(*pTask)->lock); + (void) taosThreadMutexLock(&(*pTask)->lock); streamTaskClearCheckInfo((*pTask), true); - taosThreadMutexUnlock(&(*pTask)->lock); + (void) taosThreadMutexUnlock(&(*pTask)->lock); } } @@ -210,7 +212,11 @@ int32_t streamMetaSendHbHelper(SStreamMeta* pMeta) { } addUpdateNodeIntoHbMsg(*pTask, pMsg); - taosArrayPush(pMsg->pTaskStatus, &entry); + p = taosArrayPush(pMsg->pTaskStatus, &entry); + if (p == NULL) { + stError("failed to add taskInfo:0x%x in hbMsg, vgId:%d", (*pTask)->id.taskId, pMeta->vgId); + } + if (!hasMnodeEpset) { epsetAssign(&epset, &(*pTask)->info.mnodeEpset); hasMnodeEpset = true; @@ -221,18 +227,19 @@ int32_t streamMetaSendHbHelper(SStreamMeta* pMeta) { if (hasMnodeEpset) { pInfo->msgSendTs = taosGetTimestampMs(); - doSendHbMsgInfo(pMsg, pMeta, &epset); + code = doSendHbMsgInfo(pMsg, pMeta, &epset); } else { stDebug("vgId:%d no tasks or no mnd epset, not send stream hb to mnode", pMeta->vgId); tCleanupStreamHbMsg(&pInfo->hbMsg); pInfo->msgSendTs = -1; } - return TSDB_CODE_SUCCESS; + return code; } void streamMetaHbToMnode(void* param, void* tmrId) { int64_t rid = *(int64_t*)param; + int32_t code = 0; SStreamMeta* pMeta = taosAcquireRef(streamMetaId, rid); if (pMeta == NULL) { @@ -243,15 +250,25 @@ void streamMetaHbToMnode(void* param, void* tmrId) { // need to stop, stop now if (pMeta->pHbInfo->stopFlag == STREAM_META_WILL_STOP) { // todo refactor: not need this now, use closeFlag in Meta pMeta->pHbInfo->stopFlag = STREAM_META_OK_TO_STOP; - stDebug("vgId:%d jump out of meta timer", pMeta->vgId); - taosReleaseRef(streamMetaId, rid); + code = taosReleaseRef(streamMetaId, rid); + if (code == TSDB_CODE_SUCCESS) { + stDebug("vgId:%d jump out of meta timer", pMeta->vgId); + } else { + stError("vgId:%d jump out of meta timer, failed to release the meta rid:%" PRId64, pMeta->vgId, rid); + } return; } // not leader not send msg if (pMeta->role != NODE_ROLE_LEADER) { - stInfo("vgId:%d role:%d not leader not send hb to mnode", pMeta->vgId, pMeta->role); - taosReleaseRef(streamMetaId, rid); + code = taosReleaseRef(streamMetaId, rid); + if (code == TSDB_CODE_SUCCESS) { + stInfo("vgId:%d role:%d not leader not send hb to mnode", pMeta->vgId, pMeta->role); + } else { + stError("vgId:%d role:%d not leader not send hb to mnodefailed to release the meta rid:%" PRId64, pMeta->vgId, + pMeta->role, rid); + } + pMeta->pHbInfo->hbStart = 0; return; } @@ -262,24 +279,37 @@ void streamMetaHbToMnode(void* param, void* tmrId) { } if (!waitForEnoughDuration(pMeta->pHbInfo)) { - taosTmrReset(streamMetaHbToMnode, META_HB_CHECK_INTERVAL, param, streamTimer, &pMeta->pHbInfo->hbTmr); - taosReleaseRef(streamMetaId, rid); + streamTmrReset(streamMetaHbToMnode, META_HB_CHECK_INTERVAL, param, streamTimer, &pMeta->pHbInfo->hbTmr, pMeta->vgId, + "meta-hb-tmr"); + + code = taosReleaseRef(streamMetaId, rid); + if (code) { + stError("vgId:%d in meta timer, failed to release the meta rid:%" PRId64, pMeta->vgId, rid); + } return; } streamMetaRLock(pMeta); - streamMetaSendHbHelper(pMeta); - streamMetaRUnLock(pMeta); + code = streamMetaSendHbHelper(pMeta); + if (code) { + stError("vgId:%d failed to send hmMsg to mnode, try again in 5s, code:%s", pMeta->vgId, strerror(code)); + } - taosTmrReset(streamMetaHbToMnode, META_HB_CHECK_INTERVAL, param, streamTimer, &pMeta->pHbInfo->hbTmr); - taosReleaseRef(streamMetaId, rid); + streamMetaRUnLock(pMeta); + streamTmrReset(streamMetaHbToMnode, META_HB_CHECK_INTERVAL, param, streamTimer, &pMeta->pHbInfo->hbTmr, pMeta->vgId, + "meta-hb-tmr"); + + code = taosReleaseRef(streamMetaId, rid); + if (code) { + stError("vgId:%d in meta timer, failed to release the meta rid:%" PRId64, pMeta->vgId, rid); + } } -SMetaHbInfo* createMetaHbInfo(int64_t* pRid) { +int32_t createMetaHbInfo(int64_t* pRid, SMetaHbInfo** pRes) { + *pRes = NULL; SMetaHbInfo* pInfo = taosMemoryCalloc(1, sizeof(SMetaHbInfo)); if (pInfo == NULL) { - terrno = TSDB_CODE_OUT_OF_MEMORY; - return pInfo; + return TSDB_CODE_OUT_OF_MEMORY; } pInfo->hbTmr = taosTmrStart(streamMetaHbToMnode, META_HB_CHECK_INTERVAL, pRid, streamTimer); @@ -287,22 +317,22 @@ SMetaHbInfo* createMetaHbInfo(int64_t* pRid) { pInfo->stopFlag = 0; pInfo->msgSendTs = -1; pInfo->hbCount = 0; - return pInfo; + + *pRes = pInfo; + return TSDB_CODE_SUCCESS; } -void* destroyMetaHbInfo(SMetaHbInfo* pInfo) { +void destroyMetaHbInfo(SMetaHbInfo* pInfo) { if (pInfo != NULL) { tCleanupStreamHbMsg(&pInfo->hbMsg); if (pInfo->hbTmr != NULL) { - taosTmrStop(pInfo->hbTmr); + (void) taosTmrStop(pInfo->hbTmr); pInfo->hbTmr = NULL; } taosMemoryFree(pInfo); } - - return NULL; } void streamMetaWaitForHbTmrQuit(SStreamMeta* pMeta) { diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 3ef5a4ac10..b48c90be16 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -292,32 +292,39 @@ void streamMetaRemoveDB(void* arg, char* key) { taosThreadMutexUnlock(&pMeta->backendMutex); } -SStreamMeta* streamMetaOpen(const char* path, void* ahandle, FTaskBuild buildTaskFn, FTaskExpand expandTaskFn, - int32_t vgId, int64_t stage, startComplete_fn_t fn) { +int32_t streamMetaOpen(const char* path, void* ahandle, FTaskBuild buildTaskFn, FTaskExpand expandTaskFn, int32_t vgId, + int64_t stage, startComplete_fn_t fn, SStreamMeta** p) { + *p = NULL; + int32_t code = 0; + SStreamMeta* pMeta = taosMemoryCalloc(1, sizeof(SStreamMeta)); if (pMeta == NULL) { - terrno = TSDB_CODE_OUT_OF_MEMORY; stError("vgId:%d failed to prepare stream meta, alloc size:%" PRIzu ", out of memory", vgId, sizeof(SStreamMeta)); - return NULL; + return TSDB_CODE_OUT_OF_MEMORY; } int32_t len = strlen(path) + 64; char* tpath = taosMemoryCalloc(1, len); + if (tpath == NULL) { + code = TSDB_CODE_OUT_OF_MEMORY; + goto _err; + } sprintf(tpath, "%s%s%s", path, TD_DIRSEP, "stream"); pMeta->path = tpath; - if (streamMetaOpenTdb(pMeta) < 0) { + code = streamMetaOpenTdb(pMeta); + if (code != TSDB_CODE_SUCCESS) { goto _err; } - if (streamMetaMayCvtDbFormat(pMeta) < 0) { + if ((code = streamMetaMayCvtDbFormat(pMeta)) < 0) { stError("vgId:%d convert sub info format failed, open stream meta failed, reason: %s", pMeta->vgId, tstrerror(terrno)); goto _err; } - if (streamMetaBegin(pMeta) < 0) { + if ((code = streamMetaBegin(pMeta) < 0)) { stError("vgId:%d begin trans for stream meta failed", pMeta->vgId); goto _err; } @@ -325,32 +332,32 @@ SStreamMeta* streamMetaOpen(const char* path, void* ahandle, FTaskBuild buildTas _hash_fn_t fp = taosGetDefaultHashFunction(TSDB_DATA_TYPE_VARCHAR); pMeta->pTasksMap = taosHashInit(64, fp, true, HASH_NO_LOCK); if (pMeta->pTasksMap == NULL) { - terrno = TSDB_CODE_OUT_OF_MEMORY; + code = TSDB_CODE_OUT_OF_MEMORY; goto _err; } pMeta->updateInfo.pTasks = taosHashInit(64, fp, false, HASH_NO_LOCK); if (pMeta->updateInfo.pTasks == NULL) { - terrno = TSDB_CODE_OUT_OF_MEMORY; + code = TSDB_CODE_OUT_OF_MEMORY; goto _err; } pMeta->startInfo.pReadyTaskSet = taosHashInit(64, fp, false, HASH_NO_LOCK); if (pMeta->startInfo.pReadyTaskSet == NULL) { - terrno = TSDB_CODE_OUT_OF_MEMORY; + code = TSDB_CODE_OUT_OF_MEMORY; goto _err; } pMeta->startInfo.pFailedTaskSet = taosHashInit(4, fp, false, HASH_NO_LOCK); if (pMeta->startInfo.pFailedTaskSet == NULL) { - terrno = TSDB_CODE_OUT_OF_MEMORY; + code = TSDB_CODE_OUT_OF_MEMORY; goto _err; } // task list pMeta->pTaskList = taosArrayInit(4, sizeof(SStreamTaskId)); if (pMeta->pTaskList == NULL) { - terrno = TSDB_CODE_OUT_OF_MEMORY; + code = TSDB_CODE_OUT_OF_MEMORY; goto _err; } @@ -376,10 +383,16 @@ SStreamMeta* streamMetaOpen(const char* path, void* ahandle, FTaskBuild buildTas // set the attribute when running on Linux OS TdThreadRwlockAttr attr; - taosThreadRwlockAttrInit(&attr); + code = taosThreadRwlockAttrInit(&attr); + if (code != TSDB_CODE_SUCCESS) { + goto _err; + } #ifdef LINUX - pthread_rwlockattr_setkind_np(&attr, PTHREAD_RWLOCK_PREFER_WRITER_NONRECURSIVE_NP); + code = pthread_rwlockattr_setkind_np(&attr, PTHREAD_RWLOCK_PREFER_WRITER_NONRECURSIVE_NP); + if (code != TSDB_CODE_SUCCESS) { + goto _err; + } #endif taosThreadRwlockInit(&pMeta->lock, &attr); @@ -389,9 +402,8 @@ SStreamMeta* streamMetaOpen(const char* path, void* ahandle, FTaskBuild buildTas memcpy(pRid, &pMeta->rid, sizeof(pMeta->rid)); metaRefMgtAdd(pMeta->vgId, pRid); - pMeta->pHbInfo = createMetaHbInfo(pRid); - if (pMeta->pHbInfo == NULL) { - terrno = TSDB_CODE_OUT_OF_MEMORY; + code = createMetaHbInfo(pRid, &pMeta->pHbInfo); + if (code != TSDB_CODE_SUCCESS) { goto _err; } @@ -403,7 +415,8 @@ SStreamMeta* streamMetaOpen(const char* path, void* ahandle, FTaskBuild buildTas } taosThreadMutexInit(&pMeta->backendMutex, NULL); - return pMeta; + *p = pMeta; + return code; _err: taosMemoryFree(pMeta->path); @@ -420,7 +433,7 @@ _err: taosMemoryFree(pMeta); stError("failed to open stream meta, reason:%s", tstrerror(terrno)); - return NULL; + return code; } // todo refactor: the lock shoud be restricted in one function @@ -524,7 +537,8 @@ void streamMetaCloseImpl(void* arg) { taosHashCleanup(pMeta->startInfo.pReadyTaskSet); taosHashCleanup(pMeta->startInfo.pFailedTaskSet); - pMeta->pHbInfo = destroyMetaHbInfo(pMeta->pHbInfo); + destroyMetaHbInfo(pMeta->pHbInfo); + pMeta->pHbInfo = NULL; taosMemoryFree(pMeta->path); taosThreadMutexDestroy(&pMeta->backendMutex); @@ -630,29 +644,30 @@ int32_t streamMetaGetNumOfTasks(SStreamMeta* pMeta) { return (int32_t)size; } -SStreamTask* streamMetaAcquireTaskNoLock(SStreamMeta* pMeta, int64_t streamId, int32_t taskId) { +int32_t streamMetaAcquireTaskNoLock(SStreamMeta* pMeta, int64_t streamId, int32_t taskId, SStreamTask** pTask) { STaskId id = {.streamId = streamId, .taskId = taskId}; SStreamTask** ppTask = (SStreamTask**)taosHashGet(pMeta->pTasksMap, &id, sizeof(id)); if (ppTask == NULL || streamTaskShouldStop(*ppTask)) { - return NULL; + *pTask = NULL; + return TSDB_CODE_FAILED; } int32_t ref = atomic_add_fetch_32(&(*ppTask)->refCnt, 1); stTrace("s-task:%s acquire task, ref:%d", (*ppTask)->id.idStr, ref); - return *ppTask; + *pTask = *ppTask; + return TSDB_CODE_SUCCESS; } -SStreamTask* streamMetaAcquireTask(SStreamMeta* pMeta, int64_t streamId, int32_t taskId) { +int32_t streamMetaAcquireTask(SStreamMeta* pMeta, int64_t streamId, int32_t taskId, SStreamTask** pTask) { streamMetaRLock(pMeta); - SStreamTask* p = streamMetaAcquireTaskNoLock(pMeta, streamId, taskId); + int32_t code = streamMetaAcquireTaskNoLock(pMeta, streamId, taskId, pTask); streamMetaRUnLock(pMeta); - return p; + return code; } -SStreamTask* streamMetaAcquireOneTask(SStreamTask* pTask) { +void streamMetaAcquireOneTask(SStreamTask* pTask) { int32_t ref = atomic_add_fetch_32(&pTask->refCnt, 1); stTrace("s-task:%s acquire task, ref:%d", pTask->id.idStr, ref); - return pTask; } void streamMetaReleaseTask(SStreamMeta* UNUSED_PARAM(pMeta), SStreamTask* pTask) { @@ -719,20 +734,19 @@ int32_t streamMetaUnregisterTask(SStreamMeta* pMeta, int64_t streamId, int32_t t stDebug("s-task:0x%x vgId:%d set task status:dropping and start to unregister it", taskId, pMeta->vgId); while (1) { - streamMetaRLock(pMeta); + int32_t timerActive = 0; + streamMetaRLock(pMeta); ppTask = (SStreamTask**)taosHashGet(pMeta->pTasksMap, &id, sizeof(id)); if (ppTask) { - if ((*ppTask)->status.timerActive == 0) { - streamMetaRUnLock(pMeta); - break; - } + timerActive = (*ppTask)->status.timerActive; + } + streamMetaRUnLock(pMeta); - taosMsleep(10); - stDebug("s-task:%s wait for quit from timer", (*ppTask)->id.idStr); - streamMetaRUnLock(pMeta); + if (timerActive > 0) { + taosMsleep(100); + stDebug("s-task:0x%" PRIx64 " wait for quit from timer", id.taskId); } else { - streamMetaRUnLock(pMeta); break; } } @@ -1090,13 +1104,19 @@ int32_t streamMetaAsyncExec(SStreamMeta* pMeta, __stream_async_exec_fn_t fn, voi return taosScheduleTask(pMeta->qHandle, &schedMsg); } -SArray* streamMetaSendMsgBeforeCloseTasks(SStreamMeta* pMeta) { +int32_t streamMetaSendMsgBeforeCloseTasks(SStreamMeta* pMeta, SArray** pList) { + *pList = NULL; + int32_t code = 0; SArray* pTaskList = taosArrayDup(pMeta->pTaskList, NULL); + if (pTaskList == NULL) { + stError("failed to generate the task list during send hbMsg to mnode, vgId:%d, code: out of memory", pMeta->vgId); + return TSDB_CODE_OUT_OF_MEMORY; + } bool sendMsg = pMeta->sendMsgBeforeClosing; if (!sendMsg) { stDebug("vgId:%d no need to send msg to mnode before closing tasks", pMeta->vgId); - return pTaskList; + return TSDB_CODE_SUCCESS; } stDebug("vgId:%d send msg to mnode before closing all tasks", pMeta->vgId); @@ -1105,27 +1125,29 @@ SArray* streamMetaSendMsgBeforeCloseTasks(SStreamMeta* pMeta) { int32_t numOfTasks = taosArrayGetSize(pTaskList); for (int32_t i = 0; i < numOfTasks; ++i) { SStreamTaskId* pTaskId = taosArrayGet(pTaskList, i); - SStreamTask* pTask = streamMetaAcquireTaskNoLock(pMeta, pTaskId->streamId, pTaskId->taskId); - if (pTask == NULL) { + SStreamTask* pTask = NULL; + + code = streamMetaAcquireTaskNoLock(pMeta, pTaskId->streamId, pTaskId->taskId, &pTask); + if (code != TSDB_CODE_SUCCESS) { // this error is ignored continue; } taosThreadMutexLock(&pTask->lock); - SStreamTaskState* pState = streamTaskGetStatus(pTask); - if (pState->state == TASK_STATUS__CK) { + SStreamTaskState pState = streamTaskGetStatus(pTask); + if (pState.state == TASK_STATUS__CK) { streamTaskSetFailedCheckpointId(pTask); } else { - stDebug("s-task:%s status:%s not reset the checkpoint", pTask->id.idStr, pState->name); + stDebug("s-task:%s status:%s not reset the checkpoint", pTask->id.idStr, pState.name); } taosThreadMutexUnlock(&pTask->lock); streamMetaReleaseTask(pMeta, pTask); } - streamMetaSendHbHelper(pMeta); + code = streamMetaSendHbHelper(pMeta); pMeta->sendMsgBeforeClosing = false; - return pTaskList; + return code; } void streamMetaUpdateStageRole(SStreamMeta* pMeta, int64_t stage, bool isLeader) { @@ -1201,7 +1223,8 @@ int32_t streamMetaStartAllTasks(SStreamMeta* pMeta) { // initialization, when the operation of check downstream tasks status is executed far quickly. for (int32_t i = 0; i < numOfTasks; ++i) { SStreamTaskId* pTaskId = taosArrayGet(pTaskList, i); - SStreamTask* pTask = streamMetaAcquireTask(pMeta, pTaskId->streamId, pTaskId->taskId); + SStreamTask* pTask = NULL; + code = streamMetaAcquireTask(pMeta, pTaskId->streamId, pTaskId->taskId, &pTask); if (pTask == NULL) { stError("vgId:%d failed to acquire task:0x%x during start tasks", pMeta->vgId, pTaskId->taskId); streamMetaAddFailedTask(pMeta, pTaskId->streamId, pTaskId->taskId); @@ -1223,7 +1246,8 @@ int32_t streamMetaStartAllTasks(SStreamMeta* pMeta) { for (int32_t i = 0; i < numOfTasks; ++i) { SStreamTaskId* pTaskId = taosArrayGet(pTaskList, i); - SStreamTask* pTask = streamMetaAcquireTask(pMeta, pTaskId->streamId, pTaskId->taskId); + SStreamTask* pTask = NULL; + code = streamMetaAcquireTask(pMeta, pTaskId->streamId, pTaskId->taskId, &pTask); if (pTask == NULL) { stError("vgId:%d failed to acquire task:0x%x during start tasks", pMeta->vgId, pTaskId->taskId); streamMetaAddFailedTask(pMeta, pTaskId->streamId, pTaskId->taskId); @@ -1293,13 +1317,20 @@ int32_t streamMetaStopAllTasks(SStreamMeta* pMeta) { int64_t st = taosGetTimestampMs(); // send hb msg to mnode before closing all tasks. - SArray* pTaskList = streamMetaSendMsgBeforeCloseTasks(pMeta); + SArray* pTaskList = NULL; + int32_t code = streamMetaSendMsgBeforeCloseTasks(pMeta, &pTaskList); + if (code != TSDB_CODE_SUCCESS) { + return code; + } + int32_t numOfTasks = taosArrayGetSize(pTaskList); for (int32_t i = 0; i < numOfTasks; ++i) { SStreamTaskId* pTaskId = taosArrayGet(pTaskList, i); - SStreamTask* pTask = streamMetaAcquireTaskNoLock(pMeta, pTaskId->streamId, pTaskId->taskId); - if (pTask == NULL) { + SStreamTask* pTask = NULL; + + code = streamMetaAcquireTaskNoLock(pMeta, pTaskId->streamId, pTaskId->taskId, &pTask); + if (code != TSDB_CODE_SUCCESS) { continue; } @@ -1339,7 +1370,8 @@ int32_t streamMetaStartOneTask(SStreamMeta* pMeta, int64_t streamId, int32_t tas int32_t vgId = pMeta->vgId; stInfo("vgId:%d start task:0x%x by checking it's downstream status", vgId, taskId); - SStreamTask* pTask = streamMetaAcquireTask(pMeta, streamId, taskId); + SStreamTask* pTask = NULL; + code = streamMetaAcquireTask(pMeta, streamId, taskId, &pTask); if (pTask == NULL) { stError("vgId:%d failed to acquire task:0x%x when starting task", pMeta->vgId, taskId); streamMetaAddFailedTask(pMeta, streamId, taskId); diff --git a/source/libs/stream/src/streamMsg.c b/source/libs/stream/src/streamMsg.c index 1cc48f02b6..b08280f9ed 100644 --- a/source/libs/stream/src/streamMsg.c +++ b/source/libs/stream/src/streamMsg.c @@ -99,6 +99,8 @@ int32_t tEncodeStreamTaskUpdateMsg(SEncoder* pEncoder, const SStreamTaskNodeUpda } int32_t tDecodeStreamTaskUpdateMsg(SDecoder* pDecoder, SStreamTaskNodeUpdateMsg* pMsg) { + int32_t code = 0; + if (tStartDecode(pDecoder) < 0) return -1; if (tDecodeI64(pDecoder, &pMsg->streamId) < 0) return -1; if (tDecodeI32(pDecoder, &pMsg->taskId) < 0) return -1; @@ -111,13 +113,17 @@ int32_t tDecodeStreamTaskUpdateMsg(SDecoder* pDecoder, SStreamTaskNodeUpdateMsg* if (tDecodeI32(pDecoder, &info.nodeId) < 0) return -1; if (tDecodeSEpSet(pDecoder, &info.prevEp) < 0) return -1; if (tDecodeSEpSet(pDecoder, &info.newEp) < 0) return -1; - taosArrayPush(pMsg->pNodeList, &info); + + void* p = taosArrayPush(pMsg->pNodeList, &info); + if (p == NULL) { + code = TSDB_CODE_OUT_OF_MEMORY; + } } if (tDecodeI32(pDecoder, &pMsg->transId) < 0) return -1; tEndDecode(pDecoder); - return 0; + return code; } int32_t tEncodeStreamTaskCheckReq(SEncoder* pEncoder, const SStreamTaskCheckReq* pReq) { @@ -257,8 +263,18 @@ int32_t tDecodeStreamDispatchReq(SDecoder* pDecoder, SStreamDispatchReq* pReq) { if (tDecodeI32(pDecoder, &len1) < 0) return -1; if (tDecodeBinaryAlloc(pDecoder, &data, &len2) < 0) return -1; ASSERT(len1 == len2); - taosArrayPush(pReq->dataLen, &len1); - taosArrayPush(pReq->data, &data); + + void* p = taosArrayPush(pReq->dataLen, &len1); + if (p == NULL) { + tEndDecode(pDecoder); + return TSDB_CODE_OUT_OF_MEMORY; + } + + p = taosArrayPush(pReq->data, &data); + if (p == NULL) { + tEndDecode(pDecoder); + return TSDB_CODE_OUT_OF_MEMORY; + } } tEndDecode(pDecoder); @@ -371,6 +387,8 @@ int32_t tEncodeStreamHbMsg(SEncoder* pEncoder, const SStreamHbMsg* pReq) { } int32_t tDecodeStreamHbMsg(SDecoder* pDecoder, SStreamHbMsg* pReq) { + int32_t code = 0; + if (tStartDecode(pDecoder) < 0) return -1; if (tDecodeI32(pDecoder, &pReq->vgId) < 0) return -1; if (tDecodeI32(pDecoder, &pReq->numOfTasks) < 0) return -1; @@ -413,7 +431,11 @@ int32_t tDecodeStreamHbMsg(SDecoder* pDecoder, SStreamHbMsg* pReq) { if (tDecodeI64(pDecoder, &entry.hTaskId) < 0) return -1; entry.id.taskId = taskId; - taosArrayPush(pReq->pTaskStatus, &entry); + void* p = taosArrayPush(pReq->pTaskStatus, &entry); + if (p == NULL) { + code = TSDB_CODE_OUT_OF_MEMORY; + goto _err; + } } int32_t numOfVgs = 0; @@ -424,12 +446,20 @@ int32_t tDecodeStreamHbMsg(SDecoder* pDecoder, SStreamHbMsg* pReq) { for (int j = 0; j < numOfVgs; ++j) { int32_t vgId = 0; if (tDecodeI32(pDecoder, &vgId) < 0) return -1; - taosArrayPush(pReq->pUpdateNodes, &vgId); + void* p = taosArrayPush(pReq->pUpdateNodes, &vgId); + if (p == NULL) { + code = TSDB_CODE_OUT_OF_MEMORY; + goto _err; + } } if (tDecodeI32(pDecoder, &pReq->msgId) < 0) return -1; tEndDecode(pDecoder); return 0; + + _err: + tEndDecode(pDecoder); + return code; } void tCleanupStreamHbMsg(SStreamHbMsg* pMsg) { @@ -572,7 +602,11 @@ int32_t tDecodeStreamTask(SDecoder* pDecoder, SStreamTask* pTask) { taosMemoryFreeClear(pInfo); return -1; } - taosArrayPush(pTask->upstreamInfo.pList, &pInfo); + void* p = taosArrayPush(pTask->upstreamInfo.pList, &pInfo); + if (p == NULL) { + tEndDecode(pDecoder); + return -1; + } } if (pTask->info.taskLevel != TASK_LEVEL__SINK) { diff --git a/source/libs/stream/src/streamQueue.c b/source/libs/stream/src/streamQueue.c index 7bc50417bd..b8cdcd4cf5 100644 --- a/source/libs/stream/src/streamQueue.c +++ b/source/libs/stream/src/streamQueue.c @@ -41,34 +41,34 @@ static void streamQueueCleanup(SStreamQueue* pQueue) { static void* streamQueueCurItem(SStreamQueue* queue) { return queue->qItem; } -SStreamQueue* streamQueueOpen(int64_t cap) { - int32_t code; +int32_t streamQueueOpen(int64_t cap, SStreamQueue** pQ) { + *pQ = NULL; + int32_t code = 0; SStreamQueue* pQueue = taosMemoryCalloc(1, sizeof(SStreamQueue)); if (pQueue == NULL) { - terrno = TSDB_CODE_OUT_OF_MEMORY; - return NULL; + return TSDB_CODE_OUT_OF_MEMORY; } code = taosOpenQueue(&pQueue->pQueue); if (code) { taosMemoryFreeClear(pQueue); - terrno = code; - return NULL; + return code; } code = taosAllocateQall(&pQueue->qall); if (code) { taosCloseQueue(pQueue->pQueue); taosMemoryFree(pQueue); - terrno = code; - return NULL; + return code; } pQueue->status = STREAM_QUEUE__SUCESS; taosSetQueueCapacity(pQueue->pQueue, cap); taosSetQueueMemoryCapacity(pQueue->pQueue, cap * 1024); - return pQueue; + + *pQ = pQueue; + return code; } void streamQueueClose(SStreamQueue* pQueue, int32_t taskId) { @@ -227,12 +227,11 @@ EExtractDataCode streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueIte *pInput = qItem; } else { // merge current block failed, let's handle the already merged blocks. - void* newRet = streamQueueMergeQueueItem(*pInput, qItem); - if (newRet == NULL) { - if (terrno != 0) { - stError("s-task:%s failed to merge blocks from inputQ, numOfBlocks:%d, code:%s", id, *numOfBlocks, - tstrerror(terrno)); - } + void* newRet = NULL; + int32_t code = streamQueueMergeQueueItem(*pInput, qItem, (SStreamQueueItem**)&newRet); + if (code != TSDB_CODE_SUCCESS) { + stError("s-task:%s failed to merge blocks from inputQ, numOfBlocks:%d, code:%s", id, *numOfBlocks, + tstrerror(terrno)); *blockSize = streamQueueItemGetSize(*pInput); if (taskLevel == TASK_LEVEL__SINK) { diff --git a/source/libs/stream/src/streamSched.c b/source/libs/stream/src/streamSched.c index f33c2c7b68..f11b135168 100644 --- a/source/libs/stream/src/streamSched.c +++ b/source/libs/stream/src/streamSched.c @@ -94,13 +94,13 @@ int32_t streamTaskResumeInFuture(SStreamTask* pTask) { void streamTaskResumeHelper(void* param, void* tmrId) { SStreamTask* pTask = (SStreamTask*)param; SStreamTaskId* pId = &pTask->id; - SStreamTaskState* p = streamTaskGetStatus(pTask); + SStreamTaskState p = streamTaskGetStatus(pTask); - if (p->state == TASK_STATUS__DROPPING || p->state == TASK_STATUS__STOP) { + if (p.state == TASK_STATUS__DROPPING || p.state == TASK_STATUS__STOP) { streamTaskSetSchedStatusInactive(pTask); int32_t ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1); - stDebug("s-task:%s status:%s not resume task, ref:%d", pId->idStr, p->name, ref); + stDebug("s-task:%s status:%s not resume task, ref:%d", pId->idStr, p.name, ref); streamMetaReleaseTask(pTask->pMeta, pTask); return; @@ -130,7 +130,7 @@ void streamTaskSchedHelper(void* param, void* tmrId) { return; } - if (streamTaskGetStatus(pTask)->state == TASK_STATUS__CK) { + if (streamTaskGetStatus(pTask).state == TASK_STATUS__CK) { stDebug("s-task:%s in checkpoint procedure, not retrieve result, next:%dms", id, nextTrigger); } else { if (status == TASK_TRIGGER_STATUS__ACTIVE) { diff --git a/source/libs/stream/src/streamStartHistory.c b/source/libs/stream/src/streamStartHistory.c index 30bccd6479..3c7ad2639a 100644 --- a/source/libs/stream/src/streamStartHistory.c +++ b/source/libs/stream/src/streamStartHistory.c @@ -44,13 +44,13 @@ static void doRetryLaunchFillHistoryTask(SStreamTask* pTask, SLaunc static void notRetryLaunchFillHistoryTask(SStreamTask* pTask, SLaunchHTaskInfo* pInfo, int64_t now); static int32_t streamTaskSetReady(SStreamTask* pTask) { - int32_t numOfDowns = streamTaskGetNumOfDownstream(pTask); - SStreamTaskState* p = streamTaskGetStatus(pTask); + int32_t numOfDowns = streamTaskGetNumOfDownstream(pTask); + SStreamTaskState p = streamTaskGetStatus(pTask); - if ((p->state == TASK_STATUS__SCAN_HISTORY) && pTask->info.taskLevel != TASK_LEVEL__SOURCE) { + if ((p.state == TASK_STATUS__SCAN_HISTORY) && pTask->info.taskLevel != TASK_LEVEL__SOURCE) { int32_t numOfUps = taosArrayGetSize(pTask->upstreamInfo.pList); stDebug("s-task:%s level:%d task wait for %d upstream tasks complete scan-history procedure, status:%s", - pTask->id.idStr, pTask->info.taskLevel, numOfUps, p->name); + pTask->id.idStr, pTask->info.taskLevel, numOfUps, p.name); } ASSERT(pTask->status.downstreamReady == 0); @@ -59,7 +59,7 @@ static int32_t streamTaskSetReady(SStreamTask* pTask) { pTask->execInfo.readyTs = taosGetTimestampMs(); int64_t el = (pTask->execInfo.readyTs - pTask->execInfo.checkTs); stDebug("s-task:%s all %d downstream ready, init completed, elapsed time:%" PRId64 "ms, task status:%s", - pTask->id.idStr, numOfDowns, el, p->name); + pTask->id.idStr, numOfDowns, el, p.name); return TSDB_CODE_SUCCESS; } @@ -92,10 +92,11 @@ int32_t streamExecScanHistoryInFuture(SStreamTask* pTask, int32_t idleDuration) } // add ref for task - SStreamTask* p = streamMetaAcquireTask(pTask->pMeta, pTask->id.streamId, pTask->id.taskId); + SStreamTask* p = NULL; + int32_t code = streamMetaAcquireTask(pTask->pMeta, pTask->id.streamId, pTask->id.taskId, &p); if (p == NULL) { stError("s-task:0x%x failed to acquire task, status:%s, not exec scan-history data", pTask->id.taskId, - streamTaskGetStatus(pTask)->name); + streamTaskGetStatus(pTask).name); return TSDB_CODE_SUCCESS; } @@ -117,7 +118,7 @@ int32_t streamExecScanHistoryInFuture(SStreamTask* pTask, int32_t idleDuration) int32_t streamTaskStartScanHistory(SStreamTask* pTask) { int32_t level = pTask->info.taskLevel; - ETaskStatus status = streamTaskGetStatus(pTask)->state; + ETaskStatus status = streamTaskGetStatus(pTask).state; ASSERT((pTask->status.downstreamReady == 1) && (status == TASK_STATUS__SCAN_HISTORY) && (pTask->info.fillHistory == 1)); @@ -138,8 +139,8 @@ int32_t streamTaskOnNormalTaskReady(SStreamTask* pTask) { streamTaskSetReady(pTask); streamTaskSetRangeStreamCalc(pTask); - SStreamTaskState* p = streamTaskGetStatus(pTask); - ASSERT(p->state == TASK_STATUS__READY); + SStreamTaskState p = streamTaskGetStatus(pTask); + ASSERT(p.state == TASK_STATUS__READY); int8_t schedStatus = pTask->status.schedStatus; if (pTask->info.taskLevel == TASK_LEVEL__SOURCE) { @@ -148,10 +149,10 @@ int32_t streamTaskOnNormalTaskReady(SStreamTask* pTask) { startVer = pTask->chkInfo.nextProcessVer; } - stDebug("s-task:%s status:%s, sched-status:%d, ready for data from wal ver:%" PRId64, id, p->name, schedStatus, + stDebug("s-task:%s status:%s, sched-status:%d, ready for data from wal ver:%" PRId64, id, p.name, schedStatus, startVer); } else { - stDebug("s-task:%s level:%d status:%s sched-status:%d", id, pTask->info.taskLevel, p->name, schedStatus); + stDebug("s-task:%s level:%d status:%s sched-status:%d", id, pTask->info.taskLevel, p.name, schedStatus); } return TSDB_CODE_SUCCESS; @@ -162,10 +163,10 @@ int32_t streamTaskOnScanHistoryTaskReady(SStreamTask* pTask) { streamTaskSetReady(pTask); streamTaskSetRangeStreamCalc(pTask); - SStreamTaskState* p = streamTaskGetStatus(pTask); - ASSERT((p->state == TASK_STATUS__SCAN_HISTORY) && (pTask->info.fillHistory == 1)); + SStreamTaskState p = streamTaskGetStatus(pTask); + ASSERT((p.state == TASK_STATUS__SCAN_HISTORY) && (pTask->info.fillHistory == 1)); - stDebug("s-task:%s fill-history task enters into scan-history data stage, status:%s", pTask->id.idStr, p->name); + stDebug("s-task:%s fill-history task enters into scan-history data stage, status:%s", pTask->id.idStr, p.name); streamTaskStartScanHistory(pTask); // NOTE: there will be an deadlock if launch fill history here. @@ -205,11 +206,11 @@ int32_t streamLaunchFillHistoryTask(SStreamTask* pTask) { ASSERT(hTaskId != 0); // check stream task status in the first place. - SStreamTaskState* pStatus = streamTaskGetStatus(pTask); - if (pStatus->state != TASK_STATUS__READY && pStatus->state != TASK_STATUS__HALT && - pStatus->state != TASK_STATUS__PAUSE) { + SStreamTaskState pStatus = streamTaskGetStatus(pTask); + if (pStatus.state != TASK_STATUS__READY && pStatus.state != TASK_STATUS__HALT && + pStatus.state != TASK_STATUS__PAUSE) { stDebug("s-task:%s not launch related fill-history task:0x%" PRIx64 "-0x%x, status:%s", idStr, hStreamId, hTaskId, - pStatus->name); + pStatus.name); streamMetaAddTaskLaunchResult(pMeta, hStreamId, hTaskId, pExecInfo->checkTs, pExecInfo->readyTs, false); return -1; // todo set the correct error code @@ -223,7 +224,8 @@ int32_t streamLaunchFillHistoryTask(SStreamTask* pTask) { streamMetaRUnLock(pMeta); if (pHTask != NULL) { // it is already added into stream meta store. - SStreamTask* pHisTask = streamMetaAcquireTask(pMeta, hStreamId, hTaskId); + SStreamTask* pHisTask = NULL; + code = streamMetaAcquireTask(pMeta, hStreamId, hTaskId, &pHisTask); if (pHisTask == NULL) { stDebug("s-task:%s failed acquire and start fill-history task, it may have been dropped/stopped", idStr); streamMetaAddTaskLaunchResult(pMeta, hStreamId, hTaskId, pExecInfo->checkTs, pExecInfo->readyTs, false); @@ -308,7 +310,7 @@ void doRetryLaunchFillHistoryTask(SStreamTask* pTask, SLaunchHTaskInfo* pInfo, i streamMetaAddTaskLaunchResult(pMeta, pInfo->hTaskId.streamId, pInfo->hTaskId.taskId, 0, now, false); taosMemoryFree(pInfo); } else { - char* p = streamTaskGetStatus(pTask)->name; + char* p = streamTaskGetStatus(pTask).name; int32_t hTaskId = pHTaskInfo->id.taskId; stDebug("s-task:%s status:%s failed to launch fill-history task:0x%x, retry launch:%dms, retryCount:%d", @@ -340,7 +342,7 @@ void tryLaunchHistoryTask(void* param, void* tmrId) { if (streamTaskShouldStop(*ppTask)) { ASSERT((*ppTask)->status.timerActive >= 1); - char* p = streamTaskGetStatus(*ppTask)->name; + char* p = streamTaskGetStatus(*ppTask).name; int32_t ref = atomic_sub_fetch_32(&(*ppTask)->status.timerActive, 1); stDebug("s-task:%s status:%s should stop, quit launch fill-history task timer, retry:%d, ref:%d", (*ppTask)->id.idStr, p, (*ppTask)->hTaskInfo.retryTimes, ref); @@ -353,7 +355,11 @@ void tryLaunchHistoryTask(void* param, void* tmrId) { return; } - SStreamTask* pTask = streamMetaAcquireTaskNoLock(pMeta, pInfo->id.streamId, pInfo->id.taskId); + SStreamTask* pTask = NULL; + code = streamMetaAcquireTaskNoLock(pMeta, pInfo->id.streamId, pInfo->id.taskId, &pTask); + if (code != TSDB_CODE_SUCCESS) { + // todo + } streamMetaWUnLock(pMeta); if (pTask != NULL) { @@ -373,7 +379,8 @@ void tryLaunchHistoryTask(void* param, void* tmrId) { ASSERT(pTask->status.timerActive >= 1); // abort the timer if intend to stop task - SStreamTask* pHTask = streamMetaAcquireTask(pMeta, pHTaskInfo->id.streamId, pHTaskInfo->id.taskId); + SStreamTask* pHTask = NULL; + code = streamMetaAcquireTask(pMeta, pHTaskInfo->id.streamId, pHTaskInfo->id.taskId, &pHTask); if (pHTask == NULL) { doRetryLaunchFillHistoryTask(pTask, pInfo, now); streamMetaReleaseTask(pMeta, pTask); @@ -537,10 +544,10 @@ void doExecScanhistoryInFuture(void* param, void* tmrId) { SStreamTask* pTask = param; pTask->schedHistoryInfo.numOfTicks -= 1; - SStreamTaskState* p = streamTaskGetStatus(pTask); - if (p->state == TASK_STATUS__DROPPING || p->state == TASK_STATUS__STOP) { + SStreamTaskState p = streamTaskGetStatus(pTask); + if (p.state == TASK_STATUS__DROPPING || p.state == TASK_STATUS__STOP) { int32_t ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1); - stDebug("s-task:%s status:%s not start scan-history again, ref:%d", pTask->id.idStr, p->name, ref); + stDebug("s-task:%s status:%s not start scan-history again, ref:%d", pTask->id.idStr, p.name, ref); streamMetaReleaseTask(pTask->pMeta, pTask); return; diff --git a/source/libs/stream/src/streamTask.c b/source/libs/stream/src/streamTask.c index da30caad98..39e12a9da7 100644 --- a/source/libs/stream/src/streamTask.c +++ b/source/libs/stream/src/streamTask.c @@ -91,14 +91,15 @@ static SStreamUpstreamEpInfo* createStreamTaskEpInfo(const SStreamTask* pTask) { return pEpInfo; } -SStreamTask* tNewStreamTask(int64_t streamId, int8_t taskLevel, SEpSet* pEpset, bool fillHistory, int64_t triggerParam, - SArray* pTaskList, bool hasFillhistory, int8_t subtableWithoutMd5) { +int32_t tNewStreamTask(int64_t streamId, int8_t taskLevel, SEpSet* pEpset, bool fillHistory, int64_t triggerParam, + SArray* pTaskList, bool hasFillhistory, int8_t subtableWithoutMd5, SStreamTask** p) { + *p = NULL; + SStreamTask* pTask = (SStreamTask*)taosMemoryCalloc(1, sizeof(SStreamTask)); if (pTask == NULL) { - terrno = TSDB_CODE_OUT_OF_MEMORY; stError("s-task:0x%" PRIx64 " failed malloc new stream task, size:%d, code:%s", streamId, (int32_t)sizeof(SStreamTask), tstrerror(terrno)); - return NULL; + return TSDB_CODE_OUT_OF_MEMORY; } pTask->ver = SSTREAM_TASK_VER; @@ -110,10 +111,10 @@ SStreamTask* tNewStreamTask(int64_t streamId, int8_t taskLevel, SEpSet* pEpset, pTask->info.delaySchedParam = triggerParam; pTask->subtableWithoutMd5 = subtableWithoutMd5; - pTask->status.pSM = streamCreateStateMachine(pTask); - if (pTask->status.pSM == NULL) { + int32_t code = streamCreateStateMachine(pTask); + if (pTask->status.pSM == NULL || code != TSDB_CODE_SUCCESS) { taosMemoryFreeClear(pTask); - return NULL; + return code; } char buf[128] = {0}; @@ -135,7 +136,9 @@ SStreamTask* tNewStreamTask(int64_t streamId, int8_t taskLevel, SEpSet* pEpset, epsetAssign(&(pTask->info.mnodeEpset), pEpset); addToTaskset(pTaskList, pTask); - return pTask; + *p = pTask; + + return code; } int32_t tDecodeStreamTaskChkInfo(SDecoder* pDecoder, SCheckpointInfo* pChkpInfo) { @@ -196,9 +199,9 @@ void tFreeStreamTask(SStreamTask* pTask) { ETaskStatus status1 = TASK_STATUS__UNINIT; taosThreadMutexLock(&pTask->lock); if (pTask->status.pSM != NULL) { - SStreamTaskState* pStatus = streamTaskGetStatus(pTask); - p = pStatus->name; - status1 = pStatus->state; + SStreamTaskState pStatus = streamTaskGetStatus(pTask); + p = pStatus.name; + status1 = pStatus.state; } taosThreadMutexUnlock(&pTask->lock); @@ -275,7 +278,9 @@ void tFreeStreamTask(SStreamTask* pTask) { tSimpleHashCleanup(pTask->pNameMap); } - pTask->status.pSM = streamDestroyStateMachine(pTask->status.pSM); + streamDestroyStateMachine(pTask->status.pSM); + pTask->status.pSM = NULL; + streamTaskDestroyUpstreamInfo(&pTask->upstreamInfo); taosMemoryFree(pTask->outputInfo.pTokenBucket); @@ -355,26 +360,27 @@ static void setInitialVersionInfo(SStreamTask* pTask, int64_t ver) { } int32_t streamTaskInit(SStreamTask* pTask, SStreamMeta* pMeta, SMsgCb* pMsgCb, int64_t ver) { - pTask->id.idStr = createStreamTaskIdStr(pTask->id.streamId, pTask->id.taskId); + (void) createStreamTaskIdStr(pTask->id.streamId, pTask->id.taskId, &pTask->id.idStr); pTask->refCnt = 1; pTask->inputq.status = TASK_INPUT_STATUS__NORMAL; pTask->outputq.status = TASK_OUTPUT_STATUS__NORMAL; - pTask->inputq.queue = streamQueueOpen(512 << 10); - pTask->outputq.queue = streamQueueOpen(512 << 10); - if (pTask->inputq.queue == NULL || pTask->outputq.queue == NULL) { + int32_t code1 = streamQueueOpen(512 << 10, &pTask->inputq.queue); + int32_t code2 = streamQueueOpen(512 << 10, &pTask->outputq.queue); + if (code1 || code2) { stError("s-task:%s failed to prepare the input/output queue, initialize task failed", pTask->id.idStr); return TSDB_CODE_OUT_OF_MEMORY; } pTask->status.schedStatus = TASK_SCHED_STATUS__INACTIVE; pTask->status.timerActive = 0; - pTask->status.pSM = streamCreateStateMachine(pTask); - if (pTask->status.pSM == NULL) { + + int32_t code = streamCreateStateMachine(pTask); + if (pTask->status.pSM == NULL || code != TSDB_CODE_SUCCESS) { stError("s-task:%s failed create state-machine for stream task, initialization failed, code:%s", pTask->id.idStr, - tstrerror(terrno)); - return terrno; + tstrerror(code)); + return code; } pTask->execInfo.created = taosGetTimestampMs(); @@ -392,7 +398,7 @@ int32_t streamTaskInit(SStreamTask* pTask, SStreamMeta* pMeta, SMsgCb* pMsgCb, i TdThreadMutexAttr attr = {0}; - int code = taosThreadMutexAttrInit(&attr); + code = taosThreadMutexAttrInit(&attr); if (code != 0) { stError("s-task:%s initElapsed mutex attr failed, code:%s", pTask->id.idStr, tstrerror(code)); return code; @@ -434,10 +440,10 @@ int32_t streamTaskInit(SStreamTask* pTask, SStreamMeta* pMeta, SMsgCb* pMsgCb, i } if (pTask->chkInfo.pActiveInfo == NULL) { - pTask->chkInfo.pActiveInfo = streamTaskCreateActiveChkptInfo(); + code = streamTaskCreateActiveChkptInfo(&pTask->chkInfo.pActiveInfo); } - return TSDB_CODE_SUCCESS; + return code; } int32_t streamTaskGetNumOfDownstream(const SStreamTask* pTask) { @@ -637,7 +643,13 @@ void streamTaskOpenAllUpstreamInput(SStreamTask* pTask) { } void streamTaskCloseUpstreamInput(SStreamTask* pTask, int32_t taskId) { - SStreamUpstreamEpInfo* pInfo = streamTaskGetUpstreamTaskEpInfo(pTask, taskId); + SStreamUpstreamEpInfo* pInfo = NULL; + + int32_t code = streamTaskGetUpstreamTaskEpInfo(pTask, taskId, &pInfo); + if (code != TSDB_CODE_SUCCESS) { + return; + } + if ((pInfo != NULL) && pInfo->dataAllowed) { pInfo->dataAllowed = false; int32_t t = atomic_add_fetch_32(&pTask->upstreamInfo.numOfClosed, 1); @@ -646,8 +658,14 @@ void streamTaskCloseUpstreamInput(SStreamTask* pTask, int32_t taskId) { } void streamTaskOpenUpstreamInput(SStreamTask* pTask, int32_t taskId) { - SStreamUpstreamEpInfo* pInfo = streamTaskGetUpstreamTaskEpInfo(pTask, taskId); - if ((pInfo != NULL) && (!pInfo->dataAllowed)) { + SStreamUpstreamEpInfo* pInfo = NULL; + + int32_t code = streamTaskGetUpstreamTaskEpInfo(pTask, taskId, &pInfo); + if (code != TSDB_CODE_SUCCESS) { + return; + } + + if (pInfo != NULL && (!pInfo->dataAllowed)) { int32_t t = atomic_sub_fetch_32(&pTask->upstreamInfo.numOfClosed, 1); ASSERT(t >= 0); pInfo->dataAllowed = true; @@ -711,7 +729,7 @@ int32_t streamTaskClearHTaskAttr(SStreamTask* pTask, int32_t resetRelHalt) { if (resetRelHalt) { stDebug("s-task:0x%" PRIx64 " set the persistent status attr to be ready, prev:%s, status in sm:%s", sTaskId.taskId, streamTaskGetStatusStr((*ppStreamTask)->status.taskStatus), - streamTaskGetStatus(*ppStreamTask)->name); + streamTaskGetStatus(*ppStreamTask).name); (*ppStreamTask)->status.taskStatus = TASK_STATUS__READY; } @@ -844,7 +862,7 @@ STaskStatusEntry streamTaskGetStatusEntry(SStreamTask* pTask) { STaskStatusEntry entry = { .id = streamTaskGetTaskId(pTask), - .status = streamTaskGetStatus(pTask)->state, + .status = streamTaskGetStatus(pTask).state, .nodeId = pMeta->vgId, .stage = pMeta->stage, @@ -889,12 +907,12 @@ void streamTaskPause(SStreamTask* pTask) { } void streamTaskResume(SStreamTask* pTask) { - SStreamTaskState prevState = *streamTaskGetStatus(pTask); + SStreamTaskState prevState = streamTaskGetStatus(pTask); SStreamMeta* pMeta = pTask->pMeta; int32_t code = streamTaskRestoreStatus(pTask); if (code == TSDB_CODE_SUCCESS) { - char* pNew = streamTaskGetStatus(pTask)->name; + char* pNew = streamTaskGetStatus(pTask).name; int32_t num = atomic_sub_fetch_32(&pMeta->numOfPausedTasks, 1); stInfo("s-task:%s status:%s resume from %s, paused task(s):%d", pTask->id.idStr, pNew, prevState.name, num); } else { @@ -942,17 +960,24 @@ int32_t streamTaskSendCheckpointReq(SStreamTask* pTask) { return 0; } -SStreamUpstreamEpInfo* streamTaskGetUpstreamTaskEpInfo(SStreamTask* pTask, int32_t taskId) { +int32_t streamTaskGetUpstreamTaskEpInfo(SStreamTask* pTask, int32_t taskId, SStreamUpstreamEpInfo** pEpInfo) { + *pEpInfo = NULL; + int32_t num = taosArrayGetSize(pTask->upstreamInfo.pList); for (int32_t i = 0; i < num; ++i) { SStreamUpstreamEpInfo* pInfo = taosArrayGetP(pTask->upstreamInfo.pList, i); + if (pInfo == NULL) { + return TSDB_CODE_FAILED; + } + if (pInfo->taskId == taskId) { - return pInfo; + *pEpInfo = pInfo; + return TSDB_CODE_SUCCESS; } } stError("s-task:%s failed to find upstream task:0x%x", pTask->id.idStr, taskId); - return NULL; + return TSDB_CODE_FAILED; } SEpSet* streamTaskGetDownstreamEpInfo(SStreamTask* pTask, int32_t taskId) { @@ -973,10 +998,16 @@ SEpSet* streamTaskGetDownstreamEpInfo(SStreamTask* pTask, int32_t taskId) { return NULL; } -char* createStreamTaskIdStr(int64_t streamId, int32_t taskId) { +int32_t createStreamTaskIdStr(int64_t streamId, int32_t taskId, const char** pId) { char buf[128] = {0}; sprintf(buf, "0x%" PRIx64 "-0x%x", streamId, taskId); - return taosStrdup(buf); + *pId = taosStrdup(buf); + + if (*pId == NULL) { + return TSDB_CODE_OUT_OF_MEMORY; + } else { + return TSDB_CODE_SUCCESS; + } } static int32_t streamTaskEnqueueRetrieve(SStreamTask* pTask, SStreamRetrieveReq* pReq) { @@ -1045,14 +1076,23 @@ int32_t streamTaskSetFailedChkptInfo(SStreamTask* pTask, int32_t transId, int64_ return TSDB_CODE_SUCCESS; } -SActiveCheckpointInfo* streamTaskCreateActiveChkptInfo() { +int32_t streamTaskCreateActiveChkptInfo(SActiveCheckpointInfo** pRes) { SActiveCheckpointInfo* pInfo = taosMemoryCalloc(1, sizeof(SActiveCheckpointInfo)); - taosThreadMutexInit(&pInfo->lock, NULL); + if (pInfo == NULL) { + return TSDB_CODE_OUT_OF_MEMORY; + } + + int32_t code = taosThreadMutexInit(&pInfo->lock, NULL); + if (code != TSDB_CODE_SUCCESS) { + return code; + } pInfo->pDispatchTriggerList = taosArrayInit(4, sizeof(STaskTriggerSendInfo)); pInfo->pReadyMsgList = taosArrayInit(4, sizeof(STaskCheckpointReadyInfo)); pInfo->pCheckpointReadyRecvList = taosArrayInit(4, sizeof(STaskDownstreamReadyInfo)); - return pInfo; + + *pRes = pInfo; + return TSDB_CODE_SUCCESS; } void streamTaskDestroyActiveChkptInfo(SActiveCheckpointInfo* pInfo) { diff --git a/source/libs/stream/src/streamTaskSm.c b/source/libs/stream/src/streamTaskSm.c index f2bd99cdaf..7e47857a39 100644 --- a/source/libs/stream/src/streamTaskSm.c +++ b/source/libs/stream/src/streamTaskSm.c @@ -20,7 +20,17 @@ #include "ttimer.h" #include "wal.h" +static int32_t initRes = 0; + #define GET_EVT_NAME(_ev) (StreamTaskEventList[(_ev)].name) +#define CHECK_RET_VAL(_exec) \ + do { \ + void* p = (_exec); \ + if (p == NULL) { \ + initRes = TSDB_CODE_OUT_OF_MEMORY; \ + return; \ + } \ + } while (0); SStreamTaskState StreamTaskStatusList[9] = { {.state = TASK_STATUS__READY, .name = "ready"}, @@ -67,16 +77,23 @@ static STaskStateTrans createStateTransform(ETaskStatus current, ETaskStatus nex static int32_t dummyFn(SStreamTask* UNUSED_PARAM(p)) { return TSDB_CODE_SUCCESS; } static int32_t attachWaitedEvent(SStreamTask* pTask, SFutureHandleEventInfo* pEvtInfo) { - char* p = streamTaskGetStatus(pTask)->name; + char* p = streamTaskGetStatus(pTask).name; stDebug("s-task:%s status:%s attach event:%s required status:%s, since not allowed to handle it", pTask->id.idStr, p, GET_EVT_NAME(pEvtInfo->event), StreamTaskStatusList[pEvtInfo->status].name); SArray* pList = pTask->status.pSM->pWaitingEventList; - taosArrayPush(pList, pEvtInfo); - stDebug("s-task:%s add into waiting list, total waiting events:%d", pTask->id.idStr, (int32_t)taosArrayGetSize(pList)); - return 0; + void* px = taosArrayPush(pList, pEvtInfo); + if (px == NULL) { + stError("s-task:%s failed to add into waiting list, total waiting events:%d, code: out of memory", pTask->id.idStr, + (int32_t)taosArrayGetSize(pList)); + return TSDB_CODE_OUT_OF_MEMORY; + } else { + stDebug("s-task:%s add into waiting list, total waiting events:%d", pTask->id.idStr, + (int32_t)taosArrayGetSize(pList)); + return TSDB_CODE_SUCCESS; + } } static int32_t stopTaskSuccFn(SStreamTask* pTask) { @@ -177,14 +194,14 @@ static int32_t doHandleWaitingEvent(SStreamTaskSM* pSM, const char* pEventName, GET_EVT_NAME(pEvtInfo->event), pSM->current.name); // remove it - taosArrayPop(pSM->pWaitingEventList); + (void) taosArrayPop(pSM->pWaitingEventList); STaskStateTrans* pNextTrans = streamTaskFindTransform(pSM->current.state, pEvtInfo->event); ASSERT(pSM->pActiveTrans == NULL && pNextTrans != NULL); pSM->pActiveTrans = pNextTrans; pSM->startTs = taosGetTimestampMs(); - taosThreadMutexUnlock(&pTask->lock); + (void) taosThreadMutexUnlock(&pTask->lock); code = pNextTrans->pAction(pSM->pTask); if (pNextTrans->autoInvokeEndFn) { @@ -193,7 +210,7 @@ static int32_t doHandleWaitingEvent(SStreamTaskSM* pSM, const char* pEventName, return code; } } else { - taosThreadMutexUnlock(&pTask->lock); + (void) taosThreadMutexUnlock(&pTask->lock); stDebug("s-task:%s state:%s event:%s in waiting list, req state:%s not fulfilled, put it back", pTask->id.idStr, pSM->current.name, GET_EVT_NAME(pEvtInfo->event), StreamTaskStatusList[pEvtInfo->status].name); @@ -228,9 +245,9 @@ static int32_t removeEventInWaitingList(SStreamTask* pTask, EStreamTaskEvent eve int32_t streamTaskRestoreStatus(SStreamTask* pTask) { SStreamTaskSM* pSM = pTask->status.pSM; - int32_t code = 0; + int32_t code = 0; - taosThreadMutexLock(&pTask->lock); + (void) taosThreadMutexLock(&pTask->lock); if (pSM->current.state == TASK_STATUS__PAUSE && pSM->pActiveTrans == NULL) { SStreamTaskState state = pSM->current; @@ -244,71 +261,78 @@ int32_t streamTaskRestoreStatus(SStreamTask* pTask) { if (taosArrayGetSize(pSM->pWaitingEventList) > 0) { stDebug("s-task:%s restore status, %s -> %s, and then handle waiting event", pTask->id.idStr, pSM->prev.state.name, pSM->current.name); - doHandleWaitingEvent(pSM, "restore-pause/halt", pTask); + code = doHandleWaitingEvent(pSM, "restore-pause/halt", pTask); } else { stDebug("s-task:%s restore status, %s -> %s", pTask->id.idStr, pSM->prev.state.name, pSM->current.name); } } else { - removeEventInWaitingList(pTask, TASK_EVENT_PAUSE); - code = -1; // failed to restore the status + (void)removeEventInWaitingList(pTask, TASK_EVENT_PAUSE); // ignore the return value, + code = TSDB_CODE_FAILED; // failed to restore the status, since it is not in pause status } - taosThreadMutexUnlock(&pTask->lock); + (void)taosThreadMutexUnlock(&pTask->lock); return code; } -SStreamTaskSM* streamCreateStateMachine(SStreamTask* pTask) { - initStateTransferTable(); +int32_t streamCreateStateMachine(SStreamTask* pTask) { + int32_t code = initStateTransferTable(); + if (code != TSDB_CODE_SUCCESS) { + return code; + } + const char* id = pTask->id.idStr; SStreamTaskSM* pSM = taosMemoryCalloc(1, sizeof(SStreamTaskSM)); if (pSM == NULL) { - terrno = TSDB_CODE_OUT_OF_MEMORY; stError("s-task:%s failed to create task stateMachine, size:%d, code:%s", id, (int32_t)sizeof(SStreamTaskSM), tstrerror(terrno)); - return NULL; + return TSDB_CODE_OUT_OF_MEMORY; } pSM->pTask = pTask; pSM->pWaitingEventList = taosArrayInit(4, sizeof(SFutureHandleEventInfo)); if (pSM->pWaitingEventList == NULL) { taosMemoryFree(pSM); - - terrno = TSDB_CODE_OUT_OF_MEMORY; stError("s-task:%s failed to create task stateMachine, size:%d, code:%s", id, (int32_t)sizeof(SStreamTaskSM), tstrerror(terrno)); - return NULL; + return TSDB_CODE_OUT_OF_MEMORY; } // set the initial state for the state-machine of stream task pSM->current = StreamTaskStatusList[TASK_STATUS__UNINIT]; pSM->startTs = taosGetTimestampMs(); - return pSM; + + pTask->status.pSM = pSM; + return TSDB_CODE_SUCCESS; } -void* streamDestroyStateMachine(SStreamTaskSM* pSM) { +void streamDestroyStateMachine(SStreamTaskSM* pSM) { if (pSM == NULL) { - return NULL; + return; } taosArrayDestroy(pSM->pWaitingEventList); taosMemoryFree(pSM); - return NULL; } static int32_t doHandleEvent(SStreamTaskSM* pSM, EStreamTaskEvent event, STaskStateTrans* pTrans) { SStreamTask* pTask = pSM->pTask; const char* id = pTask->id.idStr; + int32_t code = 0; if (pTrans->attachEvent.event != 0) { - attachWaitedEvent(pTask, &pTrans->attachEvent); - taosThreadMutexUnlock(&pTask->lock); + code = attachWaitedEvent(pTask, &pTrans->attachEvent); + if (code) { + return code; + } + + (void) taosThreadMutexUnlock(&pTask->lock); while (1) { // wait for the task to be here - taosThreadMutexLock(&pTask->lock); - ETaskStatus s = streamTaskGetStatus(pTask)->state; - taosThreadMutexUnlock(&pTask->lock); + (void) taosThreadMutexLock(&pTask->lock); + ETaskStatus s = streamTaskGetStatus(pTask).state; + (void) taosThreadMutexUnlock(&pTask->lock); if ((s == pTrans->next.state) && (pSM->prev.evt == pTrans->event)) {// this event has been handled already stDebug("s-task:%s attached event:%s handled", id, GET_EVT_NAME(pTrans->event)); @@ -325,42 +349,49 @@ static int32_t doHandleEvent(SStreamTaskSM* pSM, EStreamTaskEvent event, STaskSt } else { // override current active trans pSM->pActiveTrans = pTrans; pSM->startTs = taosGetTimestampMs(); - taosThreadMutexUnlock(&pTask->lock); + (void) taosThreadMutexUnlock(&pTask->lock); - int32_t code = pTrans->pAction(pTask); - // todo handle error code; + code = pTrans->pAction(pTask); if (pTrans->autoInvokeEndFn) { - streamTaskOnHandleEventSuccess(pSM, event, NULL, NULL); + int32_t c = streamTaskOnHandleEventSuccess(pSM, event, NULL, NULL); + if (code == TSDB_CODE_SUCCESS) { + code = c; + } } } - return TSDB_CODE_SUCCESS; + return code; } static int32_t doHandleEventAsync(SStreamTaskSM* pSM, EStreamTaskEvent event, STaskStateTrans* pTrans, __state_trans_user_fn callbackFn, void* param) { SStreamTask* pTask = pSM->pTask; + int32_t code = 0; + if (pTrans->attachEvent.event != 0) { SFutureHandleEventInfo info = pTrans->attachEvent; info.pParam = param; info.callBackFn = callbackFn; - attachWaitedEvent(pTask, &info); - taosThreadMutexUnlock(&pTask->lock); + code = attachWaitedEvent(pTask, &info); + (void) taosThreadMutexUnlock(&pTask->lock); } else { // override current active trans pSM->pActiveTrans = pTrans; pSM->startTs = taosGetTimestampMs(); - taosThreadMutexUnlock(&pTask->lock); + (void) taosThreadMutexUnlock(&pTask->lock); - int32_t code = pTrans->pAction(pTask); + code = pTrans->pAction(pTask); // todo handle error code; if (pTrans->autoInvokeEndFn) { - streamTaskOnHandleEventSuccess(pSM, event, callbackFn, param); + int32_t c = streamTaskOnHandleEventSuccess(pSM, event, callbackFn, param); + if (code == TSDB_CODE_SUCCESS) { + code = c; + } } } - return TSDB_CODE_SUCCESS; + return code; } int32_t streamTaskHandleEvent(SStreamTaskSM* pSM, EStreamTaskEvent event) { @@ -369,11 +400,11 @@ int32_t streamTaskHandleEvent(SStreamTaskSM* pSM, EStreamTaskEvent event) { STaskStateTrans* pTrans = NULL; while (1) { - taosThreadMutexLock(&pTask->lock); + (void) taosThreadMutexLock(&pTask->lock); if (pSM->pActiveTrans != NULL && pSM->pActiveTrans->autoInvokeEndFn) { EStreamTaskEvent evt = pSM->pActiveTrans->event; - taosThreadMutexUnlock(&pTask->lock); + (void) taosThreadMutexUnlock(&pTask->lock); stDebug("s-task:%s status:%s handling event:%s by some other thread, wait for 100ms and check if completed", pTask->id.idStr, pSM->current.name, GET_EVT_NAME(evt)); @@ -383,7 +414,7 @@ int32_t streamTaskHandleEvent(SStreamTaskSM* pSM, EStreamTaskEvent event) { pTrans = streamTaskFindTransform(pSM->current.state, event); if (pTrans == NULL) { stDebug("s-task:%s failed to handle event:%s", pTask->id.idStr, GET_EVT_NAME(event)); - taosThreadMutexUnlock(&pTask->lock); + (void) taosThreadMutexUnlock(&pTask->lock); return TSDB_CODE_STREAM_INVALID_STATETRANS; } @@ -408,11 +439,11 @@ int32_t streamTaskHandleEventAsync(SStreamTaskSM* pSM, EStreamTaskEvent event, _ STaskStateTrans* pTrans = NULL; while (1) { - taosThreadMutexLock(&pTask->lock); + (void) taosThreadMutexLock(&pTask->lock); if (pSM->pActiveTrans != NULL && pSM->pActiveTrans->autoInvokeEndFn) { EStreamTaskEvent evt = pSM->pActiveTrans->event; - taosThreadMutexUnlock(&pTask->lock); + (void) taosThreadMutexUnlock(&pTask->lock); stDebug("s-task:%s status:%s handling event:%s by some other thread, wait for 100ms and check if completed", pTask->id.idStr, pSM->current.name, GET_EVT_NAME(evt)); @@ -422,7 +453,7 @@ int32_t streamTaskHandleEventAsync(SStreamTaskSM* pSM, EStreamTaskEvent event, _ pTrans = streamTaskFindTransform(pSM->current.state, event); if (pTrans == NULL) { stDebug("s-task:%s failed to handle event:%s, status:%s", pTask->id.idStr, GET_EVT_NAME(event), pSM->current.name); - taosThreadMutexUnlock(&pTask->lock); + (void) taosThreadMutexUnlock(&pTask->lock); return TSDB_CODE_STREAM_INVALID_STATETRANS; } @@ -451,9 +482,10 @@ static void keepPrevInfo(SStreamTaskSM* pSM) { int32_t streamTaskOnHandleEventSuccess(SStreamTaskSM* pSM, EStreamTaskEvent event, __state_trans_user_fn callbackFn, void* param) { SStreamTask* pTask = pSM->pTask; const char* id = pTask->id.idStr; + int32_t code = 0; // do update the task status - taosThreadMutexLock(&pTask->lock); + (void) taosThreadMutexLock(&pTask->lock); STaskStateTrans* pTrans = pSM->pActiveTrans; if (pTrans == NULL) { @@ -465,14 +497,14 @@ int32_t streamTaskOnHandleEventSuccess(SStreamTaskSM* pSM, EStreamTaskEvent even stDebug("s-task:%s event:%s handled failed, current status:%s, trigger event:%s", id, GET_EVT_NAME(event), pSM->current.name, GET_EVT_NAME(pSM->prev.evt)); - taosThreadMutexUnlock(&pTask->lock); + (void) taosThreadMutexUnlock(&pTask->lock); return TSDB_CODE_STREAM_INVALID_STATETRANS; } if (pTrans->event != event) { stWarn("s-task:%s handle event:%s failed, current status:%s, active trans evt:%s", id, GET_EVT_NAME(event), pSM->current.name, GET_EVT_NAME(pTrans->event)); - taosThreadMutexUnlock(&pTask->lock); + (void) taosThreadMutexUnlock(&pTask->lock); return TSDB_CODE_STREAM_INVALID_STATETRANS; } @@ -482,38 +514,42 @@ int32_t streamTaskOnHandleEventSuccess(SStreamTaskSM* pSM, EStreamTaskEvent even pSM->pActiveTrans = NULL; // todo remove it + // todo: handle the error code // on success callback, add into lock if necessary, or maybe we should add an option for this? - pTrans->pSuccAction(pTask); + code = pTrans->pSuccAction(pTask); - taosThreadMutexUnlock(&pTask->lock); + (void) taosThreadMutexUnlock(&pTask->lock); // todo: add parameter to control lock // after handling the callback function assigned by invoker, go on handling the waiting tasks if (callbackFn != NULL) { stDebug("s-task:%s start to handle user-specified callback fn for event:%s", id, GET_EVT_NAME(pTrans->event)); - callbackFn(pSM->pTask, param); + int32_t ret = callbackFn(pSM->pTask, param); + if (ret != TSDB_CODE_SUCCESS) { + // todo handle error + } stDebug("s-task:%s handle user-specified callback fn for event:%s completed", id, GET_EVT_NAME(pTrans->event)); } - taosThreadMutexLock(&pTask->lock); + (void) taosThreadMutexLock(&pTask->lock); // tasks in waiting list if (taosArrayGetSize(pSM->pWaitingEventList) > 0) { - doHandleWaitingEvent(pSM, GET_EVT_NAME(pTrans->event), pTask); + code = doHandleWaitingEvent(pSM, GET_EVT_NAME(pTrans->event), pTask); } else { - taosThreadMutexUnlock(&pTask->lock); + (void) taosThreadMutexUnlock(&pTask->lock); int64_t el = (taosGetTimestampMs() - pSM->startTs); stDebug("s-task:%s handle event:%s completed, elapsed time:%" PRId64 "ms state:%s -> %s", id, GET_EVT_NAME(pTrans->event), el, pSM->prev.state.name, pSM->current.name); } - return TSDB_CODE_SUCCESS; + return code; } -SStreamTaskState* streamTaskGetStatus(const SStreamTask* pTask) { - return &pTask->status.pSM->current; // copy one obj in case of multi-thread environment +SStreamTaskState streamTaskGetStatus(const SStreamTask* pTask) { + return pTask->status.pSM->current; // copy one obj in case of multi-thread environment } ETaskStatus streamTaskGetPrevStatus(const SStreamTask* pTask) { @@ -527,14 +563,14 @@ const char* streamTaskGetStatusStr(ETaskStatus status) { void streamTaskResetStatus(SStreamTask* pTask) { SStreamTaskSM* pSM = pTask->status.pSM; - taosThreadMutexLock(&pTask->lock); + (void) taosThreadMutexLock(&pTask->lock); stDebug("s-task:%s level:%d fill-history:%d vgId:%d set uninit, prev status:%s", pTask->id.idStr, pTask->info.taskLevel, pTask->info.fillHistory, pTask->pMeta->vgId, pSM->current.name); pSM->current = StreamTaskStatusList[TASK_STATUS__UNINIT]; pSM->pActiveTrans = NULL; taosArrayClear(pSM->pWaitingEventList); - taosThreadMutexUnlock(&pTask->lock); + (void) taosThreadMutexUnlock(&pTask->lock); // clear the downstream ready status pTask->status.downstreamReady = 0; @@ -577,8 +613,7 @@ STaskStateTrans createStateTransform(ETaskStatus current, ETaskStatus next, EStr } int32_t initStateTransferTable() { - taosThreadOnce(&streamTaskStateMachineInit, doInitStateTransferTable); - return TSDB_CODE_SUCCESS; + return taosThreadOnce(&streamTaskStateMachineInit, doInitStateTransferTable); } //clang-format off @@ -587,92 +622,93 @@ void doInitStateTransferTable(void) { // initialization event handle STaskStateTrans trans = createStateTransform(TASK_STATUS__UNINIT, TASK_STATUS__READY, TASK_EVENT_INIT, streamTaskInitStatus, streamTaskOnNormalTaskReady, NULL); - taosArrayPush(streamTaskSMTrans, &trans); + CHECK_RET_VAL(taosArrayPush(streamTaskSMTrans, &trans)); + trans = createStateTransform(TASK_STATUS__UNINIT, TASK_STATUS__SCAN_HISTORY, TASK_EVENT_INIT_SCANHIST, streamTaskInitStatus, streamTaskOnScanHistoryTaskReady, NULL); - taosArrayPush(streamTaskSMTrans, &trans); + CHECK_RET_VAL(taosArrayPush(streamTaskSMTrans, &trans)); // scan-history related event trans = createStateTransform(TASK_STATUS__SCAN_HISTORY, TASK_STATUS__READY, TASK_EVENT_SCANHIST_DONE, NULL, NULL, NULL); - taosArrayPush(streamTaskSMTrans, &trans); + CHECK_RET_VAL(taosArrayPush(streamTaskSMTrans, &trans)); // halt stream task, from other task status trans = createStateTransform(TASK_STATUS__READY, TASK_STATUS__HALT, TASK_EVENT_HALT, NULL, streamTaskKeepCurrentVerInWal, NULL); - taosArrayPush(streamTaskSMTrans, &trans); + CHECK_RET_VAL(taosArrayPush(streamTaskSMTrans, &trans)); trans = createStateTransform(TASK_STATUS__HALT, TASK_STATUS__HALT, TASK_EVENT_HALT, NULL, streamTaskKeepCurrentVerInWal, NULL); - taosArrayPush(streamTaskSMTrans, &trans); + CHECK_RET_VAL(taosArrayPush(streamTaskSMTrans, &trans)); SFutureHandleEventInfo info = {.status = TASK_STATUS__READY, .event = TASK_EVENT_HALT}; trans = createStateTransform(TASK_STATUS__CK, TASK_STATUS__HALT, TASK_EVENT_HALT, NULL, streamTaskKeepCurrentVerInWal, &info); - taosArrayPush(streamTaskSMTrans, &trans); + CHECK_RET_VAL(taosArrayPush(streamTaskSMTrans, &trans)); trans = createStateTransform(TASK_STATUS__PAUSE, TASK_STATUS__HALT, TASK_EVENT_HALT, NULL, streamTaskKeepCurrentVerInWal, NULL); - taosArrayPush(streamTaskSMTrans, &trans); + CHECK_RET_VAL(taosArrayPush(streamTaskSMTrans, &trans)); // checkpoint related event trans = createStateTransform(TASK_STATUS__READY, TASK_STATUS__CK, TASK_EVENT_GEN_CHECKPOINT, NULL, NULL, NULL); - taosArrayPush(streamTaskSMTrans, &trans); + CHECK_RET_VAL(taosArrayPush(streamTaskSMTrans, &trans)); trans = createStateTransform(TASK_STATUS__HALT, TASK_STATUS__CK, TASK_EVENT_GEN_CHECKPOINT, NULL, NULL, NULL); - taosArrayPush(streamTaskSMTrans, &trans); + CHECK_RET_VAL(taosArrayPush(streamTaskSMTrans, &trans)); trans = createStateTransform(TASK_STATUS__CK, TASK_STATUS__READY, TASK_EVENT_CHECKPOINT_DONE, NULL, NULL, NULL); - taosArrayPush(streamTaskSMTrans, &trans); + CHECK_RET_VAL(taosArrayPush(streamTaskSMTrans, &trans)); // pause & resume related event handle trans = createStateTransform(TASK_STATUS__READY, TASK_STATUS__PAUSE, TASK_EVENT_PAUSE, NULL, NULL, NULL); - taosArrayPush(streamTaskSMTrans, &trans); + CHECK_RET_VAL(taosArrayPush(streamTaskSMTrans, &trans)); trans = createStateTransform(TASK_STATUS__SCAN_HISTORY, TASK_STATUS__PAUSE, TASK_EVENT_PAUSE, NULL, NULL, NULL); - taosArrayPush(streamTaskSMTrans, &trans); + CHECK_RET_VAL(taosArrayPush(streamTaskSMTrans, &trans)); info = (SFutureHandleEventInfo){.status = TASK_STATUS__READY, .event = TASK_EVENT_PAUSE}; trans = createStateTransform(TASK_STATUS__CK, TASK_STATUS__PAUSE, TASK_EVENT_PAUSE, NULL, NULL, &info); - taosArrayPush(streamTaskSMTrans, &trans); + CHECK_RET_VAL(taosArrayPush(streamTaskSMTrans, &trans)); trans = createStateTransform(TASK_STATUS__HALT, TASK_STATUS__PAUSE, TASK_EVENT_PAUSE, NULL, NULL, &info); - taosArrayPush(streamTaskSMTrans, &trans); + CHECK_RET_VAL(taosArrayPush(streamTaskSMTrans, &trans)); trans = createStateTransform(TASK_STATUS__UNINIT, TASK_STATUS__PAUSE, TASK_EVENT_PAUSE, NULL, NULL, &info); - taosArrayPush(streamTaskSMTrans, &trans); + CHECK_RET_VAL(taosArrayPush(streamTaskSMTrans, &trans)); trans = createStateTransform(TASK_STATUS__PAUSE, TASK_STATUS__PAUSE, TASK_EVENT_PAUSE, NULL, NULL, NULL); - taosArrayPush(streamTaskSMTrans, &trans); + CHECK_RET_VAL(taosArrayPush(streamTaskSMTrans, &trans)); trans = createStateTransform(TASK_STATUS__STOP, TASK_STATUS__STOP, TASK_EVENT_PAUSE, NULL, NULL, NULL); - taosArrayPush(streamTaskSMTrans, &trans); + CHECK_RET_VAL(taosArrayPush(streamTaskSMTrans, &trans)); trans = createStateTransform(TASK_STATUS__DROPPING, TASK_STATUS__DROPPING, TASK_EVENT_PAUSE, NULL, NULL, NULL); - taosArrayPush(streamTaskSMTrans, &trans); + CHECK_RET_VAL(taosArrayPush(streamTaskSMTrans, &trans)); // resume is completed by restore status of state-machine // stop related event trans = createStateTransform(TASK_STATUS__READY, TASK_STATUS__STOP, TASK_EVENT_STOP, NULL, stopTaskSuccFn, NULL); - taosArrayPush(streamTaskSMTrans, &trans); + CHECK_RET_VAL(taosArrayPush(streamTaskSMTrans, &trans)); trans = createStateTransform(TASK_STATUS__DROPPING, TASK_STATUS__STOP, TASK_EVENT_STOP, NULL, stopTaskSuccFn, NULL); - taosArrayPush(streamTaskSMTrans, &trans); + CHECK_RET_VAL(taosArrayPush(streamTaskSMTrans, &trans)); trans = createStateTransform(TASK_STATUS__UNINIT, TASK_STATUS__STOP, TASK_EVENT_STOP, NULL, stopTaskSuccFn, NULL); - taosArrayPush(streamTaskSMTrans, &trans); + CHECK_RET_VAL(taosArrayPush(streamTaskSMTrans, &trans)); trans = createStateTransform(TASK_STATUS__STOP, TASK_STATUS__STOP, TASK_EVENT_STOP, NULL, stopTaskSuccFn, NULL); - taosArrayPush(streamTaskSMTrans, &trans); + CHECK_RET_VAL(taosArrayPush(streamTaskSMTrans, &trans)); trans = createStateTransform(TASK_STATUS__SCAN_HISTORY, TASK_STATUS__STOP, TASK_EVENT_STOP, NULL, stopTaskSuccFn, NULL); - taosArrayPush(streamTaskSMTrans, &trans); + CHECK_RET_VAL(taosArrayPush(streamTaskSMTrans, &trans)); trans = createStateTransform(TASK_STATUS__HALT, TASK_STATUS__STOP, TASK_EVENT_STOP, NULL, stopTaskSuccFn, NULL); - taosArrayPush(streamTaskSMTrans, &trans); + CHECK_RET_VAL(taosArrayPush(streamTaskSMTrans, &trans)); trans = createStateTransform(TASK_STATUS__PAUSE, TASK_STATUS__STOP, TASK_EVENT_STOP, NULL, stopTaskSuccFn, NULL); - taosArrayPush(streamTaskSMTrans, &trans); + CHECK_RET_VAL(taosArrayPush(streamTaskSMTrans, &trans)); trans = createStateTransform(TASK_STATUS__CK, TASK_STATUS__STOP, TASK_EVENT_STOP, NULL, stopTaskSuccFn, NULL); - taosArrayPush(streamTaskSMTrans, &trans); + CHECK_RET_VAL(taosArrayPush(streamTaskSMTrans, &trans)); // dropping related event trans = createStateTransform(TASK_STATUS__READY, TASK_STATUS__DROPPING, TASK_EVENT_DROPPING, NULL, NULL, NULL); - taosArrayPush(streamTaskSMTrans, &trans); + CHECK_RET_VAL(taosArrayPush(streamTaskSMTrans, &trans)); trans = createStateTransform(TASK_STATUS__DROPPING, TASK_STATUS__DROPPING, TASK_EVENT_DROPPING, NULL, NULL, NULL); - taosArrayPush(streamTaskSMTrans, &trans); + CHECK_RET_VAL(taosArrayPush(streamTaskSMTrans, &trans)); trans = createStateTransform(TASK_STATUS__UNINIT, TASK_STATUS__DROPPING, TASK_EVENT_DROPPING, NULL, NULL, NULL); - taosArrayPush(streamTaskSMTrans, &trans); + CHECK_RET_VAL(taosArrayPush(streamTaskSMTrans, &trans)); trans = createStateTransform(TASK_STATUS__STOP, TASK_STATUS__DROPPING, TASK_EVENT_DROPPING, NULL, NULL, NULL); - taosArrayPush(streamTaskSMTrans, &trans); + CHECK_RET_VAL(taosArrayPush(streamTaskSMTrans, &trans)); trans = createStateTransform(TASK_STATUS__SCAN_HISTORY, TASK_STATUS__DROPPING, TASK_EVENT_DROPPING, NULL, NULL, NULL); - taosArrayPush(streamTaskSMTrans, &trans); + CHECK_RET_VAL(taosArrayPush(streamTaskSMTrans, &trans)); trans = createStateTransform(TASK_STATUS__HALT, TASK_STATUS__DROPPING, TASK_EVENT_DROPPING, NULL, NULL, NULL); - taosArrayPush(streamTaskSMTrans, &trans); + CHECK_RET_VAL(taosArrayPush(streamTaskSMTrans, &trans)); trans = createStateTransform(TASK_STATUS__PAUSE, TASK_STATUS__DROPPING, TASK_EVENT_DROPPING, NULL, NULL, NULL); - taosArrayPush(streamTaskSMTrans, &trans); + CHECK_RET_VAL(taosArrayPush(streamTaskSMTrans, &trans)); trans = createStateTransform(TASK_STATUS__CK, TASK_STATUS__DROPPING, TASK_EVENT_DROPPING, NULL, NULL, NULL); - taosArrayPush(streamTaskSMTrans, &trans); + CHECK_RET_VAL(taosArrayPush(streamTaskSMTrans, &trans)); } //clang-format on diff --git a/source/libs/stream/src/streamTimer.c b/source/libs/stream/src/streamTimer.c index 6e956e2682..c76ec92e33 100644 --- a/source/libs/stream/src/streamTimer.c +++ b/source/libs/stream/src/streamTimer.c @@ -38,3 +38,14 @@ void streamTimerCleanUp() { tmr_h streamTimerGetInstance() { return streamTimer; } + +void streamTmrReset(TAOS_TMR_CALLBACK fp, int32_t mseconds, void* param, void* handle, tmr_h* pTmrId, int32_t vgId, + const char* pMsg) { + while (1) { + bool ret = taosTmrReset(fp, mseconds, param, handle, pTmrId); + if (ret) { + break; + } + stError("vgId:%d failed to reset %s, try again", vgId, pMsg); + } +} diff --git a/source/libs/stream/test/backendTest.cpp b/source/libs/stream/test/backendTest.cpp index 38d48a2a32..104b1c27d8 100644 --- a/source/libs/stream/test/backendTest.cpp +++ b/source/libs/stream/test/backendTest.cpp @@ -43,7 +43,9 @@ SStreamState *stateCreate(const char *path) { pTask->ver = 1024; pTask->id.streamId = 1023; pTask->id.taskId = 1111111; - SStreamMeta *pMeta = streamMetaOpen((path), NULL, NULL, NULL, 0, 0, NULL); + SStreamMeta *pMeta = NULL; + + int32_t code = streamMetaOpen((path), NULL, NULL, NULL, 0, 0, NULL, &pMeta); pTask->pMeta = pMeta; SStreamState *p = streamStateOpen((char *)path, pTask, 0, 0);