From 380e7406d43f36c0b31e66e0a7926ca380cd673f Mon Sep 17 00:00:00 2001 From: liuyao <54liuyao@163.com> Date: Thu, 17 Aug 2023 09:04:05 +0800 Subject: [PATCH 01/51] multi agg --- include/libs/nodes/nodes.h | 1 + source/libs/executor/src/executil.c | 2 + .../executor/src/streamtimewindowoperator.c | 202 ++++++++++++++++++ 3 files changed, 205 insertions(+) diff --git a/include/libs/nodes/nodes.h b/include/libs/nodes/nodes.h index 8eeeff4148..205d9d85f3 100644 --- a/include/libs/nodes/nodes.h +++ b/include/libs/nodes/nodes.h @@ -257,6 +257,7 @@ typedef enum ENodeType { QUERY_NODE_PHYSICAL_PLAN_STREAM_INTERVAL, QUERY_NODE_PHYSICAL_PLAN_STREAM_FINAL_INTERVAL, QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_INTERVAL, + QUERY_NODE_PHYSICAL_PLAN_STREAM_MID_INTERVAL, QUERY_NODE_PHYSICAL_PLAN_FILL, QUERY_NODE_PHYSICAL_PLAN_STREAM_FILL, QUERY_NODE_PHYSICAL_PLAN_MERGE_SESSION, diff --git a/source/libs/executor/src/executil.c b/source/libs/executor/src/executil.c index b87af9ed04..51f624b7e8 100644 --- a/source/libs/executor/src/executil.c +++ b/source/libs/executor/src/executil.c @@ -2189,6 +2189,8 @@ char* getStreamOpName(uint16_t opType) { return "interval final"; case QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_INTERVAL: return "interval semi"; + case QUERY_NODE_PHYSICAL_PLAN_STREAM_MID_INTERVAL: + return "interval mid"; case QUERY_NODE_PHYSICAL_PLAN_STREAM_FILL: return "stream fill"; case QUERY_NODE_PHYSICAL_PLAN_STREAM_SESSION: diff --git a/source/libs/executor/src/streamtimewindowoperator.c b/source/libs/executor/src/streamtimewindowoperator.c index a718373f60..06566633c4 100644 --- a/source/libs/executor/src/streamtimewindowoperator.c +++ b/source/libs/executor/src/streamtimewindowoperator.c @@ -3193,3 +3193,205 @@ _error: pTaskInfo->code = code; return NULL; } + +static void doStreamMidIntervalAggImpl(SOperatorInfo* pOperator, SSDataBlock* pSDataBlock, SSHashObj* pUpdatedMap) { + SStreamIntervalOperatorInfo* pInfo = (SStreamIntervalOperatorInfo*)pOperator->info; + pInfo->dataVersion = TMAX(pInfo->dataVersion, pSDataBlock->info.version); + + SResultRowInfo* pResultRowInfo = &(pInfo->binfo.resultRowInfo); + SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; + SExprSupp* pSup = &pOperator->exprSupp; + int32_t numOfOutput = pSup->numOfExprs; + int32_t step = 1; + SRowBuffPos* pResPos = NULL; + SResultRow* pResult = NULL; + int32_t forwardRows = 1; + uint64_t groupId = pSDataBlock->info.id.groupId; + + SColumnInfoData* pColDataInfo = taosArrayGet(pSDataBlock->pDataBlock, pInfo->primaryTsIndex); + TSKEY* tsCol = (int64_t*)pColDataInfo->pData; + + int32_t startPos = 0; + TSKEY ts = getStartTsKey(&pSDataBlock->info.window, tsCol); + STimeWindow nextWin = getFinalTimeWindow(ts, &pInfo->interval); + + while (1) { + bool isClosed = isCloseWindow(&nextWin, &pInfo->twAggSup); + if ((pInfo->ignoreExpiredData && isClosed) || + !inSlidingWindow(&pInfo->interval, &nextWin, &pSDataBlock->info)) { + startPos = getNexWindowPos(&pInfo->interval, &pSDataBlock->info, tsCol, startPos, nextWin.ekey, &nextWin); + if (startPos < 0) { + break; + } + continue; + } + + int32_t code = setIntervalOutputBuf(pInfo->pState, &nextWin, &pResPos, groupId, pSup->pCtx, numOfOutput, + pSup->rowEntryInfoOffset, &pInfo->aggSup, &pInfo->statestore); + pResult = (SResultRow*)pResPos->pRowBuff; + if (code != TSDB_CODE_SUCCESS || pResult == NULL) { + T_LONG_JMP(pTaskInfo->env, TSDB_CODE_OUT_OF_MEMORY); + } + + SWinKey key = { + .ts = pResult->win.skey, + .groupId = groupId, + }; + if (pInfo->twAggSup.calTrigger == STREAM_TRIGGER_AT_ONCE && pUpdatedMap) { + saveWinResult(&key, pResPos, pUpdatedMap); + } + + if (pInfo->twAggSup.calTrigger == STREAM_TRIGGER_WINDOW_CLOSE) { + tSimpleHashPut(pInfo->aggSup.pResultRowHashTable, &key, sizeof(SWinKey), &pResPos, POINTER_BYTES); + } + + updateTimeWindowInfo(&pInfo->twAggSup.timeWindowData, &nextWin, 1); + applyAggFunctionOnPartialTuples(pTaskInfo, pSup->pCtx, &pInfo->twAggSup.timeWindowData, startPos, forwardRows, + pSDataBlock->info.rows, numOfOutput); + key.ts = nextWin.skey; + + if (pInfo->delKey.ts > key.ts) { + pInfo->delKey = key; + } + int32_t prevEndPos = (forwardRows - 1) * step + startPos; + if (pSDataBlock->info.window.skey <= 0 || pSDataBlock->info.window.ekey <= 0) { + qError("table uid %" PRIu64 " data block timestamp range may not be calculated! minKey %" PRId64 + ",maxKey %" PRId64, + pSDataBlock->info.id.uid, pSDataBlock->info.window.skey, pSDataBlock->info.window.ekey); + blockDataUpdateTsWindow(pSDataBlock, 0); + + // timestamp of the data is incorrect + if (pSDataBlock->info.window.skey <= 0 || pSDataBlock->info.window.ekey <= 0) { + qError("table uid %" PRIu64 " data block timestamp is out of range! minKey %" PRId64 ",maxKey %" PRId64, + pSDataBlock->info.id.uid, pSDataBlock->info.window.skey, pSDataBlock->info.window.ekey); + } + } + startPos = getNextQualifiedFinalWindow(&pInfo->interval, &nextWin, &pSDataBlock->info, tsCol, prevEndPos); + if (startPos < 0) { + break; + } + } +} + +static SSDataBlock* doStreamMidIntervalAgg(SOperatorInfo* pOperator) { + SStreamIntervalOperatorInfo* pInfo = pOperator->info; + SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; + SStorageAPI* pAPI = &pOperator->pTaskInfo->storageAPI; + SOperatorInfo* downstream = pOperator->pDownstream[0]; + SExprSupp* pSup = &pOperator->exprSupp; + + qDebug("stask:%s %s status: %d", GET_TASKID(pTaskInfo), getStreamOpName(pOperator->operatorType), pOperator->status); + + if (pOperator->status == OP_EXEC_DONE) { + return NULL; + } else if (pOperator->status == OP_RES_TO_RETURN) { + SSDataBlock* resBlock = buildIntervalResult(pOperator); + if (resBlock != NULL) { + return resBlock; + } + + setOperatorCompleted(pOperator); + clearFunctionContext(&pOperator->exprSupp); + clearStreamIntervalOperator(pInfo); + setStreamDataVersion(pTaskInfo, pInfo->dataVersion, pInfo->pState->checkPointId); + qDebug("stask:%s ===stream===%s clear", GET_TASKID(pTaskInfo), getStreamOpName(pOperator->operatorType)); + return NULL; + } else { + doBuildDeleteResult(pInfo, pInfo->pDelWins, &pInfo->delIndex, pInfo->pDelRes); + if (pInfo->pDelRes->info.rows != 0) { + printDataBlock(pInfo->pDelRes, getStreamOpName(pOperator->operatorType), GET_TASKID(pTaskInfo)); + return pInfo->pDelRes; + } + } + + if (!pInfo->pUpdated) { + pInfo->pUpdated = taosArrayInit(4096, POINTER_BYTES); + } + if (!pInfo->pUpdatedMap) { + _hash_fn_t hashFn = taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY); + pInfo->pUpdatedMap = tSimpleHashInit(4096, hashFn); + } + + while (1) { + if (isTaskKilled(pTaskInfo)) { + if (pInfo->pUpdated != NULL) { + pInfo->pUpdated = taosArrayDestroy(pInfo->pUpdated); + } + + if (pInfo->pUpdatedMap != NULL) { + tSimpleHashCleanup(pInfo->pUpdatedMap); + pInfo->pUpdatedMap = NULL; + } + + T_LONG_JMP(pTaskInfo->env, pTaskInfo->code); + } + + SSDataBlock* pBlock = downstream->fpSet.getNextFn(downstream); + if (pBlock == NULL) { + pOperator->status = OP_RES_TO_RETURN; + qDebug("===stream===return data:%s. recv datablock num:%" PRIu64, getStreamOpName(pOperator->operatorType), + pInfo->numOfDatapack); + pInfo->numOfDatapack = 0; + break; + } + pInfo->numOfDatapack++; + printSpecDataBlock(pBlock, getStreamOpName(pOperator->operatorType), "recv", GET_TASKID(pTaskInfo)); + + if (pBlock->info.type == STREAM_NORMAL || pBlock->info.type == STREAM_PULL_DATA) { + pInfo->binfo.pRes->info.type = pBlock->info.type; + } else if (pBlock->info.type == STREAM_DELETE_DATA || pBlock->info.type == STREAM_DELETE_RESULT || + pBlock->info.type == STREAM_CLEAR) { + SArray* delWins = taosArrayInit(8, sizeof(SWinKey)); + doDeleteWindows(pOperator, &pInfo->interval, pBlock, delWins, pInfo->pUpdatedMap); + removeResults(delWins, pInfo->pUpdatedMap); + taosArrayAddAll(pInfo->pDelWins, delWins); + taosArrayDestroy(delWins); + + doBuildDeleteResult(pInfo, pInfo->pDelWins, &pInfo->delIndex, pInfo->pDelRes); + if (pInfo->pDelRes->info.rows != 0) { + // process the rest of the data + printDataBlock(pInfo->pDelRes, getStreamOpName(pOperator->operatorType), GET_TASKID(pTaskInfo)); + if (pBlock->info.type == STREAM_CLEAR) { + pInfo->pDelRes->info.type = STREAM_CLEAR; + } else { + pInfo->pDelRes->info.type = STREAM_DELETE_RESULT; + } + return pInfo->pDelRes; + } + break; + } else if (pBlock->info.type == STREAM_CREATE_CHILD_TABLE) { + return pBlock; + } else { + ASSERTS(pBlock->info.type == STREAM_INVALID, "invalid SSDataBlock type"); + } + + if (pInfo->scalarSupp.pExprInfo != NULL) { + SExprSupp* pExprSup = &pInfo->scalarSupp; + projectApplyFunctions(pExprSup->pExprInfo, pBlock, pBlock, pExprSup->pCtx, pExprSup->numOfExprs, NULL); + } + setInputDataBlock(pSup, pBlock, TSDB_ORDER_ASC, MAIN_SCAN, true); + doStreamMidIntervalAggImpl(pOperator, pBlock, pInfo->pUpdatedMap); + pInfo->twAggSup.maxTs = TMAX(pInfo->twAggSup.maxTs, pBlock->info.window.ekey); + pInfo->twAggSup.maxTs = TMAX(pInfo->twAggSup.maxTs, pBlock->info.watermark); + pInfo->twAggSup.minTs = TMIN(pInfo->twAggSup.minTs, pBlock->info.window.skey); + } + + removeDeleteResults(pInfo->pUpdatedMap, pInfo->pDelWins); + pInfo->binfo.pRes->info.watermark = pInfo->twAggSup.maxTs; + + void* pIte = NULL; + int32_t iter = 0; + while ((pIte = tSimpleHashIterate(pInfo->pUpdatedMap, pIte, &iter)) != NULL) { + taosArrayPush(pInfo->pUpdated, pIte); + } + + tSimpleHashCleanup(pInfo->pUpdatedMap); + pInfo->pUpdatedMap = NULL; + taosArraySort(pInfo->pUpdated, winPosCmprImpl); + + initMultiResInfoFromArrayList(&pInfo->groupResInfo, pInfo->pUpdated); + pInfo->pUpdated = NULL; + blockDataEnsureCapacity(pInfo->binfo.pRes, pOperator->resultInfo.capacity); + + return buildIntervalResult(pOperator); +} From 1b8ec28d53125935dfed4db2ec9390169f909033 Mon Sep 17 00:00:00 2001 From: liuyao <54liuyao@163.com> Date: Thu, 17 Aug 2023 11:45:26 +0800 Subject: [PATCH 02/51] format code --- source/libs/executor/src/streamtimewindowoperator.c | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/source/libs/executor/src/streamtimewindowoperator.c b/source/libs/executor/src/streamtimewindowoperator.c index 06566633c4..25e6faf14b 100644 --- a/source/libs/executor/src/streamtimewindowoperator.c +++ b/source/libs/executor/src/streamtimewindowoperator.c @@ -914,7 +914,6 @@ static void resetUnCloseWinInfo(SSHashObj* winMap) { } } - static SSDataBlock* buildIntervalResult(SOperatorInfo* pOperator) { SStreamIntervalOperatorInfo* pInfo = pOperator->info; SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; @@ -2392,7 +2391,8 @@ SOperatorInfo* createStreamFinalSessionAggOperatorInfo(SOperatorInfo* downstream pOperator->fpSet = createOperatorFpSet(optrDummyOpenFn, doStreamSessionSemiAgg, NULL, destroyStreamSessionAggOperatorInfo, optrDefaultBufFn, NULL); } - setOperatorInfo(pOperator, getStreamOpName(pOperator->operatorType), pPhyNode->type, false, OP_NOT_OPENED, pInfo, pTaskInfo); + setOperatorInfo(pOperator, getStreamOpName(pOperator->operatorType), pPhyNode->type, false, OP_NOT_OPENED, pInfo, + pTaskInfo); if (numOfChild > 0) { pInfo->pChildren = taosArrayInit(numOfChild, sizeof(void*)); @@ -3217,8 +3217,7 @@ static void doStreamMidIntervalAggImpl(SOperatorInfo* pOperator, SSDataBlock* pS while (1) { bool isClosed = isCloseWindow(&nextWin, &pInfo->twAggSup); - if ((pInfo->ignoreExpiredData && isClosed) || - !inSlidingWindow(&pInfo->interval, &nextWin, &pSDataBlock->info)) { + if ((pInfo->ignoreExpiredData && isClosed) || !inSlidingWindow(&pInfo->interval, &nextWin, &pSDataBlock->info)) { startPos = getNexWindowPos(&pInfo->interval, &pSDataBlock->info, tsCol, startPos, nextWin.ekey, &nextWin); if (startPos < 0) { break; From 3e3c8eccbb2a346718beffb6aa17f8c876dfffa5 Mon Sep 17 00:00:00 2001 From: liuyao <54liuyao@163.com> Date: Fri, 18 Aug 2023 14:42:07 +0800 Subject: [PATCH 03/51] adj op --- source/libs/executor/src/streamtimewindowoperator.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/libs/executor/src/streamtimewindowoperator.c b/source/libs/executor/src/streamtimewindowoperator.c index 25e6faf14b..afa49302bc 100644 --- a/source/libs/executor/src/streamtimewindowoperator.c +++ b/source/libs/executor/src/streamtimewindowoperator.c @@ -3236,7 +3236,7 @@ static void doStreamMidIntervalAggImpl(SOperatorInfo* pOperator, SSDataBlock* pS .ts = pResult->win.skey, .groupId = groupId, }; - if (pInfo->twAggSup.calTrigger == STREAM_TRIGGER_AT_ONCE && pUpdatedMap) { + if (pInfo->twAggSup.calTrigger == STREAM_TRIGGER_AT_ONCE) { saveWinResult(&key, pResPos, pUpdatedMap); } From a17f0c808d23526cbedeec492f88a86edfc1d629 Mon Sep 17 00:00:00 2001 From: liuyao <54liuyao@163.com> Date: Fri, 1 Sep 2023 14:11:38 +0800 Subject: [PATCH 04/51] add node type --- include/common/tmsg.h | 1 + 1 file changed, 1 insertion(+) diff --git a/include/common/tmsg.h b/include/common/tmsg.h index 8deec53470..4788a5d9ce 100644 --- a/include/common/tmsg.h +++ b/include/common/tmsg.h @@ -395,6 +395,7 @@ typedef enum ENodeType { QUERY_NODE_PHYSICAL_PLAN_STREAM_INTERVAL, QUERY_NODE_PHYSICAL_PLAN_STREAM_FINAL_INTERVAL, QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_INTERVAL, + QUERY_NODE_PHYSICAL_PLAN_STREAM_MID_INTERVAL, QUERY_NODE_PHYSICAL_PLAN_FILL, QUERY_NODE_PHYSICAL_PLAN_STREAM_FILL, QUERY_NODE_PHYSICAL_PLAN_MERGE_SESSION, From cfcbd53024333d2b71e3f5879fb90720090e5e41 Mon Sep 17 00:00:00 2001 From: liuyao <54liuyao@163.com> Date: Fri, 1 Sep 2023 14:14:54 +0800 Subject: [PATCH 05/51] merge code --- source/libs/executor/src/streamtimewindowoperator.c | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/source/libs/executor/src/streamtimewindowoperator.c b/source/libs/executor/src/streamtimewindowoperator.c index 8c48997dc4..dec4a9c699 100644 --- a/source/libs/executor/src/streamtimewindowoperator.c +++ b/source/libs/executor/src/streamtimewindowoperator.c @@ -3797,7 +3797,7 @@ static void doStreamMidIntervalAggImpl(SOperatorInfo* pOperator, SSDataBlock* pS } int32_t code = setIntervalOutputBuf(pInfo->pState, &nextWin, &pResPos, groupId, pSup->pCtx, numOfOutput, - pSup->rowEntryInfoOffset, &pInfo->aggSup, &pInfo->statestore); + pSup->rowEntryInfoOffset, &pInfo->aggSup, &pInfo->stateStore); pResult = (SResultRow*)pResPos->pRowBuff; if (code != TSDB_CODE_SUCCESS || pResult == NULL) { T_LONG_JMP(pTaskInfo->env, TSDB_CODE_OUT_OF_MEMORY); @@ -3863,7 +3863,6 @@ static SSDataBlock* doStreamMidIntervalAgg(SOperatorInfo* pOperator) { setOperatorCompleted(pOperator); clearFunctionContext(&pOperator->exprSupp); clearStreamIntervalOperator(pInfo); - setStreamDataVersion(pTaskInfo, pInfo->dataVersion, pInfo->pState->checkPointId); qDebug("stask:%s ===stream===%s clear", GET_TASKID(pTaskInfo), getStreamOpName(pOperator->operatorType)); return NULL; } else { From fc9dfc77d079d0451be3803d9dd1e489258e0621 Mon Sep 17 00:00:00 2001 From: wangmm0220 Date: Tue, 5 Dec 2023 20:07:20 +0800 Subject: [PATCH 06/51] fix:add plan for multi agg operator in stream --- include/libs/nodes/plannodes.h | 2 + source/common/src/tglobal.c | 2 +- source/dnode/mnode/impl/src/mndStream.c | 31 +-------- source/dnode/mnode/impl/src/mndSubscribe.c | 1 + source/dnode/mnode/impl/test/CMakeLists.txt | 1 + .../mnode/impl/test/stream/CMakeLists.txt | 12 ++++ .../mnode/impl/test/stream/streamPlanTest.cpp | 66 +++++++++++++++++++ source/dnode/mnode/sdb/src/sdbHash.c | 2 +- source/libs/executor/src/executor.c | 4 +- source/libs/executor/src/operator.c | 3 + source/libs/executor/src/scanoperator.c | 1 + .../executor/src/streamtimewindowoperator.c | 9 ++- source/libs/nodes/src/nodesCloneFuncs.c | 1 + source/libs/nodes/src/nodesCodeFuncs.c | 4 ++ source/libs/nodes/src/nodesMsgFuncs.c | 2 + source/libs/nodes/src/nodesUtilFuncs.c | 3 + source/libs/planner/src/planPhysiCreater.c | 2 + source/libs/planner/src/planSpliter.c | 44 +++++++++++-- source/libs/planner/src/planValidator.c | 1 + 19 files changed, 149 insertions(+), 42 deletions(-) create mode 100644 source/dnode/mnode/impl/test/stream/CMakeLists.txt create mode 100644 source/dnode/mnode/impl/test/stream/streamPlanTest.cpp diff --git a/include/libs/nodes/plannodes.h b/include/libs/nodes/plannodes.h index e29750d8a0..fc62800dda 100644 --- a/include/libs/nodes/plannodes.h +++ b/include/libs/nodes/plannodes.h @@ -253,6 +253,7 @@ typedef enum EWindowAlgorithm { SESSION_ALGO_STREAM_FINAL, SESSION_ALGO_STREAM_SINGLE, SESSION_ALGO_MERGE, + INTERVAL_ALGO_STREAM_MID, } EWindowAlgorithm; typedef struct SWindowLogicNode { @@ -579,6 +580,7 @@ typedef SIntervalPhysiNode SMergeAlignedIntervalPhysiNode; typedef SIntervalPhysiNode SStreamIntervalPhysiNode; typedef SIntervalPhysiNode SStreamFinalIntervalPhysiNode; typedef SIntervalPhysiNode SStreamSemiIntervalPhysiNode; +typedef SIntervalPhysiNode SStreamMidIntervalPhysiNode; typedef struct SFillPhysiNode { SPhysiNode node; diff --git a/source/common/src/tglobal.c b/source/common/src/tglobal.c index d9aecd96d7..c6b964f348 100644 --- a/source/common/src/tglobal.c +++ b/source/common/src/tglobal.c @@ -147,7 +147,7 @@ bool tsEnableQueryHb = true; bool tsEnableScience = false; // on taos-cli show float and doulbe with scientific notation if true int32_t tsQuerySmaOptimize = 0; int32_t tsQueryRsmaTolerance = 1000; // the tolerance time (ms) to judge from which level to query rsma data. -bool tsQueryPlannerTrace = false; +bool tsQueryPlannerTrace = true; int32_t tsQueryNodeChunkSize = 32 * 1024; bool tsQueryUseNodeAllocator = true; bool tsKeepColumnName = false; diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index 235703428f..b424777707 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -328,35 +328,6 @@ static int32_t mndCheckCreateStreamReq(SCMCreateStreamReq *pCreate) { return 0; } -static int32_t mndStreamGetPlanString(const char *ast, int8_t triggerType, int64_t watermark, char **pStr) { - if (NULL == ast) { - return TSDB_CODE_SUCCESS; - } - - SNode * pAst = NULL; - int32_t code = nodesStringToNode(ast, &pAst); - - SQueryPlan *pPlan = NULL; - if (TSDB_CODE_SUCCESS == code) { - SPlanContext cxt = { - .pAstRoot = pAst, - .topicQuery = false, - .streamQuery = true, - .triggerType = (triggerType == STREAM_TRIGGER_MAX_DELAY) ? STREAM_TRIGGER_WINDOW_CLOSE : triggerType, - .watermark = watermark, - }; - code = qCreateQueryPlan(&cxt, &pPlan, NULL); - } - - if (TSDB_CODE_SUCCESS == code) { - code = nodesNodeToString((SNode *)pPlan, false, pStr, NULL); - } - nodesDestroyNode(pAst); - nodesDestroyNode((SNode *)pPlan); - terrno = code; - return code; -} - static int32_t mndBuildStreamObjFromCreateReq(SMnode *pMnode, SStreamObj *pObj, SCMCreateStreamReq *pCreate) { SNode * pAst = NULL; SQueryPlan *pPlan = NULL; @@ -768,6 +739,7 @@ static int32_t mndProcessCreateStreamReq(SRpcMsg *pReq) { SDbObj * pDb = NULL; SCMCreateStreamReq createStreamReq = {0}; SStreamObj streamObj = {0}; + char* sql = NULL; if (tDeserializeSCMCreateStreamReq(pReq->pCont, pReq->contLen, &createStreamReq) != 0) { terrno = TSDB_CODE_INVALID_MSG; @@ -799,7 +771,6 @@ static int32_t mndProcessCreateStreamReq(SRpcMsg *pReq) { goto _OVER; } - char* sql = NULL; int32_t sqlLen = 0; if(createStreamReq.sql != NULL){ sqlLen = strlen(createStreamReq.sql); diff --git a/source/dnode/mnode/impl/src/mndSubscribe.c b/source/dnode/mnode/impl/src/mndSubscribe.c index 408b664e50..320587bb57 100644 --- a/source/dnode/mnode/impl/src/mndSubscribe.c +++ b/source/dnode/mnode/impl/src/mndSubscribe.c @@ -161,6 +161,7 @@ static int32_t mndBuildSubChangeReq(void **pBuf, int32_t *pLen, SMqSubscribeObj static int32_t mndPersistSubChangeVgReq(SMnode *pMnode, STrans *pTrans, SMqSubscribeObj *pSub, const SMqRebOutputVg *pRebVg, SSubplan* pPlan) { if (pRebVg->oldConsumerId == pRebVg->newConsumerId) { + if(pRebVg->oldConsumerId == -1) return 0; terrno = TSDB_CODE_MND_INVALID_SUB_OPTION; return -1; } diff --git a/source/dnode/mnode/impl/test/CMakeLists.txt b/source/dnode/mnode/impl/test/CMakeLists.txt index a002b20bde..7463f1fdce 100644 --- a/source/dnode/mnode/impl/test/CMakeLists.txt +++ b/source/dnode/mnode/impl/test/CMakeLists.txt @@ -14,4 +14,5 @@ add_subdirectory(snode) add_subdirectory(stb) add_subdirectory(topic) add_subdirectory(trans) +add_subdirectory(stream) #add_subdirectory(user) diff --git a/source/dnode/mnode/impl/test/stream/CMakeLists.txt b/source/dnode/mnode/impl/test/stream/CMakeLists.txt new file mode 100644 index 0000000000..77ac366b0d --- /dev/null +++ b/source/dnode/mnode/impl/test/stream/CMakeLists.txt @@ -0,0 +1,12 @@ +aux_source_directory(. MNODE_STREAM_PLAN_TEST_SRC) +add_executable(streamPlanTest ${MNODE_STREAM_PLAN_TEST_SRC}) +target_link_libraries( + streamPlanTest + PUBLIC nodes planner gtest qcom +) + +add_test( + NAME streamPlanTest + COMMAND streamPlanTest + +) diff --git a/source/dnode/mnode/impl/test/stream/streamPlanTest.cpp b/source/dnode/mnode/impl/test/stream/streamPlanTest.cpp new file mode 100644 index 0000000000..9b3587cb3a --- /dev/null +++ b/source/dnode/mnode/impl/test/stream/streamPlanTest.cpp @@ -0,0 +1,66 @@ +/* + * Copyright (c) 2019 TAOS Data, Inc. + * + * This program is free software: you can use, redistribute, and/or modify + * it under the terms of the GNU Affero General Public License, version 3 + * or later ("AGPL"), as published by the Free Software Foundation. + * + * This program is distributed in the hope that it will be useful, but WITHOUT + * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or + * FITNESS FOR A PARTICULAR PURPOSE. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + */ + +#include + +#include +#include +#include +#include "nodes.h" +#include "planner.h" + +#pragma GCC diagnostic push +#pragma GCC diagnostic ignored "-Wwrite-strings" +#pragma GCC diagnostic ignored "-Wunused-function" +#pragma GCC diagnostic ignored "-Wunused-variable" +#pragma GCC diagnostic ignored "-Wsign-compare" + +int main(int argc, char **argv) { + testing::InitGoogleTest(&argc, argv); + + return RUN_ALL_TESTS(); +} + +TEST(testCase, plan_Test) { + char* ast = "{\"NodeType\":\"101\",\"Name\":\"SelectStmt\",\"SelectStmt\":{\"Distinct\":false,\"Projections\":[{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"9\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"#expr_1\",\"UserAlias\":\"_wstart\",\"Name\":\"_wstart\",\"Id\":\"89\",\"Type\":\"3505\",\"UdfBufSize\":\"0\"}},{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"5\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"#expr_2\",\"UserAlias\":\"sum(voltage)\",\"Name\":\"sum\",\"Id\":\"1\",\"Type\":\"14\",\"Parameters\":[{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"4\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"4\"},\"AliasName\":\"voltage\",\"UserAlias\":\"voltage\",\"TableId\":\"6555383776122680534\",\"TableType\":\"1\",\"ColId\":\"3\",\"ProjId\":\"0\",\"ColType\":\"1\",\"DbName\":\"test\",\"TableName\":\"meters\",\"TableAlias\":\"meters\",\"ColName\":\"voltage\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"4\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"4\"},\"AliasName\":\"#expr_3\",\"UserAlias\":\"groupid\",\"Name\":\"_group_key\",\"Id\":\"96\",\"Type\":\"3754\",\"Parameters\":[{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"4\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"4\"},\"AliasName\":\"#expr_3\",\"UserAlias\":\"groupid\",\"TableId\":\"6555383776122680534\",\"TableType\":\"1\",\"ColId\":\"5\",\"ProjId\":\"0\",\"ColType\":\"2\",\"DbName\":\"test\",\"TableName\":\"meters\",\"TableAlias\":\"meters\",\"ColName\":\"groupid\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}}],\"UdfBufSize\":\"0\"}}],\"From\":{\"NodeType\":\"6\",\"Name\":\"RealTable\",\"RealTable\":{\"DataType\":{\"Type\":\"0\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"0\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"DbName\":\"test\",\"tableName\":\"meters\",\"tableAlias\":\"meters\",\"MetaSize\":\"475\",\"Meta\":{\"VgId\":\"0\",\"TableType\":\"1\",\"Uid\":\"6555383776122680534\",\"Suid\":\"6555383776122680534\",\"Sversion\":\"1\",\"Tversion\":\"1\",\"ComInfo\":{\"NumOfTags\":\"2\",\"Precision\":\"0\",\"NumOfColumns\":\"4\",\"RowSize\":\"20\"},\"ColSchemas\":[{\"Type\":\"9\",\"ColId\":\"1\",\"bytes\":\"8\",\"Name\":\"ts\"},{\"Type\":\"6\",\"ColId\":\"2\",\"bytes\":\"4\",\"Name\":\"current\"},{\"Type\":\"4\",\"ColId\":\"3\",\"bytes\":\"4\",\"Name\":\"voltage\"},{\"Type\":\"6\",\"ColId\":\"4\",\"bytes\":\"4\",\"Name\":\"phase\"},{\"Type\":\"4\",\"ColId\":\"5\",\"bytes\":\"4\",\"Name\":\"groupid\"},{\"Type\":\"8\",\"ColId\":\"6\",\"bytes\":\"26\",\"Name\":\"location\"}]},\"VgroupsInfoSize\":\"1340\",\"VgroupsInfo\":{\"Num\":\"2\",\"Vgroups\":[{\"VgId\":\"2\",\"HashBegin\":\"0\",\"HashEnd\":\"2147483646\",\"EpSet\":{\"InUse\":\"0\",\"NumOfEps\":\"1\",\"Eps\":[{\"Fqdn\":\"localhost\",\"Port\":\"6030\"}]},\"NumOfTable\":\"0\"},{\"VgId\":\"3\",\"HashBegin\":\"2147483647\",\"HashEnd\":\"4294967295\",\"EpSet\":{\"InUse\":\"0\",\"NumOfEps\":\"1\",\"Eps\":[{\"Fqdn\":\"localhost\",\"Port\":\"6030\"}]},\"NumOfTable\":\"0\"}]}}},\"PartitionBy\":[{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"4\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"4\"},\"AliasName\":\"groupid\",\"UserAlias\":\"groupid\",\"TableId\":\"6555383776122680534\",\"TableType\":\"1\",\"ColId\":\"5\",\"ProjId\":\"0\",\"ColType\":\"2\",\"DbName\":\"test\",\"TableName\":\"meters\",\"TableAlias\":\"meters\",\"ColName\":\"groupid\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}}],\"Window\":{\"NodeType\":\"14\",\"Name\":\"IntervalWindow\",\"IntervalWindow\":{\"Interval\":{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"5\",\"Precision\":\"0\",\"Scale\":\"115\",\"Bytes\":\"8\"},\"AliasName\":\"c804c3a15ebe05b5baf40ad5ee12be1f\",\"UserAlias\":\"2s\",\"LiteralSize\":\"2\",\"Literal\":\"2s\",\"Duration\":true,\"Translate\":true,\"NotReserved\":false,\"IsNull\":false,\"Unit\":\"115\",\"Datum\":\"2000\"}},\"TsPk\":{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"9\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"ts\",\"UserAlias\":\"ts\",\"TableId\":\"6555383776122680534\",\"TableType\":\"1\",\"ColId\":\"1\",\"ProjId\":\"0\",\"ColType\":\"1\",\"DbName\":\"test\",\"TableName\":\"meters\",\"TableAlias\":\"meters\",\"ColName\":\"ts\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}}}},\"StmtName\":\"0x1580095ba\",\"HasAggFuncs\":true}}"; + SNode * pAst = NULL; + SQueryPlan *pPlan = NULL; + + if (taosCreateLog("taoslog", 10, "/etc/taos", NULL, NULL, NULL, NULL, 1) != 0) { + // ignore create log failed, only print + printf(" WARING: Create failed:%s. configDir\n", strerror(errno)); + } + + if (nodesStringToNode(ast, &pAst) < 0) { + ASSERT(0); + } + + SPlanContext cxt = { + .pAstRoot = pAst, + .topicQuery = false, + .streamQuery = true, + .triggerType = STREAM_TRIGGER_WINDOW_CLOSE, + .watermark = 1, + .igExpired = 1, + .deleteMark = 1, + .igCheckUpdate = 1, + }; + + // using ast and param to build physical plan + if (qCreateQueryPlan(&cxt, &pPlan, NULL) < 0) { + ASSERT(0); + } + +} diff --git a/source/dnode/mnode/sdb/src/sdbHash.c b/source/dnode/mnode/sdb/src/sdbHash.c index df5c399da8..5f275f912d 100644 --- a/source/dnode/mnode/sdb/src/sdbHash.c +++ b/source/dnode/mnode/sdb/src/sdbHash.c @@ -180,7 +180,7 @@ static int32_t sdbInsertRow(SSdb *pSdb, SHashObj *hash, SSdbRaw *pRaw, SSdbRow * pSdb->maxId[pRow->type] = TMAX(pSdb->maxId[pRow->type], *((int32_t *)pRow->pObj)); } if (pSdb->keyTypes[pRow->type] == SDB_KEY_INT64) { - pSdb->maxId[pRow->type] = TMAX(pSdb->maxId[pRow->type], *((int32_t *)pRow->pObj)); + pSdb->maxId[pRow->type] = TMAX(pSdb->maxId[pRow->type], *((int64_t *)pRow->pObj)); } pSdb->tableVer[pRow->type]++; diff --git a/source/libs/executor/src/executor.c b/source/libs/executor/src/executor.c index 1f82a9477b..c445da6e30 100644 --- a/source/libs/executor/src/executor.c +++ b/source/libs/executor/src/executor.c @@ -943,7 +943,7 @@ int32_t qSetStreamOperatorOptionForScanHistory(qTaskInfo_t tinfo) { while (1) { 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) { + type == QUERY_NODE_PHYSICAL_PLAN_STREAM_FINAL_INTERVAL || type == QUERY_NODE_PHYSICAL_PLAN_STREAM_MID_INTERVAL) { SStreamIntervalOperatorInfo* pInfo = pOperator->info; STimeWindowAggSupp* pSup = &pInfo->twAggSup; @@ -1016,7 +1016,7 @@ int32_t qRestoreStreamOperatorOption(qTaskInfo_t tinfo) { while (1) { uint16_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) { + type == QUERY_NODE_PHYSICAL_PLAN_STREAM_FINAL_INTERVAL || type == QUERY_NODE_PHYSICAL_PLAN_STREAM_MID_INTERVAL) { SStreamIntervalOperatorInfo* pInfo = pOperator->info; pInfo->twAggSup.calTrigger = pInfo->twAggSup.calTriggerSaved; pInfo->twAggSup.deleteMark = pInfo->twAggSup.deleteMarkSaved; diff --git a/source/libs/executor/src/operator.c b/source/libs/executor/src/operator.c index 69a8acb3d7..0c6671c742 100644 --- a/source/libs/executor/src/operator.c +++ b/source/libs/executor/src/operator.c @@ -489,6 +489,9 @@ SOperatorInfo* createOperator(SPhysiNode* pPhyNode, SExecTaskInfo* pTaskInfo, SR } else if (QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_INTERVAL == type) { int32_t children = 0; pOptr = createStreamFinalIntervalOperatorInfo(ops[0], pPhyNode, pTaskInfo, children, pHandle); + } else if (QUERY_NODE_PHYSICAL_PLAN_STREAM_MID_INTERVAL == type) { + int32_t children = 0; + pOptr = createStreamFinalIntervalOperatorInfo(ops[0], pPhyNode, pTaskInfo, children, pHandle); } else if (QUERY_NODE_PHYSICAL_PLAN_STREAM_FINAL_INTERVAL == type) { int32_t children = pHandle->numOfVgroups; pOptr = createStreamFinalIntervalOperatorInfo(ops[0], pPhyNode, pTaskInfo, children, pHandle); diff --git a/source/libs/executor/src/scanoperator.c b/source/libs/executor/src/scanoperator.c index 448c585869..e3068ce319 100644 --- a/source/libs/executor/src/scanoperator.c +++ b/source/libs/executor/src/scanoperator.c @@ -1134,6 +1134,7 @@ static bool isStateWindow(SStreamScanInfo* pInfo) { static bool isIntervalWindow(SStreamScanInfo* pInfo) { return pInfo->windowSup.parentType == QUERY_NODE_PHYSICAL_PLAN_STREAM_INTERVAL || pInfo->windowSup.parentType == QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_INTERVAL || + pInfo->windowSup.parentType == QUERY_NODE_PHYSICAL_PLAN_STREAM_MID_INTERVAL || pInfo->windowSup.parentType == QUERY_NODE_PHYSICAL_PLAN_STREAM_FINAL_INTERVAL; } diff --git a/source/libs/executor/src/streamtimewindowoperator.c b/source/libs/executor/src/streamtimewindowoperator.c index 92175efa86..4a88c160e6 100644 --- a/source/libs/executor/src/streamtimewindowoperator.c +++ b/source/libs/executor/src/streamtimewindowoperator.c @@ -1380,7 +1380,8 @@ static int32_t getMaxFunResSize(SExprSupp* pSup, int32_t numOfCols) { } static void streamIntervalReleaseState(SOperatorInfo* pOperator) { - if (pOperator->operatorType != QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_INTERVAL) { + if (pOperator->operatorType != QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_INTERVAL && + pOperator->operatorType != QUERY_NODE_PHYSICAL_PLAN_STREAM_MID_INTERVAL) { SStreamIntervalOperatorInfo* pInfo = pOperator->info; int32_t resSize = sizeof(TSKEY); pInfo->stateStore.streamStateSaveInfo(pInfo->pState, STREAM_INTERVAL_OP_STATE_NAME, @@ -1396,7 +1397,8 @@ static void streamIntervalReleaseState(SOperatorInfo* pOperator) { } void streamIntervalReloadState(SOperatorInfo* pOperator) { - if (pOperator->operatorType != QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_INTERVAL) { + if (pOperator->operatorType != QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_INTERVAL && + pOperator->operatorType != QUERY_NODE_PHYSICAL_PLAN_STREAM_MID_INTERVAL) { SStreamIntervalOperatorInfo* pInfo = pOperator->info; int32_t size = 0; void* pBuf = NULL; @@ -1512,7 +1514,8 @@ SOperatorInfo* createStreamFinalIntervalOperatorInfo(SOperatorInfo* downstream, pOperator->fpSet = createOperatorFpSet(NULL, doStreamFinalIntervalAgg, NULL, destroyStreamFinalIntervalOperatorInfo, optrDefaultBufFn, NULL, optrDefaultGetNextExtFn, NULL); setOperatorStreamStateFn(pOperator, streamIntervalReleaseState, streamIntervalReloadState); - if (pPhyNode->type == QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_INTERVAL) { + if (pPhyNode->type == QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_INTERVAL || + pPhyNode->type == QUERY_NODE_PHYSICAL_PLAN_STREAM_MID_INTERVAL) { initIntervalDownStream(downstream, pPhyNode->type, pInfo); } code = appendDownstream(pOperator, &downstream, 1); diff --git a/source/libs/nodes/src/nodesCloneFuncs.c b/source/libs/nodes/src/nodesCloneFuncs.c index ce23928268..785f0bc70d 100644 --- a/source/libs/nodes/src/nodesCloneFuncs.c +++ b/source/libs/nodes/src/nodesCloneFuncs.c @@ -921,6 +921,7 @@ SNode* nodesCloneNode(const SNode* pNode) { case QUERY_NODE_PHYSICAL_PLAN_STREAM_INTERVAL: case QUERY_NODE_PHYSICAL_PLAN_STREAM_FINAL_INTERVAL: case QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_INTERVAL: + case QUERY_NODE_PHYSICAL_PLAN_STREAM_MID_INTERVAL: code = physiIntervalCopy((const SIntervalPhysiNode*)pNode, (SIntervalPhysiNode*)pDst); break; case QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_SESSION: diff --git a/source/libs/nodes/src/nodesCodeFuncs.c b/source/libs/nodes/src/nodesCodeFuncs.c index f3087dd5d4..47a74a3785 100644 --- a/source/libs/nodes/src/nodesCodeFuncs.c +++ b/source/libs/nodes/src/nodesCodeFuncs.c @@ -357,6 +357,8 @@ const char* nodesNodeName(ENodeType type) { return "PhysiStreamFinalInterval"; case QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_INTERVAL: return "PhysiStreamSemiInterval"; + case QUERY_NODE_PHYSICAL_PLAN_STREAM_MID_INTERVAL: + return "PhysiStreamMidInterval"; case QUERY_NODE_PHYSICAL_PLAN_FILL: case QUERY_NODE_PHYSICAL_PLAN_STREAM_FILL: return "PhysiFill"; @@ -7082,6 +7084,7 @@ static int32_t specificNodeToJson(const void* pObj, SJson* pJson) { case QUERY_NODE_PHYSICAL_PLAN_STREAM_INTERVAL: case QUERY_NODE_PHYSICAL_PLAN_STREAM_FINAL_INTERVAL: case QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_INTERVAL: + case QUERY_NODE_PHYSICAL_PLAN_STREAM_MID_INTERVAL: return physiIntervalNodeToJson(pObj, pJson); case QUERY_NODE_PHYSICAL_PLAN_FILL: case QUERY_NODE_PHYSICAL_PLAN_STREAM_FILL: @@ -7413,6 +7416,7 @@ static int32_t jsonToSpecificNode(const SJson* pJson, void* pObj) { case QUERY_NODE_PHYSICAL_PLAN_STREAM_INTERVAL: case QUERY_NODE_PHYSICAL_PLAN_STREAM_FINAL_INTERVAL: case QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_INTERVAL: + case QUERY_NODE_PHYSICAL_PLAN_STREAM_MID_INTERVAL: return jsonToPhysiIntervalNode(pJson, pObj); case QUERY_NODE_PHYSICAL_PLAN_FILL: case QUERY_NODE_PHYSICAL_PLAN_STREAM_FILL: diff --git a/source/libs/nodes/src/nodesMsgFuncs.c b/source/libs/nodes/src/nodesMsgFuncs.c index 9804f2075b..48ebd8d886 100644 --- a/source/libs/nodes/src/nodesMsgFuncs.c +++ b/source/libs/nodes/src/nodesMsgFuncs.c @@ -4139,6 +4139,7 @@ static int32_t specificNodeToMsg(const void* pObj, STlvEncoder* pEncoder) { case QUERY_NODE_PHYSICAL_PLAN_STREAM_INTERVAL: case QUERY_NODE_PHYSICAL_PLAN_STREAM_FINAL_INTERVAL: case QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_INTERVAL: + case QUERY_NODE_PHYSICAL_PLAN_STREAM_MID_INTERVAL: code = physiIntervalNodeToMsg(pObj, pEncoder); break; case QUERY_NODE_PHYSICAL_PLAN_FILL: @@ -4293,6 +4294,7 @@ static int32_t msgToSpecificNode(STlvDecoder* pDecoder, void* pObj) { case QUERY_NODE_PHYSICAL_PLAN_STREAM_INTERVAL: case QUERY_NODE_PHYSICAL_PLAN_STREAM_FINAL_INTERVAL: case QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_INTERVAL: + case QUERY_NODE_PHYSICAL_PLAN_STREAM_MID_INTERVAL: code = msgToPhysiIntervalNode(pDecoder, pObj); break; case QUERY_NODE_PHYSICAL_PLAN_FILL: diff --git a/source/libs/nodes/src/nodesUtilFuncs.c b/source/libs/nodes/src/nodesUtilFuncs.c index d167d81c82..22417086fe 100644 --- a/source/libs/nodes/src/nodesUtilFuncs.c +++ b/source/libs/nodes/src/nodesUtilFuncs.c @@ -554,6 +554,8 @@ SNode* nodesMakeNode(ENodeType type) { return makeNode(type, sizeof(SStreamFinalIntervalPhysiNode)); case QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_INTERVAL: return makeNode(type, sizeof(SStreamSemiIntervalPhysiNode)); + case QUERY_NODE_PHYSICAL_PLAN_STREAM_MID_INTERVAL: + return makeNode(type, sizeof(SStreamMidIntervalPhysiNode)); case QUERY_NODE_PHYSICAL_PLAN_FILL: case QUERY_NODE_PHYSICAL_PLAN_STREAM_FILL: return makeNode(type, sizeof(SFillPhysiNode)); @@ -1363,6 +1365,7 @@ void nodesDestroyNode(SNode* pNode) { case QUERY_NODE_PHYSICAL_PLAN_STREAM_INTERVAL: case QUERY_NODE_PHYSICAL_PLAN_STREAM_FINAL_INTERVAL: case QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_INTERVAL: + case QUERY_NODE_PHYSICAL_PLAN_STREAM_MID_INTERVAL: destroyWinodwPhysiNode((SWindowPhysiNode*)pNode); break; case QUERY_NODE_PHYSICAL_PLAN_FILL: diff --git a/source/libs/planner/src/planPhysiCreater.c b/source/libs/planner/src/planPhysiCreater.c index 598bce3133..fac93948f5 100644 --- a/source/libs/planner/src/planPhysiCreater.c +++ b/source/libs/planner/src/planPhysiCreater.c @@ -1587,6 +1587,8 @@ static ENodeType getIntervalOperatorType(EWindowAlgorithm windowAlgo) { return QUERY_NODE_PHYSICAL_PLAN_STREAM_FINAL_INTERVAL; case INTERVAL_ALGO_STREAM_SEMI: return QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_INTERVAL; + case INTERVAL_ALGO_STREAM_MID: + return QUERY_NODE_PHYSICAL_PLAN_STREAM_MID_INTERVAL; case INTERVAL_ALGO_STREAM_SINGLE: return QUERY_NODE_PHYSICAL_PLAN_STREAM_INTERVAL; case SESSION_ALGO_STREAM_FINAL: diff --git a/source/libs/planner/src/planSpliter.c b/source/libs/planner/src/planSpliter.c index 43bd8a5589..25d575a2ca 100644 --- a/source/libs/planner/src/planSpliter.c +++ b/source/libs/planner/src/planSpliter.c @@ -635,18 +635,52 @@ static int32_t stbSplSplitIntervalForBatch(SSplitContext* pCxt, SStableSplitInfo return code; } +//static int32_t stbSplSplitIntervalForStream(SSplitContext* pCxt, SStableSplitInfo* pInfo) { +// SLogicNode* pPartWindow = NULL; +// int32_t code = stbSplCreatePartWindowNode((SWindowLogicNode*)pInfo->pSplitNode, &pPartWindow); +// if (TSDB_CODE_SUCCESS == code) { +// ((SWindowLogicNode*)pPartWindow)->windowAlgo = INTERVAL_ALGO_STREAM_SEMI; +// ((SWindowLogicNode*)pInfo->pSplitNode)->windowAlgo = INTERVAL_ALGO_STREAM_FINAL; +// code = stbSplCreateExchangeNode(pCxt, pInfo->pSplitNode, pPartWindow); +// } +// if (TSDB_CODE_SUCCESS == code) { +// code = nodesListMakeStrictAppend(&pInfo->pSubplan->pChildren, +// (SNode*)splCreateScanSubplan(pCxt, pPartWindow, SPLIT_FLAG_STABLE_SPLIT)); +// } +// pInfo->pSubplan->subplanType = SUBPLAN_TYPE_MERGE; +// ++(pCxt->groupId); +// return code; +//} + static int32_t stbSplSplitIntervalForStream(SSplitContext* pCxt, SStableSplitInfo* pInfo) { SLogicNode* pPartWindow = NULL; - int32_t code = stbSplCreatePartWindowNode((SWindowLogicNode*)pInfo->pSplitNode, &pPartWindow); + SLogicNode* pMidWindow = NULL; + int32_t code = stbSplCreatePartWindowNode((SWindowLogicNode*)pInfo->pSplitNode, &pMidWindow); if (TSDB_CODE_SUCCESS == code) { - ((SWindowLogicNode*)pPartWindow)->windowAlgo = INTERVAL_ALGO_STREAM_SEMI; + ((SWindowLogicNode*)pMidWindow)->windowAlgo = INTERVAL_ALGO_STREAM_MID; ((SWindowLogicNode*)pInfo->pSplitNode)->windowAlgo = INTERVAL_ALGO_STREAM_FINAL; - code = stbSplCreateExchangeNode(pCxt, pInfo->pSplitNode, pPartWindow); + code = stbSplCreateExchangeNode(pCxt, pInfo->pSplitNode, pMidWindow); } + if (TSDB_CODE_SUCCESS == code) { - code = nodesListMakeStrictAppend(&pInfo->pSubplan->pChildren, - (SNode*)splCreateScanSubplan(pCxt, pPartWindow, SPLIT_FLAG_STABLE_SPLIT)); + code = stbSplCreatePartWindowNode((SWindowLogicNode*)pMidWindow, &pPartWindow); + if (TSDB_CODE_SUCCESS == code) { + ((SWindowLogicNode*)pPartWindow)->windowAlgo = INTERVAL_ALGO_STREAM_SEMI; + code = stbSplCreateExchangeNode(pCxt, pMidWindow, pPartWindow); + } } + + if (TSDB_CODE_SUCCESS == code) { + SNode* subPlan = (SNode*)splCreateSubplan(pCxt, pMidWindow); + ((SLogicSubplan*)subPlan)->subplanType = SUBPLAN_TYPE_MERGE; + + code = nodesListMakeStrictAppend(&((SLogicSubplan*)subPlan)->pChildren, + (SNode*)splCreateScanSubplan(pCxt, pPartWindow, SPLIT_FLAG_STABLE_SPLIT)); + if (TSDB_CODE_SUCCESS == code) { + code = nodesListMakeStrictAppend(&pInfo->pSubplan->pChildren, subPlan); + } + } + pInfo->pSubplan->subplanType = SUBPLAN_TYPE_MERGE; ++(pCxt->groupId); return code; diff --git a/source/libs/planner/src/planValidator.c b/source/libs/planner/src/planValidator.c index a5d729ab84..4fcd064e56 100755 --- a/source/libs/planner/src/planValidator.c +++ b/source/libs/planner/src/planValidator.c @@ -95,6 +95,7 @@ int32_t doValidatePhysiNode(SValidatePlanContext* pCxt, SNode* pNode) { case QUERY_NODE_PHYSICAL_PLAN_STREAM_INTERVAL: case QUERY_NODE_PHYSICAL_PLAN_STREAM_FINAL_INTERVAL: case QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_INTERVAL: + case QUERY_NODE_PHYSICAL_PLAN_STREAM_MID_INTERVAL: case QUERY_NODE_PHYSICAL_PLAN_FILL: case QUERY_NODE_PHYSICAL_PLAN_STREAM_FILL: case QUERY_NODE_PHYSICAL_PLAN_MERGE_SESSION: From b36665c9d619b237d78f62e0294a8aea3ad4108e Mon Sep 17 00:00:00 2001 From: wangmm0220 Date: Fri, 8 Dec 2023 15:48:44 +0800 Subject: [PATCH 07/51] opti:build task logic in stream --- source/dnode/mnode/impl/src/mndScheduler.c | 481 ++++++++++----------- 1 file changed, 232 insertions(+), 249 deletions(-) diff --git a/source/dnode/mnode/impl/src/mndScheduler.c b/source/dnode/mnode/impl/src/mndScheduler.c index 3ef4c9a4d2..80bf084f1a 100644 --- a/source/dnode/mnode/impl/src/mndScheduler.c +++ b/source/dnode/mnode/impl/src/mndScheduler.c @@ -27,9 +27,6 @@ #define SINK_NODE_LEVEL (0) extern bool tsDeployOnSnode; -static int32_t doAddSinkTask(SStreamObj* pStream, SArray* pTaskList, SMnode* pMnode, int32_t vgId, SVgObj* pVgroup, - SEpSet* pEpset, bool isFillhistory); - int32_t mndConvertRsmaTask(char** pDst, int32_t* pDstLen, const char* ast, int64_t uid, int8_t triggerType, int64_t watermark, int64_t deleteMark) { SNode* pAst = NULL; @@ -157,12 +154,7 @@ int32_t mndAssignStreamTaskToVgroup(SMnode* pMnode, SStreamTask* pTask, SSubplan 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; - } - - return 0; + return qSubPlanToString(plan, &pTask->exec.qmsg, &msgLen); } SSnodeObj* mndSchedFetchOneSnode(SMnode* pMnode) { @@ -184,11 +176,7 @@ int32_t mndAssignStreamTaskToSnode(SMnode* pMnode, SStreamTask* pTask, SSubplan* 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; - return -1; - } - return 0; + return qSubPlanToString(plan, &pTask->exec.qmsg, &msgLen); } // todo random choose a node to do compute @@ -208,8 +196,26 @@ SVgObj* mndSchedFetchOneVg(SMnode* pMnode, int64_t dbUid) { return pVgroup; } +static int32_t doAddSinkTask(SStreamObj* pStream, SMnode* pMnode, int32_t vgId, SVgObj* pVgroup, + SEpSet* pEpset, bool isFillhistory) { + int64_t uid = (isFillhistory) ? pStream->hTaskUid : pStream->uid; + SArray** pTaskList = (isFillhistory) ? taosArrayGetLast(pStream->pHTasksList) : taosArrayGetLast(pStream->tasks); + + SStreamTask* pTask = tNewStreamTask(uid, TASK_LEVEL__SINK, isFillhistory, 0, *pTaskList, pStream->conf.fillHistory); + if (pTask == NULL) { + terrno = TSDB_CODE_OUT_OF_MEMORY; + return terrno; + } + + epsetAssign(&(pTask)->info.mnodeEpset, pEpset); + + pTask->info.nodeId = vgId; + pTask->info.epSet = mndGetVgroupEpset(pMnode, pVgroup); + return mndSetSinkTaskInfo(pStream, pTask); +} + // create sink node for each vgroup. -int32_t doAddShuffleSinkTask(SMnode* pMnode, SArray* pTaskList, SStreamObj* pStream, SEpSet* pEpset, bool fillHistory) { +static int32_t doAddShuffleSinkTask(SMnode* pMnode, SStreamObj* pStream, SEpSet* pEpset) { SSdb* pSdb = pMnode->pSdb; void* pIter = NULL; @@ -225,37 +231,34 @@ int32_t doAddShuffleSinkTask(SMnode* pMnode, SArray* pTaskList, SStreamObj* pStr continue; } - doAddSinkTask(pStream, pTaskList, pMnode, pVgroup->vgId, pVgroup, pEpset, fillHistory); + int32_t code = doAddSinkTask(pStream, pMnode, pVgroup->vgId, pVgroup, pEpset, false); + if(code != 0){ + sdbRelease(pSdb, pVgroup); + return code; + } + if(pStream->conf.fillHistory){ + code = doAddSinkTask(pStream, pMnode, pVgroup->vgId, pVgroup, pEpset, true); + if(code != 0){ + sdbRelease(pSdb, pVgroup); + return code; + } + } sdbRelease(pSdb, pVgroup); } - return 0; + return TDB_CODE_SUCCESS; } -int32_t doAddSinkTask(SStreamObj* pStream, SArray* pTaskList, SMnode* pMnode, int32_t vgId, SVgObj* pVgroup, - SEpSet* pEpset, bool isFillhistory) { - int64_t uid = (isFillhistory)? pStream->hTaskUid:pStream->uid; - SStreamTask* pTask = tNewStreamTask(uid, TASK_LEVEL__SINK, isFillhistory, 0, pTaskList, pStream->conf.fillHistory); +static SStreamTask* buildSourceTask(SStreamObj* pStream, SEpSet* pEpset, + int64_t firstWindowSkey, bool isFillhistory) { + uint64_t uid = (isFillhistory) ? pStream->hTaskUid : pStream->uid; + SArray** pTaskList = (isFillhistory) ? taosArrayGetLast(pStream->pHTasksList) : taosArrayGetLast(pStream->tasks); + + SStreamTask* pTask = tNewStreamTask(uid, TASK_LEVEL__SOURCE, + isFillhistory, pStream->conf.triggerParam, + *pTaskList, pStream->conf.fillHistory); if (pTask == NULL) { - terrno = TSDB_CODE_OUT_OF_MEMORY; - return -1; - } - - epsetAssign(&(pTask)->info.mnodeEpset, pEpset); - - pTask->info.nodeId = vgId; - pTask->info.epSet = mndGetVgroupEpset(pMnode, pVgroup); - mndSetSinkTaskInfo(pStream, pTask); - return 0; -} - -static int32_t addSourceTask(SMnode* pMnode, SVgObj* pVgroup, SArray* pTaskList, SArray* pSinkTaskList, - SStreamObj* pStream, SSubplan* plan, uint64_t uid, SEpSet* pEpset, bool fillHistory, - bool hasExtraSink, int64_t firstWindowSkey, bool hasFillHistory) { - SStreamTask* pTask = - tNewStreamTask(uid, TASK_LEVEL__SOURCE, fillHistory, pStream->conf.triggerParam, pTaskList, hasFillHistory); - if (pTask == NULL) { - return terrno; + return NULL; } epsetAssign(&pTask->info.mnodeEpset, pEpset); @@ -265,23 +268,7 @@ static int32_t addSourceTask(SMnode* pMnode, SVgObj* pVgroup, SArray* pTaskList, 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) { - mndAddDispatcherForInternalTask(pMnode, pStream, pSinkTaskList, pTask); - } else { - mndSetSinkTaskInfo(pStream, pTask); - } - - if (mndAssignStreamTaskToVgroup(pMnode, pTask, plan, pVgroup) < 0) { - return terrno; - } - - for(int32_t i = 0; i < taosArrayGetSize(pSinkTaskList); ++i) { - SStreamTask* pSinkTask = taosArrayGetP(pSinkTaskList, i); - streamTaskSetUpstreamInfo(pSinkTask, pTask); - } - - return TSDB_CODE_SUCCESS; + return pTask; } static SArray* addNewTaskList(SArray* pTasksList) { @@ -307,30 +294,64 @@ static void setHTasksId(SArray* pTaskList, const SArray* pHTaskList) { } } -static int32_t addSourceTasksForOneLevelStream(SMnode* pMnode, const SQueryPlan* pPlan, SStreamObj* pStream, - SEpSet* pEpset, 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); +static int32_t doAddSourceTask(SMnode* pMnode, SSubplan* plan, SStreamObj* pStream, + SEpSet* pEpset, int64_t nextWindowSkey, + SVgObj* pVgroup, bool isFillhistory ){ + // new stream task + SStreamTask* pTask = buildSourceTask(pStream, pEpset, nextWindowSkey, isFillhistory); + if(pTask == NULL){ + terrno = TSDB_CODE_OUT_OF_MEMORY; + return terrno; } + int32_t code = mndAssignStreamTaskToVgroup(pMnode, pTask, plan, pVgroup); + if(code != 0){ + terrno = code; + return terrno; + } + return TDB_CODE_SUCCESS; +} - SNodeListNode* inner = (SNodeListNode*)nodesListGetNode(pPlan->pSubplans, 0); +static SSubplan* getScanSubPlan(const SQueryPlan* pPlan, int planIndex){ + SNodeListNode* inner = (SNodeListNode*)nodesListGetNode(pPlan->pSubplans, planIndex); if (LIST_LENGTH(inner->pNodeList) != 1) { terrno = TSDB_CODE_QRY_INVALID_INPUT; - return -1; + return NULL; } SSubplan* plan = (SSubplan*)nodesListGetNode(inner->pNodeList, 0); if (plan->subplanType != SUBPLAN_TYPE_SCAN) { terrno = TSDB_CODE_QRY_INVALID_INPUT; - return -1; + return NULL; + } + return plan; +} + +static SSubplan* getFinalAggSubPlan(const SQueryPlan* pPlan){ + SNodeListNode* inner = (SNodeListNode*)nodesListGetNode(pPlan->pSubplans, 0); + if (LIST_LENGTH(inner->pNodeList) != 1) { + terrno = TSDB_CODE_QRY_INVALID_INPUT; + return NULL; + } + + SSubplan* plan = (SSubplan*)nodesListGetNode(inner->pNodeList, 0); + if (plan->subplanType != SUBPLAN_TYPE_MERGE) { + terrno = TSDB_CODE_QRY_INVALID_INPUT; + return NULL; + } + return plan; +} + +static int32_t addSourceTask(SMnode* pMnode, SSubplan* plan, SStreamObj* pStream, + SEpSet* pEpset, int64_t nextWindowSkey) { + // 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); } void* pIter = NULL; + SSdb* pSdb = pMnode->pSdb; while (1) { SVgObj* pVgroup; pIter = sdbFetch(pSdb, SDB_VGROUP, pIter, (void**)&pVgroup); @@ -343,25 +364,21 @@ static int32_t addSourceTasksForOneLevelStream(SMnode* pMnode, const SQueryPlan* continue; } - // new stream task - SArray** pSinkTaskList = taosArrayGet(pStream->tasks, SINK_NODE_LEVEL); - int32_t code = addSourceTask(pMnode, pVgroup, pTaskList, *pSinkTaskList, pStream, plan, pStream->uid, pEpset, - false, hasExtraSink, nextWindowSkey, pStream->conf.fillHistory); - if (code != TSDB_CODE_SUCCESS) { + int code = doAddSourceTask(pMnode, plan, pStream, pEpset, nextWindowSkey, pVgroup, false); + if(code != 0){ sdbRelease(pSdb, pVgroup); - return -1; + return code; } if (pStream->conf.fillHistory) { - SArray** pHSinkTaskList = taosArrayGet(pStream->pHTasksList, SINK_NODE_LEVEL); - code = addSourceTask(pMnode, pVgroup, pHTaskList, *pHSinkTaskList, pStream, plan, pStream->hTaskUid, - pEpset, true, hasExtraSink, nextWindowSkey, true); + code = doAddSourceTask(pMnode, plan, pStream, pEpset, nextWindowSkey, pVgroup, true); + if(code != 0){ + sdbRelease(pSdb, pVgroup); + return code; + } } sdbRelease(pSdb, pVgroup); - if (code != TSDB_CODE_SUCCESS) { - return -1; - } } if (pStream->conf.fillHistory) { @@ -371,76 +388,31 @@ static int32_t addSourceTasksForOneLevelStream(SMnode* pMnode, const SQueryPlan* return TSDB_CODE_SUCCESS; } -static int32_t doAddSourceTask(SArray* pTaskList, bool isFillhistory, int64_t uid, SStreamTask* pDownstreamTask, - SMnode* pMnode, SSubplan* pPlan, SVgObj* pVgroup, SEpSet* pEpset, - int64_t nextWindowSkey, bool hasFillHistory) { - SStreamTask* pTask = tNewStreamTask(uid, TASK_LEVEL__SOURCE, isFillhistory, 0, pTaskList, hasFillHistory); - if (pTask == NULL) { +static SStreamTask* buildAggTask(SStreamObj* pStream, SEpSet* pEpset, bool isFillhistory) { + uint64_t uid = (isFillhistory) ? pStream->hTaskUid : pStream->uid; + SArray** pTaskList = (isFillhistory) ? taosArrayGetLast(pStream->pHTasksList) : taosArrayGetLast(pStream->tasks); + + SStreamTask* pAggTask = tNewStreamTask(uid, TASK_LEVEL__AGG, isFillhistory, pStream->conf.triggerParam, *pTaskList, pStream->conf.fillHistory); + if (pAggTask == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; - return -1; + return NULL; } - epsetAssign(&(pTask)->info.mnodeEpset, pEpset); - - // todo set the correct ts, which should be last key of queried table. - 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, - pWindow->skey, pWindow->ekey); - - // all the source tasks dispatch result to a single agg node. - streamTaskSetFixedDownstreamInfo(pTask, pDownstreamTask); - if (mndAssignStreamTaskToVgroup(pMnode, pTask, pPlan, pVgroup) < 0) { - return -1; - } - - return streamTaskSetUpstreamInfo(pDownstreamTask, pTask); + epsetAssign(&pAggTask->info.mnodeEpset, pEpset); + return pAggTask; } -static int32_t doAddAggTask(uint64_t uid, SArray* pTaskList, SArray* pSinkNodeList, SMnode* pMnode, SStreamObj* pStream, - SEpSet* pEpset, bool fillHistory, SStreamTask** pAggTask, bool hasFillhistory) { - *pAggTask = tNewStreamTask(uid, TASK_LEVEL__AGG, fillHistory, pStream->conf.triggerParam, pTaskList, hasFillhistory); - if (*pAggTask == NULL) { - terrno = TSDB_CODE_OUT_OF_MEMORY; - return -1; - } - - epsetAssign(&(*pAggTask)->info.mnodeEpset, pEpset); - - // dispatch - if (mndAddDispatcherForInternalTask(pMnode, pStream, pSinkNodeList, *pAggTask) < 0) { - return -1; - } - - return 0; -} - -static int32_t addAggTask(SStreamObj* pStream, SMnode* pMnode, SQueryPlan* pPlan, SEpSet* pEpset, - SStreamTask** pAggTask, SStreamTask** pHAggTask) { +static int32_t addAggTask(SStreamObj* pStream, SMnode* pMnode, SSubplan* plan, SEpSet* pEpset) { SArray* pAggTaskList = addNewTaskList(pStream->tasks); - SSdb* pSdb = pMnode->pSdb; - 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; - } - - *pAggTask = NULL; - SArray* pSinkNodeList = taosArrayGetP(pStream->tasks, SINK_NODE_LEVEL); - - int32_t code = doAddAggTask(pStream->uid, pAggTaskList, pSinkNodeList, pMnode, pStream, pEpset, false, pAggTask, - pStream->conf.fillHistory); - if (code != TSDB_CODE_SUCCESS) { - return -1; + SStreamTask* pTask = buildAggTask(pStream, pEpset, false); + if (pTask == NULL) { + return terrno; } SVgObj* pVgroup = NULL; SSnodeObj* pSnode = NULL; - + int32_t code = 0; if (tsDeployOnSnode) { pSnode = mndSchedFetchOneSnode(pMnode); if (pSnode == NULL) { @@ -451,135 +423,124 @@ static int32_t addAggTask(SStreamObj* pStream, SMnode* pMnode, SQueryPlan* pPlan } if (pSnode != NULL) { - code = mndAssignStreamTaskToSnode(pMnode, *pAggTask, plan, pSnode); + code = mndAssignStreamTaskToSnode(pMnode, pTask, plan, pSnode); } else { - code = mndAssignStreamTaskToVgroup(pMnode, *pAggTask, plan, pVgroup); + code = mndAssignStreamTaskToVgroup(pMnode, pTask, plan, pVgroup); + } + if(code != 0){ + terrno = code; + goto END; } if (pStream->conf.fillHistory) { SArray* pHAggTaskList = addNewTaskList(pStream->pHTasksList); - SArray* pHSinkNodeList = taosArrayGetP(pStream->pHTasksList, SINK_NODE_LEVEL); - *pHAggTask = NULL; - code = doAddAggTask(pStream->hTaskUid, pHAggTaskList, pHSinkNodeList, pMnode, pStream, pEpset, pStream->conf.fillHistory, - pHAggTask, pStream->conf.fillHistory); - if (code != TSDB_CODE_SUCCESS) { - if (pSnode != NULL) { - sdbRelease(pSdb, pSnode); - } else { - sdbRelease(pSdb, pVgroup); - } - return code; + pTask = buildAggTask(pStream, pEpset, true); + if (pTask == NULL) { + goto END; } if (pSnode != NULL) { - code = mndAssignStreamTaskToSnode(pMnode, *pHAggTask, plan, pSnode); + code = mndAssignStreamTaskToSnode(pMnode, pTask, plan, pSnode); } else { - code = mndAssignStreamTaskToVgroup(pMnode, *pHAggTask, plan, pVgroup); + code = mndAssignStreamTaskToVgroup(pMnode, pTask, plan, pVgroup); + } + if(code != 0){ + terrno = code; + goto END; } setHTasksId(pAggTaskList, pHAggTaskList); } +END: if (pSnode != NULL) { - sdbRelease(pSdb, pSnode); + sdbRelease(pMnode->pSdb, pSnode); } else { - sdbRelease(pSdb, pVgroup); + sdbRelease(pMnode->pSdb, pVgroup); } - return code; + return terrno; } -static int32_t addSourceTasksForMultiLevelStream(SMnode* pMnode, SQueryPlan* pPlan, SStreamObj* pStream, - SStreamTask* pDownstreamTask, SStreamTask* pHDownstreamTask, - SEpSet* pEpset, int64_t nextWindowSkey) { - SArray* pSourceTaskList = addNewTaskList(pStream->tasks); +static int32_t addSinkTask(SMnode* pMnode, SStreamObj* pStream, SEpSet* pEpset){ + SArray* pSinkTaskList = addNewTaskList(pStream->tasks); - SArray* pHSourceTaskList = NULL; + SArray* pHSinkTaskList = NULL; if (pStream->conf.fillHistory) { - pHSourceTaskList = addNewTaskList(pStream->pHTasksList); + pHSinkTaskList = 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; + int32_t code = 0; + if (pStream->fixedSinkVgId == 0) { + code = doAddShuffleSinkTask(pMnode, pStream, pEpset); + if (code != 0) { + return code; } - - if (!mndVgroupInDb(pVgroup, pStream->sourceDbUid)) { - sdbRelease(pSdb, pVgroup); - continue; + } else { + code = doAddSinkTask(pStream, pMnode, pStream->fixedSinkVgId, &pStream->fixedSinkVg, pEpset, false); + if (code != 0) { + return code; } - - int32_t code = doAddSourceTask(pSourceTaskList, false, pStream->uid, pDownstreamTask, pMnode, plan, pVgroup, pEpset, - nextWindowSkey, pStream->conf.fillHistory); - if (code != TSDB_CODE_SUCCESS) { - sdbRelease(pSdb, pVgroup); - terrno = code; - return -1; - } - - if (pStream->conf.fillHistory) { - code = doAddSourceTask(pHSourceTaskList, true, pStream->hTaskUid, pHDownstreamTask, pMnode, plan, pVgroup, pEpset, - nextWindowSkey, pStream->conf.fillHistory); - if (code != TSDB_CODE_SUCCESS) { - sdbRelease(pSdb, pVgroup); + if(pStream->conf.fillHistory){ + code = doAddSinkTask(pStream, pMnode, pStream->fixedSinkVgId, &pStream->fixedSinkVg, pEpset, true); + if (code != 0) { return code; } } - - sdbRelease(pSdb, pVgroup); } if (pStream->conf.fillHistory) { - setHTasksId(pSourceTaskList, pHSourceTaskList); + setHTasksId(pSinkTaskList, pHSinkTaskList); } - - return TSDB_CODE_SUCCESS; + return TDB_CODE_SUCCESS; } -static int32_t addSinkTasks(SArray* pTasksList, SMnode* pMnode, SStreamObj* pStream, SArray** pCreatedTaskList, - SEpSet* pEpset, bool fillHistory) { - SArray* pSinkTaskList = addNewTaskList(pTasksList); - if (pStream->fixedSinkVgId == 0) { - if (doAddShuffleSinkTask(pMnode, pSinkTaskList, pStream, pEpset, fillHistory) < 0) { - // TODO free - return -1; - } - } else { - if (doAddSinkTask(pStream, pSinkTaskList, pMnode, pStream->fixedSinkVgId, &pStream->fixedSinkVg, pEpset, - fillHistory) < 0) { - // TODO free - return -1; +static void bindTaskToSinkTask(SStreamObj* pStream, SMnode* pMnode, SArray* pSinkTaskList, SStreamTask* task){ + mndAddDispatcherForInternalTask(pMnode, pStream, pSinkTaskList, task); + for(int32_t k = 0; k < taosArrayGetSize(pSinkTaskList); ++k) { + SStreamTask* pSinkTask = taosArrayGetP(pSinkTaskList, k); + streamTaskSetUpstreamInfo(pSinkTask, task); + } +} + +static void bindAggSink(SStreamObj* pStream, SMnode* pMnode, SArray* tasks) { + SArray* pSinkTaskList = taosArrayGetP(tasks, SINK_NODE_LEVEL); + SArray* pAggTaskList = taosArrayGetP(tasks, SINK_NODE_LEVEL + 1); + + for(int i = 0; i < taosArrayGetSize(pAggTaskList); i++){ + SStreamTask* pAggTask = taosArrayGetP(pAggTaskList, i); + bindTaskToSinkTask(pStream, pMnode, pSinkTaskList, pAggTask); + } +} + +static void bindSourceSink(SStreamObj* pStream, SMnode* pMnode, SArray* tasks, bool hasExtraSink) { + SArray* pSinkTaskList = taosArrayGetP(tasks, SINK_NODE_LEVEL); + SArray* pSourceTaskList = taosArrayGetP(tasks, SINK_NODE_LEVEL + 1); + + for(int i = 0; i < taosArrayGetSize(pSourceTaskList); i++){ + SStreamTask* pSourceTask = taosArrayGetP(pSourceTaskList, i); + if (hasExtraSink) { + bindTaskToSinkTask(pStream, pMnode, pSinkTaskList, pSourceTask); + } else { + mndSetSinkTaskInfo(pStream, pSourceTask); } } - - *pCreatedTaskList = pSinkTaskList; - return TSDB_CODE_SUCCESS; } -static void setSinkTaskUpstreamInfo(SArray* pTasksList, const SStreamTask* pUpstreamTask) { - if (taosArrayGetSize(pTasksList) < SINK_NODE_LEVEL || pUpstreamTask == NULL) { - return; - } +static void bindSourceAgg(SArray* tasks) { + SArray* pAggTaskList = taosArrayGetP(tasks, SINK_NODE_LEVEL + 1); + SArray* pSourceTaskList = taosArrayGetP(tasks, SINK_NODE_LEVEL + 2); - SArray* pSinkTaskList = taosArrayGetP(pTasksList, SINK_NODE_LEVEL); - for(int32_t i = 0; i < taosArrayGetSize(pSinkTaskList); ++i) { - SStreamTask* pSinkTask = taosArrayGetP(pSinkTaskList, i); - streamTaskSetUpstreamInfo(pSinkTask, pUpstreamTask); + SStreamTask* pAggTask = taosArrayGetP(pAggTaskList, 0); + for(int i = 0; i < taosArrayGetSize(pSourceTaskList); i++){ + SStreamTask* pSourceTask = taosArrayGetP(pSourceTaskList, i); + streamTaskSetFixedDownstreamInfo(pSourceTask, pAggTask); + streamTaskSetUpstreamInfo(pAggTask, pSourceTask); } } +//#define AGGNUM 2 static int32_t doScheduleStream(SStreamObj* pStream, SMnode* pMnode, SQueryPlan* pPlan, int64_t nextWindowSkey, SEpSet* pEpset) { SSdb* pSdb = pMnode->pSdb; int32_t numOfPlanLevel = LIST_LENGTH(pPlan->pSubplans); @@ -601,45 +562,67 @@ static int32_t doScheduleStream(SStreamObj* pStream, SMnode* pMnode, SQueryPlan* if (numOfPlanLevel == 2 || externalTargetDB || multiTarget || pStream->fixedSinkVgId) { // add extra sink hasExtraSink = true; - - SArray* pSinkTaskList = NULL; - int32_t code = addSinkTasks(pStream->tasks, pMnode, pStream, &pSinkTaskList, pEpset, 0); + int32_t code = addSinkTask(pMnode, pStream, pEpset); 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, pEpset, 1); - if (code != TSDB_CODE_SUCCESS) { - return code; - } - - setHTasksId(pSinkTaskList, pHSinkTaskList); - } } pStream->totalLevel = numOfPlanLevel + hasExtraSink; +// if(numOfPlanLevel == 3){ +// pDbObj = mndAcquireDb(pMnode, pStream->sourceDb); +// if (pDbObj == NULL) { +// terrno = TSDB_CODE_QRY_INVALID_INPUT; +// return -1; +// } +// +// if(pDbObj->cfg.numOfVgroups >= AGGNUM){ +// +// } +// sdbRelease(pSdb, pDbObj); +// }else if (numOfPlanLevel > 1) { - SStreamTask* pAggTask = NULL; - SStreamTask* pHAggTask = NULL; + SSubplan *plan = getFinalAggSubPlan(pPlan); + if(plan == NULL){ + return terrno; + } + int32_t code = addAggTask(pStream, pMnode, plan, pEpset); + if (code != TSDB_CODE_SUCCESS) { + return code; + } + bindAggSink(pStream, pMnode, pStream->tasks); + if (pStream->conf.fillHistory) { + bindAggSink(pStream, pMnode, pStream->pHTasksList); + } - int32_t code = addAggTask(pStream, pMnode, pPlan, pEpset, &pAggTask, &pHAggTask); + plan = getScanSubPlan(pPlan, 1); + if(plan == NULL){ + return terrno; + } + code = addSourceTask(pMnode, plan, pStream, pEpset, nextWindowSkey); if (code != TSDB_CODE_SUCCESS) { return code; } - setSinkTaskUpstreamInfo(pStream->tasks, pAggTask); - if (pHAggTask != NULL) { - setSinkTaskUpstreamInfo(pStream->pHTasksList, pHAggTask); + bindSourceAgg(pStream->tasks); + if (pStream->conf.fillHistory) { + bindSourceAgg(pStream->pHTasksList); + } + } else if (numOfPlanLevel == 1) { + SSubplan *plan = getScanSubPlan(pPlan, 0); + if(plan == NULL){ + return terrno; + } + int32_t code = addSourceTask(pMnode, plan, pStream, pEpset, nextWindowSkey); + if (code != TSDB_CODE_SUCCESS) { + return code; } - // source level - return addSourceTasksForMultiLevelStream(pMnode, pPlan, pStream, pAggTask, pHAggTask, pEpset, nextWindowSkey); - } else if (numOfPlanLevel == 1) { - return addSourceTasksForOneLevelStream(pMnode, pPlan, pStream, pEpset, hasExtraSink, nextWindowSkey); + bindSourceSink(pStream, pMnode, pStream->tasks, hasExtraSink); + if (pStream->conf.fillHistory) { + bindSourceSink(pStream, pMnode, pStream->pHTasksList, hasExtraSink); + } } return 0; From acfcfdc8b612c2e042f4e9c1c0f3ba5daff8c681 Mon Sep 17 00:00:00 2001 From: wangmm0220 Date: Fri, 8 Dec 2023 15:51:01 +0800 Subject: [PATCH 08/51] opti:build task logic in stream --- source/dnode/mnode/impl/src/mndScheduler.c | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/source/dnode/mnode/impl/src/mndScheduler.c b/source/dnode/mnode/impl/src/mndScheduler.c index 80bf084f1a..fef18574e6 100644 --- a/source/dnode/mnode/impl/src/mndScheduler.c +++ b/source/dnode/mnode/impl/src/mndScheduler.c @@ -559,7 +559,7 @@ static int32_t doScheduleStream(SStreamObj* pStream, SMnode* pMnode, SQueryPlan* pStream->tasks = taosArrayInit(numOfPlanLevel + 1, POINTER_BYTES); pStream->pHTasksList = taosArrayInit(numOfPlanLevel + 1, POINTER_BYTES); - if (numOfPlanLevel == 2 || externalTargetDB || multiTarget || pStream->fixedSinkVgId) { + if (numOfPlanLevel > 1 || externalTargetDB || multiTarget || pStream->fixedSinkVgId) { // add extra sink hasExtraSink = true; int32_t code = addSinkTask(pMnode, pStream, pEpset); @@ -596,7 +596,7 @@ static int32_t doScheduleStream(SStreamObj* pStream, SMnode* pMnode, SQueryPlan* bindAggSink(pStream, pMnode, pStream->pHTasksList); } - plan = getScanSubPlan(pPlan, 1); + plan = getScanSubPlan(pPlan, numOfPlanLevel - 1); if(plan == NULL){ return terrno; } From c3cbc1fbcec2b7cf65260f7d755bbc5ff9799b68 Mon Sep 17 00:00:00 2001 From: wangmm0220 Date: Fri, 8 Dec 2023 16:52:13 +0800 Subject: [PATCH 09/51] fix:compile error --- .../mnode/impl/test/stream/streamPlanTest.cpp | 19 +++++++++---------- 1 file changed, 9 insertions(+), 10 deletions(-) diff --git a/source/dnode/mnode/impl/test/stream/streamPlanTest.cpp b/source/dnode/mnode/impl/test/stream/streamPlanTest.cpp index 9b3587cb3a..d769ef674b 100644 --- a/source/dnode/mnode/impl/test/stream/streamPlanTest.cpp +++ b/source/dnode/mnode/impl/test/stream/streamPlanTest.cpp @@ -47,16 +47,15 @@ TEST(testCase, plan_Test) { ASSERT(0); } - SPlanContext cxt = { - .pAstRoot = pAst, - .topicQuery = false, - .streamQuery = true, - .triggerType = STREAM_TRIGGER_WINDOW_CLOSE, - .watermark = 1, - .igExpired = 1, - .deleteMark = 1, - .igCheckUpdate = 1, - }; + SPlanContext cxt = { 0 }; + cxt.pAstRoot = pAst; + cxt.topicQuery = false; + cxt.streamQuery = true; + cxt.triggerType = STREAM_TRIGGER_WINDOW_CLOSE; + cxt.watermark = 1; + cxt.igExpired = 1; + cxt.deleteMark = 1; + cxt.igCheckUpdate = 1; // using ast and param to build physical plan if (qCreateQueryPlan(&cxt, &pPlan, NULL) < 0) { From 2a506b3e5e7b96bba67ff22324729f8b0ff6c815 Mon Sep 17 00:00:00 2001 From: wangmm0220 Date: Fri, 8 Dec 2023 18:48:34 +0800 Subject: [PATCH 10/51] feat:add multi level agg operator for stream task --- source/dnode/mnode/impl/src/mndScheduler.c | 180 ++++++++++++--------- 1 file changed, 102 insertions(+), 78 deletions(-) diff --git a/source/dnode/mnode/impl/src/mndScheduler.c b/source/dnode/mnode/impl/src/mndScheduler.c index fef18574e6..8213a21d7f 100644 --- a/source/dnode/mnode/impl/src/mndScheduler.c +++ b/source/dnode/mnode/impl/src/mndScheduler.c @@ -24,6 +24,9 @@ #include "tname.h" #include "tuuid.h" +#define SINK_NODE_LEVEL (0) +#define SOURCE_NODE_LEVEL (0) +#define SINK_NODE_LEVEL (0) #define SINK_NODE_LEVEL (0) extern bool tsDeployOnSnode; @@ -311,8 +314,9 @@ static int32_t doAddSourceTask(SMnode* pMnode, SSubplan* plan, SStreamObj* pStre return TDB_CODE_SUCCESS; } -static SSubplan* getScanSubPlan(const SQueryPlan* pPlan, int planIndex){ - SNodeListNode* inner = (SNodeListNode*)nodesListGetNode(pPlan->pSubplans, planIndex); +static SSubplan* getScanSubPlan(const SQueryPlan* pPlan){ + int32_t numOfPlanLevel = LIST_LENGTH(pPlan->pSubplans); + SNodeListNode* inner = (SNodeListNode*)nodesListGetNode(pPlan->pSubplans, numOfPlanLevel - 1); if (LIST_LENGTH(inner->pNodeList) != 1) { terrno = TSDB_CODE_QRY_INVALID_INPUT; return NULL; @@ -326,8 +330,8 @@ static SSubplan* getScanSubPlan(const SQueryPlan* pPlan, int planIndex){ return plan; } -static SSubplan* getFinalAggSubPlan(const SQueryPlan* pPlan){ - SNodeListNode* inner = (SNodeListNode*)nodesListGetNode(pPlan->pSubplans, 0); +static SSubplan* getAggSubPlan(const SQueryPlan* pPlan, int index){ + SNodeListNode* inner = (SNodeListNode*)nodesListGetNode(pPlan->pSubplans, index); if (LIST_LENGTH(inner->pNodeList) != 1) { terrno = TSDB_CODE_QRY_INVALID_INPUT; return NULL; @@ -402,9 +406,7 @@ static SStreamTask* buildAggTask(SStreamObj* pStream, SEpSet* pEpset, bool isFil return pAggTask; } -static int32_t addAggTask(SStreamObj* pStream, SMnode* pMnode, SSubplan* plan, SEpSet* pEpset) { - SArray* pAggTaskList = addNewTaskList(pStream->tasks); - +static int32_t doAddAggTask(SStreamObj* pStream, SMnode* pMnode, SSubplan* plan, SEpSet* pEpset){ SStreamTask* pTask = buildAggTask(pStream, pEpset, false); if (pTask == NULL) { return terrno; @@ -428,15 +430,13 @@ static int32_t addAggTask(SStreamObj* pStream, SMnode* pMnode, SSubplan* plan, S code = mndAssignStreamTaskToVgroup(pMnode, pTask, plan, pVgroup); } if(code != 0){ - terrno = code; goto END; } if (pStream->conf.fillHistory) { - SArray* pHAggTaskList = addNewTaskList(pStream->pHTasksList); - pTask = buildAggTask(pStream, pEpset, true); if (pTask == NULL) { + code = terrno; goto END; } @@ -446,21 +446,29 @@ static int32_t addAggTask(SStreamObj* pStream, SMnode* pMnode, SSubplan* plan, S code = mndAssignStreamTaskToVgroup(pMnode, pTask, plan, pVgroup); } if(code != 0){ - terrno = code; goto END; } - setHTasksId(pAggTaskList, pHAggTaskList); + SArray** pAggTaskList = taosArrayGetLast(pStream->tasks); + SArray** pHAggTaskList = taosArrayGetLast(pStream->pHTasksList); + setHTasksId(*pAggTaskList, *pHAggTaskList); } -END: + END: if (pSnode != NULL) { sdbRelease(pMnode->pSdb, pSnode); } else { sdbRelease(pMnode->pSdb, pVgroup); } + return code; +} - return terrno; +static int32_t addAggTask(SStreamObj* pStream, SMnode* pMnode, SSubplan* plan, SEpSet* pEpset) { + addNewTaskList(pStream->tasks); + if (pStream->conf.fillHistory) { + addNewTaskList(pStream->pHTasksList); + } + return doAddAggTask(pStream, pMnode, plan, pEpset); } static int32_t addSinkTask(SMnode* pMnode, SStreamObj* pStream, SEpSet* pEpset){ @@ -498,7 +506,7 @@ static int32_t addSinkTask(SMnode* pMnode, SStreamObj* pStream, SEpSet* pEpset){ static void bindTaskToSinkTask(SStreamObj* pStream, SMnode* pMnode, SArray* pSinkTaskList, SStreamTask* task){ mndAddDispatcherForInternalTask(pMnode, pStream, pSinkTaskList, task); - for(int32_t k = 0; k < taosArrayGetSize(pSinkTaskList); ++k) { + for(int32_t k = 0; k < taosArrayGetSize(pSinkTaskList); k++) { SStreamTask* pSinkTask = taosArrayGetP(pSinkTaskList, k); streamTaskSetUpstreamInfo(pSinkTask, task); } @@ -506,17 +514,17 @@ static void bindTaskToSinkTask(SStreamObj* pStream, SMnode* pMnode, SArray* pSin static void bindAggSink(SStreamObj* pStream, SMnode* pMnode, SArray* tasks) { SArray* pSinkTaskList = taosArrayGetP(tasks, SINK_NODE_LEVEL); - SArray* pAggTaskList = taosArrayGetP(tasks, SINK_NODE_LEVEL + 1); + SArray** pAggTaskList = taosArrayGetLast(tasks); - for(int i = 0; i < taosArrayGetSize(pAggTaskList); i++){ - SStreamTask* pAggTask = taosArrayGetP(pAggTaskList, i); + for(int i = 0; i < taosArrayGetSize(*pAggTaskList); i++){ + SStreamTask* pAggTask = taosArrayGetP(*pAggTaskList, i); bindTaskToSinkTask(pStream, pMnode, pSinkTaskList, pAggTask); } } static void bindSourceSink(SStreamObj* pStream, SMnode* pMnode, SArray* tasks, bool hasExtraSink) { SArray* pSinkTaskList = taosArrayGetP(tasks, SINK_NODE_LEVEL); - SArray* pSourceTaskList = taosArrayGetP(tasks, SINK_NODE_LEVEL + 1); + SArray* pSourceTaskList = taosArrayGetP(tasks, hasExtraSink ? SINK_NODE_LEVEL + 1 : SINK_NODE_LEVEL); for(int i = 0; i < taosArrayGetSize(pSourceTaskList); i++){ SStreamTask* pSourceTask = taosArrayGetP(pSourceTaskList, i); @@ -528,23 +536,27 @@ static void bindSourceSink(SStreamObj* pStream, SMnode* pMnode, SArray* tasks, b } } -static void bindSourceAgg(SArray* tasks) { - SArray* pAggTaskList = taosArrayGetP(tasks, SINK_NODE_LEVEL + 1); - SArray* pSourceTaskList = taosArrayGetP(tasks, SINK_NODE_LEVEL + 2); +static void bindTwoLevel(SArray* tasks, int32_t begin, int32_t end) { + size_t size = taosArrayGetSize(tasks); + ASSERT(size >= 2); + SArray* pDownTaskList = taosArrayGetP(tasks, size - 1); + SArray* pUpTaskList = taosArrayGetP(tasks, size - 2); - SStreamTask* pAggTask = taosArrayGetP(pAggTaskList, 0); - for(int i = 0; i < taosArrayGetSize(pSourceTaskList); i++){ - SStreamTask* pSourceTask = taosArrayGetP(pSourceTaskList, i); - streamTaskSetFixedDownstreamInfo(pSourceTask, pAggTask); - streamTaskSetUpstreamInfo(pAggTask, pSourceTask); + SStreamTask** pDownTask = taosArrayGetLast(pDownTaskList); + for(int i = begin; i < end; i++){ + SStreamTask* pUpTask = taosArrayGetP(pUpTaskList, i); + if(pUpTask == NULL) { // out of range + break; + } + streamTaskSetFixedDownstreamInfo(pUpTask, *pDownTask); + streamTaskSetUpstreamInfo(*pDownTask, pUpTask); } } -//#define AGGNUM 2 +#define AGGNUM 2 static int32_t doScheduleStream(SStreamObj* pStream, SMnode* pMnode, SQueryPlan* pPlan, int64_t nextWindowSkey, SEpSet* pEpset) { SSdb* pSdb = pMnode->pSdb; int32_t numOfPlanLevel = LIST_LENGTH(pPlan->pSubplans); - bool hasExtraSink = false; bool externalTargetDB = strcmp(pStream->sourceDb, pStream->targetDb) != 0; SDbObj* pDbObj = mndAcquireDb(pMnode, pStream->targetDb); @@ -570,62 +582,74 @@ static int32_t doScheduleStream(SStreamObj* pStream, SMnode* pMnode, SQueryPlan* pStream->totalLevel = numOfPlanLevel + hasExtraSink; -// if(numOfPlanLevel == 3){ -// pDbObj = mndAcquireDb(pMnode, pStream->sourceDb); -// if (pDbObj == NULL) { -// terrno = TSDB_CODE_QRY_INVALID_INPUT; -// return -1; -// } -// -// if(pDbObj->cfg.numOfVgroups >= AGGNUM){ -// -// } -// sdbRelease(pSdb, pDbObj); -// }else - if (numOfPlanLevel > 1) { - SSubplan *plan = getFinalAggSubPlan(pPlan); - if(plan == NULL){ - return terrno; - } - int32_t code = addAggTask(pStream, pMnode, plan, pEpset); - if (code != TSDB_CODE_SUCCESS) { - return code; - } - bindAggSink(pStream, pMnode, pStream->tasks); - if (pStream->conf.fillHistory) { - bindAggSink(pStream, pMnode, pStream->pHTasksList); - } - - plan = getScanSubPlan(pPlan, numOfPlanLevel - 1); - if(plan == NULL){ - return terrno; - } - code = addSourceTask(pMnode, plan, pStream, pEpset, nextWindowSkey); - if (code != TSDB_CODE_SUCCESS) { - return code; - } - - bindSourceAgg(pStream->tasks); - if (pStream->conf.fillHistory) { - bindSourceAgg(pStream->pHTasksList); - } - } else if (numOfPlanLevel == 1) { - SSubplan *plan = getScanSubPlan(pPlan, 0); - if(plan == NULL){ - return terrno; - } - int32_t code = addSourceTask(pMnode, plan, pStream, pEpset, nextWindowSkey); - if (code != TSDB_CODE_SUCCESS) { - return code; - } + SSubplan* plan = getScanSubPlan(pPlan); // source plan + if (plan == NULL) { + return terrno; + } + int32_t code = addSourceTask(pMnode, plan, pStream, pEpset, nextWindowSkey); + if (code != TSDB_CODE_SUCCESS) { + return code; + } + if (numOfPlanLevel == 1) { bindSourceSink(pStream, pMnode, pStream->tasks, hasExtraSink); if (pStream->conf.fillHistory) { bindSourceSink(pStream, pMnode, pStream->pHTasksList, hasExtraSink); } + return TDB_CODE_SUCCESS; } - return 0; + if(numOfPlanLevel == 3){ + plan = getAggSubPlan(pPlan, 1); // middle agg plan + if (plan == NULL) { + return terrno; + } + do{ + SArray** list = taosArrayGetLast(pStream->tasks); + float size = (float)taosArrayGetSize(*list); + size_t cnt = (int)(size/AGGNUM + 0.5); + if(cnt <= 1) break; + + addNewTaskList(pStream->tasks); + if (pStream->conf.fillHistory) { + addNewTaskList(pStream->pHTasksList); + } + + for(int j = 0; j < cnt; j++){ + code = doAddAggTask(pStream, pMnode, plan, pEpset); + if (code != TSDB_CODE_SUCCESS) { + return code; + } + + bindTwoLevel(pStream->tasks, j*AGGNUM, (j+1)*AGGNUM); + if (pStream->conf.fillHistory) { + bindTwoLevel(pStream->pHTasksList, j*AGGNUM, (j+1)*AGGNUM); + } + } + }while(1); + } + + plan = getAggSubPlan(pPlan, 0); + if (plan == NULL) { + return terrno; + } + + SArray** list = taosArrayGetLast(pStream->tasks); + size_t size = taosArrayGetSize(*list); + code = addAggTask(pStream, pMnode, plan, pEpset); + if (code != TSDB_CODE_SUCCESS) { + return code; + } + bindTwoLevel(pStream->tasks, 0, size); + if (pStream->conf.fillHistory) { + bindTwoLevel(pStream->pHTasksList, 0, size); + } + + bindAggSink(pStream, pMnode, pStream->tasks); + if (pStream->conf.fillHistory) { + bindAggSink(pStream, pMnode, pStream->pHTasksList); + } + return TDB_CODE_SUCCESS; } int32_t mndScheduleStream(SMnode* pMnode, SStreamObj* pStream, int64_t nextWindowSkey) { From d6114f90de9679ad6b609e24ab193c91f58fb4c9 Mon Sep 17 00:00:00 2001 From: wangmm0220 Date: Fri, 8 Dec 2023 19:00:10 +0800 Subject: [PATCH 11/51] feat:add configure for agg cnt --- include/common/tglobal.h | 1 + source/common/src/tglobal.c | 5 +++++ source/dnode/mnode/impl/src/mndScheduler.c | 10 +++------- 3 files changed, 9 insertions(+), 7 deletions(-) diff --git a/include/common/tglobal.h b/include/common/tglobal.h index 33cfada338..31b98a0121 100644 --- a/include/common/tglobal.h +++ b/include/common/tglobal.h @@ -208,6 +208,7 @@ extern int32_t tsUptimeInterval; extern bool tsDisableStream; extern int64_t tsStreamBufferSize; +extern int tsStreamAggCnt; extern bool tsFilterScalarMode; extern int32_t tsMaxStreamBackendCache; extern int32_t tsPQSortMemThreshold; diff --git a/source/common/src/tglobal.c b/source/common/src/tglobal.c index 98d79c5295..4f2ae1b11f 100644 --- a/source/common/src/tglobal.c +++ b/source/common/src/tglobal.c @@ -262,6 +262,7 @@ bool tsDisableStream = false; int64_t tsStreamBufferSize = 128 * 1024 * 1024; bool tsFilterScalarMode = false; int tsResolveFQDNRetryTime = 100; // seconds +int tsStreamAggCnt = 10; char tsS3Endpoint[TSDB_FQDN_LEN] = ""; char tsS3AccessKey[TSDB_FQDN_LEN] = ""; @@ -737,6 +738,8 @@ static int32_t taosAddServerCfg(SConfig *pCfg) { if (cfgAddBool(pCfg, "disableStream", tsDisableStream, CFG_SCOPE_SERVER, CFG_DYN_ENT_SERVER) != 0) return -1; if (cfgAddInt64(pCfg, "streamBufferSize", tsStreamBufferSize, 0, INT64_MAX, CFG_SCOPE_SERVER, CFG_DYN_NONE) != 0) return -1; + if (cfgAddInt64(pCfg, "streamAggCnt", tsStreamAggCnt, 0, INT32_MAX, CFG_SCOPE_SERVER, CFG_DYN_NONE) != 0) + return -1; if (cfgAddInt32(pCfg, "checkpointInterval", tsStreamCheckpointInterval, 60, 1200, CFG_SCOPE_SERVER, CFG_DYN_ENT_SERVER) != 0) @@ -1192,6 +1195,8 @@ static int32_t taosSetServerCfg(SConfig *pCfg) { tsDisableStream = cfgGetItem(pCfg, "disableStream")->bval; tsStreamBufferSize = cfgGetItem(pCfg, "streamBufferSize")->i64; + tsStreamAggCnt = cfgGetItem(pCfg, "streamAggCnt")->i32; + tsStreamBufferSize = cfgGetItem(pCfg, "streamBufferSize")->i64; tsStreamCheckpointInterval = cfgGetItem(pCfg, "checkpointInterval")->i32; tsSinkDataRate = cfgGetItem(pCfg, "streamSinkDataRate")->fval; diff --git a/source/dnode/mnode/impl/src/mndScheduler.c b/source/dnode/mnode/impl/src/mndScheduler.c index 8213a21d7f..3de7a55820 100644 --- a/source/dnode/mnode/impl/src/mndScheduler.c +++ b/source/dnode/mnode/impl/src/mndScheduler.c @@ -24,9 +24,6 @@ #include "tname.h" #include "tuuid.h" -#define SINK_NODE_LEVEL (0) -#define SOURCE_NODE_LEVEL (0) -#define SINK_NODE_LEVEL (0) #define SINK_NODE_LEVEL (0) extern bool tsDeployOnSnode; @@ -553,7 +550,6 @@ static void bindTwoLevel(SArray* tasks, int32_t begin, int32_t end) { } } -#define AGGNUM 2 static int32_t doScheduleStream(SStreamObj* pStream, SMnode* pMnode, SQueryPlan* pPlan, int64_t nextWindowSkey, SEpSet* pEpset) { SSdb* pSdb = pMnode->pSdb; int32_t numOfPlanLevel = LIST_LENGTH(pPlan->pSubplans); @@ -607,7 +603,7 @@ static int32_t doScheduleStream(SStreamObj* pStream, SMnode* pMnode, SQueryPlan* do{ SArray** list = taosArrayGetLast(pStream->tasks); float size = (float)taosArrayGetSize(*list); - size_t cnt = (int)(size/AGGNUM + 0.5); + size_t cnt = (int)(size/tsStreamAggCnt + 0.5); if(cnt <= 1) break; addNewTaskList(pStream->tasks); @@ -621,9 +617,9 @@ static int32_t doScheduleStream(SStreamObj* pStream, SMnode* pMnode, SQueryPlan* return code; } - bindTwoLevel(pStream->tasks, j*AGGNUM, (j+1)*AGGNUM); + bindTwoLevel(pStream->tasks, j*tsStreamAggCnt, (j+1)*tsStreamAggCnt); if (pStream->conf.fillHistory) { - bindTwoLevel(pStream->pHTasksList, j*AGGNUM, (j+1)*AGGNUM); + bindTwoLevel(pStream->pHTasksList, j*tsStreamAggCnt, (j+1)*tsStreamAggCnt); } } }while(1); From 2e596c1376a942257289e9bb3ecd77ccff452b8c Mon Sep 17 00:00:00 2001 From: wangmm0220 Date: Mon, 11 Dec 2023 14:19:55 +0800 Subject: [PATCH 12/51] opti:build task logic --- source/dnode/mnode/impl/src/mndScheduler.c | 137 ++++++++++----------- 1 file changed, 67 insertions(+), 70 deletions(-) diff --git a/source/dnode/mnode/impl/src/mndScheduler.c b/source/dnode/mnode/impl/src/mndScheduler.c index 3de7a55820..ac01890b9b 100644 --- a/source/dnode/mnode/impl/src/mndScheduler.c +++ b/source/dnode/mnode/impl/src/mndScheduler.c @@ -86,6 +86,8 @@ END: int32_t mndSetSinkTaskInfo(SStreamObj* pStream, SStreamTask* pTask) { STaskOutputInfo* pInfo = &pTask->outputInfo; + mDebug("mndSetSinkTaskInfo to sma or table, taskId:%s", pTask->id.idStr); + if (pStream->smaId != 0) { pInfo->type = TASK_OUTPUT__SMA; pInfo->smaSink.smaId = pStream->smaId; @@ -196,7 +198,7 @@ SVgObj* mndSchedFetchOneVg(SMnode* pMnode, int64_t dbUid) { return pVgroup; } -static int32_t doAddSinkTask(SStreamObj* pStream, SMnode* pMnode, int32_t vgId, SVgObj* pVgroup, +static int32_t doAddSinkTask(SStreamObj* pStream, SMnode* pMnode, SVgObj* pVgroup, SEpSet* pEpset, bool isFillhistory) { int64_t uid = (isFillhistory) ? pStream->hTaskUid : pStream->uid; SArray** pTaskList = (isFillhistory) ? taosArrayGetLast(pStream->pHTasksList) : taosArrayGetLast(pStream->tasks); @@ -208,12 +210,27 @@ static int32_t doAddSinkTask(SStreamObj* pStream, SMnode* pMnode, int32_t vgId, } epsetAssign(&(pTask)->info.mnodeEpset, pEpset); + mDebug("doAddSinkTask taskId:%s, vgId:%d, isFillHistory:%d", pTask->id.idStr, pVgroup->vgId, isFillhistory); - pTask->info.nodeId = vgId; + pTask->info.nodeId = pVgroup->vgId; pTask->info.epSet = mndGetVgroupEpset(pMnode, pVgroup); return mndSetSinkTaskInfo(pStream, pTask); } +static int32_t doAddSinkTaskToVg(SMnode* pMnode, SStreamObj* pStream, SEpSet* pEpset, SVgObj* vgObj){ + int32_t code = doAddSinkTask(pStream, pMnode, vgObj, pEpset, false); + if (code != 0) { + return code; + } + if(pStream->conf.fillHistory){ + code = doAddSinkTask(pStream, pMnode, vgObj, pEpset, true); + if (code != 0) { + return code; + } + } + return TDB_CODE_SUCCESS; +} + // create sink node for each vgroup. static int32_t doAddShuffleSinkTask(SMnode* pMnode, SStreamObj* pStream, SEpSet* pEpset) { SSdb* pSdb = pMnode->pSdb; @@ -231,18 +248,12 @@ static int32_t doAddShuffleSinkTask(SMnode* pMnode, SStreamObj* pStream, SEpSet* continue; } - int32_t code = doAddSinkTask(pStream, pMnode, pVgroup->vgId, pVgroup, pEpset, false); + int32_t code = doAddSinkTaskToVg(pMnode, pStream, pEpset, pVgroup); if(code != 0){ sdbRelease(pSdb, pVgroup); return code; } - if(pStream->conf.fillHistory){ - code = doAddSinkTask(pStream, pMnode, pVgroup->vgId, pVgroup, pEpset, true); - if(code != 0){ - sdbRelease(pSdb, pVgroup); - return code; - } - } + sdbRelease(pSdb, pVgroup); } @@ -271,14 +282,20 @@ static SStreamTask* buildSourceTask(SStreamObj* pStream, SEpSet* pEpset, return pTask; } -static SArray* addNewTaskList(SArray* pTasksList) { +static void addNewTaskList(SStreamObj* pStream){ SArray* pTaskList = taosArrayInit(0, POINTER_BYTES); - taosArrayPush(pTasksList, &pTaskList); - return pTaskList; + taosArrayPush(pStream->tasks, &pTaskList); + if (pStream->conf.fillHistory) { + pTaskList = taosArrayInit(0, POINTER_BYTES); + taosArrayPush(pStream->pHTasksList, &pTaskList); + } } // set the history task id -static void setHTasksId(SArray* pTaskList, const SArray* pHTaskList) { +static void setHTasksId(SStreamObj* pStream) { + SArray* pTaskList = *(SArray**)taosArrayGetLast(pStream->tasks); + SArray* pHTaskList = *(SArray**)taosArrayGetLast(pStream->pHTasksList); + for (int32_t i = 0; i < taosArrayGetSize(pTaskList); ++i) { SStreamTask** pStreamTask = taosArrayGet(pTaskList, i); SStreamTask** pHTask = taosArrayGet(pHTaskList, i); @@ -303,6 +320,8 @@ static int32_t doAddSourceTask(SMnode* pMnode, SSubplan* plan, SStreamObj* pStre terrno = TSDB_CODE_OUT_OF_MEMORY; return terrno; } + mDebug("doAddSourceTask taskId:%s, vgId:%d, isFillHistory:%d", pTask->id.idStr, pVgroup->vgId, isFillhistory); + int32_t code = mndAssignStreamTaskToVgroup(pMnode, pTask, plan, pVgroup); if(code != 0){ terrno = code; @@ -345,12 +364,6 @@ static SSubplan* getAggSubPlan(const SQueryPlan* pPlan, int index){ static int32_t addSourceTask(SMnode* pMnode, SSubplan* plan, SStreamObj* pStream, SEpSet* pEpset, int64_t nextWindowSkey) { // 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); - } - void* pIter = NULL; SSdb* pSdb = pMnode->pSdb; while (1) { @@ -383,7 +396,7 @@ static int32_t addSourceTask(SMnode* pMnode, SSubplan* plan, SStreamObj* pStream } if (pStream->conf.fillHistory) { - setHTasksId(pTaskList, pHTaskList); + setHTasksId(pStream); } return TSDB_CODE_SUCCESS; @@ -403,12 +416,24 @@ static SStreamTask* buildAggTask(SStreamObj* pStream, SEpSet* pEpset, bool isFil return pAggTask; } -static int32_t doAddAggTask(SStreamObj* pStream, SMnode* pMnode, SSubplan* plan, SEpSet* pEpset){ - SStreamTask* pTask = buildAggTask(pStream, pEpset, false); +static int32_t doAddAggTask(SStreamObj* pStream, SMnode* pMnode, SSubplan* plan, SEpSet* pEpset, SVgObj* pVgroup, SSnodeObj* pSnode, bool isFillhistory){ + int32_t code = 0; + SStreamTask* pTask = buildAggTask(pStream, pEpset, isFillhistory); if (pTask == NULL) { return terrno; } + if (pSnode != NULL) { + code = mndAssignStreamTaskToSnode(pMnode, pTask, plan, pSnode); + mDebug("doAddAggTask taskId:%s, snode id:%d, isFillHistory:%d", pTask->id.idStr, pSnode->id, isFillhistory); + } else { + code = mndAssignStreamTaskToVgroup(pMnode, pTask, plan, pVgroup); + mDebug("doAddAggTask taskId:%s, vgId:%d, isFillHistory:%d", pTask->id.idStr, pVgroup->vgId, isFillhistory); + } + return code; +} + +static int32_t addAggTask(SStreamObj* pStream, SMnode* pMnode, SSubplan* plan, SEpSet* pEpset){ SVgObj* pVgroup = NULL; SSnodeObj* pSnode = NULL; int32_t code = 0; @@ -421,34 +446,18 @@ static int32_t doAddAggTask(SStreamObj* pStream, SMnode* pMnode, SSubplan* plan, pVgroup = mndSchedFetchOneVg(pMnode, pStream->sourceDbUid); } - if (pSnode != NULL) { - code = mndAssignStreamTaskToSnode(pMnode, pTask, plan, pSnode); - } else { - code = mndAssignStreamTaskToVgroup(pMnode, pTask, plan, pVgroup); - } + code = doAddAggTask(pStream, pMnode, plan, pEpset, pVgroup, pSnode, false); if(code != 0){ goto END; } if (pStream->conf.fillHistory) { - pTask = buildAggTask(pStream, pEpset, true); - if (pTask == NULL) { - code = terrno; - goto END; - } - - if (pSnode != NULL) { - code = mndAssignStreamTaskToSnode(pMnode, pTask, plan, pSnode); - } else { - code = mndAssignStreamTaskToVgroup(pMnode, pTask, plan, pVgroup); - } + code = doAddAggTask(pStream, pMnode, plan, pEpset, pVgroup, pSnode, true); if(code != 0){ goto END; } - SArray** pAggTaskList = taosArrayGetLast(pStream->tasks); - SArray** pHAggTaskList = taosArrayGetLast(pStream->pHTasksList); - setHTasksId(*pAggTaskList, *pHAggTaskList); + setHTasksId(pStream); } END: @@ -460,43 +469,24 @@ static int32_t doAddAggTask(SStreamObj* pStream, SMnode* pMnode, SSubplan* plan, return code; } -static int32_t addAggTask(SStreamObj* pStream, SMnode* pMnode, SSubplan* plan, SEpSet* pEpset) { - addNewTaskList(pStream->tasks); - if (pStream->conf.fillHistory) { - addNewTaskList(pStream->pHTasksList); - } - return doAddAggTask(pStream, pMnode, plan, pEpset); -} - static int32_t addSinkTask(SMnode* pMnode, SStreamObj* pStream, SEpSet* pEpset){ - SArray* pSinkTaskList = addNewTaskList(pStream->tasks); - - SArray* pHSinkTaskList = NULL; - if (pStream->conf.fillHistory) { - pHSinkTaskList = addNewTaskList(pStream->pHTasksList); - } - int32_t code = 0; + addNewTaskList(pStream); + if (pStream->fixedSinkVgId == 0) { code = doAddShuffleSinkTask(pMnode, pStream, pEpset); if (code != 0) { return code; } } else { - code = doAddSinkTask(pStream, pMnode, pStream->fixedSinkVgId, &pStream->fixedSinkVg, pEpset, false); + code = doAddSinkTaskToVg(pMnode, pStream, pEpset, &pStream->fixedSinkVg); if (code != 0) { return code; } - if(pStream->conf.fillHistory){ - code = doAddSinkTask(pStream, pMnode, pStream->fixedSinkVgId, &pStream->fixedSinkVg, pEpset, true); - if (code != 0) { - return code; - } - } } if (pStream->conf.fillHistory) { - setHTasksId(pSinkTaskList, pHSinkTaskList); + setHTasksId(pStream); } return TDB_CODE_SUCCESS; } @@ -507,6 +497,7 @@ static void bindTaskToSinkTask(SStreamObj* pStream, SMnode* pMnode, SArray* pSin SStreamTask* pSinkTask = taosArrayGetP(pSinkTaskList, k); streamTaskSetUpstreamInfo(pSinkTask, task); } + mDebug("bindTaskToSinkTask taskId:%s to sink task list", task->id.idStr); } static void bindAggSink(SStreamObj* pStream, SMnode* pMnode, SArray* tasks) { @@ -516,6 +507,7 @@ static void bindAggSink(SStreamObj* pStream, SMnode* pMnode, SArray* tasks) { for(int i = 0; i < taosArrayGetSize(*pAggTaskList); i++){ SStreamTask* pAggTask = taosArrayGetP(*pAggTaskList, i); bindTaskToSinkTask(pStream, pMnode, pSinkTaskList, pAggTask); + mDebug("bindAggSink taskId:%s to sink task list", pAggTask->id.idStr); } } @@ -525,6 +517,8 @@ static void bindSourceSink(SStreamObj* pStream, SMnode* pMnode, SArray* tasks, b for(int i = 0; i < taosArrayGetSize(pSourceTaskList); i++){ SStreamTask* pSourceTask = taosArrayGetP(pSourceTaskList, i); + mDebug("bindSourceSink taskId:%s to sink task list", pSourceTask->id.idStr); + if (hasExtraSink) { bindTaskToSinkTask(pStream, pMnode, pSinkTaskList, pSourceTask); } else { @@ -548,6 +542,7 @@ static void bindTwoLevel(SArray* tasks, int32_t begin, int32_t end) { streamTaskSetFixedDownstreamInfo(pUpTask, *pDownTask); streamTaskSetUpstreamInfo(*pDownTask, pUpTask); } + mDebug("bindTwoLevel task list(%d-%d) to taskId:%s", begin, end, (*(pDownTask))->id.idStr); } static int32_t doScheduleStream(SStreamObj* pStream, SMnode* pMnode, SQueryPlan* pPlan, int64_t nextWindowSkey, SEpSet* pEpset) { @@ -564,6 +559,8 @@ static int32_t doScheduleStream(SStreamObj* pStream, SMnode* pMnode, SQueryPlan* bool multiTarget = (pDbObj->cfg.numOfVgroups > 1); sdbRelease(pSdb, pDbObj); + mDebug("doScheduleStream numOfPlanLevel:%d, exDb:%d, multiTarget:%d, fix vgId:%d, physicalPlan:%s", + numOfPlanLevel, externalTargetDB, multiTarget, pStream->fixedSinkVgId, pStream->physicalPlan); pStream->tasks = taosArrayInit(numOfPlanLevel + 1, POINTER_BYTES); pStream->pHTasksList = taosArrayInit(numOfPlanLevel + 1, POINTER_BYTES); @@ -606,13 +603,11 @@ static int32_t doScheduleStream(SStreamObj* pStream, SMnode* pMnode, SQueryPlan* size_t cnt = (int)(size/tsStreamAggCnt + 0.5); if(cnt <= 1) break; - addNewTaskList(pStream->tasks); - if (pStream->conf.fillHistory) { - addNewTaskList(pStream->pHTasksList); - } + mDebug("doScheduleStream add middle agg, size:%d, cnt:%d", (int)size, (int)cnt); + addNewTaskList(pStream); for(int j = 0; j < cnt; j++){ - code = doAddAggTask(pStream, pMnode, plan, pEpset); + code = addAggTask(pStream, pMnode, plan, pEpset); if (code != TSDB_CODE_SUCCESS) { return code; } @@ -630,8 +625,10 @@ static int32_t doScheduleStream(SStreamObj* pStream, SMnode* pMnode, SQueryPlan* return terrno; } + mDebug("doScheduleStream add final agg"); SArray** list = taosArrayGetLast(pStream->tasks); size_t size = taosArrayGetSize(*list); + addNewTaskList(pStream); code = addAggTask(pStream, pMnode, plan, pEpset); if (code != TSDB_CODE_SUCCESS) { return code; From 623ab98392bbdc7fd9440b36b0e156f74f1e0bb0 Mon Sep 17 00:00:00 2001 From: wangmm0220 Date: Mon, 11 Dec 2023 19:58:47 +0800 Subject: [PATCH 13/51] fix:heap overflow --- source/libs/scalar/src/scalar.c | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/source/libs/scalar/src/scalar.c b/source/libs/scalar/src/scalar.c index 90cec4522f..b1cd10eac9 100644 --- a/source/libs/scalar/src/scalar.c +++ b/source/libs/scalar/src/scalar.c @@ -1367,8 +1367,8 @@ EDealRes sclRewriteCaseWhen(SNode **pNode, SScalarCtx *ctx) { } else { int32_t type = output.columnData->info.type; if (IS_VAR_DATA_TYPE(type)) { // todo refactor - res->datum.p = output.columnData->pData; - output.columnData->pData = NULL; + res->datum.p = taosMemoryCalloc(varDataTLen(output.columnData->pData) + 1, sizeof(char)); // add \0 to the end for print json value + memcpy(res->datum.p, output.columnData->pData, varDataTLen(output.columnData->pData)); } else { nodesSetValueNodeValue(res, output.columnData->pData); } From 7ba6135f2c2816ab4de1d3ac8279a7890a0f14b8 Mon Sep 17 00:00:00 2001 From: wangmm0220 Date: Tue, 12 Dec 2023 16:37:22 +0800 Subject: [PATCH 14/51] fix:error in split plan --- source/dnode/mnode/impl/src/mndScheduler.c | 3 +- source/dnode/mnode/impl/src/mndStream.c | 1 + .../mnode/impl/test/stream/streamPlanTest.cpp | 3 +- source/libs/planner/src/planSpliter.c | 49 ++++++++++++++++++- 4 files changed, 53 insertions(+), 3 deletions(-) diff --git a/source/dnode/mnode/impl/src/mndScheduler.c b/source/dnode/mnode/impl/src/mndScheduler.c index ac01890b9b..bf8e9ab4d1 100644 --- a/source/dnode/mnode/impl/src/mndScheduler.c +++ b/source/dnode/mnode/impl/src/mndScheduler.c @@ -363,7 +363,8 @@ static SSubplan* getAggSubPlan(const SQueryPlan* pPlan, int index){ static int32_t addSourceTask(SMnode* pMnode, SSubplan* plan, SStreamObj* pStream, SEpSet* pEpset, int64_t nextWindowSkey) { - // create exec stream task, since only one level, the exec task is also the source task + addNewTaskList(pStream); + void* pIter = NULL; SSdb* pSdb = pMnode->pSdb; while (1) { diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index e8d5dfd1f5..a495ccd28f 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -422,6 +422,7 @@ static int32_t mndBuildStreamObjFromCreateReq(SMnode *pMnode, SStreamObj *pObj, pCreate->sql = NULL; pCreate->ast = NULL; + qDebugL("ast:%s", pObj->ast); // deserialize ast if (nodesStringToNode(pObj->ast, &pAst) < 0) { goto FAIL; diff --git a/source/dnode/mnode/impl/test/stream/streamPlanTest.cpp b/source/dnode/mnode/impl/test/stream/streamPlanTest.cpp index d769ef674b..3adc8c9b32 100644 --- a/source/dnode/mnode/impl/test/stream/streamPlanTest.cpp +++ b/source/dnode/mnode/impl/test/stream/streamPlanTest.cpp @@ -34,7 +34,8 @@ int main(int argc, char **argv) { } TEST(testCase, plan_Test) { - char* ast = "{\"NodeType\":\"101\",\"Name\":\"SelectStmt\",\"SelectStmt\":{\"Distinct\":false,\"Projections\":[{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"9\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"#expr_1\",\"UserAlias\":\"_wstart\",\"Name\":\"_wstart\",\"Id\":\"89\",\"Type\":\"3505\",\"UdfBufSize\":\"0\"}},{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"5\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"#expr_2\",\"UserAlias\":\"sum(voltage)\",\"Name\":\"sum\",\"Id\":\"1\",\"Type\":\"14\",\"Parameters\":[{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"4\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"4\"},\"AliasName\":\"voltage\",\"UserAlias\":\"voltage\",\"TableId\":\"6555383776122680534\",\"TableType\":\"1\",\"ColId\":\"3\",\"ProjId\":\"0\",\"ColType\":\"1\",\"DbName\":\"test\",\"TableName\":\"meters\",\"TableAlias\":\"meters\",\"ColName\":\"voltage\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"4\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"4\"},\"AliasName\":\"#expr_3\",\"UserAlias\":\"groupid\",\"Name\":\"_group_key\",\"Id\":\"96\",\"Type\":\"3754\",\"Parameters\":[{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"4\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"4\"},\"AliasName\":\"#expr_3\",\"UserAlias\":\"groupid\",\"TableId\":\"6555383776122680534\",\"TableType\":\"1\",\"ColId\":\"5\",\"ProjId\":\"0\",\"ColType\":\"2\",\"DbName\":\"test\",\"TableName\":\"meters\",\"TableAlias\":\"meters\",\"ColName\":\"groupid\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}}],\"UdfBufSize\":\"0\"}}],\"From\":{\"NodeType\":\"6\",\"Name\":\"RealTable\",\"RealTable\":{\"DataType\":{\"Type\":\"0\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"0\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"DbName\":\"test\",\"tableName\":\"meters\",\"tableAlias\":\"meters\",\"MetaSize\":\"475\",\"Meta\":{\"VgId\":\"0\",\"TableType\":\"1\",\"Uid\":\"6555383776122680534\",\"Suid\":\"6555383776122680534\",\"Sversion\":\"1\",\"Tversion\":\"1\",\"ComInfo\":{\"NumOfTags\":\"2\",\"Precision\":\"0\",\"NumOfColumns\":\"4\",\"RowSize\":\"20\"},\"ColSchemas\":[{\"Type\":\"9\",\"ColId\":\"1\",\"bytes\":\"8\",\"Name\":\"ts\"},{\"Type\":\"6\",\"ColId\":\"2\",\"bytes\":\"4\",\"Name\":\"current\"},{\"Type\":\"4\",\"ColId\":\"3\",\"bytes\":\"4\",\"Name\":\"voltage\"},{\"Type\":\"6\",\"ColId\":\"4\",\"bytes\":\"4\",\"Name\":\"phase\"},{\"Type\":\"4\",\"ColId\":\"5\",\"bytes\":\"4\",\"Name\":\"groupid\"},{\"Type\":\"8\",\"ColId\":\"6\",\"bytes\":\"26\",\"Name\":\"location\"}]},\"VgroupsInfoSize\":\"1340\",\"VgroupsInfo\":{\"Num\":\"2\",\"Vgroups\":[{\"VgId\":\"2\",\"HashBegin\":\"0\",\"HashEnd\":\"2147483646\",\"EpSet\":{\"InUse\":\"0\",\"NumOfEps\":\"1\",\"Eps\":[{\"Fqdn\":\"localhost\",\"Port\":\"6030\"}]},\"NumOfTable\":\"0\"},{\"VgId\":\"3\",\"HashBegin\":\"2147483647\",\"HashEnd\":\"4294967295\",\"EpSet\":{\"InUse\":\"0\",\"NumOfEps\":\"1\",\"Eps\":[{\"Fqdn\":\"localhost\",\"Port\":\"6030\"}]},\"NumOfTable\":\"0\"}]}}},\"PartitionBy\":[{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"4\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"4\"},\"AliasName\":\"groupid\",\"UserAlias\":\"groupid\",\"TableId\":\"6555383776122680534\",\"TableType\":\"1\",\"ColId\":\"5\",\"ProjId\":\"0\",\"ColType\":\"2\",\"DbName\":\"test\",\"TableName\":\"meters\",\"TableAlias\":\"meters\",\"ColName\":\"groupid\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}}],\"Window\":{\"NodeType\":\"14\",\"Name\":\"IntervalWindow\",\"IntervalWindow\":{\"Interval\":{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"5\",\"Precision\":\"0\",\"Scale\":\"115\",\"Bytes\":\"8\"},\"AliasName\":\"c804c3a15ebe05b5baf40ad5ee12be1f\",\"UserAlias\":\"2s\",\"LiteralSize\":\"2\",\"Literal\":\"2s\",\"Duration\":true,\"Translate\":true,\"NotReserved\":false,\"IsNull\":false,\"Unit\":\"115\",\"Datum\":\"2000\"}},\"TsPk\":{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"9\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"ts\",\"UserAlias\":\"ts\",\"TableId\":\"6555383776122680534\",\"TableType\":\"1\",\"ColId\":\"1\",\"ProjId\":\"0\",\"ColType\":\"1\",\"DbName\":\"test\",\"TableName\":\"meters\",\"TableAlias\":\"meters\",\"ColName\":\"ts\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}}}},\"StmtName\":\"0x1580095ba\",\"HasAggFuncs\":true}}"; +// char* ast = "{\"NodeType\":\"101\",\"Name\":\"SelectStmt\",\"SelectStmt\":{\"Distinct\":false,\"Projections\":[{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"9\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"#expr_1\",\"UserAlias\":\"_wstart\",\"Name\":\"_wstart\",\"Id\":\"89\",\"Type\":\"3505\",\"UdfBufSize\":\"0\"}},{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"5\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"#expr_2\",\"UserAlias\":\"sum(voltage)\",\"Name\":\"sum\",\"Id\":\"1\",\"Type\":\"14\",\"Parameters\":[{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"4\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"4\"},\"AliasName\":\"voltage\",\"UserAlias\":\"voltage\",\"TableId\":\"6555383776122680534\",\"TableType\":\"1\",\"ColId\":\"3\",\"ProjId\":\"0\",\"ColType\":\"1\",\"DbName\":\"test\",\"TableName\":\"meters\",\"TableAlias\":\"meters\",\"ColName\":\"voltage\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"4\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"4\"},\"AliasName\":\"#expr_3\",\"UserAlias\":\"groupid\",\"Name\":\"_group_key\",\"Id\":\"96\",\"Type\":\"3754\",\"Parameters\":[{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"4\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"4\"},\"AliasName\":\"#expr_3\",\"UserAlias\":\"groupid\",\"TableId\":\"6555383776122680534\",\"TableType\":\"1\",\"ColId\":\"5\",\"ProjId\":\"0\",\"ColType\":\"2\",\"DbName\":\"test\",\"TableName\":\"meters\",\"TableAlias\":\"meters\",\"ColName\":\"groupid\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}}],\"UdfBufSize\":\"0\"}}],\"From\":{\"NodeType\":\"6\",\"Name\":\"RealTable\",\"RealTable\":{\"DataType\":{\"Type\":\"0\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"0\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"DbName\":\"test\",\"tableName\":\"meters\",\"tableAlias\":\"meters\",\"MetaSize\":\"475\",\"Meta\":{\"VgId\":\"0\",\"TableType\":\"1\",\"Uid\":\"6555383776122680534\",\"Suid\":\"6555383776122680534\",\"Sversion\":\"1\",\"Tversion\":\"1\",\"ComInfo\":{\"NumOfTags\":\"2\",\"Precision\":\"0\",\"NumOfColumns\":\"4\",\"RowSize\":\"20\"},\"ColSchemas\":[{\"Type\":\"9\",\"ColId\":\"1\",\"bytes\":\"8\",\"Name\":\"ts\"},{\"Type\":\"6\",\"ColId\":\"2\",\"bytes\":\"4\",\"Name\":\"current\"},{\"Type\":\"4\",\"ColId\":\"3\",\"bytes\":\"4\",\"Name\":\"voltage\"},{\"Type\":\"6\",\"ColId\":\"4\",\"bytes\":\"4\",\"Name\":\"phase\"},{\"Type\":\"4\",\"ColId\":\"5\",\"bytes\":\"4\",\"Name\":\"groupid\"},{\"Type\":\"8\",\"ColId\":\"6\",\"bytes\":\"26\",\"Name\":\"location\"}]},\"VgroupsInfoSize\":\"1340\",\"VgroupsInfo\":{\"Num\":\"2\",\"Vgroups\":[{\"VgId\":\"2\",\"HashBegin\":\"0\",\"HashEnd\":\"2147483646\",\"EpSet\":{\"InUse\":\"0\",\"NumOfEps\":\"1\",\"Eps\":[{\"Fqdn\":\"localhost\",\"Port\":\"6030\"}]},\"NumOfTable\":\"0\"},{\"VgId\":\"3\",\"HashBegin\":\"2147483647\",\"HashEnd\":\"4294967295\",\"EpSet\":{\"InUse\":\"0\",\"NumOfEps\":\"1\",\"Eps\":[{\"Fqdn\":\"localhost\",\"Port\":\"6030\"}]},\"NumOfTable\":\"0\"}]}}},\"PartitionBy\":[{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"4\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"4\"},\"AliasName\":\"groupid\",\"UserAlias\":\"groupid\",\"TableId\":\"6555383776122680534\",\"TableType\":\"1\",\"ColId\":\"5\",\"ProjId\":\"0\",\"ColType\":\"2\",\"DbName\":\"test\",\"TableName\":\"meters\",\"TableAlias\":\"meters\",\"ColName\":\"groupid\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}}],\"Window\":{\"NodeType\":\"14\",\"Name\":\"IntervalWindow\",\"IntervalWindow\":{\"Interval\":{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"5\",\"Precision\":\"0\",\"Scale\":\"115\",\"Bytes\":\"8\"},\"AliasName\":\"c804c3a15ebe05b5baf40ad5ee12be1f\",\"UserAlias\":\"2s\",\"LiteralSize\":\"2\",\"Literal\":\"2s\",\"Duration\":true,\"Translate\":true,\"NotReserved\":false,\"IsNull\":false,\"Unit\":\"115\",\"Datum\":\"2000\"}},\"TsPk\":{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"9\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"ts\",\"UserAlias\":\"ts\",\"TableId\":\"6555383776122680534\",\"TableType\":\"1\",\"ColId\":\"1\",\"ProjId\":\"0\",\"ColType\":\"1\",\"DbName\":\"test\",\"TableName\":\"meters\",\"TableAlias\":\"meters\",\"ColName\":\"ts\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}}}},\"StmtName\":\"0x1580095ba\",\"HasAggFuncs\":true}}"; + char* ast = "{\"NodeType\":\"101\",\"Name\":\"SelectStmt\",\"SelectStmt\":{\"Distinct\":false,\"Projections\":[{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"9\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"#expr_1\",\"UserAlias\":\"wstart\",\"Name\":\"_wstart\",\"Id\":\"89\",\"Type\":\"3505\",\"UdfBufSize\":\"0\"}},{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"2\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"#expr_2\",\"UserAlias\":\"min(c1)\",\"Name\":\"min\",\"Id\":\"2\",\"Type\":\"8\",\"Parameters\":[{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"2\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"c1\",\"UserAlias\":\"c1\",\"TableId\":\"5129202035162885657\",\"TableType\":\"1\",\"ColId\":\"2\",\"ProjId\":\"0\",\"ColType\":\"1\",\"DbName\":\"test\",\"TableName\":\"at_once_interval_ext_stb\",\"TableAlias\":\"at_once_interval_ext_stb\",\"ColName\":\"c1\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"3\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"2\"},\"AliasName\":\"#expr_3\",\"UserAlias\":\"max(c2)\",\"Name\":\"max\",\"Id\":\"3\",\"Type\":\"7\",\"Parameters\":[{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"3\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"2\"},\"AliasName\":\"c2\",\"UserAlias\":\"c2\",\"TableId\":\"5129202035162885657\",\"TableType\":\"1\",\"ColId\":\"3\",\"ProjId\":\"0\",\"ColType\":\"1\",\"DbName\":\"test\",\"TableName\":\"at_once_interval_ext_stb\",\"TableAlias\":\"at_once_interval_ext_stb\",\"ColName\":\"c2\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"4\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"4\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"Name\":\"cast\",\"Id\":\"77\",\"Type\":\"2000\",\"Parameters\":[{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"5\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"#expr_4\",\"UserAlias\":\"sum(c3)\",\"Name\":\"sum\",\"Id\":\"1\",\"Type\":\"14\",\"Parameters\":[{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"4\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"4\"},\"AliasName\":\"c3\",\"UserAlias\":\"c3\",\"TableId\":\"5129202035162885657\",\"TableType\":\"1\",\"ColId\":\"4\",\"ProjId\":\"0\",\"ColType\":\"1\",\"DbName\":\"test\",\"TableName\":\"at_once_interval_ext_stb\",\"TableAlias\":\"at_once_interval_ext_stb\",\"ColName\":\"c3\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"2\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":true,\"NotReserved\":true,\"IsNull\":false,\"Unit\":\"0\",\"Datum\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"5\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"#expr_5\",\"UserAlias\":\"first(c4)\",\"Name\":\"first\",\"Id\":\"33\",\"Type\":\"504\",\"Parameters\":[{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"5\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"c4\",\"UserAlias\":\"c4\",\"TableId\":\"5129202035162885657\",\"TableType\":\"1\",\"ColId\":\"5\",\"ProjId\":\"0\",\"ColType\":\"1\",\"DbName\":\"test\",\"TableName\":\"at_once_interval_ext_stb\",\"TableAlias\":\"at_once_interval_ext_stb\",\"ColName\":\"c4\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}},{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"9\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"ts\",\"UserAlias\":\"ts\",\"TableId\":\"5129202035162885657\",\"TableType\":\"1\",\"ColId\":\"1\",\"ProjId\":\"0\",\"ColType\":\"1\",\"DbName\":\"test\",\"TableName\":\"at_once_interval_ext_stb\",\"TableAlias\":\"at_once_interval_ext_stb\",\"ColName\":\"ts\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"11\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"#expr_6\",\"UserAlias\":\"last(c5)\",\"Name\":\"last\",\"Id\":\"36\",\"Type\":\"506\",\"Parameters\":[{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"11\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"c5\",\"UserAlias\":\"c5\",\"TableId\":\"5129202035162885657\",\"TableType\":\"1\",\"ColId\":\"6\",\"ProjId\":\"0\",\"ColType\":\"1\",\"DbName\":\"test\",\"TableName\":\"at_once_interval_ext_stb\",\"TableAlias\":\"at_once_interval_ext_stb\",\"ColName\":\"c5\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}},{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"9\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"ts\",\"UserAlias\":\"ts\",\"TableId\":\"5129202035162885657\",\"TableType\":\"1\",\"ColId\":\"1\",\"ProjId\":\"0\",\"ColType\":\"1\",\"DbName\":\"test\",\"TableName\":\"at_once_interval_ext_stb\",\"TableAlias\":\"at_once_interval_ext_stb\",\"ColName\":\"ts\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"12\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"2\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"Name\":\"cast\",\"Id\":\"77\",\"Type\":\"2000\",\"Parameters\":[{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"7\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"#expr_7\",\"UserAlias\":\"apercentile(c6, 50)\",\"Name\":\"apercentile\",\"Id\":\"12\",\"Type\":\"1\",\"Parameters\":[{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"12\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"2\"},\"AliasName\":\"c6\",\"UserAlias\":\"c6\",\"TableId\":\"5129202035162885657\",\"TableType\":\"1\",\"ColId\":\"7\",\"ProjId\":\"0\",\"ColType\":\"1\",\"DbName\":\"test\",\"TableName\":\"at_once_interval_ext_stb\",\"TableAlias\":\"at_once_interval_ext_stb\",\"ColName\":\"c6\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"5\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"c0c7c76d30bd3dcaefc96f40275bdc0a\",\"UserAlias\":\"50\",\"LiteralSize\":\"2\",\"Literal\":\"50\",\"Duration\":false,\"Translate\":true,\"NotReserved\":true,\"IsNull\":false,\"Unit\":\"0\",\"Datum\":\"50\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"2\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":true,\"NotReserved\":true,\"IsNull\":false,\"Unit\":\"0\",\"Datum\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"13\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"4\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"Name\":\"cast\",\"Id\":\"77\",\"Type\":\"2000\",\"Parameters\":[{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"7\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"#expr_8\",\"UserAlias\":\"avg(c7)\",\"Name\":\"avg\",\"Id\":\"8\",\"Type\":\"2\",\"Parameters\":[{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"13\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"4\"},\"AliasName\":\"c7\",\"UserAlias\":\"c7\",\"TableId\":\"5129202035162885657\",\"TableType\":\"1\",\"ColId\":\"8\",\"ProjId\":\"0\",\"ColType\":\"1\",\"DbName\":\"test\",\"TableName\":\"at_once_interval_ext_stb\",\"TableAlias\":\"at_once_interval_ext_stb\",\"ColName\":\"c7\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"2\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":true,\"NotReserved\":true,\"IsNull\":false,\"Unit\":\"0\",\"Datum\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"14\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"Name\":\"cast\",\"Id\":\"77\",\"Type\":\"2000\",\"Parameters\":[{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"5\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"#expr_9\",\"UserAlias\":\"count(c8)\",\"Name\":\"count\",\"Id\":\"0\",\"Type\":\"3\",\"Parameters\":[{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"14\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"c8\",\"UserAlias\":\"c8\",\"TableId\":\"5129202035162885657\",\"TableType\":\"1\",\"ColId\":\"9\",\"ProjId\":\"0\",\"ColType\":\"1\",\"DbName\":\"test\",\"TableName\":\"at_once_interval_ext_stb\",\"TableAlias\":\"at_once_interval_ext_stb\",\"ColName\":\"c8\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"2\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":true,\"NotReserved\":true,\"IsNull\":false,\"Unit\":\"0\",\"Datum\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"6\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"4\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"Name\":\"cast\",\"Id\":\"77\",\"Type\":\"2000\",\"Parameters\":[{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"7\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"#expr_10\",\"UserAlias\":\"spread(c1)\",\"Name\":\"spread\",\"Id\":\"17\",\"Type\":\"11\",\"Parameters\":[{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"2\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"c1\",\"UserAlias\":\"c1\",\"TableId\":\"5129202035162885657\",\"TableType\":\"1\",\"ColId\":\"2\",\"ProjId\":\"0\",\"ColType\":\"1\",\"DbName\":\"test\",\"TableName\":\"at_once_interval_ext_stb\",\"TableAlias\":\"at_once_interval_ext_stb\",\"ColName\":\"c1\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"2\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":true,\"NotReserved\":true,\"IsNull\":false,\"Unit\":\"0\",\"Datum\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"7\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"#expr_11\",\"UserAlias\":\"stddev(c2)\",\"Name\":\"stddev\",\"Id\":\"4\",\"Type\":\"12\",\"Parameters\":[{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"3\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"2\"},\"AliasName\":\"c2\",\"UserAlias\":\"c2\",\"TableId\":\"5129202035162885657\",\"TableType\":\"1\",\"ColId\":\"3\",\"ProjId\":\"0\",\"ColType\":\"1\",\"DbName\":\"test\",\"TableName\":\"at_once_interval_ext_stb\",\"TableAlias\":\"at_once_interval_ext_stb\",\"ColName\":\"c2\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"8\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"Name\":\"cast\",\"Id\":\"77\",\"Type\":\"2000\",\"Parameters\":[{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"5\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"#expr_12\",\"UserAlias\":\"hyperloglog(c11)\",\"Name\":\"hyperloglog\",\"Id\":\"43\",\"Type\":\"17\",\"Parameters\":[{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"8\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"c11\",\"UserAlias\":\"c11\",\"TableId\":\"5129202035162885657\",\"TableType\":\"1\",\"ColId\":\"12\",\"ProjId\":\"0\",\"ColType\":\"1\",\"DbName\":\"test\",\"TableName\":\"at_once_interval_ext_stb\",\"TableAlias\":\"at_once_interval_ext_stb\",\"ColName\":\"c11\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"2\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":true,\"NotReserved\":true,\"IsNull\":false,\"Unit\":\"0\",\"Datum\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"10\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"26\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"Name\":\"cast\",\"Id\":\"77\",\"Type\":\"2000\",\"Parameters\":[{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"5\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"#expr_13\",\"UserAlias\":\"timediff(1, 0, 1h)\",\"Name\":\"timediff\",\"Id\":\"81\",\"Type\":\"2501\",\"Parameters\":[{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"5\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"c4ca4238a0b923820dcc509a6f75849b\",\"UserAlias\":\"1\",\"LiteralSize\":\"1\",\"Literal\":\"1\",\"Duration\":false,\"Translate\":true,\"NotReserved\":false,\"IsNull\":false,\"Unit\":\"0\",\"Datum\":\"1\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"5\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"cfcd208495d565ef66e7dff9f98764da\",\"UserAlias\":\"0\",\"LiteralSize\":\"1\",\"Literal\":\"0\",\"Duration\":false,\"Translate\":true,\"NotReserved\":false,\"IsNull\":false,\"Unit\":\"0\",\"Datum\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"5\",\"Precision\":\"0\",\"Scale\":\"104\",\"Bytes\":\"8\"},\"AliasName\":\"7c68645d71b803bf0ba2f22519f73e08\",\"UserAlias\":\"1h\",\"LiteralSize\":\"2\",\"Literal\":\"1h\",\"Duration\":true,\"Translate\":true,\"NotReserved\":false,\"IsNull\":false,\"Unit\":\"104\",\"Datum\":\"3600000\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"2\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":true,\"NotReserved\":true,\"IsNull\":false,\"Unit\":\"0\",\"Datum\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"2\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":true,\"NotReserved\":true,\"IsNull\":false,\"Unit\":\"0\",\"Datum\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"1\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"Name\":\"cast\",\"Id\":\"77\",\"Type\":\"2000\",\"Parameters\":[{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"8\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"96\"},\"AliasName\":\"#expr_14\",\"UserAlias\":\"timezone()\",\"Name\":\"timezone\",\"Id\":\"84\",\"Type\":\"2503\",\"UdfBufSize\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"2\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":true,\"NotReserved\":true,\"IsNull\":false,\"Unit\":\"0\",\"Datum\":\"0\"}}],\"UdfBufSize\":\"0\"}}],\"From\":{\"NodeType\":\"6\",\"Name\":\"RealTable\",\"RealTable\":{\"DataType\":{\"Type\":\"0\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"0\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"DbName\":\"test\",\"tableName\":\"at_once_interval_ext_stb\",\"tableAlias\":\"at_once_interval_ext_stb\",\"MetaSize\":\"2008\",\"Meta\":{\"VgId\":\"0\",\"TableType\":\"1\",\"Uid\":\"5129202035162885657\",\"Suid\":\"5129202035162885657\",\"Sversion\":\"1\",\"Tversion\":\"1\",\"ComInfo\":{\"NumOfTags\":\"13\",\"Precision\":\"0\",\"NumOfColumns\":\"14\",\"RowSize\":\"85\"},\"ColSchemas\":[{\"Type\":\"9\",\"ColId\":\"1\",\"bytes\":\"8\",\"Name\":\"ts\"},{\"Type\":\"2\",\"ColId\":\"2\",\"bytes\":\"1\",\"Name\":\"c1\"},{\"Type\":\"3\",\"ColId\":\"3\",\"bytes\":\"2\",\"Name\":\"c2\"},{\"Type\":\"4\",\"ColId\":\"4\",\"bytes\":\"4\",\"Name\":\"c3\"},{\"Type\":\"5\",\"ColId\":\"5\",\"bytes\":\"8\",\"Name\":\"c4\"},{\"Type\":\"11\",\"ColId\":\"6\",\"bytes\":\"1\",\"Name\":\"c5\"},{\"Type\":\"12\",\"ColId\":\"7\",\"bytes\":\"2\",\"Name\":\"c6\"},{\"Type\":\"13\",\"ColId\":\"8\",\"bytes\":\"4\",\"Name\":\"c7\"},{\"Type\":\"14\",\"ColId\":\"9\",\"bytes\":\"8\",\"Name\":\"c8\"},{\"Type\":\"6\",\"ColId\":\"10\",\"bytes\":\"4\",\"Name\":\"c9\"},{\"Type\":\"7\",\"ColId\":\"11\",\"bytes\":\"8\",\"Name\":\"c10\"},{\"Type\":\"8\",\"ColId\":\"12\",\"bytes\":\"8\",\"Name\":\"c11\"},{\"Type\":\"10\",\"ColId\":\"13\",\"bytes\":\"26\",\"Name\":\"c12\"},{\"Type\":\"1\",\"ColId\":\"14\",\"bytes\":\"1\",\"Name\":\"c13\"},{\"Type\":\"2\",\"ColId\":\"15\",\"bytes\":\"1\",\"Name\":\"t1\"},{\"Type\":\"3\",\"ColId\":\"16\",\"bytes\":\"2\",\"Name\":\"t2\"},{\"Type\":\"4\",\"ColId\":\"17\",\"bytes\":\"4\",\"Name\":\"t3\"},{\"Type\":\"5\",\"ColId\":\"18\",\"bytes\":\"8\",\"Name\":\"t4\"},{\"Type\":\"11\",\"ColId\":\"19\",\"bytes\":\"1\",\"Name\":\"t5\"},{\"Type\":\"12\",\"ColId\":\"20\",\"bytes\":\"2\",\"Name\":\"t6\"},{\"Type\":\"13\",\"ColId\":\"21\",\"bytes\":\"4\",\"Name\":\"t7\"},{\"Type\":\"14\",\"ColId\":\"22\",\"bytes\":\"8\",\"Name\":\"t8\"},{\"Type\":\"6\",\"ColId\":\"23\",\"bytes\":\"4\",\"Name\":\"t9\"},{\"Type\":\"7\",\"ColId\":\"24\",\"bytes\":\"8\",\"Name\":\"t10\"},{\"Type\":\"8\",\"ColId\":\"25\",\"bytes\":\"8\",\"Name\":\"t11\"},{\"Type\":\"10\",\"ColId\":\"26\",\"bytes\":\"26\",\"Name\":\"t12\"},{\"Type\":\"1\",\"ColId\":\"27\",\"bytes\":\"1\",\"Name\":\"t13\"}]},\"VgroupsInfoSize\":\"1340\",\"VgroupsInfo\":{\"Num\":\"2\",\"Vgroups\":[{\"VgId\":\"14\",\"HashBegin\":\"0\",\"HashEnd\":\"2147483646\",\"EpSet\":{\"InUse\":\"0\",\"NumOfEps\":\"1\",\"Eps\":[{\"Fqdn\":\"localhost\",\"Port\":\"6030\"}]},\"NumOfTable\":\"0\"},{\"VgId\":\"15\",\"HashBegin\":\"2147483647\",\"HashEnd\":\"4294967295\",\"EpSet\":{\"InUse\":\"0\",\"NumOfEps\":\"1\",\"Eps\":[{\"Fqdn\":\"localhost\",\"Port\":\"6030\"}]},\"NumOfTable\":\"0\"}]}}},\"Tags\":[{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"0\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":false,\"NotReserved\":false,\"IsNull\":true,\"Unit\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"0\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":false,\"NotReserved\":false,\"IsNull\":true,\"Unit\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"0\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":false,\"NotReserved\":false,\"IsNull\":true,\"Unit\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"0\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":false,\"NotReserved\":false,\"IsNull\":true,\"Unit\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"0\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":false,\"NotReserved\":false,\"IsNull\":true,\"Unit\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"0\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":false,\"NotReserved\":false,\"IsNull\":true,\"Unit\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"0\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":false,\"NotReserved\":false,\"IsNull\":true,\"Unit\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"0\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":false,\"NotReserved\":false,\"IsNull\":true,\"Unit\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"0\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":false,\"NotReserved\":false,\"IsNull\":true,\"Unit\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"0\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":false,\"NotReserved\":false,\"IsNull\":true,\"Unit\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"0\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":false,\"NotReserved\":false,\"IsNull\":true,\"Unit\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"0\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":false,\"NotReserved\":false,\"IsNull\":true,\"Unit\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"0\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":false,\"NotReserved\":false,\"IsNull\":true,\"Unit\":\"0\"}}],\"Window\":{\"NodeType\":\"14\",\"Name\":\"IntervalWindow\",\"IntervalWindow\":{\"Interval\":{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"5\",\"Precision\":\"0\",\"Scale\":\"115\",\"Bytes\":\"8\"},\"AliasName\":\"1fd7635317edfeca9054894ac9ef9b5e\",\"UserAlias\":\"14s\",\"LiteralSize\":\"3\",\"Literal\":\"14s\",\"Duration\":true,\"Translate\":true,\"NotReserved\":false,\"IsNull\":false,\"Unit\":\"115\",\"Datum\":\"14000\"}},\"TsPk\":{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"9\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"ts\",\"UserAlias\":\"ts\",\"TableId\":\"5129202035162885657\",\"TableType\":\"1\",\"ColId\":\"1\",\"ProjId\":\"0\",\"ColType\":\"1\",\"DbName\":\"test\",\"TableName\":\"at_once_interval_ext_stb\",\"TableAlias\":\"at_once_interval_ext_stb\",\"ColName\":\"ts\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}}}},\"StmtName\":\"0x150146d14\",\"HasAggFuncs\":true}}"; SNode * pAst = NULL; SQueryPlan *pPlan = NULL; diff --git a/source/libs/planner/src/planSpliter.c b/source/libs/planner/src/planSpliter.c index 1f9c4bcae9..4cd7749b6a 100644 --- a/source/libs/planner/src/planSpliter.c +++ b/source/libs/planner/src/planSpliter.c @@ -488,6 +488,53 @@ static int32_t stbSplCreatePartWindowNode(SWindowLogicNode* pMergeWindow, SLogic return code; } +static int32_t stbSplCreateSemiWindowNode(SWindowLogicNode* pMidWindow, SLogicNode** pSemiWindow) { + SNodeList* pFunc = pMidWindow->pFuncs; + pMidWindow->pFuncs = NULL; + nodesDestroyList(pMidWindow->node.pTargets); + pMidWindow->node.pTargets = NULL; + SNodeList* pChildren = pMidWindow->node.pChildren; + pMidWindow->node.pChildren = NULL; + + SWindowLogicNode* pPartWin = (SWindowLogicNode*)nodesCloneNode((SNode*)pMidWindow); + if (NULL == pPartWin) { + return TSDB_CODE_OUT_OF_MEMORY; + } + + pPartWin->node.pChildren = pChildren; + splSetParent((SLogicNode*)pPartWin); + + int32_t index = 0; + int32_t code = stbSplRewriteFuns(pFunc, &pPartWin->pFuncs, &pMidWindow->pFuncs); + if (TSDB_CODE_SUCCESS == code) { + code = stbSplAppendWStart(pPartWin->pFuncs, &index, ((SColumnNode*)pMidWindow->pTspk)->node.resType.precision); + } + if (TSDB_CODE_SUCCESS == code) { + code = createColumnByRewriteExprs(pPartWin->pFuncs, &pPartWin->node.pTargets); + } + + if (TSDB_CODE_SUCCESS == code) { + code = createColumnByRewriteExprs(pMidWindow->pFuncs, &pMidWindow->node.pTargets); + } + + if (TSDB_CODE_SUCCESS == code) { + nodesDestroyNode(pMidWindow->pTspk); + pMidWindow->pTspk = nodesCloneNode(nodesListGetNode(pPartWin->node.pTargets, index)); + if (NULL == pMidWindow->pTspk) { + code = TSDB_CODE_OUT_OF_MEMORY; + } + } + + nodesDestroyList(pFunc); + if (TSDB_CODE_SUCCESS == code) { + *pSemiWindow = (SLogicNode*)pPartWin; + } else { + nodesDestroyNode((SNode*)pPartWin); + } + + return code; +} + static int32_t stbSplGetNumOfVgroups(SLogicNode* pNode) { if (QUERY_NODE_LOGIC_PLAN_SCAN == nodeType(pNode)) { return ((SScanLogicNode*)pNode)->pVgroupList->numOfVgroups; @@ -663,7 +710,7 @@ static int32_t stbSplSplitIntervalForStream(SSplitContext* pCxt, SStableSplitInf } if (TSDB_CODE_SUCCESS == code) { - code = stbSplCreatePartWindowNode((SWindowLogicNode*)pMidWindow, &pPartWindow); + code = stbSplCreateSemiWindowNode((SWindowLogicNode*)pMidWindow, &pPartWindow); if (TSDB_CODE_SUCCESS == code) { ((SWindowLogicNode*)pPartWindow)->windowAlgo = INTERVAL_ALGO_STREAM_SEMI; code = stbSplCreateExchangeNode(pCxt, pMidWindow, pPartWindow); From 80ee5c1f13d37c26dee2003834ba45590bcf62a7 Mon Sep 17 00:00:00 2001 From: wangmm0220 Date: Wed, 13 Dec 2023 14:51:34 +0800 Subject: [PATCH 15/51] fix:middle agg plan --- source/libs/planner/src/planSpliter.c | 124 +++++++++++++++++++++----- 1 file changed, 103 insertions(+), 21 deletions(-) diff --git a/source/libs/planner/src/planSpliter.c b/source/libs/planner/src/planSpliter.c index 4cd7749b6a..accf5bf100 100644 --- a/source/libs/planner/src/planSpliter.c +++ b/source/libs/planner/src/planSpliter.c @@ -488,53 +488,139 @@ static int32_t stbSplCreatePartWindowNode(SWindowLogicNode* pMergeWindow, SLogic return code; } -static int32_t stbSplCreateSemiWindowNode(SWindowLogicNode* pMidWindow, SLogicNode** pSemiWindow) { - SNodeList* pFunc = pMidWindow->pFuncs; - pMidWindow->pFuncs = NULL; - nodesDestroyList(pMidWindow->node.pTargets); - pMidWindow->node.pTargets = NULL; - SNodeList* pChildren = pMidWindow->node.pChildren; - pMidWindow->node.pChildren = NULL; +static int32_t stbSplCreatePartMidWindowNode(SWindowLogicNode* pMergeWindow, SLogicNode** pPartWindow, SLogicNode** pMidWindow) { + SNodeList* pFunc = pMergeWindow->pFuncs; + pMergeWindow->pFuncs = NULL; + SNodeList* pTargets = pMergeWindow->node.pTargets; + pMergeWindow->node.pTargets = NULL; + SNodeList* pChildren = pMergeWindow->node.pChildren; + pMergeWindow->node.pChildren = NULL; + SNode* pConditions = pMergeWindow->node.pConditions; + pMergeWindow->node.pConditions = NULL; - SWindowLogicNode* pPartWin = (SWindowLogicNode*)nodesCloneNode((SNode*)pMidWindow); + SWindowLogicNode* pPartWin = (SWindowLogicNode*)nodesCloneNode((SNode*)pMergeWindow); if (NULL == pPartWin) { return TSDB_CODE_OUT_OF_MEMORY; } + SWindowLogicNode* pMidWin = (SWindowLogicNode*)nodesCloneNode((SNode*)pMergeWindow); + if (NULL == pMidWin) { + return TSDB_CODE_OUT_OF_MEMORY; + } + + pPartWin->node.groupAction = GROUP_ACTION_KEEP; + pMidWin->node.groupAction = GROUP_ACTION_KEEP; + pMergeWindow->node.pTargets = pTargets; + pMergeWindow->node.pConditions = pConditions; + pPartWin->node.pChildren = pChildren; splSetParent((SLogicNode*)pPartWin); + SNodeList* pFuncPart = NULL; + SNodeList* pFuncMerge = NULL; + int32_t code = stbSplRewriteFuns(pFunc, &pFuncPart, &pFuncMerge); + pPartWin->pFuncs = pFuncPart; + pMergeWindow->pFuncs = pFuncMerge; + pMidWin->pFuncs = nodesCloneList(pFuncMerge); + int32_t index = 0; - int32_t code = stbSplRewriteFuns(pFunc, &pPartWin->pFuncs, &pMidWindow->pFuncs); if (TSDB_CODE_SUCCESS == code) { - code = stbSplAppendWStart(pPartWin->pFuncs, &index, ((SColumnNode*)pMidWindow->pTspk)->node.resType.precision); + code = stbSplAppendWStart(pPartWin->pFuncs, &index, ((SColumnNode*)pMergeWindow->pTspk)->node.resType.precision); } if (TSDB_CODE_SUCCESS == code) { code = createColumnByRewriteExprs(pPartWin->pFuncs, &pPartWin->node.pTargets); } if (TSDB_CODE_SUCCESS == code) { - code = createColumnByRewriteExprs(pMidWindow->pFuncs, &pMidWindow->node.pTargets); + nodesDestroyNode(pMidWin->pTspk); + pMidWin->pTspk = nodesCloneNode(nodesListGetNode(pPartWin->node.pTargets, index)); + if (NULL == pMidWin->pTspk) { + code = TSDB_CODE_OUT_OF_MEMORY; + } + } + + if(TSDB_CODE_SUCCESS == code){ + for (int32_t i = 0; i < LIST_LENGTH(pMidWin->pFuncs); ++i) { + SFunctionNode* pFunc1 = (SFunctionNode*)nodesListGetNode(pPartWin->pFuncs, i); + SFunctionNode* pFunc2 = (SFunctionNode*)nodesListGetNode(pMidWin->pFuncs, i); + strcpy(pFunc2->node.aliasName, pFunc1->node.aliasName); + } } if (TSDB_CODE_SUCCESS == code) { - nodesDestroyNode(pMidWindow->pTspk); - pMidWindow->pTspk = nodesCloneNode(nodesListGetNode(pPartWin->node.pTargets, index)); - if (NULL == pMidWindow->pTspk) { + code = stbSplAppendWStart(pMidWin->pFuncs, &index, ((SColumnNode*)pMergeWindow->pTspk)->node.resType.precision); + } + if (TSDB_CODE_SUCCESS == code) { + code = createColumnByRewriteExprs(pMidWin->pFuncs, &pMidWin->node.pTargets); + } + + if (TSDB_CODE_SUCCESS == code) { + nodesDestroyNode(pMergeWindow->pTspk); + pMergeWindow->pTspk = nodesCloneNode(nodesListGetNode(pMidWin->node.pTargets, index)); + if (NULL == pMergeWindow->pTspk) { code = TSDB_CODE_OUT_OF_MEMORY; } } nodesDestroyList(pFunc); if (TSDB_CODE_SUCCESS == code) { - *pSemiWindow = (SLogicNode*)pPartWin; + *pPartWindow = (SLogicNode*)pPartWin; + *pMidWindow = (SLogicNode*)pMidWin; } else { nodesDestroyNode((SNode*)pPartWin); + nodesDestroyNode((SNode*)pMidWin); } return code; } +//static int32_t stbSplCreateSemiWindowNode(SWindowLogicNode* pMidWindow, SLogicNode** pSemiWindow) { +// SNodeList* pFunc = pMidWindow->pFuncs; +// pMidWindow->pFuncs = NULL; +// nodesDestroyList(pMidWindow->node.pTargets); +// pMidWindow->node.pTargets = NULL; +// SNodeList* pChildren = pMidWindow->node.pChildren; +// pMidWindow->node.pChildren = NULL; +// +// SWindowLogicNode* pPartWin = (SWindowLogicNode*)nodesCloneNode((SNode*)pMidWindow); +// if (NULL == pPartWin) { +// return TSDB_CODE_OUT_OF_MEMORY; +// } +// +// pPartWin->node.pChildren = pChildren; +// splSetParent((SLogicNode*)pPartWin); +// +// int32_t index = 0; +// int32_t code = stbSplRewriteFuns(pFunc, &pPartWin->pFuncs, &pMidWindow->pFuncs); +// if (TSDB_CODE_SUCCESS == code) { +// code = stbSplAppendWStart(pPartWin->pFuncs, &index, ((SColumnNode*)pMidWindow->pTspk)->node.resType.precision); +// } +// if (TSDB_CODE_SUCCESS == code) { +// code = createColumnByRewriteExprs(pPartWin->pFuncs, &pPartWin->node.pTargets); +// } +// +// if (TSDB_CODE_SUCCESS == code) { +// code = createColumnByRewriteExprs(pMidWindow->pFuncs, &pMidWindow->node.pTargets); +// } +// +// if (TSDB_CODE_SUCCESS == code) { +// nodesDestroyNode(pMidWindow->pTspk); +// pMidWindow->pTspk = nodesCloneNode(nodesListGetNode(pPartWin->node.pTargets, index)); +// if (NULL == pMidWindow->pTspk) { +// code = TSDB_CODE_OUT_OF_MEMORY; +// } +// } +// +// nodesDestroyList(pFunc); +// if (TSDB_CODE_SUCCESS == code) { +// *pSemiWindow = (SLogicNode*)pPartWin; +// } else { +// nodesDestroyNode((SNode*)pPartWin); +// } +// +// return code; +//} + static int32_t stbSplGetNumOfVgroups(SLogicNode* pNode) { if (QUERY_NODE_LOGIC_PLAN_SCAN == nodeType(pNode)) { return ((SScanLogicNode*)pNode)->pVgroupList->numOfVgroups; @@ -702,17 +788,13 @@ static int32_t stbSplSplitIntervalForBatch(SSplitContext* pCxt, SStableSplitInfo static int32_t stbSplSplitIntervalForStream(SSplitContext* pCxt, SStableSplitInfo* pInfo) { SLogicNode* pPartWindow = NULL; SLogicNode* pMidWindow = NULL; - int32_t code = stbSplCreatePartWindowNode((SWindowLogicNode*)pInfo->pSplitNode, &pMidWindow); + int32_t code = stbSplCreatePartMidWindowNode((SWindowLogicNode*)pInfo->pSplitNode, &pPartWindow, &pMidWindow); if (TSDB_CODE_SUCCESS == code) { ((SWindowLogicNode*)pMidWindow)->windowAlgo = INTERVAL_ALGO_STREAM_MID; ((SWindowLogicNode*)pInfo->pSplitNode)->windowAlgo = INTERVAL_ALGO_STREAM_FINAL; + ((SWindowLogicNode*)pPartWindow)->windowAlgo = INTERVAL_ALGO_STREAM_SEMI; code = stbSplCreateExchangeNode(pCxt, pInfo->pSplitNode, pMidWindow); - } - - if (TSDB_CODE_SUCCESS == code) { - code = stbSplCreateSemiWindowNode((SWindowLogicNode*)pMidWindow, &pPartWindow); if (TSDB_CODE_SUCCESS == code) { - ((SWindowLogicNode*)pPartWindow)->windowAlgo = INTERVAL_ALGO_STREAM_SEMI; code = stbSplCreateExchangeNode(pCxt, pMidWindow, pPartWindow); } } From 64f98744e2a595adf65c0014143f516925f92aa7 Mon Sep 17 00:00:00 2001 From: wangmm0220 Date: Mon, 18 Dec 2023 10:57:36 +0800 Subject: [PATCH 16/51] fix:set trigger parms = 0 in source task if levelPlan != 1 --- source/dnode/mnode/impl/src/mndScheduler.c | 21 ++++++------ .../mnode/impl/test/stream/streamPlanTest.cpp | 7 ++-- source/libs/planner/src/planSpliter.c | 34 ++++++++++++++++++- 3 files changed, 47 insertions(+), 15 deletions(-) diff --git a/source/dnode/mnode/impl/src/mndScheduler.c b/source/dnode/mnode/impl/src/mndScheduler.c index bf8e9ab4d1..e91253e689 100644 --- a/source/dnode/mnode/impl/src/mndScheduler.c +++ b/source/dnode/mnode/impl/src/mndScheduler.c @@ -260,13 +260,13 @@ static int32_t doAddShuffleSinkTask(SMnode* pMnode, SStreamObj* pStream, SEpSet* return TDB_CODE_SUCCESS; } -static SStreamTask* buildSourceTask(SStreamObj* pStream, SEpSet* pEpset, - int64_t firstWindowSkey, bool isFillhistory) { +static SStreamTask* buildSourceTask(SStreamObj* pStream, SEpSet* pEpset, int64_t firstWindowSkey, + bool isFillhistory, bool useTriggerParam) { uint64_t uid = (isFillhistory) ? pStream->hTaskUid : pStream->uid; SArray** pTaskList = (isFillhistory) ? taosArrayGetLast(pStream->pHTasksList) : taosArrayGetLast(pStream->tasks); SStreamTask* pTask = tNewStreamTask(uid, TASK_LEVEL__SOURCE, - isFillhistory, pStream->conf.triggerParam, + isFillhistory, useTriggerParam ? pStream->conf.triggerParam : 0, *pTaskList, pStream->conf.fillHistory); if (pTask == NULL) { return NULL; @@ -311,11 +311,10 @@ static void setHTasksId(SStreamObj* pStream) { } } -static int32_t doAddSourceTask(SMnode* pMnode, SSubplan* plan, SStreamObj* pStream, - SEpSet* pEpset, int64_t nextWindowSkey, - SVgObj* pVgroup, bool isFillhistory ){ +static int32_t doAddSourceTask(SMnode* pMnode, SSubplan* plan, SStreamObj* pStream, SEpSet* pEpset, + int64_t nextWindowSkey, SVgObj* pVgroup, bool isFillhistory, bool useTriggerParam ){ // new stream task - SStreamTask* pTask = buildSourceTask(pStream, pEpset, nextWindowSkey, isFillhistory); + SStreamTask* pTask = buildSourceTask(pStream, pEpset, nextWindowSkey, isFillhistory, useTriggerParam); if(pTask == NULL){ terrno = TSDB_CODE_OUT_OF_MEMORY; return terrno; @@ -362,7 +361,7 @@ static SSubplan* getAggSubPlan(const SQueryPlan* pPlan, int index){ } static int32_t addSourceTask(SMnode* pMnode, SSubplan* plan, SStreamObj* pStream, - SEpSet* pEpset, int64_t nextWindowSkey) { + SEpSet* pEpset, int64_t nextWindowSkey, bool useTriggerParam) { addNewTaskList(pStream); void* pIter = NULL; @@ -379,14 +378,14 @@ static int32_t addSourceTask(SMnode* pMnode, SSubplan* plan, SStreamObj* pStream continue; } - int code = doAddSourceTask(pMnode, plan, pStream, pEpset, nextWindowSkey, pVgroup, false); + int code = doAddSourceTask(pMnode, plan, pStream, pEpset, nextWindowSkey, pVgroup, false, useTriggerParam); if(code != 0){ sdbRelease(pSdb, pVgroup); return code; } if (pStream->conf.fillHistory) { - code = doAddSourceTask(pMnode, plan, pStream, pEpset, nextWindowSkey, pVgroup, true); + code = doAddSourceTask(pMnode, plan, pStream, pEpset, nextWindowSkey, pVgroup, true, useTriggerParam); if(code != 0){ sdbRelease(pSdb, pVgroup); return code; @@ -580,7 +579,7 @@ static int32_t doScheduleStream(SStreamObj* pStream, SMnode* pMnode, SQueryPlan* if (plan == NULL) { return terrno; } - int32_t code = addSourceTask(pMnode, plan, pStream, pEpset, nextWindowSkey); + int32_t code = addSourceTask(pMnode, plan, pStream, pEpset, nextWindowSkey, numOfPlanLevel == 1); if (code != TSDB_CODE_SUCCESS) { return code; } diff --git a/source/dnode/mnode/impl/test/stream/streamPlanTest.cpp b/source/dnode/mnode/impl/test/stream/streamPlanTest.cpp index 3adc8c9b32..f3666338bc 100644 --- a/source/dnode/mnode/impl/test/stream/streamPlanTest.cpp +++ b/source/dnode/mnode/impl/test/stream/streamPlanTest.cpp @@ -34,8 +34,8 @@ int main(int argc, char **argv) { } TEST(testCase, plan_Test) { -// char* ast = "{\"NodeType\":\"101\",\"Name\":\"SelectStmt\",\"SelectStmt\":{\"Distinct\":false,\"Projections\":[{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"9\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"#expr_1\",\"UserAlias\":\"_wstart\",\"Name\":\"_wstart\",\"Id\":\"89\",\"Type\":\"3505\",\"UdfBufSize\":\"0\"}},{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"5\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"#expr_2\",\"UserAlias\":\"sum(voltage)\",\"Name\":\"sum\",\"Id\":\"1\",\"Type\":\"14\",\"Parameters\":[{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"4\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"4\"},\"AliasName\":\"voltage\",\"UserAlias\":\"voltage\",\"TableId\":\"6555383776122680534\",\"TableType\":\"1\",\"ColId\":\"3\",\"ProjId\":\"0\",\"ColType\":\"1\",\"DbName\":\"test\",\"TableName\":\"meters\",\"TableAlias\":\"meters\",\"ColName\":\"voltage\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"4\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"4\"},\"AliasName\":\"#expr_3\",\"UserAlias\":\"groupid\",\"Name\":\"_group_key\",\"Id\":\"96\",\"Type\":\"3754\",\"Parameters\":[{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"4\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"4\"},\"AliasName\":\"#expr_3\",\"UserAlias\":\"groupid\",\"TableId\":\"6555383776122680534\",\"TableType\":\"1\",\"ColId\":\"5\",\"ProjId\":\"0\",\"ColType\":\"2\",\"DbName\":\"test\",\"TableName\":\"meters\",\"TableAlias\":\"meters\",\"ColName\":\"groupid\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}}],\"UdfBufSize\":\"0\"}}],\"From\":{\"NodeType\":\"6\",\"Name\":\"RealTable\",\"RealTable\":{\"DataType\":{\"Type\":\"0\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"0\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"DbName\":\"test\",\"tableName\":\"meters\",\"tableAlias\":\"meters\",\"MetaSize\":\"475\",\"Meta\":{\"VgId\":\"0\",\"TableType\":\"1\",\"Uid\":\"6555383776122680534\",\"Suid\":\"6555383776122680534\",\"Sversion\":\"1\",\"Tversion\":\"1\",\"ComInfo\":{\"NumOfTags\":\"2\",\"Precision\":\"0\",\"NumOfColumns\":\"4\",\"RowSize\":\"20\"},\"ColSchemas\":[{\"Type\":\"9\",\"ColId\":\"1\",\"bytes\":\"8\",\"Name\":\"ts\"},{\"Type\":\"6\",\"ColId\":\"2\",\"bytes\":\"4\",\"Name\":\"current\"},{\"Type\":\"4\",\"ColId\":\"3\",\"bytes\":\"4\",\"Name\":\"voltage\"},{\"Type\":\"6\",\"ColId\":\"4\",\"bytes\":\"4\",\"Name\":\"phase\"},{\"Type\":\"4\",\"ColId\":\"5\",\"bytes\":\"4\",\"Name\":\"groupid\"},{\"Type\":\"8\",\"ColId\":\"6\",\"bytes\":\"26\",\"Name\":\"location\"}]},\"VgroupsInfoSize\":\"1340\",\"VgroupsInfo\":{\"Num\":\"2\",\"Vgroups\":[{\"VgId\":\"2\",\"HashBegin\":\"0\",\"HashEnd\":\"2147483646\",\"EpSet\":{\"InUse\":\"0\",\"NumOfEps\":\"1\",\"Eps\":[{\"Fqdn\":\"localhost\",\"Port\":\"6030\"}]},\"NumOfTable\":\"0\"},{\"VgId\":\"3\",\"HashBegin\":\"2147483647\",\"HashEnd\":\"4294967295\",\"EpSet\":{\"InUse\":\"0\",\"NumOfEps\":\"1\",\"Eps\":[{\"Fqdn\":\"localhost\",\"Port\":\"6030\"}]},\"NumOfTable\":\"0\"}]}}},\"PartitionBy\":[{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"4\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"4\"},\"AliasName\":\"groupid\",\"UserAlias\":\"groupid\",\"TableId\":\"6555383776122680534\",\"TableType\":\"1\",\"ColId\":\"5\",\"ProjId\":\"0\",\"ColType\":\"2\",\"DbName\":\"test\",\"TableName\":\"meters\",\"TableAlias\":\"meters\",\"ColName\":\"groupid\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}}],\"Window\":{\"NodeType\":\"14\",\"Name\":\"IntervalWindow\",\"IntervalWindow\":{\"Interval\":{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"5\",\"Precision\":\"0\",\"Scale\":\"115\",\"Bytes\":\"8\"},\"AliasName\":\"c804c3a15ebe05b5baf40ad5ee12be1f\",\"UserAlias\":\"2s\",\"LiteralSize\":\"2\",\"Literal\":\"2s\",\"Duration\":true,\"Translate\":true,\"NotReserved\":false,\"IsNull\":false,\"Unit\":\"115\",\"Datum\":\"2000\"}},\"TsPk\":{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"9\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"ts\",\"UserAlias\":\"ts\",\"TableId\":\"6555383776122680534\",\"TableType\":\"1\",\"ColId\":\"1\",\"ProjId\":\"0\",\"ColType\":\"1\",\"DbName\":\"test\",\"TableName\":\"meters\",\"TableAlias\":\"meters\",\"ColName\":\"ts\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}}}},\"StmtName\":\"0x1580095ba\",\"HasAggFuncs\":true}}"; - char* ast = "{\"NodeType\":\"101\",\"Name\":\"SelectStmt\",\"SelectStmt\":{\"Distinct\":false,\"Projections\":[{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"9\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"#expr_1\",\"UserAlias\":\"wstart\",\"Name\":\"_wstart\",\"Id\":\"89\",\"Type\":\"3505\",\"UdfBufSize\":\"0\"}},{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"2\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"#expr_2\",\"UserAlias\":\"min(c1)\",\"Name\":\"min\",\"Id\":\"2\",\"Type\":\"8\",\"Parameters\":[{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"2\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"c1\",\"UserAlias\":\"c1\",\"TableId\":\"5129202035162885657\",\"TableType\":\"1\",\"ColId\":\"2\",\"ProjId\":\"0\",\"ColType\":\"1\",\"DbName\":\"test\",\"TableName\":\"at_once_interval_ext_stb\",\"TableAlias\":\"at_once_interval_ext_stb\",\"ColName\":\"c1\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"3\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"2\"},\"AliasName\":\"#expr_3\",\"UserAlias\":\"max(c2)\",\"Name\":\"max\",\"Id\":\"3\",\"Type\":\"7\",\"Parameters\":[{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"3\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"2\"},\"AliasName\":\"c2\",\"UserAlias\":\"c2\",\"TableId\":\"5129202035162885657\",\"TableType\":\"1\",\"ColId\":\"3\",\"ProjId\":\"0\",\"ColType\":\"1\",\"DbName\":\"test\",\"TableName\":\"at_once_interval_ext_stb\",\"TableAlias\":\"at_once_interval_ext_stb\",\"ColName\":\"c2\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"4\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"4\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"Name\":\"cast\",\"Id\":\"77\",\"Type\":\"2000\",\"Parameters\":[{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"5\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"#expr_4\",\"UserAlias\":\"sum(c3)\",\"Name\":\"sum\",\"Id\":\"1\",\"Type\":\"14\",\"Parameters\":[{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"4\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"4\"},\"AliasName\":\"c3\",\"UserAlias\":\"c3\",\"TableId\":\"5129202035162885657\",\"TableType\":\"1\",\"ColId\":\"4\",\"ProjId\":\"0\",\"ColType\":\"1\",\"DbName\":\"test\",\"TableName\":\"at_once_interval_ext_stb\",\"TableAlias\":\"at_once_interval_ext_stb\",\"ColName\":\"c3\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"2\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":true,\"NotReserved\":true,\"IsNull\":false,\"Unit\":\"0\",\"Datum\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"5\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"#expr_5\",\"UserAlias\":\"first(c4)\",\"Name\":\"first\",\"Id\":\"33\",\"Type\":\"504\",\"Parameters\":[{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"5\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"c4\",\"UserAlias\":\"c4\",\"TableId\":\"5129202035162885657\",\"TableType\":\"1\",\"ColId\":\"5\",\"ProjId\":\"0\",\"ColType\":\"1\",\"DbName\":\"test\",\"TableName\":\"at_once_interval_ext_stb\",\"TableAlias\":\"at_once_interval_ext_stb\",\"ColName\":\"c4\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}},{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"9\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"ts\",\"UserAlias\":\"ts\",\"TableId\":\"5129202035162885657\",\"TableType\":\"1\",\"ColId\":\"1\",\"ProjId\":\"0\",\"ColType\":\"1\",\"DbName\":\"test\",\"TableName\":\"at_once_interval_ext_stb\",\"TableAlias\":\"at_once_interval_ext_stb\",\"ColName\":\"ts\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"11\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"#expr_6\",\"UserAlias\":\"last(c5)\",\"Name\":\"last\",\"Id\":\"36\",\"Type\":\"506\",\"Parameters\":[{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"11\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"c5\",\"UserAlias\":\"c5\",\"TableId\":\"5129202035162885657\",\"TableType\":\"1\",\"ColId\":\"6\",\"ProjId\":\"0\",\"ColType\":\"1\",\"DbName\":\"test\",\"TableName\":\"at_once_interval_ext_stb\",\"TableAlias\":\"at_once_interval_ext_stb\",\"ColName\":\"c5\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}},{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"9\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"ts\",\"UserAlias\":\"ts\",\"TableId\":\"5129202035162885657\",\"TableType\":\"1\",\"ColId\":\"1\",\"ProjId\":\"0\",\"ColType\":\"1\",\"DbName\":\"test\",\"TableName\":\"at_once_interval_ext_stb\",\"TableAlias\":\"at_once_interval_ext_stb\",\"ColName\":\"ts\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"12\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"2\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"Name\":\"cast\",\"Id\":\"77\",\"Type\":\"2000\",\"Parameters\":[{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"7\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"#expr_7\",\"UserAlias\":\"apercentile(c6, 50)\",\"Name\":\"apercentile\",\"Id\":\"12\",\"Type\":\"1\",\"Parameters\":[{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"12\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"2\"},\"AliasName\":\"c6\",\"UserAlias\":\"c6\",\"TableId\":\"5129202035162885657\",\"TableType\":\"1\",\"ColId\":\"7\",\"ProjId\":\"0\",\"ColType\":\"1\",\"DbName\":\"test\",\"TableName\":\"at_once_interval_ext_stb\",\"TableAlias\":\"at_once_interval_ext_stb\",\"ColName\":\"c6\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"5\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"c0c7c76d30bd3dcaefc96f40275bdc0a\",\"UserAlias\":\"50\",\"LiteralSize\":\"2\",\"Literal\":\"50\",\"Duration\":false,\"Translate\":true,\"NotReserved\":true,\"IsNull\":false,\"Unit\":\"0\",\"Datum\":\"50\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"2\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":true,\"NotReserved\":true,\"IsNull\":false,\"Unit\":\"0\",\"Datum\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"13\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"4\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"Name\":\"cast\",\"Id\":\"77\",\"Type\":\"2000\",\"Parameters\":[{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"7\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"#expr_8\",\"UserAlias\":\"avg(c7)\",\"Name\":\"avg\",\"Id\":\"8\",\"Type\":\"2\",\"Parameters\":[{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"13\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"4\"},\"AliasName\":\"c7\",\"UserAlias\":\"c7\",\"TableId\":\"5129202035162885657\",\"TableType\":\"1\",\"ColId\":\"8\",\"ProjId\":\"0\",\"ColType\":\"1\",\"DbName\":\"test\",\"TableName\":\"at_once_interval_ext_stb\",\"TableAlias\":\"at_once_interval_ext_stb\",\"ColName\":\"c7\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"2\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":true,\"NotReserved\":true,\"IsNull\":false,\"Unit\":\"0\",\"Datum\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"14\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"Name\":\"cast\",\"Id\":\"77\",\"Type\":\"2000\",\"Parameters\":[{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"5\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"#expr_9\",\"UserAlias\":\"count(c8)\",\"Name\":\"count\",\"Id\":\"0\",\"Type\":\"3\",\"Parameters\":[{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"14\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"c8\",\"UserAlias\":\"c8\",\"TableId\":\"5129202035162885657\",\"TableType\":\"1\",\"ColId\":\"9\",\"ProjId\":\"0\",\"ColType\":\"1\",\"DbName\":\"test\",\"TableName\":\"at_once_interval_ext_stb\",\"TableAlias\":\"at_once_interval_ext_stb\",\"ColName\":\"c8\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"2\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":true,\"NotReserved\":true,\"IsNull\":false,\"Unit\":\"0\",\"Datum\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"6\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"4\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"Name\":\"cast\",\"Id\":\"77\",\"Type\":\"2000\",\"Parameters\":[{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"7\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"#expr_10\",\"UserAlias\":\"spread(c1)\",\"Name\":\"spread\",\"Id\":\"17\",\"Type\":\"11\",\"Parameters\":[{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"2\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"c1\",\"UserAlias\":\"c1\",\"TableId\":\"5129202035162885657\",\"TableType\":\"1\",\"ColId\":\"2\",\"ProjId\":\"0\",\"ColType\":\"1\",\"DbName\":\"test\",\"TableName\":\"at_once_interval_ext_stb\",\"TableAlias\":\"at_once_interval_ext_stb\",\"ColName\":\"c1\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"2\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":true,\"NotReserved\":true,\"IsNull\":false,\"Unit\":\"0\",\"Datum\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"7\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"#expr_11\",\"UserAlias\":\"stddev(c2)\",\"Name\":\"stddev\",\"Id\":\"4\",\"Type\":\"12\",\"Parameters\":[{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"3\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"2\"},\"AliasName\":\"c2\",\"UserAlias\":\"c2\",\"TableId\":\"5129202035162885657\",\"TableType\":\"1\",\"ColId\":\"3\",\"ProjId\":\"0\",\"ColType\":\"1\",\"DbName\":\"test\",\"TableName\":\"at_once_interval_ext_stb\",\"TableAlias\":\"at_once_interval_ext_stb\",\"ColName\":\"c2\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"8\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"Name\":\"cast\",\"Id\":\"77\",\"Type\":\"2000\",\"Parameters\":[{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"5\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"#expr_12\",\"UserAlias\":\"hyperloglog(c11)\",\"Name\":\"hyperloglog\",\"Id\":\"43\",\"Type\":\"17\",\"Parameters\":[{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"8\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"c11\",\"UserAlias\":\"c11\",\"TableId\":\"5129202035162885657\",\"TableType\":\"1\",\"ColId\":\"12\",\"ProjId\":\"0\",\"ColType\":\"1\",\"DbName\":\"test\",\"TableName\":\"at_once_interval_ext_stb\",\"TableAlias\":\"at_once_interval_ext_stb\",\"ColName\":\"c11\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"2\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":true,\"NotReserved\":true,\"IsNull\":false,\"Unit\":\"0\",\"Datum\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"10\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"26\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"Name\":\"cast\",\"Id\":\"77\",\"Type\":\"2000\",\"Parameters\":[{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"5\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"#expr_13\",\"UserAlias\":\"timediff(1, 0, 1h)\",\"Name\":\"timediff\",\"Id\":\"81\",\"Type\":\"2501\",\"Parameters\":[{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"5\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"c4ca4238a0b923820dcc509a6f75849b\",\"UserAlias\":\"1\",\"LiteralSize\":\"1\",\"Literal\":\"1\",\"Duration\":false,\"Translate\":true,\"NotReserved\":false,\"IsNull\":false,\"Unit\":\"0\",\"Datum\":\"1\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"5\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"cfcd208495d565ef66e7dff9f98764da\",\"UserAlias\":\"0\",\"LiteralSize\":\"1\",\"Literal\":\"0\",\"Duration\":false,\"Translate\":true,\"NotReserved\":false,\"IsNull\":false,\"Unit\":\"0\",\"Datum\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"5\",\"Precision\":\"0\",\"Scale\":\"104\",\"Bytes\":\"8\"},\"AliasName\":\"7c68645d71b803bf0ba2f22519f73e08\",\"UserAlias\":\"1h\",\"LiteralSize\":\"2\",\"Literal\":\"1h\",\"Duration\":true,\"Translate\":true,\"NotReserved\":false,\"IsNull\":false,\"Unit\":\"104\",\"Datum\":\"3600000\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"2\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":true,\"NotReserved\":true,\"IsNull\":false,\"Unit\":\"0\",\"Datum\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"2\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":true,\"NotReserved\":true,\"IsNull\":false,\"Unit\":\"0\",\"Datum\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"1\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"Name\":\"cast\",\"Id\":\"77\",\"Type\":\"2000\",\"Parameters\":[{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"8\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"96\"},\"AliasName\":\"#expr_14\",\"UserAlias\":\"timezone()\",\"Name\":\"timezone\",\"Id\":\"84\",\"Type\":\"2503\",\"UdfBufSize\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"2\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":true,\"NotReserved\":true,\"IsNull\":false,\"Unit\":\"0\",\"Datum\":\"0\"}}],\"UdfBufSize\":\"0\"}}],\"From\":{\"NodeType\":\"6\",\"Name\":\"RealTable\",\"RealTable\":{\"DataType\":{\"Type\":\"0\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"0\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"DbName\":\"test\",\"tableName\":\"at_once_interval_ext_stb\",\"tableAlias\":\"at_once_interval_ext_stb\",\"MetaSize\":\"2008\",\"Meta\":{\"VgId\":\"0\",\"TableType\":\"1\",\"Uid\":\"5129202035162885657\",\"Suid\":\"5129202035162885657\",\"Sversion\":\"1\",\"Tversion\":\"1\",\"ComInfo\":{\"NumOfTags\":\"13\",\"Precision\":\"0\",\"NumOfColumns\":\"14\",\"RowSize\":\"85\"},\"ColSchemas\":[{\"Type\":\"9\",\"ColId\":\"1\",\"bytes\":\"8\",\"Name\":\"ts\"},{\"Type\":\"2\",\"ColId\":\"2\",\"bytes\":\"1\",\"Name\":\"c1\"},{\"Type\":\"3\",\"ColId\":\"3\",\"bytes\":\"2\",\"Name\":\"c2\"},{\"Type\":\"4\",\"ColId\":\"4\",\"bytes\":\"4\",\"Name\":\"c3\"},{\"Type\":\"5\",\"ColId\":\"5\",\"bytes\":\"8\",\"Name\":\"c4\"},{\"Type\":\"11\",\"ColId\":\"6\",\"bytes\":\"1\",\"Name\":\"c5\"},{\"Type\":\"12\",\"ColId\":\"7\",\"bytes\":\"2\",\"Name\":\"c6\"},{\"Type\":\"13\",\"ColId\":\"8\",\"bytes\":\"4\",\"Name\":\"c7\"},{\"Type\":\"14\",\"ColId\":\"9\",\"bytes\":\"8\",\"Name\":\"c8\"},{\"Type\":\"6\",\"ColId\":\"10\",\"bytes\":\"4\",\"Name\":\"c9\"},{\"Type\":\"7\",\"ColId\":\"11\",\"bytes\":\"8\",\"Name\":\"c10\"},{\"Type\":\"8\",\"ColId\":\"12\",\"bytes\":\"8\",\"Name\":\"c11\"},{\"Type\":\"10\",\"ColId\":\"13\",\"bytes\":\"26\",\"Name\":\"c12\"},{\"Type\":\"1\",\"ColId\":\"14\",\"bytes\":\"1\",\"Name\":\"c13\"},{\"Type\":\"2\",\"ColId\":\"15\",\"bytes\":\"1\",\"Name\":\"t1\"},{\"Type\":\"3\",\"ColId\":\"16\",\"bytes\":\"2\",\"Name\":\"t2\"},{\"Type\":\"4\",\"ColId\":\"17\",\"bytes\":\"4\",\"Name\":\"t3\"},{\"Type\":\"5\",\"ColId\":\"18\",\"bytes\":\"8\",\"Name\":\"t4\"},{\"Type\":\"11\",\"ColId\":\"19\",\"bytes\":\"1\",\"Name\":\"t5\"},{\"Type\":\"12\",\"ColId\":\"20\",\"bytes\":\"2\",\"Name\":\"t6\"},{\"Type\":\"13\",\"ColId\":\"21\",\"bytes\":\"4\",\"Name\":\"t7\"},{\"Type\":\"14\",\"ColId\":\"22\",\"bytes\":\"8\",\"Name\":\"t8\"},{\"Type\":\"6\",\"ColId\":\"23\",\"bytes\":\"4\",\"Name\":\"t9\"},{\"Type\":\"7\",\"ColId\":\"24\",\"bytes\":\"8\",\"Name\":\"t10\"},{\"Type\":\"8\",\"ColId\":\"25\",\"bytes\":\"8\",\"Name\":\"t11\"},{\"Type\":\"10\",\"ColId\":\"26\",\"bytes\":\"26\",\"Name\":\"t12\"},{\"Type\":\"1\",\"ColId\":\"27\",\"bytes\":\"1\",\"Name\":\"t13\"}]},\"VgroupsInfoSize\":\"1340\",\"VgroupsInfo\":{\"Num\":\"2\",\"Vgroups\":[{\"VgId\":\"14\",\"HashBegin\":\"0\",\"HashEnd\":\"2147483646\",\"EpSet\":{\"InUse\":\"0\",\"NumOfEps\":\"1\",\"Eps\":[{\"Fqdn\":\"localhost\",\"Port\":\"6030\"}]},\"NumOfTable\":\"0\"},{\"VgId\":\"15\",\"HashBegin\":\"2147483647\",\"HashEnd\":\"4294967295\",\"EpSet\":{\"InUse\":\"0\",\"NumOfEps\":\"1\",\"Eps\":[{\"Fqdn\":\"localhost\",\"Port\":\"6030\"}]},\"NumOfTable\":\"0\"}]}}},\"Tags\":[{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"0\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":false,\"NotReserved\":false,\"IsNull\":true,\"Unit\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"0\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":false,\"NotReserved\":false,\"IsNull\":true,\"Unit\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"0\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":false,\"NotReserved\":false,\"IsNull\":true,\"Unit\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"0\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":false,\"NotReserved\":false,\"IsNull\":true,\"Unit\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"0\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":false,\"NotReserved\":false,\"IsNull\":true,\"Unit\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"0\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":false,\"NotReserved\":false,\"IsNull\":true,\"Unit\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"0\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":false,\"NotReserved\":false,\"IsNull\":true,\"Unit\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"0\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":false,\"NotReserved\":false,\"IsNull\":true,\"Unit\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"0\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":false,\"NotReserved\":false,\"IsNull\":true,\"Unit\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"0\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":false,\"NotReserved\":false,\"IsNull\":true,\"Unit\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"0\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":false,\"NotReserved\":false,\"IsNull\":true,\"Unit\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"0\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":false,\"NotReserved\":false,\"IsNull\":true,\"Unit\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"0\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":false,\"NotReserved\":false,\"IsNull\":true,\"Unit\":\"0\"}}],\"Window\":{\"NodeType\":\"14\",\"Name\":\"IntervalWindow\",\"IntervalWindow\":{\"Interval\":{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"5\",\"Precision\":\"0\",\"Scale\":\"115\",\"Bytes\":\"8\"},\"AliasName\":\"1fd7635317edfeca9054894ac9ef9b5e\",\"UserAlias\":\"14s\",\"LiteralSize\":\"3\",\"Literal\":\"14s\",\"Duration\":true,\"Translate\":true,\"NotReserved\":false,\"IsNull\":false,\"Unit\":\"115\",\"Datum\":\"14000\"}},\"TsPk\":{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"9\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"ts\",\"UserAlias\":\"ts\",\"TableId\":\"5129202035162885657\",\"TableType\":\"1\",\"ColId\":\"1\",\"ProjId\":\"0\",\"ColType\":\"1\",\"DbName\":\"test\",\"TableName\":\"at_once_interval_ext_stb\",\"TableAlias\":\"at_once_interval_ext_stb\",\"ColName\":\"ts\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}}}},\"StmtName\":\"0x150146d14\",\"HasAggFuncs\":true}}"; + char* ast = "{\"NodeType\":\"101\",\"Name\":\"SelectStmt\",\"SelectStmt\":{\"Distinct\":false,\"Projections\":[{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"9\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"#expr_1\",\"UserAlias\":\"_wstart\",\"Name\":\"_wstart\",\"Id\":\"89\",\"Type\":\"3505\",\"UdfBufSize\":\"0\"}},{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"5\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"#expr_2\",\"UserAlias\":\"sum(voltage)\",\"Name\":\"sum\",\"Id\":\"1\",\"Type\":\"14\",\"Parameters\":[{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"4\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"4\"},\"AliasName\":\"voltage\",\"UserAlias\":\"voltage\",\"TableId\":\"6555383776122680534\",\"TableType\":\"1\",\"ColId\":\"3\",\"ProjId\":\"0\",\"ColType\":\"1\",\"DbName\":\"test\",\"TableName\":\"meters\",\"TableAlias\":\"meters\",\"ColName\":\"voltage\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"4\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"4\"},\"AliasName\":\"#expr_3\",\"UserAlias\":\"groupid\",\"Name\":\"_group_key\",\"Id\":\"96\",\"Type\":\"3754\",\"Parameters\":[{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"4\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"4\"},\"AliasName\":\"#expr_3\",\"UserAlias\":\"groupid\",\"TableId\":\"6555383776122680534\",\"TableType\":\"1\",\"ColId\":\"5\",\"ProjId\":\"0\",\"ColType\":\"2\",\"DbName\":\"test\",\"TableName\":\"meters\",\"TableAlias\":\"meters\",\"ColName\":\"groupid\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}}],\"UdfBufSize\":\"0\"}}],\"From\":{\"NodeType\":\"6\",\"Name\":\"RealTable\",\"RealTable\":{\"DataType\":{\"Type\":\"0\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"0\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"DbName\":\"test\",\"tableName\":\"meters\",\"tableAlias\":\"meters\",\"MetaSize\":\"475\",\"Meta\":{\"VgId\":\"0\",\"TableType\":\"1\",\"Uid\":\"6555383776122680534\",\"Suid\":\"6555383776122680534\",\"Sversion\":\"1\",\"Tversion\":\"1\",\"ComInfo\":{\"NumOfTags\":\"2\",\"Precision\":\"0\",\"NumOfColumns\":\"4\",\"RowSize\":\"20\"},\"ColSchemas\":[{\"Type\":\"9\",\"ColId\":\"1\",\"bytes\":\"8\",\"Name\":\"ts\"},{\"Type\":\"6\",\"ColId\":\"2\",\"bytes\":\"4\",\"Name\":\"current\"},{\"Type\":\"4\",\"ColId\":\"3\",\"bytes\":\"4\",\"Name\":\"voltage\"},{\"Type\":\"6\",\"ColId\":\"4\",\"bytes\":\"4\",\"Name\":\"phase\"},{\"Type\":\"4\",\"ColId\":\"5\",\"bytes\":\"4\",\"Name\":\"groupid\"},{\"Type\":\"8\",\"ColId\":\"6\",\"bytes\":\"26\",\"Name\":\"location\"}]},\"VgroupsInfoSize\":\"1340\",\"VgroupsInfo\":{\"Num\":\"2\",\"Vgroups\":[{\"VgId\":\"2\",\"HashBegin\":\"0\",\"HashEnd\":\"2147483646\",\"EpSet\":{\"InUse\":\"0\",\"NumOfEps\":\"1\",\"Eps\":[{\"Fqdn\":\"localhost\",\"Port\":\"6030\"}]},\"NumOfTable\":\"0\"},{\"VgId\":\"3\",\"HashBegin\":\"2147483647\",\"HashEnd\":\"4294967295\",\"EpSet\":{\"InUse\":\"0\",\"NumOfEps\":\"1\",\"Eps\":[{\"Fqdn\":\"localhost\",\"Port\":\"6030\"}]},\"NumOfTable\":\"0\"}]}}},\"PartitionBy\":[{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"4\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"4\"},\"AliasName\":\"groupid\",\"UserAlias\":\"groupid\",\"TableId\":\"6555383776122680534\",\"TableType\":\"1\",\"ColId\":\"5\",\"ProjId\":\"0\",\"ColType\":\"2\",\"DbName\":\"test\",\"TableName\":\"meters\",\"TableAlias\":\"meters\",\"ColName\":\"groupid\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}}],\"Window\":{\"NodeType\":\"14\",\"Name\":\"IntervalWindow\",\"IntervalWindow\":{\"Interval\":{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"5\",\"Precision\":\"0\",\"Scale\":\"115\",\"Bytes\":\"8\"},\"AliasName\":\"c804c3a15ebe05b5baf40ad5ee12be1f\",\"UserAlias\":\"2s\",\"LiteralSize\":\"2\",\"Literal\":\"2s\",\"Duration\":true,\"Translate\":true,\"NotReserved\":false,\"IsNull\":false,\"Unit\":\"115\",\"Datum\":\"2000\"}},\"TsPk\":{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"9\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"ts\",\"UserAlias\":\"ts\",\"TableId\":\"6555383776122680534\",\"TableType\":\"1\",\"ColId\":\"1\",\"ProjId\":\"0\",\"ColType\":\"1\",\"DbName\":\"test\",\"TableName\":\"meters\",\"TableAlias\":\"meters\",\"ColName\":\"ts\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}}}},\"StmtName\":\"0x1580095ba\",\"HasAggFuncs\":true}}"; +// char* ast = "{\"NodeType\":\"101\",\"Name\":\"SelectStmt\",\"SelectStmt\":{\"Distinct\":false,\"Projections\":[{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"9\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"#expr_1\",\"UserAlias\":\"wstart\",\"Name\":\"_wstart\",\"Id\":\"89\",\"Type\":\"3505\",\"UdfBufSize\":\"0\"}},{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"2\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"#expr_2\",\"UserAlias\":\"min(c1)\",\"Name\":\"min\",\"Id\":\"2\",\"Type\":\"8\",\"Parameters\":[{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"2\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"c1\",\"UserAlias\":\"c1\",\"TableId\":\"5129202035162885657\",\"TableType\":\"1\",\"ColId\":\"2\",\"ProjId\":\"0\",\"ColType\":\"1\",\"DbName\":\"test\",\"TableName\":\"at_once_interval_ext_stb\",\"TableAlias\":\"at_once_interval_ext_stb\",\"ColName\":\"c1\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"3\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"2\"},\"AliasName\":\"#expr_3\",\"UserAlias\":\"max(c2)\",\"Name\":\"max\",\"Id\":\"3\",\"Type\":\"7\",\"Parameters\":[{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"3\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"2\"},\"AliasName\":\"c2\",\"UserAlias\":\"c2\",\"TableId\":\"5129202035162885657\",\"TableType\":\"1\",\"ColId\":\"3\",\"ProjId\":\"0\",\"ColType\":\"1\",\"DbName\":\"test\",\"TableName\":\"at_once_interval_ext_stb\",\"TableAlias\":\"at_once_interval_ext_stb\",\"ColName\":\"c2\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"4\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"4\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"Name\":\"cast\",\"Id\":\"77\",\"Type\":\"2000\",\"Parameters\":[{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"5\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"#expr_4\",\"UserAlias\":\"sum(c3)\",\"Name\":\"sum\",\"Id\":\"1\",\"Type\":\"14\",\"Parameters\":[{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"4\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"4\"},\"AliasName\":\"c3\",\"UserAlias\":\"c3\",\"TableId\":\"5129202035162885657\",\"TableType\":\"1\",\"ColId\":\"4\",\"ProjId\":\"0\",\"ColType\":\"1\",\"DbName\":\"test\",\"TableName\":\"at_once_interval_ext_stb\",\"TableAlias\":\"at_once_interval_ext_stb\",\"ColName\":\"c3\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"2\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":true,\"NotReserved\":true,\"IsNull\":false,\"Unit\":\"0\",\"Datum\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"5\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"#expr_5\",\"UserAlias\":\"first(c4)\",\"Name\":\"first\",\"Id\":\"33\",\"Type\":\"504\",\"Parameters\":[{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"5\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"c4\",\"UserAlias\":\"c4\",\"TableId\":\"5129202035162885657\",\"TableType\":\"1\",\"ColId\":\"5\",\"ProjId\":\"0\",\"ColType\":\"1\",\"DbName\":\"test\",\"TableName\":\"at_once_interval_ext_stb\",\"TableAlias\":\"at_once_interval_ext_stb\",\"ColName\":\"c4\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}},{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"9\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"ts\",\"UserAlias\":\"ts\",\"TableId\":\"5129202035162885657\",\"TableType\":\"1\",\"ColId\":\"1\",\"ProjId\":\"0\",\"ColType\":\"1\",\"DbName\":\"test\",\"TableName\":\"at_once_interval_ext_stb\",\"TableAlias\":\"at_once_interval_ext_stb\",\"ColName\":\"ts\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"11\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"#expr_6\",\"UserAlias\":\"last(c5)\",\"Name\":\"last\",\"Id\":\"36\",\"Type\":\"506\",\"Parameters\":[{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"11\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"c5\",\"UserAlias\":\"c5\",\"TableId\":\"5129202035162885657\",\"TableType\":\"1\",\"ColId\":\"6\",\"ProjId\":\"0\",\"ColType\":\"1\",\"DbName\":\"test\",\"TableName\":\"at_once_interval_ext_stb\",\"TableAlias\":\"at_once_interval_ext_stb\",\"ColName\":\"c5\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}},{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"9\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"ts\",\"UserAlias\":\"ts\",\"TableId\":\"5129202035162885657\",\"TableType\":\"1\",\"ColId\":\"1\",\"ProjId\":\"0\",\"ColType\":\"1\",\"DbName\":\"test\",\"TableName\":\"at_once_interval_ext_stb\",\"TableAlias\":\"at_once_interval_ext_stb\",\"ColName\":\"ts\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"12\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"2\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"Name\":\"cast\",\"Id\":\"77\",\"Type\":\"2000\",\"Parameters\":[{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"7\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"#expr_7\",\"UserAlias\":\"apercentile(c6, 50)\",\"Name\":\"apercentile\",\"Id\":\"12\",\"Type\":\"1\",\"Parameters\":[{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"12\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"2\"},\"AliasName\":\"c6\",\"UserAlias\":\"c6\",\"TableId\":\"5129202035162885657\",\"TableType\":\"1\",\"ColId\":\"7\",\"ProjId\":\"0\",\"ColType\":\"1\",\"DbName\":\"test\",\"TableName\":\"at_once_interval_ext_stb\",\"TableAlias\":\"at_once_interval_ext_stb\",\"ColName\":\"c6\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"5\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"c0c7c76d30bd3dcaefc96f40275bdc0a\",\"UserAlias\":\"50\",\"LiteralSize\":\"2\",\"Literal\":\"50\",\"Duration\":false,\"Translate\":true,\"NotReserved\":true,\"IsNull\":false,\"Unit\":\"0\",\"Datum\":\"50\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"2\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":true,\"NotReserved\":true,\"IsNull\":false,\"Unit\":\"0\",\"Datum\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"13\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"4\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"Name\":\"cast\",\"Id\":\"77\",\"Type\":\"2000\",\"Parameters\":[{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"7\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"#expr_8\",\"UserAlias\":\"avg(c7)\",\"Name\":\"avg\",\"Id\":\"8\",\"Type\":\"2\",\"Parameters\":[{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"13\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"4\"},\"AliasName\":\"c7\",\"UserAlias\":\"c7\",\"TableId\":\"5129202035162885657\",\"TableType\":\"1\",\"ColId\":\"8\",\"ProjId\":\"0\",\"ColType\":\"1\",\"DbName\":\"test\",\"TableName\":\"at_once_interval_ext_stb\",\"TableAlias\":\"at_once_interval_ext_stb\",\"ColName\":\"c7\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"2\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":true,\"NotReserved\":true,\"IsNull\":false,\"Unit\":\"0\",\"Datum\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"14\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"Name\":\"cast\",\"Id\":\"77\",\"Type\":\"2000\",\"Parameters\":[{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"5\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"#expr_9\",\"UserAlias\":\"count(c8)\",\"Name\":\"count\",\"Id\":\"0\",\"Type\":\"3\",\"Parameters\":[{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"14\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"c8\",\"UserAlias\":\"c8\",\"TableId\":\"5129202035162885657\",\"TableType\":\"1\",\"ColId\":\"9\",\"ProjId\":\"0\",\"ColType\":\"1\",\"DbName\":\"test\",\"TableName\":\"at_once_interval_ext_stb\",\"TableAlias\":\"at_once_interval_ext_stb\",\"ColName\":\"c8\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"2\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":true,\"NotReserved\":true,\"IsNull\":false,\"Unit\":\"0\",\"Datum\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"6\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"4\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"Name\":\"cast\",\"Id\":\"77\",\"Type\":\"2000\",\"Parameters\":[{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"7\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"#expr_10\",\"UserAlias\":\"spread(c1)\",\"Name\":\"spread\",\"Id\":\"17\",\"Type\":\"11\",\"Parameters\":[{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"2\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"c1\",\"UserAlias\":\"c1\",\"TableId\":\"5129202035162885657\",\"TableType\":\"1\",\"ColId\":\"2\",\"ProjId\":\"0\",\"ColType\":\"1\",\"DbName\":\"test\",\"TableName\":\"at_once_interval_ext_stb\",\"TableAlias\":\"at_once_interval_ext_stb\",\"ColName\":\"c1\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"2\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":true,\"NotReserved\":true,\"IsNull\":false,\"Unit\":\"0\",\"Datum\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"7\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"#expr_11\",\"UserAlias\":\"stddev(c2)\",\"Name\":\"stddev\",\"Id\":\"4\",\"Type\":\"12\",\"Parameters\":[{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"3\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"2\"},\"AliasName\":\"c2\",\"UserAlias\":\"c2\",\"TableId\":\"5129202035162885657\",\"TableType\":\"1\",\"ColId\":\"3\",\"ProjId\":\"0\",\"ColType\":\"1\",\"DbName\":\"test\",\"TableName\":\"at_once_interval_ext_stb\",\"TableAlias\":\"at_once_interval_ext_stb\",\"ColName\":\"c2\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"8\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"Name\":\"cast\",\"Id\":\"77\",\"Type\":\"2000\",\"Parameters\":[{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"5\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"#expr_12\",\"UserAlias\":\"hyperloglog(c11)\",\"Name\":\"hyperloglog\",\"Id\":\"43\",\"Type\":\"17\",\"Parameters\":[{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"8\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"c11\",\"UserAlias\":\"c11\",\"TableId\":\"5129202035162885657\",\"TableType\":\"1\",\"ColId\":\"12\",\"ProjId\":\"0\",\"ColType\":\"1\",\"DbName\":\"test\",\"TableName\":\"at_once_interval_ext_stb\",\"TableAlias\":\"at_once_interval_ext_stb\",\"ColName\":\"c11\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"2\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":true,\"NotReserved\":true,\"IsNull\":false,\"Unit\":\"0\",\"Datum\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"10\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"26\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"Name\":\"cast\",\"Id\":\"77\",\"Type\":\"2000\",\"Parameters\":[{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"5\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"#expr_13\",\"UserAlias\":\"timediff(1, 0, 1h)\",\"Name\":\"timediff\",\"Id\":\"81\",\"Type\":\"2501\",\"Parameters\":[{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"5\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"c4ca4238a0b923820dcc509a6f75849b\",\"UserAlias\":\"1\",\"LiteralSize\":\"1\",\"Literal\":\"1\",\"Duration\":false,\"Translate\":true,\"NotReserved\":false,\"IsNull\":false,\"Unit\":\"0\",\"Datum\":\"1\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"5\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"cfcd208495d565ef66e7dff9f98764da\",\"UserAlias\":\"0\",\"LiteralSize\":\"1\",\"Literal\":\"0\",\"Duration\":false,\"Translate\":true,\"NotReserved\":false,\"IsNull\":false,\"Unit\":\"0\",\"Datum\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"5\",\"Precision\":\"0\",\"Scale\":\"104\",\"Bytes\":\"8\"},\"AliasName\":\"7c68645d71b803bf0ba2f22519f73e08\",\"UserAlias\":\"1h\",\"LiteralSize\":\"2\",\"Literal\":\"1h\",\"Duration\":true,\"Translate\":true,\"NotReserved\":false,\"IsNull\":false,\"Unit\":\"104\",\"Datum\":\"3600000\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"2\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":true,\"NotReserved\":true,\"IsNull\":false,\"Unit\":\"0\",\"Datum\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"2\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":true,\"NotReserved\":true,\"IsNull\":false,\"Unit\":\"0\",\"Datum\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"1\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"Name\":\"cast\",\"Id\":\"77\",\"Type\":\"2000\",\"Parameters\":[{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"8\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"96\"},\"AliasName\":\"#expr_14\",\"UserAlias\":\"timezone()\",\"Name\":\"timezone\",\"Id\":\"84\",\"Type\":\"2503\",\"UdfBufSize\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"2\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":true,\"NotReserved\":true,\"IsNull\":false,\"Unit\":\"0\",\"Datum\":\"0\"}}],\"UdfBufSize\":\"0\"}}],\"From\":{\"NodeType\":\"6\",\"Name\":\"RealTable\",\"RealTable\":{\"DataType\":{\"Type\":\"0\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"0\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"DbName\":\"test\",\"tableName\":\"at_once_interval_ext_stb\",\"tableAlias\":\"at_once_interval_ext_stb\",\"MetaSize\":\"2008\",\"Meta\":{\"VgId\":\"0\",\"TableType\":\"1\",\"Uid\":\"5129202035162885657\",\"Suid\":\"5129202035162885657\",\"Sversion\":\"1\",\"Tversion\":\"1\",\"ComInfo\":{\"NumOfTags\":\"13\",\"Precision\":\"0\",\"NumOfColumns\":\"14\",\"RowSize\":\"85\"},\"ColSchemas\":[{\"Type\":\"9\",\"ColId\":\"1\",\"bytes\":\"8\",\"Name\":\"ts\"},{\"Type\":\"2\",\"ColId\":\"2\",\"bytes\":\"1\",\"Name\":\"c1\"},{\"Type\":\"3\",\"ColId\":\"3\",\"bytes\":\"2\",\"Name\":\"c2\"},{\"Type\":\"4\",\"ColId\":\"4\",\"bytes\":\"4\",\"Name\":\"c3\"},{\"Type\":\"5\",\"ColId\":\"5\",\"bytes\":\"8\",\"Name\":\"c4\"},{\"Type\":\"11\",\"ColId\":\"6\",\"bytes\":\"1\",\"Name\":\"c5\"},{\"Type\":\"12\",\"ColId\":\"7\",\"bytes\":\"2\",\"Name\":\"c6\"},{\"Type\":\"13\",\"ColId\":\"8\",\"bytes\":\"4\",\"Name\":\"c7\"},{\"Type\":\"14\",\"ColId\":\"9\",\"bytes\":\"8\",\"Name\":\"c8\"},{\"Type\":\"6\",\"ColId\":\"10\",\"bytes\":\"4\",\"Name\":\"c9\"},{\"Type\":\"7\",\"ColId\":\"11\",\"bytes\":\"8\",\"Name\":\"c10\"},{\"Type\":\"8\",\"ColId\":\"12\",\"bytes\":\"8\",\"Name\":\"c11\"},{\"Type\":\"10\",\"ColId\":\"13\",\"bytes\":\"26\",\"Name\":\"c12\"},{\"Type\":\"1\",\"ColId\":\"14\",\"bytes\":\"1\",\"Name\":\"c13\"},{\"Type\":\"2\",\"ColId\":\"15\",\"bytes\":\"1\",\"Name\":\"t1\"},{\"Type\":\"3\",\"ColId\":\"16\",\"bytes\":\"2\",\"Name\":\"t2\"},{\"Type\":\"4\",\"ColId\":\"17\",\"bytes\":\"4\",\"Name\":\"t3\"},{\"Type\":\"5\",\"ColId\":\"18\",\"bytes\":\"8\",\"Name\":\"t4\"},{\"Type\":\"11\",\"ColId\":\"19\",\"bytes\":\"1\",\"Name\":\"t5\"},{\"Type\":\"12\",\"ColId\":\"20\",\"bytes\":\"2\",\"Name\":\"t6\"},{\"Type\":\"13\",\"ColId\":\"21\",\"bytes\":\"4\",\"Name\":\"t7\"},{\"Type\":\"14\",\"ColId\":\"22\",\"bytes\":\"8\",\"Name\":\"t8\"},{\"Type\":\"6\",\"ColId\":\"23\",\"bytes\":\"4\",\"Name\":\"t9\"},{\"Type\":\"7\",\"ColId\":\"24\",\"bytes\":\"8\",\"Name\":\"t10\"},{\"Type\":\"8\",\"ColId\":\"25\",\"bytes\":\"8\",\"Name\":\"t11\"},{\"Type\":\"10\",\"ColId\":\"26\",\"bytes\":\"26\",\"Name\":\"t12\"},{\"Type\":\"1\",\"ColId\":\"27\",\"bytes\":\"1\",\"Name\":\"t13\"}]},\"VgroupsInfoSize\":\"1340\",\"VgroupsInfo\":{\"Num\":\"2\",\"Vgroups\":[{\"VgId\":\"14\",\"HashBegin\":\"0\",\"HashEnd\":\"2147483646\",\"EpSet\":{\"InUse\":\"0\",\"NumOfEps\":\"1\",\"Eps\":[{\"Fqdn\":\"localhost\",\"Port\":\"6030\"}]},\"NumOfTable\":\"0\"},{\"VgId\":\"15\",\"HashBegin\":\"2147483647\",\"HashEnd\":\"4294967295\",\"EpSet\":{\"InUse\":\"0\",\"NumOfEps\":\"1\",\"Eps\":[{\"Fqdn\":\"localhost\",\"Port\":\"6030\"}]},\"NumOfTable\":\"0\"}]}}},\"Tags\":[{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"0\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":false,\"NotReserved\":false,\"IsNull\":true,\"Unit\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"0\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":false,\"NotReserved\":false,\"IsNull\":true,\"Unit\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"0\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":false,\"NotReserved\":false,\"IsNull\":true,\"Unit\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"0\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":false,\"NotReserved\":false,\"IsNull\":true,\"Unit\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"0\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":false,\"NotReserved\":false,\"IsNull\":true,\"Unit\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"0\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":false,\"NotReserved\":false,\"IsNull\":true,\"Unit\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"0\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":false,\"NotReserved\":false,\"IsNull\":true,\"Unit\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"0\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":false,\"NotReserved\":false,\"IsNull\":true,\"Unit\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"0\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":false,\"NotReserved\":false,\"IsNull\":true,\"Unit\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"0\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":false,\"NotReserved\":false,\"IsNull\":true,\"Unit\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"0\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":false,\"NotReserved\":false,\"IsNull\":true,\"Unit\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"0\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":false,\"NotReserved\":false,\"IsNull\":true,\"Unit\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"0\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":false,\"NotReserved\":false,\"IsNull\":true,\"Unit\":\"0\"}}],\"Window\":{\"NodeType\":\"14\",\"Name\":\"IntervalWindow\",\"IntervalWindow\":{\"Interval\":{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"5\",\"Precision\":\"0\",\"Scale\":\"115\",\"Bytes\":\"8\"},\"AliasName\":\"1fd7635317edfeca9054894ac9ef9b5e\",\"UserAlias\":\"14s\",\"LiteralSize\":\"3\",\"Literal\":\"14s\",\"Duration\":true,\"Translate\":true,\"NotReserved\":false,\"IsNull\":false,\"Unit\":\"115\",\"Datum\":\"14000\"}},\"TsPk\":{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"9\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"ts\",\"UserAlias\":\"ts\",\"TableId\":\"5129202035162885657\",\"TableType\":\"1\",\"ColId\":\"1\",\"ProjId\":\"0\",\"ColType\":\"1\",\"DbName\":\"test\",\"TableName\":\"at_once_interval_ext_stb\",\"TableAlias\":\"at_once_interval_ext_stb\",\"ColName\":\"ts\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}}}},\"StmtName\":\"0x150146d14\",\"HasAggFuncs\":true}}"; SNode * pAst = NULL; SQueryPlan *pPlan = NULL; @@ -62,5 +62,6 @@ TEST(testCase, plan_Test) { if (qCreateQueryPlan(&cxt, &pPlan, NULL) < 0) { ASSERT(0); } - + if (pAst != NULL) nodesDestroyNode(pAst); + nodesDestroyNode((SNode *)pPlan); } diff --git a/source/libs/planner/src/planSpliter.c b/source/libs/planner/src/planSpliter.c index accf5bf100..ef131b4cb3 100644 --- a/source/libs/planner/src/planSpliter.c +++ b/source/libs/planner/src/planSpliter.c @@ -785,7 +785,35 @@ static int32_t stbSplSplitIntervalForBatch(SSplitContext* pCxt, SStableSplitInfo // return code; //} +static bool isStreamMultiAgg(SLogicNode* pNode) { + if(LIST_LENGTH(pNode->pChildren) <= 0) return false; + + SNode* pChild = nodesListGetNode(pNode->pChildren, 0); + if (QUERY_NODE_LOGIC_PLAN_SCAN == nodeType(pChild)) { + qDebug("vgroups:%d", ((SScanLogicNode*)pNode)->pVgroupList->numOfVgroups); + return ((SScanLogicNode*)pNode)->pVgroupList->numOfVgroups > tsStreamAggCnt; + } + return false; +} + static int32_t stbSplSplitIntervalForStream(SSplitContext* pCxt, SStableSplitInfo* pInfo) { + SLogicNode* pPartWindow = NULL; + int32_t code = stbSplCreatePartWindowNode((SWindowLogicNode*)pInfo->pSplitNode, &pPartWindow); + if (TSDB_CODE_SUCCESS == code) { + ((SWindowLogicNode*)pPartWindow)->windowAlgo = INTERVAL_ALGO_STREAM_SEMI; + ((SWindowLogicNode*)pInfo->pSplitNode)->windowAlgo = INTERVAL_ALGO_STREAM_FINAL; + code = stbSplCreateExchangeNode(pCxt, pInfo->pSplitNode, pPartWindow); + } + if (TSDB_CODE_SUCCESS == code) { + code = nodesListMakeStrictAppend(&pInfo->pSubplan->pChildren, + (SNode*)splCreateScanSubplan(pCxt, pPartWindow, SPLIT_FLAG_STABLE_SPLIT)); + } + pInfo->pSubplan->subplanType = SUBPLAN_TYPE_MERGE; + ++(pCxt->groupId); + return code; +} + +static int32_t stbSplSplitIntervalForStreamMultiAgg(SSplitContext* pCxt, SStableSplitInfo* pInfo) { SLogicNode* pPartWindow = NULL; SLogicNode* pMidWindow = NULL; int32_t code = stbSplCreatePartMidWindowNode((SWindowLogicNode*)pInfo->pSplitNode, &pPartWindow, &pMidWindow); @@ -817,7 +845,11 @@ static int32_t stbSplSplitIntervalForStream(SSplitContext* pCxt, SStableSplitInf static int32_t stbSplSplitInterval(SSplitContext* pCxt, SStableSplitInfo* pInfo) { if (pCxt->pPlanCxt->streamQuery) { - return stbSplSplitIntervalForStream(pCxt, pInfo); +// if(isStreamMultiAgg(pInfo->pSplitNode)){ + return stbSplSplitIntervalForStreamMultiAgg(pCxt, pInfo); +// }else{ +// return stbSplSplitIntervalForStream(pCxt, pInfo); +// } } else { return stbSplSplitIntervalForBatch(pCxt, pInfo); } From 3a0af1ea49c4898913f9ab5007a4366df9d64b5e Mon Sep 17 00:00:00 2001 From: wangmm0220 Date: Tue, 19 Dec 2023 10:53:46 +0800 Subject: [PATCH 17/51] fix:aggCnt must bigger than 1 & move msg code to the end to be compatible with old version --- include/common/tmsg.h | 4 ++-- source/common/src/tglobal.c | 2 +- source/dnode/mnode/impl/src/mndScheduler.c | 7 ++----- source/libs/stream/src/streamTask.c | 2 +- 4 files changed, 6 insertions(+), 9 deletions(-) diff --git a/include/common/tmsg.h b/include/common/tmsg.h index e6ed0f250c..bd759df621 100644 --- a/include/common/tmsg.h +++ b/include/common/tmsg.h @@ -395,7 +395,6 @@ typedef enum ENodeType { QUERY_NODE_PHYSICAL_PLAN_STREAM_INTERVAL, QUERY_NODE_PHYSICAL_PLAN_STREAM_FINAL_INTERVAL, QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_INTERVAL, - QUERY_NODE_PHYSICAL_PLAN_STREAM_MID_INTERVAL, QUERY_NODE_PHYSICAL_PLAN_FILL, QUERY_NODE_PHYSICAL_PLAN_STREAM_FILL, QUERY_NODE_PHYSICAL_PLAN_MERGE_SESSION, @@ -419,7 +418,8 @@ typedef enum ENodeType { QUERY_NODE_PHYSICAL_PLAN_STREAM_EVENT, QUERY_NODE_PHYSICAL_PLAN_HASH_JOIN, QUERY_NODE_PHYSICAL_PLAN_GROUP_CACHE, - QUERY_NODE_PHYSICAL_PLAN_DYN_QUERY_CTRL + QUERY_NODE_PHYSICAL_PLAN_DYN_QUERY_CTRL, + QUERY_NODE_PHYSICAL_PLAN_STREAM_MID_INTERVAL } ENodeType; typedef struct { diff --git a/source/common/src/tglobal.c b/source/common/src/tglobal.c index 4f2ae1b11f..865528086a 100644 --- a/source/common/src/tglobal.c +++ b/source/common/src/tglobal.c @@ -738,7 +738,7 @@ static int32_t taosAddServerCfg(SConfig *pCfg) { if (cfgAddBool(pCfg, "disableStream", tsDisableStream, CFG_SCOPE_SERVER, CFG_DYN_ENT_SERVER) != 0) return -1; if (cfgAddInt64(pCfg, "streamBufferSize", tsStreamBufferSize, 0, INT64_MAX, CFG_SCOPE_SERVER, CFG_DYN_NONE) != 0) return -1; - if (cfgAddInt64(pCfg, "streamAggCnt", tsStreamAggCnt, 0, INT32_MAX, CFG_SCOPE_SERVER, CFG_DYN_NONE) != 0) + if (cfgAddInt64(pCfg, "streamAggCnt", tsStreamAggCnt, 2, INT32_MAX, CFG_SCOPE_SERVER, CFG_DYN_NONE) != 0) return -1; if (cfgAddInt32(pCfg, "checkpointInterval", tsStreamCheckpointInterval, 60, 1200, CFG_SCOPE_SERVER, diff --git a/source/dnode/mnode/impl/src/mndScheduler.c b/source/dnode/mnode/impl/src/mndScheduler.c index e91253e689..47461d7c1b 100644 --- a/source/dnode/mnode/impl/src/mndScheduler.c +++ b/source/dnode/mnode/impl/src/mndScheduler.c @@ -534,11 +534,8 @@ static void bindTwoLevel(SArray* tasks, int32_t begin, int32_t end) { SArray* pUpTaskList = taosArrayGetP(tasks, size - 2); SStreamTask** pDownTask = taosArrayGetLast(pDownTaskList); - for(int i = begin; i < end; i++){ + for(int i = begin; i < end && i < taosArrayGetSize(pUpTaskList); i++){ SStreamTask* pUpTask = taosArrayGetP(pUpTaskList, i); - if(pUpTask == NULL) { // out of range - break; - } streamTaskSetFixedDownstreamInfo(pUpTask, *pDownTask); streamTaskSetUpstreamInfo(*pDownTask, pUpTask); } @@ -600,7 +597,7 @@ static int32_t doScheduleStream(SStreamObj* pStream, SMnode* pMnode, SQueryPlan* do{ SArray** list = taosArrayGetLast(pStream->tasks); float size = (float)taosArrayGetSize(*list); - size_t cnt = (int)(size/tsStreamAggCnt + 0.5); + size_t cnt = (size_t)ceil(size/tsStreamAggCnt); if(cnt <= 1) break; mDebug("doScheduleStream add middle agg, size:%d, cnt:%d", (int)size, (int)cnt); diff --git a/source/libs/stream/src/streamTask.c b/source/libs/stream/src/streamTask.c index db0217f000..1c5d39f92c 100644 --- a/source/libs/stream/src/streamTask.c +++ b/source/libs/stream/src/streamTask.c @@ -54,7 +54,7 @@ SStreamTask* tNewStreamTask(int64_t streamId, int8_t taskLevel, bool fillHistory } char buf[128] = {0}; - sprintf(buf, "0x%" PRIx64 "-%d", pTask->id.streamId, pTask->id.taskId); + sprintf(buf, "0x%" PRIx64 "-0x%x", pTask->id.streamId, pTask->id.taskId); pTask->id.idStr = taosStrdup(buf); pTask->status.schedStatus = TASK_SCHED_STATUS__INACTIVE; From b0c71f36b1287c581a8587f154763db406cdb642 Mon Sep 17 00:00:00 2001 From: wangmm0220 Date: Tue, 19 Dec 2023 14:07:55 +0800 Subject: [PATCH 18/51] fix:disable print planner string --- source/common/src/tglobal.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/common/src/tglobal.c b/source/common/src/tglobal.c index d362a73145..d32f32d3fb 100644 --- a/source/common/src/tglobal.c +++ b/source/common/src/tglobal.c @@ -148,7 +148,7 @@ bool tsEnableQueryHb = true; bool tsEnableScience = false; // on taos-cli show float and doulbe with scientific notation if true int32_t tsQuerySmaOptimize = 0; int32_t tsQueryRsmaTolerance = 1000; // the tolerance time (ms) to judge from which level to query rsma data. -bool tsQueryPlannerTrace = true; +bool tsQueryPlannerTrace = false; int32_t tsQueryNodeChunkSize = 32 * 1024; bool tsQueryUseNodeAllocator = true; bool tsKeepColumnName = false; From 78c8ebae7f4f505e232391102f945c7df740b2c3 Mon Sep 17 00:00:00 2001 From: wangmm0220 Date: Tue, 19 Dec 2023 16:35:04 +0800 Subject: [PATCH 19/51] fix:remove useless code in split logic plan & random choose vnode in stream task & add test case --- source/dnode/mnode/impl/inc/mndDef.h | 2 + source/dnode/mnode/impl/src/mndScheduler.c | 33 +++++-- source/dnode/mnode/impl/src/mndSma.c | 1 + source/dnode/mnode/impl/src/mndStream.c | 1 + source/libs/planner/src/planSpliter.c | 51 +--------- tests/parallel_test/cases.task | 1 + .../system-test/8-stream/stream_multi_agg.py | 98 +++++++++++++++++++ 7 files changed, 130 insertions(+), 57 deletions(-) create mode 100644 tests/system-test/8-stream/stream_multi_agg.py diff --git a/source/dnode/mnode/impl/inc/mndDef.h b/source/dnode/mnode/impl/inc/mndDef.h index 8dfd03622f..8b50465b5f 100644 --- a/source/dnode/mnode/impl/inc/mndDef.h +++ b/source/dnode/mnode/impl/inc/mndDef.h @@ -694,6 +694,8 @@ typedef struct { // 3.0.5. int64_t checkpointId; + + int32_t indexForMultiAggBalance; char reserve[256]; } SStreamObj; diff --git a/source/dnode/mnode/impl/src/mndScheduler.c b/source/dnode/mnode/impl/src/mndScheduler.c index 47461d7c1b..80f77bc9a5 100644 --- a/source/dnode/mnode/impl/src/mndScheduler.c +++ b/source/dnode/mnode/impl/src/mndScheduler.c @@ -181,20 +181,39 @@ int32_t mndAssignStreamTaskToSnode(SMnode* pMnode, SStreamTask* pTask, SSubplan* return qSubPlanToString(plan, &pTask->exec.qmsg, &msgLen); } -// todo random choose a node to do compute -SVgObj* mndSchedFetchOneVg(SMnode* pMnode, int64_t dbUid) { +// random choose a node to do compute +SVgObj* mndSchedFetchOneVg(SMnode* pMnode, SStreamObj* pStream) { + SDbObj* pDbObj = mndAcquireDb(pMnode, pStream->sourceDb); + if (pDbObj == NULL) { + terrno = TSDB_CODE_QRY_INVALID_INPUT; + return NULL; + } + + if(pStream->indexForMultiAggBalance == -1){ + taosSeedRand(taosSafeRand()); + pStream->indexForMultiAggBalance = taosRand() % pDbObj->cfg.numOfVgroups; + } + + int32_t index = 0; void* pIter = NULL; SVgObj* pVgroup = NULL; while (1) { pIter = sdbFetch(pMnode->pSdb, SDB_VGROUP, pIter, (void**)&pVgroup); if (pIter == NULL) break; - if (pVgroup->dbUid != dbUid) { + if (pVgroup->dbUid != pStream->sourceDbUid) { sdbRelease(pMnode->pSdb, pVgroup); continue; } - sdbCancelFetch(pMnode->pSdb, pIter); - return pVgroup; + if (index++ == pStream->indexForMultiAggBalance){ + pStream->indexForMultiAggBalance++; + pStream->indexForMultiAggBalance %= pDbObj->cfg.numOfVgroups; + sdbCancelFetch(pMnode->pSdb, pIter); + break; + } + sdbRelease(pMnode->pSdb, pVgroup); } + sdbRelease(pMnode->pSdb, pDbObj); + return pVgroup; } @@ -440,10 +459,10 @@ static int32_t addAggTask(SStreamObj* pStream, SMnode* pMnode, SSubplan* plan, S if (tsDeployOnSnode) { pSnode = mndSchedFetchOneSnode(pMnode); if (pSnode == NULL) { - pVgroup = mndSchedFetchOneVg(pMnode, pStream->sourceDbUid); + pVgroup = mndSchedFetchOneVg(pMnode, pStream); } } else { - pVgroup = mndSchedFetchOneVg(pMnode, pStream->sourceDbUid); + pVgroup = mndSchedFetchOneVg(pMnode, pStream); } code = doAddAggTask(pStream, pMnode, plan, pEpset, pVgroup, pSnode, false); diff --git a/source/dnode/mnode/impl/src/mndSma.c b/source/dnode/mnode/impl/src/mndSma.c index 44842084c5..34ea7109d0 100644 --- a/source/dnode/mnode/impl/src/mndSma.c +++ b/source/dnode/mnode/impl/src/mndSma.c @@ -566,6 +566,7 @@ static int32_t mndCreateSma(SMnode *pMnode, SRpcMsg *pReq, SMCreateSmaReq *pCrea streamObj.conf.trigger = STREAM_TRIGGER_WINDOW_CLOSE; streamObj.conf.triggerParam = pCreate->maxDelay; streamObj.ast = taosStrdup(smaObj.ast); + streamObj.indexForMultiAggBalance = -1; // check the maxDelay if (streamObj.conf.triggerParam < TSDB_MIN_ROLLUP_MAX_DELAY) { diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index f8fba143f8..50450f0cf2 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -363,6 +363,7 @@ static int32_t mndBuildStreamObjFromCreateReq(SMnode *pMnode, SStreamObj *pObj, pObj->updateTime = pObj->createTime; pObj->version = 1; pObj->smaId = 0; + pObj->indexForMultiAggBalance = -1; pObj->uid = mndGenerateUid(pObj->name, strlen(pObj->name)); diff --git a/source/libs/planner/src/planSpliter.c b/source/libs/planner/src/planSpliter.c index ef131b4cb3..cae500d7b9 100644 --- a/source/libs/planner/src/planSpliter.c +++ b/source/libs/planner/src/planSpliter.c @@ -768,51 +768,6 @@ static int32_t stbSplSplitIntervalForBatch(SSplitContext* pCxt, SStableSplitInfo return code; } -//static int32_t stbSplSplitIntervalForStream(SSplitContext* pCxt, SStableSplitInfo* pInfo) { -// SLogicNode* pPartWindow = NULL; -// int32_t code = stbSplCreatePartWindowNode((SWindowLogicNode*)pInfo->pSplitNode, &pPartWindow); -// if (TSDB_CODE_SUCCESS == code) { -// ((SWindowLogicNode*)pPartWindow)->windowAlgo = INTERVAL_ALGO_STREAM_SEMI; -// ((SWindowLogicNode*)pInfo->pSplitNode)->windowAlgo = INTERVAL_ALGO_STREAM_FINAL; -// code = stbSplCreateExchangeNode(pCxt, pInfo->pSplitNode, pPartWindow); -// } -// if (TSDB_CODE_SUCCESS == code) { -// code = nodesListMakeStrictAppend(&pInfo->pSubplan->pChildren, -// (SNode*)splCreateScanSubplan(pCxt, pPartWindow, SPLIT_FLAG_STABLE_SPLIT)); -// } -// pInfo->pSubplan->subplanType = SUBPLAN_TYPE_MERGE; -// ++(pCxt->groupId); -// return code; -//} - -static bool isStreamMultiAgg(SLogicNode* pNode) { - if(LIST_LENGTH(pNode->pChildren) <= 0) return false; - - SNode* pChild = nodesListGetNode(pNode->pChildren, 0); - if (QUERY_NODE_LOGIC_PLAN_SCAN == nodeType(pChild)) { - qDebug("vgroups:%d", ((SScanLogicNode*)pNode)->pVgroupList->numOfVgroups); - return ((SScanLogicNode*)pNode)->pVgroupList->numOfVgroups > tsStreamAggCnt; - } - return false; -} - -static int32_t stbSplSplitIntervalForStream(SSplitContext* pCxt, SStableSplitInfo* pInfo) { - SLogicNode* pPartWindow = NULL; - int32_t code = stbSplCreatePartWindowNode((SWindowLogicNode*)pInfo->pSplitNode, &pPartWindow); - if (TSDB_CODE_SUCCESS == code) { - ((SWindowLogicNode*)pPartWindow)->windowAlgo = INTERVAL_ALGO_STREAM_SEMI; - ((SWindowLogicNode*)pInfo->pSplitNode)->windowAlgo = INTERVAL_ALGO_STREAM_FINAL; - code = stbSplCreateExchangeNode(pCxt, pInfo->pSplitNode, pPartWindow); - } - if (TSDB_CODE_SUCCESS == code) { - code = nodesListMakeStrictAppend(&pInfo->pSubplan->pChildren, - (SNode*)splCreateScanSubplan(pCxt, pPartWindow, SPLIT_FLAG_STABLE_SPLIT)); - } - pInfo->pSubplan->subplanType = SUBPLAN_TYPE_MERGE; - ++(pCxt->groupId); - return code; -} - static int32_t stbSplSplitIntervalForStreamMultiAgg(SSplitContext* pCxt, SStableSplitInfo* pInfo) { SLogicNode* pPartWindow = NULL; SLogicNode* pMidWindow = NULL; @@ -845,11 +800,7 @@ static int32_t stbSplSplitIntervalForStreamMultiAgg(SSplitContext* pCxt, SStable static int32_t stbSplSplitInterval(SSplitContext* pCxt, SStableSplitInfo* pInfo) { if (pCxt->pPlanCxt->streamQuery) { -// if(isStreamMultiAgg(pInfo->pSplitNode)){ - return stbSplSplitIntervalForStreamMultiAgg(pCxt, pInfo); -// }else{ -// return stbSplSplitIntervalForStream(pCxt, pInfo); -// } + return stbSplSplitIntervalForStreamMultiAgg(pCxt, pInfo); } else { return stbSplSplitIntervalForBatch(pCxt, pInfo); } diff --git a/tests/parallel_test/cases.task b/tests/parallel_test/cases.task index 8edfb352ab..0c66d0d1ed 100644 --- a/tests/parallel_test/cases.task +++ b/tests/parallel_test/cases.task @@ -9,6 +9,7 @@ ,,y,army,./pytest.sh python3 ./test.py -f empty.py #system test +,,y,system-test,./pytest.sh python3 ./test.py -f 8-stream/stream_multi_agg.py ,,y,system-test,./pytest.sh python3 ./test.py -f 8-stream/stream_basic.py ,,y,system-test,./pytest.sh python3 ./test.py -f 8-stream/scalar_function.py ,,y,system-test,./pytest.sh python3 ./test.py -f 8-stream/at_once_interval.py diff --git a/tests/system-test/8-stream/stream_multi_agg.py b/tests/system-test/8-stream/stream_multi_agg.py new file mode 100644 index 0000000000..5c478d7e14 --- /dev/null +++ b/tests/system-test/8-stream/stream_multi_agg.py @@ -0,0 +1,98 @@ +################################################################### +# Copyright (c) 2016 by TAOS Technologies, Inc. +# All rights reserved. +# +# This file is proprietary and confidential to TAOS Technologies. +# No part of this file may be reproduced, stored, transmitted, +# disclosed or used in any form or by any means other than as +# expressly provided by the written permission from Jianhui Tao +# +################################################################### + +# -*- coding: utf-8 -*- + + +from util.log import * +from util.cases import * +from util.sql import * +from util.common import * +from util.sqlset import * +from util.autogen import * + +import random +import time +import traceback +import os +from os import path + + +class TDTestCase: + updatecfgDict = {'debugFlag': 135, 'asynclog': 0, 'streamAggCnt': 2} + # init + def init(self, conn, logSql, replicaVar=1): + self.replicaVar = int(replicaVar) + tdLog.debug("start to execute %s" % __file__) + tdSql.init(conn.cursor(), True) + + def case1(self): + tdLog.debug("========case1 start========") + + os.system("nohup taosBenchmark -y -B 1 -t 40 -S 1000 -n 10 -i 1000 -v 12 > /dev/null 2>&1 &") + time.sleep(4) + tdSql.query("use test") + tdSql.query("create stream if not exists s1 trigger at_once ignore expired 0 ignore update 0 fill_history 1 into st1 as select _wstart,sum(voltage),groupid from meters partition by groupid interval(2s)") + tdLog.debug("========create stream useing snode and insert data ok========") + time.sleep(15) + + tdSql.query("select _wstart,sum(voltage),groupid from meters partition by groupid interval(2s) order by groupid,_wstart") + rowCnt = tdSql.getRows() + results = [] + for i in range(rowCnt): + results.append(tdSql.getData(i,1)) + + tdSql.query("select * from st1 order by groupid,_wstart") + tdSql.checkRows(rowCnt) + for i in range(rowCnt): + data1 = tdSql.getData(i,1) + data2 = results[i] + if data1 != data2: + tdLog.info("num: %d, act data: %d, expect data: %d"%(i, data1, data2)) + tdLog.exit("check data error!") + + tdLog.debug("case1 end") + + def case2(self): + tdLog.debug("========case2 start========") + + os.system("taosBenchmark -d db -t 20 -v 12 -n 1000 -y > /dev/null 2>&1") + # create stream + tdSql.execute("use db") + tdSql.execute("create stream stream1 fill_history 1 into sta as select count(*) as cnt from meters interval(10a);",show=True) + sql = "select count(*) from sta" + # loop wait max 60s to check count is ok + tdLog.info("loop wait result ...") + tdSql.checkDataLoop(0, 0, 99, sql, loopCount=120, waitTime=0.5) + + # check all data is correct + sql = "select * from sta where cnt != 200;" + tdSql.query(sql) + tdSql.checkRows(0) + + # check ts interval is correct + sql = "select * from ( select diff(_wstart) as tsdif from sta ) where tsdif != 10;" + tdSql.query(sql) + tdSql.checkRows(0) + tdLog.debug("case2 end") + +# run + def run(self): + self.case1() + self.case2() + + # stop + def stop(self): + tdSql.close() + tdLog.success("%s successfully executed" % __file__) + + +tdCases.addLinux(__file__, TDTestCase()) \ No newline at end of file From 9c2ab769ea3c79140019b74ad7f36d6869d3754c Mon Sep 17 00:00:00 2001 From: wangmm0220 Date: Tue, 19 Dec 2023 16:45:40 +0800 Subject: [PATCH 20/51] fix:change tsStreamAggCn=2 for test --- source/common/src/tglobal.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/common/src/tglobal.c b/source/common/src/tglobal.c index d32f32d3fb..dbf9838527 100644 --- a/source/common/src/tglobal.c +++ b/source/common/src/tglobal.c @@ -264,7 +264,7 @@ bool tsDisableStream = false; int64_t tsStreamBufferSize = 128 * 1024 * 1024; bool tsFilterScalarMode = false; int tsResolveFQDNRetryTime = 100; // seconds -int tsStreamAggCnt = 10; +int tsStreamAggCnt = 2; char tsS3Endpoint[TSDB_FQDN_LEN] = ""; char tsS3AccessKey[TSDB_FQDN_LEN] = ""; From 60722bcf4d0198b8f494b42d749ecd6f0740cfcc Mon Sep 17 00:00:00 2001 From: 54liuyao <54liuyao> Date: Wed, 20 Dec 2023 10:53:23 +0800 Subject: [PATCH 21/51] use mid interval --- source/libs/executor/src/streamtimewindowoperator.c | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/source/libs/executor/src/streamtimewindowoperator.c b/source/libs/executor/src/streamtimewindowoperator.c index fde3fec7d4..857b09bf51 100644 --- a/source/libs/executor/src/streamtimewindowoperator.c +++ b/source/libs/executor/src/streamtimewindowoperator.c @@ -48,6 +48,8 @@ typedef struct SPullWindowInfo { STimeWindow calWin; } SPullWindowInfo; +static SSDataBlock* doStreamMidIntervalAgg(SOperatorInfo* pOperator); + typedef int32_t (*__compare_fn_t)(void* pKey, void* data, int32_t index); static int32_t binarySearchCom(void* keyList, int num, void* pKey, int order, __compare_fn_t comparefn) { @@ -1535,8 +1537,13 @@ SOperatorInfo* createStreamFinalIntervalOperatorInfo(SOperatorInfo* downstream, pOperator->status = OP_NOT_OPENED; pOperator->info = pInfo; - pOperator->fpSet = createOperatorFpSet(NULL, doStreamFinalIntervalAgg, NULL, destroyStreamFinalIntervalOperatorInfo, - optrDefaultBufFn, NULL, optrDefaultGetNextExtFn, NULL); + if (pPhyNode->type == QUERY_NODE_PHYSICAL_PLAN_STREAM_MID_INTERVAL) { + pOperator->fpSet = createOperatorFpSet(NULL, doStreamMidIntervalAgg, NULL, destroyStreamFinalIntervalOperatorInfo, + optrDefaultBufFn, NULL, optrDefaultGetNextExtFn, NULL); + } else { + pOperator->fpSet = createOperatorFpSet(NULL, doStreamFinalIntervalAgg, NULL, destroyStreamFinalIntervalOperatorInfo, + optrDefaultBufFn, NULL, optrDefaultGetNextExtFn, NULL); + } setOperatorStreamStateFn(pOperator, streamIntervalReleaseState, streamIntervalReloadState); if (pPhyNode->type == QUERY_NODE_PHYSICAL_PLAN_STREAM_SEMI_INTERVAL || pPhyNode->type == QUERY_NODE_PHYSICAL_PLAN_STREAM_MID_INTERVAL) { From dd0afef6f2e38f4a7a902e4c29cd885fea9f8bee Mon Sep 17 00:00:00 2001 From: wangmm0220 Date: Wed, 20 Dec 2023 11:00:25 +0800 Subject: [PATCH 22/51] fix:set triggerParams=0 in mid interval --- source/dnode/mnode/impl/src/mndScheduler.c | 21 +++++---- source/dnode/mnode/impl/src/mndStream.c | 1 - .../executor/src/streamtimewindowoperator.c | 5 +- source/libs/planner/src/planSpliter.c | 47 ------------------- 4 files changed, 15 insertions(+), 59 deletions(-) diff --git a/source/dnode/mnode/impl/src/mndScheduler.c b/source/dnode/mnode/impl/src/mndScheduler.c index 80f77bc9a5..2cf5330f09 100644 --- a/source/dnode/mnode/impl/src/mndScheduler.c +++ b/source/dnode/mnode/impl/src/mndScheduler.c @@ -421,11 +421,13 @@ static int32_t addSourceTask(SMnode* pMnode, SSubplan* plan, SStreamObj* pStream return TSDB_CODE_SUCCESS; } -static SStreamTask* buildAggTask(SStreamObj* pStream, SEpSet* pEpset, bool isFillhistory) { +static SStreamTask* buildAggTask(SStreamObj* pStream, SEpSet* pEpset, bool isFillhistory, bool useTriggerParam) { uint64_t uid = (isFillhistory) ? pStream->hTaskUid : pStream->uid; SArray** pTaskList = (isFillhistory) ? taosArrayGetLast(pStream->pHTasksList) : taosArrayGetLast(pStream->tasks); - SStreamTask* pAggTask = tNewStreamTask(uid, TASK_LEVEL__AGG, isFillhistory, pStream->conf.triggerParam, *pTaskList, pStream->conf.fillHistory); + SStreamTask* pAggTask = tNewStreamTask(uid, TASK_LEVEL__AGG, isFillhistory, + useTriggerParam ? pStream->conf.triggerParam : 0, + *pTaskList, pStream->conf.fillHistory); if (pAggTask == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; return NULL; @@ -435,9 +437,10 @@ static SStreamTask* buildAggTask(SStreamObj* pStream, SEpSet* pEpset, bool isFil return pAggTask; } -static int32_t doAddAggTask(SStreamObj* pStream, SMnode* pMnode, SSubplan* plan, SEpSet* pEpset, SVgObj* pVgroup, SSnodeObj* pSnode, bool isFillhistory){ +static int32_t doAddAggTask(SStreamObj* pStream, SMnode* pMnode, SSubplan* plan, SEpSet* pEpset, + SVgObj* pVgroup, SSnodeObj* pSnode, bool isFillhistory, bool useTriggerParam){ int32_t code = 0; - SStreamTask* pTask = buildAggTask(pStream, pEpset, isFillhistory); + SStreamTask* pTask = buildAggTask(pStream, pEpset, isFillhistory, useTriggerParam); if (pTask == NULL) { return terrno; } @@ -452,7 +455,7 @@ static int32_t doAddAggTask(SStreamObj* pStream, SMnode* pMnode, SSubplan* plan, return code; } -static int32_t addAggTask(SStreamObj* pStream, SMnode* pMnode, SSubplan* plan, SEpSet* pEpset){ +static int32_t addAggTask(SStreamObj* pStream, SMnode* pMnode, SSubplan* plan, SEpSet* pEpset, bool useTriggerParam){ SVgObj* pVgroup = NULL; SSnodeObj* pSnode = NULL; int32_t code = 0; @@ -465,13 +468,13 @@ static int32_t addAggTask(SStreamObj* pStream, SMnode* pMnode, SSubplan* plan, S pVgroup = mndSchedFetchOneVg(pMnode, pStream); } - code = doAddAggTask(pStream, pMnode, plan, pEpset, pVgroup, pSnode, false); + code = doAddAggTask(pStream, pMnode, plan, pEpset, pVgroup, pSnode, false, useTriggerParam); if(code != 0){ goto END; } if (pStream->conf.fillHistory) { - code = doAddAggTask(pStream, pMnode, plan, pEpset, pVgroup, pSnode, true); + code = doAddAggTask(pStream, pMnode, plan, pEpset, pVgroup, pSnode, true, useTriggerParam); if(code != 0){ goto END; } @@ -623,7 +626,7 @@ static int32_t doScheduleStream(SStreamObj* pStream, SMnode* pMnode, SQueryPlan* addNewTaskList(pStream); for(int j = 0; j < cnt; j++){ - code = addAggTask(pStream, pMnode, plan, pEpset); + code = addAggTask(pStream, pMnode, plan, pEpset, false); if (code != TSDB_CODE_SUCCESS) { return code; } @@ -645,7 +648,7 @@ static int32_t doScheduleStream(SStreamObj* pStream, SMnode* pMnode, SQueryPlan* SArray** list = taosArrayGetLast(pStream->tasks); size_t size = taosArrayGetSize(*list); addNewTaskList(pStream); - code = addAggTask(pStream, pMnode, plan, pEpset); + code = addAggTask(pStream, pMnode, plan, pEpset, true); if (code != TSDB_CODE_SUCCESS) { return code; } diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index 50450f0cf2..1084536cc1 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -413,7 +413,6 @@ static int32_t mndBuildStreamObjFromCreateReq(SMnode *pMnode, SStreamObj *pObj, pCreate->sql = NULL; pCreate->ast = NULL; - qDebugL("ast:%s", pObj->ast); // deserialize ast if (nodesStringToNode(pObj->ast, &pAst) < 0) { goto FAIL; diff --git a/source/libs/executor/src/streamtimewindowoperator.c b/source/libs/executor/src/streamtimewindowoperator.c index fde3fec7d4..0766b92df3 100644 --- a/source/libs/executor/src/streamtimewindowoperator.c +++ b/source/libs/executor/src/streamtimewindowoperator.c @@ -1441,6 +1441,7 @@ SOperatorInfo* createStreamFinalIntervalOperatorInfo(SOperatorInfo* downstream, SIntervalPhysiNode* pIntervalPhyNode = (SIntervalPhysiNode*)pPhyNode; SStreamIntervalOperatorInfo* pInfo = taosMemoryCalloc(1, sizeof(SStreamIntervalOperatorInfo)); SOperatorInfo* pOperator = taosMemoryCalloc(1, sizeof(SOperatorInfo)); + int32_t code = 0; if (pInfo == NULL || pOperator == NULL) { goto _error; } @@ -1470,7 +1471,7 @@ SOperatorInfo* createStreamFinalIntervalOperatorInfo(SOperatorInfo* downstream, if (pIntervalPhyNode->window.pExprs != NULL) { int32_t numOfScalar = 0; SExprInfo* pScalarExprInfo = createExprInfo(pIntervalPhyNode->window.pExprs, NULL, &numOfScalar); - int32_t code = initExprSupp(&pInfo->scalarSupp, pScalarExprInfo, numOfScalar, &pTaskInfo->storageAPI.functionStore); + code = initExprSupp(&pInfo->scalarSupp, pScalarExprInfo, numOfScalar, &pTaskInfo->storageAPI.functionStore); if (code != TSDB_CODE_SUCCESS) { goto _error; } @@ -1489,7 +1490,7 @@ SOperatorInfo* createStreamFinalIntervalOperatorInfo(SOperatorInfo* downstream, qInfo("copy state %p to %p", pTaskInfo->streamInfo.pState, pInfo->pState); pAPI->stateStore.streamStateSetNumber(pInfo->pState, -1); - int32_t code = initAggSup(&pOperator->exprSupp, &pInfo->aggSup, pExprInfo, numOfCols, keyBufSize, pTaskInfo->id.str, + code = initAggSup(&pOperator->exprSupp, &pInfo->aggSup, pExprInfo, numOfCols, keyBufSize, pTaskInfo->id.str, pInfo->pState, &pTaskInfo->storageAPI.functionStore); if (code != TSDB_CODE_SUCCESS) { goto _error; diff --git a/source/libs/planner/src/planSpliter.c b/source/libs/planner/src/planSpliter.c index cae500d7b9..dc0e7da6a1 100644 --- a/source/libs/planner/src/planSpliter.c +++ b/source/libs/planner/src/planSpliter.c @@ -574,53 +574,6 @@ static int32_t stbSplCreatePartMidWindowNode(SWindowLogicNode* pMergeWindow, SLo return code; } -//static int32_t stbSplCreateSemiWindowNode(SWindowLogicNode* pMidWindow, SLogicNode** pSemiWindow) { -// SNodeList* pFunc = pMidWindow->pFuncs; -// pMidWindow->pFuncs = NULL; -// nodesDestroyList(pMidWindow->node.pTargets); -// pMidWindow->node.pTargets = NULL; -// SNodeList* pChildren = pMidWindow->node.pChildren; -// pMidWindow->node.pChildren = NULL; -// -// SWindowLogicNode* pPartWin = (SWindowLogicNode*)nodesCloneNode((SNode*)pMidWindow); -// if (NULL == pPartWin) { -// return TSDB_CODE_OUT_OF_MEMORY; -// } -// -// pPartWin->node.pChildren = pChildren; -// splSetParent((SLogicNode*)pPartWin); -// -// int32_t index = 0; -// int32_t code = stbSplRewriteFuns(pFunc, &pPartWin->pFuncs, &pMidWindow->pFuncs); -// if (TSDB_CODE_SUCCESS == code) { -// code = stbSplAppendWStart(pPartWin->pFuncs, &index, ((SColumnNode*)pMidWindow->pTspk)->node.resType.precision); -// } -// if (TSDB_CODE_SUCCESS == code) { -// code = createColumnByRewriteExprs(pPartWin->pFuncs, &pPartWin->node.pTargets); -// } -// -// if (TSDB_CODE_SUCCESS == code) { -// code = createColumnByRewriteExprs(pMidWindow->pFuncs, &pMidWindow->node.pTargets); -// } -// -// if (TSDB_CODE_SUCCESS == code) { -// nodesDestroyNode(pMidWindow->pTspk); -// pMidWindow->pTspk = nodesCloneNode(nodesListGetNode(pPartWin->node.pTargets, index)); -// if (NULL == pMidWindow->pTspk) { -// code = TSDB_CODE_OUT_OF_MEMORY; -// } -// } -// -// nodesDestroyList(pFunc); -// if (TSDB_CODE_SUCCESS == code) { -// *pSemiWindow = (SLogicNode*)pPartWin; -// } else { -// nodesDestroyNode((SNode*)pPartWin); -// } -// -// return code; -//} - static int32_t stbSplGetNumOfVgroups(SLogicNode* pNode) { if (QUERY_NODE_LOGIC_PLAN_SCAN == nodeType(pNode)) { return ((SScanLogicNode*)pNode)->pVgroupList->numOfVgroups; From de8ce3f7ec4cb575da8ca1bc9cda512636db13fd Mon Sep 17 00:00:00 2001 From: wangmm0220 Date: Wed, 20 Dec 2023 16:01:17 +0800 Subject: [PATCH 23/51] fix:send stream retrieve msg to source task --- include/libs/stream/tstream.h | 6 +- source/dnode/vnode/src/tqCommon/tqCommon.c | 16 ++- source/libs/stream/inc/streamInt.h | 2 +- source/libs/stream/src/stream.c | 24 ++-- source/libs/stream/src/streamDispatch.c | 128 ++++++++++++--------- source/libs/stream/src/streamExec.c | 2 +- 6 files changed, 101 insertions(+), 77 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index f6737b4e27..090a43fdd8 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -743,7 +743,6 @@ int32_t tEncodeStreamDispatchReq(SEncoder* pEncoder, const SStreamDispatchReq* p int32_t tDecodeStreamDispatchReq(SDecoder* pDecoder, SStreamDispatchReq* pReq); int32_t tDecodeStreamRetrieveReq(SDecoder* pDecoder, SStreamRetrieveReq* pReq); -void tDeleteStreamRetrieveReq(SStreamRetrieveReq* pReq); void tDeleteStreamDispatchReq(SStreamDispatchReq* pReq); int32_t streamSetupScheduleTrigger(SStreamTask* pTask); @@ -751,7 +750,7 @@ int32_t streamSetupScheduleTrigger(SStreamTask* pTask); int32_t streamProcessDispatchMsg(SStreamTask* pTask, SStreamDispatchReq* pReq, SRpcMsg* pMsg); int32_t streamProcessDispatchRsp(SStreamTask* pTask, SStreamDispatchRsp* pRsp, int32_t code); -int32_t streamProcessRetrieveReq(SStreamTask* pTask, SStreamRetrieveReq* pReq, SRpcMsg* pMsg); +int32_t streamProcessRetrieveReq(SStreamTask* pTask, SStreamRetrieveReq* pReq); SStreamChildEpInfo* streamTaskGetUpstreamTaskEpInfo(SStreamTask* pTask, int32_t taskId); void streamTaskInputFail(SStreamTask* pTask); @@ -868,6 +867,9 @@ int32_t buildCheckpointSourceRsp(SStreamCheckpointSourceReq* pReq, SRpcHandleInf SStreamTaskSM* streamCreateStateMachine(SStreamTask* pTask); void* streamDestroyStateMachine(SStreamTaskSM* pSM); + +int32_t broadcastRetrieveMsg(SStreamTask* pTask, SStreamRetrieveReq *req); +void sendRetrieveRsp(SStreamRetrieveReq *pReq, SRpcMsg* pRsp); #ifdef __cplusplus } #endif diff --git a/source/dnode/vnode/src/tqCommon/tqCommon.c b/source/dnode/vnode/src/tqCommon/tqCommon.c index b3c4479efc..64c4d2af5d 100644 --- a/source/dnode/vnode/src/tqCommon/tqCommon.c +++ b/source/dnode/vnode/src/tqCommon/tqCommon.c @@ -331,15 +331,25 @@ int32_t tqStreamTaskProcessRetrieveReq(SStreamMeta* pMeta, SRpcMsg* pMsg) { if (pTask == NULL) { tqError("vgId:%d process retrieve req, failed to acquire task:0x%x, it may have been dropped already", pMeta->vgId, req.dstTaskId); + taosMemoryFree(req.pRetrieve); return -1; } + int32_t code = 0; + if(pTask->info.taskLevel == TASK_LEVEL__SOURCE){ + code = streamProcessRetrieveReq(pTask, &req); + }else{ + req.srcNodeId = pTask->info.nodeId; + req.srcTaskId = pTask->id.taskId; + code = broadcastRetrieveMsg(pTask, &req); + } + SRpcMsg rsp = {.info = pMsg->info, .code = 0}; - streamProcessRetrieveReq(pTask, &req, &rsp); + sendRetrieveRsp(&req, &rsp); streamMetaReleaseTask(pMeta, pTask); - tDeleteStreamRetrieveReq(&req); - return 0; + taosMemoryFree(req.pRetrieve); + return code; } int32_t tqStreamTaskProcessScanHistoryFinishReq(SStreamMeta* pMeta, SRpcMsg* pMsg) { diff --git a/source/libs/stream/inc/streamInt.h b/source/libs/stream/inc/streamInt.h index f709741b57..0d4dbf1658 100644 --- a/source/libs/stream/inc/streamInt.h +++ b/source/libs/stream/inc/streamInt.h @@ -104,7 +104,7 @@ SStreamDataBlock* createStreamBlockFromResults(SStreamQueueItem* pItem, SStreamT void destroyStreamDataBlock(SStreamDataBlock* pBlock); int32_t streamRetrieveReqToData(const SStreamRetrieveReq* pReq, SStreamDataBlock* pData); -int32_t streamBroadcastToChildren(SStreamTask* pTask, const SSDataBlock* pBlock); +int32_t streamBroadcastToUpTasks(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 1bef42bf14..e50e373be0 100644 --- a/source/libs/stream/src/stream.c +++ b/source/libs/stream/src/stream.c @@ -172,7 +172,7 @@ static int32_t streamTaskAppendInputBlocks(SStreamTask* pTask, const SStreamDisp return status; } -int32_t streamTaskEnqueueRetrieve(SStreamTask* pTask, SStreamRetrieveReq* pReq, SRpcMsg* pRsp) { +int32_t streamTaskEnqueueRetrieve(SStreamTask* pTask, SStreamRetrieveReq* pReq) { SStreamDataBlock* pData = taosAllocateQitem(sizeof(SStreamDataBlock), DEF_QITEM, sizeof(SStreamDataBlock)); int8_t status = TASK_INPUT_STATUS__NORMAL; @@ -194,17 +194,6 @@ int32_t streamTaskEnqueueRetrieve(SStreamTask* pTask, SStreamRetrieveReq* pReq, /*status = TASK_INPUT_STATUS__FAILED;*/ } - // rsp by input status - void* buf = rpcMallocCont(sizeof(SMsgHead) + sizeof(SStreamRetrieveRsp)); - ((SMsgHead*)buf)->vgId = htonl(pReq->srcNodeId); - SStreamRetrieveRsp* pCont = POINTER_SHIFT(buf, sizeof(SMsgHead)); - pCont->streamId = pReq->streamId; - pCont->rspToTaskId = pReq->srcTaskId; - pCont->rspFromTaskId = pReq->dstTaskId; - pRsp->pCont = buf; - pRsp->contLen = sizeof(SMsgHead) + sizeof(SStreamRetrieveRsp); - tmsgSendRsp(pRsp); - return status == TASK_INPUT_STATUS__NORMAL ? 0 : -1; } @@ -269,11 +258,12 @@ int32_t streamProcessDispatchMsg(SStreamTask* pTask, SStreamDispatchReq* pReq, S return 0; } -int32_t streamProcessRetrieveReq(SStreamTask* pTask, SStreamRetrieveReq* pReq, SRpcMsg* pRsp) { - streamTaskEnqueueRetrieve(pTask, pReq, pRsp); - ASSERT(pTask->info.taskLevel != TASK_LEVEL__SINK); - streamSchedExec(pTask); - return 0; +int32_t streamProcessRetrieveReq(SStreamTask* pTask, SStreamRetrieveReq* pReq) { + int32_t code = streamTaskEnqueueRetrieve(pTask, pReq); + if(code != 0){ + return code; + } + return streamSchedExec(pTask); } void streamTaskInputFail(SStreamTask* pTask) { atomic_store_8(&pTask->inputq.status, TASK_INPUT_STATUS__FAILED); } diff --git a/source/libs/stream/src/streamDispatch.c b/source/libs/stream/src/streamDispatch.c index 5fb7db233f..1e037ee879 100644 --- a/source/libs/stream/src/streamDispatch.c +++ b/source/libs/stream/src/streamDispatch.c @@ -162,16 +162,71 @@ int32_t tDecodeStreamRetrieveReq(SDecoder* pDecoder, SStreamRetrieveReq* pReq) { return 0; } -void tDeleteStreamRetrieveReq(SStreamRetrieveReq* pReq) { taosMemoryFree(pReq->pRetrieve); } +void sendRetrieveRsp(SStreamRetrieveReq *pReq, SRpcMsg* pRsp){ + void* buf = rpcMallocCont(sizeof(SMsgHead) + sizeof(SStreamRetrieveRsp)); + ((SMsgHead*)buf)->vgId = htonl(pReq->srcNodeId); + SStreamRetrieveRsp* pCont = POINTER_SHIFT(buf, sizeof(SMsgHead)); + pCont->streamId = pReq->streamId; + pCont->rspToTaskId = pReq->srcTaskId; + pCont->rspFromTaskId = pReq->dstTaskId; + pRsp->pCont = buf; + pRsp->contLen = sizeof(SMsgHead) + sizeof(SStreamRetrieveRsp); + tmsgSendRsp(pRsp); +} -int32_t streamBroadcastToChildren(SStreamTask* pTask, const SSDataBlock* pBlock) { - int32_t code = -1; +int32_t broadcastRetrieveMsg(SStreamTask* pTask, SStreamRetrieveReq *req){ + int32_t code = 0; + void* buf = NULL; + int32_t sz = taosArrayGetSize(pTask->upstreamInfo.pList); + ASSERT(sz > 0); + for (int32_t i = 0; i < sz; i++) { + req->reqId = tGenIdPI64(); + SStreamChildEpInfo* pEpInfo = taosArrayGetP(pTask->upstreamInfo.pList, i); + req->dstNodeId = pEpInfo->nodeId; + req->dstTaskId = pEpInfo->taskId; + int32_t len; + tEncodeSize(tEncodeStreamRetrieveReq, req, len, code); + if (code != 0) { + ASSERT(0); + return code; + } + + buf = rpcMallocCont(sizeof(SMsgHead) + len); + if (buf == NULL) { + code = TSDB_CODE_OUT_OF_MEMORY; + return code; + } + + ((SMsgHead*)buf)->vgId = htonl(pEpInfo->nodeId); + void* abuf = POINTER_SHIFT(buf, sizeof(SMsgHead)); + SEncoder encoder; + tEncoderInit(&encoder, abuf, len); + tEncodeStreamRetrieveReq(&encoder, req); + tEncoderClear(&encoder); + + SRpcMsg rpcMsg = {0}; + initRpcMsg(&rpcMsg, TDMT_STREAM_RETRIEVE, buf, len + sizeof(SMsgHead)); + + code = tmsgSendReq(&pEpInfo->epSet, &rpcMsg); + if (code != 0) { + ASSERT(0); + rpcFreeCont(buf); + return code; + } + + buf = NULL; + stDebug("s-task:%s (child %d) send retrieve req to task:0x%x (vgId:%d), reqId:0x%" PRIx64, pTask->id.idStr, + pTask->info.selfChildId, pEpInfo->taskId, pEpInfo->nodeId, req->reqId); + } + return code; +} + +static int32_t buildStreamRetrieveReq(SStreamTask* pTask, const SSDataBlock* pBlock, SStreamRetrieveReq* req){ SRetrieveTableRsp* pRetrieve = NULL; - void* buf = NULL; int32_t dataStrLen = sizeof(SRetrieveTableRsp) + blockGetEncodeSize(pBlock); pRetrieve = taosMemoryCalloc(1, dataStrLen); - if (pRetrieve == NULL) return -1; + if (pRetrieve == NULL) return TSDB_CODE_OUT_OF_MEMORY; int32_t numOfCols = taosArrayGetSize(pBlock->pDataBlock); pRetrieve->useconds = 0; @@ -187,57 +242,24 @@ int32_t streamBroadcastToChildren(SStreamTask* pTask, const SSDataBlock* pBlock) int32_t actualLen = blockEncode(pBlock, pRetrieve->data, numOfCols); - SStreamRetrieveReq req = { - .streamId = pTask->id.streamId, - .srcNodeId = pTask->info.nodeId, - .srcTaskId = pTask->id.taskId, - .pRetrieve = pRetrieve, - .retrieveLen = dataStrLen, - }; + req->streamId = pTask->id.streamId; + req->srcNodeId = pTask->info.nodeId; + req->srcTaskId = pTask->id.taskId; + req->pRetrieve = pRetrieve; + req->retrieveLen = dataStrLen; + return 0; +} - int32_t sz = taosArrayGetSize(pTask->upstreamInfo.pList); - ASSERT(sz > 0); - for (int32_t i = 0; i < sz; i++) { - req.reqId = tGenIdPI64(); - SStreamChildEpInfo* pEpInfo = taosArrayGetP(pTask->upstreamInfo.pList, i); - req.dstNodeId = pEpInfo->nodeId; - req.dstTaskId = pEpInfo->taskId; - int32_t len; - tEncodeSize(tEncodeStreamRetrieveReq, &req, len, code); - if (code < 0) { - ASSERT(0); - return -1; - } - - buf = rpcMallocCont(sizeof(SMsgHead) + len); - if (buf == NULL) { - goto CLEAR; - } - - ((SMsgHead*)buf)->vgId = htonl(pEpInfo->nodeId); - void* abuf = POINTER_SHIFT(buf, sizeof(SMsgHead)); - SEncoder encoder; - tEncoderInit(&encoder, abuf, len); - tEncodeStreamRetrieveReq(&encoder, &req); - tEncoderClear(&encoder); - - SRpcMsg rpcMsg = {0}; - initRpcMsg(&rpcMsg, TDMT_STREAM_RETRIEVE, buf, len + sizeof(SMsgHead)); - - if (tmsgSendReq(&pEpInfo->epSet, &rpcMsg) < 0) { - ASSERT(0); - goto CLEAR; - } - - buf = NULL; - stDebug("s-task:%s (child %d) send retrieve req to task:0x%x (vgId:%d), reqId:0x%" PRIx64, pTask->id.idStr, - pTask->info.selfChildId, pEpInfo->taskId, pEpInfo->nodeId, req.reqId); +int32_t streamBroadcastToUpTasks(SStreamTask* pTask, const SSDataBlock* pBlock) { + SStreamRetrieveReq req; + int32_t code = buildStreamRetrieveReq(pTask, pBlock, &req); + if(code != 0){ + return code; } - code = 0; -CLEAR: - taosMemoryFree(pRetrieve); - rpcFreeCont(buf); + code = broadcastRetrieveMsg(pTask, &req); + taosMemoryFree(req.pRetrieve); + return code; } diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index 25f32195be..f7e4b7856d 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -144,7 +144,7 @@ static int32_t streamTaskExecImpl(SStreamTask* pTask, SStreamQueueItem* pItem, i } if (output->info.type == STREAM_RETRIEVE) { - if (streamBroadcastToChildren(pTask, output) < 0) { + if (streamBroadcastToUpTasks(pTask, output) < 0) { // TODO } continue; From 0eec9d5fbdf5c4548698386ce343298285bff825 Mon Sep 17 00:00:00 2001 From: wangmm0220 Date: Wed, 20 Dec 2023 16:09:36 +0800 Subject: [PATCH 24/51] fix:deal with stram_pull_over block in middle interval --- source/libs/executor/src/streamtimewindowoperator.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/libs/executor/src/streamtimewindowoperator.c b/source/libs/executor/src/streamtimewindowoperator.c index 839d315f16..baa88259f5 100644 --- a/source/libs/executor/src/streamtimewindowoperator.c +++ b/source/libs/executor/src/streamtimewindowoperator.c @@ -4254,7 +4254,7 @@ static SSDataBlock* doStreamMidIntervalAgg(SOperatorInfo* pOperator) { return pInfo->pDelRes; } break; - } else if (pBlock->info.type == STREAM_CREATE_CHILD_TABLE) { + } else if (pBlock->info.type == STREAM_CREATE_CHILD_TABLE || pBlock->info.type == STREAM_PULL_OVER) { return pBlock; } else { ASSERTS(pBlock->info.type == STREAM_INVALID, "invalid SSDataBlock type"); From a387e7e2d33902e47c7c79dd95821f4d0d6dd087 Mon Sep 17 00:00:00 2001 From: 54liuyao <54liuyao> Date: Wed, 20 Dec 2023 18:50:55 +0800 Subject: [PATCH 25/51] mid operator checkpoint --- source/libs/executor/src/streamtimewindowoperator.c | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/source/libs/executor/src/streamtimewindowoperator.c b/source/libs/executor/src/streamtimewindowoperator.c index baa88259f5..8c591702c1 100644 --- a/source/libs/executor/src/streamtimewindowoperator.c +++ b/source/libs/executor/src/streamtimewindowoperator.c @@ -4256,6 +4256,11 @@ static SSDataBlock* doStreamMidIntervalAgg(SOperatorInfo* pOperator) { break; } else if (pBlock->info.type == STREAM_CREATE_CHILD_TABLE || pBlock->info.type == STREAM_PULL_OVER) { return pBlock; + } else if (pBlock->info.type == STREAM_CHECKPOINT) { + pAPI->stateStore.streamStateCommit(pInfo->pState); + doStreamIntervalSaveCheckpoint(pOperator); + copyDataBlock(pInfo->pCheckpointRes, pBlock); + continue; } else { ASSERTS(pBlock->info.type == STREAM_INVALID, "invalid SSDataBlock type"); } From 246ad22e0ba4febf1491dc5a5d86e2972dd36fba Mon Sep 17 00:00:00 2001 From: 54liuyao <54liuyao> Date: Thu, 21 Dec 2023 14:34:37 +0800 Subject: [PATCH 26/51] mid operator igore expired data --- source/libs/executor/src/streamtimewindowoperator.c | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/source/libs/executor/src/streamtimewindowoperator.c b/source/libs/executor/src/streamtimewindowoperator.c index 8c591702c1..8870a5b1c4 100644 --- a/source/libs/executor/src/streamtimewindowoperator.c +++ b/source/libs/executor/src/streamtimewindowoperator.c @@ -4113,8 +4113,7 @@ static void doStreamMidIntervalAggImpl(SOperatorInfo* pOperator, SSDataBlock* pS STimeWindow nextWin = getFinalTimeWindow(ts, &pInfo->interval); while (1) { - bool isClosed = isCloseWindow(&nextWin, &pInfo->twAggSup); - if ((pInfo->ignoreExpiredData && isClosed) || !inSlidingWindow(&pInfo->interval, &nextWin, &pSDataBlock->info)) { + if (!inSlidingWindow(&pInfo->interval, &nextWin, &pSDataBlock->info)) { startPos = getNexWindowPos(&pInfo->interval, &pSDataBlock->info, tsCol, startPos, nextWin.ekey, &nextWin); if (startPos < 0) { break; From 036fa8d0d951c635960dd488d4427bfd93a735f0 Mon Sep 17 00:00:00 2001 From: 54liuyao <54liuyao> Date: Thu, 21 Dec 2023 19:51:09 +0800 Subject: [PATCH 27/51] sliding window --- source/libs/executor/src/streamtimewindowoperator.c | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/source/libs/executor/src/streamtimewindowoperator.c b/source/libs/executor/src/streamtimewindowoperator.c index 8870a5b1c4..919b6545dc 100644 --- a/source/libs/executor/src/streamtimewindowoperator.c +++ b/source/libs/executor/src/streamtimewindowoperator.c @@ -28,6 +28,7 @@ #include "ttime.h" #define IS_FINAL_INTERVAL_OP(op) ((op)->operatorType == QUERY_NODE_PHYSICAL_PLAN_STREAM_FINAL_INTERVAL) +#define IS_MID_INTERVAL_OP(op) ((op)->operatorType == QUERY_NODE_PHYSICAL_PLAN_STREAM_MID_INTERVAL) #define IS_FINAL_SESSION_OP(op) ((op)->operatorType == QUERY_NODE_PHYSICAL_PLAN_STREAM_FINAL_SESSION) #define DEAULT_DELETE_MARK INT64_MAX #define STREAM_INTERVAL_OP_STATE_NAME "StreamIntervalHistoryState" @@ -237,7 +238,7 @@ static void doDeleteWindows(SOperatorInfo* pOperator, SInterval* pInterval, SSDa dumyInfo.cur.pageId = -1; STimeWindow win = {0}; - if (IS_FINAL_INTERVAL_OP(pOperator)) { + if (IS_FINAL_INTERVAL_OP(pOperator) || IS_MID_INTERVAL_OP(pOperator)) { win.skey = startTsCols[i]; win.ekey = endTsCols[i]; } else { From 902504b39b14343b30b4f7fc83c6c9007e6d0d24 Mon Sep 17 00:00:00 2001 From: wangmm0220 Date: Fri, 22 Dec 2023 15:28:02 +0800 Subject: [PATCH 28/51] fix:partical func parameters for middle interval[checkStreamSTable1.sim] --- source/dnode/mnode/impl/src/mndScheduler.c | 5 ++-- source/libs/planner/src/planSpliter.c | 23 +++++++++++++++++-- .../system-test/8-stream/stream_multi_agg.py | 8 +++---- 3 files changed, 28 insertions(+), 8 deletions(-) diff --git a/source/dnode/mnode/impl/src/mndScheduler.c b/source/dnode/mnode/impl/src/mndScheduler.c index 2cf5330f09..a4721b8a11 100644 --- a/source/dnode/mnode/impl/src/mndScheduler.c +++ b/source/dnode/mnode/impl/src/mndScheduler.c @@ -556,12 +556,13 @@ static void bindTwoLevel(SArray* tasks, int32_t begin, int32_t end) { SArray* pUpTaskList = taosArrayGetP(tasks, size - 2); SStreamTask** pDownTask = taosArrayGetLast(pDownTaskList); - for(int i = begin; i < end && i < taosArrayGetSize(pUpTaskList); i++){ + end = end > taosArrayGetSize(pUpTaskList) ? taosArrayGetSize(pUpTaskList): end; + for(int i = begin; i < end; i++){ SStreamTask* pUpTask = taosArrayGetP(pUpTaskList, i); streamTaskSetFixedDownstreamInfo(pUpTask, *pDownTask); streamTaskSetUpstreamInfo(*pDownTask, pUpTask); } - mDebug("bindTwoLevel task list(%d-%d) to taskId:%s", begin, end, (*(pDownTask))->id.idStr); + mDebug("bindTwoLevel task list(%d-%d) to taskId:%s", begin, end - 1, (*(pDownTask))->id.idStr); } static int32_t doScheduleStream(SStreamObj* pStream, SMnode* pMnode, SQueryPlan* pPlan, int64_t nextWindowSkey, SEpSet* pEpset) { diff --git a/source/libs/planner/src/planSpliter.c b/source/libs/planner/src/planSpliter.c index dc0e7da6a1..d1f9901b30 100644 --- a/source/libs/planner/src/planSpliter.c +++ b/source/libs/planner/src/planSpliter.c @@ -488,6 +488,16 @@ static int32_t stbSplCreatePartWindowNode(SWindowLogicNode* pMergeWindow, SLogic return code; } +static SNode* createColumnByFunc(const SFunctionNode* pFunc) { + SColumnNode* pCol = (SColumnNode*)nodesMakeNode(QUERY_NODE_COLUMN); + if (NULL == pCol) { + return NULL; + } + strcpy(pCol->colName, pFunc->node.aliasName); + pCol->node.resType = pFunc->node.resType; + return (SNode*)pCol; +} + static int32_t stbSplCreatePartMidWindowNode(SWindowLogicNode* pMergeWindow, SLogicNode** pPartWindow, SLogicNode** pMidWindow) { SNodeList* pFunc = pMergeWindow->pFuncs; pMergeWindow->pFuncs = NULL; @@ -521,7 +531,7 @@ static int32_t stbSplCreatePartMidWindowNode(SWindowLogicNode* pMergeWindow, SLo int32_t code = stbSplRewriteFuns(pFunc, &pFuncPart, &pFuncMerge); pPartWin->pFuncs = pFuncPart; pMergeWindow->pFuncs = pFuncMerge; - pMidWin->pFuncs = nodesCloneList(pFuncMerge); + pMidWin->pFuncs = nodesCloneList(pFuncPart); int32_t index = 0; if (TSDB_CODE_SUCCESS == code) { @@ -543,7 +553,16 @@ static int32_t stbSplCreatePartMidWindowNode(SWindowLogicNode* pMergeWindow, SLo for (int32_t i = 0; i < LIST_LENGTH(pMidWin->pFuncs); ++i) { SFunctionNode* pFunc1 = (SFunctionNode*)nodesListGetNode(pPartWin->pFuncs, i); SFunctionNode* pFunc2 = (SFunctionNode*)nodesListGetNode(pMidWin->pFuncs, i); - strcpy(pFunc2->node.aliasName, pFunc1->node.aliasName); + NODES_DESTORY_LIST(pFunc2->pParameterList); + + SNodeList* pParameterList = NULL; + SNode* pRes = createColumnByFunc(pFunc1); + code = nodesListMakeStrictAppend(&pParameterList, pRes); + if(code == TSDB_CODE_SUCCESS){ + pFunc2->pParameterList = pParameterList; + }else{ + nodesDestroyNode(pRes); + } } } diff --git a/tests/system-test/8-stream/stream_multi_agg.py b/tests/system-test/8-stream/stream_multi_agg.py index 5c478d7e14..92ee4540e4 100644 --- a/tests/system-test/8-stream/stream_multi_agg.py +++ b/tests/system-test/8-stream/stream_multi_agg.py @@ -37,11 +37,11 @@ class TDTestCase: def case1(self): tdLog.debug("========case1 start========") - os.system("nohup taosBenchmark -y -B 1 -t 40 -S 1000 -n 10 -i 1000 -v 12 > /dev/null 2>&1 &") - time.sleep(4) + os.system("nohup taosBenchmark -y -B 1 -t 40 -S 1000 -n 10 -i 1000 -v 5 > /dev/null 2>&1 &") + time.sleep(10) tdSql.query("use test") tdSql.query("create stream if not exists s1 trigger at_once ignore expired 0 ignore update 0 fill_history 1 into st1 as select _wstart,sum(voltage),groupid from meters partition by groupid interval(2s)") - tdLog.debug("========create stream useing snode and insert data ok========") + tdLog.debug("========create stream and insert data ok========") time.sleep(15) tdSql.query("select _wstart,sum(voltage),groupid from meters partition by groupid interval(2s) order by groupid,_wstart") @@ -64,7 +64,7 @@ class TDTestCase: def case2(self): tdLog.debug("========case2 start========") - os.system("taosBenchmark -d db -t 20 -v 12 -n 1000 -y > /dev/null 2>&1") + os.system("taosBenchmark -d db -t 20 -v 6 -n 1000 -y > /dev/null 2>&1") # create stream tdSql.execute("use db") tdSql.execute("create stream stream1 fill_history 1 into sta as select count(*) as cnt from meters interval(10a);",show=True) From eac86f72b3538322569bb5337f6e95db02a93e44 Mon Sep 17 00:00:00 2001 From: wangmm0220 Date: Fri, 22 Dec 2023 18:30:22 +0800 Subject: [PATCH 29/51] fix:partical func parameters for middle interval[checkStreamSTable1.sim] --- include/libs/function/functionMgt.h | 2 +- source/libs/function/inc/builtins.h | 1 + source/libs/function/src/builtins.c | 28 ++++++++++++ source/libs/function/src/functionMgt.c | 35 ++++++++++++++- source/libs/planner/src/planSpliter.c | 60 +++++++++++--------------- 5 files changed, 89 insertions(+), 37 deletions(-) diff --git a/include/libs/function/functionMgt.h b/include/libs/function/functionMgt.h index 865f1b2295..878990425b 100644 --- a/include/libs/function/functionMgt.h +++ b/include/libs/function/functionMgt.h @@ -243,7 +243,7 @@ bool fmIsSkipScanCheckFunc(int32_t funcId); void getLastCacheDataType(SDataType* pType); SFunctionNode* createFunction(const char* pName, SNodeList* pParameterList); -int32_t fmGetDistMethod(const SFunctionNode* pFunc, SFunctionNode** pPartialFunc, SFunctionNode** pMergeFunc); +int32_t fmGetDistMethod(const SFunctionNode* pFunc, SFunctionNode** pPartialFunc, SFunctionNode** pMidFunc, SFunctionNode** pMergeFunc); typedef enum EFuncDataRequired { FUNC_DATA_REQUIRED_DATA_LOAD = 1, diff --git a/source/libs/function/inc/builtins.h b/source/libs/function/inc/builtins.h index e7fcc38818..b13d481254 100644 --- a/source/libs/function/inc/builtins.h +++ b/source/libs/function/inc/builtins.h @@ -43,6 +43,7 @@ typedef struct SBuiltinFuncDefinition { FExecProcess invertFunc; FExecCombine combineFunc; const char* pPartialFunc; + const char* pMiddleFunc; const char* pMergeFunc; FCreateMergeFuncParameters createMergeParaFuc; FEstimateReturnRows estimateReturnRowsFunc; diff --git a/source/libs/function/src/builtins.c b/source/libs/function/src/builtins.c index 98fda024fa..9546733c7e 100644 --- a/source/libs/function/src/builtins.c +++ b/source/libs/function/src/builtins.c @@ -422,6 +422,20 @@ static int32_t translateAvgPartial(SFunctionNode* pFunc, char* pErrBuf, int32_t return TSDB_CODE_SUCCESS; } +static int32_t translateAvgMiddle(SFunctionNode* pFunc, char* pErrBuf, int32_t len) { + if (1 != LIST_LENGTH(pFunc->pParameterList)) { + return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName); + } + + uint8_t paraType = ((SExprNode*)nodesListGetNode(pFunc->pParameterList, 0))->resType.type; + if (TSDB_DATA_TYPE_BINARY != paraType) { + return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName); + } + + pFunc->node.resType = (SDataType){.bytes = getAvgInfoSize() + VARSTR_HEADER_SIZE, .type = TSDB_DATA_TYPE_BINARY}; + return TSDB_CODE_SUCCESS; +} + static int32_t translateAvgMerge(SFunctionNode* pFunc, char* pErrBuf, int32_t len) { if (1 != LIST_LENGTH(pFunc->pParameterList)) { return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName); @@ -2485,6 +2499,7 @@ const SBuiltinFuncDefinition funcMgtBuiltins[] = { .invertFunc = avgInvertFunction, .combineFunc = avgCombine, .pPartialFunc = "_avg_partial", + .pMiddleFunc = "_avg_middle", .pMergeFunc = "_avg_merge" }, { @@ -2500,6 +2515,19 @@ const SBuiltinFuncDefinition funcMgtBuiltins[] = { .invertFunc = avgInvertFunction, .combineFunc = avgCombine, }, + { + .name = "_avg_middle", + .type = FUNCTION_TYPE_AVG_PARTIAL, + .classification = FUNC_MGT_AGG_FUNC, + .translateFunc = translateAvgMiddle, + .dataRequiredFunc = statisDataRequired, + .getEnvFunc = getAvgFuncEnv, + .initFunc = avgFunctionSetup, + .processFunc = avgFunctionMerge, + .finalizeFunc = avgPartialFinalize, + .invertFunc = avgInvertFunction, + .combineFunc = avgCombine, + }, { .name = "_avg_merge", .type = FUNCTION_TYPE_AVG_MERGE, diff --git a/source/libs/function/src/functionMgt.c b/source/libs/function/src/functionMgt.c index 036e4238d4..1f3fb2e943 100644 --- a/source/libs/function/src/functionMgt.c +++ b/source/libs/function/src/functionMgt.c @@ -422,6 +422,35 @@ static int32_t createMergeFuncPara(const SFunctionNode* pSrcFunc, const SFunctio } } +static int32_t createMidFunction(const SFunctionNode* pSrcFunc, const SFunctionNode* pPartialFunc, + SFunctionNode** pMidFunc) { + SNodeList* pParameterList = NULL; + SFunctionNode* pFunc = NULL; + + int32_t code = createMergeFuncPara(pSrcFunc, pPartialFunc, &pParameterList); + if (TSDB_CODE_SUCCESS == code) { + if(funcMgtBuiltins[pSrcFunc->funcId].pMiddleFunc != NULL){ + pFunc = createFunction(funcMgtBuiltins[pSrcFunc->funcId].pMiddleFunc, pParameterList); + }else{ + pFunc = createFunction(funcMgtBuiltins[pSrcFunc->funcId].pMergeFunc, pParameterList); + } + if (NULL == pFunc) { + code = TSDB_CODE_OUT_OF_MEMORY; + } + } + if (TSDB_CODE_SUCCESS == code) { + strcpy(pFunc->node.aliasName, pPartialFunc->node.aliasName); + } + + if (TSDB_CODE_SUCCESS == code) { + *pMidFunc = pFunc; + } else { + nodesDestroyList(pParameterList); + } + + return code; +} + static int32_t createMergeFunction(const SFunctionNode* pSrcFunc, const SFunctionNode* pPartialFunc, SFunctionNode** pMergeFunc) { SNodeList* pParameterList = NULL; @@ -451,18 +480,22 @@ static int32_t createMergeFunction(const SFunctionNode* pSrcFunc, const SFunctio return code; } -int32_t fmGetDistMethod(const SFunctionNode* pFunc, SFunctionNode** pPartialFunc, SFunctionNode** pMergeFunc) { +int32_t fmGetDistMethod(const SFunctionNode* pFunc, SFunctionNode** pPartialFunc, SFunctionNode** pMidFunc, SFunctionNode** pMergeFunc) { if (!fmIsDistExecFunc(pFunc->funcId)) { return TSDB_CODE_FAILED; } int32_t code = createPartialFunction(pFunc, pPartialFunc); + if (TSDB_CODE_SUCCESS == code) { + code = createMidFunction(pFunc, *pPartialFunc, pMidFunc); + } if (TSDB_CODE_SUCCESS == code) { code = createMergeFunction(pFunc, *pPartialFunc, pMergeFunc); } if (TSDB_CODE_SUCCESS != code) { nodesDestroyNode((SNode*)*pPartialFunc); + nodesDestroyNode((SNode*)*pMidFunc); nodesDestroyNode((SNode*)*pMergeFunc); } diff --git a/source/libs/planner/src/planSpliter.c b/source/libs/planner/src/planSpliter.c index d1f9901b30..987a1dc051 100644 --- a/source/libs/planner/src/planSpliter.c +++ b/source/libs/planner/src/planSpliter.c @@ -344,11 +344,12 @@ static bool stbSplFindSplitNode(SSplitContext* pCxt, SLogicSubplan* pSubplan, SL return false; } -static int32_t stbSplRewriteFuns(const SNodeList* pFuncs, SNodeList** pPartialFuncs, SNodeList** pMergeFuncs) { +static int32_t stbSplRewriteFuns(const SNodeList* pFuncs, SNodeList** pPartialFuncs, SNodeList** pMidFuncs, SNodeList** pMergeFuncs) { SNode* pNode = NULL; FOREACH(pNode, pFuncs) { SFunctionNode* pFunc = (SFunctionNode*)pNode; SFunctionNode* pPartFunc = NULL; + SFunctionNode* pMidFunc = NULL; SFunctionNode* pMergeFunc = NULL; int32_t code = TSDB_CODE_SUCCESS; if (fmIsWindowPseudoColumnFunc(pFunc->funcId)) { @@ -359,18 +360,33 @@ static int32_t stbSplRewriteFuns(const SNodeList* pFuncs, SNodeList** pPartialFu nodesDestroyNode((SNode*)pMergeFunc); code = TSDB_CODE_OUT_OF_MEMORY; } + if(pMidFuncs != NULL){ + pMidFunc = (SFunctionNode*)nodesCloneNode(pNode); + if (NULL == pMidFunc) { + nodesDestroyNode((SNode*)pMidFunc); + code = TSDB_CODE_OUT_OF_MEMORY; + } + } } else { - code = fmGetDistMethod(pFunc, &pPartFunc, &pMergeFunc); + code = fmGetDistMethod(pFunc, &pPartFunc, &pMidFunc, &pMergeFunc); } if (TSDB_CODE_SUCCESS == code) { code = nodesListMakeStrictAppend(pPartialFuncs, (SNode*)pPartFunc); } + if (TSDB_CODE_SUCCESS == code) { + if(pMidFuncs != NULL){ + code = nodesListMakeStrictAppend(pMidFuncs, (SNode*)pMidFunc); + }else{ + nodesDestroyNode((SNode*)pMidFunc); + } + } if (TSDB_CODE_SUCCESS == code) { code = nodesListMakeStrictAppend(pMergeFuncs, (SNode*)pMergeFunc); } if (TSDB_CODE_SUCCESS != code) { - nodesDestroyList(*pPartialFuncs); - nodesDestroyList(*pMergeFuncs); + nodesDestroyNode((SNode*)pPartFunc); + nodesDestroyNode((SNode*)pMidFunc); + nodesDestroyNode((SNode*)pMergeFunc); return code; } } @@ -463,7 +479,7 @@ static int32_t stbSplCreatePartWindowNode(SWindowLogicNode* pMergeWindow, SLogic splSetParent((SLogicNode*)pPartWin); int32_t index = 0; - int32_t code = stbSplRewriteFuns(pFunc, &pPartWin->pFuncs, &pMergeWindow->pFuncs); + int32_t code = stbSplRewriteFuns(pFunc, &pPartWin->pFuncs, NULL, &pMergeWindow->pFuncs); if (TSDB_CODE_SUCCESS == code) { code = stbSplAppendWStart(pPartWin->pFuncs, &index, ((SColumnNode*)pMergeWindow->pTspk)->node.resType.precision); } @@ -488,16 +504,6 @@ static int32_t stbSplCreatePartWindowNode(SWindowLogicNode* pMergeWindow, SLogic return code; } -static SNode* createColumnByFunc(const SFunctionNode* pFunc) { - SColumnNode* pCol = (SColumnNode*)nodesMakeNode(QUERY_NODE_COLUMN); - if (NULL == pCol) { - return NULL; - } - strcpy(pCol->colName, pFunc->node.aliasName); - pCol->node.resType = pFunc->node.resType; - return (SNode*)pCol; -} - static int32_t stbSplCreatePartMidWindowNode(SWindowLogicNode* pMergeWindow, SLogicNode** pPartWindow, SLogicNode** pMidWindow) { SNodeList* pFunc = pMergeWindow->pFuncs; pMergeWindow->pFuncs = NULL; @@ -527,11 +533,12 @@ static int32_t stbSplCreatePartMidWindowNode(SWindowLogicNode* pMergeWindow, SLo splSetParent((SLogicNode*)pPartWin); SNodeList* pFuncPart = NULL; + SNodeList* pFuncMid = NULL; SNodeList* pFuncMerge = NULL; - int32_t code = stbSplRewriteFuns(pFunc, &pFuncPart, &pFuncMerge); + int32_t code = stbSplRewriteFuns(pFunc, &pFuncPart, &pFuncMid, &pFuncMerge); pPartWin->pFuncs = pFuncPart; + pMidWin->pFuncs = pFuncMid; pMergeWindow->pFuncs = pFuncMerge; - pMidWin->pFuncs = nodesCloneList(pFuncPart); int32_t index = 0; if (TSDB_CODE_SUCCESS == code) { @@ -549,23 +556,6 @@ static int32_t stbSplCreatePartMidWindowNode(SWindowLogicNode* pMergeWindow, SLo } } - if(TSDB_CODE_SUCCESS == code){ - for (int32_t i = 0; i < LIST_LENGTH(pMidWin->pFuncs); ++i) { - SFunctionNode* pFunc1 = (SFunctionNode*)nodesListGetNode(pPartWin->pFuncs, i); - SFunctionNode* pFunc2 = (SFunctionNode*)nodesListGetNode(pMidWin->pFuncs, i); - NODES_DESTORY_LIST(pFunc2->pParameterList); - - SNodeList* pParameterList = NULL; - SNode* pRes = createColumnByFunc(pFunc1); - code = nodesListMakeStrictAppend(&pParameterList, pRes); - if(code == TSDB_CODE_SUCCESS){ - pFunc2->pParameterList = pParameterList; - }else{ - nodesDestroyNode(pRes); - } - } - } - if (TSDB_CODE_SUCCESS == code) { code = stbSplAppendWStart(pMidWin->pFuncs, &index, ((SColumnNode*)pMergeWindow->pTspk)->node.resType.precision); } @@ -978,7 +968,7 @@ static int32_t stbSplCreatePartAggNode(SAggLogicNode* pMergeAgg, SLogicNode** pO pPartAgg->node.pChildren = pChildren; splSetParent((SLogicNode*)pPartAgg); - code = stbSplRewriteFuns(pFunc, &pPartAgg->pAggFuncs, &pMergeAgg->pAggFuncs); + code = stbSplRewriteFuns(pFunc, &pPartAgg->pAggFuncs, NULL, &pMergeAgg->pAggFuncs); } if (TSDB_CODE_SUCCESS == code) { code = createColumnByRewriteExprs(pPartAgg->pAggFuncs, &pPartAgg->node.pTargets); From 5367a430c6aa5b95bc758c2a531a96d2883d7a98 Mon Sep 17 00:00:00 2001 From: wangmm0220 Date: Thu, 28 Dec 2023 16:07:19 +0800 Subject: [PATCH 30/51] fix:add function to the end --- source/libs/function/src/builtins.c | 27 +++++++++++++-------------- 1 file changed, 13 insertions(+), 14 deletions(-) diff --git a/source/libs/function/src/builtins.c b/source/libs/function/src/builtins.c index 9546733c7e..535b4bf640 100644 --- a/source/libs/function/src/builtins.c +++ b/source/libs/function/src/builtins.c @@ -2515,19 +2515,6 @@ const SBuiltinFuncDefinition funcMgtBuiltins[] = { .invertFunc = avgInvertFunction, .combineFunc = avgCombine, }, - { - .name = "_avg_middle", - .type = FUNCTION_TYPE_AVG_PARTIAL, - .classification = FUNC_MGT_AGG_FUNC, - .translateFunc = translateAvgMiddle, - .dataRequiredFunc = statisDataRequired, - .getEnvFunc = getAvgFuncEnv, - .initFunc = avgFunctionSetup, - .processFunc = avgFunctionMerge, - .finalizeFunc = avgPartialFinalize, - .invertFunc = avgInvertFunction, - .combineFunc = avgCombine, - }, { .name = "_avg_merge", .type = FUNCTION_TYPE_AVG_MERGE, @@ -3737,7 +3724,19 @@ const SBuiltinFuncDefinition funcMgtBuiltins[] = { .sprocessFunc = toCharFunction, .finalizeFunc = NULL }, - + { + .name = "_avg_middle", + .type = FUNCTION_TYPE_AVG_PARTIAL, + .classification = FUNC_MGT_AGG_FUNC, + .translateFunc = translateAvgMiddle, + .dataRequiredFunc = statisDataRequired, + .getEnvFunc = getAvgFuncEnv, + .initFunc = avgFunctionSetup, + .processFunc = avgFunctionMerge, + .finalizeFunc = avgPartialFinalize, + .invertFunc = avgInvertFunction, + .combineFunc = avgCombine, + }, }; // clang-format on From e5932ab1220d34bc7e4341c46ca881133bfc8667 Mon Sep 17 00:00:00 2001 From: wangmm0220 Date: Mon, 15 Jan 2024 18:16:09 +0800 Subject: [PATCH 31/51] fix:compile error --- source/libs/function/src/builtins.c | 24 +++++++++++++----------- 1 file changed, 13 insertions(+), 11 deletions(-) diff --git a/source/libs/function/src/builtins.c b/source/libs/function/src/builtins.c index 7be9f07ae4..cbeca3c9be 100644 --- a/source/libs/function/src/builtins.c +++ b/source/libs/function/src/builtins.c @@ -3775,17 +3775,19 @@ const SBuiltinFuncDefinition funcMgtBuiltins[] = { .finalizeFunc = NULL }, { - .name = "_avg_middle", - .type = FUNCTION_TYPE_AVG_PARTIAL, - .classification = FUNC_MGT_AGG_FUNC, - .translateFunc = translateAvgMiddle, - .dataRequiredFunc = statisDataRequired, - .getEnvFunc = getAvgFuncEnv, - .initFunc = avgFunctionSetup, - .processFunc = avgFunctionMerge, - .finalizeFunc = avgPartialFinalize, - .invertFunc = avgInvertFunction, - .combineFunc = avgCombine, + .name = "_avg_middle", + .type = FUNCTION_TYPE_AVG_PARTIAL, + .classification = FUNC_MGT_AGG_FUNC, + .translateFunc = translateAvgMiddle, + .dataRequiredFunc = statisDataRequired, + .getEnvFunc = getAvgFuncEnv, + .initFunc = avgFunctionSetup, + .processFunc = avgFunctionMerge, + .finalizeFunc = avgPartialFinalize, +#ifdef BUILD_NO_CALL + .invertFunc = avgInvertFunction, +#endif + .combineFunc = avgCombine, }, }; // clang-format on From 288b91a4c3a28fc7c358eb7d09039c630efa70f5 Mon Sep 17 00:00:00 2001 From: 54liuyao <54liuyao> Date: Mon, 22 Jan 2024 10:08:45 +0800 Subject: [PATCH 32/51] mid interval retrive --- source/libs/executor/inc/executorInt.h | 1 + .../executor/src/streamtimewindowoperator.c | 39 ++++++++++++++----- 2 files changed, 31 insertions(+), 9 deletions(-) diff --git a/source/libs/executor/inc/executorInt.h b/source/libs/executor/inc/executorInt.h index 72da249f50..e51869e418 100644 --- a/source/libs/executor/inc/executorInt.h +++ b/source/libs/executor/inc/executorInt.h @@ -556,6 +556,7 @@ typedef struct SStreamIntervalOperatorInfo { bool reCkBlock; SSDataBlock* pCheckpointRes; struct SUpdateInfo* pUpdateInfo; + bool recvRetrive; } SStreamIntervalOperatorInfo; typedef struct SDataGroupInfo { diff --git a/source/libs/executor/src/streamtimewindowoperator.c b/source/libs/executor/src/streamtimewindowoperator.c index 7f914755a0..93a233212b 100644 --- a/source/libs/executor/src/streamtimewindowoperator.c +++ b/source/libs/executor/src/streamtimewindowoperator.c @@ -1239,11 +1239,15 @@ static SSDataBlock* doStreamFinalIntervalAgg(SOperatorInfo* pOperator) { return NULL; } else { if (!IS_FINAL_INTERVAL_OP(pOperator)) { - doBuildDeleteResult(pInfo, pInfo->pDelWins, &pInfo->delIndex, pInfo->pDelRes); - if (pInfo->pDelRes->info.rows != 0) { - // process the rest of the data - printDataBlock(pInfo->pDelRes, getStreamOpName(pOperator->operatorType), GET_TASKID(pTaskInfo)); - return pInfo->pDelRes; + SSDataBlock* resBlock = buildIntervalResult(pOperator); + if (resBlock != NULL) { + return resBlock; + } + + if (pInfo->recvRetrive) { + pInfo->recvRetrive = false; + printDataBlock(pInfo->pPullDataRes, getStreamOpName(pOperator->operatorType), GET_TASKID(pTaskInfo)); + return pInfo->pPullDataRes; } } } @@ -1317,9 +1321,11 @@ static SSDataBlock* doStreamFinalIntervalAgg(SOperatorInfo* pOperator) { pInfo->recvGetAll = true; getAllIntervalWindow(pInfo->aggSup.pResultRowHashTable, pInfo->pUpdatedMap); continue; - } else if (pBlock->info.type == STREAM_RETRIEVE && !IS_FINAL_INTERVAL_OP(pOperator)) { - doDeleteWindows(pOperator, &pInfo->interval, pBlock, NULL, pInfo->pUpdatedMap); - if (taosArrayGetSize(pInfo->pUpdated) > 0) { + } else if (pBlock->info.type == STREAM_RETRIEVE) { + if(!IS_FINAL_INTERVAL_OP(pOperator)) { + pInfo->recvRetrive = true; + copyDataBlock(pInfo->pPullDataRes, pBlock); + doDeleteWindows(pOperator, &pInfo->interval, pBlock, NULL, pInfo->pUpdatedMap); break; } continue; @@ -1362,7 +1368,18 @@ static SSDataBlock* doStreamFinalIntervalAgg(SOperatorInfo* pOperator) { pInfo->pUpdated = NULL; blockDataEnsureCapacity(pInfo->binfo.pRes, pOperator->resultInfo.capacity); - return buildIntervalResult(pOperator); + SSDataBlock* resBlock = buildIntervalResult(pOperator); + if (resBlock != NULL) { + return resBlock; + } + + if (pInfo->recvRetrive) { + pInfo->recvRetrive = false; + printDataBlock(pInfo->pPullDataRes, getStreamOpName(pOperator->operatorType), GET_TASKID(pTaskInfo)); + return pInfo->pPullDataRes; + } + + return NULL; } int64_t getDeleteMark(SWindowPhysiNode* pWinPhyNode, int64_t interval) { @@ -1532,6 +1549,7 @@ SOperatorInfo* createStreamFinalIntervalOperatorInfo(SOperatorInfo* downstream, pInfo->stateStore = pTaskInfo->storageAPI.stateStore; pInfo->recvGetAll = false; pInfo->pCheckpointRes = createSpecialDataBlock(STREAM_CHECKPOINT); + pInfo->recvRetrive = false; pOperator->operatorType = pPhyNode->type; if (!IS_FINAL_INTERVAL_OP(pOperator) || numOfChild == 0) { @@ -4289,6 +4307,9 @@ static SSDataBlock* doStreamMidIntervalAgg(SOperatorInfo* pOperator) { doStreamIntervalSaveCheckpoint(pOperator); copyDataBlock(pInfo->pCheckpointRes, pBlock); continue; + } else if (pBlock->info.type == STREAM_RETRIEVE) { + doDeleteWindows(pOperator, &pInfo->interval, pBlock, NULL, pInfo->pUpdatedMap); + continue; } else { ASSERTS(pBlock->info.type == STREAM_INVALID, "invalid SSDataBlock type"); } From 7d27c3902afcb2bad9a3e083637c0ab7051aa133 Mon Sep 17 00:00:00 2001 From: Hongze Cheng Date: Thu, 25 Jan 2024 15:45:09 +0800 Subject: [PATCH 33/51] feat: avoid meta rwlock starvation --- include/os/osThread.h | 5 +++++ source/dnode/vnode/src/meta/metaOpen.c | 9 ++++++++- 2 files changed, 13 insertions(+), 1 deletion(-) diff --git a/include/os/osThread.h b/include/os/osThread.h index f0b79ac2c9..08575058ba 100644 --- a/include/os/osThread.h +++ b/include/os/osThread.h @@ -69,6 +69,11 @@ typedef pthread_key_t TdThreadKey; #define taosThreadCleanupPush pthread_cleanup_push #define taosThreadCleanupPop pthread_cleanup_pop +#if _XOPEN_SOURCE >= 500 || _POSIX_C_SOURCE >= 200809L +#define taosThreadRwlockAttrSetKindNP(A, B) pthread_rwlockattr_setkind_np(A, B) +#else +#define taosThreadRwlockAttrSetKindNP(A, B) ((void)0) +#endif #if defined(WINDOWS) && !defined(__USE_PTHREAD) #define TD_PTHREAD_MUTEX_INITIALIZER PTHREAD_MUTEX_INITIALIZER_FORBID diff --git a/source/dnode/vnode/src/meta/metaOpen.c b/source/dnode/vnode/src/meta/metaOpen.c index 8cab17c417..1f26c22167 100644 --- a/source/dnode/vnode/src/meta/metaOpen.c +++ b/source/dnode/vnode/src/meta/metaOpen.c @@ -27,7 +27,14 @@ static int taskIdxKeyCmpr(const void *pKey1, int kLen1, const void *pKey2, int k static int btimeIdxCmpr(const void *pKey1, int kLen1, const void *pKey2, int kLen2); static int ncolIdxCmpr(const void *pKey1, int kLen1, const void *pKey2, int kLen2); -static int32_t metaInitLock(SMeta *pMeta) { return taosThreadRwlockInit(&pMeta->lock, NULL); } +static int32_t metaInitLock(SMeta *pMeta) { + TdThreadRwlockAttr attr; + taosThreadRwlockAttrInit(&attr); + taosThreadRwlockAttrSetKindNP(&attr, PTHREAD_RWLOCK_PREFER_WRITER_NONRECURSIVE_NP); + taosThreadRwlockInit(&pMeta->lock, NULL); + taosThreadRwlockAttrDestroy(&attr); + return 0; +} static int32_t metaDestroyLock(SMeta *pMeta) { return taosThreadRwlockDestroy(&pMeta->lock); } static void metaCleanup(SMeta **ppMeta); From e8a8748dc9c11d67e2fff703d56c4a0e68ea0510 Mon Sep 17 00:00:00 2001 From: Hongze Cheng Date: Thu, 25 Jan 2024 15:48:45 +0800 Subject: [PATCH 34/51] more fix --- source/dnode/vnode/src/meta/metaOpen.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/dnode/vnode/src/meta/metaOpen.c b/source/dnode/vnode/src/meta/metaOpen.c index 1f26c22167..1e434b7cd8 100644 --- a/source/dnode/vnode/src/meta/metaOpen.c +++ b/source/dnode/vnode/src/meta/metaOpen.c @@ -31,7 +31,7 @@ static int32_t metaInitLock(SMeta *pMeta) { TdThreadRwlockAttr attr; taosThreadRwlockAttrInit(&attr); taosThreadRwlockAttrSetKindNP(&attr, PTHREAD_RWLOCK_PREFER_WRITER_NONRECURSIVE_NP); - taosThreadRwlockInit(&pMeta->lock, NULL); + taosThreadRwlockInit(&pMeta->lock, &attr); taosThreadRwlockAttrDestroy(&attr); return 0; } From b03b111c2a9a5c99fd1d84c14c54e9fefb30b5c9 Mon Sep 17 00:00:00 2001 From: Hongze Cheng Date: Fri, 26 Jan 2024 14:38:58 +0800 Subject: [PATCH 35/51] make it compilable on windows and macos --- include/os/osThread.h | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/include/os/osThread.h b/include/os/osThread.h index 08575058ba..4ef4550419 100644 --- a/include/os/osThread.h +++ b/include/os/osThread.h @@ -69,11 +69,19 @@ typedef pthread_key_t TdThreadKey; #define taosThreadCleanupPush pthread_cleanup_push #define taosThreadCleanupPop pthread_cleanup_pop +#if !defined(WINDOWS) +#if defined(_TD_DARWIN_64) // MACOS +#define taosThreadRwlockAttrSetKindNP(A, B) ((void)0) +#else // LINUX #if _XOPEN_SOURCE >= 500 || _POSIX_C_SOURCE >= 200809L #define taosThreadRwlockAttrSetKindNP(A, B) pthread_rwlockattr_setkind_np(A, B) #else #define taosThreadRwlockAttrSetKindNP(A, B) ((void)0) #endif +#endif +#else // WINDOWS +#define taosThreadRwlockAttrSetKindNP(A, B) ((void)0) +#endif #if defined(WINDOWS) && !defined(__USE_PTHREAD) #define TD_PTHREAD_MUTEX_INITIALIZER PTHREAD_MUTEX_INITIALIZER_FORBID From 0afed5263e846ec7ae8087330ae640fb4fea7d81 Mon Sep 17 00:00:00 2001 From: 54liuyao <54liuyao> Date: Wed, 31 Jan 2024 18:09:13 +0800 Subject: [PATCH 36/51] add mid operator retrive --- include/common/tcommon.h | 1 + source/dnode/mnode/impl/src/mndScheduler.c | 1 + source/libs/executor/inc/executorInt.h | 1 + source/libs/executor/src/operator.c | 2 +- .../executor/src/streamtimewindowoperator.c | 84 ++++++++++++++----- 5 files changed, 66 insertions(+), 23 deletions(-) diff --git a/include/common/tcommon.h b/include/common/tcommon.h index 24e5d186b9..addb02cd7d 100644 --- a/include/common/tcommon.h +++ b/include/common/tcommon.h @@ -171,6 +171,7 @@ typedef enum EStreamType { STREAM_CHECKPOINT, STREAM_CREATE_CHILD_TABLE, STREAM_TRANS_STATE, + STREAM_MID_RETRIEVE, } EStreamType; #pragma pack(push, 1) diff --git a/source/dnode/mnode/impl/src/mndScheduler.c b/source/dnode/mnode/impl/src/mndScheduler.c index a4721b8a11..9ec879ec59 100644 --- a/source/dnode/mnode/impl/src/mndScheduler.c +++ b/source/dnode/mnode/impl/src/mndScheduler.c @@ -559,6 +559,7 @@ static void bindTwoLevel(SArray* tasks, int32_t begin, int32_t end) { end = end > taosArrayGetSize(pUpTaskList) ? taosArrayGetSize(pUpTaskList): end; for(int i = begin; i < end; i++){ SStreamTask* pUpTask = taosArrayGetP(pUpTaskList, i); + pUpTask->info.selfChildId = i - begin; streamTaskSetFixedDownstreamInfo(pUpTask, *pDownTask); streamTaskSetUpstreamInfo(*pDownTask, pUpTask); } diff --git a/source/libs/executor/inc/executorInt.h b/source/libs/executor/inc/executorInt.h index e51869e418..097b8d0136 100644 --- a/source/libs/executor/inc/executorInt.h +++ b/source/libs/executor/inc/executorInt.h @@ -557,6 +557,7 @@ typedef struct SStreamIntervalOperatorInfo { SSDataBlock* pCheckpointRes; struct SUpdateInfo* pUpdateInfo; bool recvRetrive; + SSDataBlock* pMidRetriveRes; } SStreamIntervalOperatorInfo; typedef struct SDataGroupInfo { diff --git a/source/libs/executor/src/operator.c b/source/libs/executor/src/operator.c index 0c6671c742..fd4b3cd7db 100644 --- a/source/libs/executor/src/operator.c +++ b/source/libs/executor/src/operator.c @@ -490,7 +490,7 @@ SOperatorInfo* createOperator(SPhysiNode* pPhyNode, SExecTaskInfo* pTaskInfo, SR int32_t children = 0; pOptr = createStreamFinalIntervalOperatorInfo(ops[0], pPhyNode, pTaskInfo, children, pHandle); } else if (QUERY_NODE_PHYSICAL_PLAN_STREAM_MID_INTERVAL == type) { - int32_t children = 0; + int32_t children = pHandle->numOfVgroups; pOptr = createStreamFinalIntervalOperatorInfo(ops[0], pPhyNode, pTaskInfo, children, pHandle); } else if (QUERY_NODE_PHYSICAL_PLAN_STREAM_FINAL_INTERVAL == type) { int32_t children = pHandle->numOfVgroups; diff --git a/source/libs/executor/src/streamtimewindowoperator.c b/source/libs/executor/src/streamtimewindowoperator.c index 93a233212b..99b1f5bd6f 100644 --- a/source/libs/executor/src/streamtimewindowoperator.c +++ b/source/libs/executor/src/streamtimewindowoperator.c @@ -410,6 +410,7 @@ void destroyStreamFinalIntervalOperatorInfo(void* param) { blockDataDestroy(pInfo->pPullDataRes); taosArrayDestroy(pInfo->pDelWins); blockDataDestroy(pInfo->pDelRes); + blockDataDestroy(pInfo->pMidRetriveRes); pInfo->stateStore.streamFileStateDestroy(pInfo->pState->pFileState); if (pInfo->pState->dump == 1) { @@ -1246,8 +1247,8 @@ static SSDataBlock* doStreamFinalIntervalAgg(SOperatorInfo* pOperator) { if (pInfo->recvRetrive) { pInfo->recvRetrive = false; - printDataBlock(pInfo->pPullDataRes, getStreamOpName(pOperator->operatorType), GET_TASKID(pTaskInfo)); - return pInfo->pPullDataRes; + printDataBlock(pInfo->pMidRetriveRes, getStreamOpName(pOperator->operatorType), GET_TASKID(pTaskInfo)); + return pInfo->pMidRetriveRes; } } } @@ -1324,7 +1325,8 @@ static SSDataBlock* doStreamFinalIntervalAgg(SOperatorInfo* pOperator) { } else if (pBlock->info.type == STREAM_RETRIEVE) { if(!IS_FINAL_INTERVAL_OP(pOperator)) { pInfo->recvRetrive = true; - copyDataBlock(pInfo->pPullDataRes, pBlock); + copyDataBlock(pInfo->pMidRetriveRes, pBlock); + pInfo->pMidRetriveRes->info.type = STREAM_MID_RETRIEVE; doDeleteWindows(pOperator, &pInfo->interval, pBlock, NULL, pInfo->pUpdatedMap); break; } @@ -1340,6 +1342,8 @@ static SSDataBlock* doStreamFinalIntervalAgg(SOperatorInfo* pOperator) { doStreamIntervalSaveCheckpoint(pOperator); copyDataBlock(pInfo->pCheckpointRes, pBlock); continue; + } else if (IS_FINAL_INTERVAL_OP(pOperator) && pBlock->info.type == STREAM_MID_RETRIEVE) { + continue; } else { ASSERTS(pBlock->info.type == STREAM_INVALID, "invalid SSDataBlock type"); } @@ -1375,8 +1379,8 @@ static SSDataBlock* doStreamFinalIntervalAgg(SOperatorInfo* pOperator) { if (pInfo->recvRetrive) { pInfo->recvRetrive = false; - printDataBlock(pInfo->pPullDataRes, getStreamOpName(pOperator->operatorType), GET_TASKID(pTaskInfo)); - return pInfo->pPullDataRes; + printDataBlock(pInfo->pMidRetriveRes, getStreamOpName(pOperator->operatorType), GET_TASKID(pTaskInfo)); + return pInfo->pMidRetriveRes; } return NULL; @@ -1550,6 +1554,7 @@ SOperatorInfo* createStreamFinalIntervalOperatorInfo(SOperatorInfo* downstream, pInfo->recvGetAll = false; pInfo->pCheckpointRes = createSpecialDataBlock(STREAM_CHECKPOINT); pInfo->recvRetrive = false; + pInfo->pMidRetriveRes = createSpecialDataBlock(STREAM_MID_RETRIEVE); pOperator->operatorType = pPhyNode->type; if (!IS_FINAL_INTERVAL_OP(pOperator) || numOfChild == 0) { @@ -4142,16 +4147,15 @@ static void doStreamMidIntervalAggImpl(SOperatorInfo* pOperator, SSDataBlock* pS SStreamIntervalOperatorInfo* pInfo = (SStreamIntervalOperatorInfo*)pOperator->info; pInfo->dataVersion = TMAX(pInfo->dataVersion, pSDataBlock->info.version); - SResultRowInfo* pResultRowInfo = &(pInfo->binfo.resultRowInfo); - SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; - SExprSupp* pSup = &pOperator->exprSupp; - int32_t numOfOutput = pSup->numOfExprs; - int32_t step = 1; - SRowBuffPos* pResPos = NULL; - SResultRow* pResult = NULL; - int32_t forwardRows = 1; - uint64_t groupId = pSDataBlock->info.id.groupId; - + SResultRowInfo* pResultRowInfo = &(pInfo->binfo.resultRowInfo); + SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; + SExprSupp* pSup = &pOperator->exprSupp; + int32_t numOfOutput = pSup->numOfExprs; + int32_t step = 1; + SRowBuffPos* pResPos = NULL; + SResultRow* pResult = NULL; + int32_t forwardRows = 1; + uint64_t groupId = pSDataBlock->info.id.groupId; SColumnInfoData* pColDataInfo = taosArrayGet(pSDataBlock->pDataBlock, pInfo->primaryTsIndex); TSKEY* tsCol = (int64_t*)pColDataInfo->pData; @@ -4160,6 +4164,27 @@ static void doStreamMidIntervalAggImpl(SOperatorInfo* pOperator, SSDataBlock* pS STimeWindow nextWin = getFinalTimeWindow(ts, &pInfo->interval); while (1) { + SWinKey key = { + .ts = nextWin.skey, + .groupId = groupId, + }; + void* chIds = taosHashGet(pInfo->pPullDataMap, &key, sizeof(SWinKey)); + int32_t index = -1; + SArray* chArray = NULL; + int32_t chId = 0; + if (chIds) { + chArray = *(void**)chIds; + chId = getChildIndex(pSDataBlock); + index = taosArraySearchIdx(chArray, &chId, compareInt32Val, TD_EQ); + } + if (!(index == -1 || pSDataBlock->info.type == STREAM_PULL_DATA)) { + startPos = getNextQualifiedFinalWindow(&pInfo->interval, &nextWin, &pSDataBlock->info, tsCol, startPos); + if (startPos < 0) { + break; + } + continue; + } + if (!inSlidingWindow(&pInfo->interval, &nextWin, &pSDataBlock->info)) { startPos = getNexWindowPos(&pInfo->interval, &pSDataBlock->info, tsCol, startPos, nextWin.ekey, &nextWin); if (startPos < 0) { @@ -4175,10 +4200,6 @@ static void doStreamMidIntervalAggImpl(SOperatorInfo* pOperator, SSDataBlock* pS T_LONG_JMP(pTaskInfo->env, TSDB_CODE_OUT_OF_MEMORY); } - SWinKey key = { - .ts = pResult->win.skey, - .groupId = groupId, - }; if (pInfo->twAggSup.calTrigger == STREAM_TRIGGER_AT_ONCE) { saveWinResult(&key, pResPos, pUpdatedMap); } @@ -4215,6 +4236,18 @@ static void doStreamMidIntervalAggImpl(SOperatorInfo* pOperator, SSDataBlock* pS } } +static void addMidRetriveWindow(SArray* wins, SHashObj* pMidPullMap, int32_t numOfChild) { + int32_t size = taosArrayGetSize(wins); + for (int32_t i = 0; i < size; i++) { + SWinKey* winKey = taosArrayGet(wins, i); + void* chIds = taosHashGet(pMidPullMap, winKey, sizeof(SWinKey)); + if (!chIds) { + addPullWindow(pMidPullMap, winKey, numOfChild); + qDebug("===stream===prepare mid operator retrive for delete %" PRId64 ", size:%d", winKey->ts, numOfChild); + } + } +} + static SSDataBlock* doStreamMidIntervalAgg(SOperatorInfo* pOperator) { SStreamIntervalOperatorInfo* pInfo = pOperator->info; SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; @@ -4300,15 +4333,22 @@ static SSDataBlock* doStreamMidIntervalAgg(SOperatorInfo* pOperator) { return pInfo->pDelRes; } break; - } else if (pBlock->info.type == STREAM_CREATE_CHILD_TABLE || pBlock->info.type == STREAM_PULL_OVER) { + } else if (pBlock->info.type == STREAM_CREATE_CHILD_TABLE) { + return pBlock; + } else if (pBlock->info.type == STREAM_PULL_OVER) { + processPullOver(pBlock, pInfo->pPullDataMap, pInfo->pFinalPullDataMap, &pInfo->interval, pInfo->pPullWins, + pInfo->numOfChild, pOperator); return pBlock; } else if (pBlock->info.type == STREAM_CHECKPOINT) { pAPI->stateStore.streamStateCommit(pInfo->pState); doStreamIntervalSaveCheckpoint(pOperator); copyDataBlock(pInfo->pCheckpointRes, pBlock); continue; - } else if (pBlock->info.type == STREAM_RETRIEVE) { - doDeleteWindows(pOperator, &pInfo->interval, pBlock, NULL, pInfo->pUpdatedMap); + } else if (pBlock->info.type == STREAM_MID_RETRIEVE) { + SArray* delWins = taosArrayInit(8, sizeof(SWinKey)); + doDeleteWindows(pOperator, &pInfo->interval, pBlock, delWins, pInfo->pUpdatedMap); + addMidRetriveWindow(delWins, pInfo->pPullDataMap, pInfo->numOfChild); + taosArrayDestroy(delWins); continue; } else { ASSERTS(pBlock->info.type == STREAM_INVALID, "invalid SSDataBlock type"); From a6a312a1a66b35a080c5cf08aba25101fedf3096 Mon Sep 17 00:00:00 2001 From: 54liuyao <54liuyao> Date: Thu, 1 Feb 2024 16:44:00 +0800 Subject: [PATCH 37/51] fix issue --- source/libs/executor/src/streamtimewindowoperator.c | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/source/libs/executor/src/streamtimewindowoperator.c b/source/libs/executor/src/streamtimewindowoperator.c index 99b1f5bd6f..dd3ca691b6 100644 --- a/source/libs/executor/src/streamtimewindowoperator.c +++ b/source/libs/executor/src/streamtimewindowoperator.c @@ -4330,15 +4330,16 @@ static SSDataBlock* doStreamMidIntervalAgg(SOperatorInfo* pOperator) { } else { pInfo->pDelRes->info.type = STREAM_DELETE_RESULT; } + printDataBlock(pInfo->pDelRes, getStreamOpName(pOperator->operatorType), GET_TASKID(pTaskInfo)); return pInfo->pDelRes; } - break; + continue; } else if (pBlock->info.type == STREAM_CREATE_CHILD_TABLE) { return pBlock; } else if (pBlock->info.type == STREAM_PULL_OVER) { processPullOver(pBlock, pInfo->pPullDataMap, pInfo->pFinalPullDataMap, &pInfo->interval, pInfo->pPullWins, pInfo->numOfChild, pOperator); - return pBlock; + continue; } else if (pBlock->info.type == STREAM_CHECKPOINT) { pAPI->stateStore.streamStateCommit(pInfo->pState); doStreamIntervalSaveCheckpoint(pOperator); From 4f2fb15a5622107e5e35c1751fccbed0427b55ac Mon Sep 17 00:00:00 2001 From: 54liuyao <54liuyao> Date: Thu, 1 Feb 2024 18:07:19 +0800 Subject: [PATCH 38/51] delete log --- source/libs/executor/src/streamtimewindowoperator.c | 1 - 1 file changed, 1 deletion(-) diff --git a/source/libs/executor/src/streamtimewindowoperator.c b/source/libs/executor/src/streamtimewindowoperator.c index dd3ca691b6..7c9b998c7d 100644 --- a/source/libs/executor/src/streamtimewindowoperator.c +++ b/source/libs/executor/src/streamtimewindowoperator.c @@ -4330,7 +4330,6 @@ static SSDataBlock* doStreamMidIntervalAgg(SOperatorInfo* pOperator) { } else { pInfo->pDelRes->info.type = STREAM_DELETE_RESULT; } - printDataBlock(pInfo->pDelRes, getStreamOpName(pOperator->operatorType), GET_TASKID(pTaskInfo)); return pInfo->pDelRes; } continue; From a656d75ca79186cfb8415e681ca6177da4dd7756 Mon Sep 17 00:00:00 2001 From: wangmm0220 Date: Mon, 5 Feb 2024 17:07:50 +0800 Subject: [PATCH 39/51] opti:consume data excluded from some source --- include/common/tmsg.h | 7 ++++++- include/libs/executor/executor.h | 2 ++ source/client/src/clientTmq.c | 8 ++++++++ source/common/src/tmsg.c | 12 ++++++++++++ source/dnode/vnode/inc/vnode.h | 2 +- source/dnode/vnode/src/inc/tq.h | 2 +- source/dnode/vnode/src/sma/smaTimeRange.c | 2 +- source/dnode/vnode/src/tq/tqRead.c | 7 +++++-- source/dnode/vnode/src/tq/tqScan.c | 11 ++++++++++- source/dnode/vnode/src/tq/tqSink.c | 4 ++-- source/dnode/vnode/src/tq/tqUtil.c | 2 +- source/libs/executor/inc/querytask.h | 1 + source/libs/executor/src/executor.c | 5 +++++ source/libs/executor/src/scanoperator.c | 2 +- source/libs/parser/src/parInsertUtil.c | 15 +++++++++------ 15 files changed, 65 insertions(+), 17 deletions(-) diff --git a/include/common/tmsg.h b/include/common/tmsg.h index c314d82036..9af68543da 100644 --- a/include/common/tmsg.h +++ b/include/common/tmsg.h @@ -3603,6 +3603,7 @@ typedef struct { int64_t timeout; STqOffsetVal reqOffset; int8_t enableReplay; + int8_t sourceExcluded; } SMqPollReq; int32_t tSerializeSMqPollReq(void* buf, int32_t bufLen, SMqPollReq* pReq); @@ -3891,6 +3892,9 @@ int32_t tDeserializeSMqSeekReq(void* buf, int32_t bufLen, SMqSeekReq* pReq); #define SUBMIT_REQ_AUTO_CREATE_TABLE 0x1 #define SUBMIT_REQ_COLUMN_DATA_FORMAT 0x2 +#define SOURCE_NULL 0 +#define SOURCE_TAOSX 1 + typedef struct { int32_t flags; SVCreateTbReq* pCreateTbReq; @@ -3901,7 +3905,8 @@ typedef struct { SArray* aRowP; SArray* aCol; }; - int64_t ctimeMs; + int64_t ctimeMs; + int8_t source; } SSubmitTbData; typedef struct { diff --git a/include/libs/executor/executor.h b/include/libs/executor/executor.h index f78b7a3126..e06a08acba 100644 --- a/include/libs/executor/executor.h +++ b/include/libs/executor/executor.h @@ -197,6 +197,8 @@ int32_t qStreamPrepareScan(qTaskInfo_t tinfo, STqOffsetVal* pOffset, int8_t subT void qStreamSetOpen(qTaskInfo_t tinfo); +void qStreamSetSourceExcluded(qTaskInfo_t tinfo, int8_t sourceExcluded); + void qStreamExtractOffset(qTaskInfo_t tinfo, STqOffsetVal* pOffset); SMqMetaRsp* qStreamExtractMetaMsg(qTaskInfo_t tinfo); diff --git a/source/client/src/clientTmq.c b/source/client/src/clientTmq.c index 15c8903978..96d527d299 100644 --- a/source/client/src/clientTmq.c +++ b/source/client/src/clientTmq.c @@ -63,6 +63,7 @@ struct tmq_conf_t { int8_t withTbName; int8_t snapEnable; int8_t replayEnable; + int8_t sourceExcluded; // do not consume, bit uint16_t port; int32_t autoCommitInterval; char* ip; @@ -82,6 +83,7 @@ struct tmq_t { int32_t autoCommitInterval; int8_t resetOffsetCfg; int8_t replayEnable; + int8_t sourceExcluded; // do not consume, bit uint64_t consumerId; tmq_commit_cb* commitCb; void* commitCbUserParam; @@ -384,6 +386,10 @@ tmq_conf_res_t tmq_conf_set(tmq_conf_t* conf, const char* key, const char* value return TMQ_CONF_INVALID; } } + if (strcasecmp(key, "msg.consume.excluded") == 0) { + conf->sourceExcluded = taosStr2int64(value); + return TMQ_CONF_OK; + } if (strcasecmp(key, "td.connect.db") == 0) { return TMQ_CONF_OK; @@ -1081,6 +1087,7 @@ tmq_t* tmq_consumer_new(tmq_conf_t* conf, char* errstr, int32_t errstrLen) { pTmq->commitCbUserParam = conf->commitCbUserParam; pTmq->resetOffsetCfg = conf->resetOffset; pTmq->replayEnable = conf->replayEnable; + pTmq->sourceExcluded = conf->sourceExcluded; if(conf->replayEnable){ pTmq->autoCommit = false; } @@ -1549,6 +1556,7 @@ void tmqBuildConsumeReqImpl(SMqPollReq* pReq, tmq_t* tmq, int64_t timeout, SMqCl pReq->useSnapshot = tmq->useSnapshot; pReq->reqId = generateRequestId(); pReq->enableReplay = tmq->replayEnable; + pReq->sourceExcluded = tmq->sourceExcluded; } SMqMetaRspObj* tmqBuildMetaRspFromWrapper(SMqPollRspWrapper* pWrapper) { diff --git a/source/common/src/tmsg.c b/source/common/src/tmsg.c index c9e2908e8a..fa696713d6 100644 --- a/source/common/src/tmsg.c +++ b/source/common/src/tmsg.c @@ -6516,6 +6516,7 @@ int32_t tSerializeSMqPollReq(void *buf, int32_t bufLen, SMqPollReq *pReq) { if (tEncodeI64(&encoder, pReq->timeout) < 0) return -1; if (tSerializeSTqOffsetVal(&encoder, &pReq->reqOffset) < 0) return -1; if (tEncodeI8(&encoder, pReq->enableReplay) < 0) return -1; + if (tEncodeI8(&encoder, pReq->sourceExcluded) < 0) return -1; tEndEncode(&encoder); @@ -6556,6 +6557,10 @@ int32_t tDeserializeSMqPollReq(void *buf, int32_t bufLen, SMqPollReq *pReq) { if (tDecodeI8(&decoder, &pReq->enableReplay) < 0) return -1; } + if (!tDecodeIsEnd(&decoder)) { + if (tDecodeI8(&decoder, &pReq->sourceExcluded) < 0) return -1; + } + tEndDecode(&decoder); tDecoderClear(&decoder); @@ -8578,6 +8583,7 @@ static int32_t tEncodeSSubmitTbData(SEncoder *pCoder, const SSubmitTbData *pSubm } } if (tEncodeI64(pCoder, pSubmitTbData->ctimeMs) < 0) return -1; + if (tEncodeI8(pCoder, pSubmitTbData->source) < 0) return -1; tEndEncode(pCoder); return 0; @@ -8665,6 +8671,12 @@ static int32_t tDecodeSSubmitTbData(SDecoder *pCoder, SSubmitTbData *pSubmitTbDa goto _exit; } } + if (!tDecodeIsEnd(pCoder)) { + if (tDecodeI8(pCoder, &pSubmitTbData->source) < 0) { + code = TSDB_CODE_INVALID_MSG; + goto _exit; + } + } tEndDecode(pCoder); diff --git a/source/dnode/vnode/inc/vnode.h b/source/dnode/vnode/inc/vnode.h index 9a4e2edf8d..6bea092b5f 100644 --- a/source/dnode/vnode/inc/vnode.h +++ b/source/dnode/vnode/inc/vnode.h @@ -221,7 +221,7 @@ bool tqReaderIsQueriedTable(STqReader *pReader, uint64_t uid); bool tqCurrentBlockConsumed(const STqReader *pReader); int32_t tqReaderSeek(STqReader *pReader, int64_t ver, const char *id); -bool tqNextBlockInWal(STqReader *pReader, const char *idstr); +bool tqNextBlockInWal(STqReader *pReader, const char *idstr, int sourceExcluded); bool tqNextBlockImpl(STqReader *pReader, const char *idstr); SWalReader *tqGetWalReader(STqReader *pReader); SSDataBlock *tqGetResultBlock(STqReader *pReader); diff --git a/source/dnode/vnode/src/inc/tq.h b/source/dnode/vnode/src/inc/tq.h index 0ef29fcb3a..ee527a8a6e 100644 --- a/source/dnode/vnode/src/inc/tq.h +++ b/source/dnode/vnode/src/inc/tq.h @@ -119,7 +119,7 @@ int32_t tqScanData(STQ* pTq, STqHandle* pHandle, SMqDataRsp* pRsp, STqOffsetVal* int32_t tqFetchLog(STQ* pTq, STqHandle* pHandle, int64_t* fetchOffset, uint64_t reqId); // tqExec -int32_t tqTaosxScanLog(STQ* pTq, STqHandle* pHandle, SPackedData submit, STaosxRsp* pRsp, int32_t* totalRows); +int32_t tqTaosxScanLog(STQ* pTq, STqHandle* pHandle, SPackedData submit, STaosxRsp* pRsp, int32_t* totalRows, int8_t sourceExcluded); int32_t tqAddBlockDataToRsp(const SSDataBlock* pBlock, SMqDataRsp* pRsp, int32_t numOfCols, int8_t precision); int32_t tqSendDataRsp(STqHandle* pHandle, const SRpcMsg* pMsg, const SMqPollReq* pReq, const SMqDataRsp* pRsp, int32_t type, int32_t vgId); diff --git a/source/dnode/vnode/src/sma/smaTimeRange.c b/source/dnode/vnode/src/sma/smaTimeRange.c index 289986e01f..b2322260f0 100644 --- a/source/dnode/vnode/src/sma/smaTimeRange.c +++ b/source/dnode/vnode/src/sma/smaTimeRange.c @@ -193,7 +193,7 @@ int32_t smaBlockToSubmit(SVnode *pVnode, const SArray *pBlocks, const STSchema * continue; } - SSubmitTbData tbData = {.suid = suid, .uid = 0, .sver = pTSchema->version, .flags = SUBMIT_REQ_AUTO_CREATE_TABLE,}; + SSubmitTbData tbData = {.suid = suid, .uid = 0, .sver = pTSchema->version, .flags = SUBMIT_REQ_AUTO_CREATE_TABLE, .source = SOURCE_NULL}; int32_t cid = taosArrayGetSize(pDataBlock->pDataBlock) + 1; tbData.pCreateTbReq = buildAutoCreateTableReq(stbFullName, suid, cid, pDataBlock, tagArray); diff --git a/source/dnode/vnode/src/tq/tqRead.c b/source/dnode/vnode/src/tq/tqRead.c index bfa8cfdb53..b0295ba552 100644 --- a/source/dnode/vnode/src/tq/tqRead.c +++ b/source/dnode/vnode/src/tq/tqRead.c @@ -368,7 +368,7 @@ int32_t extractMsgFromWal(SWalReader* pReader, void** pItem, int64_t maxVer, con } // todo ignore the error in wal? -bool tqNextBlockInWal(STqReader* pReader, const char* id) { +bool tqNextBlockInWal(STqReader* pReader, const char* id, int sourceExcluded) { SWalReader* pWalReader = pReader->pWalReader; SSDataBlock* pDataBlock = NULL; @@ -391,7 +391,10 @@ bool tqNextBlockInWal(STqReader* pReader, const char* id) { numOfBlocks, pReader->msg.msgLen, pReader->msg.ver); SSubmitTbData* pSubmitTbData = taosArrayGet(pReader->submit.aSubmitTbData, pReader->nextBlk); - + if ((pSubmitTbData->source & sourceExcluded) != 0){ + pReader->nextBlk += 1; + continue; + } if (pReader->tbIdHash == NULL || taosHashGet(pReader->tbIdHash, &pSubmitTbData->uid, sizeof(int64_t)) != NULL) { tqTrace("tq reader return submit block, uid:%" PRId64, pSubmitTbData->uid); SSDataBlock* pRes = NULL; diff --git a/source/dnode/vnode/src/tq/tqScan.c b/source/dnode/vnode/src/tq/tqScan.c index 01866ef893..738bd0a9dd 100644 --- a/source/dnode/vnode/src/tq/tqScan.c +++ b/source/dnode/vnode/src/tq/tqScan.c @@ -94,6 +94,7 @@ int32_t tqScanData(STQ* pTq, STqHandle* pHandle, SMqDataRsp* pRsp, STqOffsetVal* return -1; } + qStreamSetSourceExcluded(task, pRequest->sourceExcluded); while (1) { SSDataBlock* pDataBlock = NULL; code = getDataBlock(task, pHandle, vgId, &pDataBlock); @@ -250,7 +251,7 @@ int32_t tqScanTaosx(STQ* pTq, const STqHandle* pHandle, STaosxRsp* pRsp, SMqMeta return 0; } -int32_t tqTaosxScanLog(STQ* pTq, STqHandle* pHandle, SPackedData submit, STaosxRsp* pRsp, int32_t* totalRows) { +int32_t tqTaosxScanLog(STQ* pTq, STqHandle* pHandle, SPackedData submit, STaosxRsp* pRsp, int32_t* totalRows, int8_t sourceExcluded) { STqExecHandle* pExec = &pHandle->execHandle; SArray* pBlocks = taosArrayInit(0, sizeof(SSDataBlock)); SArray* pSchemas = taosArrayInit(0, sizeof(void*)); @@ -265,6 +266,10 @@ int32_t tqTaosxScanLog(STQ* pTq, STqHandle* pHandle, SPackedData submit, STaosxR if (tqRetrieveTaosxBlock(pReader, pBlocks, pSchemas, &pSubmitTbDataRet) < 0) { if (terrno == TSDB_CODE_TQ_TABLE_SCHEMA_NOT_FOUND) goto loop_table; } + + if ((pSubmitTbDataRet->source & sourceExcluded) != 0){ + goto loop_table; + } if (pRsp->withTbName) { int64_t uid = pExec->pTqReader->lastBlkUid; if (tqAddTbNameToRsp(pTq, uid, pRsp, taosArrayGetSize(pBlocks)) < 0) { @@ -329,6 +334,10 @@ int32_t tqTaosxScanLog(STQ* pTq, STqHandle* pHandle, SPackedData submit, STaosxR if (tqRetrieveTaosxBlock(pReader, pBlocks, pSchemas, &pSubmitTbDataRet) < 0) { if (terrno == TSDB_CODE_TQ_TABLE_SCHEMA_NOT_FOUND) goto loop_db; } + + if ((pSubmitTbDataRet->source & sourceExcluded) != 0){ + goto loop_db; + } if (pRsp->withTbName) { int64_t uid = pExec->pTqReader->lastBlkUid; if (tqAddTbNameToRsp(pTq, uid, pRsp, taosArrayGetSize(pBlocks)) < 0) { diff --git a/source/dnode/vnode/src/tq/tqSink.c b/source/dnode/vnode/src/tq/tqSink.c index c2e48d5d92..9645ab5c70 100644 --- a/source/dnode/vnode/src/tq/tqSink.c +++ b/source/dnode/vnode/src/tq/tqSink.c @@ -791,7 +791,7 @@ void tqSinkDataIntoDstTable(SStreamTask* pTask, void* vnode, void* data) { return; } - SSubmitTbData tbData = {.suid = suid, .uid = 0, .sver = pTSchema->version}; + SSubmitTbData tbData = {.suid = suid, .uid = 0, .sver = pTSchema->version, .source = SOURCE_NULL}; code = setDstTableDataUid(pVnode, pTask, pDataBlock, stbFullName, &tbData); if (code != TSDB_CODE_SUCCESS) { continue; @@ -835,7 +835,7 @@ void tqSinkDataIntoDstTable(SStreamTask* pTask, void* vnode, void* data) { pTask->execInfo.sink.numOfBlocks += 1; uint64_t groupId = pDataBlock->info.id.groupId; - SSubmitTbData tbData = {.suid = suid, .uid = 0, .sver = pTSchema->version}; + SSubmitTbData tbData = {.suid = suid, .uid = 0, .sver = pTSchema->version, .source = SOURCE_NULL}; int32_t* index = taosHashGet(pTableIndexMap, &groupId, sizeof(groupId)); if (index == NULL) { // no data yet, append it diff --git a/source/dnode/vnode/src/tq/tqUtil.c b/source/dnode/vnode/src/tq/tqUtil.c index d18455d221..a71b45e5d7 100644 --- a/source/dnode/vnode/src/tq/tqUtil.c +++ b/source/dnode/vnode/src/tq/tqUtil.c @@ -250,7 +250,7 @@ static int32_t extractDataAndRspForDbStbSubscribe(STQ* pTq, STqHandle* pHandle, .ver = pHead->version, }; - code = tqTaosxScanLog(pTq, pHandle, submit, &taosxRsp, &totalRows); + code = tqTaosxScanLog(pTq, pHandle, submit, &taosxRsp, &totalRows, pRequest->sourceExcluded); if (code < 0) { tqError("tmq poll: tqTaosxScanLog error %" PRId64 ", in vgId:%d, subkey %s", pRequest->consumerId, vgId, pRequest->subKey); diff --git a/source/libs/executor/inc/querytask.h b/source/libs/executor/inc/querytask.h index fcafd5a4e3..0c9a5e3197 100644 --- a/source/libs/executor/inc/querytask.h +++ b/source/libs/executor/inc/querytask.h @@ -59,6 +59,7 @@ typedef struct STaskStopInfo { typedef struct { STqOffsetVal currentOffset; // for tmq SMqMetaRsp metaRsp; // for tmq fetching meta + int8_t sourceExcluded; int64_t snapshotVer; SSchemaWrapper* schema; char tbName[TSDB_TABLE_NAME_LEN]; // this is the current scan table: todo refactor diff --git a/source/libs/executor/src/executor.c b/source/libs/executor/src/executor.c index fb39de484f..87c65b94a4 100644 --- a/source/libs/executor/src/executor.c +++ b/source/libs/executor/src/executor.c @@ -1148,6 +1148,11 @@ void qStreamSetOpen(qTaskInfo_t tinfo) { pOperator->status = OP_NOT_OPENED; } +void qStreamSetSourceExcluded(qTaskInfo_t tinfo, int8_t sourceExcluded) { + SExecTaskInfo* pTaskInfo = (SExecTaskInfo*)tinfo; + pTaskInfo->streamInfo.sourceExcluded = sourceExcluded; +} + int32_t qStreamPrepareScan(qTaskInfo_t tinfo, STqOffsetVal* pOffset, int8_t subType) { SExecTaskInfo* pTaskInfo = (SExecTaskInfo*)tinfo; SStorageAPI* pAPI = &pTaskInfo->storageAPI; diff --git a/source/libs/executor/src/scanoperator.c b/source/libs/executor/src/scanoperator.c index 3ed5128858..c4c1b2c299 100644 --- a/source/libs/executor/src/scanoperator.c +++ b/source/libs/executor/src/scanoperator.c @@ -1978,7 +1978,7 @@ static SSDataBlock* doQueueScan(SOperatorInfo* pOperator) { if (pTaskInfo->streamInfo.currentOffset.type == TMQ_OFFSET__LOG) { while (1) { - bool hasResult = pAPI->tqReaderFn.tqReaderNextBlockInWal(pInfo->tqReader, id); + bool hasResult = pAPI->tqReaderFn.tqReaderNextBlockInWal(pInfo->tqReader, id, pTaskInfo->streamInfo.sourceExcluded); SSDataBlock* pRes = pAPI->tqReaderFn.tqGetResultBlock(pInfo->tqReader); struct SWalReader* pWalReader = pAPI->tqReaderFn.tqReaderGetWalReader(pInfo->tqReader); diff --git a/source/libs/parser/src/parInsertUtil.c b/source/libs/parser/src/parInsertUtil.c index a924ed68b0..ff9ece1659 100644 --- a/source/libs/parser/src/parInsertUtil.c +++ b/source/libs/parser/src/parInsertUtil.c @@ -211,6 +211,7 @@ static int32_t createTableDataCxt(STableMeta* pTableMeta, SVCreateTbReq** pCreat bool colMode, bool ignoreColVals) { STableDataCxt* pTableCxt = taosMemoryCalloc(1, sizeof(STableDataCxt)); if (NULL == pTableCxt) { + *pOutput = NULL; return TSDB_CODE_OUT_OF_MEMORY; } @@ -268,12 +269,8 @@ static int32_t createTableDataCxt(STableMeta* pTableMeta, SVCreateTbReq** pCreat } } - if (TSDB_CODE_SUCCESS == code) { - *pOutput = pTableCxt; - qDebug("tableDataCxt created, uid:%" PRId64 ", vgId:%d", pTableMeta->uid, pTableMeta->vgId); - } else { - taosMemoryFree(pTableCxt); - } + *pOutput = pTableCxt; + qDebug("tableDataCxt created, code:%d, uid:%" PRId64 ", vgId:%d", code, pTableMeta->uid, pTableMeta->vgId); return code; } @@ -288,6 +285,7 @@ static int32_t rebuildTableData(SSubmitTbData* pSrc, SSubmitTbData** pDst) { pTmp->suid = pSrc->suid; pTmp->uid = pSrc->uid; pTmp->sver = pSrc->sver; + pTmp->source = pSrc->source; pTmp->pCreateTbReq = NULL; if (pTmp->flags & SUBMIT_REQ_AUTO_CREATE_TABLE) { if (pSrc->pCreateTbReq) { @@ -344,6 +342,10 @@ int32_t insGetTableDataCxt(SHashObj* pHash, void* id, int32_t idLen, STableMeta* void* pData = *pTableCxt; // deal scan coverity code = taosHashPut(pHash, id, idLen, &pData, POINTER_BYTES); } + + if (TSDB_CODE_SUCCESS != code) { + insDestroyTableDataCxt(*pTableCxt); + } return code; } @@ -651,6 +653,7 @@ int rawBlockBindData(SQuery* query, STableMeta* pTableMeta, void* data, SVCreate goto end; } + pTableCxt->pData->source = SOURCE_TAOSX; if(tmp == NULL){ ret = initTableColSubmitData(pTableCxt); if (ret != TSDB_CODE_SUCCESS) { From 8baa27e6a365d43d5d3f2fc5f123755a1a406e52 Mon Sep 17 00:00:00 2001 From: chenhaoran Date: Mon, 5 Feb 2024 17:11:39 +0800 Subject: [PATCH 40/51] test: open annotated unit tests in ci --- tests/parallel_test/cases.task | 6 +----- tests/system-test/test.py | 2 +- tests/unit-test/test.sh | 2 +- 3 files changed, 3 insertions(+), 7 deletions(-) diff --git a/tests/parallel_test/cases.task b/tests/parallel_test/cases.task index 103e67be46..c1eb3c27a5 100644 --- a/tests/parallel_test/cases.task +++ b/tests/parallel_test/cases.task @@ -4,12 +4,8 @@ #unit-test -archOs=$(arch) -if [[ $archOs =~ "aarch64" ]]; then ,,n,unit-test,bash test.sh -else -,,y,unit-test,bash test.sh -fi + # # army-test diff --git a/tests/system-test/test.py b/tests/system-test/test.py index 795132b14e..301622cd28 100644 --- a/tests/system-test/test.py +++ b/tests/system-test/test.py @@ -683,6 +683,6 @@ if __name__ == "__main__": if conn is not None: conn.close() if asan: - tdDnodes.StopAllSigint() + # tdDnodes.StopAllSigint() tdLog.info("Address sanitizer mode finished") sys.exit(0) diff --git a/tests/unit-test/test.sh b/tests/unit-test/test.sh index 292767e00c..71f5189551 100755 --- a/tests/unit-test/test.sh +++ b/tests/unit-test/test.sh @@ -40,7 +40,7 @@ pgrep taosd || taosd >> /dev/null 2>&1 & sleep 10 -ctest -E "smlTest|funcTest|profileTest|sdbTest|showTest|geomTest|idxFstUtilUT|idxTest|idxUtilUT|idxFstUT|parserTest|plannerTest|transUT|transUtilUt" -j8 +ctest -j8 ret=$? exit $ret From 478e1a67ae34d5fcac76105eaa1d168924b78f45 Mon Sep 17 00:00:00 2001 From: wangmm0220 Date: Tue, 6 Feb 2024 18:14:06 +0800 Subject: [PATCH 41/51] fix:[TD-28590]add logic for consume excluded --- include/common/tmsg.h | 1 + source/client/src/clientTmq.c | 39 +++--- source/common/src/tmsg.c | 2 + source/dnode/mnode/impl/src/mndDef.c | 136 +++++++-------------- source/dnode/mnode/impl/src/mndSubscribe.c | 86 ++++++------- tests/system-test/7-tmq/tmq_taosx.py | 40 ++++++ utils/test/c/tmq_taosx_ci.c | 85 +++++++++++++ 7 files changed, 234 insertions(+), 155 deletions(-) diff --git a/include/common/tmsg.h b/include/common/tmsg.h index 4731a30152..812d8a02a0 100644 --- a/include/common/tmsg.h +++ b/include/common/tmsg.h @@ -3768,6 +3768,7 @@ typedef struct { int32_t vgId; STqOffsetVal offset; int64_t rows; + int64_t ever; } OffsetRows; typedef struct { diff --git a/source/client/src/clientTmq.c b/source/client/src/clientTmq.c index 0f3883d161..3c3aee3032 100644 --- a/source/client/src/clientTmq.c +++ b/source/client/src/clientTmq.c @@ -789,27 +789,26 @@ void tmqSendHbReq(void* param, void* tmrId) { req.consumerId = tmq->consumerId; req.epoch = tmq->epoch; taosRLockLatch(&tmq->lock); -// if(tmq->needReportOffsetRows){ - req.topics = taosArrayInit(taosArrayGetSize(tmq->clientTopics), sizeof(TopicOffsetRows)); - for(int i = 0; i < taosArrayGetSize(tmq->clientTopics); i++){ - SMqClientTopic* pTopic = taosArrayGet(tmq->clientTopics, i); - int32_t numOfVgroups = taosArrayGetSize(pTopic->vgs); - TopicOffsetRows* data = taosArrayReserve(req.topics, 1); - strcpy(data->topicName, pTopic->topicName); - data->offsetRows = taosArrayInit(numOfVgroups, sizeof(OffsetRows)); - for(int j = 0; j < numOfVgroups; j++){ - SMqClientVg* pVg = taosArrayGet(pTopic->vgs, j); - OffsetRows* offRows = taosArrayReserve(data->offsetRows, 1); - offRows->vgId = pVg->vgId; - offRows->rows = pVg->numOfRows; - offRows->offset = pVg->offsetInfo.beginOffset; - char buf[TSDB_OFFSET_LEN] = {0}; - tFormatOffset(buf, TSDB_OFFSET_LEN, &offRows->offset); - tscInfo("consumer:0x%" PRIx64 ",report offset: vgId:%d, offset:%s, rows:%"PRId64, tmq->consumerId, offRows->vgId, buf, offRows->rows); - } + req.topics = taosArrayInit(taosArrayGetSize(tmq->clientTopics), sizeof(TopicOffsetRows)); + for(int i = 0; i < taosArrayGetSize(tmq->clientTopics); i++){ + SMqClientTopic* pTopic = taosArrayGet(tmq->clientTopics, i); + int32_t numOfVgroups = taosArrayGetSize(pTopic->vgs); + TopicOffsetRows* data = taosArrayReserve(req.topics, 1); + strcpy(data->topicName, pTopic->topicName); + data->offsetRows = taosArrayInit(numOfVgroups, sizeof(OffsetRows)); + for(int j = 0; j < numOfVgroups; j++){ + SMqClientVg* pVg = taosArrayGet(pTopic->vgs, j); + OffsetRows* offRows = taosArrayReserve(data->offsetRows, 1); + offRows->vgId = pVg->vgId; + offRows->rows = pVg->numOfRows; + offRows->offset = pVg->offsetInfo.endOffset; + offRows->ever = pVg->offsetInfo.walVerEnd; + char buf[TSDB_OFFSET_LEN] = {0}; + tFormatOffset(buf, TSDB_OFFSET_LEN, &offRows->offset); + tscInfo("consumer:0x%" PRIx64 ",report offset, group:%s vgId:%d, offset:%s/%"PRId64", rows:%"PRId64, + tmq->consumerId, tmq->groupId, offRows->vgId, buf, offRows->ever, offRows->rows); } -// tmq->needReportOffsetRows = false; -// } + } taosRUnLockLatch(&tmq->lock); int32_t tlen = tSerializeSMqHbReq(NULL, 0, &req); diff --git a/source/common/src/tmsg.c b/source/common/src/tmsg.c index add1f12fc1..2ffa12f2c1 100644 --- a/source/common/src/tmsg.c +++ b/source/common/src/tmsg.c @@ -6252,6 +6252,7 @@ int32_t tSerializeSMqHbReq(void *buf, int32_t bufLen, SMqHbReq *pReq) { if (tEncodeI32(&encoder, offRows->vgId) < 0) return -1; if (tEncodeI64(&encoder, offRows->rows) < 0) return -1; if (tEncodeSTqOffsetVal(&encoder, &offRows->offset) < 0) return -1; + if (tEncodeI64(&encoder, offRows->ever) < 0) return -1; } } @@ -6289,6 +6290,7 @@ int32_t tDeserializeSMqHbReq(void *buf, int32_t bufLen, SMqHbReq *pReq) { if (tDecodeI32(&decoder, &offRows->vgId) < 0) return -1; if (tDecodeI64(&decoder, &offRows->rows) < 0) return -1; if (tDecodeSTqOffsetVal(&decoder, &offRows->offset) < 0) return -1; + if (tDecodeI64(&decoder, &offRows->ever) < 0) return -1; } } } diff --git a/source/dnode/mnode/impl/src/mndDef.c b/source/dnode/mnode/impl/src/mndDef.c index 172c3952ad..d59354286d 100644 --- a/source/dnode/mnode/impl/src/mndDef.c +++ b/source/dnode/mnode/impl/src/mndDef.c @@ -422,27 +422,12 @@ void *tDecodeSMqConsumerObj(const void *buf, SMqConsumerObj *pConsumer, int8_t s return (void *)buf; } -// SMqConsumerEp *tCloneSMqConsumerEp(const SMqConsumerEp *pConsumerEpOld) { -// SMqConsumerEp *pConsumerEpNew = taosMemoryMalloc(sizeof(SMqConsumerEp)); -// if (pConsumerEpNew == NULL) return NULL; -// pConsumerEpNew->consumerId = pConsumerEpOld->consumerId; -// pConsumerEpNew->vgs = taosArrayDup(pConsumerEpOld->vgs, NULL); -// return pConsumerEpNew; -// } -// -// void tDeleteSMqConsumerEp(void *data) { -// SMqConsumerEp *pConsumerEp = (SMqConsumerEp *)data; -// taosArrayDestroy(pConsumerEp->vgs); -// } - -int32_t tEncodeSMqConsumerEp(void **buf, const SMqConsumerEp *pConsumerEp) { +int32_t tEncodeOffRows(void **buf, SArray *offsetRows){ int32_t tlen = 0; - tlen += taosEncodeFixedI64(buf, pConsumerEp->consumerId); - tlen += taosEncodeArray(buf, pConsumerEp->vgs, (FEncode)tEncodeSMqVgEp); - int32_t szVgs = taosArrayGetSize(pConsumerEp->offsetRows); + int32_t szVgs = taosArrayGetSize(offsetRows); tlen += taosEncodeFixedI32(buf, szVgs); for (int32_t j = 0; j < szVgs; ++j) { - OffsetRows *offRows = taosArrayGet(pConsumerEp->offsetRows, j); + OffsetRows *offRows = taosArrayGet(offsetRows, j); tlen += taosEncodeFixedI32(buf, offRows->vgId); tlen += taosEncodeFixedI64(buf, offRows->rows); tlen += taosEncodeFixedI8(buf, offRows->offset.type); @@ -454,53 +439,54 @@ int32_t tEncodeSMqConsumerEp(void **buf, const SMqConsumerEp *pConsumerEp) { } else { // do nothing } + tlen += taosEncodeFixedI64(buf, offRows->ever); } - // #if 0 - // int32_t sz = taosArrayGetSize(pConsumerEp->vgs); - // tlen += taosEncodeFixedI32(buf, sz); - // for (int32_t i = 0; i < sz; i++) { - // SMqVgEp *pVgEp = taosArrayGetP(pConsumerEp->vgs, i); - // tlen += tEncodeSMqVgEp(buf, pVgEp); - // } - // #endif + return tlen; } +int32_t tEncodeSMqConsumerEp(void **buf, const SMqConsumerEp *pConsumerEp) { + int32_t tlen = 0; + tlen += taosEncodeFixedI64(buf, pConsumerEp->consumerId); + tlen += taosEncodeArray(buf, pConsumerEp->vgs, (FEncode)tEncodeSMqVgEp); + + + return tlen + tEncodeOffRows(buf, pConsumerEp->offsetRows); +} + +void *tDecodeOffRows(const void *buf, SArray **offsetRows, int8_t sver){ + int32_t szVgs = 0; + buf = taosDecodeFixedI32(buf, &szVgs); + if (szVgs > 0) { + *offsetRows = taosArrayInit(szVgs, sizeof(OffsetRows)); + if (NULL == *offsetRows) return NULL; + for (int32_t j = 0; j < szVgs; ++j) { + OffsetRows *offRows = taosArrayReserve(*offsetRows, 1); + buf = taosDecodeFixedI32(buf, &offRows->vgId); + buf = taosDecodeFixedI64(buf, &offRows->rows); + buf = taosDecodeFixedI8(buf, &offRows->offset.type); + if (offRows->offset.type == TMQ_OFFSET__SNAPSHOT_DATA || offRows->offset.type == TMQ_OFFSET__SNAPSHOT_META) { + buf = taosDecodeFixedI64(buf, &offRows->offset.uid); + buf = taosDecodeFixedI64(buf, &offRows->offset.ts); + } else if (offRows->offset.type == TMQ_OFFSET__LOG) { + buf = taosDecodeFixedI64(buf, &offRows->offset.version); + } else { + // do nothing + } + if(sver > 2){ + buf = taosDecodeFixedI64(buf, &offRows->ever); + } + } + } + return (void *)buf; +} + void *tDecodeSMqConsumerEp(const void *buf, SMqConsumerEp *pConsumerEp, int8_t sver) { buf = taosDecodeFixedI64(buf, &pConsumerEp->consumerId); buf = taosDecodeArray(buf, &pConsumerEp->vgs, (FDecode)tDecodeSMqVgEp, sizeof(SMqVgEp), sver); if (sver > 1) { - int32_t szVgs = 0; - buf = taosDecodeFixedI32(buf, &szVgs); - if (szVgs > 0) { - pConsumerEp->offsetRows = taosArrayInit(szVgs, sizeof(OffsetRows)); - if (NULL == pConsumerEp->offsetRows) return NULL; - for (int32_t j = 0; j < szVgs; ++j) { - OffsetRows *offRows = taosArrayReserve(pConsumerEp->offsetRows, 1); - buf = taosDecodeFixedI32(buf, &offRows->vgId); - buf = taosDecodeFixedI64(buf, &offRows->rows); - buf = taosDecodeFixedI8(buf, &offRows->offset.type); - if (offRows->offset.type == TMQ_OFFSET__SNAPSHOT_DATA || offRows->offset.type == TMQ_OFFSET__SNAPSHOT_META) { - buf = taosDecodeFixedI64(buf, &offRows->offset.uid); - buf = taosDecodeFixedI64(buf, &offRows->offset.ts); - } else if (offRows->offset.type == TMQ_OFFSET__LOG) { - buf = taosDecodeFixedI64(buf, &offRows->offset.version); - } else { - // do nothing - } - } - } + buf = tDecodeOffRows(buf, &pConsumerEp->offsetRows, sver); } - // #if 0 - // int32_t sz; - // buf = taosDecodeFixedI32(buf, &sz); - // pConsumerEp->vgs = taosArrayInit(sz, sizeof(void *)); - // for (int32_t i = 0; i < sz; i++) { - // SMqVgEp *pVgEp = taosMemoryMalloc(sizeof(SMqVgEp)); - // buf = tDecodeSMqVgEp(buf, pVgEp); - // taosArrayPush(pConsumerEp->vgs, &pVgEp); - // } - // #endif return (void *)buf; } @@ -596,22 +582,7 @@ int32_t tEncodeSubscribeObj(void **buf, const SMqSubscribeObj *pSub) { tlen += taosEncodeArray(buf, pSub->unassignedVgs, (FEncode)tEncodeSMqVgEp); tlen += taosEncodeString(buf, pSub->dbName); - int32_t szVgs = taosArrayGetSize(pSub->offsetRows); - tlen += taosEncodeFixedI32(buf, szVgs); - for (int32_t j = 0; j < szVgs; ++j) { - OffsetRows *offRows = taosArrayGet(pSub->offsetRows, j); - tlen += taosEncodeFixedI32(buf, offRows->vgId); - tlen += taosEncodeFixedI64(buf, offRows->rows); - tlen += taosEncodeFixedI8(buf, offRows->offset.type); - if (offRows->offset.type == TMQ_OFFSET__SNAPSHOT_DATA || offRows->offset.type == TMQ_OFFSET__SNAPSHOT_META) { - tlen += taosEncodeFixedI64(buf, offRows->offset.uid); - tlen += taosEncodeFixedI64(buf, offRows->offset.ts); - } else if (offRows->offset.type == TMQ_OFFSET__LOG) { - tlen += taosEncodeFixedI64(buf, offRows->offset.version); - } else { - // do nothing - } - } + tlen += tEncodeOffRows(buf, pSub->offsetRows); tlen += taosEncodeString(buf, pSub->qmsg); return tlen; } @@ -639,26 +610,7 @@ void *tDecodeSubscribeObj(const void *buf, SMqSubscribeObj *pSub, int8_t sver) { buf = taosDecodeStringTo(buf, pSub->dbName); if (sver > 1) { - int32_t szVgs = 0; - buf = taosDecodeFixedI32(buf, &szVgs); - if (szVgs > 0) { - pSub->offsetRows = taosArrayInit(szVgs, sizeof(OffsetRows)); - if (NULL == pSub->offsetRows) return NULL; - for (int32_t j = 0; j < szVgs; ++j) { - OffsetRows *offRows = taosArrayReserve(pSub->offsetRows, 1); - buf = taosDecodeFixedI32(buf, &offRows->vgId); - buf = taosDecodeFixedI64(buf, &offRows->rows); - buf = taosDecodeFixedI8(buf, &offRows->offset.type); - if (offRows->offset.type == TMQ_OFFSET__SNAPSHOT_DATA || offRows->offset.type == TMQ_OFFSET__SNAPSHOT_META) { - buf = taosDecodeFixedI64(buf, &offRows->offset.uid); - buf = taosDecodeFixedI64(buf, &offRows->offset.ts); - } else if (offRows->offset.type == TMQ_OFFSET__LOG) { - buf = taosDecodeFixedI64(buf, &offRows->offset.version); - } else { - // do nothing - } - } - } + buf = tDecodeOffRows(buf, &pSub->offsetRows, sver); buf = taosDecodeString(buf, &pSub->qmsg); } else { pSub->qmsg = taosStrdup(""); diff --git a/source/dnode/mnode/impl/src/mndSubscribe.c b/source/dnode/mnode/impl/src/mndSubscribe.c index 0909003201..fbdfd81cdf 100644 --- a/source/dnode/mnode/impl/src/mndSubscribe.c +++ b/source/dnode/mnode/impl/src/mndSubscribe.c @@ -24,7 +24,7 @@ #include "tcompare.h" #include "tname.h" -#define MND_SUBSCRIBE_VER_NUMBER 2 +#define MND_SUBSCRIBE_VER_NUMBER 3 #define MND_SUBSCRIBE_RESERVE_SIZE 64 #define MND_CONSUMER_LOST_HB_CNT 6 @@ -530,51 +530,50 @@ static int32_t mndDoRebalance(SMnode *pMnode, const SMqRebInputObj *pInput, SMqR } } -// if(taosHashGetSize(pOutput->pSub->consumerHash) == 0) { // if all consumer is removed - SMqSubscribeObj *pSub = mndAcquireSubscribeByKey(pMnode, pInput->pRebInfo->key); // put all offset rows - if (pSub) { - taosRLockLatch(&pSub->lock); - if (pOutput->pSub->offsetRows == NULL) { - pOutput->pSub->offsetRows = taosArrayInit(4, sizeof(OffsetRows)); - } - pIter = NULL; - while (1) { - pIter = taosHashIterate(pSub->consumerHash, pIter); - if (pIter == NULL) break; - SMqConsumerEp *pConsumerEp = (SMqConsumerEp *)pIter; - SMqConsumerEp *pConsumerEpNew = taosHashGet(pOutput->pSub->consumerHash, &pConsumerEp->consumerId, sizeof(int64_t)); + SMqSubscribeObj *pSub = mndAcquireSubscribeByKey(pMnode, pInput->pRebInfo->key); // put all offset rows + if (pSub) { + taosRLockLatch(&pSub->lock); + if (pOutput->pSub->offsetRows == NULL) { + pOutput->pSub->offsetRows = taosArrayInit(4, sizeof(OffsetRows)); + } + pIter = NULL; + while (1) { + pIter = taosHashIterate(pSub->consumerHash, pIter); + if (pIter == NULL) break; + SMqConsumerEp *pConsumerEp = (SMqConsumerEp *)pIter; + SMqConsumerEp *pConsumerEpNew = taosHashGet(pOutput->pSub->consumerHash, &pConsumerEp->consumerId, sizeof(int64_t)); - for (int j = 0; j < taosArrayGetSize(pConsumerEp->offsetRows); j++) { - OffsetRows *d1 = taosArrayGet(pConsumerEp->offsetRows, j); - bool jump = false; - for (int i = 0; pConsumerEpNew && i < taosArrayGetSize(pConsumerEpNew->vgs); i++){ - SMqVgEp *pVgEp = taosArrayGetP(pConsumerEpNew->vgs, i); - if(pVgEp->vgId == d1->vgId){ - jump = true; - mInfo("pSub->offsetRows jump, because consumer id:0x%"PRIx64 " and vgId:%d not change", pConsumerEp->consumerId, pVgEp->vgId); - break; - } - } - if(jump) continue; - bool find = false; - for (int i = 0; i < taosArrayGetSize(pOutput->pSub->offsetRows); i++) { - OffsetRows *d2 = taosArrayGet(pOutput->pSub->offsetRows, i); - if (d1->vgId == d2->vgId) { - d2->rows += d1->rows; - d2->offset = d1->offset; - find = true; - mInfo("pSub->offsetRows add vgId:%d, after:%"PRId64", before:%"PRId64, d2->vgId, d2->rows, d1->rows); - break; - } - } - if(!find){ - taosArrayPush(pOutput->pSub->offsetRows, d1); + for (int j = 0; j < taosArrayGetSize(pConsumerEp->offsetRows); j++) { + OffsetRows *d1 = taosArrayGet(pConsumerEp->offsetRows, j); + bool jump = false; + for (int i = 0; pConsumerEpNew && i < taosArrayGetSize(pConsumerEpNew->vgs); i++){ + SMqVgEp *pVgEp = taosArrayGetP(pConsumerEpNew->vgs, i); + if(pVgEp->vgId == d1->vgId){ + jump = true; + mInfo("pSub->offsetRows jump, because consumer id:0x%"PRIx64 " and vgId:%d not change", pConsumerEp->consumerId, pVgEp->vgId); + break; } } + if(jump) continue; + bool find = false; + for (int i = 0; i < taosArrayGetSize(pOutput->pSub->offsetRows); i++) { + OffsetRows *d2 = taosArrayGet(pOutput->pSub->offsetRows, i); + if (d1->vgId == d2->vgId) { + d2->rows += d1->rows; + d2->offset = d1->offset; + d2->ever = d1->ever; + find = true; + mInfo("pSub->offsetRows add vgId:%d, after:%"PRId64", before:%"PRId64, d2->vgId, d2->rows, d1->rows); + break; + } + } + if(!find){ + taosArrayPush(pOutput->pSub->offsetRows, d1); + } } - taosRUnLockLatch(&pSub->lock); - mndReleaseSubscribe(pMnode, pSub); -// } + } + taosRUnLockLatch(&pSub->lock); + mndReleaseSubscribe(pMnode, pSub); } // 8. generate logs @@ -1405,8 +1404,9 @@ static int32_t buildResult(SSDataBlock *pBlock, int32_t* numOfRows, int64_t cons } if(data){ // vg id - char buf[TSDB_OFFSET_LEN + VARSTR_HEADER_SIZE] = {0}; + char buf[TSDB_OFFSET_LEN*2 + VARSTR_HEADER_SIZE] = {0}; tFormatOffset(varDataVal(buf), TSDB_OFFSET_LEN, &data->offset); + sprintf(varDataVal(buf) + strlen(varDataVal(buf)), "/%"PRId64, data->ever); varDataSetLen(buf, strlen(varDataVal(buf))); pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); colDataSetVal(pColInfo, *numOfRows, (const char *)buf, false); diff --git a/tests/system-test/7-tmq/tmq_taosx.py b/tests/system-test/7-tmq/tmq_taosx.py index 86c40fdc72..5bd70a5d60 100644 --- a/tests/system-test/7-tmq/tmq_taosx.py +++ b/tests/system-test/7-tmq/tmq_taosx.py @@ -11,6 +11,7 @@ from util.sql import * from util.cases import * from util.dnodes import * from util.common import * +from taos.tmq import * sys.path.append("./7-tmq") from tmqCommon import * @@ -310,6 +311,43 @@ class TDTestCase: return + def consumeExcluded(self): + tdSql.execute(f'create topic topic_excluded as database db_taosx') + consumer_dict = { + "group.id": "g1", + "td.connect.user": "root", + "td.connect.pass": "taosdata", + "auto.offset.reset": "earliest", + "msg.consume.excluded": 1 + } + consumer = Consumer(consumer_dict) + + tdLog.debug("test subscribe topic created by other user") + exceptOccured = False + try: + consumer.subscribe(["topic_excluded"]) + except TmqError: + exceptOccured = True + + if exceptOccured: + tdLog.exit(f"subscribe error") + + try: + while True: + res = consumer.poll(1) + if not res: + break + err = res.error() + if err is not None: + raise err + val = res.value() + + for block in val: + print(block.fetchall()) + + finally: + consumer.close() + def run(self): tdSql.prepare() self.checkWal1VgroupOnlyMeta() @@ -324,6 +362,8 @@ class TDTestCase: self.checkSnapshotMultiVgroups() self.checkWalMultiVgroupsWithDropTable() + # self.consumeExcluded() + self.checkSnapshotMultiVgroupsWithDropTable() def stop(self): diff --git a/utils/test/c/tmq_taosx_ci.c b/utils/test/c/tmq_taosx_ci.c index 8a7074844a..056b7dc6cf 100644 --- a/utils/test/c/tmq_taosx_ci.c +++ b/utils/test/c/tmq_taosx_ci.c @@ -909,6 +909,88 @@ void initLogFile() { taosCloseFile(&pFile2); } +void testConsumeExcluded(int topic_type){ + TAOS* pConn = use_db(); + TAOS_RES *pRes = NULL; + + if(topic_type == 1){ + char *topic = "create topic topic_excluded with meta as database db_taosx"; + pRes = taos_query(pConn, topic); + if (taos_errno(pRes) != 0) { + printf("failed to create topic topic_excluded, reason:%s\n", taos_errstr(pRes)); + taos_close(pConn); + return; + } + taos_free_result(pRes); + }else if(topic_type == 2){ + char *topic = "create topic topic_excluded as select * from stt"; + pRes = taos_query(pConn, topic); + if (taos_errno(pRes) != 0) { + printf("failed to create topic topic_excluded, reason:%s\n", taos_errstr(pRes)); + taos_close(pConn); + return; + } + taos_free_result(pRes); + } + taos_close(pConn); + + tmq_conf_t* conf = tmq_conf_new(); + tmq_conf_set(conf, "group.id", "tg2"); + tmq_conf_set(conf, "client.id", "my app 1"); + tmq_conf_set(conf, "td.connect.user", "root"); + tmq_conf_set(conf, "td.connect.pass", "taosdata"); + tmq_conf_set(conf, "msg.with.table.name", "true"); + tmq_conf_set(conf, "enable.auto.commit", "true"); + tmq_conf_set(conf, "auto.offset.reset", "earliest"); + tmq_conf_set(conf, "msg.consume.excluded", "1"); + + + tmq_conf_set_auto_commit_cb(conf, tmq_commit_cb_print, NULL); + tmq_t* tmq = tmq_consumer_new(conf, NULL, 0); + assert(tmq); + tmq_conf_destroy(conf); + + tmq_list_t* topic_list = tmq_list_new(); + tmq_list_append(topic_list, "topic_excluded"); + + int32_t code = 0; + + if ((code = tmq_subscribe(tmq, topic_list))) { + fprintf(stderr, "%% Failed to start consuming topics: %s\n", tmq_err2str(code)); + printf("subscribe err\n"); + return; + } + while (running) { + TAOS_RES* msg = tmq_consumer_poll(tmq, 1000); + if (msg) { + tmq_raw_data raw = {0}; + tmq_get_raw(msg, &raw); + if(topic_type == 1){ + assert(raw.raw_type != 2 && raw.raw_type != 4); + }else if(topic_type == 2){ + assert(0); + } +// printf("write raw data type: %d\n", raw.raw_type); + tmq_free_raw(raw); + + taos_free_result(msg); + } else { + break; + } + } + + tmq_consumer_close(tmq); + tmq_list_destroy(topic_list); + + pConn = use_db(); + pRes = taos_query(pConn, "drop topic if exists topic_excluded"); + if (taos_errno(pRes) != 0) { + printf("error in drop topic, reason:%s\n", taos_errstr(pRes)); + taos_close(pConn); + return; + } + taos_free_result(pRes); +} int main(int argc, char* argv[]) { for (int32_t i = 1; i < argc; i++) { if (strcmp(argv[i], "-c") == 0) { @@ -942,5 +1024,8 @@ int main(int argc, char* argv[]) { tmq_list_t* topic_list = build_topic_list(); basic_consume_loop(tmq, topic_list); tmq_list_destroy(topic_list); + + testConsumeExcluded(1); + testConsumeExcluded(2); taosCloseFile(&g_fp); } From 43624bbe08f24b95a568126b8fd58f37c4083ac2 Mon Sep 17 00:00:00 2001 From: wangmm0220 Date: Wed, 7 Feb 2024 09:52:41 +0800 Subject: [PATCH 42/51] fix:cases error --- tests/system-test/7-tmq/tmqParamsTest.py | 23 ++++++++++++++++------- 1 file changed, 16 insertions(+), 7 deletions(-) diff --git a/tests/system-test/7-tmq/tmqParamsTest.py b/tests/system-test/7-tmq/tmqParamsTest.py index 9286b69278..82a5d42b47 100644 --- a/tests/system-test/7-tmq/tmqParamsTest.py +++ b/tests/system-test/7-tmq/tmqParamsTest.py @@ -133,13 +133,17 @@ class TDTestCase: if snapshot_value == "true": if offset_value != "earliest" and offset_value != "": if offset_value == "latest": - offset_value_list = list(map(lambda x: int(x[-2].replace("wal:", "").replace("earliest", "0").replace("latest", "0").replace(offset_value, "0")), subscription_info)) - tdSql.checkEqual(sum(offset_value_list) >= 0, True) + offset_value_list = list(map(lambda x: (x[-2].replace("wal:", "").replace("earliest", "0").replace("latest", "0").replace(offset_value, "0")), subscription_info)) + offset_value_list1 = list(map(lambda x: int(x.split("/")[0]), offset_value_list)) + offset_value_list2 = list(map(lambda x: int(x.split("/")[1]), offset_value_list)) + tdSql.checkEqual(offset_value_list1 == offset_value_list2, True) + tdSql.checkEqual(sum(offset_value_list1) >= 0, True) rows_value_list = list(map(lambda x: int(x[-1]), subscription_info)) tdSql.checkEqual(sum(rows_value_list), expected_res) elif offset_value == "none": offset_value_list = list(map(lambda x: x[-2], subscription_info)) - tdSql.checkEqual(offset_value_list, ['none']*len(subscription_info)) + offset_value_list1 = list(map(lambda x: (x.split("/")[0]), offset_value_list)) + tdSql.checkEqual(offset_value_list1, ['none']*len(subscription_info)) rows_value_list = list(map(lambda x: x[-1], subscription_info)) tdSql.checkEqual(rows_value_list, [0]*len(subscription_info)) else: @@ -151,18 +155,23 @@ class TDTestCase: # tdSql.checkEqual(sum(rows_value_list), expected_res) else: offset_value_list = list(map(lambda x: x[-2], subscription_info)) - tdSql.checkEqual(offset_value_list, [None]*len(subscription_info)) + offset_value_list1 = list(map(lambda x: (x.split("/")[0]), offset_value_list)) + tdSql.checkEqual(offset_value_list1, [None]*len(subscription_info)) rows_value_list = list(map(lambda x: x[-1], subscription_info)) tdSql.checkEqual(rows_value_list, [None]*len(subscription_info)) else: if offset_value != "none": - offset_value_list = list(map(lambda x: int(x[-2].replace("wal:", "").replace("earliest", "0").replace("latest", "0").replace(offset_value, "0")), subscription_info)) - tdSql.checkEqual(sum(offset_value_list) >= 0, True) + offset_value_list = list(map(lambda x: (x[-2].replace("wal:", "").replace("earliest", "0").replace("latest", "0").replace(offset_value, "0")), subscription_info)) + offset_value_list1 = list(map(lambda x: int(x.split("/")[0]), offset_value_list)) + offset_value_list2 = list(map(lambda x: int(x.split("/")[1]), offset_value_list)) + tdSql.checkEqual(offset_value_list1 == offset_value_list2, True) + tdSql.checkEqual(sum(offset_value_list1) >= 0, True) rows_value_list = list(map(lambda x: int(x[-1]), subscription_info)) tdSql.checkEqual(sum(rows_value_list), expected_res) else: offset_value_list = list(map(lambda x: x[-2], subscription_info)) - tdSql.checkEqual(offset_value_list, ['none']*len(subscription_info)) + offset_value_list1 = list(map(lambda x: (x.split("/")[0]), offset_value_list)) + tdSql.checkEqual(offset_value_list1, ['none']*len(subscription_info)) rows_value_list = list(map(lambda x: x[-1], subscription_info)) tdSql.checkEqual(rows_value_list, [0]*len(subscription_info)) tdSql.execute(f"drop topic if exists {topic_name}") From 5473c22585723ee9a9e9d1417c04de13fc07ee43 Mon Sep 17 00:00:00 2001 From: 54liuyao <54liuyao> Date: Wed, 7 Feb 2024 11:30:13 +0800 Subject: [PATCH 43/51] mid pull over --- source/libs/executor/inc/executorInt.h | 2 ++ .../executor/src/streamtimewindowoperator.c | 21 ++++++++++++++++--- 2 files changed, 20 insertions(+), 3 deletions(-) diff --git a/source/libs/executor/inc/executorInt.h b/source/libs/executor/inc/executorInt.h index 46d7588604..a9ee304656 100644 --- a/source/libs/executor/inc/executorInt.h +++ b/source/libs/executor/inc/executorInt.h @@ -559,6 +559,8 @@ typedef struct SStreamIntervalOperatorInfo { struct SUpdateInfo* pUpdateInfo; bool recvRetrive; SSDataBlock* pMidRetriveRes; + bool recvPullover; + SSDataBlock* pMidPulloverRes; } SStreamIntervalOperatorInfo; typedef struct SDataGroupInfo { diff --git a/source/libs/executor/src/streamtimewindowoperator.c b/source/libs/executor/src/streamtimewindowoperator.c index 7c9b998c7d..660b181b0e 100644 --- a/source/libs/executor/src/streamtimewindowoperator.c +++ b/source/libs/executor/src/streamtimewindowoperator.c @@ -411,6 +411,7 @@ void destroyStreamFinalIntervalOperatorInfo(void* param) { taosArrayDestroy(pInfo->pDelWins); blockDataDestroy(pInfo->pDelRes); blockDataDestroy(pInfo->pMidRetriveRes); + blockDataDestroy(pInfo->pMidPulloverRes); pInfo->stateStore.streamFileStateDestroy(pInfo->pState->pFileState); if (pInfo->pState->dump == 1) { @@ -603,7 +604,7 @@ static void doBuildPullDataBlock(SArray* array, int32_t* pIndex, SSDataBlock* pB blockDataUpdateTsWindow(pBlock, 0); } -void processPullOver(SSDataBlock* pBlock, SHashObj* pMap, SHashObj* pFinalMap, SInterval* pInterval, SArray* pPullWins, +static bool processPullOver(SSDataBlock* pBlock, SHashObj* pMap, SHashObj* pFinalMap, SInterval* pInterval, SArray* pPullWins, int32_t numOfCh, SOperatorInfo* pOperator) { SColumnInfoData* pStartCol = taosArrayGet(pBlock->pDataBlock, CALCULATE_START_TS_COLUMN_INDEX); TSKEY* tsData = (TSKEY*)pStartCol->pData; @@ -612,6 +613,7 @@ void processPullOver(SSDataBlock* pBlock, SHashObj* pMap, SHashObj* pFinalMap, S SColumnInfoData* pGroupCol = taosArrayGet(pBlock->pDataBlock, GROUPID_COLUMN_INDEX); uint64_t* groupIdData = (uint64_t*)pGroupCol->pData; int32_t chId = getChildIndex(pBlock); + bool res = false; for (int32_t i = 0; i < pBlock->info.rows; i++) { TSKEY winTs = tsData[i]; while (winTs <= tsEndData[i]) { @@ -627,6 +629,7 @@ void processPullOver(SSDataBlock* pBlock, SHashObj* pMap, SHashObj* pFinalMap, S // pull data is over taosArrayDestroy(chArray); taosHashRemove(pMap, &winRes, sizeof(SWinKey)); + res =true; qDebug("===stream===retrive pull data over.window %" PRId64, winRes.ts); void* pFinalCh = taosHashGet(pFinalMap, &winRes, sizeof(SWinKey)); @@ -650,6 +653,7 @@ void processPullOver(SSDataBlock* pBlock, SHashObj* pMap, SHashObj* pFinalMap, S winTs = taosTimeAdd(winTs, pInterval->sliding, pInterval->slidingUnit, pInterval->precision); } } + return res; } static void addRetriveWindow(SArray* wins, SStreamIntervalOperatorInfo* pInfo, int32_t childId) { @@ -1186,6 +1190,12 @@ static SSDataBlock* buildIntervalResult(SOperatorInfo* pOperator) { printDataBlock(pInfo->binfo.pRes, getStreamOpName(opType), GET_TASKID(pTaskInfo)); return pInfo->binfo.pRes; } + + if (pInfo->recvPullover) { + pInfo->recvPullover = false; + printDataBlock(pInfo->pMidPulloverRes, getStreamOpName(pOperator->operatorType), GET_TASKID(pTaskInfo)); + return pInfo->pMidPulloverRes; + } return NULL; } @@ -1555,6 +1565,7 @@ SOperatorInfo* createStreamFinalIntervalOperatorInfo(SOperatorInfo* downstream, pInfo->pCheckpointRes = createSpecialDataBlock(STREAM_CHECKPOINT); pInfo->recvRetrive = false; pInfo->pMidRetriveRes = createSpecialDataBlock(STREAM_MID_RETRIEVE); + pInfo->pMidPulloverRes = createSpecialDataBlock(STREAM_MID_RETRIEVE); pOperator->operatorType = pPhyNode->type; if (!IS_FINAL_INTERVAL_OP(pOperator) || numOfChild == 0) { @@ -4336,8 +4347,12 @@ static SSDataBlock* doStreamMidIntervalAgg(SOperatorInfo* pOperator) { } else if (pBlock->info.type == STREAM_CREATE_CHILD_TABLE) { return pBlock; } else if (pBlock->info.type == STREAM_PULL_OVER) { - processPullOver(pBlock, pInfo->pPullDataMap, pInfo->pFinalPullDataMap, &pInfo->interval, pInfo->pPullWins, - pInfo->numOfChild, pOperator); + pInfo->recvPullover = processPullOver(pBlock, pInfo->pPullDataMap, pInfo->pFinalPullDataMap, &pInfo->interval, + pInfo->pPullWins, pInfo->numOfChild, pOperator); + if (pInfo->recvPullover) { + copyDataBlock(pInfo->pMidPulloverRes, pBlock); + break; + } continue; } else if (pBlock->info.type == STREAM_CHECKPOINT) { pAPI->stateStore.streamStateCommit(pInfo->pState); From b66260d4e9d0db14966bede2d26202d0bf2fcce6 Mon Sep 17 00:00:00 2001 From: 54liuyao <54liuyao> Date: Wed, 7 Feb 2024 13:29:29 +0800 Subject: [PATCH 44/51] mid pull over --- source/libs/executor/inc/executorInt.h | 1 + source/libs/executor/src/streamtimewindowoperator.c | 8 ++++++++ 2 files changed, 9 insertions(+) diff --git a/source/libs/executor/inc/executorInt.h b/source/libs/executor/inc/executorInt.h index a9ee304656..f7ed9805e8 100644 --- a/source/libs/executor/inc/executorInt.h +++ b/source/libs/executor/inc/executorInt.h @@ -561,6 +561,7 @@ typedef struct SStreamIntervalOperatorInfo { SSDataBlock* pMidRetriveRes; bool recvPullover; SSDataBlock* pMidPulloverRes; + bool clearState; } SStreamIntervalOperatorInfo; typedef struct SDataGroupInfo { diff --git a/source/libs/executor/src/streamtimewindowoperator.c b/source/libs/executor/src/streamtimewindowoperator.c index 660b181b0e..954a9659b4 100644 --- a/source/libs/executor/src/streamtimewindowoperator.c +++ b/source/libs/executor/src/streamtimewindowoperator.c @@ -1566,6 +1566,7 @@ SOperatorInfo* createStreamFinalIntervalOperatorInfo(SOperatorInfo* downstream, pInfo->recvRetrive = false; pInfo->pMidRetriveRes = createSpecialDataBlock(STREAM_MID_RETRIEVE); pInfo->pMidPulloverRes = createSpecialDataBlock(STREAM_MID_RETRIEVE); + pInfo->clearState = false; pOperator->operatorType = pPhyNode->type; if (!IS_FINAL_INTERVAL_OP(pOperator) || numOfChild == 0) { @@ -4287,6 +4288,11 @@ static SSDataBlock* doStreamMidIntervalAgg(SOperatorInfo* pOperator) { printDataBlock(pInfo->pDelRes, getStreamOpName(pOperator->operatorType), GET_TASKID(pTaskInfo)); return pInfo->pDelRes; } + if (pInfo->clearState) { + pInfo->clearState = false; + clearFunctionContext(&pOperator->exprSupp); + clearStreamIntervalOperator(pInfo); + } } if (!pInfo->pUpdated) { @@ -4341,6 +4347,7 @@ static SSDataBlock* doStreamMidIntervalAgg(SOperatorInfo* pOperator) { } else { pInfo->pDelRes->info.type = STREAM_DELETE_RESULT; } + pInfo->clearState = true; return pInfo->pDelRes; } continue; @@ -4351,6 +4358,7 @@ static SSDataBlock* doStreamMidIntervalAgg(SOperatorInfo* pOperator) { pInfo->pPullWins, pInfo->numOfChild, pOperator); if (pInfo->recvPullover) { copyDataBlock(pInfo->pMidPulloverRes, pBlock); + pInfo->clearState = true; break; } continue; From 6e19f452ce74fdafdd123eb957a98c4b76eb76cb Mon Sep 17 00:00:00 2001 From: 54liuyao <54liuyao> Date: Wed, 7 Feb 2024 13:55:01 +0800 Subject: [PATCH 45/51] mid pull over --- .../executor/src/streamtimewindowoperator.c | 20 +++++++++++++------ 1 file changed, 14 insertions(+), 6 deletions(-) diff --git a/source/libs/executor/src/streamtimewindowoperator.c b/source/libs/executor/src/streamtimewindowoperator.c index 954a9659b4..ca39776d3d 100644 --- a/source/libs/executor/src/streamtimewindowoperator.c +++ b/source/libs/executor/src/streamtimewindowoperator.c @@ -4283,10 +4283,9 @@ static SSDataBlock* doStreamMidIntervalAgg(SOperatorInfo* pOperator) { qDebug("stask:%s ===stream===%s clear", GET_TASKID(pTaskInfo), getStreamOpName(pOperator->operatorType)); return NULL; } else { - doBuildDeleteResult(pInfo, pInfo->pDelWins, &pInfo->delIndex, pInfo->pDelRes); - if (pInfo->pDelRes->info.rows != 0) { - printDataBlock(pInfo->pDelRes, getStreamOpName(pOperator->operatorType), GET_TASKID(pTaskInfo)); - return pInfo->pDelRes; + SSDataBlock* resBlock = buildIntervalResult(pOperator); + if (resBlock != NULL) { + return resBlock; } if (pInfo->clearState) { pInfo->clearState = false; @@ -4347,7 +4346,7 @@ static SSDataBlock* doStreamMidIntervalAgg(SOperatorInfo* pOperator) { } else { pInfo->pDelRes->info.type = STREAM_DELETE_RESULT; } - pInfo->clearState = true; + ASSERT(taosArrayGetSize(pInfo->pUpdated) == 0); return pInfo->pDelRes; } continue; @@ -4405,7 +4404,16 @@ static SSDataBlock* doStreamMidIntervalAgg(SOperatorInfo* pOperator) { pInfo->pUpdated = NULL; blockDataEnsureCapacity(pInfo->binfo.pRes, pOperator->resultInfo.capacity); - return buildIntervalResult(pOperator); + SSDataBlock* resBlock = buildIntervalResult(pOperator); + if (resBlock != NULL) { + return resBlock; + } + if (pInfo->clearState) { + pInfo->clearState = false; + clearFunctionContext(&pOperator->exprSupp); + clearStreamIntervalOperator(pInfo); + } + return NULL; } void setStreamOperatorCompleted(SOperatorInfo* pOperator) { From e2352875fe7aa765177129d9942fe1c7bd9c4ec2 Mon Sep 17 00:00:00 2001 From: 54liuyao <54liuyao> Date: Wed, 7 Feb 2024 14:43:04 +0800 Subject: [PATCH 46/51] mid pull over --- .../executor/src/streamtimewindowoperator.c | 20 ++++++++++++++++++- 1 file changed, 19 insertions(+), 1 deletion(-) diff --git a/source/libs/executor/src/streamtimewindowoperator.c b/source/libs/executor/src/streamtimewindowoperator.c index ca39776d3d..57348f2902 100644 --- a/source/libs/executor/src/streamtimewindowoperator.c +++ b/source/libs/executor/src/streamtimewindowoperator.c @@ -4287,6 +4287,13 @@ static SSDataBlock* doStreamMidIntervalAgg(SOperatorInfo* pOperator) { if (resBlock != NULL) { return resBlock; } + + if (pInfo->recvRetrive) { + pInfo->recvRetrive = false; + printDataBlock(pInfo->pMidRetriveRes, getStreamOpName(pOperator->operatorType), GET_TASKID(pTaskInfo)); + return pInfo->pMidRetriveRes; + } + if (pInfo->clearState) { pInfo->clearState = false; clearFunctionContext(&pOperator->exprSupp); @@ -4371,7 +4378,11 @@ static SSDataBlock* doStreamMidIntervalAgg(SOperatorInfo* pOperator) { doDeleteWindows(pOperator, &pInfo->interval, pBlock, delWins, pInfo->pUpdatedMap); addMidRetriveWindow(delWins, pInfo->pPullDataMap, pInfo->numOfChild); taosArrayDestroy(delWins); - continue; + pInfo->recvRetrive = true; + copyDataBlock(pInfo->pMidRetriveRes, pBlock); + pInfo->pMidRetriveRes->info.type = STREAM_MID_RETRIEVE; + pInfo->clearState = true; + break; } else { ASSERTS(pBlock->info.type == STREAM_INVALID, "invalid SSDataBlock type"); } @@ -4408,6 +4419,13 @@ static SSDataBlock* doStreamMidIntervalAgg(SOperatorInfo* pOperator) { if (resBlock != NULL) { return resBlock; } + + if (pInfo->recvRetrive) { + pInfo->recvRetrive = false; + printDataBlock(pInfo->pMidRetriveRes, getStreamOpName(pOperator->operatorType), GET_TASKID(pTaskInfo)); + return pInfo->pMidRetriveRes; + } + if (pInfo->clearState) { pInfo->clearState = false; clearFunctionContext(&pOperator->exprSupp); From 56ec97d418e80ee7cbb968b865d435233e395b79 Mon Sep 17 00:00:00 2001 From: 54liuyao <54liuyao> Date: Wed, 7 Feb 2024 15:24:11 +0800 Subject: [PATCH 47/51] mid pull over --- tests/script/tsim/stream/distributeInterval0.sim | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/tests/script/tsim/stream/distributeInterval0.sim b/tests/script/tsim/stream/distributeInterval0.sim index a4e7941c28..cc8b7227d7 100644 --- a/tests/script/tsim/stream/distributeInterval0.sim +++ b/tests/script/tsim/stream/distributeInterval0.sim @@ -475,6 +475,8 @@ sql create table ts1 using st tags(1,1,1); sql create table ts2 using st tags(2,2,2); sql create stream stream_t3 trigger at_once into streamtST3 as select ts, min(a) c6, a, b, c, ta, tb, tc from st interval(10s) ; +sleep 1000 + sql insert into ts1 values(1648791211000,1,2,3); sleep 50 sql insert into ts1 values(1648791222001,2,2,3); @@ -488,6 +490,9 @@ $loop_count = 0 loop3: sql select * from streamtST3; +print $data00 $data01 $data02 $data03 +print $data10 $data11 $data12 $data13 + sleep 1000 $loop_count = $loop_count + 1 if $loop_count == 30 then From aa9d78f3cca653d9d9053e838daf50eaf6cedad2 Mon Sep 17 00:00:00 2001 From: 54liuyao <54liuyao> Date: Wed, 7 Feb 2024 15:43:00 +0800 Subject: [PATCH 48/51] mid pull over --- tests/script/tsim/stream/fillHistoryBasic2.sim | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/script/tsim/stream/fillHistoryBasic2.sim b/tests/script/tsim/stream/fillHistoryBasic2.sim index 01019334a2..b3dd947eb9 100644 --- a/tests/script/tsim/stream/fillHistoryBasic2.sim +++ b/tests/script/tsim/stream/fillHistoryBasic2.sim @@ -243,6 +243,8 @@ sql create table ts1 using st tags(1,1,1); sql create table ts2 using st tags(2,2,2); sql create stream stream_t3 trigger at_once IGNORE EXPIRED 0 IGNORE UPDATE 0 into streamtST3 as select ts, min(a) c6, a, b, c, ta, tb, tc from st interval(10s) ; +sleep 1000 + sql insert into ts1 values(1648791211000,1,2,3); sleep 50 sql insert into ts1 values(1648791222001,2,2,3); From 7108e0dc4efdc8531855a9d0c5933a8e67350071 Mon Sep 17 00:00:00 2001 From: wangmm0220 Date: Sun, 18 Feb 2024 13:53:07 +0800 Subject: [PATCH 49/51] opti:docs for tmq --- docs/zh/07-develop/07-tmq.mdx | 1284 ++++++++++++++++----------------- docs/zh/07-develop/img_5.png | Bin 0 -> 18871 bytes docs/zh/07-develop/img_6.png | Bin 0 -> 49305 bytes docs/zh/07-develop/img_7.png | Bin 0 -> 15038 bytes docs/zh/12-taos-sql/13-tmq.md | 70 +- 5 files changed, 672 insertions(+), 682 deletions(-) create mode 100644 docs/zh/07-develop/img_5.png create mode 100644 docs/zh/07-develop/img_6.png create mode 100644 docs/zh/07-develop/img_7.png diff --git a/docs/zh/07-develop/07-tmq.mdx b/docs/zh/07-develop/07-tmq.mdx index 50913e87c8..df651eab96 100644 --- a/docs/zh/07-develop/07-tmq.mdx +++ b/docs/zh/07-develop/07-tmq.mdx @@ -15,334 +15,62 @@ import Node from "./_sub_node.mdx"; import CSharp from "./_sub_cs.mdx"; import CDemo from "./_sub_c.mdx"; -为了帮助应用实时获取写入 TDengine 的数据,或者以事件到达顺序处理数据,TDengine 提供了类似消息队列产品的数据订阅、消费接口。这样在很多场景下,采用 TDengine 的时序数据处理系统不再需要集成消息队列产品,比如 kafka, 从而简化系统设计的复杂度,降低运营维护成本。 -与 kafka 一样,你需要定义 *topic*, 但 TDengine 的 *topic* 是基于一个已经存在的超级表、子表或普通表的查询条件,即一个 `SELECT` 语句。你可以使用 SQL 对标签、表名、列、表达式等条件进行过滤,以及对数据进行标量函数与 UDF 计算(不包括数据聚合)。与其他消息队列软件相比,这是 TDengine 数据订阅功能的最大的优势,它提供了更大的灵活性,数据的颗粒度可以由应用随时调整,而且数据的过滤与预处理交给 TDengine,而不是应用完成,有效的减少传输的数据量与应用的复杂度。 +为了帮助应用实时获取写入 TDengine 的数据,或者以事件到达顺序处理数据,TDengine 提供了类似 kafka 的数据订阅功能。这样在很多场景下,采用 TDengine 的时序数据处理系统不再需要集成消息队列产品,比如 kafka, 从而简化系统设计的复杂度,降低运营维护成本。 -消费者订阅 *topic* 后,可以实时获得最新的数据。多个消费者可以组成一个消费者组 (consumer group), 一个消费者组里的多个消费者共享消费进度,便于多线程、分布式地消费数据,提高消费速度。但不同消费者组中的消费者即使消费同一个 topic, 并不共享消费进度。一个消费者可以订阅多个 topic。如果订阅的是超级表,数据可能会分布在多个不同的 vnode 上,也就是多个 shard 上,这样一个消费组里有多个消费者可以提高消费效率。TDengine 的消息队列提供了消息的 ACK 机制,在宕机、重启等复杂环境下确保 at least once 消费。 +# 介绍 +## 主题 +与 kafka 一样,你需要定义 topic, TDengine 的 topic 有三种,可以是数据库,超级表,或者一个 `SELECT` 语句,具体的语法参见 [CREATE TOPIC](../../12-taos-sql/13-tmq)。与其他消息队列软件相比,这是 TDengine 数据订阅功能的最大的优势,它提供了更大的灵活性,数据的颗粒度可以由应用随时调整,而且数据的过滤与预处理交给 TDengine,而不是应用完成,有效的减少传输的数据量与应用的复杂度。 -为了实现上述功能,TDengine 会为 WAL (Write-Ahead-Log) 文件自动创建索引以支持快速随机访问,并提供了灵活可配置的文件切换与保留机制:用户可以按需指定 WAL 文件保留的时间以及大小(详见 create database 语句)。通过以上方式将 WAL 改造成了一个保留事件到达顺序的、可持久化的存储引擎(但由于 TSDB 具有远比 WAL 更高的压缩率,我们不推荐保留太长时间,一般来说,不超过几天)。 对于以 topic 形式创建的查询,TDengine 将对接 WAL 而不是 TSDB 作为其存储引擎。在消费时,TDengine 根据当前消费进度从 WAL 直接读取数据,并使用统一的查询引擎实现过滤、变换等操作,将数据推送给消费者。 +如下图,每个 topic 涉及到的数据表可能分布在多个 vnode(相当于 kafka 里的 partition) 上,每个 vnode 上的数据保存在 WAL(Write-Ahead-Log) 文件中,WAL 文件里的数据是顺序写入的(由于 WAL 文件中存储的不只有数据,还有元数据,写入消息等,所以数据的版本号不是连续的)。 -下面为关于数据订阅的一些说明,需要对TDengine的架构有一些了解,结合各个语言链接器的接口使用。(可使用时再了解) -- 一个消费组消费同一个topic下的所有数据,不同消费组之间相互独立; -- 一个消费组消费同一个topic所有的vgroup,消费组可由多个消费者组成,但一个vgroup仅被一个消费者消费,如果消费者数量超过了vgroup数量,多余的消费者不消费数据; -- 在服务端每个vgroup仅保存一个offset,每个vgroup的offset是单调递增的,但不一定连续。各个vgroup的offset之间没有关联; -- 每次poll服务端会返回一个结果block,该block属于一个vgroup,可能包含多个wal版本的数据,可以通过 offset 接口获得是该block第一条记录的offset; -- 一个消费组如果从未commit过offset,当其成员消费者重启重新拉取数据时,均从参数auto.offset.reset设定值开始消费;在一个消费者生命周期中,客户端本地记录了最近一次拉取数据的offset,不会拉取重复数据; -- 消费者如果异常终止(没有调用tmq_close),需等约12秒后触发其所属消费组rebalance,该消费者在服务端状态变为LOST,约1天后该消费者自动被删除;正常退出,退出后就会删除消费者;新增消费者,需等约2秒触发rebalance,该消费者在服务端状态变为ready; -- 消费组rebalance会对该组所有ready状态的消费者成员重新进行vgroup分配,消费者仅能对自己负责的vgroup进行assignment/seek/commit/poll操作; -- 消费者可利用 position 获得当前消费的offset,并seek到指定offset,重新消费; -- seek将position指向指定offset,不执行commit操作,一旦seek成功,可poll拉取指定offset及以后的数据; -- seek 操作之前须调用 assignment 接口获取该consumer的vgroup ID和offset范围。seek 操作会检测vgroup ID 和 offset是否合法,如非法将报错; -- position是获取当前的消费位置,是下次要取的位置,不是当前消费到的位置 -- commit是提交消费位置,不带参数的话,是提交当前消费位置(下次要取的位置,不是当前消费到的位置),带参数的话,是提交参数里的位置(也即下次退出重启后要取的位置) -- seek是设置consumer消费位置,seek到哪,position就返回哪,都是下次要取的位置 -- seek不会影响commit,commit不影响seek,相互独立,两个是不同的概念 -- begin接口为wal 第一条数据的offset,end 接口为wal 最后一条数据的offset + 1 -- offset接口获取的是记录所在结果block块里的第一条数据的offset,当seek至该offset时,将消费到这个block里的全部数据。参见第四点; -- 由于存在 WAL 过期删除机制,即使seek 操作成功,poll数据时有可能offset已失效。如果poll 的offset 小于 WAL 最小版本号,将会从WAL最小版本号消费; -- 数据订阅是从 WAL 消费数据,如果一些 WAL 文件被基于 WAL 保留策略删除,则已经删除的 WAL 文件中的数据就无法再消费到。需要根据业务需要在创建数据库时合理设置 `WAL_RETENTION_PERIOD` 或 `WAL_RETENTION_SIZE` ,并确保应用及时消费数据,这样才不会产生数据丢失的现象。数据订阅的行为与 Kafka 等广泛使用的消息队列类产品的行为相似; +![img_5.png](img_5.png) -本文档不对消息队列本身的知识做更多的介绍,如果需要了解,请自行搜索。 +TDengine 会为 WAL 文件自动创建索引以支持快速随机访问,并提供了灵活可配置的文件切换与保留机制,用户可以按需指定 WAL 文件保留的时间以及大小(详见 [CREATE DATABASE](../../12-taos-sql/02-database) 语句,由于消费是通过 WAL 实现的,所以应该根据写入消费速度来确定 WAL 的保存时长)。通过以上方式将 WAL 改造成了一个保留事件到达顺序的、可持久化的存储引擎。 -说明: +对于 `SELECT` 语句形式的 topic,在消费时,TDengine 根据当前消费进度从 WAL 直接读取数据,并使用统一的查询引擎实现过滤、变换等操作,将数据推送给消费者。 + +## 生产者 +写入 topic 相关联的数据表中数据的都是生产者,生产者实际生产的数据写入到了子表或普通表中,即表所在 vnode 的 WAL 里。 + +## 消费者 +### 消费者组 +消费者订阅 topic 后,可以消费 topic 里的所有数据(这些数据所在的表可能分布在多个 vnode 上,即 db 所在的所有 vnode)。订阅 topic 时,需要指定一个消费者组 (consumer group),如果这个消费者组里只有一个消费者,那么这个消费者会顺序的消费这些 vnode 上的数据。 + +为了提高消费速度,便于多线程、分布式地消费数据,可以在一个消费组里添加多个消费者,这些消费者将均分数据所在的 vnode 进行消费(比如数据分布在 4 个 vnode 上,有 2 个消费者的话,那么每个消费者消费 2 个 vnode;有 3 个消费者的话,2 个消费者各消费 1 个 vnode,1 个消费者消费 2 个 vnode;有 5 个消费者的话,4 个各分配 1 个 vnode 消费,另外 1 个不消费),如下图: + +![img_6.png](img_6.png) + +在一个消费组里添加一个消费者后,在 Mnode 上通过 rebalance 的机制实现消费者的重新分配,该操作对用户是透明的。 + +一个消费者可以订阅多个 topic。TDengine 的数据订阅在宕机、重启等复杂环境下确保 at least once 消费。 +### 消费进度 +在 topic 的一个消费组的一个 vnode 上有消费进度。消费者消费的同时,可以提交消费进度,消费进度即 vnode 上 WAL 的版本号(对于 kafka 里的 offset),消费进度可以手动提交,也可以通过参数(auto.commit.interval.ms)设置为周期性自动提交。 + +首次消费数据时通过订阅参数(auto.offset.reset)来确定消费位置为最新数据(latest)还是最旧数据(earliest)。 + +消费进度在一个 vnode 上对于同一个 topic 和 消费者组是唯一的。所以如果同一个 topic 和 消费者组在一个 vnode 上的消费者退出了,并且提交了消费进度。然后同一个 topic 和 消费者组里重新建了一个新的消费者消费这个 vnode,那么这个新消费者将继承之前的消费进度继续消费。 + +如果之前的消费者没有提交消费进度,那个新的消费者将根据订阅参数(auto.offset.reset)设置的值来确定起始消费位置。 + +不同消费者组中的消费者即使消费同一个 topic, 并不共享消费进度。 + +![img_7.png](img_7.png) + +作为一个数据库产品, WAL 文件中存储的不全是数据,也包括其他写入消息,元数据等,所以消费进度不是连续的。 + +##说明 从3.2.0.0版本开始,数据订阅支持vnode迁移和分裂。 -由于数据订阅依赖wal文件,而在vnode迁移和分裂的过程中,wal并不会同步过去,所以迁移或分裂后,之前没消费完的wal数据后消费不到。所以请保证之前把数据全部消费完后,再进行vnode迁移或分裂,否则,消费会丢失数据。 -## 主要数据结构和 API +由于数据订阅依赖wal文件,而在vnode迁移和分裂的过程中,wal并不会同步过去,所以迁移或分裂后,之前没消费完的wal数据后消费不到。所以请保证迁移和分裂之前把数据全部消费完后,再进行vnode迁移或分裂,否则,消费会丢失数据。 -不同语言下, TMQ 订阅相关的 API 及数据结构如下(注意consumer结构不是线程安全的,在一个线程使用consumer时,不要在另一个线程close这个consumer): +# 语法说明 - - +具体的语法参见 [数据订阅](../../12-taos-sql/13-tmq) -```c - typedef struct tmq_t tmq_t; - typedef struct tmq_conf_t tmq_conf_t; - typedef struct tmq_list_t tmq_list_t; +# 消费参数 - typedef void(tmq_commit_cb(tmq_t *tmq, int32_t code, void *param)); - - typedef enum tmq_conf_res_t { - TMQ_CONF_UNKNOWN = -2, - TMQ_CONF_INVALID = -1, - TMQ_CONF_OK = 0, - } tmq_conf_res_t; - - typedef struct tmq_topic_assignment { - int32_t vgId; - int64_t currentOffset; - int64_t begin; - int64_t end; - } tmq_topic_assignment; - - DLL_EXPORT tmq_conf_t *tmq_conf_new(); - DLL_EXPORT tmq_conf_res_t tmq_conf_set(tmq_conf_t *conf, const char *key, const char *value); - DLL_EXPORT void tmq_conf_destroy(tmq_conf_t *conf); - DLL_EXPORT void tmq_conf_set_auto_commit_cb(tmq_conf_t *conf, tmq_commit_cb *cb, void *param); - - DLL_EXPORT tmq_list_t *tmq_list_new(); - DLL_EXPORT int32_t tmq_list_append(tmq_list_t *, const char *); - DLL_EXPORT void tmq_list_destroy(tmq_list_t *); - DLL_EXPORT int32_t tmq_list_get_size(const tmq_list_t *); - DLL_EXPORT char **tmq_list_to_c_array(const tmq_list_t *); - - DLL_EXPORT tmq_t *tmq_consumer_new(tmq_conf_t *conf, char *errstr, int32_t errstrLen); - DLL_EXPORT int32_t tmq_subscribe(tmq_t *tmq, const tmq_list_t *topic_list); - DLL_EXPORT int32_t tmq_unsubscribe(tmq_t *tmq); - DLL_EXPORT int32_t tmq_subscription(tmq_t *tmq, tmq_list_t **topics); - DLL_EXPORT TAOS_RES *tmq_consumer_poll(tmq_t *tmq, int64_t timeout); - DLL_EXPORT int32_t tmq_consumer_close(tmq_t *tmq); - DLL_EXPORT int32_t tmq_commit_sync(tmq_t *tmq, const TAOS_RES *msg); - DLL_EXPORT void tmq_commit_async(tmq_t *tmq, const TAOS_RES *msg, tmq_commit_cb *cb, void *param); - DLL_EXPORT int32_t tmq_commit_offset_sync(tmq_t *tmq, const char *pTopicName, int32_t vgId, int64_t offset); - DLL_EXPORT void tmq_commit_offset_async(tmq_t *tmq, const char *pTopicName, int32_t vgId, int64_t offset, tmq_commit_cb *cb, void *param); - DLL_EXPORT int32_t tmq_get_topic_assignment(tmq_t *tmq, const char *pTopicName, tmq_topic_assignment **assignment,int32_t *numOfAssignment); - DLL_EXPORT void tmq_free_assignment(tmq_topic_assignment* pAssignment); - DLL_EXPORT int32_t tmq_offset_seek(tmq_t *tmq, const char *pTopicName, int32_t vgId, int64_t offset); - DLL_EXPORT int64_t tmq_position(tmq_t *tmq, const char *pTopicName, int32_t vgId); - DLL_EXPORT int64_t tmq_committed(tmq_t *tmq, const char *pTopicName, int32_t vgId); - - DLL_EXPORT const char *tmq_get_topic_name(TAOS_RES *res); - DLL_EXPORT const char *tmq_get_db_name(TAOS_RES *res); - DLL_EXPORT int32_t tmq_get_vgroup_id(TAOS_RES *res); - DLL_EXPORT int64_t tmq_get_vgroup_offset(TAOS_RES* res); - DLL_EXPORT const char *tmq_err2str(int32_t code); -``` - -下面介绍一下它们的具体用法(超级表和子表结构请参考“数据建模”一节),完整的示例代码请见下面 C 语言的示例代码。 - - - - -```java -void subscribe(Collection topics) throws SQLException; - -void unsubscribe() throws SQLException; - -Set subscription() throws SQLException; - -ConsumerRecords poll(Duration timeout) throws SQLException; - -Set assignment() throws SQLException; -long position(TopicPartition partition) throws SQLException; -Map position(String topic) throws SQLException; -Map beginningOffsets(String topic) throws SQLException; -Map endOffsets(String topic) throws SQLException; -Map committed(Set partitions) throws SQLException; - -void seek(TopicPartition partition, long offset) throws SQLException; -void seekToBeginning(Collection partitions) throws SQLException; -void seekToEnd(Collection partitions) throws SQLException; - -void commitSync() throws SQLException; -void commitSync(Map offsets) throws SQLException; - -void close() throws SQLException; -``` - - - - - -```python -class Consumer: - def subscribe(self, topics): - pass - - def unsubscribe(self): - pass - - def poll(self, timeout: float = 1.0): - pass - - def assignment(self): - pass - - def seek(self, partition): - pass - - def close(self): - pass - - def commit(self, message): - pass -``` - - - - - -```go -func NewConsumer(conf *tmq.ConfigMap) (*Consumer, error) - -// 出于兼容目的保留 rebalanceCb 参数,当前未使用 -func (c *Consumer) Subscribe(topic string, rebalanceCb RebalanceCb) error - -// 出于兼容目的保留 rebalanceCb 参数,当前未使用 -func (c *Consumer) SubscribeTopics(topics []string, rebalanceCb RebalanceCb) error - -func (c *Consumer) Poll(timeoutMs int) tmq.Event - -// 出于兼容目的保留 tmq.TopicPartition 参数,当前未使用 -func (c *Consumer) Commit() ([]tmq.TopicPartition, error) - -func (c *Consumer) Unsubscribe() error - -func (c *Consumer) Close() error -``` - - - - - -```rust -impl TBuilder for TmqBuilder - fn from_dsn(dsn: D) -> Result - fn build(&self) -> Result - -impl AsAsyncConsumer for Consumer - async fn subscribe, I: IntoIterator + Send>( - &mut self, - topics: I, - ) -> Result<(), Self::Error>; - fn stream( - &self, - ) -> Pin< - Box< - dyn '_ - + Send - + futures::Stream< - Item = Result<(Self::Offset, MessageSet), Self::Error>, - >, - >, - >; - async fn commit(&self, offset: Self::Offset) -> Result<(), Self::Error>; - - async fn unsubscribe(self); -``` - -可在 上查看详细 API 说明。 - - - - - -```js -function TMQConsumer(config) - -function subscribe(topic) - -function consume(timeout) - -function subscription() - -function unsubscribe() - -function commit(msg) - -function close() -``` - - - - - -```csharp -class ConsumerBuilder - -ConsumerBuilder(IEnumerable> config) - -public IConsumer Build() - -void Subscribe(IEnumerable topics) - -void Subscribe(string topic) - -ConsumeResult Consume(int millisecondsTimeout) - -List Subscription() - -void Unsubscribe() - -List Commit() - -void Close() -``` - - - - -## 写入数据 - -首先完成建库、建一张超级表和多张子表操作,然后就可以写入数据了,比如: - -```sql -DROP DATABASE IF EXISTS tmqdb; -CREATE DATABASE tmqdb WAL_RETENTION_PERIOD 3600; -CREATE TABLE tmqdb.stb (ts TIMESTAMP, c1 INT, c2 FLOAT, c3 VARCHAR(16)) TAGS(t1 INT, t3 VARCHAR(16)); -CREATE TABLE tmqdb.ctb0 USING tmqdb.stb TAGS(0, "subtable0"); -CREATE TABLE tmqdb.ctb1 USING tmqdb.stb TAGS(1, "subtable1"); -INSERT INTO tmqdb.ctb0 VALUES(now, 0, 0, 'a0')(now+1s, 0, 0, 'a00'); -INSERT INTO tmqdb.ctb1 VALUES(now, 1, 1, 'a1')(now+1s, 11, 11, 'a11'); -``` - -## 创建 *topic* - -TDengine 使用 SQL 创建一个 topic: - -```sql -CREATE TOPIC topic_name AS SELECT ts, c1, c2, c3 FROM tmqdb.stb WHERE c1 > 1; -``` -- topic创建个数有上限,通过参数 tmqMaxTopicNum 控制,默认 20 个 - -TMQ 支持多种订阅类型: - -### 列订阅 - -语法: - -```sql -CREATE TOPIC topic_name as subquery -``` - -通过 `SELECT` 语句订阅(包括 `SELECT *`,或 `SELECT ts, c1` 等指定列订阅,可以带条件过滤、标量函数计算,但不支持聚合函数、不支持时间窗口聚合)。需要注意的是: - -- 该类型 TOPIC 一旦创建则订阅数据的结构确定。 -- 被订阅或用于计算的列或标签不可被删除(`ALTER table DROP`)、修改(`ALTER table MODIFY`)。 -- 若发生表结构变更,新增的列不出现在结果中。 - -### 超级表订阅 - -语法: - -```sql -CREATE TOPIC topic_name [with meta] AS STABLE stb_name [where_condition] -``` - -与 `SELECT * from stbName` 订阅的区别是: - -- 不会限制用户的表结构变更。 -- 返回的是非结构化的数据:返回数据的结构会随之超级表的表结构变化而变化。 -- with meta 参数可选,选择时将返回创建超级表,子表等语句,主要用于taosx做超级表迁移 -- where_condition 参数可选,选择时将用来过滤符合条件的子表,订阅这些子表。where 条件里不能有普通列,只能是tag或tbname,where条件里可以用函数,用来过滤tag,但是不能是聚合函数,因为子表tag值无法做聚合。也可以是常量表达式,比如 2 > 1(订阅全部子表),或者 false(订阅0个子表) -- 返回数据不包含标签。 - -### 数据库订阅 - -语法: - -```sql -CREATE TOPIC topic_name [with meta] AS DATABASE db_name; -``` - -通过该语句可创建一个包含数据库所有表数据的订阅 - -- with meta 参数可选,选择时将返回创建数据库里所有超级表,子表的语句,主要用于taosx做数据库迁移 - -## 创建消费者 *consumer* - -消费者需要通过一系列配置选项创建,基础配置项如下表所示: +消费参数主要用于消费者创建时指定,基础配置项如下表所示: | 参数名称 | 类型 | 参数说明 | 备注 | | :----------------------------: | :-----: | -------------------------------------------------------- | ------------------------------------------- | @@ -358,515 +86,714 @@ CREATE TOPIC topic_name [with meta] AS DATABASE db_name; | `msg.with.table.name` | boolean | 是否允许从消息中解析表名, 不适用于列订阅(列订阅时可将 tbname 作为列写入 subquery 语句)(从3.2.0.0版本该参数废弃,恒为true) |默认关闭 | | `enable.replay` | boolean | 是否开启数据回放功能 |默认关闭 | -对于不同编程语言,其设置方式如下: +# 主要数据结构和 API 接口 + +不同语言下, TMQ 订阅相关的 API 及数据结构如下(详细的接口说明可以参考连接器章节,注意consumer结构不是线程安全的,在一个线程使用consumer时,不要在另一个线程close这个consumer): - + -```c -/* 根据需要,设置消费组 (group.id)、自动提交 (enable.auto.commit)、 - 自动提交时间间隔 (auto.commit.interval.ms)、用户名 (td.connect.user)、密码 (td.connect.pass) 等参数 */ -tmq_conf_t* conf = tmq_conf_new(); -tmq_conf_set(conf, "enable.auto.commit", "true"); -tmq_conf_set(conf, "auto.commit.interval.ms", "1000"); -tmq_conf_set(conf, "group.id", "cgrpName"); -tmq_conf_set(conf, "td.connect.user", "root"); -tmq_conf_set(conf, "td.connect.pass", "taosdata"); -tmq_conf_set(conf, "auto.offset.reset", "latest"); -tmq_conf_set(conf, "msg.with.table.name", "true"); -tmq_conf_set_auto_commit_cb(conf, tmq_commit_cb_print, NULL); + ```c + typedef struct tmq_t tmq_t; + typedef struct tmq_conf_t tmq_conf_t; + typedef struct tmq_list_t tmq_list_t; -tmq_t* tmq = tmq_consumer_new(conf, NULL, 0); -tmq_conf_destroy(conf); + typedef void(tmq_commit_cb(tmq_t *tmq, int32_t code, void *param)); + + typedef enum tmq_conf_res_t { + TMQ_CONF_UNKNOWN = -2, + TMQ_CONF_INVALID = -1, + TMQ_CONF_OK = 0, + } tmq_conf_res_t; + + typedef struct tmq_topic_assignment { + int32_t vgId; + int64_t currentOffset; + int64_t begin; + int64_t end; + } tmq_topic_assignment; + + DLL_EXPORT tmq_conf_t *tmq_conf_new(); + DLL_EXPORT tmq_conf_res_t tmq_conf_set(tmq_conf_t *conf, const char *key, const char *value); + DLL_EXPORT void tmq_conf_destroy(tmq_conf_t *conf); + DLL_EXPORT void tmq_conf_set_auto_commit_cb(tmq_conf_t *conf, tmq_commit_cb *cb, void *param); + + DLL_EXPORT tmq_list_t *tmq_list_new(); + DLL_EXPORT int32_t tmq_list_append(tmq_list_t *, const char *); + DLL_EXPORT void tmq_list_destroy(tmq_list_t *); + DLL_EXPORT int32_t tmq_list_get_size(const tmq_list_t *); + DLL_EXPORT char **tmq_list_to_c_array(const tmq_list_t *); + + DLL_EXPORT tmq_t *tmq_consumer_new(tmq_conf_t *conf, char *errstr, int32_t errstrLen); + DLL_EXPORT int32_t tmq_subscribe(tmq_t *tmq, const tmq_list_t *topic_list); + DLL_EXPORT int32_t tmq_unsubscribe(tmq_t *tmq); + DLL_EXPORT int32_t tmq_subscription(tmq_t *tmq, tmq_list_t **topics); + DLL_EXPORT TAOS_RES *tmq_consumer_poll(tmq_t *tmq, int64_t timeout); + DLL_EXPORT int32_t tmq_consumer_close(tmq_t *tmq); + DLL_EXPORT int32_t tmq_commit_sync(tmq_t *tmq, const TAOS_RES *msg); + DLL_EXPORT void tmq_commit_async(tmq_t *tmq, const TAOS_RES *msg, tmq_commit_cb *cb, void *param); + DLL_EXPORT int32_t tmq_commit_offset_sync(tmq_t *tmq, const char *pTopicName, int32_t vgId, int64_t offset); + DLL_EXPORT void tmq_commit_offset_async(tmq_t *tmq, const char *pTopicName, int32_t vgId, int64_t offset, tmq_commit_cb *cb, void *param); + DLL_EXPORT int32_t tmq_get_topic_assignment(tmq_t *tmq, const char *pTopicName, tmq_topic_assignment **assignment,int32_t *numOfAssignment); + DLL_EXPORT void tmq_free_assignment(tmq_topic_assignment* pAssignment); + DLL_EXPORT int32_t tmq_offset_seek(tmq_t *tmq, const char *pTopicName, int32_t vgId, int64_t offset); + DLL_EXPORT int64_t tmq_position(tmq_t *tmq, const char *pTopicName, int32_t vgId); + DLL_EXPORT int64_t tmq_committed(tmq_t *tmq, const char *pTopicName, int32_t vgId); + + DLL_EXPORT const char *tmq_get_topic_name(TAOS_RES *res); + DLL_EXPORT const char *tmq_get_db_name(TAOS_RES *res); + DLL_EXPORT int32_t tmq_get_vgroup_id(TAOS_RES *res); + DLL_EXPORT int64_t tmq_get_vgroup_offset(TAOS_RES* res); + DLL_EXPORT const char *tmq_err2str(int32_t code); + ``` + + + + + ```java + void subscribe(Collection topics) throws SQLException; + + void unsubscribe() throws SQLException; + + Set subscription() throws SQLException; + + ConsumerRecords poll(Duration timeout) throws SQLException; + + Set assignment() throws SQLException; + long position(TopicPartition partition) throws SQLException; + Map position(String topic) throws SQLException; + Map beginningOffsets(String topic) throws SQLException; + Map endOffsets(String topic) throws SQLException; + Map committed(Set partitions) throws SQLException; + + void seek(TopicPartition partition, long offset) throws SQLException; + void seekToBeginning(Collection partitions) throws SQLException; + void seekToEnd(Collection partitions) throws SQLException; + + void commitSync() throws SQLException; + void commitSync(Map offsets) throws SQLException; + + void close() throws SQLException; + ``` + + + + + + ```python + class Consumer: + def subscribe(self, topics): + pass + + def unsubscribe(self): + pass + + def poll(self, timeout: float = 1.0): + pass + + def assignment(self): + pass + + def seek(self, partition): + pass + + def close(self): + pass + + def commit(self, message): + pass + ``` + + + + + + ```go + func NewConsumer(conf *tmq.ConfigMap) (*Consumer, error) + + // 出于兼容目的保留 rebalanceCb 参数,当前未使用 + func (c *Consumer) Subscribe(topic string, rebalanceCb RebalanceCb) error + + // 出于兼容目的保留 rebalanceCb 参数,当前未使用 + func (c *Consumer) SubscribeTopics(topics []string, rebalanceCb RebalanceCb) error + + func (c *Consumer) Poll(timeoutMs int) tmq.Event + + // 出于兼容目的保留 tmq.TopicPartition 参数,当前未使用 + func (c *Consumer) Commit() ([]tmq.TopicPartition, error) + + func (c *Consumer) Unsubscribe() error + + func (c *Consumer) Close() error + ``` + + + + + + ```rust + impl TBuilder for TmqBuilder + fn from_dsn(dsn: D) -> Result + fn build(&self) -> Result + + impl AsAsyncConsumer for Consumer + async fn subscribe, I: IntoIterator + Send>( + &mut self, + topics: I, + ) -> Result<(), Self::Error>; + fn stream( + &self, + ) -> Pin< + Box< + dyn '_ + + Send + + futures::Stream< + Item = Result<(Self::Offset, MessageSet), Self::Error>, + >, + >, + >; + async fn commit(&self, offset: Self::Offset) -> Result<(), Self::Error>; + + async fn unsubscribe(self); + ``` + + 可在 上查看详细 API 说明。 + + + + + + ```js + function TMQConsumer(config) + + function subscribe(topic) + + function consume(timeout) + + function subscription() + + function unsubscribe() + + function commit(msg) + + function close() + ``` + + + + + + ```csharp + class ConsumerBuilder + + ConsumerBuilder(IEnumerable> config) + + public IConsumer Build() + + void Subscribe(IEnumerable topics) + + void Subscribe(string topic) + + ConsumeResult Consume(int millisecondsTimeout) + + List Subscription() + + void Unsubscribe() + + List Commit() + + void Close() + ``` + + + + + # 数据订阅示例 + ## 写入数据 + + 首先完成建库、建一张超级表和多张子表操作,然后就可以写入数据了,比如: + + ```sql + DROP DATABASE IF EXISTS tmqdb; + CREATE DATABASE tmqdb WAL_RETENTION_PERIOD 3600; + CREATE TABLE tmqdb.stb (ts TIMESTAMP, c1 INT, c2 FLOAT, c3 VARCHAR(16)) TAGS(t1 INT, t3 VARCHAR(16)); + CREATE TABLE tmqdb.ctb0 USING tmqdb.stb TAGS(0, "subtable0"); + CREATE TABLE tmqdb.ctb1 USING tmqdb.stb TAGS(1, "subtable1"); + INSERT INTO tmqdb.ctb0 VALUES(now, 0, 0, 'a0')(now+1s, 0, 0, 'a00'); +INSERT INTO tmqdb.ctb1 VALUES(now, 1, 1, 'a1')(now+1s, 11, 11, 'a11'); ``` +## 创建 topic - - + 使用 SQL 创建一个 topic: -对于 Java 程序,还可以使用如下配置项: + ```sql + CREATE TOPIC topic_name AS SELECT ts, c1, c2, c3 FROM tmqdb.stb WHERE c1 > 1; + ``` -| 参数名称 | 类型 | 参数说明 | -| ----------------------------- | ------ | ----------------------------------------------------------------------------------------------------------------------------- | -| `td.connect.type` | string | 连接类型,"jni" 指原生连接,"ws" 指 websocket 连接,默认值为 "jni" | -| `bootstrap.servers` | string | 连接地址,如 `localhost:6030` | -| `value.deserializer` | string | 值解析方法,使用此方法应实现 `com.taosdata.jdbc.tmq.Deserializer` 接口或继承 `com.taosdata.jdbc.tmq.ReferenceDeserializer` 类 | -| `value.deserializer.encoding` | string | 指定字符串解析的字符集 | | + ## 创建消费者 *consumer* -需要注意:此处使用 `bootstrap.servers` 替代 `td.connect.ip` 和 `td.connect.port`,以提供与 Kafka 一致的接口。 + 对于不同编程语言,其设置方式如下: -```java -Properties properties = new Properties(); -properties.setProperty("enable.auto.commit", "true"); -properties.setProperty("auto.commit.interval.ms", "1000"); -properties.setProperty("group.id", "cgrpName"); -properties.setProperty("bootstrap.servers", "127.0.0.1:6030"); -properties.setProperty("td.connect.user", "root"); -properties.setProperty("td.connect.pass", "taosdata"); -properties.setProperty("auto.offset.reset", "latest"); -properties.setProperty("msg.with.table.name", "true"); -properties.setProperty("value.deserializer", "com.taos.example.MetersDeserializer"); + + -TaosConsumer consumer = new TaosConsumer<>(properties); + ```c + /* 根据需要,设置消费组 (group.id)、自动提交 (enable.auto.commit)、 + 自动提交时间间隔 (auto.commit.interval.ms)、用户名 (td.connect.user)、密码 (td.connect.pass) 等参数 */ + tmq_conf_t* conf = tmq_conf_new(); + tmq_conf_set(conf, "enable.auto.commit", "true"); + tmq_conf_set(conf, "auto.commit.interval.ms", "1000"); + tmq_conf_set(conf, "group.id", "cgrpName"); + tmq_conf_set(conf, "td.connect.user", "root"); + tmq_conf_set(conf, "td.connect.pass", "taosdata"); + tmq_conf_set(conf, "auto.offset.reset", "latest"); + tmq_conf_set(conf, "msg.with.table.name", "true"); + tmq_conf_set_auto_commit_cb(conf, tmq_commit_cb_print, NULL); -/* value deserializer definition. */ -import com.taosdata.jdbc.tmq.ReferenceDeserializer; + tmq_t* tmq = tmq_consumer_new(conf, NULL, 0); + tmq_conf_destroy(conf); + ``` -public class MetersDeserializer extends ReferenceDeserializer { -} -``` + + - + 对于 Java 程序,还可以使用如下配置项: - + | 参数名称 | 类型 | 参数说明 | + | ----------------------------- | ------ | ----------------------------------------------------------------------------------------------------------------------------- | + | `td.connect.type` | string | 连接类型,"jni" 指原生连接,"ws" 指 websocket 连接,默认值为 "jni" | + | `bootstrap.servers` | string | 连接地址,如 `localhost:6030` | + | `value.deserializer` | string | 值解析方法,使用此方法应实现 `com.taosdata.jdbc.tmq.Deserializer` 接口或继承 `com.taosdata.jdbc.tmq.ReferenceDeserializer` 类 | + | `value.deserializer.encoding` | string | 指定字符串解析的字符集 | | -```go -conf := &tmq.ConfigMap{ - "group.id": "test", - "auto.offset.reset": "latest", - "td.connect.ip": "127.0.0.1", - "td.connect.user": "root", - "td.connect.pass": "taosdata", - "td.connect.port": "6030", - "client.id": "test_tmq_c", - "enable.auto.commit": "false", - "msg.with.table.name": "true", -} -consumer, err := NewConsumer(conf) -``` + 需要注意:此处使用 `bootstrap.servers` 替代 `td.connect.ip` 和 `td.connect.port`,以提供与 Kafka 一致的接口。 - + ```java + Properties properties = new Properties(); + properties.setProperty("enable.auto.commit", "true"); + properties.setProperty("auto.commit.interval.ms", "1000"); + properties.setProperty("group.id", "cgrpName"); + properties.setProperty("bootstrap.servers", "127.0.0.1:6030"); + properties.setProperty("td.connect.user", "root"); + properties.setProperty("td.connect.pass", "taosdata"); + properties.setProperty("auto.offset.reset", "latest"); + properties.setProperty("msg.with.table.name", "true"); + properties.setProperty("value.deserializer", "com.taos.example.MetersDeserializer"); - + TaosConsumer consumer = new TaosConsumer<>(properties); -```rust -let mut dsn: Dsn = "taos://".parse()?; -dsn.set("group.id", "group1"); -dsn.set("client.id", "test"); -dsn.set("auto.offset.reset", "latest"); + /* value deserializer definition. */ + import com.taosdata.jdbc.tmq.ReferenceDeserializer; -let tmq = TmqBuilder::from_dsn(dsn)?; + public class MetersDeserializer extends ReferenceDeserializer { + } + ``` -let mut consumer = tmq.build()?; -``` + - + - + ```go + conf := &tmq.ConfigMap{ + "group.id": "test", + "auto.offset.reset": "latest", + "td.connect.ip": "127.0.0.1", + "td.connect.user": "root", + "td.connect.pass": "taosdata", + "td.connect.port": "6030", + "client.id": "test_tmq_c", + "enable.auto.commit": "false", + "msg.with.table.name": "true", + } + consumer, err := NewConsumer(conf) + ``` -Python 语言下引入 `taos` 库的 `Consumer` 类,创建一个 Consumer 示例: + -```python -from taos.tmq import Consumer + -# Syntax: `consumer = Consumer(configs)` -# -# Example: -consumer = Consumer( - { - "group.id": "local", - "client.id": "1", - "enable.auto.commit": "true", - "auto.commit.interval.ms": "1000", - "td.connect.ip": "127.0.0.1", - "td.connect.user": "root", - "td.connect.pass": "taosdata", - "auto.offset.reset": "latest", - "msg.with.table.name": "true", - } -) -``` + ```rust + let mut dsn: Dsn = "taos://".parse()?; + dsn.set("group.id", "group1"); + dsn.set("client.id", "test"); + dsn.set("auto.offset.reset", "latest"); - + let tmq = TmqBuilder::from_dsn(dsn)?; - + let mut consumer = tmq.build()?; + ``` -```js -// 根据需要,设置消费组 (group.id)、自动提交 (enable.auto.commit)、 -// 自动提交时间间隔 (auto.commit.interval.ms)、用户名 (td.connect.user)、密码 (td.connect.pass) 等参数 + -let consumer = taos.consumer({ - 'enable.auto.commit': 'true', + + + Python 语言下引入 `taos` 库的 `Consumer` 类,创建一个 Consumer 示例: + + ```python + from taos.tmq import Consumer + + # Syntax: `consumer = Consumer(configs)` + # + # Example: + consumer = Consumer( + { + "group.id": "local", + "client.id": "1", + "enable.auto.commit": "true", + "auto.commit.interval.ms": "1000", + "td.connect.ip": "127.0.0.1", + "td.connect.user": "root", + "td.connect.pass": "taosdata", + "auto.offset.reset": "latest", + "msg.with.table.name": "true", + } + ) + ``` + + + + + + ```js + // 根据需要,设置消费组 (group.id)、自动提交 (enable.auto.commit)、 + // 自动提交时间间隔 (auto.commit.interval.ms)、用户名 (td.connect.user)、密码 (td.connect.pass) 等参数 + + let consumer = taos.consumer({ +'enable.auto.commit': 'true', 'auto.commit.interval.ms','1000', - 'group.id': 'tg2', + 'group.id': 'tg2', 'td.connect.user': 'root', 'td.connect.pass': 'taosdata', 'auto.offset.reset','latest', 'msg.with.table.name': 'true', 'td.connect.ip','127.0.0.1', - 'td.connect.port','6030' - }); -``` + 'td.connect.port','6030' + }); + ``` -```csharp -var cfg = new Dictionary() -{ - { "group.id", "group1" }, - { "auto.offset.reset", "latest" }, - { "td.connect.ip", "127.0.0.1" }, - { "td.connect.user", "root" }, - { "td.connect.pass", "taosdata" }, - { "td.connect.port", "6030" }, - { "client.id", "tmq_example" }, - { "enable.auto.commit", "true" }, - { "msg.with.table.name", "false" }, -}; -var consumer = new ConsumerBuilder>(cfg).Build(); -``` + ```csharp + var cfg = new Dictionary() + { + { "group.id", "group1" }, + { "auto.offset.reset", "latest" }, + { "td.connect.ip", "127.0.0.1" }, + { "td.connect.user", "root" }, + { "td.connect.pass", "taosdata" }, + { "td.connect.port", "6030" }, + { "client.id", "tmq_example" }, + { "enable.auto.commit", "true" }, + { "msg.with.table.name", "false" }, + }; + var consumer = new ConsumerBuilder>(cfg).Build(); + ``` - + - + -上述配置中包括 consumer group ID,如果多个 consumer 指定的 consumer group ID 一样,则自动形成一个 consumer group,共享消费进度。 + 上述配置中包括 consumer group ID,如果多个 consumer 指定的 consumer group ID 一样,则自动形成一个 consumer group,共享消费进度。 -数据回放功能说明: -- 订阅增加 replay 功能,按照数据写入的时间回放。 - 比如,如下时间写入三条数据 - ```sql - 2023/09/22 00:00:00.000 - 2023/09/22 00:00:05.000 - 2023/09/22 00:00:08.000 - ``` - 则订阅出第一条数据 5s 后返回第二条数据,获取第二条数据 3s 后返回第三条数据。 -- 仅列订阅支持数据回放 - - 回放需要保证独立时间线 - - 如果是子表订阅或者普通表订阅,只有一个vnode上有数据,保证是一个时间线 - - 如果超级表订阅,则需保证该 DB 只有一个vnode,否则报错(因为多个vnode上订阅出的数据不在一个时间线上) -- 超级表和库订阅不支持回放 -- 增加 enable.replay 参数,true表示开启订阅回放功能,false表示不开启订阅回放功能,默认不开启。 -- 回放不支持进度保存,所以回放参数 enable.replay = true 时,auto commit 自动关闭 -- 因为数据回放本身需要处理时间,所以回放的精度存在几十ms的误差 + ## 订阅 *topics* -## 订阅 *topics* + 一个 consumer 支持同时订阅多个 topic。 -一个 consumer 支持同时订阅多个 topic。 - - - + + ```c -// 创建订阅 topics 列表 -tmq_list_t* topicList = tmq_list_new(); -tmq_list_append(topicList, "topicName"); -// 启动订阅 -tmq_subscribe(tmq, topicList); -tmq_list_destroy(topicList); - -``` + // 创建订阅 topics 列表 + tmq_list_t* topicList = tmq_list_new(); + tmq_list_append(topicList, "topicName"); + // 启动订阅 + tmq_subscribe(tmq, topicList); + tmq_list_destroy(topicList); + + ``` -```java -List topics = new ArrayList<>(); -topics.add("tmq_topic"); -consumer.subscribe(topics); -``` + ```java + List topics = new ArrayList<>(); + topics.add("tmq_topic"); + consumer.subscribe(topics); + ``` -```go -err = consumer.Subscribe("example_tmq_topic", nil) -if err != nil { - panic(err) -} -``` + ```go + err = consumer.Subscribe("example_tmq_topic", nil) + if err != nil { + panic(err) + } + ``` - - + + ```rust -consumer.subscribe(["tmq_meters"]).await?; -``` + consumer.subscribe(["tmq_meters"]).await?; + ``` - + - + ```python -consumer.subscribe(['topic1', 'topic2']) -``` + consumer.subscribe(['topic1', 'topic2']) + ``` - + - + ```js -// 创建订阅 topics 列表 -let topics = ['topic_test'] + // 创建订阅 topics 列表 + let topics = ['topic_test'] -// 启动订阅 -consumer.subscribe(topics); -``` + // 启动订阅 + consumer.subscribe(topics); + ``` - + - + ```csharp -// 创建订阅 topics 列表 -List topics = new List(); -topics.add("tmq_topic"); -// 启动订阅 -consumer.Subscribe(topics); -``` + // 创建订阅 topics 列表 + List topics = new List(); + topics.add("tmq_topic"); + // 启动订阅 + consumer.Subscribe(topics); + ``` - + - + -## 消费 + ## 消费 -以下代码展示了不同语言下如何对 TMQ 消息进行消费。 + 以下代码展示了不同语言下如何对 TMQ 消息进行消费。 - - + + ```c -// 消费数据 -while (running) { - TAOS_RES* msg = tmq_consumer_poll(tmq, timeOut); - msg_process(msg); -} -``` + // 消费数据 + while (running) { + TAOS_RES* msg = tmq_consumer_poll(tmq, timeOut); + msg_process(msg); + } + ``` 这里是一个 **while** 循环,每调用一次 tmq_consumer_poll(),获取一个消息,该消息与普通查询返回的结果集完全相同,可以使用相同的解析 API 完成消息内容的解析。 -```java -while(running){ - ConsumerRecords meters = consumer.poll(Duration.ofMillis(100)); - for (Meters meter : meters) { - processMsg(meter); - } -} -``` + ```java + while(running){ + ConsumerRecords meters = consumer.poll(Duration.ofMillis(100)); + for (Meters meter : meters) { + processMsg(meter); + } + } + ``` -```go -for { - ev := consumer.Poll(0) - if ev != nil { - switch e := ev.(type) { - case *tmqcommon.DataMessage: - fmt.Println(e.Value()) - case tmqcommon.Error: - fmt.Fprintf(os.Stderr, "%% Error: %v: %v\n", e.Code(), e) - panic(e) - } - consumer.Commit() - } -} -``` + ```go + for { + ev := consumer.Poll(0) + if ev != nil { + switch e := ev.(type) { + case *tmqcommon.DataMessage: + fmt.Println(e.Value()) + case tmqcommon.Error: + fmt.Fprintf(os.Stderr, "%% Error: %v: %v\n", e.Code(), e) + panic(e) + } + consumer.Commit() + } + } + ``` -```rust -{ - let mut stream = consumer.stream(); + ```rust + { + let mut stream = consumer.stream(); - while let Some((offset, message)) = stream.try_next().await? { - // get information from offset + while let Some((offset, message)) = stream.try_next().await? { + // get information from offset - // the topic - let topic = offset.topic(); - // the vgroup id, like partition id in kafka. - let vgroup_id = offset.vgroup_id(); - println!("* in vgroup id {vgroup_id} of topic {topic}\n"); + // the topic + let topic = offset.topic(); + // the vgroup id, like partition id in kafka. + let vgroup_id = offset.vgroup_id(); + println!("* in vgroup id {vgroup_id} of topic {topic}\n"); - if let Some(data) = message.into_data() { + if let Some(data) = message.into_data() { while let Some(block) = data.fetch_raw_block().await? { - // one block for one table, get table name if needed - let name = block.table_name(); - let records: Vec = block.deserialize().try_collect()?; - println!( - "** table: {}, got {} records: {:#?}\n", - name.unwrap(), - records.len(), - records - ); - } + // one block for one table, get table name if needed + let name = block.table_name(); + let records: Vec = block.deserialize().try_collect()?; + println!( + "** table: {}, got {} records: {:#?}\n", + name.unwrap(), + records.len(), + records + ); } - consumer.commit(offset).await?; - } -} -``` + } + consumer.commit(offset).await?; + } + } + ``` -```python -while True: - res = consumer.poll(100) - if not res: + ```python + while True: + res = consumer.poll(100) + if not res: continue - err = res.error() - if err is not None: + err = res.error() + if err is not None: raise err - val = res.value() + val = res.value() - for block in val: + for block in val: print(block.fetchall()) -``` + ``` -```js -while(true){ - msg = consumer.consume(200); - // process message(consumeResult) - console.log(msg.topicPartition); - console.log(msg.block); - console.log(msg.fields) -} -``` + ```js + while(true){ + msg = consumer.consume(200); + // process message(consumeResult) + console.log(msg.topicPartition); + console.log(msg.block); + console.log(msg.fields) + } + ``` - + - + ```csharp -// 消费数据 -while (true) -{ - using (var result = consumer.Consume(500)) - { - if (result == null) continue; - ProcessMsg(result); - consumer.Commit(); - } -} -``` + // 消费数据 + while (true) + { + using (var result = consumer.Consume(500)) + { + if (result == null) continue; + ProcessMsg(result); + consumer.Commit(); + } + } + ``` - + - + -## 结束消费 + ## 结束消费 -消费结束后,应当取消订阅。 + 消费结束后,应当取消订阅。 - - + + ```c -/* 取消订阅 */ -tmq_unsubscribe(tmq); + /* 取消订阅 */ + tmq_unsubscribe(tmq); -/* 关闭消费者对象 */ -tmq_consumer_close(tmq); -``` + /* 关闭消费者对象 */ + tmq_consumer_close(tmq); + ``` - - + + ```java -/* 取消订阅 */ -consumer.unsubscribe(); + /* 取消订阅 */ + consumer.unsubscribe(); -/* 关闭消费 */ -consumer.close(); -``` + /* 关闭消费 */ + consumer.close(); + ``` - + - + ```go -/* Unsubscribe */ -_ = consumer.Unsubscribe() + /* Unsubscribe */ + _ = consumer.Unsubscribe() -/* Close consumer */ -_ = consumer.Close() -``` + /* Close consumer */ + _ = consumer.Close() + ``` - + - + ```rust -consumer.unsubscribe().await; -``` + consumer.unsubscribe().await; + ``` - + - + ```py -# 取消订阅 -consumer.unsubscribe() -# 关闭消费 -consumer.close() -``` + # 取消订阅 + consumer.unsubscribe() + # 关闭消费 + consumer.close() + ``` - - + + ```js -consumer.unsubscribe(); -consumer.close(); -``` + consumer.unsubscribe(); + consumer.close(); + ``` - + - + ```csharp -// 取消订阅 -consumer.Unsubscribe(); + // 取消订阅 + consumer.Unsubscribe(); -// 关闭消费 -consumer.Close(); -``` + // 关闭消费 + consumer.Close(); + ``` -## 删除 *topic* - -如果不再需要订阅数据,可以删除 topic,需要注意:只有当前未在订阅中的 TOPIC 才能被删除。 - -```sql -/* 删除 topic */ -DROP TOPIC topic_name; -``` - -## 状态查看 - -1、*topics*:查询已经创建的 topic - -```sql -SHOW TOPICS; -``` - -2、consumers:查询 consumer 的状态及其订阅的 topic - -```sql -SHOW CONSUMERS; -``` - -3、subscriptions:查询 consumer 与 vgroup 之间的分配关系 - -```sql -SHOW SUBSCRIPTIONS; -``` - -## 示例代码 +## 完整示例代码 以下是各语言的完整示例代码。 @@ -908,3 +835,22 @@ SHOW SUBSCRIPTIONS; + +#订阅高级功能 +##数据回放 +- 订阅支持 replay 功能,按照数据写入的时间回放。 + 比如,如下时间写入三条数据 + ```sql + 2023/09/22 00:00:00.000 + 2023/09/22 00:00:05.000 + 2023/09/22 00:00:08.000 + ``` + 则订阅出第一条数据 5s 后返回第二条数据,获取第二条数据 3s 后返回第三条数据。 +- 仅查询订阅支持数据回放 + - 回放需要保证独立时间线 + - 如果是子表订阅或者普通表订阅,只有一个vnode上有数据,保证是一个时间线 + - 如果超级表订阅,则需保证该 DB 只有一个vnode,否则报错(因为多个vnode上订阅出的数据不在一个时间线上) +- 超级表和库订阅不支持回放 +- enable.replay 参数,true表示开启订阅回放功能,false表示不开启订阅回放功能,默认不开启。 +- 回放不支持进度保存,所以回放参数 enable.replay = true 时,auto commit 自动关闭 +- 因为数据回放本身需要处理时间,所以回放的精度存在几十ms的误差 \ No newline at end of file diff --git a/docs/zh/07-develop/img_5.png b/docs/zh/07-develop/img_5.png new file mode 100644 index 0000000000000000000000000000000000000000..d9306cdb74f9876595909e3e7fde28a248561abb GIT binary patch literal 18871 zcmeEucRbZ!{J(^*eQ~YqeeIbMl50dpX7+ZCB70?pYl~!NU!)>ilw^-=Qf3I*WMy9C zcW$5W@`-}6w?{nT~pV#a8e7;Vsj`lrLqMJlGI5?#0YRY;zIJn^dYgP~f z@I?60?*tqi^$>MsMFW4!?cAvySl97!eoJSd(3kxvVoG66&4d;@JGJxs38Hi{s!Nm? zpYDGDi2k^P_^BJSNa>+un}DJGiBD?|RZ)cEQ~x~cEz4j1+pse7LtkFQZ+HNcmJ@Vm zY<5h;$1m3}_qO3ds2Cdxiim++^$0@|5H^$uCk&(^6e&|za=|0_lIP6?0Xo! zG=105(GLCTg8fI^GpkY*2uQLzToXU^>cY==SpL}lf&8ICz2A;yi>dVG*=D6VE2HOE zwn(OslZC0{`OqCHYDCPddk7n%ll^|-z?`{Ht9jY3KR=GTNlM;1`4zEMoTA-V?>e($ zo1XDh54q=750^JJ}K(tJ?y z&*0s@}OqI zDpWpjuZy;pltI#dSm7$@=!cQjX=Oq}=;djTFeO4$l#)mvZ)mByhq*u30!{~of;NQC zsfd#OY22(hm)6J-IxM^0+n2_FaG>&w?&=cbCd6ob$M4TiZJz~qt9ei%a}rC>Ual$K zdrUvPLhb6E$GBU{&?YS*!Oc7T6(8Fx!AE$(MFV7yg zoQ{}>UL2lYQovVpytI9Pdh{&!u$%0HMjT}~kR_7~!=9z@zk8FHRT+4`*AqB?Yw&@> zMJjPsQPQ0=)@#3>2NCn{jLd~z1-HzgKkfJN$1hcwe(0;DL1?mpkH+{Yo?l{iP94vF zdvFTa&1SnbuQ%%^G0FPcJLY((3PFPOteqG)(b3HNR&Kf3Pa1g@6og^F(u^#7C%){q z6NeZU<+!&K-bhsQ$kol(xNa{;TlH)3%~M%@Q*P5fS1FE?vgSvZ$uP9nlT zS;=-gFYNwwvKW8hCAZf}bzna#BK++9`gk(RB-;3sKb)sgkboW^(cBqF&A;z-{get3 z(~F&BqCHwK^LldRX7NxB@@Dley+SSL`%Q;W$4wlO6Bd56iWi4qD%c1SB`RE=$eFwh z@j6nUY-S$!L`76h~-iCms=Tc9{nF6<} z1O5=iJ_GrSX6%AvRL$Mn@TUq@;%g3uWQ9NPG-J-Wd)?)$y%w20E|tB{lJ$hxVqR2z zju1+iunZh@^Mxwk)K<9on`dXuX+*@v%>br$C*OL0=;c&ppUigM*#4pP<_n+AaVCRN zJCfjpZQ_AzV`t@yVB!wuX3r~7ybybn)(2K1`9^<_XjIW%_g{7H7@stEQnxRM=?cnR zbw`2}-xw_Li?rFfZgw_PJTfd!kI!XEhJ{`_#E?c*2;ikF9`}Uf6UVu#6MV4KnR8W1 zM?s4e5jK6T&L)ve*%QA&EBNc%0y?HYJdQ_dqzm*mgOO-TbD}@0dtsNkRCq`q@fI~` zB;q#Lv;~%PzZa8}$F#={($kbWXH2W22`(2>vEo`{ypOVQjMJ|s*O^FS+LcwN+YbcB zvxFl8_+3K19d-VE_TAx;I}kDy`MT$VY%# z2l|budiQe5F@M`=S$np9)L>WI^>F}Yl%zXOf{RE`SU}ICpueN{PM>o7i#dH7vsGbz z_Q5jNfsS(^64NtjWMLV6_~ZODV^PBJ9nAYlh1E35YwBH;9Qi@}y|(nDRE69NKFR%} ze0MHSmV)vbi{cuWDTi9ZMnu1|+0c)2Rq4`_-#M|ql*f?8zC^^VD`BhiP^e?VeV|U@ zmHk(qIum%-rQ@BHpZ8iM*&H6L*lHHrU!{!xK@*4h)RmSiOc#=H3%&o*i_(dqVk}d+ zwKiY>gs@W;&aF4u6yW6%a_;u4Q_Vo)&7k+7^(-!9sIjJ|=>2P9tw~x_AAS|6PuAIa z2o>Q(yG)qf(y-0-WGl3unGUY1h?M>FvB;P}?s`x?B!ZA6Pk1vAiqIqnLx-z-t-Xu$ z$LK2K!?U@yc(Ul`y_R``j!#o=swgFXXO(EBEw!jxpTCdA@hIN3uij$`mYDC3!US|o z{rb4JA`%_(_N^=>dyX+t+foX*%&N${A(Y0{*QDEWn626y7F1DV%6C0nZ3c$C#E|8uswqdr#0@t!JdWiGDrpaKb+|9SFC?H2izz_aqP5AC>6L?BtS(U6 zk3MDU6v#%I6X$$!jcb>iS4g3$|dNNroHYa!>gki zw^g?D+vP^}T^>=rQOrjs53V2whM)5VaxDgW8tLf0tXoZB@rS<&hwcLYXdMrz@CqS z%0MD@6YxMbTI_F(4CCsFF;E2Ee+gVON!<*nt#7gp6!BX`rGe#?6$Qoqgu5cUd*BF; zdImy#tXZRseFMU#4Ez7d|GPIv;gv0yjqj7R_%3g6m?vFn=k5@(b;qL+{U>pvKPo=| zX$;%?xRl7yM<(+dYI1LIzAR)V|9R%Aqp(CM8M3>&^cUTw?3{Euw2+AtX3F_GJhhj5 z)_je-%3LtQiq5ud5ED=05+WzuKwYWm~AggQX2D^wWv?V>XZY8#{muZuB6X&Hsv z5Dys*b!3s7^$2+5Xl2UO-)1|mAINYneI4YNio&%uSA#=F!IBb0EKfY-a-w!;_*K=t zZ7rvj@j>Q-+qqGb3x>)h1$8KPUZrM7i&zSTi*7s&N|5X`Kq$sc-9?~PvV1MsETBYM z%q{CRPc+~XCMBSX0|felFiDrm7ym3wGq*3xMn2&sxkhSZ+FZq?Z^pe%qy%9F>tdt_ zce*1{;m~dIABcq{Y16M4lxy#@+eCVoW-T4fWQcG_oC7qx<(A_7=NB40)WU z10p514dk>~KxNZ)=$Mw~TA+nl@2H8uVXv?2k>}aR8{+eSyf)wWG{h>Rz9(M}$G3J`u8zu{ zo-9PO)PNmL{GUl1j5#{}frv?`i`DT5R&Q;F<E|d(=K6V{+uMk9H}|~ zBCAYV_R~AsvfFiAdp#_}`Acu|p7Nau-Lfd*MX1obp?zfc$a{++5524GPm5~vm^$Xp%)v~V|52a@0JWgFxw4V7svC%dBI0h zTd_QOU(I5WX(3D-j4g4%(&F$_y)Z4)%J5-A@BZ`}MkdK084ruiy&mz1^2Is|ZY>wS z!$&XlN|5D&m#fX+vaW!yQETzpcX|12DM`U$r|CfZpRZyC4%<+_{i}5z*vTcZa!qrL zP6nz%Kl3=a1a?x~gXOmXOGhrtb2Zn`!7oC8MxqO5_r^lsI+R&D78)D|!pdF#c2SCd|D{z>ffT4DVD{fwQl z{hA$J-pH-_7KP8hUurx(dM;?y+~oM#Lw_PbcW(~#0SCn`-@%Jtmr-fKPFT;Y{>g0P z7Fh55R+}0ddtsPDHl+x zXxu0*On`?nauxY)S&l50++zM#scBapIM=(|oKD|!IN2zMz6;v}|yUp-1{T~68Qyx099n64itlfJWpRj*CX-*JNBiNS*%#-s_zF<>j z8!q?ny7!i6+~%9V(2YAqSyxez(8=5wXGi3qHn-$7Q91sLu?y?=z`=6#{ie=CJu{TA z(DL@XC`wEJyN4w>u7=EBx+ML&N%MIw+0>s9L!0}+1S=ulTq~bUY|v_TiD?aCL^DSvn67 zG266_6B?&(#UJf_h(Za{nMO+cT#_?|uKB+$aK1I|STFiUjd&UC5?@z_aV+Q|Zsz&;GH+!pW9{IhKK{zzqq2B( zqs=uHOfyDX%z8pw!z?K=%!MgPMhtMXdyhkZwT2bigzX7o-o`=;160{z5Uzb&{WjA` z$|zc+_r}S>fnZY%%aEc zF}QCan-}?(J@q>V7z&mDEsHD!Wfeg z?TpcnTTzQmB%T!KnESt>{*o>xTQU@gg#2Vee8ka_o>~7hBGxS{u2iQP6q_MCp+M25 zE!d2^B+uPmh}=Wk=n|a?YUN}`@Z7XX+XCHUj&gTS#_c~D@QtB8y64x0b+Nr^u2_qu zsqPB6holpKi_$6hu#O);bvR*VHN9Bb-pt^I6C*qBrI{y)B3gd-k@I?fGGBdCYP$P< zlW4*UPO53A9L!Eji?O!R!#L^tGR(#ex$;HcZ}#X%_3)P$sYtdN?NUGd5wxL0yY~w= z{N91Eqn5-4YWe=UVM-fS=e+6{g6wkq<^sB!MJnPQ&B2-bLn zZP$$P6<w#)P)HK$iNqZvAN(8uaGgqKScL=( zd&*AGfs=2D%4Tp+?eFUx|4_{;YvtL#szbvGr=o0I3d$%<%^k*E`vBIf1BI)x<17?( z$*~w=L+5_kyo?}D8sQ24&^$b8EiZV#k-quj$gJgdbh-$p1k+wqF3M+4De#K;*>1qc z#Dp`ClnqT1Ds7vLfr5k@+!B{^v)G5$M84;IXlp1FqV-i|3Wdk~*>nGAw-MLgzh*Iw zV-?lgDIYAd=7<`o_N=_X;G}m%h}sy(2qukI`s|2CThPoxKI4~GR9g*EObniZWhi?! zAnVdm)N!l}8D-9;Aku1EixK(!Wgyf4U2IOIfurr@Y1-gRqYPc6R_R+BRT{lURZDLm z_~b*Vori~kfg2Ef<#)^NBaa$Baf+-Msj(f-%>^9{p886@g38GI3Mg!_F{*cpZ>{@1 zzxVe@qEh;43O30=q7H%X{q9<<_e9 zBgez;gC%M{+;_dZ_y&!C7nAw$S=V4N7a|n61N(}4L&J#*F*=iV&ia=Yo|1a+IAHCk z3NGX78VS#36r^+{(GO$R2+MW-**IC1TH-rZjdFY&ITM>Pmx{46TwD$MQ z7b7v^Do_#I$cIH-rX{!CWG`6}ZD-S>4x*aqM4c)Vc@IV{S2OKbVR&CpYjicigv_F9!-mgYY)B$By0z(g6 zhnE|g&x*CmB<>|E=)S@27G084Kj*~c?;VmvcA40r`kZeidiwu<3p%;gA-EGtoN)iG zc&N(87vx(Gw#%9t4(Efg8_Zv$NzXdF#Li~NYR2Ue@lUA#pkvwD=}7eJSlswIyce}* zmDrSJ|7(k=`}axs+Nwdf)|hrlg$_<5qx)}U5&4A#dzkRy1qp7Aty$ z-Bp^ZX#0Ni`Vu)iEDHTRQ1-pf+lWSdR3_H7DkgQbmDqkOuf<}8u&INO>cmW;4-jGR zeP32>(XzgYmCT79tt&c`2o55Q9A(EAIpU1IJ70D2UX*G~WQtV5TxtIY>qWQ4b)Ii^ z;-Xo;-}6e6gwZ0Vz{$>Jx-AH6w<`YIy+eHBF_995LU{FL=+FuF=>q3^txgBB`qc3& zP0AQ*Oqhe_>-RhZthwKeYIg!|HF)8@P8Ca4Tc$D3)+2vl>fZE|SSkS3t+NREueK6$!eW?r^*=UMRa@P!i%~j}JxWn!80xGdsxH4@wZ5bQPe@m%Bw7fG z5{pG>+mxX@!6o2enyhdkf|o!nt+;%eys@Kn+V+gD=hh-9l}&_|yr{&s=6k&kO}H_B#-pM_ zXLP$q_x5*0R&#^(PzCRn|B}F%B9lKgTKZS0xS8a}5~1iiXtfA2hcLgwwprAd?_R4z zxoh+{JNkMNS)#=J)g~gL7aerV16DI`Ym5Z9gbSLgpEC_tGGv%coi00E-XyriKT(5O zZqQ=$4OtM?J5D3Q9AN5~q9zPiZ=*6Ff|_EzszntpS~h#E(a`v#%;@-iwnTfM1$VaU z=BoV{yF`UV{5Z%s@G;RON@iP~(^pFi zN|aM`@WL8etL+$xOMyc6ryn-uOztoD*Ir~QMO?B|O|Y$!THZkjU{Bxu|F(Ev06owSnqXjr-FP??fg&}#8YsX@u& zjc^-l2@@7^g3u`oJj08~e2D$AUG%Ln4k+9eAR27Sgy;^I&QT4K7-xVRK-kzx_-XEj zD{9^b&;&$RacwPH5v2IAsnHvZXK!;FK`|28VhZuW*NegCfr1LMx3M-R0&oK14GJDL z{P#15u@DL`VvX`&R#8qm<%|~q1B*%2@qya@#HKn?tZl5*cd@nL zPRO)|+A{w46cnEI-&00VYi(@dw)a6FF8zC0LINJ%|M!rndK5}YVr%>_5f>W^dN4T- zTdM&a(Y9T{h4iPRo*(tztq-jbqik7e6-(VI&Q2;JSV7;|so@Fq=LtwRT}&_Y4~j6N_Im`n8FjKP0LPdlD2MADua<{}X2)6ER9FUuLjw8L%A zGVh~B9EWnw*({p;t3jF5?x}3G+_3h|H0(S6)vRqIr<%DsC+lzWea(ixcQ{eD)(SeYTmxgsB&n$!Xi8P>f$)W$F9!eli%_QB~gdn zgyfJNM3jh=jynl_1<(No>n{MQ*KaGI99>q~e4CcG%3(0eb{3FUWfVL5#lN0*s;L9{CG|+b39E6fchF{}Y#Sz7cNpa83jgEam^*~NkE_Vwwe?a^8 zda>;cpns45ZnC>%mGCDvZZAo3{Tt{tNv%1SQD#el-HJ8&b>{2D_{M-h4)Gekz4<!A3%Q%tP`+Kx2^&niZyUK%_#2B z#t#+ttLhN&ovzg^IbSH^NVl-O``Tx^RlkjQZ)n)BaTJb9Sa^JQWeI(W!cFFFY}=Z; zuZUQ%hKCBk2++3jlhRE^zKRO}t4tU|6E_yqTh$sLPhRg9yJ<#KJV_3b4^b%m8uPN;iAb zJKToyq^BghdYvyVat%vYL_HC$(_yuDlNxKkl->eNI&5Y{(0WFvjX{~xSaqUnWiUJ5 z^<@5Jv$ADBuA>kJgcSUpu`>RJ6yww$VD8XbF#XoS`M+HC+@xlwcwNFsJ|4p^I10C? z`Jel0Eq8%?1u1jfaFbq2^caGleueh++DlUAazC< zIk{I_@HtVaS#c*gMU62=vFgv;-GJCHO14b^yj6Q7(c`MxZW}|I(zH=p_B368s`_W6 z&{uv4zopfuDwf0^2$2~LaK7nZz7WCGd~Hh?G>a?luNW*7d8>0*Zq{p9UVcKG-ZEf0 z?Z)%JZ^LOlUMw-77PL?{sWL?m*EZCF*YV4krd*6eCN1Vg@Sjx)eH8Af(9E0VUOLXX z>0{nNyj^<799@AWnOeB1TAI$=+sw;qxFQyhm>Y*q$N~@+otxd2~^~P7zLavqpraQ;*^+WZ5jVM+QjXB;yS9|Kex) z=YeayEyllJfpWb^dyhWHj&9p2e0SkNxdfO)GaPT5C~o$=D-lW1`DHos!+PGa(FISF zNsr^^Pb1>f1D6p)q?fyZMPuFMy{QB0Zx7(RBvFM@-aoM_g6f z2mLsn)(!myZFZbT_YcEmeaAWrZ*bpibWgMud$qELJL+@u&b#ODOXy>D=m;1S-3Y(& z&3y85y!=ecO(yrA2c{|20funXXjOy)hUpHAEN=t5m-vG!^M)rDz88lR$M5YzVoK_8mw=ECn76exg%_qgE9S*bH+}#i|CHB)_D~&Ei%UOR}HvHRtO^ zEgYu1YIO4cv|9T}(I`-q)9)$rUhb>O?xfGzSM@D_EE}(kOF?`^Hp$2x!yV5`>Lnbl zvUpdbmytxn=dB0j2BXx>Tsi3MOz#rbL_d7~#m;?oXcyi__aT zRV?fsP~m%keORU51_P1N@RdG!Z~AfNa>uP~2PpdP#B~ea94|<@_U5J~7UXTDiy*?! z)+D}nFNX0(&uOa62R=O%?kr(FDFQyA&f1nXN|PQTmY{=^BZU|}a!kw?;=Pj+K@qV= zJu`B#!b?S@W(Y%Gh3|N>6H6DC@4!;TS(` zfnpSsy=QgL?D~td=VxiFIo+&2dBYZuA^@gf5gy&foMXeE5A;PXleFbCFzJ$D9_sc# zM@$MoB)gUVe12(*ny`wDycl$vod<@uiZ<$b44R}AQrS@2wB4wgFm=voZbrh;tnt0i zZlZTytbIweIOB@K#=AVZA))4G~H|C!*e1lnf@p#;h1R=;kVNg!b7D4 zLka@J6^cdAzFxDK{mrdwfr4#+PxT%O=?_WS}0hl)Ey6Ax+ISu zCXZ{w40|EO*bi9{&%Ux)BoEO3-n7r2>0u}gGeNDF6vThnI7!$o(zd^op_MJkR43R` z-W$vsEwHs>-sJBw?8}Sw=8dnB>VJwBe~`{K)&u_5bufWa-E_4_*p5BlJHZDklY`92)s&~MOYMxc z3ir%}7X3XW-Un2FM+MmYh&pvls-c51{OI~TCH!|dNX}PNb?x+^bq&$F$DkZcfj>al zq2q_CMtD~06>PSzO0#JT^ayE6N-gRdHP_}R^5$h6-(2J} z{-r;GAi+zSK>>2yB6dh*a<0;SnS|_JbAQGPB`wd~ZDFL`E*urUthw9%mqP`v(_R0Z z^fT~B9pjO|*-^+oM_-s*5VZj!AgM2a=Es8zwL=xGIAp$mqy z66Y&zJITw2ZQ7(*yz`x_OSemf-d-CioGU6zN|<4gpiQze$H)asaHVLd((BQi)NmUMH8+ zfp6~m-6?(RS@3RTyY1Mu@ekM{5O2F(oALPYK1(*@Wp+eta-hG}bNx1^(ZZQ25hw(k z0^MJw8m0xM;ez8DUI97RRi_eo1RcslnF7&%{8BBSOG2OAxS@1VdI0lv7qjb^WD9k+ zX1MBzmQ-YI=O8%LI5AWCEvdVMnoYgq)>}wVXF(nlpQ>7X576KnMEH+&pDrzMB`4j; zBr)-Nbn$(?V2JB{PuQ()3p4n_DFFe;5hjjnWZ^}G(L}gXm4=+*UHD92*^3&zJ1n*9 zPBe-l9qT;DGujH&%c*d~&(doD6o0fgJS?5Zkn?U^;atT&L5`D>>@>eIop7f$$A9g; z{P;(XYoeXpy2R1eV-WVI8Yns_txH6QaDODLtuDbR2}8K+YJ&A{Z;~N)ty!RP;4?ag z`z`rTZ;rNlGRKsrUlbs&`F|A{%h)-%d$9lg)TT(65dV8<%lOC9%BEPFNlEm63FE1v zn3-cYr^Us}ueY}yw)~MJg$0MtaNZxh^h-0I!0YepyFVbgl*o=UE~czqUm9%ZY~S^J z{$9g(FzZ%Z-qY%QD%Kh(BG(mD?>AFO5`$~~ojU(?>UzOU3I+9mnEAu2)0{VeAFp*h z3k^A9P17dAhrPN`{wpCKqay;&u2O1^vJi6%a%RR!*2KpK2s9b*@?d;)n&L@*1#Oh@ z)bQ)$qIuD>L@EAMA1#QDzoJYcJiKSv4XS;ZBq|}=86{)HrSMI#O>#9a@X7GL``>iZ z>kS0xAEc5muQ|?>&#+j~Z}Gc`>`x`CP$G$PsfbFl&LRijgxR3l;dE&_bg8?`ZgG-* zpP$=l^NmO9r4O??Bykr(m@$Ghrl-9daj6SEh=SB z-4m%_QuTv3xLyfnsM<^4B%|nPrc0zJDyUibiS)h?N2Q#q!4rTd?BH~EGcuDP2=y)& zj)0VVI*8dr?1>Um3(#iN>-c!l`4VJm4;!A4vS0h1eQM5`?RJf0=NUR(`Pce4$Y-^$ ziob$v33nKyV}JK`s?q~Ip3^fjSb>sg(7MO%I@ZVU@e`N0Xs#!0FL+dQn-Rn0_tQh? z6xFRWsPmK6Jo75bW=>i4Yw%hI`SN;#&aS@ceYivfQwvajprBDZ4z1tu4arC!Hwvc< zn7rgsec5u?J}h>sThS4PTTuKSqliG$PkY+}&JEC~{h!w#0~|7Q8Q#u0!V5^f>{(u_ zH7xY9O&H0Ik=?%86FlItV-=%Y1aG$=kwLePK$+9=iD^y@^`lYkE`*UDbB_EPhBdZ& z@OXBKQ_DE-+dQzZe}9L?Uf>K%P!L7dI~|<3CNyR~L|adAp;;TX1>!_1{FF!@tfImP zi+(v)WT|q(IAN+lSdxj?bl2(30v0hUgSJ*!?|)#YE$fP!lXqoqAw^pw=j@j`(X0-+ zyB1RVFdNUc9G$ATrXjiSDfem5(z1QO12t}u5<<~Cx#me!$-Dw^3aN6P2XK=IE3vy%$&M2IMv|DFE4K>4JxFaR8p z-i=)Ey6+V6>sWLx1Vu-X=PC?^l0IdAbK@US^g$P9mW z&uAf;c zRsavjqCnu4g;YX36^hQc|Ij@`6)K)s?*IPf7#}QPOZp!a4-ajxGAi*e!?pj)TvEj% zeYX<}A%$Al#Zy=Wk7I3ohOM_Z6g0=^Unyi1@$71>G4Dz6Q_glIkPnnjz18u42wEkn0AF=#IjZUxU~z-F=-i4Ex>Y6 zY#7vwO>y2F_E!)%)8Rst^agY{(t}R20KgT8F;WpfnX>!&t4eqKTF1**YNA&OhV;XZ zP77W=XCBb{k8i+{{}6fn#I)i80HfGoz==$6v6l7S+AZUbD&bcw`(0jBLsUA@R75w9 zxN$4Ni{U%sjik`Q@@l_C7eVCGty6)iy>4DuyPbrO#ZveId47lUCn3=g=Dy6ZZ{>QxN% zYrQpj_~>6ik)~*dU~3BX$)^P~pbnb{NIHcr+ktBS8^}XYo)D#iuxT2|(xEeYRnG<6 z%Jk{~MvU@Ee-2&ZHi2(Vf8Y-U(HJ%V>YS3T z(hh)7013dmB&(c`eIPuuH*0uOE+`=c!YURce}bS3tK1zo0G|dlbOP@z?E>_DNXj^X zXpca@_%I;a7|r#``YI}W1Ers|Lsaj}c;*LVSBI^s8VwO>r03->OTh2s5W_@5LR34@ zmmjQM|J4LS7uJGiu@MCzsHEY9Y;iF`eCX@Sz$^VG^JwO$PQE`3jgT`9Zo>8A0LS{W z1(50CuD0&y3NEs7zhR%h|=_pI8iebr+Ls6M2(I= z7ShUSZ8-C2s{aGTWJ6fh&(#Ci0)X7M0>UhxuD?E)1)iYWuxBS??K=etOVF;OM*@Re zQ?+B@)~w&WAYLE93r$>zE596>4mZE^)_lRL-H1XVYJ1#j-YfBl!JFKUm}db;JrzNI zCv(=bUvMg3xY>W?GsJ?1h?7O(Xwojk<63}fy@GhuT`BdAL$K~0BJI&Q_ZU5tFU5im ze;CR4ms6Sn%KGxv+w1wO!}(TCce!z6AkSh66VxIM34nyou9?N`7$8I|7B`MweeDKu z6E#?_J@8l94WiZ1F)gX7_20N$Fnkcq@in8;{8No|hLEThA}od?nPuNl4}c5jKq@k| z@uTcub+vxxNq-24|7lU@HAH^#+nEc>_cmm^BN*<3#TOS3f!uP>>RmCwf2Me3j)6O| zW5S%cwdcD9!fNV*OLgF0(*VNNI9e-8S`iFr#68vg6at)((?G1`g5`Uc{Bed~R@A)|)Nxtqc@RtyS5Z^^1FHSQgk^RM%z&ml>(=9aq(R}su-(uT zN^G1;SI`ZfTyMQ*(lq>hk8|WFVjvF+Y0jfbH4HMCw3&!>jTrr6W9~hw(&fpQ0|vF3 zr^cg~D+4y;8>UW06mvTVZlTO<`q{TqkIn%8!hnHD_kDhmSZQC@!>1pI#rOj)?TU#a zXFvPViYGJ!ELwgO3;Wu##Q5a7;Z+GL37}R}*bh2{w!Z3mQ?{kF?@L?_)SJjB5;qx_k!oJ? zcK|f}J%=A>q#_Dea7~0g5@LV$#I%LiAnn$&e;2jU?@y9|6i~cV8s#{W^O11Y03abf zoITz3#F6a)S~V6}%_ZEa{gQNP3ZTTyv7?u+)DOm;c zKG+$VW{L=V81JJn^2J*Cid5>Sj>|y@xdKd>Qd_1o2O>j?ia*U4LRHt$L1zalF*p?ZWi!BB*ho0{>y`J%=t0nW5_u*q^zg1V-asTA+ z*n?qgu$a_yy?b!?d;zftDwUK^n!c|v{l4$b< zJ1po@3l7_|_z1>Ig9$z?x7lkVQi?*Pja`DEu()OA8`WSBf)Dc}M#c<*C|O0U-A)YA*Q`N?Hw+C8 zX2$D0k{4Kz-|yMQWm}e^ww-<}2zz#^oI??OVKB#R%7PRWMqN)UB+V^;%~DS$dd5TO z`fXoSO^CR`h9#uYfs)ZA+Nr89jVLK@QVCzP@S%S~eXOAY@?=CbjbBjI^PY>0;3#ZB z*Uk`#TwCBq?DRq;Fh01}1E7+zOzcG&9vV?mlpC!BYiYZDy#x3p{a zSI!C&3(A#F&ZAeEg&LN+`}AX+9a+&LB*@_Pzk&K}2FP-K=L)k1p`GCb!|ycpL0II0 zmQhiOTgFBouMa47P3nHdM0w-O&ZqXlVMM?(Su=5@e&n*q4+Pk(;np*3c<{$BZ>|$r z&(65DSfuiQ;(u2Rw5AMRoG$SrMa$|=a+>a1Xe*Ngf$6WgVh6!nhQ$#44rTFU+B~3q zJ@@*dM!dpkoh{N<1sb|j_bTAWP>QmFr?~rfY4;qN*fQjf5DKZ!1#_aSA`Qo*dkzbk z7}(Dc846n(=k2*|Y%&#~?{Fth$43h>fcu;{vZTQ3M^ZE7xu$1!-8~kjTeL9Xv&oDGL*ysIDK}0ar zQL{KGumE$^zZdhG2%ljpe8GGz95)99DVnDUKhP?yyZFn$>5iAgvXFeA>4xKs$zmZX zsifnd>fkXhDqrGaIgL4IBgKit!DV+W9rL3rz|Ju7TLsV7eVOB*UI)mZV^y0Jqt+Do zg)BaO)hu^BUp&1;wzJm3!q_@FfptQm$9K%($roJj9Gsq~z;6@Krb7lzjF!S+Jf7;uK@&gMU-eJ90V-?{hI#bMT zE9$PgW+oyEkrYE+h zQNO7GGH)Bkxuz@sW+4P|pq+PxA&jWmAryVC2KPp#_Q|6{ zJfot&Q9l!bIPMe`Y9S4%t0G^xR>T|VMIxlP#*<1}HS(8Z(qJwM7k$&JvXga75uko+ z+_l>-PLC(kP)nUQ>~NP%zLjX62zd>k9LXL>p{U(sLy{33dBw-F$Fa;lO$}T*T(rM6^S@O{N#r_S*}cHHZ%iAx$FY0D)XF;2;xB;UWgL zWMC${v?=% z`_dyfMJk*}?1)53v7KXpz8hLCgUlfa?rshjC&uiTRc%LyC6kHX(^FwWw?dmobfZ+C zC~t_UW3&xTYK3QnBnSP8-XX%A!dz*Oeo!e|zb$OBY{ZGdO-*KpHI1pdpXA$hqnKa6*d=t@qR&6%F}HRqLhMYfo{anrxA~MUT9zMXnTE{^X`Qz8i4X zuTO}AMCZTEZ#&FT5lTp1dX9UfuqL{7$6s+|haQMG;NApfnfiQ)GuMuXjDi9A8FxL4 z4y%tDt|f1L?N+ovL-oe@HQT^fQml)wvlnk2lVWv^s$FnKDP*>3bCGl&wD2>w#z5&; zK>b`i{$fj%T3&E8L}L3ZB8q}2;w2wURS|dxi6*NiHwl0TU4jJ?j9HfJ01!t%EPz^3249kLVnt(_kmIshgfpTJe?w7G#t2XbH z=D2gYb2EApXO)R(#J3SPlP(vEPSYMz!E$WMwRJ*3qU9sX{c@@XsgW1ts1OjpSNm~H zJ7uV!11)j~>qDn#JEFq#%b(|q)DzWPjbBQP6d&vZ*o++1v9oxrQRmkU_chC3jz}fC zRafdQU`5W=KDp`qEg%5+S3LrR_Mk?6(K5AUtM;d zPL8Z&gJlhTbJK@Q7?$|Tx#S?wv33wPvXT@;7~9LEO%D)Ev0wIv62NphH#I-&I5lc2 zp+zdhNc(kxLfQeS(~JT6r_1~OA{niY{7LPK?JN+zbW?&C1Uc=y{uf5|#FUmO@Mx>| zVTJiCLCYp-mi1?m3gxoIvt%NF;_Bl{m`p%)6zMc2k(SKIl74J_)q05`y0xljwtHx2Z|`}ey#w&V_GqcjN8sVaXXIPGT{>Rdu9adZ zTNvKaW>{&Oev4P0FjL_C21EiDftv^Z6{J9A&;XTUuIqQhp=%c=TL(;rZwm6Cf)M$csYTY?A9 zw(mhiO--qZ*nacx*7l0lyUpiqPk-na&b#@ZE7P`lF|{{K>`GufKnO#jN^0Bffr1Wx zqert&zkb3$7#$T|tsB{@Kgg)3-*by+`upD2S(b0}_2x{7tQn#I`u$aOE4>M0b4^~nZ-U20l1a&yL?*y)!2r=Z1X7V*|zBm|G*fAnx zQ&rXv$HwVRJ(rrx1QH)%4A>BAh=o3~Ds(pm9Q;FN-15pzKk#LXJ8BP~Vy{Y=0*a)( z%21$js{Q1(>Ts|+>^HxKyt=%Y!rpTcdbJoYd{h?cL`oP*`KRe%=o}dgG$9wI5huV` zkLz0iH$Ci^MM62EoXt2b3nw`saQObIk?=S%NKk5#}N?qAhCKb z=5i4zSN$MXn@Noy>CJs1KS2Hy-oJYfvz{2D;5OI=+75z!!ixPoP7!pES~{UCaatjFW; zdFvGofR?M_^v+4p`s)mud5S^}(70CO{`k)U2M#S8tZTJ(wE3sB8_?VCJ278Qos>{gdRuw~D0R55VXay0$;z~?(Pxx%?S z76@^`0$k14IaO=6!Q~8Ox257aM=Wf@m18PIy%1p`ZU2MDJ#Ws$l1-QUHChUq2G(HL z-zzK9kvY3>XMB+6n06X>+u@E_@*_k7L*RHH3b&sin1&wRddCvDVR_3Ec*Y^q&Tp1y z3C`!;IIpuZw|m)%HSCHf7xRWj7Ct(}&pZa+AHV9^ z`EqP&smMz9-mG0Li_mENb%hd*?YKI{jSetl^X;)Tn^lgKn3_W|@gkU2!igIDUQQA# z3UMk#8zPo&6c;A{PZIU*wMM)TCrub>d`-qvfLnG+{^io6ctL;BdYv35h;K@bh#3|A z^R=x?M*-Zl!gM3o$_c9r3=vn=loMxzCCaR1w)aNcWjqHvfP$6Vg6bIw1x}RVLvIM=s%`Uo~jFZNz_ zv`*FkT8gp@W7)njW>68Zl2NF zYr|UnMNMSJ^|^MQtGATBzc+V1|KZ*zYBwx0XFIrZOzacq5Mp;K>7FUT^+%&S)ai-# zoRUkPNBJ!iON?8J)ntIfC_js4NA~8i-q!PJN;$$T*q(cP+tc}i(i;`0J9QMP2(-K~ z1NyXKe#6(t(`N>>%GZ88tl$}xV*6Q>!Tz049sj`x%d@wgj5t*n>j$jeW`qC}iG)wi zC*aV|7oOTLA9jC!Vr&q)`rv=y4AN=PTCj{slQzw61s?VwZLAI~?t!NTffqW{qdq+K bpLLE{&g@44X8FJanHW4>{an^LB{Ts5c2S;| literal 0 HcmV?d00001 diff --git a/docs/zh/07-develop/img_6.png b/docs/zh/07-develop/img_6.png new file mode 100644 index 0000000000000000000000000000000000000000..0c9a0611070c295d7329c89b496b8869bb4c503d GIT binary patch literal 49305 zcmd43bzD^4_ckmg4MTSfD4?{YGz_JZf~2%`OAg%(ErLqNpn!A;NOwph(#Q}JN(|lc zp27S6eZSA|eLnBs?;rYM&YZK)T5GSp_S)CA)4+r1M3SdIMtTPpQvY zz#{nU=vc=~cS`H1w4mtPku3})hXelmAI>u3^6o!>qpo(D-z&oX&u0>tywHC>fzcBS z|Lb9SX3~lO9!egb_Md@4!OGfX|MMCU3nD0wwGh}2poZ0qdk&|~XecND_UcipL&jv}UCPUN=S719+xNf~{lK9Q zOAF_VUcjxehsmT9O%}<(GB%6Br6_elxTF(lUAid|uUhu+C4DTf$giafQRoU&#Q1DP|L zI9b#DA@bi72Og5AfZr-}uE3;#vd+pU^#CQnpoJl~#=`Zto9{ycW1s#`LOh5`s4cdT zT5csgfP_T8D#7nNH@m)4nYa`YwmPt!{87SnYr}hO(2DQmbY~3UFluR=FGtphF3$3a*!(@}iIltKpa05E_OI*@{6W>IOI?C*P`APp!fZ@$E%xzHcJM9g(zpz2 zCBURe;lU`Wa1y^`^AAMZ?E}ukZ~U(%97+C){`WK9Ke{R#-c#1@ya@-%#r{>lLGGWi z{xzxseD>>fjs4WtkO(3}^7?e8I+D@HrZ(W}d`4aR>K8E?*3sku`pA8+NC*S-%hd@&+W-9|k@V&6OoPX!RihX3xFCw;^|-3YC!`Tf z3jB{XZDVc?o=CbZGb;x-5SaJbEd3@h@7#K21-yL2;oIB$7cl#2ww|FF-j`zoGZLY=| zA?3b1_3KvaS-v}4i=o6>jdg5DPV$eNsn%JkKVJ`USL+%>Esy7%ThAa?Cx8*Q!pPK= zzY~I&ss?zTvEblljQ|rWA-?l*fA{!+5@G zIP`upZ|mhF=uPYOc`@R?PqciEZ75Nwb+*6f@q#b%ehc2Jq}US;DN{U*OdnvpcgGiLwU%hZ}!EHqHT7zFsV{frhDLDeX*FE%a6^qk!)ty_4S5Zd>8ubXN_ z3UJ6@NMB^Lh~%-!&AG0R$|O1gHiNqc3$6dv(bVswHr51*w>Zy1@2k9Tu1=EFK6CAE zIame`yyvb4rgw^dj~&o$UkaUR=fep)V2pVG?}VLK%>Ld?`WHlxf4!0aQqr%bF8S`| zCX6+Hw=pD0=A@HY`g90a;pFbFBop*kqT9vZAhX={<-y4B8fJLb>;4DTfKTvNS7PNl z^~N*&Mg(5&DK$`I5S%fbaqyPHrIoeQc{Zr`Ru$gok}1X}Oqm2q`uAFj&xCEfeY0;2 zpABl~qe943dal0HJ=f}j$v{>L^WvW(>g0vSFbCE7ySbw5hA_Z?aw8d^hLz?eNcpI} z>SVc_?SGsq{Rsqf{YqKxwit|^KNbE)E*H-{iK_h)jtd%T1H{g_^uIn^A2Ienny9yU zqh=b;ez`!VF2O%nW3QXWLmY^Y9*3=|I<3@>XU<%Pz0a~iRG;mC$CG)c;sr+GEE0%L!`!S`RG9U#daGURmq)e$Wf`WSbq!rbcve4vJFXJqlHNv3dPG6TI9=qc7mKrzvWapN@ zXuD_^f<(TNQ)yZZ!T&C(-+2*m{=0yzvT|dAr~cR5dbje%od#|9?f@w-3C<<~72BXz za0B!Wws)qw)Y(0#$noq~N*2>M1$!ch(hYXfiTnXn)RlOa+;Z9FlfD7H$#M zCWn6*Vy^(ApHymgw>fL!C5VOd&ZFj%4Txa%)Y=;b!=2YHg1li%w4RzcihcYKS4ygf zpN_x5+Lzc#Utg)0^(z_Q$Z3mQss)$mSF@@1K3>mA8a%l><}z37_^ragArhqH?yQ4z zaE7(D&@4{DO_J+9ra*Q7#yUBBwWzr4>9MRJh7_Eng@O-{x3tYCXZ?EL@T%DM6zJY}J$8r!CYITAl$yFlIj`juWc( zlT`Mvrmh%Uk<=bTLQ@~76w)}}%=@4G{3a~_QhyP`=k@-GqLJUI3GB zXC=d=-DYoxWE?Ah@89scT#1?Jd&sVYi#nU*|d{KVMA>42|)fSn@KIP#zqNV?~k2lW@L zD;P$eACk3*N7;>c3>VZCI%;D7Fybd?$i!1P+8S-xoTeOwZr4na28MIECa3RC)v^}- zuBDmP^p0p!n^`vwufpHRk&O0@$aa3DZikk+^mplkH?VpxjV*x~??@9U^ zBxWm`#c+T}Fv5v^smUa@o&qFYHES}Df3tp(IE|ROb9y+gezo)Rv^kH!fE+jM5pD01 z$X3^$%^Oo5p>rA>CgDH$mT0n%J04*ziq+1`V=+MPL%A;p6bUx@IeIGyGgUpMo>I%x zFRl|-_vTnLYQ#bw%O^$_+sW~DTw2=n?Rj^YRVa*^U+bHTCTGsy3<`{7%$eK`(8^t( zJqZSs$EOl!7 znyZ^8`5Pq>le~WDK7-rqC1*hx-x7bjv#YVD@OQA^84-(d44kB2;8b1wavV?A(+vG1 z#T;|P3>IGW+DD6f>6r;bp@=Siju`NxS!JioA%xHGPag*LyLY9o+({I2f)@w8?7f|o z-BNw4TQCFKcvFHMk7?j$n}Ng;K3L;nIG79Rj-s=IDCtQG^A53LC+BRP~chwy!DrWem^^RT5NbCxRo&h|ex$@{o+!(2I5)TFNt zU(uBfL|_hRX~V~Is@m9>W^KMM-Ehr;AHrWAE;dlBb@;`peUey|-aZH;hDZ_+VsAnm zKDyIfpHH>2t%HYh+eUIBScH#GjU;%Z;oinUn*K36gd=EEdz#?g>Rs&XnRaZ8ek}1p za(6 zOcewrr1U}1meY9Vw!BzH{f^@;>6^2WU!vSO)gsJGz`;U&$Q&<{edpFOLeoybd#7B(k?++YTXt4lRN+?7+v2+ms05~Mu6D6x}SSrX9V}ks|IhdAM zO*ulDVHGdS(E~7$xCgmQKD8xEkZm+o9vdQ`*HVNUKk-WiXk{{e}h>Iq*xX35;bQaUT)@GN#I3!DNjl45%ke_(J z$Sm(QWB^C;D;)CeJz61a9(TK~vD}|B3uBqcZC03M49A=>e(*T^q5)X2CWvTJ^6XNM zqs2Td{&pU8E&F|=>@cv*@{F1HBjoTxv}{(>&Bhgy)Zum5y{i;m8;<4%m!oRrU6{WI z|9lG4G7@|ukcmsnM`<|ys3BU7b!60cxWf3*0Yi)_4A(Fc3j!WI3mCn#>?v(6M~PZt zcd*;+!w!26ezyB{CK2IUL3$5%;wWGe1(xh!;!MGLML~)4jlAD`Ahb%yF%4EtLMi0! z2DcQtk1ko`h!J=v7f16cs7}Lky5772-SRC`v?`+NA+H^nSg>hyqs_maY57u&Fon%5 z`Eggnbj2QQ>7}{%H1ZV=U#TUnKAe_gN?(bH`(WBh!O)Zh>@mg9n-faJ>@bk_lsyJs ztB*sTRmIXuIRHgOfX*7Ty^QJ~-CF5qx=>eCvep4DXTz9=TlDwgV2-^?~zSD{*UdJT}R96%nj}i1P7=T(or31HjSGlPU}1g&4~=_J^x}T$2Q3mL&3TIDotTg z?;>_lh0iO@W6h<)a)1z$o?=FX8Gmc-SI_n;1Kc9IT#6g_xrNaLVo<_~WkwBt< zM>-5g=rX1c`GzO~EOV1GK|_h5NbF)$NKH98LX4JQqr&>^7ltfWo?{gyjzy%CalsV} z2TQJ>oV60;U*mMbn&3A4h`W9E>ERmW8Hq|-_#1cPWeh6KyI=ZJWrjFtEzh(i*rR3s zZa>k@Z~9OutWmOSNr*-Nu<3MkdpGe>YOvlXric!r&IrD80z7}ZXh(c)n5J-Rn??@* z+n_nWL-conhldqa##V2wM2nkjBnc>f-SDf2P)zV`B^S51rOrapsAoCj>jEox&=ld>W&JL2MfKPyr>?(O3q;nYKeT2J&e%Q{i1)bIV4FqXEsvU+M zV~Q>D(_%4n-}|^Dwua^I;QPq=MgJSXOERq3zl1h@!*YLf;>|eQh;vx6O(0=UCkn@M z&&NXOrlO9(GO}i9OML-3Fr9;N_KxViC3A(hAdL!^d$Uxuf*cnd+Dn2OZQi{>{laT6 zt4WMt5ppBJ*erCzk%g3EPXySKyI#E$C7P1S?WxDt80&eQF6{WUn>{|X*Da*_SdKh{ zv@G<(8k33jrH7aV{(Vs!`VGh?ipCen?k9!!<;PUh{#eA zf`1%kW&OcHM1$>ypB*Nv`hJ88+zD?FKe@Cq9*V~BK3f@sN?=f|xNgeM;;mH>g2Rn*9NpTwTkeu4OcuwPI@_6nfIc%@3yd$yD?P?h`loBQFr_W+ zj@goO48x7d@q$d|lRioMM-BrFFYBTxBITt96aLk0&OSj+aJax?y*@T7jw^vbJF76V z6r1p`c3gKm8OMUv&F)$finZ3uSsS_d<1lm7ZH>xA^L$kM>9K*oAwwQQ-Se*Dz$VWz zBHkCrq`YZA8u20u$U1ZUgKL%EF=HiAj?!QcuH1%E798?F8#5eQU#C71Cka|?*krkd z<+7M|mJf`6h^nd1A=~MEK0-br%A2)ucWoOL<%GcJ7;92h9kiu()<$&9vlzB`3<39+ zEe?KwObrGEtK^}SSlEJC@)`L$GRE>If6feK*clCEtm(s&PXNG>JVzzGwq0ubt4EO+_2zhb3u;LP-R#f`c|bh;JVPkb{=k2p9rJJJ~= zkaJ`uAB+8b=&^Jq7HWx)|~?PttTs^p?D$-v33CIrSzx zZ>7c=BQlUta+g&Gl6_TJR7f}5u;@x`Jsvl~0rz3}$%3!CrF!pQfR&>5njce+R$mV0 zIf^a@$CpFxdU7iC1Q^}scTRiw15-2ea)?cjJ_1R0n=>W%Ur1MX`tBYG$MzL2cou@c z0jtG*fV>Yt+HVVM0AzW%<#PX7>oXOCTTJ>EfQF;w`J!+u4}iZ%--78z|H8v#ZFmYo zn_K-aZ!z^->>tkgFV6iB;{V_l1$Gbr9~d5=bRuGT7a$t6A-4!OkIb3W7vYG%p!@C3 z+JB*Mp1;N^ztvYWswK*fe;marNdA2T=l{pOQZPFe$ku;T7Jnsw_jgGA2)3e`!+sF} z%m4aUYSO<_{zDch+zR&euc5JSzdEG2HOj-ks#E>_O7~Wh!mz(q`Ab2V{QrMITLo5O3Ke~?{3=IYu1yPJ>XeA--yrko9Iiw7TM?6h{*EDOt?3{A0A2xbX6MmIQwqS- zaHTGrlGCSG)I$ij_e-LH8DXFEhBLnk-~VSexsU^2NC00PV?AyDExt)4%sfDzqJ<(< z#(UnQ?JCeuPO~2{up6bSD`{DEr#%2`auXdl_S{$)=wCu1{5Pfe;0TNX-_siwV8DL| z(SF@enW7bB=|9-dG!7DHkW(RI*wH+9NHildW6iTU8Ij=}mO^{! z^nZ#GWp@BbZ$4YtG9?K(t}D`lY{Ksz+fYkVWoqjb4*?YGX89yC!}&ZqfvChnPS2rU z6Xgy*n5`oI)<4^;N>zd%W1Va4SMkGu^6`an~O)r zVH=cyMoREmz8xcYT4D5`w4=S%@Djsxb7;FS?>HO4OUWlJp0E|nF%)eKvH}Cy<^eMB zMPmuy_hYR`#;R)5?E}ZJ38BqSWVyo?d9R42ltGE}+PGo&zA3bFBc-y#mp5ts)~MQJ zzZqY+uK^z3^+hvvTGD%x>mVO)Ca=lcWYsM7y;t<^L>-7c8zd0kuSDWo75A>JGdtvOCMT9F0_lu4~*RulyYF~YO`-@T7arT#&?lN$k%iOL*1XKeCJTr>pd+#Z@ zgK@^znC<@k zNr3JOz0XFjt!yL=bDnc2b&k0tcfAr;HRAUW5LS;}_y-^@25L{b?X`Udw?y$<3lWaUu)5jiihK*OXKB^#NZwdtjX^Uje)L zTT^k*E#(tOM2ncr7$3;Ecyv<=rV+3k@Wl03v_^^yVHaPH5)y zTloIwbB@7gp*^t(s4Vja__uL{GslPYl zWK@m5EScQav(!h6jmVZ~t(x+<*0N*UOgE>IhzVMoj{-)H#5ZAqt8P93e|NR>Y7c$O zq{00&;vMd|BoM^A!E2J!4J?XAVJ^UOV_(no+y54z%O~>FUF$V>vQ}5LC$Z$=o5u}O z{Mk@CKDnWQ*2!6zDP2Qh#F)S?6{k+gZleJP{zeh3^9e|h_0ih!Oc(rNrO)7apRmqQQxtA$WWgs4YI2ArlI1g}_0lFPo zlD^-+BuDivRce{rz|$N}f$gEY0bAas>iRA|$;2+9=59Y}B_wB&uo_M)b44+yS++#y z6s;&VY(8j6DQ9FaV^_nRXouuhBh z7qAL}t*T$x>gz1^8b!6KC>|#l6!IGMt=1+>NS@hrS@)8zBS~E)R$)+r%D>9^1%B@320meWDhW|`x_=s>pFl~xq%LFjis3(T3Jpa{O-bq-Uv zpg0)GFWG~x&_cT3D_s)#OD((E#Z*}a#fk2@6`L^%IkYZC(5=;y3Fd=?R`-B)b1wkL z5{hF}$fufu&d^ljKbm+L!p|49ws|C>(V!m>&l7vv!GL>#B*K_aV;eeYn!N920JS> z5R`bBW;mVK45I+=OBDE&t#**^k6SsUuNg?(;gH>{UElum_kfv%9j4y|{4xo=@@Qai zQ!%CE#I-rhnIep%(zfB_BD)x~-vgpo{(3GpD2~A74wJ_Z&1!(O;gXCu{camJY%xu| z_Ob1ZI=cTL0AOG1SH@(-kWB|`WrU5t-acE4=fsfPcijP~)2p)Np>z@UUgozsIEHAT z)dSP?X$evR1`j>$%H$e*$dEu#M`~+M{SDnIa=Y&At|;H42n2xg1K`b zmM2}oww43rzv#cY2AJ>HJJ+MZKrukw){5@pluK#r^-*~>!1PF_TKp*M0b-i%cVgMt z&UzTF31$|tSVMo?q2L}-WQcr?=q}?Q9Qs_1u(}W(SYeOz2@rpd8bcOsh|;Jg?i5hl znx{O7+8eZZey?K8Ivm7OM1gg8-ebzB+9E-+c5<{&!>Mt*R`7~W#Obh+E_|G9G+z4p z^p$&(<9hRM2mbXJ|J|cR)N=czASCr;lvoD#*&rRr@w^mU3A$)_n2@CtBT? z0-P3?+m)sVF@Ze_J#G*M+6zO444*%rzqwjxxGFXfGB8d@V7dL}(c|#0fKV;=K~d4L zv(2-D6C_XW9^K znRkLbA*-r%BTY#1Grsd8mi}nFOEs{$rJ1-%KhM2!%vvpM#FO+kk-fUD4e+TPtb5ux zpMcJQ=BFU|ga-)Hlzn91qovdsP4aWZH%qGDTJV<>=DZ10#KGdj(@>&!l|OH^>2jen z{Y6D!PScTuMte_>JXnqa{(V$jU6znA&Ct17XohGnI_dev-L9e+#oz5)_xm}&2@m2c zX@U}+z|0TP)^oqV$9gE^;E=ndC9jO}8ZYcud+p){EHhq=?@G&#xi{Y-+_!I$u^7+M zl8y&#$->)0c?-*)qxKq z&FU%Tk+P%9EecwR-lvWi@pBRy#EO|h&QcI};T4|}d|zsQIs@TfW04Ina88BO`OX>A zMwk{TSD(C0=v}&j^3mcu26LqlHC^zmU!pM9@k49ws!8%=x`VOBHX_%EzFM4%L9>4& zE9lfXI!w@D!Hay^-O+}ZE%W1Pcn?lHp1`q}W_=yRb2l*T=5?UXQcP~vd>}v?vE9hY zKtjXx%!ne)M6X{`)6@GXZ^_r>#;^8sma7bRW=QH+lyn^Q6|3l%BvF|`E4Ur9qEb=z zjCmLU*LPTTwwvyG#=wLrCoj~9#}US+n3S_8Bg&0Nc#RQ4MJ49FSz1SG7I#s-W(Mhu z%)C#H7ah3N3#wGy4ZIN7lfeG66iMwGDfsG)X0W@b{HzEuQ*Y+*6!{>i{{X>hWAvf5(eo{O--TX(fi- zq@~kD-yTV|w#>1355k-5BlXX*(3r0uIky0Sg4@MxM5b(oaJPiYdMHAsSB)|D)j)b+ zIzefyQ`#FKGi8$}-1t$ILD8CC;tvfIH6rh;{oY|eSO1-lowhCyt+QX+myG|nH0Mnr zre*utH`ALVlbd&!_Urnc0T&k)*i3$WGrjekF48c*ie6^;@QLYrG>9RT@Uj-mjogJP z<0;L(q9L-k*mE6gNh!YzF~(a9m)X-SpCYOpHD1^Xau*HEY$gXUr({wFk&Fc4Z?-L3 zPxsYd>@+4hrp(=Cp-b3<^xUD{J`~89yrVgQ+1c+PFTBMYc=oKSoKX6UMQ{8eZNQQG z;E^I)#b8rb*p%U+Q%}z*tpn*qm02Vw0YX4gY}$sJPN=^-zk(MN2ie33@2k>T>v|*R z$!8b}BIYN~OHSnt5S1F)g{Hin$Db&^r;7I$L|?nN;Wy_R6c4i=A{b_*PeH zq4y%AH{oErVq$zaAADZ#n`}9D-a56~NEV15TPbYQtZxIziVUDxc!FhHAULXUtiE+Qfk#~h;!)R;Gc!cO=9Tgr5X1Eg3%1lT09R8UoPP%c^yH7hnKEV7ox4jbq`SFx4 z0HjRr~6BwVaxdiUsK3#+E1$rgR9R6jWaWBuwraG>*i z?J$ES!6j@k#H$zNoQhy9i&K8_!-6E-t^jMvMQv~(rd9L$Rq!M9ivc185B%xRqNqo$ zgn80c5zd{oS09AVt4cQC4aGdgflOl z?KV}@B=sZK9}YsmxEKAqV{UlQ1LPEh!g%J~PeU@Xijkk8Kn!`iy)kshvqlKV{91}1 zS11FsW3hgyj5&1Ihu|zM6s{#BJJc8a2>)(a5Uz{k`BvD(2AbSQ8U3C~Q2DcW(I(^5 zUUWdBp^^0`D>q=k!KK30{2&lEb2lD=&ABh#m1S}_`9j!$hJ=ywl-NOOrL)5ZJtM|m zgDX@vrL%%-E~s4?_R5CKjqZ{#p}jTZ@+`w`_(?68wv1c)aHkQO|2s%O8}=oPhtK(*ystn_+7 znC6{zZ5Al!x9WMh9~xxZrH1;N(Q8;GMQi2hmLBaw`+jeo#oq<5Q)c#dk8g3%4@(lZ z1`89;s+`hUicq%i8?NS{^R{NbNN0SEI4_}62M0M1L}OGM0KC0Ol4$a5W0ep>26 zQE!bLcy6#+j_Jkm_7mq!$F{anvV;YHakDRhMAXq1z8CclV(=Mi5^P8~JH9A4U9-mB zk4%YfQrv|OGXQ8_fn?wGSBop6Xl1KJ3R;Q%$&r|l^uKvOqbw^%6=eb>BaV5nsgVY& ztn&U&<2^f?CIp%?e_ZsfDUorI%pEe67Z^8PJYga(O1!wg!Etmeq9WnLA(8ZX2KkP6e z92dt6@TvIVIL1=>793NbkI?cDo`o#jfZm(~(y(H#cyQ#vi&oXlTl+IieeTSb+9<+ZGRJOO3} zN^MAsPxfkFcYq}G;Jd{@5MsGrKQT|IYvpoEVb{lEs4-@-xicC-s^#yt1V8^p&?Kl;KrPG{zH9!rhTd$X+N|hQAqHTP?VRwpf9&DVxyPqE8m?1+I?|Zsc=54Kx_nV zts&Z=qRDq=de$@^aw65E;=pB*O2P|}k%1@H^L%RuXdO)B7R?~oxYsX=aWgivQeV2| zC7`GoXBt2=80ZR=Y_i8=lX<|Y1l+$`jd$6>PTomrUv8a{1JRTEb(WkuH)qa9jq}3> z0!YW%OTIvz@h?~0TDu)c4V-FdO`o^gi@m`Wf8xNr9gkj)qQsVS@CDDnBvnt)NT)x5 zrkXR8E@mz_h9E24v|aDZ;mqqLP7wtcV{}Qta6G%s@C&#r<*K@I)4ePHyu-1~{!Wf}>w!r*8mC1=#iuq|K3%nQi-plAhEKiifav> z_FY8^9~r*z`v5SEP3oVswhvqpdmb%d^8*7aQ>>TLBjz78Hu?#=y}ivLfUHZ$xTVJ@+&x(Ixguy1}H4B^t|_qZaUJO(A9aW?3Uo{S;6DJb?U@EcK%_t!*t(Pq!su z_hjIjCz7rcG+`tAgND+r^Ky`pN2lE%)V+l_!Uub?A^2s?ncnzGD9$=nUoc0cjWEoG ze4!#PW>0Sv998)Ag3fZf33?#aSxGYX?Zs0F^|J*+6v@P3N++a`ke^VDlCRmR47bu} zm$W)xlqe;O0ge@2{aS@V3;>An8yOjGM_=S}^3&_WNS68e@bPzp;2off`e_|x|c0nOW9bvoXAqHOo;TxOAOB1*br_~Y6>77(^D(4B1u@Y4D zp~;)#xO+Z~*Wmkcz7)x%w)P%!J<^cmIS5TFj#b`h&==mG$ z|7asIyH3SYG_$%|U*bzKdJ#iW;vnXSm3+qaCHAE}g(l&lg5W^s1;b_Q&|#5V0SW`R z|I(!hH)u6;7Zh_B)uq)9DtX8uhzi0MLC`u?rfDH+u&Fq>8B82G7(6McHPi^8hBml2 zuB-m|wcjcFweibRV(UZn!=--7USxKwC^4j5^Kb@Gi?tc68o&x87Bn+LUr2AZMZ~Gb zP)+a=V9aSIIuuoEz0ls|G>Wm62R4k;tb;$47QkO8vGv0TZ5YHKPwpZ@-ZAb4wkpdK z0FFFYYLg`dUec-uyj>=UbdIW0lV*H)L#=RtSV5T zdS2?+F$`<$-br(1YETtFdlx=tQBgH!kz$A`S)za2qHk{09`*bdPkjJ zc-MSB@{g}8wc!GHDB#qoa+%m;-w%UfDG}Mxs;G%Gjb44_kUuN@4p`NpVLCb94im^% zC@Jza+6i)rDWo^MP_U3nU=l?u07ObO^U&gX62;%*NP3FF;e(yQFCb7!`O?+k$Ky_| zSI4jJKcJ6sf2Gi6rk1XNp!|^3`}$awPf?TmZV|HEEE=i{%$>@o;m|^%lIc?*$3HjH zG+m&vYx#|@#l>zRFj4y%m>b%VaOD|4M3=~;i4ab<+6*$FJ+Mt|Isah*Z_}=v&yc`~ zgn)i9Ua`ua|JnVy$JsMIadhjoRIc-`z4zCb{9q1QsGcS#4^3K~e7Q76|HH9g)Vn~h zQ^8a}dZ%6`6S^sxvbKE$n_ntUB-9I_mN&CL!|ePaKvPv;uh)q7Tt48rg}D3vFJwyx zmFjQ|UBYR<8+BiL$PwC>+0|CqbzUM@X%Z*^v4UF2B5p+5Z3{Brza)lJv2^z-G5HHN zuD&-srV06m=)DGkj1*;wT30`WO8@?t^xeK(zuKzSZGHIFwIGfyz#{2XWp+MB)p7rF zmano5K3l8^1RA*lbnk003SG??;AFG5NFsTo2QMN&rv>s@#scy#>hJx-S01XG7D2g% z2gM;nBL1>cExyw{GK^WIwIy6y#!t1$Iu@mKs?H=2PbKtRzBSY}Af33BwTsp1;wW3F zaP$>q+*gpg`j7jEOCxXu@jN}3g6W*Tn(btudMkcnClWUS6xBBbG|@)yH{ON1=n`gY zF==gkF`k#lugK-uWK_12D}m(C^>i2DKzb&~*Sv<7eJQlM;q9W{d)HVMSC!9jzbwi6 z^qMkN^@e;95twzhzEQf>n1w#5bSgr5%|IL{EMJ47_og7^GeG~-y4MIANo;EZ`2CBv z=Q~Y3Kr@fn+!tG|sw<$R?rE(oM!%>refhv7n>w!q5SY#YPStz`x9BM!_3>^ zi{sr+;)|c}+Uf=cH`2?9vNCxg`aE2Z0{uR$4IJhV-5482Aks*`nxsdb$E)%#OlkbS zI8dY^pG%8?TH7JKlNOr~C6VzY$sSsRjl!+(zPSN@kg{!9iSJq{`X13Yy7w-+R^^aN zY(&peazFBl(&7^UnKGuCAFK_zbx^RK^mbs4NE{5JTG-rrod9-8at8oJ)8?qq@1%*D2rMP6TN6sBZ%ApS}gDGHaGfIlj0A2-(n?@1LHnXAu(~ zyf*>>!f&p6+p%-}ud0b)@+&pW5y3~BQzOFwkNRqwZOur0J;UvJI}#A&>4-nT-X23E zQ>5djAeeNN0yd~VNCNyQJJ+1-4<62>6Gg1tEJUOe#|9*%6I~xGDezYv!9ee#tRbAX z9rLE{r>9|+H(I1c2eW%3K$J;WvIXsiQ%LhRZar${47XhzRj&Y$u$iMN6KP+dee1g; z=$g3;>*6FR zj>U>dSvEePytjd8ct$37$Yio`!eH1#FeZY_|H=;v-ki*j7;#2bMI?0w-Xr}Hb_#Hk zN52iUV}>@%TeIdj0kZhAObtb|SM09(BT)IzOTRCs?`L4e`GYr#G>|hLATperDo_2u zIRaSXu8%V9rqZBGZeXP3J^le7WS2SMCKe|j0M;<;w*L-+0tRX*a3g@O)36KnC_D2<-m;F-MU5apw&IbLtJUF5@g%dXudTOKw>g z8`U>VP1(|f6hBP(`}EfRAJRaDVviKOB41*9YKquE76!Bv0%UNKPg6_8bvQ`b*dFY%By* zE*Xfp{`tRn`O9Zf-JNS3sOk>A)BP4C@CT|z^iMY6=wnOL=(u)r$pEYBe{ZKt_J|xD z5zdRehzE8Gpfx!6`Uy~+N)aH7x(Ou-O~07g7=2Lz&i`9Eo81uF{SpuhP*gRfF^L}98d+Ot2lmWWE~Cp(Lzcccj=VZ-yFn(v@j5k%6a#N0mFmLX~WJ@ zbG2Imwmc{T!D)dlpIAeeT*7Q9fw$kOWU?jZ_eE5> zthjYmQ~m3gYQz&jAygO#+wByFcI!C(fRVy!PK~-BZZ?y}yt{4IUjAQma5r+~7vQ)y zM7Kpg4Q>DXrFhxx_X}~yxco^b999%iJ@LC94gZ-%+M2=$!3&DPqCq#USJ&rNo?$#M zr>u5r1C(G7LLYHk=>z3&sQI=5be`nx-cJN7iDubB~@ zXv%R6T~j;6yCi3cbtf}b!GYnZWcjzbxWq1@uw{$Dr=qh>qM6HP)x=T7XAZjyDLR)X zfXzg4kJJHLPU9Hv9~AFgMDb4V#}KO&hWhH&Of&%OYjt}wu665;pbiqQL9Iutc_rfM zEG@%R|5;pl6cZ_vpJ9*ubJZC4QoWuc<0)!VAmn>SG6 zuG!5%GACa%TkOzagAk&aBQIcA8I&z1;o1w^@|TGF?l&ey~UEJI3Kk9q?<% zriUS{O4fk_K$z%mdXoAMk)cT@%YWH`Vj?YUSu+b-`Cy<)X5!-+qF*4}$=<`1vB)>` zbI^M7(a@O}X7Biqq%>ji4IR>nDM(n`|eO8oL)&+rTj^;|j>0*J6F9*RzgX(H+*6Xg9Aw z`N3E^Dz{Q!Tz6;hUI0E54@nU}p1sQa-JI%3DiV5ek zef~Uf(~$(KN3$PS$NwhPVJqn-oj$#Qi=Pf!G$Ex>836+{%;^REcOvmR@O=3=i-J+7oSFFgxis^{ zLaP(7^w|}^EMpXR#hyqndITMl`=~6cRNyD_524?hQS(YJDtOpF64oTlyX&HUTRGzc zs8w;h+@cz|>^P%K_8#p}jGbr7 z105{K5D6tStl}0bnEo>Dc$~kZXdAyy5tH{~@UH-%_a4+*EV7&GWu$n>0EBy$q{=9} z?0|DygNQFR%TS}9jT;CKuvB~Uh2AGPkS@ghk^PxCr9%qTjK=(HZFD_}MF20zG6H^- zAxh73{eI^Sf79C-!Z4U!ExMjpnE;MG&@wi@g{o;QTUQVYS}@!)Z1$-E3e&li%oK`* z7ZePcuawIjrwuz>@^`Fb-`R;fo6Mbdi}rU|^RP=M0Q++OH{c-r!&fiD^yonN^PkB7 zE!Q)xBYjO39m6MF0HQ#DuPFQs*hPbzfcm%#`Czl1M>u-#Q9md#gY8<%1=JNne1@tz zrBMs~SCa(jLD(kvXBJTZcYh}Gg<^V}cyunFoxX;AHS(tcL*kHh9f-MQXTE;iS*< zkHp$Gye%%sAD+uEFtLEb_!}k+)B>FT(zBna+Tr`zATSe;TJjK8HaPSyF$z;?vf5^q&uWLl|@K{i|#o0^7-#~zh|H8+Mo1;t~uwO zV~k%6ZYJu{Awp9cz-1ept3V3WyML$8rWcuqCF)o*N*6Q#MgxBH6z#HZ!vOcekD1$u z;-t}~*KI2!>HZHqy80tqv=?G;`24gn#t(iQ*z1{J4KS{@22KM{hqlgV3W*FCb#Tn7 znV>M!-FEU-%e!B~?X1bd&iBQ1fZtoSgj5CYK!Taq{zf`Rgi_cG%f&s*Cp*^zWpMLlSA&yT*+~FG%rvv^mW+^D(%~jBA32cGC2$8)`Rpb=;91r` zS0dhk^!K&va)3UDVd>!g8|oV%$iH*LN0MpC&7rU;V!`3_BW)>Dy{!%w&9G4PQP*O8 zm~neYX_->@v;$5~dJ6@|FVsdF#R=!wdyyq{h^A~*i>D$loZhem_bno9_Nh)U?jI{Q)3b+CeA zuBbI|j)##&OEjGv`G)2l1PY7Evf)odYL}Yr|I4$T6|Nzo#byS+(z; z>AMo1_tuPBl30%ssRHrRaZN`sTp~rgeln0~7(Dso3BO>L@EZHJItGM?0KGMHKY*~L zW|0{kKcwX_;hDEvXDRz?Y2+2}xV_$;kj077fYXVI4(HgJQz`8?vm0!IpIi+Y*}1+# zSI6*5(LdvEocknxvcyw>g-Uw$3q2Y2{ePkubt3hLF(azc&lpo(vo#3=MfI}k{YKQ?17;nf zaGyU~o@?himQ#o|YZ;^$gIe|<3}#z{HiaI3O1;eUz3Mr4dCC(z2O*-1mKI-SnArBq zd+!0LF9lnVzHWg9ncmE#@p=64B~q8aXQ1^ke6{^hu?n?txlb=^y#3R1GV0}iQA2XW zJe1ojNW$%G6)U&@pU}jEbq|R>GPSQu5ZOCKUGerL>jCRn8W1B>GJxHSQyXG`ZeHJ< z)%f#(CcJ{am#EAl_*YAF22LYCj}WB4@Bhr=;ype%eL|~oCQUHYMb2JTWfQeQpO*XP zR}D*=fMX`%REwe8M#6o(+I-U?s4IYV>X+cD%h757E55t+s8xFh3+22q;^Tb17PB=H zySw(c)^k0cCM<{aikDoEU`(Ghmtg1W_~H1Gp!BU8)>^T%Q5#hkih<16Le1kiU?6Qs*AJEZP)~ zILL`NxIGTyIUyhy(;5zs z{17%uoNl)@@z#o=hn3fM@RAn6Z{(ev*Z_u!AnqBx@2Ip3-r_nwddJ>Pkmpm+1#|AQ z2xoRQH_f8m5OAr2Ajo`*9E8m>znZLw$tKXdD?dPT=t4V2eZDh)tkYjEJUlIaLA%~==!rWZ=kPnfBzF;df8KD~QdVpa>_)uIqmL_Iid!_5 zL|e0VT?;2e>fU=&bAlA=v5Ff6myN}h-yIuxx{$GaDEfJYR?m`cg)GfWYx~>JZTbs} z1V2Tb^CG%P7FcwUQ0bm=K*@H9E>r&X+Kt9WU6KC0QSmg>o7AyZEo-`Mt z+T^@oLPaPbRyv)my&dUB>5MkxhHx99&N1G*et{yiAyL>M0bO65yovk6Fk3oaYM-u zi_46EL^rd;x4;apg*QzY>v+VF>QzUG*rL#7wfm>>ys=eYJhvlGa%E&p=cHWllyF59>q0z=B%G^})(Z9_mYdCrN2bGUJb=eRES` z3=9TAF>B;`mybLV^R%o(J-b~(9e(!7e&G zvjIAi`$~iH*a#<;cN*xjCZ{?Bhsx7V2i6#v!P&>+WFxk#E!~>J0sRam_}2_ss4c5$ zogZ>6GR~B9Ohzr!w@BY~%X8T`P)ShN+Ep)0NPeQY)n2}GGtqbg3z?$+yh5|8PK!cE zrTJ2P6Hg_~cJ*&vi{6)5h6s!M36Sx~E9TK#W8R2puCzSjh4|_ULWqJJ9nns3dN@QT zO2ts5UveL@dVEqz(P?#eKK8+Cc%+k)*u{}MJ}>WYs;P!k-gr8x5Vy_^*#vj>n_T9* zuT!+}_!TXTf8mg#D3x>%p72Vn3XJD#oFS@QX ztQ+yr4eeCUF*V=X=|P?*+8(d`3>@0x9%?@L$?|FE^bw(I+6>}qdi{}RteqgBNSoCW zuYR6t%WC=o&j{67UPOB>gvNUtXq7#iDeJxnTxjCsQ4iiS4iYM2ja{=$rq-Ep?$$;e zGmsc`ic>t+PXt7w#A-x}va${>IgNG7jO33Uh!0)72%$P;rooTrzbKQ5`o5tM8KYU` z4dp$-e3z{r?Bt~fWG2U$fxIpZA6TBFc~k!uqYzW-em$eP(%_dJqSAuH%O?dCJav9% z^kF)e^oY!i(nK>P=r+SQhCd_jT3NDH%~*N7BgIJ?JI=dtMC`LP-+z9<8)Z)HRQb_~ zr$%uX?C_qMZ1#skQTid-?NH@|)2I;B0?JV&F&^4SrGM(k zEH$I{iFeua)MIJq%o`Q-znLYmo67+2gqHhi5b31dibuadm}45$DdorQT)BRd5t7tr zVlFTAy__Pe>*L2y=|WW%X2)HmnpCy8Y|%FG&Nm*jW=BWeQAXi+&CBp96m=wqoSEjB z3~-P2{+#fpSehc5cq(N|R;5G%;YEP{CY`UEn(wd=!Pkzc3yu{rAi|O6W#5jkatfDv zyQzV4`KxRPFWi^2=gQ@EcJ*f}Yumr;7AdfMi%f5XIln}DJJl>SH}y`(dH^QxlXPL? z)6C+6Xmz@+uYzYzjj8zbU=)l^i}De99XsY{wm4EnoX1I3Y>ctqLF`f!5Kv(ez|@8F$Sr$#ZdYlD~S? zW^o_!B7aYQge{I5Ma63H;mnSloS}^tR}}}F=8~O1v@+^`7?B1|f`l%bZzE(^^dg5~ z(K;+j$qrsCX-O(^covT2eJ>qKlI$^u`z_EqC>AqRR)!wx@^jIxxYQ(XrkaG!1ewi# zB$Y3$GTNPGvS)yHw0NLMOLoki?5<~moL2FbMom2N{O9!i=pqzm4O78qD~CSKi!buB z4|w_0nc`KA(mx^u`y!5Fhknd*QtEz6L|+~0vwqWoX9MY{g>jGNi9yCW%--^O()6|g z^r&zS{YU}%q<7We_rrfy@D#HQUq|gVt`n9{ClXH7V+40yvq_Eo7KJYoW}(*9;tj%& zN$E(qtq`7HVsMG0toW zyy_gfZ#~9dY$|xXA-=!AXlL4zLhC*>F^cHseiRZg`80Xa=EI?nqG`YbeU{}&P@lCu zn{PL-`F!EXm{zj%G=CL-8W)o{-VBW24F=09bX4tsA+1L_uvMiR-9*|<~5rn6Ch8BoV)1-B!6N0%87WU z*o=lA_;EC(cY@ejLh`%xQ$%5p10tqK9I0B?d)dIa5Y;-aPnOz#TiU&tX05`6^L}%ZsJ4U5+B;B$RkaY_J4OyKbjAw?PU9gzmVJuEIU*7LyG$BXa)V=3nSVY z{0#1VY>m1g`lEL3qSSZ$y#W$bhPmD|lP>4p<}_DNQ*Od_#h&Xn?&1gfih1{`V%0|- zRPhFGjmhwrmpoFH+*{=CZ&VrqVOQ&$avB&2c+tN*TV6horXH%c+#vQ$75nKVx$0xG z=L;hDYCsKQ&gf-y+G*uRIKpy5w6!G|V~pyfIz2nPl$}C^UQ@dbZkDXQ*2Po}AG5nJeo>D#~ z>yQnj1H5i>e`eKCLZve1*EObiv^7l(#{7oBVJY;Sxya{_Uay@}Brp#BZ*t^%&CG(Fh@bGse?|W5V;r0j& zBSl@{!evQbTCqV~D1IcdX~@!%2i$mbx0Ifd{@oY8y-tTdSjwYmD5!3{ys4=X^R)iF z@x={s@avv$f_a!;Hz;?2>31IS2p7nvLm+k6PaT7jVZ9mQS6eD`eJFpP*B(jUx_w@b z=&1X=pSu@snR4yaa3K*`z3dYpU8v-PYg)3KZf@weBme%v4U?d@SLN6vj(1~h*OxY( z>|NMnXQ=@_2f2+mgHzY;xof{E1pfm~>+o5>r(jY4!w00! z832rCz}&dpf=ELs13N2{-9>(E{i0)JEm>V2)T24F*$?o8Kc4$6TnX!oRf6%wGZ!Pu zrXowK$}&S~>ii=Zt$+2va!5lpHm-ib#9%t=ext=0{uvmi3%7dMCG6uiR_i{N#l@dr zPTjA;u1)*dPLnvtuYS3?Gq*3$-2(nT)@@|9Z(_|IOt${Sl$kHskQ{7@>kga1~vw5yM>5B15`ZcI~4 zwWwtTrLPlD$SG@CuyKu{#n%TcFqg3}$L}aETb5i?FGsH-_CC;OWO`OmD7;Qp0ckSt z$MN}G8d+5~a^j%rnF`Q!%3wWQ$8ePgUOiQZ;g!}}Q#xQ2+l1<7WU<)RQ}Gt)JvFzT zj3kW)Z0*N)d4=tT`d|etism!5p|+<&u==R6cCUV)2KZ#dK(!Mv1*HFF3Y2$4sik$( zHlVwgwQO+kqv-6g;4ix%aQo4}@@W1l)&*&j8T(AzYN_YpJLX0|z%x>#q@Quy|uNjw|((lIy6gWSyP zJ(l8KV+b6LmpVmYeVdIp^ujYvhVi~ff`Tk7-@UCGa2lxge9zcj6pW8CmT%~2%Zp`0 zM(mU}1VHV6TVXlMZ>%4D(3rGUIN-Ik*aFJmVVX$JkB=h!_Gz_0@b&`CO?Jn`{vz|a zsK5ETnz;B+OHxViQ+XalWHw=}Jv~{6x@%Cfm0#-@SFkV3fD!eX3)=VjPedTEw}Mh( z1XM0=t{!SYd}8qtI$YS`$S)&B0WW*WWReTlQ{n?NoucAukA18vaY2* z`QjZ^$!k>8)UgeFc4Js()eP3SB}2`GCTJsU`9}t|e-Y$)SIaAQpYu<39hbiHe5^K` z(aYj!zJ6i=eX=By=AI`+!hJjQ<;mYMYW4hgCdimCaa!F>Ifhy~rkPG`>Na6BY1Sw$ zA<1uuVC3y%K0|`lZ6;G?qK(OLE?vFV7p&Iqov1{yAwP@ni zCr$1Nw$BCU6uTn#9yG$29DN@-BMzY-2(>~gZfA5loXB54;bznkUx7Y1J26aYVS*D1 zfPYQh?mTZyukxS8Iy)D5Jri!sz3{v+ea%%HPO19wLil{%+zRS>5(PC3tKVb#{w^`O z567?GTRW7S5FRW)oQFgc&$L*-9l@N|jM}g$y874v+nj=BA;rW&nM3C2Ap%_A)}YS` z;6xI@tB^MpqA5K#%lND-2k7Nmy7vH|wglPW=Z_LV9Cv8H-WPTM<+Uf!%0B?qF9oh$4K}jx0LHkH(B=%YTH#ckkkB%IH1p-@FwsS4tyw z9wHx=xjuY&zBbekH$eIO+_8`Cb}KDAJww9qaittmz6UjZZ*>cf=Kbp5Mo)|GfK1uj z6>q7*G0V{7psoh^E;IVTwLUvFO+Xs>YhsmG_iBi6s*-)ygV-z*9?7l`1#oxtIboed zUGihOl{uWWHxBH#Mtod(;^D)#H(HRY?bXKmuYC_u;tAVDGsF5<{dYtH@Ny|y1ci`K*6av3eexpRxJAZp-MDTyH9A{E zF5x>9jaE%(wBMGiBMfX2$0uMG>{FYM<46tOccxD@pwQX@3K1yy{isZ`qWYxSZhxCA0j!@jrrJ|4Y+=6LX}#0@l=< zx!gs#7ydoTC;{h`4ac(32rs!_VkoBp_RY&i;s688{m8*pnsugWEd z8;wq#&gFy`Sx!BU`@#rmfDiW?JaNDPjTdN)H%WmD%zu_}57CUv{ycX>jE&Gy58+mQ z(kk4p;+wt|z8`p;+dFxd;r}4J<*$p1EcXNqTUcp(+Wwbcp;?Wx<{0DL&eKn1MYPnt2V0J1oBQzSC?B(M&S0=x9!%Q4KIGZ5yDAqlI{a8R9;I7uW&6tTu-8B;v385}i!0o^vX}Ol3z0a7f|V8s?T*1O(RK zxFf$oF;R)W=UvkdKm@FscX5+m=cjkgqAZ&+WXJpB39 ztS*UVrmcy$+UH>^%;s^v|HIIY=%i9|{swHjsr4s94Zx<=DM3OAut|Ygz+@@Bi%75> zpgb{(1K{i{U=y7mS$G5fr;kCf+UmEZ1X3Jjr09+8@MZcQ(p13r@?=^TlX5RpO^z4`M zw8Gy3s4)Yo`ImfNQZ`{pD!>8@fcmTdeXy>W~BLYnv8DqJr$ZAe|RLm zGca8!6EcUXjKUG#o4*kMg1mlk-v`9?_}#$n|NZsScCi&3?_)o-zC&gM-N~3l>Sw zY*x?V1`tB$Nygr<8fgC+@%ISi3h@HK+vd+UJb0?FXs|!8?%r|4$qD@vB)cZ!(fA06UIyfCaS>D@UDTMB4QvI7|<2 zq_F@mnv2jpq3c`rx-e`PB4eZTrA>{{BJLHnm1iPht^Dcpa2FN}c+?#DYZXnqB@6$8 z2D1B}tbs+cfY->_=J&T*OJBtQcSt;|2OeSU6vH{ek5NfRYGzR@*O>l1056oIX1!-F zZNyR<^j{|k2%#N+8?UVQCSAe#w4mh*fqpu$fFilM>P5!?@ZA+M<6x zjO0@8;4}it82BU)_K+}vd>3;Rh9VAD&+lb!G6(<*)DiZ1XN}Qtc`}$kmVGorL54yN>QV zdD`vWBwnjx??~$nC41>q_ikoo@P&Iyg_?GMN29D-8;qZ-CH~UMXv5#y6(&I{-BF zso&pz0sjwX!iq*3HMqVL>Qz`aDgLWQh0|Y%%!YgJ6OWp2?`SV6mG`&i_yFZbo#!^r z`(jdo^DN|()WYi+yj)}8m|%t|(ZfH1H>ppeAA`$%v)pHY-?o?R$b> z199bY9C#JDF7ujIYcs`l+9CO2q<$Fcs?Av4*&K7lzV3Q;qd4?sIZ5duejf1Yc?uVS zwGLo4e}TeCj59fSj5qtavIPxt50(K?tt_`W-P!0rqmkw-=C9uBuSk5PStya$;CoBcd>0n zPbrnOp_xA`Q$`DgvhZWAu&4L!ACi=V^Z_0tE9T898r%D;cs^05^cdb1*mHBJTvpY_ zFc@N#gp3d44V;jND(5`7T{zzan&+n1D1}!bG@ga`SMF`9^8O&TcT&3qa@Kkk^i!rT z6MLzp?wH1gHQ(jP&Xu#mesw+PJgjLfQdMKBfd+LmhjzvRIN1@*FqFvnjkS!q<;5H} z7E4W0vCxwcHs=VCsFq`%bg5~csPgJ=juqS88>Lq2xVdZ>nNp#8qpFWJoKyUH0VIS4 znW)XdS61@QY2yA{a|KuKDpV)V+5969^#VRw%RNSiY*M!XPC!2a5$&~7LTo@0N)|;s z_k{R+W)}KmQ=ynGBBVANEvaCgic5^wv0=TJTM1UVWmstQ5mW-cMK##i)yvhIpMyaB~ zcBW(j*8cFv74=2E)$(KhZ*N-Femx_YA^A(p%i-aMYY4hp<}PkQ0>JzDBxaYtcm7y% zwp{6#0N6n(Mdv@5vWDSFz;AE(J=K>x1n!2EQz78hvlN#sjC!5B57;<08Z zLA+z>qXp?2lYstY6f_}PcSWuGFBVJzY*yb7VdXnWph)vsS zQF-seCJOvpYW^qHieRM@PPHl}7?v=1|KreUmG5ri3jQ-FRifoCg5brf;pfMp{V&ou zuY5c}AeeP`2lL6g1GK^W!%0z4H2YD9Be>%GxAT-M>Ji{_7TN&C$0U{a95#fIDvgEYqnv zaMoAMSoJZN^qyZRoq$-g5Vi?S-R#%~R>6g@`0eI+i1X%G=b@gt+aCw~JS9$mJ$`#k z1UUGNBCdbZso;HpaDOdw_fx(A2p#m+CSMSQ$Z>Y?W*&gx3344}h(mG2Y~HS=*-q8A-G;e~M#29)FqF(7L4SbANW=pqHgPx$z|XC&>bXM)_%q>6 zk`ow7@bbXGkFW;rjB+L6&y>inpJ+hghjZDCGb~{!dOs8p?tX^6nR9Tj0$eH&p!GdMl~={?hI&q8g=m@msa6@6_IIk3nww%Iz9z-%(;L_epA*J6Q(o|vbr*oZv z3D^X^IfvU&epYxMYFlGd5-Wxc;q$dTI@@L|vwz*gBCTq6o(|kBc`}X2LDRC;ODDd= zYDLhBtPY)X_!PSsA%&Y$*&Ki{;6-pCWVW6XomXh@Di?A;%9MV%b?8jfynNisG&*j5|AX5E3;#7P$hX$Rn(AGd{K5Sm*4Flla;A2!uiZDhs6w@xr)ZIyul_g< zvErF4NvB-ikKCey^Y0nTR0HhYneDq4KGf29Gt--XI(biUU}RhQyrQBmwVq#l}tY6zpTUjU)5XzQnvZQQQ*3(dZ*llbcJB9cS(1C12h@RZ8BDK!>MP7_1W z_wDEt4_ITKM_6HxKWOMMmxpK8jPdA$hHb0EwI~VfEMzSI+N-;nl#fz1^1)W_t@PoW(lsCO*6N zZ?!uIgJx*1n_nB^Ae{Vg;MJS11E<)%N}# zhZko|)3QVzNC5>;#b-AH5;16oHO+iX`oT}`pPt>-*-Sfw?!_s+!oA{UMO zW4n6;BqQow5Dh}D3)QxZF`s+j_V|h9 zzV#bV;{iRSt^))D+CN#XR)W}MP@m`ZWe{u9+L0!M`cVG%&(bPOY*F0=Rc+}kizrD= z~`-=$FS-INgD;~5O?C-tG$yG$hH2Y@}M0eCt`NJwZz3_?ijMZI^=d17xSVb6zr4Zrn3-TKN-2)|XD@_TW&JK(@qk_iasFgoR-Amg)^4L) z5idS}R9IdTs_Vccrkxdw0S=Q}$3+XgnZYY?d!H|%&)sW5nR6uql&%%K^821!7Eazc&YxjbX0kXZ4{B6H`MU4!XDnBx2{iLTE;gi+pOh8 z#Z;|_lIVKRh2J6#N~jh?Bq#9=>j((~XG_x(3Y}nYFgAGcs%sG5=bz^U#OWav=1eI6 zV(ql0CLmDUPPRzJ&~w(9{#IUq#+!D%DL@@Mb$`o?%-X|PsTcu?<0?7SIYI+*uc#2HUsQ&FiYONddC!M)sdRo7kkZilEx29Jz8BYk`mRE63$p?|Yymull zam*6|Z4U;m0gt8kgV59RcB{RaZ8tqfDQtob4~i%8SI<*`oSMSek;3>gT6re6m8bSY zfS1;&%YNd!`7B}bg$NVp?+KcBzhqkxD=~ialPht>*$^$u6{o1=%UhNvIrJFJB`d2T z>qpZ}S}_ZcRLnpZgOp?0;W72)cc}Fb&h=H17yL+1E5hp0VnmMzE^COeW5Z`0pToQc zE56uZeuok`h$ld)Fal=`6Z6+lHMJiX%3s&t4<{G?apCDMut<}!!E$j z&`eL8C9Qq5FLAqyd!#`i&3@B{Yh>AY#a5^6sKD`K&|mjn{oz>5q}Y71*f@Gq)Q`gx9@tJfiGU}5|F3N1YpN~gE`y^HnWVPcxAwiu&P z=QLdrF=UnspG1ryM=?rQ=|baeYSykvLWCx>MKf>7_=3FZjFe-R_D04<81u4B#G2tS zs--Y##+c~s^lwm$v!~48l$!)XIl&`EK5K^03ut!$xaY0ncti9yh@3dFVS0YNeNf#S z`^nvbgj};Qy2P1o!Im|*HchdLvnziTqb(^FdV4t9nR=F!zW+%O8l2J9QcQW5OCn+qOEKJ1!e(>GDDCCY^eyj95H?c}hP-(2%5HQL_u9udXxO)alg&7@gx8a(9OAMUqk| zU)ds!{z*wN)+PqQ?W+1_h#GlR7Yxz%tlF4!%CU^TX0#rdR>7-wLPNMV0gvuV`NMPG zNE=s?B&e8 zbPq~oo_v!3F71uOZNXy_ou^A}Ud_$sP&T}jSs$X*)RG5Qb_-?#`K{77JEbphcLb6x z9BM+pC$f~FQ&`7P6h6n3>A&kxh74grXb1DQ`LDj&TqsLl`{hGt1uu$*i;TZ|ewE*w z-bt~rzAi*5vu9VAZW{mxGSXZXcK27Iu2bV>Y(>i z>etM+B3^W3CKwTyGfV!0q>`g~8ePDb4rt@+vK88U#0y`|#nmB4Tm>Si9;3AwA65H* zX~y>}vPE3=3}f0H8TlAucnepuRrqbms6BYFNGbL6?iRfMc^o?1&#iH^OHDDMigpJo zz7g#XylY8Nr=hDG?mg%k_uzx!y=C=6|3>lhP00oKv6wL9>hpeFn?PfqA?QwvXV?Wg zxnfo9XJ=-%!QUX+=F9}$y>1K1bkrDJXXI^6C!X`jJWg(BeNjiabtQTurUt!HIf9iB z>F5@#|kDVoTR5aYgw*B z!`L2gh)5*xyqT+;s1Z~%S9d6TVQ!Y9c(tHxT4g>sg%R6e&WN?=cvt$~j+R!tf7+%B zjlbMgo$04_R}%ulOO00$NsYGH-%1-Vp>k{0ffgbasy}q+olm^;G<%L+^f#sjj&D|+ z5EeUJKb4gw_qu;f)eZS^pEYUCJMCA0W7a=4YprSZOx9ladOGpdnY69`0{0ac42jQB zaXve5R=386YgcqG%$T|-OMyXtJ#OqoywyY9Wz^rF=Y=wGy(-60$O!(~RHLr6S1~`r$O8TR;~( z_XiGMGL70y%TS=jvoC&A^&p&iixC$?&qaQq-QQVnq-DkU=e8IpaL_MjY|@E%q+r~< z`CV#dKJTWi3=2<}2vy1?jfg$J8XqS^JFc^?{(Q4dY2>x6cspSk&n$jbHP_HX{6d*W zndcTiId+71Qt^v@)%fZHWgGf{P#L#*M%3rt@;yp9)7;_WvdsY{>+5ox&wk2C&=Scc zie$*`%*cwi)#UrVLieGTc|M>$vUK^wJ1zY}9C8a=v?we4{pqwh`l3No)lGY8tYNI> zl|fSWhlnMOx0(4Bcf8p7KO774pS_2R-Z48BorCD&?4Rltgz|SUE-bdZ*-lr^>Pgo1 zBRVPiC!|HtVl7Uf6qeWPx`e9UbSCBmjYGFgouIRSxs0jF7=4w&!)`W=sO8xrt9nS7Y}s!?gWUG>N!62f}2G`rRH zm98(ov2kKT_;j3GveGTvroQfZg$#v?_d)etlwRjesK~v{I|)}A%^#i{K9N2)4kwD` zRPFIf!5!yjZ=k((w+qi+t~8Zi?Vesenm|86F)5&x_$|{OYAFowb4;DtvD*Nj1JC7s zp0>P&kkbN{kRr<>>`MUSl2QbJ?vz6G4VyHdt)Q=`pjP}<-bjha=EX|v2Qf~?f+vpC~U_>IcZU1~Dyuo5MmzENl zw!O#)4qr87RTI&6;T375GPB6%HynAi3$Baa0-?4DGocwoC;BzF4)5}+9IZ>D3J`6u znVfZqxXVN9s<#6QY?><$FuZI~X-lwi36~vKuM5COy_U4-Z`jFI+Jk=;U4>V{PD*fY z-TwDmKe-7*;fg3kqJ^?{;-gI&4)h*9g^>cs>*B2NdJ=CgIlJP8=-OkkMdp`2Y2$^b z%w)162CTN#EPCsU;_7<@*L7lfJ0c_c>r|j7_BX4gUU&T!`~3G}yjvRL1W}@Z8c&Am zMrMJ^GPgeC{YAiL_KO!8^-ydOCR3{lo@~s~aI(s^P0VYh7;7$8*fmQTFG_~m-pO^M ziL1#4v|Zmvppk2WFJS5RR6A8z{`AgBqBOjS(Qo?CM6Hp`hbP`qRo@xw9Qd51P>?eZ6d+oxtf8jwg2O3OxnNqn6k-rB>{ zEQhZjtCYr1OmP7Dlm%gm(3}2Kr%th+|2vvw60esQy=mtBBJmib*)cJG?{(9!s&Ukl z%a~&8=q#+(`Sz_|*A6K7u_iKnNiKArU`w?J>f}{)8)ncEQ8;)}CweCf)LAvze5GIR zmDm_LQ?N}68d#V*$Vb;Hv;AWn@@B~bUllt$y?RuEXbKXY&3IdrEp#aDadg!iF|(3tdKUXI^7iK*6y&U& zI44PJP@oX7+Iz9dJhrL2EZFg&W+9x~C72YICY}l8cbJM-3Tp~inQ9NO{JS?xbvWaz zj&m2kV45Tq=Y7}~K{6aQclIxu#eqMnYjB0m!VMp92H`xGr2riW6qP%1>!hg;3TRhO zT;b2c3D=vdi1<)-U9jzuVP})I5wq(n?j=%Y+lnWtk%v$(Wf;}1NjSTIQ$VCu6eB)u z;b6MqI8ki?&vQh%l{AQ_AyAReUilR+>d>RBmZ7I5l7=c<1(I29RNHBP9;=r7Ar_5w zFqV(@t+JroX&4-ghcuP3m+jA}CYJgZp_D3CBr~pwHLip#=8U?iXX!G)_`?Qs65_g9EvXfjBu z3u;Yk8vMu+YFil|f8`&~Lk25V(=zM+Frx^o1Ypm|RYpjp52+!FD)w&I@pwtno!irr zx`5+nXI=LWd2U#5Gzb%)RN9hIq}Q-5YcrkSHf1e6(@0$N=t-@j-71x-yqMB5AKP`@ z^W241+)(IaOMg@51+(JAl$`iFalfK^Qb?T; z^T8BbG~?(CP7Xc`ZfpyZcI{g6*tnDs`P&1v6Wd{d<^2qtA8W3s7SGy5lA~JHTbudE zw-&eby$|+aiE3Kr_xm-yu-O7`>!*UBTpCTWKn{5GpK&V~Z3acSf?8ujJg!7$r# zPS4=JD_*&Mwn80y1C6T7VV;1ENf);%@B$YLTk{tki&YhyDb=2;`DIsI8jbB7W_Of( zr#ulEBN`Dfht%F1HMW8YHg5?})w@e*{m`~R!|e=+5I}pAA-sA|b#ecz`5XVuN@3HP zlcJfcVi=u*2EDm7#ie4oKMulB-a)nH=uctjb#n}r4wN~CiE&^Y)@B zV4lo&@LG26IP$`sC?4Z*KJ(ILZyAjM$$(0f zN`plXW3W=u5`C5iUszE23itB*=76_D80w*eC7B*yIp``Xlt_}hYqwDe;6nJesY>J* z{|G+9B4*WRv2rlk(0`ySk>qf=Lh;Ik+3VAn?R-Kw1>hP2!ZtbhU4lzlY|ty>y7J>p zA{j4{YWcy%_Y&R%QoJby5+qe@1cWA(T=Ft3G&_VdbBhvo?tutfuX7=X_I}O$;*02H z6ct?X)(GQ#crzyX+Y07b*_^!3UMrfT<)*s!IJEY+0a!+YNQ-onF!5{|e+pqYpYuS_ z_}c|-D*}QF6$n9}e|#Z|Fx)pdRSgLP+sru*-mT~pPvXKdAOA7s3*1 z`|I~w1Sdz^7+}eAEw;Ehi5jK=t!CNhik>=v>at~y7Q9o%=EgQ@JeBz>rGbQ?L-q0O zF@%o{S$e+*u~u?v!`eDmFPz>e9!{~Y1=yF=K41-o$O0J~nHPYkkE#dle>FMZdcm>C z&%zOKisvVQp2Pr^@ z2>{7h(|+|(Uq5)}&WBW}2TaUWPCNow0@xsoOp?Z8fax=F9&i|r`fE$+g00N?s_5Z>OmX%z3cnq6y*a;q^V=j*utbtE?y_ zML;Nt`#Us}W>K|2M~OR1%r^?yRUZLd7g#z&w;S5;4lZj(fN@XbXB!?kdGH8O@I1LQ z2k8n}!)>KvU4$D3c-6CfHt`GH>Hbe=-yKhN|NehU<1SQ2lt?IhR90q#W5gjdTaxYM zmX%FIMo3mR$2#^dAu6QEJa*;Wkv&cvBlG;Ox9-olzmM-9-^b5i9-Q-jzveZc*YnER zBX$&)8|51Qd%R3ULtN&*XG&kJtl^YudWUkQwjd4byncOu5@vg$=<-Bn&r{P{_wdUE zb)9SCzAkiP^E#d7;rsJ(Vgh^TLj{U?Q*UzTW4~6HUGjh$+!mcrS)ulelJ)`(RE5(2 zx(z5+xH6uVre??bxVw8bLM&HhhX7R|N|4Sv~nPtuFS%~_C zKWVXT7gN(wTr*0n60kk4jTR5nCb(h8-5}IKg$h`JYyVep_!((URE1q_AOhOmeq}Z{UFRKO9u3WXu5!GOhGLvOLWNpE#16`o>y=O> z{=h??S2s|F7xncazpSI2l9~5>l?E&J6JSQiTd*5$RY<~4P?rt*@nu5HnA=Pu zI^N97F8Rp(h~Wo7UVa$3|E0`Qxr`|(P1gEV1H68UiLyzM1Y6g(5n&D}so2@$dEJ3O z#TuQWYJxN=44bsO*<~VVgjt1ro@O#35xY}6Lumj6x$R^}3Yum9D*$-HW}%Sa*C+*m z{G=wAr_fNb>{5eOTikvALv50;No*Fh>1_W!04umrFCt^Xx}EtUJ$kVi@*|@=KxD<2 zn*ZpLI68kip(4jQE@QUJGr%_&Dg`1she%m@IYkPQ+_ISi)Ou*~M(JPH!49al{ktMq>8|lg+xPaj2Pwt z0eh1A_Qosz#GskS;vu%zpGcrfEv=ALYj~PPl91y_ou>~{+R?0Z*!@k9d91XeINf9M zlRJCU{f_kog^zr>sJ`NC%tX+(Ms+#qM!3v)!&p=c>6DcZ-J%y@#)==-J6&Y5+BbP$ zxlUyR($cewjI9?5y*JRVL(*1!++|og#xiT2OIFYwkw3?sUwZzjXa_I!9n+Aqa)sE; z@yX0!8(iYJNKwiJ;T5p{hRnK@`9$}iuD_d9@D`S*N{2NcMibj!g_KRunD6Z}g|k3^ z)qrY{U4S9jzAq%D1icOoghHp&6P2bj2bN!PHs1+V(P@gu4OKQyz|Gf5jxPeK;@U_n zc?Wyugod<0F5*Yu+3328c@ZOpm%O1y2YEC>aL`n!3j&_Q+{7~`#mFeuyD z-|Qjs<=L$SK?Pj~0#54K0HWH9OzS%5IlV}33H6pKOlfJBagocCDA%Lr8agwZuW^Y| zGL%~Zh_8|FLJ+@~gHkw~D=0`6E9CGO7<#|Ftd5tt_0I&$x4m1!T&KbxmN!o{oIH$L zx~l8IOmfMP5@i1_gQ__M4m5XJhS*VaZd8!c7g{;RD!(NFuxU7^{@Hc)U^s$_YpuPn zZ#9HVynr#G3Xp|W26uo>kiDv}e@5C%gx}ICr*2p;%x*q&+8YIrJEt3%#l$MWKrW`5 z;=)QUe;zQ9nl0!ot6US`mKxl!gVKm)+s;(E<^$9l4gGEoECp>z__Q~~pGprI&!p5> z<^>7o(~fU0?{8DaR#nw>h*7}a0w3c>g04k-{QhJPn(&w8)fTHaD`_y<`1(3ZhQ&I*=%xNZna@eCcdhCJ8WmGXPiz-u zG%J3M)bX+7R@6s{Pe!_7ZYWlMF~0pDTOaS3O6Yg5d`fRH*m0Wm=$$kgnyKiV_TF&m zTF3(7+Y88O>`?hRvl`dJBWZg6B2*-qI`Nbv0rlN?yzS{eOD4F>r_n7q9ySf7c;4wDYgBg;1BQC_(m z1+$C?ldEcQT4xqNM4B8ha0q>=D4tm|YUDV$*1Vyp?oI4z5G*Y2Z7@}8h!-K4YC7@z zdQueLPoA3zy!Sc&xkysuq#Eca{C*;buz-r&rKZ6v#!Wa|-M&$mCysjluQCX$n`QUD zlP%fHmS>52qlk!>gCdShAj-RtYX9gZT0BG6gu8g!zq~5zR5QTgm8ekqbmn>SE-tC|1`?p@3;OmC>XkjI=lDEMZ z@2<6fzic{O`Xa*6+SO2(VO3tpDQVi^IMW+**XL^J7$!NXC4?u{;Hw81qb)jybFgE{@(IjSh&{Tv2s8mr-(n->Y=T$rvh5TdA@*|v zMC2_gPu2m&}ql`=^ncQomy$ z0Tkw^sr8W_(|$T);^W$rmBNWRfjLW{+9OFo`lu=F{-d`Zz8c~j?DRv_jX$i zlAoB65+m10gpqG`qEYhZD9mN@uyK+X9V^?)fHLaQt5R$H^)a!E+kg}f|FddYk zcgD{t1pJwXkl}wwBZEm@Tf6EE2b5_Naco+2M9c8$mJw3isjU;>^6)Kw_>sEh$(sHfIvmyGw@5y`ZeMVC^{gLP!5x1arQN-Gja$^_4n?z> zNQeue@U)>?8bl9bgMZc74n zho6SeKwHy3X#0CZED^11KQHm-AcOZCM7xbhAK&UZTi=ludk{hU(&KbQ4{EuOv_29M z-~&6R;xJMy5Sc5jwm_=1uT0^AgEDZY=jKl!pe&dYb;M($O%7igSPVvxj& zydcGG8<4x0N#JpOG1cXL^{PW|$o!XAnsp6^+}%P454pIPG`+S1FH4BkX_~;Of$m42 zJvK~ky$@DcmxSFg-Yo&=1;tmZCI-1&j9pSyTxHR>HH1?ML^B$d3=(d57mDh?AdHdvD78(b zY@1)8&->7GnX=w_o3RXn6cu(dnwO2kF;_pU?jsys^gGm6IlR2h2qQLeJ3zS z!_%tV=Z9}6&LG__O}G$VjKzx9Ux=+rrl&8soz|f8rog+s&|lly?1fs{uZZE77tL)3 z#oEeg_W;wR|MAHf5qR6=eJ?xS%Z9^AcpkBKogK}+ss2v<40%yWdmF?+4ZNifW;?!3 zOn!0M+CTTwCsr*?gkS&rXqKWI8N3i0g! zGs=$fdJ0d2K(t{;;$1i#Wgf%&3P-hQZ%K3?jqS6k-}+3Z&a*OOdse>kn8r<3tWS-0 z%W$ACe0`r=dfdB<2Rv+bNSPuKZY*N;7U6#)rF9_{%onGvr|6#!1$=wk`_;*(jw%LP zf$ux!(jOOVTTIamQ}hPm7Wy|#@Tu=aotMWS!1?I<`BCNlNK=OGBxTPIp8)evrP<}7 zZSJo-<0AfiinpAB$Tp%=0N0+`DGyb6JUw}SaZAdKKIdAe4Tr_9>^VT+R;R-H*m@B{ zHrR&xr#RlRoD+E}XIDEf>ZGX>ign6}Hi&%7nn0vm`d+J)L|d|oP$2@QY#E3RkAv7+ z)ur6SM#g-S&CsMg(l)udhJ| zo|`m3lOf>8;K0(VO~w9GE&9igs}PvE5mkO9{FPzHwJaumV!A(65sXgb_71#;=4=9~ zM*K$0gr4kHpCQ|nBT}`439q4j{b;u6RZx!hJElxvM|;Msa~2SNq!RTfvt!ng4H%nk z?eB^Sww^pL)WhgE(+T`PY?=e_VZm&5xbmyoaS$P#j=osc-K!scC7tbL?tJNVnfT{B zNSUN;O1S)&+)1!G*h>~A+RuJI0kBOKXZ6uX8PnN*i7ES{@}oT8&jc8*+$9?5s*tl* zqaOJ@FD5t@)eLE>`avKg%t0_z>wVD**Vcw#=oLcHDfu^0y0d-XXaXq%0pkVZY_fi> zdysx?&ZUp}HK?aBpDh3RlkyY?0!n7#F>FfLzPHjh{7k`}|3zkm;5aB7DbE`K?-Eoo ze8QcLnqbs7HWBY=ANV6VqTpH@lSE-hM`7Nj(7Vxu1b2#<*s$gFt7cQKnVdttCmK>buVGRCm3Kl}gLRAqPGyvr~jYTi>A|)Xa^kWKO+A#oO`O z6zGl_n(L6e@Q!V`H`pGl^8}w`%Ze2~*7i67C(~wabb>zj?4P`cZKJBPWkVG!I$x!5 zc+3rI^_qv;$3AlgD%Ehp+`%GtAQ;~EvlMyD>!o__FHRb?ZW2${koV{9R4L!H4b*r) znJ{e`2==uMIDV~m=ESr*(ANoS&p`~3|B<*1d+s6XL+<^|QM@_+vHlPW`SlT#;1hG> z^cvlrrQAf^99eIn_!S@T`NpMcO`DUyvQ_Ubr6W#VNcoX=K_810k9q8?Z~7+83S-A7 zW~plx1>FS2aK%ayN(*jg8gE7N3!7z++f67WTFnT4EK?L4#y4B`ohVz`<9J8=fw>~r zS*EL#SRh1L?76eWSux^A*(64SS~@P07K(`fxi=FM0W**wxpe_2(nHxUGg!mOoPG_h z8p38B^PgsXq{qttQU*-?oRx>%&+cI@cx&8YC+Lq9wpCLDER?XM15 zV#DF(Z2v&K1bda79(R>{ta&v5;iDLi?`l}xD)U?mATCQgKzBCA3CuJfbuBge(X$<& z^V_t`IGCVI+pHLSaO)c- za|S2<_#S+#^@xl#-WQ;>gP!>d_UfFwUFpYMOv)`ZP&XR5z%^sm;$i6?*p`@AW@3J( ze^AwdyP4z%Q~(&0c!h5dWyn4h{;v3){Wv4}Q0eGrrdBCc+@<0tbFnI=>T@Z%;C0|W zzrxE3@@IM8+fUn|`dDz`^Gxo-2Cu??`l%NiVtxcF;!$n+emN$=Gw<#LCbOC7zd=X~ zb@-7cNRL@p6%CjpGk!cn%x(aeWNCDRvNd`6aXJqr&rj!gj2Xru?hjdNW1AavJ{6!r zpC=sz6EH5o&DNSuMxA^<<2r%w@ncDcSK^Hmia=pu%)HZQ; zm_z8eA&U65+@F5v7Q5V&F0%IogIFV_^`(m=o`?7ze*J24aBrGhI{uFaJlM%Ep7dFf z{nGTBN32X~Gflv+2a9uYMEy<@Ph~s(K;e<6l(e83k5kXO7d)L7Eyao=D8^ioF`TkE>9k2SvoF%>&#tF3*WTrrxBR- z-Zx%BAyyX;xg=hj@r{g7Jxf+QiWjjOm@=omatI^Re<8#sG_IGpgm&XdbFk|Qh}?Lx z6ibg9A3IykXWv`ASeqFaXr6dh@+erGm5Rmq)yc~gPZQ6A@=EV*_jU=kUUe>BzaMCS zhZsfww!j!nRaw#dsx!H<{+P|>S(Ez~8f;vROdjIV>66N<+qe?9C8&d7>eMv6#038T zs%G= z-BYb=36J}mmb6~>yQ_a=E7SNpVpw07UC)JSuJo3Nh6W)nGo+7_u|=dpzo-!Vi)hi7 zapxZnKS+X{MpS~%QmFNe6d8|*v;x2`_AvuMdh?g`ArC8ugZ$MTHsQ0tf#Eqywi!YK z60f(pKzo3il;gOmTOrh9)qkOMS|SK*dqyX>h5rTy(|sY_%)n585>GRKER+1QE&Q{l z!(v!N;u|o%nx&}2^~c#V56ASQncY(!qZd4*%4ig;h6YKo!@+r{V6~?4BTH6XeXgB# zo3K=uo0V4TL(Kh(;$q&V0QrWsk6GIxMX}fq9bc|I^9fFy7=0|2Fke4Sip?A2O8b#{ zB46WG0=na`BPsE|Ap0)pYPD~E<;a}kWVr6QjYbPiQ^>>NZEu&fjIl+20qEc(UL3fQbAsu`f?l9xy z8xs7{4|2lt3F4g`2FiCwaNURDipJJmhl_6cnZ&~A-gj)$ zBAZic6tI?wO{OQsg*s-QARkodkN~m9ajl|RJo}GLEvuY8XN+ZP?ZD&`#ORVo zSx|!TPfRUgP>#N16h~nazm@st2gE{S9jKu@9JuO2sWQ7_!aFR@^Kv4*n4H3Ze*HH`d6*_cSh)JH2hJ>&@_~;ohhNJJ%z7-pVu~GLAF31ovTI|M6)Dkxb2W-* z_3XyR8A_=YF<`xN6Lv_dC%-LbDd19iu$|Boox3ODL=LYc*}`a+fSYu*&x@F* z)M&_ScM|@Jz!mD`_9-I%6S@2-#6%2X*tT-FD5qLSAFN_5(un%T1+xHP68)iBxs$LV zW+Z(j@KC6MfbsKORhTVUOv{g;Wp4i;rQVevj1C3)g zP+Yc50ycSWqtMAuUW@c0FNW`-%(dChLD3&+J_+%5^BS}xzaGbJqn;h`WyA_)v>ouR zJRM#%o+Camg9DQab`Oa!=>dz!OaX(&C5tVYVXp{`0_v+JN<(ZqA+9|~V` z(6xl*z+$g@tJ2^KO%d~X2UkTMx5_<%O53@nHhYJW&~I5>BIgMS?69t4y`9jWS;ui{ zRkxVRw(y)JdtGIRqj-hvh#1a!#`bjdmEH2(FSKtm+)Yq5*EKS`OI0pnuN+fl`x-^k z@;yC3>`gqsB>D1A0MpLeq?HlR2j!F@VZTu8ionweISFwb=5ZAtY(UStnyn)783?Sq zadCWgw{xWW9oIhsp{l>)xnY^64(Xw*iZi3Uzf2c>W~Ae>C>27ENP~{?-)|3%txA^G zlCih%I?FR-5JS75;;6&;5?7m3^d^s-cseuUXJw}@V`XXyU)IRM6hI_ohM$sTdZ@ti zY-ut_NZ3>;=p>~t6Y_G}* z=N~;&Of?`TcE*O_uHb}cV-6^OPOzP$HX0+x=x8M1)o`k;1vMXrab2E62O-|u#chAE zP5H)uPYXB(YiMXc4jH660-a&pfz|_Wmp<8vt)+41CU+)(K2wD4bw-%| zhWv(i;p)C_YJ8 zY`L!U-D@gAW&sDR#=03mWLCT5JIFU6-9b#xpMC>yD^fjvI^jmd$WiQywU!^N(YKd> ztjcSXIlzZi`@y7-Kghb`D5&}J(+wG6lxb5d3tQa;KtP|Y|B$bv_F=dp+NJMwTCqy? z-sUu@CZQ-}gN)OE_Ewb14spAhpb{HfS#U-!Fm(jT!&x8kl+J0VT1yrhb-*Ky;(p&6=NH0?ma&Jq!0ZgT2Nd4HkAMZp0iaMrzh}Oz+IzAbA5?;klOOjE(A&jQl<%%K_jGhM4AiYV*XL+y{_dv{ z^c||}tS>!t&{Ps+q2H~Q>`71h?|(s32tmxk=e`|8X7cx=kc_utl^7Brzi-O{N47?@ z2zp4|(fSF4qejLG0aBw{8q97byb@B=xtIc>IiRr=kF}x7ifI@%flSl7GzBi!S!8;)=~N& zD?NN{w~JjjwBJ?+G8d6Y=eH@)sE6Gb`Y%2XaDx<9WIX_3?td{sI{A&(CPU0Q6XGo5 zsU5e0>?PBo__Z}_0we-oh5K(x5`9EXQUFbLW$hDR=Kt7kY_L7V&zls9<)1aA#$)b( zQAhC3qxBDuNaK%9F@1+TBWrGB>ba-|K(+}+q`37Pevk0kT`!H3=-p$syW{j=px7Dd z>imc&-P908hm8LTu`H%FzE&(9L=42GA#>a^{d4a(FtjA)cYuzlx)x9eR8#U&R*cBD z9EL}t(m@vtXWMDbLjaAfK>@QEW0#-J*$zrwY-5v1kw-8UqgYX-sUo0*{3jA!m+@?}B^9u&x|0_5YL_u+)^f+ubN9%k)Izj0V6@G6Xr z=C!H>6GW?KCpiZzHNVVVVM13y|Els!#kMce?t5W|Q}X(|+jHjRzbBe!th))JcqOD` z&J-kCg>gNc^!3Z0(C^SvRxbf|s^LC@9%0b5vP|M0B(b8!57`;L!c4xzevxn%K8S3( zB6a6euUiLu9=c^~ z@O#7V{Aa0!wdPv|N?03CVbYevwi{qon#KKYw%h5SV?&Y^zx>ce7zKwM$bee7B5AY% z{&3C9+pcwv%fPW3nRfF7V7`o#3BoygCvW$1z3nQ!7>eBFPydo(RcdS`3S|6FHpIgx zy_bQjC<5u(R`zElT%S>COtvhr_pc<*^WPwpm0 zV$*q|gmAQUVsgFmjR^c7?FE35?opPEaC(G4BDA2xfyGu2IhK*JTF5@B01h0)R@M=% zTs{yDGxsUEDegv#G>42s4~ULHSNPkO59|nna`fMiK-zFr3X0a%fEooIKXicVdJ81f zWSz}8Q@7ob60oTxr+U7A&FE+D9a58wEm-ME$Sj#`nMXNV4Iarss0w9c!Qj8LR`!qS zzaUO4uv-S#=N-{%-^UrE7nZ)clH4X-E0}uy{cg=a19|HMmGi?#h&$VqISmIzZBds_ikm~dGs;#k z7NTHITc`Mz<^ekBC}%+K#+8V>*9>0(2sC?500~3@O5c z;seOav1#P6z?yc))c^4heYLx@;IAbe%dNCdln@>Sp*97mVOH~3kV#z>A3p911mVqh zQ(!5#rX(BIA@JLYkSl&AkfUs^(=sRDK+bloeSZ$pvt7EL0$?+~2s;HT1%eu&H@+^e z&`)c7um~ZlebeA1W;$&5OkHIQpOr@RqycTssE0PBi929(q>!xmukHJU;X4p*_ATQG zn6ERm4n+};WVofPhNt>`Nr^-IEt+!PEO7^tHuw;X-)$RhbpJ{E{pXqhi|Sly z)0w)sn15d@-2KQENf6J5xZxYDf1prvnm479I5n!^>dh^U!@>6JD(OAFpDv2|-izXk z30s1&{&AxBaL~TCeY*-aX7|+dPF74X+BGzMhAh9Q+unSxSj4E?9zXvdri-4M1}v}wG+3~Z1ohYAiwIdC1>To-ew+Z zs=Bg1h`%S067HV^`qi)4cBij}1%SlPm4~&%GuyuWZi05eo82|%pB6hpG9Q|Lf zL8PMo!{T2lj6s7uM&erST{ZSVJ!vod^#phR3I~y@kYCD|4v{VYTxECzPVaah5Lh9G z&N3Q#PWnHV_c6z62x6NkR%QI)3gGzfx&7m%GlWfc`rqpg}Jqd*uk5yod-1ns``6;Ppb*Gx=}1;hOCN%!#N{7T0D{7ILpj zS^n3xg4!cElxK4PzI(8l$ZHD^m@djdwh;O4-x=!}(O!|dLddgk~LVmrb~)q-D8w>+Dcf-FXUcq6Ct{N=n# zDp-#^Q(e)64=hdu(Gq(<*(JT;rfEFySlz$BAsRS5V4Syh_#Q(~4JA>p-?*(9qISs6sr^$3px zsPm46DN)EMV4s3`nQm3kqI$0CKFkSmlL!*}a!UkR2joZmHZBoO_hAa{nAVKfrfDpd z7VH;qySQ1mdvuUbO#CHdbRx)N@TU-zULb!!1?(G5^f=L9**xuKu!cRo=v62eX7%s? zA|FskPQ~9p7m-u*_s>U=V*mR648z}#gFpP=z7S108WwH;jZ2*78}blUw1#5wwObGV EAF_}y761SM literal 0 HcmV?d00001 diff --git a/docs/zh/07-develop/img_7.png b/docs/zh/07-develop/img_7.png new file mode 100644 index 0000000000000000000000000000000000000000..0ddb005d6639364bd0ae789c7bcdaca5197240aa GIT binary patch literal 15038 zcmb7rbyQU0*RF~pNQZ#bz)&jPGL*oO5@OH-15#4b4IxK@JED) z3$FAt#^1Ph4Hl#fmwDlCxRE|ld{_Rk)6cKFZm)}iSK`C=V_8`x8UZ<3Awp&vJopWA z!iVq|@iGjQ2@#}E;XIl*-&4QnV7{MiS(nWO4V`@PbQAB!MQ>jEpQ2ZJUR%>ioArY$ zTO40MmjC>=^>Rk9@{P;UmAot-vuENJ0W}_T;4%X%S{4u0s+1E%niYanB4CD-upp^q zLm3sI+<~N$k#IfGm?o@(RCb?_+zNC+{GT1}Ed1?`LzN&}vs~7OgLI*{K{MQIiZP5) zJg;ggc{cH@Y{tkh7I^)_6e4NJ%){uxT+2%~6II+)%`ySF^1VsCF~szbNT=#N1i6NV z8GZIq$H&3M47C5%l<DlLp>l{DbFN+J+u%FuzOh8kMha>0MQ`Jiuc1oqSIgk}Ug@ z`F#S%^FkoLk6=!Vsx@yO5JIFR#E4N6|JJkk>~SHlK@F+TcJu90J}>{a*o))&nAVpB z3ms8(-IoiSmPFtoao(i}T6D-BpYIKZ=%QAJ^0LRuEjUb0R`N1~!+ML=s;G)2GyawecyLmkM!wHpU-*8VK9WPT7m1N{ARy!XpZW2L9_Nwf) zvVqj!>KWe%Vn&J4ipQ96exzZArDpOJCEn%X#zb}kr?!IE;dtUsDw-o~^lU$#e3n`TNe%+59kSSzp|98OvN zvk55^#Rug$c6kCSuF%b&TMd@y$2-DvvU5kD%THbi*GQl{VZQXRL0-POsU-zk^HmCezB_?EL5 zCnGS|1wGf~)2FfyDUTdXDs0BemfON8c3{&){I2VxIwfHi86viNW#^EqW5s^g?FTHb zhjdrfY6`wr7e_ldoDJuJ_!LGI$_J$){7i+ziOM@eY6Gkkz-EEv?DU-ql?5xlEjl}2Jx)hjP6XjNluklrDf9R+pW*OTg1Ti zLVbyEn7)5M3l+iRDT*n#7z}PV^|o=LIVknJx?C>pWGo=-xXY;pueO^~*)2uuMw@wT zj8|Iz&e!ZdB$Z_6cA(fEsq*i=S(0x%Mk0Ma>&DP3_MubET_*2&$!?9A&SA0N1*dMz z-uC-GhvwBk1#zagU=Jt0MA1Y-{Lc9g*QEFQ1eNchU%5>syg5kn|Fo(xAQdKrz87?O~2e?gpa9l>|v)z;~O;~c+O)~U+ za0!W^zN+$XOuq2Mw6&YAIdVC(>g-h!ijs~>(aNXs-FduzA@=LK$Xb#;^}l^7l(_jf zh}saZTtKFz?ro6qK`LKv7`fKBlb>5ZmNJfc8!Q8ETlPV#ge(R=J|f91#a~(IO^S0V z39RS6XLLrPAwhN}?Xvni$ADJq_n=~#PRmtP4E@{El0L3Ag<17G?Ws-&orbL7l{^(Q z=jJi2d(_wSdUNten0v1Oa?0Af?#DPuo!})&Tc?=WMCU%4zoOF~d=0r-7tZ zV%_a0(+2EKZ7d8?3?vP!0vU_(N(89E(mbM+{oV-OwBy)vz8$tGdNG08i^Bf8*?;Og>;|ezmV-Fza8pY)mc zlJA)mBIQ0iA$3W zGi`Nv!Da&)>6x%^WT-zF*59FsM2wmsu|WpsPNAm)m6TT6(}s%V^~-nvbA0(vKytn* zMLKK5@DLV&osVqYpA4l799?KM6OsoPoZKuHrJM*E5VMx?$XBqIW`@0o*V6pACboEu za-TeO8<)9&W#AK_gB<8!Nf+kqfL)p`3zCNzJCrRex}6yW!3s4$Dl4fnai7!uW4YS; zQ0&Yi7cx2a({=ZF!ZB-$_1^3MM%q3U4=mE2d4DwJHX|<=LL3!z+>^j5?2+knsO!Dn z8cbB6JI(oTilH@V6lW`ARdXd~?5CT3tf<@OM$rmC)NAneP#d#j`Zs-X zgb)ZbEL6a^-&Q1%IyM&aJwG0;74cVp0(=v(=n#2lG(&{z)-*%9ln<=$&3{%XAG-ft zzq&>KtM}!}63VuEiVFBBPCXES&aXho$i}B&htrALMcPf)Xg>}M75PU8kUuElMd`x>1NOX+U7`qo&~ff7llvT~AR`l4&R63r+HBQv3B(=9ib z*RQe(8~(1J#HGlZA_3noHmuhxXW-J;7p(}D%kVv2$>E}idj3UK+1Gx&;sMXRI6?GQ zD~YttxaZ-2HoywmvHIGJf0o z6V#W!fpqaHt;;63Fy-693`wt8N9&OywM30np$CSpAR2WxUa+!-nTq?KyFY9AcOI)& zDwNW0)p_hKj@}SCx`CuAv7M-b-y~sxQU0Vm&g^dH8l^JRDl)id&Nfl&ed3sWH-OEC z;kH%R2WAw&QQ5$5?7elzL!SYj$yO*_c_HL*KXCbueAqrmrZ>4pL&`61gPM|#XTk`f z>uTk(MEc+{tq@ve^@0`yti7S@j{{3P#E5rOZo)s-AIFyb9(v@yZ44>RkoNO6xA+kJ zUxsCbA-t^KTXyTWEw*;(4ZOck+QU@Ae4NzH@(ky#mvOzmPzMN4^XHuqKZya3(||4{ z7R0?b_w*EE^V83FnzF^9F8mP4S5=M>R3 zc3iB*nt#Lpy@XM^4e8`oSCmX-cWdaWxb-LrX6D)>&f54WDAdM;{K}g}@P5j@lY6Uu zto|wtul+<--+L zCAe=%Cbgma=S3N+%rENcv=ApotgESdAe+-0T>Up?;Un(wUie+kQ@kP#Rh5iK4p80O z5t++zAhd~lu6?i0@;N&^AiGldcP0f%;V|A%E)DoDo*2o%Y;i}Hl-s6(P6cpjz$-mx zL-Nr5H@6f8uYL4yHlz}D$iE#WUzx+Yym9szrM*(Ue~=e0M~d-;0w*VaG3uUxy}SDU z5lac41J(a-OlzX?+e~*T3O+te-H+Wjp*HxV-qa!vbXnHKKgX!VlB6h!6ybz%YY1WK zK$pB^l47D#+FfpK#Wn2Oe&jFhTC$=sjz|btAe8#RfKe9;Zl3+S$c{T2ko@>flSHKus%lFtO3ZptCu|#t9bgGN zKq|YlnLBbg5MhDkDd3k!GM^VR@$S?;tTDdebCA~a_5S%!SV-I+uE@L}Uyxmnor)Xt z-O663H+=J-oo%;8_CgQj27DW&IJGYT;9_|Ht&i`Tnx2&}uxDG{9;p@n6 zut&tb>`})E_omg#xD#T2jGnk7iDQJVCH1v`D?(o}9XzzC4>Uu+xf7D1@<$CZrSt?b zMX-jb>93$nBne( zEx4avL{Sb0XvG$1R=v6$WXbaAaj#3u8FMKJ1ViNSPvyVUIvMlJ<1hGXShiQy@k(KL zq*gjdtBW%+{oRE-lIKQaOxZwzG!)76!T&zmgO9T?zau?Z_^|B$+3C12cL9%bj@Rul zXMYg=tR}7Er$&1>DSH~en1xojeT{dSIb|VVxdG1AP|eW&pQTT7CORW3x)i`8zNB6e zI5N76 zzzZK#r5P|7+ z`$Y3Y5gnRu!(6ij0ZQiLG_$c-~j@o3THp7w@PkJGkh;Df4)hEuR z84A#gg(M?|L5V%pZ`K_F$9|_Pc@7q&(Zp~m$B2Ev!LtDhg*RTFS|=wwzGd$`1S3K+ zXr)Yp<$zrmVnOQ?72e$yvTV{*NWLFoTxwr_?w#c?EH{18N@rh(To373Z{97hQy`Lv zQ8KFctYjBSf^)oMvuemzh0HH;g_yN-3p`+eve1EvGOM1HrNBI_Bh)vTW z={*DHv%`%Di{HJ_VkHZ(Iaa?vN$S`Uo=D<_>z3ra&7TgfJIiYxmlClZr!-fz9xe>q zV^{x3V}9@%sbu=tz1IUUDq*|$2x{Jrq6`qNIn0y*UUbzzq3G3^fI!XCjogX8#<<$X z(2G3-n1}C4T9|zHv~aHN$;An5|Btp2Tdx&_wa|DVAUNjs{dzn@fZ&D-G?2s8kR?obViqZ7<%-fSgS{l%szbIIUD=40BgOjWkziF2rI#n!B# zUrLWY8RA$ccoM+~HgAo%EdTl}JQ8s>ZPS@zaV2DD-4`RVtCY-Oy7|6(+LNbmh#ARFB-z?Q=wMOi95d8jy(Dy0Kco#27@Euc|MyJ?;&x6;|oy!8v*)Jjh;O3z?{ z4lQL84Cx$dIs9KbVjxE4yu9e?JoO*osd&GNpXWZW(hSt_yI3=dvkh(dOJ&8LxSkap zMTIJ_1L%Q+C|$)4p^&)Ti~$6>K-W&P=;F(DKLEYErCKm7D{&f9%COrHzjf;8*?67q zuY}PAUsh*XveG{-_ODc_$4$RQp^Z7HKim8%s3?8VRqjVY!v3q}^rl4e>!Z!_N^68G zg!a|ijPDtDN}*{QZ~gwWI-{znN}HuVmnMrTJaAB@TMZukRvfLe)jt>Sl?}0PIAQ0R zU$e@#4yQw}))mVE%;8nHK4y{Wl1n!)KR$q(q@T{;hT0Yh6gfTG>AOEncWvm4+LKMi zVX-jjE^f5(T^{-`DWrnJ+03Q;W8yRWPB9@66Sts?jxv2f zW}Qb~xykd^ZOcAGB=Wp^6Y|_N{Tb>zGHs?hbfmC4&Dd}%WwrTIT}vOCD688r^KsM!lXNZ zJxjlI^am~-n%_;8doHo2Qm$YjZA1QXLS%oMXlHND&LYL3)B`l2s_AWl6D7U0x3kjB zzheYhGavi?OnD(RBd33l+{DDafV9+nd4b#W;q!Zc$%t8Oj5j{y$%OpUNpWu{QN2x5 z8j^K$VL zy}#DdMNC*o8s;pON^a$;;^e`nD$I!q)N! zC`2DY%goOj3`8~7-hB5M$wQrU=_1Osd!p6XI|HtflgDe_i(s{v3oh}ml*_=KA{4Uu zIpQ(*WiPa%xxP9d=gHwa@*~KM17qrsVZ+yNxk*c3H|`B1CC=t2ZGO^_&dL!xthtv| zaq~vHb~|bGr&Cw`6rw?WCHM-kj`Qes+aVj0syn!516a-hxeP}As7YN@@!I&UGY?@C`QXlG?G}~o^t`opj#>#ixJy^a={q+xRFmF# zA(i=YXJ&}RS*xVQNb^V~-{fGYz3J2!$%EvjjMS28r$Nabdl)`rsIn6^qDe>f5g7;b z0epOJ0`*R)j(2k3zjO0sgJXjY9AM>~3FZpYcPe4}(2W(g0OsL>;s)A~8($`k#hAUGRqs&xbmxdy#laBH1rf79!sbl^EX`XFSXH8B7t(8-$Usc*pag&5$Ai zTNG3gLGpKlHVO>oL|Sl%i9Kj5Ff%Nu`xsxPwbydvcdZt=NMkAx?o{j#J&J_iuVfRG zQgB(qa_jz*S?08I9`st*%WQyG(yyKyHh6;^Hi7VJW$w-ss z--+D%RQeK;e0aBV$jXQcy$aVPZV+vRJ97e1gv%4EuywN=fC#s6i6csC#yzQ;b3z!a z0=v}sd#_$ruT=f^d=508tyYGAFZwX#si2BAyQji=_dBW;yP-&>JODkqhdxE*G=@5h z?`J8AzE>BqSJceYlY zqB9}U#t5^UMc!$)M75~voA=D!wsp-W>_Vgkw(%B%@nuDi0hS0n)>{07qV$|b#_DAwr5t(fQcMXKoEeQ& ziYW4Dd*iF6n=6|G`0=aO^V+g3>u&J}9N$1@!ughyQ@rHKFr;J+`k4yomAUg%^yFC$ zT|}%Pgw4dgek!6@7oF;D1nxKz~Tf7W;chifzx_TjBx!1&!}GI#oo& zs{gfR5>;I5+Ry|VY}VVp@WjW}(F)K%@=)$O@^fghz76Ip=Snb+EHW+I=|1dBDp)Y0 zs~_cY-UYv0c>{(G0s}aJ0k987D#XQ0{6q3v5Lhm~E~=fSTG0h`j-G3m(=sdtjfg;_ zY^6`Va&|G;z6^49(YweA(^;@+1{u5PP3G#(CSWnx`8`2F$OboV}d?~gLGzH z-k_0Pj6cw?gw+0gUj`yB>(Y$%CNWt4zl%Zk=k;&osrqxDB+=@9!#laWILii})}4?OypZM*0!?uzGm>hgPw{SgW=j11VkM=Hp-RagvWl;{v<)c|SF5-7%~2A{Jm zz`8@^VI91U=dlI>@)1;Qz4*QQ<>qQaR_L5jZCf*-eXS1jBz^mti+h9I3YjJ_MGqKq z#QX#1JWZAY}4U4FJ;BzT~ho*IuG? zLu(t*+U%3PrA~en$oqtSIP%dBc1$=0`1}slGwU^^b-_F9lyDG0wB9X%AW%5&fbzDBCq4XK!OE; zEu)GIevo7JeM47}B;YW+eiXJ)8iP&wf;8b}b+SqwEs!J(IPGVnL7HzpQ>^WoA19-lz_+N!g{Qox_Dd zcmBva;|_%TH$BYzp~7eD>}GCNSwQ0BRAoeNDzvhzv>b+AiAM1@976{>9d8xRfdLi3 z!HSWakw_D<4ey11gh!UBd5=_)1dd)-D@##WvERcS@631qW%YGnXR>8Tb0>^R!0&gM z?_gW-=0I`lIe7i$<_HJw89^(o#rrF#?2+rbPMJm_TIf6wQNK*Modtq+k-Wumm`awS zq1Q&`U(q?62K!6jgFo6ku;VJ-Dlk@C8Hd3KQ*&9=Y_;tg5NtFO-2%+#C%+TF{zIlI zLB4WlDF5?mhe6_9qQH@q`@Xx~90*s!ILBk}2XfB16yeNx9PbBj&VvkrwMo*>D^nt` z^;~6EcJ|gT;zE-ui`mOrnvyCqv$S&4qBP#VF5nlaONH(+mz$VVvQ@bwZrqCbXQ( zjd#eKG4sg|+8{+vg3!ct(Ir~A|du9~cD~=6#N?+QH37Hcg0zWr$F%{egn=^f=W56j?-LC3t!;Nq<0F1cpgD zCzxXVoNvKR-Tv>^`*pf6%vXC&tusr7XDP#Bbh}U7fY{lNCl5O(Ldd<$rhoKz#eHZs z(1X|GcdkJ`g_oS7J+4DWn zzL@n&P2~51$_U3M8$r^LjB163eC5hvp(o)ja+$n9>wCD|Ph28eN^^V^1-9%$qwg47 z4g|_B6g8wToX>xaV)K!I^s>sJ7rnfVa8B)_NOp}3`YDg4^in2lhLelEwkcmTdrn3A zb=L>JCqj=q6vXJIeI-)G-R!VAPbm{ytK)4_5*o*t}q%clY1Yi=C~q%+wdW_0v!fwCLp?t(zR^UN_ zxZJX?vIC@;KcW4BrY1qjUe9KA|Tk5CD#5Ia+hq(NC6?=<0%w0U3^o_GB&9WQrHi+Cy8xx;^($Hw*i`1@9Gm_ zU^@z0Gd5{8@Lwk2AubSv)6f7dPdPD(;okF?rU8fj2OU1CxVAHdO(4_D+BYDd3m@kh z22yBi;cLrXtj6CL?lXFvP4V@jAoZ&CFxvs&7J8Kz0;0b&v_WJjqU&U1lt|%I z=$q}QV-|hMWZXcmd5JgH3HE}LN8R}TBBpMLonNOzs%74Dy|nX955E&bA~Qzh%?L@4 zi-o=_El@ZGFChe)`$4eg1H|0JHW-X|n)x?fB+=!fe{GU^%F(w{8GT>Ct+TP(Ahu{M znIRn%Ii!l(8%|0&JXcK<;XY)mzHuGzqmv#hI*5Qe3*Z3h?oLvxpq@~n;o?_?Ld{PC z9PxsvSpz}&2>a9hFUIND8ZX+eci%+wrb&9aIs-k@xY(}@0H&P6jaH{l1~=9vr<2`< z8k|pk0<{!17 zlnu{4Oo$rx2E=!+035)N(8>VnJ?qHo`WUv{r-gvG?dZMr>gQ-a6#=s9w*2LT5u@S< zijrkNQm}C#<#lJfiIAy{_g=r~P8<6H8xfRk^-p?T2kr#^1VgtG!FWj2?-qETqWBx! zz|mp9)vtM3Aal~GXTFjr6pqG7o!-BMtX+cRt+wm_{2mX! zA7kjJo#pHp_RjaDICuZhvTJca^*@mW$&=J$jY|u!7cE*`x3Bxs{TK-44e09~X4scG@bC zhbd80m<}`>L|2G%wkkOQ@w~Jq6=T+kq=r%ZaihsuqFnPMl~@8@QRwsUR_@I3W7&LG zEix$()THj70;d<65Q-f=r0mS>e^BpP`g_ibMHyy)Up$c=ge31ja07ku8 zeTUE3BF}ITw`hLv*}x{ziI!Vmi(SX4aYl$^s0ZEz%OD|AE0T!tR>2Z|dH+5BJ0)cp z%aKU-fU1Gz5BQ_zpW=2p;}hx-zq!Ev)Bry+PtXxHhbz*ePiPevMA)kOOX_$wAT>>v zP4K>)JTayNkuna8M@W{xz|S@Dc$9L|PHn`xRwAXdFrcDCWd7Na0myfa4))3I_F)S1 zAz_L3i7p&`GjHkxg{Rg@otaFtf@v09Ciqh>g?OVKRsh5$j+6bmFT--CW zBsqR1iC5=_&_?3#EeG@0Bex|iOLQ!;!s@v=XW`eLryLuGZ|l+LXy&c@Bg3h+u))zy z|4c6?6l2qB^dB}27Sq;M5tq?`tI1*h-m1H%t>etukGEQwnKvt5&gzZrN{n#xx=N!7 zX*Q`fRsQC^`YG{3CCC4B`H6Npxh%pRLM3HUsAoi`6TH=@&mw(wUOq9ONyU$Z45!Vr zz9Gnh+?CkYXt_5gAfa~BixdC|+Xakykx;@o(RxoD&5 zm9XCZVGTfBkR@V#>@n z$PlJd_OV+ZzE7V%EwAh70JV8YuIy-2Qb};^y9LPCWXd~&tE2h3lt9M7+{R{hcxR8J zM(n$d`}&44#fI^5{`o?Y1{DXot+=jc{ifxk+}(;;@Y6t+C_8?x#+t=ft#Y%}L0&h#&~ z`5p$|X)WdZX1x67taV%^K)pYP_ARm^U@NpNWAioK)T}N)RJ!?;s8zQ0ubi5XZd+u5k7%4(9FXA^1^@>y&AZD!y*v}Pa?E6F;`prk{`JIvcsBr>_O=4!+XLB#FTP9JV{T9j5F%r&(`q~5Tf*9|6Kjeywt zbBA!1JG<-`!{fY|pS`O@CQmAGD5jZpz7)zCmUrN4No~saI1uWLiSP#A>VdV{Rh!r( ziz96*kO{a_me8l`=R@9m9;K#)6cb+rn?KzNz zZ;hp;L=^jW32l!Qk7eaE?Yn{HDB)~E7I8SQMhGK1q9d4Op0(77ODFJSNQJjWOIIJ` zBkQ{`>GBLZHkfM5xp&AIz=C7}3C%V_ar}fnVO||7PrNK2<7x2H(AB7%ck!y~RP_gs z2;b1=NE9#PFq~BIYF=mweMe@@IY^X)tnm z=l`WR{9o~2Zc8h_M@vB*e_2zCz2g zT%q0rQ)`2H$Jf6E?yv4`FhOX(>s;9V>6E6!^&rJMvtGMc>Nj4!$x}jhwfHdqBQxWe zQYuuwd&Ng;-kp|jq+T+_kW=CHTv$!mI9bv+S;?}~)}LDAY%yfbLrzs6Z_6)lZ@7(% z%0=9W$aIUKyuL+_`KoWk!7or|NGzY_>s2E@3Xt_9jO0sC|L}N3 z;yJx!*1^@-)19(g`tI0oe706GlcYFiJ2CxkzRpGrpZWoA2(_Kz(}hxMO(eQ-!~AQc z(%ImRAV`(XT`w2Y;{^A#dled_D&1FM|PFdWBxx9h)H=Mj0 zthPxMiJNEsLh^a0Ru|ZIx|%Y)C`6la*VoK%h?IrBeZ6<=P>k!zVVSpGJY+Y-sPXZ; z6JMsSiUTR(=uUQf?-n!Jh2GTlR=4EC>$+S

x5pgJ099=@J|~?0a~Wn^@Q;Qd(2` z(e@rxmBsi5S#_)rFS-bkdiTcg_J-W&EPgnfLx1vYkDR3Gd>4}CM`Wnq)!9tgm6%%2 z^`z@AsXly2rP~a{{hkP`-)X>T9S3nScb;}WIxM-=oqP1Qm8lF9a$lB(L-lyJ9wC!< zVC2q^tg@ssY)-x9#2vo_u@zbG-8{4^(E_J;D0@P!>6?1MR@O;cSl4lQu+Cn)kx}Uo z`2x+UB*0xhyWX;arBI(ocdfwZ8AOdrCPq>04j*6Kr;tDU7Sst#OwuX;Xkj&UCe?$;fwdsbhJmtvF} z&DS=v_4fBTApndepYcUu8)2eXw{F{I;osJ}vGt`=f=m8f_3$@zEyb@-`{qNA;qaFR z=dCZ8u2S5{KYKSES$KI{^Q<9AAn)bnDpcB+zUw5Kr;s11FgEN;l#omc>pWc#P{0a* z?$EYLaUBJZhNsP97Ko9LpYtsVQGBkiPTtwmdPEUjAr~rME3|x@Ai`3 zAD5x>g|AM0a5=A0mF-g@Zkvz}e*cvj<3Aqp=#ROENnOtA?1w;FEmSoISm~{uf+@CJ%qwTk6QXu-vAQ zT+G}LyG^vc6_l)jB6uXH@G!UiVaNm&W%U%l)Hr`um3X1_PGR|1QLgqFwR_Rb>!EOw z=~VT!$2P;GUDohXDaqpi-NEyIolkh5K*>`1TgUVbkF|_b@@R$9HaVO9)R(vUX)Nt4 zGJge;vqM(Qn1h`KW(P$#kY3WM-#MB@1eEr4mvSAUL^_48I5gkJ)S6a}?`hs{a2zT~ z<^7OmnfN*AK-e?C$^7q{+BjrCD73kbjcjCMrH38!-Btp@9(~qvoKMETyEJcolmRu}&e7H$w z`g<`b_QLoF&gPv&R0gR9r)iirPY%MY{@qZ<7>Sl(x_k{cc{Yy19cLw*fataeUW%#s zy+JM=4sr#zPefehyO_|)Pel4>-$kv)LxG5-<4zPt*B7F*anFaGI!R?=iX`6jS^KA! zQi=ZC>!k?I@$LSSr%bdT8b423&M>7|RbQo%X-4QS(&vAmx@a!rnlTq2tr{!7{Uk(L z)$i4JO!)b)uV+ZgECersHL$rU{9Twnv3!{tV7 zw&BkW3#}kiGknO2HXdffm*zE22DXIA zck4hE6JKcb8PVj<%@E0`(4oV<-)1`4k5%SO&-vN? zXZgA@2eN1z=MKwAPy6FA*VT&lzoK**<`hvjINFhFT8| z!cm$CMtileTj=QV6;vuv;KF$VO@i_HQw_k2)F zHev~QjjSfsAca=LoI)d5xHpO;nHb{NV6zPvqm_*=vo2Ix$FgBw&gRoSQxb{l zfP@mFi;ovO!(k$SwyV)#vT$0>Y+{wC_6~ix9dRbIJX4B!&Z4cxvc7Y;O=dry<0ckM z>X>DUZaP};-4!Tx{aUAV1QmiBkiVd3CmfPe)(F7281cz+$RTQfixj#OL4H93Gmvh! z`7124OGEZyNs^a7qMFsm<6=eTqKrrehd9b#5&VRO;@S(99W2ca)djG`uYvBl8;-=Asz dj~y-{>55-F@dvcNfbUYTDa)(Fi)4-8|1YybW9t9_ literal 0 HcmV?d00001 diff --git a/docs/zh/12-taos-sql/13-tmq.md b/docs/zh/12-taos-sql/13-tmq.md index 571300ad8c..61135a3422 100644 --- a/docs/zh/12-taos-sql/13-tmq.md +++ b/docs/zh/12-taos-sql/13-tmq.md @@ -6,32 +6,68 @@ description: TDengine 消息队列提供的数据订阅功能 TDengine 3.0.0.0 开始对消息队列做了大幅的优化和增强以简化用户的解决方案。 -## 创建订阅主题 +## 创建 topic + +TDengine 创建 topic 的个数上限通过参数 tmqMaxTopicNum 控制,默认 20 个。 + +TDengine 使用 SQL 创建一个 topic,共有三种类型的 topic: + +### 查询 topic + +语法: ```sql -CREATE TOPIC [IF NOT EXISTS] topic_name AS subquery; +CREATE TOPIC [IF NOT EXISTS] topic_name as subquery ``` +通过 `SELECT` 语句订阅(包括 `SELECT *`,或 `SELECT ts, c1` 等指定查询订阅,可以带条件过滤、标量函数计算,但不支持聚合函数、不支持时间窗口聚合)。需要注意的是: -TOPIC 支持过滤和标量函数和 UDF 标量函数,不支持 JOIN、GROUP BY、窗口切分子句、聚合函数和 UDF 聚合函数。列订阅规则如下: +- 该类型 TOPIC 一旦创建则订阅数据的结构确定。 +- 被订阅或用于计算的列或标签不可被删除(`ALTER table DROP`)、修改(`ALTER table MODIFY`)。 +- 若发生表结构变更,新增的列不出现在结果中。 +- 对于 select \*,则订阅展开为创建时所有的列(子表、普通表为数据列,超级表为数据列加标签列) +### 超级表 topic -1. TOPIC 一旦创建则返回结果的字段确定 -2. 被订阅或用于计算的列不可被删除、修改 -3. 列可以新增,但新增的列不出现在订阅结果字段中 -4. 对于 select \*,则订阅展开为创建时所有的列(子表、普通表为数据列,超级表为数据列加标签列) - - -## 删除订阅主题 +语法: ```sql +CREATE TOPIC [IF NOT EXISTS] topic_name [with meta] AS STABLE stb_name [where_condition] +``` + +与 `SELECT * from stbName` 订阅的区别是: + +- 不会限制用户的表结构变更。 +- 返回的是非结构化的数据:返回数据的结构会随之超级表的表结构变化而变化。 +- with meta 参数可选,选择时将返回创建超级表,子表等语句,主要用于taosx做超级表迁移 +- where_condition 参数可选,选择时将用来过滤符合条件的子表,订阅这些子表。where 条件里不能有普通列,只能是tag或tbname,where条件里可以用函数,用来过滤tag,但是不能是聚合函数,因为子表tag值无法做聚合。也可以是常量表达式,比如 2 > 1(订阅全部子表),或者 false(订阅0个子表) +- 返回数据不包含标签。 + +### 数据库 topic + +语法: + +```sql +CREATE TOPIC [IF NOT EXISTS] topic_name [with meta] AS DATABASE db_name; +``` + +通过该语句可创建一个包含数据库所有表数据的订阅 + +- with meta 参数可选,选择时将返回创建数据库里所有超级表,子表的语句,主要用于taosx做数据库迁移 + +说明: 超级表订阅和库订阅属于高级订阅模式,容易出错,如确实要使用,请咨询专业人员。 + +## 删除 topic + +如果不再需要订阅数据,可以删除 topic,需要注意:只有当前未在订阅中的 TOPIC 才能被删除。 + +```sql +/* 删除 topic */ DROP TOPIC [IF EXISTS] topic_name; ``` 此时如果该订阅主题上存在 consumer,则此 consumer 会收到一个错误。 -## 查看订阅主题 - -## SHOW TOPICS +## 查看 topic ```sql SHOW TOPICS; @@ -58,3 +94,11 @@ SHOW CONSUMERS; ``` 显示当前数据库下所有活跃的消费者的信息。 + +## 查看订阅信息 + +```sql +SHOW SUBSCRIPTIONS; +``` + +显示 consumer 与 vgroup 之间的分配关系和消费信息 \ No newline at end of file From ee961270200554425163992a3d83e070b7c58884 Mon Sep 17 00:00:00 2001 From: wangmm0220 Date: Sun, 18 Feb 2024 18:38:38 +0800 Subject: [PATCH 50/51] fix:conflicts from 3.0 --- source/dnode/mnode/impl/src/mndScheduler.c | 76 ++++++++++++++++------ 1 file changed, 56 insertions(+), 20 deletions(-) diff --git a/source/dnode/mnode/impl/src/mndScheduler.c b/source/dnode/mnode/impl/src/mndScheduler.c index 2a8040cc55..88d326a5c4 100644 --- a/source/dnode/mnode/impl/src/mndScheduler.c +++ b/source/dnode/mnode/impl/src/mndScheduler.c @@ -222,13 +222,12 @@ static int32_t doAddSinkTask(SStreamObj* pStream, SMnode* pMnode, SVgObj* pVgrou int64_t uid = (isFillhistory) ? pStream->hTaskUid : pStream->uid; SArray** pTaskList = (isFillhistory) ? taosArrayGetLast(pStream->pHTasksList) : taosArrayGetLast(pStream->tasks); - SStreamTask* pTask = tNewStreamTask(uid, TASK_LEVEL__SINK, isFillhistory, 0, *pTaskList, pStream->conf.fillHistory); + SStreamTask* pTask = tNewStreamTask(uid, TASK_LEVEL__SINK, pEpset, isFillhistory, 0, *pTaskList, pStream->conf.fillHistory); if (pTask == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; return terrno; } - epsetAssign(&(pTask)->info.mnodeEpset, pEpset); mDebug("doAddSinkTask taskId:%s, vgId:%d, isFillHistory:%d", pTask->id.idStr, pVgroup->vgId, isFillhistory); pTask->info.nodeId = pVgroup->vgId; @@ -279,25 +278,60 @@ static int32_t doAddShuffleSinkTask(SMnode* pMnode, SStreamObj* pStream, SEpSet* return TDB_CODE_SUCCESS; } -static SStreamTask* buildSourceTask(SStreamObj* pStream, SEpSet* pEpset, int64_t firstWindowSkey, +static int64_t getVgroupLastVer(const SArray* pList, int32_t vgId) { + for(int32_t i = 0; i < taosArrayGetSize(pList); ++i) { + SVgroupVer* pVer = taosArrayGet(pList, i); + if (pVer->vgId == vgId) { + return pVer->ver; + } + } + + mError("failed to find the vgId:%d for extract last version", vgId); + return -1; +} + +static void streamTaskSetDataRange(SStreamTask* pTask, int64_t skey, SArray* pVerList, int32_t vgId) { + int64_t latestVer = getVgroupLastVer(pVerList, vgId); + if (latestVer < 0) { + latestVer = 0; + } + + // set the correct ts, which is the last key of queried table. + SDataRange* pRange = &pTask->dataRange; + STimeWindow* pWindow = &pRange->window; + + if (pTask->info.fillHistory) { + pWindow->skey = INT64_MIN; + pWindow->ekey = skey - 1; + + pRange->range.minVer = 0; + pRange->range.maxVer = latestVer; + mDebug("add fill-history source task 0x%x timeWindow:%" PRId64 "-%" PRId64 " verRange:%" PRId64 "-%" PRId64, + pTask->id.taskId, pWindow->skey, pWindow->ekey, pRange->range.minVer, pRange->range.maxVer); + } else { + pWindow->skey = skey; + pWindow->ekey = INT64_MAX; + + pRange->range.minVer = latestVer + 1; + pRange->range.maxVer = INT64_MAX; + + mDebug("add source task 0x%x timeWindow:%" PRId64 "-%" PRId64 " verRange:%" PRId64 "-%" PRId64, + pTask->id.taskId, pWindow->skey, pWindow->ekey, pRange->range.minVer, pRange->range.maxVer); + } +} + +static SStreamTask* buildSourceTask(SStreamObj* pStream, SEpSet* pEpset, bool isFillhistory, bool useTriggerParam) { uint64_t uid = (isFillhistory) ? pStream->hTaskUid : pStream->uid; SArray** pTaskList = (isFillhistory) ? taosArrayGetLast(pStream->pHTasksList) : taosArrayGetLast(pStream->tasks); - SStreamTask* pTask = tNewStreamTask(uid, TASK_LEVEL__SOURCE, + SStreamTask* pTask = tNewStreamTask(uid, TASK_LEVEL__SOURCE, pEpset, isFillhistory, useTriggerParam ? pStream->conf.triggerParam : 0, *pTaskList, pStream->conf.fillHistory); if (pTask == NULL) { return NULL; } - epsetAssign(&pTask->info.mnodeEpset, pEpset); - STimeWindow* pWindow = &pTask->dataRange.window; - - pWindow->skey = INT64_MIN; - pWindow->ekey = firstWindowSkey - 1; - mDebug("add source task 0x%x window:%" PRId64 " - %" PRId64, pTask->id.taskId, pWindow->skey, pWindow->ekey); - return pTask; } @@ -331,15 +365,17 @@ static void setHTasksId(SStreamObj* pStream) { } static int32_t doAddSourceTask(SMnode* pMnode, SSubplan* plan, SStreamObj* pStream, SEpSet* pEpset, - int64_t nextWindowSkey, SVgObj* pVgroup, bool isFillhistory, bool useTriggerParam ){ + int64_t skey, SArray* pVerList, SVgObj* pVgroup, bool isFillhistory, bool useTriggerParam ){ // new stream task - SStreamTask* pTask = buildSourceTask(pStream, pEpset, nextWindowSkey, isFillhistory, useTriggerParam); + SStreamTask* pTask = buildSourceTask(pStream, pEpset, isFillhistory, useTriggerParam); if(pTask == NULL){ terrno = TSDB_CODE_OUT_OF_MEMORY; return terrno; } mDebug("doAddSourceTask taskId:%s, vgId:%d, isFillHistory:%d", pTask->id.idStr, pVgroup->vgId, isFillhistory); + streamTaskSetDataRange(pTask, skey, pVerList, pVgroup->vgId); + int32_t code = mndAssignStreamTaskToVgroup(pMnode, pTask, plan, pVgroup); if(code != 0){ terrno = code; @@ -380,7 +416,7 @@ static SSubplan* getAggSubPlan(const SQueryPlan* pPlan, int index){ } static int32_t addSourceTask(SMnode* pMnode, SSubplan* plan, SStreamObj* pStream, - SEpSet* pEpset, int64_t nextWindowSkey, bool useTriggerParam) { + SEpSet* pEpset, int64_t nextWindowSkey, SArray* pVerList, bool useTriggerParam) { addNewTaskList(pStream); void* pIter = NULL; @@ -397,14 +433,14 @@ static int32_t addSourceTask(SMnode* pMnode, SSubplan* plan, SStreamObj* pStream continue; } - int code = doAddSourceTask(pMnode, plan, pStream, pEpset, nextWindowSkey, pVgroup, false, useTriggerParam); + int code = doAddSourceTask(pMnode, plan, pStream, pEpset, nextWindowSkey, pVerList, pVgroup, false, useTriggerParam); if(code != 0){ sdbRelease(pSdb, pVgroup); return code; } if (pStream->conf.fillHistory) { - code = doAddSourceTask(pMnode, plan, pStream, pEpset, nextWindowSkey, pVgroup, true, useTriggerParam); + code = doAddSourceTask(pMnode, plan, pStream, pEpset, nextWindowSkey, pVerList, pVgroup, true, useTriggerParam); if(code != 0){ sdbRelease(pSdb, pVgroup); return code; @@ -425,7 +461,7 @@ static SStreamTask* buildAggTask(SStreamObj* pStream, SEpSet* pEpset, bool isFil uint64_t uid = (isFillhistory) ? pStream->hTaskUid : pStream->uid; SArray** pTaskList = (isFillhistory) ? taosArrayGetLast(pStream->pHTasksList) : taosArrayGetLast(pStream->tasks); - SStreamTask* pAggTask = tNewStreamTask(uid, TASK_LEVEL__AGG, isFillhistory, + SStreamTask* pAggTask = tNewStreamTask(uid, TASK_LEVEL__AGG, pEpset, isFillhistory, useTriggerParam ? pStream->conf.triggerParam : 0, *pTaskList, pStream->conf.fillHistory); if (pAggTask == NULL) { @@ -433,7 +469,6 @@ static SStreamTask* buildAggTask(SStreamObj* pStream, SEpSet* pEpset, bool isFil return NULL; } - epsetAssign(&pAggTask->info.mnodeEpset, pEpset); return pAggTask; } @@ -566,7 +601,8 @@ static void bindTwoLevel(SArray* tasks, int32_t begin, int32_t end) { mDebug("bindTwoLevel task list(%d-%d) to taskId:%s", begin, end - 1, (*(pDownTask))->id.idStr); } -static int32_t doScheduleStream(SStreamObj* pStream, SMnode* pMnode, SQueryPlan* pPlan, int64_t nextWindowSkey, SEpSet* pEpset) { +static int32_t doScheduleStream(SStreamObj* pStream, SMnode* pMnode, SQueryPlan* pPlan, SEpSet* pEpset, int64_t skey, + SArray* pVerList) { SSdb* pSdb = pMnode->pSdb; int32_t numOfPlanLevel = LIST_LENGTH(pPlan->pSubplans); bool hasExtraSink = false; @@ -600,7 +636,7 @@ static int32_t doScheduleStream(SStreamObj* pStream, SMnode* pMnode, SQueryPlan* if (plan == NULL) { return terrno; } - int32_t code = addSourceTask(pMnode, plan, pStream, pEpset, nextWindowSkey, numOfPlanLevel == 1); + int32_t code = addSourceTask(pMnode, plan, pStream, pEpset, skey, pVerList, numOfPlanLevel == 1); if (code != TSDB_CODE_SUCCESS) { return code; } From fba1fee0689e7bfafe56114772aa6cc073e0f2d3 Mon Sep 17 00:00:00 2001 From: wangmm0220 Date: Sun, 18 Feb 2024 19:55:12 +0800 Subject: [PATCH 51/51] fix:cases error --- tests/system-test/8-stream/stream_multi_agg.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/system-test/8-stream/stream_multi_agg.py b/tests/system-test/8-stream/stream_multi_agg.py index 92ee4540e4..3532825493 100644 --- a/tests/system-test/8-stream/stream_multi_agg.py +++ b/tests/system-test/8-stream/stream_multi_agg.py @@ -68,10 +68,12 @@ class TDTestCase: # create stream tdSql.execute("use db") tdSql.execute("create stream stream1 fill_history 1 into sta as select count(*) as cnt from meters interval(10a);",show=True) + time.sleep(5) + sql = "select count(*) from sta" # loop wait max 60s to check count is ok tdLog.info("loop wait result ...") - tdSql.checkDataLoop(0, 0, 99, sql, loopCount=120, waitTime=0.5) + tdSql.checkDataLoop(0, 0, 100, sql, loopCount=10, waitTime=0.5) # check all data is correct sql = "select * from sta where cnt != 200;"