From aaefa71a3e0b53f7c8f2e4d46b278b5c2525bfc1 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 6 May 2024 16:40:49 +0800 Subject: [PATCH 01/30] fix(stream): set global close flag in the streamMeta. --- include/libs/stream/tstream.h | 1 + source/libs/stream/src/streamMeta.c | 12 +++++++++++- 2 files changed, 12 insertions(+), 1 deletion(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 14aae0b96ab3..aa17853454fd 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -530,6 +530,7 @@ typedef struct SStreamMeta { int32_t vgId; int64_t stage; int32_t role; + bool closeFlag; bool sendMsgBeforeClosing; // send hb to mnode before close all tasks when switch to follower. STaskStartInfo startInfo; TdThreadRwlock lock; diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 3e3454eb93dd..a27a1f7c898b 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -373,6 +373,7 @@ SStreamMeta* streamMetaOpen(const char* path, void* ahandle, FTaskExpand expandF pMeta->numOfPausedTasks = 0; pMeta->numOfStreamTasks = 0; + pMeta->closeFlag = false; stInfo("vgId:%d open stream meta succ, latest checkpoint:%" PRId64 ", stage:%" PRId64, vgId, pMeta->chkpId, stage); @@ -1282,6 +1283,8 @@ void streamMetaNotifyClose(SStreamMeta* pMeta) { streamMetaWLock(pMeta); + pMeta->closeFlag = true; + void* pIter = NULL; while (1) { pIter = taosHashIterate(pMeta->pTasksMap, pIter); @@ -1456,6 +1459,7 @@ static SArray* prepareBeforeStartTasks(SStreamMeta* pMeta) { int32_t streamMetaStartAllTasks(SStreamMeta* pMeta) { int32_t code = TSDB_CODE_SUCCESS; int32_t vgId = pMeta->vgId; + int64_t now = taosGetTimestampMs(); int32_t numOfTasks = taosArrayGetSize(pMeta->pTaskList); stInfo("vgId:%d start to check all %d stream task(s) downstream status", vgId, numOfTasks); @@ -1465,7 +1469,13 @@ int32_t streamMetaStartAllTasks(SStreamMeta* pMeta) { return TSDB_CODE_SUCCESS; } - int64_t now = taosGetTimestampMs(); + streamMetaRLock(pMeta); + if (pMeta->closeFlag) { + streamMetaRUnLock(pMeta); + stError("vgId:%d vnode is closed, not start check task(s) downstream status", vgId); + return TSDB_CODE_SUCCESS; + } + streamMetaRUnLock(pMeta); SArray* pTaskList = prepareBeforeStartTasks(pMeta); numOfTasks = taosArrayGetSize(pTaskList); From 5a4a1a6400170025c79208fc4054289c784d24ac Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 6 May 2024 17:03:09 +0800 Subject: [PATCH 02/30] fix(stream): set global close flag in the streamMeta. --- source/libs/stream/src/streamMeta.c | 25 +++++++++++++++---------- 1 file changed, 15 insertions(+), 10 deletions(-) diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index a27a1f7c898b..f0661eedd662 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -1442,10 +1442,17 @@ void streamMetaUpdateStageRole(SStreamMeta* pMeta, int64_t stage, bool isLeader) } } -static SArray* prepareBeforeStartTasks(SStreamMeta* pMeta) { +static int32_t prepareBeforeStartTasks(SStreamMeta* pMeta, SArray** pList) { streamMetaWLock(pMeta); - SArray* pTaskList = taosArrayDup(pMeta->pTaskList, NULL); + if (pMeta->closeFlag) { + streamMetaWUnLock(pMeta); + stError("vgId:%d vnode is closed, not start check task(s) downstream status", pMeta->vgId); + return -1; + } + + *pList = taosArrayDup(pMeta->pTaskList, NULL); + taosHashClear(pMeta->startInfo.pReadyTaskSet); taosHashClear(pMeta->startInfo.pFailedTaskSet); pMeta->startInfo.startTs = taosGetTimestampMs(); @@ -1453,7 +1460,7 @@ static SArray* prepareBeforeStartTasks(SStreamMeta* pMeta) { streamMetaResetTaskStatus(pMeta); streamMetaWUnLock(pMeta); - return pTaskList; + return TSDB_CODE_SUCCESS; } int32_t streamMetaStartAllTasks(SStreamMeta* pMeta) { @@ -1465,19 +1472,17 @@ int32_t streamMetaStartAllTasks(SStreamMeta* pMeta) { stInfo("vgId:%d start to check all %d stream task(s) downstream status", vgId, numOfTasks); if (numOfTasks == 0) { - stInfo("vgId:%d start tasks completed", pMeta->vgId); + stInfo("vgId:%d no tasks to be started", pMeta->vgId); return TSDB_CODE_SUCCESS; } - streamMetaRLock(pMeta); - if (pMeta->closeFlag) { - streamMetaRUnLock(pMeta); - stError("vgId:%d vnode is closed, not start check task(s) downstream status", vgId); + SArray* pTaskList = NULL; + code = prepareBeforeStartTasks(pMeta, &pTaskList); + if (code != TSDB_CODE_SUCCESS) { + ASSERT(pTaskList == NULL); return TSDB_CODE_SUCCESS; } - streamMetaRUnLock(pMeta); - SArray* pTaskList = prepareBeforeStartTasks(pMeta); numOfTasks = taosArrayGetSize(pTaskList); // broadcast the check downstream tasks msg From 24ffe96a4c8815e0e51537b1d848ecbbe8cd8bc3 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 6 May 2024 17:03:41 +0800 Subject: [PATCH 03/30] refactor: do some internal refactor. --- include/common/rsync.h | 6 +- include/dnode/vnode/tqCommon.h | 1 - include/libs/stream/tstream.h | 10 + source/common/src/rsync.c | 14 +- source/dnode/vnode/src/tq/tqStreamTask.c | 18 +- source/dnode/vnode/src/tqCommon/tqCommon.c | 80 +----- source/libs/stream/src/stream.c | 308 --------------------- source/libs/stream/src/streamCheckStatus.c | 44 +-- source/libs/stream/src/streamCheckpoint.c | 10 +- source/libs/stream/src/streamDispatch.c | 123 ++++++++ source/libs/stream/src/streamMeta.c | 26 +- source/libs/stream/src/streamQueue.c | 4 +- source/libs/stream/src/streamSched.c | 129 +++++++++ source/libs/stream/src/streamTask.c | 55 +++- source/libs/stream/src/streamTimer.c | 40 +++ 15 files changed, 416 insertions(+), 452 deletions(-) delete mode 100644 source/libs/stream/src/stream.c create mode 100644 source/libs/stream/src/streamSched.c create mode 100644 source/libs/stream/src/streamTimer.c diff --git a/include/common/rsync.h b/include/common/rsync.h index 6cce645d1e57..d570311694a5 100644 --- a/include/common/rsync.h +++ b/include/common/rsync.h @@ -13,9 +13,9 @@ extern "C" { void stopRsync(); void startRsync(); -int uploadRsync(char* id, char* path); -int downloadRsync(char* id, char* path); -int deleteRsync(char* id); +int uploadRsync(const char* id, const char* path); +int downloadRsync(const char* id, const char* path); +int deleteRsync(const char* id); #ifdef __cplusplus } diff --git a/include/dnode/vnode/tqCommon.h b/include/dnode/vnode/tqCommon.h index cb616f7afc3b..ce04ec695369 100644 --- a/include/dnode/vnode/tqCommon.h +++ b/include/dnode/vnode/tqCommon.h @@ -33,7 +33,6 @@ int32_t tqStreamTaskProcessDeployReq(SStreamMeta* pMeta, SMsgCb* cb, int64_t sve bool isLeader, bool restored); int32_t tqStreamTaskProcessDropReq(SStreamMeta* pMeta, char* msg, int32_t msgLen); int32_t tqStreamTaskProcessRunReq(SStreamMeta* pMeta, SRpcMsg* pMsg, bool isLeader); -int32_t tqStreamTaskResetStatus(SStreamMeta* pMeta); int32_t tqStartTaskCompleteCallback(SStreamMeta* pMeta); int32_t tqStreamTasksGetTotalNum(SStreamMeta* pMeta); int32_t tqStreamTaskProcessTaskResetReq(SStreamMeta* pMeta, SRpcMsg* pMsg); diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index aa17853454fd..a8a695cc5b1c 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -554,6 +554,12 @@ typedef struct SStreamMeta { void* bkdChkptMgt; } SStreamMeta; +typedef struct STaskUpdateEntry { + int64_t streamId; + int32_t taskId; + int32_t transId; +} STaskUpdateEntry; + int32_t tEncodeStreamEpInfo(SEncoder* pEncoder, const SStreamChildEpInfo* pInfo); int32_t tDecodeStreamEpInfo(SDecoder* pDecoder, SStreamChildEpInfo* pInfo); @@ -794,6 +800,8 @@ void streamTaskInputFail(SStreamTask* pTask); int32_t streamExecTask(SStreamTask* pTask); int32_t streamResumeTask(SStreamTask* pTask); int32_t streamSchedExec(SStreamTask* pTask); +int32_t streamTaskSchedTask(SMsgCb* pMsgCb, int32_t vgId, int64_t streamId, int32_t taskId, int32_t execType); + bool streamTaskShouldStop(const SStreamTask* pStatus); bool streamTaskShouldPause(const SStreamTask* pStatus); bool streamTaskIsIdle(const SStreamTask* pTask); @@ -889,6 +897,8 @@ int32_t streamMetaAddTaskLaunchResult(SStreamMeta* pMeta, int64_t streamId, int64_t endTs, bool ready); int32_t streamMetaResetTaskStatus(SStreamMeta* pMeta); int32_t streamMetaAddFailedTask(SStreamMeta* pMeta, int64_t streamId, int32_t taskId); +void streamMetaAddIntoUpdateTaskList(SStreamMeta* pMeta, SStreamTask* pTask, SStreamTask* pHTask, int32_t transId, + int64_t startTs); void streamMetaRLock(SStreamMeta* pMeta); void streamMetaRUnLock(SStreamMeta* pMeta); diff --git a/source/common/src/rsync.c b/source/common/src/rsync.c index ffab85761e03..7aec0077e7ab 100644 --- a/source/common/src/rsync.c +++ b/source/common/src/rsync.c @@ -9,7 +9,7 @@ #define ERRNO_ERR_DATA errno,strerror(errno) // deleteRsync function produce empty directories, traverse base directory to remove them -static void removeEmptyDir(){ +static void removeEmptyDir() { TdDirPtr pDir = taosOpenDir(tsCheckpointBackupDir); if (pDir == NULL) return; @@ -53,7 +53,7 @@ static void changeDirFromWindowsToLinux(char* from, char* to){ } #endif -static int generateConfigFile(char* confDir){ +static int generateConfigFile(char* confDir) { TdFilePtr pFile = taosOpenFile(confDir, TD_FILE_CREATE | TD_FILE_WRITE | TD_FILE_TRUNC); if (pFile == NULL) { uError("[rsync] open conf file error, dir:%s,"ERRNO_ERR_FORMAT, confDir, ERRNO_ERR_DATA); @@ -111,7 +111,7 @@ static int execCommand(char* command){ return code; } -void stopRsync(){ +void stopRsync() { int code = #ifdef WINDOWS system("taskkill /f /im rsync.exe"); @@ -125,7 +125,7 @@ void stopRsync(){ uDebug("[rsync] stop rsync server successful"); } -void startRsync(){ +void startRsync() { if(taosMulMkDir(tsCheckpointBackupDir) != 0){ uError("[rsync] build checkpoint backup dir failed, dir:%s,"ERRNO_ERR_FORMAT, tsCheckpointBackupDir, ERRNO_ERR_DATA); return; @@ -151,7 +151,7 @@ void startRsync(){ uDebug("[rsync] start server successful"); } -int uploadRsync(char* id, char* path){ +int uploadRsync(const char* id, const char* path) { #ifdef WINDOWS char pathTransform[PATH_MAX] = {0}; changeDirFromWindowsToLinux(path, pathTransform); @@ -188,7 +188,7 @@ int uploadRsync(char* id, char* path){ return 0; } -int downloadRsync(char* id, char* path){ +int downloadRsync(const char* id, const char* path) { #ifdef WINDOWS char pathTransform[PATH_MAX] = {0}; changeDirFromWindowsToLinux(path, pathTransform); @@ -212,7 +212,7 @@ int downloadRsync(char* id, char* path){ return 0; } -int deleteRsync(char* id){ +int deleteRsync(const char* id) { char* tmp = "./tmp_empty/"; int code = taosMkDir(tmp); if(code != 0){ diff --git a/source/dnode/vnode/src/tq/tqStreamTask.c b/source/dnode/vnode/src/tq/tqStreamTask.c index 19e53c7d1502..e22ebb9d84af 100644 --- a/source/dnode/vnode/src/tq/tqStreamTask.c +++ b/source/dnode/vnode/src/tq/tqStreamTask.c @@ -149,27 +149,13 @@ int32_t tqScanWalAsync(STQ* pTq, bool ckPause) { return 0; } - SStreamTaskRunReq* pRunReq = rpcMallocCont(sizeof(SStreamTaskRunReq)); - if (pRunReq == NULL) { - terrno = TSDB_CODE_OUT_OF_MEMORY; - tqError("vgId:%d failed to create msg to start wal scanning to launch stream tasks, code:%s", vgId, terrstr()); - streamMetaWUnLock(pMeta); - return -1; - } - tqDebug("vgId:%d create msg to start wal scan to launch stream tasks, numOfTasks:%d, vnd restored:%d", vgId, numOfTasks, alreadyRestored); - pRunReq->head.vgId = vgId; - pRunReq->streamId = 0; - pRunReq->taskId = 0; - pRunReq->reqType = STREAM_EXEC_T_EXTRACT_WAL_DATA; - - SRpcMsg msg = {.msgType = TDMT_STREAM_TASK_RUN, .pCont = pRunReq, .contLen = sizeof(SStreamTaskRunReq)}; - tmsgPutToQueue(&pTq->pVnode->msgCb, STREAM_QUEUE, &msg); + int32_t code = streamTaskSchedTask(&pTq->pVnode->msgCb, vgId, 0, 0, STREAM_EXEC_T_EXTRACT_WAL_DATA); streamMetaWUnLock(pMeta); - return 0; + return code; } int32_t tqStopStreamTasksAsync(STQ* pTq) { diff --git a/source/dnode/vnode/src/tqCommon/tqCommon.c b/source/dnode/vnode/src/tqCommon/tqCommon.c index 0b3cf6eac813..85fa5cc118ca 100644 --- a/source/dnode/vnode/src/tqCommon/tqCommon.c +++ b/source/dnode/vnode/src/tqCommon/tqCommon.c @@ -17,12 +17,6 @@ #include "tq.h" #include "tstream.h" -typedef struct STaskUpdateEntry { - int64_t streamId; - int32_t taskId; - int32_t transId; -} STaskUpdateEntry; - typedef struct SMStreamCheckpointReadyRspMsg { SMsgHead head; } SMStreamCheckpointReadyRspMsg; @@ -116,22 +110,10 @@ int32_t tqStreamTaskStartAsync(SStreamMeta* pMeta, SMsgCb* cb, bool restart) { return 0; } - SStreamTaskRunReq* pRunReq = rpcMallocCont(sizeof(SStreamTaskRunReq)); - if (pRunReq == NULL) { - terrno = TSDB_CODE_OUT_OF_MEMORY; - tqError("vgId:%d failed to create msg to start wal scanning to launch stream tasks, code:%s", vgId, terrstr()); - return -1; - } - tqDebug("vgId:%d start all %d stream task(s) async", vgId, numOfTasks); - pRunReq->head.vgId = vgId; - pRunReq->streamId = 0; - pRunReq->taskId = 0; - pRunReq->reqType = restart ? STREAM_EXEC_T_RESTART_ALL_TASKS : STREAM_EXEC_T_START_ALL_TASKS; - SRpcMsg msg = {.msgType = TDMT_STREAM_TASK_RUN, .pCont = pRunReq, .contLen = sizeof(SStreamTaskRunReq)}; - tmsgPutToQueue(cb, STREAM_QUEUE, &msg); - return 0; + int32_t type = restart ? STREAM_EXEC_T_RESTART_ALL_TASKS : STREAM_EXEC_T_START_ALL_TASKS; + return streamTaskSchedTask(cb, vgId, 0, 0, type); } int32_t tqStreamStartOneTaskAsync(SStreamMeta* pMeta, SMsgCb* cb, int64_t streamId, int32_t taskId) { @@ -143,22 +125,8 @@ int32_t tqStreamStartOneTaskAsync(SStreamMeta* pMeta, SMsgCb* cb, int64_t stream return 0; } - SStreamTaskRunReq* pRunReq = rpcMallocCont(sizeof(SStreamTaskRunReq)); - if (pRunReq == NULL) { - terrno = TSDB_CODE_OUT_OF_MEMORY; - tqError("vgId:%d failed to create msg to start task:0x%x, code:%s", vgId, taskId, terrstr()); - return -1; - } - tqDebug("vgId:%d start task:0x%x async", vgId, taskId); - pRunReq->head.vgId = vgId; - pRunReq->streamId = streamId; - pRunReq->taskId = taskId; - pRunReq->reqType = STREAM_EXEC_T_START_ONE_TASK; - - SRpcMsg msg = {.msgType = TDMT_STREAM_TASK_RUN, .pCont = pRunReq, .contLen = sizeof(SStreamTaskRunReq)}; - tmsgPutToQueue(cb, STREAM_QUEUE, &msg); - return 0; + return streamTaskSchedTask(cb, vgId, streamId, taskId, STREAM_EXEC_T_START_ONE_TASK); } int32_t tqStreamTaskProcessUpdateReq(SStreamMeta* pMeta, SMsgCb* cb, SRpcMsg* pMsg, bool restored) { @@ -259,6 +227,7 @@ int32_t tqStreamTaskProcessUpdateReq(SStreamMeta* pMeta, SMsgCb* cb, SRpcMsg* pM } } + // save if (updated) { tqDebug("s-task:%s vgId:%d save task after update epset, and stop task", idstr, vgId); streamMetaSaveTask(pMeta, pTask); @@ -269,22 +238,15 @@ int32_t tqStreamTaskProcessUpdateReq(SStreamMeta* pMeta, SMsgCb* cb, SRpcMsg* pM tqDebug("s-task:%s vgId:%d not save task since not update epset actually, stop task", idstr, vgId); } + // stop streamTaskStop(pTask); - - // keep the already updated info - taosHashPut(pMeta->updateInfo.pTasks, &entry, sizeof(entry), NULL, 0); - - int64_t now = taosGetTimestampMs(); if (ppHTask != NULL) { streamTaskStop(*ppHTask); - tqDebug("s-task:%s vgId:%d task nodeEp update completed, streamTask/fill-history closed, elapsed:%" PRId64 " ms", - idstr, vgId, now - st); - taosHashPut(pMeta->updateInfo.pTasks, &(*ppHTask)->id, sizeof(pTask->id), NULL, 0); - } else { - tqDebug("s-task:%s vgId:%d, task nodeEp update completed, streamTask closed, elapsed time:%" PRId64 "ms", idstr, - vgId, now - st); } + // keep info + streamMetaAddIntoUpdateTaskList(pMeta, pTask, (ppHTask != NULL) ? (*ppHTask) : NULL, req.transId, st); + rsp.code = 0; // possibly only handle the stream task. @@ -307,10 +269,8 @@ int32_t tqStreamTaskProcessUpdateReq(SStreamMeta* pMeta, SMsgCb* cb, SRpcMsg* pM } else { tqDebug("vgId:%d all %d task(s) nodeEp updated and closed, transId:%d", vgId, numOfTasks, req.transId); #if 0 - // for test purpose, to trigger the leader election - taosMSleep(5000); + taosMSleep(5000);// for test purpose, to trigger the leader election #endif - tqStreamTaskStartAsync(pMeta, cb, true); } } @@ -712,26 +672,6 @@ int32_t tqStreamTaskProcessDropReq(SStreamMeta* pMeta, char* msg, int32_t msgLen return 0; } -int32_t tqStreamTaskResetStatus(SStreamMeta* pMeta) { - int32_t vgId = pMeta->vgId; - int32_t numOfTasks = taosArrayGetSize(pMeta->pTaskList); - - tqDebug("vgId:%d reset all %d stream task(s) status to be uninit", vgId, numOfTasks); - if (numOfTasks == 0) { - return TSDB_CODE_SUCCESS; - } - - for (int32_t i = 0; i < numOfTasks; ++i) { - SStreamTaskId* pTaskId = taosArrayGet(pMeta->pTaskList, i); - - STaskId id = {.streamId = pTaskId->streamId, .taskId = pTaskId->taskId}; - SStreamTask** pTask = taosHashGet(pMeta->pTasksMap, &id, sizeof(id)); - streamTaskResetStatus(*pTask); - } - - return 0; -} - static int32_t restartStreamTasks(SStreamMeta* pMeta, bool isLeader) { int32_t vgId = pMeta->vgId; int32_t code = 0; @@ -781,7 +721,7 @@ static int32_t restartStreamTasks(SStreamMeta* pMeta, bool isLeader) { } else { streamMetaResetStartInfo(&pMeta->startInfo); streamMetaWUnLock(pMeta); - tqInfo("vgId:%d, follower node not start stream tasks", vgId); + tqInfo("vgId:%d, follower node not start stream tasks or stream is disabled", vgId); } code = terrno; diff --git a/source/libs/stream/src/stream.c b/source/libs/stream/src/stream.c deleted file mode 100644 index 7830bbdd39d2..000000000000 --- a/source/libs/stream/src/stream.c +++ /dev/null @@ -1,308 +0,0 @@ -/* - * Copyright (c) 2019 TAOS Data, Inc. - * - * This program is free software: you can use, redistribute, and/or modify - * it under the terms of the GNU Affero General Public License, version 3 - * or later ("AGPL"), as published by the Free Software Foundation. - * - * This program is distributed in the hope that it will be useful, but WITHOUT - * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or - * FITNESS FOR A PARTICULAR PURPOSE. - * - * You should have received a copy of the GNU Affero General Public License - * along with this program. If not, see . - */ - -#include "streamInt.h" -#include "ttimer.h" - -void* streamTimer = NULL; - -int32_t streamTimerInit() { - streamTimer = taosTmrInit(1000, 100, 10000, "STREAM"); - if (streamTimer == NULL) { - stError("init stream timer failed, code:%s", tstrerror(terrno)); - return -1; - } - - stInfo("init stream timer, %p", streamTimer); - return 0; -} - -void streamTimerCleanUp() { - stInfo("cleanup stream timer, %p", streamTimer); - taosTmrCleanUp(streamTimer); - streamTimer = NULL; -} - -tmr_h streamTimerGetInstance() { - return streamTimer; -} - -char* createStreamTaskIdStr(int64_t streamId, int32_t taskId) { - char buf[128] = {0}; - sprintf(buf, "0x%" PRIx64 "-0x%x", streamId, taskId); - return taosStrdup(buf); -} - -static void streamSchedByTimer(void* param, void* tmrId) { - SStreamTask* pTask = (void*)param; - const char* id = pTask->id.idStr; - int32_t nextTrigger = (int32_t)pTask->info.triggerParam; - - int8_t status = atomic_load_8(&pTask->schedInfo.status); - 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); - return; - } - - if (streamTaskGetStatus(pTask)->state == TASK_STATUS__CK) { - stDebug("s-task:%s in checkpoint procedure, not retrieve result, next:%dms", id, nextTrigger); - } else { - if (status == TASK_TRIGGER_STATUS__ACTIVE) { - SStreamTrigger* pTrigger = taosAllocateQitem(sizeof(SStreamTrigger), DEF_QITEM, 0); - if (pTrigger == NULL) { - stError("s-task:%s failed to prepare retrieve data trigger, code:%s, try again in %dms", id, "out of memory", - nextTrigger); - taosTmrReset(streamSchedByTimer, nextTrigger, pTask, streamTimer, &pTask->schedInfo.pDelayTimer); - return; - } - - pTrigger->type = STREAM_INPUT__GET_RES; - pTrigger->pBlock = taosMemoryCalloc(1, sizeof(SSDataBlock)); - if (pTrigger->pBlock == NULL) { - taosFreeQitem(pTrigger); - - stError("s-task:%s failed to prepare retrieve data trigger, code:%s, try again in %dms", id, "out of memory", - nextTrigger); - taosTmrReset(streamSchedByTimer, nextTrigger, pTask, streamTimer, &pTask->schedInfo.pDelayTimer); - return; - } - - atomic_store_8(&pTask->schedInfo.status, TASK_TRIGGER_STATUS__INACTIVE); - pTrigger->pBlock->info.type = STREAM_GET_ALL; - - int32_t code = streamTaskPutDataIntoInputQ(pTask, (SStreamQueueItem*)pTrigger); - if (code != TSDB_CODE_SUCCESS) { - taosTmrReset(streamSchedByTimer, nextTrigger, pTask, streamTimer, &pTask->schedInfo.pDelayTimer); - return; - } - - streamSchedExec(pTask); - } - } - - taosTmrReset(streamSchedByTimer, nextTrigger, pTask, streamTimer, &pTask->schedInfo.pDelayTimer); -} - -int32_t streamSetupScheduleTrigger(SStreamTask* pTask) { - if (pTask->info.triggerParam != 0 && pTask->info.fillHistory == 0) { - int32_t ref = atomic_add_fetch_32(&pTask->refCnt, 1); - ASSERT(ref == 2 && pTask->schedInfo.pDelayTimer == NULL); - - stDebug("s-task:%s setup scheduler trigger, delay:%" PRId64 " ms", pTask->id.idStr, pTask->info.triggerParam); - - pTask->schedInfo.pDelayTimer = taosTmrStart(streamSchedByTimer, (int32_t)pTask->info.triggerParam, pTask, streamTimer); - pTask->schedInfo.status = TASK_TRIGGER_STATUS__INACTIVE; - } - - return 0; -} - -int32_t streamSchedExec(SStreamTask* pTask) { - if (streamTaskSetSchedStatusWait(pTask)) { - SStreamTaskRunReq* pRunReq = rpcMallocCont(sizeof(SStreamTaskRunReq)); - if (pRunReq == NULL) { - terrno = TSDB_CODE_OUT_OF_MEMORY; - /*int8_t status = */streamTaskSetSchedStatusInactive(pTask); - stError("failed to create msg to aunch s-task:%s, reason out of memory", pTask->id.idStr); - return -1; - } - - pRunReq->head.vgId = pTask->info.nodeId; - pRunReq->streamId = pTask->id.streamId; - pRunReq->taskId = pTask->id.taskId; - - stDebug("trigger to run s-task:%s", pTask->id.idStr); - - SRpcMsg msg = {.msgType = TDMT_STREAM_TASK_RUN, .pCont = pRunReq, .contLen = sizeof(SStreamTaskRunReq)}; - tmsgPutToQueue(pTask->pMsgCb, STREAM_QUEUE, &msg); - } else { - stTrace("s-task:%s not launch task since sched status:%d", pTask->id.idStr, pTask->status.schedStatus); - } - - return 0; -} - -static int32_t buildDispatchRsp(const SStreamTask* pTask, const SStreamDispatchReq* pReq, int32_t status, void** pBuf) { - *pBuf = rpcMallocCont(sizeof(SMsgHead) + sizeof(SStreamDispatchRsp)); - if (*pBuf == NULL) { - return TSDB_CODE_OUT_OF_MEMORY; - } - - ((SMsgHead*)(*pBuf))->vgId = htonl(pReq->upstreamNodeId); - ASSERT(((SMsgHead*)(*pBuf))->vgId != 0); - - SStreamDispatchRsp* pDispatchRsp = POINTER_SHIFT((*pBuf), sizeof(SMsgHead)); - - pDispatchRsp->stage = htobe64(pReq->stage); - pDispatchRsp->msgId = htonl(pReq->msgId); - pDispatchRsp->inputStatus = status; - pDispatchRsp->streamId = htobe64(pReq->streamId); - pDispatchRsp->upstreamNodeId = htonl(pReq->upstreamNodeId); - pDispatchRsp->upstreamTaskId = htonl(pReq->upstreamTaskId); - pDispatchRsp->downstreamNodeId = htonl(pTask->info.nodeId); - pDispatchRsp->downstreamTaskId = htonl(pTask->id.taskId); - - return TSDB_CODE_SUCCESS; -} - -static int32_t streamTaskAppendInputBlocks(SStreamTask* pTask, const SStreamDispatchReq* pReq) { - int8_t status = 0; - - SStreamDataBlock* pBlock = createStreamBlockFromDispatchMsg(pReq, pReq->type, pReq->srcVgId); - if (pBlock == NULL) { - streamTaskInputFail(pTask); - status = TASK_INPUT_STATUS__FAILED; - stError("vgId:%d, s-task:%s failed to receive dispatch msg, reason: out of memory", pTask->pMeta->vgId, - pTask->id.idStr); - } else { - if (pBlock->type == STREAM_INPUT__TRANS_STATE) { - pTask->status.appendTranstateBlock = true; - } - - int32_t code = streamTaskPutDataIntoInputQ(pTask, (SStreamQueueItem*)pBlock); - // input queue is full, upstream is blocked now - status = (code == TSDB_CODE_SUCCESS) ? TASK_INPUT_STATUS__NORMAL : TASK_INPUT_STATUS__BLOCKED; - } - - return status; -} - -int32_t streamTaskEnqueueRetrieve(SStreamTask* pTask, SStreamRetrieveReq* pReq) { - SStreamDataBlock* pData = taosAllocateQitem(sizeof(SStreamDataBlock), DEF_QITEM, sizeof(SStreamDataBlock)); - int8_t status = TASK_INPUT_STATUS__NORMAL; - - // enqueue - if (pData != NULL) { - stDebug("s-task:%s (child %d) recv retrieve req from task:0x%x(vgId:%d), reqId:0x%" PRIx64, pTask->id.idStr, - pTask->info.selfChildId, pReq->srcTaskId, pReq->srcNodeId, pReq->reqId); - - pData->type = STREAM_INPUT__DATA_RETRIEVE; - pData->srcVgId = 0; - streamRetrieveReqToData(pReq, pData); - if (streamTaskPutDataIntoInputQ(pTask, (SStreamQueueItem*)pData) == 0) { - status = TASK_INPUT_STATUS__NORMAL; - } else { - status = TASK_INPUT_STATUS__FAILED; - } - } else { // todo handle oom - /*streamTaskInputFail(pTask);*/ - /*status = TASK_INPUT_STATUS__FAILED;*/ - } - - return status == TASK_INPUT_STATUS__NORMAL ? 0 : -1; -} - -int32_t streamProcessDispatchMsg(SStreamTask* pTask, SStreamDispatchReq* pReq, SRpcMsg* pRsp) { - int32_t status = 0; - SStreamMeta* pMeta = pTask->pMeta; - const char* id = pTask->id.idStr; - - stDebug("s-task:%s receive dispatch msg from taskId:0x%x(vgId:%d), msgLen:%" PRId64 ", msgId:%d", id, - pReq->upstreamTaskId, pReq->upstreamNodeId, pReq->totalLen, pReq->msgId); - - SStreamChildEpInfo* pInfo = streamTaskGetUpstreamTaskEpInfo(pTask, pReq->upstreamTaskId); - ASSERT(pInfo != NULL); - - if (pMeta->role == NODE_ROLE_FOLLOWER) { - stError("s-task:%s task on follower received dispatch msgs, dispatch msg rejected", id); - status = TASK_INPUT_STATUS__REFUSED; - } else { - if (pReq->stage > pInfo->stage) { - // upstream task has restarted/leader-follower switch/transferred to other dnodes - stError("s-task:%s upstream task:0x%x (vgId:%d) has restart/leader-switch/vnode-transfer, prev stage:%" PRId64 - ", current:%" PRId64 " dispatch msg rejected", - id, pReq->upstreamTaskId, pReq->upstreamNodeId, pInfo->stage, pReq->stage); - status = TASK_INPUT_STATUS__REFUSED; - } else { - if (!pInfo->dataAllowed) { - stWarn("s-task:%s data from task:0x%x is denied, since inputQ is closed for it", id, pReq->upstreamTaskId); - status = TASK_INPUT_STATUS__BLOCKED; - } else { - // This task has received the checkpoint req from the upstream task, from which all the messages should be - // blocked. Note that there is no race condition here. - if (pReq->type == STREAM_INPUT__CHECKPOINT_TRIGGER) { - atomic_add_fetch_32(&pTask->upstreamInfo.numOfClosed, 1); - streamTaskCloseUpstreamInput(pTask, pReq->upstreamTaskId); - stDebug("s-task:%s close inputQ for upstream:0x%x, msgId:%d", id, pReq->upstreamTaskId, pReq->msgId); - } else if (pReq->type == STREAM_INPUT__TRANS_STATE) { - atomic_add_fetch_32(&pTask->upstreamInfo.numOfClosed, 1); - streamTaskCloseUpstreamInput(pTask, pReq->upstreamTaskId); - - // disable the related stream task here to avoid it to receive the newly arrived data after the transfer-state - STaskId* pRelTaskId = &pTask->streamTaskId; - SStreamTask* pStreamTask = streamMetaAcquireTask(pMeta, pRelTaskId->streamId, pRelTaskId->taskId); - if (pStreamTask != NULL) { - atomic_add_fetch_32(&pStreamTask->upstreamInfo.numOfClosed, 1); - streamTaskCloseUpstreamInput(pStreamTask, pReq->upstreamRelTaskId); - streamMetaReleaseTask(pMeta, pStreamTask); - } - - stDebug("s-task:%s close inputQ for upstream:0x%x since trans-state msgId:%d recv, rel stream-task:0x%" PRIx64 - " close inputQ for upstream:0x%x", - id, pReq->upstreamTaskId, pReq->msgId, pTask->streamTaskId.taskId, pReq->upstreamRelTaskId); - } - - status = streamTaskAppendInputBlocks(pTask, pReq); - } - } - } - - // disable the data from upstream tasks -// if (streamTaskGetStatus(pTask)->state == TASK_STATUS__HALT) { -// status = TASK_INPUT_STATUS__BLOCKED; -// } - - { - // do send response with the input status - int32_t code = buildDispatchRsp(pTask, pReq, status, &pRsp->pCont); - if (code != TSDB_CODE_SUCCESS) { - stError("s-task:%s failed to build dispatch rsp, msgId:%d, code:%s", id, pReq->msgId, tstrerror(code)); - terrno = code; - return code; - } - - pRsp->contLen = sizeof(SMsgHead) + sizeof(SStreamDispatchRsp); - tmsgSendRsp(pRsp); - } - - streamSchedExec(pTask); - - return 0; -} - -int32_t streamProcessRetrieveReq(SStreamTask* pTask, SStreamRetrieveReq* pReq) { - int32_t code = streamTaskEnqueueRetrieve(pTask, pReq); - if(code != 0){ - return code; - } - return streamSchedExec(pTask); -} - -void streamTaskInputFail(SStreamTask* pTask) { atomic_store_8(&pTask->inputq.status, TASK_INPUT_STATUS__FAILED); } - -SStreamChildEpInfo* streamTaskGetUpstreamTaskEpInfo(SStreamTask* pTask, int32_t taskId) { - int32_t num = taosArrayGetSize(pTask->upstreamInfo.pList); - for (int32_t i = 0; i < num; ++i) { - SStreamChildEpInfo* pInfo = taosArrayGetP(pTask->upstreamInfo.pList, i); - if (pInfo->taskId == taskId) { - return pInfo; - } - } - - stError("s-task:%s failed to find upstream task:0x%x", pTask->id.idStr, taskId); - return NULL; -} \ No newline at end of file diff --git a/source/libs/stream/src/streamCheckStatus.c b/source/libs/stream/src/streamCheckStatus.c index d356a504c661..7a4c6becf831 100644 --- a/source/libs/stream/src/streamCheckStatus.c +++ b/source/libs/stream/src/streamCheckStatus.c @@ -342,22 +342,22 @@ int32_t streamTaskCompleteCheckRsp(STaskCheckInfo* pInfo, bool lock, const char* } if (!pInfo->inCheckProcess) { -// stWarn("s-task:%s already not in-check-procedure", id); - } - - int64_t el = (pInfo->startTs != 0) ? (taosGetTimestampMs() - pInfo->startTs) : 0; - stDebug("s-task:%s clear the in check-rsp flag, not in check-rsp anymore, elapsed time:%" PRId64 " ms", id, el); + int64_t el = (pInfo->startTs != 0) ? (taosGetTimestampMs() - pInfo->startTs) : 0; + stDebug("s-task:%s clear the in check-rsp flag, set the check-rsp done, elapsed time:%" PRId64 " ms", id, el); - pInfo->startTs = 0; - pInfo->timeoutStartTs = 0; - pInfo->notReadyTasks = 0; - pInfo->inCheckProcess = 0; - pInfo->stopCheckProcess = 0; + pInfo->startTs = 0; + pInfo->timeoutStartTs = 0; + pInfo->notReadyTasks = 0; + pInfo->inCheckProcess = 0; + pInfo->stopCheckProcess = 0; - pInfo->notReadyRetryCount = 0; - pInfo->timeoutRetryCount = 0; + pInfo->notReadyRetryCount = 0; + pInfo->timeoutRetryCount = 0; - taosArrayClear(pInfo->pList); + taosArrayClear(pInfo->pList); + } else { + stDebug("s-task:%s already not in check-rsp procedure", id); + } if (lock) { taosThreadMutexUnlock(&pInfo->checkInfoLock); @@ -527,23 +527,7 @@ void handleNotReadyDownstreamTask(SStreamTask* pTask, SArray* pNotReadyList) { // The restart of all tasks requires that all tasks should not have active timer for now. Therefore, the execution // of restart in timer thread will result in a dead lock. int32_t addDownstreamFailedStatusResultAsync(SMsgCb* pMsgCb, int32_t vgId, int64_t streamId, int32_t taskId) { - SStreamTaskRunReq* pRunReq = rpcMallocCont(sizeof(SStreamTaskRunReq)); - if (pRunReq == NULL) { - terrno = TSDB_CODE_OUT_OF_MEMORY; - stError("vgId:%d failed to create msg to stop tasks async, code:%s", vgId, terrstr()); - return -1; - } - - stDebug("vgId:%d create msg add failed s-task:0x%x", vgId, taskId); - - pRunReq->head.vgId = vgId; - pRunReq->streamId = streamId; - pRunReq->taskId = taskId; - pRunReq->reqType = STREAM_EXEC_T_ADD_FAILED_TASK; - - SRpcMsg msg = {.msgType = TDMT_STREAM_TASK_RUN, .pCont = pRunReq, .contLen = sizeof(SStreamTaskRunReq)}; - tmsgPutToQueue(pMsgCb, STREAM_QUEUE, &msg); - return 0; + return streamTaskSchedTask(pMsgCb, vgId, streamId, taskId, STREAM_EXEC_T_ADD_FAILED_TASK); } // this function is executed in timer thread diff --git a/source/libs/stream/src/streamCheckpoint.c b/source/libs/stream/src/streamCheckpoint.c index f3e83eb1907e..e66458ca1d59 100644 --- a/source/libs/stream/src/streamCheckpoint.c +++ b/source/libs/stream/src/streamCheckpoint.c @@ -30,8 +30,8 @@ typedef struct { static int32_t downloadCheckpointDataByName(const char* id, const char* fname, const char* dstName); static int32_t deleteCheckpointFile(const char* id, const char* name); -static int32_t streamTaskBackupCheckpoint(char* id, char* path); -static int32_t deleteCheckpoint(char* id); +static int32_t streamTaskBackupCheckpoint(const char* id, const char* path); +static int32_t deleteCheckpoint(const char* id); int32_t tEncodeStreamCheckpointSourceReq(SEncoder* pEncoder, const SStreamCheckpointSourceReq* pReq) { if (tStartEncode(pEncoder) < 0) return -1; @@ -578,7 +578,7 @@ int32_t streamTaskBuildCheckpoint(SStreamTask* pTask) { return code; } -static int32_t uploadCheckpointToS3(char* id, char* path) { +static int32_t uploadCheckpointToS3(const char* id, const char* path) { TdDirPtr pDir = taosOpenDir(path); if (pDir == NULL) return -1; @@ -631,7 +631,7 @@ ECHECKPOINT_BACKUP_TYPE streamGetCheckpointBackupType() { } } -int32_t streamTaskBackupCheckpoint(char* id, char* path) { +int32_t streamTaskBackupCheckpoint(const char* id, const char* path) { if (id == NULL || path == NULL || strlen(id) == 0 || strlen(path) == 0 || strlen(path) >= PATH_MAX) { stError("streamTaskBackupCheckpoint parameters invalid"); return -1; @@ -675,7 +675,7 @@ int32_t streamTaskDownloadCheckpointData(char* id, char* path) { return 0; } -int32_t deleteCheckpoint(char* id) { +int32_t deleteCheckpoint(const char* id) { if (id == NULL || strlen(id) == 0) { stError("deleteCheckpoint parameters invalid"); return -1; diff --git a/source/libs/stream/src/streamDispatch.c b/source/libs/stream/src/streamDispatch.c index 5e99c29db365..4cd21d4ab4b8 100644 --- a/source/libs/stream/src/streamDispatch.c +++ b/source/libs/stream/src/streamDispatch.c @@ -1138,6 +1138,129 @@ int32_t streamProcessDispatchRsp(SStreamTask* pTask, SStreamDispatchRsp* pRsp, i return 0; } +static int32_t buildDispatchRsp(const SStreamTask* pTask, const SStreamDispatchReq* pReq, int32_t status, void** pBuf) { + *pBuf = rpcMallocCont(sizeof(SMsgHead) + sizeof(SStreamDispatchRsp)); + if (*pBuf == NULL) { + return TSDB_CODE_OUT_OF_MEMORY; + } + + ((SMsgHead*)(*pBuf))->vgId = htonl(pReq->upstreamNodeId); + ASSERT(((SMsgHead*)(*pBuf))->vgId != 0); + + SStreamDispatchRsp* pDispatchRsp = POINTER_SHIFT((*pBuf), sizeof(SMsgHead)); + + pDispatchRsp->stage = htobe64(pReq->stage); + pDispatchRsp->msgId = htonl(pReq->msgId); + pDispatchRsp->inputStatus = status; + pDispatchRsp->streamId = htobe64(pReq->streamId); + pDispatchRsp->upstreamNodeId = htonl(pReq->upstreamNodeId); + pDispatchRsp->upstreamTaskId = htonl(pReq->upstreamTaskId); + pDispatchRsp->downstreamNodeId = htonl(pTask->info.nodeId); + pDispatchRsp->downstreamTaskId = htonl(pTask->id.taskId); + + return TSDB_CODE_SUCCESS; +} + +static int32_t streamTaskAppendInputBlocks(SStreamTask* pTask, const SStreamDispatchReq* pReq) { + int8_t status = 0; + + SStreamDataBlock* pBlock = createStreamBlockFromDispatchMsg(pReq, pReq->type, pReq->srcVgId); + if (pBlock == NULL) { + streamTaskInputFail(pTask); + status = TASK_INPUT_STATUS__FAILED; + stError("vgId:%d, s-task:%s failed to receive dispatch msg, reason: out of memory", pTask->pMeta->vgId, + pTask->id.idStr); + } else { + if (pBlock->type == STREAM_INPUT__TRANS_STATE) { + pTask->status.appendTranstateBlock = true; + } + + int32_t code = streamTaskPutDataIntoInputQ(pTask, (SStreamQueueItem*)pBlock); + // input queue is full, upstream is blocked now + status = (code == TSDB_CODE_SUCCESS) ? TASK_INPUT_STATUS__NORMAL : TASK_INPUT_STATUS__BLOCKED; + } + + return status; +} + +int32_t streamProcessDispatchMsg(SStreamTask* pTask, SStreamDispatchReq* pReq, SRpcMsg* pRsp) { + int32_t status = 0; + SStreamMeta* pMeta = pTask->pMeta; + const char* id = pTask->id.idStr; + + stDebug("s-task:%s receive dispatch msg from taskId:0x%x(vgId:%d), msgLen:%" PRId64 ", msgId:%d", id, + pReq->upstreamTaskId, pReq->upstreamNodeId, pReq->totalLen, pReq->msgId); + + SStreamChildEpInfo* pInfo = streamTaskGetUpstreamTaskEpInfo(pTask, pReq->upstreamTaskId); + ASSERT(pInfo != NULL); + + if (pMeta->role == NODE_ROLE_FOLLOWER) { + stError("s-task:%s task on follower received dispatch msgs, dispatch msg rejected", id); + status = TASK_INPUT_STATUS__REFUSED; + } else { + if (pReq->stage > pInfo->stage) { + // upstream task has restarted/leader-follower switch/transferred to other dnodes + stError("s-task:%s upstream task:0x%x (vgId:%d) has restart/leader-switch/vnode-transfer, prev stage:%" PRId64 + ", current:%" PRId64 " dispatch msg rejected", + id, pReq->upstreamTaskId, pReq->upstreamNodeId, pInfo->stage, pReq->stage); + status = TASK_INPUT_STATUS__REFUSED; + } else { + if (!pInfo->dataAllowed) { + stWarn("s-task:%s data from task:0x%x is denied, since inputQ is closed for it", id, pReq->upstreamTaskId); + status = TASK_INPUT_STATUS__BLOCKED; + } else { + // This task has received the checkpoint req from the upstream task, from which all the messages should be + // blocked. Note that there is no race condition here. + if (pReq->type == STREAM_INPUT__CHECKPOINT_TRIGGER) { + atomic_add_fetch_32(&pTask->upstreamInfo.numOfClosed, 1); + streamTaskCloseUpstreamInput(pTask, pReq->upstreamTaskId); + stDebug("s-task:%s close inputQ for upstream:0x%x, msgId:%d", id, pReq->upstreamTaskId, pReq->msgId); + } else if (pReq->type == STREAM_INPUT__TRANS_STATE) { + atomic_add_fetch_32(&pTask->upstreamInfo.numOfClosed, 1); + streamTaskCloseUpstreamInput(pTask, pReq->upstreamTaskId); + + // disable the related stream task here to avoid it to receive the newly arrived data after the transfer-state + STaskId* pRelTaskId = &pTask->streamTaskId; + SStreamTask* pStreamTask = streamMetaAcquireTask(pMeta, pRelTaskId->streamId, pRelTaskId->taskId); + if (pStreamTask != NULL) { + atomic_add_fetch_32(&pStreamTask->upstreamInfo.numOfClosed, 1); + streamTaskCloseUpstreamInput(pStreamTask, pReq->upstreamRelTaskId); + streamMetaReleaseTask(pMeta, pStreamTask); + } + + stDebug("s-task:%s close inputQ for upstream:0x%x since trans-state msgId:%d recv, rel stream-task:0x%" PRIx64 + " close inputQ for upstream:0x%x", + id, pReq->upstreamTaskId, pReq->msgId, pTask->streamTaskId.taskId, pReq->upstreamRelTaskId); + } + + status = streamTaskAppendInputBlocks(pTask, pReq); + } + } + } + + // disable the data from upstream tasks +// if (streamTaskGetStatus(pTask)->state == TASK_STATUS__HALT) { +// status = TASK_INPUT_STATUS__BLOCKED; +// } + + { + // do send response with the input status + int32_t code = buildDispatchRsp(pTask, pReq, status, &pRsp->pCont); + if (code != TSDB_CODE_SUCCESS) { + stError("s-task:%s failed to build dispatch rsp, msgId:%d, code:%s", id, pReq->msgId, tstrerror(code)); + terrno = code; + return code; + } + + pRsp->contLen = sizeof(SMsgHead) + sizeof(SStreamDispatchRsp); + tmsgSendRsp(pRsp); + } + + streamSchedExec(pTask); + + return 0; +} + int32_t tEncodeStreamTaskUpdateMsg(SEncoder* pEncoder, const SStreamTaskNodeUpdateMsg* pMsg) { if (tStartEncode(pEncoder) < 0) return -1; if (tEncodeI64(pEncoder, pMsg->streamId) < 0) return -1; diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index f0661eedd662..dd613c7ec6fb 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -1217,7 +1217,7 @@ void metaHbToMnode(void* param, void* tmrId) { } // need to stop, stop now - if (pMeta->pHbInfo->stopFlag == STREAM_META_WILL_STOP) { + if (pMeta->pHbInfo->stopFlag == STREAM_META_WILL_STOP) { // todo refactor: not need this now, use closeFlag in Meta pMeta->pHbInfo->stopFlag = STREAM_META_OK_TO_STOP; stDebug("vgId:%d jump out of meta timer", pMeta->vgId); taosReleaseRef(streamMetaId, rid); @@ -1308,7 +1308,7 @@ void streamMetaNotifyClose(SStreamMeta* pMeta) { } } - stDebug("vgId:%d start to check all tasks", vgId); + stDebug("vgId:%d start to check all tasks for closing", vgId); int64_t st = taosGetTimestampMs(); while (streamMetaTaskInTimer(pMeta)) { @@ -1759,3 +1759,25 @@ int32_t streamMetaAddFailedTask(SStreamMeta* pMeta, int64_t streamId, int32_t ta return code; } + +void streamMetaAddIntoUpdateTaskList(SStreamMeta* pMeta, SStreamTask* pTask, SStreamTask* pHTask, int32_t transId, + int64_t startTs) { + const char* id = pTask->id.idStr; + int32_t vgId = pTask->pMeta->vgId; + + // keep the already updated info + STaskUpdateEntry entry = {.streamId = pTask->id.streamId, .taskId = pTask->id.taskId, .transId = transId}; + taosHashPut(pMeta->updateInfo.pTasks, &entry, sizeof(entry), NULL, 0); + + int64_t el = taosGetTimestampMs() - startTs; + if (pHTask != NULL) { + STaskUpdateEntry hEntry = {.streamId = pHTask->id.streamId, .taskId = pHTask->id.taskId, .transId = transId}; + taosHashPut(pMeta->updateInfo.pTasks, &hEntry, sizeof(hEntry), NULL, 0); + + stDebug("s-task:%s vgId:%d transId:%d task nodeEp update completed, streamTask/hTask closed, elapsed:%" PRId64 + " ms", id, vgId, transId, el); + } else { + stDebug("s-task:%s vgId:%d transId:%d task nodeEp update completed, streamTask closed, elapsed time:%" PRId64 "ms", + id, vgId, transId, el); + } +} \ No newline at end of file diff --git a/source/libs/stream/src/streamQueue.c b/source/libs/stream/src/streamQueue.c index 9e872a1aff72..0ac282c36267 100644 --- a/source/libs/stream/src/streamQueue.c +++ b/source/libs/stream/src/streamQueue.c @@ -424,4 +424,6 @@ void streamTaskPutbackToken(STokenBucket* pBucket) { // size in KB void streamTaskConsumeQuota(STokenBucket* pBucket, int32_t bytes) { pBucket->quotaRemain -= SIZE_IN_MiB(bytes); -} \ No newline at end of file +} + +void streamTaskInputFail(SStreamTask* pTask) { atomic_store_8(&pTask->inputq.status, TASK_INPUT_STATUS__FAILED); } \ No newline at end of file diff --git a/source/libs/stream/src/streamSched.c b/source/libs/stream/src/streamSched.c new file mode 100644 index 000000000000..f8ef4e4f7165 --- /dev/null +++ b/source/libs/stream/src/streamSched.c @@ -0,0 +1,129 @@ +/* + * 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 "streamInt.h" +#include "ttimer.h" + +static void streamSchedByTimer(void* param, void* tmrId) { + SStreamTask* pTask = (void*)param; + const char* id = pTask->id.idStr; + int32_t nextTrigger = (int32_t)pTask->info.triggerParam; + + int8_t status = atomic_load_8(&pTask->schedInfo.status); + 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); + return; + } + + if (streamTaskGetStatus(pTask)->state == TASK_STATUS__CK) { + stDebug("s-task:%s in checkpoint procedure, not retrieve result, next:%dms", id, nextTrigger); + } else { + if (status == TASK_TRIGGER_STATUS__ACTIVE) { + SStreamTrigger* pTrigger = taosAllocateQitem(sizeof(SStreamTrigger), DEF_QITEM, 0); + if (pTrigger == NULL) { + stError("s-task:%s failed to prepare retrieve data trigger, code:%s, try again in %dms", id, "out of memory", + nextTrigger); + taosTmrReset(streamSchedByTimer, nextTrigger, pTask, streamTimer, &pTask->schedInfo.pDelayTimer); + return; + } + + pTrigger->type = STREAM_INPUT__GET_RES; + pTrigger->pBlock = taosMemoryCalloc(1, sizeof(SSDataBlock)); + if (pTrigger->pBlock == NULL) { + taosFreeQitem(pTrigger); + + stError("s-task:%s failed to prepare retrieve data trigger, code:%s, try again in %dms", id, "out of memory", + nextTrigger); + taosTmrReset(streamSchedByTimer, nextTrigger, pTask, streamTimer, &pTask->schedInfo.pDelayTimer); + return; + } + + atomic_store_8(&pTask->schedInfo.status, TASK_TRIGGER_STATUS__INACTIVE); + pTrigger->pBlock->info.type = STREAM_GET_ALL; + + int32_t code = streamTaskPutDataIntoInputQ(pTask, (SStreamQueueItem*)pTrigger); + if (code != TSDB_CODE_SUCCESS) { + taosTmrReset(streamSchedByTimer, nextTrigger, pTask, streamTimer, &pTask->schedInfo.pDelayTimer); + return; + } + + streamSchedExec(pTask); + } + } + + taosTmrReset(streamSchedByTimer, nextTrigger, pTask, streamTimer, &pTask->schedInfo.pDelayTimer); +} + +int32_t streamSetupScheduleTrigger(SStreamTask* pTask) { + if (pTask->info.triggerParam != 0 && pTask->info.fillHistory == 0) { + int32_t ref = atomic_add_fetch_32(&pTask->refCnt, 1); + ASSERT(ref == 2 && pTask->schedInfo.pDelayTimer == NULL); + + stDebug("s-task:%s setup scheduler trigger, delay:%" PRId64 " ms", pTask->id.idStr, pTask->info.triggerParam); + + pTask->schedInfo.pDelayTimer = taosTmrStart(streamSchedByTimer, (int32_t)pTask->info.triggerParam, pTask, streamTimer); + pTask->schedInfo.status = TASK_TRIGGER_STATUS__INACTIVE; + } + + return 0; +} + +int32_t streamSchedExec(SStreamTask* pTask) { + if (streamTaskSetSchedStatusWait(pTask)) { + SStreamTaskRunReq* pRunReq = rpcMallocCont(sizeof(SStreamTaskRunReq)); + if (pRunReq == NULL) { + terrno = TSDB_CODE_OUT_OF_MEMORY; + /*int8_t status = */streamTaskSetSchedStatusInactive(pTask); + stError("failed to create msg to aunch s-task:%s, reason out of memory", pTask->id.idStr); + return -1; + } + + pRunReq->head.vgId = pTask->info.nodeId; + pRunReq->streamId = pTask->id.streamId; + pRunReq->taskId = pTask->id.taskId; + + stDebug("trigger to run s-task:%s", pTask->id.idStr); + + SRpcMsg msg = {.msgType = TDMT_STREAM_TASK_RUN, .pCont = pRunReq, .contLen = sizeof(SStreamTaskRunReq)}; + tmsgPutToQueue(pTask->pMsgCb, STREAM_QUEUE, &msg); + } else { + stTrace("s-task:%s not launch task since sched status:%d", pTask->id.idStr, pTask->status.schedStatus); + } + + return 0; +} + +int32_t streamTaskSchedTask(SMsgCb* pMsgCb, int32_t vgId, int64_t streamId, int32_t taskId, int32_t execType) { + SStreamTaskRunReq* pRunReq = rpcMallocCont(sizeof(SStreamTaskRunReq)); + if (pRunReq == NULL) { + terrno = TSDB_CODE_OUT_OF_MEMORY; + stError("vgId:%d failed to create msg to start stream task:0x%x, type:%d, code:%s", vgId, taskId, execType, + terrstr()); + return -1; + } + + stDebug("vgId:%d create msg to start stream task:0x%x", vgId, taskId); + + pRunReq->head.vgId = vgId; + pRunReq->streamId = streamId; + pRunReq->taskId = taskId; + pRunReq->reqType = execType; + + SRpcMsg msg = {.msgType = TDMT_STREAM_TASK_RUN, .pCont = pRunReq, .contLen = sizeof(SStreamTaskRunReq)}; + tmsgPutToQueue(pMsgCb, STREAM_QUEUE, &msg); + return TSDB_CODE_SUCCESS; +} diff --git a/source/libs/stream/src/streamTask.c b/source/libs/stream/src/streamTask.c index 1e3c6f35891b..1f0cbc1695dd 100644 --- a/source/libs/stream/src/streamTask.c +++ b/source/libs/stream/src/streamTask.c @@ -993,17 +993,54 @@ int32_t streamTaskSendCheckpointReq(SStreamTask* pTask) { return 0; } -static int32_t streamTaskInitTaskCheckInfo(STaskCheckInfo* pInfo, STaskOutputInfo* pOutputInfo, int64_t startTs) { - taosArrayClear(pInfo->pList); +SStreamChildEpInfo* streamTaskGetUpstreamTaskEpInfo(SStreamTask* pTask, int32_t taskId) { + int32_t num = taosArrayGetSize(pTask->upstreamInfo.pList); + for (int32_t i = 0; i < num; ++i) { + SStreamChildEpInfo* pInfo = taosArrayGetP(pTask->upstreamInfo.pList, i); + if (pInfo->taskId == taskId) { + return pInfo; + } + } - if (pOutputInfo->type == TASK_OUTPUT__FIXED_DISPATCH) { - pInfo->notReadyTasks = 1; - } else if (pOutputInfo->type == TASK_OUTPUT__SHUFFLE_DISPATCH) { - pInfo->notReadyTasks = taosArrayGetSize(pOutputInfo->shuffleDispatcher.dbInfo.pVgroupInfos); - ASSERT(pInfo->notReadyTasks == pOutputInfo->shuffleDispatcher.dbInfo.vgNum); + stError("s-task:%s failed to find upstream task:0x%x", pTask->id.idStr, taskId); + return NULL; +} + +char* createStreamTaskIdStr(int64_t streamId, int32_t taskId) { + char buf[128] = {0}; + sprintf(buf, "0x%" PRIx64 "-0x%x", streamId, taskId); + return taosStrdup(buf); +} + +static int32_t streamTaskEnqueueRetrieve(SStreamTask* pTask, SStreamRetrieveReq* pReq) { + SStreamDataBlock* pData = taosAllocateQitem(sizeof(SStreamDataBlock), DEF_QITEM, sizeof(SStreamDataBlock)); + int8_t status = TASK_INPUT_STATUS__NORMAL; + + // enqueue + if (pData != NULL) { + stDebug("s-task:%s (child %d) recv retrieve req from task:0x%x(vgId:%d), reqId:0x%" PRIx64, pTask->id.idStr, + pTask->info.selfChildId, pReq->srcTaskId, pReq->srcNodeId, pReq->reqId); + + pData->type = STREAM_INPUT__DATA_RETRIEVE; + pData->srcVgId = 0; + streamRetrieveReqToData(pReq, pData); + if (streamTaskPutDataIntoInputQ(pTask, (SStreamQueueItem*)pData) == 0) { + status = TASK_INPUT_STATUS__NORMAL; + } else { + status = TASK_INPUT_STATUS__FAILED; + } + } else { // todo handle oom + /*streamTaskInputFail(pTask);*/ + /*status = TASK_INPUT_STATUS__FAILED;*/ } - pInfo->startTs = startTs; - return TSDB_CODE_SUCCESS; + return status == TASK_INPUT_STATUS__NORMAL ? 0 : -1; } +int32_t streamProcessRetrieveReq(SStreamTask* pTask, SStreamRetrieveReq* pReq) { + int32_t code = streamTaskEnqueueRetrieve(pTask, pReq); + if(code != 0){ + return code; + } + return streamSchedExec(pTask); +} \ No newline at end of file diff --git a/source/libs/stream/src/streamTimer.c b/source/libs/stream/src/streamTimer.c new file mode 100644 index 000000000000..6e956e268215 --- /dev/null +++ b/source/libs/stream/src/streamTimer.c @@ -0,0 +1,40 @@ +/* + * 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 "streamInt.h" +#include "ttimer.h" + +void* streamTimer = NULL; + +int32_t streamTimerInit() { + streamTimer = taosTmrInit(1000, 100, 10000, "STREAM"); + if (streamTimer == NULL) { + stError("init stream timer failed, code:%s", tstrerror(terrno)); + return -1; + } + + stInfo("init stream timer, %p", streamTimer); + return 0; +} + +void streamTimerCleanUp() { + stInfo("cleanup stream timer, %p", streamTimer); + taosTmrCleanUp(streamTimer); + streamTimer = NULL; +} + +tmr_h streamTimerGetInstance() { + return streamTimer; +} From 6e710c5d72aa995036c1499410f20a5f1e0151ea Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 6 May 2024 17:08:11 +0800 Subject: [PATCH 04/30] refactor: do some internal refactor. --- source/libs/stream/src/streamMeta.c | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index dd613c7ec6fb..7df9555e088c 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -1442,7 +1442,7 @@ void streamMetaUpdateStageRole(SStreamMeta* pMeta, int64_t stage, bool isLeader) } } -static int32_t prepareBeforeStartTasks(SStreamMeta* pMeta, SArray** pList) { +static int32_t prepareBeforeStartTasks(SStreamMeta* pMeta, SArray** pList, int64_t now) { streamMetaWLock(pMeta); if (pMeta->closeFlag) { @@ -1455,7 +1455,7 @@ static int32_t prepareBeforeStartTasks(SStreamMeta* pMeta, SArray** pList) { taosHashClear(pMeta->startInfo.pReadyTaskSet); taosHashClear(pMeta->startInfo.pFailedTaskSet); - pMeta->startInfo.startTs = taosGetTimestampMs(); + pMeta->startInfo.startTs = now; streamMetaResetTaskStatus(pMeta); streamMetaWUnLock(pMeta); @@ -1469,7 +1469,7 @@ int32_t streamMetaStartAllTasks(SStreamMeta* pMeta) { int64_t now = taosGetTimestampMs(); int32_t numOfTasks = taosArrayGetSize(pMeta->pTaskList); - stInfo("vgId:%d start to check all %d stream task(s) downstream status", vgId, numOfTasks); + stInfo("vgId:%d start to check all %d stream task(s) downstream status, start ts:%"PRId64, vgId, numOfTasks, now); if (numOfTasks == 0) { stInfo("vgId:%d no tasks to be started", pMeta->vgId); @@ -1477,7 +1477,7 @@ int32_t streamMetaStartAllTasks(SStreamMeta* pMeta) { } SArray* pTaskList = NULL; - code = prepareBeforeStartTasks(pMeta, &pTaskList); + code = prepareBeforeStartTasks(pMeta, &pTaskList, now); if (code != TSDB_CODE_SUCCESS) { ASSERT(pTaskList == NULL); return TSDB_CODE_SUCCESS; From 2253881f6f0be2cd17b19cab0b58fdb178440a9a Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 6 May 2024 18:17:38 +0800 Subject: [PATCH 05/30] refactor: do some internal refactor. --- include/libs/stream/tstream.h | 5 +- source/dnode/vnode/src/tq/tqStreamTask.c | 46 +----- source/dnode/vnode/src/tqCommon/tqCommon.c | 2 +- source/libs/stream/src/streamCheckStatus.c | 2 +- source/libs/stream/src/streamCheckpoint.c | 2 +- source/libs/stream/src/streamDispatch.c | 2 +- source/libs/stream/src/streamExec.c | 75 +--------- source/libs/stream/src/streamSched.c | 160 +++++++++++++-------- source/libs/stream/src/streamTask.c | 2 +- 9 files changed, 121 insertions(+), 175 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index a8a695cc5b1c..1db62abfc021 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -799,8 +799,11 @@ SStreamChildEpInfo* streamTaskGetUpstreamTaskEpInfo(SStreamTask* pTask, int32_t void streamTaskInputFail(SStreamTask* pTask); int32_t streamExecTask(SStreamTask* pTask); int32_t streamResumeTask(SStreamTask* pTask); -int32_t streamSchedExec(SStreamTask* pTask); +int32_t streamTrySchedExec(SStreamTask* pTask); int32_t streamTaskSchedTask(SMsgCb* pMsgCb, int32_t vgId, int64_t streamId, int32_t taskId, int32_t execType); +int32_t streamTaskResumeInFuture(SStreamTask* pTask); +void streamTaskClearSchedIdleInfo(SStreamTask* pTask); +void streamTaskSetIdleInfo(SStreamTask* pTask, int32_t idleTime); bool streamTaskShouldStop(const SStreamTask* pStatus); bool streamTaskShouldPause(const SStreamTask* pStatus); diff --git a/source/dnode/vnode/src/tq/tqStreamTask.c b/source/dnode/vnode/src/tq/tqStreamTask.c index e22ebb9d84af..5a29f67ae398 100644 --- a/source/dnode/vnode/src/tq/tqStreamTask.c +++ b/source/dnode/vnode/src/tq/tqStreamTask.c @@ -62,29 +62,14 @@ typedef struct SBuildScanWalMsgParam { } SBuildScanWalMsgParam; static void doStartScanWal(void* param, void* tmrId) { - SBuildScanWalMsgParam* pParam = (SBuildScanWalMsgParam*) param; - - int32_t vgId = pParam->pTq->pStreamMeta->vgId; - - SStreamTaskRunReq* pRunReq = rpcMallocCont(sizeof(SStreamTaskRunReq)); - if (pRunReq == NULL) { - taosMemoryFree(pParam); - terrno = TSDB_CODE_OUT_OF_MEMORY; - tqError("vgId:%d failed to create msg to start wal scanning to launch stream tasks, code:%s", vgId, terrstr()); - return; - } + SBuildScanWalMsgParam* pParam = (SBuildScanWalMsgParam*)param; + STQ* pTq = pParam->pTq; + int32_t vgId = pTq->pStreamMeta->vgId; tqDebug("vgId:%d create msg to start wal scan, numOfTasks:%d, vnd restored:%d", vgId, pParam->numOfTasks, - pParam->pTq->pVnode->restored); - - pRunReq->head.vgId = vgId; - pRunReq->streamId = 0; - pRunReq->taskId = 0; - pRunReq->reqType = STREAM_EXEC_T_EXTRACT_WAL_DATA; - - SRpcMsg msg = {.msgType = TDMT_STREAM_TASK_RUN, .pCont = pRunReq, .contLen = sizeof(SStreamTaskRunReq)}; - tmsgPutToQueue(&pParam->pTq->pVnode->msgCb, STREAM_QUEUE, &msg); + pTq->pVnode->restored); + /*int32_t code = */ streamTaskSchedTask(&pTq->pVnode->msgCb, vgId, 0, 0, STREAM_EXEC_T_EXTRACT_WAL_DATA); taosMemoryFree(pParam); } @@ -161,24 +146,7 @@ int32_t tqScanWalAsync(STQ* pTq, bool ckPause) { int32_t tqStopStreamTasksAsync(STQ* pTq) { SStreamMeta* pMeta = pTq->pStreamMeta; int32_t vgId = pMeta->vgId; - - SStreamTaskRunReq* pRunReq = rpcMallocCont(sizeof(SStreamTaskRunReq)); - if (pRunReq == NULL) { - terrno = TSDB_CODE_OUT_OF_MEMORY; - tqError("vgId:%d failed to create msg to stop tasks async, code:%s", vgId, terrstr()); - return -1; - } - - tqDebug("vgId:%d create msg to stop all tasks async", vgId); - - pRunReq->head.vgId = vgId; - pRunReq->streamId = 0; - pRunReq->taskId = 0; - pRunReq->reqType = STREAM_EXEC_T_STOP_ALL_TASKS; - - SRpcMsg msg = {.msgType = TDMT_STREAM_TASK_RUN, .pCont = pRunReq, .contLen = sizeof(SStreamTaskRunReq)}; - tmsgPutToQueue(&pTq->pVnode->msgCb, STREAM_QUEUE, &msg); - return 0; + return streamTaskSchedTask(&pTq->pVnode->msgCb, vgId, 0, 0, STREAM_EXEC_T_STOP_ALL_TASKS); } int32_t setWalReaderStartOffset(SStreamTask* pTask, int32_t vgId) { @@ -394,7 +362,7 @@ int32_t doScanWalForAllTasks(SStreamMeta* pStreamMeta, bool* pScanIdle) { if ((numOfItems > 0) || hasNewData) { noDataInWal = false; - code = streamSchedExec(pTask); + code = streamTrySchedExec(pTask); if (code != TSDB_CODE_SUCCESS) { streamMetaReleaseTask(pStreamMeta, pTask); return -1; diff --git a/source/dnode/vnode/src/tqCommon/tqCommon.c b/source/dnode/vnode/src/tqCommon/tqCommon.c index 85fa5cc118ca..b2068c99230b 100644 --- a/source/dnode/vnode/src/tqCommon/tqCommon.c +++ b/source/dnode/vnode/src/tqCommon/tqCommon.c @@ -947,7 +947,7 @@ static int32_t tqProcessTaskResumeImpl(void* handle, SStreamTask* pTask, int64_t } else if (level == TASK_LEVEL__SOURCE && (streamQueueGetNumOfItems(pTask->inputq.queue) == 0)) { tqScanWalAsync((STQ*)handle, false); } else { - streamSchedExec(pTask); + streamTrySchedExec(pTask); } } else if (status == TASK_STATUS__UNINIT) { // todo: fill-history task init ? diff --git a/source/libs/stream/src/streamCheckStatus.c b/source/libs/stream/src/streamCheckStatus.c index 7a4c6becf831..f0f12cae2bfc 100644 --- a/source/libs/stream/src/streamCheckStatus.c +++ b/source/libs/stream/src/streamCheckStatus.c @@ -341,7 +341,7 @@ int32_t streamTaskCompleteCheckRsp(STaskCheckInfo* pInfo, bool lock, const char* taosThreadMutexLock(&pInfo->checkInfoLock); } - if (!pInfo->inCheckProcess) { + if (pInfo->inCheckProcess) { int64_t el = (pInfo->startTs != 0) ? (taosGetTimestampMs() - pInfo->startTs) : 0; stDebug("s-task:%s clear the in check-rsp flag, set the check-rsp done, elapsed time:%" PRId64 " ms", id, el); diff --git a/source/libs/stream/src/streamCheckpoint.c b/source/libs/stream/src/streamCheckpoint.c index e66458ca1d59..d2f9a0cbc378 100644 --- a/source/libs/stream/src/streamCheckpoint.c +++ b/source/libs/stream/src/streamCheckpoint.c @@ -157,7 +157,7 @@ static int32_t appendCheckpointIntoInputQ(SStreamTask* pTask, int32_t checkpoint return TSDB_CODE_OUT_OF_MEMORY; } - streamSchedExec(pTask); + streamTrySchedExec(pTask); return TSDB_CODE_SUCCESS; } diff --git a/source/libs/stream/src/streamDispatch.c b/source/libs/stream/src/streamDispatch.c index 4cd21d4ab4b8..56eb7b196ef1 100644 --- a/source/libs/stream/src/streamDispatch.c +++ b/source/libs/stream/src/streamDispatch.c @@ -1256,7 +1256,7 @@ int32_t streamProcessDispatchMsg(SStreamTask* pTask, SStreamDispatchReq* pReq, S tmsgSendRsp(pRsp); } - streamSchedExec(pTask); + streamTrySchedExec(pTask); return 0; } diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index 250866005e2e..047b169ec902 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -533,8 +533,7 @@ int32_t streamProcessTransstateBlock(SStreamTask* pTask, SStreamDataBlock* pBloc return code; } -static void setTaskSchedInfo(SStreamTask* pTask, int32_t idleTime) { pTask->status.schedIdleTime = idleTime; } -static void clearTaskSchedInfo(SStreamTask* pTask) { pTask->status.schedIdleTime = 0; } +//static void streamTaskSetIdleInfo(SStreamTask* pTask, int32_t idleTime) { pTask->status.schedIdleTime = idleTime; } static void setLastExecTs(SStreamTask* pTask, int64_t ts) { pTask->status.lastExecTs = ts; } /** @@ -559,26 +558,26 @@ static int32_t doStreamExecTask(SStreamTask* pTask) { if (streamQueueIsFull(pTask->outputq.queue)) { stWarn("s-task:%s outputQ is full, idle for 500ms and retry", id); - setTaskSchedInfo(pTask, 500); + streamTaskSetIdleInfo(pTask, 500); return 0; } if (pTask->inputq.status == TASK_INPUT_STATUS__BLOCKED) { stWarn("s-task:%s downstream task inputQ blocked, idle for 1sec and retry", id); - setTaskSchedInfo(pTask, 1000); + streamTaskSetIdleInfo(pTask, 1000); return 0; } if (taosGetTimestampMs() - pTask->status.lastExecTs < MIN_INVOKE_INTERVAL) { stDebug("s-task:%s invoke with high frequency, idle and retry exec in 50ms", id); - setTaskSchedInfo(pTask, MIN_INVOKE_INTERVAL); + streamTaskSetIdleInfo(pTask, MIN_INVOKE_INTERVAL); return 0; } EExtractDataCode ret = streamTaskGetDataFromInputQ(pTask, &pInput, &numOfBlocks, &blockSize); if (ret == EXEC_AFTER_IDLE) { ASSERT(pInput == NULL && numOfBlocks == 0); - setTaskSchedInfo(pTask, MIN_INVOKE_INTERVAL); + streamTaskSetIdleInfo(pTask, MIN_INVOKE_INTERVAL); return 0; } else { if (pInput == NULL) { @@ -720,66 +719,6 @@ bool streamTaskReadyToRun(const SStreamTask* pTask, char** pStatus) { } } -static void doStreamExecTaskHelper(void* param, void* tmrId) { - SStreamTask* pTask = (SStreamTask*)param; - - SStreamTaskState* p = streamTaskGetStatus(pTask); - if (p->state == TASK_STATUS__DROPPING || p->state == TASK_STATUS__STOP) { - streamTaskSetSchedStatusInactive(pTask); - - int32_t ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1); - stDebug("s-task:%s status:%s not resume task, ref:%d", pTask->id.idStr, p->name, ref); - - streamMetaReleaseTask(pTask->pMeta, pTask); - return; - } - - // task resume running - SStreamTaskRunReq* pRunReq = rpcMallocCont(sizeof(SStreamTaskRunReq)); - if (pRunReq == NULL) { - terrno = TSDB_CODE_OUT_OF_MEMORY; - /*int8_t status = */streamTaskSetSchedStatusInactive(pTask); - - int32_t ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1); - stError("failed to create msg to resume s-task:%s, reason out of memory, ref:%d", pTask->id.idStr, ref); - - streamMetaReleaseTask(pTask->pMeta, pTask); - return; - } - - pRunReq->head.vgId = pTask->info.nodeId; - pRunReq->streamId = pTask->id.streamId; - pRunReq->taskId = pTask->id.taskId; - pRunReq->reqType = STREAM_EXEC_T_RESUME_TASK; - - int32_t ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1); - stDebug("trigger to resume s-task:%s after being idled for %dms, ref:%d", pTask->id.idStr, pTask->status.schedIdleTime, ref); - - SRpcMsg msg = {.msgType = TDMT_STREAM_TASK_RUN, .pCont = pRunReq, .contLen = sizeof(SStreamTaskRunReq)}; - tmsgPutToQueue(pTask->pMsgCb, STREAM_QUEUE, &msg); - - // release the task ref count - clearTaskSchedInfo(pTask); - streamMetaReleaseTask(pTask->pMeta, pTask); -} - -static int32_t schedTaskInFuture(SStreamTask* pTask) { - int32_t ref = atomic_add_fetch_32(&pTask->status.timerActive, 1); - stDebug("s-task:%s task should idle, add into timer to retry in %dms, ref:%d", pTask->id.idStr, - pTask->status.schedIdleTime, ref); - - // add one ref count for task - /*SStreamTask* pAddRefTask = */streamMetaAcquireOneTask(pTask); - - if (pTask->schedInfo.pIdleTimer == NULL) { - pTask->schedInfo.pIdleTimer = taosTmrStart(doStreamExecTaskHelper, pTask->status.schedIdleTime, pTask, streamTimer); - } else { - taosTmrReset(doStreamExecTaskHelper, pTask->status.schedIdleTime, pTask, streamTimer, &pTask->schedInfo.pIdleTimer); - } - - return TSDB_CODE_SUCCESS; -} - int32_t streamResumeTask(SStreamTask* pTask) { ASSERT(pTask->status.schedStatus == TASK_SCHED_STATUS__ACTIVE); const char* id = pTask->id.idStr; @@ -793,7 +732,7 @@ int32_t streamResumeTask(SStreamTask* pTask) { int32_t numOfItems = streamQueueGetNumOfItems(pTask->inputq.queue); if ((numOfItems == 0) || streamTaskShouldStop(pTask) || streamTaskShouldPause(pTask)) { atomic_store_8(&pTask->status.schedStatus, TASK_SCHED_STATUS__INACTIVE); - clearTaskSchedInfo(pTask); + streamTaskClearSchedIdleInfo(pTask); taosThreadMutexUnlock(&pTask->lock); setLastExecTs(pTask, taosGetTimestampMs()); @@ -806,7 +745,7 @@ int32_t streamResumeTask(SStreamTask* pTask) { } else { // check if this task needs to be idle for a while if (pTask->status.schedIdleTime > 0) { - schedTaskInFuture(pTask); + streamTaskResumeInFuture(pTask); taosThreadMutexUnlock(&pTask->lock); setLastExecTs(pTask, taosGetTimestampMs()); diff --git a/source/libs/stream/src/streamSched.c b/source/libs/stream/src/streamSched.c index f8ef4e4f7165..2e337234b6f2 100644 --- a/source/libs/stream/src/streamSched.c +++ b/source/libs/stream/src/streamSched.c @@ -16,7 +16,103 @@ #include "streamInt.h" #include "ttimer.h" -static void streamSchedByTimer(void* param, void* tmrId) { +static void streamTaskResumeHelper(void* param, void* tmrId); +static void streamSchedByTimer(void* param, void* tmrId); + +int32_t streamSetupScheduleTrigger(SStreamTask* pTask) { + if (pTask->info.triggerParam != 0 && pTask->info.fillHistory == 0) { + int32_t ref = atomic_add_fetch_32(&pTask->refCnt, 1); + ASSERT(ref == 2 && pTask->schedInfo.pDelayTimer == NULL); + + stDebug("s-task:%s setup scheduler trigger, delay:%" PRId64 " ms", pTask->id.idStr, pTask->info.triggerParam); + + pTask->schedInfo.pDelayTimer = taosTmrStart(streamSchedByTimer, (int32_t)pTask->info.triggerParam, pTask, streamTimer); + pTask->schedInfo.status = TASK_TRIGGER_STATUS__INACTIVE; + } + + return 0; +} + +int32_t streamTrySchedExec(SStreamTask* pTask) { + if (streamTaskSetSchedStatusWait(pTask)) { + streamTaskSchedTask(pTask->pMsgCb, pTask->info.nodeId, pTask->id.streamId, pTask->id.taskId, 0); + } else { + stTrace("s-task:%s not launch task since sched status:%d", pTask->id.idStr, pTask->status.schedStatus); + } + + return 0; +} + +int32_t streamTaskSchedTask(SMsgCb* pMsgCb, int32_t vgId, int64_t streamId, int32_t taskId, int32_t execType) { + SStreamTaskRunReq* pRunReq = rpcMallocCont(sizeof(SStreamTaskRunReq)); + if (pRunReq == NULL) { + terrno = TSDB_CODE_OUT_OF_MEMORY; + stError("vgId:%d failed to create msg to start stream task:0x%x exec, type:%d, code:%s", vgId, taskId, execType, + terrstr()); + return -1; + } + + stDebug("vgId:%d create msg to start stream task:0x%x, exec type:%d", vgId, taskId, execType); + + pRunReq->head.vgId = vgId; + pRunReq->streamId = streamId; + pRunReq->taskId = taskId; + pRunReq->reqType = execType; + + SRpcMsg msg = {.msgType = TDMT_STREAM_TASK_RUN, .pCont = pRunReq, .contLen = sizeof(SStreamTaskRunReq)}; + tmsgPutToQueue(pMsgCb, STREAM_QUEUE, &msg); + return TSDB_CODE_SUCCESS; +} + +void streamTaskClearSchedIdleInfo(SStreamTask* pTask) { pTask->status.schedIdleTime = 0; } + +void streamTaskSetIdleInfo(SStreamTask* pTask, int32_t idleTime) { pTask->status.schedIdleTime = idleTime; } + +int32_t streamTaskResumeInFuture(SStreamTask* pTask) { + int32_t ref = atomic_add_fetch_32(&pTask->status.timerActive, 1); + stDebug("s-task:%s task should idle, add into timer to retry in %dms, ref:%d", pTask->id.idStr, + pTask->status.schedIdleTime, ref); + + // add one ref count for task + /*SStreamTask* pAddRefTask = */streamMetaAcquireOneTask(pTask); + + if (pTask->schedInfo.pIdleTimer == NULL) { + pTask->schedInfo.pIdleTimer = taosTmrStart(streamTaskResumeHelper, pTask->status.schedIdleTime, pTask, streamTimer); + } else { + taosTmrReset(streamTaskResumeHelper, pTask->status.schedIdleTime, pTask, streamTimer, &pTask->schedInfo.pIdleTimer); + } + + return TSDB_CODE_SUCCESS; +} + +////////////////////////////////////////////////////////////////////////////////////////////////////////////////////// +void streamTaskResumeHelper(void* param, void* tmrId) { + SStreamTask* pTask = (SStreamTask*)param; + SStreamTaskId* pId = &pTask->id; + SStreamTaskState* p = streamTaskGetStatus(pTask); + + if (p->state == TASK_STATUS__DROPPING || p->state == TASK_STATUS__STOP) { + streamTaskSetSchedStatusInactive(pTask); + + int32_t ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1); + stDebug("s-task:%s status:%s not resume task, ref:%d", pId->idStr, p->name, ref); + + streamMetaReleaseTask(pTask->pMeta, pTask); + return; + } + + streamTaskSchedTask(pTask->pMsgCb, pTask->info.nodeId, pId->streamId, pId->taskId, STREAM_EXEC_T_RESUME_TASK); + + int32_t ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1); + stDebug("trigger to resume s-task:%s after being idled for %dms, ref:%d", pId->idStr, pTask->status.schedIdleTime, + ref); + + // release the task ref count + streamTaskClearSchedIdleInfo(pTask); + streamMetaReleaseTask(pTask->pMeta, pTask); +} + +void streamSchedByTimer(void* param, void* tmrId) { SStreamTask* pTask = (void*)param; const char* id = pTask->id.idStr; int32_t nextTrigger = (int32_t)pTask->info.triggerParam; @@ -61,69 +157,9 @@ static void streamSchedByTimer(void* param, void* tmrId) { return; } - streamSchedExec(pTask); + streamTrySchedExec(pTask); } } taosTmrReset(streamSchedByTimer, nextTrigger, pTask, streamTimer, &pTask->schedInfo.pDelayTimer); } - -int32_t streamSetupScheduleTrigger(SStreamTask* pTask) { - if (pTask->info.triggerParam != 0 && pTask->info.fillHistory == 0) { - int32_t ref = atomic_add_fetch_32(&pTask->refCnt, 1); - ASSERT(ref == 2 && pTask->schedInfo.pDelayTimer == NULL); - - stDebug("s-task:%s setup scheduler trigger, delay:%" PRId64 " ms", pTask->id.idStr, pTask->info.triggerParam); - - pTask->schedInfo.pDelayTimer = taosTmrStart(streamSchedByTimer, (int32_t)pTask->info.triggerParam, pTask, streamTimer); - pTask->schedInfo.status = TASK_TRIGGER_STATUS__INACTIVE; - } - - return 0; -} - -int32_t streamSchedExec(SStreamTask* pTask) { - if (streamTaskSetSchedStatusWait(pTask)) { - SStreamTaskRunReq* pRunReq = rpcMallocCont(sizeof(SStreamTaskRunReq)); - if (pRunReq == NULL) { - terrno = TSDB_CODE_OUT_OF_MEMORY; - /*int8_t status = */streamTaskSetSchedStatusInactive(pTask); - stError("failed to create msg to aunch s-task:%s, reason out of memory", pTask->id.idStr); - return -1; - } - - pRunReq->head.vgId = pTask->info.nodeId; - pRunReq->streamId = pTask->id.streamId; - pRunReq->taskId = pTask->id.taskId; - - stDebug("trigger to run s-task:%s", pTask->id.idStr); - - SRpcMsg msg = {.msgType = TDMT_STREAM_TASK_RUN, .pCont = pRunReq, .contLen = sizeof(SStreamTaskRunReq)}; - tmsgPutToQueue(pTask->pMsgCb, STREAM_QUEUE, &msg); - } else { - stTrace("s-task:%s not launch task since sched status:%d", pTask->id.idStr, pTask->status.schedStatus); - } - - return 0; -} - -int32_t streamTaskSchedTask(SMsgCb* pMsgCb, int32_t vgId, int64_t streamId, int32_t taskId, int32_t execType) { - SStreamTaskRunReq* pRunReq = rpcMallocCont(sizeof(SStreamTaskRunReq)); - if (pRunReq == NULL) { - terrno = TSDB_CODE_OUT_OF_MEMORY; - stError("vgId:%d failed to create msg to start stream task:0x%x, type:%d, code:%s", vgId, taskId, execType, - terrstr()); - return -1; - } - - stDebug("vgId:%d create msg to start stream task:0x%x", vgId, taskId); - - pRunReq->head.vgId = vgId; - pRunReq->streamId = streamId; - pRunReq->taskId = taskId; - pRunReq->reqType = execType; - - SRpcMsg msg = {.msgType = TDMT_STREAM_TASK_RUN, .pCont = pRunReq, .contLen = sizeof(SStreamTaskRunReq)}; - tmsgPutToQueue(pMsgCb, STREAM_QUEUE, &msg); - return TSDB_CODE_SUCCESS; -} diff --git a/source/libs/stream/src/streamTask.c b/source/libs/stream/src/streamTask.c index 1f0cbc1695dd..c7e34987ab58 100644 --- a/source/libs/stream/src/streamTask.c +++ b/source/libs/stream/src/streamTask.c @@ -1042,5 +1042,5 @@ int32_t streamProcessRetrieveReq(SStreamTask* pTask, SStreamRetrieveReq* pReq) { if(code != 0){ return code; } - return streamSchedExec(pTask); + return streamTrySchedExec(pTask); } \ No newline at end of file From 22effd5b1d387e5909648ac665107ea331992417 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 7 May 2024 09:46:12 +0800 Subject: [PATCH 06/30] refactor: do some internal refactor. --- source/libs/stream/inc/streamInt.h | 1 - source/libs/stream/src/streamCheckStatus.c | 26 +++ source/libs/stream/src/streamQueue.c | 30 +++ source/libs/stream/src/streamSched.c | 14 +- source/libs/stream/src/streamSnapshot.c | 37 ++- source/libs/stream/src/streamStartHistory.c | 241 ++++++++------------ 6 files changed, 176 insertions(+), 173 deletions(-) diff --git a/source/libs/stream/inc/streamInt.h b/source/libs/stream/inc/streamInt.h index 3a4e3d81fb1f..ceb6cd9739ab 100644 --- a/source/libs/stream/inc/streamInt.h +++ b/source/libs/stream/inc/streamInt.h @@ -158,7 +158,6 @@ typedef enum ECHECKPOINT_BACKUP_TYPE { ECHECKPOINT_BACKUP_TYPE streamGetCheckpointBackupType(); int32_t streamTaskDownloadCheckpointData(char* id, char* path); - int32_t streamTaskOnNormalTaskReady(SStreamTask* pTask); int32_t streamTaskOnScanhistoryTaskReady(SStreamTask* pTask); diff --git a/source/libs/stream/src/streamCheckStatus.c b/source/libs/stream/src/streamCheckStatus.c index f0f12cae2bfc..e6f127349c7f 100644 --- a/source/libs/stream/src/streamCheckStatus.c +++ b/source/libs/stream/src/streamCheckStatus.c @@ -162,6 +162,32 @@ int32_t streamProcessCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* pRs return 0; } +int32_t streamSendCheckRsp(const SStreamMeta* pMeta, const SStreamTaskCheckReq* pReq, SStreamTaskCheckRsp* pRsp, + SRpcHandleInfo* pRpcInfo, int32_t taskId) { + SEncoder encoder; + int32_t code; + int32_t len; + + tEncodeSize(tEncodeStreamTaskCheckRsp, pRsp, len, code); + if (code < 0) { + stError("vgId:%d failed to encode task check rsp, s-task:0x%x", pMeta->vgId, taskId); + return -1; + } + + void* buf = rpcMallocCont(sizeof(SMsgHead) + len); + ((SMsgHead*)buf)->vgId = htonl(pReq->upstreamNodeId); + + void* abuf = POINTER_SHIFT(buf, sizeof(SMsgHead)); + tEncoderInit(&encoder, (uint8_t*)abuf, len); + tEncodeStreamTaskCheckRsp(&encoder, pRsp); + tEncoderClear(&encoder); + + SRpcMsg rspMsg = {.code = 0, .pCont = buf, .contLen = sizeof(SMsgHead) + len, .info = *pRpcInfo}; + + tmsgSendRsp(&rspMsg); + return 0; +} + int32_t streamTaskStartMonitorCheckRsp(SStreamTask* pTask) { STaskCheckInfo* pInfo = &pTask->taskCheckInfo; taosThreadMutexLock(&pInfo->checkInfoLock); diff --git a/source/libs/stream/src/streamQueue.c b/source/libs/stream/src/streamQueue.c index 0ac282c36267..5596eb3dee70 100644 --- a/source/libs/stream/src/streamQueue.c +++ b/source/libs/stream/src/streamQueue.c @@ -330,6 +330,36 @@ int32_t streamTaskPutDataIntoInputQ(SStreamTask* pTask, SStreamQueueItem* pItem) return 0; } +int32_t streamTaskPutTranstateIntoInputQ(SStreamTask* pTask) { + SStreamDataBlock* pTranstate = taosAllocateQitem(sizeof(SStreamDataBlock), DEF_QITEM, sizeof(SSDataBlock)); + if (pTranstate == NULL) { + return TSDB_CODE_OUT_OF_MEMORY; + } + + SSDataBlock* pBlock = taosMemoryCalloc(1, sizeof(SSDataBlock)); + if (pBlock == NULL) { + taosFreeQitem(pTranstate); + return TSDB_CODE_OUT_OF_MEMORY; + } + + pTranstate->type = STREAM_INPUT__TRANS_STATE; + + pBlock->info.type = STREAM_TRANS_STATE; + pBlock->info.rows = 1; + pBlock->info.childId = pTask->info.selfChildId; + + pTranstate->blocks = taosArrayInit(4, sizeof(SSDataBlock)); // pBlock; + taosArrayPush(pTranstate->blocks, pBlock); + + taosMemoryFree(pBlock); + if (streamTaskPutDataIntoInputQ(pTask, (SStreamQueueItem*)pTranstate) < 0) { + return TSDB_CODE_OUT_OF_MEMORY; + } + + pTask->status.appendTranstateBlock = true; + return TSDB_CODE_SUCCESS; +} + // the result should be put into the outputQ in any cases, the result may be lost otherwise. int32_t streamTaskPutDataIntoOutputQ(SStreamTask* pTask, SStreamDataBlock* pBlock) { STaosQueue* pQueue = pTask->outputq.queue->pQueue; diff --git a/source/libs/stream/src/streamSched.c b/source/libs/stream/src/streamSched.c index 2e337234b6f2..52e7431e70b0 100644 --- a/source/libs/stream/src/streamSched.c +++ b/source/libs/stream/src/streamSched.c @@ -17,7 +17,7 @@ #include "ttimer.h" static void streamTaskResumeHelper(void* param, void* tmrId); -static void streamSchedByTimer(void* param, void* tmrId); +static void streamTaskSchedHelper(void* param, void* tmrId); int32_t streamSetupScheduleTrigger(SStreamTask* pTask) { if (pTask->info.triggerParam != 0 && pTask->info.fillHistory == 0) { @@ -26,7 +26,7 @@ int32_t streamSetupScheduleTrigger(SStreamTask* pTask) { stDebug("s-task:%s setup scheduler trigger, delay:%" PRId64 " ms", pTask->id.idStr, pTask->info.triggerParam); - pTask->schedInfo.pDelayTimer = taosTmrStart(streamSchedByTimer, (int32_t)pTask->info.triggerParam, pTask, streamTimer); + pTask->schedInfo.pDelayTimer = taosTmrStart(streamTaskSchedHelper, (int32_t)pTask->info.triggerParam, pTask, streamTimer); pTask->schedInfo.status = TASK_TRIGGER_STATUS__INACTIVE; } @@ -112,7 +112,7 @@ void streamTaskResumeHelper(void* param, void* tmrId) { streamMetaReleaseTask(pTask->pMeta, pTask); } -void streamSchedByTimer(void* param, void* tmrId) { +void streamTaskSchedHelper(void* param, void* tmrId) { SStreamTask* pTask = (void*)param; const char* id = pTask->id.idStr; int32_t nextTrigger = (int32_t)pTask->info.triggerParam; @@ -133,7 +133,7 @@ void streamSchedByTimer(void* param, void* tmrId) { if (pTrigger == NULL) { stError("s-task:%s failed to prepare retrieve data trigger, code:%s, try again in %dms", id, "out of memory", nextTrigger); - taosTmrReset(streamSchedByTimer, nextTrigger, pTask, streamTimer, &pTask->schedInfo.pDelayTimer); + taosTmrReset(streamTaskSchedHelper, nextTrigger, pTask, streamTimer, &pTask->schedInfo.pDelayTimer); return; } @@ -144,7 +144,7 @@ void streamSchedByTimer(void* param, void* tmrId) { stError("s-task:%s failed to prepare retrieve data trigger, code:%s, try again in %dms", id, "out of memory", nextTrigger); - taosTmrReset(streamSchedByTimer, nextTrigger, pTask, streamTimer, &pTask->schedInfo.pDelayTimer); + taosTmrReset(streamTaskSchedHelper, nextTrigger, pTask, streamTimer, &pTask->schedInfo.pDelayTimer); return; } @@ -153,7 +153,7 @@ void streamSchedByTimer(void* param, void* tmrId) { int32_t code = streamTaskPutDataIntoInputQ(pTask, (SStreamQueueItem*)pTrigger); if (code != TSDB_CODE_SUCCESS) { - taosTmrReset(streamSchedByTimer, nextTrigger, pTask, streamTimer, &pTask->schedInfo.pDelayTimer); + taosTmrReset(streamTaskSchedHelper, nextTrigger, pTask, streamTimer, &pTask->schedInfo.pDelayTimer); return; } @@ -161,5 +161,5 @@ void streamSchedByTimer(void* param, void* tmrId) { } } - taosTmrReset(streamSchedByTimer, nextTrigger, pTask, streamTimer, &pTask->schedInfo.pDelayTimer); + taosTmrReset(streamTaskSchedHelper, nextTrigger, pTask, streamTimer, &pTask->schedInfo.pDelayTimer); } diff --git a/source/libs/stream/src/streamSnapshot.c b/source/libs/stream/src/streamSnapshot.c index a69cb75dad56..1800324cc89a 100644 --- a/source/libs/stream/src/streamSnapshot.c +++ b/source/libs/stream/src/streamSnapshot.c @@ -32,6 +32,7 @@ typedef struct SBackendFileItem { int64_t size; int8_t ref; } SBackendFileItem; + typedef struct SBackendFile { char* pCurrent; char* pMainfest; @@ -102,6 +103,7 @@ struct SStreamSnapWriter { int64_t ever; SStreamSnapHandle handle; }; + const char* ROCKSDB_OPTIONS = "OPTIONS"; const char* ROCKSDB_MAINFEST = "MANIFEST"; const char* ROCKSDB_SST = "sst"; @@ -120,7 +122,7 @@ void streamSnapHandleDestroy(SStreamSnapHandle* handle); } while (0) int32_t streamGetFileSize(char* path, char* name, int64_t* sz) { - int ret = 0; + int32_t ret = 0; char* fullname = taosMemoryCalloc(1, strlen(path) + 32); sprintf(fullname, "%s%s%s", path, TD_DIRSEP, name); @@ -149,7 +151,7 @@ void snapFileDebugInfo(SBackendSnapFile2* pSnapFile) { if (pSnapFile->pMainfest) sprintf(buf + strlen(buf), "MANIFEST: %s,", pSnapFile->pMainfest); if (pSnapFile->pOptions) sprintf(buf + strlen(buf), "options: %s,", pSnapFile->pOptions); if (pSnapFile->pSst) { - for (int i = 0; i < taosArrayGetSize(pSnapFile->pSst); i++) { + for (int32_t i = 0; i < taosArrayGetSize(pSnapFile->pSst); i++) { char* name = taosArrayGetP(pSnapFile->pSst, i); sprintf(buf + strlen(buf), "%s,", name); } @@ -157,7 +159,7 @@ void snapFileDebugInfo(SBackendSnapFile2* pSnapFile) { sprintf(buf + strlen(buf) - 1, "]"); stInfo("%s %" PRId64 "-%" PRId64 " get file list: %s", STREAM_STATE_TRANSFER, pSnapFile->snapInfo.streamId, - pSnapFile->snapInfo.taskId, buf); + pSnapFile->snapInfo.taskId, buf); taosMemoryFree(buf); } } @@ -183,7 +185,7 @@ int32_t snapFileGenMeta(SBackendSnapFile2* pSnapFile) { streamGetFileSize(pSnapFile->path, item.name, &item.size); taosArrayPush(pSnapFile->pFileList, &item); // sst - for (int i = 0; i < taosArrayGetSize(pSnapFile->pSst); i++) { + for (int32_t i = 0; i < taosArrayGetSize(pSnapFile->pSst); i++) { char* sst = taosArrayGetP(pSnapFile->pSst, i); item.name = sst; item.type = ROCKSDB_SST_TYPE; @@ -270,12 +272,12 @@ void snapFileDestroy(SBackendSnapFile2* pSnap) { taosMemoryFree(pSnap->pMainfest); taosMemoryFree(pSnap->pOptions); taosMemoryFree(pSnap->path); - for (int i = 0; i < taosArrayGetSize(pSnap->pSst); i++) { + for (int32_t i = 0; i < taosArrayGetSize(pSnap->pSst); i++) { char* sst = taosArrayGetP(pSnap->pSst, i); taosMemoryFree(sst); } // unite read/write snap file - for (int i = 0; i < taosArrayGetSize(pSnap->pFileList); i++) { + for (int32_t i = 0; i < taosArrayGetSize(pSnap->pFileList); i++) { SBackendFileItem* pItem = taosArrayGet(pSnap->pFileList, i); if (pItem->ref == 0) { taosMemoryFree(pItem->name); @@ -297,7 +299,7 @@ int32_t streamSnapHandleInit(SStreamSnapHandle* pHandle, char* path, void* pMeta SArray* pDbSnapSet = taosArrayInit(8, sizeof(SBackendSnapFile2)); - for (int i = 0; i < taosArrayGetSize(pSnapSet); i++) { + for (int32_t i = 0; i < taosArrayGetSize(pSnapSet); i++) { SStreamTaskSnap* pSnap = taosArrayGet(pSnapSet, i); SBackendSnapFile2 snapFile = {0}; @@ -305,7 +307,7 @@ int32_t streamSnapHandleInit(SStreamSnapHandle* pHandle, char* path, void* pMeta ASSERT(code == 0); taosArrayPush(pDbSnapSet, &snapFile); } - for (int i = 0; i < taosArrayGetSize(pSnapSet); i++) { + for (int32_t i = 0; i < taosArrayGetSize(pSnapSet); i++) { SStreamTaskSnap* pSnap = taosArrayGet(pSnapSet, i); taosMemoryFree(pSnap->dbPrefixPath); } @@ -324,7 +326,7 @@ int32_t streamSnapHandleInit(SStreamSnapHandle* pHandle, char* path, void* pMeta void streamSnapHandleDestroy(SStreamSnapHandle* handle) { if (handle->pDbSnapSet) { - for (int i = 0; i < taosArrayGetSize(handle->pDbSnapSet); i++) { + for (int32_t i = 0; i < taosArrayGetSize(handle->pDbSnapSet); i++) { SBackendSnapFile2* pSnapFile = taosArrayGet(handle->pDbSnapSet, i); snapFileDebugInfo(pSnapFile); snapFileDestroy(pSnapFile); @@ -396,9 +398,9 @@ int32_t streamSnapRead(SStreamSnapReader* pReader, uint8_t** ppData, int64_t* si item = taosArrayGet(pSnapFile->pFileList, pSnapFile->currFileIdx); stDebug("%s start to read file %s, current offset:%" PRId64 ", size:%" PRId64 - ", file no.%d, total set:%d, current set idx: %d", - STREAM_STATE_TRANSFER, item->name, (int64_t)pSnapFile->offset, item->size, pSnapFile->currFileIdx, - (int)taosArrayGetSize(pHandle->pDbSnapSet), pHandle->currIdx); + ", file no.%d, total set:%d, current set idx: %d", + STREAM_STATE_TRANSFER, item->name, (int64_t)pSnapFile->offset, item->size, pSnapFile->currFileIdx, + (int32_t)taosArrayGetSize(pHandle->pDbSnapSet), pHandle->currIdx); uint8_t* buf = taosMemoryCalloc(1, sizeof(SStreamSnapBlockHdr) + kBlockSize); int64_t nread = taosPReadFile(pSnapFile->fd, buf + sizeof(SStreamSnapBlockHdr), kBlockSize, pSnapFile->offset); @@ -489,14 +491,10 @@ int32_t snapInfoEqual(SStreamTaskSnap* a, SStreamTaskSnap* b) { } int32_t streamSnapWriteImpl(SStreamSnapWriter* pWriter, uint8_t* pData, uint32_t nData, SBackendSnapFile2* pSnapFile) { - int code = -1; + int32_t code = -1; SStreamSnapBlockHdr* pHdr = (SStreamSnapBlockHdr*)pData; SStreamSnapHandle* pHandle = &pWriter->handle; - SStreamTaskSnap snapInfo = pHdr->snapInfo; - - SStreamTaskSnap* pSnapInfo = &pSnapFile->snapInfo; - - SBackendFileItem* pItem = taosArrayGet(pSnapFile->pFileList, pSnapFile->currFileIdx); + SBackendFileItem* pItem = taosArrayGet(pSnapFile->pFileList, pSnapFile->currFileIdx); if (pSnapFile->fd == 0) { pSnapFile->fd = streamOpenFile(pSnapFile->path, pItem->name, TD_FILE_CREATE | TD_FILE_WRITE | TD_FILE_APPEND); @@ -540,6 +538,7 @@ int32_t streamSnapWriteImpl(SStreamSnapWriter* pWriter, uint8_t* pData, uint32_t pSnapFile->offset += pHdr->size; } code = 0; + _EXIT: return code; } @@ -590,8 +589,8 @@ int32_t streamSnapWrite(SStreamSnapWriter* pWriter, uint8_t* pData, uint32_t nDa return streamSnapWrite(pWriter, pData, nData); } } - return code; } + int32_t streamSnapWriterClose(SStreamSnapWriter* pWriter, int8_t rollback) { if (pWriter == NULL) return 0; streamSnapHandleDestroy(&pWriter->handle); diff --git a/source/libs/stream/src/streamStartHistory.c b/source/libs/stream/src/streamStartHistory.c index b3df5755eafd..98c0d95781e8 100644 --- a/source/libs/stream/src/streamStartHistory.c +++ b/source/libs/stream/src/streamStartHistory.c @@ -38,8 +38,12 @@ static void tryLaunchHistoryTask(void* param, void* tmrId); static void doExecScanhistoryInFuture(void* param, void* tmrId); static int32_t doStartScanHistoryTask(SStreamTask* pTask); static int32_t streamTaskStartScanHistory(SStreamTask* pTask); +static void checkFillhistoryTaskStatus(SStreamTask* pTask, SStreamTask* pHTask); +static int32_t launchNotBuiltFillHistoryTask(SStreamTask* pTask); +static void doRetryLaunchFillHistoryTask(SStreamTask* pTask, SLaunchHTaskInfo* pInfo, int64_t now); +static void notRetryLaunchFillHistoryTask(SStreamTask* pTask, SLaunchHTaskInfo* pInfo, int64_t now); -int32_t streamTaskSetReady(SStreamTask* pTask) { +static int32_t streamTaskSetReady(SStreamTask* pTask) { int32_t numOfDowns = streamTaskGetNumOfDownstream(pTask); SStreamTaskState* p = streamTaskGetStatus(pTask); @@ -79,33 +83,6 @@ int32_t streamStartScanHistoryAsync(SStreamTask* pTask, int8_t igUntreated) { return 0; } -void doExecScanhistoryInFuture(void* param, void* tmrId) { - SStreamTask* pTask = param; - pTask->schedHistoryInfo.numOfTicks -= 1; - - SStreamTaskState* p = streamTaskGetStatus(pTask); - if (p->state == TASK_STATUS__DROPPING || p->state == TASK_STATUS__STOP) { - int32_t ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1); - stDebug("s-task:%s status:%s not start scan-history again, ref:%d", pTask->id.idStr, p->name, ref); - - streamMetaReleaseTask(pTask->pMeta, pTask); - return; - } - - if (pTask->schedHistoryInfo.numOfTicks <= 0) { - streamStartScanHistoryAsync(pTask, 0); - - int32_t ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1); - stDebug("s-task:%s fill-history:%d start scan-history data, out of tmr, ref:%d", pTask->id.idStr, - pTask->info.fillHistory, ref); - - // release the task. - streamMetaReleaseTask(pTask->pMeta, pTask); - } else { - taosTmrReset(doExecScanhistoryInFuture, SCANHISTORY_IDLE_TIME_SLICE, pTask, streamTimer, &pTask->schedHistoryInfo.pTimer); - } -} - int32_t streamExecScanHistoryInFuture(SStreamTask* pTask, int32_t idleDuration) { int32_t numOfTicks = idleDuration / SCANHISTORY_IDLE_TIME_SLICE; if (numOfTicks <= 0) { @@ -136,17 +113,6 @@ int32_t streamExecScanHistoryInFuture(SStreamTask* pTask, int32_t idleDuration) return TSDB_CODE_SUCCESS; } -int32_t doStartScanHistoryTask(SStreamTask* pTask) { - SVersionRange* pRange = &pTask->dataRange.range; - if (pTask->info.fillHistory) { - streamSetParamForScanHistory(pTask); - } - - streamSetParamForStreamScannerStep1(pTask, pRange, &pTask->dataRange.window); - int32_t code = streamStartScanHistoryAsync(pTask, 0); - return code; -} - int32_t streamTaskStartScanHistory(SStreamTask* pTask) { int32_t level = pTask->info.taskLevel; ETaskStatus status = streamTaskGetStatus(pTask)->state; @@ -267,32 +233,6 @@ int32_t streamTaskOnScanhistoryTaskReady(SStreamTask* pTask) { return TSDB_CODE_SUCCESS; } -int32_t streamSendCheckRsp(const SStreamMeta* pMeta, const SStreamTaskCheckReq* pReq, SStreamTaskCheckRsp* pRsp, - SRpcHandleInfo* pRpcInfo, int32_t taskId) { - SEncoder encoder; - int32_t code; - int32_t len; - - tEncodeSize(tEncodeStreamTaskCheckRsp, pRsp, len, code); - if (code < 0) { - stError("vgId:%d failed to encode task check rsp, s-task:0x%x", pMeta->vgId, taskId); - return -1; - } - - void* buf = rpcMallocCont(sizeof(SMsgHead) + len); - ((SMsgHead*)buf)->vgId = htonl(pReq->upstreamNodeId); - - void* abuf = POINTER_SHIFT(buf, sizeof(SMsgHead)); - tEncoderInit(&encoder, (uint8_t*)abuf, len); - tEncodeStreamTaskCheckRsp(&encoder, pRsp); - tEncoderClear(&encoder); - - SRpcMsg rspMsg = {.code = 0, .pCont = buf, .contLen = sizeof(SMsgHead) + len, .info = *pRpcInfo}; - - tmsgSendRsp(&rspMsg); - return 0; -} - // common int32_t streamSetParamForScanHistory(SStreamTask* pTask) { stDebug("s-task:%s set operator option for scan-history data", pTask->id.idStr); @@ -308,45 +248,64 @@ int32_t streamSetParamForStreamScannerStep2(SStreamTask* pTask, SVersionRange* p return qStreamSourceScanParamForHistoryScanStep2(pTask->exec.pExecutor, pVerRange, pWindow); } -int32_t initScanHistoryReq(SStreamTask* pTask, SStreamScanHistoryReq* pReq, int8_t igUntreated) { - pReq->msgHead.vgId = pTask->info.nodeId; - pReq->streamId = pTask->id.streamId; - pReq->taskId = pTask->id.taskId; - pReq->igUntreated = igUntreated; - return 0; -} +// an fill history task needs to be started. +int32_t streamLaunchFillHistoryTask(SStreamTask* pTask) { + SStreamMeta* pMeta = pTask->pMeta; + STaskExecStatisInfo* pExecInfo = &pTask->execInfo; + const char* idStr = pTask->id.idStr; + int64_t hStreamId = pTask->hTaskInfo.id.streamId; + int32_t hTaskId = pTask->hTaskInfo.id.taskId; + ASSERT(hTaskId != 0); -int32_t streamTaskPutTranstateIntoInputQ(SStreamTask* pTask) { - SStreamDataBlock* pTranstate = taosAllocateQitem(sizeof(SStreamDataBlock), DEF_QITEM, sizeof(SSDataBlock)); - if (pTranstate == NULL) { - return TSDB_CODE_OUT_OF_MEMORY; - } + // check stream task status in the first place. + SStreamTaskState* pStatus = streamTaskGetStatus(pTask); + if (pStatus->state != TASK_STATUS__READY && pStatus->state != TASK_STATUS__HALT) { + stDebug("s-task:%s not launch related fill-history task:0x%" PRIx64 "-0x%x, status:%s", idStr, hStreamId, hTaskId, + pStatus->name); - SSDataBlock* pBlock = taosMemoryCalloc(1, sizeof(SSDataBlock)); - if (pBlock == NULL) { - taosFreeQitem(pTranstate); - return TSDB_CODE_OUT_OF_MEMORY; + streamMetaAddTaskLaunchResult(pMeta, hStreamId, hTaskId, pExecInfo->checkTs, pExecInfo->readyTs, false); + return -1; // todo set the correct error code } - pTranstate->type = STREAM_INPUT__TRANS_STATE; + stDebug("s-task:%s start to launch related fill-history task:0x%" PRIx64 "-0x%x", idStr, hStreamId, hTaskId); - pBlock->info.type = STREAM_TRANS_STATE; - pBlock->info.rows = 1; - pBlock->info.childId = pTask->info.selfChildId; + // Set the execute conditions, including the query time window and the version range + streamMetaRLock(pMeta); + SStreamTask** pHTask = taosHashGet(pMeta->pTasksMap, &pTask->hTaskInfo.id, sizeof(pTask->hTaskInfo.id)); + streamMetaRUnLock(pMeta); - pTranstate->blocks = taosArrayInit(4, sizeof(SSDataBlock)); // pBlock; - taosArrayPush(pTranstate->blocks, pBlock); + if (pHTask != NULL) { // it is already added into stream meta store. + SStreamTask* pHisTask = streamMetaAcquireTask(pMeta, hStreamId, hTaskId); + if (pHisTask == NULL) { + stDebug("s-task:%s failed acquire and start fill-history task, it may have been dropped/stopped", idStr); + streamMetaAddTaskLaunchResult(pMeta, hStreamId, hTaskId, pExecInfo->checkTs, pExecInfo->readyTs, false); + } else { + if (pHisTask->status.downstreamReady == 1) { // it's ready now, do nothing + stDebug("s-task:%s fill-history task is ready, no need to check downstream", pHisTask->id.idStr); + streamMetaAddTaskLaunchResult(pMeta, hStreamId, hTaskId, pExecInfo->checkTs, pExecInfo->readyTs, true); + } else { // exist, but not ready, continue check downstream task status + checkFillhistoryTaskStatus(pTask, pHisTask); + } - taosMemoryFree(pBlock); - if (streamTaskPutDataIntoInputQ(pTask, (SStreamQueueItem*)pTranstate) < 0) { - return TSDB_CODE_OUT_OF_MEMORY; + streamMetaReleaseTask(pMeta, pHisTask); + } + + return TSDB_CODE_SUCCESS; + } else { + return launchNotBuiltFillHistoryTask(pTask); } +} - pTask->status.appendTranstateBlock = true; - return TSDB_CODE_SUCCESS; +/////////////////////////////////////////////////////////////////////////////////////////////////////////////////////// +int32_t initScanHistoryReq(SStreamTask* pTask, SStreamScanHistoryReq* pReq, int8_t igUntreated) { + pReq->msgHead.vgId = pTask->info.nodeId; + pReq->streamId = pTask->id.streamId; + pReq->taskId = pTask->id.taskId; + pReq->igUntreated = igUntreated; + return 0; } -static void checkFillhistoryTaskStatus(SStreamTask* pTask, SStreamTask* pHTask) { +void checkFillhistoryTaskStatus(SStreamTask* pTask, SStreamTask* pHTask) { SDataRange* pRange = &pHTask->dataRange; // the query version range should be limited to the already processed data @@ -365,7 +324,7 @@ static void checkFillhistoryTaskStatus(SStreamTask* pTask, SStreamTask* pHTask) streamTaskHandleEvent(pHTask->status.pSM, TASK_EVENT_INIT_SCANHIST); } -static void noRetryLaunchFillHistoryTask(SStreamTask* pTask, SLaunchHTaskInfo* pInfo, int64_t now) { +void notRetryLaunchFillHistoryTask(SStreamTask* pTask, SLaunchHTaskInfo* pInfo, int64_t now) { SStreamMeta* pMeta = pTask->pMeta; SHistoryTaskInfo* pHTaskInfo = &pTask->hTaskInfo; @@ -379,7 +338,7 @@ static void noRetryLaunchFillHistoryTask(SStreamTask* pTask, SLaunchHTaskInfo* p pHTaskInfo->id.streamId = 0; } -static void doRetryLaunchFillHistoryTask(SStreamTask* pTask, SLaunchHTaskInfo* pInfo, int64_t now) { +void doRetryLaunchFillHistoryTask(SStreamTask* pTask, SLaunchHTaskInfo* pInfo, int64_t now) { SStreamMeta* pMeta = pTask->pMeta; SHistoryTaskInfo* pHTaskInfo = &pTask->hTaskInfo; @@ -401,7 +360,7 @@ static void doRetryLaunchFillHistoryTask(SStreamTask* pTask, SLaunchHTaskInfo* p } } -static void tryLaunchHistoryTask(void* param, void* tmrId) { +void tryLaunchHistoryTask(void* param, void* tmrId) { SLaunchHTaskInfo* pInfo = param; SStreamMeta* pMeta = pInfo->pMeta; int64_t now = taosGetTimestampMs(); @@ -449,7 +408,7 @@ static void tryLaunchHistoryTask(void* param, void* tmrId) { } if (pHTaskInfo->retryTimes > MAX_RETRY_LAUNCH_HISTORY_TASK) { - noRetryLaunchFillHistoryTask(pTask, pInfo, now); + notRetryLaunchFillHistoryTask(pTask, pInfo, now); } else { // not reach the limitation yet, let's continue retrying launch related fill-history task. streamTaskSetRetryInfoForLaunch(pHTaskInfo); ASSERT(pTask->status.timerActive >= 1); @@ -500,7 +459,7 @@ SLaunchHTaskInfo* createHTaskLaunchInfo(SStreamMeta* pMeta, STaskId* pTaskId, in return pInfo; } -static int32_t launchNotBuiltFillHistoryTask(SStreamTask* pTask) { +int32_t launchNotBuiltFillHistoryTask(SStreamTask* pTask) { SStreamMeta* pMeta = pTask->pMeta; STaskExecStatisInfo* pExecInfo = &pTask->execInfo; const char* idStr = pTask->id.idStr; @@ -547,54 +506,6 @@ static int32_t launchNotBuiltFillHistoryTask(SStreamTask* pTask) { return TSDB_CODE_SUCCESS; } -// an fill history task needs to be started. -int32_t streamLaunchFillHistoryTask(SStreamTask* pTask) { - SStreamMeta* pMeta = pTask->pMeta; - STaskExecStatisInfo* pExecInfo = &pTask->execInfo; - const char* idStr = pTask->id.idStr; - int64_t hStreamId = pTask->hTaskInfo.id.streamId; - int32_t hTaskId = pTask->hTaskInfo.id.taskId; - ASSERT(hTaskId != 0); - - // check stream task status in the first place. - SStreamTaskState* pStatus = streamTaskGetStatus(pTask); - if (pStatus->state != TASK_STATUS__READY && pStatus->state != TASK_STATUS__HALT) { - stDebug("s-task:%s not launch related fill-history task:0x%" PRIx64 "-0x%x, status:%s", idStr, hStreamId, hTaskId, - pStatus->name); - - streamMetaAddTaskLaunchResult(pMeta, hStreamId, hTaskId, pExecInfo->checkTs, pExecInfo->readyTs, false); - return -1; // todo set the correct error code - } - - stDebug("s-task:%s start to launch related fill-history task:0x%" PRIx64 "-0x%x", idStr, hStreamId, hTaskId); - - // Set the execute conditions, including the query time window and the version range - streamMetaRLock(pMeta); - SStreamTask** pHTask = taosHashGet(pMeta->pTasksMap, &pTask->hTaskInfo.id, sizeof(pTask->hTaskInfo.id)); - streamMetaRUnLock(pMeta); - - if (pHTask != NULL) { // it is already added into stream meta store. - SStreamTask* pHisTask = streamMetaAcquireTask(pMeta, hStreamId, hTaskId); - if (pHisTask == NULL) { - stDebug("s-task:%s failed acquire and start fill-history task, it may have been dropped/stopped", idStr); - streamMetaAddTaskLaunchResult(pMeta, hStreamId, hTaskId, pExecInfo->checkTs, pExecInfo->readyTs, false); - } else { - if (pHisTask->status.downstreamReady == 1) { // it's ready now, do nothing - stDebug("s-task:%s fill-history task is ready, no need to check downstream", pHisTask->id.idStr); - streamMetaAddTaskLaunchResult(pMeta, hStreamId, hTaskId, pExecInfo->checkTs, pExecInfo->readyTs, true); - } else { // exist, but not ready, continue check downstream task status - checkFillhistoryTaskStatus(pTask, pHisTask); - } - - streamMetaReleaseTask(pMeta, pHisTask); - } - - return TSDB_CODE_SUCCESS; - } else { - return launchNotBuiltFillHistoryTask(pTask); - } -} - int32_t streamTaskResetTimewindowFilter(SStreamTask* pTask) { void* exec = pTask->exec.pExecutor; return qStreamInfoResetTimewindowFilter(exec); @@ -651,3 +562,41 @@ void streamTaskSetRangeStreamCalc(SStreamTask* pTask) { streamSetParamForStreamScannerStep2(pTask, &verRange, &win); } } + +void doExecScanhistoryInFuture(void* param, void* tmrId) { + SStreamTask* pTask = param; + pTask->schedHistoryInfo.numOfTicks -= 1; + + SStreamTaskState* p = streamTaskGetStatus(pTask); + if (p->state == TASK_STATUS__DROPPING || p->state == TASK_STATUS__STOP) { + int32_t ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1); + stDebug("s-task:%s status:%s not start scan-history again, ref:%d", pTask->id.idStr, p->name, ref); + + streamMetaReleaseTask(pTask->pMeta, pTask); + return; + } + + if (pTask->schedHistoryInfo.numOfTicks <= 0) { + streamStartScanHistoryAsync(pTask, 0); + + int32_t ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1); + stDebug("s-task:%s fill-history:%d start scan-history data, out of tmr, ref:%d", pTask->id.idStr, + pTask->info.fillHistory, ref); + + // release the task. + streamMetaReleaseTask(pTask->pMeta, pTask); + } else { + taosTmrReset(doExecScanhistoryInFuture, SCANHISTORY_IDLE_TIME_SLICE, pTask, streamTimer, &pTask->schedHistoryInfo.pTimer); + } +} + +int32_t doStartScanHistoryTask(SStreamTask* pTask) { + SVersionRange* pRange = &pTask->dataRange.range; + if (pTask->info.fillHistory) { + streamSetParamForScanHistory(pTask); + } + + streamSetParamForStreamScannerStep1(pTask, pRange, &pTask->dataRange.window); + int32_t code = streamStartScanHistoryAsync(pTask, 0); + return code; +} From 332d1877224a717eac3cf7b3237c07993ff09985 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 7 May 2024 10:50:44 +0800 Subject: [PATCH 07/30] refactor: do some internal refactor. --- include/libs/stream/tstream.h | 9 ++-- source/dnode/vnode/src/tqCommon/tqCommon.c | 44 +++---------------- source/libs/stream/src/streamCheckStatus.c | 51 ++++++++++++++++++++-- source/libs/stream/src/streamTaskSm.c | 2 +- 4 files changed, 57 insertions(+), 49 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 1db62abfc021..02fa31ef0736 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -819,9 +819,6 @@ ETaskStatus streamTaskGetPrevStatus(const SStreamTask* pTask); void initRpcMsg(SRpcMsg* pMsg, int32_t msgType, void* pCont, int32_t contLen); -// recover and fill history -void streamTaskCheckDownstream(SStreamTask* pTask); - int32_t streamTaskCheckStatus(SStreamTask* pTask, int32_t upstreamId, int32_t vgId, int64_t stage, int64_t* oldStage); bool streamTaskUpdateEpsetInfo(SStreamTask* pTask, SArray* pNodeList); void streamTaskResetUpstreamStageInfo(SStreamTask* pTask); @@ -838,8 +835,10 @@ int32_t streamTaskHandleEventAsync(SStreamTaskSM* pSM, EStreamTaskEvent event, _ int32_t streamTaskOnHandleEventSuccess(SStreamTaskSM* pSM, EStreamTaskEvent event, __state_trans_user_fn callbackFn, void* param); int32_t streamTaskRestoreStatus(SStreamTask* pTask); -int32_t streamSendCheckRsp(const SStreamMeta* pMeta, const SStreamTaskCheckReq* pReq, SStreamTaskCheckRsp* pRsp, - SRpcHandleInfo* pRpcInfo, int32_t taskId); +void streamTaskSendCheckMsg(SStreamTask* pTask); +void streamTaskProcessCheckMsg(SStreamMeta* pMeta, SStreamTaskCheckReq* pReq, SStreamTaskCheckRsp* pRsp); +int32_t streamSendCheckRsp(const SStreamMeta* pMeta, int32_t vgId, SStreamTaskCheckRsp* pRsp, SRpcHandleInfo* pRpcInfo, + int32_t taskId); int32_t streamProcessCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* pRsp); int32_t streamLaunchFillHistoryTask(SStreamTask* pTask); int32_t streamStartScanHistoryAsync(SStreamTask* pTask, int8_t igUntreated); diff --git a/source/dnode/vnode/src/tqCommon/tqCommon.c b/source/dnode/vnode/src/tqCommon/tqCommon.c index b2068c99230b..9da8d05deb07 100644 --- a/source/dnode/vnode/src/tqCommon/tqCommon.c +++ b/source/dnode/vnode/src/tqCommon/tqCommon.c @@ -406,50 +406,16 @@ int32_t tqStreamTaskProcessCheckReq(SStreamMeta* pMeta, SRpcMsg* pMsg) { int32_t msgLen = pMsg->contLen - sizeof(SMsgHead); SStreamTaskCheckReq req; - SDecoder decoder; + SStreamTaskCheckRsp rsp = {0}; + + SDecoder decoder; tDecoderInit(&decoder, (uint8_t*)msgBody, msgLen); tDecodeStreamTaskCheckReq(&decoder, &req); tDecoderClear(&decoder); - int32_t taskId = req.downstreamTaskId; - - SStreamTaskCheckRsp rsp = { - .reqId = req.reqId, - .streamId = req.streamId, - .childId = req.childId, - .downstreamNodeId = req.downstreamNodeId, - .downstreamTaskId = req.downstreamTaskId, - .upstreamNodeId = req.upstreamNodeId, - .upstreamTaskId = req.upstreamTaskId, - }; - - // only the leader node handle the check request - if (pMeta->role == NODE_ROLE_FOLLOWER) { - tqError( - "s-task:0x%x invalid check msg from upstream:0x%x(vgId:%d), vgId:%d is follower, not handle check status msg", - taskId, req.upstreamTaskId, req.upstreamNodeId, pMeta->vgId); - rsp.status = TASK_DOWNSTREAM_NOT_LEADER; - } else { - SStreamTask* pTask = streamMetaAcquireTask(pMeta, req.streamId, taskId); - if (pTask != NULL) { - rsp.status = streamTaskCheckStatus(pTask, req.upstreamTaskId, req.upstreamNodeId, req.stage, &rsp.oldStage); - streamMetaReleaseTask(pMeta, pTask); - - SStreamTaskState* pState = streamTaskGetStatus(pTask); - tqDebug("s-task:%s status:%s, stage:%" PRId64 " recv task check req(reqId:0x%" PRIx64 - ") task:0x%x (vgId:%d), check_status:%d", - pTask->id.idStr, pState->name, rsp.oldStage, rsp.reqId, rsp.upstreamTaskId, rsp.upstreamNodeId, - rsp.status); - } else { - rsp.status = TASK_DOWNSTREAM_NOT_READY; - tqDebug("tq recv task check(taskId:0x%" PRIx64 "-0x%x not built yet) req(reqId:0x%" PRIx64 - ") from task:0x%x (vgId:%d), rsp check_status %d", - req.streamId, taskId, rsp.reqId, rsp.upstreamTaskId, rsp.upstreamNodeId, rsp.status); - } - } - - return streamSendCheckRsp(pMeta, &req, &rsp, &pMsg->info, taskId); + streamTaskProcessCheckMsg(pMeta, &req, &rsp); + return streamSendCheckRsp(pMeta, req.upstreamNodeId, &rsp, &pMsg->info, req.upstreamTaskId); } int32_t tqStreamTaskProcessCheckRsp(SStreamMeta* pMeta, SRpcMsg* pMsg, bool isLeader) { diff --git a/source/libs/stream/src/streamCheckStatus.c b/source/libs/stream/src/streamCheckStatus.c index e6f127349c7f..9c3908c83372 100644 --- a/source/libs/stream/src/streamCheckStatus.c +++ b/source/libs/stream/src/streamCheckStatus.c @@ -39,7 +39,7 @@ static int32_t addDownstreamFailedStatusResultAsync(SMsgCb* pMsgCb, int32_t vgId static SDownstreamStatusInfo* findCheckRspStatus(STaskCheckInfo* pInfo, int32_t taskId); // check status -void streamTaskCheckDownstream(SStreamTask* pTask) { +void streamTaskSendCheckMsg(SStreamTask* pTask) { SDataRange* pRange = &pTask->dataRange; STimeWindow* pWindow = &pRange->window; const char* idstr = pTask->id.idStr; @@ -97,6 +97,46 @@ void streamTaskCheckDownstream(SStreamTask* pTask) { } } +void streamTaskProcessCheckMsg(SStreamMeta* pMeta, SStreamTaskCheckReq* pReq, SStreamTaskCheckRsp* pRsp) { + int32_t taskId = pReq->downstreamTaskId; + + *pRsp = (SStreamTaskCheckRsp){ + .reqId = pReq->reqId, + .streamId = pReq->streamId, + .childId = pReq->childId, + .downstreamNodeId = pReq->downstreamNodeId, + .downstreamTaskId = pReq->downstreamTaskId, + .upstreamNodeId = pReq->upstreamNodeId, + .upstreamTaskId = pReq->upstreamTaskId, + }; + + // only the leader node handle the check request + if (pMeta->role == NODE_ROLE_FOLLOWER) { + stError( + "s-task:0x%x invalid check msg from upstream:0x%x(vgId:%d), vgId:%d is follower, not handle check status msg", + taskId, pReq->upstreamTaskId, pReq->upstreamNodeId, pMeta->vgId); + pRsp->status = TASK_DOWNSTREAM_NOT_LEADER; + } else { + SStreamTask* pTask = streamMetaAcquireTask(pMeta, pReq->streamId, taskId); + if (pTask != NULL) { + pRsp->status = streamTaskCheckStatus(pTask, pReq->upstreamTaskId, pReq->upstreamNodeId, pReq->stage, &pRsp->oldStage); + streamMetaReleaseTask(pMeta, pTask); + + SStreamTaskState* pState = streamTaskGetStatus(pTask); + stDebug("s-task:%s status:%s, stage:%" PRId64 " recv task check req(reqId:0x%" PRIx64 + ") task:0x%x (vgId:%d), check_status:%d", + pTask->id.idStr, pState->name, pRsp->oldStage, pRsp->reqId, pRsp->upstreamTaskId, pRsp->upstreamNodeId, + pRsp->status); + } else { + pRsp->status = TASK_DOWNSTREAM_NOT_READY; + stDebug("tq recv task check(taskId:0x%" PRIx64 "-0x%x not built yet) req(reqId:0x%" PRIx64 + ") from task:0x%x (vgId:%d), rsp check_status %d", + pReq->streamId, taskId, pRsp->reqId, pRsp->upstreamTaskId, pRsp->upstreamNodeId, pRsp->status); + } + } + +} + int32_t streamProcessCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* pRsp) { ASSERT(pTask->id.taskId == pRsp->upstreamTaskId); @@ -152,7 +192,7 @@ int32_t streamProcessCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* pRs STaskId* pId = &pTask->hTaskInfo.id; streamMetaAddTaskLaunchResult(pTask->pMeta, pId->streamId, pId->taskId, startTs, now, false); } - } else { // TASK_DOWNSTREAM_NOT_READY, let's retry in 100ms + } else { // TASK_DOWNSTREAM_NOT_READY, rsp-check monitor will retry in 300 ms ASSERT(left > 0); stDebug("s-task:%s (vgId:%d) recv check rsp from task:0x%x (vgId:%d) status:%d, total:%d not ready:%d", id, pRsp->upstreamNodeId, pRsp->downstreamTaskId, pRsp->downstreamNodeId, pRsp->status, total, left); @@ -162,7 +202,7 @@ int32_t streamProcessCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* pRs return 0; } -int32_t streamSendCheckRsp(const SStreamMeta* pMeta, const SStreamTaskCheckReq* pReq, SStreamTaskCheckRsp* pRsp, +int32_t streamSendCheckRsp(const SStreamMeta* pMeta, int32_t vgId, SStreamTaskCheckRsp* pRsp, SRpcHandleInfo* pRpcInfo, int32_t taskId) { SEncoder encoder; int32_t code; @@ -175,7 +215,7 @@ int32_t streamSendCheckRsp(const SStreamMeta* pMeta, const SStreamTaskCheckReq* } void* buf = rpcMallocCont(sizeof(SMsgHead) + len); - ((SMsgHead*)buf)->vgId = htonl(pReq->upstreamNodeId); + ((SMsgHead*)buf)->vgId = htonl(vgId); void* abuf = POINTER_SHIFT(buf, sizeof(SMsgHead)); tEncoderInit(&encoder, (uint8_t*)abuf, len); @@ -420,6 +460,9 @@ void doSendCheckMsg(SStreamTask* pTask, SDownstreamStatusInfo* p) { .stage = pTask->pMeta->stage, }; + // update the reqId for the new check msg + p->reqId = tGenIdPI64(); + STaskOutputInfo* pOutputInfo = &pTask->outputInfo; if (pOutputInfo->type == TASK_OUTPUT__FIXED_DISPATCH) { STaskDispatcherFixed* pDispatch = &pOutputInfo->fixedDispatcher; diff --git a/source/libs/stream/src/streamTaskSm.c b/source/libs/stream/src/streamTaskSm.c index 6f6d86b10800..a875b52e26ab 100644 --- a/source/libs/stream/src/streamTaskSm.c +++ b/source/libs/stream/src/streamTaskSm.c @@ -84,7 +84,7 @@ int32_t streamTaskInitStatus(SStreamTask* pTask) { stDebug("s-task:%s start init, and check downstream tasks, set the init ts:%" PRId64, pTask->id.idStr, pTask->execInfo.checkTs); - streamTaskCheckDownstream(pTask); + streamTaskSendCheckMsg(pTask); return 0; } From eb8970eaf247507bd839d4a3dc9246bcf1ed92db Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 7 May 2024 16:33:08 +0800 Subject: [PATCH 08/30] fix(stream): always return success if repeatly recv checkpoint source msg. --- include/libs/stream/tstream.h | 5 ++-- include/libs/transport/trpc.h | 2 -- source/dnode/vnode/src/tq/tq.c | 24 ++++++++++++------- source/libs/stream/src/streamBackendRocksdb.c | 7 +++--- source/libs/stream/src/streamDispatch.c | 16 +++++++------ 5 files changed, 29 insertions(+), 25 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 02fa31ef0736..1b4636120b9a 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -923,10 +923,9 @@ int32_t streamTaskBuildCheckpoint(SStreamTask* pTask); void streamTaskClearCheckInfo(SStreamTask* pTask, bool clearChkpReadyMsg); int32_t streamAlignTransferState(SStreamTask* pTask); int32_t streamBuildAndSendDropTaskMsg(SMsgCb* pMsgCb, int32_t vgId, SStreamTaskId* pTaskId, int64_t resetRelHalt); -int32_t streamAddCheckpointSourceRspMsg(SStreamCheckpointSourceReq* pReq, SRpcHandleInfo* pRpcInfo, SStreamTask* pTask, - int8_t isSucceed); +int32_t streamAddCheckpointSourceRspMsg(SStreamCheckpointSourceReq* pReq, SRpcHandleInfo* pRpcInfo, SStreamTask* pTask); int32_t buildCheckpointSourceRsp(SStreamCheckpointSourceReq* pReq, SRpcHandleInfo* pRpcInfo, SRpcMsg* pMsg, - int8_t isSucceed); + int32_t setCode); SStreamTaskSM* streamCreateStateMachine(SStreamTask* pTask); void* streamDestroyStateMachine(SStreamTaskSM* pSM); diff --git a/include/libs/transport/trpc.h b/include/libs/transport/trpc.h index 95f70c8ff3de..c3b5beb3e3b5 100644 --- a/include/libs/transport/trpc.h +++ b/include/libs/transport/trpc.h @@ -37,7 +37,6 @@ typedef struct { int64_t applyIndex; uint64_t applyTerm; char user[TSDB_USER_LEN]; - } SRpcConnInfo; typedef struct SRpcHandleInfo { @@ -63,7 +62,6 @@ typedef struct SRpcHandleInfo { SRpcConnInfo conn; int8_t forbiddenIp; int8_t notFreeAhandle; - } SRpcHandleInfo; typedef struct SRpcMsg { diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index bb9bbfbf7589..8acee796fd3e 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1064,6 +1064,7 @@ int32_t tqStreamProgressRetrieveReq(STQ *pTq, SRpcMsg *pMsg) { return code; } +// no matter what kinds of error happened, make sure the mnode will receive the success execution code. int32_t tqProcessTaskCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg* pRsp) { int32_t vgId = TD_VID(pTq->pVnode); SStreamMeta* pMeta = pTq->pStreamMeta; @@ -1081,8 +1082,9 @@ int32_t tqProcessTaskCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg* pRsp) code = TSDB_CODE_MSG_DECODE_ERROR; tDecoderClear(&decoder); tqError("vgId:%d failed to decode checkpoint-source msg, code:%s", vgId, tstrerror(code)); + SRpcMsg rsp = {0}; - buildCheckpointSourceRsp(&req, &pMsg->info, &rsp, 0); + buildCheckpointSourceRsp(&req, &pMsg->info, &rsp, TSDB_CODE_SUCCESS); tmsgSendRsp(&rsp); // error occurs return code; } @@ -1091,7 +1093,7 @@ int32_t tqProcessTaskCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg* pRsp) if (!vnodeIsRoleLeader(pTq->pVnode)) { tqDebug("vgId:%d not leader, ignore checkpoint-source msg, s-task:0x%x", vgId, req.taskId); SRpcMsg rsp = {0}; - buildCheckpointSourceRsp(&req, &pMsg->info, &rsp, 0); + buildCheckpointSourceRsp(&req, &pMsg->info, &rsp, TSDB_CODE_SUCCESS); tmsgSendRsp(&rsp); // error occurs return TSDB_CODE_SUCCESS; } @@ -1101,7 +1103,7 @@ int32_t tqProcessTaskCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg* pRsp) ", transId:%d s-task:0x%x ignore it", vgId, req.checkpointId, req.transId, req.taskId); SRpcMsg rsp = {0}; - buildCheckpointSourceRsp(&req, &pMsg->info, &rsp, 0); + buildCheckpointSourceRsp(&req, &pMsg->info, &rsp, TSDB_CODE_SUCCESS); tmsgSendRsp(&rsp); // error occurs return TSDB_CODE_SUCCESS; } @@ -1112,7 +1114,7 @@ int32_t tqProcessTaskCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg* pRsp) " transId:%d it may have been destroyed", vgId, req.taskId, req.checkpointId, req.transId); SRpcMsg rsp = {0}; - buildCheckpointSourceRsp(&req, &pMsg->info, &rsp, 0); + buildCheckpointSourceRsp(&req, &pMsg->info, &rsp, TSDB_CODE_SUCCESS); tmsgSendRsp(&rsp); // error occurs return TSDB_CODE_SUCCESS; } @@ -1128,7 +1130,7 @@ int32_t tqProcessTaskCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg* pRsp) streamMetaReleaseTask(pMeta, pTask); SRpcMsg rsp = {0}; - buildCheckpointSourceRsp(&req, &pMsg->info, &rsp, 0); + buildCheckpointSourceRsp(&req, &pMsg->info, &rsp, TSDB_CODE_SUCCESS); tmsgSendRsp(&rsp); // error occurs return TSDB_CODE_SUCCESS; } @@ -1146,7 +1148,7 @@ int32_t tqProcessTaskCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg* pRsp) streamMetaReleaseTask(pMeta, pTask); SRpcMsg rsp = {0}; - buildCheckpointSourceRsp(&req, &pMsg->info, &rsp, 0); + buildCheckpointSourceRsp(&req, &pMsg->info, &rsp, TSDB_CODE_SUCCESS); tmsgSendRsp(&rsp); // error occurs return TSDB_CODE_SUCCESS; @@ -1171,11 +1173,15 @@ int32_t tqProcessTaskCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg* pRsp) } else { // checkpoint already finished, and not in checkpoint status if (req.checkpointId <= pTask->chkInfo.checkpointId) { tqWarn("s-task:%s repeatly recv checkpoint-source msg checkpointId:%" PRId64 - " transId:%d already handled, ignore and discard", pTask->id.idStr, req.checkpointId, req.transId); + " transId:%d already handled, return success", pTask->id.idStr, req.checkpointId, req.transId); taosThreadMutexUnlock(&pTask->lock); streamMetaReleaseTask(pMeta, pTask); + SRpcMsg rsp = {0}; + buildCheckpointSourceRsp(&req, &pMsg->info, &rsp, TSDB_CODE_SUCCESS); + tmsgSendRsp(&rsp); // error occurs + return TSDB_CODE_SUCCESS; } } @@ -1193,10 +1199,10 @@ int32_t tqProcessTaskCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg* pRsp) pTask->id.idStr, vgId, pTask->info.taskLevel, req.checkpointId, req.transId, pPrevStatus); } - code = streamAddCheckpointSourceRspMsg(&req, &pMsg->info, pTask, 1); + code = streamAddCheckpointSourceRspMsg(&req, &pMsg->info, pTask); if (code != TSDB_CODE_SUCCESS) { SRpcMsg rsp = {0}; - buildCheckpointSourceRsp(&req, &pMsg->info, &rsp, 0); + buildCheckpointSourceRsp(&req, &pMsg->info, &rsp, TSDB_CODE_SUCCESS); tmsgSendRsp(&rsp); // error occurs return code; } diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index feafebdc46e8..aa268e9d725f 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -1580,12 +1580,11 @@ int32_t valueEncode(void* value, int32_t vlen, int64_t ttl, char** dest) { key.len = compressedSize; value = dst; } - stDebug("vlen: raw size: %d, compressed size: %d", vlen, compressedSize); } if (*dest == NULL) { - char* p = taosMemoryCalloc( - 1, sizeof(key.unixTimestamp) + sizeof(key.len) + sizeof(key.rawLen) + sizeof(key.compress) + key.len); + size_t size = sizeof(key.unixTimestamp) + sizeof(key.len) + sizeof(key.rawLen) + sizeof(key.compress) + key.len; + char* p = taosMemoryCalloc(1, size); char* buf = p; len += taosEncodeFixedI64((void**)&buf, key.unixTimestamp); len += taosEncodeFixedI32((void**)&buf, key.len); @@ -1601,8 +1600,8 @@ int32_t valueEncode(void* value, int32_t vlen, int64_t ttl, char** dest) { len += taosEncodeFixedI8((void**)&buf, key.compress); len += taosEncodeBinary((void**)&buf, (char*)value, key.len); } - taosMemoryFree(dst); + taosMemoryFree(dst); return len; } diff --git a/source/libs/stream/src/streamDispatch.c b/source/libs/stream/src/streamDispatch.c index 56eb7b196ef1..4b9831a16d56 100644 --- a/source/libs/stream/src/streamDispatch.c +++ b/source/libs/stream/src/streamDispatch.c @@ -833,7 +833,7 @@ int32_t doSendDispatchMsg(SStreamTask* pTask, const SStreamDispatchReq* pReq, in } int32_t buildCheckpointSourceRsp(SStreamCheckpointSourceReq* pReq, SRpcHandleInfo* pRpcInfo, SRpcMsg* pMsg, - int8_t isSucceed) { + int32_t setCode) { int32_t len = 0; int32_t code = 0; SEncoder encoder; @@ -845,7 +845,7 @@ int32_t buildCheckpointSourceRsp(SStreamCheckpointSourceReq* pReq, SRpcHandleInf .streamId = pReq->streamId, .expireTime = pReq->expireTime, .mnodeId = pReq->mnodeId, - .success = isSucceed, + .success = (setCode == TSDB_CODE_SUCCESS) ? 1 : 0, }; tEncodeSize(tEncodeStreamCheckpointSourceRsp, &rsp, len, code); @@ -866,22 +866,24 @@ int32_t buildCheckpointSourceRsp(SStreamCheckpointSourceReq* pReq, SRpcHandleInf tEncoderClear(&encoder); initRpcMsg(pMsg, 0, pBuf, sizeof(SMsgHead) + len); + + pMsg->code = setCode; pMsg->info = *pRpcInfo; return 0; } -int32_t streamAddCheckpointSourceRspMsg(SStreamCheckpointSourceReq* pReq, SRpcHandleInfo* pRpcInfo, SStreamTask* pTask, - int8_t isSucceed) { +int32_t streamAddCheckpointSourceRspMsg(SStreamCheckpointSourceReq* pReq, SRpcHandleInfo* pRpcInfo, SStreamTask* pTask) { SStreamChkptReadyInfo info = {0}; - buildCheckpointSourceRsp(pReq, pRpcInfo, &info.msg, isSucceed); + buildCheckpointSourceRsp(pReq, pRpcInfo, &info.msg, TSDB_CODE_SUCCESS); if (pTask->pReadyMsgList == NULL) { pTask->pReadyMsgList = taosArrayInit(4, sizeof(SStreamChkptReadyInfo)); } taosArrayPush(pTask->pReadyMsgList, &info); - stDebug("s-task:%s add checkpoint source rsp msg, total:%d", pTask->id.idStr, - (int32_t)taosArrayGetSize(pTask->pReadyMsgList)); + + int32_t size = taosArrayGetSize(pTask->pReadyMsgList); + stDebug("s-task:%s add checkpoint source rsp msg, total:%d", pTask->id.idStr, size); return TSDB_CODE_SUCCESS; } From ec41a4590288a296c56dd8a6af26c1b37645fe40 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 7 May 2024 23:43:21 +0800 Subject: [PATCH 09/30] refactor: do some internal refactor. --- include/libs/stream/streammsg.h | 175 ++++++++++++++ include/libs/stream/tstream.h | 263 ++++----------------- source/dnode/mnode/impl/src/mndStreamHb.c | 4 +- source/dnode/vnode/src/tq/tq.c | 16 +- source/dnode/vnode/src/tqCommon/tqCommon.c | 16 +- source/libs/stream/inc/streamInt.h | 4 + source/libs/stream/src/streamCheckStatus.c | 4 +- source/libs/stream/src/streamDispatch.c | 14 +- source/libs/stream/src/streamMeta.c | 4 +- 9 files changed, 255 insertions(+), 245 deletions(-) create mode 100644 include/libs/stream/streammsg.h diff --git a/include/libs/stream/streammsg.h b/include/libs/stream/streammsg.h new file mode 100644 index 000000000000..d39a9b0cbaea --- /dev/null +++ b/include/libs/stream/streammsg.h @@ -0,0 +1,175 @@ +/* + * 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_STREAMMSG_H +#define TDENGINE_STREAMMSG_H + +typedef struct SStreamChildEpInfo { + int32_t nodeId; + int32_t childId; + int32_t taskId; + SEpSet epSet; + bool dataAllowed; // denote if the data from this upstream task is allowed to put into inputQ, not serialize it + int64_t stage; // upstream task stage value, to denote if the upstream node has restart/replica changed/transfer +} SStreamChildEpInfo; + +int32_t tEncodeStreamEpInfo(SEncoder* pEncoder, const SStreamChildEpInfo* pInfo); +int32_t tDecodeStreamEpInfo(SDecoder* pDecoder, SStreamChildEpInfo* pInfo); + +// mndTrigger: denote if this checkpoint is triggered by mnode or as requested from tasks when transfer-state finished +typedef struct { + int64_t streamId; + int64_t checkpointId; + int32_t taskId; + int32_t nodeId; + SEpSet mgmtEps; + int32_t mnodeId; + int32_t transId; + int8_t mndTrigger; + int64_t expireTime; +} SStreamCheckpointSourceReq; + +typedef struct { + int64_t streamId; + int64_t checkpointId; + int32_t taskId; + int32_t nodeId; + int32_t mnodeId; + int64_t expireTime; + int8_t success; +} SStreamCheckpointSourceRsp; + +int32_t tEncodeStreamCheckpointSourceReq(SEncoder* pEncoder, const SStreamCheckpointSourceReq* pReq); +int32_t tDecodeStreamCheckpointSourceReq(SDecoder* pDecoder, SStreamCheckpointSourceReq* pReq); + +int32_t tEncodeStreamCheckpointSourceRsp(SEncoder* pEncoder, const SStreamCheckpointSourceRsp* pRsp); + +typedef struct SStreamTaskNodeUpdateMsg { + int32_t transId; // to identify the msg + int64_t streamId; + int32_t taskId; + SArray* pNodeList; // SArray +} SStreamTaskNodeUpdateMsg; + +int32_t tEncodeStreamTaskUpdateMsg(SEncoder* pEncoder, const SStreamTaskNodeUpdateMsg* pMsg); +int32_t tDecodeStreamTaskUpdateMsg(SDecoder* pDecoder, SStreamTaskNodeUpdateMsg* pMsg); + +typedef struct { + int64_t reqId; + int64_t stage; + int64_t streamId; + int32_t upstreamNodeId; + int32_t upstreamTaskId; + int32_t downstreamNodeId; + int32_t downstreamTaskId; + int32_t childId; +} SStreamTaskCheckReq; + +int32_t tEncodeStreamTaskCheckReq(SEncoder* pEncoder, const SStreamTaskCheckReq* pReq); +int32_t tDecodeStreamTaskCheckReq(SDecoder* pDecoder, SStreamTaskCheckReq* pReq); + +typedef struct { + int64_t reqId; + int64_t streamId; + int32_t upstreamNodeId; + int32_t upstreamTaskId; + int32_t downstreamNodeId; + int32_t downstreamTaskId; + int32_t childId; + int64_t oldStage; + int8_t status; +} SStreamTaskCheckRsp; + +int32_t tEncodeStreamTaskCheckRsp(SEncoder* pEncoder, const SStreamTaskCheckRsp* pRsp); +int32_t tDecodeStreamTaskCheckRsp(SDecoder* pDecoder, SStreamTaskCheckRsp* pRsp); + +typedef struct { + SMsgHead msgHead; + int64_t streamId; + int64_t checkpointId; + int32_t downstreamTaskId; + int32_t downstreamNodeId; + int32_t upstreamTaskId; + int32_t upstreamNodeId; + int32_t childId; +} SStreamCheckpointReadyMsg; + +int32_t tEncodeStreamCheckpointReadyMsg(SEncoder* pEncoder, const SStreamCheckpointReadyMsg* pRsp); +int32_t tDecodeStreamCheckpointReadyMsg(SDecoder* pDecoder, SStreamCheckpointReadyMsg* pRsp); + +struct SStreamDispatchReq { + int32_t type; + int64_t stage; // nodeId from upstream task + int64_t streamId; + int32_t taskId; + int32_t msgId; // msg id to identify if the incoming msg from the same sender + int32_t srcVgId; + int32_t upstreamTaskId; + int32_t upstreamChildId; + int32_t upstreamNodeId; + int32_t upstreamRelTaskId; + int32_t blockNum; + int64_t totalLen; + SArray* dataLen; // SArray + SArray* data; // SArray +}; + +int32_t tEncodeStreamDispatchReq(SEncoder* pEncoder, const struct SStreamDispatchReq* pReq); +int32_t tDecodeStreamDispatchReq(SDecoder* pDecoder, struct SStreamDispatchReq* pReq); +void tCleanupStreamDispatchReq(struct SStreamDispatchReq* pReq); + +struct SStreamRetrieveReq { + int64_t streamId; + int64_t reqId; + int32_t srcTaskId; + int32_t srcNodeId; + int32_t dstTaskId; + int32_t dstNodeId; + int32_t retrieveLen; + SRetrieveTableRsp* pRetrieve; +}; + +int32_t tEncodeStreamRetrieveReq(SEncoder* pEncoder, const struct SStreamRetrieveReq* pReq); +int32_t tDecodeStreamRetrieveReq(SDecoder* pDecoder, struct SStreamRetrieveReq* pReq); +void tCleanupStreamRetrieveReq(struct SStreamRetrieveReq* pReq); + +typedef struct SStreamTaskCheckpointReq { + int64_t streamId; + int32_t taskId; + int32_t nodeId; +} SStreamTaskCheckpointReq; + +int32_t tEncodeStreamTaskCheckpointReq(SEncoder* pEncoder, const SStreamTaskCheckpointReq* pReq); +int32_t tDecodeStreamTaskCheckpointReq(SDecoder* pDecoder, SStreamTaskCheckpointReq* pReq); + +typedef struct SStreamHbMsg { + int32_t vgId; + int32_t numOfTasks; + SArray* pTaskStatus; // SArray + SArray* pUpdateNodes; // SArray, needs update the epsets in stream tasks for those nodes. +} SStreamHbMsg; + +int32_t tEncodeStreamHbMsg(SEncoder* pEncoder, const SStreamHbMsg* pRsp); +int32_t tDecodeStreamHbMsg(SDecoder* pDecoder, SStreamHbMsg* pRsp); +void tCleanupStreamHbMsg(SStreamHbMsg* pMsg); + +typedef struct { + SMsgHead head; + int64_t streamId; + int32_t taskId; + int32_t reqType; +} SStreamTaskRunReq; + +#endif // TDENGINE_STREAMMSG_H diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 1b4636120b9a..27a9168a9937 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -13,8 +13,8 @@ * along with this program. If not, see . */ -#ifndef _STREAM_H_ -#define _STREAM_H_ +#ifndef TDENGINE_TSTREAM_H +#define TDENGINE_TSTREAM_H #include "os.h" #include "streamState.h" @@ -24,6 +24,7 @@ #include "tmsgcb.h" #include "tqueue.h" #include "ttimer.h" +#include "streammsg.h" #ifdef __cplusplus extern "C" { @@ -150,6 +151,8 @@ typedef enum EStreamTaskEvent { TASK_EVENT_DROPPING = 0xA, } EStreamTaskEvent; +typedef int32_t (*__state_trans_user_fn)(SStreamTask*, void* param); + typedef struct { int8_t type; } SStreamQueueItem; @@ -194,30 +197,6 @@ struct SStreamQueueNode { SStreamQueueNode* next; }; -typedef struct { - SStreamQueueNode* head; - int64_t size; -} SStreamQueueRes; - -#if 0 -bool streamQueueResEmpty(const SStreamQueueRes* pRes); -int64_t streamQueueResSize(const SStreamQueueRes* pRes); -SStreamQueueNode* streamQueueResFront(SStreamQueueRes* pRes); -SStreamQueueNode* streamQueueResPop(SStreamQueueRes* pRes); -void streamQueueResClear(SStreamQueueRes* pRes); -SStreamQueueRes streamQueueBuildRes(SStreamQueueNode* pNode); -#endif - -typedef struct { - SStreamQueueNode* pHead; -} SStreamQueue1; - -#if 0 -bool streamQueueHasTask(const SStreamQueue1* pQueue); -int32_t streamQueuePush(SStreamQueue1* pQueue, SStreamQueueItem* pItem); -SStreamQueueRes streamQueueGetRes(SStreamQueue1* pQueue); -#endif - SStreamDataSubmit* streamDataSubmitNew(SPackedData* pData, int32_t type); void streamDataSubmitDestroy(SStreamDataSubmit* pDataSubmit); @@ -283,15 +262,6 @@ typedef struct { int8_t reserved; } STaskSinkFetch; -typedef struct SStreamChildEpInfo { - int32_t nodeId; - int32_t childId; - int32_t taskId; - SEpSet epSet; - bool dataAllowed; // denote if the data from this upstream task is allowed to put into inputQ, not serialize it - int64_t stage; // upstream task stage value, to denote if the upstream node has restart/replica changed/transfer -} SStreamChildEpInfo; - typedef struct STaskId { int64_t streamId; int64_t taskId; @@ -330,7 +300,6 @@ typedef struct SStreamStatus { int64_t lastExecTs; // last exec time stamp int32_t inScanHistorySentinel; bool appendTranstateBlock; // has append the transfer state data block already - bool supplementaryWalscan; // complete the supplementary wal scan or not } SStreamStatus; typedef struct SDataRange { @@ -349,6 +318,7 @@ typedef struct SSTaskBasicInfo { int64_t triggerParam; // in msec } SSTaskBasicInfo; +typedef struct SStreamRetrieveReq SStreamRetrieveReq; typedef struct SStreamDispatchReq SStreamDispatchReq; typedef struct STokenBucket STokenBucket; typedef struct SMetaHbInfo SMetaHbInfo; @@ -560,9 +530,6 @@ typedef struct STaskUpdateEntry { int32_t transId; } STaskUpdateEntry; -int32_t tEncodeStreamEpInfo(SEncoder* pEncoder, const SStreamChildEpInfo* pInfo); -int32_t tDecodeStreamEpInfo(SDecoder* pDecoder, SStreamChildEpInfo* pInfo); - SStreamTask* tNewStreamTask(int64_t streamId, int8_t taskLevel, SEpSet* pEpset, bool fillHistory, int64_t triggerParam, SArray* pTaskList, bool hasFillhistory, int8_t subtableWithoutMd5); int32_t tEncodeStreamTask(SEncoder* pEncoder, const SStreamTask* pTask); @@ -573,35 +540,12 @@ int32_t streamTaskInit(SStreamTask* pTask, SStreamMeta* pMeta, SMsgCb* pMsg int32_t tDecodeStreamTaskChkInfo(SDecoder* pDecoder, SCheckpointInfo* pChkpInfo); int32_t tDecodeStreamTaskId(SDecoder* pDecoder, STaskId* pTaskId); +// stream task queue related API int32_t streamTaskPutDataIntoInputQ(SStreamTask* pTask, SStreamQueueItem* pItem); int32_t streamTaskPutDataIntoOutputQ(SStreamTask* pTask, SStreamDataBlock* pBlock); int32_t streamTaskPutTranstateIntoInputQ(SStreamTask* pTask); bool streamQueueIsFull(const SStreamQueue* pQueue); -typedef struct { - SMsgHead head; - int64_t streamId; - int32_t taskId; - int32_t reqType; -} SStreamTaskRunReq; - -struct SStreamDispatchReq { - int32_t type; - int64_t stage; // nodeId from upstream task - int64_t streamId; - int32_t taskId; - int32_t msgId; // msg id to identify if the incoming msg from the same sender - int32_t srcVgId; - int32_t upstreamTaskId; - int32_t upstreamChildId; - int32_t upstreamNodeId; - int32_t upstreamRelTaskId; - int32_t blockNum; - int64_t totalLen; - SArray* dataLen; // SArray - SArray* data; // SArray -}; - typedef struct { int64_t streamId; int32_t upstreamNodeId; @@ -613,17 +557,6 @@ typedef struct { int64_t stage; } SStreamDispatchRsp; -typedef struct { - int64_t streamId; - int64_t reqId; - int32_t srcTaskId; - int32_t srcNodeId; - int32_t dstTaskId; - int32_t dstNodeId; - int32_t retrieveLen; - SRetrieveTableRsp* pRetrieve; -} SStreamRetrieveReq; - typedef struct { int64_t streamId; int32_t childId; @@ -631,29 +564,6 @@ typedef struct { int32_t rspToTaskId; } SStreamRetrieveRsp; -typedef struct { - int64_t reqId; - int64_t stage; - int64_t streamId; - int32_t upstreamNodeId; - int32_t upstreamTaskId; - int32_t downstreamNodeId; - int32_t downstreamTaskId; - int32_t childId; -} SStreamTaskCheckReq; - -typedef struct { - int64_t reqId; - int64_t streamId; - int32_t upstreamNodeId; - int32_t upstreamTaskId; - int32_t downstreamNodeId; - int32_t downstreamTaskId; - int32_t childId; - int64_t oldStage; - int8_t status; -} SStreamTaskCheckRsp; - typedef struct { SMsgHead msgHead; int64_t streamId; @@ -661,48 +571,6 @@ typedef struct { int8_t igUntreated; } SStreamScanHistoryReq; -// mndTrigger: denote if this checkpoint is triggered by mnode or as requested from tasks when transfer-state finished -typedef struct { - int64_t streamId; - int64_t checkpointId; - int32_t taskId; - int32_t nodeId; - SEpSet mgmtEps; - int32_t mnodeId; - int32_t transId; - int8_t mndTrigger; - int64_t expireTime; -} SStreamCheckpointSourceReq; - -typedef struct { - int64_t streamId; - int64_t checkpointId; - int32_t taskId; - int32_t nodeId; - int32_t mnodeId; - int64_t expireTime; - int8_t success; -} SStreamCheckpointSourceRsp; - -int32_t tEncodeStreamCheckpointSourceReq(SEncoder* pEncoder, const SStreamCheckpointSourceReq* pReq); -int32_t tDecodeStreamCheckpointSourceReq(SDecoder* pDecoder, SStreamCheckpointSourceReq* pReq); - -int32_t tEncodeStreamCheckpointSourceRsp(SEncoder* pEncoder, const SStreamCheckpointSourceRsp* pRsp); - -typedef struct { - SMsgHead msgHead; - int64_t streamId; - int64_t checkpointId; - int32_t downstreamTaskId; - int32_t downstreamNodeId; - int32_t upstreamTaskId; - int32_t upstreamNodeId; - int32_t childId; -} SStreamCheckpointReadyMsg; - -int32_t tEncodeStreamCheckpointReadyMsg(SEncoder* pEncoder, const SStreamCheckpointReadyMsg* pRsp); -int32_t tDecodeStreamCheckpointReadyMsg(SDecoder* pDecoder, SStreamCheckpointReadyMsg* pRsp); - typedef struct STaskCkptInfo { int64_t latestId; // saved checkpoint id int64_t latestVer; // saved checkpoint ver @@ -733,77 +601,26 @@ typedef struct STaskStatusEntry { STaskCkptInfo checkpointInfo; } STaskStatusEntry; -typedef struct SStreamHbMsg { - int32_t vgId; - int32_t numOfTasks; - SArray* pTaskStatus; // SArray - SArray* pUpdateNodes; // SArray, needs update the epsets in stream tasks for those nodes. -} SStreamHbMsg; - -int32_t tEncodeStreamHbMsg(SEncoder* pEncoder, const SStreamHbMsg* pRsp); -int32_t tDecodeStreamHbMsg(SDecoder* pDecoder, SStreamHbMsg* pRsp); -void streamMetaClearHbMsg(SStreamHbMsg* pMsg); - typedef struct SNodeUpdateInfo { int32_t nodeId; SEpSet prevEp; SEpSet newEp; } SNodeUpdateInfo; -typedef struct SStreamTaskNodeUpdateMsg { - int32_t transId; // to identify the msg - int64_t streamId; - int32_t taskId; - SArray* pNodeList; // SArray -} SStreamTaskNodeUpdateMsg; - -int32_t tEncodeStreamTaskUpdateMsg(SEncoder* pEncoder, const SStreamTaskNodeUpdateMsg* pMsg); -int32_t tDecodeStreamTaskUpdateMsg(SDecoder* pDecoder, SStreamTaskNodeUpdateMsg* pMsg); - typedef struct SStreamTaskState { ETaskStatus state; char* name; } SStreamTaskState; -int32_t tEncodeStreamTaskCheckReq(SEncoder* pEncoder, const SStreamTaskCheckReq* pReq); -int32_t tDecodeStreamTaskCheckReq(SDecoder* pDecoder, SStreamTaskCheckReq* pReq); - -int32_t tEncodeStreamTaskCheckRsp(SEncoder* pEncoder, const SStreamTaskCheckRsp* pRsp); -int32_t tDecodeStreamTaskCheckRsp(SDecoder* pDecoder, SStreamTaskCheckRsp* pRsp); - -int32_t tEncodeStreamDispatchReq(SEncoder* pEncoder, const SStreamDispatchReq* pReq); -int32_t tDecodeStreamDispatchReq(SDecoder* pDecoder, SStreamDispatchReq* pReq); -void tDeleteStreamDispatchReq(SStreamDispatchReq* pReq); - -int32_t tEncodeStreamRetrieveReq(SEncoder* pEncoder, const SStreamRetrieveReq* pReq); -int32_t tDecodeStreamRetrieveReq(SDecoder* pDecoder, SStreamRetrieveReq* pReq); -void tDeleteStreamRetrieveReq(SStreamRetrieveReq* pReq); - -typedef struct SStreamTaskCheckpointReq { - int64_t streamId; - int32_t taskId; - int32_t nodeId; -} SStreamTaskCheckpointReq; - -int32_t tEncodeStreamTaskCheckpointReq(SEncoder* pEncoder, const SStreamTaskCheckpointReq* pReq); -int32_t tDecodeStreamTaskCheckpointReq(SDecoder* pDecoder, SStreamTaskCheckpointReq* pReq); - int32_t streamSetupScheduleTrigger(SStreamTask* pTask); +// dispatch related int32_t streamProcessDispatchMsg(SStreamTask* pTask, SStreamDispatchReq* pReq, SRpcMsg* pMsg); int32_t streamProcessDispatchRsp(SStreamTask* pTask, SStreamDispatchRsp* pRsp, int32_t code); -int32_t streamProcessRetrieveReq(SStreamTask* pTask, SStreamRetrieveReq* pReq); SStreamChildEpInfo* streamTaskGetUpstreamTaskEpInfo(SStreamTask* pTask, int32_t taskId); void streamTaskInputFail(SStreamTask* pTask); -int32_t streamExecTask(SStreamTask* pTask); -int32_t streamResumeTask(SStreamTask* pTask); -int32_t streamTrySchedExec(SStreamTask* pTask); -int32_t streamTaskSchedTask(SMsgCb* pMsgCb, int32_t vgId, int64_t streamId, int32_t taskId, int32_t execType); -int32_t streamTaskResumeInFuture(SStreamTask* pTask); -void streamTaskClearSchedIdleInfo(SStreamTask* pTask); -void streamTaskSetIdleInfo(SStreamTask* pTask, int32_t idleTime); bool streamTaskShouldStop(const SStreamTask* pStatus); bool streamTaskShouldPause(const SStreamTask* pStatus); @@ -817,29 +634,38 @@ void streamTaskResetStatus(SStreamTask* pTask); void streamTaskSetStatusReady(SStreamTask* pTask); ETaskStatus streamTaskGetPrevStatus(const SStreamTask* pTask); -void initRpcMsg(SRpcMsg* pMsg, int32_t msgType, void* pCont, int32_t contLen); - -int32_t streamTaskCheckStatus(SStreamTask* pTask, int32_t upstreamId, int32_t vgId, int64_t stage, int64_t* oldStage); bool streamTaskUpdateEpsetInfo(SStreamTask* pTask, SArray* pNodeList); void streamTaskResetUpstreamStageInfo(SStreamTask* pTask); + +// stream task sched bool streamTaskIsAllUpstreamClosed(SStreamTask* pTask); bool streamTaskSetSchedStatusWait(SStreamTask* pTask); int8_t streamTaskSetSchedStatusActive(SStreamTask* pTask); int8_t streamTaskSetSchedStatusInactive(SStreamTask* pTask); int32_t streamTaskClearHTaskAttr(SStreamTask* pTask, int32_t clearRelHalt); -int32_t streamTaskHandleEvent(SStreamTaskSM* pSM, EStreamTaskEvent event); - -typedef int32_t (*__state_trans_user_fn)(SStreamTask*, void* param); -int32_t streamTaskHandleEventAsync(SStreamTaskSM* pSM, EStreamTaskEvent event, __state_trans_user_fn callbackFn, void* param); -int32_t streamTaskOnHandleEventSuccess(SStreamTaskSM* pSM, EStreamTaskEvent event, __state_trans_user_fn callbackFn, void* param); -int32_t streamTaskRestoreStatus(SStreamTask* pTask); +int32_t streamExecTask(SStreamTask* pTask); +int32_t streamResumeTask(SStreamTask* pTask); +int32_t streamTrySchedExec(SStreamTask* pTask); +int32_t streamTaskSchedTask(SMsgCb* pMsgCb, int32_t vgId, int64_t streamId, int32_t taskId, int32_t execType); +int32_t streamTaskResumeInFuture(SStreamTask* pTask); +void streamTaskClearSchedIdleInfo(SStreamTask* pTask); +void streamTaskSetIdleInfo(SStreamTask* pTask, int32_t idleTime); +// check downstream status +int32_t streamTaskCheckStatus(SStreamTask* pTask, int32_t upstreamId, int32_t vgId, int64_t stage, int64_t* oldStage); void streamTaskSendCheckMsg(SStreamTask* pTask); void streamTaskProcessCheckMsg(SStreamMeta* pMeta, SStreamTaskCheckReq* pReq, SStreamTaskCheckRsp* pRsp); -int32_t streamSendCheckRsp(const SStreamMeta* pMeta, int32_t vgId, SStreamTaskCheckRsp* pRsp, SRpcHandleInfo* pRpcInfo, - int32_t taskId); -int32_t streamProcessCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* pRsp); +int32_t streamTaskSendCheckRsp(const SStreamMeta* pMeta, int32_t vgId, SStreamTaskCheckRsp* pRsp, + SRpcHandleInfo* pRpcInfo, int32_t taskId); +int32_t streamTaskProcessCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* pRsp); + +// check downstream status +int32_t streamTaskStartMonitorCheckRsp(SStreamTask* pTask); +int32_t streamTaskStopMonitorCheckRsp(STaskCheckInfo* pInfo, const char* id); +void streamTaskCleanupCheckInfo(STaskCheckInfo* pInfo); + +// fill-history task int32_t streamLaunchFillHistoryTask(SStreamTask* pTask); int32_t streamStartScanHistoryAsync(SStreamTask* pTask, int8_t igUntreated); int32_t streamExecScanHistoryInFuture(SStreamTask* pTask, int32_t idleDuration); @@ -859,11 +685,6 @@ void streamTaskCloseUpstreamInput(SStreamTask* pTask, int32_t taskId); void streamTaskOpenAllUpstreamInput(SStreamTask* pTask); int32_t streamTaskSetDb(SStreamMeta* pMeta, void* pTask, char* key); bool streamTaskIsSinkTask(const SStreamTask* pTask); -int32_t streamTaskSendCheckpointReq(SStreamTask* pTask); - -int32_t streamTaskStartMonitorCheckRsp(SStreamTask* pTask); -int32_t streamTaskStopMonitorCheckRsp(STaskCheckInfo* pInfo, const char* id); -void streamTaskCleanupCheckInfo(STaskCheckInfo* pInfo); void streamTaskStatusInit(STaskStatusEntry* pEntry, const SStreamTask* pTask); void streamTaskStatusCopy(STaskStatusEntry* pDst, const STaskStatusEntry* pSrc); @@ -899,9 +720,8 @@ int32_t streamMetaAddTaskLaunchResult(SStreamMeta* pMeta, int64_t streamId, int64_t endTs, bool ready); int32_t streamMetaResetTaskStatus(SStreamMeta* pMeta); int32_t streamMetaAddFailedTask(SStreamMeta* pMeta, int64_t streamId, int32_t taskId); -void streamMetaAddIntoUpdateTaskList(SStreamMeta* pMeta, SStreamTask* pTask, SStreamTask* pHTask, int32_t transId, - int64_t startTs); - +void streamMetaAddIntoUpdateTaskList(SStreamMeta* pMeta, SStreamTask* pTask, SStreamTask* pHTask, int32_t transId, + int64_t startTs); void streamMetaRLock(SStreamMeta* pMeta); void streamMetaRUnLock(SStreamMeta* pMeta); void streamMetaWLock(SStreamMeta* pMeta); @@ -914,6 +734,8 @@ int32_t streamMetaStartAllTasks(SStreamMeta* pMeta); int32_t streamMetaStopAllTasks(SStreamMeta* pMeta); int32_t streamMetaStartOneTask(SStreamMeta* pMeta, int64_t streamId, int32_t taskId); bool streamMetaAllTasksReady(const SStreamMeta* pMeta); + +// timer tmr_h streamTimerGetInstance(); // checkpoint @@ -924,17 +746,26 @@ void streamTaskClearCheckInfo(SStreamTask* pTask, bool clearChkpReadyMsg); int32_t streamAlignTransferState(SStreamTask* pTask); int32_t streamBuildAndSendDropTaskMsg(SMsgCb* pMsgCb, int32_t vgId, SStreamTaskId* pTaskId, int64_t resetRelHalt); int32_t streamAddCheckpointSourceRspMsg(SStreamCheckpointSourceReq* pReq, SRpcHandleInfo* pRpcInfo, SStreamTask* pTask); -int32_t buildCheckpointSourceRsp(SStreamCheckpointSourceReq* pReq, SRpcHandleInfo* pRpcInfo, SRpcMsg* pMsg, - int32_t setCode); +int32_t streamTaskBuildCheckpointSourceRsp(SStreamCheckpointSourceReq* pReq, SRpcHandleInfo* pRpcInfo, SRpcMsg* pMsg, + int32_t setCode); +// stream task state machine, and event handling SStreamTaskSM* streamCreateStateMachine(SStreamTask* pTask); void* streamDestroyStateMachine(SStreamTaskSM* pSM); +int32_t streamTaskHandleEvent(SStreamTaskSM* pSM, EStreamTaskEvent event); +int32_t streamTaskHandleEventAsync(SStreamTaskSM* pSM, EStreamTaskEvent event, __state_trans_user_fn callbackFn, + void* param); +int32_t streamTaskOnHandleEventSuccess(SStreamTaskSM* pSM, EStreamTaskEvent event, __state_trans_user_fn callbackFn, + void* param); +int32_t streamTaskRestoreStatus(SStreamTask* pTask); -int32_t broadcastRetrieveMsg(SStreamTask* pTask, SStreamRetrieveReq *req); -void sendRetrieveRsp(SStreamRetrieveReq *pReq, SRpcMsg* pRsp); +// stream task retrieve related API +int32_t streamProcessRetrieveReq(SStreamTask* pTask, SStreamRetrieveReq* pReq); +int32_t streamTaskBroadcastRetrieveReq(SStreamTask* pTask, SStreamRetrieveReq *req); +void streamTaskSendRetrieveRsp(SStreamRetrieveReq *pReq, SRpcMsg* pRsp); #ifdef __cplusplus } #endif -#endif /* ifndef _STREAM_H_ */ +#endif /* ifndef TDENGINE_TSTREAM_H */ diff --git a/source/dnode/mnode/impl/src/mndStreamHb.c b/source/dnode/mnode/impl/src/mndStreamHb.c index 6a381ad31eeb..a81a391c3d28 100644 --- a/source/dnode/mnode/impl/src/mndStreamHb.c +++ b/source/dnode/mnode/impl/src/mndStreamHb.c @@ -235,7 +235,7 @@ int32_t mndProcessStreamHb(SRpcMsg *pReq) { tDecoderInit(&decoder, pReq->pCont, pReq->contLen); if (tDecodeStreamHbMsg(&decoder, &req) < 0) { - streamMetaClearHbMsg(&req); + tCleanupStreamHbMsg(&req); tDecoderClear(&decoder); terrno = TSDB_CODE_INVALID_MSG; return -1; @@ -349,7 +349,7 @@ int32_t mndProcessStreamHb(SRpcMsg *pReq) { } taosThreadMutexUnlock(&execInfo.lock); - streamMetaClearHbMsg(&req); + tCleanupStreamHbMsg(&req); taosArrayDestroy(pFailedTasks); taosArrayDestroy(pOrphanTasks); diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 8acee796fd3e..0a1d98425a8f 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1084,7 +1084,7 @@ int32_t tqProcessTaskCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg* pRsp) tqError("vgId:%d failed to decode checkpoint-source msg, code:%s", vgId, tstrerror(code)); SRpcMsg rsp = {0}; - buildCheckpointSourceRsp(&req, &pMsg->info, &rsp, TSDB_CODE_SUCCESS); + streamTaskBuildCheckpointSourceRsp(&req, &pMsg->info, &rsp, TSDB_CODE_SUCCESS); tmsgSendRsp(&rsp); // error occurs return code; } @@ -1093,7 +1093,7 @@ int32_t tqProcessTaskCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg* pRsp) if (!vnodeIsRoleLeader(pTq->pVnode)) { tqDebug("vgId:%d not leader, ignore checkpoint-source msg, s-task:0x%x", vgId, req.taskId); SRpcMsg rsp = {0}; - buildCheckpointSourceRsp(&req, &pMsg->info, &rsp, TSDB_CODE_SUCCESS); + streamTaskBuildCheckpointSourceRsp(&req, &pMsg->info, &rsp, TSDB_CODE_SUCCESS); tmsgSendRsp(&rsp); // error occurs return TSDB_CODE_SUCCESS; } @@ -1103,7 +1103,7 @@ int32_t tqProcessTaskCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg* pRsp) ", transId:%d s-task:0x%x ignore it", vgId, req.checkpointId, req.transId, req.taskId); SRpcMsg rsp = {0}; - buildCheckpointSourceRsp(&req, &pMsg->info, &rsp, TSDB_CODE_SUCCESS); + streamTaskBuildCheckpointSourceRsp(&req, &pMsg->info, &rsp, TSDB_CODE_SUCCESS); tmsgSendRsp(&rsp); // error occurs return TSDB_CODE_SUCCESS; } @@ -1114,7 +1114,7 @@ int32_t tqProcessTaskCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg* pRsp) " transId:%d it may have been destroyed", vgId, req.taskId, req.checkpointId, req.transId); SRpcMsg rsp = {0}; - buildCheckpointSourceRsp(&req, &pMsg->info, &rsp, TSDB_CODE_SUCCESS); + streamTaskBuildCheckpointSourceRsp(&req, &pMsg->info, &rsp, TSDB_CODE_SUCCESS); tmsgSendRsp(&rsp); // error occurs return TSDB_CODE_SUCCESS; } @@ -1130,7 +1130,7 @@ int32_t tqProcessTaskCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg* pRsp) streamMetaReleaseTask(pMeta, pTask); SRpcMsg rsp = {0}; - buildCheckpointSourceRsp(&req, &pMsg->info, &rsp, TSDB_CODE_SUCCESS); + streamTaskBuildCheckpointSourceRsp(&req, &pMsg->info, &rsp, TSDB_CODE_SUCCESS); tmsgSendRsp(&rsp); // error occurs return TSDB_CODE_SUCCESS; } @@ -1148,7 +1148,7 @@ int32_t tqProcessTaskCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg* pRsp) streamMetaReleaseTask(pMeta, pTask); SRpcMsg rsp = {0}; - buildCheckpointSourceRsp(&req, &pMsg->info, &rsp, TSDB_CODE_SUCCESS); + streamTaskBuildCheckpointSourceRsp(&req, &pMsg->info, &rsp, TSDB_CODE_SUCCESS); tmsgSendRsp(&rsp); // error occurs return TSDB_CODE_SUCCESS; @@ -1179,7 +1179,7 @@ int32_t tqProcessTaskCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg* pRsp) streamMetaReleaseTask(pMeta, pTask); SRpcMsg rsp = {0}; - buildCheckpointSourceRsp(&req, &pMsg->info, &rsp, TSDB_CODE_SUCCESS); + streamTaskBuildCheckpointSourceRsp(&req, &pMsg->info, &rsp, TSDB_CODE_SUCCESS); tmsgSendRsp(&rsp); // error occurs return TSDB_CODE_SUCCESS; @@ -1202,7 +1202,7 @@ int32_t tqProcessTaskCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg* pRsp) code = streamAddCheckpointSourceRspMsg(&req, &pMsg->info, pTask); if (code != TSDB_CODE_SUCCESS) { SRpcMsg rsp = {0}; - buildCheckpointSourceRsp(&req, &pMsg->info, &rsp, TSDB_CODE_SUCCESS); + streamTaskBuildCheckpointSourceRsp(&req, &pMsg->info, &rsp, TSDB_CODE_SUCCESS); tmsgSendRsp(&rsp); // error occurs return code; } diff --git a/source/dnode/vnode/src/tqCommon/tqCommon.c b/source/dnode/vnode/src/tqCommon/tqCommon.c index 9da8d05deb07..df7ab17b7c69 100644 --- a/source/dnode/vnode/src/tqCommon/tqCommon.c +++ b/source/dnode/vnode/src/tqCommon/tqCommon.c @@ -303,7 +303,7 @@ int32_t tqStreamTaskProcessDispatchReq(SStreamMeta* pMeta, SRpcMsg* pMsg) { if (streamProcessDispatchMsg(pTask, &req, &rsp) != 0) { return -1; } - tDeleteStreamDispatchReq(&req); + tCleanupStreamDispatchReq(&req); streamMetaReleaseTask(pMeta, pTask); return 0; } else { @@ -335,7 +335,7 @@ int32_t tqStreamTaskProcessDispatchReq(SStreamMeta* pMeta, SRpcMsg* pMsg) { tqError("s-task:0x%x send dispatch error rsp, no task", req.taskId); tmsgSendRsp(&rsp); - tDeleteStreamDispatchReq(&req); + tCleanupStreamDispatchReq(&req); return 0; } @@ -379,7 +379,7 @@ int32_t tqStreamTaskProcessRetrieveReq(SStreamMeta* pMeta, SRpcMsg* pMsg) { if (pTask == NULL) { tqError("vgId:%d process retrieve req, failed to acquire task:0x%x, it may have been dropped already", pMeta->vgId, req.dstTaskId); - tDeleteStreamRetrieveReq(&req); + tCleanupStreamRetrieveReq(&req); return -1; } @@ -389,14 +389,14 @@ int32_t tqStreamTaskProcessRetrieveReq(SStreamMeta* pMeta, SRpcMsg* pMsg) { } else { req.srcNodeId = pTask->info.nodeId; req.srcTaskId = pTask->id.taskId; - code = broadcastRetrieveMsg(pTask, &req); + code = streamTaskBroadcastRetrieveReq(pTask, &req); } SRpcMsg rsp = {.info = pMsg->info, .code = 0}; - sendRetrieveRsp(&req, &rsp); + streamTaskSendRetrieveRsp(&req, &rsp); streamMetaReleaseTask(pMeta, pTask); - tDeleteStreamRetrieveReq(&req); + tCleanupStreamRetrieveReq(&req); return code; } @@ -415,7 +415,7 @@ int32_t tqStreamTaskProcessCheckReq(SStreamMeta* pMeta, SRpcMsg* pMsg) { tDecoderClear(&decoder); streamTaskProcessCheckMsg(pMeta, &req, &rsp); - return streamSendCheckRsp(pMeta, req.upstreamNodeId, &rsp, &pMsg->info, req.upstreamTaskId); + return streamTaskSendCheckRsp(pMeta, req.upstreamNodeId, &rsp, &pMsg->info, req.upstreamTaskId); } int32_t tqStreamTaskProcessCheckRsp(SStreamMeta* pMeta, SRpcMsg* pMsg, bool isLeader) { @@ -451,7 +451,7 @@ int32_t tqStreamTaskProcessCheckRsp(SStreamMeta* pMeta, SRpcMsg* pMsg, bool isLe return streamMetaAddFailedTask(pMeta, rsp.streamId, rsp.upstreamTaskId); } - code = streamProcessCheckRsp(pTask, &rsp); + code = streamTaskProcessCheckRsp(pTask, &rsp); streamMetaReleaseTask(pMeta, pTask); return code; } diff --git a/source/libs/stream/inc/streamInt.h b/source/libs/stream/inc/streamInt.h index ceb6cd9739ab..949bc2e60b3a 100644 --- a/source/libs/stream/inc/streamInt.h +++ b/source/libs/stream/inc/streamInt.h @@ -101,6 +101,8 @@ extern int32_t taskDbWrapperId; int32_t streamTimerInit(); void streamTimerCleanUp(); +void initRpcMsg(SRpcMsg* pMsg, int32_t msgType, void* pCont, int32_t contLen); + void streamRetryDispatchData(SStreamTask* pTask, int64_t waitDuration); int32_t streamDispatchStreamBlock(SStreamTask* pTask); void destroyDispatchMsg(SStreamDispatchReq* pReq, int32_t numOfVgroups); @@ -122,6 +124,8 @@ int32_t streamSendCheckMsg(SStreamTask* pTask, const SStreamTaskCheckReq* pReq, int32_t streamAddCheckpointReadyMsg(SStreamTask* pTask, int32_t srcTaskId, int32_t index, int64_t checkpointId); int32_t streamTaskSendCheckpointReadyMsg(SStreamTask* pTask); int32_t streamTaskSendCheckpointSourceRsp(SStreamTask* pTask); +int32_t streamTaskSendCheckpointReq(SStreamTask* pTask); + void streamTaskSetFailedCheckpointId(SStreamTask* pTask); int32_t streamTaskGetNumOfDownstream(const SStreamTask* pTask); int32_t streamTaskInitTokenBucket(STokenBucket* pBucket, int32_t numCap, int32_t numRate, float quotaRate, const char*); diff --git a/source/libs/stream/src/streamCheckStatus.c b/source/libs/stream/src/streamCheckStatus.c index 9c3908c83372..1401dba8206d 100644 --- a/source/libs/stream/src/streamCheckStatus.c +++ b/source/libs/stream/src/streamCheckStatus.c @@ -137,7 +137,7 @@ void streamTaskProcessCheckMsg(SStreamMeta* pMeta, SStreamTaskCheckReq* pReq, SS } -int32_t streamProcessCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* pRsp) { +int32_t streamTaskProcessCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* pRsp) { ASSERT(pTask->id.taskId == pRsp->upstreamTaskId); int64_t now = taosGetTimestampMs(); @@ -202,7 +202,7 @@ int32_t streamProcessCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* pRs return 0; } -int32_t streamSendCheckRsp(const SStreamMeta* pMeta, int32_t vgId, SStreamTaskCheckRsp* pRsp, +int32_t streamTaskSendCheckRsp(const SStreamMeta* pMeta, int32_t vgId, SStreamTaskCheckRsp* pRsp, SRpcHandleInfo* pRpcInfo, int32_t taskId) { SEncoder encoder; int32_t code; diff --git a/source/libs/stream/src/streamDispatch.c b/source/libs/stream/src/streamDispatch.c index 4b9831a16d56..2373da024b73 100644 --- a/source/libs/stream/src/streamDispatch.c +++ b/source/libs/stream/src/streamDispatch.c @@ -129,7 +129,7 @@ static int32_t tInitStreamDispatchReq(SStreamDispatchReq* pReq, const SStreamTas return TSDB_CODE_SUCCESS; } -void tDeleteStreamDispatchReq(SStreamDispatchReq* pReq) { +void tCleanupStreamDispatchReq(SStreamDispatchReq* pReq) { taosArrayDestroyP(pReq->data, taosMemoryFree); taosArrayDestroy(pReq->dataLen); } @@ -162,9 +162,9 @@ int32_t tDecodeStreamRetrieveReq(SDecoder* pDecoder, SStreamRetrieveReq* pReq) { return 0; } -void tDeleteStreamRetrieveReq(SStreamRetrieveReq* pReq) { taosMemoryFree(pReq->pRetrieve); } +void tCleanupStreamRetrieveReq(SStreamRetrieveReq* pReq) { taosMemoryFree(pReq->pRetrieve); } -void sendRetrieveRsp(SStreamRetrieveReq *pReq, SRpcMsg* pRsp){ +void streamTaskSendRetrieveRsp(SStreamRetrieveReq *pReq, SRpcMsg* pRsp){ void* buf = rpcMallocCont(sizeof(SMsgHead) + sizeof(SStreamRetrieveRsp)); ((SMsgHead*)buf)->vgId = htonl(pReq->srcNodeId); SStreamRetrieveRsp* pCont = POINTER_SHIFT(buf, sizeof(SMsgHead)); @@ -176,7 +176,7 @@ void sendRetrieveRsp(SStreamRetrieveReq *pReq, SRpcMsg* pRsp){ tmsgSendRsp(pRsp); } -int32_t broadcastRetrieveMsg(SStreamTask* pTask, SStreamRetrieveReq* req) { +int32_t streamTaskBroadcastRetrieveReq(SStreamTask* pTask, SStreamRetrieveReq* req) { int32_t code = 0; void* buf = NULL; int32_t sz = taosArrayGetSize(pTask->upstreamInfo.pList); @@ -259,7 +259,7 @@ int32_t streamBroadcastToUpTasks(SStreamTask* pTask, const SSDataBlock* pBlock) return code; } - code = broadcastRetrieveMsg(pTask, &req); + code = streamTaskBroadcastRetrieveReq(pTask, &req); taosMemoryFree(req.pRetrieve); return code; @@ -832,7 +832,7 @@ int32_t doSendDispatchMsg(SStreamTask* pTask, const SStreamDispatchReq* pReq, in return code; } -int32_t buildCheckpointSourceRsp(SStreamCheckpointSourceReq* pReq, SRpcHandleInfo* pRpcInfo, SRpcMsg* pMsg, +int32_t streamTaskBuildCheckpointSourceRsp(SStreamCheckpointSourceReq* pReq, SRpcHandleInfo* pRpcInfo, SRpcMsg* pMsg, int32_t setCode) { int32_t len = 0; int32_t code = 0; @@ -874,7 +874,7 @@ int32_t buildCheckpointSourceRsp(SStreamCheckpointSourceReq* pReq, SRpcHandleInf int32_t streamAddCheckpointSourceRspMsg(SStreamCheckpointSourceReq* pReq, SRpcHandleInfo* pRpcInfo, SStreamTask* pTask) { SStreamChkptReadyInfo info = {0}; - buildCheckpointSourceRsp(pReq, pRpcInfo, &info.msg, TSDB_CODE_SUCCESS); + streamTaskBuildCheckpointSourceRsp(pReq, pRpcInfo, &info.msg, TSDB_CODE_SUCCESS); if (pTask->pReadyMsgList == NULL) { pTask->pReadyMsgList = taosArrayInit(4, sizeof(SStreamChkptReadyInfo)); diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 7df9555e088c..399d98437b3d 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -1046,7 +1046,7 @@ static bool waitForEnoughDuration(SMetaHbInfo* pInfo) { return false; } -void streamMetaClearHbMsg(SStreamHbMsg* pMsg) { +void tCleanupStreamHbMsg(SStreamHbMsg* pMsg) { if (pMsg == NULL) { return; } @@ -1204,7 +1204,7 @@ static int32_t metaHeartbeatToMnodeImpl(SStreamMeta* pMeta) { } _end: - streamMetaClearHbMsg(&hbMsg); + tCleanupStreamHbMsg(&hbMsg); return TSDB_CODE_SUCCESS; } From 644ffafdbee3b2bede7e81589ac30fde14c8b9b9 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 8 May 2024 09:18:36 +0800 Subject: [PATCH 10/30] fix(test): fix syntax error. --- source/dnode/mnode/impl/test/stream/stream.cpp | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/source/dnode/mnode/impl/test/stream/stream.cpp b/source/dnode/mnode/impl/test/stream/stream.cpp index ae00f47ab72b..8480f204d6e4 100644 --- a/source/dnode/mnode/impl/test/stream/stream.cpp +++ b/source/dnode/mnode/impl/test/stream/stream.cpp @@ -92,7 +92,11 @@ SRpcMsg buildHbReq() { } tEncoderClear(&encoder); - initRpcMsg(&msg1, TDMT_MND_STREAM_HEARTBEAT, buf, tlen); + { + msg1.msgType = TDMT_MND_STREAM_HEARTBEAT; + msg1.pCont = buf; + msg1.contLen = tlen; + } taosArrayDestroy(msg.pTaskStatus); return msg1; From 07615ca5f9a5ede96f675fe86d15577701d0ac4e Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 8 May 2024 10:18:19 +0800 Subject: [PATCH 11/30] fix(test): fix syntax error. --- include/libs/stream/streammsg.h | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/include/libs/stream/streammsg.h b/include/libs/stream/streammsg.h index d39a9b0cbaea..a4dead747551 100644 --- a/include/libs/stream/streammsg.h +++ b/include/libs/stream/streammsg.h @@ -16,6 +16,10 @@ #ifndef TDENGINE_STREAMMSG_H #define TDENGINE_STREAMMSG_H +#ifdef __cplusplus +extern "C" { +#endif + typedef struct SStreamChildEpInfo { int32_t nodeId; int32_t childId; @@ -143,7 +147,7 @@ struct SStreamRetrieveReq { int32_t tEncodeStreamRetrieveReq(SEncoder* pEncoder, const struct SStreamRetrieveReq* pReq); int32_t tDecodeStreamRetrieveReq(SDecoder* pDecoder, struct SStreamRetrieveReq* pReq); -void tCleanupStreamRetrieveReq(struct SStreamRetrieveReq* pReq); +void tCleanupStreamRetrieveReq(struct SStreamRetrieveReq* pReq); typedef struct SStreamTaskCheckpointReq { int64_t streamId; @@ -172,4 +176,8 @@ typedef struct { int32_t reqType; } SStreamTaskRunReq; +#ifdef __cplusplus +} +#endif + #endif // TDENGINE_STREAMMSG_H From f114028f708c86bc0eb724f3282ad64963938b09 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 8 May 2024 11:21:24 +0800 Subject: [PATCH 12/30] refactor: do some internal refactor. --- include/libs/stream/tstream.h | 23 +++----------- source/libs/stream/inc/streamInt.h | 4 +++ source/libs/stream/src/streamStartHistory.c | 34 +++++++++------------ source/libs/stream/src/streamTask.c | 5 --- 4 files changed, 23 insertions(+), 43 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 27a9168a9937..fe6a7de73406 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -59,9 +59,10 @@ extern "C" { #define STREAM_EXEC_T_RESUME_TASK (-6) #define STREAM_EXEC_T_ADD_FAILED_TASK (-7) -typedef struct SStreamTask SStreamTask; -typedef struct SStreamQueue SStreamQueue; -typedef struct SStreamTaskSM SStreamTaskSM; +typedef struct SStreamTask SStreamTask; +typedef struct SStreamQueue SStreamQueue; +typedef struct SStreamTaskSM SStreamTaskSM; +typedef struct SStreamQueueItem SStreamQueueItem; #define SSTREAM_TASK_VER 4 #define SSTREAM_TASK_INCOMPATIBLE_VER 1 @@ -153,10 +154,6 @@ typedef enum EStreamTaskEvent { typedef int32_t (*__state_trans_user_fn)(SStreamTask*, void* param); -typedef struct { - int8_t type; -} SStreamQueueItem; - typedef void FTbSink(SStreamTask* pTask, void* vnode, void* data); typedef void FSmaSink(void* vnode, int64_t smaId, const SArray* data); typedef int32_t FTaskExpand(void* ahandle, SStreamTask* pTask, int64_t ver); @@ -190,13 +187,6 @@ typedef struct { SSDataBlock* pBlock; } SStreamRefDataBlock; -typedef struct SStreamQueueNode SStreamQueueNode; - -struct SStreamQueueNode { - SStreamQueueItem* item; - SStreamQueueNode* next; -}; - SStreamDataSubmit* streamDataSubmitNew(SPackedData* pData, int32_t type); void streamDataSubmitDestroy(SStreamDataSubmit* pDataSubmit); @@ -437,7 +427,7 @@ struct SStreamTask { SCheckpointInfo chkInfo; STaskExec exec; SDataRange dataRange; - SVersionRange step2Range; + SVersionRange step2Range; // version range used to scan wal, information in dataRange should not modified. SHistoryTaskInfo hTaskInfo; STaskId streamTaskId; STaskExecStatisInfo execInfo; @@ -445,14 +435,11 @@ struct SStreamTask { TdThreadMutex lock; // secure the operation of set task status and puting data into inputQ SMsgCb* pMsgCb; // msg handle SStreamState* pState; // state backend - SArray* pRspMsgList; SUpstreamInfo upstreamInfo; STaskCheckInfo taskCheckInfo; // the followings attributes don't be serialized SScanhistorySchedInfo schedHistoryInfo; - - int32_t numOfWaitingUpstream; int32_t refCnt; int32_t transferStateAlignCnt; struct SStreamMeta* pMeta; diff --git a/source/libs/stream/inc/streamInt.h b/source/libs/stream/inc/streamInt.h index 949bc2e60b3a..3ccb25a62a85 100644 --- a/source/libs/stream/inc/streamInt.h +++ b/source/libs/stream/inc/streamInt.h @@ -88,6 +88,10 @@ struct SStreamQueue { int8_t status; }; +struct SStreamQueueItem { + int8_t type; +}; + typedef enum { EXEC_CONTINUE = 0x0, EXEC_AFTER_IDLE = 0x1, diff --git a/source/libs/stream/src/streamStartHistory.c b/source/libs/stream/src/streamStartHistory.c index 98c0d95781e8..fb5b5e57d4da 100644 --- a/source/libs/stream/src/streamStartHistory.c +++ b/source/libs/stream/src/streamStartHistory.c @@ -48,9 +48,9 @@ static int32_t streamTaskSetReady(SStreamTask* pTask) { SStreamTaskState* p = streamTaskGetStatus(pTask); if ((p->state == TASK_STATUS__SCAN_HISTORY) && pTask->info.taskLevel != TASK_LEVEL__SOURCE) { - pTask->numOfWaitingUpstream = taosArrayGetSize(pTask->upstreamInfo.pList); + int32_t numOfUps = taosArrayGetSize(pTask->upstreamInfo.pList); stDebug("s-task:%s level:%d task wait for %d upstream tasks complete scan-history procedure, status:%s", - pTask->id.idStr, pTask->info.taskLevel, pTask->numOfWaitingUpstream, p->name); + pTask->id.idStr, pTask->info.taskLevel, numOfUps, p->name); } ASSERT(pTask->status.downstreamReady == 0); @@ -117,18 +117,16 @@ int32_t streamTaskStartScanHistory(SStreamTask* pTask) { int32_t level = pTask->info.taskLevel; ETaskStatus status = streamTaskGetStatus(pTask)->state; - ASSERT((pTask->status.downstreamReady == 1) && (status == TASK_STATUS__SCAN_HISTORY)); + ASSERT((pTask->status.downstreamReady == 1) && (status == TASK_STATUS__SCAN_HISTORY) && + (pTask->info.fillHistory == 1)); if (level == TASK_LEVEL__SOURCE) { return doStartScanHistoryTask(pTask); } else if (level == TASK_LEVEL__AGG) { - if (pTask->info.fillHistory) { - streamSetParamForScanHistory(pTask); - } + return streamSetParamForScanHistory(pTask); } else if (level == TASK_LEVEL__SINK) { stDebug("s-task:%s sink task do nothing to handle scan-history", pTask->id.idStr); } - return 0; } @@ -208,27 +206,21 @@ int32_t streamTaskOnNormalTaskReady(SStreamTask* pTask) { } int32_t streamTaskOnScanhistoryTaskReady(SStreamTask* pTask) { - const char* id = pTask->id.idStr; - // set the state to be ready streamTaskSetReady(pTask); streamTaskSetRangeStreamCalc(pTask); SStreamTaskState* p = streamTaskGetStatus(pTask); - ASSERT(p->state == TASK_STATUS__SCAN_HISTORY); + ASSERT((p->state == TASK_STATUS__SCAN_HISTORY) && (pTask->info.fillHistory == 1)); - if (pTask->info.fillHistory == 1) { - stDebug("s-task:%s fill-history task enters into scan-history data stage, status:%s", id, p->name); - streamTaskStartScanHistory(pTask); - } else { - stDebug("s-task:%s scan wal data, status:%s", id, p->name); - } + stDebug("s-task:%s fill-history task enters into scan-history data stage, status:%s", pTask->id.idStr, p->name); + streamTaskStartScanHistory(pTask); // NOTE: there will be an deadlock if launch fill history here. -// // start the related fill-history task, when current task is ready -// if (HAS_RELATED_FILLHISTORY_TASK(pTask)) { -// streamLaunchFillHistoryTask(pTask); -// } + // start the related fill-history task, when current task is ready + // if (HAS_RELATED_FILLHISTORY_TASK(pTask)) { + // streamLaunchFillHistoryTask(pTask); + // } return TSDB_CODE_SUCCESS; } @@ -515,6 +507,8 @@ bool streamHistoryTaskSetVerRangeStep2(SStreamTask* pTask, int64_t nextProcessVe SVersionRange* pRange = &pTask->dataRange.range; ASSERT(nextProcessVer >= pRange->maxVer); + // maxVer for fill-history task is the version, where the last timestamp is acquired. + // it's also the maximum version to scan data in tsdb. int64_t walScanStartVer = pRange->maxVer + 1; if (walScanStartVer > nextProcessVer - 1) { stDebug( diff --git a/source/libs/stream/src/streamTask.c b/source/libs/stream/src/streamTask.c index c7e34987ab58..7f5ea52f5843 100644 --- a/source/libs/stream/src/streamTask.c +++ b/source/libs/stream/src/streamTask.c @@ -456,11 +456,6 @@ void tFreeStreamTask(SStreamTask* pTask) { tSimpleHashCleanup(pTask->pNameMap); } - if (pTask->pRspMsgList != NULL) { - taosArrayDestroyEx(pTask->pRspMsgList, freeItem); - pTask->pRspMsgList = NULL; - } - pTask->status.pSM = streamDestroyStateMachine(pTask->status.pSM); streamTaskDestroyUpstreamInfo(&pTask->upstreamInfo); From 3b85bb28e534d2cf53863d2ed53c8a8ae034de69 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 8 May 2024 13:52:20 +0800 Subject: [PATCH 13/30] refactor: do some internal refactor. --- include/common/rsync.h | 10 ++--- source/common/src/cos.c | 4 +- source/common/src/rsync.c | 44 ++++++++++--------- source/libs/stream/src/streamBackendRocksdb.c | 6 ++- source/libs/stream/src/streamCheckpoint.c | 18 ++++++-- source/libs/stream/src/streamMeta.c | 3 ++ 6 files changed, 54 insertions(+), 31 deletions(-) diff --git a/include/common/rsync.h b/include/common/rsync.h index d570311694a5..f613a35f4808 100644 --- a/include/common/rsync.h +++ b/include/common/rsync.h @@ -11,11 +11,11 @@ extern "C" { #include "tarray.h" -void stopRsync(); -void startRsync(); -int uploadRsync(const char* id, const char* path); -int downloadRsync(const char* id, const char* path); -int deleteRsync(const char* id); +void stopRsync(); +void startRsync(); +int32_t uploadRsync(const char* id, const char* path); +int32_t downloadRsync(const char* id, const char* path); +int32_t deleteRsync(const char* id); #ifdef __cplusplus } diff --git a/source/common/src/cos.c b/source/common/src/cos.c index 0db6664ab95b..b249d3eff217 100644 --- a/source/common/src/cos.c +++ b/source/common/src/cos.c @@ -1208,9 +1208,11 @@ int32_t s3GetObjectToFile(const char *object_name, const char *fileName) { TdFilePtr pFile = taosOpenFile(fileName, TD_FILE_CREATE | TD_FILE_WRITE | TD_FILE_TRUNC); if (pFile == NULL) { - uError("[s3] open file error, errno:%d, fileName:%s", errno, fileName); + terrno = TAOS_SYSTEM_ERROR(errno); + uError("[s3] open file error, errno:%d, fileName:%s", terrno, fileName); return -1; } + TS3GetData cbd = {0}; cbd.file = pFile; do { diff --git a/source/common/src/rsync.c b/source/common/src/rsync.c index 7aec0077e7ab..e448aec5e06f 100644 --- a/source/common/src/rsync.c +++ b/source/common/src/rsync.c @@ -43,7 +43,7 @@ static void removeEmptyDir() { static void changeDirFromWindowsToLinux(char* from, char* to){ to[0] = '/'; to[1] = from[0]; - for(int i = 2; i < strlen(from); i++) { + for(int32_t i = 2; i < strlen(from); i++) { if (from[i] == '\\') { to[i] = '/'; } else { @@ -53,7 +53,7 @@ static void changeDirFromWindowsToLinux(char* from, char* to){ } #endif -static int generateConfigFile(char* confDir) { +static int32_t generateConfigFile(char* confDir) { TdFilePtr pFile = taosOpenFile(confDir, TD_FILE_CREATE | TD_FILE_WRITE | TD_FILE_TRUNC); if (pFile == NULL) { uError("[rsync] open conf file error, dir:%s,"ERRNO_ERR_FORMAT, confDir, ERRNO_ERR_DATA); @@ -98,8 +98,8 @@ static int generateConfigFile(char* confDir) { return 0; } -static int execCommand(char* command){ - int try = 3; +static int32_t execCommand(char* command){ + int32_t try = 3; int32_t code = 0; while(try-- > 0) { code = system(command); @@ -112,7 +112,7 @@ static int execCommand(char* command){ } void stopRsync() { - int code = + int32_t code = #ifdef WINDOWS system("taskkill /f /im rsync.exe"); #else @@ -135,7 +135,7 @@ void startRsync() { char confDir[PATH_MAX] = {0}; snprintf(confDir, PATH_MAX, "%srsync.conf", tsCheckpointBackupDir); - int code = generateConfigFile(confDir); + int32_t code = generateConfigFile(confDir); if(code != 0){ return; } @@ -148,14 +148,16 @@ void startRsync() { uError("[rsync] start server failed, code:%d,"ERRNO_ERR_FORMAT, code, ERRNO_ERR_DATA); return; } + uDebug("[rsync] start server successful"); } -int uploadRsync(const char* id, const char* path) { +int32_t uploadRsync(const char* id, const char* path) { #ifdef WINDOWS char pathTransform[PATH_MAX] = {0}; changeDirFromWindowsToLinux(path, pathTransform); #endif + char command[PATH_MAX] = {0}; #ifdef WINDOWS if(pathTransform[strlen(pathTransform) - 1] != '/'){ @@ -169,7 +171,7 @@ int uploadRsync(const char* id, const char* path) { path #endif , tsSnodeAddress, id); - }else{ + } else { snprintf(command, PATH_MAX, "rsync -av --delete --timeout=10 --bwlimit=100000 %s rsync://%s/checkpoint/%s/", #ifdef WINDOWS pathTransform @@ -179,16 +181,17 @@ int uploadRsync(const char* id, const char* path) { , tsSnodeAddress, id); } - int code = execCommand(command); + int32_t code = execCommand(command); if(code != 0){ uError("[rsync] send failed code:%d," ERRNO_ERR_FORMAT, code, ERRNO_ERR_DATA); return -1; } + uDebug("[rsync] upload data:%s successful", id); return 0; } -int downloadRsync(const char* id, const char* path) { +int32_t downloadRsync(const char* id, const char* path) { #ifdef WINDOWS char pathTransform[PATH_MAX] = {0}; changeDirFromWindowsToLinux(path, pathTransform); @@ -203,33 +206,34 @@ int downloadRsync(const char* id, const char* path) { #endif ); - int code = execCommand(command); - if(code != 0){ + int32_t code = execCommand(command); + if (code != 0) { uError("[rsync] get failed code:%d," ERRNO_ERR_FORMAT, code, ERRNO_ERR_DATA); return -1; } + uDebug("[rsync] down data:%s successful", id); return 0; } -int deleteRsync(const char* id) { - char* tmp = "./tmp_empty/"; - int code = taosMkDir(tmp); - if(code != 0){ +int32_t deleteRsync(const char* id) { + char* tmp = "./tmp_empty/"; + int32_t code = taosMkDir(tmp); + if (code != 0) { uError("[rsync] make tmp dir failed. code:%d," ERRNO_ERR_FORMAT, code, ERRNO_ERR_DATA); return -1; } + char command[PATH_MAX] = {0}; - snprintf(command, PATH_MAX, "rsync -av --delete --timeout=10 %s rsync://%s/checkpoint/%s/", - tmp, tsSnodeAddress, id); + snprintf(command, PATH_MAX, "rsync -av --delete --timeout=10 %s rsync://%s/checkpoint/%s/", tmp, tsSnodeAddress, id); code = execCommand(command); taosRemoveDir(tmp); - if(code != 0){ + if (code != 0) { uError("[rsync] get failed code:%d," ERRNO_ERR_FORMAT, code, ERRNO_ERR_DATA); return -1; } - uDebug("[rsync] delete data:%s successful", id); + uDebug("[rsync] delete data:%s successful", id); return 0; } \ No newline at end of file diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index aa268e9d725f..c5fc6f54ce9f 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -329,6 +329,7 @@ int32_t rebuildFromRemoteChkp_rsync(char* key, char* chkpPath, int64_t chkpId, c if (taosIsDir(chkpPath)) { taosRemoveDir(chkpPath); } + if (taosIsDir(defaultPath)) { taosRemoveDir(defaultPath); } @@ -337,10 +338,11 @@ int32_t rebuildFromRemoteChkp_rsync(char* key, char* chkpPath, int64_t chkpId, c if (code != 0) { return code; } - code = backendCopyFiles(chkpPath, defaultPath); + code = backendCopyFiles(chkpPath, defaultPath); return code; } + int32_t rebuildFromRemoteChkp_s3(char* key, char* chkpPath, int64_t chkpId, char* defaultPath) { int32_t code = streamTaskDownloadCheckpointData(key, chkpPath); if (code != 0) { @@ -375,6 +377,7 @@ int32_t rebuildFromRemoteChkp_s3(char* key, char* chkpPath, int64_t chkpId, char taosMemoryFree(tmp); return code; } + int32_t rebuildFromRemoteChkp(char* key, char* chkpPath, int64_t chkpId, char* defaultPath) { ECHECKPOINT_BACKUP_TYPE type = streamGetCheckpointBackupType(); if (type == DATA_UPLOAD_S3) { @@ -1996,6 +1999,7 @@ STaskDbWrapper* taskDbOpenImpl(char* key, char* statePath, char* dbPath) { if (cfNames) rocksdb_list_column_families_destroy(cfNames, nCf); return NULL; } + STaskDbWrapper* taskDbOpen(char* path, char* key, int64_t chkpId) { char* statePath = NULL; char* dbPath = NULL; diff --git a/source/libs/stream/src/streamCheckpoint.c b/source/libs/stream/src/streamCheckpoint.c index d2f9a0cbc378..1519382fe454 100644 --- a/source/libs/stream/src/streamCheckpoint.c +++ b/source/libs/stream/src/streamCheckpoint.c @@ -32,6 +32,7 @@ static int32_t downloadCheckpointDataByName(const char* id, const char* fname, c static int32_t deleteCheckpointFile(const char* id, const char* name); static int32_t streamTaskBackupCheckpoint(const char* id, const char* path); static int32_t deleteCheckpoint(const char* id); +static int32_t downloadCheckpointByNameS3(const char* id, const char* fname, const char* dstName); int32_t tEncodeStreamCheckpointSourceReq(SEncoder* pEncoder, const SStreamCheckpointSourceReq* pReq) { if (tStartEncode(pEncoder) < 0) return -1; @@ -444,14 +445,15 @@ int32_t uploadCheckpointData(void* param) { (int8_t)(arg->type), &path, toDelFiles)) != 0) { stError("s-task:%s failed to gen upload checkpoint:%" PRId64 "", taskStr, arg->chkpId); } + if (arg->type == DATA_UPLOAD_S3) { if (code == 0 && (code = getCheckpointDataMeta(arg->taskId, path, toDelFiles)) != 0) { - stError("s-task:%s failed to get checkpoint:%" PRId64 " meta", taskStr, arg->chkpId); + stError("s-task:%s failed to get checkpointId:%" PRId64 " meta", taskStr, arg->chkpId); } } if (code == 0 && (code = streamTaskBackupCheckpoint(arg->taskId, path)) != 0) { - stError("s-task:%s failed to upload checkpoint:%" PRId64, taskStr, arg->chkpId); + stError("s-task:%s failed to upload checkpointId:%" PRId64, taskStr, arg->chkpId); } taskReleaseDb(arg->dbRefId); @@ -610,13 +612,19 @@ static int32_t uploadCheckpointToS3(const char* id, const char* path) { return 0; } -static int32_t downloadCheckpointByNameS3(const char* id, const char* fname, const char* dstName) { +int32_t downloadCheckpointByNameS3(const char* id, const char* fname, const char* dstName) { int32_t code = 0; char* buf = taosMemoryCalloc(1, strlen(id) + strlen(dstName) + 4); + if (buf == NULL) { + code = terrno = TSDB_CODE_OUT_OF_MEMORY; + return code; + } + sprintf(buf, "%s/%s", id, fname); if (s3GetObjectToFile(buf, dstName) != 0) { - code = -1; + code = errno; } + taosMemoryFree(buf); return code; } @@ -636,11 +644,13 @@ int32_t streamTaskBackupCheckpoint(const char* id, const char* path) { stError("streamTaskBackupCheckpoint parameters invalid"); return -1; } + if (strlen(tsSnodeAddress) != 0) { return uploadRsync(id, path); } else if (tsS3StreamEnabled) { return uploadCheckpointToS3(id, path); } + return 0; } diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 399d98437b3d..83a62daf27fe 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -104,6 +104,7 @@ void metaRefMgtCleanup() { int32_t metaRefMgtAdd(int64_t vgId, int64_t* rid) { taosThreadMutexLock(&gMetaRefMgt.mutex); + void* p = taosHashGet(gMetaRefMgt.pTable, &vgId, sizeof(vgId)); if (p == NULL) { SArray* list = taosArrayInit(8, sizeof(void*)); @@ -113,6 +114,7 @@ int32_t metaRefMgtAdd(int64_t vgId, int64_t* rid) { SArray* list = *(SArray**)p; taosArrayPush(list, &rid); } + taosThreadMutexUnlock(&gMetaRefMgt.mutex); return 0; } @@ -284,6 +286,7 @@ int32_t streamTaskSetDb(SStreamMeta* pMeta, void* arg, char* key) { stDebug("s-task:0x%x set backend %p", pTask->id.taskId, pBackend); return 0; } + void streamMetaRemoveDB(void* arg, char* key) { if (arg == NULL || key == NULL) return; From ececb735ee8a1fa4eabbf354f19dac02241899a7 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 8 May 2024 13:54:25 +0800 Subject: [PATCH 14/30] refactor: do some internal refactor. --- source/libs/stream/src/streamMeta.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 83a62daf27fe..eb0f148adc9a 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -770,7 +770,6 @@ int32_t streamMetaBegin(SStreamMeta* pMeta) { return code; } -// todo add error log int32_t streamMetaCommit(SStreamMeta* pMeta) { if (tdbCommit(pMeta->db, pMeta->txn) < 0) { stError("vgId:%d failed to commit stream meta", pMeta->vgId); @@ -788,6 +787,7 @@ int32_t streamMetaCommit(SStreamMeta* pMeta) { return -1; } + stDebug("vgId:%d stream meta file commit completed", pMeta->vgId); return 0; } From 52c202db899961e0c7c3e47b4a0201378f1da00a Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 8 May 2024 16:21:48 +0800 Subject: [PATCH 15/30] fix(sort): set correct output row index. --- source/libs/executor/src/tsort.c | 170 +++++++++++++++++-------------- 1 file changed, 91 insertions(+), 79 deletions(-) diff --git a/source/libs/executor/src/tsort.c b/source/libs/executor/src/tsort.c index cd1a85817512..b1688755e682 100644 --- a/source/libs/executor/src/tsort.c +++ b/source/libs/executor/src/tsort.c @@ -38,11 +38,11 @@ typedef struct SSortMemFileRegion { int32_t bufRegOffset; int32_t bufLen; - char* buf; + char* buf; } SSortMemFileRegion; typedef struct SSortMemFile { - char* writeBuf; + char* writeBuf; int32_t writeBufSize; int64_t writeFileOffset; @@ -55,7 +55,7 @@ typedef struct SSortMemFile { int32_t blockSize; FILE* pTdFile; - char memFilePath[PATH_MAX]; + char memFilePath[PATH_MAX]; } SSortMemFile; struct SSortHandle { @@ -260,6 +260,7 @@ SSortHandle* tsortCreateSortHandle(SArray* pSortInfo, int32_t type, int32_t page pSortHandle->cmpParam.orderInfo = pSortInfo; pSortHandle->cmpParam.cmpGroupId = false; pSortHandle->cmpParam.sortType = type; + if (type == SORT_BLOCK_TS_MERGE) { SBlockOrderInfo* pTsOrder = TARRAY_GET_ELEM(pSortInfo, 0); pSortHandle->cmpParam.tsSlotId = pTsOrder->slotId; @@ -522,10 +523,9 @@ static int32_t sortComparInit(SMsortComparParam* pParam, SArray* pSources, int32 static void appendOneRowToDataBlock(SSDataBlock* pBlock, const SSDataBlock* pSource, int32_t* rowIndex) { for (int32_t i = 0; i < taosArrayGetSize(pBlock->pDataBlock); ++i) { SColumnInfoData* pColInfo = taosArrayGet(pBlock->pDataBlock, i); - SColumnInfoData* pSrcColInfo = taosArrayGet(pSource->pDataBlock, i); - bool isNull = colDataIsNull(pSrcColInfo, pSource->info.rows, *rowIndex, NULL); + bool isNull = colDataIsNull(pSrcColInfo, pSource->info.rows, *rowIndex, NULL); if (isNull) { colDataSetVal(pColInfo, pBlock->info.rows, NULL, true); } else { @@ -557,7 +557,9 @@ static int32_t adjustMergeTreeForNextTuple(SSortSource* pSource, SMultiwayMergeT pSource->pageIndex = -1; pSource->src.pBlock = blockDataDestroy(pSource->src.pBlock); } else { - if (pSource->pageIndex % 512 == 0) qDebug("begin source %p page %d", pSource, pSource->pageIndex); + if (pSource->pageIndex % 512 == 0) { + qDebug("begin source %p page %d", pSource, pSource->pageIndex); + } int32_t* pPgId = taosArrayGet(pSource->pageIdList, pSource->pageIndex); @@ -635,7 +637,7 @@ static SSDataBlock* getSortedBlockDataInner(SSortHandle* pHandle, SMsortComparPa // TODO: improve this function performance -int tsortComparBlockCell(SSDataBlock* pLeftBlock, SSDataBlock* pRightBlock, +int32_t tsortComparBlockCell(SSDataBlock* pLeftBlock, SSDataBlock* pRightBlock, int32_t leftRowIndex, int32_t rightRowIndex, void* pCompareOrder) { SBlockOrderInfo* pOrder = pCompareOrder; SColumnInfoData* pLeftColInfoData = TARRAY_GET_ELEM(pLeftBlock->pDataBlock, pOrder->slotId); @@ -680,7 +682,7 @@ int tsortComparBlockCell(SSDataBlock* pLeftBlock, SSDataBlock* pRightBlock, left1 = colDataGetData(pLeftColInfoData, leftRowIndex); right1 = colDataGetData(pRightColInfoData, rightRowIndex); __compar_fn_t fn = pOrder->compFn; - int ret = fn(left1, right1); + int32_t ret = fn(left1, right1); return ret; } @@ -719,7 +721,7 @@ int32_t msortComparFn(const void* pLeft, const void* pRight, void* param) { int64_t* leftTs = (int64_t*)(pLeftTsCol->pData) + pLeftSource->src.rowIndex; int64_t* rightTs = (int64_t*)(pRightTsCol->pData) + pRightSource->src.rowIndex; - int ret = pParam->cmpTsFn(leftTs, rightTs); + int32_t ret = pParam->cmpTsFn(leftTs, rightTs); if (ret == 0 && pParam->pPkOrder) { ret = tsortComparBlockCell(pLeftBlock, pRightBlock, pLeftSource->src.rowIndex, pRightSource->src.rowIndex, (SBlockOrderInfo*)pParam->pPkOrder); @@ -782,7 +784,7 @@ int32_t msortComparFn(const void* pLeft, const void* pRight, void* param) { pOrder->compFn = fn; } - int ret = fn(left1, right1); + int32_t ret = fn(left1, right1); if (ret == 0) { continue; } else { @@ -855,7 +857,7 @@ static int32_t doInternalMergeSort(SSortHandle* pHandle) { return code; } - int nMergedRows = 0; + int32_t nMergedRows = 0; SArray* pPageIdList = taosArrayInit(4, sizeof(int32_t)); while (1) { @@ -1075,7 +1077,7 @@ static int32_t getRowBufFromExtMemFile(SSortHandle* pHandle, int32_t regionId, i } taosSeekCFile(pMemFile->pTdFile, pRegion->fileOffset, SEEK_SET); int32_t readBytes = TMIN(pMemFile->blockSize, pRegion->regionSize); - int ret = taosReadFromCFile(pRegion->buf, readBytes, 1, pMemFile->pTdFile); + int32_t ret = taosReadFromCFile(pRegion->buf, readBytes, 1, pMemFile->pTdFile); if (ret != 1) { terrno = TAOS_SYSTEM_ERROR(errno); return terrno; @@ -1095,7 +1097,7 @@ static int32_t getRowBufFromExtMemFile(SSortHandle* pHandle, int32_t regionId, i memcpy(*ppRow, pRegion->buf + tupleOffset - pRegion->bufRegOffset, szThisBlock); taosSeekCFile(pMemFile->pTdFile, pRegion->fileOffset + pRegion->bufRegOffset + pRegion->bufLen, SEEK_SET); int32_t readBytes = TMIN(pMemFile->blockSize, pRegion->regionSize - (pRegion->bufRegOffset + pRegion->bufLen)); - int ret = taosReadFromCFile(pRegion->buf, readBytes, 1, pMemFile->pTdFile); + int32_t ret = taosReadFromCFile(pRegion->buf, readBytes, 1, pMemFile->pTdFile); if (ret != 1) { taosMemoryFreeClear(*ppRow); terrno = TAOS_SYSTEM_ERROR(errno); @@ -1214,7 +1216,7 @@ static int32_t tsortCloseRegion(SSortHandle* pHandle) { pRegion->regionSize = pMemFile->currRegionOffset; int32_t writeBytes = pRegion->regionSize - (pMemFile->writeFileOffset - pRegion->fileOffset); if (writeBytes > 0) { - int ret = fwrite(pMemFile->writeBuf, writeBytes, 1, pMemFile->pTdFile); + int32_t ret = fwrite(pMemFile->writeBuf, writeBytes, 1, pMemFile->pTdFile); if (ret != 1) { terrno = TAOS_SYSTEM_ERROR(errno); return terrno; @@ -1247,7 +1249,7 @@ static int32_t saveBlockRowToExtRowsMemFile(SSortHandle* pHandle, SSDataBlock* p { if (pMemFile->currRegionOffset + pHandle->extRowBytes >= pMemFile->writeBufSize) { int32_t writeBytes = pMemFile->currRegionOffset - (pMemFile->writeFileOffset - pRegion->fileOffset); - int ret = fwrite(pMemFile->writeBuf, writeBytes, 1, pMemFile->pTdFile); + int32_t ret = fwrite(pMemFile->writeBuf, writeBytes, 1, pMemFile->pTdFile); if (ret != 1) { terrno = TAOS_SYSTEM_ERROR(errno); return terrno; @@ -1317,6 +1319,7 @@ static void initRowIdSort(SSortHandle* pHandle) { blockDataAppendColInfo(pSortInput, &offsetCol); SColumnInfoData lengthCol = createColumnInfoData(TSDB_DATA_TYPE_INT, 4, 4); blockDataAppendColInfo(pSortInput, &lengthCol); + if (pHandle->bSortPk) { pkCol = createColumnInfoData(extPkCol->info.type, extPkCol->info.bytes, 5); blockDataAppendColInfo(pSortInput, &pkCol); @@ -1324,20 +1327,21 @@ static void initRowIdSort(SSortHandle* pHandle) { blockDataDestroy(pHandle->pDataBlock); pHandle->pDataBlock = pSortInput; - int32_t rowSize = blockDataGetRowSize(pHandle->pDataBlock); - size_t nCols = taosArrayGetSize(pHandle->pDataBlock->pDataBlock); +// int32_t rowSize = blockDataGetRowSize(pHandle->pDataBlock); +// size_t nCols = taosArrayGetSize(pHandle->pDataBlock->pDataBlock); pHandle->pageSize = 256 * 1024; // 256k pHandle->numOfPages = 256; - SArray* aOrder = taosArrayInit(1, sizeof(SBlockOrderInfo)); + SArray* pOrderInfoList = taosArrayInit(1, sizeof(SBlockOrderInfo)); + + int32_t tsOrder = ((SBlockOrderInfo*)taosArrayGet(pHandle->pSortInfo, 0))->order; - SBlockOrderInfo* pTsOrder = taosArrayGet(pHandle->pSortInfo, 0); SBlockOrderInfo biTs = {0}; - biTs.order = pTsOrder->order; + biTs.order = tsOrder; biTs.slotId = 0; biTs.nullFirst = (biTs.order == TSDB_ORDER_ASC); biTs.compFn = getKeyComparFunc(TSDB_DATA_TYPE_TIMESTAMP, biTs.order); - taosArrayPush(aOrder, &biTs); + taosArrayPush(pOrderInfoList, &biTs); if (pHandle->bSortPk) { SBlockOrderInfo biPk = {0}; @@ -1345,11 +1349,11 @@ static void initRowIdSort(SSortHandle* pHandle) { biPk.slotId = 4; biPk.nullFirst = (biPk.order == TSDB_ORDER_ASC); biPk.compFn = getKeyComparFunc(pkCol.info.type, biPk.order); - taosArrayPush(aOrder, &biPk); + taosArrayPush(pOrderInfoList, &biPk); } + taosArrayDestroy(pHandle->pSortInfo); - pHandle->pSortInfo = aOrder; - return; + pHandle->pSortInfo = pOrderInfoList; } int32_t tsortSetSortByRowId(SSortHandle* pHandle, int32_t extRowsMemSize) { @@ -1441,8 +1445,8 @@ static int32_t appendDataBlockToPageBuf(SSortHandle* pHandle, SSDataBlock* blk, } static int32_t getPageBufIncForRow(SSDataBlock* blk, int32_t row, int32_t rowIdxInPage) { - int sz = 0; - int numCols = taosArrayGetSize(blk->pDataBlock); + int32_t sz = 0; + int32_t numCols = taosArrayGetSize(blk->pDataBlock); if (!blk->info.hasVarCol) { sz += numCols * ((rowIdxInPage & 0x7) == 0 ? 1: 0); sz += blockDataGetRowSize(blk); @@ -1470,42 +1474,46 @@ static int32_t getPageBufIncForRow(SSDataBlock* blk, int32_t row, int32_t rowIdx static int32_t sortBlocksToExtSource(SSortHandle* pHandle, SArray* aBlk, SArray* aExtSrc) { int32_t code = TSDB_CODE_SUCCESS; - int pgHeaderSz = sizeof(int32_t) + sizeof(int32_t) * taosArrayGetSize(pHandle->pDataBlock->pDataBlock); + int32_t pgHeaderSz = sizeof(int32_t) + sizeof(int32_t) * taosArrayGetSize(pHandle->pDataBlock->pDataBlock); int32_t rowCap = blockDataGetCapacityInRow(pHandle->pDataBlock, pHandle->pageSize, pgHeaderSz); + blockDataEnsureCapacity(pHandle->pDataBlock, rowCap); blockDataCleanup(pHandle->pDataBlock); int32_t numBlks = taosArrayGetSize(aBlk); - SBlockOrderInfo* pOrigBlockTsOrder = (!pHandle->bSortByRowId) ? - taosArrayGet(pHandle->pSortInfo, 0) : taosArrayGet(pHandle->aExtRowsOrders, 0); + SBlockOrderInfo* pOrigBlockTsOrder = + (!pHandle->bSortByRowId) ? taosArrayGet(pHandle->pSortInfo, 0) : taosArrayGet(pHandle->aExtRowsOrders, 0); + SBlockOrderInfo* pHandleBlockTsOrder = taosArrayGet(pHandle->pSortInfo, 0); SBlkMergeSupport sup = {0}; sup.aRowIdx = taosMemoryCalloc(numBlks, sizeof(int32_t)); sup.aTs = taosMemoryCalloc(numBlks, sizeof(int64_t*)); sup.tsOrder = pOrigBlockTsOrder->order; sup.aBlks = taosMemoryCalloc(numBlks, sizeof(SSDataBlock*)); - for (int i = 0; i < numBlks; ++i) { - SSDataBlock* blk = taosArrayGetP(aBlk, i); + + for (int32_t i = 0; i < numBlks; ++i) { + SSDataBlock* blk = taosArrayGetP(aBlk, i); SColumnInfoData* col = taosArrayGet(blk->pDataBlock, pOrigBlockTsOrder->slotId); sup.aTs[i] = (int64_t*)col->pData; sup.aRowIdx[i] = 0; sup.aBlks[i] = blk; } + SBlockOrderInfo* pOrigBlockPkOrder = NULL; if (pHandle->bSortPk) { - pOrigBlockPkOrder = (!pHandle->bSortByRowId) ? - taosArrayGet(pHandle->pSortInfo, 1) : taosArrayGet(pHandle->aExtRowsOrders, 1); + pOrigBlockPkOrder = + (!pHandle->bSortByRowId) ? taosArrayGet(pHandle->pSortInfo, 1) : taosArrayGet(pHandle->aExtRowsOrders, 1); } sup.pPkOrder = pOrigBlockPkOrder; int32_t totalRows = 0; - for (int i = 0; i < numBlks; ++i) { + for (int32_t i = 0; i < numBlks; ++i) { SSDataBlock* blk = taosArrayGetP(aBlk, i); totalRows += blk->info.rows; } SMultiwayMergeTreeInfo* pTree = NULL; - __merge_compare_fn_t mergeCompareFn = (!pHandle->bSortPk) ? blockCompareTsFn : blockCompareTsPkFn; + __merge_compare_fn_t mergeCompareFn = (!pHandle->bSortPk) ? blockCompareTsFn : blockCompareTsPkFn; code = tMergeTreeCreate(&pTree, taosArrayGetSize(aBlk), &sup, mergeCompareFn); if (TSDB_CODE_SUCCESS != code) { taosMemoryFree(sup.aRowIdx); @@ -1517,50 +1525,53 @@ static int32_t sortBlocksToExtSource(SSortHandle* pHandle, SArray* aBlk, SArray* SArray* aPgId = taosArrayInit(8, sizeof(int32_t)); int32_t nRows = 0; int32_t nMergedRows = 0; - bool mergeLimitReached = false; - size_t blkPgSz = pgHeaderSz; + bool mergeLimitReached = false; + size_t blkPgSz = pgHeaderSz; int64_t lastPageBufTs = (pHandleBlockTsOrder->order == TSDB_ORDER_ASC) ? INT64_MAX : INT64_MIN; - int64_t currTs = (pHandleBlockTsOrder->order == TSDB_ORDER_ASC) ? INT64_MAX : INT64_MIN; + while (nRows < totalRows) { - int32_t minIdx = tMergeTreeGetChosenIndex(pTree); + int32_t minIdx = tMergeTreeGetChosenIndex(pTree); SSDataBlock* minBlk = taosArrayGetP(aBlk, minIdx); - int32_t minRow = sup.aRowIdx[minIdx]; + int32_t minRow = sup.aRowIdx[minIdx]; SSDataBlock* incBlock = (pHandle->bSortByRowId) ? pHandle->pDataBlock : minBlk; - int32_t bufInc = getPageBufIncForRow(incBlock, minRow, pHandle->pDataBlock->info.rows); + int32_t bufInc = getPageBufIncForRow(incBlock, minRow, pHandle->pDataBlock->info.rows); if (blkPgSz <= pHandle->pageSize && blkPgSz + bufInc > pHandle->pageSize) { - SColumnInfoData* tsCol = taosArrayGet(pHandle->pDataBlock->pDataBlock, pHandleBlockTsOrder->slotId); - lastPageBufTs = ((int64_t*)tsCol->pData)[pHandle->pDataBlock->info.rows - 1]; - code = appendDataBlockToPageBuf(pHandle, pHandle->pDataBlock, aPgId); - if (code != TSDB_CODE_SUCCESS) { - taosMemoryFree(pTree); - taosArrayDestroy(aPgId); - taosMemoryFree(sup.aRowIdx); - taosMemoryFree(sup.aTs); - taosMemoryFree(sup.aBlks); - return code; + SColumnInfoData* tsCol = taosArrayGet(pHandle->pDataBlock->pDataBlock, pHandleBlockTsOrder->slotId); + lastPageBufTs = ((int64_t*)tsCol->pData)[pHandle->pDataBlock->info.rows - 1]; + code = appendDataBlockToPageBuf(pHandle, pHandle->pDataBlock, aPgId); + if (code != TSDB_CODE_SUCCESS) { + taosMemoryFree(pTree); + taosArrayDestroy(aPgId); + taosMemoryFree(sup.aRowIdx); + taosMemoryFree(sup.aTs); + taosMemoryFree(sup.aBlks); + return code; + } + + nMergedRows += pHandle->pDataBlock->info.rows; + blockDataCleanup(pHandle->pDataBlock); + blkPgSz = pgHeaderSz; + incBlock = (pHandle->bSortByRowId) ? pHandle->pDataBlock : minBlk; + bufInc = getPageBufIncForRow(incBlock, minRow, pHandle->pDataBlock->info.rows); + + if ((pHandle->mergeLimit != -1) && (nMergedRows >= pHandle->mergeLimit)) { + mergeLimitReached = true; + if ((lastPageBufTs < pHandle->currMergeLimitTs && pHandleBlockTsOrder->order == TSDB_ORDER_ASC) || + (lastPageBufTs > pHandle->currMergeLimitTs && pHandleBlockTsOrder->order == TSDB_ORDER_DESC)) { + pHandle->currMergeLimitTs = lastPageBufTs; } - nMergedRows += pHandle->pDataBlock->info.rows; - blockDataCleanup(pHandle->pDataBlock); - blkPgSz = pgHeaderSz; - incBlock = (pHandle->bSortByRowId) ? pHandle->pDataBlock : minBlk; - bufInc = getPageBufIncForRow(incBlock, minRow, 0); - - if ((pHandle->mergeLimit != -1) && (nMergedRows >= pHandle->mergeLimit)) { - mergeLimitReached = true; - if ((lastPageBufTs < pHandle->currMergeLimitTs && pHandleBlockTsOrder->order == TSDB_ORDER_ASC) || - (lastPageBufTs > pHandle->currMergeLimitTs && pHandleBlockTsOrder->order == TSDB_ORDER_DESC)) { - pHandle->currMergeLimitTs = lastPageBufTs; - } - break; - } + break; + } } + blockDataEnsureCapacity(pHandle->pDataBlock, pHandle->pDataBlock->info.rows + 1); if (!pHandle->bSortByRowId) { - appendOneRowToDataBlock(pHandle->pDataBlock, minBlk, &minRow); + appendOneRowToDataBlock(pHandle->pDataBlock, minBlk, &minRow); } else { - appendToRowIndexDataBlock(pHandle, minBlk, &minRow); + appendToRowIndexDataBlock(pHandle, minBlk, &minRow); } + blkPgSz += bufInc; ++nRows; @@ -1572,6 +1583,7 @@ static int32_t sortBlocksToExtSource(SSortHandle* pHandle, SArray* aBlk, SArray* } tMergeTreeAdjust(pTree, tMergeTreeGetAdjustIndex(pTree)); } + if (pHandle->pDataBlock->info.rows > 0) { if (!mergeLimitReached) { SColumnInfoData* tsCol = taosArrayGet(pHandle->pDataBlock->pDataBlock, pHandleBlockTsOrder->slotId); @@ -1584,14 +1596,14 @@ static int32_t sortBlocksToExtSource(SSortHandle* pHandle, SArray* aBlk, SArray* taosMemoryFree(sup.aTs); taosMemoryFree(sup.aBlks); return code; - } + } nMergedRows += pHandle->pDataBlock->info.rows; if ((pHandle->mergeLimit != -1) && (nMergedRows >= pHandle->mergeLimit)) { - mergeLimitReached = true; - if ((lastPageBufTs < pHandle->currMergeLimitTs && pHandleBlockTsOrder->order == TSDB_ORDER_ASC) || - (lastPageBufTs > pHandle->currMergeLimitTs && pHandleBlockTsOrder->order == TSDB_ORDER_DESC)) { - pHandle->currMergeLimitTs = lastPageBufTs; - } + mergeLimitReached = true; + if ((lastPageBufTs < pHandle->currMergeLimitTs && pHandleBlockTsOrder->order == TSDB_ORDER_ASC) || + (lastPageBufTs > pHandle->currMergeLimitTs && pHandleBlockTsOrder->order == TSDB_ORDER_DESC)) { + pHandle->currMergeLimitTs = lastPageBufTs; + } } } blockDataCleanup(pHandle->pDataBlock); @@ -1724,7 +1736,7 @@ static int32_t createBlocksMergeSortInitialSources(SSortHandle* pHandle) { code = sortBlocksToExtSource(pHandle, aBlkSort, aExtSrc); if (code != TSDB_CODE_SUCCESS) { - for (int i = 0; i < taosArrayGetSize(aBlkSort); ++i) { + for (int32_t i = 0; i < taosArrayGetSize(aBlkSort); ++i) { blockDataDestroy(taosArrayGetP(aBlkSort, i)); } taosArrayClear(aBlkSort); @@ -1736,7 +1748,7 @@ static int32_t createBlocksMergeSortInitialSources(SSortHandle* pHandle) { int64_t el = taosGetTimestampUs() - p; pHandle->sortElapsed += el; - for (int i = 0; i < taosArrayGetSize(aBlkSort); ++i) { + for (int32_t i = 0; i < taosArrayGetSize(aBlkSort); ++i) { blockDataDestroy(taosArrayGetP(aBlkSort, i)); } taosArrayClear(aBlkSort); @@ -1750,7 +1762,7 @@ static int32_t createBlocksMergeSortInitialSources(SSortHandle* pHandle) { if (tsortIsClosed(pHandle)) { tSimpleHashClear(mUidBlk); - for (int i = 0; i < taosArrayGetSize(aBlkSort); ++i) { + for (int32_t i = 0; i < taosArrayGetSize(aBlkSort); ++i) { blockDataDestroy(taosArrayGetP(aBlkSort, i)); } taosArrayClear(aBlkSort); @@ -1759,7 +1771,7 @@ static int32_t createBlocksMergeSortInitialSources(SSortHandle* pHandle) { } tSimpleHashCleanup(mUidBlk); - for (int i = 0; i < taosArrayGetSize(aBlkSort); ++i) { + for (int32_t i = 0; i < taosArrayGetSize(aBlkSort); ++i) { blockDataDestroy(taosArrayGetP(aBlkSort, i)); } taosArrayDestroy(aBlkSort); @@ -2048,10 +2060,10 @@ static int32_t tupleComparFn(const void* pLeft, const void* pRight, void* param) if (!lData) return pOrder->nullFirst ? -1 : 1; if (!rData) return pOrder->nullFirst ? 1 : -1; - int type = ((SColumnInfoData*)taosArrayGet(pHandle->pDataBlock->pDataBlock, pOrder->slotId))->info.type; + int32_t type = ((SColumnInfoData*)taosArrayGet(pHandle->pDataBlock->pDataBlock, pOrder->slotId))->info.type; __compar_fn_t fn = getKeyComparFunc(type, pOrder->order); - int ret = fn(lData, rData); + int32_t ret = fn(lData, rData); if (ret == 0) { continue; } else { From 36cd476c60b167ee07931240e306809cdd87737d Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 8 May 2024 16:22:02 +0800 Subject: [PATCH 16/30] refactor: do some internal refactor. --- source/libs/executor/src/scanoperator.c | 2 ++ source/libs/stream/src/streamMeta.c | 2 +- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/source/libs/executor/src/scanoperator.c b/source/libs/executor/src/scanoperator.c index 4ddee2983477..fd7047d683df 100644 --- a/source/libs/executor/src/scanoperator.c +++ b/source/libs/executor/src/scanoperator.c @@ -4318,6 +4318,7 @@ void tableMergeScanTsdbNotifyCb(ETsdReaderNotifyType type, STsdReaderNotifyInfo* int32_t startDurationForGroupTableMergeScan(SOperatorInfo* pOperator) { STableMergeScanInfo* pInfo = pOperator->info; SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; + int32_t code = TSDB_CODE_SUCCESS; int32_t numOfTable = pInfo->tableEndIndex - pInfo->tableStartIndex + 1; @@ -4337,6 +4338,7 @@ int32_t startDurationForGroupTableMergeScan(SOperatorInfo* pOperator) { return code; } } + tsortSetMergeLimit(pInfo->pSortHandle, pInfo->mergeLimit); tsortSetMergeLimitReachedFp(pInfo->pSortHandle, tableMergeScanDoSkipTable, pInfo); tsortSetAbortCheckFn(pInfo->pSortHandle, isTaskKilled, pOperator->pTaskInfo); diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index eb0f148adc9a..a373ec28c0ab 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -1735,7 +1735,7 @@ int32_t streamMetaAddFailedTask(SStreamMeta* pMeta, int64_t streamId, int32_t ta bool hasFillhistoryTask = false; STaskId hId = {0}; - stDebug("vgId:%d add failed task:0x%x", pMeta->vgId, taskId); + stDebug("vgId:%d add start failed task:0x%x", pMeta->vgId, taskId); streamMetaRLock(pMeta); From b26709a0f2a3e2b29f43c672769faf312ec10ee1 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 8 May 2024 16:37:31 +0800 Subject: [PATCH 17/30] refactor: do some internal refactor. --- source/libs/executor/src/tsort.c | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/source/libs/executor/src/tsort.c b/source/libs/executor/src/tsort.c index b1688755e682..6fe7e73f7788 100644 --- a/source/libs/executor/src/tsort.c +++ b/source/libs/executor/src/tsort.c @@ -1553,7 +1553,7 @@ static int32_t sortBlocksToExtSource(SSortHandle* pHandle, SArray* aBlk, SArray* blockDataCleanup(pHandle->pDataBlock); blkPgSz = pgHeaderSz; incBlock = (pHandle->bSortByRowId) ? pHandle->pDataBlock : minBlk; - bufInc = getPageBufIncForRow(incBlock, minRow, pHandle->pDataBlock->info.rows); + bufInc = getPageBufIncForRow(incBlock, minRow, 0); if ((pHandle->mergeLimit != -1) && (nMergedRows >= pHandle->mergeLimit)) { mergeLimitReached = true; @@ -1573,6 +1573,7 @@ static int32_t sortBlocksToExtSource(SSortHandle* pHandle, SArray* aBlk, SArray* } blkPgSz += bufInc; + ASSERT(blkPgSz == blockDataGetSize(pHandle->pDataBlock)); ++nRows; From bf89d27b9ab5836ea08158267850c547d77ff77c Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 8 May 2024 16:40:51 +0800 Subject: [PATCH 18/30] refactor: do some internal refactor. --- source/libs/executor/src/tsort.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/libs/executor/src/tsort.c b/source/libs/executor/src/tsort.c index 6fe7e73f7788..272629627aac 100644 --- a/source/libs/executor/src/tsort.c +++ b/source/libs/executor/src/tsort.c @@ -1573,7 +1573,7 @@ static int32_t sortBlocksToExtSource(SSortHandle* pHandle, SArray* aBlk, SArray* } blkPgSz += bufInc; - ASSERT(blkPgSz == blockDataGetSize(pHandle->pDataBlock)); + ASSERT(blkPgSz == blockDataGetSize(pHandle->pDataBlock) + pgHeaderSz); ++nRows; From 412a158c1a3ec03cc3783fa74e22adc5b6b62151 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 8 May 2024 18:17:22 +0800 Subject: [PATCH 19/30] fix(query): set correct row length for rowid sort. --- source/libs/executor/src/tsort.c | 90 ++++++++++++++++++++++++-------- 1 file changed, 68 insertions(+), 22 deletions(-) diff --git a/source/libs/executor/src/tsort.c b/source/libs/executor/src/tsort.c index 272629627aac..2ef40cd73899 100644 --- a/source/libs/executor/src/tsort.c +++ b/source/libs/executor/src/tsort.c @@ -1243,7 +1243,9 @@ static int32_t tsortFinalizeRegions(SSortHandle* pHandle) { return TSDB_CODE_SUCCESS; } -static int32_t saveBlockRowToExtRowsMemFile(SSortHandle* pHandle, SSDataBlock* pBlock, int32_t rowIdx, int32_t* pRegionId, int32_t* pOffset, int32_t* pLength) { +static int32_t saveBlockRowToExtRowsMemFile(SSortHandle* pHandle, SSDataBlock* pBlock, int32_t rowIdx, + int32_t* pRegionId, int32_t* pOffset, int32_t* pLength) { + SSortMemFile* pMemFile = pHandle->pExtRowsMemFile; SSortMemFileRegion* pRegion = taosArrayGet(pMemFile->aFileRegions, pMemFile->currRegionId); { @@ -1257,11 +1259,13 @@ static int32_t saveBlockRowToExtRowsMemFile(SSortHandle* pHandle, SSDataBlock* p pMemFile->writeFileOffset = pRegion->fileOffset + pMemFile->currRegionOffset; } } + *pRegionId = pMemFile->currRegionId; *pOffset = pMemFile->currRegionOffset; int32_t writeBufOffset = pMemFile->currRegionOffset - (pMemFile->writeFileOffset - pRegion->fileOffset); int32_t blockLen = blockRowToBuf(pBlock, rowIdx, pMemFile->writeBuf + writeBufOffset); *pLength = blockLen; + pMemFile->currRegionOffset += blockLen; pMemFile->bRegionDirty = true; return TSDB_CODE_SUCCESS; @@ -1324,6 +1328,7 @@ static void initRowIdSort(SSortHandle* pHandle) { pkCol = createColumnInfoData(extPkCol->info.type, extPkCol->info.bytes, 5); blockDataAppendColInfo(pSortInput, &pkCol); } + blockDataDestroy(pHandle->pDataBlock); pHandle->pDataBlock = pSortInput; @@ -1444,32 +1449,61 @@ static int32_t appendDataBlockToPageBuf(SSortHandle* pHandle, SSDataBlock* blk, return 0; } -static int32_t getPageBufIncForRow(SSDataBlock* blk, int32_t row, int32_t rowIdxInPage) { - int32_t sz = 0; - int32_t numCols = taosArrayGetSize(blk->pDataBlock); - if (!blk->info.hasVarCol) { - sz += numCols * ((rowIdxInPage & 0x7) == 0 ? 1: 0); - sz += blockDataGetRowSize(blk); +static int32_t getPageBufIncForRow(SSDataBlock* pSrcBlock, int32_t srcRowIndex, int32_t dstRowIndex) { + int32_t size = 0; + int32_t numCols = taosArrayGetSize(pSrcBlock->pDataBlock); + + if (!pSrcBlock->info.hasVarCol) { + size += numCols * ((dstRowIndex & 0x7) == 0 ? 1: 0); + size += blockDataGetRowSize(pSrcBlock); } else { for (int32_t i = 0; i < numCols; ++i) { - SColumnInfoData* pColInfoData = TARRAY_GET_ELEM(blk->pDataBlock, i); + SColumnInfoData* pColInfoData = TARRAY_GET_ELEM(pSrcBlock->pDataBlock, i); if (IS_VAR_DATA_TYPE(pColInfoData->info.type)) { - if ((pColInfoData->varmeta.offset[row] != -1) && (pColInfoData->pData)) { - char* p = colDataGetData(pColInfoData, row); - sz += varDataTLen(p); + if ((pColInfoData->varmeta.offset[srcRowIndex] != -1) && (pColInfoData->pData)) { + char* p = colDataGetData(pColInfoData, srcRowIndex); + size += varDataTLen(p); } - sz += sizeof(pColInfoData->varmeta.offset[0]); + size += sizeof(pColInfoData->varmeta.offset[0]); } else { - sz += pColInfoData->info.bytes; + size += pColInfoData->info.bytes; - if (((rowIdxInPage) & 0x07) == 0) { - sz += 1; // bitmap + if (((dstRowIndex) & 0x07) == 0) { + size += 1; // bitmap } } } } - return sz; + + return size; +} + +static int32_t getPageBufIncForRowIdSort(SSDataBlock* pDstBlock, int32_t srcRowIndex, int32_t dstRowIndex, SColumnInfoData* pPkCol) { + int32_t size = 0; + int32_t numOfCols = blockDataGetNumOfCols(pDstBlock); + + if (pPkCol == NULL) { + ASSERT(!pDstBlock->info.hasVarCol); + size += numOfCols * ((dstRowIndex & 0x7) == 0 ? 1: 0); + size += blockDataGetRowSize(pDstBlock); + } else { + if (IS_VAR_DATA_TYPE(pPkCol->info.type)) { + if ((pPkCol->varmeta.offset[srcRowIndex] != -1) && (pPkCol->pData)) { + char* p = colDataGetData(pPkCol, srcRowIndex); + size += varDataTLen(p); + } + + size += sizeof(pPkCol->varmeta.offset[0]); + } else { + size += pPkCol->info.bytes; + if (((dstRowIndex) & 0x07) == 0) { + size += 1; // bitmap + } + } + } + + return size; } static int32_t sortBlocksToExtSource(SSortHandle* pHandle, SArray* aBlk, SArray* aExtSrc) { @@ -1533,8 +1567,19 @@ static int32_t sortBlocksToExtSource(SSortHandle* pHandle, SArray* aBlk, SArray* int32_t minIdx = tMergeTreeGetChosenIndex(pTree); SSDataBlock* minBlk = taosArrayGetP(aBlk, minIdx); int32_t minRow = sup.aRowIdx[minIdx]; - SSDataBlock* incBlock = (pHandle->bSortByRowId) ? pHandle->pDataBlock : minBlk; - int32_t bufInc = getPageBufIncForRow(incBlock, minRow, pHandle->pDataBlock->info.rows); + + int32_t bufInc = 0; + if (pHandle->bSortByRowId) { + if (!pHandle->bSortPk) { + bufInc = getPageBufIncForRowIdSort(pHandle->pDataBlock, minRow, pHandle->pDataBlock->info.rows, NULL); + } else { // there may be varchar column exists, so we need to get the pk info, and then calculate the row length + SBlockOrderInfo* extRowsPkOrder = taosArrayGet(pHandle->aExtRowsOrders, 1); + SColumnInfoData* pPkCol = taosArrayGet(minBlk->pDataBlock, extRowsPkOrder->slotId); + bufInc = getPageBufIncForRowIdSort(pHandle->pDataBlock, minRow, pHandle->pDataBlock->info.rows, pPkCol); + } + } else { + bufInc = getPageBufIncForRow(minBlk, minRow, pHandle->pDataBlock->info.rows); + } if (blkPgSz <= pHandle->pageSize && blkPgSz + bufInc > pHandle->pageSize) { SColumnInfoData* tsCol = taosArrayGet(pHandle->pDataBlock->pDataBlock, pHandleBlockTsOrder->slotId); @@ -1552,7 +1597,8 @@ static int32_t sortBlocksToExtSource(SSortHandle* pHandle, SArray* aBlk, SArray* nMergedRows += pHandle->pDataBlock->info.rows; blockDataCleanup(pHandle->pDataBlock); blkPgSz = pgHeaderSz; - incBlock = (pHandle->bSortByRowId) ? pHandle->pDataBlock : minBlk; + + SSDataBlock* incBlock = (pHandle->bSortByRowId) ? pHandle->pDataBlock : minBlk; bufInc = getPageBufIncForRow(incBlock, minRow, 0); if ((pHandle->mergeLimit != -1) && (nMergedRows >= pHandle->mergeLimit)) { @@ -1566,10 +1612,10 @@ static int32_t sortBlocksToExtSource(SSortHandle* pHandle, SArray* aBlk, SArray* } blockDataEnsureCapacity(pHandle->pDataBlock, pHandle->pDataBlock->info.rows + 1); - if (!pHandle->bSortByRowId) { - appendOneRowToDataBlock(pHandle->pDataBlock, minBlk, &minRow); - } else { + if (pHandle->bSortByRowId) { appendToRowIndexDataBlock(pHandle, minBlk, &minRow); + } else { + appendOneRowToDataBlock(pHandle->pDataBlock, minBlk, &minRow); } blkPgSz += bufInc; From d19fd2fe234b8e48030b5fb2c0afa72bb5f6e1c1 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 8 May 2024 18:25:56 +0800 Subject: [PATCH 20/30] fix(query): calculate the correct dst row length. --- source/libs/executor/src/tsort.c | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/source/libs/executor/src/tsort.c b/source/libs/executor/src/tsort.c index 2ef40cd73899..906fe0626be4 100644 --- a/source/libs/executor/src/tsort.c +++ b/source/libs/executor/src/tsort.c @@ -1484,10 +1484,16 @@ static int32_t getPageBufIncForRowIdSort(SSDataBlock* pDstBlock, int32_t srcRowI int32_t numOfCols = blockDataGetNumOfCols(pDstBlock); if (pPkCol == NULL) { - ASSERT(!pDstBlock->info.hasVarCol); + ASSERT((numOfCols == 4) && (!pDstBlock->info.hasVarCol)); + size += numOfCols * ((dstRowIndex & 0x7) == 0 ? 1: 0); size += blockDataGetRowSize(pDstBlock); } else { + ASSERT(numOfCols == 5); + + size += (numOfCols - 1) * (((dstRowIndex & 0x7) == 0)? 1:0); + size += (8 + 4 + 4 + 4); // todo refactor later + if (IS_VAR_DATA_TYPE(pPkCol->info.type)) { if ((pPkCol->varmeta.offset[srcRowIndex] != -1) && (pPkCol->pData)) { char* p = colDataGetData(pPkCol, srcRowIndex); From fffb8177716f41981c034c3deef2a2b5d4f44976 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 8 May 2024 18:41:57 +0800 Subject: [PATCH 21/30] refactor: do some internal refactor. --- source/libs/executor/src/tsort.c | 50 ++++++++++++++++++++------------ 1 file changed, 32 insertions(+), 18 deletions(-) diff --git a/source/libs/executor/src/tsort.c b/source/libs/executor/src/tsort.c index 906fe0626be4..6f690b891120 100644 --- a/source/libs/executor/src/tsort.c +++ b/source/libs/executor/src/tsort.c @@ -1479,11 +1479,12 @@ static int32_t getPageBufIncForRow(SSDataBlock* pSrcBlock, int32_t srcRowIndex, return size; } -static int32_t getPageBufIncForRowIdSort(SSDataBlock* pDstBlock, int32_t srcRowIndex, int32_t dstRowIndex, SColumnInfoData* pPkCol) { +static int32_t getPageBufIncForRowIdSort(SSDataBlock* pDstBlock, int32_t srcRowIndex, int32_t dstRowIndex, + SColumnInfoData* pPkCol) { int32_t size = 0; int32_t numOfCols = blockDataGetNumOfCols(pDstBlock); - if (pPkCol == NULL) { + if (pPkCol == NULL) { // no var column ASSERT((numOfCols == 4) && (!pDstBlock->info.hasVarCol)); size += numOfCols * ((dstRowIndex & 0x7) == 0 ? 1: 0); @@ -1492,8 +1493,12 @@ static int32_t getPageBufIncForRowIdSort(SSDataBlock* pDstBlock, int32_t srcRowI ASSERT(numOfCols == 5); size += (numOfCols - 1) * (((dstRowIndex & 0x7) == 0)? 1:0); - size += (8 + 4 + 4 + 4); // todo refactor later + for(int32_t i = 0; i < numOfCols - 1; ++i) { + SColumnInfoData* pColInfo = TARRAY_GET_ELEM(pDstBlock->pDataBlock, i); + size += pColInfo->info.bytes; + } + // handle the pk column, the last column, may be the var char column if (IS_VAR_DATA_TYPE(pPkCol->info.type)) { if ((pPkCol->varmeta.offset[srcRowIndex] != -1) && (pPkCol->pData)) { char* p = colDataGetData(pPkCol, srcRowIndex); @@ -1512,6 +1517,27 @@ static int32_t getPageBufIncForRowIdSort(SSDataBlock* pDstBlock, int32_t srcRowI return size; } +static int32_t getBufIncForNewRow(SSortHandle* pHandle, int32_t dstRowIndex, SSDataBlock* pSrcBlock, + int32_t srcRowIndex) { + int32_t inc = 0; + + if (pHandle->bSortByRowId) { + SColumnInfoData* pPkCol = NULL; + + // there may be varchar column exists, so we need to get the pk info, and then calculate the row length + if (pHandle->bSortPk) { + SBlockOrderInfo* extRowsPkOrder = taosArrayGet(pHandle->aExtRowsOrders, 1); + pPkCol = taosArrayGet(pSrcBlock->pDataBlock, extRowsPkOrder->slotId); + } + + inc = getPageBufIncForRowIdSort(pHandle->pDataBlock, srcRowIndex, dstRowIndex, pPkCol); + } else { + inc = getPageBufIncForRow(pSrcBlock, srcRowIndex, dstRowIndex); + } + + return inc; +} + static int32_t sortBlocksToExtSource(SSortHandle* pHandle, SArray* aBlk, SArray* aExtSrc) { int32_t code = TSDB_CODE_SUCCESS; int32_t pgHeaderSz = sizeof(int32_t) + sizeof(int32_t) * taosArrayGetSize(pHandle->pDataBlock->pDataBlock); @@ -1574,19 +1600,7 @@ static int32_t sortBlocksToExtSource(SSortHandle* pHandle, SArray* aBlk, SArray* SSDataBlock* minBlk = taosArrayGetP(aBlk, minIdx); int32_t minRow = sup.aRowIdx[minIdx]; - int32_t bufInc = 0; - if (pHandle->bSortByRowId) { - if (!pHandle->bSortPk) { - bufInc = getPageBufIncForRowIdSort(pHandle->pDataBlock, minRow, pHandle->pDataBlock->info.rows, NULL); - } else { // there may be varchar column exists, so we need to get the pk info, and then calculate the row length - SBlockOrderInfo* extRowsPkOrder = taosArrayGet(pHandle->aExtRowsOrders, 1); - SColumnInfoData* pPkCol = taosArrayGet(minBlk->pDataBlock, extRowsPkOrder->slotId); - bufInc = getPageBufIncForRowIdSort(pHandle->pDataBlock, minRow, pHandle->pDataBlock->info.rows, pPkCol); - } - } else { - bufInc = getPageBufIncForRow(minBlk, minRow, pHandle->pDataBlock->info.rows); - } - + int32_t bufInc = getBufIncForNewRow(pHandle, pHandle->pDataBlock->info.rows, minBlk, minRow); if (blkPgSz <= pHandle->pageSize && blkPgSz + bufInc > pHandle->pageSize) { SColumnInfoData* tsCol = taosArrayGet(pHandle->pDataBlock->pDataBlock, pHandleBlockTsOrder->slotId); lastPageBufTs = ((int64_t*)tsCol->pData)[pHandle->pDataBlock->info.rows - 1]; @@ -1604,8 +1618,7 @@ static int32_t sortBlocksToExtSource(SSortHandle* pHandle, SArray* aBlk, SArray* blockDataCleanup(pHandle->pDataBlock); blkPgSz = pgHeaderSz; - SSDataBlock* incBlock = (pHandle->bSortByRowId) ? pHandle->pDataBlock : minBlk; - bufInc = getPageBufIncForRow(incBlock, minRow, 0); + bufInc = getBufIncForNewRow(pHandle, 0, minBlk, minRow); if ((pHandle->mergeLimit != -1) && (nMergedRows >= pHandle->mergeLimit)) { mergeLimitReached = true; @@ -1613,6 +1626,7 @@ static int32_t sortBlocksToExtSource(SSortHandle* pHandle, SArray* aBlk, SArray* (lastPageBufTs > pHandle->currMergeLimitTs && pHandleBlockTsOrder->order == TSDB_ORDER_DESC)) { pHandle->currMergeLimitTs = lastPageBufTs; } + break; } } From 2e414d37b8c63f6aa7f6336966f8fcf9ba3f7440 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 8 May 2024 19:04:50 +0800 Subject: [PATCH 22/30] refactor: do some internal refactor. --- source/libs/executor/src/tsort.c | 95 +++++++++++++++++++------------- 1 file changed, 56 insertions(+), 39 deletions(-) diff --git a/source/libs/executor/src/tsort.c b/source/libs/executor/src/tsort.c index 6f690b891120..21e9e5a70d68 100644 --- a/source/libs/executor/src/tsort.c +++ b/source/libs/executor/src/tsort.c @@ -1538,53 +1538,76 @@ static int32_t getBufIncForNewRow(SSortHandle* pHandle, int32_t dstRowIndex, SSD return inc; } +static int32_t initMergeSup(SBlkMergeSupport* pSup, SArray* pBlockList, int32_t tsOrder, int32_t tsSlotId, SBlockOrderInfo* pPkOrderInfo) { + memset(pSup, 0, sizeof(SBlkMergeSupport)); + + int32_t numOfBlocks = taosArrayGetSize(pBlockList); + + pSup->aRowIdx = taosMemoryCalloc(numOfBlocks, sizeof(int32_t)); + pSup->aTs = taosMemoryCalloc(numOfBlocks, sizeof(int64_t*)); + pSup->tsOrder = tsOrder; + pSup->aBlks = taosMemoryCalloc(numOfBlocks, sizeof(SSDataBlock*)); + + for (int32_t i = 0; i < numOfBlocks; ++i) { + SSDataBlock* pBlock = taosArrayGetP(pBlockList, i); + SColumnInfoData* col = taosArrayGet(pBlock->pDataBlock, tsSlotId); + pSup->aTs[i] = (int64_t*)col->pData; + pSup->aRowIdx[i] = 0; + pSup->aBlks[i] = pBlock; + } + + pSup->pPkOrder = pPkOrderInfo; + return TSDB_CODE_SUCCESS; +} + +static void cleanupMergeSup(SBlkMergeSupport* pSup) { + taosMemoryFree(pSup->aRowIdx); + taosMemoryFree(pSup->aTs); + taosMemoryFree(pSup->aBlks); +} + +static int32_t getTotalRows(SArray* pBlockList) { + int32_t totalRows = 0; + + for (int32_t i = 0; i < taosArrayGetSize(pBlockList); ++i) { + SSDataBlock* blk = taosArrayGetP(pBlockList, i); + totalRows += blk->info.rows; + } + + return totalRows; +} + static int32_t sortBlocksToExtSource(SSortHandle* pHandle, SArray* aBlk, SArray* aExtSrc) { int32_t code = TSDB_CODE_SUCCESS; - int32_t pgHeaderSz = sizeof(int32_t) + sizeof(int32_t) * taosArrayGetSize(pHandle->pDataBlock->pDataBlock); - int32_t rowCap = blockDataGetCapacityInRow(pHandle->pDataBlock, pHandle->pageSize, pgHeaderSz); + int32_t pageHeaderSize = sizeof(int32_t) + sizeof(int32_t) * blockDataGetNumOfCols(pHandle->pDataBlock); + int32_t rowCap = blockDataGetCapacityInRow(pHandle->pDataBlock, pHandle->pageSize, pageHeaderSize); blockDataEnsureCapacity(pHandle->pDataBlock, rowCap); blockDataCleanup(pHandle->pDataBlock); - int32_t numBlks = taosArrayGetSize(aBlk); + + SBlkMergeSupport sup = {0}; SBlockOrderInfo* pOrigBlockTsOrder = (!pHandle->bSortByRowId) ? taosArrayGet(pHandle->pSortInfo, 0) : taosArrayGet(pHandle->aExtRowsOrders, 0); SBlockOrderInfo* pHandleBlockTsOrder = taosArrayGet(pHandle->pSortInfo, 0); - SBlkMergeSupport sup = {0}; - sup.aRowIdx = taosMemoryCalloc(numBlks, sizeof(int32_t)); - sup.aTs = taosMemoryCalloc(numBlks, sizeof(int64_t*)); - sup.tsOrder = pOrigBlockTsOrder->order; - sup.aBlks = taosMemoryCalloc(numBlks, sizeof(SSDataBlock*)); - - for (int32_t i = 0; i < numBlks; ++i) { - SSDataBlock* blk = taosArrayGetP(aBlk, i); - SColumnInfoData* col = taosArrayGet(blk->pDataBlock, pOrigBlockTsOrder->slotId); - sup.aTs[i] = (int64_t*)col->pData; - sup.aRowIdx[i] = 0; - sup.aBlks[i] = blk; - } - + SBlockOrderInfo* pOrigBlockPkOrder = NULL; if (pHandle->bSortPk) { pOrigBlockPkOrder = (!pHandle->bSortByRowId) ? taosArrayGet(pHandle->pSortInfo, 1) : taosArrayGet(pHandle->aExtRowsOrders, 1); } - sup.pPkOrder = pOrigBlockPkOrder; - int32_t totalRows = 0; - for (int32_t i = 0; i < numBlks; ++i) { - SSDataBlock* blk = taosArrayGetP(aBlk, i); - totalRows += blk->info.rows; - } + initMergeSup(&sup, aBlk, pOrigBlockTsOrder->order, pOrigBlockTsOrder->slotId, pOrigBlockPkOrder); + + int32_t totalRows = getTotalRows(aBlk); SMultiwayMergeTreeInfo* pTree = NULL; __merge_compare_fn_t mergeCompareFn = (!pHandle->bSortPk) ? blockCompareTsFn : blockCompareTsPkFn; + code = tMergeTreeCreate(&pTree, taosArrayGetSize(aBlk), &sup, mergeCompareFn); if (TSDB_CODE_SUCCESS != code) { - taosMemoryFree(sup.aRowIdx); - taosMemoryFree(sup.aTs); - taosMemoryFree(sup.aBlks); + cleanupMergeSup(&sup); return code; } @@ -1592,7 +1615,7 @@ static int32_t sortBlocksToExtSource(SSortHandle* pHandle, SArray* aBlk, SArray* int32_t nRows = 0; int32_t nMergedRows = 0; bool mergeLimitReached = false; - size_t blkPgSz = pgHeaderSz; + size_t blkPgSz = pageHeaderSize; int64_t lastPageBufTs = (pHandleBlockTsOrder->order == TSDB_ORDER_ASC) ? INT64_MAX : INT64_MIN; while (nRows < totalRows) { @@ -1601,6 +1624,7 @@ static int32_t sortBlocksToExtSource(SSortHandle* pHandle, SArray* aBlk, SArray* int32_t minRow = sup.aRowIdx[minIdx]; int32_t bufInc = getBufIncForNewRow(pHandle, pHandle->pDataBlock->info.rows, minBlk, minRow); + if (blkPgSz <= pHandle->pageSize && blkPgSz + bufInc > pHandle->pageSize) { SColumnInfoData* tsCol = taosArrayGet(pHandle->pDataBlock->pDataBlock, pHandleBlockTsOrder->slotId); lastPageBufTs = ((int64_t*)tsCol->pData)[pHandle->pDataBlock->info.rows - 1]; @@ -1608,15 +1632,13 @@ static int32_t sortBlocksToExtSource(SSortHandle* pHandle, SArray* aBlk, SArray* if (code != TSDB_CODE_SUCCESS) { taosMemoryFree(pTree); taosArrayDestroy(aPgId); - taosMemoryFree(sup.aRowIdx); - taosMemoryFree(sup.aTs); - taosMemoryFree(sup.aBlks); + cleanupMergeSup(&sup); return code; } nMergedRows += pHandle->pDataBlock->info.rows; blockDataCleanup(pHandle->pDataBlock); - blkPgSz = pgHeaderSz; + blkPgSz = pageHeaderSize; bufInc = getBufIncForNewRow(pHandle, 0, minBlk, minRow); @@ -1639,7 +1661,7 @@ static int32_t sortBlocksToExtSource(SSortHandle* pHandle, SArray* aBlk, SArray* } blkPgSz += bufInc; - ASSERT(blkPgSz == blockDataGetSize(pHandle->pDataBlock) + pgHeaderSz); + ASSERT(blkPgSz == blockDataGetSize(pHandle->pDataBlock) + pageHeaderSize); ++nRows; @@ -1659,9 +1681,7 @@ static int32_t sortBlocksToExtSource(SSortHandle* pHandle, SArray* aBlk, SArray* if (code != TSDB_CODE_SUCCESS) { taosArrayDestroy(aPgId); taosMemoryFree(pTree); - taosMemoryFree(sup.aRowIdx); - taosMemoryFree(sup.aTs); - taosMemoryFree(sup.aBlks); + cleanupMergeSup(&sup); return code; } nMergedRows += pHandle->pDataBlock->info.rows; @@ -1679,10 +1699,7 @@ static int32_t sortBlocksToExtSource(SSortHandle* pHandle, SArray* aBlk, SArray* SSDataBlock* pMemSrcBlk = createOneDataBlock(pHandle->pDataBlock, false); doAddNewExternalMemSource(pHandle->pBuf, aExtSrc, pMemSrcBlk, &pHandle->sourceId, aPgId); - taosMemoryFree(sup.aRowIdx); - taosMemoryFree(sup.aTs); - taosMemoryFree(sup.aBlks); - + cleanupMergeSup(&sup); tMergeTreeDestroy(&pTree); return 0; From 8a9f53f51685268e0c0bca123c357f4358b0379d Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 8 May 2024 22:28:49 +0800 Subject: [PATCH 23/30] fix(query): set correct length for json value. --- source/libs/executor/src/tsort.c | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/source/libs/executor/src/tsort.c b/source/libs/executor/src/tsort.c index 21e9e5a70d68..daac98bbfc24 100644 --- a/source/libs/executor/src/tsort.c +++ b/source/libs/executor/src/tsort.c @@ -1462,7 +1462,12 @@ static int32_t getPageBufIncForRow(SSDataBlock* pSrcBlock, int32_t srcRowIndex, if (IS_VAR_DATA_TYPE(pColInfoData->info.type)) { if ((pColInfoData->varmeta.offset[srcRowIndex] != -1) && (pColInfoData->pData)) { char* p = colDataGetData(pColInfoData, srcRowIndex); - size += varDataTLen(p); + + if (pColInfoData->info.type == TSDB_DATA_TYPE_JSON) { + size += getJsonValueLen(p); + } else { + size += varDataTLen(p); + } } size += sizeof(pColInfoData->varmeta.offset[0]); From 3f70831cd612effc29d872c40bff35e143385fc4 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 9 May 2024 10:11:37 +0800 Subject: [PATCH 24/30] refactor: do some internal refactor. --- include/libs/stream/streammsg.h | 8 +- source/libs/stream/src/streamCheckStatus.c | 57 --- source/libs/stream/src/streamCheckpoint.c | 86 ---- source/libs/stream/src/streamDispatch.c | 138 ------- source/libs/stream/src/streamMeta.c | 110 ------ source/libs/stream/src/streamTask.c | 18 - source/libs/stream/src/streammsg.c | 432 +++++++++++++++++++++ 7 files changed, 437 insertions(+), 412 deletions(-) create mode 100644 source/libs/stream/src/streammsg.c diff --git a/include/libs/stream/streammsg.h b/include/libs/stream/streammsg.h index a4dead747551..543644228446 100644 --- a/include/libs/stream/streammsg.h +++ b/include/libs/stream/streammsg.h @@ -16,6 +16,8 @@ #ifndef TDENGINE_STREAMMSG_H #define TDENGINE_STREAMMSG_H +#include "tmsg.h" + #ifdef __cplusplus extern "C" { #endif @@ -45,6 +47,9 @@ typedef struct { int64_t expireTime; } SStreamCheckpointSourceReq; +int32_t tEncodeStreamCheckpointSourceReq(SEncoder* pEncoder, const SStreamCheckpointSourceReq* pReq); +int32_t tDecodeStreamCheckpointSourceReq(SDecoder* pDecoder, SStreamCheckpointSourceReq* pReq); + typedef struct { int64_t streamId; int64_t checkpointId; @@ -55,9 +60,6 @@ typedef struct { int8_t success; } SStreamCheckpointSourceRsp; -int32_t tEncodeStreamCheckpointSourceReq(SEncoder* pEncoder, const SStreamCheckpointSourceReq* pReq); -int32_t tDecodeStreamCheckpointSourceReq(SDecoder* pDecoder, SStreamCheckpointSourceReq* pReq); - int32_t tEncodeStreamCheckpointSourceRsp(SEncoder* pEncoder, const SStreamCheckpointSourceRsp* pRsp); typedef struct SStreamTaskNodeUpdateMsg { diff --git a/source/libs/stream/src/streamCheckStatus.c b/source/libs/stream/src/streamCheckStatus.c index 1401dba8206d..05cc67e069d5 100644 --- a/source/libs/stream/src/streamCheckStatus.c +++ b/source/libs/stream/src/streamCheckStatus.c @@ -720,60 +720,3 @@ void rspMonitorFn(void* param, void* tmrId) { taosArrayDestroy(pTimeoutList); } -int32_t tEncodeStreamTaskCheckReq(SEncoder* pEncoder, const SStreamTaskCheckReq* pReq) { - if (tStartEncode(pEncoder) < 0) return -1; - if (tEncodeI64(pEncoder, pReq->reqId) < 0) return -1; - if (tEncodeI64(pEncoder, pReq->streamId) < 0) return -1; - if (tEncodeI32(pEncoder, pReq->upstreamNodeId) < 0) return -1; - if (tEncodeI32(pEncoder, pReq->upstreamTaskId) < 0) return -1; - if (tEncodeI32(pEncoder, pReq->downstreamNodeId) < 0) return -1; - if (tEncodeI32(pEncoder, pReq->downstreamTaskId) < 0) return -1; - if (tEncodeI32(pEncoder, pReq->childId) < 0) return -1; - if (tEncodeI64(pEncoder, pReq->stage) < 0) return -1; - tEndEncode(pEncoder); - return pEncoder->pos; -} - -int32_t tDecodeStreamTaskCheckReq(SDecoder* pDecoder, SStreamTaskCheckReq* pReq) { - if (tStartDecode(pDecoder) < 0) return -1; - if (tDecodeI64(pDecoder, &pReq->reqId) < 0) return -1; - if (tDecodeI64(pDecoder, &pReq->streamId) < 0) return -1; - if (tDecodeI32(pDecoder, &pReq->upstreamNodeId) < 0) return -1; - if (tDecodeI32(pDecoder, &pReq->upstreamTaskId) < 0) return -1; - if (tDecodeI32(pDecoder, &pReq->downstreamNodeId) < 0) return -1; - if (tDecodeI32(pDecoder, &pReq->downstreamTaskId) < 0) return -1; - if (tDecodeI32(pDecoder, &pReq->childId) < 0) return -1; - if (tDecodeI64(pDecoder, &pReq->stage) < 0) return -1; - tEndDecode(pDecoder); - return 0; -} - -int32_t tEncodeStreamTaskCheckRsp(SEncoder* pEncoder, const SStreamTaskCheckRsp* pRsp) { - if (tStartEncode(pEncoder) < 0) return -1; - if (tEncodeI64(pEncoder, pRsp->reqId) < 0) return -1; - if (tEncodeI64(pEncoder, pRsp->streamId) < 0) return -1; - if (tEncodeI32(pEncoder, pRsp->upstreamNodeId) < 0) return -1; - if (tEncodeI32(pEncoder, pRsp->upstreamTaskId) < 0) return -1; - if (tEncodeI32(pEncoder, pRsp->downstreamNodeId) < 0) return -1; - if (tEncodeI32(pEncoder, pRsp->downstreamTaskId) < 0) return -1; - if (tEncodeI32(pEncoder, pRsp->childId) < 0) return -1; - if (tEncodeI64(pEncoder, pRsp->oldStage) < 0) return -1; - if (tEncodeI8(pEncoder, pRsp->status) < 0) return -1; - tEndEncode(pEncoder); - return pEncoder->pos; -} - -int32_t tDecodeStreamTaskCheckRsp(SDecoder* pDecoder, SStreamTaskCheckRsp* pRsp) { - if (tStartDecode(pDecoder) < 0) return -1; - if (tDecodeI64(pDecoder, &pRsp->reqId) < 0) return -1; - if (tDecodeI64(pDecoder, &pRsp->streamId) < 0) return -1; - if (tDecodeI32(pDecoder, &pRsp->upstreamNodeId) < 0) return -1; - if (tDecodeI32(pDecoder, &pRsp->upstreamTaskId) < 0) return -1; - if (tDecodeI32(pDecoder, &pRsp->downstreamNodeId) < 0) return -1; - if (tDecodeI32(pDecoder, &pRsp->downstreamTaskId) < 0) return -1; - if (tDecodeI32(pDecoder, &pRsp->childId) < 0) return -1; - if (tDecodeI64(pDecoder, &pRsp->oldStage) < 0) return -1; - if (tDecodeI8(pDecoder, &pRsp->status) < 0) return -1; - tEndDecode(pDecoder); - return 0; -} diff --git a/source/libs/stream/src/streamCheckpoint.c b/source/libs/stream/src/streamCheckpoint.c index 1519382fe454..5a4e3a543952 100644 --- a/source/libs/stream/src/streamCheckpoint.c +++ b/source/libs/stream/src/streamCheckpoint.c @@ -34,92 +34,6 @@ static int32_t streamTaskBackupCheckpoint(const char* id, const char* path); static int32_t deleteCheckpoint(const char* id); static int32_t downloadCheckpointByNameS3(const char* id, const char* fname, const char* dstName); -int32_t tEncodeStreamCheckpointSourceReq(SEncoder* pEncoder, const SStreamCheckpointSourceReq* pReq) { - if (tStartEncode(pEncoder) < 0) return -1; - if (tEncodeI64(pEncoder, pReq->streamId) < 0) return -1; - if (tEncodeI64(pEncoder, pReq->checkpointId) < 0) return -1; - if (tEncodeI32(pEncoder, pReq->taskId) < 0) return -1; - if (tEncodeI32(pEncoder, pReq->nodeId) < 0) return -1; - if (tEncodeSEpSet(pEncoder, &pReq->mgmtEps) < 0) return -1; - if (tEncodeI32(pEncoder, pReq->mnodeId) < 0) return -1; - if (tEncodeI64(pEncoder, pReq->expireTime) < 0) return -1; - if (tEncodeI32(pEncoder, pReq->transId) < 0) return -1; - if (tEncodeI8(pEncoder, pReq->mndTrigger) < 0) return -1; - tEndEncode(pEncoder); - return pEncoder->pos; -} - -int32_t tDecodeStreamCheckpointSourceReq(SDecoder* pDecoder, SStreamCheckpointSourceReq* pReq) { - if (tStartDecode(pDecoder) < 0) return -1; - if (tDecodeI64(pDecoder, &pReq->streamId) < 0) return -1; - if (tDecodeI64(pDecoder, &pReq->checkpointId) < 0) return -1; - if (tDecodeI32(pDecoder, &pReq->taskId) < 0) return -1; - if (tDecodeI32(pDecoder, &pReq->nodeId) < 0) return -1; - if (tDecodeSEpSet(pDecoder, &pReq->mgmtEps) < 0) return -1; - if (tDecodeI32(pDecoder, &pReq->mnodeId) < 0) return -1; - if (tDecodeI64(pDecoder, &pReq->expireTime) < 0) return -1; - if (tDecodeI32(pDecoder, &pReq->transId) < 0) return -1; - if (tDecodeI8(pDecoder, &pReq->mndTrigger) < 0) return -1; - tEndDecode(pDecoder); - return 0; -} - -int32_t tEncodeStreamCheckpointSourceRsp(SEncoder* pEncoder, const SStreamCheckpointSourceRsp* pRsp) { - if (tStartEncode(pEncoder) < 0) return -1; - if (tEncodeI64(pEncoder, pRsp->streamId) < 0) return -1; - if (tEncodeI64(pEncoder, pRsp->checkpointId) < 0) return -1; - if (tEncodeI32(pEncoder, pRsp->taskId) < 0) return -1; - if (tEncodeI32(pEncoder, pRsp->nodeId) < 0) return -1; - if (tEncodeI64(pEncoder, pRsp->expireTime) < 0) return -1; - if (tEncodeI8(pEncoder, pRsp->success) < 0) return -1; - tEndEncode(pEncoder); - return pEncoder->pos; -} - -int32_t tEncodeStreamCheckpointReadyMsg(SEncoder* pEncoder, const SStreamCheckpointReadyMsg* pReq) { - if (tStartEncode(pEncoder) < 0) return -1; - if (tEncodeI64(pEncoder, pReq->streamId) < 0) return -1; - if (tEncodeI64(pEncoder, pReq->checkpointId) < 0) return -1; - if (tEncodeI32(pEncoder, pReq->downstreamTaskId) < 0) return -1; - if (tEncodeI32(pEncoder, pReq->downstreamNodeId) < 0) return -1; - if (tEncodeI32(pEncoder, pReq->upstreamTaskId) < 0) return -1; - if (tEncodeI32(pEncoder, pReq->upstreamNodeId) < 0) return -1; - if (tEncodeI32(pEncoder, pReq->childId) < 0) return -1; - tEndEncode(pEncoder); - return pEncoder->pos; -} - -int32_t tDecodeStreamCheckpointReadyMsg(SDecoder* pDecoder, SStreamCheckpointReadyMsg* pRsp) { - if (tStartDecode(pDecoder) < 0) return -1; - if (tDecodeI64(pDecoder, &pRsp->streamId) < 0) return -1; - if (tDecodeI64(pDecoder, &pRsp->checkpointId) < 0) return -1; - if (tDecodeI32(pDecoder, &pRsp->downstreamTaskId) < 0) return -1; - if (tDecodeI32(pDecoder, &pRsp->downstreamNodeId) < 0) return -1; - if (tDecodeI32(pDecoder, &pRsp->upstreamTaskId) < 0) return -1; - if (tDecodeI32(pDecoder, &pRsp->upstreamNodeId) < 0) return -1; - if (tDecodeI32(pDecoder, &pRsp->childId) < 0) return -1; - tEndDecode(pDecoder); - return 0; -} - -int32_t tEncodeStreamTaskCheckpointReq(SEncoder* pEncoder, const SStreamTaskCheckpointReq* pReq) { - if (tStartEncode(pEncoder) < 0) return -1; - if (tEncodeI64(pEncoder, pReq->streamId) < 0) return -1; - if (tEncodeI32(pEncoder, pReq->taskId) < 0) return -1; - if (tEncodeI32(pEncoder, pReq->nodeId) < 0) return -1; - tEndEncode(pEncoder); - return 0; -} - -int32_t tDecodeStreamTaskCheckpointReq(SDecoder* pDecoder, SStreamTaskCheckpointReq* pReq) { - if (tStartDecode(pDecoder) < 0) return -1; - if (tDecodeI64(pDecoder, &pReq->streamId) < 0) return -1; - if (tDecodeI32(pDecoder, &pReq->taskId) < 0) return -1; - if (tDecodeI32(pDecoder, &pReq->nodeId) < 0) return -1; - tEndDecode(pDecoder); - return 0; -} - static int32_t streamAlignCheckpoint(SStreamTask* pTask) { int32_t num = taosArrayGetSize(pTask->upstreamInfo.pList); int64_t old = atomic_val_compare_exchange_32(&pTask->chkInfo.downstreamAlignNum, 0, num); diff --git a/source/libs/stream/src/streamDispatch.c b/source/libs/stream/src/streamDispatch.c index 2373da024b73..3cd5434c20ac 100644 --- a/source/libs/stream/src/streamDispatch.c +++ b/source/libs/stream/src/streamDispatch.c @@ -43,67 +43,6 @@ void initRpcMsg(SRpcMsg* pMsg, int32_t msgType, void* pCont, int32_t contLen) { pMsg->contLen = contLen; } -int32_t tEncodeStreamDispatchReq(SEncoder* pEncoder, const SStreamDispatchReq* pReq) { - if (tStartEncode(pEncoder) < 0) return -1; - if (tEncodeI64(pEncoder, pReq->stage) < 0) return -1; - if (tEncodeI32(pEncoder, pReq->msgId) < 0) return -1; - if (tEncodeI32(pEncoder, pReq->srcVgId) < 0) return -1; - if (tEncodeI32(pEncoder, pReq->type) < 0) return -1; - if (tEncodeI64(pEncoder, pReq->streamId) < 0) return -1; - if (tEncodeI32(pEncoder, pReq->taskId) < 0) return -1; - if (tEncodeI32(pEncoder, pReq->type) < 0) return -1; - if (tEncodeI32(pEncoder, pReq->upstreamTaskId) < 0) return -1; - if (tEncodeI32(pEncoder, pReq->upstreamChildId) < 0) return -1; - if (tEncodeI32(pEncoder, pReq->upstreamNodeId) < 0) return -1; - if (tEncodeI32(pEncoder, pReq->upstreamRelTaskId) < 0) return -1; - if (tEncodeI32(pEncoder, pReq->blockNum) < 0) return -1; - if (tEncodeI64(pEncoder, pReq->totalLen) < 0) return -1; - ASSERT(taosArrayGetSize(pReq->data) == pReq->blockNum); - ASSERT(taosArrayGetSize(pReq->dataLen) == pReq->blockNum); - for (int32_t i = 0; i < pReq->blockNum; i++) { - int32_t len = *(int32_t*)taosArrayGet(pReq->dataLen, i); - void* data = taosArrayGetP(pReq->data, i); - if (tEncodeI32(pEncoder, len) < 0) return -1; - if (tEncodeBinary(pEncoder, data, len) < 0) return -1; - } - tEndEncode(pEncoder); - return pEncoder->pos; -} - -int32_t tDecodeStreamDispatchReq(SDecoder* pDecoder, SStreamDispatchReq* pReq) { - if (tStartDecode(pDecoder) < 0) return -1; - if (tDecodeI64(pDecoder, &pReq->stage) < 0) return -1; - if (tDecodeI32(pDecoder, &pReq->msgId) < 0) return -1; - if (tDecodeI32(pDecoder, &pReq->srcVgId) < 0) return -1; - if (tDecodeI32(pDecoder, &pReq->type) < 0) return -1; - if (tDecodeI64(pDecoder, &pReq->streamId) < 0) return -1; - if (tDecodeI32(pDecoder, &pReq->taskId) < 0) return -1; - if (tDecodeI32(pDecoder, &pReq->type) < 0) return -1; - if (tDecodeI32(pDecoder, &pReq->upstreamTaskId) < 0) return -1; - if (tDecodeI32(pDecoder, &pReq->upstreamChildId) < 0) return -1; - if (tDecodeI32(pDecoder, &pReq->upstreamNodeId) < 0) return -1; - if (tDecodeI32(pDecoder, &pReq->upstreamRelTaskId) < 0) return -1; - if (tDecodeI32(pDecoder, &pReq->blockNum) < 0) return -1; - if (tDecodeI64(pDecoder, &pReq->totalLen) < 0) return -1; - - ASSERT(pReq->blockNum > 0); - pReq->data = taosArrayInit(pReq->blockNum, sizeof(void*)); - pReq->dataLen = taosArrayInit(pReq->blockNum, sizeof(int32_t)); - for (int32_t i = 0; i < pReq->blockNum; i++) { - int32_t len1; - uint64_t len2; - void* data; - if (tDecodeI32(pDecoder, &len1) < 0) return -1; - if (tDecodeBinaryAlloc(pDecoder, &data, &len2) < 0) return -1; - ASSERT(len1 == len2); - taosArrayPush(pReq->dataLen, &len1); - taosArrayPush(pReq->data, &data); - } - - tEndDecode(pDecoder); - return 0; -} - static int32_t tInitStreamDispatchReq(SStreamDispatchReq* pReq, const SStreamTask* pTask, int32_t vgId, int32_t numOfBlocks, int64_t dstTaskId, int32_t type) { pReq->streamId = pTask->id.streamId; @@ -129,41 +68,6 @@ static int32_t tInitStreamDispatchReq(SStreamDispatchReq* pReq, const SStreamTas return TSDB_CODE_SUCCESS; } -void tCleanupStreamDispatchReq(SStreamDispatchReq* pReq) { - taosArrayDestroyP(pReq->data, taosMemoryFree); - taosArrayDestroy(pReq->dataLen); -} - -int32_t tEncodeStreamRetrieveReq(SEncoder* pEncoder, const SStreamRetrieveReq* pReq) { - if (tStartEncode(pEncoder) < 0) return -1; - if (tEncodeI64(pEncoder, pReq->streamId) < 0) return -1; - if (tEncodeI64(pEncoder, pReq->reqId) < 0) return -1; - if (tEncodeI32(pEncoder, pReq->dstNodeId) < 0) return -1; - if (tEncodeI32(pEncoder, pReq->dstTaskId) < 0) return -1; - if (tEncodeI32(pEncoder, pReq->srcNodeId) < 0) return -1; - if (tEncodeI32(pEncoder, pReq->srcTaskId) < 0) return -1; - if (tEncodeBinary(pEncoder, (const uint8_t*)pReq->pRetrieve, pReq->retrieveLen) < 0) return -1; - tEndEncode(pEncoder); - return pEncoder->pos; -} - -int32_t tDecodeStreamRetrieveReq(SDecoder* pDecoder, SStreamRetrieveReq* pReq) { - if (tStartDecode(pDecoder) < 0) return -1; - if (tDecodeI64(pDecoder, &pReq->streamId) < 0) return -1; - if (tDecodeI64(pDecoder, &pReq->reqId) < 0) return -1; - if (tDecodeI32(pDecoder, &pReq->dstNodeId) < 0) return -1; - if (tDecodeI32(pDecoder, &pReq->dstTaskId) < 0) return -1; - if (tDecodeI32(pDecoder, &pReq->srcNodeId) < 0) return -1; - if (tDecodeI32(pDecoder, &pReq->srcTaskId) < 0) return -1; - uint64_t len = 0; - if (tDecodeBinaryAlloc(pDecoder, (void**)&pReq->pRetrieve, &len) < 0) return -1; - pReq->retrieveLen = (int32_t)len; - tEndDecode(pDecoder); - return 0; -} - -void tCleanupStreamRetrieveReq(SStreamRetrieveReq* pReq) { taosMemoryFree(pReq->pRetrieve); } - void streamTaskSendRetrieveRsp(SStreamRetrieveReq *pReq, SRpcMsg* pRsp){ void* buf = rpcMallocCont(sizeof(SMsgHead) + sizeof(SStreamRetrieveRsp)); ((SMsgHead*)buf)->vgId = htonl(pReq->srcNodeId); @@ -1263,45 +1167,3 @@ int32_t streamProcessDispatchMsg(SStreamTask* pTask, SStreamDispatchReq* pReq, S return 0; } -int32_t tEncodeStreamTaskUpdateMsg(SEncoder* pEncoder, const SStreamTaskNodeUpdateMsg* pMsg) { - if (tStartEncode(pEncoder) < 0) return -1; - if (tEncodeI64(pEncoder, pMsg->streamId) < 0) return -1; - if (tEncodeI32(pEncoder, pMsg->taskId) < 0) return -1; - - int32_t size = taosArrayGetSize(pMsg->pNodeList); - if (tEncodeI32(pEncoder, size) < 0) return -1; - - for (int32_t i = 0; i < size; ++i) { - SNodeUpdateInfo* pInfo = taosArrayGet(pMsg->pNodeList, i); - if (tEncodeI32(pEncoder, pInfo->nodeId) < 0) return -1; - 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; -} - -int32_t tDecodeStreamTaskUpdateMsg(SDecoder* pDecoder, SStreamTaskNodeUpdateMsg* pMsg) { - if (tStartDecode(pDecoder) < 0) return -1; - if (tDecodeI64(pDecoder, &pMsg->streamId) < 0) return -1; - if (tDecodeI32(pDecoder, &pMsg->taskId) < 0) return -1; - - int32_t size = 0; - if (tDecodeI32(pDecoder, &size) < 0) return -1; - pMsg->pNodeList = taosArrayInit(size, sizeof(SNodeUpdateInfo)); - for (int32_t i = 0; i < size; ++i) { - SNodeUpdateInfo info = {0}; - if (tDecodeI32(pDecoder, &info.nodeId) < 0) return -1; - if (tDecodeSEpSet(pDecoder, &info.prevEp) < 0) return -1; - if (tDecodeSEpSet(pDecoder, &info.newEp) < 0) return -1; - taosArrayPush(pMsg->pNodeList, &info); - } - - if (tDecodeI32(pDecoder, &pMsg->transId) < 0) return -1; - - tEndDecode(pDecoder); - return 0; -} diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index a373ec28c0ab..bb2c30f420ad 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -945,102 +945,6 @@ void streamMetaLoadAllTasks(SStreamMeta* pMeta) { taosArrayDestroy(pRecycleList); } -int32_t tEncodeStreamHbMsg(SEncoder* pEncoder, const SStreamHbMsg* pReq) { - if (tStartEncode(pEncoder) < 0) return -1; - if (tEncodeI32(pEncoder, pReq->vgId) < 0) return -1; - if (tEncodeI32(pEncoder, pReq->numOfTasks) < 0) return -1; - - for (int32_t i = 0; i < pReq->numOfTasks; ++i) { - STaskStatusEntry* ps = taosArrayGet(pReq->pTaskStatus, i); - if (tEncodeI64(pEncoder, ps->id.streamId) < 0) return -1; - if (tEncodeI32(pEncoder, ps->id.taskId) < 0) return -1; - if (tEncodeI32(pEncoder, ps->status) < 0) return -1; - if (tEncodeI64(pEncoder, ps->stage) < 0) return -1; - if (tEncodeI32(pEncoder, ps->nodeId) < 0) return -1; - if (tEncodeDouble(pEncoder, ps->inputQUsed) < 0) return -1; - if (tEncodeDouble(pEncoder, ps->inputRate) < 0) return -1; - if (tEncodeDouble(pEncoder, ps->sinkQuota) < 0) return -1; - if (tEncodeDouble(pEncoder, ps->sinkDataSize) < 0) return -1; - if (tEncodeI64(pEncoder, ps->processedVer) < 0) return -1; - if (tEncodeI64(pEncoder, ps->verRange.minVer) < 0) return -1; - if (tEncodeI64(pEncoder, ps->verRange.maxVer) < 0) return -1; - if (tEncodeI64(pEncoder, ps->checkpointInfo.activeId) < 0) return -1; - if (tEncodeI8(pEncoder, ps->checkpointInfo.failed) < 0) return -1; - if (tEncodeI32(pEncoder, ps->checkpointInfo.activeTransId) < 0) return -1; - if (tEncodeI64(pEncoder, ps->checkpointInfo.latestId) < 0) return -1; - if (tEncodeI64(pEncoder, ps->checkpointInfo.latestVer) < 0) return -1; - if (tEncodeI64(pEncoder, ps->checkpointInfo.latestTime) < 0) return -1; - if (tEncodeI64(pEncoder, ps->startTime) < 0) return -1; - if (tEncodeI64(pEncoder, ps->startCheckpointId) < 0) return -1; - if (tEncodeI64(pEncoder, ps->startCheckpointVer) < 0) return -1; - if (tEncodeI64(pEncoder, ps->hTaskId) < 0) return -1; - } - - int32_t numOfVgs = taosArrayGetSize(pReq->pUpdateNodes); - if (tEncodeI32(pEncoder, numOfVgs) < 0) return -1; - - for (int j = 0; j < numOfVgs; ++j) { - int32_t* pVgId = taosArrayGet(pReq->pUpdateNodes, j); - if (tEncodeI32(pEncoder, *pVgId) < 0) return -1; - } - - tEndEncode(pEncoder); - return pEncoder->pos; -} - -int32_t tDecodeStreamHbMsg(SDecoder* pDecoder, SStreamHbMsg* pReq) { - if (tStartDecode(pDecoder) < 0) return -1; - if (tDecodeI32(pDecoder, &pReq->vgId) < 0) return -1; - if (tDecodeI32(pDecoder, &pReq->numOfTasks) < 0) return -1; - - pReq->pTaskStatus = taosArrayInit(pReq->numOfTasks, sizeof(STaskStatusEntry)); - for (int32_t i = 0; i < pReq->numOfTasks; ++i) { - int32_t taskId = 0; - STaskStatusEntry entry = {0}; - - if (tDecodeI64(pDecoder, &entry.id.streamId) < 0) return -1; - if (tDecodeI32(pDecoder, &taskId) < 0) return -1; - if (tDecodeI32(pDecoder, &entry.status) < 0) return -1; - if (tDecodeI64(pDecoder, &entry.stage) < 0) return -1; - if (tDecodeI32(pDecoder, &entry.nodeId) < 0) return -1; - if (tDecodeDouble(pDecoder, &entry.inputQUsed) < 0) return -1; - if (tDecodeDouble(pDecoder, &entry.inputRate) < 0) return -1; - if (tDecodeDouble(pDecoder, &entry.sinkQuota) < 0) return -1; - if (tDecodeDouble(pDecoder, &entry.sinkDataSize) < 0) return -1; - if (tDecodeI64(pDecoder, &entry.processedVer) < 0) return -1; - if (tDecodeI64(pDecoder, &entry.verRange.minVer) < 0) return -1; - if (tDecodeI64(pDecoder, &entry.verRange.maxVer) < 0) return -1; - if (tDecodeI64(pDecoder, &entry.checkpointInfo.activeId) < 0) return -1; - if (tDecodeI8(pDecoder, &entry.checkpointInfo.failed) < 0) return -1; - if (tDecodeI32(pDecoder, &entry.checkpointInfo.activeTransId) < 0) return -1; - - if (tDecodeI64(pDecoder, &entry.checkpointInfo.latestId) < 0) return -1; - if (tDecodeI64(pDecoder, &entry.checkpointInfo.latestVer) < 0) return -1; - if (tDecodeI64(pDecoder, &entry.checkpointInfo.latestTime) < 0) return -1; - if (tDecodeI64(pDecoder, &entry.startTime) < 0) return -1; - if (tDecodeI64(pDecoder, &entry.startCheckpointId) < 0) return -1; - if (tDecodeI64(pDecoder, &entry.startCheckpointVer) < 0) return -1; - if (tDecodeI64(pDecoder, &entry.hTaskId) < 0) return -1; - - entry.id.taskId = taskId; - taosArrayPush(pReq->pTaskStatus, &entry); - } - - int32_t numOfVgs = 0; - if (tDecodeI32(pDecoder, &numOfVgs) < 0) return -1; - - pReq->pUpdateNodes = taosArrayInit(numOfVgs, sizeof(int32_t)); - - for (int j = 0; j < numOfVgs; ++j) { - int32_t vgId = 0; - if (tDecodeI32(pDecoder, &vgId) < 0) return -1; - taosArrayPush(pReq->pUpdateNodes, &vgId); - } - - tEndDecode(pDecoder); - return 0; -} - static bool waitForEnoughDuration(SMetaHbInfo* pInfo) { if ((++pInfo->tickCounter) >= META_HB_SEND_IDLE_COUNTER) { // reset the counter pInfo->tickCounter = 0; @@ -1049,20 +953,6 @@ static bool waitForEnoughDuration(SMetaHbInfo* pInfo) { return false; } -void tCleanupStreamHbMsg(SStreamHbMsg* pMsg) { - if (pMsg == NULL) { - return; - } - - if (pMsg->pUpdateNodes != NULL) { - taosArrayDestroy(pMsg->pUpdateNodes); - } - - if (pMsg->pTaskStatus != NULL) { - taosArrayDestroy(pMsg->pTaskStatus); - } -} - static bool existInHbMsg(SStreamHbMsg* pMsg, SDownstreamTaskEpset* pTaskEpset) { int32_t numOfExisted = taosArrayGetSize(pMsg->pUpdateNodes); for (int k = 0; k < numOfExisted; ++k) { diff --git a/source/libs/stream/src/streamTask.c b/source/libs/stream/src/streamTask.c index 7f5ea52f5843..a902aae9f494 100644 --- a/source/libs/stream/src/streamTask.c +++ b/source/libs/stream/src/streamTask.c @@ -137,24 +137,6 @@ SStreamTask* tNewStreamTask(int64_t streamId, int8_t taskLevel, SEpSet* pEpset, return pTask; } -int32_t tEncodeStreamEpInfo(SEncoder* pEncoder, const SStreamChildEpInfo* pInfo) { - if (tEncodeI32(pEncoder, pInfo->taskId) < 0) return -1; - if (tEncodeI32(pEncoder, pInfo->nodeId) < 0) return -1; - if (tEncodeI32(pEncoder, pInfo->childId) < 0) return -1; - if (tEncodeSEpSet(pEncoder, &pInfo->epSet) < 0) return -1; - if (tEncodeI64(pEncoder, pInfo->stage) < 0) return -1; - return 0; -} - -int32_t tDecodeStreamEpInfo(SDecoder* pDecoder, SStreamChildEpInfo* pInfo) { - if (tDecodeI32(pDecoder, &pInfo->taskId) < 0) return -1; - if (tDecodeI32(pDecoder, &pInfo->nodeId) < 0) return -1; - if (tDecodeI32(pDecoder, &pInfo->childId) < 0) return -1; - if (tDecodeSEpSet(pDecoder, &pInfo->epSet) < 0) return -1; - if (tDecodeI64(pDecoder, &pInfo->stage) < 0) return -1; - return 0; -} - int32_t tEncodeStreamTask(SEncoder* pEncoder, const SStreamTask* pTask) { if (tStartEncode(pEncoder) < 0) return -1; if (tEncodeI64(pEncoder, pTask->ver) < 0) return -1; diff --git a/source/libs/stream/src/streammsg.c b/source/libs/stream/src/streammsg.c new file mode 100644 index 000000000000..b37299f54fd3 --- /dev/null +++ b/source/libs/stream/src/streammsg.c @@ -0,0 +1,432 @@ +/* + * 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 "os.h" +#include "streammsg.h" +#include "tstream.h" + +int32_t tEncodeStreamEpInfo(SEncoder* pEncoder, const SStreamChildEpInfo* pInfo) { + if (tEncodeI32(pEncoder, pInfo->taskId) < 0) return -1; + if (tEncodeI32(pEncoder, pInfo->nodeId) < 0) return -1; + if (tEncodeI32(pEncoder, pInfo->childId) < 0) return -1; + if (tEncodeSEpSet(pEncoder, &pInfo->epSet) < 0) return -1; + if (tEncodeI64(pEncoder, pInfo->stage) < 0) return -1; + return 0; +} + +int32_t tDecodeStreamEpInfo(SDecoder* pDecoder, SStreamChildEpInfo* pInfo) { + if (tDecodeI32(pDecoder, &pInfo->taskId) < 0) return -1; + if (tDecodeI32(pDecoder, &pInfo->nodeId) < 0) return -1; + if (tDecodeI32(pDecoder, &pInfo->childId) < 0) return -1; + if (tDecodeSEpSet(pDecoder, &pInfo->epSet) < 0) return -1; + if (tDecodeI64(pDecoder, &pInfo->stage) < 0) return -1; + return 0; +} + +int32_t tEncodeStreamCheckpointSourceReq(SEncoder* pEncoder, const SStreamCheckpointSourceReq* pReq) { + if (tStartEncode(pEncoder) < 0) return -1; + if (tEncodeI64(pEncoder, pReq->streamId) < 0) return -1; + if (tEncodeI64(pEncoder, pReq->checkpointId) < 0) return -1; + if (tEncodeI32(pEncoder, pReq->taskId) < 0) return -1; + if (tEncodeI32(pEncoder, pReq->nodeId) < 0) return -1; + if (tEncodeSEpSet(pEncoder, &pReq->mgmtEps) < 0) return -1; + if (tEncodeI32(pEncoder, pReq->mnodeId) < 0) return -1; + if (tEncodeI64(pEncoder, pReq->expireTime) < 0) return -1; + if (tEncodeI32(pEncoder, pReq->transId) < 0) return -1; + if (tEncodeI8(pEncoder, pReq->mndTrigger) < 0) return -1; + tEndEncode(pEncoder); + return pEncoder->pos; +} + +int32_t tDecodeStreamCheckpointSourceReq(SDecoder* pDecoder, SStreamCheckpointSourceReq* pReq) { + if (tStartDecode(pDecoder) < 0) return -1; + if (tDecodeI64(pDecoder, &pReq->streamId) < 0) return -1; + if (tDecodeI64(pDecoder, &pReq->checkpointId) < 0) return -1; + if (tDecodeI32(pDecoder, &pReq->taskId) < 0) return -1; + if (tDecodeI32(pDecoder, &pReq->nodeId) < 0) return -1; + if (tDecodeSEpSet(pDecoder, &pReq->mgmtEps) < 0) return -1; + if (tDecodeI32(pDecoder, &pReq->mnodeId) < 0) return -1; + if (tDecodeI64(pDecoder, &pReq->expireTime) < 0) return -1; + if (tDecodeI32(pDecoder, &pReq->transId) < 0) return -1; + if (tDecodeI8(pDecoder, &pReq->mndTrigger) < 0) return -1; + tEndDecode(pDecoder); + return 0; +} + +int32_t tEncodeStreamCheckpointSourceRsp(SEncoder* pEncoder, const SStreamCheckpointSourceRsp* pRsp) { + if (tStartEncode(pEncoder) < 0) return -1; + if (tEncodeI64(pEncoder, pRsp->streamId) < 0) return -1; + if (tEncodeI64(pEncoder, pRsp->checkpointId) < 0) return -1; + if (tEncodeI32(pEncoder, pRsp->taskId) < 0) return -1; + if (tEncodeI32(pEncoder, pRsp->nodeId) < 0) return -1; + if (tEncodeI64(pEncoder, pRsp->expireTime) < 0) return -1; + if (tEncodeI8(pEncoder, pRsp->success) < 0) return -1; + tEndEncode(pEncoder); + return pEncoder->pos; +} + + +int32_t tEncodeStreamTaskUpdateMsg(SEncoder* pEncoder, const SStreamTaskNodeUpdateMsg* pMsg) { + if (tStartEncode(pEncoder) < 0) return -1; + if (tEncodeI64(pEncoder, pMsg->streamId) < 0) return -1; + if (tEncodeI32(pEncoder, pMsg->taskId) < 0) return -1; + + int32_t size = taosArrayGetSize(pMsg->pNodeList); + if (tEncodeI32(pEncoder, size) < 0) return -1; + + for (int32_t i = 0; i < size; ++i) { + SNodeUpdateInfo* pInfo = taosArrayGet(pMsg->pNodeList, i); + if (tEncodeI32(pEncoder, pInfo->nodeId) < 0) return -1; + 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; +} + + +int32_t tDecodeStreamTaskUpdateMsg(SDecoder* pDecoder, SStreamTaskNodeUpdateMsg* pMsg) { + if (tStartDecode(pDecoder) < 0) return -1; + if (tDecodeI64(pDecoder, &pMsg->streamId) < 0) return -1; + if (tDecodeI32(pDecoder, &pMsg->taskId) < 0) return -1; + + int32_t size = 0; + if (tDecodeI32(pDecoder, &size) < 0) return -1; + pMsg->pNodeList = taosArrayInit(size, sizeof(SNodeUpdateInfo)); + for (int32_t i = 0; i < size; ++i) { + SNodeUpdateInfo info = {0}; + if (tDecodeI32(pDecoder, &info.nodeId) < 0) return -1; + if (tDecodeSEpSet(pDecoder, &info.prevEp) < 0) return -1; + if (tDecodeSEpSet(pDecoder, &info.newEp) < 0) return -1; + taosArrayPush(pMsg->pNodeList, &info); + } + + if (tDecodeI32(pDecoder, &pMsg->transId) < 0) return -1; + + tEndDecode(pDecoder); + return 0; +} + +int32_t tEncodeStreamTaskCheckReq(SEncoder* pEncoder, const SStreamTaskCheckReq* pReq) { + if (tStartEncode(pEncoder) < 0) return -1; + if (tEncodeI64(pEncoder, pReq->reqId) < 0) return -1; + if (tEncodeI64(pEncoder, pReq->streamId) < 0) return -1; + if (tEncodeI32(pEncoder, pReq->upstreamNodeId) < 0) return -1; + if (tEncodeI32(pEncoder, pReq->upstreamTaskId) < 0) return -1; + if (tEncodeI32(pEncoder, pReq->downstreamNodeId) < 0) return -1; + if (tEncodeI32(pEncoder, pReq->downstreamTaskId) < 0) return -1; + if (tEncodeI32(pEncoder, pReq->childId) < 0) return -1; + if (tEncodeI64(pEncoder, pReq->stage) < 0) return -1; + tEndEncode(pEncoder); + return pEncoder->pos; +} + +int32_t tDecodeStreamTaskCheckReq(SDecoder* pDecoder, SStreamTaskCheckReq* pReq) { + if (tStartDecode(pDecoder) < 0) return -1; + if (tDecodeI64(pDecoder, &pReq->reqId) < 0) return -1; + if (tDecodeI64(pDecoder, &pReq->streamId) < 0) return -1; + if (tDecodeI32(pDecoder, &pReq->upstreamNodeId) < 0) return -1; + if (tDecodeI32(pDecoder, &pReq->upstreamTaskId) < 0) return -1; + if (tDecodeI32(pDecoder, &pReq->downstreamNodeId) < 0) return -1; + if (tDecodeI32(pDecoder, &pReq->downstreamTaskId) < 0) return -1; + if (tDecodeI32(pDecoder, &pReq->childId) < 0) return -1; + if (tDecodeI64(pDecoder, &pReq->stage) < 0) return -1; + tEndDecode(pDecoder); + return 0; +} + +int32_t tEncodeStreamTaskCheckRsp(SEncoder* pEncoder, const SStreamTaskCheckRsp* pRsp) { + if (tStartEncode(pEncoder) < 0) return -1; + if (tEncodeI64(pEncoder, pRsp->reqId) < 0) return -1; + if (tEncodeI64(pEncoder, pRsp->streamId) < 0) return -1; + if (tEncodeI32(pEncoder, pRsp->upstreamNodeId) < 0) return -1; + if (tEncodeI32(pEncoder, pRsp->upstreamTaskId) < 0) return -1; + if (tEncodeI32(pEncoder, pRsp->downstreamNodeId) < 0) return -1; + if (tEncodeI32(pEncoder, pRsp->downstreamTaskId) < 0) return -1; + if (tEncodeI32(pEncoder, pRsp->childId) < 0) return -1; + if (tEncodeI64(pEncoder, pRsp->oldStage) < 0) return -1; + if (tEncodeI8(pEncoder, pRsp->status) < 0) return -1; + tEndEncode(pEncoder); + return pEncoder->pos; +} + +int32_t tDecodeStreamTaskCheckRsp(SDecoder* pDecoder, SStreamTaskCheckRsp* pRsp) { + if (tStartDecode(pDecoder) < 0) return -1; + if (tDecodeI64(pDecoder, &pRsp->reqId) < 0) return -1; + if (tDecodeI64(pDecoder, &pRsp->streamId) < 0) return -1; + if (tDecodeI32(pDecoder, &pRsp->upstreamNodeId) < 0) return -1; + if (tDecodeI32(pDecoder, &pRsp->upstreamTaskId) < 0) return -1; + if (tDecodeI32(pDecoder, &pRsp->downstreamNodeId) < 0) return -1; + if (tDecodeI32(pDecoder, &pRsp->downstreamTaskId) < 0) return -1; + if (tDecodeI32(pDecoder, &pRsp->childId) < 0) return -1; + if (tDecodeI64(pDecoder, &pRsp->oldStage) < 0) return -1; + if (tDecodeI8(pDecoder, &pRsp->status) < 0) return -1; + tEndDecode(pDecoder); + return 0; +} + + +int32_t tEncodeStreamCheckpointReadyMsg(SEncoder* pEncoder, const SStreamCheckpointReadyMsg* pReq) { + if (tStartEncode(pEncoder) < 0) return -1; + if (tEncodeI64(pEncoder, pReq->streamId) < 0) return -1; + if (tEncodeI64(pEncoder, pReq->checkpointId) < 0) return -1; + if (tEncodeI32(pEncoder, pReq->downstreamTaskId) < 0) return -1; + if (tEncodeI32(pEncoder, pReq->downstreamNodeId) < 0) return -1; + if (tEncodeI32(pEncoder, pReq->upstreamTaskId) < 0) return -1; + if (tEncodeI32(pEncoder, pReq->upstreamNodeId) < 0) return -1; + if (tEncodeI32(pEncoder, pReq->childId) < 0) return -1; + tEndEncode(pEncoder); + return pEncoder->pos; +} + +int32_t tDecodeStreamCheckpointReadyMsg(SDecoder* pDecoder, SStreamCheckpointReadyMsg* pRsp) { + if (tStartDecode(pDecoder) < 0) return -1; + if (tDecodeI64(pDecoder, &pRsp->streamId) < 0) return -1; + if (tDecodeI64(pDecoder, &pRsp->checkpointId) < 0) return -1; + if (tDecodeI32(pDecoder, &pRsp->downstreamTaskId) < 0) return -1; + if (tDecodeI32(pDecoder, &pRsp->downstreamNodeId) < 0) return -1; + if (tDecodeI32(pDecoder, &pRsp->upstreamTaskId) < 0) return -1; + if (tDecodeI32(pDecoder, &pRsp->upstreamNodeId) < 0) return -1; + if (tDecodeI32(pDecoder, &pRsp->childId) < 0) return -1; + tEndDecode(pDecoder); + return 0; +} + +int32_t tEncodeStreamDispatchReq(SEncoder* pEncoder, const SStreamDispatchReq* pReq) { + if (tStartEncode(pEncoder) < 0) return -1; + if (tEncodeI64(pEncoder, pReq->stage) < 0) return -1; + if (tEncodeI32(pEncoder, pReq->msgId) < 0) return -1; + if (tEncodeI32(pEncoder, pReq->srcVgId) < 0) return -1; + if (tEncodeI32(pEncoder, pReq->type) < 0) return -1; + if (tEncodeI64(pEncoder, pReq->streamId) < 0) return -1; + if (tEncodeI32(pEncoder, pReq->taskId) < 0) return -1; + if (tEncodeI32(pEncoder, pReq->type) < 0) return -1; + if (tEncodeI32(pEncoder, pReq->upstreamTaskId) < 0) return -1; + if (tEncodeI32(pEncoder, pReq->upstreamChildId) < 0) return -1; + if (tEncodeI32(pEncoder, pReq->upstreamNodeId) < 0) return -1; + if (tEncodeI32(pEncoder, pReq->upstreamRelTaskId) < 0) return -1; + if (tEncodeI32(pEncoder, pReq->blockNum) < 0) return -1; + if (tEncodeI64(pEncoder, pReq->totalLen) < 0) return -1; + ASSERT(taosArrayGetSize(pReq->data) == pReq->blockNum); + ASSERT(taosArrayGetSize(pReq->dataLen) == pReq->blockNum); + for (int32_t i = 0; i < pReq->blockNum; i++) { + int32_t len = *(int32_t*)taosArrayGet(pReq->dataLen, i); + void* data = taosArrayGetP(pReq->data, i); + if (tEncodeI32(pEncoder, len) < 0) return -1; + if (tEncodeBinary(pEncoder, data, len) < 0) return -1; + } + tEndEncode(pEncoder); + return pEncoder->pos; +} + +int32_t tDecodeStreamDispatchReq(SDecoder* pDecoder, SStreamDispatchReq* pReq) { + if (tStartDecode(pDecoder) < 0) return -1; + if (tDecodeI64(pDecoder, &pReq->stage) < 0) return -1; + if (tDecodeI32(pDecoder, &pReq->msgId) < 0) return -1; + if (tDecodeI32(pDecoder, &pReq->srcVgId) < 0) return -1; + if (tDecodeI32(pDecoder, &pReq->type) < 0) return -1; + if (tDecodeI64(pDecoder, &pReq->streamId) < 0) return -1; + if (tDecodeI32(pDecoder, &pReq->taskId) < 0) return -1; + if (tDecodeI32(pDecoder, &pReq->type) < 0) return -1; + if (tDecodeI32(pDecoder, &pReq->upstreamTaskId) < 0) return -1; + if (tDecodeI32(pDecoder, &pReq->upstreamChildId) < 0) return -1; + if (tDecodeI32(pDecoder, &pReq->upstreamNodeId) < 0) return -1; + if (tDecodeI32(pDecoder, &pReq->upstreamRelTaskId) < 0) return -1; + if (tDecodeI32(pDecoder, &pReq->blockNum) < 0) return -1; + if (tDecodeI64(pDecoder, &pReq->totalLen) < 0) return -1; + + ASSERT(pReq->blockNum > 0); + pReq->data = taosArrayInit(pReq->blockNum, sizeof(void*)); + pReq->dataLen = taosArrayInit(pReq->blockNum, sizeof(int32_t)); + for (int32_t i = 0; i < pReq->blockNum; i++) { + int32_t len1; + uint64_t len2; + void* data; + if (tDecodeI32(pDecoder, &len1) < 0) return -1; + if (tDecodeBinaryAlloc(pDecoder, &data, &len2) < 0) return -1; + ASSERT(len1 == len2); + taosArrayPush(pReq->dataLen, &len1); + taosArrayPush(pReq->data, &data); + } + + tEndDecode(pDecoder); + return 0; +} + +void tCleanupStreamDispatchReq(SStreamDispatchReq* pReq) { + taosArrayDestroyP(pReq->data, taosMemoryFree); + taosArrayDestroy(pReq->dataLen); +} + +int32_t tEncodeStreamRetrieveReq(SEncoder* pEncoder, const SStreamRetrieveReq* pReq) { + if (tStartEncode(pEncoder) < 0) return -1; + if (tEncodeI64(pEncoder, pReq->streamId) < 0) return -1; + if (tEncodeI64(pEncoder, pReq->reqId) < 0) return -1; + if (tEncodeI32(pEncoder, pReq->dstNodeId) < 0) return -1; + if (tEncodeI32(pEncoder, pReq->dstTaskId) < 0) return -1; + if (tEncodeI32(pEncoder, pReq->srcNodeId) < 0) return -1; + if (tEncodeI32(pEncoder, pReq->srcTaskId) < 0) return -1; + if (tEncodeBinary(pEncoder, (const uint8_t*)pReq->pRetrieve, pReq->retrieveLen) < 0) return -1; + tEndEncode(pEncoder); + return pEncoder->pos; +} + +int32_t tDecodeStreamRetrieveReq(SDecoder* pDecoder, SStreamRetrieveReq* pReq) { + if (tStartDecode(pDecoder) < 0) return -1; + if (tDecodeI64(pDecoder, &pReq->streamId) < 0) return -1; + if (tDecodeI64(pDecoder, &pReq->reqId) < 0) return -1; + if (tDecodeI32(pDecoder, &pReq->dstNodeId) < 0) return -1; + if (tDecodeI32(pDecoder, &pReq->dstTaskId) < 0) return -1; + if (tDecodeI32(pDecoder, &pReq->srcNodeId) < 0) return -1; + if (tDecodeI32(pDecoder, &pReq->srcTaskId) < 0) return -1; + uint64_t len = 0; + if (tDecodeBinaryAlloc(pDecoder, (void**)&pReq->pRetrieve, &len) < 0) return -1; + pReq->retrieveLen = (int32_t)len; + tEndDecode(pDecoder); + return 0; +} + +void tCleanupStreamRetrieveReq(SStreamRetrieveReq* pReq) { taosMemoryFree(pReq->pRetrieve); } + +int32_t tEncodeStreamTaskCheckpointReq(SEncoder* pEncoder, const SStreamTaskCheckpointReq* pReq) { + if (tStartEncode(pEncoder) < 0) return -1; + if (tEncodeI64(pEncoder, pReq->streamId) < 0) return -1; + if (tEncodeI32(pEncoder, pReq->taskId) < 0) return -1; + if (tEncodeI32(pEncoder, pReq->nodeId) < 0) return -1; + tEndEncode(pEncoder); + return 0; +} + +int32_t tDecodeStreamTaskCheckpointReq(SDecoder* pDecoder, SStreamTaskCheckpointReq* pReq) { + if (tStartDecode(pDecoder) < 0) return -1; + if (tDecodeI64(pDecoder, &pReq->streamId) < 0) return -1; + if (tDecodeI32(pDecoder, &pReq->taskId) < 0) return -1; + if (tDecodeI32(pDecoder, &pReq->nodeId) < 0) return -1; + tEndDecode(pDecoder); + return 0; +} + +int32_t tEncodeStreamHbMsg(SEncoder* pEncoder, const SStreamHbMsg* pReq) { + if (tStartEncode(pEncoder) < 0) return -1; + if (tEncodeI32(pEncoder, pReq->vgId) < 0) return -1; + if (tEncodeI32(pEncoder, pReq->numOfTasks) < 0) return -1; + + for (int32_t i = 0; i < pReq->numOfTasks; ++i) { + STaskStatusEntry* ps = taosArrayGet(pReq->pTaskStatus, i); + if (tEncodeI64(pEncoder, ps->id.streamId) < 0) return -1; + if (tEncodeI32(pEncoder, ps->id.taskId) < 0) return -1; + if (tEncodeI32(pEncoder, ps->status) < 0) return -1; + if (tEncodeI64(pEncoder, ps->stage) < 0) return -1; + if (tEncodeI32(pEncoder, ps->nodeId) < 0) return -1; + if (tEncodeDouble(pEncoder, ps->inputQUsed) < 0) return -1; + if (tEncodeDouble(pEncoder, ps->inputRate) < 0) return -1; + if (tEncodeDouble(pEncoder, ps->sinkQuota) < 0) return -1; + if (tEncodeDouble(pEncoder, ps->sinkDataSize) < 0) return -1; + if (tEncodeI64(pEncoder, ps->processedVer) < 0) return -1; + if (tEncodeI64(pEncoder, ps->verRange.minVer) < 0) return -1; + if (tEncodeI64(pEncoder, ps->verRange.maxVer) < 0) return -1; + if (tEncodeI64(pEncoder, ps->checkpointInfo.activeId) < 0) return -1; + if (tEncodeI8(pEncoder, ps->checkpointInfo.failed) < 0) return -1; + if (tEncodeI32(pEncoder, ps->checkpointInfo.activeTransId) < 0) return -1; + if (tEncodeI64(pEncoder, ps->checkpointInfo.latestId) < 0) return -1; + if (tEncodeI64(pEncoder, ps->checkpointInfo.latestVer) < 0) return -1; + if (tEncodeI64(pEncoder, ps->checkpointInfo.latestTime) < 0) return -1; + if (tEncodeI64(pEncoder, ps->startTime) < 0) return -1; + if (tEncodeI64(pEncoder, ps->startCheckpointId) < 0) return -1; + if (tEncodeI64(pEncoder, ps->startCheckpointVer) < 0) return -1; + if (tEncodeI64(pEncoder, ps->hTaskId) < 0) return -1; + } + + int32_t numOfVgs = taosArrayGetSize(pReq->pUpdateNodes); + if (tEncodeI32(pEncoder, numOfVgs) < 0) return -1; + + for (int j = 0; j < numOfVgs; ++j) { + int32_t* pVgId = taosArrayGet(pReq->pUpdateNodes, j); + if (tEncodeI32(pEncoder, *pVgId) < 0) return -1; + } + + tEndEncode(pEncoder); + return pEncoder->pos; +} + +int32_t tDecodeStreamHbMsg(SDecoder* pDecoder, SStreamHbMsg* pReq) { + if (tStartDecode(pDecoder) < 0) return -1; + if (tDecodeI32(pDecoder, &pReq->vgId) < 0) return -1; + if (tDecodeI32(pDecoder, &pReq->numOfTasks) < 0) return -1; + + pReq->pTaskStatus = taosArrayInit(pReq->numOfTasks, sizeof(STaskStatusEntry)); + for (int32_t i = 0; i < pReq->numOfTasks; ++i) { + int32_t taskId = 0; + STaskStatusEntry entry = {0}; + + if (tDecodeI64(pDecoder, &entry.id.streamId) < 0) return -1; + if (tDecodeI32(pDecoder, &taskId) < 0) return -1; + if (tDecodeI32(pDecoder, &entry.status) < 0) return -1; + if (tDecodeI64(pDecoder, &entry.stage) < 0) return -1; + if (tDecodeI32(pDecoder, &entry.nodeId) < 0) return -1; + if (tDecodeDouble(pDecoder, &entry.inputQUsed) < 0) return -1; + if (tDecodeDouble(pDecoder, &entry.inputRate) < 0) return -1; + if (tDecodeDouble(pDecoder, &entry.sinkQuota) < 0) return -1; + if (tDecodeDouble(pDecoder, &entry.sinkDataSize) < 0) return -1; + if (tDecodeI64(pDecoder, &entry.processedVer) < 0) return -1; + if (tDecodeI64(pDecoder, &entry.verRange.minVer) < 0) return -1; + if (tDecodeI64(pDecoder, &entry.verRange.maxVer) < 0) return -1; + if (tDecodeI64(pDecoder, &entry.checkpointInfo.activeId) < 0) return -1; + if (tDecodeI8(pDecoder, &entry.checkpointInfo.failed) < 0) return -1; + if (tDecodeI32(pDecoder, &entry.checkpointInfo.activeTransId) < 0) return -1; + + if (tDecodeI64(pDecoder, &entry.checkpointInfo.latestId) < 0) return -1; + if (tDecodeI64(pDecoder, &entry.checkpointInfo.latestVer) < 0) return -1; + if (tDecodeI64(pDecoder, &entry.checkpointInfo.latestTime) < 0) return -1; + if (tDecodeI64(pDecoder, &entry.startTime) < 0) return -1; + if (tDecodeI64(pDecoder, &entry.startCheckpointId) < 0) return -1; + if (tDecodeI64(pDecoder, &entry.startCheckpointVer) < 0) return -1; + if (tDecodeI64(pDecoder, &entry.hTaskId) < 0) return -1; + + entry.id.taskId = taskId; + taosArrayPush(pReq->pTaskStatus, &entry); + } + + int32_t numOfVgs = 0; + if (tDecodeI32(pDecoder, &numOfVgs) < 0) return -1; + + pReq->pUpdateNodes = taosArrayInit(numOfVgs, sizeof(int32_t)); + + for (int j = 0; j < numOfVgs; ++j) { + int32_t vgId = 0; + if (tDecodeI32(pDecoder, &vgId) < 0) return -1; + taosArrayPush(pReq->pUpdateNodes, &vgId); + } + + tEndDecode(pDecoder); + return 0; +} + +void tCleanupStreamHbMsg(SStreamHbMsg* pMsg) { + if (pMsg == NULL) { + return; + } + + if (pMsg->pUpdateNodes != NULL) { + taosArrayDestroy(pMsg->pUpdateNodes); + } + + if (pMsg->pTaskStatus != NULL) { + taosArrayDestroy(pMsg->pTaskStatus); + } +} \ No newline at end of file From 5530ef42cdfffb11a6819a081873f964f1a99205 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 9 May 2024 10:15:34 +0800 Subject: [PATCH 25/30] refactor: do some internal refactor. --- source/libs/stream/src/streamTask.c | 155 ---------------------------- source/libs/stream/src/streammsg.c | 155 ++++++++++++++++++++++++++++ 2 files changed, 155 insertions(+), 155 deletions(-) diff --git a/source/libs/stream/src/streamTask.c b/source/libs/stream/src/streamTask.c index a902aae9f494..1e622f615d24 100644 --- a/source/libs/stream/src/streamTask.c +++ b/source/libs/stream/src/streamTask.c @@ -137,161 +137,6 @@ SStreamTask* tNewStreamTask(int64_t streamId, int8_t taskLevel, SEpSet* pEpset, return pTask; } -int32_t tEncodeStreamTask(SEncoder* pEncoder, const SStreamTask* pTask) { - if (tStartEncode(pEncoder) < 0) return -1; - if (tEncodeI64(pEncoder, pTask->ver) < 0) return -1; - if (tEncodeI64(pEncoder, pTask->id.streamId) < 0) return -1; - if (tEncodeI32(pEncoder, pTask->id.taskId) < 0) return -1; - if (tEncodeI32(pEncoder, pTask->info.totalLevel) < 0) return -1; - if (tEncodeI8(pEncoder, pTask->info.taskLevel) < 0) return -1; - if (tEncodeI8(pEncoder, pTask->outputInfo.type) < 0) return -1; - if (tEncodeI16(pEncoder, pTask->msgInfo.msgType) < 0) return -1; - - if (tEncodeI8(pEncoder, pTask->status.taskStatus) < 0) return -1; - if (tEncodeI8(pEncoder, pTask->status.schedStatus) < 0) return -1; - - if (tEncodeI32(pEncoder, pTask->info.selfChildId) < 0) return -1; - if (tEncodeI32(pEncoder, pTask->info.nodeId) < 0) return -1; - if (tEncodeSEpSet(pEncoder, &pTask->info.epSet) < 0) return -1; - if (tEncodeSEpSet(pEncoder, &pTask->info.mnodeEpset) < 0) return -1; - - if (tEncodeI64(pEncoder, pTask->chkInfo.checkpointId) < 0) return -1; - if (tEncodeI64(pEncoder, pTask->chkInfo.checkpointVer) < 0) return -1; - if (tEncodeI8(pEncoder, pTask->info.fillHistory) < 0) return -1; - - if (tEncodeI64(pEncoder, pTask->hTaskInfo.id.streamId)) return -1; - int32_t taskId = pTask->hTaskInfo.id.taskId; - if (tEncodeI32(pEncoder, taskId)) return -1; - - if (tEncodeI64(pEncoder, pTask->streamTaskId.streamId)) return -1; - taskId = pTask->streamTaskId.taskId; - if (tEncodeI32(pEncoder, taskId)) return -1; - - if (tEncodeU64(pEncoder, pTask->dataRange.range.minVer)) return -1; - if (tEncodeU64(pEncoder, pTask->dataRange.range.maxVer)) return -1; - if (tEncodeI64(pEncoder, pTask->dataRange.window.skey)) return -1; - if (tEncodeI64(pEncoder, pTask->dataRange.window.ekey)) return -1; - - int32_t epSz = taosArrayGetSize(pTask->upstreamInfo.pList); - if (tEncodeI32(pEncoder, epSz) < 0) return -1; - for (int32_t i = 0; i < epSz; i++) { - SStreamChildEpInfo* pInfo = taosArrayGetP(pTask->upstreamInfo.pList, i); - if (tEncodeStreamEpInfo(pEncoder, pInfo) < 0) return -1; - } - - if (pTask->info.taskLevel != TASK_LEVEL__SINK) { - if (tEncodeCStr(pEncoder, pTask->exec.qmsg) < 0) return -1; - } - - if (pTask->outputInfo.type == TASK_OUTPUT__TABLE) { - if (tEncodeI64(pEncoder, pTask->outputInfo.tbSink.stbUid) < 0) return -1; - if (tEncodeCStr(pEncoder, pTask->outputInfo.tbSink.stbFullName) < 0) return -1; - if (tEncodeSSchemaWrapper(pEncoder, pTask->outputInfo.tbSink.pSchemaWrapper) < 0) return -1; - } else if (pTask->outputInfo.type == TASK_OUTPUT__SMA) { - if (tEncodeI64(pEncoder, pTask->outputInfo.smaSink.smaId) < 0) return -1; - } else if (pTask->outputInfo.type == TASK_OUTPUT__FETCH) { - if (tEncodeI8(pEncoder, pTask->outputInfo.fetchSink.reserved) < 0) return -1; - } else if (pTask->outputInfo.type == TASK_OUTPUT__FIXED_DISPATCH) { - if (tEncodeI32(pEncoder, pTask->outputInfo.fixedDispatcher.taskId) < 0) return -1; - if (tEncodeI32(pEncoder, pTask->outputInfo.fixedDispatcher.nodeId) < 0) return -1; - if (tEncodeSEpSet(pEncoder, &pTask->outputInfo.fixedDispatcher.epSet) < 0) return -1; - } else if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { - if (tSerializeSUseDbRspImp(pEncoder, &pTask->outputInfo.shuffleDispatcher.dbInfo) < 0) return -1; - if (tEncodeCStr(pEncoder, pTask->outputInfo.shuffleDispatcher.stbFullName) < 0) return -1; - } - if (tEncodeI64(pEncoder, pTask->info.triggerParam) < 0) return -1; - if (tEncodeI8(pEncoder, pTask->subtableWithoutMd5) < 0) return -1; - if (tEncodeCStrWithLen(pEncoder, pTask->reserve, sizeof(pTask->reserve) - 1) < 0) return -1; - - tEndEncode(pEncoder); - return pEncoder->pos; -} - -int32_t tDecodeStreamTask(SDecoder* pDecoder, SStreamTask* pTask) { - int32_t taskId = 0; - - if (tStartDecode(pDecoder) < 0) return -1; - if (tDecodeI64(pDecoder, &pTask->ver) < 0) return -1; - if (pTask->ver <= SSTREAM_TASK_INCOMPATIBLE_VER || pTask->ver > SSTREAM_TASK_VER) return -1; - - if (tDecodeI64(pDecoder, &pTask->id.streamId) < 0) return -1; - if (tDecodeI32(pDecoder, &pTask->id.taskId) < 0) return -1; - if (tDecodeI32(pDecoder, &pTask->info.totalLevel) < 0) return -1; - if (tDecodeI8(pDecoder, &pTask->info.taskLevel) < 0) return -1; - if (tDecodeI8(pDecoder, &pTask->outputInfo.type) < 0) return -1; - if (tDecodeI16(pDecoder, &pTask->msgInfo.msgType) < 0) return -1; - - if (tDecodeI8(pDecoder, &pTask->status.taskStatus) < 0) return -1; - if (tDecodeI8(pDecoder, &pTask->status.schedStatus) < 0) return -1; - - if (tDecodeI32(pDecoder, &pTask->info.selfChildId) < 0) return -1; - if (tDecodeI32(pDecoder, &pTask->info.nodeId) < 0) return -1; - if (tDecodeSEpSet(pDecoder, &pTask->info.epSet) < 0) return -1; - if (tDecodeSEpSet(pDecoder, &pTask->info.mnodeEpset) < 0) return -1; - - if (tDecodeI64(pDecoder, &pTask->chkInfo.checkpointId) < 0) return -1; - if (tDecodeI64(pDecoder, &pTask->chkInfo.checkpointVer) < 0) return -1; - if (tDecodeI8(pDecoder, &pTask->info.fillHistory) < 0) return -1; - - if (tDecodeI64(pDecoder, &pTask->hTaskInfo.id.streamId)) return -1; - if (tDecodeI32(pDecoder, &taskId)) return -1; - pTask->hTaskInfo.id.taskId = taskId; - - if (tDecodeI64(pDecoder, &pTask->streamTaskId.streamId)) return -1; - if (tDecodeI32(pDecoder, &taskId)) return -1; - pTask->streamTaskId.taskId = taskId; - - if (tDecodeU64(pDecoder, (uint64_t*)&pTask->dataRange.range.minVer)) return -1; - if (tDecodeU64(pDecoder, (uint64_t*)&pTask->dataRange.range.maxVer)) return -1; - if (tDecodeI64(pDecoder, &pTask->dataRange.window.skey)) return -1; - if (tDecodeI64(pDecoder, &pTask->dataRange.window.ekey)) return -1; - - int32_t epSz = -1; - if (tDecodeI32(pDecoder, &epSz) < 0) return -1; - - pTask->upstreamInfo.pList = taosArrayInit(epSz, POINTER_BYTES); - for (int32_t i = 0; i < epSz; i++) { - SStreamChildEpInfo* pInfo = taosMemoryCalloc(1, sizeof(SStreamChildEpInfo)); - if (pInfo == NULL) return -1; - if (tDecodeStreamEpInfo(pDecoder, pInfo) < 0) { - taosMemoryFreeClear(pInfo); - return -1; - } - taosArrayPush(pTask->upstreamInfo.pList, &pInfo); - } - - if (pTask->info.taskLevel != TASK_LEVEL__SINK) { - if (tDecodeCStrAlloc(pDecoder, &pTask->exec.qmsg) < 0) return -1; - } - - if (pTask->outputInfo.type == TASK_OUTPUT__TABLE) { - if (tDecodeI64(pDecoder, &pTask->outputInfo.tbSink.stbUid) < 0) return -1; - if (tDecodeCStrTo(pDecoder, pTask->outputInfo.tbSink.stbFullName) < 0) return -1; - pTask->outputInfo.tbSink.pSchemaWrapper = taosMemoryCalloc(1, sizeof(SSchemaWrapper)); - if (pTask->outputInfo.tbSink.pSchemaWrapper == NULL) return -1; - if (tDecodeSSchemaWrapper(pDecoder, pTask->outputInfo.tbSink.pSchemaWrapper) < 0) return -1; - } else if (pTask->outputInfo.type == TASK_OUTPUT__SMA) { - if (tDecodeI64(pDecoder, &pTask->outputInfo.smaSink.smaId) < 0) return -1; - } else if (pTask->outputInfo.type == TASK_OUTPUT__FETCH) { - if (tDecodeI8(pDecoder, &pTask->outputInfo.fetchSink.reserved) < 0) return -1; - } else if (pTask->outputInfo.type == TASK_OUTPUT__FIXED_DISPATCH) { - if (tDecodeI32(pDecoder, &pTask->outputInfo.fixedDispatcher.taskId) < 0) return -1; - if (tDecodeI32(pDecoder, &pTask->outputInfo.fixedDispatcher.nodeId) < 0) return -1; - if (tDecodeSEpSet(pDecoder, &pTask->outputInfo.fixedDispatcher.epSet) < 0) return -1; - } else if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { - if (tDeserializeSUseDbRspImp(pDecoder, &pTask->outputInfo.shuffleDispatcher.dbInfo) < 0) return -1; - if (tDecodeCStrTo(pDecoder, pTask->outputInfo.shuffleDispatcher.stbFullName) < 0) return -1; - } - if (tDecodeI64(pDecoder, &pTask->info.triggerParam) < 0) return -1; - if (pTask->ver >= SSTREAM_TASK_SUBTABLE_CHANGED_VER){ - if (tDecodeI8(pDecoder, &pTask->subtableWithoutMd5) < 0) return -1; - } - if (tDecodeCStrTo(pDecoder, pTask->reserve) < 0) return -1; - - tEndDecode(pDecoder); - return 0; -} - int32_t tDecodeStreamTaskChkInfo(SDecoder* pDecoder, SCheckpointInfo* pChkpInfo) { int64_t skip64; int8_t skip8; diff --git a/source/libs/stream/src/streammsg.c b/source/libs/stream/src/streammsg.c index b37299f54fd3..5e52b927c6a6 100644 --- a/source/libs/stream/src/streammsg.c +++ b/source/libs/stream/src/streammsg.c @@ -429,4 +429,159 @@ void tCleanupStreamHbMsg(SStreamHbMsg* pMsg) { if (pMsg->pTaskStatus != NULL) { taosArrayDestroy(pMsg->pTaskStatus); } +} + +int32_t tEncodeStreamTask(SEncoder* pEncoder, const SStreamTask* pTask) { + if (tStartEncode(pEncoder) < 0) return -1; + if (tEncodeI64(pEncoder, pTask->ver) < 0) return -1; + if (tEncodeI64(pEncoder, pTask->id.streamId) < 0) return -1; + if (tEncodeI32(pEncoder, pTask->id.taskId) < 0) return -1; + if (tEncodeI32(pEncoder, pTask->info.totalLevel) < 0) return -1; + if (tEncodeI8(pEncoder, pTask->info.taskLevel) < 0) return -1; + if (tEncodeI8(pEncoder, pTask->outputInfo.type) < 0) return -1; + if (tEncodeI16(pEncoder, pTask->msgInfo.msgType) < 0) return -1; + + if (tEncodeI8(pEncoder, pTask->status.taskStatus) < 0) return -1; + if (tEncodeI8(pEncoder, pTask->status.schedStatus) < 0) return -1; + + if (tEncodeI32(pEncoder, pTask->info.selfChildId) < 0) return -1; + if (tEncodeI32(pEncoder, pTask->info.nodeId) < 0) return -1; + if (tEncodeSEpSet(pEncoder, &pTask->info.epSet) < 0) return -1; + if (tEncodeSEpSet(pEncoder, &pTask->info.mnodeEpset) < 0) return -1; + + if (tEncodeI64(pEncoder, pTask->chkInfo.checkpointId) < 0) return -1; + if (tEncodeI64(pEncoder, pTask->chkInfo.checkpointVer) < 0) return -1; + if (tEncodeI8(pEncoder, pTask->info.fillHistory) < 0) return -1; + + if (tEncodeI64(pEncoder, pTask->hTaskInfo.id.streamId)) return -1; + int32_t taskId = pTask->hTaskInfo.id.taskId; + if (tEncodeI32(pEncoder, taskId)) return -1; + + if (tEncodeI64(pEncoder, pTask->streamTaskId.streamId)) return -1; + taskId = pTask->streamTaskId.taskId; + if (tEncodeI32(pEncoder, taskId)) return -1; + + if (tEncodeU64(pEncoder, pTask->dataRange.range.minVer)) return -1; + if (tEncodeU64(pEncoder, pTask->dataRange.range.maxVer)) return -1; + if (tEncodeI64(pEncoder, pTask->dataRange.window.skey)) return -1; + if (tEncodeI64(pEncoder, pTask->dataRange.window.ekey)) return -1; + + int32_t epSz = taosArrayGetSize(pTask->upstreamInfo.pList); + if (tEncodeI32(pEncoder, epSz) < 0) return -1; + for (int32_t i = 0; i < epSz; i++) { + SStreamChildEpInfo* pInfo = taosArrayGetP(pTask->upstreamInfo.pList, i); + if (tEncodeStreamEpInfo(pEncoder, pInfo) < 0) return -1; + } + + if (pTask->info.taskLevel != TASK_LEVEL__SINK) { + if (tEncodeCStr(pEncoder, pTask->exec.qmsg) < 0) return -1; + } + + if (pTask->outputInfo.type == TASK_OUTPUT__TABLE) { + if (tEncodeI64(pEncoder, pTask->outputInfo.tbSink.stbUid) < 0) return -1; + if (tEncodeCStr(pEncoder, pTask->outputInfo.tbSink.stbFullName) < 0) return -1; + if (tEncodeSSchemaWrapper(pEncoder, pTask->outputInfo.tbSink.pSchemaWrapper) < 0) return -1; + } else if (pTask->outputInfo.type == TASK_OUTPUT__SMA) { + if (tEncodeI64(pEncoder, pTask->outputInfo.smaSink.smaId) < 0) return -1; + } else if (pTask->outputInfo.type == TASK_OUTPUT__FETCH) { + if (tEncodeI8(pEncoder, pTask->outputInfo.fetchSink.reserved) < 0) return -1; + } else if (pTask->outputInfo.type == TASK_OUTPUT__FIXED_DISPATCH) { + if (tEncodeI32(pEncoder, pTask->outputInfo.fixedDispatcher.taskId) < 0) return -1; + if (tEncodeI32(pEncoder, pTask->outputInfo.fixedDispatcher.nodeId) < 0) return -1; + if (tEncodeSEpSet(pEncoder, &pTask->outputInfo.fixedDispatcher.epSet) < 0) return -1; + } else if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { + if (tSerializeSUseDbRspImp(pEncoder, &pTask->outputInfo.shuffleDispatcher.dbInfo) < 0) return -1; + if (tEncodeCStr(pEncoder, pTask->outputInfo.shuffleDispatcher.stbFullName) < 0) return -1; + } + if (tEncodeI64(pEncoder, pTask->info.triggerParam) < 0) return -1; + if (tEncodeI8(pEncoder, pTask->subtableWithoutMd5) < 0) return -1; + if (tEncodeCStrWithLen(pEncoder, pTask->reserve, sizeof(pTask->reserve) - 1) < 0) return -1; + + tEndEncode(pEncoder); + return pEncoder->pos; +} + +int32_t tDecodeStreamTask(SDecoder* pDecoder, SStreamTask* pTask) { + int32_t taskId = 0; + + if (tStartDecode(pDecoder) < 0) return -1; + if (tDecodeI64(pDecoder, &pTask->ver) < 0) return -1; + if (pTask->ver <= SSTREAM_TASK_INCOMPATIBLE_VER || pTask->ver > SSTREAM_TASK_VER) return -1; + + if (tDecodeI64(pDecoder, &pTask->id.streamId) < 0) return -1; + if (tDecodeI32(pDecoder, &pTask->id.taskId) < 0) return -1; + if (tDecodeI32(pDecoder, &pTask->info.totalLevel) < 0) return -1; + if (tDecodeI8(pDecoder, &pTask->info.taskLevel) < 0) return -1; + if (tDecodeI8(pDecoder, &pTask->outputInfo.type) < 0) return -1; + if (tDecodeI16(pDecoder, &pTask->msgInfo.msgType) < 0) return -1; + + if (tDecodeI8(pDecoder, &pTask->status.taskStatus) < 0) return -1; + if (tDecodeI8(pDecoder, &pTask->status.schedStatus) < 0) return -1; + + if (tDecodeI32(pDecoder, &pTask->info.selfChildId) < 0) return -1; + if (tDecodeI32(pDecoder, &pTask->info.nodeId) < 0) return -1; + if (tDecodeSEpSet(pDecoder, &pTask->info.epSet) < 0) return -1; + if (tDecodeSEpSet(pDecoder, &pTask->info.mnodeEpset) < 0) return -1; + + if (tDecodeI64(pDecoder, &pTask->chkInfo.checkpointId) < 0) return -1; + if (tDecodeI64(pDecoder, &pTask->chkInfo.checkpointVer) < 0) return -1; + if (tDecodeI8(pDecoder, &pTask->info.fillHistory) < 0) return -1; + + if (tDecodeI64(pDecoder, &pTask->hTaskInfo.id.streamId)) return -1; + if (tDecodeI32(pDecoder, &taskId)) return -1; + pTask->hTaskInfo.id.taskId = taskId; + + if (tDecodeI64(pDecoder, &pTask->streamTaskId.streamId)) return -1; + if (tDecodeI32(pDecoder, &taskId)) return -1; + pTask->streamTaskId.taskId = taskId; + + if (tDecodeU64(pDecoder, (uint64_t*)&pTask->dataRange.range.minVer)) return -1; + if (tDecodeU64(pDecoder, (uint64_t*)&pTask->dataRange.range.maxVer)) return -1; + if (tDecodeI64(pDecoder, &pTask->dataRange.window.skey)) return -1; + if (tDecodeI64(pDecoder, &pTask->dataRange.window.ekey)) return -1; + + int32_t epSz = -1; + if (tDecodeI32(pDecoder, &epSz) < 0) return -1; + + pTask->upstreamInfo.pList = taosArrayInit(epSz, POINTER_BYTES); + for (int32_t i = 0; i < epSz; i++) { + SStreamChildEpInfo* pInfo = taosMemoryCalloc(1, sizeof(SStreamChildEpInfo)); + if (pInfo == NULL) return -1; + if (tDecodeStreamEpInfo(pDecoder, pInfo) < 0) { + taosMemoryFreeClear(pInfo); + return -1; + } + taosArrayPush(pTask->upstreamInfo.pList, &pInfo); + } + + if (pTask->info.taskLevel != TASK_LEVEL__SINK) { + if (tDecodeCStrAlloc(pDecoder, &pTask->exec.qmsg) < 0) return -1; + } + + if (pTask->outputInfo.type == TASK_OUTPUT__TABLE) { + if (tDecodeI64(pDecoder, &pTask->outputInfo.tbSink.stbUid) < 0) return -1; + if (tDecodeCStrTo(pDecoder, pTask->outputInfo.tbSink.stbFullName) < 0) return -1; + pTask->outputInfo.tbSink.pSchemaWrapper = taosMemoryCalloc(1, sizeof(SSchemaWrapper)); + if (pTask->outputInfo.tbSink.pSchemaWrapper == NULL) return -1; + if (tDecodeSSchemaWrapper(pDecoder, pTask->outputInfo.tbSink.pSchemaWrapper) < 0) return -1; + } else if (pTask->outputInfo.type == TASK_OUTPUT__SMA) { + if (tDecodeI64(pDecoder, &pTask->outputInfo.smaSink.smaId) < 0) return -1; + } else if (pTask->outputInfo.type == TASK_OUTPUT__FETCH) { + if (tDecodeI8(pDecoder, &pTask->outputInfo.fetchSink.reserved) < 0) return -1; + } else if (pTask->outputInfo.type == TASK_OUTPUT__FIXED_DISPATCH) { + if (tDecodeI32(pDecoder, &pTask->outputInfo.fixedDispatcher.taskId) < 0) return -1; + if (tDecodeI32(pDecoder, &pTask->outputInfo.fixedDispatcher.nodeId) < 0) return -1; + if (tDecodeSEpSet(pDecoder, &pTask->outputInfo.fixedDispatcher.epSet) < 0) return -1; + } else if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) { + if (tDeserializeSUseDbRspImp(pDecoder, &pTask->outputInfo.shuffleDispatcher.dbInfo) < 0) return -1; + if (tDecodeCStrTo(pDecoder, pTask->outputInfo.shuffleDispatcher.stbFullName) < 0) return -1; + } + if (tDecodeI64(pDecoder, &pTask->info.triggerParam) < 0) return -1; + if (pTask->ver >= SSTREAM_TASK_SUBTABLE_CHANGED_VER){ + if (tDecodeI8(pDecoder, &pTask->subtableWithoutMd5) < 0) return -1; + } + if (tDecodeCStrTo(pDecoder, pTask->reserve) < 0) return -1; + + tEndDecode(pDecoder); + return 0; } \ No newline at end of file From 37de0228bafe4aba7cd9078c7ea538c81d7e74b1 Mon Sep 17 00:00:00 2001 From: Ping Xiao Date: Thu, 9 May 2024 14:28:50 +0800 Subject: [PATCH 26/30] fix TD-29911 --- packaging/tools/mac_before_install.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/packaging/tools/mac_before_install.txt b/packaging/tools/mac_before_install.txt index 4ce2374b7fed..a026973de1fa 100644 --- a/packaging/tools/mac_before_install.txt +++ b/packaging/tools/mac_before_install.txt @@ -3,6 +3,7 @@ TDengine is an open-source, cloud-native time-series database optimized for Inte • To configure TDengine, edit /etc/taos/taos.cfg • To start service, run launchctl start com.tdengine.taosd • To start Taos Adapter, run launchctl start com.tdengine.taosadapter +• To start Taos Explorer, run launchctl start com.tdengine.taos-explorer • To access TDengine from your local machine, run taos If you're experiencing problems installing TDengine, check the file /var/log/taos/tdengine_install.log to help troubleshoot the installation. From 1599d80ab8dcfe59cf400f32ef2e5b28f2e6926f Mon Sep 17 00:00:00 2001 From: Shungang Li Date: Thu, 9 May 2024 18:57:45 +0800 Subject: [PATCH 27/30] fix: refactor lastcache ser/des for compatibility --- include/libs/executor/storageapi.h | 1 - source/dnode/vnode/src/inc/tsdb.h | 5 +- source/dnode/vnode/src/tsdb/tsdbCache.c | 264 ++++++++++--------- source/dnode/vnode/src/tsdb/tsdbCacheRead.c | 3 - source/libs/executor/src/cachescanoperator.c | 3 - 5 files changed, 146 insertions(+), 130 deletions(-) diff --git a/include/libs/executor/storageapi.h b/include/libs/executor/storageapi.h index fcb2e4d40583..ec92bd56dd50 100644 --- a/include/libs/executor/storageapi.h +++ b/include/libs/executor/storageapi.h @@ -35,7 +35,6 @@ extern "C" { #define CACHESCAN_RETRIEVE_TYPE_SINGLE 0x2 #define CACHESCAN_RETRIEVE_LAST_ROW 0x4 #define CACHESCAN_RETRIEVE_LAST 0x8 -#define CACHESCAN_RETRIEVE_PK 0x10 #define META_READER_LOCK 0x0 #define META_READER_NOLOCK 0x1 diff --git a/source/dnode/vnode/src/inc/tsdb.h b/source/dnode/vnode/src/inc/tsdb.h index c578b95c1a92..64dc5bc31ae7 100644 --- a/source/dnode/vnode/src/inc/tsdb.h +++ b/source/dnode/vnode/src/inc/tsdb.h @@ -894,13 +894,16 @@ typedef enum { READER_EXEC_ROWS = 0x2, } EExecMode; +#define LAST_COL_VERSION (0x1) + typedef struct { SRowKey rowKey; int8_t dirty; SColVal colVal; + int8_t version; } SLastCol; -typedef struct { +typedef struct { union { int64_t val; struct { diff --git a/source/dnode/vnode/src/tsdb/tsdbCache.c b/source/dnode/vnode/src/tsdb/tsdbCache.c index 7556f993aa8d..161be8904ac3 100644 --- a/source/dnode/vnode/src/tsdb/tsdbCache.c +++ b/source/dnode/vnode/src/tsdb/tsdbCache.c @@ -130,20 +130,14 @@ static void tsdbClosePgCache(STsdb *pTsdb) { enum { LFLAG_LAST_ROW = 0, LFLAG_LAST = 1, - LFLAG_VERSION = 1 << 2, - LFLAG_VERSION_BITS = (1 << 2 | 1 << 3), - LFLAG_PRIMARY_KEY = CACHESCAN_RETRIEVE_PK, }; -#define LAST_KEY_HAS_VERSION ((k).lflag & LFLAG_VERSION_BITS) - typedef struct { tb_uid_t uid; int16_t cid; int8_t lflag; } SLastKey; -#define HAS_PRIMARY_KEY(k) (((k).lflag & LFLAG_PRIMARY_KEY) == LFLAG_PRIMARY_KEY) #define IS_LAST_ROW_KEY(k) (((k).lflag & LFLAG_LAST) == LFLAG_LAST_ROW) #define IS_LAST_KEY(k) (((k).lflag & LFLAG_LAST) == LFLAG_LAST) @@ -336,93 +330,100 @@ static void rocksMayWrite(STsdb *pTsdb, bool force, bool read, bool lock) { } } -// note: new object do not own colVal's resource, just copy the pointer -static SLastCol *tsdbCacheConvertLastColV0(SLastColV0 *pLastColV0) { - SLastCol *pLastCol = taosMemoryCalloc(1, sizeof(SLastCol)); - if (pLastCol == NULL) return NULL; - pLastCol->rowKey.ts = pLastColV0->ts; - pLastCol->rowKey.numOfPKs = 0; - pLastCol->dirty = pLastColV0->dirty; - pLastCol->colVal.cid = pLastColV0->colVal.cid; - pLastCol->colVal.flag = pLastColV0->colVal.flag; - pLastCol->colVal.value.type = pLastColV0->colVal.type; - pLastCol->colVal.value.val = pLastColV0->colVal.value.val; - - return pLastCol; -} +static SLastColV0 *tsdbCacheDeserializeV0(char const *value, size_t *inOutOffset) { + SLastColV0 *pLastColV0 = NULL; + size_t localOffset = 0; -static SLastCol *tsdbCacheDeserializeV0(char const *value) { if (!value) { - return NULL; + goto _OUT; } - SLastColV0 *pLastColV0 = (SLastColV0 *)value; - SColValV0 *pColVal = &pLastColV0->colVal; - if (IS_VAR_DATA_TYPE(pColVal->type)) { - if (pColVal->value.nData > 0) { - pColVal->value.pData = (char *)value + sizeof(*pLastColV0); + pLastColV0 = taosMemoryMalloc(sizeof(SLastColV0)); + *pLastColV0 = *(SLastColV0 *)(value); + + localOffset = sizeof(*pLastColV0); + + SColValV0 *pColValV0 = &pLastColV0->colVal; + if (IS_VAR_DATA_TYPE(pColValV0->type)) { + if (pColValV0->value.nData > 0) { + pColValV0->value.pData = (char *)value + localOffset; + localOffset += pColValV0->value.nData; } else { - pColVal->value.pData = NULL; + pColValV0->value.pData = NULL; } } - return tsdbCacheConvertLastColV0(pLastColV0); +_OUT: + *inOutOffset += localOffset; + return pLastColV0; } -static SLastCol *tsdbCacheDeserializeV1(char const *value) { +static SLastCol *tsdbCacheDeserialize(char const *value, size_t size) { if (!value) { return NULL; } + size_t offset = 0; + + SLastColV0 *pLastColV0 = tsdbCacheDeserializeV0(value, &offset); + if (NULL == pLastColV0) { + return NULL; + } + SLastCol *pLastCol = taosMemoryMalloc(sizeof(SLastCol)); - *pLastCol = *(SLastCol *)(value); + pLastCol->rowKey.ts = pLastColV0->ts; + pLastCol->dirty = pLastColV0->dirty; + pLastCol->colVal.cid = pLastColV0->colVal.cid; + pLastCol->colVal.flag = pLastColV0->colVal.flag; + pLastCol->colVal.value.type = pLastColV0->colVal.type; + if (IS_VAR_DATA_TYPE(pLastCol->colVal.value.type)) { + pLastCol->colVal.value.nData = pLastColV0->colVal.value.nData; + pLastCol->colVal.value.pData = pLastColV0->colVal.value.pData; + } else { + pLastCol->colVal.value.val = pLastColV0->colVal.value.val; + } + + taosMemoryFreeClear(pLastColV0); + + if (offset == size) { + // version 0 + pLastCol->version = LAST_COL_VERSION; + pLastCol->rowKey.numOfPKs = 0; + memset(pLastCol->rowKey.pks, 0, sizeof(pLastCol->rowKey.pks)); + return pLastCol; + } + + pLastCol->version = *(int8_t *)(value + offset); + offset += sizeof(int8_t); + + pLastCol->rowKey.numOfPKs = *(uint8_t *)(value + offset); + offset += sizeof(uint8_t); - char *currentPos = (char *)value + sizeof(*pLastCol); for (int8_t i = 0; i < pLastCol->rowKey.numOfPKs; i++) { SValue *pValue = &pLastCol->rowKey.pks[i]; + *pValue = *(SValue *)(value + offset); + offset += sizeof(SValue); + if (IS_VAR_DATA_TYPE(pValue->type)) { if (pValue->nData > 0) { - pValue->pData = currentPos; - currentPos += pValue->nData; + pValue->pData = (char *)value + offset; + offset += pValue->nData; } else { pValue->pData = NULL; } - } - } - - SColVal *pColVal = &pLastCol->colVal; - if (IS_VAR_DATA_TYPE(pColVal->value.type)) { - if (pColVal->value.nData > 0) { - pColVal->value.pData = currentPos; - currentPos += pColVal->value.nData; } else { - pColVal->value.pData = NULL; + pValue->val = *(int64_t *)(value + offset); + offset += sizeof(int64_t); } } - return pLastCol; -} - -static SLastCol *tsdbCacheDeserialize(char const *value, int8_t lflag) { - if (!value) { + if (size < offset) { + terrno = TSDB_CODE_INVALID_DATA_FMT; + taosMemoryFreeClear(pLastCol); return NULL; } - int8_t version = lflag & LFLAG_VERSION_BITS; - - SLastCol *lastCol = NULL; - switch (version) { - case 0: - lastCol = tsdbCacheDeserializeV0(value); - break; - case LFLAG_VERSION: - lastCol = tsdbCacheDeserializeV1(value); - break; - defalut: - tsdbError("invalid last key version %" PRId8 " , lflag:%" PRId8, version, lflag); - break; - } - return lastCol; + return pLastCol; } static uint32_t tsdbCacheCopyVarData(SValue *from, SValue *to) { @@ -435,42 +436,82 @@ static uint32_t tsdbCacheCopyVarData(SValue *from, SValue *to) { return from->nData; } +static uint32_t tsdbCacheCopyVarDataToV0(SValue *from, SValueV0 *to) { + ASSERT(from->nData >= 0); + if (from->nData > 0) { + memcpy(to->pData, from->pData, from->nData); + } + to->nData = from->nData; + return from->nData; +} + +/* +typedef struct { + SLastColV0 lastColV0; + char colData[]; + int8_t version; + uint8_t numOfPKs; + SValue pks[0]; + char pk0Data[]; + SValue pks[1]; + char pk1Data[]; + ... +} SLastColDisk; +*/ static void tsdbCacheSerialize(SLastCol *pLastCol, char **value, size_t *size) { SColVal *pColVal = &pLastCol->colVal; - size_t length = sizeof(*pLastCol); + size_t length = sizeof(SLastColV0); + if (IS_VAR_DATA_TYPE(pColVal->value.type)) { + length += pColVal->value.nData; + } + + uint8_t numOfPKs = pLastCol->rowKey.numOfPKs; + + length += sizeof(int8_t) + sizeof(uint8_t) + (sizeof(SValue) * numOfPKs); // version + numOfPKs + pks + for (int8_t i = 0; i < pLastCol->rowKey.numOfPKs; i++) { if (IS_VAR_DATA_TYPE(pLastCol->rowKey.pks[i].type)) { length += pLastCol->rowKey.pks[i].nData; } } - if (IS_VAR_DATA_TYPE(pColVal->value.type)) { - length += pColVal->value.nData; - } *value = taosMemoryMalloc(length); // copy last col - SLastCol *pToLastCol = (SLastCol *)(*value); - *pToLastCol = *pLastCol; - char *currentPos = *value + sizeof(*pLastCol); + SLastColV0 *pToLastColV0 = (SLastColV0 *)(*value); + pToLastColV0->ts = pLastCol->rowKey.ts; + pToLastColV0->dirty = pLastCol->dirty; + pToLastColV0->colVal.cid = pColVal->cid; + pToLastColV0->colVal.flag = pColVal->flag; + pToLastColV0->colVal.type = pColVal->value.type; + + char *currentPos = *value + sizeof(*pToLastColV0); + + // copy var data value + if (IS_VAR_DATA_TYPE(pColVal->value.type)) { + SValue *pFromValue = &pColVal->value; + SValueV0 *pToValue = &pToLastColV0->colVal.value; + pToValue->pData = (pFromValue->nData == 0) ? NULL : currentPos; + currentPos += tsdbCacheCopyVarDataToV0(pFromValue, pToValue); + } else { + pToLastColV0->colVal.value.val = pColVal->value.val; + } + + *(int8_t *)currentPos = LAST_COL_VERSION; + currentPos += sizeof(int8_t); // copy var data pks for (int8_t i = 0; i < pLastCol->rowKey.numOfPKs; i++) { SValue *pFromValue = &pLastCol->rowKey.pks[i]; + SValue *pToValue = (SValue *)currentPos; + *pToValue = *pFromValue; + currentPos += sizeof(SValue); if (IS_VAR_DATA_TYPE(pFromValue->type)) { - SValue *pToValue = &pToLastCol->rowKey.pks[i]; pToValue->pData = (pFromValue->nData == 0) ? NULL : currentPos; currentPos += tsdbCacheCopyVarData(pFromValue, pToValue); - } + } } - // copy var data value - if (IS_VAR_DATA_TYPE(pColVal->value.type)) { - SValue *pFromValue = &pColVal->value; - SValue *pToValue = &pToLastCol->colVal.value; - pToValue->pData = (pFromValue->nData == 0) ? NULL : currentPos; - currentPos += tsdbCacheCopyVarData(pFromValue, pToValue); - } *size = length; } @@ -603,7 +644,7 @@ static int32_t tsdbCacheNewTableColumn(STsdb *pTsdb, int64_t uid, int16_t cid, i charge += pLastCol->colVal.value.nData; } - SLastKey *pLastKey = &(SLastKey){.lflag = lflag | LFLAG_VERSION, .uid = uid, .cid = cid}; + SLastKey *pLastKey = &(SLastKey){.lflag = lflag, .uid = uid, .cid = cid}; LRUStatus status = taosLRUCacheInsert(pCache, pLastKey, ROCKS_KEY_LEN, pLastCol, charge, tsdbCacheDeleter, NULL, TAOS_LRU_PRIORITY_LOW, &pTsdb->flushState); if (status != TAOS_LRU_STATUS_OK) { @@ -653,11 +694,9 @@ static int32_t tsdbCacheDropTableColumn(STsdb *pTsdb, int64_t uid, int16_t cid, size_t *keys_list_sizes = taosMemoryCalloc(2, sizeof(size_t)); const size_t klen = ROCKS_KEY_LEN; - int8_t lflag = hasPrimaryKey ? LFLAG_PRIMARY_KEY : 0; - char *keys = taosMemoryCalloc(2, sizeof(SLastKey)); - ((SLastKey *)keys)[0] = (SLastKey){.lflag = lflag | LFLAG_LAST | LFLAG_VERSION, .uid = uid, .cid = cid}; - ((SLastKey *)keys)[1] = (SLastKey){.lflag = lflag | LFLAG_LAST_ROW | LFLAG_VERSION, .uid = uid, .cid = cid}; + ((SLastKey *)keys)[0] = (SLastKey){.lflag = LFLAG_LAST, .uid = uid, .cid = cid}; + ((SLastKey *)keys)[1] = (SLastKey){.lflag = LFLAG_LAST_ROW, .uid = uid, .cid = cid}; keys_list[0] = keys; keys_list[1] = keys + sizeof(SLastKey); @@ -681,13 +720,13 @@ static int32_t tsdbCacheDropTableColumn(STsdb *pTsdb, int64_t uid, int16_t cid, rocksdb_writebatch_t *wb = pTsdb->rCache.writebatch; { - SLastCol *pLastCol = tsdbCacheDeserialize(values_list[0], ((SLastKey*)keys_list[0])->lflag); + SLastCol *pLastCol = tsdbCacheDeserialize(values_list[0], values_list_sizes[0]); if (NULL != pLastCol) { rocksdb_writebatch_delete(wb, keys_list[0], klen); } taosMemoryFreeClear(pLastCol); - pLastCol = tsdbCacheDeserialize(values_list[1], ((SLastKey*)keys_list[1])->lflag); + pLastCol = tsdbCacheDeserialize(values_list[1], values_list_sizes[1]); if (NULL != pLastCol) { rocksdb_writebatch_delete(wb, keys_list[1], klen); } @@ -733,18 +772,12 @@ int32_t tsdbCacheNewTable(STsdb *pTsdb, tb_uid_t uid, tb_uid_t suid, SSchemaWrap taosThreadMutexLock(&pTsdb->lruMutex); if (suid < 0) { - int8_t lflag = 0; - int nCols = pSchemaRow->nCols; - if (nCols >= 2) { - lflag = (pSchemaRow->pSchema[1].flags & COL_IS_KEY) ? LFLAG_PRIMARY_KEY : 0; - } - - for (int i = 0; i < nCols; ++i) { + for (int i = 0; i < pSchemaRow->nCols; ++i) { int16_t cid = pSchemaRow->pSchema[i].colId; int8_t col_type = pSchemaRow->pSchema[i].type; - (void)tsdbCacheNewTableColumn(pTsdb, uid, cid, col_type, lflag | LFLAG_LAST_ROW); - (void)tsdbCacheNewTableColumn(pTsdb, uid, cid, col_type, lflag | LFLAG_LAST); + (void)tsdbCacheNewTableColumn(pTsdb, uid, cid, col_type, LFLAG_LAST_ROW); + (void)tsdbCacheNewTableColumn(pTsdb, uid, cid, col_type, LFLAG_LAST); } } else { STSchema *pTSchema = NULL; @@ -754,18 +787,12 @@ int32_t tsdbCacheNewTable(STsdb *pTsdb, tb_uid_t uid, tb_uid_t suid, SSchemaWrap return -1; } - int8_t lflag = 0; - int nCols = pTSchema->numOfCols; - if (nCols >= 2) { - lflag = (pTSchema->columns[1].flags & COL_IS_KEY) ? LFLAG_PRIMARY_KEY : 0; - } - - for (int i = 0; i < nCols; ++i) { + for (int i = 0; i < pTSchema->numOfCols; ++i) { int16_t cid = pTSchema->columns[i].colId; int8_t col_type = pTSchema->columns[i].type; - (void)tsdbCacheNewTableColumn(pTsdb, uid, cid, col_type, lflag | LFLAG_LAST_ROW); - (void)tsdbCacheNewTableColumn(pTsdb, uid, cid, col_type, lflag | LFLAG_LAST); + (void)tsdbCacheNewTableColumn(pTsdb, uid, cid, col_type, LFLAG_LAST_ROW); + (void)tsdbCacheNewTableColumn(pTsdb, uid, cid, col_type, LFLAG_LAST); } taosMemoryFree(pTSchema); @@ -1024,14 +1051,13 @@ int32_t tsdbCacheUpdate(STsdb *pTsdb, tb_uid_t suid, tb_uid_t uid, TSDBROW *pRow STsdbRowKey tsdbRowKey = {0}; tsdbRowGetKey(pRow, &tsdbRowKey); SRowKey *pRowKey = &tsdbRowKey.key; - int8_t lflag = (pRowKey->numOfPKs != 0) ? LFLAG_PRIMARY_KEY : 0; taosThreadMutexLock(&pTsdb->lruMutex); for (int i = 0; i < num_keys; ++i) { SColVal *pColVal = (SColVal *)taosArrayGet(aColVal, i); int16_t cid = pColVal->cid; - SLastKey *key = &(SLastKey){.lflag = lflag | LFLAG_LAST_ROW | LFLAG_VERSION, .uid = uid, .cid = cid}; + SLastKey *key = &(SLastKey){.lflag = LFLAG_LAST_ROW, .uid = uid, .cid = cid}; size_t klen = ROCKS_KEY_LEN; LRUHandle *h = taosLRUCacheLookup(pCache, key, klen); if (h) { @@ -1049,7 +1075,7 @@ int32_t tsdbCacheUpdate(STsdb *pTsdb, tb_uid_t suid, tb_uid_t uid, TSDBROW *pRow } if (COL_VAL_IS_VALUE(pColVal)) { - key->lflag = lflag | LFLAG_LAST | LFLAG_VERSION; + key->lflag = LFLAG_LAST; LRUHandle *h = taosLRUCacheLookup(pCache, key, klen); if (h) { SLastCol *pLastCol = (SLastCol *)taosLRUCacheValue(pCache, h); @@ -1094,7 +1120,7 @@ int32_t tsdbCacheUpdate(STsdb *pTsdb, tb_uid_t suid, tb_uid_t uid, TSDBROW *pRow SColVal *pColVal = (SColVal *)TARRAY_DATA(aColVal) + idxKey->idx; // SColVal *pColVal = (SColVal *)taosArrayGet(aColVal, idxKey->idx); - SLastCol *pLastCol = tsdbCacheDeserialize(values_list[i], ((SLastKey*)keys_list[i])->lflag); + SLastCol *pLastCol = tsdbCacheDeserialize(values_list[i], values_list_sizes[i]); SLastCol *PToFree = pLastCol; if (IS_LAST_ROW_KEY(idxKey->key)) { @@ -1106,8 +1132,7 @@ int32_t tsdbCacheUpdate(STsdb *pTsdb, tb_uid_t suid, tb_uid_t uid, TSDBROW *pRow if (NULL == pLastCol || cmp_res < 0 || (cmp_res == 0 && !COL_VAL_IS_NONE(pColVal))) { char *value = NULL; size_t vlen = 0; - tsdbCacheSerialize(&(SLastCol){.rowKey = *pRowKey, .colVal = *pColVal}, &value, - &vlen); + tsdbCacheSerialize(&(SLastCol){.rowKey = *pRowKey, .colVal = *pColVal}, &value, &vlen); taosThreadMutexLock(&pTsdb->rCache.rMutex); @@ -1401,7 +1426,7 @@ static int32_t tsdbCacheLoadFromRaw(STsdb *pTsdb, tb_uid_t uid, SArray *pLastArr SIdxKey *idxKey = taosArrayGet(remainCols, 0); if (idxKey->key.cid != PRIMARYKEY_TIMESTAMP_COL_ID) { - SLastKey *key = &(SLastKey){.lflag = ltype | LFLAG_VERSION, .uid = uid, .cid = PRIMARYKEY_TIMESTAMP_COL_ID}; + SLastKey *key = &(SLastKey){.lflag = ltype, .uid = uid, .cid = PRIMARYKEY_TIMESTAMP_COL_ID}; taosArrayInsert(remainCols, 0, &(SIdxKey){0, *key}); } @@ -1570,7 +1595,7 @@ static int32_t tsdbCacheLoadFromRocks(STsdb *pTsdb, tb_uid_t uid, SArray *pLastA SLRUCache *pCache = pTsdb->lruCache; for (int i = 0, j = 0; i < num_keys && j < TARRAY_SIZE(remainCols); ++i) { - SLastCol *pLastCol = tsdbCacheDeserialize(values_list[i], ((SLastKey*)keys_list[i])->lflag); + SLastCol *pLastCol = tsdbCacheDeserialize(values_list[i], values_list_sizes[i]); SLastCol *PToFree = pLastCol; SIdxKey *idxKey = &((SIdxKey *)TARRAY_DATA(remainCols))[j]; if (pLastCol) { @@ -1637,7 +1662,7 @@ int32_t tsdbCacheGetBatch(STsdb *pTsdb, tb_uid_t uid, SArray *pLastArray, SCache for (int i = 0; i < num_keys; ++i) { int16_t cid = ((int16_t *)TARRAY_DATA(pCidList))[i]; - SLastKey key = {.lflag = ltype | LFLAG_VERSION, .uid = uid, .cid = cid}; + SLastKey key = {.lflag = ltype, .uid = uid, .cid = cid}; // for select last_row, last case int32_t funcType = FUNCTION_TYPE_CACHE_LAST; if (pr->pFuncTypeList != NULL && taosArrayGetSize(pr->pFuncTypeList) > i) { @@ -1726,17 +1751,12 @@ int32_t tsdbCacheDel(STsdb *pTsdb, tb_uid_t suid, tb_uid_t uid, TSKEY sKey, TSKE size_t *keys_list_sizes = taosMemoryCalloc(num_keys * 2, sizeof(size_t)); const size_t klen = ROCKS_KEY_LEN; - int8_t lflag = 0; - if (num_keys >= 2) { - lflag = (pTSchema->columns[1].flags & COL_IS_KEY) ? LFLAG_PRIMARY_KEY : 0; - } - for (int i = 0; i < num_keys; ++i) { int16_t cid = pTSchema->columns[i].colId; char *keys = taosMemoryCalloc(2, sizeof(SLastKey)); - ((SLastKey *)keys)[0] = (SLastKey){.lflag = lflag | LFLAG_LAST | LFLAG_VERSION, .uid = uid, .cid = cid}; - ((SLastKey *)keys)[1] = (SLastKey){.lflag = lflag | LFLAG_LAST_ROW | LFLAG_VERSION, .uid = uid, .cid = cid}; + ((SLastKey *)keys)[0] = (SLastKey){.lflag = LFLAG_LAST, .uid = uid, .cid = cid}; + ((SLastKey *)keys)[1] = (SLastKey){.lflag = LFLAG_LAST_ROW, .uid = uid, .cid = cid}; keys_list[i] = keys; keys_list[num_keys + i] = keys + sizeof(SLastKey); @@ -1766,14 +1786,14 @@ int32_t tsdbCacheDel(STsdb *pTsdb, tb_uid_t suid, tb_uid_t uid, TSKEY sKey, TSKE rocksdb_writebatch_t *wb = pTsdb->rCache.writebatch; for (int i = 0; i < num_keys; ++i) { - SLastCol *pLastCol = tsdbCacheDeserialize(values_list[i], ((SLastKey *)keys_list[i])->lflag); + SLastCol *pLastCol = tsdbCacheDeserialize(values_list[i], values_list_sizes[i]); taosThreadMutexLock(&pTsdb->rCache.rMutex); if (NULL != pLastCol && (pLastCol->rowKey.ts <= eKey && pLastCol->rowKey.ts >= sKey)) { rocksdb_writebatch_delete(wb, keys_list[i], klen); } taosMemoryFreeClear(pLastCol); - pLastCol = tsdbCacheDeserialize(values_list[i + num_keys], ((SLastKey *)keys_list[i + num_keys])->lflag); + pLastCol = tsdbCacheDeserialize(values_list[i + num_keys], values_list_sizes[i + num_keys]); if (NULL != pLastCol && (pLastCol->rowKey.ts <= eKey && pLastCol->rowKey.ts >= sKey)) { rocksdb_writebatch_delete(wb, keys_list[num_keys + i], klen); } diff --git a/source/dnode/vnode/src/tsdb/tsdbCacheRead.c b/source/dnode/vnode/src/tsdb/tsdbCacheRead.c index 52aabd40618d..4d6ebf721e31 100644 --- a/source/dnode/vnode/src/tsdb/tsdbCacheRead.c +++ b/source/dnode/vnode/src/tsdb/tsdbCacheRead.c @@ -385,9 +385,6 @@ int32_t tsdbRetrieveCacheRows(void* pReader, SSDataBlock* pResBlock, const int32 } int8_t ltype = (pr->type & CACHESCAN_RETRIEVE_LAST) >> 3; - if (pr->rowKey.numOfPKs > 0) { - ltype |= CACHESCAN_RETRIEVE_PK; - } STableKeyInfo* pTableList = pr->pTableList; diff --git a/source/libs/executor/src/cachescanoperator.c b/source/libs/executor/src/cachescanoperator.c index 0d0870911eaf..ad7d089da9fb 100644 --- a/source/libs/executor/src/cachescanoperator.c +++ b/source/libs/executor/src/cachescanoperator.c @@ -160,9 +160,6 @@ SOperatorInfo* createCacherowsScanOperator(SLastRowScanPhysiNode* pScanNode, SRe // partition by tbname if (oneTableForEachGroup(pTableListInfo) || (totalTables == 1)) { pInfo->retrieveType = CACHESCAN_RETRIEVE_TYPE_ALL | SCAN_ROW_TYPE(pScanNode->ignoreNull); - if (pInfo->numOfPks > 0) { - pInfo->retrieveType |= CACHESCAN_RETRIEVE_PK; - } STableKeyInfo* pList = tableListGetInfo(pTableListInfo, 0); From 30913013421ed8f739dca7d2723b34b75110fb64 Mon Sep 17 00:00:00 2001 From: Shungang Li Date: Thu, 9 May 2024 21:52:43 +0800 Subject: [PATCH 28/30] fix: adjust SValueV0 data order --- source/dnode/vnode/src/inc/tsdb.h | 24 --- source/dnode/vnode/src/tsdb/tsdbCache.c | 206 ++++++++++-------------- 2 files changed, 88 insertions(+), 142 deletions(-) diff --git a/source/dnode/vnode/src/inc/tsdb.h b/source/dnode/vnode/src/inc/tsdb.h index 64dc5bc31ae7..01db1964797c 100644 --- a/source/dnode/vnode/src/inc/tsdb.h +++ b/source/dnode/vnode/src/inc/tsdb.h @@ -900,32 +900,8 @@ typedef struct { SRowKey rowKey; int8_t dirty; SColVal colVal; - int8_t version; } SLastCol; -typedef struct { - union { - int64_t val; - struct { - uint8_t *pData; - uint32_t nData; - }; - }; -} SValueV0; - -typedef struct { - int16_t cid; - int8_t type; - int8_t flag; - SValueV0 value; -} SColValV0; - -typedef struct { - TSKEY ts; - int8_t dirty; - SColValV0 colVal; -} SLastColV0; - int32_t tsdbOpenCache(STsdb *pTsdb); void tsdbCloseCache(STsdb *pTsdb); int32_t tsdbCacheUpdate(STsdb *pTsdb, tb_uid_t suid, tb_uid_t uid, TSDBROW *row); diff --git a/source/dnode/vnode/src/tsdb/tsdbCache.c b/source/dnode/vnode/src/tsdb/tsdbCache.c index 161be8904ac3..208487b6f59c 100644 --- a/source/dnode/vnode/src/tsdb/tsdbCache.c +++ b/source/dnode/vnode/src/tsdb/tsdbCache.c @@ -330,94 +330,83 @@ static void rocksMayWrite(STsdb *pTsdb, bool force, bool read, bool lock) { } } -static SLastColV0 *tsdbCacheDeserializeV0(char const *value, size_t *inOutOffset) { - SLastColV0 *pLastColV0 = NULL; - size_t localOffset = 0; - - if (!value) { - goto _OUT; - } - - pLastColV0 = taosMemoryMalloc(sizeof(SLastColV0)); - *pLastColV0 = *(SLastColV0 *)(value); - - localOffset = sizeof(*pLastColV0); - - SColValV0 *pColValV0 = &pLastColV0->colVal; - if (IS_VAR_DATA_TYPE(pColValV0->type)) { - if (pColValV0->value.nData > 0) { - pColValV0->value.pData = (char *)value + localOffset; - localOffset += pColValV0->value.nData; - } else { - pColValV0->value.pData = NULL; - } - } - -_OUT: - *inOutOffset += localOffset; - return pLastColV0; -} - -static SLastCol *tsdbCacheDeserialize(char const *value, size_t size) { - if (!value) { - return NULL; - } - - size_t offset = 0; - - SLastColV0 *pLastColV0 = tsdbCacheDeserializeV0(value, &offset); - if (NULL == pLastColV0) { - return NULL; - } +typedef struct { + TSKEY ts; + int8_t dirty; + struct { + int16_t cid; + int8_t type; + int8_t flag; + union { + int64_t val; + struct { + uint32_t nData; + uint8_t *pData; + }; + } value; + } colVal; +} SLastColV0; + +static int32_t tsdbCacheDeserializeV0(char const *value, SLastCol *pLastCol) { + SLastColV0 *pLastColV0 = (SLastColV0 *)value; - SLastCol *pLastCol = taosMemoryMalloc(sizeof(SLastCol)); pLastCol->rowKey.ts = pLastColV0->ts; + pLastCol->rowKey.numOfPKs = 0; pLastCol->dirty = pLastColV0->dirty; pLastCol->colVal.cid = pLastColV0->colVal.cid; pLastCol->colVal.flag = pLastColV0->colVal.flag; pLastCol->colVal.value.type = pLastColV0->colVal.type; + if (IS_VAR_DATA_TYPE(pLastCol->colVal.value.type)) { pLastCol->colVal.value.nData = pLastColV0->colVal.value.nData; - pLastCol->colVal.value.pData = pLastColV0->colVal.value.pData; + pLastCol->colVal.value.pData = (uint8_t *)(&pLastColV0[1]); + return sizeof(SLastColV0) + pLastColV0->colVal.value.nData; } else { pLastCol->colVal.value.val = pLastColV0->colVal.value.val; + return sizeof(SLastColV0); + } +} + +static SLastCol *tsdbCacheDeserialize(char const *value, size_t size) { + if (!value) { + return NULL; } - taosMemoryFreeClear(pLastColV0); + SLastCol* pLastCol = taosMemoryCalloc(1, sizeof(SLastCol)); + if (NULL == pLastCol) { + return NULL; + } + int32_t offset = tsdbCacheDeserializeV0(value, pLastCol); if (offset == size) { // version 0 - pLastCol->version = LAST_COL_VERSION; - pLastCol->rowKey.numOfPKs = 0; - memset(pLastCol->rowKey.pks, 0, sizeof(pLastCol->rowKey.pks)); return pLastCol; + } else if (offset > size) { + terrno = TSDB_CODE_INVALID_DATA_FMT; + taosMemoryFreeClear(pLastCol); + return NULL; } - pLastCol->version = *(int8_t *)(value + offset); + // version + int8_t version = *(int8_t *)(value + offset); offset += sizeof(int8_t); + // numOfPKs pLastCol->rowKey.numOfPKs = *(uint8_t *)(value + offset); offset += sizeof(uint8_t); - for (int8_t i = 0; i < pLastCol->rowKey.numOfPKs; i++) { - SValue *pValue = &pLastCol->rowKey.pks[i]; - *pValue = *(SValue *)(value + offset); + // pks + for (int32_t i = 0; i < pLastCol->rowKey.numOfPKs; i++) { + pLastCol->rowKey.pks[i] = *(SValue *)(value + offset); offset += sizeof(SValue); - if (IS_VAR_DATA_TYPE(pValue->type)) { - if (pValue->nData > 0) { - pValue->pData = (char *)value + offset; - offset += pValue->nData; - } else { - pValue->pData = NULL; - } - } else { - pValue->val = *(int64_t *)(value + offset); - offset += sizeof(int64_t); + if (IS_VAR_DATA_TYPE(pLastCol->rowKey.pks[i].type)) { + pLastCol->rowKey.pks[i].pData = (uint8_t *)value + offset; + offset += pLastCol->rowKey.pks[i].nData; } } - if (size < offset) { + if (offset > size) { terrno = TSDB_CODE_INVALID_DATA_FMT; taosMemoryFreeClear(pLastCol); return NULL; @@ -426,25 +415,6 @@ static SLastCol *tsdbCacheDeserialize(char const *value, size_t size) { return pLastCol; } -static uint32_t tsdbCacheCopyVarData(SValue *from, SValue *to) { - ASSERT(from->nData >= 0); - if (from->nData > 0) { - memcpy(to->pData, from->pData, from->nData); - } - to->type = from->type; - to->nData = from->nData; - return from->nData; -} - -static uint32_t tsdbCacheCopyVarDataToV0(SValue *from, SValueV0 *to) { - ASSERT(from->nData >= 0); - if (from->nData > 0) { - memcpy(to->pData, from->pData, from->nData); - } - to->nData = from->nData; - return from->nData; -} - /* typedef struct { SLastColV0 lastColV0; @@ -458,61 +428,61 @@ typedef struct { ... } SLastColDisk; */ -static void tsdbCacheSerialize(SLastCol *pLastCol, char **value, size_t *size) { - SColVal *pColVal = &pLastCol->colVal; - size_t length = sizeof(SLastColV0); - if (IS_VAR_DATA_TYPE(pColVal->value.type)) { - length += pColVal->value.nData; +static int32_t tsdbCacheSerializeV0(char const *value, SLastCol *pLastCol) { + SLastColV0 *pLastColV0 = (SLastColV0 *)value; + + pLastColV0->ts = pLastCol->rowKey.ts; + pLastColV0->dirty = pLastCol->dirty; + pLastColV0->colVal.cid = pLastCol->colVal.cid; + pLastColV0->colVal.flag = pLastCol->colVal.flag; + pLastColV0->colVal.type = pLastCol->colVal.value.type; + if (IS_VAR_DATA_TYPE(pLastCol->colVal.value.type)) { + pLastColV0->colVal.value.nData = pLastCol->colVal.value.nData; + memcpy(&pLastColV0[1], pLastCol->colVal.value.pData, pLastCol->colVal.value.nData); + return sizeof(SLastColV0) + pLastCol->colVal.value.nData; + } else { + pLastColV0->colVal.value.val = pLastCol->colVal.value.val; + return sizeof(SLastColV0); } - uint8_t numOfPKs = pLastCol->rowKey.numOfPKs; + return 0; +} - length += sizeof(int8_t) + sizeof(uint8_t) + (sizeof(SValue) * numOfPKs); // version + numOfPKs + pks +static void tsdbCacheSerialize(SLastCol *pLastCol, char **value, size_t *size) { + *size = sizeof(SLastColV0); + if (IS_VAR_DATA_TYPE(pLastCol->colVal.value.type)) { + *size += pLastCol->colVal.value.nData; + } + *size += sizeof(uint8_t) + sizeof(uint8_t); // version + numOfPKs for (int8_t i = 0; i < pLastCol->rowKey.numOfPKs; i++) { + *size += sizeof(SValue); if (IS_VAR_DATA_TYPE(pLastCol->rowKey.pks[i].type)) { - length += pLastCol->rowKey.pks[i].nData; + *size += pLastCol->rowKey.pks[i].nData; } } - *value = taosMemoryMalloc(length); + *value = taosMemoryMalloc(*size); - // copy last col - SLastColV0 *pToLastColV0 = (SLastColV0 *)(*value); - pToLastColV0->ts = pLastCol->rowKey.ts; - pToLastColV0->dirty = pLastCol->dirty; - pToLastColV0->colVal.cid = pColVal->cid; - pToLastColV0->colVal.flag = pColVal->flag; - pToLastColV0->colVal.type = pColVal->value.type; + int32_t offset = tsdbCacheSerializeV0(*value, pLastCol); - char *currentPos = *value + sizeof(*pToLastColV0); + // version + ((uint8_t *)(*value + offset))[0] = LAST_COL_VERSION; + offset++; - // copy var data value - if (IS_VAR_DATA_TYPE(pColVal->value.type)) { - SValue *pFromValue = &pColVal->value; - SValueV0 *pToValue = &pToLastColV0->colVal.value; - pToValue->pData = (pFromValue->nData == 0) ? NULL : currentPos; - currentPos += tsdbCacheCopyVarDataToV0(pFromValue, pToValue); - } else { - pToLastColV0->colVal.value.val = pColVal->value.val; - } - - *(int8_t *)currentPos = LAST_COL_VERSION; - currentPos += sizeof(int8_t); + // numOfPKs + ((uint8_t *)(*value + offset))[0] = pLastCol->rowKey.numOfPKs; + offset++; - // copy var data pks + // pks for (int8_t i = 0; i < pLastCol->rowKey.numOfPKs; i++) { - SValue *pFromValue = &pLastCol->rowKey.pks[i]; - SValue *pToValue = (SValue *)currentPos; - *pToValue = *pFromValue; - currentPos += sizeof(SValue); - if (IS_VAR_DATA_TYPE(pFromValue->type)) { - pToValue->pData = (pFromValue->nData == 0) ? NULL : currentPos; - currentPos += tsdbCacheCopyVarData(pFromValue, pToValue); + ((SValue *)(*value + offset))[0] = pLastCol->rowKey.pks[i]; + offset += sizeof(SValue); + if (IS_VAR_DATA_TYPE(pLastCol->rowKey.pks[i].type)) { + memcpy(*value + offset, pLastCol->rowKey.pks[i].pData, pLastCol->rowKey.pks[i].nData); + offset += pLastCol->rowKey.pks[i].nData; } } - - *size = length; } static void tsdbCachePutBatch(SLastCol *pLastCol, const void *key, size_t klen, SCacheFlushState *state) { From d49bb0bb6e7091b44289d253e2dd32a2898f4662 Mon Sep 17 00:00:00 2001 From: Shungang Li Date: Thu, 9 May 2024 22:54:10 +0800 Subject: [PATCH 29/30] fix: tsdbCacheSerialize copy from null ptr --- source/dnode/vnode/src/tsdb/tsdbCache.c | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/source/dnode/vnode/src/tsdb/tsdbCache.c b/source/dnode/vnode/src/tsdb/tsdbCache.c index 208487b6f59c..4fc393ee17d7 100644 --- a/source/dnode/vnode/src/tsdb/tsdbCache.c +++ b/source/dnode/vnode/src/tsdb/tsdbCache.c @@ -438,7 +438,9 @@ static int32_t tsdbCacheSerializeV0(char const *value, SLastCol *pLastCol) { pLastColV0->colVal.type = pLastCol->colVal.value.type; if (IS_VAR_DATA_TYPE(pLastCol->colVal.value.type)) { pLastColV0->colVal.value.nData = pLastCol->colVal.value.nData; - memcpy(&pLastColV0[1], pLastCol->colVal.value.pData, pLastCol->colVal.value.nData); + if (pLastCol->colVal.value.nData > 0) { + memcpy(&pLastColV0[1], pLastCol->colVal.value.pData, pLastCol->colVal.value.nData); + } return sizeof(SLastColV0) + pLastCol->colVal.value.nData; } else { pLastColV0->colVal.value.val = pLastCol->colVal.value.val; @@ -479,7 +481,9 @@ static void tsdbCacheSerialize(SLastCol *pLastCol, char **value, size_t *size) { ((SValue *)(*value + offset))[0] = pLastCol->rowKey.pks[i]; offset += sizeof(SValue); if (IS_VAR_DATA_TYPE(pLastCol->rowKey.pks[i].type)) { - memcpy(*value + offset, pLastCol->rowKey.pks[i].pData, pLastCol->rowKey.pks[i].nData); + if (pLastCol->rowKey.pks[i].nData > 0) { + memcpy(*value + offset, pLastCol->rowKey.pks[i].pData, pLastCol->rowKey.pks[i].nData); + } offset += pLastCol->rowKey.pks[i].nData; } } From 61fd509ff50f50460e6dc4951ba774f4b11b501e Mon Sep 17 00:00:00 2001 From: Shungang Li Date: Thu, 9 May 2024 23:34:03 +0800 Subject: [PATCH 30/30] fix: tsdbcache last disk data format change --- source/dnode/vnode/src/tsdb/tsdbCache.c | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/source/dnode/vnode/src/tsdb/tsdbCache.c b/source/dnode/vnode/src/tsdb/tsdbCache.c index 4fc393ee17d7..6f26e3a63f76 100644 --- a/source/dnode/vnode/src/tsdb/tsdbCache.c +++ b/source/dnode/vnode/src/tsdb/tsdbCache.c @@ -1106,7 +1106,8 @@ int32_t tsdbCacheUpdate(STsdb *pTsdb, tb_uid_t suid, tb_uid_t uid, TSDBROW *pRow if (NULL == pLastCol || cmp_res < 0 || (cmp_res == 0 && !COL_VAL_IS_NONE(pColVal))) { char *value = NULL; size_t vlen = 0; - tsdbCacheSerialize(&(SLastCol){.rowKey = *pRowKey, .colVal = *pColVal}, &value, &vlen); + SLastCol lastColTmp = {.rowKey = *pRowKey, .colVal = *pColVal}; + tsdbCacheSerialize(&lastColTmp, &value, &vlen); taosThreadMutexLock(&pTsdb->rCache.rMutex); @@ -1114,7 +1115,7 @@ int32_t tsdbCacheUpdate(STsdb *pTsdb, tb_uid_t suid, tb_uid_t uid, TSDBROW *pRow taosThreadMutexUnlock(&pTsdb->rCache.rMutex); - pLastCol = (SLastCol *)value; + pLastCol = &lastColTmp; SLastCol *pTmpLastCol = taosMemoryCalloc(1, sizeof(SLastCol)); *pTmpLastCol = *pLastCol; pLastCol = pTmpLastCol; @@ -1146,7 +1147,8 @@ int32_t tsdbCacheUpdate(STsdb *pTsdb, tb_uid_t suid, tb_uid_t uid, TSDBROW *pRow if (NULL == pLastCol || (tRowKeyCompare(&pLastCol->rowKey, pRowKey) != 1)) { char *value = NULL; size_t vlen = 0; - tsdbCacheSerialize(&(SLastCol){.rowKey = *pRowKey, .colVal = *pColVal}, &value, &vlen); + SLastCol lastColTmp = {.rowKey = *pRowKey, .colVal = *pColVal}; + tsdbCacheSerialize(&lastColTmp, &value, &vlen); taosThreadMutexLock(&pTsdb->rCache.rMutex); @@ -1154,7 +1156,7 @@ int32_t tsdbCacheUpdate(STsdb *pTsdb, tb_uid_t suid, tb_uid_t uid, TSDBROW *pRow taosThreadMutexUnlock(&pTsdb->rCache.rMutex); - pLastCol = (SLastCol *)value; + pLastCol = &lastColTmp; SLastCol *pTmpLastCol = taosMemoryCalloc(1, sizeof(SLastCol)); *pTmpLastCol = *pLastCol; pLastCol = pTmpLastCol;