From 36aedfbb7cd6460e805830564bd0071b643fcd04 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 17 Feb 2022 15:24:00 +0800 Subject: [PATCH 01/25] [td-11818] create streamscan operator when no tables created. --- source/dnode/vnode/inc/vnode.h | 4 +++- source/libs/executor/src/executorimpl.c | 30 ++++++++++++++++--------- 2 files changed, 23 insertions(+), 11 deletions(-) diff --git a/source/dnode/vnode/inc/vnode.h b/source/dnode/vnode/inc/vnode.h index 394076f433..a414860afb 100644 --- a/source/dnode/vnode/inc/vnode.h +++ b/source/dnode/vnode/inc/vnode.h @@ -215,13 +215,15 @@ static FORCE_INLINE void tqReadHandleSetColIdList(STqReadHandle *pReadHandle, SA static FORCE_INLINE int tqReadHandleSetTbUidList(STqReadHandle *pHandle, const SArray *tbUidList) { pHandle->tbIdHash = taosHashInit(64, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BIGINT), true, HASH_NO_LOCK); if (pHandle->tbIdHash == NULL) { + terrno = TSDB_CODE_OUT_OF_MEMORY; return -1; } + for (int i = 0; i < taosArrayGetSize(tbUidList); i++) { int64_t *pKey = (int64_t *)taosArrayGet(tbUidList, i); taosHashPut(pHandle->tbIdHash, pKey, sizeof(int64_t), NULL, 0); - // pHandle->tbUid = tbUid; } + return 0; } diff --git a/source/libs/executor/src/executorimpl.c b/source/libs/executor/src/executorimpl.c index f5dc7a82b1..a4197b4414 100644 --- a/source/libs/executor/src/executorimpl.c +++ b/source/libs/executor/src/executorimpl.c @@ -5447,7 +5447,12 @@ SOperatorInfo* createStreamScanOperatorInfo(void *streamReadHandle, SArray* pExp // set the extract column id to streamHandle tqReadHandleSetColIdList((STqReadHandle* )streamReadHandle, pColList); - tqReadHandleSetTbUidList(streamReadHandle, pTableIdList); + int32_t code = tqReadHandleSetTbUidList(streamReadHandle, pTableIdList); + if (code != 0) { + tfree(pInfo); + tfree(pOperator); + return NULL; + } pInfo->readerHandle = streamReadHandle; @@ -5462,7 +5467,6 @@ SOperatorInfo* createStreamScanOperatorInfo(void *streamReadHandle, SArray* pExp return pOperator; } - void setTableScanFilterOperatorInfo(STableScanInfo* pTableScanInfo, SOperatorInfo* pDownstream) { assert(pTableScanInfo != NULL && pDownstream != NULL); @@ -8186,16 +8190,22 @@ SOperatorInfo* doCreateOperatorTreeNode(SPhyNode* pPhyNode, SExecTaskInfo* pTask STableGroupInfo groupInfo = {0}; int32_t code = doCreateTableGroup(pHandle->meta, pScanPhyNode->tableType, pScanPhyNode->uid, &groupInfo, queryId, taskId); + SArray* idList = NULL; - SArray* pa = taosArrayGetP(groupInfo.pGroupList, 0); - ASSERT(taosArrayGetSize(groupInfo.pGroupList) == 1); + if (groupInfo.numOfTables > 0) { + SArray* pa = taosArrayGetP(groupInfo.pGroupList, 0); + ASSERT(taosArrayGetSize(groupInfo.pGroupList) == 1); - // Transfer the Array of STableKeyInfo into uid list. - size_t numOfTables = taosArrayGetSize(pa); - SArray* idList = taosArrayInit(numOfTables, sizeof(uint64_t)); - for(int32_t i = 0; i < numOfTables; ++i) { - STableKeyInfo* pkeyInfo = taosArrayGet(pa, i); - taosArrayPush(idList, &pkeyInfo->uid); + // Transfer the Array of STableKeyInfo into uid list. + size_t numOfTables = taosArrayGetSize(pa); + for (int32_t i = 0; i < numOfTables; ++i) { + STableKeyInfo* pkeyInfo = taosArrayGet(pa, i); + taosArrayPush(idList, &pkeyInfo->uid); + } + + idList = taosArrayInit(numOfTables, sizeof(uint64_t)); + } else { + idList = taosArrayInit(4, sizeof(uint64_t)); } SOperatorInfo* pOperator = createStreamScanOperatorInfo(pHandle->reader, pPhyNode->pTargets, idList, pTaskInfo); From d6fa853b88ed0dfc28c58bafe90ae059d275776e Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 17 Feb 2022 18:16:50 +0800 Subject: [PATCH 02/25] [td-11818] refactor. --- include/libs/planner/plannerOp.h | 1 + source/libs/executor/inc/executorimpl.h | 38 +++--- source/libs/executor/src/executorimpl.c | 168 ++++++++---------------- 3 files changed, 74 insertions(+), 133 deletions(-) diff --git a/include/libs/planner/plannerOp.h b/include/libs/planner/plannerOp.h index 9030ffc946..840064025e 100644 --- a/include/libs/planner/plannerOp.h +++ b/include/libs/planner/plannerOp.h @@ -47,5 +47,6 @@ OP_ENUM_MACRO(AllTimeWindow) OP_ENUM_MACRO(AllMultiTableTimeInterval) OP_ENUM_MACRO(Order) OP_ENUM_MACRO(Exchange) +OP_ENUM_MACRO(SortMerge) //OP_ENUM_MACRO(TableScan) diff --git a/source/libs/executor/inc/executorimpl.h b/source/libs/executor/inc/executorimpl.h index 232b54554f..4265864f09 100644 --- a/source/libs/executor/inc/executorimpl.h +++ b/source/libs/executor/inc/executorimpl.h @@ -549,27 +549,19 @@ typedef struct SDistinctOperatorInfo { SArray* pDistinctDataInfo; } SDistinctOperatorInfo; -struct SGlobalMerger; - -typedef struct SMultiwayMergeInfo { - struct SGlobalMerger* pMerge; - SOptrBasicInfo binfo; - int32_t bufCapacity; - int64_t seed; - char** prevRow; - SArray* orderColumnList; - int32_t resultRowFactor; - - bool hasGroupColData; - char** currentGroupColData; - SArray* groupColumnList; - bool hasDataBlockForNewGroup; - SSDataBlock* pExistBlock; - - SArray* udfInfo; - bool hasPrev; - bool multiGroupResults; -} SMultiwayMergeInfo; +typedef struct SSortMergeOperatorInfo { + SOptrBasicInfo binfo; + SArray *orderInfo; // SArray + SArray* groupColumnList; + bool hasDataBlockForNewGroup; + char** currentGroupColData; + SArray* udfInfo; + int32_t numOfSources; +// char** prevRow; +// int32_t resultRowFactor; +// bool multiGroupResults; +// bool hasGroupColData; +} SSortMergeOperatorInfo; typedef struct SMsortComparParam { struct SExternalMemSource **pSources; @@ -592,6 +584,7 @@ typedef struct SOrderOperatorInfo { SMsortComparParam cmpParam; + // TODO extact struct int64_t startTs; // sort start time uint64_t sortElapsed; // sort elapsed time, time to flush to disk not included. uint64_t totalSize; // total load bytes from remote @@ -642,8 +635,7 @@ SOperatorInfo* createFilterOperatorInfo(STaskRuntimeEnv* pRuntimeEnv, SOperatorI SOperatorInfo* createJoinOperatorInfo(SOperatorInfo** pdownstream, int32_t numOfDownstream, SSchema* pSchema, int32_t numOfOutput); SOperatorInfo* createOrderOperatorInfo(SOperatorInfo* downstream, SArray* pExprInfo, SArray* pOrderVal); -SOperatorInfo* createMergeSortOperatorInfo(SOperatorInfo* downstream, SExprInfo* pExpr, int32_t numOfOutput, - SOrder* pOrderVal); +SOperatorInfo* createSortMergeOperatorInfo(SOperatorInfo* downstream, SArray* pExprInfo, void* param, SArray* pUdfInfo, SExecTaskInfo* pTaskInfo); // SSDataBlock* doGlobalAggregate(void* param, bool* newgroup); // SSDataBlock* doMultiwayMergeSort(void* param, bool* newgroup); diff --git a/source/libs/executor/src/executorimpl.c b/source/libs/executor/src/executorimpl.c index a4197b4414..593edf257a 100644 --- a/source/libs/executor/src/executorimpl.c +++ b/source/libs/executor/src/executorimpl.c @@ -5600,12 +5600,9 @@ SArray* getResultGroupCheckColumns(STaskAttr* pQuery) { } static void destroyGlobalAggOperatorInfo(void* param, int32_t numOfOutput) { - SMultiwayMergeInfo *pInfo = (SMultiwayMergeInfo*) param; + SSortMergeOperatorInfo *pInfo = (SSortMergeOperatorInfo*) param; destroyBasicOperatorInfo(&pInfo->binfo, numOfOutput); - - taosArrayDestroy(pInfo->orderColumnList); taosArrayDestroy(pInfo->groupColumnList); - tfree(pInfo->prevRow); tfree(pInfo->currentGroupColData); } @@ -5616,118 +5613,82 @@ static void destroySlimitOperatorInfo(void* param, int32_t numOfOutput) { tfree(pInfo->prevRow); } -SOperatorInfo* createGlobalAggregateOperatorInfo(STaskRuntimeEnv* pRuntimeEnv, SOperatorInfo* downstream, - SExprInfo* pExpr, int32_t numOfOutput, void* param, SArray* pUdfInfo, bool groupResultMixedUp) { - SMultiwayMergeInfo* pInfo = calloc(1, sizeof(SMultiwayMergeInfo)); - - pInfo->resultRowFactor = - (int32_t)(getRowNumForMultioutput(pRuntimeEnv->pQueryAttr, pRuntimeEnv->pQueryAttr->topBotQuery, false)); - - pRuntimeEnv->scanFlag = MERGE_STAGE; // TODO init when creating pCtx - - pInfo->multiGroupResults = groupResultMixedUp; - pInfo->pMerge = param; - pInfo->bufCapacity = 4096; - pInfo->udfInfo = pUdfInfo; - pInfo->binfo.pRes = createOutputBuf(pExpr, numOfOutput, pInfo->bufCapacity * pInfo->resultRowFactor); - pInfo->binfo.pCtx = createSqlFunctionCtx(pRuntimeEnv, pExpr, numOfOutput, &pInfo->binfo.rowCellInfoOffset); - pInfo->orderColumnList = getOrderCheckColumns(pRuntimeEnv->pQueryAttr); - pInfo->groupColumnList = getResultGroupCheckColumns(pRuntimeEnv->pQueryAttr); - - // TODO refactor - int32_t len = 0; - for(int32_t i = 0; i < numOfOutput; ++i) { -// len += pExpr[i].base.; +static SExprInfo* exprArrayDup(SArray* pExprInfo) { + size_t numOfOutput = taosArrayGetSize(pExprInfo); + SExprInfo* p = calloc(numOfOutput, sizeof(SExprInfo)); + for (int32_t i = 0; i < taosArrayGetSize(pExprInfo); ++i) { + SExprInfo* pExpr = taosArrayGetP(pExprInfo, i); + assignExprInfo(&p[i], pExpr); } - int32_t numOfCols = (pInfo->orderColumnList != NULL)? (int32_t) taosArrayGetSize(pInfo->orderColumnList):0; - pInfo->prevRow = calloc(1, (POINTER_BYTES * numOfCols + len)); + return p; +} + +static SSDataBlock* doSortMerge(void* param, bool* newgroup) { + SOperatorInfo* pOperator = (SOperatorInfo*) param; + if (pOperator->status == OP_EXEC_DONE) { + return NULL; + } + + SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; + SSortMergeOperatorInfo* pInfo = pOperator->info; + + for(int32_t i = 0; i < pInfo->numOfSources; ++i) { + SSDataBlock* pBlock = pOperator->pDownstream[i]->exec(pOperator->pDownstream[i], newgroup); + + } + + return NULL; +} + +SOperatorInfo* createSortMergeOperatorInfo(SOperatorInfo* downstream, SArray* pExprInfo, void* param, SArray* pUdfInfo, SExecTaskInfo* pTaskInfo) { + SSortMergeOperatorInfo* pInfo = calloc(1, sizeof(SSortMergeOperatorInfo)); + SOperatorInfo* pOperator = calloc(1, sizeof(SOperatorInfo)); + +// pInfo->resultRowFactor = +// (int32_t)(getRowNumForMultioutput(pRuntimeEnv->pQueryAttr, pRuntimeEnv->pQueryAttr->topBotQuery, false)); + + int32_t numOfOutput = taosArrayGetSize(pExprInfo); + pInfo->binfo.capacity = 4096; + pInfo->udfInfo = pUdfInfo; + pInfo->binfo.pRes = createOutputBuf_rv(pExprInfo, pInfo->binfo.capacity); + pInfo->binfo.pCtx = createSqlFunctionCtx_rv( pExprInfo, &pInfo->binfo.rowCellInfoOffset, &pInfo->binfo.resRowSize); + + int32_t numOfCols = (pInfo->groupColumnList != NULL)? (int32_t)taosArrayGetSize(pInfo->groupColumnList):0; +// pInfo->currentGroupColData = calloc(1, (POINTER_BYTES * numOfCols + len)); int32_t offset = POINTER_BYTES * numOfCols; - for(int32_t i = 0; i < numOfCols; ++i) { - pInfo->prevRow[i] = (char*)pInfo->prevRow + offset; - - SColIndex* index = taosArrayGet(pInfo->orderColumnList, i); - offset += pExpr[index->colIndex].base.resSchema.bytes; - } - - numOfCols = (pInfo->groupColumnList != NULL)? (int32_t)taosArrayGetSize(pInfo->groupColumnList):0; - pInfo->currentGroupColData = calloc(1, (POINTER_BYTES * numOfCols + len)); - offset = POINTER_BYTES * numOfCols; - - for(int32_t i = 0; i < numOfCols; ++i) { - pInfo->currentGroupColData[i] = (char*)pInfo->currentGroupColData + offset; - - SColIndex* index = taosArrayGet(pInfo->groupColumnList, i); - offset += pExpr[index->colIndex].base.resSchema.bytes; - } +// for(int32_t i = 0; i < numOfCols; ++i) { +// pInfo->currentGroupColData[i] = (char*)pInfo->currentGroupColData + offset; +// +// SColIndex* index = taosArrayGet(pInfo->groupColumnList, i); +// offset += pExpr[index->colIndex].base.resSchema.bytes; +// } initResultRowInfo(&pInfo->binfo.resultRowInfo, 8, TSDB_DATA_TYPE_INT); - pInfo->seed = rand(); - setDefaultOutputBuf(pRuntimeEnv, &pInfo->binfo, pInfo->seed, MERGE_STAGE); + int32_t numOfRows = 1; +// setDefaultOutputBuf_rv(pExprInfo, numOfRows); - SOperatorInfo* pOperator = calloc(1, sizeof(SOperatorInfo)); - pOperator->name = "GlobalAggregate"; -// pOperator->operatorType = OP_GlobalAggregate; + pOperator->name = "SortMerge"; + pOperator->operatorType = OP_SortMerge; pOperator->blockingOptr = true; pOperator->status = OP_IN_EXECUTING; pOperator->info = pInfo; - pOperator->pExpr = pExpr; + pOperator->pExpr = exprArrayDup(pExprInfo); pOperator->numOfOutput = numOfOutput; - pOperator->pRuntimeEnv = pRuntimeEnv; -// pOperator->exec = doGlobalAggregate; + + pOperator->pTaskInfo = pTaskInfo; + pOperator->exec = doSortMerge; pOperator->cleanupFn = destroyGlobalAggOperatorInfo; appendDownstream(pOperator, downstream); return pOperator; } -SOperatorInfo *createMultiwaySortOperatorInfo(STaskRuntimeEnv *pRuntimeEnv, SExprInfo *pExpr, int32_t numOfOutput, - int32_t numOfRows, void *merger) { - SMultiwayMergeInfo* pInfo = calloc(1, sizeof(SMultiwayMergeInfo)); - - pInfo->pMerge = merger; - pInfo->bufCapacity = numOfRows; - pInfo->orderColumnList = getResultGroupCheckColumns(pRuntimeEnv->pQueryAttr); - pInfo->binfo.pRes = createOutputBuf(pExpr, numOfOutput, numOfRows); - - { // todo extract method to create prev compare buffer - int32_t len = 0; - for(int32_t i = 0; i < numOfOutput; ++i) { -// len += pExpr[i].base.colBytes; - } - - int32_t numOfCols = (pInfo->orderColumnList != NULL)? (int32_t) taosArrayGetSize(pInfo->orderColumnList):0; - pInfo->prevRow = calloc(1, (POINTER_BYTES * numOfCols + len)); - - int32_t offset = POINTER_BYTES * numOfCols; - for(int32_t i = 0; i < numOfCols; ++i) { - pInfo->prevRow[i] = (char*)pInfo->prevRow + offset; - - SColIndex* index = taosArrayGet(pInfo->orderColumnList, i); -// offset += pExpr[index->colIndex].base.colBytes; - } - } - - SOperatorInfo* pOperator = calloc(1, sizeof(SOperatorInfo)); - pOperator->name = "MultiwaySortOperator"; -// pOperator->operatorType = OP_MultiwayMergeSort; - pOperator->blockingOptr = false; - pOperator->status = OP_IN_EXECUTING; - pOperator->info = pInfo; - pOperator->pRuntimeEnv = pRuntimeEnv; - pOperator->numOfOutput = numOfOutput; - pOperator->pExpr = pExpr; -// pOperator->exec = doMultiwayMergeSort; - pOperator->cleanupFn = destroyGlobalAggOperatorInfo; - return pOperator; -} - typedef struct SExternalMemSource { SArray* pageIdList; int32_t pageIndex; - int32_t sourceId; int32_t rowIndex; SSDataBlock *pBlock; } SExternalMemSource; @@ -5879,8 +5840,6 @@ static int32_t doAddNewSource(SOrderOperatorInfo* pInfo, SArray* pAllSources, in } pSource->pageIdList = getDataBufPagesIdList(pInfo->pSortInternalBuf, pInfo->sourceId); - pSource->sourceId = pInfo->sourceId; - pSource->pBlock = calloc(1, sizeof(SSDataBlock)); pSource->pBlock->pDataBlock = taosArrayInit(numOfCols, sizeof(SColumnInfoData)); pSource->pBlock->info.numOfCols = numOfCols; @@ -7197,17 +7156,6 @@ static int32_t initAggInfo(SAggOperatorInfo* pInfo, SArray* pExprInfo, int32_t n return TSDB_CODE_SUCCESS; } -static SExprInfo* exprArrayDup(SArray* pExprInfo) { - size_t numOfOutput = taosArrayGetSize(pExprInfo); - SExprInfo* p = calloc(numOfOutput, sizeof(SExprInfo)); - for (int32_t i = 0; i < taosArrayGetSize(pExprInfo); ++i) { - SExprInfo* pExpr = taosArrayGetP(pExprInfo, i); - assignExprInfo(&p[i], pExpr); - } - - return p; -} - SOperatorInfo* createAggregateOperatorInfo(SOperatorInfo* downstream, SArray* pExprInfo, SExecTaskInfo* pTaskInfo, const STableGroupInfo* pTableGroupInfo) { SAggOperatorInfo* pInfo = calloc(1, sizeof(SAggOperatorInfo)); @@ -8198,12 +8146,12 @@ SOperatorInfo* doCreateOperatorTreeNode(SPhyNode* pPhyNode, SExecTaskInfo* pTask // Transfer the Array of STableKeyInfo into uid list. size_t numOfTables = taosArrayGetSize(pa); + idList = taosArrayInit(numOfTables, sizeof(uint64_t)); + for (int32_t i = 0; i < numOfTables; ++i) { STableKeyInfo* pkeyInfo = taosArrayGet(pa, i); taosArrayPush(idList, &pkeyInfo->uid); } - - idList = taosArrayInit(numOfTables, sizeof(uint64_t)); } else { idList = taosArrayInit(4, sizeof(uint64_t)); } From c15c88dff74baee2c4e3b215ca93e1144327d992 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 17 Feb 2022 18:37:43 +0800 Subject: [PATCH 03/25] [td-11818] add table id for streamscan operator. --- example/src/tmq.c | 20 ++++++++++---------- include/libs/executor/executor.h | 13 ++++++++++++- source/libs/executor/src/executor.c | 22 ++++++++++++++++++++++ source/libs/executor/src/executorimpl.c | 7 +++---- 4 files changed, 47 insertions(+), 15 deletions(-) diff --git a/example/src/tmq.c b/example/src/tmq.c index 5858282aab..05540780d1 100644 --- a/example/src/tmq.c +++ b/example/src/tmq.c @@ -28,7 +28,7 @@ int32_t init_env() { return -1; } - TAOS_RES* pRes = taos_query(pConn, "create database if not exists abc1 vgroups 1"); + TAOS_RES* pRes = taos_query(pConn, "create database if not exists abc1 vgroups 2"); if (taos_errno(pRes) != 0) { printf("error in create db, reason:%s\n", taos_errstr(pRes)); return -1; @@ -56,14 +56,14 @@ int32_t init_env() { } taos_free_result(pRes); - pRes = taos_query(pConn, "create table if not exists tu2 using st1 tags(2)"); - if (taos_errno(pRes) != 0) { - printf("failed to create child table tu2, reason:%s\n", taos_errstr(pRes)); - return -1; - } - taos_free_result(pRes); +// pRes = taos_query(pConn, "create table if not exists tu6 using st1 tags(2)"); +// if (taos_errno(pRes) != 0) { +// printf("failed to create child table tu2, reason:%s\n", taos_errstr(pRes)); +// return -1; +// } +// taos_free_result(pRes); - const char* sql = "select * from tu1"; + const char* sql = "select * from st1"; pRes = tmq_create_topic(pConn, "test_stb_topic_1", sql, strlen(sql)); if (taos_errno(pRes) != 0) { printf("failed to create topic test_stb_topic_1, reason:%s\n", taos_errstr(pRes)); @@ -197,7 +197,7 @@ int main() { code = init_env(); tmq_t* tmq = build_consumer(); tmq_list_t* topic_list = build_topic_list(); - perf_loop(tmq, topic_list); - /*basic_consume_loop(tmq, topic_list);*/ +// perf_loop(tmq, topic_list); + basic_consume_loop(tmq, topic_list); /*sync_consume_loop(tmq, topic_list);*/ } diff --git a/include/libs/executor/executor.h b/include/libs/executor/executor.h index d9e1957e5d..caa8f0bc36 100644 --- a/include/libs/executor/executor.h +++ b/include/libs/executor/executor.h @@ -31,6 +31,7 @@ typedef struct SReadHandle { void* reader; void* meta; } SReadHandle; + /** * Create the exec task for streaming mode * @param pMsg @@ -40,13 +41,23 @@ typedef struct SReadHandle { qTaskInfo_t qCreateStreamExecTaskInfo(void *msg, void* streamReadHandle); /** - * + * Set the input data block for the stream scan. * @param tinfo * @param input * @return */ int32_t qSetStreamInput(qTaskInfo_t tinfo, const void* input); +/** + * Update the table id list, add or remove. + * + * @param tinfo + * @param id + * @param isAdd + * @return + */ +int32_t qUpdateTQualifiedTableId(qTaskInfo_t tinfo, SArray* tableIdList, bool isAdd); + /** * Create the exec task object according to task json * @param readHandle diff --git a/source/libs/executor/src/executor.c b/source/libs/executor/src/executor.c index c47c83ba29..29e60c8e91 100644 --- a/source/libs/executor/src/executor.c +++ b/source/libs/executor/src/executor.c @@ -90,3 +90,25 @@ qTaskInfo_t qCreateStreamExecTaskInfo(void* msg, void* streamReadHandle) { return pTaskInfo; } + +int32_t qUpdateTQualifiedTableId(qTaskInfo_t tinfo, SArray* tableIdList, bool isAdd) { + SExecTaskInfo* pTaskInfo = (SExecTaskInfo* )tinfo; + + // traverse to the streamscan node to add this table id + SOperatorInfo* pInfo = pTaskInfo->pRoot; + while(pInfo->operatorType != OP_StreamScan) { + pInfo = pInfo->pDownstream[0]; + } + + SStreamBlockScanInfo* pScanInfo = pInfo->info; + if (isAdd) { + int32_t code = tqReadHandleSetTbUidList(pScanInfo->readerHandle, tableIdList); + if (code != TSDB_CODE_SUCCESS) { + return code; + } + } else { + assert(0); + } + + return TSDB_CODE_SUCCESS; +} \ No newline at end of file diff --git a/source/libs/executor/src/executorimpl.c b/source/libs/executor/src/executorimpl.c index 593edf257a..4d8249d12f 100644 --- a/source/libs/executor/src/executorimpl.c +++ b/source/libs/executor/src/executorimpl.c @@ -5441,7 +5441,6 @@ SOperatorInfo* createStreamScanOperatorInfo(void *streamReadHandle, SArray* pExp SArray* pColList = taosArrayInit(numOfOutput, sizeof(int32_t)); for(int32_t i = 0; i < numOfOutput; ++i) { SExprInfo* pExpr = taosArrayGetP(pExprInfo, i); - taosArrayPush(pColList, &pExpr->pExpr->pSchema[0].colId); } @@ -5624,7 +5623,7 @@ static SExprInfo* exprArrayDup(SArray* pExprInfo) { return p; } -static SSDataBlock* doSortMerge(void* param, bool* newgroup) { +static SSDataBlock* doSortedMerge(void* param, bool* newgroup) { SOperatorInfo* pOperator = (SOperatorInfo*) param; if (pOperator->status == OP_EXEC_DONE) { return NULL; @@ -5635,7 +5634,7 @@ static SSDataBlock* doSortMerge(void* param, bool* newgroup) { for(int32_t i = 0; i < pInfo->numOfSources; ++i) { SSDataBlock* pBlock = pOperator->pDownstream[i]->exec(pOperator->pDownstream[i], newgroup); - + // TODO set the order input sources. } return NULL; @@ -5679,7 +5678,7 @@ SOperatorInfo* createSortMergeOperatorInfo(SOperatorInfo* downstream, SArray* pE pOperator->numOfOutput = numOfOutput; pOperator->pTaskInfo = pTaskInfo; - pOperator->exec = doSortMerge; + pOperator->exec = doSortedMerge; pOperator->cleanupFn = destroyGlobalAggOperatorInfo; appendDownstream(pOperator, downstream); From d11c44fc184be45d4479986a47b1018203d98373 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 17 Feb 2022 18:38:55 +0800 Subject: [PATCH 04/25] [td-11818] refactor. --- include/libs/executor/executor.h | 4 ++-- source/libs/executor/src/executor.c | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/include/libs/executor/executor.h b/include/libs/executor/executor.h index caa8f0bc36..4d53f40176 100644 --- a/include/libs/executor/executor.h +++ b/include/libs/executor/executor.h @@ -46,7 +46,7 @@ qTaskInfo_t qCreateStreamExecTaskInfo(void *msg, void* streamReadHandle); * @param input * @return */ -int32_t qSetStreamInput(qTaskInfo_t tinfo, const void* input); +int32_t qSetStreamInput(qTaskInfo_t tinfo, const void* input); /** * Update the table id list, add or remove. @@ -56,7 +56,7 @@ int32_t qSetStreamInput(qTaskInfo_t tinfo, const void* input); * @param isAdd * @return */ -int32_t qUpdateTQualifiedTableId(qTaskInfo_t tinfo, SArray* tableIdList, bool isAdd); +int32_t qUpdateQualifiedTableId(qTaskInfo_t tinfo, SArray* tableIdList, bool isAdd); /** * Create the exec task object according to task json diff --git a/source/libs/executor/src/executor.c b/source/libs/executor/src/executor.c index 29e60c8e91..e7e6dd7129 100644 --- a/source/libs/executor/src/executor.c +++ b/source/libs/executor/src/executor.c @@ -91,7 +91,7 @@ qTaskInfo_t qCreateStreamExecTaskInfo(void* msg, void* streamReadHandle) { return pTaskInfo; } -int32_t qUpdateTQualifiedTableId(qTaskInfo_t tinfo, SArray* tableIdList, bool isAdd) { +int32_t qUpdateQualifiedTableId(qTaskInfo_t tinfo, SArray* tableIdList, bool isAdd) { SExecTaskInfo* pTaskInfo = (SExecTaskInfo* )tinfo; // traverse to the streamscan node to add this table id From 39c06f13f658a8e0edc18765c8c5e893f7f57209 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sat, 19 Feb 2022 15:20:20 +0800 Subject: [PATCH 05/25] [td-11818] refactor sort, and sort operator. --- include/common/tep.h | 5 +- include/util/tpagedbuf.h | 2 +- source/common/src/tep.c | 17 +- source/libs/executor/inc/executorimpl.h | 16 +- source/libs/executor/inc/tsort.h | 130 ++++ source/libs/executor/src/executorimpl.c | 437 ++++------- source/libs/executor/src/tsort.c | 694 ++++++++++++++++++ source/libs/executor/test/executorTests.cpp | 6 +- .../libs/executor/test/executorUtilTests.cpp | 269 +++++++ source/util/src/tpagedbuf.c | 4 +- 10 files changed, 1254 insertions(+), 326 deletions(-) create mode 100644 source/libs/executor/inc/tsort.h create mode 100644 source/libs/executor/src/tsort.c create mode 100644 source/libs/executor/test/executorUtilTests.cpp diff --git a/include/common/tep.h b/include/common/tep.h index 584b8a5a71..a4e28dbb7c 100644 --- a/include/common/tep.h +++ b/include/common/tep.h @@ -18,9 +18,6 @@ typedef struct SBlockOrderInfo { int32_t order; int32_t colIndex; SColumnInfoData *pColData; -// int32_t type; -// int32_t bytes; -// bool hasNull; } SBlockOrderInfo; int taosGetFqdnPortFromEp(const char *ep, SEp *pEp); @@ -93,6 +90,8 @@ size_t blockDataGetRowSize(const SSDataBlock* pBlock); double blockDataGetSerialRowSize(const SSDataBlock* pBlock); size_t blockDataGetSerialMetaSize(const SSDataBlock* pBlock); +SSchema* blockDataExtractSchema(const SSDataBlock* pBlock, int32_t* numOfCols); + size_t blockDataNumOfRowsForSerialize(const SSDataBlock* pBlock, int32_t blockSize); int32_t blockDataSort(SSDataBlock* pDataBlock, SArray* pOrderInfo, bool nullFirst); diff --git a/include/util/tpagedbuf.h b/include/util/tpagedbuf.h index e989c31cd6..708c6cf741 100644 --- a/include/util/tpagedbuf.h +++ b/include/util/tpagedbuf.h @@ -157,7 +157,7 @@ void setBufPageDirty(SFilePage* pPageInfo, bool dirty); * Print the statistics when closing this buffer * @param pBuf */ -void printStatisBeforeClose(SDiskbasedBuf* pBuf); +void setPrintStatis(SDiskbasedBuf* pBuf); /** * return buf statistics. diff --git a/source/common/src/tep.c b/source/common/src/tep.c index 970b6d954f..b7f7043d26 100644 --- a/source/common/src/tep.c +++ b/source/common/src/tep.c @@ -411,7 +411,6 @@ SSDataBlock* blockDataExtractBlock(SSDataBlock* pBlock, int32_t startIndex, int3 return pDst; } - /** * * +------------------+---------------+--------------------+ @@ -522,6 +521,22 @@ size_t blockDataGetSerialMetaSize(const SSDataBlock* pBlock) { return sizeof(int32_t) + pBlock->info.numOfCols * sizeof(int32_t); } +SSchema* blockDataExtractSchema(const SSDataBlock* pBlock, int32_t* numOfCols) { + SSchema* pSchema = calloc(pBlock->info.numOfCols, sizeof(SSchema)); + for(int32_t i = 0; i < pBlock->info.numOfCols; ++i) { + SColumnInfoData* pColInfoData = taosArrayGet(pBlock->pDataBlock, i); + pSchema[i].bytes = pColInfoData->info.bytes; + pSchema[i].type = pColInfoData->info.type; + pSchema[i].colId = pColInfoData->info.colId; + } + + if (numOfCols != NULL) { + *numOfCols = pBlock->info.numOfCols; + } + + return pSchema; +} + double blockDataGetSerialRowSize(const SSDataBlock* pBlock) { ASSERT(pBlock != NULL); double rowSize = 0; diff --git a/source/libs/executor/inc/executorimpl.h b/source/libs/executor/inc/executorimpl.h index 4265864f09..adfed41f99 100644 --- a/source/libs/executor/inc/executorimpl.h +++ b/source/libs/executor/inc/executorimpl.h @@ -15,6 +15,7 @@ #ifndef TDENGINE_EXECUTORIMPL_H #define TDENGINE_EXECUTORIMPL_H +#include "tsort.h" #ifdef __cplusplus extern "C" { #endif @@ -557,10 +558,6 @@ typedef struct SSortMergeOperatorInfo { char** currentGroupColData; SArray* udfInfo; int32_t numOfSources; -// char** prevRow; -// int32_t resultRowFactor; -// bool multiGroupResults; -// bool hasGroupColData; } SSortMergeOperatorInfo; typedef struct SMsortComparParam { @@ -571,19 +568,16 @@ typedef struct SMsortComparParam { } SMsortComparParam; typedef struct SOrderOperatorInfo { - int32_t sourceId; uint32_t sortBufSize; // max buffer size for in-memory sort SSDataBlock *pDataBlock; bool hasVarCol; // has variable length column, such as binary/varchar/nchar - int32_t numOfCompleted; - SDiskbasedBuf *pSortInternalBuf; - SMultiwayMergeTreeInfo *pMergeTree; - SArray *pSources; // SArray + SArray *orderInfo; + bool nullFirst; + SSortHandle *pSortHandle; + int32_t bufPageSize; int32_t numOfRowsInRes; - SMsortComparParam cmpParam; - // TODO extact struct int64_t startTs; // sort start time uint64_t sortElapsed; // sort elapsed time, time to flush to disk not included. diff --git a/source/libs/executor/inc/tsort.h b/source/libs/executor/inc/tsort.h new file mode 100644 index 0000000000..9114746ff9 --- /dev/null +++ b/source/libs/executor/inc/tsort.h @@ -0,0 +1,130 @@ +/* + * Copyright (c) 2019 TAOS Data, Inc. + * + * This program is free software: you can use, redistribute, and/or modify + * it under the terms of the GNU Affero General Public License, version 3 + * or later ("AGPL"), as published by the Free Software Foundation. + * + * This program is distributed in the hope that it will be useful, but WITHOUT + * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or + * FITNESS FOR A PARTICULAR PURPOSE. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + */ + +#ifndef TDENGINE_TSORT_H +#define TDENGINE_TSORT_H + +#ifdef __cplusplus +extern "C" { +#endif + +#include "common.h" +#include "os.h" + +enum { + SORT_MULTIWAY_MERGE = 0x1, + SORT_SINGLESOURCE = 0x2, +}; + +typedef struct SMultiMergeSource { + int32_t type; + int32_t rowIndex; + SSDataBlock *pBlock; +} SMultiMergeSource; + +typedef struct SExternalMemSource { + SMultiMergeSource src; + SArray* pageIdList; + int32_t pageIndex; +} SExternalMemSource; + +typedef struct SOperatorSource { + SMultiMergeSource src; + void* param; +} SOperatorSource; + +typedef struct SSortHandle SSortHandle; +typedef struct STupleHandle STupleHandle; + +typedef SSDataBlock* (*_sort_fetch_block_fn_t)(void* param); +typedef int32_t (*_sort_merge_compar_fn_t)(const void* p1, const void* p2, void* param); + +/** + * + * @param type + * @return + */ +SSortHandle* createSortHandle(SArray* pOrderInfo, bool nullFirst, int32_t type, int32_t pageSize, int32_t numOfPages, SSchema* pSchema, int32_t numOfCols, const char* idstr); + +/** + * + * @param pSortHandle + */ +void destroySortHandle(SSortHandle* pSortHandle); + +/** + * + * @param pHandle + * @return + */ +int32_t sortOpen(SSortHandle* pHandle); + +/** + * + * @param pHandle + * @return + */ +int32_t sortClose(SSortHandle* pHandle); + +/** + * + * @return + */ +int32_t setFetchRawDataFp(SSortHandle* pHandle, _sort_fetch_block_fn_t fp); + +/** + * + * @param pHandle + * @param fp + * @return + */ +int32_t setComparFn(SSortHandle* pHandle, _sort_merge_compar_fn_t fp); + +/** + * + * @param pHandle + * @param pSource + * @return success or failed + */ +int32_t sortAddSource(SSortHandle* pSortHandle, void* pSource); + +/** + * + * @param pHandle + * @return + */ +STupleHandle* sortNextTuple(SSortHandle* pHandle); + +/** + * + * @param pHandle + * @param colIndex + * @return + */ +bool sortIsValueNull(STupleHandle* pVHandle, int32_t colIndex); + +/** + * + * @param pHandle + * @param colIndex + * @return + */ +void* sortGetValue(STupleHandle* pVHandle, int32_t colIndex); + +#ifdef __cplusplus +} +#endif + +#endif // TDENGINE_TSORT_H diff --git a/source/libs/executor/src/executorimpl.c b/source/libs/executor/src/executorimpl.c index 4d8249d12f..730396c070 100644 --- a/source/libs/executor/src/executorimpl.c +++ b/source/libs/executor/src/executorimpl.c @@ -13,6 +13,7 @@ * along with this program. If not, see . */ #include +#include #include "exception.h" #include "os.h" #include "parser.h" @@ -5598,7 +5599,7 @@ SArray* getResultGroupCheckColumns(STaskAttr* pQuery) { return pOrderColumns; } -static void destroyGlobalAggOperatorInfo(void* param, int32_t numOfOutput) { +static void destroySortMergeOperatorInfo(void* param, int32_t numOfOutput) { SSortMergeOperatorInfo *pInfo = (SSortMergeOperatorInfo*) param; destroyBasicOperatorInfo(&pInfo->binfo, numOfOutput); taosArrayDestroy(pInfo->groupColumnList); @@ -5632,11 +5633,21 @@ static SSDataBlock* doSortedMerge(void* param, bool* newgroup) { SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; SSortMergeOperatorInfo* pInfo = pOperator->info; + SMsortComparParam resultParam = {.orderInfo = pInfo->orderInfo}; + SArray* pSource = taosArrayInit(4, POINTER_BYTES); + for(int32_t i = 0; i < pInfo->numOfSources; ++i) { SSDataBlock* pBlock = pOperator->pDownstream[i]->exec(pOperator->pDownstream[i], newgroup); - // TODO set the order input sources. +// doAddNewOperatorSource(pSource, pBlock, pInfo->binfo.capacity); } +// sortComparInit(&resultParam, pSource, 0, pInfo->numOfSources - 1, pOperator); + +// int32_t code = tMergeTreeCreate(&pInfo->pMergeTree, pInfo->cmpParam.numOfSources, &pInfo->cmpParam, msortComparFn); +// if (code != TSDB_CODE_SUCCESS) { +// longjmp(pTaskInfo->env, code); +// } + return NULL; } @@ -5679,24 +5690,17 @@ SOperatorInfo* createSortMergeOperatorInfo(SOperatorInfo* downstream, SArray* pE pOperator->pTaskInfo = pTaskInfo; pOperator->exec = doSortedMerge; - pOperator->cleanupFn = destroyGlobalAggOperatorInfo; + pOperator->cleanupFn = destroySortMergeOperatorInfo; appendDownstream(pOperator, downstream); return pOperator; } -typedef struct SExternalMemSource { - SArray* pageIdList; - int32_t pageIndex; - int32_t rowIndex; - SSDataBlock *pBlock; -} SExternalMemSource; - int32_t msortComparFn(const void *pLeft, const void *pRight, void *param) { int32_t pLeftIdx = *(int32_t *)pLeft; int32_t pRightIdx = *(int32_t *)pRight; - SMsortComparParam *pParam = (SMsortComparParam *)param; + SMsortComparParam *pParam = (SMsortComparParam *)param; SArray *pInfo = pParam->orderInfo; @@ -5704,16 +5708,16 @@ int32_t msortComparFn(const void *pLeft, const void *pRight, void *param) { SExternalMemSource* pRightSource = pParam->pSources[pRightIdx]; // this input is exhausted, set the special value to denote this - if (pLeftSource->rowIndex == -1) { + if (pLeftSource->src.rowIndex == -1) { return 1; } - if (pRightSource->rowIndex == -1) { + if (pRightSource->src.rowIndex == -1) { return -1; } - SSDataBlock* pLeftBlock = pLeftSource->pBlock; - SSDataBlock* pRightBlock = pRightSource->pBlock; + SSDataBlock* pLeftBlock = pLeftSource->src.pBlock; + SSDataBlock* pRightBlock = pRightSource->src.pBlock; for(int32_t i = 0; i < pInfo->size; ++i) { SBlockOrderInfo* pOrder = TARRAY_GET_ELEM(pInfo, i); @@ -5722,13 +5726,13 @@ int32_t msortComparFn(const void *pLeft, const void *pRight, void *param) { bool leftNull = false; if (pLeftColInfoData->hasNull) { - leftNull = colDataIsNull(pLeftColInfoData, pLeftBlock->info.rows, pLeftSource->rowIndex, pLeftBlock->pBlockAgg); + leftNull = colDataIsNull(pLeftColInfoData, pLeftBlock->info.rows, pLeftSource->src.rowIndex, pLeftBlock->pBlockAgg); } SColumnInfoData* pRightColInfoData = TARRAY_GET_ELEM(pRightBlock->pDataBlock, pOrder->colIndex); bool rightNull = false; if (pRightColInfoData->hasNull) { - rightNull = colDataIsNull(pRightColInfoData, pRightBlock->info.rows, pRightSource->rowIndex, pRightBlock->pBlockAgg); + rightNull = colDataIsNull(pRightColInfoData, pRightBlock->info.rows, pRightSource->src.rowIndex, pRightBlock->pBlockAgg); } if (leftNull && rightNull) { @@ -5743,8 +5747,8 @@ int32_t msortComparFn(const void *pLeft, const void *pRight, void *param) { return pParam->nullFirst? -1:1; } - void* left1 = colDataGet(pLeftColInfoData, pLeftSource->rowIndex); - void* right1 = colDataGet(pRightColInfoData, pRightSource->rowIndex); + void* left1 = colDataGet(pLeftColInfoData, pLeftSource->src.rowIndex); + void* right1 = colDataGet(pRightColInfoData, pRightSource->src.rowIndex); switch(pLeftColInfoData->info.type) { case TSDB_DATA_TYPE_INT: { @@ -5767,137 +5771,108 @@ int32_t msortComparFn(const void *pLeft, const void *pRight, void *param) { } } -static int32_t adjustMergeTreeForNextTuple(SExternalMemSource *pSource, SMultiwayMergeTreeInfo *pTree, SOrderOperatorInfo* pInfo) { - /* - * load a new SDataBlock into memory of a given intermediate data-set source, - * since it's last record in buffer has been chosen to be processed, as the winner of loser-tree - */ - if (pSource->rowIndex >= pSource->pBlock->info.rows) { - pSource->rowIndex = 0; - pSource->pageIndex += 1; - - if (pSource->pageIndex >= taosArrayGetSize(pSource->pageIdList)) { - pInfo->numOfCompleted += 1; - pSource->rowIndex = -1; - pSource->pageIndex = -1; - pSource->pBlock = blockDataDestroy(pSource->pBlock); - } else { - SPageInfo* pPgInfo = *(SPageInfo**)taosArrayGet(pSource->pageIdList, pSource->pageIndex); - - SFilePage* pPage = getBufPage(pInfo->pSortInternalBuf, getPageId(pPgInfo)); - int32_t code = blockDataFromBuf(pSource->pBlock, pPage->data); - if (code != TSDB_CODE_SUCCESS) { - return code; - } - - releaseBufPage(pInfo->pSortInternalBuf, pPage); - } - } - - /* - * Adjust loser tree otherwise, according to new candidate data - * if the loser tree is rebuild completed, we do not need to adjust - */ - int32_t leafNodeIndex = tMergeTreeGetAdjustIndex(pTree); - -#ifdef _DEBUG_VIEW - printf("before adjust:\t"); - tMergeTreePrint(pTree); -#endif - - tMergeTreeAdjust(pTree, leafNodeIndex); - -#ifdef _DEBUG_VIEW - printf("\nafter adjust:\t"); - tMergeTreePrint(pTree); -#endif -} - -static void appendOneRowToDataBlock(SSDataBlock *pBlock, const SSDataBlock* pSource, int32_t* rowIndex) { +static void appendOneRowToDataBlock(SSDataBlock *pBlock, STupleHandle* pTupleHandle) { for (int32_t i = 0; i < pBlock->info.numOfCols; ++i) { SColumnInfoData* pColInfo = taosArrayGet(pBlock->pDataBlock, i); - SColumnInfoData* pSrcColInfo = taosArrayGet(pSource->pDataBlock, i); - bool isNull = colDataIsNull(pSrcColInfo, pSource->info.rows, *rowIndex, NULL); - + bool isNull = sortIsValueNull(pTupleHandle, i); if (isNull) { colDataAppend(pColInfo, pBlock->info.rows, NULL, true); } else { - char* pData = colDataGet(pSrcColInfo, *rowIndex); + char* pData = sortGetValue(pTupleHandle, i); colDataAppend(pColInfo, pBlock->info.rows, pData, false); } } pBlock->info.rows += 1; - *rowIndex += 1; } -static int32_t doAddNewSource(SOrderOperatorInfo* pInfo, SArray* pAllSources, int32_t numOfCols) { - SExternalMemSource* pSource = calloc(1, sizeof(SExternalMemSource)); - if (pSource == NULL) { - return TSDB_CODE_QRY_OUT_OF_MEMORY; - } +static SSDataBlock* createDataBlock(const SSDataBlock* pDataBlock) { + int32_t numOfCols = pDataBlock->info.numOfCols; - pSource->pageIdList = getDataBufPagesIdList(pInfo->pSortInternalBuf, pInfo->sourceId); - pSource->pBlock = calloc(1, sizeof(SSDataBlock)); - pSource->pBlock->pDataBlock = taosArrayInit(numOfCols, sizeof(SColumnInfoData)); - pSource->pBlock->info.numOfCols = numOfCols; + SSDataBlock* pBlock = calloc(1, sizeof(SSDataBlock)); + pBlock->pDataBlock = taosArrayInit(numOfCols, sizeof(SColumnInfoData)); + pBlock->info.numOfCols = numOfCols; for(int32_t i = 0; i < numOfCols; ++i) { SColumnInfoData colInfo = {0}; - SColumnInfoData* p = taosArrayGet(pInfo->pDataBlock->pDataBlock, i); + SColumnInfoData* p = taosArrayGet(pDataBlock->pDataBlock, i); colInfo.info = p->info; - taosArrayPush(pSource->pBlock->pDataBlock, &colInfo); + taosArrayPush(pBlock->pDataBlock, &colInfo); } - taosArrayPush(pAllSources, &pSource); - - pInfo->sourceId += 1; - - int32_t rowSize = blockDataGetSerialRowSize(pSource->pBlock); - int32_t numOfRows = (getBufPageSize(pInfo->pSortInternalBuf) - blockDataGetSerialMetaSize(pInfo->pDataBlock))/rowSize; - - return blockDataEnsureCapacity(pSource->pBlock, numOfRows); + return pBlock; } -void addToDiskbasedBuf(SOrderOperatorInfo* pInfo, SArray* pSources, jmp_buf env) { - int32_t start = 0; +//static int32_t doAddNewExternalMemSource(SOrderOperatorInfo* pInfo, SArray* pAllSources, SSDataBlock* pBlock) { +// SExternalMemSource* pSource = calloc(1, sizeof(SExternalMemSource)); +// if (pSource == NULL) { +// return TSDB_CODE_QRY_OUT_OF_MEMORY; +// } +// +// pSource->pageIdList = getDataBufPagesIdList(pInfo->pSortInternalBuf, pInfo->sourceId); +// pSource->src.pBlock = pBlock; +// +// taosArrayPush(pAllSources, &pSource); +// +// pInfo->sourceId += 1; +// +// int32_t rowSize = blockDataGetSerialRowSize(pSource->src.pBlock); +// int32_t numOfRows = (getBufPageSize(pInfo->pSortInternalBuf) - blockDataGetSerialMetaSize(pInfo->pDataBlock))/rowSize; +// return blockDataEnsureCapacity(pSource->src.pBlock, numOfRows); +//} - while(start < pInfo->pDataBlock->info.rows) { - int32_t stop = 0; - blockDataSplitRows(pInfo->pDataBlock, pInfo->hasVarCol, start, &stop, getBufPageSize(pInfo->pSortInternalBuf)); - SSDataBlock* p = blockDataExtractBlock(pInfo->pDataBlock, start, stop - start + 1); - if (p == NULL) { - longjmp(env, TSDB_CODE_QRY_OUT_OF_MEMORY); - } +//static int32_t doAddNewOperatorSource(SArray* pAllSources, SSDataBlock* pBlock, int32_t capacity) { +// SOperatorSource* pSource = calloc(1, sizeof(SOperatorSource)); +// if (pSource == NULL) { +// return TSDB_CODE_QRY_OUT_OF_MEMORY; +// } +// +// pSource->src.pBlock = pBlock; +// taosArrayPush(pAllSources, &pSource); +// +// return blockDataEnsureCapacity(pSource->src.pBlock, capacity); +//} - int32_t pageId = -1; - SFilePage* pPage = getNewDataBuf(pInfo->pSortInternalBuf, pInfo->sourceId, &pageId); - if (pPage == NULL) { - assert(0); - longjmp(env, terrno); - } - - int32_t size = blockDataGetSize(p) + sizeof(int32_t) + p->info.numOfCols * sizeof(int32_t); - assert(size <= getBufPageSize(pInfo->pSortInternalBuf)); - - blockDataToBuf(pPage->data, p); - - setBufPageDirty(pPage, true); - releaseBufPage(pInfo->pSortInternalBuf, pPage); - - blockDataDestroy(p); - start = stop + 1; - } - - int32_t numOfCols = pInfo->pDataBlock->info.numOfCols; - blockDataClearup(pInfo->pDataBlock, pInfo->hasVarCol); - - int32_t code = doAddNewSource(pInfo, pSources, numOfCols); - if (code != TSDB_CODE_SUCCESS) { - longjmp(env, code); - } -} +//void addToDiskbasedBuf(SOrderOperatorInfo* pInfo, SArray* pSources, jmp_buf env) { +// int32_t start = 0; +// +// while(start < pInfo->pDataBlock->info.rows) { +// int32_t stop = 0; +// blockDataSplitRows(pInfo->pDataBlock, pInfo->hasVarCol, start, &stop, getBufPageSize(pInfo->pSortInternalBuf)); +// SSDataBlock* p = blockDataExtractBlock(pInfo->pDataBlock, start, stop - start + 1); +// if (p == NULL) { +// longjmp(env, TSDB_CODE_QRY_OUT_OF_MEMORY); +// } +// +// int32_t pageId = -1; +// SFilePage* pPage = getNewDataBuf(pInfo->pSortInternalBuf, pInfo->sourceId, &pageId); +// if (pPage == NULL) { +// assert(0); +// longjmp(env, terrno); +// } +// +// int32_t size = blockDataGetSize(p) + sizeof(int32_t) + p->info.numOfCols * sizeof(int32_t); +// assert(size <= getBufPageSize(pInfo->pSortInternalBuf)); +// +// blockDataToBuf(pPage->data, p); +// +// setBufPageDirty(pPage, true); +// releaseBufPage(pInfo->pSortInternalBuf, pPage); +// +// blockDataDestroy(p); +// start = stop + 1; +// } +// +// int32_t numOfCols = pInfo->pDataBlock->info.numOfCols; +// blockDataClearup(pInfo->pDataBlock, pInfo->hasVarCol); +// +// SSDataBlock* pBlock = createDataBlock(pInfo->pDataBlock); +// int32_t code = doAddNewExternalMemSource(pInfo, pSources, pBlock); +// if (code != TSDB_CODE_SUCCESS) { +// longjmp(env, code); +// } +//} static int32_t sortComparInit(SMsortComparParam* cmpParam, SArray* pSources, int32_t startIndex, int32_t endIndex, SDiskbasedBuf* pBuf) { cmpParam->pSources = taosArrayGet(pSources, startIndex); @@ -5908,7 +5883,7 @@ static int32_t sortComparInit(SMsortComparParam* cmpParam, SArray* pSources, int SPageInfo* pPgInfo = *(SPageInfo**) taosArrayGet(pSource->pageIdList, pSource->pageIndex); SFilePage* pPage = getBufPage(pBuf, getPageId(pPgInfo)); - int32_t code = blockDataFromBuf(cmpParam->pSources[i]->pBlock, pPage->data); + int32_t code = blockDataFromBuf(cmpParam->pSources[i]->src.pBlock, pPage->data); if (code != TSDB_CODE_SUCCESS) { return code; } @@ -5922,31 +5897,23 @@ static int32_t sortComparInit(SMsortComparParam* cmpParam, SArray* pSources, int static int32_t sortComparClearup(SMsortComparParam* cmpParam) { for(int32_t i = 0; i < cmpParam->numOfSources; ++i) { SExternalMemSource* pSource = cmpParam->pSources[i]; - blockDataDestroy(pSource->pBlock); + blockDataDestroy(pSource->src.pBlock); tfree(pSource); } cmpParam->numOfSources = 0; } -static SSDataBlock* getSortedBlockData(SExecTaskInfo* pTaskInfo, SOrderOperatorInfo* pInfo, SMsortComparParam* cmpParam, int32_t capacity) { +static SSDataBlock* getSortedBlockData(SSortHandle* pHandle, SOrderOperatorInfo* pInfo, int32_t capacity) { blockDataClearup(pInfo->pDataBlock, pInfo->hasVarCol); while(1) { - if (cmpParam->numOfSources == pInfo->numOfCompleted) { - break; - } - - int32_t index = tMergeTreeGetChosenIndex(pInfo->pMergeTree); - - SExternalMemSource *pSource = (*cmpParam).pSources[index]; - appendOneRowToDataBlock(pInfo->pDataBlock, pSource->pBlock, &pSource->rowIndex); - - int32_t code = adjustMergeTreeForNextTuple(pSource, pInfo->pMergeTree, pInfo); - if (code != TSDB_CODE_SUCCESS) { - longjmp(pTaskInfo->env, code); - } + STupleHandle* pTupleHandle = sortNextTuple(pHandle); + if (pTupleHandle == NULL) { + break; + } + appendOneRowToDataBlock(pInfo->pDataBlock, pTupleHandle); if (pInfo->pDataBlock->info.rows >= capacity) { return pInfo->pDataBlock; } @@ -5955,106 +5922,10 @@ static SSDataBlock* getSortedBlockData(SExecTaskInfo* pTaskInfo, SOrderOperatorI return (pInfo->pDataBlock->info.rows > 0)? pInfo->pDataBlock:NULL; } -static int32_t doInternalSort(SExecTaskInfo* pTaskInfo, SOrderOperatorInfo* pInfo) { - size_t numOfSources = taosArrayGetSize(pInfo->pSources); - - // Calculate the I/O counts to complete the data sort. - double sortCount = floorl(log2(numOfSources) / log2(getNumOfInMemBufPages(pInfo->pSortInternalBuf))); - - pInfo->totalElapsed = taosGetTimestampUs() - pInfo->startTs; - qDebug("%s %d rounds mergesort required to complete the sort, first-round sorted data size:%"PRIzu", sort:%"PRId64", total elapsed:%"PRId64, - GET_TASKID(pTaskInfo), (int32_t) (sortCount + 1), getTotalBufSize(pInfo->pSortInternalBuf), pInfo->sortElapsed, - pInfo->totalElapsed); - - size_t pgSize = getBufPageSize(pInfo->pSortInternalBuf); - int32_t numOfRows = (pgSize - blockDataGetSerialMetaSize(pInfo->pDataBlock))/ blockDataGetSerialRowSize(pInfo->pDataBlock); - - blockDataEnsureCapacity(pInfo->pDataBlock, numOfRows); - - size_t numOfSorted = taosArrayGetSize(pInfo->pSources); - for(int32_t t = 0; t < sortCount; ++t) { - int64_t st = taosGetTimestampUs(); - - SArray* pResList = taosArrayInit(4, POINTER_BYTES); - SMsortComparParam resultParam = {.orderInfo = pInfo->cmpParam.orderInfo}; - - int32_t numOfInputSources = getNumOfInMemBufPages(pInfo->pSortInternalBuf); - int32_t sortGroup = (numOfSorted + numOfInputSources - 1) / numOfInputSources; - - // Only *numOfInputSources* can be loaded into buffer to perform the external sort. - for(int32_t i = 0; i < sortGroup; ++i) { - pInfo->sourceId += 1; - - int32_t end = (i + 1) * numOfInputSources - 1; - if (end > numOfSorted - 1) { - end = numOfSorted - 1; - } - - pInfo->cmpParam.numOfSources = end - i * numOfInputSources + 1; - - int32_t code = sortComparInit(&pInfo->cmpParam, pInfo->pSources, i * numOfInputSources, end, pInfo->pSortInternalBuf); - if (code != TSDB_CODE_SUCCESS) { - longjmp(pTaskInfo->env, code); - } - - code = tMergeTreeCreate(&pInfo->pMergeTree, pInfo->cmpParam.numOfSources, &pInfo->cmpParam, msortComparFn); - if (code != TSDB_CODE_SUCCESS) { - longjmp(pTaskInfo->env, code); - } - - while (1) { - SSDataBlock* pDataBlock = getSortedBlockData(pTaskInfo, pInfo, &pInfo->cmpParam, numOfRows); - if (pDataBlock == NULL) { - break; - } - - int32_t pageId = -1; - SFilePage* pPage = getNewDataBuf(pInfo->pSortInternalBuf, pInfo->sourceId, &pageId); - if (pPage == NULL) { - assert(0); - longjmp(pTaskInfo->env, terrno); - } - - int32_t size = blockDataGetSize(pDataBlock) + sizeof(int32_t) + pDataBlock->info.numOfCols * sizeof(int32_t); - assert(size <= getBufPageSize(pInfo->pSortInternalBuf)); - - blockDataToBuf(pPage->data, pDataBlock); - - setBufPageDirty(pPage, true); - releaseBufPage(pInfo->pSortInternalBuf, pPage); - - blockDataClearup(pDataBlock, pInfo->hasVarCol); - } - - tMergeTreeDestroy(pInfo->pMergeTree); - pInfo->numOfCompleted = 0; - - code = doAddNewSource(pInfo, pResList, pInfo->pDataBlock->info.numOfCols); - if (code != 0) { - longjmp(pTaskInfo->env, code); - } - } - - sortComparClearup(&pInfo->cmpParam); - - taosArrayClear(pInfo->pSources); - taosArrayAddAll(pInfo->pSources, pResList); - taosArrayDestroy(pResList); - - pInfo->cmpParam = resultParam; - numOfSorted = taosArrayGetSize(pInfo->pSources); - - int64_t el = taosGetTimestampUs() - st; - pInfo->totalElapsed += el; - - SDiskbasedBufStatis statis = getDBufStatis(pInfo->pSortInternalBuf); - - qDebug("%s %d round mergesort, elapsed:%"PRId64" readDisk:%.2f Kb, flushDisk:%.2f Kb", GET_TASKID(pTaskInfo), t + 1, el, statis.loadBytes/1024.0, - statis.flushBytes/1024.0); - } - - pInfo->cmpParam.numOfSources = taosArrayGetSize(pInfo->pSources); - return 0; +static SSDataBlock* loadNextDataBlock(void* param) { + bool newgroup = false; + SOperatorInfo* pOperator = (SOperatorInfo*) param; + return pOperator->pDownstream[0]->exec(pOperator->pDownstream[0], &newgroup); } static SSDataBlock* doSort(void* param, bool* newgroup) { @@ -6068,74 +5939,30 @@ static SSDataBlock* doSort(void* param, bool* newgroup) { SSDataBlock* pBlock = NULL; if (pOperator->status == OP_RES_TO_RETURN) { - return getSortedBlockData(pTaskInfo, pInfo, &pInfo->cmpParam, pInfo->numOfRowsInRes); + return getSortedBlockData(pInfo->pSortHandle, pInfo, pInfo->numOfRowsInRes); } int64_t st = taosGetTimestampUs(); - while(1) { - publishOperatorProfEvent(pOperator->pDownstream[0], QUERY_PROF_BEFORE_OPERATOR_EXEC); - pBlock = pOperator->pDownstream[0]->exec(pOperator->pDownstream[0], newgroup); - publishOperatorProfEvent(pOperator->pDownstream[0], QUERY_PROF_AFTER_OPERATOR_EXEC); + SSchema* p = blockDataExtractSchema(pInfo->pDataBlock, NULL); + int32_t numOfBufPage = pInfo->sortBufSize / pInfo->bufPageSize; + pInfo->pSortHandle = createSortHandle(pInfo->orderInfo, pInfo->nullFirst, SORT_SINGLESOURCE, pInfo->bufPageSize, + numOfBufPage, p, pInfo->pDataBlock->info.numOfCols, "GET_TASKID(pTaskInfo)"); - // start to flush data into disk and try do multiway merge sort - if (pBlock == NULL) { - break; - } + tfree(p); + setFetchRawDataFp(pInfo->pSortHandle, loadNextDataBlock); + setComparFn(pInfo->pSortHandle, msortComparFn); - int32_t code = blockDataMerge(pInfo->pDataBlock, pBlock); - if (code != TSDB_CODE_SUCCESS) { - longjmp(pOperator->pTaskInfo->env, code); - } + sortAddSource(pInfo->pSortHandle, pOperator); - size_t size = blockDataGetSize(pInfo->pDataBlock); - if (size > pInfo->sortBufSize) { - // Perform the in-memory sort and then flush data in the buffer into disk. - int64_t p = taosGetTimestampUs(); - blockDataSort(pInfo->pDataBlock, pInfo->cmpParam.orderInfo, pInfo->cmpParam.nullFirst); - - int64_t el = taosGetTimestampUs() - p; - pInfo->sortElapsed += el; - - addToDiskbasedBuf(pInfo, pInfo->pSources, pTaskInfo->env); - } - } - - if (pInfo->pDataBlock->info.rows > 0) { - // Perform the in-memory sort and then flush data in the buffer into disk. - blockDataSort(pInfo->pDataBlock, pInfo->cmpParam.orderInfo, pInfo->cmpParam.nullFirst); - - // All sorted data are resident in memory, external memory sort is not needed. - // Return to the upstream operator directly - if (isAllDataInMemBuf(pInfo->pSortInternalBuf)) { - pOperator->status = OP_EXEC_DONE; - return (pInfo->pDataBlock->info.rows == 0)? NULL:pInfo->pDataBlock; - } - - addToDiskbasedBuf(pInfo, pInfo->pSources, pTaskInfo->env); - } - - doInternalSort(pTaskInfo, pInfo); - - int32_t code = blockDataEnsureCapacity(pInfo->pDataBlock, pInfo->numOfRowsInRes); + // TODO set error code; + int32_t code = sortOpen(pInfo->pSortHandle); if (code != TSDB_CODE_SUCCESS) { - longjmp(pTaskInfo->env, code); - } - - int32_t numOfSources = taosArrayGetSize(pInfo->pSources); - ASSERT(numOfSources <= getNumOfInMemBufPages(pInfo->pSortInternalBuf)); - code = sortComparInit(&pInfo->cmpParam, pInfo->pSources, 0, numOfSources - 1, pInfo->pSortInternalBuf); - if (code != TSDB_CODE_SUCCESS) { - longjmp(pTaskInfo->env, code); - } - - code = tMergeTreeCreate(&pInfo->pMergeTree, pInfo->cmpParam.numOfSources, &pInfo->cmpParam, msortComparFn); - if (code != TSDB_CODE_SUCCESS) { - longjmp(pTaskInfo->env, code); + longjmp(pTaskInfo->env, terrno); } pOperator->status = OP_RES_TO_RETURN; - return getSortedBlockData(pTaskInfo, pInfo, &pInfo->cmpParam, pInfo->numOfRowsInRes); + return getSortedBlockData(pInfo->pSortHandle, pInfo, pInfo->numOfRowsInRes); } static SArray* createBlockOrder(SArray* pExprInfo, SArray* pOrderVal) { @@ -6176,8 +6003,7 @@ SOperatorInfo *createOrderOperatorInfo(SOperatorInfo* downstream, SArray* pExprI pInfo->numOfRowsInRes = 1024; pInfo->pDataBlock = createOutputBuf_rv(pExprInfo, pInfo->numOfRowsInRes); - pInfo->pSources = taosArrayInit(4, POINTER_BYTES); - pInfo->cmpParam.orderInfo = createBlockOrder(pExprInfo, pOrderVal); + pInfo->orderInfo = createBlockOrder(pExprInfo, pOrderVal); for(int32_t i = 0; i < taosArrayGetSize(pExprInfo); ++i) { SExprInfo* pExpr = taosArrayGetP(pExprInfo, i); @@ -6187,8 +6013,8 @@ SOperatorInfo *createOrderOperatorInfo(SOperatorInfo* downstream, SArray* pExprI } } - int32_t code = createDiskbasedBuffer(&pInfo->pSortInternalBuf, pInfo->bufPageSize, pInfo->sortBufSize, 1, "/tmp/"); - if (pInfo->pSources == NULL || code != 0 || pInfo->cmpParam.orderInfo == NULL || pInfo->pDataBlock == NULL) { +// int32_t code = createDiskbasedBuffer(&pInfo->pSortInternalBuf, pInfo->bufPageSize, pInfo->sortBufSize, 1, "/tmp/"); + if (pInfo->orderInfo == NULL || pInfo->pDataBlock == NULL) { tfree(pOperator); destroyOrderOperatorInfo(pInfo, taosArrayGetSize(pExprInfo)); tfree(pInfo); @@ -7236,10 +7062,7 @@ static void destroyOrderOperatorInfo(void* param, int32_t numOfOutput) { SOrderOperatorInfo* pInfo = (SOrderOperatorInfo*) param; pInfo->pDataBlock = blockDataDestroy(pInfo->pDataBlock); - taosArrayDestroy(pInfo->cmpParam.orderInfo); - destroyResultBuf(pInfo->pSortInternalBuf); - - tMergeTreeDestroy(pInfo->pMergeTree); + taosArrayDestroy(pInfo->orderInfo); } static void destroyConditionOperatorInfo(void* param, int32_t numOfOutput) { diff --git a/source/libs/executor/src/tsort.c b/source/libs/executor/src/tsort.c new file mode 100644 index 0000000000..4911146b54 --- /dev/null +++ b/source/libs/executor/src/tsort.c @@ -0,0 +1,694 @@ +/* + * 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 "common.h" +#include "query.h" + +#include "tsort.h" +#include "tep.h" +#include "tcfg.h" +#include "tlosertree.h" +#include "tpagedbuf.h" +#include "tutil.h" + +typedef struct SMsortComparParam { + void **pSources; + int32_t numOfSources; + SArray *orderInfo; // SArray + bool nullFirst; +} SMsortComparParam; + +typedef struct STupleHandle { + SSDataBlock* pBlock; + int32_t rowIndex; +} STupleHandle; + +typedef struct SSortHandle { + int32_t type; + + int32_t pageSize; + int32_t numOfPages; + SDiskbasedBuf *pBuf; + + SArray *pOrderInfo; + bool nullFirst; + bool hasVarCol; + + SArray *pSources; + SArray *pOrderedSource; + + _sort_fetch_block_fn_t fetchfp; + _sort_merge_compar_fn_t comparFn; + + void *pParam; + SMultiwayMergeTreeInfo *pMergeTree; + int32_t numOfCols; + + int64_t startTs; + uint64_t sortElapsed; + uint64_t totalElapsed; + + int32_t sourceId; + SSDataBlock *pDataBlock; + SMsortComparParam cmpParam; + int32_t numOfCompletedSources; + bool opened; + const char *idStr; + + bool inMemSort; + bool needAdjust; + STupleHandle tupleHandle; +} SSortHandle; + +SSDataBlock* createDataBlock_rv(SSchema* pSchema, int32_t numOfCols) { + SSDataBlock* pBlock = calloc(1, sizeof(SSDataBlock)); + pBlock->pDataBlock = taosArrayInit(numOfCols, sizeof(SColumnInfoData)); + pBlock->info.numOfCols = numOfCols; + + for(int32_t i = 0; i < numOfCols; ++i) { + SColumnInfoData colInfo = {0}; + + colInfo.info.type = pSchema[i].type; + colInfo.info.bytes = pSchema[i].bytes; + colInfo.info.colId = pSchema[i].colId; + taosArrayPush(pBlock->pDataBlock, &colInfo); + } + + return pBlock; +} + +/** + * + * @param type + * @return + */ +SSortHandle* createSortHandle(SArray* pOrderInfo, bool nullFirst, int32_t type, int32_t pageSize, int32_t numOfPages, SSchema* pSchema, int32_t numOfCols, const char* idstr) { + SSortHandle* pSortHandle = calloc(1, sizeof(SSortHandle)); + + pSortHandle->type = type; + pSortHandle->pageSize = pageSize; + pSortHandle->numOfPages = numOfPages; + pSortHandle->pOrderedSource = taosArrayInit(4, POINTER_BYTES); + pSortHandle->pSources = taosArrayInit(4, POINTER_BYTES); + pSortHandle->pOrderInfo = pOrderInfo; + pSortHandle->nullFirst = nullFirst; + pSortHandle->cmpParam.orderInfo = pOrderInfo; + + pSortHandle->pDataBlock = createDataBlock_rv(pSchema, numOfCols); + + if (idstr != NULL) { + pSortHandle->idStr = strdup(idstr); + } + + return pSortHandle; +} + +void destroySortHandle(SSortHandle* pSortHandle) { + sortClose(pSortHandle); + + taosArrayDestroy(pSortHandle->pSources); + if (pSortHandle->pMergeTree != NULL) { + tMergeTreeDestroy(pSortHandle->pMergeTree); + } + + tfree(pSortHandle->idStr); + tfree(pSortHandle); +} + +int32_t sortAddSource(SSortHandle* pSortHandle, void* pSource) { + if (pSortHandle->type == SORT_SINGLESOURCE) { + pSortHandle->pParam = pSource; + } else { + taosArrayPush(pSortHandle->pOrderedSource, &pSource); + } +} + +static SSDataBlock* createDataBlock(const SSDataBlock* pDataBlock) { + int32_t numOfCols = pDataBlock->info.numOfCols; + + SSDataBlock* pBlock = calloc(1, sizeof(SSDataBlock)); + pBlock->pDataBlock = taosArrayInit(numOfCols, sizeof(SColumnInfoData)); + pBlock->info.numOfCols = numOfCols; + + for(int32_t i = 0; i < numOfCols; ++i) { + SColumnInfoData colInfo = {0}; + SColumnInfoData* p = taosArrayGet(pDataBlock->pDataBlock, i); + colInfo.info = p->info; + taosArrayPush(pBlock->pDataBlock, &colInfo); + } + + return pBlock; +} + +static int32_t doAddNewExternalMemSource(SDiskbasedBuf *pBuf, SArray* pAllSources, SSDataBlock* pBlock, int32_t* sourceId) { + SExternalMemSource* pSource = calloc(1, sizeof(SExternalMemSource)); + if (pSource == NULL) { + return TSDB_CODE_QRY_OUT_OF_MEMORY; + } + + pSource->pageIdList = getDataBufPagesIdList(pBuf, (*sourceId)); + pSource->src.pBlock = pBlock; + + taosArrayPush(pAllSources, &pSource); + + (*sourceId) += 1; + + int32_t rowSize = blockDataGetSerialRowSize(pSource->src.pBlock); + int32_t numOfRows = (getBufPageSize(pBuf) - blockDataGetSerialMetaSize(pBlock))/rowSize; + + return blockDataEnsureCapacity(pSource->src.pBlock, numOfRows); +} + +static int32_t doAddToBuf(SSDataBlock* pDataBlock, SSortHandle* pHandle) { + int32_t start = 0; + + if (pHandle->pBuf == NULL) { + int32_t code = createDiskbasedBuffer(&pHandle->pBuf, pHandle->pageSize, pHandle->numOfPages * pHandle->pageSize, 0, "/tmp"); + if (code != TSDB_CODE_SUCCESS) { + return code; + } + } + + while(start < pDataBlock->info.rows) { + int32_t stop = 0; + blockDataSplitRows(pDataBlock, pHandle->hasVarCol, start, &stop, pHandle->pageSize); + SSDataBlock* p = blockDataExtractBlock(pDataBlock, start, stop - start + 1); + if (p == NULL) { + return terrno; + } + + int32_t pageId = -1; + SFilePage* pPage = getNewDataBuf(pHandle->pBuf, pHandle->sourceId, &pageId); + if (pPage == NULL) { + return terrno; + } + + int32_t size = blockDataGetSize(p) + sizeof(int32_t) + p->info.numOfCols * sizeof(int32_t); + assert(size <= getBufPageSize(pHandle->pBuf)); + + blockDataToBuf(pPage->data, p); + + setBufPageDirty(pPage, true); + releaseBufPage(pHandle->pBuf, pPage); + + blockDataDestroy(p); + start = stop + 1; + } + + blockDataClearup(pDataBlock, pHandle->hasVarCol); + + SSDataBlock* pBlock = createDataBlock(pDataBlock); + int32_t code = doAddNewExternalMemSource(pHandle->pBuf, pHandle->pOrderedSource, pBlock, &pHandle->sourceId); + if (code != TSDB_CODE_SUCCESS) { + return code; + } +} + +static int32_t sortComparInit(SMsortComparParam* cmpParam, SArray* pSources, int32_t startIndex, int32_t endIndex, SSortHandle* pHandle) { + cmpParam->pSources = taosArrayGet(pSources, startIndex); + cmpParam->numOfSources = (endIndex - startIndex + 1); + + int32_t code = 0; + + if (pHandle->type == SORT_SINGLESOURCE) { + for (int32_t i = 0; i < cmpParam->numOfSources; ++i) { + SExternalMemSource* pSource = cmpParam->pSources[i]; + SPageInfo* pPgInfo = *(SPageInfo**)taosArrayGet(pSource->pageIdList, pSource->pageIndex); + + SFilePage* pPage = getBufPage(pHandle->pBuf, getPageId(pPgInfo)); + code = blockDataFromBuf(pSource->src.pBlock, pPage->data); + if (code != TSDB_CODE_SUCCESS) { + return code; + } + + releaseBufPage(pHandle->pBuf, pPage); + } + } else { + // multi-pass internal merge sort is required + if (pHandle->pBuf == NULL) { + code = createDiskbasedBuffer(&pHandle->pBuf, pHandle->pageSize, pHandle->numOfPages * pHandle->pageSize, 0, "/tmp"); + if (code != TSDB_CODE_SUCCESS) { + return code; + } + } + + for (int32_t i = 0; i < cmpParam->numOfSources; ++i) { + SOperatorSource* pSource = cmpParam->pSources[i]; + pSource->src.pBlock = pHandle->fetchfp(pSource->param); + } + } + + return code; +} + +static int32_t sortComparClearup(SMsortComparParam* cmpParam) { + for(int32_t i = 0; i < cmpParam->numOfSources; ++i) { + SExternalMemSource* pSource = cmpParam->pSources[i]; + blockDataDestroy(pSource->src.pBlock); + tfree(pSource); + } + + cmpParam->numOfSources = 0; +} + +static void appendOneRowToDataBlock(SSDataBlock *pBlock, const SSDataBlock* pSource, int32_t* rowIndex) { + for (int32_t i = 0; i < pBlock->info.numOfCols; ++i) { + SColumnInfoData* pColInfo = taosArrayGet(pBlock->pDataBlock, i); + + SColumnInfoData* pSrcColInfo = taosArrayGet(pSource->pDataBlock, i); + bool isNull = colDataIsNull(pSrcColInfo, pSource->info.rows, *rowIndex, NULL); + + if (isNull) { + colDataAppend(pColInfo, pBlock->info.rows, NULL, true); + } else { + char* pData = colDataGet(pSrcColInfo, *rowIndex); + colDataAppend(pColInfo, pBlock->info.rows, pData, false); + } + } + + pBlock->info.rows += 1; + *rowIndex += 1; +} + +static int32_t adjustMergeTreeForNextTuple(SExternalMemSource *pSource, SMultiwayMergeTreeInfo *pTree, SSortHandle *pHandle, int32_t* numOfCompleted) { + /* + * load a new SDataBlock into memory of a given intermediate data-set source, + * since it's last record in buffer has been chosen to be processed, as the winner of loser-tree + */ + if (pSource->src.rowIndex >= pSource->src.pBlock->info.rows) { + pSource->src.rowIndex = 0; + pSource->pageIndex += 1; + + if (pSource->pageIndex >= taosArrayGetSize(pSource->pageIdList)) { + (*numOfCompleted) += 1; + pSource->src.rowIndex = -1; + pSource->pageIndex = -1; + pSource->src.pBlock = blockDataDestroy(pSource->src.pBlock); + } else { + if (pHandle->type == SORT_SINGLESOURCE) { + SPageInfo* pPgInfo = *(SPageInfo**)taosArrayGet(pSource->pageIdList, pSource->pageIndex); + + SFilePage* pPage = getBufPage(pHandle->pBuf, getPageId(pPgInfo)); + int32_t code = blockDataFromBuf(pSource->src.pBlock, pPage->data); + if (code != TSDB_CODE_SUCCESS) { + return code; + } + + releaseBufPage(pHandle->pBuf, pPage); + } else { + pSource->src.pBlock = pHandle->fetchfp(((SOperatorSource*)pSource)->param); + if (pSource->src.pBlock == NULL) { + (*numOfCompleted) += 1; + pSource->src.rowIndex = -1; + } + } + } + } + + /* + * Adjust loser tree otherwise, according to new candidate data + * if the loser tree is rebuild completed, we do not need to adjust + */ + int32_t leafNodeIndex = tMergeTreeGetAdjustIndex(pTree); + +#ifdef _DEBUG_VIEW + printf("before adjust:\t"); + tMergeTreePrint(pTree); +#endif + + tMergeTreeAdjust(pTree, leafNodeIndex); + +#ifdef _DEBUG_VIEW + printf("\nafter adjust:\t"); + tMergeTreePrint(pTree); +#endif +} + +static SSDataBlock* getSortedBlockData(SSortHandle* pHandle, SMsortComparParam* cmpParam, int32_t capacity) { + blockDataClearup(pHandle->pDataBlock, pHandle->hasVarCol); + + while(1) { + if (cmpParam->numOfSources == pHandle->numOfCompletedSources) { + break; + } + + int32_t index = tMergeTreeGetChosenIndex(pHandle->pMergeTree); + + SExternalMemSource *pSource = (*cmpParam).pSources[index]; + appendOneRowToDataBlock(pHandle->pDataBlock, pSource->src.pBlock, &pSource->src.rowIndex); + + int32_t code = adjustMergeTreeForNextTuple(pSource, pHandle->pMergeTree, pHandle, &pHandle->numOfCompletedSources); + if (code != TSDB_CODE_SUCCESS) { + terrno = code; + return NULL; + } + + if (pHandle->pDataBlock->info.rows >= capacity) { + return pHandle->pDataBlock; + } + } + + return (pHandle->pDataBlock->info.rows > 0)? pHandle->pDataBlock:NULL; +} + +static int32_t msortComparFn(const void *pLeft, const void *pRight, void *param) { + int32_t pLeftIdx = *(int32_t *)pLeft; + int32_t pRightIdx = *(int32_t *)pRight; + + SMsortComparParam *pParam = (SMsortComparParam *)param; + + SArray *pInfo = pParam->orderInfo; + + SExternalMemSource* pLeftSource = pParam->pSources[pLeftIdx]; + SExternalMemSource* pRightSource = pParam->pSources[pRightIdx]; + + // this input is exhausted, set the special value to denote this + if (pLeftSource->src.rowIndex == -1) { + return 1; + } + + if (pRightSource->src.rowIndex == -1) { + return -1; + } + + SSDataBlock* pLeftBlock = pLeftSource->src.pBlock; + SSDataBlock* pRightBlock = pRightSource->src.pBlock; + + for(int32_t i = 0; i < pInfo->size; ++i) { + SBlockOrderInfo* pOrder = TARRAY_GET_ELEM(pInfo, i); + + SColumnInfoData* pLeftColInfoData = TARRAY_GET_ELEM(pLeftBlock->pDataBlock, pOrder->colIndex); + + bool leftNull = false; + if (pLeftColInfoData->hasNull) { + leftNull = colDataIsNull(pLeftColInfoData, pLeftBlock->info.rows, pLeftSource->src.rowIndex, pLeftBlock->pBlockAgg); + } + + SColumnInfoData* pRightColInfoData = TARRAY_GET_ELEM(pRightBlock->pDataBlock, pOrder->colIndex); + bool rightNull = false; + if (pRightColInfoData->hasNull) { + rightNull = colDataIsNull(pRightColInfoData, pRightBlock->info.rows, pRightSource->src.rowIndex, pRightBlock->pBlockAgg); + } + + if (leftNull && rightNull) { + continue; // continue to next slot + } + + if (rightNull) { + return pParam->nullFirst? 1:-1; + } + + if (leftNull) { + return pParam->nullFirst? -1:1; + } + + void* left1 = colDataGet(pLeftColInfoData, pLeftSource->src.rowIndex); + void* right1 = colDataGet(pRightColInfoData, pRightSource->src.rowIndex); + + switch(pLeftColInfoData->info.type) { + case TSDB_DATA_TYPE_INT: { + int32_t leftv = *(int32_t*)left1; + int32_t rightv = *(int32_t*)right1; + + if (leftv == rightv) { + break; + } else { + if (pOrder->order == TSDB_ORDER_ASC) { + return leftv < rightv? -1 : 1; + } else { + return leftv < rightv? 1 : -1; + } + } + } + default: + assert(0); + } + } +} + +static int32_t doInternalMergeSort(SSortHandle* pHandle) { + size_t numOfSources = taosArrayGetSize(pHandle->pOrderedSource); + + // Calculate the I/O counts to complete the data sort. + double sortPass = floorl(log2(numOfSources) / log2(pHandle->numOfPages)); + + pHandle->totalElapsed = taosGetTimestampUs() - pHandle->startTs; + qDebug("%s %d rounds mergesort required to complete the sort, first-round sorted data size:%"PRIzu", sort:%"PRId64", total elapsed:%"PRId64, + pHandle->idStr, (int32_t) (sortPass + 1), getTotalBufSize(pHandle->pBuf), pHandle->sortElapsed, pHandle->totalElapsed); + + size_t pgSize = pHandle->pageSize; + int32_t numOfRows = (pgSize - blockDataGetSerialMetaSize(pHandle->pDataBlock))/ blockDataGetSerialRowSize(pHandle->pDataBlock); + + blockDataEnsureCapacity(pHandle->pDataBlock, numOfRows); + + size_t numOfSorted = taosArrayGetSize(pHandle->pOrderedSource); + for(int32_t t = 0; t < sortPass; ++t) { + int64_t st = taosGetTimestampUs(); + + SArray* pResList = taosArrayInit(4, POINTER_BYTES); + + int32_t numOfInputSources = pHandle->numOfPages; + int32_t sortGroup = (numOfSorted + numOfInputSources - 1) / numOfInputSources; + + // Only *numOfInputSources* can be loaded into buffer to perform the external sort. + for(int32_t i = 0; i < sortGroup; ++i) { + pHandle->sourceId += 1; + + int32_t end = (i + 1) * numOfInputSources - 1; + if (end > numOfSorted - 1) { + end = numOfSorted - 1; + } + + pHandle->cmpParam.numOfSources = end - i * numOfInputSources + 1; + + int32_t code = sortComparInit(&pHandle->cmpParam, pHandle->pOrderedSource, i * numOfInputSources, end, pHandle); + if (code != TSDB_CODE_SUCCESS) { + return code; + } + + code = tMergeTreeCreate(&pHandle->pMergeTree, pHandle->cmpParam.numOfSources, &pHandle->cmpParam, pHandle->comparFn); + if (code != TSDB_CODE_SUCCESS) { + return code; + } + + while (1) { + SSDataBlock* pDataBlock = getSortedBlockData(pHandle, &pHandle->cmpParam, numOfRows); + if (pDataBlock == NULL) { + break; + } + + int32_t pageId = -1; + SFilePage* pPage = getNewDataBuf(pHandle->pBuf, pHandle->sourceId, &pageId); + if (pPage == NULL) { + return terrno; + } + + int32_t size = blockDataGetSize(pDataBlock) + sizeof(int32_t) + pDataBlock->info.numOfCols * sizeof(int32_t); + assert(size <= getBufPageSize(pHandle->pBuf)); + + blockDataToBuf(pPage->data, pDataBlock); + + setBufPageDirty(pPage, true); + releaseBufPage(pHandle->pBuf, pPage); + + blockDataClearup(pDataBlock, pHandle->hasVarCol); + } + + tMergeTreeDestroy(pHandle->pMergeTree); + pHandle->numOfCompletedSources = 0; + + SSDataBlock* pBlock = createDataBlock(pHandle->pDataBlock); + code = doAddNewExternalMemSource(pHandle->pBuf, pResList, pBlock, &pHandle->sourceId); + if (code != 0) { + return code; + } + } + + sortComparClearup(&pHandle->cmpParam); + + taosArrayClear(pHandle->pOrderedSource); + taosArrayAddAll(pHandle->pOrderedSource, pResList); + taosArrayDestroy(pResList); + + numOfSorted = taosArrayGetSize(pHandle->pOrderedSource); + + int64_t el = taosGetTimestampUs() - st; + pHandle->totalElapsed += el; + + SDiskbasedBufStatis statis = getDBufStatis(pHandle->pBuf); + qDebug("%s %d round mergesort, elapsed:%"PRId64" readDisk:%.2f Kb, flushDisk:%.2f Kb", pHandle->idStr, t + 1, el, statis.loadBytes/1024.0, + statis.flushBytes/1024.0); + + if (pHandle->type == SORT_MULTIWAY_MERGE) { + pHandle->type = SORT_SINGLESOURCE; + pHandle->comparFn = msortComparFn; + } + } + + pHandle->cmpParam.numOfSources = taosArrayGetSize(pHandle->pOrderedSource); + return 0; +} + +int32_t sortOpen(SSortHandle* pHandle) { + if (pHandle->opened) { + return 0; + } + + if (pHandle->fetchfp == NULL || pHandle->comparFn == NULL) { + return -1; + } + + pHandle->opened = true; + + size_t sortBufSize = pHandle->numOfPages * pHandle->pageSize; + if (pHandle->type == SORT_SINGLESOURCE) { + if (pHandle->pParam == NULL) { + qError("%s sort source not set yet", pHandle->idStr); + return -1; + } + + while (1) { + SSDataBlock* pBlock = pHandle->fetchfp(pHandle->pParam); + if (pBlock == NULL) { + break; + } + + if (pHandle->pDataBlock == NULL) { + pHandle->pDataBlock = createDataBlock(pBlock); + } + + int32_t code = blockDataMerge(pHandle->pDataBlock, pBlock); + if (code != 0) { + return code; + } + + size_t size = blockDataGetSize(pHandle->pDataBlock); + if (size > sortBufSize) { + // Perform the in-memory sort and then flush data in the buffer into disk. + int64_t p = taosGetTimestampUs(); + blockDataSort(pHandle->pDataBlock, pHandle->pOrderInfo, pHandle->nullFirst); + + int64_t el = taosGetTimestampUs() - p; + pHandle->sortElapsed += el; + + doAddToBuf(pHandle->pDataBlock, pHandle); + } + } + + if (pHandle->pDataBlock->info.rows > 0) { + size_t size = blockDataGetSize(pHandle->pDataBlock); + + // Perform the in-memory sort and then flush data in the buffer into disk. + blockDataSort(pHandle->pDataBlock, pHandle->pOrderInfo, pHandle->nullFirst); + + // All sorted data can fit in memory, external memory sort is not needed. Return to directly + if (size <= sortBufSize) { + pHandle->cmpParam.numOfSources = 1; + pHandle->inMemSort = true; + + pHandle->tupleHandle.rowIndex = -1; + pHandle->tupleHandle.pBlock = pHandle->pDataBlock; + return 0; + } else { + doAddToBuf(pHandle->pDataBlock, pHandle); + } + } + } else { + // do nothing + } + + // do internal sort + int32_t code = doInternalMergeSort(pHandle); + if (code != TSDB_CODE_SUCCESS) { + return code; + } + + int32_t numOfSources = taosArrayGetSize(pHandle->pOrderedSource); + ASSERT(numOfSources <= getNumOfInMemBufPages(pHandle->pBuf)); + code = sortComparInit(&pHandle->cmpParam, pHandle->pOrderedSource, 0, numOfSources - 1, pHandle); + if (code != TSDB_CODE_SUCCESS) { + return code; + } + + code = tMergeTreeCreate(&pHandle->pMergeTree, pHandle->cmpParam.numOfSources, &pHandle->cmpParam, pHandle->comparFn); + if (code != TSDB_CODE_SUCCESS) { + return code; + } +} + +int32_t sortClose(SSortHandle* pHandle) { + // do nothing +} + +int32_t setFetchRawDataFp(SSortHandle* pHandle, _sort_fetch_block_fn_t fp) { + pHandle->fetchfp = fp; +} + +int32_t setComparFn(SSortHandle* pHandle, _sort_merge_compar_fn_t fp) { + pHandle->comparFn = fp; +} + +STupleHandle* sortNextTuple(SSortHandle* pHandle) { + if (pHandle->cmpParam.numOfSources == pHandle->numOfCompletedSources) { + return NULL; + } + + if (pHandle->inMemSort) { + pHandle->tupleHandle.rowIndex += 1; + if (pHandle->tupleHandle.rowIndex == pHandle->pDataBlock->info.rows) { + pHandle->numOfCompletedSources = 1; + return NULL; + } + + return &pHandle->tupleHandle; + } + + int32_t index = tMergeTreeGetChosenIndex(pHandle->pMergeTree); + SExternalMemSource *pSource = pHandle->cmpParam.pSources[index]; + + if (pHandle->needAdjust) { + int32_t code = adjustMergeTreeForNextTuple(pSource, pHandle->pMergeTree, pHandle, &pHandle->numOfCompletedSources); + if (code != TSDB_CODE_SUCCESS) { + terrno = code; + return NULL; + } + } + + if (pHandle->cmpParam.numOfSources == pHandle->numOfCompletedSources) { + return NULL; + } + + index = tMergeTreeGetChosenIndex(pHandle->pMergeTree); + pSource = pHandle->cmpParam.pSources[index]; + + assert(pSource->src.pBlock != NULL); + + pHandle->tupleHandle.rowIndex = pSource->src.rowIndex; + pHandle->tupleHandle.pBlock = pSource->src.pBlock; + + pHandle->needAdjust = true; + pSource->src.rowIndex += 1; + + return &pHandle->tupleHandle; +} + +bool sortIsValueNull(STupleHandle* pVHandle, int32_t colIndex) { + return false; +} + +void* sortGetValue(STupleHandle* pVHandle, int32_t colIndex) { + SColumnInfoData* pColInfo = TARRAY_GET_ELEM(pVHandle->pBlock->pDataBlock, colIndex); + return colDataGet(pColInfo, pVHandle->rowIndex); +} diff --git a/source/libs/executor/test/executorTests.cpp b/source/libs/executor/test/executorTests.cpp index ebea6755d7..22df894cdf 100644 --- a/source/libs/executor/test/executorTests.cpp +++ b/source/libs/executor/test/executorTests.cpp @@ -121,6 +121,7 @@ int main(int argc, char** argv) { return RUN_ALL_TESTS(); } +#if 0 TEST(testCase, build_executor_tree_Test) { const char* msg = "{\n" "\t\"Id\":\t{\n" @@ -330,7 +331,7 @@ TEST(testCase, external_sort_Test) { } } - printStatisBeforeClose(((SOrderOperatorInfo*) pOperator->info)->pSortInternalBuf); +// setPrintStatis(((SOrderOperatorInfo*) pOperator->info)->pSortInternalBuf); int64_t s2 = taosGetTimestampUs(); printf("total:%ld\n", s2 - s1); @@ -341,4 +342,7 @@ TEST(testCase, external_sort_Test) { taosArrayDestroy(pExprInfo); taosArrayDestroy(pOrderVal); } + +#endif + #pragma GCC diagnostic pop diff --git a/source/libs/executor/test/executorUtilTests.cpp b/source/libs/executor/test/executorUtilTests.cpp new file mode 100644 index 0000000000..a8ddafd7cd --- /dev/null +++ b/source/libs/executor/test/executorUtilTests.cpp @@ -0,0 +1,269 @@ +/* + * 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 + +#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" +#include "os.h" + +#include "executor.h" +#include "stub.h" +#include "taos.h" +#include "tdef.h" +#include "tep.h" +#include "trpc.h" +#include "tvariant.h" + +namespace { +typedef struct { + int32_t startVal; + int32_t count; + int32_t pageRows; +} _info; + +SSDataBlock* getSingleColDummyBlock(void* param) { + _info* pInfo = (_info*) param; + if (--pInfo->count < 0) { + return NULL; + } + + SSDataBlock* pBlock = static_cast(calloc(1, sizeof(SSDataBlock))); + pBlock->pDataBlock = taosArrayInit(4, sizeof(SColumnInfoData)); + + SColumnInfoData colInfo = {0}; + colInfo.info.type = TSDB_DATA_TYPE_INT; + colInfo.info.bytes = sizeof(int32_t); + colInfo.info.colId = 1; + colInfo.pData = static_cast(calloc(pInfo->pageRows, sizeof(int32_t))); + colInfo.nullbitmap = static_cast(calloc(1, (pInfo->pageRows + 7) / 8)); + + taosArrayPush(pBlock->pDataBlock, &colInfo); + + for (int32_t i = 0; i < pInfo->pageRows; ++i) { + SColumnInfoData* pColInfo = static_cast(TARRAY_GET_ELEM(pBlock->pDataBlock, 0)); + + int32_t v = ++pInfo->startVal; + colDataAppend(pColInfo, i, reinterpret_cast(&v), false); + } + + pBlock->info.rows = pInfo->pageRows; + pBlock->info.numOfCols = 1; + return pBlock; +} + +int32_t docomp(const void* p1, const void* p2, void* param) { + int32_t pLeftIdx = *(int32_t *)p1; + int32_t pRightIdx = *(int32_t *)p2; + + SMsortComparParam *pParam = (SMsortComparParam *)param; + SOperatorSource** px = reinterpret_cast(pParam->pSources); + + SArray *pInfo = pParam->orderInfo; + + SOperatorSource* pLeftSource = px[pLeftIdx]; + SOperatorSource* pRightSource = px[pRightIdx]; + + // this input is exhausted, set the special value to denote this + if (pLeftSource->src.rowIndex == -1) { + return 1; + } + + if (pRightSource->src.rowIndex == -1) { + return -1; + } + + SSDataBlock* pLeftBlock = pLeftSource->src.pBlock; + SSDataBlock* pRightBlock = pRightSource->src.pBlock; + + for(int32_t i = 0; i < pInfo->size; ++i) { + SBlockOrderInfo* pOrder = (SBlockOrderInfo*)TARRAY_GET_ELEM(pInfo, i); + + SColumnInfoData* pLeftColInfoData = (SColumnInfoData*)TARRAY_GET_ELEM(pLeftBlock->pDataBlock, pOrder->colIndex); + + bool leftNull = false; + if (pLeftColInfoData->hasNull) { + leftNull = colDataIsNull(pLeftColInfoData, pLeftBlock->info.rows, pLeftSource->src.rowIndex, pLeftBlock->pBlockAgg); + } + + SColumnInfoData* pRightColInfoData = (SColumnInfoData*) TARRAY_GET_ELEM(pRightBlock->pDataBlock, pOrder->colIndex); + bool rightNull = false; + if (pRightColInfoData->hasNull) { + rightNull = colDataIsNull(pRightColInfoData, pRightBlock->info.rows, pRightSource->src.rowIndex, pRightBlock->pBlockAgg); + } + + if (leftNull && rightNull) { + continue; // continue to next slot + } + + if (rightNull) { + return pParam->nullFirst? 1:-1; + } + + if (leftNull) { + return pParam->nullFirst? -1:1; + } + + void* left1 = colDataGet(pLeftColInfoData, pLeftSource->src.rowIndex); + void* right1 = colDataGet(pRightColInfoData, pRightSource->src.rowIndex); + + switch(pLeftColInfoData->info.type) { + case TSDB_DATA_TYPE_INT: { + int32_t leftv = *(int32_t*)left1; + int32_t rightv = *(int32_t*)right1; + + if (leftv == rightv) { + break; + } else { + if (pOrder->order == TSDB_ORDER_ASC) { + return leftv < rightv? -1 : 1; + } else { + return leftv < rightv? 1 : -1; + } + } + } + default: + assert(0); + } + } +} +} // namespace + +//TEST(testCase, inMem_sort_Test) { +// SArray* pOrderVal = taosArrayInit(4, sizeof(SOrder)); +// SOrder o = {.order = TSDB_ORDER_ASC}; +// o.col.info.colId = 1; +// o.col.info.type = TSDB_DATA_TYPE_INT; +// taosArrayPush(pOrderVal, &o); +// +// int32_t numOfRows = 1000; +// SBlockOrderInfo oi = {0}; +// oi.order = TSDB_ORDER_ASC; +// oi.colIndex = 0; +// SArray* orderInfo = taosArrayInit(1, sizeof(SBlockOrderInfo)); +// taosArrayPush(orderInfo, &oi); +// +// SSortHandle* phandle = createSortHandle(orderInfo, false, SORT_SINGLESOURCE, 1024, 5, "test_abc"); +// setFetchRawDataFp(phandle, getSingleColDummyBlock); +// sortAddSource(phandle, &numOfRows); +// +// int32_t code = sortOpen(phandle); +// int32_t row = 1; +// +// while(1) { +// STupleHandle* pTupleHandle = sortNextTuple(phandle); +// if (pTupleHandle == NULL) { +// break; +// } +// +// void* v = sortGetValue(pTupleHandle, 0); +// printf("%d: %d\n", row++, *(int32_t*) v); +// +// } +// destroySortHandle(phandle); +//} +// +//TEST(testCase, external_mem_sort_Test) { +// totalcount = 50; +// startVal = 100000; +// +// SArray* pOrderVal = taosArrayInit(4, sizeof(SOrder)); +// SOrder o = {.order = TSDB_ORDER_ASC}; +// o.col.info.colId = 1; +// o.col.info.type = TSDB_DATA_TYPE_INT; +// taosArrayPush(pOrderVal, &o); +// +// int32_t numOfRows = 1000; +// SBlockOrderInfo oi = {0}; +// oi.order = TSDB_ORDER_ASC; +// oi.colIndex = 0; +// SArray* orderInfo = taosArrayInit(1, sizeof(SBlockOrderInfo)); +// taosArrayPush(orderInfo, &oi); +// +// SSortHandle* phandle = createSortHandle(orderInfo, false, SORT_SINGLESOURCE, 1024, 5, "test_abc"); +// setFetchRawDataFp(phandle, getSingleColDummyBlock); +// sortAddSource(phandle, &numOfRows); +// +// int32_t code = sortOpen(phandle); +// int32_t row = 1; +// +// while(1) { +// STupleHandle* pTupleHandle = sortNextTuple(phandle); +// if (pTupleHandle == NULL) { +// break; +// } +// +// void* v = sortGetValue(pTupleHandle, 0); +// printf("%d: %d\n", row++, *(int32_t*) v); +// +// } +// destroySortHandle(phandle); +//} + +TEST(testCase, ordered_merge_sort_Test) { + SArray* pOrderVal = taosArrayInit(4, sizeof(SOrder)); + SOrder o = {.order = TSDB_ORDER_ASC}; + o.col.info.colId = 1; + o.col.info.type = TSDB_DATA_TYPE_INT; + taosArrayPush(pOrderVal, &o); + + int32_t numOfRows = 1000; + SBlockOrderInfo oi = {0}; + oi.order = TSDB_ORDER_ASC; + oi.colIndex = 0; + SArray* orderInfo = taosArrayInit(1, sizeof(SBlockOrderInfo)); + taosArrayPush(orderInfo, &oi); + + SSchema s = {.type = TSDB_DATA_TYPE_INT, .colId = 1, .bytes = 4}; + SSortHandle* phandle = createSortHandle(orderInfo, false, SORT_MULTIWAY_MERGE, 1024, 5, &s, 1,"test_abc"); + setFetchRawDataFp(phandle, getSingleColDummyBlock); + setComparFn(phandle, docomp); + + + for(int32_t i = 0; i < 10; ++i) { + SOperatorSource* p = static_cast(calloc(1, sizeof(SOperatorSource))); + _info* c = static_cast<_info*>(calloc(1, sizeof(_info))); + c->count = 1; + c->pageRows = 1000; + c->startVal = 0; + + p->param = c; + sortAddSource(phandle, p); + } + + int32_t code = sortOpen(phandle); + int32_t row = 1; + + while(1) { + STupleHandle* pTupleHandle = sortNextTuple(phandle); + if (pTupleHandle == NULL) { + break; + } + + void* v = sortGetValue(pTupleHandle, 0); + printf("%d: %d\n", row++, *(int32_t*) v); + + } + destroySortHandle(phandle); +} + +#pragma GCC diagnostic pop diff --git a/source/util/src/tpagedbuf.c b/source/util/src/tpagedbuf.c index 0e8d85492c..d5415c219f 100644 --- a/source/util/src/tpagedbuf.c +++ b/source/util/src/tpagedbuf.c @@ -53,7 +53,7 @@ typedef struct SDiskbasedBuf { static void printStatisData(const SDiskbasedBuf* pBuf); - int32_t createDiskbasedBuffer(SDiskbasedBuf** pBuf, int32_t pagesize, int32_t inMemBufSize, uint64_t qId, const char* dir) { +int32_t createDiskbasedBuffer(SDiskbasedBuf** pBuf, int32_t pagesize, int32_t inMemBufSize, uint64_t qId, const char* dir) { *pBuf = calloc(1, sizeof(SDiskbasedBuf)); SDiskbasedBuf* pResBuf = *pBuf; @@ -569,7 +569,7 @@ void setBufPageDirty(SFilePage* pPage, bool dirty) { ppi->dirty = dirty; } -void printStatisBeforeClose(SDiskbasedBuf* pBuf) { +void setPrintStatis(SDiskbasedBuf* pBuf) { pBuf->printStatis = true; } From bbc03a176468ab6597f4c03a4c5aeae8f846a72c Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sat, 19 Feb 2022 15:36:05 +0800 Subject: [PATCH 06/25] [td-11818] refactor code. --- source/libs/executor/inc/tsort.h | 2 +- source/libs/executor/src/executorimpl.c | 103 -------------------- source/libs/executor/src/tsort.c | 2 + source/libs/executor/test/executorTests.cpp | 4 - 4 files changed, 3 insertions(+), 108 deletions(-) diff --git a/source/libs/executor/inc/tsort.h b/source/libs/executor/inc/tsort.h index 9114746ff9..a1cd043e60 100644 --- a/source/libs/executor/inc/tsort.h +++ b/source/libs/executor/inc/tsort.h @@ -25,7 +25,7 @@ extern "C" { enum { SORT_MULTIWAY_MERGE = 0x1, - SORT_SINGLESOURCE = 0x2, + SORT_SINGLESOURCE = 0x2, }; typedef struct SMultiMergeSource { diff --git a/source/libs/executor/src/executorimpl.c b/source/libs/executor/src/executorimpl.c index 730396c070..a47928b2a2 100644 --- a/source/libs/executor/src/executorimpl.c +++ b/source/libs/executor/src/executorimpl.c @@ -5804,106 +5804,6 @@ static SSDataBlock* createDataBlock(const SSDataBlock* pDataBlock) { return pBlock; } -//static int32_t doAddNewExternalMemSource(SOrderOperatorInfo* pInfo, SArray* pAllSources, SSDataBlock* pBlock) { -// SExternalMemSource* pSource = calloc(1, sizeof(SExternalMemSource)); -// if (pSource == NULL) { -// return TSDB_CODE_QRY_OUT_OF_MEMORY; -// } -// -// pSource->pageIdList = getDataBufPagesIdList(pInfo->pSortInternalBuf, pInfo->sourceId); -// pSource->src.pBlock = pBlock; -// -// taosArrayPush(pAllSources, &pSource); -// -// pInfo->sourceId += 1; -// -// int32_t rowSize = blockDataGetSerialRowSize(pSource->src.pBlock); -// int32_t numOfRows = (getBufPageSize(pInfo->pSortInternalBuf) - blockDataGetSerialMetaSize(pInfo->pDataBlock))/rowSize; -// return blockDataEnsureCapacity(pSource->src.pBlock, numOfRows); -//} - -//static int32_t doAddNewOperatorSource(SArray* pAllSources, SSDataBlock* pBlock, int32_t capacity) { -// SOperatorSource* pSource = calloc(1, sizeof(SOperatorSource)); -// if (pSource == NULL) { -// return TSDB_CODE_QRY_OUT_OF_MEMORY; -// } -// -// pSource->src.pBlock = pBlock; -// taosArrayPush(pAllSources, &pSource); -// -// return blockDataEnsureCapacity(pSource->src.pBlock, capacity); -//} - -//void addToDiskbasedBuf(SOrderOperatorInfo* pInfo, SArray* pSources, jmp_buf env) { -// int32_t start = 0; -// -// while(start < pInfo->pDataBlock->info.rows) { -// int32_t stop = 0; -// blockDataSplitRows(pInfo->pDataBlock, pInfo->hasVarCol, start, &stop, getBufPageSize(pInfo->pSortInternalBuf)); -// SSDataBlock* p = blockDataExtractBlock(pInfo->pDataBlock, start, stop - start + 1); -// if (p == NULL) { -// longjmp(env, TSDB_CODE_QRY_OUT_OF_MEMORY); -// } -// -// int32_t pageId = -1; -// SFilePage* pPage = getNewDataBuf(pInfo->pSortInternalBuf, pInfo->sourceId, &pageId); -// if (pPage == NULL) { -// assert(0); -// longjmp(env, terrno); -// } -// -// int32_t size = blockDataGetSize(p) + sizeof(int32_t) + p->info.numOfCols * sizeof(int32_t); -// assert(size <= getBufPageSize(pInfo->pSortInternalBuf)); -// -// blockDataToBuf(pPage->data, p); -// -// setBufPageDirty(pPage, true); -// releaseBufPage(pInfo->pSortInternalBuf, pPage); -// -// blockDataDestroy(p); -// start = stop + 1; -// } -// -// int32_t numOfCols = pInfo->pDataBlock->info.numOfCols; -// blockDataClearup(pInfo->pDataBlock, pInfo->hasVarCol); -// -// SSDataBlock* pBlock = createDataBlock(pInfo->pDataBlock); -// int32_t code = doAddNewExternalMemSource(pInfo, pSources, pBlock); -// if (code != TSDB_CODE_SUCCESS) { -// longjmp(env, code); -// } -//} - -static int32_t sortComparInit(SMsortComparParam* cmpParam, SArray* pSources, int32_t startIndex, int32_t endIndex, SDiskbasedBuf* pBuf) { - cmpParam->pSources = taosArrayGet(pSources, startIndex); - cmpParam->numOfSources = (endIndex - startIndex + 1); - - for(int32_t i = 0; i < cmpParam->numOfSources; ++i) { - SExternalMemSource* pSource = cmpParam->pSources[i]; - SPageInfo* pPgInfo = *(SPageInfo**) taosArrayGet(pSource->pageIdList, pSource->pageIndex); - - SFilePage* pPage = getBufPage(pBuf, getPageId(pPgInfo)); - int32_t code = blockDataFromBuf(cmpParam->pSources[i]->src.pBlock, pPage->data); - if (code != TSDB_CODE_SUCCESS) { - return code; - } - - releaseBufPage(pBuf, pPage); - } - - return TSDB_CODE_SUCCESS; -} - -static int32_t sortComparClearup(SMsortComparParam* cmpParam) { - for(int32_t i = 0; i < cmpParam->numOfSources; ++i) { - SExternalMemSource* pSource = cmpParam->pSources[i]; - blockDataDestroy(pSource->src.pBlock); - tfree(pSource); - } - - cmpParam->numOfSources = 0; -} - static SSDataBlock* getSortedBlockData(SSortHandle* pHandle, SOrderOperatorInfo* pInfo, int32_t capacity) { blockDataClearup(pInfo->pDataBlock, pInfo->hasVarCol); @@ -5936,8 +5836,6 @@ static SSDataBlock* doSort(void* param, bool* newgroup) { SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; SOrderOperatorInfo* pInfo = pOperator->info; - SSDataBlock* pBlock = NULL; - if (pOperator->status == OP_RES_TO_RETURN) { return getSortedBlockData(pInfo->pSortHandle, pInfo, pInfo->numOfRowsInRes); } @@ -6013,7 +5911,6 @@ SOperatorInfo *createOrderOperatorInfo(SOperatorInfo* downstream, SArray* pExprI } } -// int32_t code = createDiskbasedBuffer(&pInfo->pSortInternalBuf, pInfo->bufPageSize, pInfo->sortBufSize, 1, "/tmp/"); if (pInfo->orderInfo == NULL || pInfo->pDataBlock == NULL) { tfree(pOperator); destroyOrderOperatorInfo(pInfo, taosArrayGetSize(pExprInfo)); diff --git a/source/libs/executor/src/tsort.c b/source/libs/executor/src/tsort.c index 4911146b54..5e98f6ad06 100644 --- a/source/libs/executor/src/tsort.c +++ b/source/libs/executor/src/tsort.c @@ -176,6 +176,7 @@ static int32_t doAddToBuf(SSDataBlock* pDataBlock, SSortHandle* pHandle) { if (pHandle->pBuf == NULL) { int32_t code = createDiskbasedBuffer(&pHandle->pBuf, pHandle->pageSize, pHandle->numOfPages * pHandle->pageSize, 0, "/tmp"); + setPrintStatis(pHandle->pBuf); if (code != TSDB_CODE_SUCCESS) { return code; } @@ -239,6 +240,7 @@ static int32_t sortComparInit(SMsortComparParam* cmpParam, SArray* pSources, int // multi-pass internal merge sort is required if (pHandle->pBuf == NULL) { code = createDiskbasedBuffer(&pHandle->pBuf, pHandle->pageSize, pHandle->numOfPages * pHandle->pageSize, 0, "/tmp"); + setPrintStatis(pHandle->pBuf); if (code != TSDB_CODE_SUCCESS) { return code; } diff --git a/source/libs/executor/test/executorTests.cpp b/source/libs/executor/test/executorTests.cpp index 22df894cdf..4e9f6bebcd 100644 --- a/source/libs/executor/test/executorTests.cpp +++ b/source/libs/executor/test/executorTests.cpp @@ -121,7 +121,6 @@ int main(int argc, char** argv) { return RUN_ALL_TESTS(); } -#if 0 TEST(testCase, build_executor_tree_Test) { const char* msg = "{\n" "\t\"Id\":\t{\n" @@ -331,8 +330,6 @@ TEST(testCase, external_sort_Test) { } } -// setPrintStatis(((SOrderOperatorInfo*) pOperator->info)->pSortInternalBuf); - int64_t s2 = taosGetTimestampUs(); printf("total:%ld\n", s2 - s1); @@ -343,6 +340,5 @@ TEST(testCase, external_sort_Test) { taosArrayDestroy(pOrderVal); } -#endif #pragma GCC diagnostic pop From 6d37b663edf85b72e89601040fe5d191aff1a6f2 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sat, 19 Feb 2022 16:15:09 +0800 Subject: [PATCH 07/25] [td-11818] refactor sortedmerge operator. --- include/libs/planner/plannerOp.h | 2 +- source/libs/executor/inc/executorimpl.h | 28 +- source/libs/executor/inc/tsort.h | 7 + source/libs/executor/src/executorimpl.c | 292 ++++++------------ source/libs/executor/src/tsort.c | 14 +- source/libs/executor/test/executorTests.cpp | 2 +- .../libs/executor/test/executorUtilTests.cpp | 1 - 7 files changed, 123 insertions(+), 223 deletions(-) diff --git a/include/libs/planner/plannerOp.h b/include/libs/planner/plannerOp.h index 840064025e..af8a7dccd4 100644 --- a/include/libs/planner/plannerOp.h +++ b/include/libs/planner/plannerOp.h @@ -47,6 +47,6 @@ OP_ENUM_MACRO(AllTimeWindow) OP_ENUM_MACRO(AllMultiTableTimeInterval) OP_ENUM_MACRO(Order) OP_ENUM_MACRO(Exchange) -OP_ENUM_MACRO(SortMerge) +OP_ENUM_MACRO(SortedMerge) //OP_ENUM_MACRO(TableScan) diff --git a/source/libs/executor/inc/executorimpl.h b/source/libs/executor/inc/executorimpl.h index adfed41f99..c5706c7e59 100644 --- a/source/libs/executor/inc/executorimpl.h +++ b/source/libs/executor/inc/executorimpl.h @@ -550,22 +550,20 @@ typedef struct SDistinctOperatorInfo { SArray* pDistinctDataInfo; } SDistinctOperatorInfo; -typedef struct SSortMergeOperatorInfo { - SOptrBasicInfo binfo; +typedef struct SSortedMergeOperatorInfo { + SSDataBlock *pDataBlock; + bool hasVarCol; + SArray *orderInfo; // SArray - SArray* groupColumnList; - bool hasDataBlockForNewGroup; - char** currentGroupColData; - SArray* udfInfo; + bool nullFirst; int32_t numOfSources; -} SSortMergeOperatorInfo; -typedef struct SMsortComparParam { - struct SExternalMemSource **pSources; - int32_t numOfSources; - SArray *orderInfo; // SArray - bool nullFirst; -} SMsortComparParam; + SSortHandle *pSortHandle; + + int32_t bufPageSize; + uint32_t sortBufSize; // max buffer size for in-memory sort + int32_t numOfRowsInRes; +} SSortedMergeOperatorInfo; typedef struct SOrderOperatorInfo { uint32_t sortBufSize; // max buffer size for in-memory sort @@ -628,8 +626,8 @@ SOperatorInfo* createFilterOperatorInfo(STaskRuntimeEnv* pRuntimeEnv, SOperatorI SOperatorInfo* createJoinOperatorInfo(SOperatorInfo** pdownstream, int32_t numOfDownstream, SSchema* pSchema, int32_t numOfOutput); -SOperatorInfo* createOrderOperatorInfo(SOperatorInfo* downstream, SArray* pExprInfo, SArray* pOrderVal); -SOperatorInfo* createSortMergeOperatorInfo(SOperatorInfo* downstream, SArray* pExprInfo, void* param, SArray* pUdfInfo, SExecTaskInfo* pTaskInfo); +SOperatorInfo* createOrderOperatorInfo(SOperatorInfo* downstream, SArray* pExprInfo, SArray* pOrderVal, SExecTaskInfo* pTaskInfo); +SOperatorInfo* createSortedMergeOperatorInfo(SOperatorInfo** downstream, int32_t numOfDownstream, SArray* pExprInfo, void* param, SArray* pOrderVal, SExecTaskInfo* pTaskInfo); // SSDataBlock* doGlobalAggregate(void* param, bool* newgroup); // SSDataBlock* doMultiwayMergeSort(void* param, bool* newgroup); diff --git a/source/libs/executor/inc/tsort.h b/source/libs/executor/inc/tsort.h index a1cd043e60..77a38f147e 100644 --- a/source/libs/executor/inc/tsort.h +++ b/source/libs/executor/inc/tsort.h @@ -45,6 +45,13 @@ typedef struct SOperatorSource { void* param; } SOperatorSource; +typedef struct SMsortComparParam { + void **pSources; + int32_t numOfSources; + SArray *orderInfo; // SArray + bool nullFirst; +} SMsortComparParam; + typedef struct SSortHandle SSortHandle; typedef struct STupleHandle STupleHandle; diff --git a/source/libs/executor/src/executorimpl.c b/source/libs/executor/src/executorimpl.c index a47928b2a2..26eea384e0 100644 --- a/source/libs/executor/src/executorimpl.c +++ b/source/libs/executor/src/executorimpl.c @@ -4521,13 +4521,13 @@ void queryCostStatis(SExecTaskInfo *pTaskInfo) { // return true; //} -void appendDownstream(SOperatorInfo* p, SOperatorInfo* pUpstream) { +void appendDownstream(SOperatorInfo* p, SOperatorInfo* pDownstream) { if (p->pDownstream == NULL) { assert(p->numOfDownstream == 0); } p->pDownstream = realloc(p->pDownstream, POINTER_BYTES * (p->numOfDownstream + 1)); - p->pDownstream[p->numOfDownstream++] = pUpstream; + p->pDownstream[p->numOfDownstream++] = pDownstream; } static void doDestroyTableQueryInfo(STableGroupInfo* pTableqinfoGroupInfo); @@ -5599,11 +5599,7 @@ SArray* getResultGroupCheckColumns(STaskAttr* pQuery) { return pOrderColumns; } -static void destroySortMergeOperatorInfo(void* param, int32_t numOfOutput) { - SSortMergeOperatorInfo *pInfo = (SSortMergeOperatorInfo*) param; - destroyBasicOperatorInfo(&pInfo->binfo, numOfOutput); - taosArrayDestroy(pInfo->groupColumnList); - tfree(pInfo->currentGroupColData); +static void destroySortedMergeOperatorInfo(void* param, int32_t numOfOutput) { } static void destroySlimitOperatorInfo(void* param, int32_t numOfOutput) { @@ -5624,153 +5620,6 @@ static SExprInfo* exprArrayDup(SArray* pExprInfo) { return p; } -static SSDataBlock* doSortedMerge(void* param, bool* newgroup) { - SOperatorInfo* pOperator = (SOperatorInfo*) param; - if (pOperator->status == OP_EXEC_DONE) { - return NULL; - } - - SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; - SSortMergeOperatorInfo* pInfo = pOperator->info; - - SMsortComparParam resultParam = {.orderInfo = pInfo->orderInfo}; - SArray* pSource = taosArrayInit(4, POINTER_BYTES); - - for(int32_t i = 0; i < pInfo->numOfSources; ++i) { - SSDataBlock* pBlock = pOperator->pDownstream[i]->exec(pOperator->pDownstream[i], newgroup); -// doAddNewOperatorSource(pSource, pBlock, pInfo->binfo.capacity); - } - -// sortComparInit(&resultParam, pSource, 0, pInfo->numOfSources - 1, pOperator); - -// int32_t code = tMergeTreeCreate(&pInfo->pMergeTree, pInfo->cmpParam.numOfSources, &pInfo->cmpParam, msortComparFn); -// if (code != TSDB_CODE_SUCCESS) { -// longjmp(pTaskInfo->env, code); -// } - - return NULL; -} - -SOperatorInfo* createSortMergeOperatorInfo(SOperatorInfo* downstream, SArray* pExprInfo, void* param, SArray* pUdfInfo, SExecTaskInfo* pTaskInfo) { - SSortMergeOperatorInfo* pInfo = calloc(1, sizeof(SSortMergeOperatorInfo)); - SOperatorInfo* pOperator = calloc(1, sizeof(SOperatorInfo)); - -// pInfo->resultRowFactor = -// (int32_t)(getRowNumForMultioutput(pRuntimeEnv->pQueryAttr, pRuntimeEnv->pQueryAttr->topBotQuery, false)); - - int32_t numOfOutput = taosArrayGetSize(pExprInfo); - pInfo->binfo.capacity = 4096; - pInfo->udfInfo = pUdfInfo; - pInfo->binfo.pRes = createOutputBuf_rv(pExprInfo, pInfo->binfo.capacity); - pInfo->binfo.pCtx = createSqlFunctionCtx_rv( pExprInfo, &pInfo->binfo.rowCellInfoOffset, &pInfo->binfo.resRowSize); - - int32_t numOfCols = (pInfo->groupColumnList != NULL)? (int32_t)taosArrayGetSize(pInfo->groupColumnList):0; -// pInfo->currentGroupColData = calloc(1, (POINTER_BYTES * numOfCols + len)); - int32_t offset = POINTER_BYTES * numOfCols; - -// for(int32_t i = 0; i < numOfCols; ++i) { -// pInfo->currentGroupColData[i] = (char*)pInfo->currentGroupColData + offset; -// -// SColIndex* index = taosArrayGet(pInfo->groupColumnList, i); -// offset += pExpr[index->colIndex].base.resSchema.bytes; -// } - - initResultRowInfo(&pInfo->binfo.resultRowInfo, 8, TSDB_DATA_TYPE_INT); - - int32_t numOfRows = 1; -// setDefaultOutputBuf_rv(pExprInfo, numOfRows); - - pOperator->name = "SortMerge"; - pOperator->operatorType = OP_SortMerge; - pOperator->blockingOptr = true; - pOperator->status = OP_IN_EXECUTING; - pOperator->info = pInfo; - pOperator->pExpr = exprArrayDup(pExprInfo); - pOperator->numOfOutput = numOfOutput; - - pOperator->pTaskInfo = pTaskInfo; - pOperator->exec = doSortedMerge; - pOperator->cleanupFn = destroySortMergeOperatorInfo; - appendDownstream(pOperator, downstream); - - return pOperator; -} - -int32_t msortComparFn(const void *pLeft, const void *pRight, void *param) { - int32_t pLeftIdx = *(int32_t *)pLeft; - int32_t pRightIdx = *(int32_t *)pRight; - - SMsortComparParam *pParam = (SMsortComparParam *)param; - - SArray *pInfo = pParam->orderInfo; - - SExternalMemSource* pLeftSource = pParam->pSources[pLeftIdx]; - SExternalMemSource* pRightSource = pParam->pSources[pRightIdx]; - - // this input is exhausted, set the special value to denote this - if (pLeftSource->src.rowIndex == -1) { - return 1; - } - - if (pRightSource->src.rowIndex == -1) { - return -1; - } - - SSDataBlock* pLeftBlock = pLeftSource->src.pBlock; - SSDataBlock* pRightBlock = pRightSource->src.pBlock; - - for(int32_t i = 0; i < pInfo->size; ++i) { - SBlockOrderInfo* pOrder = TARRAY_GET_ELEM(pInfo, i); - - SColumnInfoData* pLeftColInfoData = TARRAY_GET_ELEM(pLeftBlock->pDataBlock, pOrder->colIndex); - - bool leftNull = false; - if (pLeftColInfoData->hasNull) { - leftNull = colDataIsNull(pLeftColInfoData, pLeftBlock->info.rows, pLeftSource->src.rowIndex, pLeftBlock->pBlockAgg); - } - - SColumnInfoData* pRightColInfoData = TARRAY_GET_ELEM(pRightBlock->pDataBlock, pOrder->colIndex); - bool rightNull = false; - if (pRightColInfoData->hasNull) { - rightNull = colDataIsNull(pRightColInfoData, pRightBlock->info.rows, pRightSource->src.rowIndex, pRightBlock->pBlockAgg); - } - - if (leftNull && rightNull) { - continue; // continue to next slot - } - - if (rightNull) { - return pParam->nullFirst? 1:-1; - } - - if (leftNull) { - return pParam->nullFirst? -1:1; - } - - void* left1 = colDataGet(pLeftColInfoData, pLeftSource->src.rowIndex); - void* right1 = colDataGet(pRightColInfoData, pRightSource->src.rowIndex); - - switch(pLeftColInfoData->info.type) { - case TSDB_DATA_TYPE_INT: { - int32_t leftv = *(int32_t*)left1; - int32_t rightv = *(int32_t*)right1; - - if (leftv == rightv) { - break; - } else { - if (pOrder->order == TSDB_ORDER_ASC) { - return leftv < rightv? -1 : 1; - } else { - return leftv < rightv? 1 : -1; - } - } - } - default: - assert(0); - } - } -} - static void appendOneRowToDataBlock(SSDataBlock *pBlock, STupleHandle* pTupleHandle) { for (int32_t i = 0; i < pBlock->info.numOfCols; ++i) { SColumnInfoData* pColInfo = taosArrayGet(pBlock->pDataBlock, i); @@ -5787,80 +5636,59 @@ static void appendOneRowToDataBlock(SSDataBlock *pBlock, STupleHandle* pTupleHan pBlock->info.rows += 1; } -static SSDataBlock* createDataBlock(const SSDataBlock* pDataBlock) { - int32_t numOfCols = pDataBlock->info.numOfCols; - - SSDataBlock* pBlock = calloc(1, sizeof(SSDataBlock)); - pBlock->pDataBlock = taosArrayInit(numOfCols, sizeof(SColumnInfoData)); - pBlock->info.numOfCols = numOfCols; - - for(int32_t i = 0; i < numOfCols; ++i) { - SColumnInfoData colInfo = {0}; - SColumnInfoData* p = taosArrayGet(pDataBlock->pDataBlock, i); - colInfo.info = p->info; - taosArrayPush(pBlock->pDataBlock, &colInfo); - } - - return pBlock; -} - -static SSDataBlock* getSortedBlockData(SSortHandle* pHandle, SOrderOperatorInfo* pInfo, int32_t capacity) { - blockDataClearup(pInfo->pDataBlock, pInfo->hasVarCol); +static SSDataBlock* getSortedBlockData(SSortHandle* pHandle, SSDataBlock* pDataBlock, bool hasVarCol, int32_t capacity) { + blockDataClearup(pDataBlock, hasVarCol); while(1) { - STupleHandle* pTupleHandle = sortNextTuple(pHandle); - if (pTupleHandle == NULL) { - break; - } + STupleHandle* pTupleHandle = sortNextTuple(pHandle); + if (pTupleHandle == NULL) { + break; + } - appendOneRowToDataBlock(pInfo->pDataBlock, pTupleHandle); - if (pInfo->pDataBlock->info.rows >= capacity) { - return pInfo->pDataBlock; + appendOneRowToDataBlock(pDataBlock, pTupleHandle); + if (pDataBlock->info.rows >= capacity) { + return pDataBlock; } } - return (pInfo->pDataBlock->info.rows > 0)? pInfo->pDataBlock:NULL; + return (pDataBlock->info.rows > 0)? pDataBlock:NULL; } -static SSDataBlock* loadNextDataBlock(void* param) { - bool newgroup = false; - SOperatorInfo* pOperator = (SOperatorInfo*) param; - return pOperator->pDownstream[0]->exec(pOperator->pDownstream[0], &newgroup); +SSDataBlock* loadNextDataBlock(void* param) { + } -static SSDataBlock* doSort(void* param, bool* newgroup) { +static SSDataBlock* doSortedMerge(void* param, bool* newgroup) { SOperatorInfo* pOperator = (SOperatorInfo*) param; if (pOperator->status == OP_EXEC_DONE) { return NULL; } SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; - SOrderOperatorInfo* pInfo = pOperator->info; + SSortedMergeOperatorInfo* pInfo = pOperator->info; if (pOperator->status == OP_RES_TO_RETURN) { - return getSortedBlockData(pInfo->pSortHandle, pInfo, pInfo->numOfRowsInRes); + return getSortedBlockData(pInfo->pSortHandle, pInfo->pDataBlock, pInfo->hasVarCol, pInfo->numOfRowsInRes); } - int64_t st = taosGetTimestampUs(); - SSchema* p = blockDataExtractSchema(pInfo->pDataBlock, NULL); - int32_t numOfBufPage = pInfo->sortBufSize / pInfo->bufPageSize; + int32_t numOfBufPage = pInfo->sortBufSize / pInfo->bufPageSize; pInfo->pSortHandle = createSortHandle(pInfo->orderInfo, pInfo->nullFirst, SORT_SINGLESOURCE, pInfo->bufPageSize, numOfBufPage, p, pInfo->pDataBlock->info.numOfCols, "GET_TASKID(pTaskInfo)"); tfree(p); setFetchRawDataFp(pInfo->pSortHandle, loadNextDataBlock); - setComparFn(pInfo->pSortHandle, msortComparFn); - sortAddSource(pInfo->pSortHandle, pOperator); + for(int32_t i = 0; i < pOperator->numOfDownstream; ++i) { + sortAddSource(pInfo->pSortHandle, pOperator->pDownstream[i]); + } - // TODO set error code; int32_t code = sortOpen(pInfo->pSortHandle); if (code != TSDB_CODE_SUCCESS) { longjmp(pTaskInfo->env, terrno); } pOperator->status = OP_RES_TO_RETURN; - return getSortedBlockData(pInfo->pSortHandle, pInfo, pInfo->numOfRowsInRes); + return getSortedBlockData(pInfo->pSortHandle, pInfo->pDataBlock, pInfo->hasVarCol, pInfo->numOfRowsInRes); } static SArray* createBlockOrder(SArray* pExprInfo, SArray* pOrderVal) { @@ -5886,7 +5714,77 @@ static SArray* createBlockOrder(SArray* pExprInfo, SArray* pOrderVal) { return pOrderInfo; } -SOperatorInfo *createOrderOperatorInfo(SOperatorInfo* downstream, SArray* pExprInfo, SArray* pOrderVal) { +SOperatorInfo* createSortedMergeOperatorInfo(SOperatorInfo** downstream, int32_t numOfDownstream, SArray* pExprInfo, void* param, SArray* pOrderVal, SExecTaskInfo* pTaskInfo) { + SSortedMergeOperatorInfo* pInfo = calloc(1, sizeof(SSortedMergeOperatorInfo)); + SOperatorInfo* pOperator = calloc(1, sizeof(SOperatorInfo)); + if (pInfo == NULL || pOperator == NULL) { + tfree(pInfo); + + terrno = TSDB_CODE_QRY_OUT_OF_MEMORY; + return NULL; + } + +// pInfo->resultRowFactor = +// (int32_t)(getRowNumForMultioutput(pRuntimeEnv->pQueryAttr, pRuntimeEnv->pQueryAttr->topBotQuery, false)); + pInfo->sortBufSize = 1024 * 16; // 1MB + pInfo->bufPageSize = 1024; + pInfo->numOfRowsInRes = 1024; + pInfo->pDataBlock = createOutputBuf_rv(pExprInfo, pInfo->numOfRowsInRes); + pInfo->orderInfo = createBlockOrder(pExprInfo, pOrderVal); + + int32_t numOfRows = 1; +// setDefaultOutputBuf_rv(pExprInfo, numOfRows); + + pOperator->name = "SortedMerge"; + pOperator->operatorType = OP_SortedMerge; + pOperator->blockingOptr = true; + pOperator->status = OP_IN_EXECUTING; + pOperator->info = pInfo; + + pOperator->pTaskInfo = pTaskInfo; + pOperator->exec = doSortedMerge; + pOperator->cleanupFn = destroySortedMergeOperatorInfo; + + for(int32_t i = 0; i < numOfDownstream; ++i) { + appendDownstream(pOperator, downstream[i]); + } + + return pOperator; +} + +static SSDataBlock* doSort(void* param, bool* newgroup) { + SOperatorInfo* pOperator = (SOperatorInfo*) param; + if (pOperator->status == OP_EXEC_DONE) { + return NULL; + } + + SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; + SOrderOperatorInfo* pInfo = pOperator->info; + if (pOperator->status == OP_RES_TO_RETURN) { + return getSortedBlockData(pInfo->pSortHandle, pInfo->pDataBlock, pInfo->hasVarCol, pInfo->numOfRowsInRes); + } + + SSchema* p = blockDataExtractSchema(pInfo->pDataBlock, NULL); + int32_t numOfBufPage = pInfo->sortBufSize / pInfo->bufPageSize; + pInfo->pSortHandle = createSortHandle(pInfo->orderInfo, pInfo->nullFirst, SORT_SINGLESOURCE, pInfo->bufPageSize, + numOfBufPage, p, pInfo->pDataBlock->info.numOfCols, "GET_TASKID(pTaskInfo)"); + + tfree(p); + setFetchRawDataFp(pInfo->pSortHandle, loadNextDataBlock); + + sortAddSource(pInfo->pSortHandle, pOperator); + + // TODO set error code; + int32_t code = sortOpen(pInfo->pSortHandle); + if (code != TSDB_CODE_SUCCESS) { + longjmp(pTaskInfo->env, terrno); + } + + pOperator->status = OP_RES_TO_RETURN; + return getSortedBlockData(pInfo->pSortHandle, pInfo->pDataBlock, pInfo->hasVarCol, pInfo->numOfRowsInRes); +} + +SOperatorInfo *createOrderOperatorInfo(SOperatorInfo* downstream, SArray* pExprInfo, SArray* pOrderVal, SExecTaskInfo* pTaskInfo) { SOrderOperatorInfo* pInfo = calloc(1, sizeof(SOrderOperatorInfo)); SOperatorInfo* pOperator = calloc(1, sizeof(SOperatorInfo)); if (pInfo == NULL || pOperator == NULL) { @@ -5925,6 +5823,8 @@ SOperatorInfo *createOrderOperatorInfo(SOperatorInfo* downstream, SArray* pExprI pOperator->blockingOptr = true; pOperator->status = OP_IN_EXECUTING; pOperator->info = pInfo; + + pOperator->pTaskInfo = pTaskInfo; pOperator->exec = doSort; pOperator->cleanupFn = destroyOrderOperatorInfo; diff --git a/source/libs/executor/src/tsort.c b/source/libs/executor/src/tsort.c index 5e98f6ad06..1d42f1f838 100644 --- a/source/libs/executor/src/tsort.c +++ b/source/libs/executor/src/tsort.c @@ -23,13 +23,6 @@ #include "tpagedbuf.h" #include "tutil.h" -typedef struct SMsortComparParam { - void **pSources; - int32_t numOfSources; - SArray *orderInfo; // SArray - bool nullFirst; -} SMsortComparParam; - typedef struct STupleHandle { SSDataBlock* pBlock; int32_t rowIndex; @@ -72,7 +65,9 @@ typedef struct SSortHandle { STupleHandle tupleHandle; } SSortHandle; -SSDataBlock* createDataBlock_rv(SSchema* pSchema, int32_t numOfCols) { +static int32_t msortComparFn(const void *pLeft, const void *pRight, void *param); + +static SSDataBlock* createDataBlock_rv(SSchema* pSchema, int32_t numOfCols) { SSDataBlock* pBlock = calloc(1, sizeof(SSDataBlock)); pBlock->pDataBlock = taosArrayInit(numOfCols, sizeof(SColumnInfoData)); pBlock->info.numOfCols = numOfCols; @@ -107,6 +102,7 @@ SSortHandle* createSortHandle(SArray* pOrderInfo, bool nullFirst, int32_t type, pSortHandle->cmpParam.orderInfo = pOrderInfo; pSortHandle->pDataBlock = createDataBlock_rv(pSchema, numOfCols); + setComparFn(pSortHandle, msortComparFn); if (idstr != NULL) { pSortHandle->idStr = strdup(idstr); @@ -365,7 +361,7 @@ static SSDataBlock* getSortedBlockData(SSortHandle* pHandle, SMsortComparParam* return (pHandle->pDataBlock->info.rows > 0)? pHandle->pDataBlock:NULL; } -static int32_t msortComparFn(const void *pLeft, const void *pRight, void *param) { +int32_t msortComparFn(const void *pLeft, const void *pRight, void *param) { int32_t pLeftIdx = *(int32_t *)pLeft; int32_t pRightIdx = *(int32_t *)pRight; diff --git a/source/libs/executor/test/executorTests.cpp b/source/libs/executor/test/executorTests.cpp index 4e9f6bebcd..d20dc28944 100644 --- a/source/libs/executor/test/executorTests.cpp +++ b/source/libs/executor/test/executorTests.cpp @@ -298,7 +298,7 @@ TEST(testCase, external_sort_Test) { exp1->base.resSchema = createSchema(TSDB_DATA_TYPE_BINARY, 40, 2, "res1"); // taosArrayPush(pExprInfo, &exp1); - SOperatorInfo* pOperator = createOrderOperatorInfo(createDummyOperator(1500), pExprInfo, pOrderVal); + SOperatorInfo* pOperator = createOrderOperatorInfo(createDummyOperator(1500), pExprInfo, pOrderVal, NULL); bool newgroup = false; SSDataBlock* pRes = NULL; diff --git a/source/libs/executor/test/executorUtilTests.cpp b/source/libs/executor/test/executorUtilTests.cpp index a8ddafd7cd..57691d6d11 100644 --- a/source/libs/executor/test/executorUtilTests.cpp +++ b/source/libs/executor/test/executorUtilTests.cpp @@ -238,7 +238,6 @@ TEST(testCase, ordered_merge_sort_Test) { setFetchRawDataFp(phandle, getSingleColDummyBlock); setComparFn(phandle, docomp); - for(int32_t i = 0; i < 10; ++i) { SOperatorSource* p = static_cast(calloc(1, sizeof(SOperatorSource))); _info* c = static_cast<_info*>(calloc(1, sizeof(_info))); From eb017b8d57201e46581828189f25aa1e997e27a6 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sat, 19 Feb 2022 16:16:30 +0800 Subject: [PATCH 08/25] [td-11818]add test case. --- source/libs/executor/src/executorimpl.c | 1 - source/libs/executor/test/executorTests.cpp | 60 +++++++++++++++++++++ 2 files changed, 60 insertions(+), 1 deletion(-) diff --git a/source/libs/executor/src/executorimpl.c b/source/libs/executor/src/executorimpl.c index 26eea384e0..75ba620738 100644 --- a/source/libs/executor/src/executorimpl.c +++ b/source/libs/executor/src/executorimpl.c @@ -5733,7 +5733,6 @@ SOperatorInfo* createSortedMergeOperatorInfo(SOperatorInfo** downstream, int32_t pInfo->orderInfo = createBlockOrder(pExprInfo, pOrderVal); int32_t numOfRows = 1; -// setDefaultOutputBuf_rv(pExprInfo, numOfRows); pOperator->name = "SortedMerge"; pOperator->operatorType = OP_SortedMerge; diff --git a/source/libs/executor/test/executorTests.cpp b/source/libs/executor/test/executorTests.cpp index d20dc28944..160bf1f7ca 100644 --- a/source/libs/executor/test/executorTests.cpp +++ b/source/libs/executor/test/executorTests.cpp @@ -340,5 +340,65 @@ TEST(testCase, external_sort_Test) { taosArrayDestroy(pOrderVal); } +TEST(testCase, sorted_merge_Test) { + srand(time(NULL)); + + SArray* pOrderVal = taosArrayInit(4, sizeof(SOrder)); + SOrder o = {0}; + o.order = TSDB_ORDER_ASC; + o.col.info.colId = 1; + o.col.info.type = TSDB_DATA_TYPE_INT; + taosArrayPush(pOrderVal, &o); + + SArray* pExprInfo = taosArrayInit(4, sizeof(SExprInfo)); + SExprInfo *exp = static_cast(calloc(1, sizeof(SExprInfo))); + exp->base.resSchema = createSchema(TSDB_DATA_TYPE_INT, sizeof(int32_t), 1, "res"); + taosArrayPush(pExprInfo, &exp); + + SExprInfo *exp1 = static_cast(calloc(1, sizeof(SExprInfo))); + exp1->base.resSchema = createSchema(TSDB_DATA_TYPE_BINARY, 40, 2, "res1"); +// taosArrayPush(pExprInfo, &exp1); + + SOperatorInfo* pOperator = createOrderOperatorInfo(createDummyOperator(1500), pExprInfo, pOrderVal, NULL); + + bool newgroup = false; + SSDataBlock* pRes = NULL; + + int32_t total = 1; + + int64_t s1 = taosGetTimestampUs(); + int32_t t = 1; + + while(1) { + int64_t s = taosGetTimestampUs(); + pRes = pOperator->exec(pOperator, &newgroup); + + int64_t e = taosGetTimestampUs(); + if (t++ == 1) { + printf("---------------elapsed:%ld\n", e - s); + } + + if (pRes == NULL) { + break; + } + + SColumnInfoData* pCol1 = static_cast(taosArrayGet(pRes->pDataBlock, 0)); +// SColumnInfoData* pCol2 = static_cast(taosArrayGet(pRes->pDataBlock, 1)); + for (int32_t i = 0; i < pRes->info.rows; ++i) { +// char* p = colDataGet(pCol2, i); + printf("%d: %d\n", total++, ((int32_t*)pCol1->pData)[i]); +// printf("%d: %d, %s\n", total++, ((int32_t*)pCol1->pData)[i], (char*)varDataVal(p)); + } + } + + int64_t s2 = taosGetTimestampUs(); + printf("total:%ld\n", s2 - s1); + + pOperator->cleanupFn(pOperator->info, 2); + tfree(exp); + tfree(exp1); + taosArrayDestroy(pExprInfo); + taosArrayDestroy(pOrderVal); +} #pragma GCC diagnostic pop From 17ead5eda47533b479eaa6842a6de782595e42ff Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sat, 19 Feb 2022 23:11:23 +0800 Subject: [PATCH 09/25] [td-11818] refactor. --- include/util/tpagedbuf.h | 6 +- source/libs/executor/inc/executorimpl.h | 2 +- source/libs/executor/src/executorimpl.c | 20 +++-- source/libs/executor/src/tsort.c | 12 ++- source/libs/executor/test/executorTests.cpp | 86 +++++++++++------- .../libs/executor/test/executorUtilTests.cpp | 90 +++++++++---------- source/libs/function/src/tpercentile.c | 4 +- source/util/src/tpagedbuf.c | 6 +- source/util/test/pageBufferTest.cpp | 14 +-- 9 files changed, 139 insertions(+), 101 deletions(-) diff --git a/include/util/tpagedbuf.h b/include/util/tpagedbuf.h index 708c6cf741..c1c246dd64 100644 --- a/include/util/tpagedbuf.h +++ b/include/util/tpagedbuf.h @@ -55,7 +55,7 @@ typedef struct SDiskbasedBufStatis { * @param handle * @return */ -int32_t createDiskbasedBuffer(SDiskbasedBuf** pBuf, int32_t pagesize, int32_t inMemBufSize, uint64_t qId, const char* dir); +int32_t createDiskbasedBuf(SDiskbasedBuf** pBuf, int32_t pagesize, int32_t inMemBufSize, uint64_t qId, const char* dir); /** * @@ -108,13 +108,13 @@ size_t getTotalBufSize(const SDiskbasedBuf* pBuf); * @param pBuf * @return */ -size_t getNumOfResultBufGroupId(const SDiskbasedBuf* pBuf); +size_t getNumOfBufGroupId(const SDiskbasedBuf* pBuf); /** * destroy result buffer * @param pBuf */ -void destroyResultBuf(SDiskbasedBuf* pBuf); +void destroyDiskbasedBuf(SDiskbasedBuf* pBuf); /** * diff --git a/source/libs/executor/inc/executorimpl.h b/source/libs/executor/inc/executorimpl.h index c5706c7e59..c7111f987d 100644 --- a/source/libs/executor/inc/executorimpl.h +++ b/source/libs/executor/inc/executorimpl.h @@ -627,7 +627,7 @@ SOperatorInfo* createFilterOperatorInfo(STaskRuntimeEnv* pRuntimeEnv, SOperatorI SOperatorInfo* createJoinOperatorInfo(SOperatorInfo** pdownstream, int32_t numOfDownstream, SSchema* pSchema, int32_t numOfOutput); SOperatorInfo* createOrderOperatorInfo(SOperatorInfo* downstream, SArray* pExprInfo, SArray* pOrderVal, SExecTaskInfo* pTaskInfo); -SOperatorInfo* createSortedMergeOperatorInfo(SOperatorInfo** downstream, int32_t numOfDownstream, SArray* pExprInfo, void* param, SArray* pOrderVal, SExecTaskInfo* pTaskInfo); +SOperatorInfo* createSortedMergeOperatorInfo(SOperatorInfo** downstream, int32_t numOfDownstream, SArray* pExprInfo, SArray* pOrderVal, SExecTaskInfo* pTaskInfo); // SSDataBlock* doGlobalAggregate(void* param, bool* newgroup); // SSDataBlock* doMultiwayMergeSort(void* param, bool* newgroup); diff --git a/source/libs/executor/src/executorimpl.c b/source/libs/executor/src/executorimpl.c index 75ba620738..c7c72e615d 100644 --- a/source/libs/executor/src/executorimpl.c +++ b/source/libs/executor/src/executorimpl.c @@ -2215,7 +2215,7 @@ static void teardownQueryRuntimeEnv(STaskRuntimeEnv *pRuntimeEnv) { destroyScalarFuncSupport(pRuntimeEnv->scalarSup, pQueryAttr->numOfOutput); // destroyUdfInfo(pRuntimeEnv->pUdfInfo); - destroyResultBuf(pRuntimeEnv->pResultBuf); + destroyDiskbasedBuf(pRuntimeEnv->pResultBuf); doFreeQueryHandle(pRuntimeEnv); destroyTsComp(pRuntimeEnv, pQueryAttr); @@ -4629,7 +4629,7 @@ int32_t doInitQInfo(SQInfo* pQInfo, STSBuf* pTsBuf, void* tsdb, void* sourceOptr getIntermediateBufInfo(pRuntimeEnv, &ps, &pQueryAttr->intermediateResultRowSize); int32_t TENMB = 1024*1024*10; - int32_t code = createDiskbasedBuffer(&pRuntimeEnv->pResultBuf, ps, TENMB, pQInfo->qId, tsTempDir); + int32_t code = createDiskbasedBuf(&pRuntimeEnv->pResultBuf, ps, TENMB, pQInfo->qId, tsTempDir); if (code != TSDB_CODE_SUCCESS) { return code; } @@ -5600,6 +5600,10 @@ SArray* getResultGroupCheckColumns(STaskAttr* pQuery) { } static void destroySortedMergeOperatorInfo(void* param, int32_t numOfOutput) { + SSortedMergeOperatorInfo* pInfo = (SSortedMergeOperatorInfo*) param; + taosArrayDestroy(pInfo->orderInfo); + destroySortHandle(pInfo->pSortHandle); + blockDataDestroy(pInfo->pDataBlock); } static void destroySlimitOperatorInfo(void* param, int32_t numOfOutput) { @@ -5655,7 +5659,10 @@ static SSDataBlock* getSortedBlockData(SSortHandle* pHandle, SSDataBlock* pDataB } SSDataBlock* loadNextDataBlock(void* param) { + SOperatorInfo* pOperator = (SOperatorInfo*) param; + bool newgroup = false; + return pOperator->exec(pOperator, &newgroup); } static SSDataBlock* doSortedMerge(void* param, bool* newgroup) { @@ -5672,14 +5679,17 @@ static SSDataBlock* doSortedMerge(void* param, bool* newgroup) { SSchema* p = blockDataExtractSchema(pInfo->pDataBlock, NULL); int32_t numOfBufPage = pInfo->sortBufSize / pInfo->bufPageSize; - pInfo->pSortHandle = createSortHandle(pInfo->orderInfo, pInfo->nullFirst, SORT_SINGLESOURCE, pInfo->bufPageSize, + pInfo->pSortHandle = createSortHandle(pInfo->orderInfo, pInfo->nullFirst, SORT_MULTIWAY_MERGE, pInfo->bufPageSize, numOfBufPage, p, pInfo->pDataBlock->info.numOfCols, "GET_TASKID(pTaskInfo)"); tfree(p); setFetchRawDataFp(pInfo->pSortHandle, loadNextDataBlock); for(int32_t i = 0; i < pOperator->numOfDownstream; ++i) { - sortAddSource(pInfo->pSortHandle, pOperator->pDownstream[i]); + SOperatorSource* ps = calloc(1, sizeof(SOperatorSource)); + ps->param = pOperator->pDownstream[i]; + + sortAddSource(pInfo->pSortHandle, ps); } int32_t code = sortOpen(pInfo->pSortHandle); @@ -5714,7 +5724,7 @@ static SArray* createBlockOrder(SArray* pExprInfo, SArray* pOrderVal) { return pOrderInfo; } -SOperatorInfo* createSortedMergeOperatorInfo(SOperatorInfo** downstream, int32_t numOfDownstream, SArray* pExprInfo, void* param, SArray* pOrderVal, SExecTaskInfo* pTaskInfo) { +SOperatorInfo* createSortedMergeOperatorInfo(SOperatorInfo** downstream, int32_t numOfDownstream, SArray* pExprInfo, SArray* pOrderVal, SExecTaskInfo* pTaskInfo) { SSortedMergeOperatorInfo* pInfo = calloc(1, sizeof(SSortedMergeOperatorInfo)); SOperatorInfo* pOperator = calloc(1, sizeof(SOperatorInfo)); if (pInfo == NULL || pOperator == NULL) { diff --git a/source/libs/executor/src/tsort.c b/source/libs/executor/src/tsort.c index 1d42f1f838..196e22d92d 100644 --- a/source/libs/executor/src/tsort.c +++ b/source/libs/executor/src/tsort.c @@ -39,7 +39,7 @@ typedef struct SSortHandle { bool nullFirst; bool hasVarCol; - SArray *pSources; + SArray *pSources; // TODO refactor, remove it SArray *pOrderedSource; _sort_fetch_block_fn_t fetchfp; @@ -119,6 +119,7 @@ void destroySortHandle(SSortHandle* pSortHandle) { tMergeTreeDestroy(pSortHandle->pMergeTree); } + destroyDiskbasedBuf(pSortHandle->pBuf); tfree(pSortHandle->idStr); tfree(pSortHandle); } @@ -171,7 +172,7 @@ static int32_t doAddToBuf(SSDataBlock* pDataBlock, SSortHandle* pHandle) { int32_t start = 0; if (pHandle->pBuf == NULL) { - int32_t code = createDiskbasedBuffer(&pHandle->pBuf, pHandle->pageSize, pHandle->numOfPages * pHandle->pageSize, 0, "/tmp"); + int32_t code = createDiskbasedBuf(&pHandle->pBuf, pHandle->pageSize, pHandle->numOfPages * pHandle->pageSize, 0, "/tmp"); setPrintStatis(pHandle->pBuf); if (code != TSDB_CODE_SUCCESS) { return code; @@ -235,7 +236,7 @@ static int32_t sortComparInit(SMsortComparParam* cmpParam, SArray* pSources, int } else { // multi-pass internal merge sort is required if (pHandle->pBuf == NULL) { - code = createDiskbasedBuffer(&pHandle->pBuf, pHandle->pageSize, pHandle->numOfPages * pHandle->pageSize, 0, "/tmp"); + code = createDiskbasedBuf(&pHandle->pBuf, pHandle->pageSize, pHandle->numOfPages * pHandle->pageSize, 0, "/tmp"); setPrintStatis(pHandle->pBuf); if (code != TSDB_CODE_SUCCESS) { return code; @@ -614,7 +615,10 @@ int32_t sortOpen(SSortHandle* pHandle) { } int32_t numOfSources = taosArrayGetSize(pHandle->pOrderedSource); - ASSERT(numOfSources <= getNumOfInMemBufPages(pHandle->pBuf)); + if (pHandle->pBuf != NULL) { + ASSERT(numOfSources <= getNumOfInMemBufPages(pHandle->pBuf)); + } + code = sortComparInit(&pHandle->cmpParam, pHandle->pOrderedSource, 0, numOfSources - 1, pHandle); if (code != TSDB_CODE_SUCCESS) { return code; diff --git a/source/libs/executor/test/executorTests.cpp b/source/libs/executor/test/executorTests.cpp index 160bf1f7ca..2651b95b0f 100644 --- a/source/libs/executor/test/executorTests.cpp +++ b/source/libs/executor/test/executorTests.cpp @@ -35,10 +35,17 @@ namespace { +enum { + data_rand = 0x1, + data_asc = 0x2, + data_desc = 0x3, +}; + typedef struct SDummyInputInfo { int32_t max; int32_t current; int32_t startVal; + int32_t type; SSDataBlock* pBlock; } SDummyInputInfo; @@ -83,10 +90,18 @@ SSDataBlock* getDummyBlock(void* param, bool* newgroup) { char buf[128] = {0}; char b1[128] = {0}; + int32_t v = 0; for(int32_t i = 0; i < numOfRows; ++i) { SColumnInfoData* pColInfo = static_cast(TARRAY_GET_ELEM(pBlock->pDataBlock, 0)); - int32_t v = (--pInfo->startVal); + if (pInfo->type == data_desc) { + v = (--pInfo->startVal); + } else if (pInfo->type == data_asc) { + v = ++pInfo->startVal; + } else if (pInfo->type == data_rand) { + v = random(); + } + colDataAppend(pColInfo, i, reinterpret_cast(&v), false); // sprintf(buf, "this is %d row", i); @@ -103,7 +118,7 @@ SSDataBlock* getDummyBlock(void* param, bool* newgroup) { return pBlock; } -SOperatorInfo* createDummyOperator(int32_t numOfBlocks) { +SOperatorInfo* createDummyOperator(int32_t numOfBlocks, int32_t type) { SOperatorInfo* pOperator = static_cast(calloc(1, sizeof(SOperatorInfo))); pOperator->name = "dummyInputOpertor4Test"; pOperator->exec = getDummyBlock; @@ -111,6 +126,7 @@ SOperatorInfo* createDummyOperator(int32_t numOfBlocks) { SDummyInputInfo *pInfo = (SDummyInputInfo*) calloc(1, sizeof(SDummyInputInfo)); pInfo->max = numOfBlocks; pInfo->startVal = 1500000; + pInfo->type = type; pOperator->info = pInfo; return pOperator; @@ -121,6 +137,7 @@ int main(int argc, char** argv) { return RUN_ALL_TESTS(); } +#if 0 TEST(testCase, build_executor_tree_Test) { const char* msg = "{\n" "\t\"Id\":\t{\n" @@ -216,34 +233,34 @@ TEST(testCase, build_executor_tree_Test) { // int32_t code = qCreateExecTask(&handle, 2, 1, NULL, (void**) &pTaskInfo, &sinkHandle); } -//TEST(testCase, inMem_sort_Test) { -// SArray* pOrderVal = taosArrayInit(4, sizeof(SOrder)); -// SOrder o = {.order = TSDB_ORDER_ASC}; -// o.col.info.colId = 1; -// o.col.info.type = TSDB_DATA_TYPE_INT; -// taosArrayPush(pOrderVal, &o); -// -// SArray* pExprInfo = taosArrayInit(4, sizeof(SExprInfo)); -// SExprInfo *exp = static_cast(calloc(1, sizeof(SExprInfo))); -// exp->base.resSchema = createSchema(TSDB_DATA_TYPE_INT, sizeof(int32_t), 1, "res"); -// taosArrayPush(pExprInfo, &exp); -// -// SExprInfo *exp1 = static_cast(calloc(1, sizeof(SExprInfo))); -// exp1->base.resSchema = createSchema(TSDB_DATA_TYPE_BINARY, 40, 2, "res1"); -// taosArrayPush(pExprInfo, &exp1); -// -// SOperatorInfo* pOperator = createOrderOperatorInfo(createDummyOperator(5), pExprInfo, pOrderVal); -// -// bool newgroup = false; -// SSDataBlock* pRes = pOperator->exec(pOperator, &newgroup); -// -// SColumnInfoData* pCol1 = static_cast(taosArrayGet(pRes->pDataBlock, 0)); -// SColumnInfoData* pCol2 = static_cast(taosArrayGet(pRes->pDataBlock, 1)); -// for(int32_t i = 0; i < pRes->info.rows; ++i) { -// char* p = colDataGet(pCol2, i); -// printf("%d: %d, %s\n", i, ((int32_t*)pCol1->pData)[i], (char*)varDataVal(p)); -// } -//} +TEST(testCase, inMem_sort_Test) { + SArray* pOrderVal = taosArrayInit(4, sizeof(SOrder)); + SOrder o = {.order = TSDB_ORDER_ASC}; + o.col.info.colId = 1; + o.col.info.type = TSDB_DATA_TYPE_INT; + taosArrayPush(pOrderVal, &o); + + SArray* pExprInfo = taosArrayInit(4, sizeof(SExprInfo)); + SExprInfo *exp = static_cast(calloc(1, sizeof(SExprInfo))); + exp->base.resSchema = createSchema(TSDB_DATA_TYPE_INT, sizeof(int32_t), 1, "res"); + taosArrayPush(pExprInfo, &exp); + + SExprInfo *exp1 = static_cast(calloc(1, sizeof(SExprInfo))); + exp1->base.resSchema = createSchema(TSDB_DATA_TYPE_BINARY, 40, 2, "res1"); + taosArrayPush(pExprInfo, &exp1); + + SOperatorInfo* pOperator = createOrderOperatorInfo(createDummyOperator(5), pExprInfo, pOrderVal, NULL); + + bool newgroup = false; + SSDataBlock* pRes = pOperator->exec(pOperator, &newgroup); + + SColumnInfoData* pCol1 = static_cast(taosArrayGet(pRes->pDataBlock, 0)); + SColumnInfoData* pCol2 = static_cast(taosArrayGet(pRes->pDataBlock, 1)); + for(int32_t i = 0; i < pRes->info.rows; ++i) { + char* p = colDataGet(pCol2, i); + printf("%d: %d, %s\n", i, ((int32_t*)pCol1->pData)[i], (char*)varDataVal(p)); + } +} typedef struct su { int32_t v; @@ -339,6 +356,7 @@ TEST(testCase, external_sort_Test) { taosArrayDestroy(pExprInfo); taosArrayDestroy(pOrderVal); } +#endif TEST(testCase, sorted_merge_Test) { srand(time(NULL)); @@ -359,7 +377,13 @@ TEST(testCase, sorted_merge_Test) { exp1->base.resSchema = createSchema(TSDB_DATA_TYPE_BINARY, 40, 2, "res1"); // taosArrayPush(pExprInfo, &exp1); - SOperatorInfo* pOperator = createOrderOperatorInfo(createDummyOperator(1500), pExprInfo, pOrderVal, NULL); + int32_t numOfSources = 10; + SOperatorInfo** plist = (SOperatorInfo**) calloc(numOfSources, sizeof(void*)); + for(int32_t i = 0; i < numOfSources; ++i) { + plist[i] = createDummyOperator(1, data_asc); + } + + SOperatorInfo* pOperator = createSortedMergeOperatorInfo(plist, numOfSources, pExprInfo, pOrderVal, NULL); bool newgroup = false; SSDataBlock* pRes = NULL; diff --git a/source/libs/executor/test/executorUtilTests.cpp b/source/libs/executor/test/executorUtilTests.cpp index 57691d6d11..5c94b89c3d 100644 --- a/source/libs/executor/test/executorUtilTests.cpp +++ b/source/libs/executor/test/executorUtilTests.cpp @@ -219,50 +219,50 @@ int32_t docomp(const void* p1, const void* p2, void* param) { // destroySortHandle(phandle); //} -TEST(testCase, ordered_merge_sort_Test) { - SArray* pOrderVal = taosArrayInit(4, sizeof(SOrder)); - SOrder o = {.order = TSDB_ORDER_ASC}; - o.col.info.colId = 1; - o.col.info.type = TSDB_DATA_TYPE_INT; - taosArrayPush(pOrderVal, &o); - - int32_t numOfRows = 1000; - SBlockOrderInfo oi = {0}; - oi.order = TSDB_ORDER_ASC; - oi.colIndex = 0; - SArray* orderInfo = taosArrayInit(1, sizeof(SBlockOrderInfo)); - taosArrayPush(orderInfo, &oi); - - SSchema s = {.type = TSDB_DATA_TYPE_INT, .colId = 1, .bytes = 4}; - SSortHandle* phandle = createSortHandle(orderInfo, false, SORT_MULTIWAY_MERGE, 1024, 5, &s, 1,"test_abc"); - setFetchRawDataFp(phandle, getSingleColDummyBlock); - setComparFn(phandle, docomp); - - for(int32_t i = 0; i < 10; ++i) { - SOperatorSource* p = static_cast(calloc(1, sizeof(SOperatorSource))); - _info* c = static_cast<_info*>(calloc(1, sizeof(_info))); - c->count = 1; - c->pageRows = 1000; - c->startVal = 0; - - p->param = c; - sortAddSource(phandle, p); - } - - int32_t code = sortOpen(phandle); - int32_t row = 1; - - while(1) { - STupleHandle* pTupleHandle = sortNextTuple(phandle); - if (pTupleHandle == NULL) { - break; - } - - void* v = sortGetValue(pTupleHandle, 0); - printf("%d: %d\n", row++, *(int32_t*) v); - - } - destroySortHandle(phandle); -} +//TEST(testCase, ordered_merge_sort_Test) { +// SArray* pOrderVal = taosArrayInit(4, sizeof(SOrder)); +// SOrder o = {.order = TSDB_ORDER_ASC}; +// o.col.info.colId = 1; +// o.col.info.type = TSDB_DATA_TYPE_INT; +// taosArrayPush(pOrderVal, &o); +// +// int32_t numOfRows = 1000; +// SBlockOrderInfo oi = {0}; +// oi.order = TSDB_ORDER_ASC; +// oi.colIndex = 0; +// SArray* orderInfo = taosArrayInit(1, sizeof(SBlockOrderInfo)); +// taosArrayPush(orderInfo, &oi); +// +// SSchema s = {.type = TSDB_DATA_TYPE_INT, .colId = 1, .bytes = 4}; +// SSortHandle* phandle = createSortHandle(orderInfo, false, SORT_MULTIWAY_MERGE, 1024, 5, &s, 1,"test_abc"); +// setFetchRawDataFp(phandle, getSingleColDummyBlock); +// setComparFn(phandle, docomp); +// +// for(int32_t i = 0; i < 10; ++i) { +// SOperatorSource* p = static_cast(calloc(1, sizeof(SOperatorSource))); +// _info* c = static_cast<_info*>(calloc(1, sizeof(_info))); +// c->count = 1; +// c->pageRows = 1000; +// c->startVal = 0; +// +// p->param = c; +// sortAddSource(phandle, p); +// } +// +// int32_t code = sortOpen(phandle); +// int32_t row = 1; +// +// while(1) { +// STupleHandle* pTupleHandle = sortNextTuple(phandle); +// if (pTupleHandle == NULL) { +// break; +// } +// +// void* v = sortGetValue(pTupleHandle, 0); +// printf("%d: %d\n", row++, *(int32_t*) v); +// +// } +// destroySortHandle(phandle); +//} #pragma GCC diagnostic pop diff --git a/source/libs/function/src/tpercentile.c b/source/libs/function/src/tpercentile.c index 40731adc58..c6ab125362 100644 --- a/source/libs/function/src/tpercentile.c +++ b/source/libs/function/src/tpercentile.c @@ -254,7 +254,7 @@ tMemBucket *tMemBucketCreate(int16_t nElemSize, int16_t dataType, double minval, resetSlotInfo(pBucket); - int32_t ret = createDiskbasedBuffer(&pBucket->pBuffer, pBucket->bufPageSize, pBucket->bufPageSize * 512, 1, tsTempDir); + int32_t ret = createDiskbasedBuf(&pBucket->pBuffer, pBucket->bufPageSize, pBucket->bufPageSize * 512, 1, tsTempDir); if (ret != 0) { tMemBucketDestroy(pBucket); return NULL; @@ -269,7 +269,7 @@ void tMemBucketDestroy(tMemBucket *pBucket) { return; } - destroyResultBuf(pBucket->pBuffer); + destroyDiskbasedBuf(pBucket->pBuffer); tfree(pBucket->pSlots); tfree(pBucket); } diff --git a/source/util/src/tpagedbuf.c b/source/util/src/tpagedbuf.c index d5415c219f..f014d8e07e 100644 --- a/source/util/src/tpagedbuf.c +++ b/source/util/src/tpagedbuf.c @@ -53,7 +53,7 @@ typedef struct SDiskbasedBuf { static void printStatisData(const SDiskbasedBuf* pBuf); -int32_t createDiskbasedBuffer(SDiskbasedBuf** pBuf, int32_t pagesize, int32_t inMemBufSize, uint64_t qId, const char* dir) { +int32_t createDiskbasedBuf(SDiskbasedBuf** pBuf, int32_t pagesize, int32_t inMemBufSize, uint64_t qId, const char* dir) { *pBuf = calloc(1, sizeof(SDiskbasedBuf)); SDiskbasedBuf* pResBuf = *pBuf; @@ -473,7 +473,7 @@ void releaseBufPageInfo(SDiskbasedBuf* pBuf, SPageInfo* pi) { pBuf->statis.releasePages += 1; } -size_t getNumOfResultBufGroupId(const SDiskbasedBuf* pBuf) { return taosHashGetSize(pBuf->groupSet); } +size_t getNumOfBufGroupId(const SDiskbasedBuf* pBuf) { return taosHashGetSize(pBuf->groupSet); } size_t getTotalBufSize(const SDiskbasedBuf* pBuf) { return (size_t)pBuf->totalBufSize; } @@ -488,7 +488,7 @@ SIDList getDataBufPagesIdList(SDiskbasedBuf* pBuf, int32_t groupId) { } } -void destroyResultBuf(SDiskbasedBuf* pBuf) { +void destroyDiskbasedBuf(SDiskbasedBuf* pBuf) { if (pBuf == NULL) { return; } diff --git a/source/util/test/pageBufferTest.cpp b/source/util/test/pageBufferTest.cpp index 8fa8216223..d310ff450b 100644 --- a/source/util/test/pageBufferTest.cpp +++ b/source/util/test/pageBufferTest.cpp @@ -13,7 +13,7 @@ namespace { // simple test void simpleTest() { SDiskbasedBuf* pResultBuf = NULL; - int32_t ret = createDiskbasedBuffer(&pResultBuf, 1024, 4096, 1, "/tmp/"); + int32_t ret = createDiskbasedBuf(&pResultBuf, 1024, 4096, 1, "/tmp/"); int32_t pageId = 0; int32_t groupId = 0; @@ -25,7 +25,7 @@ void simpleTest() { SIDList list = getDataBufPagesIdList(pResultBuf, groupId); ASSERT_EQ(taosArrayGetSize(list), 1); - ASSERT_EQ(getNumOfResultBufGroupId(pResultBuf), 1); + ASSERT_EQ(getNumOfBufGroupId(pResultBuf), 1); releaseBufPage(pResultBuf, pBufPage); @@ -50,12 +50,12 @@ void simpleTest() { SFilePage* t4 = getBufPage(pResultBuf, pageId); ASSERT_TRUE(t4 == pBufPage5); - destroyResultBuf(pResultBuf); + destroyDiskbasedBuf(pResultBuf); } void writeDownTest() { SDiskbasedBuf* pResultBuf = NULL; - int32_t ret = createDiskbasedBuffer(&pResultBuf, 1024, 4*1024, 1, "/tmp/"); + int32_t ret = createDiskbasedBuf(&pResultBuf, 1024, 4*1024, 1, "/tmp/"); int32_t pageId = 0; int32_t writePageId = 0; @@ -97,12 +97,12 @@ void writeDownTest() { SArray* pa = getDataBufPagesIdList(pResultBuf, groupId); ASSERT_EQ(taosArrayGetSize(pa), 5); - destroyResultBuf(pResultBuf); + destroyDiskbasedBuf(pResultBuf); } void recyclePageTest() { SDiskbasedBuf* pResultBuf = NULL; - int32_t ret = createDiskbasedBuffer(&pResultBuf, 1024, 4*1024, 1, "/tmp/"); + int32_t ret = createDiskbasedBuf(&pResultBuf, 1024, 4*1024, 1, "/tmp/"); int32_t pageId = 0; int32_t writePageId = 0; @@ -150,7 +150,7 @@ void recyclePageTest() { SArray* pa = getDataBufPagesIdList(pResultBuf, groupId); ASSERT_EQ(taosArrayGetSize(pa), 6); - destroyResultBuf(pResultBuf); + destroyDiskbasedBuf(pResultBuf); } } // namespace From 6c1f75fe6efc2b06134727cec39ef31c63958d6f Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 21 Feb 2022 13:28:20 +0800 Subject: [PATCH 10/25] [td-11818] refactor tsort. --- source/libs/executor/inc/executorimpl.h | 9 +- source/libs/executor/inc/tsort.h | 10 +-- source/libs/executor/src/executorimpl.c | 33 ++++--- source/libs/executor/src/tsort.c | 71 ++++++++------- source/libs/executor/test/executorTests.cpp | 4 +- .../libs/executor/test/executorUtilTests.cpp | 88 ++++++++++--------- source/util/src/tpagedbuf.c | 2 +- 7 files changed, 119 insertions(+), 98 deletions(-) diff --git a/source/libs/executor/inc/executorimpl.h b/source/libs/executor/inc/executorimpl.h index c7111f987d..362f40696c 100644 --- a/source/libs/executor/inc/executorimpl.h +++ b/source/libs/executor/inc/executorimpl.h @@ -551,7 +551,9 @@ typedef struct SDistinctOperatorInfo { } SDistinctOperatorInfo; typedef struct SSortedMergeOperatorInfo { - SSDataBlock *pDataBlock; + SOptrBasicInfo binfo; + +// SSDataBlock *pDataBlock; bool hasVarCol; SArray *orderInfo; // SArray @@ -563,6 +565,11 @@ typedef struct SSortedMergeOperatorInfo { int32_t bufPageSize; uint32_t sortBufSize; // max buffer size for in-memory sort int32_t numOfRowsInRes; + + char** prevRow; + int32_t resultRowFactor; + bool multiGroupResults; + bool hasGroupColData; } SSortedMergeOperatorInfo; typedef struct SOrderOperatorInfo { diff --git a/source/libs/executor/inc/tsort.h b/source/libs/executor/inc/tsort.h index 77a38f147e..92258f619f 100644 --- a/source/libs/executor/inc/tsort.h +++ b/source/libs/executor/inc/tsort.h @@ -24,8 +24,8 @@ extern "C" { #include "os.h" enum { - SORT_MULTIWAY_MERGE = 0x1, - SORT_SINGLESOURCE = 0x2, + SORT_MULTISOURCE_MERGE = 0x1, + SORT_SINGLESOURCE_SORT = 0x2, }; typedef struct SMultiMergeSource { @@ -40,10 +40,10 @@ typedef struct SExternalMemSource { int32_t pageIndex; } SExternalMemSource; -typedef struct SOperatorSource { +typedef struct SGenericSource { SMultiMergeSource src; - void* param; -} SOperatorSource; + void *param; +} SGenericSource; typedef struct SMsortComparParam { void **pSources; diff --git a/source/libs/executor/src/executorimpl.c b/source/libs/executor/src/executorimpl.c index c7c72e615d..5286bd7ba1 100644 --- a/source/libs/executor/src/executorimpl.c +++ b/source/libs/executor/src/executorimpl.c @@ -5603,7 +5603,7 @@ static void destroySortedMergeOperatorInfo(void* param, int32_t numOfOutput) { SSortedMergeOperatorInfo* pInfo = (SSortedMergeOperatorInfo*) param; taosArrayDestroy(pInfo->orderInfo); destroySortHandle(pInfo->pSortHandle); - blockDataDestroy(pInfo->pDataBlock); + blockDataDestroy(pInfo->binfo.pRes); } static void destroySlimitOperatorInfo(void* param, int32_t numOfOutput) { @@ -5624,6 +5624,7 @@ static SExprInfo* exprArrayDup(SArray* pExprInfo) { return p; } +// TODO merge aggregate super table static void appendOneRowToDataBlock(SSDataBlock *pBlock, STupleHandle* pTupleHandle) { for (int32_t i = 0; i < pBlock->info.numOfCols; ++i) { SColumnInfoData* pColInfo = taosArrayGet(pBlock->pDataBlock, i); @@ -5674,21 +5675,20 @@ static SSDataBlock* doSortedMerge(void* param, bool* newgroup) { SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; SSortedMergeOperatorInfo* pInfo = pOperator->info; if (pOperator->status == OP_RES_TO_RETURN) { - return getSortedBlockData(pInfo->pSortHandle, pInfo->pDataBlock, pInfo->hasVarCol, pInfo->numOfRowsInRes); + return getSortedBlockData(pInfo->pSortHandle, pInfo->binfo.pRes, pInfo->hasVarCol, pInfo->numOfRowsInRes); } - SSchema* p = blockDataExtractSchema(pInfo->pDataBlock, NULL); + SSchema* p = blockDataExtractSchema(pInfo->binfo.pRes, NULL); int32_t numOfBufPage = pInfo->sortBufSize / pInfo->bufPageSize; - pInfo->pSortHandle = createSortHandle(pInfo->orderInfo, pInfo->nullFirst, SORT_MULTIWAY_MERGE, pInfo->bufPageSize, - numOfBufPage, p, pInfo->pDataBlock->info.numOfCols, "GET_TASKID(pTaskInfo)"); + pInfo->pSortHandle = createSortHandle(pInfo->orderInfo, pInfo->nullFirst, SORT_MULTISOURCE_MERGE, pInfo->bufPageSize, + numOfBufPage, p, pInfo->binfo.pRes->info.numOfCols, "GET_TASKID(pTaskInfo)"); tfree(p); setFetchRawDataFp(pInfo->pSortHandle, loadNextDataBlock); for(int32_t i = 0; i < pOperator->numOfDownstream; ++i) { - SOperatorSource* ps = calloc(1, sizeof(SOperatorSource)); + SGenericSource* ps = calloc(1, sizeof(SGenericSource)); ps->param = pOperator->pDownstream[i]; - sortAddSource(pInfo->pSortHandle, ps); } @@ -5698,7 +5698,7 @@ static SSDataBlock* doSortedMerge(void* param, bool* newgroup) { } pOperator->status = OP_RES_TO_RETURN; - return getSortedBlockData(pInfo->pSortHandle, pInfo->pDataBlock, pInfo->hasVarCol, pInfo->numOfRowsInRes); + return getSortedBlockData(pInfo->pSortHandle, pInfo->binfo.pRes, pInfo->hasVarCol, pInfo->numOfRowsInRes); } static SArray* createBlockOrder(SArray* pExprInfo, SArray* pOrderVal) { @@ -5729,18 +5729,22 @@ SOperatorInfo* createSortedMergeOperatorInfo(SOperatorInfo** downstream, int32_t SOperatorInfo* pOperator = calloc(1, sizeof(SOperatorInfo)); if (pInfo == NULL || pOperator == NULL) { tfree(pInfo); - + tfree(pOperator); terrno = TSDB_CODE_QRY_OUT_OF_MEMORY; return NULL; } + int32_t numOfOutput = taosArrayGetSize(pExprInfo); + pInfo->binfo.capacity = 4096; + pInfo->binfo.pCtx = createSqlFunctionCtx_rv(pExprInfo, &pInfo->binfo.rowCellInfoOffset, &pInfo->binfo.resRowSize); + // pInfo->resultRowFactor = // (int32_t)(getRowNumForMultioutput(pRuntimeEnv->pQueryAttr, pRuntimeEnv->pQueryAttr->topBotQuery, false)); pInfo->sortBufSize = 1024 * 16; // 1MB pInfo->bufPageSize = 1024; pInfo->numOfRowsInRes = 1024; - pInfo->pDataBlock = createOutputBuf_rv(pExprInfo, pInfo->numOfRowsInRes); - pInfo->orderInfo = createBlockOrder(pExprInfo, pOrderVal); + pInfo->binfo.pRes = createOutputBuf_rv(pExprInfo, pInfo->numOfRowsInRes); + pInfo->orderInfo = createBlockOrder(pExprInfo, pOrderVal); int32_t numOfRows = 1; @@ -5749,6 +5753,7 @@ SOperatorInfo* createSortedMergeOperatorInfo(SOperatorInfo** downstream, int32_t pOperator->blockingOptr = true; pOperator->status = OP_IN_EXECUTING; pOperator->info = pInfo; + pOperator->numOfOutput = numOfOutput; pOperator->pTaskInfo = pTaskInfo; pOperator->exec = doSortedMerge; @@ -5775,13 +5780,15 @@ static SSDataBlock* doSort(void* param, bool* newgroup) { SSchema* p = blockDataExtractSchema(pInfo->pDataBlock, NULL); int32_t numOfBufPage = pInfo->sortBufSize / pInfo->bufPageSize; - pInfo->pSortHandle = createSortHandle(pInfo->orderInfo, pInfo->nullFirst, SORT_SINGLESOURCE, pInfo->bufPageSize, + pInfo->pSortHandle = createSortHandle(pInfo->orderInfo, pInfo->nullFirst, SORT_SINGLESOURCE_SORT, pInfo->bufPageSize, numOfBufPage, p, pInfo->pDataBlock->info.numOfCols, "GET_TASKID(pTaskInfo)"); tfree(p); setFetchRawDataFp(pInfo->pSortHandle, loadNextDataBlock); - sortAddSource(pInfo->pSortHandle, pOperator); + SGenericSource* ps = calloc(1, sizeof(SGenericSource)); + ps->param = pOperator; + sortAddSource(pInfo->pSortHandle, ps); // TODO set error code; int32_t code = sortOpen(pInfo->pSortHandle); diff --git a/source/libs/executor/src/tsort.c b/source/libs/executor/src/tsort.c index 196e22d92d..4854f0cbed 100644 --- a/source/libs/executor/src/tsort.c +++ b/source/libs/executor/src/tsort.c @@ -38,8 +38,6 @@ typedef struct SSortHandle { SArray *pOrderInfo; bool nullFirst; bool hasVarCol; - - SArray *pSources; // TODO refactor, remove it SArray *pOrderedSource; _sort_fetch_block_fn_t fetchfp; @@ -96,7 +94,6 @@ SSortHandle* createSortHandle(SArray* pOrderInfo, bool nullFirst, int32_t type, pSortHandle->pageSize = pageSize; pSortHandle->numOfPages = numOfPages; pSortHandle->pOrderedSource = taosArrayInit(4, POINTER_BYTES); - pSortHandle->pSources = taosArrayInit(4, POINTER_BYTES); pSortHandle->pOrderInfo = pOrderInfo; pSortHandle->nullFirst = nullFirst; pSortHandle->cmpParam.orderInfo = pOrderInfo; @@ -113,8 +110,6 @@ SSortHandle* createSortHandle(SArray* pOrderInfo, bool nullFirst, int32_t type, void destroySortHandle(SSortHandle* pSortHandle) { sortClose(pSortHandle); - - taosArrayDestroy(pSortHandle->pSources); if (pSortHandle->pMergeTree != NULL) { tMergeTreeDestroy(pSortHandle->pMergeTree); } @@ -125,11 +120,7 @@ void destroySortHandle(SSortHandle* pSortHandle) { } int32_t sortAddSource(SSortHandle* pSortHandle, void* pSource) { - if (pSortHandle->type == SORT_SINGLESOURCE) { - pSortHandle->pParam = pSource; - } else { - taosArrayPush(pSortHandle->pOrderedSource, &pSource); - } + taosArrayPush(pSortHandle->pOrderedSource, &pSource); } static SSDataBlock* createDataBlock(const SSDataBlock* pDataBlock) { @@ -220,7 +211,7 @@ static int32_t sortComparInit(SMsortComparParam* cmpParam, SArray* pSources, int int32_t code = 0; - if (pHandle->type == SORT_SINGLESOURCE) { + if (pHandle->type == SORT_SINGLESOURCE_SORT) { for (int32_t i = 0; i < cmpParam->numOfSources; ++i) { SExternalMemSource* pSource = cmpParam->pSources[i]; SPageInfo* pPgInfo = *(SPageInfo**)taosArrayGet(pSource->pageIdList, pSource->pageIndex); @@ -244,7 +235,7 @@ static int32_t sortComparInit(SMsortComparParam* cmpParam, SArray* pSources, int } for (int32_t i = 0; i < cmpParam->numOfSources; ++i) { - SOperatorSource* pSource = cmpParam->pSources[i]; + SGenericSource* pSource = cmpParam->pSources[i]; pSource->src.pBlock = pHandle->fetchfp(pSource->param); } } @@ -296,7 +287,7 @@ static int32_t adjustMergeTreeForNextTuple(SExternalMemSource *pSource, SMultiwa pSource->pageIndex = -1; pSource->src.pBlock = blockDataDestroy(pSource->src.pBlock); } else { - if (pHandle->type == SORT_SINGLESOURCE) { + if (pHandle->type == SORT_SINGLESOURCE_SORT) { SPageInfo* pPgInfo = *(SPageInfo**)taosArrayGet(pSource->pageIdList, pSource->pageIndex); SFilePage* pPage = getBufPage(pHandle->pBuf, getPageId(pPgInfo)); @@ -307,7 +298,7 @@ static int32_t adjustMergeTreeForNextTuple(SExternalMemSource *pSource, SMultiwa releaseBufPage(pHandle->pBuf, pPage); } else { - pSource->src.pBlock = pHandle->fetchfp(((SOperatorSource*)pSource)->param); + pSource->src.pBlock = pHandle->fetchfp(((SGenericSource*)pSource)->param); if (pSource->src.pBlock == NULL) { (*numOfCompleted) += 1; pSource->src.rowIndex = -1; @@ -530,8 +521,8 @@ static int32_t doInternalMergeSort(SSortHandle* pHandle) { qDebug("%s %d round mergesort, elapsed:%"PRId64" readDisk:%.2f Kb, flushDisk:%.2f Kb", pHandle->idStr, t + 1, el, statis.loadBytes/1024.0, statis.flushBytes/1024.0); - if (pHandle->type == SORT_MULTIWAY_MERGE) { - pHandle->type = SORT_SINGLESOURCE; + if (pHandle->type == SORT_MULTISOURCE_MERGE) { + pHandle->type = SORT_SINGLESOURCE_SORT; pHandle->comparFn = msortComparFn; } } @@ -540,26 +531,15 @@ static int32_t doInternalMergeSort(SSortHandle* pHandle) { return 0; } -int32_t sortOpen(SSortHandle* pHandle) { - if (pHandle->opened) { - return 0; - } - - if (pHandle->fetchfp == NULL || pHandle->comparFn == NULL) { - return -1; - } - - pHandle->opened = true; - +static int32_t createInitialSortedMultiSources(SSortHandle* pHandle) { size_t sortBufSize = pHandle->numOfPages * pHandle->pageSize; - if (pHandle->type == SORT_SINGLESOURCE) { - if (pHandle->pParam == NULL) { - qError("%s sort source not set yet", pHandle->idStr); - return -1; - } + + if (pHandle->type == SORT_SINGLESOURCE_SORT) { + SGenericSource* source = taosArrayGetP(pHandle->pOrderedSource, 0); + taosArrayClear(pHandle->pOrderedSource); while (1) { - SSDataBlock* pBlock = pHandle->fetchfp(pHandle->pParam); + SSDataBlock* pBlock = pHandle->fetchfp(source->param); if (pBlock == NULL) { break; } @@ -604,12 +584,31 @@ int32_t sortOpen(SSortHandle* pHandle) { doAddToBuf(pHandle->pDataBlock, pHandle); } } - } else { - // do nothing + + tfree(source); + } + + return TSDB_CODE_SUCCESS; +} + +int32_t sortOpen(SSortHandle* pHandle) { + if (pHandle->opened) { + return 0; + } + + if (pHandle->fetchfp == NULL || pHandle->comparFn == NULL) { + return -1; + } + + pHandle->opened = true; + + int32_t code = createInitialSortedMultiSources(pHandle); + if (code != TSDB_CODE_SUCCESS) { + return code; } // do internal sort - int32_t code = doInternalMergeSort(pHandle); + code = doInternalMergeSort(pHandle); if (code != TSDB_CODE_SUCCESS) { return code; } diff --git a/source/libs/executor/test/executorTests.cpp b/source/libs/executor/test/executorTests.cpp index 2651b95b0f..455e731c05 100644 --- a/source/libs/executor/test/executorTests.cpp +++ b/source/libs/executor/test/executorTests.cpp @@ -356,7 +356,7 @@ TEST(testCase, external_sort_Test) { taosArrayDestroy(pExprInfo); taosArrayDestroy(pOrderVal); } -#endif + TEST(testCase, sorted_merge_Test) { srand(time(NULL)); @@ -424,5 +424,5 @@ TEST(testCase, sorted_merge_Test) { taosArrayDestroy(pExprInfo); taosArrayDestroy(pOrderVal); } - +#endif #pragma GCC diagnostic pop diff --git a/source/libs/executor/test/executorUtilTests.cpp b/source/libs/executor/test/executorUtilTests.cpp index 5c94b89c3d..754d4b6b80 100644 --- a/source/libs/executor/test/executorUtilTests.cpp +++ b/source/libs/executor/test/executorUtilTests.cpp @@ -76,12 +76,12 @@ int32_t docomp(const void* p1, const void* p2, void* param) { int32_t pRightIdx = *(int32_t *)p2; SMsortComparParam *pParam = (SMsortComparParam *)param; - SOperatorSource** px = reinterpret_cast(pParam->pSources); + SGenericSource** px = reinterpret_cast(pParam->pSources); SArray *pInfo = pParam->orderInfo; - SOperatorSource* pLeftSource = px[pLeftIdx]; - SOperatorSource* pRightSource = px[pRightIdx]; + SGenericSource* pLeftSource = px[pLeftIdx]; + SGenericSource* pRightSource = px[pRightIdx]; // this input is exhausted, set the special value to denote this if (pLeftSource->src.rowIndex == -1) { @@ -162,7 +162,7 @@ int32_t docomp(const void* p1, const void* p2, void* param) { // SArray* orderInfo = taosArrayInit(1, sizeof(SBlockOrderInfo)); // taosArrayPush(orderInfo, &oi); // -// SSortHandle* phandle = createSortHandle(orderInfo, false, SORT_SINGLESOURCE, 1024, 5, "test_abc"); +// SSortHandle* phandle = createSortHandle(orderInfo, false, SORT_SINGLESOURCE_SORT, 1024, 5, "test_abc"); // setFetchRawDataFp(phandle, getSingleColDummyBlock); // sortAddSource(phandle, &numOfRows); // @@ -182,42 +182,50 @@ int32_t docomp(const void* p1, const void* p2, void* param) { // destroySortHandle(phandle); //} // -//TEST(testCase, external_mem_sort_Test) { -// totalcount = 50; -// startVal = 100000; -// -// SArray* pOrderVal = taosArrayInit(4, sizeof(SOrder)); -// SOrder o = {.order = TSDB_ORDER_ASC}; -// o.col.info.colId = 1; -// o.col.info.type = TSDB_DATA_TYPE_INT; -// taosArrayPush(pOrderVal, &o); -// +TEST(testCase, external_mem_sort_Test) { + SArray* pOrderVal = taosArrayInit(4, sizeof(SOrder)); + SOrder o = {.order = TSDB_ORDER_ASC}; + o.col.info.colId = 1; + o.col.info.type = TSDB_DATA_TYPE_INT; + taosArrayPush(pOrderVal, &o); + // int32_t numOfRows = 1000; -// SBlockOrderInfo oi = {0}; -// oi.order = TSDB_ORDER_ASC; -// oi.colIndex = 0; -// SArray* orderInfo = taosArrayInit(1, sizeof(SBlockOrderInfo)); -// taosArrayPush(orderInfo, &oi); -// -// SSortHandle* phandle = createSortHandle(orderInfo, false, SORT_SINGLESOURCE, 1024, 5, "test_abc"); -// setFetchRawDataFp(phandle, getSingleColDummyBlock); -// sortAddSource(phandle, &numOfRows); -// -// int32_t code = sortOpen(phandle); -// int32_t row = 1; -// -// while(1) { -// STupleHandle* pTupleHandle = sortNextTuple(phandle); -// if (pTupleHandle == NULL) { -// break; -// } -// -// void* v = sortGetValue(pTupleHandle, 0); -// printf("%d: %d\n", row++, *(int32_t*) v); -// -// } -// destroySortHandle(phandle); -//} + SBlockOrderInfo oi = {0}; + oi.order = TSDB_ORDER_ASC; + oi.colIndex = 0; + SArray* orderInfo = taosArrayInit(1, sizeof(SBlockOrderInfo)); + taosArrayPush(orderInfo, &oi); + + SSchema s = {.type = TSDB_DATA_TYPE_INT, .colId = 1, .bytes = 4, }; + + SSortHandle* phandle = createSortHandle(orderInfo, false, SORT_SINGLESOURCE_SORT, 1024, 5, &s, 1, "test_abc"); + setFetchRawDataFp(phandle, getSingleColDummyBlock); + + _info* pInfo = (_info*) calloc(1, sizeof(_info)); + pInfo->startVal = 100000; + pInfo->pageRows = 1000; + pInfo->count = 50; + + SGenericSource* ps = static_cast(calloc(1, sizeof(SGenericSource))); + ps->param = pInfo; + + sortAddSource(phandle, ps); + + int32_t code = sortOpen(phandle); + int32_t row = 1; + + while(1) { + STupleHandle* pTupleHandle = sortNextTuple(phandle); + if (pTupleHandle == NULL) { + break; + } + + void* v = sortGetValue(pTupleHandle, 0); + printf("%d: %d\n", row++, *(int32_t*) v); + + } + destroySortHandle(phandle); +} //TEST(testCase, ordered_merge_sort_Test) { // SArray* pOrderVal = taosArrayInit(4, sizeof(SOrder)); @@ -234,7 +242,7 @@ int32_t docomp(const void* p1, const void* p2, void* param) { // taosArrayPush(orderInfo, &oi); // // SSchema s = {.type = TSDB_DATA_TYPE_INT, .colId = 1, .bytes = 4}; -// SSortHandle* phandle = createSortHandle(orderInfo, false, SORT_MULTIWAY_MERGE, 1024, 5, &s, 1,"test_abc"); +// SSortHandle* phandle = createSortHandle(orderInfo, false, SORT_MULTISOURCE_MERGE, 1024, 5, &s, 1,"test_abc"); // setFetchRawDataFp(phandle, getSingleColDummyBlock); // setComparFn(phandle, docomp); // diff --git a/source/util/src/tpagedbuf.c b/source/util/src/tpagedbuf.c index f014d8e07e..212b3f3067 100644 --- a/source/util/src/tpagedbuf.c +++ b/source/util/src/tpagedbuf.c @@ -562,7 +562,7 @@ bool isAllDataInMemBuf(const SDiskbasedBuf* pBuf) { } void setBufPageDirty(SFilePage* pPage, bool dirty) { - int32_t offset = offsetof(SPageInfo, pData); // todo extract method + int32_t offset = offsetof(SPageInfo, pData); char* p = (char*)pPage - offset; SPageInfo* ppi = ((SPageInfo**) p)[0]; From b95e95dc0ab56534072f165fe7f011db486ca8d5 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 22 Feb 2022 13:12:03 +0800 Subject: [PATCH 11/25] [td-13039] fix bug in sorted merge operator. --- include/common/tep.h | 12 +- include/libs/function/function.h | 4 +- source/common/src/tep.c | 49 +- source/common/test/commonTests.cpp | 2 +- source/libs/executor/inc/executil.h | 8 +- source/libs/executor/inc/executorimpl.h | 33 +- source/libs/executor/src/executil.c | 8 +- source/libs/executor/src/executorimpl.c | 568 +++++++++++------- source/libs/executor/src/tsort.c | 60 +- source/libs/executor/test/executorTests.cpp | 51 +- .../libs/executor/test/executorUtilTests.cpp | 168 +++--- source/libs/function/src/taggfunction.c | 5 +- source/libs/parser/src/queryInfoUtil.c | 2 +- 13 files changed, 572 insertions(+), 398 deletions(-) diff --git a/include/common/tep.h b/include/common/tep.h index a4e28dbb7c..2be86aa095 100644 --- a/include/common/tep.h +++ b/include/common/tep.h @@ -64,16 +64,16 @@ static FORCE_INLINE bool colDataIsNull(const SColumnInfoData* pColumnInfoData, u } } -#define colDataGet(p1_, r_) \ +#define colDataGetData(p1_, r_) \ ((IS_VAR_DATA_TYPE((p1_)->info.type)) ? (p1_)->pData + (p1_)->varmeta.offset[(r_)] \ - : (p1_)->pData + ((r_) * (p1_)->info.bytes)); + : (p1_)->pData + ((r_) * (p1_)->info.bytes)) int32_t colDataAppend(SColumnInfoData* pColumnInfoData, uint32_t currentRow, const char* pData, bool isNull); int32_t colDataMergeCol(SColumnInfoData* pColumnInfoData, uint32_t numOfRow1, const SColumnInfoData* pSource, uint32_t numOfRow2); int32_t blockDataUpdateTsWindow(SSDataBlock* pDataBlock); -int32_t colDataGetSize(const SColumnInfoData* pColumnInfoData, int32_t numOfRows); -void colDataTrim(SColumnInfoData* pColumnInfoData); +int32_t colDataGetLength(const SColumnInfoData* pColumnInfoData, int32_t numOfRows); +void colDataTrim(SColumnInfoData* pColumnInfoData); size_t colDataGetNumOfCols(const SSDataBlock* pBlock); size_t colDataGetNumOfRows(const SSDataBlock* pBlock); @@ -92,13 +92,13 @@ size_t blockDataGetSerialMetaSize(const SSDataBlock* pBlock); SSchema* blockDataExtractSchema(const SSDataBlock* pBlock, int32_t* numOfCols); -size_t blockDataNumOfRowsForSerialize(const SSDataBlock* pBlock, int32_t blockSize); - int32_t blockDataSort(SSDataBlock* pDataBlock, SArray* pOrderInfo, bool nullFirst); int32_t blockDataSort_rv(SSDataBlock* pDataBlock, SArray* pOrderInfo, bool nullFirst); int32_t blockDataEnsureCapacity(SSDataBlock* pDataBlock, uint32_t numOfRows); void blockDataClearup(SSDataBlock* pDataBlock, bool hasVarCol); +SSDataBlock* createOneDataBlock(const SSDataBlock* pDataBlock); +size_t blockDataGetCapacityInRow(const SSDataBlock* pBlock, size_t pageSize); void *blockDataDestroy(SSDataBlock *pBlock); #ifdef __cplusplus diff --git a/include/libs/function/function.h b/include/libs/function/function.h index aef5f7fec4..a5d45c120b 100644 --- a/include/libs/function/function.h +++ b/include/libs/function/function.h @@ -138,8 +138,10 @@ extern SFunctionFpSet fpSet[1]; // sql function runtime context typedef struct SqlFunctionCtx { + int32_t startRow; int32_t size; // number of rows - void * pInput; // input data buffer + SColumnInfoData* pInput; + uint32_t order; // asc|desc int16_t inputType; int16_t inputBytes; diff --git a/source/common/src/tep.c b/source/common/src/tep.c index b7f7043d26..89d8127a63 100644 --- a/source/common/src/tep.c +++ b/source/common/src/tep.c @@ -63,7 +63,7 @@ SEpSet getEpSet_s(SCorEpSet *pEpSet) { #define BitmapLen(_n) (((_n) + ((1<> NBIT) -int32_t colDataGetSize(const SColumnInfoData* pColumnInfoData, int32_t numOfRows) { +int32_t colDataGetLength(const SColumnInfoData* pColumnInfoData, int32_t numOfRows) { ASSERT(pColumnInfoData != NULL); if (IS_VAR_DATA_TYPE(pColumnInfoData->info.type)) { return pColumnInfoData->varmeta.length; @@ -249,8 +249,8 @@ int32_t blockDataUpdateTsWindow(SSDataBlock* pDataBlock) { } ASSERT(pColInfoData->nullbitmap == NULL); - pDataBlock->info.window.skey = *(TSKEY*) colDataGet(pColInfoData, 0); - pDataBlock->info.window.ekey = *(TSKEY*) colDataGet(pColInfoData, (pDataBlock->info.rows - 1)); + pDataBlock->info.window.skey = *(TSKEY*) colDataGetData(pColInfoData, 0); + pDataBlock->info.window.ekey = *(TSKEY*) colDataGetData(pColInfoData, (pDataBlock->info.rows - 1)); return 0; } @@ -262,8 +262,8 @@ int32_t blockDataMerge(SSDataBlock* pDest, const SSDataBlock* pSrc) { SColumnInfoData* pCol2 = taosArrayGet(pDest->pDataBlock, i); SColumnInfoData* pCol1 = taosArrayGet(pSrc->pDataBlock, i); - uint32_t oldLen = colDataGetSize(pCol2, pDest->info.rows); - uint32_t newLen = colDataGetSize(pCol1, pSrc->info.rows); + uint32_t oldLen = colDataGetLength(pCol2, pDest->info.rows); + uint32_t newLen = colDataGetLength(pCol1, pSrc->info.rows); int32_t newSize = oldLen + newLen; char* tmp = realloc(pCol2->pData, newSize); @@ -287,7 +287,7 @@ size_t blockDataGetSize(const SSDataBlock* pBlock) { for(int32_t i = 0; i < numOfCols; ++i) { SColumnInfoData* pColInfoData = taosArrayGet(pBlock->pDataBlock, i); - total += colDataGetSize(pColInfoData, pBlock->info.rows); + total += colDataGetLength(pColInfoData, pBlock->info.rows); if (IS_VAR_DATA_TYPE(pColInfoData->info.type)) { total += sizeof(int32_t) * pBlock->info.rows; @@ -336,7 +336,7 @@ int32_t blockDataSplitRows(SSDataBlock* pBlock, bool hasVarCol, int32_t startInd if (isNull) { // do nothing } else { - char* p = colDataGet(pColInfoData, j); + char* p = colDataGetData(pColInfoData, j); size += varDataTLen(p); } @@ -401,7 +401,7 @@ SSDataBlock* blockDataExtractBlock(SSDataBlock* pBlock, int32_t startIndex, int3 for (int32_t j = startIndex; j < (startIndex + rowCount); ++j) { bool isNull = colDataIsNull(pColData, pBlock->info.rows, j, pBlock->pBlockAgg); - char* p = colDataGet(pColData, j); + char* p = colDataGetData(pColData, j); colDataAppend(pDstCol, j - startIndex, p, isNull); } @@ -443,7 +443,7 @@ int32_t blockDataToBuf(char* buf, const SSDataBlock* pBlock) { pStart += BitmapLen(pBlock->info.rows); } - uint32_t dataSize = colDataGetSize(pCol, numOfRows); + uint32_t dataSize = colDataGetLength(pCol, numOfRows); *(int32_t*) pStart = dataSize; pStart += sizeof(int32_t); @@ -592,8 +592,8 @@ int32_t dataBlockCompar(const void* p1, const void* p2, const void* param) { } } - void* left1 = colDataGet(pColInfoData, left); - void* right1 = colDataGet(pColInfoData, right); + void* left1 = colDataGetData(pColInfoData, left); + void* right1 = colDataGetData(pColInfoData, right); switch(pColInfoData->info.type) { case TSDB_DATA_TYPE_INT: { @@ -632,7 +632,7 @@ static int32_t doAssignOneTuple(SColumnInfoData* pDstCols, int32_t numOfRows, co return code; } } else { - char* p = colDataGet(pSrc, tupleIndex); + char* p = colDataGetData(pSrc, tupleIndex); code = colDataAppend(pDst, numOfRows, p, false); if (code != TSDB_CODE_SUCCESS) { return code; @@ -971,8 +971,8 @@ int32_t dataBlockCompar_rv(const void* p1, const void* p2, const void* param) { // } // } -// void* left1 = colDataGet(pColInfoData, left); -// void* right1 = colDataGet(pColInfoData, right); +// void* left1 = colDataGetData(pColInfoData, left); +// void* right1 = colDataGetData(pColInfoData, right); // switch(pColInfoData->info.type) { // case TSDB_DATA_TYPE_INT: { @@ -1113,4 +1113,25 @@ void* blockDataDestroy(SSDataBlock* pBlock) { tfree(pBlock->pBlockAgg); tfree(pBlock); return NULL; +} + +SSDataBlock* createOneDataBlock(const SSDataBlock* pDataBlock) { + int32_t numOfCols = pDataBlock->info.numOfCols; + + SSDataBlock* pBlock = calloc(1, sizeof(SSDataBlock)); + pBlock->pDataBlock = taosArrayInit(numOfCols, sizeof(SColumnInfoData)); + pBlock->info.numOfCols = numOfCols; + + for(int32_t i = 0; i < numOfCols; ++i) { + SColumnInfoData colInfo = {0}; + SColumnInfoData* p = taosArrayGet(pDataBlock->pDataBlock, i); + colInfo.info = p->info; + taosArrayPush(pBlock->pDataBlock, &colInfo); + } + + return pBlock; +} + +size_t blockDataGetCapacityInRow(const SSDataBlock* pBlock, size_t pageSize) { + return pageSize / (blockDataGetSerialRowSize(pBlock) + blockDataGetSerialMetaSize(pBlock)); } \ No newline at end of file diff --git a/source/common/test/commonTests.cpp b/source/common/test/commonTests.cpp index e9e8d086b3..9b05b5a780 100644 --- a/source/common/test/commonTests.cpp +++ b/source/common/test/commonTests.cpp @@ -162,7 +162,7 @@ TEST(testCase, Datablock_test) { ASSERT_EQ(colDataGetNumOfCols(b), 2); ASSERT_EQ(colDataGetNumOfRows(b), 40); - char* pData = colDataGet(p1, 3); + char* pData = colDataGetData(p1, 3); printf("the second row of binary:%s, length:%d\n", (char*)varDataVal(pData), varDataLen(pData)); SArray* pOrderInfo = taosArrayInit(3, sizeof(SBlockOrderInfo)); diff --git a/source/libs/executor/inc/executil.h b/source/libs/executor/inc/executil.h index 10d884cb3f..e45e02cdd0 100644 --- a/source/libs/executor/inc/executil.h +++ b/source/libs/executor/inc/executil.h @@ -69,8 +69,8 @@ typedef struct SResultRow { typedef struct SResultRowInfo { SResultRow** pResult; // result list - int16_t type:8; // data type for hash key - int32_t size:24; // number of result set +// int16_t type:8; // data type for hash key + int32_t size; // number of result set int32_t capacity; // max capacity int32_t curPos; // current active result row index of pResult list } SResultRowInfo; @@ -95,7 +95,7 @@ struct SUdfInfo; int32_t getOutputInterResultBufSize(struct STaskAttr* pQueryAttr); size_t getResultRowSize(SArray* pExprInfo); -int32_t initResultRowInfo(SResultRowInfo* pResultRowInfo, int32_t size, int16_t type); +int32_t initResultRowInfo(SResultRowInfo* pResultRowInfo, int32_t size); void cleanupResultRowInfo(SResultRowInfo* pResultRowInfo); void resetResultRowInfo(struct STaskRuntimeEnv* pRuntimeEnv, SResultRowInfo* pResultRowInfo); @@ -105,7 +105,7 @@ void closeAllResultRows(SResultRowInfo* pResultRowInfo); int32_t initResultRow(SResultRow *pResultRow); void closeResultRow(SResultRowInfo* pResultRowInfo, int32_t slot); bool isResultRowClosed(SResultRowInfo *pResultRowInfo, int32_t slot); -void clearResultRow(struct STaskRuntimeEnv* pRuntimeEnv, SResultRow* pResultRow, int16_t type); +void clearResultRow(struct STaskRuntimeEnv* pRuntimeEnv, SResultRow* pResultRow); struct SResultRowEntryInfo* getResultCell(const SResultRow* pRow, int32_t index, int32_t* offset); diff --git a/source/libs/executor/inc/executorimpl.h b/source/libs/executor/inc/executorimpl.h index 362f40696c..1289de004d 100644 --- a/source/libs/executor/inc/executorimpl.h +++ b/source/libs/executor/inc/executorimpl.h @@ -445,16 +445,20 @@ typedef struct SOptrBasicInfo { int32_t capacity; } SOptrBasicInfo; -typedef struct SOptrBasicInfo STableIntervalOperatorInfo; - -typedef struct SAggOperatorInfo { - SOptrBasicInfo binfo; - SDiskbasedBuf *pResultBuf; // query result buffer based on blocked-wised disk file +typedef struct SAggSupporter { SHashObj* pResultRowHashTable; // quick locate the window object for each result SHashObj* pResultRowListSet; // used to check if current ResultRowInfo has ResultRow object or not SArray* pResultRowArrayList; // The array list that contains the Result rows char* keyBuf; // window key buffer SResultRowPool *pool; // The window result objects pool, all the resultRow Objects are allocated and managed by this object. +} SAggSupporter; + +typedef struct SOptrBasicInfo STableIntervalOperatorInfo; + +typedef struct SAggOperatorInfo { + SOptrBasicInfo binfo; + SDiskbasedBuf *pResultBuf; // query result buffer based on blocked-wised disk file + SAggSupporter aggSup; STableQueryInfo *current; uint32_t groupId; SGroupResInfo groupResInfo; @@ -552,8 +556,6 @@ typedef struct SDistinctOperatorInfo { typedef struct SSortedMergeOperatorInfo { SOptrBasicInfo binfo; - -// SSDataBlock *pDataBlock; bool hasVarCol; SArray *orderInfo; // SArray @@ -564,12 +566,16 @@ typedef struct SSortedMergeOperatorInfo { int32_t bufPageSize; uint32_t sortBufSize; // max buffer size for in-memory sort - int32_t numOfRowsInRes; - char** prevRow; int32_t resultRowFactor; - bool multiGroupResults; - bool hasGroupColData; + bool hasGroupVal; + + SDiskbasedBuf *pTupleStore; // keep the final results + int32_t numOfResPerPage; + + char** groupVal; + SArray *groupInfo; + SAggSupporter aggSup; } SSortedMergeOperatorInfo; typedef struct SOrderOperatorInfo { @@ -634,7 +640,7 @@ SOperatorInfo* createFilterOperatorInfo(STaskRuntimeEnv* pRuntimeEnv, SOperatorI SOperatorInfo* createJoinOperatorInfo(SOperatorInfo** pdownstream, int32_t numOfDownstream, SSchema* pSchema, int32_t numOfOutput); SOperatorInfo* createOrderOperatorInfo(SOperatorInfo* downstream, SArray* pExprInfo, SArray* pOrderVal, SExecTaskInfo* pTaskInfo); -SOperatorInfo* createSortedMergeOperatorInfo(SOperatorInfo** downstream, int32_t numOfDownstream, SArray* pExprInfo, SArray* pOrderVal, SExecTaskInfo* pTaskInfo); +SOperatorInfo* createSortedMergeOperatorInfo(SOperatorInfo** downstream, int32_t numOfDownstream, SArray* pExprInfo, SArray* pOrderVal, SArray* pGroupInfo, SExecTaskInfo* pTaskInfo); // SSDataBlock* doGlobalAggregate(void* param, bool* newgroup); // SSDataBlock* doMultiwayMergeSort(void* param, bool* newgroup); @@ -682,9 +688,6 @@ int32_t checkForQueryBuf(size_t numOfTables); bool checkNeedToCompressQueryCol(SQInfo* pQInfo); void setQueryStatus(STaskRuntimeEnv* pRuntimeEnv, int8_t status); -bool onlyQueryTags(STaskAttr* pQueryAttr); -// void destroyUdfInfo(struct SUdfInfo* pUdfInfo); - int32_t doDumpQueryResult(SQInfo* pQInfo, char* data, int8_t compressed, int32_t* compLen); size_t getResultSize(SQInfo* pQInfo, int64_t* numOfRows); diff --git a/source/libs/executor/src/executil.c b/source/libs/executor/src/executil.c index 52ab8493f1..e2675115e0 100644 --- a/source/libs/executor/src/executil.c +++ b/source/libs/executor/src/executil.c @@ -53,8 +53,8 @@ int32_t getOutputInterResultBufSize(STaskAttr* pQueryAttr) { return size; } -int32_t initResultRowInfo(SResultRowInfo *pResultRowInfo, int32_t size, int16_t type) { - pResultRowInfo->type = type; +int32_t initResultRowInfo(SResultRowInfo *pResultRowInfo, int32_t size) { +// pResultRowInfo->type = type; pResultRowInfo->size = 0; pResultRowInfo->curPos = -1; pResultRowInfo->capacity = size; @@ -93,7 +93,7 @@ void resetResultRowInfo(STaskRuntimeEnv *pRuntimeEnv, SResultRowInfo *pResultRow for (int32_t i = 0; i < pResultRowInfo->size; ++i) { SResultRow *pWindowRes = pResultRowInfo->pResult[i]; - clearResultRow(pRuntimeEnv, pWindowRes, pResultRowInfo->type); + clearResultRow(pRuntimeEnv, pWindowRes); int32_t groupIndex = 0; int64_t uid = 0; @@ -136,7 +136,7 @@ void closeResultRow(SResultRowInfo *pResultRowInfo, int32_t slot) { getResultRow(pResultRowInfo, slot)->closed = true; } -void clearResultRow(STaskRuntimeEnv *pRuntimeEnv, SResultRow *pResultRow, int16_t type) { +void clearResultRow(STaskRuntimeEnv *pRuntimeEnv, SResultRow *pResultRow) { if (pResultRow == NULL) { return; } diff --git a/source/libs/executor/src/executorimpl.c b/source/libs/executor/src/executorimpl.c index 5286bd7ba1..efa4afb422 100644 --- a/source/libs/executor/src/executorimpl.c +++ b/source/libs/executor/src/executorimpl.c @@ -12,14 +12,14 @@ * You should have received a copy of the GNU Affero General Public License * along with this program. If not, see . */ -#include -#include -#include "exception.h" #include "os.h" + +#include "tep.h" +#include "tsort.h" +#include "exception.h" #include "parser.h" #include "tglobal.h" #include "tmsg.h" -#include "tq.h" #include "ttime.h" #include "executorimpl.h" @@ -381,12 +381,13 @@ static bool hasNull(SColumn* pColumn, SColumnDataAgg *pStatis) { } static void prepareResultListBuffer(SResultRowInfo* pResultRowInfo, jmp_buf env) { + int64_t newCapacity = 0; + // more than the capacity, reallocate the resources if (pResultRowInfo->size < pResultRowInfo->capacity) { return; } - int64_t newCapacity = 0; if (pResultRowInfo->capacity > 10000) { newCapacity = (int64_t)(pResultRowInfo->capacity * 1.25); } else { @@ -519,12 +520,12 @@ static SResultRow* doSetResultOutBufByKey(STaskRuntimeEnv* pRuntimeEnv, SResultR } static SResultRow* doSetResultOutBufByKey_rv(SResultRowInfo* pResultRowInfo, int64_t tid, char* pData, int16_t bytes, - bool masterscan, uint64_t tableGroupId, SExecTaskInfo* pTaskInfo, bool isIntervalQuery, SAggOperatorInfo* pAggInfo) { + bool masterscan, uint64_t tableGroupId, SExecTaskInfo* pTaskInfo, bool isIntervalQuery, SAggSupporter* pSup) { bool existed = false; - SET_RES_WINDOW_KEY(pAggInfo->keyBuf, pData, bytes, tableGroupId); + SET_RES_WINDOW_KEY(pSup->keyBuf, pData, bytes, tableGroupId); SResultRow **p1 = - (SResultRow **)taosHashGet(pAggInfo->pResultRowHashTable, pAggInfo->keyBuf, GET_RES_WINDOW_KEY_LEN(bytes)); + (SResultRow **)taosHashGet(pSup->pResultRowHashTable, pSup->keyBuf, GET_RES_WINDOW_KEY_LEN(bytes)); // in case of repeat scan/reverse scan, no new time window added. if (isIntervalQuery) { @@ -540,8 +541,8 @@ static SResultRow* doSetResultOutBufByKey_rv(SResultRowInfo* pResultRowInfo, int existed = (pResultRowInfo->pResult[0] == (*p1)); pResultRowInfo->curPos = 0; } else { // check if current pResultRowInfo contains the existed pResultRow - SET_RES_EXT_WINDOW_KEY(pAggInfo->keyBuf, pData, bytes, tid, pResultRowInfo); - int64_t* index = taosHashGet(pAggInfo->pResultRowListSet, pAggInfo->keyBuf, GET_RES_EXT_WINDOW_KEY_LEN(bytes)); + SET_RES_EXT_WINDOW_KEY(pSup->keyBuf, pData, bytes, tid, pResultRowInfo); + int64_t* index = taosHashGet(pSup->pResultRowListSet, pSup->keyBuf, GET_RES_EXT_WINDOW_KEY_LEN(bytes)); if (index != NULL) { pResultRowInfo->curPos = (int32_t) *index; existed = true; @@ -562,16 +563,16 @@ static SResultRow* doSetResultOutBufByKey_rv(SResultRowInfo* pResultRowInfo, int SResultRow *pResult = NULL; if (p1 == NULL) { - pResult = getNewResultRow(pAggInfo->pool); + pResult = getNewResultRow(pSup->pool); int32_t ret = initResultRow(pResult); if (ret != TSDB_CODE_SUCCESS) { longjmp(pTaskInfo->env, TSDB_CODE_QRY_OUT_OF_MEMORY); } // add a new result set for a new group - taosHashPut(pAggInfo->pResultRowHashTable, pAggInfo->keyBuf, GET_RES_WINDOW_KEY_LEN(bytes), &pResult, POINTER_BYTES); + taosHashPut(pSup->pResultRowHashTable, pSup->keyBuf, GET_RES_WINDOW_KEY_LEN(bytes), &pResult, POINTER_BYTES); SResultRowCell cell = {.groupId = tableGroupId, .pRow = pResult}; - taosArrayPush(pAggInfo->pResultRowArrayList, &cell); + taosArrayPush(pSup->pResultRowArrayList, &cell); } else { pResult = *p1; } @@ -580,8 +581,8 @@ static SResultRow* doSetResultOutBufByKey_rv(SResultRowInfo* pResultRowInfo, int pResultRowInfo->pResult[pResultRowInfo->size++] = pResult; int64_t index = pResultRowInfo->curPos; - SET_RES_EXT_WINDOW_KEY(pAggInfo->keyBuf, pData, bytes, tid, pResultRowInfo); - taosHashPut(pAggInfo->pResultRowListSet, pAggInfo->keyBuf, GET_RES_EXT_WINDOW_KEY_LEN(bytes), &index, POINTER_BYTES); + SET_RES_EXT_WINDOW_KEY(pSup->keyBuf, pData, bytes, tid, pResultRowInfo); + taosHashPut(pSup->pResultRowListSet, pSup->keyBuf, GET_RES_EXT_WINDOW_KEY_LEN(bytes), &index, POINTER_BYTES); } // too many time window in query @@ -933,11 +934,11 @@ static void doApplyFunctions(STaskRuntimeEnv* pRuntimeEnv, SqlFunctionCtx* pCtx, pCtx[k].startTs = pWin->skey; // keep it temporarialy - char* start = pCtx[k].pInput; + char* start = NULL;//pCtx[k].pInput; int32_t pos = (QUERY_IS_ASC_QUERY(pQueryAttr)) ? offset : offset - (forwardStep - 1); if (pCtx[k].pInput != NULL) { - pCtx[k].pInput = (char *)pCtx[k].pInput + pos * pCtx[k].inputBytes; +// pCtx[k].pInput = (char *)pCtx[k].pInput + pos * pCtx[k].inputBytes; } if (tsCol != NULL) { @@ -956,7 +957,7 @@ static void doApplyFunctions(STaskRuntimeEnv* pRuntimeEnv, SqlFunctionCtx* pCtx, // restore it pCtx[k].isAggSet = hasAggregates; - pCtx[k].pInput = start; +// pCtx[k].pInput = start; } } @@ -1152,7 +1153,7 @@ static void doSetInputDataBlock(SOperatorInfo* pOperator, SqlFunctionCtx* pCtx, SColumnInfoData* p = taosArrayGet(pBlock->pDataBlock, pCtx[i].columnIndex); // in case of the block distribution query, the inputBytes is not a constant value. - pCtx[i].pInput = p->pData; + pCtx[i].pInput = p; assert(p->info.colId == pCol->info.colId); if (pCtx[i].functionId < 0) { @@ -1164,14 +1165,14 @@ static void doSetInputDataBlock(SOperatorInfo* pOperator, SqlFunctionCtx* pCtx, // uint32_t status = aAggs[pCtx[i].functionId].status; // if ((status & (FUNCSTATE_SELECTIVITY | FUNCSTATE_NEED_TS)) != 0) { - SColumnInfoData* tsInfo = taosArrayGet(pBlock->pDataBlock, 0); +// SColumnInfoData* tsInfo = taosArrayGet(pBlock->pDataBlock, 0); // In case of the top/bottom query again the nest query result, which has no timestamp column // don't set the ptsList attribute. - if (tsInfo->info.type == TSDB_DATA_TYPE_TIMESTAMP) { - pCtx[i].ptsList = (int64_t*) tsInfo->pData; - } else { - pCtx[i].ptsList = NULL; - } +// if (tsInfo->info.type == TSDB_DATA_TYPE_TIMESTAMP) { +// pCtx[i].ptsList = (int64_t*) tsInfo->pData; +// } else { +// pCtx[i].ptsList = NULL; +// } // } // } else if (TSDB_COL_IS_UD_COL(pCol->flag) && (pOperator->pRuntimeEnv->scanFlag == MERGE_STAGE)) { // SColIndex* pColIndex = &pOperator->pExpr[i].base.colInfo; @@ -2341,29 +2342,6 @@ static bool isCachedLastQuery(STaskAttr *pQueryAttr) { return true; } - - -/** - * The following 4 kinds of query are treated as the tags query - * tagprj, tid_tag query, count(tbname), 'abc' (user defined constant value column) query - */ -bool onlyQueryTags(STaskAttr* pQueryAttr) { - for(int32_t i = 0; i < pQueryAttr->numOfOutput; ++i) { - SExprInfo* pExprInfo = &pQueryAttr->pExpr1[i]; - - int32_t functionId = getExprFunctionId(pExprInfo); - - if (functionId != FUNCTION_TAGPRJ && - functionId != FUNCTION_TID_TAG && - (!(functionId == FUNCTION_COUNT && pExprInfo->base.pColumns->info.colId == TSDB_TBNAME_COLUMN_INDEX)) && - (!(functionId == FUNCTION_PRJ && TSDB_COL_IS_UD_COL(pExprInfo->base.pColumns->flag)))) { - return false; - } - } - - return true; -} - ///////////////////////////////////////////////////////////////////////////////////////////// void getAlignQueryTimeWindow(STaskAttr *pQueryAttr, int64_t key, int64_t keyFirst, int64_t keyLast, STimeWindow *win) { @@ -2864,8 +2842,6 @@ void filterColRowsInDataBlock(STaskRuntimeEnv* pRuntimeEnv, SSDataBlock* pBlock, tfree(p); } - - static SColumnInfo* doGetTagColumnInfoById(SColumnInfo* pTagColList, int32_t numOfTags, int16_t colId); static void doSetTagValueInParam(void* pTable, int32_t tagColId, SVariant *tag, int16_t type, int16_t bytes); @@ -3380,10 +3356,8 @@ void setDefaultOutputBuf(STaskRuntimeEnv *pRuntimeEnv, SOptrBasicInfo *pInfo, in initCtxOutputBuffer(pCtx, pDataBlock->info.numOfCols); } - -void setDefaultOutputBuf_rv(SAggOperatorInfo* pAggInfo, int32_t stage, SExecTaskInfo* pTaskInfo) { - SOptrBasicInfo *pInfo = &pAggInfo->binfo; - +// TODO refactor: some function move away +void setDefaultOutputBuf_rv(SOptrBasicInfo* pInfo, SAggSupporter* pSup, int32_t stage, SExecTaskInfo* pTaskInfo) { SqlFunctionCtx* pCtx = pInfo->pCtx; SSDataBlock* pDataBlock = pInfo->pRes; int32_t* rowCellInfoOffset = pInfo->rowCellInfoOffset; @@ -3391,9 +3365,7 @@ void setDefaultOutputBuf_rv(SAggOperatorInfo* pAggInfo, int32_t stage, SExecTask int64_t tid = 0; int64_t groupId = 0; - - pAggInfo->keyBuf = realloc(pAggInfo->keyBuf, sizeof(tid) + sizeof(int64_t) + POINTER_BYTES); - SResultRow* pRow = doSetResultOutBufByKey_rv(pResultRowInfo, tid, (char *)&tid, sizeof(tid), true, groupId, pTaskInfo, false, pAggInfo); + SResultRow* pRow = doSetResultOutBufByKey_rv(pResultRowInfo, tid, (char *)&tid, sizeof(tid), true, groupId, pTaskInfo, false, pSup); for (int32_t i = 0; i < pDataBlock->info.numOfCols; ++i) { SColumnInfoData* pData = taosArrayGet(pDataBlock->pDataBlock, i); @@ -3606,7 +3578,7 @@ STableQueryInfo *createTableQueryInfo(void* buf, bool groupbyColumn, STimeWindow // set more initial size of interval/groupby query // if (/*QUERY_IS_INTERVAL_QUERY(pQueryAttr) || */groupbyColumn) { int32_t initialSize = 128; - int32_t code = initResultRowInfo(&pTableQueryInfo->resInfo, initialSize, TSDB_DATA_TYPE_INT); + int32_t code = initResultRowInfo(&pTableQueryInfo->resInfo, initialSize); if (code != TSDB_CODE_SUCCESS) { return NULL; } @@ -3624,7 +3596,7 @@ STableQueryInfo* createTmpTableQueryInfo(STimeWindow win) { // set more initial size of interval/groupby query int32_t initialSize = 16; - int32_t code = initResultRowInfo(&pTableQueryInfo->resInfo, initialSize, TSDB_DATA_TYPE_INT); + int32_t code = initResultRowInfo(&pTableQueryInfo->resInfo, initialSize); if (code != TSDB_CODE_SUCCESS) { tfree(pTableQueryInfo); return NULL; @@ -3717,7 +3689,7 @@ void doSetTableGroupOutputBuf(SAggOperatorInfo* pAggInfo, int32_t numOfOutput, i int32_t* rowCellInfoOffset = pAggInfo->binfo.rowCellInfoOffset; SResultRow* pResultRow = - doSetResultOutBufByKey_rv(pResultRowInfo, tid, (char*)&tableGroupId, sizeof(tableGroupId), true, uid, pTaskInfo, false, pAggInfo); + doSetResultOutBufByKey_rv(pResultRowInfo, tid, (char*)&tableGroupId, sizeof(tableGroupId), true, uid, pTaskInfo, false, &pAggInfo->aggSup); assert (pResultRow != NULL); /* @@ -4521,13 +4493,19 @@ void queryCostStatis(SExecTaskInfo *pTaskInfo) { // return true; //} -void appendDownstream(SOperatorInfo* p, SOperatorInfo* pDownstream) { +int32_t appendDownstream(SOperatorInfo* p, SOperatorInfo** pDownstream, int32_t num) { if (p->pDownstream == NULL) { assert(p->numOfDownstream == 0); } - p->pDownstream = realloc(p->pDownstream, POINTER_BYTES * (p->numOfDownstream + 1)); - p->pDownstream[p->numOfDownstream++] = pDownstream; + p->pDownstream = calloc(1, num * POINTER_BYTES); + if (p->pDownstream == NULL) { + return TSDB_CODE_OUT_OF_MEMORY; + } + + memcpy(p->pDownstream, pDownstream, num * POINTER_BYTES); + p->numOfDownstream = num; + return TSDB_CODE_SUCCESS; } static void doDestroyTableQueryInfo(STableGroupInfo* pTableqinfoGroupInfo); @@ -5599,11 +5577,20 @@ SArray* getResultGroupCheckColumns(STaskAttr* pQuery) { return pOrderColumns; } +static int32_t initAggSup(SAggSupporter* pAggSup, SArray* pExprInfo); +static void clearupAggSup(SAggSupporter* pAggSup); + static void destroySortedMergeOperatorInfo(void* param, int32_t numOfOutput) { SSortedMergeOperatorInfo* pInfo = (SSortedMergeOperatorInfo*) param; taosArrayDestroy(pInfo->orderInfo); - destroySortHandle(pInfo->pSortHandle); + taosArrayDestroy(pInfo->groupInfo); + + if (pInfo->pSortHandle != NULL) { + destroySortHandle(pInfo->pSortHandle); + } blockDataDestroy(pInfo->binfo.pRes); + + clearupAggSup(&pInfo->aggSup); } static void destroySlimitOperatorInfo(void* param, int32_t numOfOutput) { @@ -5613,11 +5600,12 @@ static void destroySlimitOperatorInfo(void* param, int32_t numOfOutput) { tfree(pInfo->prevRow); } -static SExprInfo* exprArrayDup(SArray* pExprInfo) { - size_t numOfOutput = taosArrayGetSize(pExprInfo); +static SExprInfo* exprArrayDup(SArray* pExprList) { + size_t numOfOutput = taosArrayGetSize(pExprList); + SExprInfo* p = calloc(numOfOutput, sizeof(SExprInfo)); - for (int32_t i = 0; i < taosArrayGetSize(pExprInfo); ++i) { - SExprInfo* pExpr = taosArrayGetP(pExprInfo, i); + for (int32_t i = 0; i < numOfOutput; ++i) { + SExprInfo* pExpr = taosArrayGetP(pExprList, i); assignExprInfo(&p[i], pExpr); } @@ -5666,6 +5654,173 @@ SSDataBlock* loadNextDataBlock(void* param) { return pOperator->exec(pOperator, &newgroup); } +static bool needToMerge(SSDataBlock* pBlock, SArray* groupInfo, char **buf, int32_t rowIndex) { + size_t size = taosArrayGetSize(groupInfo); + if (size == 0) { + return true; + } + + for (int32_t i = 0; i < size; ++i) { + int32_t* index = taosArrayGet(groupInfo, i); + + SColumnInfoData* pColInfo = taosArrayGet(pBlock->pDataBlock, *index); + bool isNull = colDataIsNull(pColInfo, rowIndex, pBlock->info.rows, NULL); + + if ((isNull && buf[i] != NULL) || (!isNull && buf[i] == NULL)) { + return false; + } + + char* pCell = colDataGetData(pColInfo, rowIndex); + if (IS_VAR_DATA_TYPE(pColInfo->info.type)) { + if (varDataLen(pCell) != varDataLen(buf[i])) { + return false; + } else { + if (memcmp(varDataVal(pCell), varDataVal(buf[i]), varDataLen(pCell)) != 0) { + return false; + } + } + } else { + if (memcmp(pCell, buf[i], pColInfo->info.bytes) != 0) { + return false; + } + } + } + + return 0; +} + +static void doMergeResultImpl(SSortedMergeOperatorInfo* pInfo, SqlFunctionCtx *pCtx, int32_t numOfExpr, int32_t rowIndex) { + for (int32_t j = 0; j < numOfExpr; ++j) { // TODO set row index + pCtx[j].startRow = rowIndex; + } + + for (int32_t j = 0; j < numOfExpr; ++j) { + int32_t functionId = pCtx[j].functionId; + pCtx[j].fpSet->addInput(&pCtx[j]); + +// if (functionId < 0) { +// SUdfInfo* pUdfInfo = taosArrayGet(pInfo->udfInfo, -1 * functionId - 1); +// doInvokeUdf(pUdfInfo, &pCtx[j], 0, TSDB_UDF_FUNC_MERGE); +// } else { +// assert(!TSDB_FUNC_IS_SCALAR(functionId)); +// aAggs[functionId].mergeFunc(&pCtx[j]); +// } + } +} + +static void doFinalizeResultImpl(SqlFunctionCtx *pCtx, int32_t numOfExpr) { + for(int32_t j = 0; j < numOfExpr; ++j) { + int32_t functionId = pCtx[j].functionId; + // if (functionId == FUNC_TAG_DUMMY || functionId == FUNC_TS_DUMMY) { + // continue; + // } + + // if (functionId < 0) { + // SUdfInfo* pUdfInfo = taosArrayGet(pInfo->udfInfo, -1 * functionId - 1); + // doInvokeUdf(pUdfInfo, &pCtx[j], 0, TSDB_UDF_FUNC_FINALIZE); + // } else { + pCtx[j].fpSet->addInput(&pCtx[j]); + } +} + +static bool saveCurrentTuple(char** rowColData, SArray* pColumnList, SSDataBlock* pBlock, int32_t rowIndex) { + int32_t size = (int32_t) taosArrayGetSize(pColumnList); + + for(int32_t i = 0; i < size; ++i) { + int32_t* index = taosArrayGet(pColumnList, i); + SColumnInfoData* pColInfo = taosArrayGet(pBlock->pDataBlock, *index); + + char* data = colDataGetData(pColInfo, rowIndex); + memcpy(rowColData[i], data, colDataGetLength(pColInfo, rowIndex)); + } + + return true; +} + +static void doMergeImpl(SOperatorInfo* pOperator, int32_t numOfExpr, SSDataBlock* pBlock) { + SSortedMergeOperatorInfo* pInfo = pOperator->info; + + SqlFunctionCtx* pCtx = pInfo->binfo.pCtx; + for(int32_t i = 0; i < pBlock->info.numOfCols; ++i) { + pCtx[i].size = 1; + } + + for(int32_t i = 0; i < pBlock->info.rows; ++i) { + if (!pInfo->hasGroupVal) { + ASSERT(i == 0); + doMergeResultImpl(pInfo, pCtx, numOfExpr, i); + pInfo->hasGroupVal = saveCurrentTuple(pInfo->groupVal, pInfo->groupInfo, pBlock, i); + } else { + if (needToMerge(pBlock, pInfo->groupInfo, pInfo->groupVal, i)) { + doMergeResultImpl(pInfo, pCtx, numOfExpr, i); + } else { + doFinalizeResultImpl(pCtx, numOfExpr); + int32_t numOfRows = getNumOfResult(pInfo->binfo.pCtx, pOperator->numOfOutput); + // setTagValueForMultipleRows(pCtx, pOperator->numOfOutput, numOfRows); + + // TODO check for available buffer; + + // next group info data + pInfo->binfo.pRes->info.rows += numOfRows; + for (int32_t j = 0; j < numOfExpr; ++j) { + if (pCtx[j].functionId < 0) { + continue; + } + + pCtx[j].fpSet->addInput(&pCtx[j]); + } + + doMergeResultImpl(pInfo, pCtx, numOfExpr, i); + pInfo->hasGroupVal = saveCurrentTuple(pInfo->groupVal, pInfo->groupInfo, pBlock, i); + } + } + } +} + +static SSDataBlock* doMerge(SOperatorInfo* pOperator) { + SSortedMergeOperatorInfo* pInfo = pOperator->info; + SSortHandle* pHandle = pInfo->pSortHandle; + + SSDataBlock* pDataBlock = createOneDataBlock(pInfo->binfo.pRes); + blockDataEnsureCapacity(pDataBlock, pInfo->binfo.capacity); + + while(1) { + + blockDataClearup(pDataBlock, pInfo->hasVarCol); + while (1) { + STupleHandle* pTupleHandle = sortNextTuple(pHandle); + if (pTupleHandle == NULL) { + break; + } + + // build datablock for merge for one group + appendOneRowToDataBlock(pDataBlock, pTupleHandle); + if (pDataBlock->info.rows >= pInfo->binfo.capacity) { + break; + } + } + + if (pDataBlock->info.rows == 0) { + break; + } + + setInputDataBlock(pOperator, pInfo->binfo.pCtx, pDataBlock, TSDB_ORDER_ASC); + // updateOutputBuf(&pInfo->binfo, &pAggInfo->bufCapacity, pBlock->info.rows * pAggInfo->resultRowFactor, pOperator->pRuntimeEnv, true); + doMergeImpl(pOperator, pOperator->numOfOutput, pDataBlock); + // flush to tuple store, and after all data have been handled, return to upstream node or sink node + } + + doFinalizeResultImpl(pInfo->binfo.pCtx, pOperator->numOfOutput); + int32_t numOfRows = getNumOfResult(pInfo->binfo.pCtx, pOperator->numOfOutput); + // setTagValueForMultipleRows(pCtx, pOperator->numOfOutput, numOfRows); + + // TODO check for available buffer; + + // next group info data + pInfo->binfo.pRes->info.rows += numOfRows; + return (pInfo->binfo.pRes->info.rows > 0)? pInfo->binfo.pRes:NULL; +} + static SSDataBlock* doSortedMerge(void* param, bool* newgroup) { SOperatorInfo* pOperator = (SOperatorInfo*) param; if (pOperator->status == OP_EXEC_DONE) { @@ -5675,7 +5830,7 @@ static SSDataBlock* doSortedMerge(void* param, bool* newgroup) { SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; SSortedMergeOperatorInfo* pInfo = pOperator->info; if (pOperator->status == OP_RES_TO_RETURN) { - return getSortedBlockData(pInfo->pSortHandle, pInfo->binfo.pRes, pInfo->hasVarCol, pInfo->numOfRowsInRes); + return getSortedBlockData(pInfo->pSortHandle, pInfo->binfo.pRes, pInfo->hasVarCol, pInfo->binfo.capacity); } SSchema* p = blockDataExtractSchema(pInfo->binfo.pRes, NULL); @@ -5698,7 +5853,7 @@ static SSDataBlock* doSortedMerge(void* param, bool* newgroup) { } pOperator->status = OP_RES_TO_RETURN; - return getSortedBlockData(pInfo->pSortHandle, pInfo->binfo.pRes, pInfo->hasVarCol, pInfo->numOfRowsInRes); + return doMerge(pOperator); } static SArray* createBlockOrder(SArray* pExprInfo, SArray* pOrderVal) { @@ -5724,29 +5879,88 @@ static SArray* createBlockOrder(SArray* pExprInfo, SArray* pOrderVal) { return pOrderInfo; } -SOperatorInfo* createSortedMergeOperatorInfo(SOperatorInfo** downstream, int32_t numOfDownstream, SArray* pExprInfo, SArray* pOrderVal, SExecTaskInfo* pTaskInfo) { +static int32_t initGroupCol(SArray* pExprInfo, SArray* pGroupInfo, SSortedMergeOperatorInfo* pInfo) { + if (pGroupInfo == NULL || taosArrayGetSize(pGroupInfo) == 0) { + return 0; + } + + int32_t len = 0; + SArray* plist = taosArrayInit(3, sizeof(SColumn)); + pInfo->groupInfo = taosArrayInit(3, sizeof(int32_t)); + + if (plist == NULL || pInfo->groupInfo == NULL) { + return TSDB_CODE_OUT_OF_MEMORY; + } + + size_t numOfGroupCol = taosArrayGetSize(pInfo->groupInfo); + for(int32_t i = 0; i < numOfGroupCol; ++i) { + SColumn* pCol = taosArrayGet(pGroupInfo, i); + for(int32_t j = 0; j < taosArrayGetSize(pExprInfo); ++j) { + SExprInfo* pe = taosArrayGet(pExprInfo, j); + if (pe->base.resSchema.colId == pCol->info.colId) { + taosArrayPush(plist, pCol); + taosArrayPush(pInfo->groupInfo, &j); + len += pCol->info.bytes; + break; + } + } + } + + ASSERT(taosArrayGetSize(pGroupInfo) == taosArrayGetSize(plist)); + + pInfo->groupVal = calloc(1, (POINTER_BYTES * numOfGroupCol + len)); + if (pInfo->groupVal == NULL) { + taosArrayDestroy(plist); + return TSDB_CODE_OUT_OF_MEMORY; + } + + int32_t offset = 0; + char *start = (char*)(pInfo->groupVal + (POINTER_BYTES * numOfGroupCol)); + for(int32_t i = 0; i < numOfGroupCol; ++i) { + pInfo->groupVal[i] = start + offset; + SColumn* pCol = taosArrayGet(plist, i); + offset += pCol->info.bytes; + } + + taosArrayDestroy(plist); + + return TSDB_CODE_SUCCESS; +} + +SOperatorInfo* createSortedMergeOperatorInfo(SOperatorInfo** downstream, int32_t numOfDownstream, SArray* pExprInfo, SArray* pOrderVal, SArray* pGroupInfo, SExecTaskInfo* pTaskInfo) { SSortedMergeOperatorInfo* pInfo = calloc(1, sizeof(SSortedMergeOperatorInfo)); SOperatorInfo* pOperator = calloc(1, sizeof(SOperatorInfo)); if (pInfo == NULL || pOperator == NULL) { - tfree(pInfo); - tfree(pOperator); - terrno = TSDB_CODE_QRY_OUT_OF_MEMORY; - return NULL; + goto _error; } int32_t numOfOutput = taosArrayGetSize(pExprInfo); - pInfo->binfo.capacity = 4096; - pInfo->binfo.pCtx = createSqlFunctionCtx_rv(pExprInfo, &pInfo->binfo.rowCellInfoOffset, &pInfo->binfo.resRowSize); + pInfo->binfo.pCtx = createSqlFunctionCtx_rv(pExprInfo, &pInfo->binfo.rowCellInfoOffset, &pInfo->binfo.resRowSize); + pInfo->binfo.pRes = createOutputBuf_rv(pExprInfo, pInfo->binfo.capacity); + initResultRowInfo(&pInfo->binfo.resultRowInfo, (int32_t)1); -// pInfo->resultRowFactor = -// (int32_t)(getRowNumForMultioutput(pRuntimeEnv->pQueryAttr, pRuntimeEnv->pQueryAttr->topBotQuery, false)); + if (pInfo->binfo.pCtx == NULL || pInfo->binfo.pRes == NULL) { + goto _error; + } + + int32_t code = initAggSup(&pInfo->aggSup, pExprInfo); + if (code != TSDB_CODE_SUCCESS) { + goto _error; + } + + setDefaultOutputBuf_rv(&pInfo->binfo, &pInfo->aggSup, MAIN_SCAN, pTaskInfo); + code = initGroupCol(pExprInfo, pGroupInfo, pInfo); + if (code != TSDB_CODE_SUCCESS) { + goto _error; + } + +// pInfo->resultRowFactor = (int32_t)(getRowNumForMultioutput(pRuntimeEnv->pQueryAttr, +// pRuntimeEnv->pQueryAttr->topBotQuery, false)); pInfo->sortBufSize = 1024 * 16; // 1MB pInfo->bufPageSize = 1024; - pInfo->numOfRowsInRes = 1024; - pInfo->binfo.pRes = createOutputBuf_rv(pExprInfo, pInfo->numOfRowsInRes); pInfo->orderInfo = createBlockOrder(pExprInfo, pOrderVal); - int32_t numOfRows = 1; + pInfo->binfo.capacity = blockDataGetCapacityInRow(pInfo->binfo.pRes, pInfo->bufPageSize); pOperator->name = "SortedMerge"; pOperator->operatorType = OP_SortedMerge; @@ -5754,16 +5968,28 @@ SOperatorInfo* createSortedMergeOperatorInfo(SOperatorInfo** downstream, int32_t pOperator->status = OP_IN_EXECUTING; pOperator->info = pInfo; pOperator->numOfOutput = numOfOutput; + pOperator->pExpr = exprArrayDup(pExprInfo); pOperator->pTaskInfo = pTaskInfo; pOperator->exec = doSortedMerge; pOperator->cleanupFn = destroySortedMergeOperatorInfo; - for(int32_t i = 0; i < numOfDownstream; ++i) { - appendDownstream(pOperator, downstream[i]); + code = appendDownstream(pOperator, downstream, numOfDownstream); + if (code != TSDB_CODE_SUCCESS) { + goto _error; } return pOperator; + + _error: + if (pInfo != NULL) { + destroySortedMergeOperatorInfo(pInfo, numOfOutput); + } + + tfree(pInfo); + tfree(pOperator); + terrno = TSDB_CODE_QRY_OUT_OF_MEMORY; + return NULL; } static SSDataBlock* doSort(void* param, bool* newgroup) { @@ -5844,7 +6070,7 @@ SOperatorInfo *createOrderOperatorInfo(SOperatorInfo* downstream, SArray* pExprI pOperator->exec = doSort; pOperator->cleanupFn = destroyOrderOperatorInfo; - appendDownstream(pOperator, downstream); + int32_t code = appendDownstream(pOperator, &downstream, 1); return pOperator; } @@ -6761,18 +6987,37 @@ static void destroyOperatorInfo(SOperatorInfo* pOperator) { tfree(pOperator); } +static int32_t initAggSup(SAggSupporter* pAggSup, SArray* pExprInfo) { + _hash_fn_t hashFn = taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY); + + pAggSup->keyBuf = calloc(1, sizeof(int64_t) + sizeof(int64_t) + POINTER_BYTES); + pAggSup->pResultRowHashTable = taosHashInit(10, hashFn, true, HASH_NO_LOCK); + pAggSup->pResultRowListSet = taosHashInit(100, hashFn, false, HASH_NO_LOCK); + pAggSup->pool = initResultRowPool(getResultRowSize(pExprInfo)); + pAggSup->pResultRowArrayList = taosArrayInit(10, sizeof(SResultRowCell)); + + if (pAggSup->keyBuf == NULL || pAggSup->pResultRowArrayList == NULL || pAggSup->pResultRowListSet == NULL || + pAggSup->pResultRowHashTable == NULL || pAggSup->pool == NULL) { + return TSDB_CODE_OUT_OF_MEMORY; + } + + return TSDB_CODE_SUCCESS; +} + +static void clearupAggSup(SAggSupporter* pAggSup) { + tfree(pAggSup->keyBuf); + taosHashCleanup(pAggSup->pResultRowHashTable); + taosHashCleanup(pAggSup->pResultRowListSet); + taosArrayDestroy(pAggSup->pResultRowArrayList); + destroyResultRowPool(pAggSup->pool); +} + static int32_t initAggInfo(SAggOperatorInfo* pInfo, SArray* pExprInfo, int32_t numOfRows, const STableGroupInfo* pTableGroupInfo) { pInfo->binfo.pRes = createOutputBuf_rv(pExprInfo, numOfRows); pInfo->binfo.pCtx = createSqlFunctionCtx_rv(pExprInfo, &pInfo->binfo.rowCellInfoOffset, &pInfo->binfo.resRowSize); pInfo->binfo.capacity = 4096; - _hash_fn_t hashFn = taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY); - - pInfo->pResultRowHashTable = taosHashInit(10, hashFn, true, HASH_NO_LOCK); - pInfo->pResultRowListSet = taosHashInit(100, hashFn, false, HASH_NO_LOCK); - pInfo->pool = initResultRowPool(getResultRowSize(pExprInfo)); - pInfo->pResultRowArrayList = taosArrayInit(10, sizeof(SResultRowCell)); - + initAggSup(&pInfo->aggSup, pExprInfo); pInfo->pTableQueryInfo = calloc(pTableGroupInfo->numOfTables, sizeof(STableQueryInfo)); int32_t index = 0; @@ -6801,7 +7046,7 @@ SOperatorInfo* createAggregateOperatorInfo(SOperatorInfo* downstream, SArray* pE //(int32_t)(getRowNumForMultioutput(pQueryAttr, pQueryAttr->topBotQuery, pQueryAttr->stableQuery)); initAggInfo(pInfo, pExprInfo, numOfRows, pTableGroupInfo); - setDefaultOutputBuf_rv(pInfo, MAIN_SCAN, pTaskInfo); + setDefaultOutputBuf_rv(&pInfo->binfo, &pInfo->aggSup, MAIN_SCAN, pTaskInfo); SOperatorInfo* pOperator = calloc(1, sizeof(SOperatorInfo)); pOperator->name = "TableAggregate"; @@ -6815,7 +7060,7 @@ SOperatorInfo* createAggregateOperatorInfo(SOperatorInfo* downstream, SArray* pE pOperator->pTaskInfo = pTaskInfo; pOperator->exec = doAggregate; pOperator->cleanupFn = destroyAggOperatorInfo; - appendDownstream(pOperator, downstream); + int32_t code = appendDownstream(pOperator, &downstream, 1); return pOperator; } @@ -6899,7 +7144,7 @@ SOperatorInfo* createMultiTableAggOperatorInfo(SOperatorInfo* downstream, SArray initAggInfo(pInfo, pExprInfo, numOfRows, pTableGroupInfo); size_t tableGroup = taosArrayGetSize(pTableGroupInfo->pGroupList); - initResultRowInfo(&pInfo->binfo.resultRowInfo, (int32_t)tableGroup, TSDB_DATA_TYPE_INT); + initResultRowInfo(&pInfo->binfo.resultRowInfo, (int32_t)tableGroup); SOperatorInfo* pOperator = calloc(1, sizeof(SOperatorInfo)); pOperator->name = "MultiTableAggregate"; @@ -6912,7 +7157,7 @@ SOperatorInfo* createMultiTableAggOperatorInfo(SOperatorInfo* downstream, SArray pOperator->exec = doMultiTableAggregate; pOperator->cleanupFn = destroyAggOperatorInfo; - appendDownstream(pOperator, downstream); + int32_t code = appendDownstream(pOperator, &downstream, 1); return pOperator; } @@ -6927,7 +7172,7 @@ SOperatorInfo* createProjectOperatorInfo(STaskRuntimeEnv* pRuntimeEnv, SOperator pBInfo->pRes = createOutputBuf(pExpr, numOfOutput, pInfo->bufCapacity); pBInfo->pCtx = createSqlFunctionCtx(pRuntimeEnv, pExpr, numOfOutput, &pBInfo->rowCellInfoOffset); - initResultRowInfo(&pBInfo->resultRowInfo, 8, TSDB_DATA_TYPE_INT); + initResultRowInfo(&pBInfo->resultRowInfo, 8); setDefaultOutputBuf(pRuntimeEnv, pBInfo, pInfo->seed, MAIN_SCAN); SOperatorInfo* pOperator = calloc(1, sizeof(SOperatorInfo)); @@ -6942,7 +7187,7 @@ SOperatorInfo* createProjectOperatorInfo(STaskRuntimeEnv* pRuntimeEnv, SOperator pOperator->exec = doProjectOperation; pOperator->cleanupFn = destroyProjectOperatorInfo; - appendDownstream(pOperator, downstream); + int32_t code = appendDownstream(pOperator, &downstream, 1); return pOperator; } @@ -7000,7 +7245,7 @@ SOperatorInfo* createFilterOperatorInfo(STaskRuntimeEnv* pRuntimeEnv, SOperatorI pOperator->info = pInfo; pOperator->pRuntimeEnv = pRuntimeEnv; pOperator->cleanupFn = destroyConditionOperatorInfo; - appendDownstream(pOperator, downstream); + int32_t code = appendDownstream(pOperator, &downstream, 1); return pOperator; } @@ -7018,7 +7263,7 @@ SOperatorInfo* createLimitOperatorInfo(STaskRuntimeEnv* pRuntimeEnv, SOperatorIn pOperator->exec = doLimit; pOperator->info = pInfo; pOperator->pRuntimeEnv = pRuntimeEnv; - appendDownstream(pOperator, downstream); + int32_t code = appendDownstream(pOperator, &downstream, 1); return pOperator; } @@ -7028,7 +7273,7 @@ SOperatorInfo* createTimeIntervalOperatorInfo(STaskRuntimeEnv* pRuntimeEnv, SOpe pInfo->pCtx = createSqlFunctionCtx(pRuntimeEnv, pExpr, numOfOutput, &pInfo->rowCellInfoOffset); pInfo->pRes = createOutputBuf(pExpr, numOfOutput, pRuntimeEnv->resultInfo.capacity); - initResultRowInfo(&pInfo->resultRowInfo, 8, TSDB_DATA_TYPE_INT); + initResultRowInfo(&pInfo->resultRowInfo, 8); SOperatorInfo* pOperator = calloc(1, sizeof(SOperatorInfo)); @@ -7043,7 +7288,7 @@ SOperatorInfo* createTimeIntervalOperatorInfo(STaskRuntimeEnv* pRuntimeEnv, SOpe pOperator->exec = doIntervalAgg; pOperator->cleanupFn = destroyBasicOperatorInfo; - appendDownstream(pOperator, downstream); + int32_t code = appendDownstream(pOperator, &downstream, 1); return pOperator; } @@ -7053,7 +7298,7 @@ SOperatorInfo* createAllTimeIntervalOperatorInfo(STaskRuntimeEnv* pRuntimeEnv, S pInfo->pCtx = createSqlFunctionCtx(pRuntimeEnv, pExpr, numOfOutput, &pInfo->rowCellInfoOffset); pInfo->pRes = createOutputBuf(pExpr, numOfOutput, pRuntimeEnv->resultInfo.capacity); - initResultRowInfo(&pInfo->resultRowInfo, 8, TSDB_DATA_TYPE_INT); + initResultRowInfo(&pInfo->resultRowInfo, 8); SOperatorInfo* pOperator = calloc(1, sizeof(SOperatorInfo)); @@ -7068,7 +7313,7 @@ SOperatorInfo* createAllTimeIntervalOperatorInfo(STaskRuntimeEnv* pRuntimeEnv, S pOperator->exec = doAllIntervalAgg; pOperator->cleanupFn = destroyBasicOperatorInfo; - appendDownstream(pOperator, downstream); + int32_t code = appendDownstream(pOperator, &downstream, 1); return pOperator; } @@ -7078,7 +7323,7 @@ SOperatorInfo* createStatewindowOperatorInfo(STaskRuntimeEnv* pRuntimeEnv, SOper pInfo->reptScan = false; pInfo->binfo.pCtx = createSqlFunctionCtx(pRuntimeEnv, pExpr, numOfOutput, &pInfo->binfo.rowCellInfoOffset); pInfo->binfo.pRes = createOutputBuf(pExpr, numOfOutput, pRuntimeEnv->resultInfo.capacity); - initResultRowInfo(&pInfo->binfo.resultRowInfo, 8, TSDB_DATA_TYPE_INT); + initResultRowInfo(&pInfo->binfo.resultRowInfo, 8); SOperatorInfo* pOperator = calloc(1, sizeof(SOperatorInfo)); pOperator->name = "StateWindowOperator"; @@ -7092,7 +7337,7 @@ SOperatorInfo* createStatewindowOperatorInfo(STaskRuntimeEnv* pRuntimeEnv, SOper pOperator->exec = doStateWindowAgg; pOperator->cleanupFn = destroyStateWindowOperatorInfo; - appendDownstream(pOperator, downstream); + int32_t code = appendDownstream(pOperator, &downstream, 1); return pOperator; } SOperatorInfo* createSWindowOperatorInfo(STaskRuntimeEnv* pRuntimeEnv, SOperatorInfo* downstream, SExprInfo* pExpr, int32_t numOfOutput) { @@ -7100,7 +7345,7 @@ SOperatorInfo* createSWindowOperatorInfo(STaskRuntimeEnv* pRuntimeEnv, SOperator pInfo->binfo.pCtx = createSqlFunctionCtx(pRuntimeEnv, pExpr, numOfOutput, &pInfo->binfo.rowCellInfoOffset); pInfo->binfo.pRes = createOutputBuf(pExpr, numOfOutput, pRuntimeEnv->resultInfo.capacity); - initResultRowInfo(&pInfo->binfo.resultRowInfo, 8, TSDB_DATA_TYPE_INT); + initResultRowInfo(&pInfo->binfo.resultRowInfo, 8); pInfo->prevTs = INT64_MIN; pInfo->reptScan = false; @@ -7117,7 +7362,7 @@ SOperatorInfo* createSWindowOperatorInfo(STaskRuntimeEnv* pRuntimeEnv, SOperator pOperator->exec = doSessionWindowAgg; pOperator->cleanupFn = destroySWindowOperatorInfo; - appendDownstream(pOperator, downstream); + int32_t code = appendDownstream(pOperator, &downstream, 1); return pOperator; } @@ -7126,7 +7371,7 @@ SOperatorInfo* createMultiTableTimeIntervalOperatorInfo(STaskRuntimeEnv* pRuntim pInfo->pCtx = createSqlFunctionCtx(pRuntimeEnv, pExpr, numOfOutput, &pInfo->rowCellInfoOffset); pInfo->pRes = createOutputBuf(pExpr, numOfOutput, pRuntimeEnv->resultInfo.capacity); - initResultRowInfo(&pInfo->resultRowInfo, 8, TSDB_DATA_TYPE_INT); + initResultRowInfo(&pInfo->resultRowInfo, 8); SOperatorInfo* pOperator = calloc(1, sizeof(SOperatorInfo)); pOperator->name = "MultiTableTimeIntervalOperator"; @@ -7141,7 +7386,7 @@ SOperatorInfo* createMultiTableTimeIntervalOperatorInfo(STaskRuntimeEnv* pRuntim pOperator->exec = doSTableIntervalAgg; pOperator->cleanupFn = destroyBasicOperatorInfo; - appendDownstream(pOperator, downstream); + int32_t code = appendDownstream(pOperator, &downstream, 1); return pOperator; } @@ -7150,7 +7395,7 @@ SOperatorInfo* createAllMultiTableTimeIntervalOperatorInfo(STaskRuntimeEnv* pRun pInfo->pCtx = createSqlFunctionCtx(pRuntimeEnv, pExpr, numOfOutput, &pInfo->rowCellInfoOffset); pInfo->pRes = createOutputBuf(pExpr, numOfOutput, pRuntimeEnv->resultInfo.capacity); - initResultRowInfo(&pInfo->resultRowInfo, 8, TSDB_DATA_TYPE_INT); + initResultRowInfo(&pInfo->resultRowInfo, 8); SOperatorInfo* pOperator = calloc(1, sizeof(SOperatorInfo)); pOperator->name = "AllMultiTableTimeIntervalOperator"; @@ -7165,12 +7410,11 @@ SOperatorInfo* createAllMultiTableTimeIntervalOperatorInfo(STaskRuntimeEnv* pRun pOperator->exec = doAllSTableIntervalAgg; pOperator->cleanupFn = destroyBasicOperatorInfo; - appendDownstream(pOperator, downstream); + int32_t code = appendDownstream(pOperator, &downstream, 1); return pOperator; } - SOperatorInfo* createGroupbyOperatorInfo(STaskRuntimeEnv* pRuntimeEnv, SOperatorInfo* downstream, SExprInfo* pExpr, int32_t numOfOutput) { SGroupbyOperatorInfo* pInfo = calloc(1, sizeof(SGroupbyOperatorInfo)); pInfo->colIndex = -1; // group by column index @@ -7184,7 +7428,7 @@ SOperatorInfo* createGroupbyOperatorInfo(STaskRuntimeEnv* pRuntimeEnv, SOperator (int32_t)(getRowNumForMultioutput(pQueryAttr, pQueryAttr->topBotQuery, pQueryAttr->stableQuery))); pInfo->binfo.pRes = createOutputBuf(pExpr, numOfOutput, pRuntimeEnv->resultInfo.capacity); - initResultRowInfo(&pInfo->binfo.resultRowInfo, 8, TSDB_DATA_TYPE_INT); + initResultRowInfo(&pInfo->binfo.resultRowInfo, 8); SOperatorInfo* pOperator = calloc(1, sizeof(SOperatorInfo)); pOperator->name = "GroupbyAggOperator"; @@ -7198,7 +7442,7 @@ SOperatorInfo* createGroupbyOperatorInfo(STaskRuntimeEnv* pRuntimeEnv, SOperator pOperator->exec = hashGroupbyAggregate; pOperator->cleanupFn = destroyGroupbyOperatorInfo; - appendDownstream(pOperator, downstream); + int32_t code = appendDownstream(pOperator, &downstream, 1); return pOperator; } @@ -7237,7 +7481,7 @@ SOperatorInfo* createFillOperatorInfo(STaskRuntimeEnv* pRuntimeEnv, SOperatorInf pOperator->exec = doFill; pOperator->cleanupFn = destroySFillOperatorInfo; - appendDownstream(pOperator, downstream); + int32_t code = appendDownstream(pOperator, &downstream, 1); return pOperator; } @@ -7285,7 +7529,7 @@ SOperatorInfo* createSLimitOperatorInfo(STaskRuntimeEnv* pRuntimeEnv, SOperatorI pOperator->pRuntimeEnv = pRuntimeEnv; pOperator->cleanupFn = destroySlimitOperatorInfo; - appendDownstream(pOperator, downstream); + int32_t code = appendDownstream(pOperator, &downstream, 1); return pOperator; } @@ -7583,7 +7827,7 @@ SOperatorInfo* createDistinctOperatorInfo(STaskRuntimeEnv* pRuntimeEnv, SOperato pOperator->pExpr = pExpr; pOperator->cleanupFn = destroyDistinctOperatorInfo; - appendDownstream(pOperator, downstream); + int32_t code = appendDownstream(pOperator, &downstream, 1); return pOperator; } @@ -7963,7 +8207,6 @@ int32_t buildArithmeticExprFromMsg(SExprInfo *pExprInfo, void *pQueryMsg) { return TSDB_CODE_SUCCESS; } - static int32_t updateOutputBufForTopBotQuery(SQueriedTableInfo* pTableInfo, SColumnInfo* pTagCols, SExprInfo* pExprs, int32_t numOfOutput, int32_t tagLen, bool superTable) { for (int32_t i = 0; i < numOfOutput; ++i) { int16_t functId = getExprFunctionId(&pExprs[i]); @@ -8127,66 +8370,6 @@ int32_t createQueryFilter(char *data, uint16_t len, SFilterInfo** pFilters) { // return ret; } - -// todo refactor -int32_t createIndirectQueryFuncExprFromMsg(SQueryTableReq* pQueryMsg, int32_t numOfOutput, SExprInfo** pExprInfo, - SSqlExpr** pExpr, SExprInfo* prevExpr, struct SUdfInfo *pUdfInfo) { -// *pExprInfo = NULL; -// int32_t code = TSDB_CODE_SUCCESS; -// -// SExprInfo *pExprs = (SExprInfo *)calloc(numOfOutput, sizeof(SExprInfo)); -// if (pExprs == NULL) { -// return TSDB_CODE_QRY_OUT_OF_MEMORY; -// } -// -// bool isSuperTable = QUERY_IS_STABLE_QUERY(pQueryMsg->queryType); -// -// for (int32_t i = 0; i < numOfOutput; ++i) { -// pExprs[i].base = *pExpr[i]; -// memset(pExprs[i].base.param, 0, sizeof(SVariant) * tListLen(pExprs[i].base.param)); -// -// for (int32_t j = 0; j < pExpr[i]->numOfParams; ++j) { -// taosVariantAssign(&pExprs[i].base.param[j], &pExpr[i]->param[j]); -// } -// -// pExprs[i].base.resSchema.type = 0; -// -// int16_t type = 0; -// int16_t bytes = 0; -// -// // parse the arithmetic expression -// if (pExprs[i].base.functionId == FUNCTION_ARITHM) { -// code = buildArithmeticExprFromMsg(&pExprs[i], pQueryMsg); -// -// if (code != TSDB_CODE_SUCCESS) { -// tfree(pExprs); -// return code; -// } -// -// type = TSDB_DATA_TYPE_DOUBLE; -// bytes = tDataTypes[type].bytes; -// } else { -// int32_t index = pExprs[i].base.colInfo.colIndex; -// assert(prevExpr[index].base.resSchema.colId == pExprs[i].base.pColumns->info.colId); -// -// type = prevExpr[index].base.resSchema.type; -// bytes = prevExpr[index].base.resSchema.bytes; -// } -// -// int32_t param = (int32_t)pExprs[i].base.param[0].i; -// if (getResultDataInfo(type, bytes, functionId, param, &pExprs[i].base.resSchema.type, &pExprs[i].base.resSchema.bytes, -// &pExprs[i].base.interBytes, 0, isSuperTable, pUdfInfo) != TSDB_CODE_SUCCESS) { -// tfree(pExprs); -// return TSDB_CODE_QRY_INVALID_MSG; -// } -// -// assert(isValidDataType(pExprs[i].base.resSchema.type)); -// } -// -// *pExprInfo = pExprs; - return TSDB_CODE_SUCCESS; -} - SGroupbyExpr *createGroupbyExprFromMsg(SQueryTableReq *pQueryMsg, SColIndex *pColIndex, int32_t *code) { if (pQueryMsg->numOfGroupCols == 0) { return NULL; @@ -8595,30 +8778,3 @@ void releaseQueryBuf(size_t numOfTables) { // restore value is not enough buffer available atomic_add_fetch_64(&tsQueryBufferSizeBytes, t); } - -void freeQueryAttr(STaskAttr* pQueryAttr) { - if (pQueryAttr != NULL) { - if (pQueryAttr->fillVal != NULL) { - tfree(pQueryAttr->fillVal); - } - - pQueryAttr->pFilterInfo = doDestroyFilterInfo(pQueryAttr->pFilterInfo, pQueryAttr->numOfFilterCols); - - pQueryAttr->pExpr1 = destroyQueryFuncExpr(pQueryAttr->pExpr1, pQueryAttr->numOfOutput); - pQueryAttr->pExpr2 = destroyQueryFuncExpr(pQueryAttr->pExpr2, pQueryAttr->numOfExpr2); - pQueryAttr->pExpr3 = destroyQueryFuncExpr(pQueryAttr->pExpr3, pQueryAttr->numOfExpr3); - - tfree(pQueryAttr->tagColList); - tfree(pQueryAttr->pFilterInfo); - - pQueryAttr->tableCols = freeColumnInfo(pQueryAttr->tableCols, pQueryAttr->numOfCols); - - if (pQueryAttr->pGroupbyExpr != NULL) { - taosArrayDestroy(pQueryAttr->pGroupbyExpr->columnInfo); - tfree(pQueryAttr->pGroupbyExpr); - } - -// filterFreeInfo(pQueryAttr->pFilters); - } -} - diff --git a/source/libs/executor/src/tsort.c b/source/libs/executor/src/tsort.c index 4854f0cbed..89f555517e 100644 --- a/source/libs/executor/src/tsort.c +++ b/source/libs/executor/src/tsort.c @@ -123,23 +123,6 @@ int32_t sortAddSource(SSortHandle* pSortHandle, void* pSource) { taosArrayPush(pSortHandle->pOrderedSource, &pSource); } -static SSDataBlock* createDataBlock(const SSDataBlock* pDataBlock) { - int32_t numOfCols = pDataBlock->info.numOfCols; - - SSDataBlock* pBlock = calloc(1, sizeof(SSDataBlock)); - pBlock->pDataBlock = taosArrayInit(numOfCols, sizeof(SColumnInfoData)); - pBlock->info.numOfCols = numOfCols; - - for(int32_t i = 0; i < numOfCols; ++i) { - SColumnInfoData colInfo = {0}; - SColumnInfoData* p = taosArrayGet(pDataBlock->pDataBlock, i); - colInfo.info = p->info; - taosArrayPush(pBlock->pDataBlock, &colInfo); - } - - return pBlock; -} - static int32_t doAddNewExternalMemSource(SDiskbasedBuf *pBuf, SArray* pAllSources, SSDataBlock* pBlock, int32_t* sourceId) { SExternalMemSource* pSource = calloc(1, sizeof(SExternalMemSource)); if (pSource == NULL) { @@ -198,7 +181,7 @@ static int32_t doAddToBuf(SSDataBlock* pDataBlock, SSortHandle* pHandle) { blockDataClearup(pDataBlock, pHandle->hasVarCol); - SSDataBlock* pBlock = createDataBlock(pDataBlock); + SSDataBlock* pBlock = createOneDataBlock(pDataBlock); int32_t code = doAddNewExternalMemSource(pHandle->pBuf, pHandle->pOrderedSource, pBlock, &pHandle->sourceId); if (code != TSDB_CODE_SUCCESS) { return code; @@ -263,7 +246,7 @@ static void appendOneRowToDataBlock(SSDataBlock *pBlock, const SSDataBlock* pSou if (isNull) { colDataAppend(pColInfo, pBlock->info.rows, NULL, true); } else { - char* pData = colDataGet(pSrcColInfo, *rowIndex); + char* pData = colDataGetData(pSrcColInfo, *rowIndex); colDataAppend(pColInfo, pBlock->info.rows, pData, false); } } @@ -279,15 +262,14 @@ static int32_t adjustMergeTreeForNextTuple(SExternalMemSource *pSource, SMultiwa */ if (pSource->src.rowIndex >= pSource->src.pBlock->info.rows) { pSource->src.rowIndex = 0; - pSource->pageIndex += 1; - if (pSource->pageIndex >= taosArrayGetSize(pSource->pageIdList)) { - (*numOfCompleted) += 1; - pSource->src.rowIndex = -1; - pSource->pageIndex = -1; - pSource->src.pBlock = blockDataDestroy(pSource->src.pBlock); - } else { - if (pHandle->type == SORT_SINGLESOURCE_SORT) { + if (pHandle->type == SORT_SINGLESOURCE_SORT) { + if (pSource->pageIndex >= taosArrayGetSize(pSource->pageIdList)) { + (*numOfCompleted) += 1; + pSource->src.rowIndex = -1; + pSource->pageIndex = -1; + pSource->src.pBlock = blockDataDestroy(pSource->src.pBlock); + } else { SPageInfo* pPgInfo = *(SPageInfo**)taosArrayGet(pSource->pageIdList, pSource->pageIndex); SFilePage* pPage = getBufPage(pHandle->pBuf, getPageId(pPgInfo)); @@ -297,12 +279,12 @@ static int32_t adjustMergeTreeForNextTuple(SExternalMemSource *pSource, SMultiwa } releaseBufPage(pHandle->pBuf, pPage); - } else { - pSource->src.pBlock = pHandle->fetchfp(((SGenericSource*)pSource)->param); - if (pSource->src.pBlock == NULL) { - (*numOfCompleted) += 1; - pSource->src.rowIndex = -1; - } + } + } else { + pSource->src.pBlock = pHandle->fetchfp(((SGenericSource*)pSource)->param); + if (pSource->src.pBlock == NULL) { + (*numOfCompleted) += 1; + pSource->src.rowIndex = -1; } } } @@ -404,8 +386,8 @@ int32_t msortComparFn(const void *pLeft, const void *pRight, void *param) { return pParam->nullFirst? -1:1; } - void* left1 = colDataGet(pLeftColInfoData, pLeftSource->src.rowIndex); - void* right1 = colDataGet(pRightColInfoData, pRightSource->src.rowIndex); + void* left1 = colDataGetData(pLeftColInfoData, pLeftSource->src.rowIndex); + void* right1 = colDataGetData(pRightColInfoData, pRightSource->src.rowIndex); switch(pLeftColInfoData->info.type) { case TSDB_DATA_TYPE_INT: { @@ -499,7 +481,7 @@ static int32_t doInternalMergeSort(SSortHandle* pHandle) { tMergeTreeDestroy(pHandle->pMergeTree); pHandle->numOfCompletedSources = 0; - SSDataBlock* pBlock = createDataBlock(pHandle->pDataBlock); + SSDataBlock* pBlock = createOneDataBlock(pHandle->pDataBlock); code = doAddNewExternalMemSource(pHandle->pBuf, pResList, pBlock, &pHandle->sourceId); if (code != 0) { return code; @@ -545,7 +527,7 @@ static int32_t createInitialSortedMultiSources(SSortHandle* pHandle) { } if (pHandle->pDataBlock == NULL) { - pHandle->pDataBlock = createDataBlock(pBlock); + pHandle->pDataBlock = createOneDataBlock(pBlock); } int32_t code = blockDataMerge(pHandle->pDataBlock, pBlock); @@ -646,6 +628,7 @@ STupleHandle* sortNextTuple(SSortHandle* pHandle) { return NULL; } + // All the data are hold in the buffer, no external sort is invoked. if (pHandle->inMemSort) { pHandle->tupleHandle.rowIndex += 1; if (pHandle->tupleHandle.rowIndex == pHandle->pDataBlock->info.rows) { @@ -671,6 +654,7 @@ STupleHandle* sortNextTuple(SSortHandle* pHandle) { return NULL; } + // Get the adjusted value after the loser tree is updated. index = tMergeTreeGetChosenIndex(pHandle->pMergeTree); pSource = pHandle->cmpParam.pSources[index]; @@ -691,5 +675,5 @@ bool sortIsValueNull(STupleHandle* pVHandle, int32_t colIndex) { void* sortGetValue(STupleHandle* pVHandle, int32_t colIndex) { SColumnInfoData* pColInfo = TARRAY_GET_ELEM(pVHandle->pBlock->pDataBlock, colIndex); - return colDataGet(pColInfo, pVHandle->rowIndex); + return colDataGetData(pColInfo, pVHandle->rowIndex); } diff --git a/source/libs/executor/test/executorTests.cpp b/source/libs/executor/test/executorTests.cpp index 455e731c05..bf3f76b94e 100644 --- a/source/libs/executor/test/executorTests.cpp +++ b/source/libs/executor/test/executorTests.cpp @@ -42,22 +42,21 @@ enum { }; typedef struct SDummyInputInfo { - int32_t max; - int32_t current; - int32_t startVal; - int32_t type; + int32_t totalPages; // numOfPages + int32_t current; + int32_t startVal; + int32_t type; + int32_t numOfRowsPerPage; SSDataBlock* pBlock; } SDummyInputInfo; SSDataBlock* getDummyBlock(void* param, bool* newgroup) { SOperatorInfo* pOperator = static_cast(param); SDummyInputInfo* pInfo = static_cast(pOperator->info); - if (pInfo->current >= pInfo->max) { + if (pInfo->current >= pInfo->totalPages) { return NULL; } - int32_t numOfRows = 1000; - if (pInfo->pBlock == NULL) { pInfo->pBlock = static_cast(calloc(1, sizeof(SSDataBlock))); @@ -67,8 +66,8 @@ SSDataBlock* getDummyBlock(void* param, bool* newgroup) { colInfo.info.type = TSDB_DATA_TYPE_INT; colInfo.info.bytes = sizeof(int32_t); colInfo.info.colId = 1; - colInfo.pData = static_cast(calloc(numOfRows, sizeof(int32_t))); - colInfo.nullbitmap = static_cast(calloc(1, (numOfRows + 7) / 8)); + colInfo.pData = static_cast(calloc(pInfo->numOfRowsPerPage, sizeof(int32_t))); + colInfo.nullbitmap = static_cast(calloc(1, (pInfo->numOfRowsPerPage + 7) / 8)); taosArrayPush(pInfo->pBlock->pDataBlock, &colInfo); @@ -91,7 +90,7 @@ SSDataBlock* getDummyBlock(void* param, bool* newgroup) { char buf[128] = {0}; char b1[128] = {0}; int32_t v = 0; - for(int32_t i = 0; i < numOfRows; ++i) { + for(int32_t i = 0; i < pInfo->numOfRowsPerPage; ++i) { SColumnInfoData* pColInfo = static_cast(TARRAY_GET_ELEM(pBlock->pDataBlock, 0)); if (pInfo->type == data_desc) { @@ -111,22 +110,23 @@ SSDataBlock* getDummyBlock(void* param, bool* newgroup) { // colDataAppend(pColInfo2, i, b1, false); } - pBlock->info.rows = numOfRows; + pBlock->info.rows = pInfo->numOfRowsPerPage; pBlock->info.numOfCols = 1; pInfo->current += 1; return pBlock; } -SOperatorInfo* createDummyOperator(int32_t numOfBlocks, int32_t type) { +SOperatorInfo* createDummyOperator(int32_t startVal, int32_t numOfBlocks, int32_t rowsPerPage, int32_t type) { SOperatorInfo* pOperator = static_cast(calloc(1, sizeof(SOperatorInfo))); pOperator->name = "dummyInputOpertor4Test"; pOperator->exec = getDummyBlock; SDummyInputInfo *pInfo = (SDummyInputInfo*) calloc(1, sizeof(SDummyInputInfo)); - pInfo->max = numOfBlocks; - pInfo->startVal = 1500000; - pInfo->type = type; + pInfo->totalPages = numOfBlocks; + pInfo->startVal = startVal; + pInfo->numOfRowsPerPage = rowsPerPage; + pInfo->type = type; pOperator->info = pInfo; return pOperator; @@ -257,7 +257,7 @@ TEST(testCase, inMem_sort_Test) { SColumnInfoData* pCol1 = static_cast(taosArrayGet(pRes->pDataBlock, 0)); SColumnInfoData* pCol2 = static_cast(taosArrayGet(pRes->pDataBlock, 1)); for(int32_t i = 0; i < pRes->info.rows; ++i) { - char* p = colDataGet(pCol2, i); + char* p = colDataGetData(pCol2, i); printf("%d: %d, %s\n", i, ((int32_t*)pCol1->pData)[i], (char*)varDataVal(p)); } } @@ -341,7 +341,7 @@ TEST(testCase, external_sort_Test) { SColumnInfoData* pCol1 = static_cast(taosArrayGet(pRes->pDataBlock, 0)); // SColumnInfoData* pCol2 = static_cast(taosArrayGet(pRes->pDataBlock, 1)); for (int32_t i = 0; i < pRes->info.rows; ++i) { -// char* p = colDataGet(pCol2, i); +// char* p = colDataGetData(pCol2, i); printf("%d: %d\n", total++, ((int32_t*)pCol1->pData)[i]); // printf("%d: %d, %s\n", total++, ((int32_t*)pCol1->pData)[i], (char*)varDataVal(p)); } @@ -357,6 +357,7 @@ TEST(testCase, external_sort_Test) { taosArrayDestroy(pOrderVal); } +#endif TEST(testCase, sorted_merge_Test) { srand(time(NULL)); @@ -370,7 +371,12 @@ TEST(testCase, sorted_merge_Test) { SArray* pExprInfo = taosArrayInit(4, sizeof(SExprInfo)); SExprInfo *exp = static_cast(calloc(1, sizeof(SExprInfo))); - exp->base.resSchema = createSchema(TSDB_DATA_TYPE_INT, sizeof(int32_t), 1, "res"); + exp->base.resSchema = createSchema(TSDB_DATA_TYPE_BIGINT, sizeof(int64_t), 1, "count_result"); + exp->base.pColumns = static_cast(calloc(1, sizeof(SColumn))); + exp->base.pColumns->flag = TSDB_COL_NORMAL; + exp->base.pColumns->info = (SColumnInfo) {.colId = 1, .type = TSDB_DATA_TYPE_INT, .bytes = 4}; + exp->base.numOfCols = 1; + taosArrayPush(pExprInfo, &exp); SExprInfo *exp1 = static_cast(calloc(1, sizeof(SExprInfo))); @@ -380,10 +386,10 @@ TEST(testCase, sorted_merge_Test) { int32_t numOfSources = 10; SOperatorInfo** plist = (SOperatorInfo**) calloc(numOfSources, sizeof(void*)); for(int32_t i = 0; i < numOfSources; ++i) { - plist[i] = createDummyOperator(1, data_asc); + plist[i] = createDummyOperator(1, 1, 1, data_asc); } - SOperatorInfo* pOperator = createSortedMergeOperatorInfo(plist, numOfSources, pExprInfo, pOrderVal, NULL); + SOperatorInfo* pOperator = createSortedMergeOperatorInfo(plist, numOfSources, pExprInfo, pOrderVal, NULL, NULL); bool newgroup = false; SSDataBlock* pRes = NULL; @@ -409,8 +415,8 @@ TEST(testCase, sorted_merge_Test) { SColumnInfoData* pCol1 = static_cast(taosArrayGet(pRes->pDataBlock, 0)); // SColumnInfoData* pCol2 = static_cast(taosArrayGet(pRes->pDataBlock, 1)); for (int32_t i = 0; i < pRes->info.rows; ++i) { -// char* p = colDataGet(pCol2, i); - printf("%d: %d\n", total++, ((int32_t*)pCol1->pData)[i]); +// char* p = colDataGetData(pCol2, i); + printf("%d: %ld\n", total++, ((int64_t*)pCol1->pData)[i]); // printf("%d: %d, %s\n", total++, ((int32_t*)pCol1->pData)[i], (char*)varDataVal(p)); } } @@ -424,5 +430,4 @@ TEST(testCase, sorted_merge_Test) { taosArrayDestroy(pExprInfo); taosArrayDestroy(pOrderVal); } -#endif #pragma GCC diagnostic pop diff --git a/source/libs/executor/test/executorUtilTests.cpp b/source/libs/executor/test/executorUtilTests.cpp index 754d4b6b80..6a023170b2 100644 --- a/source/libs/executor/test/executorUtilTests.cpp +++ b/source/libs/executor/test/executorUtilTests.cpp @@ -123,8 +123,8 @@ int32_t docomp(const void* p1, const void* p2, void* param) { return pParam->nullFirst? -1:1; } - void* left1 = colDataGet(pLeftColInfoData, pLeftSource->src.rowIndex); - void* right1 = colDataGet(pRightColInfoData, pRightSource->src.rowIndex); + void* left1 = colDataGetData(pLeftColInfoData, pLeftSource->src.rowIndex); + void* right1 = colDataGetData(pRightColInfoData, pRightSource->src.rowIndex); switch(pLeftColInfoData->info.type) { case TSDB_DATA_TYPE_INT: { @@ -148,48 +148,15 @@ int32_t docomp(const void* p1, const void* p2, void* param) { } } // namespace -//TEST(testCase, inMem_sort_Test) { -// SArray* pOrderVal = taosArrayInit(4, sizeof(SOrder)); -// SOrder o = {.order = TSDB_ORDER_ASC}; -// o.col.info.colId = 1; -// o.col.info.type = TSDB_DATA_TYPE_INT; -// taosArrayPush(pOrderVal, &o); -// -// int32_t numOfRows = 1000; -// SBlockOrderInfo oi = {0}; -// oi.order = TSDB_ORDER_ASC; -// oi.colIndex = 0; -// SArray* orderInfo = taosArrayInit(1, sizeof(SBlockOrderInfo)); -// taosArrayPush(orderInfo, &oi); -// -// SSortHandle* phandle = createSortHandle(orderInfo, false, SORT_SINGLESOURCE_SORT, 1024, 5, "test_abc"); -// setFetchRawDataFp(phandle, getSingleColDummyBlock); -// sortAddSource(phandle, &numOfRows); -// -// int32_t code = sortOpen(phandle); -// int32_t row = 1; -// -// while(1) { -// STupleHandle* pTupleHandle = sortNextTuple(phandle); -// if (pTupleHandle == NULL) { -// break; -// } -// -// void* v = sortGetValue(pTupleHandle, 0); -// printf("%d: %d\n", row++, *(int32_t*) v); -// -// } -// destroySortHandle(phandle); -//} -// -TEST(testCase, external_mem_sort_Test) { +#if 0 +TEST(testCase, inMem_sort_Test) { SArray* pOrderVal = taosArrayInit(4, sizeof(SOrder)); SOrder o = {.order = TSDB_ORDER_ASC}; o.col.info.colId = 1; o.col.info.type = TSDB_DATA_TYPE_INT; taosArrayPush(pOrderVal, &o); -// int32_t numOfRows = 1000; + int32_t numOfRows = 1000; SBlockOrderInfo oi = {0}; oi.order = TSDB_ORDER_ASC; oi.colIndex = 0; @@ -197,7 +164,40 @@ TEST(testCase, external_mem_sort_Test) { taosArrayPush(orderInfo, &oi); SSchema s = {.type = TSDB_DATA_TYPE_INT, .colId = 1, .bytes = 4, }; + SSortHandle* phandle = createSortHandle(orderInfo, false, SORT_SINGLESOURCE_SORT, 1024, 5, &s, 1, "test_abc"); + setFetchRawDataFp(phandle, getSingleColDummyBlock); + sortAddSource(phandle, &numOfRows); + int32_t code = sortOpen(phandle); + int32_t row = 1; + + while(1) { + STupleHandle* pTupleHandle = sortNextTuple(phandle); + if (pTupleHandle == NULL) { + break; + } + + void* v = sortGetValue(pTupleHandle, 0); + printf("%d: %d\n", row++, *(int32_t*) v); + + } + destroySortHandle(phandle); +} + +TEST(testCase, external_mem_sort_Test) { + SArray* pOrderVal = taosArrayInit(4, sizeof(SOrder)); + SOrder o = {.order = TSDB_ORDER_ASC}; + o.col.info.colId = 1; + o.col.info.type = TSDB_DATA_TYPE_INT; + taosArrayPush(pOrderVal, &o); + + SBlockOrderInfo oi = {0}; + oi.order = TSDB_ORDER_ASC; + oi.colIndex = 0; + SArray* orderInfo = taosArrayInit(1, sizeof(SBlockOrderInfo)); + taosArrayPush(orderInfo, &oi); + + SSchema s = {.type = TSDB_DATA_TYPE_INT, .colId = 1, .bytes = 4, }; SSortHandle* phandle = createSortHandle(orderInfo, false, SORT_SINGLESOURCE_SORT, 1024, 5, &s, 1, "test_abc"); setFetchRawDataFp(phandle, getSingleColDummyBlock); @@ -227,50 +227,52 @@ TEST(testCase, external_mem_sort_Test) { destroySortHandle(phandle); } -//TEST(testCase, ordered_merge_sort_Test) { -// SArray* pOrderVal = taosArrayInit(4, sizeof(SOrder)); -// SOrder o = {.order = TSDB_ORDER_ASC}; -// o.col.info.colId = 1; -// o.col.info.type = TSDB_DATA_TYPE_INT; -// taosArrayPush(pOrderVal, &o); -// -// int32_t numOfRows = 1000; -// SBlockOrderInfo oi = {0}; -// oi.order = TSDB_ORDER_ASC; -// oi.colIndex = 0; -// SArray* orderInfo = taosArrayInit(1, sizeof(SBlockOrderInfo)); -// taosArrayPush(orderInfo, &oi); -// -// SSchema s = {.type = TSDB_DATA_TYPE_INT, .colId = 1, .bytes = 4}; -// SSortHandle* phandle = createSortHandle(orderInfo, false, SORT_MULTISOURCE_MERGE, 1024, 5, &s, 1,"test_abc"); -// setFetchRawDataFp(phandle, getSingleColDummyBlock); -// setComparFn(phandle, docomp); -// -// for(int32_t i = 0; i < 10; ++i) { -// SOperatorSource* p = static_cast(calloc(1, sizeof(SOperatorSource))); -// _info* c = static_cast<_info*>(calloc(1, sizeof(_info))); -// c->count = 1; -// c->pageRows = 1000; -// c->startVal = 0; -// -// p->param = c; -// sortAddSource(phandle, p); -// } -// -// int32_t code = sortOpen(phandle); -// int32_t row = 1; -// -// while(1) { -// STupleHandle* pTupleHandle = sortNextTuple(phandle); -// if (pTupleHandle == NULL) { -// break; -// } -// -// void* v = sortGetValue(pTupleHandle, 0); -// printf("%d: %d\n", row++, *(int32_t*) v); -// -// } -// destroySortHandle(phandle); -//} +TEST(testCase, ordered_merge_sort_Test) { + SArray* pOrderVal = taosArrayInit(4, sizeof(SOrder)); + SOrder o = {.order = TSDB_ORDER_ASC}; + o.col.info.colId = 1; + o.col.info.type = TSDB_DATA_TYPE_INT; + taosArrayPush(pOrderVal, &o); + + int32_t numOfRows = 1000; + SBlockOrderInfo oi = {0}; + oi.order = TSDB_ORDER_ASC; + oi.colIndex = 0; + SArray* orderInfo = taosArrayInit(1, sizeof(SBlockOrderInfo)); + taosArrayPush(orderInfo, &oi); + + SSchema s = {.type = TSDB_DATA_TYPE_INT, .colId = 1, .bytes = 4}; + SSortHandle* phandle = createSortHandle(orderInfo, false, SORT_MULTISOURCE_MERGE, 1024, 5, &s, 1,"test_abc"); + setFetchRawDataFp(phandle, getSingleColDummyBlock); + setComparFn(phandle, docomp); + + for(int32_t i = 0; i < 10; ++i) { + SGenericSource* p = static_cast(calloc(1, sizeof(SGenericSource))); + _info* c = static_cast<_info*>(calloc(1, sizeof(_info))); + c->count = 1; + c->pageRows = 1000; + c->startVal = 0; + + p->param = c; + sortAddSource(phandle, p); + } + + int32_t code = sortOpen(phandle); + int32_t row = 1; + + while(1) { + STupleHandle* pTupleHandle = sortNextTuple(phandle); + if (pTupleHandle == NULL) { + break; + } + + void* v = sortGetValue(pTupleHandle, 0); + printf("%d: %d\n", row++, *(int32_t*) v); + + } + destroySortHandle(phandle); +} + +#endif #pragma GCC diagnostic pop diff --git a/source/libs/function/src/taggfunction.c b/source/libs/function/src/taggfunction.c index d0d89611c4..0615ff9627 100644 --- a/source/libs/function/src/taggfunction.c +++ b/source/libs/function/src/taggfunction.c @@ -30,9 +30,10 @@ #include "tcompression.h" //#include "queryLog.h" #include "tudf.h" +#include "tep.h" #define GET_INPUT_DATA_LIST(x) ((char *)((x)->pInput)) -#define GET_INPUT_DATA(x, y) (GET_INPUT_DATA_LIST(x) + (y) * (x)->inputBytes) +#define GET_INPUT_DATA(x, y) ((char*) colDataGetData((x)->pInput, (y))) #define GET_TS_LIST(x) ((TSKEY*)((x)->ptsList)) #define GET_TS_DATA(x, y) (GET_TS_LIST(x)[(y)]) @@ -3817,7 +3818,7 @@ static void interp_function_impl(SqlFunctionCtx *pCtx) { skey = ekey; } } - assignVal(pCtx->pOutput, pCtx->pInput, pCtx->resDataInfo.bytes, pCtx->inputType); +// assignVal(pCtx->pOutput, pCtx->pInput, pCtx->resDataInfo.bytes, pCtx->inputType); } else if (type == TSDB_FILL_NEXT) { TSKEY ekey = skey; char* val = NULL; diff --git a/source/libs/parser/src/queryInfoUtil.c b/source/libs/parser/src/queryInfoUtil.c index 9a2ca2da98..9b355b0775 100644 --- a/source/libs/parser/src/queryInfoUtil.c +++ b/source/libs/parser/src/queryInfoUtil.c @@ -230,7 +230,7 @@ int32_t getExprFunctionId(SExprInfo *pExprInfo) { } void assignExprInfo(SExprInfo* dst, const SExprInfo* src) { - assert(dst != NULL && src != NULL); + assert(dst != NULL && src != NULL && src->base.numOfCols > 0); *dst = *src; #if 0 From 0c5a8dc3ff2b74c3ead9a513cbd013180bb35516 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 22 Feb 2022 13:16:46 +0800 Subject: [PATCH 12/25] [td-13039] refactor. --- source/libs/executor/inc/tsort.h | 20 ++++----- source/libs/executor/src/executorimpl.c | 26 +++++------ source/libs/executor/src/tsort.c | 24 +++++----- .../libs/executor/test/executorUtilTests.cpp | 44 +++++++++---------- 4 files changed, 57 insertions(+), 57 deletions(-) diff --git a/source/libs/executor/inc/tsort.h b/source/libs/executor/inc/tsort.h index 92258f619f..9c9368bb1a 100644 --- a/source/libs/executor/inc/tsort.h +++ b/source/libs/executor/inc/tsort.h @@ -63,33 +63,33 @@ typedef int32_t (*_sort_merge_compar_fn_t)(const void* p1, const void* p2, void* * @param type * @return */ -SSortHandle* createSortHandle(SArray* pOrderInfo, bool nullFirst, int32_t type, int32_t pageSize, int32_t numOfPages, SSchema* pSchema, int32_t numOfCols, const char* idstr); +SSortHandle* tsortCreateSortHandle(SArray* pOrderInfo, bool nullFirst, int32_t type, int32_t pageSize, int32_t numOfPages, SSchema* pSchema, int32_t numOfCols, const char* idstr); /** * * @param pSortHandle */ -void destroySortHandle(SSortHandle* pSortHandle); +void tsortDestroySortHandle(SSortHandle* pSortHandle); /** * * @param pHandle * @return */ -int32_t sortOpen(SSortHandle* pHandle); +int32_t tsortOpen(SSortHandle* pHandle); /** * * @param pHandle * @return */ -int32_t sortClose(SSortHandle* pHandle); +int32_t tsortClose(SSortHandle* pHandle); /** * * @return */ -int32_t setFetchRawDataFp(SSortHandle* pHandle, _sort_fetch_block_fn_t fp); +int32_t tsortSetFetchRawDataFp(SSortHandle* pHandle, _sort_fetch_block_fn_t fp); /** * @@ -97,7 +97,7 @@ int32_t setFetchRawDataFp(SSortHandle* pHandle, _sort_fetch_block_fn_t fp); * @param fp * @return */ -int32_t setComparFn(SSortHandle* pHandle, _sort_merge_compar_fn_t fp); +int32_t tsortSetComparFp(SSortHandle* pHandle, _sort_merge_compar_fn_t fp); /** * @@ -105,14 +105,14 @@ int32_t setComparFn(SSortHandle* pHandle, _sort_merge_compar_fn_t fp); * @param pSource * @return success or failed */ -int32_t sortAddSource(SSortHandle* pSortHandle, void* pSource); +int32_t tsortAddSource(SSortHandle* pSortHandle, void* pSource); /** * * @param pHandle * @return */ -STupleHandle* sortNextTuple(SSortHandle* pHandle); +STupleHandle* tsortNextTuple(SSortHandle* pHandle); /** * @@ -120,7 +120,7 @@ STupleHandle* sortNextTuple(SSortHandle* pHandle); * @param colIndex * @return */ -bool sortIsValueNull(STupleHandle* pVHandle, int32_t colIndex); +bool tsortIsNullVal(STupleHandle* pVHandle, int32_t colIndex); /** * @@ -128,7 +128,7 @@ bool sortIsValueNull(STupleHandle* pVHandle, int32_t colIndex); * @param colIndex * @return */ -void* sortGetValue(STupleHandle* pVHandle, int32_t colIndex); +void* tsortGetValue(STupleHandle* pVHandle, int32_t colIndex); #ifdef __cplusplus } diff --git a/source/libs/executor/src/executorimpl.c b/source/libs/executor/src/executorimpl.c index efa4afb422..f2d98ea86c 100644 --- a/source/libs/executor/src/executorimpl.c +++ b/source/libs/executor/src/executorimpl.c @@ -5586,7 +5586,7 @@ static void destroySortedMergeOperatorInfo(void* param, int32_t numOfOutput) { taosArrayDestroy(pInfo->groupInfo); if (pInfo->pSortHandle != NULL) { - destroySortHandle(pInfo->pSortHandle); + tsortDestroySortHandle(pInfo->pSortHandle); } blockDataDestroy(pInfo->binfo.pRes); @@ -5617,11 +5617,11 @@ static void appendOneRowToDataBlock(SSDataBlock *pBlock, STupleHandle* pTupleHan for (int32_t i = 0; i < pBlock->info.numOfCols; ++i) { SColumnInfoData* pColInfo = taosArrayGet(pBlock->pDataBlock, i); - bool isNull = sortIsValueNull(pTupleHandle, i); + bool isNull = tsortIsNullVal(pTupleHandle, i); if (isNull) { colDataAppend(pColInfo, pBlock->info.rows, NULL, true); } else { - char* pData = sortGetValue(pTupleHandle, i); + char* pData = tsortGetValue(pTupleHandle, i); colDataAppend(pColInfo, pBlock->info.rows, pData, false); } } @@ -5633,7 +5633,7 @@ static SSDataBlock* getSortedBlockData(SSortHandle* pHandle, SSDataBlock* pDataB blockDataClearup(pDataBlock, hasVarCol); while(1) { - STupleHandle* pTupleHandle = sortNextTuple(pHandle); + STupleHandle* pTupleHandle = tsortNextTuple(pHandle); if (pTupleHandle == NULL) { break; } @@ -5788,7 +5788,7 @@ static SSDataBlock* doMerge(SOperatorInfo* pOperator) { blockDataClearup(pDataBlock, pInfo->hasVarCol); while (1) { - STupleHandle* pTupleHandle = sortNextTuple(pHandle); + STupleHandle* pTupleHandle = tsortNextTuple(pHandle); if (pTupleHandle == NULL) { break; } @@ -5835,19 +5835,19 @@ static SSDataBlock* doSortedMerge(void* param, bool* newgroup) { SSchema* p = blockDataExtractSchema(pInfo->binfo.pRes, NULL); int32_t numOfBufPage = pInfo->sortBufSize / pInfo->bufPageSize; - pInfo->pSortHandle = createSortHandle(pInfo->orderInfo, pInfo->nullFirst, SORT_MULTISOURCE_MERGE, pInfo->bufPageSize, + pInfo->pSortHandle = tsortCreateSortHandle(pInfo->orderInfo, pInfo->nullFirst, SORT_MULTISOURCE_MERGE, pInfo->bufPageSize, numOfBufPage, p, pInfo->binfo.pRes->info.numOfCols, "GET_TASKID(pTaskInfo)"); tfree(p); - setFetchRawDataFp(pInfo->pSortHandle, loadNextDataBlock); + tsortSetFetchRawDataFp(pInfo->pSortHandle, loadNextDataBlock); for(int32_t i = 0; i < pOperator->numOfDownstream; ++i) { SGenericSource* ps = calloc(1, sizeof(SGenericSource)); ps->param = pOperator->pDownstream[i]; - sortAddSource(pInfo->pSortHandle, ps); + tsortAddSource(pInfo->pSortHandle, ps); } - int32_t code = sortOpen(pInfo->pSortHandle); + int32_t code = tsortOpen(pInfo->pSortHandle); if (code != TSDB_CODE_SUCCESS) { longjmp(pTaskInfo->env, terrno); } @@ -6006,18 +6006,18 @@ static SSDataBlock* doSort(void* param, bool* newgroup) { SSchema* p = blockDataExtractSchema(pInfo->pDataBlock, NULL); int32_t numOfBufPage = pInfo->sortBufSize / pInfo->bufPageSize; - pInfo->pSortHandle = createSortHandle(pInfo->orderInfo, pInfo->nullFirst, SORT_SINGLESOURCE_SORT, pInfo->bufPageSize, + pInfo->pSortHandle = tsortCreateSortHandle(pInfo->orderInfo, pInfo->nullFirst, SORT_SINGLESOURCE_SORT, pInfo->bufPageSize, numOfBufPage, p, pInfo->pDataBlock->info.numOfCols, "GET_TASKID(pTaskInfo)"); tfree(p); - setFetchRawDataFp(pInfo->pSortHandle, loadNextDataBlock); + tsortSetFetchRawDataFp(pInfo->pSortHandle, loadNextDataBlock); SGenericSource* ps = calloc(1, sizeof(SGenericSource)); ps->param = pOperator; - sortAddSource(pInfo->pSortHandle, ps); + tsortAddSource(pInfo->pSortHandle, ps); // TODO set error code; - int32_t code = sortOpen(pInfo->pSortHandle); + int32_t code = tsortOpen(pInfo->pSortHandle); if (code != TSDB_CODE_SUCCESS) { longjmp(pTaskInfo->env, terrno); } diff --git a/source/libs/executor/src/tsort.c b/source/libs/executor/src/tsort.c index 89f555517e..33e95a19b5 100644 --- a/source/libs/executor/src/tsort.c +++ b/source/libs/executor/src/tsort.c @@ -87,7 +87,7 @@ static SSDataBlock* createDataBlock_rv(SSchema* pSchema, int32_t numOfCols) { * @param type * @return */ -SSortHandle* createSortHandle(SArray* pOrderInfo, bool nullFirst, int32_t type, int32_t pageSize, int32_t numOfPages, SSchema* pSchema, int32_t numOfCols, const char* idstr) { +SSortHandle* tsortCreateSortHandle(SArray* pOrderInfo, bool nullFirst, int32_t type, int32_t pageSize, int32_t numOfPages, SSchema* pSchema, int32_t numOfCols, const char* idstr) { SSortHandle* pSortHandle = calloc(1, sizeof(SSortHandle)); pSortHandle->type = type; @@ -99,7 +99,7 @@ SSortHandle* createSortHandle(SArray* pOrderInfo, bool nullFirst, int32_t type, pSortHandle->cmpParam.orderInfo = pOrderInfo; pSortHandle->pDataBlock = createDataBlock_rv(pSchema, numOfCols); - setComparFn(pSortHandle, msortComparFn); + tsortSetComparFp(pSortHandle, msortComparFn); if (idstr != NULL) { pSortHandle->idStr = strdup(idstr); @@ -108,8 +108,8 @@ SSortHandle* createSortHandle(SArray* pOrderInfo, bool nullFirst, int32_t type, return pSortHandle; } -void destroySortHandle(SSortHandle* pSortHandle) { - sortClose(pSortHandle); +void tsortDestroySortHandle(SSortHandle* pSortHandle) { + tsortClose(pSortHandle); if (pSortHandle->pMergeTree != NULL) { tMergeTreeDestroy(pSortHandle->pMergeTree); } @@ -119,7 +119,7 @@ void destroySortHandle(SSortHandle* pSortHandle) { tfree(pSortHandle); } -int32_t sortAddSource(SSortHandle* pSortHandle, void* pSource) { +int32_t tsortAddSource(SSortHandle* pSortHandle, void* pSource) { taosArrayPush(pSortHandle->pOrderedSource, &pSource); } @@ -573,7 +573,7 @@ static int32_t createInitialSortedMultiSources(SSortHandle* pHandle) { return TSDB_CODE_SUCCESS; } -int32_t sortOpen(SSortHandle* pHandle) { +int32_t tsortOpen(SSortHandle* pHandle) { if (pHandle->opened) { return 0; } @@ -611,19 +611,19 @@ int32_t sortOpen(SSortHandle* pHandle) { } } -int32_t sortClose(SSortHandle* pHandle) { +int32_t tsortClose(SSortHandle* pHandle) { // do nothing } -int32_t setFetchRawDataFp(SSortHandle* pHandle, _sort_fetch_block_fn_t fp) { +int32_t tsortSetFetchRawDataFp(SSortHandle* pHandle, _sort_fetch_block_fn_t fp) { pHandle->fetchfp = fp; } -int32_t setComparFn(SSortHandle* pHandle, _sort_merge_compar_fn_t fp) { +int32_t tsortSetComparFp(SSortHandle* pHandle, _sort_merge_compar_fn_t fp) { pHandle->comparFn = fp; } -STupleHandle* sortNextTuple(SSortHandle* pHandle) { +STupleHandle* tsortNextTuple(SSortHandle* pHandle) { if (pHandle->cmpParam.numOfSources == pHandle->numOfCompletedSources) { return NULL; } @@ -669,11 +669,11 @@ STupleHandle* sortNextTuple(SSortHandle* pHandle) { return &pHandle->tupleHandle; } -bool sortIsValueNull(STupleHandle* pVHandle, int32_t colIndex) { +bool tsortIsNullVal(STupleHandle* pVHandle, int32_t colIndex) { return false; } -void* sortGetValue(STupleHandle* pVHandle, int32_t colIndex) { +void* tsortGetValue(STupleHandle* pVHandle, int32_t colIndex) { SColumnInfoData* pColInfo = TARRAY_GET_ELEM(pVHandle->pBlock->pDataBlock, colIndex); return colDataGetData(pColInfo, pVHandle->rowIndex); } diff --git a/source/libs/executor/test/executorUtilTests.cpp b/source/libs/executor/test/executorUtilTests.cpp index 6a023170b2..ec066c95b1 100644 --- a/source/libs/executor/test/executorUtilTests.cpp +++ b/source/libs/executor/test/executorUtilTests.cpp @@ -164,24 +164,24 @@ TEST(testCase, inMem_sort_Test) { taosArrayPush(orderInfo, &oi); SSchema s = {.type = TSDB_DATA_TYPE_INT, .colId = 1, .bytes = 4, }; - SSortHandle* phandle = createSortHandle(orderInfo, false, SORT_SINGLESOURCE_SORT, 1024, 5, &s, 1, "test_abc"); - setFetchRawDataFp(phandle, getSingleColDummyBlock); - sortAddSource(phandle, &numOfRows); + SSortHandle* phandle = tsortCreateSortHandle(orderInfo, false, SORT_SINGLESOURCE_SORT, 1024, 5, &s, 1, "test_abc"); + tsortSetFetchRawDataFp(phandle, getSingleColDummyBlock); + tsortAddSource(phandle, &numOfRows); - int32_t code = sortOpen(phandle); + int32_t code = tsortOpen(phandle); int32_t row = 1; while(1) { - STupleHandle* pTupleHandle = sortNextTuple(phandle); + STupleHandle* pTupleHandle = tsortNextTuple(phandle); if (pTupleHandle == NULL) { break; } - void* v = sortGetValue(pTupleHandle, 0); + void* v = tsortGetValue(pTupleHandle, 0); printf("%d: %d\n", row++, *(int32_t*) v); } - destroySortHandle(phandle); + tsortDestroySortHandle(phandle); } TEST(testCase, external_mem_sort_Test) { @@ -198,8 +198,8 @@ TEST(testCase, external_mem_sort_Test) { taosArrayPush(orderInfo, &oi); SSchema s = {.type = TSDB_DATA_TYPE_INT, .colId = 1, .bytes = 4, }; - SSortHandle* phandle = createSortHandle(orderInfo, false, SORT_SINGLESOURCE_SORT, 1024, 5, &s, 1, "test_abc"); - setFetchRawDataFp(phandle, getSingleColDummyBlock); + SSortHandle* phandle = tsortCreateSortHandle(orderInfo, false, SORT_SINGLESOURCE_SORT, 1024, 5, &s, 1, "test_abc"); + tsortSetFetchRawDataFp(phandle, getSingleColDummyBlock); _info* pInfo = (_info*) calloc(1, sizeof(_info)); pInfo->startVal = 100000; @@ -209,22 +209,22 @@ TEST(testCase, external_mem_sort_Test) { SGenericSource* ps = static_cast(calloc(1, sizeof(SGenericSource))); ps->param = pInfo; - sortAddSource(phandle, ps); + tsortAddSource(phandle, ps); - int32_t code = sortOpen(phandle); + int32_t code = tsortOpen(phandle); int32_t row = 1; while(1) { - STupleHandle* pTupleHandle = sortNextTuple(phandle); + STupleHandle* pTupleHandle = tsortNextTuple(phandle); if (pTupleHandle == NULL) { break; } - void* v = sortGetValue(pTupleHandle, 0); + void* v = tsortGetValue(pTupleHandle, 0); printf("%d: %d\n", row++, *(int32_t*) v); } - destroySortHandle(phandle); + tsortDestroySortHandle(phandle); } TEST(testCase, ordered_merge_sort_Test) { @@ -242,9 +242,9 @@ TEST(testCase, ordered_merge_sort_Test) { taosArrayPush(orderInfo, &oi); SSchema s = {.type = TSDB_DATA_TYPE_INT, .colId = 1, .bytes = 4}; - SSortHandle* phandle = createSortHandle(orderInfo, false, SORT_MULTISOURCE_MERGE, 1024, 5, &s, 1,"test_abc"); - setFetchRawDataFp(phandle, getSingleColDummyBlock); - setComparFn(phandle, docomp); + SSortHandle* phandle = tsortCreateSortHandle(orderInfo, false, SORT_MULTISOURCE_MERGE, 1024, 5, &s, 1,"test_abc"); + tsortSetFetchRawDataFp(phandle, getSingleColDummyBlock); + tsortSetComparFp(phandle, docomp); for(int32_t i = 0; i < 10; ++i) { SGenericSource* p = static_cast(calloc(1, sizeof(SGenericSource))); @@ -254,23 +254,23 @@ TEST(testCase, ordered_merge_sort_Test) { c->startVal = 0; p->param = c; - sortAddSource(phandle, p); + tsortAddSource(phandle, p); } - int32_t code = sortOpen(phandle); + int32_t code = tsortOpen(phandle); int32_t row = 1; while(1) { - STupleHandle* pTupleHandle = sortNextTuple(phandle); + STupleHandle* pTupleHandle = tsortNextTuple(phandle); if (pTupleHandle == NULL) { break; } - void* v = sortGetValue(pTupleHandle, 0); + void* v = tsortGetValue(pTupleHandle, 0); printf("%d: %d\n", row++, *(int32_t*) v); } - destroySortHandle(phandle); + tsortDestroySortHandle(phandle); } #endif From af0180f2163c42cf6587af34cc021307b3a32976 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 23 Feb 2022 15:10:15 +0800 Subject: [PATCH 13/25] [td-13039] add linear hash. --- include/util/tarray.h | 4 +- include/util/tpagedbuf.h | 19 +- source/libs/executor/inc/tlinearhash.h | 44 +++ source/libs/executor/src/executorimpl.c | 6 +- source/libs/executor/src/tlinearhash.c | 350 ++++++++++++++++++ source/libs/executor/src/tsort.c | 8 +- source/libs/executor/test/executorTests.cpp | 4 +- source/libs/executor/test/lhashTests.cpp | 65 ++++ .../{executorUtilTests.cpp => sortTests.cpp} | 0 source/libs/function/src/tpercentile.c | 4 +- source/util/src/tpagedbuf.c | 60 +-- source/util/test/pageBufferTest.cpp | 34 +- 12 files changed, 542 insertions(+), 56 deletions(-) create mode 100644 source/libs/executor/inc/tlinearhash.h create mode 100644 source/libs/executor/src/tlinearhash.c create mode 100644 source/libs/executor/test/lhashTests.cpp rename source/libs/executor/test/{executorUtilTests.cpp => sortTests.cpp} (100%) diff --git a/include/util/tarray.h b/include/util/tarray.h index 6d6120a49b..117e60dede 100644 --- a/include/util/tarray.h +++ b/include/util/tarray.h @@ -43,8 +43,8 @@ extern "C" { typedef struct SArray { size_t size; - size_t capacity; - size_t elemSize; + uint32_t capacity; + uint32_t elemSize; void* pData; } SArray; diff --git a/include/util/tpagedbuf.h b/include/util/tpagedbuf.h index c1c246dd64..d9e233f8bb 100644 --- a/include/util/tpagedbuf.h +++ b/include/util/tpagedbuf.h @@ -30,7 +30,6 @@ typedef struct SPageInfo SPageInfo; typedef struct SDiskbasedBuf SDiskbasedBuf; #define DEFAULT_INTERN_BUF_PAGE_SIZE (1024L) // in bytes -#define DEFAULT_PAGE_SIZE (16384L) typedef struct SFilePage { int64_t num; @@ -64,7 +63,7 @@ int32_t createDiskbasedBuf(SDiskbasedBuf** pBuf, int32_t pagesize, int32_t inMem * @param pageId * @return */ -SFilePage* getNewDataBuf(SDiskbasedBuf* pBuf, int32_t groupId, int32_t* pageId); +void* getNewBufPage(SDiskbasedBuf* pBuf, int32_t groupId, int32_t* pageId); /** * @@ -80,7 +79,7 @@ SIDList getDataBufPagesIdList(SDiskbasedBuf* pBuf, int32_t groupId); * @param id * @return */ -SFilePage* getBufPage(SDiskbasedBuf* pBuf, int32_t id); +void* getBufPage(SDiskbasedBuf* pBuf, int32_t id); /** * release the referenced buf pages @@ -151,19 +150,27 @@ bool isAllDataInMemBuf(const SDiskbasedBuf* pBuf); * @param pPageInfo * @param dirty */ -void setBufPageDirty(SFilePage* pPageInfo, bool dirty); +void setBufPageDirty(void* pPageInfo, bool dirty); /** * Print the statistics when closing this buffer * @param pBuf */ -void setPrintStatis(SDiskbasedBuf* pBuf); +void dBufSetPrintInfo(SDiskbasedBuf* pBuf); /** - * return buf statistics. + * Return buf statistics. + * @param pBuf + * @return */ SDiskbasedBufStatis getDBufStatis(const SDiskbasedBuf* pBuf); +/** + * Print the buffer statistics information + * @param pBuf + */ +void dBufPrintStatis(const SDiskbasedBuf* pBuf); + #ifdef __cplusplus } #endif diff --git a/source/libs/executor/inc/tlinearhash.h b/source/libs/executor/inc/tlinearhash.h new file mode 100644 index 0000000000..a419058216 --- /dev/null +++ b/source/libs/executor/inc/tlinearhash.h @@ -0,0 +1,44 @@ +/* + * Copyright (c) 2019 TAOS Data, Inc. + * + * This program is free software: you can use, redistribute, and/or modify + * it under the terms of the GNU Affero General Public License, version 3 + * or later ("AGPL"), as published by the Free Software Foundation. + * + * This program is distributed in the hope that it will be useful, but WITHOUT + * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or + * FITNESS FOR A PARTICULAR PURPOSE. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + */ + +#ifndef TDENGINE_TLINEARHASH_H +#define TDENGINE_TLINEARHASH_H + +#ifdef __cplusplus +extern "C" { +#endif + +#include "thash.h" + +enum { + LINEAR_HASH_STATIS = 0x1, + LINEAR_HASH_DATA = 0x2, +}; + +typedef struct SLHashObj SLHashObj; + +SLHashObj* tHashInit(int32_t inMemPages, int32_t pageSize, _hash_fn_t fn, int32_t numOfTuplePerPage); +void* tHashCleanup(SLHashObj* pHashObj); + +int32_t tHashPut(SLHashObj* pHashObj, const void *key, size_t keyLen, void *data, size_t size); +char* tHashGet(SLHashObj* pHashObj, const void *key, size_t keyLen); +int32_t tHashRemove(SLHashObj* pHashObj, const void *key, size_t keyLen); + +void tHashPrint(const SLHashObj* pHashObj, int32_t type); + +#ifdef __cplusplus +} +#endif +#endif // TDENGINE_TLINEARHASH_H diff --git a/source/libs/executor/src/executorimpl.c b/source/libs/executor/src/executorimpl.c index f2d98ea86c..6f1a05805a 100644 --- a/source/libs/executor/src/executorimpl.c +++ b/source/libs/executor/src/executorimpl.c @@ -705,7 +705,7 @@ static int32_t addNewWindowResultBuf(SResultRow *pWindowRes, SDiskbasedBuf *pRes SIDList list = getDataBufPagesIdList(pResultBuf, tid); if (taosArrayGetSize(list) == 0) { - pData = getNewDataBuf(pResultBuf, tid, &pageId); + pData = getNewBufPage(pResultBuf, tid, &pageId); } else { SPageInfo* pi = getLastPageInfo(list); pData = getBufPage(pResultBuf, getPageId(pi)); @@ -714,7 +714,7 @@ static int32_t addNewWindowResultBuf(SResultRow *pWindowRes, SDiskbasedBuf *pRes if (pData->num + size > getBufPageSize(pResultBuf)) { // release current page first, and prepare the next one releaseBufPageInfo(pResultBuf, pi); - pData = getNewDataBuf(pResultBuf, tid, &pageId); + pData = getNewBufPage(pResultBuf, tid, &pageId); if (pData != NULL) { assert(pData->num == 0); // number of elements must be 0 for new allocated buffer } @@ -4603,7 +4603,7 @@ int32_t doInitQInfo(SQInfo* pQInfo, STSBuf* pTsBuf, void* tsdb, void* sourceOptr tsBufSetTraverseOrder(pRuntimeEnv->pTsBuf, order); } - int32_t ps = DEFAULT_PAGE_SIZE; + int32_t ps = 4096; getIntermediateBufInfo(pRuntimeEnv, &ps, &pQueryAttr->intermediateResultRowSize); int32_t TENMB = 1024*1024*10; diff --git a/source/libs/executor/src/tlinearhash.c b/source/libs/executor/src/tlinearhash.c new file mode 100644 index 0000000000..b644801a6e --- /dev/null +++ b/source/libs/executor/src/tlinearhash.c @@ -0,0 +1,350 @@ +/* + * 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 "tlinearhash.h" +#include "tcfg.h" +#include "taoserror.h" +#include "tpagedbuf.h" + +#define LHASH_CAP_RATIO 0.85 + +// Always located in memory +typedef struct SLHashBucket { + SArray *pPageIdList; + int32_t size; // the number of element in this entry +} SLHashBucket; + +typedef struct SLHashObj { + SDiskbasedBuf *pBuf; + _hash_fn_t hashFn; + int32_t tuplesPerPage; + SLHashBucket **pBucket; // entry list + int32_t numOfAlloc; // number of allocated bucket ptr slot + int32_t bits; // the number of bits used in hash + int32_t numOfBuckets; // the number of buckets + int64_t size; // the number of total items +} SLHashObj; + +/** + * the data struct for each hash node + * +-----------+-------+--------+ + * | SLHashNode| key | data | + * +-----------+-------+--------+ + */ +typedef struct SLHashNode { + int32_t keyLen; + int32_t dataLen; +} SLHashNode; + +#define GET_LHASH_NODE_KEY(_n) (((char*)(_n)) + sizeof(SLHashNode)) +#define GET_LHASH_NODE_DATA(_n) ((char*)(_n) + sizeof(SLHashNode) + (_n)->keyLen) +#define GET_LHASH_NODE_LEN(_n) (sizeof(SLHashNode) + ((SLHashNode*)(_n))->keyLen + ((SLHashNode*)(_n))->dataLen) + +static int32_t doAddNewBucket(SLHashObj* pHashObj); + +static int32_t doGetBucketIdFromHashVal(int32_t hashv, int32_t bits) { + return hashv & ((1ul << (bits)) - 1); +} + +static int32_t doGetAlternativeBucketId(int32_t bucketId, int32_t bits, int32_t numOfBuckets) { + int32_t v = bucketId - (1ul << (bits - 1)); + + ASSERT(v < numOfBuckets); + return v; +} + +SLHashObj* tHashInit(int32_t inMemPages, int32_t pageSize, _hash_fn_t fn, int32_t numOfTuplePerPage) { + SLHashObj* pHashObj = calloc(1, sizeof(SLHashObj)); + if (pHashObj == NULL) { + terrno = TSDB_CODE_OUT_OF_MEMORY; + return NULL; + } + + int32_t code = createDiskbasedBuf(&pHashObj->pBuf, pageSize, inMemPages * pageSize, 0, "/tmp"); + if (code != 0) { + terrno = code; + return NULL; + } + + /** + * The number of bits in the hash value, which is used to decide the exact bucket where the object should be located in. + * The initial value is 0. + */ + pHashObj->bits = 0; + pHashObj->hashFn = fn; + pHashObj->tuplesPerPage = numOfTuplePerPage; + + pHashObj->numOfAlloc = 4; // initial allocated array list + pHashObj->pBucket = calloc(pHashObj->numOfAlloc, POINTER_BYTES); + + code = doAddNewBucket(pHashObj); + if (code != TSDB_CODE_SUCCESS) { + destroyDiskbasedBuf(pHashObj->pBuf); + tfree(pHashObj); + terrno = code; + return NULL; + } + + return pHashObj; +} + +void* tHashCleanup(SLHashObj* pHashObj) { + destroyDiskbasedBuf(pHashObj->pBuf); + for(int32_t i = 0; i < pHashObj->numOfBuckets; ++i) { + taosArrayDestroy(pHashObj->pBucket[i]->pPageIdList); + tfree(pHashObj->pBucket[i]); + } + + tfree(pHashObj->pBucket); + tfree(pHashObj); + return NULL; +} + +static void doCopyObject(char* p, const void* key, int32_t keyLen, const void* data, int32_t size) { + *(int32_t*) p = keyLen; + p += sizeof(int32_t); + *(int32_t*) p = size; + p += sizeof(int32_t); + + memcpy(p, key, keyLen); + p += keyLen; + + memcpy(p, data, size); +} + +static int32_t doAddToBucket(SLHashObj* pHashObj, SLHashBucket* pBucket, int32_t index, const void* key, int32_t keyLen, + const void* data, int32_t size) { + int32_t pageId = *(int32_t*)taosArrayGetLast(pBucket->pPageIdList); + + SFilePage* pPage = getBufPage(pHashObj->pBuf, pageId); + ASSERT (pPage != NULL); + + // put to current buf page + size_t nodeSize = sizeof(SLHashNode) + keyLen + size; + ASSERT(nodeSize <= getBufPageSize(pHashObj->pBuf)); + + if (pPage->num + nodeSize > getBufPageSize(pHashObj->pBuf)) { + releaseBufPage(pHashObj->pBuf, pPage); + + // allocate the overflow buffer page to hold this k/v. + int32_t newPageId = -1; + SFilePage* pNewPage = getNewBufPage(pHashObj->pBuf, 0, &newPageId); + if (pNewPage == 0) { + // TODO handle error + } + + taosArrayPush(pBucket->pPageIdList, &newPageId); + + doCopyObject(pNewPage->data, key, keyLen, data, size); + pNewPage->num = nodeSize; + + setBufPageDirty(pNewPage, true); + releaseBufPage(pHashObj->pBuf, pNewPage); + } else { + char* p = pPage->data + pPage->num; + doCopyObject(p, key, keyLen, data, size); + pPage->num += nodeSize; + setBufPageDirty(pPage, true); + releaseBufPage(pHashObj->pBuf, pPage); + } + + pBucket->size += 1; +// printf("===> add to bucket:0x%x, num:%d, key:%d\n", index, pBucket->size, *(int*) key); +} + +// TODO merge the fragments on multiple pages to recycle the empty disk page ASAP +static void doRemoveFromBucket(SFilePage* pPage, SLHashNode* pNode, SLHashBucket* pBucket) { + ASSERT(pPage != NULL && pNode != NULL); + + int32_t len = GET_LHASH_NODE_LEN(pNode); + char* p = (char*) pNode + len; + + char* pEnd = pPage->data + pPage->num; + memmove(pNode, p, (pEnd - p)); + + pPage->num -= len; + if (pPage->num == 0) { + // this page is empty, could be recycle in the future. + } + + setBufPageDirty(pPage, true); + pBucket->size -= 1; +} + +static int32_t doAddNewBucket(SLHashObj* pHashObj) { + if (pHashObj->numOfBuckets + 1 > pHashObj->numOfAlloc) { + int32_t newLen = pHashObj->numOfAlloc * 1.25; + if (newLen == pHashObj->numOfAlloc) { + newLen += 4; + } + + char* p = realloc(pHashObj->pBucket, POINTER_BYTES * newLen); + if (p == NULL) { + return TSDB_CODE_OUT_OF_MEMORY; + } + + memset(p + POINTER_BYTES * pHashObj->numOfBuckets, 0, newLen - pHashObj->numOfBuckets); + pHashObj->pBucket = (SLHashBucket**) p; + pHashObj->numOfAlloc = newLen; + } + + SLHashBucket* pBucket = calloc(1, sizeof(SLHashBucket)); + pHashObj->pBucket[pHashObj->numOfBuckets] = pBucket; + + pBucket->pPageIdList = taosArrayInit(2, sizeof(int32_t)); + if (pBucket->pPageIdList == NULL || pBucket == NULL) { + return TSDB_CODE_OUT_OF_MEMORY; + } + + int32_t pageId = -1; + SFilePage* p = getNewBufPage(pHashObj->pBuf, 0, &pageId); + releaseBufPage(pHashObj->pBuf, p); + + taosArrayPush(pBucket->pPageIdList, &pageId); + + pHashObj->numOfBuckets += 1; +// printf("---------------add new bucket, id:0x%x, total:%d\n", pHashObj->numOfBuckets - 1, pHashObj->numOfBuckets); + return TSDB_CODE_SUCCESS; +} + +int32_t tHashPut(SLHashObj* pHashObj, const void *key, size_t keyLen, void *data, size_t size) { + ASSERT(pHashObj != NULL && key != NULL); + + if (pHashObj->bits == 0) { + SLHashBucket* pBucket = pHashObj->pBucket[0]; + doAddToBucket(pHashObj, pBucket, 0, key, keyLen, data, size); + } else { + int32_t hashVal = pHashObj->hashFn(key, keyLen); + int32_t v = doGetBucketIdFromHashVal(hashVal, pHashObj->bits); + + if (pHashObj->numOfBuckets > v) { + SLHashBucket* pBucket = pHashObj->pBucket[v]; + + // TODO check return code + doAddToBucket(pHashObj, pBucket, v, key, keyLen, data, size); + } else { // no matched bucket exists, find the candidate bucket + int32_t bucketId = doGetAlternativeBucketId(v, pHashObj->bits, pHashObj->numOfBuckets); +// printf("bucketId: 0x%x not exists, put it into 0x%x instead\n", v, bucketId); + + SLHashBucket* pBucket = pHashObj->pBucket[bucketId]; + doAddToBucket(pHashObj, pBucket, bucketId, key, keyLen, data, size); + } + } + + pHashObj->size += 1; + + // Too many records, needs to bucket split + if ((pHashObj->numOfBuckets * LHASH_CAP_RATIO * pHashObj->tuplesPerPage) < pHashObj->size) { + int32_t newBucketId = pHashObj->numOfBuckets; + + int32_t code = doAddNewBucket(pHashObj); + int32_t numOfBits = ceil(log(pHashObj->numOfBuckets) / log(2)); + if (numOfBits > pHashObj->bits) { +// printf("extend the bits from %d to %d, new bucket:%d\n", pHashObj->bits, numOfBits, newBucketId); + + ASSERT(numOfBits == pHashObj->bits + 1); + pHashObj->bits = numOfBits; + } + + int32_t splitBucketId = (1ul << (pHashObj->bits - 1)) ^ newBucketId; + + // load all data in this bucket and check if the data needs to relocated into the new bucket + SLHashBucket* pBucket = pHashObj->pBucket[splitBucketId]; +// printf("split %d items' bucket:0x%x to new bucket:0x%x\n", pBucket->size, splitBucketId, newBucketId); + + for (int32_t i = 0; i < taosArrayGetSize(pBucket->pPageIdList); ++i) { + int32_t pageId = *(int32_t*)taosArrayGet(pBucket->pPageIdList, i); + SFilePage* p = getBufPage(pHashObj->pBuf, pageId); + + char* pStart = p->data; + while (pStart - p->data < p->num) { + SLHashNode* pNode = (SLHashNode*)pStart; + + char* k = GET_LHASH_NODE_KEY(pNode); + int32_t hashv = pHashObj->hashFn(k, pNode->keyLen); + + int32_t v1 = hashv & ((1ul << (pHashObj->bits)) - 1); + if (v1 != splitBucketId) { // place it into the new bucket + ASSERT(v1 == newBucketId); +// printf("move key:%d to 0x%x bucket, remain items:%d\n", *(int32_t*)k, v1, pBucket->size - 1); + + SLHashBucket* pNewBucket = pHashObj->pBucket[newBucketId]; + doAddToBucket(pHashObj, pNewBucket, newBucketId, (void*)GET_LHASH_NODE_KEY(pNode), pNode->keyLen, + GET_LHASH_NODE_KEY(pNode), pNode->dataLen); + doRemoveFromBucket(p, pNode, pBucket); + } else { +// printf("check key:%d, located into: %d, skip it\n", *(int*) k, v1); + + int32_t nodeSize = GET_LHASH_NODE_LEN(pStart); + pStart += nodeSize; + } + } + releaseBufPage(pHashObj->pBuf, p); + } + } +} + +char* tHashGet(SLHashObj* pHashObj, const void *key, size_t keyLen) { + ASSERT(pHashObj != NULL && key != NULL && keyLen > 0); + int32_t hashv = pHashObj->hashFn(key, keyLen); + + int32_t bucketId = doGetBucketIdFromHashVal(hashv, pHashObj->bits); + if (bucketId >= pHashObj->numOfBuckets) { + bucketId = doGetAlternativeBucketId(bucketId, pHashObj->bits, pHashObj->numOfBuckets); + } + + SLHashBucket* pBucket = pHashObj->pBucket[bucketId]; + for (int32_t i = 0; i < taosArrayGetSize(pBucket->pPageIdList); ++i) { + int32_t pageId = *(int32_t*)taosArrayGet(pBucket->pPageIdList, i); + SFilePage* p = getBufPage(pHashObj->pBuf, pageId); + + char* pStart = p->data; + while (pStart - p->data < p->num) { + SLHashNode* pNode = (SLHashNode*)pStart; + + char* k = GET_LHASH_NODE_KEY(pNode); + if (pNode->keyLen == keyLen && (memcmp(key, k, keyLen) == 0)) { + releaseBufPage(pHashObj->pBuf, p); + return GET_LHASH_NODE_DATA(pNode); + } else { + pStart += GET_LHASH_NODE_LEN(pStart); + } + } + + releaseBufPage(pHashObj->pBuf, p); + } + + return NULL; +} + +int32_t tHashRemove(SLHashObj* pHashObj, const void *key, size_t keyLen) { + +} + +void tHashPrint(const SLHashObj* pHashObj, int32_t type) { + printf("==================== linear hash ====================\n"); + printf("total bucket:%d, size:%ld, ratio:%.2f\n", pHashObj->numOfBuckets, pHashObj->size, LHASH_CAP_RATIO); + + dBufSetPrintInfo(pHashObj->pBuf); + + if (type == LINEAR_HASH_DATA) { + for (int32_t i = 0; i < pHashObj->numOfBuckets; ++i) { +// printf("bucket: 0x%x, obj:%d, page:%d\n", i, pHashObj->pBucket[i]->size, +// (int)taosArrayGetSize(pHashObj->pBucket[i]->pPageIdList)); + } + } else { + dBufPrintStatis(pHashObj->pBuf); + } +} \ No newline at end of file diff --git a/source/libs/executor/src/tsort.c b/source/libs/executor/src/tsort.c index 33e95a19b5..6e7b6b4659 100644 --- a/source/libs/executor/src/tsort.c +++ b/source/libs/executor/src/tsort.c @@ -147,7 +147,7 @@ static int32_t doAddToBuf(SSDataBlock* pDataBlock, SSortHandle* pHandle) { if (pHandle->pBuf == NULL) { int32_t code = createDiskbasedBuf(&pHandle->pBuf, pHandle->pageSize, pHandle->numOfPages * pHandle->pageSize, 0, "/tmp"); - setPrintStatis(pHandle->pBuf); + dBufSetPrintInfo(pHandle->pBuf); if (code != TSDB_CODE_SUCCESS) { return code; } @@ -162,7 +162,7 @@ static int32_t doAddToBuf(SSDataBlock* pDataBlock, SSortHandle* pHandle) { } int32_t pageId = -1; - SFilePage* pPage = getNewDataBuf(pHandle->pBuf, pHandle->sourceId, &pageId); + SFilePage* pPage = getNewBufPage(pHandle->pBuf, pHandle->sourceId, &pageId); if (pPage == NULL) { return terrno; } @@ -211,7 +211,7 @@ static int32_t sortComparInit(SMsortComparParam* cmpParam, SArray* pSources, int // multi-pass internal merge sort is required if (pHandle->pBuf == NULL) { code = createDiskbasedBuf(&pHandle->pBuf, pHandle->pageSize, pHandle->numOfPages * pHandle->pageSize, 0, "/tmp"); - setPrintStatis(pHandle->pBuf); + dBufSetPrintInfo(pHandle->pBuf); if (code != TSDB_CODE_SUCCESS) { return code; } @@ -462,7 +462,7 @@ static int32_t doInternalMergeSort(SSortHandle* pHandle) { } int32_t pageId = -1; - SFilePage* pPage = getNewDataBuf(pHandle->pBuf, pHandle->sourceId, &pageId); + SFilePage* pPage = getNewBufPage(pHandle->pBuf, pHandle->sourceId, &pageId); if (pPage == NULL) { return terrno; } diff --git a/source/libs/executor/test/executorTests.cpp b/source/libs/executor/test/executorTests.cpp index bf3f76b94e..c0fb899a2d 100644 --- a/source/libs/executor/test/executorTests.cpp +++ b/source/libs/executor/test/executorTests.cpp @@ -357,7 +357,7 @@ TEST(testCase, external_sort_Test) { taosArrayDestroy(pOrderVal); } -#endif + TEST(testCase, sorted_merge_Test) { srand(time(NULL)); @@ -430,4 +430,6 @@ TEST(testCase, sorted_merge_Test) { taosArrayDestroy(pExprInfo); taosArrayDestroy(pOrderVal); } + +#endif #pragma GCC diagnostic pop diff --git a/source/libs/executor/test/lhashTests.cpp b/source/libs/executor/test/lhashTests.cpp new file mode 100644 index 0000000000..be847d2959 --- /dev/null +++ b/source/libs/executor/test/lhashTests.cpp @@ -0,0 +1,65 @@ +/* + * 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 "executorimpl.h" +#include "tlinearhash.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" +#include "os.h" + +TEST(testCase, linear_hash_Tests) { + srand(time(NULL)); + + _hash_fn_t fn = taosGetDefaultHashFunction(TSDB_DATA_TYPE_INT); +#if 1 + SLHashObj* pHashObj = tHashInit(100, 64 + 8, fn, 4); + for(int32_t i = 0; i < 5000; ++i) { + tHashPut(pHashObj, &i, sizeof(i), &i, sizeof(i)); + } + + tHashPrint(pHashObj, LINEAR_HASH_STATIS); + + for(int32_t i = 0; i < 100; ++i) { + char* v = tHashGet(pHashObj, &i, sizeof(i)); + if (v != NULL) { +// printf("find value: %d, key:%d\n", *(int32_t*) v, i); + } else { + printf("failed to found key:%d in hash\n", i); + } + } + + tHashPrint(pHashObj, LINEAR_HASH_DATA); + tHashCleanup(pHashObj); +#endif + +#if 0 + SHashObj* pHashObj = taosHashInit(1000, fn, false, HASH_NO_LOCK); + for(int32_t i = 0; i < 500000; ++i) { + taosHashPut(pHashObj, &i, sizeof(i), &i, sizeof(i)); + } + + for(int32_t i = 0; i < 10000; ++i) { + void* v = taosHashGet(pHashObj, &i, sizeof(i)); + } + taosHashCleanup(pHashObj); +#endif + +} \ No newline at end of file diff --git a/source/libs/executor/test/executorUtilTests.cpp b/source/libs/executor/test/sortTests.cpp similarity index 100% rename from source/libs/executor/test/executorUtilTests.cpp rename to source/libs/executor/test/sortTests.cpp diff --git a/source/libs/function/src/tpercentile.c b/source/libs/function/src/tpercentile.c index c6ab125362..eab05ad039 100644 --- a/source/libs/function/src/tpercentile.c +++ b/source/libs/function/src/tpercentile.c @@ -221,7 +221,7 @@ tMemBucket *tMemBucketCreate(int16_t nElemSize, int16_t dataType, double minval, } pBucket->numOfSlots = DEFAULT_NUM_OF_SLOT; - pBucket->bufPageSize = DEFAULT_PAGE_SIZE * 4; // 4k per page + pBucket->bufPageSize = 16384 * 4; // 16k per page pBucket->type = dataType; pBucket->bytes = nElemSize; @@ -347,7 +347,7 @@ int32_t tMemBucketPut(tMemBucket *pBucket, const void *data, size_t size) { pSlot->info.data = NULL; } - pSlot->info.data = getNewDataBuf(pBucket->pBuffer, groupId, &pageId); + pSlot->info.data = getNewBufPage(pBucket->pBuffer, groupId, &pageId); pSlot->info.pageId = pageId; } diff --git a/source/util/src/tpagedbuf.c b/source/util/src/tpagedbuf.c index 212b3f3067..cc3d1747a1 100644 --- a/source/util/src/tpagedbuf.c +++ b/source/util/src/tpagedbuf.c @@ -18,7 +18,7 @@ typedef struct SPageDiskInfo { int32_t length; } SPageDiskInfo; -typedef struct SPageInfo { +struct SPageInfo { SListNode* pn; // point to list node void* pData; int64_t offset; @@ -26,9 +26,9 @@ typedef struct SPageInfo { int32_t length:30; bool used:1; // set current page is in used bool dirty:1; // set current buffer page is dirty or not -} SPageInfo; +}; -typedef struct SDiskbasedBuf { +struct SDiskbasedBuf { int32_t numOfPages; int64_t totalBufSize; uint64_t fileSize; // disk file size @@ -49,9 +49,7 @@ typedef struct SDiskbasedBuf { uint64_t qId; // for debug purpose bool printStatis; // Print statistics info when closing this buffer. SDiskbasedBufStatis statis; -} SDiskbasedBuf; - -static void printStatisData(const SDiskbasedBuf* pBuf); +}; int32_t createDiskbasedBuf(SDiskbasedBuf** pBuf, int32_t pagesize, int32_t inMemBufSize, uint64_t qId, const char* dir) { *pBuf = calloc(1, sizeof(SDiskbasedBuf)); @@ -121,7 +119,7 @@ static char* doDecompressData(void* data, int32_t srcSize, int32_t *dst, SDiskba return data; } - *dst = tsDecompressString(data, srcSize, 1, pBuf->assistBuf, pBuf->pageSize+sizeof(SFilePage), ONE_STAGE_COMP, NULL, 0); + *dst = tsDecompressString(data, srcSize, 1, pBuf->assistBuf, pBuf->pageSize, ONE_STAGE_COMP, NULL, 0); if (*dst > 0) { memcpy(data, pBuf->assistBuf, *dst); } @@ -151,14 +149,23 @@ static uint64_t allocatePositionInFile(SDiskbasedBuf* pBuf, size_t size) { } } +/** + * +--------------------------+-------------------+--------------+ + * | PTR to SPageInfo (8bytes)| Payload (PageSize)| 2 Extra Bytes| + * +--------------------------+-------------------+--------------+ + * @param pBuf + * @param pg + * @return + */ static char* doFlushPageToDisk(SDiskbasedBuf* pBuf, SPageInfo* pg) { assert(!pg->used && pg->pData != NULL); int32_t size = -1; char* t = NULL; if (pg->offset == -1 || pg->dirty) { - SFilePage* pPage = (SFilePage*) GET_DATA_PAYLOAD(pg); - t = doCompressData(pPage->data, pBuf->pageSize, &size, pBuf); + void* payload = GET_DATA_PAYLOAD(pg); + t = doCompressData(payload, pBuf->pageSize, &size, pBuf); + assert(size >= 0); } // this page is flushed to disk for the first time @@ -217,10 +224,14 @@ static char* doFlushPageToDisk(SDiskbasedBuf* pBuf, SPageInfo* pg) { pBuf->statis.flushBytes += size; pBuf->statis.flushPages += 1; + } else { + size = pg->length; } + assert(size >= 0); + char* pDataBuf = pg->pData; - memset(pDataBuf, 0, pBuf->pageSize + sizeof(SFilePage)); + memset(pDataBuf, 0, pBuf->pageSize); pg->pData = NULL; // this means the data is not in buffer pg->length = size; @@ -251,8 +262,8 @@ static int32_t loadPageFromDisk(SDiskbasedBuf* pBuf, SPageInfo* pg) { return ret; } - SFilePage* pPage = (SFilePage*) GET_DATA_PAYLOAD(pg); - ret = (int32_t)fread(pPage->data, 1, pg->length, pBuf->file); + void* pPage = (void*) GET_DATA_PAYLOAD(pg); + ret = (int32_t)fread(pPage, 1, pg->length, pBuf->file); if (ret != pg->length) { ret = TAOS_SYSTEM_ERROR(errno); return ret; @@ -262,7 +273,7 @@ static int32_t loadPageFromDisk(SDiskbasedBuf* pBuf, SPageInfo* pg) { pBuf->statis.loadPages += 1; int32_t fullSize = 0; - doDecompressData(pPage->data, pg->length, &fullSize, pBuf); + doDecompressData(pPage, pg->length, &fullSize, pBuf); return 0; } @@ -288,7 +299,7 @@ static SPageInfo* registerPage(SDiskbasedBuf* pBuf, int32_t groupId, int32_t pag pBuf->numOfPages += 1; - SPageInfo* ppi = malloc(sizeof(SPageInfo));//{ .info = PAGE_INFO_INITIALIZER, .pageId = pageId, .pn = NULL}; + SPageInfo* ppi = malloc(sizeof(SPageInfo)); ppi->pageId = pageId; ppi->pData = NULL; @@ -302,6 +313,7 @@ static SPageInfo* registerPage(SDiskbasedBuf* pBuf, int32_t groupId, int32_t pag static SListNode* getEldestUnrefedPage(SDiskbasedBuf* pBuf) { SListIter iter = {0}; + tdListInitIter(pBuf->lruList, &iter, TD_LIST_BACKWARD); SListNode* pn = NULL; @@ -313,6 +325,8 @@ static SListNode* getEldestUnrefedPage(SDiskbasedBuf* pBuf) { if (!pageInfo->used) { break; + } else { + printf("page %d is used, dirty:%d\n", pageInfo->pageId, pageInfo->dirty); } } @@ -360,10 +374,10 @@ static void lruListMoveToFront(SList *pList, SPageInfo* pi) { } static FORCE_INLINE size_t getAllocPageSize(int32_t pageSize) { - return pageSize + POINTER_BYTES + 2 + sizeof(SFilePage); + return pageSize + POINTER_BYTES + 2; } -SFilePage* getNewDataBuf(SDiskbasedBuf* pBuf, int32_t groupId, int32_t* pageId) { +void* getNewBufPage(SDiskbasedBuf* pBuf, int32_t groupId, int32_t* pageId) { pBuf->statis.getPages += 1; char* availablePage = NULL; @@ -379,6 +393,10 @@ SFilePage* getNewDataBuf(SDiskbasedBuf* pBuf, int32_t groupId, int32_t* pageId) // register new id in this group *pageId = (++pBuf->allocateId); + if (*pageId == 11) { + printf("page is allocated, id:%d\n", *pageId); + } + // register page id info SPageInfo* pi = registerPage(pBuf, groupId, *pageId); @@ -404,7 +422,7 @@ SFilePage* getNewDataBuf(SDiskbasedBuf* pBuf, int32_t groupId, int32_t* pageId) return (void *)(GET_DATA_PAYLOAD(pi)); } -SFilePage* getBufPage(SDiskbasedBuf* pBuf, int32_t id) { +void* getBufPage(SDiskbasedBuf* pBuf, int32_t id) { assert(pBuf != NULL && id >= 0); pBuf->statis.getPages += 1; @@ -493,7 +511,7 @@ void destroyDiskbasedBuf(SDiskbasedBuf* pBuf) { return; } - printStatisData(pBuf); + dBufPrintStatis(pBuf); if (pBuf->file != NULL) { uDebug("Paged buffer closed, total:%.2f Kb (%d Pages), inmem size:%.2f Kb (%d Pages), file size:%.2f Kb, page size:%.2f Kb, %"PRIx64"\n", @@ -561,7 +579,7 @@ bool isAllDataInMemBuf(const SDiskbasedBuf* pBuf) { return pBuf->fileSize == 0; } -void setBufPageDirty(SFilePage* pPage, bool dirty) { +void setBufPageDirty(void* pPage, bool dirty) { int32_t offset = offsetof(SPageInfo, pData); char* p = (char*)pPage - offset; @@ -569,7 +587,7 @@ void setBufPageDirty(SFilePage* pPage, bool dirty) { ppi->dirty = dirty; } -void setPrintStatis(SDiskbasedBuf* pBuf) { +void dBufSetPrintInfo(SDiskbasedBuf* pBuf) { pBuf->printStatis = true; } @@ -577,7 +595,7 @@ SDiskbasedBufStatis getDBufStatis(const SDiskbasedBuf* pBuf) { return pBuf->statis; } -void printStatisData(const SDiskbasedBuf* pBuf) { +void dBufPrintStatis(const SDiskbasedBuf* pBuf) { if (!pBuf->printStatis) { return; } diff --git a/source/util/test/pageBufferTest.cpp b/source/util/test/pageBufferTest.cpp index d310ff450b..e86fcf8653 100644 --- a/source/util/test/pageBufferTest.cpp +++ b/source/util/test/pageBufferTest.cpp @@ -18,7 +18,7 @@ void simpleTest() { int32_t pageId = 0; int32_t groupId = 0; - SFilePage* pBufPage = getNewDataBuf(pResultBuf, groupId, &pageId); + SFilePage* pBufPage = getNewBufPage(pResultBuf, groupId, &pageId); ASSERT_TRUE(pBufPage != NULL); ASSERT_EQ(getTotalBufSize(pResultBuf), 1024); @@ -29,24 +29,24 @@ void simpleTest() { releaseBufPage(pResultBuf, pBufPage); - SFilePage* pBufPage1 = getNewDataBuf(pResultBuf, groupId, &pageId); + SFilePage* pBufPage1 = getNewBufPage(pResultBuf, groupId, &pageId); SFilePage* t = getBufPage(pResultBuf, pageId); ASSERT_TRUE(t == pBufPage1); - SFilePage* pBufPage2 = getNewDataBuf(pResultBuf, groupId, &pageId); + SFilePage* pBufPage2 = getNewBufPage(pResultBuf, groupId, &pageId); SFilePage* t1 = getBufPage(pResultBuf, pageId); ASSERT_TRUE(t1 == pBufPage2); - SFilePage* pBufPage3 = getNewDataBuf(pResultBuf, groupId, &pageId); + SFilePage* pBufPage3 = getNewBufPage(pResultBuf, groupId, &pageId); SFilePage* t2 = getBufPage(pResultBuf, pageId); ASSERT_TRUE(t2 == pBufPage3); - SFilePage* pBufPage4 = getNewDataBuf(pResultBuf, groupId, &pageId); + SFilePage* pBufPage4 = getNewBufPage(pResultBuf, groupId, &pageId); SFilePage* t3 = getBufPage(pResultBuf, pageId); ASSERT_TRUE(t3 == pBufPage4); - SFilePage* pBufPage5 = getNewDataBuf(pResultBuf, groupId, &pageId); + SFilePage* pBufPage5 = getNewBufPage(pResultBuf, groupId, &pageId); SFilePage* t4 = getBufPage(pResultBuf, pageId); ASSERT_TRUE(t4 == pBufPage5); @@ -62,29 +62,29 @@ void writeDownTest() { int32_t groupId = 0; int32_t nx = 12345; - SFilePage* pBufPage = getNewDataBuf(pResultBuf, groupId, &pageId); + SFilePage* pBufPage = getNewBufPage(pResultBuf, groupId, &pageId); ASSERT_TRUE(pBufPage != NULL); *(int32_t*)(pBufPage->data) = nx; writePageId = pageId; releaseBufPage(pResultBuf, pBufPage); - SFilePage* pBufPage1 = getNewDataBuf(pResultBuf, groupId, &pageId); + SFilePage* pBufPage1 = getNewBufPage(pResultBuf, groupId, &pageId); SFilePage* t1 = getBufPage(pResultBuf, pageId); ASSERT_TRUE(t1 == pBufPage1); ASSERT_TRUE(pageId == 1); - SFilePage* pBufPage2 = getNewDataBuf(pResultBuf, groupId, &pageId); + SFilePage* pBufPage2 = getNewBufPage(pResultBuf, groupId, &pageId); SFilePage* t2 = getBufPage(pResultBuf, pageId); ASSERT_TRUE(t2 == pBufPage2); ASSERT_TRUE(pageId == 2); - SFilePage* pBufPage3 = getNewDataBuf(pResultBuf, groupId, &pageId); + SFilePage* pBufPage3 = getNewBufPage(pResultBuf, groupId, &pageId); SFilePage* t3 = getBufPage(pResultBuf, pageId); ASSERT_TRUE(t3 == pBufPage3); ASSERT_TRUE(pageId == 3); - SFilePage* pBufPage4 = getNewDataBuf(pResultBuf, groupId, &pageId); + SFilePage* pBufPage4 = getNewBufPage(pResultBuf, groupId, &pageId); SFilePage* t4 = getBufPage(pResultBuf, pageId); ASSERT_TRUE(t4 == pBufPage4); ASSERT_TRUE(pageId == 4); @@ -109,32 +109,32 @@ void recyclePageTest() { int32_t groupId = 0; int32_t nx = 12345; - SFilePage* pBufPage = getNewDataBuf(pResultBuf, groupId, &pageId); + SFilePage* pBufPage = getNewBufPage(pResultBuf, groupId, &pageId); ASSERT_TRUE(pBufPage != NULL); releaseBufPage(pResultBuf, pBufPage); - SFilePage* pBufPage1 = getNewDataBuf(pResultBuf, groupId, &pageId); + SFilePage* pBufPage1 = getNewBufPage(pResultBuf, groupId, &pageId); SFilePage* t1 = getBufPage(pResultBuf, pageId); ASSERT_TRUE(t1 == pBufPage1); ASSERT_TRUE(pageId == 1); - SFilePage* pBufPage2 = getNewDataBuf(pResultBuf, groupId, &pageId); + SFilePage* pBufPage2 = getNewBufPage(pResultBuf, groupId, &pageId); SFilePage* t2 = getBufPage(pResultBuf, pageId); ASSERT_TRUE(t2 == pBufPage2); ASSERT_TRUE(pageId == 2); - SFilePage* pBufPage3 = getNewDataBuf(pResultBuf, groupId, &pageId); + SFilePage* pBufPage3 = getNewBufPage(pResultBuf, groupId, &pageId); SFilePage* t3 = getBufPage(pResultBuf, pageId); ASSERT_TRUE(t3 == pBufPage3); ASSERT_TRUE(pageId == 3); - SFilePage* pBufPage4 = getNewDataBuf(pResultBuf, groupId, &pageId); + SFilePage* pBufPage4 = getNewBufPage(pResultBuf, groupId, &pageId); SFilePage* t4 = getBufPage(pResultBuf, pageId); ASSERT_TRUE(t4 == pBufPage4); ASSERT_TRUE(pageId == 4); releaseBufPage(pResultBuf, t4); - SFilePage* pBufPage5 = getNewDataBuf(pResultBuf, groupId, &pageId); + SFilePage* pBufPage5 = getNewBufPage(pResultBuf, groupId, &pageId); SFilePage* t5 = getBufPage(pResultBuf, pageId); ASSERT_TRUE(t5 == pBufPage5); ASSERT_TRUE(pageId == 5); From 73b9bf64985cc5bb27c3610fdf5494fa666675a5 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 23 Feb 2022 15:13:00 +0800 Subject: [PATCH 14/25] [td-13039] add unit test. --- source/libs/executor/test/lhashTests.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/source/libs/executor/test/lhashTests.cpp b/source/libs/executor/test/lhashTests.cpp index be847d2959..11fc38a37d 100644 --- a/source/libs/executor/test/lhashTests.cpp +++ b/source/libs/executor/test/lhashTests.cpp @@ -31,13 +31,13 @@ TEST(testCase, linear_hash_Tests) { _hash_fn_t fn = taosGetDefaultHashFunction(TSDB_DATA_TYPE_INT); #if 1 SLHashObj* pHashObj = tHashInit(100, 64 + 8, fn, 4); - for(int32_t i = 0; i < 5000; ++i) { + for(int32_t i = 0; i < 500000; ++i) { tHashPut(pHashObj, &i, sizeof(i), &i, sizeof(i)); } tHashPrint(pHashObj, LINEAR_HASH_STATIS); - for(int32_t i = 0; i < 100; ++i) { + for(int32_t i = 0; i < 10000; ++i) { char* v = tHashGet(pHashObj, &i, sizeof(i)); if (v != NULL) { // printf("find value: %d, key:%d\n", *(int32_t*) v, i); From a97735a3586508105e127b104024e6c6ae1a6553 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 23 Feb 2022 15:38:35 +0800 Subject: [PATCH 15/25] [td-13039] fix compiler error. --- source/util/test/pageBufferTest.cpp | 68 ++++++++++++++--------------- 1 file changed, 34 insertions(+), 34 deletions(-) diff --git a/source/util/test/pageBufferTest.cpp b/source/util/test/pageBufferTest.cpp index e86fcf8653..4b9d8c5f51 100644 --- a/source/util/test/pageBufferTest.cpp +++ b/source/util/test/pageBufferTest.cpp @@ -18,7 +18,7 @@ void simpleTest() { int32_t pageId = 0; int32_t groupId = 0; - SFilePage* pBufPage = getNewBufPage(pResultBuf, groupId, &pageId); + SFilePage* pBufPage = static_cast(getNewBufPage(pResultBuf, groupId, &pageId)); ASSERT_TRUE(pBufPage != NULL); ASSERT_EQ(getTotalBufSize(pResultBuf), 1024); @@ -29,25 +29,25 @@ void simpleTest() { releaseBufPage(pResultBuf, pBufPage); - SFilePage* pBufPage1 = getNewBufPage(pResultBuf, groupId, &pageId); + SFilePage* pBufPage1 = static_cast(getNewBufPage(pResultBuf, groupId, &pageId)); - SFilePage* t = getBufPage(pResultBuf, pageId); + SFilePage* t = static_cast(getBufPage(pResultBuf, pageId)); ASSERT_TRUE(t == pBufPage1); - SFilePage* pBufPage2 = getNewBufPage(pResultBuf, groupId, &pageId); - SFilePage* t1 = getBufPage(pResultBuf, pageId); + SFilePage* pBufPage2 = static_cast(getNewBufPage(pResultBuf, groupId, &pageId)); + SFilePage* t1 = static_cast(getBufPage(pResultBuf, pageId)); ASSERT_TRUE(t1 == pBufPage2); - SFilePage* pBufPage3 = getNewBufPage(pResultBuf, groupId, &pageId); - SFilePage* t2 = getBufPage(pResultBuf, pageId); + SFilePage* pBufPage3 = static_cast(getNewBufPage(pResultBuf, groupId, &pageId)); + SFilePage* t2 = static_cast(getBufPage(pResultBuf, pageId)); ASSERT_TRUE(t2 == pBufPage3); - SFilePage* pBufPage4 = getNewBufPage(pResultBuf, groupId, &pageId); - SFilePage* t3 = getBufPage(pResultBuf, pageId); + SFilePage* pBufPage4 = static_cast(getNewBufPage(pResultBuf, groupId, &pageId)); + SFilePage* t3 = static_cast(getBufPage(pResultBuf, pageId)); ASSERT_TRUE(t3 == pBufPage4); - SFilePage* pBufPage5 = getNewBufPage(pResultBuf, groupId, &pageId); - SFilePage* t4 = getBufPage(pResultBuf, pageId); + SFilePage* pBufPage5 = static_cast(getNewBufPage(pResultBuf, groupId, &pageId)); + SFilePage* t4 = static_cast(getBufPage(pResultBuf, pageId)); ASSERT_TRUE(t4 == pBufPage5); destroyDiskbasedBuf(pResultBuf); @@ -62,36 +62,36 @@ void writeDownTest() { int32_t groupId = 0; int32_t nx = 12345; - SFilePage* pBufPage = getNewBufPage(pResultBuf, groupId, &pageId); + SFilePage* pBufPage = static_cast(getNewBufPage(pResultBuf, groupId, &pageId)); ASSERT_TRUE(pBufPage != NULL); *(int32_t*)(pBufPage->data) = nx; writePageId = pageId; releaseBufPage(pResultBuf, pBufPage); - SFilePage* pBufPage1 = getNewBufPage(pResultBuf, groupId, &pageId); - SFilePage* t1 = getBufPage(pResultBuf, pageId); + SFilePage* pBufPage1 = static_cast(getNewBufPage(pResultBuf, groupId, &pageId)); + SFilePage* t1 = static_cast(getBufPage(pResultBuf, pageId)); ASSERT_TRUE(t1 == pBufPage1); ASSERT_TRUE(pageId == 1); - SFilePage* pBufPage2 = getNewBufPage(pResultBuf, groupId, &pageId); - SFilePage* t2 = getBufPage(pResultBuf, pageId); + SFilePage* pBufPage2 = static_cast(getNewBufPage(pResultBuf, groupId, &pageId)); + SFilePage* t2 = static_cast(getBufPage(pResultBuf, pageId)); ASSERT_TRUE(t2 == pBufPage2); ASSERT_TRUE(pageId == 2); - SFilePage* pBufPage3 = getNewBufPage(pResultBuf, groupId, &pageId); - SFilePage* t3 = getBufPage(pResultBuf, pageId); + SFilePage* pBufPage3 = static_cast(getNewBufPage(pResultBuf, groupId, &pageId)); + SFilePage* t3 = static_cast(getBufPage(pResultBuf, pageId)); ASSERT_TRUE(t3 == pBufPage3); ASSERT_TRUE(pageId == 3); - SFilePage* pBufPage4 = getNewBufPage(pResultBuf, groupId, &pageId); - SFilePage* t4 = getBufPage(pResultBuf, pageId); + SFilePage* pBufPage4 = static_cast(getNewBufPage(pResultBuf, groupId, &pageId)); + SFilePage* t4 = static_cast(getBufPage(pResultBuf, pageId)); ASSERT_TRUE(t4 == pBufPage4); ASSERT_TRUE(pageId == 4); releaseBufPage(pResultBuf, t4); // flush the written page to disk, and read it out again - SFilePage* pBufPagex = getBufPage(pResultBuf, writePageId); + SFilePage* pBufPagex = static_cast(getBufPage(pResultBuf, writePageId)); ASSERT_EQ(*(int32_t*)pBufPagex->data, nx); SArray* pa = getDataBufPagesIdList(pResultBuf, groupId); @@ -109,43 +109,43 @@ void recyclePageTest() { int32_t groupId = 0; int32_t nx = 12345; - SFilePage* pBufPage = getNewBufPage(pResultBuf, groupId, &pageId); + SFilePage* pBufPage = static_cast(getNewBufPage(pResultBuf, groupId, &pageId)); ASSERT_TRUE(pBufPage != NULL); releaseBufPage(pResultBuf, pBufPage); - SFilePage* pBufPage1 = getNewBufPage(pResultBuf, groupId, &pageId); - SFilePage* t1 = getBufPage(pResultBuf, pageId); + SFilePage* pBufPage1 = static_cast(getNewBufPage(pResultBuf, groupId, &pageId)); + SFilePage* t1 = static_cast(getBufPage(pResultBuf, pageId)); ASSERT_TRUE(t1 == pBufPage1); ASSERT_TRUE(pageId == 1); - SFilePage* pBufPage2 = getNewBufPage(pResultBuf, groupId, &pageId); - SFilePage* t2 = getBufPage(pResultBuf, pageId); + SFilePage* pBufPage2 = static_cast(getNewBufPage(pResultBuf, groupId, &pageId)); + SFilePage* t2 = static_cast(getBufPage(pResultBuf, pageId)); ASSERT_TRUE(t2 == pBufPage2); ASSERT_TRUE(pageId == 2); - SFilePage* pBufPage3 = getNewBufPage(pResultBuf, groupId, &pageId); - SFilePage* t3 = getBufPage(pResultBuf, pageId); + SFilePage* pBufPage3 = static_cast(getNewBufPage(pResultBuf, groupId, &pageId)); + SFilePage* t3 = static_cast(getBufPage(pResultBuf, pageId)); ASSERT_TRUE(t3 == pBufPage3); ASSERT_TRUE(pageId == 3); - SFilePage* pBufPage4 = getNewBufPage(pResultBuf, groupId, &pageId); - SFilePage* t4 = getBufPage(pResultBuf, pageId); + SFilePage* pBufPage4 = static_cast(getNewBufPage(pResultBuf, groupId, &pageId)); + SFilePage* t4 = static_cast(getBufPage(pResultBuf, pageId)); ASSERT_TRUE(t4 == pBufPage4); ASSERT_TRUE(pageId == 4); releaseBufPage(pResultBuf, t4); - SFilePage* pBufPage5 = getNewBufPage(pResultBuf, groupId, &pageId); - SFilePage* t5 = getBufPage(pResultBuf, pageId); + SFilePage* pBufPage5 = static_cast(getNewBufPage(pResultBuf, groupId, &pageId)); + SFilePage* t5 = static_cast(getBufPage(pResultBuf, pageId)); ASSERT_TRUE(t5 == pBufPage5); ASSERT_TRUE(pageId == 5); // flush the written page to disk, and read it out again - SFilePage* pBufPagex = getBufPage(pResultBuf, writePageId); + SFilePage* pBufPagex = static_cast(getBufPage(pResultBuf, writePageId)); *(int32_t*)(pBufPagex->data) = nx; writePageId = pageId; // update the data releaseBufPage(pResultBuf, pBufPagex); - SFilePage* pBufPagex1 = getBufPage(pResultBuf, 1); + SFilePage* pBufPagex1 = static_cast(getBufPage(pResultBuf, 1)); SArray* pa = getDataBufPagesIdList(pResultBuf, groupId); ASSERT_EQ(taosArrayGetSize(pa), 6); From fb77bb1a0b377a583dc37011388cc44ac0208a4f Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 23 Feb 2022 15:40:50 +0800 Subject: [PATCH 16/25] [td-13039]fix compiling error. --- source/libs/executor/test/sortTests.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/source/libs/executor/test/sortTests.cpp b/source/libs/executor/test/sortTests.cpp index ec066c95b1..fc366e4cc8 100644 --- a/source/libs/executor/test/sortTests.cpp +++ b/source/libs/executor/test/sortTests.cpp @@ -145,6 +145,8 @@ int32_t docomp(const void* p1, const void* p2, void* param) { assert(0); } } + + return 0; } } // namespace From 1d46e0a70a40aae5f61642bd606ca5dde82d1f81 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 23 Feb 2022 15:54:59 +0800 Subject: [PATCH 17/25] [td-13039]Add api to close file when flushing data out of buffer. --- include/util/tpagedbuf.h | 6 ++++++ source/libs/executor/src/tlinearhash.c | 2 ++ 2 files changed, 8 insertions(+) diff --git a/include/util/tpagedbuf.h b/include/util/tpagedbuf.h index d9e233f8bb..4cd0797df3 100644 --- a/include/util/tpagedbuf.h +++ b/include/util/tpagedbuf.h @@ -152,6 +152,12 @@ bool isAllDataInMemBuf(const SDiskbasedBuf* pBuf); */ void setBufPageDirty(void* pPageInfo, bool dirty); +/** + * Set the compress/ no-compress flag for paged buffer, when flushing data in disk. + * @param pBuf + */ +void setBufPageCompressOnDisk(SDiskbasedBuf* pBuf, bool comp); + /** * Print the statistics when closing this buffer * @param pBuf diff --git a/source/libs/executor/src/tlinearhash.c b/source/libs/executor/src/tlinearhash.c index b644801a6e..152102f0c3 100644 --- a/source/libs/executor/src/tlinearhash.c +++ b/source/libs/executor/src/tlinearhash.c @@ -78,6 +78,8 @@ SLHashObj* tHashInit(int32_t inMemPages, int32_t pageSize, _hash_fn_t fn, int32_ return NULL; } + setBufPageCompressOnDisk(pHashObj->pBuf, false); + /** * The number of bits in the hash value, which is used to decide the exact bucket where the object should be located in. * The initial value is 0. From bbf77ff193f5ae9d11e6c25e403e6037538fc874 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 23 Feb 2022 15:56:14 +0800 Subject: [PATCH 18/25] [td-13039] add impl --- source/util/src/tpagedbuf.c | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/source/util/src/tpagedbuf.c b/source/util/src/tpagedbuf.c index cc3d1747a1..967443e93f 100644 --- a/source/util/src/tpagedbuf.c +++ b/source/util/src/tpagedbuf.c @@ -587,6 +587,11 @@ void setBufPageDirty(void* pPage, bool dirty) { ppi->dirty = dirty; } +void setBufPageCompressOnDisk(SDiskbasedBuf* pBuf, bool comp) { + pBuf->comp = comp; +} + + void dBufSetPrintInfo(SDiskbasedBuf* pBuf) { pBuf->printStatis = true; } From d3228cf9581aefb83a446277bfbbc926560bc5ba Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 23 Feb 2022 16:39:21 +0800 Subject: [PATCH 19/25] [td-13039] udpate test case. --- source/libs/executor/test/lhashTests.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/libs/executor/test/lhashTests.cpp b/source/libs/executor/test/lhashTests.cpp index 11fc38a37d..533fc67916 100644 --- a/source/libs/executor/test/lhashTests.cpp +++ b/source/libs/executor/test/lhashTests.cpp @@ -30,7 +30,7 @@ TEST(testCase, linear_hash_Tests) { _hash_fn_t fn = taosGetDefaultHashFunction(TSDB_DATA_TYPE_INT); #if 1 - SLHashObj* pHashObj = tHashInit(100, 64 + 8, fn, 4); + SLHashObj* pHashObj = tHashInit(220000, 64 + 8, fn, 4); for(int32_t i = 0; i < 500000; ++i) { tHashPut(pHashObj, &i, sizeof(i), &i, sizeof(i)); } From c440e55395123270137af62a455da3d329744eb5 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 23 Feb 2022 19:39:43 +0800 Subject: [PATCH 20/25] [td-13039] add page compression. --- include/util/tpagedbuf.h | 16 +- source/libs/executor/src/tlinearhash.c | 315 +++++++++++++---------- source/libs/executor/test/lhashTests.cpp | 26 +- source/util/src/tpagedbuf.c | 249 +++++++++--------- 4 files changed, 344 insertions(+), 262 deletions(-) diff --git a/include/util/tpagedbuf.h b/include/util/tpagedbuf.h index 4cd0797df3..decf952bcd 100644 --- a/include/util/tpagedbuf.h +++ b/include/util/tpagedbuf.h @@ -136,6 +136,11 @@ int32_t getPageId(const SPageInfo* pPgInfo); */ int32_t getBufPageSize(const SDiskbasedBuf* pBuf); +/** + * + * @param pBuf + * @return + */ int32_t getNumOfInMemBufPages(const SDiskbasedBuf* pBuf); /** @@ -147,10 +152,10 @@ bool isAllDataInMemBuf(const SDiskbasedBuf* pBuf); /** * Set the buffer page is dirty, and needs to be flushed to disk when swap out. - * @param pPageInfo + * @param pPage * @param dirty */ -void setBufPageDirty(void* pPageInfo, bool dirty); +void setBufPageDirty(void* pPage, bool dirty); /** * Set the compress/ no-compress flag for paged buffer, when flushing data in disk. @@ -158,6 +163,13 @@ void setBufPageDirty(void* pPageInfo, bool dirty); */ void setBufPageCompressOnDisk(SDiskbasedBuf* pBuf, bool comp); +/** + * Set the pageId page buffer is not need + * @param pBuf + * @param pageId + */ +void dBufSetBufPageRecycled(SDiskbasedBuf *pBuf, void* pPage); + /** * Print the statistics when closing this buffer * @param pBuf diff --git a/source/libs/executor/src/tlinearhash.c b/source/libs/executor/src/tlinearhash.c index 152102f0c3..803ce8bba2 100644 --- a/source/libs/executor/src/tlinearhash.c +++ b/source/libs/executor/src/tlinearhash.c @@ -49,7 +49,7 @@ typedef struct SLHashNode { } SLHashNode; #define GET_LHASH_NODE_KEY(_n) (((char*)(_n)) + sizeof(SLHashNode)) -#define GET_LHASH_NODE_DATA(_n) ((char*)(_n) + sizeof(SLHashNode) + (_n)->keyLen) +#define GET_LHASH_NODE_DATA(_n) ((char*)(_n) + sizeof(SLHashNode) + ((SLHashNode*)(_n))->keyLen) #define GET_LHASH_NODE_LEN(_n) (sizeof(SLHashNode) + ((SLHashNode*)(_n))->keyLen + ((SLHashNode*)(_n))->dataLen) static int32_t doAddNewBucket(SLHashObj* pHashObj); @@ -60,11 +60,166 @@ static int32_t doGetBucketIdFromHashVal(int32_t hashv, int32_t bits) { static int32_t doGetAlternativeBucketId(int32_t bucketId, int32_t bits, int32_t numOfBuckets) { int32_t v = bucketId - (1ul << (bits - 1)); - ASSERT(v < numOfBuckets); return v; } +static int32_t doGetRelatedSplitBucketId(int32_t bucketId, int32_t bits) { + int32_t splitBucketId = (1ul << (bits - 1)) ^ bucketId; + return splitBucketId; +} + +static void doCopyObject(char* p, const void* key, int32_t keyLen, const void* data, int32_t size) { + *(int32_t*) p = keyLen; + p += sizeof(int32_t); + *(int32_t*) p = size; + p += sizeof(int32_t); + + memcpy(p, key, keyLen); + p += keyLen; + + memcpy(p, data, size); +} + +static int32_t doAddToBucket(SLHashObj* pHashObj, SLHashBucket* pBucket, int32_t index, const void* key, int32_t keyLen, + const void* data, int32_t size) { + int32_t pageId = *(int32_t*)taosArrayGetLast(pBucket->pPageIdList); + + SFilePage* pPage = getBufPage(pHashObj->pBuf, pageId); + ASSERT (pPage != NULL); + + // put to current buf page + size_t nodeSize = sizeof(SLHashNode) + keyLen + size; + ASSERT(nodeSize + sizeof(SFilePage) <= getBufPageSize(pHashObj->pBuf)); + + if (pPage->num + nodeSize > getBufPageSize(pHashObj->pBuf)) { + releaseBufPage(pHashObj->pBuf, pPage); + + // allocate the overflow buffer page to hold this k/v. + int32_t newPageId = -1; + SFilePage* pNewPage = getNewBufPage(pHashObj->pBuf, 0, &newPageId); + if (pNewPage == NULL) { + return TSDB_CODE_OUT_OF_MEMORY; + } + + taosArrayPush(pBucket->pPageIdList, &newPageId); + + doCopyObject(pNewPage->data, key, keyLen, data, size); + pNewPage->num = sizeof(SFilePage) + nodeSize; + + setBufPageDirty(pNewPage, true); + releaseBufPage(pHashObj->pBuf, pNewPage); + } else { + char* p = (char*) pPage + pPage->num; + doCopyObject(p, key, keyLen, data, size); + pPage->num += nodeSize; + setBufPageDirty(pPage, true); + releaseBufPage(pHashObj->pBuf, pPage); + } + + pBucket->size += 1; + printf("===> add to bucket:0x%x, num:%d, key:%d\n", index, pBucket->size, *(int*) key); + + return TSDB_CODE_SUCCESS; +} + +static void doRemoveFromBucket(SFilePage* pPage, SLHashNode* pNode, SLHashBucket* pBucket) { + ASSERT(pPage != NULL && pNode != NULL && pBucket->size >= 1); + + int32_t len = GET_LHASH_NODE_LEN(pNode); + char* p = (char*) pNode + len; + + char* pEnd = (char*)pPage + pPage->num; + memmove(pNode, p, (pEnd - p)); + + pPage->num -= len; + if (pPage->num == 0) { + // this page is empty, could be recycle in the future. + } + + setBufPageDirty(pPage, true); + + pBucket->size -= 1; +} + +static void doCompressBucketPages(SLHashObj *pHashObj, SLHashBucket* pBucket) { + size_t numOfPages = taosArrayGetSize(pBucket->pPageIdList); + if (numOfPages <= 1) { + return; + } + + int32_t* firstPage = taosArrayGet(pBucket->pPageIdList, 0); + SFilePage* pFirst = getBufPage(pHashObj->pBuf, *firstPage); + + int32_t* pageId = taosArrayGetLast(pBucket->pPageIdList); + SFilePage* pLast = getBufPage(pHashObj->pBuf, *pageId); + + char* pStart = pLast->data; + int32_t nodeSize = GET_LHASH_NODE_LEN(pStart); + while (1) { + if (pFirst->num + nodeSize < getBufPageSize(pHashObj->pBuf)) { + char* p = ((char*)pFirst) + pFirst->num; + + SLHashNode* pNode = (SLHashNode*)pStart; + doCopyObject(p, GET_LHASH_NODE_KEY(pStart), pNode->keyLen, GET_LHASH_NODE_DATA(pStart), pNode->dataLen); + setBufPageDirty(pFirst, true); + + pFirst->num += nodeSize; + pLast->num -= nodeSize; + pStart += nodeSize; + if (pStart - pLast->data >= pLast->num) { + // this is empty + dBufSetBufPageRecycled(pHashObj->pBuf, pLast); + taosArrayRemove(pBucket->pPageIdList, numOfPages - 1); + break; + } + + nodeSize = GET_LHASH_NODE_LEN(pStart); + } else { // move to the front of pLast page + memmove(pLast->data, pStart,(((char*)pLast) + pLast->num - pStart)); + break; + } + } +} + +static int32_t doAddNewBucket(SLHashObj* pHashObj) { + if (pHashObj->numOfBuckets + 1 > pHashObj->numOfAlloc) { + int32_t newLen = pHashObj->numOfAlloc * 1.25; + if (newLen == pHashObj->numOfAlloc) { + newLen += 4; + } + + char* p = realloc(pHashObj->pBucket, POINTER_BYTES * newLen); + if (p == NULL) { + return TSDB_CODE_OUT_OF_MEMORY; + } + + memset(p + POINTER_BYTES * pHashObj->numOfBuckets, 0, newLen - pHashObj->numOfBuckets); + pHashObj->pBucket = (SLHashBucket**) p; + pHashObj->numOfAlloc = newLen; + } + + SLHashBucket* pBucket = calloc(1, sizeof(SLHashBucket)); + pHashObj->pBucket[pHashObj->numOfBuckets] = pBucket; + + pBucket->pPageIdList = taosArrayInit(2, sizeof(int32_t)); + if (pBucket->pPageIdList == NULL || pBucket == NULL) { + return TSDB_CODE_OUT_OF_MEMORY; + } + + int32_t pageId = -1; + SFilePage* p = getNewBufPage(pHashObj->pBuf, 0, &pageId); + p->num = sizeof(SFilePage); + setBufPageDirty(p, true); + + releaseBufPage(pHashObj->pBuf, p); + taosArrayPush(pBucket->pPageIdList, &pageId); + + pHashObj->numOfBuckets += 1; + printf("---------------add new bucket, id:0x%x, total:%d\n", pHashObj->numOfBuckets - 1, pHashObj->numOfBuckets); + return TSDB_CODE_SUCCESS; +} + SLHashObj* tHashInit(int32_t inMemPages, int32_t pageSize, _hash_fn_t fn, int32_t numOfTuplePerPage) { SLHashObj* pHashObj = calloc(1, sizeof(SLHashObj)); if (pHashObj == NULL) { @@ -114,113 +269,6 @@ void* tHashCleanup(SLHashObj* pHashObj) { return NULL; } -static void doCopyObject(char* p, const void* key, int32_t keyLen, const void* data, int32_t size) { - *(int32_t*) p = keyLen; - p += sizeof(int32_t); - *(int32_t*) p = size; - p += sizeof(int32_t); - - memcpy(p, key, keyLen); - p += keyLen; - - memcpy(p, data, size); -} - -static int32_t doAddToBucket(SLHashObj* pHashObj, SLHashBucket* pBucket, int32_t index, const void* key, int32_t keyLen, - const void* data, int32_t size) { - int32_t pageId = *(int32_t*)taosArrayGetLast(pBucket->pPageIdList); - - SFilePage* pPage = getBufPage(pHashObj->pBuf, pageId); - ASSERT (pPage != NULL); - - // put to current buf page - size_t nodeSize = sizeof(SLHashNode) + keyLen + size; - ASSERT(nodeSize <= getBufPageSize(pHashObj->pBuf)); - - if (pPage->num + nodeSize > getBufPageSize(pHashObj->pBuf)) { - releaseBufPage(pHashObj->pBuf, pPage); - - // allocate the overflow buffer page to hold this k/v. - int32_t newPageId = -1; - SFilePage* pNewPage = getNewBufPage(pHashObj->pBuf, 0, &newPageId); - if (pNewPage == 0) { - // TODO handle error - } - - taosArrayPush(pBucket->pPageIdList, &newPageId); - - doCopyObject(pNewPage->data, key, keyLen, data, size); - pNewPage->num = nodeSize; - - setBufPageDirty(pNewPage, true); - releaseBufPage(pHashObj->pBuf, pNewPage); - } else { - char* p = pPage->data + pPage->num; - doCopyObject(p, key, keyLen, data, size); - pPage->num += nodeSize; - setBufPageDirty(pPage, true); - releaseBufPage(pHashObj->pBuf, pPage); - } - - pBucket->size += 1; -// printf("===> add to bucket:0x%x, num:%d, key:%d\n", index, pBucket->size, *(int*) key); -} - -// TODO merge the fragments on multiple pages to recycle the empty disk page ASAP -static void doRemoveFromBucket(SFilePage* pPage, SLHashNode* pNode, SLHashBucket* pBucket) { - ASSERT(pPage != NULL && pNode != NULL); - - int32_t len = GET_LHASH_NODE_LEN(pNode); - char* p = (char*) pNode + len; - - char* pEnd = pPage->data + pPage->num; - memmove(pNode, p, (pEnd - p)); - - pPage->num -= len; - if (pPage->num == 0) { - // this page is empty, could be recycle in the future. - } - - setBufPageDirty(pPage, true); - pBucket->size -= 1; -} - -static int32_t doAddNewBucket(SLHashObj* pHashObj) { - if (pHashObj->numOfBuckets + 1 > pHashObj->numOfAlloc) { - int32_t newLen = pHashObj->numOfAlloc * 1.25; - if (newLen == pHashObj->numOfAlloc) { - newLen += 4; - } - - char* p = realloc(pHashObj->pBucket, POINTER_BYTES * newLen); - if (p == NULL) { - return TSDB_CODE_OUT_OF_MEMORY; - } - - memset(p + POINTER_BYTES * pHashObj->numOfBuckets, 0, newLen - pHashObj->numOfBuckets); - pHashObj->pBucket = (SLHashBucket**) p; - pHashObj->numOfAlloc = newLen; - } - - SLHashBucket* pBucket = calloc(1, sizeof(SLHashBucket)); - pHashObj->pBucket[pHashObj->numOfBuckets] = pBucket; - - pBucket->pPageIdList = taosArrayInit(2, sizeof(int32_t)); - if (pBucket->pPageIdList == NULL || pBucket == NULL) { - return TSDB_CODE_OUT_OF_MEMORY; - } - - int32_t pageId = -1; - SFilePage* p = getNewBufPage(pHashObj->pBuf, 0, &pageId); - releaseBufPage(pHashObj->pBuf, p); - - taosArrayPush(pBucket->pPageIdList, &pageId); - - pHashObj->numOfBuckets += 1; -// printf("---------------add new bucket, id:0x%x, total:%d\n", pHashObj->numOfBuckets - 1, pHashObj->numOfBuckets); - return TSDB_CODE_SUCCESS; -} - int32_t tHashPut(SLHashObj* pHashObj, const void *key, size_t keyLen, void *data, size_t size) { ASSERT(pHashObj != NULL && key != NULL); @@ -231,17 +279,16 @@ int32_t tHashPut(SLHashObj* pHashObj, const void *key, size_t keyLen, void *data int32_t hashVal = pHashObj->hashFn(key, keyLen); int32_t v = doGetBucketIdFromHashVal(hashVal, pHashObj->bits); - if (pHashObj->numOfBuckets > v) { - SLHashBucket* pBucket = pHashObj->pBucket[v]; + if (v >= pHashObj->numOfBuckets) { + int32_t newBucketId = doGetAlternativeBucketId(v, pHashObj->bits, pHashObj->numOfBuckets); + printf("bucketId: 0x%x not exists, put it into 0x%x instead\n", v, newBucketId); + v = newBucketId; + } - // TODO check return code - doAddToBucket(pHashObj, pBucket, v, key, keyLen, data, size); - } else { // no matched bucket exists, find the candidate bucket - int32_t bucketId = doGetAlternativeBucketId(v, pHashObj->bits, pHashObj->numOfBuckets); -// printf("bucketId: 0x%x not exists, put it into 0x%x instead\n", v, bucketId); - - SLHashBucket* pBucket = pHashObj->pBucket[bucketId]; - doAddToBucket(pHashObj, pBucket, bucketId, key, keyLen, data, size); + SLHashBucket* pBucket = pHashObj->pBucket[v]; + int32_t code = doAddToBucket(pHashObj, pBucket, v, key, keyLen, data, size); + if (code != TSDB_CODE_SUCCESS) { + return code; } } @@ -252,42 +299,46 @@ int32_t tHashPut(SLHashObj* pHashObj, const void *key, size_t keyLen, void *data int32_t newBucketId = pHashObj->numOfBuckets; int32_t code = doAddNewBucket(pHashObj); + if (code != TSDB_CODE_SUCCESS) { + return code; + } + int32_t numOfBits = ceil(log(pHashObj->numOfBuckets) / log(2)); if (numOfBits > pHashObj->bits) { -// printf("extend the bits from %d to %d, new bucket:%d\n", pHashObj->bits, numOfBits, newBucketId); - + printf("extend the bits from %d to %d, new bucket:%d\n", pHashObj->bits, numOfBits, newBucketId); ASSERT(numOfBits == pHashObj->bits + 1); pHashObj->bits = numOfBits; } - int32_t splitBucketId = (1ul << (pHashObj->bits - 1)) ^ newBucketId; + int32_t splitBucketId = doGetRelatedSplitBucketId(newBucketId, pHashObj->bits); // load all data in this bucket and check if the data needs to relocated into the new bucket SLHashBucket* pBucket = pHashObj->pBucket[splitBucketId]; -// printf("split %d items' bucket:0x%x to new bucket:0x%x\n", pBucket->size, splitBucketId, newBucketId); + printf("split %d items' bucket:0x%x to new bucket:0x%x\n", pBucket->size, splitBucketId, newBucketId); for (int32_t i = 0; i < taosArrayGetSize(pBucket->pPageIdList); ++i) { int32_t pageId = *(int32_t*)taosArrayGet(pBucket->pPageIdList, i); SFilePage* p = getBufPage(pHashObj->pBuf, pageId); char* pStart = p->data; - while (pStart - p->data < p->num) { + while (pStart - ((char*) p) < p->num) { SLHashNode* pNode = (SLHashNode*)pStart; + ASSERT(pNode->keyLen > 0 && pNode->dataLen >= 0); char* k = GET_LHASH_NODE_KEY(pNode); int32_t hashv = pHashObj->hashFn(k, pNode->keyLen); + int32_t v1 = doGetBucketIdFromHashVal(hashv, pHashObj->bits); - int32_t v1 = hashv & ((1ul << (pHashObj->bits)) - 1); if (v1 != splitBucketId) { // place it into the new bucket ASSERT(v1 == newBucketId); -// printf("move key:%d to 0x%x bucket, remain items:%d\n", *(int32_t*)k, v1, pBucket->size - 1); + printf("move key:%d to 0x%x bucket, remain items:%d\n", *(int32_t*)k, v1, pBucket->size - 1); SLHashBucket* pNewBucket = pHashObj->pBucket[newBucketId]; doAddToBucket(pHashObj, pNewBucket, newBucketId, (void*)GET_LHASH_NODE_KEY(pNode), pNode->keyLen, GET_LHASH_NODE_KEY(pNode), pNode->dataLen); doRemoveFromBucket(p, pNode, pBucket); } else { -// printf("check key:%d, located into: %d, skip it\n", *(int*) k, v1); + printf("check key:%d, located into: %d, skip it\n", *(int*) k, v1); int32_t nodeSize = GET_LHASH_NODE_LEN(pStart); pStart += nodeSize; @@ -295,7 +346,11 @@ int32_t tHashPut(SLHashObj* pHashObj, const void *key, size_t keyLen, void *data } releaseBufPage(pHashObj->pBuf, p); } + + doCompressBucketPages(pHashObj, pBucket); } + + return TSDB_CODE_SUCCESS; } char* tHashGet(SLHashObj* pHashObj, const void *key, size_t keyLen) { @@ -332,7 +387,7 @@ char* tHashGet(SLHashObj* pHashObj, const void *key, size_t keyLen) { } int32_t tHashRemove(SLHashObj* pHashObj, const void *key, size_t keyLen) { - + // todo } void tHashPrint(const SLHashObj* pHashObj, int32_t type) { @@ -343,8 +398,8 @@ void tHashPrint(const SLHashObj* pHashObj, int32_t type) { if (type == LINEAR_HASH_DATA) { for (int32_t i = 0; i < pHashObj->numOfBuckets; ++i) { -// printf("bucket: 0x%x, obj:%d, page:%d\n", i, pHashObj->pBucket[i]->size, -// (int)taosArrayGetSize(pHashObj->pBucket[i]->pPageIdList)); + printf("bucket: 0x%x, obj:%d, page:%d\n", i, pHashObj->pBucket[i]->size, + (int)taosArrayGetSize(pHashObj->pBucket[i]->pPageIdList)); } } else { dBufPrintStatis(pHashObj->pBuf); diff --git a/source/libs/executor/test/lhashTests.cpp b/source/libs/executor/test/lhashTests.cpp index 533fc67916..d0fe9c5ac0 100644 --- a/source/libs/executor/test/lhashTests.cpp +++ b/source/libs/executor/test/lhashTests.cpp @@ -23,28 +23,28 @@ #pragma GCC diagnostic ignored "-Wunused-function" #pragma GCC diagnostic ignored "-Wunused-variable" #pragma GCC diagnostic ignored "-Wsign-compare" -#include "os.h" TEST(testCase, linear_hash_Tests) { srand(time(NULL)); _hash_fn_t fn = taosGetDefaultHashFunction(TSDB_DATA_TYPE_INT); #if 1 - SLHashObj* pHashObj = tHashInit(220000, 64 + 8, fn, 4); - for(int32_t i = 0; i < 500000; ++i) { - tHashPut(pHashObj, &i, sizeof(i), &i, sizeof(i)); + SLHashObj* pHashObj = tHashInit(10, 128 + 8, fn, 8); + for(int32_t i = 0; i < 100; ++i) { + int32_t code = tHashPut(pHashObj, &i, sizeof(i), &i, sizeof(i)); + assert(code == 0); } - tHashPrint(pHashObj, LINEAR_HASH_STATIS); +// tHashPrint(pHashObj, LINEAR_HASH_STATIS); - for(int32_t i = 0; i < 10000; ++i) { - char* v = tHashGet(pHashObj, &i, sizeof(i)); - if (v != NULL) { -// printf("find value: %d, key:%d\n", *(int32_t*) v, i); - } else { - printf("failed to found key:%d in hash\n", i); - } - } +// for(int32_t i = 0; i < 10000; ++i) { +// char* v = tHashGet(pHashObj, &i, sizeof(i)); +// if (v != NULL) { +//// printf("find value: %d, key:%d\n", *(int32_t*) v, i); +// } else { +// printf("failed to found key:%d in hash\n", i); +// } +// } tHashPrint(pHashObj, LINEAR_HASH_DATA); tHashCleanup(pHashObj); diff --git a/source/util/src/tpagedbuf.c b/source/util/src/tpagedbuf.c index 967443e93f..3fe3ddccde 100644 --- a/source/util/src/tpagedbuf.c +++ b/source/util/src/tpagedbuf.c @@ -5,25 +5,26 @@ #include "tcompression.h" #include "thash.h" +//enum { +// true = 0x1, +// BUF_PAGE_RELEASED = 0x2, +// true = 0x3, +//}; + #define GET_DATA_PAYLOAD(_p) ((char *)(_p)->pData + POINTER_BYTES) #define NO_IN_MEM_AVAILABLE_PAGES(_b) (listNEles((_b)->lruList) >= (_b)->inMemPages) -typedef struct SFreeListItem { - int32_t offset; - int32_t len; -} SFreeListItem; - typedef struct SPageDiskInfo { int64_t offset; int32_t length; -} SPageDiskInfo; +} SPageDiskInfo, SFreeListItem; struct SPageInfo { SListNode* pn; // point to list node void* pData; int64_t offset; int32_t pageId; - int32_t length:30; + int32_t length:29; bool used:1; // set current page is in used bool dirty:1; // set current buffer page is dirty or not }; @@ -51,46 +52,6 @@ struct SDiskbasedBuf { SDiskbasedBufStatis statis; }; -int32_t createDiskbasedBuf(SDiskbasedBuf** pBuf, int32_t pagesize, int32_t inMemBufSize, uint64_t qId, const char* dir) { - *pBuf = calloc(1, sizeof(SDiskbasedBuf)); - - SDiskbasedBuf* pResBuf = *pBuf; - if (pResBuf == NULL) { - return TSDB_CODE_OUT_OF_MEMORY; - } - - pResBuf->pageSize = pagesize; - pResBuf->numOfPages = 0; // all pages are in buffer in the first place - pResBuf->totalBufSize = 0; - pResBuf->inMemPages = inMemBufSize/pagesize; // maximum allowed pages, it is a soft limit. - pResBuf->allocateId = -1; - pResBuf->comp = true; - pResBuf->file = NULL; - pResBuf->qId = qId; - pResBuf->fileSize = 0; - - // at least more than 2 pages must be in memory - assert(inMemBufSize >= pagesize * 2); - - pResBuf->lruList = tdListNew(POINTER_BYTES); - - // init id hash table - pResBuf->groupSet = taosHashInit(10, taosGetDefaultHashFunction(TSDB_DATA_TYPE_INT), true, false); - pResBuf->assistBuf = malloc(pResBuf->pageSize + 2); // EXTRA BYTES - pResBuf->all = taosHashInit(10, taosGetDefaultHashFunction(TSDB_DATA_TYPE_INT), true, false); - - char path[PATH_MAX] = {0}; - taosGetTmpfilePath(dir, "qbuf", path); - pResBuf->path = strdup(path); - - pResBuf->emptyDummyIdList = taosArrayInit(1, sizeof(int32_t)); - -// qDebug("QInfo:0x%"PRIx64" create resBuf for output, page size:%d, inmem buf pages:%d, file:%s", qId, pResBuf->pageSize, -// pResBuf->inMemPages, pResBuf->path); - - return TSDB_CODE_SUCCESS; -} - static int32_t createDiskFile(SDiskbasedBuf* pBuf) { pBuf->file = fopen(pBuf->path, "wb+"); if (pBuf->file == NULL) { @@ -135,10 +96,10 @@ static uint64_t allocatePositionInFile(SDiskbasedBuf* pBuf, size_t size) { size_t num = taosArrayGetSize(pBuf->pFree); for(int32_t i = 0; i < num; ++i) { SFreeListItem* pi = taosArrayGet(pBuf->pFree, i); - if (pi->len >= size) { + if (pi->length >= size) { offset = pi->offset; pi->offset += (int32_t)size; - pi->len -= (int32_t)size; + pi->length -= (int32_t)size; return offset; } @@ -160,7 +121,7 @@ static uint64_t allocatePositionInFile(SDiskbasedBuf* pBuf, size_t size) { static char* doFlushPageToDisk(SDiskbasedBuf* pBuf, SPageInfo* pg) { assert(!pg->used && pg->pData != NULL); - int32_t size = -1; + int32_t size = pBuf->pageSize; char* t = NULL; if (pg->offset == -1 || pg->dirty) { void* payload = GET_DATA_PAYLOAD(pg); @@ -169,66 +130,68 @@ static char* doFlushPageToDisk(SDiskbasedBuf* pBuf, SPageInfo* pg) { } // this page is flushed to disk for the first time - if (pg->offset == -1) { - assert(pg->dirty == true); + if (pg->dirty) { + if (pg->offset == -1) { + assert(pg->dirty == true); - pg->offset = allocatePositionInFile(pBuf, size); - pBuf->nextPos += size; - - int32_t ret = fseek(pBuf->file, pg->offset, SEEK_SET); - if (ret != 0) { - terrno = TAOS_SYSTEM_ERROR(errno); - return NULL; - } - - ret = (int32_t) fwrite(t, 1, size, pBuf->file); - if (ret != size) { - terrno = TAOS_SYSTEM_ERROR(errno); - return NULL; - } - - if (pBuf->fileSize < pg->offset + size) { - pBuf->fileSize = pg->offset + size; - } - - pBuf->statis.flushBytes += size; - pBuf->statis.flushPages += 1; - } else if (pg->dirty) { - // length becomes greater, current space is not enough, allocate new place, otherwise, do nothing - if (pg->length < size) { - // 1. add current space to free list - SPageDiskInfo dinfo = {.length = pg->length, .offset = pg->offset}; - taosArrayPush(pBuf->pFree, &dinfo); - - // 2. allocate new position, and update the info pg->offset = allocatePositionInFile(pBuf, size); pBuf->nextPos += size; - } - // 3. write to disk. - int32_t ret = fseek(pBuf->file, pg->offset, SEEK_SET); - if (ret != 0) { - terrno = TAOS_SYSTEM_ERROR(errno); - return NULL; - } + int32_t ret = fseek(pBuf->file, pg->offset, SEEK_SET); + if (ret != 0) { + terrno = TAOS_SYSTEM_ERROR(errno); + return NULL; + } - ret = (int32_t)fwrite(t, 1, size, pBuf->file); - if (ret != size) { - terrno = TAOS_SYSTEM_ERROR(errno); - return NULL; - } + ret = (int32_t)fwrite(t, 1, size, pBuf->file); + if (ret != size) { + terrno = TAOS_SYSTEM_ERROR(errno); + return NULL; + } - if (pBuf->fileSize < pg->offset + size) { - pBuf->fileSize = pg->offset + size; - } + if (pBuf->fileSize < pg->offset + size) { + pBuf->fileSize = pg->offset + size; + } - pBuf->statis.flushBytes += size; - pBuf->statis.flushPages += 1; - } else { + pBuf->statis.flushBytes += size; + pBuf->statis.flushPages += 1; + } else { + // length becomes greater, current space is not enough, allocate new place, otherwise, do nothing + if (pg->length < size) { + // 1. add current space to free list + SPageDiskInfo dinfo = {.length = pg->length, .offset = pg->offset}; + taosArrayPush(pBuf->pFree, &dinfo); + + // 2. allocate new position, and update the info + pg->offset = allocatePositionInFile(pBuf, size); + pBuf->nextPos += size; + } + + // 3. write to disk. + int32_t ret = fseek(pBuf->file, pg->offset, SEEK_SET); + if (ret != 0) { + terrno = TAOS_SYSTEM_ERROR(errno); + return NULL; + } + + ret = (int32_t)fwrite(t, 1, size, pBuf->file); + if (ret != size) { + terrno = TAOS_SYSTEM_ERROR(errno); + return NULL; + } + + if (pBuf->fileSize < pg->offset + size) { + pBuf->fileSize = pg->offset + size; + } + + pBuf->statis.flushBytes += size; + pBuf->statis.flushPages += 1; + } + } else {// NOTE: the size may be -1, the this recycle page has not been flushed to disk yet. size = pg->length; } - assert(size >= 0); + ASSERT(size > 0 || (pg->offset == -1 && pg->length == -1)); char* pDataBuf = pg->pData; memset(pDataBuf, 0, pBuf->pageSize); @@ -313,13 +276,10 @@ static SPageInfo* registerPage(SDiskbasedBuf* pBuf, int32_t groupId, int32_t pag static SListNode* getEldestUnrefedPage(SDiskbasedBuf* pBuf) { SListIter iter = {0}; - tdListInitIter(pBuf->lruList, &iter, TD_LIST_BACKWARD); SListNode* pn = NULL; while((pn = tdListNext(&iter)) != NULL) { - assert(pn != NULL); - SPageInfo* pageInfo = *(SPageInfo**) pn->data; assert(pageInfo->pageId >= 0 && pageInfo->pn == pn); @@ -377,6 +337,56 @@ static FORCE_INLINE size_t getAllocPageSize(int32_t pageSize) { return pageSize + POINTER_BYTES + 2; } +static SPageInfo* getPageInfoFromPayload(void* page) { + int32_t offset = offsetof(SPageInfo, pData); + char* p = page - offset; + + SPageInfo* ppi = ((SPageInfo**) p)[0]; + return ppi; +} + +int32_t createDiskbasedBuf(SDiskbasedBuf** pBuf, int32_t pagesize, int32_t inMemBufSize, uint64_t qId, const char* dir) { + *pBuf = calloc(1, sizeof(SDiskbasedBuf)); + + SDiskbasedBuf* pResBuf = *pBuf; + if (pResBuf == NULL) { + return TSDB_CODE_OUT_OF_MEMORY; + } + + pResBuf->pageSize = pagesize; + pResBuf->numOfPages = 0; // all pages are in buffer in the first place + pResBuf->totalBufSize = 0; + pResBuf->inMemPages = inMemBufSize/pagesize; // maximum allowed pages, it is a soft limit. + pResBuf->allocateId = -1; + pResBuf->comp = true; + pResBuf->file = NULL; + pResBuf->qId = qId; + pResBuf->fileSize = 0; + pResBuf->pFree = taosArrayInit(4, sizeof(SFreeListItem)); + + // at least more than 2 pages must be in memory + assert(inMemBufSize >= pagesize * 2); + + pResBuf->lruList = tdListNew(POINTER_BYTES); + + // init id hash table + _hash_fn_t fn = taosGetDefaultHashFunction(TSDB_DATA_TYPE_INT); + pResBuf->groupSet = taosHashInit(10, fn, true, false); + pResBuf->assistBuf = malloc(pResBuf->pageSize + 2); // EXTRA BYTES + pResBuf->all = taosHashInit(10, fn, true, false); + + char path[PATH_MAX] = {0}; + taosGetTmpfilePath(dir, "paged-buf", path); + pResBuf->path = strdup(path); + + pResBuf->emptyDummyIdList = taosArrayInit(1, sizeof(int32_t)); + +// qDebug("QInfo:0x%"PRIx64" create resBuf for output, page size:%d, inmem buf pages:%d, file:%s", qId, pResBuf->pageSize, +// pResBuf->inMemPages, pResBuf->path); + + return TSDB_CODE_SUCCESS; +} + void* getNewBufPage(SDiskbasedBuf* pBuf, int32_t groupId, int32_t* pageId) { pBuf->statis.getPages += 1; @@ -386,6 +396,7 @@ void* getNewBufPage(SDiskbasedBuf* pBuf, int32_t groupId, int32_t* pageId) { // Failed to allocate a new buffer page, and there is an error occurs. if (availablePage == NULL) { + assert(0); return NULL; } } @@ -393,10 +404,6 @@ void* getNewBufPage(SDiskbasedBuf* pBuf, int32_t groupId, int32_t* pageId) { // register new id in this group *pageId = (++pBuf->allocateId); - if (*pageId == 11) { - printf("page is allocated, id:%d\n", *pageId); - } - // register page id info SPageInfo* pi = registerPage(pBuf, groupId, *pageId); @@ -443,7 +450,6 @@ void* getBufPage(SDiskbasedBuf* pBuf, int32_t id) { (*pi)->used = true; return (void *)(GET_DATA_PAYLOAD(*pi)); - } else { // not in memory assert((*pi)->pData == NULL && (*pi)->pn == NULL && (*pi)->length >= 0 && (*pi)->offset >= 0); @@ -477,15 +483,12 @@ void* getBufPage(SDiskbasedBuf* pBuf, int32_t id) { void releaseBufPage(SDiskbasedBuf* pBuf, void* page) { assert(pBuf != NULL && page != NULL); - int32_t offset = offsetof(SPageInfo, pData); - char* p = page - offset; - - SPageInfo* ppi = ((SPageInfo**) p)[0]; + SPageInfo* ppi = getPageInfoFromPayload(page); releaseBufPageInfo(pBuf, ppi); } void releaseBufPageInfo(SDiskbasedBuf* pBuf, SPageInfo* pi) { - assert(pi->pData != NULL && pi->used); + assert(pi->pData != NULL && pi->used == true); pi->used = false; pBuf->statis.releasePages += 1; @@ -549,6 +552,8 @@ void destroyDiskbasedBuf(SDiskbasedBuf* pBuf) { tdListFree(pBuf->lruList); taosArrayDestroy(pBuf->emptyDummyIdList); + taosArrayDestroy(pBuf->pFree); + taosHashCleanup(pBuf->groupSet); taosHashCleanup(pBuf->all); @@ -580,10 +585,7 @@ bool isAllDataInMemBuf(const SDiskbasedBuf* pBuf) { } void setBufPageDirty(void* pPage, bool dirty) { - int32_t offset = offsetof(SPageInfo, pData); - char* p = (char*)pPage - offset; - - SPageInfo* ppi = ((SPageInfo**) p)[0]; + SPageInfo* ppi = getPageInfoFromPayload(pPage); ppi->dirty = dirty; } @@ -591,6 +593,18 @@ void setBufPageCompressOnDisk(SDiskbasedBuf* pBuf, bool comp) { pBuf->comp = comp; } +void dBufSetBufPageRecycled(SDiskbasedBuf *pBuf, void* pPage) { + SPageInfo* ppi = getPageInfoFromPayload(pPage); + + ppi->used = false; + ppi->dirty = false; + + // it is a in-memory page that has not been flushed to disk yet. + if (ppi->length != -1 && ppi->offset != -1) { + SFreeListItem item = {.length = ppi->length, .offset = ppi->offset}; + taosArrayPush(pBuf->pFree, &item); + } +} void dBufSetPrintInfo(SDiskbasedBuf* pBuf) { pBuf->printStatis = true; @@ -618,3 +632,4 @@ void dBufPrintStatis(const SDiskbasedBuf* pBuf) { ps->getPages, ps->releasePages, ps->flushBytes / 1024.0f, ps->flushPages, ps->loadBytes / 1024.0f, ps->loadPages, ps->loadBytes / (1024.0 * ps->loadPages)); } + From 2a7e0171d04b06bc4c275aa3a70bdecdf5f78cb3 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 24 Feb 2022 17:18:56 +0800 Subject: [PATCH 21/25] [td-13039] support interval query. --- 2.0/src/query/inc/qExecutor.h | 2 +- 2.0/src/query/src/qExecutor.c | 4 +- include/util/tpagedbuf.h | 2 +- source/common/src/tep.c | 1 + source/libs/executor/inc/executil.h | 1 + source/libs/executor/inc/executorimpl.h | 18 +- source/libs/executor/src/executil.c | 1 - source/libs/executor/src/executorimpl.c | 534 ++++++++++---------- source/libs/executor/src/tlinearhash.c | 54 +- source/libs/executor/test/executorTests.cpp | 164 +++++- source/libs/executor/test/lhashTests.cpp | 10 +- source/libs/function/src/taggfunction.c | 2 +- source/util/src/tpagedbuf.c | 137 +++-- 13 files changed, 585 insertions(+), 345 deletions(-) diff --git a/2.0/src/query/inc/qExecutor.h b/2.0/src/query/inc/qExecutor.h index 9c738dad98..970b826303 100644 --- a/2.0/src/query/inc/qExecutor.h +++ b/2.0/src/query/inc/qExecutor.h @@ -589,7 +589,7 @@ SOperatorInfo* createTableSeqScanOperatorInfo(void* pTsdbQueryHandle, SQueryRunt SOperatorInfo* createAggregateOperatorInfo(SQueryRuntimeEnv* pRuntimeEnv, SOperatorInfo* upstream, SExprInfo* pExpr, int32_t numOfOutput); SOperatorInfo* createProjectOperatorInfo(SQueryRuntimeEnv* pRuntimeEnv, SOperatorInfo* upstream, SExprInfo* pExpr, int32_t numOfOutput); SOperatorInfo* createLimitOperatorInfo(SQueryRuntimeEnv* pRuntimeEnv, SOperatorInfo* upstream); -SOperatorInfo* createTimeIntervalOperatorInfo(SQueryRuntimeEnv* pRuntimeEnv, SOperatorInfo* upstream, SExprInfo* pExpr, int32_t numOfOutput); +SOperatorInfo* createIntervalOperatorInfo(SQueryRuntimeEnv* pRuntimeEnv, SOperatorInfo* upstream, SExprInfo* pExpr, int32_t numOfOutput); SOperatorInfo* createAllTimeIntervalOperatorInfo(SQueryRuntimeEnv* pRuntimeEnv, SOperatorInfo* upstream, SExprInfo* pExpr, int32_t numOfOutput); SOperatorInfo* createSWindowOperatorInfo(SQueryRuntimeEnv* pRuntimeEnv, SOperatorInfo* upstream, SExprInfo* pExpr, int32_t numOfOutput); SOperatorInfo* createFillOperatorInfo(SQueryRuntimeEnv* pRuntimeEnv, SOperatorInfo* upstream, SExprInfo* pExpr, int32_t numOfOutput, bool multigroupResult); diff --git a/2.0/src/query/src/qExecutor.c b/2.0/src/query/src/qExecutor.c index ca656b81ff..490584c75a 100644 --- a/2.0/src/query/src/qExecutor.c +++ b/2.0/src/query/src/qExecutor.c @@ -2166,7 +2166,7 @@ static int32_t setupQueryRuntimeEnv(SQueryRuntimeEnv *pRuntimeEnv, int32_t numOf } case OP_TimeWindow: { pRuntimeEnv->proot = - createTimeIntervalOperatorInfo(pRuntimeEnv, pRuntimeEnv->proot, pQueryAttr->pExpr1, pQueryAttr->numOfOutput); + createIntervalOperatorInfo(pRuntimeEnv, pRuntimeEnv->proot, pQueryAttr->pExpr1, pQueryAttr->numOfOutput); int32_t opType = pRuntimeEnv->proot->upstream[0]->operatorType; if (opType != OP_DummyInput && opType != OP_Join) { setTableScanFilterOperatorInfo(pRuntimeEnv->proot->upstream[0]->info, pRuntimeEnv->proot); @@ -6756,7 +6756,7 @@ SOperatorInfo* createLimitOperatorInfo(SQueryRuntimeEnv* pRuntimeEnv, SOperatorI return pOperator; } -SOperatorInfo* createTimeIntervalOperatorInfo(SQueryRuntimeEnv* pRuntimeEnv, SOperatorInfo* upstream, SExprInfo* pExpr, int32_t numOfOutput) { +SOperatorInfo* createIntervalOperatorInfo(SQueryRuntimeEnv* pRuntimeEnv, SOperatorInfo* upstream, SExprInfo* pExpr, int32_t numOfOutput) { STableIntervalOperatorInfo* pInfo = calloc(1, sizeof(STableIntervalOperatorInfo)); pInfo->pCtx = createSQLFunctionCtx(pRuntimeEnv, pExpr, numOfOutput, &pInfo->rowCellInfoOffset); diff --git a/include/util/tpagedbuf.h b/include/util/tpagedbuf.h index decf952bcd..76501b51ad 100644 --- a/include/util/tpagedbuf.h +++ b/include/util/tpagedbuf.h @@ -32,7 +32,7 @@ typedef struct SDiskbasedBuf SDiskbasedBuf; #define DEFAULT_INTERN_BUF_PAGE_SIZE (1024L) // in bytes typedef struct SFilePage { - int64_t num; + int32_t num; char data[]; } SFilePage; diff --git a/source/common/src/tep.c b/source/common/src/tep.c index 89d8127a63..d325f57c91 100644 --- a/source/common/src/tep.c +++ b/source/common/src/tep.c @@ -127,6 +127,7 @@ int32_t colDataAppend(SColumnInfoData* pColumnInfoData, uint32_t currentRow, con case TSDB_DATA_TYPE_USMALLINT: {*(int16_t*) p = *(int16_t*) pData;break;} case TSDB_DATA_TYPE_INT: case TSDB_DATA_TYPE_UINT: {*(int32_t*) p = *(int32_t*) pData;break;} + case TSDB_DATA_TYPE_TIMESTAMP: case TSDB_DATA_TYPE_BIGINT: case TSDB_DATA_TYPE_UBIGINT: {*(int64_t*) p = *(int64_t*) pData;break;} default: diff --git a/source/libs/executor/inc/executil.h b/source/libs/executor/inc/executil.h index e45e02cdd0..e729c868a7 100644 --- a/source/libs/executor/inc/executil.h +++ b/source/libs/executor/inc/executil.h @@ -68,6 +68,7 @@ typedef struct SResultRow { } SResultRow; typedef struct SResultRowInfo { + SResultRow *pCurResult; // current active result row info SResultRow** pResult; // result list // int16_t type:8; // data type for hash key int32_t size; // number of result set diff --git a/source/libs/executor/inc/executorimpl.h b/source/libs/executor/inc/executorimpl.h index 1289de004d..56fc1fcdb0 100644 --- a/source/libs/executor/inc/executorimpl.h +++ b/source/libs/executor/inc/executorimpl.h @@ -453,7 +453,19 @@ typedef struct SAggSupporter { SResultRowPool *pool; // The window result objects pool, all the resultRow Objects are allocated and managed by this object. } SAggSupporter; -typedef struct SOptrBasicInfo STableIntervalOperatorInfo; +typedef struct STableIntervalOperatorInfo { + SOptrBasicInfo binfo; + SDiskbasedBuf *pResultBuf; // query result buffer based on blocked-wised disk file + SGroupResInfo groupResInfo; + SInterval interval; + STimeWindow win; + int32_t precision; + bool timeWindowInterpo; + char **pRow; + SAggSupporter aggSup; + STableQueryInfo *pCurrent; + int32_t order; +} STableIntervalOperatorInfo; typedef struct SAggOperatorInfo { SOptrBasicInfo binfo; @@ -606,8 +618,8 @@ SOperatorInfo* createMultiTableAggOperatorInfo(SOperatorInfo* downstream, SArray SOperatorInfo* createProjectOperatorInfo(STaskRuntimeEnv* pRuntimeEnv, SOperatorInfo* downstream, SExprInfo* pExpr, int32_t numOfOutput); SOperatorInfo* createLimitOperatorInfo(STaskRuntimeEnv* pRuntimeEnv, SOperatorInfo* downstream); -SOperatorInfo* createTimeIntervalOperatorInfo(STaskRuntimeEnv* pRuntimeEnv, SOperatorInfo* downstream, SExprInfo* pExpr, - int32_t numOfOutput); +SOperatorInfo* createIntervalOperatorInfo(SOperatorInfo* downstream, SArray* pExprInfo, SExecTaskInfo* pTaskInfo); + SOperatorInfo* createAllTimeIntervalOperatorInfo(STaskRuntimeEnv* pRuntimeEnv, SOperatorInfo* downstream, SExprInfo* pExpr, int32_t numOfOutput); SOperatorInfo* createSWindowOperatorInfo(STaskRuntimeEnv* pRuntimeEnv, SOperatorInfo* downstream, SExprInfo* pExpr, diff --git a/source/libs/executor/src/executil.c b/source/libs/executor/src/executil.c index e2675115e0..1b901ee9f6 100644 --- a/source/libs/executor/src/executil.c +++ b/source/libs/executor/src/executil.c @@ -54,7 +54,6 @@ int32_t getOutputInterResultBufSize(STaskAttr* pQueryAttr) { } int32_t initResultRowInfo(SResultRowInfo *pResultRowInfo, int32_t size) { -// pResultRowInfo->type = type; pResultRowInfo->size = 0; pResultRowInfo->curPos = -1; pResultRowInfo->capacity = size; diff --git a/source/libs/executor/src/executorimpl.c b/source/libs/executor/src/executorimpl.c index 6f1a05805a..bd2e0893af 100644 --- a/source/libs/executor/src/executorimpl.c +++ b/source/libs/executor/src/executorimpl.c @@ -141,19 +141,19 @@ static int32_t getExprFunctionId(SExprInfo *pExprInfo) { return 0; } -static void getNextTimeWindow(STaskAttr* pQueryAttr, STimeWindow* tw) { - int32_t factor = GET_FORWARD_DIRECTION_FACTOR(pQueryAttr->order.order); - if (pQueryAttr->interval.intervalUnit != 'n' && pQueryAttr->interval.intervalUnit != 'y') { - tw->skey += pQueryAttr->interval.sliding * factor; - tw->ekey = tw->skey + pQueryAttr->interval.interval - 1; +static void getNextTimeWindow(SInterval* pInterval, int32_t precision, int32_t order, STimeWindow* tw) { + int32_t factor = GET_FORWARD_DIRECTION_FACTOR(order); + if (pInterval->intervalUnit != 'n' && pInterval->intervalUnit != 'y') { + tw->skey += pInterval->sliding * factor; + tw->ekey = tw->skey + pInterval->interval - 1; return; } - int64_t key = tw->skey, interval = pQueryAttr->interval.interval; + int64_t key = tw->skey, interval = pInterval->interval; //convert key to second - key = convertTimePrecision(key, pQueryAttr->precision, TSDB_TIME_PRECISION_MILLI) / 1000; + key = convertTimePrecision(key, precision, TSDB_TIME_PRECISION_MILLI) / 1000; - if (pQueryAttr->interval.intervalUnit == 'y') { + if (pInterval->intervalUnit == 'y') { interval *= 12; } @@ -164,12 +164,12 @@ static void getNextTimeWindow(STaskAttr* pQueryAttr, STimeWindow* tw) { int mon = (int)(tm.tm_year * 12 + tm.tm_mon + interval * factor); tm.tm_year = mon / 12; tm.tm_mon = mon % 12; - tw->skey = convertTimePrecision((int64_t)mktime(&tm) * 1000L, TSDB_TIME_PRECISION_MILLI, pQueryAttr->precision); + tw->skey = convertTimePrecision((int64_t)mktime(&tm) * 1000L, TSDB_TIME_PRECISION_MILLI, precision); mon = (int)(mon + interval); tm.tm_year = mon / 12; tm.tm_mon = mon % 12; - tw->ekey = convertTimePrecision((int64_t)mktime(&tm) * 1000L, TSDB_TIME_PRECISION_MILLI, pQueryAttr->precision); + tw->ekey = convertTimePrecision((int64_t)mktime(&tm) * 1000L, TSDB_TIME_PRECISION_MILLI, precision); tw->ekey -= 1; } @@ -179,7 +179,7 @@ static void setResultOutputBuf(STaskRuntimeEnv* pRuntimeEnv, SResultRow* pResult int32_t numOfCols, int32_t* rowCellInfoOffset); void setResultRowOutputBufInitCtx(STaskRuntimeEnv *pRuntimeEnv, SResultRow *pResult, SqlFunctionCtx* pCtx, int32_t numOfOutput, int32_t* rowCellInfoOffset); -static bool functionNeedToExecute(STaskRuntimeEnv *pRuntimeEnv, SqlFunctionCtx *pCtx); +static bool functionNeedToExecute(SqlFunctionCtx *pCtx); static void setBlockStatisInfo(SqlFunctionCtx *pCtx, SSDataBlock* pSDataBlock, SColumn* pColumn); @@ -222,7 +222,8 @@ static int32_t getGroupbyColumnIndex(SGroupbyExpr *pGroupbyExpr, SSDataBlock* pD static int32_t setGroupResultOutputBuf(STaskRuntimeEnv *pRuntimeEnv, SOptrBasicInfo *binf, int32_t numOfCols, char *pData, int16_t type, int16_t bytes, int32_t groupIndex); static void initCtxOutputBuffer(SqlFunctionCtx* pCtx, int32_t size); -static void getAlignQueryTimeWindow(STaskAttr *pQueryAttr, int64_t key, int64_t keyFirst, int64_t keyLast, STimeWindow *win); +static void getAlignQueryTimeWindow(SInterval* pInterval, int32_t precision, int64_t key, int64_t keyFirst, int64_t keyLast, STimeWindow *win); + static void setResultBufSize(STaskAttr* pQueryAttr, SRspResultInfo* pResultInfo); static void setCtxTagForJoin(STaskRuntimeEnv* pRuntimeEnv, SqlFunctionCtx* pCtx, SExprInfo* pExprInfo, void* pTable); static void setParamForStableStddev(STaskRuntimeEnv* pRuntimeEnv, SqlFunctionCtx* pCtx, int32_t numOfOutput, SExprInfo* pExpr); @@ -394,6 +395,10 @@ static void prepareResultListBuffer(SResultRowInfo* pResultRowInfo, jmp_buf env) newCapacity = (int64_t)(pResultRowInfo->capacity * 1.5); } + if (newCapacity == pResultRowInfo->capacity) { + newCapacity += 4; + } + char *t = realloc(pResultRowInfo->pResult, (size_t)(newCapacity * POINTER_BYTES)); if (t == NULL) { longjmp(env, TSDB_CODE_QRY_OUT_OF_MEMORY); @@ -593,19 +598,19 @@ static SResultRow* doSetResultOutBufByKey_rv(SResultRowInfo* pResultRowInfo, int return pResultRowInfo->pResult[pResultRowInfo->curPos]; } -static void getInitialStartTimeWindow(STaskAttr* pQueryAttr, TSKEY ts, STimeWindow* w) { - if (QUERY_IS_ASC_QUERY(pQueryAttr)) { - getAlignQueryTimeWindow(pQueryAttr, ts, ts, pQueryAttr->window.ekey, w); +static void getInitialStartTimeWindow(SInterval* pInterval, int32_t precision, TSKEY ts, STimeWindow* w, TSKEY ekey, bool ascQuery) { + if (ascQuery) { + getAlignQueryTimeWindow(pInterval, precision, ts, ts, ekey, w); } else { // the start position of the first time window in the endpoint that spreads beyond the queried last timestamp - getAlignQueryTimeWindow(pQueryAttr, ts, pQueryAttr->window.ekey, ts, w); + getAlignQueryTimeWindow(pInterval, precision, ts, ekey, ts, w); int64_t key = w->skey; while(key < ts) { // moving towards end - if (pQueryAttr->interval.intervalUnit == 'n' || pQueryAttr->interval.intervalUnit == 'y') { - key = taosTimeAdd(key, pQueryAttr->interval.sliding, pQueryAttr->interval.slidingUnit, pQueryAttr->precision); + if (pInterval->intervalUnit == 'n' || pInterval->intervalUnit == 'y') { + key = taosTimeAdd(key, pInterval->sliding, pInterval->slidingUnit, precision); } else { - key += pQueryAttr->interval.sliding; + key += pInterval->sliding; } if (key >= ts) { @@ -618,39 +623,39 @@ static void getInitialStartTimeWindow(STaskAttr* pQueryAttr, TSKEY ts, STimeWind } // get the correct time window according to the handled timestamp -static STimeWindow getActiveTimeWindow(SResultRowInfo * pResultRowInfo, int64_t ts, STaskAttr *pQueryAttr) { +static STimeWindow getActiveTimeWindow(SResultRowInfo * pResultRowInfo, int64_t ts, SInterval* pInterval, int32_t precision, STimeWindow* win) { STimeWindow w = {0}; if (pResultRowInfo->curPos == -1) { // the first window, from the previous stored value - getInitialStartTimeWindow(pQueryAttr, ts, &w); + getInitialStartTimeWindow(pInterval, precision, ts, &w, win->ekey, true); - if (pQueryAttr->interval.intervalUnit == 'n' || pQueryAttr->interval.intervalUnit == 'y') { - w.ekey = taosTimeAdd(w.skey, pQueryAttr->interval.interval, pQueryAttr->interval.intervalUnit, pQueryAttr->precision) - 1; + if (pInterval->intervalUnit == 'n' || pInterval->intervalUnit == 'y') { + w.ekey = taosTimeAdd(w.skey, pInterval->interval, pInterval->intervalUnit, precision) - 1; } else { - w.ekey = w.skey + pQueryAttr->interval.interval - 1; + w.ekey = w.skey + pInterval->interval - 1; } } else { w = getResultRow(pResultRowInfo, pResultRowInfo->curPos)->win; } if (w.skey > ts || w.ekey < ts) { - if (pQueryAttr->interval.intervalUnit == 'n' || pQueryAttr->interval.intervalUnit == 'y') { - w.skey = taosTimeTruncate(ts, &pQueryAttr->interval, pQueryAttr->precision); - w.ekey = taosTimeAdd(w.skey, pQueryAttr->interval.interval, pQueryAttr->interval.intervalUnit, pQueryAttr->precision) - 1; + if (pInterval->intervalUnit == 'n' || pInterval->intervalUnit == 'y') { + w.skey = taosTimeTruncate(ts, pInterval, precision); + w.ekey = taosTimeAdd(w.skey, pInterval->interval, pInterval->intervalUnit, precision) - 1; } else { int64_t st = w.skey; if (st > ts) { - st -= ((st - ts + pQueryAttr->interval.sliding - 1) / pQueryAttr->interval.sliding) * pQueryAttr->interval.sliding; + st -= ((st - ts + pInterval->sliding - 1) / pInterval->sliding) * pInterval->sliding; } - int64_t et = st + pQueryAttr->interval.interval - 1; + int64_t et = st + pInterval->interval - 1; if (et < ts) { - st += ((ts - et + pQueryAttr->interval.sliding - 1) / pQueryAttr->interval.sliding) * pQueryAttr->interval.sliding; + st += ((ts - et + pInterval->sliding - 1) / pInterval->sliding) * pInterval->sliding; } w.skey = st; - w.ekey = w.skey + pQueryAttr->interval.interval - 1; + w.ekey = w.skey + pInterval->interval - 1; } } @@ -658,10 +663,7 @@ static STimeWindow getActiveTimeWindow(SResultRowInfo * pResultRowInfo, int64_t * query border check, skey should not be bounded by the query time range, since the value skey will * be used as the time window index value. So we only change ekey of time window accordingly. */ - if (w.ekey > pQueryAttr->window.ekey && QUERY_IS_ASC_QUERY(pQueryAttr)) { - w.ekey = pQueryAttr->window.ekey; - } - +// ASSERT(win->skey <= win->ekey); // todo no need this return w; } @@ -670,7 +672,7 @@ static STimeWindow getCurrentActiveTimeWindow(SResultRowInfo * pResultRowInfo, i STimeWindow w = {0}; if (pResultRowInfo->curPos == -1) { // the first window, from the previous stored value - getInitialStartTimeWindow(pQueryAttr, ts, &w); +// getInitialStartTimeWindow(pQueryAttr, ts, &w); if (pQueryAttr->interval.intervalUnit == 'n' || pQueryAttr->interval.intervalUnit == 'y') { w.ekey = taosTimeAdd(w.skey, pQueryAttr->interval.interval, pQueryAttr->interval.intervalUnit, pQueryAttr->precision) - 1; @@ -772,6 +774,37 @@ static int32_t setResultOutputBufByKey(STaskRuntimeEnv *pRuntimeEnv, SResultRowI return TSDB_CODE_SUCCESS; } +static void setResultRowOutputBufInitCtx_rv(SDiskbasedBuf * pBuf, SResultRow *pResult, SqlFunctionCtx* pCtx, int32_t numOfOutput, int32_t* rowCellInfoOffset); + +static int32_t setResultOutputBufByKey_rv(SResultRowInfo *pResultRowInfo, int64_t id, STimeWindow *win, + bool masterscan, SResultRow **pResult, int64_t tableGroupId, SqlFunctionCtx* pCtx, + int32_t numOfOutput, int32_t* rowCellInfoOffset, SDiskbasedBuf *pBuf, SAggSupporter *pAggSup, SExecTaskInfo* pTaskInfo) { + assert(win->skey <= win->ekey); + SResultRow *pResultRow = doSetResultOutBufByKey_rv(pResultRowInfo, id, (char *)&win->skey, TSDB_KEYSIZE, masterscan, tableGroupId, + pTaskInfo, true, pAggSup); + + if (pResultRow == NULL) { + *pResult = NULL; + return TSDB_CODE_SUCCESS; + } + + // not assign result buffer yet, add new result buffer + if (pResultRow->pageId == -1) { // todo intermediate result size + int32_t ret = addNewWindowResultBuf(pResultRow, pBuf, (int32_t) tableGroupId, 0); + if (ret != TSDB_CODE_SUCCESS) { + return -1; + } + } + + // set time window for current result + pResultRow->win = (*win); + *pResult = pResultRow; + setResultRowOutputBufInitCtx_rv(pBuf, pResultRow, pCtx, numOfOutput, rowCellInfoOffset); + + return TSDB_CODE_SUCCESS; +} + + static void setResultRowInterpo(SResultRow* pResult, SResultTsInterpType type) { assert(pResult != NULL && (type == RESULT_ROW_START_INTERP || type == RESULT_ROW_END_INTERP)); if (type == RESULT_ROW_START_INTERP) { @@ -873,48 +906,44 @@ static void doUpdateResultRowIndex(SResultRowInfo*pResultRowInfo, TSKEY lastKey, } } -static void updateResultRowInfoActiveIndex(SResultRowInfo* pResultRowInfo, STaskAttr* pQueryAttr, TSKEY lastKey) { - bool ascQuery = QUERY_IS_ASC_QUERY(pQueryAttr); - if ((lastKey > pQueryAttr->window.ekey && ascQuery) || (lastKey < pQueryAttr->window.ekey && (!ascQuery))) { +static void updateResultRowInfoActiveIndex(SResultRowInfo* pResultRowInfo, const STimeWindow* pWin, TSKEY lastKey, bool ascQuery, bool interp) { + if ((lastKey > pWin->ekey && ascQuery) || (lastKey < pWin->ekey && (!ascQuery))) { closeAllResultRows(pResultRowInfo); pResultRowInfo->curPos = pResultRowInfo->size - 1; } else { int32_t step = ascQuery ? 1 : -1; - doUpdateResultRowIndex(pResultRowInfo, lastKey - step, ascQuery, pQueryAttr->timeWindowInterpo); + doUpdateResultRowIndex(pResultRowInfo, lastKey - step, ascQuery, interp); } } -static int32_t getNumOfRowsInTimeWindow(STaskRuntimeEnv* pRuntimeEnv, SDataBlockInfo *pDataBlockInfo, TSKEY *pPrimaryColumn, - int32_t startPos, TSKEY ekey, __block_search_fn_t searchFn, bool updateLastKey) { +static int32_t getNumOfRowsInTimeWindow(SDataBlockInfo *pDataBlockInfo, TSKEY *pPrimaryColumn, + int32_t startPos, TSKEY ekey, __block_search_fn_t searchFn, STableQueryInfo* item, int32_t order) { assert(startPos >= 0 && startPos < pDataBlockInfo->rows); - STaskAttr* pQueryAttr = pRuntimeEnv->pQueryAttr; - STableQueryInfo* item = pRuntimeEnv->current; int32_t num = -1; - int32_t order = pQueryAttr->order.order; int32_t step = GET_FORWARD_DIRECTION_FACTOR(order); - if (QUERY_IS_ASC_QUERY(pQueryAttr)) { + if (order == TSDB_ORDER_ASC) { if (ekey < pDataBlockInfo->window.ekey && pPrimaryColumn) { num = getForwardStepsInBlock(pDataBlockInfo->rows, searchFn, ekey, startPos, order, pPrimaryColumn); - if (updateLastKey) { // update the last key + if (item != NULL) { item->lastKey = pPrimaryColumn[startPos + (num - 1)] + step; } } else { num = pDataBlockInfo->rows - startPos; - if (updateLastKey) { + if (item != NULL) { item->lastKey = pDataBlockInfo->window.ekey + step; } } } else { // desc if (ekey > pDataBlockInfo->window.skey && pPrimaryColumn) { num = getForwardStepsInBlock(pDataBlockInfo->rows, searchFn, ekey, startPos, order, pPrimaryColumn); - if (updateLastKey) { // update the last key + if (item != NULL) { item->lastKey = pPrimaryColumn[startPos - (num - 1)] + step; } } else { num = startPos + 1; - if (updateLastKey) { + if (item != NULL) { item->lastKey = pDataBlockInfo->window.skey + step; } } @@ -924,22 +953,18 @@ static int32_t getNumOfRowsInTimeWindow(STaskRuntimeEnv* pRuntimeEnv, SDataBlock return num; } -static void doApplyFunctions(STaskRuntimeEnv* pRuntimeEnv, SqlFunctionCtx* pCtx, STimeWindow* pWin, int32_t offset, - int32_t forwardStep, TSKEY* tsCol, int32_t numOfTotal, int32_t numOfOutput) { - STaskAttr *pQueryAttr = pRuntimeEnv->pQueryAttr; - bool hasAggregates = pCtx[0].isAggSet; - +static void doApplyFunctions(SqlFunctionCtx* pCtx, STimeWindow* pWin, int32_t offset, int32_t forwardStep, TSKEY* tsCol, + int32_t numOfTotal, int32_t numOfOutput, int32_t order) { for (int32_t k = 0; k < numOfOutput; ++k) { pCtx[k].size = forwardStep; pCtx[k].startTs = pWin->skey; // keep it temporarialy - char* start = NULL;//pCtx[k].pInput; + int32_t startOffset = pCtx[k].startRow; + bool hasAgg = pCtx[k].isAggSet; - int32_t pos = (QUERY_IS_ASC_QUERY(pQueryAttr)) ? offset : offset - (forwardStep - 1); - if (pCtx[k].pInput != NULL) { -// pCtx[k].pInput = (char *)pCtx[k].pInput + pos * pCtx[k].inputBytes; - } + int32_t pos = (order == TSDB_ORDER_ASC) ? offset : offset - (forwardStep - 1); + pCtx[k].startRow = pos; if (tsCol != NULL) { pCtx[k].ptsList = &tsCol[pos]; @@ -951,88 +976,80 @@ static void doApplyFunctions(STaskRuntimeEnv* pRuntimeEnv, SqlFunctionCtx* pCtx, pCtx[k].isAggSet = false; } - if (functionNeedToExecute(pRuntimeEnv, &pCtx[k])) { + if (functionNeedToExecute(&pCtx[k])) { pCtx[k].fpSet->addInput(&pCtx[k]); } // restore it - pCtx[k].isAggSet = hasAggregates; -// pCtx[k].pInput = start; + pCtx[k].isAggSet = hasAgg; + pCtx[k].startRow = startOffset; } } -static int32_t getNextQualifiedWindow(STaskAttr* pQueryAttr, STimeWindow* pNext, SDataBlockInfo* pDataBlockInfo, - TSKEY* primaryKeys, __block_search_fn_t searchFn, int32_t prevPosition) { - getNextTimeWindow(pQueryAttr, pNext); +static int32_t getNextQualifiedWindow(SInterval* pInterval, STimeWindow* pNext, SDataBlockInfo* pDataBlockInfo, + TSKEY* primaryKeys, int32_t prevPosition, STableIntervalOperatorInfo* pInfo) { + int32_t order = pInfo->order; + bool ascQuery = (order == TSDB_ORDER_ASC); + + int32_t precision = pInfo->precision; + getNextTimeWindow(pInterval, precision, order, pNext); // next time window is not in current block - if ((pNext->skey > pDataBlockInfo->window.ekey && QUERY_IS_ASC_QUERY(pQueryAttr)) || - (pNext->ekey < pDataBlockInfo->window.skey && !QUERY_IS_ASC_QUERY(pQueryAttr))) { + if ((pNext->skey > pDataBlockInfo->window.ekey && order == TSDB_ORDER_ASC) || + (pNext->ekey < pDataBlockInfo->window.skey && order == TSDB_ORDER_DESC)) { return -1; } - TSKEY startKey = -1; - if (QUERY_IS_ASC_QUERY(pQueryAttr)) { - startKey = pNext->skey; - if (startKey < pQueryAttr->window.skey) { - startKey = pQueryAttr->window.skey; - } - } else { - startKey = pNext->ekey; - if (startKey > pQueryAttr->window.skey) { - startKey = pQueryAttr->window.skey; - } - } - + TSKEY startKey = ascQuery? pNext->skey:pNext->ekey; int32_t startPos = 0; // tumbling time window query, a special case of sliding time window query - if (pQueryAttr->interval.sliding == pQueryAttr->interval.interval && prevPosition != -1) { - int32_t factor = GET_FORWARD_DIRECTION_FACTOR(pQueryAttr->order.order); + if (pInterval->sliding == pInterval->interval && prevPosition != -1) { + int32_t factor = GET_FORWARD_DIRECTION_FACTOR(order); startPos = prevPosition + factor; } else { - if (startKey <= pDataBlockInfo->window.skey && QUERY_IS_ASC_QUERY(pQueryAttr)) { + if (startKey <= pDataBlockInfo->window.skey && ascQuery) { startPos = 0; - } else if (startKey >= pDataBlockInfo->window.ekey && !QUERY_IS_ASC_QUERY(pQueryAttr)) { + } else if (startKey >= pDataBlockInfo->window.ekey && !ascQuery) { startPos = pDataBlockInfo->rows - 1; } else { - startPos = searchFn((char *)primaryKeys, pDataBlockInfo->rows, startKey, pQueryAttr->order.order); + startPos = binarySearchForKey((char *)primaryKeys, pDataBlockInfo->rows, startKey, order); } } /* interp query with fill should not skip time window */ - if (pQueryAttr->pointInterpQuery && pQueryAttr->fillType != TSDB_FILL_NONE) { - return startPos; - } +// if (pQueryAttr->pointInterpQuery && pQueryAttr->fillType != TSDB_FILL_NONE) { +// return startPos; +// } /* * This time window does not cover any data, try next time window, * this case may happen when the time window is too small */ if (primaryKeys == NULL) { - if (QUERY_IS_ASC_QUERY(pQueryAttr)) { + if (ascQuery) { assert(pDataBlockInfo->window.skey <= pNext->ekey); } else { assert(pDataBlockInfo->window.ekey >= pNext->skey); } } else { - if (QUERY_IS_ASC_QUERY(pQueryAttr) && primaryKeys[startPos] > pNext->ekey) { + if (ascQuery && primaryKeys[startPos] > pNext->ekey) { TSKEY next = primaryKeys[startPos]; - if (pQueryAttr->interval.intervalUnit == 'n' || pQueryAttr->interval.intervalUnit == 'y') { - pNext->skey = taosTimeTruncate(next, &pQueryAttr->interval, pQueryAttr->precision); - pNext->ekey = taosTimeAdd(pNext->skey, pQueryAttr->interval.interval, pQueryAttr->interval.intervalUnit, pQueryAttr->precision) - 1; + if (pInterval->intervalUnit == 'n' || pInterval->intervalUnit == 'y') { + pNext->skey = taosTimeTruncate(next, pInterval, precision); + pNext->ekey = taosTimeAdd(pNext->skey, pInterval->interval, pInterval->intervalUnit, precision) - 1; } else { - pNext->ekey += ((next - pNext->ekey + pQueryAttr->interval.sliding - 1)/pQueryAttr->interval.sliding) * pQueryAttr->interval.sliding; - pNext->skey = pNext->ekey - pQueryAttr->interval.interval + 1; + pNext->ekey += ((next - pNext->ekey + pInterval->sliding - 1)/pInterval->sliding) * pInterval->sliding; + pNext->skey = pNext->ekey - pInterval->interval + 1; } - } else if ((!QUERY_IS_ASC_QUERY(pQueryAttr)) && primaryKeys[startPos] < pNext->skey) { + } else if ((!ascQuery) && primaryKeys[startPos] < pNext->skey) { TSKEY next = primaryKeys[startPos]; - if (pQueryAttr->interval.intervalUnit == 'n' || pQueryAttr->interval.intervalUnit == 'y') { - pNext->skey = taosTimeTruncate(next, &pQueryAttr->interval, pQueryAttr->precision); - pNext->ekey = taosTimeAdd(pNext->skey, pQueryAttr->interval.interval, pQueryAttr->interval.intervalUnit, pQueryAttr->precision) - 1; + if (pInterval->intervalUnit == 'n' || pInterval->intervalUnit == 'y') { + pNext->skey = taosTimeTruncate(next, pInterval, precision); + pNext->ekey = taosTimeAdd(pNext->skey, pInterval->interval, pInterval->intervalUnit, precision) - 1; } else { - pNext->skey -= ((pNext->skey - next + pQueryAttr->interval.sliding - 1) / pQueryAttr->interval.sliding) * pQueryAttr->interval.sliding; - pNext->ekey = pNext->skey + pQueryAttr->interval.interval - 1; + pNext->skey -= ((pNext->skey - next + pInterval->sliding - 1) / pInterval->sliding) * pInterval->sliding; + pNext->ekey = pNext->skey + pInterval->interval - 1; } } } @@ -1069,23 +1086,19 @@ static void setNotInterpoWindowKey(SqlFunctionCtx* pCtx, int32_t numOfOutput, in } } -static void saveDataBlockLastRow(STaskRuntimeEnv* pRuntimeEnv, SDataBlockInfo* pDataBlockInfo, SArray* pDataBlock, - int32_t rowIndex) { +static void saveDataBlockLastRow(char** pRow, SArray* pDataBlock, int32_t rowIndex, int32_t numOfCols) { if (pDataBlock == NULL) { return; } - STaskAttr* pQueryAttr = pRuntimeEnv->pQueryAttr; - for (int32_t k = 0; k < pQueryAttr->numOfCols; ++k) { + for (int32_t k = 0; k < numOfCols; ++k) { SColumnInfoData *pColInfo = taosArrayGet(pDataBlock, k); - memcpy(pRuntimeEnv->prevRow[k], ((char*)pColInfo->pData) + (pColInfo->info.bytes * rowIndex), pColInfo->info.bytes); + memcpy(pRow[k], ((char*)pColInfo->pData) + (pColInfo->info.bytes * rowIndex), pColInfo->info.bytes); } } -static TSKEY getStartTsKey(STaskAttr* pQueryAttr, STimeWindow* win, const TSKEY* tsCols, int32_t rows) { +static TSKEY getStartTsKey(STimeWindow* win, const TSKEY* tsCols, int32_t rows, bool ascQuery) { TSKEY ts = TSKEY_INITIAL_VAL; - - bool ascQuery = QUERY_IS_ASC_QUERY(pQueryAttr); if (tsCols == NULL) { ts = ascQuery? win->skey : win->ekey; } else { @@ -1191,7 +1204,7 @@ static void doSetInputDataBlock(SOperatorInfo* pOperator, SqlFunctionCtx* pCtx, static void doAggregateImpl(SOperatorInfo* pOperator, TSKEY startTs, SqlFunctionCtx* pCtx, SSDataBlock* pSDataBlock) { for (int32_t k = 0; k < pOperator->numOfOutput; ++k) { - if (functionNeedToExecute(NULL, &pCtx[k])) { + if (functionNeedToExecute(&pCtx[k])) { pCtx[k].startTs = startTs;// this can be set during create the struct pCtx[k].fpSet->addInput(&pCtx[k]); } @@ -1350,20 +1363,19 @@ static bool setTimeWindowInterpolationEndTs(SOperatorInfo* pOperatorInfo, SqlFun } static void doWindowBorderInterpolation(SOperatorInfo* pOperatorInfo, SSDataBlock* pBlock, SqlFunctionCtx* pCtx, - SResultRow* pResult, STimeWindow* win, int32_t startPos, int32_t forwardStep) { - STaskRuntimeEnv* pRuntimeEnv = pOperatorInfo->pRuntimeEnv; - STaskAttr* pQueryAttr = pRuntimeEnv->pQueryAttr; - if (!pQueryAttr->timeWindowInterpo) { + SResultRow* pResult, STimeWindow* win, int32_t startPos, int32_t forwardStep, int32_t order, bool timeWindowInterpo) { + if (!timeWindowInterpo) { return; } assert(pBlock != NULL); - int32_t step = GET_FORWARD_DIRECTION_FACTOR(pQueryAttr->order.order); + int32_t step = GET_FORWARD_DIRECTION_FACTOR(order); if (pBlock->pDataBlock == NULL){ // tscError("pBlock->pDataBlock == NULL"); return; } + SColumnInfoData *pColInfo = taosArrayGet(pBlock->pDataBlock, 0); TSKEY *tsCols = (TSKEY *)(pColInfo->pData); @@ -1376,38 +1388,37 @@ static void doWindowBorderInterpolation(SOperatorInfo* pOperatorInfo, SSDataBloc setResultRowInterpo(pResult, RESULT_ROW_START_INTERP); } } else { - setNotInterpoWindowKey(pCtx, pQueryAttr->numOfOutput, RESULT_ROW_START_INTERP); + setNotInterpoWindowKey(pCtx, pOperatorInfo->numOfOutput, RESULT_ROW_START_INTERP); } // point interpolation does not require the end key time window interpolation. - if (pQueryAttr->pointInterpQuery) { - return; - } +// if (pointInterpQuery) { +// return; +// } // interpolation query does not generate the time window end interpolation done = resultRowInterpolated(pResult, RESULT_ROW_END_INTERP); if (!done) { int32_t endRowIndex = startPos + (forwardStep - 1) * step; - TSKEY endKey = QUERY_IS_ASC_QUERY(pQueryAttr)? pBlock->info.window.ekey:pBlock->info.window.skey; + TSKEY endKey = (order == TSDB_ORDER_ASC)? pBlock->info.window.ekey:pBlock->info.window.skey; bool interp = setTimeWindowInterpolationEndTs(pOperatorInfo, pCtx, endRowIndex, pBlock->pDataBlock, tsCols, endKey, win); if (interp) { setResultRowInterpo(pResult, RESULT_ROW_END_INTERP); } } else { - setNotInterpoWindowKey(pCtx, pQueryAttr->numOfOutput, RESULT_ROW_END_INTERP); + setNotInterpoWindowKey(pCtx, pOperatorInfo->numOfOutput, RESULT_ROW_END_INTERP); } } static void hashIntervalAgg(SOperatorInfo* pOperatorInfo, SResultRowInfo* pResultRowInfo, SSDataBlock* pSDataBlock, int32_t tableGroupId) { STableIntervalOperatorInfo* pInfo = (STableIntervalOperatorInfo*) pOperatorInfo->info; - STaskRuntimeEnv* pRuntimeEnv = pOperatorInfo->pRuntimeEnv; - int32_t numOfOutput = pOperatorInfo->numOfOutput; - STaskAttr* pQueryAttr = pRuntimeEnv->pQueryAttr; + SExecTaskInfo* pTaskInfo = pOperatorInfo->pTaskInfo; + int32_t numOfOutput = pOperatorInfo->numOfOutput; - int32_t step = GET_FORWARD_DIRECTION_FACTOR(pQueryAttr->order.order); - bool ascQuery = QUERY_IS_ASC_QUERY(pQueryAttr); + int32_t step = 1; + bool ascQuery = true; int32_t prevIndex = pResultRowInfo->curPos; @@ -1420,26 +1431,26 @@ static void hashIntervalAgg(SOperatorInfo* pOperatorInfo, SResultRowInfo* pResul } int32_t startPos = ascQuery? 0 : (pSDataBlock->info.rows - 1); - TSKEY ts = getStartTsKey(pQueryAttr, &pSDataBlock->info.window, tsCols, pSDataBlock->info.rows); + TSKEY ts = getStartTsKey(&pSDataBlock->info.window, tsCols, pSDataBlock->info.rows, ascQuery); - STimeWindow win = getActiveTimeWindow(pResultRowInfo, ts, pQueryAttr); - bool masterScan = IS_MAIN_SCAN(pRuntimeEnv); + STimeWindow win = getActiveTimeWindow(pResultRowInfo, ts, &pInfo->interval, pInfo->precision, &pInfo->win); + bool masterScan = true; SResultRow* pResult = NULL; - int32_t ret = setResultOutputBufByKey(pRuntimeEnv, pResultRowInfo, pSDataBlock->info.uid, &win, masterScan, &pResult, tableGroupId, pInfo->pCtx, - numOfOutput, pInfo->rowCellInfoOffset); + int32_t ret = setResultOutputBufByKey_rv(pResultRowInfo, pSDataBlock->info.uid, &win, masterScan, &pResult, tableGroupId, pInfo->binfo.pCtx, + numOfOutput, pInfo->binfo.rowCellInfoOffset, pInfo->pResultBuf, &pInfo->aggSup, pTaskInfo); if (ret != TSDB_CODE_SUCCESS || pResult == NULL) { - longjmp(pRuntimeEnv->env, TSDB_CODE_QRY_OUT_OF_MEMORY); + longjmp(pTaskInfo->env, TSDB_CODE_QRY_OUT_OF_MEMORY); } int32_t forwardStep = 0; - TSKEY ekey = reviseWindowEkey(pQueryAttr, &win); + TSKEY ekey = win.ekey; forwardStep = - getNumOfRowsInTimeWindow(pRuntimeEnv, &pSDataBlock->info, tsCols, startPos, ekey, binarySearchForKey, true); + getNumOfRowsInTimeWindow(&pSDataBlock->info, tsCols, startPos, ekey, binarySearchForKey, NULL, TSDB_ORDER_ASC); // prev time window not interpolation yet. int32_t curIndex = pResultRowInfo->curPos; - if (prevIndex != -1 && prevIndex < curIndex && pQueryAttr->timeWindowInterpo) { + if (prevIndex != -1 && prevIndex < curIndex && pInfo->timeWindowInterpo) { for (int32_t j = prevIndex; j < curIndex; ++j) { // previous time window may be all closed already. SResultRow* pRes = getResultRow(pResultRowInfo, j); if (pRes->closed) { @@ -1448,64 +1459,64 @@ static void hashIntervalAgg(SOperatorInfo* pOperatorInfo, SResultRowInfo* pResul } STimeWindow w = pRes->win; - ret = setResultOutputBufByKey(pRuntimeEnv, pResultRowInfo, pSDataBlock->info.uid, &w, masterScan, &pResult, - tableGroupId, pInfo->pCtx, numOfOutput, pInfo->rowCellInfoOffset); + ret = setResultOutputBufByKey_rv(pResultRowInfo, pSDataBlock->info.uid, &w, masterScan, &pResult, + tableGroupId, pInfo->binfo.pCtx, numOfOutput, pInfo->binfo.rowCellInfoOffset, pInfo->pResultBuf, &pInfo->aggSup, pTaskInfo); if (ret != TSDB_CODE_SUCCESS) { - longjmp(pRuntimeEnv->env, TSDB_CODE_QRY_OUT_OF_MEMORY); + longjmp(pTaskInfo->env, TSDB_CODE_QRY_OUT_OF_MEMORY); } assert(!resultRowInterpolated(pResult, RESULT_ROW_END_INTERP)); - doTimeWindowInterpolation(pOperatorInfo, pInfo, pSDataBlock->pDataBlock, *(TSKEY*)pRuntimeEnv->prevRow[0], -1, + doTimeWindowInterpolation(pOperatorInfo, &pInfo->binfo, pSDataBlock->pDataBlock, *(TSKEY*)pInfo->pRow[0], -1, tsCols[startPos], startPos, w.ekey, RESULT_ROW_END_INTERP); setResultRowInterpo(pResult, RESULT_ROW_END_INTERP); - setNotInterpoWindowKey(pInfo->pCtx, pQueryAttr->numOfOutput, RESULT_ROW_START_INTERP); + setNotInterpoWindowKey(pInfo->binfo.pCtx, pOperatorInfo->numOfOutput, RESULT_ROW_START_INTERP); - doApplyFunctions(pRuntimeEnv, pInfo->pCtx, &w, startPos, 0, tsCols, pSDataBlock->info.rows, numOfOutput); + doApplyFunctions(pInfo->binfo.pCtx, &w, startPos, 0, tsCols, pSDataBlock->info.rows, numOfOutput, TSDB_ORDER_ASC); } // restore current time window - ret = setResultOutputBufByKey(pRuntimeEnv, pResultRowInfo, pSDataBlock->info.uid, &win, masterScan, &pResult, tableGroupId, pInfo->pCtx, - numOfOutput, pInfo->rowCellInfoOffset); + ret = setResultOutputBufByKey_rv(pResultRowInfo, pSDataBlock->info.uid, &win, masterScan, &pResult, tableGroupId, pInfo->binfo.pCtx, + numOfOutput, pInfo->binfo.rowCellInfoOffset, pInfo->pResultBuf, &pInfo->aggSup, pTaskInfo); if (ret != TSDB_CODE_SUCCESS) { - longjmp(pRuntimeEnv->env, TSDB_CODE_QRY_OUT_OF_MEMORY); + longjmp(pTaskInfo->env, TSDB_CODE_QRY_OUT_OF_MEMORY); } } // window start key interpolation - doWindowBorderInterpolation(pOperatorInfo, pSDataBlock, pInfo->pCtx, pResult, &win, startPos, forwardStep); - doApplyFunctions(pRuntimeEnv, pInfo->pCtx, &win, startPos, forwardStep, tsCols, pSDataBlock->info.rows, numOfOutput); + doWindowBorderInterpolation(pOperatorInfo, pSDataBlock, pInfo->binfo.pCtx, pResult, &win, startPos, forwardStep, pInfo->order, false); + doApplyFunctions(pInfo->binfo.pCtx, &win, startPos, forwardStep, tsCols, pSDataBlock->info.rows, numOfOutput, TSDB_ORDER_ASC); STimeWindow nextWin = win; while (1) { int32_t prevEndPos = (forwardStep - 1) * step + startPos; - startPos = getNextQualifiedWindow(pQueryAttr, &nextWin, &pSDataBlock->info, tsCols, binarySearchForKey, prevEndPos); + startPos = getNextQualifiedWindow(&pInfo->interval, &nextWin, &pSDataBlock->info, tsCols, prevEndPos, pInfo); if (startPos < 0) { break; } // null data, failed to allocate more memory buffer - int32_t code = setResultOutputBufByKey(pRuntimeEnv, pResultRowInfo, pSDataBlock->info.uid, &nextWin, masterScan, &pResult, tableGroupId, - pInfo->pCtx, numOfOutput, pInfo->rowCellInfoOffset); + int32_t code = setResultOutputBufByKey_rv(pResultRowInfo, pSDataBlock->info.uid, &nextWin, masterScan, &pResult, tableGroupId, + pInfo->binfo.pCtx, numOfOutput, pInfo->binfo.rowCellInfoOffset, pInfo->pResultBuf, &pInfo->aggSup, pTaskInfo); if (code != TSDB_CODE_SUCCESS || pResult == NULL) { - longjmp(pRuntimeEnv->env, TSDB_CODE_QRY_OUT_OF_MEMORY); + longjmp(pTaskInfo->env, TSDB_CODE_QRY_OUT_OF_MEMORY); } - ekey = reviseWindowEkey(pQueryAttr, &nextWin); - forwardStep = getNumOfRowsInTimeWindow(pRuntimeEnv, &pSDataBlock->info, tsCols, startPos, ekey, binarySearchForKey, true); + ekey = nextWin.ekey;//reviseWindowEkey(pQueryAttr, &nextWin); + forwardStep = getNumOfRowsInTimeWindow(&pSDataBlock->info, tsCols, startPos, ekey, binarySearchForKey, NULL, TSDB_ORDER_ASC); // window start(end) key interpolation - doWindowBorderInterpolation(pOperatorInfo, pSDataBlock, pInfo->pCtx, pResult, &nextWin, startPos, forwardStep); - doApplyFunctions(pRuntimeEnv, pInfo->pCtx, &nextWin, startPos, forwardStep, tsCols, pSDataBlock->info.rows, numOfOutput); + doWindowBorderInterpolation(pOperatorInfo, pSDataBlock, pInfo->binfo.pCtx, pResult, &nextWin, startPos, forwardStep, pInfo->order, false); + doApplyFunctions(pInfo->binfo.pCtx, &nextWin, startPos, forwardStep, tsCols, pSDataBlock->info.rows, numOfOutput, TSDB_ORDER_ASC); } - if (pQueryAttr->timeWindowInterpo) { + if (pInfo->timeWindowInterpo) { int32_t rowIndex = ascQuery? (pSDataBlock->info.rows-1):0; - saveDataBlockLastRow(pRuntimeEnv, &pSDataBlock->info, pSDataBlock->pDataBlock, rowIndex); + saveDataBlockLastRow(pInfo->pRow, pSDataBlock->pDataBlock, rowIndex, pSDataBlock->info.numOfCols); } - updateResultRowInfoActiveIndex(pResultRowInfo, pQueryAttr, pRuntimeEnv->current->lastKey); +// updateResultRowInfoActiveIndex(pResultRowInfo, &pInfo->win, pRuntimeEnv->current->lastKey, true, false); } @@ -1528,7 +1539,7 @@ static void hashAllIntervalAgg(SOperatorInfo* pOperatorInfo, SResultRowInfo* pRe } int32_t startPos = ascQuery? 0 : (pSDataBlock->info.rows - 1); - TSKEY ts = getStartTsKey(pQueryAttr, &pSDataBlock->info.window, tsCols, pSDataBlock->info.rows); + TSKEY ts = getStartTsKey(&pSDataBlock->info.window, tsCols, pSDataBlock->info.rows, ascQuery); STimeWindow win = getCurrentActiveTimeWindow(pResultRowInfo, ts, pQueryAttr); bool masterScan = IS_MAIN_SCAN(pRuntimeEnv); @@ -1541,25 +1552,25 @@ static void hashAllIntervalAgg(SOperatorInfo* pOperatorInfo, SResultRowInfo* pRe while (1) { // null data, failed to allocate more memory buffer ret = setResultOutputBufByKey(pRuntimeEnv, pResultRowInfo, pSDataBlock->info.uid, &win, masterScan, &pResult, - tableGroupId, pInfo->pCtx, numOfOutput, pInfo->rowCellInfoOffset); + tableGroupId, pInfo->binfo.pCtx, numOfOutput, pInfo->binfo.rowCellInfoOffset); if (ret != TSDB_CODE_SUCCESS) { longjmp(pRuntimeEnv->env, TSDB_CODE_QRY_OUT_OF_MEMORY); } TSKEY ekey = reviseWindowEkey(pQueryAttr, &win); - forwardStep = getNumOfRowsInTimeWindow(pRuntimeEnv, &pSDataBlock->info, tsCols, startPos, ekey, binarySearchForKey, true); +// forwardStep = getNumOfRowsInTimeWindow(pRuntimeEnv, &pSDataBlock->info, tsCols, startPos, ekey, binarySearchForKey, true); // window start(end) key interpolation - doWindowBorderInterpolation(pOperatorInfo, pSDataBlock, pInfo->pCtx, pResult, &win, startPos, forwardStep); - doApplyFunctions(pRuntimeEnv, pInfo->pCtx, ascQuery ? &win : &preWin, startPos, forwardStep, tsCols, pSDataBlock->info.rows, numOfOutput); +// doWindowBorderInterpolation(pOperatorInfo, pSDataBlock, pInfo->binfo.pCtx, pResult, &win, startPos, forwardStep); +// doApplyFunctions(pRuntimeEnv, pInfo->binfo.pCtx, ascQuery ? &win : &preWin, startPos, forwardStep, tsCols, pSDataBlock->info.rows, numOfOutput); preWin = win; int32_t prevEndPos = (forwardStep - 1) * step + startPos; - startPos = getNextQualifiedWindow(pQueryAttr, &win, &pSDataBlock->info, tsCols, binarySearchForKey, prevEndPos); +// startPos = getNextQualifiedWindow(pQueryAttr, &win, &pSDataBlock->info, tsCols, binarySearchForKey, prevEndPos); if (startPos < 0) { if ((ascQuery && win.skey <= pQueryAttr->window.ekey) || ((!ascQuery) && win.ekey >= pQueryAttr->window.ekey)) { int32_t code = setResultOutputBufByKey(pRuntimeEnv, pResultRowInfo, pSDataBlock->info.uid, &win, masterScan, &pResult, tableGroupId, - pInfo->pCtx, numOfOutput, pInfo->rowCellInfoOffset); + pInfo->binfo.pCtx, numOfOutput, pInfo->binfo.rowCellInfoOffset); if (code != TSDB_CODE_SUCCESS || pResult == NULL) { longjmp(pRuntimeEnv->env, TSDB_CODE_QRY_OUT_OF_MEMORY); } @@ -1567,8 +1578,8 @@ static void hashAllIntervalAgg(SOperatorInfo* pOperatorInfo, SResultRowInfo* pRe startPos = pSDataBlock->info.rows - 1; // window start(end) key interpolation - doWindowBorderInterpolation(pOperatorInfo, pSDataBlock, pInfo->pCtx, pResult, &win, startPos, forwardStep); - doApplyFunctions(pRuntimeEnv, pInfo->pCtx, ascQuery ? &win : &preWin, startPos, forwardStep, tsCols, pSDataBlock->info.rows, numOfOutput); +// doWindowBorderInterpolation(pOperatorInfo, pSDataBlock, pInfo->binfo.pCtx, pResult, &win, startPos, forwardStep); +// doApplyFunctions(pRuntimeEnv, pInfo->binfo.pCtx, ascQuery ? &win : &preWin, startPos, forwardStep, tsCols, pSDataBlock->info.rows, numOfOutput); } break; @@ -1578,10 +1589,10 @@ static void hashAllIntervalAgg(SOperatorInfo* pOperatorInfo, SResultRowInfo* pRe if (pQueryAttr->timeWindowInterpo) { int32_t rowIndex = ascQuery? (pSDataBlock->info.rows-1):0; - saveDataBlockLastRow(pRuntimeEnv, &pSDataBlock->info, pSDataBlock->pDataBlock, rowIndex); +// saveDataBlockLastRow(pRuntimeEnv, &pSDataBlock->info, pSDataBlock->pDataBlock, rowIndex); } - updateResultRowInfoActiveIndex(pResultRowInfo, pQueryAttr, pRuntimeEnv->current->lastKey); +// updateResultRowInfoActiveIndex(pResultRowInfo, pQueryAttr, pRuntimeEnv->current->lastKey); } @@ -1643,7 +1654,7 @@ static void doHashGroupbyAgg(SOperatorInfo* pOperator, SGroupbyOperatorInfo *pIn longjmp(pRuntimeEnv->env, TSDB_CODE_QRY_APP_ERROR); } - doApplyFunctions(pRuntimeEnv, pInfo->binfo.pCtx, &w, j - num, num, tsList, pSDataBlock->info.rows, pOperator->numOfOutput); +// doApplyFunctions(pRuntimeEnv, pInfo->binfo.pCtx, &w, j - num, num, tsList, pSDataBlock->info.rows, pOperator->numOfOutput); num = 1; memcpy(pInfo->prevData, val, bytes); @@ -1662,7 +1673,7 @@ static void doHashGroupbyAgg(SOperatorInfo* pOperator, SGroupbyOperatorInfo *pIn longjmp(pRuntimeEnv->env, TSDB_CODE_QRY_APP_ERROR); } - doApplyFunctions(pRuntimeEnv, pInfo->binfo.pCtx, &w, pSDataBlock->info.rows - num, num, tsList, pSDataBlock->info.rows, pOperator->numOfOutput); +// doApplyFunctions(pRuntimeEnv, pInfo->binfo.pCtx, &w, pSDataBlock->info.rows - num, num, tsList, pSDataBlock->info.rows, pOperator->numOfOutput); } tfree(pInfo->prevData); @@ -1712,8 +1723,8 @@ static void doSessionWindowAggImpl(SOperatorInfo* pOperator, SSWindowOperatorInf longjmp(pRuntimeEnv->env, TSDB_CODE_QRY_APP_ERROR); } - doApplyFunctions(pRuntimeEnv, pBInfo->pCtx, &pInfo->curWindow, pInfo->start, pInfo->numOfRows, tsList, - pSDataBlock->info.rows, pOperator->numOfOutput); +// doApplyFunctions(pRuntimeEnv, pBInfo->pCtx, &pInfo->curWindow, pInfo->start, pInfo->numOfRows, tsList, +// pSDataBlock->info.rows, pOperator->numOfOutput); pInfo->curWindow.skey = tsList[j]; pInfo->curWindow.ekey = tsList[j]; @@ -1733,8 +1744,8 @@ static void doSessionWindowAggImpl(SOperatorInfo* pOperator, SSWindowOperatorInf longjmp(pRuntimeEnv->env, TSDB_CODE_QRY_APP_ERROR); } - doApplyFunctions(pRuntimeEnv, pBInfo->pCtx, &pInfo->curWindow, pInfo->start, pInfo->numOfRows, tsList, - pSDataBlock->info.rows, pOperator->numOfOutput); +// doApplyFunctions(pRuntimeEnv, pBInfo->pCtx, &pInfo->curWindow, pInfo->start, pInfo->numOfRows, tsList, +// pSDataBlock->info.rows, pOperator->numOfOutput); } static void setResultRowKey(SResultRow* pResultRow, char* pData, int16_t type) { @@ -1808,7 +1819,7 @@ static int32_t getGroupbyColumnIndex(SGroupbyExpr *pGroupbyExpr, SSDataBlock* pD return -1; } -static bool functionNeedToExecute(STaskRuntimeEnv *pRuntimeEnv, SqlFunctionCtx *pCtx) { +static bool functionNeedToExecute(SqlFunctionCtx *pCtx) { struct SResultRowEntryInfo *pResInfo = GET_RES_INFO(pCtx); // in case of timestamp column, always generated results. @@ -2048,6 +2059,10 @@ static SqlFunctionCtx* createSqlFunctionCtx_rv(SArray* pExprInfo, int32_t** rowC pCtx->resDataInfo.type = pSqlExpr->resSchema.type; pCtx->order = TSDB_ORDER_ASC; + if (i == 0) { + pCtx->functionId = FUNCTION_TS; + } + // pCtx->functionId = pSqlExpr->functionId; // pCtx->stableQuery = pQueryAttr->stableQuery; pCtx->resDataInfo.intermediateBytes = pSqlExpr->interBytes; @@ -2343,22 +2358,22 @@ static bool isCachedLastQuery(STaskAttr *pQueryAttr) { } ///////////////////////////////////////////////////////////////////////////////////////////// - -void getAlignQueryTimeWindow(STaskAttr *pQueryAttr, int64_t key, int64_t keyFirst, int64_t keyLast, STimeWindow *win) { - assert(key >= keyFirst && key <= keyLast && pQueryAttr->interval.sliding <= pQueryAttr->interval.interval); - win->skey = taosTimeTruncate(key, &pQueryAttr->interval, pQueryAttr->precision); +//todo refactor : return window +void getAlignQueryTimeWindow(SInterval* pInterval, int32_t precision, int64_t key, int64_t keyFirst, int64_t keyLast, STimeWindow *win) { + assert(key >= keyFirst && key <= keyLast && pInterval->sliding <= pInterval->interval); + win->skey = taosTimeTruncate(key, pInterval, precision); /* - * if the realSkey > INT64_MAX - pQueryAttr->interval.interval, the query duration between + * if the realSkey > INT64_MAX - pInterval->interval, the query duration between * realSkey and realEkey must be less than one interval.Therefore, no need to adjust the query ranges. */ - if (keyFirst > (INT64_MAX - pQueryAttr->interval.interval)) { - assert(keyLast - keyFirst < pQueryAttr->interval.interval); + if (keyFirst > (INT64_MAX - pInterval->interval)) { + assert(keyLast - keyFirst < pInterval->interval); win->ekey = INT64_MAX; - } else if (pQueryAttr->interval.intervalUnit == 'n' || pQueryAttr->interval.intervalUnit == 'y') { - win->ekey = taosTimeAdd(win->skey, pQueryAttr->interval.interval, pQueryAttr->interval.intervalUnit, pQueryAttr->precision) - 1; + } else if (pInterval->intervalUnit == 'n' || pInterval->intervalUnit == 'y') { + win->ekey = taosTimeAdd(win->skey, pInterval->interval, pInterval->intervalUnit, precision) - 1; } else { - win->ekey = win->skey + pQueryAttr->interval.interval - 1; + win->ekey = win->skey + pInterval->interval - 1; } } @@ -2584,7 +2599,7 @@ static bool overlapWithTimeWindow(STaskAttr* pQueryAttr, SDataBlockInfo* pBlockI TSKEY ek = TMAX(pQueryAttr->window.skey, pQueryAttr->window.ekey); if (QUERY_IS_ASC_QUERY(pQueryAttr)) { - getAlignQueryTimeWindow(pQueryAttr, pBlockInfo->window.skey, sk, ek, &w); +// getAlignQueryTimeWindow(pQueryAttr, pBlockInfo->window.skey, sk, ek, &w); assert(w.ekey >= pBlockInfo->window.skey); if (w.ekey < pBlockInfo->window.ekey) { @@ -2592,7 +2607,7 @@ static bool overlapWithTimeWindow(STaskAttr* pQueryAttr, SDataBlockInfo* pBlockI } while(1) { - getNextTimeWindow(pQueryAttr, &w); +// getNextTimeWindow(pQueryAttr, &w); if (w.skey > pBlockInfo->window.ekey) { break; } @@ -2603,7 +2618,7 @@ static bool overlapWithTimeWindow(STaskAttr* pQueryAttr, SDataBlockInfo* pBlockI } } } else { - getAlignQueryTimeWindow(pQueryAttr, pBlockInfo->window.ekey, sk, ek, &w); +// getAlignQueryTimeWindow(pQueryAttr, pBlockInfo->window.ekey, sk, ek, &w); assert(w.skey <= pBlockInfo->window.ekey); if (w.skey > pBlockInfo->window.skey) { @@ -2611,7 +2626,7 @@ static bool overlapWithTimeWindow(STaskAttr* pQueryAttr, SDataBlockInfo* pBlockI } while(1) { - getNextTimeWindow(pQueryAttr, &w); +// getNextTimeWindow(pQueryAttr, &w); if (w.ekey < pBlockInfo->window.skey) { break; } @@ -3513,9 +3528,6 @@ static void setupEnvForReverseScan(STableScanInfo *pTableScanInfo, SqlFunctionCt } void finalizeQueryResult(SOperatorInfo* pOperator, SqlFunctionCtx* pCtx, SResultRowInfo* pResultRowInfo, int32_t* rowCellInfoOffset) { - STaskRuntimeEnv *pRuntimeEnv = pOperator->pRuntimeEnv; -// STaskAttr *pQueryAttr = pRuntimeEnv->pQueryAttr; - int32_t numOfOutput = pOperator->numOfOutput; // if (pQueryAttr->groupbyColumn || QUERY_IS_INTERVAL_QUERY(pQueryAttr) || pQueryAttr->sw.gap > 0 || pQueryAttr->stateWindow) { // // for each group result, call the finalize function for each column @@ -3908,7 +3920,7 @@ void setIntervalQueryRange(STaskRuntimeEnv *pRuntimeEnv, TSKEY key) { TSKEY sk = TMIN(win.skey, win.ekey); TSKEY ek = TMAX(win.skey, win.ekey); - getAlignQueryTimeWindow(pQueryAttr, win.skey, sk, ek, &w); +// getAlignQueryTimeWindow(pQueryAttr, win.skey, sk, ek, &w); // if (pResultRowInfo->prevSKey == TSKEY_INITIAL_VAL) { // if (!QUERY_IS_ASC_QUERY(pQueryAttr)) { @@ -6364,22 +6376,19 @@ static SSDataBlock* doIntervalAgg(void* param, bool* newgroup) { return NULL; } - STableIntervalOperatorInfo* pIntervalInfo = pOperator->info; + STableIntervalOperatorInfo* pInfo = pOperator->info; - STaskRuntimeEnv* pRuntimeEnv = pOperator->pRuntimeEnv; if (pOperator->status == OP_RES_TO_RETURN) { // toSDatablock(pAggInfo->pGroupResInfo, pAggInfo->pResultBuf, pInfo->pRes, pAggInfo->binfo.capacity); - if (pIntervalInfo->pRes->info.rows == 0 || !hasRemainDataInCurrentGroup(&pRuntimeEnv->groupResInfo)) { + if (pInfo->binfo.pRes->info.rows == 0 || !hasRemainDataInCurrentGroup(&pInfo->groupResInfo)) { doSetOperatorCompleted(pOperator); } - return pIntervalInfo->pRes; + return pInfo->binfo.pRes; } - STaskAttr* pQueryAttr = pRuntimeEnv->pQueryAttr; - int32_t order = pQueryAttr->order.order; - STimeWindow win = pQueryAttr->window; - +// int32_t order = pQueryAttr->order.order; +// STimeWindow win = pQueryAttr->window; SOperatorInfo* downstream = pOperator->pDownstream[0]; while(1) { @@ -6391,30 +6400,30 @@ static SSDataBlock* doIntervalAgg(void* param, bool* newgroup) { break; } -// setTagValue(pOperator, pRuntimeEnv->current->pTable, pIntervalInfo->pCtx, pOperator->numOfOutput); +// setTagValue(pOperator, pRuntimeEnv->current->pTable, pInfo->pCtx, pOperator->numOfOutput); // the pDataBlock are always the same one, no need to call this again - setInputDataBlock(pOperator, pIntervalInfo->pCtx, pBlock, pQueryAttr->order.order); - hashIntervalAgg(pOperator, &pIntervalInfo->resultRowInfo, pBlock, 0); + setInputDataBlock(pOperator, pInfo->binfo.pCtx, pBlock, TSDB_ORDER_ASC); + hashIntervalAgg(pOperator, &pInfo->binfo.resultRowInfo, pBlock, 0); } // restore the value - pQueryAttr->order.order = order; - pQueryAttr->window = win; +// pQueryAttr->order.order = order; +// pQueryAttr->window = win; pOperator->status = OP_RES_TO_RETURN; - closeAllResultRows(&pIntervalInfo->resultRowInfo); + closeAllResultRows(&pInfo->binfo.resultRowInfo); setTaskStatus(pOperator->pTaskInfo, TASK_COMPLETED); - finalizeQueryResult(pOperator, pIntervalInfo->pCtx, &pIntervalInfo->resultRowInfo, pIntervalInfo->rowCellInfoOffset); + finalizeQueryResult(pOperator, pInfo->binfo.pCtx, &pInfo->binfo.resultRowInfo, pInfo->binfo.rowCellInfoOffset); - initGroupResInfo(&pRuntimeEnv->groupResInfo, &pIntervalInfo->resultRowInfo); -// toSDatablock(&pRuntimeEnv->groupResInfo, pRuntimeEnv, pIntervalInfo->pRes); + initGroupResInfo(&pInfo->groupResInfo, &pInfo->binfo.resultRowInfo); + toSDatablock(&pInfo->groupResInfo, pInfo->pResultBuf, pInfo->binfo.pRes, pInfo->binfo.capacity); - if (pIntervalInfo->pRes->info.rows == 0 || !hasRemainDataInCurrentGroup(&pRuntimeEnv->groupResInfo)) { + if (pInfo->binfo.pRes->info.rows == 0 || !hasRemainDataInCurrentGroup(&pInfo->groupResInfo)) { doSetOperatorCompleted(pOperator); } - return pIntervalInfo->pRes->info.rows == 0? NULL:pIntervalInfo->pRes; + return pInfo->binfo.pRes->info.rows == 0? NULL:pInfo->binfo.pRes; } static SSDataBlock* doAllIntervalAgg(void* param, bool* newgroup) { @@ -6429,11 +6438,11 @@ static SSDataBlock* doAllIntervalAgg(void* param, bool* newgroup) { if (pOperator->status == OP_RES_TO_RETURN) { // toSDatablock(&pRuntimeEnv->groupResInfo, pRuntimeEnv, pIntervalInfo->pRes); - if (pIntervalInfo->pRes->info.rows == 0 || !hasRemainDataInCurrentGroup(&pRuntimeEnv->groupResInfo)) { + if (pIntervalInfo->binfo.pRes->info.rows == 0 || !hasRemainDataInCurrentGroup(&pRuntimeEnv->groupResInfo)) { doSetOperatorCompleted(pOperator); } - return pIntervalInfo->pRes; + return pIntervalInfo->binfo.pRes; } STaskAttr* pQueryAttr = pRuntimeEnv->pQueryAttr; @@ -6454,8 +6463,8 @@ static SSDataBlock* doAllIntervalAgg(void* param, bool* newgroup) { // setTagValue(pOperator, pRuntimeEnv->current->pTable, pIntervalInfo->pCtx, pOperator->numOfOutput); // the pDataBlock are always the same one, no need to call this again - setInputDataBlock(pOperator, pIntervalInfo->pCtx, pBlock, pQueryAttr->order.order); - hashAllIntervalAgg(pOperator, &pIntervalInfo->resultRowInfo, pBlock, 0); + setInputDataBlock(pOperator, pIntervalInfo->binfo.pCtx, pBlock, pQueryAttr->order.order); + hashAllIntervalAgg(pOperator, &pIntervalInfo->binfo.resultRowInfo, pBlock, 0); } // restore the value @@ -6463,18 +6472,18 @@ static SSDataBlock* doAllIntervalAgg(void* param, bool* newgroup) { pQueryAttr->window = win; pOperator->status = OP_RES_TO_RETURN; - closeAllResultRows(&pIntervalInfo->resultRowInfo); + closeAllResultRows(&pIntervalInfo->binfo.resultRowInfo); setTaskStatus(pOperator->pTaskInfo, TASK_COMPLETED); - finalizeQueryResult(pOperator, pIntervalInfo->pCtx, &pIntervalInfo->resultRowInfo, pIntervalInfo->rowCellInfoOffset); + finalizeQueryResult(pOperator, pIntervalInfo->binfo.pCtx, &pIntervalInfo->binfo.resultRowInfo, pIntervalInfo->binfo.rowCellInfoOffset); - initGroupResInfo(&pRuntimeEnv->groupResInfo, &pIntervalInfo->resultRowInfo); + initGroupResInfo(&pRuntimeEnv->groupResInfo, &pIntervalInfo->binfo.resultRowInfo); // toSDatablock(&pRuntimeEnv->groupResInfo, pRuntimeEnv, pIntervalInfo->pRes); - if (pIntervalInfo->pRes->info.rows == 0 || !hasRemainDataInCurrentGroup(&pRuntimeEnv->groupResInfo)) { + if (pIntervalInfo->binfo.pRes->info.rows == 0 || !hasRemainDataInCurrentGroup(&pRuntimeEnv->groupResInfo)) { pOperator->status = OP_EXEC_DONE; } - return pIntervalInfo->pRes->info.rows == 0? NULL:pIntervalInfo->pRes; + return pIntervalInfo->binfo.pRes->info.rows == 0? NULL:pIntervalInfo->binfo.pRes; } static SSDataBlock* doSTableIntervalAgg(void* param, bool* newgroup) { @@ -6490,14 +6499,14 @@ static SSDataBlock* doSTableIntervalAgg(void* param, bool* newgroup) { int64_t st = taosGetTimestampUs(); // copyToSDataBlock(NULL, 3000, pIntervalInfo->pRes, pIntervalInfo->rowCellInfoOffset); - if (pIntervalInfo->pRes->info.rows == 0 || !hasRemainData(&pRuntimeEnv->groupResInfo)) { + if (pIntervalInfo->binfo.pRes->info.rows == 0 || !hasRemainData(&pRuntimeEnv->groupResInfo)) { doSetOperatorCompleted(pOperator); } SQInfo* pQInfo = pRuntimeEnv->qinfo; pQInfo->summary.firstStageMergeTime += (taosGetTimestampUs() - st); - return pIntervalInfo->pRes; + return pIntervalInfo->binfo.pRes; } STaskAttr* pQueryAttr = pRuntimeEnv->pQueryAttr; @@ -6518,7 +6527,7 @@ static SSDataBlock* doSTableIntervalAgg(void* param, bool* newgroup) { STableQueryInfo* pTableQueryInfo = pRuntimeEnv->current; // setTagValue(pOperator, pTableQueryInfo->pTable, pIntervalInfo->pCtx, pOperator->numOfOutput); - setInputDataBlock(pOperator, pIntervalInfo->pCtx, pBlock, pQueryAttr->order.order); + setInputDataBlock(pOperator, pIntervalInfo->binfo.pCtx, pBlock, pQueryAttr->order.order); setIntervalQueryRange(pRuntimeEnv, pBlock->info.window.skey); hashIntervalAgg(pOperator, &pTableQueryInfo->resInfo, pBlock, pTableQueryInfo->groupIndex); @@ -6530,11 +6539,11 @@ static SSDataBlock* doSTableIntervalAgg(void* param, bool* newgroup) { setTaskStatus(pOperator->pTaskInfo, TASK_COMPLETED); // copyToSDataBlock(pRuntimeEnv, 3000, pIntervalInfo->pRes, pIntervalInfo->rowCellInfoOffset); - if (pIntervalInfo->pRes->info.rows == 0 || !hasRemainData(&pRuntimeEnv->groupResInfo)) { + if (pIntervalInfo->binfo.pRes->info.rows == 0 || !hasRemainData(&pRuntimeEnv->groupResInfo)) { pOperator->status = OP_EXEC_DONE; } - return pIntervalInfo->pRes; + return pIntervalInfo->binfo.pRes; } static SSDataBlock* doAllSTableIntervalAgg(void* param, bool* newgroup) { @@ -6548,11 +6557,11 @@ static SSDataBlock* doAllSTableIntervalAgg(void* param, bool* newgroup) { if (pOperator->status == OP_RES_TO_RETURN) { // copyToSDataBlock(pRuntimeEnv, 3000, pIntervalInfo->pRes, pIntervalInfo->rowCellInfoOffset); - if (pIntervalInfo->pRes->info.rows == 0 || !hasRemainData(&pRuntimeEnv->groupResInfo)) { + if (pIntervalInfo->binfo.pRes->info.rows == 0 || !hasRemainData(&pRuntimeEnv->groupResInfo)) { pOperator->status = OP_EXEC_DONE; } - return pIntervalInfo->pRes; + return pIntervalInfo->binfo.pRes; } STaskAttr* pQueryAttr = pRuntimeEnv->pQueryAttr; @@ -6573,7 +6582,7 @@ static SSDataBlock* doAllSTableIntervalAgg(void* param, bool* newgroup) { STableQueryInfo* pTableQueryInfo = pRuntimeEnv->current; // setTagValue(pOperator, pTableQueryInfo->pTable, pIntervalInfo->pCtx, pOperator->numOfOutput); - setInputDataBlock(pOperator, pIntervalInfo->pCtx, pBlock, pQueryAttr->order.order); + setInputDataBlock(pOperator, pIntervalInfo->binfo.pCtx, pBlock, pQueryAttr->order.order); setIntervalQueryRange(pRuntimeEnv, pBlock->info.window.skey); hashAllIntervalAgg(pOperator, &pTableQueryInfo->resInfo, pBlock, pTableQueryInfo->groupIndex); @@ -6586,14 +6595,14 @@ static SSDataBlock* doAllSTableIntervalAgg(void* param, bool* newgroup) { int64_t st = taosGetTimestampUs(); // copyToSDataBlock(pRuntimeEnv, 3000, pIntervalInfo->pRes, pIntervalInfo->rowCellInfoOffset); - if (pIntervalInfo->pRes->info.rows == 0 || !hasRemainData(&pRuntimeEnv->groupResInfo)) { + if (pIntervalInfo->binfo.pRes->info.rows == 0 || !hasRemainData(&pRuntimeEnv->groupResInfo)) { pOperator->status = OP_EXEC_DONE; } SQInfo* pQInfo = pRuntimeEnv->qinfo; pQInfo->summary.firstStageMergeTime += (taosGetTimestampUs() - st); - return pIntervalInfo->pRes; + return pIntervalInfo->binfo.pRes; } static void doStateWindowAggImpl(SOperatorInfo* pOperator, SStateWindowOperatorInfo *pInfo, SSDataBlock *pSDataBlock) { @@ -6645,8 +6654,8 @@ static void doStateWindowAggImpl(SOperatorInfo* pOperator, SStateWindowOperatorI if (ret != TSDB_CODE_SUCCESS) { // null data, too many state code longjmp(pRuntimeEnv->env, TSDB_CODE_QRY_APP_ERROR); } - doApplyFunctions(pRuntimeEnv, pBInfo->pCtx, &pInfo->curWindow, pInfo->start, pInfo->numOfRows, tsList, - pSDataBlock->info.rows, pOperator->numOfOutput); +// doApplyFunctions(pRuntimeEnv, pBInfo->pCtx, &pInfo->curWindow, pInfo->start, pInfo->numOfRows, tsList, +// pSDataBlock->info.rows, pOperator->numOfOutput); pInfo->curWindow.skey = tsList[j]; pInfo->curWindow.ekey = tsList[j]; @@ -6667,8 +6676,8 @@ static void doStateWindowAggImpl(SOperatorInfo* pOperator, SStateWindowOperatorI longjmp(pRuntimeEnv->env, TSDB_CODE_QRY_APP_ERROR); } - doApplyFunctions(pRuntimeEnv, pBInfo->pCtx, &pInfo->curWindow, pInfo->start, pInfo->numOfRows, tsList, - pSDataBlock->info.rows, pOperator->numOfOutput); +// doApplyFunctions(pRuntimeEnv, pBInfo->pCtx, &pInfo->curWindow, pInfo->start, pInfo->numOfRows, tsList, +// pSDataBlock->info.rows, pOperator->numOfOutput); } static SSDataBlock* doStateWindowAgg(void *param, bool* newgroup) { @@ -7268,27 +7277,44 @@ SOperatorInfo* createLimitOperatorInfo(STaskRuntimeEnv* pRuntimeEnv, SOperatorIn return pOperator; } -SOperatorInfo* createTimeIntervalOperatorInfo(STaskRuntimeEnv* pRuntimeEnv, SOperatorInfo* downstream, SExprInfo* pExpr, int32_t numOfOutput) { +SOperatorInfo* createIntervalOperatorInfo(SOperatorInfo* downstream, SArray* pExprInfo, SExecTaskInfo* pTaskInfo) { STableIntervalOperatorInfo* pInfo = calloc(1, sizeof(STableIntervalOperatorInfo)); - pInfo->pCtx = createSqlFunctionCtx(pRuntimeEnv, pExpr, numOfOutput, &pInfo->rowCellInfoOffset); - pInfo->pRes = createOutputBuf(pExpr, numOfOutput, pRuntimeEnv->resultInfo.capacity); - initResultRowInfo(&pInfo->resultRowInfo, 8); + initAggSup(&pInfo->aggSup, pExprInfo); + + // todo: + pInfo->order = TSDB_ORDER_ASC; + pInfo->precision = TSDB_TIME_PRECISION_MICRO; + pInfo->win.skey = INT64_MIN; + pInfo->win.ekey = INT64_MAX; + pInfo->interval.intervalUnit = 's'; + pInfo->interval.slidingUnit = 's'; + pInfo->interval.interval = 1000; + pInfo->interval.sliding = 1000; + + int32_t code = createDiskbasedBuf(&pInfo->pResultBuf, 4096, 4096 * 256, 0, "/tmp/"); + + int32_t numOfOutput = taosArrayGetSize(pExprInfo); + pInfo->binfo.pCtx = createSqlFunctionCtx_rv(pExprInfo, &pInfo->binfo.rowCellInfoOffset, &pInfo->binfo.resRowSize); + pInfo->binfo.pRes = createOutputBuf_rv(pExprInfo, pInfo->binfo.capacity); + + initResultRowInfo(&pInfo->binfo.resultRowInfo, (int32_t)1); SOperatorInfo* pOperator = calloc(1, sizeof(SOperatorInfo)); pOperator->name = "TimeIntervalAggOperator"; -// pOperator->operatorType = OP_TimeWindow; + pOperator->operatorType = OP_TimeWindow; pOperator->blockingOptr = true; pOperator->status = OP_IN_EXECUTING; - pOperator->pExpr = pExpr; + pOperator->pExpr = exprArrayDup(pExprInfo); + + pOperator->pTaskInfo = pTaskInfo; pOperator->numOfOutput = numOfOutput; pOperator->info = pInfo; - pOperator->pRuntimeEnv = pRuntimeEnv; pOperator->exec = doIntervalAgg; pOperator->cleanupFn = destroyBasicOperatorInfo; - int32_t code = appendDownstream(pOperator, &downstream, 1); + code = appendDownstream(pOperator, &downstream, 1); return pOperator; } @@ -7296,9 +7322,9 @@ SOperatorInfo* createTimeIntervalOperatorInfo(STaskRuntimeEnv* pRuntimeEnv, SOpe SOperatorInfo* createAllTimeIntervalOperatorInfo(STaskRuntimeEnv* pRuntimeEnv, SOperatorInfo* downstream, SExprInfo* pExpr, int32_t numOfOutput) { STableIntervalOperatorInfo* pInfo = calloc(1, sizeof(STableIntervalOperatorInfo)); - pInfo->pCtx = createSqlFunctionCtx(pRuntimeEnv, pExpr, numOfOutput, &pInfo->rowCellInfoOffset); - pInfo->pRes = createOutputBuf(pExpr, numOfOutput, pRuntimeEnv->resultInfo.capacity); - initResultRowInfo(&pInfo->resultRowInfo, 8); + pInfo->binfo.pCtx = createSqlFunctionCtx(pRuntimeEnv, pExpr, numOfOutput, &pInfo->binfo.rowCellInfoOffset); + pInfo->binfo.pRes = createOutputBuf(pExpr, numOfOutput, pRuntimeEnv->resultInfo.capacity); + initResultRowInfo(&pInfo->binfo.resultRowInfo, 8); SOperatorInfo* pOperator = calloc(1, sizeof(SOperatorInfo)); @@ -7369,9 +7395,9 @@ SOperatorInfo* createSWindowOperatorInfo(STaskRuntimeEnv* pRuntimeEnv, SOperator SOperatorInfo* createMultiTableTimeIntervalOperatorInfo(STaskRuntimeEnv* pRuntimeEnv, SOperatorInfo* downstream, SExprInfo* pExpr, int32_t numOfOutput) { STableIntervalOperatorInfo* pInfo = calloc(1, sizeof(STableIntervalOperatorInfo)); - pInfo->pCtx = createSqlFunctionCtx(pRuntimeEnv, pExpr, numOfOutput, &pInfo->rowCellInfoOffset); - pInfo->pRes = createOutputBuf(pExpr, numOfOutput, pRuntimeEnv->resultInfo.capacity); - initResultRowInfo(&pInfo->resultRowInfo, 8); + pInfo->binfo.pCtx = createSqlFunctionCtx(pRuntimeEnv, pExpr, numOfOutput, &pInfo->binfo.rowCellInfoOffset); + pInfo->binfo.pRes = createOutputBuf(pExpr, numOfOutput, pRuntimeEnv->resultInfo.capacity); + initResultRowInfo(&pInfo->binfo.resultRowInfo, 8); SOperatorInfo* pOperator = calloc(1, sizeof(SOperatorInfo)); pOperator->name = "MultiTableTimeIntervalOperator"; @@ -7393,9 +7419,9 @@ SOperatorInfo* createMultiTableTimeIntervalOperatorInfo(STaskRuntimeEnv* pRuntim SOperatorInfo* createAllMultiTableTimeIntervalOperatorInfo(STaskRuntimeEnv* pRuntimeEnv, SOperatorInfo* downstream, SExprInfo* pExpr, int32_t numOfOutput) { STableIntervalOperatorInfo* pInfo = calloc(1, sizeof(STableIntervalOperatorInfo)); - pInfo->pCtx = createSqlFunctionCtx(pRuntimeEnv, pExpr, numOfOutput, &pInfo->rowCellInfoOffset); - pInfo->pRes = createOutputBuf(pExpr, numOfOutput, pRuntimeEnv->resultInfo.capacity); - initResultRowInfo(&pInfo->resultRowInfo, 8); + pInfo->binfo.pCtx = createSqlFunctionCtx(pRuntimeEnv, pExpr, numOfOutput, &pInfo->binfo.rowCellInfoOffset); + pInfo->binfo.pRes = createOutputBuf(pExpr, numOfOutput, pRuntimeEnv->resultInfo.capacity); + initResultRowInfo(&pInfo->binfo.resultRowInfo, 8); SOperatorInfo* pOperator = calloc(1, sizeof(SOperatorInfo)); pOperator->name = "AllMultiTableTimeIntervalOperator"; @@ -7458,7 +7484,7 @@ SOperatorInfo* createFillOperatorInfo(STaskRuntimeEnv* pRuntimeEnv, SOperatorInf TSKEY sk = TMIN(pQueryAttr->window.skey, pQueryAttr->window.ekey); TSKEY ek = TMAX(pQueryAttr->window.skey, pQueryAttr->window.ekey); - getAlignQueryTimeWindow(pQueryAttr, pQueryAttr->window.skey, sk, ek, &w); +// getAlignQueryTimeWindow(pQueryAttr, pQueryAttr->window.skey, sk, ek, &w); pInfo->pFillInfo = taosCreateFillInfo(pQueryAttr->order.order, w.skey, 0, (int32_t)pRuntimeEnv->resultInfo.capacity, numOfOutput, diff --git a/source/libs/executor/src/tlinearhash.c b/source/libs/executor/src/tlinearhash.c index 803ce8bba2..3a58253d81 100644 --- a/source/libs/executor/src/tlinearhash.c +++ b/source/libs/executor/src/tlinearhash.c @@ -44,8 +44,8 @@ typedef struct SLHashObj { * +-----------+-------+--------+ */ typedef struct SLHashNode { - int32_t keyLen; - int32_t dataLen; + uint16_t keyLen; + uint16_t dataLen; } SLHashNode; #define GET_LHASH_NODE_KEY(_n) (((char*)(_n)) + sizeof(SLHashNode)) @@ -70,10 +70,10 @@ static int32_t doGetRelatedSplitBucketId(int32_t bucketId, int32_t bits) { } static void doCopyObject(char* p, const void* key, int32_t keyLen, const void* data, int32_t size) { - *(int32_t*) p = keyLen; - p += sizeof(int32_t); - *(int32_t*) p = size; - p += sizeof(int32_t); + *(uint16_t*) p = keyLen; + p += sizeof(uint16_t); + *(uint16_t*) p = size; + p += sizeof(uint16_t); memcpy(p, key, keyLen); p += keyLen; @@ -118,7 +118,7 @@ static int32_t doAddToBucket(SLHashObj* pHashObj, SLHashBucket* pBucket, int32_t } pBucket->size += 1; - printf("===> add to bucket:0x%x, num:%d, key:%d\n", index, pBucket->size, *(int*) key); +// printf("===> add to bucket:0x%x, num:%d, key:%d\n", index, pBucket->size, *(int*) key); return TSDB_CODE_SUCCESS; } @@ -154,6 +154,14 @@ static void doCompressBucketPages(SLHashObj *pHashObj, SLHashBucket* pBucket) { int32_t* pageId = taosArrayGetLast(pBucket->pPageIdList); SFilePage* pLast = getBufPage(pHashObj->pBuf, *pageId); + if (pLast->num <= sizeof(SFilePage)) { + // this is empty + dBufSetBufPageRecycled(pHashObj->pBuf, pLast); + releaseBufPage(pHashObj->pBuf, pFirst); + taosArrayRemove(pBucket->pPageIdList, numOfPages - 1); + return; + } + char* pStart = pLast->data; int32_t nodeSize = GET_LHASH_NODE_LEN(pStart); while (1) { @@ -162,21 +170,33 @@ static void doCompressBucketPages(SLHashObj *pHashObj, SLHashBucket* pBucket) { SLHashNode* pNode = (SLHashNode*)pStart; doCopyObject(p, GET_LHASH_NODE_KEY(pStart), pNode->keyLen, GET_LHASH_NODE_DATA(pStart), pNode->dataLen); + setBufPageDirty(pFirst, true); + setBufPageDirty(pLast, true); + + ASSERT(pLast->num >= nodeSize + sizeof(SFilePage)); pFirst->num += nodeSize; pLast->num -= nodeSize; + pStart += nodeSize; - if (pStart - pLast->data >= pLast->num) { + if (pLast->num <= sizeof(SFilePage)) { // this is empty dBufSetBufPageRecycled(pHashObj->pBuf, pLast); + releaseBufPage(pHashObj->pBuf, pFirst); taosArrayRemove(pBucket->pPageIdList, numOfPages - 1); break; } nodeSize = GET_LHASH_NODE_LEN(pStart); } else { // move to the front of pLast page - memmove(pLast->data, pStart,(((char*)pLast) + pLast->num - pStart)); + if (pStart != pLast->data) { + memmove(pLast->data, pStart, (((char*)pLast) + pLast->num - pStart)); + setBufPageDirty(pLast, true); + } + + releaseBufPage(pHashObj->pBuf, pLast); + releaseBufPage(pHashObj->pBuf, pFirst); break; } } @@ -216,7 +236,7 @@ static int32_t doAddNewBucket(SLHashObj* pHashObj) { taosArrayPush(pBucket->pPageIdList, &pageId); pHashObj->numOfBuckets += 1; - printf("---------------add new bucket, id:0x%x, total:%d\n", pHashObj->numOfBuckets - 1, pHashObj->numOfBuckets); +// printf("---------------add new bucket, id:0x%x, total:%d\n", pHashObj->numOfBuckets - 1, pHashObj->numOfBuckets); return TSDB_CODE_SUCCESS; } @@ -281,7 +301,7 @@ int32_t tHashPut(SLHashObj* pHashObj, const void *key, size_t keyLen, void *data if (v >= pHashObj->numOfBuckets) { int32_t newBucketId = doGetAlternativeBucketId(v, pHashObj->bits, pHashObj->numOfBuckets); - printf("bucketId: 0x%x not exists, put it into 0x%x instead\n", v, newBucketId); +// printf("bucketId: 0x%x not exists, put it into 0x%x instead\n", v, newBucketId); v = newBucketId; } @@ -305,7 +325,7 @@ int32_t tHashPut(SLHashObj* pHashObj, const void *key, size_t keyLen, void *data int32_t numOfBits = ceil(log(pHashObj->numOfBuckets) / log(2)); if (numOfBits > pHashObj->bits) { - printf("extend the bits from %d to %d, new bucket:%d\n", pHashObj->bits, numOfBits, newBucketId); +// printf("extend the bits from %d to %d, new bucket:%d\n", pHashObj->bits, numOfBits, newBucketId); ASSERT(numOfBits == pHashObj->bits + 1); pHashObj->bits = numOfBits; } @@ -314,7 +334,7 @@ int32_t tHashPut(SLHashObj* pHashObj, const void *key, size_t keyLen, void *data // load all data in this bucket and check if the data needs to relocated into the new bucket SLHashBucket* pBucket = pHashObj->pBucket[splitBucketId]; - printf("split %d items' bucket:0x%x to new bucket:0x%x\n", pBucket->size, splitBucketId, newBucketId); +// printf("split %d items' bucket:0x%x to new bucket:0x%x\n", pBucket->size, splitBucketId, newBucketId); for (int32_t i = 0; i < taosArrayGetSize(pBucket->pPageIdList); ++i) { int32_t pageId = *(int32_t*)taosArrayGet(pBucket->pPageIdList, i); @@ -331,14 +351,14 @@ int32_t tHashPut(SLHashObj* pHashObj, const void *key, size_t keyLen, void *data if (v1 != splitBucketId) { // place it into the new bucket ASSERT(v1 == newBucketId); - printf("move key:%d to 0x%x bucket, remain items:%d\n", *(int32_t*)k, v1, pBucket->size - 1); +// printf("move key:%d to 0x%x bucket, remain items:%d\n", *(int32_t*)k, v1, pBucket->size - 1); SLHashBucket* pNewBucket = pHashObj->pBucket[newBucketId]; doAddToBucket(pHashObj, pNewBucket, newBucketId, (void*)GET_LHASH_NODE_KEY(pNode), pNode->keyLen, GET_LHASH_NODE_KEY(pNode), pNode->dataLen); doRemoveFromBucket(p, pNode, pBucket); } else { - printf("check key:%d, located into: %d, skip it\n", *(int*) k, v1); +// printf("check key:%d, located into: %d, skip it\n", *(int*) k, v1); int32_t nodeSize = GET_LHASH_NODE_LEN(pStart); pStart += nodeSize; @@ -398,8 +418,8 @@ void tHashPrint(const SLHashObj* pHashObj, int32_t type) { if (type == LINEAR_HASH_DATA) { for (int32_t i = 0; i < pHashObj->numOfBuckets; ++i) { - printf("bucket: 0x%x, obj:%d, page:%d\n", i, pHashObj->pBucket[i]->size, - (int)taosArrayGetSize(pHashObj->pBucket[i]->pPageIdList)); +// printf("bucket: 0x%x, obj:%d, page:%d\n", i, pHashObj->pBucket[i]->size, +// (int)taosArrayGetSize(pHashObj->pBucket[i]->pPageIdList)); } } else { dBufPrintStatis(pHashObj->pBuf); diff --git a/source/libs/executor/test/executorTests.cpp b/source/libs/executor/test/executorTests.cpp index c0fb899a2d..c3757134d5 100644 --- a/source/libs/executor/test/executorTests.cpp +++ b/source/libs/executor/test/executorTests.cpp @@ -14,6 +14,7 @@ */ #include +#include #include #include #include @@ -47,6 +48,8 @@ typedef struct SDummyInputInfo { int32_t startVal; int32_t type; int32_t numOfRowsPerPage; + int32_t numOfCols; // number of columns + int64_t tsStart; SSDataBlock* pBlock; } SDummyInputInfo; @@ -117,16 +120,96 @@ SSDataBlock* getDummyBlock(void* param, bool* newgroup) { return pBlock; } -SOperatorInfo* createDummyOperator(int32_t startVal, int32_t numOfBlocks, int32_t rowsPerPage, int32_t type) { +SSDataBlock* get2ColsDummyBlock(void* param, bool* newgroup) { + SOperatorInfo* pOperator = static_cast(param); + SDummyInputInfo* pInfo = static_cast(pOperator->info); + if (pInfo->current >= pInfo->totalPages) { + return NULL; + } + + if (pInfo->pBlock == NULL) { + pInfo->pBlock = static_cast(calloc(1, sizeof(SSDataBlock))); + + pInfo->pBlock->pDataBlock = taosArrayInit(4, sizeof(SColumnInfoData)); + + SColumnInfoData colInfo = {0}; + colInfo.info.type = TSDB_DATA_TYPE_TIMESTAMP; + colInfo.info.bytes = sizeof(int64_t); + colInfo.info.colId = 1; + colInfo.pData = static_cast(calloc(pInfo->numOfRowsPerPage, sizeof(int64_t))); +// colInfo.nullbitmap = static_cast(calloc(1, (pInfo->numOfRowsPerPage + 7) / 8)); + + taosArrayPush(pInfo->pBlock->pDataBlock, &colInfo); + + SColumnInfoData colInfo1 = {0}; + colInfo1.info.type = TSDB_DATA_TYPE_INT; + colInfo1.info.bytes = 4; + colInfo1.info.colId = 2; + + colInfo1.pData = static_cast(calloc(pInfo->numOfRowsPerPage, sizeof(int32_t))); + colInfo1.nullbitmap = static_cast(calloc(1, (pInfo->numOfRowsPerPage + 7) / 8)); + + taosArrayPush(pInfo->pBlock->pDataBlock, &colInfo1); + } else { + blockDataClearup(pInfo->pBlock, false); + } + + SSDataBlock* pBlock = pInfo->pBlock; + + char buf[128] = {0}; + char b1[128] = {0}; + int64_t ts = 0; + int32_t v = 0; + for(int32_t i = 0; i < pInfo->numOfRowsPerPage; ++i) { + SColumnInfoData* pColInfo = static_cast(TARRAY_GET_ELEM(pBlock->pDataBlock, 0)); + + ts = (++pInfo->tsStart); + colDataAppend(pColInfo, i, reinterpret_cast(&ts), false); + + SColumnInfoData* pColInfo1 = static_cast(TARRAY_GET_ELEM(pBlock->pDataBlock, 1)); + if (pInfo->type == data_desc) { + v = (--pInfo->startVal); + } else if (pInfo->type == data_asc) { + v = ++pInfo->startVal; + } else if (pInfo->type == data_rand) { + v = random(); + } + + colDataAppend(pColInfo1, i, reinterpret_cast(&v), false); + +// sprintf(buf, "this is %d row", i); +// STR_TO_VARSTR(b1, buf); +// +// SColumnInfoData* pColInfo2 = static_cast(TARRAY_GET_ELEM(pBlock->pDataBlock, 1)); +// colDataAppend(pColInfo2, i, b1, false); + } + + pBlock->info.rows = pInfo->numOfRowsPerPage; + pBlock->info.numOfCols = 1; + + pInfo->current += 1; + + blockDataUpdateTsWindow(pBlock); + return pBlock; + +} + +SOperatorInfo* createDummyOperator(int32_t startVal, int32_t numOfBlocks, int32_t rowsPerPage, int32_t type, int32_t numOfCols) { SOperatorInfo* pOperator = static_cast(calloc(1, sizeof(SOperatorInfo))); pOperator->name = "dummyInputOpertor4Test"; - pOperator->exec = getDummyBlock; + + if (numOfCols == 1) { + pOperator->exec = getDummyBlock; + } else { + pOperator->exec = get2ColsDummyBlock; + } SDummyInputInfo *pInfo = (SDummyInputInfo*) calloc(1, sizeof(SDummyInputInfo)); pInfo->totalPages = numOfBlocks; pInfo->startVal = startVal; pInfo->numOfRowsPerPage = rowsPerPage; pInfo->type = type; + pInfo->tsStart = 1620000000000; pOperator->info = pInfo; return pOperator; @@ -357,8 +440,6 @@ TEST(testCase, external_sort_Test) { taosArrayDestroy(pOrderVal); } - - TEST(testCase, sorted_merge_Test) { srand(time(NULL)); @@ -432,4 +513,79 @@ TEST(testCase, sorted_merge_Test) { } #endif + +TEST(testCase, time_interval_Operator_Test) { + srand(time(NULL)); + + SArray* pOrderVal = taosArrayInit(4, sizeof(SOrder)); + SOrder o = {0}; + o.order = TSDB_ORDER_ASC; + o.col.info.colId = 1; + o.col.info.type = TSDB_DATA_TYPE_INT; + taosArrayPush(pOrderVal, &o); + + SArray* pExprInfo = taosArrayInit(4, sizeof(SExprInfo)); + SExprInfo *exp = static_cast(calloc(1, sizeof(SExprInfo))); + exp->base.resSchema = createSchema(TSDB_DATA_TYPE_TIMESTAMP, sizeof(int64_t), 1, "ts"); + exp->base.pColumns = static_cast(calloc(1, sizeof(SColumn))); + exp->base.pColumns->flag = TSDB_COL_NORMAL; + exp->base.pColumns->info = (SColumnInfo) {.colId = 1, .type = TSDB_DATA_TYPE_TIMESTAMP, .bytes = 8}; + exp->base.numOfCols = 1; + + taosArrayPush(pExprInfo, &exp); + + SExprInfo *exp1 = static_cast(calloc(1, sizeof(SExprInfo))); + exp1->base.resSchema = createSchema(TSDB_DATA_TYPE_BIGINT, 8, 2, "res1"); + exp1->base.pColumns = static_cast(calloc(1, sizeof(SColumn))); + exp1->base.pColumns->flag = TSDB_COL_NORMAL; + exp1->base.pColumns->info = (SColumnInfo) {.colId = 1, .type = TSDB_DATA_TYPE_INT, .bytes = 4}; + exp1->base.numOfCols = 1; + + taosArrayPush(pExprInfo, &exp1); + + SOperatorInfo* p = createDummyOperator(1, 1, 2000, data_asc, 2); + + SExecTaskInfo ti = {0}; + SOperatorInfo* pOperator = createIntervalOperatorInfo(p, pExprInfo, &ti); + + bool newgroup = false; + SSDataBlock* pRes = NULL; + + int32_t total = 1; + + int64_t s1 = taosGetTimestampUs(); + int32_t t = 1; + + while(1) { + int64_t s = taosGetTimestampUs(); + pRes = pOperator->exec(pOperator, &newgroup); + + int64_t e = taosGetTimestampUs(); + if (t++ == 1) { + printf("---------------elapsed:%ld\n", e - s); + } + + if (pRes == NULL) { + break; + } + + SColumnInfoData* pCol1 = static_cast(taosArrayGet(pRes->pDataBlock, 0)); +// SColumnInfoData* pCol2 = static_cast(taosArrayGet(pRes->pDataBlock, 1)); + for (int32_t i = 0; i < pRes->info.rows; ++i) { +// char* p = colDataGetData(pCol2, i); + printf("%d: %ld\n", total++, ((int64_t*)pCol1->pData)[i]); +// printf("%d: %d, %s\n", total++, ((int32_t*)pCol1->pData)[i], (char*)varDataVal(p)); + } + } + + int64_t s2 = taosGetTimestampUs(); + printf("total:%ld\n", s2 - s1); + + pOperator->cleanupFn(pOperator->info, 2); + tfree(exp); + tfree(exp1); + taosArrayDestroy(pExprInfo); + taosArrayDestroy(pOrderVal); +} + #pragma GCC diagnostic pop diff --git a/source/libs/executor/test/lhashTests.cpp b/source/libs/executor/test/lhashTests.cpp index d0fe9c5ac0..66ef3b0877 100644 --- a/source/libs/executor/test/lhashTests.cpp +++ b/source/libs/executor/test/lhashTests.cpp @@ -28,9 +28,9 @@ TEST(testCase, linear_hash_Tests) { srand(time(NULL)); _hash_fn_t fn = taosGetDefaultHashFunction(TSDB_DATA_TYPE_INT); -#if 1 - SLHashObj* pHashObj = tHashInit(10, 128 + 8, fn, 8); - for(int32_t i = 0; i < 100; ++i) { +#if 0 + SLHashObj* pHashObj = tHashInit(256, 4096, fn, 320); + for(int32_t i = 0; i < 5000000; ++i) { int32_t code = tHashPut(pHashObj, &i, sizeof(i), &i, sizeof(i)); assert(code == 0); } @@ -46,13 +46,13 @@ TEST(testCase, linear_hash_Tests) { // } // } - tHashPrint(pHashObj, LINEAR_HASH_DATA); + tHashPrint(pHashObj, LINEAR_HASH_STATIS); tHashCleanup(pHashObj); #endif #if 0 SHashObj* pHashObj = taosHashInit(1000, fn, false, HASH_NO_LOCK); - for(int32_t i = 0; i < 500000; ++i) { + for(int32_t i = 0; i < 1000000; ++i) { taosHashPut(pHashObj, &i, sizeof(i), &i, sizeof(i)); } diff --git a/source/libs/function/src/taggfunction.c b/source/libs/function/src/taggfunction.c index 0615ff9627..0c2ce821d2 100644 --- a/source/libs/function/src/taggfunction.c +++ b/source/libs/function/src/taggfunction.c @@ -4395,7 +4395,7 @@ SFunctionFpSet fpSet[1] = { .addInput = count_function, .finalize = doFinalizer, .combine = count_func_merge, - } + }, }; SAggFunctionInfo aggFunc[35] = {{ diff --git a/source/util/src/tpagedbuf.c b/source/util/src/tpagedbuf.c index 3fe3ddccde..d2d1b5a367 100644 --- a/source/util/src/tpagedbuf.c +++ b/source/util/src/tpagedbuf.c @@ -5,12 +5,6 @@ #include "tcompression.h" #include "thash.h" -//enum { -// true = 0x1, -// BUF_PAGE_RELEASED = 0x2, -// true = 0x3, -//}; - #define GET_DATA_PAYLOAD(_p) ((char *)(_p)->pData + POINTER_BYTES) #define NO_IN_MEM_AVAILABLE_PAGES(_b) (listNEles((_b)->lruList) >= (_b)->inMemPages) @@ -20,7 +14,7 @@ typedef struct SPageDiskInfo { } SPageDiskInfo, SFreeListItem; struct SPageInfo { - SListNode* pn; // point to list node + SListNode* pn; // point to list node struct void* pData; int64_t offset; int32_t pageId; @@ -38,7 +32,8 @@ struct SDiskbasedBuf { char* path; // file path int32_t pageSize; // current used page size int32_t inMemPages; // numOfPages that are allocated in memory - SHashObj* groupSet; // id hash table + SList* freePgList; // free page list + SHashObj* groupSet; // id hash table, todo remove it SHashObj* all; SList* lruList; void* emptyDummyIdList; // dummy id list @@ -110,6 +105,14 @@ static uint64_t allocatePositionInFile(SDiskbasedBuf* pBuf, size_t size) { } } +static void setPageNotInBuf(SPageInfo* pPageInfo) { + pPageInfo->pData = NULL; +} + +static FORCE_INLINE size_t getAllocPageSize(int32_t pageSize) { + return pageSize + POINTER_BYTES + 2; +} + /** * +--------------------------+-------------------+--------------+ * | PTR to SPageInfo (8bytes)| Payload (PageSize)| 2 Extra Bytes| @@ -189,17 +192,17 @@ static char* doFlushPageToDisk(SDiskbasedBuf* pBuf, SPageInfo* pg) { } } else {// NOTE: the size may be -1, the this recycle page has not been flushed to disk yet. size = pg->length; + if (size == -1) { + printf("----\n"); + } } ASSERT(size > 0 || (pg->offset == -1 && pg->length == -1)); char* pDataBuf = pg->pData; - memset(pDataBuf, 0, pBuf->pageSize); - - pg->pData = NULL; // this means the data is not in buffer - pg->length = size; - pg->dirty = false; + memset(pDataBuf, 0, getAllocPageSize(pBuf->pageSize)); + pg->length = size; // on disk size return pDataBuf; } @@ -214,7 +217,11 @@ static char* flushPageToDisk(SDiskbasedBuf* pBuf, SPageInfo* pg) { } } - return doFlushPageToDisk(pBuf, pg); + char* p = doFlushPageToDisk(pBuf, pg); + setPageNotInBuf(pg); + pg->dirty = false; + + return p; } // load file block data in disk @@ -284,12 +291,23 @@ static SListNode* getEldestUnrefedPage(SDiskbasedBuf* pBuf) { assert(pageInfo->pageId >= 0 && pageInfo->pn == pn); if (!pageInfo->used) { +// printf("%d is chosen\n", pageInfo->pageId); break; } else { - printf("page %d is used, dirty:%d\n", pageInfo->pageId, pageInfo->dirty); +// printf("page %d is used, dirty:%d\n", pageInfo->pageId, pageInfo->dirty); } } +// int32_t pos = listNEles(pBuf->lruList); +// SListIter iter1 = {0}; +// tdListInitIter(pBuf->lruList, &iter1, TD_LIST_BACKWARD); +// SListNode* pn1 = NULL; +// while((pn1 = tdListNext(&iter1)) != NULL) { +// SPageInfo* pageInfo = *(SPageInfo**) pn1->data; +// printf("page %d is used, dirty:%d, pos:%d\n", pageInfo->pageId, pageInfo->dirty, pos - 1); +// pos -= 1; +// } + return pn; } @@ -333,10 +351,6 @@ static void lruListMoveToFront(SList *pList, SPageInfo* pi) { tdListPrependNode(pList, pi->pn); } -static FORCE_INLINE size_t getAllocPageSize(int32_t pageSize) { - return pageSize + POINTER_BYTES + 2; -} - static SPageInfo* getPageInfoFromPayload(void* page) { int32_t offset = offsetof(SPageInfo, pData); char* p = page - offset; @@ -348,41 +362,42 @@ static SPageInfo* getPageInfoFromPayload(void* page) { int32_t createDiskbasedBuf(SDiskbasedBuf** pBuf, int32_t pagesize, int32_t inMemBufSize, uint64_t qId, const char* dir) { *pBuf = calloc(1, sizeof(SDiskbasedBuf)); - SDiskbasedBuf* pResBuf = *pBuf; - if (pResBuf == NULL) { + SDiskbasedBuf* pPBuf = *pBuf; + if (pPBuf == NULL) { return TSDB_CODE_OUT_OF_MEMORY; } - pResBuf->pageSize = pagesize; - pResBuf->numOfPages = 0; // all pages are in buffer in the first place - pResBuf->totalBufSize = 0; - pResBuf->inMemPages = inMemBufSize/pagesize; // maximum allowed pages, it is a soft limit. - pResBuf->allocateId = -1; - pResBuf->comp = true; - pResBuf->file = NULL; - pResBuf->qId = qId; - pResBuf->fileSize = 0; - pResBuf->pFree = taosArrayInit(4, sizeof(SFreeListItem)); + pPBuf->pageSize = pagesize; + pPBuf->numOfPages = 0; // all pages are in buffer in the first place + pPBuf->totalBufSize = 0; + pPBuf->inMemPages = inMemBufSize/pagesize; // maximum allowed pages, it is a soft limit. + pPBuf->allocateId = -1; + pPBuf->comp = true; + pPBuf->file = NULL; + pPBuf->qId = qId; + pPBuf->fileSize = 0; + pPBuf->pFree = taosArrayInit(4, sizeof(SFreeListItem)); + pPBuf->freePgList = tdListNew(POINTER_BYTES); // at least more than 2 pages must be in memory assert(inMemBufSize >= pagesize * 2); - pResBuf->lruList = tdListNew(POINTER_BYTES); + pPBuf->lruList = tdListNew(POINTER_BYTES); // init id hash table _hash_fn_t fn = taosGetDefaultHashFunction(TSDB_DATA_TYPE_INT); - pResBuf->groupSet = taosHashInit(10, fn, true, false); - pResBuf->assistBuf = malloc(pResBuf->pageSize + 2); // EXTRA BYTES - pResBuf->all = taosHashInit(10, fn, true, false); + pPBuf->groupSet = taosHashInit(10, fn, true, false); + pPBuf->assistBuf = malloc(pPBuf->pageSize + 2); // EXTRA BYTES + pPBuf->all = taosHashInit(10, fn, true, false); char path[PATH_MAX] = {0}; taosGetTmpfilePath(dir, "paged-buf", path); - pResBuf->path = strdup(path); + pPBuf->path = strdup(path); - pResBuf->emptyDummyIdList = taosArrayInit(1, sizeof(int32_t)); + pPBuf->emptyDummyIdList = taosArrayInit(1, sizeof(int32_t)); -// qDebug("QInfo:0x%"PRIx64" create resBuf for output, page size:%d, inmem buf pages:%d, file:%s", qId, pResBuf->pageSize, -// pResBuf->inMemPages, pResBuf->path); +// qDebug("QInfo:0x%"PRIx64" create resBuf for output, page size:%d, inmem buf pages:%d, file:%s", qId, pPBuf->pageSize, +// pPBuf->inMemPages, pPBuf->path); return TSDB_CODE_SUCCESS; } @@ -401,19 +416,29 @@ void* getNewBufPage(SDiskbasedBuf* pBuf, int32_t groupId, int32_t* pageId) { } } - // register new id in this group - *pageId = (++pBuf->allocateId); + SPageInfo* pi = NULL; + if (listNEles(pBuf->freePgList) != 0) { + SListNode* pItem = tdListPopHead(pBuf->freePgList); + pi = *(SPageInfo**) pItem->data; + pi->used = true; + *pageId = pi->pageId; + tfree(pItem); + } else {// create a new pageinfo + // register new id in this group + *pageId = (++pBuf->allocateId); - // register page id info - SPageInfo* pi = registerPage(pBuf, groupId, *pageId); + // register page id info + pi = registerPage(pBuf, groupId, *pageId); + + // add to hash map + taosHashPut(pBuf->all, pageId, sizeof(int32_t), &pi, POINTER_BYTES); + pBuf->totalBufSize += pBuf->pageSize; + } // add to LRU list assert(listNEles(pBuf->lruList) < pBuf->inMemPages && pBuf->inMemPages > 0); lruListPushFront(pBuf->lruList, pi); - // add to hash map - taosHashPut(pBuf->all, pageId, sizeof(int32_t), &pi, POINTER_BYTES); - // allocate buf if (availablePage == NULL) { pi->pData = calloc(1, getAllocPageSize(pBuf->pageSize)); // add extract bytes in case of zipped buffer increased. @@ -421,11 +446,7 @@ void* getNewBufPage(SDiskbasedBuf* pBuf, int32_t groupId, int32_t* pageId) { pi->pData = availablePage; } - pBuf->totalBufSize += pBuf->pageSize; - ((void**)pi->pData)[0] = pi; - pi->used = true; - return (void *)(GET_DATA_PAYLOAD(pi)); } @@ -467,6 +488,7 @@ void* getBufPage(SDiskbasedBuf* pBuf, int32_t id) { (*pi)->pData = availablePage; } + // set the ptr to the new SPageInfo ((void**)((*pi)->pData))[0] = (*pi); lruListPushFront(pBuf->lruList, *pi); @@ -551,6 +573,8 @@ void destroyDiskbasedBuf(SDiskbasedBuf* pBuf) { } tdListFree(pBuf->lruList); + tdListFree(pBuf->freePgList); + taosArrayDestroy(pBuf->emptyDummyIdList); taosArrayDestroy(pBuf->pFree); @@ -596,14 +620,15 @@ void setBufPageCompressOnDisk(SDiskbasedBuf* pBuf, bool comp) { void dBufSetBufPageRecycled(SDiskbasedBuf *pBuf, void* pPage) { SPageInfo* ppi = getPageInfoFromPayload(pPage); - ppi->used = false; + ppi->used = false; ppi->dirty = false; - // it is a in-memory page that has not been flushed to disk yet. - if (ppi->length != -1 && ppi->offset != -1) { - SFreeListItem item = {.length = ppi->length, .offset = ppi->offset}; - taosArrayPush(pBuf->pFree, &item); - } + // add this pageinfo into the free page info list + SListNode* pNode = tdListPopNode(pBuf->lruList, ppi->pn); + tfree(ppi->pData); + tfree(pNode); + + tdListAppend(pBuf->freePgList, &ppi); } void dBufSetPrintInfo(SDiskbasedBuf* pBuf) { From 90d231eb7ebed3be976fcf0554f280e166a8ef42 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 24 Feb 2022 17:58:16 +0800 Subject: [PATCH 22/25] [td-13039] merge 3.0 --- source/libs/executor/src/executorimpl.c | 2 +- source/libs/function/src/tpercentile.c | 2 +- source/libs/scalar/inc/filterInt.h | 2 +- source/libs/scalar/src/filter.c | 13 +++++++------ source/libs/scalar/src/sclvector.c | 2 +- 5 files changed, 11 insertions(+), 10 deletions(-) diff --git a/source/libs/executor/src/executorimpl.c b/source/libs/executor/src/executorimpl.c index d8d418370f..6858f68baa 100644 --- a/source/libs/executor/src/executorimpl.c +++ b/source/libs/executor/src/executorimpl.c @@ -4620,7 +4620,7 @@ int32_t doInitQInfo(SQInfo* pQInfo, STSBuf* pTsBuf, void* tsdb, void* sourceOptr getIntermediateBufInfo(pRuntimeEnv, &ps, &pQueryAttr->intermediateResultRowSize); int32_t TENMB = 1024*1024*10; - int32_t code = createDiskbasedBuf(&pRuntimeEnv->pResultBuf, ps, TENMB, pQInfo->qId, tsTempDir); + int32_t code = createDiskbasedBuf(&pRuntimeEnv->pResultBuf, ps, TENMB, pQInfo->qId, osTempDir()); if (code != TSDB_CODE_SUCCESS) { return code; } diff --git a/source/libs/function/src/tpercentile.c b/source/libs/function/src/tpercentile.c index eab05ad039..27cdc15d94 100644 --- a/source/libs/function/src/tpercentile.c +++ b/source/libs/function/src/tpercentile.c @@ -254,7 +254,7 @@ tMemBucket *tMemBucketCreate(int16_t nElemSize, int16_t dataType, double minval, resetSlotInfo(pBucket); - int32_t ret = createDiskbasedBuf(&pBucket->pBuffer, pBucket->bufPageSize, pBucket->bufPageSize * 512, 1, tsTempDir); + int32_t ret = createDiskbasedBuf(&pBucket->pBuffer, pBucket->bufPageSize, pBucket->bufPageSize * 512, 1, "/tmp"); if (ret != 0) { tMemBucketDestroy(pBucket); return NULL; diff --git a/source/libs/scalar/inc/filterInt.h b/source/libs/scalar/inc/filterInt.h index f9aa994283..fde2273241 100644 --- a/source/libs/scalar/inc/filterInt.h +++ b/source/libs/scalar/inc/filterInt.h @@ -306,7 +306,7 @@ typedef struct SFilterInfo { #define FILTER_GET_COL_FIELD_ID(fi) (((SColumnRefNode *)((fi)->desc))->columnId) #define FILTER_GET_COL_FIELD_SLOT_ID(fi) (((SColumnRefNode *)((fi)->desc))->slotId) #define FILTER_GET_COL_FIELD_DESC(fi) ((SColumnRefNode *)((fi)->desc)) -#define FILTER_GET_COL_FIELD_DATA(fi, ri) (colDataGet(((SColumnInfoData *)(fi)->data), (ri))) +#define FILTER_GET_COL_FIELD_DATA(fi, ri) (colDataGetData(((SColumnInfoData *)(fi)->data), (ri))) #define FILTER_GET_VAL_FIELD_TYPE(fi) (((SValueNode *)((fi)->desc))->node.resType.type) #define FILTER_GET_VAL_FIELD_DATA(fi) ((char *)(fi)->data) #define FILTER_GET_JSON_VAL_FIELD_DATA(fi) ((char *)(fi)->desc) diff --git a/source/libs/scalar/src/filter.c b/source/libs/scalar/src/filter.c index c8f94d2d97..8dba4aaba9 100644 --- a/source/libs/scalar/src/filter.c +++ b/source/libs/scalar/src/filter.c @@ -19,6 +19,7 @@ #include "tcompare.h" #include "filterInt.h" #include "filter.h" +#include "tep.h" OptrStr gOptrStr[] = { {0, "invalid"}, @@ -2776,7 +2777,7 @@ bool filterExecuteBasedOnStatisImpl(void *pinfo, int32_t numOfRows, int8_t** p, uint32_t unitNum = *(unitIdx++); for (uint32_t u = 0; u < unitNum; ++u) { SFilterComUnit *cunit = &info->cunits[*(unitIdx + u)]; - void *colData = colDataGet((SColumnInfoData *)cunit->colData, i); + void *colData = colDataGetData((SColumnInfoData *)cunit->colData, i); //if (FILTER_UNIT_GET_F(info, uidx)) { // p[i] = FILTER_UNIT_GET_R(info, uidx); @@ -2874,7 +2875,7 @@ static FORCE_INLINE bool filterExecuteImplIsNull(void *pinfo, int32_t numOfRows, for (int32_t i = 0; i < numOfRows; ++i) { uint32_t uidx = info->groups[0].unitIdxs[0]; - void *colData = colDataGet((SColumnInfoData *)info->cunits[uidx].colData, i); + void *colData = colDataGetData((SColumnInfoData *)info->cunits[uidx].colData, i); if(info->cunits[uidx].dataType == TSDB_DATA_TYPE_JSON){ if (!colData){ // for json->'key' is null (*p)[i] = 1; @@ -2908,7 +2909,7 @@ static FORCE_INLINE bool filterExecuteImplNotNull(void *pinfo, int32_t numOfRows for (int32_t i = 0; i < numOfRows; ++i) { uint32_t uidx = info->groups[0].unitIdxs[0]; - void *colData = colDataGet((SColumnInfoData *)info->cunits[uidx].colData, i); + void *colData = colDataGetData((SColumnInfoData *)info->cunits[uidx].colData, i); if(info->cunits[uidx].dataType == TSDB_DATA_TYPE_JSON){ if (!colData) { // for json->'key' is not null @@ -2949,7 +2950,7 @@ bool filterExecuteImplRange(void *pinfo, int32_t numOfRows, int8_t** p, SColumnD } for (int32_t i = 0; i < numOfRows; ++i) { - void *colData = colDataGet((SColumnInfoData *)info->cunits[0].colData, i); + void *colData = colDataGetData((SColumnInfoData *)info->cunits[0].colData, i); if (colData == NULL || isNull(colData, info->cunits[0].dataType)) { all = false; @@ -2980,7 +2981,7 @@ bool filterExecuteImplMisc(void *pinfo, int32_t numOfRows, int8_t** p, SColumnDa for (int32_t i = 0; i < numOfRows; ++i) { uint32_t uidx = info->groups[0].unitIdxs[0]; - void *colData = colDataGet((SColumnInfoData *)info->cunits[uidx].colData, i); + void *colData = colDataGetData((SColumnInfoData *)info->cunits[uidx].colData, i); if (colData == NULL || isNull(colData, info->cunits[uidx].dataType)) { (*p)[i] = 0; all = false; @@ -3031,7 +3032,7 @@ bool filterExecuteImpl(void *pinfo, int32_t numOfRows, int8_t** p, SColumnDataAg for (uint32_t u = 0; u < group->unitNum; ++u) { uint32_t uidx = group->unitIdxs[u]; SFilterComUnit *cunit = &info->cunits[uidx]; - void *colData = colDataGet((SColumnInfoData *)(cunit->colData), i); + void *colData = colDataGetData((SColumnInfoData *)(cunit->colData), i); //if (FILTER_UNIT_GET_F(info, uidx)) { // p[i] = FILTER_UNIT_GET_R(info, uidx); diff --git a/source/libs/scalar/src/sclvector.c b/source/libs/scalar/src/sclvector.c index 85af663313..26ffeb2d60 100644 --- a/source/libs/scalar/src/sclvector.c +++ b/source/libs/scalar/src/sclvector.c @@ -226,7 +226,7 @@ void* getVectorValueAddr_default(void *src, int32_t index) { return src; } void* getVectorValueAddr_VAR(void *src, int32_t index) { - return colDataGet((SColumnInfoData *)src, index); + return colDataGetData((SColumnInfoData *)src, index); } _getValueAddr_fn_t getVectorValueAddrFn(int32_t srcType) { From 30dd3305554b12f2b89ec056f71d4761b92fc775 Mon Sep 17 00:00:00 2001 From: afwerar <1296468573@qq.com> Date: Sun, 27 Feb 2022 15:41:21 +0800 Subject: [PATCH 23/25] invalid read --- source/libs/transport/src/transCli.c | 2 +- source/util/src/tconfig.c | 33 +++++++----------- source/util/src/tlog.c | 52 +++++++++------------------- 3 files changed, 30 insertions(+), 57 deletions(-) diff --git a/source/libs/transport/src/transCli.c b/source/libs/transport/src/transCli.c index d5fa4c8c47..fb76f38fe5 100644 --- a/source/libs/transport/src/transCli.c +++ b/source/libs/transport/src/transCli.c @@ -127,7 +127,7 @@ static void clientHandleResp(SCliConn* conn) { // buf's mem alread translated to rpcMsg.pCont transClearBuffer(&conn->readBuf); - SRpcMsg rpcMsg; + SRpcMsg rpcMsg = {0}; rpcMsg.contLen = transContLenFromMsg(pHead->msgLen); rpcMsg.pCont = transContFromHead((char*)pHead); rpcMsg.code = pHead->code; diff --git a/source/util/src/tconfig.c b/source/util/src/tconfig.c index 0ee1b9e1d6..dc9fa33595 100644 --- a/source/util/src/tconfig.c +++ b/source/util/src/tconfig.c @@ -96,6 +96,7 @@ void cfgCleanup(SConfig *pCfg) { SConfigItem *pItem = taosHashIterate(pCfg->hash, NULL); while (pItem != NULL) { cfgFreeItem(pItem); + tfree(pItem->name); pItem = taosHashIterate(pCfg->hash, pItem); } taosHashCleanup(pCfg->hash); @@ -249,9 +250,7 @@ static int32_t cfgSetString(SConfigItem *pItem, const char *value, ECfgSrcType s } static int32_t cfgSetDir(SConfigItem *pItem, const char *value, ECfgSrcType stype) { - char *tmp = strdup(value); - if (tmp == NULL || cfgCheckAndSetDir(pItem, value) != 0) { - free(tmp); + if (cfgCheckAndSetDir(pItem, value) != 0) { terrno = TSDB_CODE_OUT_OF_MEMORY; uError("cfg:%s, type:%s src:%s value:%s failed to dup since %s, use last src:%s value:%s", pItem->name, cfgDtypeStr(pItem->dtype), cfgStypeStr(stype), value, terrstr(), cfgStypeStr(pItem->stype), pItem->str); @@ -263,9 +262,7 @@ static int32_t cfgSetDir(SConfigItem *pItem, const char *value, ECfgSrcType styp } static int32_t cfgSetLocale(SConfigItem *pItem, const char *value, ECfgSrcType stype) { - char *tmp = strdup(value); - if (tmp == NULL || cfgCheckAndSetLocale(pItem, value) != 0) { - free(tmp); + if (cfgCheckAndSetLocale(pItem, value) != 0) { terrno = TSDB_CODE_OUT_OF_MEMORY; uError("cfg:%s, type:%s src:%s value:%s failed to dup since %s, use last src:%s value:%s", pItem->name, cfgDtypeStr(pItem->dtype), cfgStypeStr(stype), value, terrstr(), cfgStypeStr(pItem->stype), pItem->str); @@ -277,9 +274,7 @@ static int32_t cfgSetLocale(SConfigItem *pItem, const char *value, ECfgSrcType s } static int32_t cfgSetCharset(SConfigItem *pItem, const char *value, ECfgSrcType stype) { - char *tmp = strdup(value); - if (tmp == NULL || cfgCheckAndSetCharset(pItem, value) != 0) { - free(tmp); + if (cfgCheckAndSetCharset(pItem, value) != 0) { terrno = TSDB_CODE_OUT_OF_MEMORY; uError("cfg:%s, type:%s src:%s value:%s failed to dup since %s, use last src:%s value:%s", pItem->name, cfgDtypeStr(pItem->dtype), cfgStypeStr(stype), value, terrstr(), cfgStypeStr(pItem->stype), pItem->str); @@ -291,9 +286,7 @@ static int32_t cfgSetCharset(SConfigItem *pItem, const char *value, ECfgSrcType } static int32_t cfgSetTimezone(SConfigItem *pItem, const char *value, ECfgSrcType stype) { - char *tmp = strdup(value); - if (tmp == NULL || cfgCheckAndSetTimezone(pItem, value) != 0) { - free(tmp); + if (cfgCheckAndSetTimezone(pItem, value) != 0) { terrno = TSDB_CODE_OUT_OF_MEMORY; uError("cfg:%s, type:%s src:%s value:%s failed to dup since %s, use last src:%s value:%s", pItem->name, cfgDtypeStr(pItem->dtype), cfgStypeStr(stype), value, terrstr(), cfgStypeStr(pItem->stype), pItem->str); @@ -366,11 +359,11 @@ int32_t cfgSetItem(SConfig *pCfg, const char *name, const char *value, ECfgSrcTy } SConfigItem *cfgGetItem(SConfig *pCfg, const char *name) { - char lowcaseName[CFG_NAME_MAX_LEN + 1] = {0}; - memcpy(lowcaseName, name, CFG_NAME_MAX_LEN); - strntolower(lowcaseName, name, CFG_NAME_MAX_LEN); + int32_t len = strlen(name); + char lowcaseName[CFG_NAME_MAX_LEN + 1] = {0}; + strntolower(lowcaseName, name, TMIN(CFG_NAME_MAX_LEN, len)); - SConfigItem *pItem = taosHashGet(pCfg->hash, lowcaseName, strlen(lowcaseName) + 1); + SConfigItem *pItem = taosHashGet(pCfg->hash, lowcaseName, len + 1); if (pItem == NULL) { terrno = TSDB_CODE_CFG_NOT_FOUND; } @@ -386,11 +379,11 @@ static int32_t cfgAddItem(SConfig *pCfg, SConfigItem *pItem, const char *name) { return -1; } - char lowcaseName[CFG_NAME_MAX_LEN + 1] = {0}; - memcpy(lowcaseName, name, CFG_NAME_MAX_LEN); - strntolower(lowcaseName, name, CFG_NAME_MAX_LEN); + int32_t len = strlen(name); + char lowcaseName[CFG_NAME_MAX_LEN + 1] = {0}; + strntolower(lowcaseName, name, TMIN(CFG_NAME_MAX_LEN, len)); - if (taosHashPut(pCfg->hash, lowcaseName, strlen(lowcaseName) + 1, pItem, sizeof(SConfigItem)) != 0) { + if (taosHashPut(pCfg->hash, lowcaseName, len + 1, pItem, sizeof(SConfigItem)) != 0) { if (pItem->dtype == CFG_DTYPE_STRING) { free(pItem->str); } diff --git a/source/util/src/tlog.c b/source/util/src/tlog.c index 11b4555737..2ed8d6e347 100644 --- a/source/util/src/tlog.c +++ b/source/util/src/tlog.c @@ -64,16 +64,17 @@ typedef struct { pthread_mutex_t logMutex; } SLogObj; -int8_t tscEmbeddedInUtil = 0; +static int8_t tsLogInited = 0; +static SLogObj tsLogObj = {.fileNum = 1}; +int8_t tscEmbeddedInUtil = 0; int32_t tsLogKeepDays = 0; bool tsAsyncLog = true; -int8_t tsLogInited = 0; -int64_t asyncLogLostLines = 0; -int32_t writeInterval = LOG_DEFAULT_INTERVAL; +int32_t tsNumOfLogLines = 10000000; +int64_t tsAsyncLogLostLines = 0; +int32_t tsWriteInterval = LOG_DEFAULT_INTERVAL; // log -int32_t tsNumOfLogLines = 10000000; int32_t dDebugFlag = 135; int32_t vDebugFlag = 135; int32_t mDebugFlag = 131; @@ -95,13 +96,11 @@ int64_t dbgSmallWN = 0; int64_t dbgBigWN = 0; int64_t dbgWSize = 0; -static SLogObj tsLogObj = {.fileNum = 1}; static void *taosAsyncOutputLog(void *param); static int32_t taosPushLogBuffer(SLogBuff *tLogBuff, char *msg, int32_t msgLen); static SLogBuff *taosLogBuffNew(int32_t bufSize); static void taosCloseLogByFd(TdFilePtr pFile); static int32_t taosOpenLogFile(char *fn, int32_t maxLines, int32_t maxFileNum); -extern void taosPrintCfg(); static int32_t taosCompressFile(char *srcFileName, char *destFileName); static int32_t taosStartLog() { @@ -125,7 +124,6 @@ int32_t taosInitLog(const char *logName, int maxFiles) { if (tsLogObj.logHandle == NULL) return -1; if (taosOpenLogFile(fullName, tsNumOfLogLines, maxFiles) < 0) return -1; if (taosStartLog() < 0) return -1; - tsLogInited = true; return 0; } @@ -137,8 +135,6 @@ static void taosStopLog() { void taosCloseLog() { taosStopLog(); - // tsem_post(&(tsLogObj.logHandle->buffNotEmpty)); - taosMsleep(LOG_MAX_INTERVAL / 1000); if (taosCheckPthreadValid(tsLogObj.logHandle->asyncThread)) { pthread_join(tsLogObj.logHandle->asyncThread, NULL); } @@ -218,7 +214,6 @@ static void *taosThreadToOpenNewFile(void *param) { uInfo(" new log file:%d is opened", tsLogObj.flag); uInfo("=================================="); - // taosPrintCfg(); taosKeepOldLog(keepName); return NULL; @@ -498,12 +493,6 @@ void taosPrintLongString(const char *flags, int32_t dflag, const char *format, . if (dflag & DEBUG_SCREEN) write(1, buffer, (uint32_t)len); } -#if 0 -void taosCloseLog() { - taosCloseLogByFd(tsLogObj.logHandle->pFile); -} -#endif - static void taosCloseLogByFd(TdFilePtr pFile) { if (pFile != NULL) { taosUnLockLogFile(pFile); @@ -536,15 +525,6 @@ _err: return NULL; } -#if 0 -static void taosLogBuffDestroy(SLogBuff *tLogBuff) { - tsem_destroy(&(tLogBuff->buffNotEmpty)); - pthread_mutex_destroy(&(tLogBuff->buffMutex)); - free(tLogBuff->buffer); - tfree(tLogBuff); -} -#endif - static void taosCopyLogBuffer(SLogBuff *tLogBuff, int32_t start, int32_t end, char *msg, int32_t msgLen) { if (start > end) { memcpy(LOG_BUF_BUFFER(tLogBuff) + end, msg, msgLen); @@ -582,7 +562,7 @@ static int32_t taosPushLogBuffer(SLogBuff *tLogBuff, char *msg, int32_t msgLen) if (remainSize <= msgLen || ((lostLine > 0) && (remainSize <= (msgLen + tmpBufLen)))) { lostLine++; - asyncLogLostLines++; + tsAsyncLogLostLines++; pthread_mutex_unlock(&LOG_BUF_MUTEX(tLogBuff)); return -1; } @@ -627,13 +607,13 @@ static void taosWriteLog(SLogBuff *tLogBuff) { if (start == end) { dbgEmptyW++; - writeInterval = LOG_MAX_INTERVAL; + tsWriteInterval = LOG_MAX_INTERVAL; return; } pollSize = taosGetLogRemainSize(tLogBuff, start, end); if (pollSize < tLogBuff->minBuffSize) { - lastDuration += writeInterval; + lastDuration += tsWriteInterval; if (lastDuration < LOG_MAX_WAIT_MSEC) { break; } @@ -656,15 +636,15 @@ static void taosWriteLog(SLogBuff *tLogBuff) { if (pollSize < tLogBuff->minBuffSize) { dbgSmallWN++; - if (writeInterval < LOG_MAX_INTERVAL) { - writeInterval += LOG_INTERVAL_STEP; + if (tsWriteInterval < LOG_MAX_INTERVAL) { + tsWriteInterval += LOG_INTERVAL_STEP; } } else if (pollSize > LOG_BUF_SIZE(tLogBuff) / 3) { dbgBigWN++; - writeInterval = LOG_MIN_INTERVAL; + tsWriteInterval = LOG_MIN_INTERVAL; } else if (pollSize > LOG_BUF_SIZE(tLogBuff) / 4) { - if (writeInterval > LOG_MIN_INTERVAL) { - writeInterval -= LOG_INTERVAL_STEP; + if (tsWriteInterval > LOG_MIN_INTERVAL) { + tsWriteInterval -= LOG_INTERVAL_STEP; } } @@ -678,7 +658,7 @@ static void taosWriteLog(SLogBuff *tLogBuff) { break; } - writeInterval = LOG_MIN_INTERVAL; + tsWriteInterval = LOG_MIN_INTERVAL; remainChecked = 1; } while (1); @@ -689,7 +669,7 @@ static void *taosAsyncOutputLog(void *param) { setThreadName("log"); while (1) { - taosMsleep(writeInterval); + taosMsleep(tsWriteInterval); // Polling the buffer taosWriteLog(tLogBuff); From c4ace1885e8da418361b2e05e032f0df20ec958a Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 28 Feb 2022 13:38:02 +0800 Subject: [PATCH 24/25] [td-13039] remove assert. --- source/libs/executor/src/executorimpl.c | 1 - source/libs/parser/src/queryInfoUtil.c | 8 +++++--- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/source/libs/executor/src/executorimpl.c b/source/libs/executor/src/executorimpl.c index bb1ed09d20..a4a09c91fe 100644 --- a/source/libs/executor/src/executorimpl.c +++ b/source/libs/executor/src/executorimpl.c @@ -4522,7 +4522,6 @@ int32_t appendDownstream(SOperatorInfo* p, SOperatorInfo** pDownstream, int32_t static void doDestroyTableQueryInfo(STableGroupInfo* pTableqinfoGroupInfo); -void createResultBlock(const SArray* pExprInfo, SExchangeInfo* pInfo, const SOperatorInfo* pOperator, size_t size); static int32_t setupQueryHandle(void* tsdb, STaskRuntimeEnv* pRuntimeEnv, int64_t qId, bool isSTableQuery) { STaskAttr *pQueryAttr = pRuntimeEnv->pQueryAttr; #if 0 diff --git a/source/libs/parser/src/queryInfoUtil.c b/source/libs/parser/src/queryInfoUtil.c index 9b355b0775..fc46058aa3 100644 --- a/source/libs/parser/src/queryInfoUtil.c +++ b/source/libs/parser/src/queryInfoUtil.c @@ -230,7 +230,7 @@ int32_t getExprFunctionId(SExprInfo *pExprInfo) { } void assignExprInfo(SExprInfo* dst, const SExprInfo* src) { - assert(dst != NULL && src != NULL && src->base.numOfCols > 0); + assert(dst != NULL && src != NULL/* && src->base.numOfCols > 0*/); *dst = *src; #if 0 @@ -241,8 +241,10 @@ void assignExprInfo(SExprInfo* dst, const SExprInfo* src) { #endif dst->pExpr = exprdup(src->pExpr); - dst->base.pColumns = calloc(src->base.numOfCols, sizeof(SColumn)); - memcpy(dst->base.pColumns, src->base.pColumns, sizeof(SColumn) * src->base.numOfCols); + if (src->base.numOfCols > 0) { + dst->base.pColumns = calloc(src->base.numOfCols, sizeof(SColumn)); + memcpy(dst->base.pColumns, src->base.pColumns, sizeof(SColumn) * src->base.numOfCols); + } memset(dst->base.param, 0, sizeof(SVariant) * tListLen(dst->base.param)); for (int32_t j = 0; j < src->base.numOfParams; ++j) { From ccd5727ba8ae605e8628e77550beff19c597cda1 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 28 Feb 2022 15:03:32 +0800 Subject: [PATCH 25/25] [td-13039] fix bug. --- source/libs/parser/src/queryInfoUtil.c | 2 ++ 1 file changed, 2 insertions(+) diff --git a/source/libs/parser/src/queryInfoUtil.c b/source/libs/parser/src/queryInfoUtil.c index fc46058aa3..05aebb2cb2 100644 --- a/source/libs/parser/src/queryInfoUtil.c +++ b/source/libs/parser/src/queryInfoUtil.c @@ -244,6 +244,8 @@ void assignExprInfo(SExprInfo* dst, const SExprInfo* src) { if (src->base.numOfCols > 0) { dst->base.pColumns = calloc(src->base.numOfCols, sizeof(SColumn)); memcpy(dst->base.pColumns, src->base.pColumns, sizeof(SColumn) * src->base.numOfCols); + } else { + dst->base.pColumns = NULL; } memset(dst->base.param, 0, sizeof(SVariant) * tListLen(dst->base.param));