From 2977a4d07f18ffb1ef9bfc3280a64b7d04e659a6 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 30 May 2023 09:41:19 +0800 Subject: [PATCH 01/83] refactor: do some internal refactor. --- source/dnode/mnode/impl/inc/mndDef.h | 23 +++-- source/dnode/mnode/impl/src/mndDef.c | 20 ++--- source/dnode/mnode/impl/src/mndDump.c | 8 +- source/dnode/mnode/impl/src/mndScheduler.c | 95 ++++++++++---------- source/dnode/mnode/impl/src/mndSma.c | 20 ++--- source/dnode/mnode/impl/src/mndStream.c | 25 +++--- source/libs/stream/src/streamTask.c | 100 +++++++++++++++++++++ 7 files changed, 199 insertions(+), 92 deletions(-) diff --git a/source/dnode/mnode/impl/inc/mndDef.h b/source/dnode/mnode/impl/inc/mndDef.h index 82b714e6eb..5c288ec9e5 100644 --- a/source/dnode/mnode/impl/inc/mndDef.h +++ b/source/dnode/mnode/impl/inc/mndDef.h @@ -637,6 +637,14 @@ typedef struct { SMqSubActionLogEntry* pLogEntry; } SMqRebOutputObj; +typedef struct SStreamConf { + int8_t igExpired; + int8_t trigger; + int8_t fillHistory; + int64_t triggerParam; + int64_t watermark; +} SStreamConf; + typedef struct { char name[TSDB_STREAM_FNAME_LEN]; // ctl @@ -650,12 +658,7 @@ typedef struct { // info int64_t uid; int8_t status; - // config - int8_t igExpired; - int8_t trigger; - int8_t fillHistory; - int64_t triggerParam; - int64_t watermark; + SStreamConf conf; // source and target int64_t sourceDbUid; int64_t targetDbUid; @@ -665,14 +668,18 @@ typedef struct { int64_t targetStbUid; // fixedSinkVg is not applicable for encode and decode - SVgObj fixedSinkVg; + SVgObj fixedSinkVg; int32_t fixedSinkVgId; // 0 for shuffle // transformation char* sql; char* ast; char* physicalPlan; - SArray* tasks; // SArray> + SArray* tasks; // SArray> + + SArray* pBatchTask; // generate the results for already stored ts data + int64_t batchTaskUid; // stream task for history ts data + SSchemaWrapper outputSchema; SSchemaWrapper tagSchema; diff --git a/source/dnode/mnode/impl/src/mndDef.c b/source/dnode/mnode/impl/src/mndDef.c index 6dab018236..589d35afb8 100644 --- a/source/dnode/mnode/impl/src/mndDef.c +++ b/source/dnode/mnode/impl/src/mndDef.c @@ -30,11 +30,11 @@ int32_t tEncodeSStreamObj(SEncoder *pEncoder, const SStreamObj *pObj) { if (tEncodeI64(pEncoder, pObj->uid) < 0) return -1; if (tEncodeI8(pEncoder, pObj->status) < 0) return -1; - if (tEncodeI8(pEncoder, pObj->igExpired) < 0) return -1; - if (tEncodeI8(pEncoder, pObj->trigger) < 0) return -1; - if (tEncodeI8(pEncoder, pObj->fillHistory) < 0) return -1; - if (tEncodeI64(pEncoder, pObj->triggerParam) < 0) return -1; - if (tEncodeI64(pEncoder, pObj->watermark) < 0) return -1; + if (tEncodeI8(pEncoder, pObj->conf.igExpired) < 0) return -1; + if (tEncodeI8(pEncoder, pObj->conf.trigger) < 0) return -1; + if (tEncodeI8(pEncoder, pObj->conf.fillHistory) < 0) return -1; + if (tEncodeI64(pEncoder, pObj->conf.triggerParam) < 0) return -1; + if (tEncodeI64(pEncoder, pObj->conf.watermark) < 0) return -1; if (tEncodeI64(pEncoder, pObj->sourceDbUid) < 0) return -1; if (tEncodeI64(pEncoder, pObj->targetDbUid) < 0) return -1; @@ -97,11 +97,11 @@ int32_t tDecodeSStreamObj(SDecoder *pDecoder, SStreamObj *pObj, int32_t sver) { if (tDecodeI64(pDecoder, &pObj->uid) < 0) return -1; if (tDecodeI8(pDecoder, &pObj->status) < 0) return -1; - if (tDecodeI8(pDecoder, &pObj->igExpired) < 0) return -1; - if (tDecodeI8(pDecoder, &pObj->trigger) < 0) return -1; - if (tDecodeI8(pDecoder, &pObj->fillHistory) < 0) return -1; - if (tDecodeI64(pDecoder, &pObj->triggerParam) < 0) return -1; - if (tDecodeI64(pDecoder, &pObj->watermark) < 0) return -1; + if (tDecodeI8(pDecoder, &pObj->conf.igExpired) < 0) return -1; + if (tDecodeI8(pDecoder, &pObj->conf.trigger) < 0) return -1; + if (tDecodeI8(pDecoder, &pObj->conf.fillHistory) < 0) return -1; + if (tDecodeI64(pDecoder, &pObj->conf.triggerParam) < 0) return -1; + if (tDecodeI64(pDecoder, &pObj->conf.watermark) < 0) return -1; if (tDecodeI64(pDecoder, &pObj->sourceDbUid) < 0) return -1; if (tDecodeI64(pDecoder, &pObj->targetDbUid) < 0) return -1; diff --git a/source/dnode/mnode/impl/src/mndDump.c b/source/dnode/mnode/impl/src/mndDump.c index d57053bb5b..62b5cb00e6 100644 --- a/source/dnode/mnode/impl/src/mndDump.c +++ b/source/dnode/mnode/impl/src/mndDump.c @@ -367,10 +367,10 @@ void dumpStream(SSdb *pSdb, SJson *json) { tjsonAddStringToObject(item, "smaId", i642str(pObj->smaId)); tjsonAddStringToObject(item, "uid", i642str(pObj->uid)); tjsonAddStringToObject(item, "status", i642str(pObj->status)); - tjsonAddStringToObject(item, "igExpired", i642str(pObj->igExpired)); - tjsonAddStringToObject(item, "trigger", i642str(pObj->trigger)); - tjsonAddStringToObject(item, "triggerParam", i642str(pObj->triggerParam)); - tjsonAddStringToObject(item, "watermark", i642str(pObj->watermark)); + tjsonAddStringToObject(item, "igExpired", i642str(pObj->conf.igExpired)); + tjsonAddStringToObject(item, "trigger", i642str(pObj->conf.trigger)); + tjsonAddStringToObject(item, "triggerParam", i642str(pObj->conf.triggerParam)); + tjsonAddStringToObject(item, "watermark", i642str(pObj->conf.watermark)); tjsonAddStringToObject(item, "sourceDbUid", i642str(pObj->sourceDbUid)); tjsonAddStringToObject(item, "targetDbUid", i642str(pObj->targetDbUid)); tjsonAddStringToObject(item, "sourceDb", mndGetDbStr(pObj->sourceDb)); diff --git a/source/dnode/mnode/impl/src/mndScheduler.c b/source/dnode/mnode/impl/src/mndScheduler.c index df8c11a6f6..a6a9c9b6e2 100644 --- a/source/dnode/mnode/impl/src/mndScheduler.c +++ b/source/dnode/mnode/impl/src/mndScheduler.c @@ -22,9 +22,10 @@ #include "tname.h" #include "tuuid.h" +#define SINK_NODE_LEVEL (0) extern bool tsDeployOnSnode; -static int32_t mndAddSinkTaskToStream(SStreamObj* pStream, SMnode* pMnode, int32_t vgId, SVgObj* pVgroup); +static int32_t mndAddSinkTaskToStream(SStreamObj* pStream, SArray* pTaskList, SMnode* pMnode, int32_t vgId, SVgObj* pVgroup); static void setFixedDownstreamEpInfo(SStreamTask* pDstTask, const SStreamTask* pTask); int32_t mndConvertRsmaTask(char** pDst, int32_t* pDstLen, const char* ast, int64_t uid, int8_t triggerType, @@ -100,9 +101,7 @@ int32_t mndSetSinkTaskInfo(SStreamObj* pStream, SStreamTask* pTask) { return 0; } -#define SINK_NODE_LEVEL (0) - -int32_t mndAddDispatcherForInnerTask(SMnode* pMnode, SStreamObj* pStream, SStreamTask* pTask) { +int32_t mndAddDispatcherForInnerTask(SMnode* pMnode, SStreamObj* pStream, SArray* pSinkNodeList, SStreamTask* pTask) { bool isShuffle = false; if (pStream->fixedSinkVgId == 0) { @@ -120,7 +119,6 @@ int32_t mndAddDispatcherForInnerTask(SMnode* pMnode, SStreamObj* pStream, SStrea sdbRelease(pMnode->pSdb, pDb); } - SArray* pSinkNodeList = taosArrayGetP(pStream->tasks, SINK_NODE_LEVEL); int32_t numOfSinkNodes = taosArrayGetSize(pSinkNodeList); if (isShuffle) { @@ -187,6 +185,7 @@ int32_t mndAssignTaskToSnode(SMnode* pMnode, SStreamTask* pTask, SSubplan* plan, return 0; } +// todo random choose a node to do compute SVgObj* mndSchedFetchOneVg(SMnode* pMnode, int64_t dbUid) { void* pIter = NULL; SVgObj* pVgroup = NULL; @@ -203,7 +202,7 @@ SVgObj* mndSchedFetchOneVg(SMnode* pMnode, int64_t dbUid) { } // create sink node for each vgroup. -int32_t mndAddShuffleSinkTasksToStream(SMnode* pMnode, SStreamObj* pStream) { +int32_t mndAddShuffleSinkTasksToStream(SMnode* pMnode, SArray* pTaskList, SStreamObj* pStream) { SSdb* pSdb = pMnode->pSdb; void* pIter = NULL; @@ -219,17 +218,15 @@ int32_t mndAddShuffleSinkTasksToStream(SMnode* pMnode, SStreamObj* pStream) { continue; } - mndAddSinkTaskToStream(pStream, pMnode, pVgroup->vgId, pVgroup); + mndAddSinkTaskToStream(pStream, pTaskList, pMnode, pVgroup->vgId, pVgroup); sdbRelease(pSdb, pVgroup); } return 0; } -int32_t mndAddSinkTaskToStream(SStreamObj* pStream, SMnode* pMnode, int32_t vgId, SVgObj* pVgroup) { - SArray* pTaskList = taosArrayGetP(pStream->tasks, SINK_NODE_LEVEL); - - SStreamTask* pTask = tNewStreamTask(pStream->uid, TASK_LEVEL__SINK, pStream->fillHistory, 0, pTaskList); +int32_t mndAddSinkTaskToStream(SStreamObj* pStream, SArray* pTaskList, SMnode* pMnode, int32_t vgId, SVgObj* pVgroup) { + SStreamTask* pTask = tNewStreamTask(pStream->uid, TASK_LEVEL__SINK, pStream->conf.fillHistory, 0, pTaskList); if (pTask == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; return -1; @@ -245,17 +242,18 @@ static int32_t mndScheduleFillHistoryStreamTask(SMnode* pMnode, SStreamObj* pStr return 0; } -static int32_t addSourceStreamTask(SMnode* pMnode, SVgObj* pVgroup, SArray* pTaskList, SStreamObj* pStream, +static int32_t addSourceStreamTask(SMnode* pMnode, SVgObj* pVgroup, SArray* pTaskList, SArray* pSinkTaskList, SStreamObj* pStream, SSubplan* plan, uint64_t uid, int8_t taskLevel, int8_t fillHistory, bool hasExtraSink) { - SStreamTask* pTask = tNewStreamTask(uid, taskLevel, fillHistory, pStream->triggerParam, pTaskList); + SStreamTask* pTask = tNewStreamTask(uid, taskLevel, fillHistory, pStream->conf.triggerParam, pTaskList); if (pTask == NULL) { return terrno; } // sink or dispatch if (hasExtraSink) { - mndAddDispatcherForInnerTask(pMnode, pStream, pTask); + + mndAddDispatcherForInnerTask(pMnode, pStream, pSinkTaskList, pTask); } else { mndSetSinkTaskInfo(pStream, pTask); } @@ -292,31 +290,23 @@ void setFixedDownstreamEpInfo(SStreamTask* pDstTask, const SStreamTask* pTask) { pDstTask->dispatchMsgType = TDMT_STREAM_TASK_DISPATCH; } -int32_t appendToUpstream(SStreamTask* pTask, SStreamTask* pUpstream) { +int32_t appendToDownstream(SStreamTask* pTask, SStreamTask* pDownstream) { SStreamChildEpInfo* pEpInfo = createStreamTaskEpInfo(pTask); if (pEpInfo == NULL) { return TSDB_CODE_OUT_OF_MEMORY; } - if(pUpstream->childEpInfo == NULL) { - pUpstream->childEpInfo = taosArrayInit(4, POINTER_BYTES); + if(pDownstream->childEpInfo == NULL) { + pDownstream->childEpInfo = taosArrayInit(4, POINTER_BYTES); } - taosArrayPush(pUpstream->childEpInfo, &pEpInfo); + taosArrayPush(pDownstream->childEpInfo, &pEpInfo); return TSDB_CODE_SUCCESS; } -int32_t mndScheduleStream(SMnode* pMnode, SStreamObj* pStream) { +static int32_t doScheduleStream(uint64_t uid, SArray* pTasksList, SStreamObj* pStream, SMnode* pMnode, SQueryPlan* pPlan) { SSdb* pSdb = pMnode->pSdb; - - SQueryPlan* pPlan = qStringToQueryPlan(pStream->physicalPlan); - if (pPlan == NULL) { - terrno = TSDB_CODE_QRY_INVALID_INPUT; - return -1; - } - int32_t planTotLevel = LIST_LENGTH(pPlan->pSubplans); - pStream->tasks = taosArrayInit(planTotLevel, POINTER_BYTES); bool hasExtraSink = false; bool externalTargetDB = strcmp(pStream->sourceDb, pStream->targetDb) != 0; @@ -331,17 +321,17 @@ int32_t mndScheduleStream(SMnode* pMnode, SStreamObj* pStream) { if (planTotLevel == 2 || externalTargetDB || multiTarget || pStream->fixedSinkVgId) { SArray* taskOneLevel = taosArrayInit(0, POINTER_BYTES); - taosArrayPush(pStream->tasks, &taskOneLevel); + taosArrayPush(pTasksList, &taskOneLevel); // add extra sink hasExtraSink = true; if (pStream->fixedSinkVgId == 0) { - if (mndAddShuffleSinkTasksToStream(pMnode, pStream) < 0) { + if (mndAddShuffleSinkTasksToStream(pMnode, taskOneLevel, pStream) < 0) { // TODO free return -1; } } else { - if (mndAddSinkTaskToStream(pStream, pMnode, pStream->fixedSinkVgId, &pStream->fixedSinkVg) < 0) { + if (mndAddSinkTaskToStream(pStream, taskOneLevel, pMnode, pStream->fixedSinkVgId, &pStream->fixedSinkVg) < 0) { // TODO free return -1; } @@ -355,7 +345,7 @@ int32_t mndScheduleStream(SMnode* pMnode, SStreamObj* pStream) { // inner level { SArray* taskInnerLevel = taosArrayInit(0, POINTER_BYTES); - taosArrayPush(pStream->tasks, &taskInnerLevel); + taosArrayPush(pTasksList, &taskInnerLevel); SNodeListNode* inner = (SNodeListNode*)nodesListGetNode(pPlan->pSubplans, 0); SSubplan* plan = (SSubplan*)nodesListGetNode(inner->pNodeList, 0); @@ -364,16 +354,15 @@ int32_t mndScheduleStream(SMnode* pMnode, SStreamObj* pStream) { return -1; } - pInnerTask = tNewStreamTask(pStream->uid, TASK_LEVEL__AGG, pStream->fillHistory, pStream->triggerParam, taskInnerLevel); + pInnerTask = tNewStreamTask(uid, TASK_LEVEL__AGG, pStream->conf.fillHistory, pStream->conf.triggerParam, taskInnerLevel); if (pInnerTask == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; - qDestroyQueryPlan(pPlan); return -1; } // dispatch - if (mndAddDispatcherForInnerTask(pMnode, pStream, pInnerTask) < 0) { - qDestroyQueryPlan(pPlan); + SArray* pSinkNodeList = taosArrayGet(pTasksList, SINK_NODE_LEVEL); + if (mndAddDispatcherForInnerTask(pMnode, pStream, pSinkNodeList, pInnerTask) < 0) { return -1; } @@ -383,14 +372,12 @@ int32_t mndScheduleStream(SMnode* pMnode, SStreamObj* pStream) { SVgObj* pVgroup = mndSchedFetchOneVg(pMnode, pStream->sourceDbUid); if (mndAssignStreamTaskToVgroup(pMnode, pInnerTask, plan, pVgroup) < 0) { sdbRelease(pSdb, pVgroup); - qDestroyQueryPlan(pPlan); return -1; } sdbRelease(pSdb, pVgroup); } else { if (mndAssignTaskToSnode(pMnode, pInnerTask, plan, pSnode) < 0) { sdbRelease(pSdb, pSnode); - qDestroyQueryPlan(pPlan); return -1; } } @@ -398,7 +385,6 @@ int32_t mndScheduleStream(SMnode* pMnode, SStreamObj* pStream) { SVgObj* pVgroup = mndSchedFetchOneVg(pMnode, pStream->sourceDbUid); if (mndAssignStreamTaskToVgroup(pMnode, pInnerTask, plan, pVgroup) < 0) { sdbRelease(pSdb, pVgroup); - qDestroyQueryPlan(pPlan); return -1; } @@ -408,7 +394,7 @@ int32_t mndScheduleStream(SMnode* pMnode, SStreamObj* pStream) { // source level SArray* taskSourceLevel = taosArrayInit(0, POINTER_BYTES); - taosArrayPush(pStream->tasks, &taskSourceLevel); + taosArrayPush(pTasksList, &taskSourceLevel); SNodeListNode* inner = (SNodeListNode*)nodesListGetNode(pPlan->pSubplans, 1); SSubplan* plan = (SSubplan*)nodesListGetNode(inner->pNodeList, 0); @@ -430,11 +416,10 @@ int32_t mndScheduleStream(SMnode* pMnode, SStreamObj* pStream) { continue; } - SStreamTask* pTask = tNewStreamTask(pStream->uid, TASK_LEVEL__SOURCE, pStream->fillHistory, 0, taskSourceLevel); + SStreamTask* pTask = tNewStreamTask(uid, TASK_LEVEL__SOURCE, pStream->conf.fillHistory, 0, taskSourceLevel); if (pTask == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; sdbRelease(pSdb, pVgroup); - qDestroyQueryPlan(pPlan); return -1; } @@ -443,23 +428,21 @@ int32_t mndScheduleStream(SMnode* pMnode, SStreamObj* pStream) { if (mndAssignStreamTaskToVgroup(pMnode, pTask, plan, pVgroup) < 0) { sdbRelease(pSdb, pVgroup); - qDestroyQueryPlan(pPlan); return -1; } - int32_t code = appendToUpstream(pTask, pInnerTask); + int32_t code = appendToDownstream(pTask, pInnerTask); sdbRelease(pSdb, pVgroup); if (code != TSDB_CODE_SUCCESS) { terrno = code; - qDestroyQueryPlan(pPlan); return -1; } } } else if (planTotLevel == 1) { // create exec stream task, since only one level, the exec task is also the source task SArray* pTaskList = taosArrayInit(0, POINTER_BYTES); - taosArrayPush(pStream->tasks, &pTaskList); + taosArrayPush(pTasksList, &pTaskList); SNodeListNode* inner = (SNodeListNode*)nodesListGetNode(pPlan->pSubplans, 0); if (LIST_LENGTH(inner->pNodeList) != 1) { @@ -487,20 +470,36 @@ int32_t mndScheduleStream(SMnode* pMnode, SStreamObj* pStream) { } // new stream task - int32_t code = addSourceStreamTask(pMnode, pVgroup, pTaskList, pStream, plan, pStream->uid, TASK_LEVEL__SOURCE, pStream->fillHistory, hasExtraSink); + SArray* pSinkNodeTaskList = taosArrayGet(pTasksList, SINK_NODE_LEVEL); + int32_t code = addSourceStreamTask(pMnode, pVgroup, pTaskList, pSinkNodeTaskList, pStream, plan, uid, TASK_LEVEL__SOURCE, pStream->conf.fillHistory, hasExtraSink); sdbRelease(pSdb, pVgroup); if (code != TSDB_CODE_SUCCESS) { - qDestroyQueryPlan(pPlan); return -1; } } } - qDestroyQueryPlan(pPlan); return 0; } +int32_t mndScheduleStream(SMnode* pMnode, SStreamObj* pStream) { + SQueryPlan* pPlan = qStringToQueryPlan(pStream->physicalPlan); + if (pPlan == NULL) { + terrno = TSDB_CODE_QRY_INVALID_INPUT; + return -1; + } + + int32_t code = doScheduleStream(pStream->uid, pStream->tasks, pStream, pMnode, pPlan); + + if (code == TSDB_CODE_SUCCESS) { + code = doScheduleStream(pStream->batchTaskUid, pStream->pBatchTask, pStream, pMnode, pPlan); + } + + qDestroyQueryPlan(pPlan); + return code; +} + int32_t mndSchedInitSubEp(SMnode* pMnode, const SMqTopicObj* pTopic, SMqSubscribeObj* pSub) { SSdb* pSdb = pMnode->pSdb; SVgObj* pVgroup = NULL; diff --git a/source/dnode/mnode/impl/src/mndSma.c b/source/dnode/mnode/impl/src/mndSma.c index 42ad9e24d5..42325cb926 100644 --- a/source/dnode/mnode/impl/src/mndSma.c +++ b/source/dnode/mnode/impl/src/mndSma.c @@ -555,20 +555,20 @@ static int32_t mndCreateSma(SMnode *pMnode, SRpcMsg *pReq, SMCreateSmaReq *pCrea streamObj.version = 1; streamObj.sql = taosStrdup(pCreate->sql); streamObj.smaId = smaObj.uid; - streamObj.watermark = pCreate->watermark; + streamObj.conf.watermark = pCreate->watermark; streamObj.deleteMark = pCreate->deleteMark; - streamObj.fillHistory = STREAM_FILL_HISTORY_ON; - streamObj.trigger = STREAM_TRIGGER_WINDOW_CLOSE; - streamObj.triggerParam = pCreate->maxDelay; + streamObj.conf.fillHistory = STREAM_FILL_HISTORY_ON; + streamObj.conf.trigger = STREAM_TRIGGER_WINDOW_CLOSE; + streamObj.conf.triggerParam = pCreate->maxDelay; streamObj.ast = taosStrdup(smaObj.ast); // check the maxDelay - if (streamObj.triggerParam < TSDB_MIN_ROLLUP_MAX_DELAY) { + if (streamObj.conf.triggerParam < TSDB_MIN_ROLLUP_MAX_DELAY) { int64_t msInterval = convertTimeFromPrecisionToUnit(pCreate->interval, pDb->cfg.precision, TIME_UNIT_MILLISECOND); - streamObj.triggerParam = msInterval > TSDB_MIN_ROLLUP_MAX_DELAY ? msInterval : TSDB_MIN_ROLLUP_MAX_DELAY; + streamObj.conf.triggerParam = msInterval > TSDB_MIN_ROLLUP_MAX_DELAY ? msInterval : TSDB_MIN_ROLLUP_MAX_DELAY; } - if (streamObj.triggerParam > TSDB_MAX_ROLLUP_MAX_DELAY) { - streamObj.triggerParam = TSDB_MAX_ROLLUP_MAX_DELAY; + if (streamObj.conf.triggerParam > TSDB_MAX_ROLLUP_MAX_DELAY) { + streamObj.conf.triggerParam = TSDB_MAX_ROLLUP_MAX_DELAY; } if (mndAllocSmaVgroup(pMnode, pDb, &streamObj.fixedSinkVg) != 0) { @@ -597,8 +597,8 @@ static int32_t mndCreateSma(SMnode *pMnode, SRpcMsg *pReq, SMCreateSmaReq *pCrea .pAstRoot = pAst, .topicQuery = false, .streamQuery = true, - .triggerType = streamObj.trigger, - .watermark = streamObj.watermark, + .triggerType = streamObj.conf.trigger, + .watermark = streamObj.conf.watermark, .deleteMark = streamObj.deleteMark, }; diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index 39a1fa223f..f5cd673225 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -239,7 +239,7 @@ static void mndShowStreamStatus(char *dst, SStreamObj *pStream) { } static void mndShowStreamTrigger(char *dst, SStreamObj *pStream) { - int8_t trigger = pStream->trigger; + int8_t trigger = pStream->conf.trigger; if (trigger == STREAM_TRIGGER_AT_ONCE) { strcpy(dst, "at once"); } else if (trigger == STREAM_TRIGGER_WINDOW_CLOSE) { @@ -301,11 +301,11 @@ static int32_t mndBuildStreamObjFromCreateReq(SMnode *pMnode, SStreamObj *pObj, pObj->uid = mndGenerateUid(pObj->name, strlen(pObj->name)); pObj->status = 0; - pObj->igExpired = pCreate->igExpired; - pObj->trigger = pCreate->triggerType; - pObj->triggerParam = pCreate->maxDelay; - pObj->watermark = pCreate->watermark; - pObj->fillHistory = pCreate->fillHistory; + pObj->conf.igExpired = pCreate->igExpired; + pObj->conf.trigger = pCreate->triggerType; + pObj->conf.triggerParam = pCreate->maxDelay; + pObj->conf.watermark = pCreate->watermark; + pObj->conf.fillHistory = pCreate->fillHistory; pObj->deleteMark = pCreate->deleteMark; pObj->igCheckUpdate = pCreate->igUpdate; @@ -387,9 +387,9 @@ static int32_t mndBuildStreamObjFromCreateReq(SMnode *pMnode, SStreamObj *pObj, .pAstRoot = pAst, .topicQuery = false, .streamQuery = true, - .triggerType = pObj->trigger == STREAM_TRIGGER_MAX_DELAY ? STREAM_TRIGGER_WINDOW_CLOSE : pObj->trigger, - .watermark = pObj->watermark, - .igExpired = pObj->igExpired, + .triggerType = pObj->conf.trigger == STREAM_TRIGGER_MAX_DELAY ? STREAM_TRIGGER_WINDOW_CLOSE : pObj->conf.trigger, + .watermark = pObj->conf.watermark, + .igExpired = pObj->conf.igExpired, .deleteMark = pObj->deleteMark, .igCheckUpdate = pObj->igCheckUpdate, }; @@ -459,8 +459,9 @@ int32_t mndPersistStreamTasks(SMnode *pMnode, STrans *pTrans, SStreamObj *pStrea int32_t level = taosArrayGetSize(pStream->tasks); for (int32_t i = 0; i < level; i++) { SArray *pLevel = taosArrayGetP(pStream->tasks, i); - int32_t sz = taosArrayGetSize(pLevel); - for (int32_t j = 0; j < sz; j++) { + + int32_t numOfTasks = taosArrayGetSize(pLevel); + for (int32_t j = 0; j < numOfTasks; j++) { SStreamTask *pTask = taosArrayGetP(pLevel, j); if (mndPersistTaskDeployReq(pTrans, pTask) < 0) { return -1; @@ -1157,7 +1158,7 @@ static int32_t mndRetrieveStream(SRpcMsg *pReq, SShowObj *pShow, SSDataBlock *pB } pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); - colDataSetVal(pColInfo, numOfRows, (const char *)&pStream->watermark, false); + colDataSetVal(pColInfo, numOfRows, (const char *)&pStream->conf.watermark, false); char trigger[20 + VARSTR_HEADER_SIZE] = {0}; char trigger2[20] = {0}; diff --git a/source/libs/stream/src/streamTask.c b/source/libs/stream/src/streamTask.c index a0caffd41f..6d8ec11f44 100644 --- a/source/libs/stream/src/streamTask.c +++ b/source/libs/stream/src/streamTask.c @@ -49,6 +49,106 @@ SStreamTask* tNewStreamTask(int64_t streamId, int8_t taskLevel, int8_t fillHisto return pTask; } +SStreamTask* streamTaskClone(SStreamTask* pTask) { + SStreamTask* pDst = taosMemoryCalloc(1, sizeof(SStreamTask)); + /* pDst-> + + SStreamId id; + int32_t totalLevel; + int8_t taskLevel; + int8_t outputType; + int16_t dispatchMsgType; + SStreamStatus status; + int32_t selfChildId; + int32_t nodeId; // vgroup id + SEpSet epSet; + SCheckpointInfo chkInfo; + STaskExec exec; + int8_t fillHistory; // fill history + int64_t ekey; // end ts key + int64_t endVer; // end version + + // children info + SArray* childEpInfo; // SArray + int32_t nextCheckId; + SArray* checkpointInfo; // SArray + + // output + union { + STaskDispatcherFixedEp fixedEpDispatcher; + STaskDispatcherShuffle shuffleDispatcher; + STaskSinkTb tbSink; + STaskSinkSma smaSink; + STaskSinkFetch fetchSink; + }; + + int8_t inputStatus; + int8_t outputStatus; + SStreamQueue* inputQueue; + SStreamQueue* outputQueue; + + // trigger + int8_t triggerStatus; + int64_t triggerParam; + void* timer; + SMsgCb* pMsgCb; // msg handle + SStreamState* pState; // state backend + + // the followings attributes don't be serialized + int32_t recoverTryingDownstream; + int32_t recoverWaitingUpstream; + int64_t checkReqId; + SArray* checkReqIds; // shuffle + int32_t refCnt; + int64_t checkpointingId; + int32_t checkpointAlignCnt; + struct SStreamMeta* pMeta; + + int32_t status = atomic_load_8((int8_t*)&(pTask->status.taskStatus)); + if (pTask->inputQueue) { + streamQueueClose(pTask->inputQueue); + } + if (pTask->outputQueue) { + streamQueueClose(pTask->outputQueue); + } + if (pTask->exec.qmsg) { + taosMemoryFree(pTask->exec.qmsg); + } + + if (pTask->exec.pExecutor) { + qDestroyTask(pTask->exec.pExecutor); + pTask->exec.pExecutor = NULL; + } + + if (pTask->exec.pWalReader != NULL) { + walCloseReader(pTask->exec.pWalReader); + } + + taosArrayDestroyP(pTask->childEpInfo, taosMemoryFree); + if (pTask->outputType == TASK_OUTPUT__TABLE) { + tDeleteSchemaWrapper(pTask->tbSink.pSchemaWrapper); + taosMemoryFree(pTask->tbSink.pTSchema); + tSimpleHashCleanup(pTask->tbSink.pTblInfo); + } + + if (pTask->outputType == TASK_OUTPUT__SHUFFLE_DISPATCH) { + taosArrayDestroy(pTask->shuffleDispatcher.dbInfo.pVgroupInfos); + taosArrayDestroy(pTask->checkReqIds); + pTask->checkReqIds = NULL; + } + + if (pTask->pState) { + streamStateClose(pTask->pState, status == TASK_STATUS__DROPPING); + } + + if (pTask->id.idStr != NULL) { + taosMemoryFree((void*)pTask->id.idStr); + } + + taosMemoryFree(pTask);*/ + return NULL; +} + int32_t tEncodeStreamEpInfo(SEncoder* pEncoder, const SStreamChildEpInfo* pInfo) { if (tEncodeI32(pEncoder, pInfo->taskId) < 0) return -1; if (tEncodeI32(pEncoder, pInfo->nodeId) < 0) return -1; From be8fd9e48bb25aaf49008aeab8c6f39186a081cd Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 30 May 2023 14:47:10 +0800 Subject: [PATCH 02/83] enh(stream): create additional task for history data processing. --- include/libs/stream/tstream.h | 3 +- source/dnode/mnode/impl/inc/mndDef.h | 4 +- source/dnode/mnode/impl/src/mndScheduler.c | 447 +++++++++++++-------- source/dnode/mnode/impl/src/mndStream.c | 30 ++ 4 files changed, 320 insertions(+), 164 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 1d4bbf073e..6d54790b2f 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -289,9 +289,10 @@ struct SStreamTask { SCheckpointInfo chkInfo; STaskExec exec; int8_t fillHistory; // fill history + int64_t ekey; // end ts key int64_t endVer; // end version - + SStreamId historyTaskId; // children info SArray* childEpInfo; // SArray int32_t nextCheckId; diff --git a/source/dnode/mnode/impl/inc/mndDef.h b/source/dnode/mnode/impl/inc/mndDef.h index 5c288ec9e5..0343e91ac2 100644 --- a/source/dnode/mnode/impl/inc/mndDef.h +++ b/source/dnode/mnode/impl/inc/mndDef.h @@ -677,8 +677,8 @@ typedef struct { char* physicalPlan; SArray* tasks; // SArray> - SArray* pBatchTask; // generate the results for already stored ts data - int64_t batchTaskUid; // stream task for history ts data + SArray* pHTasksList; // generate the results for already stored ts data + int64_t hTaskUid; // stream task for history ts data SSchemaWrapper outputSchema; SSchemaWrapper tagSchema; diff --git a/source/dnode/mnode/impl/src/mndScheduler.c b/source/dnode/mnode/impl/src/mndScheduler.c index a6a9c9b6e2..e8cb8604ea 100644 --- a/source/dnode/mnode/impl/src/mndScheduler.c +++ b/source/dnode/mnode/impl/src/mndScheduler.c @@ -169,7 +169,7 @@ SSnodeObj* mndSchedFetchOneSnode(SMnode* pMnode) { return pObj; } -int32_t mndAssignTaskToSnode(SMnode* pMnode, SStreamTask* pTask, SSubplan* plan, const SSnodeObj* pSnode) { +int32_t mndAssignStreamTaskToSnode(SMnode* pMnode, SStreamTask* pTask, SSubplan* plan, const SSnodeObj* pSnode) { int32_t msgLen; pTask->nodeId = SNODE_HANDLE; @@ -242,17 +242,16 @@ static int32_t mndScheduleFillHistoryStreamTask(SMnode* pMnode, SStreamObj* pStr return 0; } -static int32_t addSourceStreamTask(SMnode* pMnode, SVgObj* pVgroup, SArray* pTaskList, SArray* pSinkTaskList, SStreamObj* pStream, - SSubplan* plan, uint64_t uid, int8_t taskLevel, int8_t fillHistory, +static int32_t addSourceStreamTask(SMnode* pMnode, SVgObj* pVgroup, SArray* pTaskList, SArray* pSinkTaskList, + SStreamObj* pStream, SSubplan* plan, uint64_t uid, int8_t fillHistory, bool hasExtraSink) { - SStreamTask* pTask = tNewStreamTask(uid, taskLevel, fillHistory, pStream->conf.triggerParam, pTaskList); + SStreamTask* pTask = tNewStreamTask(uid, TASK_LEVEL__SOURCE, fillHistory, pStream->conf.triggerParam, pTaskList); if (pTask == NULL) { return terrno; } // sink or dispatch if (hasExtraSink) { - mndAddDispatcherForInnerTask(pMnode, pStream, pSinkTaskList, pTask); } else { mndSetSinkTaskInfo(pStream, pTask); @@ -290,7 +289,7 @@ void setFixedDownstreamEpInfo(SStreamTask* pDstTask, const SStreamTask* pTask) { pDstTask->dispatchMsgType = TDMT_STREAM_TASK_DISPATCH; } -int32_t appendToDownstream(SStreamTask* pTask, SStreamTask* pDownstream) { +int32_t setEpToDownstreamTask(SStreamTask* pTask, SStreamTask* pDownstream) { SStreamChildEpInfo* pEpInfo = createStreamTaskEpInfo(pTask); if (pEpInfo == NULL) { return TSDB_CODE_OUT_OF_MEMORY; @@ -304,9 +303,260 @@ int32_t appendToDownstream(SStreamTask* pTask, SStreamTask* pDownstream) { return TSDB_CODE_SUCCESS; } -static int32_t doScheduleStream(uint64_t uid, SArray* pTasksList, SStreamObj* pStream, SMnode* pMnode, SQueryPlan* pPlan) { +static SArray* addNewTaskList(SArray* pTasksList) { + SArray* pTaskList = taosArrayInit(0, POINTER_BYTES); + taosArrayPush(pTasksList, &pTaskList); + return pTaskList; +} + +// set the history task id +static void setHTasksId(SArray* pTaskList, const SArray* pHTaskList) { + for(int32_t i = 0; i < taosArrayGetSize(pTaskList); ++i) { + SStreamTask* pStreamTask = taosArrayGet(pTaskList, i); + SStreamTask* pHTask = taosArrayGet(pHTaskList, i); + + pStreamTask->historyTaskId.taskId = pHTask->id.taskId; + pStreamTask->historyTaskId.streamId = pHTask->id.streamId; + } +} + +static int32_t addSourceTasksForSingleLevelStream(SMnode* pMnode, const SQueryPlan* pPlan, SStreamObj* pStream, + bool hasExtraSink) { + // create exec stream task, since only one level, the exec task is also the source task + SArray* pTaskList = addNewTaskList(pStream->tasks); + + SArray* pHTaskList = NULL; + if (pStream->conf.fillHistory) { + pHTaskList = addNewTaskList(pStream->pHTasksList); + } + SSdb* pSdb = pMnode->pSdb; - int32_t planTotLevel = LIST_LENGTH(pPlan->pSubplans); + SNodeListNode* inner = (SNodeListNode*)nodesListGetNode(pPlan->pSubplans, 0); + if (LIST_LENGTH(inner->pNodeList) != 1) { + terrno = TSDB_CODE_QRY_INVALID_INPUT; + return -1; + } + + SSubplan* plan = (SSubplan*)nodesListGetNode(inner->pNodeList, 0); + if (plan->subplanType != SUBPLAN_TYPE_SCAN) { + terrno = TSDB_CODE_QRY_INVALID_INPUT; + return -1; + } + + void* pIter = NULL; + while (1) { + SVgObj* pVgroup; + pIter = sdbFetch(pSdb, SDB_VGROUP, pIter, (void**)&pVgroup); + if (pIter == NULL) { + break; + } + + if (!mndVgroupInDb(pVgroup, pStream->sourceDbUid)) { + sdbRelease(pSdb, pVgroup); + continue; + } + + // new stream task + SArray** pSinkTaskList = taosArrayGet(pStream->tasks, SINK_NODE_LEVEL); + int32_t code = addSourceStreamTask(pMnode, pVgroup, pTaskList, *pSinkTaskList, pStream, plan, pStream->uid, + pStream->conf.fillHistory, hasExtraSink); + if (code != TSDB_CODE_SUCCESS) { + sdbRelease(pSdb, pVgroup); + return -1; + } + + if (pStream->conf.fillHistory) { + SArray** pHSinkTaskList = taosArrayGet(pStream->pHTasksList, SINK_NODE_LEVEL); + code = addSourceStreamTask(pMnode, pVgroup, pHTaskList, *pHSinkTaskList, pStream, plan, pStream->hTaskUid, 0, + hasExtraSink); + setHTasksId(pTaskList, pHTaskList); + } + + sdbRelease(pSdb, pVgroup); + if (code != TSDB_CODE_SUCCESS) { + return -1; + } + } + + return TSDB_CODE_SUCCESS; +} + +static int32_t doAddSourceTask(SArray* pTaskList, int8_t fillHistory, int64_t uid, SStreamTask* pDownstreamTask, SMnode* pMnode, + SSubplan* pPlan, SVgObj* pVgroup) { + SStreamTask* pTask = tNewStreamTask(uid, TASK_LEVEL__SOURCE, fillHistory, 0, pTaskList); + if (pTask == NULL) { + terrno = TSDB_CODE_OUT_OF_MEMORY; + return -1; + } + + // all the source tasks dispatch result to a single agg node. + setFixedDownstreamEpInfo(pTask, pDownstreamTask); + if (mndAssignStreamTaskToVgroup(pMnode, pTask, pPlan, pVgroup) < 0) { + return -1; + } + + return setEpToDownstreamTask(pTask, pDownstreamTask); +} + +static int32_t doAddAggTask(uint64_t uid, SArray* pTaskList, SArray* pSinkNodeList, SMnode* pMnode, SStreamObj* pStream, + SStreamTask** pAggTask) { + *pAggTask = tNewStreamTask(uid, TASK_LEVEL__AGG, pStream->conf.fillHistory, + pStream->conf.triggerParam, pTaskList); + if (*pAggTask == NULL) { + terrno = TSDB_CODE_OUT_OF_MEMORY; + return -1; + } + + // dispatch + if (mndAddDispatcherForInnerTask(pMnode, pStream, pSinkNodeList, *pAggTask) < 0) { + return -1; + } + + return 0; +} + +static int32_t addAggTask(SStreamObj* pStream, SMnode* pMnode, SQueryPlan* pPlan) { + SArray* pAggTaskList = addNewTaskList(pStream->tasks); + SSdb* pSdb = pMnode->pSdb; + + SNodeListNode* inner = (SNodeListNode*)nodesListGetNode(pPlan->pSubplans, 0); + SSubplan* plan = (SSubplan*)nodesListGetNode(inner->pNodeList, 0); + if (plan->subplanType != SUBPLAN_TYPE_MERGE) { + terrno = TSDB_CODE_QRY_INVALID_INPUT; + return -1; + } + + SStreamTask* pAggTask = NULL; + SArray* pSinkNodeList = taosArrayGet(pStream->tasks, SINK_NODE_LEVEL); + + int32_t code = doAddAggTask(pStream->uid, pAggTaskList, pSinkNodeList, pMnode, pStream, &pAggTask); + if (code != TSDB_CODE_SUCCESS) { + return -1; + } + + SVgObj* pVgroup = NULL; + SSnodeObj* pSnode = NULL; + + if (tsDeployOnSnode) { + pSnode = mndSchedFetchOneSnode(pMnode); + if (pSnode == NULL) { + pVgroup = mndSchedFetchOneVg(pMnode, pStream->sourceDbUid); + } + } else { + pVgroup = mndSchedFetchOneVg(pMnode, pStream->sourceDbUid); + } + + if (pSnode != NULL) { + code = mndAssignStreamTaskToSnode(pMnode, pAggTask, plan, pSnode); + } else { + code = mndAssignStreamTaskToVgroup(pMnode, pAggTask, plan, pVgroup); + } + + if (pStream->conf.fillHistory) { + SArray* pHAggTaskList = addNewTaskList(pStream->pHTasksList); + + SStreamTask* pHAggTask = NULL; + code = doAddAggTask(pStream->uid, pAggTaskList, pSinkNodeList, pMnode, pStream, &pHAggTask); + if (code != TSDB_CODE_SUCCESS) { + if (pSnode != NULL) { + sdbRelease(pSdb, pSnode); + } else { + sdbRelease(pSdb, pVgroup); + } + return code; + } + + if (pSnode != NULL) { + code = mndAssignStreamTaskToSnode(pMnode, pHAggTask, plan, pSnode); + } else { + code = mndAssignStreamTaskToVgroup(pMnode, pHAggTask, plan, pVgroup); + } + + setHTasksId(pAggTaskList, pHAggTaskList); + } + + if (pSnode != NULL) { + sdbRelease(pSdb, pSnode); + } else { + sdbRelease(pSdb, pVgroup); + } + + return code; +} + +static int32_t addSourceTasksForMultiLevelStream(SMnode* pMnode, SQueryPlan* pPlan, SStreamObj* pStream, + SStreamTask* pDownstreamTask, SStreamTask* pHDownstreamTask) { + SArray* pSourceTaskList = addNewTaskList(pStream->tasks); + + SArray* pHSourceTaskList = NULL; + if (pStream->conf.fillHistory) { + pHSourceTaskList = addNewTaskList(pStream->pHTasksList); + } + + SSdb* pSdb = pMnode->pSdb; + SNodeListNode* inner = (SNodeListNode*)nodesListGetNode(pPlan->pSubplans, 1); + SSubplan* plan = (SSubplan*)nodesListGetNode(inner->pNodeList, 0); + if (plan->subplanType != SUBPLAN_TYPE_SCAN) { + terrno = TSDB_CODE_QRY_INVALID_INPUT; + return -1; + } + + void* pIter = NULL; + while (1) { + SVgObj* pVgroup; + pIter = sdbFetch(pSdb, SDB_VGROUP, pIter, (void**)&pVgroup); + if (pIter == NULL) { + break; + } + + if (!mndVgroupInDb(pVgroup, pStream->sourceDbUid)) { + sdbRelease(pSdb, pVgroup); + continue; + } + + int32_t code = doAddSourceTask(pSourceTaskList, pStream->conf.fillHistory, pStream->uid, pDownstreamTask, pMnode, plan, pVgroup); + if (code != TSDB_CODE_SUCCESS) { + sdbRelease(pSdb, pVgroup); + terrno = code; + return -1; + } + + if (pStream->conf.fillHistory) { + code = doAddSourceTask(pHSourceTaskList, 0, pStream->hTaskUid, pHDownstreamTask, pMnode, plan, pVgroup); + sdbRelease(pSdb, pVgroup); + + if (code != TSDB_CODE_SUCCESS) { + return code; + } + + setHTasksId(pSourceTaskList, pHSourceTaskList); + } + } + + return TSDB_CODE_SUCCESS; +} + +static int32_t addSinkTasks(SArray* pTasksList, SMnode* pMnode, SStreamObj* pStream, SArray** pCreatedTaskList) { + SArray* pSinkTaskList = addNewTaskList(pTasksList); + if (pStream->fixedSinkVgId == 0) { + if (mndAddShuffleSinkTasksToStream(pMnode, pSinkTaskList, pStream) < 0) { + // TODO free + return -1; + } + } else { + if (mndAddSinkTaskToStream(pStream, pSinkTaskList, pMnode, pStream->fixedSinkVgId, &pStream->fixedSinkVg) < 0) { + // TODO free + return -1; + } + } + + *pCreatedTaskList = pSinkTaskList; + return TSDB_CODE_SUCCESS; +} + +static int32_t doScheduleStream(SStreamObj* pStream, SMnode* pMnode, SQueryPlan* pPlan) { + SSdb* pSdb = pMnode->pSdb; + int32_t numOfPlanLevel = LIST_LENGTH(pPlan->pSubplans); bool hasExtraSink = false; bool externalTargetDB = strcmp(pStream->sourceDb, pStream->targetDb) != 0; @@ -319,165 +569,44 @@ static int32_t doScheduleStream(uint64_t uid, SArray* pTasksList, SStreamObj* pS bool multiTarget = (pDbObj->cfg.numOfVgroups > 1); sdbRelease(pSdb, pDbObj); - if (planTotLevel == 2 || externalTargetDB || multiTarget || pStream->fixedSinkVgId) { - SArray* taskOneLevel = taosArrayInit(0, POINTER_BYTES); - taosArrayPush(pTasksList, &taskOneLevel); + pStream->tasks = taosArrayInit(numOfPlanLevel + 1, POINTER_BYTES); + pStream->pHTasksList = taosArrayInit(numOfPlanLevel + 1, POINTER_BYTES); + if (numOfPlanLevel == 2 || externalTargetDB || multiTarget || pStream->fixedSinkVgId) { // add extra sink hasExtraSink = true; - if (pStream->fixedSinkVgId == 0) { - if (mndAddShuffleSinkTasksToStream(pMnode, taskOneLevel, pStream) < 0) { - // TODO free - return -1; - } - } else { - if (mndAddSinkTaskToStream(pStream, taskOneLevel, pMnode, pStream->fixedSinkVgId, &pStream->fixedSinkVg) < 0) { - // TODO free - return -1; + + SArray* pSinkTaskList = NULL; + int32_t code = addSinkTasks(pStream->tasks, pMnode, pStream, &pSinkTaskList); + if (code != TSDB_CODE_SUCCESS) { + return code; + } + + // check for fill history + if (pStream->conf.fillHistory) { + SArray* pHSinkTaskList = NULL; + code = addSinkTasks(pStream->pHTasksList, pMnode, pStream, &pHSinkTaskList); + if (code != TSDB_CODE_SUCCESS) { + return code; } + + setHTasksId(pSinkTaskList, pHSinkTaskList); } } - pStream->totalLevel = planTotLevel + hasExtraSink; + pStream->totalLevel = numOfPlanLevel + hasExtraSink; - if (planTotLevel > 1) { + if (numOfPlanLevel > 1) { SStreamTask* pInnerTask; - // inner level - { - SArray* taskInnerLevel = taosArrayInit(0, POINTER_BYTES); - taosArrayPush(pTasksList, &taskInnerLevel); - - SNodeListNode* inner = (SNodeListNode*)nodesListGetNode(pPlan->pSubplans, 0); - SSubplan* plan = (SSubplan*)nodesListGetNode(inner->pNodeList, 0); - if (plan->subplanType != SUBPLAN_TYPE_MERGE) { - terrno = TSDB_CODE_QRY_INVALID_INPUT; - return -1; - } - - pInnerTask = tNewStreamTask(uid, TASK_LEVEL__AGG, pStream->conf.fillHistory, pStream->conf.triggerParam, taskInnerLevel); - if (pInnerTask == NULL) { - terrno = TSDB_CODE_OUT_OF_MEMORY; - return -1; - } - - // dispatch - SArray* pSinkNodeList = taosArrayGet(pTasksList, SINK_NODE_LEVEL); - if (mndAddDispatcherForInnerTask(pMnode, pStream, pSinkNodeList, pInnerTask) < 0) { - return -1; - } - - if (tsDeployOnSnode) { - SSnodeObj* pSnode = mndSchedFetchOneSnode(pMnode); - if (pSnode == NULL) { - SVgObj* pVgroup = mndSchedFetchOneVg(pMnode, pStream->sourceDbUid); - if (mndAssignStreamTaskToVgroup(pMnode, pInnerTask, plan, pVgroup) < 0) { - sdbRelease(pSdb, pVgroup); - return -1; - } - sdbRelease(pSdb, pVgroup); - } else { - if (mndAssignTaskToSnode(pMnode, pInnerTask, plan, pSnode) < 0) { - sdbRelease(pSdb, pSnode); - return -1; - } - } - } else { - SVgObj* pVgroup = mndSchedFetchOneVg(pMnode, pStream->sourceDbUid); - if (mndAssignStreamTaskToVgroup(pMnode, pInnerTask, plan, pVgroup) < 0) { - sdbRelease(pSdb, pVgroup); - return -1; - } - - sdbRelease(pSdb, pVgroup); - } + int32_t code = addAggTask(pStream, pMnode, pPlan); + if (code != TSDB_CODE_SUCCESS) { + return code; } // source level - SArray* taskSourceLevel = taosArrayInit(0, POINTER_BYTES); - taosArrayPush(pTasksList, &taskSourceLevel); - - SNodeListNode* inner = (SNodeListNode*)nodesListGetNode(pPlan->pSubplans, 1); - SSubplan* plan = (SSubplan*)nodesListGetNode(inner->pNodeList, 0); - if (plan->subplanType != SUBPLAN_TYPE_SCAN) { - terrno = TSDB_CODE_QRY_INVALID_INPUT; - return -1; - } - - void* pIter = NULL; - while (1) { - SVgObj* pVgroup; - pIter = sdbFetch(pSdb, SDB_VGROUP, pIter, (void**)&pVgroup); - if (pIter == NULL) { - break; - } - - if (!mndVgroupInDb(pVgroup, pStream->sourceDbUid)) { - sdbRelease(pSdb, pVgroup); - continue; - } - - SStreamTask* pTask = tNewStreamTask(uid, TASK_LEVEL__SOURCE, pStream->conf.fillHistory, 0, taskSourceLevel); - if (pTask == NULL) { - terrno = TSDB_CODE_OUT_OF_MEMORY; - sdbRelease(pSdb, pVgroup); - return -1; - } - - // all the source tasks dispatch result to a single agg node. - setFixedDownstreamEpInfo(pTask, pInnerTask); - - if (mndAssignStreamTaskToVgroup(pMnode, pTask, plan, pVgroup) < 0) { - sdbRelease(pSdb, pVgroup); - return -1; - } - - int32_t code = appendToDownstream(pTask, pInnerTask); - sdbRelease(pSdb, pVgroup); - - if (code != TSDB_CODE_SUCCESS) { - terrno = code; - return -1; - } - } - } else if (planTotLevel == 1) { - // create exec stream task, since only one level, the exec task is also the source task - SArray* pTaskList = taosArrayInit(0, POINTER_BYTES); - taosArrayPush(pTasksList, &pTaskList); - - SNodeListNode* inner = (SNodeListNode*)nodesListGetNode(pPlan->pSubplans, 0); - if (LIST_LENGTH(inner->pNodeList) != 1) { - terrno = TSDB_CODE_QRY_INVALID_INPUT; - return -1; - } - - SSubplan* plan = (SSubplan*)nodesListGetNode(inner->pNodeList, 0); - if (plan->subplanType != SUBPLAN_TYPE_SCAN) { - terrno = TSDB_CODE_QRY_INVALID_INPUT; - return -1; - } - - void* pIter = NULL; - while (1) { - SVgObj* pVgroup; - pIter = sdbFetch(pSdb, SDB_VGROUP, pIter, (void**)&pVgroup); - if (pIter == NULL) { - break; - } - - if (!mndVgroupInDb(pVgroup, pStream->sourceDbUid)) { - sdbRelease(pSdb, pVgroup); - continue; - } - - // new stream task - SArray* pSinkNodeTaskList = taosArrayGet(pTasksList, SINK_NODE_LEVEL); - int32_t code = addSourceStreamTask(pMnode, pVgroup, pTaskList, pSinkNodeTaskList, pStream, plan, uid, TASK_LEVEL__SOURCE, pStream->conf.fillHistory, hasExtraSink); - sdbRelease(pSdb, pVgroup); - - if (code != TSDB_CODE_SUCCESS) { - return -1; - } - } + return addSourceTasksForMultiLevelStream(pMnode, pPlan, pStream, pInnerTask, NULL); + } else if (numOfPlanLevel == 1) { + return addSourceTasksForSingleLevelStream(pMnode, pPlan, pStream, hasExtraSink); } return 0; @@ -490,13 +619,9 @@ int32_t mndScheduleStream(SMnode* pMnode, SStreamObj* pStream) { return -1; } - int32_t code = doScheduleStream(pStream->uid, pStream->tasks, pStream, pMnode, pPlan); - - if (code == TSDB_CODE_SUCCESS) { - code = doScheduleStream(pStream->batchTaskUid, pStream->pBatchTask, pStream, pMnode, pPlan); - } - + int32_t code = doScheduleStream(pStream, pMnode, pPlan); qDestroyQueryPlan(pPlan); + return code; } diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index f5cd673225..81b985f515 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -428,17 +428,22 @@ int32_t mndPersistTaskDeployReq(STrans *pTrans, const SStreamTask *pTask) { SEncoder encoder; tEncoderInit(&encoder, NULL, 0); tEncodeStreamTask(&encoder, pTask); + int32_t size = encoder.pos; int32_t tlen = sizeof(SMsgHead) + size; tEncoderClear(&encoder); + void *buf = taosMemoryCalloc(1, tlen); if (buf == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; return -1; } + ((SMsgHead *)buf)->vgId = htonl(pTask->nodeId); + void *abuf = POINTER_SHIFT(buf, sizeof(SMsgHead)); tEncoderInit(&encoder, abuf, size); + tEncodeStreamTask(&encoder, pTask); tEncoderClear(&encoder); @@ -448,10 +453,12 @@ int32_t mndPersistTaskDeployReq(STrans *pTrans, const SStreamTask *pTask) { action.pCont = buf; action.contLen = tlen; action.msgType = TDMT_STREAM_TASK_DEPLOY; + if (mndTransAppendRedoAction(pTrans, &action) != 0) { taosMemoryFree(buf); return -1; } + return 0; } @@ -468,6 +475,25 @@ int32_t mndPersistStreamTasks(SMnode *pMnode, STrans *pTrans, SStreamObj *pStrea } } } + + // persistent stream task for history data + if (pStream->conf.fillHistory) { + level = taosArrayGetSize(pStream->pHTasksList); + + for (int32_t i = 0; i < level; i++) { + SArray *pLevel = taosArrayGetP(pStream->pHTasksList, i); + + int32_t numOfTasks = taosArrayGetSize(pLevel); + for (int32_t j = 0; j < numOfTasks; j++) { + SStreamTask *pTask = taosArrayGetP(pLevel, j); + if (mndPersistTaskDeployReq(pTrans, pTask) < 0) { + return -1; + } + } + } + } + + return 0; } @@ -475,11 +501,13 @@ int32_t mndPersistStream(SMnode *pMnode, STrans *pTrans, SStreamObj *pStream) { if (mndPersistStreamTasks(pMnode, pTrans, pStream) < 0) { return -1; } + SSdbRaw *pCommitRaw = mndStreamActionEncode(pStream); if (pCommitRaw == NULL || mndTransAppendCommitlog(pTrans, pCommitRaw) != 0) { mError("trans:%d, failed to append commit log since %s", pTrans->id, terrstr()); return -1; } + (void)sdbSetRawStatus(pCommitRaw, SDB_STATUS_READY); return 0; } @@ -491,6 +519,7 @@ int32_t mndPersistDropStreamLog(SMnode *pMnode, STrans *pTrans, SStreamObj *pStr mndTransDrop(pTrans); return -1; } + (void)sdbSetRawStatus(pCommitRaw, SDB_STATUS_DROPPED); return 0; } @@ -733,6 +762,7 @@ static int32_t mndProcessCreateStreamReq(SRpcMsg *pReq) { mError("stream:%s, failed to create since %s", createStreamReq.name, terrstr()); goto _OVER; } + mInfo("trans:%d, used to create stream:%s", pTrans->id, createStreamReq.name); mndTransSetDbName(pTrans, createStreamReq.sourceDB, streamObj.targetDb); From eda06081ffdf782a3f21415326b705b1dccc5732 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 30 May 2023 16:55:37 +0800 Subject: [PATCH 03/83] enh(stream): refactor and serialize the attributes of history tasks. --- include/common/tcommon.h | 5 +++++ include/libs/stream/tstream.h | 11 ++++++++--- source/dnode/mnode/impl/src/mndStream.c | 4 ++-- source/dnode/vnode/src/inc/tsdb.h | 6 ------ source/libs/stream/src/streamTask.c | 14 ++++++++++++++ 5 files changed, 29 insertions(+), 11 deletions(-) diff --git a/include/common/tcommon.h b/include/common/tcommon.h index 2f93f8c3e3..8a1e95a661 100644 --- a/include/common/tcommon.h +++ b/include/common/tcommon.h @@ -54,6 +54,11 @@ typedef struct SSessionKey { uint64_t groupId; } SSessionKey; +typedef struct SVersionRange { + uint64_t minVer; + uint64_t maxVer; +} SVersionRange; + static inline int winKeyCmprImpl(const void* pKey1, const void* pKey2) { SWinKey* pWin1 = (SWinKey*)pKey1; SWinKey* pWin2 = (SWinKey*)pKey2; diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 6d54790b2f..2e2663f87a 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -276,6 +276,11 @@ typedef struct SStreamStatus { int8_t keepTaskStatus; } SStreamStatus; +typedef struct SHistoryDataRange { + SVersionRange range; + STimeWindow window; +} SHistoryDataRange; + struct SStreamTask { SStreamId id; int32_t totalLevel; @@ -290,9 +295,9 @@ struct SStreamTask { STaskExec exec; int8_t fillHistory; // fill history - int64_t ekey; // end ts key - int64_t endVer; // end version - SStreamId historyTaskId; + SHistoryDataRange dataRange; + SStreamId historyTaskId; + // children info SArray* childEpInfo; // SArray int32_t nextCheckId; diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index 81b985f515..1ce4ce2b7e 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -476,7 +476,7 @@ int32_t mndPersistStreamTasks(SMnode *pMnode, STrans *pTrans, SStreamObj *pStrea } } - // persistent stream task for history data + // persistent stream task for already stored ts data if (pStream->conf.fillHistory) { level = taosArrayGetSize(pStream->pHTasksList); @@ -493,7 +493,6 @@ int32_t mndPersistStreamTasks(SMnode *pMnode, STrans *pTrans, SStreamObj *pStrea } } - return 0; } @@ -639,6 +638,7 @@ static int32_t mndPersistTaskDropReq(STrans *pTrans, SStreamTask *pTask) { terrno = TSDB_CODE_OUT_OF_MEMORY; return -1; } + pReq->head.vgId = htonl(pTask->nodeId); pReq->taskId = pTask->id.taskId; STransAction action = {0}; diff --git a/source/dnode/vnode/src/inc/tsdb.h b/source/dnode/vnode/src/inc/tsdb.h index faf550ab75..f9dd80a056 100644 --- a/source/dnode/vnode/src/inc/tsdb.h +++ b/source/dnode/vnode/src/inc/tsdb.h @@ -64,7 +64,6 @@ typedef struct STsdbReadSnap STsdbReadSnap; typedef struct SBlockInfo SBlockInfo; typedef struct SSmaInfo SSmaInfo; typedef struct SBlockCol SBlockCol; -typedef struct SVersionRange SVersionRange; typedef struct SLDataIter SLDataIter; typedef struct SDiskCol SDiskCol; typedef struct SDiskData SDiskData; @@ -376,11 +375,6 @@ struct TSDBKEY { TSKEY ts; }; -struct SVersionRange { - uint64_t minVer; - uint64_t maxVer; -}; - typedef struct SMemSkipListNode SMemSkipListNode; struct SMemSkipListNode { int8_t level; diff --git a/source/libs/stream/src/streamTask.c b/source/libs/stream/src/streamTask.c index 6d8ec11f44..4f883b76e4 100644 --- a/source/libs/stream/src/streamTask.c +++ b/source/libs/stream/src/streamTask.c @@ -187,6 +187,13 @@ int32_t tEncodeStreamTask(SEncoder* pEncoder, const SStreamTask* pTask) { if (tEncodeI64(pEncoder, pTask->chkInfo.version) < 0) return -1; if (tEncodeI8(pEncoder, pTask->fillHistory) < 0) return -1; + if (tEncodeI64(pEncoder, pTask->historyTaskId.streamId)) return -1; + if (tEncodeI32(pEncoder, pTask->historyTaskId.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->childEpInfo); if (tEncodeI32(pEncoder, epSz) < 0) return -1; for (int32_t i = 0; i < epSz; i++) { @@ -240,6 +247,13 @@ int32_t tDecodeStreamTask(SDecoder* pDecoder, SStreamTask* pTask) { if (tDecodeI64(pDecoder, &pTask->chkInfo.version) < 0) return -1; if (tDecodeI8(pDecoder, &pTask->fillHistory) < 0) return -1; + if (tDecodeI64(pDecoder, &pTask->historyTaskId.streamId)) return -1; + if (tDecodeI32(pDecoder, &pTask->historyTaskId.taskId)) return -1; + if (tDecodeU64(pDecoder, &pTask->dataRange.range.minVer)) return -1; + if (tDecodeU64(pDecoder, &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; if (tDecodeI32(pDecoder, &epSz) < 0) return -1; pTask->childEpInfo = taosArrayInit(epSz, sizeof(void*)); From e237b4ac39e29537bdf820296d1061bb2deff1fd Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 31 May 2023 09:19:05 +0800 Subject: [PATCH 04/83] refactor: do some internal refactor. --- source/dnode/vnode/src/tq/tq.c | 13 ++++++++----- 1 file changed, 8 insertions(+), 5 deletions(-) diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index dcfc578ac7..4f2be441e0 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -978,17 +978,20 @@ int32_t tqProcessTaskDeployReq(STQ* pTq, int64_t sversion, char* msg, int32_t ms tDecoderClear(&decoder); + SStreamMeta* pStreamMeta = pTq->pStreamMeta; + // 2.save task, use the newest commit version as the initial start version of stream task. - taosWLockLatch(&pTq->pStreamMeta->lock); - code = streamMetaAddDeployedTask(pTq->pStreamMeta, sversion, pTask); - int32_t numOfTasks = streamMetaGetNumOfTasks(pTq->pStreamMeta); + taosWLockLatch(&pStreamMeta->lock); + code = streamMetaAddDeployedTask(pStreamMeta, sversion, pTask); + + int32_t numOfTasks = streamMetaGetNumOfTasks(pStreamMeta); if (code < 0) { tqError("vgId:%d failed to add s-task:%s, total:%d", vgId, pTask->id.idStr, numOfTasks); - taosWUnLockLatch(&pTq->pStreamMeta->lock); + taosWUnLockLatch(&pStreamMeta->lock); return -1; } - taosWUnLockLatch(&pTq->pStreamMeta->lock); + taosWUnLockLatch(&pStreamMeta->lock); // 3.go through recover steps to fill history if (pTask->fillHistory) { From bbb6359ea2386965aecd1e1dcf6191c4dc26ce37 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 31 May 2023 17:03:39 +0800 Subject: [PATCH 05/83] fix(query): add check for version range when dumping partial rows of file block to data block --- source/dnode/vnode/src/tsdb/tsdbRead.c | 52 +++++++++++++++++++++++++- 1 file changed, 50 insertions(+), 2 deletions(-) diff --git a/source/dnode/vnode/src/tsdb/tsdbRead.c b/source/dnode/vnode/src/tsdb/tsdbRead.c index 362934ec84..56a953519e 100644 --- a/source/dnode/vnode/src/tsdb/tsdbRead.c +++ b/source/dnode/vnode/src/tsdb/tsdbRead.c @@ -1121,6 +1121,27 @@ static int32_t getEndPosInDataBlock(STsdbReader* pReader, SBlockData* pBlockData endPos = doBinarySearchKey(pBlockData->aTSKEY, pBlock->nRow, pos, key, pReader->order); } + if ((pReader->verRange.maxVer >= pBlock->minVer && pReader->verRange.maxVer < pBlock->maxVer)|| + (pReader->verRange.minVer <= pBlock->maxVer && pReader->verRange.minVer > pBlock->minVer)) { + int32_t i = endPos; + + if (asc) { + for(; i >= 0; --i) { + if (pBlockData->aVersion[i] <= pReader->verRange.maxVer) { + break; + } + } + } else { + for(; i < pBlock->nRow; ++i) { + if (pBlockData->aVersion[i] >= pReader->verRange.minVer) { + break; + } + } + } + + endPos = i; + } + return endPos; } @@ -1260,10 +1281,11 @@ static int32_t copyBlockDataToSDataBlock(STsdbReader* pReader) { return 0; } + // row index of dump info remain the initial position, let's find the appropriate start position. if ((pDumpInfo->rowIndex == 0 && asc) || (pDumpInfo->rowIndex == pBlock->nRow - 1 && (!asc))) { - if (asc && pReader->window.skey <= pBlock->minKey.ts) { + if (asc && pReader->window.skey <= pBlock->minKey.ts && pReader->verRange.minVer <= pBlock->minVer) { // pDumpInfo->rowIndex = 0; - } else if (!asc && pReader->window.ekey >= pBlock->maxKey.ts) { + } else if (!asc && pReader->window.ekey >= pBlock->maxKey.ts && pReader->verRange.maxVer >= pBlock->maxVer) { // pDumpInfo->rowIndex = pBlock->nRow - 1; } else { // find the appropriate the start position in current block, and set it to be the current rowIndex int32_t pos = asc ? pBlock->nRow - 1 : 0; @@ -1279,6 +1301,29 @@ static int32_t copyBlockDataToSDataBlock(STsdbReader* pReader) { pBlock->maxVer, pReader->idStr); return TSDB_CODE_INVALID_PARA; } + + ASSERT(pReader->verRange.minVer <= pBlock->maxVer && pReader->verRange.maxVer >= pBlock->minVer); + + // find the appropriate start position that satisfies the version requirement. + if ((pReader->verRange.maxVer >= pBlock->minVer && pReader->verRange.maxVer < pBlock->maxVer)|| + (pReader->verRange.minVer <= pBlock->maxVer && pReader->verRange.minVer > pBlock->minVer)) { + int32_t i = pDumpInfo->rowIndex; + if (asc) { + for(; i < pBlock->nRow; ++i) { + if (pBlockData->aVersion[i] >= pReader->verRange.minVer) { + break; + } + } + } else { + for(; i >= 0; --i) { + if (pBlockData->aVersion[i] <= pReader->verRange.maxVer) { + break; + } + } + } + + pDumpInfo->rowIndex = i; + } } } @@ -1293,6 +1338,9 @@ static int32_t copyBlockDataToSDataBlock(STsdbReader* pReader) { int32_t dumpedRows = asc ? (endIndex - pDumpInfo->rowIndex) : (pDumpInfo->rowIndex - endIndex); if (dumpedRows > pReader->resBlockInfo.capacity) { // output buffer check dumpedRows = pReader->resBlockInfo.capacity; + } else if (dumpedRows <= 0) { // no qualified rows in current data block, abort directly. + setBlockAllDumped(pDumpInfo, pReader->window.ekey, pReader->order); + return TSDB_CODE_SUCCESS; } int32_t i = 0; From fcc706c45ce3cbe6ac00520d5e20835f7bf6326b Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 5 Jun 2023 15:59:01 +0800 Subject: [PATCH 06/83] enh(stream): refactor the fill history task. --- include/libs/executor/executor.h | 2 +- include/libs/stream/tstream.h | 4 ++-- source/dnode/mnode/impl/src/mndScheduler.c | 12 +++++----- source/dnode/mnode/impl/src/mndStream.c | 5 ++++ source/dnode/vnode/src/tq/tq.c | 27 ++++++++++++++++++++-- source/libs/executor/inc/querytask.h | 1 + source/libs/executor/src/executor.c | 3 ++- source/libs/executor/src/scanoperator.c | 10 ++++---- source/libs/stream/src/streamMeta.c | 1 + source/libs/stream/src/streamRecover.c | 24 +++++++++---------- 10 files changed, 60 insertions(+), 29 deletions(-) diff --git a/include/libs/executor/executor.h b/include/libs/executor/executor.h index 3f53976c67..2e4fb23207 100644 --- a/include/libs/executor/executor.h +++ b/include/libs/executor/executor.h @@ -221,7 +221,7 @@ void* qExtractReaderFromStreamScanner(void* scanner); int32_t qExtractStreamScanner(qTaskInfo_t tinfo, void** scanner); int32_t qStreamSetParamForRecover(qTaskInfo_t tinfo); -int32_t qStreamSourceRecoverStep1(qTaskInfo_t tinfo, int64_t ver); +int32_t qStreamSourceRecoverStep1(qTaskInfo_t tinfo, int64_t ver, int64_t ekey); int32_t qStreamSourceRecoverStep2(qTaskInfo_t tinfo, int64_t ver); int32_t qStreamRecoverFinish(qTaskInfo_t tinfo); int32_t qStreamRestoreParam(qTaskInfo_t tinfo); diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 579a4a65cb..5a5e930d42 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -553,7 +553,7 @@ int32_t streamScanExec(SStreamTask* pTask, int32_t batchSz); // recover and fill history int32_t streamTaskCheckDownstream(SStreamTask* pTask, int64_t version); -int32_t streamTaskLaunchRecover(SStreamTask* pTask, int64_t version); +int32_t streamTaskLaunchRecover(SStreamTask* pTask); int32_t streamTaskCheckStatus(SStreamTask* pTask); int32_t streamProcessTaskCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* pRsp, int64_t version); @@ -562,7 +562,7 @@ int32_t streamSetParamForRecover(SStreamTask* pTask); int32_t streamRestoreParam(SStreamTask* pTask); int32_t streamSetStatusNormal(SStreamTask* pTask); // source level -int32_t streamSourceRecoverPrepareStep1(SStreamTask* pTask, int64_t ver); +int32_t streamSourceRecoverPrepareStep1(SStreamTask* pTask, int64_t ver, int64_t ekey); int32_t streamBuildSourceRecover1Req(SStreamTask* pTask, SStreamRecoverStep1Req* pReq); int32_t streamSourceRecoverScanStep1(SStreamTask* pTask); int32_t streamBuildSourceRecover2Req(SStreamTask* pTask, SStreamRecoverStep2Req* pReq); diff --git a/source/dnode/mnode/impl/src/mndScheduler.c b/source/dnode/mnode/impl/src/mndScheduler.c index e52ef25725..8ae3827936 100644 --- a/source/dnode/mnode/impl/src/mndScheduler.c +++ b/source/dnode/mnode/impl/src/mndScheduler.c @@ -312,11 +312,11 @@ static SArray* addNewTaskList(SArray* pTasksList) { // set the history task id static void setHTasksId(SArray* pTaskList, const SArray* pHTaskList) { for(int32_t i = 0; i < taosArrayGetSize(pTaskList); ++i) { - SStreamTask* pStreamTask = taosArrayGet(pTaskList, i); - SStreamTask* pHTask = taosArrayGet(pHTaskList, i); + SStreamTask** pStreamTask = taosArrayGet(pTaskList, i); + SStreamTask** pHTask = taosArrayGet(pHTaskList, i); - pStreamTask->historyTaskId.taskId = pHTask->id.taskId; - pStreamTask->historyTaskId.streamId = pHTask->id.streamId; + (*pStreamTask)->historyTaskId.taskId = (*pHTask)->id.taskId; + (*pStreamTask)->historyTaskId.streamId = (*pHTask)->id.streamId; } } @@ -359,7 +359,7 @@ static int32_t addSourceTasksForSingleLevelStream(SMnode* pMnode, const SQueryPl // new stream task SArray** pSinkTaskList = taosArrayGet(pStream->tasks, SINK_NODE_LEVEL); int32_t code = addSourceStreamTask(pMnode, pVgroup, pTaskList, *pSinkTaskList, pStream, plan, pStream->uid, - pStream->conf.fillHistory, hasExtraSink); + 0, hasExtraSink); if (code != TSDB_CODE_SUCCESS) { sdbRelease(pSdb, pVgroup); return -1; @@ -367,7 +367,7 @@ static int32_t addSourceTasksForSingleLevelStream(SMnode* pMnode, const SQueryPl if (pStream->conf.fillHistory) { SArray** pHSinkTaskList = taosArrayGet(pStream->pHTasksList, SINK_NODE_LEVEL); - code = addSourceStreamTask(pMnode, pVgroup, pHTaskList, *pHSinkTaskList, pStream, plan, pStream->hTaskUid, 0, + code = addSourceStreamTask(pMnode, pVgroup, pHTaskList, *pHSinkTaskList, pStream, plan, pStream->hTaskUid, pStream->conf.fillHistory, hasExtraSink); setHTasksId(pTaskList, pHTaskList); } diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index 1ce4ce2b7e..d4c1e033dc 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -299,6 +299,11 @@ static int32_t mndBuildStreamObjFromCreateReq(SMnode *pMnode, SStreamObj *pObj, pObj->smaId = 0; pObj->uid = mndGenerateUid(pObj->name, strlen(pObj->name)); + + char p[TSDB_STREAM_FNAME_LEN + 32] = {0}; + snprintf(p, tListLen(p), "%s_%s", pObj->name, "fillhistory"); + + pObj->hTaskUid = mndGenerateUid(pObj->name, strlen(pObj->name)); pObj->status = 0; pObj->conf.igExpired = pCreate->igExpired; diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 5a8abe2bec..ce3362b677 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1027,13 +1027,36 @@ int32_t tqProcessTaskDeployReq(STQ* pTq, int64_t sversion, char* msg, int32_t ms taosWUnLockLatch(&pStreamMeta->lock); - // 3.go through recover steps to fill history + // 3. for fill history task, do nothing. wait for the main task to start it if (pTask->fillHistory) { - streamTaskCheckDownstream(pTask, sversion); + tqDebug("s-task:%s fill history task, wait for being launched", pTask->id.idStr); + } else { + if (pTask->historyTaskId.taskId != 0) { + // todo fix the bug: 1. maybe failed to located the fill history task, since it is not built yet. 2. race condition + + // an fill history task needs to be started. + // Set the execute conditions, including the query time window and the version range + SStreamTask* pHTask = taosHashGet(pStreamMeta->pTasks, &pTask->historyTaskId.taskId, sizeof(pTask->historyTaskId.taskId)); + + pHTask->dataRange.range.minVer = 0; + pHTask->dataRange.range.maxVer = sversion; + + pHTask->dataRange.window.skey = INT64_MIN; + pHTask->dataRange.window.ekey = 1000000; + + tqDebug("s-task:%s set the launch condition for fill history task:%s, window:%" PRId64 " - %" PRId64 + " verrange:%" PRId64 " - %" PRId64, + pTask->id.idStr, pHTask->id.idStr, pHTask->dataRange.window.skey, pHTask->dataRange.window.ekey, + pHTask->dataRange.range.minVer, pHTask->dataRange.range.maxVer); + + // check if downstream tasks have been ready + streamTaskCheckDownstream(pHTask, sversion); + } } tqDebug("vgId:%d s-task:%s is deployed and add meta from mnd, status:%d, total:%d", vgId, pTask->id.idStr, pTask->status.taskStatus, numOfTasks); + return 0; } diff --git a/source/libs/executor/inc/querytask.h b/source/libs/executor/inc/querytask.h index 6497bd90b4..2b9256f08e 100644 --- a/source/libs/executor/inc/querytask.h +++ b/source/libs/executor/inc/querytask.h @@ -65,6 +65,7 @@ typedef struct { int8_t recoverScanFinished; SQueryTableDataCond tableCond; int64_t fillHistoryVer1; + int64_t fillHisotryeKey1; int64_t fillHistoryVer2; SStreamState* pState; int64_t dataVersion; diff --git a/source/libs/executor/src/executor.c b/source/libs/executor/src/executor.c index c8b66836d5..0ea6ae7144 100644 --- a/source/libs/executor/src/executor.c +++ b/source/libs/executor/src/executor.c @@ -869,10 +869,11 @@ int32_t qExtractStreamScanner(qTaskInfo_t tinfo, void** scanner) { } } -int32_t qStreamSourceRecoverStep1(qTaskInfo_t tinfo, int64_t ver) { +int32_t qStreamSourceRecoverStep1(qTaskInfo_t tinfo, int64_t ver, int64_t ekey) { SExecTaskInfo* pTaskInfo = (SExecTaskInfo*)tinfo; ASSERT(pTaskInfo->execModel == OPTR_EXEC_MODEL_STREAM); pTaskInfo->streamInfo.fillHistoryVer1 = ver; + pTaskInfo->streamInfo.fillHisotryeKey1 = ekey; pTaskInfo->streamInfo.recoverStep = STREAM_RECOVER_STEP__PREPARE1; return 0; } diff --git a/source/libs/executor/src/scanoperator.c b/source/libs/executor/src/scanoperator.c index c93b9f4c73..4b53a9918d 100644 --- a/source/libs/executor/src/scanoperator.c +++ b/source/libs/executor/src/scanoperator.c @@ -1779,7 +1779,7 @@ static SSDataBlock* doStreamScan(SOperatorInfo* pOperator) { SStorageAPI* pAPI = &pTaskInfo->storageAPI; SStreamScanInfo* pInfo = pOperator->info; - qDebug("stream scan started, %s", GET_TASKID(pTaskInfo)); + qDebug("stream scan started, %s", id); if (pTaskInfo->streamInfo.recoverStep == STREAM_RECOVER_STEP__PREPARE1 || pTaskInfo->streamInfo.recoverStep == STREAM_RECOVER_STEP__PREPARE2) { @@ -1788,14 +1788,14 @@ static SSDataBlock* doStreamScan(SOperatorInfo* pOperator) { if (pTaskInfo->streamInfo.recoverStep == STREAM_RECOVER_STEP__PREPARE1) { pTSInfo->base.cond.startVersion = 0; pTSInfo->base.cond.endVersion = pTaskInfo->streamInfo.fillHistoryVer1; - qDebug("stream recover step1, verRange:%" PRId64 " - %" PRId64, pTSInfo->base.cond.startVersion, - pTSInfo->base.cond.endVersion); + qDebug("stream recover step1, verRange:%" PRId64 " - %" PRId64 ", %s", pTSInfo->base.cond.startVersion, + pTSInfo->base.cond.endVersion, id); pTaskInfo->streamInfo.recoverStep = STREAM_RECOVER_STEP__SCAN1; } else { pTSInfo->base.cond.startVersion = pTaskInfo->streamInfo.fillHistoryVer1 + 1; pTSInfo->base.cond.endVersion = pTaskInfo->streamInfo.fillHistoryVer2; - qDebug("stream recover step2, verRange:%" PRId64 " - %" PRId64, pTSInfo->base.cond.startVersion, - pTSInfo->base.cond.endVersion); + qDebug("stream recover step2, verRange:%" PRId64 " - %" PRId64", %s", pTSInfo->base.cond.startVersion, + pTSInfo->base.cond.endVersion, id); pTaskInfo->streamInfo.recoverStep = STREAM_RECOVER_STEP__SCAN2; } diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 8c26052fdb..c984bdba54 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -386,6 +386,7 @@ int32_t streamLoadTasks(SStreamMeta* pMeta, int64_t ver) { return -1; } + // todo handle the fill history task if (pTask->fillHistory) { ASSERT(pTask->status.taskStatus == TASK_STATUS__WAIT_DOWNSTREAM); streamTaskCheckDownstream(pTask, ver); diff --git a/source/libs/stream/src/streamRecover.c b/source/libs/stream/src/streamRecover.c index eb2535782e..e288694887 100644 --- a/source/libs/stream/src/streamRecover.c +++ b/source/libs/stream/src/streamRecover.c @@ -15,7 +15,7 @@ #include "streamInc.h" -int32_t streamTaskLaunchRecover(SStreamTask* pTask, int64_t version) { +int32_t streamTaskLaunchRecover(SStreamTask* pTask) { qDebug("s-task:%s at node %d launch recover", pTask->id.idStr, pTask->nodeId); if (pTask->taskLevel == TASK_LEVEL__SOURCE) { @@ -23,7 +23,7 @@ int32_t streamTaskLaunchRecover(SStreamTask* pTask, int64_t version) { qDebug("s-task:%s set task status:%d and start to recover", pTask->id.idStr, pTask->status.taskStatus); streamSetParamForRecover(pTask); - streamSourceRecoverPrepareStep1(pTask, version); + streamSourceRecoverPrepareStep1(pTask, pTask->dataRange.range.maxVer, pTask->dataRange.window.ekey); SStreamRecoverStep1Req req; streamBuildSourceRecover1Req(pTask, &req); @@ -54,8 +54,8 @@ int32_t streamTaskLaunchRecover(SStreamTask* pTask, int64_t version) { } // checkstatus -int32_t streamTaskCheckDownstream(SStreamTask* pTask, int64_t version) { - qDebug("s-task:%s in fill history stage, ver:%"PRId64, pTask->id.idStr, version); +int32_t streamTaskCheckDownstream(SStreamTask* pTask, int64_t ver) { + qDebug("s-task:%s in fill history stage, ver:%"PRId64, pTask->id.idStr, ver); SStreamTaskCheckReq req = { .streamId = pTask->id.streamId, @@ -88,13 +88,13 @@ int32_t streamTaskCheckDownstream(SStreamTask* pTask, int64_t version) { taosArrayPush(pTask->checkReqIds, &req.reqId); req.downstreamNodeId = pVgInfo->vgId; req.downstreamTaskId = pVgInfo->taskId; - qDebug("s-task:%s at node %d check downstream task:0x%x at node %d (shuffle)", pTask->id.idStr, pTask->nodeId, + qDebug("s-task:%s (vgId:%d) check downstream task:0x%x at node %d (shuffle)", pTask->id.idStr, pTask->nodeId, req.downstreamTaskId, req.downstreamNodeId); streamDispatchCheckMsg(pTask, &req, pVgInfo->vgId, &pVgInfo->epSet); } } else { - qDebug("s-task:%s at node %d direct launch recover since no downstream", pTask->id.idStr, pTask->nodeId); - streamTaskLaunchRecover(pTask, version); + qDebug("s-task:%s (vgId:%d) direct launch recover since no downstream", pTask->id.idStr, pTask->nodeId); + streamTaskLaunchRecover(pTask); } return 0; @@ -135,7 +135,7 @@ int32_t streamTaskCheckStatus(SStreamTask* pTask) { return atomic_load_8(&pTask->status.taskStatus) == TASK_STATUS__NORMAL? 1:0; } -int32_t streamProcessTaskCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* pRsp, int64_t version) { +int32_t streamProcessTaskCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* pRsp, int64_t ver) { ASSERT(pTask->id.taskId == pRsp->upstreamTaskId); qDebug("s-task:%s at node %d recv check rsp from task:0x%x at node %d: status %d", pTask->id.idStr, @@ -166,14 +166,14 @@ int32_t streamProcessTaskCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* pTask->checkReqIds = NULL; qDebug("s-task:%s all %d downstream tasks are ready, now enter into recover stage", pTask->id.idStr, numOfReqs); - streamTaskLaunchRecover(pTask, version); + streamTaskLaunchRecover(pTask); } } else if (pTask->outputType == TASK_OUTPUT__FIXED_DISPATCH) { if (pRsp->reqId != pTask->checkReqId) { return -1; } - streamTaskLaunchRecover(pTask, version); + streamTaskLaunchRecover(pTask); } else { ASSERT(0); } @@ -204,9 +204,9 @@ int32_t streamSetStatusNormal(SStreamTask* pTask) { } // source -int32_t streamSourceRecoverPrepareStep1(SStreamTask* pTask, int64_t ver) { +int32_t streamSourceRecoverPrepareStep1(SStreamTask* pTask, int64_t ver, int64_t ekey) { void* exec = pTask->exec.pExecutor; - return qStreamSourceRecoverStep1(exec, ver); + return qStreamSourceRecoverStep1(exec, ver, ekey); } int32_t streamBuildSourceRecover1Req(SStreamTask* pTask, SStreamRecoverStep1Req* pReq) { From e8549ce511dd2a4017bbd180af879b2b22d40e8c Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 5 Jun 2023 16:19:37 +0800 Subject: [PATCH 07/83] refactor: do some internal refactor. --- source/libs/stream/src/stream.c | 13 +++++++------ source/libs/stream/src/streamRecover.c | 2 +- 2 files changed, 8 insertions(+), 7 deletions(-) diff --git a/source/libs/stream/src/stream.c b/source/libs/stream/src/stream.c index 7457b2197e..b64468a7f4 100644 --- a/source/libs/stream/src/stream.c +++ b/source/libs/stream/src/stream.c @@ -16,8 +16,8 @@ #include "streamInc.h" #include "ttimer.h" -#define STREAM_TASK_INPUT_QUEUEU_CAPACITY 20480 -#define STREAM_TASK_INPUT_QUEUEU_CAPACITY_IN_SIZE (50) +#define STREAM_TASK_INPUT_QUEUE_CAPACITY 20480 +#define STREAM_TASK_INPUT_QUEUE_CAPACITY_IN_SIZE (50) #define ONE_MB_F (1048576.0) #define QUEUE_MEM_SIZE_IN_MB(_q) (taosQueueMemorySize(_q) / ONE_MB_F) @@ -36,6 +36,7 @@ int32_t streamInit() { } atomic_store_8(&streamEnv.inited, 1); } + return 0; } @@ -284,9 +285,9 @@ int32_t streamProcessRetrieveReq(SStreamTask* pTask, SStreamRetrieveReq* pReq, S } bool tInputQueueIsFull(const SStreamTask* pTask) { - bool isFull = taosQueueItemSize((pTask->inputQueue->queue)) >= STREAM_TASK_INPUT_QUEUEU_CAPACITY; + bool isFull = taosQueueItemSize((pTask->inputQueue->queue)) >= STREAM_TASK_INPUT_QUEUE_CAPACITY; double size = QUEUE_MEM_SIZE_IN_MB(pTask->inputQueue->queue); - return (isFull || size >= STREAM_TASK_INPUT_QUEUEU_CAPACITY_IN_SIZE); + return (isFull || size >= STREAM_TASK_INPUT_QUEUE_CAPACITY_IN_SIZE); } int32_t tAppendDataToInputQueue(SStreamTask* pTask, SStreamQueueItem* pItem) { @@ -298,7 +299,7 @@ int32_t tAppendDataToInputQueue(SStreamTask* pTask, SStreamQueueItem* pItem) { SStreamDataSubmit* px = (SStreamDataSubmit*)pItem; if ((pTask->taskLevel == TASK_LEVEL__SOURCE) && tInputQueueIsFull(pTask)) { qError("s-task:%s input queue is full, capacity(size:%d num:%dMiB), current(blocks:%d, size:%.2fMiB) stop to push data", - pTask->id.idStr, STREAM_TASK_INPUT_QUEUEU_CAPACITY, STREAM_TASK_INPUT_QUEUEU_CAPACITY_IN_SIZE, total, + pTask->id.idStr, STREAM_TASK_INPUT_QUEUE_CAPACITY, STREAM_TASK_INPUT_QUEUE_CAPACITY_IN_SIZE, total, size); streamDataSubmitDestroy(px); taosFreeQitem(pItem); @@ -318,7 +319,7 @@ int32_t tAppendDataToInputQueue(SStreamTask* pTask, SStreamQueueItem* pItem) { type == STREAM_INPUT__REF_DATA_BLOCK) { if ((pTask->taskLevel == TASK_LEVEL__SOURCE) && (tInputQueueIsFull(pTask))) { qError("s-task:%s input queue is full, capacity:%d size:%d MiB, current(blocks:%d, size:%.2fMiB) abort", - pTask->id.idStr, STREAM_TASK_INPUT_QUEUEU_CAPACITY, STREAM_TASK_INPUT_QUEUEU_CAPACITY_IN_SIZE, total, + pTask->id.idStr, STREAM_TASK_INPUT_QUEUE_CAPACITY, STREAM_TASK_INPUT_QUEUE_CAPACITY_IN_SIZE, total, size); destroyStreamDataBlock((SStreamDataBlock*) pItem); return -1; diff --git a/source/libs/stream/src/streamRecover.c b/source/libs/stream/src/streamRecover.c index e288694887..d3472a6a87 100644 --- a/source/libs/stream/src/streamRecover.c +++ b/source/libs/stream/src/streamRecover.c @@ -53,7 +53,7 @@ int32_t streamTaskLaunchRecover(SStreamTask* pTask) { return 0; } -// checkstatus +// check status int32_t streamTaskCheckDownstream(SStreamTask* pTask, int64_t ver) { qDebug("s-task:%s in fill history stage, ver:%"PRId64, pTask->id.idStr, ver); From 0dd933013cd53a5814a75e2ac415dec3aaf3abb5 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 5 Jun 2023 17:58:00 +0800 Subject: [PATCH 08/83] enh(stream): make history task for stream running. --- include/libs/stream/tstream.h | 6 +- source/dnode/mgmt/mgmt_vnode/src/vmWorker.c | 2 +- source/dnode/mnode/impl/src/mndScheduler.c | 9 +-- source/dnode/vnode/inc/vnode.h | 1 + source/dnode/vnode/src/tq/tq.c | 29 +++------ source/dnode/vnode/src/vnd/vnodeSvr.c | 34 +++++++++++ source/libs/stream/inc/streamInc.h | 2 +- source/libs/stream/src/stream.c | 2 + source/libs/stream/src/streamMeta.c | 3 +- source/libs/stream/src/streamRecover.c | 66 ++++++++++++++++++++- 10 files changed, 120 insertions(+), 34 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 5a5e930d42..5d2c0e4e33 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -552,10 +552,11 @@ bool streamTaskShouldPause(const SStreamStatus* pStatus); int32_t streamScanExec(SStreamTask* pTask, int32_t batchSz); // recover and fill history -int32_t streamTaskCheckDownstream(SStreamTask* pTask, int64_t version); +int32_t streamTaskCheckDownstreamTasks(SStreamTask* pTask); int32_t streamTaskLaunchRecover(SStreamTask* pTask); int32_t streamTaskCheckStatus(SStreamTask* pTask); -int32_t streamProcessTaskCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* pRsp, int64_t version); +int32_t streamProcessCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* pRsp); +int32_t streamTaskStartHistoryTask(SStreamTask* pTask, int64_t ver); // common int32_t streamSetParamForRecover(SStreamTask* pTask); @@ -570,7 +571,6 @@ int32_t streamSourceRecoverScanStep2(SStreamTask* pTask, int64_t ver); int32_t streamDispatchRecoverFinishReq(SStreamTask* pTask); // agg level int32_t streamAggRecoverPrepare(SStreamTask* pTask); -// int32_t streamAggChildrenRecoverFinish(SStreamTask* pTask); int32_t streamProcessRecoverFinishReq(SStreamTask* pTask, int32_t childId); void streamMetaInit(); diff --git a/source/dnode/mgmt/mgmt_vnode/src/vmWorker.c b/source/dnode/mgmt/mgmt_vnode/src/vmWorker.c index e41c9e10d7..02c70b9eca 100644 --- a/source/dnode/mgmt/mgmt_vnode/src/vmWorker.c +++ b/source/dnode/mgmt/mgmt_vnode/src/vmWorker.c @@ -92,7 +92,7 @@ static void vmProcessStreamQueue(SQueueInfo *pInfo, SRpcMsg *pMsg) { const STraceId *trace = &pMsg->info.traceId; dGTrace("vgId:%d, msg:%p get from vnode-stream queue", pVnode->vgId, pMsg); - int32_t code = vnodeProcessFetchMsg(pVnode->pImpl, pMsg, pInfo); + int32_t code = vnodeProcessStreamMsg(pVnode->pImpl, pMsg, pInfo); if (code != 0) { if (terrno != 0) code = terrno; dGError("vgId:%d, msg:%p failed to process stream msg %s since %s", pVnode->vgId, pMsg, TMSG_INFO(pMsg->msgType), diff --git a/source/dnode/mnode/impl/src/mndScheduler.c b/source/dnode/mnode/impl/src/mndScheduler.c index 8ae3827936..8886687f01 100644 --- a/source/dnode/mnode/impl/src/mndScheduler.c +++ b/source/dnode/mnode/impl/src/mndScheduler.c @@ -238,10 +238,6 @@ int32_t mndAddSinkTaskToStream(SStreamObj* pStream, SArray* pTaskList, SMnode* p return 0; } -static int32_t mndScheduleFillHistoryStreamTask(SMnode* pMnode, SStreamObj* pStream) { - return 0; -} - static int32_t addSourceStreamTask(SMnode* pMnode, SVgObj* pVgroup, SArray* pTaskList, SArray* pSinkTaskList, SStreamObj* pStream, SSubplan* plan, uint64_t uid, int8_t fillHistory, bool hasExtraSink) { @@ -250,6 +246,11 @@ static int32_t addSourceStreamTask(SMnode* pMnode, SVgObj* pVgroup, SArray* pTas return terrno; } + if (fillHistory) { // todo set the correct ts, which should be last key of queried table. + pTask->dataRange.window.skey = INT64_MIN; + pTask->dataRange.window.ekey = taosGetTimestampMs(); + } + // sink or dispatch if (hasExtraSink) { mndAddDispatcherForInnerTask(pMnode, pStream, pSinkTaskList, pTask); diff --git a/source/dnode/vnode/inc/vnode.h b/source/dnode/vnode/inc/vnode.h index 7e19425d56..0187a9ac6e 100644 --- a/source/dnode/vnode/inc/vnode.h +++ b/source/dnode/vnode/inc/vnode.h @@ -95,6 +95,7 @@ int32_t vnodeProcessWriteMsg(SVnode *pVnode, SRpcMsg *pMsg, int64_t version, SRp int32_t vnodeProcessSyncMsg(SVnode *pVnode, SRpcMsg *pMsg, SRpcMsg **pRsp); int32_t vnodeProcessQueryMsg(SVnode *pVnode, SRpcMsg *pMsg); int32_t vnodeProcessFetchMsg(SVnode *pVnode, SRpcMsg *pMsg, SQueueInfo *pInfo); +int32_t vnodeProcessStreamMsg(SVnode *pVnode, SRpcMsg *pMsg, SQueueInfo *pInfo); void vnodeProposeWriteMsg(SQueueInfo *pInfo, STaosQall *qall, int32_t numOfMsgs); void vnodeApplyWriteMsg(SQueueInfo *pInfo, STaosQall *qall, int32_t numOfMsgs); void vnodeProposeCommitOnNeed(SVnode *pVnode, bool atExit); diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index ce3362b677..4addda25b7 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -979,7 +979,7 @@ int32_t tqProcessStreamTaskCheckRsp(STQ* pTq, int64_t sversion, char* msg, int32 return -1; } - code = streamProcessTaskCheckRsp(pTask, &rsp, sversion); + code = streamProcessCheckRsp(pTask, &rsp); streamMetaReleaseTask(pTq->pStreamMeta, pTask); return code; } @@ -1032,25 +1032,7 @@ int32_t tqProcessTaskDeployReq(STQ* pTq, int64_t sversion, char* msg, int32_t ms tqDebug("s-task:%s fill history task, wait for being launched", pTask->id.idStr); } else { if (pTask->historyTaskId.taskId != 0) { - // todo fix the bug: 1. maybe failed to located the fill history task, since it is not built yet. 2. race condition - - // an fill history task needs to be started. - // Set the execute conditions, including the query time window and the version range - SStreamTask* pHTask = taosHashGet(pStreamMeta->pTasks, &pTask->historyTaskId.taskId, sizeof(pTask->historyTaskId.taskId)); - - pHTask->dataRange.range.minVer = 0; - pHTask->dataRange.range.maxVer = sversion; - - pHTask->dataRange.window.skey = INT64_MIN; - pHTask->dataRange.window.ekey = 1000000; - - tqDebug("s-task:%s set the launch condition for fill history task:%s, window:%" PRId64 " - %" PRId64 - " verrange:%" PRId64 " - %" PRId64, - pTask->id.idStr, pHTask->id.idStr, pHTask->dataRange.window.skey, pHTask->dataRange.window.ekey, - pHTask->dataRange.range.minVer, pHTask->dataRange.range.maxVer); - - // check if downstream tasks have been ready - streamTaskCheckDownstream(pHTask, sversion); + streamTaskStartHistoryTask(pTask, sversion); } } @@ -1091,8 +1073,11 @@ int32_t tqProcessTaskRecover1Req(STQ* pTq, SRpcMsg* pMsg) { } double el = (taosGetTimestampMs() - st) / 1000.0; - tqDebug("s-task:%s non-blocking recover stage(step 1) ended, elapsed time:%.2fs", pTask->id.idStr, el); + tqDebug("s-task:%s history scan stage(step 1) ended, elapsed time:%.2fs", pTask->id.idStr, el); + // todo transfer the executor status, and then destroy this stream task + +#if 0 // build msg to launch next step SStreamRecoverStep2Req req; code = streamBuildSourceRecover2Req(pTask, &req); @@ -1123,6 +1108,8 @@ int32_t tqProcessTaskRecover1Req(STQ* pTq, SRpcMsg* pMsg) { SRpcMsg rpcMsg = { .code = 0, .contLen = len, .msgType = TDMT_VND_STREAM_RECOVER_BLOCKING_STAGE, .pCont = serializedReq}; tmsgPutToQueue(&pTq->pVnode->msgCb, WRITE_QUEUE, &rpcMsg); +#endif + return 0; } diff --git a/source/dnode/vnode/src/vnd/vnodeSvr.c b/source/dnode/vnode/src/vnd/vnodeSvr.c index b950437f23..28a5becfd4 100644 --- a/source/dnode/vnode/src/vnd/vnodeSvr.c +++ b/source/dnode/vnode/src/vnd/vnodeSvr.c @@ -588,6 +588,40 @@ int32_t vnodeProcessFetchMsg(SVnode *pVnode, SRpcMsg *pMsg, SQueueInfo *pInfo) { } } +int32_t vnodeProcessStreamMsg(SVnode *pVnode, SRpcMsg *pMsg, SQueueInfo *pInfo) { + vTrace("vgId:%d, msg:%p in fetch queue is processing", pVnode->config.vgId, pMsg); + if ((pMsg->msgType == TDMT_SCH_FETCH || pMsg->msgType == TDMT_VND_TABLE_META || pMsg->msgType == TDMT_VND_TABLE_CFG || + pMsg->msgType == TDMT_VND_BATCH_META) && + !syncIsReadyForRead(pVnode->sync)) { + vnodeRedirectRpcMsg(pVnode, pMsg, terrno); + return 0; + } + + switch (pMsg->msgType) { + case TDMT_STREAM_TASK_RUN: + return tqProcessTaskRunReq(pVnode->pTq, pMsg); + case TDMT_STREAM_TASK_DISPATCH: + return tqProcessTaskDispatchReq(pVnode->pTq, pMsg, true); + case TDMT_STREAM_TASK_CHECK: + return tqProcessStreamTaskCheckReq(pVnode->pTq, pMsg); + case TDMT_STREAM_TASK_DISPATCH_RSP: + return tqProcessTaskDispatchRsp(pVnode->pTq, pMsg); + case TDMT_STREAM_RETRIEVE: + return tqProcessTaskRetrieveReq(pVnode->pTq, pMsg); + case TDMT_STREAM_RETRIEVE_RSP: + return tqProcessTaskRetrieveRsp(pVnode->pTq, pMsg); + case TDMT_VND_STREAM_RECOVER_NONBLOCKING_STAGE: + return tqProcessTaskRecover1Req(pVnode->pTq, pMsg); + case TDMT_STREAM_RECOVER_FINISH: + return tqProcessTaskRecoverFinishReq(pVnode->pTq, pMsg); + case TDMT_STREAM_RECOVER_FINISH_RSP: + return tqProcessTaskRecoverFinishRsp(pVnode->pTq, pMsg); + default: + vError("unknown msg type:%d in fetch queue", pMsg->msgType); + return TSDB_CODE_APP_ERROR; + } +} + // TODO: remove the function void smaHandleRes(void *pVnode, int64_t smaId, const SArray *data) { // TODO diff --git a/source/libs/stream/inc/streamInc.h b/source/libs/stream/inc/streamInc.h index 2c1956998a..4fc4d3ccf3 100644 --- a/source/libs/stream/inc/streamInc.h +++ b/source/libs/stream/inc/streamInc.h @@ -31,7 +31,7 @@ typedef struct { void* timer; } SStreamGlobalEnv; -static SStreamGlobalEnv streamEnv; +extern SStreamGlobalEnv streamEnv; int32_t streamDispatchStreamBlock(SStreamTask* pTask); diff --git a/source/libs/stream/src/stream.c b/source/libs/stream/src/stream.c index b64468a7f4..7d0a44d2b8 100644 --- a/source/libs/stream/src/stream.c +++ b/source/libs/stream/src/stream.c @@ -21,6 +21,8 @@ #define ONE_MB_F (1048576.0) #define QUEUE_MEM_SIZE_IN_MB(_q) (taosQueueMemorySize(_q) / ONE_MB_F) +SStreamGlobalEnv streamEnv; + int32_t streamInit() { int8_t old; while (1) { diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index c984bdba54..ef3ab2ae46 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -387,9 +387,10 @@ int32_t streamLoadTasks(SStreamMeta* pMeta, int64_t ver) { } // todo handle the fill history task + ASSERT(0); if (pTask->fillHistory) { ASSERT(pTask->status.taskStatus == TASK_STATUS__WAIT_DOWNSTREAM); - streamTaskCheckDownstream(pTask, ver); + streamTaskCheckDownstreamTasks(pTask); } } diff --git a/source/libs/stream/src/streamRecover.c b/source/libs/stream/src/streamRecover.c index d3472a6a87..5d366aca05 100644 --- a/source/libs/stream/src/streamRecover.c +++ b/source/libs/stream/src/streamRecover.c @@ -14,6 +14,7 @@ */ #include "streamInc.h" +#include "ttimer.h" int32_t streamTaskLaunchRecover(SStreamTask* pTask) { qDebug("s-task:%s at node %d launch recover", pTask->id.idStr, pTask->nodeId); @@ -54,8 +55,9 @@ int32_t streamTaskLaunchRecover(SStreamTask* pTask) { } // check status -int32_t streamTaskCheckDownstream(SStreamTask* pTask, int64_t ver) { - qDebug("s-task:%s in fill history stage, ver:%"PRId64, pTask->id.idStr, ver); +int32_t streamTaskCheckDownstreamTasks(SStreamTask* pTask) { + qDebug("s-task:%s in fill history stage, ver:%"PRId64" ekey:%"PRId64, pTask->id.idStr, pTask->dataRange.range.maxVer, + pTask->dataRange.window.ekey); SStreamTaskCheckReq req = { .streamId = pTask->id.streamId, @@ -135,7 +137,7 @@ int32_t streamTaskCheckStatus(SStreamTask* pTask) { return atomic_load_8(&pTask->status.taskStatus) == TASK_STATUS__NORMAL? 1:0; } -int32_t streamProcessTaskCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* pRsp, int64_t ver) { +int32_t streamProcessCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* pRsp) { ASSERT(pTask->id.taskId == pRsp->upstreamTaskId); qDebug("s-task:%s at node %d recv check rsp from task:0x%x at node %d: status %d", pTask->id.idStr, @@ -297,6 +299,64 @@ int32_t streamProcessRecoverFinishReq(SStreamTask* pTask, int32_t childId) { return 0; } +static void doCheckDownstreamStatus(SStreamTask* pTask, SStreamTask* pHTask) { + pHTask->dataRange.range.minVer = 0; + pHTask->dataRange.range.maxVer = pTask->chkInfo.currentVer; + + qDebug("s-task:%s set the launch condition for fill history task:%s, window:%" PRId64 " - %" PRId64 + " verrange:%" PRId64 " - %" PRId64, + pTask->id.idStr, pHTask->id.idStr, pHTask->dataRange.window.skey, pHTask->dataRange.window.ekey, + pHTask->dataRange.range.minVer, pHTask->dataRange.range.maxVer); + + // check if downstream tasks have been ready + streamTaskCheckDownstreamTasks(pHTask); +} + +static void tryLaunchHistoryTask(void* param, void* tmrId) { + SStreamTask* pTask = param; + + SStreamMeta* pMeta = pTask->pMeta; + SStreamTask** pHTask = taosHashGet(pMeta->pTasks, &pTask->historyTaskId.taskId, sizeof(pTask->historyTaskId.taskId)); + if (pHTask == NULL) { + qWarn("s-task:%s vgId:%d failed to launch history task:0x%x, since it is not built yet", pTask->id.idStr, + pMeta->vgId, pTask->historyTaskId.taskId); + + taosTmrReset(tryLaunchHistoryTask, (int32_t)pTask->triggerParam, pTask, streamEnv.timer, &pTask->timer); + return; + } + + doCheckDownstreamStatus(pTask, *pHTask); +} + +// todo fix the bug: 2. race condition +// an fill history task needs to be started. +int32_t streamTaskStartHistoryTask(SStreamTask* pTask, int64_t ver) { + SStreamMeta* pMeta = pTask->pMeta; + if (pTask->historyTaskId.taskId == 0) { + return TSDB_CODE_SUCCESS; + } + + // Set the execute conditions, including the query time window and the version range + SStreamTask** pHTask = taosHashGet(pMeta->pTasks, &pTask->historyTaskId.taskId, sizeof(pTask->historyTaskId.taskId)); + if (pHTask == NULL) { + qWarn("s-task:%s vgId:%d failed to launch history task:0x%x, since it is not built yet", pTask->id.idStr, + pMeta->vgId, pTask->historyTaskId.taskId); + + if (pTask->timer == NULL) { + pTask->timer = taosTmrStart(tryLaunchHistoryTask, 100, pTask, streamEnv.timer); + if (pTask->timer == NULL) { + // todo failed to create timer + } + } + + // try again in 500ms + return TSDB_CODE_SUCCESS; + } + + doCheckDownstreamStatus(pTask, *pHTask); + return TSDB_CODE_SUCCESS; +} + int32_t tEncodeSStreamTaskCheckReq(SEncoder* pEncoder, const SStreamTaskCheckReq* pReq) { if (tStartEncode(pEncoder) < 0) return -1; if (tEncodeI64(pEncoder, pReq->reqId) < 0) return -1; From 6a3c935b4a65f3cecd1fde700fec25148e11f157 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 7 Jun 2023 09:15:55 +0800 Subject: [PATCH 09/83] fix(stream): fix error in fill history scan. --- include/libs/executor/executor.h | 2 +- include/libs/executor/storageapi.h | 22 ----- include/libs/stream/tstream.h | 5 +- source/dnode/mnode/impl/src/mndScheduler.c | 96 ++++++++++-------- source/dnode/vnode/inc/vnode.h | 4 +- source/dnode/vnode/src/inc/vnodeInt.h | 3 +- source/dnode/vnode/src/meta/metaQuery.c | 20 ++-- source/dnode/vnode/src/meta/metaSma.c | 2 +- source/dnode/vnode/src/meta/metaTable.c | 2 +- source/dnode/vnode/src/sma/smaRollup.c | 4 +- source/dnode/vnode/src/tq/tq.c | 67 +++++++++---- source/dnode/vnode/src/tq/tqRead.c | 2 +- source/dnode/vnode/src/tq/tqScan.c | 2 +- source/dnode/vnode/src/tq/tqSink.c | 2 +- source/dnode/vnode/src/tsdb/tsdbCache.c | 2 +- source/dnode/vnode/src/tsdb/tsdbRead.c | 56 +---------- source/dnode/vnode/src/vnd/vnodeInitApi.c | 2 +- source/dnode/vnode/src/vnd/vnodeQuery.c | 8 +- source/dnode/vnode/src/vnd/vnodeSvr.c | 20 +--- source/libs/executor/inc/querytask.h | 6 +- source/libs/executor/src/executor.c | 6 +- source/libs/executor/src/scanoperator.c | 16 ++- source/libs/stream/inc/streamInc.h | 2 +- source/libs/stream/src/streamDispatch.c | 9 +- source/libs/stream/src/streamRecover.c | 110 +++++++++++++++------ source/libs/stream/src/streamTask.c | 104 +------------------ 26 files changed, 242 insertions(+), 332 deletions(-) diff --git a/include/libs/executor/executor.h b/include/libs/executor/executor.h index 2e4fb23207..7a8c074283 100644 --- a/include/libs/executor/executor.h +++ b/include/libs/executor/executor.h @@ -221,7 +221,7 @@ void* qExtractReaderFromStreamScanner(void* scanner); int32_t qExtractStreamScanner(qTaskInfo_t tinfo, void** scanner); int32_t qStreamSetParamForRecover(qTaskInfo_t tinfo); -int32_t qStreamSourceRecoverStep1(qTaskInfo_t tinfo, int64_t ver, int64_t ekey); +int32_t qStreamSourceRecoverStep1(qTaskInfo_t tinfo, SVersionRange *pVerRange, STimeWindow* pWindow); int32_t qStreamSourceRecoverStep2(qTaskInfo_t tinfo, int64_t ver); int32_t qStreamRecoverFinish(qTaskInfo_t tinfo); int32_t qStreamRestoreParam(qTaskInfo_t tinfo); diff --git a/include/libs/executor/storageapi.h b/include/libs/executor/storageapi.h index 4cfbf2c2fa..e9340a33c3 100644 --- a/include/libs/executor/storageapi.h +++ b/include/libs/executor/storageapi.h @@ -233,28 +233,6 @@ typedef struct SStoreSnapshotFn { int32_t (*getTableInfoFromSnapshot)(SSnapContext* ctx, void** pBuf, int32_t* contLen, int16_t* type, int64_t* uid); } SStoreSnapshotFn; -/** -void metaReaderInit(SMetaReader *pReader, SMeta *pMeta, int32_t flags); -void metaReaderReleaseLock(SMetaReader *pReader); -void metaReaderClear(SMetaReader *pReader); -int32_t metaReaderGetTableEntryByUid(SMetaReader *pReader, tb_uid_t uid); -int32_t metaReaderGetTableEntryByUidCache(SMetaReader *pReader, tb_uid_t uid); -int32_t metaGetTableTags(SMeta *pMeta, uint64_t suid, SArray *uidList); -const void *metaGetTableTagVal(void *tag, int16_t type, STagVal *tagVal); -int metaGetTableNameByUid(void *meta, uint64_t uid, char *tbName); - -int metaGetTableUidByName(void *meta, char *tbName, uint64_t *uid); -int metaGetTableTypeByName(void *meta, char *tbName, ETableType *tbType); -bool metaIsTableExist(SMeta *pMeta, tb_uid_t uid); -int32_t metaGetCachedTableUidList(SMeta *pMeta, tb_uid_t suid, const uint8_t *key, int32_t keyLen, SArray *pList, - bool *acquired); -int32_t metaUidFilterCachePut(SMeta *pMeta, uint64_t suid, const void *pKey, int32_t keyLen, void *pPayload, - int32_t payloadLen, double selectivityRatio); -tb_uid_t metaGetTableEntryUidByName(SMeta *pMeta, const char *name); -int32_t metaGetCachedTbGroup(SMeta* pMeta, tb_uid_t suid, const uint8_t* pKey, int32_t keyLen, SArray** pList); -int32_t metaPutTbGroupToCache(SMeta* pMeta, uint64_t suid, const void* pKey, int32_t keyLen, void* pPayload, int32_t payloadLen); - */ - typedef struct SStoreMeta { SMTbCursor *(*openTableMetaCursor)(void *pVnode); // metaOpenTbCursor void (*closeTableMetaCursor)(SMTbCursor *pTbCur); // metaCloseTbCursor diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 5d2c0e4e33..898db47e86 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -557,18 +557,19 @@ int32_t streamTaskLaunchRecover(SStreamTask* pTask); int32_t streamTaskCheckStatus(SStreamTask* pTask); int32_t streamProcessCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* pRsp); int32_t streamTaskStartHistoryTask(SStreamTask* pTask, int64_t ver); +int32_t streamTaskScanHistoryDataComplete(SStreamTask* pTask); // common int32_t streamSetParamForRecover(SStreamTask* pTask); int32_t streamRestoreParam(SStreamTask* pTask); int32_t streamSetStatusNormal(SStreamTask* pTask); // source level -int32_t streamSourceRecoverPrepareStep1(SStreamTask* pTask, int64_t ver, int64_t ekey); +int32_t streamSourceRecoverPrepareStep1(SStreamTask* pTask, SVersionRange *pVerRange, STimeWindow* pWindow); int32_t streamBuildSourceRecover1Req(SStreamTask* pTask, SStreamRecoverStep1Req* pReq); int32_t streamSourceRecoverScanStep1(SStreamTask* pTask); int32_t streamBuildSourceRecover2Req(SStreamTask* pTask, SStreamRecoverStep2Req* pReq); int32_t streamSourceRecoverScanStep2(SStreamTask* pTask, int64_t ver); -int32_t streamDispatchRecoverFinishReq(SStreamTask* pTask); +int32_t streamDispatchRecoverFinishMsg(SStreamTask* pTask); // agg level int32_t streamAggRecoverPrepare(SStreamTask* pTask); int32_t streamProcessRecoverFinishReq(SStreamTask* pTask, int32_t childId); diff --git a/source/dnode/mnode/impl/src/mndScheduler.c b/source/dnode/mnode/impl/src/mndScheduler.c index 8886687f01..a1c81a999f 100644 --- a/source/dnode/mnode/impl/src/mndScheduler.c +++ b/source/dnode/mnode/impl/src/mndScheduler.c @@ -25,7 +25,7 @@ #define SINK_NODE_LEVEL (0) extern bool tsDeployOnSnode; -static int32_t mndAddSinkTaskToStream(SStreamObj* pStream, SArray* pTaskList, SMnode* pMnode, int32_t vgId, SVgObj* pVgroup); +static int32_t mndAddSinkTaskToStream(SStreamObj* pStream, SArray* pTaskList, SMnode* pMnode, int32_t vgId, SVgObj* pVgroup, int32_t fillHistory); static void setFixedDownstreamEpInfo(SStreamTask* pDstTask, const SStreamTask* pTask); int32_t mndConvertRsmaTask(char** pDst, int32_t* pDstLen, const char* ast, int64_t uid, int8_t triggerType, @@ -101,7 +101,7 @@ int32_t mndSetSinkTaskInfo(SStreamObj* pStream, SStreamTask* pTask) { return 0; } -int32_t mndAddDispatcherForInnerTask(SMnode* pMnode, SStreamObj* pStream, SArray* pSinkNodeList, SStreamTask* pTask) { +int32_t mndAddDispatcherForInternalTask(SMnode* pMnode, SStreamObj* pStream, SArray* pSinkNodeList, SStreamTask* pTask) { bool isShuffle = false; if (pStream->fixedSinkVgId == 0) { @@ -202,7 +202,7 @@ SVgObj* mndSchedFetchOneVg(SMnode* pMnode, int64_t dbUid) { } // create sink node for each vgroup. -int32_t mndAddShuffleSinkTasksToStream(SMnode* pMnode, SArray* pTaskList, SStreamObj* pStream) { +int32_t mndAddShuffleSinkTasksToStream(SMnode* pMnode, SArray* pTaskList, SStreamObj* pStream, int32_t fillHistory) { SSdb* pSdb = pMnode->pSdb; void* pIter = NULL; @@ -218,15 +218,15 @@ int32_t mndAddShuffleSinkTasksToStream(SMnode* pMnode, SArray* pTaskList, SStrea continue; } - mndAddSinkTaskToStream(pStream, pTaskList, pMnode, pVgroup->vgId, pVgroup); + mndAddSinkTaskToStream(pStream, pTaskList, pMnode, pVgroup->vgId, pVgroup, fillHistory); sdbRelease(pSdb, pVgroup); } return 0; } -int32_t mndAddSinkTaskToStream(SStreamObj* pStream, SArray* pTaskList, SMnode* pMnode, int32_t vgId, SVgObj* pVgroup) { - SStreamTask* pTask = tNewStreamTask(pStream->uid, TASK_LEVEL__SINK, pStream->conf.fillHistory, 0, pTaskList); +int32_t mndAddSinkTaskToStream(SStreamObj* pStream, SArray* pTaskList, SMnode* pMnode, int32_t vgId, SVgObj* pVgroup, int32_t fillHistory) { + SStreamTask* pTask = tNewStreamTask(pStream->uid, TASK_LEVEL__SINK, fillHistory, 0, pTaskList); if (pTask == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; return -1; @@ -246,14 +246,15 @@ static int32_t addSourceStreamTask(SMnode* pMnode, SVgObj* pVgroup, SArray* pTas return terrno; } - if (fillHistory) { // todo set the correct ts, which should be last key of queried table. - pTask->dataRange.window.skey = INT64_MIN; - pTask->dataRange.window.ekey = taosGetTimestampMs(); - } + // todo set the correct ts, which should be last key of queried table. + pTask->dataRange.window.skey = INT64_MIN; + pTask->dataRange.window.ekey = 1685959190000;//taosGetTimestampMs(); + + mDebug("0x%x----------------window:%"PRId64" - %"PRId64, pTask->id.taskId, pTask->dataRange.window.skey, pTask->dataRange.window.ekey); // sink or dispatch if (hasExtraSink) { - mndAddDispatcherForInnerTask(pMnode, pStream, pSinkTaskList, pTask); + mndAddDispatcherForInternalTask(pMnode, pStream, pSinkTaskList, pTask); } else { mndSetSinkTaskInfo(pStream, pTask); } @@ -318,11 +319,12 @@ static void setHTasksId(SArray* pTaskList, const SArray* pHTaskList) { (*pStreamTask)->historyTaskId.taskId = (*pHTask)->id.taskId; (*pStreamTask)->historyTaskId.streamId = (*pHTask)->id.streamId; + mDebug("s-task:0x%x related history task:0x%x", (*pStreamTask)->id.taskId, (*pHTask)->id.taskId); } } -static int32_t addSourceTasksForSingleLevelStream(SMnode* pMnode, const SQueryPlan* pPlan, SStreamObj* pStream, - bool hasExtraSink) { +static int32_t addSourceTasksForOneLevelStream(SMnode* pMnode, const SQueryPlan* pPlan, SStreamObj* pStream, + bool hasExtraSink) { // create exec stream task, since only one level, the exec task is also the source task SArray* pTaskList = addNewTaskList(pStream->tasks); @@ -368,8 +370,8 @@ static int32_t addSourceTasksForSingleLevelStream(SMnode* pMnode, const SQueryPl if (pStream->conf.fillHistory) { SArray** pHSinkTaskList = taosArrayGet(pStream->pHTasksList, SINK_NODE_LEVEL); - code = addSourceStreamTask(pMnode, pVgroup, pHTaskList, *pHSinkTaskList, pStream, plan, pStream->hTaskUid, pStream->conf.fillHistory, - hasExtraSink); + code = addSourceStreamTask(pMnode, pVgroup, pHTaskList, *pHSinkTaskList, pStream, plan, pStream->hTaskUid, + pStream->conf.fillHistory, hasExtraSink); setHTasksId(pTaskList, pHTaskList); } @@ -390,6 +392,13 @@ static int32_t doAddSourceTask(SArray* pTaskList, int8_t fillHistory, int64_t ui return -1; } + // todo set the correct ts, which should be last key of queried table. + pTask->dataRange.window.skey = INT64_MIN; + pTask->dataRange.window.ekey = 1685959190000;//taosGetTimestampMs(); + + mDebug("0x%x----------------window:%"PRId64" - %"PRId64, pTask->id.taskId, pTask->dataRange.window.skey, pTask->dataRange.window.ekey); + + // all the source tasks dispatch result to a single agg node. setFixedDownstreamEpInfo(pTask, pDownstreamTask); if (mndAssignStreamTaskToVgroup(pMnode, pTask, pPlan, pVgroup) < 0) { @@ -399,38 +408,38 @@ static int32_t doAddSourceTask(SArray* pTaskList, int8_t fillHistory, int64_t ui return setEpToDownstreamTask(pTask, pDownstreamTask); } -static int32_t doAddAggTask(uint64_t uid, SArray* pTaskList, SArray* pSinkNodeList, SMnode* pMnode, SStreamObj* pStream, +static int32_t doAddAggTask(uint64_t uid, SArray* pTaskList, SArray* pSinkNodeList, SMnode* pMnode, SStreamObj* pStream, int32_t fillHistory, SStreamTask** pAggTask) { - *pAggTask = tNewStreamTask(uid, TASK_LEVEL__AGG, pStream->conf.fillHistory, - pStream->conf.triggerParam, pTaskList); + *pAggTask = tNewStreamTask(uid, TASK_LEVEL__AGG, fillHistory, pStream->conf.triggerParam, pTaskList); if (*pAggTask == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; return -1; } // dispatch - if (mndAddDispatcherForInnerTask(pMnode, pStream, pSinkNodeList, *pAggTask) < 0) { + if (mndAddDispatcherForInternalTask(pMnode, pStream, pSinkNodeList, *pAggTask) < 0) { return -1; } return 0; } -static int32_t addAggTask(SStreamObj* pStream, SMnode* pMnode, SQueryPlan* pPlan) { +static int32_t addAggTask(SStreamObj* pStream, SMnode* pMnode, SQueryPlan* pPlan, SStreamTask** pAggTask, + SStreamTask** pHAggTask) { SArray* pAggTaskList = addNewTaskList(pStream->tasks); SSdb* pSdb = pMnode->pSdb; - SNodeListNode* inner = (SNodeListNode*)nodesListGetNode(pPlan->pSubplans, 0); - SSubplan* plan = (SSubplan*)nodesListGetNode(inner->pNodeList, 0); + SNodeListNode* pInnerNode = (SNodeListNode*)nodesListGetNode(pPlan->pSubplans, 0); + SSubplan* plan = (SSubplan*)nodesListGetNode(pInnerNode->pNodeList, 0); if (plan->subplanType != SUBPLAN_TYPE_MERGE) { terrno = TSDB_CODE_QRY_INVALID_INPUT; return -1; } - SStreamTask* pAggTask = NULL; - SArray* pSinkNodeList = taosArrayGet(pStream->tasks, SINK_NODE_LEVEL); + *pAggTask = NULL; + SArray* pSinkNodeList = taosArrayGetP(pStream->tasks, SINK_NODE_LEVEL); - int32_t code = doAddAggTask(pStream->uid, pAggTaskList, pSinkNodeList, pMnode, pStream, &pAggTask); + int32_t code = doAddAggTask(pStream->uid, pAggTaskList, pSinkNodeList, pMnode, pStream, 0, pAggTask); if (code != TSDB_CODE_SUCCESS) { return -1; } @@ -448,16 +457,17 @@ static int32_t addAggTask(SStreamObj* pStream, SMnode* pMnode, SQueryPlan* pPlan } if (pSnode != NULL) { - code = mndAssignStreamTaskToSnode(pMnode, pAggTask, plan, pSnode); + code = mndAssignStreamTaskToSnode(pMnode, *pAggTask, plan, pSnode); } else { - code = mndAssignStreamTaskToVgroup(pMnode, pAggTask, plan, pVgroup); + code = mndAssignStreamTaskToVgroup(pMnode, *pAggTask, plan, pVgroup); } if (pStream->conf.fillHistory) { SArray* pHAggTaskList = addNewTaskList(pStream->pHTasksList); + SArray* pHSinkNodeList = taosArrayGetP(pStream->pHTasksList, SINK_NODE_LEVEL); - SStreamTask* pHAggTask = NULL; - code = doAddAggTask(pStream->uid, pAggTaskList, pSinkNodeList, pMnode, pStream, &pHAggTask); + *pHAggTask = NULL; + code = doAddAggTask(pStream->hTaskUid, pHAggTaskList, pHSinkNodeList, pMnode, pStream, pStream->conf.fillHistory, pHAggTask); if (code != TSDB_CODE_SUCCESS) { if (pSnode != NULL) { sdbRelease(pSdb, pSnode); @@ -468,9 +478,9 @@ static int32_t addAggTask(SStreamObj* pStream, SMnode* pMnode, SQueryPlan* pPlan } if (pSnode != NULL) { - code = mndAssignStreamTaskToSnode(pMnode, pHAggTask, plan, pSnode); + code = mndAssignStreamTaskToSnode(pMnode, *pHAggTask, plan, pSnode); } else { - code = mndAssignStreamTaskToVgroup(pMnode, pHAggTask, plan, pVgroup); + code = mndAssignStreamTaskToVgroup(pMnode, *pHAggTask, plan, pVgroup); } setHTasksId(pAggTaskList, pHAggTaskList); @@ -515,7 +525,7 @@ static int32_t addSourceTasksForMultiLevelStream(SMnode* pMnode, SQueryPlan* pPl continue; } - int32_t code = doAddSourceTask(pSourceTaskList, pStream->conf.fillHistory, pStream->uid, pDownstreamTask, pMnode, plan, pVgroup); + int32_t code = doAddSourceTask(pSourceTaskList, 0, pStream->uid, pDownstreamTask, pMnode, plan, pVgroup); if (code != TSDB_CODE_SUCCESS) { sdbRelease(pSdb, pVgroup); terrno = code; @@ -523,7 +533,7 @@ static int32_t addSourceTasksForMultiLevelStream(SMnode* pMnode, SQueryPlan* pPl } if (pStream->conf.fillHistory) { - code = doAddSourceTask(pHSourceTaskList, 0, pStream->hTaskUid, pHDownstreamTask, pMnode, plan, pVgroup); + code = doAddSourceTask(pHSourceTaskList, pStream->conf.fillHistory, pStream->hTaskUid, pHDownstreamTask, pMnode, plan, pVgroup); sdbRelease(pSdb, pVgroup); if (code != TSDB_CODE_SUCCESS) { @@ -537,15 +547,15 @@ static int32_t addSourceTasksForMultiLevelStream(SMnode* pMnode, SQueryPlan* pPl return TSDB_CODE_SUCCESS; } -static int32_t addSinkTasks(SArray* pTasksList, SMnode* pMnode, SStreamObj* pStream, SArray** pCreatedTaskList) { +static int32_t addSinkTasks(SArray* pTasksList, SMnode* pMnode, SStreamObj* pStream, SArray** pCreatedTaskList, int32_t fillHistory) { SArray* pSinkTaskList = addNewTaskList(pTasksList); if (pStream->fixedSinkVgId == 0) { - if (mndAddShuffleSinkTasksToStream(pMnode, pSinkTaskList, pStream) < 0) { + if (mndAddShuffleSinkTasksToStream(pMnode, pSinkTaskList, pStream, fillHistory) < 0) { // TODO free return -1; } } else { - if (mndAddSinkTaskToStream(pStream, pSinkTaskList, pMnode, pStream->fixedSinkVgId, &pStream->fixedSinkVg) < 0) { + if (mndAddSinkTaskToStream(pStream, pSinkTaskList, pMnode, pStream->fixedSinkVgId, &pStream->fixedSinkVg, fillHistory) < 0) { // TODO free return -1; } @@ -578,7 +588,7 @@ static int32_t doScheduleStream(SStreamObj* pStream, SMnode* pMnode, SQueryPlan* hasExtraSink = true; SArray* pSinkTaskList = NULL; - int32_t code = addSinkTasks(pStream->tasks, pMnode, pStream, &pSinkTaskList); + int32_t code = addSinkTasks(pStream->tasks, pMnode, pStream, &pSinkTaskList, 0); if (code != TSDB_CODE_SUCCESS) { return code; } @@ -586,7 +596,7 @@ static int32_t doScheduleStream(SStreamObj* pStream, SMnode* pMnode, SQueryPlan* // check for fill history if (pStream->conf.fillHistory) { SArray* pHSinkTaskList = NULL; - code = addSinkTasks(pStream->pHTasksList, pMnode, pStream, &pHSinkTaskList); + code = addSinkTasks(pStream->pHTasksList, pMnode, pStream, &pHSinkTaskList, pStream->conf.fillHistory); if (code != TSDB_CODE_SUCCESS) { return code; } @@ -598,16 +608,18 @@ static int32_t doScheduleStream(SStreamObj* pStream, SMnode* pMnode, SQueryPlan* pStream->totalLevel = numOfPlanLevel + hasExtraSink; if (numOfPlanLevel > 1) { - SStreamTask* pInnerTask; - int32_t code = addAggTask(pStream, pMnode, pPlan); + SStreamTask* pAggTask = NULL; + SStreamTask* pHAggTask = NULL; + + int32_t code = addAggTask(pStream, pMnode, pPlan, &pAggTask, &pHAggTask); if (code != TSDB_CODE_SUCCESS) { return code; } // source level - return addSourceTasksForMultiLevelStream(pMnode, pPlan, pStream, pInnerTask, NULL); + return addSourceTasksForMultiLevelStream(pMnode, pPlan, pStream, pAggTask, pHAggTask); } else if (numOfPlanLevel == 1) { - return addSourceTasksForSingleLevelStream(pMnode, pPlan, pStream, hasExtraSink); + return addSourceTasksForOneLevelStream(pMnode, pPlan, pStream, hasExtraSink); } return 0; diff --git a/source/dnode/vnode/inc/vnode.h b/source/dnode/vnode/inc/vnode.h index 0187a9ac6e..e29d2f7a8f 100644 --- a/source/dnode/vnode/inc/vnode.h +++ b/source/dnode/vnode/inc/vnode.h @@ -101,7 +101,7 @@ void vnodeApplyWriteMsg(SQueueInfo *pInfo, STaosQall *qall, int32_t numOfMsgs void vnodeProposeCommitOnNeed(SVnode *pVnode, bool atExit); // meta -void _metaReaderInit(SMetaReader *pReader, void *pVnode, int32_t flags, SStoreMeta* pAPI); +void metaReaderInit(SMetaReader *pReader, void *pVnode, int32_t flags, SStoreMeta* pAPI); void metaReaderReleaseLock(SMetaReader *pReader); void metaReaderClear(SMetaReader *pReader); int32_t metaReaderGetTableEntryByUid(SMetaReader *pReader, tb_uid_t uid); @@ -125,8 +125,6 @@ int32_t metaGetCachedTbGroup(void *pVnode, tb_uid_t suid, const uint8_t *pKey, int32_t metaPutTbGroupToCache(void* pVnode, uint64_t suid, const void *pKey, int32_t keyLen, void *pPayload, int32_t payloadLen); -int64_t metaGetTbNum(SMeta *pMeta); - int32_t metaGetStbStats(void *pVnode, int64_t uid, int64_t *numOfTables); // tsdb diff --git a/source/dnode/vnode/src/inc/vnodeInt.h b/source/dnode/vnode/src/inc/vnodeInt.h index db285dc124..607ff8665a 100644 --- a/source/dnode/vnode/src/inc/vnodeInt.h +++ b/source/dnode/vnode/src/inc/vnodeInt.h @@ -175,7 +175,8 @@ void* metaGetIdx(SMeta* pMeta); void* metaGetIvtIdx(SMeta* pMeta); int metaTtlSmaller(SMeta* pMeta, uint64_t time, SArray* uidList); -void metaReaderInit(SMetaReader *pReader, SMeta *pMeta, int32_t flags); +int64_t metaGetTbNum(SMeta *pMeta); +void metaReaderDoInit(SMetaReader *pReader, SMeta *pMeta, int32_t flags); int32_t metaCreateTSma(SMeta* pMeta, int64_t version, SSmaCfg* pCfg); int32_t metaDropTSma(SMeta* pMeta, int64_t indexUid); diff --git a/source/dnode/vnode/src/meta/metaQuery.c b/source/dnode/vnode/src/meta/metaQuery.c index fa9eea5e29..8cfe41340a 100644 --- a/source/dnode/vnode/src/meta/metaQuery.c +++ b/source/dnode/vnode/src/meta/metaQuery.c @@ -17,13 +17,13 @@ #include "osMemory.h" #include "tencode.h" -void _metaReaderInit(SMetaReader* pReader, void* pVnode, int32_t flags, SStoreMeta* pAPI) { +void metaReaderInit(SMetaReader* pReader, void* pVnode, int32_t flags, SStoreMeta* pAPI) { SMeta* pMeta = ((SVnode*)pVnode)->pMeta; - metaReaderInit(pReader, pMeta, flags); + metaReaderDoInit(pReader, pMeta, flags); pReader->pAPI = pAPI; } -void metaReaderInit(SMetaReader *pReader, SMeta *pMeta, int32_t flags) { +void metaReaderDoInit(SMetaReader *pReader, SMeta *pMeta, int32_t flags) { memset(pReader, 0, sizeof(*pReader)); pReader->pMeta = pMeta; pReader->flags = flags; @@ -143,7 +143,7 @@ tb_uid_t metaGetTableEntryUidByName(SMeta *pMeta, const char *name) { int metaGetTableNameByUid(void *pVnode, uint64_t uid, char *tbName) { int code = 0; SMetaReader mr = {0}; - metaReaderInit(&mr, ((SVnode*)pVnode)->pMeta, 0); + metaReaderDoInit(&mr, ((SVnode*)pVnode)->pMeta, 0); code = metaReaderGetTableEntryByUid(&mr, uid); if (code < 0) { metaReaderClear(&mr); @@ -159,7 +159,7 @@ int metaGetTableNameByUid(void *pVnode, uint64_t uid, char *tbName) { int metaGetTableSzNameByUid(void *meta, uint64_t uid, char *tbName) { int code = 0; SMetaReader mr = {0}; - metaReaderInit(&mr, (SMeta *)meta, 0); + metaReaderDoInit(&mr, (SMeta *)meta, 0); code = metaReaderGetTableEntryByUid(&mr, uid); if (code < 0) { metaReaderClear(&mr); @@ -174,7 +174,7 @@ int metaGetTableSzNameByUid(void *meta, uint64_t uid, char *tbName) { int metaGetTableUidByName(void *pVnode, char *tbName, uint64_t *uid) { int code = 0; SMetaReader mr = {0}; - metaReaderInit(&mr, ((SVnode *)pVnode)->pMeta, 0); + metaReaderDoInit(&mr, ((SVnode *)pVnode)->pMeta, 0); SMetaReader *pReader = &mr; @@ -195,7 +195,7 @@ int metaGetTableUidByName(void *pVnode, char *tbName, uint64_t *uid) { int metaGetTableTypeByName(void *pVnode, char *tbName, ETableType *tbType) { int code = 0; SMetaReader mr = {0}; - metaReaderInit(&mr, ((SVnode*)pVnode)->pMeta, 0); + metaReaderDoInit(&mr, ((SVnode*)pVnode)->pMeta, 0); code = metaGetTableEntryByName(&mr, tbName); if (code == 0) *tbType = mr.me.type; @@ -222,7 +222,7 @@ SMTbCursor *metaOpenTbCursor(void *pVnode) { } SVnode* pVnodeObj = pVnode; - metaReaderInit(&pTbCur->mr, pVnodeObj->pMeta, 0); + metaReaderDoInit(&pTbCur->mr, pVnodeObj->pMeta, 0); tdbTbcOpen(pVnodeObj->pMeta->pUidIdx, (TBC **)&pTbCur->pDbc, NULL); @@ -760,7 +760,7 @@ STSmaWrapper *metaGetSmaInfoByTable(SMeta *pMeta, tb_uid_t uid, bool deepCopy) { } SMetaReader mr = {0}; - metaReaderInit(&mr, pMeta, 0); + metaReaderDoInit(&mr, pMeta, 0); int64_t smaId; int smaIdx = 0; STSma *pTSma = NULL; @@ -815,7 +815,7 @@ _err: STSma *metaGetSmaInfoByIndex(SMeta *pMeta, int64_t indexUid) { STSma *pTSma = NULL; SMetaReader mr = {0}; - metaReaderInit(&mr, pMeta, 0); + metaReaderDoInit(&mr, pMeta, 0); if (metaReaderGetTableEntryByUid(&mr, indexUid) < 0) { metaWarn("vgId:%d, failed to get table entry for smaId:%" PRIi64, TD_VID(pMeta->pVnode), indexUid); metaReaderClear(&mr); diff --git a/source/dnode/vnode/src/meta/metaSma.c b/source/dnode/vnode/src/meta/metaSma.c index a49848f442..91704f5c7a 100644 --- a/source/dnode/vnode/src/meta/metaSma.c +++ b/source/dnode/vnode/src/meta/metaSma.c @@ -37,7 +37,7 @@ int32_t metaCreateTSma(SMeta *pMeta, int64_t version, SSmaCfg *pCfg) { // validate req // save smaIndex - metaReaderInit(&mr, pMeta, 0); + metaReaderDoInit(&mr, pMeta, 0); if (metaReaderGetTableEntryByUidCache(&mr, pCfg->indexUid) == 0) { #if 1 terrno = TSDB_CODE_TSMA_ALREADY_EXIST; diff --git a/source/dnode/vnode/src/meta/metaTable.c b/source/dnode/vnode/src/meta/metaTable.c index df16304e28..def008311a 100644 --- a/source/dnode/vnode/src/meta/metaTable.c +++ b/source/dnode/vnode/src/meta/metaTable.c @@ -709,7 +709,7 @@ int metaCreateTable(SMeta *pMeta, int64_t ver, SVCreateTbReq *pReq, STableMetaRs } // validate req - metaReaderInit(&mr, pMeta, 0); + metaReaderDoInit(&mr, pMeta, 0); if (metaGetTableEntryByName(&mr, pReq->name) == 0) { if (pReq->type == TSDB_CHILD_TABLE && pReq->ctb.suid != mr.me.ctbEntry.suid) { terrno = TSDB_CODE_TDB_TABLE_IN_OTHER_STABLE; diff --git a/source/dnode/vnode/src/sma/smaRollup.c b/source/dnode/vnode/src/sma/smaRollup.c index 39aa5c3043..d393f4b6bc 100644 --- a/source/dnode/vnode/src/sma/smaRollup.c +++ b/source/dnode/vnode/src/sma/smaRollup.c @@ -896,7 +896,7 @@ static int32_t tdRSmaInfoClone(SSma *pSma, SRSmaInfo *pInfo) { return TSDB_CODE_SUCCESS; } - metaReaderInit(&mr, SMA_META(pSma), 0); + metaReaderDoInit(&mr, SMA_META(pSma), 0); smaDebug("vgId:%d, rsma clone qTaskInfo for suid:%" PRIi64, SMA_VID(pSma), pInfo->suid); if (metaReaderGetTableEntryByUidCache(&mr, pInfo->suid) < 0) { code = terrno; @@ -1116,7 +1116,7 @@ static int32_t tdRSmaRestoreQTaskInfoInit(SSma *pSma, int64_t *nTables) { } int64_t nRsmaTables = 0; - metaReaderInit(&mr, SMA_META(pSma), 0); + metaReaderDoInit(&mr, SMA_META(pSma), 0); if (!(uidStore.tbUids = taosArrayInit(1024, sizeof(tb_uid_t)))) { code = TSDB_CODE_OUT_OF_MEMORY; TSDB_CHECK_CODE(code, lino, _exit); diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 4addda25b7..84e37649ac 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -812,11 +812,15 @@ int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t ver) { pTask->outputStatus = TASK_OUTPUT_STATUS__NORMAL; pTask->pMsgCb = &pTq->pVnode->msgCb; pTask->pMeta = pTq->pStreamMeta; + pTask->chkInfo.version = ver; pTask->chkInfo.currentVer = ver; + pTask->dataRange.range.maxVer = ver; + pTask->dataRange.range.minVer = ver; + // expand executor - pTask->status.taskStatus = (pTask->fillHistory) ? TASK_STATUS__WAIT_DOWNSTREAM : TASK_STATUS__NORMAL; + pTask->status.taskStatus = /*(pTask->fillHistory) ? */TASK_STATUS__WAIT_DOWNSTREAM /*: TASK_STATUS__NORMAL*/; if (pTask->taskLevel == TASK_LEVEL__SOURCE) { pTask->pState = streamStateOpen(pTq->pStreamMeta->path, pTask, false, -1, -1); @@ -919,16 +923,12 @@ int32_t tqProcessStreamTaskCheckReq(STQ* pTq, SRpcMsg* pMsg) { rsp.status = streamTaskCheckStatus(pTask); streamMetaReleaseTask(pTq->pStreamMeta, pTask); - tqDebug("s-task:%s recv task check req(reqId:0x%" PRIx64 - ") %d at node %d task status:%d, check req from task %d at node %d, rsp status %d", - pTask->id.idStr, rsp.reqId, rsp.downstreamTaskId, rsp.downstreamNodeId, pTask->status.taskStatus, - rsp.upstreamTaskId, rsp.upstreamNodeId, rsp.status); + tqDebug("s-task:%s recv task check req(reqId:0x%" PRIx64 ") task:0x%x (vgId:%d), status:%d, rsp status %d", + pTask->id.idStr, rsp.reqId, rsp.upstreamTaskId, rsp.upstreamNodeId, pTask->status.taskStatus, rsp.status); } else { rsp.status = 0; - tqDebug("tq recv task check(taskId:0x%x not built yet) req(reqId:0x%" PRIx64 - ") %d at node %d, check req from task:0x%x at node %d, rsp status %d", - taskId, rsp.reqId, rsp.downstreamTaskId, rsp.downstreamNodeId, rsp.upstreamTaskId, rsp.upstreamNodeId, - rsp.status); + tqDebug("tq recv task check(taskId:0x%x not built yet) req(reqId:0x%" PRIx64 ") from task:0x%x (vgId:%d), rsp status %d", + taskId, rsp.reqId, rsp.upstreamTaskId, rsp.upstreamNodeId, rsp.status); } SEncoder encoder; @@ -969,12 +969,12 @@ int32_t tqProcessStreamTaskCheckRsp(STQ* pTq, int64_t sversion, char* msg, int32 } tDecoderClear(&decoder); - tqDebug("tq recv task check rsp(reqId:0x%" PRIx64 ") %d at node %d check req from task:0x%x at node %d, status %d", + tqDebug("tq recv task check rsp(reqId:0x%" PRIx64 ") %d (vgId:%d) check req from task:0x%x (vgId:%d), status %d", rsp.reqId, rsp.downstreamTaskId, rsp.downstreamNodeId, rsp.upstreamTaskId, rsp.upstreamNodeId, rsp.status); SStreamTask* pTask = streamMetaAcquireTask(pTq->pStreamMeta, rsp.upstreamTaskId); if (pTask == NULL) { - tqError("tq failed to locate the stream task:0x%x vgId:%d, it may have been destroyed", rsp.upstreamTaskId, + tqError("tq failed to locate the stream task:0x%x (vgId:%d), it may have been destroyed", rsp.upstreamTaskId, pTq->pStreamMeta->vgId); return -1; } @@ -1027,13 +1027,27 @@ int32_t tqProcessTaskDeployReq(STQ* pTq, int64_t sversion, char* msg, int32_t ms taosWUnLockLatch(&pStreamMeta->lock); - // 3. for fill history task, do nothing. wait for the main task to start it + // 3. It's an fill history task, do nothing. wait for the main task to start it if (pTask->fillHistory) { tqDebug("s-task:%s fill history task, wait for being launched", pTask->id.idStr); } else { + // calculate the correct start time window, and start the handle the history data for the main task. if (pTask->historyTaskId.taskId != 0) { + // launch the history fill stream task streamTaskStartHistoryTask(pTask, sversion); + + // launch current task + SHistoryDataRange* pRange = &pTask->dataRange; + int64_t ekey = pRange->window.ekey; + int64_t ver = pRange->range.minVer; + + pRange->window.skey = ekey; + pRange->window.ekey = INT64_MAX; + pRange->range.minVer = 0; + pRange->range.maxVer = ver; } + + streamTaskCheckDownstreamTasks(pTask); } tqDebug("vgId:%d s-task:%s is deployed and add meta from mnd, status:%d, total:%d", vgId, pTask->id.idStr, @@ -1043,20 +1057,24 @@ int32_t tqProcessTaskDeployReq(STQ* pTq, int64_t sversion, char* msg, int32_t ms } int32_t tqProcessTaskRecover1Req(STQ* pTq, SRpcMsg* pMsg) { - int32_t code; + int32_t code = TSDB_CODE_SUCCESS; char* msg = pMsg->pCont; int32_t msgLen = pMsg->contLen; + SStreamMeta* pMeta = pTq->pStreamMeta; SStreamRecoverStep1Req* pReq = (SStreamRecoverStep1Req*)msg; - SStreamTask* pTask = streamMetaAcquireTask(pTq->pStreamMeta, pReq->taskId); + + SStreamTask* pTask = streamMetaAcquireTask(pMeta, pReq->taskId); if (pTask == NULL) { + tqError("vgId:%d failed to acquire stream task:0x%x during stream recover, task may have been destroyed", + pMeta->vgId, pReq->taskId); return -1; } // check param int64_t fillVer1 = pTask->chkInfo.version; if (fillVer1 <= 0) { - streamMetaReleaseTask(pTq->pStreamMeta, pTask); + streamMetaReleaseTask(pMeta, pTask); return -1; } @@ -1068,14 +1086,27 @@ int32_t tqProcessTaskRecover1Req(STQ* pTq, SRpcMsg* pMsg) { if (atomic_load_8(&pTask->status.taskStatus) == TASK_STATUS__DROPPING) { tqDebug("s-task:%s is dropped, abort recover in step1", pTask->id.idStr); - streamMetaReleaseTask(pTq->pStreamMeta, pTask); + streamMetaReleaseTask(pMeta, pTask); return 0; } double el = (taosGetTimestampMs() - st) / 1000.0; tqDebug("s-task:%s history scan stage(step 1) ended, elapsed time:%.2fs", pTask->id.idStr, el); - // todo transfer the executor status, and then destroy this stream task + if (pTask->fillHistory) { + // todo transfer the executor status, and then destroy this stream task + } else { + // todo update the chkInfo version for current task. + // this task has an associated history stream task, so we need to scan wal from the end version of + // history scan. The current version of chkInfo.current is not updated during the history scan + tqDebug("s-task:%s history data scan completed, now start to scan data from wal, start ver:%" PRId64 + ", window:%" PRId64 " - %" PRId64, + pTask->id.idStr, pTask->chkInfo.currentVer, pTask->dataRange.window.skey, pTask->dataRange.window.ekey); + + code = streamTaskScanHistoryDataComplete(pTask); + streamMetaReleaseTask(pMeta, pTask); + return code; + } #if 0 // build msg to launch next step @@ -1162,7 +1193,7 @@ int32_t tqProcessTaskRecover2Req(STQ* pTq, int64_t sversion, char* msg, int32_t tqDebug("s-task:%s step2 recover finished, el:%.2fs", pTask->id.idStr, el); // dispatch recover finish req to all related downstream task - code = streamDispatchRecoverFinishReq(pTask); + code = streamDispatchRecoverFinishMsg(pTask); if (code < 0) { streamMetaReleaseTask(pTq->pStreamMeta, pTask); return -1; diff --git a/source/dnode/vnode/src/tq/tqRead.c b/source/dnode/vnode/src/tq/tqRead.c index 77a966715e..155ef92ae5 100644 --- a/source/dnode/vnode/src/tq/tqRead.c +++ b/source/dnode/vnode/src/tq/tqRead.c @@ -114,7 +114,7 @@ bool isValValidForTable(STqHandle* pHandle, SWalCont* pHead) { } SMetaReader mr = {0}; - metaReaderInit(&mr, pHandle->execHandle.pTqReader->pVnodeMeta, 0); + metaReaderDoInit(&mr, pHandle->execHandle.pTqReader->pVnodeMeta, 0); if (metaGetTableEntryByName(&mr, req.tbName) < 0) { metaReaderClear(&mr); diff --git a/source/dnode/vnode/src/tq/tqScan.c b/source/dnode/vnode/src/tq/tqScan.c index 1ff78c586f..2fd78f1309 100644 --- a/source/dnode/vnode/src/tq/tqScan.c +++ b/source/dnode/vnode/src/tq/tqScan.c @@ -48,7 +48,7 @@ static int32_t tqAddBlockSchemaToRsp(const STqExecHandle* pExec, STaosxRsp* pRsp static int32_t tqAddTbNameToRsp(const STQ* pTq, int64_t uid, STaosxRsp* pRsp, int32_t n) { SMetaReader mr = {0}; - metaReaderInit(&mr, pTq->pVnode->pMeta, 0); + metaReaderDoInit(&mr, pTq->pVnode->pMeta, 0); // TODO add reference to gurantee success if (metaReaderGetTableEntryByUidCache(&mr, uid) < 0) { diff --git a/source/dnode/vnode/src/tq/tqSink.c b/source/dnode/vnode/src/tq/tqSink.c index db1b5ed902..604b171daf 100644 --- a/source/dnode/vnode/src/tq/tqSink.c +++ b/source/dnode/vnode/src/tq/tqSink.c @@ -311,7 +311,7 @@ void tqSinkToTablePipeline(SStreamTask* pTask, void* vnode, int64_t ver, void* d tbData.uid = pTableSinkInfo->uid; } else { SMetaReader mr = {0}; - metaReaderInit(&mr, pVnode->pMeta, 0); + metaReaderDoInit(&mr, pVnode->pMeta, 0); if (metaGetTableEntryByName(&mr, ctbName) < 0) { metaReaderClear(&mr); taosMemoryFree(pTableSinkInfo); diff --git a/source/dnode/vnode/src/tsdb/tsdbCache.c b/source/dnode/vnode/src/tsdb/tsdbCache.c index c659c8f4a2..fc1600dc3a 100644 --- a/source/dnode/vnode/src/tsdb/tsdbCache.c +++ b/source/dnode/vnode/src/tsdb/tsdbCache.c @@ -1431,7 +1431,7 @@ static tb_uid_t getTableSuidByUid(tb_uid_t uid, STsdb *pTsdb) { tb_uid_t suid = 0; SMetaReader mr = {0}; - metaReaderInit(&mr, pTsdb->pVnode->pMeta, 0); + metaReaderDoInit(&mr, pTsdb->pVnode->pMeta, 0); if (metaReaderGetTableEntryByUidCache(&mr, uid) < 0) { metaReaderClear(&mr); // table not esist return 0; diff --git a/source/dnode/vnode/src/tsdb/tsdbRead.c b/source/dnode/vnode/src/tsdb/tsdbRead.c index 2500015ec1..10d19da4c9 100644 --- a/source/dnode/vnode/src/tsdb/tsdbRead.c +++ b/source/dnode/vnode/src/tsdb/tsdbRead.c @@ -5452,7 +5452,7 @@ int64_t tsdbGetNumOfRowsInMemTable(STsdbReader* pReader) { int32_t tsdbGetTableSchema(void* pVnode, int64_t uid, STSchema** pSchema, int64_t* suid) { SMetaReader mr = {0}; - metaReaderInit(&mr, ((SVnode*)pVnode)->pMeta, 0); + metaReaderDoInit(&mr, ((SVnode*)pVnode)->pMeta, 0); int32_t code = metaReaderGetTableEntryByUidCache(&mr, uid); if (code != TSDB_CODE_SUCCESS) { terrno = TSDB_CODE_TDB_INVALID_TABLE_ID; @@ -5584,57 +5584,3 @@ void tsdbReaderSetId(STsdbReader* pReader, const char* idstr) { void tsdbReaderSetCloseFlag(STsdbReader* pReader) { pReader->code = TSDB_CODE_TSC_QUERY_CANCELLED; } -/*-------------todo:refactor the implementation of those APIs in this file to seperate the API into two files------*/ -// opt perf, do NOT create so many readers -int64_t tsdbGetLastTimestamp(SVnode* pVnode, void* pTableList, int32_t numOfTables, const char* pIdStr) { - SQueryTableDataCond cond = {.type = TIMEWINDOW_RANGE_CONTAINED, .numOfCols = 1, .order = TSDB_ORDER_DESC, - .startVersion = -1, .endVersion = -1}; - cond.twindows.skey = INT64_MIN; - cond.twindows.ekey = INT64_MAX; - - cond.colList = taosMemoryCalloc(1, sizeof(SColumnInfo)); - cond.pSlotList = taosMemoryMalloc(sizeof(int32_t) * cond.numOfCols); - if (cond.colList == NULL || cond.pSlotList == NULL) { - // todo - } - - cond.colList[0].colId = 1; - cond.colList[0].slotId = 0; - cond.colList[0].type = TSDB_DATA_TYPE_TIMESTAMP; - - cond.pSlotList[0] = 0; - - STableKeyInfo* pTableKeyInfo = pTableList; - STsdbReader* pReader = NULL; - SSDataBlock* pBlock = createDataBlock(); - - SColumnInfoData data = {0}; - data.info = (SColumnInfo) {.type = TSDB_DATA_TYPE_TIMESTAMP, .colId = 1, .bytes = TSDB_KEYSIZE}; - blockDataAppendColInfo(pBlock, &data); - - int64_t key = INT64_MIN; - - for(int32_t i = 0; i < numOfTables; ++i) { - int32_t code = tsdbReaderOpen(pVnode, &cond, &pTableKeyInfo[i], 1, pBlock, (void**)&pReader, pIdStr, false, NULL); - if (code != TSDB_CODE_SUCCESS) { - return code; - } - - bool hasData = false; - code = tsdbNextDataBlock(pReader, &hasData); - if (!hasData || code != TSDB_CODE_SUCCESS) { - continue; - } - - SColumnInfoData* pCol = taosArrayGet(pBlock->pDataBlock, 0); - int64_t k = *(int64_t*)pCol->pData; - - if (key < k) { - key = k; - } - - tsdbReaderClose(pReader); - } - - return 0; -} diff --git a/source/dnode/vnode/src/vnd/vnodeInitApi.c b/source/dnode/vnode/src/vnd/vnodeInitApi.c index 526b9b4e2d..45dec6ebee 100644 --- a/source/dnode/vnode/src/vnd/vnodeInitApi.c +++ b/source/dnode/vnode/src/vnd/vnodeInitApi.c @@ -204,7 +204,7 @@ void initStateStoreAPI(SStateStore* pStore) { } void initMetaReaderAPI(SStoreMetaReader* pMetaReader) { - pMetaReader->initReader = _metaReaderInit; + pMetaReader->initReader = metaReaderInit; pMetaReader->clearReader = metaReaderClear; pMetaReader->getTableEntryByUid = metaReaderGetTableEntryByUid; diff --git a/source/dnode/vnode/src/vnd/vnodeQuery.c b/source/dnode/vnode/src/vnd/vnodeQuery.c index 022fc4c951..c122a98a12 100644 --- a/source/dnode/vnode/src/vnd/vnodeQuery.c +++ b/source/dnode/vnode/src/vnd/vnodeQuery.c @@ -62,7 +62,7 @@ int vnodeGetTableMeta(SVnode *pVnode, SRpcMsg *pMsg, bool direct) { } // query meta - metaReaderInit(&mer1, pVnode->pMeta, 0); + metaReaderDoInit(&mer1, pVnode->pMeta, 0); if (metaGetTableEntryByName(&mer1, infoReq.tbName) < 0) { code = terrno; @@ -79,7 +79,7 @@ int vnodeGetTableMeta(SVnode *pVnode, SRpcMsg *pMsg, bool direct) { schemaTag = mer1.me.stbEntry.schemaTag; metaRsp.suid = mer1.me.uid; } else if (mer1.me.type == TSDB_CHILD_TABLE) { - metaReaderInit(&mer2, pVnode->pMeta, META_READER_NOLOCK); + metaReaderDoInit(&mer2, pVnode->pMeta, META_READER_NOLOCK); if (metaReaderGetTableEntryByUid(&mer2, mer1.me.ctbEntry.suid) < 0) goto _exit; strcpy(metaRsp.stbName, mer2.me.name); @@ -175,7 +175,7 @@ int vnodeGetTableCfg(SVnode *pVnode, SRpcMsg *pMsg, bool direct) { } // query meta - metaReaderInit(&mer1, pVnode->pMeta, 0); + metaReaderDoInit(&mer1, pVnode->pMeta, 0); if (metaGetTableEntryByName(&mer1, cfgReq.tbName) < 0) { code = terrno; @@ -188,7 +188,7 @@ int vnodeGetTableCfg(SVnode *pVnode, SRpcMsg *pMsg, bool direct) { code = TSDB_CODE_VND_HASH_MISMATCH; goto _exit; } else if (mer1.me.type == TSDB_CHILD_TABLE) { - metaReaderInit(&mer2, pVnode->pMeta, 0); + metaReaderDoInit(&mer2, pVnode->pMeta, 0); if (metaReaderGetTableEntryByUid(&mer2, mer1.me.ctbEntry.suid) < 0) goto _exit; strcpy(cfgRsp.stbName, mer2.me.name); diff --git a/source/dnode/vnode/src/vnd/vnodeSvr.c b/source/dnode/vnode/src/vnd/vnodeSvr.c index 28a5becfd4..6d68e94989 100644 --- a/source/dnode/vnode/src/vnd/vnodeSvr.c +++ b/source/dnode/vnode/src/vnd/vnodeSvr.c @@ -564,24 +564,6 @@ int32_t vnodeProcessFetchMsg(SVnode *pVnode, SRpcMsg *pMsg, SQueueInfo *pInfo) { return tqProcessPollReq(pVnode->pTq, pMsg); case TDMT_VND_TMQ_VG_WALINFO: return tqProcessVgWalInfoReq(pVnode->pTq, pMsg); - case TDMT_STREAM_TASK_RUN: - return tqProcessTaskRunReq(pVnode->pTq, pMsg); - case TDMT_STREAM_TASK_DISPATCH: - return tqProcessTaskDispatchReq(pVnode->pTq, pMsg, true); - case TDMT_STREAM_TASK_CHECK: - return tqProcessStreamTaskCheckReq(pVnode->pTq, pMsg); - case TDMT_STREAM_TASK_DISPATCH_RSP: - return tqProcessTaskDispatchRsp(pVnode->pTq, pMsg); - case TDMT_STREAM_RETRIEVE: - return tqProcessTaskRetrieveReq(pVnode->pTq, pMsg); - case TDMT_STREAM_RETRIEVE_RSP: - return tqProcessTaskRetrieveRsp(pVnode->pTq, pMsg); - case TDMT_VND_STREAM_RECOVER_NONBLOCKING_STAGE: - return tqProcessTaskRecover1Req(pVnode->pTq, pMsg); - case TDMT_STREAM_RECOVER_FINISH: - return tqProcessTaskRecoverFinishReq(pVnode->pTq, pMsg); - case TDMT_STREAM_RECOVER_FINISH_RSP: - return tqProcessTaskRecoverFinishRsp(pVnode->pTq, pMsg); default: vError("unknown msg type:%d in fetch queue", pMsg->msgType); return TSDB_CODE_APP_ERROR; @@ -1651,7 +1633,7 @@ static int32_t vnodeProcessBatchDeleteReq(SVnode *pVnode, int64_t ver, void *pRe tDecodeSBatchDeleteReq(&decoder, &deleteReq); SMetaReader mr = {0}; - metaReaderInit(&mr, pVnode->pMeta, META_READER_NOLOCK); + metaReaderDoInit(&mr, pVnode->pMeta, META_READER_NOLOCK); int32_t sz = taosArrayGetSize(deleteReq.deleteReqs); for (int32_t i = 0; i < sz; i++) { diff --git a/source/libs/executor/inc/querytask.h b/source/libs/executor/inc/querytask.h index 2b9256f08e..6c2a16bb0c 100644 --- a/source/libs/executor/inc/querytask.h +++ b/source/libs/executor/inc/querytask.h @@ -64,8 +64,10 @@ typedef struct { int8_t recoverStep; int8_t recoverScanFinished; SQueryTableDataCond tableCond; - int64_t fillHistoryVer1; - int64_t fillHisotryeKey1; + SVersionRange fillHistoryVer; + STimeWindow fillHistoryWindow; +// int64_t fillHistoryVer1; +// int64_t fillHisotryeKey1; int64_t fillHistoryVer2; SStreamState* pState; int64_t dataVersion; diff --git a/source/libs/executor/src/executor.c b/source/libs/executor/src/executor.c index 0ea6ae7144..d77323d737 100644 --- a/source/libs/executor/src/executor.c +++ b/source/libs/executor/src/executor.c @@ -869,11 +869,11 @@ int32_t qExtractStreamScanner(qTaskInfo_t tinfo, void** scanner) { } } -int32_t qStreamSourceRecoverStep1(qTaskInfo_t tinfo, int64_t ver, int64_t ekey) { +int32_t qStreamSourceRecoverStep1(qTaskInfo_t tinfo, SVersionRange *pVerRange, STimeWindow* pWindow) { SExecTaskInfo* pTaskInfo = (SExecTaskInfo*)tinfo; ASSERT(pTaskInfo->execModel == OPTR_EXEC_MODEL_STREAM); - pTaskInfo->streamInfo.fillHistoryVer1 = ver; - pTaskInfo->streamInfo.fillHisotryeKey1 = ekey; + pTaskInfo->streamInfo.fillHistoryVer = *pVerRange; + pTaskInfo->streamInfo.fillHistoryWindow = *pWindow; pTaskInfo->streamInfo.recoverStep = STREAM_RECOVER_STEP__PREPARE1; return 0; } diff --git a/source/libs/executor/src/scanoperator.c b/source/libs/executor/src/scanoperator.c index 4b53a9918d..c892617783 100644 --- a/source/libs/executor/src/scanoperator.c +++ b/source/libs/executor/src/scanoperator.c @@ -1785,14 +1785,17 @@ static SSDataBlock* doStreamScan(SOperatorInfo* pOperator) { pTaskInfo->streamInfo.recoverStep == STREAM_RECOVER_STEP__PREPARE2) { STableScanInfo* pTSInfo = pInfo->pTableScanOp->info; memcpy(&pTSInfo->base.cond, &pTaskInfo->streamInfo.tableCond, sizeof(SQueryTableDataCond)); + if (pTaskInfo->streamInfo.recoverStep == STREAM_RECOVER_STEP__PREPARE1) { - pTSInfo->base.cond.startVersion = 0; - pTSInfo->base.cond.endVersion = pTaskInfo->streamInfo.fillHistoryVer1; - qDebug("stream recover step1, verRange:%" PRId64 " - %" PRId64 ", %s", pTSInfo->base.cond.startVersion, - pTSInfo->base.cond.endVersion, id); + pTSInfo->base.cond.startVersion = pTaskInfo->streamInfo.fillHistoryVer.minVer; + pTSInfo->base.cond.endVersion = pTaskInfo->streamInfo.fillHistoryVer.maxVer; + + pTSInfo->base.cond.twindows = pTaskInfo->streamInfo.fillHistoryWindow; + qDebug("stream recover step1, verRange:%" PRId64 "-%" PRId64 " window:%"PRId64"-%"PRId64", %s", pTSInfo->base.cond.startVersion, + pTSInfo->base.cond.endVersion, pTSInfo->base.cond.twindows.skey, pTSInfo->base.cond.twindows.ekey, id); pTaskInfo->streamInfo.recoverStep = STREAM_RECOVER_STEP__SCAN1; } else { - pTSInfo->base.cond.startVersion = pTaskInfo->streamInfo.fillHistoryVer1 + 1; + pTSInfo->base.cond.startVersion = pTaskInfo->streamInfo.fillHistoryVer.minVer + 1; pTSInfo->base.cond.endVersion = pTaskInfo->streamInfo.fillHistoryVer2; qDebug("stream recover step2, verRange:%" PRId64 " - %" PRId64", %s", pTSInfo->base.cond.startVersion, pTSInfo->base.cond.endVersion, id); @@ -2085,8 +2088,11 @@ FETCH_NEXT_BLOCK: return pInfo->pCreateTbRes; } + // todo apply time window range filter + doCheckUpdate(pInfo, pBlockInfo->window.ekey, pBlock); doFilter(pBlock, pOperator->exprSupp.pFilterInfo, NULL); + pBlock->info.dataLoad = 1; blockDataUpdateTsWindow(pBlock, pInfo->primaryTsIndex); diff --git a/source/libs/stream/inc/streamInc.h b/source/libs/stream/inc/streamInc.h index 4fc4d3ccf3..e12a0fdd43 100644 --- a/source/libs/stream/inc/streamInc.h +++ b/source/libs/stream/inc/streamInc.h @@ -48,7 +48,7 @@ int32_t tEncodeStreamRetrieveReq(SEncoder* pEncoder, const SStreamRetrieveReq* p int32_t streamDispatchCheckMsg(SStreamTask* pTask, const SStreamTaskCheckReq* pReq, int32_t nodeId, SEpSet* pEpSet); -int32_t streamDispatchOneRecoverFinishReq(SStreamTask* pTask, const SStreamRecoverFinishReq* pReq, int32_t vgId, +int32_t streamDoDispatchRecoverFinishMsg(SStreamTask* pTask, const SStreamRecoverFinishReq* pReq, int32_t vgId, SEpSet* pEpSet); SStreamQueueItem* streamMergeQueueItem(SStreamQueueItem* dst, SStreamQueueItem* pElem); diff --git a/source/libs/stream/src/streamDispatch.c b/source/libs/stream/src/streamDispatch.c index 9cb0a56644..26dd19ce7e 100644 --- a/source/libs/stream/src/streamDispatch.c +++ b/source/libs/stream/src/streamDispatch.c @@ -164,7 +164,7 @@ int32_t streamBroadcastToChildren(SStreamTask* pTask, const SSDataBlock* pBlock) } buf = NULL; - qDebug("s-task:%s (child %d) send retrieve req to task %d at node %d, reqId:0x%" PRIx64, pTask->id.idStr, + qDebug("s-task:%s (child %d) send retrieve req to task:0x%x (vgId:%d), reqId:0x%" PRIx64, pTask->id.idStr, pTask->selfChildId, pEpInfo->taskId, pEpInfo->nodeId, req.reqId); } code = 0; @@ -238,14 +238,14 @@ int32_t streamDispatchCheckMsg(SStreamTask* pTask, const SStreamTaskCheckReq* pR msg.pCont = buf; msg.msgType = TDMT_STREAM_TASK_CHECK; - qDebug("s-task:%s dispatch check msg to downstream s-task:%" PRIx64 ":%d node %d: check msg", pTask->id.idStr, + qDebug("s-task:%s dispatch check msg to s-task:%" PRIx64 ":0x%x (vgId:%d)", pTask->id.idStr, pReq->streamId, pReq->downstreamTaskId, nodeId); tmsgSendReq(pEpSet, &msg); return 0; } -int32_t streamDispatchOneRecoverFinishReq(SStreamTask* pTask, const SStreamRecoverFinishReq* pReq, int32_t vgId, +int32_t streamDoDispatchRecoverFinishMsg(SStreamTask* pTask, const SStreamRecoverFinishReq* pReq, int32_t vgId, SEpSet* pEpSet) { void* buf = NULL; int32_t code = -1; @@ -283,8 +283,7 @@ int32_t streamDispatchOneRecoverFinishReq(SStreamTask* pTask, const SStreamRecov msg.info.noResp = 1; tmsgSendReq(pEpSet, &msg); - qDebug("s-task:%s dispatch recover finish msg to downstream taskId:0x%x node %d: recover finish msg", pTask->id.idStr, - pReq->taskId, vgId); + qDebug("s-task:%s dispatch recover finish msg to taskId:0x%x (vgId:%d)", pTask->id.idStr, pReq->taskId, vgId); return 0; } diff --git a/source/libs/stream/src/streamRecover.c b/source/libs/stream/src/streamRecover.c index 5d366aca05..0bcb078d51 100644 --- a/source/libs/stream/src/streamRecover.c +++ b/source/libs/stream/src/streamRecover.c @@ -15,16 +15,30 @@ #include "streamInc.h" #include "ttimer.h" +#include "wal.h" +const char* streamGetTaskStatusStr(int32_t status) { + switch(status) { + case TASK_STATUS__NORMAL: return "normal"; + case TASK_STATUS__WAIT_DOWNSTREAM: return "wait-for-downstream"; + case TASK_STATUS__RECOVER_PREPARE: return "scan-history-prepare"; + case TASK_STATUS__RECOVER1: return "scan-history-data"; + default:return ""; + } +} int32_t streamTaskLaunchRecover(SStreamTask* pTask) { - qDebug("s-task:%s at node %d launch recover", pTask->id.idStr, pTask->nodeId); + qDebug("s-task:%s (vgId:%d) launch recover", pTask->id.idStr, pTask->nodeId); if (pTask->taskLevel == TASK_LEVEL__SOURCE) { atomic_store_8(&pTask->status.taskStatus, TASK_STATUS__RECOVER_PREPARE); - qDebug("s-task:%s set task status:%d and start to recover", pTask->id.idStr, pTask->status.taskStatus); + + SVersionRange* pRange = &pTask->dataRange.range; + qDebug("s-task:%s set task status:%s and start to recover, ver:%" PRId64 "-%" PRId64, pTask->id.idStr, + streamGetTaskStatusStr(pTask->status.taskStatus), pTask->dataRange.range.minVer, + pTask->dataRange.range.maxVer); streamSetParamForRecover(pTask); - streamSourceRecoverPrepareStep1(pTask, pTask->dataRange.range.maxVer, pTask->dataRange.window.ekey); + streamSourceRecoverPrepareStep1(pTask, pRange, &pTask->dataRange.window); SStreamRecoverStep1Req req; streamBuildSourceRecover1Req(pTask, &req); @@ -43,12 +57,12 @@ int32_t streamTaskLaunchRecover(SStreamTask* pTask) { } } else if (pTask->taskLevel == TASK_LEVEL__AGG) { - atomic_store_8(&pTask->status.taskStatus, TASK_STATUS__NORMAL); + streamSetStatusNormal(pTask); streamSetParamForRecover(pTask); streamAggRecoverPrepare(pTask); } else if (pTask->taskLevel == TASK_LEVEL__SINK) { - // sink nodes has no specified operation for fill history - atomic_store_8(&pTask->status.taskStatus, TASK_STATUS__NORMAL); + streamSetStatusNormal(pTask); + qDebug("s-task:%s sink task convert to normal immediately", pTask->id.idStr); } return 0; @@ -56,8 +70,9 @@ int32_t streamTaskLaunchRecover(SStreamTask* pTask) { // check status int32_t streamTaskCheckDownstreamTasks(SStreamTask* pTask) { - qDebug("s-task:%s in fill history stage, ver:%"PRId64" ekey:%"PRId64, pTask->id.idStr, pTask->dataRange.range.maxVer, - pTask->dataRange.window.ekey); + qDebug("s-task:%s in fill history stage, ver:%" PRId64 "-%"PRId64" window:%" PRId64"-%"PRId64, pTask->id.idStr, + pTask->dataRange.range.minVer, pTask->dataRange.range.maxVer, pTask->dataRange.window.skey, + pTask->dataRange.window.ekey); SStreamTaskCheckReq req = { .streamId = pTask->id.streamId, @@ -74,7 +89,7 @@ int32_t streamTaskCheckDownstreamTasks(SStreamTask* pTask) { req.downstreamTaskId = pTask->fixedEpDispatcher.taskId; pTask->checkReqId = req.reqId; - qDebug("s-task:%s at node %d check downstream task:0x%x at node %d", pTask->id.idStr, pTask->nodeId, req.downstreamTaskId, + qDebug("s-task:%s (vgId:%d) check downstream task:0x%x (vgId:%d)", pTask->id.idStr, pTask->nodeId, req.downstreamTaskId, req.downstreamNodeId); streamDispatchCheckMsg(pTask, &req, pTask->fixedEpDispatcher.nodeId, &pTask->fixedEpDispatcher.epSet); } else if (pTask->outputType == TASK_OUTPUT__SHUFFLE_DISPATCH) { @@ -90,7 +105,7 @@ int32_t streamTaskCheckDownstreamTasks(SStreamTask* pTask) { taosArrayPush(pTask->checkReqIds, &req.reqId); req.downstreamNodeId = pVgInfo->vgId; req.downstreamTaskId = pVgInfo->taskId; - qDebug("s-task:%s (vgId:%d) check downstream task:0x%x at node %d (shuffle)", pTask->id.idStr, pTask->nodeId, + qDebug("s-task:%s (vgId:%d) check downstream task:0x%x (vgId:%d) (shuffle)", pTask->id.idStr, pTask->nodeId, req.downstreamTaskId, req.downstreamNodeId); streamDispatchCheckMsg(pTask, &req, pVgInfo->vgId, &pVgInfo->epSet); } @@ -113,7 +128,7 @@ int32_t streamRecheckOneDownstream(SStreamTask* pTask, const SStreamTaskCheckRsp .childId = pRsp->childId, }; - qDebug("s-task:%s at node %d check downstream task:0x%x at node %d (recheck)", pTask->id.idStr, pTask->nodeId, + qDebug("s-task:%s (vgId:%d) check downstream task:0x%x (vgId:%d) (recheck)", pTask->id.idStr, pTask->nodeId, req.downstreamTaskId, req.downstreamNodeId); if (pTask->outputType == TASK_OUTPUT__FIXED_DISPATCH) { @@ -139,9 +154,7 @@ int32_t streamTaskCheckStatus(SStreamTask* pTask) { int32_t streamProcessCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* pRsp) { ASSERT(pTask->id.taskId == pRsp->upstreamTaskId); - - qDebug("s-task:%s at node %d recv check rsp from task:0x%x at node %d: status %d", pTask->id.idStr, - pRsp->upstreamNodeId, pRsp->downstreamTaskId, pRsp->downstreamNodeId, pRsp->status); + const char* id = pTask->id.idStr; if (pRsp->status == 1) { if (pTask->outputType == TASK_OUTPUT__SHUFFLE_DISPATCH) { @@ -167,21 +180,25 @@ int32_t streamProcessCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* pRs taosArrayDestroy(pTask->checkReqIds); pTask->checkReqIds = NULL; - qDebug("s-task:%s all %d downstream tasks are ready, now enter into recover stage", pTask->id.idStr, numOfReqs); + qDebug("s-task:%s all %d downstream tasks are ready, now enter into recover stage", id, numOfReqs); streamTaskLaunchRecover(pTask); + } else { + qDebug("s-task:%s (vgId:%d) recv check rsp from task:0x%x (vgId:%d) status:%d, remain not ready:%d", id, + pRsp->upstreamNodeId, pRsp->downstreamTaskId, pRsp->downstreamNodeId, pRsp->status, left); } } else if (pTask->outputType == TASK_OUTPUT__FIXED_DISPATCH) { if (pRsp->reqId != pTask->checkReqId) { return -1; } + qDebug("s-task:%s fixed downstream tasks is ready, now enter into recover stage", id); streamTaskLaunchRecover(pTask); } else { ASSERT(0); } - } else { // not ready, wait for 100ms and retry - qDebug("s-task:%s downstream taskId:0x%x (vgId:%d) not ready, wait for 100ms and retry", pTask->id.idStr, - pRsp->downstreamTaskId, pRsp->downstreamNodeId); + } else { // not ready, wait for 100ms and retry + qDebug("s-task:%s downstream taskId:0x%x (vgId:%d) not ready, wait for 100ms and retry", id, pRsp->downstreamTaskId, + pRsp->downstreamNodeId); taosMsleep(100); streamRecheckOneDownstream(pTask, pRsp); @@ -206,9 +223,9 @@ int32_t streamSetStatusNormal(SStreamTask* pTask) { } // source -int32_t streamSourceRecoverPrepareStep1(SStreamTask* pTask, int64_t ver, int64_t ekey) { +int32_t streamSourceRecoverPrepareStep1(SStreamTask* pTask, SVersionRange *pVerRange, STimeWindow* pWindow) { void* exec = pTask->exec.pExecutor; - return qStreamSourceRecoverStep1(exec, ver, ekey); + return qStreamSourceRecoverStep1(exec, pVerRange, pWindow); } int32_t streamBuildSourceRecover1Req(SStreamTask* pTask, SStreamRecoverStep1Req* pReq) { @@ -246,23 +263,23 @@ int32_t streamSourceRecoverScanStep2(SStreamTask* pTask, int64_t ver) { return code; } -int32_t streamDispatchRecoverFinishReq(SStreamTask* pTask) { +int32_t streamDispatchRecoverFinishMsg(SStreamTask* pTask) { SStreamRecoverFinishReq req = { .streamId = pTask->id.streamId, .childId = pTask->selfChildId }; // serialize if (pTask->outputType == TASK_OUTPUT__FIXED_DISPATCH) { - qDebug("s-task:%s send recover finish msg to downstream (fix-dispatch) to taskId:%d, status:%d", pTask->id.idStr, + qDebug("s-task:%s send recover finish msg to downstream (fix-dispatch) to taskId:0x%x, status:%d", pTask->id.idStr, pTask->fixedEpDispatcher.taskId, pTask->status.taskStatus); req.taskId = pTask->fixedEpDispatcher.taskId; - streamDispatchOneRecoverFinishReq(pTask, &req, pTask->fixedEpDispatcher.nodeId, &pTask->fixedEpDispatcher.epSet); + streamDoDispatchRecoverFinishMsg(pTask, &req, pTask->fixedEpDispatcher.nodeId, &pTask->fixedEpDispatcher.epSet); } else if (pTask->outputType == TASK_OUTPUT__SHUFFLE_DISPATCH) { SArray* vgInfo = pTask->shuffleDispatcher.dbInfo.pVgroupInfos; int32_t vgSz = taosArrayGetSize(vgInfo); for (int32_t i = 0; i < vgSz; i++) { SVgroupInfo* pVgInfo = taosArrayGet(vgInfo, i); req.taskId = pVgInfo->taskId; - streamDispatchOneRecoverFinishReq(pTask, &req, pVgInfo->vgId, &pVgInfo->epSet); + streamDoDispatchRecoverFinishMsg(pTask, &req, pVgInfo->vgId, &pVgInfo->epSet); } } return 0; @@ -271,7 +288,8 @@ int32_t streamDispatchRecoverFinishReq(SStreamTask* pTask) { // agg int32_t streamAggRecoverPrepare(SStreamTask* pTask) { pTask->recoverWaitingUpstream = taosArrayGetSize(pTask->childEpInfo); - qDebug("s-task:%s wait for %d upstreams", pTask->id.idStr, pTask->recoverWaitingUpstream); + qDebug("s-task:%s agg task is ready and wait for %d upstream tasks complete fill history procedure", pTask->id.idStr, + pTask->recoverWaitingUpstream); return 0; } @@ -303,8 +321,8 @@ static void doCheckDownstreamStatus(SStreamTask* pTask, SStreamTask* pHTask) { pHTask->dataRange.range.minVer = 0; pHTask->dataRange.range.maxVer = pTask->chkInfo.currentVer; - qDebug("s-task:%s set the launch condition for fill history task:%s, window:%" PRId64 " - %" PRId64 - " verrange:%" PRId64 " - %" PRId64, + qDebug("s-task:%s set the launch condition for fill history s-task:%s, window:%" PRId64 "-%" PRId64 + " verrange:%" PRId64 "-%" PRId64, pTask->id.idStr, pHTask->id.idStr, pHTask->dataRange.window.skey, pHTask->dataRange.window.ekey, pHTask->dataRange.range.minVer, pHTask->dataRange.range.maxVer); @@ -321,7 +339,7 @@ static void tryLaunchHistoryTask(void* param, void* tmrId) { qWarn("s-task:%s vgId:%d failed to launch history task:0x%x, since it is not built yet", pTask->id.idStr, pMeta->vgId, pTask->historyTaskId.taskId); - taosTmrReset(tryLaunchHistoryTask, (int32_t)pTask->triggerParam, pTask, streamEnv.timer, &pTask->timer); + taosTmrReset(tryLaunchHistoryTask, 100, pTask, streamEnv.timer, &pTask->timer); return; } @@ -357,6 +375,42 @@ int32_t streamTaskStartHistoryTask(SStreamTask* pTask, int64_t ver) { return TSDB_CODE_SUCCESS; } +int32_t streamTaskScanHistoryDataComplete(SStreamTask* pTask) { + SStreamMeta* pMeta = pTask->pMeta; + + qDebug("s-task:%s set start wal scan start ver:%" PRId64, pTask->id.idStr, pTask->chkInfo.currentVer); + ASSERT(walReaderGetCurrentVer(pTask->exec.pWalReader) == -1); + +// walReaderSeekVer(pTask->exec.pWalReader, sversion); +// pTask->chkInfo.currentVer = sversion; + + if (atomic_load_8(&pTask->status.taskStatus) == TASK_STATUS__DROPPING) { + return 0; + } + + // restore param + int32_t code = streamRestoreParam(pTask); + if (code < 0) { + return -1; + } + + // dispatch recover finish req to all related downstream task + code = streamDispatchRecoverFinishMsg(pTask); + if (code < 0) { + return -1; + } + + // set status normal + qDebug("s-task:%s set the status to be normal, and start wal scan", pTask->id.idStr); + code = streamSetStatusNormal(pTask); + if (code < 0) { + return -1; + } + + streamMetaSaveTask(pMeta, pTask); + return 0; +} + int32_t tEncodeSStreamTaskCheckReq(SEncoder* pEncoder, const SStreamTaskCheckReq* pReq) { if (tStartEncode(pEncoder) < 0) return -1; if (tEncodeI64(pEncoder, pReq->reqId) < 0) return -1; diff --git a/source/libs/stream/src/streamTask.c b/source/libs/stream/src/streamTask.c index 4f883b76e4..de10f021d2 100644 --- a/source/libs/stream/src/streamTask.c +++ b/source/libs/stream/src/streamTask.c @@ -17,7 +17,7 @@ #include "tstream.h" #include "wal.h" -static int32_t mndAddToTaskset(SArray* pArray, SStreamTask* pTask) { +static int32_t addToTaskset(SArray* pArray, SStreamTask* pTask) { int32_t childId = taosArrayGetSize(pArray); pTask->selfChildId = childId; taosArrayPush(pArray, &pTask); @@ -45,110 +45,10 @@ SStreamTask* tNewStreamTask(int64_t streamId, int8_t taskLevel, int8_t fillHisto pTask->inputStatus = TASK_INPUT_STATUS__NORMAL; pTask->outputStatus = TASK_OUTPUT_STATUS__NORMAL; - mndAddToTaskset(pTaskList, pTask); + addToTaskset(pTaskList, pTask); return pTask; } -SStreamTask* streamTaskClone(SStreamTask* pTask) { - SStreamTask* pDst = taosMemoryCalloc(1, sizeof(SStreamTask)); - /* pDst-> - - SStreamId id; - int32_t totalLevel; - int8_t taskLevel; - int8_t outputType; - int16_t dispatchMsgType; - SStreamStatus status; - int32_t selfChildId; - int32_t nodeId; // vgroup id - SEpSet epSet; - SCheckpointInfo chkInfo; - STaskExec exec; - int8_t fillHistory; // fill history - int64_t ekey; // end ts key - int64_t endVer; // end version - - // children info - SArray* childEpInfo; // SArray - int32_t nextCheckId; - SArray* checkpointInfo; // SArray - - // output - union { - STaskDispatcherFixedEp fixedEpDispatcher; - STaskDispatcherShuffle shuffleDispatcher; - STaskSinkTb tbSink; - STaskSinkSma smaSink; - STaskSinkFetch fetchSink; - }; - - int8_t inputStatus; - int8_t outputStatus; - SStreamQueue* inputQueue; - SStreamQueue* outputQueue; - - // trigger - int8_t triggerStatus; - int64_t triggerParam; - void* timer; - SMsgCb* pMsgCb; // msg handle - SStreamState* pState; // state backend - - // the followings attributes don't be serialized - int32_t recoverTryingDownstream; - int32_t recoverWaitingUpstream; - int64_t checkReqId; - SArray* checkReqIds; // shuffle - int32_t refCnt; - int64_t checkpointingId; - int32_t checkpointAlignCnt; - struct SStreamMeta* pMeta; - - int32_t status = atomic_load_8((int8_t*)&(pTask->status.taskStatus)); - if (pTask->inputQueue) { - streamQueueClose(pTask->inputQueue); - } - if (pTask->outputQueue) { - streamQueueClose(pTask->outputQueue); - } - if (pTask->exec.qmsg) { - taosMemoryFree(pTask->exec.qmsg); - } - - if (pTask->exec.pExecutor) { - qDestroyTask(pTask->exec.pExecutor); - pTask->exec.pExecutor = NULL; - } - - if (pTask->exec.pWalReader != NULL) { - walCloseReader(pTask->exec.pWalReader); - } - - taosArrayDestroyP(pTask->childEpInfo, taosMemoryFree); - if (pTask->outputType == TASK_OUTPUT__TABLE) { - tDeleteSchemaWrapper(pTask->tbSink.pSchemaWrapper); - taosMemoryFree(pTask->tbSink.pTSchema); - tSimpleHashCleanup(pTask->tbSink.pTblInfo); - } - - if (pTask->outputType == TASK_OUTPUT__SHUFFLE_DISPATCH) { - taosArrayDestroy(pTask->shuffleDispatcher.dbInfo.pVgroupInfos); - taosArrayDestroy(pTask->checkReqIds); - pTask->checkReqIds = NULL; - } - - if (pTask->pState) { - streamStateClose(pTask->pState, status == TASK_STATUS__DROPPING); - } - - if (pTask->id.idStr != NULL) { - taosMemoryFree((void*)pTask->id.idStr); - } - - taosMemoryFree(pTask);*/ - return NULL; -} - int32_t tEncodeStreamEpInfo(SEncoder* pEncoder, const SStreamChildEpInfo* pInfo) { if (tEncodeI32(pEncoder, pInfo->taskId) < 0) return -1; if (tEncodeI32(pEncoder, pInfo->nodeId) < 0) return -1; From 06cf3588da6b428332704fda6ae9991667b2ca72 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 7 Jun 2023 15:03:04 +0800 Subject: [PATCH 10/83] refactor: do some internal refactor. --- include/libs/stream/tstream.h | 51 +++++----- source/dnode/mnode/impl/src/mndScheduler.c | 40 ++++---- source/dnode/mnode/impl/src/mndStream.c | 44 ++++----- source/dnode/snode/src/snode.c | 8 +- source/dnode/vnode/src/tq/tq.c | 24 ++--- source/dnode/vnode/src/tq/tqRead.c | 2 +- source/dnode/vnode/src/tq/tqRestore.c | 4 +- source/libs/qworker/CMakeLists.txt | 12 +-- source/libs/stream/inc/streamInc.h | 2 - source/libs/stream/src/stream.c | 38 +++---- source/libs/stream/src/streamCheckpoint.c | 4 +- source/libs/stream/src/streamData.c | 4 +- source/libs/stream/src/streamDispatch.c | 110 +++++++++++---------- source/libs/stream/src/streamExec.c | 24 ++--- source/libs/stream/src/streamMeta.c | 2 +- source/libs/stream/src/streamRecover.c | 34 +++---- source/libs/stream/src/streamTask.c | 49 ++++----- 17 files changed, 227 insertions(+), 225 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 898db47e86..e67423ca6e 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -47,7 +47,6 @@ enum { TASK_STATUS__WAIT_DOWNSTREAM, TASK_STATUS__RECOVER_PREPARE, TASK_STATUS__RECOVER1, - TASK_STATUS__RECOVER2, TASK_STATUS__PAUSE, }; @@ -203,7 +202,7 @@ static FORCE_INLINE void streamQueueProcessFail(SStreamQueue* queue) { atomic_store_8(&queue->status, STREAM_QUEUE__FAILED); } -void* streamQueueNextItem(SStreamQueue* queue); +void* streamQueueNextItem(SStreamQueue* pQueue); SStreamDataSubmit* streamDataSubmitNew(SPackedData* pData, int32_t type); void streamDataSubmitDestroy(SStreamDataSubmit* pDataSubmit); @@ -251,7 +250,7 @@ typedef struct { int8_t reserved; } STaskSinkFetch; -typedef struct { +typedef struct SStreamChildEpInfo { int32_t nodeId; int32_t childId; int32_t taskId; @@ -276,32 +275,38 @@ typedef struct SStreamStatus { int8_t keepTaskStatus; } SStreamStatus; -typedef struct SHistoryDataRange { +typedef struct SHistDataRange { SVersionRange range; STimeWindow window; -} SHistoryDataRange; +} SHistDataRange; -struct SStreamTask { - SStreamId id; +typedef struct SSTaskBasicInfo { + int32_t nodeId; // vgroup id or snode id + SEpSet epSet; + int32_t selfChildId; int32_t totalLevel; int8_t taskLevel; - int8_t outputType; - int16_t dispatchMsgType; - SStreamStatus status; - int32_t selfChildId; - int32_t nodeId; // vgroup id - SEpSet epSet; - SCheckpointInfo chkInfo; - STaskExec exec; - int8_t fillHistory; // fill history + int8_t fillHistory; // is fill history task or not +} SSTaskBasicInfo; - SHistoryDataRange dataRange; - SStreamId historyTaskId; +typedef struct SDispatchMsgInfo { + void* pData; // current dispatch data + int16_t msgType; // dispatch msg type +} SDispatchMsgInfo; - // children info - SArray* childEpInfo; // SArray - int32_t nextCheckId; - SArray* checkpointInfo; // SArray +struct SStreamTask { + SStreamId id; + SSTaskBasicInfo info; + int8_t outputType; + SDispatchMsgInfo msgInfo; + SStreamStatus status; + SCheckpointInfo chkInfo; + STaskExec exec; + SHistDataRange dataRange; + SStreamId historyTaskId; + SArray* pUpstreamEpInfoList; // SArray, // children info + int32_t nextCheckId; + SArray* checkpointInfo; // SArray // output union { @@ -326,7 +331,7 @@ struct SStreamTask { // the followings attributes don't be serialized int32_t recoverTryingDownstream; - int32_t recoverWaitingUpstream; + int32_t numOfWaitingUpstream; int64_t checkReqId; SArray* checkReqIds; // shuffle int32_t refCnt; diff --git a/source/dnode/mnode/impl/src/mndScheduler.c b/source/dnode/mnode/impl/src/mndScheduler.c index a1c81a999f..c6d8bb5ffe 100644 --- a/source/dnode/mnode/impl/src/mndScheduler.c +++ b/source/dnode/mnode/impl/src/mndScheduler.c @@ -110,7 +110,7 @@ int32_t mndAddDispatcherForInternalTask(SMnode* pMnode, SStreamObj* pStream, SAr isShuffle = true; pTask->outputType = TASK_OUTPUT__SHUFFLE_DISPATCH; - pTask->dispatchMsgType = TDMT_STREAM_TASK_DISPATCH; + pTask->msgInfo.msgType = TDMT_STREAM_TASK_DISPATCH; if (mndExtractDbInfo(pMnode, pDb, &pTask->shuffleDispatcher.dbInfo, NULL) < 0) { return -1; } @@ -131,7 +131,7 @@ int32_t mndAddDispatcherForInternalTask(SMnode* pMnode, SStreamObj* pStream, SAr for (int32_t j = 0; j < numOfSinkNodes; j++) { SStreamTask* pSinkTask = taosArrayGetP(pSinkNodeList, j); - if (pSinkTask->nodeId == pVgInfo->vgId) { + if (pSinkTask->info.nodeId == pVgInfo->vgId) { pVgInfo->taskId = pSinkTask->id.taskId; break; } @@ -148,11 +148,11 @@ int32_t mndAddDispatcherForInternalTask(SMnode* pMnode, SStreamObj* pStream, SAr int32_t mndAssignStreamTaskToVgroup(SMnode* pMnode, SStreamTask* pTask, SSubplan* plan, const SVgObj* pVgroup) { int32_t msgLen; - pTask->nodeId = pVgroup->vgId; - pTask->epSet = mndGetVgroupEpset(pMnode, pVgroup); + pTask->info.nodeId = pVgroup->vgId; + pTask->info.epSet = mndGetVgroupEpset(pMnode, pVgroup); - plan->execNode.nodeId = pTask->nodeId; - plan->execNode.epSet = pTask->epSet; + plan->execNode.nodeId = pTask->info.nodeId; + plan->execNode.epSet = pTask->info.epSet; if (qSubPlanToString(plan, &pTask->exec.qmsg, &msgLen) < 0) { terrno = TSDB_CODE_QRY_INVALID_INPUT; return -1; @@ -172,11 +172,11 @@ SSnodeObj* mndSchedFetchOneSnode(SMnode* pMnode) { int32_t mndAssignStreamTaskToSnode(SMnode* pMnode, SStreamTask* pTask, SSubplan* plan, const SSnodeObj* pSnode) { int32_t msgLen; - pTask->nodeId = SNODE_HANDLE; - pTask->epSet = mndAcquireEpFromSnode(pMnode, pSnode); + pTask->info.nodeId = SNODE_HANDLE; + pTask->info.epSet = mndAcquireEpFromSnode(pMnode, pSnode); plan->execNode.nodeId = SNODE_HANDLE; - plan->execNode.epSet = pTask->epSet; + plan->execNode.epSet = pTask->info.epSet; if (qSubPlanToString(plan, &pTask->exec.qmsg, &msgLen) < 0) { terrno = TSDB_CODE_QRY_INVALID_INPUT; @@ -232,8 +232,8 @@ int32_t mndAddSinkTaskToStream(SStreamObj* pStream, SArray* pTaskList, SMnode* p return -1; } - pTask->nodeId = vgId; - pTask->epSet = mndGetVgroupEpset(pMnode, pVgroup); + pTask->info.nodeId = vgId; + pTask->info.epSet = mndGetVgroupEpset(pMnode, pVgroup); mndSetSinkTaskInfo(pStream, pTask); return 0; } @@ -273,9 +273,9 @@ static SStreamChildEpInfo* createStreamTaskEpInfo(SStreamTask* pTask) { return NULL; } - pEpInfo->childId = pTask->selfChildId; - pEpInfo->epSet = pTask->epSet; - pEpInfo->nodeId = pTask->nodeId; + pEpInfo->childId = pTask->info.selfChildId; + pEpInfo->epSet = pTask->info.epSet; + pEpInfo->nodeId = pTask->info.nodeId; pEpInfo->taskId = pTask->id.taskId; return pEpInfo; @@ -284,11 +284,11 @@ static SStreamChildEpInfo* createStreamTaskEpInfo(SStreamTask* pTask) { void setFixedDownstreamEpInfo(SStreamTask* pDstTask, const SStreamTask* pTask) { STaskDispatcherFixedEp* pDispatcher = &pDstTask->fixedEpDispatcher; pDispatcher->taskId = pTask->id.taskId; - pDispatcher->nodeId = pTask->nodeId; - pDispatcher->epSet = pTask->epSet; + pDispatcher->nodeId = pTask->info.nodeId; + pDispatcher->epSet = pTask->info.epSet; pDstTask->outputType = TASK_OUTPUT__FIXED_DISPATCH; - pDstTask->dispatchMsgType = TDMT_STREAM_TASK_DISPATCH; + pDstTask->msgInfo.msgType = TDMT_STREAM_TASK_DISPATCH; } int32_t setEpToDownstreamTask(SStreamTask* pTask, SStreamTask* pDownstream) { @@ -297,11 +297,11 @@ int32_t setEpToDownstreamTask(SStreamTask* pTask, SStreamTask* pDownstream) { return TSDB_CODE_OUT_OF_MEMORY; } - if(pDownstream->childEpInfo == NULL) { - pDownstream->childEpInfo = taosArrayInit(4, POINTER_BYTES); + if(pDownstream->pUpstreamEpInfoList == NULL) { + pDownstream->pUpstreamEpInfoList = taosArrayInit(4, POINTER_BYTES); } - taosArrayPush(pDownstream->childEpInfo, &pEpInfo); + taosArrayPush(pDownstream->pUpstreamEpInfoList, &pEpInfo); return TSDB_CODE_SUCCESS; } diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index d4c1e033dc..74138aab5e 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -444,7 +444,7 @@ int32_t mndPersistTaskDeployReq(STrans *pTrans, const SStreamTask *pTask) { return -1; } - ((SMsgHead *)buf)->vgId = htonl(pTask->nodeId); + ((SMsgHead *)buf)->vgId = htonl(pTask->info.nodeId); void *abuf = POINTER_SHIFT(buf, sizeof(SMsgHead)); tEncoderInit(&encoder, abuf, size); @@ -454,7 +454,7 @@ int32_t mndPersistTaskDeployReq(STrans *pTrans, const SStreamTask *pTask) { STransAction action = {0}; action.mTraceId = pTrans->mTraceId; - memcpy(&action.epSet, &pTask->epSet, sizeof(SEpSet)); + memcpy(&action.epSet, &pTask->info.epSet, sizeof(SEpSet)); action.pCont = buf; action.contLen = tlen; action.msgType = TDMT_STREAM_TASK_DEPLOY; @@ -637,17 +637,17 @@ _OVER: static int32_t mndPersistTaskDropReq(STrans *pTrans, SStreamTask *pTask) { // vnode - /*if (pTask->nodeId > 0) {*/ + /*if (pTask->info.nodeId > 0) {*/ SVDropStreamTaskReq *pReq = taosMemoryCalloc(1, sizeof(SVDropStreamTaskReq)); if (pReq == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; return -1; } - pReq->head.vgId = htonl(pTask->nodeId); + pReq->head.vgId = htonl(pTask->info.nodeId); pReq->taskId = pTask->id.taskId; STransAction action = {0}; - memcpy(&action.epSet, &pTask->epSet, sizeof(SEpSet)); + memcpy(&action.epSet, &pTask->info.epSet, sizeof(SEpSet)); action.pCont = pReq; action.contLen = sizeof(SVDropStreamTaskReq); action.msgType = TDMT_STREAM_TASK_DROP; @@ -870,7 +870,7 @@ static int32_t mndBuildStreamCheckpointSourceReq(void **pBuf, int32_t *pLen, con SMStreamDoCheckpointMsg *pMsg) { SStreamCheckpointSourceReq req = {0}; req.checkpointId = pMsg->checkpointId; - req.nodeId = pTask->nodeId; + req.nodeId = pTask->info.nodeId; req.expireTime = -1; req.streamId = pTask->streamId; req.taskId = pTask->taskId; @@ -899,7 +899,7 @@ static int32_t mndBuildStreamCheckpointSourceReq(void **pBuf, int32_t *pLen, con SMsgHead *pMsgHead = (SMsgHead *)buf; pMsgHead->contLen = htonl(tlen); - pMsgHead->vgId = htonl(pTask->nodeId); + pMsgHead->vgId = htonl(pTask->info.nodeId); tEncoderClear(&encoder); @@ -938,12 +938,12 @@ static int32_t mndProcessStreamDoCheckpoint(SRpcMsg *pReq) { for (int32_t i = 0; i < totLevel; i++) { SArray *pLevel = taosArrayGetP(pStream->tasks, i); SStreamTask *pTask = taosArrayGetP(pLevel, 0); - if (pTask->taskLevel == TASK_LEVEL__SOURCE) { + if (pTask->info.taskLevel == TASK_LEVEL__SOURCE) { int32_t sz = taosArrayGetSize(pLevel); for (int32_t j = 0; j < sz; j++) { SStreamTask *pTask = taosArrayGetP(pLevel, j); - /*A(pTask->nodeId > 0);*/ - SVgObj *pVgObj = mndAcquireVgroup(pMnode, pTask->nodeId); + /*A(pTask->info.nodeId > 0);*/ + SVgObj *pVgObj = mndAcquireVgroup(pMnode, pTask->info.nodeId); if (pVgObj == NULL) { taosRUnLockLatch(&pStream->lock); mndReleaseStream(pMnode, pStream); @@ -1262,7 +1262,7 @@ static int32_t mndRetrieveStreamTask(SRpcMsg *pReq, SShowObj *pShow, SSDataBlock char nodeType[20 + VARSTR_HEADER_SIZE] = {0}; varDataSetLen(nodeType, 5); pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); - if (pTask->nodeId > 0) { + if (pTask->info.nodeId > 0) { memcpy(varDataVal(nodeType), "vnode", 5); } else { memcpy(varDataVal(nodeType), "snode", 5); @@ -1271,21 +1271,21 @@ static int32_t mndRetrieveStreamTask(SRpcMsg *pReq, SShowObj *pShow, SSDataBlock // node id pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); - int64_t nodeId = TMAX(pTask->nodeId, 0); + int64_t nodeId = TMAX(pTask->info.nodeId, 0); colDataSetVal(pColInfo, numOfRows, (const char *)&nodeId, false); // level char level[20 + VARSTR_HEADER_SIZE] = {0}; - if (pTask->taskLevel == TASK_LEVEL__SOURCE) { + if (pTask->info.taskLevel == TASK_LEVEL__SOURCE) { memcpy(varDataVal(level), "source", 6); varDataSetLen(level, 6); - } else if (pTask->taskLevel == TASK_LEVEL__AGG) { + } else if (pTask->info.taskLevel == TASK_LEVEL__AGG) { memcpy(varDataVal(level), "agg", 3); varDataSetLen(level, 3); - } else if (pTask->taskLevel == TASK_LEVEL__SINK) { + } else if (pTask->info.taskLevel == TASK_LEVEL__SINK) { memcpy(varDataVal(level), "sink", 4); varDataSetLen(level, 4); - } else if (pTask->taskLevel == TASK_LEVEL__SINK) { + } else if (pTask->info.taskLevel == TASK_LEVEL__SINK) { } pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); colDataSetVal(pColInfo, numOfRows, (const char *)&level, false); @@ -1323,10 +1323,10 @@ static int32_t mndPauseStreamTask(STrans *pTrans, SStreamTask *pTask) { terrno = TSDB_CODE_OUT_OF_MEMORY; return -1; } - pReq->head.vgId = htonl(pTask->nodeId); + pReq->head.vgId = htonl(pTask->info.nodeId); pReq->taskId = pTask->id.taskId; STransAction action = {0}; - memcpy(&action.epSet, &pTask->epSet, sizeof(SEpSet)); + memcpy(&action.epSet, &pTask->info.epSet, sizeof(SEpSet)); action.pCont = pReq; action.contLen = sizeof(SVPauseStreamTaskReq); action.msgType = TDMT_STREAM_TASK_PAUSE; @@ -1344,7 +1344,7 @@ int32_t mndPauseAllStreamTasks(STrans *pTrans, SStreamObj *pStream) { int32_t sz = taosArrayGetSize(pTasks); for (int32_t j = 0; j < sz; j++) { SStreamTask *pTask = taosArrayGetP(pTasks, j); - if (pTask->taskLevel != TASK_LEVEL__SINK && mndPauseStreamTask(pTrans, pTask) < 0) { + if (pTask->info.taskLevel != TASK_LEVEL__SINK && mndPauseStreamTask(pTrans, pTask) < 0) { return -1; } } @@ -1446,11 +1446,11 @@ static int32_t mndResumeStreamTask(STrans *pTrans, SStreamTask *pTask, int8_t ig terrno = TSDB_CODE_OUT_OF_MEMORY; return -1; } - pReq->head.vgId = htonl(pTask->nodeId); + pReq->head.vgId = htonl(pTask->info.nodeId); pReq->taskId = pTask->id.taskId; pReq->igUntreated = igUntreated; STransAction action = {0}; - memcpy(&action.epSet, &pTask->epSet, sizeof(SEpSet)); + memcpy(&action.epSet, &pTask->info.epSet, sizeof(SEpSet)); action.pCont = pReq; action.contLen = sizeof(SVResumeStreamTaskReq); action.msgType = TDMT_STREAM_TASK_RESUME; @@ -1468,7 +1468,7 @@ int32_t mndResumeAllStreamTasks(STrans *pTrans, SStreamObj *pStream, int8_t igUn int32_t sz = taosArrayGetSize(pTasks); for (int32_t j = 0; j < sz; j++) { SStreamTask *pTask = taosArrayGetP(pTasks, j); - if (pTask->taskLevel != TASK_LEVEL__SINK && mndResumeStreamTask(pTrans, pTask, igUntreated) < 0) { + if (pTask->info.taskLevel != TASK_LEVEL__SINK && mndResumeStreamTask(pTrans, pTask, igUntreated) < 0) { return -1; } } diff --git a/source/dnode/snode/src/snode.c b/source/dnode/snode/src/snode.c index 678dd34e4a..cf1481a113 100644 --- a/source/dnode/snode/src/snode.c +++ b/source/dnode/snode/src/snode.c @@ -62,7 +62,7 @@ FAIL: } int32_t sndExpandTask(SSnode *pSnode, SStreamTask *pTask, int64_t ver) { - ASSERT(pTask->taskLevel == TASK_LEVEL__AGG && taosArrayGetSize(pTask->childEpInfo) != 0); + ASSERT(pTask->info.taskLevel == TASK_LEVEL__AGG && taosArrayGetSize(pTask->pUpstreamEpInfoList) != 0); pTask->refCnt = 1; pTask->status.schedStatus = TASK_SCHED_STATUS__INACTIVE; @@ -85,7 +85,7 @@ int32_t sndExpandTask(SSnode *pSnode, SStreamTask *pTask, int64_t ver) { return -1; } - int32_t numOfChildEp = taosArrayGetSize(pTask->childEpInfo); + int32_t numOfChildEp = taosArrayGetSize(pTask->pUpstreamEpInfoList); SReadHandle handle = { .vnode = NULL, .numOfVgroups = numOfChildEp, .pStateBackend = pTask->pState }; initStreamStateAPI(&handle.api); @@ -151,7 +151,7 @@ int32_t sndProcessTaskDeployReq(SSnode *pSnode, char *msg, int32_t msgLen) { } tDecoderClear(&decoder); - ASSERT(pTask->taskLevel == TASK_LEVEL__AGG); + ASSERT(pTask->info.taskLevel == TASK_LEVEL__AGG); // 2.save task taosWLockLatch(&pSnode->pMeta->lock); @@ -164,7 +164,7 @@ int32_t sndProcessTaskDeployReq(SSnode *pSnode, char *msg, int32_t msgLen) { taosWUnLockLatch(&pSnode->pMeta->lock); // 3.go through recover steps to fill history - if (pTask->fillHistory) { + if (pTask->info.fillHistory) { streamSetParamForRecover(pTask); streamAggRecoverPrepare(pTask); } diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 84e37649ac..da05e950ce 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -820,9 +820,9 @@ int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t ver) { pTask->dataRange.range.minVer = ver; // expand executor - pTask->status.taskStatus = /*(pTask->fillHistory) ? */TASK_STATUS__WAIT_DOWNSTREAM /*: TASK_STATUS__NORMAL*/; + pTask->status.taskStatus = /*(pTask->info.fillHistory) ? */TASK_STATUS__WAIT_DOWNSTREAM /*: TASK_STATUS__NORMAL*/; - if (pTask->taskLevel == TASK_LEVEL__SOURCE) { + if (pTask->info.taskLevel == TASK_LEVEL__SOURCE) { pTask->pState = streamStateOpen(pTq->pStreamMeta->path, pTask, false, -1, -1); if (pTask->pState == NULL) { return -1; @@ -837,13 +837,13 @@ int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t ver) { } qSetTaskId(pTask->exec.pExecutor, pTask->id.taskId, pTask->id.streamId); - } else if (pTask->taskLevel == TASK_LEVEL__AGG) { + } else if (pTask->info.taskLevel == TASK_LEVEL__AGG) { pTask->pState = streamStateOpen(pTq->pStreamMeta->path, pTask, false, -1, -1); if (pTask->pState == NULL) { return -1; } - int32_t numOfVgroups = (int32_t)taosArrayGetSize(pTask->childEpInfo); + int32_t numOfVgroups = (int32_t)taosArrayGetSize(pTask->pUpstreamEpInfoList); SReadHandle handle = {.vnode = NULL, .numOfVgroups = numOfVgroups, .pStateBackend = pTask->pState}; initStorageAPI(&handle.api); @@ -879,7 +879,7 @@ int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t ver) { tSimpleHashSetFreeFp(pTask->tbSink.pTblInfo, freePtr); } - if (pTask->taskLevel == TASK_LEVEL__SOURCE) { + if (pTask->info.taskLevel == TASK_LEVEL__SOURCE) { SWalFilterCond cond = {.deleteMsg = 1}; // delete msg also extract from wal files pTask->exec.pWalReader = walOpenReader(pTq->pVnode->pWal, &cond); } @@ -887,7 +887,7 @@ int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t ver) { streamSetupTrigger(pTask); tqInfo("vgId:%d expand stream task, s-task:%s, checkpoint ver:%" PRId64 " child id:%d, level:%d", vgId, - pTask->id.idStr, pTask->chkInfo.version, pTask->selfChildId, pTask->taskLevel); + pTask->id.idStr, pTask->chkInfo.version, pTask->info.selfChildId, pTask->info.taskLevel); // next valid version will add one pTask->chkInfo.version += 1; @@ -1028,7 +1028,7 @@ int32_t tqProcessTaskDeployReq(STQ* pTq, int64_t sversion, char* msg, int32_t ms taosWUnLockLatch(&pStreamMeta->lock); // 3. It's an fill history task, do nothing. wait for the main task to start it - if (pTask->fillHistory) { + if (pTask->info.fillHistory) { tqDebug("s-task:%s fill history task, wait for being launched", pTask->id.idStr); } else { // calculate the correct start time window, and start the handle the history data for the main task. @@ -1037,7 +1037,7 @@ int32_t tqProcessTaskDeployReq(STQ* pTq, int64_t sversion, char* msg, int32_t ms streamTaskStartHistoryTask(pTask, sversion); // launch current task - SHistoryDataRange* pRange = &pTask->dataRange; + SHistDataRange* pRange = &pTask->dataRange; int64_t ekey = pRange->window.ekey; int64_t ver = pRange->range.minVer; @@ -1093,7 +1093,7 @@ int32_t tqProcessTaskRecover1Req(STQ* pTq, SRpcMsg* pMsg) { double el = (taosGetTimestampMs() - st) / 1000.0; tqDebug("s-task:%s history scan stage(step 1) ended, elapsed time:%.2fs", pTask->id.idStr, el); - if (pTask->fillHistory) { + if (pTask->info.fillHistory) { // todo transfer the executor status, and then destroy this stream task } else { // todo update the chkInfo version for current task. @@ -1199,7 +1199,7 @@ int32_t tqProcessTaskRecover2Req(STQ* pTq, int64_t sversion, char* msg, int32_t return -1; } - atomic_store_8(&pTask->fillHistory, 0); + atomic_store_8(&pTask->info.fillHistory, 0); streamMetaSaveTask(pTq->pStreamMeta, pTask); streamMetaReleaseTask(pTq->pStreamMeta, pTask); @@ -1392,7 +1392,7 @@ int32_t tqProcessTaskResumeReq(STQ* pTq, int64_t sversion, char* msg, int32_t ms atomic_store_8(&pTask->status.taskStatus, pTask->status.keepTaskStatus); // no lock needs to secure the access of the version - if (pReq->igUntreated && pTask->taskLevel == TASK_LEVEL__SOURCE) { + if (pReq->igUntreated && pTask->info.taskLevel == TASK_LEVEL__SOURCE) { // discard all the data when the stream task is suspended. walReaderSetSkipToVersion(pTask->exec.pWalReader, sversion); tqDebug("vgId:%d s-task:%s resume to exec, prev paused version:%" PRId64 ", start from vnode ver:%" PRId64 @@ -1403,7 +1403,7 @@ int32_t tqProcessTaskResumeReq(STQ* pTq, int64_t sversion, char* msg, int32_t ms vgId, pTask->id.idStr, pTask->chkInfo.currentVer, sversion, pTask->status.schedStatus); } - if (pTask->taskLevel == TASK_LEVEL__SOURCE && taosQueueItemSize(pTask->inputQueue->queue) == 0) { + if (pTask->info.taskLevel == TASK_LEVEL__SOURCE && taosQueueItemSize(pTask->inputQueue->queue) == 0) { tqStartStreamTasks(pTq); } else { streamSchedExec(pTask); diff --git a/source/dnode/vnode/src/tq/tqRead.c b/source/dnode/vnode/src/tq/tqRead.c index 155ef92ae5..e562264711 100644 --- a/source/dnode/vnode/src/tq/tqRead.c +++ b/source/dnode/vnode/src/tq/tqRead.c @@ -1109,7 +1109,7 @@ int32_t tqUpdateTbUidList(STQ* pTq, const SArray* tbUidList, bool isAdd) { } SStreamTask* pTask = *(SStreamTask**)pIter; - if (pTask->taskLevel == TASK_LEVEL__SOURCE) { + if (pTask->info.taskLevel == TASK_LEVEL__SOURCE) { int32_t code = qUpdateTableListForStreamScanner(pTask->exec.pExecutor, tbUidList, isAdd); if (code != 0) { tqError("vgId:%d, s-task:%s update qualified table error for stream task", vgId, pTask->id.idStr); diff --git a/source/dnode/vnode/src/tq/tqRestore.c b/source/dnode/vnode/src/tq/tqRestore.c index fe80f48691..cafb64e44a 100644 --- a/source/dnode/vnode/src/tq/tqRestore.c +++ b/source/dnode/vnode/src/tq/tqRestore.c @@ -129,8 +129,8 @@ int32_t createStreamTaskRunReq(SStreamMeta* pStreamMeta, bool* pScanIdle) { } int32_t status = pTask->status.taskStatus; - if (pTask->taskLevel != TASK_LEVEL__SOURCE) { -// tqTrace("s-task:%s level:%d not source task, no need to start", pTask->id.idStr, pTask->taskLevel); + if (pTask->info.taskLevel != TASK_LEVEL__SOURCE) { +// tqTrace("s-task:%s level:%d not source task, no need to start", pTask->id.idStr, pTask->info.taskLevel); streamMetaReleaseTask(pStreamMeta, pTask); continue; } diff --git a/source/libs/qworker/CMakeLists.txt b/source/libs/qworker/CMakeLists.txt index 8ba8b79ab8..7a984cd000 100644 --- a/source/libs/qworker/CMakeLists.txt +++ b/source/libs/qworker/CMakeLists.txt @@ -7,15 +7,9 @@ target_include_directories( PRIVATE "${CMAKE_CURRENT_SOURCE_DIR}/inc" ) -IF (TD_GRANT) - TARGET_LINK_LIBRARIES(qworker - PRIVATE os util transport nodes planner qcom executor index grant - ) -ELSE () - TARGET_LINK_LIBRARIES(qworker - PRIVATE os util transport nodes planner qcom executor index - ) -ENDIF() +TARGET_LINK_LIBRARIES(qworker + PRIVATE os util transport nodes planner qcom executor index + ) if(${BUILD_TEST}) ADD_SUBDIRECTORY(test) diff --git a/source/libs/stream/inc/streamInc.h b/source/libs/stream/inc/streamInc.h index e12a0fdd43..3ed7bc8f7e 100644 --- a/source/libs/stream/inc/streamInc.h +++ b/source/libs/stream/inc/streamInc.h @@ -40,8 +40,6 @@ SStreamDataBlock* createStreamBlockFromResults(SStreamQueueItem* pItem, SStreamT void destroyStreamDataBlock(SStreamDataBlock* pBlock); int32_t streamRetrieveReqToData(const SStreamRetrieveReq* pReq, SStreamDataBlock* pData); -int32_t streamDispatchAllBlocks(SStreamTask* pTask, const SStreamDataBlock* data); - int32_t streamBroadcastToChildren(SStreamTask* pTask, const SSDataBlock* pBlock); int32_t tEncodeStreamRetrieveReq(SEncoder* pEncoder, const SStreamRetrieveReq* pReq); diff --git a/source/libs/stream/src/stream.c b/source/libs/stream/src/stream.c index 7d0a44d2b8..bdf7358f90 100644 --- a/source/libs/stream/src/stream.c +++ b/source/libs/stream/src/stream.c @@ -110,7 +110,7 @@ int32_t streamSchedExec(SStreamTask* pTask) { return -1; } - pRunReq->head.vgId = pTask->nodeId; + pRunReq->head.vgId = pTask->info.nodeId; pRunReq->streamId = pTask->id.streamId; pRunReq->taskId = pTask->id.taskId; @@ -146,7 +146,7 @@ int32_t streamTaskEnqueueBlocks(SStreamTask* pTask, const SStreamDispatchReq* pR pDispatchRsp->streamId = htobe64(pReq->streamId); pDispatchRsp->upstreamNodeId = htonl(pReq->upstreamNodeId); pDispatchRsp->upstreamTaskId = htonl(pReq->upstreamTaskId); - pDispatchRsp->downstreamNodeId = htonl(pTask->nodeId); + pDispatchRsp->downstreamNodeId = htonl(pTask->info.nodeId); pDispatchRsp->downstreamTaskId = htonl(pTask->id.taskId); pRsp->pCont = buf; @@ -162,7 +162,7 @@ int32_t streamTaskEnqueueRetrieve(SStreamTask* pTask, SStreamRetrieveReq* pReq, // enqueue if (pData != NULL) { - qDebug("s-task:%s (child %d) recv retrieve req from task:0x%x, reqId %" PRId64, pTask->id.idStr, pTask->selfChildId, + qDebug("s-task:%s (child %d) recv retrieve req from task:0x%x, reqId %" PRId64, pTask->id.idStr, pTask->info.selfChildId, pReq->srcTaskId, pReq->reqId); pData->type = STREAM_INPUT__DATA_RETRIEVE; @@ -278,10 +278,10 @@ int32_t streamProcessRunReq(SStreamTask* pTask) { } int32_t streamProcessRetrieveReq(SStreamTask* pTask, SStreamRetrieveReq* pReq, SRpcMsg* pRsp) { - qDebug("s-task:%s receive retrieve req from node %d taskId:0x%x", pTask->id.idStr, pReq->srcNodeId, pReq->srcTaskId); + qDebug("s-task:%s receive retrieve req from taskId:0x%x (vgId:%d)", pTask->id.idStr, pReq->srcTaskId, pReq->srcNodeId); streamTaskEnqueueRetrieve(pTask, pReq, pRsp); - ASSERT(pTask->taskLevel != TASK_LEVEL__SINK); + ASSERT(pTask->info.taskLevel != TASK_LEVEL__SINK); streamSchedExec(pTask); return 0; } @@ -299,7 +299,7 @@ int32_t tAppendDataToInputQueue(SStreamTask* pTask, SStreamQueueItem* pItem) { if (type == STREAM_INPUT__DATA_SUBMIT) { SStreamDataSubmit* px = (SStreamDataSubmit*)pItem; - if ((pTask->taskLevel == TASK_LEVEL__SOURCE) && tInputQueueIsFull(pTask)) { + if ((pTask->info.taskLevel == TASK_LEVEL__SOURCE) && tInputQueueIsFull(pTask)) { qError("s-task:%s input queue is full, capacity(size:%d num:%dMiB), current(blocks:%d, size:%.2fMiB) stop to push data", pTask->id.idStr, STREAM_TASK_INPUT_QUEUE_CAPACITY, STREAM_TASK_INPUT_QUEUE_CAPACITY_IN_SIZE, total, size); @@ -319,7 +319,7 @@ int32_t tAppendDataToInputQueue(SStreamTask* pTask, SStreamQueueItem* pItem) { px->submit.msgLen, px->submit.ver, total, size + px->submit.msgLen/1048576.0); } else if (type == STREAM_INPUT__DATA_BLOCK || type == STREAM_INPUT__DATA_RETRIEVE || type == STREAM_INPUT__REF_DATA_BLOCK) { - if ((pTask->taskLevel == TASK_LEVEL__SOURCE) && (tInputQueueIsFull(pTask))) { + if ((pTask->info.taskLevel == TASK_LEVEL__SOURCE) && (tInputQueueIsFull(pTask))) { qError("s-task:%s input queue is full, capacity:%d size:%d MiB, current(blocks:%d, size:%.2fMiB) abort", pTask->id.idStr, STREAM_TASK_INPUT_QUEUE_CAPACITY, STREAM_TASK_INPUT_QUEUE_CAPACITY_IN_SIZE, total, size); @@ -350,19 +350,21 @@ int32_t tAppendDataToInputQueue(SStreamTask* pTask, SStreamQueueItem* pItem) { static void* streamQueueCurItem(SStreamQueue* queue) { return queue->qItem; } -void* streamQueueNextItem(SStreamQueue* queue) { - int8_t dequeueFlag = atomic_exchange_8(&queue->status, STREAM_QUEUE__PROCESSING); - if (dequeueFlag == STREAM_QUEUE__FAILED) { - ASSERT(queue->qItem != NULL); - return streamQueueCurItem(queue); +void* streamQueueNextItem(SStreamQueue* pQueue) { + int8_t flag = atomic_exchange_8(&pQueue->status, STREAM_QUEUE__PROCESSING); + + if (flag == STREAM_QUEUE__FAILED) { + ASSERT(pQueue->qItem != NULL); + return streamQueueCurItem(pQueue); } else { - queue->qItem = NULL; - taosGetQitem(queue->qall, &queue->qItem); - if (queue->qItem == NULL) { - taosReadAllQitems(queue->queue, queue->qall); - taosGetQitem(queue->qall, &queue->qItem); + pQueue->qItem = NULL; + taosGetQitem(pQueue->qall, &pQueue->qItem); + if (pQueue->qItem == NULL) { + taosReadAllQitems(pQueue->queue, pQueue->qall); + taosGetQitem(pQueue->qall, &pQueue->qItem); } - return streamQueueCurItem(queue); + + return streamQueueCurItem(pQueue); } } diff --git a/source/libs/stream/src/streamCheckpoint.c b/source/libs/stream/src/streamCheckpoint.c index 670cfbead1..722c557b8f 100644 --- a/source/libs/stream/src/streamCheckpoint.c +++ b/source/libs/stream/src/streamCheckpoint.c @@ -123,7 +123,7 @@ int32_t tDecodeSStreamCheckpointRsp(SDecoder* pDecoder, SStreamCheckpointRsp* pR static int32_t streamAlignCheckpoint(SStreamTask* pTask, int64_t checkpointId, int32_t childId) { if (pTask->checkpointingId == 0) { pTask->checkpointingId = checkpointId; - pTask->checkpointAlignCnt = taosArrayGetSize(pTask->childEpInfo); + pTask->checkpointAlignCnt = taosArrayGetSize(pTask->pUpstreamEpInfoList); } ASSERT(pTask->checkpointingId == checkpointId); @@ -165,7 +165,7 @@ int32_t streamProcessCheckpointReq(SStreamMeta* pMeta, SStreamTask* pTask, SStre int64_t checkpointId = pReq->checkpointId; int32_t childId = pReq->childId; - if (taosArrayGetSize(pTask->childEpInfo) > 0) { + if (taosArrayGetSize(pTask->pUpstreamEpInfoList) > 0) { code = streamAlignCheckpoint(pTask, checkpointId, childId); if (code > 0) { return 0; diff --git a/source/libs/stream/src/streamData.c b/source/libs/stream/src/streamData.c index 7c06e7deb3..84b5eb3ab7 100644 --- a/source/libs/stream/src/streamData.c +++ b/source/libs/stream/src/streamData.c @@ -64,11 +64,11 @@ SStreamDataBlock* createStreamBlockFromResults(SStreamQueueItem* pItem, SStreamT if (pItem->type == STREAM_INPUT__DATA_SUBMIT) { SStreamDataSubmit* pSubmit = (SStreamDataSubmit*)pItem; - pStreamBlocks->childId = pTask->selfChildId; + pStreamBlocks->childId = pTask->info.selfChildId; pStreamBlocks->sourceVer = pSubmit->ver; } else if (pItem->type == STREAM_INPUT__MERGED_SUBMIT) { SStreamMergedSubmit* pMerged = (SStreamMergedSubmit*)pItem; - pStreamBlocks->childId = pTask->selfChildId; + pStreamBlocks->childId = pTask->info.selfChildId; pStreamBlocks->sourceVer = pMerged->ver; } diff --git a/source/libs/stream/src/streamDispatch.c b/source/libs/stream/src/streamDispatch.c index 26dd19ce7e..25ce470b33 100644 --- a/source/libs/stream/src/streamDispatch.c +++ b/source/libs/stream/src/streamDispatch.c @@ -15,7 +15,9 @@ #include "streamInc.h" -int32_t tEncodeStreamDispatchReq(SEncoder* pEncoder, const SStreamDispatchReq* pReq) { +static int32_t streamDispatchAllBlocks(SStreamTask* pTask, const SStreamDataBlock* pData); + +static int32_t tEncodeStreamDispatchReq(SEncoder* pEncoder, const SStreamDispatchReq* pReq) { if (tStartEncode(pEncoder) < 0) return -1; if (tEncodeI64(pEncoder, pReq->streamId) < 0) return -1; if (tEncodeI32(pEncoder, pReq->taskId) < 0) return -1; @@ -37,6 +39,37 @@ int32_t tEncodeStreamDispatchReq(SEncoder* pEncoder, const SStreamDispatchReq* p return pEncoder->pos; } +static int32_t streamAddBlockIntoDispatchMsg(const SSDataBlock* pBlock, SStreamDispatchReq* pReq) { + int32_t dataStrLen = sizeof(SRetrieveTableRsp) + blockGetEncodeSize(pBlock); + void* buf = taosMemoryCalloc(1, dataStrLen); + if (buf == NULL) return -1; + + SRetrieveTableRsp* pRetrieve = (SRetrieveTableRsp*)buf; + pRetrieve->useconds = 0; + pRetrieve->precision = TSDB_DEFAULT_PRECISION; + pRetrieve->compressed = 0; + pRetrieve->completed = 1; + pRetrieve->streamBlockType = pBlock->info.type; + pRetrieve->numOfRows = htobe64((int64_t)pBlock->info.rows); + pRetrieve->skey = htobe64(pBlock->info.window.skey); + pRetrieve->ekey = htobe64(pBlock->info.window.ekey); + pRetrieve->version = htobe64(pBlock->info.version); + pRetrieve->watermark = htobe64(pBlock->info.watermark); + memcpy(pRetrieve->parTbName, pBlock->info.parTbName, TSDB_TABLE_NAME_LEN); + + int32_t numOfCols = (int32_t)taosArrayGetSize(pBlock->pDataBlock); + pRetrieve->numOfCols = htonl(numOfCols); + + int32_t actualLen = blockEncode(pBlock, pRetrieve->data, numOfCols); + actualLen += sizeof(SRetrieveTableRsp); + ASSERT(actualLen <= dataStrLen); + taosArrayPush(pReq->dataLen, &actualLen); + taosArrayPush(pReq->data, &buf); + + pReq->totalLen += dataStrLen; + return 0; +} + int32_t tDecodeStreamDispatchReq(SDecoder* pDecoder, SStreamDispatchReq* pReq) { if (tStartDecode(pDecoder) < 0) return -1; if (tDecodeI64(pDecoder, &pReq->streamId) < 0) return -1; @@ -125,17 +158,17 @@ int32_t streamBroadcastToChildren(SStreamTask* pTask, const SSDataBlock* pBlock) SStreamRetrieveReq req = { .streamId = pTask->id.streamId, - .srcNodeId = pTask->nodeId, + .srcNodeId = pTask->info.nodeId, .srcTaskId = pTask->id.taskId, .pRetrieve = pRetrieve, .retrieveLen = dataStrLen, }; - int32_t sz = taosArrayGetSize(pTask->childEpInfo); + int32_t sz = taosArrayGetSize(pTask->pUpstreamEpInfoList); ASSERT(sz > 0); for (int32_t i = 0; i < sz; i++) { req.reqId = tGenIdPI64(); - SStreamChildEpInfo* pEpInfo = taosArrayGetP(pTask->childEpInfo, i); + SStreamChildEpInfo* pEpInfo = taosArrayGetP(pTask->pUpstreamEpInfoList, i); req.dstNodeId = pEpInfo->nodeId; req.dstTaskId = pEpInfo->taskId; int32_t len; @@ -165,7 +198,7 @@ int32_t streamBroadcastToChildren(SStreamTask* pTask, const SSDataBlock* pBlock) buf = NULL; qDebug("s-task:%s (child %d) send retrieve req to task:0x%x (vgId:%d), reqId:0x%" PRIx64, pTask->id.idStr, - pTask->selfChildId, pEpInfo->taskId, pEpInfo->nodeId, req.reqId); + pTask->info.selfChildId, pEpInfo->taskId, pEpInfo->nodeId, req.reqId); } code = 0; @@ -175,37 +208,6 @@ CLEAR: return code; } -static int32_t streamAddBlockIntoDispatchMsg(const SSDataBlock* pBlock, SStreamDispatchReq* pReq) { - int32_t dataStrLen = sizeof(SRetrieveTableRsp) + blockGetEncodeSize(pBlock); - void* buf = taosMemoryCalloc(1, dataStrLen); - if (buf == NULL) return -1; - - SRetrieveTableRsp* pRetrieve = (SRetrieveTableRsp*)buf; - pRetrieve->useconds = 0; - pRetrieve->precision = TSDB_DEFAULT_PRECISION; - pRetrieve->compressed = 0; - pRetrieve->completed = 1; - pRetrieve->streamBlockType = pBlock->info.type; - pRetrieve->numOfRows = htobe64((int64_t)pBlock->info.rows); - pRetrieve->skey = htobe64(pBlock->info.window.skey); - pRetrieve->ekey = htobe64(pBlock->info.window.ekey); - pRetrieve->version = htobe64(pBlock->info.version); - pRetrieve->watermark = htobe64(pBlock->info.watermark); - memcpy(pRetrieve->parTbName, pBlock->info.parTbName, TSDB_TABLE_NAME_LEN); - - int32_t numOfCols = (int32_t)taosArrayGetSize(pBlock->pDataBlock); - pRetrieve->numOfCols = htonl(numOfCols); - - int32_t actualLen = blockEncode(pBlock, pRetrieve->data, numOfCols); - actualLen += sizeof(SRetrieveTableRsp); - ASSERT(actualLen <= dataStrLen); - taosArrayPush(pReq->dataLen, &actualLen); - taosArrayPush(pReq->data, &buf); - - pReq->totalLen += dataStrLen; - return 0; -} - int32_t streamDispatchCheckMsg(SStreamTask* pTask, const SStreamTaskCheckReq* pReq, int32_t nodeId, SEpSet* pEpSet) { void* buf = NULL; int32_t code = -1; @@ -315,7 +317,7 @@ int32_t doSendDispatchMsg(SStreamTask* pTask, const SStreamDispatchReq* pReq, in msg.contLen = tlen + sizeof(SMsgHead); msg.pCont = buf; - msg.msgType = pTask->dispatchMsgType; + msg.msgType = pTask->msgInfo.msgType; qDebug("s-task:%s dispatch msg to taskId:0x%x vgId:%d data msg", pTask->id.idStr, pReq->taskId, vgId); tmsgSendReq(pEpSet, &msg); @@ -383,12 +385,12 @@ int32_t streamDispatchAllBlocks(SStreamTask* pTask, const SStreamDataBlock* pDat .streamId = pTask->id.streamId, .dataSrcVgId = pData->srcVgId, .upstreamTaskId = pTask->id.taskId, - .upstreamChildId = pTask->selfChildId, - .upstreamNodeId = pTask->nodeId, + .upstreamChildId = pTask->info.selfChildId, + .upstreamNodeId = pTask->info.nodeId, .blockNum = numOfBlocks, }; - req.data = taosArrayInit(numOfBlocks, sizeof(void*)); + req.data = taosArrayInit(numOfBlocks, POINTER_BYTES); req.dataLen = taosArrayInit(numOfBlocks, sizeof(int32_t)); if (req.data == NULL || req.dataLen == NULL) { taosArrayDestroyP(req.data, taosMemoryFree); @@ -413,8 +415,8 @@ int32_t streamDispatchAllBlocks(SStreamTask* pTask, const SStreamDataBlock* pDat req.taskId = downstreamTaskId; - qDebug("s-task:%s (child taskId:%d) fix-dispatch %d block(s) to down stream s-task:0x%x in vgId:%d", pTask->id.idStr, - pTask->selfChildId, numOfBlocks, downstreamTaskId, vgId); + qDebug("s-task:%s (child taskId:%d) fix-dispatch %d block(s) to s-task:0x%x (vgId:%d)", pTask->id.idStr, + pTask->info.selfChildId, numOfBlocks, downstreamTaskId, vgId); code = doSendDispatchMsg(pTask, &req, vgId, pEpSet); taosArrayDestroyP(req.data, taosMemoryFree); @@ -436,8 +438,8 @@ int32_t streamDispatchAllBlocks(SStreamTask* pTask, const SStreamDataBlock* pDat pReqs[i].streamId = pTask->id.streamId; pReqs[i].dataSrcVgId = pData->srcVgId; pReqs[i].upstreamTaskId = pTask->id.taskId; - pReqs[i].upstreamChildId = pTask->selfChildId; - pReqs[i].upstreamNodeId = pTask->nodeId; + pReqs[i].upstreamChildId = pTask->info.selfChildId; + pReqs[i].upstreamNodeId = pTask->info.nodeId; pReqs[i].blockNum = 0; pReqs[i].data = taosArrayInit(0, sizeof(void*)); pReqs[i].dataLen = taosArrayInit(0, sizeof(int32_t)); @@ -471,13 +473,13 @@ int32_t streamDispatchAllBlocks(SStreamTask* pTask, const SStreamDataBlock* pDat } } - qDebug("s-task:%s (child taskId:%d) shuffle-dispatch blocks:%d to %d vgroups", pTask->id.idStr, pTask->selfChildId, + qDebug("s-task:%s (child taskId:%d) shuffle-dispatch blocks:%d to %d vgroups", pTask->id.idStr, pTask->info.selfChildId, numOfBlocks, vgSz); for (int32_t i = 0; i < vgSz; i++) { if (pReqs[i].blockNum > 0) { SVgroupInfo* pVgInfo = taosArrayGet(vgInfo, i); - qDebug("s-task:%s (child taskId:%d) shuffle-dispatch blocks:%d to vgId:%d", pTask->id.idStr, pTask->selfChildId, + qDebug("s-task:%s (child taskId:%d) shuffle-dispatch blocks:%d to vgId:%d", pTask->id.idStr, pTask->info.selfChildId, pReqs[i].blockNum, pVgInfo->vgId); if (doSendDispatchMsg(pTask, &pReqs[i], pVgInfo->vgId, &pVgInfo->epSet) < 0) { @@ -506,26 +508,26 @@ int32_t streamDispatchStreamBlock(SStreamTask* pTask) { numOfElems); } + // to make sure only one dispatch is running int8_t old = atomic_val_compare_exchange_8(&pTask->outputStatus, TASK_OUTPUT_STATUS__NORMAL, TASK_OUTPUT_STATUS__WAIT); if (old != TASK_OUTPUT_STATUS__NORMAL) { - qDebug("s-task:%s task wait for dispatch rsp, not dispatch now, output status:%d", pTask->id.idStr, old); + qDebug("s-task:%s wait for dispatch rsp, not dispatch now, output status:%d", pTask->id.idStr, old); return 0; } qDebug("s-task:%s start to dispatch msg, set output status:%d", pTask->id.idStr, pTask->outputStatus); - SStreamDataBlock* pDispatchedBlock = streamQueueNextItem(pTask->outputQueue); - if (pDispatchedBlock == NULL) { + SStreamDataBlock* pBlock = streamQueueNextItem(pTask->outputQueue); + if (pBlock == NULL) { atomic_store_8(&pTask->outputStatus, TASK_OUTPUT_STATUS__NORMAL); - qDebug("s-task:%s stop dispatching since no output in output queue, output status:%d", pTask->id.idStr, - pTask->outputStatus); + qDebug("s-task:%s not dispatch since no elems in outputQ, output status:%d", pTask->id.idStr, pTask->outputStatus); return 0; } - ASSERT(pDispatchedBlock->type == STREAM_INPUT__DATA_BLOCK); + ASSERT(pBlock->type == STREAM_INPUT__DATA_BLOCK); - int32_t code = streamDispatchAllBlocks(pTask, pDispatchedBlock); + int32_t code = streamDispatchAllBlocks(pTask, pBlock); if (code != TSDB_CODE_SUCCESS) { streamQueueProcessFail(pTask->outputQueue); atomic_store_8(&pTask->outputStatus, TASK_OUTPUT_STATUS__NORMAL); @@ -533,6 +535,6 @@ int32_t streamDispatchStreamBlock(SStreamTask* pTask) { } // this block can be freed only when it has been pushed to down stream. - destroyStreamDataBlock(pDispatchedBlock); + destroyStreamDataBlock(pBlock); return code; } diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index 46290c306f..096ec25af3 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -108,10 +108,10 @@ static int32_t streamTaskExecImpl(SStreamTask* pTask, SStreamQueueItem* pItem, i assignOneDataBlock(&block, taosArrayGet(pRetrieveBlock->blocks, 0)); block.info.type = STREAM_PULL_OVER; - block.info.childId = pTask->selfChildId; + block.info.childId = pTask->info.selfChildId; taosArrayPush(pRes, &block); numOfBlocks += 1; - qDebug("s-task:%s(child %d) processed retrieve, reqId:0x%" PRIx64, pTask->id.idStr, pTask->selfChildId, + qDebug("s-task:%s(child %d) processed retrieve, reqId:0x%" PRIx64, pTask->id.idStr, pTask->info.selfChildId, pRetrieveBlock->reqId); } @@ -127,7 +127,7 @@ static int32_t streamTaskExecImpl(SStreamTask* pTask, SStreamQueueItem* pItem, i SSDataBlock block = {0}; assignOneDataBlock(&block, output); - block.info.childId = pTask->selfChildId; + block.info.childId = pTask->info.selfChildId; size += blockDataGetSize(output) + sizeof(SSDataBlock) + sizeof(SColumnInfoData) * blockDataGetNumOfCols(&block); numOfBlocks += 1; @@ -135,7 +135,7 @@ static int32_t streamTaskExecImpl(SStreamTask* pTask, SStreamQueueItem* pItem, i taosArrayPush(pRes, &block); qDebug("s-task:%s (child %d) executed and get %d result blocks, size:%.2fMiB", pTask->id.idStr, - pTask->selfChildId, numOfBlocks, size / 1048576.0); + pTask->info.selfChildId, numOfBlocks, size / 1048576.0); // current output should be dispatched to down stream nodes if (numOfBlocks >= MAX_STREAM_RESULT_DUMP_THRESHOLD) { @@ -164,7 +164,7 @@ static int32_t streamTaskExecImpl(SStreamTask* pTask, SStreamQueueItem* pItem, i int32_t streamScanExec(SStreamTask* pTask, int32_t batchSz) { int32_t code = 0; - ASSERT(pTask->taskLevel == TASK_LEVEL__SOURCE); + ASSERT(pTask->info.taskLevel == TASK_LEVEL__SOURCE); void* exec = pTask->exec.pExecutor; qSetStreamOpOpen(exec); @@ -200,7 +200,7 @@ int32_t streamScanExec(SStreamTask* pTask, int32_t batchSz) { SSDataBlock block = {0}; assignOneDataBlock(&block, output); - block.info.childId = pTask->selfChildId; + block.info.childId = pTask->info.selfChildId; taosArrayPush(pRes, &block); batchCnt++; @@ -275,7 +275,7 @@ int32_t streamBatchExec(SStreamTask* pTask, int32_t batchLimit) { return -1; } - if (pTask->taskLevel == TASK_LEVEL__SINK) { + if (pTask->info.taskLevel == TASK_LEVEL__SINK) { ASSERT(((SStreamQueueItem*)pItem)->type == STREAM_INPUT__DATA_BLOCK); streamTaskOutputResultBlock(pTask, (SStreamDataBlock*)pItem); } @@ -344,7 +344,7 @@ int32_t streamExecForAll(SStreamTask* pTask) { SStreamQueueItem* qItem = streamQueueNextItem(pTask->inputQueue); if (qItem == NULL) { - if (pTask->taskLevel == TASK_LEVEL__SOURCE && batchSize < MIN_STREAM_EXEC_BATCH_NUM && times < 5) { + if (pTask->info.taskLevel == TASK_LEVEL__SOURCE && batchSize < MIN_STREAM_EXEC_BATCH_NUM && times < 5) { times++; taosMsleep(10); qDebug("===stream===try again batchSize:%d", batchSize); @@ -358,7 +358,7 @@ int32_t streamExecForAll(SStreamTask* pTask) { if (pInput == NULL) { pInput = qItem; streamQueueProcessSuccess(pTask->inputQueue); - if (pTask->taskLevel == TASK_LEVEL__SINK) { + if (pTask->info.taskLevel == TASK_LEVEL__SINK) { break; } } else { @@ -392,7 +392,7 @@ int32_t streamExecForAll(SStreamTask* pTask) { break; } - if (pTask->taskLevel == TASK_LEVEL__SINK) { + if (pTask->info.taskLevel == TASK_LEVEL__SINK) { ASSERT(pInput->type == STREAM_INPUT__DATA_BLOCK); qDebug("s-task:%s sink task start to sink %d blocks", id, batchSize); streamTaskOutputResultBlock(pTask, (SStreamDataBlock*)pInput); @@ -400,7 +400,7 @@ int32_t streamExecForAll(SStreamTask* pTask) { } // wait for the task to be ready to go - while (pTask->taskLevel == TASK_LEVEL__SOURCE) { + while (pTask->info.taskLevel == TASK_LEVEL__SOURCE) { int8_t status = atomic_load_8(&pTask->status.taskStatus); if (status != TASK_STATUS__NORMAL && status != TASK_STATUS__PAUSE) { qError("stream task wait for the end of fill history, s-task:%s, status:%d", id, @@ -423,7 +423,7 @@ int32_t streamExecForAll(SStreamTask* pTask) { const SStreamTrigger* pTrigger = (const SStreamTrigger*)pInput; qSetMultiStreamInput(pExecutor, pTrigger->pBlock, 1, STREAM_INPUT__DATA_BLOCK); } else if (pItem->type == STREAM_INPUT__DATA_SUBMIT) { - ASSERT(pTask->taskLevel == TASK_LEVEL__SOURCE); + ASSERT(pTask->info.taskLevel == TASK_LEVEL__SOURCE); const SStreamDataSubmit* pSubmit = (const SStreamDataSubmit*)pInput; qSetMultiStreamInput(pExecutor, &pSubmit->submit, 1, STREAM_INPUT__DATA_SUBMIT); qDebug("s-task:%s set submit blocks as source block completed, %p %p len:%d ver:%" PRId64, id, pSubmit, diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index ef3ab2ae46..a4af4c30a1 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -388,7 +388,7 @@ int32_t streamLoadTasks(SStreamMeta* pMeta, int64_t ver) { // todo handle the fill history task ASSERT(0); - if (pTask->fillHistory) { + if (pTask->info.fillHistory) { ASSERT(pTask->status.taskStatus == TASK_STATUS__WAIT_DOWNSTREAM); streamTaskCheckDownstreamTasks(pTask); } diff --git a/source/libs/stream/src/streamRecover.c b/source/libs/stream/src/streamRecover.c index 0bcb078d51..ec52e6fbe8 100644 --- a/source/libs/stream/src/streamRecover.c +++ b/source/libs/stream/src/streamRecover.c @@ -27,9 +27,9 @@ const char* streamGetTaskStatusStr(int32_t status) { } } int32_t streamTaskLaunchRecover(SStreamTask* pTask) { - qDebug("s-task:%s (vgId:%d) launch recover", pTask->id.idStr, pTask->nodeId); + qDebug("s-task:%s (vgId:%d) launch recover", pTask->id.idStr, pTask->info.nodeId); - if (pTask->taskLevel == TASK_LEVEL__SOURCE) { + if (pTask->info.taskLevel == TASK_LEVEL__SOURCE) { atomic_store_8(&pTask->status.taskStatus, TASK_STATUS__RECOVER_PREPARE); SVersionRange* pRange = &pTask->dataRange.range; @@ -56,11 +56,11 @@ int32_t streamTaskLaunchRecover(SStreamTask* pTask) { /*ASSERT(0);*/ } - } else if (pTask->taskLevel == TASK_LEVEL__AGG) { + } else if (pTask->info.taskLevel == TASK_LEVEL__AGG) { streamSetStatusNormal(pTask); streamSetParamForRecover(pTask); streamAggRecoverPrepare(pTask); - } else if (pTask->taskLevel == TASK_LEVEL__SINK) { + } else if (pTask->info.taskLevel == TASK_LEVEL__SINK) { streamSetStatusNormal(pTask); qDebug("s-task:%s sink task convert to normal immediately", pTask->id.idStr); } @@ -77,8 +77,8 @@ int32_t streamTaskCheckDownstreamTasks(SStreamTask* pTask) { SStreamTaskCheckReq req = { .streamId = pTask->id.streamId, .upstreamTaskId = pTask->id.taskId, - .upstreamNodeId = pTask->nodeId, - .childId = pTask->selfChildId, + .upstreamNodeId = pTask->info.nodeId, + .childId = pTask->info.selfChildId, }; // serialize @@ -89,7 +89,7 @@ int32_t streamTaskCheckDownstreamTasks(SStreamTask* pTask) { req.downstreamTaskId = pTask->fixedEpDispatcher.taskId; pTask->checkReqId = req.reqId; - qDebug("s-task:%s (vgId:%d) check downstream task:0x%x (vgId:%d)", pTask->id.idStr, pTask->nodeId, req.downstreamTaskId, + qDebug("s-task:%s (vgId:%d) check downstream task:0x%x (vgId:%d)", pTask->id.idStr, pTask->info.nodeId, req.downstreamTaskId, req.downstreamNodeId); streamDispatchCheckMsg(pTask, &req, pTask->fixedEpDispatcher.nodeId, &pTask->fixedEpDispatcher.epSet); } else if (pTask->outputType == TASK_OUTPUT__SHUFFLE_DISPATCH) { @@ -105,12 +105,12 @@ int32_t streamTaskCheckDownstreamTasks(SStreamTask* pTask) { taosArrayPush(pTask->checkReqIds, &req.reqId); req.downstreamNodeId = pVgInfo->vgId; req.downstreamTaskId = pVgInfo->taskId; - qDebug("s-task:%s (vgId:%d) check downstream task:0x%x (vgId:%d) (shuffle)", pTask->id.idStr, pTask->nodeId, + qDebug("s-task:%s (vgId:%d) check downstream task:0x%x (vgId:%d) (shuffle)", pTask->id.idStr, pTask->info.nodeId, req.downstreamTaskId, req.downstreamNodeId); streamDispatchCheckMsg(pTask, &req, pVgInfo->vgId, &pVgInfo->epSet); } } else { - qDebug("s-task:%s (vgId:%d) direct launch recover since no downstream", pTask->id.idStr, pTask->nodeId); + qDebug("s-task:%s (vgId:%d) direct launch recover since no downstream", pTask->id.idStr, pTask->info.nodeId); streamTaskLaunchRecover(pTask); } @@ -128,7 +128,7 @@ int32_t streamRecheckOneDownstream(SStreamTask* pTask, const SStreamTaskCheckRsp .childId = pRsp->childId, }; - qDebug("s-task:%s (vgId:%d) check downstream task:0x%x (vgId:%d) (recheck)", pTask->id.idStr, pTask->nodeId, + qDebug("s-task:%s (vgId:%d) check downstream task:0x%x (vgId:%d) (recheck)", pTask->id.idStr, pTask->info.nodeId, req.downstreamTaskId, req.downstreamNodeId); if (pTask->outputType == TASK_OUTPUT__FIXED_DISPATCH) { @@ -229,7 +229,7 @@ int32_t streamSourceRecoverPrepareStep1(SStreamTask* pTask, SVersionRange *pVerR } int32_t streamBuildSourceRecover1Req(SStreamTask* pTask, SStreamRecoverStep1Req* pReq) { - pReq->msgHead.vgId = pTask->nodeId; + pReq->msgHead.vgId = pTask->info.nodeId; pReq->streamId = pTask->id.streamId; pReq->taskId = pTask->id.taskId; return 0; @@ -240,7 +240,7 @@ int32_t streamSourceRecoverScanStep1(SStreamTask* pTask) { } int32_t streamBuildSourceRecover2Req(SStreamTask* pTask, SStreamRecoverStep2Req* pReq) { - pReq->msgHead.vgId = pTask->nodeId; + pReq->msgHead.vgId = pTask->info.nodeId; pReq->streamId = pTask->id.streamId; pReq->taskId = pTask->id.taskId; return 0; @@ -264,7 +264,7 @@ int32_t streamSourceRecoverScanStep2(SStreamTask* pTask, int64_t ver) { } int32_t streamDispatchRecoverFinishMsg(SStreamTask* pTask) { - SStreamRecoverFinishReq req = { .streamId = pTask->id.streamId, .childId = pTask->selfChildId }; + SStreamRecoverFinishReq req = { .streamId = pTask->id.streamId, .childId = pTask->info.selfChildId }; // serialize if (pTask->outputType == TASK_OUTPUT__FIXED_DISPATCH) { @@ -287,9 +287,9 @@ int32_t streamDispatchRecoverFinishMsg(SStreamTask* pTask) { // agg int32_t streamAggRecoverPrepare(SStreamTask* pTask) { - pTask->recoverWaitingUpstream = taosArrayGetSize(pTask->childEpInfo); + pTask->numOfWaitingUpstream = taosArrayGetSize(pTask->pUpstreamEpInfoList); qDebug("s-task:%s agg task is ready and wait for %d upstream tasks complete fill history procedure", pTask->id.idStr, - pTask->recoverWaitingUpstream); + pTask->numOfWaitingUpstream); return 0; } @@ -306,8 +306,8 @@ int32_t streamAggChildrenRecoverFinish(SStreamTask* pTask) { } int32_t streamProcessRecoverFinishReq(SStreamTask* pTask, int32_t childId) { - if (pTask->taskLevel == TASK_LEVEL__AGG) { - int32_t left = atomic_sub_fetch_32(&pTask->recoverWaitingUpstream, 1); + if (pTask->info.taskLevel == TASK_LEVEL__AGG) { + int32_t left = atomic_sub_fetch_32(&pTask->numOfWaitingUpstream, 1); qDebug("s-task:%s remain unfinished child tasks:%d", pTask->id.idStr, left); ASSERT(left >= 0); if (left == 0) { diff --git a/source/libs/stream/src/streamTask.c b/source/libs/stream/src/streamTask.c index de10f021d2..7397c90f71 100644 --- a/source/libs/stream/src/streamTask.c +++ b/source/libs/stream/src/streamTask.c @@ -19,7 +19,7 @@ static int32_t addToTaskset(SArray* pArray, SStreamTask* pTask) { int32_t childId = taosArrayGetSize(pArray); - pTask->selfChildId = childId; + pTask->info.selfChildId = childId; taosArrayPush(pArray, &pTask); return 0; } @@ -33,8 +33,8 @@ SStreamTask* tNewStreamTask(int64_t streamId, int8_t taskLevel, int8_t fillHisto pTask->id.taskId = tGenIdPI32(); pTask->id.streamId = streamId; - pTask->taskLevel = taskLevel; - pTask->fillHistory = fillHistory; + pTask->info.taskLevel = taskLevel; + pTask->info.fillHistory = fillHistory; pTask->triggerParam = triggerParam; char buf[128] = {0}; @@ -71,21 +71,21 @@ int32_t tEncodeStreamTask(SEncoder* pEncoder, const SStreamTask* pTask) { if (tStartEncode(pEncoder) < 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->totalLevel) < 0) return -1; - if (tEncodeI8(pEncoder, pTask->taskLevel) < 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->outputType) < 0) return -1; - if (tEncodeI16(pEncoder, pTask->dispatchMsgType) < 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->selfChildId) < 0) return -1; - if (tEncodeI32(pEncoder, pTask->nodeId) < 0) return -1; - if (tEncodeSEpSet(pEncoder, &pTask->epSet) < 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 (tEncodeI64(pEncoder, pTask->chkInfo.id) < 0) return -1; if (tEncodeI64(pEncoder, pTask->chkInfo.version) < 0) return -1; - if (tEncodeI8(pEncoder, pTask->fillHistory) < 0) return -1; + if (tEncodeI8(pEncoder, pTask->info.fillHistory) < 0) return -1; if (tEncodeI64(pEncoder, pTask->historyTaskId.streamId)) return -1; if (tEncodeI32(pEncoder, pTask->historyTaskId.taskId)) return -1; @@ -94,14 +94,14 @@ int32_t tEncodeStreamTask(SEncoder* pEncoder, const SStreamTask* pTask) { if (tEncodeI64(pEncoder, pTask->dataRange.window.skey)) return -1; if (tEncodeI64(pEncoder, pTask->dataRange.window.ekey)) return -1; - int32_t epSz = taosArrayGetSize(pTask->childEpInfo); + int32_t epSz = taosArrayGetSize(pTask->pUpstreamEpInfoList); if (tEncodeI32(pEncoder, epSz) < 0) return -1; for (int32_t i = 0; i < epSz; i++) { - SStreamChildEpInfo* pInfo = taosArrayGetP(pTask->childEpInfo, i); + SStreamChildEpInfo* pInfo = taosArrayGetP(pTask->pUpstreamEpInfoList, i); if (tEncodeStreamEpInfo(pEncoder, pInfo) < 0) return -1; } - if (pTask->taskLevel != TASK_LEVEL__SINK) { + if (pTask->info.taskLevel != TASK_LEVEL__SINK) { if (tEncodeCStr(pEncoder, pTask->exec.qmsg) < 0) return -1; } @@ -131,21 +131,21 @@ int32_t tDecodeStreamTask(SDecoder* pDecoder, SStreamTask* pTask) { if (tStartDecode(pDecoder) < 0) return -1; if (tDecodeI64(pDecoder, &pTask->id.streamId) < 0) return -1; if (tDecodeI32(pDecoder, &pTask->id.taskId) < 0) return -1; - if (tDecodeI32(pDecoder, &pTask->totalLevel) < 0) return -1; - if (tDecodeI8(pDecoder, &pTask->taskLevel) < 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->outputType) < 0) return -1; - if (tDecodeI16(pDecoder, &pTask->dispatchMsgType) < 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->selfChildId) < 0) return -1; - if (tDecodeI32(pDecoder, &pTask->nodeId) < 0) return -1; - if (tDecodeSEpSet(pDecoder, &pTask->epSet) < 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 (tDecodeI64(pDecoder, &pTask->chkInfo.id) < 0) return -1; if (tDecodeI64(pDecoder, &pTask->chkInfo.version) < 0) return -1; - if (tDecodeI8(pDecoder, &pTask->fillHistory) < 0) return -1; + if (tDecodeI8(pDecoder, &pTask->info.fillHistory) < 0) return -1; if (tDecodeI64(pDecoder, &pTask->historyTaskId.streamId)) return -1; if (tDecodeI32(pDecoder, &pTask->historyTaskId.taskId)) return -1; @@ -156,7 +156,8 @@ int32_t tDecodeStreamTask(SDecoder* pDecoder, SStreamTask* pTask) { int32_t epSz; if (tDecodeI32(pDecoder, &epSz) < 0) return -1; - pTask->childEpInfo = taosArrayInit(epSz, sizeof(void*)); + + pTask->pUpstreamEpInfoList = taosArrayInit(epSz, POINTER_BYTES); for (int32_t i = 0; i < epSz; i++) { SStreamChildEpInfo* pInfo = taosMemoryCalloc(1, sizeof(SStreamChildEpInfo)); if (pInfo == NULL) return -1; @@ -164,10 +165,10 @@ int32_t tDecodeStreamTask(SDecoder* pDecoder, SStreamTask* pTask) { taosMemoryFreeClear(pInfo); return -1; } - taosArrayPush(pTask->childEpInfo, &pInfo); + taosArrayPush(pTask->pUpstreamEpInfoList, &pInfo); } - if (pTask->taskLevel != TASK_LEVEL__SINK) { + if (pTask->info.taskLevel != TASK_LEVEL__SINK) { if (tDecodeCStrAlloc(pDecoder, &pTask->exec.qmsg) < 0) return -1; } @@ -217,7 +218,7 @@ void tFreeStreamTask(SStreamTask* pTask) { walCloseReader(pTask->exec.pWalReader); } - taosArrayDestroyP(pTask->childEpInfo, taosMemoryFree); + taosArrayDestroyP(pTask->pUpstreamEpInfoList, taosMemoryFree); if (pTask->outputType == TASK_OUTPUT__TABLE) { tDeleteSchemaWrapper(pTask->tbSink.pSchemaWrapper); taosMemoryFree(pTask->tbSink.pTSchema); From cb26dd9fa2ee8241fb3dcead768114d4588a8978 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 8 Jun 2023 10:17:21 +0800 Subject: [PATCH 11/83] refactor(stream): do some internal refactor. --- include/libs/stream/tstream.h | 12 ++- source/libs/stream/inc/streamInc.h | 2 + source/libs/stream/src/stream.c | 43 +++++++-- source/libs/stream/src/streamDispatch.c | 121 +++++++++++++++--------- source/libs/stream/src/streamExec.c | 20 +--- 5 files changed, 127 insertions(+), 71 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index e67423ca6e..1d80601178 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -292,8 +292,16 @@ typedef struct SSTaskBasicInfo { typedef struct SDispatchMsgInfo { void* pData; // current dispatch data int16_t msgType; // dispatch msg type + int32_t retryCount; // retry send data count + int64_t blockingTs; // output blocking timestamp } SDispatchMsgInfo; +typedef struct { + int8_t outputType; + int8_t outputStatus; + SStreamQueue* outputQueue; +} SSTaskOutputInfo; + struct SStreamTask { SStreamId id; SSTaskBasicInfo info; @@ -536,7 +544,9 @@ int32_t tDecodeStreamDispatchReq(SDecoder* pDecoder, SStreamDispatchReq* pReq); int32_t tDecodeStreamRetrieveReq(SDecoder* pDecoder, SStreamRetrieveReq* pReq); void tDeleteStreamRetrieveReq(SStreamRetrieveReq* pReq); -void tDeleteStreamDispatchReq(SStreamDispatchReq* pReq); +int32_t tInitStreamDispatchReq(SStreamDispatchReq* pReq, const SStreamTask* pTask, int32_t vgId, int32_t numOfBlocks, + int64_t dstTaskId); +void tDeleteStreamDispatchReq(SStreamDispatchReq* pReq); int32_t streamSetupTrigger(SStreamTask* pTask); diff --git a/source/libs/stream/inc/streamInc.h b/source/libs/stream/inc/streamInc.h index 3ed7bc8f7e..e51a618452 100644 --- a/source/libs/stream/inc/streamInc.h +++ b/source/libs/stream/inc/streamInc.h @@ -33,6 +33,7 @@ typedef struct { extern SStreamGlobalEnv streamEnv; +void streamRetryDispatchStreamBlock(SStreamTask* pTask, int64_t waitDuration); int32_t streamDispatchStreamBlock(SStreamTask* pTask); SStreamDataBlock* createStreamDataFromDispatchMsg(const SStreamDispatchReq* pReq, int32_t blockType, int32_t srcVg); @@ -44,6 +45,7 @@ int32_t streamBroadcastToChildren(SStreamTask* pTask, const SSDataBlock* pBlock) int32_t tEncodeStreamRetrieveReq(SEncoder* pEncoder, const SStreamRetrieveReq* pReq); +int32_t streamDispatchAllBlocks(SStreamTask* pTask, const SStreamDataBlock* pData); int32_t streamDispatchCheckMsg(SStreamTask* pTask, const SStreamTaskCheckReq* pReq, int32_t nodeId, SEpSet* pEpSet); int32_t streamDoDispatchRecoverFinishMsg(SStreamTask* pTask, const SStreamRecoverFinishReq* pReq, int32_t vgId, diff --git a/source/libs/stream/src/stream.c b/source/libs/stream/src/stream.c index bdf7358f90..3befd5d55d 100644 --- a/source/libs/stream/src/stream.c +++ b/source/libs/stream/src/stream.c @@ -235,9 +235,26 @@ int32_t streamProcessDispatchMsg(SStreamTask* pTask, SStreamDispatchReq* pReq, S return 0; } +// todo record the idle time for dispatch data int32_t streamProcessDispatchRsp(SStreamTask* pTask, SStreamDispatchRsp* pRsp, int32_t code) { + if (code != TSDB_CODE_SUCCESS) { + // dispatch message failed: network error, or node not available. + // in case of the input queue is full, the code will be TSDB_CODE_SUCCESS, the and pRsp>inputStatus will be set + // flag. here we need to retry dispatch this message to downstream task immediately. handle the case the failure + // happened too fast. todo handle the shuffle dispatch failure + qError("s-task:%s failed to dispatch msg to task:0x%x, code:%s, retry cnt:%d", pTask->id.idStr, + pRsp->downstreamTaskId, tstrerror(code), ++pTask->msgInfo.retryCount); + int32_t ret = streamDispatchAllBlocks(pTask, pTask->msgInfo.pData); + if (ret != TSDB_CODE_SUCCESS) { + + } + + return TSDB_CODE_SUCCESS; + } + qDebug("s-task:%s receive dispatch rsp, output status:%d code:%d", pTask->id.idStr, pRsp->inputStatus, code); + // there are other dispatch message not response yet if (pTask->outputType == TASK_OUTPUT__SHUFFLE_DISPATCH) { int32_t leftRsp = atomic_sub_fetch_32(&pTask->shuffleDispatcher.waitingRspCnt, 1); qDebug("s-task:%s is shuffle, left waiting rsp %d", pTask->id.idStr, leftRsp); @@ -246,23 +263,31 @@ int32_t streamProcessDispatchRsp(SStreamTask* pTask, SStreamDispatchRsp* pRsp, i } } + pTask->msgInfo.retryCount = 0; int8_t old = atomic_exchange_8(&pTask->outputStatus, pRsp->inputStatus); ASSERT(old == TASK_OUTPUT_STATUS__WAIT); + qDebug("s-task:%s output status is set to:%d", pTask->id.idStr, pTask->outputStatus); + // the input queue of the (down stream) task that receive the output data is full, so the TASK_INPUT_STATUS_BLOCKED is rsp - // todo we need to send EMPTY PACKAGE to detect if the input queue is available for output of upstream task, every 50 ms. - if (pRsp->inputStatus == TASK_INPUT_STATUS__BLOCKED) { - // TODO: init recover timer - qError("s-task:%s inputQ of downstream task:0x%x is full, need to block output", pTask->id.idStr, pRsp->downstreamTaskId); + if (pTask->outputStatus == TASK_INPUT_STATUS__BLOCKED) { + pTask->msgInfo.blockingTs = taosGetTimestampMs(); // record the blocking start time + + int32_t waitDuration = 300; // 300 ms + qError("s-task:%s inputQ of downstream task:0x%x is full, time:%" PRId64 "wait for %dms and retry dispatch data", + pTask->id.idStr, pRsp->downstreamTaskId, pTask->msgInfo.blockingTs, waitDuration); + streamRetryDispatchStreamBlock(pTask, waitDuration); + } else { // pipeline send data in output queue + // this message has been sent successfully, let's try next one. + destroyStreamDataBlock(pTask->msgInfo.pData); + pTask->msgInfo.pData = NULL; atomic_store_8(&pTask->outputStatus, TASK_OUTPUT_STATUS__NORMAL); - qError("s-task:%s ignore error, and reset task output status:%d", pTask->id.idStr, pTask->outputStatus); - return 0; + // otherwise, continue dispatch the first block to down stream task in pipeline + streamDispatchStreamBlock(pTask); } - // otherwise, continue dispatch the first block to down stream task in pipeline - streamDispatchStreamBlock(pTask); return 0; } @@ -271,7 +296,7 @@ int32_t streamProcessRunReq(SStreamTask* pTask) { return -1; } - /*if (pTask->outputType == TASK_OUTPUT__FIXED_DISPATCH || pTask->outputType == TASK_OUTPUT__SHUFFLE_DISPATCH) {*/ + /*if (pTask->dispatchType == TASK_OUTPUT__FIXED_DISPATCH || pTask->dispatchType == TASK_OUTPUT__SHUFFLE_DISPATCH) {*/ /*streamDispatchStreamBlock(pTask);*/ /*}*/ return 0; diff --git a/source/libs/stream/src/streamDispatch.c b/source/libs/stream/src/streamDispatch.c index 25ce470b33..3d01d4411d 100644 --- a/source/libs/stream/src/streamDispatch.c +++ b/source/libs/stream/src/streamDispatch.c @@ -13,10 +13,9 @@ * along with this program. If not, see . */ +#include #include "streamInc.h" -static int32_t streamDispatchAllBlocks(SStreamTask* pTask, const SStreamDataBlock* pData); - static int32_t tEncodeStreamDispatchReq(SEncoder* pEncoder, const SStreamDispatchReq* pReq) { if (tStartEncode(pEncoder) < 0) return -1; if (tEncodeI64(pEncoder, pReq->streamId) < 0) return -1; @@ -98,6 +97,27 @@ int32_t tDecodeStreamDispatchReq(SDecoder* pDecoder, SStreamDispatchReq* pReq) { return 0; } +int32_t tInitStreamDispatchReq(SStreamDispatchReq* pReq, const SStreamTask* pTask, int32_t vgId, int32_t numOfBlocks, + int64_t dstTaskId) { + pReq->streamId = pTask->id.streamId; + pReq->dataSrcVgId = vgId; + pReq->upstreamTaskId = pTask->id.taskId; + pReq->upstreamChildId = pTask->info.selfChildId; + pReq->upstreamNodeId = pTask->info.nodeId; + pReq->blockNum = numOfBlocks; + pReq->taskId = dstTaskId; + + pReq->data = taosArrayInit(numOfBlocks, POINTER_BYTES); + pReq->dataLen = taosArrayInit(numOfBlocks, sizeof(int32_t)); + if (pReq->data == NULL || pReq->dataLen == NULL) { + taosArrayDestroyP(pReq->data, taosMemoryFree); + taosArrayDestroy(pReq->dataLen); + return TSDB_CODE_OUT_OF_MEMORY; + } + + return TSDB_CODE_SUCCESS; +} + void tDeleteStreamDispatchReq(SStreamDispatchReq* pReq) { taosArrayDestroyP(pReq->data, taosMemoryFree); taosArrayDestroy(pReq->dataLen); @@ -248,7 +268,7 @@ int32_t streamDispatchCheckMsg(SStreamTask* pTask, const SStreamTaskCheckReq* pR } int32_t streamDoDispatchRecoverFinishMsg(SStreamTask* pTask, const SStreamRecoverFinishReq* pReq, int32_t vgId, - SEpSet* pEpSet) { + SEpSet* pEpSet) { void* buf = NULL; int32_t code = -1; SRpcMsg msg = {0}; @@ -377,25 +397,17 @@ int32_t streamSearchAndAddBlock(SStreamTask* pTask, SStreamDispatchReq* pReqs, S int32_t streamDispatchAllBlocks(SStreamTask* pTask, const SStreamDataBlock* pData) { int32_t code = 0; + int32_t numOfBlocks = taosArrayGetSize(pData->blocks); ASSERT(numOfBlocks != 0); if (pTask->outputType == TASK_OUTPUT__FIXED_DISPATCH) { - SStreamDispatchReq req = { - .streamId = pTask->id.streamId, - .dataSrcVgId = pData->srcVgId, - .upstreamTaskId = pTask->id.taskId, - .upstreamChildId = pTask->info.selfChildId, - .upstreamNodeId = pTask->info.nodeId, - .blockNum = numOfBlocks, - }; + SStreamDispatchReq req = {0}; - req.data = taosArrayInit(numOfBlocks, POINTER_BYTES); - req.dataLen = taosArrayInit(numOfBlocks, sizeof(int32_t)); - if (req.data == NULL || req.dataLen == NULL) { - taosArrayDestroyP(req.data, taosMemoryFree); - taosArrayDestroy(req.dataLen); - return TSDB_CODE_OUT_OF_MEMORY; + int32_t downstreamTaskId = pTask->fixedEpDispatcher.taskId; + code = tInitStreamDispatchReq(&req, pTask, pData->srcVgId, numOfBlocks, downstreamTaskId); + if (code != TSDB_CODE_SUCCESS) { + return code; } for (int32_t i = 0; i < numOfBlocks; i++) { @@ -411,9 +423,6 @@ int32_t streamDispatchAllBlocks(SStreamTask* pTask, const SStreamDataBlock* pDat int32_t vgId = pTask->fixedEpDispatcher.nodeId; SEpSet* pEpSet = &pTask->fixedEpDispatcher.epSet; - int32_t downstreamTaskId = pTask->fixedEpDispatcher.taskId; - - req.taskId = downstreamTaskId; qDebug("s-task:%s (child taskId:%d) fix-dispatch %d block(s) to s-task:0x%x (vgId:%d)", pTask->id.idStr, pTask->info.selfChildId, numOfBlocks, downstreamTaskId, vgId); @@ -426,8 +435,9 @@ int32_t streamDispatchAllBlocks(SStreamTask* pTask, const SStreamDataBlock* pDat int32_t rspCnt = atomic_load_32(&pTask->shuffleDispatcher.waitingRspCnt); ASSERT(rspCnt == 0); - SArray* vgInfo = pTask->shuffleDispatcher.dbInfo.pVgroupInfos; - int32_t vgSz = taosArrayGetSize(vgInfo); + SArray* vgInfo = pTask->shuffleDispatcher.dbInfo.pVgroupInfos; + int32_t vgSz = taosArrayGetSize(vgInfo); + SStreamDispatchReq* pReqs = taosMemoryCalloc(vgSz, sizeof(SStreamDispatchReq)); if (pReqs == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; @@ -435,20 +445,11 @@ int32_t streamDispatchAllBlocks(SStreamTask* pTask, const SStreamDataBlock* pDat } for (int32_t i = 0; i < vgSz; i++) { - pReqs[i].streamId = pTask->id.streamId; - pReqs[i].dataSrcVgId = pData->srcVgId; - pReqs[i].upstreamTaskId = pTask->id.taskId; - pReqs[i].upstreamChildId = pTask->info.selfChildId; - pReqs[i].upstreamNodeId = pTask->info.nodeId; - pReqs[i].blockNum = 0; - pReqs[i].data = taosArrayInit(0, sizeof(void*)); - pReqs[i].dataLen = taosArrayInit(0, sizeof(int32_t)); - if (pReqs[i].data == NULL || pReqs[i].dataLen == NULL) { + SVgroupInfo* pVgInfo = taosArrayGet(vgInfo, i); + code = tInitStreamDispatchReq(&pReqs[i], pTask, pData->srcVgId, 0, pVgInfo->taskId); + if (code != TSDB_CODE_SUCCESS) { goto FAIL_SHUFFLE_DISPATCH; } - - SVgroupInfo* pVgInfo = taosArrayGet(vgInfo, i); - pReqs[i].taskId = pVgInfo->taskId; } for (int32_t i = 0; i < numOfBlocks; i++) { @@ -456,15 +457,18 @@ int32_t streamDispatchAllBlocks(SStreamTask* pTask, const SStreamDataBlock* pDat // TODO: do not use broadcast if (pDataBlock->info.type == STREAM_DELETE_RESULT) { + for (int32_t j = 0; j < vgSz; j++) { if (streamAddBlockIntoDispatchMsg(pDataBlock, &pReqs[j]) < 0) { goto FAIL_SHUFFLE_DISPATCH; } + if (pReqs[j].blockNum == 0) { atomic_add_fetch_32(&pTask->shuffleDispatcher.waitingRspCnt, 1); } pReqs[j].blockNum++; } + continue; } @@ -495,13 +499,31 @@ int32_t streamDispatchAllBlocks(SStreamTask* pTask, const SStreamDataBlock* pDat taosArrayDestroyP(pReqs[i].data, taosMemoryFree); taosArrayDestroy(pReqs[i].dataLen); } + taosMemoryFree(pReqs); } + return code; } +static void doRetryDispatchData(void* param, void* tmrId) { + SStreamTask* pTask = param; + ASSERT(pTask->outputStatus == TASK_OUTPUT_STATUS__WAIT); + + int32_t code = streamDispatchAllBlocks(pTask, pTask->msgInfo.pData); + if (code != TSDB_CODE_SUCCESS) { + streamRetryDispatchStreamBlock(pTask, 300); + } +} + +void streamRetryDispatchStreamBlock(SStreamTask* pTask, int64_t waitDuration) { + qError("s-task:%s dispatch data in %"PRId64"ms", pTask->id.idStr, waitDuration); + taosTmrReset(doRetryDispatchData, waitDuration, pTask, streamEnv.timer, &pTask->timer); +} + int32_t streamDispatchStreamBlock(SStreamTask* pTask) { - ASSERT(pTask->outputType == TASK_OUTPUT__FIXED_DISPATCH || pTask->outputType == TASK_OUTPUT__SHUFFLE_DISPATCH); + ASSERT((pTask->outputType == TASK_OUTPUT__FIXED_DISPATCH || pTask->outputType == TASK_OUTPUT__SHUFFLE_DISPATCH)); + int32_t numOfElems = taosQueueItemSize(pTask->outputQueue->queue); if (numOfElems > 0) { qDebug("s-task:%s try to dispatch intermediate result block to downstream, elem in outputQ:%d", pTask->id.idStr, @@ -516,6 +538,7 @@ int32_t streamDispatchStreamBlock(SStreamTask* pTask) { return 0; } + ASSERT(pTask->msgInfo.pData == NULL); qDebug("s-task:%s start to dispatch msg, set output status:%d", pTask->id.idStr, pTask->outputStatus); SStreamDataBlock* pBlock = streamQueueNextItem(pTask->outputQueue); @@ -525,16 +548,28 @@ int32_t streamDispatchStreamBlock(SStreamTask* pTask) { return 0; } + pTask->msgInfo.pData = pBlock; ASSERT(pBlock->type == STREAM_INPUT__DATA_BLOCK); - int32_t code = streamDispatchAllBlocks(pTask, pBlock); - if (code != TSDB_CODE_SUCCESS) { - streamQueueProcessFail(pTask->outputQueue); - atomic_store_8(&pTask->outputStatus, TASK_OUTPUT_STATUS__NORMAL); - qDebug("s-task:%s failed to dispatch msg to downstream, output status:%d", pTask->id.idStr, pTask->outputStatus); + int32_t retryCount = 0; + + while (1) { + int32_t code = streamDispatchAllBlocks(pTask, pBlock); + if (code == TSDB_CODE_SUCCESS) { + break; + } + + qDebug("s-task:%s failed to dispatch msg to downstream, code:%s, output status:%d, retry cnt:%d", pTask->id.idStr, + tstrerror(code), pTask->outputStatus, retryCount); + + if (++retryCount > 5) { // add to timer to retry + qDebug("s-task:%s failed to dispatch msg to downstream for %d times, code:%s, retry in %dms", pTask->id.idStr, + retryCount, tstrerror(code), 300); + streamRetryDispatchStreamBlock(pTask, 300); + break; + } } - // this block can be freed only when it has been pushed to down stream. - destroyStreamDataBlock(pBlock); - return code; + // this block can not be deleted until it has been sent to downstream task successfully. + return TSDB_CODE_SUCCESS; } diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index 096ec25af3..a1e72015ff 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -237,11 +237,6 @@ int32_t streamScanExec(SStreamTask* pTask, int32_t batchSz) { taosFreeQitem(qRes); return code; } -// -// if (pTask->outputType == TASK_OUTPUT__FIXED_DISPATCH || pTask->outputType == TASK_OUTPUT__SHUFFLE_DISPATCH) { -// qDebug("s-task:%s scan exec dispatch blocks:%d", pTask->id.idStr, batchCnt); -// streamDispatchStreamBlock(pTask); -// } if (finished) { break; @@ -334,7 +329,8 @@ int32_t streamExecForAll(SStreamTask* pTask) { qDebug("s-task:%s start to extract data block from inputQ", id); while (1) { - if (streamTaskShouldPause(&pTask->status)) { + // downstream task's input queue is blocked, stop immediately + if (streamTaskShouldPause(&pTask->status) || (pTask->outputStatus == TASK_INPUT_STATUS__BLOCKED)) { if (batchSize > 1) { break; } else { @@ -399,18 +395,6 @@ int32_t streamExecForAll(SStreamTask* pTask) { continue; } - // wait for the task to be ready to go - while (pTask->info.taskLevel == TASK_LEVEL__SOURCE) { - int8_t status = atomic_load_8(&pTask->status.taskStatus); - if (status != TASK_STATUS__NORMAL && status != TASK_STATUS__PAUSE) { - qError("stream task wait for the end of fill history, s-task:%s, status:%d", id, - atomic_load_8(&pTask->status.taskStatus)); - taosMsleep(100); - } else { - break; - } - } - int64_t st = taosGetTimestampMs(); qDebug("s-task:%s start to process batch of blocks, num:%d", id, batchSize); From f582705febb3a887dc3296d03f4e04bbb44d9cc3 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 8 Jun 2023 14:18:06 +0800 Subject: [PATCH 12/83] refactor: do some internal refactor. --- include/common/tdatablock.h | 9 +- source/common/src/tdatablock.c | 510 +++++++----------------- source/libs/executor/src/executorInt.c | 141 +------ source/libs/executor/src/scanoperator.c | 34 +- 4 files changed, 177 insertions(+), 517 deletions(-) diff --git a/include/common/tdatablock.h b/include/common/tdatablock.h index 33c571fc1b..1fa5b63a9a 100644 --- a/include/common/tdatablock.h +++ b/include/common/tdatablock.h @@ -186,7 +186,6 @@ int32_t colDataAssign(SColumnInfoData* pColumnInfoData, const SColumnInfoData* p int32_t blockDataUpdateTsWindow(SSDataBlock* pDataBlock, int32_t tsColumnIndex); int32_t colDataGetLength(const SColumnInfoData* pColumnInfoData, int32_t numOfRows); -void colDataTrim(SColumnInfoData* pColumnInfoData); size_t blockDataGetNumOfCols(const SSDataBlock* pBlock); size_t blockDataGetNumOfRows(const SSDataBlock* pBlock); @@ -206,7 +205,6 @@ double blockDataGetSerialRowSize(const SSDataBlock* pBlock); size_t blockDataGetSerialMetaSize(uint32_t numOfCols); int32_t blockDataSort(SSDataBlock* pDataBlock, SArray* pOrderInfo); -int32_t blockDataSort_rv(SSDataBlock* pDataBlock, SArray* pOrderInfo, bool nullFirst); int32_t colInfoDataEnsureCapacity(SColumnInfoData* pColumn, uint32_t numOfRows, bool clearPayload); int32_t blockDataEnsureCapacity(SSDataBlock* pDataBlock, uint32_t numOfRows); @@ -235,11 +233,10 @@ int32_t blockDataAppendColInfo(SSDataBlock* pBlock, SColumnInfoData* pColIn SColumnInfoData createColumnInfoData(int16_t type, int32_t bytes, int16_t colId); SColumnInfoData* bdGetColumnInfoData(const SSDataBlock* pBlock, int32_t index); +int32_t blockGetEncodeSize(const SSDataBlock* pBlock); int32_t blockEncode(const SSDataBlock* pBlock, char* data, int32_t numOfCols); const char* blockDecode(SSDataBlock* pBlock, const char* pData); -void blockDebugShowDataBlock(SSDataBlock* pBlock, const char* flag); -void blockDebugShowDataBlocks(const SArray* dataBlocks, const char* flag); // for debug char* dumpBlockData(SSDataBlock* pDataBlock, const char* flag, char** dumpBuf); @@ -248,9 +245,7 @@ int32_t buildSubmitReqFromDataBlock(SSubmitReq2** pReq, const SSDataBlock* pData char* buildCtbNameByGroupId(const char* stbName, uint64_t groupId); -static FORCE_INLINE int32_t blockGetEncodeSize(const SSDataBlock* pBlock) { - return blockDataGetSerialMetaSize(taosArrayGetSize(pBlock->pDataBlock)) + blockDataGetSize(pBlock); -} +void trimDataBlock(SSDataBlock* pBlock, int32_t totalRows, const bool* pBoolList); #ifdef __cplusplus } diff --git a/source/common/src/tdatablock.c b/source/common/src/tdatablock.c index 311c79381c..fe38225ac8 100644 --- a/source/common/src/tdatablock.c +++ b/source/common/src/tdatablock.c @@ -33,7 +33,7 @@ int32_t colDataGetLength(const SColumnInfoData* pColumnInfoData, int32_t numOfRo } } -int32_t colDataGetFullLength(const SColumnInfoData* pColumnInfoData, int32_t numOfRows) { +static int32_t colDataGetFullLength(const SColumnInfoData* pColumnInfoData, int32_t numOfRows) { if (IS_VAR_DATA_TYPE(pColumnInfoData->info.type)) { return pColumnInfoData->varmeta.length + sizeof(int32_t) * numOfRows; } else { @@ -42,10 +42,6 @@ int32_t colDataGetFullLength(const SColumnInfoData* pColumnInfoData, int32_t num } } -void colDataTrim(SColumnInfoData* pColumnInfoData) { - // TODO -} - int32_t getJsonValueLen(const char* data) { int32_t dataLen = 0; if (*data == TSDB_DATA_TYPE_NULL) { @@ -820,41 +816,8 @@ int32_t dataBlockCompar(const void* p1, const void* p2, const void* param) { return 0; } -static int32_t doAssignOneTuple(SColumnInfoData* pDstCols, int32_t numOfRows, const SSDataBlock* pSrcBlock, - int32_t tupleIndex) { - int32_t code = 0; - size_t numOfCols = taosArrayGetSize(pSrcBlock->pDataBlock); - - for (int32_t i = 0; i < numOfCols; ++i) { - SColumnInfoData* pDst = &pDstCols[i]; - SColumnInfoData* pSrc = taosArrayGet(pSrcBlock->pDataBlock, i); - - if (pSrc->hasNull && colDataIsNull(pSrc, pSrcBlock->info.rows, tupleIndex, pSrcBlock->pBlockAgg[i])) { - code = colDataSetVal(pDst, numOfRows, NULL, true); - if (code != TSDB_CODE_SUCCESS) { - return code; - } - } else { - char* p = colDataGetData(pSrc, tupleIndex); - code = colDataSetVal(pDst, numOfRows, p, false); - if (code != TSDB_CODE_SUCCESS) { - return code; - } - } - } - - return TSDB_CODE_SUCCESS; -} - static int32_t blockDataAssign(SColumnInfoData* pCols, const SSDataBlock* pDataBlock, const int32_t* index) { -#if 0 - for (int32_t i = 0; i < pDataBlock->info.rows; ++i) { - int32_t code = doAssignOneTuple(pCols, i, pDataBlock, index[i]); - if (code != TSDB_CODE_SUCCESS) { - return code; - } - } -#else + size_t numOfCols = taosArrayGetSize(pDataBlock->pDataBlock); for (int32_t i = 0; i < numOfCols; ++i) { SColumnInfoData* pDst = &pCols[i]; @@ -879,7 +842,7 @@ static int32_t blockDataAssign(SColumnInfoData* pCols, const SSDataBlock* pDataB } } } -#endif + return TSDB_CODE_SUCCESS; } @@ -1039,114 +1002,6 @@ int32_t blockDataSort(SSDataBlock* pDataBlock, SArray* pOrderInfo) { return TSDB_CODE_SUCCESS; } -#if 0 -typedef struct SHelper { - int32_t index; - union { - char* pData; - int64_t i64; - double d64; - }; -} SHelper; - -SHelper* createTupleIndex_rv(int32_t numOfRows, SArray* pOrderInfo, SSDataBlock* pBlock) { - int32_t sortValLengthPerRow = 0; - int32_t numOfCols = taosArrayGetSize(pOrderInfo); - - for (int32_t i = 0; i < numOfCols; ++i) { - SBlockOrderInfo* pInfo = taosArrayGet(pOrderInfo, i); - SColumnInfoData* pColInfo = taosArrayGet(pBlock->pDataBlock, pInfo->slotId); - pInfo->pColData = pColInfo; - sortValLengthPerRow += pColInfo->info.bytes; - } - - size_t len = sortValLengthPerRow * pBlock->info.rows; - - char* buf = taosMemoryCalloc(1, len); - SHelper* phelper = taosMemoryCalloc(numOfRows, sizeof(SHelper)); - for (int32_t i = 0; i < numOfRows; ++i) { - phelper[i].index = i; - phelper[i].pData = buf + sortValLengthPerRow * i; - } - - int32_t offset = 0; - for (int32_t i = 0; i < numOfCols; ++i) { - SBlockOrderInfo* pInfo = taosArrayGet(pOrderInfo, i); - for (int32_t j = 0; j < numOfRows; ++j) { - phelper[j].i64 = *(int32_t*)pInfo->pColData->pData + pInfo->pColData->info.bytes * j; - // memcpy(phelper[j].pData + offset, pInfo->pColData->pData + pInfo->pColData->info.bytes * j, - // pInfo->pColData->info.bytes); - } - - offset += pInfo->pColData->info.bytes; - } - - taosMemoryFree(buf); - return phelper; -} - -int32_t dataBlockCompar_rv(const void* p1, const void* p2, const void* param) { - const SSDataBlockSortHelper* pHelper = (const SSDataBlockSortHelper*)param; - - SHelper* left = (SHelper*)p1; - SHelper* right = (SHelper*)p2; - - SArray* pInfo = pHelper->orderInfo; - - int32_t offset = 0; - int32_t leftx = *(int32_t*)left->pData; //*(int32_t*)(left->pData + offset); - int32_t rightx = *(int32_t*)right->pData; //*(int32_t*)(right->pData + offset); - - if (leftx == rightx) { - return 0; - } else { - return (leftx < rightx) ? -1 : 1; - } - return 0; -} - -int32_t blockDataSort_rv(SSDataBlock* pDataBlock, SArray* pOrderInfo, bool nullFirst) { - // Allocate the additional buffer. - int64_t p0 = taosGetTimestampUs(); - - SSDataBlockSortHelper helper = {.pDataBlock = pDataBlock, .orderInfo = pOrderInfo}; - - uint32_t rows = pDataBlock->info.rows; - SHelper* index = createTupleIndex_rv(rows, helper.orderInfo, pDataBlock); - if (index == NULL) { - terrno = TSDB_CODE_OUT_OF_MEMORY; - return terrno; - } - - taosqsort(index, rows, sizeof(SHelper), &helper, dataBlockCompar_rv); - - int64_t p1 = taosGetTimestampUs(); - SColumnInfoData* pCols = createHelpColInfoData(pDataBlock); - if (pCols == NULL) { - terrno = TSDB_CODE_OUT_OF_MEMORY; - return terrno; - } - - int64_t p2 = taosGetTimestampUs(); - - // int32_t code = blockDataAssign(pCols, pDataBlock, index); - // if (code != TSDB_CODE_SUCCESS) { - // terrno = code; - // return code; - // } - - int64_t p3 = taosGetTimestampUs(); - - copyBackToBlock(pDataBlock, pCols); - int64_t p4 = taosGetTimestampUs(); - - printf("sort:%" PRId64 ", create:%" PRId64 ", assign:%" PRId64 ", copyback:%" PRId64 ", rows:%d\n", p1 - p0, p2 - p1, - p3 - p2, p4 - p3, rows); - // destroyTupleIndex(index); - return 0; -} -#endif - void blockDataCleanup(SSDataBlock* pDataBlock) { blockDataEmpty(pDataBlock); SDataBlockInfo* pInfo = &pDataBlock->info; @@ -1299,6 +1154,7 @@ void* blockDataDestroy(SSDataBlock* pBlock) { return NULL; } +// todo remove it int32_t assignOneDataBlock(SSDataBlock* dst, const SSDataBlock* src) { dst->info = src->info; dst->info.rows = 0; @@ -1679,16 +1535,6 @@ static void colDataKeepFirstNRows(SColumnInfoData* pColInfoData, size_t n, size_ if (IS_VAR_DATA_TYPE(pColInfoData->info.type)) { pColInfoData->varmeta.length = colDataMoveVarData(pColInfoData, 0, n); memset(&pColInfoData->varmeta.offset[n], 0, total - n); - } else { // reset the bitmap value - /*int32_t stopIndex = BitmapLen(n) * 8; - for(int32_t i = n; i < stopIndex; ++i) { - colDataClearNull_f(pColInfoData->nullbitmap, i); - } - - int32_t remain = BitmapLen(total) - BitmapLen(n); - if (remain > 0) { - memset(pColInfoData->nullbitmap+BitmapLen(n), 0, remain); - }*/ } } @@ -1782,32 +1628,6 @@ void* tDecodeDataBlock(const void* buf, SSDataBlock* pBlock) { return (void*)buf; } -int32_t tEncodeDataBlocks(void** buf, const SArray* blocks) { - int32_t tlen = 0; - int32_t sz = taosArrayGetSize(blocks); - tlen += taosEncodeFixedI32(buf, sz); - - for (int32_t i = 0; i < sz; i++) { - SSDataBlock* pBlock = taosArrayGet(blocks, i); - tlen += tEncodeDataBlock(buf, pBlock); - } - - return tlen; -} - -void* tDecodeDataBlocks(const void* buf, SArray** blocks) { - int32_t sz; - buf = taosDecodeFixedI32(buf, &sz); - - *blocks = taosArrayInit(sz, sizeof(SSDataBlock)); - for (int32_t i = 0; i < sz; i++) { - SSDataBlock pBlock = {0}; - buf = tDecodeDataBlock(buf, &pBlock); - taosArrayPush(*blocks, &pBlock); - } - return (void*)buf; -} - static char* formatTimestamp(char* buf, int64_t val, int precision) { time_t tt; int32_t ms = 0; @@ -2060,182 +1880,6 @@ char* dumpBlockData(SSDataBlock* pDataBlock, const char* flag, char** pDataBuf) return dumpBuf; } -/** - * @brief TODO: Assume that the final generated result it less than 3M - * - * @param pReq - * @param pDataBlocks - * @param vgId - * @param suid - * - */ -#if 0 -int32_t buildSubmitReqFromDataBlock(SSubmitReq** pReq, const SSDataBlock* pDataBlock, STSchema* pTSchema, int32_t vgId, - tb_uid_t suid) { - int32_t bufSize = sizeof(SSubmitReq); - int32_t sz = 1; - for (int32_t i = 0; i < sz; ++i) { - const SDataBlockInfo* pBlkInfo = &pDataBlock->info; - - int32_t colNum = taosArrayGetSize(pDataBlock->pDataBlock); - bufSize += pBlkInfo->rows * (TD_ROW_HEAD_LEN + pBlkInfo->rowSize + BitmapLen(colNum)); - bufSize += sizeof(SSubmitBlk); - } - - *pReq = taosMemoryCalloc(1, bufSize); - if (!(*pReq)) { - terrno = TSDB_CODE_OUT_OF_MEMORY; - return TSDB_CODE_FAILED; - } - void* pDataBuf = *pReq; - - int32_t msgLen = sizeof(SSubmitReq); - int32_t numOfBlks = 0; - SRowBuilder rb = {0}; - tdSRowInit(&rb, pTSchema->version); - - for (int32_t i = 0; i < sz; ++i) { - int32_t colNum = taosArrayGetSize(pDataBlock->pDataBlock); - int32_t rows = pDataBlock->info.rows; - - if (colNum <= 1) { - // invalid if only with TS col - continue; - } - - if (rb.nCols != colNum) { - tdSRowSetTpInfo(&rb, colNum, pTSchema->flen); - } - - SSubmitBlk* pSubmitBlk = POINTER_SHIFT(pDataBuf, msgLen); - pSubmitBlk->suid = suid; - pSubmitBlk->uid = pDataBlock->info.id.groupId; - pSubmitBlk->numOfRows = rows; - pSubmitBlk->sversion = pTSchema->version; - - msgLen += sizeof(SSubmitBlk); - int32_t dataLen = 0; - for (int32_t j = 0; j < rows; ++j) { // iterate by row - tdSRowResetBuf(&rb, POINTER_SHIFT(pDataBuf, msgLen + dataLen)); // set row buf - bool isStartKey = false; - int32_t offset = 0; - for (int32_t k = 0; k < colNum; ++k) { // iterate by column - SColumnInfoData* pColInfoData = taosArrayGet(pDataBlock->pDataBlock, k); - STColumn* pCol = &pTSchema->columns[k]; - void* var = POINTER_SHIFT(pColInfoData->pData, j * pColInfoData->info.bytes); - switch (pColInfoData->info.type) { - case TSDB_DATA_TYPE_TIMESTAMP: - if (!isStartKey) { - isStartKey = true; - tdAppendColValToRow(&rb, PRIMARYKEY_TIMESTAMP_COL_ID, TSDB_DATA_TYPE_TIMESTAMP, TD_VTYPE_NORM, var, true, - offset, k); - continue; // offset should keep 0 for next column - - } else if (colDataIsNull_s(pColInfoData, j)) { - tdAppendColValToRow(&rb, PRIMARYKEY_TIMESTAMP_COL_ID + k, TSDB_DATA_TYPE_TIMESTAMP, TD_VTYPE_NULL, NULL, - false, offset, k); - } else { - tdAppendColValToRow(&rb, PRIMARYKEY_TIMESTAMP_COL_ID + k, TSDB_DATA_TYPE_TIMESTAMP, TD_VTYPE_NORM, var, - true, offset, k); - } - break; - case TSDB_DATA_TYPE_NCHAR: - case TSDB_DATA_TYPE_VARCHAR: // TSDB_DATA_TYPE_BINARY - case TSDB_DATA_TYPE_GEOMETRY: { - if (colDataIsNull_s(pColInfoData, j)) { - tdAppendColValToRow(&rb, PRIMARYKEY_TIMESTAMP_COL_ID + k, pColInfoData->info.type, TD_VTYPE_NULL, NULL, - false, offset, k); - } else { - void* data = colDataGetData(pColInfoData, j); - tdAppendColValToRow(&rb, PRIMARYKEY_TIMESTAMP_COL_ID + k, pColInfoData->info.type, TD_VTYPE_NORM, data, - true, offset, k); - } - break; - } - case TSDB_DATA_TYPE_VARBINARY: - case TSDB_DATA_TYPE_DECIMAL: - case TSDB_DATA_TYPE_BLOB: - case TSDB_DATA_TYPE_JSON: - case TSDB_DATA_TYPE_MEDIUMBLOB: - uError("the column type %" PRIi16 " is defined but not implemented yet", pColInfoData->info.type); - break; - default: - if (pColInfoData->info.type < TSDB_DATA_TYPE_MAX && pColInfoData->info.type > TSDB_DATA_TYPE_NULL) { - if (colDataIsNull_s(pColInfoData, j)) { - tdAppendColValToRow(&rb, PRIMARYKEY_TIMESTAMP_COL_ID + k, pCol->type, TD_VTYPE_NULL, NULL, false, - offset, k); - } else if (pCol->type == pColInfoData->info.type) { - tdAppendColValToRow(&rb, PRIMARYKEY_TIMESTAMP_COL_ID + k, pCol->type, TD_VTYPE_NORM, var, true, offset, - k); - } else { - char tv[8] = {0}; - if (pColInfoData->info.type == TSDB_DATA_TYPE_FLOAT) { - float v = 0; - GET_TYPED_DATA(v, float, pColInfoData->info.type, var); - SET_TYPED_DATA(&tv, pCol->type, v); - } else if (pColInfoData->info.type == TSDB_DATA_TYPE_DOUBLE) { - double v = 0; - GET_TYPED_DATA(v, double, pColInfoData->info.type, var); - SET_TYPED_DATA(&tv, pCol->type, v); - } else if (IS_SIGNED_NUMERIC_TYPE(pColInfoData->info.type)) { - int64_t v = 0; - GET_TYPED_DATA(v, int64_t, pColInfoData->info.type, var); - SET_TYPED_DATA(&tv, pCol->type, v); - } else { - uint64_t v = 0; - GET_TYPED_DATA(v, uint64_t, pColInfoData->info.type, var); - SET_TYPED_DATA(&tv, pCol->type, v); - } - tdAppendColValToRow(&rb, PRIMARYKEY_TIMESTAMP_COL_ID + k, pCol->type, TD_VTYPE_NORM, tv, true, offset, - k); - } - } else { - uError("the column type %" PRIi16 " is undefined\n", pColInfoData->info.type); - } - break; - } - offset += TYPE_BYTES[pCol->type]; // sum/avg would convert to int64_t/uint64_t/double during aggregation - } - tdSRowEnd(&rb); - dataLen += TD_ROW_LEN(rb.pBuf); -#ifdef TD_DEBUG_PRINT_ROW - tdSRowPrint(rb.pBuf, pTSchema, __func__); -#endif - } - - ++numOfBlks; - - pSubmitBlk->dataLen = dataLen; - msgLen += pSubmitBlk->dataLen; - } - - if (numOfBlks > 0) { - (*pReq)->length = msgLen; - - (*pReq)->header.vgId = htonl(vgId); - (*pReq)->header.contLen = htonl(msgLen); - (*pReq)->length = (*pReq)->header.contLen; - (*pReq)->numOfBlocks = htonl(numOfBlks); - SSubmitBlk* blk = (SSubmitBlk*)((*pReq) + 1); - while (numOfBlks--) { - int32_t dataLen = blk->dataLen; - blk->uid = htobe64(blk->uid); - blk->suid = htobe64(blk->suid); - blk->sversion = htonl(blk->sversion); - blk->dataLen = htonl(blk->dataLen); - blk->schemaLen = htonl(blk->schemaLen); - blk->numOfRows = htonl(blk->numOfRows); - blk = (SSubmitBlk*)(blk->data + dataLen); - } - } else { - // no valid rows - taosMemoryFreeClear(*pReq); - } - - return TSDB_CODE_SUCCESS; -} -#endif - int32_t buildSubmitReqFromDataBlock(SSubmitReq2** ppReq, const SSDataBlock* pDataBlock, const STSchema* pTSchema, int64_t uid, int32_t vgId, tb_uid_t suid) { SSubmitReq2* pReq = *ppReq; @@ -2610,3 +2254,149 @@ const char* blockDecode(SSDataBlock* pBlock, const char* pData) { ASSERT(pStart - pData == dataLen); return pStart; } + +void trimDataBlock(SSDataBlock* pBlock, int32_t totalRows, const bool* pBoolList) { +// int32_t totalRows = pBlock->info.rows; + int32_t bmLen = BitmapLen(totalRows); + char* pBitmap = NULL; + int32_t maxRows = 0; + + size_t numOfCols = taosArrayGetSize(pBlock->pDataBlock); + for (int32_t i = 0; i < numOfCols; ++i) { + SColumnInfoData* pDst = taosArrayGet(pBlock->pDataBlock, i); + // it is a reserved column for scalar function, and no data in this column yet. + if (pDst->pData == NULL) { + continue; + } + + int32_t numOfRows = 0; + if (IS_VAR_DATA_TYPE(pDst->info.type)) { + int32_t j = 0; + pDst->varmeta.length = 0; + + while (j < totalRows) { + if (pBoolList[j] == 0) { + j += 1; + continue; + } + + if (colDataIsNull_var(pDst, j)) { + colDataSetNull_var(pDst, numOfRows); + } else { + // fix address sanitizer error. p1 may point to memory that will change during realloc of colDataSetVal, first copy it to p2 + char* p1 = colDataGetVarData(pDst, j); + int32_t len = 0; + if (pDst->info.type == TSDB_DATA_TYPE_JSON) { + len = getJsonValueLen(p1); + } else { + len = varDataTLen(p1); + } + char* p2 = taosMemoryMalloc(len); + memcpy(p2, p1, len); + colDataSetVal(pDst, numOfRows, p2, false); + taosMemoryFree(p2); + } + numOfRows += 1; + j += 1; + } + + if (maxRows < numOfRows) { + maxRows = numOfRows; + } + } else { + if (pBitmap == NULL) { + pBitmap = taosMemoryCalloc(1, bmLen); + } + + memcpy(pBitmap, pDst->nullbitmap, bmLen); + memset(pDst->nullbitmap, 0, bmLen); + + int32_t j = 0; + + switch (pDst->info.type) { + case TSDB_DATA_TYPE_BIGINT: + case TSDB_DATA_TYPE_UBIGINT: + case TSDB_DATA_TYPE_DOUBLE: + case TSDB_DATA_TYPE_TIMESTAMP: + while (j < totalRows) { + if (pBoolList[j] == 0) { + j += 1; + continue; + } + + if (colDataIsNull_f(pBitmap, j)) { + colDataSetNull_f(pDst->nullbitmap, numOfRows); + } else { + ((int64_t*)pDst->pData)[numOfRows] = ((int64_t*)pDst->pData)[j]; + } + numOfRows += 1; + j += 1; + } + break; + case TSDB_DATA_TYPE_FLOAT: + case TSDB_DATA_TYPE_INT: + case TSDB_DATA_TYPE_UINT: + while (j < totalRows) { + if (pBoolList[j] == 0) { + j += 1; + continue; + } + if (colDataIsNull_f(pBitmap, j)) { + colDataSetNull_f(pDst->nullbitmap, numOfRows); + } else { + ((int32_t*)pDst->pData)[numOfRows] = ((int32_t*)pDst->pData)[j]; + } + numOfRows += 1; + j += 1; + } + break; + case TSDB_DATA_TYPE_SMALLINT: + case TSDB_DATA_TYPE_USMALLINT: + while (j < totalRows) { + if (pBoolList[j] == 0) { + j += 1; + continue; + } + if (colDataIsNull_f(pBitmap, j)) { + colDataSetNull_f(pDst->nullbitmap, numOfRows); + } else { + ((int16_t*)pDst->pData)[numOfRows] = ((int16_t*)pDst->pData)[j]; + } + numOfRows += 1; + j += 1; + } + break; + case TSDB_DATA_TYPE_BOOL: + case TSDB_DATA_TYPE_TINYINT: + case TSDB_DATA_TYPE_UTINYINT: + while (j < totalRows) { + if (pBoolList[j] == 0) { + j += 1; + continue; + } + if (colDataIsNull_f(pBitmap, j)) { + colDataSetNull_f(pDst->nullbitmap, numOfRows); + } else { + ((int8_t*)pDst->pData)[numOfRows] = ((int8_t*)pDst->pData)[j]; + } + numOfRows += 1; + j += 1; + } + break; + } + } + + if (maxRows < numOfRows) { + maxRows = numOfRows; + } + } + + pBlock->info.rows = maxRows; + if (pBitmap != NULL) { + taosMemoryFree(pBitmap); + } +} + +int32_t blockGetEncodeSize(const SSDataBlock* pBlock) { + return blockDataGetSerialMetaSize(taosArrayGetSize(pBlock->pDataBlock)) + blockDataGetSize(pBlock); +} \ No newline at end of file diff --git a/source/libs/executor/src/executorInt.c b/source/libs/executor/src/executorInt.c index 4f1a0254e4..2606d0ad0c 100644 --- a/source/libs/executor/src/executorInt.c +++ b/source/libs/executor/src/executorInt.c @@ -540,151 +540,12 @@ void extractQualifiedTupleByFilterResult(SSDataBlock* pBlock, const SColumnInfoD } int8_t* pIndicator = (int8_t*)p->pData; - int32_t totalRows = pBlock->info.rows; - if (status == FILTER_RESULT_ALL_QUALIFIED) { // here nothing needs to be done } else if (status == FILTER_RESULT_NONE_QUALIFIED) { pBlock->info.rows = 0; } else { - int32_t bmLen = BitmapLen(totalRows); - char* pBitmap = NULL; - int32_t maxRows = 0; - - size_t numOfCols = taosArrayGetSize(pBlock->pDataBlock); - for (int32_t i = 0; i < numOfCols; ++i) { - SColumnInfoData* pDst = taosArrayGet(pBlock->pDataBlock, i); - // it is a reserved column for scalar function, and no data in this column yet. - if (pDst->pData == NULL) { - continue; - } - - int32_t numOfRows = 0; - if (IS_VAR_DATA_TYPE(pDst->info.type)) { - int32_t j = 0; - pDst->varmeta.length = 0; - - while (j < totalRows) { - if (pIndicator[j] == 0) { - j += 1; - continue; - } - - if (colDataIsNull_var(pDst, j)) { - colDataSetNull_var(pDst, numOfRows); - } else { - // fix address sanitizer error. p1 may point to memory that will change during realloc of colDataSetVal, first copy it to p2 - char* p1 = colDataGetVarData(pDst, j); - int32_t len = 0; - if (pDst->info.type == TSDB_DATA_TYPE_JSON) { - len = getJsonValueLen(p1); - } else { - len = varDataTLen(p1); - } - char* p2 = taosMemoryMalloc(len); - memcpy(p2, p1, len); - colDataSetVal(pDst, numOfRows, p2, false); - taosMemoryFree(p2); - } - numOfRows += 1; - j += 1; - } - - if (maxRows < numOfRows) { - maxRows = numOfRows; - } - } else { - if (pBitmap == NULL) { - pBitmap = taosMemoryCalloc(1, bmLen); - } - - memcpy(pBitmap, pDst->nullbitmap, bmLen); - memset(pDst->nullbitmap, 0, bmLen); - - int32_t j = 0; - - switch (pDst->info.type) { - case TSDB_DATA_TYPE_BIGINT: - case TSDB_DATA_TYPE_UBIGINT: - case TSDB_DATA_TYPE_DOUBLE: - case TSDB_DATA_TYPE_TIMESTAMP: - while (j < totalRows) { - if (pIndicator[j] == 0) { - j += 1; - continue; - } - - if (colDataIsNull_f(pBitmap, j)) { - colDataSetNull_f(pDst->nullbitmap, numOfRows); - } else { - ((int64_t*)pDst->pData)[numOfRows] = ((int64_t*)pDst->pData)[j]; - } - numOfRows += 1; - j += 1; - } - break; - case TSDB_DATA_TYPE_FLOAT: - case TSDB_DATA_TYPE_INT: - case TSDB_DATA_TYPE_UINT: - while (j < totalRows) { - if (pIndicator[j] == 0) { - j += 1; - continue; - } - if (colDataIsNull_f(pBitmap, j)) { - colDataSetNull_f(pDst->nullbitmap, numOfRows); - } else { - ((int32_t*)pDst->pData)[numOfRows] = ((int32_t*)pDst->pData)[j]; - } - numOfRows += 1; - j += 1; - } - break; - case TSDB_DATA_TYPE_SMALLINT: - case TSDB_DATA_TYPE_USMALLINT: - while (j < totalRows) { - if (pIndicator[j] == 0) { - j += 1; - continue; - } - if (colDataIsNull_f(pBitmap, j)) { - colDataSetNull_f(pDst->nullbitmap, numOfRows); - } else { - ((int16_t*)pDst->pData)[numOfRows] = ((int16_t*)pDst->pData)[j]; - } - numOfRows += 1; - j += 1; - } - break; - case TSDB_DATA_TYPE_BOOL: - case TSDB_DATA_TYPE_TINYINT: - case TSDB_DATA_TYPE_UTINYINT: - while (j < totalRows) { - if (pIndicator[j] == 0) { - j += 1; - continue; - } - if (colDataIsNull_f(pBitmap, j)) { - colDataSetNull_f(pDst->nullbitmap, numOfRows); - } else { - ((int8_t*)pDst->pData)[numOfRows] = ((int8_t*)pDst->pData)[j]; - } - numOfRows += 1; - j += 1; - } - break; - } - } - - if (maxRows < numOfRows) { - maxRows = numOfRows; - } - } - - pBlock->info.rows = maxRows; - if (pBitmap != NULL) { - taosMemoryFree(pBitmap); - } + trimDataBlock(pBlock, pBlock->info.rows, (bool*) pIndicator); } } diff --git a/source/libs/executor/src/scanoperator.c b/source/libs/executor/src/scanoperator.c index c892617783..4434a29870 100644 --- a/source/libs/executor/src/scanoperator.c +++ b/source/libs/executor/src/scanoperator.c @@ -485,12 +485,12 @@ int32_t addTagPseudoColumnData(SReadHandle* pHandle, const SExprInfo* pExpr, int } int32_t code = 0; + bool freeReader = false; // backup the rows int32_t backupRows = pBlock->info.rows; pBlock->info.rows = rows; - bool freeReader = false; STableCachedVal val = {0}; SMetaReader mr = {0}; @@ -1553,13 +1553,13 @@ static int32_t setBlockIntoRes(SStreamScanInfo* pInfo, const SSDataBlock* pBlock blockDataEnsureCapacity(pInfo->pRes, pBlock->info.rows); - pInfo->pRes->info.rows = pBlock->info.rows; - pInfo->pRes->info.id.uid = pBlock->info.id.uid; - pInfo->pRes->info.type = STREAM_NORMAL; - pInfo->pRes->info.version = pBlock->info.version; + pBlockInfo->rows = pBlock->info.rows; + pBlockInfo->id.uid = pBlock->info.id.uid; + pBlockInfo->type = STREAM_NORMAL; + pBlockInfo->version = pBlock->info.version; STableScanInfo* pTableScanInfo = pInfo->pTableScanOp->info; - pInfo->pRes->info.id.groupId = getTableGroupId(pTableScanInfo->base.pTableListInfo, pBlock->info.id.uid); + pBlockInfo->id.groupId = getTableGroupId(pTableScanInfo->base.pTableListInfo, pBlock->info.id.uid); // todo extract method for (int32_t i = 0; i < taosArrayGetSize(pInfo->matchInfo.pList); ++i) { @@ -1589,7 +1589,7 @@ static int32_t setBlockIntoRes(SStreamScanInfo* pInfo, const SSDataBlock* pBlock // currently only the tbname pseudo column if (pInfo->numOfPseudoExpr > 0) { int32_t code = addTagPseudoColumnData(&pInfo->readHandle, pInfo->pPseudoExpr, pInfo->numOfPseudoExpr, pInfo->pRes, - pInfo->pRes->info.rows, GET_TASKID(pTaskInfo), &pTableScanInfo->base.metaCache); + pBlockInfo->rows, GET_TASKID(pTaskInfo), &pTableScanInfo->base.metaCache); // ignore the table not exists error, since this table may have been dropped during the scan procedure. if (code != TSDB_CODE_SUCCESS && code != TSDB_CODE_PAR_TABLE_NOT_EXIST) { blockDataFreeRes((SSDataBlock*)pBlock); @@ -1606,7 +1606,6 @@ static int32_t setBlockIntoRes(SStreamScanInfo* pInfo, const SSDataBlock* pBlock pInfo->pRes->info.dataLoad = 1; blockDataUpdateTsWindow(pInfo->pRes, pInfo->primaryTsIndex); -// blockDataFreeRes((SSDataBlock*)pBlock); calBlockTbName(pInfo, pInfo->pRes); return 0; @@ -2088,11 +2087,26 @@ FETCH_NEXT_BLOCK: return pInfo->pCreateTbRes; } - // todo apply time window range filter - doCheckUpdate(pInfo, pBlockInfo->window.ekey, pBlock); doFilter(pBlock, pOperator->exprSupp.pFilterInfo, NULL); + { // do additional time window filter + STimeWindow* pWindow = &pTaskInfo->streamInfo.fillHistoryWindow; + + if (pWindow->skey != 0) { + bool* p = taosMemoryCalloc(pBlock->info.rows, sizeof(bool)); + + SColumnInfoData* pCol = taosArrayGet(pBlock->pDataBlock, pInfo->primaryTsIndex); + for(int32_t i = 0; i < pBlock->info.rows; ++i) { + int64_t* ts = (int64_t*) colDataGetData(pCol, i); + p[i] = (*ts >= pWindow->skey); + } + + trimDataBlock(pBlock, pBlock->info.rows, p); + taosMemoryFree(p); + } + } + pBlock->info.dataLoad = 1; blockDataUpdateTsWindow(pBlock, pInfo->primaryTsIndex); From 2b6ace653a81a3972298879ba7d92790152bedf8 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 8 Jun 2023 16:14:27 +0800 Subject: [PATCH 13/83] refactor: do some internal refactor. --- source/common/src/tdatablock.c | 97 +------------------------ source/dnode/vnode/src/tq/tq.c | 11 ++- source/libs/executor/src/scanoperator.c | 12 ++- source/libs/stream/src/streamRecover.c | 4 +- 4 files changed, 23 insertions(+), 101 deletions(-) diff --git a/source/common/src/tdatablock.c b/source/common/src/tdatablock.c index fe38225ac8..c4396387b1 100644 --- a/source/common/src/tdatablock.c +++ b/source/common/src/tdatablock.c @@ -122,7 +122,7 @@ int32_t colDataSetVal(SColumnInfoData* pColumnInfoData, uint32_t rowIndex, const return 0; } -int32_t colDataReserve(SColumnInfoData* pColumnInfoData, size_t newSize) { +static int32_t colDataReserve(SColumnInfoData* pColumnInfoData, size_t newSize) { if (!IS_VAR_DATA_TYPE(pColumnInfoData->info.type)) { return TSDB_CODE_SUCCESS; } @@ -1677,101 +1677,6 @@ static char* formatTimestamp(char* buf, int64_t val, int precision) { return buf; } -#if 0 -void blockDebugShowDataBlock(SSDataBlock* pBlock, const char* flag) { - SArray* dataBlocks = taosArrayInit(1, sizeof(SSDataBlock*)); - taosArrayPush(dataBlocks, &pBlock); - blockDebugShowDataBlocks(dataBlocks, flag); - taosArrayDestroy(dataBlocks); -} - -void blockDebugShowDataBlocks(const SArray* dataBlocks, const char* flag) { - char pBuf[128] = {0}; - int32_t sz = taosArrayGetSize(dataBlocks); - for (int32_t i = 0; i < sz; i++) { - SSDataBlock* pDataBlock = taosArrayGet(dataBlocks, i); - size_t numOfCols = taosArrayGetSize(pDataBlock->pDataBlock); - - int32_t rows = pDataBlock->info.rows; - printf("%s |block ver %" PRIi64 " |block type %d |child id %d|group id %" PRIu64 "\n", flag, - pDataBlock->info.version, (int32_t)pDataBlock->info.type, pDataBlock->info.childId, - pDataBlock->info.id.groupId); - for (int32_t j = 0; j < rows; j++) { - printf("%s |", flag); - for (int32_t k = 0; k < numOfCols; k++) { - SColumnInfoData* pColInfoData = taosArrayGet(pDataBlock->pDataBlock, k); - void* var = POINTER_SHIFT(pColInfoData->pData, j * pColInfoData->info.bytes); - if (k == 0) { - printf("cols:%d |", (int32_t)numOfCols); - } - if (colDataIsNull(pColInfoData, rows, j, NULL)) { - printf(" %15s |", "NULL"); - continue; - } - - switch (pColInfoData->info.type) { - case TSDB_DATA_TYPE_TIMESTAMP: - formatTimestamp(pBuf, *(uint64_t*)var, TSDB_TIME_PRECISION_MILLI); - printf(" %25s |", pBuf); - break; - case TSDB_DATA_TYPE_BOOL: - printf(" %15" PRIi8 " |", *(int8_t*)var); - break; - case TSDB_DATA_TYPE_TINYINT: - printf(" %15" PRIi8 " |", *(int8_t*)var); - break; - case TSDB_DATA_TYPE_SMALLINT: - printf(" %15" PRIi16 " |", *(int16_t*)var); - break; - case TSDB_DATA_TYPE_INT: - printf(" %15d |", *(int32_t*)var); - break; - case TSDB_DATA_TYPE_UTINYINT: - printf(" %15" PRIu8 " |", *(uint8_t*)var); - break; - case TSDB_DATA_TYPE_USMALLINT: - printf(" %15" PRIu16 " |", *(uint16_t*)var); - break; - case TSDB_DATA_TYPE_UINT: - printf(" %15u |", *(uint32_t*)var); - break; - case TSDB_DATA_TYPE_BIGINT: - printf(" %15" PRId64 " |", *(int64_t*)var); - break; - case TSDB_DATA_TYPE_UBIGINT: - printf(" %15" PRIu64 " |", *(uint64_t*)var); - break; - case TSDB_DATA_TYPE_FLOAT: - printf(" %15f |", *(float*)var); - break; - case TSDB_DATA_TYPE_DOUBLE: - printf(" %15lf |", *(double*)var); - break; - case TSDB_DATA_TYPE_VARCHAR: - case TSDB_DATA_TYPE_GEOMETRY: { - char* pData = colDataGetVarData(pColInfoData, j); - int32_t dataSize = TMIN(sizeof(pBuf) - 1, varDataLen(pData)); - memset(pBuf, 0, dataSize + 1); - strncpy(pBuf, varDataVal(pData), dataSize); - printf(" %15s |", pBuf); - } break; - case TSDB_DATA_TYPE_NCHAR: { - char* pData = colDataGetVarData(pColInfoData, j); - int32_t dataSize = TMIN(sizeof(pBuf), varDataLen(pData)); - memset(pBuf, 0, dataSize); - (void)taosUcs4ToMbs((TdUcs4*)varDataVal(pData), dataSize, pBuf); - printf(" %15s |", pBuf); - } break; - default: - break; - } - } - printf("\n"); - } - } -} -#endif - // for debug char* dumpBlockData(SSDataBlock* pDataBlock, const char* flag, char** pDataBuf) { int32_t size = 2048*1024; diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index da05e950ce..18291ac70f 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1045,12 +1045,21 @@ int32_t tqProcessTaskDeployReq(STQ* pTq, int64_t sversion, char* msg, int32_t ms pRange->window.ekey = INT64_MAX; pRange->range.minVer = 0; pRange->range.maxVer = ver; + + tqDebug("s-task:%s fill-history task exists, update stream time window:%" PRId64 " - %" PRId64 + ", ver range:%" PRId64 " - %" PRId64, + pTask->id.idStr, pRange->window.skey, pRange->window.ekey, pRange->range.minVer, pRange->range.maxVer); + } else { + SHistDataRange* pRange = &pTask->dataRange; + tqDebug("s-task:%s no associated task, stream time window:%" PRId64 " - %" PRId64 ", ver range:%" PRId64 + " - %" PRId64, + pTask->id.idStr, pRange->window.skey, pRange->window.ekey, pRange->range.minVer, pRange->range.maxVer); } streamTaskCheckDownstreamTasks(pTask); } - tqDebug("vgId:%d s-task:%s is deployed and add meta from mnd, status:%d, total:%d", vgId, pTask->id.idStr, + tqDebug("vgId:%d s-task:%s is deployed and add meta from mnd, status:%d, numOfTasks:%d", vgId, pTask->id.idStr, pTask->status.taskStatus, numOfTasks); return 0; diff --git a/source/libs/executor/src/scanoperator.c b/source/libs/executor/src/scanoperator.c index 4434a29870..12e56800aa 100644 --- a/source/libs/executor/src/scanoperator.c +++ b/source/libs/executor/src/scanoperator.c @@ -2093,16 +2093,24 @@ FETCH_NEXT_BLOCK: { // do additional time window filter STimeWindow* pWindow = &pTaskInfo->streamInfo.fillHistoryWindow; - if (pWindow->skey != 0) { + if (pWindow->skey != INT64_MIN) { bool* p = taosMemoryCalloc(pBlock->info.rows, sizeof(bool)); + bool hasUnqualified = false; SColumnInfoData* pCol = taosArrayGet(pBlock->pDataBlock, pInfo->primaryTsIndex); for(int32_t i = 0; i < pBlock->info.rows; ++i) { int64_t* ts = (int64_t*) colDataGetData(pCol, i); p[i] = (*ts >= pWindow->skey); + + if (!p[i]) { + hasUnqualified = true; + } + } + + if (hasUnqualified) { + trimDataBlock(pBlock, pBlock->info.rows, p); } - trimDataBlock(pBlock, pBlock->info.rows, p); taosMemoryFree(p); } } diff --git a/source/libs/stream/src/streamRecover.c b/source/libs/stream/src/streamRecover.c index ec52e6fbe8..f1e43df230 100644 --- a/source/libs/stream/src/streamRecover.c +++ b/source/libs/stream/src/streamRecover.c @@ -321,8 +321,8 @@ static void doCheckDownstreamStatus(SStreamTask* pTask, SStreamTask* pHTask) { pHTask->dataRange.range.minVer = 0; pHTask->dataRange.range.maxVer = pTask->chkInfo.currentVer; - qDebug("s-task:%s set the launch condition for fill history s-task:%s, window:%" PRId64 "-%" PRId64 - " verrange:%" PRId64 "-%" PRId64, + qDebug("s-task:%s set the launch condition for fill history s-task:%s, window:%" PRId64 " - %" PRId64 + " ver range:%" PRId64 " - %" PRId64, pTask->id.idStr, pHTask->id.idStr, pHTask->dataRange.window.skey, pHTask->dataRange.window.ekey, pHTask->dataRange.range.minVer, pHTask->dataRange.range.maxVer); From 6c13491428174a039eb9fa598dadd5353d8f259e Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 8 Jun 2023 16:55:41 +0800 Subject: [PATCH 14/83] refactor: do some internal refactor. --- include/libs/stream/tstream.h | 3 --- source/dnode/mnode/impl/src/mndScheduler.c | 3 +-- source/libs/stream/src/stream.c | 2 +- source/libs/stream/src/streamData.c | 20 -------------------- 4 files changed, 2 insertions(+), 26 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 1d80601178..3ce4367724 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -132,7 +132,6 @@ typedef struct { // ref data block, for delete typedef struct { int8_t type; - int64_t ver; SSDataBlock* pBlock; } SStreamRefDataBlock; @@ -207,8 +206,6 @@ void* streamQueueNextItem(SStreamQueue* pQueue); SStreamDataSubmit* streamDataSubmitNew(SPackedData* pData, int32_t type); void streamDataSubmitDestroy(SStreamDataSubmit* pDataSubmit); -SStreamDataSubmit* streamSubmitBlockClone(SStreamDataSubmit* pSubmit); - typedef struct { char* qmsg; void* pExecutor; // not applicable to encoder and decoder diff --git a/source/dnode/mnode/impl/src/mndScheduler.c b/source/dnode/mnode/impl/src/mndScheduler.c index c6d8bb5ffe..1ab11154f0 100644 --- a/source/dnode/mnode/impl/src/mndScheduler.c +++ b/source/dnode/mnode/impl/src/mndScheduler.c @@ -396,8 +396,7 @@ static int32_t doAddSourceTask(SArray* pTaskList, int8_t fillHistory, int64_t ui pTask->dataRange.window.skey = INT64_MIN; pTask->dataRange.window.ekey = 1685959190000;//taosGetTimestampMs(); - mDebug("0x%x----------------window:%"PRId64" - %"PRId64, pTask->id.taskId, pTask->dataRange.window.skey, pTask->dataRange.window.ekey); - + mDebug("s-task:0x%x set time window:%"PRId64" - %"PRId64, pTask->id.taskId, pTask->dataRange.window.skey, pTask->dataRange.window.ekey); // all the source tasks dispatch result to a single agg node. setFixedDownstreamEpInfo(pTask, pDownstreamTask); diff --git a/source/libs/stream/src/stream.c b/source/libs/stream/src/stream.c index 3befd5d55d..bfb28c0919 100644 --- a/source/libs/stream/src/stream.c +++ b/source/libs/stream/src/stream.c @@ -17,7 +17,7 @@ #include "ttimer.h" #define STREAM_TASK_INPUT_QUEUE_CAPACITY 20480 -#define STREAM_TASK_INPUT_QUEUE_CAPACITY_IN_SIZE (50) +#define STREAM_TASK_INPUT_QUEUE_CAPACITY_IN_SIZE (30) #define ONE_MB_F (1048576.0) #define QUEUE_MEM_SIZE_IN_MB(_q) (taosQueueMemorySize(_q) / ONE_MB_F) diff --git a/source/libs/stream/src/streamData.c b/source/libs/stream/src/streamData.c index 84b5eb3ab7..37923ca807 100644 --- a/source/libs/stream/src/streamData.c +++ b/source/libs/stream/src/streamData.c @@ -164,26 +164,6 @@ int32_t streamMergeSubmit(SStreamMergedSubmit* pMerged, SStreamDataSubmit* pSubm return 0; } -static FORCE_INLINE void streamDataSubmitRefInc(SStreamDataSubmit* pDataSubmit) { - atomic_add_fetch_32(pDataSubmit->dataRef, 1); -} - -SStreamDataSubmit* streamSubmitBlockClone(SStreamDataSubmit* pSubmit) { - int32_t len = 0; - if (pSubmit->type == STREAM_INPUT__DATA_SUBMIT) { - len = pSubmit->submit.msgLen; - } - - SStreamDataSubmit* pSubmitClone = taosAllocateQitem(sizeof(SStreamDataSubmit), DEF_QITEM, len); - if (pSubmitClone == NULL) { - return NULL; - } - - streamDataSubmitRefInc(pSubmit); - memcpy(pSubmitClone, pSubmit, sizeof(SStreamDataSubmit)); - return pSubmitClone; -} - SStreamQueueItem* streamMergeQueueItem(SStreamQueueItem* dst, SStreamQueueItem* pElem) { if (dst->type == STREAM_INPUT__DATA_BLOCK && pElem->type == STREAM_INPUT__DATA_BLOCK) { SStreamDataBlock* pBlock = (SStreamDataBlock*)dst; From 6726edae0c73c0c7c007c7dbffb5a758019ff46c Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 8 Jun 2023 17:54:22 +0800 Subject: [PATCH 15/83] fix(stream): add more logs. --- source/dnode/vnode/src/tq/tqSink.c | 2 +- source/libs/stream/src/tstreamFileState.c | 4 +++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/source/dnode/vnode/src/tq/tqSink.c b/source/dnode/vnode/src/tq/tqSink.c index 604b171daf..33f8a1aa6c 100644 --- a/source/dnode/vnode/src/tq/tqSink.c +++ b/source/dnode/vnode/src/tq/tqSink.c @@ -412,7 +412,7 @@ void tqSinkToTablePipeline(SStreamTask* pTask, void* vnode, int64_t ver, void* d if (k == 0) { SColumnInfoData* pColData = taosArrayGet(pDataBlock->pDataBlock, dataIndex); void* colData = colDataGetData(pColData, j); - tqDebug("tq sink pipe2, row %d, col %d ts %" PRId64, j, k, *(int64_t*)colData); + tqTrace("tq sink pipe2, row %d, col %d ts %" PRId64, j, k, *(int64_t*)colData); } if (IS_SET_NULL(pCol)) { SColVal cv = COL_VAL_NULL(pCol->colId, pCol->type); diff --git a/source/libs/stream/src/tstreamFileState.c b/source/libs/stream/src/tstreamFileState.c index bc84509728..87a0aa94fd 100644 --- a/source/libs/stream/src/tstreamFileState.c +++ b/source/libs/stream/src/tstreamFileState.c @@ -361,11 +361,13 @@ int32_t flushSnapshot(SStreamFileState* pFileState, SStreamSnapshot* pSnapshot, SStateKey sKey = {.key = *((SWinKey*)pPos->pKey), .opNum = ((SStreamState*)pFileState->pFileStore)->number}; code = streamStatePutBatch(pFileState->pFileStore, "state", batch, &sKey, pPos->pRowBuff, pFileState->rowSize, 0); - qDebug("===stream===put %" PRId64 " to disc, res %d", sKey.key.ts, code); + qDebug("===stream===put %" PRId64 " to disc, code:%d, size:%d", sKey.key.ts, code, pFileState->rowSize); } + if (streamStateGetBatchSize(batch) > 0) { code = streamStatePutBatch_rocksdb(pFileState->pFileStore, batch); } + streamStateClearBatch(batch); if (flushState) { From c9aa59cbb9c1d37b591dc6a562e62c1314673e20 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 8 Jun 2023 18:59:25 +0800 Subject: [PATCH 16/83] refactor: do some internal refactor. --- include/libs/executor/storageapi.h | 2 +- include/libs/stream/tstreamFileState.h | 3 +- source/libs/executor/src/timewindowoperator.c | 8 ++-- source/libs/stream/src/tstreamFileState.c | 39 ++++++++++++++----- 4 files changed, 37 insertions(+), 15 deletions(-) diff --git a/include/libs/executor/storageapi.h b/include/libs/executor/storageapi.h index e9340a33c3..7f3039829a 100644 --- a/include/libs/executor/storageapi.h +++ b/include/libs/executor/storageapi.h @@ -367,7 +367,7 @@ typedef struct SStateStore { SStreamStateCur* (*streamStateSessionSeekKeyCurrentNext)(SStreamState* pState, const SSessionKey* key); struct SStreamFileState* (*streamFileStateInit)(int64_t memSize, uint32_t keySize, uint32_t rowSize, - uint32_t selectRowSize, GetTsFun fp, void* pFile, TSKEY delMark); + uint32_t selectRowSize, GetTsFun fp, void* pFile, TSKEY delMark, const char*id); void (*streamFileStateDestroy)(struct SStreamFileState* pFileState); void (*streamFileStateClear)(struct SStreamFileState* pFileState); diff --git a/include/libs/stream/tstreamFileState.h b/include/libs/stream/tstreamFileState.h index 0dbacf6c9f..8496b0ea62 100644 --- a/include/libs/stream/tstreamFileState.h +++ b/include/libs/stream/tstreamFileState.h @@ -28,11 +28,10 @@ extern "C" { #endif typedef struct SStreamFileState SStreamFileState; - typedef SList SStreamSnapshot; SStreamFileState* streamFileStateInit(int64_t memSize, uint32_t keySize, uint32_t rowSize, uint32_t selectRowSize, - GetTsFun fp, void* pFile, TSKEY delMark); + GetTsFun fp, void* pFile, TSKEY delMark, const char* id); void streamFileStateDestroy(SStreamFileState* pFileState); void streamFileStateClear(SStreamFileState* pFileState); bool needClearDiskBuff(SStreamFileState* pFileState); diff --git a/source/libs/executor/src/timewindowoperator.c b/source/libs/executor/src/timewindowoperator.c index 3a83472079..4f6e7a62dc 100644 --- a/source/libs/executor/src/timewindowoperator.c +++ b/source/libs/executor/src/timewindowoperator.c @@ -2785,7 +2785,7 @@ SOperatorInfo* createStreamFinalIntervalOperatorInfo(SOperatorInfo* downstream, pInfo->pUpdatedMap = NULL; int32_t funResSize= getMaxFunResSize(&pOperator->exprSupp, numOfCols); pInfo->pState->pFileState = pAPI->stateStore.streamFileStateInit(tsStreamBufferSize, sizeof(SWinKey), pInfo->aggSup.resultRowSize, funResSize, - compareTs, pInfo->pState, pInfo->twAggSup.deleteMark); + compareTs, pInfo->pState, pInfo->twAggSup.deleteMark, GET_TASKID(pTaskInfo)); pInfo->dataVersion = 0; pInfo->statestore = pTaskInfo->storageAPI.stateStore; pInfo->recvGetAll = false; @@ -4974,8 +4974,10 @@ SOperatorInfo* createStreamIntervalOperatorInfo(SOperatorInfo* downstream, SPhys pInfo->pUpdated = NULL; pInfo->pUpdatedMap = NULL; int32_t funResSize= getMaxFunResSize(pSup, numOfCols); - pInfo->pState->pFileState = pTaskInfo->storageAPI.stateStore.streamFileStateInit(tsStreamBufferSize, sizeof(SWinKey), pInfo->aggSup.resultRowSize, funResSize, - compareTs, pInfo->pState, pInfo->twAggSup.deleteMark); + + pInfo->pState->pFileState = pTaskInfo->storageAPI.stateStore.streamFileStateInit( + tsStreamBufferSize, sizeof(SWinKey), pInfo->aggSup.resultRowSize, funResSize, compareTs, pInfo->pState, + pInfo->twAggSup.deleteMark, GET_TASKID(pTaskInfo)); setOperatorInfo(pOperator, "StreamIntervalOperator", QUERY_NODE_PHYSICAL_PLAN_STREAM_INTERVAL, true, OP_NOT_OPENED, pInfo, pTaskInfo); diff --git a/source/libs/stream/src/tstreamFileState.c b/source/libs/stream/src/tstreamFileState.c index 87a0aa94fd..77440b968b 100644 --- a/source/libs/stream/src/tstreamFileState.c +++ b/source/libs/stream/src/tstreamFileState.c @@ -43,12 +43,13 @@ struct SStreamFileState { uint64_t maxRowCount; uint64_t curRowCount; GetTsFun getTs; + char* id; }; typedef SRowBuffPos SRowBuffInfo; SStreamFileState* streamFileStateInit(int64_t memSize, uint32_t keySize, uint32_t rowSize, uint32_t selectRowSize, - GetTsFun fp, void* pFile, TSKEY delMark) { + GetTsFun fp, void* pFile, TSKEY delMark, const char* idstr) { if (memSize <= 0) { memSize = DEFAULT_MAX_STREAM_BUFFER_SIZE; } @@ -70,6 +71,7 @@ SStreamFileState* streamFileStateInit(int64_t memSize, uint32_t keySize, uint32_ if (!pFileState->usedBuffs || !pFileState->freeBuffs || !pFileState->rowBuffMap) { goto _error; } + pFileState->keyLen = keySize; pFileState->rowSize = rowSize; pFileState->selectivityRowSize = selectRowSize; @@ -81,6 +83,8 @@ SStreamFileState* streamFileStateInit(int64_t memSize, uint32_t keySize, uint32_ pFileState->deleteMark = delMark; pFileState->flushMark = INT64_MIN; pFileState->maxTs = INT64_MIN; + pFileState->id = taosStrdup(idstr); + recoverSnapshot(pFileState); return pFileState; @@ -124,6 +128,8 @@ void streamFileStateDestroy(SStreamFileState* pFileState) { if (!pFileState) { return; } + + taosMemoryFree(pFileState->id); tdListFreeP(pFileState->usedBuffs, destroyRowBuffAllPosPtr); tdListFreeP(pFileState->freeBuffs, destroyRowBuff); tSimpleHashCleanup(pFileState->rowBuffMap); @@ -177,7 +183,8 @@ void popUsedBuffs(SStreamFileState* pFileState, SStreamSnapshot* pFlushList, uin i++; } } - qInfo("do stream state flush %d rows to disck. is used: %d", listNEles(pFlushList), used); + + qInfo("stream state flush %d rows to disk. is used:%d", listNEles(pFlushList), used); } int32_t flushRowBuff(SStreamFileState* pFileState) { @@ -185,13 +192,17 @@ int32_t flushRowBuff(SStreamFileState* pFileState) { if (!pFlushList) { return TSDB_CODE_OUT_OF_MEMORY; } + uint64_t num = (uint64_t)(pFileState->curRowCount * FLUSH_RATIO); num = TMAX(num, FLUSH_NUM); popUsedBuffs(pFileState, pFlushList, num, false); + if (isListEmpty(pFlushList)) { popUsedBuffs(pFileState, pFlushList, num, true); } + flushSnapshot(pFileState, pFlushList, false); + SListIter fIter = {0}; tdListInitIter(pFlushList, &fIter, TD_LIST_FORWARD); SListNode* pNode = NULL; @@ -201,6 +212,7 @@ int32_t flushRowBuff(SStreamFileState* pFileState) { tdListAppend(pFileState->freeBuffs, &pPos->pRowBuff); pPos->pRowBuff = NULL; } + tdListFreeP(pFlushList, destroyRowBuffPosPtr); return TSDB_CODE_SUCCESS; } @@ -269,13 +281,13 @@ int32_t getRowBuff(SStreamFileState* pFileState, void* pKey, int32_t keyLen, voi TSKEY ts = pFileState->getTs(pKey); if (ts > pFileState->maxTs - pFileState->deleteMark && ts < pFileState->flushMark) { int32_t len = 0; - void* pVal = NULL; - int32_t code = streamStateGet_rocksdb(pFileState->pFileStore, pKey, &pVal, &len); + void* p = NULL; + int32_t code = streamStateGet_rocksdb(pFileState->pFileStore, pKey, &p, &len); qDebug("===stream===get %" PRId64 " from disc, res %d", ts, code); if (code == TSDB_CODE_SUCCESS) { - memcpy(pNewPos->pRowBuff, pVal, len); + memcpy(pNewPos->pRowBuff, p, len); } - taosMemoryFree(pVal); + taosMemoryFree(p); } tSimpleHashPut(pFileState->rowBuffMap, pKey, keyLen, &pNewPos, POINTER_BYTES); @@ -348,12 +360,16 @@ int32_t flushSnapshot(SStreamFileState* pFileState, SStreamSnapshot* pSnapshot, tdListInitIter(pSnapshot, &iter, TD_LIST_FORWARD); const int32_t BATCH_LIMIT = 256; - SListNode* pNode = NULL; + + int64_t st = taosGetTimestampMs(); + int32_t numOfElems = listNEles(pSnapshot); + SListNode* pNode = NULL; void* batch = streamStateCreateBatch(); while ((pNode = tdListNext(&iter)) != NULL && code == TSDB_CODE_SUCCESS) { SRowBuffPos* pPos = *(SRowBuffPos**)pNode->data; ASSERT(pPos->pRowBuff && pFileState->rowSize > 0); + if (streamStateGetBatchSize(batch) >= BATCH_LIMIT) { code = streamStatePutBatch_rocksdb(pFileState->pFileStore, batch); streamStateClearBatch(batch); @@ -361,7 +377,8 @@ int32_t flushSnapshot(SStreamFileState* pFileState, SStreamSnapshot* pSnapshot, SStateKey sKey = {.key = *((SWinKey*)pPos->pKey), .opNum = ((SStreamState*)pFileState->pFileStore)->number}; code = streamStatePutBatch(pFileState->pFileStore, "state", batch, &sKey, pPos->pRowBuff, pFileState->rowSize, 0); - qDebug("===stream===put %" PRId64 " to disc, code:%d, size:%d", sKey.key.ts, code, pFileState->rowSize); + // todo handle failure +// qDebug("===stream===put %" PRId64 " to disc, code:%d, size:%d", sKey.key.ts, code, pFileState->rowSize); } if (streamStateGetBatchSize(batch) > 0) { @@ -370,6 +387,10 @@ int32_t flushSnapshot(SStreamFileState* pFileState, SStreamSnapshot* pSnapshot, streamStateClearBatch(batch); + int64_t elapsed = taosGetTimestampMs() - st; + qDebug("%s flush to disk in batch model completed, rows:%d, batch size:%d, elapsed time:%"PRId64"ms", pFileState->id, numOfElems, + BATCH_LIMIT, elapsed); + if (flushState) { const char* taskKey = "streamFileState"; { @@ -391,8 +412,8 @@ int32_t flushSnapshot(SStreamFileState* pFileState, SStreamSnapshot* pSnapshot, } streamStatePutBatch_rocksdb(pFileState->pFileStore, batch); } - streamStateDestroyBatch(batch); + streamStateDestroyBatch(batch); return code; } From 0fd11aaefe356cf333d51130c2b4aa6065896a77 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 8 Jun 2023 19:14:36 +0800 Subject: [PATCH 17/83] other: add some logs. --- source/libs/stream/src/streamExec.c | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index a1e72015ff..b8ae8fc7fd 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -330,7 +330,8 @@ int32_t streamExecForAll(SStreamTask* pTask) { while (1) { // downstream task's input queue is blocked, stop immediately - if (streamTaskShouldPause(&pTask->status) || (pTask->outputStatus == TASK_INPUT_STATUS__BLOCKED)) { + if (streamTaskShouldPause(&pTask->status) || (pTask->outputStatus == TASK_INPUT_STATUS__BLOCKED) || + streamTaskShouldStop(&pTask->status)) { if (batchSize > 1) { break; } else { @@ -461,7 +462,7 @@ int32_t streamTryExec(SStreamTask* pTask) { // todo the task should be commit here atomic_store_8(&pTask->status.schedStatus, TASK_SCHED_STATUS__INACTIVE); - qDebug("s-task:%s exec completed", pTask->id.idStr); + qDebug("s-task:%s exec completed, status:%d", pTask->id.idStr, pTask->status.taskStatus); if (!taosQueueEmpty(pTask->inputQueue->queue) && (!streamTaskShouldStop(&pTask->status)) && (!streamTaskShouldPause(&pTask->status))) { From 63ef04593dd0d98309512d3e05d1155470a9548b Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Fri, 9 Jun 2023 09:48:18 +0800 Subject: [PATCH 18/83] fix: fix syntax error. --- source/dnode/vnode/inc/vnode.h | 2 +- source/dnode/vnode/src/meta/metaQuery.c | 10 +++++----- source/dnode/vnode/src/vnd/vnodeInitApi.c | 2 +- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/source/dnode/vnode/inc/vnode.h b/source/dnode/vnode/inc/vnode.h index e29d2f7a8f..52553fb7f2 100644 --- a/source/dnode/vnode/inc/vnode.h +++ b/source/dnode/vnode/inc/vnode.h @@ -101,7 +101,7 @@ void vnodeApplyWriteMsg(SQueueInfo *pInfo, STaosQall *qall, int32_t numOfMsgs void vnodeProposeCommitOnNeed(SVnode *pVnode, bool atExit); // meta -void metaReaderInit(SMetaReader *pReader, void *pVnode, int32_t flags, SStoreMeta* pAPI); +void _metaReaderInit(SMetaReader *pReader, void *pVnode, int32_t flags, SStoreMeta* pAPI); void metaReaderReleaseLock(SMetaReader *pReader); void metaReaderClear(SMetaReader *pReader); int32_t metaReaderGetTableEntryByUid(SMetaReader *pReader, tb_uid_t uid); diff --git a/source/dnode/vnode/src/meta/metaQuery.c b/source/dnode/vnode/src/meta/metaQuery.c index d146c7cd8c..1f34c60b5e 100644 --- a/source/dnode/vnode/src/meta/metaQuery.c +++ b/source/dnode/vnode/src/meta/metaQuery.c @@ -17,7 +17,7 @@ #include "osMemory.h" #include "tencode.h" -void metaReaderInit(SMetaReader* pReader, void* pVnode, int32_t flags, SStoreMeta* pAPI) { +void _metaReaderInit(SMetaReader* pReader, void* pVnode, int32_t flags, SStoreMeta* pAPI) { SMeta* pMeta = ((SVnode*)pVnode)->pMeta; metaReaderDoInit(pReader, pMeta, flags); pReader->pAPI = pAPI; @@ -143,7 +143,7 @@ tb_uid_t metaGetTableEntryUidByName(SMeta *pMeta, const char *name) { int metaGetTableNameByUid(void *pVnode, uint64_t uid, char *tbName) { int code = 0; SMetaReader mr = {0}; - metaReaderInit(&mr, ((SVnode*)pVnode)->pMeta, 0); + metaReaderDoInit(&mr, ((SVnode*)pVnode)->pMeta, 0); code = metaReaderGetTableEntryByUid(&mr, uid); if (code < 0) { metaReaderClear(&mr); @@ -195,7 +195,7 @@ int metaGetTableUidByName(void *pVnode, char *tbName, uint64_t *uid) { int metaGetTableTypeByName(void *pVnode, char *tbName, ETableType *tbType) { int code = 0; SMetaReader mr = {0}; - metaReaderInit(&mr, ((SVnode*)pVnode)->pMeta, 0); + metaReaderDoInit(&mr, ((SVnode*)pVnode)->pMeta, 0); code = metaGetTableEntryByName(&mr, tbName); if (code == 0) *tbType = mr.me.type; @@ -222,7 +222,7 @@ SMTbCursor *metaOpenTbCursor(void *pVnode) { } SVnode* pVnodeObj = pVnode; - metaReaderInit(&pTbCur->mr, pVnodeObj->pMeta, 0); + metaReaderDoInit(&pTbCur->mr, pVnodeObj->pMeta, 0); // tdbTbcMoveToFirst((TBC *)pTbCur->pDbc); pTbCur->pMeta = pVnodeObj->pMeta; @@ -254,7 +254,7 @@ void metaPauseTbCursor(SMTbCursor *pTbCur) { } void metaResumeTbCursor(SMTbCursor *pTbCur, int8_t first) { if (pTbCur->paused) { - metaReaderInit(&pTbCur->mr, pTbCur->pMeta, 0); + metaReaderDoInit(&pTbCur->mr, pTbCur->pMeta, 0); tdbTbcOpen(((SMeta *)pTbCur->pMeta)->pUidIdx, (TBC **)&pTbCur->pDbc, NULL); diff --git a/source/dnode/vnode/src/vnd/vnodeInitApi.c b/source/dnode/vnode/src/vnd/vnodeInitApi.c index f6462c3071..d2db6368a2 100644 --- a/source/dnode/vnode/src/vnd/vnodeInitApi.c +++ b/source/dnode/vnode/src/vnd/vnodeInitApi.c @@ -206,7 +206,7 @@ void initStateStoreAPI(SStateStore* pStore) { } void initMetaReaderAPI(SStoreMetaReader* pMetaReader) { - pMetaReader->initReader = metaReaderInit; + pMetaReader->initReader = _metaReaderInit; pMetaReader->clearReader = metaReaderClear; pMetaReader->getTableEntryByUid = metaReaderGetTableEntryByUid; From 69c9eda7af8545d9d8cbbe212634bcdd36bbab33 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Fri, 9 Jun 2023 11:37:38 +0800 Subject: [PATCH 19/83] fix(stream): fix race condition. --- include/libs/stream/tstream.h | 1 + source/dnode/mnode/impl/src/mndScheduler.c | 4 +++ source/dnode/vnode/src/tq/tq.c | 29 +++++++++++++++++++--- source/libs/stream/src/stream.c | 5 ++-- source/libs/stream/src/streamExec.c | 4 +-- 5 files changed, 34 insertions(+), 9 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 745247b425..9e2705461d 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -309,6 +309,7 @@ struct SStreamTask { STaskExec exec; SHistDataRange dataRange; SStreamId historyTaskId; + SStreamId streamTaskId; SArray* pUpstreamEpInfoList; // SArray, // children info int32_t nextCheckId; SArray* checkpointInfo; // SArray diff --git a/source/dnode/mnode/impl/src/mndScheduler.c b/source/dnode/mnode/impl/src/mndScheduler.c index 1ab11154f0..b6274b57b8 100644 --- a/source/dnode/mnode/impl/src/mndScheduler.c +++ b/source/dnode/mnode/impl/src/mndScheduler.c @@ -319,6 +319,10 @@ static void setHTasksId(SArray* pTaskList, const SArray* pHTaskList) { (*pStreamTask)->historyTaskId.taskId = (*pHTask)->id.taskId; (*pStreamTask)->historyTaskId.streamId = (*pHTask)->id.streamId; + + (*pHTask)->streamTaskId.taskId = (*pStreamTask)->id.taskId; + (*pHTask)->streamTaskId.streamId = (*pStreamTask)->id.streamId; + mDebug("s-task:0x%x related history task:0x%x", (*pStreamTask)->id.taskId, (*pHTask)->id.taskId); } } diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 18291ac70f..a7adf247c8 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1088,7 +1088,7 @@ int32_t tqProcessTaskRecover1Req(STQ* pTq, SRpcMsg* pMsg) { } // do recovery step 1 - tqDebug("s-task:%s start non-blocking recover stage(step 1) scan", pTask->id.idStr); + tqDebug("s-task:%s start history data scan stage(step 1)", pTask->id.idStr); int64_t st = taosGetTimestampMs(); streamSourceRecoverScanStep1(pTask); @@ -1100,10 +1100,31 @@ int32_t tqProcessTaskRecover1Req(STQ* pTq, SRpcMsg* pMsg) { } double el = (taosGetTimestampMs() - st) / 1000.0; - tqDebug("s-task:%s history scan stage(step 1) ended, elapsed time:%.2fs", pTask->id.idStr, el); + tqDebug("s-task:%s history data scan stage(step 1) ended, elapsed time:%.2fs", pTask->id.idStr, el); - if (pTask->info.fillHistory) { - // todo transfer the executor status, and then destroy this stream task + if (pTask->info.fillHistory) {/* + // 1. stop the related stream task, get the current scan wal version of stream task, ver1. + SStreamTask* pStreamTask = streamMetaAcquireTask(pMeta, pTask->streamTaskId.taskId); + if (pStreamTask == NULL) { + // todo handle error + } + + pStreamTask->status.taskStatus = TASK_STATUS__PAUSE; + + + // if it's an source task, extract the last version in wal. + + // 2. wait for downstream tasks to completed + + + // 3. do secondary scan of the history data scan, the time window remain, and the version range is updated to [pTask->dataRange.range.maxVer, ver1] + + + // 4. 1) transfer the ownership of executor state, 2) update the scan data range for source task. + + + // 5. resume the related stream task. +*/ } else { // todo update the chkInfo version for current task. // this task has an associated history stream task, so we need to scan wal from the end version of diff --git a/source/libs/stream/src/stream.c b/source/libs/stream/src/stream.c index bfb28c0919..510c18ab65 100644 --- a/source/libs/stream/src/stream.c +++ b/source/libs/stream/src/stream.c @@ -264,13 +264,12 @@ int32_t streamProcessDispatchRsp(SStreamTask* pTask, SStreamDispatchRsp* pRsp, i } pTask->msgInfo.retryCount = 0; - int8_t old = atomic_exchange_8(&pTask->outputStatus, pRsp->inputStatus); - ASSERT(old == TASK_OUTPUT_STATUS__WAIT); + ASSERT(pTask->outputStatus == TASK_OUTPUT_STATUS__WAIT); qDebug("s-task:%s output status is set to:%d", pTask->id.idStr, pTask->outputStatus); // the input queue of the (down stream) task that receive the output data is full, so the TASK_INPUT_STATUS_BLOCKED is rsp - if (pTask->outputStatus == TASK_INPUT_STATUS__BLOCKED) { + if (pRsp->inputStatus == TASK_INPUT_STATUS__BLOCKED) { pTask->msgInfo.blockingTs = taosGetTimestampMs(); // record the blocking start time int32_t waitDuration = 300; // 300 ms diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index b8ae8fc7fd..e4d28e7dd7 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -440,8 +440,8 @@ int32_t streamExecForAll(SStreamTask* pTask) { streamTaskExecImpl(pTask, pInput, &resSize, &totalBlocks); double el = (taosGetTimestampMs() - st) / 1000.0; - qDebug("s-task:%s batch of input blocks exec end, elapsed time:%.2fs, result size:%.2fMiB, numOfBlocks:%d", - id, el, resSize / 1048576.0, totalBlocks); + qDebug("s-task:%s batch of (%d)input blocks exec end, elapsed time:%.2fs, result size:%.2fMiB, numOfBlocks:%d", + id, batchSize, el, resSize / 1048576.0, totalBlocks); streamFreeQitem(pInput); } From 78a240be8f5e0be9c6bc7be543fe9867205c362f Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 12 Jun 2023 13:12:45 +0800 Subject: [PATCH 20/83] enh(stream): add dispatch msg. --- include/common/tmsgdef.h | 1 + include/libs/stream/tstream.h | 10 ++- source/dnode/mgmt/mgmt_vnode/src/vmHandle.c | 3 +- source/dnode/snode/src/snode.c | 2 +- source/dnode/vnode/src/inc/vnodeInt.h | 2 +- source/dnode/vnode/src/tq/tq.c | 75 ++++++++------------- source/dnode/vnode/src/vnd/vnodeSvr.c | 11 +-- source/libs/stream/src/streamDispatch.c | 10 ++- source/libs/stream/src/streamExec.c | 17 +++++ source/libs/stream/src/streamRecover.c | 70 ++++++++++++++++++- 10 files changed, 130 insertions(+), 71 deletions(-) diff --git a/include/common/tmsgdef.h b/include/common/tmsgdef.h index 1f2d597496..11756c47ba 100644 --- a/include/common/tmsgdef.h +++ b/include/common/tmsgdef.h @@ -253,6 +253,7 @@ enum { TD_DEF_MSG_TYPE(TDMT_STREAM_UNUSED1, "stream-unused1", NULL, NULL) TD_DEF_MSG_TYPE(TDMT_STREAM_RETRIEVE, "stream-retrieve", NULL, NULL) TD_DEF_MSG_TYPE(TDMT_STREAM_RECOVER_FINISH, "stream-recover-finish", NULL, NULL) + TD_DEF_MSG_TYPE(TDMT_STREAM_TRANSFER_STATE, "stream-transfer-state", NULL, NULL) TD_DEF_MSG_TYPE(TDMT_STREAM_TASK_CHECK, "stream-task-check", NULL, NULL) TD_DEF_MSG_TYPE(TDMT_STREAM_TASK_CHECKPOINT, "stream-checkpoint", NULL, NULL) TD_DEF_MSG_TYPE(TDMT_STREAM_TASK_REPORT_CHECKPOINT, "stream-report-checkpoint", NULL, NULL) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 9e2705461d..e1af12389b 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -270,6 +270,7 @@ typedef struct SStreamStatus { int8_t taskStatus; int8_t schedStatus; int8_t keepTaskStatus; + bool transferState; } SStreamStatus; typedef struct SHistDataRange { @@ -454,10 +455,10 @@ typedef struct { int64_t streamId; int32_t taskId; int32_t childId; -} SStreamRecoverFinishReq; +} SStreamRecoverFinishReq, SStreamTransferReq; -int32_t tEncodeSStreamRecoverFinishReq(SEncoder* pEncoder, const SStreamRecoverFinishReq* pReq); -int32_t tDecodeSStreamRecoverFinishReq(SDecoder* pDecoder, SStreamRecoverFinishReq* pReq); +int32_t tEncodeStreamRecoverFinishReq(SEncoder* pEncoder, const SStreamRecoverFinishReq* pReq); +int32_t tDecodeStreamRecoverFinishReq(SDecoder* pDecoder, SStreamRecoverFinishReq* pReq); typedef struct { int64_t streamId; @@ -583,6 +584,9 @@ int32_t streamSourceRecoverScanStep1(SStreamTask* pTask); int32_t streamBuildSourceRecover2Req(SStreamTask* pTask, SStreamRecoverStep2Req* pReq); int32_t streamSourceRecoverScanStep2(SStreamTask* pTask, int64_t ver); int32_t streamDispatchRecoverFinishMsg(SStreamTask* pTask); + +int32_t streamDispatchTransferStateMsg(SStreamTask* pTask); + // agg level int32_t streamAggRecoverPrepare(SStreamTask* pTask); int32_t streamProcessRecoverFinishReq(SStreamTask* pTask, int32_t childId); diff --git a/source/dnode/mgmt/mgmt_vnode/src/vmHandle.c b/source/dnode/mgmt/mgmt_vnode/src/vmHandle.c index 814a155cfb..d9081e6e43 100644 --- a/source/dnode/mgmt/mgmt_vnode/src/vmHandle.c +++ b/source/dnode/mgmt/mgmt_vnode/src/vmHandle.c @@ -728,8 +728,9 @@ SArray *vmGetMsgHandles() { if (dmSetMgmtHandle(pArray, TDMT_STREAM_RETRIEVE, vmPutMsgToStreamQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_STREAM_RETRIEVE_RSP, vmPutMsgToStreamQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_STREAM_RECOVER_FINISH, vmPutMsgToStreamQueue, 0) == NULL) goto _OVER; + if (dmSetMgmtHandle(pArray, TDMT_STREAM_TRANSFER_STATE, vmPutMsgToStreamQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_STREAM_TASK_CHECK, vmPutMsgToStreamQueue, 0) == NULL) goto _OVER; - if (dmSetMgmtHandle(pArray, TDMT_STREAM_TASK_CHECK_RSP, vmPutMsgToWriteQueue, 0) == NULL) goto _OVER; + if (dmSetMgmtHandle(pArray, TDMT_STREAM_TASK_CHECK_RSP, vmPutMsgToStreamQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_VND_STREAM_TRIGGER, vmPutMsgToStreamQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_STREAM_TASK_PAUSE, vmPutMsgToWriteQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_STREAM_TASK_RESUME, vmPutMsgToWriteQueue, 0) == NULL) goto _OVER; diff --git a/source/dnode/snode/src/snode.c b/source/dnode/snode/src/snode.c index cf1481a113..41fb50a0df 100644 --- a/source/dnode/snode/src/snode.c +++ b/source/dnode/snode/src/snode.c @@ -277,7 +277,7 @@ int32_t sndProcessTaskRecoverFinishReq(SSnode *pSnode, SRpcMsg *pMsg) { SDecoder decoder; tDecoderInit(&decoder, msg, msgLen); - tDecodeSStreamRecoverFinishReq(&decoder, &req); + tDecodeStreamRecoverFinishReq(&decoder, &req); tDecoderClear(&decoder); // find task diff --git a/source/dnode/vnode/src/inc/vnodeInt.h b/source/dnode/vnode/src/inc/vnodeInt.h index 635893fee2..57e7ad8abf 100644 --- a/source/dnode/vnode/src/inc/vnodeInt.h +++ b/source/dnode/vnode/src/inc/vnodeInt.h @@ -245,7 +245,7 @@ int32_t tqProcessTaskDispatchRsp(STQ* pTq, SRpcMsg* pMsg); int32_t tqProcessTaskRetrieveReq(STQ* pTq, SRpcMsg* pMsg); int32_t tqProcessTaskRetrieveRsp(STQ* pTq, SRpcMsg* pMsg); int32_t tqProcessTaskRecover1Req(STQ* pTq, SRpcMsg* pMsg); -int32_t tqProcessTaskRecover2Req(STQ* pTq, int64_t version, char* msg, int32_t msgLen); +int32_t tqProcessTaskTransferStateReq(STQ* pTq, int64_t version, char* msg, int32_t msgLen); int32_t tqProcessTaskRecoverFinishReq(STQ* pTq, SRpcMsg* pMsg); int32_t tqProcessTaskRecoverFinishRsp(STQ* pTq, SRpcMsg* pMsg); int32_t tqCheckLogInWal(STQ* pTq, int64_t version); diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index a7adf247c8..71e28c3bad 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1102,29 +1102,35 @@ int32_t tqProcessTaskRecover1Req(STQ* pTq, SRpcMsg* pMsg) { double el = (taosGetTimestampMs() - st) / 1000.0; tqDebug("s-task:%s history data scan stage(step 1) ended, elapsed time:%.2fs", pTask->id.idStr, el); - if (pTask->info.fillHistory) {/* - // 1. stop the related stream task, get the current scan wal version of stream task, ver1. + if (pTask->info.fillHistory) { + // 1. stop the related stream task, get the current scan wal version of stream task, ver. SStreamTask* pStreamTask = streamMetaAcquireTask(pMeta, pTask->streamTaskId.taskId); if (pStreamTask == NULL) { // todo handle error } + // todo here we should use another flag to avoid user resume the task pStreamTask->status.taskStatus = TASK_STATUS__PAUSE; - // if it's an source task, extract the last version in wal. + int64_t ver = pTask->dataRange.range.maxVer; + int64_t latestVer = walReaderGetCurrentVer(pStreamTask->exec.pWalReader); + ASSERT(latestVer >= ver); + ver = latestVer; - // 2. wait for downstream tasks to completed + // 2. do secondary scan of the history data, the time window remain, and the version range is updated to [pTask->dataRange.range.maxVer, ver1] - // 3. do secondary scan of the history data scan, the time window remain, and the version range is updated to [pTask->dataRange.range.maxVer, ver1] - + // 3. notify the downstream tasks to transfer executor state after handle all history blocks. + code = streamDispatchTransferStateMsg(pTask); + if (code != TSDB_CODE_SUCCESS) { + // todo handle error + } // 4. 1) transfer the ownership of executor state, 2) update the scan data range for source task. - - // 5. resume the related stream task. -*/ + + streamMetaReleaseTask(pMeta, pTask); } else { // todo update the chkInfo version for current task. // this task has an associated history stream task, so we need to scan wal from the end version of @@ -1138,53 +1144,23 @@ int32_t tqProcessTaskRecover1Req(STQ* pTq, SRpcMsg* pMsg) { return code; } -#if 0 - // build msg to launch next step - SStreamRecoverStep2Req req; - code = streamBuildSourceRecover2Req(pTask, &req); - if (code < 0) { - streamMetaReleaseTask(pTq->pStreamMeta, pTask); - return -1; - } - - streamMetaReleaseTask(pTq->pStreamMeta, pTask); - if (atomic_load_8(&pTask->status.taskStatus) == TASK_STATUS__DROPPING) { - return 0; - } - - // serialize msg - int32_t len = sizeof(SStreamRecoverStep1Req); - - void* serializedReq = rpcMallocCont(len); - if (serializedReq == NULL) { - tqError("s-task:%s failed to prepare the step2 stage, out of memory", pTask->id.idStr); - return -1; - } - - memcpy(serializedReq, &req, len); - - // dispatch msg - tqDebug("s-task:%s start recover block stage", pTask->id.idStr); - - SRpcMsg rpcMsg = { - .code = 0, .contLen = len, .msgType = TDMT_VND_STREAM_RECOVER_BLOCKING_STAGE, .pCont = serializedReq}; - tmsgPutToQueue(&pTq->pVnode->msgCb, WRITE_QUEUE, &rpcMsg); -#endif - return 0; } -int32_t tqProcessTaskRecover2Req(STQ* pTq, int64_t sversion, char* msg, int32_t msgLen) { - int32_t code = 0; - - SStreamRecoverStep2Req* pReq = (SStreamRecoverStep2Req*)msg; +// notify the downstream tasks to transfer executor state after handle all history blocks. +int32_t tqProcessTaskTransferStateReq(STQ* pTq, int64_t sversion, char* msg, int32_t msgLen) { + SStreamTransferReq* pReq = (SStreamTransferReq*)msg; SStreamTask* pTask = streamMetaAcquireTask(pTq->pStreamMeta, pReq->taskId); if (pTask == NULL) { return -1; } - // do recovery step 2 + ASSERT(pTask->streamTaskId.taskId != 0); + pTask->status.transferState = true; // persistent data? + +#if 0 + // do check if current task handle all data in the input queue int64_t st = taosGetTimestampMs(); tqDebug("s-task:%s start step2 recover, ts:%" PRId64, pTask->id.idStr, st); @@ -1231,8 +1207,11 @@ int32_t tqProcessTaskRecover2Req(STQ* pTq, int64_t sversion, char* msg, int32_t atomic_store_8(&pTask->info.fillHistory, 0); streamMetaSaveTask(pTq->pStreamMeta, pTask); +#endif + streamSchedExec(pTask); streamMetaReleaseTask(pTq->pStreamMeta, pTask); + return 0; } @@ -1245,7 +1224,7 @@ int32_t tqProcessTaskRecoverFinishReq(STQ* pTq, SRpcMsg* pMsg) { SDecoder decoder; tDecoderInit(&decoder, (uint8_t*)msg, msgLen); - tDecodeSStreamRecoverFinishReq(&decoder, &req); + tDecodeStreamRecoverFinishReq(&decoder, &req); tDecoderClear(&decoder); // find task diff --git a/source/dnode/vnode/src/vnd/vnodeSvr.c b/source/dnode/vnode/src/vnd/vnodeSvr.c index 3a806ff0de..47b899aa3d 100644 --- a/source/dnode/vnode/src/vnd/vnodeSvr.c +++ b/source/dnode/vnode/src/vnd/vnodeSvr.c @@ -326,10 +326,6 @@ int32_t vnodeProcessWriteMsg(SVnode *pVnode, SRpcMsg *pMsg, int64_t ver, SRpcMsg if (!syncUtilUserCommit(pMsg->msgType)) goto _exit; - if (pMsg->msgType == TDMT_VND_STREAM_RECOVER_BLOCKING_STAGE || pMsg->msgType == TDMT_STREAM_TASK_CHECK_RSP) { - if (tqCheckLogInWal(pVnode->pTq, ver)) return 0; - } - // skip header pReq = POINTER_SHIFT(pMsg->pCont, sizeof(SMsgHead)); len = pMsg->contLen - sizeof(SMsgHead); @@ -425,11 +421,6 @@ int32_t vnodeProcessWriteMsg(SVnode *pVnode, SRpcMsg *pMsg, int64_t ver, SRpcMsg goto _err; } } break; - case TDMT_VND_STREAM_RECOVER_BLOCKING_STAGE: { - if (tqProcessTaskRecover2Req(pVnode->pTq, ver, pMsg->pCont, pMsg->contLen) < 0) { - goto _err; - } - } break; case TDMT_STREAM_TASK_CHECK_RSP: { if (tqProcessStreamTaskCheckRsp(pVnode->pTq, ver, pReq, len) < 0) { goto _err; @@ -593,6 +584,8 @@ int32_t vnodeProcessStreamMsg(SVnode *pVnode, SRpcMsg *pMsg, SQueueInfo *pInfo) return tqProcessTaskRetrieveRsp(pVnode->pTq, pMsg); case TDMT_VND_STREAM_RECOVER_NONBLOCKING_STAGE: return tqProcessTaskRecover1Req(pVnode->pTq, pMsg); + case TDMT_VND_STREAM_RECOVER_BLOCKING_STAGE: + return tqProcessTaskTransferStateReq(pVnode->pTq, 0, pMsg->pCont, pMsg->contLen); case TDMT_STREAM_RECOVER_FINISH: return tqProcessTaskRecoverFinishReq(pVnode->pTq, pMsg); case TDMT_STREAM_RECOVER_FINISH_RSP: diff --git a/source/libs/stream/src/streamDispatch.c b/source/libs/stream/src/streamDispatch.c index c01a95b288..c866e2cc21 100644 --- a/source/libs/stream/src/streamDispatch.c +++ b/source/libs/stream/src/streamDispatch.c @@ -267,8 +267,8 @@ int32_t streamDispatchCheckMsg(SStreamTask* pTask, const SStreamTaskCheckReq* pR msg.pCont = buf; msg.msgType = TDMT_STREAM_TASK_CHECK; - qDebug("s-task:%s dispatch check msg to s-task:%" PRIx64 ":0x%x (vgId:%d)", pTask->id.idStr, - pReq->streamId, pReq->downstreamTaskId, nodeId); + qDebug("s-task:%s (level:%d) dispatch check msg to s-task:%" PRIx64 ":0x%x (vgId:%d)", pTask->id.idStr, + pTask->info.taskLevel, pReq->streamId, pReq->downstreamTaskId, nodeId); tmsgSendReq(pEpSet, &msg); return 0; @@ -281,7 +281,7 @@ int32_t streamDoDispatchRecoverFinishMsg(SStreamTask* pTask, const SStreamRecove SRpcMsg msg = {0}; int32_t tlen; - tEncodeSize(tEncodeSStreamRecoverFinishReq, pReq, tlen, code); + tEncodeSize(tEncodeStreamRecoverFinishReq, pReq, tlen, code); if (code < 0) { return -1; } @@ -297,7 +297,7 @@ int32_t streamDoDispatchRecoverFinishMsg(SStreamTask* pTask, const SStreamRecove SEncoder encoder; tEncoderInit(&encoder, abuf, tlen); - if ((code = tEncodeSStreamRecoverFinishReq(&encoder, pReq)) < 0) { + if ((code = tEncodeStreamRecoverFinishReq(&encoder, pReq)) < 0) { if (buf) { rpcFreeCont(buf); } @@ -385,8 +385,6 @@ int32_t streamSearchAndAddBlock(SStreamTask* pTask, SStreamDispatchReq* pReqs, S snprintf(ctbName, TSDB_TABLE_NAME_LEN, "%s.%s", pTask->shuffleDispatcher.dbInfo.db, pDataBlock->info.parTbName); } - SArray* vgInfo = pTask->shuffleDispatcher.dbInfo.pVgroupInfos; - /*uint32_t hashValue = MurmurHash3_32(ctbName, strlen(ctbName));*/ SUseDbRsp* pDbInfo = &pTask->shuffleDispatcher.dbInfo; hashValue = diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index e4d28e7dd7..5264a5f043 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -386,6 +386,23 @@ int32_t streamExecForAll(SStreamTask* pTask) { } if (pInput == NULL) { + if (pTask->info.fillHistory && pTask->status.transferState) { + // todo transfer task state here + SStreamTask* pStreamTask = streamMetaAcquireTask(pTask->pMeta, pTask->streamTaskId.taskId); + ASSERT(pStreamTask != NULL && pStreamTask->historyTaskId.taskId == pTask->id.taskId); + + ASSERT(pStreamTask->status.taskStatus == STREAM_STATUS__PAUSE); + + // update the scan data range for source task. + + + streamSetStatusNormal(pStreamTask); + streamSchedExec(pStreamTask); + + streamMetaReleaseTask(pTask->pMeta, pStreamTask); + + // todo set the task with specified status, to avoid execute this process again + } break; } diff --git a/source/libs/stream/src/streamRecover.c b/source/libs/stream/src/streamRecover.c index f1e43df230..9efe6fec42 100644 --- a/source/libs/stream/src/streamRecover.c +++ b/source/libs/stream/src/streamRecover.c @@ -285,6 +285,72 @@ int32_t streamDispatchRecoverFinishMsg(SStreamTask* pTask) { return 0; } +static int32_t doDispatchTransferMsg(SStreamTask* pTask, const SStreamTransferReq* pReq, int32_t vgId, SEpSet* pEpSet) { + void* buf = NULL; + int32_t code = -1; + SRpcMsg msg = {0}; + + int32_t tlen; + tEncodeSize(tEncodeStreamRecoverFinishReq, pReq, tlen, code); + if (code < 0) { + return -1; + } + + buf = rpcMallocCont(sizeof(SMsgHead) + tlen); + if (buf == NULL) { + terrno = TSDB_CODE_OUT_OF_MEMORY; + return -1; + } + + ((SMsgHead*)buf)->vgId = htonl(vgId); + void* abuf = POINTER_SHIFT(buf, sizeof(SMsgHead)); + + SEncoder encoder; + tEncoderInit(&encoder, abuf, tlen); + if ((code = tEncodeStreamRecoverFinishReq(&encoder, pReq)) < 0) { + if (buf) { + rpcFreeCont(buf); + } + return code; + } + + tEncoderClear(&encoder); + + msg.contLen = tlen + sizeof(SMsgHead); + msg.pCont = buf; + msg.msgType = TDMT_STREAM_TRANSFER_STATE; + msg.info.noResp = 1; + + tmsgSendReq(pEpSet, &msg); + qDebug("s-task:%s dispatch transfer state msg to taskId:0x%x (vgId:%d)", pTask->id.idStr, pReq->taskId, vgId); + + return 0; +} + +int32_t streamDispatchTransferStateMsg(SStreamTask* pTask) { + SStreamTransferReq req = { .streamId = pTask->id.streamId, .childId = pTask->info.selfChildId }; + + // serialize + if (pTask->outputType == TASK_OUTPUT__FIXED_DISPATCH) { + qDebug("s-task:%s send recover finish msg to downstream (fix-dispatch) to taskId:0x%x, status:%d", pTask->id.idStr, + pTask->fixedEpDispatcher.taskId, pTask->status.taskStatus); + + req.taskId = pTask->fixedEpDispatcher.taskId; + doDispatchTransferMsg(pTask, &req, pTask->fixedEpDispatcher.nodeId, &pTask->fixedEpDispatcher.epSet); + } else if (pTask->outputType == TASK_OUTPUT__SHUFFLE_DISPATCH) { + SArray* vgInfo = pTask->shuffleDispatcher.dbInfo.pVgroupInfos; + + int32_t numOfVgs = taosArrayGetSize(vgInfo); + for (int32_t i = 0; i < numOfVgs; i++) { + SVgroupInfo* pVgInfo = taosArrayGet(vgInfo, i); + req.taskId = pVgInfo->taskId; + doDispatchTransferMsg(pTask, &req, pVgInfo->vgId, &pVgInfo->epSet); + } + } + + return 0; +} + // agg int32_t streamAggRecoverPrepare(SStreamTask* pTask) { pTask->numOfWaitingUpstream = taosArrayGetSize(pTask->pUpstreamEpInfoList); @@ -465,7 +531,7 @@ int32_t tDecodeSStreamTaskCheckRsp(SDecoder* pDecoder, SStreamTaskCheckRsp* pRsp return 0; } -int32_t tEncodeSStreamRecoverFinishReq(SEncoder* pEncoder, const SStreamRecoverFinishReq* pReq) { +int32_t tEncodeStreamRecoverFinishReq(SEncoder* pEncoder, const SStreamRecoverFinishReq* pReq) { if (tStartEncode(pEncoder) < 0) return -1; if (tEncodeI64(pEncoder, pReq->streamId) < 0) return -1; if (tEncodeI32(pEncoder, pReq->taskId) < 0) return -1; @@ -473,7 +539,7 @@ int32_t tEncodeSStreamRecoverFinishReq(SEncoder* pEncoder, const SStreamRecoverF tEndEncode(pEncoder); return pEncoder->pos; } -int32_t tDecodeSStreamRecoverFinishReq(SDecoder* pDecoder, SStreamRecoverFinishReq* pReq) { +int32_t tDecodeStreamRecoverFinishReq(SDecoder* pDecoder, SStreamRecoverFinishReq* pReq) { if (tStartDecode(pDecoder) < 0) return -1; if (tDecodeI64(pDecoder, &pReq->streamId) < 0) return -1; if (tDecodeI32(pDecoder, &pReq->taskId) < 0) return -1; From fb24ed161d896fad08ffc18b25f920ba71c3650d Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 12 Jun 2023 22:48:23 +0800 Subject: [PATCH 21/83] enh(stream): set correct task status. --- include/common/tmsgdef.h | 5 +++-- include/libs/stream/tstream.h | 6 +++-- source/dnode/vnode/src/inc/vnodeInt.h | 2 +- source/dnode/vnode/src/tq/tq.c | 31 +++++++++++++++++++------- source/dnode/vnode/src/tq/tqRestore.c | 2 +- source/dnode/vnode/src/vnd/vnodeSvr.c | 19 +++++++--------- source/libs/stream/src/stream.c | 2 ++ source/libs/stream/src/streamExec.c | 18 +++++++++------ source/libs/stream/src/streamRecover.c | 10 +++++---- source/libs/stream/src/streamTask.c | 6 +++++ 10 files changed, 65 insertions(+), 36 deletions(-) diff --git a/include/common/tmsgdef.h b/include/common/tmsgdef.h index 11756c47ba..3103b6fd76 100644 --- a/include/common/tmsgdef.h +++ b/include/common/tmsgdef.h @@ -252,6 +252,7 @@ enum { TD_DEF_MSG_TYPE(TDMT_STREAM_TASK_DISPATCH, "stream-task-dispatch", NULL, NULL) TD_DEF_MSG_TYPE(TDMT_STREAM_UNUSED1, "stream-unused1", NULL, NULL) TD_DEF_MSG_TYPE(TDMT_STREAM_RETRIEVE, "stream-retrieve", NULL, NULL) + TD_DEF_MSG_TYPE(TDMT_STREAM_SCAN_HISTORY, "stream-scan-history", NULL, NULL) TD_DEF_MSG_TYPE(TDMT_STREAM_RECOVER_FINISH, "stream-recover-finish", NULL, NULL) TD_DEF_MSG_TYPE(TDMT_STREAM_TRANSFER_STATE, "stream-transfer-state", NULL, NULL) TD_DEF_MSG_TYPE(TDMT_STREAM_TASK_CHECK, "stream-task-check", NULL, NULL) @@ -298,8 +299,8 @@ enum { TD_NEW_MSG_SEG(TDMT_VND_STREAM_MSG) TD_DEF_MSG_TYPE(TDMT_VND_STREAM_TRIGGER, "vnode-stream-trigger", NULL, NULL) - TD_DEF_MSG_TYPE(TDMT_VND_STREAM_RECOVER_NONBLOCKING_STAGE, "vnode-stream-recover1", NULL, NULL) - TD_DEF_MSG_TYPE(TDMT_VND_STREAM_RECOVER_BLOCKING_STAGE, "vnode-stream-recover2", NULL, NULL) + TD_DEF_MSG_TYPE(TDMT_VND_STREAM_SCAN_HISTORY, "vnode-stream-scan-history", NULL, NULL) + TD_DEF_MSG_TYPE(TDMT_VND_STREAM_TRANSFER_STATE, "vnode-stream-transfer-state", NULL, NULL) TD_DEF_MSG_TYPE(TDMT_VND_STREAM_CHECK_POINT_SOURCE, "vnode-stream-checkpoint-source", NULL, NULL) TD_DEF_MSG_TYPE(TDMT_VND_STREAM_MAX_MSG, "vnd-stream-max", NULL, NULL) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index e1af12389b..63af587794 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -45,8 +45,8 @@ enum { TASK_STATUS__FAIL, TASK_STATUS__STOP, TASK_STATUS__WAIT_DOWNSTREAM, - TASK_STATUS__RECOVER_PREPARE, - TASK_STATUS__RECOVER1, + TASK_STATUS__SCAN_HISTORY_PREPARE, + TASK_STATUS__HALT, // stream task halt to wait for the secondary scan history, this status is invisible for user TASK_STATUS__PAUSE, }; @@ -577,6 +577,8 @@ int32_t streamTaskScanHistoryDataComplete(SStreamTask* pTask); int32_t streamSetParamForRecover(SStreamTask* pTask); int32_t streamRestoreParam(SStreamTask* pTask); int32_t streamSetStatusNormal(SStreamTask* pTask); +const char* streamGetTaskStatusStr(int32_t status); + // source level int32_t streamSourceRecoverPrepareStep1(SStreamTask* pTask, SVersionRange *pVerRange, STimeWindow* pWindow); int32_t streamBuildSourceRecover1Req(SStreamTask* pTask, SStreamRecoverStep1Req* pReq); diff --git a/source/dnode/vnode/src/inc/vnodeInt.h b/source/dnode/vnode/src/inc/vnodeInt.h index 57e7ad8abf..cd97bd5753 100644 --- a/source/dnode/vnode/src/inc/vnodeInt.h +++ b/source/dnode/vnode/src/inc/vnodeInt.h @@ -244,7 +244,7 @@ int32_t tqProcessTaskDispatchReq(STQ* pTq, SRpcMsg* pMsg, bool exec); int32_t tqProcessTaskDispatchRsp(STQ* pTq, SRpcMsg* pMsg); int32_t tqProcessTaskRetrieveReq(STQ* pTq, SRpcMsg* pMsg); int32_t tqProcessTaskRetrieveRsp(STQ* pTq, SRpcMsg* pMsg); -int32_t tqProcessTaskRecover1Req(STQ* pTq, SRpcMsg* pMsg); +int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg); int32_t tqProcessTaskTransferStateReq(STQ* pTq, int64_t version, char* msg, int32_t msgLen); int32_t tqProcessTaskRecoverFinishReq(STQ* pTq, SRpcMsg* pMsg); int32_t tqProcessTaskRecoverFinishRsp(STQ* pTq, SRpcMsg* pMsg); diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 71e28c3bad..93b078ad7d 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1065,7 +1065,7 @@ int32_t tqProcessTaskDeployReq(STQ* pTq, int64_t sversion, char* msg, int32_t ms return 0; } -int32_t tqProcessTaskRecover1Req(STQ* pTq, SRpcMsg* pMsg) { +int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { int32_t code = TSDB_CODE_SUCCESS; char* msg = pMsg->pCont; int32_t msgLen = pMsg->contLen; @@ -1091,10 +1091,17 @@ int32_t tqProcessTaskRecover1Req(STQ* pTq, SRpcMsg* pMsg) { tqDebug("s-task:%s start history data scan stage(step 1)", pTask->id.idStr); int64_t st = taosGetTimestampMs(); + // todo set the correct status flag + int8_t schedStatus = atomic_val_compare_exchange_8(&pTask->status.schedStatus, TASK_SCHED_STATUS__INACTIVE, + TASK_SCHED_STATUS__WAITING); + if (schedStatus != TASK_SCHED_STATUS__INACTIVE) { + ASSERT(0); + return 0; + } + streamSourceRecoverScanStep1(pTask); if (atomic_load_8(&pTask->status.taskStatus) == TASK_STATUS__DROPPING) { tqDebug("s-task:%s is dropped, abort recover in step1", pTask->id.idStr); - streamMetaReleaseTask(pMeta, pTask); return 0; } @@ -1109,19 +1116,23 @@ int32_t tqProcessTaskRecover1Req(STQ* pTq, SRpcMsg* pMsg) { // todo handle error } - // todo here we should use another flag to avoid user resume the task - pStreamTask->status.taskStatus = TASK_STATUS__PAUSE; + pStreamTask->status.taskStatus = TASK_STATUS__HALT; // if it's an source task, extract the last version in wal. int64_t ver = pTask->dataRange.range.maxVer; int64_t latestVer = walReaderGetCurrentVer(pStreamTask->exec.pWalReader); - ASSERT(latestVer >= ver); - ver = latestVer; + if (latestVer >= ver) { + ver = latestVer; + } else { + ASSERT(latestVer == -1); + } // 2. do secondary scan of the history data, the time window remain, and the version range is updated to [pTask->dataRange.range.maxVer, ver1] // 3. notify the downstream tasks to transfer executor state after handle all history blocks. + pTask->status.transferState = true; + code = streamDispatchTransferStateMsg(pTask); if (code != TSDB_CODE_SUCCESS) { // todo handle error @@ -1129,7 +1140,9 @@ int32_t tqProcessTaskRecover1Req(STQ* pTq, SRpcMsg* pMsg) { // 4. 1) transfer the ownership of executor state, 2) update the scan data range for source task. // 5. resume the related stream task. + streamTryExec(pTask); + streamMetaReleaseTask(pMeta, pStreamTask); streamMetaReleaseTask(pMeta, pTask); } else { // todo update the chkInfo version for current task. @@ -1318,10 +1331,12 @@ int32_t tqProcessTaskRunReq(STQ* pTq, SRpcMsg* pMsg) { pTask->chkInfo.version); streamProcessRunReq(pTask); } else { - if (streamTaskShouldPause(&pTask->status)) { + if (streamTaskShouldPause(&pTask->status) || (pTask->status.taskStatus == TASK_STATUS__HALT)) { atomic_store_8(&pTask->status.schedStatus, TASK_SCHED_STATUS__INACTIVE); } - tqDebug("vgId:%d s-task:%s ignore run req since not in ready state", vgId, pTask->id.idStr); + + tqDebug("vgId:%d s-task:%s ignore run req since not in ready state, status:%s, sched-status:%d", vgId, + pTask->id.idStr, streamGetTaskStatusStr(pTask->status.taskStatus), pTask->status.schedStatus); } streamMetaReleaseTask(pTq->pStreamMeta, pTask); diff --git a/source/dnode/vnode/src/tq/tqRestore.c b/source/dnode/vnode/src/tq/tqRestore.c index cafb64e44a..bf560a3fb9 100644 --- a/source/dnode/vnode/src/tq/tqRestore.c +++ b/source/dnode/vnode/src/tq/tqRestore.c @@ -135,7 +135,7 @@ int32_t createStreamTaskRunReq(SStreamMeta* pStreamMeta, bool* pScanIdle) { continue; } - if (streamTaskShouldStop(&pTask->status) || status == TASK_STATUS__RECOVER_PREPARE || + if (streamTaskShouldStop(&pTask->status) || status == TASK_STATUS__SCAN_HISTORY_PREPARE || status == TASK_STATUS__WAIT_DOWNSTREAM || streamTaskShouldPause(&pTask->status)) { tqDebug("s-task:%s not ready for new submit block from wal, status:%d", pTask->id.idStr, status); streamMetaReleaseTask(pStreamMeta, pTask); diff --git a/source/dnode/vnode/src/vnd/vnodeSvr.c b/source/dnode/vnode/src/vnd/vnodeSvr.c index 47b899aa3d..99957431f8 100644 --- a/source/dnode/vnode/src/vnd/vnodeSvr.c +++ b/source/dnode/vnode/src/vnd/vnodeSvr.c @@ -421,11 +421,6 @@ int32_t vnodeProcessWriteMsg(SVnode *pVnode, SRpcMsg *pMsg, int64_t ver, SRpcMsg goto _err; } } break; - case TDMT_STREAM_TASK_CHECK_RSP: { - if (tqProcessStreamTaskCheckRsp(pVnode->pTq, ver, pReq, len) < 0) { - goto _err; - } - } break; case TDMT_VND_ALTER_CONFIRM: needCommit = pVnode->config.hashChange; if (vnodeProcessAlterConfirmReq(pVnode, ver, pReq, len, pRsp) < 0) { @@ -574,24 +569,26 @@ int32_t vnodeProcessStreamMsg(SVnode *pVnode, SRpcMsg *pMsg, SQueueInfo *pInfo) return tqProcessTaskRunReq(pVnode->pTq, pMsg); case TDMT_STREAM_TASK_DISPATCH: return tqProcessTaskDispatchReq(pVnode->pTq, pMsg, true); - case TDMT_STREAM_TASK_CHECK: - return tqProcessStreamTaskCheckReq(pVnode->pTq, pMsg); case TDMT_STREAM_TASK_DISPATCH_RSP: return tqProcessTaskDispatchRsp(pVnode->pTq, pMsg); + case TDMT_STREAM_TASK_CHECK: + return tqProcessStreamTaskCheckReq(pVnode->pTq, pMsg); + case TDMT_STREAM_TASK_CHECK_RSP: + return tqProcessStreamTaskCheckRsp(pVnode->pTq, 0, pMsg->pCont, pMsg->contLen); case TDMT_STREAM_RETRIEVE: return tqProcessTaskRetrieveReq(pVnode->pTq, pMsg); case TDMT_STREAM_RETRIEVE_RSP: return tqProcessTaskRetrieveRsp(pVnode->pTq, pMsg); - case TDMT_VND_STREAM_RECOVER_NONBLOCKING_STAGE: - return tqProcessTaskRecover1Req(pVnode->pTq, pMsg); - case TDMT_VND_STREAM_RECOVER_BLOCKING_STAGE: + case TDMT_VND_STREAM_SCAN_HISTORY: + return tqProcessTaskScanHistory(pVnode->pTq, pMsg); + case TDMT_VND_STREAM_TRANSFER_STATE: return tqProcessTaskTransferStateReq(pVnode->pTq, 0, pMsg->pCont, pMsg->contLen); case TDMT_STREAM_RECOVER_FINISH: return tqProcessTaskRecoverFinishReq(pVnode->pTq, pMsg); case TDMT_STREAM_RECOVER_FINISH_RSP: return tqProcessTaskRecoverFinishRsp(pVnode->pTq, pMsg); default: - vError("unknown msg type:%d in fetch queue", pMsg->msgType); + vError("unknown msg type:%d in stream queue", pMsg->msgType); return TSDB_CODE_APP_ERROR; } } diff --git a/source/libs/stream/src/stream.c b/source/libs/stream/src/stream.c index 510c18ab65..e93c280d62 100644 --- a/source/libs/stream/src/stream.c +++ b/source/libs/stream/src/stream.c @@ -117,6 +117,8 @@ int32_t streamSchedExec(SStreamTask* pTask) { SRpcMsg msg = {.msgType = TDMT_STREAM_TASK_RUN, .pCont = pRunReq, .contLen = sizeof(SStreamTaskRunReq)}; tmsgPutToQueue(pTask->pMsgCb, STREAM_QUEUE, &msg); qDebug("trigger to run s-task:%s", pTask->id.idStr); + } else { + qDebug("s-task:%s not launch task since sched status:%d", pTask->id.idStr, pTask->status.schedStatus); } return 0; diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index 5264a5f043..688491fbdb 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -388,20 +388,23 @@ int32_t streamExecForAll(SStreamTask* pTask) { if (pInput == NULL) { if (pTask->info.fillHistory && pTask->status.transferState) { // todo transfer task state here - SStreamTask* pStreamTask = streamMetaAcquireTask(pTask->pMeta, pTask->streamTaskId.taskId); - ASSERT(pStreamTask != NULL && pStreamTask->historyTaskId.taskId == pTask->id.taskId); - ASSERT(pStreamTask->status.taskStatus == STREAM_STATUS__PAUSE); + SStreamTask* pStreamTask = streamMetaAcquireTask(pTask->pMeta, pTask->streamTaskId.taskId); + qDebug("s-task:%s scan history task end, update stream task:%s info and launch it", pTask->id.idStr, pStreamTask->id.idStr); + + ASSERT(pStreamTask != NULL && pStreamTask->historyTaskId.taskId == pTask->id.taskId); + ASSERT(pStreamTask->status.taskStatus == TASK_STATUS__HALT); // update the scan data range for source task. - + STimeWindow* pTimeWindow = &pStreamTask->dataRange.window; + qDebug("s-task:%s stream task window %"PRId64" - %"PRId64" transfer to %"PRId64" - %"PRId64", status:%d, sched-status:%d", pStreamTask->id.idStr, + pTimeWindow->skey, pTimeWindow->ekey, INT64_MIN, pTimeWindow->ekey, TASK_STATUS__NORMAL, pStreamTask->status.schedStatus); + pTimeWindow->skey = INT64_MIN; streamSetStatusNormal(pStreamTask); streamSchedExec(pStreamTask); streamMetaReleaseTask(pTask->pMeta, pStreamTask); - - // todo set the task with specified status, to avoid execute this process again } break; } @@ -479,7 +482,8 @@ int32_t streamTryExec(SStreamTask* pTask) { // todo the task should be commit here atomic_store_8(&pTask->status.schedStatus, TASK_SCHED_STATUS__INACTIVE); - qDebug("s-task:%s exec completed, status:%d", pTask->id.idStr, pTask->status.taskStatus); + qDebug("s-task:%s exec completed, status:%d, sched-status:%d", pTask->id.idStr, pTask->status.taskStatus, + pTask->status.schedStatus); if (!taosQueueEmpty(pTask->inputQueue->queue) && (!streamTaskShouldStop(&pTask->status)) && (!streamTaskShouldPause(&pTask->status))) { diff --git a/source/libs/stream/src/streamRecover.c b/source/libs/stream/src/streamRecover.c index 9efe6fec42..c5f1004784 100644 --- a/source/libs/stream/src/streamRecover.c +++ b/source/libs/stream/src/streamRecover.c @@ -21,16 +21,17 @@ const char* streamGetTaskStatusStr(int32_t status) { switch(status) { case TASK_STATUS__NORMAL: return "normal"; case TASK_STATUS__WAIT_DOWNSTREAM: return "wait-for-downstream"; - case TASK_STATUS__RECOVER_PREPARE: return "scan-history-prepare"; - case TASK_STATUS__RECOVER1: return "scan-history-data"; + case TASK_STATUS__SCAN_HISTORY_PREPARE: return "scan-history-prepare"; + case TASK_STATUS__HALT: return "halt"; default:return ""; } } + int32_t streamTaskLaunchRecover(SStreamTask* pTask) { qDebug("s-task:%s (vgId:%d) launch recover", pTask->id.idStr, pTask->info.nodeId); if (pTask->info.taskLevel == TASK_LEVEL__SOURCE) { - atomic_store_8(&pTask->status.taskStatus, TASK_STATUS__RECOVER_PREPARE); + atomic_store_8(&pTask->status.taskStatus, TASK_STATUS__SCAN_HISTORY_PREPARE); SVersionRange* pRange = &pTask->dataRange.range; qDebug("s-task:%s set task status:%s and start to recover, ver:%" PRId64 "-%" PRId64, pTask->id.idStr, @@ -51,7 +52,7 @@ int32_t streamTaskLaunchRecover(SStreamTask* pTask) { memcpy(serializedReq, &req, len); - SRpcMsg rpcMsg = { .contLen = len, .pCont = serializedReq, .msgType = TDMT_VND_STREAM_RECOVER_NONBLOCKING_STAGE }; + SRpcMsg rpcMsg = { .contLen = len, .pCont = serializedReq, .msgType = TDMT_VND_STREAM_SCAN_HISTORY }; if (tmsgPutToQueue(pTask->pMsgCb, STREAM_QUEUE, &rpcMsg) < 0) { /*ASSERT(0);*/ } @@ -473,6 +474,7 @@ int32_t streamTaskScanHistoryDataComplete(SStreamTask* pTask) { return -1; } + atomic_store_8(&pTask->status.schedStatus, TASK_SCHED_STATUS__INACTIVE); streamMetaSaveTask(pMeta, pTask); return 0; } diff --git a/source/libs/stream/src/streamTask.c b/source/libs/stream/src/streamTask.c index 7261fd31cc..0ab096aea3 100644 --- a/source/libs/stream/src/streamTask.c +++ b/source/libs/stream/src/streamTask.c @@ -89,6 +89,9 @@ int32_t tEncodeStreamTask(SEncoder* pEncoder, const SStreamTask* pTask) { if (tEncodeI64(pEncoder, pTask->historyTaskId.streamId)) return -1; if (tEncodeI32(pEncoder, pTask->historyTaskId.taskId)) return -1; + if (tEncodeI64(pEncoder, pTask->streamTaskId.streamId)) return -1; + if (tEncodeI32(pEncoder, pTask->streamTaskId.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; @@ -149,6 +152,9 @@ int32_t tDecodeStreamTask(SDecoder* pDecoder, SStreamTask* pTask) { if (tDecodeI64(pDecoder, &pTask->historyTaskId.streamId)) return -1; if (tDecodeI32(pDecoder, &pTask->historyTaskId.taskId)) return -1; + if (tDecodeI64(pDecoder, &pTask->streamTaskId.streamId)) return -1; + if (tDecodeI32(pDecoder, &pTask->streamTaskId.taskId)) return -1; + if (tDecodeU64(pDecoder, &pTask->dataRange.range.minVer)) return -1; if (tDecodeU64(pDecoder, &pTask->dataRange.range.maxVer)) return -1; if (tDecodeI64(pDecoder, &pTask->dataRange.window.skey)) return -1; From c58bde971502370f62c55d84e093c584c96f3711 Mon Sep 17 00:00:00 2001 From: liuyao <54liuyao@163.com> Date: Tue, 13 Jun 2023 10:57:36 +0800 Subject: [PATCH 22/83] stream operator fill history --- source/libs/executor/inc/executorInt.h | 4 + source/libs/executor/inc/operator.h | 4 + source/libs/executor/src/operator.c | 7 + source/libs/executor/src/scanoperator.c | 43 +++++ source/libs/executor/src/timewindowoperator.c | 169 +++++++++++++++++- 5 files changed, 224 insertions(+), 3 deletions(-) diff --git a/source/libs/executor/inc/executorInt.h b/source/libs/executor/inc/executorInt.h index 38890f8c34..07507db836 100644 --- a/source/libs/executor/inc/executorInt.h +++ b/source/libs/executor/inc/executorInt.h @@ -503,6 +503,8 @@ typedef struct SStreamSessionAggOperatorInfo { SArray* pUpdated; SSHashObj* pStUpdated; int64_t dataVersion; + SArray* historyWins; + bool isHistoryOp; } SStreamSessionAggOperatorInfo; typedef struct SStreamStateAggOperatorInfo { @@ -522,6 +524,8 @@ typedef struct SStreamStateAggOperatorInfo { SArray* pUpdated; SSHashObj* pSeUpdated; int64_t dataVersion; + bool isHistoryOp; + SArray* historyWins; } SStreamStateAggOperatorInfo; typedef struct SStreamPartitionOperatorInfo { diff --git a/source/libs/executor/inc/operator.h b/source/libs/executor/inc/operator.h index 1d2685b8c6..4042dc45c7 100644 --- a/source/libs/executor/inc/operator.h +++ b/source/libs/executor/inc/operator.h @@ -35,6 +35,7 @@ typedef SSDataBlock* (*__optr_fn_t)(struct SOperatorInfo* pOptr); typedef void (*__optr_close_fn_t)(void* param); typedef int32_t (*__optr_explain_fn_t)(struct SOperatorInfo* pOptr, void** pOptrExplain, uint32_t* len); typedef int32_t (*__optr_reqBuf_fn_t)(struct SOperatorInfo* pOptr); +typedef void (*__optr_state_fn_t)(struct SOperatorInfo* pOptr); typedef struct SOperatorFpSet { __optr_open_fn_t _openFn; // DO NOT invoke this function directly @@ -45,6 +46,8 @@ typedef struct SOperatorFpSet { __optr_encode_fn_t encodeResultRow; __optr_decode_fn_t decodeResultRow; __optr_explain_fn_t getExplainFn; + __optr_state_fn_t releaseStreamStateFn; + __optr_state_fn_t reloadStreamStateFn; } SOperatorFpSet; enum { @@ -143,6 +146,7 @@ SOperatorInfo* createEventwindowOperatorInfo(SOperatorInfo* downstream, SPhysiNo SOperatorFpSet createOperatorFpSet(__optr_open_fn_t openFn, __optr_fn_t nextFn, __optr_fn_t cleanup, __optr_close_fn_t closeFn, __optr_reqBuf_fn_t reqBufFn, __optr_explain_fn_t explain); +void setOperatorStreamStateFn(SOperatorInfo* pOperator, __optr_state_fn_t relaseFn, __optr_state_fn_t reloadFn); int32_t optrDummyOpenFn(SOperatorInfo* pOperator); int32_t appendDownstream(SOperatorInfo* p, SOperatorInfo** pDownstream, int32_t num); void setOperatorCompleted(SOperatorInfo* pOperator); diff --git a/source/libs/executor/src/operator.c b/source/libs/executor/src/operator.c index 730252c7ee..7486a4ad5f 100644 --- a/source/libs/executor/src/operator.c +++ b/source/libs/executor/src/operator.c @@ -38,11 +38,18 @@ SOperatorFpSet createOperatorFpSet(__optr_open_fn_t openFn, __optr_fn_t nextFn, .closeFn = closeFn, .reqBufFn = reqBufFn, .getExplainFn = explain, + .releaseStreamStateFn = NULL, + .reloadStreamStateFn = NULL, }; return fpSet; } +void setOperatorStreamStateFn(SOperatorInfo* pOperator, __optr_state_fn_t relaseFn, __optr_state_fn_t reloadFn) { + pOperator->fpSet.releaseStreamStateFn = relaseFn; + pOperator->fpSet.reloadStreamStateFn = reloadFn; +} + int32_t optrDummyOpenFn(SOperatorInfo* pOperator) { OPTR_SET_OPENED(pOperator); pOperator->cost.openCost = 0; diff --git a/source/libs/executor/src/scanoperator.c b/source/libs/executor/src/scanoperator.c index 1f2e4cbe39..817b2a8d5f 100644 --- a/source/libs/executor/src/scanoperator.c +++ b/source/libs/executor/src/scanoperator.c @@ -44,6 +44,7 @@ int32_t scanDebug = 0; #define SET_REVERSE_SCAN_FLAG(_info) ((_info)->scanFlag = REVERSE_SCAN) #define SWITCH_ORDER(n) (((n) = ((n) == TSDB_ORDER_ASC) ? TSDB_ORDER_DESC : TSDB_ORDER_ASC)) #define STREAM_SCAN_OP_NAME "StreamScanOperator" +#define STREAM_SCAN_OP_STATE_NAME "StreamScanFillHistoryState" typedef struct STableMergeScanExecInfo { SFileBlockLoadRecorder blockRecorder; @@ -2317,6 +2318,47 @@ static void destroyStreamScanOperatorInfo(void* param) { taosMemoryFree(pStreamScan); } +void streamScanReleaseState(SOperatorInfo* pOperator) { + SStreamScanInfo* pInfo = pOperator->info; + if (!pInfo->pUpdateInfo) { + return; + } + int32_t len = pInfo->stateStore.updateInfoSerialize(NULL, 0, pInfo->pUpdateInfo); + void* pBuff = taosMemoryCalloc(1, len); + pInfo->stateStore.updateInfoSerialize(pBuff, len, pInfo->pUpdateInfo); + pInfo->stateStore.streamStateSaveInfo(pInfo->pState, STREAM_SCAN_OP_STATE_NAME, strlen(STREAM_SCAN_OP_STATE_NAME), pBuff, len); +} + +void streamScanReloadState(SOperatorInfo* pOperator) { + SStreamScanInfo* pInfo = pOperator->info; + void* pBuff = NULL; + int32_t len = 0; + pInfo->stateStore.streamStateGetInfo(pInfo->pState, STREAM_SCAN_OP_STATE_NAME, strlen(STREAM_SCAN_OP_STATE_NAME), &pBuff, &len); + SUpdateInfo* pUpInfo = pInfo->stateStore.updateInfoInit(0, TSDB_TIME_PRECISION_MILLI, 0); + int32_t code = pInfo->stateStore.updateInfoDeserialize(pBuff, len, pUpInfo); + if (code == TSDB_CODE_SUCCESS && pInfo->pUpdateInfo) { + if (pInfo->pUpdateInfo->minTS < 0) { + pInfo->stateStore.updateInfoDestroy(pInfo->pUpdateInfo); + pInfo->pUpdateInfo = pUpInfo; + } else { + pInfo->pUpdateInfo->minTS = TMAX(pInfo->pUpdateInfo->minTS, pUpInfo->minTS); + pInfo->pUpdateInfo->maxDataVersion = TMAX(pInfo->pUpdateInfo->maxDataVersion, pUpInfo->maxDataVersion); + SHashObj* curMap = pInfo->pUpdateInfo->pMap; + void *pIte = taosHashIterate(curMap, NULL); + while (pIte != NULL) { + size_t keySize = 0; + int64_t* pUid = taosHashGetKey(pIte, &keySize); + taosHashPut(pUpInfo->pMap, pUid, sizeof(int64_t), pIte, sizeof(TSKEY)); + pIte = taosHashIterate(curMap, pIte); + } + taosHashCleanup(curMap); + pInfo->pUpdateInfo->pMap = pUpInfo->pMap; + pUpInfo->pMap = NULL; + pInfo->stateStore.updateInfoDestroy(pUpInfo); + } + } +} + SOperatorInfo* createStreamScanOperatorInfo(SReadHandle* pHandle, STableScanPhysiNode* pTableScanNode, SNode* pTagCond, STableListInfo* pTableListInfo, SExecTaskInfo* pTaskInfo) { SArray* pColIds = NULL; @@ -2489,6 +2531,7 @@ SOperatorInfo* createStreamScanOperatorInfo(SReadHandle* pHandle, STableScanPhys setOperatorInfo(pOperator, STREAM_SCAN_OP_NAME, QUERY_NODE_PHYSICAL_PLAN_STREAM_SCAN, false, OP_NOT_OPENED, pInfo, pTaskInfo); + setOperatorStreamStateFn(pOperator, streamScanReleaseState, streamScanReloadState); pOperator->exprSupp.numOfExprs = taosArrayGetSize(pInfo->pRes->pDataBlock); __optr_fn_t nextFn = (pTaskInfo->execModel == OPTR_EXEC_MODEL_STREAM) ? doStreamScan : doQueueScan; diff --git a/source/libs/executor/src/timewindowoperator.c b/source/libs/executor/src/timewindowoperator.c index 7463475a54..593d735039 100644 --- a/source/libs/executor/src/timewindowoperator.c +++ b/source/libs/executor/src/timewindowoperator.c @@ -28,6 +28,8 @@ #define IS_FINAL_OP(op) ((op)->isFinal) #define DEAULT_DELETE_MARK (1000LL * 60LL * 60LL * 24LL * 365LL * 10LL); +#define STREAM_SESSION_OP_STATE_NAME "StreamSessionHistoryState" +#define STREAM_STATE_OP_STATE_NAME "StreamStateHistoryState" typedef struct SStateWindowInfo { SResultWindowInfo winInfo; @@ -2721,6 +2723,26 @@ int32_t getMaxFunResSize(SExprSupp* pSup, int32_t numOfCols) { return size; } +void streamIntervalReleaseState(SOperatorInfo* pOperator) { + if (pOperator->operatorType == QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_INTERVAL) { + return; + } + SStreamIntervalOperatorInfo* pInfo = pOperator->info; + SStorageAPI* pAPI = &pOperator->pTaskInfo->storageAPI; + pAPI->stateStore.streamStateCommit(pInfo->pState); + SOperatorInfo* downstream = pOperator->pDownstream[0]; + if (downstream->fpSet.releaseStreamStateFn) { + downstream->fpSet.releaseStreamStateFn(downstream); + } +} + +void streamIntervalReloadState(SOperatorInfo* pOperator) { + SOperatorInfo* downstream = pOperator->pDownstream[0]; + if (downstream->fpSet.reloadStreamStateFn) { + downstream->fpSet.reloadStreamStateFn(downstream); + } +} + SOperatorInfo* createStreamFinalIntervalOperatorInfo(SOperatorInfo* downstream, SPhysiNode* pPhyNode, SExecTaskInfo* pTaskInfo, int32_t numOfChild) { SIntervalPhysiNode* pIntervalPhyNode = (SIntervalPhysiNode*)pPhyNode; @@ -2830,6 +2852,7 @@ SOperatorInfo* createStreamFinalIntervalOperatorInfo(SOperatorInfo* downstream, pOperator->fpSet = createOperatorFpSet(NULL, doStreamFinalIntervalAgg, NULL, destroyStreamFinalIntervalOperatorInfo, optrDefaultBufFn, NULL); + setOperatorStreamStateFn(pOperator, streamIntervalReleaseState, streamIntervalReloadState); if (pPhyNode->type == QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_INTERVAL) { initIntervalDownStream(downstream, pPhyNode->type, pInfo); } @@ -3463,6 +3486,26 @@ void doBuildSessionResult(SOperatorInfo* pOperator, void* pState, SGroupResInfo* pBlock->info.id.groupId = 0; buildSessionResultDataBlock(pOperator, pState, pBlock, &pOperator->exprSupp, pGroupResInfo); } +void getMaxTsWins(const SArray* pAllWins, SArray* pMaxWins) { + int32_t size = taosArrayGetSize(pAllWins); + if (size == 0) { + return; + } + + SSessionKey* pSeKey = taosArrayGet(pAllWins, size - 1); + taosArrayPush(pMaxWins, pSeKey); + if (pSeKey->groupId == 0) { + return; + } + uint64_t preGpId = pSeKey->groupId; + for (int32_t i = size - 2; i >= 0; i--) { + pSeKey = taosArrayGet(pAllWins, i); + if (preGpId != pSeKey->groupId) { + taosArrayPush(pMaxWins, pSeKey); + preGpId = pSeKey->groupId; + } + } +} static SSDataBlock* doStreamSessionAgg(SOperatorInfo* pOperator) { SExprSupp* pSup = &pOperator->exprSupp; @@ -3563,6 +3606,9 @@ static SSDataBlock* doStreamSessionAgg(SOperatorInfo* pOperator) { removeSessionResults(pInfo->pStDeleted, pInfo->pUpdated); tSimpleHashCleanup(pInfo->pStUpdated); pInfo->pStUpdated = NULL; + if(pInfo->isHistoryOp) { + getMaxTsWins(pInfo->pUpdated, pInfo->historyWins); + } initGroupResInfoFromArrayList(&pInfo->groupResInfo, pInfo->pUpdated); pInfo->pUpdated = NULL; blockDataEnsureCapacity(pInfo->binfo.pRes, pOperator->resultInfo.capacity); @@ -3589,6 +3635,42 @@ static SSDataBlock* doStreamSessionAgg(SOperatorInfo* pOperator) { return NULL; } +void streamSessionReleaseState(SOperatorInfo* pOperator) { + if (pOperator->operatorType != QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_SESSION) { + SStreamSessionAggOperatorInfo* pInfo = pOperator->info; + int32_t resSize = taosArrayGetSize(pInfo->historyWins) * sizeof(SSessionKey); + pInfo->streamAggSup.stateStore.streamStateSaveInfo(pInfo->streamAggSup.pState, STREAM_SESSION_OP_STATE_NAME, strlen(STREAM_SESSION_OP_STATE_NAME), pInfo->historyWins->pData, resSize); + } + SOperatorInfo* downstream = pOperator->pDownstream[0]; + if (downstream->fpSet.releaseStreamStateFn) { + downstream->fpSet.releaseStreamStateFn(downstream); + } +} + +void streamSessionReloadState(SOperatorInfo* pOperator) { + SStreamSessionAggOperatorInfo* pInfo = pOperator->info; + SStreamAggSupporter* pAggSup = &pInfo->streamAggSup; + SResultWindowInfo winInfo = {0}; + SSessionKey seKey = {.win.skey = INT64_MIN, .win.ekey = INT64_MIN, .groupId = 0}; + int32_t size = 0; + void* pBuf = NULL; + int32_t code = pAggSup->stateStore.streamStateGetInfo(pAggSup->pState, STREAM_SESSION_OP_STATE_NAME, + strlen(STREAM_SESSION_OP_STATE_NAME), &pBuf, &size); + int32_t num = size / sizeof(SSessionKey); + SSessionKey* pSeKeyBuf = (SSessionKey*) pBuf; + ASSERT(size == num * sizeof(SSessionKey)); + for (int32_t i = 0; i < num; i++) { + SResultWindowInfo winInfo = {0}; + setSessionOutputBuf(pAggSup, pSeKeyBuf[i].win.skey, pSeKeyBuf[i].win.ekey, pSeKeyBuf[i].groupId, &winInfo); + compactSessionWindow(pOperator, &winInfo, pInfo->pStUpdated, pInfo->pStDeleted); + } + + SOperatorInfo* downstream = pOperator->pDownstream[0]; + if (downstream->fpSet.reloadStreamStateFn) { + downstream->fpSet.reloadStreamStateFn(downstream); + } +} + SOperatorInfo* createStreamSessionAggOperatorInfo(SOperatorInfo* downstream, SPhysiNode* pPhyNode, SExecTaskInfo* pTaskInfo) { SSessionWinodwPhysiNode* pSessionNode = (SSessionWinodwPhysiNode*)pPhyNode; @@ -3653,11 +3735,17 @@ SOperatorInfo* createStreamSessionAggOperatorInfo(SOperatorInfo* downstream, SPh pInfo->pUpdated = NULL; pInfo->pStUpdated = NULL; pInfo->dataVersion = 0; + pInfo->historyWins = taosArrayInit(4, sizeof(SSessionKey)); + if (!pInfo->historyWins) { + goto _error; + } + pInfo->isHistoryOp = false; setOperatorInfo(pOperator, "StreamSessionWindowAggOperator", QUERY_NODE_PHYSICAL_PLAN_STREAM_SESSION, true, OP_NOT_OPENED, pInfo, pTaskInfo); pOperator->fpSet = createOperatorFpSet(optrDummyOpenFn, doStreamSessionAgg, NULL, destroyStreamSessionAggOperatorInfo, optrDefaultBufFn, NULL); + setOperatorStreamStateFn(pOperator, streamSessionReleaseState, streamSessionReloadState); if (downstream) { initDownStream(downstream, &pInfo->streamAggSup, pOperator->operatorType, pInfo->primaryTsIndex, &pInfo->twAggSup); @@ -3765,7 +3853,6 @@ static SSDataBlock* doStreamSessionSemiAgg(SOperatorInfo* pOperator) { removeSessionResults(pInfo->pStDeleted, pInfo->pUpdated); tSimpleHashCleanup(pInfo->pStUpdated); pInfo->pStUpdated = NULL; - initGroupResInfoFromArrayList(&pInfo->groupResInfo, pInfo->pUpdated); pInfo->pUpdated = NULL; blockDataEnsureCapacity(pBInfo->pRes, pOperator->resultInfo.capacity); @@ -3815,7 +3902,7 @@ SOperatorInfo* createStreamFinalSessionAggOperatorInfo(SOperatorInfo* downstream pOperator->fpSet = createOperatorFpSet(optrDummyOpenFn, doStreamSessionSemiAgg, NULL, destroyStreamSessionAggOperatorInfo, optrDefaultBufFn, NULL); } - + setOperatorStreamStateFn(pOperator, streamSessionReleaseState, streamSessionReloadState); setOperatorInfo(pOperator, name, pPhyNode->type, false, OP_NOT_OPENED, pInfo, pTaskInfo); pOperator->operatorType = pPhyNode->type; @@ -3879,6 +3966,9 @@ bool isEqualStateKey(SStateWindowInfo* pWin, char* pKeyData) { } bool compareStateKey(void* data, void* key) { + if (!data || !key) { + return true; + } SStateKeys* stateKey = (SStateKeys*)key; stateKey->pData = (char*)key + sizeof(SStateKeys); return compareVal(data, stateKey); @@ -3902,7 +3992,7 @@ void setStateOutputBuf(SStreamAggSupporter* pAggSup, TSKEY ts, uint64_t groupId, if (code == TSDB_CODE_SUCCESS) { pCurWin->winInfo.isOutput = true; - } else { + } else if (pKeyData) { if (IS_VAR_DATA_TYPE(pAggSup->stateKeyType)) { varDataCopy(pCurWin->pStateKey->pData, pKeyData); } else { @@ -4100,6 +4190,10 @@ static SSDataBlock* doStreamStateAgg(SOperatorInfo* pOperator) { tSimpleHashCleanup(pInfo->pSeUpdated); pInfo->pSeUpdated = NULL; + if(pInfo->isHistoryOp) { + getMaxTsWins(pInfo->pUpdated, pInfo->historyWins); + } + initGroupResInfoFromArrayList(&pInfo->groupResInfo, pInfo->pUpdated); pInfo->pUpdated = NULL; blockDataEnsureCapacity(pInfo->binfo.pRes, pOperator->resultInfo.capacity); @@ -4125,6 +4219,68 @@ static SSDataBlock* doStreamStateAgg(SOperatorInfo* pOperator) { return NULL; } +void streamStateReleaseState(SOperatorInfo* pOperator) { + SStreamStateAggOperatorInfo* pInfo = pOperator->info; + int32_t resSize = taosArrayGetSize(pInfo->historyWins) * sizeof(SSessionKey); + pInfo->streamAggSup.stateStore.streamStateSaveInfo(pInfo->streamAggSup.pState, STREAM_STATE_OP_STATE_NAME, strlen(STREAM_STATE_OP_STATE_NAME), pInfo->historyWins->pData, resSize); + SOperatorInfo* downstream = pOperator->pDownstream[0]; + if (downstream->fpSet.releaseStreamStateFn) { + downstream->fpSet.releaseStreamStateFn(downstream); + } +} + +static void compactStateWindow(SOperatorInfo* pOperator, SResultWindowInfo* pCurWin, SResultWindowInfo* pNextWin, + SSHashObj* pStUpdated, SSHashObj* pStDeleted) { + SExprSupp* pSup = &pOperator->exprSupp; + SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; + SStorageAPI* pAPI = &pOperator->pTaskInfo->storageAPI; + + SStreamStateAggOperatorInfo* pInfo = pOperator->info; + SResultRow* pCurResult = NULL; + int32_t numOfOutput = pOperator->exprSupp.numOfExprs; + SStreamAggSupporter* pAggSup = &pInfo->streamAggSup; + initSessionOutputBuf(pCurWin, &pCurResult, pSup->pCtx, numOfOutput, pSup->rowEntryInfoOffset); + SResultRow* pWinResult = NULL; + initSessionOutputBuf(pNextWin, &pWinResult, pAggSup->pDummyCtx, numOfOutput, pSup->rowEntryInfoOffset); + + updateTimeWindowInfo(&pInfo->twAggSup.timeWindowData, &pCurWin->sessionWin.win, true); + compactFunctions(pSup->pCtx, pAggSup->pDummyCtx, numOfOutput, pTaskInfo, &pInfo->twAggSup.timeWindowData); + tSimpleHashRemove(pStUpdated, &pNextWin->sessionWin, sizeof(SSessionKey)); + if (pNextWin->isOutput && pStDeleted) { + saveDeleteRes(pStDeleted, pNextWin->sessionWin); + } + removeSessionResult(pStUpdated, pAggSup->pResultRows, pNextWin->sessionWin); + doDeleteSessionWindow(pAggSup, &pNextWin->sessionWin); + taosMemoryFree(pNextWin->pOutputBuf); + saveSessionOutputBuf(pAggSup, pCurWin); +} + +void streamStateReloadState(SOperatorInfo* pOperator) { + SStreamSessionAggOperatorInfo* pInfo = pOperator->info; + SStreamAggSupporter* pAggSup = &pInfo->streamAggSup; + SSessionKey seKey = {.win.skey = INT64_MIN, .win.ekey = INT64_MIN, .groupId = 0}; + int32_t size = 0; + void* pBuf = NULL; + int32_t code = pAggSup->stateStore.streamStateGetInfo(pAggSup->pState, STREAM_STATE_OP_STATE_NAME, + strlen(STREAM_STATE_OP_STATE_NAME), &pBuf, &size); + int32_t num = size / sizeof(SSessionKey); + SSessionKey* pSeKeyBuf = (SSessionKey*) pBuf; + ASSERT(size == num * sizeof(SSessionKey)); + for (int32_t i = 0; i < num; i++) { + SStateWindowInfo curInfo = {0}; + SStateWindowInfo nextInfo = {0}; + setStateOutputBuf(pAggSup, pSeKeyBuf[i].win.skey, pSeKeyBuf[i].groupId, NULL, &curInfo, &nextInfo); + if (compareStateKey(curInfo.pStateKey,nextInfo.pStateKey)) { + compactStateWindow(pOperator, &curInfo.winInfo, &nextInfo.winInfo, pInfo->pStUpdated, pInfo->pStDeleted); + } + } + + SOperatorInfo* downstream = pOperator->pDownstream[0]; + if (downstream->fpSet.reloadStreamStateFn) { + downstream->fpSet.reloadStreamStateFn(downstream); + } +} + SOperatorInfo* createStreamStateAggOperatorInfo(SOperatorInfo* downstream, SPhysiNode* pPhyNode, SExecTaskInfo* pTaskInfo) { SStreamStateWinodwPhysiNode* pStateNode = (SStreamStateWinodwPhysiNode*)pPhyNode; @@ -4186,11 +4342,17 @@ SOperatorInfo* createStreamStateAggOperatorInfo(SOperatorInfo* downstream, SPhys pInfo->pUpdated = NULL; pInfo->pSeUpdated = NULL; pInfo->dataVersion = 0; + pInfo->historyWins = taosArrayInit(4, sizeof(SSessionKey)); + if (!pInfo->historyWins) { + goto _error; + } + pInfo->isHistoryOp = false; setOperatorInfo(pOperator, "StreamStateAggOperator", QUERY_NODE_PHYSICAL_PLAN_STREAM_STATE, true, OP_NOT_OPENED, pInfo, pTaskInfo); pOperator->fpSet = createOperatorFpSet(optrDummyOpenFn, doStreamStateAgg, NULL, destroyStreamStateOperatorInfo, optrDefaultBufFn, NULL); + setOperatorStreamStateFn(pOperator, streamStateReleaseState, streamSessionReloadState); initDownStream(downstream, &pInfo->streamAggSup, pOperator->operatorType, pInfo->primaryTsIndex, &pInfo->twAggSup); code = appendDownstream(pOperator, &downstream, 1); if (code != TSDB_CODE_SUCCESS) { @@ -5017,6 +5179,7 @@ SOperatorInfo* createStreamIntervalOperatorInfo(SOperatorInfo* downstream, SPhys pInfo, pTaskInfo); pOperator->fpSet = createOperatorFpSet(optrDummyOpenFn, doStreamIntervalAgg, NULL, destroyStreamFinalIntervalOperatorInfo, optrDefaultBufFn, NULL); + setOperatorStreamStateFn(pOperator, streamIntervalReleaseState, streamIntervalReloadState); pInfo->statestore = pTaskInfo->storageAPI.stateStore; pInfo->recvGetAll = false; From 97da2a8face14c3be6879d2acaaef3932082fd43 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 13 Jun 2023 16:10:55 +0800 Subject: [PATCH 23/83] fix(stream): set the fill history status. --- include/common/tmsgdef.h | 1 - include/libs/stream/tstream.h | 5 +- source/dnode/vnode/src/tq/tq.c | 33 ++++++++--- source/dnode/vnode/src/tq/tqRestore.c | 5 +- source/dnode/vnode/src/vnd/vnodeSvr.c | 14 +++-- source/libs/stream/src/streamDispatch.c | 3 +- source/libs/stream/src/streamExec.c | 37 +++++++++--- source/libs/stream/src/streamRecover.c | 76 +++++++++++++------------ 8 files changed, 110 insertions(+), 64 deletions(-) diff --git a/include/common/tmsgdef.h b/include/common/tmsgdef.h index 3103b6fd76..bfc0eb57ec 100644 --- a/include/common/tmsgdef.h +++ b/include/common/tmsgdef.h @@ -300,7 +300,6 @@ enum { TD_NEW_MSG_SEG(TDMT_VND_STREAM_MSG) TD_DEF_MSG_TYPE(TDMT_VND_STREAM_TRIGGER, "vnode-stream-trigger", NULL, NULL) TD_DEF_MSG_TYPE(TDMT_VND_STREAM_SCAN_HISTORY, "vnode-stream-scan-history", NULL, NULL) - TD_DEF_MSG_TYPE(TDMT_VND_STREAM_TRANSFER_STATE, "vnode-stream-transfer-state", NULL, NULL) TD_DEF_MSG_TYPE(TDMT_VND_STREAM_CHECK_POINT_SOURCE, "vnode-stream-checkpoint-source", NULL, NULL) TD_DEF_MSG_TYPE(TDMT_VND_STREAM_MAX_MSG, "vnd-stream-max", NULL, NULL) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 63af587794..4c773ea30f 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -45,7 +45,7 @@ enum { TASK_STATUS__FAIL, TASK_STATUS__STOP, TASK_STATUS__WAIT_DOWNSTREAM, - TASK_STATUS__SCAN_HISTORY_PREPARE, + TASK_STATUS__SCAN_HISTORY, TASK_STATUS__HALT, // stream task halt to wait for the secondary scan history, this status is invisible for user TASK_STATUS__PAUSE, }; @@ -271,6 +271,7 @@ typedef struct SStreamStatus { int8_t schedStatus; int8_t keepTaskStatus; bool transferState; + TdThreadMutex lock; } SStreamStatus; typedef struct SHistDataRange { @@ -585,7 +586,7 @@ int32_t streamBuildSourceRecover1Req(SStreamTask* pTask, SStreamRecoverStep1Req* int32_t streamSourceRecoverScanStep1(SStreamTask* pTask); int32_t streamBuildSourceRecover2Req(SStreamTask* pTask, SStreamRecoverStep2Req* pReq); int32_t streamSourceRecoverScanStep2(SStreamTask* pTask, int64_t ver); -int32_t streamDispatchRecoverFinishMsg(SStreamTask* pTask); +int32_t streamDispatchScanHistoryFinishMsg(SStreamTask* pTask); int32_t streamDispatchTransferStateMsg(SStreamTask* pTask); diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 93b078ad7d..1d29f245e2 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -820,7 +820,7 @@ int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t ver) { pTask->dataRange.range.minVer = ver; // expand executor - pTask->status.taskStatus = /*(pTask->info.fillHistory) ? */TASK_STATUS__WAIT_DOWNSTREAM /*: TASK_STATUS__NORMAL*/; + pTask->status.taskStatus = TASK_STATUS__WAIT_DOWNSTREAM; if (pTask->info.taskLevel == TASK_LEVEL__SOURCE) { pTask->pState = streamStateOpen(pTq->pStreamMeta->path, pTask, false, -1, -1); @@ -923,8 +923,8 @@ int32_t tqProcessStreamTaskCheckReq(STQ* pTq, SRpcMsg* pMsg) { rsp.status = streamTaskCheckStatus(pTask); streamMetaReleaseTask(pTq->pStreamMeta, pTask); - tqDebug("s-task:%s recv task check req(reqId:0x%" PRIx64 ") task:0x%x (vgId:%d), status:%d, rsp status %d", - pTask->id.idStr, rsp.reqId, rsp.upstreamTaskId, rsp.upstreamNodeId, pTask->status.taskStatus, rsp.status); + tqDebug("s-task:%s recv task check req(reqId:0x%" PRIx64 ") task:0x%x (vgId:%d), status:%s, rsp status %d", + pTask->id.idStr, rsp.reqId, rsp.upstreamTaskId, rsp.upstreamNodeId, streamGetTaskStatusStr(pTask->status.taskStatus), rsp.status); } else { rsp.status = 0; tqDebug("tq recv task check(taskId:0x%x not built yet) req(reqId:0x%" PRIx64 ") from task:0x%x (vgId:%d), rsp status %d", @@ -1088,7 +1088,8 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { } // do recovery step 1 - tqDebug("s-task:%s start history data scan stage(step 1)", pTask->id.idStr); + tqDebug("s-task:%s start history data scan stage(step 1), status:%s", pTask->id.idStr, streamGetTaskStatusStr(pTask->status.taskStatus)); + int64_t st = taosGetTimestampMs(); // todo set the correct status flag @@ -1116,7 +1117,19 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { // todo handle error } + ASSERT(pStreamTask->info.taskLevel == TASK_LEVEL__SOURCE); + + // wait for the stream task get ready for scan history data + while (pStreamTask->status.taskStatus == TASK_STATUS__WAIT_DOWNSTREAM || + pStreamTask->status.taskStatus == TASK_STATUS__SCAN_HISTORY) { + tqDebug("s-task:%s level:%d not ready for halt, wait for 100ms and recheck", pStreamTask->id.idStr, + pStreamTask->info.taskLevel); + taosMsleep(100); + } + pStreamTask->status.taskStatus = TASK_STATUS__HALT; + tqDebug("s-task:%s level:%d status is set to halt by history scan task:%s", pStreamTask->id.idStr, + pStreamTask->info.taskLevel, pTask->id.idStr); // if it's an source task, extract the last version in wal. int64_t ver = pTask->dataRange.range.maxVer; @@ -1158,14 +1171,20 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { } return 0; + } // notify the downstream tasks to transfer executor state after handle all history blocks. int32_t tqProcessTaskTransferStateReq(STQ* pTq, int64_t sversion, char* msg, int32_t msgLen) { - SStreamTransferReq* pReq = (SStreamTransferReq*)msg; + SStreamTransferReq req; - SStreamTask* pTask = streamMetaAcquireTask(pTq->pStreamMeta, pReq->taskId); + SDecoder decoder; + tDecoderInit(&decoder, (uint8_t*)msg, msgLen); + int32_t code = tDecodeStreamRecoverFinishReq(&decoder, &req); + + SStreamTask* pTask = streamMetaAcquireTask(pTq->pStreamMeta, req.taskId); if (pTask == NULL) { + tqError("failed to find task:0x%x", req.taskId); return -1; } @@ -1212,7 +1231,7 @@ int32_t tqProcessTaskTransferStateReq(STQ* pTq, int64_t sversion, char* msg, int tqDebug("s-task:%s step2 recover finished, el:%.2fs", pTask->id.idStr, el); // dispatch recover finish req to all related downstream task - code = streamDispatchRecoverFinishMsg(pTask); + code = streamDispatchScanHistoryFinishMsg(pTask); if (code < 0) { streamMetaReleaseTask(pTq->pStreamMeta, pTask); return -1; diff --git a/source/dnode/vnode/src/tq/tqRestore.c b/source/dnode/vnode/src/tq/tqRestore.c index bf560a3fb9..1a9a4ec612 100644 --- a/source/dnode/vnode/src/tq/tqRestore.c +++ b/source/dnode/vnode/src/tq/tqRestore.c @@ -135,9 +135,8 @@ int32_t createStreamTaskRunReq(SStreamMeta* pStreamMeta, bool* pScanIdle) { continue; } - if (streamTaskShouldStop(&pTask->status) || status == TASK_STATUS__SCAN_HISTORY_PREPARE || - status == TASK_STATUS__WAIT_DOWNSTREAM || streamTaskShouldPause(&pTask->status)) { - tqDebug("s-task:%s not ready for new submit block from wal, status:%d", pTask->id.idStr, status); + if (status != TASK_STATUS__NORMAL) { + tqDebug("s-task:%s not ready for new submit block from wal, status:%s", pTask->id.idStr, streamGetTaskStatusStr(status)); streamMetaReleaseTask(pStreamMeta, pTask); continue; } diff --git a/source/dnode/vnode/src/vnd/vnodeSvr.c b/source/dnode/vnode/src/vnd/vnodeSvr.c index 99957431f8..96113eeb7f 100644 --- a/source/dnode/vnode/src/vnd/vnodeSvr.c +++ b/source/dnode/vnode/src/vnd/vnodeSvr.c @@ -573,16 +573,22 @@ int32_t vnodeProcessStreamMsg(SVnode *pVnode, SRpcMsg *pMsg, SQueueInfo *pInfo) return tqProcessTaskDispatchRsp(pVnode->pTq, pMsg); case TDMT_STREAM_TASK_CHECK: return tqProcessStreamTaskCheckReq(pVnode->pTq, pMsg); - case TDMT_STREAM_TASK_CHECK_RSP: - return tqProcessStreamTaskCheckRsp(pVnode->pTq, 0, pMsg->pCont, pMsg->contLen); + case TDMT_STREAM_TASK_CHECK_RSP: { + char* pReq = POINTER_SHIFT(pMsg->pCont, sizeof(SMsgHead)); + int32_t len = pMsg->contLen - sizeof(SMsgHead); + return tqProcessStreamTaskCheckRsp(pVnode->pTq, 0, pReq, len); + } case TDMT_STREAM_RETRIEVE: return tqProcessTaskRetrieveReq(pVnode->pTq, pMsg); case TDMT_STREAM_RETRIEVE_RSP: return tqProcessTaskRetrieveRsp(pVnode->pTq, pMsg); case TDMT_VND_STREAM_SCAN_HISTORY: return tqProcessTaskScanHistory(pVnode->pTq, pMsg); - case TDMT_VND_STREAM_TRANSFER_STATE: - return tqProcessTaskTransferStateReq(pVnode->pTq, 0, pMsg->pCont, pMsg->contLen); + case TDMT_STREAM_TRANSFER_STATE: { + char* pReq = POINTER_SHIFT(pMsg->pCont, sizeof(SMsgHead)); + int32_t len = pMsg->contLen - sizeof(SMsgHead); + return tqProcessTaskTransferStateReq(pVnode->pTq, 0, pReq, len); + } case TDMT_STREAM_RECOVER_FINISH: return tqProcessTaskRecoverFinishReq(pVnode->pTq, pMsg); case TDMT_STREAM_RECOVER_FINISH_RSP: diff --git a/source/libs/stream/src/streamDispatch.c b/source/libs/stream/src/streamDispatch.c index c866e2cc21..ae3f094d12 100644 --- a/source/libs/stream/src/streamDispatch.c +++ b/source/libs/stream/src/streamDispatch.c @@ -312,8 +312,7 @@ int32_t streamDoDispatchRecoverFinishMsg(SStreamTask* pTask, const SStreamRecove msg.info.noResp = 1; tmsgSendReq(pEpSet, &msg); - qDebug("s-task:%s dispatch recover finish msg to taskId:0x%x (vgId:%d)", pTask->id.idStr, pReq->taskId, vgId); - + qDebug("s-task:%s dispatch scan-history-data finish msg to taskId:0x%x (vgId:%d)", pTask->id.idStr, pReq->taskId, vgId); return 0; } diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index 688491fbdb..2cbe72e0be 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -326,15 +326,16 @@ int32_t streamExecForAll(SStreamTask* pTask) { SStreamQueueItem* pInput = NULL; // merge multiple input data if possible in the input queue. - qDebug("s-task:%s start to extract data block from inputQ", id); + qDebug("s-task:%s start to extract data block from inputQ, status:%s", id, streamGetTaskStatusStr(pTask->status.taskStatus)); while (1) { // downstream task's input queue is blocked, stop immediately - if (streamTaskShouldPause(&pTask->status) || (pTask->outputStatus == TASK_INPUT_STATUS__BLOCKED) || + if (streamTaskShouldPause(&pTask->status) || (pTask->outputStatus == TASK_OUTPUT_STATUS__BLOCKED) || streamTaskShouldStop(&pTask->status)) { if (batchSize > 1) { break; } else { + qDebug("123 %s", pTask->id.idStr); return 0; } } @@ -382,23 +383,40 @@ int32_t streamExecForAll(SStreamTask* pTask) { if (pInput) { streamFreeQitem(pInput); } + + qDebug("456 %s", pTask->id.idStr); return 0; } if (pInput == NULL) { - if (pTask->info.fillHistory && pTask->status.transferState) { + qDebug("789 %s", pTask->id.idStr); + + if (pTask->info.fillHistory && pTask->status.transferState) { // todo transfer task state here SStreamTask* pStreamTask = streamMetaAcquireTask(pTask->pMeta, pTask->streamTaskId.taskId); qDebug("s-task:%s scan history task end, update stream task:%s info and launch it", pTask->id.idStr, pStreamTask->id.idStr); ASSERT(pStreamTask != NULL && pStreamTask->historyTaskId.taskId == pTask->id.taskId); - ASSERT(pStreamTask->status.taskStatus == TASK_STATUS__HALT); - - // update the scan data range for source task. STimeWindow* pTimeWindow = &pStreamTask->dataRange.window; - qDebug("s-task:%s stream task window %"PRId64" - %"PRId64" transfer to %"PRId64" - %"PRId64", status:%d, sched-status:%d", pStreamTask->id.idStr, - pTimeWindow->skey, pTimeWindow->ekey, INT64_MIN, pTimeWindow->ekey, TASK_STATUS__NORMAL, pStreamTask->status.schedStatus); + + if (pStreamTask->info.taskLevel == TASK_LEVEL__SOURCE) { + ASSERT(pStreamTask->status.taskStatus == TASK_STATUS__HALT); + + // update the scan data range for source task. + qDebug("s-task:%s level:%d stream task window %" PRId64 " - %" PRId64 " transfer to %" PRId64 " - %" PRId64 + ", status:%s, sched-status:%d", + pStreamTask->id.idStr, TASK_LEVEL__SOURCE, pTimeWindow->skey, pTimeWindow->ekey, INT64_MIN, + pTimeWindow->ekey, streamGetTaskStatusStr(TASK_STATUS__NORMAL), pStreamTask->status.schedStatus); + } else { + // for agg task and sink task, they are continue to execute, no need to be halt. + // the process should be stopped for a while, during the term of transfer task state. + // OR wait for the inputQ && outputQ of agg tasks are all consumed, and then start the state transfer + + + qDebug("s-task:%s no need to update time window", pStreamTask->id.idStr); + } + pTimeWindow->skey = INT64_MIN; streamSetStatusNormal(pStreamTask); @@ -406,6 +424,7 @@ int32_t streamExecForAll(SStreamTask* pTask) { streamMetaReleaseTask(pTask->pMeta, pStreamTask); } + break; } @@ -482,7 +501,7 @@ int32_t streamTryExec(SStreamTask* pTask) { // todo the task should be commit here atomic_store_8(&pTask->status.schedStatus, TASK_SCHED_STATUS__INACTIVE); - qDebug("s-task:%s exec completed, status:%d, sched-status:%d", pTask->id.idStr, pTask->status.taskStatus, + qDebug("s-task:%s exec completed, status:%s, sched-status:%d", pTask->id.idStr, streamGetTaskStatusStr(pTask->status.taskStatus), pTask->status.schedStatus); if (!taosQueueEmpty(pTask->inputQueue->queue) && (!streamTaskShouldStop(&pTask->status)) && diff --git a/source/libs/stream/src/streamRecover.c b/source/libs/stream/src/streamRecover.c index c5f1004784..5e278cd7f6 100644 --- a/source/libs/stream/src/streamRecover.c +++ b/source/libs/stream/src/streamRecover.c @@ -21,22 +21,19 @@ const char* streamGetTaskStatusStr(int32_t status) { switch(status) { case TASK_STATUS__NORMAL: return "normal"; case TASK_STATUS__WAIT_DOWNSTREAM: return "wait-for-downstream"; - case TASK_STATUS__SCAN_HISTORY_PREPARE: return "scan-history-prepare"; + case TASK_STATUS__SCAN_HISTORY: return "scan-history"; case TASK_STATUS__HALT: return "halt"; + case TASK_STATUS__PAUSE: return "paused"; default:return ""; } } int32_t streamTaskLaunchRecover(SStreamTask* pTask) { - qDebug("s-task:%s (vgId:%d) launch recover", pTask->id.idStr, pTask->info.nodeId); - if (pTask->info.taskLevel == TASK_LEVEL__SOURCE) { - atomic_store_8(&pTask->status.taskStatus, TASK_STATUS__SCAN_HISTORY_PREPARE); - SVersionRange* pRange = &pTask->dataRange.range; - qDebug("s-task:%s set task status:%s and start to recover, ver:%" PRId64 "-%" PRId64, pTask->id.idStr, - streamGetTaskStatusStr(pTask->status.taskStatus), pTask->dataRange.range.minVer, - pTask->dataRange.range.maxVer); + qDebug("s-task:%s vgId:%d task status:%s and start to scan-history-data task, ver:%" PRId64 " - %" PRId64, + pTask->id.idStr, pTask->info.nodeId, streamGetTaskStatusStr(pTask->status.taskStatus), + pTask->dataRange.range.minVer, pTask->dataRange.range.maxVer); streamSetParamForRecover(pTask); streamSourceRecoverPrepareStep1(pTask, pRange, &pTask->dataRange.window); @@ -63,7 +60,7 @@ int32_t streamTaskLaunchRecover(SStreamTask* pTask) { streamAggRecoverPrepare(pTask); } else if (pTask->info.taskLevel == TASK_LEVEL__SINK) { streamSetStatusNormal(pTask); - qDebug("s-task:%s sink task convert to normal immediately", pTask->id.idStr); + qDebug("s-task:%s sink task convert to normal status immediately", pTask->id.idStr); } return 0; @@ -181,7 +178,7 @@ int32_t streamProcessCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* pRs taosArrayDestroy(pTask->checkReqIds); pTask->checkReqIds = NULL; - qDebug("s-task:%s all %d downstream tasks are ready, now enter into recover stage", id, numOfReqs); + qDebug("s-task:%s all %d downstream tasks are ready, now enter into scan-history-data stage", id, numOfReqs); streamTaskLaunchRecover(pTask); } else { qDebug("s-task:%s (vgId:%d) recv check rsp from task:0x%x (vgId:%d) status:%d, remain not ready:%d", id, @@ -192,7 +189,12 @@ int32_t streamProcessCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* pRs return -1; } - qDebug("s-task:%s fixed downstream tasks is ready, now enter into recover stage", id); + ASSERT(pTask->status.taskStatus != TASK_STATUS__HALT); + pTask->status.taskStatus = TASK_STATUS__SCAN_HISTORY; + + qDebug("s-task:%s fixed downstream task is ready, now enter into scan-history-data stage, status:%s", id, + streamGetTaskStatusStr(pTask->status.taskStatus)); + streamTaskLaunchRecover(pTask); } else { ASSERT(0); @@ -219,6 +221,7 @@ int32_t streamRestoreParam(SStreamTask* pTask) { } int32_t streamSetStatusNormal(SStreamTask* pTask) { + qDebug("s-task:%s set task status to be normal, prev:%s", pTask->id.idStr, streamGetTaskStatusStr(pTask->status.taskStatus)); atomic_store_8(&pTask->status.taskStatus, TASK_STATUS__NORMAL); return 0; } @@ -264,25 +267,29 @@ int32_t streamSourceRecoverScanStep2(SStreamTask* pTask, int64_t ver) { return code; } -int32_t streamDispatchRecoverFinishMsg(SStreamTask* pTask) { +int32_t streamDispatchScanHistoryFinishMsg(SStreamTask* pTask) { SStreamRecoverFinishReq req = { .streamId = pTask->id.streamId, .childId = pTask->info.selfChildId }; // serialize if (pTask->outputType == TASK_OUTPUT__FIXED_DISPATCH) { - qDebug("s-task:%s send recover finish msg to downstream (fix-dispatch) to taskId:0x%x, status:%d", pTask->id.idStr, - pTask->fixedEpDispatcher.taskId, pTask->status.taskStatus); + qDebug("s-task:%s send scan-history-data complete msg to downstream (fix-dispatch) to taskId:0x%x, status:%s", pTask->id.idStr, + pTask->fixedEpDispatcher.taskId, streamGetTaskStatusStr(pTask->status.taskStatus)); req.taskId = pTask->fixedEpDispatcher.taskId; streamDoDispatchRecoverFinishMsg(pTask, &req, pTask->fixedEpDispatcher.nodeId, &pTask->fixedEpDispatcher.epSet); } else if (pTask->outputType == TASK_OUTPUT__SHUFFLE_DISPATCH) { SArray* vgInfo = pTask->shuffleDispatcher.dbInfo.pVgroupInfos; - int32_t vgSz = taosArrayGetSize(vgInfo); - for (int32_t i = 0; i < vgSz; i++) { + int32_t numOfVgs = taosArrayGetSize(vgInfo); + + qDebug("s-task:%s send scan-history-data complete msg to downstream (shuffle-dispatch) %d tasks, status:%s", pTask->id.idStr, + numOfVgs, streamGetTaskStatusStr(pTask->status.taskStatus)); + for (int32_t i = 0; i < numOfVgs; i++) { SVgroupInfo* pVgInfo = taosArrayGet(vgInfo, i); req.taskId = pVgInfo->taskId; streamDoDispatchRecoverFinishMsg(pTask, &req, pVgInfo->vgId, &pVgInfo->epSet); } } + return 0; } @@ -333,7 +340,7 @@ int32_t streamDispatchTransferStateMsg(SStreamTask* pTask) { // serialize if (pTask->outputType == TASK_OUTPUT__FIXED_DISPATCH) { - qDebug("s-task:%s send recover finish msg to downstream (fix-dispatch) to taskId:0x%x, status:%d", pTask->id.idStr, + qDebug("s-task:%s send transfer state msg to downstream (fix-dispatch) to taskId:0x%x, status:%d", pTask->id.idStr, pTask->fixedEpDispatcher.taskId, pTask->status.taskStatus); req.taskId = pTask->fixedEpDispatcher.taskId; @@ -375,11 +382,15 @@ int32_t streamAggChildrenRecoverFinish(SStreamTask* pTask) { int32_t streamProcessRecoverFinishReq(SStreamTask* pTask, int32_t childId) { if (pTask->info.taskLevel == TASK_LEVEL__AGG) { int32_t left = atomic_sub_fetch_32(&pTask->numOfWaitingUpstream, 1); - qDebug("s-task:%s remain unfinished child tasks:%d", pTask->id.idStr, left); ASSERT(left >= 0); + if (left == 0) { + qDebug("s-task:%s all %d upstream tasks finish scan-history data", pTask->id.idStr, left); streamAggChildrenRecoverFinish(pTask); + } else { + qDebug("s-task:%s remain unfinished upstream tasks:%d", pTask->id.idStr, left); } + } return 0; } @@ -388,10 +399,14 @@ static void doCheckDownstreamStatus(SStreamTask* pTask, SStreamTask* pHTask) { pHTask->dataRange.range.minVer = 0; pHTask->dataRange.range.maxVer = pTask->chkInfo.currentVer; - qDebug("s-task:%s set the launch condition for fill history s-task:%s, window:%" PRId64 " - %" PRId64 - " ver range:%" PRId64 " - %" PRId64, - pTask->id.idStr, pHTask->id.idStr, pHTask->dataRange.window.skey, pHTask->dataRange.window.ekey, - pHTask->dataRange.range.minVer, pHTask->dataRange.range.maxVer); + if (pTask->info.taskLevel == TASK_LEVEL__SOURCE) { + qDebug("s-task:%s set the launch condition for fill history s-task:%s, window:%" PRId64 " - %" PRId64 + " ver range:%" PRId64 " - %" PRId64, + pTask->id.idStr, pHTask->id.idStr, pHTask->dataRange.window.skey, pHTask->dataRange.window.ekey, + pHTask->dataRange.range.minVer, pHTask->dataRange.range.maxVer); + } else { + qDebug("s-task:%s no fill history condition for non-source task:%s", pTask->id.idStr, pHTask->id.idStr); + } // check if downstream tasks have been ready streamTaskCheckDownstreamTasks(pHTask); @@ -444,13 +459,6 @@ int32_t streamTaskStartHistoryTask(SStreamTask* pTask, int64_t ver) { int32_t streamTaskScanHistoryDataComplete(SStreamTask* pTask) { SStreamMeta* pMeta = pTask->pMeta; - - qDebug("s-task:%s set start wal scan start ver:%" PRId64, pTask->id.idStr, pTask->chkInfo.currentVer); - ASSERT(walReaderGetCurrentVer(pTask->exec.pWalReader) == -1); - -// walReaderSeekVer(pTask->exec.pWalReader, sversion); -// pTask->chkInfo.currentVer = sversion; - if (atomic_load_8(&pTask->status.taskStatus) == TASK_STATUS__DROPPING) { return 0; } @@ -462,17 +470,13 @@ int32_t streamTaskScanHistoryDataComplete(SStreamTask* pTask) { } // dispatch recover finish req to all related downstream task - code = streamDispatchRecoverFinishMsg(pTask); + code = streamDispatchScanHistoryFinishMsg(pTask); if (code < 0) { return -1; } - // set status normal - qDebug("s-task:%s set the status to be normal, and start wal scan", pTask->id.idStr); - code = streamSetStatusNormal(pTask); - if (code < 0) { - return -1; - } + ASSERT(pTask->status.taskStatus == TASK_STATUS__SCAN_HISTORY); + /*code = */streamSetStatusNormal(pTask); atomic_store_8(&pTask->status.schedStatus, TASK_SCHED_STATUS__INACTIVE); streamMetaSaveTask(pMeta, pTask); From b44447e6f3b79785b0673a3a680cb46687c06f51 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 14 Jun 2023 10:19:03 +0800 Subject: [PATCH 24/83] enh(stream): support restore from disk. --- include/libs/stream/tstream.h | 13 +-- source/dnode/mnode/impl/src/mndScheduler.c | 6 +- source/dnode/vnode/src/inc/tq.h | 22 +--- source/dnode/vnode/src/inc/vnodeInt.h | 1 + source/dnode/vnode/src/tq/tq.c | 91 ++++++---------- source/dnode/vnode/src/tq/tqRestore.c | 107 ++++++++++++++++++- source/dnode/vnode/src/vnd/vnodeSync.c | 2 +- source/libs/stream/src/streamDispatch.c | 4 +- source/libs/stream/src/streamExec.c | 10 +- source/libs/stream/src/streamMeta.c | 8 +- source/libs/stream/src/streamRecover.c | 115 +++++++++++++-------- source/libs/stream/src/streamTask.c | 1 + 12 files changed, 237 insertions(+), 143 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 4c773ea30f..6ed97ac547 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -268,6 +268,7 @@ typedef struct SCheckpointInfo { typedef struct SStreamStatus { int8_t taskStatus; + int8_t checkDownstream; int8_t schedStatus; int8_t keepTaskStatus; bool transferState; @@ -528,11 +529,11 @@ typedef struct { SArray* checkpointVer; // SArray } SStreamRecoverDownstreamRsp; -int32_t tEncodeSStreamTaskCheckReq(SEncoder* pEncoder, const SStreamTaskCheckReq* pReq); -int32_t tDecodeSStreamTaskCheckReq(SDecoder* pDecoder, SStreamTaskCheckReq* pReq); +int32_t tEncodeStreamTaskCheckReq(SEncoder* pEncoder, const SStreamTaskCheckReq* pReq); +int32_t tDecodeStreamTaskCheckReq(SDecoder* pDecoder, SStreamTaskCheckReq* pReq); -int32_t tEncodeSStreamTaskCheckRsp(SEncoder* pEncoder, const SStreamTaskCheckRsp* pRsp); -int32_t tDecodeSStreamTaskCheckRsp(SDecoder* pDecoder, SStreamTaskCheckRsp* pRsp); +int32_t tEncodeStreamTaskCheckRsp(SEncoder* pEncoder, const SStreamTaskCheckRsp* pRsp); +int32_t tDecodeStreamTaskCheckRsp(SDecoder* pDecoder, SStreamTaskCheckRsp* pRsp); int32_t tEncodeSStreamTaskRecoverReq(SEncoder* pEncoder, const SStreamRecoverDownstreamReq* pReq); int32_t tDecodeSStreamTaskRecoverReq(SDecoder* pDecoder, SStreamRecoverDownstreamReq* pReq); @@ -568,10 +569,10 @@ int32_t streamScanExec(SStreamTask* pTask, int32_t batchSz); // recover and fill history int32_t streamTaskCheckDownstreamTasks(SStreamTask* pTask); -int32_t streamTaskLaunchRecover(SStreamTask* pTask); +int32_t streamTaskLaunchScanHistory(SStreamTask* pTask); int32_t streamTaskCheckStatus(SStreamTask* pTask); int32_t streamProcessCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* pRsp); -int32_t streamTaskStartHistoryTask(SStreamTask* pTask, int64_t ver); +int32_t streamTaskStartHistoryTask(SStreamTask* pTask); int32_t streamTaskScanHistoryDataComplete(SStreamTask* pTask); // common diff --git a/source/dnode/mnode/impl/src/mndScheduler.c b/source/dnode/mnode/impl/src/mndScheduler.c index b6274b57b8..24b2a60898 100644 --- a/source/dnode/mnode/impl/src/mndScheduler.c +++ b/source/dnode/mnode/impl/src/mndScheduler.c @@ -250,7 +250,8 @@ static int32_t addSourceStreamTask(SMnode* pMnode, SVgObj* pVgroup, SArray* pTas pTask->dataRange.window.skey = INT64_MIN; pTask->dataRange.window.ekey = 1685959190000;//taosGetTimestampMs(); - mDebug("0x%x----------------window:%"PRId64" - %"PRId64, pTask->id.taskId, pTask->dataRange.window.skey, pTask->dataRange.window.ekey); + mDebug("add source task 0x%x window:%" PRId64 " - %" PRId64, pTask->id.taskId, pTask->dataRange.window.skey, + pTask->dataRange.window.ekey); // sink or dispatch if (hasExtraSink) { @@ -323,7 +324,8 @@ static void setHTasksId(SArray* pTaskList, const SArray* pHTaskList) { (*pHTask)->streamTaskId.taskId = (*pStreamTask)->id.taskId; (*pHTask)->streamTaskId.streamId = (*pStreamTask)->id.streamId; - mDebug("s-task:0x%x related history task:0x%x", (*pStreamTask)->id.taskId, (*pHTask)->id.taskId); + mDebug("s-task:0x%x related history task:0x%x, level:%d", (*pStreamTask)->id.taskId, (*pHTask)->id.taskId, + (*pHTask)->info.taskLevel); } } diff --git a/source/dnode/vnode/src/inc/tq.h b/source/dnode/vnode/src/inc/tq.h index 4ba8d6d69f..fc1cf8e6e2 100644 --- a/source/dnode/vnode/src/inc/tq.h +++ b/source/dnode/vnode/src/inc/tq.h @@ -45,27 +45,10 @@ extern "C" { typedef struct STqOffsetStore STqOffsetStore; // tqPush - -// typedef struct { -// // msg info -// int64_t consumerId; -// int64_t reqOffset; -// int64_t processedVer; -// int32_t epoch; -// // rpc info -// int64_t reqId; -// SRpcHandleInfo rpcInfo; -// tmr_h timerId; -// int8_t tmrStopped; -// // exec -// int8_t inputStatus; -// int8_t execStatus; -// SStreamQueue inputQ; -// SRWLatch lock; -// } STqPushHandle; +#define EXTRACT_DATA_FROM_WAL_ID (-1) +#define STREAM_TASK_STATUS_CHECK_ID (-2) // tqExec - typedef struct { char* qmsg; // SubPlanToString } STqExecCol; @@ -181,6 +164,7 @@ int32_t tqOffsetRestoreFromFile(STqOffsetStore* pStore, const char* fname); // tqStream int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t ver); int32_t tqStreamTasksScanWal(STQ* pTq); +int32_t tqStreamTasksStatusCheck(STQ* pTq); // tq util int32_t extractDelDataBlock(const void* pData, int32_t len, int64_t ver, SStreamRefDataBlock** pRefBlock); diff --git a/source/dnode/vnode/src/inc/vnodeInt.h b/source/dnode/vnode/src/inc/vnodeInt.h index cd97bd5753..f63bddb8b8 100644 --- a/source/dnode/vnode/src/inc/vnodeInt.h +++ b/source/dnode/vnode/src/inc/vnodeInt.h @@ -218,6 +218,7 @@ int tqPushMsg(STQ*, void* msg, int32_t msgLen, tmsg_t msgType, int64_t ver); int tqRegisterPushHandle(STQ* pTq, void* handle, SRpcMsg* pMsg); int tqUnregisterPushHandle(STQ* pTq, void* pHandle); int tqStartStreamTasks(STQ* pTq); // restore all stream tasks after vnode launching completed. +int tqCheckforStreamStatus(STQ* pTq); int tqCommit(STQ*); int32_t tqUpdateTbUidList(STQ* pTq, const SArray* tbUidList, bool isAdd); diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 1d29f245e2..dd36075eb1 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -18,8 +18,6 @@ // 0: not init // 1: already inited // 2: wait to be inited or cleaup -#define WAL_READ_TASKS_ID (-1) - static int32_t tqInitialize(STQ* pTq); static FORCE_INLINE bool tqIsHandleExec(STqHandle* pHandle) { return TMQ_HANDLE_STATUS_EXEC == pHandle->status; } @@ -819,9 +817,6 @@ int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t ver) { pTask->dataRange.range.maxVer = ver; pTask->dataRange.range.minVer = ver; - // expand executor - pTask->status.taskStatus = TASK_STATUS__WAIT_DOWNSTREAM; - if (pTask->info.taskLevel == TASK_LEVEL__SOURCE) { pTask->pState = streamStateOpen(pTq->pStreamMeta->path, pTask, false, -1, -1); if (pTask->pState == NULL) { @@ -903,10 +898,11 @@ int32_t tqProcessStreamTaskCheckReq(STQ* pTq, SRpcMsg* pMsg) { SDecoder decoder; tDecoderInit(&decoder, (uint8_t*)msgBody, msgLen); - tDecodeSStreamTaskCheckReq(&decoder, &req); + tDecodeStreamTaskCheckReq(&decoder, &req); tDecoderClear(&decoder); - int32_t taskId = req.downstreamTaskId; + int32_t taskId = req.downstreamTaskId; + SStreamTaskCheckRsp rsp = { .reqId = req.reqId, .streamId = req.streamId, @@ -924,7 +920,8 @@ int32_t tqProcessStreamTaskCheckReq(STQ* pTq, SRpcMsg* pMsg) { streamMetaReleaseTask(pTq->pStreamMeta, pTask); tqDebug("s-task:%s recv task check req(reqId:0x%" PRIx64 ") task:0x%x (vgId:%d), status:%s, rsp status %d", - pTask->id.idStr, rsp.reqId, rsp.upstreamTaskId, rsp.upstreamNodeId, streamGetTaskStatusStr(pTask->status.taskStatus), rsp.status); + pTask->id.idStr, rsp.reqId, rsp.upstreamTaskId, rsp.upstreamNodeId, + streamGetTaskStatusStr(pTask->status.taskStatus), rsp.status); } else { rsp.status = 0; tqDebug("tq recv task check(taskId:0x%x not built yet) req(reqId:0x%" PRIx64 ") from task:0x%x (vgId:%d), rsp status %d", @@ -935,7 +932,7 @@ int32_t tqProcessStreamTaskCheckReq(STQ* pTq, SRpcMsg* pMsg) { int32_t code; int32_t len; - tEncodeSize(tEncodeSStreamTaskCheckRsp, &rsp, len, code); + tEncodeSize(tEncodeStreamTaskCheckRsp, &rsp, len, code); if (code < 0) { tqError("vgId:%d failed to encode task check rsp, task:0x%x", pTq->pStreamMeta->vgId, taskId); return -1; @@ -946,7 +943,7 @@ int32_t tqProcessStreamTaskCheckReq(STQ* pTq, SRpcMsg* pMsg) { void* abuf = POINTER_SHIFT(buf, sizeof(SMsgHead)); tEncoderInit(&encoder, (uint8_t*)abuf, len); - tEncodeSStreamTaskCheckRsp(&encoder, &rsp); + tEncodeStreamTaskCheckRsp(&encoder, &rsp); tEncoderClear(&encoder); SRpcMsg rspMsg = {.code = 0, .pCont = buf, .contLen = sizeof(SMsgHead) + len, .info = pMsg->info}; @@ -961,7 +958,7 @@ int32_t tqProcessStreamTaskCheckRsp(STQ* pTq, int64_t sversion, char* msg, int32 SDecoder decoder; tDecoderInit(&decoder, (uint8_t*)msg, msgLen); - code = tDecodeSStreamTaskCheckRsp(&decoder, &rsp); + code = tDecodeStreamTaskCheckRsp(&decoder, &rsp); if (code < 0) { tDecoderClear(&decoder); @@ -969,8 +966,8 @@ int32_t tqProcessStreamTaskCheckRsp(STQ* pTq, int64_t sversion, char* msg, int32 } tDecoderClear(&decoder); - tqDebug("tq recv task check rsp(reqId:0x%" PRIx64 ") %d (vgId:%d) check req from task:0x%x (vgId:%d), status %d", - rsp.reqId, rsp.downstreamTaskId, rsp.downstreamNodeId, rsp.upstreamTaskId, rsp.upstreamNodeId, rsp.status); + tqDebug("tq task:0x%x (vgId:%d) recv check rsp(reqId:0x%" PRIx64 ") from 0x%x (vgId:%d) status %d", + rsp.upstreamTaskId, rsp.upstreamNodeId, rsp.reqId, rsp.downstreamTaskId, rsp.downstreamNodeId, rsp.status); SStreamTask* pTask = streamMetaAcquireTask(pTq->pStreamMeta, rsp.upstreamTaskId); if (pTask == NULL) { @@ -1034,7 +1031,7 @@ int32_t tqProcessTaskDeployReq(STQ* pTq, int64_t sversion, char* msg, int32_t ms // calculate the correct start time window, and start the handle the history data for the main task. if (pTask->historyTaskId.taskId != 0) { // launch the history fill stream task - streamTaskStartHistoryTask(pTask, sversion); + streamTaskStartHistoryTask(pTask); // launch current task SHistDataRange* pRange = &pTask->dataRange; @@ -1056,11 +1053,12 @@ int32_t tqProcessTaskDeployReq(STQ* pTq, int64_t sversion, char* msg, int32_t ms pTask->id.idStr, pRange->window.skey, pRange->window.ekey, pRange->range.minVer, pRange->range.maxVer); } + ASSERT(pTask->status.checkDownstream == 0); streamTaskCheckDownstreamTasks(pTask); } - tqDebug("vgId:%d s-task:%s is deployed and add meta from mnd, status:%d, numOfTasks:%d", vgId, pTask->id.idStr, - pTask->status.taskStatus, numOfTasks); + tqDebug("vgId:%d s-task:%s is deployed and add into meta, status:%s, numOfTasks:%d", vgId, pTask->id.idStr, + streamGetTaskStatusStr(pTask->status.taskStatus), numOfTasks); return 0; } @@ -1120,8 +1118,7 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { ASSERT(pStreamTask->info.taskLevel == TASK_LEVEL__SOURCE); // wait for the stream task get ready for scan history data - while (pStreamTask->status.taskStatus == TASK_STATUS__WAIT_DOWNSTREAM || - pStreamTask->status.taskStatus == TASK_STATUS__SCAN_HISTORY) { + while (pStreamTask->status.checkDownstream == 0 || pStreamTask->status.taskStatus == TASK_STATUS__SCAN_HISTORY) { tqDebug("s-task:%s level:%d not ready for halt, wait for 100ms and recheck", pStreamTask->id.idStr, pStreamTask->info.taskLevel); taosMsleep(100); @@ -1155,8 +1152,17 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { // 5. resume the related stream task. streamTryExec(pTask); - streamMetaReleaseTask(pMeta, pStreamTask); + pTask->status.taskStatus = TASK_STATUS__DROPPING; + tqDebug("s-task:%s set status to be dropping", pTask->id.idStr); + + streamMetaSaveTask(pMeta, pTask); streamMetaReleaseTask(pMeta, pTask); +// streamMetaRemoveTask(pMeta, pTask->id.taskId); + + streamMetaReleaseTask(pMeta, pStreamTask); + if (streamMetaCommit(pTask->pMeta) < 0) { + + } } else { // todo update the chkInfo version for current task. // this task has an associated history stream task, so we need to scan wal from the end version of @@ -1338,7 +1344,12 @@ int32_t tqProcessTaskRunReq(STQ* pTq, SRpcMsg* pMsg) { int32_t taskId = pReq->taskId; int32_t vgId = TD_VID(pTq->pVnode); - if (taskId == WAL_READ_TASKS_ID) { // all tasks are extracted submit data from the wal + if (taskId == STREAM_TASK_STATUS_CHECK_ID) { + tqStreamTasksStatusCheck(pTq); + return 0; + } + + if (taskId == EXTRACT_DATA_FROM_WAL_ID) { // all tasks are extracted submit data from the wal tqStreamTasksScanWal(pTq); return 0; } @@ -1553,43 +1564,3 @@ FAIL: int32_t tqCheckLogInWal(STQ* pTq, int64_t sversion) { return sversion <= pTq->walLogLastVer; } -int32_t tqStartStreamTasks(STQ* pTq) { - int32_t vgId = TD_VID(pTq->pVnode); - SStreamMeta* pMeta = pTq->pStreamMeta; - - taosWLockLatch(&pMeta->lock); - - int32_t numOfTasks = taosArrayGetSize(pMeta->pTaskList); - if (numOfTasks == 0) { - tqInfo("vgId:%d no stream tasks exist", vgId); - taosWUnLockLatch(&pMeta->lock); - return 0; - } - - pMeta->walScanCounter += 1; - - if (pMeta->walScanCounter > 1) { - tqDebug("vgId:%d wal read task has been launched, remain scan times:%d", vgId, pMeta->walScanCounter); - taosWUnLockLatch(&pMeta->lock); - 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()); - taosWUnLockLatch(&pMeta->lock); - return -1; - } - - tqDebug("vgId:%d create msg to start wal scan to launch stream tasks, numOfTasks:%d", vgId, numOfTasks); - pRunReq->head.vgId = vgId; - pRunReq->streamId = 0; - pRunReq->taskId = WAL_READ_TASKS_ID; - - SRpcMsg msg = {.msgType = TDMT_STREAM_TASK_RUN, .pCont = pRunReq, .contLen = sizeof(SStreamTaskRunReq)}; - tmsgPutToQueue(&pTq->pVnode->msgCb, STREAM_QUEUE, &msg); - taosWUnLockLatch(&pMeta->lock); - - return 0; -} diff --git a/source/dnode/vnode/src/tq/tqRestore.c b/source/dnode/vnode/src/tq/tqRestore.c index 1a9a4ec612..6fbc4197ee 100644 --- a/source/dnode/vnode/src/tq/tqRestore.c +++ b/source/dnode/vnode/src/tq/tqRestore.c @@ -16,6 +16,7 @@ #include "tq.h" static int32_t createStreamTaskRunReq(SStreamMeta* pStreamMeta, bool* pScanIdle); +static int32_t doSetOffsetForWalReader(SStreamTask *pTask, int32_t vgId); // this function should be executed by stream threads. // extract submit block from WAL, and add them into the input queue for the sources tasks. @@ -57,7 +58,110 @@ int32_t tqStreamTasksScanWal(STQ* pTq) { return 0; } -static int32_t doSetOffsetForWalReader(SStreamTask *pTask, int32_t vgId) { +int32_t tqStreamTasksStatusCheck(STQ* pTq) { + int32_t vgId = TD_VID(pTq->pVnode); + SStreamMeta* pMeta = pTq->pStreamMeta; + + int32_t numOfTasks = taosArrayGetSize(pMeta->pTaskList); + tqDebug("vgId:%d start to check all (%d) stream tasks downstream status", vgId, numOfTasks); + if (numOfTasks == 0) { + return TSDB_CODE_SUCCESS; + } + + SArray* pTaskList = NULL; + taosWLockLatch(&pMeta->lock); + pTaskList = taosArrayDup(pMeta->pTaskList, NULL); + taosWUnLockLatch(&pMeta->lock); + + for (int32_t i = 0; i < numOfTasks; ++i) { + int32_t* pTaskId = taosArrayGet(pTaskList, i); + SStreamTask* pTask = streamMetaAcquireTask(pMeta, *pTaskId); + if (pTask == NULL) { + continue; + } + + streamTaskCheckDownstreamTasks(pTask); + streamMetaReleaseTask(pMeta, pTask); + } + + return 0; +} + +int32_t tqCheckforStreamStatus(STQ* pTq) { + int32_t vgId = TD_VID(pTq->pVnode); + SStreamMeta* pMeta = pTq->pStreamMeta; + + taosWLockLatch(&pMeta->lock); + + int32_t numOfTasks = taosArrayGetSize(pMeta->pTaskList); + if (numOfTasks == 0) { + tqInfo("vgId:%d no stream tasks exist", vgId); + taosWUnLockLatch(&pMeta->lock); + 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()); + taosWUnLockLatch(&pMeta->lock); + return -1; + } + + tqDebug("vgId:%d check for stream tasks status, numOfTasks:%d", vgId, numOfTasks); + pRunReq->head.vgId = vgId; + pRunReq->streamId = 0; + pRunReq->taskId = STREAM_TASK_STATUS_CHECK_ID; + + SRpcMsg msg = {.msgType = TDMT_STREAM_TASK_RUN, .pCont = pRunReq, .contLen = sizeof(SStreamTaskRunReq)}; + tmsgPutToQueue(&pTq->pVnode->msgCb, STREAM_QUEUE, &msg); + taosWUnLockLatch(&pMeta->lock); + + return 0; +} + +int32_t tqStartStreamTasks(STQ* pTq) { + int32_t vgId = TD_VID(pTq->pVnode); + SStreamMeta* pMeta = pTq->pStreamMeta; + + taosWLockLatch(&pMeta->lock); + + int32_t numOfTasks = taosArrayGetSize(pMeta->pTaskList); + if (numOfTasks == 0) { + tqInfo("vgId:%d no stream tasks exist", vgId); + taosWUnLockLatch(&pMeta->lock); + return 0; + } + + pMeta->walScanCounter += 1; + + if (pMeta->walScanCounter > 1) { + tqDebug("vgId:%d wal read task has been launched, remain scan times:%d", vgId, pMeta->walScanCounter); + taosWUnLockLatch(&pMeta->lock); + 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()); + taosWUnLockLatch(&pMeta->lock); + return -1; + } + + tqDebug("vgId:%d create msg to start wal scan to launch stream tasks, numOfTasks:%d", vgId, numOfTasks); + pRunReq->head.vgId = vgId; + pRunReq->streamId = 0; + pRunReq->taskId = EXTRACT_DATA_FROM_WAL_ID; + + SRpcMsg msg = {.msgType = TDMT_STREAM_TASK_RUN, .pCont = pRunReq, .contLen = sizeof(SStreamTaskRunReq)}; + tmsgPutToQueue(&pTq->pVnode->msgCb, STREAM_QUEUE, &msg); + taosWUnLockLatch(&pMeta->lock); + + return 0; +} + +int32_t doSetOffsetForWalReader(SStreamTask *pTask, int32_t vgId) { // seek the stored version and extract data from WAL int64_t firstVer = walReaderGetValidFirstVer(pTask->exec.pWalReader); if (pTask->chkInfo.currentVer < firstVer) { @@ -192,3 +296,4 @@ int32_t createStreamTaskRunReq(SStreamMeta* pStreamMeta, bool* pScanIdle) { taosArrayDestroy(pTaskList); return 0; } + diff --git a/source/dnode/vnode/src/vnd/vnodeSync.c b/source/dnode/vnode/src/vnd/vnodeSync.c index 29f1ddc50f..045a7298c1 100644 --- a/source/dnode/vnode/src/vnd/vnodeSync.c +++ b/source/dnode/vnode/src/vnd/vnodeSync.c @@ -554,7 +554,7 @@ static void vnodeRestoreFinish(const SSyncFSM *pFsm, const SyncIndex commitIdx) vInfo("vgId:%d, not launch stream tasks, since stream tasks are disabled", pVnode->config.vgId); } else { vInfo("vgId:%d start to launch stream tasks", pVnode->config.vgId); - tqStartStreamTasks(pVnode->pTq); + tqCheckforStreamStatus(pVnode->pTq); } } diff --git a/source/libs/stream/src/streamDispatch.c b/source/libs/stream/src/streamDispatch.c index ae3f094d12..95f68cce2d 100644 --- a/source/libs/stream/src/streamDispatch.c +++ b/source/libs/stream/src/streamDispatch.c @@ -241,7 +241,7 @@ int32_t streamDispatchCheckMsg(SStreamTask* pTask, const SStreamTaskCheckReq* pR SRpcMsg msg = {0}; int32_t tlen; - tEncodeSize(tEncodeSStreamTaskCheckReq, pReq, tlen, code); + tEncodeSize(tEncodeStreamTaskCheckReq, pReq, tlen, code); if (code < 0) { return -1; } @@ -256,7 +256,7 @@ int32_t streamDispatchCheckMsg(SStreamTask* pTask, const SStreamTaskCheckReq* pR SEncoder encoder; tEncoderInit(&encoder, abuf, tlen); - if ((code = tEncodeSStreamTaskCheckReq(&encoder, pReq)) < 0) { + if ((code = tEncodeStreamTaskCheckReq(&encoder, pReq)) < 0) { rpcFreeCont(buf); return code; } diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index 2cbe72e0be..aee3070d7f 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -389,8 +389,6 @@ int32_t streamExecForAll(SStreamTask* pTask) { } if (pInput == NULL) { - qDebug("789 %s", pTask->id.idStr); - if (pTask->info.fillHistory && pTask->status.transferState) { // todo transfer task state here @@ -414,14 +412,18 @@ int32_t streamExecForAll(SStreamTask* pTask) { // OR wait for the inputQ && outputQ of agg tasks are all consumed, and then start the state transfer - qDebug("s-task:%s no need to update time window", pStreamTask->id.idStr); + qDebug("s-task:%s no need to update time window, for non-source task", pStreamTask->id.idStr); } pTimeWindow->skey = INT64_MIN; streamSetStatusNormal(pStreamTask); - streamSchedExec(pStreamTask); + streamMetaSaveTask(pTask->pMeta, pStreamTask); + if (streamMetaCommit(pTask->pMeta)) { + // persistent to disk for + } + streamSchedExec(pStreamTask); streamMetaReleaseTask(pTask->pMeta, pStreamTask); } diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index aa3f1dbe07..b7bc35d13d 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -377,6 +377,7 @@ int32_t streamLoadTasks(SStreamMeta* pMeta, int64_t ver) { taosMemoryFree(pTask); continue; } + if (taosHashPut(pMeta->pTasks, &pTask->id.taskId, sizeof(pTask->id.taskId), &pTask, sizeof(void*)) < 0) { tdbFree(pKey); tdbFree(pVal); @@ -385,12 +386,7 @@ int32_t streamLoadTasks(SStreamMeta* pMeta, int64_t ver) { return -1; } - // todo handle the fill history task - ASSERT(0); - if (pTask->info.fillHistory) { - ASSERT(pTask->status.taskStatus == TASK_STATUS__WAIT_DOWNSTREAM); - streamTaskCheckDownstreamTasks(pTask); - } + ASSERT(pTask->status.checkDownstream == 0); } tdbFree(pKey); diff --git a/source/libs/stream/src/streamRecover.c b/source/libs/stream/src/streamRecover.c index 5e278cd7f6..82b9941c96 100644 --- a/source/libs/stream/src/streamRecover.c +++ b/source/libs/stream/src/streamRecover.c @@ -28,32 +28,45 @@ const char* streamGetTaskStatusStr(int32_t status) { } } -int32_t streamTaskLaunchRecover(SStreamTask* pTask) { +static int32_t doLaunchScanHistoryTask(SStreamTask* pTask) { + SVersionRange* pRange = &pTask->dataRange.range; + + qDebug("s-task:%s vgId:%d task status:%s and start to scan-history-data task, ver:%" PRId64 " - %" PRId64, + pTask->id.idStr, pTask->info.nodeId, streamGetTaskStatusStr(pTask->status.taskStatus), + pTask->dataRange.range.minVer, pTask->dataRange.range.maxVer); + + streamSetParamForRecover(pTask); + streamSourceRecoverPrepareStep1(pTask, pRange, &pTask->dataRange.window); + + SStreamRecoverStep1Req req; + streamBuildSourceRecover1Req(pTask, &req); + int32_t len = sizeof(SStreamRecoverStep1Req); + + void* serializedReq = rpcMallocCont(len); + if (serializedReq == NULL) { + return -1; + } + + memcpy(serializedReq, &req, len); + + SRpcMsg rpcMsg = {.contLen = len, .pCont = serializedReq, .msgType = TDMT_VND_STREAM_SCAN_HISTORY}; + if (tmsgPutToQueue(pTask->pMsgCb, STREAM_QUEUE, &rpcMsg) < 0) { + /*ASSERT(0);*/ + } + + return 0; +} + +int32_t streamTaskLaunchScanHistory(SStreamTask* pTask) { if (pTask->info.taskLevel == TASK_LEVEL__SOURCE) { - SVersionRange* pRange = &pTask->dataRange.range; - qDebug("s-task:%s vgId:%d task status:%s and start to scan-history-data task, ver:%" PRId64 " - %" PRId64, - pTask->id.idStr, pTask->info.nodeId, streamGetTaskStatusStr(pTask->status.taskStatus), - pTask->dataRange.range.minVer, pTask->dataRange.range.maxVer); - - streamSetParamForRecover(pTask); - streamSourceRecoverPrepareStep1(pTask, pRange, &pTask->dataRange.window); - - SStreamRecoverStep1Req req; - streamBuildSourceRecover1Req(pTask, &req); - int32_t len = sizeof(SStreamRecoverStep1Req); - - void* serializedReq = rpcMallocCont(len); - if (serializedReq == NULL) { - return -1; + if (pTask->status.taskStatus == TASK_STATUS__SCAN_HISTORY) { + return doLaunchScanHistoryTask(pTask); + } else { + ASSERT(pTask->status.taskStatus == TASK_STATUS__NORMAL); + qDebug("s-task:%s no need to scan-history-data, status:%s, sched-status:%d, ver:%" PRId64, pTask->id.idStr, + streamGetTaskStatusStr(pTask->status.taskStatus), pTask->status.schedStatus, + walReaderGetCurrentVer(pTask->exec.pWalReader)); } - - memcpy(serializedReq, &req, len); - - SRpcMsg rpcMsg = { .contLen = len, .pCont = serializedReq, .msgType = TDMT_VND_STREAM_SCAN_HISTORY }; - if (tmsgPutToQueue(pTask->pMsgCb, STREAM_QUEUE, &rpcMsg) < 0) { - /*ASSERT(0);*/ - } - } else if (pTask->info.taskLevel == TASK_LEVEL__AGG) { streamSetStatusNormal(pTask); streamSetParamForRecover(pTask); @@ -68,9 +81,9 @@ int32_t streamTaskLaunchRecover(SStreamTask* pTask) { // check status int32_t streamTaskCheckDownstreamTasks(SStreamTask* pTask) { - qDebug("s-task:%s in fill history stage, ver:%" PRId64 "-%"PRId64" window:%" PRId64"-%"PRId64, pTask->id.idStr, - pTask->dataRange.range.minVer, pTask->dataRange.range.maxVer, pTask->dataRange.window.skey, - pTask->dataRange.window.ekey); + SHistDataRange* pRange = &pTask->dataRange; + qDebug("s-task:%s check downstream tasks, ver:%" PRId64 "-%" PRId64 " window:%" PRId64 "-%" PRId64, + pTask->id.idStr, pRange->range.minVer, pRange->range.maxVer, pRange->window.skey, pRange->window.ekey); SStreamTaskCheckReq req = { .streamId = pTask->id.streamId, @@ -108,8 +121,10 @@ int32_t streamTaskCheckDownstreamTasks(SStreamTask* pTask) { streamDispatchCheckMsg(pTask, &req, pVgInfo->vgId, &pVgInfo->epSet); } } else { - qDebug("s-task:%s (vgId:%d) direct launch recover since no downstream", pTask->id.idStr, pTask->info.nodeId); - streamTaskLaunchRecover(pTask); + pTask->status.checkDownstream = 1; + qDebug("s-task:%s (vgId:%d) set downstream task checked for task without downstream tasks, try to launch scan-history-data, status:%s", + pTask->id.idStr, pTask->info.nodeId, streamGetTaskStatusStr(pTask->status.taskStatus)); + streamTaskLaunchScanHistory(pTask); } return 0; @@ -178,8 +193,15 @@ int32_t streamProcessCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* pRs taosArrayDestroy(pTask->checkReqIds); pTask->checkReqIds = NULL; - qDebug("s-task:%s all %d downstream tasks are ready, now enter into scan-history-data stage", id, numOfReqs); - streamTaskLaunchRecover(pTask); + if (pTask->status.taskStatus == TASK_STATUS__SCAN_HISTORY) { + qDebug("s-task:%s all %d downstream tasks are ready, now enter into scan-history-data stage, status:%s", id, numOfReqs, + streamGetTaskStatusStr(pTask->status.taskStatus)); + streamTaskLaunchScanHistory(pTask); + } else { + ASSERT(pTask->status.taskStatus == TASK_STATUS__NORMAL); + qDebug("s-task:%s fixed downstream task is ready, now ready for data from wal, status:%s", id, + streamGetTaskStatusStr(pTask->status.taskStatus)); + } } else { qDebug("s-task:%s (vgId:%d) recv check rsp from task:0x%x (vgId:%d) status:%d, remain not ready:%d", id, pRsp->upstreamNodeId, pRsp->downstreamTaskId, pRsp->downstreamNodeId, pRsp->status, left); @@ -189,13 +211,20 @@ int32_t streamProcessCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* pRs return -1; } + ASSERT(pTask->status.checkDownstream == 0); + + pTask->status.checkDownstream = 1; ASSERT(pTask->status.taskStatus != TASK_STATUS__HALT); - pTask->status.taskStatus = TASK_STATUS__SCAN_HISTORY; - qDebug("s-task:%s fixed downstream task is ready, now enter into scan-history-data stage, status:%s", id, - streamGetTaskStatusStr(pTask->status.taskStatus)); - - streamTaskLaunchRecover(pTask); + if (pTask->status.taskStatus == TASK_STATUS__SCAN_HISTORY) { + qDebug("s-task:%s fixed downstream task is ready, now enter into scan-history-data stage, status:%s", id, + streamGetTaskStatusStr(pTask->status.taskStatus)); + streamTaskLaunchScanHistory(pTask); + } else { + ASSERT(pTask->status.taskStatus == TASK_STATUS__NORMAL); + qDebug("s-task:%s fixed downstream task is ready, now ready for data from wal, status:%s", id, + streamGetTaskStatusStr(pTask->status.taskStatus)); + } } else { ASSERT(0); } @@ -401,7 +430,7 @@ static void doCheckDownstreamStatus(SStreamTask* pTask, SStreamTask* pHTask) { if (pTask->info.taskLevel == TASK_LEVEL__SOURCE) { qDebug("s-task:%s set the launch condition for fill history s-task:%s, window:%" PRId64 " - %" PRId64 - " ver range:%" PRId64 " - %" PRId64, + " ver range:%" PRId64 " - %" PRId64, pTask->id.idStr, pHTask->id.idStr, pHTask->dataRange.window.skey, pHTask->dataRange.window.ekey, pHTask->dataRange.range.minVer, pHTask->dataRange.range.maxVer); } else { @@ -430,7 +459,7 @@ static void tryLaunchHistoryTask(void* param, void* tmrId) { // todo fix the bug: 2. race condition // an fill history task needs to be started. -int32_t streamTaskStartHistoryTask(SStreamTask* pTask, int64_t ver) { +int32_t streamTaskStartHistoryTask(SStreamTask* pTask) { SStreamMeta* pMeta = pTask->pMeta; if (pTask->historyTaskId.taskId == 0) { return TSDB_CODE_SUCCESS; @@ -479,11 +508,13 @@ int32_t streamTaskScanHistoryDataComplete(SStreamTask* pTask) { /*code = */streamSetStatusNormal(pTask); atomic_store_8(&pTask->status.schedStatus, TASK_SCHED_STATUS__INACTIVE); + + // todo check rsp streamMetaSaveTask(pMeta, pTask); return 0; } -int32_t tEncodeSStreamTaskCheckReq(SEncoder* pEncoder, const SStreamTaskCheckReq* pReq) { +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; @@ -496,7 +527,7 @@ int32_t tEncodeSStreamTaskCheckReq(SEncoder* pEncoder, const SStreamTaskCheckReq return pEncoder->pos; } -int32_t tDecodeSStreamTaskCheckReq(SDecoder* pDecoder, SStreamTaskCheckReq* pReq) { +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; @@ -509,7 +540,7 @@ int32_t tDecodeSStreamTaskCheckReq(SDecoder* pDecoder, SStreamTaskCheckReq* pReq return 0; } -int32_t tEncodeSStreamTaskCheckRsp(SEncoder* pEncoder, const SStreamTaskCheckRsp* pRsp) { +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; @@ -523,7 +554,7 @@ int32_t tEncodeSStreamTaskCheckRsp(SEncoder* pEncoder, const SStreamTaskCheckRsp return pEncoder->pos; } -int32_t tDecodeSStreamTaskCheckRsp(SDecoder* pDecoder, SStreamTaskCheckRsp* pRsp) { +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; diff --git a/source/libs/stream/src/streamTask.c b/source/libs/stream/src/streamTask.c index 0ab096aea3..06da72188c 100644 --- a/source/libs/stream/src/streamTask.c +++ b/source/libs/stream/src/streamTask.c @@ -42,6 +42,7 @@ SStreamTask* tNewStreamTask(int64_t streamId, int8_t taskLevel, int8_t fillHisto pTask->id.idStr = taosStrdup(buf); pTask->status.schedStatus = TASK_SCHED_STATUS__INACTIVE; + pTask->status.taskStatus = TASK_STATUS__SCAN_HISTORY; pTask->inputStatus = TASK_INPUT_STATUS__NORMAL; pTask->outputStatus = TASK_OUTPUT_STATUS__NORMAL; From 528249266331e9e33746086116a3fc81cfda90a6 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 14 Jun 2023 10:35:03 +0800 Subject: [PATCH 25/83] refactor: do some internal refactor. --- source/libs/stream/src/streamExec.c | 2 -- 1 file changed, 2 deletions(-) diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index aee3070d7f..b717ea456a 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -383,8 +383,6 @@ int32_t streamExecForAll(SStreamTask* pTask) { if (pInput) { streamFreeQitem(pInput); } - - qDebug("456 %s", pTask->id.idStr); return 0; } From 11f0c3b33648e7f6977f2075efbbab949ded328b Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 14 Jun 2023 10:46:46 +0800 Subject: [PATCH 26/83] refactor: do some internal refactor. --- include/libs/stream/tstream.h | 7 ++--- source/dnode/snode/src/snode.c | 2 +- source/libs/stream/inc/streamInc.h | 2 +- source/libs/stream/src/streamDispatch.c | 6 ++-- source/libs/stream/src/streamRecover.c | 37 +++++++++++-------------- 5 files changed, 24 insertions(+), 30 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 6ed97ac547..223fd3c3b7 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -339,7 +339,7 @@ struct SStreamTask { SStreamState* pState; // state backend // the followings attributes don't be serialized - int32_t recoverTryingDownstream; + int32_t notReadyTasks; int32_t numOfWaitingUpstream; int64_t checkReqId; SArray* checkReqIds; // shuffle @@ -576,7 +576,7 @@ int32_t streamTaskStartHistoryTask(SStreamTask* pTask); int32_t streamTaskScanHistoryDataComplete(SStreamTask* pTask); // common -int32_t streamSetParamForRecover(SStreamTask* pTask); +int32_t streamSetParamForScanHistoryData(SStreamTask* pTask); int32_t streamRestoreParam(SStreamTask* pTask); int32_t streamSetStatusNormal(SStreamTask* pTask); const char* streamGetTaskStatusStr(int32_t status); @@ -585,8 +585,7 @@ const char* streamGetTaskStatusStr(int32_t status); int32_t streamSourceRecoverPrepareStep1(SStreamTask* pTask, SVersionRange *pVerRange, STimeWindow* pWindow); int32_t streamBuildSourceRecover1Req(SStreamTask* pTask, SStreamRecoverStep1Req* pReq); int32_t streamSourceRecoverScanStep1(SStreamTask* pTask); -int32_t streamBuildSourceRecover2Req(SStreamTask* pTask, SStreamRecoverStep2Req* pReq); -int32_t streamSourceRecoverScanStep2(SStreamTask* pTask, int64_t ver); +//int32_t streamSourceRecoverScanStep2(SStreamTask* pTask, int64_t ver); int32_t streamDispatchScanHistoryFinishMsg(SStreamTask* pTask); int32_t streamDispatchTransferStateMsg(SStreamTask* pTask); diff --git a/source/dnode/snode/src/snode.c b/source/dnode/snode/src/snode.c index 41fb50a0df..42bb606c4e 100644 --- a/source/dnode/snode/src/snode.c +++ b/source/dnode/snode/src/snode.c @@ -165,7 +165,7 @@ int32_t sndProcessTaskDeployReq(SSnode *pSnode, char *msg, int32_t msgLen) { // 3.go through recover steps to fill history if (pTask->info.fillHistory) { - streamSetParamForRecover(pTask); + streamSetParamForScanHistoryData(pTask); streamAggRecoverPrepare(pTask); } diff --git a/source/libs/stream/inc/streamInc.h b/source/libs/stream/inc/streamInc.h index b3a28aea43..aeb35d9ad5 100644 --- a/source/libs/stream/inc/streamInc.h +++ b/source/libs/stream/inc/streamInc.h @@ -49,7 +49,7 @@ int32_t tEncodeStreamRetrieveReq(SEncoder* pEncoder, const SStreamRetrieveReq* p int32_t streamDispatchAllBlocks(SStreamTask* pTask, const SStreamDataBlock* pData); int32_t streamDispatchCheckMsg(SStreamTask* pTask, const SStreamTaskCheckReq* pReq, int32_t nodeId, SEpSet* pEpSet); -int32_t streamDoDispatchRecoverFinishMsg(SStreamTask* pTask, const SStreamRecoverFinishReq* pReq, int32_t vgId, +int32_t streamDoDispatchScanHistoryFinishMsg(SStreamTask* pTask, const SStreamRecoverFinishReq* pReq, int32_t vgId, SEpSet* pEpSet); SStreamQueueItem* streamMergeQueueItem(SStreamQueueItem* dst, SStreamQueueItem* pElem); diff --git a/source/libs/stream/src/streamDispatch.c b/source/libs/stream/src/streamDispatch.c index 95f68cce2d..d48bfbb189 100644 --- a/source/libs/stream/src/streamDispatch.c +++ b/source/libs/stream/src/streamDispatch.c @@ -13,7 +13,7 @@ * along with this program. If not, see . */ -#include +#include "ttimer.h" #include "streamInc.h" #define MAX_BLOCK_NAME_NUM 1024 @@ -274,8 +274,8 @@ int32_t streamDispatchCheckMsg(SStreamTask* pTask, const SStreamTaskCheckReq* pR return 0; } -int32_t streamDoDispatchRecoverFinishMsg(SStreamTask* pTask, const SStreamRecoverFinishReq* pReq, int32_t vgId, - SEpSet* pEpSet) { +int32_t streamDoDispatchScanHistoryFinishMsg(SStreamTask* pTask, const SStreamRecoverFinishReq* pReq, int32_t vgId, + SEpSet* pEpSet) { void* buf = NULL; int32_t code = -1; SRpcMsg msg = {0}; diff --git a/source/libs/stream/src/streamRecover.c b/source/libs/stream/src/streamRecover.c index 82b9941c96..f59ffa68a0 100644 --- a/source/libs/stream/src/streamRecover.c +++ b/source/libs/stream/src/streamRecover.c @@ -33,9 +33,9 @@ static int32_t doLaunchScanHistoryTask(SStreamTask* pTask) { qDebug("s-task:%s vgId:%d task status:%s and start to scan-history-data task, ver:%" PRId64 " - %" PRId64, pTask->id.idStr, pTask->info.nodeId, streamGetTaskStatusStr(pTask->status.taskStatus), - pTask->dataRange.range.minVer, pTask->dataRange.range.maxVer); + pRange->minVer, pRange->maxVer); - streamSetParamForRecover(pTask); + streamSetParamForScanHistoryData(pTask); streamSourceRecoverPrepareStep1(pTask, pRange, &pTask->dataRange.window); SStreamRecoverStep1Req req; @@ -69,11 +69,11 @@ int32_t streamTaskLaunchScanHistory(SStreamTask* pTask) { } } else if (pTask->info.taskLevel == TASK_LEVEL__AGG) { streamSetStatusNormal(pTask); - streamSetParamForRecover(pTask); + streamSetParamForScanHistoryData(pTask); streamAggRecoverPrepare(pTask); } else if (pTask->info.taskLevel == TASK_LEVEL__SINK) { streamSetStatusNormal(pTask); - qDebug("s-task:%s sink task convert to normal status immediately", pTask->id.idStr); + qDebug("s-task:%s sink task convert to normal immediately", pTask->id.idStr); } return 0; @@ -107,7 +107,7 @@ int32_t streamTaskCheckDownstreamTasks(SStreamTask* pTask) { SArray* vgInfo = pTask->shuffleDispatcher.dbInfo.pVgroupInfos; int32_t numOfVgs = taosArrayGetSize(vgInfo); - pTask->recoverTryingDownstream = numOfVgs; + pTask->notReadyTasks = numOfVgs; pTask->checkReqIds = taosArrayInit(numOfVgs, sizeof(int64_t)); for (int32_t i = 0; i < numOfVgs; i++) { @@ -130,7 +130,7 @@ int32_t streamTaskCheckDownstreamTasks(SStreamTask* pTask) { return 0; } -int32_t streamRecheckOneDownstream(SStreamTask* pTask, const SStreamTaskCheckRsp* pRsp) { +int32_t streamRecheckDownstream(SStreamTask* pTask, const SStreamTaskCheckRsp* pRsp) { SStreamTaskCheckReq req = { .reqId = pRsp->reqId, .streamId = pRsp->streamId, @@ -186,7 +186,7 @@ int32_t streamProcessCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* pRs return -1; } - int32_t left = atomic_sub_fetch_32(&pTask->recoverTryingDownstream, 1); + int32_t left = atomic_sub_fetch_32(&pTask->notReadyTasks, 1); ASSERT(left >= 0); if (left == 0) { @@ -203,8 +203,9 @@ int32_t streamProcessCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* pRs streamGetTaskStatusStr(pTask->status.taskStatus)); } } else { - qDebug("s-task:%s (vgId:%d) recv check rsp from task:0x%x (vgId:%d) status:%d, remain not ready:%d", id, - pRsp->upstreamNodeId, pRsp->downstreamTaskId, pRsp->downstreamNodeId, pRsp->status, left); + int32_t total = taosArrayGetSize(pTask->shuffleDispatcher.dbInfo.pVgroupInfos); + qDebug("s-task:%s (vgId:%d) recv check rsp from task:0x%x (vgId:%d) status:%d, total:%d not ready:%d", id, + pRsp->upstreamNodeId, pRsp->downstreamTaskId, pRsp->downstreamNodeId, pRsp->status, total, left); } } else if (pTask->outputType == TASK_OUTPUT__FIXED_DISPATCH) { if (pRsp->reqId != pTask->checkReqId) { @@ -212,8 +213,8 @@ int32_t streamProcessCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* pRs } ASSERT(pTask->status.checkDownstream == 0); - pTask->status.checkDownstream = 1; + ASSERT(pTask->status.taskStatus != TASK_STATUS__HALT); if (pTask->status.taskStatus == TASK_STATUS__SCAN_HISTORY) { @@ -233,17 +234,18 @@ int32_t streamProcessCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* pRs pRsp->downstreamNodeId); taosMsleep(100); - streamRecheckOneDownstream(pTask, pRsp); + streamRecheckDownstream(pTask, pRsp); } return 0; } // common -int32_t streamSetParamForRecover(SStreamTask* pTask) { +int32_t streamSetParamForScanHistoryData(SStreamTask* pTask) { void* exec = pTask->exec.pExecutor; return qStreamSetParamForRecover(exec); } + int32_t streamRestoreParam(SStreamTask* pTask) { void* exec = pTask->exec.pExecutor; return qStreamRestoreParam(exec); @@ -272,13 +274,6 @@ int32_t streamSourceRecoverScanStep1(SStreamTask* pTask) { return streamScanExec(pTask, 100); } -int32_t streamBuildSourceRecover2Req(SStreamTask* pTask, SStreamRecoverStep2Req* pReq) { - pReq->msgHead.vgId = pTask->info.nodeId; - pReq->streamId = pTask->id.streamId; - pReq->taskId = pTask->id.taskId; - return 0; -} - int32_t streamSourceRecoverScanStep2(SStreamTask* pTask, int64_t ver) { void* exec = pTask->exec.pExecutor; const char* id = pTask->id.idStr; @@ -305,7 +300,7 @@ int32_t streamDispatchScanHistoryFinishMsg(SStreamTask* pTask) { pTask->fixedEpDispatcher.taskId, streamGetTaskStatusStr(pTask->status.taskStatus)); req.taskId = pTask->fixedEpDispatcher.taskId; - streamDoDispatchRecoverFinishMsg(pTask, &req, pTask->fixedEpDispatcher.nodeId, &pTask->fixedEpDispatcher.epSet); + streamDoDispatchScanHistoryFinishMsg(pTask, &req, pTask->fixedEpDispatcher.nodeId, &pTask->fixedEpDispatcher.epSet); } else if (pTask->outputType == TASK_OUTPUT__SHUFFLE_DISPATCH) { SArray* vgInfo = pTask->shuffleDispatcher.dbInfo.pVgroupInfos; int32_t numOfVgs = taosArrayGetSize(vgInfo); @@ -315,7 +310,7 @@ int32_t streamDispatchScanHistoryFinishMsg(SStreamTask* pTask) { for (int32_t i = 0; i < numOfVgs; i++) { SVgroupInfo* pVgInfo = taosArrayGet(vgInfo, i); req.taskId = pVgInfo->taskId; - streamDoDispatchRecoverFinishMsg(pTask, &req, pVgInfo->vgId, &pVgInfo->epSet); + streamDoDispatchScanHistoryFinishMsg(pTask, &req, pVgInfo->vgId, &pVgInfo->epSet); } } From 7c6fbd77728d2fa03b358561a7f32a9e0cda326f Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 14 Jun 2023 14:10:09 +0800 Subject: [PATCH 27/83] enh(stream): do some internal refactor and support secondary scan for history data. --- include/libs/executor/executor.h | 6 +- include/libs/stream/tstream.h | 8 +-- source/dnode/vnode/src/tq/tq.c | 44 +++++++++---- source/libs/executor/src/executor.c | 86 +++++++++++++++----------- source/libs/stream/src/streamRecover.c | 25 ++++---- 5 files changed, 101 insertions(+), 68 deletions(-) diff --git a/include/libs/executor/executor.h b/include/libs/executor/executor.h index 7a8c074283..852257f5df 100644 --- a/include/libs/executor/executor.h +++ b/include/libs/executor/executor.h @@ -220,11 +220,11 @@ void* qExtractReaderFromStreamScanner(void* scanner); int32_t qExtractStreamScanner(qTaskInfo_t tinfo, void** scanner); -int32_t qStreamSetParamForRecover(qTaskInfo_t tinfo); -int32_t qStreamSourceRecoverStep1(qTaskInfo_t tinfo, SVersionRange *pVerRange, STimeWindow* pWindow); +int32_t qSetStreamOperatorOptionForScanHistory(qTaskInfo_t tinfo); +int32_t qStreamSourceScanParamForHistoryScan(qTaskInfo_t tinfo, SVersionRange *pVerRange, STimeWindow* pWindow); int32_t qStreamSourceRecoverStep2(qTaskInfo_t tinfo, int64_t ver); int32_t qStreamRecoverFinish(qTaskInfo_t tinfo); -int32_t qStreamRestoreParam(qTaskInfo_t tinfo); +int32_t qRestoreStreamOperatorOption(qTaskInfo_t tinfo); bool qStreamRecoverScanFinished(qTaskInfo_t tinfo); void qStreamCloseTsdbReader(void* task); void resetTaskInfo(qTaskInfo_t tinfo); diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 223fd3c3b7..0891c35716 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -451,7 +451,7 @@ typedef struct { SMsgHead msgHead; int64_t streamId; int32_t taskId; -} SStreamRecoverStep1Req, SStreamRecoverStep2Req; +} SStreamScanHistoryReq, SStreamRecoverStep2Req; typedef struct { int64_t streamId; @@ -582,9 +582,9 @@ int32_t streamSetStatusNormal(SStreamTask* pTask); const char* streamGetTaskStatusStr(int32_t status); // source level -int32_t streamSourceRecoverPrepareStep1(SStreamTask* pTask, SVersionRange *pVerRange, STimeWindow* pWindow); -int32_t streamBuildSourceRecover1Req(SStreamTask* pTask, SStreamRecoverStep1Req* pReq); -int32_t streamSourceRecoverScanStep1(SStreamTask* pTask); +int32_t streamSetParamForStreamScanner(SStreamTask* pTask, SVersionRange *pVerRange, STimeWindow* pWindow); +int32_t streamBuildSourceRecover1Req(SStreamTask* pTask, SStreamScanHistoryReq* pReq); +int32_t streamSourceScanHistoryData(SStreamTask* pTask); //int32_t streamSourceRecoverScanStep2(SStreamTask* pTask, int64_t ver); int32_t streamDispatchScanHistoryFinishMsg(SStreamTask* pTask); diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index dd36075eb1..a8d712ae01 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1066,10 +1066,9 @@ int32_t tqProcessTaskDeployReq(STQ* pTq, int64_t sversion, char* msg, int32_t ms int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { int32_t code = TSDB_CODE_SUCCESS; char* msg = pMsg->pCont; - int32_t msgLen = pMsg->contLen; SStreamMeta* pMeta = pTq->pStreamMeta; - SStreamRecoverStep1Req* pReq = (SStreamRecoverStep1Req*)msg; + SStreamScanHistoryReq* pReq = (SStreamScanHistoryReq*)msg; SStreamTask* pTask = streamMetaAcquireTask(pMeta, pReq->taskId); if (pTask == NULL) { @@ -1089,16 +1088,14 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { tqDebug("s-task:%s start history data scan stage(step 1), status:%s", pTask->id.idStr, streamGetTaskStatusStr(pTask->status.taskStatus)); int64_t st = taosGetTimestampMs(); - - // todo set the correct status flag - int8_t schedStatus = atomic_val_compare_exchange_8(&pTask->status.schedStatus, TASK_SCHED_STATUS__INACTIVE, - TASK_SCHED_STATUS__WAITING); + int8_t schedStatus = atomic_val_compare_exchange_8(&pTask->status.schedStatus, TASK_SCHED_STATUS__INACTIVE, + TASK_SCHED_STATUS__WAITING); if (schedStatus != TASK_SCHED_STATUS__INACTIVE) { ASSERT(0); return 0; } - streamSourceRecoverScanStep1(pTask); + streamSourceScanHistoryData(pTask); if (atomic_load_8(&pTask->status.taskStatus) == TASK_STATUS__DROPPING) { tqDebug("s-task:%s is dropped, abort recover in step1", pTask->id.idStr); streamMetaReleaseTask(pMeta, pTask); @@ -1124,21 +1121,47 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { taosMsleep(100); } + taosMsleep(10000); + + // now we can stop the stream task execution pStreamTask->status.taskStatus = TASK_STATUS__HALT; tqDebug("s-task:%s level:%d status is set to halt by history scan task:%s", pStreamTask->id.idStr, pStreamTask->info.taskLevel, pTask->id.idStr); // if it's an source task, extract the last version in wal. - int64_t ver = pTask->dataRange.range.maxVer; + int64_t ver = pTask->dataRange.range.maxVer + 1; int64_t latestVer = walReaderGetCurrentVer(pStreamTask->exec.pWalReader); if (latestVer >= ver) { ver = latestVer; - } else { - ASSERT(latestVer == -1); } // 2. do secondary scan of the history data, the time window remain, and the version range is updated to [pTask->dataRange.range.maxVer, ver1] + SVersionRange* pRange = &pTask->dataRange.range; + pRange->minVer = pRange->maxVer + 1; + pRange->maxVer = ver; + if (pRange->minVer == pRange->maxVer) { + tqDebug("s-task:%s no need to perform secondary scan-history-data(step 2), since no new data ingest", pTask->id.idStr); + } else { + tqDebug("s-task:%s level:%d verRange:%" PRId64 " - %" PRId64 + " do secondary scan-history-data after halt the related stream task:%s", + pTask->id.idStr, pTask->info.taskLevel, pRange->minVer, pRange->maxVer, pStreamTask->id.idStr); + + ASSERT(pTask->status.schedStatus == TASK_SCHED_STATUS__WAITING); + st = taosGetTimestampMs(); + + streamSetParamForStreamScanner(pTask, pRange, &pTask->dataRange.window); + + streamSourceScanHistoryData(pTask); + if (atomic_load_8(&pTask->status.taskStatus) == TASK_STATUS__DROPPING) { + tqDebug("s-task:%s is dropped, abort recover in step1", pTask->id.idStr); + streamMetaReleaseTask(pMeta, pTask); + return 0; + } + + el = (taosGetTimestampMs() - st) / 1000.0; + tqDebug("s-task:%s history data scan stage(step 2) ended, elapsed time:%.2fs", pTask->id.idStr, el); + } // 3. notify the downstream tasks to transfer executor state after handle all history blocks. pTask->status.transferState = true; @@ -1177,7 +1200,6 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { } return 0; - } // notify the downstream tasks to transfer executor state after handle all history blocks. diff --git a/source/libs/executor/src/executor.c b/source/libs/executor/src/executor.c index d77323d737..f4135f58d1 100644 --- a/source/libs/executor/src/executor.c +++ b/source/libs/executor/src/executor.c @@ -92,6 +92,7 @@ static int32_t doSetStreamOpOpen(SOperatorInfo* pOperator, char* id) { qError("join not supported for stream block scan, %s" PRIx64, id); return TSDB_CODE_APP_ERROR; } + pOperator->status = OP_NOT_OPENED; return doSetStreamOpOpen(pOperator->pDownstream[0], id); } @@ -869,12 +870,20 @@ int32_t qExtractStreamScanner(qTaskInfo_t tinfo, void** scanner) { } } -int32_t qStreamSourceRecoverStep1(qTaskInfo_t tinfo, SVersionRange *pVerRange, STimeWindow* pWindow) { +int32_t qStreamSourceScanParamForHistoryScan(qTaskInfo_t tinfo, SVersionRange *pVerRange, STimeWindow* pWindow) { SExecTaskInfo* pTaskInfo = (SExecTaskInfo*)tinfo; ASSERT(pTaskInfo->execModel == OPTR_EXEC_MODEL_STREAM); - pTaskInfo->streamInfo.fillHistoryVer = *pVerRange; - pTaskInfo->streamInfo.fillHistoryWindow = *pWindow; - pTaskInfo->streamInfo.recoverStep = STREAM_RECOVER_STEP__PREPARE1; + + SStreamTaskInfo* pStreamInfo = &pTaskInfo->streamInfo; + + pStreamInfo->fillHistoryVer = *pVerRange; + pStreamInfo->fillHistoryWindow = *pWindow; + pStreamInfo->recoverStep = STREAM_RECOVER_STEP__PREPARE1; + + qDebug("%s set param for stream scanner for scan history data, Ver:%" PRId64 " - %" PRId64 ", window:%" PRId64 + " - %" PRId64, + GET_TASKID(pTaskInfo), pStreamInfo->fillHistoryVer.minVer, pStreamInfo->fillHistoryVer.maxVer, pWindow->skey, + pWindow->ekey); return 0; } @@ -893,55 +902,58 @@ int32_t qStreamRecoverFinish(qTaskInfo_t tinfo) { return 0; } -int32_t qStreamSetParamForRecover(qTaskInfo_t tinfo) { +int32_t qSetStreamOperatorOptionForScanHistory(qTaskInfo_t tinfo) { SExecTaskInfo* pTaskInfo = (SExecTaskInfo*)tinfo; SOperatorInfo* pOperator = pTaskInfo->pRoot; while (1) { - if (pOperator->operatorType == QUERY_NODE_PHYSICAL_PLAN_STREAM_INTERVAL || - pOperator->operatorType == QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_INTERVAL || - pOperator->operatorType == QUERY_NODE_PHYSICAL_PLAN_STREAM_FINAL_INTERVAL) { + int32_t type = pOperator->operatorType; + if (type == QUERY_NODE_PHYSICAL_PLAN_STREAM_INTERVAL || type == QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_INTERVAL || + type == QUERY_NODE_PHYSICAL_PLAN_STREAM_FINAL_INTERVAL) { SStreamIntervalOperatorInfo* pInfo = pOperator->info; - ASSERT(pInfo->twAggSup.calTrigger == STREAM_TRIGGER_AT_ONCE || - pInfo->twAggSup.calTrigger == STREAM_TRIGGER_WINDOW_CLOSE); - ASSERT(pInfo->twAggSup.calTriggerSaved == 0 && pInfo->twAggSup.deleteMarkSaved == 0); + STimeWindowAggSupp* pSup = &pInfo->twAggSup; - qInfo("save stream param for interval: %d, %" PRId64, pInfo->twAggSup.calTrigger, pInfo->twAggSup.deleteMark); + ASSERT(pSup->calTrigger == STREAM_TRIGGER_AT_ONCE || pSup->calTrigger == STREAM_TRIGGER_WINDOW_CLOSE); + ASSERT(pSup->calTriggerSaved == 0 && pSup->deleteMarkSaved == 0); - pInfo->twAggSup.calTriggerSaved = pInfo->twAggSup.calTrigger; - pInfo->twAggSup.deleteMarkSaved = pInfo->twAggSup.deleteMark; - pInfo->twAggSup.calTrigger = STREAM_TRIGGER_AT_ONCE; - pInfo->twAggSup.deleteMark = INT64_MAX; + qInfo("save stream param for interval: %d, %" PRId64, pSup->calTrigger, pSup->deleteMark); + + pSup->calTriggerSaved = pSup->calTrigger; + pSup->deleteMarkSaved = pSup->deleteMark; + pSup->calTrigger = STREAM_TRIGGER_AT_ONCE; + pSup->deleteMark = INT64_MAX; pInfo->ignoreExpiredDataSaved = pInfo->ignoreExpiredData; pInfo->ignoreExpiredData = false; - } else if (pOperator->operatorType == QUERY_NODE_PHYSICAL_PLAN_STREAM_SESSION || - pOperator->operatorType == QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_SESSION || - pOperator->operatorType == QUERY_NODE_PHYSICAL_PLAN_STREAM_FINAL_SESSION) { + } else if (type == QUERY_NODE_PHYSICAL_PLAN_STREAM_SESSION || + type == QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_SESSION || + type == QUERY_NODE_PHYSICAL_PLAN_STREAM_FINAL_SESSION) { SStreamSessionAggOperatorInfo* pInfo = pOperator->info; - ASSERT(pInfo->twAggSup.calTrigger == STREAM_TRIGGER_AT_ONCE || - pInfo->twAggSup.calTrigger == STREAM_TRIGGER_WINDOW_CLOSE); + STimeWindowAggSupp* pSup = &pInfo->twAggSup; - ASSERT(pInfo->twAggSup.calTriggerSaved == 0 && pInfo->twAggSup.deleteMarkSaved == 0); - qInfo("save stream param for session: %d, %" PRId64, pInfo->twAggSup.calTrigger, pInfo->twAggSup.deleteMark); + ASSERT(pSup->calTrigger == STREAM_TRIGGER_AT_ONCE || pSup->calTrigger == STREAM_TRIGGER_WINDOW_CLOSE); + ASSERT(pSup->calTriggerSaved == 0 && pSup->deleteMarkSaved == 0); - pInfo->twAggSup.calTriggerSaved = pInfo->twAggSup.calTrigger; - pInfo->twAggSup.deleteMarkSaved = pInfo->twAggSup.deleteMark; - pInfo->twAggSup.calTrigger = STREAM_TRIGGER_AT_ONCE; - pInfo->twAggSup.deleteMark = INT64_MAX; + qInfo("save stream param for session: %d, %" PRId64, pSup->calTrigger, pSup->deleteMark); + + pSup->calTriggerSaved = pSup->calTrigger; + pSup->deleteMarkSaved = pSup->deleteMark; + pSup->calTrigger = STREAM_TRIGGER_AT_ONCE; + pSup->deleteMark = INT64_MAX; pInfo->ignoreExpiredDataSaved = pInfo->ignoreExpiredData; pInfo->ignoreExpiredData = false; - } else if (pOperator->operatorType == QUERY_NODE_PHYSICAL_PLAN_STREAM_STATE) { + } else if (type == QUERY_NODE_PHYSICAL_PLAN_STREAM_STATE) { SStreamStateAggOperatorInfo* pInfo = pOperator->info; - ASSERT(pInfo->twAggSup.calTrigger == STREAM_TRIGGER_AT_ONCE || - pInfo->twAggSup.calTrigger == STREAM_TRIGGER_WINDOW_CLOSE); - ASSERT(pInfo->twAggSup.calTriggerSaved == 0 && pInfo->twAggSup.deleteMarkSaved == 0); + STimeWindowAggSupp* pSup = &pInfo->twAggSup; - qInfo("save stream param for state: %d, %" PRId64, pInfo->twAggSup.calTrigger, pInfo->twAggSup.deleteMark); + ASSERT(pSup->calTrigger == STREAM_TRIGGER_AT_ONCE || pSup->calTrigger == STREAM_TRIGGER_WINDOW_CLOSE); + ASSERT(pSup->calTriggerSaved == 0 && pSup->deleteMarkSaved == 0); - pInfo->twAggSup.calTriggerSaved = pInfo->twAggSup.calTrigger; - pInfo->twAggSup.deleteMarkSaved = pInfo->twAggSup.deleteMark; - pInfo->twAggSup.calTrigger = STREAM_TRIGGER_AT_ONCE; - pInfo->twAggSup.deleteMark = INT64_MAX; + qInfo("save stream param for state: %d, %" PRId64, pSup->calTrigger, pSup->deleteMark); + + pSup->calTriggerSaved = pSup->calTrigger; + pSup->deleteMarkSaved = pSup->deleteMark; + pSup->calTrigger = STREAM_TRIGGER_AT_ONCE; + pSup->deleteMark = INT64_MAX; pInfo->ignoreExpiredDataSaved = pInfo->ignoreExpiredData; pInfo->ignoreExpiredData = false; } @@ -962,7 +974,7 @@ int32_t qStreamSetParamForRecover(qTaskInfo_t tinfo) { return 0; } -int32_t qStreamRestoreParam(qTaskInfo_t tinfo) { +int32_t qRestoreStreamOperatorOption(qTaskInfo_t tinfo) { SExecTaskInfo* pTaskInfo = (SExecTaskInfo*)tinfo; SOperatorInfo* pOperator = pTaskInfo->pRoot; diff --git a/source/libs/stream/src/streamRecover.c b/source/libs/stream/src/streamRecover.c index f59ffa68a0..38c6ad6f29 100644 --- a/source/libs/stream/src/streamRecover.c +++ b/source/libs/stream/src/streamRecover.c @@ -36,11 +36,11 @@ static int32_t doLaunchScanHistoryTask(SStreamTask* pTask) { pRange->minVer, pRange->maxVer); streamSetParamForScanHistoryData(pTask); - streamSourceRecoverPrepareStep1(pTask, pRange, &pTask->dataRange.window); + streamSetParamForStreamScanner(pTask, pRange, &pTask->dataRange.window); - SStreamRecoverStep1Req req; + SStreamScanHistoryReq req; streamBuildSourceRecover1Req(pTask, &req); - int32_t len = sizeof(SStreamRecoverStep1Req); + int32_t len = sizeof(SStreamScanHistoryReq); void* serializedReq = rpcMallocCont(len); if (serializedReq == NULL) { @@ -242,13 +242,13 @@ int32_t streamProcessCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* pRs // common int32_t streamSetParamForScanHistoryData(SStreamTask* pTask) { - void* exec = pTask->exec.pExecutor; - return qStreamSetParamForRecover(exec); + qDebug("s-task:%s set operator option for scan-history-data", pTask->id.idStr); + return qSetStreamOperatorOptionForScanHistory(pTask->exec.pExecutor); } int32_t streamRestoreParam(SStreamTask* pTask) { - void* exec = pTask->exec.pExecutor; - return qStreamRestoreParam(exec); + qDebug("s-task:%s restore operator param after scan-history-data", pTask->id.idStr); + return qRestoreStreamOperatorOption(pTask->exec.pExecutor); } int32_t streamSetStatusNormal(SStreamTask* pTask) { @@ -258,19 +258,18 @@ int32_t streamSetStatusNormal(SStreamTask* pTask) { } // source -int32_t streamSourceRecoverPrepareStep1(SStreamTask* pTask, SVersionRange *pVerRange, STimeWindow* pWindow) { - void* exec = pTask->exec.pExecutor; - return qStreamSourceRecoverStep1(exec, pVerRange, pWindow); +int32_t streamSetParamForStreamScanner(SStreamTask* pTask, SVersionRange *pVerRange, STimeWindow* pWindow) { + return qStreamSourceScanParamForHistoryScan(pTask->exec.pExecutor, pVerRange, pWindow); } -int32_t streamBuildSourceRecover1Req(SStreamTask* pTask, SStreamRecoverStep1Req* pReq) { +int32_t streamBuildSourceRecover1Req(SStreamTask* pTask, SStreamScanHistoryReq* pReq) { pReq->msgHead.vgId = pTask->info.nodeId; pReq->streamId = pTask->id.streamId; pReq->taskId = pTask->id.taskId; return 0; } -int32_t streamSourceRecoverScanStep1(SStreamTask* pTask) { +int32_t streamSourceScanHistoryData(SStreamTask* pTask) { return streamScanExec(pTask, 100); } @@ -393,7 +392,7 @@ int32_t streamAggRecoverPrepare(SStreamTask* pTask) { int32_t streamAggChildrenRecoverFinish(SStreamTask* pTask) { void* exec = pTask->exec.pExecutor; - if (qStreamRestoreParam(exec) < 0) { + if (qRestoreStreamOperatorOption(exec) < 0) { return -1; } if (qStreamRecoverFinish(exec) < 0) { From 9a3708e17b13524f0c7b1d6d9e13ebc7e4b819ec Mon Sep 17 00:00:00 2001 From: liuyao <54liuyao@163.com> Date: Wed, 14 Jun 2023 15:54:07 +0800 Subject: [PATCH 28/83] pause&resume fill history --- include/libs/stream/tstream.h | 6 ++-- source/dnode/vnode/src/tq/tq.c | 6 ++-- source/libs/stream/src/streamRecover.c | 42 +++++++++++++++----------- 3 files changed, 32 insertions(+), 22 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 0891c35716..93b1314cad 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -451,7 +451,8 @@ typedef struct { SMsgHead msgHead; int64_t streamId; int32_t taskId; -} SStreamScanHistoryReq, SStreamRecoverStep2Req; + int8_t igUntreated; +} SStreamScanHistoryReq; typedef struct { int64_t streamId; @@ -574,6 +575,7 @@ int32_t streamTaskCheckStatus(SStreamTask* pTask); int32_t streamProcessCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* pRsp); int32_t streamTaskStartHistoryTask(SStreamTask* pTask); int32_t streamTaskScanHistoryDataComplete(SStreamTask* pTask); +int32_t streamStartRecoverTask(SStreamTask* pTask, int8_t igUntreated); // common int32_t streamSetParamForScanHistoryData(SStreamTask* pTask); @@ -583,7 +585,7 @@ const char* streamGetTaskStatusStr(int32_t status); // source level int32_t streamSetParamForStreamScanner(SStreamTask* pTask, SVersionRange *pVerRange, STimeWindow* pWindow); -int32_t streamBuildSourceRecover1Req(SStreamTask* pTask, SStreamScanHistoryReq* pReq); +int32_t streamBuildSourceRecover1Req(SStreamTask* pTask, SStreamScanHistoryReq* pReq, int8_t igUntreated); int32_t streamSourceScanHistoryData(SStreamTask* pTask); //int32_t streamSourceRecoverScanStep2(SStreamTask* pTask, int64_t ver); int32_t streamDispatchScanHistoryFinishMsg(SStreamTask* pTask); diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index a8d712ae01..648b8f4ec2 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1468,7 +1468,7 @@ int32_t tqProcessTaskResumeReq(STQ* pTq, int64_t sversion, char* msg, int32_t ms atomic_store_8(&pTask->status.taskStatus, pTask->status.keepTaskStatus); // no lock needs to secure the access of the version - if (pReq->igUntreated && pTask->info.taskLevel == TASK_LEVEL__SOURCE) { + if (pReq->igUntreated && pTask->info.taskLevel == TASK_LEVEL__SOURCE && !pTask->info.fillHistory) { // discard all the data when the stream task is suspended. walReaderSetSkipToVersion(pTask->exec.pWalReader, sversion); tqDebug("vgId:%d s-task:%s resume to exec, prev paused version:%" PRId64 ", start from vnode ver:%" PRId64 @@ -1479,7 +1479,9 @@ int32_t tqProcessTaskResumeReq(STQ* pTq, int64_t sversion, char* msg, int32_t ms vgId, pTask->id.idStr, pTask->chkInfo.currentVer, sversion, pTask->status.schedStatus); } - if (pTask->info.taskLevel == TASK_LEVEL__SOURCE && taosQueueItemSize(pTask->inputQueue->queue) == 0) { + if (pTask->info.fillHistory && pTask->info.taskLevel == TASK_LEVEL__SOURCE) { + streamStartRecoverTask(pTask, pReq->igUntreated); + } else if (pTask->info.taskLevel == TASK_LEVEL__SOURCE && taosQueueItemSize(pTask->inputQueue->queue) == 0) { tqStartStreamTasks(pTq); } else { streamSchedExec(pTask); diff --git a/source/libs/stream/src/streamRecover.c b/source/libs/stream/src/streamRecover.c index 38c6ad6f29..46b6798c64 100644 --- a/source/libs/stream/src/streamRecover.c +++ b/source/libs/stream/src/streamRecover.c @@ -17,6 +17,26 @@ #include "ttimer.h" #include "wal.h" +int32_t streamStartRecoverTask(SStreamTask* pTask, int8_t igUntreated) { + SStreamScanHistoryReq req; + streamBuildSourceRecover1Req(pTask, &req, igUntreated); + int32_t len = sizeof(SStreamScanHistoryReq); + + void* serializedReq = rpcMallocCont(len); + if (serializedReq == NULL) { + return -1; + } + + memcpy(serializedReq, &req, len); + + SRpcMsg rpcMsg = {.contLen = len, .pCont = serializedReq, .msgType = TDMT_VND_STREAM_SCAN_HISTORY}; + if (tmsgPutToQueue(pTask->pMsgCb, STREAM_QUEUE, &rpcMsg) < 0) { + /*ASSERT(0);*/ + } + + return 0; +} + const char* streamGetTaskStatusStr(int32_t status) { switch(status) { case TASK_STATUS__NORMAL: return "normal"; @@ -38,23 +58,8 @@ static int32_t doLaunchScanHistoryTask(SStreamTask* pTask) { streamSetParamForScanHistoryData(pTask); streamSetParamForStreamScanner(pTask, pRange, &pTask->dataRange.window); - SStreamScanHistoryReq req; - streamBuildSourceRecover1Req(pTask, &req); - int32_t len = sizeof(SStreamScanHistoryReq); - - void* serializedReq = rpcMallocCont(len); - if (serializedReq == NULL) { - return -1; - } - - memcpy(serializedReq, &req, len); - - SRpcMsg rpcMsg = {.contLen = len, .pCont = serializedReq, .msgType = TDMT_VND_STREAM_SCAN_HISTORY}; - if (tmsgPutToQueue(pTask->pMsgCb, STREAM_QUEUE, &rpcMsg) < 0) { - /*ASSERT(0);*/ - } - - return 0; + int32_t code = streamStartRecoverTask(pTask, 0); + return code; } int32_t streamTaskLaunchScanHistory(SStreamTask* pTask) { @@ -262,10 +267,11 @@ int32_t streamSetParamForStreamScanner(SStreamTask* pTask, SVersionRange *pVerRa return qStreamSourceScanParamForHistoryScan(pTask->exec.pExecutor, pVerRange, pWindow); } -int32_t streamBuildSourceRecover1Req(SStreamTask* pTask, SStreamScanHistoryReq* pReq) { +int32_t streamBuildSourceRecover1Req(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; } From 53377c2c1f739422afc6c1005f5b3b10a93625a8 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 15 Jun 2023 01:14:41 +0800 Subject: [PATCH 29/83] fix(stream): wait for stream task completed. --- include/libs/stream/tstream.h | 6 ++-- source/dnode/vnode/src/tq/tq.c | 9 +++--- source/libs/stream/src/streamExec.c | 40 ++++++++++++++++++++++++-- source/libs/stream/src/streamRecover.c | 19 ++++++------ 4 files changed, 55 insertions(+), 19 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 93b1314cad..3cfde016f0 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -44,9 +44,8 @@ enum { TASK_STATUS__DROPPING, TASK_STATUS__FAIL, TASK_STATUS__STOP, - TASK_STATUS__WAIT_DOWNSTREAM, - TASK_STATUS__SCAN_HISTORY, - TASK_STATUS__HALT, // stream task halt to wait for the secondary scan history, this status is invisible for user + TASK_STATUS__SCAN_HISTORY, // stream task scan history data by using tsdbread in the stream scanner + TASK_STATUS__HALT, // stream task will handle all data in the input queue, and then paused TASK_STATUS__PAUSE, }; @@ -565,6 +564,7 @@ int32_t streamSchedExec(SStreamTask* pTask); int32_t streamTaskOutputResultBlock(SStreamTask* pTask, SStreamDataBlock* pBlock); bool streamTaskShouldStop(const SStreamStatus* pStatus); bool streamTaskShouldPause(const SStreamStatus* pStatus); +bool streamTaskIsIdle(const SStreamTask* pTask); int32_t streamScanExec(SStreamTask* pTask, int32_t batchSz); diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 648b8f4ec2..e1cf46a5f4 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1121,8 +1121,6 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { taosMsleep(100); } - taosMsleep(10000); - // now we can stop the stream task execution pStreamTask->status.taskStatus = TASK_STATUS__HALT; tqDebug("s-task:%s level:%d status is set to halt by history scan task:%s", pStreamTask->id.idStr, @@ -1378,12 +1376,13 @@ int32_t tqProcessTaskRunReq(STQ* pTq, SRpcMsg* pMsg) { SStreamTask* pTask = streamMetaAcquireTask(pTq->pStreamMeta, taskId); if (pTask != NULL) { - if (pTask->status.taskStatus == TASK_STATUS__NORMAL) { - tqDebug("vgId:%d s-task:%s start to process block from wal, last chk point:%" PRId64, vgId, pTask->id.idStr, + int8_t status = pTask->status.taskStatus; + if (status == TASK_STATUS__NORMAL || status == TASK_STATUS__HALT) { + tqDebug("vgId:%d s-task:%s start to process block from inputQ, last chk point:%" PRId64, vgId, pTask->id.idStr, pTask->chkInfo.version); streamProcessRunReq(pTask); } else { - if (streamTaskShouldPause(&pTask->status) || (pTask->status.taskStatus == TASK_STATUS__HALT)) { + if (streamTaskShouldPause(&pTask->status)) { atomic_store_8(&pTask->status.schedStatus, TASK_SCHED_STATUS__INACTIVE); } diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index b717ea456a..b68d15328d 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -189,6 +189,7 @@ int32_t streamScanExec(SStreamTask* pTask, int32_t batchSz) { if (qExecTask(exec, &output, &ts) < 0) { continue; } + if (output == NULL) { if (qStreamRecoverScanFinished(exec)) { finished = true; @@ -396,16 +397,30 @@ int32_t streamExecForAll(SStreamTask* pTask) { ASSERT(pStreamTask != NULL && pStreamTask->historyTaskId.taskId == pTask->id.taskId); STimeWindow* pTimeWindow = &pStreamTask->dataRange.window; + // here we need to wait for the stream task handle all data in the input queue. if (pStreamTask->info.taskLevel == TASK_LEVEL__SOURCE) { ASSERT(pStreamTask->status.taskStatus == TASK_STATUS__HALT); + } else { + ASSERT(pStreamTask->status.taskStatus == TASK_STATUS__NORMAL); + pStreamTask->status.taskStatus = TASK_STATUS__HALT; + } + {// wait for the stream task to be idle + while(!streamTaskIsIdle(pStreamTask)) { + qDebug("s-task:%s level:%d wait for stream task:%s to be idle, check again in 100ms", pTask->id.idStr, + pTask->info.taskLevel, pStreamTask->id.idStr); + taosMsleep(100); + } + } + + if (pStreamTask->info.taskLevel == TASK_LEVEL__SOURCE) { // update the scan data range for source task. qDebug("s-task:%s level:%d stream task window %" PRId64 " - %" PRId64 " transfer to %" PRId64 " - %" PRId64 ", status:%s, sched-status:%d", pStreamTask->id.idStr, TASK_LEVEL__SOURCE, pTimeWindow->skey, pTimeWindow->ekey, INT64_MIN, pTimeWindow->ekey, streamGetTaskStatusStr(TASK_STATUS__NORMAL), pStreamTask->status.schedStatus); } else { - // for agg task and sink task, they are continue to execute, no need to be halt. + // for sink tasks, they are continue to execute, no need to be halt. // the process should be stopped for a while, during the term of transfer task state. // OR wait for the inputQ && outputQ of agg tasks are all consumed, and then start the state transfer @@ -413,12 +428,13 @@ int32_t streamExecForAll(SStreamTask* pTask) { qDebug("s-task:%s no need to update time window, for non-source task", pStreamTask->id.idStr); } + // expand the query time window for stream scanner pTimeWindow->skey = INT64_MIN; streamSetStatusNormal(pStreamTask); streamMetaSaveTask(pTask->pMeta, pStreamTask); if (streamMetaCommit(pTask->pMeta)) { - // persistent to disk for + // persistent to disk } streamSchedExec(pStreamTask); @@ -481,12 +497,32 @@ int32_t streamExecForAll(SStreamTask* pTask) { double el = (taosGetTimestampMs() - st) / 1000.0; qDebug("s-task:%s batch of (%d)input blocks exec end, elapsed time:%.2fs, result size:%.2fMiB, numOfBlocks:%d", id, batchSize, el, resSize / 1048576.0, totalBlocks); + streamFreeQitem(pInput); } return 0; } +bool streamTaskIsIdle(const SStreamTask* pTask) { + int32_t numOfItems = taosQueueItemSize(pTask->inputQueue->queue); + if (numOfItems > 0) { + return false; + } + + numOfItems = taosQallItemSize(pTask->inputQueue->qall); + if (numOfItems > 0) { + return false; + } + + // blocked by downstream task + if (pTask->outputStatus == TASK_OUTPUT_STATUS__BLOCKED) { + return false; + } + + return (pTask->status.schedStatus == TASK_SCHED_STATUS__INACTIVE); +} + int32_t streamTryExec(SStreamTask* pTask) { // this function may be executed by multi-threads, so status check is required. int8_t schedStatus = diff --git a/source/libs/stream/src/streamRecover.c b/source/libs/stream/src/streamRecover.c index 46b6798c64..f041b17d2e 100644 --- a/source/libs/stream/src/streamRecover.c +++ b/source/libs/stream/src/streamRecover.c @@ -40,7 +40,6 @@ int32_t streamStartRecoverTask(SStreamTask* pTask, int8_t igUntreated) { const char* streamGetTaskStatusStr(int32_t status) { switch(status) { case TASK_STATUS__NORMAL: return "normal"; - case TASK_STATUS__WAIT_DOWNSTREAM: return "wait-for-downstream"; case TASK_STATUS__SCAN_HISTORY: return "scan-history"; case TASK_STATUS__HALT: return "halt"; case TASK_STATUS__PAUSE: return "paused"; @@ -217,18 +216,17 @@ int32_t streamProcessCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* pRs return -1; } + // set the downstream tasks have been checked flag ASSERT(pTask->status.checkDownstream == 0); pTask->status.checkDownstream = 1; - ASSERT(pTask->status.taskStatus != TASK_STATUS__HALT); - + ASSERT(pTask->status.taskStatus == TASK_STATUS__SCAN_HISTORY || pTask->status.taskStatus == TASK_STATUS__NORMAL); if (pTask->status.taskStatus == TASK_STATUS__SCAN_HISTORY) { qDebug("s-task:%s fixed downstream task is ready, now enter into scan-history-data stage, status:%s", id, streamGetTaskStatusStr(pTask->status.taskStatus)); streamTaskLaunchScanHistory(pTask); } else { - ASSERT(pTask->status.taskStatus == TASK_STATUS__NORMAL); - qDebug("s-task:%s fixed downstream task is ready, now ready for data from wal, status:%s", id, + qDebug("s-task:%s fixed downstream task is ready, ready for data from inputQ, status:%s", id, streamGetTaskStatusStr(pTask->status.taskStatus)); } } else { @@ -396,15 +394,17 @@ int32_t streamAggRecoverPrepare(SStreamTask* pTask) { return 0; } -int32_t streamAggChildrenRecoverFinish(SStreamTask* pTask) { +int32_t streamAggUpstreamScanHistoryFinish(SStreamTask* pTask) { void* exec = pTask->exec.pExecutor; if (qRestoreStreamOperatorOption(exec) < 0) { return -1; } + if (qStreamRecoverFinish(exec) < 0) { return -1; } - streamSetStatusNormal(pTask); + +// streamSetStatusNormal(pTask); return 0; } @@ -414,8 +414,9 @@ int32_t streamProcessRecoverFinishReq(SStreamTask* pTask, int32_t childId) { ASSERT(left >= 0); if (left == 0) { - qDebug("s-task:%s all %d upstream tasks finish scan-history data", pTask->id.idStr, left); - streamAggChildrenRecoverFinish(pTask); + int32_t numOfTasks = taosArrayGetSize(pTask->pUpstreamEpInfoList); + qDebug("s-task:%s all %d upstream tasks finish scan-history data", pTask->id.idStr, numOfTasks); + streamAggUpstreamScanHistoryFinish(pTask); } else { qDebug("s-task:%s remain unfinished upstream tasks:%d", pTask->id.idStr, left); } From 13028744df296a441372e02b2909389628ca20f8 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 15 Jun 2023 10:24:21 +0800 Subject: [PATCH 30/83] refactor: do some internal refactor. --- source/dnode/vnode/src/tq/tq.c | 7 +- source/libs/stream/src/streamExec.c | 116 ++++++++++++++++------------ 2 files changed, 69 insertions(+), 54 deletions(-) diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index e1cf46a5f4..6f05d67b5e 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1177,12 +1177,13 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { tqDebug("s-task:%s set status to be dropping", pTask->id.idStr); streamMetaSaveTask(pMeta, pTask); + streamMetaSaveTask(pMeta, pStreamTask); + streamMetaReleaseTask(pMeta, pTask); -// streamMetaRemoveTask(pMeta, pTask->id.taskId); - streamMetaReleaseTask(pMeta, pStreamTask); - if (streamMetaCommit(pTask->pMeta) < 0) { + if (streamMetaCommit(pTask->pMeta) < 0) { + // persist to disk } } else { // todo update the chkInfo version for current task. diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index b68d15328d..907360dff0 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -313,6 +313,69 @@ int32_t updateCheckPointInfo(SStreamTask* pTask) { return TSDB_CODE_SUCCESS; } +static void waitForTaskTobeIdle(SStreamTask* pTask, SStreamTask* pStreamTask) { + // wait for the stream task to be idle + int64_t st = taosGetTimestampMs(); + + while (!streamTaskIsIdle(pStreamTask)) { + qDebug("s-task:%s level:%d wait for stream task:%s to be idle, check again in 100ms", pTask->id.idStr, + pTask->info.taskLevel, pStreamTask->id.idStr); + taosMsleep(100); + } + + double el = (taosGetTimestampMs() - st) / 1000.0; + if (el > 0) { + qDebug("s-task:%s wait for stream task:%s for %.2fs to execute all data in inputQ", pTask->id.idStr, + pStreamTask->id.idStr, el); + } +} + +static int32_t streamTransferStateToStreamTask(SStreamTask* pTask) { + SStreamTask* pStreamTask = streamMetaAcquireTask(pTask->pMeta, pTask->streamTaskId.taskId); + qDebug("s-task:%s scan history task end, update stream task:%s info and launch it", pTask->id.idStr, pStreamTask->id.idStr); + + // todo handle stream task is dropped here + + ASSERT(pStreamTask != NULL && pStreamTask->historyTaskId.taskId == pTask->id.taskId); + STimeWindow* pTimeWindow = &pStreamTask->dataRange.window; + + // here we need to wait for the stream task handle all data in the input queue. + if (pStreamTask->info.taskLevel == TASK_LEVEL__SOURCE) { + ASSERT(pStreamTask->status.taskStatus == TASK_STATUS__HALT); + } else { + ASSERT(pStreamTask->status.taskStatus == TASK_STATUS__NORMAL); + pStreamTask->status.taskStatus = TASK_STATUS__HALT; + } + + // wait for the stream task to be idle + waitForTaskTobeIdle(pTask, pStreamTask); + + if (pStreamTask->info.taskLevel == TASK_LEVEL__SOURCE) { + // update the scan data range for source task. + qDebug("s-task:%s level:%d stream task window %" PRId64 " - %" PRId64 " transfer to %" PRId64 " - %" PRId64 + ", status:%s, sched-status:%d", + pStreamTask->id.idStr, TASK_LEVEL__SOURCE, pTimeWindow->skey, pTimeWindow->ekey, INT64_MIN, + pTimeWindow->ekey, streamGetTaskStatusStr(TASK_STATUS__NORMAL), pStreamTask->status.schedStatus); + + // todo transfer state + } else { + // for sink tasks, they are continue to execute, no need to be halt. + // the process should be stopped for a while, during the term of transfer task state. + // OR wait for the inputQ && outputQ of agg tasks are all consumed, and then start the state transfer + qDebug("s-task:%s no need to update time window, for non-source task", pStreamTask->id.idStr); + + // todo transfer state + } + + // expand the query time window for stream scanner + pTimeWindow->skey = INT64_MIN; + + streamSetStatusNormal(pStreamTask); + streamSchedExec(pStreamTask); + streamMetaReleaseTask(pTask->pMeta, pStreamTask); + return TSDB_CODE_SUCCESS; +} + /** * todo: the batch of blocks should be tuned dynamic, according to the total elapsed time of each batch of blocks, the * appropriate batch of blocks should be handled in 5 to 10 sec. @@ -388,57 +451,8 @@ int32_t streamExecForAll(SStreamTask* pTask) { } if (pInput == NULL) { - if (pTask->info.fillHistory && pTask->status.transferState) { - // todo transfer task state here - - SStreamTask* pStreamTask = streamMetaAcquireTask(pTask->pMeta, pTask->streamTaskId.taskId); - qDebug("s-task:%s scan history task end, update stream task:%s info and launch it", pTask->id.idStr, pStreamTask->id.idStr); - - ASSERT(pStreamTask != NULL && pStreamTask->historyTaskId.taskId == pTask->id.taskId); - STimeWindow* pTimeWindow = &pStreamTask->dataRange.window; - - // here we need to wait for the stream task handle all data in the input queue. - if (pStreamTask->info.taskLevel == TASK_LEVEL__SOURCE) { - ASSERT(pStreamTask->status.taskStatus == TASK_STATUS__HALT); - } else { - ASSERT(pStreamTask->status.taskStatus == TASK_STATUS__NORMAL); - pStreamTask->status.taskStatus = TASK_STATUS__HALT; - } - - {// wait for the stream task to be idle - while(!streamTaskIsIdle(pStreamTask)) { - qDebug("s-task:%s level:%d wait for stream task:%s to be idle, check again in 100ms", pTask->id.idStr, - pTask->info.taskLevel, pStreamTask->id.idStr); - taosMsleep(100); - } - } - - if (pStreamTask->info.taskLevel == TASK_LEVEL__SOURCE) { - // update the scan data range for source task. - qDebug("s-task:%s level:%d stream task window %" PRId64 " - %" PRId64 " transfer to %" PRId64 " - %" PRId64 - ", status:%s, sched-status:%d", - pStreamTask->id.idStr, TASK_LEVEL__SOURCE, pTimeWindow->skey, pTimeWindow->ekey, INT64_MIN, - pTimeWindow->ekey, streamGetTaskStatusStr(TASK_STATUS__NORMAL), pStreamTask->status.schedStatus); - } else { - // for sink tasks, they are continue to execute, no need to be halt. - // the process should be stopped for a while, during the term of transfer task state. - // OR wait for the inputQ && outputQ of agg tasks are all consumed, and then start the state transfer - - - qDebug("s-task:%s no need to update time window, for non-source task", pStreamTask->id.idStr); - } - - // expand the query time window for stream scanner - pTimeWindow->skey = INT64_MIN; - - streamSetStatusNormal(pStreamTask); - streamMetaSaveTask(pTask->pMeta, pStreamTask); - if (streamMetaCommit(pTask->pMeta)) { - // persistent to disk - } - - streamSchedExec(pStreamTask); - streamMetaReleaseTask(pTask->pMeta, pStreamTask); + if (pTask->info.fillHistory && pTask->status.transferState) { + int32_t code = streamTransferStateToStreamTask(pTask); } break; From 55724157c4082d9bc8e83fb078d95939ded5d2e8 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 15 Jun 2023 10:40:25 +0800 Subject: [PATCH 31/83] refactor: remove one function. --- include/common/tdatablock.h | 1 - source/common/src/tdatablock.c | 4 ---- source/dnode/mnode/impl/src/mndIndex.c | 14 +++++++------- source/dnode/mnode/impl/src/mndSma.c | 4 ++-- source/dnode/mnode/impl/src/mndVgroup.c | 2 +- source/libs/command/src/command.c | 2 +- source/libs/executor/src/groupoperator.c | 2 +- source/libs/executor/src/sysscanoperator.c | 12 ++++++------ source/libs/executor/src/timesliceoperator.c | 2 +- source/libs/function/src/builtinsimpl.c | 2 +- source/libs/geometry/src/geomFunc.c | 8 ++++---- source/libs/geometry/test/geomFuncTestUtil.cpp | 2 +- 12 files changed, 25 insertions(+), 30 deletions(-) diff --git a/include/common/tdatablock.h b/include/common/tdatablock.h index b59ba28671..169a9ee17f 100644 --- a/include/common/tdatablock.h +++ b/include/common/tdatablock.h @@ -177,7 +177,6 @@ static FORCE_INLINE void colDataSetDouble(SColumnInfoData* pColumnInfoData, uint int32_t getJsonValueLen(const char* data); int32_t colDataSetVal(SColumnInfoData* pColumnInfoData, uint32_t rowIndex, const char* pData, bool isNull); -int32_t colDataAppend(SColumnInfoData* pColumnInfoData, uint32_t rowIndex, const char* pData, bool isNull); int32_t colDataReassignVal(SColumnInfoData* pColumnInfoData, uint32_t dstRowIdx, uint32_t srcRowIdx, const char* pData); int32_t colDataSetNItems(SColumnInfoData* pColumnInfoData, uint32_t rowIndex, const char* pData, uint32_t numOfRows, bool trimValue); int32_t colDataMergeCol(SColumnInfoData* pColumnInfoData, int32_t numOfRow1, int32_t* capacity, diff --git a/source/common/src/tdatablock.c b/source/common/src/tdatablock.c index b7e6e9f8b9..093ffd5198 100644 --- a/source/common/src/tdatablock.c +++ b/source/common/src/tdatablock.c @@ -74,10 +74,6 @@ int32_t getJsonValueLen(const char* data) { return dataLen; } -int32_t colDataAppend(SColumnInfoData* pColumnInfoData, uint32_t rowIndex, const char* pData, bool isNull) { - return colDataSetVal(pColumnInfoData, rowIndex, pData, isNull); -} - int32_t colDataSetVal(SColumnInfoData* pColumnInfoData, uint32_t rowIndex, const char* pData, bool isNull) { if (isNull) { // There is a placehold for each NULL value of binary or nchar type. diff --git a/source/dnode/mnode/impl/src/mndIndex.c b/source/dnode/mnode/impl/src/mndIndex.c index 83172acf64..8dc24ddb34 100644 --- a/source/dnode/mnode/impl/src/mndIndex.c +++ b/source/dnode/mnode/impl/src/mndIndex.c @@ -542,32 +542,32 @@ int32_t mndRetrieveTagIdx(SRpcMsg *pReq, SShowObj *pShow, SSDataBlock *pBlock, i STR_TO_VARSTR(n3, (char *)tNameGetTableName(&stbName)); SColumnInfoData *pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); - colDataAppend(pColInfo, numOfRows, (const char *)n1, false); + colDataSetVal(pColInfo, numOfRows, (const char *)n1, false); pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); - colDataAppend(pColInfo, numOfRows, (const char *)n2, false); + colDataSetVal(pColInfo, numOfRows, (const char *)n2, false); pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); - colDataAppend(pColInfo, numOfRows, (const char *)n3, false); + colDataSetVal(pColInfo, numOfRows, (const char *)n3, false); pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); - colDataAppend(pColInfo, numOfRows, (const char *)&invalid, false); + colDataSetVal(pColInfo, numOfRows, (const char *)&invalid, false); pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); - colDataAppend(pColInfo, numOfRows, (const char *)&pIdx->createdTime, false); + colDataSetVal(pColInfo, numOfRows, (const char *)&pIdx->createdTime, false); char col[TSDB_TABLE_FNAME_LEN + VARSTR_HEADER_SIZE] = {0}; STR_TO_VARSTR(col, (char *)pIdx->colName); pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); - colDataAppend(pColInfo, numOfRows, (const char *)col, false); + colDataSetVal(pColInfo, numOfRows, (const char *)col, false); pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); char tag[TSDB_TABLE_FNAME_LEN + VARSTR_HEADER_SIZE] = {0}; STR_TO_VARSTR(tag, (char *)"tag_index"); - colDataAppend(pColInfo, numOfRows, (const char *)tag, false); + colDataSetVal(pColInfo, numOfRows, (const char *)tag, false); numOfRows++; sdbRelease(pSdb, pIdx); diff --git a/source/dnode/mnode/impl/src/mndSma.c b/source/dnode/mnode/impl/src/mndSma.c index 42325cb926..76ad57d688 100644 --- a/source/dnode/mnode/impl/src/mndSma.c +++ b/source/dnode/mnode/impl/src/mndSma.c @@ -1278,13 +1278,13 @@ static int32_t mndRetrieveSma(SRpcMsg *pReq, SShowObj *pShow, SSDataBlock *pBloc STR_TO_VARSTR(col, (char *)""); pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); - colDataAppend(pColInfo, numOfRows, (const char *)col, false); + colDataSetVal(pColInfo, numOfRows, (const char *)col, false); pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); char tag[TSDB_TABLE_FNAME_LEN + VARSTR_HEADER_SIZE] = {0}; STR_TO_VARSTR(tag, (char *)"sma_index"); - colDataAppend(pColInfo, numOfRows, (const char *)tag, false); + colDataSetVal(pColInfo, numOfRows, (const char *)tag, false); numOfRows++; sdbRelease(pSdb, pSma); diff --git a/source/dnode/mnode/impl/src/mndVgroup.c b/source/dnode/mnode/impl/src/mndVgroup.c index b7a6378bd8..166306a8ee 100644 --- a/source/dnode/mnode/impl/src/mndVgroup.c +++ b/source/dnode/mnode/impl/src/mndVgroup.c @@ -874,7 +874,7 @@ static int32_t mndRetrieveVgroups(SRpcMsg *pReq, SShowObj *pShow, SSDataBlock *p // if (pDb == NULL || pDb->compactStartTime <= 0) { // colDataSetNULL(pColInfo, numOfRows); // } else { - // colDataAppend(pColInfo, numOfRows, (const char *)&pDb->compactStartTime, false); + // colDataSetVal(pColInfo, numOfRows, (const char *)&pDb->compactStartTime, false); // } numOfRows++; diff --git a/source/libs/command/src/command.c b/source/libs/command/src/command.c index f59653700b..dad20c915c 100644 --- a/source/libs/command/src/command.c +++ b/source/libs/command/src/command.c @@ -392,7 +392,7 @@ static int32_t setAliveResultIntoDataBlock(int64_t* pConnId, SSDataBlock* pBlock int32_t status = 0; int32_t code = getAliveStatusFromApi(pConnId, dbName, &status); if (code == TSDB_CODE_SUCCESS) { - colDataAppend(pCol1, 0, (const char*)&status, false); + colDataSetVal(pCol1, 0, (const char*)&status, false); } return code; } diff --git a/source/libs/executor/src/groupoperator.c b/source/libs/executor/src/groupoperator.c index c448ea0160..934ea7baf1 100644 --- a/source/libs/executor/src/groupoperator.c +++ b/source/libs/executor/src/groupoperator.c @@ -1033,7 +1033,7 @@ void appendCreateTableRow(void* pState, SExprSupp* pTableSup, SExprSupp* pTagSup } void* pGpIdCol = taosArrayGet(pDestBlock->pDataBlock, UD_GROUPID_COLUMN_INDEX); - colDataAppend(pGpIdCol, pDestBlock->info.rows, (const char*)&groupId, false); + colDataSetVal(pGpIdCol, pDestBlock->info.rows, (const char*)&groupId, false); pDestBlock->info.rows++; blockDataDestroy(pTmpBlock); } else { diff --git a/source/libs/executor/src/sysscanoperator.c b/source/libs/executor/src/sysscanoperator.c index 23a7d2c9e9..c158ecd9e8 100644 --- a/source/libs/executor/src/sysscanoperator.c +++ b/source/libs/executor/src/sysscanoperator.c @@ -966,20 +966,20 @@ static int32_t sysTableUserColsFillOneTableCols(const SSysTableScanInfo* pInfo, // table name pColInfoData = taosArrayGet(dataBlock->pDataBlock, 0); - colDataAppend(pColInfoData, numOfRows, tName, false); + colDataSetVal(pColInfoData, numOfRows, tName, false); // database name pColInfoData = taosArrayGet(dataBlock->pDataBlock, 1); - colDataAppend(pColInfoData, numOfRows, dbname, false); + colDataSetVal(pColInfoData, numOfRows, dbname, false); pColInfoData = taosArrayGet(dataBlock->pDataBlock, 2); - colDataAppend(pColInfoData, numOfRows, tableType, false); + colDataSetVal(pColInfoData, numOfRows, tableType, false); // col name char colName[TSDB_COL_NAME_LEN + VARSTR_HEADER_SIZE] = {0}; STR_TO_VARSTR(colName, schemaRow->pSchema[i].name); pColInfoData = taosArrayGet(dataBlock->pDataBlock, 3); - colDataAppend(pColInfoData, numOfRows, colName, false); + colDataSetVal(pColInfoData, numOfRows, colName, false); // col type int8_t colType = schemaRow->pSchema[i].type; @@ -994,10 +994,10 @@ static int32_t sysTableUserColsFillOneTableCols(const SSysTableScanInfo* pInfo, (int32_t)((schemaRow->pSchema[i].bytes - VARSTR_HEADER_SIZE) / TSDB_NCHAR_SIZE)); } varDataSetLen(colTypeStr, colTypeLen); - colDataAppend(pColInfoData, numOfRows, (char*)colTypeStr, false); + colDataSetVal(pColInfoData, numOfRows, (char*)colTypeStr, false); pColInfoData = taosArrayGet(dataBlock->pDataBlock, 5); - colDataAppend(pColInfoData, numOfRows, (const char*)&schemaRow->pSchema[i].bytes, false); + colDataSetVal(pColInfoData, numOfRows, (const char*)&schemaRow->pSchema[i].bytes, false); for (int32_t j = 6; j <= 8; ++j) { pColInfoData = taosArrayGet(dataBlock->pDataBlock, j); diff --git a/source/libs/executor/src/timesliceoperator.c b/source/libs/executor/src/timesliceoperator.c index 2421343bd7..9c059c26c2 100644 --- a/source/libs/executor/src/timesliceoperator.c +++ b/source/libs/executor/src/timesliceoperator.c @@ -270,7 +270,7 @@ static bool genInterpolationResult(STimeSliceOperatorInfo* pSliceInfo, SExprSupp continue; } else if (isIsfilledPseudoColumn(pExprInfo)) { bool isFilled = true; - colDataAppend(pDst, pResBlock->info.rows, (char*)&isFilled, false); + colDataSetVal(pDst, pResBlock->info.rows, (char*)&isFilled, false); continue; } else if (!isInterpFunc(pExprInfo)) { if (isGroupKeyFunc(pExprInfo)) { diff --git a/source/libs/function/src/builtinsimpl.c b/source/libs/function/src/builtinsimpl.c index 1265c64c8c..743b9225a6 100644 --- a/source/libs/function/src/builtinsimpl.c +++ b/source/libs/function/src/builtinsimpl.c @@ -1712,7 +1712,7 @@ int32_t percentileFinalize(SqlFunctionCtx* pCtx, SSDataBlock* pBlock) { SColumnInfoData* pCol = taosArrayGet(pBlock->pDataBlock, slotId); varDataSetLen(buf, len); - colDataAppend(pCol, pBlock->info.rows, buf, false); + colDataSetVal(pCol, pBlock->info.rows, buf, false); tMemBucketDestroy(pMemBucket); return pResInfo->numOfRes; diff --git a/source/libs/geometry/src/geomFunc.c b/source/libs/geometry/src/geomFunc.c index 0e2bac584d..3588bf8b7d 100644 --- a/source/libs/geometry/src/geomFunc.c +++ b/source/libs/geometry/src/geomFunc.c @@ -145,7 +145,7 @@ int32_t executeMakePointFunc(SColumnInfoData *pInputData[], int32_t iLeft, int32 goto _exit; } - colDataAppend(pOutputData, TMAX(iLeft, iRight), output, (output == NULL)); + colDataSetVal(pOutputData, TMAX(iLeft, iRight), output, (output == NULL)); _exit: if (output) { @@ -165,7 +165,7 @@ int32_t executeGeomFromTextFunc(SColumnInfoData *pInputData, int32_t i, SColumnI goto _exit; } - colDataAppend(pOutputData, i, output, (output == NULL)); + colDataSetVal(pOutputData, i, output, (output == NULL)); _exit: if (output) { @@ -185,7 +185,7 @@ int32_t executeAsTextFunc(SColumnInfoData *pInputData, int32_t i, SColumnInfoDat goto _exit; } - colDataAppend(pOutputData, i, output, (output == NULL)); + colDataSetVal(pOutputData, i, output, (output == NULL)); _exit: if (output) { @@ -213,7 +213,7 @@ int32_t executeRelationFunc(const GEOSGeometry *geom1, const GEOSPreparedGeometr } } - colDataAppend(pOutputData, i, &res, (res==-1)); + colDataSetVal(pOutputData, i, &res, (res==-1)); return code; } diff --git a/source/libs/geometry/test/geomFuncTestUtil.cpp b/source/libs/geometry/test/geomFuncTestUtil.cpp index cb59ea098f..0918781499 100644 --- a/source/libs/geometry/test/geomFuncTestUtil.cpp +++ b/source/libs/geometry/test/geomFuncTestUtil.cpp @@ -84,7 +84,7 @@ void setScalarParam(SScalarParam *sclParam, int32_t type, void *valueArray, TDRo } else { const char *val = (const char *)valueArray + (i * bytes); - colDataAppend(sclParam->columnData, i, val, false); + colDataSetVal(sclParam->columnData, i, val, false); } } } From 1742874790984a51a9772757fdbe11dbe8f8e54c Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 15 Jun 2023 10:49:34 +0800 Subject: [PATCH 32/83] fix(test): fix unit test error. --- source/libs/scheduler/test/CMakeLists.txt | 16 ++++++++++++---- 1 file changed, 12 insertions(+), 4 deletions(-) diff --git a/source/libs/scheduler/test/CMakeLists.txt b/source/libs/scheduler/test/CMakeLists.txt index ce92886221..703bd5932b 100644 --- a/source/libs/scheduler/test/CMakeLists.txt +++ b/source/libs/scheduler/test/CMakeLists.txt @@ -7,10 +7,18 @@ IF(NOT TD_DARWIN) AUX_SOURCE_DIRECTORY(${CMAKE_CURRENT_SOURCE_DIR} SOURCE_LIST) ADD_EXECUTABLE(schedulerTest ${SOURCE_LIST}) - TARGET_LINK_LIBRARIES( - schedulerTest - PUBLIC os util common catalog transport gtest qcom taos_static planner scheduler - ) + + IF (TD_GRANT) + TARGET_LINK_LIBRARIES( + schedulerTest + PUBLIC os util common catalog transport gtest qcom taos_static planner scheduler grant + ) + ELSE () + TARGET_LINK_LIBRARIES( + schedulerTest + PUBLIC os util common catalog transport gtest qcom taos_static planner scheduler + ) + ENDIF() TARGET_INCLUDE_DIRECTORIES( schedulerTest From 3b2f2f0a946630c194e88effec30ddd901c283c3 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 15 Jun 2023 11:06:03 +0800 Subject: [PATCH 33/83] fix(test): fix link error in unit test. --- source/libs/planner/test/CMakeLists.txt | 15 +++++++++++---- 1 file changed, 11 insertions(+), 4 deletions(-) diff --git a/source/libs/planner/test/CMakeLists.txt b/source/libs/planner/test/CMakeLists.txt index b9d5c85717..73aca8572a 100644 --- a/source/libs/planner/test/CMakeLists.txt +++ b/source/libs/planner/test/CMakeLists.txt @@ -12,10 +12,17 @@ IF(NOT TD_DARWIN) "${SOURCE_LIST}/../../../parser/test/mockCatalogService.cpp" ) - TARGET_LINK_LIBRARIES( - plannerTest - PUBLIC os util common nodes planner parser catalog transport gtest function qcom - ) + IF (TD_GRANT) + TARGET_LINK_LIBRARIES( + plannerTest + PUBLIC os util common nodes planner parser catalog transport gtest function qcom grant + ) + ELSE () + TARGET_LINK_LIBRARIES( + plannerTest + PUBLIC os util common nodes planner parser catalog transport gtest function qcom + ) + ENDIF() TARGET_INCLUDE_DIRECTORIES( plannerTest From 693942b8215867465bf8dfc38cae9dac4c35cd57 Mon Sep 17 00:00:00 2001 From: liuyao <54liuyao@163.com> Date: Thu, 15 Jun 2023 14:13:17 +0800 Subject: [PATCH 34/83] trans state --- include/libs/executor/executor.h | 3 +++ include/libs/executor/storageapi.h | 1 + include/libs/stream/streamState.h | 2 ++ include/libs/stream/tstream.h | 3 +++ include/libs/stream/tstreamFileState.h | 1 + source/dnode/snode/src/snodeInitApi.c | 1 + source/dnode/vnode/src/tq/tq.c | 2 ++ source/dnode/vnode/src/vnd/vnodeInitApi.c | 1 + source/libs/executor/src/executor.c | 14 +++++++++++++- source/libs/executor/src/projectoperator.c | 15 +++++++++++++++ source/libs/executor/src/timewindowoperator.c | 19 +++++++++++++++---- source/libs/stream/src/streamExec.c | 12 ++++++++++++ source/libs/stream/src/streamState.c | 4 ++++ source/libs/stream/src/tstreamFileState.c | 5 +++++ 14 files changed, 78 insertions(+), 5 deletions(-) diff --git a/include/libs/executor/executor.h b/include/libs/executor/executor.h index 852257f5df..e03718b571 100644 --- a/include/libs/executor/executor.h +++ b/include/libs/executor/executor.h @@ -229,6 +229,9 @@ bool qStreamRecoverScanFinished(qTaskInfo_t tinfo); void qStreamCloseTsdbReader(void* task); void resetTaskInfo(qTaskInfo_t tinfo); +int32_t qStreamOperatorReleaseState(qTaskInfo_t tInfo); +int32_t qStreamOperatorReloadState(qTaskInfo_t tInfo); + #ifdef __cplusplus } #endif diff --git a/include/libs/executor/storageapi.h b/include/libs/executor/storageapi.h index 2b4eb06f68..bd31997640 100644 --- a/include/libs/executor/storageapi.h +++ b/include/libs/executor/storageapi.h @@ -392,6 +392,7 @@ typedef struct SStateStore { int32_t (*streamStateCommit)(SStreamState* pState); void (*streamStateDestroy)(SStreamState* pState, bool remove); int32_t (*streamStateDeleteCheckPoint)(SStreamState* pState, TSKEY mark); + void (*streamStateReloadInfo)(SStreamState* pState, TSKEY ts); } SStateStore; typedef struct SStorageAPI { diff --git a/include/libs/stream/streamState.h b/include/libs/stream/streamState.h index 7f9d20a9dd..7747df8595 100644 --- a/include/libs/stream/streamState.h +++ b/include/libs/stream/streamState.h @@ -138,6 +138,8 @@ int32_t streamStateCurPrev(SStreamState* pState, SStreamStateCur* pCur); int32_t streamStatePutParName(SStreamState* pState, int64_t groupId, const char* tbname); int32_t streamStateGetParName(SStreamState* pState, int64_t groupId, void** pVal); +void streamStateReloadInfo(SStreamState* pState, TSKEY ts); + /***compare func **/ typedef struct SStateChekpoint { diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 3cfde016f0..09583572ed 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -620,6 +620,9 @@ int32_t streamProcessCheckpointSourceReq(SStreamMeta* pMeta, SStreamTask* pTask, int32_t streamProcessCheckpointReq(SStreamMeta* pMeta, SStreamTask* pTask, SStreamCheckpointReq* pReq); int32_t streamProcessCheckpointRsp(SStreamMeta* pMeta, SStreamTask* pTask, SStreamCheckpointRsp* pRsp); +int32_t streamTaskReleaseState(SStreamTask* pTask); +int32_t streamTaskReloadState(SStreamTask* pTask); + #ifdef __cplusplus } #endif diff --git a/include/libs/stream/tstreamFileState.h b/include/libs/stream/tstreamFileState.h index 8496b0ea62..b2255013ca 100644 --- a/include/libs/stream/tstreamFileState.h +++ b/include/libs/stream/tstreamFileState.h @@ -49,6 +49,7 @@ int32_t recoverSnapshot(SStreamFileState* pFileState); int32_t getSnapshotIdList(SStreamFileState* pFileState, SArray* list); int32_t deleteExpiredCheckPoint(SStreamFileState* pFileState, TSKEY mark); int32_t streamFileStateGeSelectRowSize(SStreamFileState* pFileState); +void streamFileStateReloadInfo(SStreamFileState* pFileState, TSKEY ts); #ifdef __cplusplus } diff --git a/source/dnode/snode/src/snodeInitApi.c b/source/dnode/snode/src/snodeInitApi.c index f5e9245252..c046505630 100644 --- a/source/dnode/snode/src/snodeInitApi.c +++ b/source/dnode/snode/src/snodeInitApi.c @@ -101,6 +101,7 @@ void initStateStoreAPI(SStateStore* pStore) { pStore->streamStateCommit = streamStateCommit; pStore->streamStateDestroy= streamStateDestroy; pStore->streamStateDeleteCheckPoint = streamStateDeleteCheckPoint; + pStore->streamStateReloadInfo = streamStateReloadInfo; } void initFunctionStateStore(SFunctionStateStore* pStore) { diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 6f05d67b5e..54afdfed4e 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1111,6 +1111,8 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { if (pStreamTask == NULL) { // todo handle error } + // streamTaskReleaseState(pTask); + // streamTaskReloadState(pStreamTask); ASSERT(pStreamTask->info.taskLevel == TASK_LEVEL__SOURCE); diff --git a/source/dnode/vnode/src/vnd/vnodeInitApi.c b/source/dnode/vnode/src/vnd/vnodeInitApi.c index d2db6368a2..28a88561af 100644 --- a/source/dnode/vnode/src/vnd/vnodeInitApi.c +++ b/source/dnode/vnode/src/vnd/vnodeInitApi.c @@ -203,6 +203,7 @@ void initStateStoreAPI(SStateStore* pStore) { pStore->streamStateCommit = streamStateCommit; pStore->streamStateDestroy = streamStateDestroy; pStore->streamStateDeleteCheckPoint = streamStateDeleteCheckPoint; + pStore->streamStateReloadInfo = streamStateReloadInfo; } void initMetaReaderAPI(SStoreMetaReader* pMetaReader) { diff --git a/source/libs/executor/src/executor.c b/source/libs/executor/src/executor.c index f4135f58d1..9542a10389 100644 --- a/source/libs/executor/src/executor.c +++ b/source/libs/executor/src/executor.c @@ -1326,4 +1326,16 @@ SArray* getTableListInfo(const SExecTaskInfo* pTaskInfo) { SOperatorInfo* pOperator = pTaskInfo->pRoot; extractTableList(pArray, pOperator); return pArray; -} \ No newline at end of file +} + +int32_t qStreamOperatorReleaseState(qTaskInfo_t tInfo) { + SExecTaskInfo* pTaskInfo = (SExecTaskInfo*) tInfo; + pTaskInfo->pRoot->fpSet.releaseStreamStateFn(pTaskInfo->pRoot); + return 0; +} + +int32_t qStreamOperatorReloadState(qTaskInfo_t tInfo) { + SExecTaskInfo* pTaskInfo = (SExecTaskInfo*) tInfo; + pTaskInfo->pRoot->fpSet.reloadStreamStateFn(pTaskInfo->pRoot); + return 0; +} diff --git a/source/libs/executor/src/projectoperator.c b/source/libs/executor/src/projectoperator.c index e7de826d4b..c3459ace23 100644 --- a/source/libs/executor/src/projectoperator.c +++ b/source/libs/executor/src/projectoperator.c @@ -73,6 +73,20 @@ static void destroyIndefinitOperatorInfo(void* param) { taosMemoryFreeClear(param); } +void streamOperatorReleaseState(SOperatorInfo* pOperator) { + SOperatorInfo* downstream = pOperator->pDownstream[0]; + if (downstream->fpSet.releaseStreamStateFn) { + downstream->fpSet.releaseStreamStateFn(downstream); + } +} + +void streamOperatorReloadState(SOperatorInfo* pOperator) { + SOperatorInfo* downstream = pOperator->pDownstream[0]; + if (downstream->fpSet.reloadStreamStateFn) { + downstream->fpSet.reloadStreamStateFn(downstream); + } +} + SOperatorInfo* createProjectOperatorInfo(SOperatorInfo* downstream, SProjectPhysiNode* pProjPhyNode, SExecTaskInfo* pTaskInfo) { int32_t code = TSDB_CODE_SUCCESS; @@ -134,6 +148,7 @@ SOperatorInfo* createProjectOperatorInfo(SOperatorInfo* downstream, SProjectPhys pTaskInfo); pOperator->fpSet = createOperatorFpSet(optrDummyOpenFn, doProjectOperation, NULL, destroyProjectOperatorInfo, optrDefaultBufFn, NULL); + setOperatorStreamStateFn(pOperator, streamOperatorReleaseState, streamOperatorReloadState); code = appendDownstream(pOperator, &downstream, 1); if (code != TSDB_CODE_SUCCESS) { diff --git a/source/libs/executor/src/timewindowoperator.c b/source/libs/executor/src/timewindowoperator.c index 593d735039..50b9ca1c31 100644 --- a/source/libs/executor/src/timewindowoperator.c +++ b/source/libs/executor/src/timewindowoperator.c @@ -28,6 +28,7 @@ #define IS_FINAL_OP(op) ((op)->isFinal) #define DEAULT_DELETE_MARK (1000LL * 60LL * 60LL * 24LL * 365LL * 10LL); +#define STREAM_INTERVAL_OP_STATE_NAME "StreamIntervalHistoryState" #define STREAM_SESSION_OP_STATE_NAME "StreamSessionHistoryState" #define STREAM_STATE_OP_STATE_NAME "StreamStateHistoryState" @@ -2724,8 +2725,10 @@ int32_t getMaxFunResSize(SExprSupp* pSup, int32_t numOfCols) { } void streamIntervalReleaseState(SOperatorInfo* pOperator) { - if (pOperator->operatorType == QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_INTERVAL) { - return; + if (pOperator->operatorType != QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_INTERVAL) { + SStreamIntervalOperatorInfo* pInfo = pOperator->info; + int32_t resSize = sizeof(TSKEY); + pInfo->statestore.streamStateSaveInfo(pInfo->pState, STREAM_INTERVAL_OP_STATE_NAME, strlen(STREAM_INTERVAL_OP_STATE_NAME), &pInfo->twAggSup.maxTs, resSize); } SStreamIntervalOperatorInfo* pInfo = pOperator->info; SStorageAPI* pAPI = &pOperator->pTaskInfo->storageAPI; @@ -2737,6 +2740,15 @@ void streamIntervalReleaseState(SOperatorInfo* pOperator) { } void streamIntervalReloadState(SOperatorInfo* pOperator) { + if (pOperator->operatorType != QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_INTERVAL) { + SStreamIntervalOperatorInfo* pInfo = pOperator->info; + int32_t size = 0; + void* pBuf = NULL; + int32_t code = pInfo->statestore.streamStateGetInfo(pInfo->pState, STREAM_INTERVAL_OP_STATE_NAME, + strlen(STREAM_INTERVAL_OP_STATE_NAME), &pBuf, &size); + TSKEY ts = *(TSKEY*)pBuf; + pInfo->statestore.streamStateReloadInfo(pInfo->pState, ts); + } SOperatorInfo* downstream = pOperator->pDownstream[0]; if (downstream->fpSet.reloadStreamStateFn) { downstream->fpSet.reloadStreamStateFn(downstream); @@ -3651,7 +3663,6 @@ void streamSessionReloadState(SOperatorInfo* pOperator) { SStreamSessionAggOperatorInfo* pInfo = pOperator->info; SStreamAggSupporter* pAggSup = &pInfo->streamAggSup; SResultWindowInfo winInfo = {0}; - SSessionKey seKey = {.win.skey = INT64_MIN, .win.ekey = INT64_MIN, .groupId = 0}; int32_t size = 0; void* pBuf = NULL; int32_t code = pAggSup->stateStore.streamStateGetInfo(pAggSup->pState, STREAM_SESSION_OP_STATE_NAME, @@ -4352,7 +4363,7 @@ SOperatorInfo* createStreamStateAggOperatorInfo(SOperatorInfo* downstream, SPhys pInfo, pTaskInfo); pOperator->fpSet = createOperatorFpSet(optrDummyOpenFn, doStreamStateAgg, NULL, destroyStreamStateOperatorInfo, optrDefaultBufFn, NULL); - setOperatorStreamStateFn(pOperator, streamStateReleaseState, streamSessionReloadState); + setOperatorStreamStateFn(pOperator, streamStateReleaseState, streamStateReloadState); initDownStream(downstream, &pInfo->streamAggSup, pOperator->operatorType, pInfo->primaryTsIndex, &pInfo->twAggSup); code = appendDownstream(pOperator, &downstream, 1); if (code != TSDB_CODE_SUCCESS) { diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index 907360dff0..96a10b9cd1 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -562,3 +562,15 @@ int32_t streamTryExec(SStreamTask* pTask) { return 0; } + +int32_t streamTaskReleaseState(SStreamTask* pTask) { + void* exec = pTask->exec.pExecutor; + int32_t code = qStreamOperatorReleaseState(exec); + return code; +} + +int32_t streamTaskReloadState(SStreamTask* pTask) { + void* exec = pTask->exec.pExecutor; + int32_t code = qStreamOperatorReloadState(exec); + return code; +} diff --git a/source/libs/stream/src/streamState.c b/source/libs/stream/src/streamState.c index 967c7733c9..aeb25c2368 100644 --- a/source/libs/stream/src/streamState.c +++ b/source/libs/stream/src/streamState.c @@ -1113,6 +1113,10 @@ int32_t streamStateDeleteCheckPoint(SStreamState* pState, TSKEY mark) { #endif } +void streamStateReloadInfo(SStreamState* pState, TSKEY ts) { + streamFileStateReloadInfo(pState->pFileState, ts); +} + #if 0 char* streamStateSessionDump(SStreamState* pState) { SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); diff --git a/source/libs/stream/src/tstreamFileState.c b/source/libs/stream/src/tstreamFileState.c index 2d730f3eda..0799671bce 100644 --- a/source/libs/stream/src/tstreamFileState.c +++ b/source/libs/stream/src/tstreamFileState.c @@ -524,3 +524,8 @@ int32_t recoverSnapshot(SStreamFileState* pFileState) { } int32_t streamFileStateGeSelectRowSize(SStreamFileState* pFileState) { return pFileState->selectivityRowSize; } + +void streamFileStateReloadInfo(SStreamFileState* pFileState, TSKEY ts) { + pFileState->flushMark = TMAX(pFileState->flushMark, ts); + pFileState->maxTs = TMAX(pFileState->maxTs, ts); +} From f3df37bf284aa3ee2df908c7b6fc17b046dd3d2a Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 15 Jun 2023 17:56:58 +0800 Subject: [PATCH 35/83] fix(stream): fix memory leak. --- source/dnode/mnode/impl/src/mndDef.c | 1 + source/libs/executor/src/timewindowoperator.c | 2 ++ 2 files changed, 3 insertions(+) diff --git a/source/dnode/mnode/impl/src/mndDef.c b/source/dnode/mnode/impl/src/mndDef.c index af82040760..dfd54ab254 100644 --- a/source/dnode/mnode/impl/src/mndDef.c +++ b/source/dnode/mnode/impl/src/mndDef.c @@ -176,6 +176,7 @@ void tFreeStreamObj(SStreamObj *pStream) { } taosArrayDestroy(pStream->tasks); + taosArrayDestroy(pStream->pHTasksList); // tagSchema.pSchema if (pStream->tagSchema.nCols > 0) { diff --git a/source/libs/executor/src/timewindowoperator.c b/source/libs/executor/src/timewindowoperator.c index 50b9ca1c31..23932dcbc7 100644 --- a/source/libs/executor/src/timewindowoperator.c +++ b/source/libs/executor/src/timewindowoperator.c @@ -2903,12 +2903,14 @@ void destroyStreamSessionAggOperatorInfo(void* param) { } taosArrayDestroy(pInfo->pChildren); } + colDataDestroy(&pInfo->twAggSup.timeWindowData); blockDataDestroy(pInfo->pDelRes); blockDataDestroy(pInfo->pWinBlock); blockDataDestroy(pInfo->pUpdateRes); tSimpleHashCleanup(pInfo->pStDeleted); + taosArrayDestroy(pInfo->historyWins); taosMemoryFreeClear(param); } From a3e1882fb2e20c49b6ca0bc6ee635f91d87a5313 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 15 Jun 2023 18:17:38 +0800 Subject: [PATCH 36/83] fix: fix dead lock. --- source/dnode/vnode/src/meta/metaQuery.c | 2 -- 1 file changed, 2 deletions(-) diff --git a/source/dnode/vnode/src/meta/metaQuery.c b/source/dnode/vnode/src/meta/metaQuery.c index 1f34c60b5e..ac2b533462 100644 --- a/source/dnode/vnode/src/meta/metaQuery.c +++ b/source/dnode/vnode/src/meta/metaQuery.c @@ -222,8 +222,6 @@ SMTbCursor *metaOpenTbCursor(void *pVnode) { } SVnode* pVnodeObj = pVnode; - metaReaderDoInit(&pTbCur->mr, pVnodeObj->pMeta, 0); - // tdbTbcMoveToFirst((TBC *)pTbCur->pDbc); pTbCur->pMeta = pVnodeObj->pMeta; pTbCur->paused = 1; From 0aa141ec77c88bd20ed8f91a009f3e256767f1b0 Mon Sep 17 00:00:00 2001 From: liuyao <54liuyao@163.com> Date: Thu, 15 Jun 2023 18:24:40 +0800 Subject: [PATCH 37/83] use stream task state --- source/dnode/vnode/src/tq/tq.c | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 417a582d78..b938b2b3b0 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -778,7 +778,11 @@ int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t ver) { pTask->dataRange.range.minVer = ver; if (pTask->info.taskLevel == TASK_LEVEL__SOURCE) { - pTask->pState = streamStateOpen(pTq->pStreamMeta->path, pTask, false, -1, -1); + SStreamTask* pSateTask = pTask; + // if (pTask->info.fillHistory) { + // pSateTask = *(SStreamTask**)taosHashGet(pTq->pStreamMeta->pTasks, &pTask->streamTaskId.taskId, sizeof(int32_t)); + // } + pTask->pState = streamStateOpen(pTq->pStreamMeta->path, pSateTask, false, -1, -1); if (pTask->pState == NULL) { return -1; } @@ -793,7 +797,11 @@ int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t ver) { qSetTaskId(pTask->exec.pExecutor, pTask->id.taskId, pTask->id.streamId); } else if (pTask->info.taskLevel == TASK_LEVEL__AGG) { - pTask->pState = streamStateOpen(pTq->pStreamMeta->path, pTask, false, -1, -1); + SStreamTask* pSateTask = pTask; + // if (pTask->info.fillHistory) { + // pSateTask = *(SStreamTask**)taosHashGet(pTq->pStreamMeta->pTasks, &pTask->streamTaskId.taskId, sizeof(int32_t)); + // } + pTask->pState = streamStateOpen(pTq->pStreamMeta->path, pSateTask, false, -1, -1); if (pTask->pState == NULL) { return -1; } From 9f9171719a956f0816c52412b90a5eb347f336c5 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Thu, 15 Jun 2023 21:29:52 +0800 Subject: [PATCH 38/83] support fill history --- cmake/rocksdb_CMakeLists.txt.in | 17 +- include/libs/function/function.h | 54 +- include/libs/stream/tstream.h | 75 +-- source/dnode/mnode/impl/src/mndScheduler.c | 63 +- source/libs/stream/inc/streamBackendRocksdb.h | 1 + source/libs/stream/inc/streamInc.h | 3 +- source/libs/stream/src/streamBackendRocksdb.c | 574 ++++++++++-------- source/libs/stream/src/streamMeta.c | 17 +- source/libs/stream/src/streamState.c | 85 +-- 9 files changed, 506 insertions(+), 383 deletions(-) diff --git a/cmake/rocksdb_CMakeLists.txt.in b/cmake/rocksdb_CMakeLists.txt.in index ba4a404af6..af6ec3c56c 100644 --- a/cmake/rocksdb_CMakeLists.txt.in +++ b/cmake/rocksdb_CMakeLists.txt.in @@ -1,11 +1,14 @@ # rocksdb ExternalProject_Add(rocksdb - GIT_REPOSITORY https://github.com/facebook/rocksdb.git - GIT_TAG v8.1.1 - SOURCE_DIR "${TD_CONTRIB_DIR}/rocksdb" - CONFIGURE_COMMAND "" - BUILD_COMMAND "" - INSTALL_COMMAND "" - TEST_COMMAND "" + URL https://github.com/facebook/rocksdb/archive/refs/tags/v8.1.1.tar.gz + URL_HASH MD5=3b4c97ee45df9c8a5517308d31ab008b + DOWNLOAD_NO_PROGRESS 1 + DOWNLOAD_DIR "${TD_CONTRIB_DIR}/deps-download" + SOURCE_DIR "${TD_CONTRIB_DIR}/rocksdb" + CONFIGURE_COMMAND "" + BUILD_COMMAND "" + INSTALL_COMMAND "" + TEST_COMMAND "" ) + diff --git a/include/libs/function/function.h b/include/libs/function/function.h index c92ce254a8..8b08db8f78 100644 --- a/include/libs/function/function.h +++ b/include/libs/function/function.h @@ -129,30 +129,38 @@ typedef struct SSerializeDataHandle { } SSerializeDataHandle; // incremental state storage -typedef struct STdbState { - void *rocksdb; - void **pHandle; - void *writeOpts; - void *readOpts; - void **cfOpts; - void *dbOpt; - struct SStreamTask *pOwner; - void *param; - void *env; - SListNode *pComparNode; - void *pBackend; - char idstr[64]; - void *compactFactory; - TdThreadRwlock rwLock; - void *db; - void *pStateDb; - void *pFuncStateDb; - void *pFillStateDb; // todo refactor - void *pSessionStateDb; - void *pParNameDb; - void *pParTagDb; - void *txn; +typedef struct SBackendWrapper { + void *rocksdb; + void **pHandle; + void *writeOpts; + void *readOpts; + void **cfOpts; + void *dbOpt; + void *param; + void *env; + SListNode *pComparNode; + void *pBackend; + void *compactFactory; + TdThreadRwlock rwLock; + bool remove; + int64_t backendId; + char idstr[64]; +} SBackendWrapper; +typedef struct STdbState { + SBackendWrapper *pBackendWrapper; + int64_t backendWrapperId; + char idstr[64]; + + struct SStreamTask *pOwner; + void *db; + void *pStateDb; + void *pFuncStateDb; + void *pFillStateDb; // todo refactor + void *pSessionStateDb; + void *pParNameDb; + void *pParTagDb; + void *txn; } STdbState; typedef struct { diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 09583572ed..f668588e1a 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -44,8 +44,8 @@ enum { TASK_STATUS__DROPPING, TASK_STATUS__FAIL, TASK_STATUS__STOP, - TASK_STATUS__SCAN_HISTORY, // stream task scan history data by using tsdbread in the stream scanner - TASK_STATUS__HALT, // stream task will handle all data in the input queue, and then paused + TASK_STATUS__SCAN_HISTORY, // stream task scan history data by using tsdbread in the stream scanner + TASK_STATUS__HALT, // stream task will handle all data in the input queue, and then paused TASK_STATUS__PAUSE, }; @@ -266,11 +266,11 @@ typedef struct SCheckpointInfo { } SCheckpointInfo; typedef struct SStreamStatus { - int8_t taskStatus; - int8_t checkDownstream; - int8_t schedStatus; - int8_t keepTaskStatus; - bool transferState; + int8_t taskStatus; + int8_t checkDownstream; + int8_t schedStatus; + int8_t keepTaskStatus; + bool transferState; TdThreadMutex lock; } SStreamStatus; @@ -280,19 +280,19 @@ typedef struct SHistDataRange { } SHistDataRange; typedef struct SSTaskBasicInfo { - int32_t nodeId; // vgroup id or snode id - SEpSet epSet; - int32_t selfChildId; - int32_t totalLevel; - int8_t taskLevel; - int8_t fillHistory; // is fill history task or not + int32_t nodeId; // vgroup id or snode id + SEpSet epSet; + int32_t selfChildId; + int32_t totalLevel; + int8_t taskLevel; + int8_t fillHistory; // is fill history task or not } SSTaskBasicInfo; typedef struct SDispatchMsgInfo { - void* pData; // current dispatch data - int16_t msgType; // dispatch msg type - int32_t retryCount; // retry send data count - int64_t blockingTs; // output blocking timestamp + void* pData; // current dispatch data + int16_t msgType; // dispatch msg type + int32_t retryCount; // retry send data count + int64_t blockingTs; // output blocking timestamp } SDispatchMsgInfo; typedef struct { @@ -351,21 +351,22 @@ struct SStreamTask { // meta typedef struct SStreamMeta { - char* path; - TDB* db; - TTB* pTaskDb; - TTB* pCheckpointDb; - SHashObj* pTasks; - SArray* pTaskList; // SArray - void* ahandle; - TXN* txn; - FTaskExpand* expandFunc; - int32_t vgId; - SRWLatch lock; - int32_t walScanCounter; - void* streamBackend; - int64_t streamBackendRid; - SHashObj* pTaskBackendUnique; + char* path; + TDB* db; + TTB* pTaskDb; + TTB* pCheckpointDb; + SHashObj* pTasks; + SArray* pTaskList; // SArray + void* ahandle; + TXN* txn; + FTaskExpand* expandFunc; + int32_t vgId; + SRWLatch lock; + int32_t walScanCounter; + void* streamBackend; + int64_t streamBackendRid; + SHashObj* pTaskBackendUnique; + TdThreadMutex backendMutex; } SStreamMeta; int32_t tEncodeStreamEpInfo(SEncoder* pEncoder, const SStreamChildEpInfo* pInfo); @@ -578,16 +579,16 @@ int32_t streamTaskScanHistoryDataComplete(SStreamTask* pTask); int32_t streamStartRecoverTask(SStreamTask* pTask, int8_t igUntreated); // common -int32_t streamSetParamForScanHistoryData(SStreamTask* pTask); -int32_t streamRestoreParam(SStreamTask* pTask); -int32_t streamSetStatusNormal(SStreamTask* pTask); +int32_t streamSetParamForScanHistoryData(SStreamTask* pTask); +int32_t streamRestoreParam(SStreamTask* pTask); +int32_t streamSetStatusNormal(SStreamTask* pTask); const char* streamGetTaskStatusStr(int32_t status); // source level -int32_t streamSetParamForStreamScanner(SStreamTask* pTask, SVersionRange *pVerRange, STimeWindow* pWindow); +int32_t streamSetParamForStreamScanner(SStreamTask* pTask, SVersionRange* pVerRange, STimeWindow* pWindow); int32_t streamBuildSourceRecover1Req(SStreamTask* pTask, SStreamScanHistoryReq* pReq, int8_t igUntreated); int32_t streamSourceScanHistoryData(SStreamTask* pTask); -//int32_t streamSourceRecoverScanStep2(SStreamTask* pTask, int64_t ver); +// int32_t streamSourceRecoverScanStep2(SStreamTask* pTask, int64_t ver); int32_t streamDispatchScanHistoryFinishMsg(SStreamTask* pTask); int32_t streamDispatchTransferStateMsg(SStreamTask* pTask); diff --git a/source/dnode/mnode/impl/src/mndScheduler.c b/source/dnode/mnode/impl/src/mndScheduler.c index 24b2a60898..f41a878b4a 100644 --- a/source/dnode/mnode/impl/src/mndScheduler.c +++ b/source/dnode/mnode/impl/src/mndScheduler.c @@ -25,8 +25,9 @@ #define SINK_NODE_LEVEL (0) extern bool tsDeployOnSnode; -static int32_t mndAddSinkTaskToStream(SStreamObj* pStream, SArray* pTaskList, SMnode* pMnode, int32_t vgId, SVgObj* pVgroup, int32_t fillHistory); -static void setFixedDownstreamEpInfo(SStreamTask* pDstTask, const SStreamTask* pTask); +static int32_t mndAddSinkTaskToStream(SStreamObj* pStream, SArray* pTaskList, SMnode* pMnode, int32_t vgId, + SVgObj* pVgroup, int32_t fillHistory); +static void setFixedDownstreamEpInfo(SStreamTask* pDstTask, const SStreamTask* pTask); int32_t mndConvertRsmaTask(char** pDst, int32_t* pDstLen, const char* ast, int64_t uid, int8_t triggerType, int64_t watermark, int64_t deleteMark) { @@ -101,13 +102,13 @@ int32_t mndSetSinkTaskInfo(SStreamObj* pStream, SStreamTask* pTask) { return 0; } -int32_t mndAddDispatcherForInternalTask(SMnode* pMnode, SStreamObj* pStream, SArray* pSinkNodeList, SStreamTask* pTask) { +int32_t mndAddDispatcherForInternalTask(SMnode* pMnode, SStreamObj* pStream, SArray* pSinkNodeList, + SStreamTask* pTask) { bool isShuffle = false; if (pStream->fixedSinkVgId == 0) { SDbObj* pDb = mndAcquireDb(pMnode, pStream->targetDb); if (pDb != NULL && pDb->cfg.numOfVgroups > 1) { - isShuffle = true; pTask->outputType = TASK_OUTPUT__SHUFFLE_DISPATCH; pTask->msgInfo.msgType = TDMT_STREAM_TASK_DISPATCH; @@ -203,8 +204,8 @@ SVgObj* mndSchedFetchOneVg(SMnode* pMnode, int64_t dbUid) { // create sink node for each vgroup. int32_t mndAddShuffleSinkTasksToStream(SMnode* pMnode, SArray* pTaskList, SStreamObj* pStream, int32_t fillHistory) { - SSdb* pSdb = pMnode->pSdb; - void* pIter = NULL; + SSdb* pSdb = pMnode->pSdb; + void* pIter = NULL; while (1) { SVgObj* pVgroup = NULL; @@ -225,7 +226,8 @@ int32_t mndAddShuffleSinkTasksToStream(SMnode* pMnode, SArray* pTaskList, SStrea return 0; } -int32_t mndAddSinkTaskToStream(SStreamObj* pStream, SArray* pTaskList, SMnode* pMnode, int32_t vgId, SVgObj* pVgroup, int32_t fillHistory) { +int32_t mndAddSinkTaskToStream(SStreamObj* pStream, SArray* pTaskList, SMnode* pMnode, int32_t vgId, SVgObj* pVgroup, + int32_t fillHistory) { SStreamTask* pTask = tNewStreamTask(pStream->uid, TASK_LEVEL__SINK, fillHistory, 0, pTaskList); if (pTask == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; @@ -248,7 +250,7 @@ static int32_t addSourceStreamTask(SMnode* pMnode, SVgObj* pVgroup, SArray* pTas // todo set the correct ts, which should be last key of queried table. pTask->dataRange.window.skey = INT64_MIN; - pTask->dataRange.window.ekey = 1685959190000;//taosGetTimestampMs(); + pTask->dataRange.window.ekey = 1685959190000; // taosGetTimestampMs(); mDebug("add source task 0x%x window:%" PRId64 " - %" PRId64, pTask->id.taskId, pTask->dataRange.window.skey, pTask->dataRange.window.ekey); @@ -298,10 +300,10 @@ int32_t setEpToDownstreamTask(SStreamTask* pTask, SStreamTask* pDownstream) { return TSDB_CODE_OUT_OF_MEMORY; } - if(pDownstream->pUpstreamEpInfoList == NULL) { + if (pDownstream->pUpstreamEpInfoList == NULL) { pDownstream->pUpstreamEpInfoList = taosArrayInit(4, POINTER_BYTES); } - + taosArrayPush(pDownstream->pUpstreamEpInfoList, &pEpInfo); return TSDB_CODE_SUCCESS; } @@ -314,7 +316,7 @@ static SArray* addNewTaskList(SArray* pTasksList) { // set the history task id static void setHTasksId(SArray* pTaskList, const SArray* pHTaskList) { - for(int32_t i = 0; i < taosArrayGetSize(pTaskList); ++i) { + for (int32_t i = 0; i < taosArrayGetSize(pTaskList); ++i) { SStreamTask** pStreamTask = taosArrayGet(pTaskList, i); SStreamTask** pHTask = taosArrayGet(pHTaskList, i); @@ -339,7 +341,7 @@ static int32_t addSourceTasksForOneLevelStream(SMnode* pMnode, const SQueryPlan* pHTaskList = addNewTaskList(pStream->pHTasksList); } - SSdb* pSdb = pMnode->pSdb; + SSdb* pSdb = pMnode->pSdb; SNodeListNode* inner = (SNodeListNode*)nodesListGetNode(pPlan->pSubplans, 0); if (LIST_LENGTH(inner->pNodeList) != 1) { terrno = TSDB_CODE_QRY_INVALID_INPUT; @@ -367,8 +369,8 @@ static int32_t addSourceTasksForOneLevelStream(SMnode* pMnode, const SQueryPlan* // new stream task SArray** pSinkTaskList = taosArrayGet(pStream->tasks, SINK_NODE_LEVEL); - int32_t code = addSourceStreamTask(pMnode, pVgroup, pTaskList, *pSinkTaskList, pStream, plan, pStream->uid, - 0, hasExtraSink); + int32_t code = + addSourceStreamTask(pMnode, pVgroup, pTaskList, *pSinkTaskList, pStream, plan, pStream->uid, 0, hasExtraSink); if (code != TSDB_CODE_SUCCESS) { sdbRelease(pSdb, pVgroup); return -1; @@ -390,8 +392,8 @@ static int32_t addSourceTasksForOneLevelStream(SMnode* pMnode, const SQueryPlan* return TSDB_CODE_SUCCESS; } -static int32_t doAddSourceTask(SArray* pTaskList, int8_t fillHistory, int64_t uid, SStreamTask* pDownstreamTask, SMnode* pMnode, - SSubplan* pPlan, SVgObj* pVgroup) { +static int32_t doAddSourceTask(SArray* pTaskList, int8_t fillHistory, int64_t uid, SStreamTask* pDownstreamTask, + SMnode* pMnode, SSubplan* pPlan, SVgObj* pVgroup) { SStreamTask* pTask = tNewStreamTask(uid, TASK_LEVEL__SOURCE, fillHistory, 0, pTaskList); if (pTask == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; @@ -400,9 +402,10 @@ static int32_t doAddSourceTask(SArray* pTaskList, int8_t fillHistory, int64_t ui // todo set the correct ts, which should be last key of queried table. pTask->dataRange.window.skey = INT64_MIN; - pTask->dataRange.window.ekey = 1685959190000;//taosGetTimestampMs(); + pTask->dataRange.window.ekey = 1685959190000; // taosGetTimestampMs(); - mDebug("s-task:0x%x set time window:%"PRId64" - %"PRId64, pTask->id.taskId, pTask->dataRange.window.skey, pTask->dataRange.window.ekey); + mDebug("s-task:0x%x set time window:%" PRId64 " - %" PRId64, pTask->id.taskId, pTask->dataRange.window.skey, + pTask->dataRange.window.ekey); // all the source tasks dispatch result to a single agg node. setFixedDownstreamEpInfo(pTask, pDownstreamTask); @@ -413,8 +416,8 @@ static int32_t doAddSourceTask(SArray* pTaskList, int8_t fillHistory, int64_t ui return setEpToDownstreamTask(pTask, pDownstreamTask); } -static int32_t doAddAggTask(uint64_t uid, SArray* pTaskList, SArray* pSinkNodeList, SMnode* pMnode, SStreamObj* pStream, int32_t fillHistory, - SStreamTask** pAggTask) { +static int32_t doAddAggTask(uint64_t uid, SArray* pTaskList, SArray* pSinkNodeList, SMnode* pMnode, SStreamObj* pStream, + int32_t fillHistory, SStreamTask** pAggTask) { *pAggTask = tNewStreamTask(uid, TASK_LEVEL__AGG, fillHistory, pStream->conf.triggerParam, pTaskList); if (*pAggTask == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; @@ -472,7 +475,8 @@ static int32_t addAggTask(SStreamObj* pStream, SMnode* pMnode, SQueryPlan* pPlan SArray* pHSinkNodeList = taosArrayGetP(pStream->pHTasksList, SINK_NODE_LEVEL); *pHAggTask = NULL; - code = doAddAggTask(pStream->hTaskUid, pHAggTaskList, pHSinkNodeList, pMnode, pStream, pStream->conf.fillHistory, pHAggTask); + code = doAddAggTask(pStream->hTaskUid, pHAggTaskList, pHSinkNodeList, pMnode, pStream, pStream->conf.fillHistory, + pHAggTask); if (code != TSDB_CODE_SUCCESS) { if (pSnode != NULL) { sdbRelease(pSdb, pSnode); @@ -538,7 +542,8 @@ static int32_t addSourceTasksForMultiLevelStream(SMnode* pMnode, SQueryPlan* pPl } if (pStream->conf.fillHistory) { - code = doAddSourceTask(pHSourceTaskList, pStream->conf.fillHistory, pStream->hTaskUid, pHDownstreamTask, pMnode, plan, pVgroup); + code = doAddSourceTask(pHSourceTaskList, pStream->conf.fillHistory, pStream->hTaskUid, pHDownstreamTask, pMnode, + plan, pVgroup); sdbRelease(pSdb, pVgroup); if (code != TSDB_CODE_SUCCESS) { @@ -552,7 +557,8 @@ static int32_t addSourceTasksForMultiLevelStream(SMnode* pMnode, SQueryPlan* pPl return TSDB_CODE_SUCCESS; } -static int32_t addSinkTasks(SArray* pTasksList, SMnode* pMnode, SStreamObj* pStream, SArray** pCreatedTaskList, int32_t fillHistory) { +static int32_t addSinkTasks(SArray* pTasksList, SMnode* pMnode, SStreamObj* pStream, SArray** pCreatedTaskList, + int32_t fillHistory) { SArray* pSinkTaskList = addNewTaskList(pTasksList); if (pStream->fixedSinkVgId == 0) { if (mndAddShuffleSinkTasksToStream(pMnode, pSinkTaskList, pStream, fillHistory) < 0) { @@ -560,7 +566,8 @@ static int32_t addSinkTasks(SArray* pTasksList, SMnode* pMnode, SStreamObj* pStr return -1; } } else { - if (mndAddSinkTaskToStream(pStream, pSinkTaskList, pMnode, pStream->fixedSinkVgId, &pStream->fixedSinkVg, fillHistory) < 0) { + if (mndAddSinkTaskToStream(pStream, pSinkTaskList, pMnode, pStream->fixedSinkVgId, &pStream->fixedSinkVg, + fillHistory) < 0) { // TODO free return -1; } @@ -571,7 +578,7 @@ static int32_t addSinkTasks(SArray* pTasksList, SMnode* pMnode, SStreamObj* pStr } static int32_t doScheduleStream(SStreamObj* pStream, SMnode* pMnode, SQueryPlan* pPlan) { - SSdb* pSdb = pMnode->pSdb; + SSdb* pSdb = pMnode->pSdb; int32_t numOfPlanLevel = LIST_LENGTH(pPlan->pSubplans); bool hasExtraSink = false; @@ -655,8 +662,8 @@ int32_t mndSchedInitSubEp(SMnode* pMnode, const SMqTopicObj* pTopic, SMqSubscrib terrno = TSDB_CODE_QRY_INVALID_INPUT; return -1; } - }else if(pTopic->subType == TOPIC_SUB_TYPE__TABLE && pTopic->ast != NULL){ - SNode *pAst = NULL; + } else if (pTopic->subType == TOPIC_SUB_TYPE__TABLE && pTopic->ast != NULL) { + SNode* pAst = NULL; if (nodesStringToNode(pTopic->ast, &pAst) != 0) { mError("topic:%s, failed to create since %s", pTopic->name, terrstr()); return -1; @@ -671,7 +678,7 @@ int32_t mndSchedInitSubEp(SMnode* pMnode, const SMqTopicObj* pTopic, SMqSubscrib nodesDestroyNode(pAst); } - if(pPlan){ + if (pPlan) { int32_t levelNum = LIST_LENGTH(pPlan->pSubplans); if (levelNum != 1) { qDestroyQueryPlan(pPlan); diff --git a/source/libs/stream/inc/streamBackendRocksdb.h b/source/libs/stream/inc/streamBackendRocksdb.h index da4e442f1a..6195200ba5 100644 --- a/source/libs/stream/inc/streamBackendRocksdb.h +++ b/source/libs/stream/inc/streamBackendRocksdb.h @@ -46,6 +46,7 @@ typedef struct { void* streamBackendInit(const char* path); void streamBackendCleanup(void* arg); +void streamBackendHandleCleanup(void* arg); SListNode* streamBackendAddCompare(void* backend, void* arg); void streamBackendDelCompare(void* backend, void* arg); diff --git a/source/libs/stream/inc/streamInc.h b/source/libs/stream/inc/streamInc.h index aeb35d9ad5..19a8044b71 100644 --- a/source/libs/stream/inc/streamInc.h +++ b/source/libs/stream/inc/streamInc.h @@ -50,11 +50,12 @@ int32_t streamDispatchAllBlocks(SStreamTask* pTask, const SStreamDataBlock* pDat int32_t streamDispatchCheckMsg(SStreamTask* pTask, const SStreamTaskCheckReq* pReq, int32_t nodeId, SEpSet* pEpSet); int32_t streamDoDispatchScanHistoryFinishMsg(SStreamTask* pTask, const SStreamRecoverFinishReq* pReq, int32_t vgId, - SEpSet* pEpSet); + SEpSet* pEpSet); SStreamQueueItem* streamMergeQueueItem(SStreamQueueItem* dst, SStreamQueueItem* pElem); extern int32_t streamBackendId; +extern int32_t streamBackendWrapperId; #ifdef __cplusplus } diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index c4e3f147e0..e5aa08edaf 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -40,16 +40,8 @@ typedef struct { rocksdb_comparator_t** pCompares; } RocksdbCfInst; -uint32_t nextPow2(uint32_t x) { - x = x - 1; - x = x | (x >> 1); - x = x | (x >> 2); - x = x | (x >> 4); - x = x | (x >> 8); - x = x | (x >> 16); - return x + 1; -} -int32_t streamStateOpenBackendCf(void* backend, char* name, char** cfs, int32_t nCf); +uint32_t nextPow2(uint32_t x); +int32_t streamStateOpenBackendCf(void* backend, char* name, char** cfs, int32_t nCf); void destroyRocksdbCfInst(RocksdbCfInst* inst); @@ -71,7 +63,22 @@ typedef int (*BackendCmpFunc)(void* state, const char* aBuf, size_t aLen, const typedef void (*DestroyFunc)(void* state); typedef int32_t (*EncodeValueFunc)(void* value, int32_t vlen, int64_t ttl, char** dest); typedef int32_t (*DecodeValueFunc)(void* value, int32_t vlen, int64_t* ttl, char** dest); +typedef struct { + const char* key; + int32_t len; + int idx; + BackendCmpFunc cmpFunc; + EncodeFunc enFunc; + DecodeFunc deFunc; + ToStringFunc toStrFunc; + CompareName cmpName; + DestroyFunc detroyFunc; + EncodeValueFunc enValueFunc; + DecodeValueFunc deValueFunc; +} SCfInit; + +#define GEN_COLUMN_FAMILY_NAME(name, idstr, SUFFIX) sprintf(name, "%s_%s", idstr, (SUFFIX)); const char* compareDefaultName(void* name); const char* compareStateName(void* name); const char* compareWinKeyName(void* name); @@ -80,6 +87,62 @@ const char* compareFuncKeyName(void* name); const char* compareParKeyName(void* name); const char* comparePartagKeyName(void* name); +int defaultKeyComp(void* state, const char* aBuf, size_t aLen, const char* bBuf, size_t bLen); +int defaultKeyEncode(void* k, char* buf); +int defaultKeyDecode(void* k, char* buf); +int defaultKeyToString(void* k, char* buf); + +int stateKeyDBComp(void* state, const char* aBuf, size_t aLen, const char* bBuf, size_t bLen); +int stateKeyEncode(void* k, char* buf); +int stateKeyDecode(void* k, char* buf); +int stateKeyToString(void* k, char* buf); + +int stateSessionKeyDBComp(void* state, const char* aBuf, size_t aLen, const char* bBuf, size_t bLen); +int stateSessionKeyEncode(void* ses, char* buf); +int stateSessionKeyDecode(void* ses, char* buf); +int stateSessionKeyToString(void* k, char* buf); + +int winKeyDBComp(void* state, const char* aBuf, size_t aLen, const char* bBuf, size_t bLen); +int winKeyEncode(void* k, char* buf); +int winKeyDecode(void* k, char* buf); +int winKeyToString(void* k, char* buf); + +int tupleKeyDBComp(void* state, const char* aBuf, size_t aLen, const char* bBuf, size_t bLen); +int tupleKeyEncode(void* k, char* buf); +int tupleKeyDecode(void* k, char* buf); +int tupleKeyToString(void* k, char* buf); + +int parKeyDBComp(void* state, const char* aBuf, size_t aLen, const char* bBuf, size_t bLen); +int parKeyEncode(void* k, char* buf); +int parKeyDecode(void* k, char* buf); +int parKeyToString(void* k, char* buf); + +int stremaValueEncode(void* k, char* buf); +int streamValueDecode(void* k, char* buf); +int32_t streamValueToString(void* k, char* buf); +int32_t streaValueIsStale(void* k, int64_t ts); +void destroyFunc(void* arg); + +int32_t encodeValueFunc(void* value, int32_t vlen, int64_t ttl, char** dest); +int32_t decodeValueFunc(void* value, int32_t vlen, int64_t* ttl, char** dest); + +SCfInit ginitDict[] = { + {"default", 7, 0, defaultKeyComp, defaultKeyEncode, defaultKeyDecode, defaultKeyToString, compareDefaultName, + destroyFunc, encodeValueFunc, decodeValueFunc}, + {"state", 5, 1, stateKeyDBComp, stateKeyEncode, stateKeyDecode, stateKeyToString, compareStateName, destroyFunc, + encodeValueFunc, decodeValueFunc}, + {"fill", 4, 2, winKeyDBComp, winKeyEncode, winKeyDecode, winKeyToString, compareWinKeyName, destroyFunc, + encodeValueFunc, decodeValueFunc}, + {"sess", 4, 3, stateSessionKeyDBComp, stateSessionKeyEncode, stateSessionKeyDecode, stateSessionKeyToString, + compareSessionKeyName, destroyFunc, encodeValueFunc, decodeValueFunc}, + {"func", 4, 4, tupleKeyDBComp, tupleKeyEncode, tupleKeyDecode, tupleKeyToString, compareFuncKeyName, destroyFunc, + encodeValueFunc, decodeValueFunc}, + {"parname", 7, 5, parKeyDBComp, parKeyEncode, parKeyDecode, parKeyToString, compareParKeyName, destroyFunc, + encodeValueFunc, decodeValueFunc}, + {"partag", 6, 6, parKeyDBComp, parKeyEncode, parKeyDecode, parKeyToString, comparePartagKeyName, destroyFunc, + encodeValueFunc, decodeValueFunc}, +}; + void* streamBackendInit(const char* path) { qDebug("start to init stream backend at %s", path); SBackendHandle* pHandle = taosMemoryCalloc(1, sizeof(SBackendHandle)); @@ -189,6 +252,69 @@ void streamBackendCleanup(void* arg) { qDebug("destroy stream backend backend:%p", pHandle); return; } +void streamBackendHandleCleanup(void* arg) { + SBackendWrapper* wrapper = arg; + bool remove = false; + + qDebug("start to do-close backendwrapper %p, %s", wrapper, wrapper->idstr); + if (wrapper->rocksdb == NULL) { + return; + } + + int cfLen = sizeof(ginitDict) / sizeof(ginitDict[0]); + + char* err = NULL; + if (wrapper->remove) { + for (int i = 0; i < cfLen; i++) { + if (wrapper->pHandle[i] != NULL) + rocksdb_drop_column_family(wrapper->rocksdb, ((rocksdb_column_family_handle_t**)wrapper->pHandle)[i], &err); + if (err != NULL) { + // qError("failed to create cf:%s_%s, reason:%s", wrapper->idstr, ginitDict[i].key, err); + taosMemoryFreeClear(err); + } + } + } else { + rocksdb_flushoptions_t* flushOpt = rocksdb_flushoptions_create(); + for (int i = 0; i < cfLen; i++) { + if (wrapper->pHandle[i] != NULL) rocksdb_flush_cf(wrapper->rocksdb, flushOpt, wrapper->pHandle[i], &err); + if (err != NULL) { + qError("failed to create cf:%s_%s, reason:%s", wrapper->idstr, ginitDict[i].key, err); + taosMemoryFreeClear(err); + } + } + rocksdb_flushoptions_destroy(flushOpt); + } + + for (int i = 0; i < cfLen; i++) { + if (wrapper->pHandle[i] != NULL) { + rocksdb_column_family_handle_destroy(wrapper->pHandle[i]); + } + } + taosMemoryFreeClear(wrapper->pHandle); + for (int i = 0; i < cfLen; i++) { + rocksdb_options_destroy(wrapper->cfOpts[i]); + rocksdb_block_based_options_destroy(((RocksdbCfParam*)wrapper->param)[i].tableOpt); + } + + if (remove) { + streamBackendDelCompare(wrapper->pBackend, wrapper->pComparNode); + } + rocksdb_writeoptions_destroy(wrapper->writeOpts); + wrapper->writeOpts = NULL; + + rocksdb_readoptions_destroy(wrapper->readOpts); + wrapper->readOpts = NULL; + taosMemoryFreeClear(wrapper->cfOpts); + taosMemoryFreeClear(wrapper->param); + + taosThreadRwlockDestroy(&wrapper->rwLock); + wrapper->rocksdb = NULL; + taosReleaseRef(streamBackendId, wrapper->backendId); + + qDebug("end to do-close backendwrapper %p, %s", wrapper, wrapper->idstr); + taosMemoryFree(wrapper); + return; +} SListNode* streamBackendAddCompare(void* backend, void* arg) { SBackendHandle* pHandle = (SBackendHandle*)backend; SListNode* node = NULL; @@ -537,23 +663,6 @@ void destroyFunc(void* arg) { return; } -typedef struct { - const char* key; - int32_t len; - int idx; - BackendCmpFunc cmpFunc; - EncodeFunc enFunc; - DecodeFunc deFunc; - ToStringFunc toStrFunc; - CompareName cmpName; - DestroyFunc detroyFunc; - EncodeValueFunc enValueFunc; - DecodeValueFunc deValueFunc; - -} SCfInit; - -#define GEN_COLUMN_FAMILY_NAME(name, idstr, SUFFIX) sprintf(name, "%s_%s", idstr, (SUFFIX)); - int32_t encodeValueFunc(void* value, int32_t vlen, int64_t ttl, char** dest) { SStreamValue key = {.unixTimestamp = ttl, .len = vlen, .data = (char*)(value)}; int32_t len = 0; @@ -608,22 +717,6 @@ int32_t decodeValueFunc(void* value, int32_t vlen, int64_t* ttl, char** dest) { } return key.len; } -SCfInit ginitDict[] = { - {"default", 7, 0, defaultKeyComp, defaultKeyEncode, defaultKeyDecode, defaultKeyToString, compareDefaultName, - destroyFunc, encodeValueFunc, decodeValueFunc}, - {"state", 5, 1, stateKeyDBComp, stateKeyEncode, stateKeyDecode, stateKeyToString, compareStateName, destroyFunc, - encodeValueFunc, decodeValueFunc}, - {"fill", 4, 2, winKeyDBComp, winKeyEncode, winKeyDecode, winKeyToString, compareWinKeyName, destroyFunc, - encodeValueFunc, decodeValueFunc}, - {"sess", 4, 3, stateSessionKeyDBComp, stateSessionKeyEncode, stateSessionKeyDecode, stateSessionKeyToString, - compareSessionKeyName, destroyFunc, encodeValueFunc, decodeValueFunc}, - {"func", 4, 4, tupleKeyDBComp, tupleKeyEncode, tupleKeyDecode, tupleKeyToString, compareFuncKeyName, destroyFunc, - encodeValueFunc, decodeValueFunc}, - {"parname", 7, 5, parKeyDBComp, parKeyEncode, parKeyDecode, parKeyToString, compareParKeyName, destroyFunc, - encodeValueFunc, decodeValueFunc}, - {"partag", 6, 6, parKeyDBComp, parKeyEncode, parKeyDecode, parKeyToString, comparePartagKeyName, destroyFunc, - encodeValueFunc, decodeValueFunc}, -}; const char* compareDefaultName(void* arg) { (void)arg; @@ -816,23 +909,30 @@ int32_t streamStateOpenBackendCf(void* backend, char* name, char** cfs, int32_t int streamStateOpenBackend(void* backend, SStreamState* pState) { qInfo("start to open state %p on backend %p 0x%" PRIx64 "-%d", pState, backend, pState->streamId, pState->taskId); taosAcquireRef(streamBackendId, pState->streamBackendRid); - SBackendHandle* handle = backend; - - sprintf(pState->pTdbState->idstr, "0x%" PRIx64 "-%d", pState->streamId, pState->taskId); + SBackendHandle* handle = backend; + SBackendWrapper* pBackendWrapper = taosMemoryCalloc(1, sizeof(SBackendWrapper)); taosThreadMutexLock(&handle->cfMutex); + RocksdbCfInst** ppInst = taosHashGet(handle->cfInst, pState->pTdbState->idstr, strlen(pState->pTdbState->idstr) + 1); if (ppInst != NULL && *ppInst != NULL) { RocksdbCfInst* inst = *ppInst; - pState->pTdbState->rocksdb = inst->db; - pState->pTdbState->pHandle = (void**)inst->pHandle; - pState->pTdbState->writeOpts = inst->wOpt; - pState->pTdbState->readOpts = inst->rOpt; - pState->pTdbState->cfOpts = (void**)(inst->cfOpt); - pState->pTdbState->dbOpt = handle->dbOpt; - pState->pTdbState->param = inst->param; - pState->pTdbState->pBackend = handle; - pState->pTdbState->pComparNode = inst->pCompareNode; + pBackendWrapper->rocksdb = inst->db; + pBackendWrapper->pHandle = (void**)inst->pHandle; + pBackendWrapper->writeOpts = inst->wOpt; + pBackendWrapper->readOpts = inst->rOpt; + pBackendWrapper->cfOpts = (void**)(inst->cfOpt); + pBackendWrapper->dbOpt = handle->dbOpt; + pBackendWrapper->param = inst->param; + pBackendWrapper->pBackend = handle; + pBackendWrapper->pComparNode = inst->pCompareNode; taosThreadMutexUnlock(&handle->cfMutex); + pBackendWrapper->backendId = pState->streamBackendRid; + memcpy(pBackendWrapper->idstr, pState->pTdbState->idstr, sizeof(pState->pTdbState->idstr)); + + int64_t id = taosAddRef(streamBackendWrapperId, pBackendWrapper); + pState->pTdbState->backendWrapperId = id; + pState->pTdbState->pBackendWrapper = pBackendWrapper; + qInfo("succ to open state %p on backendWrapper, %p, %s", pState, pBackendWrapper, pBackendWrapper->idstr); return 0; } taosThreadMutexUnlock(&handle->cfMutex); @@ -865,92 +965,43 @@ int streamStateOpenBackend(void* backend, SStreamState* pState) { pCompare[i] = compare; } rocksdb_column_family_handle_t** cfHandle = taosMemoryCalloc(cfLen, sizeof(rocksdb_column_family_handle_t*)); - pState->pTdbState->rocksdb = handle->db; - pState->pTdbState->pHandle = (void**)cfHandle; - pState->pTdbState->writeOpts = rocksdb_writeoptions_create(); - pState->pTdbState->readOpts = rocksdb_readoptions_create(); - pState->pTdbState->cfOpts = (void**)cfOpt; - pState->pTdbState->dbOpt = handle->dbOpt; - pState->pTdbState->param = param; - pState->pTdbState->pBackend = handle; - - taosThreadRwlockInit(&pState->pTdbState->rwLock, NULL); + pBackendWrapper->rocksdb = handle->db; + pBackendWrapper->pHandle = (void**)cfHandle; + pBackendWrapper->writeOpts = rocksdb_writeoptions_create(); + pBackendWrapper->readOpts = rocksdb_readoptions_create(); + pBackendWrapper->cfOpts = (void**)cfOpt; + pBackendWrapper->dbOpt = handle->dbOpt; + pBackendWrapper->param = param; + pBackendWrapper->pBackend = handle; + pBackendWrapper->backendId = pState->streamBackendRid; + taosThreadRwlockInit(&pBackendWrapper->rwLock, NULL); SCfComparator compare = {.comp = pCompare, .numOfComp = cfLen}; - pState->pTdbState->pComparNode = streamBackendAddCompare(handle, &compare); - rocksdb_writeoptions_disable_WAL(pState->pTdbState->writeOpts, 1); - qInfo("succ to open state %p on backend, %p, 0x%" PRIx64 "-%d", pState, handle, pState->streamId, pState->taskId); + pBackendWrapper->pComparNode = streamBackendAddCompare(handle, &compare); + rocksdb_writeoptions_disable_WAL(pBackendWrapper->writeOpts, 1); + memcpy(pBackendWrapper->idstr, pState->pTdbState->idstr, sizeof(pState->pTdbState->idstr)); + + int64_t id = taosAddRef(streamBackendWrapperId, pBackendWrapper); + pState->pTdbState->backendWrapperId = id; + pState->pTdbState->pBackendWrapper = pBackendWrapper; + qInfo("succ to open state %p on backendWrapper, %p, %s", pState, pBackendWrapper, pBackendWrapper->idstr); return 0; } void streamStateCloseBackend(SStreamState* pState, bool remove) { - SBackendHandle* pHandle = pState->pTdbState->pBackend; + SBackendWrapper* wrapper = pState->pTdbState->pBackendWrapper; + SBackendHandle* pHandle = wrapper->pBackend; taosThreadMutexLock(&pHandle->cfMutex); - RocksdbCfInst** ppInst = taosHashGet(pHandle->cfInst, pState->pTdbState->idstr, strlen(pState->pTdbState->idstr) + 1); + RocksdbCfInst** ppInst = taosHashGet(pHandle->cfInst, wrapper->idstr, strlen(pState->pTdbState->idstr) + 1); if (ppInst != NULL && *ppInst != NULL) { RocksdbCfInst* inst = *ppInst; taosMemoryFree(inst); taosHashRemove(pHandle->cfInst, pState->pTdbState->idstr, strlen(pState->pTdbState->idstr) + 1); } taosThreadMutexUnlock(&pHandle->cfMutex); - char* status[] = {"close", "drop"}; - qInfo("start to close %s state %p on backend %p 0x%" PRIx64 "-%d", status[remove == false ? 0 : 1], pState, pHandle, - pState->streamId, pState->taskId); - if (pState->pTdbState->rocksdb == NULL) { - return; - } - - int cfLen = sizeof(ginitDict) / sizeof(ginitDict[0]); - - char* err = NULL; - if (remove) { - for (int i = 0; i < cfLen; i++) { - if (pState->pTdbState->pHandle[i] != NULL) - rocksdb_drop_column_family(pState->pTdbState->rocksdb, - ((rocksdb_column_family_handle_t**)pState->pTdbState->pHandle)[i], &err); - if (err != NULL) { - qError("failed to create cf:%s_%s, reason:%s", pState->pTdbState->idstr, ginitDict[i].key, err); - taosMemoryFreeClear(err); - } - } - } else { - rocksdb_flushoptions_t* flushOpt = rocksdb_flushoptions_create(); - for (int i = 0; i < cfLen; i++) { - if (pState->pTdbState->pHandle[i] != NULL) - rocksdb_flush_cf(pState->pTdbState->rocksdb, flushOpt, pState->pTdbState->pHandle[i], &err); - if (err != NULL) { - qError("failed to create cf:%s_%s, reason:%s", pState->pTdbState->idstr, ginitDict[i].key, err); - taosMemoryFreeClear(err); - } - } - rocksdb_flushoptions_destroy(flushOpt); - } - - for (int i = 0; i < cfLen; i++) { - if (pState->pTdbState->pHandle[i] != NULL) { - rocksdb_column_family_handle_destroy(pState->pTdbState->pHandle[i]); - } - } - taosMemoryFreeClear(pState->pTdbState->pHandle); - for (int i = 0; i < cfLen; i++) { - rocksdb_options_destroy(pState->pTdbState->cfOpts[i]); - rocksdb_block_based_options_destroy(((RocksdbCfParam*)pState->pTdbState->param)[i].tableOpt); - } - - if (remove) { - streamBackendDelCompare(pState->pTdbState->pBackend, pState->pTdbState->pComparNode); - } - rocksdb_writeoptions_destroy(pState->pTdbState->writeOpts); - pState->pTdbState->writeOpts = NULL; - - rocksdb_readoptions_destroy(pState->pTdbState->readOpts); - pState->pTdbState->readOpts = NULL; - taosMemoryFreeClear(pState->pTdbState->cfOpts); - taosMemoryFreeClear(pState->pTdbState->param); - - taosThreadRwlockDestroy(&pState->pTdbState->rwLock); - pState->pTdbState->rocksdb = NULL; - taosReleaseRef(streamBackendId, pState->streamBackendRid); + qInfo("start to close %s state %p on backendWrapper %p %s", status[remove == false ? 0 : 1], pState, wrapper, + wrapper->idstr); + taosReleaseRef(streamBackendWrapperId, pState->pTdbState->backendWrapperId); } void streamStateDestroyCompar(void* arg) { SCfComparator* comp = (SCfComparator*)arg; @@ -969,27 +1020,27 @@ int streamStateGetCfIdx(SStreamState* pState, const char* funcName) { break; } } + SBackendWrapper* wrapper = pState->pTdbState->pBackendWrapper; if (pState != NULL && idx != -1) { rocksdb_column_family_handle_t* cf = NULL; - taosThreadRwlockRdlock(&pState->pTdbState->rwLock); - cf = pState->pTdbState->pHandle[idx]; - taosThreadRwlockUnlock(&pState->pTdbState->rwLock); + taosThreadRwlockRdlock(&wrapper->rwLock); + cf = wrapper->pHandle[idx]; + taosThreadRwlockUnlock(&wrapper->rwLock); if (cf == NULL) { char buf[128] = {0}; - GEN_COLUMN_FAMILY_NAME(buf, pState->pTdbState->idstr, ginitDict[idx].key); + GEN_COLUMN_FAMILY_NAME(buf, wrapper->idstr, ginitDict[idx].key); char* err = NULL; - taosThreadRwlockWrlock(&pState->pTdbState->rwLock); - cf = rocksdb_create_column_family(pState->pTdbState->rocksdb, pState->pTdbState->cfOpts[idx], buf, &err); + taosThreadRwlockWrlock(&wrapper->rwLock); + cf = rocksdb_create_column_family(wrapper->rocksdb, wrapper->cfOpts[idx], buf, &err); if (err != NULL) { idx = -1; - qError("failed to to open cf, %p 0x%" PRIx64 "-%d_%s, reason:%s", pState, pState->streamId, pState->taskId, - funcName, err); + qError("failed to to open cf, %p %s_%s, reason:%s", pState, wrapper->idstr, funcName, err); taosMemoryFree(err); } else { - pState->pTdbState->pHandle[idx] = cf; + wrapper->pHandle[idx] = cf; } - taosThreadRwlockUnlock(&pState->pTdbState->rwLock); + taosThreadRwlockUnlock(&wrapper->rwLock); } } @@ -1009,8 +1060,9 @@ rocksdb_iterator_t* streamStateIterCreate(SStreamState* pState, const char* cfNa rocksdb_readoptions_t** readOpt) { int idx = streamStateGetCfIdx(pState, cfName); + SBackendWrapper* wrapper = pState->pTdbState->pBackendWrapper; if (snapshot != NULL) { - *snapshot = (rocksdb_snapshot_t*)rocksdb_create_snapshot(pState->pTdbState->rocksdb); + *snapshot = (rocksdb_snapshot_t*)rocksdb_create_snapshot(wrapper->rocksdb); } rocksdb_readoptions_t* rOpt = rocksdb_readoptions_create(); *readOpt = rOpt; @@ -1018,8 +1070,7 @@ rocksdb_iterator_t* streamStateIterCreate(SStreamState* pState, const char* cfNa rocksdb_readoptions_set_snapshot(rOpt, *snapshot); rocksdb_readoptions_set_fill_cache(rOpt, 0); - return rocksdb_create_iterator_cf(pState->pTdbState->rocksdb, rOpt, - ((rocksdb_column_family_handle_t**)pState->pTdbState->pHandle)[idx]); + return rocksdb_create_iterator_cf(wrapper->rocksdb, rOpt, ((rocksdb_column_family_handle_t**)wrapper->pHandle)[idx]); } #define STREAM_STATE_PUT_ROCKSDB(pState, funcname, key, value, vLen) \ @@ -1033,15 +1084,15 @@ rocksdb_iterator_t* streamStateIterCreate(SStreamState* pState, const char* cfNa code = -1; \ break; \ } \ - char toString[128] = {0}; \ + SBackendWrapper* wrapper = pState->pTdbState->pBackendWrapper; \ + char toString[128] = {0}; \ if (qDebugFlag & DEBUG_TRACE) ginitDict[i].toStrFunc((void*)key, toString); \ int32_t klen = ginitDict[i].enFunc((void*)key, buf); \ - rocksdb_column_family_handle_t* pHandle = \ - ((rocksdb_column_family_handle_t**)pState->pTdbState->pHandle)[ginitDict[i].idx]; \ - rocksdb_t* db = pState->pTdbState->rocksdb; \ - rocksdb_writeoptions_t* opts = pState->pTdbState->writeOpts; \ - char* ttlV = NULL; \ - int32_t ttlVLen = ginitDict[i].enValueFunc((char*)value, vLen, 0, &ttlV); \ + rocksdb_column_family_handle_t* pHandle = ((rocksdb_column_family_handle_t**)wrapper->pHandle)[ginitDict[i].idx]; \ + rocksdb_t* db = wrapper->rocksdb; \ + rocksdb_writeoptions_t* opts = wrapper->writeOpts; \ + char* ttlV = NULL; \ + int32_t ttlVLen = ginitDict[i].enValueFunc((char*)value, vLen, 0, &ttlV); \ rocksdb_put_cf(db, opts, pHandle, (const char*)buf, klen, (const char*)ttlV, (size_t)ttlVLen, &err); \ if (err != NULL) { \ taosMemoryFree(err); \ @@ -1053,81 +1104,76 @@ rocksdb_iterator_t* streamStateIterCreate(SStreamState* pState, const char* cfNa taosMemoryFree(ttlV); \ } while (0); -#define STREAM_STATE_GET_ROCKSDB(pState, funcname, key, pVal, vLen) \ - do { \ - code = 0; \ - char buf[128] = {0}; \ - char* err = NULL; \ - int i = streamStateGetCfIdx(pState, funcname); \ - if (i < 0) { \ - qWarn("streamState failed to get cf name: %s", funcname); \ - code = -1; \ - break; \ - } \ - char toString[128] = {0}; \ - if (qDebugFlag & DEBUG_TRACE) ginitDict[i].toStrFunc((void*)key, toString); \ - int32_t klen = ginitDict[i].enFunc((void*)key, buf); \ - rocksdb_column_family_handle_t* pHandle = \ - ((rocksdb_column_family_handle_t**)pState->pTdbState->pHandle)[ginitDict[i].idx]; \ - rocksdb_t* db = pState->pTdbState->rocksdb; \ - rocksdb_readoptions_t* opts = pState->pTdbState->readOpts; \ - size_t len = 0; \ - char* val = rocksdb_get_cf(db, opts, pHandle, (const char*)buf, klen, (size_t*)&len, &err); \ - if (val == NULL || len == 0) { \ - if (err == NULL) { \ - qTrace("streamState str: %s failed to read from %s_%s, err: not exist", toString, pState->pTdbState->idstr, \ - funcname); \ - } else { \ - qError("streamState str: %s failed to read from %s_%s, err: %s", toString, pState->pTdbState->idstr, funcname, \ - err); \ - taosMemoryFreeClear(err); \ - } \ - code = -1; \ - } else { \ - char* p = NULL; \ - int32_t tlen = ginitDict[i].deValueFunc(val, len, NULL, (char**)pVal); \ - if (tlen <= 0) { \ - qError("streamState str: %s failed to read from %s_%s, err: already ttl ", toString, pState->pTdbState->idstr, \ - funcname); \ - code = -1; \ - } else { \ - qTrace("streamState str: %s succ to read from %s_%s, valLen:%d", toString, pState->pTdbState->idstr, funcname, \ - tlen); \ - } \ - taosMemoryFree(val); \ - if (vLen != NULL) *vLen = tlen; \ - } \ - if (code == 0) \ - qDebug("streamState str: %s succ to read from %s_%s", toString, pState->pTdbState->idstr, funcname); \ +#define STREAM_STATE_GET_ROCKSDB(pState, funcname, key, pVal, vLen) \ + do { \ + code = 0; \ + char buf[128] = {0}; \ + char* err = NULL; \ + int i = streamStateGetCfIdx(pState, funcname); \ + if (i < 0) { \ + qWarn("streamState failed to get cf name: %s", funcname); \ + code = -1; \ + break; \ + } \ + SBackendWrapper* wrapper = pState->pTdbState->pBackendWrapper; \ + char toString[128] = {0}; \ + if (qDebugFlag & DEBUG_TRACE) ginitDict[i].toStrFunc((void*)key, toString); \ + int32_t klen = ginitDict[i].enFunc((void*)key, buf); \ + rocksdb_column_family_handle_t* pHandle = ((rocksdb_column_family_handle_t**)wrapper->pHandle)[ginitDict[i].idx]; \ + rocksdb_t* db = wrapper->rocksdb; \ + rocksdb_readoptions_t* opts = wrapper->readOpts; \ + size_t len = 0; \ + char* val = rocksdb_get_cf(db, opts, pHandle, (const char*)buf, klen, (size_t*)&len, &err); \ + if (val == NULL || len == 0) { \ + if (err == NULL) { \ + qTrace("streamState str: %s failed to read from %s_%s, err: not exist", toString, wrapper->idstr, funcname); \ + } else { \ + qError("streamState str: %s failed to read from %s_%s, err: %s", toString, wrapper->idstr, funcname, err); \ + taosMemoryFreeClear(err); \ + } \ + code = -1; \ + } else { \ + char* p = NULL; \ + int32_t tlen = ginitDict[i].deValueFunc(val, len, NULL, (char**)pVal); \ + if (tlen <= 0) { \ + qError("streamState str: %s failed to read from %s_%s, err: already ttl ", toString, wrapper->idstr, \ + funcname); \ + code = -1; \ + } else { \ + qTrace("streamState str: %s succ to read from %s_%s, valLen:%d", toString, wrapper->idstr, funcname, tlen); \ + } \ + taosMemoryFree(val); \ + if (vLen != NULL) *vLen = tlen; \ + } \ + if (code == 0) qDebug("streamState str: %s succ to read from %s_%s", toString, wrapper->idstr, funcname); \ } while (0); -#define STREAM_STATE_DEL_ROCKSDB(pState, funcname, key) \ - do { \ - code = 0; \ - char buf[128] = {0}; \ - char* err = NULL; \ - int i = streamStateGetCfIdx(pState, funcname); \ - if (i < 0) { \ - qWarn("streamState failed to get cf name: %s_%s", pState->pTdbState->idstr, funcname); \ - code = -1; \ - break; \ - } \ - char toString[128] = {0}; \ - if (qDebugFlag & DEBUG_TRACE) ginitDict[i].toStrFunc((void*)key, toString); \ - int32_t klen = ginitDict[i].enFunc((void*)key, buf); \ - rocksdb_column_family_handle_t* pHandle = \ - ((rocksdb_column_family_handle_t**)pState->pTdbState->pHandle)[ginitDict[i].idx]; \ - rocksdb_t* db = pState->pTdbState->rocksdb; \ - rocksdb_writeoptions_t* opts = pState->pTdbState->writeOpts; \ - rocksdb_delete_cf(db, opts, pHandle, (const char*)buf, klen, &err); \ - if (err != NULL) { \ - qError("streamState str: %s failed to del from %s_%s, err: %s", toString, pState->pTdbState->idstr, funcname, \ - err); \ - taosMemoryFree(err); \ - code = -1; \ - } else { \ - qTrace("streamState str: %s succ to del from %s_%s", toString, pState->pTdbState->idstr, funcname); \ - } \ +#define STREAM_STATE_DEL_ROCKSDB(pState, funcname, key) \ + do { \ + code = 0; \ + char buf[128] = {0}; \ + char* err = NULL; \ + int i = streamStateGetCfIdx(pState, funcname); \ + if (i < 0) { \ + qWarn("streamState failed to get cf name: %s_%s", pState->pTdbState->idstr, funcname); \ + code = -1; \ + break; \ + } \ + SBackendWrapper* wrapper = pState->pTdbState->pBackendWrapper; \ + char toString[128] = {0}; \ + if (qDebugFlag & DEBUG_TRACE) ginitDict[i].toStrFunc((void*)key, toString); \ + int32_t klen = ginitDict[i].enFunc((void*)key, buf); \ + rocksdb_column_family_handle_t* pHandle = ((rocksdb_column_family_handle_t**)wrapper->pHandle)[ginitDict[i].idx]; \ + rocksdb_t* db = wrapper->rocksdb; \ + rocksdb_writeoptions_t* opts = wrapper->writeOpts; \ + rocksdb_delete_cf(db, opts, pHandle, (const char*)buf, klen, &err); \ + if (err != NULL) { \ + qError("streamState str: %s failed to del from %s_%s, err: %s", toString, wrapper->idstr, funcname, err); \ + taosMemoryFree(err); \ + code = -1; \ + } else { \ + qTrace("streamState str: %s succ to del from %s_%s", toString, wrapper->idstr, funcname); \ + } \ } while (0); // state cf @@ -1153,18 +1199,19 @@ int32_t streamStateDel_rocksdb(SStreamState* pState, const SWinKey* key) { int32_t streamStateClear_rocksdb(SStreamState* pState) { qDebug("streamStateClear_rocksdb"); - char sKeyStr[128] = {0}; - char eKeyStr[128] = {0}; - SStateKey sKey = {.key = {.ts = 0, .groupId = 0}, .opNum = pState->number}; - SStateKey eKey = {.key = {.ts = INT64_MAX, .groupId = UINT64_MAX}, .opNum = pState->number}; + SBackendWrapper* wrapper = pState->pTdbState->pBackendWrapper; + char sKeyStr[128] = {0}; + char eKeyStr[128] = {0}; + SStateKey sKey = {.key = {.ts = 0, .groupId = 0}, .opNum = pState->number}; + SStateKey eKey = {.key = {.ts = INT64_MAX, .groupId = UINT64_MAX}, .opNum = pState->number}; int sLen = stateKeyEncode(&sKey, sKeyStr); int eLen = stateKeyEncode(&eKey, eKeyStr); - if (pState->pTdbState->pHandle[1] != NULL) { + if (wrapper->pHandle[1] != NULL) { char* err = NULL; - rocksdb_delete_range_cf(pState->pTdbState->rocksdb, pState->pTdbState->writeOpts, pState->pTdbState->pHandle[1], - sKeyStr, sLen, eKeyStr, eLen, &err); + rocksdb_delete_range_cf(wrapper->rocksdb, wrapper->writeOpts, wrapper->pHandle[1], sKeyStr, sLen, eKeyStr, eLen, + &err); if (err != NULL) { char toStringStart[128] = {0}; char toStringEnd[128] = {0}; @@ -1174,7 +1221,7 @@ int32_t streamStateClear_rocksdb(SStreamState* pState) { qWarn("failed to delete range cf(state) start: %s, end:%s, reason:%s", toStringStart, toStringEnd, err); taosMemoryFree(err); } else { - rocksdb_compact_range_cf(pState->pTdbState->rocksdb, pState->pTdbState->pHandle[1], sKeyStr, sLen, eKeyStr, eLen); + rocksdb_compact_range_cf(wrapper->rocksdb, wrapper->pHandle[1], sKeyStr, sLen, eKeyStr, eLen); } } @@ -1268,8 +1315,9 @@ SStreamStateCur* streamStateSeekKeyNext_rocksdb(SStreamState* pState, const SWin if (pCur == NULL) { return NULL; } + SBackendWrapper* wrapper = pState->pTdbState->pBackendWrapper; pCur->number = pState->number; - pCur->db = pState->pTdbState->rocksdb; + pCur->db = wrapper->rocksdb; pCur->iter = streamStateIterCreate(pState, "state", (rocksdb_snapshot_t**)&pCur->snapshot, (rocksdb_readoptions_t**)&pCur->readOpt); @@ -1302,15 +1350,16 @@ SStreamStateCur* streamStateSeekKeyNext_rocksdb(SStreamState* pState, const SWin SStreamStateCur* streamStateSeekToLast_rocksdb(SStreamState* pState, const SWinKey* key) { qDebug("streamStateGetCur_rocksdb"); - int32_t code = 0; + int32_t code = 0; + SBackendWrapper* wrapper = pState->pTdbState->pBackendWrapper; + const SStateKey maxStateKey = {.key = {.groupId = UINT64_MAX, .ts = INT64_MAX}, .opNum = INT64_MAX}; STREAM_STATE_PUT_ROCKSDB(pState, "state", &maxStateKey, "", 0); - char buf[128] = {0}; - int32_t klen = stateKeyEncode((void*)&maxStateKey, buf); - + char buf[128] = {0}; + int32_t klen = stateKeyEncode((void*)&maxStateKey, buf); SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); if (pCur == NULL) return NULL; - pCur->db = pState->pTdbState->rocksdb; + pCur->db = wrapper->rocksdb; pCur->iter = streamStateIterCreate(pState, "state", (rocksdb_snapshot_t**)&pCur->snapshot, (rocksdb_readoptions_t**)&pCur->readOpt); rocksdb_iter_seek(pCur->iter, buf, (size_t)klen); @@ -1330,10 +1379,11 @@ SStreamStateCur* streamStateSeekToLast_rocksdb(SStreamState* pState, const SWinK SStreamStateCur* streamStateGetCur_rocksdb(SStreamState* pState, const SWinKey* key) { qDebug("streamStateGetCur_rocksdb"); + SBackendWrapper* wrapper = pState->pTdbState->pBackendWrapper; SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); if (pCur == NULL) return NULL; - pCur->db = pState->pTdbState->rocksdb; + pCur->db = wrapper->rocksdb; pCur->iter = streamStateIterCreate(pState, "state", (rocksdb_snapshot_t**)&pCur->snapshot, (rocksdb_readoptions_t**)&pCur->readOpt); @@ -1421,12 +1471,14 @@ int32_t streamStateSessionDel_rocksdb(SStreamState* pState, const SSessionKey* k } SStreamStateCur* streamStateSessionSeekKeyCurrentPrev_rocksdb(SStreamState* pState, const SSessionKey* key) { qDebug("streamStateSessionSeekKeyCurrentPrev_rocksdb"); + + SBackendWrapper* wrapper = pState->pTdbState->pBackendWrapper; SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); if (pCur == NULL) { return NULL; } pCur->number = pState->number; - pCur->db = pState->pTdbState->rocksdb; + pCur->db = wrapper->rocksdb; pCur->iter = streamStateIterCreate(pState, "sess", (rocksdb_snapshot_t**)&pCur->snapshot, (rocksdb_readoptions_t**)&pCur->readOpt); @@ -1462,11 +1514,12 @@ SStreamStateCur* streamStateSessionSeekKeyCurrentPrev_rocksdb(SStreamState* pSta } SStreamStateCur* streamStateSessionSeekKeyCurrentNext_rocksdb(SStreamState* pState, SSessionKey* key) { qDebug("streamStateSessionSeekKeyCurrentNext_rocksdb"); + SBackendWrapper* wrapper = pState->pTdbState->pBackendWrapper; SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); if (pCur == NULL) { return NULL; } - pCur->db = pState->pTdbState->rocksdb; + pCur->db = wrapper->rocksdb; pCur->iter = streamStateIterCreate(pState, "sess", (rocksdb_snapshot_t**)&pCur->snapshot, (rocksdb_readoptions_t**)&pCur->readOpt); pCur->number = pState->number; @@ -1499,11 +1552,12 @@ SStreamStateCur* streamStateSessionSeekKeyCurrentNext_rocksdb(SStreamState* pSta SStreamStateCur* streamStateSessionSeekKeyNext_rocksdb(SStreamState* pState, const SSessionKey* key) { qDebug("streamStateSessionSeekKeyNext_rocksdb"); + SBackendWrapper* wrapper = pState->pTdbState->pBackendWrapper; SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); if (pCur == NULL) { return NULL; } - pCur->db = pState->pTdbState->rocksdb; + pCur->db = wrapper->rocksdb; pCur->iter = streamStateIterCreate(pState, "sess", (rocksdb_snapshot_t**)&pCur->snapshot, (rocksdb_readoptions_t**)&pCur->readOpt); pCur->number = pState->number; @@ -1593,10 +1647,11 @@ int32_t streamStateFillDel_rocksdb(SStreamState* pState, const SWinKey* key) { SStreamStateCur* streamStateFillGetCur_rocksdb(SStreamState* pState, const SWinKey* key) { qDebug("streamStateFillGetCur_rocksdb"); SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); + SBackendWrapper* wrapper = pState->pTdbState->pBackendWrapper; if (pCur == NULL) return NULL; - pCur->db = pState->pTdbState->rocksdb; + pCur->db = wrapper->rocksdb; pCur->iter = streamStateIterCreate(pState, "fill", (rocksdb_snapshot_t**)&pCur->snapshot, (rocksdb_readoptions_t**)&pCur->readOpt); @@ -1651,12 +1706,13 @@ int32_t streamStateFillGetKVByCur_rocksdb(SStreamStateCur* pCur, SWinKey* pKey, SStreamStateCur* streamStateFillSeekKeyNext_rocksdb(SStreamState* pState, const SWinKey* key) { qDebug("streamStateFillSeekKeyNext_rocksdb"); + SBackendWrapper* wrapper = pState->pTdbState->pBackendWrapper; SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); if (!pCur) { return NULL; } - pCur->db = pState->pTdbState->rocksdb; + pCur->db = wrapper->rocksdb; pCur->iter = streamStateIterCreate(pState, "fill", (rocksdb_snapshot_t**)&pCur->snapshot, (rocksdb_readoptions_t**)&pCur->readOpt); @@ -1687,12 +1743,13 @@ SStreamStateCur* streamStateFillSeekKeyNext_rocksdb(SStreamState* pState, const } SStreamStateCur* streamStateFillSeekKeyPrev_rocksdb(SStreamState* pState, const SWinKey* key) { qDebug("streamStateFillSeekKeyPrev_rocksdb"); + SBackendWrapper* wrapper = pState->pTdbState->pBackendWrapper; SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); if (pCur == NULL) { return NULL; } - pCur->db = pState->pTdbState->rocksdb; + pCur->db = wrapper->rocksdb; pCur->iter = streamStateIterCreate(pState, "fill", (rocksdb_snapshot_t**)&pCur->snapshot, (rocksdb_readoptions_t**)&pCur->readOpt); @@ -1723,12 +1780,13 @@ SStreamStateCur* streamStateFillSeekKeyPrev_rocksdb(SStreamState* pState, const } int32_t streamStateSessionGetKeyByRange_rocksdb(SStreamState* pState, const SSessionKey* key, SSessionKey* curKey) { qDebug("streamStateSessionGetKeyByRange_rocksdb"); + SBackendWrapper* wrapper = pState->pTdbState->pBackendWrapper; SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); if (pCur == NULL) { return -1; } pCur->number = pState->number; - pCur->db = pState->pTdbState->rocksdb; + pCur->db = wrapper->rocksdb; pCur->iter = streamStateIterCreate(pState, "sess", (rocksdb_snapshot_t**)&pCur->snapshot, (rocksdb_readoptions_t**)&pCur->readOpt); @@ -1959,6 +2017,7 @@ int32_t streamDefaultIterGet_rocksdb(SStreamState* pState, const void* start, co int code = 0; char* err = NULL; + SBackendWrapper* wrapper = pState->pTdbState->pBackendWrapper; rocksdb_snapshot_t* snapshot = NULL; rocksdb_readoptions_t* readopts = NULL; rocksdb_iterator_t* pIter = streamStateIterCreate(pState, "default", &snapshot, &readopts); @@ -1991,15 +2050,16 @@ int32_t streamDefaultIterGet_rocksdb(SStreamState* pState, const void* start, co } rocksdb_iter_next(pIter); } - rocksdb_release_snapshot(pState->pTdbState->rocksdb, snapshot); + rocksdb_release_snapshot(wrapper->rocksdb, snapshot); rocksdb_readoptions_destroy(readopts); rocksdb_iter_destroy(pIter); return code; } void* streamDefaultIterCreate_rocksdb(SStreamState* pState) { SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); + SBackendWrapper* wrapper = pState->pTdbState->pBackendWrapper; - pCur->db = pState->pTdbState->rocksdb; + pCur->db = wrapper->rocksdb; pCur->iter = streamStateIterCreate(pState, "default", (rocksdb_snapshot_t**)&pCur->snapshot, (rocksdb_readoptions_t**)&pCur->readOpt); return pCur; @@ -2046,7 +2106,8 @@ void streamStateClearBatch(void* pBatch) { rocksdb_writebatch_clear((rocksdb_ void streamStateDestroyBatch(void* pBatch) { rocksdb_writebatch_destroy((rocksdb_writebatch_t*)pBatch); } int32_t streamStatePutBatch(SStreamState* pState, const char* cfName, rocksdb_writebatch_t* pBatch, void* key, void* val, int32_t vlen, int64_t ttl) { - int i = streamStateGetCfIdx(pState, cfName); + SBackendWrapper* wrapper = pState->pTdbState->pBackendWrapper; + int i = streamStateGetCfIdx(pState, cfName); if (i < 0) { qError("streamState failed to put to cf name:%s", cfName); @@ -2057,7 +2118,7 @@ int32_t streamStatePutBatch(SStreamState* pState, const char* cfName, rocksdb_wr char* ttlV = NULL; int32_t ttlVLen = ginitDict[i].enValueFunc(val, vlen, ttl, &ttlV); - rocksdb_column_family_handle_t* pCf = pState->pTdbState->pHandle[ginitDict[i].idx]; + rocksdb_column_family_handle_t* pCf = wrapper->pHandle[ginitDict[i].idx]; rocksdb_writebatch_put_cf((rocksdb_writebatch_t*)pBatch, pCf, buf, (size_t)klen, ttlV, (size_t)ttlVLen); taosMemoryFree(ttlV); return 0; @@ -2069,7 +2130,9 @@ int32_t streamStatePutBatchOptimize(SStreamState* pState, int32_t cfIdx, rocksdb char* ttlV = tmpBuf; int32_t ttlVLen = ginitDict[cfIdx].enValueFunc(val, vlen, ttl, &ttlV); - rocksdb_column_family_handle_t* pCf = pState->pTdbState->pHandle[ginitDict[cfIdx].idx]; + SBackendWrapper* wrapper = pState->pTdbState->pBackendWrapper; + + rocksdb_column_family_handle_t* pCf = wrapper->pHandle[ginitDict[cfIdx].idx]; rocksdb_writebatch_put_cf((rocksdb_writebatch_t*)pBatch, pCf, buf, (size_t)klen, ttlV, (size_t)ttlVLen); if (tmpBuf == NULL) { @@ -2078,8 +2141,9 @@ int32_t streamStatePutBatchOptimize(SStreamState* pState, int32_t cfIdx, rocksdb return 0; } int32_t streamStatePutBatch_rocksdb(SStreamState* pState, void* pBatch) { - char* err = NULL; - rocksdb_write(pState->pTdbState->rocksdb, pState->pTdbState->writeOpts, (rocksdb_writebatch_t*)pBatch, &err); + char* err = NULL; + SBackendWrapper* wrapper = pState->pTdbState->pBackendWrapper; + rocksdb_write(wrapper->rocksdb, wrapper->writeOpts, (rocksdb_writebatch_t*)pBatch, &err); if (err != NULL) { qError("streamState failed to write batch, err:%s", err); taosMemoryFree(err); @@ -2087,3 +2151,13 @@ int32_t streamStatePutBatch_rocksdb(SStreamState* pState, void* pBatch) { } return 0; } + +uint32_t nextPow2(uint32_t x) { + x = x - 1; + x = x | (x >> 1); + x = x | (x >> 2); + x = x | (x >> 4); + x = x | (x >> 8); + x = x | (x >> 16); + return x + 1; +} \ No newline at end of file diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index b7bc35d13d..5a5180b8c5 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -21,10 +21,17 @@ static TdThreadOnce streamMetaModuleInit = PTHREAD_ONCE_INIT; int32_t streamBackendId = 0; -static void streamMetaEnvInit() { streamBackendId = taosOpenRef(20, streamBackendCleanup); } +int32_t streamBackendWrapperId = 0; +static void streamMetaEnvInit() { + streamBackendId = taosOpenRef(64, streamBackendCleanup); + streamBackendWrapperId = taosOpenRef(64, streamBackendHandleCleanup); +} void streamMetaInit() { taosThreadOnce(&streamMetaModuleInit, streamMetaEnvInit); } -void streamMetaCleanup() { taosCloseRef(streamBackendId); } +void streamMetaCleanup() { + taosCloseRef(streamBackendId); + taosCloseRef(streamBackendWrapperId); +} SStreamMeta* streamMetaOpen(const char* path, void* ahandle, FTaskExpand expandFunc, int32_t vgId) { int32_t code = -1; @@ -90,10 +97,14 @@ SStreamMeta* streamMetaOpen(const char* path, void* ahandle, FTaskExpand expandF pMeta->streamBackend = streamBackendInit(streamPath); pMeta->streamBackendRid = taosAddRef(streamBackendId, pMeta->streamBackend); + pMeta->pTaskBackendUnique = + taosHashInit(64, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY), false, HASH_ENTRY_LOCK); taosMemoryFree(streamPath); taosInitRWLatch(&pMeta->lock); + taosThreadMutexInit(&pMeta->backendMutex, NULL); + return pMeta; _err: @@ -136,6 +147,8 @@ void streamMetaClose(SStreamMeta* pMeta) { taosRemoveRef(streamBackendId, pMeta->streamBackendRid); pMeta->pTaskList = taosArrayDestroy(pMeta->pTaskList); taosMemoryFree(pMeta->path); + taosThreadMutexDestroy(&pMeta->backendMutex); + taosHashCleanup(pMeta->pTaskBackendUnique); taosMemoryFree(pMeta); } diff --git a/source/libs/stream/src/streamState.c b/source/libs/stream/src/streamState.c index aeb25c2368..448d693b0a 100644 --- a/source/libs/stream/src/streamState.c +++ b/source/libs/stream/src/streamState.c @@ -116,16 +116,33 @@ SStreamState* streamStateOpen(char* path, void* pTask, bool specPath, int32_t sz pState->taskId = pStreamTask->id.taskId; pState->streamId = pStreamTask->id.streamId; + sprintf(pState->pTdbState->idstr, "0x%" PRIx64 "-%d", pState->streamId, pState->taskId); #ifdef USE_ROCKSDB SStreamMeta* pMeta = pStreamTask->pMeta; pState->streamBackendRid = pMeta->streamBackendRid; - int code = streamStateOpenBackend(pMeta->streamBackend, pState); - if (code == -1) { - taosReleaseRef(streamBackendId, pMeta->streamBackendRid); - taosMemoryFree(pState); - pState = NULL; + // taosWLockLatch(&pMeta->lock); + taosThreadMutexLock(&pMeta->backendMutex); + void* uniqueId = + taosHashGet(pMeta->pTaskBackendUnique, pState->pTdbState->idstr, strlen(pState->pTdbState->idstr) + 1); + if (uniqueId == NULL) { + int code = streamStateOpenBackend(pMeta->streamBackend, pState); + if (code == -1) { + taosReleaseRef(streamBackendId, pState->streamBackendRid); + taosThreadMutexUnlock(&pMeta->backendMutex); + taosMemoryFree(pState); + return NULL; + } + taosHashPut(pMeta->pTaskBackendUnique, pState->pTdbState->idstr, strlen(pState->pTdbState->idstr) + 1, + &pState->pTdbState->backendWrapperId, sizeof(pState->pTdbState->backendWrapperId)); + } else { + int64_t id = *(int64_t*)uniqueId; + pState->pTdbState->backendWrapperId = id; + pState->pTdbState->pBackendWrapper = taosAcquireRef(streamBackendWrapperId, id); + + taosAcquireRef(streamBackendId, pState->streamBackendRid); } + taosThreadMutexUnlock(&pMeta->backendMutex); pState->pTdbState->pOwner = pTask; pState->pFileState = NULL; @@ -385,8 +402,8 @@ int32_t streamStateClear(SStreamState* pState) { streamStatePut(pState, &key, NULL, 0); while (1) { SStreamStateCur* pCur = streamStateSeekKeyNext(pState, &key); - SWinKey delKey = {0}; - int32_t code = streamStateGetKVByCur(pCur, &delKey, NULL, 0); + SWinKey delKey = {0}; + int32_t code = streamStateGetKVByCur(pCur, &delKey, NULL, 0); streamStateFreeCur(pCur); if (code == 0) { streamStateDel(pState, &delKey); @@ -498,7 +515,7 @@ int32_t streamStateGetKVByCur(SStreamStateCur* pCur, SWinKey* pKey, const void** return -1; } const SStateKey* pKTmp = NULL; - int32_t kLen; + int32_t kLen; if (tdbTbcGet(pCur->pCur, (const void**)&pKTmp, &kLen, pVal, pVLen) < 0) { return -1; } @@ -518,7 +535,7 @@ int32_t streamStateFillGetKVByCur(SStreamStateCur* pCur, SWinKey* pKey, const vo return -1; } const SWinKey* pKTmp = NULL; - int32_t kLen; + int32_t kLen; if (tdbTbcGet(pCur->pCur, (const void**)&pKTmp, &kLen, pVal, pVLen) < 0) { return -1; } @@ -535,7 +552,7 @@ int32_t streamStateGetGroupKVByCur(SStreamStateCur* pCur, SWinKey* pKey, const v return -1; } uint64_t groupId = pKey->groupId; - int32_t code = streamStateFillGetKVByCur(pCur, pKey, pVal, pVLen); + int32_t code = streamStateFillGetKVByCur(pCur, pKey, pVal, pVLen); if (code == 0) { if (pKey->groupId == groupId) { return 0; @@ -553,7 +570,7 @@ int32_t streamStateGetFirst(SStreamState* pState, SWinKey* key) { SWinKey tmp = {.ts = 0, .groupId = 0}; streamStatePut(pState, &tmp, NULL, 0); SStreamStateCur* pCur = streamStateSeekKeyNext(pState, &tmp); - int32_t code = streamStateGetKVByCur(pCur, key, NULL, 0); + int32_t code = streamStateGetKVByCur(pCur, key, NULL, 0); streamStateFreeCur(pCur); streamStateDel(pState, &tmp); return code; @@ -593,7 +610,7 @@ SStreamStateCur* streamStateSeekKeyNext(SStreamState* pState, const SWinKey* key } SStateKey sKey = {.key = *key, .opNum = pState->number}; - int32_t c = 0; + int32_t c = 0; if (tdbTbcMoveTo(pCur->pCur, &sKey, sizeof(SStateKey), &c) < 0) { streamStateFreeCur(pCur); return NULL; @@ -726,9 +743,9 @@ int32_t streamStateSessionGet(SStreamState* pState, SSessionKey* key, void** pVa #else SStreamStateCur* pCur = streamStateSessionSeekKeyCurrentNext(pState, key); - SSessionKey resKey = *key; - void* tmp = NULL; - int32_t code = streamStateSessionGetKVByCur(pCur, &resKey, &tmp, pVLen); + SSessionKey resKey = *key; + void* tmp = NULL; + int32_t code = streamStateSessionGetKVByCur(pCur, &resKey, &tmp, pVLen); if (code == 0) { if (key->win.skey != resKey.win.skey) { code = -1; @@ -767,7 +784,7 @@ SStreamStateCur* streamStateSessionSeekKeyCurrentPrev(SStreamState* pState, cons } SStateSessionKey sKey = {.key = *key, .opNum = pState->number}; - int32_t c = 0; + int32_t c = 0; if (tdbTbcMoveTo(pCur->pCur, &sKey, sizeof(SStateSessionKey), &c) < 0) { streamStateFreeCur(pCur); return NULL; @@ -798,7 +815,7 @@ SStreamStateCur* streamStateSessionSeekKeyCurrentNext(SStreamState* pState, cons } SStateSessionKey sKey = {.key = *key, .opNum = pState->number}; - int32_t c = 0; + int32_t c = 0; if (tdbTbcMoveTo(pCur->pCur, &sKey, sizeof(SStateSessionKey), &c) < 0) { streamStateFreeCur(pCur); return NULL; @@ -830,7 +847,7 @@ SStreamStateCur* streamStateSessionSeekKeyNext(SStreamState* pState, const SSess } SStateSessionKey sKey = {.key = *key, .opNum = pState->number}; - int32_t c = 0; + int32_t c = 0; if (tdbTbcMoveTo(pCur->pCur, &sKey, sizeof(SStateSessionKey), &c) < 0) { streamStateFreeCur(pCur); return NULL; @@ -854,7 +871,7 @@ int32_t streamStateSessionGetKVByCur(SStreamStateCur* pCur, SSessionKey* pKey, v return -1; } SStateSessionKey* pKTmp = NULL; - int32_t kLen; + int32_t kLen; if (tdbTbcGet(pCur->pCur, (const void**)&pKTmp, &kLen, (const void**)pVal, pVLen) < 0) { return -1; } @@ -873,13 +890,13 @@ int32_t streamStateSessionClear(SStreamState* pState) { #ifdef USE_ROCKSDB return streamStateSessionClear_rocksdb(pState); #else - SSessionKey key = {.win.skey = 0, .win.ekey = 0, .groupId = 0}; + SSessionKey key = {.win.skey = 0, .win.ekey = 0, .groupId = 0}; SStreamStateCur* pCur = streamStateSessionSeekKeyCurrentNext(pState, &key); while (1) { SSessionKey delKey = {0}; - void* buf = NULL; - int32_t size = 0; - int32_t code = streamStateSessionGetKVByCur(pCur, &delKey, &buf, &size); + void* buf = NULL; + int32_t size = 0; + int32_t code = streamStateSessionGetKVByCur(pCur, &delKey, &buf, &size); if (code == 0 && size > 0) { memset(buf, 0, size); streamStateSessionPut(pState, &delKey, buf, size); @@ -908,14 +925,14 @@ int32_t streamStateSessionGetKeyByRange(SStreamState* pState, const SSessionKey* } SStateSessionKey sKey = {.key = *key, .opNum = pState->number}; - int32_t c = 0; + int32_t c = 0; if (tdbTbcMoveTo(pCur->pCur, &sKey, sizeof(SStateSessionKey), &c) < 0) { streamStateFreeCur(pCur); return -1; } SSessionKey resKey = *key; - int32_t code = streamStateSessionGetKVByCur(pCur, &resKey, NULL, 0); + int32_t code = streamStateSessionGetKVByCur(pCur, &resKey, NULL, 0); if (code == 0 && sessionRangeKeyCmpr(key, &resKey) == 0) { *curKey = resKey; streamStateFreeCur(pCur); @@ -951,19 +968,19 @@ int32_t streamStateSessionAddIfNotExist(SStreamState* pState, SSessionKey* key, return streamStateSessionAddIfNotExist_rocksdb(pState, key, gap, pVal, pVLen); #else // todo refactor - int32_t res = 0; + int32_t res = 0; SSessionKey originKey = *key; SSessionKey searchKey = *key; searchKey.win.skey = key->win.skey - gap; searchKey.win.ekey = key->win.ekey + gap; int32_t valSize = *pVLen; - void* tmp = tdbRealloc(NULL, valSize); + void* tmp = tdbRealloc(NULL, valSize); if (!tmp) { return -1; } SStreamStateCur* pCur = streamStateSessionSeekKeyCurrentPrev(pState, key); - int32_t code = streamStateSessionGetKVByCur(pCur, key, pVal, pVLen); + int32_t code = streamStateSessionGetKVByCur(pCur, key, pVal, pVLen); if (code == 0) { if (sessionRangeKeyCmpr(&searchKey, key) == 0) { memcpy(tmp, *pVal, valSize); @@ -1006,16 +1023,16 @@ int32_t streamStateStateAddIfNotExist(SStreamState* pState, SSessionKey* key, ch #ifdef USE_ROCKSDB return streamStateStateAddIfNotExist_rocksdb(pState, key, pKeyData, keyDataLen, fn, pVal, pVLen); #else - int32_t res = 0; + int32_t res = 0; SSessionKey tmpKey = *key; - int32_t valSize = *pVLen; - void* tmp = tdbRealloc(NULL, valSize); + int32_t valSize = *pVLen; + void* tmp = tdbRealloc(NULL, valSize); if (!tmp) { return -1; } SStreamStateCur* pCur = streamStateSessionSeekKeyCurrentPrev(pState, key); - int32_t code = streamStateSessionGetKVByCur(pCur, key, pVal, pVLen); + int32_t code = streamStateSessionGetKVByCur(pCur, key, pVal, pVLen); if (code == 0) { if (key->win.skey <= tmpKey.win.skey && tmpKey.win.ekey <= key->win.ekey) { memcpy(tmp, *pVal, valSize); @@ -1113,9 +1130,7 @@ int32_t streamStateDeleteCheckPoint(SStreamState* pState, TSKEY mark) { #endif } -void streamStateReloadInfo(SStreamState* pState, TSKEY ts) { - streamFileStateReloadInfo(pState->pFileState, ts); -} +void streamStateReloadInfo(SStreamState* pState, TSKEY ts) { streamFileStateReloadInfo(pState->pFileState, ts); } #if 0 char* streamStateSessionDump(SStreamState* pState) { From 8cb8c05428bcb9e60911efe2d1c0aae0485c067d Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Thu, 15 Jun 2023 21:55:46 +0800 Subject: [PATCH 39/83] support fill history --- source/libs/stream/src/streamBackendRocksdb.c | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index e5aa08edaf..4697c5b130 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -254,7 +254,6 @@ void streamBackendCleanup(void* arg) { } void streamBackendHandleCleanup(void* arg) { SBackendWrapper* wrapper = arg; - bool remove = false; qDebug("start to do-close backendwrapper %p, %s", wrapper, wrapper->idstr); if (wrapper->rocksdb == NULL) { @@ -296,7 +295,7 @@ void streamBackendHandleCleanup(void* arg) { rocksdb_block_based_options_destroy(((RocksdbCfParam*)wrapper->param)[i].tableOpt); } - if (remove) { + if (wrapper->remove) { streamBackendDelCompare(wrapper->pBackend, wrapper->pComparNode); } rocksdb_writeoptions_destroy(wrapper->writeOpts); @@ -1001,6 +1000,7 @@ void streamStateCloseBackend(SStreamState* pState, bool remove) { char* status[] = {"close", "drop"}; qInfo("start to close %s state %p on backendWrapper %p %s", status[remove == false ? 0 : 1], pState, wrapper, wrapper->idstr); + wrapper->remove |= remove; // update by other pState taosReleaseRef(streamBackendWrapperId, pState->pTdbState->backendWrapperId); } void streamStateDestroyCompar(void* arg) { From 0ef8afbdaaa74b4692d4ca2115acb00d82071ea0 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Fri, 16 Jun 2023 09:32:38 +0800 Subject: [PATCH 40/83] support fill history --- include/libs/function/function.h | 10 +- source/libs/stream/inc/streamBackendRocksdb.h | 2 +- source/libs/stream/inc/streamInc.h | 2 +- source/libs/stream/src/streamBackendRocksdb.c | 186 +++++++++--------- source/libs/stream/src/streamMeta.c | 6 +- source/libs/stream/src/streamState.c | 6 +- 6 files changed, 106 insertions(+), 106 deletions(-) diff --git a/include/libs/function/function.h b/include/libs/function/function.h index 8b08db8f78..2e3cd670d7 100644 --- a/include/libs/function/function.h +++ b/include/libs/function/function.h @@ -130,7 +130,7 @@ typedef struct SSerializeDataHandle { // incremental state storage -typedef struct SBackendWrapper { +typedef struct SBackendCfWrapper { void *rocksdb; void **pHandle; void *writeOpts; @@ -146,11 +146,11 @@ typedef struct SBackendWrapper { bool remove; int64_t backendId; char idstr[64]; -} SBackendWrapper; +} SBackendCfWrapper; typedef struct STdbState { - SBackendWrapper *pBackendWrapper; - int64_t backendWrapperId; - char idstr[64]; + SBackendCfWrapper *pBackendCfWrapper; + int64_t backendCfWrapperId; + char idstr[64]; struct SStreamTask *pOwner; void *db; diff --git a/source/libs/stream/inc/streamBackendRocksdb.h b/source/libs/stream/inc/streamBackendRocksdb.h index 6195200ba5..b6bc9c888b 100644 --- a/source/libs/stream/inc/streamBackendRocksdb.h +++ b/source/libs/stream/inc/streamBackendRocksdb.h @@ -42,7 +42,7 @@ typedef struct { TdThreadMutex cfMutex; SHashObj* cfInst; int64_t defaultCfInit; -} SBackendHandle; +} SBackendWrapper; void* streamBackendInit(const char* path); void streamBackendCleanup(void* arg); diff --git a/source/libs/stream/inc/streamInc.h b/source/libs/stream/inc/streamInc.h index 19a8044b71..eec37d7dbb 100644 --- a/source/libs/stream/inc/streamInc.h +++ b/source/libs/stream/inc/streamInc.h @@ -55,7 +55,7 @@ int32_t streamDoDispatchScanHistoryFinishMsg(SStreamTask* pTask, const SStreamRe SStreamQueueItem* streamMergeQueueItem(SStreamQueueItem* dst, SStreamQueueItem* pElem); extern int32_t streamBackendId; -extern int32_t streamBackendWrapperId; +extern int32_t streamBackendCfWrapperId; #ifdef __cplusplus } diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index 4697c5b130..884100808f 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -145,7 +145,7 @@ SCfInit ginitDict[] = { void* streamBackendInit(const char* path) { qDebug("start to init stream backend at %s", path); - SBackendHandle* pHandle = taosMemoryCalloc(1, sizeof(SBackendHandle)); + SBackendWrapper* pHandle = taosMemoryCalloc(1, sizeof(SBackendWrapper)); pHandle->list = tdListNew(sizeof(SCfComparator)); taosThreadMutexInit(&pHandle->mutex, NULL); taosThreadMutexInit(&pHandle->cfMutex, NULL); @@ -212,8 +212,8 @@ _EXIT: return NULL; } void streamBackendCleanup(void* arg) { - SBackendHandle* pHandle = (SBackendHandle*)arg; - RocksdbCfInst** pIter = (RocksdbCfInst**)taosHashIterate(pHandle->cfInst, NULL); + SBackendWrapper* pHandle = (SBackendWrapper*)arg; + RocksdbCfInst** pIter = (RocksdbCfInst**)taosHashIterate(pHandle->cfInst, NULL); while (pIter != NULL) { RocksdbCfInst* inst = *pIter; destroyRocksdbCfInst(inst); @@ -253,8 +253,8 @@ void streamBackendCleanup(void* arg) { return; } void streamBackendHandleCleanup(void* arg) { - SBackendWrapper* wrapper = arg; - + SBackendCfWrapper* wrapper = arg; + bool remove = wrapper->remove; qDebug("start to do-close backendwrapper %p, %s", wrapper, wrapper->idstr); if (wrapper->rocksdb == NULL) { return; @@ -263,7 +263,7 @@ void streamBackendHandleCleanup(void* arg) { int cfLen = sizeof(ginitDict) / sizeof(ginitDict[0]); char* err = NULL; - if (wrapper->remove) { + if (remove) { for (int i = 0; i < cfLen; i++) { if (wrapper->pHandle[i] != NULL) rocksdb_drop_column_family(wrapper->rocksdb, ((rocksdb_column_family_handle_t**)wrapper->pHandle)[i], &err); @@ -295,7 +295,7 @@ void streamBackendHandleCleanup(void* arg) { rocksdb_block_based_options_destroy(((RocksdbCfParam*)wrapper->param)[i].tableOpt); } - if (wrapper->remove) { + if (remove) { streamBackendDelCompare(wrapper->pBackend, wrapper->pComparNode); } rocksdb_writeoptions_destroy(wrapper->writeOpts); @@ -315,16 +315,16 @@ void streamBackendHandleCleanup(void* arg) { return; } SListNode* streamBackendAddCompare(void* backend, void* arg) { - SBackendHandle* pHandle = (SBackendHandle*)backend; - SListNode* node = NULL; + SBackendWrapper* pHandle = (SBackendWrapper*)backend; + SListNode* node = NULL; taosThreadMutexLock(&pHandle->mutex); node = tdListAdd(pHandle->list, arg); taosThreadMutexUnlock(&pHandle->mutex); return node; } void streamBackendDelCompare(void* backend, void* arg) { - SBackendHandle* pHandle = (SBackendHandle*)backend; - SListNode* node = NULL; + SBackendWrapper* pHandle = (SBackendWrapper*)backend; + SListNode* node = NULL; taosThreadMutexLock(&pHandle->mutex); node = tdListPopNode(pHandle->list, arg); taosThreadMutexUnlock(&pHandle->mutex); @@ -784,11 +784,11 @@ void destroyRocksdbCfInst(RocksdbCfInst* inst) { } int32_t streamStateOpenBackendCf(void* backend, char* name, char** cfs, int32_t nCf) { - SBackendHandle* handle = backend; - char* err = NULL; - int64_t streamId; - int32_t taskId, dummy = 0; - char suffix[64] = {0}; + SBackendWrapper* handle = backend; + char* err = NULL; + int64_t streamId; + int32_t taskId, dummy = 0; + char suffix[64] = {0}; rocksdb_options_t** cfOpts = taosMemoryCalloc(nCf, sizeof(rocksdb_options_t*)); RocksdbCfParam* params = taosMemoryCalloc(nCf, sizeof(RocksdbCfParam*)); @@ -908,30 +908,30 @@ int32_t streamStateOpenBackendCf(void* backend, char* name, char** cfs, int32_t int streamStateOpenBackend(void* backend, SStreamState* pState) { qInfo("start to open state %p on backend %p 0x%" PRIx64 "-%d", pState, backend, pState->streamId, pState->taskId); taosAcquireRef(streamBackendId, pState->streamBackendRid); - SBackendHandle* handle = backend; - SBackendWrapper* pBackendWrapper = taosMemoryCalloc(1, sizeof(SBackendWrapper)); + SBackendWrapper* handle = backend; + SBackendCfWrapper* pBackendCfWrapper = taosMemoryCalloc(1, sizeof(SBackendCfWrapper)); taosThreadMutexLock(&handle->cfMutex); RocksdbCfInst** ppInst = taosHashGet(handle->cfInst, pState->pTdbState->idstr, strlen(pState->pTdbState->idstr) + 1); if (ppInst != NULL && *ppInst != NULL) { RocksdbCfInst* inst = *ppInst; - pBackendWrapper->rocksdb = inst->db; - pBackendWrapper->pHandle = (void**)inst->pHandle; - pBackendWrapper->writeOpts = inst->wOpt; - pBackendWrapper->readOpts = inst->rOpt; - pBackendWrapper->cfOpts = (void**)(inst->cfOpt); - pBackendWrapper->dbOpt = handle->dbOpt; - pBackendWrapper->param = inst->param; - pBackendWrapper->pBackend = handle; - pBackendWrapper->pComparNode = inst->pCompareNode; + pBackendCfWrapper->rocksdb = inst->db; + pBackendCfWrapper->pHandle = (void**)inst->pHandle; + pBackendCfWrapper->writeOpts = inst->wOpt; + pBackendCfWrapper->readOpts = inst->rOpt; + pBackendCfWrapper->cfOpts = (void**)(inst->cfOpt); + pBackendCfWrapper->dbOpt = handle->dbOpt; + pBackendCfWrapper->param = inst->param; + pBackendCfWrapper->pBackend = handle; + pBackendCfWrapper->pComparNode = inst->pCompareNode; taosThreadMutexUnlock(&handle->cfMutex); - pBackendWrapper->backendId = pState->streamBackendRid; - memcpy(pBackendWrapper->idstr, pState->pTdbState->idstr, sizeof(pState->pTdbState->idstr)); + pBackendCfWrapper->backendId = pState->streamBackendRid; + memcpy(pBackendCfWrapper->idstr, pState->pTdbState->idstr, sizeof(pState->pTdbState->idstr)); - int64_t id = taosAddRef(streamBackendWrapperId, pBackendWrapper); - pState->pTdbState->backendWrapperId = id; - pState->pTdbState->pBackendWrapper = pBackendWrapper; - qInfo("succ to open state %p on backendWrapper, %p, %s", pState, pBackendWrapper, pBackendWrapper->idstr); + int64_t id = taosAddRef(streamBackendCfWrapperId, pBackendCfWrapper); + pState->pTdbState->backendCfWrapperId = id; + pState->pTdbState->pBackendCfWrapper = pBackendCfWrapper; + qInfo("succ to open state %p on backendWrapper, %p, %s", pState, pBackendCfWrapper, pBackendCfWrapper->idstr); return 0; } taosThreadMutexUnlock(&handle->cfMutex); @@ -964,31 +964,31 @@ int streamStateOpenBackend(void* backend, SStreamState* pState) { pCompare[i] = compare; } rocksdb_column_family_handle_t** cfHandle = taosMemoryCalloc(cfLen, sizeof(rocksdb_column_family_handle_t*)); - pBackendWrapper->rocksdb = handle->db; - pBackendWrapper->pHandle = (void**)cfHandle; - pBackendWrapper->writeOpts = rocksdb_writeoptions_create(); - pBackendWrapper->readOpts = rocksdb_readoptions_create(); - pBackendWrapper->cfOpts = (void**)cfOpt; - pBackendWrapper->dbOpt = handle->dbOpt; - pBackendWrapper->param = param; - pBackendWrapper->pBackend = handle; - pBackendWrapper->backendId = pState->streamBackendRid; - taosThreadRwlockInit(&pBackendWrapper->rwLock, NULL); + pBackendCfWrapper->rocksdb = handle->db; + pBackendCfWrapper->pHandle = (void**)cfHandle; + pBackendCfWrapper->writeOpts = rocksdb_writeoptions_create(); + pBackendCfWrapper->readOpts = rocksdb_readoptions_create(); + pBackendCfWrapper->cfOpts = (void**)cfOpt; + pBackendCfWrapper->dbOpt = handle->dbOpt; + pBackendCfWrapper->param = param; + pBackendCfWrapper->pBackend = handle; + pBackendCfWrapper->backendId = pState->streamBackendRid; + taosThreadRwlockInit(&pBackendCfWrapper->rwLock, NULL); SCfComparator compare = {.comp = pCompare, .numOfComp = cfLen}; - pBackendWrapper->pComparNode = streamBackendAddCompare(handle, &compare); - rocksdb_writeoptions_disable_WAL(pBackendWrapper->writeOpts, 1); - memcpy(pBackendWrapper->idstr, pState->pTdbState->idstr, sizeof(pState->pTdbState->idstr)); + pBackendCfWrapper->pComparNode = streamBackendAddCompare(handle, &compare); + rocksdb_writeoptions_disable_WAL(pBackendCfWrapper->writeOpts, 1); + memcpy(pBackendCfWrapper->idstr, pState->pTdbState->idstr, sizeof(pState->pTdbState->idstr)); - int64_t id = taosAddRef(streamBackendWrapperId, pBackendWrapper); - pState->pTdbState->backendWrapperId = id; - pState->pTdbState->pBackendWrapper = pBackendWrapper; - qInfo("succ to open state %p on backendWrapper, %p, %s", pState, pBackendWrapper, pBackendWrapper->idstr); + int64_t id = taosAddRef(streamBackendCfWrapperId, pBackendCfWrapper); + pState->pTdbState->backendCfWrapperId = id; + pState->pTdbState->pBackendCfWrapper = pBackendCfWrapper; + qInfo("succ to open state %p on backendWrapper %p %s", pState, pBackendCfWrapper, pBackendCfWrapper->idstr); return 0; } void streamStateCloseBackend(SStreamState* pState, bool remove) { - SBackendWrapper* wrapper = pState->pTdbState->pBackendWrapper; - SBackendHandle* pHandle = wrapper->pBackend; + SBackendCfWrapper* wrapper = pState->pTdbState->pBackendCfWrapper; + SBackendWrapper* pHandle = wrapper->pBackend; taosThreadMutexLock(&pHandle->cfMutex); RocksdbCfInst** ppInst = taosHashGet(pHandle->cfInst, wrapper->idstr, strlen(pState->pTdbState->idstr) + 1); if (ppInst != NULL && *ppInst != NULL) { @@ -1001,7 +1001,7 @@ void streamStateCloseBackend(SStreamState* pState, bool remove) { qInfo("start to close %s state %p on backendWrapper %p %s", status[remove == false ? 0 : 1], pState, wrapper, wrapper->idstr); wrapper->remove |= remove; // update by other pState - taosReleaseRef(streamBackendWrapperId, pState->pTdbState->backendWrapperId); + taosReleaseRef(streamBackendCfWrapperId, pState->pTdbState->backendCfWrapperId); } void streamStateDestroyCompar(void* arg) { SCfComparator* comp = (SCfComparator*)arg; @@ -1020,7 +1020,7 @@ int streamStateGetCfIdx(SStreamState* pState, const char* funcName) { break; } } - SBackendWrapper* wrapper = pState->pTdbState->pBackendWrapper; + SBackendCfWrapper* wrapper = pState->pTdbState->pBackendCfWrapper; if (pState != NULL && idx != -1) { rocksdb_column_family_handle_t* cf = NULL; taosThreadRwlockRdlock(&wrapper->rwLock); @@ -1060,7 +1060,7 @@ rocksdb_iterator_t* streamStateIterCreate(SStreamState* pState, const char* cfNa rocksdb_readoptions_t** readOpt) { int idx = streamStateGetCfIdx(pState, cfName); - SBackendWrapper* wrapper = pState->pTdbState->pBackendWrapper; + SBackendCfWrapper* wrapper = pState->pTdbState->pBackendCfWrapper; if (snapshot != NULL) { *snapshot = (rocksdb_snapshot_t*)rocksdb_create_snapshot(wrapper->rocksdb); } @@ -1084,8 +1084,8 @@ rocksdb_iterator_t* streamStateIterCreate(SStreamState* pState, const char* cfNa code = -1; \ break; \ } \ - SBackendWrapper* wrapper = pState->pTdbState->pBackendWrapper; \ - char toString[128] = {0}; \ + SBackendCfWrapper* wrapper = pState->pTdbState->pBackendCfWrapper; \ + char toString[128] = {0}; \ if (qDebugFlag & DEBUG_TRACE) ginitDict[i].toStrFunc((void*)key, toString); \ int32_t klen = ginitDict[i].enFunc((void*)key, buf); \ rocksdb_column_family_handle_t* pHandle = ((rocksdb_column_family_handle_t**)wrapper->pHandle)[ginitDict[i].idx]; \ @@ -1115,8 +1115,8 @@ rocksdb_iterator_t* streamStateIterCreate(SStreamState* pState, const char* cfNa code = -1; \ break; \ } \ - SBackendWrapper* wrapper = pState->pTdbState->pBackendWrapper; \ - char toString[128] = {0}; \ + SBackendCfWrapper* wrapper = pState->pTdbState->pBackendCfWrapper; \ + char toString[128] = {0}; \ if (qDebugFlag & DEBUG_TRACE) ginitDict[i].toStrFunc((void*)key, toString); \ int32_t klen = ginitDict[i].enFunc((void*)key, buf); \ rocksdb_column_family_handle_t* pHandle = ((rocksdb_column_family_handle_t**)wrapper->pHandle)[ginitDict[i].idx]; \ @@ -1159,8 +1159,8 @@ rocksdb_iterator_t* streamStateIterCreate(SStreamState* pState, const char* cfNa code = -1; \ break; \ } \ - SBackendWrapper* wrapper = pState->pTdbState->pBackendWrapper; \ - char toString[128] = {0}; \ + SBackendCfWrapper* wrapper = pState->pTdbState->pBackendCfWrapper; \ + char toString[128] = {0}; \ if (qDebugFlag & DEBUG_TRACE) ginitDict[i].toStrFunc((void*)key, toString); \ int32_t klen = ginitDict[i].enFunc((void*)key, buf); \ rocksdb_column_family_handle_t* pHandle = ((rocksdb_column_family_handle_t**)wrapper->pHandle)[ginitDict[i].idx]; \ @@ -1199,11 +1199,11 @@ int32_t streamStateDel_rocksdb(SStreamState* pState, const SWinKey* key) { int32_t streamStateClear_rocksdb(SStreamState* pState) { qDebug("streamStateClear_rocksdb"); - SBackendWrapper* wrapper = pState->pTdbState->pBackendWrapper; - char sKeyStr[128] = {0}; - char eKeyStr[128] = {0}; - SStateKey sKey = {.key = {.ts = 0, .groupId = 0}, .opNum = pState->number}; - SStateKey eKey = {.key = {.ts = INT64_MAX, .groupId = UINT64_MAX}, .opNum = pState->number}; + SBackendCfWrapper* wrapper = pState->pTdbState->pBackendCfWrapper; + char sKeyStr[128] = {0}; + char eKeyStr[128] = {0}; + SStateKey sKey = {.key = {.ts = 0, .groupId = 0}, .opNum = pState->number}; + SStateKey eKey = {.key = {.ts = INT64_MAX, .groupId = UINT64_MAX}, .opNum = pState->number}; int sLen = stateKeyEncode(&sKey, sKeyStr); int eLen = stateKeyEncode(&eKey, eKeyStr); @@ -1315,7 +1315,7 @@ SStreamStateCur* streamStateSeekKeyNext_rocksdb(SStreamState* pState, const SWin if (pCur == NULL) { return NULL; } - SBackendWrapper* wrapper = pState->pTdbState->pBackendWrapper; + SBackendCfWrapper* wrapper = pState->pTdbState->pBackendCfWrapper; pCur->number = pState->number; pCur->db = wrapper->rocksdb; pCur->iter = streamStateIterCreate(pState, "state", (rocksdb_snapshot_t**)&pCur->snapshot, @@ -1350,8 +1350,8 @@ SStreamStateCur* streamStateSeekKeyNext_rocksdb(SStreamState* pState, const SWin SStreamStateCur* streamStateSeekToLast_rocksdb(SStreamState* pState, const SWinKey* key) { qDebug("streamStateGetCur_rocksdb"); - int32_t code = 0; - SBackendWrapper* wrapper = pState->pTdbState->pBackendWrapper; + int32_t code = 0; + SBackendCfWrapper* wrapper = pState->pTdbState->pBackendCfWrapper; const SStateKey maxStateKey = {.key = {.groupId = UINT64_MAX, .ts = INT64_MAX}, .opNum = INT64_MAX}; STREAM_STATE_PUT_ROCKSDB(pState, "state", &maxStateKey, "", 0); @@ -1379,8 +1379,8 @@ SStreamStateCur* streamStateSeekToLast_rocksdb(SStreamState* pState, const SWinK SStreamStateCur* streamStateGetCur_rocksdb(SStreamState* pState, const SWinKey* key) { qDebug("streamStateGetCur_rocksdb"); - SBackendWrapper* wrapper = pState->pTdbState->pBackendWrapper; - SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); + SBackendCfWrapper* wrapper = pState->pTdbState->pBackendCfWrapper; + SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); if (pCur == NULL) return NULL; pCur->db = wrapper->rocksdb; @@ -1472,8 +1472,8 @@ int32_t streamStateSessionDel_rocksdb(SStreamState* pState, const SSessionKey* k SStreamStateCur* streamStateSessionSeekKeyCurrentPrev_rocksdb(SStreamState* pState, const SSessionKey* key) { qDebug("streamStateSessionSeekKeyCurrentPrev_rocksdb"); - SBackendWrapper* wrapper = pState->pTdbState->pBackendWrapper; - SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); + SBackendCfWrapper* wrapper = pState->pTdbState->pBackendCfWrapper; + SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); if (pCur == NULL) { return NULL; } @@ -1514,8 +1514,8 @@ SStreamStateCur* streamStateSessionSeekKeyCurrentPrev_rocksdb(SStreamState* pSta } SStreamStateCur* streamStateSessionSeekKeyCurrentNext_rocksdb(SStreamState* pState, SSessionKey* key) { qDebug("streamStateSessionSeekKeyCurrentNext_rocksdb"); - SBackendWrapper* wrapper = pState->pTdbState->pBackendWrapper; - SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); + SBackendCfWrapper* wrapper = pState->pTdbState->pBackendCfWrapper; + SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); if (pCur == NULL) { return NULL; } @@ -1552,8 +1552,8 @@ SStreamStateCur* streamStateSessionSeekKeyCurrentNext_rocksdb(SStreamState* pSta SStreamStateCur* streamStateSessionSeekKeyNext_rocksdb(SStreamState* pState, const SSessionKey* key) { qDebug("streamStateSessionSeekKeyNext_rocksdb"); - SBackendWrapper* wrapper = pState->pTdbState->pBackendWrapper; - SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); + SBackendCfWrapper* wrapper = pState->pTdbState->pBackendCfWrapper; + SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); if (pCur == NULL) { return NULL; } @@ -1646,8 +1646,8 @@ int32_t streamStateFillDel_rocksdb(SStreamState* pState, const SWinKey* key) { SStreamStateCur* streamStateFillGetCur_rocksdb(SStreamState* pState, const SWinKey* key) { qDebug("streamStateFillGetCur_rocksdb"); - SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); - SBackendWrapper* wrapper = pState->pTdbState->pBackendWrapper; + SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); + SBackendCfWrapper* wrapper = pState->pTdbState->pBackendCfWrapper; if (pCur == NULL) return NULL; @@ -1706,8 +1706,8 @@ int32_t streamStateFillGetKVByCur_rocksdb(SStreamStateCur* pCur, SWinKey* pKey, SStreamStateCur* streamStateFillSeekKeyNext_rocksdb(SStreamState* pState, const SWinKey* key) { qDebug("streamStateFillSeekKeyNext_rocksdb"); - SBackendWrapper* wrapper = pState->pTdbState->pBackendWrapper; - SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); + SBackendCfWrapper* wrapper = pState->pTdbState->pBackendCfWrapper; + SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); if (!pCur) { return NULL; } @@ -1743,8 +1743,8 @@ SStreamStateCur* streamStateFillSeekKeyNext_rocksdb(SStreamState* pState, const } SStreamStateCur* streamStateFillSeekKeyPrev_rocksdb(SStreamState* pState, const SWinKey* key) { qDebug("streamStateFillSeekKeyPrev_rocksdb"); - SBackendWrapper* wrapper = pState->pTdbState->pBackendWrapper; - SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); + SBackendCfWrapper* wrapper = pState->pTdbState->pBackendCfWrapper; + SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); if (pCur == NULL) { return NULL; } @@ -1780,8 +1780,8 @@ SStreamStateCur* streamStateFillSeekKeyPrev_rocksdb(SStreamState* pState, const } int32_t streamStateSessionGetKeyByRange_rocksdb(SStreamState* pState, const SSessionKey* key, SSessionKey* curKey) { qDebug("streamStateSessionGetKeyByRange_rocksdb"); - SBackendWrapper* wrapper = pState->pTdbState->pBackendWrapper; - SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); + SBackendCfWrapper* wrapper = pState->pTdbState->pBackendCfWrapper; + SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); if (pCur == NULL) { return -1; } @@ -2017,7 +2017,7 @@ int32_t streamDefaultIterGet_rocksdb(SStreamState* pState, const void* start, co int code = 0; char* err = NULL; - SBackendWrapper* wrapper = pState->pTdbState->pBackendWrapper; + SBackendCfWrapper* wrapper = pState->pTdbState->pBackendCfWrapper; rocksdb_snapshot_t* snapshot = NULL; rocksdb_readoptions_t* readopts = NULL; rocksdb_iterator_t* pIter = streamStateIterCreate(pState, "default", &snapshot, &readopts); @@ -2056,8 +2056,8 @@ int32_t streamDefaultIterGet_rocksdb(SStreamState* pState, const void* start, co return code; } void* streamDefaultIterCreate_rocksdb(SStreamState* pState) { - SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); - SBackendWrapper* wrapper = pState->pTdbState->pBackendWrapper; + SStreamStateCur* pCur = taosMemoryCalloc(1, sizeof(SStreamStateCur)); + SBackendCfWrapper* wrapper = pState->pTdbState->pBackendCfWrapper; pCur->db = wrapper->rocksdb; pCur->iter = streamStateIterCreate(pState, "default", (rocksdb_snapshot_t**)&pCur->snapshot, @@ -2106,8 +2106,8 @@ void streamStateClearBatch(void* pBatch) { rocksdb_writebatch_clear((rocksdb_ void streamStateDestroyBatch(void* pBatch) { rocksdb_writebatch_destroy((rocksdb_writebatch_t*)pBatch); } int32_t streamStatePutBatch(SStreamState* pState, const char* cfName, rocksdb_writebatch_t* pBatch, void* key, void* val, int32_t vlen, int64_t ttl) { - SBackendWrapper* wrapper = pState->pTdbState->pBackendWrapper; - int i = streamStateGetCfIdx(pState, cfName); + SBackendCfWrapper* wrapper = pState->pTdbState->pBackendCfWrapper; + int i = streamStateGetCfIdx(pState, cfName); if (i < 0) { qError("streamState failed to put to cf name:%s", cfName); @@ -2130,7 +2130,7 @@ int32_t streamStatePutBatchOptimize(SStreamState* pState, int32_t cfIdx, rocksdb char* ttlV = tmpBuf; int32_t ttlVLen = ginitDict[cfIdx].enValueFunc(val, vlen, ttl, &ttlV); - SBackendWrapper* wrapper = pState->pTdbState->pBackendWrapper; + SBackendCfWrapper* wrapper = pState->pTdbState->pBackendCfWrapper; rocksdb_column_family_handle_t* pCf = wrapper->pHandle[ginitDict[cfIdx].idx]; rocksdb_writebatch_put_cf((rocksdb_writebatch_t*)pBatch, pCf, buf, (size_t)klen, ttlV, (size_t)ttlVLen); @@ -2141,8 +2141,8 @@ int32_t streamStatePutBatchOptimize(SStreamState* pState, int32_t cfIdx, rocksdb return 0; } int32_t streamStatePutBatch_rocksdb(SStreamState* pState, void* pBatch) { - char* err = NULL; - SBackendWrapper* wrapper = pState->pTdbState->pBackendWrapper; + char* err = NULL; + SBackendCfWrapper* wrapper = pState->pTdbState->pBackendCfWrapper; rocksdb_write(wrapper->rocksdb, wrapper->writeOpts, (rocksdb_writebatch_t*)pBatch, &err); if (err != NULL) { qError("streamState failed to write batch, err:%s", err); diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 5a5180b8c5..4c743e46e6 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -21,16 +21,16 @@ static TdThreadOnce streamMetaModuleInit = PTHREAD_ONCE_INIT; int32_t streamBackendId = 0; -int32_t streamBackendWrapperId = 0; +int32_t streamBackendCfWrapperId = 0; static void streamMetaEnvInit() { streamBackendId = taosOpenRef(64, streamBackendCleanup); - streamBackendWrapperId = taosOpenRef(64, streamBackendHandleCleanup); + streamBackendCfWrapperId = taosOpenRef(64, streamBackendHandleCleanup); } void streamMetaInit() { taosThreadOnce(&streamMetaModuleInit, streamMetaEnvInit); } void streamMetaCleanup() { taosCloseRef(streamBackendId); - taosCloseRef(streamBackendWrapperId); + taosCloseRef(streamBackendCfWrapperId); } SStreamMeta* streamMetaOpen(const char* path, void* ahandle, FTaskExpand expandFunc, int32_t vgId) { diff --git a/source/libs/stream/src/streamState.c b/source/libs/stream/src/streamState.c index 448d693b0a..216e3b8a11 100644 --- a/source/libs/stream/src/streamState.c +++ b/source/libs/stream/src/streamState.c @@ -134,11 +134,11 @@ SStreamState* streamStateOpen(char* path, void* pTask, bool specPath, int32_t sz return NULL; } taosHashPut(pMeta->pTaskBackendUnique, pState->pTdbState->idstr, strlen(pState->pTdbState->idstr) + 1, - &pState->pTdbState->backendWrapperId, sizeof(pState->pTdbState->backendWrapperId)); + &pState->pTdbState->backendCfWrapperId, sizeof(pState->pTdbState->backendCfWrapperId)); } else { int64_t id = *(int64_t*)uniqueId; - pState->pTdbState->backendWrapperId = id; - pState->pTdbState->pBackendWrapper = taosAcquireRef(streamBackendWrapperId, id); + pState->pTdbState->backendCfWrapperId = id; + pState->pTdbState->pBackendCfWrapper = taosAcquireRef(streamBackendCfWrapperId, id); taosAcquireRef(streamBackendId, pState->streamBackendRid); } From f4b9d630c03025d00d74517c9fbaf55a7391aa95 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Fri, 16 Jun 2023 09:36:44 +0800 Subject: [PATCH 41/83] support fill history --- source/libs/stream/src/streamBackendRocksdb.c | 1 + 1 file changed, 1 insertion(+) diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index 884100808f..a770a29627 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -997,6 +997,7 @@ void streamStateCloseBackend(SStreamState* pState, bool remove) { taosHashRemove(pHandle->cfInst, pState->pTdbState->idstr, strlen(pState->pTdbState->idstr) + 1); } taosThreadMutexUnlock(&pHandle->cfMutex); + char* status[] = {"close", "drop"}; qInfo("start to close %s state %p on backendWrapper %p %s", status[remove == false ? 0 : 1], pState, wrapper, wrapper->idstr); From 933801269a3d993696b1be082a90e1aa4c5f1096 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Fri, 16 Jun 2023 09:55:11 +0800 Subject: [PATCH 42/83] fix(stream): fix memory leak and failed to close vnode. --- source/dnode/mnode/impl/src/mndDef.c | 32 +++++++++++-------- source/dnode/vnode/src/tq/tq.c | 3 +- source/libs/executor/src/timewindowoperator.c | 1 + 3 files changed, 21 insertions(+), 15 deletions(-) diff --git a/source/dnode/mnode/impl/src/mndDef.c b/source/dnode/mnode/impl/src/mndDef.c index dfd54ab254..81f434a184 100644 --- a/source/dnode/mnode/impl/src/mndDef.c +++ b/source/dnode/mnode/impl/src/mndDef.c @@ -154,18 +154,10 @@ int32_t tDecodeSStreamObj(SDecoder *pDecoder, SStreamObj *pObj, int32_t sver) { return 0; } -void tFreeStreamObj(SStreamObj *pStream) { - taosMemoryFree(pStream->sql); - taosMemoryFree(pStream->ast); - taosMemoryFree(pStream->physicalPlan); - - if (pStream->outputSchema.nCols) { - taosMemoryFree(pStream->outputSchema.pSchema); - } - - int32_t sz = taosArrayGetSize(pStream->tasks); - for (int32_t i = 0; i < sz; i++) { - SArray *pLevel = taosArrayGetP(pStream->tasks, i); +static void* freeStreamTasks(SArray* pTaskLevel) { + int32_t numOfLevel = taosArrayGetSize(pTaskLevel); + for (int32_t i = 0; i < numOfLevel; i++) { + SArray *pLevel = taosArrayGetP(pTaskLevel, i); int32_t taskSz = taosArrayGetSize(pLevel); for (int32_t j = 0; j < taskSz; j++) { SStreamTask *pTask = taosArrayGetP(pLevel, j); @@ -175,8 +167,20 @@ void tFreeStreamObj(SStreamObj *pStream) { taosArrayDestroy(pLevel); } - taosArrayDestroy(pStream->tasks); - taosArrayDestroy(pStream->pHTasksList); + return taosArrayDestroy(pTaskLevel); +} + +void tFreeStreamObj(SStreamObj *pStream) { + taosMemoryFree(pStream->sql); + taosMemoryFree(pStream->ast); + taosMemoryFree(pStream->physicalPlan); + + if (pStream->outputSchema.nCols) { + taosMemoryFree(pStream->outputSchema.pSchema); + } + + pStream->tasks = freeStreamTasks(pStream->tasks); + pStream->pHTasksList = freeStreamTasks(pStream->pHTasksList); // tagSchema.pSchema if (pStream->tagSchema.nCols > 0) { diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 417a582d78..b6a3c95897 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1077,7 +1077,8 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { ASSERT(pStreamTask->info.taskLevel == TASK_LEVEL__SOURCE); // wait for the stream task get ready for scan history data - while (pStreamTask->status.checkDownstream == 0 || pStreamTask->status.taskStatus == TASK_STATUS__SCAN_HISTORY) { + while (((pStreamTask->status.checkDownstream == 0) && (pStreamTask->status.taskStatus != TASK_STATUS__STOP)) || + pStreamTask->status.taskStatus == TASK_STATUS__SCAN_HISTORY) { tqDebug("s-task:%s level:%d not ready for halt, wait for 100ms and recheck", pStreamTask->id.idStr, pStreamTask->info.taskLevel); taosMsleep(100); diff --git a/source/libs/executor/src/timewindowoperator.c b/source/libs/executor/src/timewindowoperator.c index 23932dcbc7..aa364375ae 100644 --- a/source/libs/executor/src/timewindowoperator.c +++ b/source/libs/executor/src/timewindowoperator.c @@ -3963,6 +3963,7 @@ void destroyStreamStateOperatorInfo(void* param) { } colDataDestroy(&pInfo->twAggSup.timeWindowData); blockDataDestroy(pInfo->pDelRes); + taosArrayDestroy(pInfo->historyWins); tSimpleHashCleanup(pInfo->pSeDeleted); taosMemoryFreeClear(param); } From 1e05f5c693abf1f96082dfe209215825a35ccba5 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Fri, 16 Jun 2023 10:00:16 +0800 Subject: [PATCH 43/83] support fill history --- source/libs/stream/src/streamMeta.c | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 4c743e46e6..5d0cb37351 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -22,7 +22,8 @@ static TdThreadOnce streamMetaModuleInit = PTHREAD_ONCE_INIT; int32_t streamBackendId = 0; int32_t streamBackendCfWrapperId = 0; -static void streamMetaEnvInit() { + +static void streamMetaEnvInit() { streamBackendId = taosOpenRef(64, streamBackendCleanup); streamBackendCfWrapperId = taosOpenRef(64, streamBackendHandleCleanup); } From 64365666b7e5bcce96a5c1ba40a9ee6eadf291f2 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Fri, 16 Jun 2023 16:04:43 +0800 Subject: [PATCH 44/83] fix(stream): update the info, and do some internal refactor. --- include/common/tmsgdef.h | 2 +- include/libs/stream/tstream.h | 5 +- source/dnode/mgmt/mgmt_snode/src/smHandle.c | 3 + source/dnode/mgmt/mgmt_vnode/src/vmHandle.c | 2 +- source/dnode/mnode/impl/src/mndScheduler.c | 19 +-- source/dnode/snode/src/snode.c | 142 +++++++++++++++++--- source/dnode/vnode/src/inc/tq.h | 1 - source/dnode/vnode/src/inc/vnodeInt.h | 4 +- source/dnode/vnode/src/tq/tq.c | 52 +++---- source/dnode/vnode/src/tq/tqRestore.c | 2 +- source/dnode/vnode/src/tq/tqUtil.c | 6 - source/dnode/vnode/src/vnd/vnodeSvr.c | 11 +- source/dnode/vnode/src/vnd/vnodeSync.c | 2 +- source/libs/stream/src/stream.c | 6 + source/libs/stream/src/streamDispatch.c | 2 +- source/libs/stream/src/streamRecover.c | 46 ++++++- tests/script/tsim/stream/partitionby.sim | 1 + 17 files changed, 216 insertions(+), 90 deletions(-) diff --git a/include/common/tmsgdef.h b/include/common/tmsgdef.h index 4e24ff5b37..e98453f571 100644 --- a/include/common/tmsgdef.h +++ b/include/common/tmsgdef.h @@ -253,7 +253,7 @@ enum { TD_DEF_MSG_TYPE(TDMT_STREAM_UNUSED1, "stream-unused1", NULL, NULL) TD_DEF_MSG_TYPE(TDMT_STREAM_RETRIEVE, "stream-retrieve", NULL, NULL) TD_DEF_MSG_TYPE(TDMT_STREAM_SCAN_HISTORY, "stream-scan-history", NULL, NULL) - TD_DEF_MSG_TYPE(TDMT_STREAM_RECOVER_FINISH, "stream-recover-finish", NULL, NULL) + TD_DEF_MSG_TYPE(TDMT_STREAM_SCAN_HISTORY_FINISH, "stream-scan-history-finish", NULL, NULL) TD_DEF_MSG_TYPE(TDMT_STREAM_TRANSFER_STATE, "stream-transfer-state", NULL, NULL) TD_DEF_MSG_TYPE(TDMT_STREAM_TASK_CHECK, "stream-task-check", NULL, NULL) TD_DEF_MSG_TYPE(TDMT_STREAM_TASK_CHECKPOINT, "stream-checkpoint", NULL, NULL) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 09583572ed..62a4dcc50d 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -568,12 +568,15 @@ bool streamTaskIsIdle(const SStreamTask* pTask); int32_t streamScanExec(SStreamTask* pTask, int32_t batchSz); +char* createStreamTaskIdStr(int64_t streamId, int32_t taskId); + // recover and fill history +void streamPrepareNdoCheckDownstream(SStreamTask* pTask); int32_t streamTaskCheckDownstreamTasks(SStreamTask* pTask); int32_t streamTaskLaunchScanHistory(SStreamTask* pTask); int32_t streamTaskCheckStatus(SStreamTask* pTask); int32_t streamProcessCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* pRsp); -int32_t streamTaskStartHistoryTask(SStreamTask* pTask); +int32_t streamCheckHistoryTaskDownstrem(SStreamTask* pTask); int32_t streamTaskScanHistoryDataComplete(SStreamTask* pTask); int32_t streamStartRecoverTask(SStreamTask* pTask, int8_t igUntreated); diff --git a/source/dnode/mgmt/mgmt_snode/src/smHandle.c b/source/dnode/mgmt/mgmt_snode/src/smHandle.c index c098d546b6..b2fb7243ff 100644 --- a/source/dnode/mgmt/mgmt_snode/src/smHandle.c +++ b/source/dnode/mgmt/mgmt_snode/src/smHandle.c @@ -76,6 +76,9 @@ SArray *smGetMsgHandles() { if (dmSetMgmtHandle(pArray, TDMT_STREAM_RETRIEVE_RSP, smPutNodeMsgToStreamQueue, 1) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_STREAM_TASK_PAUSE, smPutNodeMsgToMgmtQueue, 1) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_STREAM_TASK_RESUME, smPutNodeMsgToMgmtQueue, 1) == NULL) goto _OVER; + if (dmSetMgmtHandle(pArray, TDMT_STREAM_TASK_CHECK, smPutNodeMsgToStreamQueue, 1) == NULL) goto _OVER; + if (dmSetMgmtHandle(pArray, TDMT_STREAM_TASK_CHECK_RSP, smPutNodeMsgToStreamQueue, 1) == NULL) goto _OVER; + if (dmSetMgmtHandle(pArray, TDMT_STREAM_SCAN_HISTORY_FINISH, smPutNodeMsgToStreamQueue, 1) == NULL) goto _OVER; code = 0; _OVER: diff --git a/source/dnode/mgmt/mgmt_vnode/src/vmHandle.c b/source/dnode/mgmt/mgmt_vnode/src/vmHandle.c index 7267bd5ebf..86681903c0 100644 --- a/source/dnode/mgmt/mgmt_vnode/src/vmHandle.c +++ b/source/dnode/mgmt/mgmt_vnode/src/vmHandle.c @@ -730,7 +730,7 @@ SArray *vmGetMsgHandles() { if (dmSetMgmtHandle(pArray, TDMT_STREAM_TASK_DISPATCH_RSP, vmPutMsgToStreamQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_STREAM_RETRIEVE, vmPutMsgToStreamQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_STREAM_RETRIEVE_RSP, vmPutMsgToStreamQueue, 0) == NULL) goto _OVER; - if (dmSetMgmtHandle(pArray, TDMT_STREAM_RECOVER_FINISH, vmPutMsgToStreamQueue, 0) == NULL) goto _OVER; + if (dmSetMgmtHandle(pArray, TDMT_STREAM_SCAN_HISTORY_FINISH, vmPutMsgToStreamQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_STREAM_TRANSFER_STATE, vmPutMsgToStreamQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_STREAM_TASK_CHECK, vmPutMsgToStreamQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_STREAM_TASK_CHECK_RSP, vmPutMsgToStreamQueue, 0) == NULL) goto _OVER; diff --git a/source/dnode/mnode/impl/src/mndScheduler.c b/source/dnode/mnode/impl/src/mndScheduler.c index 58532cf94c..f347b4d036 100644 --- a/source/dnode/mnode/impl/src/mndScheduler.c +++ b/source/dnode/mnode/impl/src/mndScheduler.c @@ -177,6 +177,7 @@ int32_t mndAssignStreamTaskToSnode(SMnode* pMnode, SStreamTask* pTask, SSubplan* plan->execNode.nodeId = SNODE_HANDLE; plan->execNode.epSet = pTask->info.epSet; + mDebug("s-task:0x%x set the agg task to snode:%d", pTask->id.taskId, SNODE_HANDLE); if (qSubPlanToString(plan, &pTask->exec.qmsg, &msgLen) < 0) { terrno = TSDB_CODE_QRY_INVALID_INPUT; @@ -240,7 +241,7 @@ int32_t mndAddSinkTaskToStream(SStreamObj* pStream, SArray* pTaskList, SMnode* p static int32_t addSourceStreamTask(SMnode* pMnode, SVgObj* pVgroup, SArray* pTaskList, SArray* pSinkTaskList, SStreamObj* pStream, SSubplan* plan, uint64_t uid, int8_t fillHistory, - bool hasExtraSink) { + bool hasExtraSink, int64_t firstWindowSkey) { SStreamTask* pTask = tNewStreamTask(uid, TASK_LEVEL__SOURCE, fillHistory, pStream->conf.triggerParam, pTaskList); if (pTask == NULL) { return terrno; @@ -249,6 +250,7 @@ static int32_t addSourceStreamTask(SMnode* pMnode, SVgObj* pVgroup, SArray* pTas // todo set the correct ts, which should be last key of queried table. pTask->dataRange.window.skey = INT64_MIN; pTask->dataRange.window.ekey = 1685959190000;//taosGetTimestampMs(); +// pTask->dataRange.window.ekey = firstWindowSkey - 1;//taosGetTimestampMs(); mDebug("add source task 0x%x window:%" PRId64 " - %" PRId64, pTask->id.taskId, pTask->dataRange.window.skey, pTask->dataRange.window.ekey); @@ -330,7 +332,7 @@ static void setHTasksId(SArray* pTaskList, const SArray* pHTaskList) { } static int32_t addSourceTasksForOneLevelStream(SMnode* pMnode, const SQueryPlan* pPlan, SStreamObj* pStream, - bool hasExtraSink) { + bool hasExtraSink, int64_t lastTs) { // create exec stream task, since only one level, the exec task is also the source task SArray* pTaskList = addNewTaskList(pStream->tasks); @@ -368,7 +370,7 @@ static int32_t addSourceTasksForOneLevelStream(SMnode* pMnode, const SQueryPlan* // new stream task SArray** pSinkTaskList = taosArrayGet(pStream->tasks, SINK_NODE_LEVEL); int32_t code = addSourceStreamTask(pMnode, pVgroup, pTaskList, *pSinkTaskList, pStream, plan, pStream->uid, - 0, hasExtraSink); + 0, hasExtraSink, lastTs); if (code != TSDB_CODE_SUCCESS) { sdbRelease(pSdb, pVgroup); return -1; @@ -377,7 +379,7 @@ static int32_t addSourceTasksForOneLevelStream(SMnode* pMnode, const SQueryPlan* if (pStream->conf.fillHistory) { SArray** pHSinkTaskList = taosArrayGet(pStream->pHTasksList, SINK_NODE_LEVEL); code = addSourceStreamTask(pMnode, pVgroup, pHTaskList, *pHSinkTaskList, pStream, plan, pStream->hTaskUid, - pStream->conf.fillHistory, hasExtraSink); + pStream->conf.fillHistory, hasExtraSink, lastTs); setHTasksId(pTaskList, pHTaskList); } @@ -402,7 +404,8 @@ static int32_t doAddSourceTask(SArray* pTaskList, int8_t fillHistory, int64_t ui pTask->dataRange.window.skey = INT64_MIN; pTask->dataRange.window.ekey = 1685959190000;//taosGetTimestampMs(); - mDebug("s-task:0x%x set time window:%"PRId64" - %"PRId64, pTask->id.taskId, pTask->dataRange.window.skey, pTask->dataRange.window.ekey); + mDebug("s-task:0x%x level:%d set time window:%" PRId64 " - %" PRId64, pTask->id.taskId, pTask->info.taskLevel, + pTask->dataRange.window.skey, pTask->dataRange.window.ekey); // all the source tasks dispatch result to a single agg node. setFixedDownstreamEpInfo(pTask, pDownstreamTask); @@ -570,7 +573,7 @@ static int32_t addSinkTasks(SArray* pTasksList, SMnode* pMnode, SStreamObj* pStr return TSDB_CODE_SUCCESS; } -static int32_t doScheduleStream(SStreamObj* pStream, SMnode* pMnode, SQueryPlan* pPlan) { +static int32_t doScheduleStream(SStreamObj* pStream, SMnode* pMnode, SQueryPlan* pPlan, int64_t lastTs) { SSdb* pSdb = pMnode->pSdb; int32_t numOfPlanLevel = LIST_LENGTH(pPlan->pSubplans); @@ -624,7 +627,7 @@ static int32_t doScheduleStream(SStreamObj* pStream, SMnode* pMnode, SQueryPlan* // source level return addSourceTasksForMultiLevelStream(pMnode, pPlan, pStream, pAggTask, pHAggTask); } else if (numOfPlanLevel == 1) { - return addSourceTasksForOneLevelStream(pMnode, pPlan, pStream, hasExtraSink); + return addSourceTasksForOneLevelStream(pMnode, pPlan, pStream, hasExtraSink, lastTs); } return 0; @@ -637,7 +640,7 @@ int32_t mndScheduleStream(SMnode* pMnode, SStreamObj* pStream) { return -1; } - int32_t code = doScheduleStream(pStream, pMnode, pPlan); + int32_t code = doScheduleStream(pStream, pMnode, pPlan, 0); qDestroyQueryPlan(pPlan); return code; diff --git a/source/dnode/snode/src/snode.c b/source/dnode/snode/src/snode.c index 42bb606c4e..ad57bdbdc6 100644 --- a/source/dnode/snode/src/snode.c +++ b/source/dnode/snode/src/snode.c @@ -52,10 +52,7 @@ void sndEnqueueStreamDispatch(SSnode *pSnode, SRpcMsg *pMsg) { FAIL: if (pMsg->info.handle == NULL) return; - SRpcMsg rsp = { - .code = code, - .info = pMsg->info, - }; + SRpcMsg rsp = { .code = code, .info = pMsg->info}; tmsgSendRsp(&rsp); rpcFreeCont(pMsg->pCont); taosFreeQitem(pMsg); @@ -65,10 +62,11 @@ int32_t sndExpandTask(SSnode *pSnode, SStreamTask *pTask, int64_t ver) { ASSERT(pTask->info.taskLevel == TASK_LEVEL__AGG && taosArrayGetSize(pTask->pUpstreamEpInfoList) != 0); pTask->refCnt = 1; - pTask->status.schedStatus = TASK_SCHED_STATUS__INACTIVE; + pTask->id.idStr = createStreamTaskIdStr(pTask->id.streamId, pTask->id.taskId); - pTask->inputQueue = streamQueueOpen(0); - pTask->outputQueue = streamQueueOpen(0); + pTask->status.schedStatus = TASK_SCHED_STATUS__INACTIVE; + pTask->inputQueue = streamQueueOpen(512 << 10); + pTask->outputQueue = streamQueueOpen(512 << 10); if (pTask->inputQueue == NULL || pTask->outputQueue == NULL) { return -1; @@ -93,6 +91,10 @@ int32_t sndExpandTask(SSnode *pSnode, SStreamTask *pTask, int64_t ver) { ASSERT(pTask->exec.pExecutor); streamSetupTrigger(pTask); + + qDebug("snode:%d expand stream task on snode, s-task:%s, checkpoint ver:%" PRId64 " child id:%d, level:%d", SNODE_HANDLE, + pTask->id.idStr, pTask->chkInfo.version, pTask->info.selfChildId, pTask->info.taskLevel); + return 0; } @@ -149,6 +151,7 @@ int32_t sndProcessTaskDeployReq(SSnode *pSnode, char *msg, int32_t msgLen) { taosMemoryFree(pTask); return -1; } + tDecoderClear(&decoder); ASSERT(pTask->info.taskLevel == TASK_LEVEL__AGG); @@ -161,19 +164,20 @@ int32_t sndProcessTaskDeployReq(SSnode *pSnode, char *msg, int32_t msgLen) { return -1; } + int32_t numOfTasks = streamMetaGetNumOfTasks(pSnode->pMeta); taosWUnLockLatch(&pSnode->pMeta->lock); - // 3.go through recover steps to fill history - if (pTask->info.fillHistory) { - streamSetParamForScanHistoryData(pTask); - streamAggRecoverPrepare(pTask); - } + streamPrepareNdoCheckDownstream(pTask); + qDebug("snode:%d s-task:%s is deployed on snode and add into meta, status:%s, numOfTasks:%d", SNODE_HANDLE, pTask->id.idStr, + streamGetTaskStatusStr(pTask->status.taskStatus), numOfTasks); return 0; } int32_t sndProcessTaskDropReq(SSnode *pSnode, char *msg, int32_t msgLen) { SVDropStreamTaskReq *pReq = (SVDropStreamTaskReq *)msg; + qDebug("snode:%d receive msg to drop stream task:0x%x", pSnode->pMeta->vgId, pReq->taskId); + streamMetaRemoveTask(pSnode->pMeta, pReq->taskId); return 0; } @@ -255,13 +259,15 @@ int32_t sndProcessTaskRetrieveRsp(SSnode *pSnode, SRpcMsg *pMsg) { } int32_t sndProcessWriteMsg(SSnode *pSnode, SRpcMsg *pMsg, SRpcMsg *pRsp) { - void *pReq = POINTER_SHIFT(pMsg->pCont, sizeof(SMsgHead)); - int32_t len = pMsg->contLen - sizeof(SMsgHead); switch (pMsg->msgType) { - case TDMT_STREAM_TASK_DEPLOY: + case TDMT_STREAM_TASK_DEPLOY: { + void *pReq = POINTER_SHIFT(pMsg->pCont, sizeof(SMsgHead)); + int32_t len = pMsg->contLen - sizeof(SMsgHead); return sndProcessTaskDeployReq(pSnode, pReq, len); + } + case TDMT_STREAM_TASK_DROP: - return sndProcessTaskDropReq(pSnode, pReq, len); + return sndProcessTaskDropReq(pSnode, pMsg->pCont, pMsg->contLen); default: ASSERT(0); } @@ -300,6 +306,102 @@ int32_t sndProcessTaskRecoverFinishRsp(SSnode *pSnode, SRpcMsg *pMsg) { return 0; } +int32_t sndProcessStreamTaskCheckReq(SSnode *pSnode, SRpcMsg *pMsg) { + char *msgStr = pMsg->pCont; + char *msgBody = POINTER_SHIFT(msgStr, sizeof(SMsgHead)); + int32_t msgLen = pMsg->contLen - sizeof(SMsgHead); + + SStreamTaskCheckReq req; + 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, + }; + + SStreamTask *pTask = streamMetaAcquireTask(pSnode->pMeta, taskId); + + if (pTask != NULL) { + rsp.status = streamTaskCheckStatus(pTask); + streamMetaReleaseTask(pSnode->pMeta, pTask); + + qDebug("s-task:%s recv task check req(reqId:0x%" PRIx64 ") task:0x%x (vgId:%d), status:%s, rsp status %d", + pTask->id.idStr, rsp.reqId, rsp.upstreamTaskId, rsp.upstreamNodeId, + streamGetTaskStatusStr(pTask->status.taskStatus), rsp.status); + } else { + rsp.status = 0; + qDebug("tq recv task check(taskId:0x%x not built yet) req(reqId:0x%" PRIx64 + ") from task:0x%x (vgId:%d), rsp status %d", + taskId, rsp.reqId, rsp.upstreamTaskId, rsp.upstreamNodeId, rsp.status); + } + + SEncoder encoder; + int32_t code; + int32_t len; + + tEncodeSize(tEncodeStreamTaskCheckRsp, &rsp, len, code); + if (code < 0) { + qError("vgId:%d failed to encode task check rsp, task:0x%x", pSnode->pMeta->vgId, taskId); + return -1; + } + + void *buf = rpcMallocCont(sizeof(SMsgHead) + len); + ((SMsgHead *)buf)->vgId = htonl(req.upstreamNodeId); + + void *abuf = POINTER_SHIFT(buf, sizeof(SMsgHead)); + tEncoderInit(&encoder, (uint8_t *)abuf, len); + tEncodeStreamTaskCheckRsp(&encoder, &rsp); + tEncoderClear(&encoder); + + SRpcMsg rspMsg = {.code = 0, .pCont = buf, .contLen = sizeof(SMsgHead) + len, .info = pMsg->info}; + + tmsgSendRsp(&rspMsg); + return 0; +} + +int32_t sndProcessStreamTaskCheckRsp(SSnode* pSnode, SRpcMsg* pMsg) { + char* pReq = POINTER_SHIFT(pMsg->pCont, sizeof(SMsgHead)); + int32_t len = pMsg->contLen - sizeof(SMsgHead); + + int32_t code; + SStreamTaskCheckRsp rsp; + + SDecoder decoder; + tDecoderInit(&decoder, (uint8_t*)pReq, len); + code = tDecodeStreamTaskCheckRsp(&decoder, &rsp); + + if (code < 0) { + tDecoderClear(&decoder); + return -1; + } + + tDecoderClear(&decoder); + qDebug("tq task:0x%x (vgId:%d) recv check rsp(reqId:0x%" PRIx64 ") from 0x%x (vgId:%d) status %d", + rsp.upstreamTaskId, rsp.upstreamNodeId, rsp.reqId, rsp.downstreamTaskId, rsp.downstreamNodeId, rsp.status); + + SStreamTask* pTask = streamMetaAcquireTask(pSnode->pMeta, rsp.upstreamTaskId); + if (pTask == NULL) { + qError("tq failed to locate the stream task:0x%x (vgId:%d), it may have been destroyed", rsp.upstreamTaskId, + pSnode->pMeta->vgId); + return -1; + } + + code = streamProcessCheckRsp(pTask, &rsp); + streamMetaReleaseTask(pSnode->pMeta, pTask); + return code; +} + int32_t sndProcessStreamMsg(SSnode *pSnode, SRpcMsg *pMsg) { switch (pMsg->msgType) { case TDMT_STREAM_TASK_RUN: @@ -312,10 +414,14 @@ int32_t sndProcessStreamMsg(SSnode *pSnode, SRpcMsg *pMsg) { return sndProcessTaskRetrieveReq(pSnode, pMsg); case TDMT_STREAM_RETRIEVE_RSP: return sndProcessTaskRetrieveRsp(pSnode, pMsg); - case TDMT_STREAM_RECOVER_FINISH: + case TDMT_STREAM_SCAN_HISTORY_FINISH: return sndProcessTaskRecoverFinishReq(pSnode, pMsg); - case TDMT_STREAM_RECOVER_FINISH_RSP: + case TDMT_STREAM_SCAN_HISTORY_FINISH_RSP: return sndProcessTaskRecoverFinishRsp(pSnode, pMsg); + case TDMT_STREAM_TASK_CHECK: + return sndProcessStreamTaskCheckReq(pSnode, pMsg); + case TDMT_STREAM_TASK_CHECK_RSP: + return sndProcessStreamTaskCheckRsp(pSnode, pMsg); default: ASSERT(0); } diff --git a/source/dnode/vnode/src/inc/tq.h b/source/dnode/vnode/src/inc/tq.h index 22baea736d..7f95e48c41 100644 --- a/source/dnode/vnode/src/inc/tq.h +++ b/source/dnode/vnode/src/inc/tq.h @@ -171,7 +171,6 @@ int32_t tqStreamTasksStatusCheck(STQ* pTq); // tq util int32_t extractDelDataBlock(const void* pData, int32_t len, int64_t ver, SStreamRefDataBlock** pRefBlock); -char* createStreamTaskIdStr(int64_t streamId, int32_t taskId); int32_t tqAddInputBlockNLaunchTask(SStreamTask* pTask, SStreamQueueItem* pQueueItem); int32_t tqExtractDataForMq(STQ* pTq, STqHandle* pHandle, const SMqPollReq* pRequest, SRpcMsg* pMsg); int32_t tqDoSendDataRsp(const SRpcHandleInfo* pRpcHandleInfo, const SMqDataRsp* pRsp, int32_t epoch, int64_t consumerId, diff --git a/source/dnode/vnode/src/inc/vnodeInt.h b/source/dnode/vnode/src/inc/vnodeInt.h index 0eb2817e22..d85b78b80c 100644 --- a/source/dnode/vnode/src/inc/vnodeInt.h +++ b/source/dnode/vnode/src/inc/vnodeInt.h @@ -218,7 +218,7 @@ int tqPushMsg(STQ*, void* msg, int32_t msgLen, tmsg_t msgType, int64_t ver); int tqRegisterPushHandle(STQ* pTq, void* handle, SRpcMsg* pMsg); int tqUnregisterPushHandle(STQ* pTq, void* pHandle); int tqStartStreamTasks(STQ* pTq); // restore all stream tasks after vnode launching completed. -int tqCheckforStreamStatus(STQ* pTq); +int tqCheckStreamStatus(STQ* pTq); int tqCommit(STQ*); int32_t tqUpdateTbUidList(STQ* pTq, const SArray* tbUidList, bool isAdd); @@ -240,7 +240,7 @@ int32_t tqProcessTaskDropReq(STQ* pTq, int64_t version, char* msg, int32_t msgLe int32_t tqProcessTaskPauseReq(STQ* pTq, int64_t version, char* msg, int32_t msgLen); int32_t tqProcessTaskResumeReq(STQ* pTq, int64_t version, char* msg, int32_t msgLen); int32_t tqProcessStreamTaskCheckReq(STQ* pTq, SRpcMsg* pMsg); -int32_t tqProcessStreamTaskCheckRsp(STQ* pTq, int64_t version, char* msg, int32_t msgLen); +int32_t tqProcessStreamTaskCheckRsp(STQ* pTq, int64_t version, SRpcMsg* pMsg); int32_t tqProcessTaskRunReq(STQ* pTq, SRpcMsg* pMsg); int32_t tqProcessTaskDispatchReq(STQ* pTq, SRpcMsg* pMsg, bool exec); int32_t tqProcessTaskDispatchRsp(STQ* pTq, SRpcMsg* pMsg); diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index e389f77a30..b4eb5957c7 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -920,12 +920,15 @@ int32_t tqProcessStreamTaskCheckReq(STQ* pTq, SRpcMsg* pMsg) { return 0; } -int32_t tqProcessStreamTaskCheckRsp(STQ* pTq, int64_t sversion, char* msg, int32_t msgLen) { +int32_t tqProcessStreamTaskCheckRsp(STQ* pTq, int64_t sversion, SRpcMsg* pMsg) { + char* pReq = POINTER_SHIFT(pMsg->pCont, sizeof(SMsgHead)); + int32_t len = pMsg->contLen - sizeof(SMsgHead); + int32_t code; SStreamTaskCheckRsp rsp; SDecoder decoder; - tDecoderInit(&decoder, (uint8_t*)msg, msgLen); + tDecoderInit(&decoder, (uint8_t*)pReq, len); code = tDecodeStreamTaskCheckRsp(&decoder, &rsp); if (code < 0) { @@ -993,37 +996,7 @@ int32_t tqProcessTaskDeployReq(STQ* pTq, int64_t sversion, char* msg, int32_t ms taosWUnLockLatch(&pStreamMeta->lock); // 3. It's an fill history task, do nothing. wait for the main task to start it - if (pTask->info.fillHistory) { - tqDebug("s-task:%s fill history task, wait for being launched", pTask->id.idStr); - } else { - // calculate the correct start time window, and start the handle the history data for the main task. - if (pTask->historyTaskId.taskId != 0) { - // launch the history fill stream task - streamTaskStartHistoryTask(pTask); - - // launch current task - SHistDataRange* pRange = &pTask->dataRange; - int64_t ekey = pRange->window.ekey; - int64_t ver = pRange->range.minVer; - - pRange->window.skey = ekey; - pRange->window.ekey = INT64_MAX; - pRange->range.minVer = 0; - pRange->range.maxVer = ver; - - tqDebug("s-task:%s fill-history task exists, update stream time window:%" PRId64 " - %" PRId64 - ", ver range:%" PRId64 " - %" PRId64, - pTask->id.idStr, pRange->window.skey, pRange->window.ekey, pRange->range.minVer, pRange->range.maxVer); - } else { - SHistDataRange* pRange = &pTask->dataRange; - tqDebug("s-task:%s no associated task, stream time window:%" PRId64 " - %" PRId64 ", ver range:%" PRId64 - " - %" PRId64, - pTask->id.idStr, pRange->window.skey, pRange->window.ekey, pRange->range.minVer, pRange->range.maxVer); - } - - ASSERT(pTask->status.checkDownstream == 0); - streamTaskCheckDownstreamTasks(pTask); - } + streamPrepareNdoCheckDownstream(pTask); tqDebug("vgId:%d s-task:%s is deployed and add into meta, status:%s, numOfTasks:%d", vgId, pTask->id.idStr, streamGetTaskStatusStr(pTask->status.taskStatus), numOfTasks); @@ -1160,9 +1133,16 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { // todo update the chkInfo version for current task. // this task has an associated history stream task, so we need to scan wal from the end version of // history scan. The current version of chkInfo.current is not updated during the history scan - tqDebug("s-task:%s history data scan completed, now start to scan data from wal, start ver:%" PRId64 - ", window:%" PRId64 " - %" PRId64, - pTask->id.idStr, pTask->chkInfo.currentVer, pTask->dataRange.window.skey, pTask->dataRange.window.ekey); + if (pTask->historyTaskId.taskId == 0) { + pTask->dataRange.window.ekey = INT64_MAX; + pTask->dataRange.window.skey = INT64_MIN; + tqDebug("s-task:%s without associated stream task, reset the time window:%"PRId64" - %"PRId64, pTask->id.idStr, + pTask->dataRange.window.skey, pTask->dataRange.window.ekey); + } else { + tqDebug("s-task:%s history data scan completed, now start to scan data from wal, start ver:%" PRId64 + ", window:%" PRId64 " - %" PRId64, + pTask->id.idStr, pTask->chkInfo.currentVer, pTask->dataRange.window.skey, pTask->dataRange.window.ekey); + } code = streamTaskScanHistoryDataComplete(pTask); streamMetaReleaseTask(pMeta, pTask); diff --git a/source/dnode/vnode/src/tq/tqRestore.c b/source/dnode/vnode/src/tq/tqRestore.c index 6fbc4197ee..5c0e735a81 100644 --- a/source/dnode/vnode/src/tq/tqRestore.c +++ b/source/dnode/vnode/src/tq/tqRestore.c @@ -87,7 +87,7 @@ int32_t tqStreamTasksStatusCheck(STQ* pTq) { return 0; } -int32_t tqCheckforStreamStatus(STQ* pTq) { +int32_t tqCheckStreamStatus(STQ* pTq) { int32_t vgId = TD_VID(pTq->pVnode); SStreamMeta* pMeta = pTq->pStreamMeta; diff --git a/source/dnode/vnode/src/tq/tqUtil.c b/source/dnode/vnode/src/tq/tqUtil.c index a301d82c30..723e844346 100644 --- a/source/dnode/vnode/src/tq/tqUtil.c +++ b/source/dnode/vnode/src/tq/tqUtil.c @@ -20,12 +20,6 @@ static int32_t tqSendMetaPollRsp(STqHandle* pHandle, const SRpcMsg* pMsg, const SMqPollReq* pReq, const SMqMetaRsp* pRsp, int32_t vgId); -char* createStreamTaskIdStr(int64_t streamId, int32_t taskId) { - char buf[128] = {0}; - sprintf(buf, "0x%" PRIx64 "-0x%x", streamId, taskId); - return taosStrdup(buf); -} - int32_t tqAddInputBlockNLaunchTask(SStreamTask* pTask, SStreamQueueItem* pQueueItem) { int32_t code = tAppendDataToInputQueue(pTask, pQueueItem); if (code < 0) { diff --git a/source/dnode/vnode/src/vnd/vnodeSvr.c b/source/dnode/vnode/src/vnd/vnodeSvr.c index a099f3123e..0f7bcd25a0 100644 --- a/source/dnode/vnode/src/vnd/vnodeSvr.c +++ b/source/dnode/vnode/src/vnd/vnodeSvr.c @@ -579,11 +579,8 @@ int32_t vnodeProcessStreamMsg(SVnode *pVnode, SRpcMsg *pMsg, SQueueInfo *pInfo) return tqProcessTaskDispatchRsp(pVnode->pTq, pMsg); case TDMT_STREAM_TASK_CHECK: return tqProcessStreamTaskCheckReq(pVnode->pTq, pMsg); - case TDMT_STREAM_TASK_CHECK_RSP: { - char* pReq = POINTER_SHIFT(pMsg->pCont, sizeof(SMsgHead)); - int32_t len = pMsg->contLen - sizeof(SMsgHead); - return tqProcessStreamTaskCheckRsp(pVnode->pTq, 0, pReq, len); - } + case TDMT_STREAM_TASK_CHECK_RSP: + return tqProcessStreamTaskCheckRsp(pVnode->pTq, 0, pMsg); case TDMT_STREAM_RETRIEVE: return tqProcessTaskRetrieveReq(pVnode->pTq, pMsg); case TDMT_STREAM_RETRIEVE_RSP: @@ -595,9 +592,9 @@ int32_t vnodeProcessStreamMsg(SVnode *pVnode, SRpcMsg *pMsg, SQueueInfo *pInfo) int32_t len = pMsg->contLen - sizeof(SMsgHead); return tqProcessTaskTransferStateReq(pVnode->pTq, 0, pReq, len); } - case TDMT_STREAM_RECOVER_FINISH: + case TDMT_STREAM_SCAN_HISTORY_FINISH: return tqProcessTaskRecoverFinishReq(pVnode->pTq, pMsg); - case TDMT_STREAM_RECOVER_FINISH_RSP: + case TDMT_STREAM_SCAN_HISTORY_FINISH_RSP: return tqProcessTaskRecoverFinishRsp(pVnode->pTq, pMsg); default: vError("unknown msg type:%d in stream queue", pMsg->msgType); diff --git a/source/dnode/vnode/src/vnd/vnodeSync.c b/source/dnode/vnode/src/vnd/vnodeSync.c index 3e4655ff75..360da41482 100644 --- a/source/dnode/vnode/src/vnd/vnodeSync.c +++ b/source/dnode/vnode/src/vnd/vnodeSync.c @@ -554,7 +554,7 @@ static void vnodeRestoreFinish(const SSyncFSM *pFsm, const SyncIndex commitIdx) vInfo("vgId:%d, not launch stream tasks, since stream tasks are disabled", pVnode->config.vgId); } else { vInfo("vgId:%d start to launch stream tasks", pVnode->config.vgId); - tqCheckforStreamStatus(pVnode->pTq); + tqCheckStreamStatus(pVnode->pTq); } } diff --git a/source/libs/stream/src/stream.c b/source/libs/stream/src/stream.c index e93c280d62..5043486135 100644 --- a/source/libs/stream/src/stream.c +++ b/source/libs/stream/src/stream.c @@ -55,6 +55,12 @@ void streamCleanUp() { } } +char* createStreamTaskIdStr(int64_t streamId, int32_t taskId) { + char buf[128] = {0}; + sprintf(buf, "0x%" PRIx64 "-0x%x", streamId, taskId); + return taosStrdup(buf); +} + void streamSchedByTimer(void* param, void* tmrId) { SStreamTask* pTask = (void*)param; diff --git a/source/libs/stream/src/streamDispatch.c b/source/libs/stream/src/streamDispatch.c index d48bfbb189..7ce85b4e65 100644 --- a/source/libs/stream/src/streamDispatch.c +++ b/source/libs/stream/src/streamDispatch.c @@ -308,7 +308,7 @@ int32_t streamDoDispatchScanHistoryFinishMsg(SStreamTask* pTask, const SStreamRe msg.contLen = tlen + sizeof(SMsgHead); msg.pCont = buf; - msg.msgType = TDMT_STREAM_RECOVER_FINISH; + msg.msgType = TDMT_STREAM_SCAN_HISTORY_FINISH; msg.info.noResp = 1; tmsgSendReq(pEpSet, &msg); diff --git a/source/libs/stream/src/streamRecover.c b/source/libs/stream/src/streamRecover.c index f041b17d2e..577022bbef 100644 --- a/source/libs/stream/src/streamRecover.c +++ b/source/libs/stream/src/streamRecover.c @@ -98,7 +98,6 @@ int32_t streamTaskCheckDownstreamTasks(SStreamTask* pTask) { // serialize if (pTask->outputType == TASK_OUTPUT__FIXED_DISPATCH) { - req.reqId = tGenIdPI64(); req.downstreamNodeId = pTask->fixedEpDispatcher.nodeId; req.downstreamTaskId = pTask->fixedEpDispatcher.taskId; @@ -126,8 +125,9 @@ int32_t streamTaskCheckDownstreamTasks(SStreamTask* pTask) { } } else { pTask->status.checkDownstream = 1; - qDebug("s-task:%s (vgId:%d) set downstream task checked for task without downstream tasks, try to launch scan-history-data, status:%s", + qDebug("s-task:%s (vgId:%d) no downstream tasks, set downstream checked, try to launch scan-history-data, status:%s", pTask->id.idStr, pTask->info.nodeId, streamGetTaskStatusStr(pTask->status.taskStatus)); + streamTaskLaunchScanHistory(pTask); } @@ -460,11 +460,8 @@ static void tryLaunchHistoryTask(void* param, void* tmrId) { // todo fix the bug: 2. race condition // an fill history task needs to be started. -int32_t streamTaskStartHistoryTask(SStreamTask* pTask) { +int32_t streamCheckHistoryTaskDownstrem(SStreamTask* pTask) { SStreamMeta* pMeta = pTask->pMeta; - if (pTask->historyTaskId.taskId == 0) { - return TSDB_CODE_SUCCESS; - } // Set the execute conditions, including the query time window and the version range SStreamTask** pHTask = taosHashGet(pMeta->pTasks, &pTask->historyTaskId.taskId, sizeof(pTask->historyTaskId.taskId)); @@ -585,3 +582,40 @@ int32_t tDecodeStreamRecoverFinishReq(SDecoder* pDecoder, SStreamRecoverFinishRe tEndDecode(pDecoder); return 0; } + +void streamPrepareNdoCheckDownstream(SStreamTask* pTask) { + if (pTask->info.fillHistory) { + qDebug("s-task:%s fill history task, wait for being launched", pTask->id.idStr); + } else { + // calculate the correct start time window, and start the handle the history data for the main task. + if (pTask->historyTaskId.taskId != 0) { + // check downstream tasks for associated scan-history-data tasks + streamCheckHistoryTaskDownstrem(pTask); + + // launch current task + SHistDataRange* pRange = &pTask->dataRange; + int64_t ekey = pRange->window.ekey; + int64_t ver = pRange->range.minVer; + + pRange->window.skey = ekey; + pRange->window.ekey = INT64_MAX; + pRange->range.minVer = 0; + pRange->range.maxVer = ver; + + qDebug("s-task:%s level:%d fill-history task exists, update stream time window:%" PRId64 " - %" PRId64 + ", ver range:%" PRId64 " - %" PRId64, + pTask->id.idStr, pTask->info.taskLevel, pRange->window.skey, pRange->window.ekey, pRange->range.minVer, + pRange->range.maxVer); + } else { + SHistDataRange* pRange = &pTask->dataRange; + qDebug("s-task:%s no associated scan-history task, stream time window:%" PRId64 " - %" PRId64 ", ver range:%" PRId64 + " - %" PRId64, + pTask->id.idStr, pRange->window.skey, pRange->window.ekey, pRange->range.minVer, pRange->range.maxVer); + } + + ASSERT(pTask->status.checkDownstream == 0); + + // check downstream tasks for itself + streamTaskCheckDownstreamTasks(pTask); + } +} diff --git a/tests/script/tsim/stream/partitionby.sim b/tests/script/tsim/stream/partitionby.sim index df4b60314f..9a660741e7 100644 --- a/tests/script/tsim/stream/partitionby.sim +++ b/tests/script/tsim/stream/partitionby.sim @@ -14,6 +14,7 @@ sql create table ts3 using st tags(3,2,2); sql create table ts4 using st tags(4,2,2); sql create stream stream_t1 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into test0.streamtST1 as select _wstart, count(*) c1, count(d) c2 , sum(a) c3 , max(b) c4, min(c) c5 from st partition by ta,tb,tc interval(10s); +sleep 500 sql insert into ts1 values(1648791213001,1,12,3,1.0); sql insert into ts2 values(1648791213001,1,12,3,1.0); From 41c193945f862807e723364b1a7dc5a16927d594 Mon Sep 17 00:00:00 2001 From: liuyao <54liuyao@163.com> Date: Mon, 19 Jun 2023 17:28:19 +0800 Subject: [PATCH 45/83] fill history pause&resume --- include/libs/executor/executor.h | 6 +- include/libs/stream/tstream.h | 6 +- source/dnode/mnode/impl/src/mndStream.c | 17 +- source/dnode/vnode/src/tq/tq.c | 213 +++++++++++++++--------- source/libs/executor/inc/querytask.h | 2 + source/libs/executor/src/executor.c | 42 ++++- source/libs/stream/src/streamExec.c | 6 +- source/libs/stream/src/streamRecover.c | 25 ++- 8 files changed, 223 insertions(+), 94 deletions(-) diff --git a/include/libs/executor/executor.h b/include/libs/executor/executor.h index e03718b571..faaea12b1a 100644 --- a/include/libs/executor/executor.h +++ b/include/libs/executor/executor.h @@ -221,11 +221,15 @@ void* qExtractReaderFromStreamScanner(void* scanner); int32_t qExtractStreamScanner(qTaskInfo_t tinfo, void** scanner); int32_t qSetStreamOperatorOptionForScanHistory(qTaskInfo_t tinfo); -int32_t qStreamSourceScanParamForHistoryScan(qTaskInfo_t tinfo, SVersionRange *pVerRange, STimeWindow* pWindow); +int32_t qStreamSourceScanParamForHistoryScanStep1(qTaskInfo_t tinfo, SVersionRange *pVerRange, STimeWindow* pWindow); +int32_t qStreamSourceScanParamForHistoryScanStep2(qTaskInfo_t tinfo, SVersionRange *pVerRange, STimeWindow* pWindow); int32_t qStreamSourceRecoverStep2(qTaskInfo_t tinfo, int64_t ver); int32_t qStreamRecoverFinish(qTaskInfo_t tinfo); int32_t qRestoreStreamOperatorOption(qTaskInfo_t tinfo); bool qStreamRecoverScanFinished(qTaskInfo_t tinfo); +bool qStreamRecoverScanStep1Finished(qTaskInfo_t tinfo); +bool qStreamRecoverScanStep2Finished(qTaskInfo_t tinfo); +int32_t qStreamRecoverSetAllStepFinished(qTaskInfo_t tinfo); void qStreamCloseTsdbReader(void* task); void resetTaskInfo(qTaskInfo_t tinfo); diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index a5718b4195..78acea1cd9 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -580,6 +580,9 @@ int32_t streamProcessCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* pRs int32_t streamCheckHistoryTaskDownstrem(SStreamTask* pTask); int32_t streamTaskScanHistoryDataComplete(SStreamTask* pTask); int32_t streamStartRecoverTask(SStreamTask* pTask, int8_t igUntreated); +bool streamTaskRecoverScanStep1Finished(SStreamTask* pTask); +bool streamTaskRecoverScanStep2Finished(SStreamTask* pTask); +int32_t streamTaskRecoverSetAllStepFinished(SStreamTask* pTask); // common int32_t streamSetParamForScanHistoryData(SStreamTask* pTask); @@ -588,7 +591,8 @@ int32_t streamSetStatusNormal(SStreamTask* pTask); const char* streamGetTaskStatusStr(int32_t status); // source level -int32_t streamSetParamForStreamScanner(SStreamTask* pTask, SVersionRange* pVerRange, STimeWindow* pWindow); +int32_t streamSetParamForStreamScannerStep1(SStreamTask* pTask, SVersionRange* pVerRange, STimeWindow* pWindow); +int32_t streamSetParamForStreamScannerStep2(SStreamTask* pTask, SVersionRange* pVerRange, STimeWindow* pWindow); int32_t streamBuildSourceRecover1Req(SStreamTask* pTask, SStreamScanHistoryReq* pReq, int8_t igUntreated); int32_t streamSourceScanHistoryData(SStreamTask* pTask); // int32_t streamSourceRecoverScanStep2(SStreamTask* pTask, int64_t ver); diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index acd9e851d3..e78411096f 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -1337,10 +1337,10 @@ static int32_t mndPauseStreamTask(STrans *pTrans, SStreamTask *pTask) { return 0; } -int32_t mndPauseAllStreamTasks(STrans *pTrans, SStreamObj *pStream) { - int32_t size = taosArrayGetSize(pStream->tasks); +int32_t mndPauseAllStreamTaskImpl(STrans *pTrans, SArray* tasks) { + int32_t size = taosArrayGetSize(tasks); for (int32_t i = 0; i < size; i++) { - SArray *pTasks = taosArrayGetP(pStream->tasks, i); + SArray *pTasks = taosArrayGetP(tasks, i); int32_t sz = taosArrayGetSize(pTasks); for (int32_t j = 0; j < sz; j++) { SStreamTask *pTask = taosArrayGetP(pTasks, j); @@ -1352,6 +1352,16 @@ int32_t mndPauseAllStreamTasks(STrans *pTrans, SStreamObj *pStream) { return 0; } +int32_t mndPauseAllStreamTasks(STrans *pTrans, SStreamObj *pStream) { + int32_t code = mndPauseAllStreamTaskImpl(pTrans, pStream->tasks); + if (code != 0) { + return code; + } + // pStream->pHTasksList is null + // code = mndPauseAllStreamTaskImpl(pTrans, pStream->pHTasksList); + return code; +} + static int32_t mndPersistStreamLog(STrans *pTrans, const SStreamObj *pStream, int8_t status) { SStreamObj streamObj = {0}; memcpy(streamObj.name, pStream->name, TSDB_STREAM_FNAME_LEN); @@ -1473,6 +1483,7 @@ int32_t mndResumeAllStreamTasks(STrans *pTrans, SStreamObj *pStream, int8_t igUn } } } + // pStream->pHTasksList is null return 0; } diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index b4eb5957c7..36abd581ee 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1036,9 +1036,12 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { return 0; } - streamSourceScanHistoryData(pTask); - if (atomic_load_8(&pTask->status.taskStatus) == TASK_STATUS__DROPPING) { - tqDebug("s-task:%s is dropped, abort recover in step1", pTask->id.idStr); + if (!pReq->igUntreated && !streamTaskRecoverScanStep1Finished(pTask)) { + streamSourceScanHistoryData(pTask); + } + if (atomic_load_8(&pTask->status.taskStatus) == TASK_STATUS__DROPPING || streamTaskShouldPause(&pTask->status)) { + tqDebug("s-task:%s is dropped or paused, abort recover in step1", pTask->id.idStr); + atomic_store_8(&pTask->status.schedStatus, TASK_SCHED_STATUS__INACTIVE); streamMetaReleaseTask(pMeta, pTask); return 0; } @@ -1047,44 +1050,50 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { tqDebug("s-task:%s history data scan stage(step 1) ended, elapsed time:%.2fs", pTask->id.idStr, el); if (pTask->info.fillHistory) { - // 1. stop the related stream task, get the current scan wal version of stream task, ver. - SStreamTask* pStreamTask = streamMetaAcquireTask(pMeta, pTask->streamTaskId.taskId); - if (pStreamTask == NULL) { - // todo handle error + SVersionRange* pRange = NULL; + SStreamTask* pStreamTask = NULL; + if (!pReq->igUntreated && !streamTaskRecoverScanStep1Finished(pTask)) { + // 1. stop the related stream task, get the current scan wal version of stream task, ver. + pStreamTask = streamMetaAcquireTask(pMeta, pTask->streamTaskId.taskId); + if (pStreamTask == NULL) { + // todo handle error + } + + ASSERT(pStreamTask->info.taskLevel == TASK_LEVEL__SOURCE); + + // wait for the stream task get ready for scan history data + while (((pStreamTask->status.checkDownstream == 0) && (pStreamTask->status.taskStatus != TASK_STATUS__STOP)) || + pStreamTask->status.taskStatus == TASK_STATUS__SCAN_HISTORY) { + tqDebug("s-task:%s level:%d not ready for halt, wait for 100ms and recheck", pStreamTask->id.idStr, + pStreamTask->info.taskLevel); + taosMsleep(100); + } + + // now we can stop the stream task execution + pStreamTask->status.taskStatus = TASK_STATUS__HALT; + tqDebug("s-task:%s level:%d status is set to halt by history scan task:%s", pStreamTask->id.idStr, + pStreamTask->info.taskLevel, pTask->id.idStr); + + // if it's an source task, extract the last version in wal. + int64_t ver = pTask->dataRange.range.maxVer + 1; + int64_t latestVer = walReaderGetCurrentVer(pStreamTask->exec.pWalReader); + if (latestVer >= ver) { + ver = latestVer; + } + + // 2. do secondary scan of the history data, the time window remain, and the version range is updated to [pTask->dataRange.range.maxVer, ver1] + pRange = &pTask->dataRange.range; + + pRange->minVer = pRange->maxVer + 1; + pRange->maxVer = ver; + if (pRange->minVer == pRange->maxVer) { + streamTaskRecoverSetAllStepFinished(pTask); + tqDebug("s-task:%s no need to perform secondary scan-history-data(step 2), since no new data ingest", + pTask->id.idStr); + } } - // streamTaskReleaseState(pTask); - // streamTaskReloadState(pStreamTask); - - ASSERT(pStreamTask->info.taskLevel == TASK_LEVEL__SOURCE); - - // wait for the stream task get ready for scan history data - while (((pStreamTask->status.checkDownstream == 0) && (pStreamTask->status.taskStatus != TASK_STATUS__STOP)) || - pStreamTask->status.taskStatus == TASK_STATUS__SCAN_HISTORY) { - tqDebug("s-task:%s level:%d not ready for halt, wait for 100ms and recheck", pStreamTask->id.idStr, - pStreamTask->info.taskLevel); - taosMsleep(100); - } - - // now we can stop the stream task execution - pStreamTask->status.taskStatus = TASK_STATUS__HALT; - tqDebug("s-task:%s level:%d status is set to halt by history scan task:%s", pStreamTask->id.idStr, - pStreamTask->info.taskLevel, pTask->id.idStr); - - // if it's an source task, extract the last version in wal. - int64_t ver = pTask->dataRange.range.maxVer + 1; - int64_t latestVer = walReaderGetCurrentVer(pStreamTask->exec.pWalReader); - if (latestVer >= ver) { - ver = latestVer; - } - - // 2. do secondary scan of the history data, the time window remain, and the version range is updated to [pTask->dataRange.range.maxVer, ver1] - SVersionRange* pRange = &pTask->dataRange.range; - - pRange->minVer = pRange->maxVer + 1; - pRange->maxVer = ver; - if (pRange->minVer == pRange->maxVer) { - tqDebug("s-task:%s no need to perform secondary scan-history-data(step 2), since no new data ingest", pTask->id.idStr); - } else { + + if (!streamTaskRecoverScanStep1Finished(pTask)) { tqDebug("s-task:%s level:%d verRange:%" PRId64 " - %" PRId64 " do secondary scan-history-data after halt the related stream task:%s", pTask->id.idStr, pTask->info.taskLevel, pRange->minVer, pRange->maxVer, pStreamTask->id.idStr); @@ -1092,25 +1101,30 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { ASSERT(pTask->status.schedStatus == TASK_SCHED_STATUS__WAITING); st = taosGetTimestampMs(); - streamSetParamForStreamScanner(pTask, pRange, &pTask->dataRange.window); + streamSetParamForStreamScannerStep2(pTask, pRange, &pTask->dataRange.window); + } + if(!streamTaskRecoverScanStep2Finished(pTask)) { streamSourceScanHistoryData(pTask); - if (atomic_load_8(&pTask->status.taskStatus) == TASK_STATUS__DROPPING) { - tqDebug("s-task:%s is dropped, abort recover in step1", pTask->id.idStr); + if (atomic_load_8(&pTask->status.taskStatus) == TASK_STATUS__DROPPING || streamTaskShouldPause(&pTask->status)) { + tqDebug("s-task:%s is dropped or paused, abort recover in step1", pTask->id.idStr); streamMetaReleaseTask(pMeta, pTask); return 0; } - - el = (taosGetTimestampMs() - st) / 1000.0; - tqDebug("s-task:%s history data scan stage(step 2) ended, elapsed time:%.2fs", pTask->id.idStr, el); + streamTaskRecoverSetAllStepFinished(pTask); } - // 3. notify the downstream tasks to transfer executor state after handle all history blocks. - pTask->status.transferState = true; + el = (taosGetTimestampMs() - st) / 1000.0; + tqDebug("s-task:%s history data scan stage(step 2) ended, elapsed time:%.2fs", pTask->id.idStr, el); - code = streamDispatchTransferStateMsg(pTask); - if (code != TSDB_CODE_SUCCESS) { - // todo handle error + if (!pTask->status.transferState) { + // 3. notify the downstream tasks to transfer executor state after handle all history blocks. + pTask->status.transferState = true; + + code = streamDispatchTransferStateMsg(pTask); + if (code != TSDB_CODE_SUCCESS) { + // todo handle error + } } // 4. 1) transfer the ownership of executor state, 2) update the scan data range for source task. @@ -1119,7 +1133,10 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { pTask->status.taskStatus = TASK_STATUS__DROPPING; tqDebug("s-task:%s set status to be dropping", pTask->id.idStr); - + // transfer the ownership of executor state + // todo(liuyao) + // streamTaskReleaseState(pTask); + // streamTaskReloadState(pStreamTask); streamMetaSaveTask(pMeta, pTask); streamMetaSaveTask(pMeta, pStreamTask); @@ -1165,6 +1182,11 @@ int32_t tqProcessTaskTransferStateReq(STQ* pTq, int64_t sversion, char* msg, int tqError("failed to find task:0x%x", req.taskId); return -1; } + // transfer the ownership of executor state + // todo(liuyao) + // streamTaskReleaseState(pTask); + // SStreamTask* pStreamTask = streamMetaAcquireTask(pTq->pStreamMeta, pTask->streamTaskId.taskId); + // streamTaskReloadState(pStreamTask); ASSERT(pTask->streamTaskId.taskId != 0); pTask->status.transferState = true; // persistent data? @@ -1398,51 +1420,76 @@ int32_t tqProcessTaskDropReq(STQ* pTq, int64_t sversion, char* msg, int32_t msgL return 0; } +int32_t tqProcessTaskPauseImpl(SStreamMeta* pStreamMeta, SStreamTask* pTask) { + if (pTask) { + if (!streamTaskShouldPause(&pTask->status)) { + tqDebug("vgId:%d s-task:%s set pause flag", pStreamMeta->vgId, pTask->id.idStr); + atomic_store_8(&pTask->status.keepTaskStatus, pTask->status.taskStatus); + atomic_store_8(&pTask->status.taskStatus, TASK_STATUS__PAUSE); + } + streamMetaReleaseTask(pStreamMeta, pTask); + } else { + return -1; + } + return 0; +} + int32_t tqProcessTaskPauseReq(STQ* pTq, int64_t sversion, char* msg, int32_t msgLen) { SVPauseStreamTaskReq* pReq = (SVPauseStreamTaskReq*)msg; SStreamTask* pTask = streamMetaAcquireTask(pTq->pStreamMeta, pReq->taskId); + int32_t code = tqProcessTaskPauseImpl(pTq->pStreamMeta, pTask); + if (code != 0) { + return code; + } + SStreamTask* pHistoryTask = streamMetaAcquireTask(pTq->pStreamMeta, pTask->historyTaskId.taskId); + code = tqProcessTaskPauseImpl(pTq->pStreamMeta, pHistoryTask); + return code; +} + +int32_t tqProcessTaskResumeImpl(STQ* pTq, SStreamTask* pTask, int64_t sversion, int8_t igUntreated) { + int32_t vgId = pTq->pStreamMeta->vgId; if (pTask) { - tqDebug("vgId:%d s-task:%s set pause flag", pTq->pStreamMeta->vgId, pTask->id.idStr); - atomic_store_8(&pTask->status.keepTaskStatus, pTask->status.taskStatus); - atomic_store_8(&pTask->status.taskStatus, TASK_STATUS__PAUSE); + if (streamTaskShouldPause(&pTask->status)) { + atomic_store_8(&pTask->status.taskStatus, pTask->status.keepTaskStatus); + + // no lock needs to secure the access of the version + if (igUntreated && pTask->info.taskLevel == TASK_LEVEL__SOURCE && !pTask->info.fillHistory) { + // discard all the data when the stream task is suspended. + walReaderSetSkipToVersion(pTask->exec.pWalReader, sversion); + tqDebug("vgId:%d s-task:%s resume to exec, prev paused version:%" PRId64 ", start from vnode ver:%" PRId64 + ", schedStatus:%d", + vgId, pTask->id.idStr, pTask->chkInfo.currentVer, sversion, pTask->status.schedStatus); + } else { // from the previous paused version and go on + tqDebug("vgId:%d s-task:%s resume to exec, from paused ver:%" PRId64 ", vnode ver:%" PRId64 ", schedStatus:%d", + vgId, pTask->id.idStr, pTask->chkInfo.currentVer, sversion, pTask->status.schedStatus); + } + + if (pTask->info.fillHistory && pTask->info.taskLevel == TASK_LEVEL__SOURCE) { + streamStartRecoverTask(pTask, igUntreated); + } else if (pTask->info.taskLevel == TASK_LEVEL__SOURCE && taosQueueItemSize(pTask->inputQueue->queue) == 0) { + tqStartStreamTasks(pTq); + } else { + streamSchedExec(pTask); + } + } streamMetaReleaseTask(pTq->pStreamMeta, pTask); + } else { + return -1; } return 0; } int32_t tqProcessTaskResumeReq(STQ* pTq, int64_t sversion, char* msg, int32_t msgLen) { SVResumeStreamTaskReq* pReq = (SVResumeStreamTaskReq*)msg; - - int32_t vgId = pTq->pStreamMeta->vgId; SStreamTask* pTask = streamMetaAcquireTask(pTq->pStreamMeta, pReq->taskId); - if (pTask) { - atomic_store_8(&pTask->status.taskStatus, pTask->status.keepTaskStatus); - - // no lock needs to secure the access of the version - if (pReq->igUntreated && pTask->info.taskLevel == TASK_LEVEL__SOURCE && !pTask->info.fillHistory) { - // discard all the data when the stream task is suspended. - walReaderSetSkipToVersion(pTask->exec.pWalReader, sversion); - tqDebug("vgId:%d s-task:%s resume to exec, prev paused version:%" PRId64 ", start from vnode ver:%" PRId64 - ", schedStatus:%d", - vgId, pTask->id.idStr, pTask->chkInfo.currentVer, sversion, pTask->status.schedStatus); - } else { // from the previous paused version and go on - tqDebug("vgId:%d s-task:%s resume to exec, from paused ver:%" PRId64 ", vnode ver:%" PRId64 ", schedStatus:%d", - vgId, pTask->id.idStr, pTask->chkInfo.currentVer, sversion, pTask->status.schedStatus); - } - - if (pTask->info.fillHistory && pTask->info.taskLevel == TASK_LEVEL__SOURCE) { - streamStartRecoverTask(pTask, pReq->igUntreated); - } else if (pTask->info.taskLevel == TASK_LEVEL__SOURCE && taosQueueItemSize(pTask->inputQueue->queue) == 0) { - tqStartStreamTasks(pTq); - } else { - streamSchedExec(pTask); - } - streamMetaReleaseTask(pTq->pStreamMeta, pTask); - } else { - tqError("vgId:%d failed to find the s-task:0x%x for resume stream task", vgId, pReq->taskId); + int32_t code = tqProcessTaskResumeImpl(pTq, pTask, sversion, pReq->igUntreated); + if (code != 0) { + return code; } - return 0; + SStreamTask* pHistoryTask = streamMetaAcquireTask(pTq->pStreamMeta, pTask->historyTaskId.taskId); + code = tqProcessTaskResumeImpl(pTq, pHistoryTask, sversion, pReq->igUntreated); + return code; } int32_t tqProcessTaskRetrieveReq(STQ* pTq, SRpcMsg* pMsg) { diff --git a/source/libs/executor/inc/querytask.h b/source/libs/executor/inc/querytask.h index 6c2a16bb0c..9f8b749171 100644 --- a/source/libs/executor/inc/querytask.h +++ b/source/libs/executor/inc/querytask.h @@ -62,6 +62,8 @@ typedef struct { SSchemaWrapper* schema; char tbName[TSDB_TABLE_NAME_LEN]; // this is the current scan table: todo refactor int8_t recoverStep; + bool recoverStep1Finished; + bool recoverStep2Finished; int8_t recoverScanFinished; SQueryTableDataCond tableCond; SVersionRange fillHistoryVer; diff --git a/source/libs/executor/src/executor.c b/source/libs/executor/src/executor.c index 9542a10389..ad79bc87d7 100644 --- a/source/libs/executor/src/executor.c +++ b/source/libs/executor/src/executor.c @@ -870,7 +870,7 @@ int32_t qExtractStreamScanner(qTaskInfo_t tinfo, void** scanner) { } } -int32_t qStreamSourceScanParamForHistoryScan(qTaskInfo_t tinfo, SVersionRange *pVerRange, STimeWindow* pWindow) { +int32_t qStreamSourceScanParamForHistoryScanStep1(qTaskInfo_t tinfo, SVersionRange *pVerRange, STimeWindow* pWindow) { SExecTaskInfo* pTaskInfo = (SExecTaskInfo*)tinfo; ASSERT(pTaskInfo->execModel == OPTR_EXEC_MODEL_STREAM); @@ -879,8 +879,29 @@ int32_t qStreamSourceScanParamForHistoryScan(qTaskInfo_t tinfo, SVersionRange *p pStreamInfo->fillHistoryVer = *pVerRange; pStreamInfo->fillHistoryWindow = *pWindow; pStreamInfo->recoverStep = STREAM_RECOVER_STEP__PREPARE1; + pStreamInfo->recoverStep1Finished = false; + pStreamInfo->recoverStep2Finished = false; - qDebug("%s set param for stream scanner for scan history data, Ver:%" PRId64 " - %" PRId64 ", window:%" PRId64 + qDebug("%s step 1. set param for stream scanner for scan history data, Ver:%" PRId64 " - %" PRId64 ", window:%" PRId64 + " - %" PRId64, + GET_TASKID(pTaskInfo), pStreamInfo->fillHistoryVer.minVer, pStreamInfo->fillHistoryVer.maxVer, pWindow->skey, + pWindow->ekey); + return 0; +} + +int32_t qStreamSourceScanParamForHistoryScanStep2(qTaskInfo_t tinfo, SVersionRange *pVerRange, STimeWindow* pWindow) { + SExecTaskInfo* pTaskInfo = (SExecTaskInfo*)tinfo; + ASSERT(pTaskInfo->execModel == OPTR_EXEC_MODEL_STREAM); + + SStreamTaskInfo* pStreamInfo = &pTaskInfo->streamInfo; + + pStreamInfo->fillHistoryVer = *pVerRange; + pStreamInfo->fillHistoryWindow = *pWindow; + pStreamInfo->recoverStep = STREAM_RECOVER_STEP__PREPARE2; + pStreamInfo->recoverStep1Finished = true; + pStreamInfo->recoverStep2Finished = false; + + qDebug("%s step 2. set param for stream scanner for scan history data, Ver:%" PRId64 " - %" PRId64 ", window:%" PRId64 " - %" PRId64, GET_TASKID(pTaskInfo), pStreamInfo->fillHistoryVer.minVer, pStreamInfo->fillHistoryVer.maxVer, pWindow->skey, pWindow->ekey); @@ -1022,6 +1043,23 @@ bool qStreamRecoverScanFinished(qTaskInfo_t tinfo) { return pTaskInfo->streamInfo.recoverScanFinished; } +bool qStreamRecoverScanStep1Finished(qTaskInfo_t tinfo) { + SExecTaskInfo* pTaskInfo = (SExecTaskInfo*)tinfo; + return pTaskInfo->streamInfo.recoverStep1Finished; +} + +bool qStreamRecoverScanStep2Finished(qTaskInfo_t tinfo) { + SExecTaskInfo* pTaskInfo = (SExecTaskInfo*)tinfo; + return pTaskInfo->streamInfo.recoverStep2Finished; +} + +int32_t qStreamRecoverSetAllStepFinished(qTaskInfo_t tinfo) { + SExecTaskInfo* pTaskInfo = (SExecTaskInfo*)tinfo; + pTaskInfo->streamInfo.recoverStep1Finished = true; + pTaskInfo->streamInfo.recoverStep2Finished = true; + return 0; +} + void* qExtractReaderFromStreamScanner(void* scanner) { SStreamScanInfo* pInfo = scanner; return (void*)pInfo->tqReader; diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index 96a10b9cd1..8793b11ab1 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -179,7 +179,7 @@ int32_t streamScanExec(SStreamTask* pTask, int32_t batchSz) { int32_t batchCnt = 0; while (1) { - if (streamTaskShouldStop(&pTask->status) || streamTaskShouldPause(&pTask->status)) { + if (streamTaskShouldStop(&pTask->status)) { taosArrayDestroyEx(pRes, (FDelete)blockDataFreeRes); return 0; } @@ -195,6 +195,10 @@ int32_t streamScanExec(SStreamTask* pTask, int32_t batchSz) { finished = true; } else { qSetStreamOpOpen(exec); + if (streamTaskShouldPause(&pTask->status)) { + taosArrayDestroyEx(pRes, (FDelete)blockDataFreeRes); + return 0; + } } break; } diff --git a/source/libs/stream/src/streamRecover.c b/source/libs/stream/src/streamRecover.c index 577022bbef..45a8ad0dfa 100644 --- a/source/libs/stream/src/streamRecover.c +++ b/source/libs/stream/src/streamRecover.c @@ -55,7 +55,7 @@ static int32_t doLaunchScanHistoryTask(SStreamTask* pTask) { pRange->minVer, pRange->maxVer); streamSetParamForScanHistoryData(pTask); - streamSetParamForStreamScanner(pTask, pRange, &pTask->dataRange.window); + streamSetParamForStreamScannerStep1(pTask, pRange, &pTask->dataRange.window); int32_t code = streamStartRecoverTask(pTask, 0); return code; @@ -261,8 +261,12 @@ int32_t streamSetStatusNormal(SStreamTask* pTask) { } // source -int32_t streamSetParamForStreamScanner(SStreamTask* pTask, SVersionRange *pVerRange, STimeWindow* pWindow) { - return qStreamSourceScanParamForHistoryScan(pTask->exec.pExecutor, pVerRange, pWindow); +int32_t streamSetParamForStreamScannerStep1(SStreamTask* pTask, SVersionRange *pVerRange, STimeWindow* pWindow) { + return qStreamSourceScanParamForHistoryScanStep1(pTask->exec.pExecutor, pVerRange, pWindow); +} + +int32_t streamSetParamForStreamScannerStep2(SStreamTask* pTask, SVersionRange *pVerRange, STimeWindow* pWindow) { + return qStreamSourceScanParamForHistoryScanStep2(pTask->exec.pExecutor, pVerRange, pWindow); } int32_t streamBuildSourceRecover1Req(SStreamTask* pTask, SStreamScanHistoryReq* pReq, int8_t igUntreated) { @@ -512,6 +516,21 @@ int32_t streamTaskScanHistoryDataComplete(SStreamTask* pTask) { return 0; } +bool streamTaskRecoverScanStep1Finished(SStreamTask* pTask) { + void* exec = pTask->exec.pExecutor; + return qStreamRecoverScanStep1Finished(exec); +} + +bool streamTaskRecoverScanStep2Finished(SStreamTask* pTask) { + void* exec = pTask->exec.pExecutor; + return qStreamRecoverScanStep2Finished(exec); +} + +int32_t streamTaskRecoverSetAllStepFinished(SStreamTask* pTask) { + void* exec = pTask->exec.pExecutor; + return qStreamRecoverSetAllStepFinished(exec); +} + int32_t tEncodeStreamTaskCheckReq(SEncoder* pEncoder, const SStreamTaskCheckReq* pReq) { if (tStartEncode(pEncoder) < 0) return -1; if (tEncodeI64(pEncoder, pReq->reqId) < 0) return -1; From 870d75bf82455dea3e7daef7f6754ffeb8b83462 Mon Sep 17 00:00:00 2001 From: liuyao <54liuyao@163.com> Date: Tue, 20 Jun 2023 11:34:09 +0800 Subject: [PATCH 46/83] fill history pause&resume --- source/dnode/vnode/src/tq/tq.c | 24 +++++++++++------------- source/libs/stream/src/streamExec.c | 16 +++++++++++++++- 2 files changed, 26 insertions(+), 14 deletions(-) diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 36abd581ee..57d5cd8eb0 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -779,9 +779,9 @@ int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t ver) { if (pTask->info.taskLevel == TASK_LEVEL__SOURCE) { SStreamTask* pSateTask = pTask; - // if (pTask->info.fillHistory) { - // pSateTask = *(SStreamTask**)taosHashGet(pTq->pStreamMeta->pTasks, &pTask->streamTaskId.taskId, sizeof(int32_t)); - // } + if (pTask->info.fillHistory) { + pSateTask = *(SStreamTask**)taosHashGet(pTq->pStreamMeta->pTasks, &pTask->streamTaskId.taskId, sizeof(int32_t)); + } pTask->pState = streamStateOpen(pTq->pStreamMeta->path, pSateTask, false, -1, -1); if (pTask->pState == NULL) { return -1; @@ -798,9 +798,9 @@ int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t ver) { qSetTaskId(pTask->exec.pExecutor, pTask->id.taskId, pTask->id.streamId); } else if (pTask->info.taskLevel == TASK_LEVEL__AGG) { SStreamTask* pSateTask = pTask; - // if (pTask->info.fillHistory) { - // pSateTask = *(SStreamTask**)taosHashGet(pTq->pStreamMeta->pTasks, &pTask->streamTaskId.taskId, sizeof(int32_t)); - // } + if (pTask->info.fillHistory) { + pSateTask = *(SStreamTask**)taosHashGet(pTq->pStreamMeta->pTasks, &pTask->streamTaskId.taskId, sizeof(int32_t)); + } pTask->pState = streamStateOpen(pTq->pStreamMeta->path, pSateTask, false, -1, -1); if (pTask->pState == NULL) { return -1; @@ -1134,9 +1134,8 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { pTask->status.taskStatus = TASK_STATUS__DROPPING; tqDebug("s-task:%s set status to be dropping", pTask->id.idStr); // transfer the ownership of executor state - // todo(liuyao) - // streamTaskReleaseState(pTask); - // streamTaskReloadState(pStreamTask); + streamTaskReleaseState(pTask); + streamTaskReloadState(pStreamTask); streamMetaSaveTask(pMeta, pTask); streamMetaSaveTask(pMeta, pStreamTask); @@ -1183,10 +1182,9 @@ int32_t tqProcessTaskTransferStateReq(STQ* pTq, int64_t sversion, char* msg, int return -1; } // transfer the ownership of executor state - // todo(liuyao) - // streamTaskReleaseState(pTask); - // SStreamTask* pStreamTask = streamMetaAcquireTask(pTq->pStreamMeta, pTask->streamTaskId.taskId); - // streamTaskReloadState(pStreamTask); + streamTaskReleaseState(pTask); + SStreamTask* pStreamTask = streamMetaAcquireTask(pTq->pStreamMeta, pTask->streamTaskId.taskId); + streamTaskReloadState(pStreamTask); ASSERT(pTask->streamTaskId.taskId != 0); pTask->status.transferState = true; // persistent data? diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index 8793b11ab1..1af5948268 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -196,7 +196,21 @@ int32_t streamScanExec(SStreamTask* pTask, int32_t batchSz) { } else { qSetStreamOpOpen(exec); if (streamTaskShouldPause(&pTask->status)) { - taosArrayDestroyEx(pRes, (FDelete)blockDataFreeRes); + SStreamDataBlock* qRes = taosAllocateQitem(sizeof(SStreamDataBlock), DEF_QITEM, 0); + if (qRes == NULL) { + taosArrayDestroyEx(pRes, (FDelete)blockDataFreeRes); + terrno = TSDB_CODE_OUT_OF_MEMORY; + return -1; + } + + qRes->type = STREAM_INPUT__DATA_BLOCK; + qRes->blocks = pRes; + code = streamTaskOutputResultBlock(pTask, qRes); + if (code == TSDB_CODE_UTIL_QUEUE_OUT_OF_MEMORY) { + taosArrayDestroyEx(pRes, (FDelete)blockDataFreeRes); + taosFreeQitem(qRes); + return code; + } return 0; } } From b6d4e98e68a8569f98345f8241032546b13888ad Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Tue, 20 Jun 2023 13:47:46 +0800 Subject: [PATCH 47/83] fix restart crash --- source/libs/stream/src/streamBackendRocksdb.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/libs/stream/src/streamBackendRocksdb.c b/source/libs/stream/src/streamBackendRocksdb.c index 151edbdaf0..4646af641f 100644 --- a/source/libs/stream/src/streamBackendRocksdb.c +++ b/source/libs/stream/src/streamBackendRocksdb.c @@ -1026,8 +1026,8 @@ int streamStateGetCfIdx(SStreamState* pState, const char* funcName) { break; } } - SBackendCfWrapper* wrapper = pState->pTdbState->pBackendCfWrapper; if (pState != NULL && idx != -1) { + SBackendCfWrapper* wrapper = pState->pTdbState->pBackendCfWrapper; rocksdb_column_family_handle_t* cf = NULL; taosThreadRwlockRdlock(&wrapper->rwLock); cf = wrapper->pHandle[idx]; From aafbdcb492c721aafb5bd6839e9d4e7e975e30a9 Mon Sep 17 00:00:00 2001 From: liuyao <54liuyao@163.com> Date: Tue, 20 Jun 2023 15:20:04 +0800 Subject: [PATCH 48/83] set task status --- source/dnode/vnode/src/tq/tq.c | 8 ++++++-- source/libs/stream/src/streamTask.c | 6 +++++- 2 files changed, 11 insertions(+), 3 deletions(-) diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 57d5cd8eb0..90790132f1 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1440,7 +1440,9 @@ int32_t tqProcessTaskPauseReq(STQ* pTq, int64_t sversion, char* msg, int32_t msg return code; } SStreamTask* pHistoryTask = streamMetaAcquireTask(pTq->pStreamMeta, pTask->historyTaskId.taskId); - code = tqProcessTaskPauseImpl(pTq->pStreamMeta, pHistoryTask); + if (pHistoryTask) { + code = tqProcessTaskPauseImpl(pTq->pStreamMeta, pHistoryTask); + } return code; } @@ -1486,7 +1488,9 @@ int32_t tqProcessTaskResumeReq(STQ* pTq, int64_t sversion, char* msg, int32_t ms } SStreamTask* pHistoryTask = streamMetaAcquireTask(pTq->pStreamMeta, pTask->historyTaskId.taskId); - code = tqProcessTaskResumeImpl(pTq, pHistoryTask, sversion, pReq->igUntreated); + if (pHistoryTask) { + code = tqProcessTaskResumeImpl(pTq, pHistoryTask, sversion, pReq->igUntreated); + } return code; } diff --git a/source/libs/stream/src/streamTask.c b/source/libs/stream/src/streamTask.c index 06da72188c..d6f23bb44a 100644 --- a/source/libs/stream/src/streamTask.c +++ b/source/libs/stream/src/streamTask.c @@ -42,7 +42,11 @@ SStreamTask* tNewStreamTask(int64_t streamId, int8_t taskLevel, int8_t fillHisto pTask->id.idStr = taosStrdup(buf); pTask->status.schedStatus = TASK_SCHED_STATUS__INACTIVE; - pTask->status.taskStatus = TASK_STATUS__SCAN_HISTORY; + if (fillHistory) { + pTask->status.taskStatus = TASK_STATUS__SCAN_HISTORY; + } else { + pTask->status.taskStatus = TASK_STATUS__NORMAL; + } pTask->inputStatus = TASK_INPUT_STATUS__NORMAL; pTask->outputStatus = TASK_OUTPUT_STATUS__NORMAL; From 74bea443009ad69f9b34c6c8a5e214d2ae88bbd2 Mon Sep 17 00:00:00 2001 From: liuyao <54liuyao@163.com> Date: Tue, 20 Jun 2023 17:30:59 +0800 Subject: [PATCH 49/83] stream op transform --- source/dnode/vnode/src/tq/tq.c | 12 ++++++++++-- source/libs/executor/inc/executorInt.h | 2 ++ source/libs/executor/src/executorInt.c | 14 ++++++++++++++ source/libs/executor/src/filloperator.c | 1 + source/libs/executor/src/groupoperator.c | 1 + source/libs/executor/src/scanoperator.c | 2 +- 6 files changed, 29 insertions(+), 3 deletions(-) diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 90790132f1..092a10b1cf 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -779,8 +779,12 @@ int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t ver) { if (pTask->info.taskLevel == TASK_LEVEL__SOURCE) { SStreamTask* pSateTask = pTask; + SStreamTask task = {0}; if (pTask->info.fillHistory) { - pSateTask = *(SStreamTask**)taosHashGet(pTq->pStreamMeta->pTasks, &pTask->streamTaskId.taskId, sizeof(int32_t)); + task.id = pTask->streamTaskId; + SStreamMeta meta = {0}; + task.pMeta = pTask->pMeta; + pSateTask = &task; } pTask->pState = streamStateOpen(pTq->pStreamMeta->path, pSateTask, false, -1, -1); if (pTask->pState == NULL) { @@ -798,8 +802,12 @@ int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t ver) { qSetTaskId(pTask->exec.pExecutor, pTask->id.taskId, pTask->id.streamId); } else if (pTask->info.taskLevel == TASK_LEVEL__AGG) { SStreamTask* pSateTask = pTask; + SStreamTask task = {0}; if (pTask->info.fillHistory) { - pSateTask = *(SStreamTask**)taosHashGet(pTq->pStreamMeta->pTasks, &pTask->streamTaskId.taskId, sizeof(int32_t)); + task.id = pTask->streamTaskId; + SStreamMeta meta = {0}; + task.pMeta = pTask->pMeta; + pSateTask = &task; } pTask->pState = streamStateOpen(pTq->pStreamMeta->path, pSateTask, false, -1, -1); if (pTask->pState == NULL) { diff --git a/source/libs/executor/inc/executorInt.h b/source/libs/executor/inc/executorInt.h index 07507db836..d2c1812205 100644 --- a/source/libs/executor/inc/executorInt.h +++ b/source/libs/executor/inc/executorInt.h @@ -682,6 +682,8 @@ void doUpdateNumOfRows(SqlFunctionCtx* pCtx, SResultRow* pRow, int32_t numOfExpr void doClearBufferedBlocks(SStreamScanInfo* pInfo); uint64_t calcGroupId(char* pData, int32_t len); +void streamOpReleaseState(struct SOperatorInfo* pOperator); +void streamOpReloadState(struct SOperatorInfo* pOperator); #ifdef __cplusplus } diff --git a/source/libs/executor/src/executorInt.c b/source/libs/executor/src/executorInt.c index 1b23f3ea57..edda6ee03f 100644 --- a/source/libs/executor/src/executorInt.c +++ b/source/libs/executor/src/executorInt.c @@ -1086,3 +1086,17 @@ void qStreamCloseTsdbReader(void* task) { } } } + +void streamOpReleaseState(SOperatorInfo* pOperator) { + SOperatorInfo* downstream = pOperator->pDownstream[0]; + if (downstream->fpSet.releaseStreamStateFn) { + downstream->fpSet.releaseStreamStateFn(downstream); + } +} + +void streamOpReloadState(SOperatorInfo* pOperator) { + SOperatorInfo* downstream = pOperator->pDownstream[0]; + if (downstream->fpSet.reloadStreamStateFn) { + downstream->fpSet.reloadStreamStateFn(downstream); + } +} diff --git a/source/libs/executor/src/filloperator.c b/source/libs/executor/src/filloperator.c index f9e8a32520..bde59013ee 100644 --- a/source/libs/executor/src/filloperator.c +++ b/source/libs/executor/src/filloperator.c @@ -1560,6 +1560,7 @@ SOperatorInfo* createStreamFillOperatorInfo(SOperatorInfo* downstream, SStreamFi pTaskInfo); pOperator->fpSet = createOperatorFpSet(optrDummyOpenFn, doStreamFill, NULL, destroyStreamFillOperatorInfo, optrDefaultBufFn, NULL); + setOperatorStreamStateFn(pOperator, streamOpReleaseState, streamOpReloadState); code = appendDownstream(pOperator, &downstream, 1); if (code != TSDB_CODE_SUCCESS) { diff --git a/source/libs/executor/src/groupoperator.c b/source/libs/executor/src/groupoperator.c index 4f29ada7ef..0a98d86af5 100644 --- a/source/libs/executor/src/groupoperator.c +++ b/source/libs/executor/src/groupoperator.c @@ -1326,6 +1326,7 @@ SOperatorInfo* createStreamPartitionOperatorInfo(SOperatorInfo* downstream, SStr pOperator->exprSupp.pExprInfo = pExprInfo; pOperator->fpSet = createOperatorFpSet(optrDummyOpenFn, doStreamHashPartition, NULL, destroyStreamPartitionOperatorInfo, optrDefaultBufFn, NULL); + setOperatorStreamStateFn(pOperator, streamOpReleaseState, streamOpReloadState); initParDownStream(downstream, &pInfo->partitionSup, &pInfo->scalarSup); code = appendDownstream(pOperator, &downstream, 1); diff --git a/source/libs/executor/src/scanoperator.c b/source/libs/executor/src/scanoperator.c index 817b2a8d5f..0e68b8d0a6 100644 --- a/source/libs/executor/src/scanoperator.c +++ b/source/libs/executor/src/scanoperator.c @@ -2531,12 +2531,12 @@ SOperatorInfo* createStreamScanOperatorInfo(SReadHandle* pHandle, STableScanPhys setOperatorInfo(pOperator, STREAM_SCAN_OP_NAME, QUERY_NODE_PHYSICAL_PLAN_STREAM_SCAN, false, OP_NOT_OPENED, pInfo, pTaskInfo); - setOperatorStreamStateFn(pOperator, streamScanReleaseState, streamScanReloadState); pOperator->exprSupp.numOfExprs = taosArrayGetSize(pInfo->pRes->pDataBlock); __optr_fn_t nextFn = (pTaskInfo->execModel == OPTR_EXEC_MODEL_STREAM) ? doStreamScan : doQueueScan; pOperator->fpSet = createOperatorFpSet(optrDummyOpenFn, nextFn, NULL, destroyStreamScanOperatorInfo, optrDefaultBufFn, NULL); + setOperatorStreamStateFn(pOperator, streamScanReleaseState, streamScanReloadState); return pOperator; From 3cfda2c5794072d2e1c9d199cf03fd9ef6914fb3 Mon Sep 17 00:00:00 2001 From: liuyao <54liuyao@163.com> Date: Tue, 20 Jun 2023 17:44:41 +0800 Subject: [PATCH 50/83] scan op transform --- source/libs/executor/src/scanoperator.c | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/source/libs/executor/src/scanoperator.c b/source/libs/executor/src/scanoperator.c index 0e68b8d0a6..db9c384776 100644 --- a/source/libs/executor/src/scanoperator.c +++ b/source/libs/executor/src/scanoperator.c @@ -2320,6 +2320,9 @@ static void destroyStreamScanOperatorInfo(void* param) { void streamScanReleaseState(SOperatorInfo* pOperator) { SStreamScanInfo* pInfo = pOperator->info; + if (!pInfo->pState) { + return; + } if (!pInfo->pUpdateInfo) { return; } @@ -2331,6 +2334,9 @@ void streamScanReleaseState(SOperatorInfo* pOperator) { void streamScanReloadState(SOperatorInfo* pOperator) { SStreamScanInfo* pInfo = pOperator->info; + if (!pInfo->pState) { + return; + } void* pBuff = NULL; int32_t len = 0; pInfo->stateStore.streamStateGetInfo(pInfo->pState, STREAM_SCAN_OP_STATE_NAME, strlen(STREAM_SCAN_OP_STATE_NAME), &pBuff, &len); From fbed0acace0cc1170a8c090b6cb90b12a36221c3 Mon Sep 17 00:00:00 2001 From: liuyao <54liuyao@163.com> Date: Tue, 20 Jun 2023 19:23:30 +0800 Subject: [PATCH 51/83] free mem --- source/libs/executor/src/scanoperator.c | 2 ++ source/libs/executor/src/timewindowoperator.c | 3 +++ 2 files changed, 5 insertions(+) diff --git a/source/libs/executor/src/scanoperator.c b/source/libs/executor/src/scanoperator.c index db9c384776..2348d67434 100644 --- a/source/libs/executor/src/scanoperator.c +++ b/source/libs/executor/src/scanoperator.c @@ -2330,6 +2330,7 @@ void streamScanReleaseState(SOperatorInfo* pOperator) { void* pBuff = taosMemoryCalloc(1, len); pInfo->stateStore.updateInfoSerialize(pBuff, len, pInfo->pUpdateInfo); pInfo->stateStore.streamStateSaveInfo(pInfo->pState, STREAM_SCAN_OP_STATE_NAME, strlen(STREAM_SCAN_OP_STATE_NAME), pBuff, len); + taosMemoryFree(pBuff); } void streamScanReloadState(SOperatorInfo* pOperator) { @@ -2342,6 +2343,7 @@ void streamScanReloadState(SOperatorInfo* pOperator) { pInfo->stateStore.streamStateGetInfo(pInfo->pState, STREAM_SCAN_OP_STATE_NAME, strlen(STREAM_SCAN_OP_STATE_NAME), &pBuff, &len); SUpdateInfo* pUpInfo = pInfo->stateStore.updateInfoInit(0, TSDB_TIME_PRECISION_MILLI, 0); int32_t code = pInfo->stateStore.updateInfoDeserialize(pBuff, len, pUpInfo); + taosMemoryFree(pBuff); if (code == TSDB_CODE_SUCCESS && pInfo->pUpdateInfo) { if (pInfo->pUpdateInfo->minTS < 0) { pInfo->stateStore.updateInfoDestroy(pInfo->pUpdateInfo); diff --git a/source/libs/executor/src/timewindowoperator.c b/source/libs/executor/src/timewindowoperator.c index aa364375ae..67bd73b262 100644 --- a/source/libs/executor/src/timewindowoperator.c +++ b/source/libs/executor/src/timewindowoperator.c @@ -2747,6 +2747,7 @@ void streamIntervalReloadState(SOperatorInfo* pOperator) { int32_t code = pInfo->statestore.streamStateGetInfo(pInfo->pState, STREAM_INTERVAL_OP_STATE_NAME, strlen(STREAM_INTERVAL_OP_STATE_NAME), &pBuf, &size); TSKEY ts = *(TSKEY*)pBuf; + taosMemoryFree(pBuf); pInfo->statestore.streamStateReloadInfo(pInfo->pState, ts); } SOperatorInfo* downstream = pOperator->pDownstream[0]; @@ -3677,6 +3678,7 @@ void streamSessionReloadState(SOperatorInfo* pOperator) { setSessionOutputBuf(pAggSup, pSeKeyBuf[i].win.skey, pSeKeyBuf[i].win.ekey, pSeKeyBuf[i].groupId, &winInfo); compactSessionWindow(pOperator, &winInfo, pInfo->pStUpdated, pInfo->pStDeleted); } + taosMemoryFree(pBuf); SOperatorInfo* downstream = pOperator->pDownstream[0]; if (downstream->fpSet.reloadStreamStateFn) { @@ -4288,6 +4290,7 @@ void streamStateReloadState(SOperatorInfo* pOperator) { compactStateWindow(pOperator, &curInfo.winInfo, &nextInfo.winInfo, pInfo->pStUpdated, pInfo->pStDeleted); } } + taosMemoryFree(pBuf); SOperatorInfo* downstream = pOperator->pDownstream[0]; if (downstream->fpSet.reloadStreamStateFn) { From 1bed2b1e404cf718e548a3a1cf5aee7137f4d111 Mon Sep 17 00:00:00 2001 From: liuyao <54liuyao@163.com> Date: Wed, 21 Jun 2023 09:13:46 +0800 Subject: [PATCH 52/83] free mem --- source/libs/executor/src/scanoperator.c | 2 ++ 1 file changed, 2 insertions(+) diff --git a/source/libs/executor/src/scanoperator.c b/source/libs/executor/src/scanoperator.c index 2348d67434..e787e73434 100644 --- a/source/libs/executor/src/scanoperator.c +++ b/source/libs/executor/src/scanoperator.c @@ -2364,6 +2364,8 @@ void streamScanReloadState(SOperatorInfo* pOperator) { pUpInfo->pMap = NULL; pInfo->stateStore.updateInfoDestroy(pUpInfo); } + } else { + pInfo->stateStore.updateInfoDestroy(pUpInfo); } } From d23c80b12fc38edd0623479047c4c4b6df8d482d Mon Sep 17 00:00:00 2001 From: liuyao <54liuyao@163.com> Date: Wed, 21 Jun 2023 10:24:34 +0800 Subject: [PATCH 53/83] free mem --- source/libs/executor/src/scanoperator.c | 4 ++-- source/libs/stream/test/tstreamUpdateTest.cpp | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/source/libs/executor/src/scanoperator.c b/source/libs/executor/src/scanoperator.c index e787e73434..1059677108 100644 --- a/source/libs/executor/src/scanoperator.c +++ b/source/libs/executor/src/scanoperator.c @@ -1764,7 +1764,7 @@ void streamScanOperatorDecode(void* pBuff, int32_t len, SStreamScanInfo* pInfo) return; } - void* pUpInfo = pInfo->stateStore.updateInfoInit(0, TSDB_TIME_PRECISION_MILLI, 0); + void* pUpInfo = taosMemoryCalloc(1, sizeof(SUpdateInfo)); int32_t code = pInfo->stateStore.updateInfoDeserialize(pBuff, len, pUpInfo); if (code == TSDB_CODE_SUCCESS) { pInfo->pUpdateInfo = pUpInfo; @@ -2341,7 +2341,7 @@ void streamScanReloadState(SOperatorInfo* pOperator) { void* pBuff = NULL; int32_t len = 0; pInfo->stateStore.streamStateGetInfo(pInfo->pState, STREAM_SCAN_OP_STATE_NAME, strlen(STREAM_SCAN_OP_STATE_NAME), &pBuff, &len); - SUpdateInfo* pUpInfo = pInfo->stateStore.updateInfoInit(0, TSDB_TIME_PRECISION_MILLI, 0); + SUpdateInfo* pUpInfo = taosMemoryCalloc(1, sizeof(SUpdateInfo)); int32_t code = pInfo->stateStore.updateInfoDeserialize(pBuff, len, pUpInfo); taosMemoryFree(pBuff); if (code == TSDB_CODE_SUCCESS && pInfo->pUpdateInfo) { diff --git a/source/libs/stream/test/tstreamUpdateTest.cpp b/source/libs/stream/test/tstreamUpdateTest.cpp index 18c60aff28..0e84d6b8bd 100644 --- a/source/libs/stream/test/tstreamUpdateTest.cpp +++ b/source/libs/stream/test/tstreamUpdateTest.cpp @@ -158,7 +158,7 @@ TEST(TD_STREAM_UPDATE_TEST, update) { // void *buf = taosMemoryCalloc(1, bufLen); // int32_t resSize = updateInfoSerialize(buf, bufLen, pSU7); - // SUpdateInfo *pSU6 = updateInfoInit(0, TSDB_TIME_PRECISION_MILLI, 0); + // SUpdateInfo *pSU6 = taosMemoryCalloc(1, sizeof(SUpdateInfo)); // int32_t desSize = updateInfoDeserialize(buf, bufLen, pSU6); // GTEST_ASSERT_EQ(desSize, 0); From 13d9136ad617eb3a877856aa885122065ec4ef8e Mon Sep 17 00:00:00 2001 From: liuyao <54liuyao@163.com> Date: Wed, 21 Jun 2023 11:40:08 +0800 Subject: [PATCH 54/83] add ci --- .../tsim/stream/fillHistoryTransform.sim | 405 ++++++++++++++++++ 1 file changed, 405 insertions(+) create mode 100644 tests/script/tsim/stream/fillHistoryTransform.sim diff --git a/tests/script/tsim/stream/fillHistoryTransform.sim b/tests/script/tsim/stream/fillHistoryTransform.sim new file mode 100644 index 0000000000..fe58b76b78 --- /dev/null +++ b/tests/script/tsim/stream/fillHistoryTransform.sim @@ -0,0 +1,405 @@ +system sh/stop_dnodes.sh +system sh/deploy.sh -n dnode1 -i 1 +system sh/exec.sh -n dnode1 -s start +sleep 50 +sql connect + +print =============== create database +sql create database test vgroups 1; +sql select * from information_schema.ins_databases +if $rows != 3 then + return -1 +endi + +print $data00 $data01 $data02 + +sql use test; + +print =====step1 + +sql create table t1(ts timestamp, a int, b int , c int, d double); + +sql insert into t1 values(1648791213000,10,2,3,1.0); + +sql create stream stream0 trigger at_once fill_history 1 IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt as select _wstart, sum(a) from t1 interval(10s); + +$loop_count = 0 +loop00: + +sleep 1000 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +sql select * from streamt; + +if $rows != 1 then + print ======$rows + print data00,data01, data02 + print data10,data11, data12 + print data20,data21, data22 + goto loop00 +endi + +if $data01 != 10 then + print =====data01=$data01 + goto loop00 +endi + +sql insert into t1 values(1648791213000,1,2,3,1.0); + +$loop_count = 0 +loop0: + +sleep 1000 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +sql select * from streamt; + +if $rows != 1 then + print ======$rows + print data00,data01, data02 + print data10,data11, data12 + print data20,data21, data22 + goto loop0 +endi + +if $data01 != 1 then + print =====data01=$data01 + goto loop0 +endi + +sql insert into t1 values(1648791213001,2,2,3,1.0); + +$loop_count = 0 +loop1: + +sleep 1000 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +sql select * from streamt; + +if $rows != 1 then + print ======$rows + print data00,data01, data02 + print data10,data11, data12 + print data20,data21, data22 + goto loop1 +endi + +if $data01 != 3 then + print ======$data01 + goto loop1 +endi + + +sql insert into t1 values(1648791223001,3,2,3,1.0); + +sql insert into t1 values(1648791223002,4,2,3,1.0); + +$loop_count = 0 +loop2: + +sleep 1000 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +sql select * from streamt; + +if $rows != 2 then + print ======$rows + print data00,data01, data02 + print data10,data11, data12 + print data20,data21, data22 + goto loop2 +endi + +if $data01 != 3 then + print ======$data01 + goto loop2 +endi + +if $data11 != 7 then + print ======$data01 + goto loop2 +endi + +print =====step1 over + +print =====step2 + +sql create database test1 vgroups 4; + +sql use test1; + +sql create stable st(ts timestamp,a int,b int,c int,d double) tags(ta int,tb int,tc int); +sql create table t1 using st tags(1,1,1); +sql create table t2 using st tags(2,2,2); + +sql insert into t1 values(1648791213000,10,2,3,1.0); + +sql create stream stream1 trigger at_once fill_history 1 IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt1 as select _wstart, sum(a) from st interval(10s); + +$loop_count = 0 +loop00: + +sleep 1000 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +sql select * from streamt1; + +if $rows != 1 then + print ======$rows + print data00,data01, data02 + print data10,data11, data12 + print data20,data21, data22 + goto loop00 +endi + +if $data01 != 10 then + print =====data01=$data01 + goto loop00 +endi + +sql insert into t1 values(1648791213000,1,2,3,1.0); + +$loop_count = 0 +loop0: + +sleep 1000 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +sql select * from streamt1; + +if $rows != 1 then + print ======$rows + print data00,data01, data02 + print data10,data11, data12 + print data20,data21, data22 + goto loop0 +endi + +if $data01 != 1 then + print =====data01=$data01 + goto loop0 +endi + +sql insert into t1 values(1648791213001,2,2,3,1.0); + +$loop_count = 0 +loop1: + +sleep 1000 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +sql select * from streamt1; + +if $rows != 1 then + print ======$rows + print data00,data01, data02 + print data10,data11, data12 + print data20,data21, data22 + goto loop1 +endi + +if $data01 != 3 then + print ======$data01 + goto loop1 +endi + + +sql insert into t1 values(1648791223001,3,2,3,1.0); + +sql insert into t1 values(1648791223002,4,2,3,1.0); + +$loop_count = 0 +loop2: + +sleep 1000 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +sql select * from streamt1; + +if $rows != 2 then + print ======$rows + print data00,data01, data02 + print data10,data11, data12 + print data20,data21, data22 + goto loop2 +endi + +if $data01 != 3 then + print ======$data01 + goto loop2 +endi + +if $data11 != 7 then + print ======$data01 + goto loop2 +endi + +print =====step2 over + +print =====step3 + +sql create database test2 vgroups 4; + +sql use test2; + +sql create stable st(ts timestamp,a int,b int,c int,d double) tags(ta int,tb int,tc int); +sql create table t1 using st tags(1,1,1); +sql create table t2 using st tags(2,2,2); + +sql insert into t1 values(1648791213000,10,2,3,1.0); + +sql create stream stream2 trigger at_once fill_history 1 IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamt2 as select _wstart, sum(a) from st partition by ta interval(10s); + +$loop_count = 0 +loop00: + +sleep 1000 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +sql select * from streamt2; + +if $rows != 1 then + print ======$rows + print data00,data01, data02 + print data10,data11, data12 + print data20,data21, data22 + goto loop00 +endi + +if $data01 != 10 then + print =====data01=$data01 + goto loop00 +endi + +sql insert into t1 values(1648791213000,1,2,3,1.0); + +$loop_count = 0 +loop0: + +sleep 1000 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +sql select * from streamt2; + +if $rows != 1 then + print ======$rows + print data00,data01, data02 + print data10,data11, data12 + print data20,data21, data22 + goto loop0 +endi + +if $data01 != 1 then + print =====data01=$data01 + goto loop0 +endi + +sql insert into t1 values(1648791213001,2,2,3,1.0); + +$loop_count = 0 +loop1: + +sleep 1000 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +sql select * from streamt2; + +if $rows != 1 then + print ======$rows + print data00,data01, data02 + print data10,data11, data12 + print data20,data21, data22 + goto loop1 +endi + +if $data01 != 3 then + print ======$data01 + goto loop1 +endi + + +sql insert into t1 values(1648791223001,3,2,3,1.0); + +sql insert into t1 values(1648791223002,4,2,3,1.0); + +$loop_count = 0 +loop2: + +sleep 1000 + +$loop_count = $loop_count + 1 +if $loop_count == 20 then + return -1 +endi + +sql select * from streamt2; + +if $rows != 2 then + print ======$rows + print data00,data01, data02 + print data10,data11, data12 + print data20,data21, data22 + goto loop2 +endi + +if $data01 != 3 then + print ======$data01 + goto loop2 +endi + +if $data11 != 7 then + print ======$data01 + goto loop2 +endi + +print =====step3 over + +print =====over + + +system sh/stop_dnodes.sh From ea7b8e91d322b90277a4d9109247e8ee5a8f7933 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 21 Jun 2023 12:03:26 +0800 Subject: [PATCH 55/83] fix(stream): fix memory leak. --- source/dnode/vnode/src/tq/tq.c | 2 ++ 1 file changed, 2 insertions(+) diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 092a10b1cf..8a44b223e2 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1183,12 +1183,14 @@ int32_t tqProcessTaskTransferStateReq(STQ* pTq, int64_t sversion, char* msg, int SDecoder decoder; tDecoderInit(&decoder, (uint8_t*)msg, msgLen); int32_t code = tDecodeStreamRecoverFinishReq(&decoder, &req); + tDecoderClear(&decoder); SStreamTask* pTask = streamMetaAcquireTask(pTq->pStreamMeta, req.taskId); if (pTask == NULL) { tqError("failed to find task:0x%x", req.taskId); return -1; } + // transfer the ownership of executor state streamTaskReleaseState(pTask); SStreamTask* pStreamTask = streamMetaAcquireTask(pTq->pStreamMeta, pTask->streamTaskId.taskId); From 0322fdc1fc9a54e92b2933e52de570a6337d471b Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sat, 24 Jun 2023 23:43:43 +0800 Subject: [PATCH 56/83] fix(stream): fix memory leak. --- source/common/src/tdatablock.c | 3 +- source/libs/stream/src/streamDispatch.c | 47 +++++++++++++++++-------- 2 files changed, 34 insertions(+), 16 deletions(-) diff --git a/source/common/src/tdatablock.c b/source/common/src/tdatablock.c index cc49716644..dc6e0d2cb7 100644 --- a/source/common/src/tdatablock.c +++ b/source/common/src/tdatablock.c @@ -1185,8 +1185,7 @@ void blockDataFreeRes(SSDataBlock* pBlock) { colDataDestroy(pColInfoData); } - taosArrayDestroy(pBlock->pDataBlock); - pBlock->pDataBlock = NULL; + pBlock->pDataBlock = taosArrayDestroy(pBlock->pDataBlock); taosMemoryFreeClear(pBlock->pBlockAgg); memset(&pBlock->info, 0, sizeof(SDataBlockInfo)); } diff --git a/source/libs/stream/src/streamDispatch.c b/source/libs/stream/src/streamDispatch.c index 7ce85b4e65..c42320ad13 100644 --- a/source/libs/stream/src/streamDispatch.c +++ b/source/libs/stream/src/streamDispatch.c @@ -16,7 +16,9 @@ #include "ttimer.h" #include "streamInc.h" -#define MAX_BLOCK_NAME_NUM 1024 +#define MAX_BLOCK_NAME_NUM 1024 +#define DISPATCH_RETRY_INTERVAL_MS 300 +#define MAX_CONTINUE_RETRY_COUNT 5 typedef struct SBlockName { uint32_t hashValue; @@ -324,7 +326,10 @@ int32_t doSendDispatchMsg(SStreamTask* pTask, const SStreamDispatchReq* pReq, in // serialize int32_t tlen; tEncodeSize(tEncodeStreamDispatchReq, pReq, tlen, code); - if (code < 0) goto FAIL; + if (code < 0) { + goto FAIL; + } + code = -1; buf = rpcMallocCont(sizeof(SMsgHead) + tlen); if (buf == NULL) { @@ -346,13 +351,13 @@ int32_t doSendDispatchMsg(SStreamTask* pTask, const SStreamDispatchReq* pReq, in msg.msgType = pTask->msgInfo.msgType; qDebug("s-task:%s dispatch msg to taskId:0x%x vgId:%d data msg", pTask->id.idStr, pReq->taskId, vgId); - tmsgSendReq(pEpSet, &msg); - - code = 0; - return 0; + return tmsgSendReq(pEpSet, &msg); FAIL: - if (buf) rpcFreeCont(buf); + if (buf) { + rpcFreeCont(buf); + } + return code; } @@ -403,13 +408,16 @@ int32_t streamSearchAndAddBlock(SStreamTask* pTask, SStreamDispatchReq* pReqs, S for (j = 0; j < vgSz; j++) { SVgroupInfo* pVgInfo = taosArrayGet(vgInfo, j); ASSERT(pVgInfo->vgId > 0); + if (hashValue >= pVgInfo->hashBegin && hashValue <= pVgInfo->hashEnd) { if (streamAddBlockIntoDispatchMsg(pDataBlock, &pReqs[j]) < 0) { return -1; } + if (pReqs[j].blockNum == 0) { atomic_add_fetch_32(&pTask->shuffleDispatcher.waitingRspCnt, 1); } + pReqs[j].blockNum++; found = true; break; @@ -510,7 +518,8 @@ int32_t streamDispatchAllBlocks(SStreamTask* pTask, const SStreamDataBlock* pDat qDebug("s-task:%s (child taskId:%d) shuffle-dispatch blocks:%d to vgId:%d", pTask->id.idStr, pTask->info.selfChildId, pReqs[i].blockNum, pVgInfo->vgId); - if (doSendDispatchMsg(pTask, &pReqs[i], pVgInfo->vgId, &pVgInfo->epSet) < 0) { + code = doSendDispatchMsg(pTask, &pReqs[i], pVgInfo->vgId, &pVgInfo->epSet); + if (code < 0) { goto FAIL_SHUFFLE_DISPATCH; } } @@ -536,7 +545,9 @@ static void doRetryDispatchData(void* param, void* tmrId) { int32_t code = streamDispatchAllBlocks(pTask, pTask->msgInfo.pData); if (code != TSDB_CODE_SUCCESS) { - streamRetryDispatchStreamBlock(pTask, 300); + qDebug("s-task:%s reset the waitRspCnt to be 0 before launch retry dispatch", pTask->id.idStr); + atomic_store_32(&pTask->shuffleDispatcher.waitingRspCnt, 0); + streamRetryDispatchStreamBlock(pTask, DISPATCH_RETRY_INTERVAL_MS); } } @@ -584,12 +595,20 @@ int32_t streamDispatchStreamBlock(SStreamTask* pTask) { } qDebug("s-task:%s failed to dispatch msg to downstream, code:%s, output status:%d, retry cnt:%d", pTask->id.idStr, - tstrerror(code), pTask->outputStatus, retryCount); + tstrerror(terrno), pTask->outputStatus, retryCount); - if (++retryCount > 5) { // add to timer to retry - qDebug("s-task:%s failed to dispatch msg to downstream for %d times, code:%s, retry in %dms", pTask->id.idStr, - retryCount, tstrerror(code), 300); - streamRetryDispatchStreamBlock(pTask, 300); + // todo deal with only partially success dispatch case + atomic_store_32(&pTask->shuffleDispatcher.waitingRspCnt, 0); + if (terrno == TSDB_CODE_APP_IS_STOPPING) { // in case of this error, do not retry anymore + destroyStreamDataBlock(pTask->msgInfo.pData); + pTask->msgInfo.pData = NULL; + return code; + } + + if (++retryCount > MAX_CONTINUE_RETRY_COUNT) { // add to timer to retry + qDebug("s-task:%s failed to dispatch msg to downstream for %d times, code:%s, add timer to retry in %dms", pTask->id.idStr, + retryCount, tstrerror(terrno), DISPATCH_RETRY_INTERVAL_MS); + streamRetryDispatchStreamBlock(pTask, DISPATCH_RETRY_INTERVAL_MS); break; } } From 7c23630fe46806286ef726ea242e45c56d4b6df7 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sun, 25 Jun 2023 09:26:37 +0800 Subject: [PATCH 57/83] fix(stream): set correct stream error code. --- source/dnode/vnode/src/tq/tq.c | 1 + 1 file changed, 1 insertion(+) diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 8a44b223e2..fc3b9a0be4 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -952,6 +952,7 @@ int32_t tqProcessStreamTaskCheckRsp(STQ* pTq, int64_t sversion, SRpcMsg* pMsg) { if (pTask == NULL) { tqError("tq failed to locate the stream task:0x%x (vgId:%d), it may have been destroyed", rsp.upstreamTaskId, pTq->pStreamMeta->vgId); + terrno = TSDB_CODE_STREAM_TASK_NOT_EXIST; return -1; } From 6ac179501aabe368ef56e5c4d798a25d452092cc Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sun, 25 Jun 2023 10:46:21 +0800 Subject: [PATCH 58/83] fix: fix syntax error. --- source/dnode/vnode/src/meta/metaQuery.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/dnode/vnode/src/meta/metaQuery.c b/source/dnode/vnode/src/meta/metaQuery.c index 9550b44c6d..c26bb45c2b 100644 --- a/source/dnode/vnode/src/meta/metaQuery.c +++ b/source/dnode/vnode/src/meta/metaQuery.c @@ -215,7 +215,7 @@ int metaReadNext(SMetaReader *pReader) { int metaGetTableTtlByUid(void *meta, uint64_t uid, int64_t *ttlDays) { int code = -1; SMetaReader mr = {0}; - metaReaderInit(&mr, (SMeta *)meta, 0); + metaReaderDoInit(&mr, (SMeta *)meta, 0); code = metaReaderGetTableEntryByUid(&mr, uid); if (code < 0) { goto _exit; From e9d3198e052388c6c8bf9c73a3a54d8ce6f9f524 Mon Sep 17 00:00:00 2001 From: liuyao <54liuyao@163.com> Date: Sun, 25 Jun 2023 10:57:10 +0800 Subject: [PATCH 59/83] mem leak --- source/dnode/mnode/impl/src/mndDef.c | 2 +- source/dnode/vnode/src/tq/tqRestore.c | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/source/dnode/mnode/impl/src/mndDef.c b/source/dnode/mnode/impl/src/mndDef.c index 81f434a184..a81ab30455 100644 --- a/source/dnode/mnode/impl/src/mndDef.c +++ b/source/dnode/mnode/impl/src/mndDef.c @@ -175,7 +175,7 @@ void tFreeStreamObj(SStreamObj *pStream) { taosMemoryFree(pStream->ast); taosMemoryFree(pStream->physicalPlan); - if (pStream->outputSchema.nCols) { + if (pStream->outputSchema.nCols || pStream->outputSchema.pSchema) { taosMemoryFree(pStream->outputSchema.pSchema); } diff --git a/source/dnode/vnode/src/tq/tqRestore.c b/source/dnode/vnode/src/tq/tqRestore.c index 5c0e735a81..833f59fe8d 100644 --- a/source/dnode/vnode/src/tq/tqRestore.c +++ b/source/dnode/vnode/src/tq/tqRestore.c @@ -83,6 +83,7 @@ int32_t tqStreamTasksStatusCheck(STQ* pTq) { streamTaskCheckDownstreamTasks(pTask); streamMetaReleaseTask(pMeta, pTask); } + taosArrayDestroy(pTaskList); return 0; } From 7e7feeffdfcf66fea19af4759e5215673d57f1be Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sun, 25 Jun 2023 14:38:31 +0800 Subject: [PATCH 60/83] fix: fix memory leak. --- source/dnode/vnode/src/tsdb/tsdbUtil.c | 32 ++++++++++++-------------- source/libs/executor/src/executil.c | 2 +- 2 files changed, 16 insertions(+), 18 deletions(-) diff --git a/source/dnode/vnode/src/tsdb/tsdbUtil.c b/source/dnode/vnode/src/tsdb/tsdbUtil.c index 556ec33526..84671197d8 100644 --- a/source/dnode/vnode/src/tsdb/tsdbUtil.c +++ b/source/dnode/vnode/src/tsdb/tsdbUtil.c @@ -528,25 +528,25 @@ void tsdbFidKeyRange(int32_t fid, int32_t minutes, int8_t precision, TSKEY *minK *maxKey = *minKey + tsTickPerMin[precision] * minutes - 1; } -int32_t tsdbFidLevel(int32_t fid, STsdbKeepCfg *pKeepCfg, int64_t now) { +int32_t tsdbFidLevel(int32_t fid, STsdbKeepCfg *pKeepCfg, int64_t nowSec) { int32_t aFid[3]; TSKEY key; if (pKeepCfg->precision == TSDB_TIME_PRECISION_MILLI) { - now = now * 1000; + nowSec = nowSec * 1000; } else if (pKeepCfg->precision == TSDB_TIME_PRECISION_MICRO) { - now = now * 1000000l; + nowSec = nowSec * 1000000l; } else if (pKeepCfg->precision == TSDB_TIME_PRECISION_NANO) { - now = now * 1000000000l; + nowSec = nowSec * 1000000000l; } else { ASSERT(0); } - key = now - pKeepCfg->keep0 * tsTickPerMin[pKeepCfg->precision]; + key = nowSec - pKeepCfg->keep0 * tsTickPerMin[pKeepCfg->precision]; aFid[0] = tsdbKeyFid(key, pKeepCfg->days, pKeepCfg->precision); - key = now - pKeepCfg->keep1 * tsTickPerMin[pKeepCfg->precision]; + key = nowSec - pKeepCfg->keep1 * tsTickPerMin[pKeepCfg->precision]; aFid[1] = tsdbKeyFid(key, pKeepCfg->days, pKeepCfg->precision); - key = now - pKeepCfg->keep2 * tsTickPerMin[pKeepCfg->precision]; + key = nowSec - pKeepCfg->keep2 * tsTickPerMin[pKeepCfg->precision]; aFid[2] = tsdbKeyFid(key, pKeepCfg->days, pKeepCfg->precision); if (fid >= aFid[0]) { @@ -640,7 +640,7 @@ SColVal *tsdbRowIterNext(STSDBRowIter *pIter) { int32_t tsdbRowMergerAdd(SRowMerger *pMerger, TSDBROW *pRow, STSchema *pTSchema) { int32_t code = 0; TSDBKEY key = TSDBROW_KEY(pRow); - SColVal * pColVal = &(SColVal){0}; + SColVal *pColVal = &(SColVal){0}; STColumn *pTColumn; int32_t iCol, jCol = 1; @@ -764,7 +764,7 @@ int32_t tsdbRowMergerAdd(SRowMerger *pMerger, TSDBROW *pRow, STSchema *pTSchema) } } -int32_t tsdbRowMergerInit(SRowMerger* pMerger, STSchema *pSchema) { +int32_t tsdbRowMergerInit(SRowMerger *pMerger, STSchema *pSchema) { pMerger->pTSchema = pSchema; pMerger->pArray = taosArrayInit(pSchema->numOfCols, sizeof(SColVal)); if (pMerger->pArray == NULL) { @@ -774,7 +774,7 @@ int32_t tsdbRowMergerInit(SRowMerger* pMerger, STSchema *pSchema) { } } -void tsdbRowMergerClear(SRowMerger* pMerger) { +void tsdbRowMergerClear(SRowMerger *pMerger) { for (int32_t iCol = 1; iCol < pMerger->pTSchema->numOfCols; iCol++) { SColVal *pTColVal = taosArrayGet(pMerger->pArray, iCol); if (IS_VAR_DATA_TYPE(pTColVal->type)) { @@ -785,7 +785,7 @@ void tsdbRowMergerClear(SRowMerger* pMerger) { taosArrayClear(pMerger->pArray); } -void tsdbRowMergerCleanup(SRowMerger* pMerger) { +void tsdbRowMergerCleanup(SRowMerger *pMerger) { int32_t numOfCols = taosArrayGetSize(pMerger->pArray); for (int32_t iCol = 1; iCol < numOfCols; iCol++) { SColVal *pTColVal = taosArrayGet(pMerger->pArray, iCol); @@ -1041,8 +1041,6 @@ int32_t tsdbBuildDeleteSkyline2(SArray *aDelData, int32_t sidx, int32_t eidx, SA // SBlockData ====================================================== int32_t tBlockDataCreate(SBlockData *pBlockData) { - int32_t code = 0; - pBlockData->suid = 0; pBlockData->uid = 0; pBlockData->nRow = 0; @@ -1051,7 +1049,7 @@ int32_t tBlockDataCreate(SBlockData *pBlockData) { pBlockData->aTSKEY = NULL; pBlockData->nColData = 0; pBlockData->aColData = NULL; - return code; + return 0; } void tBlockDataDestroy(SBlockData *pBlockData) { @@ -1107,8 +1105,8 @@ int32_t tBlockDataInit(SBlockData *pBlockData, TABLEID *pId, STSchema *pTSchema, int32_t iColumn = 1; STColumn *pTColumn = &pTSchema->columns[iColumn]; for (int32_t iCid = 0; iCid < nCid; iCid++) { - - // aCid array (from taos client catalog) contains columns that does not exist in the pTSchema. the pTSchema is newer + // aCid array (from taos client catalog) contains columns that does not exist in the pTSchema. the pTSchema is + // newer if (pTColumn == NULL) { continue; } @@ -1239,7 +1237,7 @@ int32_t tBlockDataAppendRow(SBlockData *pBlockData, TSDBROW *pRow, STSchema *pTS _exit: return code; } -static int32_t tBlockDataUpdateRow(SBlockData *pBlockData, TSDBROW *pRow, STSchema *pTSchema) { +int32_t tBlockDataUpdateRow(SBlockData *pBlockData, TSDBROW *pRow, STSchema *pTSchema) { int32_t code = 0; // version diff --git a/source/libs/executor/src/executil.c b/source/libs/executor/src/executil.c index 331a2fa7ab..fa0c1782c4 100644 --- a/source/libs/executor/src/executil.c +++ b/source/libs/executor/src/executil.c @@ -1128,7 +1128,7 @@ int32_t getTableList(void* pVnode, SScanPhysiNode* pScanNode, SNode* pTagCond, S memcpy(pPayload + sizeof(int32_t), taosArrayGet(pUidList, 0), numOfTables * sizeof(uint64_t)); } -// metaUidFilterCachePut(metaHandle, pScanNode->suid, context.digest, tListLen(context.digest), pPayload, size, 1); + pStorageAPI->metaFn.putCachedTableList(pVnode, pScanNode->suid, context.digest, tListLen(context.digest), pPayload, size, 1); digest[0] = 1; memcpy(digest + 1, context.digest, tListLen(context.digest)); } From e236bf818c80a901ccdfa76b2a0daac660ce9d6c Mon Sep 17 00:00:00 2001 From: liuyao <54liuyao@163.com> Date: Sun, 25 Jun 2023 17:47:10 +0800 Subject: [PATCH 61/83] set fill history info --- include/libs/executor/executor.h | 2 ++ source/dnode/snode/src/snode.c | 2 +- source/dnode/vnode/src/tq/tq.c | 4 ++-- source/libs/executor/inc/operator.h | 6 ++--- source/libs/executor/src/executorInt.c | 6 +++-- source/libs/executor/src/operator.c | 8 +++---- source/libs/executor/src/timewindowoperator.c | 23 ++++++++++++------- source/libs/stream/src/streamState.c | 1 + 8 files changed, 32 insertions(+), 20 deletions(-) diff --git a/include/libs/executor/executor.h b/include/libs/executor/executor.h index faaea12b1a..5fd9a4a8c1 100644 --- a/include/libs/executor/executor.h +++ b/include/libs/executor/executor.h @@ -55,6 +55,8 @@ typedef struct { void* pStateBackend; struct SStorageAPI api; + + int8_t fillHistory; } SReadHandle; // in queue mode, data streams are seperated by msg diff --git a/source/dnode/snode/src/snode.c b/source/dnode/snode/src/snode.c index ad57bdbdc6..7146ba3f5b 100644 --- a/source/dnode/snode/src/snode.c +++ b/source/dnode/snode/src/snode.c @@ -84,7 +84,7 @@ int32_t sndExpandTask(SSnode *pSnode, SStreamTask *pTask, int64_t ver) { } int32_t numOfChildEp = taosArrayGetSize(pTask->pUpstreamEpInfoList); - SReadHandle handle = { .vnode = NULL, .numOfVgroups = numOfChildEp, .pStateBackend = pTask->pState }; + SReadHandle handle = { .vnode = NULL, .numOfVgroups = numOfChildEp, .pStateBackend = pTask->pState, .fillHistory = pTask->info.fillHistory }; initStreamStateAPI(&handle.api); pTask->exec.pExecutor = qCreateStreamExecTaskInfo(pTask->exec.qmsg, &handle, 0); diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index fc3b9a0be4..891fb8dd20 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -791,7 +791,7 @@ int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t ver) { return -1; } - SReadHandle handle = {.vnode = pTq->pVnode, .initTqReader = 1, .pStateBackend = pTask->pState}; + SReadHandle handle = {.vnode = pTq->pVnode, .initTqReader = 1, .pStateBackend = pTask->pState, .fillHistory = pTask->info.fillHistory}; initStorageAPI(&handle.api); pTask->exec.pExecutor = qCreateStreamExecTaskInfo(pTask->exec.qmsg, &handle, vgId); @@ -815,7 +815,7 @@ int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t ver) { } int32_t numOfVgroups = (int32_t)taosArrayGetSize(pTask->pUpstreamEpInfoList); - SReadHandle handle = {.vnode = NULL, .numOfVgroups = numOfVgroups, .pStateBackend = pTask->pState}; + SReadHandle handle = {.vnode = NULL, .numOfVgroups = numOfVgroups, .pStateBackend = pTask->pState, .fillHistory = pTask->info.fillHistory}; initStorageAPI(&handle.api); pTask->exec.pExecutor = qCreateStreamExecTaskInfo(pTask->exec.qmsg, &handle, vgId); diff --git a/source/libs/executor/inc/operator.h b/source/libs/executor/inc/operator.h index 4042dc45c7..e6c3405d7f 100644 --- a/source/libs/executor/inc/operator.h +++ b/source/libs/executor/inc/operator.h @@ -129,13 +129,13 @@ SOperatorInfo* createTimeSliceOperatorInfo(SOperatorInfo* downstream, SPhysiNode SOperatorInfo* createMergeJoinOperatorInfo(SOperatorInfo** pDownstream, int32_t numOfDownstream, SSortMergeJoinPhysiNode* pJoinNode, SExecTaskInfo* pTaskInfo); -SOperatorInfo* createStreamSessionAggOperatorInfo(SOperatorInfo* downstream, SPhysiNode* pPhyNode, SExecTaskInfo* pTaskInfo); +SOperatorInfo* createStreamSessionAggOperatorInfo(SOperatorInfo* downstream, SPhysiNode* pPhyNode, SExecTaskInfo* pTaskInfo, SReadHandle* pHandle); -SOperatorInfo* createStreamFinalSessionAggOperatorInfo(SOperatorInfo* downstream, SPhysiNode* pPhyNode, SExecTaskInfo* pTaskInfo, int32_t numOfChild); +SOperatorInfo* createStreamFinalSessionAggOperatorInfo(SOperatorInfo* downstream, SPhysiNode* pPhyNode, SExecTaskInfo* pTaskInfo, int32_t numOfChild, SReadHandle* pHandle); SOperatorInfo* createStreamIntervalOperatorInfo(SOperatorInfo* downstream, SPhysiNode* pPhyNode, SExecTaskInfo* pTaskInfo); -SOperatorInfo* createStreamStateAggOperatorInfo(SOperatorInfo* downstream, SPhysiNode* pPhyNode, SExecTaskInfo* pTaskInfo); +SOperatorInfo* createStreamStateAggOperatorInfo(SOperatorInfo* downstream, SPhysiNode* pPhyNode, SExecTaskInfo* pTaskInfo, SReadHandle* pHandle); SOperatorInfo* createStreamFillOperatorInfo(SOperatorInfo* downstream, SStreamFillPhysiNode* pPhyFillNode, SExecTaskInfo* pTaskInfo); diff --git a/source/libs/executor/src/executorInt.c b/source/libs/executor/src/executorInt.c index edda6ee03f..472e7fed0a 100644 --- a/source/libs/executor/src/executorInt.c +++ b/source/libs/executor/src/executorInt.c @@ -977,9 +977,11 @@ int32_t buildSessionResultDataBlock(SOperatorInfo* pOperator, void* pState, SSDa int32_t size = 0; void* pVal = NULL; int32_t code = pAPI->stateStore.streamStateSessionGet(pState, pKey, &pVal, &size); - ASSERT(code == 0); + // ASSERT(code == 0); if (code == -1) { - // coverity scan + // for history + qWarn("===stream===not found session result key:%" PRId64 ", ekey:%" PRId64 ", groupId:%" PRIu64, pKey->win.skey, + pKey->win.ekey, pKey->groupId); pGroupResInfo->index += 1; continue; } diff --git a/source/libs/executor/src/operator.c b/source/libs/executor/src/operator.c index 7486a4ad5f..2db5ea2f1e 100644 --- a/source/libs/executor/src/operator.c +++ b/source/libs/executor/src/operator.c @@ -492,13 +492,13 @@ SOperatorInfo* createOperator(SPhysiNode* pPhyNode, SExecTaskInfo* pTaskInfo, SR SSessionWinodwPhysiNode* pSessionNode = (SSessionWinodwPhysiNode*)pPhyNode; pOptr = createSessionAggOperatorInfo(ops[0], pSessionNode, pTaskInfo); } else if (QUERY_NODE_PHYSICAL_PLAN_STREAM_SESSION == type) { - pOptr = createStreamSessionAggOperatorInfo(ops[0], pPhyNode, pTaskInfo); + pOptr = createStreamSessionAggOperatorInfo(ops[0], pPhyNode, pTaskInfo, pHandle); } else if (QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_SESSION == type) { int32_t children = 0; - pOptr = createStreamFinalSessionAggOperatorInfo(ops[0], pPhyNode, pTaskInfo, children); + pOptr = createStreamFinalSessionAggOperatorInfo(ops[0], pPhyNode, pTaskInfo, children, pHandle); } else if (QUERY_NODE_PHYSICAL_PLAN_STREAM_FINAL_SESSION == type) { int32_t children = pHandle->numOfVgroups; - pOptr = createStreamFinalSessionAggOperatorInfo(ops[0], pPhyNode, pTaskInfo, children); + pOptr = createStreamFinalSessionAggOperatorInfo(ops[0], pPhyNode, pTaskInfo, children, pHandle); } else if (QUERY_NODE_PHYSICAL_PLAN_PARTITION == type) { pOptr = createPartitionOperatorInfo(ops[0], (SPartitionPhysiNode*)pPhyNode, pTaskInfo); } else if (QUERY_NODE_PHYSICAL_PLAN_STREAM_PARTITION == type) { @@ -507,7 +507,7 @@ SOperatorInfo* createOperator(SPhysiNode* pPhyNode, SExecTaskInfo* pTaskInfo, SR SStateWinodwPhysiNode* pStateNode = (SStateWinodwPhysiNode*)pPhyNode; pOptr = createStatewindowOperatorInfo(ops[0], pStateNode, pTaskInfo); } else if (QUERY_NODE_PHYSICAL_PLAN_STREAM_STATE == type) { - pOptr = createStreamStateAggOperatorInfo(ops[0], pPhyNode, pTaskInfo); + pOptr = createStreamStateAggOperatorInfo(ops[0], pPhyNode, pTaskInfo, pHandle); } else if (QUERY_NODE_PHYSICAL_PLAN_MERGE_JOIN == type) { pOptr = createMergeJoinOperatorInfo(ops, size, (SSortMergeJoinPhysiNode*)pPhyNode, pTaskInfo); } else if (QUERY_NODE_PHYSICAL_PLAN_FILL == type) { diff --git a/source/libs/executor/src/timewindowoperator.c b/source/libs/executor/src/timewindowoperator.c index a8a0184e4e..aac0660328 100644 --- a/source/libs/executor/src/timewindowoperator.c +++ b/source/libs/executor/src/timewindowoperator.c @@ -3506,6 +3506,9 @@ void doBuildSessionResult(SOperatorInfo* pOperator, void* pState, SGroupResInfo* // clear the existed group id pBlock->info.id.groupId = 0; buildSessionResultDataBlock(pOperator, pState, pBlock, &pOperator->exprSupp, pGroupResInfo); + if (pBlock->info.rows == 0) { + cleanupGroupResInfo(pGroupResInfo); + } } void getMaxTsWins(const SArray* pAllWins, SArray* pMaxWins) { int32_t size = taosArrayGetSize(pAllWins); @@ -3605,7 +3608,7 @@ static SSDataBlock* doStreamSessionAgg(SOperatorInfo* pOperator) { // if chIndex + 1 - size > 0, add new child for (int32_t i = 0; i < chIndex + 1 - size; i++) { SOperatorInfo* pChildOp = - createStreamFinalSessionAggOperatorInfo(NULL, pInfo->pPhyNode, pOperator->pTaskInfo, 0); + createStreamFinalSessionAggOperatorInfo(NULL, pInfo->pPhyNode, pOperator->pTaskInfo, 0, NULL); if (!pChildOp) { T_LONG_JMP(pOperator->pTaskInfo->env, TSDB_CODE_OUT_OF_MEMORY); } @@ -3693,7 +3696,7 @@ void streamSessionReloadState(SOperatorInfo* pOperator) { } SOperatorInfo* createStreamSessionAggOperatorInfo(SOperatorInfo* downstream, SPhysiNode* pPhyNode, - SExecTaskInfo* pTaskInfo) { + SExecTaskInfo* pTaskInfo, SReadHandle* pHandle) { SSessionWinodwPhysiNode* pSessionNode = (SSessionWinodwPhysiNode*)pPhyNode; int32_t numOfCols = 0; int32_t code = TSDB_CODE_OUT_OF_MEMORY; @@ -3760,7 +3763,9 @@ SOperatorInfo* createStreamSessionAggOperatorInfo(SOperatorInfo* downstream, SPh if (!pInfo->historyWins) { goto _error; } - pInfo->isHistoryOp = false; + if (pHandle) { + pInfo->isHistoryOp = pHandle->fillHistory; + } setOperatorInfo(pOperator, "StreamSessionWindowAggOperator", QUERY_NODE_PHYSICAL_PLAN_STREAM_SESSION, true, OP_NOT_OPENED, pInfo, pTaskInfo); @@ -3904,9 +3909,9 @@ static SSDataBlock* doStreamSessionSemiAgg(SOperatorInfo* pOperator) { } SOperatorInfo* createStreamFinalSessionAggOperatorInfo(SOperatorInfo* downstream, SPhysiNode* pPhyNode, - SExecTaskInfo* pTaskInfo, int32_t numOfChild) { + SExecTaskInfo* pTaskInfo, int32_t numOfChild, SReadHandle* pHandle) { int32_t code = TSDB_CODE_OUT_OF_MEMORY; - SOperatorInfo* pOperator = createStreamSessionAggOperatorInfo(downstream, pPhyNode, pTaskInfo); + SOperatorInfo* pOperator = createStreamSessionAggOperatorInfo(downstream, pPhyNode, pTaskInfo, pHandle); if (pOperator == NULL) { goto _error; } @@ -3930,7 +3935,7 @@ SOperatorInfo* createStreamFinalSessionAggOperatorInfo(SOperatorInfo* downstream if (numOfChild > 0) { pInfo->pChildren = taosArrayInit(numOfChild, sizeof(void*)); for (int32_t i = 0; i < numOfChild; i++) { - SOperatorInfo* pChildOp = createStreamFinalSessionAggOperatorInfo(NULL, pPhyNode, pTaskInfo, 0); + SOperatorInfo* pChildOp = createStreamFinalSessionAggOperatorInfo(NULL, pPhyNode, pTaskInfo, 0, NULL); if (pChildOp == NULL) { goto _error; } @@ -4305,7 +4310,7 @@ void streamStateReloadState(SOperatorInfo* pOperator) { } SOperatorInfo* createStreamStateAggOperatorInfo(SOperatorInfo* downstream, SPhysiNode* pPhyNode, - SExecTaskInfo* pTaskInfo) { + SExecTaskInfo* pTaskInfo, SReadHandle* pHandle) { SStreamStateWinodwPhysiNode* pStateNode = (SStreamStateWinodwPhysiNode*)pPhyNode; int32_t tsSlotId = ((SColumnNode*)pStateNode->window.pTspk)->slotId; SColumnNode* pColNode = (SColumnNode*)((STargetNode*)pStateNode->pStateKey)->pExpr; @@ -4369,7 +4374,9 @@ SOperatorInfo* createStreamStateAggOperatorInfo(SOperatorInfo* downstream, SPhys if (!pInfo->historyWins) { goto _error; } - pInfo->isHistoryOp = false; + if (pHandle) { + pInfo->isHistoryOp = pHandle->fillHistory; + } setOperatorInfo(pOperator, "StreamStateAggOperator", QUERY_NODE_PHYSICAL_PLAN_STREAM_STATE, true, OP_NOT_OPENED, pInfo, pTaskInfo); diff --git a/source/libs/stream/src/streamState.c b/source/libs/stream/src/streamState.c index 216e3b8a11..9873e7b4c8 100644 --- a/source/libs/stream/src/streamState.c +++ b/source/libs/stream/src/streamState.c @@ -762,6 +762,7 @@ int32_t streamStateSessionGet(SStreamState* pState, SSessionKey* key, void** pVa int32_t streamStateSessionDel(SStreamState* pState, const SSessionKey* key) { #ifdef USE_ROCKSDB + qDebug("===stream===delete skey:%" PRId64 ", ekey:%" PRId64 ", groupId:%" PRIu64, key->win.skey,key->win.ekey, key->groupId); return streamStateSessionDel_rocksdb(pState, key); #else SStateSessionKey sKey = {.key = *key, .opNum = pState->number}; From 000325f7169ce02599a0063ad3535ac94c064076 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sun, 25 Jun 2023 19:30:21 +0800 Subject: [PATCH 62/83] fix(stream): fix memory leak. --- source/dnode/mnode/impl/src/mndScheduler.c | 4 +++- source/dnode/mnode/impl/src/mndStream.c | 2 -- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/source/dnode/mnode/impl/src/mndScheduler.c b/source/dnode/mnode/impl/src/mndScheduler.c index e69f874f41..c2e5716c23 100644 --- a/source/dnode/mnode/impl/src/mndScheduler.c +++ b/source/dnode/mnode/impl/src/mndScheduler.c @@ -546,14 +546,16 @@ static int32_t addSourceTasksForMultiLevelStream(SMnode* pMnode, SQueryPlan* pPl if (pStream->conf.fillHistory) { code = doAddSourceTask(pHSourceTaskList, pStream->conf.fillHistory, pStream->hTaskUid, pHDownstreamTask, pMnode, plan, pVgroup); - sdbRelease(pSdb, pVgroup); if (code != TSDB_CODE_SUCCESS) { + sdbRelease(pSdb, pVgroup); return code; } setHTasksId(pSourceTaskList, pHSourceTaskList); } + + sdbRelease(pSdb, pVgroup); } return TSDB_CODE_SUCCESS; diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index e78411096f..433f3fe272 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -1001,8 +1001,6 @@ static int32_t mndProcessStreamDoCheckpoint(SRpcMsg *pReq) { static int32_t mndProcessDropStreamReq(SRpcMsg *pReq) { SMnode *pMnode = pReq->info.node; SStreamObj *pStream = NULL; - /*SDbObj *pDb = NULL;*/ - /*SUserObj *pUser = NULL;*/ SMDropStreamReq dropReq = {0}; if (tDeserializeSMDropStreamReq(pReq->pCont, pReq->contLen, &dropReq) < 0) { From 402c091df04933328d5fda98990921475042bf33 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sun, 25 Jun 2023 19:41:44 +0800 Subject: [PATCH 63/83] fix(stream): fix sma error. --- include/libs/stream/tstream.h | 4 +- source/dnode/vnode/src/tq/tq.c | 39 +++++++++++- source/libs/stream/src/streamMeta.c | 41 +++++++++++- source/libs/stream/src/streamRecover.c | 62 ++++++++++++++++--- tests/system-test/0-others/backquote_check.py | 6 +- 5 files changed, 133 insertions(+), 19 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 78acea1cd9..925bfa24c6 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -267,11 +267,11 @@ typedef struct SCheckpointInfo { typedef struct SStreamStatus { int8_t taskStatus; - int8_t checkDownstream; + int8_t checkDownstream; // downstream tasks are all ready now, if this flag is set int8_t schedStatus; int8_t keepTaskStatus; bool transferState; - TdThreadMutex lock; + int8_t timerActive; // timer is active } SStreamStatus; typedef struct SHistDataRange { diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 891fb8dd20..f3e0bcaa0c 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -168,16 +168,53 @@ void tqNotifyClose(STQ* pTq) { } SStreamTask* pTask = *(SStreamTask**)pIter; - tqDebug("vgId:%d s-task:%s set dropping flag", pTq->pStreamMeta->vgId, pTask->id.idStr); + tqDebug("vgId:%d s-task:%s set closing flag", pTq->pStreamMeta->vgId, pTask->id.idStr); pTask->status.taskStatus = TASK_STATUS__STOP; int64_t st = taosGetTimestampMs(); qKillTask(pTask->exec.pExecutor, TSDB_CODE_SUCCESS); + int64_t el = taosGetTimestampMs() - st; tqDebug("vgId:%d s-task:%s is closed in %" PRId64 " ms", pTq->pStreamMeta->vgId, pTask->id.idStr, el); } taosWUnLockLatch(&pTq->pStreamMeta->lock); + + tqDebug("vgId:%d start to check all tasks", pTq->pStreamMeta->vgId); + + int64_t st = taosGetTimestampMs(); + while(1) { + taosMsleep(1000); + + bool inTimer = false; + + taosWLockLatch(&pTq->pStreamMeta->lock); + pIter = NULL; + + while(1) { + pIter = taosHashIterate(pTq->pStreamMeta->pTasks, pIter); + if (pIter == NULL) { + break; + } + + SStreamTask* pTask = *(SStreamTask**)pIter; + if (pTask->status.timerActive == 1) { + inTimer = true; + } + } + + taosWUnLockLatch(&pTq->pStreamMeta->lock); + + if (inTimer) { + tqDebug("vgId:%d some tasks in timer, wait for 1sec and recheck", pTq->pStreamMeta->vgId); + } else { + break; + } + } + + int64_t el = taosGetTimestampMs() - st; + tqDebug("vgId:%d all stream tasks are not in timer, continue close, elapsed time:%"PRId64" ms", pTq->pStreamMeta->vgId, el); + } } diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 725f918f39..593f84cb8a 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -280,18 +280,53 @@ void streamMetaReleaseTask(SStreamMeta* pMeta, SStreamTask* pTask) { } void streamMetaRemoveTask(SStreamMeta* pMeta, int32_t taskId) { - taosWLockLatch(&pMeta->lock); + SStreamTask* pTask = NULL; + // pre-delete operation + taosWLockLatch(&pMeta->lock); SStreamTask** ppTask = (SStreamTask**)taosHashGet(pMeta->pTasks, &taskId, sizeof(int32_t)); if (ppTask) { - SStreamTask* pTask = *ppTask; + pTask = *ppTask; + atomic_store_8(&pTask->status.taskStatus, TASK_STATUS__DROPPING); + } else { + qDebug("vgId:%d failed to find the task:0x%x, it may be dropped already", pMeta->vgId, taskId); + taosWUnLockLatch(&pMeta->lock); + return; + } + taosWUnLockLatch(&pMeta->lock); + qDebug("s-task:0x%x set task status:%s", taskId, streamGetTaskStatusStr(TASK_STATUS__DROPPING)); + + while(1) { + taosRLockLatch(&pMeta->lock); + ppTask = (SStreamTask**)taosHashGet(pMeta->pTasks, &taskId, sizeof(int32_t)); + + if (ppTask) { + if ((*ppTask)->status.timerActive == 0) { + taosRUnLockLatch(&pMeta->lock); + break; + } + + taosMsleep(10); + qDebug("s-task:%s wait for quit from timer", (*ppTask)->id.idStr); + taosRUnLockLatch(&pMeta->lock); + } else { + taosRUnLockLatch(&pMeta->lock); + break; + } + } + + // let's do delete of stream task + taosWLockLatch(&pMeta->lock); + ppTask = (SStreamTask**)taosHashGet(pMeta->pTasks, &taskId, sizeof(int32_t)); + if (ppTask) { taosHashRemove(pMeta->pTasks, &taskId, sizeof(int32_t)); tdbTbDelete(pMeta->pTaskDb, &taskId, sizeof(int32_t), pMeta->txn); atomic_store_8(&pTask->status.taskStatus, TASK_STATUS__DROPPING); - int32_t num = taosArrayGetSize(pMeta->pTaskList); + ASSERT(pTask->status.timerActive == 0); + int32_t num = taosArrayGetSize(pMeta->pTaskList); qDebug("s-task:%s set the drop task flag, remain running s-task:%d", pTask->id.idStr, num - 1); for (int32_t i = 0; i < num; ++i) { int32_t* pTaskId = taosArrayGet(pMeta->pTaskList, i); diff --git a/source/libs/stream/src/streamRecover.c b/source/libs/stream/src/streamRecover.c index 45a8ad0dfa..a033556353 100644 --- a/source/libs/stream/src/streamRecover.c +++ b/source/libs/stream/src/streamRecover.c @@ -446,20 +446,54 @@ static void doCheckDownstreamStatus(SStreamTask* pTask, SStreamTask* pHTask) { streamTaskCheckDownstreamTasks(pHTask); } +typedef struct SStreamTaskRetryInfo { + SStreamMeta* pMeta; + int32_t taskId; +} SStreamTaskRetryInfo; + static void tryLaunchHistoryTask(void* param, void* tmrId) { - SStreamTask* pTask = param; + SStreamTaskRetryInfo* pInfo = param; + SStreamMeta* pMeta = pInfo->pMeta; - SStreamMeta* pMeta = pTask->pMeta; - SStreamTask** pHTask = taosHashGet(pMeta->pTasks, &pTask->historyTaskId.taskId, sizeof(pTask->historyTaskId.taskId)); - if (pHTask == NULL) { - qWarn("s-task:%s vgId:%d failed to launch history task:0x%x, since it is not built yet", pTask->id.idStr, - pMeta->vgId, pTask->historyTaskId.taskId); + qDebug("s-task:0x%x in timer to launch history task", pInfo->taskId); - taosTmrReset(tryLaunchHistoryTask, 100, pTask, streamEnv.timer, &pTask->timer); - return; + taosWLockLatch(&pMeta->lock); + SStreamTask** ppTask = (SStreamTask**)taosHashGet(pMeta->pTasks, &pInfo->taskId, sizeof(int32_t)); + if (ppTask) { + ASSERT((*ppTask)->status.timerActive == 1); + if (streamTaskShouldStop(&(*ppTask)->status)) { + qDebug("s-task:%s status:%s quit timer task", (*ppTask)->id.idStr, + streamGetTaskStatusStr((*ppTask)->status.taskStatus)); + (*ppTask)->status.timerActive = 0; + taosWUnLockLatch(&pMeta->lock); + return; + } } + taosWUnLockLatch(&pMeta->lock); - doCheckDownstreamStatus(pTask, *pHTask); + SStreamTask* pTask = streamMetaAcquireTask(pMeta, pInfo->taskId); + if (pTask != NULL) { + ASSERT(pTask->status.timerActive == 1); + + // abort the timer if intend to stop task + SStreamTask* pHTask = streamMetaAcquireTask(pMeta, pTask->historyTaskId.taskId); + if (pHTask == NULL && pTask->status.taskStatus == TASK_STATUS__NORMAL) { + qWarn("s-task:%s vgId:%d failed to launch history task:0x%x, since it may not be built or have been destroyed", + pTask->id.idStr, pMeta->vgId, pTask->historyTaskId.taskId); + + taosTmrReset(tryLaunchHistoryTask, 100, pInfo, streamEnv.timer, &pTask->timer); + return; + } + + doCheckDownstreamStatus(pTask, pHTask); + + // not in timer anymore + pTask->status.timerActive = 0; + streamMetaReleaseTask(pMeta, pHTask); + streamMetaReleaseTask(pMeta, pTask); + } else { + qError("s-task:0x%x failed to load task", pInfo->taskId); + } } // todo fix the bug: 2. race condition @@ -474,9 +508,16 @@ int32_t streamCheckHistoryTaskDownstrem(SStreamTask* pTask) { pMeta->vgId, pTask->historyTaskId.taskId); if (pTask->timer == NULL) { - pTask->timer = taosTmrStart(tryLaunchHistoryTask, 100, pTask, streamEnv.timer); + SStreamTaskRetryInfo* pInfo = taosMemoryCalloc(1, sizeof(SStreamTaskRetryInfo)); + pInfo->taskId = pTask->id.taskId; + pInfo->pMeta = pTask->pMeta; + + pTask->timer = taosTmrStart(tryLaunchHistoryTask, 100, pInfo, streamEnv.timer); if (pTask->timer == NULL) { // todo failed to create timer + } else { + pTask->status.timerActive = 1; // timer is active + qDebug("s-task:%s set time active flag", pTask->id.idStr); } } @@ -602,6 +643,7 @@ int32_t tDecodeStreamRecoverFinishReq(SDecoder* pDecoder, SStreamRecoverFinishRe return 0; } +// todo handle race condition, this task may be destroyed void streamPrepareNdoCheckDownstream(SStreamTask* pTask) { if (pTask->info.fillHistory) { qDebug("s-task:%s fill history task, wait for being launched", pTask->id.idStr); diff --git a/tests/system-test/0-others/backquote_check.py b/tests/system-test/0-others/backquote_check.py index be8590f913..3076d246d1 100644 --- a/tests/system-test/0-others/backquote_check.py +++ b/tests/system-test/0-others/backquote_check.py @@ -22,7 +22,7 @@ class TDTestCase: def init(self, conn, logSql, replicaVar=1): self.replicaVar = int(replicaVar) tdLog.debug("start to execute %s" % __file__) - tdSql.init(conn.cursor()) + tdSql.init(conn.cursor(), True) self.dbname = 'db' self.setsql = TDSetSql() self.stbname = 'stb' @@ -76,8 +76,8 @@ class TDTestCase: tdSql.execute(f'drop database {self.dbname}') def run(self): - self.topic_name_check() - self.db_name_check() + # self.topic_name_check() + # self.db_name_check() self.stream_name_check() def stop(self): tdSql.close() From f9be16bef41431f2c382f7354082ec4b6123c4ba Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sun, 25 Jun 2023 19:42:10 +0800 Subject: [PATCH 64/83] fix: update test case. --- tests/system-test/0-others/backquote_check.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/system-test/0-others/backquote_check.py b/tests/system-test/0-others/backquote_check.py index 3076d246d1..7c91fd9e8c 100644 --- a/tests/system-test/0-others/backquote_check.py +++ b/tests/system-test/0-others/backquote_check.py @@ -76,8 +76,8 @@ class TDTestCase: tdSql.execute(f'drop database {self.dbname}') def run(self): - # self.topic_name_check() - # self.db_name_check() + self.topic_name_check() + self.db_name_check() self.stream_name_check() def stop(self): tdSql.close() From daafe240ed5f71cd82c6614e0c5128adba19ae3d Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 26 Jun 2023 14:18:47 +0800 Subject: [PATCH 65/83] fix(stream): fix memory leak. --- source/dnode/mnode/impl/inc/mndScheduler.h | 2 +- source/dnode/mnode/impl/src/mndScheduler.c | 53 ++++++++++++---------- source/dnode/mnode/impl/src/mndSma.c | 2 +- source/dnode/mnode/impl/src/mndStream.c | 2 +- source/libs/stream/src/streamExec.c | 20 +++++--- source/libs/stream/src/streamRecover.c | 25 ++++++---- 6 files changed, 62 insertions(+), 42 deletions(-) diff --git a/source/dnode/mnode/impl/inc/mndScheduler.h b/source/dnode/mnode/impl/inc/mndScheduler.h index 23085c53ee..14517a99d3 100644 --- a/source/dnode/mnode/impl/inc/mndScheduler.h +++ b/source/dnode/mnode/impl/inc/mndScheduler.h @@ -30,7 +30,7 @@ int32_t mndSchedInitSubEp(SMnode* pMnode, const SMqTopicObj* pTopic, SMqSubscrib int32_t mndConvertRsmaTask(char** pDst, int32_t* pDstLen, const char* ast, int64_t uid, int8_t triggerType, int64_t watermark, int64_t deleteMark); -int32_t mndScheduleStream(SMnode* pMnode, SStreamObj* pStream); +int32_t mndScheduleStream(SMnode* pMnode, SStreamObj* pStream, int64_t nextWindowSkey); #ifdef __cplusplus } diff --git a/source/dnode/mnode/impl/src/mndScheduler.c b/source/dnode/mnode/impl/src/mndScheduler.c index c2e5716c23..36ed013549 100644 --- a/source/dnode/mnode/impl/src/mndScheduler.c +++ b/source/dnode/mnode/impl/src/mndScheduler.c @@ -250,12 +250,11 @@ static int32_t addSourceStreamTask(SMnode* pMnode, SVgObj* pVgroup, SArray* pTas } // todo set the correct ts, which should be last key of queried table. - pTask->dataRange.window.skey = INT64_MIN; - pTask->dataRange.window.ekey = 1685959190000; // taosGetTimestampMs(); - // pTask->dataRange.window.ekey = firstWindowSkey - 1;//taosGetTimestampMs(); + STimeWindow* pWindow = &pTask->dataRange.window; - mDebug("add source task 0x%x window:%" PRId64 " - %" PRId64, pTask->id.taskId, pTask->dataRange.window.skey, - pTask->dataRange.window.ekey); + pWindow->skey = INT64_MIN; + pWindow->ekey = firstWindowSkey - 1; + mDebug("add source task 0x%x window:%" PRId64 " - %" PRId64, pTask->id.taskId, pWindow->skey, pWindow->ekey); // sink or dispatch if (hasExtraSink) { @@ -334,16 +333,16 @@ static void setHTasksId(SArray* pTaskList, const SArray* pHTaskList) { } static int32_t addSourceTasksForOneLevelStream(SMnode* pMnode, const SQueryPlan* pPlan, SStreamObj* pStream, - bool hasExtraSink, int64_t lastTs) { + bool hasExtraSink, int64_t nextWindowSkey) { // create exec stream task, since only one level, the exec task is also the source task SArray* pTaskList = addNewTaskList(pStream->tasks); + SSdb* pSdb = pMnode->pSdb; SArray* pHTaskList = NULL; if (pStream->conf.fillHistory) { pHTaskList = addNewTaskList(pStream->pHTasksList); } - SSdb* pSdb = pMnode->pSdb; SNodeListNode* inner = (SNodeListNode*)nodesListGetNode(pPlan->pSubplans, 0); if (LIST_LENGTH(inner->pNodeList) != 1) { terrno = TSDB_CODE_QRY_INVALID_INPUT; @@ -372,7 +371,7 @@ static int32_t addSourceTasksForOneLevelStream(SMnode* pMnode, const SQueryPlan* // new stream task SArray** pSinkTaskList = taosArrayGet(pStream->tasks, SINK_NODE_LEVEL); int32_t code = addSourceStreamTask(pMnode, pVgroup, pTaskList, *pSinkTaskList, pStream, plan, pStream->uid, 0, - hasExtraSink, lastTs); + hasExtraSink, nextWindowSkey); if (code != TSDB_CODE_SUCCESS) { sdbRelease(pSdb, pVgroup); return -1; @@ -381,8 +380,7 @@ static int32_t addSourceTasksForOneLevelStream(SMnode* pMnode, const SQueryPlan* if (pStream->conf.fillHistory) { SArray** pHSinkTaskList = taosArrayGet(pStream->pHTasksList, SINK_NODE_LEVEL); code = addSourceStreamTask(pMnode, pVgroup, pHTaskList, *pHSinkTaskList, pStream, plan, pStream->hTaskUid, - pStream->conf.fillHistory, hasExtraSink, lastTs); - setHTasksId(pTaskList, pHTaskList); + 1, hasExtraSink, nextWindowSkey); } sdbRelease(pSdb, pVgroup); @@ -391,11 +389,15 @@ static int32_t addSourceTasksForOneLevelStream(SMnode* pMnode, const SQueryPlan* } } + if (pStream->conf.fillHistory) { + setHTasksId(pTaskList, pHTaskList); + } + return TSDB_CODE_SUCCESS; } static int32_t doAddSourceTask(SArray* pTaskList, int8_t fillHistory, int64_t uid, SStreamTask* pDownstreamTask, - SMnode* pMnode, SSubplan* pPlan, SVgObj* pVgroup) { + SMnode* pMnode, SSubplan* pPlan, SVgObj* pVgroup, int64_t nextWindowSkey) { SStreamTask* pTask = tNewStreamTask(uid, TASK_LEVEL__SOURCE, fillHistory, 0, pTaskList); if (pTask == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; @@ -403,11 +405,12 @@ static int32_t doAddSourceTask(SArray* pTaskList, int8_t fillHistory, int64_t ui } // todo set the correct ts, which should be last key of queried table. - pTask->dataRange.window.skey = INT64_MIN; - pTask->dataRange.window.ekey = 1685959190000; // taosGetTimestampMs(); + STimeWindow* pWindow = &pTask->dataRange.window; + pWindow->skey = INT64_MIN; + pWindow->ekey = nextWindowSkey - 1; mDebug("s-task:0x%x level:%d set time window:%" PRId64 " - %" PRId64, pTask->id.taskId, pTask->info.taskLevel, - pTask->dataRange.window.skey, pTask->dataRange.window.ekey); + pWindow->skey, pWindow->ekey); // all the source tasks dispatch result to a single agg node. setFixedDownstreamEpInfo(pTask, pDownstreamTask); @@ -507,7 +510,7 @@ static int32_t addAggTask(SStreamObj* pStream, SMnode* pMnode, SQueryPlan* pPlan } static int32_t addSourceTasksForMultiLevelStream(SMnode* pMnode, SQueryPlan* pPlan, SStreamObj* pStream, - SStreamTask* pDownstreamTask, SStreamTask* pHDownstreamTask) { + SStreamTask* pDownstreamTask, SStreamTask* pHDownstreamTask, int64_t nextWindowSkey) { SArray* pSourceTaskList = addNewTaskList(pStream->tasks); SArray* pHSourceTaskList = NULL; @@ -536,7 +539,8 @@ static int32_t addSourceTasksForMultiLevelStream(SMnode* pMnode, SQueryPlan* pPl continue; } - int32_t code = doAddSourceTask(pSourceTaskList, 0, pStream->uid, pDownstreamTask, pMnode, plan, pVgroup); + int32_t code = + doAddSourceTask(pSourceTaskList, 0, pStream->uid, pDownstreamTask, pMnode, plan, pVgroup, nextWindowSkey); if (code != TSDB_CODE_SUCCESS) { sdbRelease(pSdb, pVgroup); terrno = code; @@ -544,9 +548,8 @@ static int32_t addSourceTasksForMultiLevelStream(SMnode* pMnode, SQueryPlan* pPl } if (pStream->conf.fillHistory) { - code = doAddSourceTask(pHSourceTaskList, pStream->conf.fillHistory, pStream->hTaskUid, pHDownstreamTask, pMnode, - plan, pVgroup); - + code = doAddSourceTask(pHSourceTaskList, 1, pStream->hTaskUid, pHDownstreamTask, pMnode, plan, pVgroup, + nextWindowSkey); if (code != TSDB_CODE_SUCCESS) { sdbRelease(pSdb, pVgroup); return code; @@ -581,7 +584,7 @@ static int32_t addSinkTasks(SArray* pTasksList, SMnode* pMnode, SStreamObj* pStr return TSDB_CODE_SUCCESS; } -static int32_t doScheduleStream(SStreamObj* pStream, SMnode* pMnode, SQueryPlan* pPlan, int64_t lastTs) { +static int32_t doScheduleStream(SStreamObj* pStream, SMnode* pMnode, SQueryPlan* pPlan, int64_t nextWindowSkey) { SSdb* pSdb = pMnode->pSdb; int32_t numOfPlanLevel = LIST_LENGTH(pPlan->pSubplans); @@ -612,7 +615,7 @@ static int32_t doScheduleStream(SStreamObj* pStream, SMnode* pMnode, SQueryPlan* // check for fill history if (pStream->conf.fillHistory) { SArray* pHSinkTaskList = NULL; - code = addSinkTasks(pStream->pHTasksList, pMnode, pStream, &pHSinkTaskList, pStream->conf.fillHistory); + code = addSinkTasks(pStream->pHTasksList, pMnode, pStream, &pHSinkTaskList, 1); if (code != TSDB_CODE_SUCCESS) { return code; } @@ -633,22 +636,22 @@ static int32_t doScheduleStream(SStreamObj* pStream, SMnode* pMnode, SQueryPlan* } // source level - return addSourceTasksForMultiLevelStream(pMnode, pPlan, pStream, pAggTask, pHAggTask); + return addSourceTasksForMultiLevelStream(pMnode, pPlan, pStream, pAggTask, pHAggTask, nextWindowSkey); } else if (numOfPlanLevel == 1) { - return addSourceTasksForOneLevelStream(pMnode, pPlan, pStream, hasExtraSink, lastTs); + return addSourceTasksForOneLevelStream(pMnode, pPlan, pStream, hasExtraSink, nextWindowSkey); } return 0; } -int32_t mndScheduleStream(SMnode* pMnode, SStreamObj* pStream) { +int32_t mndScheduleStream(SMnode* pMnode, SStreamObj* pStream, int64_t nextWindowSkey) { SQueryPlan* pPlan = qStringToQueryPlan(pStream->physicalPlan); if (pPlan == NULL) { terrno = TSDB_CODE_QRY_INVALID_INPUT; return -1; } - int32_t code = doScheduleStream(pStream, pMnode, pPlan, 0); + int32_t code = doScheduleStream(pStream, pMnode, pPlan, nextWindowSkey); qDestroyQueryPlan(pPlan); return code; diff --git a/source/dnode/mnode/impl/src/mndSma.c b/source/dnode/mnode/impl/src/mndSma.c index f0b8ed11be..68b697ca67 100644 --- a/source/dnode/mnode/impl/src/mndSma.c +++ b/source/dnode/mnode/impl/src/mndSma.c @@ -633,7 +633,7 @@ static int32_t mndCreateSma(SMnode *pMnode, SRpcMsg *pReq, SMCreateSmaReq *pCrea if (mndSetCreateSmaVgroupCommitLogs(pMnode, pTrans, &streamObj.fixedSinkVg) != 0) goto _OVER; if (mndSetUpdateSmaStbCommitLogs(pMnode, pTrans, pStb) != 0) goto _OVER; if (mndSetCreateSmaVgroupRedoActions(pMnode, pTrans, pDb, &streamObj.fixedSinkVg, &smaObj) != 0) goto _OVER; - if (mndScheduleStream(pMnode, &streamObj) != 0) goto _OVER; + if (mndScheduleStream(pMnode, &streamObj, 1685959190000) != 0) goto _OVER; if (mndPersistStream(pMnode, pTrans, &streamObj) != 0) goto _OVER; if (mndTransPrepare(pMnode, pTrans) != 0) goto _OVER; diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index 433f3fe272..f3bb8336ae 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -784,7 +784,7 @@ static int32_t mndProcessCreateStreamReq(SRpcMsg *pReq) { } // schedule stream task for stream obj - if (mndScheduleStream(pMnode, &streamObj) < 0) { + if (mndScheduleStream(pMnode, &streamObj, createStreamReq.lastTs) < 0) { mError("stream:%s, failed to schedule since %s", createStreamReq.name, terrstr()); mndTransDrop(pTrans); goto _OVER; diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index 1af5948268..cfbfb40ee6 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -582,13 +582,21 @@ int32_t streamTryExec(SStreamTask* pTask) { } int32_t streamTaskReleaseState(SStreamTask* pTask) { - void* exec = pTask->exec.pExecutor; - int32_t code = qStreamOperatorReleaseState(exec); - return code; + void* pExecutor = pTask->exec.pExecutor; + if (pExecutor != NULL) { + int32_t code = qStreamOperatorReleaseState(pExecutor); + return code; + } else { + return TSDB_CODE_SUCCESS; + } } int32_t streamTaskReloadState(SStreamTask* pTask) { - void* exec = pTask->exec.pExecutor; - int32_t code = qStreamOperatorReloadState(exec); - return code; + void* pExecutor = pTask->exec.pExecutor; + if (pExecutor != NULL) { + int32_t code = qStreamOperatorReloadState(pExecutor); + return code; + } else { + return TSDB_CODE_SUCCESS; + } } diff --git a/source/libs/stream/src/streamRecover.c b/source/libs/stream/src/streamRecover.c index a033556353..575cb7105d 100644 --- a/source/libs/stream/src/streamRecover.c +++ b/source/libs/stream/src/streamRecover.c @@ -461,9 +461,11 @@ static void tryLaunchHistoryTask(void* param, void* tmrId) { SStreamTask** ppTask = (SStreamTask**)taosHashGet(pMeta->pTasks, &pInfo->taskId, sizeof(int32_t)); if (ppTask) { ASSERT((*ppTask)->status.timerActive == 1); + if (streamTaskShouldStop(&(*ppTask)->status)) { - qDebug("s-task:%s status:%s quit timer task", (*ppTask)->id.idStr, - streamGetTaskStatusStr((*ppTask)->status.taskStatus)); + const char* pStatus = streamGetTaskStatusStr((*ppTask)->status.taskStatus); + qDebug("s-task:%s status:%s quit timer task", (*ppTask)->id.idStr, pStatus); + (*ppTask)->status.timerActive = 0; taosWUnLockLatch(&pMeta->lock); return; @@ -494,24 +496,27 @@ static void tryLaunchHistoryTask(void* param, void* tmrId) { } else { qError("s-task:0x%x failed to load task", pInfo->taskId); } + + taosMemoryFree(pInfo); } // todo fix the bug: 2. race condition // an fill history task needs to be started. int32_t streamCheckHistoryTaskDownstrem(SStreamTask* pTask) { SStreamMeta* pMeta = pTask->pMeta; + int32_t hTaskId = pTask->historyTaskId.taskId; // Set the execute conditions, including the query time window and the version range - SStreamTask** pHTask = taosHashGet(pMeta->pTasks, &pTask->historyTaskId.taskId, sizeof(pTask->historyTaskId.taskId)); + SStreamTask** pHTask = taosHashGet(pMeta->pTasks, &hTaskId, sizeof(hTaskId)); if (pHTask == NULL) { qWarn("s-task:%s vgId:%d failed to launch history task:0x%x, since it is not built yet", pTask->id.idStr, - pMeta->vgId, pTask->historyTaskId.taskId); + pMeta->vgId, hTaskId); + + SStreamTaskRetryInfo* pInfo = taosMemoryCalloc(1, sizeof(SStreamTaskRetryInfo)); + pInfo->taskId = pTask->id.taskId; + pInfo->pMeta = pTask->pMeta; if (pTask->timer == NULL) { - SStreamTaskRetryInfo* pInfo = taosMemoryCalloc(1, sizeof(SStreamTaskRetryInfo)); - pInfo->taskId = pTask->id.taskId; - pInfo->pMeta = pTask->pMeta; - pTask->timer = taosTmrStart(tryLaunchHistoryTask, 100, pInfo, streamEnv.timer); if (pTask->timer == NULL) { // todo failed to create timer @@ -519,6 +524,10 @@ int32_t streamCheckHistoryTaskDownstrem(SStreamTask* pTask) { pTask->status.timerActive = 1; // timer is active qDebug("s-task:%s set time active flag", pTask->id.idStr); } + } else { // timer exists + pTask->status.timerActive = 1; + qDebug("s-task:%s set time active flag", pTask->id.idStr); + taosTmrReset(tryLaunchHistoryTask, 100, pInfo, streamEnv.timer, &pTask->timer); } // try again in 500ms From 437eb93a4d77993707508eb07d057bdce71e3286 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 26 Jun 2023 18:44:59 +0800 Subject: [PATCH 66/83] fix(stream): fix error while fill history exists. --- source/dnode/mnode/impl/src/mndScheduler.c | 6 ++++-- source/libs/stream/src/streamExec.c | 2 +- source/libs/stream/src/streamRecover.c | 20 +++++++++++++------- source/libs/stream/src/streamTask.c | 6 +----- 4 files changed, 19 insertions(+), 15 deletions(-) diff --git a/source/dnode/mnode/impl/src/mndScheduler.c b/source/dnode/mnode/impl/src/mndScheduler.c index 36ed013549..33905bad86 100644 --- a/source/dnode/mnode/impl/src/mndScheduler.c +++ b/source/dnode/mnode/impl/src/mndScheduler.c @@ -554,13 +554,15 @@ static int32_t addSourceTasksForMultiLevelStream(SMnode* pMnode, SQueryPlan* pPl sdbRelease(pSdb, pVgroup); return code; } - - setHTasksId(pSourceTaskList, pHSourceTaskList); } sdbRelease(pSdb, pVgroup); } + if (pStream->conf.fillHistory) { + setHTasksId(pSourceTaskList, pHSourceTaskList); + } + return TSDB_CODE_SUCCESS; } diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index cfbfb40ee6..4768f5aed3 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -343,7 +343,7 @@ static void waitForTaskTobeIdle(SStreamTask* pTask, SStreamTask* pStreamTask) { double el = (taosGetTimestampMs() - st) / 1000.0; if (el > 0) { - qDebug("s-task:%s wait for stream task:%s for %.2fs to execute all data in inputQ", pTask->id.idStr, + qDebug("s-task:%s wait for stream task:%s for %.2fs to handle all data in inputQ", pTask->id.idStr, pStreamTask->id.idStr, el); } } diff --git a/source/libs/stream/src/streamRecover.c b/source/libs/stream/src/streamRecover.c index 575cb7105d..9f2d6b5908 100644 --- a/source/libs/stream/src/streamRecover.c +++ b/source/libs/stream/src/streamRecover.c @@ -479,22 +479,28 @@ static void tryLaunchHistoryTask(void* param, void* tmrId) { // abort the timer if intend to stop task SStreamTask* pHTask = streamMetaAcquireTask(pMeta, pTask->historyTaskId.taskId); - if (pHTask == NULL && pTask->status.taskStatus == TASK_STATUS__NORMAL) { - qWarn("s-task:%s vgId:%d failed to launch history task:0x%x, since it may not be built or have been destroyed", - pTask->id.idStr, pMeta->vgId, pTask->historyTaskId.taskId); + if (pHTask == NULL && (!streamTaskShouldStop(&pTask->status))) { + const char* pStatus = streamGetTaskStatusStr(pTask->status.taskStatus); + qWarn( + "s-task:%s vgId:%d status:%s failed to launch history task:0x%x, since it may not be built, or have been " + "destroyed, or should stop exec", + pTask->id.idStr, pMeta->vgId, pStatus, pTask->historyTaskId.taskId); taosTmrReset(tryLaunchHistoryTask, 100, pInfo, streamEnv.timer, &pTask->timer); + streamMetaReleaseTask(pMeta, pTask); return; } - doCheckDownstreamStatus(pTask, pHTask); + if (pHTask != NULL) { + doCheckDownstreamStatus(pTask, pHTask); + streamMetaReleaseTask(pMeta, pHTask); + } // not in timer anymore pTask->status.timerActive = 0; - streamMetaReleaseTask(pMeta, pHTask); streamMetaReleaseTask(pMeta, pTask); } else { - qError("s-task:0x%x failed to load task", pInfo->taskId); + qError("s-task:0x%x failed to load task, it may have been destoryed", pInfo->taskId); } taosMemoryFree(pInfo); @@ -664,7 +670,7 @@ void streamPrepareNdoCheckDownstream(SStreamTask* pTask) { // launch current task SHistDataRange* pRange = &pTask->dataRange; - int64_t ekey = pRange->window.ekey; + int64_t ekey = pRange->window.ekey + 1; int64_t ver = pRange->range.minVer; pRange->window.skey = ekey; diff --git a/source/libs/stream/src/streamTask.c b/source/libs/stream/src/streamTask.c index d6f23bb44a..06da72188c 100644 --- a/source/libs/stream/src/streamTask.c +++ b/source/libs/stream/src/streamTask.c @@ -42,11 +42,7 @@ SStreamTask* tNewStreamTask(int64_t streamId, int8_t taskLevel, int8_t fillHisto pTask->id.idStr = taosStrdup(buf); pTask->status.schedStatus = TASK_SCHED_STATUS__INACTIVE; - if (fillHistory) { - pTask->status.taskStatus = TASK_STATUS__SCAN_HISTORY; - } else { - pTask->status.taskStatus = TASK_STATUS__NORMAL; - } + pTask->status.taskStatus = TASK_STATUS__SCAN_HISTORY; pTask->inputStatus = TASK_INPUT_STATUS__NORMAL; pTask->outputStatus = TASK_OUTPUT_STATUS__NORMAL; From b73444b2911c3801762a5ec8e85d9c615c97792d Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 27 Jun 2023 09:44:51 +0800 Subject: [PATCH 67/83] fix(stream): start stream task in case of scan history completing. --- include/libs/stream/tstream.h | 2 +- source/dnode/snode/src/snode.c | 2 +- source/dnode/vnode/src/tq/tq.c | 65 +++++++++++++++---------- source/libs/stream/src/stream.c | 8 ++- source/libs/stream/src/streamDispatch.c | 5 +- source/libs/stream/src/streamRecover.c | 32 ++++++------ tests/script/tsim/stream/sliding.sim | 1 - 7 files changed, 67 insertions(+), 48 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 925bfa24c6..9a72f785ae 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -602,7 +602,7 @@ int32_t streamDispatchTransferStateMsg(SStreamTask* pTask); // agg level int32_t streamAggRecoverPrepare(SStreamTask* pTask); -int32_t streamProcessRecoverFinishReq(SStreamTask* pTask, int32_t childId); +int32_t streamProcessRecoverFinishReq(SStreamTask* pTask, int32_t taskId, int32_t childId); void streamMetaInit(); void streamMetaCleanup(); diff --git a/source/dnode/snode/src/snode.c b/source/dnode/snode/src/snode.c index 7146ba3f5b..d2fcf166b4 100644 --- a/source/dnode/snode/src/snode.c +++ b/source/dnode/snode/src/snode.c @@ -292,7 +292,7 @@ int32_t sndProcessTaskRecoverFinishReq(SSnode *pSnode, SRpcMsg *pMsg) { return -1; } // do process request - if (streamProcessRecoverFinishReq(pTask, req.childId) < 0) { + if (streamProcessRecoverFinishReq(pTask, req.taskId, req.childId) < 0) { streamMetaReleaseTask(pSnode->pMeta, pTask); return -1; } diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index f3e0bcaa0c..d474f0ca90 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -894,8 +894,9 @@ int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t ver) { streamSetupTrigger(pTask); - tqInfo("vgId:%d expand stream task, s-task:%s, checkpoint ver:%" PRId64 " child id:%d, level:%d", vgId, - pTask->id.idStr, pTask->chkInfo.version, pTask->info.selfChildId, pTask->info.taskLevel); + tqInfo("vgId:%d expand stream task, s-task:%s, checkpoint ver:%" PRId64 " child id:%d, level:%d, scan-history:%d", + vgId, pTask->id.idStr, pTask->chkInfo.version, pTask->info.selfChildId, pTask->info.taskLevel, + pTask->info.fillHistory); // next valid version will add one pTask->chkInfo.version += 1; @@ -1072,7 +1073,9 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { } // do recovery step 1 - tqDebug("s-task:%s start history data scan stage(step 1), status:%s", pTask->id.idStr, streamGetTaskStatusStr(pTask->status.taskStatus)); + const char* pId = pTask->id.idStr; + tqDebug("s-task:%s start history data scan stage(step 1), status:%s", pId, + streamGetTaskStatusStr(pTask->status.taskStatus)); int64_t st = taosGetTimestampMs(); int8_t schedStatus = atomic_val_compare_exchange_8(&pTask->status.schedStatus, TASK_SCHED_STATUS__INACTIVE, @@ -1085,15 +1088,16 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { if (!pReq->igUntreated && !streamTaskRecoverScanStep1Finished(pTask)) { streamSourceScanHistoryData(pTask); } + if (atomic_load_8(&pTask->status.taskStatus) == TASK_STATUS__DROPPING || streamTaskShouldPause(&pTask->status)) { - tqDebug("s-task:%s is dropped or paused, abort recover in step1", pTask->id.idStr); + tqDebug("s-task:%s is dropped or paused, abort recover in step1", pId); atomic_store_8(&pTask->status.schedStatus, TASK_SCHED_STATUS__INACTIVE); streamMetaReleaseTask(pMeta, pTask); return 0; } double el = (taosGetTimestampMs() - st) / 1000.0; - tqDebug("s-task:%s history data scan stage(step 1) ended, elapsed time:%.2fs", pTask->id.idStr, el); + tqDebug("s-task:%s history data scan stage(step 1) ended, elapsed time:%.2fs", pId, el); if (pTask->info.fillHistory) { SVersionRange* pRange = NULL; @@ -1118,7 +1122,7 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { // now we can stop the stream task execution pStreamTask->status.taskStatus = TASK_STATUS__HALT; tqDebug("s-task:%s level:%d status is set to halt by history scan task:%s", pStreamTask->id.idStr, - pStreamTask->info.taskLevel, pTask->id.idStr); + pStreamTask->info.taskLevel, pId); // if it's an source task, extract the last version in wal. int64_t ver = pTask->dataRange.range.maxVer + 1; @@ -1135,14 +1139,14 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { if (pRange->minVer == pRange->maxVer) { streamTaskRecoverSetAllStepFinished(pTask); tqDebug("s-task:%s no need to perform secondary scan-history-data(step 2), since no new data ingest", - pTask->id.idStr); + pId); } } if (!streamTaskRecoverScanStep1Finished(pTask)) { tqDebug("s-task:%s level:%d verRange:%" PRId64 " - %" PRId64 " do secondary scan-history-data after halt the related stream task:%s", - pTask->id.idStr, pTask->info.taskLevel, pRange->minVer, pRange->maxVer, pStreamTask->id.idStr); + pId, pTask->info.taskLevel, pRange->minVer, pRange->maxVer, pStreamTask->id.idStr); ASSERT(pTask->status.schedStatus == TASK_SCHED_STATUS__WAITING); st = taosGetTimestampMs(); @@ -1153,7 +1157,7 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { if(!streamTaskRecoverScanStep2Finished(pTask)) { streamSourceScanHistoryData(pTask); if (atomic_load_8(&pTask->status.taskStatus) == TASK_STATUS__DROPPING || streamTaskShouldPause(&pTask->status)) { - tqDebug("s-task:%s is dropped or paused, abort recover in step1", pTask->id.idStr); + tqDebug("s-task:%s is dropped or paused, abort recover in step1", pId); streamMetaReleaseTask(pMeta, pTask); return 0; } @@ -1161,12 +1165,11 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { } el = (taosGetTimestampMs() - st) / 1000.0; - tqDebug("s-task:%s history data scan stage(step 2) ended, elapsed time:%.2fs", pTask->id.idStr, el); + tqDebug("s-task:%s history data scan stage(step 2) ended, elapsed time:%.2fs", pId, el); if (!pTask->status.transferState) { // 3. notify the downstream tasks to transfer executor state after handle all history blocks. pTask->status.transferState = true; - code = streamDispatchTransferStateMsg(pTask); if (code != TSDB_CODE_SUCCESS) { // todo handle error @@ -1178,7 +1181,9 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { streamTryExec(pTask); pTask->status.taskStatus = TASK_STATUS__DROPPING; - tqDebug("s-task:%s set status to be dropping", pTask->id.idStr); + + tqDebug("s-task:%s set status to be dropping", pId); + // transfer the ownership of executor state streamTaskReleaseState(pTask); streamTaskReloadState(pStreamTask); @@ -1195,19 +1200,25 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { // todo update the chkInfo version for current task. // this task has an associated history stream task, so we need to scan wal from the end version of // history scan. The current version of chkInfo.current is not updated during the history scan + STimeWindow* pWindow = &pTask->dataRange.window; + if (pTask->historyTaskId.taskId == 0) { - pTask->dataRange.window.ekey = INT64_MAX; - pTask->dataRange.window.skey = INT64_MIN; - tqDebug("s-task:%s without associated stream task, reset the time window:%"PRId64" - %"PRId64, pTask->id.idStr, - pTask->dataRange.window.skey, pTask->dataRange.window.ekey); + *pWindow = (STimeWindow){INT64_MIN, INT64_MAX}; + tqDebug("s-task:%s without associated stream task, reset the time window:%" PRId64 " - %" PRId64, pId, + pWindow->skey, pWindow->ekey); } else { tqDebug("s-task:%s history data scan completed, now start to scan data from wal, start ver:%" PRId64 - ", window:%" PRId64 " - %" PRId64, - pTask->id.idStr, pTask->chkInfo.currentVer, pTask->dataRange.window.skey, pTask->dataRange.window.ekey); + ", window:%" PRId64 " - %" PRId64, pId, pTask->chkInfo.currentVer, pWindow->skey, pWindow->ekey); } code = streamTaskScanHistoryDataComplete(pTask); streamMetaReleaseTask(pMeta, pTask); + + // let's start the stream task by extracting data from wal + if (pTask->info.taskLevel == TASK_LEVEL__SOURCE) { + tqStartStreamTasks(pTq); + } + return code; } @@ -1311,7 +1322,7 @@ int32_t tqProcessTaskRecoverFinishReq(STQ* pTq, SRpcMsg* pMsg) { return -1; } // do process request - if (streamProcessRecoverFinishReq(pTask, req.childId) < 0) { + if (streamProcessRecoverFinishReq(pTask, req.taskId, req.childId) < 0) { streamMetaReleaseTask(pTq->pStreamMeta, pTask); return -1; } @@ -1402,9 +1413,9 @@ int32_t tqProcessTaskRunReq(STQ* pTq, SRpcMsg* pMsg) { pTask->chkInfo.version); streamProcessRunReq(pTask); } else { - if (streamTaskShouldPause(&pTask->status)) { +// if (streamTaskShouldPause(&pTask->status)) { atomic_store_8(&pTask->status.schedStatus, TASK_SCHED_STATUS__INACTIVE); - } +// } tqDebug("vgId:%d s-task:%s ignore run req since not in ready state, status:%s, sched-status:%d", vgId, pTask->id.idStr, streamGetTaskStatusStr(pTask->status.taskStatus), pTask->status.schedStatus); @@ -1543,19 +1554,23 @@ int32_t tqProcessTaskResumeReq(STQ* pTq, int64_t sversion, char* msg, int32_t ms } int32_t tqProcessTaskRetrieveReq(STQ* pTq, SRpcMsg* pMsg) { - char* msgStr = pMsg->pCont; - char* msgBody = POINTER_SHIFT(msgStr, sizeof(SMsgHead)); - int32_t msgLen = pMsg->contLen - sizeof(SMsgHead); + char* msgStr = pMsg->pCont; + char* msgBody = POINTER_SHIFT(msgStr, sizeof(SMsgHead)); + int32_t msgLen = pMsg->contLen - sizeof(SMsgHead); + SDecoder decoder; + SStreamRetrieveReq req; - SDecoder decoder; tDecoderInit(&decoder, (uint8_t*)msgBody, msgLen); tDecodeStreamRetrieveReq(&decoder, &req); tDecoderClear(&decoder); + int32_t taskId = req.dstTaskId; SStreamTask* pTask = streamMetaAcquireTask(pTq->pStreamMeta, taskId); + if (pTask) { SRpcMsg rsp = {.info = pMsg->info, .code = 0}; streamProcessRetrieveReq(pTask, &req, &rsp); + streamMetaReleaseTask(pTq->pStreamMeta, pTask); tDeleteStreamRetrieveReq(&req); return 0; diff --git a/source/libs/stream/src/stream.c b/source/libs/stream/src/stream.c index 5043486135..a1cc74054a 100644 --- a/source/libs/stream/src/stream.c +++ b/source/libs/stream/src/stream.c @@ -170,21 +170,18 @@ int32_t streamTaskEnqueueRetrieve(SStreamTask* pTask, SStreamRetrieveReq* pReq, // enqueue if (pData != NULL) { - qDebug("s-task:%s (child %d) recv retrieve req from task:0x%x, reqId %" PRId64, pTask->id.idStr, pTask->info.selfChildId, + qDebug("s-task:%s (child %d) recv retrieve req from task:0x%x, reqId:0x%" PRIx64, pTask->id.idStr, pTask->info.selfChildId, pReq->srcTaskId, pReq->reqId); pData->type = STREAM_INPUT__DATA_RETRIEVE; pData->srcVgId = 0; - // decode - /*pData->blocks = pReq->data;*/ - /*pBlock->sourceVer = pReq->sourceVer;*/ streamRetrieveReqToData(pReq, pData); if (tAppendDataToInputQueue(pTask, (SStreamQueueItem*)pData) == 0) { status = TASK_INPUT_STATUS__NORMAL; } else { status = TASK_INPUT_STATUS__FAILED; } - } else { + } else { // todo handle oom /*streamTaskInputFail(pTask);*/ /*status = TASK_INPUT_STATUS__FAILED;*/ } @@ -199,6 +196,7 @@ int32_t streamTaskEnqueueRetrieve(SStreamTask* pTask, SStreamRetrieveReq* pReq, pRsp->pCont = buf; pRsp->contLen = sizeof(SMsgHead) + sizeof(SStreamRetrieveRsp); tmsgSendRsp(pRsp); + return status == TASK_INPUT_STATUS__NORMAL ? 0 : -1; } diff --git a/source/libs/stream/src/streamDispatch.c b/source/libs/stream/src/streamDispatch.c index c42320ad13..358932cf20 100644 --- a/source/libs/stream/src/streamDispatch.c +++ b/source/libs/stream/src/streamDispatch.c @@ -314,7 +314,10 @@ int32_t streamDoDispatchScanHistoryFinishMsg(SStreamTask* pTask, const SStreamRe msg.info.noResp = 1; tmsgSendReq(pEpSet, &msg); - qDebug("s-task:%s dispatch scan-history-data finish msg to taskId:0x%x (vgId:%d)", pTask->id.idStr, pReq->taskId, vgId); + + const char* pStatus = streamGetTaskStatusStr(pTask->status.taskStatus); + qDebug("s-task:%s status:%s dispatch scan-history-data finish msg to taskId:0x%x (vgId:%d)", pTask->id.idStr, pStatus, + pReq->taskId, vgId); return 0; } diff --git a/source/libs/stream/src/streamRecover.c b/source/libs/stream/src/streamRecover.c index 9f2d6b5908..a025d01004 100644 --- a/source/libs/stream/src/streamRecover.c +++ b/source/libs/stream/src/streamRecover.c @@ -86,8 +86,7 @@ int32_t streamTaskLaunchScanHistory(SStreamTask* pTask) { // check status int32_t streamTaskCheckDownstreamTasks(SStreamTask* pTask) { SHistDataRange* pRange = &pTask->dataRange; - qDebug("s-task:%s check downstream tasks, ver:%" PRId64 "-%" PRId64 " window:%" PRId64 "-%" PRId64, - pTask->id.idStr, pRange->range.minVer, pRange->range.maxVer, pRange->window.skey, pRange->window.ekey); + STimeWindow* pWindow = &pRange->window; SStreamTaskCheckReq req = { .streamId = pTask->id.streamId, @@ -98,13 +97,16 @@ int32_t streamTaskCheckDownstreamTasks(SStreamTask* pTask) { // serialize if (pTask->outputType == TASK_OUTPUT__FIXED_DISPATCH) { + qDebug("s-task:%s check single downstream task:0x%x(vgId:%d) ver:%" PRId64 "-%" PRId64 " window:%" PRId64 + "-%" PRId64, + pTask->id.idStr, req.downstreamTaskId, req.downstreamNodeId, pRange->range.minVer, pRange->range.maxVer, + pWindow->skey, pWindow->ekey); + req.reqId = tGenIdPI64(); req.downstreamNodeId = pTask->fixedEpDispatcher.nodeId; req.downstreamTaskId = pTask->fixedEpDispatcher.taskId; pTask->checkReqId = req.reqId; - qDebug("s-task:%s (vgId:%d) check downstream task:0x%x (vgId:%d)", pTask->id.idStr, pTask->info.nodeId, req.downstreamTaskId, - req.downstreamNodeId); streamDispatchCheckMsg(pTask, &req, pTask->fixedEpDispatcher.nodeId, &pTask->fixedEpDispatcher.epSet); } else if (pTask->outputType == TASK_OUTPUT__SHUFFLE_DISPATCH) { SArray* vgInfo = pTask->shuffleDispatcher.dbInfo.pVgroupInfos; @@ -113,14 +115,17 @@ int32_t streamTaskCheckDownstreamTasks(SStreamTask* pTask) { pTask->notReadyTasks = numOfVgs; pTask->checkReqIds = taosArrayInit(numOfVgs, sizeof(int64_t)); + qDebug("s-task:%s check %d downstream tasks, ver:%" PRId64 "-%" PRId64 " window:%" PRId64 "-%" PRId64, + pTask->id.idStr, numOfVgs, pRange->range.minVer, pRange->range.maxVer, pWindow->skey, pWindow->ekey); + for (int32_t i = 0; i < numOfVgs; i++) { SVgroupInfo* pVgInfo = taosArrayGet(vgInfo, i); req.reqId = tGenIdPI64(); taosArrayPush(pTask->checkReqIds, &req.reqId); req.downstreamNodeId = pVgInfo->vgId; req.downstreamTaskId = pVgInfo->taskId; - qDebug("s-task:%s (vgId:%d) check downstream task:0x%x (vgId:%d) (shuffle)", pTask->id.idStr, pTask->info.nodeId, - req.downstreamTaskId, req.downstreamNodeId); + qDebug("s-task:%s (vgId:%d) check downstream task:0x%x (vgId:%d) (shuffle), idx:%d", pTask->id.idStr, pTask->info.nodeId, + req.downstreamTaskId, req.downstreamNodeId, i); streamDispatchCheckMsg(pTask, &req, pVgInfo->vgId, &pVgInfo->epSet); } } else { @@ -303,9 +308,6 @@ int32_t streamDispatchScanHistoryFinishMsg(SStreamTask* pTask) { // serialize if (pTask->outputType == TASK_OUTPUT__FIXED_DISPATCH) { - qDebug("s-task:%s send scan-history-data complete msg to downstream (fix-dispatch) to taskId:0x%x, status:%s", pTask->id.idStr, - pTask->fixedEpDispatcher.taskId, streamGetTaskStatusStr(pTask->status.taskStatus)); - req.taskId = pTask->fixedEpDispatcher.taskId; streamDoDispatchScanHistoryFinishMsg(pTask, &req, pTask->fixedEpDispatcher.nodeId, &pTask->fixedEpDispatcher.epSet); } else if (pTask->outputType == TASK_OUTPUT__SHUFFLE_DISPATCH) { @@ -393,7 +395,7 @@ int32_t streamDispatchTransferStateMsg(SStreamTask* pTask) { // agg int32_t streamAggRecoverPrepare(SStreamTask* pTask) { pTask->numOfWaitingUpstream = taosArrayGetSize(pTask->pUpstreamEpInfoList); - qDebug("s-task:%s agg task is ready and wait for %d upstream tasks complete fill history procedure", pTask->id.idStr, + qDebug("s-task:%s agg task is ready and wait for %d upstream tasks complete scan-history procedure", pTask->id.idStr, pTask->numOfWaitingUpstream); return 0; } @@ -412,7 +414,7 @@ int32_t streamAggUpstreamScanHistoryFinish(SStreamTask* pTask) { return 0; } -int32_t streamProcessRecoverFinishReq(SStreamTask* pTask, int32_t childId) { +int32_t streamProcessRecoverFinishReq(SStreamTask* pTask, int32_t taskId, int32_t childId) { if (pTask->info.taskLevel == TASK_LEVEL__AGG) { int32_t left = atomic_sub_fetch_32(&pTask->numOfWaitingUpstream, 1); ASSERT(left >= 0); @@ -422,7 +424,8 @@ int32_t streamProcessRecoverFinishReq(SStreamTask* pTask, int32_t childId) { qDebug("s-task:%s all %d upstream tasks finish scan-history data", pTask->id.idStr, numOfTasks); streamAggUpstreamScanHistoryFinish(pTask); } else { - qDebug("s-task:%s remain unfinished upstream tasks:%d", pTask->id.idStr, left); + qDebug("s-task:%s receive scan-history data finish msg from upstream:0x%x(index:%d), unfinished:%d", + pTask->id.idStr, taskId, childId, left); } } @@ -563,11 +566,12 @@ int32_t streamTaskScanHistoryDataComplete(SStreamTask* pTask) { } ASSERT(pTask->status.taskStatus == TASK_STATUS__SCAN_HISTORY); - /*code = */streamSetStatusNormal(pTask); + // ready to process data from inputQ + streamSetStatusNormal(pTask); atomic_store_8(&pTask->status.schedStatus, TASK_SCHED_STATUS__INACTIVE); - // todo check rsp + // todo check rsp, commit data streamMetaSaveTask(pMeta, pTask); return 0; } diff --git a/tests/script/tsim/stream/sliding.sim b/tests/script/tsim/stream/sliding.sim index 05eb7dacba..7617233dc6 100644 --- a/tests/script/tsim/stream/sliding.sim +++ b/tests/script/tsim/stream/sliding.sim @@ -576,7 +576,6 @@ $loop_count = 0 print step 7 - sql create database test3 vgroups 6; sql use test3; sql create stable st(ts timestamp, a int, b int, c int, d double) tags(ta int,tb int,tc int); From 32fddeff1800d38873129998503092ec27139206 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 27 Jun 2023 16:24:47 +0800 Subject: [PATCH 68/83] fix(stream): fix error retrieve data from source task.And optimize the test cases. --- source/common/src/systable.c | 2 +- source/dnode/mnode/impl/src/mndStream.c | 19 +++++++-- source/dnode/vnode/src/tq/tq.c | 4 +- source/dnode/vnode/src/tq/tqRestore.c | 41 +++++++++++-------- source/libs/executor/src/executor.c | 3 +- source/libs/stream/src/stream.c | 7 ++-- source/libs/stream/src/streamExec.c | 3 +- tests/script/tsim/stream/sliding.sim | 1 - .../system-test/1-insert/db_tb_name_check.py | 4 +- 9 files changed, 51 insertions(+), 33 deletions(-) diff --git a/source/common/src/systable.c b/source/common/src/systable.c index 7cd33955c1..12ec97080f 100644 --- a/source/common/src/systable.c +++ b/source/common/src/systable.c @@ -160,7 +160,7 @@ static const SSysDbTableSchema streamSchema[] = { static const SSysDbTableSchema streamTaskSchema[] = { {.name = "stream_name", .bytes = SYSTABLE_SCH_DB_NAME_LEN, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = false}, - {.name = "task_id", .bytes = 8, .type = TSDB_DATA_TYPE_INT, .sysInfo = false}, + {.name = "task_id", .bytes = 32, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = false}, {.name = "node_type", .bytes = SYSTABLE_SCH_DB_NAME_LEN, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = false}, {.name = "node_id", .bytes = 8, .type = TSDB_DATA_TYPE_INT, .sysInfo = false}, {.name = "level", .bytes = 20 + VARSTR_HEADER_SIZE, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = false}, diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index a3f03a46a9..be6cfae0be 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -1221,12 +1221,16 @@ static int32_t mndRetrieveStreamTask(SRpcMsg *pReq, SShowObj *pShow, SSDataBlock while (numOfRows < rowsCapacity) { pShow->pIter = sdbFetch(pSdb, SDB_STREAM, pShow->pIter, (void **)&pStream); - if (pShow->pIter == NULL) break; + if (pShow->pIter == NULL) { + break; + } // lock taosRLockLatch(&pStream->lock); + // count task num int32_t sz = taosArrayGetSize(pStream->tasks); + int32_t count = 0; for (int32_t i = 0; i < sz; i++) { SArray *pLevel = taosArrayGetP(pStream->tasks, i); @@ -1236,10 +1240,12 @@ static int32_t mndRetrieveStreamTask(SRpcMsg *pReq, SShowObj *pShow, SSDataBlock if (numOfRows + count > rowsCapacity) { blockDataEnsureCapacity(pBlock, numOfRows + count); } + // add row for each task for (int32_t i = 0; i < sz; i++) { SArray *pLevel = taosArrayGetP(pStream->tasks, i); int32_t levelCnt = taosArrayGetSize(pLevel); + for (int32_t j = 0; j < levelCnt; j++) { SStreamTask *pTask = taosArrayGetP(pLevel, j); @@ -1249,12 +1255,19 @@ static int32_t mndRetrieveStreamTask(SRpcMsg *pReq, SShowObj *pShow, SSDataBlock // stream name char streamName[TSDB_TABLE_NAME_LEN + VARSTR_HEADER_SIZE] = {0}; STR_WITH_MAXSIZE_TO_VARSTR(streamName, mndGetDbStr(pStream->name), sizeof(streamName)); + pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); colDataSetVal(pColInfo, numOfRows, (const char *)streamName, false); // task id pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); - colDataSetVal(pColInfo, numOfRows, (const char *)&pTask->id.taskId, false); + + char idstr[128] = {0}; + int32_t len = tintToHex(pTask->id.taskId, &idstr[4]); + idstr[2] = '0'; + idstr[3] = 'x'; + varDataSetLen(idstr, len + 2); + colDataSetVal(pColInfo, numOfRows, idstr, false); // node type char nodeType[20 + VARSTR_HEADER_SIZE] = {0}; @@ -1283,8 +1296,8 @@ static int32_t mndRetrieveStreamTask(SRpcMsg *pReq, SShowObj *pShow, SSDataBlock } else if (pTask->info.taskLevel == TASK_LEVEL__SINK) { memcpy(varDataVal(level), "sink", 4); varDataSetLen(level, 4); - } else if (pTask->info.taskLevel == TASK_LEVEL__SINK) { } + pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); colDataSetVal(pColInfo, numOfRows, (const char *)&level, false); diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 4d5e32c7cd..8ddf920985 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1203,8 +1203,8 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { if (pTask->historyTaskId.taskId == 0) { *pWindow = (STimeWindow){INT64_MIN, INT64_MAX}; - tqDebug("s-task:%s without associated stream task, reset the time window:%" PRId64 " - %" PRId64, pId, - pWindow->skey, pWindow->ekey); + tqDebug("s-task:%s no associated task, reset the time window:%" PRId64 " - %" PRId64, pId, pWindow->skey, + pWindow->ekey); } else { tqDebug("s-task:%s history data scan completed, now start to scan data from wal, start ver:%" PRId64 ", window:%" PRId64 " - %" PRId64, pId, pTask->chkInfo.currentVer, pWindow->skey, pWindow->ekey); diff --git a/source/dnode/vnode/src/tq/tqRestore.c b/source/dnode/vnode/src/tq/tqRestore.c index 833f59fe8d..ea7327f741 100644 --- a/source/dnode/vnode/src/tq/tqRestore.c +++ b/source/dnode/vnode/src/tq/tqRestore.c @@ -207,7 +207,7 @@ int32_t doSetOffsetForWalReader(SStreamTask *pTask, int32_t vgId) { int32_t createStreamTaskRunReq(SStreamMeta* pStreamMeta, bool* pScanIdle) { *pScanIdle = true; - bool noNewDataInWal = true; + bool noDataInWal = true; int32_t vgId = pStreamMeta->vgId; int32_t numOfTasks = taosArrayGetSize(pStreamMeta->pTaskList); @@ -235,7 +235,6 @@ int32_t createStreamTaskRunReq(SStreamMeta* pStreamMeta, bool* pScanIdle) { int32_t status = pTask->status.taskStatus; if (pTask->info.taskLevel != TASK_LEVEL__SOURCE) { -// tqTrace("s-task:%s level:%d not source task, no need to start", pTask->id.idStr, pTask->info.taskLevel); streamMetaReleaseTask(pStreamMeta, pTask); continue; } @@ -261,36 +260,44 @@ int32_t createStreamTaskRunReq(SStreamMeta* pStreamMeta, bool* pScanIdle) { continue; } + int32_t numOfItemsInQ = taosQueueItemSize(pTask->inputQueue->queue); + // append the data for the stream SStreamQueueItem* pItem = NULL; code = extractMsgFromWal(pTask->exec.pWalReader, (void**) &pItem, pTask->id.idStr); - if (code != TSDB_CODE_SUCCESS) { // failed, continue + + if ((code != TSDB_CODE_SUCCESS || pItem == NULL) && (numOfItemsInQ == 0)) { // failed, continue streamMetaReleaseTask(pStreamMeta, pTask); continue; } - // delete ignore - if (pItem == NULL) { - streamMetaReleaseTask(pStreamMeta, pTask); - continue; + noDataInWal = false; + + if (pItem != NULL) { + code = tAppendDataToInputQueue(pTask, pItem); + if (code == TSDB_CODE_SUCCESS) { + pTask->chkInfo.currentVer = walReaderGetCurrentVer(pTask->exec.pWalReader); + tqDebug("s-task:%s set the ver:%" PRId64 " from WALReader after extract block from WAL", pTask->id.idStr, + pTask->chkInfo.currentVer); + } else { + tqError("s-task:%s append input queue failed, too many in inputQ, ver:%" PRId64, pTask->id.idStr, + pTask->chkInfo.currentVer); + } } - noNewDataInWal = false; - - code = tqAddInputBlockNLaunchTask(pTask, pItem); - if (code == TSDB_CODE_SUCCESS) { - pTask->chkInfo.currentVer = walReaderGetCurrentVer(pTask->exec.pWalReader); - tqDebug("s-task:%s set the ver:%" PRId64 " from WALReader after extract block from WAL", pTask->id.idStr, - pTask->chkInfo.currentVer); - } else { - tqError("s-task:%s append input queue failed, ver:%" PRId64, pTask->id.idStr, pTask->chkInfo.currentVer); + if ((code == TSDB_CODE_SUCCESS) || (numOfItemsInQ > 0)) { + code = streamSchedExec(pTask); + if (code != TSDB_CODE_SUCCESS) { + streamMetaReleaseTask(pStreamMeta, pTask); + return -1; + } } streamMetaReleaseTask(pStreamMeta, pTask); } // all wal are checked, and no new data available in wal. - if (noNewDataInWal) { + if (noDataInWal) { *pScanIdle = true; } diff --git a/source/libs/executor/src/executor.c b/source/libs/executor/src/executor.c index ad79bc87d7..fab608c2b5 100644 --- a/source/libs/executor/src/executor.c +++ b/source/libs/executor/src/executor.c @@ -131,10 +131,9 @@ static int32_t doSetStreamBlock(SOperatorInfo* pOperator, void* input, size_t nu return doSetStreamBlock(pOperator->pDownstream[0], input, numOfBlocks, type, id); } else { pOperator->status = OP_NOT_OPENED; - SStreamScanInfo* pInfo = pOperator->info; - qDebug("s-task:%s in this batch, all %d blocks need to be processed and dump results", id, (int32_t)numOfBlocks); + qDebug("s-task:%s in this batch, %d blocks need to be processed", id, (int32_t)numOfBlocks); ASSERT(pInfo->validBlockIndex == 0 && taosArrayGetSize(pInfo->pBlockLists) == 0); if (type == STREAM_INPUT__MERGED_SUBMIT) { diff --git a/source/libs/stream/src/stream.c b/source/libs/stream/src/stream.c index a1cc74054a..b5c83bc10b 100644 --- a/source/libs/stream/src/stream.c +++ b/source/libs/stream/src/stream.c @@ -113,6 +113,7 @@ int32_t streamSchedExec(SStreamTask* pTask) { if (pRunReq == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; atomic_store_8(&pTask->status.schedStatus, TASK_SCHED_STATUS__INACTIVE); + qError("failed to create msg to aunch s-task:%s, reason out of memory", pTask->id.idStr); return -1; } @@ -170,8 +171,8 @@ int32_t streamTaskEnqueueRetrieve(SStreamTask* pTask, SStreamRetrieveReq* pReq, // enqueue if (pData != NULL) { - qDebug("s-task:%s (child %d) recv retrieve req from task:0x%x, reqId:0x%" PRIx64, pTask->id.idStr, pTask->info.selfChildId, - pReq->srcTaskId, pReq->reqId); + qDebug("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; @@ -308,9 +309,7 @@ int32_t streamProcessRunReq(SStreamTask* pTask) { } int32_t streamProcessRetrieveReq(SStreamTask* pTask, SStreamRetrieveReq* pReq, SRpcMsg* pRsp) { - qDebug("s-task:%s receive retrieve req from taskId:0x%x (vgId:%d)", pTask->id.idStr, pReq->srcTaskId, pReq->srcNodeId); streamTaskEnqueueRetrieve(pTask, pReq, pRsp); - ASSERT(pTask->info.taskLevel != TASK_LEVEL__SINK); streamSchedExec(pTask); return 0; diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index 4768f5aed3..3c150e924f 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -111,7 +111,8 @@ static int32_t streamTaskExecImpl(SStreamTask* pTask, SStreamQueueItem* pItem, i block.info.childId = pTask->info.selfChildId; taosArrayPush(pRes, &block); numOfBlocks += 1; - qDebug("s-task:%s(child %d) processed retrieve, reqId:0x%" PRIx64, pTask->id.idStr, pTask->info.selfChildId, + + qDebug("s-task:%s(child %d) retrieve process completed, reqId:0x%" PRIx64" dump results", pTask->id.idStr, pTask->info.selfChildId, pRetrieveBlock->reqId); } diff --git a/tests/script/tsim/stream/sliding.sim b/tests/script/tsim/stream/sliding.sim index 7617233dc6..18893245fa 100644 --- a/tests/script/tsim/stream/sliding.sim +++ b/tests/script/tsim/stream/sliding.sim @@ -575,7 +575,6 @@ endi $loop_count = 0 print step 7 - sql create database test3 vgroups 6; sql use test3; sql create stable st(ts timestamp, a int, b int, c int, d double) tags(ta int,tb int,tc int); diff --git a/tests/system-test/1-insert/db_tb_name_check.py b/tests/system-test/1-insert/db_tb_name_check.py index 23bb539620..fa43603e25 100644 --- a/tests/system-test/1-insert/db_tb_name_check.py +++ b/tests/system-test/1-insert/db_tb_name_check.py @@ -44,7 +44,7 @@ class TDTestCase: new_dbname = list(dbname) new_dbname.insert(i,j) dbname_1 = ''.join(new_dbname) - tdSql.execute(f'create database if not exists `{dbname_1}`') + tdSql.execute(f'create database if not exists `{dbname_1}` vgroups 1 replica 1') tdSql.query('select * from information_schema.ins_databases') tdSql.checkEqual(tdSql.queryResult[2][0],str(dbname_1)) tdSql.execute(f'drop database `{dbname_1}`') @@ -56,7 +56,7 @@ class TDTestCase: def tb_name_check(self): dbname = tdCom.getLongName(10) - tdSql.execute(f'create database if not exists `{dbname}`') + tdSql.execute(f'create database if not exists `{dbname}` vgroups 1 replica 1') tdSql.execute(f'use `{dbname}`') tbname = tdCom.getLongName(5) for i in self.special_name: From 4bb78df27c1f9db75c71552242cfd0cd2e8d15c6 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 27 Jun 2023 16:44:35 +0800 Subject: [PATCH 69/83] fix(stream): reduce the sleep time. --- source/dnode/vnode/src/tq/tq.c | 53 +++++++++++++++++----------------- 1 file changed, 26 insertions(+), 27 deletions(-) diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 8ddf920985..677ce88457 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -156,6 +156,29 @@ void tqClose(STQ* pTq) { taosMemoryFree(pTq); } +static bool hasStreamTaskInTimer(SStreamMeta* pMeta) { + bool inTimer = false; + + taosWLockLatch(&pMeta->lock); + + void* pIter = NULL; + while(1) { + pIter = taosHashIterate(pMeta->pTasks, pIter); + if (pIter == NULL) { + break; + } + + SStreamTask* pTask = *(SStreamTask**)pIter; + if (pTask->status.timerActive == 1) { + inTimer = true; + } + } + + taosWUnLockLatch(&pMeta->lock); + + return inTimer; +} + void tqNotifyClose(STQ* pTq) { if (pTq != NULL) { taosWLockLatch(&pTq->pStreamMeta->lock); @@ -183,38 +206,14 @@ void tqNotifyClose(STQ* pTq) { tqDebug("vgId:%d start to check all tasks", pTq->pStreamMeta->vgId); int64_t st = taosGetTimestampMs(); - while(1) { - taosMsleep(1000); - bool inTimer = false; - - taosWLockLatch(&pTq->pStreamMeta->lock); - pIter = NULL; - - while(1) { - pIter = taosHashIterate(pTq->pStreamMeta->pTasks, pIter); - if (pIter == NULL) { - break; - } - - SStreamTask* pTask = *(SStreamTask**)pIter; - if (pTask->status.timerActive == 1) { - inTimer = true; - } - } - - taosWUnLockLatch(&pTq->pStreamMeta->lock); - - if (inTimer) { - tqDebug("vgId:%d some tasks in timer, wait for 1sec and recheck", pTq->pStreamMeta->vgId); - } else { - break; - } + while(hasStreamTaskInTimer(pTq->pStreamMeta)) { + tqDebug("vgId:%d some tasks in timer, wait for 100ms and recheck", pTq->pStreamMeta->vgId); + taosMsleep(100); } int64_t el = taosGetTimestampMs() - st; tqDebug("vgId:%d all stream tasks are not in timer, continue close, elapsed time:%"PRId64" ms", pTq->pStreamMeta->vgId, el); - } } From a19e63fd235ed1a0050f895e5371461a9203c1c4 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 27 Jun 2023 23:00:14 +0800 Subject: [PATCH 70/83] fix(stream): fix error in handling fill history. --- include/libs/executor/executor.h | 12 ++------ include/libs/stream/tstream.h | 2 +- source/dnode/vnode/src/tq/tq.c | 21 +++++++------ source/dnode/vnode/src/tq/tqRestore.c | 3 +- source/libs/executor/inc/querytask.h | 3 -- source/libs/executor/src/executor.c | 26 ++++++++++------- source/libs/executor/src/executorInt.c | 38 ------------------------ source/libs/executor/src/scanoperator.c | 12 ++++---- source/libs/stream/src/stream.c | 3 +- source/libs/stream/src/streamExec.c | 9 +++--- source/libs/stream/src/streamMeta.c | 2 +- source/libs/stream/src/streamRecover.c | 39 +++++++------------------ 12 files changed, 58 insertions(+), 112 deletions(-) diff --git a/include/libs/executor/executor.h b/include/libs/executor/executor.h index 5fd9a4a8c1..f4713f7a6f 100644 --- a/include/libs/executor/executor.h +++ b/include/libs/executor/executor.h @@ -195,14 +195,6 @@ int32_t qDeserializeTaskStatus(qTaskInfo_t tinfo, const char* pInput, int32_t le void getNextTimeWindow(const SInterval* pInterval, STimeWindow* tw, int32_t order); void getInitialStartTimeWindow(SInterval* pInterval, TSKEY ts, STimeWindow* w, bool ascQuery); STimeWindow getAlignQueryTimeWindow(const SInterval* pInterval, int64_t key); -/** - * return the scan info, in the form of tuple of two items, including table uid and current timestamp - * @param tinfo - * @param uid - * @param ts - * @return - */ -int32_t qGetStreamScanStatus(qTaskInfo_t tinfo, uint64_t* uid, int64_t* ts); SArray* qGetQueriedTableListInfo(qTaskInfo_t tinfo); @@ -225,16 +217,16 @@ int32_t qExtractStreamScanner(qTaskInfo_t tinfo, void** scanner); int32_t qSetStreamOperatorOptionForScanHistory(qTaskInfo_t tinfo); int32_t qStreamSourceScanParamForHistoryScanStep1(qTaskInfo_t tinfo, SVersionRange *pVerRange, STimeWindow* pWindow); int32_t qStreamSourceScanParamForHistoryScanStep2(qTaskInfo_t tinfo, SVersionRange *pVerRange, STimeWindow* pWindow); -int32_t qStreamSourceRecoverStep2(qTaskInfo_t tinfo, int64_t ver); int32_t qStreamRecoverFinish(qTaskInfo_t tinfo); int32_t qRestoreStreamOperatorOption(qTaskInfo_t tinfo); bool qStreamRecoverScanFinished(qTaskInfo_t tinfo); bool qStreamRecoverScanStep1Finished(qTaskInfo_t tinfo); bool qStreamRecoverScanStep2Finished(qTaskInfo_t tinfo); int32_t qStreamRecoverSetAllStepFinished(qTaskInfo_t tinfo); -void qStreamCloseTsdbReader(void* task); void resetTaskInfo(qTaskInfo_t tinfo); +void qResetStreamInfoTimeWindow(qTaskInfo_t tinfo); + int32_t qStreamOperatorReleaseState(qTaskInfo_t tInfo); int32_t qStreamOperatorReloadState(qTaskInfo_t tInfo); diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 9a72f785ae..ef533dc969 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -267,7 +267,7 @@ typedef struct SCheckpointInfo { typedef struct SStreamStatus { int8_t taskStatus; - int8_t checkDownstream; // downstream tasks are all ready now, if this flag is set + int8_t downstreamReady; // downstream tasks are all ready now, if this flag is set int8_t schedStatus; int8_t keepTaskStatus; bool transferState; diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 677ce88457..f68b725cdd 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -817,10 +817,10 @@ int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t ver) { SStreamTask task = {0}; if (pTask->info.fillHistory) { task.id = pTask->streamTaskId; - SStreamMeta meta = {0}; task.pMeta = pTask->pMeta; pSateTask = &task; } + pTask->pState = streamStateOpen(pTq->pStreamMeta->path, pSateTask, false, -1, -1); if (pTask->pState == NULL) { return -1; @@ -840,7 +840,6 @@ int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t ver) { SStreamTask task = {0}; if (pTask->info.fillHistory) { task.id = pTask->streamTaskId; - SStreamMeta meta = {0}; task.pMeta = pTask->pMeta; pSateTask = &task; } @@ -1100,6 +1099,7 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { if (pTask->info.fillHistory) { SVersionRange* pRange = NULL; SStreamTask* pStreamTask = NULL; + if (!pReq->igUntreated && !streamTaskRecoverScanStep1Finished(pTask)) { // 1. stop the related stream task, get the current scan wal version of stream task, ver. pStreamTask = streamMetaAcquireTask(pMeta, pTask->streamTaskId.taskId); @@ -1110,7 +1110,7 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { ASSERT(pStreamTask->info.taskLevel == TASK_LEVEL__SOURCE); // wait for the stream task get ready for scan history data - while (((pStreamTask->status.checkDownstream == 0) && (pStreamTask->status.taskStatus != TASK_STATUS__STOP)) || + while (((pStreamTask->status.downstreamReady == 0) && (pStreamTask->status.taskStatus != TASK_STATUS__STOP)) || pStreamTask->status.taskStatus == TASK_STATUS__SCAN_HISTORY) { tqDebug("s-task:%s level:%d not ready for halt, wait for 100ms and recheck", pStreamTask->id.idStr, pStreamTask->info.taskLevel); @@ -1136,8 +1136,7 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { pRange->maxVer = ver; if (pRange->minVer == pRange->maxVer) { streamTaskRecoverSetAllStepFinished(pTask); - tqDebug("s-task:%s no need to perform secondary scan-history-data(step 2), since no new data ingest", - pId); + tqDebug("s-task:%s no need to perform secondary scan-history-data(step 2), since no new data ingest", pId); } } @@ -1147,31 +1146,34 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { pId, pTask->info.taskLevel, pRange->minVer, pRange->maxVer, pStreamTask->id.idStr); ASSERT(pTask->status.schedStatus == TASK_SCHED_STATUS__WAITING); - st = taosGetTimestampMs(); + st = taosGetTimestampMs(); streamSetParamForStreamScannerStep2(pTask, pRange, &pTask->dataRange.window); } if(!streamTaskRecoverScanStep2Finished(pTask)) { + streamSourceScanHistoryData(pTask); if (atomic_load_8(&pTask->status.taskStatus) == TASK_STATUS__DROPPING || streamTaskShouldPause(&pTask->status)) { tqDebug("s-task:%s is dropped or paused, abort recover in step1", pId); streamMetaReleaseTask(pMeta, pTask); return 0; } + streamTaskRecoverSetAllStepFinished(pTask); } el = (taosGetTimestampMs() - st) / 1000.0; tqDebug("s-task:%s history data scan stage(step 2) ended, elapsed time:%.2fs", pId, el); + // 3. notify the downstream tasks to transfer executor state after handle all history blocks. if (!pTask->status.transferState) { - // 3. notify the downstream tasks to transfer executor state after handle all history blocks. - pTask->status.transferState = true; code = streamDispatchTransferStateMsg(pTask); if (code != TSDB_CODE_SUCCESS) { // todo handle error } + + pTask->status.transferState = true; } // 4. 1) transfer the ownership of executor state, 2) update the scan data range for source task. @@ -1179,12 +1181,12 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { streamTryExec(pTask); pTask->status.taskStatus = TASK_STATUS__DROPPING; - tqDebug("s-task:%s set status to be dropping", pId); // transfer the ownership of executor state streamTaskReleaseState(pTask); streamTaskReloadState(pStreamTask); + streamMetaSaveTask(pMeta, pTask); streamMetaSaveTask(pMeta, pStreamTask); @@ -1405,6 +1407,7 @@ int32_t tqProcessTaskRunReq(STQ* pTq, SRpcMsg* pMsg) { SStreamTask* pTask = streamMetaAcquireTask(pTq->pStreamMeta, taskId); if (pTask != NULL) { + // even in halt status, the data in inputQ must be processed int8_t status = pTask->status.taskStatus; if (status == TASK_STATUS__NORMAL || status == TASK_STATUS__HALT) { tqDebug("vgId:%d s-task:%s start to process block from inputQ, last chk point:%" PRId64, vgId, pTask->id.idStr, diff --git a/source/dnode/vnode/src/tq/tqRestore.c b/source/dnode/vnode/src/tq/tqRestore.c index ea7327f741..5db3e735cc 100644 --- a/source/dnode/vnode/src/tq/tqRestore.c +++ b/source/dnode/vnode/src/tq/tqRestore.c @@ -271,9 +271,8 @@ int32_t createStreamTaskRunReq(SStreamMeta* pStreamMeta, bool* pScanIdle) { continue; } - noDataInWal = false; - if (pItem != NULL) { + noDataInWal = false; code = tAppendDataToInputQueue(pTask, pItem); if (code == TSDB_CODE_SUCCESS) { pTask->chkInfo.currentVer = walReaderGetCurrentVer(pTask->exec.pWalReader); diff --git a/source/libs/executor/inc/querytask.h b/source/libs/executor/inc/querytask.h index 9f8b749171..cdf37bcc6b 100644 --- a/source/libs/executor/inc/querytask.h +++ b/source/libs/executor/inc/querytask.h @@ -68,9 +68,6 @@ typedef struct { SQueryTableDataCond tableCond; SVersionRange fillHistoryVer; STimeWindow fillHistoryWindow; -// int64_t fillHistoryVer1; -// int64_t fillHisotryeKey1; - int64_t fillHistoryVer2; SStreamState* pState; int64_t dataVersion; int64_t checkPointId; diff --git a/source/libs/executor/src/executor.c b/source/libs/executor/src/executor.c index fab608c2b5..5f86f195b6 100644 --- a/source/libs/executor/src/executor.c +++ b/source/libs/executor/src/executor.c @@ -116,6 +116,16 @@ void resetTaskInfo(qTaskInfo_t tinfo) { clearStreamBlock(pTaskInfo->pRoot); } +void qResetStreamInfoTimeWindow(qTaskInfo_t tinfo) { + SExecTaskInfo* pTaskInfo = (SExecTaskInfo*) tinfo; + if (pTaskInfo == NULL) { + return; + } + + qDebug("%s set fill history start key:%"PRId64, GET_TASKID(pTaskInfo), INT64_MIN); + pTaskInfo->streamInfo.fillHistoryWindow.skey = INT64_MIN; +} + static int32_t doSetStreamBlock(SOperatorInfo* pOperator, void* input, size_t numOfBlocks, int32_t type, const char* id) { if (pOperator->operatorType != QUERY_NODE_PHYSICAL_PLAN_STREAM_SCAN) { if (pOperator->numOfDownstream == 0) { @@ -265,6 +275,7 @@ qTaskInfo_t qCreateQueueExecTaskInfo(void* msg, SReadHandle* pReaderHandle, int3 terrno = TSDB_CODE_OUT_OF_MEMORY; return NULL; } + pTaskInfo->pRoot = createRawScanOperatorInfo(pReaderHandle, pTaskInfo); if (NULL == pTaskInfo->pRoot) { terrno = TSDB_CODE_OUT_OF_MEMORY; @@ -314,8 +325,8 @@ qTaskInfo_t qCreateStreamExecTaskInfo(void* msg, SReadHandle* readers, int32_t v return NULL; } - struct SSubplan* pPlan = NULL; - int32_t code = qStringToSubplan(msg, &pPlan); + SSubplan* pPlan = NULL; + int32_t code = qStringToSubplan(msg, &pPlan); if (code != TSDB_CODE_SUCCESS) { terrno = code; return NULL; @@ -907,14 +918,6 @@ int32_t qStreamSourceScanParamForHistoryScanStep2(qTaskInfo_t tinfo, SVersionRan return 0; } -int32_t qStreamSourceRecoverStep2(qTaskInfo_t tinfo, int64_t ver) { - SExecTaskInfo* pTaskInfo = (SExecTaskInfo*)tinfo; - ASSERT(pTaskInfo->execModel == OPTR_EXEC_MODEL_STREAM); - pTaskInfo->streamInfo.fillHistoryVer2 = ver; - pTaskInfo->streamInfo.recoverStep = STREAM_RECOVER_STEP__PREPARE2; - return 0; -} - int32_t qStreamRecoverFinish(qTaskInfo_t tinfo) { SExecTaskInfo* pTaskInfo = (SExecTaskInfo*)tinfo; ASSERT(pTaskInfo->execModel == OPTR_EXEC_MODEL_STREAM); @@ -1056,6 +1059,9 @@ int32_t qStreamRecoverSetAllStepFinished(qTaskInfo_t tinfo) { SExecTaskInfo* pTaskInfo = (SExecTaskInfo*)tinfo; pTaskInfo->streamInfo.recoverStep1Finished = true; pTaskInfo->streamInfo.recoverStep2Finished = true; + + // reset the time window + pTaskInfo->streamInfo.fillHistoryWindow.skey = INT64_MIN; return 0; } diff --git a/source/libs/executor/src/executorInt.c b/source/libs/executor/src/executorInt.c index 472e7fed0a..eb55ab5e08 100644 --- a/source/libs/executor/src/executorInt.c +++ b/source/libs/executor/src/executorInt.c @@ -1051,44 +1051,6 @@ int32_t buildSessionResultDataBlock(SOperatorInfo* pOperator, void* pState, SSDa return TSDB_CODE_SUCCESS; } -void qStreamCloseTsdbReader(void* task) { - if (task == NULL) { - return; - } - - SExecTaskInfo* pTaskInfo = (SExecTaskInfo*)task; - SOperatorInfo* pOp = pTaskInfo->pRoot; - - qDebug("stream close tsdb reader, reset status uid:%" PRId64 " ts:%" PRId64, pTaskInfo->streamInfo.currentOffset.uid, - pTaskInfo->streamInfo.currentOffset.ts); - - // todo refactor, other thread may already use this read to extract data. - pTaskInfo->streamInfo.currentOffset = (STqOffsetVal){0}; - while (pOp->numOfDownstream == 1 && pOp->pDownstream[0]) { - SOperatorInfo* pDownstreamOp = pOp->pDownstream[0]; - if (pDownstreamOp->operatorType == QUERY_NODE_PHYSICAL_PLAN_STREAM_SCAN) { - SStreamScanInfo* pInfo = pDownstreamOp->info; - if (pInfo->pTableScanOp) { - STableScanInfo* pTSInfo = pInfo->pTableScanOp->info; - - setOperatorCompleted(pInfo->pTableScanOp); - while (pTaskInfo->owner != 0) { - taosMsleep(100); - qDebug("wait for the reader stopping"); - } - - pTaskInfo->storageAPI.tsdReader.tsdReaderClose(pTSInfo->base.dataReader); - pTSInfo->base.dataReader = NULL; - - // restore the status, todo refactor. - pInfo->pTableScanOp->status = OP_OPENED; - pTaskInfo->status = TASK_NOT_COMPLETED; - return; - } - } - } -} - void streamOpReleaseState(SOperatorInfo* pOperator) { SOperatorInfo* downstream = pOperator->pDownstream[0]; if (downstream->fpSet.releaseStreamStateFn) { diff --git a/source/libs/executor/src/scanoperator.c b/source/libs/executor/src/scanoperator.c index ca83ef6d51..7f510d6745 100644 --- a/source/libs/executor/src/scanoperator.c +++ b/source/libs/executor/src/scanoperator.c @@ -1800,8 +1800,8 @@ static SSDataBlock* doStreamScan(SOperatorInfo* pOperator) { pTSInfo->base.cond.endVersion, pTSInfo->base.cond.twindows.skey, pTSInfo->base.cond.twindows.ekey, id); pTaskInfo->streamInfo.recoverStep = STREAM_RECOVER_STEP__SCAN1; } else { - pTSInfo->base.cond.startVersion = pTaskInfo->streamInfo.fillHistoryVer.minVer + 1; - pTSInfo->base.cond.endVersion = pTaskInfo->streamInfo.fillHistoryVer2; + pTSInfo->base.cond.startVersion = pTaskInfo->streamInfo.fillHistoryVer.minVer; + pTSInfo->base.cond.endVersion = pTaskInfo->streamInfo.fillHistoryVer.maxVer; qDebug("stream recover step2, verRange:%" PRId64 " - %" PRId64", %s", pTSInfo->base.cond.startVersion, pTSInfo->base.cond.endVersion, id); pTaskInfo->streamInfo.recoverStep = STREAM_RECOVER_STEP__SCAN2; @@ -1873,7 +1873,7 @@ static SSDataBlock* doStreamScan(SOperatorInfo* pOperator) { TSKEY maxTs = pAPI->stateStore.updateInfoFillBlockData(pInfo->pUpdateInfo, pInfo->pRecoverRes, pInfo->primaryTsIndex); pInfo->twAggSup.maxTs = TMAX(pInfo->twAggSup.maxTs, maxTs); } else { - pInfo->pUpdateInfo->maxDataVersion = TMAX(pInfo->pUpdateInfo->maxDataVersion, pTaskInfo->streamInfo.fillHistoryVer2); + pInfo->pUpdateInfo->maxDataVersion = TMAX(pInfo->pUpdateInfo->maxDataVersion, pTaskInfo->streamInfo.fillHistoryVer.maxVer); doCheckUpdate(pInfo, pInfo->pRecoverRes->info.window.ekey, pInfo->pRecoverRes); } } @@ -2100,6 +2100,8 @@ FETCH_NEXT_BLOCK: STimeWindow* pWindow = &pTaskInfo->streamInfo.fillHistoryWindow; if (pWindow->skey != INT64_MIN) { + qDebug("%s filter for additional history window, skey:%"PRId64, id, pWindow->skey); + bool* p = taosMemoryCalloc(pBlock->info.rows, sizeof(bool)); bool hasUnqualified = false; @@ -2124,8 +2126,8 @@ FETCH_NEXT_BLOCK: pBlock->info.dataLoad = 1; blockDataUpdateTsWindow(pBlock, pInfo->primaryTsIndex); - qDebug("%" PRId64 " rows in datablock, update res:%" PRId64 " %s", pBlockInfo->rows, - pInfo->pUpdateDataRes->info.rows, id); + qDebug("%s %" PRId64 " rows in datablock, update res:%" PRId64, id, pBlockInfo->rows, + pInfo->pUpdateDataRes->info.rows); if (pBlockInfo->rows > 0 || pInfo->pUpdateDataRes->info.rows > 0) { break; } diff --git a/source/libs/stream/src/stream.c b/source/libs/stream/src/stream.c index b5c83bc10b..d0c65cc256 100644 --- a/source/libs/stream/src/stream.c +++ b/source/libs/stream/src/stream.c @@ -121,9 +121,10 @@ int32_t streamSchedExec(SStreamTask* pTask) { pRunReq->streamId = pTask->id.streamId; pRunReq->taskId = pTask->id.taskId; + qDebug("trigger to run s-task:%s", pTask->id.idStr); + SRpcMsg msg = {.msgType = TDMT_STREAM_TASK_RUN, .pCont = pRunReq, .contLen = sizeof(SStreamTaskRunReq)}; tmsgPutToQueue(pTask->pMsgCb, STREAM_QUEUE, &msg); - qDebug("trigger to run s-task:%s", pTask->id.idStr); } else { qDebug("s-task:%s not launch task since sched status:%d", pTask->id.idStr, pTask->status.schedStatus); } diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index 3c150e924f..4db4e0d9a6 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -332,7 +332,7 @@ int32_t updateCheckPointInfo(SStreamTask* pTask) { return TSDB_CODE_SUCCESS; } -static void waitForTaskTobeIdle(SStreamTask* pTask, SStreamTask* pStreamTask) { +static void waitForTaskIdle(SStreamTask* pTask, SStreamTask* pStreamTask) { // wait for the stream task to be idle int64_t st = taosGetTimestampMs(); @@ -367,12 +367,12 @@ static int32_t streamTransferStateToStreamTask(SStreamTask* pTask) { } // wait for the stream task to be idle - waitForTaskTobeIdle(pTask, pStreamTask); + waitForTaskIdle(pTask, pStreamTask); if (pStreamTask->info.taskLevel == TASK_LEVEL__SOURCE) { // update the scan data range for source task. - qDebug("s-task:%s level:%d stream task window %" PRId64 " - %" PRId64 " transfer to %" PRId64 " - %" PRId64 - ", status:%s, sched-status:%d", + qDebug("s-task:%s level:%d stream task window %" PRId64 " - %" PRId64 " update to %" PRId64 " - %" PRId64 + ", status:%s, sched-status:%d", pStreamTask->id.idStr, TASK_LEVEL__SOURCE, pTimeWindow->skey, pTimeWindow->ekey, INT64_MIN, pTimeWindow->ekey, streamGetTaskStatusStr(TASK_STATUS__NORMAL), pStreamTask->status.schedStatus); @@ -388,6 +388,7 @@ static int32_t streamTransferStateToStreamTask(SStreamTask* pTask) { // expand the query time window for stream scanner pTimeWindow->skey = INT64_MIN; + qResetStreamInfoTimeWindow(pStreamTask->exec.pExecutor); streamSetStatusNormal(pStreamTask); streamSchedExec(pStreamTask); diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 593f84cb8a..a245b2d4f6 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -438,7 +438,7 @@ int32_t streamLoadTasks(SStreamMeta* pMeta, int64_t ver) { return -1; } - ASSERT(pTask->status.checkDownstream == 0); + ASSERT(pTask->status.downstreamReady == 0); } tdbFree(pKey); diff --git a/source/libs/stream/src/streamRecover.c b/source/libs/stream/src/streamRecover.c index a025d01004..6ff7a365a9 100644 --- a/source/libs/stream/src/streamRecover.c +++ b/source/libs/stream/src/streamRecover.c @@ -97,16 +97,16 @@ int32_t streamTaskCheckDownstreamTasks(SStreamTask* pTask) { // serialize if (pTask->outputType == TASK_OUTPUT__FIXED_DISPATCH) { - qDebug("s-task:%s check single downstream task:0x%x(vgId:%d) ver:%" PRId64 "-%" PRId64 " window:%" PRId64 - "-%" PRId64, - pTask->id.idStr, req.downstreamTaskId, req.downstreamNodeId, pRange->range.minVer, pRange->range.maxVer, - pWindow->skey, pWindow->ekey); - req.reqId = tGenIdPI64(); req.downstreamNodeId = pTask->fixedEpDispatcher.nodeId; req.downstreamTaskId = pTask->fixedEpDispatcher.taskId; pTask->checkReqId = req.reqId; + qDebug("s-task:%s check single downstream task:0x%x(vgId:%d) ver:%" PRId64 "-%" PRId64 " window:%" PRId64 + "-%" PRId64 ", req:0x%" PRIx64, + pTask->id.idStr, req.downstreamTaskId, req.downstreamNodeId, pRange->range.minVer, pRange->range.maxVer, + pWindow->skey, pWindow->ekey, req.reqId); + streamDispatchCheckMsg(pTask, &req, pTask->fixedEpDispatcher.nodeId, &pTask->fixedEpDispatcher.epSet); } else if (pTask->outputType == TASK_OUTPUT__SHUFFLE_DISPATCH) { SArray* vgInfo = pTask->shuffleDispatcher.dbInfo.pVgroupInfos; @@ -129,7 +129,7 @@ int32_t streamTaskCheckDownstreamTasks(SStreamTask* pTask) { streamDispatchCheckMsg(pTask, &req, pVgInfo->vgId, &pVgInfo->epSet); } } else { - pTask->status.checkDownstream = 1; + pTask->status.downstreamReady = 1; qDebug("s-task:%s (vgId:%d) no downstream tasks, set downstream checked, try to launch scan-history-data, status:%s", pTask->id.idStr, pTask->info.nodeId, streamGetTaskStatusStr(pTask->status.taskStatus)); @@ -222,8 +222,8 @@ int32_t streamProcessCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* pRs } // set the downstream tasks have been checked flag - ASSERT(pTask->status.checkDownstream == 0); - pTask->status.checkDownstream = 1; + ASSERT(pTask->status.downstreamReady == 0); + pTask->status.downstreamReady = 1; ASSERT(pTask->status.taskStatus == TASK_STATUS__SCAN_HISTORY || pTask->status.taskStatus == TASK_STATUS__NORMAL); if (pTask->status.taskStatus == TASK_STATUS__SCAN_HISTORY) { @@ -286,23 +286,6 @@ int32_t streamSourceScanHistoryData(SStreamTask* pTask) { return streamScanExec(pTask, 100); } -int32_t streamSourceRecoverScanStep2(SStreamTask* pTask, int64_t ver) { - void* exec = pTask->exec.pExecutor; - const char* id = pTask->id.idStr; - - int64_t st = taosGetTimestampMs(); - qDebug("s-task:%s recover step2(blocking stage) started", id); - if (qStreamSourceRecoverStep2(exec, ver) < 0) { - } - - int32_t code = streamScanExec(pTask, 100); - - double el = (taosGetTimestampMs() - st) / 1000.0; - qDebug("s-task:%s recover step2(blocking stage) ended, elapsed time:%.2fs", id, el); - - return code; -} - int32_t streamDispatchScanHistoryFinishMsg(SStreamTask* pTask) { SStreamRecoverFinishReq req = { .streamId = pTask->id.streamId, .childId = pTask->info.selfChildId }; @@ -373,8 +356,8 @@ int32_t streamDispatchTransferStateMsg(SStreamTask* pTask) { // serialize if (pTask->outputType == TASK_OUTPUT__FIXED_DISPATCH) { - qDebug("s-task:%s send transfer state msg to downstream (fix-dispatch) to taskId:0x%x, status:%d", pTask->id.idStr, - pTask->fixedEpDispatcher.taskId, pTask->status.taskStatus); + qDebug("s-task:%s send transfer state msg to downstream (fix-dispatch) to taskId:0x%x, status:%s", pTask->id.idStr, + pTask->fixedEpDispatcher.taskId, streamGetTaskStatusStr(pTask->status.taskStatus)); req.taskId = pTask->fixedEpDispatcher.taskId; doDispatchTransferMsg(pTask, &req, pTask->fixedEpDispatcher.nodeId, &pTask->fixedEpDispatcher.epSet); @@ -693,7 +676,7 @@ void streamPrepareNdoCheckDownstream(SStreamTask* pTask) { pTask->id.idStr, pRange->window.skey, pRange->window.ekey, pRange->range.minVer, pRange->range.maxVer); } - ASSERT(pTask->status.checkDownstream == 0); + ASSERT(pTask->status.downstreamReady == 0); // check downstream tasks for itself streamTaskCheckDownstreamTasks(pTask); From 3710ea4acaff9240f8baca94b7e922c030b38219 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 28 Jun 2023 09:41:08 +0800 Subject: [PATCH 71/83] refactor: do some internal refactor. --- source/dnode/vnode/src/tq/tq.c | 14 ++++++-------- source/libs/stream/src/streamExec.c | 9 ++++++++- 2 files changed, 14 insertions(+), 9 deletions(-) diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index f68b725cdd..ed37605d08 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1112,8 +1112,8 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { // wait for the stream task get ready for scan history data while (((pStreamTask->status.downstreamReady == 0) && (pStreamTask->status.taskStatus != TASK_STATUS__STOP)) || pStreamTask->status.taskStatus == TASK_STATUS__SCAN_HISTORY) { - tqDebug("s-task:%s level:%d not ready for halt, wait for 100ms and recheck", pStreamTask->id.idStr, - pStreamTask->info.taskLevel); + tqDebug("s-task:%s level:%d related stream task:%s not ready for halt, wait for it continue and recheck in 100ms", + pTask->id.idStr, pStreamTask->id.idStr, pStreamTask->info.taskLevel); taosMsleep(100); } @@ -1181,11 +1181,7 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { streamTryExec(pTask); pTask->status.taskStatus = TASK_STATUS__DROPPING; - tqDebug("s-task:%s set status to be dropping", pId); - - // transfer the ownership of executor state - streamTaskReleaseState(pTask); - streamTaskReloadState(pStreamTask); + tqDebug("s-task:%s scan-history-task set status to be dropping", pId); streamMetaSaveTask(pMeta, pTask); streamMetaSaveTask(pMeta, pStreamTask); @@ -1236,12 +1232,14 @@ int32_t tqProcessTaskTransferStateReq(STQ* pTq, int64_t sversion, char* msg, int SStreamTask* pTask = streamMetaAcquireTask(pTq->pStreamMeta, req.taskId); if (pTask == NULL) { - tqError("failed to find task:0x%x", req.taskId); + tqError("failed to find task:0x%x, it may have been dropped already", req.taskId); return -1; } // transfer the ownership of executor state streamTaskReleaseState(pTask); + tqDebug("s-task:%s receive state transfer req", pTask->id.idStr); + SStreamTask* pStreamTask = streamMetaAcquireTask(pTq->pStreamMeta, pTask->streamTaskId.taskId); streamTaskReloadState(pStreamTask); diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index 4db4e0d9a6..6e1804b08e 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -351,7 +351,7 @@ static void waitForTaskIdle(SStreamTask* pTask, SStreamTask* pStreamTask) { static int32_t streamTransferStateToStreamTask(SStreamTask* pTask) { SStreamTask* pStreamTask = streamMetaAcquireTask(pTask->pMeta, pTask->streamTaskId.taskId); - qDebug("s-task:%s scan history task end, update stream task:%s info and launch it", pTask->id.idStr, pStreamTask->id.idStr); + qDebug("s-task:%s scan history task end, update stream task:%s info, transfer exec state", pTask->id.idStr, pStreamTask->id.idStr); // todo handle stream task is dropped here @@ -390,7 +390,12 @@ static int32_t streamTransferStateToStreamTask(SStreamTask* pTask) { pTimeWindow->skey = INT64_MIN; qResetStreamInfoTimeWindow(pStreamTask->exec.pExecutor); + // transfer the ownership of executor state + streamTaskReleaseState(pTask); + streamTaskReloadState(pStreamTask); + streamSetStatusNormal(pStreamTask); + streamSchedExec(pStreamTask); streamMetaReleaseTask(pTask->pMeta, pStreamTask); return TSDB_CODE_SUCCESS; @@ -584,6 +589,7 @@ int32_t streamTryExec(SStreamTask* pTask) { } int32_t streamTaskReleaseState(SStreamTask* pTask) { + qDebug("s-task:%s release exec state", pTask->id.idStr); void* pExecutor = pTask->exec.pExecutor; if (pExecutor != NULL) { int32_t code = qStreamOperatorReleaseState(pExecutor); @@ -594,6 +600,7 @@ int32_t streamTaskReleaseState(SStreamTask* pTask) { } int32_t streamTaskReloadState(SStreamTask* pTask) { + qDebug("s-task:%s reload exec state", pTask->id.idStr); void* pExecutor = pTask->exec.pExecutor; if (pExecutor != NULL) { int32_t code = qStreamOperatorReloadState(pExecutor); From 57fcd553f8a7e7b99b34030d39f45708fa721d9c Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 28 Jun 2023 13:13:13 +0800 Subject: [PATCH 72/83] refactor: do some internal refactor. --- include/libs/stream/tstream.h | 5 +++-- source/dnode/snode/src/snode.c | 2 +- source/dnode/vnode/src/tq/tq.c | 9 ++++++--- source/libs/stream/src/stream.c | 25 ++++++++++++++++++------- source/libs/stream/src/streamDispatch.c | 2 +- source/libs/stream/src/streamMeta.c | 11 ++++++++--- source/libs/stream/src/streamRecover.c | 18 +++++++++--------- tests/script/sh/deploy.sh | 2 +- 8 files changed, 47 insertions(+), 27 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index ef533dc969..dbcc31a35e 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -333,7 +333,8 @@ struct SStreamTask { // trigger int8_t triggerStatus; int64_t triggerParam; - void* timer; + void* schedTimer; + void* launchTaskTimer; SMsgCb* pMsgCb; // msg handle SStreamState* pState; // state backend @@ -550,7 +551,7 @@ int32_t tInitStreamDispatchReq(SStreamDispatchReq* pReq, const SStreamTask* pTas int64_t dstTaskId); void tDeleteStreamDispatchReq(SStreamDispatchReq* pReq); -int32_t streamSetupTrigger(SStreamTask* pTask); +int32_t streamSetupScheduleTrigger(SStreamTask* pTask); int32_t streamProcessRunReq(SStreamTask* pTask); int32_t streamProcessDispatchMsg(SStreamTask* pTask, SStreamDispatchReq* pReq, SRpcMsg* pMsg, bool exec); diff --git a/source/dnode/snode/src/snode.c b/source/dnode/snode/src/snode.c index d2fcf166b4..3d9adf8156 100644 --- a/source/dnode/snode/src/snode.c +++ b/source/dnode/snode/src/snode.c @@ -90,7 +90,7 @@ int32_t sndExpandTask(SSnode *pSnode, SStreamTask *pTask, int64_t ver) { pTask->exec.pExecutor = qCreateStreamExecTaskInfo(pTask->exec.qmsg, &handle, 0); ASSERT(pTask->exec.pExecutor); - streamSetupTrigger(pTask); + streamSetupScheduleTrigger(pTask); qDebug("snode:%d expand stream task on snode, s-task:%s, checkpoint ver:%" PRId64 " child id:%d, level:%d", SNODE_HANDLE, pTask->id.idStr, pTask->chkInfo.version, pTask->info.selfChildId, pTask->info.taskLevel); diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index ed37605d08..47a2eb6d3f 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -889,11 +889,12 @@ int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t ver) { pTask->exec.pWalReader = walOpenReader(pTq->pVnode->pWal, &cond); } - streamSetupTrigger(pTask); + streamSetupScheduleTrigger(pTask); - tqInfo("vgId:%d expand stream task, s-task:%s, checkpoint ver:%" PRId64 " child id:%d, level:%d, scan-history:%d", + tqInfo("vgId:%d expand stream task, s-task:%s, checkpoint ver:%" PRId64 + " child id:%d, level:%d, scan-history:%d, trigger:%" PRId64 " ms", vgId, pTask->id.idStr, pTask->chkInfo.version, pTask->info.selfChildId, pTask->info.taskLevel, - pTask->info.fillHistory); + pTask->info.fillHistory, pTask->triggerParam); // next valid version will add one pTask->chkInfo.version += 1; @@ -1189,9 +1190,11 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { streamMetaReleaseTask(pMeta, pTask); streamMetaReleaseTask(pMeta, pStreamTask); + taosWLockLatch(&pMeta->lock); if (streamMetaCommit(pTask->pMeta) < 0) { // persist to disk } + taosWUnLockLatch(&pMeta->lock); } else { // todo update the chkInfo version for current task. // this task has an associated history stream task, so we need to scan wal from the end version of diff --git a/source/libs/stream/src/stream.c b/source/libs/stream/src/stream.c index d0c65cc256..f528080afe 100644 --- a/source/libs/stream/src/stream.c +++ b/source/libs/stream/src/stream.c @@ -64,14 +64,21 @@ char* createStreamTaskIdStr(int64_t streamId, int32_t taskId) { void streamSchedByTimer(void* param, void* tmrId) { SStreamTask* pTask = (void*)param; + int8_t status = atomic_load_8(&pTask->triggerStatus); + qDebug("s-task:%s in scheduler timer, trigger status:%d", pTask->id.idStr, status); + if (streamTaskShouldStop(&pTask->status) || streamTaskShouldPause(&pTask->status)) { streamMetaReleaseTask(NULL, pTask); + qDebug("s-task:%s jump out of schedTimer", pTask->id.idStr); return; } - if (atomic_load_8(&pTask->triggerStatus) == TASK_TRIGGER_STATUS__ACTIVE) { + if (status == TASK_TRIGGER_STATUS__ACTIVE) { SStreamTrigger* trigger = taosAllocateQitem(sizeof(SStreamTrigger), DEF_QITEM, 0); - if (trigger == NULL) return; + if (trigger == NULL) { + return; + } + trigger->type = STREAM_INPUT__GET_RES; trigger->pBlock = taosMemoryCalloc(1, sizeof(SSDataBlock)); if (trigger->pBlock == NULL) { @@ -84,23 +91,27 @@ void streamSchedByTimer(void* param, void* tmrId) { if (tAppendDataToInputQueue(pTask, (SStreamQueueItem*)trigger) < 0) { taosFreeQitem(trigger); - taosTmrReset(streamSchedByTimer, (int32_t)pTask->triggerParam, pTask, streamEnv.timer, &pTask->timer); + taosTmrReset(streamSchedByTimer, (int32_t)pTask->triggerParam, pTask, streamEnv.timer, &pTask->schedTimer); return; } streamSchedExec(pTask); } - taosTmrReset(streamSchedByTimer, (int32_t)pTask->triggerParam, pTask, streamEnv.timer, &pTask->timer); + taosTmrReset(streamSchedByTimer, (int32_t)pTask->triggerParam, pTask, streamEnv.timer, &pTask->schedTimer); } -int32_t streamSetupTrigger(SStreamTask* pTask) { +int32_t streamSetupScheduleTrigger(SStreamTask* pTask) { if (pTask->triggerParam != 0) { int32_t ref = atomic_add_fetch_32(&pTask->refCnt, 1); - ASSERT(ref == 2); - pTask->timer = taosTmrStart(streamSchedByTimer, (int32_t)pTask->triggerParam, pTask, streamEnv.timer); + ASSERT(ref == 2 && pTask->schedTimer == NULL); + + qDebug("s-task:%s setup scheduler trigger, delay:%d ms", pTask->id.idStr, pTask->triggerParam); + + pTask->schedTimer = taosTmrStart(streamSchedByTimer, (int32_t)pTask->triggerParam, pTask, streamEnv.timer); pTask->triggerStatus = TASK_TRIGGER_STATUS__INACTIVE; } + return 0; } diff --git a/source/libs/stream/src/streamDispatch.c b/source/libs/stream/src/streamDispatch.c index 57acc6223f..c7e156bcf6 100644 --- a/source/libs/stream/src/streamDispatch.c +++ b/source/libs/stream/src/streamDispatch.c @@ -557,7 +557,7 @@ static void doRetryDispatchData(void* param, void* tmrId) { void streamRetryDispatchStreamBlock(SStreamTask* pTask, int64_t waitDuration) { qError("s-task:%s dispatch data in %"PRId64"ms", pTask->id.idStr, waitDuration); - taosTmrReset(doRetryDispatchData, waitDuration, pTask, streamEnv.timer, &pTask->timer); + taosTmrReset(doRetryDispatchData, waitDuration, pTask, streamEnv.timer, &pTask->launchTaskTimer); } int32_t streamDispatchStreamBlock(SStreamTask* pTask) { diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index a245b2d4f6..8242f84312 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -139,9 +139,14 @@ void streamMetaClose(SStreamMeta* pMeta) { } SStreamTask* pTask = *(SStreamTask**)pIter; - if (pTask->timer) { - taosTmrStop(pTask->timer); - pTask->timer = NULL; + if (pTask->schedTimer) { + taosTmrStop(pTask->schedTimer); + pTask->schedTimer = NULL; + } + + if (pTask->launchTaskTimer) { + taosTmrStop(pTask->launchTaskTimer); + pTask->launchTaskTimer = NULL; } tFreeStreamTask(pTask); diff --git a/source/libs/stream/src/streamRecover.c b/source/libs/stream/src/streamRecover.c index 6ff7a365a9..8e4d50bcf2 100644 --- a/source/libs/stream/src/streamRecover.c +++ b/source/libs/stream/src/streamRecover.c @@ -441,7 +441,7 @@ static void tryLaunchHistoryTask(void* param, void* tmrId) { SStreamTaskRetryInfo* pInfo = param; SStreamMeta* pMeta = pInfo->pMeta; - qDebug("s-task:0x%x in timer to launch history task", pInfo->taskId); + qDebug("s-task:0x%x in timer to launch related history task", pInfo->taskId); taosWLockLatch(&pMeta->lock); SStreamTask** ppTask = (SStreamTask**)taosHashGet(pMeta->pTasks, &pInfo->taskId, sizeof(int32_t)); @@ -472,7 +472,7 @@ static void tryLaunchHistoryTask(void* param, void* tmrId) { "destroyed, or should stop exec", pTask->id.idStr, pMeta->vgId, pStatus, pTask->historyTaskId.taskId); - taosTmrReset(tryLaunchHistoryTask, 100, pInfo, streamEnv.timer, &pTask->timer); + taosTmrReset(tryLaunchHistoryTask, 100, pInfo, streamEnv.timer, &pTask->launchTaskTimer); streamMetaReleaseTask(pMeta, pTask); return; } @@ -486,7 +486,7 @@ static void tryLaunchHistoryTask(void* param, void* tmrId) { pTask->status.timerActive = 0; streamMetaReleaseTask(pMeta, pTask); } else { - qError("s-task:0x%x failed to load task, it may have been destoryed", pInfo->taskId); + qError("s-task:0x%x failed to load task, it may have been destroyed", pInfo->taskId); } taosMemoryFree(pInfo); @@ -508,18 +508,18 @@ int32_t streamCheckHistoryTaskDownstrem(SStreamTask* pTask) { pInfo->taskId = pTask->id.taskId; pInfo->pMeta = pTask->pMeta; - if (pTask->timer == NULL) { - pTask->timer = taosTmrStart(tryLaunchHistoryTask, 100, pInfo, streamEnv.timer); - if (pTask->timer == NULL) { + if (pTask->launchTaskTimer == NULL) { + pTask->launchTaskTimer = taosTmrStart(tryLaunchHistoryTask, 100, pInfo, streamEnv.timer); + if (pTask->launchTaskTimer == NULL) { // todo failed to create timer } else { pTask->status.timerActive = 1; // timer is active - qDebug("s-task:%s set time active flag", pTask->id.idStr); + qDebug("s-task:%s set timer active flag", pTask->id.idStr); } } else { // timer exists pTask->status.timerActive = 1; - qDebug("s-task:%s set time active flag", pTask->id.idStr); - taosTmrReset(tryLaunchHistoryTask, 100, pInfo, streamEnv.timer, &pTask->timer); + qDebug("s-task:%s set timer active flag, task timer not null", pTask->id.idStr); + taosTmrReset(tryLaunchHistoryTask, 100, pInfo, streamEnv.timer, &pTask->launchTaskTimer); } // try again in 500ms diff --git a/tests/script/sh/deploy.sh b/tests/script/sh/deploy.sh index 7da8da09bf..5b1773e664 100755 --- a/tests/script/sh/deploy.sh +++ b/tests/script/sh/deploy.sh @@ -118,7 +118,7 @@ echo "statusInterval 1" >> $TAOS_CFG echo "dataDir $DATA_DIR" >> $TAOS_CFG echo "logDir $LOG_DIR" >> $TAOS_CFG echo "debugFlag 0" >> $TAOS_CFG -echo "tmrDebugFlag 131" >> $TAOS_CFG +echo "tmrDebugFlag 143" >> $TAOS_CFG echo "uDebugFlag 143" >> $TAOS_CFG echo "rpcDebugFlag 143" >> $TAOS_CFG echo "jniDebugFlag 143" >> $TAOS_CFG From f4309603e317831e99218864a7c89e9a178aabdd Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 28 Jun 2023 14:05:38 +0800 Subject: [PATCH 73/83] fix(stream): fix syntax error. --- 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 daf06c81d1..c0be5f99c1 100644 --- a/source/libs/executor/src/tsort.c +++ b/source/libs/executor/src/tsort.c @@ -1063,7 +1063,7 @@ static STupleHandle* tsortPQSortNextTuple(SSortHandle* pHandle) { if (!pData) { colDataSetNULL(bdGetColumnInfoData(pHandle->pDataBlock, i), 0); } else { - colDataAppend(bdGetColumnInfoData(pHandle->pDataBlock, i), 0, pData, false); + colDataSetVal(bdGetColumnInfoData(pHandle->pDataBlock, i), 0, pData, false); } } pHandle->pDataBlock->info.rows++; From 78739e4bdb8e70a1425f31a7f4ccaa6ecf565f15 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 28 Jun 2023 14:21:11 +0800 Subject: [PATCH 74/83] fix(stream): fix syntax error. --- source/libs/stream/src/stream.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/libs/stream/src/stream.c b/source/libs/stream/src/stream.c index f528080afe..728f435d76 100644 --- a/source/libs/stream/src/stream.c +++ b/source/libs/stream/src/stream.c @@ -106,7 +106,7 @@ int32_t streamSetupScheduleTrigger(SStreamTask* pTask) { int32_t ref = atomic_add_fetch_32(&pTask->refCnt, 1); ASSERT(ref == 2 && pTask->schedTimer == NULL); - qDebug("s-task:%s setup scheduler trigger, delay:%d ms", pTask->id.idStr, pTask->triggerParam); + qDebug("s-task:%s setup scheduler trigger, delay:%"PRId64" ms", pTask->id.idStr, pTask->triggerParam); pTask->schedTimer = taosTmrStart(streamSchedByTimer, (int32_t)pTask->triggerParam, pTask, streamEnv.timer); pTask->triggerStatus = TASK_TRIGGER_STATUS__INACTIVE; From 8867a572620bb2feafceaa99dabf888c04d41e76 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 28 Jun 2023 14:48:59 +0800 Subject: [PATCH 75/83] fix(stream): fix syntax error. --- source/dnode/vnode/src/tq/tq.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 47a2eb6d3f..761def6dc6 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1114,7 +1114,7 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { while (((pStreamTask->status.downstreamReady == 0) && (pStreamTask->status.taskStatus != TASK_STATUS__STOP)) || pStreamTask->status.taskStatus == TASK_STATUS__SCAN_HISTORY) { tqDebug("s-task:%s level:%d related stream task:%s not ready for halt, wait for it continue and recheck in 100ms", - pTask->id.idStr, pStreamTask->id.idStr, pStreamTask->info.taskLevel); + pTask->id.idStr, pTask->info.taskLevel, pStreamTask->id.idStr); taosMsleep(100); } From 2fc5eeb8106127aed2596ce4f3195a6ae2fff2d4 Mon Sep 17 00:00:00 2001 From: liuyao <54liuyao@163.com> Date: Wed, 28 Jun 2023 17:20:08 +0800 Subject: [PATCH 76/83] session win range --- include/libs/executor/executor.h | 3 +- source/dnode/vnode/src/tq/tq.c | 12 +++- source/libs/executor/inc/executorInt.h | 2 + source/libs/executor/src/timewindowoperator.c | 55 +++++++++++++++++-- 4 files changed, 63 insertions(+), 9 deletions(-) diff --git a/include/libs/executor/executor.h b/include/libs/executor/executor.h index f4713f7a6f..3bef15f3a7 100644 --- a/include/libs/executor/executor.h +++ b/include/libs/executor/executor.h @@ -56,7 +56,8 @@ typedef struct { void* pStateBackend; struct SStorageAPI api; - int8_t fillHistory; + int8_t fillHistory; + STimeWindow winRange; } SReadHandle; // in queue mode, data streams are seperated by msg diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 761def6dc6..250c94c2f9 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -826,7 +826,11 @@ int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t ver) { return -1; } - SReadHandle handle = {.vnode = pTq->pVnode, .initTqReader = 1, .pStateBackend = pTask->pState, .fillHistory = pTask->info.fillHistory}; + SReadHandle handle = {.vnode = pTq->pVnode, + .initTqReader = 1, + .pStateBackend = pTask->pState, + .fillHistory = pTask->info.fillHistory, + .winRange = pTask->dataRange.window}; initStorageAPI(&handle.api); pTask->exec.pExecutor = qCreateStreamExecTaskInfo(pTask->exec.qmsg, &handle, vgId); @@ -849,7 +853,11 @@ int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t ver) { } int32_t numOfVgroups = (int32_t)taosArrayGetSize(pTask->pUpstreamEpInfoList); - SReadHandle handle = {.vnode = NULL, .numOfVgroups = numOfVgroups, .pStateBackend = pTask->pState, .fillHistory = pTask->info.fillHistory}; + SReadHandle handle = {.vnode = NULL, + .numOfVgroups = numOfVgroups, + .pStateBackend = pTask->pState, + .fillHistory = pTask->info.fillHistory, + .winRange = pTask->dataRange.window}; initStorageAPI(&handle.api); pTask->exec.pExecutor = qCreateStreamExecTaskInfo(pTask->exec.qmsg, &handle, vgId); diff --git a/source/libs/executor/inc/executorInt.h b/source/libs/executor/inc/executorInt.h index 522f648ccc..0ba9aae133 100644 --- a/source/libs/executor/inc/executorInt.h +++ b/source/libs/executor/inc/executorInt.h @@ -285,6 +285,8 @@ typedef struct SStreamAggSupporter { int16_t stateKeyType; SDiskbasedBuf* pResultBuf; SStateStore stateStore; + STimeWindow winRange; + SStorageAPI* pSessionAPI; } SStreamAggSupporter; typedef struct SWindowSupporter { diff --git a/source/libs/executor/src/timewindowoperator.c b/source/libs/executor/src/timewindowoperator.c index 13d24aa531..c4111ded92 100644 --- a/source/libs/executor/src/timewindowoperator.c +++ b/source/libs/executor/src/timewindowoperator.c @@ -2966,7 +2966,7 @@ void initDownStream(SOperatorInfo* downstream, SStreamAggSupporter* pAggSup, uin } int32_t initStreamAggSupporter(SStreamAggSupporter* pSup, SqlFunctionCtx* pCtx, int32_t numOfOutput, int64_t gap, - SStreamState* pState, int32_t keySize, int16_t keyType, SStateStore* pStore) { + SStreamState* pState, int32_t keySize, int16_t keyType, SStateStore* pStore, SReadHandle* pHandle, SStorageAPI* pApi) { pSup->resultRowSize = keySize + getResultRowSize(pCtx, numOfOutput); pSup->pScanBlock = createSpecialDataBlock(STREAM_CLEAR); pSup->gap = gap; @@ -3008,6 +3008,16 @@ int32_t initStreamAggSupporter(SStreamAggSupporter* pSup, SqlFunctionCtx* pCtx, pCtx[i].saveHandle.pBuf = pSup->pResultBuf; } + if (pHandle) { + pSup->winRange = pHandle->winRange; + // temporary + if (pSup->winRange.ekey <= 0) { + pSup->winRange.ekey = INT64_MAX; + } + } + + pSup->pSessionAPI = pApi; + return TSDB_CODE_SUCCESS; } @@ -3035,6 +3045,13 @@ void getCurSessionWindow(SStreamAggSupporter* pAggSup, TSKEY startTs, TSKEY endT bool isInvalidSessionWin(SResultWindowInfo* pWinInfo) { return pWinInfo->sessionWin.win.skey == 0; } +bool inWinRange(STimeWindow* range, STimeWindow* cur) { + if (cur->skey >= range->skey && cur->ekey <= range->ekey) { + return true; + } + return false; +} + void setSessionOutputBuf(SStreamAggSupporter* pAggSup, TSKEY startTs, TSKEY endTs, uint64_t groupId, SResultWindowInfo* pCurWin) { pCurWin->sessionWin.groupId = groupId; @@ -3043,6 +3060,12 @@ void setSessionOutputBuf(SStreamAggSupporter* pAggSup, TSKEY startTs, TSKEY endT int32_t size = pAggSup->resultRowSize; int32_t code = pAggSup->stateStore.streamStateSessionAddIfNotExist(pAggSup->pState, &pCurWin->sessionWin, pAggSup->gap, &pCurWin->pOutputBuf, &size); + if (code == TSDB_CODE_SUCCESS && !inWinRange(&pAggSup->winRange, &pCurWin->sessionWin.win)) { + code = TSDB_CODE_FAILED; + releaseOutputBuf(pAggSup->pState, NULL, (SResultRow*)pCurWin->pOutputBuf, &pAggSup->pSessionAPI->stateStore); + pCurWin->pOutputBuf = taosMemoryMalloc(size); + } + if (code == TSDB_CODE_SUCCESS) { pCurWin->isOutput = true; } else { @@ -3189,7 +3212,8 @@ static void compactSessionWindow(SOperatorInfo* pOperator, SResultWindowInfo* pC while (1) { SResultWindowInfo winInfo = {0}; SStreamStateCur* pCur = getNextSessionWinInfo(pAggSup, pStUpdated, pCurWin, &winInfo); - if (!IS_VALID_SESSION_WIN(winInfo) || !isInWindow(pCurWin, winInfo.sessionWin.win.skey, pAggSup->gap)) { + if (!IS_VALID_SESSION_WIN(winInfo) || !isInWindow(pCurWin, winInfo.sessionWin.win.skey, pAggSup->gap) || + !inWinRange(&pAggSup->winRange, &winInfo.sessionWin.win)) { taosMemoryFree(winInfo.pOutputBuf); pAPI->stateStore.streamStateFreeCur(pCur); break; @@ -3413,8 +3437,12 @@ static void rebuildSessionWindow(SOperatorInfo* pOperator, SArray* pWinArray, SS SResultWindowInfo childWin = {0}; childWin.sessionWin = *pWinKey; int32_t code = getSessionWinBuf(pChAggSup, pCur, &childWin); - if (code == TSDB_CODE_SUCCESS && pWinKey->win.skey <= childWin.sessionWin.win.skey && - childWin.sessionWin.win.ekey <= pWinKey->win.ekey) { + + if (code == TSDB_CODE_SUCCESS && !inWinRange(&pAggSup->winRange, &childWin.sessionWin.win)) { + continue; + } + + if (code == TSDB_CODE_SUCCESS && inWinRange(&pWinKey->win, &childWin.sessionWin.win)) { if (num == 0) { setSessionOutputBuf(pAggSup, pWinKey->win.skey, pWinKey->win.ekey, pWinKey->groupId, &parentWin); code = initSessionOutputBuf(&parentWin, &pResult, pSup->pCtx, numOfOutput, pSup->rowEntryInfoOffset); @@ -3678,9 +3706,16 @@ void streamSessionReleaseState(SOperatorInfo* pOperator) { } } +void resetWinRange(STimeWindow* winRange) { + winRange->skey = INT16_MIN; + winRange->skey = INT16_MAX; +} + void streamSessionReloadState(SOperatorInfo* pOperator) { SStreamSessionAggOperatorInfo* pInfo = pOperator->info; SStreamAggSupporter* pAggSup = &pInfo->streamAggSup; + resetWinRange(&pAggSup->winRange); + SResultWindowInfo winInfo = {0}; int32_t size = 0; void* pBuf = NULL; @@ -3734,7 +3769,7 @@ SOperatorInfo* createStreamSessionAggOperatorInfo(SOperatorInfo* downstream, SPh } code = initStreamAggSupporter(&pInfo->streamAggSup, pSup->pCtx, numOfCols, pSessionNode->gap, - pTaskInfo->streamInfo.pState, 0, 0, &pTaskInfo->storageAPI.stateStore); + pTaskInfo->streamInfo.pState, 0, 0, &pTaskInfo->storageAPI.stateStore, pHandle, &pTaskInfo->storageAPI); if (code != TSDB_CODE_SUCCESS) { goto _error; } @@ -4024,6 +4059,12 @@ void setStateOutputBuf(SStreamAggSupporter* pAggSup, TSKEY ts, uint64_t groupId, pCurWin->pStateKey->pData = (char*)pCurWin->pStateKey + sizeof(SStateKeys); pCurWin->pStateKey->isNull = false; + if (code == TSDB_CODE_SUCCESS && !inWinRange(&pAggSup->winRange, &pCurWin->winInfo.sessionWin.win)) { + code = TSDB_CODE_FAILED; + releaseOutputBuf(pAggSup->pState, NULL, (SResultRow*)pCurWin->winInfo.pOutputBuf, &pAggSup->pSessionAPI->stateStore); + pCurWin->winInfo.pOutputBuf = taosMemoryMalloc(size); + } + if (code == TSDB_CODE_SUCCESS) { pCurWin->winInfo.isOutput = true; } else if (pKeyData) { @@ -4292,6 +4333,8 @@ static void compactStateWindow(SOperatorInfo* pOperator, SResultWindowInfo* pCur void streamStateReloadState(SOperatorInfo* pOperator) { SStreamSessionAggOperatorInfo* pInfo = pOperator->info; SStreamAggSupporter* pAggSup = &pInfo->streamAggSup; + resetWinRange(&pAggSup->winRange); + SSessionKey seKey = {.win.skey = INT64_MIN, .win.ekey = INT64_MIN, .groupId = 0}; int32_t size = 0; void* pBuf = NULL; @@ -4361,7 +4404,7 @@ SOperatorInfo* createStreamStateAggOperatorInfo(SOperatorInfo* downstream, SPhys int32_t keySize = sizeof(SStateKeys) + pColNode->node.resType.bytes; int16_t type = pColNode->node.resType.type; code = initStreamAggSupporter(&pInfo->streamAggSup, pSup->pCtx, numOfCols, 0, pTaskInfo->streamInfo.pState, keySize, - type, &pTaskInfo->storageAPI.stateStore); + type, &pTaskInfo->storageAPI.stateStore, pHandle, &pTaskInfo->storageAPI); if (code != TSDB_CODE_SUCCESS) { goto _error; } From 863da2ae05f1a6252148f14fddc4cbe13164cf4d Mon Sep 17 00:00:00 2001 From: liuyao <54liuyao@163.com> Date: Mon, 3 Jul 2023 11:10:38 +0800 Subject: [PATCH 77/83] set task status of ins_stream_tasks --- source/dnode/mnode/impl/src/mndStream.c | 43 +++++++++++++++++++++++-- 1 file changed, 40 insertions(+), 3 deletions(-) diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index be6cfae0be..8c6701edd9 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -1303,9 +1303,29 @@ static int32_t mndRetrieveStreamTask(SRpcMsg *pReq, SShowObj *pShow, SSDataBlock // status char status[20 + VARSTR_HEADER_SIZE] = {0}; - char status2[20] = {0}; - strcpy(status, "normal"); - STR_WITH_MAXSIZE_TO_VARSTR(status, status2, sizeof(status)); + int8_t taskStatus = atomic_load_8(&pTask->status.taskStatus); + if (taskStatus == TASK_STATUS__NORMAL) { + memcpy(varDataVal(status), "normal", 6); + varDataSetLen(status, 6); + } else if (taskStatus == TASK_STATUS__DROPPING) { + memcpy(varDataVal(status), "dropping", 8); + varDataSetLen(status, 8); + } else if (taskStatus == TASK_STATUS__FAIL) { + memcpy(varDataVal(status), "fail", 4); + varDataSetLen(status, 4); + } else if (taskStatus == TASK_STATUS__STOP) { + memcpy(varDataVal(status), "stop", 4); + varDataSetLen(status, 4); + } else if (taskStatus == TASK_STATUS__SCAN_HISTORY) { + memcpy(varDataVal(status), "history", 6); + varDataSetLen(status, 6); + } else if (taskStatus == TASK_STATUS__HALT) { + memcpy(varDataVal(status), "halt", 4); + varDataSetLen(status, 4); + } else if (taskStatus == TASK_STATUS__PAUSE) { + memcpy(varDataVal(status), "pause", 5); + varDataSetLen(status, 5); + } pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); colDataSetVal(pColInfo, numOfRows, (const char *)&status, false); @@ -1358,6 +1378,11 @@ int32_t mndPauseAllStreamTaskImpl(STrans *pTrans, SArray* tasks) { if (pTask->info.taskLevel != TASK_LEVEL__SINK && mndPauseStreamTask(pTrans, pTask) < 0) { return -1; } + + if (atomic_load_8(&pTask->status.taskStatus) != TASK_STATUS__PAUSE) { + atomic_store_8(&pTask->status.keepTaskStatus, pTask->status.taskStatus); + atomic_store_8(&pTask->status.taskStatus, TASK_STATUS__PAUSE); + } } } return 0; @@ -1412,6 +1437,10 @@ static int32_t mndProcessPauseStreamReq(SRpcMsg *pReq) { } } + if (pStream->status == STREAM_STATUS__PAUSE) { + return 0; + } + if (mndCheckDbPrivilegeByName(pMnode, pReq->info.conn.user, MND_OPER_WRITE_DB, pStream->targetDb) != 0) { sdbRelease(pMnode->pSdb, pStream); return -1; @@ -1492,6 +1521,10 @@ int32_t mndResumeAllStreamTasks(STrans *pTrans, SStreamObj *pStream, int8_t igUn if (pTask->info.taskLevel != TASK_LEVEL__SINK && mndResumeStreamTask(pTrans, pTask, igUntreated) < 0) { return -1; } + + if (atomic_load_8(&pTask->status.taskStatus) == TASK_STATUS__PAUSE) { + atomic_store_8(&pTask->status.taskStatus, pTask->status.keepTaskStatus); + } } } // pStream->pHTasksList is null @@ -1521,6 +1554,10 @@ static int32_t mndProcessResumeStreamReq(SRpcMsg *pReq) { } } + if (pStream->status != STREAM_STATUS__PAUSE) { + return 0; + } + if (mndCheckDbPrivilegeByName(pMnode, pReq->info.conn.user, MND_OPER_WRITE_DB, pStream->targetDb) != 0) { sdbRelease(pMnode->pSdb, pStream); return -1; From bc5e709d3166c91d9156c5d87ad8723ee164712f Mon Sep 17 00:00:00 2001 From: liuyao <54liuyao@163.com> Date: Mon, 3 Jul 2023 14:00:30 +0800 Subject: [PATCH 78/83] set task status of ins_stream_tasks --- source/dnode/mnode/impl/src/mndStream.c | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index 8c6701edd9..9d8948650a 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -1317,8 +1317,8 @@ static int32_t mndRetrieveStreamTask(SRpcMsg *pReq, SShowObj *pShow, SSDataBlock memcpy(varDataVal(status), "stop", 4); varDataSetLen(status, 4); } else if (taskStatus == TASK_STATUS__SCAN_HISTORY) { - memcpy(varDataVal(status), "history", 6); - varDataSetLen(status, 6); + memcpy(varDataVal(status), "history", 7); + varDataSetLen(status, 7); } else if (taskStatus == TASK_STATUS__HALT) { memcpy(varDataVal(status), "halt", 4); varDataSetLen(status, 4); From 2dbccc15734187c7b3480e8eec979c776d1b3dad Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 3 Jul 2023 17:52:17 +0800 Subject: [PATCH 79/83] refactor: do some internal refactor. --- source/libs/stream/src/stream.c | 3 ++- source/libs/stream/src/streamDispatch.c | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/source/libs/stream/src/stream.c b/source/libs/stream/src/stream.c index 728f435d76..0a02c39bdf 100644 --- a/source/libs/stream/src/stream.c +++ b/source/libs/stream/src/stream.c @@ -287,7 +287,8 @@ int32_t streamProcessDispatchRsp(SStreamTask* pTask, SStreamDispatchRsp* pRsp, i qDebug("s-task:%s output status is set to:%d", pTask->id.idStr, pTask->outputStatus); - // the input queue of the (down stream) task that receive the output data is full, so the TASK_INPUT_STATUS_BLOCKED is rsp + // the input queue of the (down stream) task that receive the output data is full, + // so the TASK_INPUT_STATUS_BLOCKED is rsp if (pRsp->inputStatus == TASK_INPUT_STATUS__BLOCKED) { pTask->msgInfo.blockingTs = taosGetTimestampMs(); // record the blocking start time diff --git a/source/libs/stream/src/streamDispatch.c b/source/libs/stream/src/streamDispatch.c index c7e156bcf6..d93de7b1e5 100644 --- a/source/libs/stream/src/streamDispatch.c +++ b/source/libs/stream/src/streamDispatch.c @@ -321,7 +321,7 @@ int32_t streamDoDispatchScanHistoryFinishMsg(SStreamTask* pTask, const SStreamRe return 0; } -int32_t doSendDispatchMsg(SStreamTask* pTask, const SStreamDispatchReq* pReq, int32_t vgId, SEpSet* pEpSet) { +static int32_t doSendDispatchMsg(SStreamTask* pTask, const SStreamDispatchReq* pReq, int32_t vgId, SEpSet* pEpSet) { void* buf = NULL; int32_t code = -1; SRpcMsg msg = {0}; From 418849ed3a5b70871f6785ed8e5cf2da966974f6 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 4 Jul 2023 10:57:43 +0800 Subject: [PATCH 80/83] test: disable stream meta storage format compatible test cases. --- tests/system-test/0-others/compatibility.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/system-test/0-others/compatibility.py b/tests/system-test/0-others/compatibility.py index 22e319fdaf..49f6a76b78 100644 --- a/tests/system-test/0-others/compatibility.py +++ b/tests/system-test/0-others/compatibility.py @@ -138,9 +138,9 @@ class TDTestCase: tdLog.printNoPrefix(f"==========step1:prepare and check data in old version-{BASEVERSION}") tdLog.info(f" LD_LIBRARY_PATH=/usr/lib taosBenchmark -t {tableNumbers} -n {recordNumbers1} -y ") os.system(f"LD_LIBRARY_PATH=/usr/lib taosBenchmark -t {tableNumbers} -n {recordNumbers1} -y ") - os.system(f"LD_LIBRARY_PATH=/usr/lib taos -s 'use test;create stream current_stream into current_stream_output_stb as select _wstart as `start`, _wend as wend, max(current) as max_current from meters where voltage <= 220 interval (5s);' ") - os.system('LD_LIBRARY_PATH=/usr/lib taos -s "use test;create stream power_stream into power_stream_output_stb as select ts, concat_ws(\\".\\", location, tbname) as meter_location, current*voltage*cos(phase) as active_power, current*voltage*sin(phase) as reactive_power from meters partition by tbname;" ') - os.system('LD_LIBRARY_PATH=/usr/lib taos -s "use test;show streams;" ') + # os.system(f"LD_LIBRARY_PATH=/usr/lib taos -s 'use test;create stream current_stream into current_stream_output_stb as select _wstart as `start`, _wend as wend, max(current) as max_current from meters where voltage <= 220 interval (5s);' ") + # os.system('LD_LIBRARY_PATH=/usr/lib taos -s "use test;create stream power_stream into power_stream_output_stb as select ts, concat_ws(\\".\\", location, tbname) as meter_location, current*voltage*cos(phase) as active_power, current*voltage*sin(phase) as reactive_power from meters partition by tbname;" ') + # os.system('LD_LIBRARY_PATH=/usr/lib taos -s "use test;show streams;" ') os.system(f"sed -i 's/\/etc\/taos/{cPath}/' 0-others/tmqBasic.json ") # os.system("LD_LIBRARY_PATH=/usr/lib taosBenchmark -f 0-others/tmqBasic.json -y ") os.system('LD_LIBRARY_PATH=/usr/lib taos -s "create topic if not exists tmq_test_topic as select current,voltage,phase from test.meters where voltage <= 106 and current <= 5;" ') From 77530e19565ef9f2bb2b4bd44c86e4139aa7b121 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 4 Jul 2023 10:58:11 +0800 Subject: [PATCH 81/83] refactor: record the downstream input Queue blocking time. --- source/libs/stream/src/stream.c | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/source/libs/stream/src/stream.c b/source/libs/stream/src/stream.c index 0a02c39bdf..691d31e64c 100644 --- a/source/libs/stream/src/stream.c +++ b/source/libs/stream/src/stream.c @@ -289,6 +289,7 @@ int32_t streamProcessDispatchRsp(SStreamTask* pTask, SStreamDispatchRsp* pRsp, i // the input queue of the (down stream) task that receive the output data is full, // so the TASK_INPUT_STATUS_BLOCKED is rsp + // todo blocking the output status if (pRsp->inputStatus == TASK_INPUT_STATUS__BLOCKED) { pTask->msgInfo.blockingTs = taosGetTimestampMs(); // record the blocking start time @@ -301,6 +302,13 @@ int32_t streamProcessDispatchRsp(SStreamTask* pTask, SStreamDispatchRsp* pRsp, i destroyStreamDataBlock(pTask->msgInfo.pData); pTask->msgInfo.pData = NULL; + if (pTask->msgInfo.blockingTs != 0) { + int64_t el = taosGetTimestampMs() - pTask->msgInfo.blockingTs; + qDebug("s-task:%s resume to normal from inputQ blocking, idle time:%"PRId64"ms", pTask->id.idStr, el); + pTask->msgInfo.blockingTs = 0; + } + + // now ready for next data output atomic_store_8(&pTask->outputStatus, TASK_OUTPUT_STATUS__NORMAL); // otherwise, continue dispatch the first block to down stream task in pipeline From 86b1e4958934c72617ba311c9afa24472a51f5f2 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 4 Jul 2023 11:36:05 +0800 Subject: [PATCH 82/83] test: update test cases. --- tests/system-test/0-others/compatibility.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/system-test/0-others/compatibility.py b/tests/system-test/0-others/compatibility.py index 49f6a76b78..cd71de0c06 100644 --- a/tests/system-test/0-others/compatibility.py +++ b/tests/system-test/0-others/compatibility.py @@ -224,7 +224,7 @@ class TDTestCase: args = (caller.filename, caller.lineno) tdLog.exit("%s(%d) failed" % args) tdsql.query("show streams;") - tdsql.checkRows(2) + tdsql.checkRows(0) tdsql.query("select *,tbname from d0.almlog where mcid='m0103';") tdsql.checkRows(6) expectList = [0,3003,20031,20032,20033,30031] From ff2bf356037a720f45d59e09f9d505eca9e978db Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 4 Jul 2023 15:08:39 +0800 Subject: [PATCH 83/83] fix(stream): fix error in set the version range for secondary scan. --- source/dnode/vnode/src/tq/tq.c | 42 ++++++++++++------------- source/dnode/vnode/src/tsdb/tsdbRead.c | 9 ++++-- source/libs/executor/src/executor.c | 4 +-- source/libs/executor/src/scanoperator.c | 40 +++++++++++------------ source/libs/stream/src/streamRecover.c | 9 +++--- 5 files changed, 53 insertions(+), 51 deletions(-) diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 250c94c2f9..9702779470 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1061,7 +1061,7 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { int32_t code = TSDB_CODE_SUCCESS; char* msg = pMsg->pCont; - SStreamMeta* pMeta = pTq->pStreamMeta; + SStreamMeta* pMeta = pTq->pStreamMeta; SStreamScanHistoryReq* pReq = (SStreamScanHistoryReq*)msg; SStreamTask* pTask = streamMetaAcquireTask(pMeta, pReq->taskId); @@ -1107,7 +1107,7 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { if (pTask->info.fillHistory) { SVersionRange* pRange = NULL; - SStreamTask* pStreamTask = NULL; + SStreamTask* pStreamTask = NULL; if (!pReq->igUntreated && !streamTaskRecoverScanStep1Finished(pTask)) { // 1. stop the related stream task, get the current scan wal version of stream task, ver. @@ -1120,9 +1120,10 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { // wait for the stream task get ready for scan history data while (((pStreamTask->status.downstreamReady == 0) && (pStreamTask->status.taskStatus != TASK_STATUS__STOP)) || - pStreamTask->status.taskStatus == TASK_STATUS__SCAN_HISTORY) { - tqDebug("s-task:%s level:%d related stream task:%s not ready for halt, wait for it continue and recheck in 100ms", - pTask->id.idStr, pTask->info.taskLevel, pStreamTask->id.idStr); + pStreamTask->status.taskStatus == TASK_STATUS__SCAN_HISTORY) { + tqDebug( + "s-task:%s level:%d related stream task:%s not ready for halt, wait for it continue and recheck in 100ms", + pTask->id.idStr, pTask->info.taskLevel, pStreamTask->id.idStr); taosMsleep(100); } @@ -1132,36 +1133,34 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { pStreamTask->info.taskLevel, pId); // if it's an source task, extract the last version in wal. - int64_t ver = pTask->dataRange.range.maxVer + 1; - int64_t latestVer = walReaderGetCurrentVer(pStreamTask->exec.pWalReader); - if (latestVer >= ver) { - ver = latestVer; - } - - // 2. do secondary scan of the history data, the time window remain, and the version range is updated to [pTask->dataRange.range.maxVer, ver1] pRange = &pTask->dataRange.range; + int64_t latestVer = walReaderGetCurrentVer(pStreamTask->exec.pWalReader); + ASSERT(latestVer >= pRange->maxVer); - pRange->minVer = pRange->maxVer + 1; - pRange->maxVer = ver; - if (pRange->minVer == pRange->maxVer) { + int64_t nextStartVer = pRange->maxVer + 1; + if (nextStartVer > latestVer - 1) { + // no input data yet. no need to execute the secondardy scan while stream task halt streamTaskRecoverSetAllStepFinished(pTask); - tqDebug("s-task:%s no need to perform secondary scan-history-data(step 2), since no new data ingest", pId); + tqDebug("s-task:%s no need to perform secondary scan-history-data(step 2), since no data ingest during secondary scan", pId); + } else { + // 2. do secondary scan of the history data, the time window remain, and the version range is updated to + // [pTask->dataRange.range.maxVer, ver1] + pRange->minVer = nextStartVer; + pRange->maxVer = latestVer - 1; } } - + if (!streamTaskRecoverScanStep1Finished(pTask)) { tqDebug("s-task:%s level:%d verRange:%" PRId64 " - %" PRId64 " do secondary scan-history-data after halt the related stream task:%s", pId, pTask->info.taskLevel, pRange->minVer, pRange->maxVer, pStreamTask->id.idStr); - ASSERT(pTask->status.schedStatus == TASK_SCHED_STATUS__WAITING); st = taosGetTimestampMs(); streamSetParamForStreamScannerStep2(pTask, pRange, &pTask->dataRange.window); } - if(!streamTaskRecoverScanStep2Finished(pTask)) { - + if (!streamTaskRecoverScanStep2Finished(pTask)) { streamSourceScanHistoryData(pTask); if (atomic_load_8(&pTask->status.taskStatus) == TASK_STATUS__DROPPING || streamTaskShouldPause(&pTask->status)) { tqDebug("s-task:%s is dropped or paused, abort recover in step1", pId); @@ -1215,7 +1214,8 @@ int32_t tqProcessTaskScanHistory(STQ* pTq, SRpcMsg* pMsg) { pWindow->ekey); } else { tqDebug("s-task:%s history data scan completed, now start to scan data from wal, start ver:%" PRId64 - ", window:%" PRId64 " - %" PRId64, pId, pTask->chkInfo.currentVer, pWindow->skey, pWindow->ekey); + ", window:%" PRId64 " - %" PRId64, + pId, pTask->chkInfo.currentVer, pWindow->skey, pWindow->ekey); } code = streamTaskScanHistoryDataComplete(pTask); diff --git a/source/dnode/vnode/src/tsdb/tsdbRead.c b/source/dnode/vnode/src/tsdb/tsdbRead.c index b04e643d29..165448fb7b 100644 --- a/source/dnode/vnode/src/tsdb/tsdbRead.c +++ b/source/dnode/vnode/src/tsdb/tsdbRead.c @@ -248,7 +248,7 @@ static int32_t initDelSkylineIterator(STableBlockScanInfo* pBlockScanInfo, STsdb STbData* piMemTbData); static STsdb* getTsdbByRetentions(SVnode* pVnode, TSKEY winSKey, SRetention* retentions, const char* idstr, int8_t* pLevel); -static SVersionRange getQueryVerRange(SVnode* pVnode, SQueryTableDataCond* pCond, int8_t level); +static SVersionRange getQueryVerRange(SVnode* pVnode, SQueryTableDataCond* pCond, const char* id); static bool hasDataInLastBlock(SLastBlockReader* pLastBlockReader); static int32_t doBuildDataBlock(STsdbReader* pReader); static TSDBKEY getCurrentKeyInBuf(STableBlockScanInfo* pScanInfo, STsdbReader* pReader); @@ -775,7 +775,7 @@ static int32_t tsdbReaderCreate(SVnode* pVnode, SQueryTableDataCond* pCond, void pReader->order = pCond->order; pReader->idStr = (idstr != NULL) ? taosStrdup(idstr) : NULL; - pReader->verRange = getQueryVerRange(pVnode, pCond, level); + pReader->verRange = getQueryVerRange(pVnode, pCond, idstr); pReader->type = pCond->type; pReader->window = updateQueryTimeWindow(pReader->pTsdb, &pCond->twindows); pReader->blockInfoBuf.numPerBucket = 1000; // 1000 tables per bucket @@ -3721,7 +3721,7 @@ static STsdb* getTsdbByRetentions(SVnode* pVnode, TSKEY winSKey, SRetention* ret return VND_TSDB(pVnode); } -SVersionRange getQueryVerRange(SVnode* pVnode, SQueryTableDataCond* pCond, int8_t level) { +SVersionRange getQueryVerRange(SVnode* pVnode, SQueryTableDataCond* pCond, const char* id) { int64_t startVer = (pCond->startVersion == -1) ? 0 : pCond->startVersion; int64_t endVer = 0; @@ -3732,6 +3732,9 @@ SVersionRange getQueryVerRange(SVnode* pVnode, SQueryTableDataCond* pCond, int8_ endVer = (pCond->endVersion > pVnode->state.applied) ? pVnode->state.applied : pCond->endVersion; } + tsdbDebug("queried verRange:%"PRId64"-%"PRId64", revised query verRange:%"PRId64"-%"PRId64", %s", pCond->startVersion, + pCond->endVersion, startVer, endVer, id); + return (SVersionRange){.minVer = startVer, .maxVer = endVer}; } diff --git a/source/libs/executor/src/executor.c b/source/libs/executor/src/executor.c index d67088ebe1..a3d94a0891 100644 --- a/source/libs/executor/src/executor.c +++ b/source/libs/executor/src/executor.c @@ -892,7 +892,7 @@ int32_t qStreamSourceScanParamForHistoryScanStep1(qTaskInfo_t tinfo, SVersionRan pStreamInfo->recoverStep1Finished = false; pStreamInfo->recoverStep2Finished = false; - qDebug("%s step 1. set param for stream scanner for scan history data, Ver:%" PRId64 " - %" PRId64 ", window:%" PRId64 + qDebug("%s step 1. set param for stream scanner for scan history data, verRange:%" PRId64 " - %" PRId64 ", window:%" PRId64 " - %" PRId64, GET_TASKID(pTaskInfo), pStreamInfo->fillHistoryVer.minVer, pStreamInfo->fillHistoryVer.maxVer, pWindow->skey, pWindow->ekey); @@ -911,7 +911,7 @@ int32_t qStreamSourceScanParamForHistoryScanStep2(qTaskInfo_t tinfo, SVersionRan pStreamInfo->recoverStep1Finished = true; pStreamInfo->recoverStep2Finished = false; - qDebug("%s step 2. set param for stream scanner for scan history data, Ver:%" PRId64 " - %" PRId64 ", window:%" PRId64 + qDebug("%s step 2. set param for stream scanner for scan history data, verRange:%" PRId64 " - %" PRId64 ", window:%" PRId64 " - %" PRId64, GET_TASKID(pTaskInfo), pStreamInfo->fillHistoryVer.minVer, pStreamInfo->fillHistoryVer.maxVer, pWindow->skey, pWindow->ekey); diff --git a/source/libs/executor/src/scanoperator.c b/source/libs/executor/src/scanoperator.c index bfe4ed0533..689a3afeb1 100644 --- a/source/libs/executor/src/scanoperator.c +++ b/source/libs/executor/src/scanoperator.c @@ -1783,28 +1783,28 @@ static SSDataBlock* doStreamScan(SOperatorInfo* pOperator) { SStorageAPI* pAPI = &pTaskInfo->storageAPI; SStreamScanInfo* pInfo = pOperator->info; + SStreamTaskInfo* pStreamInfo = &pTaskInfo->streamInfo; qDebug("stream scan started, %s", id); - if (pTaskInfo->streamInfo.recoverStep == STREAM_RECOVER_STEP__PREPARE1 || - pTaskInfo->streamInfo.recoverStep == STREAM_RECOVER_STEP__PREPARE2) { + if (pStreamInfo->recoverStep == STREAM_RECOVER_STEP__PREPARE1 || pStreamInfo->recoverStep == STREAM_RECOVER_STEP__PREPARE2) { STableScanInfo* pTSInfo = pInfo->pTableScanOp->info; - memcpy(&pTSInfo->base.cond, &pTaskInfo->streamInfo.tableCond, sizeof(SQueryTableDataCond)); + memcpy(&pTSInfo->base.cond, &pStreamInfo->tableCond, sizeof(SQueryTableDataCond)); - if (pTaskInfo->streamInfo.recoverStep == STREAM_RECOVER_STEP__PREPARE1) { - pTSInfo->base.cond.startVersion = pTaskInfo->streamInfo.fillHistoryVer.minVer; - pTSInfo->base.cond.endVersion = pTaskInfo->streamInfo.fillHistoryVer.maxVer; + if (pStreamInfo->recoverStep == STREAM_RECOVER_STEP__PREPARE1) { + pTSInfo->base.cond.startVersion = pStreamInfo->fillHistoryVer.minVer; + pTSInfo->base.cond.endVersion = pStreamInfo->fillHistoryVer.maxVer; - pTSInfo->base.cond.twindows = pTaskInfo->streamInfo.fillHistoryWindow; + pTSInfo->base.cond.twindows = pStreamInfo->fillHistoryWindow; qDebug("stream recover step1, verRange:%" PRId64 "-%" PRId64 " window:%"PRId64"-%"PRId64", %s", pTSInfo->base.cond.startVersion, pTSInfo->base.cond.endVersion, pTSInfo->base.cond.twindows.skey, pTSInfo->base.cond.twindows.ekey, id); - pTaskInfo->streamInfo.recoverStep = STREAM_RECOVER_STEP__SCAN1; + pStreamInfo->recoverStep = STREAM_RECOVER_STEP__SCAN1; } else { - pTSInfo->base.cond.startVersion = pTaskInfo->streamInfo.fillHistoryVer.minVer; - pTSInfo->base.cond.endVersion = pTaskInfo->streamInfo.fillHistoryVer.maxVer; + pTSInfo->base.cond.startVersion = pStreamInfo->fillHistoryVer.minVer; + pTSInfo->base.cond.endVersion = pStreamInfo->fillHistoryVer.maxVer; qDebug("stream recover step2, verRange:%" PRId64 " - %" PRId64", %s", pTSInfo->base.cond.startVersion, pTSInfo->base.cond.endVersion, id); - pTaskInfo->streamInfo.recoverStep = STREAM_RECOVER_STEP__SCAN2; + pStreamInfo->recoverStep = STREAM_RECOVER_STEP__SCAN2; } pAPI->tsdReader.tsdReaderClose(pTSInfo->base.dataReader); @@ -1814,11 +1814,11 @@ static SSDataBlock* doStreamScan(SOperatorInfo* pOperator) { pTSInfo->scanTimes = 0; pTSInfo->currentGroupId = -1; - pTaskInfo->streamInfo.recoverScanFinished = false; + pStreamInfo->recoverScanFinished = false; } - if (pTaskInfo->streamInfo.recoverStep == STREAM_RECOVER_STEP__SCAN1 || - pTaskInfo->streamInfo.recoverStep == STREAM_RECOVER_STEP__SCAN2) { + if (pStreamInfo->recoverStep == STREAM_RECOVER_STEP__SCAN1 || + pStreamInfo->recoverStep == STREAM_RECOVER_STEP__SCAN2) { if (pInfo->blockRecoverContiCnt > 100) { pInfo->blockRecoverTotCnt += pInfo->blockRecoverContiCnt; pInfo->blockRecoverContiCnt = 0; @@ -1869,11 +1869,11 @@ static SSDataBlock* doStreamScan(SOperatorInfo* pOperator) { pInfo->blockRecoverContiCnt++; calBlockTbName(pInfo, pInfo->pRecoverRes); if (!pInfo->igCheckUpdate && pInfo->pUpdateInfo) { - if (pTaskInfo->streamInfo.recoverStep == STREAM_RECOVER_STEP__SCAN1) { + if (pStreamInfo->recoverStep == STREAM_RECOVER_STEP__SCAN1) { TSKEY maxTs = pAPI->stateStore.updateInfoFillBlockData(pInfo->pUpdateInfo, pInfo->pRecoverRes, pInfo->primaryTsIndex); pInfo->twAggSup.maxTs = TMAX(pInfo->twAggSup.maxTs, maxTs); } else { - pInfo->pUpdateInfo->maxDataVersion = TMAX(pInfo->pUpdateInfo->maxDataVersion, pTaskInfo->streamInfo.fillHistoryVer.maxVer); + pInfo->pUpdateInfo->maxDataVersion = TMAX(pInfo->pUpdateInfo->maxDataVersion, pStreamInfo->fillHistoryVer.maxVer); doCheckUpdate(pInfo, pInfo->pRecoverRes->info.window.ekey, pInfo->pRecoverRes); } } @@ -1887,7 +1887,7 @@ static SSDataBlock* doStreamScan(SOperatorInfo* pOperator) { printDataBlock(pInfo->pRecoverRes, "scan recover"); return pInfo->pRecoverRes; } - pTaskInfo->streamInfo.recoverStep = STREAM_RECOVER_STEP__NONE; + pStreamInfo->recoverStep = STREAM_RECOVER_STEP__NONE; STableScanInfo* pTSInfo = pInfo->pTableScanOp->info; pAPI->tsdReader.tsdReaderClose(pTSInfo->base.dataReader); @@ -1896,7 +1896,7 @@ static SSDataBlock* doStreamScan(SOperatorInfo* pOperator) { pTSInfo->base.cond.startVersion = -1; pTSInfo->base.cond.endVersion = -1; - pTaskInfo->streamInfo.recoverScanFinished = true; + pStreamInfo->recoverScanFinished = true; return NULL; } @@ -1915,7 +1915,7 @@ FETCH_NEXT_BLOCK: SPackedData* pPacked = taosArrayGet(pInfo->pBlockLists, current); SSDataBlock* pBlock = pPacked->pDataBlock; if (pBlock->info.parTbName[0]) { - pAPI->stateStore.streamStatePutParName(pTaskInfo->streamInfo.pState, pBlock->info.id.groupId, pBlock->info.parTbName); + pAPI->stateStore.streamStatePutParName(pStreamInfo->pState, pBlock->info.id.groupId, pBlock->info.parTbName); } // TODO move into scan @@ -2097,7 +2097,7 @@ FETCH_NEXT_BLOCK: doFilter(pBlock, pOperator->exprSupp.pFilterInfo, NULL); { // do additional time window filter - STimeWindow* pWindow = &pTaskInfo->streamInfo.fillHistoryWindow; + STimeWindow* pWindow = &pStreamInfo->fillHistoryWindow; if (pWindow->skey != INT64_MIN) { qDebug("%s filter for additional history window, skey:%"PRId64, id, pWindow->skey); diff --git a/source/libs/stream/src/streamRecover.c b/source/libs/stream/src/streamRecover.c index 8e4d50bcf2..9ded58597f 100644 --- a/source/libs/stream/src/streamRecover.c +++ b/source/libs/stream/src/streamRecover.c @@ -50,9 +50,8 @@ const char* streamGetTaskStatusStr(int32_t status) { static int32_t doLaunchScanHistoryTask(SStreamTask* pTask) { SVersionRange* pRange = &pTask->dataRange.range; - qDebug("s-task:%s vgId:%d task status:%s and start to scan-history-data task, ver:%" PRId64 " - %" PRId64, - pTask->id.idStr, pTask->info.nodeId, streamGetTaskStatusStr(pTask->status.taskStatus), - pRange->minVer, pRange->maxVer); + qDebug("s-task:%s vgId:%d status:%s, start scan-history-data task, verRange:%" PRId64 " - %" PRId64, pTask->id.idStr, + pTask->info.nodeId, streamGetTaskStatusStr(pTask->status.taskStatus), pRange->minVer, pRange->maxVer); streamSetParamForScanHistoryData(pTask); streamSetParamForStreamScannerStep1(pTask, pRange, &pTask->dataRange.window); @@ -671,8 +670,8 @@ void streamPrepareNdoCheckDownstream(SStreamTask* pTask) { pRange->range.maxVer); } else { SHistDataRange* pRange = &pTask->dataRange; - qDebug("s-task:%s no associated scan-history task, stream time window:%" PRId64 " - %" PRId64 ", ver range:%" PRId64 - " - %" PRId64, + qDebug("s-task:%s no associated scan-history task, stream time window:%" PRId64 " - %" PRId64 + ", ver range:%" PRId64 " - %" PRId64, pTask->id.idStr, pRange->window.skey, pRange->window.ekey, pRange->range.minVer, pRange->range.maxVer); }