diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 8e7dd0bb0d..1d4bbf073e 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -78,11 +78,11 @@ enum { TASK_TRIGGER_STATUS__ACTIVE, }; -enum { +typedef enum { TASK_LEVEL__SOURCE = 1, TASK_LEVEL__AGG, TASK_LEVEL__SINK, -}; +} ETASK_LEVEL; enum { TASK_OUTPUT__FIXED_DISPATCH = 1, @@ -284,13 +284,13 @@ struct SStreamTask { int16_t dispatchMsgType; SStreamStatus status; int32_t selfChildId; - int32_t nodeId; + int32_t nodeId; // vgroup id SEpSet epSet; SCheckpointInfo chkInfo; STaskExec exec; - - // fill history - int8_t fillHistory; + int8_t fillHistory; // fill history + int64_t ekey; // end ts key + int64_t endVer; // end version // children info SArray* childEpInfo; // SArray @@ -351,7 +351,7 @@ typedef struct SStreamMeta { int32_t tEncodeStreamEpInfo(SEncoder* pEncoder, const SStreamChildEpInfo* pInfo); int32_t tDecodeStreamEpInfo(SDecoder* pDecoder, SStreamChildEpInfo* pInfo); -SStreamTask* tNewStreamTask(int64_t streamId); +SStreamTask* tNewStreamTask(int64_t streamId, int8_t taskLevel, int8_t fillHistory, int64_t triggerParam, SArray* pTaskList); int32_t tEncodeStreamTask(SEncoder* pEncoder, const SStreamTask* pTask); int32_t tDecodeStreamTask(SDecoder* pDecoder, SStreamTask* pTask); void tFreeStreamTask(SStreamTask* pTask); diff --git a/source/dnode/mnode/impl/inc/mndDef.h b/source/dnode/mnode/impl/inc/mndDef.h index 7d707f4cba..82b714e6eb 100644 --- a/source/dnode/mnode/impl/inc/mndDef.h +++ b/source/dnode/mnode/impl/inc/mndDef.h @@ -663,9 +663,10 @@ typedef struct { char targetDb[TSDB_DB_FNAME_LEN]; char targetSTbName[TSDB_TABLE_FNAME_LEN]; int64_t targetStbUid; - int32_t fixedSinkVgId; // 0 for shuffle + // fixedSinkVg is not applicable for encode and decode SVgObj fixedSinkVg; + int32_t fixedSinkVgId; // 0 for shuffle // transformation char* sql; diff --git a/source/dnode/mnode/impl/src/mndScheduler.c b/source/dnode/mnode/impl/src/mndScheduler.c index 734f624be0..df8c11a6f6 100644 --- a/source/dnode/mnode/impl/src/mndScheduler.c +++ b/source/dnode/mnode/impl/src/mndScheduler.c @@ -14,18 +14,8 @@ */ #include "mndScheduler.h" -#include "mndConsumer.h" #include "mndDb.h" -#include "mndDnode.h" -#include "mndMnode.h" -#include "mndShow.h" #include "mndSnode.h" -#include "mndStb.h" -#include "mndStream.h" -#include "mndSubscribe.h" -#include "mndTopic.h" -#include "mndTrans.h" -#include "mndUser.h" #include "mndVgroup.h" #include "parser.h" #include "tcompare.h" @@ -34,12 +24,8 @@ extern bool tsDeployOnSnode; -static int32_t mndAddTaskToTaskSet(SArray* pArray, SStreamTask* pTask) { - int32_t childId = taosArrayGetSize(pArray); - pTask->selfChildId = childId; - taosArrayPush(pArray, &pTask); - return 0; -} +static int32_t mndAddSinkTaskToStream(SStreamObj* pStream, 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, int64_t watermark, int64_t deleteMark) { @@ -97,7 +83,7 @@ END: return terrno; } -int32_t mndAddSinkToTask(SMnode* pMnode, SStreamObj* pStream, SStreamTask* pTask) { +int32_t mndSetSinkTaskInfo(SStreamObj* pStream, SStreamTask* pTask) { if (pStream->smaId != 0) { pTask->outputType = TASK_OUTPUT__SMA; pTask->smaSink.smaId = pStream->smaId; @@ -106,16 +92,23 @@ int32_t mndAddSinkToTask(SMnode* pMnode, SStreamObj* pStream, SStreamTask* pTask pTask->tbSink.stbUid = pStream->targetStbUid; memcpy(pTask->tbSink.stbFullName, pStream->targetSTbName, TSDB_TABLE_FNAME_LEN); pTask->tbSink.pSchemaWrapper = tCloneSSchemaWrapper(&pStream->outputSchema); + if (pTask->tbSink.pSchemaWrapper == NULL) { + return TSDB_CODE_OUT_OF_MEMORY; + } } + return 0; } -int32_t mndAddDispatcherToInnerTask(SMnode* pMnode, SStreamObj* pStream, SStreamTask* pTask) { +#define SINK_NODE_LEVEL (0) + +int32_t mndAddDispatcherForInnerTask(SMnode* pMnode, SStreamObj* pStream, 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->dispatchMsgType = TDMT_STREAM_TASK_DISPATCH; @@ -127,47 +120,46 @@ int32_t mndAddDispatcherToInnerTask(SMnode* pMnode, SStreamObj* pStream, SStream sdbRelease(pMnode->pSdb, pDb); } + SArray* pSinkNodeList = taosArrayGetP(pStream->tasks, SINK_NODE_LEVEL); + int32_t numOfSinkNodes = taosArrayGetSize(pSinkNodeList); + if (isShuffle) { memcpy(pTask->shuffleDispatcher.stbFullName, pStream->targetSTbName, TSDB_TABLE_FNAME_LEN); SArray* pVgs = pTask->shuffleDispatcher.dbInfo.pVgroupInfos; - int32_t sz = taosArrayGetSize(pVgs); - SArray* sinkLv = taosArrayGetP(pStream->tasks, 0); - int32_t sinkLvSize = taosArrayGetSize(sinkLv); - for (int32_t i = 0; i < sz; i++) { + + int32_t numOfVgroups = taosArrayGetSize(pVgs); + for (int32_t i = 0; i < numOfVgroups; i++) { SVgroupInfo* pVgInfo = taosArrayGet(pVgs, i); - for (int32_t j = 0; j < sinkLvSize; j++) { - SStreamTask* pLastLevelTask = taosArrayGetP(sinkLv, j); - if (pLastLevelTask->nodeId == pVgInfo->vgId) { - pVgInfo->taskId = pLastLevelTask->id.taskId; + + for (int32_t j = 0; j < numOfSinkNodes; j++) { + SStreamTask* pSinkTask = taosArrayGetP(pSinkNodeList, j); + if (pSinkTask->nodeId == pVgInfo->vgId) { + pVgInfo->taskId = pSinkTask->id.taskId; break; } } } } else { - pTask->outputType = TASK_OUTPUT__FIXED_DISPATCH; - pTask->dispatchMsgType = TDMT_STREAM_TASK_DISPATCH; - SArray* pArray = taosArrayGetP(pStream->tasks, 0); - // one sink only - SStreamTask* lastLevelTask = taosArrayGetP(pArray, 0); - pTask->fixedEpDispatcher.taskId = lastLevelTask->id.taskId; - pTask->fixedEpDispatcher.nodeId = lastLevelTask->nodeId; - pTask->fixedEpDispatcher.epSet = lastLevelTask->epSet; + SStreamTask* pOneSinkTask = taosArrayGetP(pSinkNodeList, 0); + setFixedDownstreamEpInfo(pTask, pOneSinkTask); } + return 0; } -int32_t mndAssignTaskToVg(SMnode* pMnode, SStreamTask* pTask, SSubplan* plan, const SVgObj* pVgroup) { +int32_t mndAssignStreamTaskToVgroup(SMnode* pMnode, SStreamTask* pTask, SSubplan* plan, const SVgObj* pVgroup) { int32_t msgLen; + pTask->nodeId = pVgroup->vgId; pTask->epSet = mndGetVgroupEpset(pMnode, pVgroup); - plan->execNode.nodeId = pVgroup->vgId; + plan->execNode.nodeId = pTask->nodeId; plan->execNode.epSet = pTask->epSet; - if (qSubPlanToString(plan, &pTask->exec.qmsg, &msgLen) < 0) { terrno = TSDB_CODE_QRY_INVALID_INPUT; return -1; } + return 0; } @@ -210,100 +202,121 @@ SVgObj* mndSchedFetchOneVg(SMnode* pMnode, int64_t dbUid) { return pVgroup; } +// create sink node for each vgroup. int32_t mndAddShuffleSinkTasksToStream(SMnode* pMnode, SStreamObj* pStream) { SSdb* pSdb = pMnode->pSdb; void* pIter = NULL; - SArray* tasks = taosArrayGetP(pStream->tasks, 0); while (1) { SVgObj* pVgroup = NULL; pIter = sdbFetch(pSdb, SDB_VGROUP, pIter, (void**)&pVgroup); - if (pIter == NULL) break; + if (pIter == NULL) { + break; + } + if (!mndVgroupInDb(pVgroup, pStream->targetDbUid)) { sdbRelease(pSdb, pVgroup); continue; } - SStreamTask* pTask = tNewStreamTask(pStream->uid); - if (pTask == NULL) { - sdbRelease(pSdb, pVgroup); - terrno = TSDB_CODE_OUT_OF_MEMORY; - return -1; - } - pTask->fillHistory = pStream->fillHistory; - mndAddTaskToTaskSet(tasks, pTask); - - pTask->nodeId = pVgroup->vgId; - pTask->epSet = mndGetVgroupEpset(pMnode, pVgroup); - - // type - pTask->taskLevel = TASK_LEVEL__SINK; - - // sink - if (pStream->smaId != 0) { - pTask->outputType = TASK_OUTPUT__SMA; - pTask->smaSink.smaId = pStream->smaId; - } else { - pTask->outputType = TASK_OUTPUT__TABLE; - pTask->tbSink.stbUid = pStream->targetStbUid; - memcpy(pTask->tbSink.stbFullName, pStream->targetSTbName, TSDB_TABLE_FNAME_LEN); - pTask->tbSink.pSchemaWrapper = tCloneSSchemaWrapper(&pStream->outputSchema); - if (pTask->tbSink.pSchemaWrapper == NULL) { - terrno = TSDB_CODE_OUT_OF_MEMORY; - return -1; - } - } + mndAddSinkTaskToStream(pStream, pMnode, pVgroup->vgId, pVgroup); sdbRelease(pSdb, pVgroup); } + return 0; } -int32_t mndAddFixedSinkTaskToStream(SMnode* pMnode, SStreamObj* pStream) { - SArray* tasks = taosArrayGetP(pStream->tasks, 0); - SStreamTask* pTask = tNewStreamTask(pStream->uid); +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); if (pTask == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; return -1; } - pTask->fillHistory = pStream->fillHistory; - mndAddTaskToTaskSet(tasks, pTask); - pTask->nodeId = pStream->fixedSinkVgId; -#if 0 - SVgObj* pVgroup = mndAcquireVgroup(pMnode, pStream->fixedSinkVgId); - if (pVgroup == NULL) { - return -1; - } + pTask->nodeId = vgId; pTask->epSet = mndGetVgroupEpset(pMnode, pVgroup); -#endif - pTask->epSet = mndGetVgroupEpset(pMnode, &pStream->fixedSinkVg); - - pTask->taskLevel = TASK_LEVEL__SINK; - - // sink - if (pStream->smaId != 0) { - pTask->outputType = TASK_OUTPUT__SMA; - pTask->smaSink.smaId = pStream->smaId; - } else { - pTask->outputType = TASK_OUTPUT__TABLE; - pTask->tbSink.stbUid = pStream->targetStbUid; - memcpy(pTask->tbSink.stbFullName, pStream->targetSTbName, TSDB_TABLE_FNAME_LEN); - pTask->tbSink.pSchemaWrapper = tCloneSSchemaWrapper(&pStream->outputSchema); - } - + mndSetSinkTaskInfo(pStream, pTask); return 0; } +static int32_t mndScheduleFillHistoryStreamTask(SMnode* pMnode, SStreamObj* pStream) { + return 0; +} + +static int32_t addSourceStreamTask(SMnode* pMnode, SVgObj* pVgroup, SArray* pTaskList, SStreamObj* pStream, + SSubplan* plan, uint64_t uid, int8_t taskLevel, int8_t fillHistory, + bool hasExtraSink) { + SStreamTask* pTask = tNewStreamTask(uid, taskLevel, fillHistory, pStream->triggerParam, pTaskList); + if (pTask == NULL) { + return terrno; + } + + // sink or dispatch + if (hasExtraSink) { + mndAddDispatcherForInnerTask(pMnode, pStream, pTask); + } else { + mndSetSinkTaskInfo(pStream, pTask); + } + + if (mndAssignStreamTaskToVgroup(pMnode, pTask, plan, pVgroup) < 0) { + return terrno; + } + + return TSDB_CODE_SUCCESS; +} + +static SStreamChildEpInfo* createStreamTaskEpInfo(SStreamTask* pTask) { + SStreamChildEpInfo* pEpInfo = taosMemoryMalloc(sizeof(SStreamChildEpInfo)); + if (pEpInfo == NULL) { + terrno = TSDB_CODE_OUT_OF_MEMORY; + return NULL; + } + + pEpInfo->childId = pTask->selfChildId; + pEpInfo->epSet = pTask->epSet; + pEpInfo->nodeId = pTask->nodeId; + pEpInfo->taskId = pTask->id.taskId; + + return pEpInfo; +} + +void setFixedDownstreamEpInfo(SStreamTask* pDstTask, const SStreamTask* pTask) { + STaskDispatcherFixedEp* pDispatcher = &pDstTask->fixedEpDispatcher; + pDispatcher->taskId = pTask->id.taskId; + pDispatcher->nodeId = pTask->nodeId; + pDispatcher->epSet = pTask->epSet; + + pDstTask->outputType = TASK_OUTPUT__FIXED_DISPATCH; + pDstTask->dispatchMsgType = TDMT_STREAM_TASK_DISPATCH; +} + +int32_t appendToUpstream(SStreamTask* pTask, SStreamTask* pUpstream) { + SStreamChildEpInfo* pEpInfo = createStreamTaskEpInfo(pTask); + if (pEpInfo == NULL) { + return TSDB_CODE_OUT_OF_MEMORY; + } + + if(pUpstream->childEpInfo == NULL) { + pUpstream->childEpInfo = taosArrayInit(4, POINTER_BYTES); + } + + taosArrayPush(pUpstream->childEpInfo, &pEpInfo); + return TSDB_CODE_SUCCESS; +} + int32_t mndScheduleStream(SMnode* pMnode, SStreamObj* pStream) { - SSdb* pSdb = pMnode->pSdb; + 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, sizeof(void*)); + int32_t planTotLevel = LIST_LENGTH(pPlan->pSubplans); + pStream->tasks = taosArrayInit(planTotLevel, POINTER_BYTES); bool hasExtraSink = false; bool externalTargetDB = strcmp(pStream->sourceDb, pStream->targetDb) != 0; @@ -313,13 +326,13 @@ int32_t mndScheduleStream(SMnode* pMnode, SStreamObj* pStream) { return -1; } - bool multiTarget = pDbObj->cfg.numOfVgroups > 1; + bool multiTarget = (pDbObj->cfg.numOfVgroups > 1); sdbRelease(pSdb, pDbObj); if (planTotLevel == 2 || externalTargetDB || multiTarget || pStream->fixedSinkVgId) { - /*if (true) {*/ - SArray* taskOneLevel = taosArrayInit(0, sizeof(void*)); + SArray* taskOneLevel = taosArrayInit(0, POINTER_BYTES); taosArrayPush(pStream->tasks, &taskOneLevel); + // add extra sink hasExtraSink = true; if (pStream->fixedSinkVgId == 0) { @@ -328,19 +341,20 @@ int32_t mndScheduleStream(SMnode* pMnode, SStreamObj* pStream) { return -1; } } else { - if (mndAddFixedSinkTaskToStream(pMnode, pStream) < 0) { + if (mndAddSinkTaskToStream(pStream, pMnode, pStream->fixedSinkVgId, &pStream->fixedSinkVg) < 0) { // TODO free return -1; } } } + pStream->totalLevel = planTotLevel + hasExtraSink; if (planTotLevel > 1) { SStreamTask* pInnerTask; // inner level { - SArray* taskInnerLevel = taosArrayInit(0, sizeof(void*)); + SArray* taskInnerLevel = taosArrayInit(0, POINTER_BYTES); taosArrayPush(pStream->tasks, &taskInnerLevel); SNodeListNode* inner = (SNodeListNode*)nodesListGetNode(pPlan->pSubplans, 0); @@ -350,25 +364,15 @@ int32_t mndScheduleStream(SMnode* pMnode, SStreamObj* pStream) { return -1; } - pInnerTask = tNewStreamTask(pStream->uid); + pInnerTask = tNewStreamTask(pStream->uid, TASK_LEVEL__AGG, pStream->fillHistory, pStream->triggerParam, taskInnerLevel); if (pInnerTask == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; qDestroyQueryPlan(pPlan); return -1; } - pInnerTask->fillHistory = pStream->fillHistory; - mndAddTaskToTaskSet(taskInnerLevel, pInnerTask); - - pInnerTask->childEpInfo = taosArrayInit(0, sizeof(void*)); - - pInnerTask->taskLevel = TASK_LEVEL__AGG; - - // trigger - pInnerTask->triggerParam = pStream->triggerParam; - // dispatch - if (mndAddDispatcherToInnerTask(pMnode, pStream, pInnerTask) < 0) { + if (mndAddDispatcherForInnerTask(pMnode, pStream, pInnerTask) < 0) { qDestroyQueryPlan(pPlan); return -1; } @@ -377,7 +381,7 @@ int32_t mndScheduleStream(SMnode* pMnode, SStreamObj* pStream) { SSnodeObj* pSnode = mndSchedFetchOneSnode(pMnode); if (pSnode == NULL) { SVgObj* pVgroup = mndSchedFetchOneVg(pMnode, pStream->sourceDbUid); - if (mndAssignTaskToVg(pMnode, pInnerTask, plan, pVgroup) < 0) { + if (mndAssignStreamTaskToVgroup(pMnode, pInnerTask, plan, pVgroup) < 0) { sdbRelease(pSdb, pVgroup); qDestroyQueryPlan(pPlan); return -1; @@ -392,17 +396,18 @@ int32_t mndScheduleStream(SMnode* pMnode, SStreamObj* pStream) { } } else { SVgObj* pVgroup = mndSchedFetchOneVg(pMnode, pStream->sourceDbUid); - if (mndAssignTaskToVg(pMnode, pInnerTask, plan, pVgroup) < 0) { + if (mndAssignStreamTaskToVgroup(pMnode, pInnerTask, plan, pVgroup) < 0) { sdbRelease(pSdb, pVgroup); qDestroyQueryPlan(pPlan); return -1; } + sdbRelease(pSdb, pVgroup); } } // source level - SArray* taskSourceLevel = taosArrayInit(0, sizeof(void*)); + SArray* taskSourceLevel = taosArrayInit(0, POINTER_BYTES); taosArrayPush(pStream->tasks, &taskSourceLevel); SNodeListNode* inner = (SNodeListNode*)nodesListGetNode(pPlan->pSubplans, 1); @@ -416,66 +421,52 @@ int32_t mndScheduleStream(SMnode* pMnode, SStreamObj* pStream) { while (1) { SVgObj* pVgroup; pIter = sdbFetch(pSdb, SDB_VGROUP, pIter, (void**)&pVgroup); - if (pIter == NULL) break; + if (pIter == NULL) { + break; + } + if (!mndVgroupInDb(pVgroup, pStream->sourceDbUid)) { sdbRelease(pSdb, pVgroup); continue; } - SStreamTask* pTask = tNewStreamTask(pStream->uid); + SStreamTask* pTask = tNewStreamTask(pStream->uid, TASK_LEVEL__SOURCE, pStream->fillHistory, 0, taskSourceLevel); if (pTask == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; sdbRelease(pSdb, pVgroup); qDestroyQueryPlan(pPlan); return -1; } - pTask->fillHistory = pStream->fillHistory; - mndAddTaskToTaskSet(taskSourceLevel, pTask); - pTask->triggerParam = 0; + // all the source tasks dispatch result to a single agg node. + setFixedDownstreamEpInfo(pTask, pInnerTask); - // source - pTask->taskLevel = TASK_LEVEL__SOURCE; - - // add fixed vg dispatch - pTask->dispatchMsgType = TDMT_STREAM_TASK_DISPATCH; - pTask->outputType = TASK_OUTPUT__FIXED_DISPATCH; - - pTask->fixedEpDispatcher.taskId = pInnerTask->id.taskId; - pTask->fixedEpDispatcher.nodeId = pInnerTask->nodeId; - pTask->fixedEpDispatcher.epSet = pInnerTask->epSet; - - if (mndAssignTaskToVg(pMnode, pTask, plan, pVgroup) < 0) { + if (mndAssignStreamTaskToVgroup(pMnode, pTask, plan, pVgroup) < 0) { sdbRelease(pSdb, pVgroup); qDestroyQueryPlan(pPlan); return -1; } - SStreamChildEpInfo* pEpInfo = taosMemoryMalloc(sizeof(SStreamChildEpInfo)); - if (pEpInfo == NULL) { - terrno = TSDB_CODE_OUT_OF_MEMORY; - sdbRelease(pSdb, pVgroup); - qDestroyQueryPlan(pPlan); - return -1; - } - pEpInfo->childId = pTask->selfChildId; - pEpInfo->epSet = pTask->epSet; - pEpInfo->nodeId = pTask->nodeId; - pEpInfo->taskId = pTask->id.taskId; - taosArrayPush(pInnerTask->childEpInfo, &pEpInfo); + int32_t code = appendToUpstream(pTask, pInnerTask); sdbRelease(pSdb, pVgroup); - } - } - if (planTotLevel == 1) { - SArray* taskOneLevel = taosArrayInit(0, sizeof(void*)); - taosArrayPush(pStream->tasks, &taskOneLevel); + 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); 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; @@ -486,42 +477,26 @@ int32_t mndScheduleStream(SMnode* pMnode, SStreamObj* pStream) { while (1) { SVgObj* pVgroup; pIter = sdbFetch(pSdb, SDB_VGROUP, pIter, (void**)&pVgroup); - if (pIter == NULL) break; + if (pIter == NULL) { + break; + } + if (!mndVgroupInDb(pVgroup, pStream->sourceDbUid)) { sdbRelease(pSdb, pVgroup); continue; } - SStreamTask* pTask = tNewStreamTask(pStream->uid); - if (pTask == NULL) { - sdbRelease(pSdb, pVgroup); - qDestroyQueryPlan(pPlan); - return -1; - } - pTask->fillHistory = pStream->fillHistory; - mndAddTaskToTaskSet(taskOneLevel, pTask); - - // source - pTask->taskLevel = TASK_LEVEL__SOURCE; - - // trigger - pTask->triggerParam = pStream->triggerParam; - - // sink or dispatch - if (hasExtraSink) { - mndAddDispatcherToInnerTask(pMnode, pStream, pTask); - } else { - mndAddSinkToTask(pMnode, pStream, pTask); - } - - if (mndAssignTaskToVg(pMnode, pTask, plan, pVgroup) < 0) { - sdbRelease(pSdb, pVgroup); - qDestroyQueryPlan(pPlan); - return -1; - } + // new stream task + int32_t code = addSourceStreamTask(pMnode, pVgroup, pTaskList, pStream, plan, pStream->uid, TASK_LEVEL__SOURCE, pStream->fillHistory, hasExtraSink); sdbRelease(pSdb, pVgroup); + + if (code != TSDB_CODE_SUCCESS) { + qDestroyQueryPlan(pPlan); + return -1; + } } } + qDestroyQueryPlan(pPlan); return 0; } diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index 0713150b48..39a1fa223f 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -700,6 +700,7 @@ static int32_t mndProcessCreateStreamReq(SRpcMsg *pReq) { if (pStream->sourceDbUid == streamObj.sourceDbUid) { ++numOfStream; } + sdbRelease(pMnode->pSdb, pStream); if (numOfStream > MND_STREAM_MAX_NUM) { mError("too many streams, no more than %d for each database", MND_STREAM_MAX_NUM); @@ -723,6 +724,7 @@ static int32_t mndProcessCreateStreamReq(SRpcMsg *pReq) { pDb = NULL; goto _OVER; } + mndReleaseDb(pMnode, pDb); STrans *pTrans = mndTransCreate(pMnode, TRN_POLICY_ROLLBACK, TRN_CONFLICT_DB_INSIDE, pReq, "create-stream"); diff --git a/source/libs/stream/src/streamTask.c b/source/libs/stream/src/streamTask.c index 8a03896978..a0caffd41f 100644 --- a/source/libs/stream/src/streamTask.c +++ b/source/libs/stream/src/streamTask.c @@ -17,14 +17,25 @@ #include "tstream.h" #include "wal.h" -SStreamTask* tNewStreamTask(int64_t streamId) { +static int32_t mndAddToTaskset(SArray* pArray, SStreamTask* pTask) { + int32_t childId = taosArrayGetSize(pArray); + pTask->selfChildId = childId; + taosArrayPush(pArray, &pTask); + return 0; +} + +SStreamTask* tNewStreamTask(int64_t streamId, int8_t taskLevel, int8_t fillHistory, int64_t triggerParam, SArray* pTaskList) { SStreamTask* pTask = (SStreamTask*)taosMemoryCalloc(1, sizeof(SStreamTask)); if (pTask == NULL) { + terrno = TSDB_CODE_OUT_OF_MEMORY; return NULL; } pTask->id.taskId = tGenIdPI32(); pTask->id.streamId = streamId; + pTask->taskLevel = taskLevel; + pTask->fillHistory = fillHistory; + pTask->triggerParam = triggerParam; char buf[128] = {0}; sprintf(buf, "0x%" PRIx64 "-%d", pTask->id.streamId, pTask->id.taskId); @@ -34,6 +45,7 @@ SStreamTask* tNewStreamTask(int64_t streamId) { pTask->inputStatus = TASK_INPUT_STATUS__NORMAL; pTask->outputStatus = TASK_OUTPUT_STATUS__NORMAL; + mndAddToTaskset(pTaskList, pTask); return pTask; }