From 7b7ce10475228a735caed6a32f9ca284be4bb09f Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 17 Oct 2023 11:23:04 +0800 Subject: [PATCH 01/41] refactor(stream): do some internal refactor. --- include/libs/stream/tstream.h | 22 +++-- source/libs/stream/inc/streamsm.h | 72 ++++++++++++++ source/libs/stream/src/streamTask.c | 12 ++- source/libs/stream/src/streamTaskSm.c | 135 ++++++++++++++++++++++++++ 4 files changed, 230 insertions(+), 11 deletions(-) create mode 100644 source/libs/stream/inc/streamsm.h create mode 100644 source/libs/stream/src/streamTaskSm.c diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 629efa00b3..97bc58b90f 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -44,8 +44,9 @@ extern "C" { #define NODE_ROLE_LEADER 0x2 #define NODE_ROLE_FOLLOWER 0x3 -typedef struct SStreamTask SStreamTask; -typedef struct SStreamQueue SStreamQueue; +typedef struct SStreamTask SStreamTask; +typedef struct SStreamQueue SStreamQueue; +typedef struct SStreamTaskSM SStreamTaskSM; #define SSTREAM_TASK_VER 2 enum { @@ -265,14 +266,15 @@ typedef struct SCheckpointInfo { } SCheckpointInfo; typedef struct SStreamStatus { - int8_t taskStatus; - int8_t downstreamReady; // downstream tasks are all ready now, if this flag is set - int8_t schedStatus; - int8_t keepTaskStatus; - bool appendTranstateBlock; // has append the transfer state data block already, todo: remove it - int8_t pauseAllowed; // allowed task status to be set to be paused - int32_t timerActive; // timer is active - int32_t inScanHistorySentinel; + SStreamTaskSM* pSM; + int8_t taskStatus; + int8_t downstreamReady; // downstream tasks are all ready now, if this flag is set + int8_t schedStatus; + int8_t keepTaskStatus; + bool appendTranstateBlock; // has append the transfer state data block already, todo: remove it + int8_t pauseAllowed; // allowed task status to be set to be paused + int32_t timerActive; // timer is active + int32_t inScanHistorySentinel; } SStreamStatus; typedef struct SDataRange { diff --git a/source/libs/stream/inc/streamsm.h b/source/libs/stream/inc/streamsm.h new file mode 100644 index 0000000000..4b87dcfe84 --- /dev/null +++ b/source/libs/stream/inc/streamsm.h @@ -0,0 +1,72 @@ +/* + * Copyright (c) 2019 TAOS Data, Inc. + * + * This program is free software: you can use, redistribute, and/or modify + * it under the terms of the GNU Affero General Public License, version 3 + * or later ("AGPL"), as published by the Free Software Foundation. + * + * This program is distributed in the hope that it will be useful, but WITHOUT + * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or + * FITNESS FOR A PARTICULAR PURPOSE. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + */ + +#ifndef TDENGINE_STREAMSM_H +#define TDENGINE_STREAMSM_H + +#include "tstream.h" + +#ifdef __cplusplus +extern "C" { +#endif + +// moore finite state machine for stream task +typedef struct SStreamTaskState { + ETaskStatus state; + char* name; +} SStreamTaskState; + +typedef enum EStreamTaskEvent { + TASK_EVENT_INIT = 0x1, + TASK_EVENT_START = 0x2, + TASK_EVENT_STOP = 0x3, + TASK_EVENT_GEN_CHECKPOINT = 0x4, + TASK_EVENT_PAUSE = 0x5, + TASK_EVENT_RESUME = 0x6, + TASK_EVENT_HALT = 0x7, + TASK_EVENT_TRANS_STATE = 0x8, + TASK_EVENT_SCAN_TSDB = 0x9, + TASK_EVENT_SCAN_WAL = 0x10, +} EStreamTaskEvent; + +typedef int32_t (*__state_trans_fn)(SStreamTask*); + +typedef struct STaskStateTrans { + SStreamTaskState state; + EStreamTaskEvent event; + SStreamTaskState next; + __state_trans_fn pAction; +} STaskStateTrans; + +struct SStreamTaskSM { + SStreamTaskState current; + SArray* pTransList; // SArray + int64_t stateTs; + SStreamTask* pTask; +}; + +typedef struct SStreamEventInfo { + EStreamTaskEvent event; + const char* name; + bool isTrans; +} SStreamEventInfo; + +SStreamTaskSM* streamCreateStateMachine(SStreamTask* pTask); + +#ifdef __cplusplus +} +#endif + +#endif // TDENGINE_STREAMSM_H diff --git a/source/libs/stream/src/streamTask.c b/source/libs/stream/src/streamTask.c index 57103e5a96..c9eb76b6c3 100644 --- a/source/libs/stream/src/streamTask.c +++ b/source/libs/stream/src/streamTask.c @@ -19,6 +19,7 @@ #include "tstream.h" #include "ttimer.h" #include "wal.h" +#include "streamsm.h" static void streamTaskDestroyUpstreamInfo(SUpstreamInfo* pUpstreamInfo); @@ -34,8 +35,11 @@ SStreamTask* tNewStreamTask(int64_t streamId, int8_t taskLevel, bool fillHistory 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; } + pTask->ver = SSTREAM_TASK_VER; pTask->id.taskId = tGenIdPI32(); pTask->id.streamId = streamId; @@ -43,12 +47,18 @@ SStreamTask* tNewStreamTask(int64_t streamId, int8_t taskLevel, bool fillHistory pTask->info.fillHistory = fillHistory; pTask->info.triggerParam = triggerParam; + pTask->status.pSM = streamCreateStateMachine(pTask); + if (pTask->status.pSM == NULL) { + taosMemoryFreeClear(pTask); + return NULL; + } + char buf[128] = {0}; sprintf(buf, "0x%" PRIx64 "-%d", pTask->id.streamId, pTask->id.taskId); pTask->id.idStr = taosStrdup(buf); pTask->status.schedStatus = TASK_SCHED_STATUS__INACTIVE; - pTask->status.taskStatus = (fillHistory || hasFillhistory)? TASK_STATUS__SCAN_HISTORY:TASK_STATUS__NORMAL; + pTask->status.taskStatus = (fillHistory || hasFillhistory) ? TASK_STATUS__SCAN_HISTORY : TASK_STATUS__NORMAL; pTask->inputInfo.status = TASK_INPUT_STATUS__NORMAL; pTask->outputq.status = TASK_OUTPUT_STATUS__NORMAL; diff --git a/source/libs/stream/src/streamTaskSm.c b/source/libs/stream/src/streamTaskSm.c new file mode 100644 index 0000000000..b6bd53eb87 --- /dev/null +++ b/source/libs/stream/src/streamTaskSm.c @@ -0,0 +1,135 @@ +/* + * Copyright (c) 2019 TAOS Data, Inc. + * + * This program is free software: you can use, redistribute, and/or modify + * it under the terms of the GNU Affero General Public License, version 3 + * or later ("AGPL"), as published by the Free Software Foundation. + * + * This program is distributed in the hope that it will be useful, but WITHOUT + * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or + * FITNESS FOR A PARTICULAR PURPOSE. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + */ + +#include "executor.h" +#include "streamInt.h" +#include "tmisce.h" +#include "tstream.h" +#include "ttimer.h" +#include "wal.h" +#include "streamsm.h" + +SStreamTaskState StreamTaskStatusList[8] = { + {.state = TASK_STATUS__NORMAL, .name = "normal"}, + {.state = TASK_STATUS__DROPPING, .name = "dropping"}, + {.state = TASK_STATUS__UNINIT, .name = "uninit"}, + {.state = TASK_STATUS__STOP, .name = "stop"}, + {.state = TASK_STATUS__SCAN_HISTORY, .name = "scan-history"}, + {.state = TASK_STATUS__HALT, .name = "halt"}, + {.state = TASK_STATUS__PAUSE, .name = "paused"}, + {.state = TASK_STATUS__CK, .name = "checkpoint"}, +}; + +static STaskStateTrans createStateTransform(ETaskStatus current, ETaskStatus next, EStreamTaskEvent event, __state_trans_fn fn); +static int32_t initStateTransferTable(SStreamTaskSM* pSM); + +static int32_t dummyFn(SStreamTask* UNUSED_PARAM(p)) { return 0; } +static int32_t streamTaskStartCheckDownstream(SStreamTask* pTask) { + stDebug("s-task:%s start to check downstream tasks", pTask->id.idStr); + return 0; +} +static int32_t streamTaskDoPause(SStreamTask* pTask) { + stDebug("s-task:%s start to pause tasks", pTask->id.idStr); + return 0; +} +static int32_t streamTaskDoResume(SStreamTask* pTask) { + stDebug("s-task:%s start to resume tasks", pTask->id.idStr); + return 0; +} +static int32_t streamTaskDoCheckpoint(SStreamTask* pTask) { + stDebug("s-task:%s start to do checkpoint", pTask->id.idStr); + return 0; +} + +// todo optimize the perf of find the trans objs by using hash table +static STaskStateTrans* streamTaskFindTransform(const SStreamTaskSM* pState, const EStreamTaskEvent* pEvent) { + int32_t numOfTrans = taosArrayGetSize(pState->pTransList); + for(int32_t i = 0; i < numOfTrans; ++i) { + STaskStateTrans* pTrans = taosArrayGet(pState->pTransList, i); + if (pTrans->state.state == pState->current.state && pTrans->event == *pEvent) { + return pTrans; + } + } + + ASSERT(0); + return NULL; +} + +SStreamTaskSM* streamCreateStateMachine(SStreamTask* pTask) { + SStreamTaskSM* pSM = taosMemoryCalloc(1, sizeof(SStreamTaskSM)); + if (pSM == NULL) { + terrno = TSDB_CODE_OUT_OF_MEMORY; + return NULL; + } + + pSM->pTask = pTask; + + // set the initial state for the state-machine of stream task + pSM->current = StreamTaskStatusList[TASK_STATUS__UNINIT]; + pSM->stateTs = taosGetTimestampMs(); + int32_t code = initStateTransferTable(pSM); + if (code != TSDB_CODE_SUCCESS) { + taosMemoryFree(pSM); + return NULL; + } + + return pSM; +} + +int32_t taskSMHandleEvent(SStreamTaskSM* pSM, const EStreamTaskEvent* pEvent) { + STaskStateTrans* pTrans = streamTaskFindTransform(pSM, pEvent); + qDebug("start to handle event:%d", *pEvent); + + pSM->current = pTrans->next; + pSM->stateTs = taosGetTimestampMs(); + qDebug("new state:%s from %s", pTrans->next.name, pSM->current.name); + + return pTrans->pAction(pSM->pTask); +} + +STaskStateTrans createStateTransform(ETaskStatus current, ETaskStatus next, EStreamTaskEvent event, __state_trans_fn fn) { + STaskStateTrans trans = {0}; + trans.state = StreamTaskStatusList[current]; + trans.next = StreamTaskStatusList[next]; + trans.event = event; + trans.pAction = (fn != NULL)? fn : dummyFn; + return trans; +} + +int32_t initStateTransferTable(SStreamTaskSM* pSM) { + if (pSM->pTransList == NULL) { + pSM->pTransList = taosArrayInit(8, sizeof(STaskStateTrans)); + if (pSM->pTransList == NULL) { + return TSDB_CODE_OUT_OF_MEMORY; + } + } + + STaskStateTrans trans = createStateTransform(TASK_STATUS__UNINIT, TASK_STATUS__NORMAL, TASK_EVENT_INIT, streamTaskStartCheckDownstream); + taosArrayPush(pSM->pTransList, &trans); + + trans = createStateTransform(TASK_STATUS__NORMAL, TASK_STATUS__PAUSE, TASK_EVENT_PAUSE, streamTaskDoPause); + taosArrayPush(pSM->pTransList, &trans); + + trans = createStateTransform(TASK_STATUS__PAUSE, TASK_STATUS__NORMAL, TASK_EVENT_RESUME, streamTaskDoResume); + taosArrayPush(pSM->pTransList, &trans); + + trans = createStateTransform(TASK_STATUS__NORMAL, TASK_STATUS__CK, TASK_EVENT_GEN_CHECKPOINT, streamTaskDoCheckpoint); + taosArrayPush(pSM->pTransList, &trans); + + trans = createStateTransform(TASK_STATUS__CK, TASK_STATUS__NORMAL, TASK_EVENT_PAUSE, NULL); + taosArrayPush(pSM->pTransList, &trans); + + return 0; +} \ No newline at end of file From 64ed7e49a2ba6bea41eb8ff7bd3b91b553d6b0cc Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 17 Oct 2023 13:47:55 +0800 Subject: [PATCH 02/41] enh(stream): sm for stream tasks. --- source/libs/stream/inc/streamsm.h | 1 + source/libs/stream/src/streamTaskSm.c | 23 ++++++++++++++++++----- 2 files changed, 19 insertions(+), 5 deletions(-) diff --git a/source/libs/stream/inc/streamsm.h b/source/libs/stream/inc/streamsm.h index 4b87dcfe84..c2a351a297 100644 --- a/source/libs/stream/inc/streamsm.h +++ b/source/libs/stream/inc/streamsm.h @@ -55,6 +55,7 @@ struct SStreamTaskSM { SArray* pTransList; // SArray int64_t stateTs; SStreamTask* pTask; + STaskStateTrans* pActiveTrans; }; typedef struct SStreamEventInfo { diff --git a/source/libs/stream/src/streamTaskSm.c b/source/libs/stream/src/streamTaskSm.c index b6bd53eb87..c219556c6f 100644 --- a/source/libs/stream/src/streamTaskSm.c +++ b/source/libs/stream/src/streamTaskSm.c @@ -38,6 +38,7 @@ static int32_t initStateTransferTable(SStreamTaskSM* pSM); static int32_t dummyFn(SStreamTask* UNUSED_PARAM(p)) { return 0; } static int32_t streamTaskStartCheckDownstream(SStreamTask* pTask) { stDebug("s-task:%s start to check downstream tasks", pTask->id.idStr); + streamTaskCheckDownstream(pTask); return 0; } static int32_t streamTaskDoPause(SStreamTask* pTask) { @@ -84,21 +85,33 @@ SStreamTaskSM* streamCreateStateMachine(SStreamTask* pTask) { taosMemoryFree(pSM); return NULL; } - return pSM; } int32_t taskSMHandleEvent(SStreamTaskSM* pSM, const EStreamTaskEvent* pEvent) { STaskStateTrans* pTrans = streamTaskFindTransform(pSM, pEvent); - qDebug("start to handle event:%d", *pEvent); + ASSERT(pSM->pActiveTrans == NULL); - pSM->current = pTrans->next; + stDebug("start to handle event:%d, state:%s", *pEvent, pSM->current.name); + pSM->pActiveTrans = pTrans; pSM->stateTs = taosGetTimestampMs(); - qDebug("new state:%s from %s", pTrans->next.name, pSM->current.name); - return pTrans->pAction(pSM->pTask); } +int32_t taskSMOnHandleEventSuccess(SStreamTaskSM* pSM) { + STaskStateTrans* pTrans = pSM->pActiveTrans; + EStreamTaskEvent* pEvent = &pTrans->event; + + int64_t el = (taosGetTimestampMs() - pSM->stateTs); + stDebug("handle event:%d completed, elapsd time:%" PRId64 "ms new state:%s from %s", *pEvent, el, pTrans->next.name, + pSM->current.name); + + // todo: add lock + pSM->current = pTrans->next; +} + + + STaskStateTrans createStateTransform(ETaskStatus current, ETaskStatus next, EStreamTaskEvent event, __state_trans_fn fn) { STaskStateTrans trans = {0}; trans.state = StreamTaskStatusList[current]; From 54ec679b5809ddd58245cb520cdbd9a8f93f61d0 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 18 Oct 2023 15:34:53 +0800 Subject: [PATCH 03/41] refactor(stream): do some internal refactor. --- include/libs/stream/tstream.h | 40 +++- source/dnode/snode/src/snode.c | 27 ++- source/dnode/vnode/src/tq/tq.c | 116 +++++------ source/dnode/vnode/src/tq/tqStreamTask.c | 25 ++- source/libs/stream/inc/streamsm.h | 24 +-- source/libs/stream/src/stream.c | 2 +- source/libs/stream/src/streamDispatch.c | 9 +- source/libs/stream/src/streamExec.c | 32 +-- source/libs/stream/src/streamMeta.c | 3 +- source/libs/stream/src/streamStart.c | 235 ++++++++++++++--------- source/libs/stream/src/streamTask.c | 19 +- source/libs/stream/src/streamTaskSm.c | 187 ++++++++++++++---- 12 files changed, 465 insertions(+), 254 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 6eb1ed6e9b..41f260fdc3 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -44,11 +44,19 @@ extern "C" { #define NODE_ROLE_LEADER 0x2 #define NODE_ROLE_FOLLOWER 0x3 +#define HAS_RELATED_FILLHISTORY_TASK(_t) ((_t)->hTaskInfo.id.taskId != 0) +#define CLEAR_RELATED_FILLHISTORY_TASK(_t) \ + do { \ + (_t)->hTaskInfo.id.taskId = 0; \ + (_t)->hTaskInfo.id.streamId = 0; \ + } while (0); + typedef struct SStreamTask SStreamTask; typedef struct SStreamQueue SStreamQueue; typedef struct SStreamTaskSM SStreamTaskSM; #define SSTREAM_TASK_VER 2 + enum { STREAM_STATUS__NORMAL = 0, STREAM_STATUS__STOP, @@ -119,6 +127,21 @@ enum { STREAM_META_OK_TO_STOP = 2, }; +typedef enum EStreamTaskEvent { + TASK_EVENT_INIT = 0x1, + TASK_EVENT_INIT_SCAN_HISTORY = 0x2, + TASK_EVENT_SCANHIST_COMPLETED = 0x3, + TASK_EVENT_START = 0x4, + TASK_EVENT_STOP = 0x5, + TASK_EVENT_GEN_CHECKPOINT = 0x6, + TASK_EVENT_PAUSE = 0x7, + TASK_EVENT_RESUME = 0x8, + TASK_EVENT_HALT = 0x9, + TASK_EVENT_TRANS_STATE = 0xA, + TASK_EVENT_SCAN_TSDB = 0xB, + TASK_EVENT_SCAN_WAL = 0xC, +} EStreamTaskEvent; + typedef struct { int8_t type; } SStreamQueueItem; @@ -351,6 +374,7 @@ typedef struct SHistoryTaskInfo { int32_t tickCount; int32_t retryTimes; int32_t waitInterval; + int64_t haltVer; // offset in wal when halt the stream task } SHistoryTaskInfo; typedef struct STaskOutputInfo { @@ -692,20 +716,28 @@ bool streamTaskShouldStop(const SStreamStatus* pStatus); bool streamTaskShouldPause(const SStreamStatus* pStatus); bool streamTaskIsIdle(const SStreamTask* pTask); -void initRpcMsg(SRpcMsg* pMsg, int32_t msgType, void* pCont, int32_t contLen); +char* createStreamTaskIdStr(int64_t streamId, int32_t taskId); +ETaskStatus streamTaskGetStatus(SStreamTask* pTask, char** pStr); -char* createStreamTaskIdStr(int64_t streamId, int32_t taskId); +void initRpcMsg(SRpcMsg* pMsg, int32_t msgType, void* pCont, int32_t contLen); // recover and fill history void streamTaskCheckDownstream(SStreamTask* pTask); -int32_t streamTaskStartScanHistory(SStreamTask* pTask); +int32_t onNormalTaskReady(SStreamTask* pTask); +int32_t onScanhistoryTaskReady(SStreamTask* pTask); + +//int32_t streamTaskStartScanHistory(SStreamTask* pTask); int32_t streamTaskCheckStatus(SStreamTask* pTask, int32_t upstreamTaskId, int32_t vgId, int64_t stage); int32_t streamTaskUpdateEpsetInfo(SStreamTask* pTask, SArray* pNodeList); void streamTaskResetUpstreamStageInfo(SStreamTask* pTask); bool streamTaskAllUpstreamClosed(SStreamTask* pTask); bool streamTaskSetSchedStatusWait(SStreamTask* pTask); int8_t streamTaskSetSchedStatusActive(SStreamTask* pTask); -int8_t streamTaskSetSchedStatusInActive(SStreamTask* pTask); +int8_t streamTaskSetSchedStatusInactive(SStreamTask* pTask); + +int32_t streamTaskHandleEvent(SStreamTaskSM* pSM, EStreamTaskEvent event); +int32_t streamTaskOnHandleEventSuccess(SStreamTaskSM* pSM); +void streamTaskRestoreStatus(SStreamTask* pTask); int32_t streamTaskStop(SStreamTask* pTask); int32_t streamSendCheckRsp(const SStreamMeta* pMeta, const SStreamTaskCheckReq* pReq, SStreamTaskCheckRsp* pRsp, diff --git a/source/dnode/snode/src/snode.c b/source/dnode/snode/src/snode.c index 6451dba2da..4015ba9c61 100644 --- a/source/dnode/snode/src/snode.c +++ b/source/dnode/snode/src/snode.c @@ -92,11 +92,13 @@ int32_t sndExpandTask(SSnode *pSnode, SStreamTask *pTask, int64_t nextProcessVer } } - qInfo("snode:%d expand stream task, s-task:%s, checkpointId:%" PRId64 " checkpointVer:%" PRId64 " nextProcessVer:%" PRId64 - " child id:%d, level:%d, status:%s fill-history:%d, trigger:%" PRId64 " ms", + char* p = NULL; + streamTaskGetStatus(pTask, &p); + + qInfo("snode:%d expand stream task, s-task:%s, checkpointId:%" PRId64 " checkpointVer:%" PRId64 + " nextProcessVer:%" PRId64 " child id:%d, level:%d, status:%s fill-history:%d, trigger:%" PRId64 " ms", SNODE_HANDLE, pTask->id.idStr, pChkInfo->checkpointId, pChkInfo->checkpointVer, pChkInfo->nextProcessVer, - pTask->info.selfChildId, pTask->info.taskLevel, streamGetTaskStatusStr(pTask->status.taskStatus), - pTask->info.fillHistory, pTask->info.triggerParam); + pTask->info.selfChildId, pTask->info.taskLevel, p, pTask->info.fillHistory, pTask->info.triggerParam); return 0; } @@ -174,10 +176,15 @@ int32_t sndProcessTaskDeployReq(SSnode *pSnode, char *msg, int32_t msgLen) { int32_t numOfTasks = streamMetaGetNumOfTasks(pSnode->pMeta); taosWUnLockLatch(&pSnode->pMeta->lock); - qDebug("snode:%d s-task:%s is deployed on snode and add into meta, status:%s, numOfTasks:%d", SNODE_HANDLE, pTask->id.idStr, - streamGetTaskStatusStr(pTask->status.taskStatus), numOfTasks); - streamTaskCheckDownstream(pTask); + char* p = NULL; + streamTaskGetStatus(pTask, &p); + + qDebug("snode:%d s-task:%s is deployed on snode and add into meta, status:%s, numOfTasks:%d", SNODE_HANDLE, + pTask->id.idStr, p, numOfTasks); + + ASSERT(0); +// streamTaskCheckDownstream(pTask); return 0; } @@ -352,10 +359,10 @@ int32_t sndProcessStreamTaskCheckReq(SSnode *pSnode, SRpcMsg *pMsg) { if (pTask != NULL) { rsp.status = streamTaskCheckStatus(pTask, req.upstreamTaskId, req.upstreamNodeId, req.stage); streamMetaReleaseTask(pSnode->pMeta, pTask); - - const char* pStatus = streamGetTaskStatusStr(pTask->status.taskStatus); + char* p = NULL; + streamTaskGetStatus(pTask, &p); qDebug("s-task:%s status:%s, recv task check req(reqId:0x%" PRIx64 ") task:0x%x (vgId:%d), ready:%d", - pTask->id.idStr, pStatus, rsp.reqId, rsp.upstreamTaskId, rsp.upstreamNodeId, rsp.status); + pTask->id.idStr, p, rsp.reqId, rsp.upstreamTaskId, rsp.upstreamNodeId, rsp.status); } else { rsp.status = TASK_DOWNSTREAM_NOT_READY; qDebug("recv task check(taskId:0x%x not built yet) req(reqId:0x%" PRIx64 ") from task:0x%x (vgId:%d), rsp status %d", diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 1c90812d95..01dea11235 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -817,28 +817,29 @@ int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t nextProcessVer) { } // sink - if (pTask->outputInfo.type == TASK_OUTPUT__SMA) { - pTask->outputInfo.smaSink.vnode = pTq->pVnode; - pTask->outputInfo.smaSink.smaSink = smaHandleRes; - } else if (pTask->outputInfo.type == TASK_OUTPUT__TABLE) { - pTask->outputInfo.tbSink.vnode = pTq->pVnode; - pTask->outputInfo.tbSink.tbSinkFunc = tqSinkDataIntoDstTable; + STaskOutputInfo* pOutputInfo = &pTask->outputInfo; + if (pOutputInfo->type == TASK_OUTPUT__SMA) { + pOutputInfo->smaSink.vnode = pTq->pVnode; + pOutputInfo->smaSink.smaSink = smaHandleRes; + } else if (pOutputInfo->type == TASK_OUTPUT__TABLE) { + pOutputInfo->tbSink.vnode = pTq->pVnode; + pOutputInfo->tbSink.tbSinkFunc = tqSinkDataIntoDstTable; int32_t ver1 = 1; SMetaInfo info = {0}; - code = metaGetInfo(pTq->pVnode->pMeta, pTask->outputInfo.tbSink.stbUid, &info, NULL); + code = metaGetInfo(pTq->pVnode->pMeta, pOutputInfo->tbSink.stbUid, &info, NULL); if (code == TSDB_CODE_SUCCESS) { ver1 = info.skmVer; } - SSchemaWrapper* pschemaWrapper = pTask->outputInfo.tbSink.pSchemaWrapper; - pTask->outputInfo.tbSink.pTSchema = tBuildTSchema(pschemaWrapper->pSchema, pschemaWrapper->nCols, ver1); - if (pTask->outputInfo.tbSink.pTSchema == NULL) { + SSchemaWrapper* pschemaWrapper = pOutputInfo->tbSink.pSchemaWrapper; + pOutputInfo->tbSink.pTSchema = tBuildTSchema(pschemaWrapper->pSchema, pschemaWrapper->nCols, ver1); + if (pOutputInfo->tbSink.pTSchema == NULL) { return -1; } - pTask->outputInfo.tbSink.pTblInfo = tSimpleHashInit(10240, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BIGINT)); - tSimpleHashSetFreeFp(pTask->outputInfo.tbSink.pTblInfo, freePtr); + pOutputInfo->tbSink.pTblInfo = tSimpleHashInit(10240, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BIGINT)); + tSimpleHashSetFreeFp(pOutputInfo->tbSink.pTblInfo, freePtr); } if (pTask->info.taskLevel == TASK_LEVEL__SOURCE) { @@ -863,20 +864,23 @@ int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t nextProcessVer) { pChkInfo->checkpointId, pChkInfo->checkpointVer, pChkInfo->nextProcessVer); } + char* p = NULL; + streamTaskGetStatus(pTask, &p); + if (pTask->info.fillHistory) { tqInfo("vgId:%d expand stream task, s-task:%s, checkpointId:%" PRId64 " checkpointVer:%" PRId64 " nextProcessVer:%" PRId64 " child id:%d, level:%d, status:%s fill-history:%d, related stream task:0x%x trigger:%" PRId64 " ms", vgId, pTask->id.idStr, pChkInfo->checkpointId, pChkInfo->checkpointVer, pChkInfo->nextProcessVer, - pTask->info.selfChildId, pTask->info.taskLevel, streamGetTaskStatusStr(pTask->status.taskStatus), - pTask->info.fillHistory, (int32_t)pTask->streamTaskId.taskId, pTask->info.triggerParam); + pTask->info.selfChildId, pTask->info.taskLevel, p, pTask->info.fillHistory, + (int32_t)pTask->streamTaskId.taskId, pTask->info.triggerParam); } else { tqInfo("vgId:%d expand stream task, s-task:%s, checkpointId:%" PRId64 " checkpointVer:%" PRId64 " nextProcessVer:%" PRId64 " child id:%d, level:%d, status:%s fill-history:%d, related fill-task:0x%x trigger:%" PRId64 " ms", vgId, pTask->id.idStr, pChkInfo->checkpointId, pChkInfo->checkpointVer, pChkInfo->nextProcessVer, - pTask->info.selfChildId, pTask->info.taskLevel, streamGetTaskStatusStr(pTask->status.taskStatus), - pTask->info.fillHistory, (int32_t)pTask->hTaskInfo.id.taskId, pTask->info.triggerParam); + pTask->info.selfChildId, pTask->info.taskLevel, p, pTask->info.fillHistory, + (int32_t)pTask->hTaskInfo.id.taskId, pTask->info.triggerParam); } return 0; @@ -918,9 +922,10 @@ int32_t tqProcessTaskCheckReq(STQ* pTq, SRpcMsg* pMsg) { rsp.status = streamTaskCheckStatus(pTask, req.upstreamTaskId, req.upstreamNodeId, req.stage); streamMetaReleaseTask(pMeta, pTask); - const char* pStatus = streamGetTaskStatusStr(pTask->status.taskStatus); + char* p = NULL; + streamTaskGetStatus(pTask, &p); tqDebug("s-task:%s status:%s, stage:%d recv task check req(reqId:0x%" PRIx64 ") task:0x%x (vgId:%d), check_status:%d", - pTask->id.idStr, pStatus, rsp.oldStage, rsp.reqId, rsp.upstreamTaskId, rsp.upstreamNodeId, rsp.status); + pTask->id.idStr, p, rsp.oldStage, rsp.reqId, rsp.upstreamTaskId, rsp.upstreamNodeId, rsp.status); } else { rsp.status = TASK_DOWNSTREAM_NOT_READY; tqDebug("tq recv task check(taskId:0x%" PRIx64 "-0x%x not built yet) req(reqId:0x%" PRIx64 @@ -1023,11 +1028,9 @@ int32_t tqProcessTaskDeployReq(STQ* pTq, int64_t sversion, char* msg, int32_t ms SStreamTask* p = streamMetaAcquireTask(pStreamMeta, streamId, taskId); bool restored = pTq->pVnode->restored; - if (p != NULL && restored) { - p->execInfo.init = taosGetTimestampMs(); - tqDebug("s-task:%s set the init ts:%"PRId64, p->id.idStr, p->execInfo.init); - - streamTaskCheckDownstream(p); + if (p != NULL && restored && p->info.fillHistory == 0) { + EStreamTaskEvent event = (p->hTaskInfo.id.taskId == 0) ? TASK_EVENT_INIT : TASK_EVENT_INIT_SCAN_HISTORY; + streamTaskHandleEvent(p->status.pSM, event); } else if (!restored) { tqWarn("s-task:%s not launched since vnode(vgId:%d) not ready", p->id.idStr, vgId); } @@ -1061,7 +1064,8 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { // do recovery step1 const char* id = pTask->id.idStr; - const char* pStatus = streamGetTaskStatusStr(pTask->status.taskStatus); + char* pStatus = NULL; + streamTaskGetStatus(pTask, &pStatus); // avoid multi-thread exec while(1) { @@ -1115,7 +1119,7 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { double el = (taosGetTimestampMs() - pTask->execInfo.step1Start) / 1000.0; if (pTask->status.taskStatus == TASK_STATUS__PAUSE) { - int8_t status = streamTaskSetSchedStatusInActive(pTask); + int8_t status = streamTaskSetSchedStatusInactive(pTask); tqDebug("s-task:%s is paused in the step1, elapsed time:%.2fs, sched-status:%d", pTask->id.idStr, el, status); atomic_store_32(&pTask->status.inScanHistorySentinel, 0); @@ -1124,12 +1128,11 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { } // the following procedure should be executed, no matter status is stop/pause or not - tqDebug("s-task:%s scan-history stage(step 1) ended, elapsed time:%.2fs", id, el); + tqDebug("s-task:%s scan-history(step 1) ended, elapsed time:%.2fs", id, el); if (pTask->info.fillHistory) { SVersionRange* pRange = NULL; SStreamTask* pStreamTask = NULL; - bool done = false; // 1. get the related stream task pStreamTask = streamMetaAcquireTask(pMeta, pTask->streamTaskId.streamId, pTask->streamTaskId.taskId); @@ -1148,10 +1151,10 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { } ASSERT(pStreamTask->info.taskLevel == TASK_LEVEL__SOURCE); - +#if 0 // 2. it cannot be paused, when the stream task in TASK_STATUS__SCAN_HISTORY status. Let's wait for the // stream task get ready for scan history data - while (pStreamTask->status.taskStatus == TASK_STATUS__SCAN_HISTORY) { + while (streamTaskGetStatus(pStreamTask, NULL) == TASK_STATUS__SCAN_HISTORY) { tqDebug( "s-task:%s level:%d related stream task:%s(status:%s) not ready for halt, wait for it and recheck in 100ms", id, pTask->info.taskLevel, pStreamTask->id.idStr, streamGetTaskStatusStr(pStreamTask->status.taskStatus)); @@ -1209,21 +1212,26 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { taosThreadMutexUnlock(&pStreamTask->lock); break; } +#endif + + streamTaskHandleEvent(pStreamTask->status.pSM, TASK_EVENT_HALT); + int64_t nextProcessedVer = pStreamTask->hTaskInfo.haltVer; // if it's an source task, extract the last version in wal. pRange = &pTask->dataRange.range; - done = streamHistoryTaskSetVerRangeStep2(pTask, nextProcessedVer); + bool done = streamHistoryTaskSetVerRangeStep2(pTask, nextProcessedVer); pTask->execInfo.step2Start = taosGetTimestampMs(); if (done) { qDebug("s-task:%s scan-history from WAL stage(step 2) ended, elapsed time:%.2fs", id, 0.0); streamTaskPutTranstateIntoInputQ(pTask); + streamTaskRestoreStatus(pTask); - if (pTask->status.taskStatus == TASK_STATUS__PAUSE) { - pTask->status.keepTaskStatus = TASK_STATUS__NORMAL; - qDebug("s-task:%s prev status is %s, update the kept status to be:%s when after step 2", id, - streamGetTaskStatusStr(TASK_STATUS__PAUSE), streamGetTaskStatusStr(pTask->status.keepTaskStatus)); - } +// if (pTask->status.taskStatus == TASK_STATUS__PAUSE) { +// pTask->status.keepTaskStatus = TASK_STATUS__NORMAL; +// qDebug("s-task:%s prev status is %s, update the kept status to be:%s when after step 2", id, +// streamGetTaskStatusStr(TASK_STATUS__PAUSE), streamGetTaskStatusStr(pTask->status.keepTaskStatus)); +// } streamExecTask(pTask); // exec directly } else { @@ -1243,35 +1251,29 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { tqDebug("s-task:%s wal reader start scan WAL verRange:%" PRId64 "-%" PRId64 ", set sched-status:%d", id, dstVer, pTask->dataRange.range.maxVer, TASK_SCHED_STATUS__INACTIVE); - /*int8_t status = */streamTaskSetSchedStatusInActive(pTask); - + /*int8_t status = */streamTaskSetSchedStatusInactive(pTask); +#if 0 // the fill-history task starts to process data in wal, let's set it status to be normal now if (pTask->info.fillHistory == 1 && !streamTaskShouldStop(&pTask->status)) { streamSetStatusNormal(pTask); } +#endif + streamTaskHandleEvent(pTask->status.pSM, TASK_EVENT_SCANHIST_COMPLETED); tqScanWalAsync(pTq, false); } streamMetaReleaseTask(pMeta, pStreamTask); } else { STimeWindow* pWindow = &pTask->dataRange.window; + ASSERT(HAS_RELATED_FILLHISTORY_TASK(pTask)); - if (pTask->hTaskInfo.id.taskId == 0) { - *pWindow = (STimeWindow){INT64_MIN, INT64_MAX}; - tqDebug( - "s-task:%s scan-history in stream time window completed, no related fill-history task, reset the time " - "window:%" PRId64 " - %" PRId64, - id, pWindow->skey, pWindow->ekey); - qStreamInfoResetTimewindowFilter(pTask->exec.pExecutor); - } else { - // when related fill-history task exists, update the fill-history time window only when the - // state transfer is completed. - tqDebug( - "s-task:%s scan-history in stream time window completed, now start to handle data from WAL, start " - "ver:%" PRId64 ", window:%" PRId64 " - %" PRId64, - id, pTask->chkInfo.nextProcessVer, pWindow->skey, pWindow->ekey); - } + // Not update the fill-history time window until the state transfer is completed if the related fill-history task + // exists. + tqDebug( + "s-task:%s scan-history in stream time window completed, now start to handle data from WAL, start " + "ver:%" PRId64 ", window:%" PRId64 " - %" PRId64, + id, pTask->chkInfo.nextProcessVer, pWindow->skey, pWindow->ekey); code = streamTaskScanHistoryDataComplete(pTask); } @@ -1360,17 +1362,17 @@ int32_t tqProcessTaskRunReq(STQ* pTq, SRpcMsg* pMsg) { } SStreamTask* pTask = streamMetaAcquireTask(pTq->pStreamMeta, pReq->streamId, taskId); - if (pTask != NULL) { - // even in halt status, the data in inputQ must be processed - int8_t st = pTask->status.taskStatus; + if (pTask != NULL) { // even in halt status, the data in inputQ must be processed + char* p = NULL; + ETaskStatus st = streamTaskGetStatus(pTask, &p); if (st == TASK_STATUS__NORMAL || st == TASK_STATUS__SCAN_HISTORY || st == TASK_STATUS__CK) { tqDebug("vgId:%d s-task:%s start to process block from inputQ, next checked ver:%" PRId64, vgId, pTask->id.idStr, pTask->chkInfo.nextProcessVer); streamExecTask(pTask); } else { - int8_t status = streamTaskSetSchedStatusInActive(pTask); + int8_t status = streamTaskSetSchedStatusInactive(pTask); tqDebug("vgId:%d s-task:%s ignore run req since not in ready state, status:%s, sched-status:%d", vgId, - pTask->id.idStr, streamGetTaskStatusStr(st), status); + pTask->id.idStr, p, status); } streamMetaReleaseTask(pTq->pStreamMeta, pTask); diff --git a/source/dnode/vnode/src/tq/tqStreamTask.c b/source/dnode/vnode/src/tq/tqStreamTask.c index b9cb22e7a4..29de14e3c3 100644 --- a/source/dnode/vnode/src/tq/tqStreamTask.c +++ b/source/dnode/vnode/src/tq/tqStreamTask.c @@ -99,12 +99,8 @@ int32_t tqCheckAndRunStreamTask(STQ* pTq) { continue; } - pTask->execInfo.init = taosGetTimestampMs(); - tqDebug("s-task:%s start check downstream tasks, set the init ts:%"PRId64, pTask->id.idStr, pTask->execInfo.init); - - streamSetStatusNormal(pTask); - streamTaskCheckDownstream(pTask); - + EStreamTaskEvent event = (pTask->hTaskInfo.id.taskId == 0) ? TASK_EVENT_INIT : TASK_EVENT_INIT_SCAN_HISTORY; + streamTaskHandleEvent(pTask->status.pSM, event); streamMetaReleaseTask(pMeta, pTask); } @@ -113,8 +109,8 @@ int32_t tqCheckAndRunStreamTask(STQ* pTq) { } int32_t tqCheckAndRunStreamTaskAsync(STQ* pTq) { - int32_t vgId = TD_VID(pTq->pVnode); SStreamMeta* pMeta = pTq->pStreamMeta; + int32_t vgId = pMeta->vgId; int32_t numOfTasks = taosArrayGetSize(pMeta->pTaskList); if (numOfTasks == 0) { @@ -328,9 +324,11 @@ static bool taskReadyForDataFromWal(SStreamTask* pTask) { } // not in ready state, do not handle the data from wal - int32_t status = pTask->status.taskStatus; - if (status != TASK_STATUS__NORMAL) { - tqTrace("s-task:%s not ready for submit block in wal, status:%s", pTask->id.idStr, streamGetTaskStatusStr(status)); +// int32_t status = pTask->status.taskStatus; + char* p = NULL; + int32_t status = streamTaskGetStatus(pTask, &p); + if (streamTaskGetStatus(pTask, &p) != TASK_STATUS__NORMAL) { + tqTrace("s-task:%s not ready for submit block in wal, status:%s", pTask->id.idStr, p); return false; } @@ -449,9 +447,10 @@ int32_t doScanWalForAllTasks(SStreamMeta* pStreamMeta, bool* pScanIdle) { taosThreadMutexLock(&pTask->lock); - const char* pStatus = streamGetTaskStatusStr(pTask->status.taskStatus); - if (pTask->status.taskStatus != TASK_STATUS__NORMAL) { - tqDebug("s-task:%s not ready for submit block from wal, status:%s", pTask->id.idStr, pStatus); + char* p = NULL; + ETaskStatus status = streamTaskGetStatus(pTask, &p); + if (status != TASK_STATUS__NORMAL) { + tqDebug("s-task:%s not ready for submit block from wal, status:%s", pTask->id.idStr, p); taosThreadMutexUnlock(&pTask->lock); streamMetaReleaseTask(pStreamMeta, pTask); continue; diff --git a/source/libs/stream/inc/streamsm.h b/source/libs/stream/inc/streamsm.h index c2a351a297..39ea2a77f8 100644 --- a/source/libs/stream/inc/streamsm.h +++ b/source/libs/stream/inc/streamsm.h @@ -28,34 +28,26 @@ typedef struct SStreamTaskState { char* name; } SStreamTaskState; -typedef enum EStreamTaskEvent { - TASK_EVENT_INIT = 0x1, - TASK_EVENT_START = 0x2, - TASK_EVENT_STOP = 0x3, - TASK_EVENT_GEN_CHECKPOINT = 0x4, - TASK_EVENT_PAUSE = 0x5, - TASK_EVENT_RESUME = 0x6, - TASK_EVENT_HALT = 0x7, - TASK_EVENT_TRANS_STATE = 0x8, - TASK_EVENT_SCAN_TSDB = 0x9, - TASK_EVENT_SCAN_WAL = 0x10, -} EStreamTaskEvent; - typedef int32_t (*__state_trans_fn)(SStreamTask*); +typedef int32_t (*__state_trans_succ_fn)(SStreamTask*); typedef struct STaskStateTrans { + bool autoInvokeEndFn; SStreamTaskState state; EStreamTaskEvent event; SStreamTaskState next; + __state_trans_fn preAction; __state_trans_fn pAction; + __state_trans_succ_fn pSuccAction; } STaskStateTrans; struct SStreamTaskSM { - SStreamTaskState current; - SArray* pTransList; // SArray - int64_t stateTs; SStreamTask* pTask; + SArray* pTransList; // SArray STaskStateTrans* pActiveTrans; + int64_t startTs; + SStreamTaskState current; + SStreamTaskState prev; }; typedef struct SStreamEventInfo { diff --git a/source/libs/stream/src/stream.c b/source/libs/stream/src/stream.c index 03ba796b2c..8910616f01 100644 --- a/source/libs/stream/src/stream.c +++ b/source/libs/stream/src/stream.c @@ -127,7 +127,7 @@ int32_t streamSchedExec(SStreamTask* pTask) { SStreamTaskRunReq* pRunReq = rpcMallocCont(sizeof(SStreamTaskRunReq)); if (pRunReq == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; - /*int8_t status = */streamTaskSetSchedStatusInActive(pTask); + /*int8_t status = */streamTaskSetSchedStatusInactive(pTask); stError("failed to create msg to aunch s-task:%s, reason out of memory", pTask->id.idStr); return -1; } diff --git a/source/libs/stream/src/streamDispatch.c b/source/libs/stream/src/streamDispatch.c index a7a06dd884..39afc3c98a 100644 --- a/source/libs/stream/src/streamDispatch.c +++ b/source/libs/stream/src/streamDispatch.c @@ -662,8 +662,10 @@ int32_t streamDispatchScanHistoryFinishMsg(SStreamTask* pTask) { int32_t numOfVgs = taosArrayGetSize(vgInfo); pTask->notReadyTasks = numOfVgs; + char* p = NULL; + streamTaskGetStatus(pTask, &p); stDebug("s-task:%s send scan-history data complete msg to downstream (shuffle-dispatch) %d tasks, status:%s", pTask->id.idStr, - numOfVgs, streamGetTaskStatusStr(pTask->status.taskStatus)); + numOfVgs, p); for (int32_t i = 0; i < numOfVgs; i++) { SVgroupInfo* pVgInfo = taosArrayGet(vgInfo, i); req.downstreamTaskId = pVgInfo->taskId; @@ -775,8 +777,9 @@ int32_t doDispatchScanHistoryFinishMsg(SStreamTask* pTask, const SStreamScanHist initRpcMsg(&msg, TDMT_VND_STREAM_SCAN_HISTORY_FINISH, buf, tlen + sizeof(SMsgHead)); tmsgSendReq(pEpSet, &msg); - const char* pStatus = streamGetTaskStatusStr(pTask->status.taskStatus); - stDebug("s-task:%s status:%s dispatch scan-history finish msg to taskId:0x%x (vgId:%d)", pTask->id.idStr, pStatus, + char* p = NULL; + streamTaskGetStatus(pTask, &p); + stDebug("s-task:%s status:%s dispatch scan-history finish msg to taskId:0x%x (vgId:%d)", pTask->id.idStr, p, pReq->downstreamTaskId, vgId); return 0; } diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index c49c647906..ca67e19901 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -318,12 +318,13 @@ int32_t streamDoTransferStateToStreamTask(SStreamTask* pTask) { // todo. the dropping status should be append to the status after the halt completed. // It must be halted for a source stream task, since when the related scan-history-data task start scan the history // for the step 2. - int8_t status = pStreamTask->status.taskStatus; + int8_t status = streamTaskGetStatus(pStreamTask, NULL);//pStreamTask->status.taskStatus; if (pStreamTask->info.taskLevel == TASK_LEVEL__SOURCE) { ASSERT(status == TASK_STATUS__HALT || status == TASK_STATUS__DROPPING); } else { ASSERT(status == TASK_STATUS__NORMAL); - pStreamTask->status.taskStatus = TASK_STATUS__HALT; + streamTaskHandleEvent(pStreamTask->status.pSM, TASK_EVENT_HALT); +// pStreamTask->status.taskStatus = TASK_STATUS__HALT; stDebug("s-task:%s halt by related fill-history task:%s", pStreamTask->id.idStr, pTask->id.idStr); } @@ -337,9 +338,9 @@ int32_t streamDoTransferStateToStreamTask(SStreamTask* pTask) { if (pStreamTask->info.taskLevel == TASK_LEVEL__SOURCE) { // update the scan data range for source task. stDebug("s-task:%s level:%d stream task window %" PRId64 " - %" PRId64 " update to %" PRId64 " - %" PRId64 - ", status:%s, sched-status:%d", - pStreamTask->id.idStr, TASK_LEVEL__SOURCE, pTimeWindow->skey, pTimeWindow->ekey, INT64_MIN, - pTimeWindow->ekey, streamGetTaskStatusStr(TASK_STATUS__NORMAL), pStreamTask->status.schedStatus); + ", status:normal, sched-status:%d", + pStreamTask->id.idStr, TASK_LEVEL__SOURCE, pTimeWindow->skey, pTimeWindow->ekey, INT64_MIN, + pTimeWindow->ekey, pStreamTask->status.schedStatus); } else { stDebug("s-task:%s no need to update time window for non-source task", pStreamTask->id.idStr); } @@ -362,8 +363,7 @@ int32_t streamDoTransferStateToStreamTask(SStreamTask* pTask) { streamBuildAndSendDropTaskMsg(pTask->pMsgCb, pMeta->vgId, &pTask->id); // 5. clear the link between fill-history task and stream task info - pStreamTask->hTaskInfo.id.taskId = 0; - pStreamTask->hTaskInfo.id.streamId = 0; + CLEAR_RELATED_FILLHISTORY_TASK(pStreamTask); // 6. save to disk taosWLockLatch(&pMeta->lock); @@ -505,7 +505,7 @@ int32_t streamProcessTranstateBlock(SStreamTask* pTask, SStreamDataBlock* pBlock code = streamTransferStateToStreamTask(pTask); if (code != TSDB_CODE_SUCCESS) { - /*int8_t status = */ streamTaskSetSchedStatusInActive(pTask); + /*int8_t status = */ streamTaskSetSchedStatusInactive(pTask); } } @@ -592,8 +592,9 @@ int32_t streamExecForAll(SStreamTask* pTask) { // todo other thread may change the status // do nothing after sync executor state to storage backend, untill the vnode-level checkpoint is completed. if (type == STREAM_INPUT__CHECKPOINT) { - stDebug("s-task:%s checkpoint block received, set status:%s", pTask->id.idStr, - streamGetTaskStatusStr(pTask->status.taskStatus)); + char* p = NULL; + streamTaskGetStatus(pTask, &p); + stDebug("s-task:%s checkpoint block received, set status:%s", pTask->id.idStr, p); streamTaskBuildCheckpoint(pTask); return 0; } @@ -628,15 +629,18 @@ int32_t streamExecTask(SStreamTask* pTask) { atomic_store_8(&pTask->status.schedStatus, TASK_SCHED_STATUS__INACTIVE); taosThreadMutexUnlock(&pTask->lock); - stDebug("s-task:%s exec completed, status:%s, sched-status:%d", id, - streamGetTaskStatusStr(pTask->status.taskStatus), pTask->status.schedStatus); + char* p = NULL; + streamTaskGetStatus(pTask, &p); + stDebug("s-task:%s exec completed, status:%s, sched-status:%d", id, p, pTask->status.schedStatus); return 0; } taosThreadMutexUnlock(&pTask->lock); } } else { - stDebug("s-task:%s already started to exec by other thread, status:%s, sched-status:%d", id, - streamGetTaskStatusStr(pTask->status.taskStatus), pTask->status.schedStatus); + char* p = NULL; + streamTaskGetStatus(pTask, &p); + stDebug("s-task:%s already started to exec by other thread, status:%s, sched-status:%d", id, p, + pTask->status.schedStatus); } return 0; diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index f7b0cdb0f1..2773912437 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -518,8 +518,7 @@ int32_t streamMetaUnregisterTask(SStreamMeta* pMeta, int64_t streamId, int32_t t } taosWUnLockLatch(&pMeta->lock); - stDebug("s-task:0x%x set task status:%s and start to unregister it", taskId, - streamGetTaskStatusStr(TASK_STATUS__DROPPING)); + stDebug("s-task:0x%x set task status:dropping and start to unregister it", taskId); while (1) { taosRLockLatch(&pMeta->lock); diff --git a/source/libs/stream/src/streamStart.c b/source/libs/stream/src/streamStart.c index 061efc94fb..8ebc1c0094 100644 --- a/source/libs/stream/src/streamStart.c +++ b/source/libs/stream/src/streamStart.c @@ -17,6 +17,7 @@ #include "trpc.h" #include "ttimer.h" #include "wal.h" +#include "streamsm.h" typedef struct SLaunchHTaskInfo { SStreamMeta* pMeta; @@ -34,16 +35,17 @@ static void streamTaskSetRangeStreamCalc(SStreamTask* pTask); static int32_t initScanHistoryReq(SStreamTask* pTask, SStreamScanHistoryReq* pReq, int8_t igUntreated); static SLaunchHTaskInfo* createHTaskLaunchInfo(SStreamMeta* pMeta, int64_t streamId, int32_t taskId); static void tryLaunchHistoryTask(void* param, void* tmrId); +static int32_t updateTaskReadyInMeta(SStreamTask* pTask); -static void streamTaskSetReady(SStreamTask* pTask, int32_t numOfReqs) { - SStreamMeta* pMeta = pTask->pMeta; - int32_t vgId = pMeta->vgId; +int32_t streamTaskSetReady(SStreamTask* pTask) { + char* p = NULL; + int32_t numOfDowns = streamTaskGetNumOfDownstream(pTask); + ETaskStatus status = streamTaskGetStatus(pTask, &p); - if (pTask->status.taskStatus == TASK_STATUS__SCAN_HISTORY && pTask->info.taskLevel != TASK_LEVEL__SOURCE) { + if (status == TASK_STATUS__SCAN_HISTORY && pTask->info.taskLevel != TASK_LEVEL__SOURCE) { pTask->numOfWaitingUpstream = taosArrayGetSize(pTask->upstreamInfo.pList); stDebug("s-task:%s level:%d task wait for %d upstream tasks complete scan-history procedure, status:%s", - pTask->id.idStr, pTask->info.taskLevel, pTask->numOfWaitingUpstream, - streamGetTaskStatusStr(pTask->status.taskStatus)); + pTask->id.idStr, pTask->info.taskLevel, pTask->numOfWaitingUpstream, p); } ASSERT(pTask->status.downstreamReady == 0); @@ -52,34 +54,10 @@ static void streamTaskSetReady(SStreamTask* pTask, int32_t numOfReqs) { pTask->execInfo.start = taosGetTimestampMs(); int64_t el = (pTask->execInfo.start - pTask->execInfo.init); stDebug("s-task:%s all %d downstream ready, init completed, elapsed time:%" PRId64 "ms, task status:%s", - pTask->id.idStr, numOfReqs, el, streamGetTaskStatusStr(pTask->status.taskStatus)); + pTask->id.idStr, numOfDowns, el, p); - taosWLockLatch(&pMeta->lock); - - STaskId id = streamTaskExtractKey(pTask); - taosHashPut(pMeta->startInfo.pReadyTaskSet, &id, sizeof(id), NULL, 0); - - int32_t numOfTotal = streamMetaGetNumOfTasks(pMeta); - - if (taosHashGetSize(pMeta->startInfo.pReadyTaskSet) == numOfTotal) { - STaskStartInfo* pStartInfo = &pMeta->startInfo; - pStartInfo->readyTs = pTask->execInfo.start; - - if (pStartInfo->startTs != 0) { - pStartInfo->elapsedTime = pStartInfo->readyTs - pStartInfo->startTs; - } else { - pStartInfo->elapsedTime = 0; - } - - streamMetaResetStartInfo(pStartInfo); - - stDebug("vgId:%d all %d task(s) are started successfully, last ready task:%s level:%d, startTs:%" PRId64 - ", readyTs:%" PRId64 " total elapsed time:%.2fs", - vgId, numOfTotal, pTask->id.idStr, pTask->info.taskLevel, pStartInfo->startTs, pStartInfo->readyTs, - pStartInfo->elapsedTime / 1000.0); - } - - taosWUnLockLatch(&pMeta->lock); + updateTaskReadyInMeta(pTask); + return TSDB_CODE_SUCCESS; } int32_t streamStartScanHistoryAsync(SStreamTask* pTask, int8_t igUntreated) { @@ -114,28 +92,19 @@ static int32_t doStartScanHistoryTask(SStreamTask* pTask) { } int32_t streamTaskStartScanHistory(SStreamTask* pTask) { - ASSERT(pTask->status.downstreamReady == 1); + ASSERT(pTask->status.downstreamReady == 1 && streamTaskGetStatus(pTask, NULL) == TASK_STATUS__SCAN_HISTORY); if (pTask->info.taskLevel == TASK_LEVEL__SOURCE) { - if (pTask->status.taskStatus == TASK_STATUS__SCAN_HISTORY) { - return doStartScanHistoryTask(pTask); - } else { - ASSERT(pTask->status.taskStatus == TASK_STATUS__NORMAL); - stDebug("s-task:%s no need to scan-history data, status:%s, sched-status:%d, ver:%" PRId64, pTask->id.idStr, - streamGetTaskStatusStr(pTask->status.taskStatus), pTask->status.schedStatus, - walReaderGetCurrentVer(pTask->exec.pWalReader)); - streamTaskEnablePause(pTask); - } + return doStartScanHistoryTask(pTask); } else if (pTask->info.taskLevel == TASK_LEVEL__AGG) { if (pTask->info.fillHistory) { streamSetParamForScanHistory(pTask); streamTaskEnablePause(pTask); } } else if (pTask->info.taskLevel == TASK_LEVEL__SINK) { - if (pTask->status.taskStatus == TASK_STATUS__SCAN_HISTORY) { - stDebug("s-task:%s sink task do nothing to handle scan-history", pTask->id.idStr); - } + stDebug("s-task:%s sink task do nothing to handle scan-history", pTask->id.idStr); } + return 0; } @@ -152,6 +121,8 @@ static int32_t doCheckDownstreamStatus(SStreamTask* pTask) { .stage = pTask->pMeta->stage, }; + ASSERT(pTask->status.downstreamReady == 0); + // serialize streamProcessScanHistoryFinishRsp if (pTask->outputInfo.type == TASK_OUTPUT__FIXED_DISPATCH) { req.reqId = tGenIdPI64(); @@ -187,11 +158,7 @@ static int32_t doCheckDownstreamStatus(SStreamTask* pTask) { } } else { stDebug("s-task:%s (vgId:%d) set downstream ready, since no downstream", pTask->id.idStr, pTask->info.nodeId); - - streamTaskSetReady(pTask, 0); - streamTaskSetRangeStreamCalc(pTask); - streamTaskStartScanHistory(pTask); - streamLaunchFillHistoryTask(pTask); + streamTaskOnHandleEventSuccess(pTask->status.pSM); } return 0; @@ -288,8 +255,57 @@ int32_t streamTaskCheckStatus(SStreamTask* pTask, int32_t upstreamTaskId, int32_ } } -static void doProcessDownstreamReadyRsp(SStreamTask* pTask, int32_t numOfReqs) { - streamTaskSetReady(pTask, numOfReqs); +int32_t onNormalTaskReady(SStreamTask* pTask) { + const char* id = pTask->id.idStr; + + streamTaskSetReady(pTask); + streamTaskSetRangeStreamCalc(pTask); + + char* p = NULL; + ETaskStatus status = streamTaskGetStatus(pTask, &p); + ASSERT(status == TASK_STATUS__NORMAL); + + // todo refactor: remove this later + if (pTask->info.fillHistory == 1) { + stDebug("s-task:%s fill-history is set normal when start it, try to remove it,set it task to be dropping", id); + pTask->status.taskStatus = TASK_STATUS__DROPPING; + ASSERT(pTask->hTaskInfo.id.taskId == 0); + } + + stDebug("s-task:%s no need to scan-history data, status:%s, sched-status:%d, ready for data from wal ver:%" PRId64, + id, p, pTask->status.schedStatus, walReaderGetCurrentVer(pTask->exec.pWalReader)); + + streamTaskEnablePause(pTask); + return TSDB_CODE_SUCCESS; +} + +int32_t onScanhistoryTaskReady(SStreamTask* pTask) { + const char* id = pTask->id.idStr; + + // set the state to be ready + streamTaskSetReady(pTask); + streamTaskSetRangeStreamCalc(pTask); + + char* p = NULL; + ETaskStatus status = streamTaskGetStatus(pTask, &p); + ASSERT(status == TASK_STATUS__SCAN_HISTORY); + + stDebug("s-task:%s enter into scan-history data stage, status:%s", id, p); + streamTaskStartScanHistory(pTask); + + // start the related fill-history task, when current task is ready + if (pTask->hTaskInfo.id.taskId != 0) { + streamLaunchFillHistoryTask(pTask); + } + + return TSDB_CODE_SUCCESS; +} + +// todo: refactor this function. +static void doProcessDownstreamReadyRsp(SStreamTask* pTask) { + streamTaskOnHandleEventSuccess(pTask->status.pSM); + +#if 0 const char* id = pTask->id.idStr; int8_t status = pTask->status.taskStatus; @@ -314,6 +330,7 @@ static void doProcessDownstreamReadyRsp(SStreamTask* pTask, int32_t numOfReqs) { streamTaskEnablePause(pTask); } } +#endif } int32_t streamProcessCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* pRsp) { @@ -349,7 +366,7 @@ int32_t streamProcessCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* pRs taosArrayDestroy(pTask->checkReqIds); pTask->checkReqIds = NULL; - doProcessDownstreamReadyRsp(pTask, numOfReqs); + doProcessDownstreamReadyRsp(pTask); } else { int32_t total = taosArrayGetSize(pTask->outputInfo.shuffleDispatcher.dbInfo.pVgroupInfos); stDebug("s-task:%s (vgId:%d) recv check rsp from task:0x%x (vgId:%d) status:%d, total:%d not ready:%d", id, @@ -361,7 +378,7 @@ int32_t streamProcessCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* pRs return -1; } - doProcessDownstreamReadyRsp(pTask, 1); + doProcessDownstreamReadyRsp(pTask); } } else { // not ready, wait for 100ms and retry if (pRsp->status == TASK_DOWNSTREAM_NOT_LEADER) { @@ -438,18 +455,6 @@ int32_t streamSetStatusNormal(SStreamTask* pTask) { } } -int32_t streamSetStatusUnint(SStreamTask* pTask) { - int32_t status = atomic_load_8(&pTask->status.taskStatus); - if (status == TASK_STATUS__DROPPING) { - stError("s-task:%s cannot be set uninit, since in dropping state", pTask->id.idStr); - return -1; - } else { - stDebug("s-task:%s set task status to be uninit, prev:%s", pTask->id.idStr, streamGetTaskStatusStr(status)); - atomic_store_8(&pTask->status.taskStatus, TASK_STATUS__UNINIT); - return 0; - } -} - // source int32_t streamSetParamForStreamScannerStep1(SStreamTask* pTask, SVersionRange *pVerRange, STimeWindow* pWindow) { return qStreamSourceScanParamForHistoryScanStep1(pTask->exec.pExecutor, pVerRange, pWindow); @@ -515,9 +520,11 @@ int32_t streamProcessScanHistoryFinishReq(SStreamTask* pTask, SStreamScanHistory int32_t taskLevel = pTask->info.taskLevel; ASSERT(taskLevel == TASK_LEVEL__AGG || taskLevel == TASK_LEVEL__SINK); - if (pTask->status.taskStatus != TASK_STATUS__SCAN_HISTORY) { + char* p = NULL; + ETaskStatus status = streamTaskGetStatus(pTask, &p); + if (status != TASK_STATUS__SCAN_HISTORY) { stError("s-task:%s not in scan-history status, status:%s return upstream:0x%x scan-history finish directly", - pTask->id.idStr, streamGetTaskStatusStr(pTask->status.taskStatus), pReq->upstreamTaskId); + pTask->id.idStr, p, pReq->upstreamTaskId); void* pBuf = NULL; int32_t len = 0; @@ -571,12 +578,12 @@ int32_t streamProcessScanHistoryFinishReq(SStreamTask* pTask, SStreamScanHistory } int32_t streamProcessScanHistoryFinishRsp(SStreamTask* pTask) { - ASSERT(pTask->status.taskStatus == TASK_STATUS__SCAN_HISTORY); + ASSERT(/*pTask->status.taskStatus*/ streamTaskGetStatus(pTask, NULL) == TASK_STATUS__SCAN_HISTORY); SStreamMeta* pMeta = pTask->pMeta; // execute in the scan history complete call back msg, ready to process data from inputQ - streamSetStatusNormal(pTask); - streamTaskSetSchedStatusInActive(pTask); + streamTaskHandleEvent(pTask->status.pSM, TASK_EVENT_SCANHIST_COMPLETED); + streamTaskSetSchedStatusInactive(pTask); taosWLockLatch(&pMeta->lock); streamMetaSaveTask(pMeta, pTask); @@ -604,15 +611,15 @@ static void checkFillhistoryTaskStatus(SStreamTask* pTask, SStreamTask* pHTask) if (pTask->info.taskLevel == TASK_LEVEL__SOURCE) { stDebug("s-task:%s set the launch condition for fill-history s-task:%s, window:%" PRId64 " - %" PRId64 - " ver range:%" PRId64 " - %" PRId64", init:%"PRId64, + " verRange:%" PRId64 " - %" PRId64", init:%"PRId64, pTask->id.idStr, pHTask->id.idStr, pRange->window.skey, pRange->window.ekey, pRange->range.minVer, pRange->range.maxVer, pHTask->execInfo.init); } else { - stDebug("s-task:%s no fill history condition for non-source task:%s", pTask->id.idStr, pHTask->id.idStr); + stDebug("s-task:%s no fill-history condition for non-source task:%s", pTask->id.idStr, pHTask->id.idStr); } // check if downstream tasks have been ready - doCheckDownstreamStatus(pHTask); + streamTaskHandleEvent(pHTask->status.pSM, TASK_EVENT_INIT_SCAN_HISTORY); } static void tryLaunchHistoryTask(void* param, void* tmrId) { @@ -625,11 +632,11 @@ static void tryLaunchHistoryTask(void* param, void* tmrId) { ASSERT((*ppTask)->status.timerActive >= 1); if (streamTaskShouldStop(&(*ppTask)->status)) { - const char* pStatus = streamGetTaskStatusStr((*ppTask)->status.taskStatus); - + char* p = NULL; + streamTaskGetStatus((*ppTask), &p); int32_t ref = atomic_sub_fetch_32(&(*ppTask)->status.timerActive, 1); stDebug("s-task:%s status:%s should stop, quit launch fill-history task timer, retry:%d, ref:%d", - (*ppTask)->id.idStr, pStatus, (*ppTask)->hTaskInfo.retryTimes, ref); + (*ppTask)->id.idStr, p, (*ppTask)->hTaskInfo.retryTimes, ref); taosMemoryFree(pInfo); taosWUnLockLatch(&pMeta->lock); @@ -666,8 +673,10 @@ static void tryLaunchHistoryTask(void* param, void* tmrId) { // abort the timer if intend to stop task SStreamTask* pHTask = streamMetaAcquireTask(pMeta, pHTaskInfo->id.streamId, pHTaskInfo->id.taskId); if (pHTask == NULL && (!streamTaskShouldStop(&pTask->status))) { - const char* p = streamGetTaskStatusStr(pTask->status.taskStatus); - int32_t hTaskId = pHTaskInfo->id.taskId; + char* p = NULL; + int32_t hTaskId = pHTaskInfo->id.taskId; + + streamTaskGetStatus(pTask, &p); stDebug( "s-task:%s status:%s failed to launch fill-history task:0x%x, retry launch:%dms, retryCount:%d", pTask->id.idStr, p, hTaskId, pHTaskInfo->waitInterval, pHTaskInfo->retryTimes); @@ -713,11 +722,8 @@ SLaunchHTaskInfo* createHTaskLaunchInfo(SStreamMeta* pMeta, int64_t streamId, in int32_t streamLaunchFillHistoryTask(SStreamTask* pTask) { SStreamMeta* pMeta = pTask->pMeta; int32_t hTaskId = pTask->hTaskInfo.id.taskId; - if (hTaskId == 0) { - return TSDB_CODE_SUCCESS; - } - ASSERT(pTask->status.downstreamReady == 1); + ASSERT((hTaskId != 0) && (pTask->status.downstreamReady == 1)); stDebug("s-task:%s start to launch related fill-history task:0x%" PRIx64 "-0x%x", pTask->id.idStr, pTask->hTaskInfo.id.streamId, hTaskId); @@ -931,12 +937,12 @@ void streamTaskSetRangeStreamCalc(SStreamTask* pTask) { // only the downstream tasks are ready, set the task to be ready to work. void streamTaskCheckDownstream(SStreamTask* pTask) { - if (pTask->info.fillHistory) { - stDebug("s-task:%s fill history task, wait for being launched", pTask->id.idStr); - return; - } +// if (pTask->info.fillHistory) { +// ASSERT(0); +// stDebug("s-task:%s fill history task, wait for being launched", pTask->id.idStr); +// return; +// } - ASSERT(pTask->status.downstreamReady == 0); doCheckDownstreamStatus(pTask); } @@ -1047,13 +1053,50 @@ void streamTaskEnablePause(SStreamTask* pTask) { void streamTaskResumeFromHalt(SStreamTask* pTask) { const char* id = pTask->id.idStr; - int8_t status = pTask->status.taskStatus; - if (status != TASK_STATUS__HALT) { - stError("s-task:%s not in halt status, status:%s", id, streamGetTaskStatusStr(status)); - return; + char* p = NULL; + + ASSERT(streamTaskGetStatus(pTask, NULL) == TASK_STATUS__HALT); +// int8_t status = pTask->status.taskStatus; +// if (status != TASK_STATUS__HALT) { +// stError("s-task:%s not in halt status, status:%s", id, streamGetTaskStatusStr(status)); +// return; +// } + +// pTask->status.taskStatus = pTask->status.keepTaskStatus; +// pTask->status.keepTaskStatus = TASK_STATUS__NORMAL; + streamTaskRestoreStatus(pTask); + streamTaskGetStatus(pTask, &p); + stDebug("s-task:%s resume from halt, current status:%s", id, p); +} + +int32_t updateTaskReadyInMeta(SStreamTask* pTask) { + SStreamMeta* pMeta = pTask->pMeta; + + taosWLockLatch(&pMeta->lock); + + STaskId id = streamTaskExtractKey(pTask); + taosHashPut(pMeta->startInfo.pReadyTaskSet, &id, sizeof(id), NULL, 0); + + int32_t numOfTotal = streamMetaGetNumOfTasks(pMeta); + + if (taosHashGetSize(pMeta->startInfo.pReadyTaskSet) == numOfTotal) { + STaskStartInfo* pStartInfo = &pMeta->startInfo; + pStartInfo->readyTs = pTask->execInfo.start; + + if (pStartInfo->startTs != 0) { + pStartInfo->elapsedTime = pStartInfo->readyTs - pStartInfo->startTs; + } else { + pStartInfo->elapsedTime = 0; + } + + streamMetaResetStartInfo(pStartInfo); + + stDebug("vgId:%d all %d task(s) are started successfully, last ready task:%s level:%d, startTs:%" PRId64 + ", readyTs:%" PRId64 " total elapsed time:%.2fs", + pMeta->vgId, numOfTotal, pTask->id.idStr, pTask->info.taskLevel, pStartInfo->startTs, pStartInfo->readyTs, + pStartInfo->elapsedTime / 1000.0); } - pTask->status.taskStatus = pTask->status.keepTaskStatus; - pTask->status.keepTaskStatus = TASK_STATUS__NORMAL; - stDebug("s-task:%s resume from halt, current status:%s", id, streamGetTaskStatusStr(pTask->status.taskStatus)); + taosWUnLockLatch(&pMeta->lock); + return TSDB_CODE_SUCCESS; } diff --git a/source/libs/stream/src/streamTask.c b/source/libs/stream/src/streamTask.c index 5898a02ea1..4b39d979b6 100644 --- a/source/libs/stream/src/streamTask.c +++ b/source/libs/stream/src/streamTask.c @@ -304,11 +304,11 @@ static void freeUpstreamItem(void* p) { void tFreeStreamTask(SStreamTask* pTask) { int32_t taskId = pTask->id.taskId; + char* p = NULL; + streamTaskGetStatus(pTask, &p); STaskExecStatisInfo* pStatis = &pTask->execInfo; - - stDebug("start to free s-task:0x%x, %p, state:%p, status:%s", taskId, pTask, pTask->pState, - streamGetTaskStatusStr(pTask->status.taskStatus)); + stDebug("start to free s-task:0x%x, %p, state:%p, status:%s", taskId, pTask, pTask->pState, p); stDebug("s-task:0x%x task exec summary: create:%" PRId64 ", init:%" PRId64 ", start:%" PRId64 ", updateCount:%d latestUpdate:%" PRId64 ", latestCheckPoint:%" PRId64 ", ver:%" PRId64 @@ -417,6 +417,13 @@ int32_t streamTaskInit(SStreamTask* pTask, SStreamMeta* pMeta, SMsgCb* pMsgCb, i return TSDB_CODE_OUT_OF_MEMORY; } + pTask->status.pSM = streamCreateStateMachine(pTask); + if (pTask->status.pSM == NULL) { + stError("s-task:%s failed create state-machine for stream task, initialization failed, code:%s", pTask->id.idStr, + tstrerror(terrno)); + return terrno; + } + pTask->execInfo.created = taosGetTimestampMs(); pTask->inputInfo.status = TASK_INPUT_STATUS__NORMAL; pTask->outputq.status = TASK_OUTPUT_STATUS__NORMAL; @@ -463,7 +470,9 @@ int32_t streamTaskGetNumOfDownstream(const SStreamTask* pTask) { return 0; } else { int32_t type = pTask->outputInfo.type; - if (type == TASK_OUTPUT__FIXED_DISPATCH || type == TASK_OUTPUT__TABLE) { + if (type == TASK_OUTPUT__TABLE) { + return 0; + } else if (type == TASK_OUTPUT__FIXED_DISPATCH) { return 1; } else { SArray* vgInfo = pTask->outputInfo.shuffleDispatcher.dbInfo.pVgroupInfos; @@ -677,7 +686,7 @@ int8_t streamTaskSetSchedStatusActive(SStreamTask* pTask) { return status; } -int8_t streamTaskSetSchedStatusInActive(SStreamTask* pTask) { +int8_t streamTaskSetSchedStatusInactive(SStreamTask* pTask) { taosThreadMutexLock(&pTask->lock); int8_t status = pTask->status.schedStatus; ASSERT(status == TASK_SCHED_STATUS__WAITING || status == TASK_SCHED_STATUS__ACTIVE || diff --git a/source/libs/stream/src/streamTaskSm.c b/source/libs/stream/src/streamTaskSm.c index c219556c6f..b4b4d026c2 100644 --- a/source/libs/stream/src/streamTaskSm.c +++ b/source/libs/stream/src/streamTaskSm.c @@ -13,7 +13,6 @@ * along with this program. If not, see . */ -#include "executor.h" #include "streamInt.h" #include "tmisce.h" #include "tstream.h" @@ -32,19 +31,42 @@ SStreamTaskState StreamTaskStatusList[8] = { {.state = TASK_STATUS__CK, .name = "checkpoint"}, }; -static STaskStateTrans createStateTransform(ETaskStatus current, ETaskStatus next, EStreamTaskEvent event, __state_trans_fn fn); -static int32_t initStateTransferTable(SStreamTaskSM* pSM); +SStreamEventInfo StreamTaskEventList[8] = { + {}, // dummy event, place holder + {.event = TASK_EVENT_INIT, .name = "initialize"}, + {.event = TASK_EVENT_INIT_SCAN_HISTORY, .name = "scan-history-initialize"}, + {.event = TASK_EVENT_SCANHIST_COMPLETED, .name = "scan-history-completed"}, +}; -static int32_t dummyFn(SStreamTask* UNUSED_PARAM(p)) { return 0; } -static int32_t streamTaskStartCheckDownstream(SStreamTask* pTask) { - stDebug("s-task:%s start to check downstream tasks", pTask->id.idStr); +static int32_t initStateTransferTable(SStreamTaskSM* pSM); +static STaskStateTrans createStateTransform(ETaskStatus current, ETaskStatus next, EStreamTaskEvent event, + __state_trans_fn preFn, __state_trans_fn fn, __state_trans_succ_fn succFn, + bool autoInvoke); +static int32_t streamTaskInitStatus(SStreamTask* pTask); +static int32_t streamTaskKeepCurrentVerInWal(SStreamTask* pTask); + +static int32_t dummyFn(SStreamTask* UNUSED_PARAM(p)) { return TSDB_CODE_SUCCESS; } + +int32_t streamTaskInitStatus(SStreamTask* pTask) { + pTask->execInfo.init = taosGetTimestampMs(); + + stDebug("s-task:%s start init, and check downstream tasks, set the init ts:%" PRId64, pTask->id.idStr, + pTask->execInfo.init); streamTaskCheckDownstream(pTask); return 0; } + +int32_t streamTaskSetReadyForWal(SStreamTask* pTask) { + stDebug("s-task:%s ready for extract data from wal", pTask->id.idStr); + streamSetStatusNormal(pTask); // todo remove it + return TSDB_CODE_SUCCESS; +} + static int32_t streamTaskDoPause(SStreamTask* pTask) { stDebug("s-task:%s start to pause tasks", pTask->id.idStr); return 0; } + static int32_t streamTaskDoResume(SStreamTask* pTask) { stDebug("s-task:%s start to resume tasks", pTask->id.idStr); return 0; @@ -54,12 +76,32 @@ static int32_t streamTaskDoCheckpoint(SStreamTask* pTask) { return 0; } +int32_t streamTaskWaitBeforeHalt(SStreamTask* pTask) { + char* p = NULL; + while (streamTaskGetStatus(pTask, &p) != TASK_STATUS__NORMAL) { + stDebug("related stream task:%s(status:%s) not ready for halt, wait for 100ms and retry", pTask->id.idStr, p); + taosMsleep(100); + } + + return TSDB_CODE_SUCCESS; +} + +int32_t streamTaskKeepCurrentVerInWal(SStreamTask* pTask) { + ASSERT(HAS_RELATED_FILLHISTORY_TASK(pTask)); + + pTask->hTaskInfo.haltVer = walReaderGetCurrentVer(pTask->exec.pWalReader); + if (pTask->hTaskInfo.haltVer == -1) { + pTask->hTaskInfo.haltVer = pTask->dataRange.range.maxVer + 1; + } + return TSDB_CODE_SUCCESS; +} + // todo optimize the perf of find the trans objs by using hash table -static STaskStateTrans* streamTaskFindTransform(const SStreamTaskSM* pState, const EStreamTaskEvent* pEvent) { +static STaskStateTrans* streamTaskFindTransform(const SStreamTaskSM* pState, const EStreamTaskEvent event) { int32_t numOfTrans = taosArrayGetSize(pState->pTransList); for(int32_t i = 0; i < numOfTrans; ++i) { STaskStateTrans* pTrans = taosArrayGet(pState->pTransList, i); - if (pTrans->state.state == pState->current.state && pTrans->event == *pEvent) { + if (pTrans->state.state == pState->current.state && pTrans->event == event) { return pTrans; } } @@ -68,6 +110,20 @@ static STaskStateTrans* streamTaskFindTransform(const SStreamTaskSM* pState, con return NULL; } +void streamTaskRestoreStatus(SStreamTask* pTask) { + SStreamTaskSM* pSM = pTask->status.pSM; + taosThreadMutexLock(&pTask->lock); + ASSERT(pSM->pActiveTrans == NULL); + + SStreamTaskState state = pSM->current; + pSM->current = pSM->prev; + pSM->prev = state; + pSM->startTs = taosGetTimestampMs(); + + taosThreadMutexUnlock(&pTask->lock); + stDebug("s-task:%s restore status, %s -> %s", pTask->id.idStr, pSM->prev.name, pSM->current.name); +} + SStreamTaskSM* streamCreateStateMachine(SStreamTask* pTask) { SStreamTaskSM* pSM = taosMemoryCalloc(1, sizeof(SStreamTaskSM)); if (pSM == NULL) { @@ -79,7 +135,7 @@ SStreamTaskSM* streamCreateStateMachine(SStreamTask* pTask) { // set the initial state for the state-machine of stream task pSM->current = StreamTaskStatusList[TASK_STATUS__UNINIT]; - pSM->stateTs = taosGetTimestampMs(); + pSM->startTs = taosGetTimestampMs(); int32_t code = initStateTransferTable(pSM); if (code != TSDB_CODE_SUCCESS) { taosMemoryFree(pSM); @@ -88,36 +144,71 @@ SStreamTaskSM* streamCreateStateMachine(SStreamTask* pTask) { return pSM; } -int32_t taskSMHandleEvent(SStreamTaskSM* pSM, const EStreamTaskEvent* pEvent) { - STaskStateTrans* pTrans = streamTaskFindTransform(pSM, pEvent); - ASSERT(pSM->pActiveTrans == NULL); - - stDebug("start to handle event:%d, state:%s", *pEvent, pSM->current.name); - pSM->pActiveTrans = pTrans; - pSM->stateTs = taosGetTimestampMs(); - return pTrans->pAction(pSM->pTask); -} - -int32_t taskSMOnHandleEventSuccess(SStreamTaskSM* pSM) { - STaskStateTrans* pTrans = pSM->pActiveTrans; - EStreamTaskEvent* pEvent = &pTrans->event; - - int64_t el = (taosGetTimestampMs() - pSM->stateTs); - stDebug("handle event:%d completed, elapsd time:%" PRId64 "ms new state:%s from %s", *pEvent, el, pTrans->next.name, +int32_t streamTaskHandleEvent(SStreamTaskSM* pSM, EStreamTaskEvent event) { + STaskStateTrans* pTrans = streamTaskFindTransform(pSM, event); + stDebug("s-task:%s start to handle event:%s, state:%s", pSM->pTask->id.idStr, StreamTaskEventList[event].name, pSM->current.name); - // todo: add lock - pSM->current = pTrans->next; + int32_t code = pTrans->preAction(pSM->pTask); + + taosThreadMutexLock(&pSM->pTask->lock); + ASSERT(pSM->pActiveTrans == NULL); + pSM->pActiveTrans = pTrans; + pSM->startTs = taosGetTimestampMs(); + taosThreadMutexUnlock(&pSM->pTask->lock); + + code = pTrans->pAction(pSM->pTask); + // todo handle error code; + + if (pTrans->autoInvokeEndFn) { + streamTaskOnHandleEventSuccess(pSM); + } + + return code; } +int32_t streamTaskOnHandleEventSuccess(SStreamTaskSM* pSM) { + STaskStateTrans* pTrans = pSM->pActiveTrans; + SStreamTask* pTask = pSM->pTask; + // do update the task status + taosThreadMutexLock(&pTask->lock); + SStreamTaskState current = pSM->current; -STaskStateTrans createStateTransform(ETaskStatus current, ETaskStatus next, EStreamTaskEvent event, __state_trans_fn fn) { + pSM->prev = pSM->current; + pSM->current = pTrans->next; + pSM->pActiveTrans = NULL; + + // on success callback, add into lock if necessary, or maybe we should add an option for this? + pTrans->pSuccAction(pTask); + taosThreadMutexUnlock(&pTask->lock); + + int64_t el = (taosGetTimestampMs() - pSM->startTs); + stDebug("s-task:%s handle event:%s completed, elapsed time:%" PRId64 "ms state:%s -> %s", pTask->id.idStr, + StreamTaskEventList[pTrans->event].name, el, current.name, pSM->current.name); + return TSDB_CODE_SUCCESS; +} + +ETaskStatus streamTaskGetStatus(SStreamTask* pTask, char** pStr) { + SStreamTaskState s = pTask->status.pSM->current; // copy one obj in case of multi-thread environment + if (pStr != NULL) { + *pStr = s.name; + } + return s.state; +} + +STaskStateTrans createStateTransform(ETaskStatus current, ETaskStatus next, EStreamTaskEvent event, + __state_trans_fn preFn, __state_trans_fn fn, __state_trans_succ_fn succFn, + bool autoInvoke) { STaskStateTrans trans = {0}; trans.state = StreamTaskStatusList[current]; trans.next = StreamTaskStatusList[next]; trans.event = event; + + trans.preAction = (preFn != NULL)? preFn:dummyFn; trans.pAction = (fn != NULL)? fn : dummyFn; + trans.pSuccAction = (succFn != NULL)? succFn:dummyFn; + trans.autoInvokeEndFn = autoInvoke; return trans; } @@ -129,20 +220,50 @@ int32_t initStateTransferTable(SStreamTaskSM* pSM) { } } - STaskStateTrans trans = createStateTransform(TASK_STATUS__UNINIT, TASK_STATUS__NORMAL, TASK_EVENT_INIT, streamTaskStartCheckDownstream); + // initialization event handle + STaskStateTrans trans = createStateTransform(TASK_STATUS__UNINIT, TASK_STATUS__NORMAL, TASK_EVENT_INIT, NULL, + streamTaskInitStatus, onNormalTaskReady, false); taosArrayPush(pSM->pTransList, &trans); - trans = createStateTransform(TASK_STATUS__NORMAL, TASK_STATUS__PAUSE, TASK_EVENT_PAUSE, streamTaskDoPause); + trans = createStateTransform(TASK_STATUS__UNINIT, TASK_STATUS__SCAN_HISTORY, TASK_EVENT_INIT_SCAN_HISTORY, NULL, + streamTaskInitStatus, onScanhistoryTaskReady, false); taosArrayPush(pSM->pTransList, &trans); - trans = createStateTransform(TASK_STATUS__PAUSE, TASK_STATUS__NORMAL, TASK_EVENT_RESUME, streamTaskDoResume); + trans = createStateTransform(TASK_STATUS__SCAN_HISTORY, TASK_STATUS__NORMAL, TASK_EVENT_SCANHIST_COMPLETED, NULL, + streamTaskSetReadyForWal, NULL, true); taosArrayPush(pSM->pTransList, &trans); - trans = createStateTransform(TASK_STATUS__NORMAL, TASK_STATUS__CK, TASK_EVENT_GEN_CHECKPOINT, streamTaskDoCheckpoint); + // pause & resume related event handle + trans = createStateTransform(TASK_STATUS__NORMAL, TASK_STATUS__PAUSE, TASK_EVENT_PAUSE, NULL, streamTaskDoPause, NULL, + true); taosArrayPush(pSM->pTransList, &trans); - trans = createStateTransform(TASK_STATUS__CK, TASK_STATUS__NORMAL, TASK_EVENT_PAUSE, NULL); + trans = createStateTransform(TASK_STATUS__PAUSE, TASK_STATUS__NORMAL, TASK_EVENT_RESUME, NULL, streamTaskDoResume, + NULL, true); taosArrayPush(pSM->pTransList, &trans); + trans = createStateTransform(TASK_STATUS__NORMAL, TASK_STATUS__CK, TASK_EVENT_GEN_CHECKPOINT, NULL, + streamTaskDoCheckpoint, NULL, true); + taosArrayPush(pSM->pTransList, &trans); + + trans = createStateTransform(TASK_STATUS__CK, TASK_STATUS__NORMAL, TASK_EVENT_PAUSE, NULL, NULL, NULL, true); + taosArrayPush(pSM->pTransList, &trans); + + // halt stream task, from other task status + trans = createStateTransform(TASK_STATUS__NORMAL, TASK_STATUS__HALT, TASK_EVENT_HALT, NULL, NULL, + streamTaskKeepCurrentVerInWal, true); + taosArrayPush(pSM->pTransList, &trans); + + trans = createStateTransform(TASK_STATUS__SCAN_HISTORY, TASK_STATUS__HALT, TASK_EVENT_HALT, streamTaskWaitBeforeHalt, + NULL, streamTaskKeepCurrentVerInWal, true); + taosArrayPush(pSM->pTransList, &trans); + + trans = createStateTransform(TASK_STATUS__CK, TASK_STATUS__HALT, TASK_EVENT_HALT, streamTaskWaitBeforeHalt, NULL, + streamTaskKeepCurrentVerInWal, true); + taosArrayPush(pSM->pTransList, &trans); + + trans = createStateTransform(TASK_STATUS__PAUSE, TASK_STATUS__HALT, TASK_EVENT_HALT, NULL, NULL, + streamTaskKeepCurrentVerInWal, true); + taosArrayPush(pSM->pTransList, &trans); return 0; } \ No newline at end of file From f4caeca24af7566c666e08984f6a3177cb0582a0 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 19 Oct 2023 10:34:20 +0800 Subject: [PATCH 04/41] refactor(stream): add state machine to manage the state of stream tasks. --- include/libs/stream/tstream.h | 20 +- source/dnode/vnode/src/tq/tq.c | 11 +- source/dnode/vnode/src/tq/tqStreamTask.c | 2 +- source/libs/stream/inc/streamsm.h | 21 +- source/libs/stream/src/stream.c | 5 +- source/libs/stream/src/streamCheckpoint.c | 24 +- source/libs/stream/src/streamDispatch.c | 4 +- source/libs/stream/src/streamExec.c | 5 +- source/libs/stream/src/streamMeta.c | 2 +- source/libs/stream/src/streamStart.c | 71 ++++-- source/libs/stream/src/streamTask.c | 6 +- source/libs/stream/src/streamTaskSm.c | 279 ++++++++++++++++------ 12 files changed, 307 insertions(+), 143 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 41f260fdc3..3b076fc9c3 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -75,6 +75,7 @@ typedef enum ETaskStatus { TASK_STATUS__HALT, // pause, but not be manipulated by user command TASK_STATUS__PAUSE, // pause TASK_STATUS__CK, // stream task is in checkpoint status, no data are allowed to put into inputQ anymore + TASK_STATUS_STREAM_SCAN_HISTORY, } ETaskStatus; enum { @@ -129,17 +130,18 @@ enum { typedef enum EStreamTaskEvent { TASK_EVENT_INIT = 0x1, - TASK_EVENT_INIT_SCAN_HISTORY = 0x2, - TASK_EVENT_SCANHIST_COMPLETED = 0x3, - TASK_EVENT_START = 0x4, + TASK_EVENT_INIT_SCANHIST = 0x2, + TASK_EVENT_INIT_STREAM_SCANHIST = 0x3, + TASK_EVENT_SCANHIST_DONE = 0x4, TASK_EVENT_STOP = 0x5, TASK_EVENT_GEN_CHECKPOINT = 0x6, - TASK_EVENT_PAUSE = 0x7, - TASK_EVENT_RESUME = 0x8, - TASK_EVENT_HALT = 0x9, - TASK_EVENT_TRANS_STATE = 0xA, - TASK_EVENT_SCAN_TSDB = 0xB, - TASK_EVENT_SCAN_WAL = 0xC, + TASK_EVENT_CHECKPOINT_DONE = 0x7, + TASK_EVENT_PAUSE = 0x8, + TASK_EVENT_RESUME = 0x9, + TASK_EVENT_HALT = 0xA, + TASK_EVENT_TRANS_STATE = 0xB, + TASK_EVENT_SCAN_TSDB = 0xC, + TASK_EVENT_SCAN_WAL = 0xD, } EStreamTaskEvent; typedef struct { diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 01dea11235..5e0a81314f 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1029,7 +1029,7 @@ int32_t tqProcessTaskDeployReq(STQ* pTq, int64_t sversion, char* msg, int32_t ms bool restored = pTq->pVnode->restored; if (p != NULL && restored && p->info.fillHistory == 0) { - EStreamTaskEvent event = (p->hTaskInfo.id.taskId == 0) ? TASK_EVENT_INIT : TASK_EVENT_INIT_SCAN_HISTORY; + EStreamTaskEvent event = (p->hTaskInfo.id.taskId == 0) ? TASK_EVENT_INIT : TASK_EVENT_INIT_SCANHIST; streamTaskHandleEvent(p->status.pSM, event); } else if (!restored) { tqWarn("s-task:%s not launched since vnode(vgId:%d) not ready", p->id.idStr, vgId); @@ -1259,7 +1259,7 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { } #endif - streamTaskHandleEvent(pTask->status.pSM, TASK_EVENT_SCANHIST_COMPLETED); + streamTaskHandleEvent(pTask->status.pSM, TASK_EVENT_SCANHIST_DONE); tqScanWalAsync(pTq, false); } streamMetaReleaseTask(pMeta, pStreamTask); @@ -1683,7 +1683,6 @@ FAIL: return -1; } -// todo error code cannot be return, since this is invoked by an mnode-launched transaction. int32_t tqProcessTaskCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg* pRsp) { int32_t vgId = TD_VID(pTq->pVnode); SStreamMeta* pMeta = pTq->pStreamMeta; @@ -1694,7 +1693,6 @@ int32_t tqProcessTaskCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg* pRsp) // disable auto rsp to mnode pRsp->info.handle = NULL; - // todo: add counter to make sure other tasks would not be trapped in checkpoint state SStreamCheckpointSourceReq req = {0}; if (!vnodeIsRoleLeader(pTq->pVnode)) { tqDebug("vgId:%d not leader, ignore checkpoint-source msg", vgId); @@ -1725,6 +1723,7 @@ int32_t tqProcessTaskCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg* pRsp) } tDecoderClear(&decoder); + // todo handle failure to reset from checkpoint procedure SStreamTask* pTask = streamMetaAcquireTask(pMeta, req.streamId, req.taskId); if (pTask == NULL) { tqError("vgId:%d failed to find s-task:0x%x, ignore checkpoint msg. it may have been destroyed already", vgId, @@ -1735,6 +1734,7 @@ int32_t tqProcessTaskCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg* pRsp) return TSDB_CODE_SUCCESS; } + // todo handle failure to reset from checkpoint procedure // downstream not ready, current the stream tasks are not all ready. Ignore this checkpoint req. if (pTask->status.downstreamReady != 1) { pTask->chkInfo.failedId = req.checkpointId; // record the latest failed checkpoint id @@ -1750,8 +1750,9 @@ int32_t tqProcessTaskCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg* pRsp) return TSDB_CODE_SUCCESS; } + // todo save the checkpoint failed info taosThreadMutexLock(&pTask->lock); - if (pTask->status.taskStatus == TASK_STATUS__HALT) { + if (streamTaskGetStatus(pTask, NULL) == TASK_STATUS__HALT) { qError("s-task:%s not ready for checkpoint, since it is halt, ignore this checkpoint:%" PRId64 ", set it failure", pTask->id.idStr, req.checkpointId); taosThreadMutexUnlock(&pTask->lock); diff --git a/source/dnode/vnode/src/tq/tqStreamTask.c b/source/dnode/vnode/src/tq/tqStreamTask.c index 29de14e3c3..be64ec20a4 100644 --- a/source/dnode/vnode/src/tq/tqStreamTask.c +++ b/source/dnode/vnode/src/tq/tqStreamTask.c @@ -99,7 +99,7 @@ int32_t tqCheckAndRunStreamTask(STQ* pTq) { continue; } - EStreamTaskEvent event = (pTask->hTaskInfo.id.taskId == 0) ? TASK_EVENT_INIT : TASK_EVENT_INIT_SCAN_HISTORY; + EStreamTaskEvent event = (pTask->hTaskInfo.id.taskId == 0) ? TASK_EVENT_INIT : TASK_EVENT_INIT_SCANHIST; streamTaskHandleEvent(pTask->status.pSM, event); streamMetaReleaseTask(pMeta, pTask); } diff --git a/source/libs/stream/inc/streamsm.h b/source/libs/stream/inc/streamsm.h index 39ea2a77f8..b1ccc19e23 100644 --- a/source/libs/stream/inc/streamsm.h +++ b/source/libs/stream/inc/streamsm.h @@ -31,14 +31,19 @@ typedef struct SStreamTaskState { typedef int32_t (*__state_trans_fn)(SStreamTask*); typedef int32_t (*__state_trans_succ_fn)(SStreamTask*); -typedef struct STaskStateTrans { - bool autoInvokeEndFn; - SStreamTaskState state; +typedef struct SAttachedEventInfo { + ETaskStatus status; EStreamTaskEvent event; - SStreamTaskState next; - __state_trans_fn preAction; - __state_trans_fn pAction; +} SAttachedEventInfo; + +typedef struct STaskStateTrans { + bool autoInvokeEndFn; + SStreamTaskState state; + EStreamTaskEvent event; + SStreamTaskState next; + __state_trans_fn pAction; __state_trans_succ_fn pSuccAction; + SAttachedEventInfo attachEvent; } STaskStateTrans; struct SStreamTaskSM { @@ -48,6 +53,8 @@ struct SStreamTaskSM { int64_t startTs; SStreamTaskState current; SStreamTaskState prev; + // register the next handled event, if current state is not allowed to handle this event + SArray* eventList; }; typedef struct SStreamEventInfo { @@ -57,7 +64,7 @@ typedef struct SStreamEventInfo { } SStreamEventInfo; SStreamTaskSM* streamCreateStateMachine(SStreamTask* pTask); - +void* streamDestroyStateMachine(SStreamTaskSM* pSM); #ifdef __cplusplus } #endif diff --git a/source/libs/stream/src/stream.c b/source/libs/stream/src/stream.c index 8910616f01..91205a216b 100644 --- a/source/libs/stream/src/stream.c +++ b/source/libs/stream/src/stream.c @@ -69,7 +69,7 @@ static void streamSchedByTimer(void* param, void* tmrId) { return; } - if (pTask->status.taskStatus == TASK_STATUS__CK) { + if (streamTaskGetStatus(pTask, NULL) == TASK_STATUS__CK) { stDebug("s-task:%s in checkpoint procedure, not retrieve result, next:%dms", id, nextTrigger); } else { if (status == TASK_TRIGGER_STATUS__ACTIVE) { @@ -267,8 +267,7 @@ int32_t streamProcessDispatchMsg(SStreamTask* pTask, SStreamDispatchReq* pReq, S } // disable the data from upstream tasks - int8_t st = pTask->status.taskStatus; - if (st == TASK_STATUS__HALT) { + if (streamTaskGetStatus(pTask, NULL) == TASK_STATUS__HALT) { status = TASK_INPUT_STATUS__BLOCKED; } diff --git a/source/libs/stream/src/streamCheckpoint.c b/source/libs/stream/src/streamCheckpoint.c index 2cde368195..7026ac7119 100644 --- a/source/libs/stream/src/streamCheckpoint.c +++ b/source/libs/stream/src/streamCheckpoint.c @@ -135,12 +135,13 @@ static int32_t appendCheckpointIntoInputQ(SStreamTask* pTask, int32_t checkpoint int32_t streamProcessCheckpointSourceReq(SStreamTask* pTask, SStreamCheckpointSourceReq* pReq) { ASSERT(pTask->info.taskLevel == TASK_LEVEL__SOURCE); + // todo this status may not be set here. // 1. set task status to be prepared for check point, no data are allowed to put into inputQ. - pTask->status.taskStatus = TASK_STATUS__CK; + streamTaskHandleEvent(pTask->status.pSM, TASK_EVENT_GEN_CHECKPOINT); + pTask->checkpointingId = pReq->checkpointId; pTask->checkpointNotReadyTasks = streamTaskGetNumOfDownstream(pTask); pTask->chkInfo.startTs = taosGetTimestampMs(); - pTask->execInfo.checkpoint += 1; // 2. Put the checkpoint block into inputQ, to make sure all blocks with less version have been handled by this task @@ -171,11 +172,11 @@ int32_t streamProcessCheckpointBlock(SStreamTask* pTask, SStreamDataBlock* pBloc const char* id = pTask->id.idStr; int32_t code = TSDB_CODE_SUCCESS; - // set the task status - pTask->checkpointingId = checkpointId; - // set task status - pTask->status.taskStatus = TASK_STATUS__CK; + if (streamTaskGetStatus(pTask, NULL) != TASK_STATUS__CK) { + pTask->checkpointingId = checkpointId; + streamTaskHandleEvent(pTask->status.pSM, TASK_EVENT_GEN_CHECKPOINT); + } { // todo: remove this when the pipeline checkpoint generating is used. SStreamMeta* pMeta = pTask->pMeta; @@ -195,6 +196,7 @@ int32_t streamProcessCheckpointBlock(SStreamTask* pTask, SStreamDataBlock* pBloc stDebug("s-task:%s set childIdx:%d, and add checkpoint-trigger block into outputQ", id, pTask->info.selfChildId); continueDispatchCheckpointBlock(pBlock, pTask); } else { // only one task exists, no need to dispatch downstream info + atomic_add_fetch_32(&pTask->checkpointNotReadyTasks, 1); streamProcessCheckpointReadyMsg(pTask); streamFreeQitem((SStreamQueueItem*)pBlock); } @@ -288,12 +290,14 @@ int32_t streamSaveAllTaskStatus(SStreamMeta* pMeta, int64_t checkpointId) { continue; } - int8_t prev = p->status.taskStatus; ASSERT(p->chkInfo.checkpointId < p->checkpointingId && p->checkpointingId == checkpointId); p->chkInfo.checkpointId = p->checkpointingId; streamTaskClearCheckInfo(p); - streamSetStatusNormal(p); + + char* str = NULL; + streamTaskGetStatus(p, &str); + streamTaskHandleEvent(p->status.pSM, TASK_EVENT_CHECKPOINT_DONE); // save the task streamMetaSaveTask(pMeta, p); @@ -302,13 +306,13 @@ int32_t streamSaveAllTaskStatus(SStreamMeta* pMeta, int64_t checkpointId) { "vgId:%d s-task:%s level:%d open upstream inputQ, commit task status after checkpoint completed, " "checkpointId:%" PRId64 ", Ver(saved):%" PRId64 " currentVer:%" PRId64 ", status to be normal, prev:%s", pMeta->vgId, p->id.idStr, p->info.taskLevel, checkpointId, p->chkInfo.checkpointVer, p->chkInfo.nextProcessVer, - streamGetTaskStatusStr(prev)); + str); } if (streamMetaCommit(pMeta) < 0) { taosWUnLockLatch(&pMeta->lock); stError("vgId:%d failed to commit stream meta after do checkpoint, checkpointId:%" PRId64 ", since %s", pMeta->vgId, - checkpointId, terrstr()); + checkpointId, terrstr()); return -1; } else { taosWUnLockLatch(&pMeta->lock); diff --git a/source/libs/stream/src/streamDispatch.c b/source/libs/stream/src/streamDispatch.c index 39afc3c98a..b339cb6969 100644 --- a/source/libs/stream/src/streamDispatch.c +++ b/source/libs/stream/src/streamDispatch.c @@ -1011,7 +1011,7 @@ int32_t streamAddEndScanHistoryMsg(SStreamTask* pTask, SRpcHandleInfo* pRpcInfo, taosThreadMutexUnlock(&pTask->lock); int32_t num = taosArrayGetSize(pTask->pRspMsgList); - stDebug("s-task:%s add scan history finish rsp msg for task:0x%x, total:%d", pTask->id.idStr, pReq->upstreamTaskId, + stDebug("s-task:%s add scan-history finish rsp msg for task:0x%x, total:%d", pTask->id.idStr, pReq->upstreamTaskId, num); return TSDB_CODE_SUCCESS; } @@ -1027,7 +1027,7 @@ int32_t streamNotifyUpstreamContinue(SStreamTask* pTask) { SStreamContinueExecInfo* pInfo = taosArrayGet(pTask->pRspMsgList, i); tmsgSendRsp(&pInfo->msg); - stDebug("s-task:%s level:%d notify upstream:0x%x continuing scan data in WAL", id, level, pInfo->taskId); + stDebug("s-task:%s level:%d notify upstream:0x%x continuing handle data in WAL", id, level, pInfo->taskId); } taosArrayClear(pTask->pRspMsgList); diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index ca67e19901..da7e2ece0d 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -324,7 +324,6 @@ int32_t streamDoTransferStateToStreamTask(SStreamTask* pTask) { } else { ASSERT(status == TASK_STATUS__NORMAL); streamTaskHandleEvent(pStreamTask->status.pSM, TASK_EVENT_HALT); -// pStreamTask->status.taskStatus = TASK_STATUS__HALT; stDebug("s-task:%s halt by related fill-history task:%s", pStreamTask->id.idStr, pTask->id.idStr); } @@ -375,7 +374,7 @@ int32_t streamDoTransferStateToStreamTask(SStreamTask* pTask) { // 7. pause allowed. streamTaskEnablePause(pStreamTask); - if (taosQueueEmpty(pStreamTask->inputInfo.queue->pQueue)) { + if ((pStreamTask->info.taskLevel == TASK_LEVEL__SOURCE) && taosQueueEmpty(pStreamTask->inputInfo.queue->pQueue)) { SStreamRefDataBlock* pItem = taosAllocateQitem(sizeof(SStreamRefDataBlock), DEF_QITEM, 0); SSDataBlock* pDelBlock = createSpecialDataBlock(STREAM_DELETE_DATA); @@ -492,7 +491,7 @@ int32_t streamProcessTranstateBlock(SStreamTask* pTask, SStreamDataBlock* pBlock code = taosWriteQitem(pTask->outputq.queue->pQueue, pBlock); if (code == 0) { streamDispatchStreamBlock(pTask); - } else { + } else { // todo put into queue failed, retry streamFreeQitem((SStreamQueueItem*)pBlock); } } else { // level == TASK_LEVEL__SINK diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 2773912437..a80e3a05f0 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -894,7 +894,7 @@ void metaHbToMnode(void* param, void* tmrId) { STaskStatusEntry entry = { .id = *pId, - .status = (*pTask)->status.taskStatus, + .status = streamTaskGetStatus(*pTask, NULL), .nodeId = pMeta->vgId, .stage = pMeta->stage, .inputQUsed = SIZE_IN_MiB(streamQueueGetItemSize((*pTask)->inputInfo.queue)), diff --git a/source/libs/stream/src/streamStart.c b/source/libs/stream/src/streamStart.c index 8ebc1c0094..003596ce90 100644 --- a/source/libs/stream/src/streamStart.c +++ b/source/libs/stream/src/streamStart.c @@ -272,8 +272,12 @@ int32_t onNormalTaskReady(SStreamTask* pTask) { ASSERT(pTask->hTaskInfo.id.taskId == 0); } - stDebug("s-task:%s no need to scan-history data, status:%s, sched-status:%d, ready for data from wal ver:%" PRId64, - id, p, pTask->status.schedStatus, walReaderGetCurrentVer(pTask->exec.pWalReader)); + if (pTask->info.taskLevel == TASK_LEVEL__SOURCE) { + stDebug("s-task:%s no need to scan-history data, status:%s, sched-status:%d, ready for data from wal ver:%" PRId64, + id, p, pTask->status.schedStatus, walReaderGetCurrentVer(pTask->exec.pWalReader)); + } else { + stDebug("s-task:%s level:%d status:%s sched-status:%d", id, pTask->info.taskLevel, p, pTask->status.schedStatus); + } streamTaskEnablePause(pTask); return TSDB_CODE_SUCCESS; @@ -520,11 +524,13 @@ int32_t streamProcessScanHistoryFinishReq(SStreamTask* pTask, SStreamScanHistory int32_t taskLevel = pTask->info.taskLevel; ASSERT(taskLevel == TASK_LEVEL__AGG || taskLevel == TASK_LEVEL__SINK); - char* p = NULL; + const char* id = pTask->id.idStr; + char* p = NULL; ETaskStatus status = streamTaskGetStatus(pTask, &p); + if (status != TASK_STATUS__SCAN_HISTORY) { stError("s-task:%s not in scan-history status, status:%s return upstream:0x%x scan-history finish directly", - pTask->id.idStr, p, pReq->upstreamTaskId); + id, p, pReq->upstreamTaskId); void* pBuf = NULL; int32_t len = 0; @@ -534,8 +540,8 @@ int32_t streamProcessScanHistoryFinishReq(SStreamTask* pTask, SStreamScanHistory initRpcMsg(&msg, 0, pBuf, sizeof(SMsgHead) + len); tmsgSendRsp(&msg); - stDebug("s-task:%s level:%d notify upstream:0x%x(vgId:%d) to continue process data in WAL", pTask->id.idStr, - pTask->info.taskLevel, pReq->upstreamTaskId, pReq->upstreamNodeId); + stDebug("s-task:%s level:%d notify upstream:0x%x(vgId:%d) to continue process data in WAL", id, + taskLevel, pReq->upstreamTaskId, pReq->upstreamNodeId); return 0; } @@ -547,13 +553,15 @@ int32_t streamProcessScanHistoryFinishReq(SStreamTask* pTask, SStreamScanHistory if (left == 0) { int32_t numOfTasks = taosArrayGetSize(pTask->upstreamInfo.pList); - stDebug( - "s-task:%s all %d upstream tasks finish scan-history data, set param for agg task for stream data and send " - "rsp to all upstream tasks", - pTask->id.idStr, numOfTasks); - - if (pTask->info.taskLevel == TASK_LEVEL__AGG) { + if (taskLevel == TASK_LEVEL__AGG) { + stDebug( + "s-task:%s all %d upstream tasks finish scan-history data, set param for agg task for stream data processing " + "and send rsp to all upstream tasks", + id, numOfTasks); streamAggUpstreamScanHistoryFinish(pTask); + } else { + stDebug("s-task:%s all %d upstream task(s) finish scan-history data, and rsp to all upstream tasks", id, + numOfTasks); } // all upstream tasks have completed the scan-history task in the stream time window, let's start to extract data @@ -564,14 +572,11 @@ int32_t streamProcessScanHistoryFinishReq(SStreamTask* pTask, SStreamScanHistory if (taskLevel == TASK_LEVEL__AGG) { /*int32_t code = */streamTaskScanHistoryDataComplete(pTask); } else { // for sink task, set normal - if (pTask->status.taskStatus != TASK_STATUS__PAUSE && pTask->status.taskStatus != TASK_STATUS__STOP && - pTask->status.taskStatus != TASK_STATUS__DROPPING) { - streamSetStatusNormal(pTask); - } + streamTaskHandleEvent(pTask->status.pSM, TASK_EVENT_SCANHIST_DONE); } } else { stDebug("s-task:%s receive scan-history data finish msg from upstream:0x%x(index:%d), unfinished:%d", - pTask->id.idStr, pReq->upstreamTaskId, pReq->childId, left); + id, pReq->upstreamTaskId, pReq->childId, left); } return 0; @@ -582,7 +587,7 @@ int32_t streamProcessScanHistoryFinishRsp(SStreamTask* pTask) { SStreamMeta* pMeta = pTask->pMeta; // execute in the scan history complete call back msg, ready to process data from inputQ - streamTaskHandleEvent(pTask->status.pSM, TASK_EVENT_SCANHIST_COMPLETED); + streamTaskHandleEvent(pTask->status.pSM, TASK_EVENT_SCANHIST_DONE); streamTaskSetSchedStatusInactive(pTask); taosWLockLatch(&pMeta->lock); @@ -619,7 +624,7 @@ static void checkFillhistoryTaskStatus(SStreamTask* pTask, SStreamTask* pHTask) } // check if downstream tasks have been ready - streamTaskHandleEvent(pHTask->status.pSM, TASK_EVENT_INIT_SCAN_HISTORY); + streamTaskHandleEvent(pHTask->status.pSM, TASK_EVENT_INIT_SCANHIST); } static void tryLaunchHistoryTask(void* param, void* tmrId) { @@ -903,7 +908,7 @@ int32_t tDecodeStreamScanHistoryFinishReq(SDecoder* pDecoder, SStreamScanHistory void streamTaskSetRangeStreamCalc(SStreamTask* pTask) { SDataRange* pRange = &pTask->dataRange; - if (pTask->hTaskInfo.id.taskId == 0) { + if (!HAS_RELATED_FILLHISTORY_TASK(pTask)) { if (pTask->info.fillHistory == 1) { stDebug("s-task:%s fill-history task, time window:%" PRId64 "-%" PRId64 ", verRange:%" PRId64 "-%" PRId64, pTask->id.idStr, pRange->window.skey, pRange->window.ekey, pRange->range.minVer, pRange->range.maxVer); @@ -949,7 +954,7 @@ void streamTaskCheckDownstream(SStreamTask* pTask) { // normal -> pause, pause/stop/dropping -> pause, halt -> pause, scan-history -> pause void streamTaskPause(SStreamTask* pTask, SStreamMeta* pMeta) { int64_t st = taosGetTimestampMs(); - +#if 0 int8_t status = pTask->status.taskStatus; if (status == TASK_STATUS__DROPPING) { stDebug("vgId:%d s-task:%s task already dropped, do nothing", pMeta->vgId, pTask->id.idStr); @@ -1007,6 +1012,13 @@ void streamTaskPause(SStreamTask* pTask, SStreamMeta* pMeta) { stInfo("vgId:%d s-task:%s pause stream task. pause task num:%d", pMeta->vgId, pTask->id.idStr, num); taosWUnLockLatch(&pMeta->lock); +#endif + + streamTaskHandleEvent(pTask->status.pSM, TASK_EVENT_PAUSE); + + int32_t num = atomic_add_fetch_32(&pMeta->numOfPausedTasks, 1); + stInfo("vgId:%d s-task:%s pause stream task. pause task num:%d", pMeta->vgId, pTask->id.idStr, num); + // in case of fill-history task, stop the tsdb file scan operation. if (pTask->info.fillHistory == 1) { void* pExecutor = pTask->exec.pExecutor; @@ -1019,6 +1031,21 @@ void streamTaskPause(SStreamTask* pTask, SStreamMeta* pMeta) { } void streamTaskResume(SStreamTask* pTask, SStreamMeta* pMeta) { + char* p = NULL; + ETaskStatus status = streamTaskGetStatus(pTask, &p); + + if (status == TASK_STATUS__PAUSE) { + streamTaskRestoreStatus(pTask); + + streamTaskGetStatus(pTask, &p); + int32_t num = atomic_sub_fetch_32(&pMeta->numOfPausedTasks, 1); + stInfo("vgId:%d s-task:%s sink task.resume from pause, status:%s. pause task num:%d", pMeta->vgId, pTask->id.idStr, + p, num); + } else { + stDebug("s-task:%s status:%s not in pause status, no need to resume", pTask->id.idStr, p); + } + +#if 0 int8_t status = pTask->status.taskStatus; if (status == TASK_STATUS__PAUSE) { pTask->status.taskStatus = pTask->status.keepTaskStatus; @@ -1031,6 +1058,8 @@ void streamTaskResume(SStreamTask* pTask, SStreamMeta* pMeta) { } else { stError("s-task:%s not in pause, failed to resume, status:%s", pTask->id.idStr, streamGetTaskStatusStr(status)); } +#endif + } // todo fix race condition diff --git a/source/libs/stream/src/streamTask.c b/source/libs/stream/src/streamTask.c index 4b39d979b6..b20bd9421c 100644 --- a/source/libs/stream/src/streamTask.c +++ b/source/libs/stream/src/streamTask.c @@ -304,11 +304,9 @@ static void freeUpstreamItem(void* p) { void tFreeStreamTask(SStreamTask* pTask) { int32_t taskId = pTask->id.taskId; - char* p = NULL; - streamTaskGetStatus(pTask, &p); STaskExecStatisInfo* pStatis = &pTask->execInfo; - stDebug("start to free s-task:0x%x, %p, state:%p, status:%s", taskId, pTask, pTask->pState, p); + stDebug("start to free s-task:0x%x, %p, state:%p", taskId, pTask, pTask->pState); stDebug("s-task:0x%x task exec summary: create:%" PRId64 ", init:%" PRId64 ", start:%" PRId64 ", updateCount:%d latestUpdate:%" PRId64 ", latestCheckPoint:%" PRId64 ", ver:%" PRId64 @@ -394,6 +392,8 @@ void tFreeStreamTask(SStreamTask* pTask) { pTask->pRspMsgList = NULL; } + pTask->status.pSM = streamDestroyStateMachine(pTask->status.pSM); + streamTaskDestroyUpstreamInfo(&pTask->upstreamInfo); pTask->msgInfo.pRetryList = taosArrayDestroy(pTask->msgInfo.pRetryList); diff --git a/source/libs/stream/src/streamTaskSm.c b/source/libs/stream/src/streamTaskSm.c index b4b4d026c2..bb7c168922 100644 --- a/source/libs/stream/src/streamTaskSm.c +++ b/source/libs/stream/src/streamTaskSm.c @@ -14,39 +14,63 @@ */ #include "streamInt.h" +#include "streamsm.h" #include "tmisce.h" #include "tstream.h" #include "ttimer.h" #include "wal.h" -#include "streamsm.h" -SStreamTaskState StreamTaskStatusList[8] = { +SStreamTaskState StreamTaskStatusList[9] = { {.state = TASK_STATUS__NORMAL, .name = "normal"}, - {.state = TASK_STATUS__DROPPING, .name = "dropping"}, + {.state = TASK_STATUS__DROPPING, .name = "dropped"}, {.state = TASK_STATUS__UNINIT, .name = "uninit"}, {.state = TASK_STATUS__STOP, .name = "stop"}, {.state = TASK_STATUS__SCAN_HISTORY, .name = "scan-history"}, {.state = TASK_STATUS__HALT, .name = "halt"}, {.state = TASK_STATUS__PAUSE, .name = "paused"}, {.state = TASK_STATUS__CK, .name = "checkpoint"}, + {.state = TASK_STATUS_STREAM_SCAN_HISTORY, .name = "stream-scan-history"}, }; -SStreamEventInfo StreamTaskEventList[8] = { - {}, // dummy event, place holder +SStreamEventInfo StreamTaskEventList[10] = { + {}, // dummy event, place holder {.event = TASK_EVENT_INIT, .name = "initialize"}, - {.event = TASK_EVENT_INIT_SCAN_HISTORY, .name = "scan-history-initialize"}, - {.event = TASK_EVENT_SCANHIST_COMPLETED, .name = "scan-history-completed"}, + {.event = TASK_EVENT_INIT_SCANHIST, .name = "scan-history-initialize"}, + {.event = TASK_EVENT_SCANHIST_DONE, .name = "scan-history-completed"}, + {.event = TASK_EVENT_STOP, .name = "stopping"}, + {.event = TASK_EVENT_GEN_CHECKPOINT, .name = "checkpoint"}, + {.event = TASK_EVENT_CHECKPOINT_DONE, .name = "checkpoint-done"}, + {.event = TASK_EVENT_PAUSE, .name = "pausing"}, + {.event = TASK_EVENT_RESUME, .name = "resuming"}, + {.event = TASK_EVENT_HALT, .name = "halting"}, }; -static int32_t initStateTransferTable(SStreamTaskSM* pSM); -static STaskStateTrans createStateTransform(ETaskStatus current, ETaskStatus next, EStreamTaskEvent event, - __state_trans_fn preFn, __state_trans_fn fn, __state_trans_succ_fn succFn, - bool autoInvoke); static int32_t streamTaskInitStatus(SStreamTask* pTask); static int32_t streamTaskKeepCurrentVerInWal(SStreamTask* pTask); +static int32_t initStateTransferTable(SStreamTaskSM* pSM); + +static STaskStateTrans createStateTransform(ETaskStatus current, ETaskStatus next, EStreamTaskEvent event, + __state_trans_fn fn, __state_trans_succ_fn succFn, + SAttachedEventInfo* pEventInfo, bool autoInvoke); static int32_t dummyFn(SStreamTask* UNUSED_PARAM(p)) { return TSDB_CODE_SUCCESS; } +static int32_t attachEvent(SStreamTask* pTask, SAttachedEventInfo* pEvtInfo) { + char* p = NULL; + streamTaskGetStatus(pTask, &p); + + stDebug("s-task:%s status:%s attach event:%s required status:%s, since not allowed to handle it", pTask->id.idStr, p, + StreamTaskEventList[pEvtInfo->event].name, StreamTaskStatusList[pEvtInfo->status].name); + + SStreamTaskSM* pSM = pTask->status.pSM; + if (pSM->eventList == NULL) { + + } + + taosArrayPush(pSM->eventList, pEvtInfo); + return 0; +} + int32_t streamTaskInitStatus(SStreamTask* pTask) { pTask->execInfo.init = taosGetTimestampMs(); @@ -57,49 +81,36 @@ int32_t streamTaskInitStatus(SStreamTask* pTask) { } int32_t streamTaskSetReadyForWal(SStreamTask* pTask) { - stDebug("s-task:%s ready for extract data from wal", pTask->id.idStr); - streamSetStatusNormal(pTask); // todo remove it + if (pTask->info.taskLevel == TASK_LEVEL__SOURCE) { + stDebug("s-task:%s ready for extract data from wal", pTask->id.idStr); + } + + streamSetStatusNormal(pTask); // todo remove it return TSDB_CODE_SUCCESS; } -static int32_t streamTaskDoPause(SStreamTask* pTask) { - stDebug("s-task:%s start to pause tasks", pTask->id.idStr); - return 0; -} - -static int32_t streamTaskDoResume(SStreamTask* pTask) { - stDebug("s-task:%s start to resume tasks", pTask->id.idStr); - return 0; -} static int32_t streamTaskDoCheckpoint(SStreamTask* pTask) { stDebug("s-task:%s start to do checkpoint", pTask->id.idStr); return 0; } -int32_t streamTaskWaitBeforeHalt(SStreamTask* pTask) { - char* p = NULL; - while (streamTaskGetStatus(pTask, &p) != TASK_STATUS__NORMAL) { - stDebug("related stream task:%s(status:%s) not ready for halt, wait for 100ms and retry", pTask->id.idStr, p); - taosMsleep(100); - } - - return TSDB_CODE_SUCCESS; -} - int32_t streamTaskKeepCurrentVerInWal(SStreamTask* pTask) { ASSERT(HAS_RELATED_FILLHISTORY_TASK(pTask)); - pTask->hTaskInfo.haltVer = walReaderGetCurrentVer(pTask->exec.pWalReader); - if (pTask->hTaskInfo.haltVer == -1) { - pTask->hTaskInfo.haltVer = pTask->dataRange.range.maxVer + 1; + if (pTask->info.taskLevel == TASK_LEVEL__SOURCE) { + pTask->hTaskInfo.haltVer = walReaderGetCurrentVer(pTask->exec.pWalReader); + if (pTask->hTaskInfo.haltVer == -1) { + pTask->hTaskInfo.haltVer = pTask->dataRange.range.maxVer + 1; + } } + return TSDB_CODE_SUCCESS; } // todo optimize the perf of find the trans objs by using hash table static STaskStateTrans* streamTaskFindTransform(const SStreamTaskSM* pState, const EStreamTaskEvent event) { int32_t numOfTrans = taosArrayGetSize(pState->pTransList); - for(int32_t i = 0; i < numOfTrans; ++i) { + for (int32_t i = 0; i < numOfTrans; ++i) { STaskStateTrans* pTrans = taosArrayGet(pState->pTransList, i); if (pTrans->state.state == pState->current.state && pTrans->event == event) { return pTrans; @@ -115,6 +126,8 @@ void streamTaskRestoreStatus(SStreamTask* pTask) { taosThreadMutexLock(&pTask->lock); ASSERT(pSM->pActiveTrans == NULL); + ASSERT(pSM->current.state == TASK_STATUS__PAUSE || pSM->current.state == TASK_STATUS__HALT); + SStreamTaskState state = pSM->current; pSM->current = pSM->prev; pSM->prev = state; @@ -125,46 +138,89 @@ void streamTaskRestoreStatus(SStreamTask* pTask) { } SStreamTaskSM* streamCreateStateMachine(SStreamTask* pTask) { + 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; } pSM->pTask = pTask; + pSM->eventList = taosArrayInit(4, sizeof(SAttachedEventInfo)); + if (pSM->eventList == 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; + } // set the initial state for the state-machine of stream task pSM->current = StreamTaskStatusList[TASK_STATUS__UNINIT]; + pSM->startTs = taosGetTimestampMs(); int32_t code = initStateTransferTable(pSM); if (code != TSDB_CODE_SUCCESS) { + taosArrayDestroy(pSM->eventList); taosMemoryFree(pSM); return NULL; } return pSM; } +void* streamDestroyStateMachine(SStreamTaskSM* pSM) { + if (pSM == NULL) { + return NULL; + } + + taosArrayDestroy(pSM->eventList); + taosArrayDestroy(pSM->pTransList); + taosMemoryFree(pSM); + return NULL; +} + int32_t streamTaskHandleEvent(SStreamTaskSM* pSM, EStreamTaskEvent event) { + taosThreadMutexLock(&pSM->pTask->lock); + STaskStateTrans* pTrans = streamTaskFindTransform(pSM, event); stDebug("s-task:%s start to handle event:%s, state:%s", pSM->pTask->id.idStr, StreamTaskEventList[event].name, pSM->current.name); - int32_t code = pTrans->preAction(pSM->pTask); + if (pTrans->attachEvent.event != 0) { + attachEvent(pSM->pTask, &pTrans->attachEvent); + taosThreadMutexUnlock(&pSM->pTask->lock); - taosThreadMutexLock(&pSM->pTask->lock); - ASSERT(pSM->pActiveTrans == NULL); - pSM->pActiveTrans = pTrans; - pSM->startTs = taosGetTimestampMs(); - taosThreadMutexUnlock(&pSM->pTask->lock); + while (1) { + // wait for the task to be here + ETaskStatus s = streamTaskGetStatus(pSM->pTask, NULL); + if (s == pTrans->attachEvent.status) { + return TSDB_CODE_SUCCESS; + } else {// this event has been handled already + stDebug("s-task:%s not handle event:%s yet, wait for 100ms and recheck", pSM->pTask->id.idStr, + StreamTaskEventList[event].name); + taosMsleep(100); + } + } - code = pTrans->pAction(pSM->pTask); - // todo handle error code; + } else { + ASSERT(pSM->pActiveTrans == NULL); + pSM->pActiveTrans = pTrans; + pSM->startTs = taosGetTimestampMs(); + taosThreadMutexUnlock(&pSM->pTask->lock); - if (pTrans->autoInvokeEndFn) { - streamTaskOnHandleEventSuccess(pSM); + int32_t code = pTrans->pAction(pSM->pTask); + // todo handle error code; + + if (pTrans->autoInvokeEndFn) { + streamTaskOnHandleEventSuccess(pSM); + } } - return code; + return TSDB_CODE_SUCCESS; } int32_t streamTaskOnHandleEventSuccess(SStreamTaskSM* pSM) { @@ -181,11 +237,42 @@ int32_t streamTaskOnHandleEventSuccess(SStreamTaskSM* pSM) { // on success callback, add into lock if necessary, or maybe we should add an option for this? pTrans->pSuccAction(pTask); - taosThreadMutexUnlock(&pTask->lock); - int64_t el = (taosGetTimestampMs() - pSM->startTs); - stDebug("s-task:%s handle event:%s completed, elapsed time:%" PRId64 "ms state:%s -> %s", pTask->id.idStr, - StreamTaskEventList[pTrans->event].name, el, current.name, pSM->current.name); + if (taosArrayGetSize(pSM->eventList) > 0) { + int64_t el = (taosGetTimestampMs() - pSM->startTs); + stDebug("s-task:%s handle event:%s completed, elapsed time:%" PRId64 "ms state:%s -> %s", pTask->id.idStr, + StreamTaskEventList[pTrans->event].name, el, current.name, pSM->current.name); + + SAttachedEventInfo* pEvtInfo = taosArrayPop(pSM->eventList); + + // OK, let's handle the attached event, since the task has reached the required status now + if (pSM->current.state == pEvtInfo->status) { + stDebug("s-task:%s handle the attached event:%s, state:%s", pTask->id.idStr, + StreamTaskEventList[pEvtInfo->event].name, pSM->current.name); + + STaskStateTrans* pNextTrans = streamTaskFindTransform(pSM, pEvtInfo->event); + ASSERT(pSM->pActiveTrans == NULL); + pSM->pActiveTrans = pNextTrans; + pSM->startTs = taosGetTimestampMs(); + + taosThreadMutexUnlock(&pTask->lock); + + int32_t code = pNextTrans->pAction(pSM->pTask); + + if (pTrans->autoInvokeEndFn) { + return streamTaskOnHandleEventSuccess(pSM); + } else { + return code; + } + } + } else { + taosThreadMutexUnlock(&pTask->lock); + + int64_t el = (taosGetTimestampMs() - pSM->startTs); + stDebug("s-task:%s handle event:%s completed, elapsed time:%" PRId64 "ms state:%s -> %s", pTask->id.idStr, + StreamTaskEventList[pTrans->event].name, el, current.name, pSM->current.name); + } + return TSDB_CODE_SUCCESS; } @@ -197,17 +284,19 @@ ETaskStatus streamTaskGetStatus(SStreamTask* pTask, char** pStr) { return s.state; } -STaskStateTrans createStateTransform(ETaskStatus current, ETaskStatus next, EStreamTaskEvent event, - __state_trans_fn preFn, __state_trans_fn fn, __state_trans_succ_fn succFn, - bool autoInvoke) { +STaskStateTrans createStateTransform(ETaskStatus current, ETaskStatus next, EStreamTaskEvent event, __state_trans_fn fn, + __state_trans_succ_fn succFn, SAttachedEventInfo* pEventInfo, bool autoInvoke) { STaskStateTrans trans = {0}; trans.state = StreamTaskStatusList[current]; trans.next = StreamTaskStatusList[next]; trans.event = event; - trans.preAction = (preFn != NULL)? preFn:dummyFn; - trans.pAction = (fn != NULL)? fn : dummyFn; - trans.pSuccAction = (succFn != NULL)? succFn:dummyFn; + if (pEventInfo != NULL) { + trans.attachEvent = *pEventInfo; + } + + trans.pAction = (fn != NULL) ? fn : dummyFn; + trans.pSuccAction = (succFn != NULL) ? succFn : dummyFn; trans.autoInvokeEndFn = autoInvoke; return trans; } @@ -221,49 +310,83 @@ int32_t initStateTransferTable(SStreamTaskSM* pSM) { } // initialization event handle - STaskStateTrans trans = createStateTransform(TASK_STATUS__UNINIT, TASK_STATUS__NORMAL, TASK_EVENT_INIT, NULL, - streamTaskInitStatus, onNormalTaskReady, false); + STaskStateTrans trans = createStateTransform(TASK_STATUS__UNINIT, TASK_STATUS__NORMAL, TASK_EVENT_INIT, + streamTaskInitStatus, onNormalTaskReady, false, false); taosArrayPush(pSM->pTransList, &trans); - trans = createStateTransform(TASK_STATUS__UNINIT, TASK_STATUS__SCAN_HISTORY, TASK_EVENT_INIT_SCAN_HISTORY, NULL, - streamTaskInitStatus, onScanhistoryTaskReady, false); + trans = createStateTransform(TASK_STATUS__UNINIT, TASK_STATUS__SCAN_HISTORY, TASK_EVENT_INIT_SCANHIST, + streamTaskInitStatus, onScanhistoryTaskReady, false, false); taosArrayPush(pSM->pTransList, &trans); - trans = createStateTransform(TASK_STATUS__SCAN_HISTORY, TASK_STATUS__NORMAL, TASK_EVENT_SCANHIST_COMPLETED, NULL, - streamTaskSetReadyForWal, NULL, true); + trans = createStateTransform(TASK_STATUS__UNINIT, TASK_STATUS_STREAM_SCAN_HISTORY, TASK_EVENT_INIT_STREAM_SCANHIST, + streamTaskInitStatus, onScanhistoryTaskReady, false, false); taosArrayPush(pSM->pTransList, &trans); - // pause & resume related event handle - trans = createStateTransform(TASK_STATUS__NORMAL, TASK_STATUS__PAUSE, TASK_EVENT_PAUSE, NULL, streamTaskDoPause, NULL, - true); + trans = createStateTransform(TASK_STATUS__SCAN_HISTORY, TASK_STATUS__NORMAL, TASK_EVENT_SCANHIST_DONE, + streamTaskSetReadyForWal, NULL, NULL, true); taosArrayPush(pSM->pTransList, &trans); - trans = createStateTransform(TASK_STATUS__PAUSE, TASK_STATUS__NORMAL, TASK_EVENT_RESUME, NULL, streamTaskDoResume, - NULL, true); + trans = createStateTransform(TASK_STATUS_STREAM_SCAN_HISTORY, TASK_STATUS__NORMAL, TASK_EVENT_SCANHIST_DONE, + streamTaskSetReadyForWal, NULL, NULL, true); taosArrayPush(pSM->pTransList, &trans); + // halt stream task, from other task status + trans = createStateTransform(TASK_STATUS__NORMAL, TASK_STATUS__HALT, TASK_EVENT_HALT, NULL, + streamTaskKeepCurrentVerInWal, NULL, true); + taosArrayPush(pSM->pTransList, &trans); + + SAttachedEventInfo info = {.status = TASK_STATUS__NORMAL, .event = TASK_EVENT_HALT}; + trans = createStateTransform(TASK_STATUS_STREAM_SCAN_HISTORY, TASK_STATUS__HALT, TASK_EVENT_HALT, NULL, + streamTaskKeepCurrentVerInWal, &info, true); + taosArrayPush(pSM->pTransList, &trans); + + trans = createStateTransform(TASK_STATUS__CK, TASK_STATUS__HALT, TASK_EVENT_HALT, NULL, streamTaskKeepCurrentVerInWal, + &info, true); + taosArrayPush(pSM->pTransList, &trans); + + trans = createStateTransform(TASK_STATUS__PAUSE, TASK_STATUS__HALT, TASK_EVENT_HALT, NULL, + streamTaskKeepCurrentVerInWal, NULL, true); + taosArrayPush(pSM->pTransList, &trans); + + // checkpoint related event trans = createStateTransform(TASK_STATUS__NORMAL, TASK_STATUS__CK, TASK_EVENT_GEN_CHECKPOINT, NULL, streamTaskDoCheckpoint, NULL, true); taosArrayPush(pSM->pTransList, &trans); - trans = createStateTransform(TASK_STATUS__CK, TASK_STATUS__NORMAL, TASK_EVENT_PAUSE, NULL, NULL, NULL, true); + trans = + createStateTransform(TASK_STATUS__CK, TASK_STATUS__NORMAL, TASK_EVENT_CHECKPOINT_DONE, NULL, NULL, NULL, true); taosArrayPush(pSM->pTransList, &trans); - // halt stream task, from other task status - trans = createStateTransform(TASK_STATUS__NORMAL, TASK_STATUS__HALT, TASK_EVENT_HALT, NULL, NULL, - streamTaskKeepCurrentVerInWal, true); + // pause & resume related event handle + trans = createStateTransform(TASK_STATUS__NORMAL, TASK_STATUS__PAUSE, TASK_EVENT_PAUSE, NULL, NULL, NULL, true); taosArrayPush(pSM->pTransList, &trans); - trans = createStateTransform(TASK_STATUS__SCAN_HISTORY, TASK_STATUS__HALT, TASK_EVENT_HALT, streamTaskWaitBeforeHalt, - NULL, streamTaskKeepCurrentVerInWal, true); + trans = createStateTransform(TASK_STATUS__SCAN_HISTORY, TASK_STATUS__PAUSE, TASK_EVENT_PAUSE, NULL, NULL, NULL, true); taosArrayPush(pSM->pTransList, &trans); - trans = createStateTransform(TASK_STATUS__CK, TASK_STATUS__HALT, TASK_EVENT_HALT, streamTaskWaitBeforeHalt, NULL, - streamTaskKeepCurrentVerInWal, true); + info = (SAttachedEventInfo){.status = TASK_STATUS__NORMAL, .event = TASK_EVENT_PAUSE}; + trans = createStateTransform(TASK_STATUS_STREAM_SCAN_HISTORY, TASK_STATUS__PAUSE, TASK_EVENT_PAUSE, NULL, NULL, &info, true); taosArrayPush(pSM->pTransList, &trans); - trans = createStateTransform(TASK_STATUS__PAUSE, TASK_STATUS__HALT, TASK_EVENT_HALT, NULL, NULL, - streamTaskKeepCurrentVerInWal, true); + trans = createStateTransform(TASK_STATUS__CK, TASK_STATUS__PAUSE, TASK_EVENT_PAUSE, NULL, NULL, &info, true); taosArrayPush(pSM->pTransList, &trans); + + trans = createStateTransform(TASK_STATUS__HALT, TASK_STATUS__PAUSE, TASK_EVENT_PAUSE, NULL, NULL, &info, true); + taosArrayPush(pSM->pTransList, &trans); + + trans = createStateTransform(TASK_STATUS__UNINIT, TASK_STATUS__PAUSE, TASK_EVENT_PAUSE, NULL, NULL, &info, true); + taosArrayPush(pSM->pTransList, &trans); + + trans = createStateTransform(TASK_STATUS__PAUSE, TASK_STATUS__PAUSE, TASK_EVENT_PAUSE, NULL, NULL, NULL, true); + taosArrayPush(pSM->pTransList, &trans); + + trans = createStateTransform(TASK_STATUS__STOP, TASK_STATUS__STOP, TASK_EVENT_PAUSE, NULL, NULL, NULL, true); + taosArrayPush(pSM->pTransList, &trans); + + trans = createStateTransform(TASK_STATUS__DROPPING, TASK_STATUS__DROPPING, TASK_EVENT_PAUSE, NULL, NULL, NULL, true); + taosArrayPush(pSM->pTransList, &trans); + + // resume is completed by restore status of state-machine + return 0; } \ No newline at end of file From 402aefd95debbfced124faaea9c92fedd4f0f7af Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 19 Oct 2023 15:09:47 +0800 Subject: [PATCH 05/41] refactor(stream): do some internal refactor. --- include/libs/stream/tstream.h | 13 +- source/common/src/tglobal.c | 2 +- source/dnode/mnode/impl/src/mndStream.c | 6 +- source/dnode/vnode/src/inc/tq.h | 2 +- source/dnode/vnode/src/tq/tq.c | 34 +++-- source/dnode/vnode/src/tq/tqSink.c | 6 +- source/dnode/vnode/src/tq/tqStreamTask.c | 15 +- source/libs/stream/inc/streamsm.h | 2 +- source/libs/stream/src/stream.c | 2 +- source/libs/stream/src/streamDispatch.c | 6 +- source/libs/stream/src/streamExec.c | 47 +++--- source/libs/stream/src/streamMeta.c | 22 +-- source/libs/stream/src/streamQueue.c | 4 +- source/libs/stream/src/streamStart.c | 54 +++---- source/libs/stream/src/streamTask.c | 12 +- source/libs/stream/src/streamTaskSm.c | 173 ++++++++++++++--------- 16 files changed, 213 insertions(+), 187 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 3b076fc9c3..0a541a34ab 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -67,7 +67,7 @@ enum { }; typedef enum ETaskStatus { - TASK_STATUS__NORMAL = 0, + TASK_STATUS__READY = 0, TASK_STATUS__DROPPING, TASK_STATUS__UNINIT, // not used, an placeholder TASK_STATUS__STOP, @@ -139,7 +139,7 @@ typedef enum EStreamTaskEvent { TASK_EVENT_PAUSE = 0x8, TASK_EVENT_RESUME = 0x9, TASK_EVENT_HALT = 0xA, - TASK_EVENT_TRANS_STATE = 0xB, + TASK_EVENT_DROPPING = 0xB, TASK_EVENT_SCAN_TSDB = 0xC, TASK_EVENT_SCAN_WAL = 0xD, } EStreamTaskEvent; @@ -714,12 +714,14 @@ SStreamChildEpInfo* streamTaskGetUpstreamTaskEpInfo(SStreamTask* pTask, int32_t void streamTaskInputFail(SStreamTask* pTask); int32_t streamExecTask(SStreamTask* pTask); int32_t streamSchedExec(SStreamTask* pTask); -bool streamTaskShouldStop(const SStreamStatus* pStatus); -bool streamTaskShouldPause(const SStreamStatus* pStatus); +bool streamTaskShouldStop(const SStreamTask* pStatus); +bool streamTaskShouldPause(const SStreamTask* pStatus); bool streamTaskIsIdle(const SStreamTask* pTask); char* createStreamTaskIdStr(int64_t streamId, int32_t taskId); -ETaskStatus streamTaskGetStatus(SStreamTask* pTask, char** pStr); +ETaskStatus streamTaskGetStatus(const SStreamTask* pTask, char** pStr); +void streamTaskResetStatus(SStreamTask* pTask); +void streamTaskSetStatusReady(SStreamTask* pTask); void initRpcMsg(SRpcMsg* pMsg, int32_t msgType, void* pCont, int32_t contLen); @@ -753,7 +755,6 @@ int32_t streamQueueGetNumOfItems(const SStreamQueue* pQueue); // common int32_t streamRestoreParam(SStreamTask* pTask); -int32_t streamSetStatusNormal(SStreamTask* pTask); int32_t streamSetStatusUnint(SStreamTask* pTask); const char* streamGetTaskStatusStr(int32_t status); void streamTaskPause(SStreamTask* pTask, SStreamMeta* pMeta); diff --git a/source/common/src/tglobal.c b/source/common/src/tglobal.c index 568808405e..4fb02bb021 100644 --- a/source/common/src/tglobal.c +++ b/source/common/src/tglobal.c @@ -244,7 +244,7 @@ int32_t tsTtlBatchDropNum = 10000; // number of tables dropped per batch // internal int32_t tsTransPullupInterval = 2; int32_t tsMqRebalanceInterval = 2; -int32_t tsStreamCheckpointInterval = 300; +int32_t tsStreamCheckpointInterval = 60; float tsSinkDataRate = 2.0; int32_t tsStreamNodeCheckInterval = 30; int32_t tsTtlUnit = 86400; diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index f23d596449..0a9e3c5336 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -1170,7 +1170,7 @@ static int32_t mndProcessStreamDoCheckpoint(SRpcMsg *pReq) { } } - { // check if all tasks are in TASK_STATUS__NORMAL status + { // check if all tasks are in TASK_STATUS__READY status bool ready = true; taosThreadMutexLock(&execNodeList.lock); @@ -1181,7 +1181,7 @@ static int32_t mndProcessStreamDoCheckpoint(SRpcMsg *pReq) { continue; } - if (pEntry->status != TASK_STATUS__NORMAL) { + if (pEntry->status != TASK_STATUS__READY) { mDebug("s-task:0x%" PRIx64 "-0x%x (nodeId:%d) status:%s not ready, checkpoint msg not issued", pEntry->id.streamId, (int32_t)pEntry->id.taskId, 0, streamGetTaskStatusStr(pEntry->status)); ready = false; @@ -2614,7 +2614,7 @@ int32_t mndProcessStreamHb(SRpcMsg *pReq) { } pEntry->status = p->status; - if (p->status != TASK_STATUS__NORMAL) { + if (p->status != TASK_STATUS__READY) { mDebug("received s-task:0x%"PRIx64" not in ready status:%s", p->id.taskId, streamGetTaskStatusStr(p->status)); } } diff --git a/source/dnode/vnode/src/inc/tq.h b/source/dnode/vnode/src/inc/tq.h index 96274ec102..9aed87d811 100644 --- a/source/dnode/vnode/src/inc/tq.h +++ b/source/dnode/vnode/src/inc/tq.h @@ -159,7 +159,7 @@ int32_t tqOffsetRestoreFromFile(STqOffsetStore* pStore, const char* fname); // tqStream int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t ver); int32_t tqScanWal(STQ* pTq); -int32_t tqCheckAndRunStreamTask(STQ* pTq); +int32_t tqStartStreamTask(STQ* pTq); int32_t tqStartStreamTasks(STQ* pTq); int32_t tqStopStreamTasks(STQ* pTq); diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 5e0a81314f..750a9d942a 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -847,11 +847,11 @@ int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t nextProcessVer) { pTask->exec.pWalReader = walOpenReader(pTq->pVnode->pWal, &cond, pTask->id.taskId); } - // reset the task status from unfinished transaction - if (pTask->status.taskStatus == TASK_STATUS__PAUSE) { - tqWarn("s-task:%s reset task status to be normal, status kept in taskMeta: Paused", pTask->id.idStr); - pTask->status.taskStatus = TASK_STATUS__NORMAL; - } +// // reset the task status from unfinished transaction +// if (pTask->status.taskStatus == TASK_STATUS__PAUSE) { +// tqWarn("s-task:%s reset task status to be normal, status kept in taskMeta: Paused", pTask->id.idStr); +// pTask->status.taskStatus = TASK_STATUS__READY; +// } streamTaskResetUpstreamStageInfo(pTask); streamSetupScheduleTrigger(pTask); @@ -1029,7 +1029,7 @@ int32_t tqProcessTaskDeployReq(STQ* pTq, int64_t sversion, char* msg, int32_t ms bool restored = pTq->pVnode->restored; if (p != NULL && restored && p->info.fillHistory == 0) { - EStreamTaskEvent event = (p->hTaskInfo.id.taskId == 0) ? TASK_EVENT_INIT : TASK_EVENT_INIT_SCANHIST; + EStreamTaskEvent event = (p->hTaskInfo.id.taskId == 0) ? TASK_EVENT_INIT : TASK_EVENT_INIT_STREAM_SCANHIST; streamTaskHandleEvent(p->status.pSM, event); } else if (!restored) { tqWarn("s-task:%s not launched since vnode(vgId:%d) not ready", p->id.idStr, vgId); @@ -1118,7 +1118,7 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { streamScanHistoryData(pTask); double el = (taosGetTimestampMs() - pTask->execInfo.step1Start) / 1000.0; - if (pTask->status.taskStatus == TASK_STATUS__PAUSE) { + if (streamTaskGetStatus(pTask, NULL) == TASK_STATUS__PAUSE) { int8_t status = streamTaskSetSchedStatusInactive(pTask); tqDebug("s-task:%s is paused in the step1, elapsed time:%.2fs, sched-status:%d", pTask->id.idStr, el, status); @@ -1228,7 +1228,7 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { streamTaskRestoreStatus(pTask); // if (pTask->status.taskStatus == TASK_STATUS__PAUSE) { -// pTask->status.keepTaskStatus = TASK_STATUS__NORMAL; +// pTask->status.keepTaskStatus = TASK_STATUS__READY; // qDebug("s-task:%s prev status is %s, update the kept status to be:%s when after step 2", id, // streamGetTaskStatusStr(TASK_STATUS__PAUSE), streamGetTaskStatusStr(pTask->status.keepTaskStatus)); // } @@ -1352,7 +1352,7 @@ int32_t tqProcessTaskRunReq(STQ* pTq, SRpcMsg* pMsg) { int32_t vgId = TD_VID(pTq->pVnode); if (taskId == STREAM_EXEC_TASK_STATUS_CHECK_ID) { - tqCheckAndRunStreamTask(pTq); + tqStartStreamTask(pTq); return 0; } @@ -1365,7 +1365,7 @@ int32_t tqProcessTaskRunReq(STQ* pTq, SRpcMsg* pMsg) { if (pTask != NULL) { // even in halt status, the data in inputQ must be processed char* p = NULL; ETaskStatus st = streamTaskGetStatus(pTask, &p); - if (st == TASK_STATUS__NORMAL || st == TASK_STATUS__SCAN_HISTORY || st == TASK_STATUS__CK) { + if (st == TASK_STATUS__READY || st == TASK_STATUS__SCAN_HISTORY || st == TASK_STATUS__CK) { tqDebug("vgId:%d s-task:%s start to process block from inputQ, next checked ver:%" PRId64, vgId, pTask->id.idStr, pTask->chkInfo.nextProcessVer); streamExecTask(pTask); @@ -1514,7 +1514,6 @@ int32_t tqProcessTaskResumeImpl(STQ* pTq, SStreamTask* pTask, int64_t sversion, return -1; } - // todo: handle the case: resume from halt to pause/ from halt to normal/ from pause to normal streamTaskResume(pTask, pTq->pStreamMeta); int32_t level = pTask->info.taskLevel; @@ -1523,8 +1522,8 @@ int32_t tqProcessTaskResumeImpl(STQ* pTq, SStreamTask* pTask, int64_t sversion, return 0; } - int8_t status = pTask->status.taskStatus; - if (status == TASK_STATUS__NORMAL || status == TASK_STATUS__SCAN_HISTORY || status == TASK_STATUS__CK) { + ETaskStatus status = streamTaskGetStatus(pTask, NULL); + if (status == TASK_STATUS__READY || status == TASK_STATUS__SCAN_HISTORY || status == TASK_STATUS__CK) { // no lock needs to secure the access of the version if (igUntreated && level == TASK_LEVEL__SOURCE && !pTask->info.fillHistory) { // discard all the data when the stream task is suspended. @@ -1537,8 +1536,7 @@ int32_t tqProcessTaskResumeImpl(STQ* pTq, SStreamTask* pTask, int64_t sversion, vgId, pTask->id.idStr, pTask->chkInfo.nextProcessVer, sversion, pTask->status.schedStatus); } - if (level == TASK_LEVEL__SOURCE && pTask->info.fillHistory && - pTask->status.taskStatus == TASK_STATUS__SCAN_HISTORY) { + if (level == TASK_LEVEL__SOURCE && pTask->info.fillHistory && status == TASK_STATUS__SCAN_HISTORY) { streamStartScanHistoryAsync(pTask, igUntreated); } else if (level == TASK_LEVEL__SOURCE && (streamQueueGetNumOfItems(pTask->inputInfo.queue) == 0)) { tqScanWalAsync(pTq, false); @@ -1867,7 +1865,7 @@ int32_t tqProcessTaskUpdateReq(STQ* pTq, SRpcMsg* pMsg) { tqDebug("s-task:%s receive nodeEp update msg from mnode", pTask->id.idStr); streamTaskUpdateEpsetInfo(pTask, req.pNodeList); - streamSetStatusNormal(pTask); + streamTaskResetStatus(pTask); SStreamTask** ppHTask = NULL; if (pTask->hTaskInfo.id.taskId != 0) { @@ -1971,10 +1969,10 @@ int32_t tqProcessTaskResetReq(STQ* pTq, SRpcMsg* pMsg) { tqDebug("s-task:%s receive task-reset msg from mnode, reset status and ready for data processing", pTask->id.idStr); // clear flag set during do checkpoint, and open inputQ for all upstream tasks - if (pTask->status.taskStatus == TASK_STATUS__CK) { + if (streamTaskGetStatus(pTask, NULL) == TASK_STATUS__CK) { streamTaskClearCheckInfo(pTask); taosArrayClear(pTask->pReadyMsgList); - streamSetStatusNormal(pTask); + streamTaskSetStatusReady(pTask); } streamMetaReleaseTask(pMeta, pTask); diff --git a/source/dnode/vnode/src/tq/tqSink.c b/source/dnode/vnode/src/tq/tqSink.c index 7d1c754005..5dafe6a4a0 100644 --- a/source/dnode/vnode/src/tq/tqSink.c +++ b/source/dnode/vnode/src/tq/tqSink.c @@ -592,7 +592,7 @@ int32_t doWaitForDstTableCreated(SVnode* pVnode, SStreamTask* pTask, STableSinkI const char* id = pTask->id.idStr; while (pTableSinkInfo->uid == 0) { - if (streamTaskShouldStop(&pTask->status)) { + if (streamTaskShouldStop(pTask)) { tqDebug("s-task:%s task will stop, quit from waiting for table:%s create", id, dstTableName); return TSDB_CODE_STREAM_EXEC_CANCELLED; } @@ -773,7 +773,7 @@ void tqSinkDataIntoDstTable(SStreamTask* pTask, void* vnode, void* data) { numOfBlocks); for(int32_t i = 0; i < numOfBlocks; ++i) { - if (streamTaskShouldStop(&pTask->status)) { + if (streamTaskShouldStop(pTask)) { return; } @@ -823,7 +823,7 @@ void tqSinkDataIntoDstTable(SStreamTask* pTask, void* vnode, void* data) { bool hasSubmit = false; for (int32_t i = 0; i < numOfBlocks; i++) { - if (streamTaskShouldStop(&pTask->status)) { + if (streamTaskShouldStop(pTask)) { taosHashCleanup(pTableIndexMap); tDestroySubmitReq(&submitReq, TSDB_MSG_FLG_ENCODE); return; diff --git a/source/dnode/vnode/src/tq/tqStreamTask.c b/source/dnode/vnode/src/tq/tqStreamTask.c index be64ec20a4..8dfa1e2670 100644 --- a/source/dnode/vnode/src/tq/tqStreamTask.c +++ b/source/dnode/vnode/src/tq/tqStreamTask.c @@ -60,7 +60,7 @@ int32_t tqScanWal(STQ* pTq) { return 0; } -int32_t tqCheckAndRunStreamTask(STQ* pTq) { +int32_t tqStartStreamTask(STQ* pTq) { int32_t vgId = TD_VID(pTq->pVnode); SStreamMeta* pMeta = pTq->pStreamMeta; @@ -99,7 +99,7 @@ int32_t tqCheckAndRunStreamTask(STQ* pTq) { continue; } - EStreamTaskEvent event = (pTask->hTaskInfo.id.taskId == 0) ? TASK_EVENT_INIT : TASK_EVENT_INIT_SCANHIST; + EStreamTaskEvent event = (pTask->hTaskInfo.id.taskId == 0) ? TASK_EVENT_INIT : TASK_EVENT_INIT_STREAM_SCANHIST; streamTaskHandleEvent(pTask->status.pSM, event); streamMetaReleaseTask(pMeta, pTask); } @@ -240,9 +240,8 @@ int32_t tqStartStreamTasks(STQ* pTq) { STaskId id = {.streamId = pTaskId->streamId, .taskId = pTaskId->taskId}; SStreamTask** pTask = taosHashGet(pMeta->pTasksMap, &id, sizeof(id)); - int8_t status = (*pTask)->status.taskStatus; - if (status == TASK_STATUS__STOP && (*pTask)->info.fillHistory != 1) { - streamSetStatusNormal(*pTask); + if ((*pTask)->info.fillHistory != 1) { + streamTaskResetStatus(*pTask); } } @@ -327,14 +326,14 @@ static bool taskReadyForDataFromWal(SStreamTask* pTask) { // int32_t status = pTask->status.taskStatus; char* p = NULL; int32_t status = streamTaskGetStatus(pTask, &p); - if (streamTaskGetStatus(pTask, &p) != TASK_STATUS__NORMAL) { + if (streamTaskGetStatus(pTask, &p) != TASK_STATUS__READY) { tqTrace("s-task:%s not ready for submit block in wal, status:%s", pTask->id.idStr, p); return false; } // fill-history task has entered into the last phase, no need to anything if ((pTask->info.fillHistory == 1) && pTask->status.appendTranstateBlock) { - ASSERT(status == TASK_STATUS__NORMAL); + ASSERT(status == 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); @@ -449,7 +448,7 @@ int32_t doScanWalForAllTasks(SStreamMeta* pStreamMeta, bool* pScanIdle) { char* p = NULL; ETaskStatus status = streamTaskGetStatus(pTask, &p); - if (status != TASK_STATUS__NORMAL) { + if (status != TASK_STATUS__READY) { tqDebug("s-task:%s not ready for submit block from wal, status:%s", pTask->id.idStr, p); taosThreadMutexUnlock(&pTask->lock); streamMetaReleaseTask(pStreamMeta, pTask); diff --git a/source/libs/stream/inc/streamsm.h b/source/libs/stream/inc/streamsm.h index b1ccc19e23..19eb3c0029 100644 --- a/source/libs/stream/inc/streamsm.h +++ b/source/libs/stream/inc/streamsm.h @@ -48,7 +48,7 @@ typedef struct STaskStateTrans { struct SStreamTaskSM { SStreamTask* pTask; - SArray* pTransList; // SArray +// SArray* pTransList; // SArray STaskStateTrans* pActiveTrans; int64_t startTs; SStreamTaskState current; diff --git a/source/libs/stream/src/stream.c b/source/libs/stream/src/stream.c index 91205a216b..307e1c47c9 100644 --- a/source/libs/stream/src/stream.c +++ b/source/libs/stream/src/stream.c @@ -64,7 +64,7 @@ static void streamSchedByTimer(void* param, void* tmrId) { int8_t status = atomic_load_8(&pTask->schedInfo.status); stDebug("s-task:%s in scheduler, trigger status:%d, next:%dms", id, status, nextTrigger); - if (streamTaskShouldStop(&pTask->status) || streamTaskShouldPause(&pTask->status)) { + if (streamTaskShouldStop(pTask) || streamTaskShouldPause(pTask)) { stDebug("s-task:%s jump out of schedTimer", id); return; } diff --git a/source/libs/stream/src/streamDispatch.c b/source/libs/stream/src/streamDispatch.c index b339cb6969..e6378d309e 100644 --- a/source/libs/stream/src/streamDispatch.c +++ b/source/libs/stream/src/streamDispatch.c @@ -420,7 +420,7 @@ static void doRetryDispatchData(void* param, void* tmrId) { const char* id = pTask->id.idStr; int32_t msgId = pTask->execInfo.dispatch; - if (streamTaskShouldStop(&pTask->status)) { + if (streamTaskShouldStop(pTask)) { int32_t ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1); stDebug("s-task:%s should stop, abort from timer, ref:%d", pTask->id.idStr, ref); return; @@ -474,10 +474,10 @@ static void doRetryDispatchData(void* param, void* tmrId) { } if (code != TSDB_CODE_SUCCESS) { - if (!streamTaskShouldStop(&pTask->status)) { + if (!streamTaskShouldStop(pTask)) { // stDebug("s-task:%s reset the waitRspCnt to be 0 before launch retry dispatch", pTask->id.idStr); // atomic_store_32(&pTask->outputInfo.shuffleDispatcher.waitingRspCnt, 0); - if (streamTaskShouldPause(&pTask->status)) { + if (streamTaskShouldPause(pTask)) { streamRetryDispatchData(pTask, DISPATCH_RETRY_INTERVAL_MS * 10); } else { streamRetryDispatchData(pTask, DISPATCH_RETRY_INTERVAL_MS); diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index da7e2ece0d..346f6cefcb 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -22,14 +22,13 @@ static int32_t streamDoTransferStateToStreamTask(SStreamTask* pTask); -bool streamTaskShouldStop(const SStreamStatus* pStatus) { - int32_t status = atomic_load_8((int8_t*)&pStatus->taskStatus); - return (status == TASK_STATUS__STOP) || (status == TASK_STATUS__DROPPING); +bool streamTaskShouldStop(const SStreamTask* pTask) { + ETaskStatus s = streamTaskGetStatus(pTask, NULL); + return (s == TASK_STATUS__STOP) || (s == TASK_STATUS__DROPPING); } -bool streamTaskShouldPause(const SStreamStatus* pStatus) { - int32_t status = atomic_load_8((int8_t*)&pStatus->taskStatus); - return (status == TASK_STATUS__PAUSE); +bool streamTaskShouldPause(const SStreamTask* pTask) { + return (streamTaskGetStatus(pTask, NULL) == TASK_STATUS__PAUSE); } static int32_t doOutputResultBlockImpl(SStreamTask* pTask, SStreamDataBlock* pBlock) { @@ -102,7 +101,7 @@ static int32_t streamTaskExecImpl(SStreamTask* pTask, SStreamQueueItem* pItem, i pRes = taosArrayInit(4, sizeof(SSDataBlock)); } - if (streamTaskShouldStop(&pTask->status)) { + if (streamTaskShouldStop(pTask)) { taosArrayDestroyEx(pRes, (FDelete)blockDataFreeRes); return 0; } @@ -198,7 +197,7 @@ int32_t streamScanHistoryData(SStreamTask* pTask) { qSetStreamOpOpen(exec); while (!finished) { - if (streamTaskShouldPause(&pTask->status)) { + if (streamTaskShouldPause(pTask)) { double el = (taosGetTimestampMs() - pTask->execInfo.step1Start) / 1000.0; stDebug("s-task:%s paused from the scan-history task, elapsed time:%.2fsec", pTask->id.idStr, el); break; @@ -213,7 +212,7 @@ int32_t streamScanHistoryData(SStreamTask* pTask) { int32_t size = 0; int32_t numOfBlocks = 0; while (1) { - if (streamTaskShouldStop(&pTask->status)) { + if (streamTaskShouldStop(pTask)) { taosArrayDestroyEx(pRes, (FDelete)blockDataFreeRes); return 0; } @@ -309,20 +308,18 @@ int32_t streamDoTransferStateToStreamTask(SStreamTask* pTask) { pStreamTask->id.idStr); } - ASSERT(((pStreamTask->status.taskStatus == TASK_STATUS__STOP) || - (pStreamTask->hTaskInfo.id.taskId == pTask->id.taskId)) && + int8_t status = streamTaskGetStatus(pStreamTask, NULL); + ASSERT(((status == TASK_STATUS__DROPPING) || (pStreamTask->hTaskInfo.id.taskId == pTask->id.taskId)) && pTask->status.appendTranstateBlock == true); STimeWindow* pTimeWindow = &pStreamTask->dataRange.window; - // todo. the dropping status should be append to the status after the halt completed. // It must be halted for a source stream task, since when the related scan-history-data task start scan the history // for the step 2. - int8_t status = streamTaskGetStatus(pStreamTask, NULL);//pStreamTask->status.taskStatus; if (pStreamTask->info.taskLevel == TASK_LEVEL__SOURCE) { - ASSERT(status == TASK_STATUS__HALT || status == TASK_STATUS__DROPPING); + ASSERT(status == TASK_STATUS__HALT || status == TASK_STATUS__DROPPING || status == TASK_STATUS__STOP); } else { - ASSERT(status == TASK_STATUS__NORMAL); + ASSERT(status == TASK_STATUS__READY|| status == TASK_STATUS__DROPPING || status == TASK_STATUS__STOP); streamTaskHandleEvent(pStreamTask->status.pSM, TASK_EVENT_HALT); stDebug("s-task:%s halt by related fill-history task:%s", pStreamTask->id.idStr, pTask->id.idStr); } @@ -333,13 +330,14 @@ int32_t streamDoTransferStateToStreamTask(SStreamTask* pTask) { // In case of sink tasks, no need to halt them. // In case of source tasks and agg tasks, we should HALT them, and wait for them to be idle. And then, it's safe to // start the task state transfer procedure. - // When a task is idle with halt status, all data in inputQ are consumed. + char* p = NULL; + streamTaskGetStatus(pStreamTask, &p); if (pStreamTask->info.taskLevel == TASK_LEVEL__SOURCE) { // update the scan data range for source task. stDebug("s-task:%s level:%d stream task window %" PRId64 " - %" PRId64 " update to %" PRId64 " - %" PRId64 - ", status:normal, sched-status:%d", + ", status:%s, sched-status:%d", pStreamTask->id.idStr, TASK_LEVEL__SOURCE, pTimeWindow->skey, pTimeWindow->ekey, INT64_MIN, - pTimeWindow->ekey, pStreamTask->status.schedStatus); + pTimeWindow->ekey, p, pStreamTask->status.schedStatus); } else { stDebug("s-task:%s no need to update time window for non-source task", pStreamTask->id.idStr); } @@ -366,6 +364,8 @@ int32_t streamDoTransferStateToStreamTask(SStreamTask* pTask) { // 6. save to disk taosWLockLatch(&pMeta->lock); + + pStreamTask->status.taskStatus = streamTaskGetStatus(pStreamTask, NULL); streamMetaSaveTask(pMeta, pStreamTask); if (streamMetaCommit(pMeta) < 0) { // persist to disk @@ -525,7 +525,7 @@ int32_t streamExecForAll(SStreamTask* pTask) { int32_t blockSize = 0; int32_t numOfBlocks = 0; SStreamQueueItem* pInput = NULL; - if (streamTaskShouldStop(&pTask->status)) { + if (streamTaskShouldStop(pTask)) { stDebug("s-task:%s stream task is stopped", id); break; } @@ -605,8 +605,9 @@ int32_t streamExecForAll(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) { - return (pTask->status.schedStatus == TASK_SCHED_STATUS__INACTIVE || pTask->status.taskStatus == TASK_STATUS__STOP || - pTask->status.taskStatus == TASK_STATUS__DROPPING); + ETaskStatus status = streamTaskGetStatus(pTask, NULL); + return (pTask->status.schedStatus == TASK_SCHED_STATUS__INACTIVE || status == TASK_STATUS__STOP || + status == TASK_STATUS__DROPPING); } int32_t streamExecTask(SStreamTask* pTask) { @@ -623,8 +624,8 @@ int32_t streamExecTask(SStreamTask* pTask) { } taosThreadMutexLock(&pTask->lock); - if ((streamQueueGetNumOfItems(pTask->inputInfo.queue) == 0) || streamTaskShouldStop(&pTask->status) || - streamTaskShouldPause(&pTask->status)) { + if ((streamQueueGetNumOfItems(pTask->inputInfo.queue) == 0) || streamTaskShouldStop(pTask) || + streamTaskShouldPause(pTask)) { atomic_store_8(&pTask->status.schedStatus, TASK_SCHED_STATUS__INACTIVE); taosThreadMutexUnlock(&pTask->lock); diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index a80e3a05f0..ee4f1e2340 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -461,7 +461,7 @@ SStreamTask* streamMetaAcquireTask(SStreamMeta* pMeta, int64_t streamId, int32_t STaskId id = {.streamId = streamId, .taskId = taskId}; SStreamTask** ppTask = (SStreamTask**)taosHashGet(pMeta->pTasksMap, &id, sizeof(id)); if (ppTask != NULL) { - if (!streamTaskShouldStop(&(*ppTask)->status)) { + if (!streamTaskShouldStop(*ppTask)) { int32_t ref = atomic_add_fetch_32(&(*ppTask)->refCnt, 1); taosRUnLockLatch(&pMeta->lock); stTrace("s-task:%s acquire task, ref:%d", (*ppTask)->id.idStr, ref); @@ -478,7 +478,7 @@ void streamMetaReleaseTask(SStreamMeta* UNUSED_PARAM(pMeta), SStreamTask* pTask) if (ref > 0) { stTrace("s-task:%s release task, ref:%d", pTask->id.idStr, ref); } else if (ref == 0) { - ASSERT(streamTaskShouldStop(&pTask->status)); + ASSERT(streamTaskShouldStop(pTask)); stTrace("s-task:%s all refs are gone, free it", pTask->id.idStr); tFreeStreamTask(pTask); } else if (ref < 0) { @@ -506,11 +506,15 @@ int32_t streamMetaUnregisterTask(SStreamMeta* pMeta, int64_t streamId, int32_t t SStreamTask** ppTask = (SStreamTask**)taosHashGet(pMeta->pTasksMap, &id, sizeof(id)); if (ppTask) { pTask = *ppTask; - if (streamTaskShouldPause(&pTask->status)) { + + // desc the paused task counter + if (streamTaskShouldPause(pTask)) { int32_t num = atomic_sub_fetch_32(&pMeta->numOfPausedTasks, 1); stInfo("vgId:%d s-task:%s drop stream task. pause task num:%d", pMeta->vgId, pTask->id.idStr, num); } - atomic_store_8(&pTask->status.taskStatus, TASK_STATUS__DROPPING); + + // handle the dropping event + streamTaskHandleEvent(pTask->status.pSM, TASK_EVENT_DROPPING); } else { stDebug("vgId:%d failed to find the task:0x%x, it may be dropped already", pMeta->vgId, taskId); taosWUnLockLatch(&pMeta->lock); @@ -522,8 +526,8 @@ int32_t streamMetaUnregisterTask(SStreamMeta* pMeta, int64_t streamId, int32_t t while (1) { taosRLockLatch(&pMeta->lock); - ppTask = (SStreamTask**)taosHashGet(pMeta->pTasksMap, &id, sizeof(id)); + ppTask = (SStreamTask**)taosHashGet(pMeta->pTasksMap, &id, sizeof(id)); if (ppTask) { if ((*ppTask)->status.timerActive == 0) { taosRUnLockLatch(&pMeta->lock); @@ -548,15 +552,13 @@ int32_t streamMetaUnregisterTask(SStreamMeta* pMeta, int64_t streamId, int32_t t STaskId streamTaskId = {.streamId = (*ppTask)->streamTaskId.streamId, .taskId = (*ppTask)->streamTaskId.taskId}; SStreamTask** ppStreamTask = (SStreamTask**)taosHashGet(pMeta->pTasksMap, &streamTaskId, sizeof(streamTaskId)); if (ppStreamTask != NULL) { - (*ppStreamTask)->hTaskInfo.id.taskId = 0; - (*ppStreamTask)->hTaskInfo.id.streamId = 0; + CLEAR_RELATED_FILLHISTORY_TASK((*ppStreamTask)); } } else { atomic_sub_fetch_32(&pMeta->numOfStreamTasks, 1); } taosHashRemove(pMeta->pTasksMap, &id, sizeof(id)); - atomic_store_8(&pTask->status.taskStatus, TASK_STATUS__DROPPING); ASSERT(pTask->status.timerActive == 0); doRemoveIdFromList(pMeta, (int32_t)taosArrayGetSize(pMeta->pTaskList), &pTask->id); @@ -702,8 +704,8 @@ int32_t streamMetaLoadAllTasks(SStreamMeta* pMeta) { tFreeStreamTask(pTask); STaskId id = streamTaskExtractKey(pTask); - taosArrayPush(pRecycleList, &id); + int32_t total = taosArrayGetSize(pRecycleList); stDebug("s-task:0x%x is already dropped, add into recycle list, total:%d", taskId, total); continue; @@ -739,7 +741,7 @@ int32_t streamMetaLoadAllTasks(SStreamMeta* pMeta) { atomic_add_fetch_32(&pMeta->numOfStreamTasks, 1); } - if (streamTaskShouldPause(&pTask->status)) { + if (streamTaskShouldPause(pTask)) { atomic_add_fetch_32(&pMeta->numOfPausedTasks, 1); } diff --git a/source/libs/stream/src/streamQueue.c b/source/libs/stream/src/streamQueue.c index e396ac77b4..70a065c22e 100644 --- a/source/libs/stream/src/streamQueue.c +++ b/source/libs/stream/src/streamQueue.c @@ -165,7 +165,7 @@ int32_t streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueItem** pInpu } while (1) { - if (streamTaskShouldPause(&pTask->status) || streamTaskShouldStop(&pTask->status)) { + if (streamTaskShouldPause(pTask) || streamTaskShouldStop(pTask)) { stDebug("s-task:%s task should pause, extract input blocks:%d", pTask->id.idStr, *numOfBlocks); return TSDB_CODE_SUCCESS; } @@ -346,7 +346,7 @@ int32_t streamTaskPutDataIntoOutputQ(SStreamTask* pTask, SStreamDataBlock* pBloc STaosQueue* pQueue = pTask->outputq.queue->pQueue; while (streamQueueIsFull(pTask->outputq.queue)) { - if (streamTaskShouldStop(&pTask->status)) { + if (streamTaskShouldStop(pTask)) { stInfo("s-task:%s discard result block due to task stop", pTask->id.idStr); return TSDB_CODE_STREAM_EXEC_CANCELLED; } diff --git a/source/libs/stream/src/streamStart.c b/source/libs/stream/src/streamStart.c index 003596ce90..58ea042079 100644 --- a/source/libs/stream/src/streamStart.c +++ b/source/libs/stream/src/streamStart.c @@ -263,14 +263,14 @@ int32_t onNormalTaskReady(SStreamTask* pTask) { char* p = NULL; ETaskStatus status = streamTaskGetStatus(pTask, &p); - ASSERT(status == TASK_STATUS__NORMAL); + ASSERT(status == TASK_STATUS__READY); // todo refactor: remove this later - if (pTask->info.fillHistory == 1) { - stDebug("s-task:%s fill-history is set normal when start it, try to remove it,set it task to be dropping", id); - pTask->status.taskStatus = TASK_STATUS__DROPPING; - ASSERT(pTask->hTaskInfo.id.taskId == 0); - } +// if (pTask->info.fillHistory == 1) { +// stDebug("s-task:%s fill-history is set normal when start it, try to remove it,set it task to be dropping", id); +// pTask->status.taskStatus = TASK_STATUS__DROPPING; +// ASSERT(pTask->hTaskInfo.id.taskId == 0); +// } if (pTask->info.taskLevel == TASK_LEVEL__SOURCE) { stDebug("s-task:%s no need to scan-history data, status:%s, sched-status:%d, ready for data from wal ver:%" PRId64, @@ -315,7 +315,7 @@ static void doProcessDownstreamReadyRsp(SStreamTask* pTask) { int8_t status = pTask->status.taskStatus; const char* str = streamGetTaskStatusStr(status); - ASSERT(status == TASK_STATUS__SCAN_HISTORY || status == TASK_STATUS__NORMAL); + ASSERT(status == TASK_STATUS__SCAN_HISTORY || status == TASK_STATUS__READY); streamTaskSetRangeStreamCalc(pTask); if (status == TASK_STATUS__SCAN_HISTORY) { @@ -341,7 +341,7 @@ int32_t streamProcessCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* pRs ASSERT(pTask->id.taskId == pRsp->upstreamTaskId); const char* id = pTask->id.idStr; - if (streamTaskShouldStop(&pTask->status)) { + if (streamTaskShouldStop(pTask)) { stDebug("s-task:%s should stop, do not do check downstream again", id); return TSDB_CODE_SUCCESS; } @@ -447,18 +447,6 @@ int32_t streamRestoreParam(SStreamTask* pTask) { return qRestoreStreamOperatorOption(pTask->exec.pExecutor); } -int32_t streamSetStatusNormal(SStreamTask* pTask) { - int32_t status = atomic_load_8(&pTask->status.taskStatus); - if (status == TASK_STATUS__DROPPING) { - stError("s-task:%s cannot be set normal, since in dropping state", pTask->id.idStr); - return -1; - } else { - stDebug("s-task:%s set task status to be normal, prev:%s", pTask->id.idStr, streamGetTaskStatusStr(status)); - atomic_store_8(&pTask->status.taskStatus, TASK_STATUS__NORMAL); - return 0; - } -} - // source int32_t streamSetParamForStreamScannerStep1(SStreamTask* pTask, SVersionRange *pVerRange, STimeWindow* pWindow) { return qStreamSourceScanParamForHistoryScanStep1(pTask->exec.pExecutor, pVerRange, pWindow); @@ -636,7 +624,7 @@ static void tryLaunchHistoryTask(void* param, void* tmrId) { if (ppTask) { ASSERT((*ppTask)->status.timerActive >= 1); - if (streamTaskShouldStop(&(*ppTask)->status)) { + if (streamTaskShouldStop(*ppTask)) { char* p = NULL; streamTaskGetStatus((*ppTask), &p); int32_t ref = atomic_sub_fetch_32(&(*ppTask)->status.timerActive, 1); @@ -677,7 +665,7 @@ static void tryLaunchHistoryTask(void* param, void* tmrId) { // abort the timer if intend to stop task SStreamTask* pHTask = streamMetaAcquireTask(pMeta, pHTaskInfo->id.streamId, pHTaskInfo->id.taskId); - if (pHTask == NULL && (!streamTaskShouldStop(&pTask->status))) { + if (pHTask == NULL && (!streamTaskShouldStop(pTask))) { char* p = NULL; int32_t hTaskId = pHTaskInfo->id.taskId; @@ -776,7 +764,7 @@ int32_t streamLaunchFillHistoryTask(SStreamTask* pTask) { } int32_t streamTaskScanHistoryDataComplete(SStreamTask* pTask) { - if (atomic_load_8(&pTask->status.taskStatus) == TASK_STATUS__DROPPING) { + if (streamTaskGetStatus(pTask, NULL) == TASK_STATUS__DROPPING) { return 0; } @@ -1049,7 +1037,7 @@ void streamTaskResume(SStreamTask* pTask, SStreamMeta* pMeta) { int8_t status = pTask->status.taskStatus; if (status == TASK_STATUS__PAUSE) { pTask->status.taskStatus = pTask->status.keepTaskStatus; - pTask->status.keepTaskStatus = TASK_STATUS__NORMAL; + pTask->status.keepTaskStatus = TASK_STATUS__READY; int32_t num = atomic_sub_fetch_32(&pMeta->numOfPausedTasks, 1); stInfo("vgId:%d s-task:%s resume from pause, status:%s. pause task num:%d", pMeta->vgId, pTask->id.idStr, streamGetTaskStatusStr(status), num); } else if (pTask->info.taskLevel == TASK_LEVEL__SINK) { @@ -1065,14 +1053,14 @@ void streamTaskResume(SStreamTask* pTask, SStreamMeta* pMeta) { // todo fix race condition void streamTaskDisablePause(SStreamTask* pTask) { // pre-condition check - const char* id = pTask->id.idStr; - while (pTask->status.taskStatus == TASK_STATUS__PAUSE) { - stDebug("s-task:%s already in pause, wait for pause being cancelled, and set pause disabled, recheck in 100ms", id); - taosMsleep(100); - } - - stDebug("s-task:%s disable task pause", id); - pTask->status.pauseAllowed = 0; +// const char* id = pTask->id.idStr; +// while (pTask->status.taskStatus == TASK_STATUS__PAUSE) { +// stDebug("s-task:%s already in pause, wait for pause being cancelled, and set pause disabled, recheck in 100ms", id); +// taosMsleep(100); +// } +// +// stDebug("s-task:%s disable task pause", id); +// pTask->status.pauseAllowed = 0; } void streamTaskEnablePause(SStreamTask* pTask) { @@ -1092,7 +1080,7 @@ void streamTaskResumeFromHalt(SStreamTask* pTask) { // } // pTask->status.taskStatus = pTask->status.keepTaskStatus; -// pTask->status.keepTaskStatus = TASK_STATUS__NORMAL; +// pTask->status.keepTaskStatus = TASK_STATUS__READY; streamTaskRestoreStatus(pTask); streamTaskGetStatus(pTask, &p); stDebug("s-task:%s resume from halt, current status:%s", id, p); diff --git a/source/libs/stream/src/streamTask.c b/source/libs/stream/src/streamTask.c index b20bd9421c..fdcfcfa9a9 100644 --- a/source/libs/stream/src/streamTask.c +++ b/source/libs/stream/src/streamTask.c @@ -58,7 +58,7 @@ SStreamTask* tNewStreamTask(int64_t streamId, int8_t taskLevel, bool fillHistory pTask->id.idStr = taosStrdup(buf); pTask->status.schedStatus = TASK_SCHED_STATUS__INACTIVE; - pTask->status.taskStatus = (fillHistory || hasFillhistory) ? TASK_STATUS__SCAN_HISTORY : TASK_STATUS__NORMAL; + pTask->status.taskStatus = (fillHistory || hasFillhistory) ? TASK_STATUS__SCAN_HISTORY : TASK_STATUS__READY; pTask->inputInfo.status = TASK_INPUT_STATUS__NORMAL; pTask->outputq.status = TASK_OUTPUT_STATUS__NORMAL; @@ -581,13 +581,7 @@ int32_t streamTaskStop(SStreamTask* pTask) { int64_t st = taosGetTimestampMs(); const char* id = pTask->id.idStr; - taosThreadMutexLock(&pTask->lock); - if (pTask->status.taskStatus == TASK_STATUS__CK) { - stDebug("s-task:%s in checkpoint will be discarded since task is stopped", id); - } - pTask->status.taskStatus = TASK_STATUS__STOP; - taosThreadMutexUnlock(&pTask->lock); - + streamTaskHandleEvent(pTask->status.pSM, TASK_EVENT_STOP); qKillTask(pTask->exec.pExecutor, TSDB_CODE_SUCCESS); while (/*pTask->status.schedStatus != TASK_SCHED_STATUS__INACTIVE */ !streamTaskIsIdle(pTask)) { stDebug("s-task:%s level:%d wait for task to be idle and then close, check again in 100ms", id, @@ -740,7 +734,7 @@ void streamTaskSetRetryInfoForLaunch(SHistoryTaskInfo* pInfo) { const char* streamGetTaskStatusStr(int32_t status) { switch(status) { - case TASK_STATUS__NORMAL: return "normal"; + case TASK_STATUS__READY: return "normal"; case TASK_STATUS__SCAN_HISTORY: return "scan-history"; case TASK_STATUS__HALT: return "halt"; case TASK_STATUS__PAUSE: return "paused"; diff --git a/source/libs/stream/src/streamTaskSm.c b/source/libs/stream/src/streamTaskSm.c index bb7c168922..49a434af94 100644 --- a/source/libs/stream/src/streamTaskSm.c +++ b/source/libs/stream/src/streamTaskSm.c @@ -21,7 +21,7 @@ #include "wal.h" SStreamTaskState StreamTaskStatusList[9] = { - {.state = TASK_STATUS__NORMAL, .name = "normal"}, + {.state = TASK_STATUS__READY, .name = "ready"}, {.state = TASK_STATUS__DROPPING, .name = "dropped"}, {.state = TASK_STATUS__UNINIT, .name = "uninit"}, {.state = TASK_STATUS__STOP, .name = "stop"}, @@ -45,9 +45,13 @@ SStreamEventInfo StreamTaskEventList[10] = { {.event = TASK_EVENT_HALT, .name = "halting"}, }; +static TdThreadOnce streamTaskStateMachineInit = PTHREAD_ONCE_INIT; +static SArray* streamTaskSMTrans = NULL; + static int32_t streamTaskInitStatus(SStreamTask* pTask); static int32_t streamTaskKeepCurrentVerInWal(SStreamTask* pTask); -static int32_t initStateTransferTable(SStreamTaskSM* pSM); +static int32_t initStateTransferTable(); +static void doInitStateTransferTable(void); static STaskStateTrans createStateTransform(ETaskStatus current, ETaskStatus next, EStreamTaskEvent event, __state_trans_fn fn, __state_trans_succ_fn succFn, @@ -61,13 +65,7 @@ static int32_t attachEvent(SStreamTask* pTask, SAttachedEventInfo* pEvtInfo) { stDebug("s-task:%s status:%s attach event:%s required status:%s, since not allowed to handle it", pTask->id.idStr, p, StreamTaskEventList[pEvtInfo->event].name, StreamTaskStatusList[pEvtInfo->status].name); - - SStreamTaskSM* pSM = pTask->status.pSM; - if (pSM->eventList == NULL) { - - } - - taosArrayPush(pSM->eventList, pEvtInfo); + taosArrayPush(pTask->status.pSM->eventList, pEvtInfo); return 0; } @@ -84,8 +82,6 @@ int32_t streamTaskSetReadyForWal(SStreamTask* pTask) { if (pTask->info.taskLevel == TASK_LEVEL__SOURCE) { stDebug("s-task:%s ready for extract data from wal", pTask->id.idStr); } - - streamSetStatusNormal(pTask); // todo remove it return TSDB_CODE_SUCCESS; } @@ -109,9 +105,9 @@ int32_t streamTaskKeepCurrentVerInWal(SStreamTask* pTask) { // todo optimize the perf of find the trans objs by using hash table static STaskStateTrans* streamTaskFindTransform(const SStreamTaskSM* pState, const EStreamTaskEvent event) { - int32_t numOfTrans = taosArrayGetSize(pState->pTransList); + int32_t numOfTrans = taosArrayGetSize(streamTaskSMTrans); for (int32_t i = 0; i < numOfTrans; ++i) { - STaskStateTrans* pTrans = taosArrayGet(pState->pTransList, i); + STaskStateTrans* pTrans = taosArrayGet(streamTaskSMTrans, i); if (pTrans->state.state == pState->current.state && pTrans->event == event) { return pTrans; } @@ -138,6 +134,7 @@ void streamTaskRestoreStatus(SStreamTask* pTask) { } SStreamTaskSM* streamCreateStateMachine(SStreamTask* pTask) { + initStateTransferTable(); const char* id = pTask->id.idStr; SStreamTaskSM* pSM = taosMemoryCalloc(1, sizeof(SStreamTaskSM)); @@ -161,14 +158,7 @@ SStreamTaskSM* streamCreateStateMachine(SStreamTask* pTask) { // set the initial state for the state-machine of stream task pSM->current = StreamTaskStatusList[TASK_STATUS__UNINIT]; - pSM->startTs = taosGetTimestampMs(); - int32_t code = initStateTransferTable(pSM); - if (code != TSDB_CODE_SUCCESS) { - taosArrayDestroy(pSM->eventList); - taosMemoryFree(pSM); - return NULL; - } return pSM; } @@ -178,7 +168,6 @@ void* streamDestroyStateMachine(SStreamTaskSM* pSM) { } taosArrayDestroy(pSM->eventList); - taosArrayDestroy(pSM->pTransList); taosMemoryFree(pSM); return NULL; } @@ -276,7 +265,7 @@ int32_t streamTaskOnHandleEventSuccess(SStreamTaskSM* pSM) { return TSDB_CODE_SUCCESS; } -ETaskStatus streamTaskGetStatus(SStreamTask* pTask, char** pStr) { +ETaskStatus streamTaskGetStatus(const SStreamTask* pTask, char** pStr) { SStreamTaskState s = pTask->status.pSM->current; // copy one obj in case of multi-thread environment if (pStr != NULL) { *pStr = s.name; @@ -284,6 +273,28 @@ ETaskStatus streamTaskGetStatus(SStreamTask* pTask, char** pStr) { return s.state; } +void streamTaskResetStatus(SStreamTask* pTask) { + SStreamTaskSM* pSM = pTask->status.pSM; + pSM->current = StreamTaskStatusList[TASK_STATUS__UNINIT]; + pSM->pActiveTrans = NULL; + taosArrayClear(pSM->eventList); +} + +void streamTaskSetStatusReady(SStreamTask* pTask) { + SStreamTaskSM* pSM = pTask->status.pSM; + if (pSM->current.state == TASK_STATUS__DROPPING) { + stError("s-task:%s task in dropping state, cannot be set ready", pTask->id.idStr); + return; + } + + pSM->prev = pSM->current; + + pSM->current = StreamTaskStatusList[TASK_STATUS__READY]; + pSM->startTs = taosGetTimestampMs(); + pSM->pActiveTrans = NULL; + taosArrayClear(pSM->eventList); +} + STaskStateTrans createStateTransform(ETaskStatus current, ETaskStatus next, EStreamTaskEvent event, __state_trans_fn fn, __state_trans_succ_fn succFn, SAttachedEventInfo* pEventInfo, bool autoInvoke) { STaskStateTrans trans = {0}; @@ -301,92 +312,124 @@ STaskStateTrans createStateTransform(ETaskStatus current, ETaskStatus next, EStr return trans; } -int32_t initStateTransferTable(SStreamTaskSM* pSM) { - if (pSM->pTransList == NULL) { - pSM->pTransList = taosArrayInit(8, sizeof(STaskStateTrans)); - if (pSM->pTransList == NULL) { - return TSDB_CODE_OUT_OF_MEMORY; - } - } +int32_t initStateTransferTable() { + taosThreadOnce(&streamTaskStateMachineInit, doInitStateTransferTable); + return TSDB_CODE_SUCCESS; +} + +void doInitStateTransferTable(void) { + streamTaskSMTrans = taosArrayInit(8, sizeof(STaskStateTrans)); // initialization event handle - STaskStateTrans trans = createStateTransform(TASK_STATUS__UNINIT, TASK_STATUS__NORMAL, TASK_EVENT_INIT, + STaskStateTrans trans = createStateTransform(TASK_STATUS__UNINIT, TASK_STATUS__READY, TASK_EVENT_INIT, streamTaskInitStatus, onNormalTaskReady, false, false); - taosArrayPush(pSM->pTransList, &trans); + taosArrayPush(streamTaskSMTrans, &trans); trans = createStateTransform(TASK_STATUS__UNINIT, TASK_STATUS__SCAN_HISTORY, TASK_EVENT_INIT_SCANHIST, streamTaskInitStatus, onScanhistoryTaskReady, false, false); - taosArrayPush(pSM->pTransList, &trans); + taosArrayPush(streamTaskSMTrans, &trans); trans = createStateTransform(TASK_STATUS__UNINIT, TASK_STATUS_STREAM_SCAN_HISTORY, TASK_EVENT_INIT_STREAM_SCANHIST, streamTaskInitStatus, onScanhistoryTaskReady, false, false); - taosArrayPush(pSM->pTransList, &trans); + taosArrayPush(streamTaskSMTrans, &trans); - trans = createStateTransform(TASK_STATUS__SCAN_HISTORY, TASK_STATUS__NORMAL, TASK_EVENT_SCANHIST_DONE, + trans = createStateTransform(TASK_STATUS__SCAN_HISTORY, TASK_STATUS__READY, TASK_EVENT_SCANHIST_DONE, streamTaskSetReadyForWal, NULL, NULL, true); - taosArrayPush(pSM->pTransList, &trans); + taosArrayPush(streamTaskSMTrans, &trans); - trans = createStateTransform(TASK_STATUS_STREAM_SCAN_HISTORY, TASK_STATUS__NORMAL, TASK_EVENT_SCANHIST_DONE, + trans = createStateTransform(TASK_STATUS_STREAM_SCAN_HISTORY, TASK_STATUS__READY, TASK_EVENT_SCANHIST_DONE, streamTaskSetReadyForWal, NULL, NULL, true); - taosArrayPush(pSM->pTransList, &trans); + taosArrayPush(streamTaskSMTrans, &trans); // halt stream task, from other task status - trans = createStateTransform(TASK_STATUS__NORMAL, TASK_STATUS__HALT, TASK_EVENT_HALT, NULL, + trans = createStateTransform(TASK_STATUS__READY, TASK_STATUS__HALT, TASK_EVENT_HALT, NULL, streamTaskKeepCurrentVerInWal, NULL, true); - taosArrayPush(pSM->pTransList, &trans); + taosArrayPush(streamTaskSMTrans, &trans); - SAttachedEventInfo info = {.status = TASK_STATUS__NORMAL, .event = TASK_EVENT_HALT}; + SAttachedEventInfo info = {.status = TASK_STATUS__READY, .event = TASK_EVENT_HALT}; trans = createStateTransform(TASK_STATUS_STREAM_SCAN_HISTORY, TASK_STATUS__HALT, TASK_EVENT_HALT, NULL, streamTaskKeepCurrentVerInWal, &info, true); - taosArrayPush(pSM->pTransList, &trans); + taosArrayPush(streamTaskSMTrans, &trans); trans = createStateTransform(TASK_STATUS__CK, TASK_STATUS__HALT, TASK_EVENT_HALT, NULL, streamTaskKeepCurrentVerInWal, &info, true); - taosArrayPush(pSM->pTransList, &trans); + taosArrayPush(streamTaskSMTrans, &trans); trans = createStateTransform(TASK_STATUS__PAUSE, TASK_STATUS__HALT, TASK_EVENT_HALT, NULL, streamTaskKeepCurrentVerInWal, NULL, true); - taosArrayPush(pSM->pTransList, &trans); + taosArrayPush(streamTaskSMTrans, &trans); // checkpoint related event - trans = createStateTransform(TASK_STATUS__NORMAL, TASK_STATUS__CK, TASK_EVENT_GEN_CHECKPOINT, NULL, + trans = createStateTransform(TASK_STATUS__READY, TASK_STATUS__CK, TASK_EVENT_GEN_CHECKPOINT, NULL, streamTaskDoCheckpoint, NULL, true); - taosArrayPush(pSM->pTransList, &trans); + taosArrayPush(streamTaskSMTrans, &trans); trans = - createStateTransform(TASK_STATUS__CK, TASK_STATUS__NORMAL, TASK_EVENT_CHECKPOINT_DONE, NULL, NULL, NULL, true); - taosArrayPush(pSM->pTransList, &trans); + createStateTransform(TASK_STATUS__CK, TASK_STATUS__READY, TASK_EVENT_CHECKPOINT_DONE, NULL, NULL, NULL, true); + taosArrayPush(streamTaskSMTrans, &trans); // pause & resume related event handle - trans = createStateTransform(TASK_STATUS__NORMAL, TASK_STATUS__PAUSE, TASK_EVENT_PAUSE, NULL, NULL, NULL, true); - taosArrayPush(pSM->pTransList, &trans); - + trans = createStateTransform(TASK_STATUS__READY, TASK_STATUS__PAUSE, TASK_EVENT_PAUSE, NULL, NULL, NULL, true); + taosArrayPush(streamTaskSMTrans, &trans); trans = createStateTransform(TASK_STATUS__SCAN_HISTORY, TASK_STATUS__PAUSE, TASK_EVENT_PAUSE, NULL, NULL, NULL, true); - taosArrayPush(pSM->pTransList, &trans); + taosArrayPush(streamTaskSMTrans, &trans); - info = (SAttachedEventInfo){.status = TASK_STATUS__NORMAL, .event = TASK_EVENT_PAUSE}; + info = (SAttachedEventInfo){.status = TASK_STATUS__READY, .event = TASK_EVENT_PAUSE}; trans = createStateTransform(TASK_STATUS_STREAM_SCAN_HISTORY, TASK_STATUS__PAUSE, TASK_EVENT_PAUSE, NULL, NULL, &info, true); - taosArrayPush(pSM->pTransList, &trans); - + taosArrayPush(streamTaskSMTrans, &trans); trans = createStateTransform(TASK_STATUS__CK, TASK_STATUS__PAUSE, TASK_EVENT_PAUSE, NULL, NULL, &info, true); - taosArrayPush(pSM->pTransList, &trans); - + taosArrayPush(streamTaskSMTrans, &trans); trans = createStateTransform(TASK_STATUS__HALT, TASK_STATUS__PAUSE, TASK_EVENT_PAUSE, NULL, NULL, &info, true); - taosArrayPush(pSM->pTransList, &trans); - + taosArrayPush(streamTaskSMTrans, &trans); trans = createStateTransform(TASK_STATUS__UNINIT, TASK_STATUS__PAUSE, TASK_EVENT_PAUSE, NULL, NULL, &info, true); - taosArrayPush(pSM->pTransList, &trans); + taosArrayPush(streamTaskSMTrans, &trans); trans = createStateTransform(TASK_STATUS__PAUSE, TASK_STATUS__PAUSE, TASK_EVENT_PAUSE, NULL, NULL, NULL, true); - taosArrayPush(pSM->pTransList, &trans); - + taosArrayPush(streamTaskSMTrans, &trans); trans = createStateTransform(TASK_STATUS__STOP, TASK_STATUS__STOP, TASK_EVENT_PAUSE, NULL, NULL, NULL, true); - taosArrayPush(pSM->pTransList, &trans); - + taosArrayPush(streamTaskSMTrans, &trans); trans = createStateTransform(TASK_STATUS__DROPPING, TASK_STATUS__DROPPING, TASK_EVENT_PAUSE, NULL, NULL, NULL, true); - taosArrayPush(pSM->pTransList, &trans); + taosArrayPush(streamTaskSMTrans, &trans); // resume is completed by restore status of state-machine - return 0; + // stop related event + trans = createStateTransform(TASK_STATUS__READY, TASK_STATUS__STOP, TASK_EVENT_STOP, NULL, NULL, NULL, true); + taosArrayPush(streamTaskSMTrans, &trans); + trans = createStateTransform(TASK_STATUS__DROPPING, TASK_STATUS__STOP, TASK_EVENT_STOP, NULL, NULL, NULL, true); + taosArrayPush(streamTaskSMTrans, &trans); + trans = createStateTransform(TASK_STATUS__UNINIT, TASK_STATUS__STOP, TASK_EVENT_STOP, NULL, NULL, NULL, true); + taosArrayPush(streamTaskSMTrans, &trans); + trans = createStateTransform(TASK_STATUS__STOP, TASK_STATUS__STOP, TASK_EVENT_STOP, NULL, NULL, NULL, true); + taosArrayPush(streamTaskSMTrans, &trans); + trans = createStateTransform(TASK_STATUS__SCAN_HISTORY, TASK_STATUS__STOP, TASK_EVENT_STOP, NULL, NULL, NULL, true); + taosArrayPush(streamTaskSMTrans, &trans); + trans = createStateTransform(TASK_STATUS__HALT, TASK_STATUS__STOP, TASK_EVENT_STOP, NULL, NULL, NULL, true); + taosArrayPush(streamTaskSMTrans, &trans); + trans = createStateTransform(TASK_STATUS__PAUSE, TASK_STATUS__STOP, TASK_EVENT_STOP, NULL, NULL, NULL, true); + taosArrayPush(streamTaskSMTrans, &trans); + trans = createStateTransform(TASK_STATUS__CK, TASK_STATUS__STOP, TASK_EVENT_STOP, NULL, NULL, NULL, true); + taosArrayPush(streamTaskSMTrans, &trans); + trans = createStateTransform(TASK_STATUS_STREAM_SCAN_HISTORY, TASK_STATUS__STOP, TASK_EVENT_STOP, NULL, NULL, NULL, true); + taosArrayPush(streamTaskSMTrans, &trans); + + // dropping related event + trans = createStateTransform(TASK_STATUS__READY, TASK_STATUS__DROPPING, TASK_EVENT_DROPPING, NULL, NULL, NULL, true); + taosArrayPush(streamTaskSMTrans, &trans); + trans = createStateTransform(TASK_STATUS__DROPPING, TASK_STATUS__DROPPING, TASK_EVENT_DROPPING, NULL, NULL, NULL, true); + taosArrayPush(streamTaskSMTrans, &trans); + trans = createStateTransform(TASK_STATUS__UNINIT, TASK_STATUS__DROPPING, TASK_EVENT_DROPPING, NULL, NULL, NULL, true); + taosArrayPush(streamTaskSMTrans, &trans); + trans = createStateTransform(TASK_STATUS__STOP, TASK_STATUS__DROPPING, TASK_EVENT_DROPPING, NULL, NULL, NULL, true); + taosArrayPush(streamTaskSMTrans, &trans); + trans = createStateTransform(TASK_STATUS__SCAN_HISTORY, TASK_STATUS__DROPPING, TASK_EVENT_DROPPING, NULL, NULL, NULL, true); + taosArrayPush(streamTaskSMTrans, &trans); + trans = createStateTransform(TASK_STATUS__HALT, TASK_STATUS__DROPPING, TASK_EVENT_DROPPING, NULL, NULL, NULL, true); + taosArrayPush(streamTaskSMTrans, &trans); + trans = createStateTransform(TASK_STATUS__PAUSE, TASK_STATUS__DROPPING, TASK_EVENT_DROPPING, NULL, NULL, NULL, true); + taosArrayPush(streamTaskSMTrans, &trans); + trans = createStateTransform(TASK_STATUS__CK, TASK_STATUS__DROPPING, TASK_EVENT_DROPPING, NULL, NULL, NULL, true); + taosArrayPush(streamTaskSMTrans, &trans); + trans = createStateTransform(TASK_STATUS_STREAM_SCAN_HISTORY, TASK_STATUS__DROPPING, TASK_EVENT_DROPPING, NULL, NULL, NULL, true); + taosArrayPush(streamTaskSMTrans, &trans); } \ No newline at end of file From c1cebae6ba198b376b77dd14a9f4184ab843626d Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 19 Oct 2023 16:20:27 +0800 Subject: [PATCH 06/41] fix(stream): fix error in stream. --- include/libs/stream/tstream.h | 5 +-- source/dnode/mnode/impl/src/mndStream.c | 6 +-- source/dnode/vnode/src/tq/tq.c | 6 +-- source/libs/stream/src/streamExec.c | 4 +- source/libs/stream/src/streamMeta.c | 1 + source/libs/stream/src/streamStart.c | 31 ++++++++------- source/libs/stream/src/streamTask.c | 14 ------- source/libs/stream/src/streamTaskSm.c | 53 +++++++++++++++++-------- 8 files changed, 64 insertions(+), 56 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 0a541a34ab..6564bcc769 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -75,7 +75,7 @@ typedef enum ETaskStatus { TASK_STATUS__HALT, // pause, but not be manipulated by user command TASK_STATUS__PAUSE, // pause TASK_STATUS__CK, // stream task is in checkpoint status, no data are allowed to put into inputQ anymore - TASK_STATUS_STREAM_SCAN_HISTORY, + TASK_STATUS__STREAM_SCAN_HISTORY, } ETaskStatus; enum { @@ -720,6 +720,7 @@ bool streamTaskIsIdle(const SStreamTask* pTask); char* createStreamTaskIdStr(int64_t streamId, int32_t taskId); ETaskStatus streamTaskGetStatus(const SStreamTask* pTask, char** pStr); +const char* streamTaskGetStatusStr(ETaskStatus status); void streamTaskResetStatus(SStreamTask* pTask); void streamTaskSetStatusReady(SStreamTask* pTask); @@ -755,8 +756,6 @@ int32_t streamQueueGetNumOfItems(const SStreamQueue* pQueue); // common int32_t streamRestoreParam(SStreamTask* pTask); -int32_t streamSetStatusUnint(SStreamTask* pTask); -const char* streamGetTaskStatusStr(int32_t status); void streamTaskPause(SStreamTask* pTask, SStreamMeta* pMeta); void streamTaskResume(SStreamTask* pTask, SStreamMeta* pMeta); void streamTaskResumeFromHalt(SStreamTask* pTask); diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index 0a9e3c5336..7e254e5efb 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -1183,7 +1183,7 @@ static int32_t mndProcessStreamDoCheckpoint(SRpcMsg *pReq) { if (pEntry->status != TASK_STATUS__READY) { mDebug("s-task:0x%" PRIx64 "-0x%x (nodeId:%d) status:%s not ready, checkpoint msg not issued", - pEntry->id.streamId, (int32_t)pEntry->id.taskId, 0, streamGetTaskStatusStr(pEntry->status)); + pEntry->id.streamId, (int32_t)pEntry->id.taskId, 0, streamTaskGetStatusStr(pEntry->status)); ready = false; break; } @@ -1567,7 +1567,7 @@ static int32_t mndRetrieveStreamTask(SRpcMsg *pReq, SShowObj *pShow, SSDataBlock continue; } - const char* pStatus = streamGetTaskStatusStr(pe->status); + const char* pStatus = streamTaskGetStatusStr(pe->status); STR_TO_VARSTR(status, pStatus); // status @@ -2615,7 +2615,7 @@ int32_t mndProcessStreamHb(SRpcMsg *pReq) { pEntry->status = p->status; if (p->status != TASK_STATUS__READY) { - mDebug("received s-task:0x%"PRIx64" not in ready status:%s", p->id.taskId, streamGetTaskStatusStr(p->status)); + mDebug("received s-task:0x%"PRIx64" not in ready status:%s", p->id.taskId, streamTaskGetStatusStr(p->status)); } } diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 750a9d942a..8f7657f98c 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1446,7 +1446,7 @@ int32_t tqProcessTaskDropReq(STQ* pTq, char* msg, int32_t msgLen) { SStreamTask* pTask = streamMetaAcquireTask(pMeta, pReq->streamId, pReq->taskId); if (pTask != NULL) { // drop the related fill-history task firstly - if (pTask->hTaskInfo.id.taskId != 0) { + if (HAS_RELATED_FILLHISTORY_TASK(pTask)) { STaskId* pHTaskId = &pTask->hTaskInfo.id; streamMetaUnregisterTask(pMeta, pHTaskId->streamId, pHTaskId->taskId); tqDebug("vgId:%d drop fill-history task:0x%x dropped firstly", vgId, (int32_t)pHTaskId->taskId); @@ -1486,7 +1486,7 @@ int32_t tqProcessTaskPauseReq(STQ* pTq, int64_t sversion, char* msg, int32_t msg streamTaskPause(pTask, pMeta); SStreamTask* pHistoryTask = NULL; - if (pTask->hTaskInfo.id.taskId != 0) { + if (HAS_RELATED_FILLHISTORY_TASK(pTask)) { pHistoryTask = streamMetaAcquireTask(pMeta, pTask->hTaskInfo.id.streamId, pTask->hTaskInfo.id.taskId); if (pHistoryTask == NULL) { tqError("vgId:%d process pause req, failed to acquire fill-history task:0x%" PRIx64 @@ -1868,7 +1868,7 @@ int32_t tqProcessTaskUpdateReq(STQ* pTq, SRpcMsg* pMsg) { streamTaskResetStatus(pTask); SStreamTask** ppHTask = NULL; - if (pTask->hTaskInfo.id.taskId != 0) { + if (HAS_RELATED_FILLHISTORY_TASK(pTask)) { ppHTask = (SStreamTask**)taosHashGet(pMeta->pTasksMap, &pTask->hTaskInfo.id, sizeof(pTask->hTaskInfo.id)); if (ppHTask == NULL || *ppHTask == NULL) { tqError("vgId:%d failed to acquire fill-history task:0x%x when handling update, it may have been dropped already", diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index 346f6cefcb..8c37e785dd 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -308,7 +308,7 @@ int32_t streamDoTransferStateToStreamTask(SStreamTask* pTask) { pStreamTask->id.idStr); } - int8_t status = streamTaskGetStatus(pStreamTask, NULL); + ETaskStatus status = streamTaskGetStatus(pStreamTask, NULL); ASSERT(((status == TASK_STATUS__DROPPING) || (pStreamTask->hTaskInfo.id.taskId == pTask->id.taskId)) && pTask->status.appendTranstateBlock == true); @@ -352,7 +352,7 @@ int32_t streamDoTransferStateToStreamTask(SStreamTask* pTask) { // 3. resume the state of stream task, after this function, the stream task will run immidately. But it can not be // pause, since the pause allowed attribute is not set yet. - streamTaskResumeFromHalt(pStreamTask); + streamTaskResumeFromHalt(pStreamTask); // todo refactor: use streamTaskResume. stDebug("s-task:%s fill-history task set status to be dropping, save the state into disk", pTask->id.idStr); diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index ee4f1e2340..206f1fcfc6 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -348,6 +348,7 @@ void streamMetaCloseImpl(void* arg) { stDebug("end to close stream meta"); } +// todo let's check the status for each task int32_t streamMetaSaveTask(SStreamMeta* pMeta, SStreamTask* pTask) { void* buf = NULL; int32_t len; diff --git a/source/libs/stream/src/streamStart.c b/source/libs/stream/src/streamStart.c index 58ea042079..66865c8e25 100644 --- a/source/libs/stream/src/streamStart.c +++ b/source/libs/stream/src/streamStart.c @@ -42,7 +42,8 @@ int32_t streamTaskSetReady(SStreamTask* pTask) { int32_t numOfDowns = streamTaskGetNumOfDownstream(pTask); ETaskStatus status = streamTaskGetStatus(pTask, &p); - if (status == TASK_STATUS__SCAN_HISTORY && pTask->info.taskLevel != TASK_LEVEL__SOURCE) { + if ((status == TASK_STATUS__SCAN_HISTORY || status == TASK_STATUS__STREAM_SCAN_HISTORY) && + pTask->info.taskLevel != TASK_LEVEL__SOURCE) { pTask->numOfWaitingUpstream = taosArrayGetSize(pTask->upstreamInfo.pList); stDebug("s-task:%s level:%d task wait for %d upstream tasks complete scan-history procedure, status:%s", pTask->id.idStr, pTask->info.taskLevel, pTask->numOfWaitingUpstream, p); @@ -92,16 +93,20 @@ static int32_t doStartScanHistoryTask(SStreamTask* pTask) { } int32_t streamTaskStartScanHistory(SStreamTask* pTask) { - ASSERT(pTask->status.downstreamReady == 1 && streamTaskGetStatus(pTask, NULL) == TASK_STATUS__SCAN_HISTORY); + int32_t level = pTask->info.taskLevel; + ETaskStatus status = streamTaskGetStatus(pTask, NULL); - if (pTask->info.taskLevel == TASK_LEVEL__SOURCE) { + ASSERT(pTask->status.downstreamReady == 1 && + ((status == TASK_STATUS__SCAN_HISTORY) || (status == TASK_STATUS__STREAM_SCAN_HISTORY))); + + if (level == TASK_LEVEL__SOURCE) { return doStartScanHistoryTask(pTask); - } else if (pTask->info.taskLevel == TASK_LEVEL__AGG) { + } else if (level == TASK_LEVEL__AGG) { if (pTask->info.fillHistory) { streamSetParamForScanHistory(pTask); streamTaskEnablePause(pTask); } - } else if (pTask->info.taskLevel == TASK_LEVEL__SINK) { + } else if (level == TASK_LEVEL__SINK) { stDebug("s-task:%s sink task do nothing to handle scan-history", pTask->id.idStr); } @@ -292,13 +297,13 @@ int32_t onScanhistoryTaskReady(SStreamTask* pTask) { char* p = NULL; ETaskStatus status = streamTaskGetStatus(pTask, &p); - ASSERT(status == TASK_STATUS__SCAN_HISTORY); + ASSERT(status == TASK_STATUS__SCAN_HISTORY || status == TASK_STATUS__STREAM_SCAN_HISTORY); stDebug("s-task:%s enter into scan-history data stage, status:%s", id, p); streamTaskStartScanHistory(pTask); // start the related fill-history task, when current task is ready - if (pTask->hTaskInfo.id.taskId != 0) { + if (HAS_RELATED_FILLHISTORY_TASK(pTask)) { streamLaunchFillHistoryTask(pTask); } @@ -516,7 +521,7 @@ int32_t streamProcessScanHistoryFinishReq(SStreamTask* pTask, SStreamScanHistory char* p = NULL; ETaskStatus status = streamTaskGetStatus(pTask, &p); - if (status != TASK_STATUS__SCAN_HISTORY) { + if (status != TASK_STATUS__SCAN_HISTORY && status != TASK_STATUS__STREAM_SCAN_HISTORY) { stError("s-task:%s not in scan-history status, status:%s return upstream:0x%x scan-history finish directly", id, p, pReq->upstreamTaskId); @@ -571,7 +576,9 @@ int32_t streamProcessScanHistoryFinishReq(SStreamTask* pTask, SStreamScanHistory } int32_t streamProcessScanHistoryFinishRsp(SStreamTask* pTask) { - ASSERT(/*pTask->status.taskStatus*/ streamTaskGetStatus(pTask, NULL) == TASK_STATUS__SCAN_HISTORY); + ETaskStatus status = streamTaskGetStatus(pTask, NULL); + + ASSERT(status == TASK_STATUS__SCAN_HISTORY || status == TASK_STATUS__STREAM_SCAN_HISTORY); SStreamMeta* pMeta = pTask->pMeta; // execute in the scan history complete call back msg, ready to process data from inputQ @@ -939,9 +946,7 @@ void streamTaskCheckDownstream(SStreamTask* pTask) { doCheckDownstreamStatus(pTask); } -// normal -> pause, pause/stop/dropping -> pause, halt -> pause, scan-history -> pause void streamTaskPause(SStreamTask* pTask, SStreamMeta* pMeta) { - int64_t st = taosGetTimestampMs(); #if 0 int8_t status = pTask->status.taskStatus; if (status == TASK_STATUS__DROPPING) { @@ -1013,9 +1018,7 @@ void streamTaskPause(SStreamTask* pTask, SStreamMeta* pMeta) { qKillTask(pExecutor, TSDB_CODE_SUCCESS); } - int64_t el = taosGetTimestampMs() - st; - stDebug("vgId:%d s-task:%s set pause flag, prev:%s, pause elapsed time:%dms", pMeta->vgId, pTask->id.idStr, - streamGetTaskStatusStr(pTask->status.keepTaskStatus), (int32_t)el); + stDebug("vgId:%d s-task:%s set pause flag and pause task", pMeta->vgId, pTask->id.idStr); } void streamTaskResume(SStreamTask* pTask, SStreamMeta* pMeta) { diff --git a/source/libs/stream/src/streamTask.c b/source/libs/stream/src/streamTask.c index fdcfcfa9a9..1d12401d12 100644 --- a/source/libs/stream/src/streamTask.c +++ b/source/libs/stream/src/streamTask.c @@ -732,20 +732,6 @@ void streamTaskSetRetryInfoForLaunch(SHistoryTaskInfo* pInfo) { pInfo->retryTimes += 1; } -const char* streamGetTaskStatusStr(int32_t status) { - switch(status) { - case TASK_STATUS__READY: return "normal"; - case TASK_STATUS__SCAN_HISTORY: return "scan-history"; - case TASK_STATUS__HALT: return "halt"; - case TASK_STATUS__PAUSE: return "paused"; - case TASK_STATUS__CK: return "check-point"; - case TASK_STATUS__DROPPING: return "dropping"; - case TASK_STATUS__STOP: return "stop"; - case TASK_STATUS__UNINIT: return "uninitialized"; - default:return ""; - } -} - void streamTaskStatusInit(STaskStatusEntry* pEntry, const SStreamTask* pTask) { pEntry->id.streamId = pTask->id.streamId; pEntry->id.taskId = pTask->id.taskId; diff --git a/source/libs/stream/src/streamTaskSm.c b/source/libs/stream/src/streamTaskSm.c index 49a434af94..addd563388 100644 --- a/source/libs/stream/src/streamTaskSm.c +++ b/source/libs/stream/src/streamTaskSm.c @@ -29,10 +29,10 @@ SStreamTaskState StreamTaskStatusList[9] = { {.state = TASK_STATUS__HALT, .name = "halt"}, {.state = TASK_STATUS__PAUSE, .name = "paused"}, {.state = TASK_STATUS__CK, .name = "checkpoint"}, - {.state = TASK_STATUS_STREAM_SCAN_HISTORY, .name = "stream-scan-history"}, + {.state = TASK_STATUS__STREAM_SCAN_HISTORY, .name = "stream-scan-history"}, }; -SStreamEventInfo StreamTaskEventList[10] = { +SStreamEventInfo StreamTaskEventList[11] = { {}, // dummy event, place holder {.event = TASK_EVENT_INIT, .name = "initialize"}, {.event = TASK_EVENT_INIT_SCANHIST, .name = "scan-history-initialize"}, @@ -43,6 +43,7 @@ SStreamEventInfo StreamTaskEventList[10] = { {.event = TASK_EVENT_PAUSE, .name = "pausing"}, {.event = TASK_EVENT_RESUME, .name = "resuming"}, {.event = TASK_EVENT_HALT, .name = "halting"}, + {.event = TASK_EVENT_DROPPING, .name = "dropping"}, }; static TdThreadOnce streamTaskStateMachineInit = PTHREAD_ONCE_INIT; @@ -173,35 +174,45 @@ void* streamDestroyStateMachine(SStreamTaskSM* pSM) { } int32_t streamTaskHandleEvent(SStreamTaskSM* pSM, EStreamTaskEvent event) { - taosThreadMutexLock(&pSM->pTask->lock); + SStreamTask* pTask = pSM->pTask; + + taosThreadMutexLock(&pTask->lock); STaskStateTrans* pTrans = streamTaskFindTransform(pSM, event); - stDebug("s-task:%s start to handle event:%s, state:%s", pSM->pTask->id.idStr, StreamTaskEventList[event].name, + stDebug("s-task:%s start to handle event:%s, state:%s", pTask->id.idStr, StreamTaskEventList[event].name, pSM->current.name); if (pTrans->attachEvent.event != 0) { - attachEvent(pSM->pTask, &pTrans->attachEvent); - taosThreadMutexUnlock(&pSM->pTask->lock); + attachEvent(pTask, &pTrans->attachEvent); + taosThreadMutexUnlock(&pTask->lock); while (1) { // wait for the task to be here - ETaskStatus s = streamTaskGetStatus(pSM->pTask, NULL); + taosThreadMutexLock(&pTask->lock); + ETaskStatus s = streamTaskGetStatus(pTask, NULL); + taosThreadMutexUnlock(&pTask->lock); + if (s == pTrans->attachEvent.status) { return TSDB_CODE_SUCCESS; } else {// this event has been handled already - stDebug("s-task:%s not handle event:%s yet, wait for 100ms and recheck", pSM->pTask->id.idStr, + stDebug("s-task:%s not handle event:%s yet, wait for 100ms and recheck", pTask->id.idStr, StreamTaskEventList[event].name); taosMsleep(100); } } } else { - ASSERT(pSM->pActiveTrans == NULL); + if (pSM->pActiveTrans != NULL) { + ASSERT(!pSM->pActiveTrans->autoInvokeEndFn); + stWarn("s-task:%s status:%s handle event:%s is interrupted, handle the new event:%s", pTask->id.idStr, + pSM->current.name, StreamTaskEventList[pSM->pActiveTrans->event].name, StreamTaskEventList[event].name); + } + pSM->pActiveTrans = pTrans; pSM->startTs = taosGetTimestampMs(); - taosThreadMutexUnlock(&pSM->pTask->lock); + taosThreadMutexUnlock(&pTask->lock); - int32_t code = pTrans->pAction(pSM->pTask); + int32_t code = pTrans->pAction(pTask); // todo handle error code; if (pTrans->autoInvokeEndFn) { @@ -273,6 +284,10 @@ ETaskStatus streamTaskGetStatus(const SStreamTask* pTask, char** pStr) { return s.state; } +const char* streamTaskGetStatusStr(ETaskStatus status) { + return StreamTaskStatusList[status].name; +} + void streamTaskResetStatus(SStreamTask* pTask) { SStreamTaskSM* pSM = pTask->status.pSM; pSM->current = StreamTaskStatusList[TASK_STATUS__UNINIT]; @@ -304,6 +319,9 @@ STaskStateTrans createStateTransform(ETaskStatus current, ETaskStatus next, EStr if (pEventInfo != NULL) { trans.attachEvent = *pEventInfo; + } else { + trans.attachEvent.event = 0; + trans.attachEvent.status = 0; } trans.pAction = (fn != NULL) ? fn : dummyFn; @@ -329,15 +347,16 @@ void doInitStateTransferTable(void) { streamTaskInitStatus, onScanhistoryTaskReady, false, false); taosArrayPush(streamTaskSMTrans, &trans); - trans = createStateTransform(TASK_STATUS__UNINIT, TASK_STATUS_STREAM_SCAN_HISTORY, TASK_EVENT_INIT_STREAM_SCANHIST, + trans = createStateTransform(TASK_STATUS__UNINIT, TASK_STATUS__STREAM_SCAN_HISTORY, TASK_EVENT_INIT_STREAM_SCANHIST, streamTaskInitStatus, onScanhistoryTaskReady, false, false); taosArrayPush(streamTaskSMTrans, &trans); + // scan-history related event trans = createStateTransform(TASK_STATUS__SCAN_HISTORY, TASK_STATUS__READY, TASK_EVENT_SCANHIST_DONE, streamTaskSetReadyForWal, NULL, NULL, true); taosArrayPush(streamTaskSMTrans, &trans); - trans = createStateTransform(TASK_STATUS_STREAM_SCAN_HISTORY, TASK_STATUS__READY, TASK_EVENT_SCANHIST_DONE, + trans = createStateTransform(TASK_STATUS__STREAM_SCAN_HISTORY, TASK_STATUS__READY, TASK_EVENT_SCANHIST_DONE, streamTaskSetReadyForWal, NULL, NULL, true); taosArrayPush(streamTaskSMTrans, &trans); @@ -347,7 +366,7 @@ void doInitStateTransferTable(void) { taosArrayPush(streamTaskSMTrans, &trans); SAttachedEventInfo info = {.status = TASK_STATUS__READY, .event = TASK_EVENT_HALT}; - trans = createStateTransform(TASK_STATUS_STREAM_SCAN_HISTORY, TASK_STATUS__HALT, TASK_EVENT_HALT, NULL, + trans = createStateTransform(TASK_STATUS__STREAM_SCAN_HISTORY, TASK_STATUS__HALT, TASK_EVENT_HALT, NULL, streamTaskKeepCurrentVerInWal, &info, true); taosArrayPush(streamTaskSMTrans, &trans); @@ -375,7 +394,7 @@ void doInitStateTransferTable(void) { taosArrayPush(streamTaskSMTrans, &trans); info = (SAttachedEventInfo){.status = TASK_STATUS__READY, .event = TASK_EVENT_PAUSE}; - trans = createStateTransform(TASK_STATUS_STREAM_SCAN_HISTORY, TASK_STATUS__PAUSE, TASK_EVENT_PAUSE, NULL, NULL, &info, true); + trans = createStateTransform(TASK_STATUS__STREAM_SCAN_HISTORY, TASK_STATUS__PAUSE, TASK_EVENT_PAUSE, NULL, NULL, &info, true); taosArrayPush(streamTaskSMTrans, &trans); trans = createStateTransform(TASK_STATUS__CK, TASK_STATUS__PAUSE, TASK_EVENT_PAUSE, NULL, NULL, &info, true); taosArrayPush(streamTaskSMTrans, &trans); @@ -410,7 +429,7 @@ void doInitStateTransferTable(void) { taosArrayPush(streamTaskSMTrans, &trans); trans = createStateTransform(TASK_STATUS__CK, TASK_STATUS__STOP, TASK_EVENT_STOP, NULL, NULL, NULL, true); taosArrayPush(streamTaskSMTrans, &trans); - trans = createStateTransform(TASK_STATUS_STREAM_SCAN_HISTORY, TASK_STATUS__STOP, TASK_EVENT_STOP, NULL, NULL, NULL, true); + trans = createStateTransform(TASK_STATUS__STREAM_SCAN_HISTORY, TASK_STATUS__STOP, TASK_EVENT_STOP, NULL, NULL, NULL, true); taosArrayPush(streamTaskSMTrans, &trans); // dropping related event @@ -430,6 +449,6 @@ void doInitStateTransferTable(void) { taosArrayPush(streamTaskSMTrans, &trans); trans = createStateTransform(TASK_STATUS__CK, TASK_STATUS__DROPPING, TASK_EVENT_DROPPING, NULL, NULL, NULL, true); taosArrayPush(streamTaskSMTrans, &trans); - trans = createStateTransform(TASK_STATUS_STREAM_SCAN_HISTORY, TASK_STATUS__DROPPING, TASK_EVENT_DROPPING, NULL, NULL, NULL, true); + trans = createStateTransform(TASK_STATUS__STREAM_SCAN_HISTORY, TASK_STATUS__DROPPING, TASK_EVENT_DROPPING, NULL, NULL, NULL, true); taosArrayPush(streamTaskSMTrans, &trans); } \ No newline at end of file From d5059826a84eafcef74c5c984277172a9d6a36b2 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 19 Oct 2023 17:59:04 +0800 Subject: [PATCH 07/41] fix(stream): fix bugs caused by refactor. --- source/dnode/mnode/impl/src/mndScheduler.c | 8 ++- source/dnode/vnode/src/tq/tq.c | 7 +-- source/libs/stream/inc/streamsm.h | 12 +++-- source/libs/stream/src/streamTaskSm.c | 50 +++++++++++-------- source/util/src/tarray.c | 2 +- tests/system-test/8-stream/scalar_function.py | 2 +- 6 files changed, 50 insertions(+), 31 deletions(-) diff --git a/source/dnode/mnode/impl/src/mndScheduler.c b/source/dnode/mnode/impl/src/mndScheduler.c index 2931f6be6b..404198a523 100644 --- a/source/dnode/mnode/impl/src/mndScheduler.c +++ b/source/dnode/mnode/impl/src/mndScheduler.c @@ -569,6 +569,10 @@ static int32_t addSinkTasks(SArray* pTasksList, SMnode* pMnode, SStreamObj* pStr } static void setSinkTaskUpstreamInfo(SArray* pTasksList, const SStreamTask* pUpstreamTask) { + if (taosArrayGetSize(pTasksList) < SINK_NODE_LEVEL || pUpstreamTask == NULL) { + return; + } + SArray* pSinkTaskList = taosArrayGetP(pTasksList, SINK_NODE_LEVEL); for(int32_t i = 0; i < taosArrayGetSize(pSinkTaskList); ++i) { SStreamTask* pSinkTask = taosArrayGetP(pSinkTaskList, i); @@ -628,7 +632,9 @@ static int32_t doScheduleStream(SStreamObj* pStream, SMnode* pMnode, SQueryPlan* } setSinkTaskUpstreamInfo(pStream->tasks, pAggTask); - setSinkTaskUpstreamInfo(pStream->pHTasksList, pHAggTask); + if (pHAggTask != NULL) { + setSinkTaskUpstreamInfo(pStream->pHTasksList, pHAggTask); + } // source level return addSourceTasksForMultiLevelStream(pMnode, pPlan, pStream, pAggTask, pHAggTask, pEpset, nextWindowSkey); diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 8f7657f98c..c4e74f84e5 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1225,7 +1225,7 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { if (done) { qDebug("s-task:%s scan-history from WAL stage(step 2) ended, elapsed time:%.2fs", id, 0.0); streamTaskPutTranstateIntoInputQ(pTask); - streamTaskRestoreStatus(pTask); +// streamTaskRestoreStatus(pTask); // if (pTask->status.taskStatus == TASK_STATUS__PAUSE) { // pTask->status.keepTaskStatus = TASK_STATUS__READY; @@ -1259,6 +1259,7 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { } #endif + // now the fill-history task starts to scan data from wal files. streamTaskHandleEvent(pTask->status.pSM, TASK_EVENT_SCANHIST_DONE); tqScanWalAsync(pTq, false); } @@ -1271,8 +1272,8 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { // Not update the fill-history time window until the state transfer is completed if the related fill-history task // exists. tqDebug( - "s-task:%s scan-history in stream time window completed, now start to handle data from WAL, start " - "ver:%" PRId64 ", window:%" PRId64 " - %" PRId64, + "s-task:%s scan-history in stream time window completed, now start to handle data from WAL, startVer:%" PRId64 + ", window:%" PRId64 " - %" PRId64, id, pTask->chkInfo.nextProcessVer, pWindow->skey, pWindow->ekey); code = streamTaskScanHistoryDataComplete(pTask); diff --git a/source/libs/stream/inc/streamsm.h b/source/libs/stream/inc/streamsm.h index 19eb3c0029..83c4f51a1d 100644 --- a/source/libs/stream/inc/streamsm.h +++ b/source/libs/stream/inc/streamsm.h @@ -32,8 +32,8 @@ typedef int32_t (*__state_trans_fn)(SStreamTask*); typedef int32_t (*__state_trans_succ_fn)(SStreamTask*); typedef struct SAttachedEventInfo { - ETaskStatus status; - EStreamTaskEvent event; + ETaskStatus status; // required status that this event can be handled + EStreamTaskEvent event; // the delayed handled event } SAttachedEventInfo; typedef struct STaskStateTrans { @@ -48,13 +48,15 @@ typedef struct STaskStateTrans { struct SStreamTaskSM { SStreamTask* pTask; -// SArray* pTransList; // SArray STaskStateTrans* pActiveTrans; int64_t startTs; SStreamTaskState current; - SStreamTaskState prev; + struct { + SStreamTaskState state; + EStreamTaskEvent evt; + } prev; // register the next handled event, if current state is not allowed to handle this event - SArray* eventList; + SArray* pWaitingEventList; }; typedef struct SStreamEventInfo { diff --git a/source/libs/stream/src/streamTaskSm.c b/source/libs/stream/src/streamTaskSm.c index addd563388..463b7ae771 100644 --- a/source/libs/stream/src/streamTaskSm.c +++ b/source/libs/stream/src/streamTaskSm.c @@ -32,10 +32,11 @@ SStreamTaskState StreamTaskStatusList[9] = { {.state = TASK_STATUS__STREAM_SCAN_HISTORY, .name = "stream-scan-history"}, }; -SStreamEventInfo StreamTaskEventList[11] = { +SStreamEventInfo StreamTaskEventList[12] = { {}, // dummy event, place holder {.event = TASK_EVENT_INIT, .name = "initialize"}, - {.event = TASK_EVENT_INIT_SCANHIST, .name = "scan-history-initialize"}, + {.event = TASK_EVENT_INIT_SCANHIST, .name = "scan-history-init"}, + {.event = TASK_EVENT_INIT_STREAM_SCANHIST, .name = "stream-scan-history-init"}, {.event = TASK_EVENT_SCANHIST_DONE, .name = "scan-history-completed"}, {.event = TASK_EVENT_STOP, .name = "stopping"}, {.event = TASK_EVENT_GEN_CHECKPOINT, .name = "checkpoint"}, @@ -66,7 +67,7 @@ static int32_t attachEvent(SStreamTask* pTask, SAttachedEventInfo* pEvtInfo) { stDebug("s-task:%s status:%s attach event:%s required status:%s, since not allowed to handle it", pTask->id.idStr, p, StreamTaskEventList[pEvtInfo->event].name, StreamTaskStatusList[pEvtInfo->status].name); - taosArrayPush(pTask->status.pSM->eventList, pEvtInfo); + taosArrayPush(pTask->status.pSM->pWaitingEventList, pEvtInfo); return 0; } @@ -126,12 +127,15 @@ void streamTaskRestoreStatus(SStreamTask* pTask) { ASSERT(pSM->current.state == TASK_STATUS__PAUSE || pSM->current.state == TASK_STATUS__HALT); SStreamTaskState state = pSM->current; - pSM->current = pSM->prev; - pSM->prev = state; + pSM->current = pSM->prev.state; + + pSM->prev.state = state; + pSM->prev.evt = 0; + pSM->startTs = taosGetTimestampMs(); taosThreadMutexUnlock(&pTask->lock); - stDebug("s-task:%s restore status, %s -> %s", pTask->id.idStr, pSM->prev.name, pSM->current.name); + stDebug("s-task:%s restore status, %s -> %s", pTask->id.idStr, pSM->prev.state.name, pSM->current.name); } SStreamTaskSM* streamCreateStateMachine(SStreamTask* pTask) { @@ -147,8 +151,8 @@ SStreamTaskSM* streamCreateStateMachine(SStreamTask* pTask) { } pSM->pTask = pTask; - pSM->eventList = taosArrayInit(4, sizeof(SAttachedEventInfo)); - if (pSM->eventList == NULL) { + pSM->pWaitingEventList = taosArrayInit(4, sizeof(SAttachedEventInfo)); + if (pSM->pWaitingEventList == NULL) { taosMemoryFree(pSM); terrno = TSDB_CODE_OUT_OF_MEMORY; @@ -168,7 +172,7 @@ void* streamDestroyStateMachine(SStreamTaskSM* pSM) { return NULL; } - taosArrayDestroy(pSM->eventList); + taosArrayDestroy(pSM->pWaitingEventList); taosMemoryFree(pSM); return NULL; } @@ -192,7 +196,8 @@ int32_t streamTaskHandleEvent(SStreamTaskSM* pSM, EStreamTaskEvent event) { ETaskStatus s = streamTaskGetStatus(pTask, NULL); taosThreadMutexUnlock(&pTask->lock); - if (s == pTrans->attachEvent.status) { + if ((s == pTrans->next.state) && (pSM->prev.evt == pTrans->event)) { + stDebug("s-task:%s attached event:%s handled", pTask->id.idStr, StreamTaskEventList[pTrans->event].name); return TSDB_CODE_SUCCESS; } else {// this event has been handled already stDebug("s-task:%s not handle event:%s yet, wait for 100ms and recheck", pTask->id.idStr, @@ -223,27 +228,32 @@ int32_t streamTaskHandleEvent(SStreamTaskSM* pSM, EStreamTaskEvent event) { return TSDB_CODE_SUCCESS; } +static void keepPrevInfo(SStreamTaskSM* pSM) { + STaskStateTrans* pTrans = pSM->pActiveTrans; + + pSM->prev.state = pSM->current; + pSM->prev.evt = pTrans->event; +} int32_t streamTaskOnHandleEventSuccess(SStreamTaskSM* pSM) { STaskStateTrans* pTrans = pSM->pActiveTrans; SStreamTask* pTask = pSM->pTask; // do update the task status taosThreadMutexLock(&pTask->lock); - SStreamTaskState current = pSM->current; + keepPrevInfo(pSM); - pSM->prev = pSM->current; pSM->current = pTrans->next; pSM->pActiveTrans = NULL; // on success callback, add into lock if necessary, or maybe we should add an option for this? pTrans->pSuccAction(pTask); - if (taosArrayGetSize(pSM->eventList) > 0) { + if (taosArrayGetSize(pSM->pWaitingEventList) > 0) { int64_t el = (taosGetTimestampMs() - pSM->startTs); stDebug("s-task:%s handle event:%s completed, elapsed time:%" PRId64 "ms state:%s -> %s", pTask->id.idStr, - StreamTaskEventList[pTrans->event].name, el, current.name, pSM->current.name); + StreamTaskEventList[pTrans->event].name, el, pSM->prev.state.name, pSM->current.name); - SAttachedEventInfo* pEvtInfo = taosArrayPop(pSM->eventList); + SAttachedEventInfo* pEvtInfo = taosArrayPop(pSM->pWaitingEventList); // OK, let's handle the attached event, since the task has reached the required status now if (pSM->current.state == pEvtInfo->status) { @@ -258,7 +268,6 @@ int32_t streamTaskOnHandleEventSuccess(SStreamTaskSM* pSM) { taosThreadMutexUnlock(&pTask->lock); int32_t code = pNextTrans->pAction(pSM->pTask); - if (pTrans->autoInvokeEndFn) { return streamTaskOnHandleEventSuccess(pSM); } else { @@ -270,7 +279,7 @@ int32_t streamTaskOnHandleEventSuccess(SStreamTaskSM* pSM) { int64_t el = (taosGetTimestampMs() - pSM->startTs); stDebug("s-task:%s handle event:%s completed, elapsed time:%" PRId64 "ms state:%s -> %s", pTask->id.idStr, - StreamTaskEventList[pTrans->event].name, el, current.name, pSM->current.name); + StreamTaskEventList[pTrans->event].name, el, pSM->prev.state.name, pSM->current.name); } return TSDB_CODE_SUCCESS; @@ -292,7 +301,7 @@ void streamTaskResetStatus(SStreamTask* pTask) { SStreamTaskSM* pSM = pTask->status.pSM; pSM->current = StreamTaskStatusList[TASK_STATUS__UNINIT]; pSM->pActiveTrans = NULL; - taosArrayClear(pSM->eventList); + taosArrayClear(pSM->pWaitingEventList); } void streamTaskSetStatusReady(SStreamTask* pTask) { @@ -302,12 +311,13 @@ void streamTaskSetStatusReady(SStreamTask* pTask) { return; } - pSM->prev = pSM->current; + pSM->prev.state = pSM->current; + pSM->prev.evt = 0; pSM->current = StreamTaskStatusList[TASK_STATUS__READY]; pSM->startTs = taosGetTimestampMs(); pSM->pActiveTrans = NULL; - taosArrayClear(pSM->eventList); + taosArrayClear(pSM->pWaitingEventList); } STaskStateTrans createStateTransform(ETaskStatus current, ETaskStatus next, EStreamTaskEvent event, __state_trans_fn fn, diff --git a/source/util/src/tarray.c b/source/util/src/tarray.c index a7c28df22b..26d149b5b5 100644 --- a/source/util/src/tarray.c +++ b/source/util/src/tarray.c @@ -191,7 +191,7 @@ void* taosArrayGet(const SArray* pArray, size_t index) { } if (index >= pArray->size) { - uError("index is out of range, current:%" PRIzu " max:%d", index, pArray->capacity); + uError("index is out of range, current:%" PRIzu " max:%"PRIzu, index, pArray->size); return NULL; } diff --git a/tests/system-test/8-stream/scalar_function.py b/tests/system-test/8-stream/scalar_function.py index 3bc44a7dc7..eda643f661 100644 --- a/tests/system-test/8-stream/scalar_function.py +++ b/tests/system-test/8-stream/scalar_function.py @@ -7,7 +7,7 @@ from util.common import * class TDTestCase: updatecfgDict = {'vdebugFlag': 143, 'qdebugflag':135, 'tqdebugflag':135, 'udebugflag':135, 'rpcdebugflag':135, - 'asynclog': 0} + 'asynclog': 0, 'stdebugflag':135} def init(self, conn, logSql, replicaVar=1): self.replicaVar = int(replicaVar) tdLog.debug("start to execute %s" % __file__) From 4d5d078fbe986155a2bfee1cc2ce1912998f8ae5 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 19 Oct 2023 18:00:54 +0800 Subject: [PATCH 08/41] fix(stream): fix bugs caused by refactor. --- source/libs/stream/inc/streamsm.h | 1 - 1 file changed, 1 deletion(-) diff --git a/source/libs/stream/inc/streamsm.h b/source/libs/stream/inc/streamsm.h index 83c4f51a1d..be3665fde7 100644 --- a/source/libs/stream/inc/streamsm.h +++ b/source/libs/stream/inc/streamsm.h @@ -62,7 +62,6 @@ struct SStreamTaskSM { typedef struct SStreamEventInfo { EStreamTaskEvent event; const char* name; - bool isTrans; } SStreamEventInfo; SStreamTaskSM* streamCreateStateMachine(SStreamTask* pTask); From 270a84780903e595b298dbea209eaf7d75f4dd76 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Fri, 20 Oct 2023 10:23:48 +0800 Subject: [PATCH 09/41] fix(stream): fix bugs caused by refactor sm. --- include/libs/stream/tstream.h | 3 ++- source/dnode/snode/src/snode.c | 5 +++-- source/dnode/vnode/src/tq/tq.c | 2 +- source/dnode/vnode/src/tq/tqStreamTask.c | 2 +- source/libs/stream/src/streamExec.c | 12 ++++++------ source/libs/stream/src/streamMeta.c | 12 +++++------- source/libs/stream/src/streamTask.c | 21 ++++++++++++++++++++- source/libs/stream/src/streamTaskSm.c | 8 ++++---- 8 files changed, 42 insertions(+), 23 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 6564bcc769..6c2cec6292 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -49,7 +49,7 @@ extern "C" { do { \ (_t)->hTaskInfo.id.taskId = 0; \ (_t)->hTaskInfo.id.streamId = 0; \ - } while (0); + } while (0) typedef struct SStreamTask SStreamTask; typedef struct SStreamQueue SStreamQueue; @@ -739,6 +739,7 @@ bool streamTaskAllUpstreamClosed(SStreamTask* pTask); bool streamTaskSetSchedStatusWait(SStreamTask* pTask); int8_t streamTaskSetSchedStatusActive(SStreamTask* pTask); int8_t streamTaskSetSchedStatusInactive(SStreamTask* pTask); +int32_t streamTaskClearHTaskAttr(SStreamTask* pTask); int32_t streamTaskHandleEvent(SStreamTaskSM* pSM, EStreamTaskEvent event); int32_t streamTaskOnHandleEventSuccess(SStreamTaskSM* pSM); diff --git a/source/dnode/snode/src/snode.c b/source/dnode/snode/src/snode.c index 4015ba9c61..4e84b4cd26 100644 --- a/source/dnode/snode/src/snode.c +++ b/source/dnode/snode/src/snode.c @@ -183,8 +183,9 @@ int32_t sndProcessTaskDeployReq(SSnode *pSnode, char *msg, int32_t msgLen) { qDebug("snode:%d s-task:%s is deployed on snode and add into meta, status:%s, numOfTasks:%d", SNODE_HANDLE, pTask->id.idStr, p, numOfTasks); - ASSERT(0); -// streamTaskCheckDownstream(pTask); + EStreamTaskEvent event = (HAS_RELATED_FILLHISTORY_TASK(pTask)) ? TASK_EVENT_INIT_STREAM_SCANHIST : TASK_EVENT_INIT; + streamTaskHandleEvent(pTask->status.pSM, event); + streamTaskCheckDownstream(pTask); return 0; } diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index c4e74f84e5..e99503df33 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1029,7 +1029,7 @@ int32_t tqProcessTaskDeployReq(STQ* pTq, int64_t sversion, char* msg, int32_t ms bool restored = pTq->pVnode->restored; if (p != NULL && restored && p->info.fillHistory == 0) { - EStreamTaskEvent event = (p->hTaskInfo.id.taskId == 0) ? TASK_EVENT_INIT : TASK_EVENT_INIT_STREAM_SCANHIST; + EStreamTaskEvent event = (HAS_RELATED_FILLHISTORY_TASK(p)) ? TASK_EVENT_INIT_STREAM_SCANHIST : TASK_EVENT_INIT; streamTaskHandleEvent(p->status.pSM, event); } else if (!restored) { tqWarn("s-task:%s not launched since vnode(vgId:%d) not ready", p->id.idStr, vgId); diff --git a/source/dnode/vnode/src/tq/tqStreamTask.c b/source/dnode/vnode/src/tq/tqStreamTask.c index 8dfa1e2670..4bc386fb9a 100644 --- a/source/dnode/vnode/src/tq/tqStreamTask.c +++ b/source/dnode/vnode/src/tq/tqStreamTask.c @@ -99,7 +99,7 @@ int32_t tqStartStreamTask(STQ* pTq) { continue; } - EStreamTaskEvent event = (pTask->hTaskInfo.id.taskId == 0) ? TASK_EVENT_INIT : TASK_EVENT_INIT_STREAM_SCANHIST; + EStreamTaskEvent event = (HAS_RELATED_FILLHISTORY_TASK(pTask)) ? TASK_EVENT_INIT_STREAM_SCANHIST : TASK_EVENT_INIT; streamTaskHandleEvent(pTask->status.pSM, event); streamMetaReleaseTask(pMeta, pTask); } diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index 8c37e785dd..2b1ea7c911 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -360,16 +360,16 @@ int32_t streamDoTransferStateToStreamTask(SStreamTask* pTask) { streamBuildAndSendDropTaskMsg(pTask->pMsgCb, pMeta->vgId, &pTask->id); // 5. clear the link between fill-history task and stream task info - CLEAR_RELATED_FILLHISTORY_TASK(pStreamTask); +// CLEAR_RELATED_FILLHISTORY_TASK(pStreamTask); // 6. save to disk taosWLockLatch(&pMeta->lock); pStreamTask->status.taskStatus = streamTaskGetStatus(pStreamTask, NULL); - streamMetaSaveTask(pMeta, pStreamTask); - if (streamMetaCommit(pMeta) < 0) { +// streamMetaSaveTask(pMeta, pStreamTask); +// if (streamMetaCommit(pMeta) < 0) { // persist to disk - } +// } taosWUnLockLatch(&pMeta->lock); // 7. pause allowed. @@ -499,10 +499,10 @@ int32_t streamProcessTranstateBlock(SStreamTask* pTask, SStreamDataBlock* pBlock } } else { // non-dispatch task, do task state transfer directly streamFreeQitem((SStreamQueueItem*)pBlock); - stDebug("s-task:%s non-dispatch task, start to transfer state directly", id); + stDebug("s-task:%s non-dispatch task, level:%d start to transfer state directly", id, pTask->info.taskLevel); ASSERT(pTask->info.fillHistory == 1); - code = streamTransferStateToStreamTask(pTask); + code = streamTransferStateToStreamTask(pTask); if (code != TSDB_CODE_SUCCESS) { /*int8_t status = */ streamTaskSetSchedStatusInactive(pTask); } diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 206f1fcfc6..a1ed9a1458 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -548,21 +548,19 @@ int32_t streamMetaUnregisterTask(SStreamMeta* pMeta, int64_t streamId, int32_t t taosWLockLatch(&pMeta->lock); ppTask = (SStreamTask**)taosHashGet(pMeta->pTasksMap, &id, sizeof(id)); if (ppTask) { + pTask = *ppTask; + // it is an fill-history task, remove the related stream task's id that points to it - if ((*ppTask)->info.fillHistory == 1) { - STaskId streamTaskId = {.streamId = (*ppTask)->streamTaskId.streamId, .taskId = (*ppTask)->streamTaskId.taskId}; - SStreamTask** ppStreamTask = (SStreamTask**)taosHashGet(pMeta->pTasksMap, &streamTaskId, sizeof(streamTaskId)); - if (ppStreamTask != NULL) { - CLEAR_RELATED_FILLHISTORY_TASK((*ppStreamTask)); - } + if (pTask->info.fillHistory == 1) { + streamTaskClearHTaskAttr(pTask); } else { atomic_sub_fetch_32(&pMeta->numOfStreamTasks, 1); } taosHashRemove(pMeta->pTasksMap, &id, sizeof(id)); + doRemoveIdFromList(pMeta, (int32_t)taosArrayGetSize(pMeta->pTaskList), &pTask->id); ASSERT(pTask->status.timerActive == 0); - doRemoveIdFromList(pMeta, (int32_t)taosArrayGetSize(pMeta->pTaskList), &pTask->id); if (pTask->info.triggerParam != 0 && pTask->info.fillHistory == 0) { stDebug("s-task:%s stop schedTimer, and (before) desc ref:%d", pTask->id.idStr, pTask->refCnt); diff --git a/source/libs/stream/src/streamTask.c b/source/libs/stream/src/streamTask.c index 1d12401d12..231f1ce299 100644 --- a/source/libs/stream/src/streamTask.c +++ b/source/libs/stream/src/streamTask.c @@ -691,6 +691,25 @@ int8_t streamTaskSetSchedStatusInactive(SStreamTask* pTask) { return status; } +int32_t streamTaskClearHTaskAttr(SStreamTask* pTask) { + SStreamMeta* pMeta = pTask->pMeta; + if (pTask->info.fillHistory == 0) { + return TSDB_CODE_SUCCESS; + } + + STaskId sTaskId = {.streamId = pTask->streamTaskId.streamId, .taskId = pTask->streamTaskId.taskId}; + SStreamTask** ppStreamTask = (SStreamTask**)taosHashGet(pMeta->pTasksMap, &sTaskId, sizeof(sTaskId)); + + if (ppStreamTask != NULL) { + CLEAR_RELATED_FILLHISTORY_TASK((*ppStreamTask)); + streamMetaSaveTask(pMeta, *ppStreamTask); + stDebug("s-task:%s clear the related stream task:0x%x attr to fill-history task", pTask->id.idStr, + (int32_t)sTaskId.taskId); + } + + return TSDB_CODE_SUCCESS; +} + int32_t streamBuildAndSendDropTaskMsg(SMsgCb* pMsgCb, int32_t vgId, SStreamTaskId* pTaskId) { SVDropStreamTaskReq *pReq = rpcMallocCont(sizeof(SVDropStreamTaskReq)); if (pReq == NULL) { @@ -709,7 +728,7 @@ int32_t streamBuildAndSendDropTaskMsg(SMsgCb* pMsgCb, int32_t vgId, SStreamTaskI return code; } - stDebug("vgId:%d build and send drop table:0x%x msg", vgId, pTaskId->taskId); + stDebug("vgId:%d build and send drop task:0x%x msg", vgId, pTaskId->taskId); return code; } diff --git a/source/libs/stream/src/streamTaskSm.c b/source/libs/stream/src/streamTaskSm.c index 463b7ae771..7abdd155b9 100644 --- a/source/libs/stream/src/streamTaskSm.c +++ b/source/libs/stream/src/streamTaskSm.c @@ -261,14 +261,14 @@ int32_t streamTaskOnHandleEventSuccess(SStreamTaskSM* pSM) { StreamTaskEventList[pEvtInfo->event].name, pSM->current.name); STaskStateTrans* pNextTrans = streamTaskFindTransform(pSM, pEvtInfo->event); - ASSERT(pSM->pActiveTrans == NULL); + ASSERT(pSM->pActiveTrans == NULL && pNextTrans != NULL); + pSM->pActiveTrans = pNextTrans; pSM->startTs = taosGetTimestampMs(); - taosThreadMutexUnlock(&pTask->lock); - + int32_t code = pNextTrans->pAction(pSM->pTask); - if (pTrans->autoInvokeEndFn) { + if (pNextTrans->autoInvokeEndFn) { return streamTaskOnHandleEventSuccess(pSM); } else { return code; From 3b8c85f632ae25ca1529a9afbff95d3c952260d2 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Fri, 20 Oct 2023 14:19:57 +0800 Subject: [PATCH 10/41] fix(stream): fix bugs caused by refactor sm. --- include/libs/stream/tstream.h | 4 +- source/dnode/vnode/src/tq/tq.c | 6 +-- source/libs/stream/src/streamExec.c | 8 +++- source/libs/stream/src/streamStart.c | 57 +++++++--------------------- 4 files changed, 26 insertions(+), 49 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 6c2cec6292..0ca401e3a4 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -717,6 +717,7 @@ int32_t streamSchedExec(SStreamTask* pTask); bool streamTaskShouldStop(const SStreamTask* pStatus); bool streamTaskShouldPause(const SStreamTask* pStatus); bool streamTaskIsIdle(const SStreamTask* pTask); +bool streamTaskReadyToRun(const SStreamTask* pTask, char** pStatus); char* createStreamTaskIdStr(int64_t streamId, int32_t taskId); ETaskStatus streamTaskGetStatus(const SStreamTask* pTask, char** pStr); @@ -758,8 +759,7 @@ int32_t streamQueueGetNumOfItems(const SStreamQueue* pQueue); // common int32_t streamRestoreParam(SStreamTask* pTask); void streamTaskPause(SStreamTask* pTask, SStreamMeta* pMeta); -void streamTaskResume(SStreamTask* pTask, SStreamMeta* pMeta); -void streamTaskResumeFromHalt(SStreamTask* pTask); +void streamTaskResume(SStreamTask* pTask); void streamTaskDisablePause(SStreamTask* pTask); void streamTaskEnablePause(SStreamTask* pTask); int32_t streamTaskSetUpstreamInfo(SStreamTask* pTask, const SStreamTask* pUpstreamTask); diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index e99503df33..f9abb5f2c9 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1365,8 +1365,7 @@ int32_t tqProcessTaskRunReq(STQ* pTq, SRpcMsg* pMsg) { SStreamTask* pTask = streamMetaAcquireTask(pTq->pStreamMeta, pReq->streamId, taskId); if (pTask != NULL) { // even in halt status, the data in inputQ must be processed char* p = NULL; - ETaskStatus st = streamTaskGetStatus(pTask, &p); - if (st == TASK_STATUS__READY || st == TASK_STATUS__SCAN_HISTORY || st == TASK_STATUS__CK) { + if (streamTaskReadyToRun(pTask, &p)) { tqDebug("vgId:%d s-task:%s start to process block from inputQ, next checked ver:%" PRId64, vgId, pTask->id.idStr, pTask->chkInfo.nextProcessVer); streamExecTask(pTask); @@ -1515,7 +1514,7 @@ int32_t tqProcessTaskResumeImpl(STQ* pTq, SStreamTask* pTask, int64_t sversion, return -1; } - streamTaskResume(pTask, pTq->pStreamMeta); + streamTaskResume(pTask); int32_t level = pTask->info.taskLevel; if (level == TASK_LEVEL__SINK) { @@ -1874,6 +1873,7 @@ int32_t tqProcessTaskUpdateReq(STQ* pTq, SRpcMsg* pMsg) { if (ppHTask == NULL || *ppHTask == NULL) { tqError("vgId:%d failed to acquire fill-history task:0x%x when handling update, it may have been dropped already", pMeta->vgId, req.taskId); + CLEAR_RELATED_FILLHISTORY_TASK(pTask); } else { tqDebug("s-task:%s fill-history task update nodeEp along with stream task", (*ppHTask)->id.idStr); streamTaskUpdateEpsetInfo(*ppHTask, req.pNodeList); diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index 2b1ea7c911..2d5788fc4d 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -352,7 +352,7 @@ int32_t streamDoTransferStateToStreamTask(SStreamTask* pTask) { // 3. resume the state of stream task, after this function, the stream task will run immidately. But it can not be // pause, since the pause allowed attribute is not set yet. - streamTaskResumeFromHalt(pStreamTask); // todo refactor: use streamTaskResume. + streamTaskResume(pStreamTask); // todo refactor: use streamTaskResume. stDebug("s-task:%s fill-history task set status to be dropping, save the state into disk", pTask->id.idStr); @@ -610,6 +610,12 @@ bool streamTaskIsIdle(const SStreamTask* pTask) { status == TASK_STATUS__DROPPING); } +bool streamTaskReadyToRun(const SStreamTask* pTask, char** pStatus) { + ETaskStatus st = streamTaskGetStatus(pTask, NULL); + return (st == TASK_STATUS__READY || st == TASK_STATUS__SCAN_HISTORY || st == TASK_STATUS__STREAM_SCAN_HISTORY || + st == TASK_STATUS__CK); +} + int32_t streamExecTask(SStreamTask* pTask) { // this function may be executed by multi-threads, so status check is required. const char* id = pTask->id.idStr; diff --git a/source/libs/stream/src/streamStart.c b/source/libs/stream/src/streamStart.c index 66865c8e25..b52d9177b7 100644 --- a/source/libs/stream/src/streamStart.c +++ b/source/libs/stream/src/streamStart.c @@ -1021,36 +1021,25 @@ void streamTaskPause(SStreamTask* pTask, SStreamMeta* pMeta) { stDebug("vgId:%d s-task:%s set pause flag and pause task", pMeta->vgId, pTask->id.idStr); } -void streamTaskResume(SStreamTask* pTask, SStreamMeta* pMeta) { - char* p = NULL; - ETaskStatus status = streamTaskGetStatus(pTask, &p); +void streamTaskResume(SStreamTask* pTask) { + char* p = NULL; + ETaskStatus status = streamTaskGetStatus(pTask, &p); + SStreamMeta* pMeta = pTask->pMeta; - if (status == TASK_STATUS__PAUSE) { + if (status == TASK_STATUS__PAUSE || status == TASK_STATUS__HALT) { streamTaskRestoreStatus(pTask); - streamTaskGetStatus(pTask, &p); - int32_t num = atomic_sub_fetch_32(&pMeta->numOfPausedTasks, 1); - stInfo("vgId:%d s-task:%s sink task.resume from pause, status:%s. pause task num:%d", pMeta->vgId, pTask->id.idStr, - p, num); + char* pNew = NULL; + streamTaskGetStatus(pTask, &pNew); + if (status == TASK_STATUS__PAUSE) { + int32_t num = atomic_sub_fetch_32(&pMeta->numOfPausedTasks, 1); + stInfo("s-task:%s status:%s resume from %s, paused task(s):%d", pTask->id.idStr, pNew, p, num); + } else { + stInfo("s-task:%s status:%s resume from %s", pTask->id.idStr, pNew, p); + } } else { - stDebug("s-task:%s status:%s not in pause status, no need to resume", pTask->id.idStr, p); + stDebug("s-task:%s status:%s not in pause/halt status, no need to resume", pTask->id.idStr, p); } - -#if 0 - int8_t status = pTask->status.taskStatus; - if (status == TASK_STATUS__PAUSE) { - pTask->status.taskStatus = pTask->status.keepTaskStatus; - pTask->status.keepTaskStatus = TASK_STATUS__READY; - int32_t num = atomic_sub_fetch_32(&pMeta->numOfPausedTasks, 1); - stInfo("vgId:%d s-task:%s resume from pause, status:%s. pause task num:%d", pMeta->vgId, pTask->id.idStr, streamGetTaskStatusStr(status), num); - } else if (pTask->info.taskLevel == TASK_LEVEL__SINK) { - int32_t num = atomic_sub_fetch_32(&pMeta->numOfPausedTasks, 1); - stInfo("vgId:%d s-task:%s sink task.resume from pause, status:%s. pause task num:%d", pMeta->vgId, pTask->id.idStr, streamGetTaskStatusStr(status), num); - } else { - stError("s-task:%s not in pause, failed to resume, status:%s", pTask->id.idStr, streamGetTaskStatusStr(status)); - } -#endif - } // todo fix race condition @@ -1071,24 +1060,6 @@ void streamTaskEnablePause(SStreamTask* pTask) { pTask->status.pauseAllowed = 1; } -void streamTaskResumeFromHalt(SStreamTask* pTask) { - const char* id = pTask->id.idStr; - char* p = NULL; - - ASSERT(streamTaskGetStatus(pTask, NULL) == TASK_STATUS__HALT); -// int8_t status = pTask->status.taskStatus; -// if (status != TASK_STATUS__HALT) { -// stError("s-task:%s not in halt status, status:%s", id, streamGetTaskStatusStr(status)); -// return; -// } - -// pTask->status.taskStatus = pTask->status.keepTaskStatus; -// pTask->status.keepTaskStatus = TASK_STATUS__READY; - streamTaskRestoreStatus(pTask); - streamTaskGetStatus(pTask, &p); - stDebug("s-task:%s resume from halt, current status:%s", id, p); -} - int32_t updateTaskReadyInMeta(SStreamTask* pTask) { SStreamMeta* pMeta = pTask->pMeta; From c4268ca3959d98795fb996ac1b5b819dde22e700 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Fri, 20 Oct 2023 15:54:30 +0800 Subject: [PATCH 11/41] fix(stream): fix bugs caused by refactor. --- source/dnode/vnode/src/tq/tqStreamTask.c | 2 +- source/libs/stream/src/streamTaskSm.c | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/source/dnode/vnode/src/tq/tqStreamTask.c b/source/dnode/vnode/src/tq/tqStreamTask.c index 4bc386fb9a..cb313b7eed 100644 --- a/source/dnode/vnode/src/tq/tqStreamTask.c +++ b/source/dnode/vnode/src/tq/tqStreamTask.c @@ -91,7 +91,7 @@ int32_t tqStartStreamTask(STQ* pTq) { continue; } - if (pTask->status.downstreamReady == 1) { + if (pTask->status.downstreamReady == 1 && HAS_RELATED_FILLHISTORY_TASK(pTask)) { tqDebug("s-task:%s downstream ready, no need to check downstream, check only related fill-history task", pTask->id.idStr); streamLaunchFillHistoryTask(pTask); diff --git a/source/libs/stream/src/streamTaskSm.c b/source/libs/stream/src/streamTaskSm.c index 7abdd155b9..3e148c0a35 100644 --- a/source/libs/stream/src/streamTaskSm.c +++ b/source/libs/stream/src/streamTaskSm.c @@ -33,7 +33,7 @@ SStreamTaskState StreamTaskStatusList[9] = { }; SStreamEventInfo StreamTaskEventList[12] = { - {}, // dummy event, place holder + {.event = 0, .name = ""}, // dummy event, place holder {.event = TASK_EVENT_INIT, .name = "initialize"}, {.event = TASK_EVENT_INIT_SCANHIST, .name = "scan-history-init"}, {.event = TASK_EVENT_INIT_STREAM_SCANHIST, .name = "stream-scan-history-init"}, From 4feebffb05dea58531292efed77aa26c5867bee5 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Fri, 20 Oct 2023 15:59:43 +0800 Subject: [PATCH 12/41] refactor: add ref in log. --- source/libs/stream/src/streamTask.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/libs/stream/src/streamTask.c b/source/libs/stream/src/streamTask.c index 231f1ce299..2f14f38fdc 100644 --- a/source/libs/stream/src/streamTask.c +++ b/source/libs/stream/src/streamTask.c @@ -317,7 +317,7 @@ void tFreeStreamTask(SStreamTask* pTask) { // remove the ref by timer while (pTask->status.timerActive > 0) { - stDebug("s-task:%s wait for task stop timer activities", pTask->id.idStr); + stDebug("s-task:%s wait for task stop timer activities, ref:%d", pTask->id.idStr, pTask->status.timerActive); taosMsleep(10); } From 3c719da5b55975db8b6966be7875228ab6f860b5 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Fri, 20 Oct 2023 16:18:17 +0800 Subject: [PATCH 13/41] change stDebugFlag of system test --- tests/pytest/util/dnodes.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/pytest/util/dnodes.py b/tests/pytest/util/dnodes.py index c4fc1ce654..87dd284368 100644 --- a/tests/pytest/util/dnodes.py +++ b/tests/pytest/util/dnodes.py @@ -135,6 +135,7 @@ class TDDnode: "vDebugFlag": "143", "tqDebugFlag": "143", "cDebugFlag": "143", + "stDebugFlag": "143", "jniDebugFlag": "143", "qDebugFlag": "143", "rpcDebugFlag": "143", From c68c0eda79b171dcfb43343bc9442a365d948db3 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Fri, 20 Oct 2023 16:27:39 +0800 Subject: [PATCH 14/41] fix(stream): fix bugs caused by refactor sm. --- source/libs/stream/src/streamCheckpoint.c | 13 ++++++++++--- source/libs/stream/src/streamTask.c | 2 +- source/libs/stream/src/streamTaskSm.c | 15 ++++++++++++--- 3 files changed, 23 insertions(+), 7 deletions(-) diff --git a/source/libs/stream/src/streamCheckpoint.c b/source/libs/stream/src/streamCheckpoint.c index 7026ac7119..28b67029ce 100644 --- a/source/libs/stream/src/streamCheckpoint.c +++ b/source/libs/stream/src/streamCheckpoint.c @@ -276,6 +276,8 @@ void streamTaskClearCheckInfo(SStreamTask* pTask) { } int32_t streamSaveAllTaskStatus(SStreamMeta* pMeta, int64_t checkpointId) { + int32_t vgId = pMeta->vgId; + taosWLockLatch(&pMeta->lock); for (int32_t i = 0; i < taosArrayGetSize(pMeta->pTaskList); ++i) { @@ -297,10 +299,15 @@ int32_t streamSaveAllTaskStatus(SStreamMeta* pMeta, int64_t checkpointId) { char* str = NULL; streamTaskGetStatus(p, &str); - streamTaskHandleEvent(p->status.pSM, TASK_EVENT_CHECKPOINT_DONE); - // save the task - streamMetaSaveTask(pMeta, p); + int32_t code = streamTaskHandleEvent(p->status.pSM, TASK_EVENT_CHECKPOINT_DONE); + if (code != TSDB_CODE_SUCCESS) { + stDebug("s-task:%s vgId:%d save task status failed, since handle event failed", p->id.idStr, vgId); + taosWUnLockLatch(&pMeta->lock); + return -1; + } else { // save the task + streamMetaSaveTask(pMeta, p); + } stDebug( "vgId:%d s-task:%s level:%d open upstream inputQ, commit task status after checkpoint completed, " diff --git a/source/libs/stream/src/streamTask.c b/source/libs/stream/src/streamTask.c index 2f14f38fdc..ec9715c068 100644 --- a/source/libs/stream/src/streamTask.c +++ b/source/libs/stream/src/streamTask.c @@ -318,7 +318,7 @@ void tFreeStreamTask(SStreamTask* pTask) { // remove the ref by timer while (pTask->status.timerActive > 0) { stDebug("s-task:%s wait for task stop timer activities, ref:%d", pTask->id.idStr, pTask->status.timerActive); - taosMsleep(10); + taosMsleep(100); } if (pTask->schedInfo.pTimer != NULL) { diff --git a/source/libs/stream/src/streamTaskSm.c b/source/libs/stream/src/streamTaskSm.c index 3e148c0a35..f42e6bbb3b 100644 --- a/source/libs/stream/src/streamTaskSm.c +++ b/source/libs/stream/src/streamTaskSm.c @@ -115,7 +115,11 @@ static STaskStateTrans* streamTaskFindTransform(const SStreamTaskSM* pState, con } } - ASSERT(0); + if (event == TASK_EVENT_CHECKPOINT_DONE && pState->current.state == TASK_STATUS__STOP) { + + } else { + ASSERT(0); + } return NULL; } @@ -183,8 +187,13 @@ int32_t streamTaskHandleEvent(SStreamTaskSM* pSM, EStreamTaskEvent event) { taosThreadMutexLock(&pTask->lock); STaskStateTrans* pTrans = streamTaskFindTransform(pSM, event); - stDebug("s-task:%s start to handle event:%s, state:%s", pTask->id.idStr, StreamTaskEventList[event].name, - pSM->current.name); + if (pTrans == NULL) { + stWarn("s-task:%s status:%s not allowed handle event:%s", pTask->id.idStr, pSM->current.name, StreamTaskEventList[event].name); + return -1; + } else { + stDebug("s-task:%s start to handle event:%s, state:%s", pTask->id.idStr, StreamTaskEventList[event].name, + pSM->current.name); + } if (pTrans->attachEvent.event != 0) { attachEvent(pTask, &pTrans->attachEvent); From e95735eea4e2a0dbc4596cf9a8df7ceb3751669b Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Fri, 20 Oct 2023 20:49:01 +0800 Subject: [PATCH 15/41] fix mem leak --- source/libs/stream/src/streamBackendRocksdb.c | 30 ++++++++++++++----- 1 file changed, 22 insertions(+), 8 deletions(-) diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index ce4feb38eb..c3c5b5cb76 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -993,7 +993,7 @@ int32_t streamBackendDoCheckpoint(void* arg, uint64_t checkpointId) { stDebug("stream backend:%p end to do checkpoint at:%s, time cost:%" PRId64 "ms", pHandle, pChkpIdDir, taosGetTimestampMs() - st); } - } else { + } else { stError("stream backend:%p failed to flush db at:%s", pHandle, pChkpIdDir); } // release all ref to cfWrapper; @@ -1467,16 +1467,24 @@ rocksdb_compactionfilter_t* compactFilteFactoryCreateFilter(void* arg, rocksdb_c void destroyRocksdbCfInst(RocksdbCfInst* inst) { int cfLen = sizeof(ginitDict) / sizeof(ginitDict[0]); - for (int i = 0; i < cfLen; i++) { - if (inst->pHandle[i]) rocksdb_column_family_handle_destroy((inst->pHandle)[i]); + if (inst->pHandle) { + for (int i = 0; i < cfLen; i++) { + if (inst->pHandle[i]) rocksdb_column_family_handle_destroy((inst->pHandle)[i]); + } + taosMemoryFree(inst->pHandle); } - rocksdb_writeoptions_destroy(inst->wOpt); - inst->wOpt = NULL; + if (inst->cfOpt) { + for (int i = 0; i < cfLen; i++) { + rocksdb_options_destroy(inst->cfOpt[i]); + rocksdb_block_based_options_destroy(((RocksdbCfParam*)inst->param)[i].tableOpt); + } + taosMemoryFreeClear(inst->cfOpt); + taosMemoryFreeClear(inst->param); + } + if (inst->wOpt) rocksdb_writeoptions_destroy(inst->wOpt); + if (inst->rOpt) rocksdb_readoptions_destroy(inst->rOpt); - rocksdb_readoptions_destroy(inst->rOpt); - taosMemoryFree(inst->cfOpt); - taosMemoryFreeClear(inst->param); taosMemoryFree(inst); } @@ -1645,6 +1653,12 @@ int streamStateOpenBackend(void* backend, SStreamState* pState) { pState->pTdbState->backendCfWrapperId = id; pState->pTdbState->pBackendCfWrapper = pBackendCfWrapper; stInfo("succ to open state %p on backendWrapper, %p, %s", pState, pBackendCfWrapper, pBackendCfWrapper->idstr); + + inst->pHandle = NULL; + inst->wOpt = NULL; + inst->rOpt = NULL; + inst->cfOpt = NULL; + inst->param = NULL; return 0; } taosThreadMutexUnlock(&handle->cfMutex); From 49014d4b6892f57c474599bfe509afd86ae46742 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sat, 21 Oct 2023 01:42:43 +0800 Subject: [PATCH 16/41] fix(stream): update the nodeEp even during checkpoint trans. --- source/dnode/mnode/impl/src/mndStream.c | 68 ++++++++++++++----------- source/libs/stream/src/streamDispatch.c | 12 ++--- 2 files changed, 43 insertions(+), 37 deletions(-) diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index 7e254e5efb..34041b26fb 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -83,14 +83,14 @@ static SArray *mndTakeVgroupSnapshot(SMnode *pMnode); static SVgroupChangeInfo mndFindChangedNodeInfo(SMnode *pMnode, const SArray *pPrevNodeList, const SArray *pNodeList); static STrans *doCreateTrans(SMnode *pMnode, SStreamObj *pStream, const char *name); -static STrans *doCreateTrans1(SMnode *pMnode, const char *name, const char* pDbName); static int32_t mndPersistTransLog(SStreamObj *pStream, STrans *pTrans); static void initTransAction(STransAction *pAction, void *pCont, int32_t contLen, int32_t msgType, const SEpSet *pEpset); static int32_t createStreamUpdateTrans(SStreamObj *pStream, SVgroupChangeInfo *pInfo, STrans *pTrans); -static void removeStreamTasksInBuf(SStreamObj* pStream, SStreamExecNodeInfo * pExecNode); -static void keepStreamTasksInBuf(SStreamObj *pStream, SStreamExecNodeInfo *pExecNode); +static void removeStreamTasksInBuf(SStreamObj *pStream, SStreamExecNodeInfo *pExecNode); +static void keepStreamTasksInBuf(SStreamObj *pStream, SStreamExecNodeInfo *pExecNode); static int32_t removeExpirednodeEntryAndTask(SArray *pNodeSnapshot); +static int32_t doKillActiveCheckpointTrans(SMnode *pMnode); int32_t mndInitStream(SMnode *pMnode) { SSdbTable table = { @@ -1189,7 +1189,6 @@ static int32_t mndProcessStreamDoCheckpoint(SRpcMsg *pReq) { } } taosThreadMutexUnlock(&execNodeList.lock); - if (!ready) { return 0; } @@ -1203,7 +1202,8 @@ static int32_t mndProcessStreamDoCheckpoint(SRpcMsg *pReq) { mError("failed to trigger checkpoint, reason: %s", tstrerror(TSDB_CODE_OUT_OF_MEMORY)); return -1; } - mDebug("start to trigger checkpoint, checkpointId: %" PRId64 "", checkpointId); + + mDebug("start to trigger checkpoint, checkpointId: %" PRId64, checkpointId); const char *pDb = mndGetStreamDB(pMnode); mndTransSetDbName(pTrans, pDb, "checkpoint"); @@ -2082,8 +2082,7 @@ static int32_t mndProcessVgroupChange(SMnode *pMnode, SVgroupChangeInfo *pChange // check all streams that involved this vnode should update the epset info SStreamObj *pStream = NULL; void *pIter = NULL; - - STrans *pTrans = NULL; + STrans *pTrans = NULL; while (1) { pIter = sdbFetch(pSdb, SDB_STREAM, pIter, (void **)&pStream); @@ -2091,6 +2090,7 @@ static int32_t mndProcessVgroupChange(SMnode *pMnode, SVgroupChangeInfo *pChange break; } + // here create only one trans if (pTrans == NULL) { pTrans = doCreateTrans(pMnode, pStream, "stream-task-update"); if (pTrans == NULL) { @@ -2137,8 +2137,6 @@ static int32_t mndProcessVgroupChange(SMnode *pMnode, SVgroupChangeInfo *pChange sdbRelease(pMnode->pSdb, pStream); mndTransDrop(pTrans); - // return TSDB_CODE_ACTION_IN_PROGRESS; - return 0; } @@ -2295,7 +2293,6 @@ static int32_t mndProcessNodeCheckReq(SRpcMsg *pMsg) { if (execNodeList.pNodeEntryList != NULL) { execNodeList.pNodeEntryList = taosArrayDestroy(execNodeList.pNodeEntryList); } - execNodeList.pNodeEntryList = extractNodeListFromStream(pMnode); } @@ -2313,6 +2310,9 @@ static int32_t mndProcessNodeCheckReq(SRpcMsg *pMsg) { SVgroupChangeInfo changeInfo = mndFindChangedNodeInfo(pMnode, execNodeList.pNodeEntryList, pNodeSnapshot); if (taosArrayGetSize(changeInfo.pUpdateNodeList) > 0) { + + // kill current active checkpoint transaction, since the transaction is vnode wide. + doKillActiveCheckpointTrans(pMnode); code = mndProcessVgroupChange(pMnode, &changeInfo); // keep the new vnode snapshot @@ -2498,29 +2498,26 @@ int32_t createStreamResetStatusTrans(SMnode* pMnode, SStreamObj* pStream) { return TSDB_CODE_ACTION_IN_PROGRESS; } -int32_t mndResetFromCheckpoint(SMnode* pMnode) { - // find the checkpoint trans id +int32_t doKillActiveCheckpointTrans(SMnode *pMnode) { int32_t transId = 0; + SSdb *pSdb = pMnode->pSdb; + STrans *pTrans = NULL; + void *pIter = NULL; - { - SSdb *pSdb = pMnode->pSdb; - STrans *pTrans = NULL; - void* pIter = NULL; - while (1) { - pIter = sdbFetch(pSdb, SDB_TRANS, pIter, (void **)&pTrans); - if (pIter == NULL) { - break; - } - - if (strncmp(pTrans->opername, MND_STREAM_CHECKPOINT_NAME, tListLen(pTrans->opername) - 1) == 0) { - transId = pTrans->id; - sdbRelease(pSdb, pTrans); - sdbCancelFetch(pSdb, pIter); - break; - } - - sdbRelease(pSdb, pTrans); + while (1) { + pIter = sdbFetch(pSdb, SDB_TRANS, pIter, (void **)&pTrans); + if (pIter == NULL) { + break; } + + if (strncmp(pTrans->opername, MND_STREAM_CHECKPOINT_NAME, tListLen(pTrans->opername) - 1) == 0) { + transId = pTrans->id; + sdbRelease(pSdb, pTrans); + sdbCancelFetch(pSdb, pIter); + break; + } + + sdbRelease(pSdb, pTrans); } if (transId == 0) { @@ -2528,8 +2525,16 @@ int32_t mndResetFromCheckpoint(SMnode* pMnode) { return TSDB_CODE_SUCCESS; } - STrans* pTrans = mndAcquireTrans(pMnode, transId); + pTrans = mndAcquireTrans(pMnode, transId); + mInfo("kill checkpoint trans:%d", transId); + mndKillTrans(pMnode, pTrans); + mndReleaseTrans(pMnode, pTrans); + return TSDB_CODE_SUCCESS; +} + +int32_t mndResetFromCheckpoint(SMnode* pMnode) { + doKillActiveCheckpointTrans(pMnode); // set all tasks status to be normal, refactor later to be stream level, instead of vnode level. SSdb *pSdb = pMnode->pSdb; @@ -2541,6 +2546,7 @@ int32_t mndResetFromCheckpoint(SMnode* pMnode) { break; } + // todo this transaction should exist be only one mDebug("stream:%s (0x%" PRIx64 ") reset checkpoint procedure, create reset trans", pStream->name, pStream->uid); int32_t code = createStreamResetStatusTrans(pMnode, pStream); if (code != TSDB_CODE_SUCCESS) { diff --git a/source/libs/stream/src/streamDispatch.c b/source/libs/stream/src/streamDispatch.c index e6378d309e..750f5d6a43 100644 --- a/source/libs/stream/src/streamDispatch.c +++ b/source/libs/stream/src/streamDispatch.c @@ -1116,16 +1116,16 @@ int32_t streamProcessDispatchRsp(SStreamTask* pTask, SStreamDispatchRsp* pRsp, i ASSERT(leftRsp >= 0); if (leftRsp > 0) { - stDebug( "s-task:%s recv dispatch rsp, msgId:%d from 0x%x(vgId:%d), downstream task input status:%d code:%d, waiting for %d rsp", - id, msgId, pRsp->downstreamTaskId, pRsp->downstreamNodeId, pRsp->inputStatus, code, leftRsp); + stDebug( "s-task:%s recv dispatch rsp, msgId:%d from 0x%x(vgId:%d), downstream task input status:%d code:%s, waiting for %d rsp", + id, msgId, pRsp->downstreamTaskId, pRsp->downstreamNodeId, pRsp->inputStatus, tstrerror(code), leftRsp); } else { stDebug( - "s-task:%s recv dispatch rsp, msgId:%d from 0x%x(vgId:%d), downstream task input status:%d code:%d, all rsp", - id, msgId, pRsp->downstreamTaskId, pRsp->downstreamNodeId, pRsp->inputStatus, code); + "s-task:%s recv dispatch rsp, msgId:%d from 0x%x(vgId:%d), downstream task input status:%d code:%s, all rsp", + id, msgId, pRsp->downstreamTaskId, pRsp->downstreamNodeId, pRsp->inputStatus, tstrerror(code)); } } else { - stDebug("s-task:%s recv fix-dispatch rsp, msgId:%d from 0x%x(vgId:%d), downstream task input status:%d code:%d", - id, msgId, pRsp->downstreamTaskId, pRsp->downstreamNodeId, pRsp->inputStatus, code); + stDebug("s-task:%s recv fix-dispatch rsp, msgId:%d from 0x%x(vgId:%d), downstream task input status:%d code:%s", + id, msgId, pRsp->downstreamTaskId, pRsp->downstreamNodeId, pRsp->inputStatus, tstrerror(code)); } ASSERT(leftRsp >= 0); From 21e98e358e327890af850af07a101cb08fac6332 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sat, 21 Oct 2023 02:41:03 +0800 Subject: [PATCH 17/41] fix(stream): clear the flag. --- source/libs/stream/src/streamTaskSm.c | 3 +++ 1 file changed, 3 insertions(+) diff --git a/source/libs/stream/src/streamTaskSm.c b/source/libs/stream/src/streamTaskSm.c index f42e6bbb3b..c3286407e4 100644 --- a/source/libs/stream/src/streamTaskSm.c +++ b/source/libs/stream/src/streamTaskSm.c @@ -311,6 +311,9 @@ void streamTaskResetStatus(SStreamTask* pTask) { pSM->current = StreamTaskStatusList[TASK_STATUS__UNINIT]; pSM->pActiveTrans = NULL; taosArrayClear(pSM->pWaitingEventList); + + // clear the downstream ready status + pTask->status.downstreamReady = 0; } void streamTaskSetStatusReady(SStreamTask* pTask) { From 74572dd6a76d76fc38e1e8d0d965c2a0395d310c Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sat, 21 Oct 2023 15:21:02 +0800 Subject: [PATCH 18/41] refactor: do some internal refactor. --- source/dnode/mnode/impl/src/mndStream.c | 156 ++++++++++++------------ 1 file changed, 80 insertions(+), 76 deletions(-) diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index 34041b26fb..2fcfdc9b71 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -42,22 +42,22 @@ typedef struct SNodeEntry { int64_t hbTimestamp; // second } SNodeEntry; -typedef struct SStreamExecNodeInfo { +typedef struct SStreamExecInfo { SArray *pNodeEntryList; int64_t ts; // snapshot ts int64_t activeCheckpoint; // active check point id SHashObj *pTaskMap; SArray *pTaskList; TdThreadMutex lock; -} SStreamExecNodeInfo; +} SStreamExecInfo; typedef struct SVgroupChangeInfo { SHashObj *pDBMap; SArray *pUpdateNodeList; // SArray } SVgroupChangeInfo; -static int32_t mndNodeCheckSentinel = 0; -static SStreamExecNodeInfo execNodeList; +static int32_t mndNodeCheckSentinel = 0; +static SStreamExecInfo execInfo; static int32_t mndStreamActionInsert(SSdb *pSdb, SStreamObj *pStream); static int32_t mndStreamActionDelete(SSdb *pSdb, SStreamObj *pStream); @@ -77,18 +77,17 @@ static int32_t mndBuildStreamCheckpointSourceReq2(void **pBuf, int32_t *pLen, in int64_t streamId, int32_t taskId); static int32_t mndProcessNodeCheck(SRpcMsg *pReq); static int32_t mndProcessNodeCheckReq(SRpcMsg *pMsg); +static SArray *extractNodeListFromStream(SMnode *pMnode); +static SArray *mndTakeVgroupSnapshot(SMnode *pMnode); -static SArray *extractNodeListFromStream(SMnode *pMnode); -static SArray *mndTakeVgroupSnapshot(SMnode *pMnode); static SVgroupChangeInfo mndFindChangedNodeInfo(SMnode *pMnode, const SArray *pPrevNodeList, const SArray *pNodeList); static STrans *doCreateTrans(SMnode *pMnode, SStreamObj *pStream, const char *name); static int32_t mndPersistTransLog(SStreamObj *pStream, STrans *pTrans); static void initTransAction(STransAction *pAction, void *pCont, int32_t contLen, int32_t msgType, const SEpSet *pEpset); static int32_t createStreamUpdateTrans(SStreamObj *pStream, SVgroupChangeInfo *pInfo, STrans *pTrans); - -static void removeStreamTasksInBuf(SStreamObj *pStream, SStreamExecNodeInfo *pExecNode); -static void keepStreamTasksInBuf(SStreamObj *pStream, SStreamExecNodeInfo *pExecNode); +static void removeStreamTasksInBuf(SStreamObj *pStream, SStreamExecInfo *pExecNode); +static void keepStreamTasksInBuf(SStreamObj *pStream, SStreamExecInfo *pExecNode); static int32_t removeExpirednodeEntryAndTask(SArray *pNodeSnapshot); static int32_t doKillActiveCheckpointTrans(SMnode *pMnode); @@ -130,18 +129,18 @@ int32_t mndInitStream(SMnode *pMnode) { mndAddShowRetrieveHandle(pMnode, TSDB_MGMT_TABLE_STREAM_TASKS, mndRetrieveStreamTask); mndAddShowFreeIterHandle(pMnode, TSDB_MGMT_TABLE_STREAM_TASKS, mndCancelGetNextStreamTask); - taosThreadMutexInit(&execNodeList.lock, NULL); - execNodeList.pTaskMap = taosHashInit(64, taosGetDefaultHashFunction(TSDB_DATA_TYPE_VARCHAR), true, HASH_NO_LOCK); - execNodeList.pTaskList = taosArrayInit(4, sizeof(STaskId)); + taosThreadMutexInit(&execInfo.lock, NULL); + execInfo.pTaskMap = taosHashInit(64, taosGetDefaultHashFunction(TSDB_DATA_TYPE_VARCHAR), true, HASH_NO_LOCK); + execInfo.pTaskList = taosArrayInit(4, sizeof(STaskId)); return sdbSetTable(pMnode->pSdb, table); } void mndCleanupStream(SMnode *pMnode) { - taosArrayDestroy(execNodeList.pTaskList); - taosHashCleanup(execNodeList.pTaskMap); - taosThreadMutexDestroy(&execNodeList.lock); - mDebug("mnd stream cleanup"); + taosArrayDestroy(execInfo.pTaskList); + taosHashCleanup(execInfo.pTaskMap); + taosThreadMutexDestroy(&execInfo.lock); + mDebug("mnd stream exec info cleanup"); } SSdbRaw *mndStreamActionEncode(SStreamObj *pStream) { @@ -848,10 +847,10 @@ static int32_t mndProcessCreateStreamReq(SRpcMsg *pReq) { mndTransDrop(pTrans); - taosThreadMutexLock(&execNodeList.lock); + taosThreadMutexLock(&execInfo.lock); mDebug("register to stream task node list"); - keepStreamTasksInBuf(&streamObj, &execNodeList); - taosThreadMutexUnlock(&execNodeList.lock); + keepStreamTasksInBuf(&streamObj, &execInfo); + taosThreadMutexUnlock(&execInfo.lock); code = TSDB_CODE_ACTION_IN_PROGRESS; @@ -883,9 +882,8 @@ static int32_t mndProcessStreamCheckpointTmr(SRpcMsg *pReq) { return 0; } - int64_t checkpointId = taosGetTimestampMs(); SMStreamDoCheckpointMsg *pMsg = rpcMallocCont(sizeof(SMStreamDoCheckpointMsg)); - pMsg->checkpointId = checkpointId; + pMsg->checkpointId = taosGetTimestampMs(); int32_t size = sizeof(SMStreamDoCheckpointMsg); SRpcMsg rpcMsg = {.msgType = TDMT_MND_STREAM_BEGIN_CHECKPOINT, .pCont = pMsg, .contLen = size}; @@ -1085,6 +1083,7 @@ static int32_t mndAddStreamCheckpointToTrans(STrans *pTrans, SStreamObj *pStream pStream->checkpointId = chkptId; pStream->checkpointFreq = taosGetTimestampMs(); pStream->currentTick = 0; + // 3. commit log: stream checkpoint info pStream->version = pStream->version + 1; @@ -1134,22 +1133,22 @@ static int32_t mndProcessStreamDoCheckpoint(SRpcMsg *pReq) { { // check if the node update happens or not int64_t ts = taosGetTimestampSec(); - if (execNodeList.pNodeEntryList == NULL || (taosArrayGetSize(execNodeList.pNodeEntryList) == 0)) { - if (execNodeList.pNodeEntryList != NULL) { - execNodeList.pNodeEntryList = taosArrayDestroy(execNodeList.pNodeEntryList); + if (execInfo.pNodeEntryList == NULL || (taosArrayGetSize(execInfo.pNodeEntryList) == 0)) { + if (execInfo.pNodeEntryList != NULL) { + execInfo.pNodeEntryList = taosArrayDestroy(execInfo.pNodeEntryList); } - execNodeList.pNodeEntryList = extractNodeListFromStream(pMnode); + execInfo.pNodeEntryList = extractNodeListFromStream(pMnode); } - if (taosArrayGetSize(execNodeList.pNodeEntryList) == 0) { + if (taosArrayGetSize(execInfo.pNodeEntryList) == 0) { mDebug("stream task node change checking done, no vgroups exist, do nothing"); - execNodeList.ts = ts; + execInfo.ts = ts; return 0; } - for(int32_t i = 0; i < taosArrayGetSize(execNodeList.pNodeEntryList); ++i) { - SNodeEntry* pNodeEntry = taosArrayGet(execNodeList.pNodeEntryList, i); + for(int32_t i = 0; i < taosArrayGetSize(execInfo.pNodeEntryList); ++i) { + SNodeEntry* pNodeEntry = taosArrayGet(execInfo.pNodeEntryList, i); if (pNodeEntry->stageUpdated) { mDebug("stream task not ready due to node update detected, checkpoint not issued"); return 0; @@ -1158,7 +1157,7 @@ static int32_t mndProcessStreamDoCheckpoint(SRpcMsg *pReq) { SArray *pNodeSnapshot = mndTakeVgroupSnapshot(pMnode); - SVgroupChangeInfo changeInfo = mndFindChangedNodeInfo(pMnode, execNodeList.pNodeEntryList, pNodeSnapshot); + SVgroupChangeInfo changeInfo = mndFindChangedNodeInfo(pMnode, execInfo.pNodeEntryList, pNodeSnapshot); bool nodeUpdated = (taosArrayGetSize(changeInfo.pUpdateNodeList) > 0); taosArrayDestroy(changeInfo.pUpdateNodeList); taosHashCleanup(changeInfo.pDBMap); @@ -1173,10 +1172,10 @@ static int32_t mndProcessStreamDoCheckpoint(SRpcMsg *pReq) { { // check if all tasks are in TASK_STATUS__READY status bool ready = true; - taosThreadMutexLock(&execNodeList.lock); - for (int32_t i = 0; i < taosArrayGetSize(execNodeList.pTaskList); ++i) { - STaskId *p = taosArrayGet(execNodeList.pTaskList, i); - STaskStatusEntry* pEntry = taosHashGet(execNodeList.pTaskMap, p, sizeof(*p)); + taosThreadMutexLock(&execInfo.lock); + for (int32_t i = 0; i < taosArrayGetSize(execInfo.pTaskList); ++i) { + STaskId *p = taosArrayGet(execInfo.pTaskList, i); + STaskStatusEntry* pEntry = taosHashGet(execInfo.pTaskMap, p, sizeof(*p)); if (pEntry == NULL) { continue; } @@ -1188,7 +1187,7 @@ static int32_t mndProcessStreamDoCheckpoint(SRpcMsg *pReq) { break; } } - taosThreadMutexUnlock(&execNodeList.lock); + taosThreadMutexUnlock(&execInfo.lock); if (!ready) { return 0; } @@ -1229,11 +1228,16 @@ static int32_t mndProcessStreamDoCheckpoint(SRpcMsg *pReq) { if (code == 0) { if (mndTransPrepare(pMnode, pTrans) != 0) { - mError("failed to prepre trans rebalance since %s", terrstr()); + mError("failed to prepare trans rebalance since %s", terrstr()); } } mndTransDrop(pTrans); + + // only one trans here + taosThreadMutexLock(&execInfo.lock); + execInfo.activeCheckpoint = checkpointId; + taosThreadMutexUnlock(&execInfo.lock); return code; } @@ -1311,7 +1315,7 @@ static int32_t mndProcessDropStreamReq(SRpcMsg *pReq) { return -1; } - removeStreamTasksInBuf(pStream, &execNodeList); + removeStreamTasksInBuf(pStream, &execInfo); SName name = {0}; tNameFromString(&name, dropReq.name, T_NAME_ACCT | T_NAME_DB); @@ -1562,7 +1566,7 @@ static int32_t mndRetrieveStreamTask(SRpcMsg *pReq, SShowObj *pShow, SSDataBlock char status[20 + VARSTR_HEADER_SIZE] = {0}; STaskId id = {.streamId = pTask->id.streamId, .taskId = pTask->id.taskId}; - STaskStatusEntry* pe = taosHashGet(execNodeList.pTaskMap, &id, sizeof(id)); + STaskStatusEntry* pe = taosHashGet(execInfo.pTaskMap, &id, sizeof(id)); if (pe == NULL) { continue; } @@ -2196,12 +2200,12 @@ static void doExtractTasksFromStream(SMnode *pMnode) { break; } - keepStreamTasksInBuf(pStream, &execNodeList); + keepStreamTasksInBuf(pStream, &execInfo); sdbRelease(pSdb, pStream); } } -static int32_t doRemoveFromTask(SStreamExecNodeInfo* pExecNode, STaskId* pRemovedId) { +static int32_t doRemoveFromTask(SStreamExecInfo* pExecNode, STaskId* pRemovedId) { void *p = taosHashGet(pExecNode->pTaskMap, pRemovedId, sizeof(*pRemovedId)); if (p != NULL) { @@ -2236,10 +2240,10 @@ static bool taskNodeExists(SArray* pList, int32_t nodeId) { int32_t removeExpirednodeEntryAndTask(SArray *pNodeSnapshot) { SArray* pRemoveTaskList = taosArrayInit(4, sizeof(STaskId)); - int32_t numOfTask = taosArrayGetSize(execNodeList.pTaskList); + int32_t numOfTask = taosArrayGetSize(execInfo.pTaskList); for(int32_t i = 0; i < numOfTask; ++i) { - STaskId* pId = taosArrayGet(execNodeList.pTaskList, i); - STaskStatusEntry* pEntry = taosHashGet(execNodeList.pTaskMap, pId, sizeof(*pId)); + STaskId* pId = taosArrayGet(execInfo.pTaskList, i); + STaskStatusEntry* pEntry = taosHashGet(execInfo.pTaskMap, pId, sizeof(*pId)); bool existed = taskNodeExists(pNodeSnapshot, pEntry->nodeId); if (!existed) { @@ -2249,16 +2253,16 @@ int32_t removeExpirednodeEntryAndTask(SArray *pNodeSnapshot) { for(int32_t i = 0; i < taosArrayGetSize(pRemoveTaskList); ++i) { STaskId* pId = taosArrayGet(pRemoveTaskList, i); - doRemoveFromTask(&execNodeList, pId); + doRemoveFromTask(&execInfo, pId); } mDebug("remove invalid stream tasks:%d, remain:%d", (int32_t)taosArrayGetSize(pRemoveTaskList), - (int32_t) taosArrayGetSize(execNodeList.pTaskList)); + (int32_t) taosArrayGetSize(execInfo.pTaskList)); int32_t size = taosArrayGetSize(pNodeSnapshot); SArray* pValidNodeEntryList = taosArrayInit(4, sizeof(SNodeEntry)); - for(int32_t i = 0; i < taosArrayGetSize(execNodeList.pNodeEntryList); ++i) { - SNodeEntry* p = taosArrayGet(execNodeList.pNodeEntryList, i); + for(int32_t i = 0; i < taosArrayGetSize(execInfo.pNodeEntryList); ++i) { + SNodeEntry* p = taosArrayGet(execInfo.pNodeEntryList, i); for(int32_t j = 0; j < size; ++j) { SNodeEntry* pEntry = taosArrayGet(pNodeSnapshot, j); @@ -2269,8 +2273,8 @@ int32_t removeExpirednodeEntryAndTask(SArray *pNodeSnapshot) { } } - execNodeList.pNodeEntryList = taosArrayDestroy(execNodeList.pNodeEntryList); - execNodeList.pNodeEntryList = pValidNodeEntryList; + execInfo.pNodeEntryList = taosArrayDestroy(execInfo.pNodeEntryList); + execInfo.pNodeEntryList = pValidNodeEntryList; taosArrayDestroy(pRemoveTaskList); return 0; @@ -2289,26 +2293,26 @@ static int32_t mndProcessNodeCheckReq(SRpcMsg *pMsg) { int64_t ts = taosGetTimestampSec(); SMnode *pMnode = pMsg->info.node; - if (execNodeList.pNodeEntryList == NULL || (taosArrayGetSize(execNodeList.pNodeEntryList) == 0)) { - if (execNodeList.pNodeEntryList != NULL) { - execNodeList.pNodeEntryList = taosArrayDestroy(execNodeList.pNodeEntryList); + if (execInfo.pNodeEntryList == NULL || (taosArrayGetSize(execInfo.pNodeEntryList) == 0)) { + if (execInfo.pNodeEntryList != NULL) { + execInfo.pNodeEntryList = taosArrayDestroy(execInfo.pNodeEntryList); } - execNodeList.pNodeEntryList = extractNodeListFromStream(pMnode); + execInfo.pNodeEntryList = extractNodeListFromStream(pMnode); } - if (taosArrayGetSize(execNodeList.pNodeEntryList) == 0) { + if (taosArrayGetSize(execInfo.pNodeEntryList) == 0) { mDebug("end to do stream task node change checking, no vgroup exists, do nothing"); - execNodeList.ts = ts; + execInfo.ts = ts; atomic_store_32(&mndNodeCheckSentinel, 0); return 0; } SArray *pNodeSnapshot = mndTakeVgroupSnapshot(pMnode); - taosThreadMutexLock(&execNodeList.lock); + taosThreadMutexLock(&execInfo.lock); removeExpirednodeEntryAndTask(pNodeSnapshot); - SVgroupChangeInfo changeInfo = mndFindChangedNodeInfo(pMnode, execNodeList.pNodeEntryList, pNodeSnapshot); + SVgroupChangeInfo changeInfo = mndFindChangedNodeInfo(pMnode, execInfo.pNodeEntryList, pNodeSnapshot); if (taosArrayGetSize(changeInfo.pUpdateNodeList) > 0) { // kill current active checkpoint transaction, since the transaction is vnode wide. @@ -2318,16 +2322,16 @@ static int32_t mndProcessNodeCheckReq(SRpcMsg *pMsg) { // keep the new vnode snapshot if (code == TSDB_CODE_SUCCESS || code == TSDB_CODE_ACTION_IN_PROGRESS) { mDebug("create trans successfully, update cached node list"); - taosArrayDestroy(execNodeList.pNodeEntryList); - execNodeList.pNodeEntryList = pNodeSnapshot; - execNodeList.ts = ts; + taosArrayDestroy(execInfo.pNodeEntryList); + execInfo.pNodeEntryList = pNodeSnapshot; + execInfo.ts = ts; } } else { mDebug("no update found in nodeList"); taosArrayDestroy(pNodeSnapshot); } - taosThreadMutexUnlock(&execNodeList.lock); + taosThreadMutexUnlock(&execInfo.lock); taosArrayDestroy(changeInfo.pUpdateNodeList); taosHashCleanup(changeInfo.pDBMap); @@ -2359,7 +2363,7 @@ static int32_t mndProcessNodeCheck(SRpcMsg *pReq) { return 0; } -void keepStreamTasksInBuf(SStreamObj *pStream, SStreamExecNodeInfo *pExecNode) { +void keepStreamTasksInBuf(SStreamObj *pStream, SStreamExecInfo *pExecNode) { int32_t level = taosArrayGetSize(pStream->tasks); for (int32_t i = 0; i < level; i++) { @@ -2384,7 +2388,7 @@ void keepStreamTasksInBuf(SStreamObj *pStream, SStreamExecNodeInfo *pExecNode) { } } -void removeStreamTasksInBuf(SStreamObj* pStream, SStreamExecNodeInfo * pExecNode) { +void removeStreamTasksInBuf(SStreamObj* pStream, SStreamExecInfo * pExecNode) { int32_t level = taosArrayGetSize(pStream->tasks); for (int32_t i = 0; i < level; i++) { SArray *pLevel = taosArrayGetP(pStream->tasks, i); @@ -2577,24 +2581,24 @@ int32_t mndProcessStreamHb(SRpcMsg *pReq) { mTrace("receive stream-meta hb from vgId:%d, active numOfTasks:%d", req.vgId, req.numOfTasks); - taosThreadMutexLock(&execNodeList.lock); - int32_t numOfExisted = taosHashGetSize(execNodeList.pTaskMap); + taosThreadMutexLock(&execInfo.lock); + int32_t numOfExisted = taosHashGetSize(execInfo.pTaskMap); if (numOfExisted == 0) { doExtractTasksFromStream(pMnode); } for (int32_t i = 0; i < req.numOfTasks; ++i) { STaskStatusEntry *p = taosArrayGet(req.pTaskStatus, i); - STaskStatusEntry *pEntry = taosHashGet(execNodeList.pTaskMap, &p->id, sizeof(p->id)); + STaskStatusEntry *pEntry = taosHashGet(execInfo.pTaskMap, &p->id, sizeof(p->id)); if (pEntry == NULL) { mError("s-task:0x%" PRIx64 " not found in mnode task list", p->id.taskId); continue; } if (p->stage != pEntry->stage && pEntry->stage != -1) { - int32_t numOfNodes = taosArrayGetSize(execNodeList.pNodeEntryList); + int32_t numOfNodes = taosArrayGetSize(execInfo.pNodeEntryList); for(int32_t j = 0; j < numOfNodes; ++j) { - SNodeEntry* pNodeEntry = taosArrayGet(execNodeList.pNodeEntryList, j); + SNodeEntry* pNodeEntry = taosArrayGet(execInfo.pNodeEntryList, j); if (pNodeEntry->nodeId == pEntry->nodeId) { mInfo("vgId:%d stage updated, from %d to %d, nodeUpdate trigger by s-task:0x%" PRIx64, pEntry->nodeId, pEntry->stage, p->stage, pEntry->id.taskId); @@ -2628,16 +2632,16 @@ int32_t mndProcessStreamHb(SRpcMsg *pReq) { // current checkpoint is failed, rollback from the checkpoint trans // kill the checkpoint trans and then set all tasks status to be normal if (checkpointFailed && activeCheckpointId != 0) { - if (execNodeList.activeCheckpoint != activeCheckpointId) { - mInfo("checkpointId:%"PRId64" failed, issue task-reset trans to reset all tasks status", activeCheckpointId); - execNodeList.activeCheckpoint = activeCheckpointId; - mndResetFromCheckpoint(pMnode); - } else { - mDebug("checkpoint:%"PRId64" reset has issued already, ignore it", activeCheckpointId); - } + ASSERT(execInfo.activeCheckpoint == activeCheckpointId); + mInfo("checkpointId:%" PRId64 " failed, issue task-reset trans to reset all tasks status", activeCheckpointId); + // execInfo.activeCheckpoint = activeCheckpointId; + mndResetFromCheckpoint(pMnode); + // } else { + // mDebug("checkpoint:%"PRId64" reset has issued already, ignore it", activeCheckpointId); + // } } - taosThreadMutexUnlock(&execNodeList.lock); + taosThreadMutexUnlock(&execInfo.lock); taosArrayDestroy(req.pTaskStatus); return TSDB_CODE_SUCCESS; From 06cd67e26eb36cf3babdc8c119efe4860a5f6fb0 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sat, 21 Oct 2023 16:20:03 +0800 Subject: [PATCH 19/41] refactor: do some internal refactor. --- include/util/tqueue.h | 8 ++++++-- source/libs/stream/src/streamQueue.c | 3 +-- source/util/src/tqueue.c | 16 ++++++++++++++++ 3 files changed, 23 insertions(+), 4 deletions(-) diff --git a/include/util/tqueue.h b/include/util/tqueue.h index 503d15e793..9f09bd2930 100644 --- a/include/util/tqueue.h +++ b/include/util/tqueue.h @@ -101,6 +101,9 @@ struct STaosQall { STaosQnode *current; STaosQnode *start; int32_t numOfItems; + int64_t memOfItems; + int32_t unAccessedNumOfItems; + int64_t unAccessMemOfItems; }; STaosQueue *taosOpenQueue(); @@ -123,6 +126,9 @@ int32_t taosReadAllQitems(STaosQueue *queue, STaosQall *qall); int32_t taosGetQitem(STaosQall *qall, void **ppItem); void taosResetQitems(STaosQall *qall); int32_t taosQallItemSize(STaosQall *qall); +int64_t taosQallMemSize(STaosQall *qll); +int64_t taosQallUnAccessedItemSize(STaosQall *qall); +int64_t taosQallUnAccessedMemSize(STaosQall *qall); STaosQset *taosOpenQset(); void taosCloseQset(STaosQset *qset); @@ -135,8 +141,6 @@ int32_t taosReadQitemFromQset(STaosQset *qset, void **ppItem, SQueueInfo *qinfo) int32_t taosReadAllQitemsFromQset(STaosQset *qset, STaosQall *qall, SQueueInfo *qinfo); void taosResetQsetThread(STaosQset *qset, void *pItem); -extern int64_t tsRpcQueueMemoryAllowed; - #ifdef __cplusplus } #endif diff --git a/source/libs/stream/src/streamQueue.c b/source/libs/stream/src/streamQueue.c index 70a065c22e..7305e9db83 100644 --- a/source/libs/stream/src/streamQueue.c +++ b/source/libs/stream/src/streamQueue.c @@ -119,9 +119,8 @@ int32_t streamQueueGetNumOfItems(const SStreamQueue* pQueue) { return numOfItems1 + numOfItems2; } -// todo: fix it: data in Qall is not included here int32_t streamQueueGetItemSize(const SStreamQueue* pQueue) { - return taosQueueMemorySize(pQueue->pQueue); + return taosQueueMemorySize(pQueue->pQueue) + taosQallMemSize(pQueue->qall); } int32_t streamQueueItemGetSize(const SStreamQueueItem* pItem) { diff --git a/source/util/src/tqueue.c b/source/util/src/tqueue.c index 81350dddd2..1dfdd637b6 100644 --- a/source/util/src/tqueue.c +++ b/source/util/src/tqueue.c @@ -242,6 +242,11 @@ int32_t taosReadAllQitems(STaosQueue *queue, STaosQall *qall) { qall->current = queue->head; qall->start = queue->head; qall->numOfItems = queue->numOfItems; + qall->memOfItems = queue->memOfItems; + + qall->unAccessedNumOfItems = queue->numOfItems; + qall->unAccessMemOfItems = queue->memOfItems; + numOfItems = qall->numOfItems; queue->head = NULL; @@ -274,6 +279,10 @@ int32_t taosGetQitem(STaosQall *qall, void **ppItem) { if (pNode) { *ppItem = pNode->item; num = 1; + + qall->unAccessedNumOfItems -= 1; + qall->unAccessMemOfItems -= pNode->dataSize; + uTrace("item:%p is fetched", *ppItem); } else { *ppItem = NULL; @@ -449,6 +458,8 @@ int32_t taosReadAllQitemsFromQset(STaosQset *qset, STaosQall *qall, SQueueInfo * qall->current = queue->head; qall->start = queue->head; qall->numOfItems = queue->numOfItems; + qall->memOfItems = queue->memOfItems; + code = qall->numOfItems; qinfo->ahandle = queue->ahandle; qinfo->fp = queue->itemsFp; @@ -476,6 +487,11 @@ int32_t taosReadAllQitemsFromQset(STaosQset *qset, STaosQall *qall, SQueueInfo * } int32_t taosQallItemSize(STaosQall *qall) { return qall->numOfItems; } +int64_t taosQallMemSize(STaosQall *qall) { return qall->memOfItems; } + +int64_t taosQallUnAccessedItemSize(STaosQall *qall) {return qall->unAccessedNumOfItems;} +int64_t taosQallUnAccessedMemSize(STaosQall *qall) {return qall->unAccessMemOfItems;} + void taosResetQitems(STaosQall *qall) { qall->current = qall->start; } int32_t taosGetQueueNumber(STaosQset *qset) { return qset->numOfQueues; } From 4826b34a2bb5d06ab4f6a0f58f3592dd7c447618 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sat, 21 Oct 2023 16:32:44 +0800 Subject: [PATCH 20/41] refactor: do some internal refactor. --- source/libs/stream/src/streamQueue.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/libs/stream/src/streamQueue.c b/source/libs/stream/src/streamQueue.c index 7305e9db83..e4c13699c9 100644 --- a/source/libs/stream/src/streamQueue.c +++ b/source/libs/stream/src/streamQueue.c @@ -120,7 +120,7 @@ int32_t streamQueueGetNumOfItems(const SStreamQueue* pQueue) { } int32_t streamQueueGetItemSize(const SStreamQueue* pQueue) { - return taosQueueMemorySize(pQueue->pQueue) + taosQallMemSize(pQueue->qall); + return taosQueueMemorySize(pQueue->pQueue) + taosQallUnAccessedMemSize(pQueue->qall); } int32_t streamQueueItemGetSize(const SStreamQueueItem* pItem) { From 825195b351285713be4258a91c136542e2885a8c Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sat, 21 Oct 2023 16:38:46 +0800 Subject: [PATCH 21/41] refactor: do some internal refactor. --- source/dnode/mnode/impl/src/mndStream.c | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index 2fcfdc9b71..28c22d5248 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -2632,9 +2632,12 @@ int32_t mndProcessStreamHb(SRpcMsg *pReq) { // current checkpoint is failed, rollback from the checkpoint trans // kill the checkpoint trans and then set all tasks status to be normal if (checkpointFailed && activeCheckpointId != 0) { - ASSERT(execInfo.activeCheckpoint == activeCheckpointId); + // if the execInfo.activeCheckpoint == 0, the checkpoint is restoring from wal + if (execInfo.activeCheckpoint != 0) { + ASSERT(execInfo.activeCheckpoint == activeCheckpointId); + } + mInfo("checkpointId:%" PRId64 " failed, issue task-reset trans to reset all tasks status", activeCheckpointId); - // execInfo.activeCheckpoint = activeCheckpointId; mndResetFromCheckpoint(pMnode); // } else { // mDebug("checkpoint:%"PRId64" reset has issued already, ignore it", activeCheckpointId); From 059e94e428ac1b88e46a5cd42dc963d68e5a2914 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sun, 22 Oct 2023 01:25:00 +0800 Subject: [PATCH 22/41] fix(stream): clear flag for checkpoint. --- include/libs/stream/tstream.h | 2 +- source/libs/stream/src/streamMeta.c | 3 +++ source/libs/stream/src/streamStart.c | 4 ++-- 3 files changed, 6 insertions(+), 3 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 0ca401e3a4..54ca7f1566 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -38,7 +38,7 @@ extern "C" { #define TASK_DOWNSTREAM_READY 0x0 #define TASK_DOWNSTREAM_NOT_READY 0x1 #define TASK_DOWNSTREAM_NOT_LEADER 0x2 -#define TASK_SELF_NEW_STAGE 0x3 +#define TASK_UPSTREAM_NEW_STAGE 0x3 #define NODE_ROLE_UNINIT 0x1 #define NODE_ROLE_LEADER 0x2 diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index a1ed9a1458..4307f74709 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -299,6 +299,9 @@ void streamMetaClear(SStreamMeta* pMeta) { taosArrayClear(pMeta->chkpInUse); pMeta->numOfStreamTasks = 0; pMeta->numOfPausedTasks = 0; + pMeta->chkptNotReadyTasks = 0; + + streamMetaResetStartInfo(&pMeta->startInfo); } void streamMetaClose(SStreamMeta* pMeta) { diff --git a/source/libs/stream/src/streamStart.c b/source/libs/stream/src/streamStart.c index b52d9177b7..9ebd617be0 100644 --- a/source/libs/stream/src/streamStart.c +++ b/source/libs/stream/src/streamStart.c @@ -252,7 +252,7 @@ int32_t streamTaskCheckStatus(SStreamTask* pTask, int32_t upstreamTaskId, int32_ } if (pInfo->stage != stage) { - return TASK_SELF_NEW_STAGE; + return TASK_UPSTREAM_NEW_STAGE; } else if (pTask->status.downstreamReady != 1) { return TASK_DOWNSTREAM_NOT_READY; } else { @@ -396,7 +396,7 @@ int32_t streamProcessCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* pRs "roll-back needed", id, pRsp->downstreamTaskId, pRsp->downstreamNodeId); } else { - if (pRsp->status == TASK_SELF_NEW_STAGE) { + if (pRsp->status == TASK_UPSTREAM_NEW_STAGE) { stError( "s-task:%s vnode-transfer/leader-change/restart detected, old stage:%d, current stage:%d, continue check " "till downstream nodeUpdate", From a882c639bebb0fc767979adba74edede16ea13a4 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sun, 22 Oct 2023 16:16:59 +0800 Subject: [PATCH 23/41] fix(stream): remove invalid assert. --- source/dnode/mnode/impl/src/mndStream.c | 9 +-------- 1 file changed, 1 insertion(+), 8 deletions(-) diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index 28c22d5248..194c1021f6 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -2633,15 +2633,8 @@ int32_t mndProcessStreamHb(SRpcMsg *pReq) { // kill the checkpoint trans and then set all tasks status to be normal if (checkpointFailed && activeCheckpointId != 0) { // if the execInfo.activeCheckpoint == 0, the checkpoint is restoring from wal - if (execInfo.activeCheckpoint != 0) { - ASSERT(execInfo.activeCheckpoint == activeCheckpointId); - } - - mInfo("checkpointId:%" PRId64 " failed, issue task-reset trans to reset all tasks status", activeCheckpointId); + mInfo("checkpointId:%" PRId64 " failed, issue task-reset trans to reset all tasks status", execInfo.activeCheckpoint); mndResetFromCheckpoint(pMnode); - // } else { - // mDebug("checkpoint:%"PRId64" reset has issued already, ignore it", activeCheckpointId); - // } } taosThreadMutexUnlock(&execInfo.lock); From 7a8b046a358c96721e5ce03bdd0acb5b390c29e2 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 24 Oct 2023 09:40:38 +0800 Subject: [PATCH 24/41] fix(sma): fix bugs in sma and do some internal refactor. --- source/dnode/vnode/src/inc/tq.h | 6 + source/dnode/vnode/src/sma/smaTimeRange.c | 157 +++++------------- source/dnode/vnode/src/tq/tq.c | 1 - source/dnode/vnode/src/tq/tqSink.c | 150 ++++++++--------- source/libs/stream/src/stream.c | 2 +- source/libs/stream/src/streamBackendRocksdb.c | 6 - source/libs/stream/src/streamStart.c | 3 +- tests/pytest/util/dnodes.py | 1 + tests/system-test/0-others/timeRangeWise.py | 3 +- tests/system-test/test.py | 2 +- 10 files changed, 118 insertions(+), 213 deletions(-) diff --git a/source/dnode/vnode/src/inc/tq.h b/source/dnode/vnode/src/inc/tq.h index 9aed87d811..84f3b3e85a 100644 --- a/source/dnode/vnode/src/inc/tq.h +++ b/source/dnode/vnode/src/inc/tq.h @@ -170,6 +170,12 @@ int32_t tqDoSendDataRsp(const SRpcHandleInfo* pRpcHandleInfo, const SMqDataRsp* int32_t type, int64_t sver, int64_t ever); int32_t tqInitDataRsp(SMqDataRsp* pRsp, STqOffsetVal pOffset); void tqUpdateNodeStage(STQ* pTq, bool isLeader); +int32_t setDstTableDataPayload(uint64_t suid, const STSchema* pTSchema, int32_t blockIndex, SSDataBlock* pDataBlock, + SSubmitTbData* pTableData, const char* id); +int32_t doMergeExistedRows(SSubmitTbData* pExisted, const SSubmitTbData* pNew, const char* id); + +SVCreateTbReq* buildAutoCreateTableReq(const char* stbFullName, int64_t suid, int32_t numOfCols, + SSDataBlock* pDataBlock); #ifdef __cplusplus } diff --git a/source/dnode/vnode/src/sma/smaTimeRange.c b/source/dnode/vnode/src/sma/smaTimeRange.c index 0c37008344..f18843bc35 100644 --- a/source/dnode/vnode/src/sma/smaTimeRange.c +++ b/source/dnode/vnode/src/sma/smaTimeRange.c @@ -35,8 +35,8 @@ int32_t tdProcessTSmaInsert(SSma *pSma, int64_t indexUid, const char *msg) { return code; } -int32_t tdProcessTSmaCreate(SSma *pSma, int64_t version, const char *msg) { - int32_t code = tdProcessTSmaCreateImpl(pSma, version, msg); +int32_t tdProcessTSmaCreate(SSma *pSma, int64_t ver, const char *msg) { + int32_t code = tdProcessTSmaCreateImpl(pSma, ver, msg); return code; } @@ -109,7 +109,7 @@ _exit: * @param pMsg * @return int32_t */ -static int32_t tdProcessTSmaCreateImpl(SSma *pSma, int64_t version, const char *pMsg) { +static int32_t tdProcessTSmaCreateImpl(SSma *pSma, int64_t ver, const char *pMsg) { int32_t code = 0; int32_t lino = 0; SSmaCfg *pCfg = (SSmaCfg *)pMsg; @@ -118,7 +118,7 @@ static int32_t tdProcessTSmaCreateImpl(SSma *pSma, int64_t version, const char * if (TD_VID(pSma->pVnode) == pCfg->dstVgId) { // create tsma meta in dstVgId - if (metaCreateTSma(SMA_META(pSma), version, pCfg) < 0) { + if (metaCreateTSma(SMA_META(pSma), ver, pCfg) < 0) { code = terrno; TSDB_CHECK_CODE(code, lino, _exit); } @@ -130,7 +130,7 @@ static int32_t tdProcessTSmaCreateImpl(SSma *pSma, int64_t version, const char * pReq.schemaRow = pCfg->schemaRow; pReq.schemaTag = pCfg->schemaTag; - if (metaCreateSTable(SMA_META(pSma), version, &pReq) < 0) { + if (metaCreateSTable(SMA_META(pSma), ver, &pReq) < 0) { code = terrno; TSDB_CHECK_CODE(code, lino, _exit); } @@ -154,9 +154,8 @@ _exit: return code; } -int32_t smaBlockToSubmit(SVnode *pVnode, const SArray *pBlocks, const STSchema *pTSchema, - SSchemaWrapper *pTagSchemaWrapper, bool createTb, int64_t suid, const char *stbFullName, - SBatchDeleteReq *pDeleteReq, void **ppData, int32_t *pLen) { +int32_t smaBlockToSubmit(SVnode *pVnode, const SArray *pBlocks, const STSchema *pTSchema, bool createTb, int64_t suid, + const char *stbFullName, SBatchDeleteReq *pDeleteReq, void **ppData, int32_t *pLen) { int32_t code = 0; int32_t lino = 0; void *pBuf = NULL; @@ -166,82 +165,23 @@ int32_t smaBlockToSubmit(SVnode *pVnode, const SArray *pBlocks, const STSchema * SArray *createTbArray = NULL; SArray *pVals = NULL; - int32_t sz = taosArrayGetSize(pBlocks); + int32_t numOfBlocks = taosArrayGetSize(pBlocks); tagArray = taosArrayInit(1, sizeof(STagVal)); - createTbArray = taosArrayInit(sz, POINTER_BYTES); + createTbArray = taosArrayInit(numOfBlocks, POINTER_BYTES); pReq = taosMemoryCalloc(1, sizeof(SSubmitReq2)); pReq->aSubmitTbData = taosArrayInit(1, sizeof(SSubmitTbData)); - if(!tagArray || !createTbArray || !pReq || !pReq->aSubmitTbData) { + if (!tagArray || !createTbArray || !pReq || !pReq->aSubmitTbData) { code = terrno == TSDB_CODE_SUCCESS ? TSDB_CODE_OUT_OF_MEMORY : terrno; TSDB_CHECK_CODE(code, lino, _exit); } - // create table req - if (createTb) { - for (int32_t i = 0; i < sz; ++i) { - SSDataBlock *pDataBlock = taosArrayGet(pBlocks, i); - SVCreateTbReq *pCreateTbReq = NULL; - if (pDataBlock->info.type == STREAM_DELETE_RESULT) { - taosArrayPush(createTbArray, &pCreateTbReq); - continue; - } - - if (!(pCreateTbReq = taosMemoryCalloc(1, sizeof(SVCreateStbReq)))) { - code = TSDB_CODE_OUT_OF_MEMORY; - TSDB_CHECK_CODE(code, lino, _exit); - }; - - // don't move to the end of loop as to destroy in the end of func when error occur - taosArrayPush(createTbArray, &pCreateTbReq); - - // set const - pCreateTbReq->flags = 0; - pCreateTbReq->type = TSDB_CHILD_TABLE; - pCreateTbReq->ctb.suid = suid; - - // set super table name - SName name = {0}; - tNameFromString(&name, stbFullName, T_NAME_ACCT | T_NAME_DB | T_NAME_TABLE); - pCreateTbReq->ctb.stbName = taosStrdup((char *)tNameGetTableName(&name)); // taosStrdup(stbFullName); - - // set tag content - taosArrayClear(tagArray); - STagVal tagVal = { - .cid = taosArrayGetSize(pDataBlock->pDataBlock) + 1, - .type = TSDB_DATA_TYPE_UBIGINT, - .i64 = (int64_t)pDataBlock->info.id.groupId, - }; - taosArrayPush(tagArray, &tagVal); - pCreateTbReq->ctb.tagNum = taosArrayGetSize(tagArray); - - STag *pTag = NULL; - tTagNew(tagArray, 1, false, &pTag); - if (pTag == NULL) { - code = TSDB_CODE_OUT_OF_MEMORY; - TSDB_CHECK_CODE(code, lino, _exit); - } - pCreateTbReq->ctb.pTag = (uint8_t *)pTag; - - // set tag name - SArray *tagName = taosArrayInit(1, TSDB_COL_NAME_LEN); - char tagNameStr[TSDB_COL_NAME_LEN] = {0}; - strcpy(tagNameStr, "group_id"); - taosArrayPush(tagName, tagNameStr); - pCreateTbReq->ctb.tagName = tagName; - - // set table name - if (pDataBlock->info.parTbName[0]) { - pCreateTbReq->name = taosStrdup(pDataBlock->info.parTbName); - } else { - pCreateTbReq->name = buildCtbNameByGroupId(stbFullName, pDataBlock->info.id.groupId); - } - } - } + SHashObj *pTableIndexMap = + taosHashInit(numOfBlocks, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BIGINT), false, HASH_NO_LOCK); // SSubmitTbData req - for (int32_t i = 0; i < sz; ++i) { + for (int32_t i = 0; i < numOfBlocks; ++i) { SSDataBlock *pDataBlock = taosArrayGet(pBlocks, i); if (pDataBlock->info.type == STREAM_DELETE_RESULT) { pDeleteReq->suid = suid; @@ -250,57 +190,38 @@ int32_t smaBlockToSubmit(SVnode *pVnode, const SArray *pBlocks, const STSchema * continue; } - int32_t rows = pDataBlock->info.rows; + SSubmitTbData tbData = {.suid = suid, .uid = 0, .sver = pTSchema->version, .flags = SUBMIT_REQ_AUTO_CREATE_TABLE,}; + int32_t cid = taosArrayGetSize(pDataBlock->pDataBlock) + 1; - SSubmitTbData tbData = {0}; + tbData.pCreateTbReq = + buildAutoCreateTableReq(stbFullName, suid, taosArrayGetSize(pDataBlock->pDataBlock) + 1, pDataBlock); - if (!(tbData.aRowP = taosArrayInit(rows, sizeof(SRow *)))) { - code = terrno; - TSDB_CHECK_CODE(code, lino, _exit); - } - tbData.suid = suid; - tbData.uid = 0; // uid is assigned by vnode - tbData.sver = pTSchema->version; + { + uint64_t groupId = pDataBlock->info.id.groupId; - if (createTb) { - tbData.pCreateTbReq = taosArrayGetP(createTbArray, i); - if (tbData.pCreateTbReq) tbData.flags = SUBMIT_REQ_AUTO_CREATE_TABLE; - } + int32_t *index = taosHashGet(pTableIndexMap, &groupId, sizeof(groupId)); + if (index == NULL) { // no data yet, append it + code = setDstTableDataPayload(suid, pTSchema, i, pDataBlock, &tbData, ""); + if (code != TSDB_CODE_SUCCESS) { + continue; + } - if (!pVals && !(pVals = taosArrayInit(pTSchema->numOfCols, sizeof(SColVal)))) { - taosArrayDestroy(tbData.aRowP); - code = terrno; - TSDB_CHECK_CODE(code, lino, _exit); - } + taosArrayPush(pReq->aSubmitTbData, &tbData); - for (int32_t j = 0; j < rows; ++j) { - taosArrayClear(pVals); - for (int32_t k = 0; k < pTSchema->numOfCols; k++) { - const STColumn *pCol = &pTSchema->columns[k]; - SColumnInfoData *pColData = taosArrayGet(pDataBlock->pDataBlock, k); - if (colDataIsNull_s(pColData, j)) { - SColVal cv = COL_VAL_NULL(pCol->colId, pCol->type); - taosArrayPush(pVals, &cv); - } else { - void *data = colDataGetData(pColData, j); - if (IS_STR_DATA_TYPE(pCol->type)) { - SValue sv = (SValue){.nData = varDataLen(data), .pData = varDataVal(data)}; // address copy, no value - SColVal cv = COL_VAL_VALUE(pCol->colId, pCol->type, sv); - taosArrayPush(pVals, &cv); - } else { - SValue sv; - memcpy(&sv.val, data, tDataTypes[pCol->type].bytes); - SColVal cv = COL_VAL_VALUE(pCol->colId, pCol->type, sv); - taosArrayPush(pVals, &cv); - } + int32_t size = (int32_t)taosArrayGetSize(pReq->aSubmitTbData) - 1; + taosHashPut(pTableIndexMap, &groupId, sizeof(groupId), &size, sizeof(size)); + } else { + code = setDstTableDataPayload(suid, pTSchema, i, pDataBlock, &tbData, ""); + if (code != TSDB_CODE_SUCCESS) { + continue; + } + + SSubmitTbData *pExisted = taosArrayGet(pReq->aSubmitTbData, *index); + code = doMergeExistedRows(pExisted, &tbData, "id"); + if (code != TSDB_CODE_SUCCESS) { + continue; } } - SRow *pRow = NULL; - if ((code = tRowBuild(pVals, (STSchema *)pTSchema, &pRow)) < 0) { - tDestroySubmitTbData(&tbData, TSDB_MSG_FLG_ENCODE); - TSDB_CHECK_CODE(code, lino, _exit); - } - taosArrayPush(tbData.aRowP, &pRow); } taosArrayPush(pReq->aSubmitTbData, &tbData); @@ -442,7 +363,7 @@ static int32_t tdProcessTSmaInsertImpl(SSma *pSma, int64_t indexUid, const char void *pSubmitReq = NULL; int32_t contLen = 0; - code = smaBlockToSubmit(pSma->pVnode, (const SArray *)msg, pTsmaStat->pTSchema, &pTsmaStat->pTSma->schemaTag, true, + code = smaBlockToSubmit(pSma->pVnode, (const SArray *)msg, pTsmaStat->pTSchema, true, pTsmaStat->pTSma->dstTbUid, pTsmaStat->pTSma->dstTbName, &deleteReq, &pSubmitReq, &contLen); TSDB_CHECK_CODE(code, lino, _exit); diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index f9abb5f2c9..7c67f3cd9b 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1831,7 +1831,6 @@ int32_t tqProcessTaskUpdateReq(STQ* pTq, SRpcMsg* pMsg) { char* msg = POINTER_SHIFT(pMsg->pCont, sizeof(SMsgHead)); int32_t len = pMsg->contLen - sizeof(SMsgHead); SRpcMsg rsp = {.info = pMsg->info, .code = TSDB_CODE_SUCCESS}; - bool allStopped = false; SStreamTaskNodeUpdateMsg req = {0}; diff --git a/source/dnode/vnode/src/tq/tqSink.c b/source/dnode/vnode/src/tq/tqSink.c index 5dafe6a4a0..65484f4842 100644 --- a/source/dnode/vnode/src/tq/tqSink.c +++ b/source/dnode/vnode/src/tq/tqSink.c @@ -13,6 +13,7 @@ * along with this program. If not, see . */ +#include #include "tcommon.h" #include "tmsg.h" #include "tq.h" @@ -28,19 +29,19 @@ static bool hasOnlySubmitData(const SArray* pBlocks, int32_t numOfBlocks); static int32_t tsAscendingSortFn(const void* p1, const void* p2); static int32_t setDstTableDataUid(SVnode* pVnode, SStreamTask* pTask, SSDataBlock* pDataBlock, char* stbFullName, SSubmitTbData* pTableData); -static int32_t setDstTableDataPayload(SStreamTask* pTask, int32_t blockIndex, SSDataBlock* pDataBlock, - SSubmitTbData* pTableData); static int32_t doBuildAndSendDeleteMsg(SVnode* pVnode, char* stbFullName, SSDataBlock* pDataBlock, SStreamTask* pTask, int64_t suid); static int32_t doBuildAndSendSubmitMsg(SVnode* pVnode, SStreamTask* pTask, SSubmitReq2* pReq, int32_t numOfBlocks); static int32_t buildSubmitMsgImpl(SSubmitReq2* pSubmitReq, int32_t vgId, void** pMsg, int32_t* msgLen); -static int32_t doConvertRows(SSubmitTbData* pTableData, STSchema* pTSchema, SSDataBlock* pDataBlock, const char* id); +static int32_t doConvertRows(SSubmitTbData* pTableData, const STSchema* pTSchema, SSDataBlock* pDataBlock, const char* id); static int32_t doWaitForDstTableCreated(SVnode* pVnode, SStreamTask* pTask, STableSinkInfo* pTableSinkInfo, const char* dstTableName, int64_t* uid); static int32_t doPutIntoCache(SSHashObj* pSinkTableMap, STableSinkInfo* pTableSinkInfo, uint64_t groupId, const char* id); -static int32_t doMergeExistedRows(SSubmitTbData* pExisted, const SSubmitTbData* pNew, const char* id); static bool isValidDstChildTable(SMetaReader* pReader, int32_t vgId, const char* ctbName, int64_t suid); -static SVCreateTbReq* buildAutoCreateTableReq(char* stbFullName, int64_t suid, int32_t numOfCols, SSDataBlock* pDataBlock); +static int32_t initCreateTableMsg(SVCreateTbReq* pCreateTableReq, uint64_t suid, const char* stbFullName, int32_t numOfTags); +static SArray* createDefaultTagColName(); +static void setCreateTableMsgTableName(SVCreateTbReq* pCreateTableReq, SSDataBlock* pDataBlock, const char* stbFullName, + int64_t gid); int32_t tqBuildDeleteReq(const char* stbFullName, const SSDataBlock* pDataBlock, SBatchDeleteReq* deleteReq, const char* pIdStr) { @@ -138,61 +139,68 @@ static int32_t tqPutReqToQueue(SVnode* pVnode, SVCreateTbBatchReq* pReqs) { return TSDB_CODE_SUCCESS; } +int32_t initCreateTableMsg(SVCreateTbReq* pCreateTableReq, uint64_t suid, const char* stbFullName, int32_t numOfTags) { + pCreateTableReq->flags = 0; + pCreateTableReq->type = TSDB_CHILD_TABLE; + pCreateTableReq->ctb.suid = suid; + + // set super table name + SName name = {0}; + tNameFromString(&name, stbFullName, T_NAME_ACCT | T_NAME_DB | T_NAME_TABLE); + pCreateTableReq->ctb.stbName = taosStrdup((char*)tNameGetTableName(&name)); + + pCreateTableReq->ctb.tagNum = numOfTags; + return TSDB_CODE_SUCCESS; +} + +SArray* createDefaultTagColName() { + SArray* pTagColNameList = taosArrayInit(1, TSDB_COL_NAME_LEN); + char tagNameStr[TSDB_COL_NAME_LEN] = "group_id"; + taosArrayPush(pTagColNameList, tagNameStr); + return pTagColNameList; +} + +void setCreateTableMsgTableName(SVCreateTbReq* pCreateTableReq, SSDataBlock* pDataBlock, const char* stbFullName, + int64_t gid) { + if (pDataBlock->info.parTbName[0]) { + pCreateTableReq->name = taosStrdup(pDataBlock->info.parTbName); + } else { + pCreateTableReq->name = buildCtbNameByGroupId(stbFullName, gid); + } +} + static int32_t doBuildAndSendCreateTableMsg(SVnode* pVnode, char* stbFullName, SSDataBlock* pDataBlock, SStreamTask* pTask, int64_t suid) { tqDebug("s-task:%s build create table msg", pTask->id.idStr); STSchema* pTSchema = pTask->outputInfo.tbSink.pTSchema; int32_t rows = pDataBlock->info.rows; - SArray* tagArray = NULL; + SArray* tagArray = taosArrayInit(4, sizeof(STagVal));; int32_t code = 0; SVCreateTbBatchReq reqs = {0}; - SArray* crTblArray = reqs.pArray = taosArrayInit(1, sizeof(SVCreateTbReq)); if (NULL == reqs.pArray) { + tqError("s-task:%s failed to init create table msg, code:%s", pTask->id.idStr, tstrerror(terrno)); goto _end; } for (int32_t rowId = 0; rowId < rows; rowId++) { SVCreateTbReq* pCreateTbReq = &((SVCreateTbReq){0}); - // set const - pCreateTbReq->flags = 0; - pCreateTbReq->type = TSDB_CHILD_TABLE; - pCreateTbReq->ctb.suid = suid; - - // set super table name - SName name = {0}; - tNameFromString(&name, stbFullName, T_NAME_ACCT | T_NAME_DB | T_NAME_TABLE); - pCreateTbReq->ctb.stbName = taosStrdup((char*)tNameGetTableName(&name)); // taosStrdup(stbFullName); - - // set tag content int32_t size = taosArrayGetSize(pDataBlock->pDataBlock); - if (size == 2) { - tagArray = taosArrayInit(1, sizeof(STagVal)); - if (!tagArray) { - tdDestroySVCreateTbReq(pCreateTbReq); - goto _end; - } + int32_t numOfTags = TMAX(size - UD_TAG_COLUMN_INDEX, 1); + initCreateTableMsg(pCreateTbReq, suid, stbFullName, numOfTags); + taosArrayClear(tagArray); + + if (size == 2) { STagVal tagVal = { .cid = pTSchema->numOfCols + 1, .type = TSDB_DATA_TYPE_UBIGINT, .i64 = pDataBlock->info.id.groupId}; taosArrayPush(tagArray, &tagVal); - - // set tag name - SArray* tagName = taosArrayInit(1, TSDB_COL_NAME_LEN); - char tagNameStr[TSDB_COL_NAME_LEN] = "group_id"; - taosArrayPush(tagName, tagNameStr); - pCreateTbReq->ctb.tagName = tagName; + pCreateTbReq->ctb.tagName = createDefaultTagColName(); } else { - tagArray = taosArrayInit(size - 1, sizeof(STagVal)); - if (!tagArray) { - tdDestroySVCreateTbReq(pCreateTbReq); - goto _end; - } - for (int32_t tagId = UD_TAG_COLUMN_INDEX, step = 1; tagId < size; tagId++, step++) { SColumnInfoData* pTagData = taosArrayGet(pDataBlock->pDataBlock, tagId); @@ -209,29 +217,25 @@ static int32_t doBuildAndSendCreateTableMsg(SVnode* pVnode, char* stbFullName, S taosArrayPush(tagArray, &tagVal); } } - pCreateTbReq->ctb.tagNum = TMAX(size - UD_TAG_COLUMN_INDEX, 1); - STag* pTag = NULL; - tTagNew(tagArray, 1, false, &pTag); + tTagNew(tagArray, 1, false, (STag**)&pCreateTbReq->ctb.pTag); tagArray = taosArrayDestroy(tagArray); - if (pTag == NULL) { + if (pCreateTbReq->ctb.pTag == NULL) { tdDestroySVCreateTbReq(pCreateTbReq); code = TSDB_CODE_OUT_OF_MEMORY; goto _end; } - pCreateTbReq->ctb.pTag = (uint8_t*)pTag; - - // set table name - if (!pDataBlock->info.parTbName[0]) { + uint64_t gid = pDataBlock->info.id.groupId; + if (taosArrayGetSize(pDataBlock->pDataBlock) > UD_GROUPID_COLUMN_INDEX) { SColumnInfoData* pGpIdColInfo = taosArrayGet(pDataBlock->pDataBlock, UD_GROUPID_COLUMN_INDEX); void* pGpIdData = colDataGetData(pGpIdColInfo, rowId); - pCreateTbReq->name = buildCtbNameByGroupId(stbFullName, *(uint64_t*)pGpIdData); - } else { - pCreateTbReq->name = taosStrdup(pDataBlock->info.parTbName); + ASSERT(gid == *(int64_t*)pGpIdData); } + setCreateTableMsgTableName(pCreateTbReq, pDataBlock, stbFullName, gid); + taosArrayPush(reqs.pArray, pCreateTbReq); tqDebug("s-task:%s build create table:%s msg complete", pTask->id.idStr, pCreateTbReq->name); } @@ -390,10 +394,8 @@ bool isValidDstChildTable(SMetaReader* pReader, int32_t vgId, const char* ctbNam return true; } -SVCreateTbReq* buildAutoCreateTableReq(char* stbFullName, int64_t suid, int32_t numOfCols, SSDataBlock* pDataBlock) { - char* ctbName = pDataBlock->info.parTbName; - - SVCreateTbReq* pCreateTbReq = taosMemoryCalloc(1, sizeof(SVCreateStbReq)); +SVCreateTbReq* buildAutoCreateTableReq(const char* stbFullName, int64_t suid, int32_t numOfCols, SSDataBlock* pDataBlock) { + SVCreateTbReq* pCreateTbReq = taosMemoryCalloc(1, sizeof(SVCreateTbReq)); if (pCreateTbReq == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; return NULL; @@ -408,42 +410,25 @@ SVCreateTbReq* buildAutoCreateTableReq(char* stbFullName, int64_t suid, int32_t return NULL; } - // set const - pCreateTbReq->flags = 0; - pCreateTbReq->type = TSDB_CHILD_TABLE; - pCreateTbReq->ctb.suid = suid; - - // set super table name - SName name = {0}; - tNameFromString(&name, stbFullName, T_NAME_ACCT | T_NAME_DB | T_NAME_TABLE); - pCreateTbReq->ctb.stbName = taosStrdup((char*)tNameGetTableName(&name)); + initCreateTableMsg(pCreateTbReq, suid, stbFullName, 1); STagVal tagVal = { .cid = numOfCols, .type = TSDB_DATA_TYPE_UBIGINT, .i64 = pDataBlock->info.id.groupId}; taosArrayPush(tagArray, &tagVal); - pCreateTbReq->ctb.tagNum = taosArrayGetSize(tagArray); - STag* pTag = NULL; - tTagNew(tagArray, 1, false, &pTag); + tTagNew(tagArray, 1, false, (STag**) &pCreateTbReq->ctb.pTag); taosArrayDestroy(tagArray); - if (pTag == NULL) { + if (pCreateTbReq->ctb.pTag == NULL) { tdDestroySVCreateTbReq(pCreateTbReq); taosMemoryFreeClear(pCreateTbReq); terrno = TSDB_CODE_OUT_OF_MEMORY; return NULL; } - pCreateTbReq->ctb.pTag = (uint8_t*)pTag; - - // set tag name - SArray* tagName = taosArrayInit(1, TSDB_COL_NAME_LEN); - char k[TSDB_COL_NAME_LEN] = "group_id"; - taosArrayPush(tagName, k); - - pCreateTbReq->ctb.tagName = tagName; + pCreateTbReq->ctb.tagName = createDefaultTagColName();; // set table name - pCreateTbReq->name = taosStrdup(ctbName); + setCreateTableMsgTableName(pCreateTbReq, pDataBlock, stbFullName, pDataBlock->info.id.groupId); return pCreateTbReq; } @@ -514,7 +499,7 @@ int32_t tsAscendingSortFn(const void* p1, const void* p2) { } } -int32_t doConvertRows(SSubmitTbData* pTableData, STSchema* pTSchema, SSDataBlock* pDataBlock, const char* id) { +int32_t doConvertRows(SSubmitTbData* pTableData, const STSchema* pTSchema, SSDataBlock* pDataBlock, const char* id) { int32_t numOfRows = pDataBlock->info.rows; int32_t code = TSDB_CODE_SUCCESS; @@ -724,17 +709,16 @@ int32_t setDstTableDataUid(SVnode* pVnode, SStreamTask* pTask, SSDataBlock* pDat return TSDB_CODE_SUCCESS; } -int32_t setDstTableDataPayload(SStreamTask* pTask, int32_t blockIndex, SSDataBlock* pDataBlock, - SSubmitTbData* pTableData) { - int32_t numOfRows = pDataBlock->info.rows; - const char* id = pTask->id.idStr; +int32_t setDstTableDataPayload(uint64_t suid, const STSchema *pTSchema, int32_t blockIndex, SSDataBlock* pDataBlock, + SSubmitTbData* pTableData, const char* id) { + int32_t numOfRows = pDataBlock->info.rows; tqDebug("s-task:%s sink data pipeline, build submit msg from %dth resBlock, including %d rows, dst suid:%" PRId64, - id, blockIndex + 1, numOfRows, pTask->outputInfo.tbSink.stbUid); + id, blockIndex + 1, numOfRows, suid); char* dstTableName = pDataBlock->info.parTbName; // convert all rows - int32_t code = doConvertRows(pTableData, pTask->outputInfo.tbSink.pTSchema, pDataBlock, id); + int32_t code = doConvertRows(pTableData, pTSchema, pDataBlock, id); if (code != TSDB_CODE_SUCCESS) { tqError("s-task:%s failed to convert rows from result block, code:%s", id, tstrerror(terrno)); return code; @@ -800,7 +784,7 @@ void tqSinkDataIntoDstTable(SStreamTask* pTask, void* vnode, void* data) { continue; } - code = setDstTableDataPayload(pTask, i, pDataBlock, &tbData); + code = setDstTableDataPayload(suid, pTSchema, i, pDataBlock, &tbData, id); if (code != TSDB_CODE_SUCCESS) { continue; } @@ -847,7 +831,7 @@ void tqSinkDataIntoDstTable(SStreamTask* pTask, void* vnode, void* data) { continue; } - code = setDstTableDataPayload(pTask, i, pDataBlock, &tbData); + code = setDstTableDataPayload(suid, pTSchema, i, pDataBlock, &tbData, id); if (code != TSDB_CODE_SUCCESS) { continue; } @@ -857,7 +841,7 @@ void tqSinkDataIntoDstTable(SStreamTask* pTask, void* vnode, void* data) { int32_t size = (int32_t)taosArrayGetSize(submitReq.aSubmitTbData) - 1; taosHashPut(pTableIndexMap, &groupId, sizeof(groupId), &size, sizeof(size)); } else { - code = setDstTableDataPayload(pTask, i, pDataBlock, &tbData); + code = setDstTableDataPayload(suid, pTSchema, i, pDataBlock, &tbData, id); if (code != TSDB_CODE_SUCCESS) { continue; } diff --git a/source/libs/stream/src/stream.c b/source/libs/stream/src/stream.c index 307e1c47c9..fcb61dc892 100644 --- a/source/libs/stream/src/stream.c +++ b/source/libs/stream/src/stream.c @@ -62,7 +62,7 @@ static void streamSchedByTimer(void* param, void* tmrId) { int32_t nextTrigger = (int32_t)pTask->info.triggerParam; int8_t status = atomic_load_8(&pTask->schedInfo.status); - stDebug("s-task:%s in scheduler, trigger status:%d, next:%dms", id, status, nextTrigger); + stTrace("s-task:%s in scheduler, trigger status:%d, next:%dms", id, status, nextTrigger); if (streamTaskShouldStop(pTask) || streamTaskShouldPause(pTask)) { stDebug("s-task:%s jump out of schedTimer", id); diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index c3c5b5cb76..8401de5baf 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -2006,14 +2006,12 @@ int32_t streamStateAddIfNotExist_rocksdb(SStreamState* pState, const SWinKey* ke return 0; } int32_t streamStateCurPrev_rocksdb(SStreamStateCur* pCur) { - stDebug("streamStateCurPrev_rocksdb"); if (!pCur) return -1; rocksdb_iter_prev(pCur->iter); return 0; } int32_t streamStateGetKVByCur_rocksdb(SStreamStateCur* pCur, SWinKey* pKey, const void** pVal, int32_t* pVLen) { - stDebug("streamStateGetKVByCur_rocksdb"); if (!pCur) return -1; SStateKey tkey; SStateKey* pKtmp = &tkey; @@ -2038,7 +2036,6 @@ int32_t streamStateGetKVByCur_rocksdb(SStreamStateCur* pCur, SWinKey* pKey, cons return -1; } SStreamStateCur* streamStateGetAndCheckCur_rocksdb(SStreamState* pState, SWinKey* key) { - stDebug("streamStateGetAndCheckCur_rocksdb"); SStreamStateCur* pCur = streamStateFillGetCur_rocksdb(pState, key); if (pCur) { int32_t code = streamStateGetGroupKVByCur_rocksdb(pCur, key, NULL, 0); @@ -2049,7 +2046,6 @@ SStreamStateCur* streamStateGetAndCheckCur_rocksdb(SStreamState* pState, SWinKey } SStreamStateCur* streamStateSeekKeyNext_rocksdb(SStreamState* pState, const SWinKey* key) { - stDebug("streamStateSeekKeyNext_rocksdb"); SStreamStateCur* pCur = createStreamStateCursor(); if (pCur == NULL) { return NULL; @@ -2088,7 +2084,6 @@ SStreamStateCur* streamStateSeekKeyNext_rocksdb(SStreamState* pState, const SWin } SStreamStateCur* streamStateSeekToLast_rocksdb(SStreamState* pState) { - stDebug("streamStateGetCur_rocksdb"); int32_t code = 0; const SStateKey maxStateKey = {.key = {.groupId = UINT64_MAX, .ts = INT64_MAX}, .opNum = INT64_MAX}; @@ -2129,7 +2124,6 @@ SStreamStateCur* streamStateSeekToLast_rocksdb(SStreamState* pState) { } SStreamStateCur* streamStateGetCur_rocksdb(SStreamState* pState, const SWinKey* key) { - stDebug("streamStateGetCur_rocksdb"); SBackendCfWrapper* wrapper = pState->pTdbState->pBackendCfWrapper; SStreamStateCur* pCur = createStreamStateCursor(); diff --git a/source/libs/stream/src/streamStart.c b/source/libs/stream/src/streamStart.c index 9ebd617be0..965f116572 100644 --- a/source/libs/stream/src/streamStart.c +++ b/source/libs/stream/src/streamStart.c @@ -407,8 +407,7 @@ int32_t streamProcessCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* pRs int32_t ref = atomic_add_fetch_32(&pTask->status.timerActive, 1); stDebug("s-task:%s downstream taskId:0x%x (vgId:%d) not ready, stage:%d, retry in 100ms, ref:%d ", id, pRsp->downstreamTaskId, pRsp->downstreamNodeId, pRsp->oldStage, ref); - - taosTmrReset(recheckDownstreamTasks, CHECK_DOWNSTREAM_INTERVAL, pInfo, streamEnv.timer, &pInfo->checkTimer); + pInfo->checkTimer = taosTmrStart(recheckDownstreamTasks, CHECK_DOWNSTREAM_INTERVAL, pInfo, streamEnv.timer); } } diff --git a/tests/pytest/util/dnodes.py b/tests/pytest/util/dnodes.py index 87dd284368..92577744ac 100644 --- a/tests/pytest/util/dnodes.py +++ b/tests/pytest/util/dnodes.py @@ -136,6 +136,7 @@ class TDDnode: "tqDebugFlag": "143", "cDebugFlag": "143", "stDebugFlag": "143", + "smaDebugFlag": "143", "jniDebugFlag": "143", "qDebugFlag": "143", "rpcDebugFlag": "143", diff --git a/tests/system-test/0-others/timeRangeWise.py b/tests/system-test/0-others/timeRangeWise.py index 5ef5aa4a75..7a258f5dd3 100644 --- a/tests/system-test/0-others/timeRangeWise.py +++ b/tests/system-test/0-others/timeRangeWise.py @@ -23,7 +23,8 @@ from util.cases import * from util.sql import * class TDTestCase: - + updatecfgDict = {'vdebugFlag': 143, 'qdebugflag':135, 'tqdebugflag':135, 'udebugflag':135, 'rpcdebugflag':135, + 'asynclog': 0, 'stdebugflag':135} # random string def random_string(self, count): letters = string.ascii_letters diff --git a/tests/system-test/test.py b/tests/system-test/test.py index 6813530a5c..81f98fea22 100644 --- a/tests/system-test/test.py +++ b/tests/system-test/test.py @@ -398,7 +398,7 @@ if __name__ == "__main__": tdDnodes.setValgrind(valgrind) tdDnodes.stopAll() for dnode in tdDnodes.dnodes: - tdDnodes.deploy(dnode.index,{}) + tdDnodes.deploy(dnode.index, updateCfgDict) for dnode in tdDnodes.dnodes: tdDnodes.starttaosd(dnode.index) tdCases.logSql(logSql) From f446ff1c368ba2ee0f30bd59755c4bcca1d1d770 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 24 Oct 2023 09:48:10 +0800 Subject: [PATCH 25/41] refactor(stream): do some internal refactor. --- source/dnode/vnode/src/inc/tq.h | 2 +- source/dnode/vnode/src/sma/smaTimeRange.c | 11 ++++------ source/dnode/vnode/src/tq/tqSink.c | 25 ++++++++++------------- 3 files changed, 16 insertions(+), 22 deletions(-) diff --git a/source/dnode/vnode/src/inc/tq.h b/source/dnode/vnode/src/inc/tq.h index 84f3b3e85a..24c31b95c8 100644 --- a/source/dnode/vnode/src/inc/tq.h +++ b/source/dnode/vnode/src/inc/tq.h @@ -175,7 +175,7 @@ int32_t setDstTableDataPayload(uint64_t suid, const STSchema* pTSchema, int32_t int32_t doMergeExistedRows(SSubmitTbData* pExisted, const SSubmitTbData* pNew, const char* id); SVCreateTbReq* buildAutoCreateTableReq(const char* stbFullName, int64_t suid, int32_t numOfCols, - SSDataBlock* pDataBlock); + SSDataBlock* pDataBlock, SArray* pTagArray); #ifdef __cplusplus } diff --git a/source/dnode/vnode/src/sma/smaTimeRange.c b/source/dnode/vnode/src/sma/smaTimeRange.c index f18843bc35..b7f7d16ddd 100644 --- a/source/dnode/vnode/src/sma/smaTimeRange.c +++ b/source/dnode/vnode/src/sma/smaTimeRange.c @@ -162,17 +162,15 @@ int32_t smaBlockToSubmit(SVnode *pVnode, const SArray *pBlocks, const STSchema * int32_t len = 0; SSubmitReq2 *pReq = NULL; SArray *tagArray = NULL; - SArray *createTbArray = NULL; SArray *pVals = NULL; int32_t numOfBlocks = taosArrayGetSize(pBlocks); tagArray = taosArrayInit(1, sizeof(STagVal)); - createTbArray = taosArrayInit(numOfBlocks, POINTER_BYTES); pReq = taosMemoryCalloc(1, sizeof(SSubmitReq2)); pReq->aSubmitTbData = taosArrayInit(1, sizeof(SSubmitTbData)); - if (!tagArray || !createTbArray || !pReq || !pReq->aSubmitTbData) { + if (!tagArray || !pReq || !pReq->aSubmitTbData) { code = terrno == TSDB_CODE_SUCCESS ? TSDB_CODE_OUT_OF_MEMORY : terrno; TSDB_CHECK_CODE(code, lino, _exit); } @@ -191,10 +189,9 @@ int32_t smaBlockToSubmit(SVnode *pVnode, const SArray *pBlocks, const STSchema * } SSubmitTbData tbData = {.suid = suid, .uid = 0, .sver = pTSchema->version, .flags = SUBMIT_REQ_AUTO_CREATE_TABLE,}; - int32_t cid = taosArrayGetSize(pDataBlock->pDataBlock) + 1; - tbData.pCreateTbReq = - buildAutoCreateTableReq(stbFullName, suid, taosArrayGetSize(pDataBlock->pDataBlock) + 1, pDataBlock); + int32_t cid = taosArrayGetSize(pDataBlock->pDataBlock) + 1; + tbData.pCreateTbReq = buildAutoCreateTableReq(stbFullName, suid, cid, pDataBlock, tagArray); { uint64_t groupId = pDataBlock->info.id.groupId; @@ -248,8 +245,8 @@ int32_t smaBlockToSubmit(SVnode *pVnode, const SArray *pBlocks, const STSchema * } tEncoderClear(&encoder); } + _exit: - taosArrayDestroy(createTbArray); taosArrayDestroy(tagArray); taosArrayDestroy(pVals); if (pReq) { diff --git a/source/dnode/vnode/src/tq/tqSink.c b/source/dnode/vnode/src/tq/tqSink.c index 65484f4842..97e3376663 100644 --- a/source/dnode/vnode/src/tq/tqSink.c +++ b/source/dnode/vnode/src/tq/tqSink.c @@ -394,29 +394,21 @@ bool isValidDstChildTable(SMetaReader* pReader, int32_t vgId, const char* ctbNam return true; } -SVCreateTbReq* buildAutoCreateTableReq(const char* stbFullName, int64_t suid, int32_t numOfCols, SSDataBlock* pDataBlock) { +SVCreateTbReq* buildAutoCreateTableReq(const char* stbFullName, int64_t suid, int32_t numOfCols, + SSDataBlock* pDataBlock, SArray* pTagArray) { SVCreateTbReq* pCreateTbReq = taosMemoryCalloc(1, sizeof(SVCreateTbReq)); if (pCreateTbReq == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; return NULL; } - // set tag content - SArray* tagArray = taosArrayInit(1, sizeof(STagVal)); - if (tagArray == NULL) { - terrno = TSDB_CODE_OUT_OF_MEMORY; - tdDestroySVCreateTbReq(pCreateTbReq); - taosMemoryFreeClear(pCreateTbReq); - return NULL; - } - + taosArrayClear(pTagArray); initCreateTableMsg(pCreateTbReq, suid, stbFullName, 1); STagVal tagVal = { .cid = numOfCols, .type = TSDB_DATA_TYPE_UBIGINT, .i64 = pDataBlock->info.id.groupId}; - taosArrayPush(tagArray, &tagVal); + taosArrayPush(pTagArray, &tagVal); - tTagNew(tagArray, 1, false, (STag**) &pCreateTbReq->ctb.pTag); - taosArrayDestroy(tagArray); + tTagNew(pTagArray, 1, false, (STag**) &pCreateTbReq->ctb.pTag); if (pCreateTbReq->ctb.pTag == NULL) { tdDestroySVCreateTbReq(pCreateTbReq); @@ -678,8 +670,13 @@ int32_t setDstTableDataUid(SVnode* pVnode, SStreamTask* pTask, SSDataBlock* pDat tqDebug("s-task:%s stream write into table:%s, table auto created", id, dstTableName); + SArray* pTagArray = taosArrayInit(pTSchema->numOfCols + 1, sizeof(STagVal)); + pTableData->flags = SUBMIT_REQ_AUTO_CREATE_TABLE; - pTableData->pCreateTbReq = buildAutoCreateTableReq(stbFullName, suid, pTSchema->numOfCols + 1, pDataBlock); + pTableData->pCreateTbReq = + buildAutoCreateTableReq(stbFullName, suid, pTSchema->numOfCols + 1, pDataBlock, pTagArray); + taosArrayDestroy(pTagArray); + if (pTableData->pCreateTbReq == NULL) { tqError("s-task:%s failed to build auto create table req, code:%s", id, tstrerror(terrno)); taosMemoryFree(pTableSinkInfo); From 0dae8c3c76926733b64326d2270d43076937a149 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 24 Oct 2023 09:49:14 +0800 Subject: [PATCH 26/41] refactor: do some internal refactor. --- source/dnode/vnode/src/sma/smaTimeRange.c | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/source/dnode/vnode/src/sma/smaTimeRange.c b/source/dnode/vnode/src/sma/smaTimeRange.c index b7f7d16ddd..2395a7cfb9 100644 --- a/source/dnode/vnode/src/sma/smaTimeRange.c +++ b/source/dnode/vnode/src/sma/smaTimeRange.c @@ -154,7 +154,7 @@ _exit: return code; } -int32_t smaBlockToSubmit(SVnode *pVnode, const SArray *pBlocks, const STSchema *pTSchema, bool createTb, int64_t suid, +int32_t smaBlockToSubmit(SVnode *pVnode, const SArray *pBlocks, const STSchema *pTSchema, int64_t suid, const char *stbFullName, SBatchDeleteReq *pDeleteReq, void **ppData, int32_t *pLen) { int32_t code = 0; int32_t lino = 0; @@ -360,8 +360,8 @@ static int32_t tdProcessTSmaInsertImpl(SSma *pSma, int64_t indexUid, const char void *pSubmitReq = NULL; int32_t contLen = 0; - code = smaBlockToSubmit(pSma->pVnode, (const SArray *)msg, pTsmaStat->pTSchema, true, - pTsmaStat->pTSma->dstTbUid, pTsmaStat->pTSma->dstTbName, &deleteReq, &pSubmitReq, &contLen); + code = smaBlockToSubmit(pSma->pVnode, (const SArray *)msg, pTsmaStat->pTSchema, pTsmaStat->pTSma->dstTbUid, + pTsmaStat->pTSma->dstTbName, &deleteReq, &pSubmitReq, &contLen); TSDB_CHECK_CODE(code, lino, _exit); if ((terrno = tsmaProcessDelReq(pSma, indexUid, &deleteReq)) != 0) { From e92b276e3e70add2dad520b1854791c006455989 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 24 Oct 2023 10:14:29 +0800 Subject: [PATCH 27/41] fix(stream): set checkpoint failure when in pause status. --- source/dnode/vnode/src/tq/tq.c | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 7c67f3cd9b..c8fec925db 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1750,7 +1750,9 @@ int32_t tqProcessTaskCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg* pRsp) // todo save the checkpoint failed info taosThreadMutexLock(&pTask->lock); - if (streamTaskGetStatus(pTask, NULL) == TASK_STATUS__HALT) { + ETaskStatus status = streamTaskGetStatus(pTask, NULL); + + if (status == TASK_STATUS__HALT || status == TASK_STATUS__PAUSE) { qError("s-task:%s not ready for checkpoint, since it is halt, ignore this checkpoint:%" PRId64 ", set it failure", pTask->id.idStr, req.checkpointId); taosThreadMutexUnlock(&pTask->lock); From a6e706ba2d90f040d87f053e537ab7ab5b2defbb Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 24 Oct 2023 11:38:14 +0800 Subject: [PATCH 28/41] fix(stream): ignore the check status when is ready. --- source/dnode/vnode/src/tq/tqStreamTask.c | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/source/dnode/vnode/src/tq/tqStreamTask.c b/source/dnode/vnode/src/tq/tqStreamTask.c index cb313b7eed..7df8bdf891 100644 --- a/source/dnode/vnode/src/tq/tqStreamTask.c +++ b/source/dnode/vnode/src/tq/tqStreamTask.c @@ -91,10 +91,13 @@ int32_t tqStartStreamTask(STQ* pTq) { continue; } - if (pTask->status.downstreamReady == 1 && HAS_RELATED_FILLHISTORY_TASK(pTask)) { - tqDebug("s-task:%s downstream ready, no need to check downstream, check only related fill-history task", - pTask->id.idStr); - streamLaunchFillHistoryTask(pTask); + if (pTask->status.downstreamReady == 1) { + if (HAS_RELATED_FILLHISTORY_TASK(pTask)) { + tqDebug("s-task:%s downstream ready, no need to check downstream, check only related fill-history task", + pTask->id.idStr); + streamLaunchFillHistoryTask(pTask); + } + streamMetaReleaseTask(pMeta, pTask); continue; } From 0f9328330b044c893d89bd16f8015ab43f525641 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 24 Oct 2023 11:56:36 +0800 Subject: [PATCH 29/41] fix(stream): disable continue check for downstream tasks. --- source/libs/stream/src/streamStart.c | 17 +++++++++-------- 1 file changed, 9 insertions(+), 8 deletions(-) diff --git a/source/libs/stream/src/streamStart.c b/source/libs/stream/src/streamStart.c index 965f116572..b060121680 100644 --- a/source/libs/stream/src/streamStart.c +++ b/source/libs/stream/src/streamStart.c @@ -390,20 +390,21 @@ int32_t streamProcessCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* pRs doProcessDownstreamReadyRsp(pTask); } } else { // not ready, wait for 100ms and retry - if (pRsp->status == TASK_DOWNSTREAM_NOT_LEADER) { + if (pRsp->status == TASK_UPSTREAM_NEW_STAGE) { stError( - "s-task:%s downstream taskId:0x%x (vgId:%d) vnode-transfer/leader-change detected, not send check again, " - "roll-back needed", - id, pRsp->downstreamTaskId, pRsp->downstreamNodeId); + "s-task:%s vgId:%d self vnode-transfer/leader-change/restart detected, old stage:%d, current stage:%d, " + "not check wait for downstream task nodeUpdate, and all tasks restart", + id, pRsp->upstreamNodeId, pRsp->oldStage, (int32_t)pTask->pMeta->stage); } else { - if (pRsp->status == TASK_UPSTREAM_NEW_STAGE) { + if (pRsp->status == TASK_DOWNSTREAM_NOT_LEADER) { stError( - "s-task:%s vnode-transfer/leader-change/restart detected, old stage:%d, current stage:%d, continue check " - "till downstream nodeUpdate", - id, pRsp->oldStage, (int32_t)pTask->pMeta->stage); + "s-task:%s downstream taskId:0x%x (vgId:%d) vnode-transfer/leader-change detected, not send check again, " + "roll-back needed", + id, pRsp->downstreamTaskId, pRsp->downstreamNodeId); } STaskRecheckInfo* pInfo = createRecheckInfo(pTask, pRsp); + int32_t ref = atomic_add_fetch_32(&pTask->status.timerActive, 1); stDebug("s-task:%s downstream taskId:0x%x (vgId:%d) not ready, stage:%d, retry in 100ms, ref:%d ", id, pRsp->downstreamTaskId, pRsp->downstreamNodeId, pRsp->oldStage, ref); From f1498f89294539010ec1d56975f21c9a87e2dcd3 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 24 Oct 2023 15:24:34 +0800 Subject: [PATCH 30/41] fix(stream): fix the invalid write in sma --- source/common/src/tmsg.c | 2 +- source/dnode/vnode/src/sma/smaTimeRange.c | 17 ++++++++------ source/dnode/vnode/src/tq/tqSink.c | 3 ++- source/libs/stream/src/streamTaskSm.c | 27 ++++++++++++++++++++--- 4 files changed, 37 insertions(+), 12 deletions(-) diff --git a/source/common/src/tmsg.c b/source/common/src/tmsg.c index 9b66bd1fb3..c67b9e5e68 100644 --- a/source/common/src/tmsg.c +++ b/source/common/src/tmsg.c @@ -8422,7 +8422,7 @@ void tDestroySubmitTbData(SSubmitTbData *pTbData, int32_t flag) { } else { tDestroySVCreateTbReq(pTbData->pCreateTbReq, TSDB_MSG_FLG_DECODE); } - taosMemoryFree(pTbData->pCreateTbReq); + taosMemoryFreeClear(pTbData->pCreateTbReq); } if (pTbData->flags & SUBMIT_REQ_COLUMN_DATA_FORMAT) { diff --git a/source/dnode/vnode/src/sma/smaTimeRange.c b/source/dnode/vnode/src/sma/smaTimeRange.c index 2395a7cfb9..08ddc4bd7b 100644 --- a/source/dnode/vnode/src/sma/smaTimeRange.c +++ b/source/dnode/vnode/src/sma/smaTimeRange.c @@ -162,15 +162,19 @@ int32_t smaBlockToSubmit(SVnode *pVnode, const SArray *pBlocks, const STSchema * int32_t len = 0; SSubmitReq2 *pReq = NULL; SArray *tagArray = NULL; - SArray *pVals = NULL; int32_t numOfBlocks = taosArrayGetSize(pBlocks); tagArray = taosArrayInit(1, sizeof(STagVal)); pReq = taosMemoryCalloc(1, sizeof(SSubmitReq2)); - pReq->aSubmitTbData = taosArrayInit(1, sizeof(SSubmitTbData)); - if (!tagArray || !pReq || !pReq->aSubmitTbData) { + if (!tagArray || !pReq) { + code = terrno == TSDB_CODE_SUCCESS ? TSDB_CODE_OUT_OF_MEMORY : terrno; + TSDB_CHECK_CODE(code, lino, _exit); + } + + pReq->aSubmitTbData = taosArrayInit(1, sizeof(SSubmitTbData)); + if (pReq->aSubmitTbData == NULL) { code = terrno == TSDB_CODE_SUCCESS ? TSDB_CODE_OUT_OF_MEMORY : terrno; TSDB_CHECK_CODE(code, lino, _exit); } @@ -220,10 +224,10 @@ int32_t smaBlockToSubmit(SVnode *pVnode, const SArray *pBlocks, const STSchema * } } } - - taosArrayPush(pReq->aSubmitTbData, &tbData); } + taosHashCleanup(pTableIndexMap); + // encode tEncodeSize(tEncodeSubmitReq, pReq, len, code); if (TSDB_CODE_SUCCESS == code) { @@ -248,8 +252,7 @@ int32_t smaBlockToSubmit(SVnode *pVnode, const SArray *pBlocks, const STSchema * _exit: taosArrayDestroy(tagArray); - taosArrayDestroy(pVals); - if (pReq) { + if (pReq != NULL) { tDestroySubmitReq(pReq, TSDB_MSG_FLG_ENCODE); taosMemoryFree(pReq); } diff --git a/source/dnode/vnode/src/tq/tqSink.c b/source/dnode/vnode/src/tq/tqSink.c index 97e3376663..742b170a8c 100644 --- a/source/dnode/vnode/src/tq/tqSink.c +++ b/source/dnode/vnode/src/tq/tqSink.c @@ -230,6 +230,7 @@ static int32_t doBuildAndSendCreateTableMsg(SVnode* pVnode, char* stbFullName, S if (taosArrayGetSize(pDataBlock->pDataBlock) > UD_GROUPID_COLUMN_INDEX) { SColumnInfoData* pGpIdColInfo = taosArrayGet(pDataBlock->pDataBlock, UD_GROUPID_COLUMN_INDEX); + // todo remove this void* pGpIdData = colDataGetData(pGpIdColInfo, rowId); ASSERT(gid == *(int64_t*)pGpIdData); } @@ -417,7 +418,7 @@ SVCreateTbReq* buildAutoCreateTableReq(const char* stbFullName, int64_t suid, in return NULL; } - pCreateTbReq->ctb.tagName = createDefaultTagColName();; + pCreateTbReq->ctb.tagName = createDefaultTagColName(); // set table name setCreateTableMsgTableName(pCreateTbReq, pDataBlock, stbFullName, pDataBlock->info.id.groupId); diff --git a/source/libs/stream/src/streamTaskSm.c b/source/libs/stream/src/streamTaskSm.c index c3286407e4..bc832c178c 100644 --- a/source/libs/stream/src/streamTaskSm.c +++ b/source/libs/stream/src/streamTaskSm.c @@ -13,6 +13,7 @@ * along with this program. If not, see . */ +#include #include "streamInt.h" #include "streamsm.h" #include "tmisce.h" @@ -243,12 +244,25 @@ static void keepPrevInfo(SStreamTaskSM* pSM) { pSM->prev.state = pSM->current; pSM->prev.evt = pTrans->event; } + int32_t streamTaskOnHandleEventSuccess(SStreamTaskSM* pSM) { - STaskStateTrans* pTrans = pSM->pActiveTrans; - SStreamTask* pTask = pSM->pTask; + SStreamTask* pTask = pSM->pTask; // do update the task status taosThreadMutexLock(&pTask->lock); + STaskStateTrans* pTrans = pSM->pActiveTrans; + + if (pTrans == NULL) { + ETaskStatus s = pSM->current.state; + ASSERT(s == TASK_STATUS__DROPPING || s == TASK_STATUS__PAUSE || s == TASK_STATUS__STOP); + // the pSM->prev.evt may be 0, so print string is not appropriate. + stDebug("status not handled success, current status:%s, trigger event:%d, %s", pSM->current.name, pSM->prev.evt, + pTask->id.idStr); + + taosThreadMutexUnlock(&pTask->lock); + return TSDB_CODE_INVALID_PARA; + } + keepPrevInfo(pSM); pSM->current = pTrans->next; @@ -275,7 +289,7 @@ int32_t streamTaskOnHandleEventSuccess(SStreamTaskSM* pSM) { pSM->pActiveTrans = pNextTrans; pSM->startTs = taosGetTimestampMs(); taosThreadMutexUnlock(&pTask->lock); - + int32_t code = pNextTrans->pAction(pSM->pTask); if (pNextTrans->autoInvokeEndFn) { return streamTaskOnHandleEventSuccess(pSM); @@ -308,9 +322,12 @@ const char* streamTaskGetStatusStr(ETaskStatus status) { void streamTaskResetStatus(SStreamTask* pTask) { SStreamTaskSM* pSM = pTask->status.pSM; + + taosThreadMutexLock(&pTask->lock); pSM->current = StreamTaskStatusList[TASK_STATUS__UNINIT]; pSM->pActiveTrans = NULL; taosArrayClear(pSM->pWaitingEventList); + taosThreadMutexUnlock(&pTask->lock); // clear the downstream ready status pTask->status.downstreamReady = 0; @@ -323,6 +340,8 @@ void streamTaskSetStatusReady(SStreamTask* pTask) { return; } + taosThreadMutexLock(&pTask->lock); + pSM->prev.state = pSM->current; pSM->prev.evt = 0; @@ -330,6 +349,8 @@ void streamTaskSetStatusReady(SStreamTask* pTask) { pSM->startTs = taosGetTimestampMs(); pSM->pActiveTrans = NULL; taosArrayClear(pSM->pWaitingEventList); + + taosThreadMutexUnlock(&pTask->lock); } STaskStateTrans createStateTransform(ETaskStatus current, ETaskStatus next, EStreamTaskEvent event, __state_trans_fn fn, From 6ffb5ee4760024f0a37afe25ef9e2b310ac9006b Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 24 Oct 2023 15:49:50 +0800 Subject: [PATCH 31/41] fix(stream): do send check info if downstream is not leader. --- source/libs/stream/src/streamStart.c | 1 + 1 file changed, 1 insertion(+) diff --git a/source/libs/stream/src/streamStart.c b/source/libs/stream/src/streamStart.c index b060121680..c386a49e2d 100644 --- a/source/libs/stream/src/streamStart.c +++ b/source/libs/stream/src/streamStart.c @@ -401,6 +401,7 @@ int32_t streamProcessCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* pRs "s-task:%s downstream taskId:0x%x (vgId:%d) vnode-transfer/leader-change detected, not send check again, " "roll-back needed", id, pRsp->downstreamTaskId, pRsp->downstreamNodeId); + return 0; } STaskRecheckInfo* pInfo = createRecheckInfo(pTask, pRsp); From 8cde39eebdbdf59a116684c237c8ece7cd335a77 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 24 Oct 2023 19:11:24 +0800 Subject: [PATCH 32/41] fix(stream): fix dead lock when shutdown all tasks. --- include/libs/stream/tstream.h | 6 +----- source/dnode/vnode/src/tq/tq.c | 9 ++++++++- source/libs/stream/inc/streamInt.h | 5 +++++ source/libs/stream/src/streamMeta.c | 11 +++++------ 4 files changed, 19 insertions(+), 12 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 54ca7f1566..a027b4f9ba 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -181,11 +181,6 @@ typedef struct { SSDataBlock* pBlock; } SStreamRefDataBlock; -typedef struct { - int8_t type; - SSDataBlock* pBlock; -} SStreamTrigger; - typedef struct SStreamQueueNode SStreamQueueNode; struct SStreamQueueNode { @@ -804,6 +799,7 @@ int32_t streamMetaLoadAllTasks(SStreamMeta* pMeta); void streamMetaNotifyClose(SStreamMeta* pMeta); void streamMetaStartHb(SStreamMeta* pMeta); void streamMetaInitForSnode(SStreamMeta* pMeta); +bool streamMetaTaskInTimer(SStreamMeta* pMeta); // checkpoint int32_t streamProcessCheckpointSourceReq(SStreamTask* pTask, SStreamCheckpointSourceReq* pReq); diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index c8fec925db..dbd1e02732 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1924,8 +1924,15 @@ int32_t tqProcessTaskUpdateReq(STQ* pTq, SRpcMsg* pMsg) { taosWUnLockLatch(&pMeta->lock); } else { tqDebug("vgId:%d tasks are all updated and stopped, restart them", vgId); - terrno = 0; + taosWUnLockLatch(&pMeta->lock); + + while (streamMetaTaskInTimer(pMeta)) { + qDebug("vgId:%d some tasks in timer, wait for 100ms and recheck", pMeta->vgId); + taosMsleep(100); + } + + taosWLockLatch(&pMeta->lock); int32_t code = streamMetaReopen(pMeta); if (code != 0) { tqError("vgId:%d failed to reopen stream meta", vgId); diff --git a/source/libs/stream/inc/streamInt.h b/source/libs/stream/inc/streamInt.h index d98fa2f436..c63b51d745 100644 --- a/source/libs/stream/inc/streamInt.h +++ b/source/libs/stream/inc/streamInt.h @@ -52,6 +52,11 @@ extern "C" { #define stTrace(...) do { if (stDebugFlag & DEBUG_TRACE) { taosPrintLog("STM ", DEBUG_TRACE, tqDebugFlag, __VA_ARGS__); }} while(0) // clang-format on +typedef struct { + int8_t type; + SSDataBlock* pBlock; +} SStreamTrigger; + typedef struct SStreamGlobalEnv { int8_t inited; void* timer; diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 4307f74709..c536d01516 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -274,6 +274,7 @@ int32_t streamMetaReopen(SStreamMeta* pMeta) { } void streamMetaClear(SStreamMeta* pMeta) { + // remove all existed tasks in this vnode void* pIter = NULL; while ((pIter = taosHashIterate(pMeta->pTasksMap, pIter)) != NULL) { SStreamTask* p = *(SStreamTask**)pIter; @@ -694,8 +695,7 @@ int32_t streamMetaLoadAllTasks(SStreamMeta* pMeta) { tFreeStreamTask(pTask); stError( "vgId:%d stream read incompatible data, rm %s/vnode/vnode*/tq/stream if taosd cannot start, and rebuild " - "stream " - "manually", + "stream manually", vgId, tsDataDir); return -1; } @@ -857,7 +857,7 @@ void metaHbToMnode(void* param, void* tmrId) { } // not leader not send msg - if (pMeta->role == NODE_ROLE_FOLLOWER) { + if (pMeta->role != NODE_ROLE_LEADER) { stInfo("vgId:%d follower not send hb to mnode", pMeta->vgId); taosReleaseRef(streamMetaId, rid); pMeta->pHbInfo->hbStart = 0; @@ -980,9 +980,8 @@ void metaHbToMnode(void* param, void* tmrId) { taosReleaseRef(streamMetaId, rid); } -static bool hasStreamTaskInTimer(SStreamMeta* pMeta) { +bool streamMetaTaskInTimer(SStreamMeta* pMeta) { bool inTimer = false; - taosWLockLatch(&pMeta->lock); void* pIter = NULL; @@ -1036,7 +1035,7 @@ void streamMetaNotifyClose(SStreamMeta* pMeta) { stDebug("vgId:%d start to check all tasks", vgId); int64_t st = taosGetTimestampMs(); - while (hasStreamTaskInTimer(pMeta)) { + while (streamMetaTaskInTimer(pMeta)) { stDebug("vgId:%d some tasks in timer, wait for 100ms and recheck", pMeta->vgId); taosMsleep(100); } From 13d979a2eeb3771348896fad933ba40d7430cc0e Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 25 Oct 2023 00:03:46 +0800 Subject: [PATCH 33/41] fix(stream): add some logs. --- source/dnode/vnode/src/tq/tq.c | 6 ++++++ source/libs/stream/src/streamStart.c | 5 +++-- 2 files changed, 9 insertions(+), 2 deletions(-) diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index dbd1e02732..711c9a52bc 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -959,6 +959,12 @@ int32_t tqProcessTaskCheckRsp(STQ* pTq, SRpcMsg* pMsg) { tqDebug("tq task:0x%x (vgId:%d) recv check rsp(reqId:0x%" PRIx64 ") from 0x%x (vgId:%d) status %d", rsp.upstreamTaskId, rsp.upstreamNodeId, rsp.reqId, rsp.downstreamTaskId, rsp.downstreamNodeId, rsp.status); + if (!vnodeIsRoleLeader(pTq->pVnode)) { + tqError("vgId:%d not leader, task:0x%x not handle the check rsp, downstream:0x%x (vgId:%d)", vgId, + rsp.upstreamTaskId, rsp.downstreamTaskId, rsp.downstreamNodeId); + return code; + } + SStreamTask* pTask = streamMetaAcquireTask(pTq->pStreamMeta, rsp.streamId, rsp.upstreamTaskId); if (pTask == NULL) { tqError("tq failed to locate the stream task:0x%" PRIx64 "-0x%x (vgId:%d), it may have been destroyed or stopped", diff --git a/source/libs/stream/src/streamStart.c b/source/libs/stream/src/streamStart.c index c386a49e2d..56910c0c53 100644 --- a/source/libs/stream/src/streamStart.c +++ b/source/libs/stream/src/streamStart.c @@ -254,6 +254,7 @@ int32_t streamTaskCheckStatus(SStreamTask* pTask, int32_t upstreamTaskId, int32_ if (pInfo->stage != stage) { return TASK_UPSTREAM_NEW_STAGE; } else if (pTask->status.downstreamReady != 1) { + stDebug("s-task:%s vgId:%d leader:%d, downstream not ready", id, vgId, (pTask->pMeta->role == NODE_ROLE_LEADER)); return TASK_DOWNSTREAM_NOT_READY; } else { return TASK_DOWNSTREAM_READY; @@ -398,8 +399,8 @@ int32_t streamProcessCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* pRs } else { if (pRsp->status == TASK_DOWNSTREAM_NOT_LEADER) { stError( - "s-task:%s downstream taskId:0x%x (vgId:%d) vnode-transfer/leader-change detected, not send check again, " - "roll-back needed", + "s-task:%s downstream taskId:0x%x (vgId:%d) detects current task vnode-transfer/leader-change/restart, not " + "send check again, roll-back needed", id, pRsp->downstreamTaskId, pRsp->downstreamNodeId); return 0; } From 5b42204dbae029db23c3bfac2399113cdbc8b176 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 25 Oct 2023 00:32:46 +0800 Subject: [PATCH 34/41] fix(stream): not clear role flag. --- source/libs/stream/src/streamMeta.c | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index c536d01516..f2dba090cf 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -230,9 +230,9 @@ _err: int32_t streamMetaReopen(SStreamMeta* pMeta) { streamMetaClear(pMeta); + // NOTE: role should not be changed during reopen meta pMeta->streamBackendRid = -1; pMeta->streamBackend = NULL; - pMeta->role = NODE_ROLE_UNINIT; char* defaultPath = taosMemoryCalloc(1, strlen(pMeta->path) + 128); sprintf(defaultPath, "%s%s%s", pMeta->path, TD_DIRSEP, "state"); @@ -858,7 +858,7 @@ void metaHbToMnode(void* param, void* tmrId) { // not leader not send msg if (pMeta->role != NODE_ROLE_LEADER) { - stInfo("vgId:%d follower not send hb to mnode", pMeta->vgId); + stInfo("vgId:%d role:%d not leader not send hb to mnode", pMeta->vgId, pMeta->role); taosReleaseRef(streamMetaId, rid); pMeta->pHbInfo->hbStart = 0; return; From 03165d53272efa841184074dca74fe3bbbab7c3c Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 25 Oct 2023 09:55:02 +0800 Subject: [PATCH 35/41] fix(stream): handle the repeatly issued nodeUpdate msg from dnode. --- include/libs/stream/tstream.h | 56 ++++++++++++++----------- source/dnode/mnode/impl/src/mndStream.c | 17 ++++---- source/dnode/vnode/src/inc/tq.h | 2 - source/dnode/vnode/src/tq/tq.c | 37 +++++++++++++--- source/libs/stream/src/streamDispatch.c | 5 +++ source/libs/stream/src/streamMeta.c | 8 ++-- 6 files changed, 80 insertions(+), 45 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index a027b4f9ba..d3717bc1e7 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -436,33 +436,38 @@ typedef struct STaskStartInfo { int32_t elapsedTime; } STaskStartInfo; +typedef struct STaskUpdateInfo { + SHashObj* pTasks; + int32_t transId; +} STaskUpdateInfo; + // meta typedef struct SStreamMeta { - char* path; - TDB* db; - TTB* pTaskDb; - TTB* pCheckpointDb; - SHashObj* pTasksMap; - SArray* pTaskList; // SArray - void* ahandle; - TXN* txn; - FTaskExpand* expandFunc; - int32_t vgId; - int64_t stage; - int32_t role; - STaskStartInfo startInfo; - SRWLatch lock; - int32_t walScanCounter; - void* streamBackend; - int64_t streamBackendRid; - SHashObj* pTaskBackendUnique; - TdThreadMutex backendMutex; - SMetaHbInfo* pHbInfo; - SHashObj* pUpdateTaskSet; - int32_t numOfStreamTasks; // this value should be increased when a new task is added into the meta - int32_t numOfPausedTasks; - int32_t chkptNotReadyTasks; - int64_t rid; + char* path; + TDB* db; + TTB* pTaskDb; + TTB* pCheckpointDb; + SHashObj* pTasksMap; + SArray* pTaskList; // SArray + void* ahandle; + TXN* txn; + FTaskExpand* expandFunc; + int32_t vgId; + int64_t stage; + int32_t role; + STaskStartInfo startInfo; + SRWLatch lock; + int32_t walScanCounter; + void* streamBackend; + int64_t streamBackendRid; + SHashObj* pTaskBackendUnique; + TdThreadMutex backendMutex; + SMetaHbInfo* pHbInfo; + STaskUpdateInfo updateInfo; + int32_t numOfStreamTasks; // this value should be increased when a new task is added into the meta + int32_t numOfPausedTasks; + int32_t chkptNotReadyTasks; + int64_t rid; int64_t chkpId; SArray* chkpSaved; @@ -664,6 +669,7 @@ typedef struct SNodeUpdateInfo { } SNodeUpdateInfo; typedef struct SStreamTaskNodeUpdateMsg { + int32_t transId; // to identify the msg int64_t streamId; int32_t taskId; SArray* pNodeList; // SArray diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index 194c1021f6..a2aa56dd6e 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -1883,18 +1883,19 @@ static int32_t mndProcessResumeStreamReq(SRpcMsg *pReq) { return TSDB_CODE_ACTION_IN_PROGRESS; } -static void initNodeUpdateMsg(SStreamTaskNodeUpdateMsg *pMsg, const SVgroupChangeInfo *pInfo, int64_t streamId, - int32_t taskId) { - pMsg->streamId = streamId; - pMsg->taskId = taskId; +static void initNodeUpdateMsg(SStreamTaskNodeUpdateMsg *pMsg, const SVgroupChangeInfo *pInfo, SStreamTaskId *pId, + int32_t transId) { + pMsg->streamId = pId->streamId; + pMsg->taskId = pId->taskId; + pMsg->transId = transId; pMsg->pNodeList = taosArrayInit(taosArrayGetSize(pInfo->pUpdateNodeList), sizeof(SNodeUpdateInfo)); taosArrayAddAll(pMsg->pNodeList, pInfo->pUpdateNodeList); } static int32_t doBuildStreamTaskUpdateMsg(void **pBuf, int32_t *pLen, SVgroupChangeInfo *pInfo, int32_t nodeId, - int64_t streamId, int32_t taskId) { + SStreamTaskId* pId, int32_t transId) { SStreamTaskNodeUpdateMsg req = {0}; - initNodeUpdateMsg(&req, pInfo, streamId, taskId); + initNodeUpdateMsg(&req, pInfo, pId, transId); int32_t code = 0; int32_t blen; @@ -1968,7 +1969,7 @@ void initTransAction(STransAction *pAction, void *pCont, int32_t contLen, int32_ // todo extract method: traverse stream tasks // build trans to update the epset static int32_t createStreamUpdateTrans(SStreamObj *pStream, SVgroupChangeInfo *pInfo, STrans* pTrans) { - mDebug("start to build stream:0x%" PRIx64 " task DAG update", pStream->uid); + mDebug("start to build stream:0x%" PRIx64 " tasks epset update", pStream->uid); taosWLockLatch(&pStream->lock); int32_t numOfLevels = taosArrayGetSize(pStream->tasks); @@ -1983,7 +1984,7 @@ static int32_t createStreamUpdateTrans(SStreamObj *pStream, SVgroupChangeInfo *p void *pBuf = NULL; int32_t len = 0; streamTaskUpdateEpsetInfo(pTask, pInfo->pUpdateNodeList); - doBuildStreamTaskUpdateMsg(&pBuf, &len, pInfo, pTask->info.nodeId, pTask->id.streamId, pTask->id.taskId); + doBuildStreamTaskUpdateMsg(&pBuf, &len, pInfo, pTask->info.nodeId, &pTask->id, pTrans->id); STransAction action = {0}; initTransAction(&action, pBuf, len, TDMT_VND_STREAM_TASK_UPDATE, &pTask->info.epSet); diff --git a/source/dnode/vnode/src/inc/tq.h b/source/dnode/vnode/src/inc/tq.h index 24c31b95c8..eac6603e8b 100644 --- a/source/dnode/vnode/src/inc/tq.h +++ b/source/dnode/vnode/src/inc/tq.h @@ -125,7 +125,6 @@ int32_t tqTaosxScanLog(STQ* pTq, STqHandle* pHandle, SPackedData submit, STaosxR int32_t tqAddBlockDataToRsp(const SSDataBlock* pBlock, SMqDataRsp* pRsp, int32_t numOfCols, int8_t precision); int32_t tqSendDataRsp(STqHandle* pHandle, const SRpcMsg* pMsg, const SMqPollReq* pReq, const SMqDataRsp* pRsp, int32_t type, int32_t vgId); -//int32_t tqPushDataRsp(STqHandle* pHandle, int32_t vgId); int32_t tqPushEmptyDataRsp(STqHandle* pHandle, int32_t vgId); // tqMeta @@ -133,7 +132,6 @@ int32_t tqMetaOpen(STQ* pTq); int32_t tqMetaClose(STQ* pTq); int32_t tqMetaSaveHandle(STQ* pTq, const char* key, const STqHandle* pHandle); int32_t tqMetaDeleteHandle(STQ* pTq, const char* key); -//int32_t tqMetaRestoreHandle(STQ* pTq); int32_t tqMetaSaveCheckInfo(STQ* pTq, const char* key, const void* value, int32_t vLen); int32_t tqMetaDeleteCheckInfo(STQ* pTq, const char* key); int32_t tqMetaRestoreCheckInfo(STQ* pTq); diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 711c9a52bc..907dc8d88a 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -20,6 +20,12 @@ typedef struct { int8_t inited; } STqMgmt; +typedef struct STaskUpdateEntry { + int64_t streamId; + int32_t taskId; + int32_t transId; +} STaskUpdateEntry; + static STqMgmt tqMgmt = {0}; // 0: not init @@ -1869,7 +1875,26 @@ int32_t tqProcessTaskUpdateReq(STQ* pTq, SRpcMsg* pMsg) { } SStreamTask* pTask = *ppTask; - tqDebug("s-task:%s receive nodeEp update msg from mnode", pTask->id.idStr); + + if (pMeta->updateInfo.transId != req.transId) { + pMeta->updateInfo.transId = req.transId; + tqDebug("s-task:%s receive new trans to update nodeEp msg from mnode, transId:%d", pTask->id.idStr, req.transId); + // info needs to be kept till the new trans to update the nodeEp arrived. + taosHashClear(pMeta->updateInfo.pTasks); + } else { + tqDebug("s-task:%s recv trans to update nodeEp from mnode, transId:%d", pTask->id.idStr, req.transId); + } + + STaskUpdateEntry entry = {.streamId = req.streamId, .taskId = req.taskId, .transId = req.transId}; + void* exist = taosHashGet(pMeta->updateInfo.pTasks, &entry, sizeof(STaskUpdateEntry)); + if (exist != NULL) { + tqDebug("s-task:%s (vgId:%d) already update in trans:%d, discard the nodeEp update msg", pTask->id.idStr, vgId, + req.transId); + rsp.code = TSDB_CODE_SUCCESS; + taosWUnLockLatch(&pMeta->lock); + taosArrayDestroy(req.pNodeList); + return rsp.code; + } streamTaskUpdateEpsetInfo(pTask, req.pNodeList); streamTaskResetStatus(pTask); @@ -1899,12 +1924,14 @@ int32_t tqProcessTaskUpdateReq(STQ* pTq, SRpcMsg* pMsg) { } streamTaskStop(pTask); - taosHashPut(pMeta->pUpdateTaskSet, &pTask->id, sizeof(pTask->id), NULL, 0); + + // keep the already handled info + taosHashPut(pMeta->updateInfo.pTasks, &entry, sizeof(entry), NULL, 0); if (ppHTask != NULL) { streamTaskStop(*ppHTask); tqDebug("s-task:%s task nodeEp update completed, streamTask and related fill-history task closed", pTask->id.idStr); - taosHashPut(pMeta->pUpdateTaskSet, &(*ppHTask)->id, sizeof(pTask->id), NULL, 0); + taosHashPut(pMeta->updateInfo.pTasks, &(*ppHTask)->id, sizeof(pTask->id), NULL, 0); } else { tqDebug("s-task:%s task nodeEp update completed, streamTask closed", pTask->id.idStr); } @@ -1913,7 +1940,7 @@ int32_t tqProcessTaskUpdateReq(STQ* pTq, SRpcMsg* pMsg) { // possibly only handle the stream task. int32_t numOfTasks = streamMetaGetNumOfTasks(pMeta); - int32_t updateTasks = taosHashGetSize(pMeta->pUpdateTaskSet); + int32_t updateTasks = taosHashGetSize(pMeta->updateInfo.pTasks); pMeta->startInfo.startedAfterNodeUpdate = 1; @@ -1922,8 +1949,6 @@ int32_t tqProcessTaskUpdateReq(STQ* pTq, SRpcMsg* pMsg) { updateTasks, (numOfTasks - updateTasks)); taosWUnLockLatch(&pMeta->lock); } else { - taosHashClear(pMeta->pUpdateTaskSet); - if (!pTq->pVnode->restored) { tqDebug("vgId:%d vnode restore not completed, not restart the tasks, clear the start after nodeUpdate flag", vgId); pMeta->startInfo.startedAfterNodeUpdate = 0; diff --git a/source/libs/stream/src/streamDispatch.c b/source/libs/stream/src/streamDispatch.c index 750f5d6a43..cd69bc0d92 100644 --- a/source/libs/stream/src/streamDispatch.c +++ b/source/libs/stream/src/streamDispatch.c @@ -1175,6 +1175,9 @@ int32_t tEncodeStreamTaskUpdateMsg(SEncoder* pEncoder, const SStreamTaskNodeUpda if (tEncodeI64(pEncoder, pMsg->streamId) < 0) return -1; if (tEncodeI32(pEncoder, pMsg->taskId) < 0) return -1; + // todo this new attribute will be result in being incompatible with previous version + if (tEncodeI32(pEncoder, pMsg->transId) < 0) return -1; + int32_t size = taosArrayGetSize(pMsg->pNodeList); if (tEncodeI32(pEncoder, size) < 0) return -1; @@ -1193,6 +1196,8 @@ int32_t tDecodeStreamTaskUpdateMsg(SDecoder* pDecoder, SStreamTaskNodeUpdateMsg* if (tDecodeI64(pDecoder, &pMsg->streamId) < 0) return -1; if (tDecodeI32(pDecoder, &pMsg->taskId) < 0) return -1; + if (tDecodeI32(pDecoder, &pMsg->transId) < 0) return -1; + int32_t size = 0; if (tDecodeI32(pDecoder, &size) < 0) return -1; pMsg->pNodeList = taosArrayInit(size, sizeof(SNodeUpdateInfo)); diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index f2dba090cf..b13e49beb4 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -143,8 +143,8 @@ SStreamMeta* streamMetaOpen(const char* path, void* ahandle, FTaskExpand expandF goto _err; } - pMeta->pUpdateTaskSet = taosHashInit(64, fp, false, HASH_NO_LOCK); - if (pMeta->pUpdateTaskSet == NULL) { + pMeta->updateInfo.pTasks = taosHashInit(64, fp, false, HASH_NO_LOCK); + if (pMeta->updateInfo.pTasks == NULL) { goto _err; } @@ -219,7 +219,7 @@ _err: if (pMeta->pCheckpointDb) tdbTbClose(pMeta->pCheckpointDb); if (pMeta->db) tdbClose(pMeta->db); if (pMeta->pHbInfo) taosMemoryFreeClear(pMeta->pHbInfo); - if (pMeta->pUpdateTaskSet) taosHashCleanup(pMeta->pUpdateTaskSet); + if (pMeta->updateInfo.pTasks) taosHashCleanup(pMeta->updateInfo.pTasks); if (pMeta->startInfo.pReadyTaskSet) taosHashCleanup(pMeta->startInfo.pReadyTaskSet); taosMemoryFree(pMeta); @@ -340,7 +340,7 @@ void streamMetaCloseImpl(void* arg) { taosHashCleanup(pMeta->pTasksMap); taosHashCleanup(pMeta->pTaskBackendUnique); - taosHashCleanup(pMeta->pUpdateTaskSet); + taosHashCleanup(pMeta->updateInfo.pTasks); taosHashCleanup(pMeta->startInfo.pReadyTaskSet); taosMemoryFree(pMeta->pHbInfo); From 985371e8418ce22cbc5eb367a951f3b262be5c24 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 25 Oct 2023 10:01:20 +0800 Subject: [PATCH 36/41] fix(stream): resolve the nodeUpdate msg compatible issue. --- source/libs/stream/src/streamDispatch.c | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/source/libs/stream/src/streamDispatch.c b/source/libs/stream/src/streamDispatch.c index cd69bc0d92..e34ec07eac 100644 --- a/source/libs/stream/src/streamDispatch.c +++ b/source/libs/stream/src/streamDispatch.c @@ -1175,9 +1175,6 @@ int32_t tEncodeStreamTaskUpdateMsg(SEncoder* pEncoder, const SStreamTaskNodeUpda if (tEncodeI64(pEncoder, pMsg->streamId) < 0) return -1; if (tEncodeI32(pEncoder, pMsg->taskId) < 0) return -1; - // todo this new attribute will be result in being incompatible with previous version - if (tEncodeI32(pEncoder, pMsg->transId) < 0) return -1; - int32_t size = taosArrayGetSize(pMsg->pNodeList); if (tEncodeI32(pEncoder, size) < 0) return -1; @@ -1187,6 +1184,9 @@ int32_t tEncodeStreamTaskUpdateMsg(SEncoder* pEncoder, const SStreamTaskNodeUpda if (tEncodeSEpSet(pEncoder, &pInfo->prevEp) < 0) return -1; if (tEncodeSEpSet(pEncoder, &pInfo->newEp) < 0) return -1; } + + // todo this new attribute will be result in being incompatible with previous version + if (tEncodeI32(pEncoder, pMsg->transId) < 0) return -1; tEndEncode(pEncoder); return pEncoder->pos; } @@ -1196,8 +1196,6 @@ int32_t tDecodeStreamTaskUpdateMsg(SDecoder* pDecoder, SStreamTaskNodeUpdateMsg* if (tDecodeI64(pDecoder, &pMsg->streamId) < 0) return -1; if (tDecodeI32(pDecoder, &pMsg->taskId) < 0) return -1; - if (tDecodeI32(pDecoder, &pMsg->transId) < 0) return -1; - int32_t size = 0; if (tDecodeI32(pDecoder, &size) < 0) return -1; pMsg->pNodeList = taosArrayInit(size, sizeof(SNodeUpdateInfo)); @@ -1209,6 +1207,8 @@ int32_t tDecodeStreamTaskUpdateMsg(SDecoder* pDecoder, SStreamTaskNodeUpdateMsg* taosArrayPush(pMsg->pNodeList, &info); } + if (tDecodeI32(pDecoder, &pMsg->transId) < 0) return -1; + tEndDecode(pDecoder); return 0; } From 07e9d56e077a538a95c626e0fb9b9fecf7d41e38 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 25 Oct 2023 11:42:46 +0800 Subject: [PATCH 37/41] fix(stream): set the retrycode for checkpoint source transaction. --- source/dnode/mnode/impl/src/mndStream.c | 27 ++++++++++++++----------- 1 file changed, 15 insertions(+), 12 deletions(-) diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index a2aa56dd6e..a4f69a1064 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -84,7 +84,8 @@ static SVgroupChangeInfo mndFindChangedNodeInfo(SMnode *pMnode, const SArray *pP static STrans *doCreateTrans(SMnode *pMnode, SStreamObj *pStream, const char *name); static int32_t mndPersistTransLog(SStreamObj *pStream, STrans *pTrans); -static void initTransAction(STransAction *pAction, void *pCont, int32_t contLen, int32_t msgType, const SEpSet *pEpset); +static void initTransAction(STransAction *pAction, void *pCont, int32_t contLen, int32_t msgType, const SEpSet *pEpset, + int32_t retryCode); static int32_t createStreamUpdateTrans(SStreamObj *pStream, SVgroupChangeInfo *pInfo, STrans *pTrans); static void removeStreamTasksInBuf(SStreamObj *pStream, SStreamExecInfo *pExecNode); static void keepStreamTasksInBuf(SStreamObj *pStream, SStreamExecInfo *pExecNode); @@ -516,7 +517,7 @@ int32_t mndPersistTaskDeployReq(STrans *pTrans, SStreamTask *pTask) { STransAction action = {0}; action.mTraceId = pTrans->mTraceId; - initTransAction(&action, buf, tlen, TDMT_STREAM_TASK_DEPLOY, &pTask->info.epSet); + initTransAction(&action, buf, tlen, TDMT_STREAM_TASK_DEPLOY, &pTask->info.epSet, 0); if (mndTransAppendRedoAction(pTrans, &action) != 0) { taosMemoryFree(buf); return -1; @@ -688,7 +689,7 @@ static int32_t mndPersistTaskDropReq(STrans *pTrans, SStreamTask *pTask) { pReq->streamId = pTask->id.streamId; STransAction action = {0}; - initTransAction(&action, pReq, sizeof(SVDropStreamTaskReq), TDMT_STREAM_TASK_DROP, &pTask->info.epSet); + initTransAction(&action, pReq, sizeof(SVDropStreamTaskReq), TDMT_STREAM_TASK_DROP, &pTask->info.epSet, 0); if (mndTransAppendRedoAction(pTrans, &action) != 0) { taosMemoryFree(pReq); return -1; @@ -1068,7 +1069,7 @@ static int32_t mndAddStreamCheckpointToTrans(STrans *pTrans, SStreamObj *pStream STransAction action = {0}; SEpSet epset = mndGetVgroupEpset(pMnode, pVgObj); - initTransAction(&action, buf, tlen, TDMT_VND_STREAM_CHECK_POINT_SOURCE, &epset); + initTransAction(&action, buf, tlen, TDMT_VND_STREAM_CHECK_POINT_SOURCE, &epset, TSDB_CODE_SYN_PROPOSE_NOT_READY); mndReleaseVgroup(pMnode, pVgObj); if (mndTransAppendRedoAction(pTrans, &action) != 0) { @@ -1646,7 +1647,7 @@ static int32_t mndPauseStreamTask(STrans *pTrans, SStreamTask *pTask) { pReq->streamId = pTask->id.streamId; STransAction action = {0}; - initTransAction(&action, pReq, sizeof(SVPauseStreamTaskReq), TDMT_STREAM_TASK_PAUSE, &pTask->info.epSet); + initTransAction(&action, pReq, sizeof(SVPauseStreamTaskReq), TDMT_STREAM_TASK_PAUSE, &pTask->info.epSet, 0); if (mndTransAppendRedoAction(pTrans, &action) != 0) { taosMemoryFree(pReq); return -1; @@ -1779,7 +1780,7 @@ static int32_t mndResumeStreamTask(STrans *pTrans, SStreamTask *pTask, int8_t ig pReq->igUntreated = igUntreated; STransAction action = {0}; - initTransAction(&action, pReq, sizeof(SVResumeStreamTaskReq), TDMT_STREAM_TASK_RESUME, &pTask->info.epSet); + initTransAction(&action, pReq, sizeof(SVResumeStreamTaskReq), TDMT_STREAM_TASK_RESUME, &pTask->info.epSet, 0); if (mndTransAppendRedoAction(pTrans, &action) != 0) { taosMemoryFree(pReq); return -1; @@ -1959,19 +1960,21 @@ int32_t mndPersistTransLog(SStreamObj *pStream, STrans *pTrans) { return 0; } -void initTransAction(STransAction *pAction, void *pCont, int32_t contLen, int32_t msgType, const SEpSet *pEpset) { +void initTransAction(STransAction *pAction, void *pCont, int32_t contLen, int32_t msgType, const SEpSet *pEpset, + int32_t retryCode) { pAction->epSet = *pEpset; pAction->contLen = contLen; pAction->pCont = pCont; pAction->msgType = msgType; + pAction->retryCode = retryCode; } // todo extract method: traverse stream tasks // build trans to update the epset -static int32_t createStreamUpdateTrans(SStreamObj *pStream, SVgroupChangeInfo *pInfo, STrans* pTrans) { - mDebug("start to build stream:0x%" PRIx64 " tasks epset update", pStream->uid); +static int32_t createStreamUpdateTrans(SStreamObj *pStream, SVgroupChangeInfo *pInfo, STrans *pTrans) { + mDebug("start to build stream:0x%" PRIx64 " tasks epset update", pStream->uid); - taosWLockLatch(&pStream->lock); + taosWLockLatch(&pStream->lock); int32_t numOfLevels = taosArrayGetSize(pStream->tasks); for (int32_t j = 0; j < numOfLevels; ++j) { @@ -1987,7 +1990,7 @@ static int32_t createStreamUpdateTrans(SStreamObj *pStream, SVgroupChangeInfo *p doBuildStreamTaskUpdateMsg(&pBuf, &len, pInfo, pTask->info.nodeId, &pTask->id, pTrans->id); STransAction action = {0}; - initTransAction(&action, pBuf, len, TDMT_VND_STREAM_TASK_UPDATE, &pTask->info.epSet); + initTransAction(&action, pBuf, len, TDMT_VND_STREAM_TASK_UPDATE, &pTask->info.epSet, 0); if (mndTransAppendRedoAction(pTrans, &action) != 0) { taosMemoryFree(pBuf); taosWUnLockLatch(&pStream->lock); @@ -2472,7 +2475,7 @@ int32_t createStreamResetStatusTrans(SMnode* pMnode, SStreamObj* pStream) { pReq->streamId = pTask->id.streamId; STransAction action = {0}; - initTransAction(&action, pReq, sizeof(SVResetStreamTaskReq), TDMT_VND_STREAM_TASK_RESET, &pTask->info.epSet); + initTransAction(&action, pReq, sizeof(SVResetStreamTaskReq), TDMT_VND_STREAM_TASK_RESET, &pTask->info.epSet, 0); if (mndTransAppendRedoAction(pTrans, &action) != 0) { taosMemoryFree(pReq); taosWUnLockLatch(&pStream->lock); From f36b24ed24616e5fc995fa317114d0bef30c758b Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 25 Oct 2023 14:16:36 +0800 Subject: [PATCH 38/41] fix(stream): disable follower nodes exec reset. --- source/dnode/vnode/src/vnd/vnodeSvr.c | 2 +- source/libs/stream/src/streamStart.c | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/source/dnode/vnode/src/vnd/vnodeSvr.c b/source/dnode/vnode/src/vnd/vnodeSvr.c index efa722d41a..98e40a7eab 100644 --- a/source/dnode/vnode/src/vnd/vnodeSvr.c +++ b/source/dnode/vnode/src/vnd/vnodeSvr.c @@ -584,7 +584,7 @@ int32_t vnodeProcessWriteMsg(SVnode *pVnode, SRpcMsg *pMsg, int64_t ver, SRpcMsg } } break; case TDMT_VND_STREAM_TASK_RESET: { - if (pVnode->restored/* && vnodeIsLeader(pVnode)*/) { + if (pVnode->restored && vnodeIsLeader(pVnode)) { tqProcessTaskResetReq(pVnode->pTq, pMsg); } } break; diff --git a/source/libs/stream/src/streamStart.c b/source/libs/stream/src/streamStart.c index 56910c0c53..8b2fb4b9b2 100644 --- a/source/libs/stream/src/streamStart.c +++ b/source/libs/stream/src/streamStart.c @@ -399,8 +399,8 @@ int32_t streamProcessCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* pRs } else { if (pRsp->status == TASK_DOWNSTREAM_NOT_LEADER) { stError( - "s-task:%s downstream taskId:0x%x (vgId:%d) detects current task vnode-transfer/leader-change/restart, not " - "send check again, roll-back needed", + "s-task:%s downstream taskId:0x%x (vgId:%d) not leader, self dispatch epset needs to be updated, not check " + "downstream again, nodeUpdate needed", id, pRsp->downstreamTaskId, pRsp->downstreamNodeId); return 0; } From 92207313137db2a85089152936bb9b8fb177078e Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 25 Oct 2023 14:51:59 +0800 Subject: [PATCH 39/41] other: add some logs. --- source/dnode/mnode/impl/src/mndStream.c | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index a4f69a1064..8fe55c2598 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -2038,7 +2038,7 @@ static SVgroupChangeInfo mndFindChangedNodeInfo(SMnode *pMnode, const SArray *pP char buf[256] = {0}; EPSET_TO_STR(&pCurrent->epset, buf); - mDebug("nodeId:%d epset changed detected, old:%s:%d -> new:%s", pCurrent->nodeId, pPrevEp->fqdn, + mDebug("nodeId:%d restart/epset changed detected, old:%s:%d -> new:%s", pCurrent->nodeId, pPrevEp->fqdn, pPrevEp->port, buf); SNodeUpdateInfo updateInfo = {.nodeId = pPrevEntry->nodeId}; @@ -2077,6 +2077,9 @@ static SArray *mndTakeVgroupSnapshot(SMnode *pMnode) { entry.nodeId = pVgroup->vgId; entry.hbTimestamp = -1; + char buf[256] = {0}; + EPSET_TO_STR(&entry.epset, buf); + mDebug("take node snapshot, nodeId:%d %s", entry.nodeId, buf); taosArrayPush(pVgroupListSnapshot, &entry); sdbRelease(pSdb, pVgroup); } @@ -2187,6 +2190,10 @@ static SArray *extractNodeListFromStream(SMnode *pMnode) { while ((pIter = taosHashIterate(pHash, pIter)) != NULL) { SNodeEntry *pEntry = (SNodeEntry *)pIter; taosArrayPush(plist, pEntry); + + char buf[256] = {0}; + EPSET_TO_STR(&pEntry->epset, buf); + mDebug("extract nodeInfo from stream obj, nodeId:%d, %s", pEntry->nodeId, buf); } taosHashCleanup(pHash); From 9baff82f85048189699a4f3fc8597fb5dd50c2ba Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 25 Oct 2023 18:14:05 +0800 Subject: [PATCH 40/41] test:update test case. --- tests/script/tsim/stream/distributeIntervalRetrive0.sim | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/script/tsim/stream/distributeIntervalRetrive0.sim b/tests/script/tsim/stream/distributeIntervalRetrive0.sim index 052bf441d5..39c3353429 100644 --- a/tests/script/tsim/stream/distributeIntervalRetrive0.sim +++ b/tests/script/tsim/stream/distributeIntervalRetrive0.sim @@ -267,6 +267,7 @@ sql create table t1 using st tags(1,1,1); sql create table t2 using st tags(2,2,2); sql create stream streams1 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 delete_mark 20s into streamt1 as select _wstart as c0, count(*) c1, count(a) c2 from st interval(10s) ; +sleep 1000 sql insert into t1 values(1648791211000,1,2,3); sql insert into t1 values(1262275200000,2,2,3); From c95cd8da8bf91d974c8c5d1e084b643026e6debd Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 25 Oct 2023 18:14:28 +0800 Subject: [PATCH 41/41] refactor(stream): do some internal refactor. --- source/dnode/mnode/impl/src/mndStream.c | 24 ++++++++++++++---------- 1 file changed, 14 insertions(+), 10 deletions(-) diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index 8fe55c2598..14bdb73b4f 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -2038,8 +2038,8 @@ static SVgroupChangeInfo mndFindChangedNodeInfo(SMnode *pMnode, const SArray *pP char buf[256] = {0}; EPSET_TO_STR(&pCurrent->epset, buf); - mDebug("nodeId:%d restart/epset changed detected, old:%s:%d -> new:%s", pCurrent->nodeId, pPrevEp->fqdn, - pPrevEp->port, buf); + mDebug("nodeId:%d restart/epset changed detected, old:%s:%d -> new:%s, stageUpdate:%d", pCurrent->nodeId, + pPrevEp->fqdn, pPrevEp->port, buf, pPrevEntry->stageUpdated); SNodeUpdateInfo updateInfo = {.nodeId = pPrevEntry->nodeId}; epsetAssign(&updateInfo.prevEp, &pPrevEntry->epset); @@ -2216,7 +2216,7 @@ static void doExtractTasksFromStream(SMnode *pMnode) { } } -static int32_t doRemoveFromTask(SStreamExecInfo* pExecNode, STaskId* pRemovedId) { +static int32_t doRemoveTasks(SStreamExecInfo* pExecNode, STaskId* pRemovedId) { void *p = taosHashGet(pExecNode->pTaskMap, pRemovedId, sizeof(*pRemovedId)); if (p != NULL) { @@ -2249,7 +2249,7 @@ static bool taskNodeExists(SArray* pList, int32_t nodeId) { } int32_t removeExpirednodeEntryAndTask(SArray *pNodeSnapshot) { - SArray* pRemoveTaskList = taosArrayInit(4, sizeof(STaskId)); + SArray* pRemovedTasks = taosArrayInit(4, sizeof(STaskId)); int32_t numOfTask = taosArrayGetSize(execInfo.pTaskList); for(int32_t i = 0; i < numOfTask; ++i) { @@ -2258,16 +2258,16 @@ int32_t removeExpirednodeEntryAndTask(SArray *pNodeSnapshot) { bool existed = taskNodeExists(pNodeSnapshot, pEntry->nodeId); if (!existed) { - taosArrayPush(pRemoveTaskList, pId); + taosArrayPush(pRemovedTasks, pId); } } - for(int32_t i = 0; i < taosArrayGetSize(pRemoveTaskList); ++i) { - STaskId* pId = taosArrayGet(pRemoveTaskList, i); - doRemoveFromTask(&execInfo, pId); + for(int32_t i = 0; i < taosArrayGetSize(pRemovedTasks); ++i) { + STaskId* pId = taosArrayGet(pRemovedTasks, i); + doRemoveTasks(&execInfo, pId); } - mDebug("remove invalid stream tasks:%d, remain:%d", (int32_t)taosArrayGetSize(pRemoveTaskList), + mDebug("remove invalid stream tasks:%d, remain:%d", (int32_t)taosArrayGetSize(pRemovedTasks), (int32_t) taosArrayGetSize(execInfo.pTaskList)); int32_t size = taosArrayGetSize(pNodeSnapshot); @@ -2287,7 +2287,8 @@ int32_t removeExpirednodeEntryAndTask(SArray *pNodeSnapshot) { execInfo.pNodeEntryList = taosArrayDestroy(execInfo.pNodeEntryList); execInfo.pNodeEntryList = pValidNodeEntryList; - taosArrayDestroy(pRemoveTaskList); + mDebug("remain %d valid node entries", (int32_t) taosArrayGetSize(pValidNodeEntryList)); + taosArrayDestroy(pRemovedTasks); return 0; } @@ -2336,6 +2337,9 @@ static int32_t mndProcessNodeCheckReq(SRpcMsg *pMsg) { taosArrayDestroy(execInfo.pNodeEntryList); execInfo.pNodeEntryList = pNodeSnapshot; execInfo.ts = ts; + } else { + mDebug("unexpect code during create nodeUpdate trans, code:%s", tstrerror(code)); + taosArrayDestroy(pNodeSnapshot); } } else { mDebug("no update found in nodeList");