From dd70c63dfe1d320814c8e35983c50cd2a256c653 Mon Sep 17 00:00:00 2001 From: shenglian zhou Date: Tue, 23 Jan 2024 16:34:14 +0800 Subject: [PATCH 01/41] feat: table merge scan with one less disk usage --- source/libs/executor/inc/executorInt.h | 32 ++++ source/libs/executor/src/scanoperator.c | 235 ++++++++++++++++++++++++ 2 files changed, 267 insertions(+) diff --git a/source/libs/executor/inc/executorInt.h b/source/libs/executor/inc/executorInt.h index 64c14456b6..b20e1a98c0 100644 --- a/source/libs/executor/inc/executorInt.h +++ b/source/libs/executor/inc/executorInt.h @@ -42,6 +42,7 @@ extern "C" { // #include "tstream.h" // #include "tstreamUpdate.h" #include "tlrucache.h" +#include "tdatablock.h" typedef int32_t (*__block_search_fn_t)(char* data, int32_t num, int64_t key, int32_t order); @@ -284,6 +285,34 @@ typedef struct STableScanInfo { bool needCountEmptyTable; } STableScanInfo; +typedef enum ESubTableInputType { + SUB_TABLE_MEM_BLOCK, + SUB_TABLE_EXT_PAGES, +} ESubTableInputType; + +typedef struct STmsSubTableInput { + STsdbReader* pReader; + ESubTableInputType type; + SSDataBlock* pBlock; + SArray* aBlockPages; + + int32_t pageIdx; + int32_t rowIdx; + int64_t* aTs; +} STmsSubTableInput; + +typedef struct STmsSubTablesMergeInfo { + SBlockOrderInfo* pOrderInfo; + + int32_t numSubTables; + STmsSubTableInput* aInputs; + SMultiwayMergeTreeInfo* pTree; + int32_t numTablesCompleted; + + int32_t numTableBlocksInMem; + SDiskbasedBuf* pBlocksBuf; +} STmsSubTablesMergeInfo; + typedef struct STableMergeScanInfo { int32_t tableStartIndex; int32_t tableEndIndex; @@ -318,6 +347,9 @@ typedef struct STableMergeScanInfo { SSDataBlock* nextDurationBlocks[2]; bool rtnNextDurationBlocks; int32_t nextDurationBlocksIdx; + + + STmsSubTablesMergeInfo* pSubTablesMergeInfo; } STableMergeScanInfo; typedef struct STagScanFilterContext { diff --git a/source/libs/executor/src/scanoperator.c b/source/libs/executor/src/scanoperator.c index 09f4f6ac3c..87749d3c81 100644 --- a/source/libs/executor/src/scanoperator.c +++ b/source/libs/executor/src/scanoperator.c @@ -3311,6 +3311,241 @@ _error: return NULL; } +// table merge scan operator + +// table merge scan operator +// TODO: limit / duration optimization +// TODO: get block from tsdReader function, with task killed, func_data all filter out, skip, finish +// TODO: error processing, memory freeing +// TODO: add log for error and perf + +static int32_t subTblRowCompareFn(const void* pLeft, const void* pRight, void* param) { + int32_t left = *(int32_t*)pLeft; + int32_t right = *(int32_t*)pRight; + STmsSubTablesMergeInfo* pInfo = (STmsSubTablesMergeInfo*)param; + + int32_t leftIdx = pInfo->aInputs[left].rowIdx; + int32_t rightIdx = pInfo->aInputs[right].rowIdx; + + if (leftIdx == -1) { + return 1; + } else if (rightIdx == -1) { + return -1; + } + + int64_t leftTs = pInfo->aInputs[left].aTs[leftIdx]; + int64_t rightTs = pInfo->aInputs[right].aTs[rightIdx]; + int32_t ret = leftTs>rightTs ? 1 : ((leftTs < rightTs) ? -1 : 0); + if (pInfo->pOrderInfo->order == TSDB_ORDER_DESC) { + ret = -1 * ret; + } + return ret; +} + +static int32_t fetchNextSubTableBlockFromReader(SOperatorInfo* pOperator, STmsSubTableInput* pInput, bool* pHasNext) { + const STableMergeScanInfo* pInfo = pOperator->info; + STmsSubTablesMergeInfo* pSubTblsInfo = pInfo->pSubTablesMergeInfo; + SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; + const SStorageAPI* pAPI= &pTaskInfo->storageAPI; + + blockDataCleanup(pInput->pBlock); + bool hasNext = false; + int32_t code = pAPI->tsdReader.tsdNextDataBlock(pInput->pReader, &hasNext); + *pHasNext = hasNext; + + if (!hasNext) { + pInput->rowIdx = -1; + ++pSubTblsInfo->numTablesCompleted; + return code; + } + uint32_t status = 0; + code = loadDataBlock(pOperator, &pInfo->base, pInput->pBlock, &status); + + return code; +} + +static int32_t saveSubTableBlock(const STableMergeScanInfo* pInfo, STmsSubTablesMergeInfo* pSubTblsInfo, + STmsSubTableInput* pInput) { + pInput->aBlockPages = taosArrayInit(4, sizeof(int32_t)); + int32_t start = 0; + SSDataBlock* pDataBlock = pInput->pBlock; + while (start < pDataBlock->info.rows) { + int32_t stop = 0; + blockDataSplitRows(pDataBlock, pDataBlock->info.hasVarCol, start, &stop, pInfo->bufPageSize); + SSDataBlock* p = blockDataExtractBlock(pDataBlock, start, stop - start + 1); + + int32_t pageId = -1; + void* pPage = getNewBufPage(pSubTblsInfo->pBlocksBuf, &pageId); + + taosArrayPush(pInput->aBlockPages, &pageId); + + int32_t size = blockDataGetSize(p) + sizeof(int32_t) + taosArrayGetSize(p->pDataBlock) * sizeof(int32_t); + ASSERT(size <= getBufPageSize(pSubTblsInfo->pBlocksBuf)); + + blockDataToBuf(pPage, p); + + setBufPageDirty(pPage, true); + releaseBufPage(pSubTblsInfo->pBlocksBuf, pPage); + + blockDataDestroy(p); + start = stop + 1; + } + blockDataCleanup(pInput->pBlock); + return TSDB_CODE_SUCCESS; +} + +static int32_t fetchNextSubTableBlockFromPage(STmsSubTablesMergeInfo* pSubTblsInfo, STmsSubTableInput* pInput) { + int32_t pageId = *(int32_t*)taosArrayGet(pInput->aBlockPages, pInput->pageIdx); + void* page = getBufPage(pSubTblsInfo->pBlocksBuf, pageId); + blockDataFromBuf(pInput->pBlock, page); + releaseBufPage(pSubTblsInfo->pBlocksBuf, page); + pInput->rowIdx = 0; + return TSDB_CODE_SUCCESS; +} + +static void setGroupStartEndIndex(STableMergeScanInfo* pInfo) { + size_t numOfTables = tableListGetSize(pInfo->base.pTableListInfo); + int32_t i = pInfo->tableStartIndex + 1; + for (; i < numOfTables; ++i) { + STableKeyInfo* tableKeyInfo = tableListGetInfo(pInfo->base.pTableListInfo, i); + if (tableKeyInfo->groupId != pInfo->groupId) { + break; + } + } + pInfo->tableEndIndex = i - 1; +} + +static int32_t initSubTablesMergeSort(STmsSubTablesMergeInfo* pSubTblsInfo) { + for (int32_t i = 0; i < pSubTblsInfo->numSubTables; ++i) { + STmsSubTableInput * pInput = pSubTblsInfo->aInputs + i; + if (pInput->rowIdx == -1) { + continue; + } + if (pInput->type == SUB_TABLE_EXT_PAGES) { + pInput->pageIdx = 0; + fetchNextSubTableBlockFromPage(pSubTblsInfo, pInput); + } else { + pInput->rowIdx = 0; + pInput->pageIdx = -1; + } + SColumnInfoData* col = taosArrayGet(pInput->pBlock->pDataBlock, pSubTblsInfo->pOrderInfo->slotId); + pInput->aTs = (int64_t*)col->pData; + } + tMergeTreeCreate(&pSubTblsInfo->pTree, pSubTblsInfo->numSubTables, pSubTblsInfo, subTblRowCompareFn); + return TSDB_CODE_SUCCESS; +} + +static int32_t startSubTablesTms(SOperatorInfo* pOperator) { + STableMergeScanInfo* pInfo = pOperator->info; + STmsSubTablesMergeInfo* pSubTblsInfo = pInfo->pSubTablesMergeInfo; + + SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; + SReadHandle* pHandle = &pInfo->base.readHandle; + SStorageAPI* pAPI = &pTaskInfo->storageAPI; + + setGroupStartEndIndex(pInfo); + + pSubTblsInfo->pOrderInfo = taosArrayGet(pInfo->pSortInfo, 0); + pSubTblsInfo->numSubTables = pInfo->tableEndIndex - pInfo->tableStartIndex + 1; + pSubTblsInfo->aInputs = taosMemoryCalloc(pSubTblsInfo->numSubTables, sizeof(STmsSubTableInput)); + int32_t bufPageSize = pInfo->bufPageSize; + int32_t inMemSize = (pSubTblsInfo->numSubTables - pSubTblsInfo->numTableBlocksInMem) * bufPageSize; + createDiskbasedBuf(&pSubTblsInfo->pBlocksBuf, pInfo->bufPageSize, inMemSize, "blocksExternalBuf", tsTempDir); + for (int32_t i = pInfo->tableStartIndex; i <= pInfo->tableEndIndex; ++i) { + int32_t idx = i - pInfo->tableStartIndex; + STmsSubTableInput* pInput = pSubTblsInfo->aInputs + idx; + pInput->type = SUB_TABLE_MEM_BLOCK; + pInput->pBlock = createOneDataBlock(pInfo->pResBlock, false); + STableKeyInfo* keyInfo = tableListGetInfo(pInfo->base.pTableListInfo, i); + pAPI->tsdReader.tsdReaderOpen(pHandle->vnode, &pInfo->base.cond, keyInfo, 1, pInfo->pReaderBlock, + (void**)&pInput->pReader, GET_TASKID(pTaskInfo), NULL); + bool hasNext = true; + fetchNextSubTableBlockFromReader(pOperator, pInput, &hasNext); + + if (idx + 1 > pSubTblsInfo->numTableBlocksInMem && hasNext) { + saveSubTableBlock(pInfo, pSubTblsInfo, pInput); + pInput->type = SUB_TABLE_EXT_PAGES; + } + } + + initSubTablesMergeSort(pSubTblsInfo); + + return TSDB_CODE_SUCCESS; +} + +static int32_t stopSubTablesTms(SOperatorInfo* pOperator) { + STableMergeScanInfo* pInfo = pOperator->info; + STmsSubTablesMergeInfo* pSubTblsInfo = pInfo->pSubTablesMergeInfo; + + SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; + SReadHandle* pHandle = &pInfo->base.readHandle; + SStorageAPI* pAPI = &pTaskInfo->storageAPI; + tMergeTreeDestroy(pSubTblsInfo->pTree); + for (int32_t i = 0; i <= pSubTblsInfo->numSubTables; ++i) { + STmsSubTableInput* pInput = pSubTblsInfo->aInputs + i; + + blockDataDestroy(pInput->pBlock); + taosArrayDestroy(pInput->aBlockPages); + pAPI->tsdReader.tsdReaderClose(pInput->pReader); + } + destroyDiskbasedBuf(pSubTblsInfo->pBlocksBuf); + taosMemoryFree(pSubTblsInfo->aInputs); + return TSDB_CODE_SUCCESS; +} + +static int32_t adjustSubTableForNextRow(SOperatorInfo* pOperatorInfo, STmsSubTablesMergeInfo* pSubTblsInfo, int32_t *pNumCompleted) { + STableMergeScanInfo* pInfo = pOperatorInfo->info; + + STmsSubTableInput* pInput = pSubTblsInfo->aInputs + tMergeTreeGetChosenIndex(pSubTblsInfo->pTree); + if (pInput->rowIdx < pInput->pBlock->info.rows - 1) { + ++pInput->rowIdx; + return TSDB_CODE_SUCCESS; + } else if (pInput->rowIdx == pInput->pBlock->info.rows -1 ) { + if (pInput->type == SUB_TABLE_MEM_BLOCK) { + bool hasNext = true; + fetchNextSubTableBlockFromReader(pOperatorInfo, pInput, &hasNext); + } else if (pInput->type == SUB_TABLE_EXT_PAGES) { + ++pInput->pageIdx; + if (pInput->pageIdx >= taosArrayGetSize(pInput->aBlockPages)) { + bool hasNext = true; + fetchNextSubTableBlockFromReader(pOperatorInfo, pInput, &hasNext); + if (hasNext) { + saveSubTableBlock(pInfo, pSubTblsInfo, pInput); + pInput->pageIdx = 0; + fetchNextSubTableBlockFromPage(pSubTblsInfo, pInput); + } + } else { + fetchNextSubTableBlockFromPage(pSubTblsInfo, pInput); + } + } + SColumnInfoData* col = taosArrayGet(pInput->pBlock->pDataBlock, pSubTblsInfo->pOrderInfo->slotId); + pInput->aTs = (int64_t*)col->pData; + } + tMergeTreeAdjust(pSubTblsInfo->pTree, tMergeTreeGetAdjustIndex(pSubTblsInfo->pTree)); + return TSDB_CODE_SUCCESS; +} + +static int32_t appendChosenRowToDataBlock(STmsSubTablesMergeInfo* pSubTblsInfo, SSDataBlock* pBlock) { + STmsSubTableInput* pInput = pSubTblsInfo->aInputs + tMergeTreeGetChosenIndex(pSubTblsInfo->pTree); + for (int32_t i = 0; i < taosArrayGetSize(pBlock->pDataBlock); ++i) { + SColumnInfoData* pColInfo = taosArrayGet(pBlock->pDataBlock, i); + + SColumnInfoData* pSrcColInfo = taosArrayGet(pInput->pBlock->pDataBlock, i); + bool isNull = colDataIsNull(pSrcColInfo, pInput->pBlock->info.rows, pInput->rowIdx, NULL); + + if (isNull) { + colDataSetVal(pColInfo, pBlock->info.rows, NULL, true); + } else { + if (!pSrcColInfo->pData) continue; + char* pData = colDataGetData(pSrcColInfo, pInput->rowIdx); + colDataSetVal(pColInfo, pBlock->info.rows, pData, false); + } + } + + pBlock->info.rows += 1; + return TSDB_CODE_SUCCESS; +} + static int32_t tableMergeScanDoSkipTable(STableMergeScanInfo* pInfo, SSDataBlock* pBlock) { int64_t nRows = 0; void* pNum = tSimpleHashGet(pInfo->mTableNumRows, &pBlock->info.id.uid, sizeof(pBlock->info.id.uid)); From 174673470d4c07440c13ed908b89c328bb03b542 Mon Sep 17 00:00:00 2001 From: slzhou Date: Tue, 23 Jan 2024 17:12:53 +0800 Subject: [PATCH 02/41] fix: linux compilation --- source/libs/executor/src/scanoperator.c | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/source/libs/executor/src/scanoperator.c b/source/libs/executor/src/scanoperator.c index 87749d3c81..0f87175fd2 100644 --- a/source/libs/executor/src/scanoperator.c +++ b/source/libs/executor/src/scanoperator.c @@ -3343,7 +3343,7 @@ static int32_t subTblRowCompareFn(const void* pLeft, const void* pRight, void* p } static int32_t fetchNextSubTableBlockFromReader(SOperatorInfo* pOperator, STmsSubTableInput* pInput, bool* pHasNext) { - const STableMergeScanInfo* pInfo = pOperator->info; + STableMergeScanInfo* pInfo = pOperator->info; STmsSubTablesMergeInfo* pSubTblsInfo = pInfo->pSubTablesMergeInfo; SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; const SStorageAPI* pAPI= &pTaskInfo->storageAPI; @@ -3480,7 +3480,7 @@ static int32_t stopSubTablesTms(SOperatorInfo* pOperator) { SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; SReadHandle* pHandle = &pInfo->base.readHandle; SStorageAPI* pAPI = &pTaskInfo->storageAPI; - tMergeTreeDestroy(pSubTblsInfo->pTree); + tMergeTreeDestroy(&pSubTblsInfo->pTree); for (int32_t i = 0; i <= pSubTblsInfo->numSubTables; ++i) { STmsSubTableInput* pInput = pSubTblsInfo->aInputs + i; From 3f2b469b5ec0a086138c0c1935a8f9845892173d Mon Sep 17 00:00:00 2001 From: slzhou Date: Wed, 24 Jan 2024 09:12:27 +0800 Subject: [PATCH 03/41] feat: refactor code --- source/libs/executor/src/scanoperator.c | 167 +++++++++++++++++------- 1 file changed, 122 insertions(+), 45 deletions(-) diff --git a/source/libs/executor/src/scanoperator.c b/source/libs/executor/src/scanoperator.c index 0f87175fd2..0f1d48c840 100644 --- a/source/libs/executor/src/scanoperator.c +++ b/source/libs/executor/src/scanoperator.c @@ -3342,26 +3342,47 @@ static int32_t subTblRowCompareFn(const void* pLeft, const void* pRight, void* p return ret; } -static int32_t fetchNextSubTableBlockFromReader(SOperatorInfo* pOperator, STmsSubTableInput* pInput, bool* pHasNext) { +static int32_t fetchNextSubTableBlockFromReader(SOperatorInfo* pOperator, STmsSubTableInput* pInput, bool* pSubTableHasBlock) { STableMergeScanInfo* pInfo = pOperator->info; STmsSubTablesMergeInfo* pSubTblsInfo = pInfo->pSubTablesMergeInfo; SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; const SStorageAPI* pAPI= &pTaskInfo->storageAPI; blockDataCleanup(pInput->pBlock); - bool hasNext = false; - int32_t code = pAPI->tsdReader.tsdNextDataBlock(pInput->pReader, &hasNext); - *pHasNext = hasNext; - if (!hasNext) { - pInput->rowIdx = -1; - ++pSubTblsInfo->numTablesCompleted; - return code; + while (true) { + bool hasNext = false; + int32_t code = pAPI->tsdReader.tsdNextDataBlock(pInput->pReader, &hasNext); + if (code != 0) { + pAPI->tsdReader.tsdReaderReleaseDataBlock(pInput->pReader); + T_LONG_JMP(pTaskInfo->env, code); + } + if (!hasNext || isTaskKilled(pTaskInfo)) { + if (isTaskKilled(pTaskInfo)) { + pAPI->tsdReader.tsdReaderReleaseDataBlock(pInput->pReader); + } + break; + } + + uint32_t status = 0; + code = loadDataBlock(pOperator, &pInfo->base, pInput->pBlock, &status); + if (code != 0) { + T_LONG_JMP(pTaskInfo->env, code); + } + if (status == FUNC_DATA_REQUIRED_ALL_FILTEROUT) { + break; + } + if (status == FUNC_DATA_REQUIRED_FILTEROUT || pInput->pBlock->info.rows == 0) { + continue; + } + + *pSubTableHasBlock = true; + pInput->pBlock->info.id.groupId = tableListGetTableGroupId(pInfo->base.pTableListInfo, pInput->pBlock->info.id.uid); + pOperator->resultInfo.totalRows += pInput->pBlock->info.rows; + return TSDB_CODE_SUCCESS; } - uint32_t status = 0; - code = loadDataBlock(pOperator, &pInfo->base, pInput->pBlock, &status); - - return code; + *pSubTableHasBlock = false; + return TSDB_CODE_SUCCESS; } static int32_t saveSubTableBlock(const STableMergeScanInfo* pInfo, STmsSubTablesMergeInfo* pSubTblsInfo, @@ -3399,7 +3420,6 @@ static int32_t fetchNextSubTableBlockFromPage(STmsSubTablesMergeInfo* pSubTblsIn void* page = getBufPage(pSubTblsInfo->pBlocksBuf, pageId); blockDataFromBuf(pInput->pBlock, page); releaseBufPage(pSubTblsInfo->pBlocksBuf, page); - pInput->rowIdx = 0; return TSDB_CODE_SUCCESS; } @@ -3424,6 +3444,7 @@ static int32_t initSubTablesMergeSort(STmsSubTablesMergeInfo* pSubTblsInfo) { if (pInput->type == SUB_TABLE_EXT_PAGES) { pInput->pageIdx = 0; fetchNextSubTableBlockFromPage(pSubTblsInfo, pInput); + pInput->rowIdx = 0; } else { pInput->rowIdx = 0; pInput->pageIdx = -1; @@ -3435,40 +3456,62 @@ static int32_t initSubTablesMergeSort(STmsSubTablesMergeInfo* pSubTblsInfo) { return TSDB_CODE_SUCCESS; } -static int32_t startSubTablesTms(SOperatorInfo* pOperator) { - STableMergeScanInfo* pInfo = pOperator->info; - STmsSubTablesMergeInfo* pSubTblsInfo = pInfo->pSubTablesMergeInfo; - - SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; - SReadHandle* pHandle = &pInfo->base.readHandle; - SStorageAPI* pAPI = &pTaskInfo->storageAPI; +static int32_t initSubTablesMergeInfo(STableMergeScanInfo* pInfo) { + STmsSubTablesMergeInfo* pSubTblsInfo = taosMemoryCalloc(1, sizeof(STmsSubTablesMergeInfo)); + pInfo->pSubTablesMergeInfo = pSubTblsInfo; setGroupStartEndIndex(pInfo); pSubTblsInfo->pOrderInfo = taosArrayGet(pInfo->pSortInfo, 0); pSubTblsInfo->numSubTables = pInfo->tableEndIndex - pInfo->tableStartIndex + 1; pSubTblsInfo->aInputs = taosMemoryCalloc(pSubTblsInfo->numSubTables, sizeof(STmsSubTableInput)); + int32_t bufPageSize = pInfo->bufPageSize; int32_t inMemSize = (pSubTblsInfo->numSubTables - pSubTblsInfo->numTableBlocksInMem) * bufPageSize; createDiskbasedBuf(&pSubTblsInfo->pBlocksBuf, pInfo->bufPageSize, inMemSize, "blocksExternalBuf", tsTempDir); - for (int32_t i = pInfo->tableStartIndex; i <= pInfo->tableEndIndex; ++i) { - int32_t idx = i - pInfo->tableStartIndex; - STmsSubTableInput* pInput = pSubTblsInfo->aInputs + idx; + + pSubTblsInfo->numTableBlocksInMem = pSubTblsInfo->numSubTables; + return TSDB_CODE_SUCCESS; +} + +static int32_t initSubTableInputs(SOperatorInfo* pOperator, STableMergeScanInfo* pInfo) { + SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; + SReadHandle* pHandle = &pInfo->base.readHandle; + SStorageAPI* pAPI = &pTaskInfo->storageAPI; + + STmsSubTablesMergeInfo* pSubTblsInfo = pInfo->pSubTablesMergeInfo; + + for (int32_t i = 0; i < pSubTblsInfo->numSubTables; ++i) { + STmsSubTableInput* pInput = pSubTblsInfo->aInputs + i; pInput->type = SUB_TABLE_MEM_BLOCK; pInput->pBlock = createOneDataBlock(pInfo->pResBlock, false); - STableKeyInfo* keyInfo = tableListGetInfo(pInfo->base.pTableListInfo, i); + STableKeyInfo* keyInfo = tableListGetInfo(pInfo->base.pTableListInfo, i + pInfo->tableStartIndex); pAPI->tsdReader.tsdReaderOpen(pHandle->vnode, &pInfo->base.cond, keyInfo, 1, pInfo->pReaderBlock, (void**)&pInput->pReader, GET_TASKID(pTaskInfo), NULL); bool hasNext = true; fetchNextSubTableBlockFromReader(pOperator, pInput, &hasNext); - - if (idx + 1 > pSubTblsInfo->numTableBlocksInMem && hasNext) { + if (!hasNext) { + pInput->rowIdx = -1; + ++pSubTblsInfo->numTablesCompleted; + continue; + } else { + pInput->rowIdx = 0; + pInput->pageIdx = -1; + } + if (i + 1 > pSubTblsInfo->numTableBlocksInMem && hasNext) { saveSubTableBlock(pInfo, pSubTblsInfo, pInput); pInput->type = SUB_TABLE_EXT_PAGES; } - } + } + return TSDB_CODE_SUCCESS; +} - initSubTablesMergeSort(pSubTblsInfo); +static int32_t startSubTablesTms(SOperatorInfo* pOperator) { + STableMergeScanInfo* pInfo = pOperator->info; + + initSubTablesMergeInfo(pInfo); + initSubTableInputs(pOperator, pInfo); + initSubTablesMergeSort(pInfo->pSubTablesMergeInfo); return TSDB_CODE_SUCCESS; } @@ -3490,38 +3533,72 @@ static int32_t stopSubTablesTms(SOperatorInfo* pOperator) { } destroyDiskbasedBuf(pSubTblsInfo->pBlocksBuf); taosMemoryFree(pSubTblsInfo->aInputs); + + taosMemoryFree(pSubTblsInfo); + pInfo->pSubTablesMergeInfo = NULL; return TSDB_CODE_SUCCESS; } +static int32_t adjustSubTableFromMemBlock(SOperatorInfo* pOperatorInfo, STmsSubTablesMergeInfo* pSubTblsInfo) { + STableMergeScanInfo* pInfo = pOperatorInfo->info; + STmsSubTableInput* pInput = pSubTblsInfo->aInputs + tMergeTreeGetChosenIndex(pSubTblsInfo->pTree); + + bool hasNext = true; + fetchNextSubTableBlockFromReader(pOperatorInfo, pInput, &hasNext); + if (!hasNext) { + pInput->rowIdx = -1; + ++pSubTblsInfo->numTablesCompleted; + } else { + pInput->rowIdx = 0; + } + + return TSDB_CODE_SUCCESS; +} + +static int32_t adjustSubTableFromExtPages(SOperatorInfo* pOperatorInfo, STmsSubTablesMergeInfo* pSubTblsInfo) { + STableMergeScanInfo* pInfo = pOperatorInfo->info; + STmsSubTableInput* pInput = pSubTblsInfo->aInputs + tMergeTreeGetChosenIndex(pSubTblsInfo->pTree); + if (pInput->pageIdx >= taosArrayGetSize(pInput->aBlockPages)) { + bool hasNext = true; + fetchNextSubTableBlockFromReader(pOperatorInfo, pInput, &hasNext); + if (!hasNext) { + pInput->rowIdx = -1; + ++pSubTblsInfo->numTablesCompleted; + } else { + saveSubTableBlock(pInfo, pSubTblsInfo, pInput); + pInput->pageIdx = 0; + fetchNextSubTableBlockFromPage(pSubTblsInfo, pInput); + pInput->rowIdx = 0; + + } + } else { + fetchNextSubTableBlockFromPage(pSubTblsInfo, pInput); + pInput->rowIdx = 0; + } + return TSDB_CODE_SUCCESS; +} + static int32_t adjustSubTableForNextRow(SOperatorInfo* pOperatorInfo, STmsSubTablesMergeInfo* pSubTblsInfo, int32_t *pNumCompleted) { STableMergeScanInfo* pInfo = pOperatorInfo->info; - STmsSubTableInput* pInput = pSubTblsInfo->aInputs + tMergeTreeGetChosenIndex(pSubTblsInfo->pTree); + if (pInput->rowIdx < pInput->pBlock->info.rows - 1) { ++pInput->rowIdx; - return TSDB_CODE_SUCCESS; } else if (pInput->rowIdx == pInput->pBlock->info.rows -1 ) { if (pInput->type == SUB_TABLE_MEM_BLOCK) { - bool hasNext = true; - fetchNextSubTableBlockFromReader(pOperatorInfo, pInput, &hasNext); + adjustSubTableFromMemBlock(pOperatorInfo, pSubTblsInfo); } else if (pInput->type == SUB_TABLE_EXT_PAGES) { ++pInput->pageIdx; - if (pInput->pageIdx >= taosArrayGetSize(pInput->aBlockPages)) { - bool hasNext = true; - fetchNextSubTableBlockFromReader(pOperatorInfo, pInput, &hasNext); - if (hasNext) { - saveSubTableBlock(pInfo, pSubTblsInfo, pInput); - pInput->pageIdx = 0; - fetchNextSubTableBlockFromPage(pSubTblsInfo, pInput); - } - } else { - fetchNextSubTableBlockFromPage(pSubTblsInfo, pInput); - } + adjustSubTableFromExtPages(pOperatorInfo, pSubTblsInfo); + } + if (pInput->rowIdx != -1) { + SColumnInfoData* col = taosArrayGet(pInput->pBlock->pDataBlock, pSubTblsInfo->pOrderInfo->slotId); + pInput->aTs = (int64_t*)col->pData; } - SColumnInfoData* col = taosArrayGet(pInput->pBlock->pDataBlock, pSubTblsInfo->pOrderInfo->slotId); - pInput->aTs = (int64_t*)col->pData; } + tMergeTreeAdjust(pSubTblsInfo->pTree, tMergeTreeGetAdjustIndex(pSubTblsInfo->pTree)); + return TSDB_CODE_SUCCESS; } From 622dd10b195d8b9c123a9019e58b3b7801a0439e Mon Sep 17 00:00:00 2001 From: slzhou Date: Wed, 24 Jan 2024 10:21:17 +0800 Subject: [PATCH 04/41] feat: add operator next function --- source/libs/executor/inc/executorInt.h | 4 +- source/libs/executor/src/scanoperator.c | 184 ++++++++++++++++++------ 2 files changed, 143 insertions(+), 45 deletions(-) diff --git a/source/libs/executor/inc/executorInt.h b/source/libs/executor/inc/executorInt.h index b20e1a98c0..b86ff2f7fa 100644 --- a/source/libs/executor/inc/executorInt.h +++ b/source/libs/executor/inc/executorInt.h @@ -42,7 +42,6 @@ extern "C" { // #include "tstream.h" // #include "tstreamUpdate.h" #include "tlrucache.h" -#include "tdatablock.h" typedef int32_t (*__block_search_fn_t)(char* data, int32_t num, int64_t key, int32_t order); @@ -301,13 +300,14 @@ typedef struct STmsSubTableInput { int64_t* aTs; } STmsSubTableInput; +typedef struct SBlockOrderInfo SBlockOrderInfo; typedef struct STmsSubTablesMergeInfo { SBlockOrderInfo* pOrderInfo; int32_t numSubTables; STmsSubTableInput* aInputs; SMultiwayMergeTreeInfo* pTree; - int32_t numTablesCompleted; + int32_t numSubTablesCompleted; int32_t numTableBlocksInMem; SDiskbasedBuf* pBlocksBuf; diff --git a/source/libs/executor/src/scanoperator.c b/source/libs/executor/src/scanoperator.c index 0f1d48c840..d73b43ee3e 100644 --- a/source/libs/executor/src/scanoperator.c +++ b/source/libs/executor/src/scanoperator.c @@ -3375,7 +3375,7 @@ static int32_t fetchNextSubTableBlockFromReader(SOperatorInfo* pOperator, STmsSu if (status == FUNC_DATA_REQUIRED_FILTEROUT || pInput->pBlock->info.rows == 0) { continue; } - + *pSubTableHasBlock = true; pInput->pBlock->info.id.groupId = tableListGetTableGroupId(pInfo->base.pTableListInfo, pInput->pBlock->info.id.uid); pOperator->resultInfo.totalRows += pInput->pBlock->info.rows; @@ -3415,8 +3415,8 @@ static int32_t saveSubTableBlock(const STableMergeScanInfo* pInfo, STmsSubTables return TSDB_CODE_SUCCESS; } -static int32_t fetchNextSubTableBlockFromPage(STmsSubTablesMergeInfo* pSubTblsInfo, STmsSubTableInput* pInput) { - int32_t pageId = *(int32_t*)taosArrayGet(pInput->aBlockPages, pInput->pageIdx); +static int32_t fetchNextSubTableBlockFromPage(STmsSubTablesMergeInfo* pSubTblsInfo, STmsSubTableInput* pInput, int32_t pageIndex) { + int32_t pageId = *(int32_t*)taosArrayGet(pInput->aBlockPages, pageIndex); void* page = getBufPage(pSubTblsInfo->pBlocksBuf, pageId); blockDataFromBuf(pInput->pBlock, page); releaseBufPage(pSubTblsInfo->pBlocksBuf, page); @@ -3424,6 +3424,8 @@ static int32_t fetchNextSubTableBlockFromPage(STmsSubTablesMergeInfo* pSubTblsIn } static void setGroupStartEndIndex(STableMergeScanInfo* pInfo) { + pInfo->bGroupProcessed = false; + size_t numOfTables = tableListGetSize(pInfo->base.pTableListInfo); int32_t i = pInfo->tableStartIndex + 1; for (; i < numOfTables; ++i) { @@ -3443,7 +3445,7 @@ static int32_t initSubTablesMergeSort(STmsSubTablesMergeInfo* pSubTblsInfo) { } if (pInput->type == SUB_TABLE_EXT_PAGES) { pInput->pageIdx = 0; - fetchNextSubTableBlockFromPage(pSubTblsInfo, pInput); + fetchNextSubTableBlockFromPage(pSubTblsInfo, pInput, pInput->pageIdx); pInput->rowIdx = 0; } else { pInput->rowIdx = 0; @@ -3492,7 +3494,7 @@ static int32_t initSubTableInputs(SOperatorInfo* pOperator, STableMergeScanInfo* fetchNextSubTableBlockFromReader(pOperator, pInput, &hasNext); if (!hasNext) { pInput->rowIdx = -1; - ++pSubTblsInfo->numTablesCompleted; + ++pSubTblsInfo->numSubTablesCompleted; continue; } else { pInput->rowIdx = 0; @@ -3506,39 +3508,6 @@ static int32_t initSubTableInputs(SOperatorInfo* pOperator, STableMergeScanInfo* return TSDB_CODE_SUCCESS; } -static int32_t startSubTablesTms(SOperatorInfo* pOperator) { - STableMergeScanInfo* pInfo = pOperator->info; - - initSubTablesMergeInfo(pInfo); - initSubTableInputs(pOperator, pInfo); - initSubTablesMergeSort(pInfo->pSubTablesMergeInfo); - - return TSDB_CODE_SUCCESS; -} - -static int32_t stopSubTablesTms(SOperatorInfo* pOperator) { - STableMergeScanInfo* pInfo = pOperator->info; - STmsSubTablesMergeInfo* pSubTblsInfo = pInfo->pSubTablesMergeInfo; - - SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; - SReadHandle* pHandle = &pInfo->base.readHandle; - SStorageAPI* pAPI = &pTaskInfo->storageAPI; - tMergeTreeDestroy(&pSubTblsInfo->pTree); - for (int32_t i = 0; i <= pSubTblsInfo->numSubTables; ++i) { - STmsSubTableInput* pInput = pSubTblsInfo->aInputs + i; - - blockDataDestroy(pInput->pBlock); - taosArrayDestroy(pInput->aBlockPages); - pAPI->tsdReader.tsdReaderClose(pInput->pReader); - } - destroyDiskbasedBuf(pSubTblsInfo->pBlocksBuf); - taosMemoryFree(pSubTblsInfo->aInputs); - - taosMemoryFree(pSubTblsInfo); - pInfo->pSubTablesMergeInfo = NULL; - return TSDB_CODE_SUCCESS; -} - static int32_t adjustSubTableFromMemBlock(SOperatorInfo* pOperatorInfo, STmsSubTablesMergeInfo* pSubTblsInfo) { STableMergeScanInfo* pInfo = pOperatorInfo->info; STmsSubTableInput* pInput = pSubTblsInfo->aInputs + tMergeTreeGetChosenIndex(pSubTblsInfo->pTree); @@ -3547,7 +3516,7 @@ static int32_t adjustSubTableFromMemBlock(SOperatorInfo* pOperatorInfo, STmsSubT fetchNextSubTableBlockFromReader(pOperatorInfo, pInput, &hasNext); if (!hasNext) { pInput->rowIdx = -1; - ++pSubTblsInfo->numTablesCompleted; + ++pSubTblsInfo->numSubTablesCompleted; } else { pInput->rowIdx = 0; } @@ -3563,22 +3532,22 @@ static int32_t adjustSubTableFromExtPages(SOperatorInfo* pOperatorInfo, STmsSubT fetchNextSubTableBlockFromReader(pOperatorInfo, pInput, &hasNext); if (!hasNext) { pInput->rowIdx = -1; - ++pSubTblsInfo->numTablesCompleted; + ++pSubTblsInfo->numSubTablesCompleted; } else { saveSubTableBlock(pInfo, pSubTblsInfo, pInput); pInput->pageIdx = 0; - fetchNextSubTableBlockFromPage(pSubTblsInfo, pInput); + fetchNextSubTableBlockFromPage(pSubTblsInfo, pInput, pInput->pageIdx); pInput->rowIdx = 0; } } else { - fetchNextSubTableBlockFromPage(pSubTblsInfo, pInput); + fetchNextSubTableBlockFromPage(pSubTblsInfo, pInput, pInput->pageIdx); pInput->rowIdx = 0; } return TSDB_CODE_SUCCESS; } -static int32_t adjustSubTableForNextRow(SOperatorInfo* pOperatorInfo, STmsSubTablesMergeInfo* pSubTblsInfo, int32_t *pNumCompleted) { +static int32_t adjustSubTableForNextRow(SOperatorInfo* pOperatorInfo, STmsSubTablesMergeInfo* pSubTblsInfo) { STableMergeScanInfo* pInfo = pOperatorInfo->info; STmsSubTableInput* pInput = pSubTblsInfo->aInputs + tMergeTreeGetChosenIndex(pSubTblsInfo->pTree); @@ -3623,6 +3592,135 @@ static int32_t appendChosenRowToDataBlock(STmsSubTablesMergeInfo* pSubTblsInfo, return TSDB_CODE_SUCCESS; } +static SSDataBlock* getSubTablesSortedBlock(SOperatorInfo* pOperator, SSDataBlock* pResBlock, int32_t capacity) { + STableMergeScanInfo* pInfo = pOperator->info; + SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; + STmsSubTablesMergeInfo* pSubTblsInfo = pInfo->pSubTablesMergeInfo; + + blockDataCleanup(pResBlock); + bool finished = false; + while (true) { + while (1) { + if (pSubTblsInfo->numSubTablesCompleted >= pSubTblsInfo->numSubTables) { + finished = true; + break; + } + + appendChosenRowToDataBlock(pSubTblsInfo, pResBlock); + adjustSubTableForNextRow(pOperator, pSubTblsInfo); + + if (pResBlock->info.rows >= capacity) { + break; + } + } + bool limitReached = applyLimitOffset(&pInfo->limitInfo, pResBlock, pTaskInfo); + if (finished || limitReached || pResBlock->info.rows > 0) { + break; + } + } + return (pResBlock->info.rows > 0) ? pResBlock : NULL; +} + +static int32_t startSubTablesTableMergeScan(SOperatorInfo* pOperator) { + STableMergeScanInfo* pInfo = pOperator->info; + + initSubTablesMergeInfo(pInfo); + + initSubTableInputs(pOperator, pInfo); + + initSubTablesMergeSort(pInfo->pSubTablesMergeInfo); + + return TSDB_CODE_SUCCESS; +} + +static int32_t stopSubTablesTableMergeScan(SOperatorInfo* pOperator) { + STableMergeScanInfo* pInfo = pOperator->info; + SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; + STmsSubTablesMergeInfo* pSubTblsInfo = pInfo->pSubTablesMergeInfo; + + SReadHandle* pHandle = &pInfo->base.readHandle; + SStorageAPI* pAPI = &pTaskInfo->storageAPI; + + tMergeTreeDestroy(&pSubTblsInfo->pTree); + + for (int32_t i = 0; i <= pSubTblsInfo->numSubTables; ++i) { + STmsSubTableInput* pInput = pSubTblsInfo->aInputs + i; + + blockDataDestroy(pInput->pBlock); + taosArrayDestroy(pInput->aBlockPages); + pAPI->tsdReader.tsdReaderClose(pInput->pReader); + } + + destroyDiskbasedBuf(pSubTblsInfo->pBlocksBuf); + taosMemoryFree(pSubTblsInfo->aInputs); + + taosMemoryFree(pSubTblsInfo); + pInfo->pSubTablesMergeInfo = NULL; + return TSDB_CODE_SUCCESS; +} + +SSDataBlock* doTableMergeScanSubTables(SOperatorInfo* pOperator) { + if (pOperator->status == OP_EXEC_DONE) { + return NULL; + } + + SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; + STableMergeScanInfo* pInfo = pOperator->info; + + int32_t code = pOperator->fpSet._openFn(pOperator); + if (code != TSDB_CODE_SUCCESS) { + T_LONG_JMP(pTaskInfo->env, code); + } + + size_t tableListSize = tableListGetSize(pInfo->base.pTableListInfo); + if (!pInfo->hasGroupId) { + pInfo->hasGroupId = true; + + if (tableListSize == 0) { + setOperatorCompleted(pOperator); + return NULL; + } + pInfo->tableStartIndex = 0; + pInfo->groupId = ((STableKeyInfo*)tableListGetInfo(pInfo->base.pTableListInfo, pInfo->tableStartIndex))->groupId; + startSubTablesTableMergeScan(pOperator); + } + + SSDataBlock* pBlock = NULL; + while (pInfo->tableStartIndex < tableListSize) { + if (isTaskKilled(pTaskInfo)) { + T_LONG_JMP(pTaskInfo->env, pTaskInfo->code); + } + + pBlock = getSubTablesSortedBlock(pOperator, pInfo->pResBlock, pOperator->resultInfo.capacity); + if (pBlock == NULL && !pInfo->bGroupProcessed && pInfo->needCountEmptyTable) { + STableKeyInfo* tbInfo = tableListGetInfo(pInfo->base.pTableListInfo, pInfo->tableStartIndex); + pBlock = getOneRowResultBlock(pTaskInfo, &pInfo->base, pInfo->pResBlock, tbInfo); + } + if (pBlock != NULL) { + pBlock->info.id.groupId = pInfo->groupId; + pOperator->resultInfo.totalRows += pBlock->info.rows; + pInfo->bGroupProcessed = true; + return pBlock; + } else { + // Data of this group are all dumped, let's try the next group + stopSubTablesTableMergeScan(pOperator); + if (pInfo->tableEndIndex >= tableListSize - 1) { + setOperatorCompleted(pOperator); + break; + } + + pInfo->tableStartIndex = pInfo->tableEndIndex + 1; + pInfo->groupId = tableListGetInfo(pInfo->base.pTableListInfo, pInfo->tableStartIndex)->groupId; + startSubTablesTableMergeScan(pOperator); + resetLimitInfoForNextGroup(&pInfo->limitInfo); + } + } + + return pBlock; +} + + +// table merge scan : one reader, save blocks to disk static int32_t tableMergeScanDoSkipTable(STableMergeScanInfo* pInfo, SSDataBlock* pBlock) { int64_t nRows = 0; void* pNum = tSimpleHashGet(pInfo->mTableNumRows, &pBlock->info.id.uid, sizeof(pBlock->info.id.uid)); From 033ebb5fa59362bdf3a4ddf982e67342687dec39 Mon Sep 17 00:00:00 2001 From: slzhou Date: Wed, 24 Jan 2024 11:11:45 +0800 Subject: [PATCH 05/41] feat: prepare for switch between one reader and multi readers --- source/libs/executor/inc/executorInt.h | 2 - source/libs/executor/src/scanoperator.c | 76 +++++++++++++------------ 2 files changed, 39 insertions(+), 39 deletions(-) diff --git a/source/libs/executor/inc/executorInt.h b/source/libs/executor/inc/executorInt.h index b86ff2f7fa..013fe41f4e 100644 --- a/source/libs/executor/inc/executorInt.h +++ b/source/libs/executor/inc/executorInt.h @@ -326,7 +326,6 @@ typedef struct STableMergeScanInfo { SSDataBlock* pSortInputBlock; SSDataBlock* pReaderBlock; int64_t startTs; // sort start time - SArray* sortSourceParams; SLimitInfo limitInfo; int64_t numOfRows; SScanInfo scanInfo; @@ -348,7 +347,6 @@ typedef struct STableMergeScanInfo { bool rtnNextDurationBlocks; int32_t nextDurationBlocksIdx; - STmsSubTablesMergeInfo* pSubTablesMergeInfo; } STableMergeScanInfo; diff --git a/source/libs/executor/src/scanoperator.c b/source/libs/executor/src/scanoperator.c index d73b43ee3e..068bdeb075 100644 --- a/source/libs/executor/src/scanoperator.c +++ b/source/libs/executor/src/scanoperator.c @@ -3437,7 +3437,7 @@ static void setGroupStartEndIndex(STableMergeScanInfo* pInfo) { pInfo->tableEndIndex = i - 1; } -static int32_t initSubTablesMergeSort(STmsSubTablesMergeInfo* pSubTblsInfo) { +static int32_t openSubTablesMergeSort(STmsSubTablesMergeInfo* pSubTblsInfo) { for (int32_t i = 0; i < pSubTblsInfo->numSubTables; ++i) { STmsSubTableInput * pInput = pSubTblsInfo->aInputs + i; if (pInput->rowIdx == -1) { @@ -3628,34 +3628,31 @@ static int32_t startSubTablesTableMergeScan(SOperatorInfo* pOperator) { initSubTableInputs(pOperator, pInfo); - initSubTablesMergeSort(pInfo->pSubTablesMergeInfo); + openSubTablesMergeSort(pInfo->pSubTablesMergeInfo); return TSDB_CODE_SUCCESS; } -static int32_t stopSubTablesTableMergeScan(SOperatorInfo* pOperator) { - STableMergeScanInfo* pInfo = pOperator->info; - SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; +static int32_t stopSubTablesTableMergeScan(STableMergeScanInfo* pInfo) { STmsSubTablesMergeInfo* pSubTblsInfo = pInfo->pSubTablesMergeInfo; + if (pSubTblsInfo != NULL) { + tMergeTreeDestroy(&pSubTblsInfo->pTree); - SReadHandle* pHandle = &pInfo->base.readHandle; - SStorageAPI* pAPI = &pTaskInfo->storageAPI; + for (int32_t i = 0; i <= pSubTblsInfo->numSubTables; ++i) { + STmsSubTableInput* pInput = pSubTblsInfo->aInputs + i; - tMergeTreeDestroy(&pSubTblsInfo->pTree); + blockDataDestroy(pInput->pBlock); + taosArrayDestroy(pInput->aBlockPages); + pInfo->base.readerAPI.tsdReaderClose(pInput->pReader); + pInput->pReader = NULL; + } - for (int32_t i = 0; i <= pSubTblsInfo->numSubTables; ++i) { - STmsSubTableInput* pInput = pSubTblsInfo->aInputs + i; + destroyDiskbasedBuf(pSubTblsInfo->pBlocksBuf); + taosMemoryFree(pSubTblsInfo->aInputs); - blockDataDestroy(pInput->pBlock); - taosArrayDestroy(pInput->aBlockPages); - pAPI->tsdReader.tsdReaderClose(pInput->pReader); + taosMemoryFree(pSubTblsInfo); + pInfo->pSubTablesMergeInfo = NULL; } - - destroyDiskbasedBuf(pSubTblsInfo->pBlocksBuf); - taosMemoryFree(pSubTblsInfo->aInputs); - - taosMemoryFree(pSubTblsInfo); - pInfo->pSubTablesMergeInfo = NULL; return TSDB_CODE_SUCCESS; } @@ -4136,10 +4133,8 @@ SSDataBlock* doTableMergeScan(SOperatorInfo* pOperator) { void destroyTableMergeScanOperatorInfo(void* param) { STableMergeScanInfo* pTableScanInfo = (STableMergeScanInfo*)param; - cleanupQueryTableDataCond(&pTableScanInfo->base.cond); - - int32_t numOfTable = taosArrayGetSize(pTableScanInfo->sortSourceParams); + // start one reader variable pTableScanInfo->base.readerAPI.tsdReaderClose(pTableScanInfo->base.dataReader); pTableScanInfo->base.dataReader = NULL; @@ -4150,20 +4145,24 @@ void destroyTableMergeScanOperatorInfo(void* param) { } } - taosArrayDestroy(pTableScanInfo->sortSourceParams); tsortDestroySortHandle(pTableScanInfo->pSortHandle); pTableScanInfo->pSortHandle = NULL; tSimpleHashCleanup(pTableScanInfo->mTableNumRows); pTableScanInfo->mTableNumRows = NULL; taosHashCleanup(pTableScanInfo->mSkipTables); pTableScanInfo->mSkipTables = NULL; + pTableScanInfo->pSortInputBlock = blockDataDestroy(pTableScanInfo->pSortInputBlock); + // end one reader variable + + cleanupQueryTableDataCond(&pTableScanInfo->base.cond); destroyTableScanBase(&pTableScanInfo->base, &pTableScanInfo->base.readerAPI); pTableScanInfo->pResBlock = blockDataDestroy(pTableScanInfo->pResBlock); - pTableScanInfo->pSortInputBlock = blockDataDestroy(pTableScanInfo->pSortInputBlock); pTableScanInfo->pReaderBlock = blockDataDestroy(pTableScanInfo->pReaderBlock); - taosArrayDestroy(pTableScanInfo->pSortInfo); + + stopSubTablesTableMergeScan(pTableScanInfo); + taosMemoryFreeClear(param); } @@ -4237,35 +4236,38 @@ SOperatorInfo* createTableMergeScanOperatorInfo(STableScanPhysiNode* pTableScanN goto _error; } - initResultSizeInfo(&pOperator->resultInfo, 1024); - pInfo->pResBlock = createDataBlockFromDescNode(pDescNode); - blockDataEnsureCapacity(pInfo->pResBlock, pOperator->resultInfo.capacity); - - pInfo->sortSourceParams = taosArrayInit(64, sizeof(STableMergeScanSortSourceParam)); - - pInfo->pSortInfo = generateSortByTsInfo(pInfo->base.matchInfo.pList, pInfo->base.cond.order); - pInfo->pSortInputBlock = createOneDataBlock(pInfo->pResBlock, false); initLimitInfo(pTableScanNode->scan.node.pLimit, pTableScanNode->scan.node.pSlimit, &pInfo->limitInfo); - pInfo->mTableNumRows = tSimpleHashInit(1024, - taosGetDefaultHashFunction(TSDB_DATA_TYPE_UBIGINT)); pInfo->mergeLimit = -1; bool hasLimit = pInfo->limitInfo.limit.limit != -1 || pInfo->limitInfo.limit.offset != -1; if (hasLimit) { pInfo->mergeLimit = pInfo->limitInfo.limit.limit + pInfo->limitInfo.limit.offset; pInfo->mSkipTables = NULL; } + + initResultSizeInfo(&pOperator->resultInfo, 1024); + pInfo->pResBlock = createDataBlockFromDescNode(pDescNode); + blockDataEnsureCapacity(pInfo->pResBlock, pOperator->resultInfo.capacity); + + pInfo->pSortInfo = generateSortByTsInfo(pInfo->base.matchInfo.pList, pInfo->base.cond.order); pInfo->pReaderBlock = createOneDataBlock(pInfo->pResBlock, false); + pInfo->needCountEmptyTable = tsCountAlwaysReturnValue && pTableScanNode->needCountEmptyTable; + int32_t rowSize = pInfo->pResBlock->info.rowSize; uint32_t nCols = taosArrayGetSize(pInfo->pResBlock->pDataBlock); pInfo->bufPageSize = getProperSortPageSize(rowSize, nCols); + + //start one reader variable + pInfo->pSortInputBlock = createOneDataBlock(pInfo->pResBlock, false); + pInfo->mTableNumRows = tSimpleHashInit(1024, + taosGetDefaultHashFunction(TSDB_DATA_TYPE_UBIGINT)); + if (!tsExperimental) { pInfo->filesetDelimited = false; } else { pInfo->filesetDelimited = pTableScanNode->filesetDelimited; } - pInfo->needCountEmptyTable = tsCountAlwaysReturnValue && pTableScanNode->needCountEmptyTable; - + // end one reader variable setOperatorInfo(pOperator, "TableMergeScanOperator", QUERY_NODE_PHYSICAL_PLAN_TABLE_MERGE_SCAN, false, OP_NOT_OPENED, pInfo, pTaskInfo); pOperator->exprSupp.numOfExprs = numOfCols; From 4484e99e6c735ecd839bcee3900ea43c1c1b9b65 Mon Sep 17 00:00:00 2001 From: slzhou Date: Wed, 24 Jan 2024 11:13:37 +0800 Subject: [PATCH 06/41] fix: compilation error --- source/libs/executor/src/scanoperator.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/libs/executor/src/scanoperator.c b/source/libs/executor/src/scanoperator.c index 068bdeb075..b52f942438 100644 --- a/source/libs/executor/src/scanoperator.c +++ b/source/libs/executor/src/scanoperator.c @@ -3700,7 +3700,7 @@ SSDataBlock* doTableMergeScanSubTables(SOperatorInfo* pOperator) { return pBlock; } else { // Data of this group are all dumped, let's try the next group - stopSubTablesTableMergeScan(pOperator); + stopSubTablesTableMergeScan(pInfo); if (pInfo->tableEndIndex >= tableListSize - 1) { setOperatorCompleted(pOperator); break; From 2f6a2923b750f719486a3164224ee5da1b527944 Mon Sep 17 00:00:00 2001 From: slzhou Date: Wed, 24 Jan 2024 15:23:03 +0800 Subject: [PATCH 07/41] fix: first running --- source/libs/executor/src/scanoperator.c | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/source/libs/executor/src/scanoperator.c b/source/libs/executor/src/scanoperator.c index b52f942438..34a70f28e4 100644 --- a/source/libs/executor/src/scanoperator.c +++ b/source/libs/executor/src/scanoperator.c @@ -3318,6 +3318,7 @@ _error: // TODO: get block from tsdReader function, with task killed, func_data all filter out, skip, finish // TODO: error processing, memory freeing // TODO: add log for error and perf +// TODO: tsdb reader open/close dynamically static int32_t subTblRowCompareFn(const void* pLeft, const void* pRight, void* param) { int32_t left = *(int32_t*)pLeft; @@ -3350,11 +3351,14 @@ static int32_t fetchNextSubTableBlockFromReader(SOperatorInfo* pOperator, STmsSu blockDataCleanup(pInput->pBlock); + pInfo->base.dataReader = pInput->pReader; + while (true) { bool hasNext = false; int32_t code = pAPI->tsdReader.tsdNextDataBlock(pInput->pReader, &hasNext); if (code != 0) { pAPI->tsdReader.tsdReaderReleaseDataBlock(pInput->pReader); + pInfo->base.dataReader = NULL; T_LONG_JMP(pTaskInfo->env, code); } if (!hasNext || isTaskKilled(pTaskInfo)) { @@ -3367,6 +3371,7 @@ static int32_t fetchNextSubTableBlockFromReader(SOperatorInfo* pOperator, STmsSu uint32_t status = 0; code = loadDataBlock(pOperator, &pInfo->base, pInput->pBlock, &status); if (code != 0) { + pInfo->base.dataReader = NULL; T_LONG_JMP(pTaskInfo->env, code); } if (status == FUNC_DATA_REQUIRED_ALL_FILTEROUT) { @@ -3379,8 +3384,10 @@ static int32_t fetchNextSubTableBlockFromReader(SOperatorInfo* pOperator, STmsSu *pSubTableHasBlock = true; pInput->pBlock->info.id.groupId = tableListGetTableGroupId(pInfo->base.pTableListInfo, pInput->pBlock->info.id.uid); pOperator->resultInfo.totalRows += pInput->pBlock->info.rows; + pInfo->base.dataReader = NULL; return TSDB_CODE_SUCCESS; } + pInfo->base.dataReader = NULL; *pSubTableHasBlock = false; return TSDB_CODE_SUCCESS; } @@ -3488,7 +3495,7 @@ static int32_t initSubTableInputs(SOperatorInfo* pOperator, STableMergeScanInfo* pInput->type = SUB_TABLE_MEM_BLOCK; pInput->pBlock = createOneDataBlock(pInfo->pResBlock, false); STableKeyInfo* keyInfo = tableListGetInfo(pInfo->base.pTableListInfo, i + pInfo->tableStartIndex); - pAPI->tsdReader.tsdReaderOpen(pHandle->vnode, &pInfo->base.cond, keyInfo, 1, pInfo->pReaderBlock, + pAPI->tsdReader.tsdReaderOpen(pHandle->vnode, &pInfo->base.cond, keyInfo, 1, pInput->pBlock, (void**)&pInput->pReader, GET_TASKID(pTaskInfo), NULL); bool hasNext = true; fetchNextSubTableBlockFromReader(pOperator, pInput, &hasNext); @@ -3638,7 +3645,7 @@ static int32_t stopSubTablesTableMergeScan(STableMergeScanInfo* pInfo) { if (pSubTblsInfo != NULL) { tMergeTreeDestroy(&pSubTblsInfo->pTree); - for (int32_t i = 0; i <= pSubTblsInfo->numSubTables; ++i) { + for (int32_t i = 0; i < pSubTblsInfo->numSubTables; ++i) { STmsSubTableInput* pInput = pSubTblsInfo->aInputs + i; blockDataDestroy(pInput->pBlock); @@ -4272,7 +4279,7 @@ SOperatorInfo* createTableMergeScanOperatorInfo(STableScanPhysiNode* pTableScanN pInfo, pTaskInfo); pOperator->exprSupp.numOfExprs = numOfCols; - pOperator->fpSet = createOperatorFpSet(optrDummyOpenFn, doTableMergeScan, NULL, destroyTableMergeScanOperatorInfo, + pOperator->fpSet = createOperatorFpSet(optrDummyOpenFn, doTableMergeScanSubTables, NULL, destroyTableMergeScanOperatorInfo, optrDefaultBufFn, getTableMergeScanExplainExecInfo, optrDefaultGetNextExtFn, NULL); pOperator->cost.openCost = 0; return pOperator; From 305d4a962aec7e4f43379a2a29dc4310bb3ed88f Mon Sep 17 00:00:00 2001 From: slzhou Date: Wed, 24 Jan 2024 17:16:18 +0800 Subject: [PATCH 08/41] fix: page block and reader block for each child table input --- source/libs/executor/inc/executorInt.h | 4 +- source/libs/executor/src/scanoperator.c | 49 +++++++++++++++---------- 2 files changed, 32 insertions(+), 21 deletions(-) diff --git a/source/libs/executor/inc/executorInt.h b/source/libs/executor/inc/executorInt.h index 013fe41f4e..f59f0b9c57 100644 --- a/source/libs/executor/inc/executorInt.h +++ b/source/libs/executor/inc/executorInt.h @@ -292,8 +292,10 @@ typedef enum ESubTableInputType { typedef struct STmsSubTableInput { STsdbReader* pReader; ESubTableInputType type; - SSDataBlock* pBlock; + SSDataBlock* pReaderBlock; + SArray* aBlockPages; + SSDataBlock* pPageBlock; int32_t pageIdx; int32_t rowIdx; diff --git a/source/libs/executor/src/scanoperator.c b/source/libs/executor/src/scanoperator.c index 34a70f28e4..54552e8289 100644 --- a/source/libs/executor/src/scanoperator.c +++ b/source/libs/executor/src/scanoperator.c @@ -3319,6 +3319,7 @@ _error: // TODO: error processing, memory freeing // TODO: add log for error and perf // TODO: tsdb reader open/close dynamically +// TODO: blockdata deep cleanup static int32_t subTblRowCompareFn(const void* pLeft, const void* pRight, void* param) { int32_t left = *(int32_t*)pLeft; @@ -3349,7 +3350,7 @@ static int32_t fetchNextSubTableBlockFromReader(SOperatorInfo* pOperator, STmsSu SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; const SStorageAPI* pAPI= &pTaskInfo->storageAPI; - blockDataCleanup(pInput->pBlock); + blockDataCleanup(pInput->pReaderBlock); pInfo->base.dataReader = pInput->pReader; @@ -3369,7 +3370,7 @@ static int32_t fetchNextSubTableBlockFromReader(SOperatorInfo* pOperator, STmsSu } uint32_t status = 0; - code = loadDataBlock(pOperator, &pInfo->base, pInput->pBlock, &status); + code = loadDataBlock(pOperator, &pInfo->base, pInput->pReaderBlock, &status); if (code != 0) { pInfo->base.dataReader = NULL; T_LONG_JMP(pTaskInfo->env, code); @@ -3377,13 +3378,13 @@ static int32_t fetchNextSubTableBlockFromReader(SOperatorInfo* pOperator, STmsSu if (status == FUNC_DATA_REQUIRED_ALL_FILTEROUT) { break; } - if (status == FUNC_DATA_REQUIRED_FILTEROUT || pInput->pBlock->info.rows == 0) { + if (status == FUNC_DATA_REQUIRED_FILTEROUT || pInput->pReaderBlock->info.rows == 0) { continue; } *pSubTableHasBlock = true; - pInput->pBlock->info.id.groupId = tableListGetTableGroupId(pInfo->base.pTableListInfo, pInput->pBlock->info.id.uid); - pOperator->resultInfo.totalRows += pInput->pBlock->info.rows; + pInput->pReaderBlock->info.id.groupId = tableListGetTableGroupId(pInfo->base.pTableListInfo, pInput->pReaderBlock->info.id.uid); + pOperator->resultInfo.totalRows += pInput->pReaderBlock->info.rows; pInfo->base.dataReader = NULL; return TSDB_CODE_SUCCESS; } @@ -3394,9 +3395,9 @@ static int32_t fetchNextSubTableBlockFromReader(SOperatorInfo* pOperator, STmsSu static int32_t saveSubTableBlock(const STableMergeScanInfo* pInfo, STmsSubTablesMergeInfo* pSubTblsInfo, STmsSubTableInput* pInput) { - pInput->aBlockPages = taosArrayInit(4, sizeof(int32_t)); + taosArrayClear(pInput->aBlockPages); int32_t start = 0; - SSDataBlock* pDataBlock = pInput->pBlock; + SSDataBlock* pDataBlock = pInput->pReaderBlock; while (start < pDataBlock->info.rows) { int32_t stop = 0; blockDataSplitRows(pDataBlock, pDataBlock->info.hasVarCol, start, &stop, pInfo->bufPageSize); @@ -3418,14 +3419,16 @@ static int32_t saveSubTableBlock(const STableMergeScanInfo* pInfo, STmsSubTables blockDataDestroy(p); start = stop + 1; } - blockDataCleanup(pInput->pBlock); + blockDataCleanup(pInput->pReaderBlock); return TSDB_CODE_SUCCESS; } static int32_t fetchNextSubTableBlockFromPage(STmsSubTablesMergeInfo* pSubTblsInfo, STmsSubTableInput* pInput, int32_t pageIndex) { int32_t pageId = *(int32_t*)taosArrayGet(pInput->aBlockPages, pageIndex); void* page = getBufPage(pSubTblsInfo->pBlocksBuf, pageId); - blockDataFromBuf(pInput->pBlock, page); + + blockDataFromBuf(pInput->pPageBlock, page); + releaseBufPage(pSubTblsInfo->pBlocksBuf, page); return TSDB_CODE_SUCCESS; } @@ -3458,7 +3461,8 @@ static int32_t openSubTablesMergeSort(STmsSubTablesMergeInfo* pSubTblsInfo) { pInput->rowIdx = 0; pInput->pageIdx = -1; } - SColumnInfoData* col = taosArrayGet(pInput->pBlock->pDataBlock, pSubTblsInfo->pOrderInfo->slotId); + SSDataBlock* pInputBlock = (pInput->type == SUB_TABLE_MEM_BLOCK) ? pInput->pReaderBlock : pInput->pPageBlock; + SColumnInfoData* col = taosArrayGet(pInputBlock->pDataBlock, pSubTblsInfo->pOrderInfo->slotId); pInput->aTs = (int64_t*)col->pData; } tMergeTreeCreate(&pSubTblsInfo->pTree, pSubTblsInfo->numSubTables, pSubTblsInfo, subTblRowCompareFn); @@ -3479,7 +3483,7 @@ static int32_t initSubTablesMergeInfo(STableMergeScanInfo* pInfo) { int32_t inMemSize = (pSubTblsInfo->numSubTables - pSubTblsInfo->numTableBlocksInMem) * bufPageSize; createDiskbasedBuf(&pSubTblsInfo->pBlocksBuf, pInfo->bufPageSize, inMemSize, "blocksExternalBuf", tsTempDir); - pSubTblsInfo->numTableBlocksInMem = pSubTblsInfo->numSubTables; + pSubTblsInfo->numTableBlocksInMem = 0; return TSDB_CODE_SUCCESS; } @@ -3493,9 +3497,10 @@ static int32_t initSubTableInputs(SOperatorInfo* pOperator, STableMergeScanInfo* for (int32_t i = 0; i < pSubTblsInfo->numSubTables; ++i) { STmsSubTableInput* pInput = pSubTblsInfo->aInputs + i; pInput->type = SUB_TABLE_MEM_BLOCK; - pInput->pBlock = createOneDataBlock(pInfo->pResBlock, false); + pInput->pReaderBlock = createOneDataBlock(pInfo->pResBlock, false); + pInput->pPageBlock = createOneDataBlock(pInfo->pResBlock, false); STableKeyInfo* keyInfo = tableListGetInfo(pInfo->base.pTableListInfo, i + pInfo->tableStartIndex); - pAPI->tsdReader.tsdReaderOpen(pHandle->vnode, &pInfo->base.cond, keyInfo, 1, pInput->pBlock, + pAPI->tsdReader.tsdReaderOpen(pHandle->vnode, &pInfo->base.cond, keyInfo, 1, pInput->pReaderBlock, (void**)&pInput->pReader, GET_TASKID(pTaskInfo), NULL); bool hasNext = true; fetchNextSubTableBlockFromReader(pOperator, pInput, &hasNext); @@ -3508,6 +3513,7 @@ static int32_t initSubTableInputs(SOperatorInfo* pOperator, STableMergeScanInfo* pInput->pageIdx = -1; } if (i + 1 > pSubTblsInfo->numTableBlocksInMem && hasNext) { + pInput->aBlockPages = taosArrayInit(32, sizeof(int32_t)); saveSubTableBlock(pInfo, pSubTblsInfo, pInput); pInput->type = SUB_TABLE_EXT_PAGES; } @@ -3518,7 +3524,6 @@ static int32_t initSubTableInputs(SOperatorInfo* pOperator, STableMergeScanInfo* static int32_t adjustSubTableFromMemBlock(SOperatorInfo* pOperatorInfo, STmsSubTablesMergeInfo* pSubTblsInfo) { STableMergeScanInfo* pInfo = pOperatorInfo->info; STmsSubTableInput* pInput = pSubTblsInfo->aInputs + tMergeTreeGetChosenIndex(pSubTblsInfo->pTree); - bool hasNext = true; fetchNextSubTableBlockFromReader(pOperatorInfo, pInput, &hasNext); if (!hasNext) { @@ -3558,9 +3563,10 @@ static int32_t adjustSubTableForNextRow(SOperatorInfo* pOperatorInfo, STmsSubTab STableMergeScanInfo* pInfo = pOperatorInfo->info; STmsSubTableInput* pInput = pSubTblsInfo->aInputs + tMergeTreeGetChosenIndex(pSubTblsInfo->pTree); - if (pInput->rowIdx < pInput->pBlock->info.rows - 1) { + SSDataBlock* pInputBlock = (pInput->type == SUB_TABLE_MEM_BLOCK) ? pInput->pReaderBlock : pInput->pPageBlock; + if (pInput->rowIdx < pInputBlock->info.rows - 1) { ++pInput->rowIdx; - } else if (pInput->rowIdx == pInput->pBlock->info.rows -1 ) { + } else if (pInput->rowIdx == pInputBlock->info.rows -1 ) { if (pInput->type == SUB_TABLE_MEM_BLOCK) { adjustSubTableFromMemBlock(pOperatorInfo, pSubTblsInfo); } else if (pInput->type == SUB_TABLE_EXT_PAGES) { @@ -3568,7 +3574,7 @@ static int32_t adjustSubTableForNextRow(SOperatorInfo* pOperatorInfo, STmsSubTab adjustSubTableFromExtPages(pOperatorInfo, pSubTblsInfo); } if (pInput->rowIdx != -1) { - SColumnInfoData* col = taosArrayGet(pInput->pBlock->pDataBlock, pSubTblsInfo->pOrderInfo->slotId); + SColumnInfoData* col = taosArrayGet(pInputBlock->pDataBlock, pSubTblsInfo->pOrderInfo->slotId); pInput->aTs = (int64_t*)col->pData; } } @@ -3580,11 +3586,13 @@ static int32_t adjustSubTableForNextRow(SOperatorInfo* pOperatorInfo, STmsSubTab static int32_t appendChosenRowToDataBlock(STmsSubTablesMergeInfo* pSubTblsInfo, SSDataBlock* pBlock) { STmsSubTableInput* pInput = pSubTblsInfo->aInputs + tMergeTreeGetChosenIndex(pSubTblsInfo->pTree); + SSDataBlock* pInputBlock = (pInput->type == SUB_TABLE_MEM_BLOCK) ? pInput->pReaderBlock : pInput->pPageBlock; + for (int32_t i = 0; i < taosArrayGetSize(pBlock->pDataBlock); ++i) { SColumnInfoData* pColInfo = taosArrayGet(pBlock->pDataBlock, i); - SColumnInfoData* pSrcColInfo = taosArrayGet(pInput->pBlock->pDataBlock, i); - bool isNull = colDataIsNull(pSrcColInfo, pInput->pBlock->info.rows, pInput->rowIdx, NULL); + SColumnInfoData* pSrcColInfo = taosArrayGet(pInputBlock->pDataBlock, i); + bool isNull = colDataIsNull(pSrcColInfo, pInputBlock->info.rows, pInput->rowIdx, NULL); if (isNull) { colDataSetVal(pColInfo, pBlock->info.rows, NULL, true); @@ -3648,7 +3656,8 @@ static int32_t stopSubTablesTableMergeScan(STableMergeScanInfo* pInfo) { for (int32_t i = 0; i < pSubTblsInfo->numSubTables; ++i) { STmsSubTableInput* pInput = pSubTblsInfo->aInputs + i; - blockDataDestroy(pInput->pBlock); + blockDataDestroy(pInput->pReaderBlock); + blockDataDestroy(pInput->pPageBlock); taosArrayDestroy(pInput->aBlockPages); pInfo->base.readerAPI.tsdReaderClose(pInput->pReader); pInput->pReader = NULL; From 1eedb9e7fa5aad8fb110665ffc79ef391976ab57 Mon Sep 17 00:00:00 2001 From: slzhou Date: Fri, 26 Jan 2024 10:55:14 +0800 Subject: [PATCH 09/41] fix: preallocated columns does not have data --- source/common/src/tdatablock.c | 3 ++- tests/script/tsim/parser/groupby.sim | 2 ++ 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/source/common/src/tdatablock.c b/source/common/src/tdatablock.c index 5382259899..200fd2bc46 100644 --- a/source/common/src/tdatablock.c +++ b/source/common/src/tdatablock.c @@ -573,6 +573,7 @@ int32_t blockDataSplitRows(SSDataBlock* pBlock, bool hasVarCol, int32_t startInd for (int32_t j = startIndex; j < numOfRows; ++j) { for (int32_t i = 0; i < numOfCols; ++i) { SColumnInfoData* pColInfoData = TARRAY_GET_ELEM(pBlock->pDataBlock, i); + if (pColInfoData->pData == NULL) continue; if (IS_VAR_DATA_TYPE(pColInfoData->info.type)) { if (pColInfoData->varmeta.offset[j] != -1) { char* p = colDataGetData(pColInfoData, j); @@ -631,7 +632,7 @@ SSDataBlock* blockDataExtractBlock(SSDataBlock* pBlock, int32_t startIndex, int3 for (int32_t i = 0; i < numOfCols; ++i) { SColumnInfoData* pColData = taosArrayGet(pBlock->pDataBlock, i); SColumnInfoData* pDstCol = taosArrayGet(pDst->pDataBlock, i); - + if (pColData->pData == NULL) continue; for (int32_t j = startIndex; j < (startIndex + rowCount); ++j) { bool isNull = false; if (pBlock->pBlockAgg == NULL) { diff --git a/tests/script/tsim/parser/groupby.sim b/tests/script/tsim/parser/groupby.sim index 637b7553fd..8b23160e04 100644 --- a/tests/script/tsim/parser/groupby.sim +++ b/tests/script/tsim/parser/groupby.sim @@ -656,7 +656,9 @@ if $data31 != 4 then endi sql_error select irate(c) from st where t1="1" and ts >= '2020-03-27 04:11:17.732' and ts < '2020-03-27 05:11:17.732' interval(1m) sliding(15s) group by tbname,c; +print select _wstart, irate(c), tbname, t1, t2 from st where t1=1 and ts >= '2020-03-27 04:11:17.732' and ts < '2020-03-27 05:11:17.732' partition by tbname,t1,t2 interval(1m) sliding(15s) order by tbname; sql select _wstart, irate(c), tbname, t1, t2 from st where t1=1 and ts >= '2020-03-27 04:11:17.732' and ts < '2020-03-27 05:11:17.732' partition by tbname,t1,t2 interval(1m) sliding(15s) order by tbname; +print $rows if $rows != 40 then return -1 endi From a06ba844ba377017978ac5c7d8b541e857e80d46 Mon Sep 17 00:00:00 2001 From: slzhou Date: Fri, 26 Jan 2024 13:11:05 +0800 Subject: [PATCH 10/41] fix: set dataload flag --- source/libs/executor/src/scanoperator.c | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/source/libs/executor/src/scanoperator.c b/source/libs/executor/src/scanoperator.c index 5fbb089755..a8fa42c6c6 100644 --- a/source/libs/executor/src/scanoperator.c +++ b/source/libs/executor/src/scanoperator.c @@ -3602,7 +3602,8 @@ static int32_t appendChosenRowToDataBlock(STmsSubTablesMergeInfo* pSubTblsInfo, colDataSetVal(pColInfo, pBlock->info.rows, pData, false); } } - + pBlock->info.dataLoad = 1; + pBlock->info.scanFlag = pInputBlock->info.scanFlag; pBlock->info.rows += 1; return TSDB_CODE_SUCCESS; } From c46769ef3789d02f42c588b72ac895fe6b446891 Mon Sep 17 00:00:00 2001 From: slzhou Date: Fri, 26 Jan 2024 14:10:02 +0800 Subject: [PATCH 11/41] fix: reader can be in memory or not --- source/libs/executor/inc/executorInt.h | 5 ++ source/libs/executor/src/scanoperator.c | 69 ++++++++++++++++++------- 2 files changed, 56 insertions(+), 18 deletions(-) diff --git a/source/libs/executor/inc/executorInt.h b/source/libs/executor/inc/executorInt.h index f59f0b9c57..6418654ed5 100644 --- a/source/libs/executor/inc/executorInt.h +++ b/source/libs/executor/inc/executorInt.h @@ -291,6 +291,9 @@ typedef enum ESubTableInputType { typedef struct STmsSubTableInput { STsdbReader* pReader; + SQueryTableDataCond tblCond; + STableKeyInfo* pKeyInfo; + bool bInMemReader; ESubTableInputType type; SSDataBlock* pReaderBlock; @@ -313,6 +316,8 @@ typedef struct STmsSubTablesMergeInfo { int32_t numTableBlocksInMem; SDiskbasedBuf* pBlocksBuf; + + int32_t numInMemReaders; } STmsSubTablesMergeInfo; typedef struct STableMergeScanInfo { diff --git a/source/libs/executor/src/scanoperator.c b/source/libs/executor/src/scanoperator.c index a8fa42c6c6..031ac9a5e7 100644 --- a/source/libs/executor/src/scanoperator.c +++ b/source/libs/executor/src/scanoperator.c @@ -3344,31 +3344,57 @@ static int32_t subTblRowCompareFn(const void* pLeft, const void* pRight, void* p return ret; } +int32_t dumpQueryTableCond(const SQueryTableDataCond* src, SQueryTableDataCond* dst) { + memcpy((void*)dst, (void*)src, sizeof(SQueryTableDataCond)); + dst->colList = taosMemoryCalloc(src->numOfCols, sizeof(SColumnInfo)); + for (int i = 0; i < src->numOfCols; i++) { + dst->colList[i] = src->colList[i]; + } + return 0; +} + static int32_t fetchNextSubTableBlockFromReader(SOperatorInfo* pOperator, STmsSubTableInput* pInput, bool* pSubTableHasBlock) { + int32_t code = 0; + STableMergeScanInfo* pInfo = pOperator->info; + SReadHandle* pHandle = &pInfo->base.readHandle; STmsSubTablesMergeInfo* pSubTblsInfo = pInfo->pSubTablesMergeInfo; SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; const SStorageAPI* pAPI= &pTaskInfo->storageAPI; blockDataCleanup(pInput->pReaderBlock); + if (!pInput->bInMemReader) { + code = pAPI->tsdReader.tsdReaderOpen(pHandle->vnode, &pInput->tblCond, pInput->pKeyInfo, 1, pInput->pReaderBlock, + (void**)&pInput->pReader, GET_TASKID(pTaskInfo), NULL); + if (code != 0) { + T_LONG_JMP(pTaskInfo->env, code); + } + } pInfo->base.dataReader = pInput->pReader; while (true) { bool hasNext = false; - int32_t code = pAPI->tsdReader.tsdNextDataBlock(pInput->pReader, &hasNext); + int32_t code = pAPI->tsdReader.tsdNextDataBlock(pInfo->base.dataReader, &hasNext); if (code != 0) { - pAPI->tsdReader.tsdReaderReleaseDataBlock(pInput->pReader); + pAPI->tsdReader.tsdReaderReleaseDataBlock(pInfo->base.dataReader); pInfo->base.dataReader = NULL; T_LONG_JMP(pTaskInfo->env, code); } if (!hasNext || isTaskKilled(pTaskInfo)) { if (isTaskKilled(pTaskInfo)) { - pAPI->tsdReader.tsdReaderReleaseDataBlock(pInput->pReader); + pAPI->tsdReader.tsdReaderReleaseDataBlock(pInfo->base.dataReader); } + *pSubTableHasBlock = false; break; } + if (pInput->tblCond.order == TSDB_ORDER_ASC) { + pInput->tblCond.twindows.skey = pInput->pReaderBlock->info.window.ekey + 1; + } else { + pInput->tblCond.twindows.ekey = pInput->pReaderBlock->info.window.skey - 1; + } + uint32_t status = 0; code = loadDataBlock(pOperator, &pInfo->base, pInput->pReaderBlock, &status); if (code != 0) { @@ -3376,6 +3402,7 @@ static int32_t fetchNextSubTableBlockFromReader(SOperatorInfo* pOperator, STmsSu T_LONG_JMP(pTaskInfo->env, code); } if (status == FUNC_DATA_REQUIRED_ALL_FILTEROUT) { + *pSubTableHasBlock = false; break; } if (status == FUNC_DATA_REQUIRED_FILTEROUT || pInput->pReaderBlock->info.rows == 0) { @@ -3383,13 +3410,19 @@ static int32_t fetchNextSubTableBlockFromReader(SOperatorInfo* pOperator, STmsSu } *pSubTableHasBlock = true; + break; + } + + if (*pSubTableHasBlock) { pInput->pReaderBlock->info.id.groupId = tableListGetTableGroupId(pInfo->base.pTableListInfo, pInput->pReaderBlock->info.id.uid); pOperator->resultInfo.totalRows += pInput->pReaderBlock->info.rows; - pInfo->base.dataReader = NULL; - return TSDB_CODE_SUCCESS; } + if (!pInput->bInMemReader || !*pSubTableHasBlock) { + pAPI->tsdReader.tsdReaderClose(pInput->pReader); + pInput->pReader = NULL; + } + pInfo->base.dataReader = NULL; - *pSubTableHasBlock = false; return TSDB_CODE_SUCCESS; } @@ -3483,7 +3516,8 @@ static int32_t initSubTablesMergeInfo(STableMergeScanInfo* pInfo) { int32_t inMemSize = (pSubTblsInfo->numSubTables - pSubTblsInfo->numTableBlocksInMem) * bufPageSize; createDiskbasedBuf(&pSubTblsInfo->pBlocksBuf, pInfo->bufPageSize, inMemSize, "blocksExternalBuf", tsTempDir); - pSubTblsInfo->numTableBlocksInMem = 0; + pSubTblsInfo->numTableBlocksInMem = pSubTblsInfo->numSubTables; + pSubTblsInfo->numInMemReaders = pSubTblsInfo->numSubTables; return TSDB_CODE_SUCCESS; } @@ -3497,11 +3531,19 @@ static int32_t initSubTableInputs(SOperatorInfo* pOperator, STableMergeScanInfo* for (int32_t i = 0; i < pSubTblsInfo->numSubTables; ++i) { STmsSubTableInput* pInput = pSubTblsInfo->aInputs + i; pInput->type = SUB_TABLE_MEM_BLOCK; + dumpQueryTableCond(&pInfo->base.cond, &pInput->tblCond); pInput->pReaderBlock = createOneDataBlock(pInfo->pResBlock, false); pInput->pPageBlock = createOneDataBlock(pInfo->pResBlock, false); STableKeyInfo* keyInfo = tableListGetInfo(pInfo->base.pTableListInfo, i + pInfo->tableStartIndex); - pAPI->tsdReader.tsdReaderOpen(pHandle->vnode, &pInfo->base.cond, keyInfo, 1, pInput->pReaderBlock, + pInput->pKeyInfo = keyInfo; + if (i + 1 < pSubTblsInfo->numInMemReaders) { + pAPI->tsdReader.tsdReaderOpen(pHandle->vnode, &pInput->tblCond, keyInfo, 1, pInput->pReaderBlock, (void**)&pInput->pReader, GET_TASKID(pTaskInfo), NULL); + pInput->bInMemReader = true; + } else { + pInput->pReader = NULL; + pInput->bInMemReader = false; + } bool hasNext = true; fetchNextSubTableBlockFromReader(pOperator, pInput, &hasNext); if (!hasNext) { @@ -3656,7 +3698,7 @@ static int32_t stopSubTablesTableMergeScan(STableMergeScanInfo* pInfo) { for (int32_t i = 0; i < pSubTblsInfo->numSubTables; ++i) { STmsSubTableInput* pInput = pSubTblsInfo->aInputs + i; - + taosMemoryFree(pInput->tblCond.colList); blockDataDestroy(pInput->pReaderBlock); blockDataDestroy(pInput->pPageBlock); taosArrayDestroy(pInput->aBlockPages); @@ -3903,15 +3945,6 @@ SArray* generateSortByTsInfo(SArray* colMatchInfo, int32_t order) { return pList; } -int32_t dumpQueryTableCond(const SQueryTableDataCond* src, SQueryTableDataCond* dst) { - memcpy((void*)dst, (void*)src, sizeof(SQueryTableDataCond)); - dst->colList = taosMemoryCalloc(src->numOfCols, sizeof(SColumnInfo)); - for (int i = 0; i < src->numOfCols; i++) { - dst->colList[i] = src->colList[i]; - } - return 0; -} - void tableMergeScanTsdbNotifyCb(ETsdReaderNotifyType type, STsdReaderNotifyInfo* info, void* param) { STableMergeScanInfo* pTmsInfo = param; if (type == TSD_READER_NOTIFY_DURATION_START) { From 82871f7ee3364341495bef73f649fc089240cb2f Mon Sep 17 00:00:00 2001 From: slzhou Date: Fri, 26 Jan 2024 14:23:34 +0800 Subject: [PATCH 12/41] feat: no in mem reader and blocks are to disk --- source/libs/executor/src/scanoperator.c | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/source/libs/executor/src/scanoperator.c b/source/libs/executor/src/scanoperator.c index 031ac9a5e7..f0bc4010be 100644 --- a/source/libs/executor/src/scanoperator.c +++ b/source/libs/executor/src/scanoperator.c @@ -3516,8 +3516,8 @@ static int32_t initSubTablesMergeInfo(STableMergeScanInfo* pInfo) { int32_t inMemSize = (pSubTblsInfo->numSubTables - pSubTblsInfo->numTableBlocksInMem) * bufPageSize; createDiskbasedBuf(&pSubTblsInfo->pBlocksBuf, pInfo->bufPageSize, inMemSize, "blocksExternalBuf", tsTempDir); - pSubTblsInfo->numTableBlocksInMem = pSubTblsInfo->numSubTables; - pSubTblsInfo->numInMemReaders = pSubTblsInfo->numSubTables; + pSubTblsInfo->numTableBlocksInMem = 0; + pSubTblsInfo->numInMemReaders = 0; return TSDB_CODE_SUCCESS; } From 686258b26e818e750936ae48c6d3a9a3e74e6158 Mon Sep 17 00:00:00 2001 From: slzhou Date: Fri, 26 Jan 2024 14:52:24 +0800 Subject: [PATCH 13/41] feat: deep clean up for data blocks --- source/libs/executor/src/scanoperator.c | 21 +++++++++++++++++++-- 1 file changed, 19 insertions(+), 2 deletions(-) diff --git a/source/libs/executor/src/scanoperator.c b/source/libs/executor/src/scanoperator.c index f0bc4010be..fca9d4c55c 100644 --- a/source/libs/executor/src/scanoperator.c +++ b/source/libs/executor/src/scanoperator.c @@ -3344,6 +3344,23 @@ static int32_t subTblRowCompareFn(const void* pLeft, const void* pRight, void* p return ret; } +static void blockDataDeepCleanup(SSDataBlock* pDataBlock) { + size_t numOfCols = taosArrayGetSize(pDataBlock->pDataBlock); + for (int32_t i = 0; i < numOfCols; ++i) { + SColumnInfoData* p = taosArrayGet(pDataBlock->pDataBlock, i); + taosMemoryFreeClear(p->pData); + if (IS_VAR_DATA_TYPE(p->info.type)) { + taosMemoryFreeClear(p->varmeta.offset); + p->varmeta.length = 0; + p->varmeta.allocLen = 0; + } else { + taosMemoryFreeClear(p->nullbitmap); + } + } + pDataBlock->info.capacity = 0; + pDataBlock->info.rows = 0; +} + int32_t dumpQueryTableCond(const SQueryTableDataCond* src, SQueryTableDataCond* dst) { memcpy((void*)dst, (void*)src, sizeof(SQueryTableDataCond)); dst->colList = taosMemoryCalloc(src->numOfCols, sizeof(SColumnInfo)); @@ -3362,7 +3379,7 @@ static int32_t fetchNextSubTableBlockFromReader(SOperatorInfo* pOperator, STmsSu SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; const SStorageAPI* pAPI= &pTaskInfo->storageAPI; - blockDataCleanup(pInput->pReaderBlock); + blockDataDeepCleanup(pInput->pReaderBlock); if (!pInput->bInMemReader) { code = pAPI->tsdReader.tsdReaderOpen(pHandle->vnode, &pInput->tblCond, pInput->pKeyInfo, 1, pInput->pReaderBlock, (void**)&pInput->pReader, GET_TASKID(pTaskInfo), NULL); @@ -3452,7 +3469,7 @@ static int32_t saveSubTableBlock(const STableMergeScanInfo* pInfo, STmsSubTables blockDataDestroy(p); start = stop + 1; } - blockDataCleanup(pInput->pReaderBlock); + blockDataDeepCleanup(pInput->pReaderBlock); return TSDB_CODE_SUCCESS; } From f19e66909d063d20463853aa6e7594bbe4355fa3 Mon Sep 17 00:00:00 2001 From: slzhou Date: Fri, 26 Jan 2024 17:01:08 +0800 Subject: [PATCH 14/41] fix: only deep cleanup datablock when not inmem reader --- source/libs/executor/src/scanoperator.c | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/source/libs/executor/src/scanoperator.c b/source/libs/executor/src/scanoperator.c index fca9d4c55c..caf48174df 100644 --- a/source/libs/executor/src/scanoperator.c +++ b/source/libs/executor/src/scanoperator.c @@ -3379,7 +3379,7 @@ static int32_t fetchNextSubTableBlockFromReader(SOperatorInfo* pOperator, STmsSu SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; const SStorageAPI* pAPI= &pTaskInfo->storageAPI; - blockDataDeepCleanup(pInput->pReaderBlock); + blockDataCleanup(pInput->pReaderBlock); if (!pInput->bInMemReader) { code = pAPI->tsdReader.tsdReaderOpen(pHandle->vnode, &pInput->tblCond, pInput->pKeyInfo, 1, pInput->pReaderBlock, (void**)&pInput->pReader, GET_TASKID(pTaskInfo), NULL); @@ -3469,7 +3469,11 @@ static int32_t saveSubTableBlock(const STableMergeScanInfo* pInfo, STmsSubTables blockDataDestroy(p); start = stop + 1; } - blockDataDeepCleanup(pInput->pReaderBlock); + if (!pInput->bInMemReader) { + blockDataDeepCleanup(pInput->pReaderBlock); + } else { + blockDataCleanup(pInput->pReaderBlock); + } return TSDB_CODE_SUCCESS; } @@ -3533,8 +3537,8 @@ static int32_t initSubTablesMergeInfo(STableMergeScanInfo* pInfo) { int32_t inMemSize = (pSubTblsInfo->numSubTables - pSubTblsInfo->numTableBlocksInMem) * bufPageSize; createDiskbasedBuf(&pSubTblsInfo->pBlocksBuf, pInfo->bufPageSize, inMemSize, "blocksExternalBuf", tsTempDir); - pSubTblsInfo->numTableBlocksInMem = 0; - pSubTblsInfo->numInMemReaders = 0; + pSubTblsInfo->numTableBlocksInMem = pSubTblsInfo->numSubTables; + pSubTblsInfo->numInMemReaders = pSubTblsInfo->numSubTables; return TSDB_CODE_SUCCESS; } From 6849ccd280f6c3d4a34bd9c008563ab3f352e885 Mon Sep 17 00:00:00 2001 From: slzhou Date: Mon, 29 Jan 2024 11:02:38 +0800 Subject: [PATCH 15/41] fix: pre-allocated columns has no data --- source/libs/executor/src/mergejoinoperator.c | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/source/libs/executor/src/mergejoinoperator.c b/source/libs/executor/src/mergejoinoperator.c index b4461f20b1..7f12d9d225 100644 --- a/source/libs/executor/src/mergejoinoperator.c +++ b/source/libs/executor/src/mergejoinoperator.c @@ -399,7 +399,9 @@ static void mergeJoinJoinLeftRight(struct SOperatorInfo* pOperator, SSDataBlock* colDataSetNULL(pDst, currRow); } else { char* p = colDataGetData(pSrc, rowIndex); - colDataSetVal(pDst, currRow, p, false); + if (p != NULL) { + colDataSetVal(pDst, currRow, p, false); + } } } } From 7d0edcc54e898fe4d9971de931b47b52a56f6a35 Mon Sep 17 00:00:00 2001 From: slzhou Date: Mon, 29 Jan 2024 12:56:48 +0800 Subject: [PATCH 16/41] fix: fix memory sanitizer error --- source/libs/executor/src/scanoperator.c | 4 ---- 1 file changed, 4 deletions(-) diff --git a/source/libs/executor/src/scanoperator.c b/source/libs/executor/src/scanoperator.c index d5f815dc85..892eb38c88 100644 --- a/source/libs/executor/src/scanoperator.c +++ b/source/libs/executor/src/scanoperator.c @@ -4038,8 +4038,6 @@ int32_t startGroupTableMergeScan(SOperatorInfo* pOperator) { int32_t tableStartIdx = pInfo->tableStartIndex; int32_t tableEndIdx = pInfo->tableEndIndex; - tSimpleHashClear(pInfo->mTableNumRows); - int32_t numOfTable = tableEndIdx - tableStartIdx + 1; STableKeyInfo* startKeyInfo = tableListGetInfo(pInfo->base.pTableListInfo, tableStartIdx); pAPI->tsdReader.tsdReaderOpen(pHandle->vnode, &pInfo->base.cond, startKeyInfo, numOfTable, pInfo->pReaderBlock, @@ -4315,8 +4313,6 @@ SOperatorInfo* createTableMergeScanOperatorInfo(STableScanPhysiNode* pTableScanN //start one reader variable pInfo->pSortInputBlock = createOneDataBlock(pInfo->pResBlock, false); - pInfo->mTableNumRows = tSimpleHashInit(1024, - taosGetDefaultHashFunction(TSDB_DATA_TYPE_UBIGINT)); if (!tsExperimental) { pInfo->filesetDelimited = false; From af422e77c6b017413cd948b36f03527910357c2a Mon Sep 17 00:00:00 2001 From: slzhou Date: Mon, 29 Jan 2024 14:36:40 +0800 Subject: [PATCH 17/41] fix: restore pseudo column pData and not skip and data block cleanup --- source/common/src/tdatablock.c | 2 -- source/libs/executor/src/mergejoinoperator.c | 4 +-- source/libs/executor/src/scanoperator.c | 30 ++++---------------- 3 files changed, 6 insertions(+), 30 deletions(-) diff --git a/source/common/src/tdatablock.c b/source/common/src/tdatablock.c index 200fd2bc46..d712a381e7 100644 --- a/source/common/src/tdatablock.c +++ b/source/common/src/tdatablock.c @@ -573,7 +573,6 @@ int32_t blockDataSplitRows(SSDataBlock* pBlock, bool hasVarCol, int32_t startInd for (int32_t j = startIndex; j < numOfRows; ++j) { for (int32_t i = 0; i < numOfCols; ++i) { SColumnInfoData* pColInfoData = TARRAY_GET_ELEM(pBlock->pDataBlock, i); - if (pColInfoData->pData == NULL) continue; if (IS_VAR_DATA_TYPE(pColInfoData->info.type)) { if (pColInfoData->varmeta.offset[j] != -1) { char* p = colDataGetData(pColInfoData, j); @@ -632,7 +631,6 @@ SSDataBlock* blockDataExtractBlock(SSDataBlock* pBlock, int32_t startIndex, int3 for (int32_t i = 0; i < numOfCols; ++i) { SColumnInfoData* pColData = taosArrayGet(pBlock->pDataBlock, i); SColumnInfoData* pDstCol = taosArrayGet(pDst->pDataBlock, i); - if (pColData->pData == NULL) continue; for (int32_t j = startIndex; j < (startIndex + rowCount); ++j) { bool isNull = false; if (pBlock->pBlockAgg == NULL) { diff --git a/source/libs/executor/src/mergejoinoperator.c b/source/libs/executor/src/mergejoinoperator.c index 7f12d9d225..b4461f20b1 100644 --- a/source/libs/executor/src/mergejoinoperator.c +++ b/source/libs/executor/src/mergejoinoperator.c @@ -399,9 +399,7 @@ static void mergeJoinJoinLeftRight(struct SOperatorInfo* pOperator, SSDataBlock* colDataSetNULL(pDst, currRow); } else { char* p = colDataGetData(pSrc, rowIndex); - if (p != NULL) { - colDataSetVal(pDst, currRow, p, false); - } + colDataSetVal(pDst, currRow, p, false); } } } diff --git a/source/libs/executor/src/scanoperator.c b/source/libs/executor/src/scanoperator.c index 892eb38c88..a4bb94e494 100644 --- a/source/libs/executor/src/scanoperator.c +++ b/source/libs/executor/src/scanoperator.c @@ -3344,23 +3344,6 @@ static int32_t subTblRowCompareFn(const void* pLeft, const void* pRight, void* p return ret; } -static void blockDataDeepCleanup(SSDataBlock* pDataBlock) { - size_t numOfCols = taosArrayGetSize(pDataBlock->pDataBlock); - for (int32_t i = 0; i < numOfCols; ++i) { - SColumnInfoData* p = taosArrayGet(pDataBlock->pDataBlock, i); - taosMemoryFreeClear(p->pData); - if (IS_VAR_DATA_TYPE(p->info.type)) { - taosMemoryFreeClear(p->varmeta.offset); - p->varmeta.length = 0; - p->varmeta.allocLen = 0; - } else { - taosMemoryFreeClear(p->nullbitmap); - } - } - pDataBlock->info.capacity = 0; - pDataBlock->info.rows = 0; -} - int32_t dumpQueryTableCond(const SQueryTableDataCond* src, SQueryTableDataCond* dst) { memcpy((void*)dst, (void*)src, sizeof(SQueryTableDataCond)); dst->colList = taosMemoryCalloc(src->numOfCols, sizeof(SColumnInfo)); @@ -3469,11 +3452,7 @@ static int32_t saveSubTableBlock(const STableMergeScanInfo* pInfo, STmsSubTables blockDataDestroy(p); start = stop + 1; } - if (!pInput->bInMemReader) { - blockDataDeepCleanup(pInput->pReaderBlock); - } else { - blockDataCleanup(pInput->pReaderBlock); - } + return TSDB_CODE_SUCCESS; } @@ -3660,9 +3639,10 @@ static int32_t appendChosenRowToDataBlock(STmsSubTablesMergeInfo* pSubTblsInfo, if (isNull) { colDataSetVal(pColInfo, pBlock->info.rows, NULL, true); } else { - if (!pSrcColInfo->pData) continue; - char* pData = colDataGetData(pSrcColInfo, pInput->rowIdx); - colDataSetVal(pColInfo, pBlock->info.rows, pData, false); + if (pSrcColInfo->pData != NULL) { + char* pData = colDataGetData(pSrcColInfo, pInput->rowIdx); + colDataSetVal(pColInfo, pBlock->info.rows, pData, false); + } } } pBlock->info.dataLoad = 1; From 7dcb47793307a4dc14a870eb89395ce26ea52b75 Mon Sep 17 00:00:00 2001 From: shenglian zhou Date: Thu, 1 Feb 2024 09:38:35 +0800 Subject: [PATCH 18/41] feat: add front end for ordered blocks in mem sort --- include/common/ttokendef.h | 1 + include/libs/nodes/plannodes.h | 2 ++ include/libs/nodes/querynodes.h | 1 + source/libs/executor/src/scanoperator.c | 12 +++++++++--- source/libs/nodes/src/nodesCloneFuncs.c | 2 ++ source/libs/nodes/src/nodesCodeFuncs.c | 14 ++++++++++++++ source/libs/nodes/src/nodesMsgFuncs.c | 6 ++++++ source/libs/parser/src/parAstCreater.c | 11 +++++++++++ source/libs/parser/src/parTokenizer.c | 1 + source/libs/planner/inc/planInt.h | 1 + source/libs/planner/src/planLogicCreater.c | 2 +- source/libs/planner/src/planPhysiCreater.c | 1 + source/libs/planner/src/planUtil.c | 12 ++++++++++++ 13 files changed, 62 insertions(+), 4 deletions(-) diff --git a/include/common/ttokendef.h b/include/common/ttokendef.h index 445fe0737b..3bd8438062 100644 --- a/include/common/ttokendef.h +++ b/include/common/ttokendef.h @@ -374,6 +374,7 @@ #define TK_NO_BATCH_SCAN 607 #define TK_SORT_FOR_GROUP 608 #define TK_PARTITION_FIRST 609 +#define TK_SEQ_BLOCKS_SORT 610 #define TK_NK_NIL 65535 diff --git a/include/libs/nodes/plannodes.h b/include/libs/nodes/plannodes.h index 8e375e9da8..38afccb91f 100644 --- a/include/libs/nodes/plannodes.h +++ b/include/libs/nodes/plannodes.h @@ -121,6 +121,7 @@ typedef struct SScanLogicNode { bool filesetDelimited; // returned blocks delimited by fileset bool isCountByTag; // true if selectstmt hasCountFunc & part by tag/tbname SArray* pFuncTypes; // for last, last_row + bool seqBlocksSort; // for table merge scan } SScanLogicNode; typedef struct SJoinLogicNode { @@ -439,6 +440,7 @@ typedef struct STableScanPhysiNode { int8_t igCheckUpdate; bool filesetDelimited; bool needCountEmptyTable; + bool seqBlocksSort; } STableScanPhysiNode; typedef STableScanPhysiNode STableSeqScanPhysiNode; diff --git a/include/libs/nodes/querynodes.h b/include/libs/nodes/querynodes.h index 9647c0adac..45619a8501 100644 --- a/include/libs/nodes/querynodes.h +++ b/include/libs/nodes/querynodes.h @@ -128,6 +128,7 @@ typedef enum EHintOption { HINT_BATCH_SCAN, HINT_SORT_FOR_GROUP, HINT_PARTITION_FIRST, + HINT_SEQ_BLOCKS_SORT } EHintOption; typedef struct SHintNode { diff --git a/source/libs/executor/src/scanoperator.c b/source/libs/executor/src/scanoperator.c index a4bb94e494..9e48bbb214 100644 --- a/source/libs/executor/src/scanoperator.c +++ b/source/libs/executor/src/scanoperator.c @@ -3716,7 +3716,7 @@ static int32_t stopSubTablesTableMergeScan(STableMergeScanInfo* pInfo) { return TSDB_CODE_SUCCESS; } -SSDataBlock* doTableMergeScanSubTables(SOperatorInfo* pOperator) { +SSDataBlock* doTableMergeScanSeqBlocksInMem(SOperatorInfo* pOperator) { if (pOperator->status == OP_EXEC_DONE) { return NULL; } @@ -4304,8 +4304,14 @@ SOperatorInfo* createTableMergeScanOperatorInfo(STableScanPhysiNode* pTableScanN pInfo, pTaskInfo); pOperator->exprSupp.numOfExprs = numOfCols; - pOperator->fpSet = createOperatorFpSet(optrDummyOpenFn, doTableMergeScanSubTables, NULL, destroyTableMergeScanOperatorInfo, - optrDefaultBufFn, getTableMergeScanExplainExecInfo, optrDefaultGetNextExtFn, NULL); + pOperator->fpSet = + createOperatorFpSet(optrDummyOpenFn, + pTableScanNode->seqBlocksSort ? doTableMergeScanSeqBlocksInMem : doTableMergeScan, + NULL, + destroyTableMergeScanOperatorInfo, + optrDefaultBufFn, + getTableMergeScanExplainExecInfo, + optrDefaultGetNextExtFn, NULL); pOperator->cost.openCost = 0; return pOperator; diff --git a/source/libs/nodes/src/nodesCloneFuncs.c b/source/libs/nodes/src/nodesCloneFuncs.c index bc9839792c..e79fd1e1e3 100644 --- a/source/libs/nodes/src/nodesCloneFuncs.c +++ b/source/libs/nodes/src/nodesCloneFuncs.c @@ -449,6 +449,7 @@ static int32_t logicScanCopy(const SScanLogicNode* pSrc, SScanLogicNode* pDst) { COPY_SCALAR_FIELD(filesetDelimited); COPY_SCALAR_FIELD(isCountByTag); CLONE_OBJECT_FIELD(pFuncTypes, functParamClone); + COPY_SCALAR_FIELD(seqBlocksSort); return TSDB_CODE_SUCCESS; } @@ -679,6 +680,7 @@ static int32_t physiTableScanCopy(const STableScanPhysiNode* pSrc, STableScanPhy COPY_SCALAR_FIELD(igExpired); COPY_SCALAR_FIELD(filesetDelimited); COPY_SCALAR_FIELD(needCountEmptyTable); + COPY_SCALAR_FIELD(seqBlocksSort); return TSDB_CODE_SUCCESS; } diff --git a/source/libs/nodes/src/nodesCodeFuncs.c b/source/libs/nodes/src/nodesCodeFuncs.c index d36a1bd6b9..bf73bc5f50 100644 --- a/source/libs/nodes/src/nodesCodeFuncs.c +++ b/source/libs/nodes/src/nodesCodeFuncs.c @@ -682,6 +682,7 @@ static const char* jkScanLogicPlanTagCond = "TagCond"; static const char* jkScanLogicPlanGroupTags = "GroupTags"; static const char* jkScanLogicPlanOnlyMetaCtbIdx = "OnlyMetaCtbIdx"; static const char* jkScanLogicPlanFilesetDelimited = "FilesetDelimited"; +static const char* jkScanLogicPlanSeqBlocksSort = "SeqBlocksSort"; static int32_t logicScanNodeToJson(const void* pObj, SJson* pJson) { const SScanLogicNode* pNode = (const SScanLogicNode*)pObj; @@ -729,6 +730,9 @@ static int32_t logicScanNodeToJson(const void* pObj, SJson* pJson) { if (TSDB_CODE_SUCCESS == code) { code = tjsonAddBoolToObject(pJson, jkScanLogicPlanFilesetDelimited, pNode->filesetDelimited); } + if (TSDB_CODE_SUCCESS == code) { + code = tjsonAddBoolToObject(pJson, jkScanLogicPlanFilesetDelimited, pNode->seqBlocksSort); + } return code; } @@ -779,6 +783,9 @@ static int32_t jsonToLogicScanNode(const SJson* pJson, void* pObj) { if (TSDB_CODE_SUCCESS == code) { code = tjsonGetBoolValue(pJson, jkScanLogicPlanFilesetDelimited, &pNode->filesetDelimited); } + if (TSDB_CODE_SUCCESS == code) { + code = tjsonGetBoolValue(pJson, jkScanLogicPlanFilesetDelimited, &pNode->seqBlocksSort); + } return code; } @@ -1872,6 +1879,7 @@ static const char* jkTableScanPhysiPlanAssignBlockUid = "AssignBlockUid"; static const char* jkTableScanPhysiPlanIgnoreUpdate = "IgnoreUpdate"; static const char* jkTableScanPhysiPlanFilesetDelimited = "FilesetDelimited"; static const char* jkTableScanPhysiPlanNeedCountEmptyTable = "NeedCountEmptyTable"; +static const char* jkTableScanPhysiPlanSeqBlocksSort = "SeqBlocksSort"; static int32_t physiTableScanNodeToJson(const void* pObj, SJson* pJson) { const STableScanPhysiNode* pNode = (const STableScanPhysiNode*)pObj; @@ -1946,6 +1954,9 @@ static int32_t physiTableScanNodeToJson(const void* pObj, SJson* pJson) { if (TSDB_CODE_SUCCESS == code) { code = tjsonAddBoolToObject(pJson, jkTableScanPhysiPlanNeedCountEmptyTable, pNode->needCountEmptyTable); } + if (TSDB_CODE_SUCCESS == code) { + code = tjsonAddBoolToObject(pJson, jkTableScanPhysiPlanSeqBlocksSort, pNode->seqBlocksSort); + } return code; } @@ -2022,6 +2033,9 @@ static int32_t jsonToPhysiTableScanNode(const SJson* pJson, void* pObj) { if (TSDB_CODE_SUCCESS == code) { code = tjsonGetBoolValue(pJson, jkTableScanPhysiPlanNeedCountEmptyTable, &pNode->needCountEmptyTable); } + if (TSDB_CODE_SUCCESS == code) { + code = tjsonGetBoolValue(pJson, jkTableScanPhysiPlanSeqBlocksSort, &pNode->seqBlocksSort); + } return code; } diff --git a/source/libs/nodes/src/nodesMsgFuncs.c b/source/libs/nodes/src/nodesMsgFuncs.c index 3fd219b8d8..963e2af67b 100644 --- a/source/libs/nodes/src/nodesMsgFuncs.c +++ b/source/libs/nodes/src/nodesMsgFuncs.c @@ -2185,6 +2185,9 @@ static int32_t physiTableScanNodeInlineToMsg(const void* pObj, STlvEncoder* pEnc if (TSDB_CODE_SUCCESS == code) { code = tlvEncodeValueBool(pEncoder, pNode->needCountEmptyTable); } + if (TSDB_CODE_SUCCESS == code) { + code = tlvEncodeValueBool(pEncoder, pNode->seqBlocksSort); + } return code; } @@ -2269,6 +2272,9 @@ static int32_t msgToPhysiTableScanNodeInline(STlvDecoder* pDecoder, void* pObj) if (TSDB_CODE_SUCCESS == code) { code = tlvDecodeValueBool(pDecoder, &pNode->needCountEmptyTable); } + if (TSDB_CODE_SUCCESS == code) { + code = tlvDecodeValueBool(pDecoder, &pNode->seqBlocksSort); + } return code; } diff --git a/source/libs/parser/src/parAstCreater.c b/source/libs/parser/src/parAstCreater.c index 8e89ae1f53..ade937f552 100644 --- a/source/libs/parser/src/parAstCreater.c +++ b/source/libs/parser/src/parAstCreater.c @@ -401,6 +401,9 @@ bool addHintNodeToList(SAstCreateContext* pCxt, SNodeList** ppHintList, EHintOpt case HINT_PARTITION_FIRST: if (paramNum > 0 || hasHint(*ppHintList, HINT_SORT_FOR_GROUP)) return true; break; + case HINT_SEQ_BLOCKS_SORT: + if (paramNum > 0 || hasHint(*ppHintList, HINT_SEQ_BLOCKS_SORT)) return true; + break; default: return true; } @@ -479,6 +482,14 @@ SNodeList* createHintNodeList(SAstCreateContext* pCxt, const SToken* pLiteral) { } opt = HINT_PARTITION_FIRST; break; + case TK_SEQ_BLOCKS_SORT: + lastComma = false; + if (0 != opt || inParamList) { + quit = true; + break; + } + opt = HINT_SEQ_BLOCKS_SORT; + break; case TK_NK_LP: lastComma = false; if (0 == opt || inParamList) { diff --git a/source/libs/parser/src/parTokenizer.c b/source/libs/parser/src/parTokenizer.c index 072892fe7f..cebf82c5f3 100644 --- a/source/libs/parser/src/parTokenizer.c +++ b/source/libs/parser/src/parTokenizer.c @@ -199,6 +199,7 @@ static SKeyword keywordTable[] = { {"SCHEMALESS", TK_SCHEMALESS}, {"SCORES", TK_SCORES}, {"SELECT", TK_SELECT}, + {"SEQ_BLOCKS_SORT", TK_SEQ_BLOCKS_SORT}, {"SERVER_STATUS", TK_SERVER_STATUS}, {"SERVER_VERSION", TK_SERVER_VERSION}, {"SESSION", TK_SESSION}, diff --git a/source/libs/planner/inc/planInt.h b/source/libs/planner/inc/planInt.h index d4074e1373..6ca766a98d 100644 --- a/source/libs/planner/inc/planInt.h +++ b/source/libs/planner/inc/planInt.h @@ -47,6 +47,7 @@ int32_t validateQueryPlan(SPlanContext* pCxt, SQueryPlan* pPlan); bool getBatchScanOptionFromHint(SNodeList* pList); bool getSortForGroupOptHint(SNodeList* pList); +bool getSeqBlocksSortHint(SNodeList* pList); bool getOptHint(SNodeList* pList, EHintOption hint); SLogicNode* getLogicNodeRootNode(SLogicNode* pCurr); int32_t collectTableAliasFromNodes(SNode* pNode, SSHashObj** ppRes); diff --git a/source/libs/planner/src/planLogicCreater.c b/source/libs/planner/src/planLogicCreater.c index 12b7360165..6852c9c359 100644 --- a/source/libs/planner/src/planLogicCreater.c +++ b/source/libs/planner/src/planLogicCreater.c @@ -501,7 +501,7 @@ static int32_t createScanLogicNode(SLogicPlanContext* pCxt, SSelectStmt* pSelect } else { nodesDestroyNode((SNode*)pScan); } - + pScan->seqBlocksSort = getSeqBlocksSortHint(pSelect->pHint); pCxt->hasScan = true; return code; diff --git a/source/libs/planner/src/planPhysiCreater.c b/source/libs/planner/src/planPhysiCreater.c index 21c637116f..c560a29234 100644 --- a/source/libs/planner/src/planPhysiCreater.c +++ b/source/libs/planner/src/planPhysiCreater.c @@ -651,6 +651,7 @@ static int32_t createTableScanPhysiNode(SPhysiPlanContext* pCxt, SSubplan* pSubp pTableScan->assignBlockUid = pCxt->pPlanCxt->rSmaQuery ? true : false; pTableScan->filesetDelimited = pScanLogicNode->filesetDelimited; pTableScan->needCountEmptyTable = pScanLogicNode->isCountByTag; + pTableScan->seqBlocksSort = pScanLogicNode->seqBlocksSort; int32_t code = createScanPhysiNodeFinalize(pCxt, pSubplan, pScanLogicNode, (SScanPhysiNode*)pTableScan, pPhyNode); if (TSDB_CODE_SUCCESS == code) { diff --git a/source/libs/planner/src/planUtil.c b/source/libs/planner/src/planUtil.c index aeef3f2487..266eefc930 100644 --- a/source/libs/planner/src/planUtil.c +++ b/source/libs/planner/src/planUtil.c @@ -455,6 +455,18 @@ bool getOptHint(SNodeList* pList, EHintOption hint) { return false; } +bool getSeqBlocksSortOptHint(SNodeList* pList) { + if (!pList) return false; + SNode* pNode; + FOREACH(pNode, pList) { + SHintNode* pHint = (SHintNode*)pNode; + if (pHint->option == HINT_SEQ_BLOCKS_SORT) { + return true; + } + } + return false; +} + int32_t collectTableAliasFromNodes(SNode* pNode, SSHashObj** ppRes) { int32_t code = TSDB_CODE_SUCCESS; SLogicNode* pCurr = (SLogicNode*)pNode; From 1e30ac9408631cadc098a627a3c43fa5f7b0c00e Mon Sep 17 00:00:00 2001 From: slzhou Date: Thu, 1 Feb 2024 10:30:07 +0800 Subject: [PATCH 19/41] fix: compilation error --- source/libs/planner/inc/planInt.h | 2 +- source/libs/planner/src/planLogicCreater.c | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/source/libs/planner/inc/planInt.h b/source/libs/planner/inc/planInt.h index 6ca766a98d..34a8a4d70f 100644 --- a/source/libs/planner/inc/planInt.h +++ b/source/libs/planner/inc/planInt.h @@ -47,7 +47,7 @@ int32_t validateQueryPlan(SPlanContext* pCxt, SQueryPlan* pPlan); bool getBatchScanOptionFromHint(SNodeList* pList); bool getSortForGroupOptHint(SNodeList* pList); -bool getSeqBlocksSortHint(SNodeList* pList); +bool getSeqBlocksSortOptHint(SNodeList* pList); bool getOptHint(SNodeList* pList, EHintOption hint); SLogicNode* getLogicNodeRootNode(SLogicNode* pCurr); int32_t collectTableAliasFromNodes(SNode* pNode, SSHashObj** ppRes); diff --git a/source/libs/planner/src/planLogicCreater.c b/source/libs/planner/src/planLogicCreater.c index 6852c9c359..ed7b266198 100644 --- a/source/libs/planner/src/planLogicCreater.c +++ b/source/libs/planner/src/planLogicCreater.c @@ -501,7 +501,7 @@ static int32_t createScanLogicNode(SLogicPlanContext* pCxt, SSelectStmt* pSelect } else { nodesDestroyNode((SNode*)pScan); } - pScan->seqBlocksSort = getSeqBlocksSortHint(pSelect->pHint); + pScan->seqBlocksSort = getSeqBlocksSortOptHint(pSelect->pHint); pCxt->hasScan = true; return code; From bcba3007dcd937da329c6df1d3e3a0698bcf19f3 Mon Sep 17 00:00:00 2001 From: slzhou Date: Thu, 1 Feb 2024 16:40:53 +0800 Subject: [PATCH 20/41] fix: query and free RSS quickly --- source/libs/executor/src/scanoperator.c | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/source/libs/executor/src/scanoperator.c b/source/libs/executor/src/scanoperator.c index bb01836881..8c7c93e4d5 100644 --- a/source/libs/executor/src/scanoperator.c +++ b/source/libs/executor/src/scanoperator.c @@ -3387,6 +3387,8 @@ static int32_t fetchNextSubTableBlockFromReader(SOperatorInfo* pOperator, STmsSu if (!hasNext || isTaskKilled(pTaskInfo)) { if (isTaskKilled(pTaskInfo)) { pAPI->tsdReader.tsdReaderReleaseDataBlock(pInfo->base.dataReader); + pInfo->base.dataReader = NULL; + T_LONG_JMP(pTaskInfo->env, pTaskInfo->code); } *pSubTableHasBlock = false; break; @@ -3539,6 +3541,11 @@ static int32_t initSubTableInputs(SOperatorInfo* pOperator, STableMergeScanInfo* pInput->pPageBlock = createOneDataBlock(pInfo->pResBlock, false); STableKeyInfo* keyInfo = tableListGetInfo(pInfo->base.pTableListInfo, i + pInfo->tableStartIndex); pInput->pKeyInfo = keyInfo; + + if (isTaskKilled(pTaskInfo)) { + T_LONG_JMP(pTaskInfo->env, pTaskInfo->code); + } + if (i + 1 < pSubTblsInfo->numInMemReaders) { pAPI->tsdReader.tsdReaderOpen(pHandle->vnode, &pInput->tblCond, keyInfo, 1, pInput->pReaderBlock, (void**)&pInput->pReader, GET_TASKID(pTaskInfo), NULL); @@ -3675,6 +3682,11 @@ static SSDataBlock* getSubTablesSortedBlock(SOperatorInfo* pOperator, SSDataBloc break; } } + + if (isTaskKilled(pTaskInfo)) { + T_LONG_JMP(pOperator->pTaskInfo->env, pTaskInfo->code); + } + bool limitReached = applyLimitOffset(&pInfo->limitInfo, pResBlock, pTaskInfo); if (finished || limitReached || pResBlock->info.rows > 0) { break; @@ -3716,6 +3728,7 @@ static int32_t stopSubTablesTableMergeScan(STableMergeScanInfo* pInfo) { taosMemoryFree(pSubTblsInfo); pInfo->pSubTablesMergeInfo = NULL; } + taosMemoryTrim(0); return TSDB_CODE_SUCCESS; } From f77b602da950c9cd5417c51b2eed9c4117eaf8e1 Mon Sep 17 00:00:00 2001 From: shenglian zhou Date: Fri, 2 Feb 2024 14:45:04 +0800 Subject: [PATCH 21/41] fix: change from seq_blocks_sort to para_tables_sort --- include/common/ttokendef.h | 2 +- include/libs/nodes/plannodes.h | 4 ++-- include/libs/nodes/querynodes.h | 2 +- source/libs/executor/src/scanoperator.c | 2 +- source/libs/nodes/src/nodesCloneFuncs.c | 4 ++-- source/libs/nodes/src/nodesCodeFuncs.c | 12 ++++++------ source/libs/nodes/src/nodesMsgFuncs.c | 4 ++-- source/libs/parser/src/parAstCreater.c | 8 ++++---- source/libs/parser/src/parTokenizer.c | 2 +- source/libs/planner/inc/planInt.h | 2 +- source/libs/planner/src/planLogicCreater.c | 2 +- source/libs/planner/src/planPhysiCreater.c | 2 +- source/libs/planner/src/planUtil.c | 4 ++-- 13 files changed, 25 insertions(+), 25 deletions(-) diff --git a/include/common/ttokendef.h b/include/common/ttokendef.h index 3bd8438062..e1e5fc8f42 100644 --- a/include/common/ttokendef.h +++ b/include/common/ttokendef.h @@ -374,7 +374,7 @@ #define TK_NO_BATCH_SCAN 607 #define TK_SORT_FOR_GROUP 608 #define TK_PARTITION_FIRST 609 -#define TK_SEQ_BLOCKS_SORT 610 +#define TK_PARA_TABLES_SORT 610 #define TK_NK_NIL 65535 diff --git a/include/libs/nodes/plannodes.h b/include/libs/nodes/plannodes.h index 38afccb91f..0eabb0f426 100644 --- a/include/libs/nodes/plannodes.h +++ b/include/libs/nodes/plannodes.h @@ -121,7 +121,7 @@ typedef struct SScanLogicNode { bool filesetDelimited; // returned blocks delimited by fileset bool isCountByTag; // true if selectstmt hasCountFunc & part by tag/tbname SArray* pFuncTypes; // for last, last_row - bool seqBlocksSort; // for table merge scan + bool paraTablesSort; // for table merge scan } SScanLogicNode; typedef struct SJoinLogicNode { @@ -440,7 +440,7 @@ typedef struct STableScanPhysiNode { int8_t igCheckUpdate; bool filesetDelimited; bool needCountEmptyTable; - bool seqBlocksSort; + bool paraTablesSort; } STableScanPhysiNode; typedef STableScanPhysiNode STableSeqScanPhysiNode; diff --git a/include/libs/nodes/querynodes.h b/include/libs/nodes/querynodes.h index 45619a8501..d4a3b10cf2 100644 --- a/include/libs/nodes/querynodes.h +++ b/include/libs/nodes/querynodes.h @@ -128,7 +128,7 @@ typedef enum EHintOption { HINT_BATCH_SCAN, HINT_SORT_FOR_GROUP, HINT_PARTITION_FIRST, - HINT_SEQ_BLOCKS_SORT + HINT_PARA_TABLES_SORT } EHintOption; typedef struct SHintNode { diff --git a/source/libs/executor/src/scanoperator.c b/source/libs/executor/src/scanoperator.c index 8c7c93e4d5..55cad4236e 100644 --- a/source/libs/executor/src/scanoperator.c +++ b/source/libs/executor/src/scanoperator.c @@ -4322,7 +4322,7 @@ SOperatorInfo* createTableMergeScanOperatorInfo(STableScanPhysiNode* pTableScanN pOperator->fpSet = createOperatorFpSet(optrDummyOpenFn, - pTableScanNode->seqBlocksSort ? doTableMergeScanSeqBlocksInMem : doTableMergeScan, + pTableScanNode->paraTablesSort ? doTableMergeScanSeqBlocksInMem : doTableMergeScan, NULL, destroyTableMergeScanOperatorInfo, optrDefaultBufFn, diff --git a/source/libs/nodes/src/nodesCloneFuncs.c b/source/libs/nodes/src/nodesCloneFuncs.c index e79fd1e1e3..65330b22b9 100644 --- a/source/libs/nodes/src/nodesCloneFuncs.c +++ b/source/libs/nodes/src/nodesCloneFuncs.c @@ -449,7 +449,7 @@ static int32_t logicScanCopy(const SScanLogicNode* pSrc, SScanLogicNode* pDst) { COPY_SCALAR_FIELD(filesetDelimited); COPY_SCALAR_FIELD(isCountByTag); CLONE_OBJECT_FIELD(pFuncTypes, functParamClone); - COPY_SCALAR_FIELD(seqBlocksSort); + COPY_SCALAR_FIELD(paraTablesSort); return TSDB_CODE_SUCCESS; } @@ -680,7 +680,7 @@ static int32_t physiTableScanCopy(const STableScanPhysiNode* pSrc, STableScanPhy COPY_SCALAR_FIELD(igExpired); COPY_SCALAR_FIELD(filesetDelimited); COPY_SCALAR_FIELD(needCountEmptyTable); - COPY_SCALAR_FIELD(seqBlocksSort); + COPY_SCALAR_FIELD(paraTablesSort); return TSDB_CODE_SUCCESS; } diff --git a/source/libs/nodes/src/nodesCodeFuncs.c b/source/libs/nodes/src/nodesCodeFuncs.c index bf73bc5f50..31f9129c75 100644 --- a/source/libs/nodes/src/nodesCodeFuncs.c +++ b/source/libs/nodes/src/nodesCodeFuncs.c @@ -682,7 +682,7 @@ static const char* jkScanLogicPlanTagCond = "TagCond"; static const char* jkScanLogicPlanGroupTags = "GroupTags"; static const char* jkScanLogicPlanOnlyMetaCtbIdx = "OnlyMetaCtbIdx"; static const char* jkScanLogicPlanFilesetDelimited = "FilesetDelimited"; -static const char* jkScanLogicPlanSeqBlocksSort = "SeqBlocksSort"; +static const char* jkScanLogicPlanparaTablesSort = "paraTablesSort"; static int32_t logicScanNodeToJson(const void* pObj, SJson* pJson) { const SScanLogicNode* pNode = (const SScanLogicNode*)pObj; @@ -731,7 +731,7 @@ static int32_t logicScanNodeToJson(const void* pObj, SJson* pJson) { code = tjsonAddBoolToObject(pJson, jkScanLogicPlanFilesetDelimited, pNode->filesetDelimited); } if (TSDB_CODE_SUCCESS == code) { - code = tjsonAddBoolToObject(pJson, jkScanLogicPlanFilesetDelimited, pNode->seqBlocksSort); + code = tjsonAddBoolToObject(pJson, jkScanLogicPlanFilesetDelimited, pNode->paraTablesSort); } return code; } @@ -784,7 +784,7 @@ static int32_t jsonToLogicScanNode(const SJson* pJson, void* pObj) { code = tjsonGetBoolValue(pJson, jkScanLogicPlanFilesetDelimited, &pNode->filesetDelimited); } if (TSDB_CODE_SUCCESS == code) { - code = tjsonGetBoolValue(pJson, jkScanLogicPlanFilesetDelimited, &pNode->seqBlocksSort); + code = tjsonGetBoolValue(pJson, jkScanLogicPlanFilesetDelimited, &pNode->paraTablesSort); } return code; } @@ -1879,7 +1879,7 @@ static const char* jkTableScanPhysiPlanAssignBlockUid = "AssignBlockUid"; static const char* jkTableScanPhysiPlanIgnoreUpdate = "IgnoreUpdate"; static const char* jkTableScanPhysiPlanFilesetDelimited = "FilesetDelimited"; static const char* jkTableScanPhysiPlanNeedCountEmptyTable = "NeedCountEmptyTable"; -static const char* jkTableScanPhysiPlanSeqBlocksSort = "SeqBlocksSort"; +static const char* jkTableScanPhysiPlanparaTablesSort = "paraTablesSort"; static int32_t physiTableScanNodeToJson(const void* pObj, SJson* pJson) { const STableScanPhysiNode* pNode = (const STableScanPhysiNode*)pObj; @@ -1955,7 +1955,7 @@ static int32_t physiTableScanNodeToJson(const void* pObj, SJson* pJson) { code = tjsonAddBoolToObject(pJson, jkTableScanPhysiPlanNeedCountEmptyTable, pNode->needCountEmptyTable); } if (TSDB_CODE_SUCCESS == code) { - code = tjsonAddBoolToObject(pJson, jkTableScanPhysiPlanSeqBlocksSort, pNode->seqBlocksSort); + code = tjsonAddBoolToObject(pJson, jkTableScanPhysiPlanparaTablesSort, pNode->paraTablesSort); } return code; } @@ -2034,7 +2034,7 @@ static int32_t jsonToPhysiTableScanNode(const SJson* pJson, void* pObj) { code = tjsonGetBoolValue(pJson, jkTableScanPhysiPlanNeedCountEmptyTable, &pNode->needCountEmptyTable); } if (TSDB_CODE_SUCCESS == code) { - code = tjsonGetBoolValue(pJson, jkTableScanPhysiPlanSeqBlocksSort, &pNode->seqBlocksSort); + code = tjsonGetBoolValue(pJson, jkTableScanPhysiPlanparaTablesSort, &pNode->paraTablesSort); } return code; } diff --git a/source/libs/nodes/src/nodesMsgFuncs.c b/source/libs/nodes/src/nodesMsgFuncs.c index 963e2af67b..86ebac2cbc 100644 --- a/source/libs/nodes/src/nodesMsgFuncs.c +++ b/source/libs/nodes/src/nodesMsgFuncs.c @@ -2186,7 +2186,7 @@ static int32_t physiTableScanNodeInlineToMsg(const void* pObj, STlvEncoder* pEnc code = tlvEncodeValueBool(pEncoder, pNode->needCountEmptyTable); } if (TSDB_CODE_SUCCESS == code) { - code = tlvEncodeValueBool(pEncoder, pNode->seqBlocksSort); + code = tlvEncodeValueBool(pEncoder, pNode->paraTablesSort); } return code; } @@ -2273,7 +2273,7 @@ static int32_t msgToPhysiTableScanNodeInline(STlvDecoder* pDecoder, void* pObj) code = tlvDecodeValueBool(pDecoder, &pNode->needCountEmptyTable); } if (TSDB_CODE_SUCCESS == code) { - code = tlvDecodeValueBool(pDecoder, &pNode->seqBlocksSort); + code = tlvDecodeValueBool(pDecoder, &pNode->paraTablesSort); } return code; } diff --git a/source/libs/parser/src/parAstCreater.c b/source/libs/parser/src/parAstCreater.c index ade937f552..be8c7bcd89 100644 --- a/source/libs/parser/src/parAstCreater.c +++ b/source/libs/parser/src/parAstCreater.c @@ -401,8 +401,8 @@ bool addHintNodeToList(SAstCreateContext* pCxt, SNodeList** ppHintList, EHintOpt case HINT_PARTITION_FIRST: if (paramNum > 0 || hasHint(*ppHintList, HINT_SORT_FOR_GROUP)) return true; break; - case HINT_SEQ_BLOCKS_SORT: - if (paramNum > 0 || hasHint(*ppHintList, HINT_SEQ_BLOCKS_SORT)) return true; + case HINT_PARA_TABLES_SORT: + if (paramNum > 0 || hasHint(*ppHintList, HINT_PARA_TABLES_SORT)) return true; break; default: return true; @@ -482,13 +482,13 @@ SNodeList* createHintNodeList(SAstCreateContext* pCxt, const SToken* pLiteral) { } opt = HINT_PARTITION_FIRST; break; - case TK_SEQ_BLOCKS_SORT: + case TK_PARA_TABLES_SORT: lastComma = false; if (0 != opt || inParamList) { quit = true; break; } - opt = HINT_SEQ_BLOCKS_SORT; + opt = HINT_PARA_TABLES_SORT; break; case TK_NK_LP: lastComma = false; diff --git a/source/libs/parser/src/parTokenizer.c b/source/libs/parser/src/parTokenizer.c index cebf82c5f3..94ba6af205 100644 --- a/source/libs/parser/src/parTokenizer.c +++ b/source/libs/parser/src/parTokenizer.c @@ -169,6 +169,7 @@ static SKeyword keywordTable[] = { {"OUTPUTTYPE", TK_OUTPUTTYPE}, {"PAGES", TK_PAGES}, {"PAGESIZE", TK_PAGESIZE}, + {"PARA_TABLES_SORT", TK_PARA_TABLES_SORT}, {"PARTITION", TK_PARTITION}, {"PARTITION_FIRST", TK_PARTITION_FIRST}, {"PASS", TK_PASS}, @@ -199,7 +200,6 @@ static SKeyword keywordTable[] = { {"SCHEMALESS", TK_SCHEMALESS}, {"SCORES", TK_SCORES}, {"SELECT", TK_SELECT}, - {"SEQ_BLOCKS_SORT", TK_SEQ_BLOCKS_SORT}, {"SERVER_STATUS", TK_SERVER_STATUS}, {"SERVER_VERSION", TK_SERVER_VERSION}, {"SESSION", TK_SESSION}, diff --git a/source/libs/planner/inc/planInt.h b/source/libs/planner/inc/planInt.h index 34a8a4d70f..fcccdcf23e 100644 --- a/source/libs/planner/inc/planInt.h +++ b/source/libs/planner/inc/planInt.h @@ -47,7 +47,7 @@ int32_t validateQueryPlan(SPlanContext* pCxt, SQueryPlan* pPlan); bool getBatchScanOptionFromHint(SNodeList* pList); bool getSortForGroupOptHint(SNodeList* pList); -bool getSeqBlocksSortOptHint(SNodeList* pList); +bool getparaTablesSortOptHint(SNodeList* pList); bool getOptHint(SNodeList* pList, EHintOption hint); SLogicNode* getLogicNodeRootNode(SLogicNode* pCurr); int32_t collectTableAliasFromNodes(SNode* pNode, SSHashObj** ppRes); diff --git a/source/libs/planner/src/planLogicCreater.c b/source/libs/planner/src/planLogicCreater.c index ed7b266198..546206ad78 100644 --- a/source/libs/planner/src/planLogicCreater.c +++ b/source/libs/planner/src/planLogicCreater.c @@ -501,7 +501,7 @@ static int32_t createScanLogicNode(SLogicPlanContext* pCxt, SSelectStmt* pSelect } else { nodesDestroyNode((SNode*)pScan); } - pScan->seqBlocksSort = getSeqBlocksSortOptHint(pSelect->pHint); + pScan->paraTablesSort = getparaTablesSortOptHint(pSelect->pHint); pCxt->hasScan = true; return code; diff --git a/source/libs/planner/src/planPhysiCreater.c b/source/libs/planner/src/planPhysiCreater.c index c560a29234..5b7f3c7b30 100644 --- a/source/libs/planner/src/planPhysiCreater.c +++ b/source/libs/planner/src/planPhysiCreater.c @@ -651,7 +651,7 @@ static int32_t createTableScanPhysiNode(SPhysiPlanContext* pCxt, SSubplan* pSubp pTableScan->assignBlockUid = pCxt->pPlanCxt->rSmaQuery ? true : false; pTableScan->filesetDelimited = pScanLogicNode->filesetDelimited; pTableScan->needCountEmptyTable = pScanLogicNode->isCountByTag; - pTableScan->seqBlocksSort = pScanLogicNode->seqBlocksSort; + pTableScan->paraTablesSort = pScanLogicNode->paraTablesSort; int32_t code = createScanPhysiNodeFinalize(pCxt, pSubplan, pScanLogicNode, (SScanPhysiNode*)pTableScan, pPhyNode); if (TSDB_CODE_SUCCESS == code) { diff --git a/source/libs/planner/src/planUtil.c b/source/libs/planner/src/planUtil.c index 266eefc930..7ccc1432e1 100644 --- a/source/libs/planner/src/planUtil.c +++ b/source/libs/planner/src/planUtil.c @@ -455,12 +455,12 @@ bool getOptHint(SNodeList* pList, EHintOption hint) { return false; } -bool getSeqBlocksSortOptHint(SNodeList* pList) { +bool getparaTablesSortOptHint(SNodeList* pList) { if (!pList) return false; SNode* pNode; FOREACH(pNode, pList) { SHintNode* pHint = (SHintNode*)pNode; - if (pHint->option == HINT_SEQ_BLOCKS_SORT) { + if (pHint->option == HINT_PARA_TABLES_SORT) { return true; } } From ba339077765381d582596097cd36a14f5ea0df53 Mon Sep 17 00:00:00 2001 From: shenglian zhou Date: Fri, 2 Feb 2024 15:00:09 +0800 Subject: [PATCH 22/41] fix: remove pages block using disk --- source/libs/executor/inc/executorInt.h | 2 +- source/libs/executor/src/scanoperator.c | 91 ++----------------------- 2 files changed, 7 insertions(+), 86 deletions(-) diff --git a/source/libs/executor/inc/executorInt.h b/source/libs/executor/inc/executorInt.h index 6418654ed5..dab0499aad 100644 --- a/source/libs/executor/inc/executorInt.h +++ b/source/libs/executor/inc/executorInt.h @@ -299,8 +299,8 @@ typedef struct STmsSubTableInput { SArray* aBlockPages; SSDataBlock* pPageBlock; - int32_t pageIdx; + int32_t rowIdx; int64_t* aTs; } STmsSubTableInput; diff --git a/source/libs/executor/src/scanoperator.c b/source/libs/executor/src/scanoperator.c index 55cad4236e..5c490b9d3b 100644 --- a/source/libs/executor/src/scanoperator.c +++ b/source/libs/executor/src/scanoperator.c @@ -3431,46 +3431,6 @@ static int32_t fetchNextSubTableBlockFromReader(SOperatorInfo* pOperator, STmsSu return TSDB_CODE_SUCCESS; } -static int32_t saveSubTableBlock(const STableMergeScanInfo* pInfo, STmsSubTablesMergeInfo* pSubTblsInfo, - STmsSubTableInput* pInput) { - taosArrayClear(pInput->aBlockPages); - int32_t start = 0; - SSDataBlock* pDataBlock = pInput->pReaderBlock; - while (start < pDataBlock->info.rows) { - int32_t stop = 0; - blockDataSplitRows(pDataBlock, pDataBlock->info.hasVarCol, start, &stop, pInfo->bufPageSize); - SSDataBlock* p = blockDataExtractBlock(pDataBlock, start, stop - start + 1); - - int32_t pageId = -1; - void* pPage = getNewBufPage(pSubTblsInfo->pBlocksBuf, &pageId); - - taosArrayPush(pInput->aBlockPages, &pageId); - - int32_t size = blockDataGetSize(p) + sizeof(int32_t) + taosArrayGetSize(p->pDataBlock) * sizeof(int32_t); - ASSERT(size <= getBufPageSize(pSubTblsInfo->pBlocksBuf)); - - blockDataToBuf(pPage, p); - - setBufPageDirty(pPage, true); - releaseBufPage(pSubTblsInfo->pBlocksBuf, pPage); - - blockDataDestroy(p); - start = stop + 1; - } - - return TSDB_CODE_SUCCESS; -} - -static int32_t fetchNextSubTableBlockFromPage(STmsSubTablesMergeInfo* pSubTblsInfo, STmsSubTableInput* pInput, int32_t pageIndex) { - int32_t pageId = *(int32_t*)taosArrayGet(pInput->aBlockPages, pageIndex); - void* page = getBufPage(pSubTblsInfo->pBlocksBuf, pageId); - - blockDataFromBuf(pInput->pPageBlock, page); - - releaseBufPage(pSubTblsInfo->pBlocksBuf, page); - return TSDB_CODE_SUCCESS; -} - static void setGroupStartEndIndex(STableMergeScanInfo* pInfo) { pInfo->bGroupProcessed = false; @@ -3491,11 +3451,7 @@ static int32_t openSubTablesMergeSort(STmsSubTablesMergeInfo* pSubTblsInfo) { if (pInput->rowIdx == -1) { continue; } - if (pInput->type == SUB_TABLE_EXT_PAGES) { - pInput->pageIdx = 0; - fetchNextSubTableBlockFromPage(pSubTblsInfo, pInput, pInput->pageIdx); - pInput->rowIdx = 0; - } else { + if (pInput->type == SUB_TABLE_MEM_BLOCK) { pInput->rowIdx = 0; pInput->pageIdx = -1; } @@ -3564,11 +3520,6 @@ static int32_t initSubTableInputs(SOperatorInfo* pOperator, STableMergeScanInfo* pInput->rowIdx = 0; pInput->pageIdx = -1; } - if (i + 1 > pSubTblsInfo->numTableBlocksInMem && hasNext) { - pInput->aBlockPages = taosArrayInit(32, sizeof(int32_t)); - saveSubTableBlock(pInfo, pSubTblsInfo, pInput); - pInput->type = SUB_TABLE_EXT_PAGES; - } } return TSDB_CODE_SUCCESS; } @@ -3588,29 +3539,6 @@ static int32_t adjustSubTableFromMemBlock(SOperatorInfo* pOperatorInfo, STmsSubT return TSDB_CODE_SUCCESS; } -static int32_t adjustSubTableFromExtPages(SOperatorInfo* pOperatorInfo, STmsSubTablesMergeInfo* pSubTblsInfo) { - STableMergeScanInfo* pInfo = pOperatorInfo->info; - STmsSubTableInput* pInput = pSubTblsInfo->aInputs + tMergeTreeGetChosenIndex(pSubTblsInfo->pTree); - if (pInput->pageIdx >= taosArrayGetSize(pInput->aBlockPages)) { - bool hasNext = true; - fetchNextSubTableBlockFromReader(pOperatorInfo, pInput, &hasNext); - if (!hasNext) { - pInput->rowIdx = -1; - ++pSubTblsInfo->numSubTablesCompleted; - } else { - saveSubTableBlock(pInfo, pSubTblsInfo, pInput); - pInput->pageIdx = 0; - fetchNextSubTableBlockFromPage(pSubTblsInfo, pInput, pInput->pageIdx); - pInput->rowIdx = 0; - - } - } else { - fetchNextSubTableBlockFromPage(pSubTblsInfo, pInput, pInput->pageIdx); - pInput->rowIdx = 0; - } - return TSDB_CODE_SUCCESS; -} - static int32_t adjustSubTableForNextRow(SOperatorInfo* pOperatorInfo, STmsSubTablesMergeInfo* pSubTblsInfo) { STableMergeScanInfo* pInfo = pOperatorInfo->info; STmsSubTableInput* pInput = pSubTblsInfo->aInputs + tMergeTreeGetChosenIndex(pSubTblsInfo->pTree); @@ -3621,9 +3549,6 @@ static int32_t adjustSubTableForNextRow(SOperatorInfo* pOperatorInfo, STmsSubTab } else if (pInput->rowIdx == pInputBlock->info.rows -1 ) { if (pInput->type == SUB_TABLE_MEM_BLOCK) { adjustSubTableFromMemBlock(pOperatorInfo, pSubTblsInfo); - } else if (pInput->type == SUB_TABLE_EXT_PAGES) { - ++pInput->pageIdx; - adjustSubTableFromExtPages(pOperatorInfo, pSubTblsInfo); } if (pInput->rowIdx != -1) { SColumnInfoData* col = taosArrayGet(pInputBlock->pDataBlock, pSubTblsInfo->pOrderInfo->slotId); @@ -3732,7 +3657,7 @@ static int32_t stopSubTablesTableMergeScan(STableMergeScanInfo* pInfo) { return TSDB_CODE_SUCCESS; } -SSDataBlock* doTableMergeScanSeqBlocksInMem(SOperatorInfo* pOperator) { +SSDataBlock* doTableMergeScanParaSubTables(SOperatorInfo* pOperator) { if (pOperator->status == OP_EXEC_DONE) { return NULL; } @@ -4320,14 +4245,10 @@ SOperatorInfo* createTableMergeScanOperatorInfo(STableScanPhysiNode* pTableScanN pInfo, pTaskInfo); pOperator->exprSupp.numOfExprs = numOfCols; - pOperator->fpSet = - createOperatorFpSet(optrDummyOpenFn, - pTableScanNode->paraTablesSort ? doTableMergeScanSeqBlocksInMem : doTableMergeScan, - NULL, - destroyTableMergeScanOperatorInfo, - optrDefaultBufFn, - getTableMergeScanExplainExecInfo, - optrDefaultGetNextExtFn, NULL); + pOperator->fpSet = createOperatorFpSet( + optrDummyOpenFn, pTableScanNode->paraTablesSort ? doTableMergeScanParaSubTables : doTableMergeScan, NULL, + destroyTableMergeScanOperatorInfo, optrDefaultBufFn, getTableMergeScanExplainExecInfo, optrDefaultGetNextExtFn, + NULL); pOperator->cost.openCost = 0; return pOperator; From d236f7c9cc979f78ce090b9757784602b8180c3a Mon Sep 17 00:00:00 2001 From: shenglian zhou Date: Sun, 4 Feb 2024 09:20:28 +0800 Subject: [PATCH 23/41] enhance: error processing --- source/libs/executor/src/scanoperator.c | 24 +++++++++++++++++------- 1 file changed, 17 insertions(+), 7 deletions(-) diff --git a/source/libs/executor/src/scanoperator.c b/source/libs/executor/src/scanoperator.c index 5c490b9d3b..dc9fceee84 100644 --- a/source/libs/executor/src/scanoperator.c +++ b/source/libs/executor/src/scanoperator.c @@ -3464,21 +3464,31 @@ static int32_t openSubTablesMergeSort(STmsSubTablesMergeInfo* pSubTblsInfo) { } static int32_t initSubTablesMergeInfo(STableMergeScanInfo* pInfo) { - STmsSubTablesMergeInfo* pSubTblsInfo = taosMemoryCalloc(1, sizeof(STmsSubTablesMergeInfo)); - pInfo->pSubTablesMergeInfo = pSubTblsInfo; - setGroupStartEndIndex(pInfo); - + STmsSubTablesMergeInfo* pSubTblsInfo = taosMemoryCalloc(1, sizeof(STmsSubTablesMergeInfo)); + if (pSubTblsInfo == NULL) { + return TSDB_CODE_OUT_OF_MEMORY; + } pSubTblsInfo->pOrderInfo = taosArrayGet(pInfo->pSortInfo, 0); pSubTblsInfo->numSubTables = pInfo->tableEndIndex - pInfo->tableStartIndex + 1; pSubTblsInfo->aInputs = taosMemoryCalloc(pSubTblsInfo->numSubTables, sizeof(STmsSubTableInput)); - + if (pSubTblsInfo->aInputs == NULL) { + taosMemoryFree(pSubTblsInfo); + return TSDB_CODE_OUT_OF_MEMORY; + } int32_t bufPageSize = pInfo->bufPageSize; int32_t inMemSize = (pSubTblsInfo->numSubTables - pSubTblsInfo->numTableBlocksInMem) * bufPageSize; - createDiskbasedBuf(&pSubTblsInfo->pBlocksBuf, pInfo->bufPageSize, inMemSize, "blocksExternalBuf", tsTempDir); - + int32_t code = + createDiskbasedBuf(&pSubTblsInfo->pBlocksBuf, pInfo->bufPageSize, inMemSize, "blocksExternalBuf", tsTempDir); + if (code != TSDB_CODE_SUCCESS) { + taosMemoryFree(pSubTblsInfo->aInputs); + taosMemoryFree(pSubTblsInfo); + return code; + } pSubTblsInfo->numTableBlocksInMem = pSubTblsInfo->numSubTables; pSubTblsInfo->numInMemReaders = pSubTblsInfo->numSubTables; + + pInfo->pSubTablesMergeInfo = pSubTblsInfo; return TSDB_CODE_SUCCESS; } From 03bc2d7c6ddb1272c8067747c542bec547b77847 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 20 Feb 2024 10:27:02 +0800 Subject: [PATCH 24/41] test(stream): add unit test cases. --- source/dnode/mnode/impl/inc/mndStream.h | 1 + source/dnode/mnode/impl/src/mndStreamHb.c | 4 +- source/dnode/mnode/impl/src/mndStreamTrans.c | 34 ++++-- source/dnode/mnode/impl/src/mndStreamUtil.c | 2 + .../dnode/mnode/impl/test/stream/stream.cpp | 114 +++++++++++++++--- 5 files changed, 121 insertions(+), 34 deletions(-) diff --git a/source/dnode/mnode/impl/inc/mndStream.h b/source/dnode/mnode/impl/inc/mndStream.h index 1084340dc2..aed49809dd 100644 --- a/source/dnode/mnode/impl/inc/mndStream.h +++ b/source/dnode/mnode/impl/inc/mndStream.h @@ -119,6 +119,7 @@ int32_t mndStreamSetPauseAction(SMnode *pMnode, STrans *pTrans, SStreamObj * int32_t mndStreamSetDropAction(SMnode *pMnode, STrans *pTrans, SStreamObj *pStream); int32_t mndStreamSetDropActionFromList(SMnode *pMnode, STrans *pTrans, SArray *pList); int32_t mndStreamSetResetTaskAction(SMnode *pMnode, STrans *pTrans, SStreamObj *pStream); +int32_t mndCreateStreamResetStatusTrans(SMnode *pMnode, SStreamObj *pStream); SStreamTaskIter *createStreamTaskIter(SStreamObj *pStream); void destroyStreamTaskIter(SStreamTaskIter *pIter); diff --git a/source/dnode/mnode/impl/src/mndStreamHb.c b/source/dnode/mnode/impl/src/mndStreamHb.c index 5de442951c..3f3a6edf35 100644 --- a/source/dnode/mnode/impl/src/mndStreamHb.c +++ b/source/dnode/mnode/impl/src/mndStreamHb.c @@ -65,7 +65,7 @@ static void addIntoCheckpointList(SArray* pList, const SFailedCheckpointInfo* pI taosArrayPush(pList, pInfo); } -int32_t createStreamResetStatusTrans(SMnode *pMnode, SStreamObj *pStream) { +int32_t mndCreateStreamResetStatusTrans(SMnode *pMnode, SStreamObj *pStream) { STrans *pTrans = doCreateTrans(pMnode, pStream, NULL, MND_STREAM_TASK_RESET_NAME, " reset from failed checkpoint"); if (pTrans == NULL) { return terrno; @@ -115,7 +115,7 @@ static int32_t mndResetStatusFromCheckpoint(SMnode *pMnode, int64_t streamId, in } else { mDebug("stream:%s (0x%" PRIx64 ") reset checkpoint procedure, transId:%d, create reset trans", pStream->name, pStream->uid, transId); - code = createStreamResetStatusTrans(pMnode, pStream); + code = mndCreateStreamResetStatusTrans(pMnode, pStream); } } diff --git a/source/dnode/mnode/impl/src/mndStreamTrans.c b/source/dnode/mnode/impl/src/mndStreamTrans.c index 5bfd3933b5..2241d93465 100644 --- a/source/dnode/mnode/impl/src/mndStreamTrans.c +++ b/source/dnode/mnode/impl/src/mndStreamTrans.c @@ -261,22 +261,30 @@ int32_t setTransAction(STrans *pTrans, void *pCont, int32_t contLen, int32_t msg return mndTransAppendRedoAction(pTrans, &action); } +static bool identicalName(const char* pDb, const char* pParam, int32_t len) { + return (strlen(pDb) == len) && (strncmp(pDb, pParam, len) == 0); +} + int32_t doKillCheckpointTrans(SMnode *pMnode, const char *pDBName, size_t len) { - // data in the hash table will be removed automatically, no need to remove it here. - SStreamTransInfo *pTransInfo = taosHashGet(execInfo.transMgmt.pDBTrans, pDBName, len); - if (pTransInfo == NULL) { - return TSDB_CODE_SUCCESS; - } + void *pIter = NULL; - // not checkpoint trans, ignore - if (strcmp(pTransInfo->name, MND_STREAM_CHECKPOINT_NAME) != 0) { - mDebug("not checkpoint trans, not kill it, name:%s, transId:%d", pTransInfo->name, pTransInfo->transId); - return TSDB_CODE_SUCCESS; - } + while ((pIter = taosHashIterate(execInfo.transMgmt.pDBTrans, pIter)) != NULL) { + SStreamTransInfo *pTransInfo = (SStreamTransInfo *)pIter; + if (strcmp(pTransInfo->name, MND_STREAM_CHECKPOINT_NAME) != 0) { + continue; + } - char *pDupDBName = strndup(pDBName, len); - mndKillTransImpl(pMnode, pTransInfo->transId, pDupDBName); - taosMemoryFree(pDupDBName); + SStreamObj *pStream = mndGetStreamObj(pMnode, pTransInfo->streamId); + if (pStream != NULL) { + if (identicalName(pStream->sourceDb, pDBName, len)) { + mndKillTransImpl(pMnode, pTransInfo->transId, pStream->sourceDb); + } else if (identicalName(pStream->targetDb, pDBName, len)) { + mndKillTransImpl(pMnode, pTransInfo->transId, pStream->targetDb); + } + + mndReleaseStream(pMnode, pStream); + } + } return TSDB_CODE_SUCCESS; } diff --git a/source/dnode/mnode/impl/src/mndStreamUtil.c b/source/dnode/mnode/impl/src/mndStreamUtil.c index 3cabce2201..1ae85a2cc6 100644 --- a/source/dnode/mnode/impl/src/mndStreamUtil.c +++ b/source/dnode/mnode/impl/src/mndStreamUtil.c @@ -231,6 +231,8 @@ static int32_t doSetResumeAction(STrans *pTrans, SMnode *pMnode, SStreamTask *pT taosMemoryFree(pReq); return -1; } + + mDebug("set the resume action for trans:%d", pTrans->id); return 0; } diff --git a/source/dnode/mnode/impl/test/stream/stream.cpp b/source/dnode/mnode/impl/test/stream/stream.cpp index e3bfdb5d6c..369053c7bb 100644 --- a/source/dnode/mnode/impl/test/stream/stream.cpp +++ b/source/dnode/mnode/impl/test/stream/stream.cpp @@ -29,6 +29,9 @@ #include "../../inc/mndStream.h" namespace { + +static int64_t defStreamId = 999; + SRpcMsg buildHbReq() { SStreamHbMsg msg = {0}; msg.vgId = 1; @@ -40,7 +43,7 @@ SRpcMsg buildHbReq() { entry.nodeId = i + 1; entry.stage = 1; entry.id.taskId = i + 1; - entry.id.streamId = 999; + entry.id.streamId = defStreamId; if (i == 0) { entry.stage = 4; @@ -57,7 +60,7 @@ SRpcMsg buildHbReq() { entry.stage = 1; entry.id.taskId = 5; - entry.id.streamId = 999; + entry.id.streamId = defStreamId; entry.checkpointId = 1; entry.checkpointFailed = true; @@ -118,15 +121,16 @@ void setTask(SStreamTask* pTask, int32_t nodeId, int64_t streamId, int32_t taskI taosHashPut(pExecNode->pTaskMap, &id, sizeof(id), &entry, sizeof(entry)); taosArrayPush(pExecNode->pTaskList, &id); } + void initStreamExecInfo() { SStreamExecInfo* pExecNode = &execInfo; SStreamTask task = {0}; - setTask(&task, 1, 999, 1); - setTask(&task, 1, 999, 2); - setTask(&task, 1, 999, 3); - setTask(&task, 1, 999, 4); - setTask(&task, 2, 999, 5); + setTask(&task, 1, defStreamId, 1); + setTask(&task, 1, defStreamId, 2); + setTask(&task, 1, defStreamId, 3); + setTask(&task, 1, defStreamId, 4); + setTask(&task, 2, defStreamId, 5); } void initNodeInfo() { @@ -138,38 +142,109 @@ void initNodeInfo() { } } // namespace +class StreamTest : public testing::Test { // 继承了 testing::Test + protected: + + static void SetUpTestSuite() { + mndInitExecInfo(); + initStreamExecInfo(); + initNodeInfo(); + + std::cout<<"setup env for streamTest suite"<(taosMemoryCalloc(1, sizeof(SMnode))); + {// init sdb + SSdbOpt opt = {0}; + opt.path = pMnode->path; + opt.pMnode = pMnode; + opt.pWal = pMnode->pWal; + + pMnode->pSdb = sdbInit(&opt); + } + + SVgroupChangeInfo info; + info.pUpdateNodeList = taosArrayInit(4, sizeof(SNodeUpdateInfo)); + info.pDBMap = taosHashInit(32, taosGetDefaultHashFunction(TSDB_DATA_TYPE_VARCHAR), true, HASH_NO_LOCK); + + const char* pDbName = "test_db_name"; + int32_t len = strlen(pDbName); + + taosHashPut(info.pDBMap, pDbName, len, NULL, 0); + + killAllCheckpointTrans(pMnode, &info); + + SStreamObj stream; + memset(&stream, 0, sizeof(SStreamObj)); + + stream.uid = defStreamId; + stream.lock = 0; + stream.tasks = taosArrayInit(1, POINTER_BYTES); + stream.pHTasksList = taosArrayInit(1, POINTER_BYTES); + + SArray* pLevel = taosArrayInit(1, POINTER_BYTES); + SStreamTask* pTask = static_cast(taosMemoryCalloc(1, sizeof(SStreamTask))); + pTask->id.streamId = defStreamId; + pTask->id.taskId = 1; + pTask->exec.qmsg = (char*)taosMemoryMalloc(1); + + taosArrayPush(pLevel, &pTask); + + taosArrayPush(stream.tasks, &pLevel); + mndCreateStreamResetStatusTrans(pMnode, &stream); + + tFreeStreamObj(&stream); + sdbCleanup(pMnode->pSdb); + taosMemoryFree(pMnode); + + taosArrayDestroy(info.pUpdateNodeList); + taosHashCleanup(info.pDBMap); +} + +TEST_F(StreamTest, plan_Test) { char* ast = "{\"NodeType\":\"101\",\"Name\":\"SelectStmt\",\"SelectStmt\":{\"Distinct\":false,\"Projections\":[{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"9\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"#expr_1\",\"UserAlias\":\"_wstart\",\"Name\":\"_wstart\",\"Id\":\"89\",\"Type\":\"3505\",\"UdfBufSize\":\"0\"}},{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"5\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"#expr_2\",\"UserAlias\":\"sum(voltage)\",\"Name\":\"sum\",\"Id\":\"1\",\"Type\":\"14\",\"Parameters\":[{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"4\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"4\"},\"AliasName\":\"voltage\",\"UserAlias\":\"voltage\",\"TableId\":\"6555383776122680534\",\"TableType\":\"1\",\"ColId\":\"3\",\"ProjId\":\"0\",\"ColType\":\"1\",\"DbName\":\"test\",\"TableName\":\"meters\",\"TableAlias\":\"meters\",\"ColName\":\"voltage\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"4\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"4\"},\"AliasName\":\"#expr_3\",\"UserAlias\":\"groupid\",\"Name\":\"_group_key\",\"Id\":\"96\",\"Type\":\"3754\",\"Parameters\":[{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"4\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"4\"},\"AliasName\":\"#expr_3\",\"UserAlias\":\"groupid\",\"TableId\":\"6555383776122680534\",\"TableType\":\"1\",\"ColId\":\"5\",\"ProjId\":\"0\",\"ColType\":\"2\",\"DbName\":\"test\",\"TableName\":\"meters\",\"TableAlias\":\"meters\",\"ColName\":\"groupid\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}}],\"UdfBufSize\":\"0\"}}],\"From\":{\"NodeType\":\"6\",\"Name\":\"RealTable\",\"RealTable\":{\"DataType\":{\"Type\":\"0\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"0\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"DbName\":\"test\",\"tableName\":\"meters\",\"tableAlias\":\"meters\",\"MetaSize\":\"475\",\"Meta\":{\"VgId\":\"0\",\"TableType\":\"1\",\"Uid\":\"6555383776122680534\",\"Suid\":\"6555383776122680534\",\"Sversion\":\"1\",\"Tversion\":\"1\",\"ComInfo\":{\"NumOfTags\":\"2\",\"Precision\":\"0\",\"NumOfColumns\":\"4\",\"RowSize\":\"20\"},\"ColSchemas\":[{\"Type\":\"9\",\"ColId\":\"1\",\"bytes\":\"8\",\"Name\":\"ts\"},{\"Type\":\"6\",\"ColId\":\"2\",\"bytes\":\"4\",\"Name\":\"current\"},{\"Type\":\"4\",\"ColId\":\"3\",\"bytes\":\"4\",\"Name\":\"voltage\"},{\"Type\":\"6\",\"ColId\":\"4\",\"bytes\":\"4\",\"Name\":\"phase\"},{\"Type\":\"4\",\"ColId\":\"5\",\"bytes\":\"4\",\"Name\":\"groupid\"},{\"Type\":\"8\",\"ColId\":\"6\",\"bytes\":\"26\",\"Name\":\"location\"}]},\"VgroupsInfoSize\":\"1340\",\"VgroupsInfo\":{\"Num\":\"2\",\"Vgroups\":[{\"VgId\":\"2\",\"HashBegin\":\"0\",\"HashEnd\":\"2147483646\",\"EpSet\":{\"InUse\":\"0\",\"NumOfEps\":\"1\",\"Eps\":[{\"Fqdn\":\"localhost\",\"Port\":\"6030\"}]},\"NumOfTable\":\"0\"},{\"VgId\":\"3\",\"HashBegin\":\"2147483647\",\"HashEnd\":\"4294967295\",\"EpSet\":{\"InUse\":\"0\",\"NumOfEps\":\"1\",\"Eps\":[{\"Fqdn\":\"localhost\",\"Port\":\"6030\"}]},\"NumOfTable\":\"0\"}]}}},\"PartitionBy\":[{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"4\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"4\"},\"AliasName\":\"groupid\",\"UserAlias\":\"groupid\",\"TableId\":\"6555383776122680534\",\"TableType\":\"1\",\"ColId\":\"5\",\"ProjId\":\"0\",\"ColType\":\"2\",\"DbName\":\"test\",\"TableName\":\"meters\",\"TableAlias\":\"meters\",\"ColName\":\"groupid\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}}],\"Window\":{\"NodeType\":\"14\",\"Name\":\"IntervalWindow\",\"IntervalWindow\":{\"Interval\":{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"5\",\"Precision\":\"0\",\"Scale\":\"115\",\"Bytes\":\"8\"},\"AliasName\":\"c804c3a15ebe05b5baf40ad5ee12be1f\",\"UserAlias\":\"2s\",\"LiteralSize\":\"2\",\"Literal\":\"2s\",\"Duration\":true,\"Translate\":true,\"NotReserved\":false,\"IsNull\":false,\"Unit\":\"115\",\"Datum\":\"2000\"}},\"TsPk\":{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"9\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"ts\",\"UserAlias\":\"ts\",\"TableId\":\"6555383776122680534\",\"TableType\":\"1\",\"ColId\":\"1\",\"ProjId\":\"0\",\"ColType\":\"1\",\"DbName\":\"test\",\"TableName\":\"meters\",\"TableAlias\":\"meters\",\"ColName\":\"ts\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}}}},\"StmtName\":\"0x1580095ba\",\"HasAggFuncs\":true}}"; // char* ast = "{\"NodeType\":\"101\",\"Name\":\"SelectStmt\",\"SelectStmt\":{\"Distinct\":false,\"Projections\":[{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"9\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"#expr_1\",\"UserAlias\":\"wstart\",\"Name\":\"_wstart\",\"Id\":\"89\",\"Type\":\"3505\",\"UdfBufSize\":\"0\"}},{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"2\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"#expr_2\",\"UserAlias\":\"min(c1)\",\"Name\":\"min\",\"Id\":\"2\",\"Type\":\"8\",\"Parameters\":[{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"2\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"c1\",\"UserAlias\":\"c1\",\"TableId\":\"5129202035162885657\",\"TableType\":\"1\",\"ColId\":\"2\",\"ProjId\":\"0\",\"ColType\":\"1\",\"DbName\":\"test\",\"TableName\":\"at_once_interval_ext_stb\",\"TableAlias\":\"at_once_interval_ext_stb\",\"ColName\":\"c1\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"3\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"2\"},\"AliasName\":\"#expr_3\",\"UserAlias\":\"max(c2)\",\"Name\":\"max\",\"Id\":\"3\",\"Type\":\"7\",\"Parameters\":[{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"3\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"2\"},\"AliasName\":\"c2\",\"UserAlias\":\"c2\",\"TableId\":\"5129202035162885657\",\"TableType\":\"1\",\"ColId\":\"3\",\"ProjId\":\"0\",\"ColType\":\"1\",\"DbName\":\"test\",\"TableName\":\"at_once_interval_ext_stb\",\"TableAlias\":\"at_once_interval_ext_stb\",\"ColName\":\"c2\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"4\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"4\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"Name\":\"cast\",\"Id\":\"77\",\"Type\":\"2000\",\"Parameters\":[{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"5\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"#expr_4\",\"UserAlias\":\"sum(c3)\",\"Name\":\"sum\",\"Id\":\"1\",\"Type\":\"14\",\"Parameters\":[{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"4\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"4\"},\"AliasName\":\"c3\",\"UserAlias\":\"c3\",\"TableId\":\"5129202035162885657\",\"TableType\":\"1\",\"ColId\":\"4\",\"ProjId\":\"0\",\"ColType\":\"1\",\"DbName\":\"test\",\"TableName\":\"at_once_interval_ext_stb\",\"TableAlias\":\"at_once_interval_ext_stb\",\"ColName\":\"c3\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"2\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":true,\"NotReserved\":true,\"IsNull\":false,\"Unit\":\"0\",\"Datum\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"5\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"#expr_5\",\"UserAlias\":\"first(c4)\",\"Name\":\"first\",\"Id\":\"33\",\"Type\":\"504\",\"Parameters\":[{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"5\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"c4\",\"UserAlias\":\"c4\",\"TableId\":\"5129202035162885657\",\"TableType\":\"1\",\"ColId\":\"5\",\"ProjId\":\"0\",\"ColType\":\"1\",\"DbName\":\"test\",\"TableName\":\"at_once_interval_ext_stb\",\"TableAlias\":\"at_once_interval_ext_stb\",\"ColName\":\"c4\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}},{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"9\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"ts\",\"UserAlias\":\"ts\",\"TableId\":\"5129202035162885657\",\"TableType\":\"1\",\"ColId\":\"1\",\"ProjId\":\"0\",\"ColType\":\"1\",\"DbName\":\"test\",\"TableName\":\"at_once_interval_ext_stb\",\"TableAlias\":\"at_once_interval_ext_stb\",\"ColName\":\"ts\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"11\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"#expr_6\",\"UserAlias\":\"last(c5)\",\"Name\":\"last\",\"Id\":\"36\",\"Type\":\"506\",\"Parameters\":[{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"11\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"c5\",\"UserAlias\":\"c5\",\"TableId\":\"5129202035162885657\",\"TableType\":\"1\",\"ColId\":\"6\",\"ProjId\":\"0\",\"ColType\":\"1\",\"DbName\":\"test\",\"TableName\":\"at_once_interval_ext_stb\",\"TableAlias\":\"at_once_interval_ext_stb\",\"ColName\":\"c5\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}},{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"9\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"ts\",\"UserAlias\":\"ts\",\"TableId\":\"5129202035162885657\",\"TableType\":\"1\",\"ColId\":\"1\",\"ProjId\":\"0\",\"ColType\":\"1\",\"DbName\":\"test\",\"TableName\":\"at_once_interval_ext_stb\",\"TableAlias\":\"at_once_interval_ext_stb\",\"ColName\":\"ts\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"12\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"2\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"Name\":\"cast\",\"Id\":\"77\",\"Type\":\"2000\",\"Parameters\":[{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"7\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"#expr_7\",\"UserAlias\":\"apercentile(c6, 50)\",\"Name\":\"apercentile\",\"Id\":\"12\",\"Type\":\"1\",\"Parameters\":[{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"12\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"2\"},\"AliasName\":\"c6\",\"UserAlias\":\"c6\",\"TableId\":\"5129202035162885657\",\"TableType\":\"1\",\"ColId\":\"7\",\"ProjId\":\"0\",\"ColType\":\"1\",\"DbName\":\"test\",\"TableName\":\"at_once_interval_ext_stb\",\"TableAlias\":\"at_once_interval_ext_stb\",\"ColName\":\"c6\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"5\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"c0c7c76d30bd3dcaefc96f40275bdc0a\",\"UserAlias\":\"50\",\"LiteralSize\":\"2\",\"Literal\":\"50\",\"Duration\":false,\"Translate\":true,\"NotReserved\":true,\"IsNull\":false,\"Unit\":\"0\",\"Datum\":\"50\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"2\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":true,\"NotReserved\":true,\"IsNull\":false,\"Unit\":\"0\",\"Datum\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"13\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"4\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"Name\":\"cast\",\"Id\":\"77\",\"Type\":\"2000\",\"Parameters\":[{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"7\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"#expr_8\",\"UserAlias\":\"avg(c7)\",\"Name\":\"avg\",\"Id\":\"8\",\"Type\":\"2\",\"Parameters\":[{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"13\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"4\"},\"AliasName\":\"c7\",\"UserAlias\":\"c7\",\"TableId\":\"5129202035162885657\",\"TableType\":\"1\",\"ColId\":\"8\",\"ProjId\":\"0\",\"ColType\":\"1\",\"DbName\":\"test\",\"TableName\":\"at_once_interval_ext_stb\",\"TableAlias\":\"at_once_interval_ext_stb\",\"ColName\":\"c7\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"2\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":true,\"NotReserved\":true,\"IsNull\":false,\"Unit\":\"0\",\"Datum\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"14\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"Name\":\"cast\",\"Id\":\"77\",\"Type\":\"2000\",\"Parameters\":[{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"5\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"#expr_9\",\"UserAlias\":\"count(c8)\",\"Name\":\"count\",\"Id\":\"0\",\"Type\":\"3\",\"Parameters\":[{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"14\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"c8\",\"UserAlias\":\"c8\",\"TableId\":\"5129202035162885657\",\"TableType\":\"1\",\"ColId\":\"9\",\"ProjId\":\"0\",\"ColType\":\"1\",\"DbName\":\"test\",\"TableName\":\"at_once_interval_ext_stb\",\"TableAlias\":\"at_once_interval_ext_stb\",\"ColName\":\"c8\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"2\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":true,\"NotReserved\":true,\"IsNull\":false,\"Unit\":\"0\",\"Datum\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"6\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"4\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"Name\":\"cast\",\"Id\":\"77\",\"Type\":\"2000\",\"Parameters\":[{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"7\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"#expr_10\",\"UserAlias\":\"spread(c1)\",\"Name\":\"spread\",\"Id\":\"17\",\"Type\":\"11\",\"Parameters\":[{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"2\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"c1\",\"UserAlias\":\"c1\",\"TableId\":\"5129202035162885657\",\"TableType\":\"1\",\"ColId\":\"2\",\"ProjId\":\"0\",\"ColType\":\"1\",\"DbName\":\"test\",\"TableName\":\"at_once_interval_ext_stb\",\"TableAlias\":\"at_once_interval_ext_stb\",\"ColName\":\"c1\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"2\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":true,\"NotReserved\":true,\"IsNull\":false,\"Unit\":\"0\",\"Datum\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"7\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"#expr_11\",\"UserAlias\":\"stddev(c2)\",\"Name\":\"stddev\",\"Id\":\"4\",\"Type\":\"12\",\"Parameters\":[{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"3\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"2\"},\"AliasName\":\"c2\",\"UserAlias\":\"c2\",\"TableId\":\"5129202035162885657\",\"TableType\":\"1\",\"ColId\":\"3\",\"ProjId\":\"0\",\"ColType\":\"1\",\"DbName\":\"test\",\"TableName\":\"at_once_interval_ext_stb\",\"TableAlias\":\"at_once_interval_ext_stb\",\"ColName\":\"c2\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"8\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"Name\":\"cast\",\"Id\":\"77\",\"Type\":\"2000\",\"Parameters\":[{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"5\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"#expr_12\",\"UserAlias\":\"hyperloglog(c11)\",\"Name\":\"hyperloglog\",\"Id\":\"43\",\"Type\":\"17\",\"Parameters\":[{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"8\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"c11\",\"UserAlias\":\"c11\",\"TableId\":\"5129202035162885657\",\"TableType\":\"1\",\"ColId\":\"12\",\"ProjId\":\"0\",\"ColType\":\"1\",\"DbName\":\"test\",\"TableName\":\"at_once_interval_ext_stb\",\"TableAlias\":\"at_once_interval_ext_stb\",\"ColName\":\"c11\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"2\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":true,\"NotReserved\":true,\"IsNull\":false,\"Unit\":\"0\",\"Datum\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"10\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"26\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"Name\":\"cast\",\"Id\":\"77\",\"Type\":\"2000\",\"Parameters\":[{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"5\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"#expr_13\",\"UserAlias\":\"timediff(1, 0, 1h)\",\"Name\":\"timediff\",\"Id\":\"81\",\"Type\":\"2501\",\"Parameters\":[{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"5\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"c4ca4238a0b923820dcc509a6f75849b\",\"UserAlias\":\"1\",\"LiteralSize\":\"1\",\"Literal\":\"1\",\"Duration\":false,\"Translate\":true,\"NotReserved\":false,\"IsNull\":false,\"Unit\":\"0\",\"Datum\":\"1\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"5\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"cfcd208495d565ef66e7dff9f98764da\",\"UserAlias\":\"0\",\"LiteralSize\":\"1\",\"Literal\":\"0\",\"Duration\":false,\"Translate\":true,\"NotReserved\":false,\"IsNull\":false,\"Unit\":\"0\",\"Datum\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"5\",\"Precision\":\"0\",\"Scale\":\"104\",\"Bytes\":\"8\"},\"AliasName\":\"7c68645d71b803bf0ba2f22519f73e08\",\"UserAlias\":\"1h\",\"LiteralSize\":\"2\",\"Literal\":\"1h\",\"Duration\":true,\"Translate\":true,\"NotReserved\":false,\"IsNull\":false,\"Unit\":\"104\",\"Datum\":\"3600000\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"2\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":true,\"NotReserved\":true,\"IsNull\":false,\"Unit\":\"0\",\"Datum\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"2\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":true,\"NotReserved\":true,\"IsNull\":false,\"Unit\":\"0\",\"Datum\":\"0\"}}],\"UdfBufSize\":\"0\"}},{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"1\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"Name\":\"cast\",\"Id\":\"77\",\"Type\":\"2000\",\"Parameters\":[{\"NodeType\":\"5\",\"Name\":\"Function\",\"Function\":{\"DataType\":{\"Type\":\"8\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"96\"},\"AliasName\":\"#expr_14\",\"UserAlias\":\"timezone()\",\"Name\":\"timezone\",\"Id\":\"84\",\"Type\":\"2503\",\"UdfBufSize\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"2\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":true,\"NotReserved\":true,\"IsNull\":false,\"Unit\":\"0\",\"Datum\":\"0\"}}],\"UdfBufSize\":\"0\"}}],\"From\":{\"NodeType\":\"6\",\"Name\":\"RealTable\",\"RealTable\":{\"DataType\":{\"Type\":\"0\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"0\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"DbName\":\"test\",\"tableName\":\"at_once_interval_ext_stb\",\"tableAlias\":\"at_once_interval_ext_stb\",\"MetaSize\":\"2008\",\"Meta\":{\"VgId\":\"0\",\"TableType\":\"1\",\"Uid\":\"5129202035162885657\",\"Suid\":\"5129202035162885657\",\"Sversion\":\"1\",\"Tversion\":\"1\",\"ComInfo\":{\"NumOfTags\":\"13\",\"Precision\":\"0\",\"NumOfColumns\":\"14\",\"RowSize\":\"85\"},\"ColSchemas\":[{\"Type\":\"9\",\"ColId\":\"1\",\"bytes\":\"8\",\"Name\":\"ts\"},{\"Type\":\"2\",\"ColId\":\"2\",\"bytes\":\"1\",\"Name\":\"c1\"},{\"Type\":\"3\",\"ColId\":\"3\",\"bytes\":\"2\",\"Name\":\"c2\"},{\"Type\":\"4\",\"ColId\":\"4\",\"bytes\":\"4\",\"Name\":\"c3\"},{\"Type\":\"5\",\"ColId\":\"5\",\"bytes\":\"8\",\"Name\":\"c4\"},{\"Type\":\"11\",\"ColId\":\"6\",\"bytes\":\"1\",\"Name\":\"c5\"},{\"Type\":\"12\",\"ColId\":\"7\",\"bytes\":\"2\",\"Name\":\"c6\"},{\"Type\":\"13\",\"ColId\":\"8\",\"bytes\":\"4\",\"Name\":\"c7\"},{\"Type\":\"14\",\"ColId\":\"9\",\"bytes\":\"8\",\"Name\":\"c8\"},{\"Type\":\"6\",\"ColId\":\"10\",\"bytes\":\"4\",\"Name\":\"c9\"},{\"Type\":\"7\",\"ColId\":\"11\",\"bytes\":\"8\",\"Name\":\"c10\"},{\"Type\":\"8\",\"ColId\":\"12\",\"bytes\":\"8\",\"Name\":\"c11\"},{\"Type\":\"10\",\"ColId\":\"13\",\"bytes\":\"26\",\"Name\":\"c12\"},{\"Type\":\"1\",\"ColId\":\"14\",\"bytes\":\"1\",\"Name\":\"c13\"},{\"Type\":\"2\",\"ColId\":\"15\",\"bytes\":\"1\",\"Name\":\"t1\"},{\"Type\":\"3\",\"ColId\":\"16\",\"bytes\":\"2\",\"Name\":\"t2\"},{\"Type\":\"4\",\"ColId\":\"17\",\"bytes\":\"4\",\"Name\":\"t3\"},{\"Type\":\"5\",\"ColId\":\"18\",\"bytes\":\"8\",\"Name\":\"t4\"},{\"Type\":\"11\",\"ColId\":\"19\",\"bytes\":\"1\",\"Name\":\"t5\"},{\"Type\":\"12\",\"ColId\":\"20\",\"bytes\":\"2\",\"Name\":\"t6\"},{\"Type\":\"13\",\"ColId\":\"21\",\"bytes\":\"4\",\"Name\":\"t7\"},{\"Type\":\"14\",\"ColId\":\"22\",\"bytes\":\"8\",\"Name\":\"t8\"},{\"Type\":\"6\",\"ColId\":\"23\",\"bytes\":\"4\",\"Name\":\"t9\"},{\"Type\":\"7\",\"ColId\":\"24\",\"bytes\":\"8\",\"Name\":\"t10\"},{\"Type\":\"8\",\"ColId\":\"25\",\"bytes\":\"8\",\"Name\":\"t11\"},{\"Type\":\"10\",\"ColId\":\"26\",\"bytes\":\"26\",\"Name\":\"t12\"},{\"Type\":\"1\",\"ColId\":\"27\",\"bytes\":\"1\",\"Name\":\"t13\"}]},\"VgroupsInfoSize\":\"1340\",\"VgroupsInfo\":{\"Num\":\"2\",\"Vgroups\":[{\"VgId\":\"14\",\"HashBegin\":\"0\",\"HashEnd\":\"2147483646\",\"EpSet\":{\"InUse\":\"0\",\"NumOfEps\":\"1\",\"Eps\":[{\"Fqdn\":\"localhost\",\"Port\":\"6030\"}]},\"NumOfTable\":\"0\"},{\"VgId\":\"15\",\"HashBegin\":\"2147483647\",\"HashEnd\":\"4294967295\",\"EpSet\":{\"InUse\":\"0\",\"NumOfEps\":\"1\",\"Eps\":[{\"Fqdn\":\"localhost\",\"Port\":\"6030\"}]},\"NumOfTable\":\"0\"}]}}},\"Tags\":[{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"0\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":false,\"NotReserved\":false,\"IsNull\":true,\"Unit\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"0\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":false,\"NotReserved\":false,\"IsNull\":true,\"Unit\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"0\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":false,\"NotReserved\":false,\"IsNull\":true,\"Unit\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"0\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":false,\"NotReserved\":false,\"IsNull\":true,\"Unit\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"0\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":false,\"NotReserved\":false,\"IsNull\":true,\"Unit\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"0\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":false,\"NotReserved\":false,\"IsNull\":true,\"Unit\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"0\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":false,\"NotReserved\":false,\"IsNull\":true,\"Unit\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"0\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":false,\"NotReserved\":false,\"IsNull\":true,\"Unit\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"0\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":false,\"NotReserved\":false,\"IsNull\":true,\"Unit\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"0\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":false,\"NotReserved\":false,\"IsNull\":true,\"Unit\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"0\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":false,\"NotReserved\":false,\"IsNull\":true,\"Unit\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"0\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":false,\"NotReserved\":false,\"IsNull\":true,\"Unit\":\"0\"}},{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"0\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"1\"},\"AliasName\":\"\",\"UserAlias\":\"\",\"LiteralSize\":\"0\",\"Duration\":false,\"Translate\":false,\"NotReserved\":false,\"IsNull\":true,\"Unit\":\"0\"}}],\"Window\":{\"NodeType\":\"14\",\"Name\":\"IntervalWindow\",\"IntervalWindow\":{\"Interval\":{\"NodeType\":\"2\",\"Name\":\"Value\",\"Value\":{\"DataType\":{\"Type\":\"5\",\"Precision\":\"0\",\"Scale\":\"115\",\"Bytes\":\"8\"},\"AliasName\":\"1fd7635317edfeca9054894ac9ef9b5e\",\"UserAlias\":\"14s\",\"LiteralSize\":\"3\",\"Literal\":\"14s\",\"Duration\":true,\"Translate\":true,\"NotReserved\":false,\"IsNull\":false,\"Unit\":\"115\",\"Datum\":\"14000\"}},\"TsPk\":{\"NodeType\":\"1\",\"Name\":\"Column\",\"Column\":{\"DataType\":{\"Type\":\"9\",\"Precision\":\"0\",\"Scale\":\"0\",\"Bytes\":\"8\"},\"AliasName\":\"ts\",\"UserAlias\":\"ts\",\"TableId\":\"5129202035162885657\",\"TableType\":\"1\",\"ColId\":\"1\",\"ProjId\":\"0\",\"ColType\":\"1\",\"DbName\":\"test\",\"TableName\":\"at_once_interval_ext_stb\",\"TableAlias\":\"at_once_interval_ext_stb\",\"ColName\":\"ts\",\"DataBlockId\":\"0\",\"SlotId\":\"0\"}}}},\"StmtName\":\"0x150146d14\",\"HasAggFuncs\":true}}"; SNode * pAst = NULL; SQueryPlan *pPlan = NULL; if (taosCreateLog("taoslog", 10, "/etc/taos", NULL, NULL, NULL, NULL, 1) != 0) { - // ignore create log failed, only print - printf(" WARING: Create failed:%s. configDir\n", strerror(errno)); + // ignore create log failed, only print + printf(" WARING: Create failed:%s. configDir\n", strerror(errno)); } if (nodesStringToNode(ast, &pAst) < 0) { - ASSERT(0); + ASSERT(0); } - SPlanContext cxt = { 0 }; + SPlanContext cxt = {0}; cxt.pAstRoot = pAst; cxt.topicQuery = false; cxt.streamQuery = true; @@ -181,10 +256,11 @@ TEST(testCase, plan_Test) { // using ast and param to build physical plan if (qCreateQueryPlan(&cxt, &pPlan, NULL) < 0) { - ASSERT(0); + ASSERT(0); } + if (pAst != NULL) nodesDestroyNode(pAst); - nodesDestroyNode((SNode *)pPlan); + nodesDestroyNode((SNode*)pPlan); } #pragma GCC diagnostic pop \ No newline at end of file From e8142dec0e5ffd2ce1c668a3852b375b784811ed Mon Sep 17 00:00:00 2001 From: slzhou Date: Tue, 20 Feb 2024 10:39:10 +0800 Subject: [PATCH 25/41] enhance: add test case --- tests/parallel_test/cases.task | 2 + tests/system-test/2-query/para_tms.py | 4936 +++++++++++++++++++ tests/system-test/2-query/para_tms2.py | 6209 ++++++++++++++++++++++++ 3 files changed, 11147 insertions(+) create mode 100755 tests/system-test/2-query/para_tms.py create mode 100755 tests/system-test/2-query/para_tms2.py diff --git a/tests/parallel_test/cases.task b/tests/parallel_test/cases.task index 91a0ac46e5..d2f0777408 100644 --- a/tests/parallel_test/cases.task +++ b/tests/parallel_test/cases.task @@ -61,6 +61,8 @@ fi ,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/hint.py -Q 2 ,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/hint.py -Q 3 ,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/hint.py -Q 4 +,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/para_tms.py +,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/para_tms2.py ,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/nestedQuery.py ,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/nestedQuery_str.py ,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/nestedQuery_math.py diff --git a/tests/system-test/2-query/para_tms.py b/tests/system-test/2-query/para_tms.py new file mode 100755 index 0000000000..fd62e43f38 --- /dev/null +++ b/tests/system-test/2-query/para_tms.py @@ -0,0 +1,4936 @@ +################################################################### +# Copyright (c) 2016 by TAOS Technologies, Inc. +# All rights reserved. +# +# This file is proprietary and confidential to TAOS Technologies. +# No part of this file may be reproduced, stored, transmitted, +# disclosed or used in any form or by any means other than as +# expressly provided by the written permission from Jianhui Tao +# +################################################################### + +# -*- coding: utf-8 -*- + +import random +import os +import time +import taos +import subprocess +from faker import Faker +from util.log import tdLog +from util.cases import tdCases +from util.sql import tdSql +from util.dnodes import tdDnodes +from util.dnodes import * + +class TDTestCase: + updatecfgDict = {'maxSQLLength':1048576,'debugFlag': 131 ,"cDebugFlag":131,"uDebugFlag":131 ,"rpcDebugFlag":131 , "tmrDebugFlag":131 , + "jniDebugFlag":131 ,"simDebugFlag":131,"dDebugFlag":131, "dDebugFlag":131,"vDebugFlag":131,"mDebugFlag":131,"qDebugFlag":131, + "wDebugFlag":131,"sDebugFlag":131,"tsdbDebugFlag":131,"tqDebugFlag":131 ,"fsDebugFlag":131 ,"fnDebugFlag":131} + + def init(self, conn, logSql, replicaVar=1): + self.replicaVar = int(replicaVar) + tdLog.debug("start to execute %s" % __file__) + tdSql.init(conn.cursor(), logSql) + + self.testcasePath = os.path.split(__file__)[0] + self.testcaseFilename = os.path.split(__file__)[-1] + os.system("rm -rf %s/%s.sql" % (self.testcasePath,self.testcaseFilename)) + + self.num = 10 + self.fornum = 5 + + self.db_nest = "stab" + self.dropandcreateDB_random("%s" %self.db_nest, 1) + + # regular column select + self.q_select= ['q_int', 'q_bigint' , 'q_bigint' , 'q_smallint' , 'q_tinyint' , 'q_bool' , 'q_binary' , 'q_nchar' ,'q_float' , 'q_double' ,'q_ts ', 'q_int_null ', 'q_bigint_null ' , 'q_bigint_null ' , 'q_smallint_null ' , 'q_tinyint_null ' , 'q_bool_null ' , 'q_binary_null ' , 'q_nchar_null ' ,'q_float_null ' , 'q_double_null ' ,'q_ts_null '] + + # tag column select + self.t_select= ['loc','t_int', 't_bigint' , 't_bigint' , 't_smallint' , 't_tinyint' , 't_bool' , 't_binary' , 't_nchar' ,'t_float' , 't_double' ,'t_ts '] + + # regular and tag column select + self.qt_select= self.q_select + self.t_select + + # distinct regular column select + self.dq_select= ['distinct q_int', 'distinct q_bigint' , 'distinct q_smallint' , 'distinct q_tinyint' , + 'distinct q_bool' , 'distinct q_binary' , 'distinct q_nchar' ,'distinct q_float' , 'distinct q_double' ,'distinct q_ts '] + + # distinct tag column select + self.dt_select= ['distinct loc', 'distinct t_int', 'distinct t_bigint' , 'distinct t_smallint' , 'distinct t_tinyint' , + 'distinct t_bool' , 'distinct t_binary' , 'distinct t_nchar' ,'distinct t_float' , 'distinct t_double' ,'distinct t_ts '] + + # distinct regular and tag column select + self.dqt_select= self.dq_select + self.dt_select + + # special column select + self.s_r_select= ['_c0', '_rowts' , '_C0' ] + self.s_s_select= ['tbname' , '_rowts' , '_c0', '_C0' ] + self.unionall_or_union= [ ' union ' , ' union all ' ] + + # regular column where + self.q_where = ['ts < now +1s','q_bigint >= -9223372036854775807 and q_bigint <= 9223372036854775807', 'q_int <= 2147483647 and q_int >= -2147483647', + 'q_smallint >= -32767 and q_smallint <= 32767','q_tinyint >= -127 and q_tinyint <= 127','q_float >= -1.7E308 and q_float <= 1.7E308', + 'q_double >= -1.7E308 and q_double <= 1.7E308', 'q_binary like \'binary%\' or q_binary = \'0\' ' , 'q_nchar like \'nchar%\' or q_nchar = \'0\' ' , + 'q_bool = true or q_bool = false' , 'q_bool in (0 , 1)' , 'q_bool in ( true , false)' , 'q_bool = 0 or q_bool = 1', + 'q_bigint between -9223372036854775807 and 9223372036854775807',' q_int between -2147483647 and 2147483647','q_smallint between -32767 and 32767', + 'q_bigint not between 9223372036854775807 and -9223372036854775807','q_int not between 2147483647 and -2147483647','q_smallint not between 32767 and -32767', + 'q_tinyint between -127 and 127 ','q_float >= -3.4E38 ','q_float <= 3.4E38 ','q_double >= -1.7E308 ', + 'q_double <= 1.7E308 ','q_float between -3.4E38 and 3.4E38 ','q_double between -1.7E308 and 1.7E308 ' ,'q_float not between 3.4E38 and -3.4E38 ','q_double not between 1.7E308 and -1.7E308 ', + 'q_float is not null ' ,'q_double is not null ' ,'q_binary match \'binary\' ','q_binary nmatch \'binarynchar\' ','q_nchar match \'nchar\' ','q_nchar nmatch \'binarynchar\' ', + 'q_binary like \'binary%\' ','(q_binary like \'binary%\' or q_nchar = \'0\' or q_binary = \'binary_\' ) ','q_nchar like \'nchar%\' ','(q_nchar like \'nchar%\' or q_binary = \'0\' or q_nchar = \'nchar_\' ) ', + 'q_binary match \'[binary]\'','q_binary nmatch \'[binarynchar]\' ',] + #TD-6201 ,'q_bool between 0 and 1' + + # regular column where for test union,join + self.q_u_where = ['t1.ts < now +1s' , 't2.ts < now +1s','t1.q_bigint >= -9223372036854775807 and t1.q_bigint <= 9223372036854775807 and t2.q_bigint >= -9223372036854775807 and t2.q_bigint <= 9223372036854775807', + 't1.q_int <= 2147483647 and t1.q_int >= -2147483647 and t2.q_int <= 2147483647 and t2.q_int >= -2147483647', + 't1.q_smallint >= -32767 and t1.q_smallint <= 32767 and t2.q_smallint >= -32767 and t2.q_smallint <= 32767', + 't1.q_tinyint >= -127 and t1.q_tinyint <= 127 and t2.q_tinyint >= -127 and t2.q_tinyint <= 127', + 't1.q_float >= - 1.7E308 and t1.q_float <= 1.7E308 and t2.q_float >= - 1.7E308 and t2.q_float <= 1.7E308', + 't1.q_double >= - 1.7E308 and t1.q_double <= 1.7E308 and t2.q_double >= - 1.7E308 and t2.q_double <= 1.7E308', + 't1.q_binary like \'binary%\' and t2.q_binary like \'binary%\' ' , + 't1.q_nchar like \'nchar%\' and t2.q_nchar like \'nchar%\' ' , + 't1.q_bool in (0 , 1) and t2.q_bool in (0 , 1)' , 't1.q_bool in ( true , false) and t2.q_bool in ( true , false)' , + 't1.q_bigint between -9223372036854775807 and 9223372036854775807 and t2.q_bigint between -9223372036854775807 and 9223372036854775807', + 't1.q_int between -2147483647 and 2147483647 and t2.q_int between -2147483647 and 2147483647', + 't1.q_smallint between -32767 and 32767 and t2.q_smallint between -32767 and 32767', + 't1.q_tinyint between -127 and 127 and t2.q_tinyint between -127 and 127 ','t1.q_float between -1.7E308 and 1.7E308 and t2.q_float between -1.7E308 and 1.7E308', + 't1.q_double between -1.7E308 and 1.7E308 and t2.q_double between -1.7E308 and 1.7E308', + 't1.q_bigint not between 9223372036854775807 and -9223372036854775807 and t2.q_bigint not between 9223372036854775807 and -9223372036854775807', + 't1.q_int not between 2147483647 and -2147483647 and t2.q_int not between 2147483647 and -2147483647', + 't1.q_smallint not between 32767 and -32767 and t2.q_smallint not between 32767 and -32767', + 't1.q_tinyint not between 127 and -127 and t2.q_tinyint not between 127 and -127 ','t1.q_float not between -1.7E308 and -1.7E308 and t2.q_float not between 1.7E308 and -1.7E308', + 't1.q_double not between 1.7E308 and -1.7E308 and t2.q_double not between 1.7E308 and -1.7E308'] + #TD-6201 ,'t1.q_bool between 0 and 1 or t2.q_bool between 0 and 1'] + #'t1.q_bool = true and t1.q_bool = false and t2.q_bool = true and t2.q_bool = false' , 't1.q_bool = 0 and t1.q_bool = 1 and t2.q_bool = 0 and t2.q_bool = 1' , + + self.q_u_or_where = ['(t1.q_binary like \'binary%\' or t1.q_binary = \'0\' or t2.q_binary like \'binary%\' or t2.q_binary = \'0\' )' , + '(t1.q_nchar like \'nchar%\' or t1.q_nchar = \'0\' or t2.q_nchar like \'nchar%\' or t2.q_nchar = \'0\' )' , '(t1.q_bool = true or t1.q_bool = false or t2.q_bool = true or t2.q_bool = false)' , + '(t1.q_bool in (0 , 1) or t2.q_bool in (0 , 1))' , '(t1.q_bool in ( true , false) or t2.q_bool in ( true , false))' , '(t1.q_bool = 0 or t1.q_bool = 1 or t2.q_bool = 0 or t2.q_bool = 1)' , + '(t1.q_bigint between -9223372036854775807 and 9223372036854775807 or t2.q_bigint between -9223372036854775807 and 9223372036854775807)', + '(t1.q_int between -2147483647 and 2147483647 or t2.q_int between -2147483647 and 2147483647)', + '(t1.q_smallint between -32767 and 32767 or t2.q_smallint between -32767 and 32767)', + '(t1.q_tinyint between -127 and 127 or t2.q_tinyint between -127 and 127 )','(t1.q_float between -1.7E308 and 1.7E308 or t2.q_float between -1.7E308 and 1.7E308)', + '(t1.q_double between -1.7E308 and 1.7E308 or t2.q_double between -1.7E308 and 1.7E308)'] + + # tag column where + self.t_where = ['ts < now +1s','t_bigint >= -9223372036854775807 and t_bigint <= 9223372036854775807','t_int <= 2147483647 and t_int >= -2147483647', + 't_smallint >= -32767 and t_smallint <= 32767','q_tinyint >= -127 and t_tinyint <= 127','t_float >= -1.7E308 and t_float <= 1.7E308', + 't_double >= -1.7E308 and t_double <= 1.7E308', 't_binary like \'binary%\' or t_binary = \'0\' ' , 't_nchar like \'nchar%\' or t_nchar = \'0\'' , + 't_bool = true or t_bool = false' , 't_bool in (0 , 1)' , 't_bool in ( true , false)' , 't_bool = 0 or t_bool = 1', + 't_bigint between -9223372036854775807 and 9223372036854775807',' t_int between -2147483647 and 2147483647','t_smallint between -32767 and 32767', + 't_tinyint between -127 and 127 ','t_float between -1.7E308 and 1.7E308','t_double between -1.7E308 and 1.7E308', + 't_binary match \'binary\' ','t_binary nmatch \'binarynchar\' ','t_nchar match \'nchar\' ','t_nchar nmatch \'binarynchar\' ', + 't_binary like \'binary%\' ','t_nchar like \'nchar%\' ','(t_binary like \'binary%\' or t_nchar = \'0\' ) ','(t_nchar like \'nchar%\' or t_binary = \'0\' ) ', + 'tbname match \'[stable]\' ','tbname nmatch \'[qwwerrwee]\' ','loc match \'[stable]\' ','loc nmatch \'[qwwerrwee]\' ', + 'tbname match \'[^stable]\' ','tbname nmatch \'[^qwwerrwee]\' ','loc match \'[^stable]\' ','loc nmatch \'[^qwwerrwee]\' ',] + + # tag column where for test union,join | this is not support + self.t_u_where = ['t1.ts < now +1s' , 't2.ts < now +1s','t1.t_bigint >= -9223372036854775807 and t1.t_bigint <= 9223372036854775807 and t2.t_bigint >= -9223372036854775807 and t2.t_bigint <= 9223372036854775807', + 't1.t_int <= 2147483647 and t1.t_int >= -2147483647 and t2.t_int <= 2147483647 and t2.t_int >= -2147483647', + 't1.t_smallint >= -32767 and t1.t_smallint <= 32767 and t2.t_smallint >= -32767 and t2.t_smallint <= 32767', + 't1.t_tinyint >= -127 and t1.t_tinyint <= 127 and t2.t_tinyint >= -127 and t2.t_tinyint <= 127', + 't1.t_float >= -1.7E308 and t1.t_float <= 1.7E308 and t2.t_float >= -1.7E308 and t2.t_float <= 1.7E308', + 't1.t_double >= -1.7E308 and t1.t_double <= 1.7E308 and t2.t_double >= -1.7E308 and t2.t_double <= 1.7E308', + '(t1.t_binary like \'binary%\' or t1.t_binary = \'0\' or t2.t_binary like \'binary%\' or t2.t_binary = \'0\') ' , + '(t1.t_nchar like \'nchar%\' or t1.t_nchar = \'0\' or t2.t_nchar like \'nchar%\' or t2.t_nchar = \'0\' )' , '(t1.t_bool = true or t1.t_bool = false or t2.t_bool = true or t2.t_bool = false)' , + 't1.t_bool in (0 , 1) and t2.t_bool in (0 , 1)' , 't1.t_bool in ( true , false) and t2.t_bool in ( true , false)' , '(t1.t_bool = 0 or t1.t_bool = 1 or t2.t_bool = 0 or t2.t_bool = 1)', + 't1.t_bigint between -9223372036854775807 and 9223372036854775807 and t2.t_bigint between -9223372036854775807 and 9223372036854775807', + 't1.t_int between -2147483647 and 2147483647 and t2.t_int between -2147483647 and 2147483647', + 't1.t_smallint between -32767 and 32767 and t2.t_smallint between -32767 and 32767', + '(t1.t_tinyint between -127 and 127 and t2.t_tinyint between -127 and 127) ','t1.t_float between -1.7E308 and 1.7E308 and t2.t_float between -1.7E308 and 1.7E308', + '(t1.t_double between -1.7E308 and 1.7E308 and t2.t_double between -1.7E308 and 1.7E308)', + '(t1.loc match \'[stable]\' and t2.loc match \'[stable]\')','(t1.loc nmatch \'[qqeqweq]\' and t2.loc nmatch \'[eqeqweq]\')', + '(t1.loc match \'[^stable]\' and t2.loc match \'[^stable]\')','(t1.loc nmatch \'[^qqeqweq]\' and t2.loc nmatch \'[^eqeqweq]\')', + '(t1.t_binary match \'[stable]\' and t2.t_binary match \'[stable]\')','(t1.t_binary nmatch \'[qqeqweq]\' and t2.t_binary nmatch \'[eqeqweq]\')', + '(t1.t_binary match \'[^stable]\' and t2.t_binary match \'[^stable]\')','(t1.t_binary nmatch \'[^qqeqweq]\' and t2.t_binary nmatch \'[^eqeqweq]\')', + '(t1.t_nchar match \'[stable]\' and t2.t_nchar match \'[stable]\')','(t1.t_nchar nmatch \'[qqeqweq]\' and t2.t_nchar nmatch \'[eqeqweq]\')', + '(t1.t_nchar match \'[^stable]\' and t2.t_nchar match \'[^stable]\')','(t1.t_nchar nmatch \'[^qqeqweq]\' and t2.t_nchar nmatch \'[^eqeqweq]\')'] + + self.t_u_or_where = ['(t1.t_binary like \'binary%\' or t1.t_binary = \'0\' or t2.t_binary like \'binary%\' or t2.t_binary = \'0\' )' , + '(t1.t_nchar like \'nchar%\' or t1.t_nchar = \'0\' or t2.t_nchar like \'nchar%\' or t2.t_nchar = \'0\' )' , '(t1.t_bool = true or t1.t_bool = false or t2.t_bool = true or t2.t_bool = false)' , + '(t1.t_bool in (0 , 1) or t2.t_bool in (0 , 1))' , '(t1.t_bool in ( true , false) or t2.t_bool in ( true , false))' , '(t1.t_bool = 0 or t1.t_bool = 1 or t2.t_bool = 0 or t2.t_bool = 1)', + '(t1.t_bigint between -9223372036854775807 and 9223372036854775807 or t2.t_bigint between -9223372036854775807 and 9223372036854775807)', + '(t1.t_int between -2147483647 and 2147483647 or t2.t_int between -2147483647 and 2147483647)', + '(t1.t_smallint between -32767 and 32767 or t2.t_smallint between -32767 and 32767)', + '(t1.t_tinyint between -127 and 127 or t2.t_tinyint between -127 and 127 )','(t1.t_float between -1.7E308 and 1.7E308 or t2.t_float between -1.7E308 and 1.7E308)', + '(t1.t_double between -1.7E308 and 1.7E308 or t2.t_double between -1.7E308 and 1.7E308)', + '(t1.loc match \'[stable]\' or t2.loc match \'[stable]\')','(t1.loc nmatch \'[qqeqweq]\' or t2.loc nmatch \'[eqeqweq]\')', + '(t1.loc match \'[^stable]\' or t2.loc match \'[^stable]\')','(t1.loc nmatch \'[^qqeqweq]\' or t2.loc nmatch \'[^eqeqweq]\')' , + '(t1.t_binary match \'[stable]\' or t2.t_binary match \'[stable]\')','(t1.t_binary nmatch \'[qqeqweq]\' or t2.t_binary nmatch \'[eqeqweq]\')', + '(t1.t_binary match \'[^stable]\' or t2.t_binary match \'[^stable]\')','(t1.t_binary nmatch \'[^qqeqweq]\' or t2.t_binary nmatch \'[^eqeqweq]\')', + '(t1.t_nchar match \'[stable]\' or t2.t_nchar match \'[stable]\')','(t1.t_nchar nmatch \'[qqeqweq]\' or t2.t_nchar nmatch \'[eqeqweq]\')', + '(t1.t_nchar match \'[^stable]\' or t2.t_nchar match \'[^stable]\')','(t1.t_nchar nmatch \'[^qqeqweq]\' or t2.t_nchar nmatch \'[^eqeqweq]\')'] + + # regular and tag column where + self.qt_where = self.q_where + self.t_where + self.qt_u_where = self.q_u_where + self.t_u_where + self.qt_u_or_where = self.q_u_or_where + self.t_u_or_where + + # tag column where for test super join | this is support , 't1.t_bool = t2.t_bool ' ??? + self.t_join_where = ['t1.t_bigint = t2.t_bigint ', 't1.t_int = t2.t_int ', 't1.t_smallint = t2.t_smallint ', 't1.t_tinyint = t2.t_tinyint ', + 't1.t_float = t2.t_float ', 't1.t_double = t2.t_double ', 't1.t_binary = t2.t_binary ' , 't1.t_nchar = t2.t_nchar ' ] + + # session && fill + self.session_where = ['session(ts,10a)' , 'session(ts,10s)', 'session(ts,10m)' , 'session(ts,10h)','session(ts,10d)' , 'session(ts,10w)'] + self.session_u_where = ['session(t1.ts,10a)' , 'session(t1.ts,10s)', 'session(t1.ts,10m)' , 'session(t1.ts,10h)','session(t1.ts,10d)' , 'session(t1.ts,10w)', + 'session(t2.ts,10a)' , 'session(t2.ts,10s)', 'session(t2.ts,10m)' , 'session(t2.ts,10h)','session(t2.ts,10d)' , 'session(t2.ts,10w)'] + + self.fill_where = ['FILL(NONE)','FILL(PREV)','FILL(NULL)','FILL(LINEAR)','FILL(NEXT)','FILL(VALUE, 1.23)'] + + self.state_window = ['STATE_WINDOW(q_tinyint)','STATE_WINDOW(q_bigint)','STATE_WINDOW(q_int)','STATE_WINDOW(q_bool)','STATE_WINDOW(q_smallint)'] + self.state_u_window = ['STATE_WINDOW(t1.q_tinyint)','STATE_WINDOW(t1.q_bigint)','STATE_WINDOW(t1.q_int)','STATE_WINDOW(t1.q_bool)','STATE_WINDOW(t1.q_smallint)', + 'STATE_WINDOW(t2.q_tinyint)','STATE_WINDOW(t2.q_bigint)','STATE_WINDOW(t2.q_int)','STATE_WINDOW(t2.q_bool)','STATE_WINDOW(t2.q_smallint)'] + + # order by where + self.order_where = ['order by ts' , 'order by ts asc'] + self.order_u_where = ['order by t1.ts' , 'order by t1.ts asc' , 'order by t2.ts' , 'order by t2.ts asc'] + self.order_desc_where = ['order by ts' , 'order by ts asc' , 'order by ts desc' ] + self.orders_desc_where = ['order by ts' , 'order by ts asc' , 'order by ts desc' , 'order by ts,loc' , 'order by ts,loc asc' , 'order by ts,loc desc'] + + self.group_where = ['group by tbname , loc' , 'group by tbname', 'group by tbname, t_bigint', 'group by tbname,t_int', 'group by tbname, t_smallint', 'group by tbname,t_tinyint', + 'group by tbname,t_float', 'group by tbname,t_double' , 'group by tbname,t_binary', 'group by tbname,t_nchar', 'group by tbname,t_bool' ,'group by tbname ,loc ,t_bigint', + 'group by tbname,t_binary ,t_nchar ,t_bool' , 'group by tbname,t_int ,t_smallint ,t_tinyint' , 'group by tbname,t_float ,t_double ' , + 'PARTITION BY tbname , loc' , 'PARTITION BY tbname', 'PARTITION BY tbname, t_bigint', 'PARTITION BY tbname,t_int', 'PARTITION BY tbname, t_smallint', 'PARTITION BY tbname,t_tinyint', + 'PARTITION BY tbname,t_float', 'PARTITION BY tbname,t_double' , 'PARTITION BY tbname,t_binary', 'PARTITION BY tbname,t_nchar', 'PARTITION BY tbname,t_bool' ,'PARTITION BY tbname ,loc ,t_bigint', + 'PARTITION BY tbname,t_binary ,t_nchar ,t_bool' , 'PARTITION BY tbname,t_int ,t_smallint ,t_tinyint' , 'PARTITION BY tbname,t_float ,t_double '] + self.group_where_j = ['group by t1.loc' , 'group by t1.t_bigint', 'group by t1.t_int', 'group by t1.t_smallint', 'group by t1.t_tinyint', + 'group by t1.t_float', 'group by t1.t_double' , 'group by t1.t_binary', 'group by t1.t_nchar', 'group by t1.t_bool' ,'group by t1.loc ,t1.t_bigint', + 'group by t1.t_binary ,t1.t_nchar ,t1.t_bool' , 'group by t1.t_int ,t1.t_smallint ,t1.t_tinyint' , 'group by t1.t_float ,t1.t_double ' , + 'PARTITION BY t1.loc' , 'PARTITION by t1.t_bigint', 'PARTITION by t1.t_int', 'PARTITION by t1.t_smallint', 'PARTITION by t1.t_tinyint', + 'PARTITION by t1.t_float', 'PARTITION by t1.t_double' , 'PARTITION by t1.t_binary', 'PARTITION by t1.t_nchar', 'PARTITION by t1.t_bool' ,'PARTITION BY t1.loc ,t1.t_bigint', + 'PARTITION by t1.t_binary ,t1.t_nchar ,t1.t_bool' , 'PARTITION by t1.t_int ,t1.t_smallint ,t1.t_tinyint' , 'PARTITION by t1.t_float ,t1.t_double ', + 'group by t2.loc' , 'group by t2.t_bigint', 'group by t2.t_int', 'group by t2.t_smallint', 'group by t2.t_tinyint', + 'group by t2.t_float', 'group by t2.t_double' , 'group by t2.t_binary', 'group by t2.t_nchar', 'group by t2.t_bool' ,'group by t2.loc ,t2.t_bigint', + 'group by t2.t_binary ,t2.t_nchar ,t2.t_bool' , 'group by t2.t_int ,t2.t_smallint ,t2.t_tinyint' , 'group by t2.t_float ,t2.t_double ' , + 'PARTITION BY t2.loc' , 'PARTITION by t2.t_bigint', 'PARTITION by t2.t_int', 'PARTITION by t2.t_smallint', 'PARTITION by t2.t_tinyint', + 'PARTITION by t2.t_float', 'PARTITION by t2.t_double' , 'PARTITION by t2.t_binary', 'PARTITION by t2.t_nchar', 'PARTITION by t2.t_bool' ,'PARTITION BY t2.loc ,t2.t_bigint', + 'PARTITION by t2.t_binary ,t2.t_nchar ,t2.t_bool' , 'PARTITION by t2.t_int ,t2.t_smallint ,t2.t_tinyint' , 'PARTITION by t2.t_float ,t2.t_double '] + + self.group_only_where = ['group by tbname , loc' , 'group by tbname', 'group by tbname, t_bigint', 'group by tbname,t_int', 'group by tbname, t_smallint', 'group by tbname,t_tinyint', + 'group by tbname,t_float', 'group by tbname,t_double' , 'group by tbname,t_binary', 'group by tbname,t_nchar', 'group by tbname,t_bool' ,'group by tbname ,loc ,t_bigint', + 'group by tbname,t_binary ,t_nchar ,t_bool' , 'group by tbname,t_int ,t_smallint ,t_tinyint' , 'group by tbname,t_float ,t_double ' ] + self.group_only_where_j = ['group by t1.loc' , 'group by t1.t_bigint', 'group by t1.t_int', 'group by t1.t_smallint', 'group by t1.t_tinyint', + 'group by t1.t_float', 'group by t1.t_double' , 'group by t1.t_binary', 'group by t1.t_nchar', 'group by t1.t_bool' ,'group by t1.loc ,t1.t_bigint', + 'group by t1.t_binary ,t1.t_nchar ,t1.t_bool' , 'group by t1.t_int ,t1.t_smallint ,t1.t_tinyint' , 'group by t1.t_float ,t1.t_double ' , + 'group by t2.loc' , 'group by t2.t_bigint', 'group by t2.t_int', 'group by t2.t_smallint', 'group by t2.t_tinyint', + 'group by t2.t_float', 'group by t2.t_double' , 'group by t2.t_binary', 'group by t2.t_nchar', 'group by t2.t_bool' ,'group by t2.loc ,t2.t_bigint', + 'group by t2.t_binary ,t2.t_nchar ,t2.t_bool' , 'group by t2.t_int ,t2.t_smallint ,t2.t_tinyint' , 'group by t2.t_float ,t2.t_double ' ] + + self.partiton_where = ['PARTITION BY tbname , loc' , 'PARTITION BY tbname', 'PARTITION BY tbname, t_bigint', 'PARTITION BY tbname,t_int', 'PARTITION BY tbname, t_smallint', 'PARTITION BY tbname,t_tinyint', + 'PARTITION BY tbname,t_float', 'PARTITION BY tbname,t_double' , 'PARTITION BY tbname,t_binary', 'PARTITION BY tbname,t_nchar', 'PARTITION BY tbname,t_bool' ,'PARTITION BY tbname ,loc ,t_bigint', + 'PARTITION BY tbname,t_binary ,t_nchar ,t_bool' , 'PARTITION BY tbname,t_int ,t_smallint ,t_tinyint' , 'PARTITION BY tbname,t_float ,t_double '] + self.partiton_where_j = ['PARTITION BY t1.loc' , 'PARTITION by t1.t_bigint', 'PARTITION by t1.t_int', 'PARTITION by t1.t_smallint', 'PARTITION by t1.t_tinyint', + 'PARTITION by t1.t_float', 'PARTITION by t1.t_double' , 'PARTITION by t1.t_binary', 'PARTITION by t1.t_nchar', 'PARTITION by t1.t_bool' ,'PARTITION BY t1.loc ,t1.t_bigint', + 'PARTITION by t1.t_binary ,t1.t_nchar ,t1.t_bool' , 'PARTITION by t1.t_int ,t1.t_smallint ,t1.t_tinyint' , 'PARTITION by t1.t_float ,t1.t_double ', + 'PARTITION BY t2.loc' , 'PARTITION by t2.t_bigint', 'PARTITION by t2.t_int', 'PARTITION by t2.t_smallint', 'PARTITION by t2.t_tinyint', + 'PARTITION by t2.t_float', 'PARTITION by t2.t_double' , 'PARTITION by t2.t_binary', 'PARTITION by t2.t_nchar', 'PARTITION by t2.t_bool' ,'PARTITION BY t2.loc ,t2.t_bigint', + 'PARTITION by t2.t_binary ,t2.t_nchar ,t2.t_bool' , 'PARTITION by t2.t_int ,t2.t_smallint ,t2.t_tinyint' , 'PARTITION by t2.t_float ,t2.t_double '] + + self.group_where_regular = ['group by tbname ' , 'group by tbname', 'group by tbname, q_bigint', 'group by tbname,q_int', 'group by tbname, q_smallint', 'group by tbname,q_tinyint', + 'group by tbname,q_float', 'group by tbname,q_double' , 'group by tbname,q_binary', 'group by tbname,q_nchar', 'group by tbname,q_bool' ,'group by tbname ,q_bigint', + 'group by tbname,q_binary ,q_nchar ,q_bool' , 'group by tbname,q_int ,q_smallint ,q_tinyint' , 'group by tbname,q_float ,q_double ' , + 'PARTITION BY tbname ' , 'PARTITION BY tbname', 'PARTITION BY tbname, q_bigint', 'PARTITION BY tbname,q_int', 'PARTITION BY tbname, q_smallint', 'PARTITION BY tbname,q_tinyint', + 'PARTITION BY tbname,q_float', 'PARTITION BY tbname,q_double' , 'PARTITION BY tbname,q_binary', 'PARTITION BY tbname,q_nchar', 'PARTITION BY tbname,q_bool' ,'PARTITION BY tbname ,q_bigint', + 'PARTITION BY tbname,q_binary ,q_nchar ,q_bool' , 'PARTITION BY tbname,q_int ,q_smallint ,q_tinyint' , 'PARTITION BY tbname,q_float ,q_double '] + self.group_where_regular_j = ['group by t1.q_bigint', 'group by t1.q_int', 'group by t1.q_smallint', 'group by t1.q_tinyint', + 'group by t1.q_float', 'group by t1.q_double' , 'group by t1.q_binary', 'group by t1.q_nchar', 'group by t1.q_bool' ,'group by t1.q_bigint', + 'group by t1.q_binary ,t1.q_nchar ,t1.q_bool' , 'group by t1.q_int ,t1.q_smallint ,t1.q_tinyint' , 'group by t1.q_float ,t1.q_double ' , + 'PARTITION by t1.q_bigint', 'PARTITION by t1.q_int', 'PARTITION by t1.q_smallint', 'PARTITION by t1.q_tinyint', + 'PARTITION by t1.q_float', 'PARTITION by t1.q_double' , 'PARTITION by t1.q_binary', 'PARTITION by t1.q_nchar', 'PARTITION by t1.q_bool' ,'PARTITION BY t1.q_bigint', + 'PARTITION by t1.q_binary ,t1.q_nchar ,t1.q_bool' , 'PARTITION by t1.q_int ,t1.q_smallint ,t1.q_tinyint' , 'PARTITION by t1.q_float ,t1.q_double ', + 'group by t2.q_bigint', 'group by t2.q_int', 'group by t2.q_smallint', 'group by t2.q_tinyint', + 'group by t2.q_float', 'group by t2.q_double' , 'group by t2.q_binary', 'group by t2.q_nchar', 'group by t2.q_bool' ,'group by t2.q_bigint', + 'group by t2.q_binary ,t2.q_nchar ,t2.q_bool' , 'group by t2.q_int ,t2.q_smallint ,t2.q_tinyint' , 'group by t2.q_float ,t2.q_double ' , + 'PARTITION by t2.q_bigint', 'PARTITION by t2.q_int', 'PARTITION by t2.q_smallint', 'PARTITION by t2.q_tinyint', + 'PARTITION by t2.q_float', 'PARTITION by t2.q_double' , 'PARTITION by t2.q_binary', 'PARTITION by t2.q_nchar', 'PARTITION by t2.q_bool' ,'PARTITION BY t2.q_bigint', + 'PARTITION by t2.q_binary ,t2.q_nchar ,t2.q_bool' , 'PARTITION by t2.q_int ,t2.q_smallint ,t2.q_tinyint' , 'PARTITION by t2.q_float ,t2.q_double '] + + self.partiton_where_regular = ['PARTITION BY tbname ' , 'PARTITION BY tbname', 'PARTITION BY tbname, q_bigint', 'PARTITION BY tbname,q_int', 'PARTITION BY tbname, q_smallint', 'PARTITION BY tbname,q_tinyint', + 'PARTITION BY tbname,q_float', 'PARTITION BY tbname,q_double' , 'PARTITION BY tbname,q_binary', 'PARTITION BY tbname,q_nchar', 'PARTITION BY tbname,q_bool' ,'PARTITION BY tbname ,q_bigint', + 'PARTITION BY tbname,q_binary ,q_nchar ,q_bool' , 'PARTITION BY tbname,q_int ,q_smallint ,q_tinyint' , 'PARTITION BY tbname,q_float ,q_double '] + self.partiton_where_regular_j = ['PARTITION by t1.q_bigint', 'PARTITION by t1.q_int', 'PARTITION by t1.q_smallint', 'PARTITION by t1.q_tinyint', + 'PARTITION by t1.q_float', 'PARTITION by t1.q_double' , 'PARTITION by t1.q_binary', 'PARTITION by t1.q_nchar', 'PARTITION by t1.q_bool' ,'PARTITION BY t1.q_bigint', + 'PARTITION by t1.q_binary ,t1.q_nchar ,t1.q_bool' , 'PARTITION by t1.q_int ,t1.q_smallint ,t1.q_tinyint' , 'PARTITION by t1.q_float ,t1.q_double ', + 'PARTITION by t2.q_bigint', 'PARTITION by t2.q_int', 'PARTITION by t2.q_smallint', 'PARTITION by t2.q_tinyint', + 'PARTITION by t2.q_float', 'PARTITION by t2.q_double' , 'PARTITION by t2.q_binary', 'PARTITION by t2.q_nchar', 'PARTITION by t2.q_bool' ,'PARTITION BY t2.q_bigint', + 'PARTITION by t2.q_binary ,t2.q_nchar ,t2.q_bool' , 'PARTITION by t2.q_int ,t2.q_smallint ,t2.q_tinyint' , 'PARTITION by t2.q_float ,t2.q_double '] + + self.having_support = ['having count(q_int) > 0','having count(q_bigint) > 0','having count(q_smallint) > 0','having count(q_tinyint) > 0','having count(q_float) > 0','having count(q_double) > 0','having count(q_bool) > 0', + 'having avg(q_int) > 0','having avg(q_bigint) > 0','having avg(q_smallint) > 0','having avg(q_tinyint) > 0','having avg(q_float) > 0','having avg(q_double) > 0', + 'having sum(q_int) > 0','having sum(q_bigint) > 0','having sum(q_smallint) > 0','having sum(q_tinyint) > 0','having sum(q_float) > 0','having sum(q_double) > 0', + 'having STDDEV(q_int) > 0','having STDDEV(q_bigint) > 0','having STDDEV(q_smallint) > 0','having STDDEV(q_tinyint) > 0','having STDDEV(q_float) > 0','having STDDEV(q_double) > 0', + 'having TWA(q_int) > 0','having TWA(q_bigint) > 0','having TWA(q_smallint) > 0','having TWA(q_tinyint) > 0','having TWA(q_float) > 0','having TWA(q_double) > 0', + 'having IRATE(q_int) > 0','having IRATE(q_bigint) > 0','having IRATE(q_smallint) > 0','having IRATE(q_tinyint) > 0','having IRATE(q_float) > 0','having IRATE(q_double) > 0', + 'having MIN(q_int) > 0','having MIN(q_bigint) > 0','having MIN(q_smallint) > 0','having MIN(q_tinyint) > 0','having MIN(q_float) > 0','having MIN(q_double) > 0', + 'having MAX(q_int) > 0','having MAX(q_bigint) > 0','having MAX(q_smallint) > 0','having MAX(q_tinyint) > 0','having MAX(q_float) > 0','having MAX(q_double) > 0', + 'having FIRST(q_int) > 0','having FIRST(q_bigint) > 0','having FIRST(q_smallint) > 0','having FIRST(q_tinyint) > 0','having FIRST(q_float) > 0','having FIRST(q_double) > 0', + 'having LAST(q_int) > 0','having LAST(q_bigint) > 0','having LAST(q_smallint) > 0','having LAST(q_tinyint) > 0','having LAST(q_float) > 0','having LAST(q_double) > 0', + 'having APERCENTILE(q_int,10) > 0','having APERCENTILE(q_bigint,10) > 0','having APERCENTILE(q_smallint,10) > 0','having APERCENTILE(q_tinyint,10) > 0','having APERCENTILE(q_float,10) > 0','having APERCENTILE(q_double,10) > 0', + 'having count(q_int_null) > 0','having count(q_bigint_null) > 0','having count(q_smallint_null) > 0','having count(q_tinyint_null) > 0','having count(q_float_null) > 0','having count(q_double_null) > 0','having count(q_bool_null) > 0', + 'having avg(q_int_null) > 0','having avg(q_bigint_null) > 0','having avg(q_smallint_null) > 0','having avg(q_tinyint_null) > 0','having avg(q_float_null) > 0','having avg(q_double_null) > 0', + 'having sum(q_int_null) > 0','having sum(q_bigint_null) > 0','having sum(q_smallint_null) > 0','having sum(q_tinyint_null) > 0','having sum(q_float_null) > 0','having sum(q_double_null) > 0', + 'having STDDEV(q_int_null) > 0','having STDDEV(q_bigint_null) > 0','having STDDEV(q_smallint_null) > 0','having STDDEV(q_tinyint_null) > 0','having STDDEV(q_float_null) > 0','having STDDEV(q_double_null) > 0', + 'having TWA(q_int_null) > 0','having TWA(q_bigint_null) > 0','having TWA(q_smallint_null) > 0','having TWA(q_tinyint_null) > 0','having TWA(q_float_null) > 0','having TWA(q_double_null) > 0', + 'having IRATE(q_int_null) > 0','having IRATE(q_bigint_null) > 0','having IRATE(q_smallint_null) > 0','having IRATE(q_tinyint_null) > 0','having IRATE(q_float_null) > 0','having IRATE(q_double_null) > 0', + 'having MIN(q_int_null) > 0','having MIN(q_bigint_null) > 0','having MIN(q_smallint_null) > 0','having MIN(q_tinyint_null) > 0','having MIN(q_float_null) > 0','having MIN(q_double_null) > 0', + 'having MAX(q_int_null) > 0','having MAX(q_bigint_null) > 0','having MAX(q_smallint_null) > 0','having MAX(q_tinyint_null) > 0','having MAX(q_float_null) > 0','having MAX(q_double_null) > 0', + 'having FIRST(q_int_null) > 0','having FIRST(q_bigint_null) > 0','having FIRST(q_smallint_null) > 0','having FIRST(q_tinyint_null) > 0','having FIRST(q_float_null) > 0','having FIRST(q_double_null) > 0', + 'having LAST(q_int_null) > 0','having LAST(q_bigint_null) > 0','having LAST(q_smallint_null) > 0','having LAST(q_tinyint_null) > 0','having LAST(q_float_null) > 0','having LAST(q_double_null) > 0', + 'having APERCENTILE(q_int_null,10) > 0','having APERCENTILE(q_bigint_null,10) > 0','having APERCENTILE(q_smallint_null,10) > 0','having APERCENTILE(q_tinyint_null,10) > 0','having APERCENTILE(q_float_null,10) > 0','having APERCENTILE(q_double_null,10) > 0'] + self.having_not_support = ['having TOP(q_int,10) > 0','having TOP(q_bigint,10) > 0','having TOP(q_smallint,10) > 0','having TOP(q_tinyint,10) > 0','having TOP(q_float,10) > 0','having TOP(q_double,10) > 0','having TOP(q_bool,10) > 0', + 'having BOTTOM(q_int,10) > 0','having BOTTOM(q_bigint,10) > 0','having BOTTOM(q_smallint,10) > 0','having BOTTOM(q_tinyint,10) > 0','having BOTTOM(q_float,10) > 0','having BOTTOM(q_double,10) > 0','having BOTTOM(q_bool,10) > 0', + 'having LEASTSQUARES(q_int) > 0','having LEASTSQUARES(q_bigint) > 0','having LEASTSQUARES(q_smallint) > 0','having LEASTSQUARES(q_tinyint) > 0','having LEASTSQUARES(q_float) > 0','having LEASTSQUARES(q_double) > 0','having LEASTSQUARES(q_bool) > 0', + 'having FIRST(q_bool) > 0','having IRATE(q_bool) > 0','having PERCENTILE(q_bool,10) > 0','having avg(q_bool) > 0','having LAST_ROW(q_bool) > 0','having sum(q_bool) > 0','having STDDEV(q_bool) > 0','having APERCENTILE(q_bool,10) > 0','having TWA(q_bool) > 0','having LAST(q_bool) > 0', + 'having PERCENTILE(q_int,10) > 0','having PERCENTILE(q_bigint,10) > 0','having PERCENTILE(q_smallint,10) > 0','having PERCENTILE(q_tinyint,10) > 0','having PERCENTILE(q_float,10) > 0','having PERCENTILE(q_double,10) > 0', + 'having TOP(q_int_null,10) > 0','having TOP(q_bigint_null,10) > 0','having TOP(q_smallint_null,10) > 0','having TOP(q_tinyint_null,10) > 0','having TOP(q_float_null,10) > 0','having TOP(q_double_null,10) > 0','having TOP(q_bool_null,10) > 0', + 'having BOTTOM(q_int_null,10) > 0','having BOTTOM(q_bigint_null,10) > 0','having BOTTOM(q_smallint_null,10) > 0','having BOTTOM(q_tinyint_null,10) > 0','having BOTTOM(q_float_null,10) > 0','having BOTTOM(q_double_null,10) > 0','having BOTTOM(q_bool_null,10) > 0', + 'having LEASTSQUARES(q_int_null) > 0','having LEASTSQUARES(q_bigint_null) > 0','having LEASTSQUARES(q_smallint_null) > 0','having LEASTSQUARES(q_tinyint_null) > 0','having LEASTSQUARES(q_float_null) > 0','having LEASTSQUARES(q_double_null) > 0','having LEASTSQUARES(q_bool_null) > 0', + 'having FIRST(q_bool_null) > 0','having IRATE(q_bool_null) > 0','having PERCENTILE(q_bool_null,10) > 0','having avg(q_bool_null) > 0','having LAST_ROW(q_bool_null) > 0','having sum(q_bool_null) > 0','having STDDEV(q_bool_null) > 0','having APERCENTILE(q_bool_null,10) > 0','having TWA(q_bool_null) > 0','having LAST(q_bool_null) > 0', + 'having PERCENTILE(q_int_null,10) > 0','having PERCENTILE(q_bigint_null,10) > 0','having PERCENTILE(q_smallint_null,10) > 0','having PERCENTILE(q_tinyint_null,10) > 0','having PERCENTILE(q_float_null,10) > 0','having PERCENTILE(q_double_null,10) > 0'] + self.having_tagnot_support = ['having LAST_ROW(q_int) > 0','having LAST_ROW(q_bigint) > 0','having LAST_ROW(q_smallint) > 0','having LAST_ROW(q_tinyint) > 0','having LAST_ROW(q_float) > 0','having LAST_ROW(q_double) > 0', + 'having LAST_ROW(q_int_null) > 0','having LAST_ROW(q_bigint_null) > 0','having LAST_ROW(q_smallint_null) > 0','having LAST_ROW(q_tinyint_null) > 0','having LAST_ROW(q_float_null) > 0','having LAST_ROW(q_double_null) > 0'] + + self.having_support_j = ['having count(t1.q_int) > 0','having count(t1.q_bigint) > 0','having count(t1.q_smallint) > 0','having count(t1.q_tinyint) > 0','having count(t1.q_float) > 0','having count(t1.q_double) > 0','having count(t1.q_bool) > 0', + 'having avg(t1.q_int) > 0','having avg(t1.q_bigint) > 0','having avg(t1.q_smallint) > 0','having avg(t1.q_tinyint) > 0','having avg(t1.q_float) > 0','having avg(t1.q_double) > 0', + 'having sum(t1.q_int) > 0','having sum(t1.q_bigint) > 0','having sum(t1.q_smallint) > 0','having sum(t1.q_tinyint) > 0','having sum(t1.q_float) > 0','having sum(t1.q_double) > 0', + 'having STDDEV(t1.q_int) > 0','having STDDEV(t1.q_bigint) > 0','having STDDEV(t1.q_smallint) > 0','having STDDEV(t1.q_tinyint) > 0','having STDDEV(t1.q_float) > 0','having STDDEV(t1.q_double) > 0', + 'having TWA(t1.q_int) > 0','having TWA(t1.q_bigint) > 0','having TWA(t1.q_smallint) > 0','having TWA(t1.q_tinyint) > 0','having TWA(t1.q_float) > 0','having TWA(t1.q_double) > 0', + 'having IRATE(t1.q_int) > 0','having IRATE(t1.q_bigint) > 0','having IRATE(t1.q_smallint) > 0','having IRATE(t1.q_tinyint) > 0','having IRATE(t1.q_float) > 0','having IRATE(t1.q_double) > 0', + 'having MIN(t1.q_int) > 0','having MIN(t1.q_bigint) > 0','having MIN(t1.q_smallint) > 0','having MIN(t1.q_tinyint) > 0','having MIN(t1.q_float) > 0','having MIN(t1.q_double) > 0', + 'having MAX(t1.q_int) > 0','having MAX(t1.q_bigint) > 0','having MAX(t1.q_smallint) > 0','having MAX(t1.q_tinyint) > 0','having MAX(t1.q_float) > 0','having MAX(t1.q_double) > 0', + 'having FIRST(t1.q_int) > 0','having FIRST(t1.q_bigint) > 0','having FIRST(t1.q_smallint) > 0','having FIRST(t1.q_tinyint) > 0','having FIRST(t1.q_float) > 0','having FIRST(t1.q_double) > 0', + 'having LAST(t1.q_int) > 0','having LAST(t1.q_bigint) > 0','having LAST(t1.q_smallint) > 0','having LAST(t1.q_tinyint) > 0','having LAST(t1.q_float) > 0','having LAST(t1.q_double) > 0', + 'having APERCENTILE(t1.q_int,10) > 0','having APERCENTILE(t1.q_bigint,10) > 0','having APERCENTILE(t1.q_smallint,10) > 0','having APERCENTILE(t1.q_tinyint,10) > 0','having APERCENTILE(t1.q_float,10) > 0','having APERCENTILE(t1.q_double,10) > 0'] + + # limit offset where + self.limit_where = ['limit 1 offset 1' , 'limit 1' , 'limit 2 offset 1' , 'limit 2', 'limit 12 offset 1' , 'limit 20', 'limit 20 offset 10' , 'limit 200'] + self.limit1_where = ['limit 1 offset 1' , 'limit 1' ] + self.limit_u_where = ['limit 100 offset 10' , 'limit 50' , 'limit 100' , 'limit 10' ] + + # slimit soffset where + self.slimit_where = ['slimit 1 soffset 1' , 'slimit 1' , 'slimit 2 soffset 1' , 'slimit 2'] + self.slimit1_where = ['slimit 2 soffset 1' , 'slimit 1' ] + + self.calc_select_all = ['bottom(q_int,20)' , 'bottom(q_bigint,20)' , 'bottom(q_smallint,20)' , 'bottom(q_tinyint,20)' ,'bottom(q_float,20)' , 'bottom(q_double,20)' , + 'top(q_int,20)' , 'top(q_bigint,20)' , 'top(q_smallint,20)' ,'top(q_tinyint,20)' ,'top(q_float,20)' ,'top(q_double,20)' , + 'first(q_int)' , 'first(q_bigint)' , 'first(q_smallint)' , 'first(q_tinyint)' , 'first(q_float)' ,'first(q_double)' ,'first(q_binary)' ,'first(q_nchar)' ,'first(q_bool)' ,'first(q_ts)' , + 'last(q_int)' , 'last(q_bigint)' , 'last(q_smallint)' , 'last(q_tinyint)' , 'last(q_float)' ,'last(q_double)' , 'last(q_binary)' ,'last(q_nchar)' ,'last(q_bool)' ,'last(q_ts)' , + 'min(q_int)' , 'min(q_bigint)' , 'min(q_smallint)' , 'min(q_tinyint)' , 'min(q_float)' ,'min(q_double)' , + 'max(q_int)' , 'max(q_bigint)' , 'max(q_smallint)' , 'max(q_tinyint)' ,'max(q_float)' ,'max(q_double)' , + 'apercentile(q_int,20)' , 'apercentile(q_bigint,20)' ,'apercentile(q_smallint,20)' ,'apercentile(q_tinyint,20)' ,'apercentile(q_float,20)' ,'apercentile(q_double,20)' , + 'last_row(q_int)' , 'last_row(q_bigint)' , 'last_row(q_smallint)' , 'last_row(q_tinyint)' , 'last_row(q_float)' , + 'last_row(q_double)' , 'last_row(q_bool)' ,'last_row(q_binary)' ,'last_row(q_nchar)' ,'last_row(q_ts)', + 'bottom(q_int_null,20)' , 'bottom(q_bigint_null,20)' , 'bottom(q_smallint_null,20)' , 'bottom(q_tinyint_null,20)' ,'bottom(q_float_null,20)' , 'bottom(q_double_null,20)' , + 'top(q_int_null,20)' , 'top(q_bigint_null,20)' , 'top(q_smallint_null,20)' ,'top(q_tinyint_null,20)' ,'top(q_float_null,20)' ,'top(q_double_null,20)' , + 'first(q_int_null)' , 'first(q_bigint_null)' , 'first(q_smallint_null)' , 'first(q_tinyint_null)' , 'first(q_float_null)' ,'first(q_double_null)' ,'first(q_binary_null)' ,'first(q_nchar_null)' ,'first(q_bool_null)' ,'first(q_ts_null)' , + 'last(q_int_null)' , 'last(q_bigint_null)' , 'last(q_smallint_null)' , 'last(q_tinyint_null)' , 'last(q_float_null)' ,'last(q_double_null)' , 'last(q_binary_null)' ,'last(q_nchar_null)' ,'last(q_bool_null)' ,'last(q_ts_null)' , + 'min(q_int_null)' , 'min(q_bigint_null)' , 'min(q_smallint_null)' , 'min(q_tinyint_null)' , 'min(q_float_null)' ,'min(q_double_null)' , + 'max(q_int_null)' , 'max(q_bigint_null)' , 'max(q_smallint_null)' , 'max(q_tinyint_null)' ,'max(q_float_null)' ,'max(q_double_null)' , + 'last_row(q_int_null)' , 'last_row(q_bigint_null)' , 'last_row(q_smallint_null)' , 'last_row(q_tinyint_null)' , 'last_row(q_float_null)' , + 'last_row(q_double_null)' , 'last_row(q_bool_null)' ,'last_row(q_binary_null)' ,'last_row(q_nchar_null)' ,'last_row(q_ts_null)', + 'apercentile(q_int_null,20)' , 'apercentile(q_bigint_null,20)' ,'apercentile(q_smallint_null,20)' ,'apercentile(q_tinyint_null,20)' ,'apercentile(q_float_null,20)' ,'apercentile(q_double_null,20)' ,] + + self.calc_select_in_ts = ['bottom(q_int,20)' , 'bottom(q_bigint,20)' , 'bottom(q_smallint,20)' , 'bottom(q_tinyint,20)' ,'bottom(q_float,20)' , 'bottom(q_double,20)' , + 'top(q_int,20)' , 'top(q_bigint,20)' , 'top(q_smallint,20)' ,'top(q_tinyint,20)' ,'top(q_float,20)' ,'top(q_double,20)' , + 'bottom(q_int_null,20)' , 'bottom(q_bigint_null,20)' , 'bottom(q_smallint_null,20)' , 'bottom(q_tinyint_null,20)' ,'bottom(q_float_null,20)' , 'bottom(q_double_null,20)' , + 'top(q_int_null,20)' , 'top(q_bigint_null,20)' , 'top(q_smallint_null,20)' ,'top(q_tinyint_null,20)' ,'top(q_float_null,20)' ,'top(q_double_null,20)' , + 'first(q_int)' , 'first(q_bigint)' , 'first(q_smallint)' , 'first(q_tinyint)' , 'first(q_float)' ,'first(q_double)' ,'first(q_binary)' ,'first(q_nchar)' ,'first(q_bool)' ,'first(q_ts)' , + 'last(q_int)' , 'last(q_bigint)' , 'last(q_smallint)' , 'last(q_tinyint)' , 'last(q_float)' ,'last(q_double)' , 'last(q_binary)' ,'last(q_nchar)' ,'last(q_bool)' ,'last(q_ts)' , + 'first(q_int_null)' , 'first(q_bigint_null)' , 'first(q_smallint_null)' , 'first(q_tinyint_null)' , 'first(q_float_null)' ,'first(q_double_null)' ,'first(q_binary_null)' ,'first(q_nchar_null)' ,'first(q_bool_null)' ,'first(q_ts_null)' , + 'last(q_int_null)' , 'last(q_bigint_null)' , 'last(q_smallint_null)' , 'last(q_tinyint_null)' , 'last(q_float_null)' ,'last(q_double_null)' , 'last(q_binary_null)' ,'last(q_nchar_null)' ,'last(q_bool_null)' ,'last(q_ts_null)' ] + + self.calc_select_in = ['min(q_int)' , 'min(q_bigint)' , 'min(q_smallint)' , 'min(q_tinyint)' , 'min(q_float)' ,'min(q_double)' , + 'max(q_int)' , 'max(q_bigint)' , 'max(q_smallint)' , 'max(q_tinyint)' ,'max(q_float)' ,'max(q_double)' , + 'apercentile(q_int,20)' , 'apercentile(q_bigint,20)' ,'apercentile(q_smallint,20)' ,'apercentile(q_tinyint,20)' ,'apercentile(q_float,20)' ,'apercentile(q_double,20)' , + 'last_row(q_int)' , 'last_row(q_bigint)' , 'last_row(q_smallint)' , 'last_row(q_tinyint)' , 'last_row(q_float)' , + 'last_row(q_double)' , 'last_row(q_bool)' ,'last_row(q_binary)' ,'last_row(q_nchar)' ,'last_row(q_ts)', + 'min(q_int_null)' , 'min(q_bigint_null)' , 'min(q_smallint_null)' , 'min(q_tinyint_null)' , 'min(q_float_null)' ,'min(q_double_null)' , + 'max(q_int_null)' , 'max(q_bigint_null)' , 'max(q_smallint_null)' , 'max(q_tinyint_null)' ,'max(q_float_null)' ,'max(q_double_null)' , + 'apercentile(q_int_null,20)' , 'apercentile(q_bigint_null,20)' ,'apercentile(q_smallint_null,20)' ,'apercentile(q_tinyint_null,20)' ,'apercentile(q_float_null,20)' ,'apercentile(q_double_null,20)' , + 'last_row(q_int_null)' , 'last_row(q_bigint_null)' , 'last_row(q_smallint_null)' , 'last_row(q_tinyint_null)' , 'last_row(q_float_null)' , + 'last_row(q_double_null)' , 'last_row(q_bool_null)' ,'last_row(q_binary_null)' ,'last_row(q_nchar_null)' ,'last_row(q_ts_null)'] + + self.calc_select_not_support_ts = ['first(q_int)' , 'first(q_bigint)' , 'first(q_smallint)' , 'first(q_tinyint)' , 'first(q_float)' ,'first(q_double)' ,'first(q_binary)' ,'first(q_nchar)' ,'first(q_bool)' ,'first(q_ts)' , + 'last(q_int)' , 'last(q_bigint)' , 'last(q_smallint)' , 'last(q_tinyint)' , 'last(q_float)' ,'last(q_double)' , 'last(q_binary)' ,'last(q_nchar)' ,'last(q_bool)' ,'last(q_ts)' , + 'last_row(q_int)' , 'last_row(q_bigint)' , 'last_row(q_smallint)' , 'last_row(q_tinyint)' , 'last_row(q_float)' , + 'last_row(q_double)' , 'last_row(q_bool)' ,'last_row(q_binary)' ,'last_row(q_nchar)' ,'last_row(q_ts)', + 'apercentile(q_int,20)' , 'apercentile(q_bigint,20)' ,'apercentile(q_smallint,20)' ,'apercentile(q_tinyint,20)' ,'apercentile(q_float,20)' ,'apercentile(q_double,20)', + 'first(q_int_null)' , 'first(q_bigint_null)' , 'first(q_smallint_null)' , 'first(q_tinyint_null)' , 'first(q_float_null)' ,'first(q_double_null)' ,'first(q_binary_null)' ,'first(q_nchar_null)' ,'first(q_bool_null)' ,'first(q_ts_null)' , + 'last(q_int_null)' , 'last(q_bigint_null)' , 'last(q_smallint_null)' , 'last(q_tinyint_null)' , 'last(q_float_null)' ,'last(q_double_null)' , 'last(q_binary_null)' ,'last(q_nchar_null)' ,'last(q_bool_null)' ,'last(q_ts_null)' , + 'last_row(q_int_null)' , 'last_row(q_bigint_null)' , 'last_row(q_smallint_null)' , 'last_row(q_tinyint_null)' , 'last_row(q_float_null)' , + 'last_row(q_double_null)' , 'last_row(q_bool_null)' ,'last_row(q_binary_null)' ,'last_row(q_nchar_null)' ,'last_row(q_ts_null)', + 'apercentile(q_int_null,20)' , 'apercentile(q_bigint_null,20)' ,'apercentile(q_smallint_null,20)' ,'apercentile(q_tinyint_null,20)' ,'apercentile(q_float_null,20)' ,'apercentile(q_double_null,20)'] + + self.calc_select_support_ts = ['bottom(q_int,20)' , 'bottom(q_bigint,20)' , 'bottom(q_smallint,20)' , 'bottom(q_tinyint,20)' ,'bottom(q_float,20)' , 'bottom(q_double,20)' , + 'top(q_int,20)' , 'top(q_bigint,20)' , 'top(q_smallint,20)' ,'top(q_tinyint,20)' ,'top(q_float,20)' ,'top(q_double,20)' , + 'bottom(q_int_null,20)' , 'bottom(q_bigint_null,20)' , 'bottom(q_smallint_null,20)' , 'bottom(q_tinyint_null,20)' ,'bottom(q_float_null,20)' , 'bottom(q_double_null,20)' , + 'top(q_int_null,20)' , 'top(q_bigint_null,20)' , 'top(q_smallint_null,20)' ,'top(q_tinyint_null,20)' ,'top(q_float_null,20)' ,'top(q_double_null,20)' , + 'min(q_int)' , 'min(q_bigint)' , 'min(q_smallint)' , 'min(q_tinyint)' , 'min(q_float)' ,'min(q_double)' , + 'max(q_int)' , 'max(q_bigint)' , 'max(q_smallint)' , 'max(q_tinyint)' ,'max(q_float)' ,'max(q_double)' , + 'min(q_int_null)' , 'min(q_bigint_null)' , 'min(q_smallint_null)' , 'min(q_tinyint_null)' , 'min(q_float_null)' ,'min(q_double_null)' , + 'max(q_int_null)' , 'max(q_bigint_null)' , 'max(q_smallint_null)' , 'max(q_tinyint_null)' ,'max(q_float_null)' ,'max(q_double_null)'] + + self.calc_select_regular = [ 'PERCENTILE(q_int,10)' ,'PERCENTILE(q_bigint,20)' , 'PERCENTILE(q_smallint,30)' ,'PERCENTILE(q_tinyint,40)' ,'PERCENTILE(q_float,50)' ,'PERCENTILE(q_double,60)', + 'PERCENTILE(q_int_null,10)' ,'PERCENTILE(q_bigint_null,20)' , 'PERCENTILE(q_smallint_null,30)' ,'PERCENTILE(q_tinyint_null,40)' ,'PERCENTILE(q_float_null,50)' ,'PERCENTILE(q_double_null,60)'] + + + self.calc_select_fill = ['INTERP(q_int)' ,'INTERP(q_bigint)' ,'INTERP(q_smallint)' ,'INTERP(q_tinyint)', 'INTERP(q_float)' ,'INTERP(q_double)'] + self.interp_where = ['ts = now' , 'ts = \'2020-09-13 20:26:40.000\'' , 'ts = \'2020-09-13 20:26:40.009\'' ,'tbname in (\'table_1\') and ts = now' ,'tbname in (\'table_0\' ,\'table_1\',\'table_2\',\'table_3\',\'table_4\',\'table_5\') and ts = \'2020-09-13 20:26:40.000\'','tbname like \'table%\' and ts = \'2020-09-13 20:26:40.002\''] + + #two table join + self.calc_select_in_ts_j = ['bottom(t1.q_int,20)' , 'bottom(t1.q_bigint,20)' , 'bottom(t1.q_smallint,20)' , 'bottom(t1.q_tinyint,20)' ,'bottom(t1.q_float,20)' , 'bottom(t1.q_double,20)' , + 'top(t1.q_int,20)' , 'top(t1.q_bigint,20)' , 'top(t1.q_smallint,20)' ,'top(t1.q_tinyint,20)' ,'top(t1.q_float,20)' ,'top(t1.q_double,20)' , + 'first(t1.q_int)' , 'first(t1.q_bigint)' , 'first(t1.q_smallint)' , 'first(t1.q_tinyint)' , 'first(t1.q_float)' ,'first(t1.q_double)' ,'first(t1.q_binary)' ,'first(t1.q_nchar)' ,'first(t1.q_bool)' ,'first(t1.q_ts)' , + 'last(t1.q_int)' , 'last(t1.q_bigint)' , 'last(t1.q_smallint)' , 'last(t1.q_tinyint)' , 'last(t1.q_float)' ,'last(t1.q_double)' , 'last(t1.q_binary)' ,'last(t1.q_nchar)' ,'last(t1.q_bool)' ,'last(t1.q_ts)' , + 'bottom(t2.q_int,20)' , 'bottom(t2.q_bigint,20)' , 'bottom(t2.q_smallint,20)' , 'bottom(t2.q_tinyint,20)' ,'bottom(t2.q_float,20)' , 'bottom(t2.q_double,20)' , + 'top(t2.q_int,20)' , 'top(t2.q_bigint,20)' , 'top(t2.q_smallint,20)' ,'top(t2.q_tinyint,20)' ,'top(t2.q_float,20)' ,'top(t2.q_double,20)' , + 'first(t2.q_int)' , 'first(t2.q_bigint)' , 'first(t2.q_smallint)' , 'first(t2.q_tinyint)' , 'first(t2.q_float)' ,'first(t2.q_double)' ,'first(t2.q_binary)' ,'first(t2.q_nchar)' ,'first(t2.q_bool)' ,'first(t2.q_ts)' , + 'last(t2.q_int)' , 'last(t2.q_bigint)' , 'last(t2.q_smallint)' , 'last(t2.q_tinyint)' , 'last(t2.q_float)' ,'last(t2.q_double)' , 'last(t2.q_binary)' ,'last(t2.q_nchar)' ,'last(t2.q_bool)' ,'last(t2.q_ts)', + 'bottom(t1.q_int_null,20)' , 'bottom(t1.q_bigint_null,20)' , 'bottom(t1.q_smallint_null,20)' , 'bottom(t1.q_tinyint_null,20)' ,'bottom(t1.q_float_null,20)' , 'bottom(t1.q_double_null,20)' , + 'top(t1.q_int_null,20)' , 'top(t1.q_bigint_null,20)' , 'top(t1.q_smallint_null,20)' ,'top(t1.q_tinyint_null,20)' ,'top(t1.q_float_null,20)' ,'top(t1.q_double_null,20)' , + 'first(t1.q_int_null)' , 'first(t1.q_bigint_null)' , 'first(t1.q_smallint_null)' , 'first(t1.q_tinyint_null)' , 'first(t1.q_float_null)' ,'first(t1.q_double_null)' ,'first(t1.q_binary_null)' ,'first(t1.q_nchar_null))' ,'first(t1.q_bool_null)' ,'first(t1.q_ts_null)' , + 'last(t1.q_int_null)' , 'last(t1.q_bigint_null)' , 'last(t1.q_smallint_null)' , 'last(t1.q_tinyint_null)' , 'last(t1.q_float_null)' ,'last(t1.q_double_null)' , 'last(t1.q_binary_null)' ,'last(t1.q_nchar_null))' ,'last(t1.q_bool_null)' ,'last(t1.q_ts_null)' , + 'bottom(t2.q_int_null,20)' , 'bottom(t2.q_bigint_null,20)' , 'bottom(t2.q_smallint_null,20)' , 'bottom(t2.q_tinyint_null,20)' ,'bottom(t2.q_float_null,20)' , 'bottom(t2.q_double_null,20)' , + 'top(t2.q_int_null,20)' , 'top(t2.q_bigint_null,20)' , 'top(t2.q_smallint_null,20)' ,'top(t2.q_tinyint_null,20)' ,'top(t2.q_float_null,20)' ,'top(t2.q_double_null,20)' , + 'first(t2.q_int_null)' , 'first(t2.q_bigint_null)' , 'first(t2.q_smallint_null)' , 'first(t2.q_tinyint_null)' , 'first(t2.q_float_null)' ,'first(t2.q_double_null)' ,'first(t2.q_binary_null)' ,'first(t2.q_nchar_null))' ,'first(t2.q_bool_null)' ,'first(t2.q_ts_null)' , + 'last(t2.q_int_null)' , 'last(t2.q_bigint_null)' , 'last(t2.q_smallint_null)' , 'last(t2.q_tinyint_null)' , 'last(t2.q_float_null)' ,'last(t2.q_double_null)' , 'last(t2.q_binary_null)' ,'last(t2.q_nchar_null))' ,'last(t2.q_bool_null)' ,'last(t2.q_ts_null)'] + + self.calc_select_in_support_ts_j = ['bottom(t1.q_int,20)' , 'bottom(t1.q_bigint,20)' , 'bottom(t1.q_smallint,20)' , 'bottom(t1.q_tinyint,20)' ,'bottom(t1.q_float,20)' , 'bottom(t1.q_double,20)' , + 'top(t1.q_int,20)' , 'top(t1.q_bigint,20)' , 'top(t1.q_smallint,20)' ,'top(t1.q_tinyint,20)' ,'top(t1.q_float,20)' ,'top(t1.q_double,20)' , + 'min(t1.q_int)' , 'min(t1.q_bigint)' , 'min(t1.q_smallint)' , 'min(t1.q_tinyint)' , 'min(t1.q_float)' ,'min(t1.q_double)' , + 'max(t1.q_int)' , 'max(t1.q_bigint)' , 'max(t1.q_smallint)' , 'max(t1.q_tinyint)' ,'max(t1.q_float)' ,'max(t1.q_double)' , + 'bottom(t2.q_int,20)' , 'bottom(t2.q_bigint,20)' , 'bottom(t2.q_smallint,20)' , 'bottom(t2.q_tinyint,20)' ,'bottom(t2.q_float,20)' , 'bottom(t2.q_double,20)' , + 'top(t2.q_int,20)' , 'top(t2.q_bigint,20)' , 'top(t2.q_smallint,20)' ,'top(t2.q_tinyint,20)' ,'top(t2.q_float,20)' ,'top(t2.q_double,20)' , + 'min(t2.q_int)' , 'min(t2.q_bigint)' , 'min(t2.q_smallint)' , 'min(t2.q_tinyint)' , 'min(t2.q_float)' ,'min(t2.q_double)' , + 'max(t2.q_int)' , 'max(t2.q_bigint)' , 'max(t2.q_smallint)' , 'max(t2.q_tinyint)' ,'max(t2.q_float)' ,'max(t2.q_double)' , + 'bottom(t1.q_int_null,20)' , 'bottom(t1.q_bigint_null,20)' , 'bottom(t1.q_smallint_null,20)' , 'bottom(t1.q_tinyint_null,20)' ,'bottom(t1.q_float_null,20)' , 'bottom(t1.q_double_null,20)' , + 'top(t1.q_int_null,20)' , 'top(t1.q_bigint_null,20)' , 'top(t1.q_smallint_null,20)' ,'top(t1.q_tinyint_null,20)' ,'top(t1.q_float_null,20)' ,'top(t1.q_double_null,20)' , + 'bottom(t2.q_int_null,20)' , 'bottom(t2.q_bigint_null,20)' , 'bottom(t2.q_smallint_null,20)' , 'bottom(t2.q_tinyint_null,20)' ,'bottom(t2.q_float_null,20)' , 'bottom(t2.q_double_null,20)' , + 'top(t2.q_int_null,20)' , 'top(t2.q_bigint_null,20)' , 'top(t2.q_smallint_null,20)' ,'top(t2.q_tinyint_null,20)' ,'top(t2.q_float_null,20)' ,'top(t2.q_double_null,20)' , + 'min(t1.q_int_null)' , 'min(t1.q_bigint_null)' , 'min(t1.q_smallint_null)' , 'min(t1.q_tinyint_null)' , 'min(t1.q_float_null)' ,'min(t1.q_double_null)' , + 'max(t1.q_int_null)' , 'max(t1.q_bigint_null)' , 'max(t1.q_smallint_null)' , 'max(t1.q_tinyint_null)' ,'max(t1.q_float_null)' ,'max(t1.q_double_null)' , + 'min(t2.q_int_null)' , 'min(t2.q_bigint_null)' , 'min(t2.q_smallint_null)' , 'min(t2.q_tinyint_null)' , 'min(t2.q_float_null)' ,'min(t2.q_double_null)' , + 'max(t2.q_int_null)' , 'max(t2.q_bigint_null)' , 'max(t2.q_smallint_null)' , 'max(t2.q_tinyint_null)' ,'max(t2.q_float_null)' ,'max(t2.q_double_null)' ] + + self.calc_select_in_not_support_ts_j = ['apercentile(t1.q_int,20)' , 'apercentile(t1.q_bigint,20)' ,'apercentile(t1.q_smallint,20)' ,'apercentile(t1.q_tinyint,20)' ,'apercentile(t1.q_float,20)' ,'apercentile(t1.q_double,20)' , + 'apercentile(t1.q_int_null,20)' , 'apercentile(t1.q_bigint_null,20)' ,'apercentile(t1.q_smallint_null,20)' ,'apercentile(t1.q_tinyint_null,20)' ,'apercentile(t1.q_float_null,20)' ,'apercentile(t1.q_double_null,20)' , + 'last_row(t1.q_int)' , 'last_row(t1.q_bigint)' , 'last_row(t1.q_smallint)' , 'last_row(t1.q_tinyint)' , 'last_row(t1.q_float)' , + 'last_row(t1.q_double)' , 'last_row(t1.q_bool)' ,'last_row(t1.q_binary)' ,'last_row(t1.q_nchar)' ,'last_row(t1.q_ts)' , + 'last_row(t1.q_int_null)' , 'last_row(t1.q_bigint_null)' , 'last_row(t1.q_smallint_null)' , 'last_row(t1.q_tinyint_null)' , 'last_row(t1.q_float_null)' , + 'last_row(t1.q_double_null)' , 'last_row(t1.q_bool_null)' ,'last_row(t1.q_binary_null)' ,'last_row(t1.q_nchar_null)' ,'last_row(t1.q_ts_null)' , + 'apercentile(t2.q_int,20)' , 'apercentile(t2.q_bigint,20)' ,'apercentile(t2.q_smallint,20)' ,'apercentile(t2.q_tinyint,20)' ,'apercentile(t2.q_float,20)' ,'apercentile(t2.q_double,20)' , + 'apercentile(t2.q_int_null,20)' , 'apercentile(t2.q_bigint_null,20)' ,'apercentile(t2.q_smallint_null,20)' ,'apercentile(t2.q_tinyint_null,20)' ,'apercentile(t2.q_float_null,20)' ,'apercentile(t2.q_double_null,20)' , + 'last_row(t2.q_int)' , 'last_row(t2.q_bigint)' , 'last_row(t2.q_smallint)' , 'last_row(t2.q_tinyint)' , 'last_row(t2.q_float)' , + 'last_row(t2.q_double)' , 'last_row(t2.q_bool)' ,'last_row(t2.q_binary)' ,'last_row(t2.q_nchar)' ,'last_row(t2.q_ts)', + 'last_row(t2.q_int_null)' , 'last_row(t2.q_bigint_null)' , 'last_row(t2.q_smallint_null)' , 'last_row(t2.q_tinyint_null)' , 'last_row(t2.q_float_null)' , + 'last_row(t2.q_double_null)' , 'last_row(t2.q_bool_null)' ,'last_row(t2.q_binary_null)' ,'last_row(t2.q_nchar_null)' ,'last_row(t2.q_ts_null)'] + + self.calc_select_in_j = ['min(t1.q_int)' , 'min(t1.q_bigint)' , 'min(t1.q_smallint)' , 'min(t1.q_tinyint)' , 'min(t1.q_float)' ,'min(t1.q_double)' , + 'max(t1.q_int)' , 'max(t1.q_bigint)' , 'max(t1.q_smallint)' , 'max(t1.q_tinyint)' ,'max(t1.q_float)' ,'max(t1.q_double)' , + 'apercentile(t1.q_int,20)' , 'apercentile(t1.q_bigint,20)' ,'apercentile(t1.q_smallint,20)' ,'apercentile(t1.q_tinyint,20)' ,'apercentile(t1.q_float,20)' ,'apercentile(t1.q_double,20)' , + 'min(t1.q_int_null)' , 'min(t1.q_bigint_null)' , 'min(t1.q_smallint_null)' , 'min(t1.q_tinyint_null)' , 'min(t1.q_float_null)' ,'min(t1.q_double_null)' , + 'max(t1.q_int_null)' , 'max(t1.q_bigint_null)' , 'max(t1.q_smallint_null)' , 'max(t1.q_tinyint_null)' ,'max(t1.q_float_null)' ,'max(t1.q_double_null)' , + 'apercentile(t1.q_int_null,20)' , 'apercentile(t1.q_bigint_null,20)' ,'apercentile(t1.q_smallint_null,20)' ,'apercentile(t1.q_tinyint_null,20)' ,'apercentile(t1.q_float_null,20)' ,'apercentile(t1.q_double_null,20)' , + 'last_row(t1.q_int)' , 'last_row(t1.q_bigint)' , 'last_row(t1.q_smallint)' , 'last_row(t1.q_tinyint)' , 'last_row(t1.q_float)' , + 'last_row(t1.q_double)' , 'last_row(t1.q_bool)' ,'last_row(t1.q_binary)' ,'last_row(t1.q_nchar)' ,'last_row(t1.q_ts)' , + 'min(t2.q_int)' , 'min(t2.q_bigint)' , 'min(t2.q_smallint)' , 'min(t2.q_tinyint)' , 'min(t2.q_float)' ,'min(t2.q_double)' , + 'max(t2.q_int)' , 'max(t2.q_bigint)' , 'max(t2.q_smallint)' , 'max(t2.q_tinyint)' ,'max(t2.q_float)' ,'max(t2.q_double)' , + 'last_row(t1.q_int_null)' , 'last_row(t1.q_bigint_null)' , 'last_row(t1.q_smallint_null)' , 'last_row(t1.q_tinyint_null)' , 'last_row(t1.q_float_null)' , + 'last_row(t1.q_double_null)' , 'last_row(t1.q_bool_null)' ,'last_row(t1.q_binary_null)' ,'last_row(t1.q_nchar_null)' ,'last_row(t1.q_ts_null)' , + 'min(t2.q_int_null)' , 'min(t2.q_bigint_null)' , 'min(t2.q_smallint_null)' , 'min(t2.q_tinyint_null)' , 'min(t2.q_float_null)' ,'min(t2.q_double_null)' , + 'max(t2.q_int_null)' , 'max(t2.q_bigint_null)' , 'max(t2.q_smallint_null)' , 'max(t2.q_tinyint_null)' ,'max(t2.q_float_null)' ,'max(t2.q_double_null)' , + 'apercentile(t2.q_int,20)' , 'apercentile(t2.q_bigint,20)' ,'apercentile(t2.q_smallint,20)' ,'apercentile(t2.q_tinyint,20)' ,'apercentile(t2.q_float,20)' ,'apercentile(t2.q_double,20)' , + 'apercentile(t2.q_int_null,20)' , 'apercentile(t2.q_bigint_null,20)' ,'apercentile(t2.q_smallint_null,20)' ,'apercentile(t2.q_tinyint_null,20)' ,'apercentile(t2.q_float_null,20)' ,'apercentile(t2.q_double_null,20)' , + 'last_row(t2.q_int)' , 'last_row(t2.q_bigint)' , 'last_row(t2.q_smallint)' , 'last_row(t2.q_tinyint)' , 'last_row(t2.q_float)' , + 'last_row(t2.q_double)' , 'last_row(t2.q_bool)' ,'last_row(t2.q_binary)' ,'last_row(t2.q_nchar)' ,'last_row(t2.q_ts)', + 'last_row(t2.q_int_null)' , 'last_row(t2.q_bigint_null)' , 'last_row(t2.q_smallint_null)' , 'last_row(t2.q_tinyint_null)' , 'last_row(t2.q_float_null)' , + 'last_row(t2.q_double_null)' , 'last_row(t2.q_bool_null)' ,'last_row(t2.q_binary_null)' ,'last_row(t2.q_nchar_null)' ,'last_row(t2.q_ts_null)'] + self.calc_select_all_j = self.calc_select_in_ts_j + self.calc_select_in_j + + self.calc_select_regular_j = [ 'PERCENTILE(t1.q_int,10)' ,'PERCENTILE(t1.q_bigint,20)' , 'PERCENTILE(t1.q_smallint,30)' ,'PERCENTILE(t1.q_tinyint,40)' ,'PERCENTILE(t1.q_float,50)' ,'PERCENTILE(t1.q_double,60)' , + 'PERCENTILE(t2.q_int,10)' ,'PERCENTILE(t2.q_bigint,20)' , 'PERCENTILE(t2.q_smallint,30)' ,'PERCENTILE(t2.q_tinyint,40)' ,'PERCENTILE(t2.q_float,50)' ,'PERCENTILE(t2.q_double,60)', + 'PERCENTILE(t1.q_int_null,10)' ,'PERCENTILE(t1.q_bigint_null,20)' , 'PERCENTILE(t1.q_smallint_null,30)' ,'PERCENTILE(t1.q_tinyint_null,40)' ,'PERCENTILE(t1.q_float_null,50)' ,'PERCENTILE(t1.q_double_null,60)' , + 'PERCENTILE(t2.q_int_null,10)' ,'PERCENTILE(t2.q_bigint_null,20)' , 'PERCENTILE(t2.q_smallint_null,30)' ,'PERCENTILE(t2.q_tinyint_null,40)' ,'PERCENTILE(t2.q_float_null,50)' ,'PERCENTILE(t2.q_double_null,60)'] + + + self.calc_select_fill_j = ['INTERP(t1.q_int)' ,'INTERP(t1.q_bigint)' ,'INTERP(t1.q_smallint)' ,'INTERP(t1.q_tinyint)', 'INTERP(t1.q_float)' ,'INTERP(t1.q_double)' , + 'INTERP(t2.q_int)' ,'INTERP(t2.q_bigint)' ,'INTERP(t2.q_smallint)' ,'INTERP(t2.q_tinyint)', 'INTERP(t2.q_float)' ,'INTERP(t2.q_double)'] + self.interp_where_j = ['t1.ts = now' , 't1.ts = \'2020-09-13 20:26:40.000\'' , 't1.ts = \'2020-09-13 20:26:40.009\'' ,'t2.ts = now' , 't2.ts = \'2020-09-13 20:26:40.000\'' , 't2.ts = \'2020-09-13 20:26:40.009\'' , + 't1.tbname in (\'table_1\') and t1.ts = now' ,'t1.tbname in (\'table_0\' ,\'table_1\',\'table_2\',\'table_3\',\'table_4\',\'table_5\') and t1.ts = \'2020-09-13 20:26:40.000\'','t1.tbname like \'table%\' and t1.ts = \'2020-09-13 20:26:40.002\'', + 't2.tbname in (\'table_1\') and t2.ts = now' ,'t2.tbname in (\'table_0\' ,\'table_1\',\'table_2\',\'table_3\',\'table_4\',\'table_5\') and t2.ts = \'2020-09-13 20:26:40.000\'','t2.tbname like \'table%\' and t2.ts = \'2020-09-13 20:26:40.002\''] + + self.calc_aggregate_all = ['count(*)' , 'count(q_int)' ,'count(q_bigint)' , 'count(q_smallint)' ,'count(q_tinyint)' ,'count(q_float)' , + 'count(q_double)' ,'count(q_binary)' ,'count(q_nchar)' ,'count(q_bool)' ,'count(q_ts)' , + 'avg(q_int)' ,'avg(q_bigint)' , 'avg(q_smallint)' ,'avg(q_tinyint)' ,'avg(q_float)' ,'avg(q_double)' , + 'sum(q_int)' ,'sum(q_bigint)' , 'sum(q_smallint)' ,'sum(q_tinyint)' ,'sum(q_float)' ,'sum(q_double)' , + 'STDDEV(q_int)' ,'STDDEV(q_bigint)' , 'STDDEV(q_smallint)' ,'STDDEV(q_tinyint)' ,'STDDEV(q_float)' ,'STDDEV(q_double)', + 'APERCENTILE(q_int,10)' ,'APERCENTILE(q_bigint,20)' , 'APERCENTILE(q_smallint,30)' ,'APERCENTILE(q_tinyint,40)' ,'APERCENTILE(q_float,50)' ,'APERCENTILE(q_double,60)', + 'count(q_int_null)' ,'count(q_bigint_null)' , 'count(q_smallint_null)' ,'count(q_tinyint_null)' ,'count(q_float_null)' , + 'count(q_double_null)' ,'count(q_binary_null)' ,'count(q_nchar_null)' ,'count(q_bool_null)' ,'count(q_ts_null)' , + 'avg(q_int_null)' ,'avg(q_bigint_null)' , 'avg(q_smallint_null)' ,'avg(q_tinyint_null)' ,'avg(q_float_null)' ,'avg(q_double_null)' , + 'sum(q_int_null)' ,'sum(q_bigint_null)' , 'sum(q_smallint_null)' ,'sum(q_tinyint_null)' ,'sum(q_float_null)' ,'sum(q_double_null)' , + 'STDDEV(q_int_null)' ,'STDDEV(q_bigint_null)' , 'STDDEV(q_smallint_null)' ,'STDDEV(q_tinyint_null)' ,'STDDEV(q_float_null)' ,'STDDEV(q_double_null)', + 'APERCENTILE(q_int_null,10)' ,'APERCENTILE(q_bigint_null,20)' , 'APERCENTILE(q_smallint_null,30)' ,'APERCENTILE(q_tinyint_null,40)' ,'APERCENTILE(q_float_null,50)' ,'APERCENTILE(q_double_null,60)'] + + self.calc_aggregate_regular = ['twa(q_int)' ,'twa(q_bigint)' , 'twa(q_smallint)' ,'twa(q_tinyint)' ,'twa (q_float)' ,'twa(q_double)' , + 'IRATE(q_int)' ,'IRATE(q_bigint)' , 'IRATE(q_smallint)' ,'IRATE(q_tinyint)' ,'IRATE (q_float)' ,'IRATE(q_double)' , + 'twa(q_int_null)' ,'twa(q_bigint_null)' , 'twa(q_smallint_null)' ,'twa(q_tinyint_null)' ,'twa (q_float_null)' ,'twa(q_double_null)' , + 'IRATE(q_int_null)' ,'IRATE(q_bigint_null)' , 'IRATE(q_smallint_null)' ,'IRATE(q_tinyint_null)' ,'IRATE (q_float_null)' ,'IRATE(q_double_null)' , + 'LEASTSQUARES(q_int,15,3)' , 'LEASTSQUARES(q_bigint,10,1)' , 'LEASTSQUARES(q_smallint,20,3)' ,'LEASTSQUARES(q_tinyint,10,4)' ,'LEASTSQUARES(q_float,6,4)' ,'LEASTSQUARES(q_double,3,1)' , + 'PERCENTILE(q_int,10)' ,'PERCENTILE(q_bigint,20)' , 'PERCENTILE(q_smallint,30)' ,'PERCENTILE(q_tinyint,40)' ,'PERCENTILE(q_float,50)' ,'PERCENTILE(q_double,60)', + 'LEASTSQUARES(q_int_null,15,3)' , 'LEASTSQUARES(q_bigint_null,10,1)' , 'LEASTSQUARES(q_smallint_null,20,3)' ,'LEASTSQUARES(q_tinyint_null,10,4)' ,'LEASTSQUARES(q_float_null,6,4)' ,'LEASTSQUARES(q_double_null,3,1)' , + 'PERCENTILE(q_int_null,10)' ,'PERCENTILE(q_bigint_null,20)' , 'PERCENTILE(q_smallint_null,30)' ,'PERCENTILE(q_tinyint_null,40)' ,'PERCENTILE(q_float_null,50)' ,'PERCENTILE(q_double_null,60)'] + + self.calc_aggregate_groupbytbname = ['twa(q_int)' ,'twa(q_bigint)' , 'twa(q_smallint)' ,'twa(q_tinyint)' ,'twa (q_float)' ,'twa(q_double)' , + 'IRATE(q_int)' ,'IRATE(q_bigint)' , 'IRATE(q_smallint)' ,'IRATE(q_tinyint)' ,'IRATE (q_float)' ,'IRATE(q_double)', + 'twa(q_int_null)' ,'twa(q_bigint_null)' , 'twa(q_smallint_null)' ,'twa(q_tinyint_null)' ,'twa (q_float_null)' ,'twa(q_double_null)' , + 'IRATE(q_int_null)' ,'IRATE(q_bigint_null)' , 'IRATE(q_smallint_null)' ,'IRATE(q_tinyint_null)' ,'IRATE (q_float_null)' ,'IRATE(q_double_null)'] + + #two table join + self.calc_aggregate_all_j = ['count(t1.*)' , 'count(t1.q_int)' ,'count(t1.q_bigint)' , 'count(t1.q_smallint)' ,'count(t1.q_tinyint)' ,'count(t1.q_float)' , + 'count(t1.q_double)' ,'count(t1.q_binary)' ,'count(t1.q_nchar)' ,'count(t1.q_bool)' ,'count(t1.q_ts)' , + 'avg(t1.q_int)' ,'avg(t1.q_bigint)' , 'avg(t1.q_smallint)' ,'avg(t1.q_tinyint)' ,'avg(t1.q_float)' ,'avg(t1.q_double)' , + 'sum(t1.q_int)' ,'sum(t1.q_bigint)' , 'sum(t1.q_smallint)' ,'sum(t1.q_tinyint)' ,'sum(t1.q_float)' ,'sum(t1.q_double)' , + 'STDDEV(t1.q_int)' ,'STDDEV(t1.q_bigint)' , 'STDDEV(t1.q_smallint)' ,'STDDEV(t1.q_tinyint)' ,'STDDEV(t1.q_float)' ,'STDDEV(t1.q_double)', + 'APERCENTILE(t1.q_int,10)' ,'APERCENTILE(t1.q_bigint,20)' , 'APERCENTILE(t1.q_smallint,30)' ,'APERCENTILE(t1.q_tinyint,40)' ,'APERCENTILE(t1.q_float,50)' ,'APERCENTILE(t1.q_double,60)' , + 'count(t1.q_int_null)' ,'count(t1.q_bigint_null)' , 'count(t1.q_smallint_null)' ,'count(t1.q_tinyint_null)' ,'count(t1.q_float_null)' , + 'count(t1.q_double_null)' ,'count(t1.q_binary_null)' ,'count(t1.q_nchar_null)' ,'count(t1.q_bool_null)' ,'count(t1.q_ts_null)' , + 'avg(t1.q_int_null)' ,'avg(t1.q_bigint_null)' , 'avg(t1.q_smallint_null)' ,'avg(t1.q_tinyint_null)' ,'avg(t1.q_float_null)' ,'avg(t1.q_double_null)' , + 'sum(t1.q_int_null)' ,'sum(t1.q_bigint_null)' , 'sum(t1.q_smallint_null)' ,'sum(t1.q_tinyint_null)' ,'sum(t1.q_float_null)' ,'sum(t1.q_double_null)' , + 'STDDEV(t1.q_int_null)' ,'STDDEV(t1.q_bigint_null)' , 'STDDEV(t1.q_smallint_null)' ,'STDDEV(t1.q_tinyint_null)' ,'STDDEV(t1.q_float_null)' ,'STDDEV(t1.q_double_null)', + 'APERCENTILE(t1.q_int_null,10)' ,'APERCENTILE(t1.q_bigint_null,20)' , 'APERCENTILE(t1.q_smallint_null,30)' ,'APERCENTILE(t1.q_tinyint_null,40)' ,'APERCENTILE(t1.q_float_null,50)' ,'APERCENTILE(t1.q_double,60)' , + 'count(t2.*)' , 'count(t2.q_int)' ,'count(t2.q_bigint)' , 'count(t2.q_smallint)' ,'count(t2.q_tinyint)' ,'count(t2.q_float)' , + 'count(t2.q_double)' ,'count(t2.q_binary)' ,'count(t2.q_nchar)' ,'count(t2.q_bool)' ,'count(t2.q_ts)' , + 'avg(t2.q_int)' ,'avg(t2.q_bigint)' , 'avg(t2.q_smallint)' ,'avg(t2.q_tinyint)' ,'avg(t2.q_float)' ,'avg(t2.q_double)' , + 'sum(t2.q_int)' ,'sum(t2.q_bigint)' , 'sum(t2.q_smallint)' ,'sum(t2.q_tinyint)' ,'sum(t2.q_float)' ,'sum(t2.q_double)' , + 'STDDEV(t2.q_int)' ,'STDDEV(t2.q_bigint)' , 'STDDEV(t2.q_smallint)' ,'STDDEV(t2.q_tinyint)' ,'STDDEV(t2.q_float)' ,'STDDEV(t2.q_double)', + 'APERCENTILE(t2.q_int,10)' ,'APERCENTILE(t2.q_bigint,20)' , 'APERCENTILE(t2.q_smallint,30)' ,'APERCENTILE(t2.q_tinyint,40)' ,'APERCENTILE(t2.q_float,50)' ,'APERCENTILE(t2.q_double,60)', + 'count(t2.q_int_null)' ,'count(t2.q_bigint_null)' , 'count(t2.q_smallint_null)' ,'count(t2.q_tinyint_null)' ,'count(t2.q_float_null)' , + 'count(t2.q_double_null)' ,'count(t2.q_binary_null)' ,'count(t2.q_nchar_null)' ,'count(t2.q_bool_null)' ,'count(t2.q_ts_null)' , + 'avg(t2.q_int_null)' ,'avg(t2.q_bigint_null)' , 'avg(t2.q_smallint_null)' ,'avg(t2.q_tinyint_null)' ,'avg(t2.q_float_null)' ,'avg(t2.q_double_null)' , + 'sum(t2.q_int_null)' ,'sum(t2.q_bigint_null)' , 'sum(t2.q_smallint_null)' ,'sum(t2.q_tinyint_null)' ,'sum(t2.q_float_null)' ,'sum(t2.q_double_null)' , + 'STDDEV(t2.q_int_null)' ,'STDDEV(t2.q_bigint_null)' , 'STDDEV(t2.q_smallint_null)' ,'STDDEV(t2.q_tinyint_null)' ,'STDDEV(t2.q_float_null)' ,'STDDEV(t2.q_double_null)', + 'APERCENTILE(t2.q_int_null,10)' ,'APERCENTILE(t2.q_bigint_null,20)' , 'APERCENTILE(t2.q_smallint_null,30)' ,'APERCENTILE(t2.q_tinyint_null,40)' ,'APERCENTILE(t2.q_float_null,50)' ,'APERCENTILE(t2.q_double,60)'] + + self.calc_aggregate_regular_j = ['twa(t1.q_int)' ,'twa(t1.q_bigint)' , 'twa(t1.q_smallint)' ,'twa(t1.q_tinyint)' ,'twa (t1.q_float)' ,'twa(t1.q_double)' , + 'IRATE(t1.q_int)' ,'IRATE(t1.q_bigint)' , 'IRATE(t1.q_smallint)' ,'IRATE(t1.q_tinyint)' ,'IRATE (t1.q_float)' ,'IRATE(t1.q_double)' , + 'LEASTSQUARES(t1.q_int,15,3)' , 'LEASTSQUARES(t1.q_bigint,10,1)' , 'LEASTSQUARES(t1.q_smallint,20,3)' ,'LEASTSQUARES(t1.q_tinyint,10,4)' ,'LEASTSQUARES(t1.q_float,6,4)' ,'LEASTSQUARES(t1.q_double,3,1)' , + 'twa(t2.q_int)' ,'twa(t2.q_bigint)' , 'twa(t2.q_smallint)' ,'twa(t2.q_tinyint)' ,'twa (t2.q_float)' ,'twa(t2.q_double)' , + 'IRATE(t2.q_int)' ,'IRATE(t2.q_bigint)' , 'IRATE(t2.q_smallint)' ,'IRATE(t2.q_tinyint)' ,'IRATE (t2.q_float)' ,'IRATE(t2.q_double)', + 'LEASTSQUARES(t2.q_int,15,3)' , 'LEASTSQUARES(t2.q_bigint,10,1)' , 'LEASTSQUARES(t2.q_smallint,20,3)' ,'LEASTSQUARES(t2.q_tinyint,10,4)' ,'LEASTSQUARES(t2.q_float,6,4)' ,'LEASTSQUARES(t2.q_double,3,1)' , + 'twa(t1.q_int_null)' ,'twa(t1.q_bigint_null)' , 'twa(t1.q_smallint_null)' ,'twa(t1.q_tinyint_null)' ,'twa (t1.q_float_null)' ,'twa(t1.q_double_null)' , + 'IRATE(t1.q_int_null)' ,'IRATE(t1.q_bigint_null)' , 'IRATE(t1.q_smallint_null)' ,'IRATE(t1.q_tinyint_null)' ,'IRATE (t1.q_float_null)' ,'IRATE(t1.q_double_null)' , + 'LEASTSQUARES(t1.q_int_null,15,3)' , 'LEASTSQUARES(t1.q_bigint_null,10,1)' , 'LEASTSQUARES(t1.q_smallint_null,20,3)' ,'LEASTSQUARES(t1.q_tinyint_null,10,4)' ,'LEASTSQUARES(t1.q_float_null,6,4)' ,'LEASTSQUARES(t1.q_double_null,3,1)' , + 'twa(t2.q_int_null)' ,'twa(t2.q_bigint_null)' , 'twa(t2.q_smallint_null)' ,'twa(t2.q_tinyint_null)' ,'twa (t2.q_float_null)' ,'twa(t2.q_double_null)' , + 'IRATE(t2.q_int_null)' ,'IRATE(t2.q_bigint_null)' , 'IRATE(t2.q_smallint_null)' ,'IRATE(t2.q_tinyint_null)' ,'IRATE (t2.q_float_null)' ,'IRATE(t2.q_double_null)', + 'LEASTSQUARES(t2.q_int_null,15,3)' , 'LEASTSQUARES(t2.q_bigint_null,10,1)' , 'LEASTSQUARES(t2.q_smallint_null,20,3)' ,'LEASTSQUARES(t2.q_tinyint_null,10,4)' ,'LEASTSQUARES(t2.q_float_null,6,4)' ,'LEASTSQUARES(t2.q_double_null,3,1)' ] + + self.calc_aggregate_groupbytbname_j = ['twa(t1.q_int)' ,'twa(t1.q_bigint)' , 'twa(t1.q_smallint)' ,'twa(t1.q_tinyint)' ,'twa (t1.q_float)' ,'twa(t1.q_double)' , + 'IRATE(t1.q_int)' ,'IRATE(t1.q_bigint)' , 'IRATE(t1.q_smallint)' ,'IRATE(t1.q_tinyint)' ,'IRATE (t1.q_float)' ,'IRATE(t1.q_double)' , + 'twa(t2.q_int)' ,'twa(t2.q_bigint)' , 'twa(t2.q_smallint)' ,'twa(t2.q_tinyint)' ,'twa (t2.q_float)' ,'twa(t2.q_double)' , + 'IRATE(t2.q_int)' ,'IRATE(t2.q_bigint)' , 'IRATE(t2.q_smallint)' ,'IRATE(t2.q_tinyint)' ,'IRATE (t2.q_float)' ,'IRATE(t2.q_double)' , + 'twa(t1.q_int_null)' ,'twa(t1.q_bigint_null)' , 'twa(t1.q_smallint_null)' ,'twa(t1.q_tinyint_null)' ,'twa (t1.q_float_null)' ,'twa(t1.q_double_null)' , + 'IRATE(t1.q_int_null)' ,'IRATE(t1.q_bigint_null)' , 'IRATE(t1.q_smallint_null)' ,'IRATE(t1.q_tinyint_null)' ,'IRATE (t1.q_float_null)' ,'IRATE(t1.q_double_null)' , + 'twa(t2.q_int_null)' ,'twa(t2.q_bigint_null)' , 'twa(t2.q_smallint_null)' ,'twa(t2.q_tinyint_null)' ,'twa (t2.q_float_null)' ,'twa(t2.q_double_null)' , + 'IRATE(t2.q_int_null)' ,'IRATE(t2.q_bigint_null)' , 'IRATE(t2.q_smallint_null)' ,'IRATE(t2.q_tinyint_null)' ,'IRATE (t2.q_float_null)' ,'IRATE(t2.q_double_null)' ] + + self.calc_calculate_all = ['SPREAD(ts)' , 'SPREAD(q_ts)' , 'SPREAD(q_int)' ,'SPREAD(q_bigint)' , 'SPREAD(q_smallint)' ,'SPREAD(q_tinyint)' ,'SPREAD(q_float)' ,'SPREAD(q_double)' , + '(SPREAD(q_int) + SPREAD(q_bigint))' , '(SPREAD(q_smallint) - SPREAD(q_float))', '(SPREAD(q_double) * SPREAD(q_tinyint))' , '(SPREAD(q_double) / SPREAD(q_float))', + 'SPREAD(q_ts_null)' , 'SPREAD(q_int_null)' ,'SPREAD(q_bigint_null)' , 'SPREAD(q_smallint_null)' ,'SPREAD(q_tinyint_null)' ,'SPREAD(q_float_null)' ,'SPREAD(q_double_null)' , + '(SPREAD(q_int_null) + SPREAD(q_bigint_null))' , '(SPREAD(q_smallint_null) - SPREAD(q_float_null))', '(SPREAD(q_double_null) * SPREAD(q_tinyint_null))' , '(SPREAD(q_double_null) / SPREAD(q_float_null))'] + self.calc_calculate_regular = ['DIFF(q_int)' ,'DIFF(q_bigint)' , 'DIFF(q_smallint)' ,'DIFF(q_tinyint)' ,'DIFF(q_float)' ,'DIFF(q_double)' , + 'DIFF(q_int,0)' ,'DIFF(q_bigint,0)' , 'DIFF(q_smallint,0)' ,'DIFF(q_tinyint,0)' ,'DIFF(q_float,0)' ,'DIFF(q_double,0)' , + 'DIFF(q_int,1)' ,'DIFF(q_bigint,1)' , 'DIFF(q_smallint,1)' ,'DIFF(q_tinyint,1)' ,'DIFF(q_float,1)' ,'DIFF(q_double,1)' , + 'DERIVATIVE(q_int,15s,0)' , 'DERIVATIVE(q_bigint,10s,1)' , 'DERIVATIVE(q_smallint,20s,0)' ,'DERIVATIVE(q_tinyint,10s,1)' ,'DERIVATIVE(q_float,6s,0)' ,'DERIVATIVE(q_double,3s,1)', + 'DIFF(q_int_null)' ,'DIFF(q_bigint_null)' , 'DIFF(q_smallint_null)' ,'DIFF(q_tinyint_null)' ,'DIFF(q_float_null)' ,'DIFF(q_double_null)' , + 'DIFF(q_int_null,0)' ,'DIFF(q_bigint_null,0)' , 'DIFF(q_smallint_null,0)' ,'DIFF(q_tinyint_null,0)' ,'DIFF(q_float_null,0)' ,'DIFF(q_double_null,0)' , + 'DIFF(q_int_null,1)' ,'DIFF(q_bigint_null,1)' , 'DIFF(q_smallint_null,1)' ,'DIFF(q_tinyint_null,1)' ,'DIFF(q_float_null,1)' ,'DIFF(q_double_null,1)' , + 'DERIVATIVE(q_int_null,15s,0)' , 'DERIVATIVE(q_bigint_null,10s,1)' , 'DERIVATIVE(q_smallint_null,20s,0)' ,'DERIVATIVE(q_tinyint_null,10s,1)' ,'DERIVATIVE(q_float_null,6s,0)' ,'DERIVATIVE(q_double_null,3s,1)'] + self.calc_calculate_groupbytbname = self.calc_calculate_regular + + #two table join + self.calc_calculate_all_j = ['SPREAD(t1.ts)' , 'SPREAD(t1.q_ts)' , 'SPREAD(t1.q_int)' ,'SPREAD(t1.q_bigint)' , 'SPREAD(t1.q_smallint)' ,'SPREAD(t1.q_tinyint)' ,'SPREAD(t1.q_float)' ,'SPREAD(t1.q_double)' , + 'SPREAD(t2.ts)' , 'SPREAD(t2.q_ts)' , 'SPREAD(t2.q_int)' ,'SPREAD(t2.q_bigint)' , 'SPREAD(t2.q_smallint)' ,'SPREAD(t2.q_tinyint)' ,'SPREAD(t2.q_float)' ,'SPREAD(t2.q_double)' , + '(SPREAD(t1.q_int) + SPREAD(t1.q_bigint))' , '(SPREAD(t1.q_tinyint) - SPREAD(t1.q_float))', '(SPREAD(t1.q_double) * SPREAD(t1.q_tinyint))' , '(SPREAD(t1.q_double) / SPREAD(t1.q_tinyint))', + '(SPREAD(t2.q_int) + SPREAD(t2.q_bigint))' , '(SPREAD(t2.q_smallint) - SPREAD(t2.q_float))', '(SPREAD(t2.q_double) * SPREAD(t2.q_tinyint))' , '(SPREAD(t2.q_double) / SPREAD(t2.q_tinyint))', + '(SPREAD(t1.q_int) + SPREAD(t1.q_smallint))' , '(SPREAD(t2.q_smallint) - SPREAD(t2.q_float))', '(SPREAD(t1.q_double) * SPREAD(t1.q_tinyint))' , '(SPREAD(t1.q_double) / SPREAD(t1.q_float))', + 'SPREAD(t1.q_ts_null)' , 'SPREAD(t1.q_int_null)' ,'SPREAD(t1.q_bigint_null)' , 'SPREAD(t1.q_smallint_null)' ,'SPREAD(t1.q_tinyint_null)' ,'SPREAD(t1.q_float_null)' ,'SPREAD(t1.q_double_null)' , + 'SPREAD(t2.q_ts_null)' , 'SPREAD(t2.q_int_null)' ,'SPREAD(t2.q_bigint_null)' , 'SPREAD(t2.q_smallint_null)' ,'SPREAD(t2.q_tinyint_null)' ,'SPREAD(t2.q_float_null)' ,'SPREAD(t2.q_double_null)' , + '(SPREAD(t1.q_int_null) + SPREAD(t1.q_bigint_null))' , '(SPREAD(t1.q_tinyint_null) - SPREAD(t1.q_float_null))', '(SPREAD(t1.q_double_null) * SPREAD(t1.q_tinyint_null))' , '(SPREAD(t1.q_double_null) / SPREAD(t1.q_tinyint_null))', + '(SPREAD(t2.q_int_null) + SPREAD(t2.q_bigint_null))' , '(SPREAD(t2.q_smallint_null) - SPREAD(t2.q_float_null))', '(SPREAD(t2.q_double_null) * SPREAD(t2.q_tinyint_null))' , '(SPREAD(t2.q_double_null) / SPREAD(t2.q_tinyint_null))', + '(SPREAD(t1.q_int_null) + SPREAD(t1.q_smallint_null))' , '(SPREAD(t2.q_smallint_null) - SPREAD(t2.q_float_null))', '(SPREAD(t1.q_double_null) * SPREAD(t1.q_tinyint_null))' , '(SPREAD(t1.q_double_null) / SPREAD(t1.q_float_null))'] + self.calc_calculate_regular_j = ['DIFF(t1.q_int)' ,'DIFF(t1.q_bigint)' , 'DIFF(t1.q_smallint)' ,'DIFF(t1.q_tinyint)' ,'DIFF(t1.q_float)' ,'DIFF(t1.q_double)' , + 'DIFF(t1.q_int,0)' ,'DIFF(t1.q_bigint,0)' , 'DIFF(t1.q_smallint,0)' ,'DIFF(t1.q_tinyint,0)' ,'DIFF(t1.q_float,0)' ,'DIFF(t1.q_double,0)' , + 'DIFF(t1.q_int,1)' ,'DIFF(t1.q_bigint,1)' , 'DIFF(t1.q_smallint,1)' ,'DIFF(t1.q_tinyint,1)' ,'DIFF(t1.q_float,1)' ,'DIFF(t1.q_double,1)' , + 'DERIVATIVE(t1.q_int,15s,0)' , 'DERIVATIVE(t1.q_bigint,10s,1)' , 'DERIVATIVE(t1.q_smallint,20s,0)' ,'DERIVATIVE(t1.q_tinyint,10s,1)' ,'DERIVATIVE(t1.q_float,6s,0)' ,'DERIVATIVE(t1.q_double,3s,1)' , + 'DIFF(t2.q_int)' ,'DIFF(t2.q_bigint)' , 'DIFF(t2.q_smallint)' ,'DIFF(t2.q_tinyint)' ,'DIFF(t2.q_float)' ,'DIFF(t2.q_double)' , + 'DIFF(t2.q_int,0)' ,'DIFF(t2.q_bigint,0)' , 'DIFF(t2.q_smallint,0)' ,'DIFF(t2.q_tinyint,0)' ,'DIFF(t2.q_float,0)' ,'DIFF(t2.q_double,0)' , + 'DIFF(t2.q_int,1)' ,'DIFF(t2.q_bigint,1)' , 'DIFF(t2.q_smallint,1)' ,'DIFF(t2.q_tinyint,1)' ,'DIFF(t2.q_float,1)' ,'DIFF(t2.q_double,1)' , + 'DERIVATIVE(t2.q_int,15s,0)' , 'DERIVATIVE(t2.q_bigint,10s,1)' , 'DERIVATIVE(t2.q_smallint,20s,0)' ,'DERIVATIVE(t2.q_tinyint,10s,1)' ,'DERIVATIVE(t2.q_float,6s,0)' ,'DERIVATIVE(t2.q_double,3s,1)' , + 'DIFF(t1.q_int_null)' ,'DIFF(t1.q_bigint_null)' , 'DIFF(t1.q_smallint_null)' ,'DIFF(t1.q_tinyint_null)' ,'DIFF(t1.q_float_null)' ,'DIFF(t1.q_double_null)' , + 'DIFF(t1.q_int_null,0)' ,'DIFF(t1.q_bigint_null,0)' , 'DIFF(t1.q_smallint_null,0)' ,'DIFF(t1.q_tinyint_null,0)' ,'DIFF(t1.q_float_null,0)' ,'DIFF(t1.q_double_null,0)' , + 'DIFF(t1.q_int_null,1)' ,'DIFF(t1.q_bigint_null,1)' , 'DIFF(t1.q_smallint_null,1)' ,'DIFF(t1.q_tinyint_null,1)' ,'DIFF(t1.q_float_null,1)' ,'DIFF(t1.q_double_null,1)' , + 'DERIVATIVE(t1.q_int_null,15s,0)' , 'DERIVATIVE(t1.q_bigint_null,10s,1)' , 'DERIVATIVE(t1.q_smallint_null,20s,0)' ,'DERIVATIVE(t1.q_tinyint_null,10s,1)' ,'DERIVATIVE(t1.q_float_null,6s,0)' ,'DERIVATIVE(t1.q_double_null,3s,1)' , + 'DIFF(t2.q_int_null)' ,'DIFF(t2.q_bigint_null)' , 'DIFF(t2.q_smallint_null)' ,'DIFF(t2.q_tinyint_null)' ,'DIFF(t2.q_float_null)' ,'DIFF(t2.q_double_null)' , + 'DIFF(t2.q_int_null,0)' ,'DIFF(t2.q_bigint_null,0)' , 'DIFF(t2.q_smallint_null,0)' ,'DIFF(t2.q_tinyint_null,0)' ,'DIFF(t2.q_float_null,0)' ,'DIFF(t2.q_double_null,0)' , + 'DIFF(t2.q_int_null,1)' ,'DIFF(t2.q_bigint_null,1)' , 'DIFF(t2.q_smallint_null,1)' ,'DIFF(t2.q_tinyint_null,1)' ,'DIFF(t2.q_float_null,1)' ,'DIFF(t2.q_double_null,1)' , + 'DERIVATIVE(t2.q_int_null,15s,0)' , 'DERIVATIVE(t2.q_bigint_null,10s,1)' , 'DERIVATIVE(t2.q_smallint_null,20s,0)' ,'DERIVATIVE(t2.q_tinyint_null,10s,1)' ,'DERIVATIVE(t2.q_float_null,6s,0)' ,'DERIVATIVE(t2.q_double_null,3s,1)'] + self.calc_calculate_groupbytbname_j = self.calc_calculate_regular_j + + self.interval_sliding = ['interval(4w) sliding(1w) ','interval(1w) sliding(1d) ','interval(1d) sliding(1h) ' , + 'interval(1h) sliding(1m) ','interval(1m) sliding(1s) ','interval(1s) sliding(10a) ', + 'interval(1y) ','interval(1n) ','interval(1w) ','interval(1d) ','interval(1h) ','interval(1m) ','interval(1s) ' ,'interval(10a)', + 'interval(1y,1n) ','interval(1n,1w) ','interval(1w,1d) ','interval(1d,1h) ','interval(1h,1m) ','interval(1m,1s) ','interval(1s,10a) ' ,'interval(100a,30a)'] + + self.conn1 = taos.connect(host="127.0.0.1", user="root", password="taosdata", config="/etc/taos/") + self.cur1 = self.conn1.cursor() + self.cur1.execute("use %s ;" %self.db_nest) + sql = 'select /*+ para_tables_sort() */* from stable_1 limit 5;' + self.cur1.execute(sql) + + + def dropandcreateDB_random(self,database,n): + ts = 1630000000000 + num_random = 100 + fake = Faker('zh_CN') + tdSql.execute('''drop database if exists %s ;''' %database) + tdSql.execute('''create database %s keep 36500;'''%database) + tdSql.execute('''use %s;'''%database) + + tdSql.execute('''create stable stable_1 (ts timestamp , q_int int , q_bigint bigint , q_smallint smallint , q_tinyint tinyint , q_float float , q_double double , q_bool bool , q_binary binary(100) , q_nchar nchar(100) , q_ts timestamp , \ + q_binary1 binary(100) , q_nchar1 nchar(100) ,q_binary2 binary(100) , q_nchar2 nchar(100) ,q_binary3 binary(100) , q_nchar3 nchar(100) ,q_binary4 binary(100) , q_nchar4 nchar(100) ,\ + q_binary5 binary(100) , q_nchar5 nchar(100) ,q_binary6 binary(100) , q_nchar6 nchar(100) ,q_binary7 binary(100) , q_nchar7 nchar(100) ,q_binary8 binary(100) , q_nchar8 nchar(100) ,\ + q_int_null int , q_bigint_null bigint , q_smallint_null smallint , q_tinyint_null tinyint, q_float_null float , q_double_null double , q_bool_null bool , q_binary_null binary(20) , q_nchar_null nchar(20) , q_ts_null timestamp) \ + tags(loc nchar(100) , t_int int , t_bigint bigint , t_smallint smallint , t_tinyint tinyint, t_bool bool , t_binary binary(100) , t_nchar nchar(100) ,t_float float , t_double double , t_ts timestamp);''') + tdSql.execute('''create stable stable_2 (ts timestamp , q_int int , q_bigint bigint , q_smallint smallint , q_tinyint tinyint , q_float float , q_double double , q_bool bool , q_binary binary(100) , q_nchar nchar(100) , q_ts timestamp , \ + q_binary1 binary(100) , q_nchar1 nchar(100) ,q_binary2 binary(100) , q_nchar2 nchar(100) ,q_binary3 binary(100) , q_nchar3 nchar(100) ,q_binary4 binary(100) , q_nchar4 nchar(100) ,\ + q_binary5 binary(100) , q_nchar5 nchar(100) ,q_binary6 binary(100) , q_nchar6 nchar(100) ,q_binary7 binary(100) , q_nchar7 nchar(100) ,q_binary8 binary(100) , q_nchar8 nchar(100) ,\ + q_int_null int , q_bigint_null bigint , q_smallint_null smallint , q_tinyint_null tinyint, q_float_null float , q_double_null double , q_bool_null bool , q_binary_null binary(20) , q_nchar_null nchar(20) , q_ts_null timestamp) \ + tags(loc nchar(100) , t_int int , t_bigint bigint , t_smallint smallint , t_tinyint tinyint, t_bool bool , t_binary binary(100) , t_nchar nchar(100) ,t_float float , t_double double , t_ts timestamp);''') + + tdSql.execute('''create stable stable_null_data (ts timestamp , q_int int , q_bigint bigint , q_smallint smallint , q_tinyint tinyint , q_float float , q_double double , q_bool bool , q_binary binary(100) , q_nchar nchar(100) , q_ts timestamp , \ + q_binary1 binary(100) , q_nchar1 nchar(100) ,q_binary2 binary(100) , q_nchar2 nchar(100) ,q_binary3 binary(100) , q_nchar3 nchar(100) ,q_binary4 binary(100) , q_nchar4 nchar(100) ,\ + q_binary5 binary(100) , q_nchar5 nchar(100) ,q_binary6 binary(100) , q_nchar6 nchar(100) ,q_binary7 binary(100) , q_nchar7 nchar(100) ,q_binary8 binary(100) , q_nchar8 nchar(100) ,\ + q_int_null int , q_bigint_null bigint , q_smallint_null smallint , q_tinyint_null tinyint, q_float_null float , q_double_null double , q_bool_null bool , q_binary_null binary(20) , q_nchar_null nchar(20) , q_ts_null timestamp) \ + tags(loc nchar(100) , t_int int , t_bigint bigint , t_smallint smallint , t_tinyint tinyint, t_bool bool , t_binary binary(100) , t_nchar nchar(100) ,t_float float , t_double double , t_ts timestamp);''') + + tdSql.execute('''create stable stable_null_childtable (ts timestamp , q_int int , q_bigint bigint , q_smallint smallint , q_tinyint tinyint , q_float float , q_double double , q_bool bool , q_binary binary(100) , q_nchar nchar(100) , q_ts timestamp , \ + q_binary1 binary(100) , q_nchar1 nchar(100) ,q_binary2 binary(100) , q_nchar2 nchar(100) ,q_binary3 binary(100) , q_nchar3 nchar(100) ,q_binary4 binary(100) , q_nchar4 nchar(100) ,\ + q_binary5 binary(100) , q_nchar5 nchar(100) ,q_binary6 binary(100) , q_nchar6 nchar(100) ,q_binary7 binary(100) , q_nchar7 nchar(100) ,q_binary8 binary(100) , q_nchar8 nchar(100) ,\ + q_int_null int , q_bigint_null bigint , q_smallint_null smallint , q_tinyint_null tinyint, q_float_null float , q_double_null double , q_bool_null bool , q_binary_null binary(20) , q_nchar_null nchar(20) , q_ts_null timestamp) \ + tags(loc nchar(100) , t_int int , t_bigint bigint , t_smallint smallint , t_tinyint tinyint, t_bool bool , t_binary binary(100) , t_nchar nchar(100) ,t_float float , t_double double , t_ts timestamp);''') + + tdSql.execute('''create table stable_1_1 using stable_1 tags('stable_1_1', '%d' , '%d', '%d' , '%d' , 0 , 'binary1.%s' , 'nchar1.%s' , '%f', '%f' ,'%d') ;''' + %(fake.random_int(min=-2147483647, max=2147483647, step=1), fake.random_int(min=-9223372036854775807, max=9223372036854775807, step=1), + fake.random_int(min=-32767, max=32767, step=1) , fake.random_int(min=-127, max=127, step=1) , + fake.pystr() ,fake.pystr() ,fake.pyfloat(),fake.pyfloat(),fake.random_int(min=-2147483647, max=2147483647, step=1))) + tdSql.execute('''create table stable_1_2 using stable_1 tags('stable_1_2', '2147483647' , '9223372036854775807' , '32767' , '127' , 1 , 'binary2' , 'nchar2' , '2' , '22' , \'1999-09-09 09:09:09.090\') ;''') + tdSql.execute('''create table stable_1_3 using stable_1 tags('stable_1_3', '-2147483647' , '-9223372036854775807' , '-32767' , '-127' , false , 'binary3' , 'nchar3nchar3' , '-3.3' , '-33.33' , \'2099-09-09 09:09:09.090\') ;''') + tdSql.execute('''create table stable_1_4 using stable_1 tags('stable_1_4', '%d' , '%d', '%d' , '%d' , 1 , 'binary1.%s' , 'nchar1.%s' , '%f', '%f' ,'%d') ;''' + %(fake.random_int(min=-2147483647, max=2147483647, step=1), fake.random_int(min=-9223372036854775807, max=9223372036854775807, step=1), + fake.random_int(min=-32767, max=32767, step=1) , fake.random_int(min=-127, max=127, step=1) , + fake.pystr() ,fake.pystr() ,fake.pyfloat(),fake.pyfloat(),fake.random_int(min=-2147483647, max=2147483647, step=1))) + + tdSql.execute('''create table stable_2_1 using stable_2 tags('stable_2_1' , '0' , '0' , '0' , '0' , 0 , 'binary21' , 'nchar21' , '0' , '0' ,\'2099-09-09 09:09:09.090\') ;''') + tdSql.execute('''create table stable_2_2 using stable_2 tags('stable_2_2' , '%d' , '%d', '%d' , '%d' , 0 , 'binary2.%s' , 'nchar2.%s' , '%f', '%f' ,'%d') ;''' + %(fake.random_int(min=-2147483647, max=2147483647, step=1), fake.random_int(min=-9223372036854775807, max=9223372036854775807, step=1), + fake.random_int(min=-32767, max=32767, step=1) , fake.random_int(min=-127, max=127, step=1) , + fake.pystr() ,fake.pystr() ,fake.pyfloat(),fake.pyfloat(),fake.random_int(min=-2147483647, max=2147483647, step=1))) + + tdSql.execute('''create table stable_null_data_1 using stable_null_data tags('stable_null_data_1', '%d' , '%d', '%d' , '%d' , 1 , 'binary1.%s' , 'nchar1.%s' , '%f', '%f' ,'%d') ;''' + %(fake.random_int(min=-2147483647, max=2147483647, step=1), fake.random_int(min=-9223372036854775807, max=9223372036854775807, step=1), + fake.random_int(min=-32767, max=32767, step=1) , fake.random_int(min=-127, max=127, step=1) , + fake.pystr() ,fake.pystr() ,fake.pyfloat(),fake.pyfloat(),fake.random_int(min=-2147483647, max=2147483647, step=1))) + + #regular table + tdSql.execute('''create table regular_table_1 \ + (ts timestamp , q_int int , q_bigint bigint , q_smallint smallint , q_tinyint tinyint , q_float float , q_double double , q_bool bool , q_binary binary(100) , q_nchar nchar(100) , q_ts timestamp , \ + q_binary1 binary(100) , q_nchar1 nchar(100) ,q_binary2 binary(100) , q_nchar2 nchar(100) ,q_binary3 binary(100) , q_nchar3 nchar(100) ,q_binary4 binary(100) , q_nchar4 nchar(100) ,\ + q_binary5 binary(100) , q_nchar5 nchar(100) ,q_binary6 binary(100) , q_nchar6 nchar(100) ,q_binary7 binary(100) , q_nchar7 nchar(100) ,q_binary8 binary(100) , q_nchar8 nchar(100) ,\ + q_int_null int , q_bigint_null bigint , q_smallint_null smallint , q_tinyint_null tinyint, q_float_null float , q_double_null double , q_bool_null bool , q_binary_null binary(20) , q_nchar_null nchar(20) , q_ts_null timestamp) ;''') + tdSql.execute('''create table regular_table_2 \ + (ts timestamp , q_int int , q_bigint bigint , q_smallint smallint , q_tinyint tinyint , q_float float , q_double double , q_bool bool , q_binary binary(100) , q_nchar nchar(100) , q_ts timestamp , \ + q_binary1 binary(100) , q_nchar1 nchar(100) ,q_binary2 binary(100) , q_nchar2 nchar(100) ,q_binary3 binary(100) , q_nchar3 nchar(100) ,q_binary4 binary(100) , q_nchar4 nchar(100) ,\ + q_binary5 binary(100) , q_nchar5 nchar(100) ,q_binary6 binary(100) , q_nchar6 nchar(100) ,q_binary7 binary(100) , q_nchar7 nchar(100) ,q_binary8 binary(100) , q_nchar8 nchar(100) ,\ + q_int_null int , q_bigint_null bigint , q_smallint_null smallint , q_tinyint_null tinyint, q_float_null float , q_double_null double , q_bool_null bool , q_binary_null binary(20) , q_nchar_null nchar(20) , q_ts_null timestamp) ;''') + tdSql.execute('''create table regular_table_3 \ + (ts timestamp , q_int int , q_bigint bigint , q_smallint smallint , q_tinyint tinyint , q_float float , q_double double , q_bool bool , q_binary binary(100) , q_nchar nchar(100) , q_ts timestamp , \ + q_binary1 binary(100) , q_nchar1 nchar(100) ,q_binary2 binary(100) , q_nchar2 nchar(100) ,q_binary3 binary(100) , q_nchar3 nchar(100) ,q_binary4 binary(100) , q_nchar4 nchar(100) ,\ + q_binary5 binary(100) , q_nchar5 nchar(100) ,q_binary6 binary(100) , q_nchar6 nchar(100) ,q_binary7 binary(100) , q_nchar7 nchar(100) ,q_binary8 binary(100) , q_nchar8 nchar(100) ,\ + q_int_null int , q_bigint_null bigint , q_smallint_null smallint , q_tinyint_null tinyint, q_float_null float , q_double_null double , q_bool_null bool , q_binary_null binary(20) , q_nchar_null nchar(20) , q_ts_null timestamp) ;''') + + tdSql.execute('''create table regular_table_null \ + (ts timestamp , q_int int , q_bigint bigint , q_smallint smallint , q_tinyint tinyint , q_float float , q_double double , q_bool bool , q_binary binary(100) , q_nchar nchar(100) , q_ts timestamp , \ + q_binary1 binary(100) , q_nchar1 nchar(100) ,q_binary2 binary(100) , q_nchar2 nchar(100) ,q_binary3 binary(100) , q_nchar3 nchar(100) ,q_binary4 binary(100) , q_nchar4 nchar(100) ,\ + q_binary5 binary(100) , q_nchar5 nchar(100) ,q_binary6 binary(100) , q_nchar6 nchar(100) ,q_binary7 binary(100) , q_nchar7 nchar(100) ,q_binary8 binary(100) , q_nchar8 nchar(100) ,\ + q_int_null int , q_bigint_null bigint , q_smallint_null smallint , q_tinyint_null tinyint, q_float_null float , q_double_null double , q_bool_null bool , q_binary_null binary(20) , q_nchar_null nchar(20) , q_ts_null timestamp) ;''') + + + for i in range(1, num_random*n + 1): + tdSql.execute('''insert into stable_1_1 (ts , q_int , q_bigint , q_smallint , q_tinyint , q_float , q_double , q_bool , q_binary , q_nchar, q_ts,\ + q_binary1 , q_nchar1 , q_binary2 , q_nchar2 , q_binary3 , q_nchar3 , q_binary4 , q_nchar4 , q_binary5 , q_nchar5 , q_binary6 , q_nchar6 , q_binary7 , q_nchar7, q_binary8 , q_nchar8) \ + values(%d, %d, %d, %d, %d, %f, %f, 0, 'binary.%s', 'nchar.%s', %d, 'binary1.%s', 'nchar1.%s', 'binary2.%s', 'nchar2.%s', 'binary3.%s', 'nchar3.%s', \ + 'binary4.%s', 'nchar4.%s', 'binary5.%s', 'nchar5.%s', 'binary6.%s', 'nchar6.%s', 'binary7.%s', 'nchar7.%s', 'binary8.%s', 'nchar8.%s') ;''' + % (ts + i*1000, fake.random_int(min=-2147483647, max=2147483647, step=1), + fake.random_int(min=-9223372036854775807, max=9223372036854775807, step=1), + fake.random_int(min=-32767, max=32767, step=1) , fake.random_int(min=-127, max=127, step=1) , + fake.pyfloat() , fake.pyfloat() , fake.pystr() , fake.pystr() , ts + i , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , + fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr())) + tdSql.execute('''insert into regular_table_1 (ts , q_int , q_bigint , q_smallint , q_tinyint , q_float , q_double, q_bool , q_binary , q_nchar, q_ts,\ + q_binary1 , q_nchar1 , q_binary2 , q_nchar2 , q_binary3 , q_nchar3 , q_binary4 , q_nchar4 , q_binary5 , q_nchar5 , q_binary6 , q_nchar6 , q_binary7 , q_nchar7, q_binary8 , q_nchar8) \ + values(%d, %d, %d, %d, %d, %f, %f, 0, 'binary.%s', 'nchar.%s', %d, 'binary1.%s', 'nchar1.%s', 'binary2.%s', 'nchar2.%s', 'binary3.%s', 'nchar3.%s', \ + 'binary4.%s', 'nchar4.%s', 'binary5.%s', 'nchar5.%s', 'binary6.%s', 'nchar6.%s', 'binary7.%s', 'nchar7.%s', 'binary8.%s', 'nchar8.%s') ;''' + % (ts + i*1000, fake.random_int(min=-2147483647, max=2147483647, step=1) , + fake.random_int(min=-9223372036854775807, max=9223372036854775807, step=1) , + fake.random_int(min=-32767, max=32767, step=1) , fake.random_int(min=-127, max=127, step=1) , + fake.pyfloat() , fake.pyfloat() , fake.pystr() , fake.pystr() , ts + i, fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , + fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr())) + + tdSql.execute('''insert into stable_1_2 (ts , q_int , q_bigint , q_smallint , q_tinyint , q_float , q_double, q_bool , q_binary , q_nchar, q_ts,\ + q_binary1 , q_nchar1 , q_binary2 , q_nchar2 , q_binary3 , q_nchar3 , q_binary4 , q_nchar4 , q_binary5 , q_nchar5 , q_binary6 , q_nchar6 , q_binary7 , q_nchar7, q_binary8 , q_nchar8)\ + values(%d, %d, %d, %d, %d, %f, %f, 1, 'binary.%s', 'nchar.%s', %d, 'binary1.%s', 'nchar1.%s', 'binary2.%s', 'nchar2.%s', 'binary3.%s', 'nchar3.%s', \ + 'binary4.%s', 'nchar4.%s', 'binary5.%s', 'nchar5.%s', 'binary6.%s', 'nchar6.%s', 'binary7.%s', 'nchar7.%s', 'binary8.%s', 'nchar8.%s') ;''' + % (ts + i*1000*60*60*2, fake.random_int(min=0, max=2147483647, step=1), + fake.random_int(min=0, max=9223372036854775807, step=1), + fake.random_int(min=0, max=32767, step=1) , fake.random_int(min=0, max=127, step=1) , + fake.pyfloat() , fake.pyfloat() , fake.pystr() , fake.pystr() , ts + i, fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , + fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr())) + tdSql.execute('''insert into regular_table_2 (ts , q_int , q_bigint , q_smallint , q_tinyint , q_float , q_double, q_bool , q_binary , q_nchar, q_ts,\ + q_binary1 , q_nchar1 , q_binary2 , q_nchar2 , q_binary3 , q_nchar3 , q_binary4 , q_nchar4 , q_binary5 , q_nchar5 , q_binary6 , q_nchar6 , q_binary7 , q_nchar7, q_binary8 , q_nchar8) \ + values(%d, %d, %d, %d, %d, %f, %f, 1, 'binary.%s', 'nchar.%s', %d, 'binary1.%s', 'nchar1.%s', 'binary2.%s', 'nchar2.%s', 'binary3.%s', 'nchar3.%s', \ + 'binary4.%s', 'nchar4.%s', 'binary5.%s', 'nchar5.%s', 'binary6.%s', 'nchar6.%s', 'binary7.%s', 'nchar7.%s', 'binary8.%s', 'nchar8.%s') ;''' + % (ts + i*1000, fake.random_int(min=0, max=2147483647, step=1), + fake.random_int(min=0, max=9223372036854775807, step=1), + fake.random_int(min=0, max=32767, step=1) , fake.random_int(min=0, max=127, step=1) , + fake.pyfloat() , fake.pyfloat() , fake.pystr() , fake.pystr() , ts + i, fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , + fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr())) + + tdSql.execute('''insert into stable_1_2 (ts , q_int , q_bigint , q_smallint , q_tinyint , q_float , q_double, q_bool , q_binary , q_nchar, q_ts,\ + q_binary1 , q_nchar1 , q_binary2 , q_nchar2 , q_binary3 , q_nchar3 , q_binary4 , q_nchar4 , q_binary5 , q_nchar5 , q_binary6 , q_nchar6 , q_binary7 , q_nchar7, q_binary8 , q_nchar8) \ + values(%d, %d, %d, %d, %d, %f, %f, 1, 'binary.%s', 'nchar.%s', %d, 'binary1.%s', 'nchar1.%s', 'binary2.%s', 'nchar2.%s', 'binary3.%s', 'nchar3.%s', \ + 'binary4.%s', 'nchar4.%s', 'binary5.%s', 'nchar5.%s', 'binary6.%s', 'nchar6.%s', 'binary7.%s', 'nchar7.%s', 'binary8.%s', 'nchar8.%s') ;''' + % (ts + i*1000*60*60*2 +1, fake.random_int(min=-2147483647, max=0, step=1), + fake.random_int(min=-9223372036854775807, max=0, step=1), + fake.random_int(min=-32767, max=0, step=1) , fake.random_int(min=-127, max=0, step=1) , + fake.pyfloat() , fake.pyfloat() , fake.pystr() , fake.pystr() , ts + i +1, fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , + fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr())) + tdSql.execute('''insert into regular_table_2 (ts , q_int , q_bigint , q_smallint , q_tinyint , q_float , q_double, q_bool , q_binary , q_nchar, q_ts,\ + q_binary1 , q_nchar1 , q_binary2 , q_nchar2 , q_binary3 , q_nchar3 , q_binary4 , q_nchar4 , q_binary5 , q_nchar5 , q_binary6 , q_nchar6 , q_binary7 , q_nchar7, q_binary8 , q_nchar8) \ + values(%d, %d, %d, %d, %d, %f, %f, 1, 'binary.%s', 'nchar.%s', %d, 'binary1.%s', 'nchar1.%s', 'binary2.%s', 'nchar2.%s', 'binary3.%s', 'nchar3.%s', \ + 'binary4.%s', 'nchar4.%s', 'binary5.%s', 'nchar5.%s', 'binary6.%s', 'nchar6.%s', 'binary7.%s', 'nchar7.%s', 'binary8.%s', 'nchar8.%s') ;''' + % (ts + i*1000 +1, fake.random_int(min=-2147483647, max=0, step=1), + fake.random_int(min=-9223372036854775807, max=0, step=1), + fake.random_int(min=-32767, max=0, step=1) , fake.random_int(min=-127, max=0, step=1) , + fake.pyfloat() , fake.pyfloat() , fake.pystr() , fake.pystr() , ts + i +1, fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , + fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr())) + + tdSql.execute('''insert into stable_2_1 (ts , q_int , q_bigint , q_smallint , q_tinyint , q_float , q_double, q_bool , q_binary , q_nchar, q_ts,\ + q_binary1 , q_nchar1 , q_binary2 , q_nchar2 , q_binary3 , q_nchar3 , q_binary4 , q_nchar4 , q_binary5 , q_nchar5 , q_binary6 , q_nchar6 , q_binary7 , q_nchar7, q_binary8 , q_nchar8) \ + values(%d, %d, %d, %d, %d, %f, %f, 0, 'binary.%s', 'nchar.%s', %d, 'binary1.%s', 'nchar1.%s', 'binary2.%s', 'nchar2.%s', 'binary3.%s', 'nchar3.%s', \ + 'binary4.%s', 'nchar4.%s', 'binary5.%s', 'nchar5.%s', 'binary6.%s', 'nchar6.%s', 'binary7.%s', 'nchar7.%s', 'binary8.%s', 'nchar8.%s') ;''' + % (ts + i*1000*60*60*4, fake.random_int(min=-0, max=2147483647, step=1), + fake.random_int(min=-0, max=9223372036854775807, step=1), + fake.random_int(min=-0, max=32767, step=1) , fake.random_int(min=-0, max=127, step=1) , + fake.pyfloat() , fake.pyfloat() , fake.pystr() , fake.pystr() , ts + i, fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , + fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr())) + + tdSql.execute('''insert into stable_2_1 (ts , q_int , q_bigint , q_smallint , q_tinyint , q_float , q_double, q_bool , q_binary , q_nchar, q_ts,\ + q_binary1 , q_nchar1 , q_binary2 , q_nchar2 , q_binary3 , q_nchar3 , q_binary4 , q_nchar4 , q_binary5 , q_nchar5 , q_binary6 , q_nchar6 , q_binary7 , q_nchar7, q_binary8 , q_nchar8) \ + values(%d, %d, %d, %d, %d, %f, %f, 0, 'binary.%s', 'nchar.%s', %d, 'binary1.%s', 'nchar1.%s', 'binary2.%s', 'nchar2.%s', 'binary3.%s', 'nchar3.%s', \ + 'binary4.%s', 'nchar4.%s', 'binary5.%s', 'nchar5.%s', 'binary6.%s', 'nchar6.%s', 'binary7.%s', 'nchar7.%s', 'binary8.%s', 'nchar8.%s') ;''' + % (ts + i*1000*60*60*4 +1, fake.random_int(min=-0, max=2147483647, step=1), + fake.random_int(min=-0, max=9223372036854775807, step=1), + fake.random_int(min=-0, max=32767, step=1) , fake.random_int(min=-0, max=127, step=1) , + fake.pyfloat() , fake.pyfloat() , fake.pystr() , fake.pystr() , ts + i, fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , + fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr())) + + tdSql.execute('''insert into stable_2_1 (ts , q_int , q_bigint , q_smallint , q_tinyint , q_float , q_double, q_bool , q_binary , q_nchar, q_ts,\ + q_binary1 , q_nchar1 , q_binary2 , q_nchar2 , q_binary3 , q_nchar3 , q_binary4 , q_nchar4 , q_binary5 , q_nchar5 , q_binary6 , q_nchar6 , q_binary7 , q_nchar7, q_binary8 , q_nchar8) \ + values(%d, %d, %d, %d, %d, %f, %f, 0, 'binary.%s', 'nchar.%s', %d, 'binary1.%s', 'nchar1.%s', 'binary2.%s', 'nchar2.%s', 'binary3.%s', 'nchar3.%s', \ + 'binary4.%s', 'nchar4.%s', 'binary5.%s', 'nchar5.%s', 'binary6.%s', 'nchar6.%s', 'binary7.%s', 'nchar7.%s', 'binary8.%s', 'nchar8.%s') ;''' + % (ts + i*1000*60*60*4 +10, fake.random_int(min=-0, max=2147483647, step=1), + fake.random_int(min=-0, max=9223372036854775807, step=1), + fake.random_int(min=-0, max=32767, step=1) , fake.random_int(min=-0, max=127, step=1) , + fake.pyfloat() , fake.pyfloat() , fake.pystr() , fake.pystr() , ts + i, fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , + fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr())) + + tdSql.query("select /*+ para_tables_sort() */count(*) from stable_1;") + tdSql.checkData(0,0,3*num_random*n) + tdSql.query("select /*+ para_tables_sort() */count(*) from regular_table_1;") + tdSql.checkData(0,0,num_random*n) + + def explain_sql(self,sql): + # #执行sql解析 + sql = "explain " + sql + tdLog.info(sql) + tdSql.query(sql) + + def data_check(self,sql,mark='mark') : + tdLog.info("========mark==%s==="% mark); + try: + tdSql.query(sql,queryTimes=1) + except: + tdLog.info("sql is not support :=====%s; " %sql) + tdSql.error(sql) + + + def math_nest(self,mathlist): + + print("==========%s===start=============" %mathlist) + os.system("rm -rf %s/%s.sql" % (self.testcasePath,self.testcaseFilename)) + + #self.dropandcreateDB_random("%s" %self.db_nest, 1) + + if (mathlist == ['ABS','SQRT']) or (mathlist == ['SIN','COS','TAN','ASIN','ACOS','ATAN']) or (mathlist == ['FLOOR','CEIL','ROUND']) \ + or (mathlist == ['CSUM']) : + math_functions = mathlist + fun_fix_column = ['(q_bigint)','(q_smallint)','(q_tinyint)','(q_int)','(q_float)','(q_double)','(q_bigint_null)','(q_smallint_null)','(q_tinyint_null)','(q_int_null)','(q_float_null)','(q_double_null)'] + fun_column_1 = random.sample(math_functions,1)+random.sample(fun_fix_column,1) + math_fun_1 = str(fun_column_1).replace("[","").replace("]","").replace("'","").replace(", ","") + fun_column_2 = random.sample(math_functions,1)+random.sample(fun_fix_column,1) + math_fun_2 = str(fun_column_2).replace("[","").replace("]","").replace("'","").replace(", ","") + + fun_fix_column_j = ['(t1.q_bigint)','(t1.q_smallint)','(t1.q_tinyint)','(t1.q_int)','(t1.q_float)','(t1.q_double)','(t1.q_bigint_null)','(t1.q_smallint_null)','(t1.q_tinyint_null)','(t1.q_int_null)','(t1.q_float_null)','(t1.q_double_null)', + '(t2.q_bigint)','(t2.q_smallint)','(t2.q_tinyint)','(t2.q_int)','(t2.q_float)','(t2.q_double)','(t2.q_bigint_null)','(t2.q_smallint_null)','(t2.q_tinyint_null)','(t2.q_int_null)','(t2.q_float_null)','(t2.q_double_null)'] + fun_column_join_1 = random.sample(math_functions,1)+random.sample(fun_fix_column_j,1) + math_fun_join_1 = str(fun_column_join_1).replace("[","").replace("]","").replace("'","").replace(", ","") + fun_column_join_2 = random.sample(math_functions,1)+random.sample(fun_fix_column_j,1) + math_fun_join_2 = str(fun_column_join_2).replace("[","").replace("]","").replace("'","").replace(", ","") + + elif (mathlist == ['UNIQUE']) or (mathlist == ['HYPERLOGLOG']) or (mathlist == ['MODE']) : + math_functions = mathlist + fun_fix_column = ['(q_bigint)','(q_smallint)','(q_tinyint)','(q_int)','(q_float)','(q_double)','(q_binary)','(q_nchar)','(q_bool)','(q_ts)', + '(q_bigint_null)','(q_smallint_null)','(q_tinyint_null)','(q_int_null)','(q_float_null)','(q_double_null)','(q_binary_null)','(q_nchar_null)','(q_bool_null)','(q_ts_null)'] + fun_column_1 = random.sample(math_functions,1)+random.sample(fun_fix_column,1) + math_fun_1 = str(fun_column_1).replace("[","").replace("]","").replace("'","").replace(", ","") + fun_column_2 = random.sample(math_functions,1)+random.sample(fun_fix_column,1) + math_fun_2 = str(fun_column_2).replace("[","").replace("]","").replace("'","").replace(", ","") + + fun_fix_column_j = ['(t1.q_bigint)','(t1.q_smallint)','(t1.q_tinyint)','(t1.q_int)','(t1.q_float)','(t1.q_double)','(t1.q_bigint_null)','(t1.q_smallint_null)','(t1.q_tinyint_null)','(t1.q_int_null)','(t1.q_float_null)','(t1.q_double_null)','(t1.q_ts)','(t1.q_ts_null)', + '(t2.q_bigint)','(t2.q_smallint)','(t2.q_tinyint)','(t2.q_int)','(t2.q_float)','(t2.q_double)','(t2.q_bigint_null)','(t2.q_smallint_null)','(t2.q_tinyint_null)','(t2.q_int_null)','(t2.q_float_null)','(t2.q_double_null)','(t2.q_ts)','(t2.q_ts_null)'] + fun_column_join_1 = random.sample(math_functions,1)+random.sample(fun_fix_column_j,1) + math_fun_join_1 = str(fun_column_join_1).replace("[","").replace("]","").replace("'","").replace(", ","") + fun_column_join_2 = random.sample(math_functions,1)+random.sample(fun_fix_column_j,1) + math_fun_join_2 = str(fun_column_join_2).replace("[","").replace("]","").replace("'","").replace(", ","") + + elif (mathlist == ['TAIL']): + math_functions = mathlist + num = random.randint(1, 100) + offset_rows = random.randint(0, 100) + fun_fix_column = ['(q_bigint,num)','(q_smallint,num)','(q_tinyint,num)','(q_int,num)','(q_float,num)','(q_double,num)','(q_binary,num)','(q_nchar,num)','(q_bool,num)','(q_ts,num)', + '(q_bigint_null,num)','(q_smallint_null,num)','(q_tinyint_null,num)','(q_int_null,num)','(q_float_null,num)','(q_double_null,num)','(q_binary_null,num)','(q_nchar_null,num)','(q_bool_null,num)','(q_ts_null,num)', + '(q_bigint,num,offset_rows)','(q_smallint,num,offset_rows)','(q_tinyint,num,offset_rows)','(q_int,num,offset_rows)','(q_float,num,offset_rows)','(q_double,num,offset_rows)','(q_binary,num,offset_rows)','(q_nchar,num,offset_rows)','(q_bool,num,offset_rows)','(q_ts,num,offset_rows)', + '(q_bigint_null,num,offset_rows)','(q_smallint_null,num,offset_rows)','(q_tinyint_null,num,offset_rows)','(q_int_null,num,offset_rows)','(q_float_null,num,offset_rows)','(q_double_null,num,offset_rows)','(q_binary_null,num,offset_rows)','(q_nchar_null,num,offset_rows)','(q_bool_null,num,offset_rows)','(q_ts_null,num,offset_rows)'] + fun_column_1 = random.sample(math_functions,1)+random.sample(fun_fix_column,1) + math_fun_1 = str(fun_column_1).replace("[","").replace("]","").replace("'","").replace(", ","").replace("num",str(num)).replace("offset_rows",str(offset_rows)) + fun_column_2 = random.sample(math_functions,1)+random.sample(fun_fix_column,1) + math_fun_2 = str(fun_column_2).replace("[","").replace("]","").replace("'","").replace(", ","").replace("num",str(num)).replace("offset_rows",str(offset_rows)) + + fun_fix_column_j = ['(t1.q_bigint,num)','(t1.q_smallint,num)','(t1.q_tinyint,num)','(t1.q_int,num)','(t1.q_float,num)','(t1.q_double,num)','(t1.q_binary,num)','(t1.q_nchar,num)','(t1.q_bool,num)','(t1.q_ts,num)', + '(t1.q_bigint_null,num)','(t1.q_smallint_null,num)','(t1.q_tinyint_null,num)','(t1.q_int_null,num)','(t1.q_float_null,num)','(t1.q_double_null,num)','(t1.q_binary_null,num)','(t1.q_nchar_null,num)','(t1.q_bool_null,num)','(t1.q_ts_null,num)', + '(t2.q_bigint,num)','(t2.q_smallint,num)','(t2.q_tinyint,num)','(t2.q_int,num)','(t2.q_float,num)','(t2.q_double,num)','(t2.q_binary,num)','(t2.q_nchar,num)','(t2.q_bool,num)','(t2.q_ts,num)', + '(t2.q_bigint_null,num)','(t2.q_smallint_null,num)','(t2.q_tinyint_null,num)','(t2.q_int_null,num)','(t2.q_float_null,num)','(t2.q_double_null,num)','(t2.q_binary_null,num)','(t2.q_nchar_null,num)','(t2.q_bool_null,num)','(t2.q_ts_null,num)', + '(t1.q_bigint,num,offset_rows)','(t1.q_smallint,num,offset_rows)','(t1.q_tinyint,num,offset_rows)','(t1.q_int,num,offset_rows)','(t1.q_float,num,offset_rows)','(t1.q_double,num,offset_rows)','(t1.q_binary,num,offset_rows)','(t1.q_nchar,num,offset_rows)','(t1.q_bool,num,offset_rows)','(t1.q_ts,num,offset_rows)', + '(t1.q_bigint_null,num,offset_rows)','(t1.q_smallint_null,num,offset_rows)','(t1.q_tinyint_null,num,offset_rows)','(t1.q_int_null,num,offset_rows)','(t1.q_float_null,num,offset_rows)','(t1.q_double_null,num,offset_rows)','(t1.q_binary_null,num,offset_rows)','(t1.q_nchar_null,num,offset_rows)','(t1.q_bool_null,num,offset_rows)','(t1.q_ts_null,num,offset_rows)', + '(t2.q_bigint,num,offset_rows)','(t2.q_smallint,num,offset_rows)','(t2.q_tinyint,num,offset_rows)','(t2.q_int,num,offset_rows)','(t2.q_float,num,offset_rows)','(t2.q_double,num,offset_rows)','(t2.q_binary,num,offset_rows)','(t2.q_nchar,num,offset_rows)','(t2.q_bool,num,offset_rows)','(t2.q_ts,num,offset_rows)', + '(t2.q_bigint_null,num,offset_rows)','(t2.q_smallint_null,num,offset_rows)','(t2.q_tinyint_null,num,offset_rows)','(t2.q_int_null,num,offset_rows)','(t2.q_float_null,num,offset_rows)','(t2.q_double_null,num,offset_rows)','(t2.q_binary_null,num,offset_rows)','(t2.q_nchar_null,num,offset_rows)','(t2.q_bool_null,num,offset_rows)','(t2.q_ts_null,num,offset_rows)'] + fun_column_join_1 = random.sample(math_functions,1)+random.sample(fun_fix_column_j,1) + math_fun_join_1 = str(fun_column_join_1).replace("[","").replace("]","").replace("'","").replace(", ","").replace("num",str(num)).replace("offset_rows",str(offset_rows)) + fun_column_join_2 = random.sample(math_functions,1)+random.sample(fun_fix_column_j,1) + math_fun_join_2 = str(fun_column_join_2).replace("[","").replace("]","").replace("'","").replace(", ","").replace("num",str(num)).replace("offset_rows",str(offset_rows)) + + elif (mathlist == ['POW','LOG']) or (mathlist == ['MAVG']) or (mathlist == ['SAMPLE']) : + math_functions = mathlist + num = random.randint(1, 1000) + fun_fix_column = ['(q_bigint,num)','(q_smallint,num)','(q_tinyint,num)','(q_int,num)','(q_float,num)','(q_double,num)', + '(q_bigint_null,num)','(q_smallint_null,num)','(q_tinyint_null,num)','(q_int_null,num)','(q_float_null,num)','(q_double_null,num)'] + fun_column_1 = random.sample(math_functions,1)+random.sample(fun_fix_column,1) + math_fun_1 = str(fun_column_1).replace("[","").replace("]","").replace("'","").replace(", ","").replace("num",str(num)) + fun_column_2 = random.sample(math_functions,1)+random.sample(fun_fix_column,1) + math_fun_2 = str(fun_column_2).replace("[","").replace("]","").replace("'","").replace(", ","").replace("num",str(num)) + + fun_fix_column_j = ['(t1.q_bigint,num)','(t1.q_smallint,num)','(t1.q_tinyint,num)','(t1.q_int,num)','(t1.q_float,num)','(t1.q_double,num)', + '(t1.q_bigint_null,num)','(t1.q_smallint_null,num)','(t1.q_tinyint_null,num)','(t1.q_int_null,num)','(t1.q_float_null,num)','(t1.q_double_null,num)', + '(t2.q_bigint,num)','(t2.q_smallint,num)','(t2.q_tinyint,num)','(t2.q_int,num)','(t2.q_float,num)','(t2.q_double,num)', + '(t2.q_bigint_null,num)','(t2.q_smallint_null,num)','(t2.q_tinyint_null,num)','(t2.q_int_null,num)','(t2.q_float_null,num)','(t2.q_double_null,num)'] + fun_column_join_1 = random.sample(math_functions,1)+random.sample(fun_fix_column_j,1) + math_fun_join_1 = str(fun_column_join_1).replace("[","").replace("]","").replace("'","").replace(", ","").replace("num",str(num)) + fun_column_join_2 = random.sample(math_functions,1)+random.sample(fun_fix_column_j,1) + math_fun_join_2 = str(fun_column_join_2).replace("[","").replace("]","").replace("'","").replace(", ","").replace("num",str(num)) + + elif (mathlist == ['statecount','stateduration']): + math_functions = mathlist + num = random.randint(-1000, 1000) + + operator = ['LT' , 'GT' ,'GE','NE','EQ'] + oper = str(random.sample(operator,1)).replace("[","").replace("]","")#.replace("'","") + + fun_fix_column = ['(q_bigint,oper,num,time)','(q_smallint,oper,num,time)','(q_tinyint,oper,num,time)','(q_int,oper,num,time)','(q_float,oper,num,time)','(q_double,oper,num,time)', + '(q_bigint_null,oper,num,time)','(q_smallint_null,oper,num,time)','(q_tinyint_null,oper,num,time)','(q_int_null,oper,num,time)','(q_float_null,oper,num,time)','(q_double_null,oper,num,time)'] + + hanshu_select1 = random.sample(math_functions,1) + fun_column_1 = random.sample(hanshu_select1,1)+random.sample(fun_fix_column,1) + math_fun_1 = str(fun_column_1).replace("[","").replace("]","").replace("'","").replace(", ","") + + if str(hanshu_select1).replace("[","").replace("]","").replace("'","") == 'statecount': + math_fun_1 = math_fun_1.replace("oper","%s" %oper).replace(",time","").replace("num",str(num)) + elif str(hanshu_select1).replace("[","").replace("]","").replace("'","") == 'stateduration': + timeunit = ['1s' , '1m' ,'1h'] + time = str(random.sample(timeunit,1)).replace("[","").replace("]","").replace("'","") + math_fun_1 = math_fun_1.replace("oper","%s" %oper).replace("time","%s" %time).replace("num",str(num)) + + hanshu_select2 = random.sample(math_functions,1) + fun_column_2 = random.sample(hanshu_select2,1)+random.sample(fun_fix_column,1) + math_fun_2 = str(fun_column_2).replace("[","").replace("]","").replace("'","").replace(", ","") + + if str(hanshu_select2).replace("[","").replace("]","").replace("'","") == 'statecount': + math_fun_2 = math_fun_2.replace("oper","%s" %oper).replace(",time","").replace("num",str(num)) + elif str(hanshu_select2).replace("[","").replace("]","").replace("'","") == 'stateduration': + timeunit = ['1s' , '1m' ,'1h'] + time = str(random.sample(timeunit,1)).replace("[","").replace("]","").replace("'","") + math_fun_2 = math_fun_2.replace("oper","%s" %oper).replace("time","%s" %time).replace("num",str(num)) + + fun_fix_column_j = ['(t1.q_bigint,oper,num,time)','(t1.q_smallint,oper,num,time)','(t1.q_tinyint,oper,num,time)','(t1.q_int,oper,num,time)','(t1.q_float,oper,num,time)','(t1.q_double,oper,num,time)', + '(t1.q_bigint_null,oper,num,time)','(t1.q_smallint_null,oper,num,time)','(t1.q_tinyint_null,oper,num,time)','(t1.q_int_null,oper,num,time)','(t1.q_float_null,oper,num,time)','(t1.q_double_null,oper,num,time)', + '(t2.q_bigint,oper,num,time)','(t2.q_smallint,oper,num,time)','(t2.q_tinyint,oper,num,time)','(t2.q_int,oper,num,time)','(t2.q_float,oper,num,time)','(t2.q_double,oper,num,time)', + '(t2.q_bigint_null,oper,num,time)','(t2.q_smallint_null,oper,num,time)','(t2.q_tinyint_null,oper,num,time)','(t2.q_int_null,oper,num,time)','(t2.q_float_null,oper,num,time)','(t2.q_double_null,oper,num,time)'] + + hanshu_select_join_1 = random.sample(math_functions,1) + fun_column_join_1 = random.sample(hanshu_select_join_1,1)+random.sample(fun_fix_column_j,1) + math_fun_join_1 = str(fun_column_join_1).replace("[","").replace("]","").replace("'","").replace(", ","") + + if str(hanshu_select_join_1).replace("[","").replace("]","").replace("'","") == 'statecount': + math_fun_join_1 = math_fun_join_1.replace("oper","%s" %oper).replace(",time","").replace("num",str(num)) + elif str(hanshu_select_join_1).replace("[","").replace("]","").replace("'","") == 'stateduration': + timeunit = ['1s' , '1m' ,'1h'] + time = str(random.sample(timeunit,1)).replace("[","").replace("]","").replace("'","") + math_fun_join_1 = math_fun_join_1.replace("oper","%s" %oper).replace("time","%s" %time).replace("num",str(num)) + + hanshu_select_join_2 = random.sample(math_functions,1) + fun_column_join_2 = random.sample(hanshu_select_join_2,1)+random.sample(fun_fix_column_j,1) + math_fun_join_2 = str(fun_column_join_2).replace("[","").replace("]","").replace("'","").replace(", ","") + + if str(hanshu_select_join_2).replace("[","").replace("]","").replace("'","") == 'statecount': + math_fun_join_2 = math_fun_join_2.replace("oper","%s" %oper).replace(",time","").replace("num",str(num)) + elif str(hanshu_select_join_2).replace("[","").replace("]","").replace("'","") == 'stateduration': + timeunit = ['1s' , '1m' ,'1h'] + time = str(random.sample(timeunit,1)).replace("[","").replace("]","").replace("'","") + math_fun_join_2 = math_fun_join_2.replace("oper","%s" %oper).replace("time","%s" %time).replace("num",str(num)) + + elif(mathlist == ['HISTOGRAM']) : + math_functions = mathlist + fun_fix_column = ['(q_bigint','(q_smallint','(q_tinyint','(q_int','(q_float','(q_double','(q_bigint_null','(q_smallint_null','(q_tinyint_null','(q_int_null','(q_float_null','(q_double_null'] + + fun_fix_column_j = ['(t1.q_bigint','(t1.q_smallint','(t1.q_tinyint','(t1.q_int','(t1.q_float','(t1.q_double','(t1.q_bigint_null','(t1.q_smallint_null','(t1.q_tinyint_null','(t1.q_int_null','(t1.q_float_null','(t1.q_double_null', + '(t2.q_bigint','(t2.q_smallint','(t2.q_tinyint','(t2.q_int','(t2.q_float','(t2.q_double','(t2.q_bigint_null','(t2.q_smallint_null','(t2.q_tinyint_null','(t2.q_int_null','(t2.q_float_null','(t2.q_double_null'] + + normalized = random.randint(0, 1) + + i = random.randint(1,3) + if i == 1: + bin_type = 'user_input' + bin_description = {-11111119395555977777} #9一会转译成, + fun_column_1 = [math_functions , random.sample(fun_fix_column,1), ',',"'%s'" %bin_type, ',',"'%s'" % bin_description, ',', "%d" %normalized,')'] + math_fun_1 = str(fun_column_1).replace("[","").replace("]","").replace("'","").replace(", ","").replace("{","[").replace("}","]").replace("9",",") + + fun_column_2 = [math_functions , random.sample(fun_fix_column,1), ',',"'%s'" %bin_type, ',',"'%s'" % bin_description, ',', "%d" %normalized,')'] + math_fun_2 = str(fun_column_2).replace("[","").replace("]","").replace("'","").replace(", ","").replace("{","[").replace("}","]").replace("9",",") + + fun_column_join_1 = [math_functions , random.sample(fun_fix_column_j,1), ',',"'%s'" %bin_type, ',',"'%s'" % bin_description, ',', "%d" %normalized,')'] + math_fun_join_1 = str(fun_column_join_1).replace("[","").replace("]","").replace("'","").replace(", ","").replace("{","[").replace("}","]").replace("9",",") + + fun_column_join_2 = [math_functions , random.sample(fun_fix_column_j,1), ',',"'%s'" %bin_type, ',',"'%s'" % bin_description, ',', "%d" %normalized,')'] + math_fun_join_2 = str(fun_column_join_2).replace("[","").replace("]","").replace("'","").replace(", ","").replace("{","[").replace("}","]").replace("9",",") + + elif i == 2: + bin_type = 'linear_bin' + true_false = random.randint(10, 11) + bin_description = {"ZstartZ": -333339, "ZwidthZ":559, "ZcountZ":59, "ZinfinityZ":'%d' %true_false} #Z一会转译成" ,9一会转译成 , + fun_column_1 = [math_functions , random.sample(fun_fix_column,1), ',',"'%s'" %bin_type, ',','%s' % bin_description, ',', "%d" %normalized,')'] + math_fun_1 = str(fun_column_1).replace("[","").replace("]","").replace("'","").replace(", ","").replace("9",",").replace("Z","\"").replace("10","false").replace("11","true").replace("\"{","'{").replace("}\"","}'") + + fun_column_2 = [math_functions , random.sample(fun_fix_column,1), ',',"'%s'" %bin_type, ',','%s' % bin_description, ',', "%d" %normalized,')'] + math_fun_2 = str(fun_column_2).replace("[","").replace("]","").replace("'","").replace(", ","").replace("9",",").replace("Z","\"").replace("10","false").replace("11","true").replace("\"{","'{").replace("}\"","}'") + + fun_column_join_1 = [math_functions , random.sample(fun_fix_column_j,1), ',',"'%s'" %bin_type, ',','%s' % bin_description, ',', "%d" %normalized,')'] + math_fun_join_1 = str(fun_column_join_1).replace("[","").replace("]","").replace("'","").replace(", ","").replace("9",",").replace("Z","\"").replace("10","false").replace("11","true").replace("\"{","'{").replace("}\"","}'") + + fun_column_join_2 = [math_functions , random.sample(fun_fix_column_j,1), ',',"'%s'" %bin_type, ',','%s' % bin_description, ',', "%d" %normalized,')'] + math_fun_join_2 = str(fun_column_join_2).replace("[","").replace("]","").replace("'","").replace(", ","").replace("9",",").replace("Z","\"").replace("10","false").replace("11","true").replace("\"{","'{").replace("}\"","}'") + + elif i == 3: + bin_type = 'log_bin' + true_false = random.randint(10, 11) + bin_description = {"ZstartZ": -333339, "ZfactorZ":559, "ZcountZ":59, "ZinfinityZ":'%d' %true_false} #Z一会转译成" ,9一会转译成 , + fun_column_1 = [math_functions , random.sample(fun_fix_column,1), ',',"'%s'" %bin_type, ',','%s' % bin_description, ',', "%d" %normalized,')'] + math_fun_1 = str(fun_column_1).replace("[","").replace("]","").replace("'","").replace(", ","").replace("9",",").replace("Z","\"").replace("10","false").replace("11","true").replace("\"{","'{").replace("}\"","}'") + + fun_column_2 = [math_functions , random.sample(fun_fix_column,1), ',',"'%s'" %bin_type, ',','%s' % bin_description, ',', "%d" %normalized,')'] + math_fun_2 = str(fun_column_2).replace("[","").replace("]","").replace("'","").replace(", ","").replace("9",",").replace("Z","\"").replace("10","false").replace("11","true").replace("\"{","'{").replace("}\"","}'") + + fun_column_join_1 = [math_functions , random.sample(fun_fix_column_j,1), ',',"'%s'" %bin_type, ',','%s' % bin_description, ',', "%d" %normalized,')'] + math_fun_join_1 = str(fun_column_join_1).replace("[","").replace("]","").replace("'","").replace(", ","").replace("9",",").replace("Z","\"").replace("10","false").replace("11","true").replace("\"{","'{").replace("}\"","}'") + + fun_column_join_2 = [math_functions , random.sample(fun_fix_column_j,1), ',',"'%s'" %bin_type, ',','%s' % bin_description, ',', "%d" %normalized,')'] + math_fun_join_2 = str(fun_column_join_2).replace("[","").replace("]","").replace("'","").replace(", ","").replace("9",",").replace("Z","\"").replace("10","false").replace("11","true").replace("\"{","'{").replace("}\"","}'") + + tdSql.query("select /*+ para_tables_sort() */1-1 as math_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (mathlist == ['SIN','COS','TAN','ASIN','ACOS','ATAN']) or (mathlist == ['ABS','SQRT']) \ + or (mathlist == ['POW','LOG']) or (mathlist == ['FLOOR','CEIL','ROUND']): + sql = "select /*+ para_tables_sort() */ %s as asct1, " % math_fun_1 + sql += "%s as asct2, " % math_fun_2 + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts as ts1 from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + elif (mathlist == ['MAVG']) or (mathlist == ['SAMPLE']) or (mathlist == ['TAIL']) or (mathlist == ['CSUM']) or (mathlist == ['HISTOGRAM']) \ + or (mathlist == ['HYPERLOGLOG']) or (mathlist == ['UNIQUE']) or (mathlist == ['MODE']) or (mathlist == ['statecount','stateduration']) : + sql = "select /*+ para_tables_sort() */ %s as asct1 " % math_fun_1 + sql += " from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-2 as math_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (mathlist == ['SIN','COS','TAN','ASIN','ACOS','ATAN']) or (mathlist == ['ABS','SQRT']) \ + or (mathlist == ['POW','LOG']) or (mathlist == ['FLOOR','CEIL','ROUND']) : + sql = "(select /*+ para_tables_sort() */ %s as asct1, " % math_fun_1 + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts ts from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s )" % random.choice(self.order_where) + sql += "%s " % random.choice(self.unionall_or_union) + sql += "(select /*+ para_tables_sort() */ %s as asct2, " % math_fun_2 + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts ts from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ")" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + elif (mathlist == ['MAVG']) or (mathlist == ['SAMPLE']) or (mathlist == ['TAIL']) or (mathlist == ['CSUM']) or (mathlist == ['HISTOGRAM']) \ + or (mathlist == ['HYPERLOGLOG']) or (mathlist == ['UNIQUE']) or (mathlist == ['MODE']) or (mathlist == ['statecount','stateduration']): + sql = "(select /*+ para_tables_sort() */ %s as asct1 " % math_fun_1 + sql += "from regular_table_1 where " + sql += "%s )" % random.choice(self.q_where) + sql += "%s " % random.choice(self.unionall_or_union) + sql += "(select /*+ para_tables_sort() */ %s as asct2 " % math_fun_2 + sql += " from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ")" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-3 as math_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (mathlist == ['SIN','COS','TAN','ASIN','ACOS','ATAN']) or (mathlist == ['ABS','SQRT']) \ + or (mathlist == ['POW','LOG']) or (mathlist == ['FLOOR','CEIL','ROUND']) : + sql = "(select /*+ para_tables_sort() */ %s as asct1, ts ," % math_fun_1 + sql += "%s as asct2, " % math_fun_2 + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s " % random.choice(self.q_select) + sql += " from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += ") %s (select /*+ para_tables_sort() */" % random.choice(self.unionall_or_union) + sql += "%s as asct2, ts ," % math_fun_2 + sql += "%s as asct1, " % math_fun_1 + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s " % random.choice(self.q_select) + sql += " from regular_table_2 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += ")" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + elif (mathlist == ['MODE']) : + sql = "(select /*+ para_tables_sort() */%s as asct1," % math_fun_1 + sql += "%s as asct2 " % math_fun_2 + sql += " from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += ") %s (select /*+ para_tables_sort() */" % random.choice(self.unionall_or_union) + sql += "%s as asct2 ," % math_fun_2 + sql += "%s as asct1 " % math_fun_1 + sql += " from regular_table_2 where " + sql += "%s " % random.choice(self.q_where) + sql += ")" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + elif (mathlist == ['MAVG']) or (mathlist == ['statecount','stateduration']) or (mathlist == ['CSUM']) or (mathlist == ['HISTOGRAM']) \ + or (mathlist == ['TAIL']) or (mathlist == ['HYPERLOGLOG']) or (mathlist == ['UNIQUE']) or (mathlist == ['SAMPLE']): + sql = "(select /*+ para_tables_sort() */%s as asct1 " % math_fun_1 + sql += " from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += ") %s (select /*+ para_tables_sort() */" % random.choice(self.unionall_or_union) + sql += "%s as asct1 " % math_fun_2 + sql += " from regular_table_2 where " + sql += "%s " % random.choice(self.q_where) + sql += " order by asct1 asc " + sql += ")" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-4 as math_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (mathlist == ['SIN','COS','TAN','ASIN','ACOS','ATAN']) or (mathlist == ['ABS','SQRT']) \ + or (mathlist == ['POW','LOG']) or (mathlist == ['FLOOR','CEIL','ROUND']) : + sql = "select /*+ para_tables_sort() */ ts1,ts2 ,timediff(ts1,ts2), asct1 from ( select /*+ para_tables_sort() */ t1.ts as ts1," + sql += "%s as asct0, " % math_fun_join_1 + sql += "%s as asct1, " % math_fun_join_2 + sql += "%s as asct2, " % math_fun_join_1 + sql += "t1.%s as asct12, " % random.choice(self.q_select) + sql += "t2.%s as asct22, " % random.choice(self.q_select) + sql += "t2.ts as ts2 from regular_table_1 t1 , regular_table_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_where) + sql += "and %s " % random.choice(self.q_u_or_where) + sql += "%s " % random.choice(self.order_u_where) + sql += ");" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + elif (mathlist == ['MAVG']) or (mathlist == ['SAMPLE'])or (mathlist == ['TAIL']) or (mathlist == ['CSUM']) or (mathlist == ['HISTOGRAM']) \ + or (mathlist == ['HYPERLOGLOG']) or (mathlist == ['UNIQUE']) or (mathlist == ['MODE']) or (mathlist == ['statecount','stateduration']) : + sql = "select /*+ para_tables_sort() */ count(asct1) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1 " % math_fun_join_2 + sql += "from regular_table_1 t1 , regular_table_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_where) + sql += "and %s " % random.choice(self.q_u_or_where) + sql += ");" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-5 as math_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (mathlist == ['SIN','COS','TAN','ASIN','ACOS','ATAN']) or (mathlist == ['ABS','SQRT']) \ + or (mathlist == ['POW','LOG']) or (mathlist == ['FLOOR','CEIL','ROUND']) : + sql = "select /*+ para_tables_sort() */ ts ," + sql += "%s, " % math_fun_1 + sql += "%s as asct1, " % random.choice(self.q_select) + sql += "%s as asct2, " % random.choice(self.q_select) + sql += "%s " % math_fun_2 + sql += " from regular_table_1" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + elif (mathlist == ['MAVG']) or (mathlist == ['SAMPLE'])or (mathlist == ['TAIL']) or (mathlist == ['CSUM']) or (mathlist == ['HISTOGRAM']) \ + or (mathlist == ['HYPERLOGLOG']) or (mathlist == ['UNIQUE']) or (mathlist == ['MODE']) or (mathlist == ['statecount','stateduration']) : + sql = "select /*+ para_tables_sort() */" + sql += "%s " % math_fun_2 + sql += " from regular_table_1" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-6 as math_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (mathlist == ['SIN','COS','TAN','ASIN','ACOS','ATAN']) or (mathlist == ['ABS','SQRT']) \ + or (mathlist == ['POW','LOG']) or (mathlist == ['FLOOR','CEIL','ROUND']) : + sql = "select /*+ para_tables_sort() */ ts1 ,timediff(ts1,ts2), max(asct1) from ( select /*+ para_tables_sort() */ t1.ts,t1.ts as ts1," + sql += "%s as asct0, " % math_fun_join_1 + sql += "%s as asct1, " % math_fun_join_2 + sql += "t1.%s as asct11, " % random.choice(self.q_select) + sql += "t2.%s as asct12, " % random.choice(self.q_select) + sql += "%s as asct13, " % math_fun_join_1 + sql += "t2.ts as ts2 from regular_table_1 t1 , regular_table_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_where) + sql += "and %s )" % random.choice(self.q_u_or_where) + sql += "%s " % random.choice(self.order_where) + sql += "%s " % random.choice(self.limit1_where) + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + elif (mathlist == ['MAVG']) or (mathlist == ['SAMPLE'])or (mathlist == ['TAIL']) or (mathlist == ['CSUM']) or (mathlist == ['HISTOGRAM']) \ + or (mathlist == ['HYPERLOGLOG']) or (mathlist == ['UNIQUE']) or (mathlist == ['MODE']) or (mathlist == ['statecount','stateduration']) : + sql = "select /*+ para_tables_sort() */ count(asct1) from ( select /*+ para_tables_sort() */" + sql += "%s as asct1 " % math_fun_join_2 + sql += "from regular_table_1 t1 , regular_table_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_where) + sql += "and %s )" % random.choice(self.q_u_or_where) + sql += "%s " % random.choice(self.limit1_where) + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-7 as math_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (mathlist == ['SIN','COS','TAN','ASIN','ACOS','ATAN']) or (mathlist == ['ABS','SQRT']) \ + or (mathlist == ['POW','LOG']) or (mathlist == ['FLOOR','CEIL','ROUND']) : + sql = "select /*+ para_tables_sort() */ ts1,ts2 , abs(asct1) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1, ts as ts1," % math_fun_1 + sql += "%s as asct2, " % math_fun_2 + sql += "%s, " % random.choice(self.q_select) + sql += "%s, " % random.choice(self.t_select) + sql += "ts as ts2 from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += "%s " % random.choice(self.order_where) + sql += ") ;" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + elif (mathlist == ['MAVG']) or (mathlist == ['SAMPLE']) or (mathlist == ['TAIL']) or (mathlist == ['CSUM']) or (mathlist == ['HISTOGRAM']) \ + or (mathlist == ['HYPERLOGLOG']) or (mathlist == ['UNIQUE']) or (mathlist == ['MODE']) or (mathlist == ['statecount','stateduration']) : + sql = "select /*+ para_tables_sort() */ count(asct1) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1 " % math_fun_1 + sql += "from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += ") ;" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-8 as math_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (mathlist == ['SIN','COS','TAN','ASIN','ACOS','ATAN']) or (mathlist == ['ABS','SQRT']) \ + or (mathlist == ['POW','LOG']) or (mathlist == ['FLOOR','CEIL','ROUND']) : + sql = "select /*+ para_tables_sort() */ %s, " % random.choice(self.s_s_select) + sql += "%s as asct1, ts as ts1," % math_fun_1 + sql += "%s as asct2, " % math_fun_2 + sql += "%s, " % random.choice(self.q_select) + sql += "%s, " % random.choice(self.t_select) + sql += "ts as ts2 from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += "%s " % random.choice(self.order_where) + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + elif (mathlist == ['MAVG']) or (mathlist == ['SAMPLE'])or (mathlist == ['TAIL']) or (mathlist == ['CSUM']) or (mathlist == ['HISTOGRAM']) \ + or (mathlist == ['HYPERLOGLOG']) or (mathlist == ['UNIQUE']) or (mathlist == ['MODE']) or (mathlist == ['statecount','stateduration']) : + sql = "select /*+ para_tables_sort() */ %s as asct1 " % math_fun_1 + sql += " from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-9 as math_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (mathlist == ['SIN','COS','TAN','ASIN','ACOS','ATAN']) or (mathlist == ['ABS','SQRT']) \ + or (mathlist == ['POW','LOG']) or (mathlist == ['FLOOR','CEIL','ROUND']) : + sql = "select /*+ para_tables_sort() */ %s, " % math_fun_join_1 + sql += "%s as asct1, " % math_fun_join_2 + sql += "t1.%s as asct11, " % random.choice(self.q_select) + sql += "t1.%s as asct12, " % random.choice(self.q_select) + sql += "t2.%s as asct21, " % random.choice(self.q_select) + sql += "t2.%s as asct22, " % random.choice(self.q_select) + sql += "t2.ts as ts2 from stable_1 t1 , stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.t_join_where) + sql += "and %s " % random.choice(self.t_u_where) + sql += "and %s " % random.choice(self.t_u_or_where) + sql += "%s " % random.choice(self.order_u_where) + sql += "%s " % random.choice(self.limit1_where) + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + elif (mathlist == ['MAVG']) or (mathlist == ['SAMPLE'])or (mathlist == ['TAIL']) or (mathlist == ['CSUM']) or (mathlist == ['HISTOGRAM']) \ + or (mathlist == ['HYPERLOGLOG']) or (mathlist == ['UNIQUE']) or (mathlist == ['MODE']) or (mathlist == ['statecount','stateduration']) : + sql = "select /*+ para_tables_sort() */ %s as asct1 " % math_fun_join_2 + sql += "from stable_1 t1 , stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.t_join_where) + sql += "and %s " % random.choice(self.t_u_where) + sql += "and %s " % random.choice(self.t_u_or_where) + sql += "%s " % random.choice(self.limit1_where) + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-10 as math_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (mathlist == ['SIN','COS','TAN','ASIN','ACOS','ATAN']) or (mathlist == ['ABS','SQRT']) \ + or (mathlist == ['POW','LOG']) or (mathlist == ['FLOOR','CEIL','ROUND']) : + sql = "(select /*+ para_tables_sort() */ %s as asct1 ," % math_fun_1 + sql += "%s as asct2, " % math_fun_2 + sql += "%s, " % random.choice(self.s_r_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += ") %s " % random.choice(self.unionall_or_union) + sql += "(select /*+ para_tables_sort() */ " + sql += "%s as asct1 ," % math_fun_1 + sql += "%s as asct2, " % math_fun_2 + sql += "%s, " % random.choice(self.s_r_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts from stable_2 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ")" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + elif (mathlist == ['MAVG']) or (mathlist == ['SAMPLE'])or (mathlist == ['TAIL']) or (mathlist == ['CSUM']) or (mathlist == ['HISTOGRAM']) \ + or (mathlist == ['HYPERLOGLOG']) or (mathlist == ['UNIQUE']) or (mathlist == ['MODE']) or (mathlist == ['statecount','stateduration']) : + sql = "(select /*+ para_tables_sort() */%s as asct1 " % math_fun_1 + sql += " from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += ") %s " % random.choice(self.unionall_or_union) + sql += "(select /*+ para_tables_sort() */ " + sql += "%s as asct2 " % math_fun_2 + sql += "from stable_2 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ")" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + #3 inter union not support + tdSql.query("select /*+ para_tables_sort() */1-11 as math_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (mathlist == ['SIN','COS','TAN','ASIN','ACOS','ATAN']) or (mathlist == ['ABS','SQRT']) \ + or (mathlist == ['POW','LOG']) or (mathlist == ['FLOOR','CEIL','ROUND']) : + sql = "(select /*+ para_tables_sort() */ %s as asct1, ts ," % math_fun_1 + sql += "%s as asct2, " % math_fun_2 + sql += "%s, " % random.choice(self.s_r_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts as t2ts from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += " ) %s (" % random.choice(self.unionall_or_union) + sql += " select /*+ para_tables_sort() */ " + sql += "%s as asct1, ts as t1ts," % math_fun_1 + sql += "%s as asct2, " % math_fun_2 + sql += "%s, " % random.choice(self.s_r_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts as t2ts from stable_2 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ")" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + elif (mathlist == ['MAVG']) or (mathlist == ['SAMPLE'])or (mathlist == ['TAIL']) or (mathlist == ['CSUM']) or (mathlist == ['HISTOGRAM']) \ + or (mathlist == ['HYPERLOGLOG']) or (mathlist == ['UNIQUE']) or (mathlist == ['MODE']) or (mathlist == ['statecount','stateduration']) : + sql = "select /*+ para_tables_sort() */%s as asct1 " % math_fun_1 + sql += " from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += " %s " % random.choice(self.unionall_or_union) + sql += " select /*+ para_tables_sort() */ " + sql += "%s as asct2 " % math_fun_2 + sql += " from stable_2 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.limit1_where) + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-12 as math_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (mathlist == ['SIN','COS','TAN','ASIN','ACOS','ATAN']) or (mathlist == ['ABS','SQRT']) \ + or (mathlist == ['POW','LOG']) or (mathlist == ['FLOOR','CEIL','ROUND']) : + sql = "select /*+ para_tables_sort() */ ts1,ts2 ,timediff(ts1,ts2), max(asct1) from ( select /*+ para_tables_sort() */ t1.ts as ts1," + sql += "%s, " % math_fun_join_1 + sql += "%s as asct1, " % math_fun_join_2 + sql += "t1.%s as asct11, " % random.choice(self.q_select) + sql += "t1.%s as asct12, " % random.choice(self.q_select) + sql += "t2.%s as asct21, " % random.choice(self.q_select) + sql += "t2.%s as asct111, " % random.choice(self.q_select) + sql += "t2.ts as ts2 from stable_1 t1 , stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.t_u_where) + sql += "and %s " % random.choice(self.t_u_or_where) + sql += "%s " % random.choice(self.order_u_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ");" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + elif (mathlist == ['MAVG']) or (mathlist == ['SAMPLE'])or (mathlist == ['TAIL']) or (mathlist == ['CSUM']) or (mathlist == ['HISTOGRAM']) \ + or (mathlist == ['HYPERLOGLOG']) or (mathlist == ['UNIQUE']) or (mathlist == ['MODE']) or (mathlist == ['statecount','stateduration']) : + sql = "select /*+ para_tables_sort() */count(asct1) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1 " % math_fun_join_2 + sql += " from stable_1 t1 , stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.t_u_where) + sql += "and %s " % random.choice(self.t_u_or_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ");" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-13 as math_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (mathlist == ['SIN','COS','TAN','ASIN','ACOS','ATAN']) or (mathlist == ['ABS','SQRT']) \ + or (mathlist == ['POW','LOG']) or (mathlist == ['FLOOR','CEIL','ROUND']) : + sql = "select /*+ para_tables_sort() */ ts ," + sql += "%s as asct11, " % math_fun_1 + sql += "%s as asct12, " % random.choice(self.q_select) + sql += "%s as asct13, " % random.choice(self.q_select) + sql += "%s as asct14, " % math_fun_2 + sql += "%s as asct15 " % random.choice(self.t_select) + sql += " from stable_1 " + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + elif (mathlist == ['MAVG']) or (mathlist == ['SAMPLE'])or (mathlist == ['TAIL']) or (mathlist == ['CSUM']) or (mathlist == ['HISTOGRAM']) \ + or (mathlist == ['HYPERLOGLOG']) or (mathlist == ['UNIQUE']) or (mathlist == ['MODE']) or (mathlist == ['statecount','stateduration']) : + sql = "select /*+ para_tables_sort() */ " + sql += "%s " % math_fun_2 + sql += "%s " % random.choice(self.t_select) + sql += " from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-14 as math_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (mathlist == ['SIN','COS','TAN','ASIN','ACOS','ATAN']) or (mathlist == ['ABS','SQRT']) \ + or (mathlist == ['POW','LOG']) or (mathlist == ['FLOOR','CEIL','ROUND']) : + sql = "select /*+ para_tables_sort() */ avg(asct1),count(asct2) from ( select /*+ para_tables_sort() */" + sql += "%s as asct1, " % math_fun_1 + sql += "%s as asct2" % math_fun_2 + sql += " from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.partiton_where) + sql += "%s " % random.choice(self.order_desc_where) + sql += "%s " % random.choice([self.limit_where[2] , self.limit_where[3]] ) + sql += " ) ;" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + elif (mathlist == ['MAVG']) or (mathlist == ['SAMPLE'])or (mathlist == ['TAIL']) or (mathlist == ['CSUM']) or (mathlist == ['HISTOGRAM']) \ + or (mathlist == ['HYPERLOGLOG']) or (mathlist == ['UNIQUE']) or (mathlist == ['MODE']) or (mathlist == ['statecount','stateduration']) : + sql = "select /*+ para_tables_sort() */ count(asct1) from ( select /*+ para_tables_sort() */" + sql += "%s as asct1 " % math_fun_1 + sql += " from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.partiton_where) + sql += "%s " % random.choice([self.limit_where[2] , self.limit_where[3]] ) + sql += " ) ;" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-15 as math_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (mathlist == ['SIN','COS','TAN','ASIN','ACOS','ATAN']) or (mathlist == ['ABS','SQRT']) \ + or (mathlist == ['POW','LOG']) or (mathlist == ['FLOOR','CEIL','ROUND']) : + sql = "select /*+ para_tables_sort() */ ts1,ts ,timediff(ts1,ts), max(asct1) from ( select /*+ para_tables_sort() */ t1.ts as ts1," + sql += "%s, " % math_fun_join_1 + sql += "%s as asct1, " % math_fun_join_2 + sql += "t1.%s as asct11, " % random.choice(self.q_select) + sql += "t1.%s as asct12, " % random.choice(self.q_select) + sql += "t2.%s as asct13, " % random.choice(self.q_select) + sql += "t2.%s as asct14, " % random.choice(self.q_select) + sql += "t2.ts from stable_1 t1, stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.t_join_where) + sql += " and %s " % random.choice(self.qt_u_or_where) + sql += "%s " % random.choice(self.partiton_where_j) + sql += "%s " % random.choice(self.slimit1_where) + sql += ") " + sql += "%s " % random.choice(self.order_desc_where) + sql += "%s ;" % random.choice(self.limit_u_where) + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + elif (mathlist == ['MAVG']) or (mathlist == ['SAMPLE']) or (mathlist == ['TAIL']) or (mathlist == ['CSUM']) or (mathlist == ['HISTOGRAM']) \ + or (mathlist == ['HYPERLOGLOG']) or (mathlist == ['UNIQUE']) or (mathlist == ['MODE']) or (mathlist == ['statecount','stateduration']) : + sql = "select /*+ para_tables_sort() */ count(asct1) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1 " % math_fun_join_2 + sql += "from stable_1 t1, stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.t_join_where) + sql += " and %s " % random.choice(self.qt_u_or_where) + sql += "%s " % random.choice(self.partiton_where_j) + sql += "%s " % random.choice(self.slimit1_where) + sql += ") " + sql += "%s ;" % random.choice(self.limit_u_where) + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + #taos -f sql + # startTime_taosf = time.time() + print("taos -f %s sql start!" %mathlist) + # taos_cmd1 = "taos -f %s/%s.sql" % (self.testcasePath,self.testcaseFilename) + # _ = subprocess.check_output(taos_cmd1, shell=True) + print("taos -f %s sql over!" %mathlist) + + print("=========%s====over=============" %mathlist) + + + def str_nest(self,strlist): + + print("==========%s===start=============" %strlist) + os.system("rm -rf %s/%s.sql" % (self.testcasePath,self.testcaseFilename)) + + #self.dropandcreateDB_random("%s" %self.db_nest, 1) + + if (strlist == ['LTRIM','RTRIM','LOWER','UPPER']) or (strlist == ['LENGTH','CHAR_LENGTH']) \ + or (strlist == ['']): + str_functions = strlist + fun_fix_column = ['(q_nchar)','(q_binary)','(q_nchar_null)','(q_binary_null)'] + fun_column_1 = random.sample(str_functions,1)+random.sample(fun_fix_column,1) + str_fun_1 = str(fun_column_1).replace("[","").replace("]","").replace("'","").replace(", ","") + fun_column_2 = random.sample(str_functions,1)+random.sample(fun_fix_column,1) + str_fun_2 = str(fun_column_2).replace("[","").replace("]","").replace("'","").replace(", ","") + + fun_fix_column_j = ['(t1.q_nchar)','(t1.q_binary)','(t1.q_nchar_null)','(t1.q_binary_null)', + '(t2.q_nchar)','(t2.q_binary)','(t2.q_nchar_null)','(t2.q_binary_null)'] + fun_column_join_1 = random.sample(str_functions,1)+random.sample(fun_fix_column_j,1) + str_fun_join_1 = str(fun_column_join_1).replace("[","").replace("]","").replace("'","").replace(", ","") + fun_column_join_2 = random.sample(str_functions,1)+random.sample(fun_fix_column_j,1) + str_fun_join_2 = str(fun_column_join_2).replace("[","").replace("]","").replace("'","").replace(", ","") + + fun_fix_column_s = ['(q_nchar)','(q_binary)','(q_nchar_null)','(q_binary_null)','(loc)','(tbname)'] + fun_column_s_1 = random.sample(str_functions,1)+random.sample(fun_fix_column_s,1) + str_fun_s_1 = str(fun_column_s_1).replace("[","").replace("]","").replace("'","").replace(", ","") + fun_column_s_2 = random.sample(str_functions,1)+random.sample(fun_fix_column_s,1) + str_fun_s_2 = str(fun_column_s_2).replace("[","").replace("]","").replace("'","").replace(", ","") + + fun_fix_column_s_j = ['(t1.q_nchar)','(t1.q_binary)','(t1.q_nchar_null)','(t1.q_binary_null)','(t1.loc)','(t1.tbname)', + '(t2.q_nchar)','(t2.q_binary)','(t2.q_nchar_null)','(t2.q_binary_null)','(t2.loc)','(t2.tbname)'] + fun_column_join_s_1 = random.sample(str_functions,1)+random.sample(fun_fix_column_j,1) + str_fun_join_s_1 = str(fun_column_join_1).replace("[","").replace("]","").replace("'","").replace(", ","") + fun_column_join_s_2 = random.sample(str_functions,1)+random.sample(fun_fix_column_j,1) + str_fun_join_s_2 = str(fun_column_join_2).replace("[","").replace("]","").replace("'","").replace(", ","") + + elif (strlist == ['SUBSTR']) : + str_functions = strlist + pos = random.randint(1, 20) + sub_len = random.randint(1, 10) + fun_fix_column = ['(q_nchar,pos)','(q_binary,pos)','(q_nchar_null,pos)','(q_binary_null,pos)', + '(q_nchar,pos,sub_len)','(q_binary,pos,sub_len)','(q_nchar_null,pos,sub_len)','(q_binary_null,pos,sub_len)',] + fun_column_1 = random.sample(str_functions,1)+random.sample(fun_fix_column,1) + str_fun_1 = str(fun_column_1).replace("[","").replace("]","").replace("'","").replace(", ","").replace("pos",str(pos)).replace("sub_len",str(sub_len)) + fun_column_2 = random.sample(str_functions,1)+random.sample(fun_fix_column,1) + str_fun_2 = str(fun_column_2).replace("[","").replace("]","").replace("'","").replace(", ","").replace("pos",str(pos)).replace("sub_len",str(sub_len)) + + fun_fix_column_j = ['(t1.q_nchar,pos)','(t1.q_binary,pos)','(t1.q_nchar_null,pos)','(t1.q_binary_null,pos)', + '(t1.q_nchar,pos,sub_len)','(t1.q_binary,pos,sub_len)','(t1.q_nchar_null,pos,sub_len)','(t1.q_binary_null,pos,sub_len)', + '(t2.q_nchar,pos)','(t2.q_binary,pos)','(t2.q_nchar_null,pos)','(t2.q_binary_null,pos)', + '(t2.q_nchar,pos,sub_len)','(t2.q_binary,pos,sub_len)','(t2.q_nchar_null,pos,sub_len)','(t2.q_binary_null,pos,sub_len)'] + fun_column_join_1 = random.sample(str_functions,1)+random.sample(fun_fix_column_j,1) + str_fun_join_1 = str(fun_column_join_1).replace("[","").replace("]","").replace("'","").replace(", ","").replace("pos",str(pos)).replace("sub_len",str(sub_len)) + fun_column_join_2 = random.sample(str_functions,1)+random.sample(fun_fix_column_j,1) + str_fun_join_2 = str(fun_column_join_2).replace("[","").replace("]","").replace("'","").replace(", ","").replace("pos",str(pos)).replace("sub_len",str(sub_len)) + + fun_fix_column_s = ['(q_nchar,pos)','(q_binary,pos)','(q_nchar_null,pos)','(q_binary_null,pos)','(loc,pos)', + '(q_nchar,pos,sub_len)','(q_binary,pos,sub_len)','(q_nchar_null,pos,sub_len)','(q_binary_null,pos,sub_len)','(loc,pos,sub_len)',] + fun_column_s_1 = random.sample(str_functions,1)+random.sample(fun_fix_column_s,1) + str_fun_s_1 = str(fun_column_1).replace("[","").replace("]","").replace("'","").replace(", ","").replace("pos",str(pos)).replace("sub_len",str(sub_len)) + fun_column_s_2 = random.sample(str_functions,1)+random.sample(fun_fix_column_s,1) + str_fun_s_2 = str(fun_column_2).replace("[","").replace("]","").replace("'","").replace(", ","").replace("pos",str(pos)).replace("sub_len",str(sub_len)) + + fun_fix_column_s_j = ['(t1.q_nchar,pos)','(t1.q_binary,pos)','(t1.q_nchar_null,pos)','(t1.q_binary_null,pos)','(t1.loc,pos)', + '(t1.q_nchar,pos,sub_len)','(t1.q_binary,pos,sub_len)','(t1.q_nchar_null,pos,sub_len)','(t1.q_binary_null,pos,sub_len)','(t1.loc,pos,sub_len)', + '(t2.q_nchar,pos)','(t2.q_binary,pos)','(t2.q_nchar_null,pos)','(t2.q_binary_null,pos)','(t2.loc,pos)', + '(t2.q_nchar,pos,sub_len)','(t2.q_binary,pos,sub_len)','(t2.q_nchar_null,pos,sub_len)','(t2.q_binary_null,pos,sub_len)','(t2.loc,pos,sub_len)'] + fun_column_join_s_1 = random.sample(str_functions,1)+random.sample(fun_fix_column_s_j,1) + str_fun_join_s_1 = str(fun_column_join_1).replace("[","").replace("]","").replace("'","").replace(", ","").replace("pos",str(pos)).replace("sub_len",str(sub_len)) + fun_column_join_s_2 = random.sample(str_functions,1)+random.sample(fun_fix_column_s_j,1) + str_fun_join_s_2 = str(fun_column_join_2).replace("[","").replace("]","").replace("'","").replace(", ","").replace("pos",str(pos)).replace("sub_len",str(sub_len)) + + elif (strlist == ['CONCAT']) : + str_functions = strlist + i = random.randint(2,4) + fun_fix_column = ['q_nchar','q_nchar1','q_nchar2','q_nchar3','q_nchar4','q_nchar5','q_nchar6','q_nchar7','q_nchar8','q_nchar_null', + 'q_binary','q_binary1','q_binary2','q_binary3','q_binary4','q_binary5','q_binary6','q_binary7','q_binary8','q_binary_null'] + + column1 = str(random.sample(fun_fix_column,i)).replace("[","").replace("]","").replace("'","") + fun_column_1 = str(random.sample(str_functions,1))+'('+column1+')' + str_fun_1 = str(fun_column_1).replace("[","").replace("]","").replace("'","") + + column2 = str(random.sample(fun_fix_column,i)).replace("[","").replace("]","").replace("'","") + fun_column_2 = str(random.sample(str_functions,1))+'('+column2+')' + str_fun_2 = str(fun_column_2).replace("[","").replace("]","").replace("'","") + + fun_fix_column_j = ['(t1.q_nchar)','(t1.q_nchar1)','(t1.q_nchar2)','(t1.q_nchar3)','(t1.q_nchar4)','(t1.q_nchar5)','(t1.q_nchar6)','(t1.q_nchar7)','(t1.q_nchar8)','(t1.q_nchar_null)', + '(t2.q_nchar)','(t2.q_nchar1)','(t2.q_nchar2)','(t2.q_nchar3)','(t2.q_nchar4)','(t2.q_nchar5)','(t2.q_nchar6)','(t2.q_nchar7)','(t2.q_nchar8)','(t2.q_nchar_null)', + '(t1.q_binary)','(t1.q_binary1)','(t1.q_binary2)','(t1.q_binary3)','(t1.q_binary4)','(t1.q_binary5)','(t1.q_binary6)','(t1.q_binary7)','(t1.q_binary8)','(t1.q_binary_null)', + '(t2.q_binary)','(t2.q_binary1)','(t2.q_binary2)','(t2.q_binary3)','(t2.q_binary4)','(t2.q_binary5)','(t2.q_binary6)','(t2.q_binary7)','(t2.q_binary8)','(t2.q_binary_null)'] + + column_j1 = str(random.sample(fun_fix_column_j,i)).replace("[","").replace("]","").replace("'","") + fun_column_join_1 = str(random.sample(str_functions,1))+'('+column_j1+')' + str_fun_join_1 = str(fun_column_join_1).replace("[","").replace("]","").replace("'","") + + column_j2 = str(random.sample(fun_fix_column_j,i)).replace("[","").replace("]","").replace("'","") + fun_column_join_2 = str(random.sample(str_functions,1))+'('+column_j2+')' + str_fun_join_2 = str(fun_column_join_2).replace("[","").replace("]","").replace("'","") + + fun_fix_column_s = ['q_nchar','q_nchar1','q_nchar2','q_nchar3','q_nchar4','q_nchar5','q_nchar6','q_nchar7','q_nchar8','loc','q_nchar_null', + 'q_binary','q_binary1','q_binary2','q_binary3','q_binary4','q_binary5','q_binary6','q_binary7','q_binary8','q_binary_null'] + + column_s1 = str(random.sample(fun_fix_column,i)).replace("[","").replace("]","").replace("'","") + fun_column_s_1 = str(random.sample(str_functions,1))+'('+column_s1+')' + str_fun_s_1 = str(fun_column_s_1).replace("[","").replace("]","").replace("'","") + + column_s2 = str(random.sample(fun_fix_column,i)).replace("[","").replace("]","").replace("'","") + fun_column_s_2 = str(random.sample(str_functions,1))+'('+column_s2+')' + str_fun_s_2 = str(fun_column_s_2).replace("[","").replace("]","").replace("'","") + + fun_fix_column_s_j = ['(t1.q_nchar)','(t1.q_nchar1)','(t1.q_nchar2)','(t1.q_nchar3)','(t1.q_nchar4)','(t1.q_nchar5)','(t1.q_nchar6)','(t1.q_nchar7)','(t1.q_nchar8)','(t1.q_nchar_null)','(t1.loc)', + '(t2.q_nchar)','(t2.q_nchar1)','(t2.q_nchar2)','(t2.q_nchar3)','(t2.q_nchar4)','(t2.q_nchar5)','(t2.q_nchar6)','(t2.q_nchar7)','(t2.q_nchar8)','(t2.q_nchar_null)','(t2.loc)', + '(t1.q_binary)','(t1.q_binary1)','(t1.q_binary2)','(t1.q_binary3)','(t1.q_binary4)','(t1.q_binary5)','(t1.q_binary6)','(t1.q_binary7)','(t1.q_binary8)','(t1.q_binary_null)', + '(t2.q_binary)','(t2.q_binary1)','(t2.q_binary2)','(t2.q_binary3)','(t2.q_binary4)','(t2.q_binary5)','(t2.q_binary6)','(t2.q_binary7)','(t2.q_binary8)','(t2.q_binary_null)'] + + column_j_s1 = str(random.sample(fun_fix_column_s_j,i)).replace("[","").replace("]","").replace("'","") + fun_column_join_s_1 = str(random.sample(str_functions,1))+'('+column_j_s1+')' + str_fun_join_s_1 = str(fun_column_join_s_1).replace("[","").replace("]","").replace("'","") + + column_j_s2 = str(random.sample(fun_fix_column_s_j,i)).replace("[","").replace("]","").replace("'","") + fun_column_join_s_2 = str(random.sample(str_functions,1))+'('+column_j_s2+')' + str_fun_join_s_2 = str(fun_column_join_s_2).replace("[","").replace("]","").replace("'","") + + elif (strlist == ['CONCAT_WS']): + str_functions = strlist + i = random.randint(2,4) + fun_fix_column = ['q_nchar','q_nchar1','q_nchar2','q_nchar3','q_nchar4','q_nchar5','q_nchar6','q_nchar7','q_nchar8','q_nchar_null', + 'q_binary','q_binary1','q_binary2','q_binary3','q_binary4','q_binary5','q_binary6','q_binary7','q_binary8','q_binary_null'] + + separators = ['',' ','abc','123','!','@','#','$','%','^','&','*','(',')','-','_','+','=','{', + '[','}',']','|',';',':',',','.','<','>','?','/','~','`','taos','涛思'] + separator = str(random.sample(separators,i)).replace("[","").replace("]","") + + column1 = str(random.sample(fun_fix_column,i)).replace("[","").replace("]","").replace("'","") + fun_column_1 = str(random.sample(str_functions,1))+'('+'\"'+separator+'\",'+column1+')' + str_fun_1 = str(fun_column_1).replace("[","").replace("]","").replace("'","") + + column2 = str(random.sample(fun_fix_column,i)).replace("[","").replace("]","").replace("'","") + fun_column_2 = str(random.sample(str_functions,1))+'('+'\"'+separator+'\",'+column2+')' + str_fun_2 = str(fun_column_2).replace("[","").replace("]","").replace("'","") + + fun_fix_column_j = ['(t1.q_nchar)','(t1.q_nchar1)','(t1.q_nchar2)','(t1.q_nchar3)','(t1.q_nchar4)','(t1.q_nchar5)','(t1.q_nchar6)','(t1.q_nchar7)','(t1.q_nchar8)','(t1.q_nchar_null)', + '(t2.q_nchar)','(t2.q_nchar1)','(t2.q_nchar2)','(t2.q_nchar3)','(t2.q_nchar4)','(t2.q_nchar5)','(t2.q_nchar6)','(t2.q_nchar7)','(t2.q_nchar8)','(t2.q_nchar_null)', + '(t1.q_binary)','(t1.q_binary1)','(t1.q_binary2)','(t1.q_binary3)','(t1.q_binary4)','(t1.q_binary5)','(t1.q_binary6)','(t1.q_binary7)','(t1.q_binary8)','(t1.q_binary_null)', + '(t2.q_binary)','(t2.q_binary1)','(t2.q_binary2)','(t2.q_binary3)','(t2.q_binary4)','(t2.q_binary5)','(t2.q_binary6)','(t2.q_binary7)','(t2.q_binary8)','(t2.q_binary_null)'] + + column_j1 = str(random.sample(fun_fix_column_j,i)).replace("[","").replace("]","").replace("'","") + fun_column_join_1 = str(random.sample(str_functions,1))+'('+'\"'+separator+'\",'+column_j1+')' + str_fun_join_1 = str(fun_column_join_1).replace("[","").replace("]","").replace("'","") + + column_j2 = str(random.sample(fun_fix_column_j,i)).replace("[","").replace("]","").replace("'","") + fun_column_join_2 = str(random.sample(str_functions,1))+'('+'\"'+separator+'\",'+column_j2+')' + str_fun_join_2 = str(fun_column_join_2).replace("[","").replace("]","").replace("'","") + + fun_fix_column_s = ['q_nchar','q_nchar1','q_nchar2','q_nchar3','q_nchar4','q_nchar5','q_nchar6','q_nchar7','q_nchar8','loc','q_nchar_null', + 'q_binary','q_binary1','q_binary2','q_binary3','q_binary4','q_binary5','q_binary6','q_binary7','q_binary8','q_binary_null'] + + column_s1 = str(random.sample(fun_fix_column,i)).replace("[","").replace("]","").replace("'","") + fun_column_s_1 = str(random.sample(str_functions,1))+'('+'\"'+separator+'\",'+column_s1+')' + str_fun_s_1 = str(fun_column_s_1).replace("[","").replace("]","").replace("'","") + + column_s2 = str(random.sample(fun_fix_column,i)).replace("[","").replace("]","").replace("'","") + fun_column_s_2 = str(random.sample(str_functions,1))+'('+'\"'+separator+'\",'+column_s2+')' + str_fun_s_2 = str(fun_column_s_2).replace("[","").replace("]","").replace("'","") + + fun_fix_column_s_j = ['(t1.q_nchar)','(t1.q_nchar1)','(t1.q_nchar2)','(t1.q_nchar3)','(t1.q_nchar4)','(t1.q_nchar5)','(t1.q_nchar6)','(t1.q_nchar7)','(t1.q_nchar8)','(t1.q_nchar_null)','(t1.loc)', + '(t2.q_nchar)','(t2.q_nchar1)','(t2.q_nchar2)','(t2.q_nchar3)','(t2.q_nchar4)','(t2.q_nchar5)','(t2.q_nchar6)','(t2.q_nchar7)','(t2.q_nchar8)','(t2.q_nchar_null)','(t2.loc)', + '(t1.q_binary)','(t1.q_binary1)','(t1.q_binary2)','(t1.q_binary3)','(t1.q_binary4)','(t1.q_binary5)','(t1.q_binary6)','(t1.q_binary7)','(t1.q_binary8)','(t1.q_binary_null)', + '(t2.q_binary)','(t2.q_binary1)','(t2.q_binary2)','(t2.q_binary3)','(t2.q_binary4)','(t2.q_binary5)','(t2.q_binary6)','(t2.q_binary7)','(t2.q_binary8)','(t2.q_binary_null)'] + + column_j_s1 = str(random.sample(fun_fix_column_s_j,i)).replace("[","").replace("]","").replace("'","") + fun_column_join_s_1 = str(random.sample(str_functions,1))+'('+'\"'+separator+'\",'+column_j_s1+')' + str_fun_join_s_1 = str(fun_column_join_s_1).replace("[","").replace("]","").replace("'","") + + column_j_s2 = str(random.sample(fun_fix_column_s_j,i)).replace("[","").replace("]","").replace("'","") + fun_column_join_s_2 = str(random.sample(str_functions,1))+'('+'\"'+separator+'\",'+column_j_s2+')' + str_fun_join_s_2 = str(fun_column_join_s_2).replace("[","").replace("]","").replace("'","") + + + tdSql.query("select /*+ para_tables_sort() */1-1 as str_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (strlist == ['LTRIM','RTRIM','LOWER','UPPER']) or (strlist == ['SUBSTR']) or (strlist == ['CONCAT']) or (strlist == ['CONCAT_WS']) : + sql = "select /*+ para_tables_sort() */ t1s , LTRIM(asct1), LOWER(asct1), RTRIM(asct2), UPPER(asct2) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1, " % str_fun_1 + sql += "%s as asct2, " % str_fun_2 + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts as t1s from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += ");" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + elif (strlist == ['LENGTH','CHAR_LENGTH']): + sql = "select /*+ para_tables_sort() */ sum(asct1), min(asct1), max(asct2), avg(asct2) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1, " % str_fun_1 + sql += "%s as asct2, " % str_fun_2 + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += ");" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-2 as str_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (strlist == ['LTRIM','RTRIM','LOWER','UPPER']) or (strlist == ['SUBSTR']) or (strlist == ['CONCAT']) or (strlist == ['CONCAT_WS']) : + sql = "select /*+ para_tables_sort() */ ts , asct1 from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1, " % str_fun_1 + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts ts from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s )" % random.choice(self.order_where) + sql += "%s " % random.choice(self.unionall_or_union) + sql += "select /*+ para_tables_sort() */ ts , asct2 from ( select /*+ para_tables_sort() */ " + sql += "%s as asct2, " % str_fun_2 + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts ts from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ")" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + elif (strlist == ['LENGTH','CHAR_LENGTH']): + sql = "select /*+ para_tables_sort() */ sum(asct1), min(asct1) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1, " % str_fun_1 + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts ts from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s )" % random.choice(self.order_where) + sql += "%s " % random.choice(self.unionall_or_union) + sql += "select /*+ para_tables_sort() */ sum(asct2), min(asct2) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct2, " % str_fun_2 + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts ts from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ")" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-3 as str_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (strlist == ['LTRIM','RTRIM','LOWER','UPPER']) or (strlist == ['SUBSTR']) or (strlist == ['CONCAT']) or (strlist == ['CONCAT_WS']): + sql = "select /*+ para_tables_sort() */ ts , LTRIM(asct1), LOWER(asct1), RTRIM(asct2), UPPER(asct2) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1 ," % str_fun_1 + sql += "%s as asct2, " % str_fun_2 + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s select /*+ para_tables_sort() */" % random.choice(self.unionall_or_union) + sql += "%s as asct2 ," % str_fun_2 + sql += "%s as asct1, " % str_fun_1 + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts from regular_table_2 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += ")" + tdLog.info(sql) + tdSql.query(sql) + elif (strlist == ['LENGTH','CHAR_LENGTH']): + sql = "select /*+ para_tables_sort() */ sum(asct1), min(asct1), max(asct2), avg(asct2) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1 ," % str_fun_1 + sql += "%s as asct2, " % str_fun_2 + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s select /*+ para_tables_sort() */" % random.choice(self.unionall_or_union) + sql += "%s as asct2 ," % str_fun_2 + sql += "%s as asct1, " % str_fun_1 + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts from regular_table_2 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += ")" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-4 as str_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (strlist == ['LTRIM','RTRIM','LOWER','UPPER']) or (strlist == ['SUBSTR']) or (strlist == ['CONCAT']) or (strlist == ['CONCAT_WS']): + sql = "select /*+ para_tables_sort() */ ts1,ts2 ,timediff(ts1,ts2), LTRIM(asct1), LOWER(asct1), RTRIM(asct2), UPPER(asct2) from ( select /*+ para_tables_sort() */ t1.ts as ts1," + sql += "%s as asct2, " % str_fun_join_1 + sql += "%s as asct1, " % str_fun_join_2 + sql += "%s, " % str_fun_join_1 + sql += "t1.%s as asct11, " % random.choice(self.q_select) + sql += "t2.%s as asct12, " % random.choice(self.q_select) + sql += "t2.ts as ts2 from regular_table_1 t1 , regular_table_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_where) + sql += "and %s " % random.choice(self.q_u_or_where) + sql += "%s " % random.choice(self.order_u_where) + sql += ");" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + elif (strlist == ['LENGTH','CHAR_LENGTH']): + sql = "select /*+ para_tables_sort() */ sum(asct1), min(asct1), max(asct2), avg(asct2) from ( select /*+ para_tables_sort() */ t1.ts as ts1," + sql += "%s as asct2, " % str_fun_join_1 + sql += "%s as asct1, " % str_fun_join_2 + sql += "%s, " % str_fun_join_1 + sql += "t1.%s as asct21, " % random.choice(self.q_select) + sql += "t2.%s as asct22, " % random.choice(self.q_select) + sql += "t2.ts as ts2 from regular_table_1 t1 , regular_table_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_where) + sql += "and %s " % random.choice(self.q_u_or_where) + sql += "%s " % random.choice(self.order_u_where) + sql += ");" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-5 as str_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (strlist == ['LTRIM','RTRIM','LOWER','UPPER']) or (strlist == ['SUBSTR']) or (strlist == ['CONCAT']) or (strlist == ['CONCAT_WS']): + sql = "select /*+ para_tables_sort() */ ts ," + sql += "%s, " % str_fun_1 + sql += "%s as asct21, " % random.choice(self.q_select) + sql += "%s as asct22, " % random.choice(self.q_select) + sql += "%s " % str_fun_2 + sql += " from ( select /*+ para_tables_sort() */ * from regular_table_1 ) where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += " ;" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + elif (strlist == ['LENGTH','CHAR_LENGTH']): + sql = "select /*+ para_tables_sort() */ ts ," + sql += "%s, " % str_fun_1 + sql += "%s as asct22, " % random.choice(self.q_select) + sql += "%s as asct21, " % random.choice(self.q_select) + sql += "%s " % str_fun_2 + sql += " from ( select /*+ para_tables_sort() */ * from regular_table_1 ) where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += " ;" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-6 as str_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (strlist == ['LTRIM','RTRIM','LOWER','UPPER']) or (strlist == ['SUBSTR']) or (strlist == ['CONCAT']) or (strlist == ['CONCAT_WS']): + sql = "select /*+ para_tables_sort() */ ts1,ts ,timediff(ts1,ts), LTRIM(asct1), LOWER(asct1), RTRIM(asct2), UPPER(asct2) from ( select /*+ para_tables_sort() */ t1.ts as ts1," + sql += "%s as asct2, " % str_fun_join_1 + sql += "%s as asct1, " % str_fun_join_2 + sql += "t1.%s as asct22, " % random.choice(self.q_select) + sql += "t2.%s as asct21, " % random.choice(self.q_select) + sql += "%s, " % str_fun_join_1 + sql += "t2.ts from regular_table_1 t1 , regular_table_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_where) + sql += "and %s )" % random.choice(self.q_u_or_where) + sql += "%s " % random.choice(self.order_where) + sql += "%s " % random.choice(self.limit1_where) + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + elif (strlist == ['LENGTH','CHAR_LENGTH']): + sql = "select /*+ para_tables_sort() */ sum(asct1), min(asct1), max(asct2), avg(asct2) from ( select /*+ para_tables_sort() */ t1.ts as ts1," + sql += "%s as asct2, " % str_fun_join_1 + sql += "%s as asct1, " % str_fun_join_2 + sql += "t1.%s as asct22, " % random.choice(self.q_select) + sql += "t2.%s as asct21, " % random.choice(self.q_select) + sql += "%s, " % str_fun_join_1 + sql += "t2.ts as ts2 from regular_table_1 t1 , regular_table_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_where) + sql += "and %s )" % random.choice(self.q_u_or_where) + sql += "%s " % random.choice(self.limit1_where) + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-7 as str_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (strlist == ['LTRIM','RTRIM','LOWER','UPPER']) or (strlist == ['SUBSTR']) or (strlist == ['CONCAT']) or (strlist == ['CONCAT_WS']): + sql = "select /*+ para_tables_sort() */ t1s ,ts1, LTRIM(asct1), LOWER(asct1), RTRIM(asct2), UPPER(asct2) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1, ts as t1s," % str_fun_s_1 + sql += "%s as asct2, " % str_fun_s_2 + sql += "%s, " % random.choice(self.q_select) + sql += "%s, " % random.choice(self.t_select) + sql += "ts as ts1 from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += "%s " % random.choice(self.order_where) + sql += ") ;" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + elif (strlist == ['LENGTH','CHAR_LENGTH']): + sql = "select /*+ para_tables_sort() */ sum(asct1), min(asct1), max(asct2), avg(asct2) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1, ts as ts1," % str_fun_s_1 + sql += "%s as asct2, " % str_fun_s_2 + sql += "%s, " % random.choice(self.q_select) + sql += "%s, " % random.choice(self.t_select) + sql += "ts as t1s from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += "%s " % random.choice(self.order_where) + sql += ") ;" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-8 as str_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (strlist == ['LTRIM','RTRIM','LOWER','UPPER']) or (strlist == ['SUBSTR']) or (strlist == ['CONCAT']) or (strlist == ['CONCAT_WS']): + sql = "select /*+ para_tables_sort() */ ts1,st1, LTRIM(asct1), LOWER(asct1), RTRIM(asct2), UPPER(asct2) " + sql += "from ( select /*+ para_tables_sort() */ " + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s as asct1, ts as st1," % str_fun_s_1 + sql += "%s as asct2, " % str_fun_s_2 + sql += "%s, " % random.choice(self.q_select) + sql += "%s, " % random.choice(self.t_select) + sql += "ts as ts1 from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += "%s " % random.choice(self.order_where) + sql += ") ;" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + elif (strlist == ['LENGTH','CHAR_LENGTH']): + sql = "select /*+ para_tables_sort() */ sum(asct1), min(asct1), max(asct2), avg(asct2) " + sql += "from ( select /*+ para_tables_sort() */ " + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s as asct1, ts as ts1," % str_fun_s_1 + sql += "%s as asct2, " % str_fun_s_2 + sql += "%s, " % random.choice(self.q_select) + sql += "%s, " % random.choice(self.t_select) + sql += "ts as st1 from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += "%s " % random.choice(self.order_where) + sql += ") ;" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-9 as str_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (strlist == ['LTRIM','RTRIM','LOWER','UPPER']) or (strlist == ['SUBSTR']) or (strlist == ['CONCAT']) or (strlist == ['CONCAT_WS']): + sql = "select /*+ para_tables_sort() */ ts1,ts2 ,timediff(ts1,ts2), LTRIM(asct1), LOWER(asct1), RTRIM(asct2), UPPER(asct2) from ( select /*+ para_tables_sort() */ t1.ts as ts1," + sql += "%s as asct2, " % str_fun_join_s_1 + sql += "%s as asct1, " % str_fun_join_s_2 + sql += "t1.%s as asct21, " % random.choice(self.q_select) + sql += "t1.%s as asct22, " % random.choice(self.q_select) + sql += "t2.%s as asct23, " % random.choice(self.q_select) + sql += "t2.%s as asct24, " % random.choice(self.q_select) + sql += "t2.ts as ts2 from stable_1 t1 , stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.t_join_where) + sql += "and %s " % random.choice(self.t_u_where) + sql += "and %s " % random.choice(self.t_u_or_where) + sql += "%s " % random.choice(self.order_u_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ") ;" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + elif (strlist == ['LENGTH','CHAR_LENGTH']): + sql = "select /*+ para_tables_sort() */ sum(asct1), min(asct1), max(asct2), avg(asct2) from ( select /*+ para_tables_sort() */ t1.ts as ts1," + sql += "%s as asct2, " % str_fun_join_s_1 + sql += "%s as asct1, " % str_fun_join_s_2 + sql += "t1.%s as asct21, " % random.choice(self.q_select) + sql += "t1.%s as asct22, " % random.choice(self.q_select) + sql += "t2.%s as asct23, " % random.choice(self.q_select) + sql += "t2.%s as asct24, " % random.choice(self.q_select) + sql += "t2.ts as ts2 from stable_1 t1 , stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.t_join_where) + sql += "and %s " % random.choice(self.t_u_where) + sql += "and %s " % random.choice(self.t_u_or_where) + sql += "%s " % random.choice(self.order_u_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ") ;" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + + tdSql.query("select /*+ para_tables_sort() */1-10 as str_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (strlist == ['LTRIM','RTRIM','LOWER','UPPER']) or (strlist == ['SUBSTR']) or (strlist == ['CONCAT']) or (strlist == ['CONCAT_WS']): + sql = "select /*+ para_tables_sort() */ ts , LTRIM(asct1), LOWER(asct1), RTRIM(asct2), UPPER(asct2) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1 ," % str_fun_s_1 + sql += "%s as asct2, " % str_fun_s_2 + sql += "%s, " % random.choice(self.s_r_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += ") %s " % random.choice(self.unionall_or_union) + sql += "select /*+ para_tables_sort() */ ts , LTRIM(asct1), LOWER(asct1), RTRIM(asct2), UPPER(asct2) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1 ," % str_fun_1 + sql += "%s as asct2, " % str_fun_2 + sql += "%s, " % random.choice(self.s_r_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts from stable_2 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ")" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + elif (strlist == ['LENGTH','CHAR_LENGTH']): + sql = "select /*+ para_tables_sort() */ sum(asct1), min(asct1), max(asct2), avg(asct2) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1 ," % str_fun_s_1 + sql += "%s as asct2, " % str_fun_s_2 + sql += "%s, " % random.choice(self.s_r_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += ") %s " % random.choice(self.unionall_or_union) + sql += "select /*+ para_tables_sort() */ sum(asct1), min(asct1), max(asct2), avg(asct2) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1 ," % str_fun_1 + sql += "%s as asct2, " % str_fun_2 + sql += "%s, " % random.choice(self.s_r_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts from stable_2 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ")" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + #3 inter union not support + tdSql.query("select /*+ para_tables_sort() */1-11 as str_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (strlist == ['LTRIM','RTRIM','LOWER','UPPER']) or (strlist == ['SUBSTR']) or (strlist == ['CONCAT']) or (strlist == ['CONCAT_WS']): + sql = "select /*+ para_tables_sort() */ ts , LTRIM(asct1), LOWER(asct1), RTRIM(asct2), UPPER(asct2) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1, ts ," % str_fun_s_1 + sql += "%s as asct2, " % str_fun_s_2 + sql += "%s, " % random.choice(self.s_r_select) + sql += "%s " % random.choice(self.q_select) + sql += " from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += " %s " % random.choice(self.unionall_or_union) + sql += " select /*+ para_tables_sort() */ " + sql += "%s as asct1, ts ," % str_fun_1 + sql += "%s as asct2, " % str_fun_2 + sql += "%s, " % random.choice(self.s_r_select) + sql += "%s " % random.choice(self.q_select) + sql += " from stable_2 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ")" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + elif (strlist == ['LENGTH','CHAR_LENGTH']): + sql = "select /*+ para_tables_sort() */sum(asct1), min(asct1), max(asct2), avg(asct2) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1 ," % str_fun_s_1 + sql += "%s as asct2, " % str_fun_s_2 + sql += "%s, " % random.choice(self.s_r_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += " %s " % random.choice(self.unionall_or_union) + sql += " select /*+ para_tables_sort() */ " + sql += "%s as asct1 ," % str_fun_1 + sql += "%s as asct2, " % str_fun_2 + sql += "%s, " % random.choice(self.s_r_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts from stable_2 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ")" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-12 as str_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (strlist == ['LTRIM','RTRIM','LOWER','UPPER']) or (strlist == ['SUBSTR']) or (strlist == ['CONCAT']) or (strlist == ['CONCAT_WS']): + sql = "select /*+ para_tables_sort() */ ts1,ts2 ,timediff(ts1,ts2), LTRIM(asct1), LOWER(asct1), RTRIM(asct2), UPPER(asct2) from ( select /*+ para_tables_sort() */ t1.ts as ts1," + sql += "%s as asct2, " % str_fun_join_s_1 + sql += "%s as asct1, " % str_fun_join_s_2 + sql += "t1.%s as asct21, " % random.choice(self.q_select) + sql += "t1.%s as asct22, " % random.choice(self.q_select) + sql += "t2.%s as asct23, " % random.choice(self.q_select) + sql += "t2.%s as asct24, " % random.choice(self.q_select) + sql += "t2.ts as ts2 from stable_1 t1 , stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.t_u_where) + sql += "and %s " % random.choice(self.t_u_or_where) + sql += "%s " % random.choice(self.order_u_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ");" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + elif (strlist == ['LENGTH','CHAR_LENGTH']): + sql = "select /*+ para_tables_sort() */ sum(asct1), min(asct1), max(asct2), avg(asct2) from ( select /*+ para_tables_sort() */ t1.ts as ts1," + sql += "%s as asct2, " % str_fun_join_s_1 + sql += "%s as asct1, " % str_fun_join_s_2 + sql += "t1.%s as asct21, " % random.choice(self.q_select) + sql += "t1.%s as asct22, " % random.choice(self.q_select) + sql += "t2.%s as asct23, " % random.choice(self.q_select) + sql += "t2.%s as asct24, " % random.choice(self.q_select) + sql += "t2.ts as ts2 from stable_1 t1 , stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.t_u_where) + sql += "and %s " % random.choice(self.t_u_or_where) + sql += "%s " % random.choice(self.order_u_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ");" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-13 as str_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (strlist == ['LTRIM','RTRIM','LOWER','UPPER']) or (strlist == ['SUBSTR']) or (strlist == ['CONCAT']) or (strlist == ['CONCAT_WS']): + sql = "select /*+ para_tables_sort() */ ts ," + sql += "%s as asct10, " % str_fun_1 + sql += "%s as asct1, " % random.choice(self.q_select) + sql += "%s as asct12, " % random.choice(self.q_select) + sql += "%s as asct13, " % str_fun_2 + sql += "%s as asct14 " % random.choice(self.t_select) + sql += " from ( select /*+ para_tables_sort() */ * from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += "%s " % random.choice(self.order_where) + sql += ") ;" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + elif (strlist == ['LENGTH','CHAR_LENGTH']): + sql = "select /*+ para_tables_sort() */ ts ," + sql += "%s as asct1, " % str_fun_1 + sql += "%s as asct11, " % random.choice(self.q_select) + sql += "%s as asct12, " % random.choice(self.q_select) + sql += "%s as asct13, " % str_fun_2 + sql += "%s as asct14 " % random.choice(self.t_select) + sql += " from ( select /*+ para_tables_sort() */ * from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += "%s " % random.choice(self.order_where) + sql += ") ;" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-14 as str_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (strlist == ['LTRIM','RTRIM','LOWER','UPPER']) or (strlist == ['SUBSTR']) or (strlist == ['CONCAT']) or (strlist == ['CONCAT_WS']): + sql = "select /*+ para_tables_sort() */ LTRIM(asct1), LOWER(asct1), RTRIM(asct2), UPPER(asct2) from ( select /*+ para_tables_sort() */" + sql += "%s as asct1, " % str_fun_s_1 + sql += "%s as asct2" % str_fun_s_2 + sql += " from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.partiton_where) + sql += "%s " % random.choice(self.order_desc_where) + sql += "%s " % random.choice([self.limit_where[2] , self.limit_where[3]] ) + sql += " ) ;" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + elif (strlist == ['LENGTH','CHAR_LENGTH']): + sql = "select /*+ para_tables_sort() */ sum(asct1), min(asct1), max(asct2), avg(asct2) from ( select /*+ para_tables_sort() */" + sql += "%s as asct1, " % str_fun_s_1 + sql += "%s as asct2" % str_fun_s_2 + sql += " from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.partiton_where) + sql += "%s " % random.choice(self.order_desc_where) + sql += "%s " % random.choice([self.limit_where[2] , self.limit_where[3]] ) + sql += " ) ;" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-15 as str_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (strlist == ['LTRIM','RTRIM','LOWER','UPPER']) or (strlist == ['SUBSTR']) or (strlist == ['CONCAT']) or (strlist == ['CONCAT_WS']): + sql = "select /*+ para_tables_sort() */ ts,ts2 ,timediff(ts,ts2), LTRIM(asct1), LOWER(asct1), RTRIM(asct2), UPPER(asct2) from ( select /*+ para_tables_sort() */ t1.ts ," + sql += "%s as asct2, " % str_fun_join_s_1 + sql += "%s as asct1, " % str_fun_join_s_2 + sql += "t1.%s as asct11, " % random.choice(self.q_select) + sql += "t1.%s as asct12, " % random.choice(self.q_select) + sql += "t2.%s as asct13, " % random.choice(self.q_select) + sql += "t2.%s as asct14, " % random.choice(self.q_select) + sql += "t2.ts as ts2 from stable_1 t1, stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.t_join_where) + sql += " and %s " % random.choice(self.qt_u_or_where) + sql += "%s " % random.choice(self.partiton_where_j) + sql += "%s " % random.choice(self.slimit1_where) + sql += ") " + sql += "%s " % random.choice(self.order_desc_where) + sql += "%s ;" % random.choice(self.limit_u_where) + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + elif (strlist == ['LENGTH','CHAR_LENGTH']): + sql = "select /*+ para_tables_sort() */ sum(asct1), min(asct1), max(asct2), avg(asct2) from ( select /*+ para_tables_sort() */ t1.ts as ts1," + sql += "%s as asct2, " % str_fun_join_s_1 + sql += "%s as asct1, " % str_fun_join_s_2 + sql += "t1.%s as asct11, " % random.choice(self.q_select) + sql += "t1.%s as asct12, " % random.choice(self.q_select) + sql += "t2.%s as asct13, " % random.choice(self.q_select) + sql += "t2.%s as asct14 " % random.choice(self.q_select) + sql += "from stable_1 t1, stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.t_join_where) + sql += " and %s " % random.choice(self.qt_u_or_where) + sql += "%s " % random.choice(self.partiton_where_j) + sql += "%s " % random.choice(self.slimit1_where) + sql += ") " + sql += "%s ;" % random.choice(self.limit_u_where) + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + #taos -f sql + startTime_taos_f = time.time() + print("taos -f %s sql start!" %strlist) + # taos_cmd1 = "taos -f %s/%s.sql" % (self.testcasePath,self.testcaseFilename) + # _ = subprocess.check_output(taos_cmd1, shell=True) + print("taos -f %s sql over!" %strlist) + endTime_taos_f = time.time() + print("taos_f total time %ds" % (endTime_taos_f - startTime_taos_f)) + + print("=========%s====over=============" %strlist) + + def time_nest(self,timelist): + + print("==========%s===start=============" %timelist) + os.system("rm -rf %s/%s.sql" % (self.testcasePath,self.testcaseFilename)) + + #self.dropandcreateDB_random("%s" %self.db_nest, 1) + + if (timelist == ['NOW','TODAY']) or (timelist == ['TIMEZONE']): + time_functions = timelist + fun_fix_column = ['()'] + fun_column_1 = random.sample(time_functions,1)+random.sample(fun_fix_column,1) + time_fun_1 = str(fun_column_1).replace("[","").replace("]","").replace("'","").replace(", ","") + fun_column_2 = random.sample(time_functions,1)+random.sample(fun_fix_column,1) + time_fun_2 = str(fun_column_2).replace("[","").replace("]","").replace("'","").replace(", ","") + + fun_fix_column_j = ['()'] + fun_column_join_1 = random.sample(time_functions,1)+random.sample(fun_fix_column_j,1) + time_fun_join_1 = str(fun_column_join_1).replace("[","").replace("]","").replace("'","").replace(", ","") + fun_column_join_2 = random.sample(time_functions,1)+random.sample(fun_fix_column_j,1) + time_fun_join_2 = str(fun_column_join_2).replace("[","").replace("]","").replace("'","").replace(", ","") + + elif (timelist == ['TIMETRUNCATE']): + time_functions = timelist + + t = time.time() + t_to_s = time.strftime('%Y-%m-%d %H:%M:%S', time.localtime(t)) + fun_fix_column = ['q_ts','ts','_c0','_C0','_rowts','1600000000000','1600000000000000','1600000000000000000', + '%d' %t, '%d000' %t, '%d000000' %t,'t_to_s'] + + timeunits = ['1a' ,'1s', '1m' ,'1h', '1d'] + timeunit = str(random.sample(timeunits,1)).replace("[","").replace("]","").replace("'","") + + column_1 = ['(%s,timeutil)'%(random.sample(fun_fix_column,1))] + fun_column_1 = random.sample(time_functions,1)+random.sample(column_1,1) + time_fun_1 = str(fun_column_1).replace("[","").replace("]","").replace("'","").replace(", ","").replace("\"","").replace("t_to_s","'t_to_s'") + time_fun_1 = str(time_fun_1).replace("timeutil","%s" %timeunit).replace("t_to_s","%s" %t_to_s) + + column_2 = ['(%s,timeutil)'%(random.sample(fun_fix_column,1))] + fun_column_2 = random.sample(time_functions,1)+random.sample(column_2,1) + time_fun_2 = str(fun_column_2).replace("[","").replace("]","").replace("'","").replace(", ","").replace("\"","").replace("t_to_s","'t_to_s'") + time_fun_2 = str(time_fun_2).replace("timeutil","%s" %timeunit).replace("t_to_s","%s" %t_to_s) + + + fun_fix_column_j = ['(t1.q_ts)','(t1.ts)', '(t2.q_ts)','(t2.ts)','(1600000000000)','(1600000000000000)','(1600000000000000000)', + '(%d)' %t, '(%d000)' %t, '(%d000000)' %t,'t_to_s'] + + column_j1 = ['(%s,timeutil)'%(random.sample(fun_fix_column_j,1))] + fun_column_join_1 = random.sample(time_functions,1)+random.sample(column_j1,1) + time_fun_join_1 = str(fun_column_join_1).replace("[","").replace("]","").replace("'","").replace(", ","").replace("\"","").replace("t_to_s","'t_to_s'") + time_fun_join_1 = str(time_fun_join_1).replace("timeutil","%s" %timeunit).replace("t_to_s","%s" %t_to_s) + + column_j2 = ['(%s,timeutil)'%(random.sample(fun_fix_column_j,1))] + fun_column_join_2 = random.sample(time_functions,1)+random.sample(column_j2,1) + time_fun_join_2 = str(fun_column_join_2).replace("[","").replace("]","").replace("'","").replace(", ","").replace("\"","").replace("t_to_s","'t_to_s'") + time_fun_join_2 = str(time_fun_join_2).replace("timeutil","%s" %timeunit).replace("t_to_s","%s" %t_to_s) + + elif (timelist == ['TO_ISO8601']): + time_functions = timelist + + t = time.time() + fun_fix_column = ['(now())','(ts)','(q_ts)','(_rowts)','(_c0)','(_C0)', + '(1600000000000)','(1600000000000000)','(1600000000000000000)', + '(%d)' %t, '(%d000)' %t, '(%d000000)' %t] + + fun_column_1 = random.sample(time_functions,1)+random.sample(fun_fix_column,1) + time_fun_1 = str(fun_column_1).replace("[","").replace("]","").replace("'","").replace(", ","") + + fun_column_2 = random.sample(time_functions,1)+random.sample(fun_fix_column,1) + time_fun_2 = str(fun_column_2).replace("[","").replace("]","").replace("'","").replace(", ","") + + fun_fix_column_j = ['(t1.q_ts)','(t1.ts)', '(t2.q_ts)','(t2.ts)','(1600000000000)','(1600000000000000)','(1600000000000000000)','(now())', + '(%d)' %t, '(%d000)' %t, '(%d000000)' %t] + + fun_column_join_1 = random.sample(time_functions,1)+random.sample(fun_fix_column_j,1) + time_fun_join_1 = str(fun_column_join_1).replace("[","").replace("]","").replace("'","").replace(", ","") + + fun_column_join_2 = random.sample(time_functions,1)+random.sample(fun_fix_column_j,1) + time_fun_join_2 = str(fun_column_join_2).replace("[","").replace("]","").replace("'","").replace(", ","") + + elif (timelist == ['TO_UNIXTIMESTAMP']): + time_functions = timelist + + t = time.time() + t_to_s = time.strftime('%Y-%m-%d %H:%M:%S', time.localtime(t)) + fun_fix_column = ['(q_nchar)','(q_nchar1)','(q_nchar2)','(q_nchar3)','(q_nchar4)','(q_nchar_null)','(q_binary)','(q_binary5)','(q_binary6)','(q_binary7)','(q_binary8)','(q_binary_null)','(t_to_s)'] + + fun_column_1 = random.sample(time_functions,1)+random.sample(fun_fix_column,1) + time_fun_1 = str(fun_column_1).replace("[","").replace("]","").replace("'","").replace(", ","").replace("t_to_s","'t_to_s'") + time_fun_1 = str(time_fun_1).replace("t_to_s","%s" %t_to_s) + + fun_column_2 = random.sample(time_functions,1)+random.sample(fun_fix_column,1) + time_fun_2 = str(fun_column_2).replace("[","").replace("]","").replace("'","").replace(", ","").replace("t_to_s","'t_to_s'") + time_fun_2 = str(time_fun_2).replace("t_to_s","%s" %t_to_s) + + fun_fix_column_j = ['(t1.q_nchar)','(t1.q_binary)', '(t2.q_nchar)','(t2.q_binary)','(t_to_s)'] + + fun_column_join_1 = random.sample(time_functions,1)+random.sample(fun_fix_column_j,1) + time_fun_join_1 = str(fun_column_join_1).replace("[","").replace("]","").replace("'","").replace(", ","").replace("t_to_s","'t_to_s'") + time_fun_join_1 = str(time_fun_join_1).replace("t_to_s","%s" %t_to_s) + + fun_column_join_2 = random.sample(time_functions,1)+random.sample(fun_fix_column_j,1) + time_fun_join_2 = str(fun_column_join_2).replace("[","").replace("]","").replace("'","").replace(", ","").replace("t_to_s","'t_to_s'") + time_fun_join_2 = str(time_fun_join_2).replace("t_to_s","%s" %t_to_s) + + elif (timelist == ['TIMEDIFF_1']): + time_functions = timelist + + t = time.time() + t_to_s = time.strftime('%Y-%m-%d %H:%M:%S', time.localtime(t)) + timeunits = [ '1a' ,'1s', '1m' ,'1h', '1d'] + timeunit = str(random.sample(timeunits,1)).replace("[","").replace("]","").replace("'","") + + fun_fix_column = ['q_ts','ts','_c0','_C0','_rowts','1600000000000','1600000000000000','1600000000000000000', + '%d' %t, '%d000' %t, '%d000000' %t,'t_to_s'] + + column_1,column_2 = random.sample(fun_fix_column,1),random.sample(fun_fix_column,1) + column_12 = ['(%s,%s,timeutil)'%(column_1,column_2)] + fun_column_1 = random.sample(time_functions,1)+random.sample(column_12,1) + time_fun_1 = str(fun_column_1).replace("[","").replace("]","").replace("'","").replace(", ","").replace("\"","").replace("t_to_s","'t_to_s'") + time_fun_1 = str(time_fun_1).replace("timeutil","%s" %timeunit).replace("t_to_s","%s" %t_to_s).replace("_1","") + + column_3,column_4 = random.sample(fun_fix_column,1),random.sample(fun_fix_column,1) + column_34 = ['(%s,%s,timeutil)'%(column_3,column_4)] + fun_column_2 = random.sample(time_functions,1)+random.sample(column_34,1) + time_fun_2 = str(fun_column_2).replace("[","").replace("]","").replace("'","").replace(", ","").replace("\"","").replace("t_to_s","'t_to_s'") + time_fun_2 = str(time_fun_2).replace("timeutil","%s" %timeunit).replace("t_to_s","%s" %t_to_s).replace("_1","") + + fun_fix_column_j = ['(t1.q_ts)','(t1.ts)', '(t2.q_ts)','(t2.ts)','1600000000000','1600000000000000','1600000000000000000', + '%d' %t, '%d000' %t, '%d000000' %t,'t_to_s'] + + column_j1,column_j2 = random.sample(fun_fix_column_j,1),random.sample(fun_fix_column_j,1) + column_j12 = ['(%s,%s,timeutil)'%(column_j1,column_j2)] + fun_column_join_1 = random.sample(time_functions,1)+random.sample(column_j12,1) + time_fun_join_1 = str(fun_column_join_1).replace("[","").replace("]","").replace("'","").replace(", ","").replace("\"","").replace("t_to_s","'t_to_s'") + time_fun_join_1 = str(time_fun_join_1).replace("timeutil","%s" %timeunit).replace("t_to_s","%s" %t_to_s).replace("_1","") + + column_j3,column_j4 = random.sample(fun_fix_column_j,1),random.sample(fun_fix_column_j,1) + column_j34 = ['(%s,%s,timeutil)'%(column_j3,column_j4)] + fun_column_join_2 = random.sample(time_functions,1)+random.sample(column_j34,1) + time_fun_join_2 = str(fun_column_join_2).replace("[","").replace("]","").replace("'","").replace(", ","").replace("\"","").replace("t_to_s","'t_to_s'") + time_fun_join_2 = str(time_fun_join_2).replace("timeutil","%s" %timeunit).replace("t_to_s","%s" %t_to_s).replace("_1","") + + elif (timelist == ['TIMEDIFF_2']): + time_functions = timelist + + t = time.time() + t_to_s = time.strftime('%Y-%m-%d %H:%M:%S', time.localtime(t)) + + fun_fix_column = ['q_ts','ts','_c0','_C0','_rowts','1600000000000','1600000000000000','1600000000000000000', + '%d' %t, '%d000' %t, '%d000000' %t,'t_to_s'] + + column_1,column_2 = random.sample(fun_fix_column,1),random.sample(fun_fix_column,1) + column_12 = ['(%s,%s)'%(column_1,column_2)] + fun_column_1 = random.sample(time_functions,1)+random.sample(column_12,1) + time_fun_1 = str(fun_column_1).replace("[","").replace("]","").replace("'","").replace(", ","").replace("\"","").replace("t_to_s","'t_to_s'") + time_fun_1 = str(time_fun_1).replace("t_to_s","%s" %t_to_s).replace("_2","") + + column_3,column_4 = random.sample(fun_fix_column,1),random.sample(fun_fix_column,1) + column_34 = ['(%s,%s)'%(column_3,column_4)] + fun_column_2 = random.sample(time_functions,1)+random.sample(column_34,1) + time_fun_2 = str(fun_column_2).replace("[","").replace("]","").replace("'","").replace(", ","").replace("\"","").replace("t_to_s","'t_to_s'") + time_fun_2 = str(time_fun_2).replace("t_to_s","%s" %t_to_s).replace("_2","") + + fun_fix_column_j = ['(t1.q_ts)','(t1.ts)', '(t2.q_ts)','(t2.ts)','1600000000000','1600000000000000','1600000000000000000', + '%d' %t, '%d000' %t, '%d000000' %t,'t_to_s'] + + column_j1,column_j2 = random.sample(fun_fix_column_j,1),random.sample(fun_fix_column_j,1) + column_j12 = ['(%s,%s)'%(column_j1,column_j2)] + fun_column_join_1 = random.sample(time_functions,1)+random.sample(column_j12,1) + time_fun_join_1 = str(fun_column_join_1).replace("[","").replace("]","").replace("'","").replace(", ","").replace("\"","").replace("t_to_s","'t_to_s'") + time_fun_join_1 = str(time_fun_join_1).replace("t_to_s","%s" %t_to_s).replace("_2","") + + column_j3,column_j4 = random.sample(fun_fix_column_j,1),random.sample(fun_fix_column_j,1) + column_j34 = ['(%s,%s)'%(column_j3,column_j4)] + fun_column_join_2 = random.sample(time_functions,1)+random.sample(column_j34,1) + time_fun_join_2 = str(fun_column_join_2).replace("[","").replace("]","").replace("'","").replace(", ","").replace("\"","").replace("t_to_s","'t_to_s'") + time_fun_join_2 = str(time_fun_join_2).replace("t_to_s","%s" %t_to_s).replace("_2","") + + elif (timelist == ['ELAPSED']): + time_functions = timelist + + fun_fix_column = ['(ts)','(_c0)','(_C0)','(_rowts)','(ts,time_unit)','(_c0,time_unit)','(_C0,time_unit)','(_rowts,time_unit)'] + + time_units = ['1s','1m','1h','1d','1a'] + time_unit1 = str(random.sample(time_units,1)).replace("[","").replace("]","").replace("'","") + time_unit2 = str(random.sample(time_units,1)).replace("[","").replace("]","").replace("'","") + + fun_column_1 = random.sample(time_functions,1)+random.sample(fun_fix_column,1) + time_fun_1 = str(fun_column_1).replace("[","").replace("]","").replace("'","").replace(", ","").replace("time_unit","%s" %time_unit1) + + fun_column_2 = random.sample(time_functions,1)+random.sample(fun_fix_column,1) + time_fun_2 = str(fun_column_2).replace("[","").replace("]","").replace("'","").replace(", ","").replace("time_unit","%s" %time_unit2) + + + fun_fix_column_j = ['(t1.ts)', '(t2.ts)','(t1.ts,time_unit)','(t1.ts,time_unit)','(t2.ts,time_unit)','(t2.ts,time_unit)'] + + fun_column_join_1 = random.sample(time_functions,1)+random.sample(fun_fix_column_j,1) + time_fun_join_1 = str(fun_column_join_1).replace("[","").replace("]","").replace("'","").replace(", ","").replace("time_unit","%s" %time_unit1) + + fun_column_join_2 = random.sample(time_functions,1)+random.sample(fun_fix_column_j,1) + time_fun_join_2 = str(fun_column_join_2).replace("[","").replace("]","").replace("'","").replace(", ","").replace("time_unit","%s" %time_unit2) + + + elif (timelist == ['CAST']) : + str_functions = timelist + #下面的4个是全的,这个只是1个 + i = random.randint(1,4) + if i ==1: + print('===========cast_1===========') + fun_fix_column = ['q_bool','q_bool_null','q_bigint','q_bigint_null','q_smallint','q_smallint_null', + 'q_tinyint','q_tinyint_null','q_int','q_int_null','q_float','q_float_null','q_double','q_double_null'] + type_names = ['BIGINT','BINARY(100)','TIMESTAMP','NCHAR(100)','BIGINT UNSIGNED'] + + type_name1 = str(random.sample(type_names,1)).replace("[","").replace("]","").replace("'","") + fun_column_1 = str(random.sample(str_functions,1))+'('+str(random.sample(fun_fix_column,1))+' AS '+type_name1+')' + time_fun_1 = str(fun_column_1).replace("[","").replace("]","").replace("'","") + + type_name2 = str(random.sample(type_names,1)).replace("[","").replace("]","").replace("'","") + fun_column_2 = str(random.sample(str_functions,1))+'('+str(random.sample(fun_fix_column,1))+' AS '+type_name2+')' + time_fun_2 = str(fun_column_2).replace("[","").replace("]","").replace("'","") + + fun_fix_column_j = ['t1.q_bool','t1.q_bool_null','t1.q_bigint','t1.q_bigint_null','t1.q_smallint','t1.q_smallint_null', + 't1.q_tinyint','t1.q_tinyint_null','t1.q_int','t1.q_int_null','t1.q_float','t1.q_float_null','t1.q_double','t1.q_double_null', + 't2.q_bool','t2.q_bool_null','t2.q_bigint','t2.q_bigint_null','t2.q_smallint','t2.q_smallint_null', + 't2.q_tinyint','t2.q_tinyint_null','t2.q_int','t2.q_int_null','t2.q_float','t2.q_float_null','t2.q_double','t2.q_double_null'] + + type_name_j1 = str(random.sample(type_names,1)).replace("[","").replace("]","").replace("'","") + fun_column_join_1 = str(random.sample(str_functions,1))+'('+str(random.sample(fun_fix_column_j,1))+' AS '+type_name_j1+')' + time_fun_join_1 = str(fun_column_join_1).replace("[","").replace("]","").replace("'","") + + type_name_j2 = str(random.sample(type_names,1)).replace("[","").replace("]","").replace("'","") + fun_column_join_2 = str(random.sample(str_functions,1))+'('+str(random.sample(fun_fix_column_j,1))+' AS '+type_name_j2+')' + time_fun_join_2 = str(fun_column_join_2).replace("[","").replace("]","").replace("'","") + + elif i==2: + print('===========cast_2===========') + fun_fix_column = ['q_binary','q_binary_null','q_binary1','q_binary2','q_binary3','q_binary4'] + type_names = ['BIGINT','BINARY(100)','NCHAR(100)','BIGINT UNSIGNED'] + + type_name1 = str(random.sample(type_names,1)).replace("[","").replace("]","").replace("'","") + fun_column_1 = str(random.sample(str_functions,1))+'('+str(random.sample(fun_fix_column,1))+' AS '+type_name1+')' + time_fun_1 = str(fun_column_1).replace("[","").replace("]","").replace("'","") + + type_name2 = str(random.sample(type_names,1)).replace("[","").replace("]","").replace("'","") + fun_column_2 = str(random.sample(str_functions,1))+'('+str(random.sample(fun_fix_column,1))+' AS '+type_name2+')' + time_fun_2 = str(fun_column_2).replace("[","").replace("]","").replace("'","") + + fun_fix_column_j = ['t1.q_binary','t1.q_binary_null','t1.q_binary1','t1.q_binary2','t1.q_binary3','t1.q_binary4', + 't2.q_binary','t2.q_binary_null','t2.q_binary1','t2.q_binary2','t2.q_binary3','t2.q_binary4'] + + type_name_j1 = str(random.sample(type_names,1)).replace("[","").replace("]","").replace("'","") + fun_column_join_1 = str(random.sample(str_functions,1))+'('+str(random.sample(fun_fix_column_j,1))+' AS '+type_name_j1+')' + time_fun_join_1 = str(fun_column_join_1).replace("[","").replace("]","").replace("'","") + + type_name_j2 = str(random.sample(type_names,1)).replace("[","").replace("]","").replace("'","") + fun_column_join_2 = str(random.sample(str_functions,1))+'('+str(random.sample(fun_fix_column_j,1))+' AS '+type_name_j2+')' + time_fun_join_2 = str(fun_column_join_2).replace("[","").replace("]","").replace("'","") + + elif i==3: + print('===========cast_3===========') + fun_fix_column = ['q_nchar','q_nchar_null','q_nchar5','q_nchar6','q_nchar7','q_nchar8'] + type_names = ['BIGINT','NCHAR(100)','BIGINT UNSIGNED'] + + type_name1 = str(random.sample(type_names,1)).replace("[","").replace("]","").replace("'","") + fun_column_1 = str(random.sample(str_functions,1))+'('+str(random.sample(fun_fix_column,1))+' AS '+type_name1+')' + time_fun_1 = str(fun_column_1).replace("[","").replace("]","").replace("'","") + + type_name2 = str(random.sample(type_names,1)).replace("[","").replace("]","").replace("'","") + fun_column_2 = str(random.sample(str_functions,1))+'('+str(random.sample(fun_fix_column,1))+' AS '+type_name2+')' + time_fun_2 = str(fun_column_2).replace("[","").replace("]","").replace("'","") + + fun_fix_column_j = ['t1.q_nchar','t1.q_nchar_null','t1.q_nchar5','t1.q_nchar6','t1.q_nchar7','t1.q_nchar8', + 't2.q_nchar','t2.q_nchar_null','t2.q_nchar5','t2.q_nchar6','t2.q_nchar7','t2.q_nchar8'] + + type_name_j1 = str(random.sample(type_names,1)).replace("[","").replace("]","").replace("'","") + fun_column_join_1 = str(random.sample(str_functions,1))+'('+str(random.sample(fun_fix_column_j,1))+' AS '+type_name_j1+')' + time_fun_join_1 = str(fun_column_join_1).replace("[","").replace("]","").replace("'","") + + type_name_j2 = str(random.sample(type_names,1)).replace("[","").replace("]","").replace("'","") + fun_column_join_2 = str(random.sample(str_functions,1))+'('+str(random.sample(fun_fix_column_j,1))+' AS '+type_name_j2+')' + time_fun_join_2 = str(fun_column_join_2).replace("[","").replace("]","").replace("'","") + + elif i==4: + print('===========cast_4===========') + fun_fix_column = ['q_ts','q_ts_null','_C0','_c0','ts','_rowts'] + type_names = ['BIGINT','TIMESTAMP','BIGINT UNSIGNED'] + + type_name1 = str(random.sample(type_names,1)).replace("[","").replace("]","").replace("'","") + fun_column_1 = str(random.sample(str_functions,1))+'('+str(random.sample(fun_fix_column,1))+' AS '+type_name1+')' + time_fun_1 = str(fun_column_1).replace("[","").replace("]","").replace("'","") + + type_name2 = str(random.sample(type_names,1)).replace("[","").replace("]","").replace("'","") + fun_column_2 = str(random.sample(str_functions,1))+'('+str(random.sample(fun_fix_column,1))+' AS '+type_name2+')' + time_fun_2 = str(fun_column_2).replace("[","").replace("]","").replace("'","") + + fun_fix_column_j = ['t1.q_ts','t1.q_ts_null','t1.ts','t2.q_ts','t2.q_ts_null','t2.ts'] + + type_name_j1 = str(random.sample(type_names,1)).replace("[","").replace("]","").replace("'","") + fun_column_join_1 = str(random.sample(str_functions,1))+'('+str(random.sample(fun_fix_column_j,1))+' AS '+type_name_j1+')' + time_fun_join_1 = str(fun_column_join_1).replace("[","").replace("]","").replace("'","") + + type_name_j2 = str(random.sample(type_names,1)).replace("[","").replace("]","").replace("'","") + fun_column_join_2 = str(random.sample(str_functions,1))+'('+str(random.sample(fun_fix_column_j,1))+' AS '+type_name_j2+')' + time_fun_join_2 = str(fun_column_join_2).replace("[","").replace("]","").replace("'","") + + elif (timelist == ['CAST_1']) : + str_functions = timelist + + print('===========cast_1===========') + fun_fix_column = ['q_bool','q_bool_null','q_bigint','q_bigint_null','q_smallint','q_smallint_null', + 'q_tinyint','q_tinyint_null','q_int','q_int_null','q_float','q_float_null','q_double','q_double_null'] + type_names = ['BIGINT','BINARY(100)','TIMESTAMP','NCHAR(100)','BIGINT UNSIGNED'] + + type_name1 = str(random.sample(type_names,1)).replace("[","").replace("]","").replace("'","") + fun_column_1 = str(random.sample(str_functions,1))+'('+str(random.sample(fun_fix_column,1))+' AS '+type_name1+')' + time_fun_1 = str(fun_column_1).replace("[","").replace("]","").replace("'","").replace("_1","") + + type_name2 = str(random.sample(type_names,1)).replace("[","").replace("]","").replace("'","") + fun_column_2 = str(random.sample(str_functions,1))+'('+str(random.sample(fun_fix_column,1))+' AS '+type_name2+')' + time_fun_2 = str(fun_column_2).replace("[","").replace("]","").replace("'","").replace("_1","") + + fun_fix_column_j = ['t1.q_bool','t1.q_bool_null','t1.q_bigint','t1.q_bigint_null','t1.q_smallint','t1.q_smallint_null', + 't1.q_tinyint','t1.q_tinyint_null','t1.q_int','t1.q_int_null','t1.q_float','t1.q_float_null','t1.q_double','t1.q_double_null', + 't2.q_bool','t2.q_bool_null','t2.q_bigint','t2.q_bigint_null','t2.q_smallint','t2.q_smallint_null', + 't2.q_tinyint','t2.q_tinyint_null','t2.q_int','t2.q_int_null','t2.q_float','t2.q_float_null','t2.q_double','t2.q_double_null'] + + type_name_j1 = str(random.sample(type_names,1)).replace("[","").replace("]","").replace("'","") + fun_column_join_1 = str(random.sample(str_functions,1))+'('+str(random.sample(fun_fix_column_j,1))+' AS '+type_name_j1+')' + time_fun_join_1 = str(fun_column_join_1).replace("[","").replace("]","").replace("'","").replace("_1","") + + type_name_j2 = str(random.sample(type_names,1)).replace("[","").replace("]","").replace("'","") + fun_column_join_2 = str(random.sample(str_functions,1))+'('+str(random.sample(fun_fix_column_j,1))+' AS '+type_name_j2+')' + time_fun_join_2 = str(fun_column_join_2).replace("[","").replace("]","").replace("'","").replace("_1","") + + elif (timelist == ['CAST_2']) : + str_functions = timelist + print('===========cast_2===========') + fun_fix_column = ['q_binary','q_binary_null','q_binary1','q_binary2','q_binary3','q_binary4'] + type_names = ['BIGINT','BINARY(100)','NCHAR(100)','BIGINT UNSIGNED'] + + type_name1 = str(random.sample(type_names,1)).replace("[","").replace("]","").replace("'","") + fun_column_1 = str(random.sample(str_functions,1))+'('+str(random.sample(fun_fix_column,1))+' AS '+type_name1+')' + time_fun_1 = str(fun_column_1).replace("[","").replace("]","").replace("'","").replace("_2","") + + type_name2 = str(random.sample(type_names,1)).replace("[","").replace("]","").replace("'","") + fun_column_2 = str(random.sample(str_functions,1))+'('+str(random.sample(fun_fix_column,1))+' AS '+type_name2+')' + time_fun_2 = str(fun_column_2).replace("[","").replace("]","").replace("'","").replace("_2","") + + fun_fix_column_j = ['t1.q_binary','t1.q_binary_null','t1.q_binary1','t1.q_binary2','t1.q_binary3','t1.q_binary4', + 't2.q_binary','t2.q_binary_null','t2.q_binary1','t2.q_binary2','t2.q_binary3','t2.q_binary4'] + + type_name_j1 = str(random.sample(type_names,1)).replace("[","").replace("]","").replace("'","") + fun_column_join_1 = str(random.sample(str_functions,1))+'('+str(random.sample(fun_fix_column_j,1))+' AS '+type_name_j1+')' + time_fun_join_1 = str(fun_column_join_1).replace("[","").replace("]","").replace("'","").replace("_2","") + + type_name_j2 = str(random.sample(type_names,1)).replace("[","").replace("]","").replace("'","") + fun_column_join_2 = str(random.sample(str_functions,1))+'('+str(random.sample(fun_fix_column_j,1))+' AS '+type_name_j2+')' + time_fun_join_2 = str(fun_column_join_2).replace("[","").replace("]","").replace("'","").replace("_2","") + + elif (timelist == ['CAST_3']) : + str_functions = timelist + print('===========cast_3===========') + fun_fix_column = ['q_nchar','q_nchar_null','q_nchar5','q_nchar6','q_nchar7','q_nchar8'] + type_names = ['BIGINT','NCHAR(100)','BIGINT UNSIGNED'] + + type_name1 = str(random.sample(type_names,1)).replace("[","").replace("]","").replace("'","") + fun_column_1 = str(random.sample(str_functions,1))+'('+str(random.sample(fun_fix_column,1))+' AS '+type_name1+')' + time_fun_1 = str(fun_column_1).replace("[","").replace("]","").replace("'","").replace("_3","") + + type_name2 = str(random.sample(type_names,1)).replace("[","").replace("]","").replace("'","") + fun_column_2 = str(random.sample(str_functions,1))+'('+str(random.sample(fun_fix_column,1))+' AS '+type_name2+')' + time_fun_2 = str(fun_column_2).replace("[","").replace("]","").replace("'","").replace("_3","") + + fun_fix_column_j = ['t1.q_nchar','t1.q_nchar_null','t1.q_nchar5','t1.q_nchar6','t1.q_nchar7','t1.q_nchar8', + 't2.q_nchar','t2.q_nchar_null','t2.q_nchar5','t2.q_nchar6','t2.q_nchar7','t2.q_nchar8'] + + type_name_j1 = str(random.sample(type_names,1)).replace("[","").replace("]","").replace("'","") + fun_column_join_1 = str(random.sample(str_functions,1))+'('+str(random.sample(fun_fix_column_j,1))+' AS '+type_name_j1+')' + time_fun_join_1 = str(fun_column_join_1).replace("[","").replace("]","").replace("'","").replace("_3","") + + type_name_j2 = str(random.sample(type_names,1)).replace("[","").replace("]","").replace("'","") + fun_column_join_2 = str(random.sample(str_functions,1))+'('+str(random.sample(fun_fix_column_j,1))+' AS '+type_name_j2+')' + time_fun_join_2 = str(fun_column_join_2).replace("[","").replace("]","").replace("'","").replace("_3","") + + elif (timelist == ['CAST_4']) : + str_functions = timelist + print('===========cast_4===========') + fun_fix_column = ['q_ts','q_ts_null','_C0','_c0','ts','_rowts'] + type_names = ['BIGINT','TIMESTAMP','BIGINT UNSIGNED'] + + type_name1 = str(random.sample(type_names,1)).replace("[","").replace("]","").replace("'","") + fun_column_1 = str(random.sample(str_functions,1))+'('+str(random.sample(fun_fix_column,1))+' AS '+type_name1+')' + time_fun_1 = str(fun_column_1).replace("[","").replace("]","").replace("'","").replace("_4","") + + type_name2 = str(random.sample(type_names,1)).replace("[","").replace("]","").replace("'","") + fun_column_2 = str(random.sample(str_functions,1))+'('+str(random.sample(fun_fix_column,1))+' AS '+type_name2+')' + time_fun_2 = str(fun_column_2).replace("[","").replace("]","").replace("'","").replace("_4","") + + fun_fix_column_j = ['t1.q_ts','t1.q_ts_null','t1.ts','t2.q_ts','t2.q_ts_null','t2.ts'] + + type_name_j1 = str(random.sample(type_names,1)).replace("[","").replace("]","").replace("'","") + fun_column_join_1 = str(random.sample(str_functions,1))+'('+str(random.sample(fun_fix_column_j,1))+' AS '+type_name_j1+')' + time_fun_join_1 = str(fun_column_join_1).replace("[","").replace("]","").replace("'","").replace("_4","") + + type_name_j2 = str(random.sample(type_names,1)).replace("[","").replace("]","").replace("'","") + fun_column_join_2 = str(random.sample(str_functions,1))+'('+str(random.sample(fun_fix_column_j,1))+' AS '+type_name_j2+')' + time_fun_join_2 = str(fun_column_join_2).replace("[","").replace("]","").replace("'","").replace("_4","") + + tdSql.query("select /*+ para_tables_sort() */1-1 as time_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (timelist == ['NOW','TODAY']) or (timelist == ['TIMETRUNCATE']) or (timelist == ['TO_ISO8601'])\ + or (timelist == ['TO_UNIXTIMESTAMP']) or (timelist == ['TIMEDIFF_1']) or (timelist == ['TIMEDIFF_2']): + sql = "select /*+ para_tables_sort() */ ts1 , timediff(asct1,now) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1, " % time_fun_1 + sql += "%s as asct2, " % time_fun_2 + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts as ts1 from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += ");" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + elif (timelist == ['TIMEZONE']) \ + or (timelist == ['CAST']) or (timelist == ['CAST_1']) or (timelist == ['CAST_2']) or (timelist == ['CAST_3']) or (timelist == ['CAST_4']): + sql = "select /*+ para_tables_sort() */ ts2 , asct1,now(),today(),timezone() from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1, " % time_fun_1 + sql += "%s as asct2, " % time_fun_2 + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts as ts2 from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += ");" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + elif (timelist == ['ELAPSED']) : + sql = "select /*+ para_tables_sort() */ max(asct1),now(),today(),timezone() from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1, " % time_fun_1 + sql += "%s as asct2 " % time_fun_2 + sql += "from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += ");" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-2 as time_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (timelist == ['NOW','TODAY']) or (timelist == ['TIMETRUNCATE']) or (timelist == ['TO_ISO8601'])\ + or (timelist == ['TO_UNIXTIMESTAMP']) or (timelist == ['TIMEDIFF_1']) or (timelist == ['TIMEDIFF_2']): + sql = "select /*+ para_tables_sort() */ ts , timediff(asct1,now),now(),today(),timezone() from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1, " % time_fun_1 + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts ts from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s )" % random.choice(self.order_where) + sql += "%s " % random.choice(self.unionall_or_union) + sql += "select /*+ para_tables_sort() */ ts , timediff(asct2,now),now(),today(),timezone() from ( select /*+ para_tables_sort() */ " + sql += "%s as asct2, " % time_fun_2 + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts ts from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ")" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + elif (timelist == ['TIMEZONE']) or (timelist == ['CAST']) or (timelist == ['CAST_1']) or (timelist == ['CAST_2']) or (timelist == ['CAST_3']) or (timelist == ['CAST_4']): + sql = "select /*+ para_tables_sort() */ ts , (asct1),now(),today(),timezone() from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1, " % time_fun_1 + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts ts from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s )" % random.choice(self.order_where) + sql += "%s " % random.choice(self.unionall_or_union) + sql += "select /*+ para_tables_sort() */ ts , asct2,now(),today(),timezone() from ( select /*+ para_tables_sort() */ " + sql += "%s as asct2, " % time_fun_2 + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts ts from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ")" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + elif (timelist == ['ELAPSED']) : + sql = "select /*+ para_tables_sort() */ min(asct1),now(),today(),timezone() from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1 " % time_fun_1 + sql += " from regular_table_1 where " + sql += "%s )" % random.choice(self.q_where) + sql += "%s " % random.choice(self.unionall_or_union) + sql += "select /*+ para_tables_sort() */ avg(asct2),now(),today(),timezone() from ( select /*+ para_tables_sort() */ " + sql += "%s as asct2 " % time_fun_2 + sql += " from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ")" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-3 as time_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (timelist == ['NOW','TODAY']) or (timelist == ['TIMETRUNCATE']) or (timelist == ['TO_ISO8601'])\ + or (timelist == ['TO_UNIXTIMESTAMP']) or (timelist == ['TIMEDIFF_1']) or (timelist == ['TIMEDIFF_2']): + sql = "select /*+ para_tables_sort() */ ts , timediff(asct1,now) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1, ts ," % time_fun_1 + sql += "%s as asct2, " % time_fun_2 + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s " % random.choice(self.q_select) + sql += "from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s select /*+ para_tables_sort() */" % random.choice(self.unionall_or_union) + sql += "%s as asct2, ts ," % time_fun_2 + sql += "%s as asct1, " % time_fun_1 + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s " % random.choice(self.q_select) + sql += "from regular_table_2 where " + sql += "%s " % random.choice(self.q_where) + sql += " order by asct1 desc " + sql += ")" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + elif (timelist == ['TIMEZONE']) or (timelist == ['CAST']) or (timelist == ['CAST_1']) or (timelist == ['CAST_2']) or (timelist == ['CAST_3']) or (timelist == ['CAST_4']): + sql = "select /*+ para_tables_sort() */ ts , (asct1),now(),today(),timezone() from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1, ts ," % time_fun_1 + sql += "%s as asct2, " % time_fun_2 + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s " % random.choice(self.q_select) + sql += " from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s select /*+ para_tables_sort() */" % random.choice(self.unionall_or_union) + sql += "%s as asct2, ts ," % time_fun_2 + sql += "%s as asct1, " % time_fun_1 + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s " % random.choice(self.q_select) + sql += "from regular_table_2 where " + sql += "%s " % random.choice(self.q_where) + sql += " order by asct1 desc " + sql += ")" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + elif (timelist == ['ELAPSED']) : + sql = "select /*+ para_tables_sort() */ abs(asct1),now(),today(),timezone() from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1," % time_fun_1 + sql += "%s as asct2 " % time_fun_2 + sql += "from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s select /*+ para_tables_sort() */" % random.choice(self.unionall_or_union) + sql += "%s as asct2," % time_fun_2 + sql += "%s as asct1 " % time_fun_1 + sql += "from regular_table_2 where " + sql += "%s " % random.choice(self.q_where) + sql += " order by asct1 asc " + sql += ")" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-4 as time_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (timelist == ['NOW','TODAY']) or (timelist == ['TIMETRUNCATE']) or (timelist == ['TO_ISO8601'])\ + or (timelist == ['TO_UNIXTIMESTAMP']) or (timelist == ['TIMEDIFF_1']) or (timelist == ['TIMEDIFF_2']): + sql = "select /*+ para_tables_sort() */ ts1,ts2 ,timediff(ts1,ts2), timediff(asct1,now) from ( select /*+ para_tables_sort() */ t1.ts as ts1," + sql += "%s as asct11, " % time_fun_join_1 + sql += "%s as asct1, " % time_fun_join_2 + sql += "%s as asct12, " % time_fun_join_1 + sql += "t1.%s as asct111, " % random.choice(self.q_select) + sql += "t2.%s as asct121, " % random.choice(self.q_select) + sql += "t2.ts as ts2 from regular_table_1 t1 , regular_table_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_where) + sql += "and %s " % random.choice(self.q_u_or_where) + sql += "%s " % random.choice(self.order_u_where) + sql += ");" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + elif (timelist == ['TIMEZONE']) or (timelist == ['CAST']) or (timelist == ['CAST_1']) or (timelist == ['CAST_2']) or (timelist == ['CAST_3']) or (timelist == ['CAST_4']): + sql = "select /*+ para_tables_sort() */ ts1,ts2 ,timediff(ts1,ts2), (asct1) from ( select /*+ para_tables_sort() */ t1.ts as ts1," + sql += "%s as asct10, " % time_fun_join_1 + sql += "%s as asct1, " % time_fun_join_2 + sql += "%s as asct11, " % time_fun_join_1 + sql += "t1.%s as asct12, " % random.choice(self.q_select) + sql += "t2.%s as asct13, " % random.choice(self.q_select) + sql += "t2.ts as ts2 from regular_table_1 t1 , regular_table_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_where) + sql += "and %s " % random.choice(self.q_u_or_where) + sql += "%s " % random.choice(self.order_u_where) + sql += ");" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + elif (timelist == ['ELAPSED']) : + sql = "select /*+ para_tables_sort() */ floor(asct1) from ( select /*+ para_tables_sort() */" + sql += "%s as asct10, " % time_fun_join_1 + sql += "%s as asct1, " % time_fun_join_2 + sql += "%s as asct11" % time_fun_join_1 + sql += " from regular_table_1 t1 , regular_table_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_where) + sql += "and %s " % random.choice(self.q_u_or_where) + sql += ");" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-5 as time_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (timelist == ['ELAPSED']) : + sql = "select /*+ para_tables_sort() */ now(),today(),timezone(), " + sql += "%s, " % time_fun_1 + sql += "%s " % time_fun_2 + sql += " from ( select /*+ para_tables_sort() */ * from regular_table_1 ) where " + sql += "%s " % random.choice(self.q_where) + sql += " ;" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + else: + sql = "select /*+ para_tables_sort() */ ts ,now(),today(),timezone(), " + sql += "%s as asct11, " % time_fun_1 + sql += "%s as asct12, " % random.choice(self.q_select) + sql += "%s as asct13, " % random.choice(self.q_select) + sql += "%s as asct14 " % time_fun_2 + sql += " from ( select /*+ para_tables_sort() */ * from regular_table_1 ) where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += " ;" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-6 as time_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (timelist == ['NOW','TODAY']) or (timelist == ['TIMETRUNCATE']) or (timelist == ['TO_ISO8601'])\ + or (timelist == ['TO_UNIXTIMESTAMP']) or (timelist == ['TIMEDIFF_1']) or (timelist == ['TIMEDIFF_2']): + sql = "select /*+ para_tables_sort() */ ts1,ts ,timediff(ts1,ts), timediff(asct1,now) from ( select /*+ para_tables_sort() */ t1.ts as ts1," + sql += "%s as asct121, " % time_fun_join_1 + sql += "%s as asct1, " % time_fun_join_2 + sql += "t1.%s as asct11, " % random.choice(self.q_select) + sql += "t2.%s as asct12, " % random.choice(self.q_select) + sql += "%s as asct13, " % time_fun_join_1 + sql += "t2.ts from regular_table_1 t1 , regular_table_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_where) + sql += "and %s )" % random.choice(self.q_u_or_where) + sql += "%s " % random.choice(self.order_where) + sql += "%s " % random.choice(self.limit1_where) + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + elif (timelist == ['TIMEZONE']) or (timelist == ['CAST']) or (timelist == ['CAST_1']) or (timelist == ['CAST_2']) or (timelist == ['CAST_3']) or (timelist == ['CAST_4']): + sql = "select /*+ para_tables_sort() */ ts1,ts ,timediff(ts1,ts), (asct1) from ( select /*+ para_tables_sort() */ t1.ts as ts1," + sql += "%s as asct121, " % time_fun_join_1 + sql += "%s as asct1, " % time_fun_join_2 + sql += "t1.%s as asct11, " % random.choice(self.q_select) + sql += "t2.%s as asct12, " % random.choice(self.q_select) + sql += "%s as asct13, " % time_fun_join_1 + sql += "t2.ts from regular_table_1 t1 , regular_table_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_where) + sql += "and %s )" % random.choice(self.q_u_or_where) + sql += "%s " % random.choice(self.order_where) + sql += "%s " % random.choice(self.limit1_where) + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + elif (timelist == ['ELAPSED']) : + sql = "select /*+ para_tables_sort() */ (asct1)*111 from ( select /*+ para_tables_sort() */ " + sql += "%s as asct121, " % time_fun_join_1 + sql += "%s as asct1, " % time_fun_join_2 + sql += "%s as asct122 " % time_fun_join_1 + sql += " from regular_table_1 t1 , regular_table_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_where) + sql += "and %s )" % random.choice(self.q_u_or_where) + sql += "%s " % random.choice(self.limit1_where) + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-7 as time_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (timelist == ['NOW','TODAY']) or (timelist == ['TIMETRUNCATE']) or (timelist == ['TO_ISO8601'])\ + or (timelist == ['TO_UNIXTIMESTAMP']) or (timelist == ['TIMEDIFF_1']) or (timelist == ['TIMEDIFF_2']): + sql = "select /*+ para_tables_sort() */ ts1,m1 , timediff(asct1,now) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1, ts as m1," % time_fun_1 + sql += "%s as asct2, " % time_fun_2 + sql += "%s as asct11, " % random.choice(self.q_select) + sql += "%s as asct12, " % random.choice(self.t_select) + sql += "ts as ts1 from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += "%s " % random.choice(self.order_where) + sql += ") ;" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + elif (timelist == ['TIMEZONE']) or (timelist == ['CAST']) or (timelist == ['CAST_1']) or (timelist == ['CAST_2']) or (timelist == ['CAST_3']) or (timelist == ['CAST_4']): + sql = "select /*+ para_tables_sort() */ tm1,tm2 , (asct1),now(),today(),timezone() from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1, ts as tm1," % time_fun_1 + sql += "%s as asct2, " % time_fun_2 + sql += "%s as asct11, " % random.choice(self.q_select) + sql += "%s as asct12, " % random.choice(self.t_select) + sql += "ts as tm2 from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += "%s " % random.choice(self.order_where) + sql += ") ;" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + elif (timelist == ['ELAPSED']) : + sql = "select /*+ para_tables_sort() */ (asct1)/asct2 ,now(),today(),timezone() from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1, " % time_fun_1 + sql += "%s as asct2 " % time_fun_2 + sql += "from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += ") ;" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-8 as time_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (timelist == ['NOW','TODAY']) or (timelist == ['TIMETRUNCATE']) or (timelist == ['TO_ISO8601'])\ + or (timelist == ['TO_UNIXTIMESTAMP']) or (timelist == ['TIMEDIFF_1']) or (timelist == ['TIMEDIFF_2']): + sql = "select /*+ para_tables_sort() */ tm1,tm2 , timediff(asct1,now) " + sql += "from ( select /*+ para_tables_sort() */ " + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s as asct1, ts as tm1," % time_fun_1 + sql += "%s as asct2, " % time_fun_2 + sql += "%s as asct11, " % random.choice(self.q_select) + sql += "%s as asct12, " % random.choice(self.t_select) + sql += "ts as tm2 from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += "%s " % random.choice(self.order_where) + sql += ") ;" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + elif (timelist == ['TIMEZONE']) or (timelist == ['CAST']) or (timelist == ['CAST_1']) or (timelist == ['CAST_2']) or (timelist == ['CAST_3']) or (timelist == ['CAST_4']): + sql = "select /*+ para_tables_sort() */ ts1,ts2 , (asct1),now(),today(),timezone() " + sql += "from ( select /*+ para_tables_sort() */ " + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s as asct1, ts as ts1," % time_fun_1 + sql += "%s as asct2, " % time_fun_2 + sql += "%s as asct11, " % random.choice(self.q_select) + sql += "%s as asct12, " % random.choice(self.t_select) + sql += "ts as ts2 from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += "%s " % random.choice(self.order_where) + sql += ") ;" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + elif (timelist == ['ELAPSED']) : + sql = "select /*+ para_tables_sort() */ floor(abs(asct1)),now(),today(),timezone() " + sql += "from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1, " % time_fun_1 + sql += "%s as asct2 " % time_fun_2 + sql += "from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += ") ;" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-9 as time_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (timelist == ['NOW','TODAY']) or (timelist == ['TIMETRUNCATE']) or (timelist == ['TO_ISO8601'])\ + or (timelist == ['TO_UNIXTIMESTAMP']) or (timelist == ['TIMEDIFF_1']) or (timelist == ['TIMEDIFF_2']): + sql = "select /*+ para_tables_sort() */ ts1,ts2 ,timediff(ts1,ts2), timediff(asct1,now) from ( select /*+ para_tables_sort() */ t1.ts as ts1," + sql += "%s as asct121, " % time_fun_join_1 + sql += "%s as asct1, " % time_fun_join_2 + sql += "t1.%s as asct11, " % random.choice(self.q_select) + sql += "t1.%s as asct12, " % random.choice(self.q_select) + sql += "t2.%s as asct13, " % random.choice(self.q_select) + sql += "t2.%s as asct14, " % random.choice(self.q_select) + sql += "t2.ts as ts2 from stable_1 t1 , stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.t_join_where) + sql += "and %s " % random.choice(self.t_u_where) + sql += "and %s " % random.choice(self.t_u_or_where) + sql += "%s " % random.choice(self.order_u_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ") ;" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + elif (timelist == ['TIMEZONE']) or (timelist == ['CAST']) or (timelist == ['CAST_1']) or (timelist == ['CAST_2']) or (timelist == ['CAST_3']) or (timelist == ['CAST_4']): + sql = "select /*+ para_tables_sort() */ ts1,ts2 ,timediff(ts1,ts2), asct1 from ( select /*+ para_tables_sort() */ t1.ts as ts1," + sql += "%s as asct121, " % time_fun_join_1 + sql += "%s as asct1, " % time_fun_join_2 + sql += "t1.%s as asct11, " % random.choice(self.q_select) + sql += "t1.%s as asct12, " % random.choice(self.q_select) + sql += "t2.%s as asct13, " % random.choice(self.q_select) + sql += "t2.%s as asct14, " % random.choice(self.q_select) + sql += "t2.ts as ts2 from stable_1 t1 , stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.t_join_where) + sql += "and %s " % random.choice(self.t_u_where) + sql += "and %s " % random.choice(self.t_u_or_where) + sql += "%s " % random.choice(self.order_u_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ") ;" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + elif (timelist == ['ELAPSED']) : + sql = "select /*+ para_tables_sort() */ min(asct1*110) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct121, " % time_fun_join_1 + sql += "%s as asct1 " % time_fun_join_2 + sql += "from stable_1 t1 , stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.t_join_where) + sql += "and %s " % random.choice(self.t_u_where) + sql += "and %s " % random.choice(self.t_u_or_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ") ;" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + + tdSql.query("select /*+ para_tables_sort() */1-10 as time_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (timelist == ['NOW','TODAY']) or (timelist == ['TIMETRUNCATE']) or (timelist == ['TO_ISO8601'])\ + or (timelist == ['TO_UNIXTIMESTAMP']) or (timelist == ['TIMEDIFF_1']) or (timelist == ['TIMEDIFF_2']): + sql = "select /*+ para_tables_sort() */ ts , timediff(asct1,now) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1 ," % time_fun_1 + sql += "%s as asct2, " % time_fun_2 + sql += "%s as asct121, " % random.choice(self.s_r_select) + sql += "%s as asct122, " % random.choice(self.q_select) + sql += "ts from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += ") %s " % random.choice(self.unionall_or_union) + sql += "select /*+ para_tables_sort() */ ts , timediff(asct1,now) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1 ," % time_fun_1 + sql += "%s as asct2, " % time_fun_2 + sql += "%s as asct121, " % random.choice(self.s_r_select) + sql += "%s as asct122, " % random.choice(self.q_select) + sql += "ts from stable_2 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ")" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + elif (timelist == ['TIMEZONE']) or (timelist == ['CAST']) or (timelist == ['CAST_1']) or (timelist == ['CAST_2']) or (timelist == ['CAST_3']) or (timelist == ['CAST_4']): + sql = "select /*+ para_tables_sort() */ ts , (asct1),now(),today(),timezone() from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1 ," % time_fun_1 + sql += "%s as asct2, " % time_fun_2 + sql += "%s as asct121, " % random.choice(self.s_r_select) + sql += "%s as asct122, " % random.choice(self.q_select) + sql += "ts from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += ") %s " % random.choice(self.unionall_or_union) + sql += "select /*+ para_tables_sort() */ ts , (asct2),now(),today(),timezone() from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1 ," % time_fun_1 + sql += "%s as asct2, " % time_fun_2 + sql += "%s as asct121, " % random.choice(self.s_r_select) + sql += "%s as asct122, " % random.choice(self.q_select) + sql += "ts from stable_2 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ")" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + elif (timelist == ['ELAPSED']) : + sql = "select /*+ para_tables_sort() */ abs(asct1),now(),today(),timezone() from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1 ," % time_fun_1 + sql += "%s as asct2 " % time_fun_2 + sql += " from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += ") %s " % random.choice(self.unionall_or_union) + sql += "select /*+ para_tables_sort() */ max(asct2),now(),today(),timezone() from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1 ," % time_fun_1 + sql += "%s as asct2 " % time_fun_2 + sql += "from stable_2 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ")" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + #3 inter union not support + tdSql.query("select /*+ para_tables_sort() */1-11 as time_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (timelist == ['NOW','TODAY']) or (timelist == ['TIMETRUNCATE']) or (timelist == ['TO_ISO8601'])\ + or (timelist == ['TO_UNIXTIMESTAMP']) or (timelist == ['TIMEDIFF_1']) or (timelist == ['TIMEDIFF_2']): + sql = "select /*+ para_tables_sort() */ ts , timediff(asct1,now), timediff(now,asct2) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1 ," % time_fun_1 + sql += "%s as asct2, " % time_fun_2 + sql += "%s as asct121, " % random.choice(self.s_r_select) + sql += "%s as asct122, " % random.choice(self.q_select) + sql += "ts from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += " %s " % random.choice(self.unionall_or_union) + sql += " select /*+ para_tables_sort() */ " + sql += "%s as asct1 ," % time_fun_1 + sql += "%s as asct2, " % time_fun_2 + sql += "%s as asct121, " % random.choice(self.s_r_select) + sql += "%s as asct122, " % random.choice(self.q_select) + sql += "ts from stable_2 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ")" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + elif (timelist == ['TIMEZONE']) or (timelist == ['CAST']) or (timelist == ['CAST_1']) or (timelist == ['CAST_2']) or (timelist == ['CAST_3']) or (timelist == ['CAST_4']): + sql = "select /*+ para_tables_sort() */ ts , asct1,now(),now(),asct2 from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1 ," % time_fun_1 + sql += "%s as asct2, " % time_fun_2 + sql += "%s as asct121, " % random.choice(self.s_r_select) + sql += "%s as asct122, " % random.choice(self.q_select) + sql += "ts from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += " %s " % random.choice(self.unionall_or_union) + sql += " select /*+ para_tables_sort() */ " + sql += "%s as asct1 ," % time_fun_1 + sql += "%s as asct2, " % time_fun_2 + sql += "%s as asct121, " % random.choice(self.s_r_select) + sql += "%s as asct122, " % random.choice(self.q_select) + sql += "ts from stable_2 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ")" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + elif (timelist == ['ELAPSED']) : + sql = "select /*+ para_tables_sort() */ asct1+asct2,now(),today(),timezone() from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1, " % time_fun_1 + sql += "%s as asct2 " % time_fun_2 + sql += " from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += " %s " % random.choice(self.unionall_or_union) + sql += " select /*+ para_tables_sort() */ " + sql += "%s as asct1 ," % time_fun_1 + sql += "%s as asct2 " % time_fun_2 + sql += " from stable_2 where " + sql += "%s " % random.choice(self.q_where) + sql += "order by asct1 " + sql += "%s " % random.choice(self.limit1_where) + sql += ")" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-12 as time_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (timelist == ['NOW','TODAY']) or (timelist == ['TIMETRUNCATE']) or (timelist == ['TO_ISO8601'])\ + or (timelist == ['TO_UNIXTIMESTAMP']) or (timelist == ['TIMEDIFF_1']) or (timelist == ['TIMEDIFF_2']): + sql = "select /*+ para_tables_sort() */ ts1,ts2 ,timediff(ts1,ts2), timediff(asct1,now) from ( select /*+ para_tables_sort() */ t1.ts as ts1," + sql += "%s, " % time_fun_join_1 + sql += "%s as asct1, " % time_fun_join_2 + sql += "t1.%s as asct11, " % random.choice(self.q_select) + sql += "t1.%s as asct12, " % random.choice(self.q_select) + sql += "t2.%s as asct13, " % random.choice(self.q_select) + sql += "t2.%s as asct14, " % random.choice(self.q_select) + sql += "t2.ts as ts2 from stable_1 t1 , stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.t_u_where) + sql += "and %s " % random.choice(self.t_u_or_where) + sql += "%s " % random.choice(self.order_u_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ");" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + elif (timelist == ['TIMEZONE']) or (timelist == ['CAST']) or (timelist == ['CAST_1']) or (timelist == ['CAST_2']) or (timelist == ['CAST_3']) or (timelist == ['CAST_4']): + sql = "select /*+ para_tables_sort() */ ts1,ts2 ,timediff(ts1,ts2), asct1,now() from ( select /*+ para_tables_sort() */ t1.ts as ts1," + sql += "%s, " % time_fun_join_1 + sql += "%s as asct1, " % time_fun_join_2 + sql += "t1.%s as asct11, " % random.choice(self.q_select) + sql += "t1.%s as asct12, " % random.choice(self.q_select) + sql += "t2.%s as asct13, " % random.choice(self.q_select) + sql += "t2.%s as asct14, " % random.choice(self.q_select) + sql += "t2.ts as ts2 from stable_1 t1 , stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.t_u_where) + sql += "and %s " % random.choice(self.t_u_or_where) + sql += "%s " % random.choice(self.order_u_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ");" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + elif (timelist == ['ELAPSED']) : + sql = "select /*+ para_tables_sort() */min(floor(asct1)),now() from ( select /*+ para_tables_sort() */" + sql += "%s as asct121, " % time_fun_join_1 + sql += "%s as asct1 " % time_fun_join_2 + sql += " from stable_1 t1 , stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.t_u_where) + sql += "and %s " % random.choice(self.t_u_or_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ");" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-13 as time_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (timelist == ['NOW','TODAY']) or (timelist == ['TIMETRUNCATE']) or (timelist == ['TO_ISO8601'])\ + or (timelist == ['TO_UNIXTIMESTAMP']) or (timelist == ['TIMEDIFF_1']) or (timelist == ['TIMEDIFF_2']): + sql = "select /*+ para_tables_sort() */ ts , timediff(%s,now)," % time_fun_2 + sql += "%s as asct1, " % time_fun_1 + sql += "%s as asct11, " % random.choice(self.q_select) + sql += "%s as asct12, " % random.choice(self.q_select) + sql += "%s as asct13, " % time_fun_2 + sql += "%s as asct122 " % random.choice(self.t_select) + sql += " from ( select /*+ para_tables_sort() */ * from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += "%s " % random.choice(self.order_where) + sql += ") ;" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + elif (timelist == ['TIMEZONE']) or (timelist == ['CAST']) or (timelist == ['CAST_1']) or (timelist == ['CAST_2']) or (timelist == ['CAST_3']) or (timelist == ['CAST_4']): + sql = "select /*+ para_tables_sort() */ ts ,now(),today(),timezone(), " + sql += "%s as asct1, " % time_fun_1 + sql += "%s as asct11, " % random.choice(self.q_select) + sql += "%s as asct12, " % random.choice(self.q_select) + sql += "%s as asct13, " % time_fun_2 + sql += "%s as asct122 " % random.choice(self.t_select) + sql += " from ( select /*+ para_tables_sort() */ * from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += "%s " % random.choice(self.order_where) + sql += ") ;" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + elif (timelist == ['ELAPSED']) : + sql = "select /*+ para_tables_sort() */ now(),today(),timezone(), " + sql += "%s as asct1, " % time_fun_1 + sql += "%s as asct12 " % time_fun_2 + sql += " from ( select /*+ para_tables_sort() */ * from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += "%s " % random.choice(self.order_where) + sql += ") ;" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-14 as time_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (timelist == ['NOW','TODAY']) or (timelist == ['TIMETRUNCATE']) or (timelist == ['TO_ISO8601'])\ + or (timelist == ['TO_UNIXTIMESTAMP']) or (timelist == ['TIMEDIFF_1']) or (timelist == ['TIMEDIFF_2']): + sql = "select /*+ para_tables_sort() */ ts , timediff(asct1,now),timediff(now,asct2) from ( select /*+ para_tables_sort() */ts ts ," + sql += "%s as asct1, " % time_fun_1 + sql += "%s as asct2" % time_fun_2 + sql += " from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.partiton_where) + sql += "%s " % random.choice(self.order_desc_where) + sql += "%s " % random.choice([self.limit_where[2] , self.limit_where[3]] ) + sql += " ) ;" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + elif (timelist == ['TIMEZONE']) or (timelist == ['CAST']) or (timelist == ['CAST_1']) or (timelist == ['CAST_2']) or (timelist == ['CAST_3']) or (timelist == ['CAST_4']): + sql = "select /*+ para_tables_sort() */ ts , (asct1),now(),(now()),asct2 from ( select /*+ para_tables_sort() */ts ts ," + sql += "%s as asct1, " % time_fun_1 + sql += "%s as asct2" % time_fun_2 + sql += " from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.partiton_where) + sql += "%s " % random.choice(self.order_desc_where) + sql += "%s " % random.choice([self.limit_where[2] , self.limit_where[3]] ) + sql += " ) ;" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + elif (timelist == ['ELAPSED']) : + sql = "select /*+ para_tables_sort() */ (asct1)*asct2,now(),(now()) from ( select /*+ para_tables_sort() */" + sql += "%s as asct1, " % time_fun_1 + sql += "%s as asct2" % time_fun_2 + sql += " from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.partiton_where) + sql += "%s " % random.choice([self.limit_where[2] , self.limit_where[3]] ) + sql += " ) ;" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-15 as time_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (timelist == ['NOW','TODAY']) or (timelist == ['TIMETRUNCATE']) or (timelist == ['TO_ISO8601'])\ + or (timelist == ['TO_UNIXTIMESTAMP']) or (timelist == ['TIMEDIFF_1']) or (timelist == ['TIMEDIFF_2']): + sql = "select /*+ para_tables_sort() */ ts1,ts ,timediff(ts1,ts), timediff(asct1,now),timediff(now,asct2) from ( select /*+ para_tables_sort() */ t1.ts as ts1," + sql += "%s as asct2, " % time_fun_join_1 + sql += "%s as asct1, " % time_fun_join_2 + sql += "t1.%s as asct11, " % random.choice(self.q_select) + sql += "t1.%s as asct12, " % random.choice(self.q_select) + sql += "t2.%s as asct13, " % random.choice(self.q_select) + sql += "t2.%s as asct14, " % random.choice(self.q_select) + sql += "t2.ts from stable_1 t1, stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.t_join_where) + sql += " and %s " % random.choice(self.qt_u_or_where) + sql += "%s " % random.choice(self.partiton_where_j) + sql += "%s " % random.choice(self.slimit1_where) + sql += ") " + sql += "%s " % random.choice(self.order_desc_where) + sql += "%s ;" % random.choice(self.limit_u_where) + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + elif (timelist == ['TIMEZONE']) or (timelist == ['CAST']) or (timelist == ['CAST_1']) or (timelist == ['CAST_2']) or (timelist == ['CAST_3']) or (timelist == ['CAST_4']): + sql = "select /*+ para_tables_sort() */ ts1,ts ,timediff(ts1,ts), asct1,(now()),(now()),asct2 ,now(),today(),timezone() from ( select /*+ para_tables_sort() */ t1.ts as ts1," + sql += "%s as asct2, " % time_fun_join_1 + sql += "%s as asct1, " % time_fun_join_2 + sql += "t1.%s as asct11, " % random.choice(self.q_select) + sql += "t1.%s as asct12, " % random.choice(self.q_select) + sql += "t2.%s as asct13, " % random.choice(self.q_select) + sql += "t2.%s as asct14, " % random.choice(self.q_select) + sql += "t2.ts from stable_1 t1, stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.t_join_where) + sql += " and %s " % random.choice(self.qt_u_or_where) + sql += "%s " % random.choice(self.partiton_where_j) + sql += "%s " % random.choice(self.slimit1_where) + sql += ") " + sql += "%s " % random.choice(self.order_desc_where) + sql += "%s ;" % random.choice(self.limit_u_where) + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + elif (timelist == ['ELAPSED']) : + sql = "select /*+ para_tables_sort() */ asct1,(now()),(now()),asct2 ,now(),today(),timezone() from ( select /*+ para_tables_sort() */" + sql += "%s as asct2, " % time_fun_join_1 + sql += "%s as asct1 " % time_fun_join_2 + sql += "from stable_1 t1, stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.t_join_where) + sql += " and %s " % random.choice(self.qt_u_or_where) + sql += "%s " % random.choice(self.partiton_where_j) + sql += "%s " % random.choice(self.slimit1_where) + sql += ") " + sql += "%s ;" % random.choice(self.limit_u_where) + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + #taos -f sql + startTime_taos_f = time.time() + print("taos -f %s sql start!" %timelist) + # taos_cmd1 = "taos -f %s/%s.sql" % (self.testcasePath,self.testcaseFilename) + # _ = subprocess.check_output(taos_cmd1, shell=True) + print("taos -f %s sql over!" %timelist) + endTime_taos_f = time.time() + print("taos_f total time %ds" % (endTime_taos_f - startTime_taos_f)) + + print("=========%s====over=============" %timelist) + + def function_before_26(self): + + print('=====================2.6 old function start ===========') + os.system("rm -rf %s/%s.sql" % (self.testcasePath,self.testcaseFilename)) + + self.dropandcreateDB_random("%s" %self.db_nest, 1) + + #1 select /*+ para_tables_sort() */* from (select /*+ para_tables_sort() */column form regular_table where <\>\in\and\or order by) + tdSql.query("select /*+ para_tables_sort() */1-1 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ tas from ( select /*+ para_tables_sort() */ " + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts as tas from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += ");" + tdLog.info(sql) + tdSql.query(sql,queryTimes=1) + self.explain_sql(sql) + + #1 outer union not support + tdSql.query("select /*+ para_tables_sort() */1-2 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ t1s from ( select /*+ para_tables_sort() */ " + sql += "%s, " % random.choice(self.s_r_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts as t1s from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += ") union " + sql += "select /*+ para_tables_sort() */ t2s from ( select /*+ para_tables_sort() */ " + sql += "%s, " % random.choice(self.s_r_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts as t2s from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += ")" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + #self.dropandcreateDB_random("%s" %db, 1) + tdSql.query("select /*+ para_tables_sort() */1-2 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ ts from ( select /*+ para_tables_sort() */ " + sql += "%s, " % random.choice(self.s_r_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += ") union all " + sql += "select /*+ para_tables_sort() */ ts from ( select /*+ para_tables_sort() */ " + sql += "%s, " % random.choice(self.s_r_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += ")" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + #1 inter union not support + tdSql.query("select /*+ para_tables_sort() */1-3 from stable_1;") + for i in range(self.fornum): + #sql = "select /*+ para_tables_sort() */ ts , * from ( select /*+ para_tables_sort() */ " + sql = "select /*+ para_tables_sort() */ ts from ( select /*+ para_tables_sort() */ " + sql += "%s, " % random.choice(self.s_r_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "" + sql += " union all select /*+ para_tables_sort() */ " + sql += "%s, " % random.choice(self.s_r_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts from regular_table_2 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += ")" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-3 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ ts from ( select /*+ para_tables_sort() */ " + sql += "%s, " % random.choice(self.s_r_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += " union all select /*+ para_tables_sort() */ " + sql += "%s, " % random.choice(self.s_r_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts from regular_table_2 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += ")" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + #join:select /*+ para_tables_sort() */* from (select /*+ para_tables_sort() */column form regular_table1,regular_table2 where t1.ts=t2.ts and <\>\in\and\or order by) + tdSql.query("select /*+ para_tables_sort() */1-4 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */ t1.ts as t1ts," + sql += "t1.%s as t11, " % random.choice(self.q_select) + sql += "t1.%s as t12, " % random.choice(self.q_select) + sql += "t2.%s as t21, " % random.choice(self.q_select) + sql += "t2.%s as t22, " % random.choice(self.q_select) + sql += "t2.ts as t2ts from regular_table_1 t1 , regular_table_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_where) + sql += "and %s " % random.choice(self.q_u_or_where) + sql += "%s " % random.choice(self.order_u_where) + sql += ");" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + #2 select /*+ para_tables_sort() */column from (select /*+ para_tables_sort() */* form regular_table ) where <\>\in\and\or order by + tdSql.query("select /*+ para_tables_sort() */2-1 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ ts ," + sql += "%s, " % random.choice(self.s_r_select) + sql += "%s " % random.choice(self.q_select) + sql += " from ( select /*+ para_tables_sort() */ * from regular_table_1 ) where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += " ;" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + #3 select /*+ para_tables_sort() */* from (select /*+ para_tables_sort() */column\tag form stable where <\>\in\and\or order by ) + #self.dropandcreateDB_random("%s" %db, 1) + tdSql.query("select /*+ para_tables_sort() */3-1 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */ " + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s, " % random.choice(self.q_select) + sql += "%s, " % random.choice(self.t_select) + sql += "ts from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += "%s " % random.choice(self.order_where) + sql += ") ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.explain_sql(sql) + tdSql.query("select /*+ para_tables_sort() */3-1 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ ts, " + sql += "%s " % random.choice(self.s_r_select) + sql += "from ( select /*+ para_tables_sort() */ " + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s, " % random.choice(self.q_select) + sql += "%s, " % random.choice(self.t_select) + sql += "ts from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += "%s " % random.choice(self.order_where) + sql += ") ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.explain_sql(sql) + + # select /*+ para_tables_sort() */ts,* from (select /*+ para_tables_sort() */column\tag form stable1,stable2 where t1.ts = t2.ts and <\>\in\and\or order by ) + #self.dropandcreateDB_random("%s" %db, 1) + tdSql.query("select /*+ para_tables_sort() */3-2 from stable_1;") + for i in range(self.fornum): + #sql = "select /*+ para_tables_sort() */ ts , * from ( select /*+ para_tables_sort() */ t1.ts as t1ts , " + sql = "select /*+ para_tables_sort() */ t1ts , t2ts from ( select /*+ para_tables_sort() */ t1.ts as t1ts , " + sql += "t1.%s as t11, " % random.choice(self.t_select) + sql += "t1.%s as t12, " % random.choice(self.q_select) + sql += "t2.%s as t13, " % random.choice(self.t_select) + sql += "t2.%s as t14, " % random.choice(self.q_select) + sql += "t2.ts as t2ts from stable_1 t1 , stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.t_join_where) + sql += "%s " % random.choice(self.order_u_where) + sql += ") ;" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + #3 outer union not support + + tdSql.query("select /*+ para_tables_sort() */3-3 from stable_1;") + for i in range(self.fornum): + #sql = "select /*+ para_tables_sort() */ ts , * from ( select /*+ para_tables_sort() */ " + sql = "select /*+ para_tables_sort() */ ts1 from ( select /*+ para_tables_sort() */ " + sql += "%s, " % random.choice(self.s_r_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts as ts1 from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += ") union " + sql += "select /*+ para_tables_sort() */ ts2 from ( select /*+ para_tables_sort() */ " + sql += "%s, " % random.choice(self.s_r_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts as ts2 from stable_2 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += ")" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ ts1 from ( select /*+ para_tables_sort() */ " + sql += "%s, " % random.choice(self.s_r_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts as ts1 from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += ") union all " + sql += "select /*+ para_tables_sort() */ ts2 from ( select /*+ para_tables_sort() */ " + sql += "%s, " % random.choice(self.s_r_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts as ts2 from stable_2 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += ")" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + #3 inter union not support + tdSql.query("select /*+ para_tables_sort() */3-4 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */ " + sql += "%s, " % random.choice(self.s_r_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += " %s " % random.choice(self.unionall_or_union) + sql += " select /*+ para_tables_sort() */ " + sql += "%s, " % random.choice(self.s_r_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts from stable_2 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += ")" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + #join:select /*+ para_tables_sort() */* from (select /*+ para_tables_sort() */column form stable1,stable2 where t1.ts=t2.ts and <\>\in\and\or order by) + tdSql.query("select /*+ para_tables_sort() */3-5 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */ t1.ts as t1ts," + sql += "t1.%s as t11, " % random.choice(self.q_select) + sql += "t1.%s as t12, " % random.choice(self.q_select) + sql += "t2.%s as t21, " % random.choice(self.q_select) + sql += "t2.%s as t22, " % random.choice(self.q_select) + sql += "t2.ts as t2ts from stable_1 t1 , stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.t_u_where) + sql += "%s " % random.choice(self.order_u_where) + sql += ");" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */3-6 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */ t1.ts as t1ts ," + sql += "t1.%s as t11, " % random.choice(self.q_select) + sql += "t1.%s as t12, " % random.choice(self.q_select) + sql += "t2.%s as t21, " % random.choice(self.q_select) + sql += "t2.%s as t22, " % random.choice(self.q_select) + sql += "t2.ts as t2ts from stable_1 t1 , stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.t_u_or_where) + sql += "%s " % random.choice(self.order_u_where) + sql += ");" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + #4 select /*+ para_tables_sort() */column from (select /*+ para_tables_sort() */* form stable where <\>\in\and\or order by ) + tdSql.query("select /*+ para_tables_sort() */4-1 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ ts , " + sql += "%s as t11, " % random.choice(self.q_select) + sql += "%s as t12, " % random.choice(self.q_select) + sql += "%s " % random.choice(self.t_select) + sql += " from ( select /*+ para_tables_sort() */ * from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += "%s " % random.choice(self.order_where) + sql += ") ;" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + #5 select /*+ para_tables_sort() */distinct column\tag from (select /*+ para_tables_sort() */* form stable where <\>\in\and\or order by limit offset ) + tdSql.query("select /*+ para_tables_sort() */5-1 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ " + sql += "%s " % random.choice(self.dqt_select) + sql += " from ( select /*+ para_tables_sort() */ * from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += "%s " % random.choice(self.order_where) + sql += ") ;" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + #5-1 select /*+ para_tables_sort() */distinct column\tag from (select /*+ para_tables_sort() */calc form stable where <\>\in\and\or order by limit offset ) + tdSql.query("select /*+ para_tables_sort() */5-2 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ distinct c5_1 " + sql += " from ( select /*+ para_tables_sort() */ " + sql += "%s " % random.choice(self.calc_select_in_ts) + sql += " as c5_1 from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += ") ;" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + #6 select /*+ para_tables_sort() */* from (select /*+ para_tables_sort() */distinct(tag) form stable where <\>\in\and\or order by limit ) + tdSql.query("select /*+ para_tables_sort() */6-1 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */ " + sql += "%s " % random.choice(self.dt_select) + sql += " from stable_1 where " + sql += "%s ) ;" % random.choice(self.qt_where) + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + #7 select /*+ para_tables_sort() */* from (select /*+ para_tables_sort() */distinct(tag) form stable where <\>\in\and\or order by limit ) + tdSql.query("select /*+ para_tables_sort() */7-1 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */ " + sql += "%s " % random.choice(self.dq_select) + sql += " from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += "%s " % random.choice([self.limit_where[0] , self.limit_where[1]] ) + sql += ") ;" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + #calc_select,TWA/Diff/Derivative/Irate are not allowed to apply to super table directly + #8 select /*+ para_tables_sort() */* from (select /*+ para_tables_sort() */ts,calc form ragular_table where <\>\in\and\or order by ) + tdSql.query("select /*+ para_tables_sort() */8-1 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */ ts ," + sql += "%s " % random.choice(self.calc_select_support_ts) + sql += "from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ") ;" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + tdSql.query("select /*+ para_tables_sort() */8-1 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */ " + sql += "%s " % random.choice(self.calc_select_not_support_ts) + sql += "from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ") ;" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */ " + sql += "%s " % random.choice(self.calc_select_in_ts) + sql += "from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ") ;" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */8-2 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */t1.ts, " + sql += "%s " % random.choice(self.calc_select_in_support_ts_j) + sql += "from regular_table_1 t1, regular_table_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_where) + sql += "%s " % random.choice(self.order_u_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ") ;" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */ " + sql += "%s " % random.choice(self.calc_select_in_not_support_ts_j) + sql += "from regular_table_1 t1, regular_table_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ") ;" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + #9 select /*+ para_tables_sort() */* from (select /*+ para_tables_sort() */ts,calc form stable where <\>\in\and\or order by ) + # self.dropandcreateDB_random("%s" %db, 1) + tdSql.query("select /*+ para_tables_sort() */9-1 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */ " + sql += "%s " % random.choice(self.calc_select_not_support_ts) + sql += "from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ") ;" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + tdSql.query("select /*+ para_tables_sort() */9-2 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */ ts ," + sql += "%s " % random.choice(self.calc_select_support_ts) + sql += "from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += "%s " % random.choice(self.order_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ") ;" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */9-3 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */ " + sql += "%s " % random.choice(self.calc_select_in_not_support_ts_j) + sql += "from stable_1 t1, stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.t_join_where) + sql += " and %s " % random.choice(self.qt_u_or_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ") ;" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + tdSql.query("select /*+ para_tables_sort() */9-4 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */ t1.ts," + sql += "%s " % random.choice(self.calc_select_in_support_ts_j) + sql += "from stable_1 t1, stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.t_join_where) + sql += " and %s " % random.choice(self.qt_u_or_where) + sql += "%s " % random.choice(self.order_u_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ") ;" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + #10 select /*+ para_tables_sort() */calc from (select /*+ para_tables_sort() */* form regualr_table where <\>\in\and\or order by ) + tdSql.query("select /*+ para_tables_sort() */10-1 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ " + sql += "%s " % random.choice(self.calc_select_in_ts) + sql += "as calc10_1 from ( select /*+ para_tables_sort() */* from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_desc_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ") ;" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + #10-1 select /*+ para_tables_sort() */calc from (select /*+ para_tables_sort() */* form regualr_table where <\>\in\and\or order by ) + tdSql.query("select /*+ para_tables_sort() */10-2 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ " + sql += "%s " % random.choice(self.calc_select_all) + sql += "as calc10_2 from ( select /*+ para_tables_sort() */* from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_desc_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ") ;" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + #10-2 select /*+ para_tables_sort() */calc from (select /*+ para_tables_sort() */* form regualr_tables where <\>\in\and\or order by ) + tdSql.query("select /*+ para_tables_sort() */10-3 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ " + sql += "count(*) as calc10_3 " + sql += " from ( select /*+ para_tables_sort() */t1.ts as t11, t2.ts as t22 from regular_table_1 t1, regular_table_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_where) + sql += " and %s " % random.choice(self.q_u_or_where) + sql += "%s " % random.choice(self.order_u_where) + sql += "%s " % random.choice(self.limit_u_where) + sql += ") " + sql += "%s ;" % random.choice(self.limit1_where) + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + #11 select /*+ para_tables_sort() */calc from (select /*+ para_tables_sort() */* form stable where <\>\in\and\or order by limit ) + tdSql.query("select /*+ para_tables_sort() */11-1 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ " + sql += "%s " % random.choice(self.calc_select_in_ts) + sql += "as calc11_1 from ( select /*+ para_tables_sort() */* from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += "%s " % random.choice(self.order_desc_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ") ;" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + #11-1 select /*+ para_tables_sort() */calc from (select /*+ para_tables_sort() */* form stable where <\>\in\and\or order by limit ) + tdSql.query("select /*+ para_tables_sort() */11-2 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ " + sql += "%s " % random.choice(self.calc_select_all) + sql += "as calc11_1 from ( select /*+ para_tables_sort() */* from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += "%s " % random.choice(self.order_desc_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ") ;" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + #12 select /*+ para_tables_sort() */calc-diff from (select /*+ para_tables_sort() */* form regualr_table where <\>\in\and\or order by limit ) + tdSql.query("select /*+ para_tables_sort() */12-1 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ " + sql += "%s " % random.choice(self.calc_calculate_regular) + sql += " from ( select /*+ para_tables_sort() */* from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_desc_where) + sql += "%s " % random.choice([self.limit_where[2] , self.limit_where[3]] ) + sql += ") ;" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + #12-1 select /*+ para_tables_sort() */calc-diff from (select /*+ para_tables_sort() */* form stable where <\>\in\and\or order by limit ) + tdSql.query("select /*+ para_tables_sort() */12-3 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */" + sql += "%s " % random.choice(self.calc_calculate_regular) + sql += " from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.partiton_where) + sql += ") " + sql += "%s " % random.choice([self.limit_where[2] , self.limit_where[3]] ) + sql += " ;" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */12-4 from stable_1;") + #join query does not support group by + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */" + sql += "%s " % random.choice(self.calc_calculate_regular_j) + sql += " from stable_1 t1, stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.t_join_where) + sql += "%s " % random.choice(self.partiton_where_j) + sql += ") " + sql += "%s " % random.choice([self.limit_where[2] , self.limit_where[3]] ) + sql += " ;" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */12-5 from stable_1;") + #join query does not support group by + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */" + sql += "%s " % random.choice(self.calc_calculate_regular_j) + sql += " from stable_1 t1, stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.qt_u_or_where) + sql += ") " + sql += "%s " % random.choice([self.limit_where[2] , self.limit_where[3]] ) + sql += " ;" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + #13 select /*+ para_tables_sort() */calc-diff as diffns from (select /*+ para_tables_sort() */* form stable where <\>\in\and\or order by limit ) + tdSql.query("select /*+ para_tables_sort() */13-1 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ " + sql += "%s " % random.choice(self.calc_calculate_regular) + sql += " as calc13_1 from ( select /*+ para_tables_sort() */* from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += "%s " % random.choice(self.orders_desc_where) + sql += "%s " % random.choice([self.limit_where[2] , self.limit_where[3]] ) + sql += ") ;" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + #14 select /*+ para_tables_sort() */* from (select /*+ para_tables_sort() */calc_aggregate_alls as agg from stable where <\>\in\and\or group by order by slimit soffset ) + tdSql.query("select /*+ para_tables_sort() */14-1 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */" + sql += "%s as calc14_1, " % random.choice(self.calc_aggregate_all) + sql += "%s as calc14_2, " % random.choice(self.calc_aggregate_all) + sql += "%s " % random.choice(self.calc_aggregate_all) + sql += " as calc14_3 from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += "%s " % random.choice(self.group_where) + sql += ") ;" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + # error group by in out query + tdSql.query("select /*+ para_tables_sort() */14-2 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */" + sql += "%s as calc14_1, " % random.choice(self.calc_aggregate_all) + sql += "%s as calc14_2, " % random.choice(self.calc_aggregate_all) + sql += "%s " % random.choice(self.calc_aggregate_all) + sql += " as calc14_3 from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += "%s " % random.choice(self.partiton_where_regular) + sql += "%s " % random.choice(self.slimit1_where) + sql += ") " + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + #14-2 select /*+ para_tables_sort() */* from (select /*+ para_tables_sort() */calc_aggregate_all_js as agg from stables where <\>\in\and\or group by order by slimit soffset ) + tdSql.query("select /*+ para_tables_sort() */14-3 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */" + sql += "%s as calc14_1, " % random.choice(self.calc_aggregate_all_j) + sql += "%s as calc14_2, " % random.choice(self.calc_aggregate_all_j) + sql += "%s " % random.choice(self.calc_aggregate_all_j) + sql += " as calc14_3 from stable_1 t1, stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.t_join_where) + sql += "%s " % random.choice(self.partiton_where_j) + sql += "%s " % random.choice(self.slimit1_where) + sql += ") " + sql += "%s ;" % random.choice(self.limit_u_where) + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */14-4 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */" + sql += "%s as calc14_1, " % random.choice(self.calc_aggregate_all_j) + sql += "%s as calc14_2, " % random.choice(self.calc_aggregate_all_j) + sql += "%s " % random.choice(self.calc_aggregate_all_j) + sql += " as calc14_3 from stable_1 t1, stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.qt_u_or_where) + sql += "%s " % random.choice(self.partiton_where_j) + sql += "%s " % random.choice(self.slimit1_where) + sql += ") " + sql += "%s ;" % random.choice(self.limit_u_where) + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + #15 select /*+ para_tables_sort() */* from (select /*+ para_tables_sort() */calc_aggregate_regulars as agg from regular_table where <\>\in\and\or order by slimit soffset ) + tdSql.query("select /*+ para_tables_sort() */15-1 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */" + sql += "%s as calc15_1, " % random.choice(self.calc_aggregate_regular) + sql += "%s as calc15_2, " % random.choice(self.calc_aggregate_regular) + sql += "%s " % random.choice(self.calc_aggregate_regular) + sql += " as calc15_3 from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.group_where_regular) + sql += ") ;" + tdLog.info(sql) + self.data_check(sql,mark='15-1') + + tdSql.query("select /*+ para_tables_sort() */15-2 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */" + sql += "%s as calc15_2 " % random.choice(self.calc_aggregate_regular_j) + sql += "from regular_table_1 t1, regular_table_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_where) + sql += "%s " % random.choice(self.group_where_regular_j) + sql += "%s " % random.choice(self.limit_u_where) + sql += ") " + sql += "%s ;" % random.choice(self.limit_u_where) + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */15-2.2 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */" + sql += "%s as calc15_1, " % random.choice(self.calc_aggregate_regular_j) + sql += "%s as calc15_2, " % random.choice(self.calc_aggregate_regular_j) + sql += "%s " % random.choice(self.calc_aggregate_regular_j) + sql += " as calc15_3 from regular_table_1 t1, regular_table_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_or_where) + sql += "%s " % random.choice(self.group_where_regular_j) + sql += "%s " % random.choice(self.limit_u_where) + sql += ") " + sql += "%s ;" % random.choice(self.limit_u_where) + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */15-3 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */" + sql += "%s as calc15_1, " % random.choice(self.calc_aggregate_groupbytbname) + sql += "%s as calc15_2, " % random.choice(self.calc_aggregate_groupbytbname) + sql += "%s " % random.choice(self.calc_aggregate_groupbytbname) + sql += " as calc15_3 from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.group_only_where) + sql += "%s " % random.choice(self.having_support) + sql += ") " + sql += "order by calc15_1 " + sql += "%s " % random.choice(self.limit_where) + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */15-4 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */" + sql += "%s as calc15_1, " % random.choice(self.calc_aggregate_groupbytbname_j) + sql += "%s as calc15_2, " % random.choice(self.calc_aggregate_groupbytbname_j) + sql += "%s " % random.choice(self.calc_aggregate_groupbytbname_j) + sql += " as calc15_3 from stable_1 t1, stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.t_join_where) + sql += "%s " % random.choice(self.group_only_where_j) + sql += "%s " % random.choice(self.having_support_j) + sql += ") " + sql += "order by calc15_1 " + sql += "%s " % random.choice(self.limit_u_where) + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */15-5 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */" + sql += "%s as calc15_1, " % random.choice(self.calc_aggregate_groupbytbname) + sql += "%s as calc15_2, " % random.choice(self.calc_aggregate_groupbytbname) + sql += "%s " % random.choice(self.calc_aggregate_groupbytbname) + sql += " as calc15_3 from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.group_where) + sql += ") " + sql += "order by calc15_1 " + sql += "%s " % random.choice(self.limit_where) + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + #16 select /*+ para_tables_sort() */* from (select /*+ para_tables_sort() */calc_aggregate_regulars as agg from regular_table where <\>\in\and\or order by limit offset ) + tdSql.query("select /*+ para_tables_sort() */16-1 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */ " + sql += "%s as calc16_0 , " % random.choice(self.calc_calculate_all) + sql += "%s as calc16_1 , " % random.choice(self.calc_aggregate_all) + sql += "%s as calc16_2 " % random.choice(self.calc_select_in) + sql += " from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.group_where) + sql += ") " + sql += "order by calc16_0 " + sql += "%s " % random.choice(self.limit1_where) + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */16-2 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */ " + sql += "%s as calc16_0 " % random.choice(self.calc_calculate_all_j) + sql += ", %s as calc16_1 " % random.choice(self.calc_aggregate_all_j) + sql += " from stable_1 t1, stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.t_join_where) + sql += ") " + sql += "order by calc16_0 " + sql += "%s " % random.choice(self.limit1_where) + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */16-2.2 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */ " + sql += "%s as calc16_0 " % random.choice(self.calc_calculate_all_j) + sql += ", %s as calc16_1 " % random.choice(self.calc_aggregate_all_j) + sql += " from stable_1 t1, stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.qt_u_or_where) + sql += ") " + sql += "order by calc16_0 " + sql += "%s " % random.choice(self.limit1_where) + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */16-3 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */ " + sql += "%s as calc16_1 " % random.choice(self.calc_calculate_regular) + sql += " from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "limit 2 ) " + sql += "%s " % random.choice(self.limit1_where) + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */16-4 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */ " + sql += "%s as calc16_1 " % random.choice(self.calc_calculate_regular_j) + sql += " from regular_table_1 t1, regular_table_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_where) + sql += "limit 2 ) " + sql += "%s " % random.choice(self.limit1_where) + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */16-4.2 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */ " + sql += "%s as calc16_1 " % random.choice(self.calc_calculate_regular_j) + sql += " from regular_table_1 t1, regular_table_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_or_where) + sql += "limit 2 ) " + sql += "%s " % random.choice(self.limit1_where) + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */16-5 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */ " + sql += "%s as calc16_1 , " % random.choice(self.calc_calculate_all) + sql += "%s as calc16_2 , " % random.choice(self.calc_calculate_all) + sql += "%s as calc16_3 " % random.choice(self.calc_calculate_all) + sql += " from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.group_where) + sql += ") " + sql += "order by calc16_1 " + sql += "%s " % random.choice(self.limit1_where) + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */16-6 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */ " + sql += "%s as calc16_1 " % random.choice(self.calc_calculate_groupbytbname) + sql += " from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.partiton_where) + sql += "limit 2 ) " + sql += "%s " % random.choice(self.limit1_where) + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */16-7 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */ " + sql += "%s as calc16_1 " % random.choice(self.calc_calculate_groupbytbname_j) + sql += " from stable_1 t1, stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.t_join_where) + sql += "limit 2 ) " + sql += "%s " % random.choice(self.limit1_where) + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */16-8 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */ " + sql += "%s as calc16_1 " % random.choice(self.calc_calculate_groupbytbname_j) + sql += " from stable_1 t1, stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.qt_u_or_where) + sql += "limit 2 ) " + sql += "%s " % random.choice(self.limit1_where) + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + #17 select /*+ para_tables_sort() */apercentile from (select /*+ para_tables_sort() */calc_aggregate_alls form regualr_table or stable where <\>\in\and\or interval_sliding group by having order by limit offset )interval_sliding + tdSql.query("select /*+ para_tables_sort() */17-1 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ apercentile(cal17_0, %d)/10 ,apercentile(cal17_1, %d)/1000 ,apercentile(cal17_2, %d)*10+%d from ( select /*+ para_tables_sort() */ " %(random.randint(0,100) , random.randint(0,100) , random.randint(0,100) ,random.randint(-1000,1000)) + sql += "%s as cal17_0 , " % random.choice(self.calc_calculate_all) + sql += "%s as cal17_1 ," % random.choice(self.calc_aggregate_all) + sql += "%s as cal17_2 " % random.choice(self.calc_aggregate_all) + sql += " from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += "%s " % random.choice(self.partiton_where) + sql += "%s " % random.choice(self.interval_sliding) + sql += "%s " % random.choice(self.limit1_where) + sql += ") " + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */17-2 from stable_1;") + for i in range(self.fornum): + #this is having_support , but tag-select /*+ para_tables_sort() */cannot mix with last_row,other select /*+ para_tables_sort() */can + sql = "select /*+ para_tables_sort() */ apercentile(cal17_0, %d)/10 ,apercentile(cal17_1, %d)/1000 ,apercentile(cal17_2, %d)*10+%d from ( select /*+ para_tables_sort() */ " %(random.randint(0,100) , random.randint(0,100) , random.randint(0,100) ,random.randint(-1000,1000)) + sql += "%s as cal17_0 , " % random.choice(self.calc_calculate_all_j) + sql += "%s as cal17_1 ," % random.choice(self.calc_aggregate_all_j) + sql += "%s as cal17_2 " % random.choice(self.calc_aggregate_all_j) + sql += " from stable_1 t1, stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.t_join_where) + sql += "%s " % random.choice(self.interval_sliding) + sql += "%s " % random.choice(self.limit_u_where) + sql += ") " + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */17-2.2 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ apercentile(cal17_0, %d)/10 ,apercentile(cal17_1, %d)/1000 ,apercentile(cal17_2, %d)*10+%d from ( select /*+ para_tables_sort() */ " %(random.randint(0,100) , random.randint(0,100) , random.randint(0,100) ,random.randint(-1000,1000)) + sql += "%s as cal17_0 , " % random.choice(self.calc_calculate_all_j) + sql += "%s as cal17_1 ," % random.choice(self.calc_aggregate_all_j) + sql += "%s as cal17_2 " % random.choice(self.calc_aggregate_all_j) + sql += " from stable_1 t1, stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.qt_u_or_where) + sql += "%s " % random.choice(self.interval_sliding) + sql += "%s " % random.choice(self.limit_u_where) + sql += ") " + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */17-3 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ apercentile(cal17_1, %d)/1000 ,apercentile(cal17_2, %d)*10+%d from ( select /*+ para_tables_sort() */ " %(random.randint(0,100) , random.randint(0,100) ,random.randint(-1000,1000)) + sql += "%s as cal17_1 ," % random.choice(self.calc_aggregate_all) + sql += "%s as cal17_2 " % random.choice(self.calc_aggregate_all) + sql += " from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.partiton_where) + sql += "%s " % random.choice(self.interval_sliding) + sql += "%s " % random.choice(self.limit1_where) + sql += ") " + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */17-4 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ apercentile(cal17_1, %d)/1000 ,apercentile(cal17_2, %d)*10+%d from ( select /*+ para_tables_sort() */ " %(random.randint(0,100) , random.randint(0,100) ,random.randint(-1000,1000)) + sql += "%s as cal17_1 ," % random.choice(self.calc_aggregate_all_j) + sql += "%s as cal17_2 " % random.choice(self.calc_aggregate_all_j) + sql += " from stable_1 t1, stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.t_join_where) + sql += "%s " % random.choice(self.interval_sliding) + sql += "%s " % random.choice(self.limit_u_where) + sql += ") " + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */17-4.2 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ apercentile(cal17_1, %d)/1000 ,apercentile(cal17_2, %d)*10+%d from ( select /*+ para_tables_sort() */ " %(random.randint(0,100) , random.randint(0,100) ,random.randint(-1000,1000)) + sql += "%s as cal17_1 ," % random.choice(self.calc_aggregate_all_j) + sql += "%s as cal17_2 " % random.choice(self.calc_aggregate_all_j) + sql += " from stable_1 t1, stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.qt_u_or_where) + sql += "%s " % random.choice(self.interval_sliding) + sql += "%s " % random.choice(self.limit_u_where) + sql += ") " + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */17-5 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ apercentile(cal17_1, %d)/1000 ,apercentile(cal17_2, %d)*10+%d from ( select /*+ para_tables_sort() */ " %(random.randint(0,100) , random.randint(0,100) ,random.randint(-1000,1000)) + sql += "%s as cal17_1 ," % random.choice(self.calc_aggregate_all) + sql += "%s as cal17_2 " % random.choice(self.calc_aggregate_all) + sql += " from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += "%s " % random.choice(self.partiton_where) + sql += "%s " % random.choice(self.interval_sliding) + sql += "%s " % random.choice(self.limit1_where) + sql += ") " + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */17-6 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ apercentile(cal17_1, %d)/1000 ,apercentile(cal17_2, %d)*10+%d from ( select /*+ para_tables_sort() */ " %(random.randint(0,100) , random.randint(0,100) ,random.randint(-1000,1000)) + sql += "%s as cal17_1 ," % random.choice(self.calc_aggregate_all) + sql += "%s as cal17_2 " % random.choice(self.calc_aggregate_all) + sql += " from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.interval_sliding) + sql += "%s " % random.choice(self.limit1_where) + sql += ") " + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */17-7 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ apercentile(cal17_1, %d)/1000 ,apercentile(cal17_2, %d)*10+%d from ( select /*+ para_tables_sort() */ " %(random.randint(0,100) , random.randint(0,100) ,random.randint(-1000,1000)) + sql += "%s as cal17_1 ," % random.choice(self.calc_aggregate_all_j) + sql += "%s as cal17_2 " % random.choice(self.calc_aggregate_all_j) + sql += " from stable_1_1 t1, stable_1_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_where) + sql += "%s " % random.choice(self.interval_sliding) + sql += "%s " % random.choice(self.limit1_where) + sql += ") " + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */17-7.2 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ apercentile(cal17_1, %d)/1000 ,apercentile(cal17_2, %d)*10+%d from ( select /*+ para_tables_sort() */ " %(random.randint(0,100) , random.randint(0,100) ,random.randint(-1000,1000)) + sql += "%s as cal17_1 ," % random.choice(self.calc_aggregate_all_j) + sql += "%s as cal17_2 " % random.choice(self.calc_aggregate_all_j) + sql += " from stable_1_1 t1, stable_1_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_or_where) + sql += "%s " % random.choice(self.interval_sliding) + sql += "%s " % random.choice(self.limit1_where) + sql += ") " + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */17-8 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ apercentile(cal17_1, %d)/1000 ,apercentile(cal17_2, %d)*10+%d from ( select /*+ para_tables_sort() */ " %(random.randint(0,100) , random.randint(0,100) ,random.randint(-1000,1000)) + sql += "%s as cal17_1 ," % random.choice(self.calc_aggregate_all) + sql += "%s as cal17_2 " % random.choice(self.calc_aggregate_all) + sql += " from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.interval_sliding) + sql += "%s " % random.choice(self.limit1_where) + sql += ") " + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */17-9 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ apercentile(cal17_1, %d)/1000 ,apercentile(cal17_2, %d)*10+%d from ( select /*+ para_tables_sort() */ " %(random.randint(0,100) , random.randint(0,100) ,random.randint(-1000,1000)) + sql += "%s as cal17_1 ," % random.choice(self.calc_aggregate_all_j) + sql += "%s as cal17_2 " % random.choice(self.calc_aggregate_all_j) + sql += " from regular_table_1 t1, regular_table_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_where) + sql += "%s " % random.choice(self.interval_sliding) + sql += "%s " % random.choice(self.limit_u_where) + sql += ") " + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */17-10 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ apercentile(cal17_1, %d)/1000 ,apercentile(cal17_2, %d)*10+%d from ( select /*+ para_tables_sort() */ " %(random.randint(0,100) , random.randint(0,100) ,random.randint(-1000,1000)) + sql += "%s as cal17_1 ," % random.choice(self.calc_aggregate_all_j) + sql += "%s as cal17_2 " % random.choice(self.calc_aggregate_all_j) + sql += " from regular_table_1 t1, regular_table_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_or_where) + sql += "%s " % random.choice(self.interval_sliding) + sql += "%s " % random.choice(self.limit_u_where) + sql += ") " + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */18-1 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ apercentile(cal18_1, %d)/1000 ,apercentile(cal18_2, %d)*10+%d from ( select /*+ para_tables_sort() */ " %(random.randint(0,100) , random.randint(0,100) ,random.randint(-1000,1000)) + sql += "%s as cal18_1 ," % random.choice(self.calc_aggregate_all) + sql += "%s as cal18_2 " % random.choice(self.calc_aggregate_all) + sql += " from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.session_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ") " + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */18-2 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ apercentile(cal18_1, %d)/1000 ,apercentile(cal18_2, %d)*10+%d from ( select /*+ para_tables_sort() */ " %(random.randint(0,100) , random.randint(0,100) ,random.randint(-1000,1000)) + sql += "%s as cal18_1 ," % random.choice(self.calc_aggregate_all_j) + sql += "%s as cal18_2 " % random.choice(self.calc_aggregate_all_j) + sql += " from regular_table_1 t1, regular_table_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_where) + sql += "%s " % random.choice(self.session_u_where) + sql += "%s " % random.choice(self.limit_u_where) + sql += ") " + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */18-2.2 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ apercentile(cal18_1, %d)/1000 ,apercentile(cal18_2, %d)*10+%d from ( select /*+ para_tables_sort() */ " %(random.randint(0,100) , random.randint(0,100) ,random.randint(-1000,1000)) + sql += "%s as cal18_1 ," % random.choice(self.calc_aggregate_all_j) + sql += "%s as cal18_2 " % random.choice(self.calc_aggregate_all_j) + sql += " from regular_table_1 t1, regular_table_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_or_where) + sql += "%s " % random.choice(self.session_u_where) + sql += "%s " % random.choice(self.limit_u_where) + sql += ") " + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */18-3 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ apercentile(cal18_1, %d)/1000 ,apercentile(cal18_2, %d)*10+%d from ( select /*+ para_tables_sort() */ " %(random.randint(0,100) , random.randint(0,100) ,random.randint(-1000,1000)) + sql += "%s as cal18_1 ," % random.choice(self.calc_aggregate_all) + sql += "%s as cal18_2 " % random.choice(self.calc_aggregate_all) + sql += " from stable_1_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.session_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ") " + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */18-4 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ apercentile(cal18_1, %d)/1000 ,apercentile(cal18_2, %d)*10+%d from ( select /*+ para_tables_sort() */ " %(random.randint(0,100) , random.randint(0,100) ,random.randint(-1000,1000)) + sql += "%s as cal18_1 ," % random.choice(self.calc_aggregate_all_j) + sql += "%s as cal18_2 " % random.choice(self.calc_aggregate_all_j) + sql += " from stable_1_1 t1, regular_table_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_where) + sql += "%s " % random.choice(self.session_u_where) + sql += "%s " % random.choice(self.limit_u_where) + sql += ") " + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */18-4.2 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ apercentile(cal18_1, %d)/1000 ,apercentile(cal18_2, %d)*10+%d from ( select /*+ para_tables_sort() */ " %(random.randint(0,100) , random.randint(0,100) ,random.randint(-1000,1000)) + sql += "%s as cal18_1 ," % random.choice(self.calc_aggregate_all_j) + sql += "%s as cal18_2 " % random.choice(self.calc_aggregate_all_j) + sql += " from stable_1_1 t1, regular_table_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_or_where) + sql += "%s " % random.choice(self.session_u_where) + sql += "%s " % random.choice(self.limit_u_where) + sql += ") " + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */18-5 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ apercentile(cal18_1, %d)/1000 ,apercentile(cal18_2, %d)*10+%d from ( select /*+ para_tables_sort() */ " %(random.randint(0,100) , random.randint(0,100) ,random.randint(-1000,1000)) + sql += "%s as cal18_1 ," % random.choice(self.calc_aggregate_all) + sql += "%s as cal18_2 " % random.choice(self.calc_aggregate_all) + sql += " from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.session_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ") " + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */18-6 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ apercentile(cal18_1, %d)/1000 ,apercentile(cal18_2, %d)*10+%d from ( select /*+ para_tables_sort() */ " %(random.randint(0,100) , random.randint(0,100) ,random.randint(-1000,1000)) + sql += "%s as cal18_1 ," % random.choice(self.calc_aggregate_all_j) + sql += "%s as cal18_2 " % random.choice(self.calc_aggregate_all_j) + sql += " from stable_1 t1, stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.t_join_where) + sql += "%s " % random.choice(self.session_u_where) + sql += "%s " % random.choice(self.limit_u_where) + sql += ") " + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */18-7 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ apercentile(cal18_1, %d)/1000 ,apercentile(cal18_2, %d)*10+%d from ( select /*+ para_tables_sort() */ " %(random.randint(0,100) , random.randint(0,100) ,random.randint(-1000,1000)) + sql += "%s as cal18_1 ," % random.choice(self.calc_aggregate_all_j) + sql += "%s as cal18_2 " % random.choice(self.calc_aggregate_all_j) + sql += " from stable_1 t1, stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.qt_u_or_where) + sql += "%s " % random.choice(self.session_u_where) + sql += "%s " % random.choice(self.limit_u_where) + sql += ") " + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + #19 select /*+ para_tables_sort() */apercentile from (select /*+ para_tables_sort() */calc_aggregate_alls form regualr_table or stable where <\>\in\and\or session order by limit )interval_sliding + tdSql.query("select /*+ para_tables_sort() */19-1 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ apercentile(cal19_1, %d)/1000 ,apercentile(cal19_2, %d)*10+%d from ( select /*+ para_tables_sort() */ " %(random.randint(0,100) , random.randint(0,100) ,random.randint(-1000,1000)) + sql += "%s as cal19_1 ," % random.choice(self.calc_aggregate_all) + sql += "%s as cal19_2 " % random.choice(self.calc_aggregate_all) + sql += " from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.state_window) + sql += "%s " % random.choice(self.limit1_where) + sql += ") " + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */19-2 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ apercentile(cal19_1, %d)/1000 ,apercentile(cal19_2, %d)*10+%d from ( select /*+ para_tables_sort() */ " %(random.randint(0,100) , random.randint(0,100) ,random.randint(-1000,1000)) + sql += "%s as cal19_1 ," % random.choice(self.calc_aggregate_all_j) + sql += "%s as cal19_2 " % random.choice(self.calc_aggregate_all_j) + sql += " from regular_table_1 t1, regular_table_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_where) + sql += "%s " % random.choice(self.state_u_window) + sql += "%s " % random.choice(self.limit_u_where) + sql += ") " + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */19-2.2 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ apercentile(cal19_1, %d)/1000 ,apercentile(cal19_2, %d)*10+%d from ( select /*+ para_tables_sort() */ " %(random.randint(0,100) , random.randint(0,100) ,random.randint(-1000,1000)) + sql += "%s as cal19_1 ," % random.choice(self.calc_aggregate_all_j) + sql += "%s as cal19_2 " % random.choice(self.calc_aggregate_all_j) + sql += " from regular_table_1 t1, regular_table_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_or_where) + sql += "%s " % random.choice(self.state_u_window) + sql += "%s " % random.choice(self.limit_u_where) + sql += ") " + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */19-3 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ apercentile(cal19_1, %d)/1000 ,apercentile(cal19_2, %d)*10+%d from ( select /*+ para_tables_sort() */ " %(random.randint(0,100) , random.randint(0,100) ,random.randint(-1000,1000)) + sql += "%s as cal19_1 ," % random.choice(self.calc_aggregate_all) + sql += "%s as cal19_2 " % random.choice(self.calc_aggregate_all) + sql += " from stable_1_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.state_window) + sql += "%s " % random.choice(self.limit1_where) + sql += ") " + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */19-4 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ apercentile(cal19_1, %d)/1000 ,apercentile(cal19_2, %d)*10+%d from ( select /*+ para_tables_sort() */ " %(random.randint(0,100) , random.randint(0,100) ,random.randint(-1000,1000)) + sql += "%s as cal19_1 ," % random.choice(self.calc_aggregate_all_j) + sql += "%s as cal19_2 " % random.choice(self.calc_aggregate_all_j) + sql += " from stable_1_1 t1, stable_1_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_where) + sql += "%s " % random.choice(self.limit_u_where) + sql += ") " + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */19-4.2 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ apercentile(cal19_1, %d)/1000 ,apercentile(cal19_2, %d)*10+%d from ( select /*+ para_tables_sort() */ " %(random.randint(0,100) , random.randint(0,100) ,random.randint(-1000,1000)) + sql += "%s as cal19_1 ," % random.choice(self.calc_aggregate_all_j) + sql += "%s as cal19_2 " % random.choice(self.calc_aggregate_all_j) + sql += " from stable_1_1 t1, stable_1_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_or_where) + sql += "%s " % random.choice(self.limit_u_where) + sql += ") " + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */19-6 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ apercentile(cal19_1, %d)/1000 ,apercentile(cal19_2, %d)*10+%d from ( select /*+ para_tables_sort() */ " %(random.randint(0,100) , random.randint(0,100) ,random.randint(-1000,1000)) + sql += "%s as cal19_1 ," % random.choice(self.calc_aggregate_all_j) + sql += "%s as cal19_2 " % random.choice(self.calc_aggregate_all_j) + sql += " from stable_1 t1 , stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_where) + sql += "%s " % random.choice(self.limit_u_where) + sql += ") " + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */19-7 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ apercentile(cal19_1, %d)/1000 ,apercentile(cal19_2, %d)*10+%d from ( select /*+ para_tables_sort() */ " %(random.randint(0,100) , random.randint(0,100) ,random.randint(-1000,1000)) + sql += "%s as cal19_1 ," % random.choice(self.calc_aggregate_all_j) + sql += "%s as cal19_2 " % random.choice(self.calc_aggregate_all_j) + sql += " from stable_1 t1 , stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.qt_u_or_where) + sql += "%s " % random.choice(self.limit_u_where) + sql += ") " + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + #1 select /*+ para_tables_sort() */* from (select /*+ para_tables_sort() */* from (select /*+ para_tables_sort() */* form regular_table where <\>\in\and\or order by limit )) + tdSql.query("select /*+ para_tables_sort() */1-1 from stable_1;") + for i in range(self.fornum): + for_num = random.randint(1, 15); + sql = "select /*+ para_tables_sort() */ * from (" * for_num + sql += "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */* from ( select /*+ para_tables_sort() */ " + sql += "%s, " % random.choice(self.s_r_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts as ttt from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += ")) " + sql += ")" * for_num + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + sql2 = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */* from ( select /*+ para_tables_sort() */ " + sql2 += "%s, " % random.choice(self.s_r_select) + sql2 += "%s, " % random.choice(self.q_select) + sql2 += "ts as tin from regular_table_1 where " + sql2 += "%s " % random.choice(self.q_where) + sql2 += ")) " + tdLog.info(sql2) + tdSql.query(sql2) + self.cur1.execute(sql2) + self.explain_sql(sql2) + + tdLog.info("=====1-1==over=========") + + for i in range(self.fornum): + for_num = random.randint(1, 15); + sql = "select /*+ para_tables_sort() */ ts2 from (" * for_num + sql += "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */* from ( select /*+ para_tables_sort() */ " + sql += "%s, " % random.choice(self.s_r_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts as ts2 from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += ")) " + sql += ")" * for_num + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + sql2 = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */* from ( select /*+ para_tables_sort() */ " + sql2 += "%s, " % random.choice(self.s_r_select) + sql2 += "%s, " % random.choice(self.q_select) + sql2 += "ts as tt from regular_table_1 where " + sql2 += "%s " % random.choice(self.q_where) + sql2 += ")) " + tdLog.info(sql2) + tdSql.query(sql2) + self.explain_sql(sql2) + tdLog.info("=====1-2==over=========") + + #2 select /*+ para_tables_sort() */* from (select /*+ para_tables_sort() */* from (select /*+ para_tables_sort() */* form stable where <\>\in\and\or order by limit )) + tdSql.query("select /*+ para_tables_sort() */2-1 from stable_1;") + for i in range(self.fornum): + for_num = random.randint(1, 15); + sql = "select /*+ para_tables_sort() */ * from (" * for_num + sql += "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */* from ( select /*+ para_tables_sort() */ " + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s, " % random.choice(self.qt_select) + sql += "ts as tss from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += ")) " + sql += ")" * for_num + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + sql2 = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */* from ( select /*+ para_tables_sort() */ " + sql2 += "%s, " % random.choice(self.s_s_select) + sql2 += "%s, " % random.choice(self.qt_select) + sql2 += "ts as tst from stable_1 where " + sql2 += "%s " % random.choice(self.q_where) + sql2 += ")) " + tdLog.info(sql2) + tdSql.query(sql2) + self.explain_sql(sql2) + + tdLog.info("=====2-1==over=========") + + for i in range(self.fornum): + for_num = random.randint(1, 15); + sql = "select /*+ para_tables_sort() */ tsn from (" * for_num + sql += "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */* from ( select /*+ para_tables_sort() */ " + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s, " % random.choice(self.qt_select) + sql += "ts as tsn from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += ")) " + sql += ")" * for_num + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + sql2 = "select /*+ para_tables_sort() */ ts1 from ( select /*+ para_tables_sort() */* from ( select /*+ para_tables_sort() */ " + sql2 += "%s, " % random.choice(self.s_s_select) + sql2 += "%s, " % random.choice(self.qt_select) + sql2 += "ts as ts1 from stable_1 where " + sql2 += "%s " % random.choice(self.q_where) + sql2 += ")) " + tdLog.info(sql2) + tdSql.query(sql2) + self.cur1.execute(sql2) + self.explain_sql(sql2) + tdLog.info("=====2-2==over=========") + + #3 select /*+ para_tables_sort() */ts ,calc from (select /*+ para_tables_sort() */* form stable where <\>\in\and\or order by limit ) + tdSql.query("select /*+ para_tables_sort() */3-1 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ " + sql += "%s " % random.choice(self.calc_calculate_regular) + sql += " from ( select /*+ para_tables_sort() */* from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += "%s " % random.choice(self.orders_desc_where) + sql += "%s " % random.choice(self.limit_where) + sql += ") ;" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + #4 select /*+ para_tables_sort() */* from (select /*+ para_tables_sort() */calc form stable where <\>\in\and\or order by limit ) + tdSql.query("select /*+ para_tables_sort() */4-1 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */ " + sql += "%s " % random.choice(self.calc_select_in_ts) + sql += "from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += "%s " % random.choice(self.limit_where) + sql += ") ;" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + #5 select /*+ para_tables_sort() */ts ,tbname from (select /*+ para_tables_sort() */* form stable where <\>\in\and\or order by limit ) + tdSql.query("select /*+ para_tables_sort() */5-1 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ ts , tbname , " + sql += "%s ," % random.choice(self.calc_calculate_regular) + sql += "%s ," % random.choice(self.dqt_select) + sql += "%s " % random.choice(self.qt_select) + sql += " from ( select /*+ para_tables_sort() */* from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += "%s " % random.choice(self.orders_desc_where) + sql += "%s " % random.choice(self.limit_where) + sql += ") ;" + tdLog.info(sql) + tdSql.error(sql) + + #special sql + tdSql.query("select /*+ para_tables_sort() */6-1 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */* from ( select /*+ para_tables_sort() */_block_dist() from stable_1);" + tdSql.error(sql) + sql = "select /*+ para_tables_sort() */_block_dist() from (select /*+ para_tables_sort() */* from stable_1);" + tdSql.error(sql) + sql = "select /*+ para_tables_sort() */* from (select /*+ para_tables_sort() */database());" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + sql = "select /*+ para_tables_sort() */* from (select /*+ para_tables_sort() */client_version());" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + sql = "select /*+ para_tables_sort() */* from (select /*+ para_tables_sort() */client_version() as version);" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + sql = "select /*+ para_tables_sort() */* from (select /*+ para_tables_sort() */server_version());" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + sql = "select /*+ para_tables_sort() */* from (select /*+ para_tables_sort() */server_version() as version);" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + sql = "select /*+ para_tables_sort() */* from (select /*+ para_tables_sort() */server_status());" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + sql = "select /*+ para_tables_sort() */* from (select /*+ para_tables_sort() */server_status() as status);" + tdLog.info(sql) + tdSql.query(sql) + self.explain_sql(sql) + + #taos -f sql + startTime_taos_f = time.time() + print("taos -f sql start!") + taos_cmd1 = "taos -f %s/%s.sql" % (self.testcasePath,self.testcaseFilename) + _ = subprocess.check_output(taos_cmd1, shell=True) + print("taos -f sql over!") + endTime_taos_f = time.time() + print("taos_f total time %ds" % (endTime_taos_f - startTime_taos_f)) + + print('=====================2.6 old function end ===========') + + + def run(self): + tdSql.prepare() + + startTime = time.time() + + #self.function_before_26() + + self.dropandcreateDB_random("%s" %self.db_nest, 1) + + self.math_nest(['UNIQUE']) + self.math_nest(['MODE']) + self.math_nest(['SAMPLE']) + + self.math_nest(['ABS','SQRT']) + self.math_nest(['SIN','COS','TAN','ASIN','ACOS','ATAN']) + self.math_nest(['POW','LOG']) + self.math_nest(['FLOOR','CEIL','ROUND']) + self.math_nest(['MAVG']) + self.math_nest(['HYPERLOGLOG']) + self.math_nest(['TAIL']) + self.math_nest(['CSUM']) + self.math_nest(['statecount','stateduration']) + self.math_nest(['HISTOGRAM']) + + self.str_nest(['LTRIM','RTRIM','LOWER','UPPER']) + self.str_nest(['LENGTH','CHAR_LENGTH']) + self.str_nest(['SUBSTR']) + self.str_nest(['CONCAT']) + self.str_nest(['CONCAT_WS']) + self.time_nest(['CAST']) + self.time_nest(['CAST_1']) + self.time_nest(['CAST_2']) + self.time_nest(['CAST_3']) + self.time_nest(['CAST_4']) + + self.time_nest(['NOW','TODAY']) + self.time_nest(['TIMEZONE']) + self.time_nest(['TIMETRUNCATE']) + self.time_nest(['TO_ISO8601']) + self.time_nest(['TO_UNIXTIMESTAMP']) + self.time_nest(['ELAPSED']) + self.time_nest(['TIMEDIFF_1']) + self.time_nest(['TIMEDIFF_2']) + + endTime = time.time() + print("total time %ds" % (endTime - startTime)) + + + + + def stop(self): + tdSql.close() + tdLog.success("%s successfully executed" % __file__) + + +tdCases.addWindows(__file__, TDTestCase()) +tdCases.addLinux(__file__, TDTestCase()) diff --git a/tests/system-test/2-query/para_tms2.py b/tests/system-test/2-query/para_tms2.py new file mode 100755 index 0000000000..f7f278c419 --- /dev/null +++ b/tests/system-test/2-query/para_tms2.py @@ -0,0 +1,6209 @@ +################################################################### +# Copyright (c) 2016 by TAOS Technologies, Inc. +# All rights reserved. +# +# This file is proprietary and confidential to TAOS Technologies. +# No part of this file may be reproduced, stored, transmitted, +# disclosed or used in any form or by any means other than as +# expressly provided by the written permission from Jianhui Tao +# +################################################################### + +# -*- coding: utf-8 -*- + +import random +import os +import time +import taos +import subprocess +from faker import Faker +from util.log import tdLog +from util.cases import tdCases +from util.sql import tdSql +from util.dnodes import tdDnodes +from util.dnodes import * + +class TDTestCase: + updatecfgDict = {'maxSQLLength':1048576,'debugFlag': 131 ,"cDebugFlag":131,"uDebugFlag":131 ,"rpcDebugFlag":131 , "tmrDebugFlag":131 , + "jniDebugFlag":131 ,"simDebugFlag":131,"dDebugFlag":131, "dDebugFlag":131,"vDebugFlag":131,"mDebugFlag":131,"qDebugFlag":131, + "wDebugFlag":131,"sDebugFlag":131,"tsdbDebugFlag":131,"tqDebugFlag":131 ,"fsDebugFlag":131 ,"fnDebugFlag":131} + + def init(self, conn, logSql, replicaVar=1): + self.replicaVar = int(replicaVar) + tdLog.debug("start to execute %s" % __file__) + tdSql.init(conn.cursor(), logSql) + + self.testcasePath = os.path.split(__file__)[0] + self.testcaseFilename = os.path.split(__file__)[-1] + os.system("rm -rf %s/%s.sql" % (self.testcasePath,self.testcaseFilename)) + + self.num = 10 + self.fornum = 15 + + self.db_nest = "nest" + self.dropandcreateDB_random("%s" %self.db_nest, 1) + + # regular column select + #q_select= ['ts' , '*' , 'q_int', 'q_bigint' , 'q_bigint' , 'q_smallint' , 'q_tinyint' , 'q_bool' , 'q_binary' , 'q_nchar' ,'q_float' , 'q_double' ,'q_ts '] + self.q_select= ['q_int', 'q_bigint' , 'q_bigint' , 'q_smallint' , 'q_tinyint' , 'q_bool' , 'q_binary' , 'q_nchar' ,'q_float' , 'q_double' ,'q_ts ', 'q_int_null ', 'q_bigint_null ' , 'q_bigint_null ' , 'q_smallint_null ' , 'q_tinyint_null ' , 'q_bool_null ' , 'q_binary_null ' , 'q_nchar_null ' ,'q_float_null ' , 'q_double_null ' ,'q_ts_null '] + + # tag column select + #t_select= ['*' , 'loc' ,'t_int', 't_bigint' , 't_bigint' , 't_smallint' , 't_tinyint' , 't_bool' , 't_binary' , 't_nchar' ,'t_float' , 't_double' ,'t_ts '] + self.t_select= ['loc','t_int', 't_bigint' , 't_bigint' , 't_smallint' , 't_tinyint' , 't_bool' , 't_binary' , 't_nchar' ,'t_float' , 't_double' ,'t_ts '] + + # regular and tag column select + self.qt_select= self.q_select + self.t_select + + # distinct regular column select + self.dq_select= ['distinct q_int', 'distinct q_bigint' , 'distinct q_smallint' , 'distinct q_tinyint' , + 'distinct q_bool' , 'distinct q_binary' , 'distinct q_nchar' ,'distinct q_float' , 'distinct q_double' ,'distinct q_ts '] + + # distinct tag column select + self.dt_select= ['distinct loc', 'distinct t_int', 'distinct t_bigint' , 'distinct t_smallint' , 'distinct t_tinyint' , + 'distinct t_bool' , 'distinct t_binary' , 'distinct t_nchar' ,'distinct t_float' , 'distinct t_double' ,'distinct t_ts '] + + # distinct regular and tag column select + self.dqt_select= self.dq_select + self.dt_select + + # special column select + self.s_r_select= ['_c0', '_rowts' , '_C0' ] + self.s_s_select= ['tbname' , '_rowts' , '_c0', '_C0' ] + self.unionall_or_union= [ ' union ' , ' union all ' ] + + # regular column where + self.q_where = ['ts < now +1s','q_bigint >= -9223372036854775807 and q_bigint <= 9223372036854775807', 'q_int <= 2147483647 and q_int >= -2147483647', + 'q_smallint >= -32767 and q_smallint <= 32767','q_tinyint >= -127 and q_tinyint <= 127','q_float >= -1.7E308 and q_float <= 1.7E308', + 'q_double >= -1.7E308 and q_double <= 1.7E308', 'q_binary like \'binary%\' or q_binary = \'0\' ' , 'q_nchar like \'nchar%\' or q_nchar = \'0\' ' , + 'q_bool = true or q_bool = false' , 'q_bool in (0 , 1)' , 'q_bool in ( true , false)' , 'q_bool = 0 or q_bool = 1', + 'q_bigint between -9223372036854775807 and 9223372036854775807',' q_int between -2147483647 and 2147483647','q_smallint between -32767 and 32767', + 'q_bigint not between 9223372036854775807 and -9223372036854775807','q_int not between 2147483647 and -2147483647','q_smallint not between 32767 and -32767', + 'q_tinyint between -127 and 127 ','q_float >= -3.4E38 ','q_float <= 3.4E38 ','q_double >= -1.7E308 ', + 'q_double <= 1.7E308 ','q_float between -3.4E38 and 3.4E38 ','q_double between -1.7E308 and 1.7E308 ' ,'q_float not between 3.4E38 and -3.4E38 ','q_double not between 1.7E308 and -1.7E308 ', + 'q_float is not null ' ,'q_double is not null ' ,'q_binary match \'binary\' ','q_binary nmatch \'binarynchar\' ','q_nchar match \'nchar\' ','q_nchar nmatch \'binarynchar\' ', + 'q_binary like \'binary%\' ','(q_binary like \'binary%\' or q_nchar = \'0\' or q_binary = \'binary_\' ) ','q_nchar like \'nchar%\' ','(q_nchar like \'nchar%\' or q_binary = \'0\' or q_nchar = \'nchar_\' ) ',] + #TD-6201 ,'q_bool between 0 and 1' + + # regular column where for test union,join + self.q_u_where = ['t1.ts < now +1s' , 't2.ts < now +1s','t1.q_bigint >= -9223372036854775807 and t1.q_bigint <= 9223372036854775807 and t2.q_bigint >= -9223372036854775807 and t2.q_bigint <= 9223372036854775807', + 't1.q_int <= 2147483647 and t1.q_int >= -2147483647 and t2.q_int <= 2147483647 and t2.q_int >= -2147483647', + 't1.q_smallint >= -32767 and t1.q_smallint <= 32767 and t2.q_smallint >= -32767 and t2.q_smallint <= 32767', + 't1.q_tinyint >= -127 and t1.q_tinyint <= 127 and t2.q_tinyint >= -127 and t2.q_tinyint <= 127', + 't1.q_float >= - 1.7E308 and t1.q_float <= 1.7E308 and t2.q_float >= - 1.7E308 and t2.q_float <= 1.7E308', + 't1.q_double >= - 1.7E308 and t1.q_double <= 1.7E308 and t2.q_double >= - 1.7E308 and t2.q_double <= 1.7E308', + 't1.q_binary like \'binary%\' and t2.q_binary like \'binary%\' ' , + 't1.q_nchar like \'nchar%\' and t2.q_nchar like \'nchar%\' ' , + 't1.q_bool in (0 , 1) and t2.q_bool in (0 , 1)' , 't1.q_bool in ( true , false) and t2.q_bool in ( true , false)' , + 't1.q_bigint between -9223372036854775807 and 9223372036854775807 and t2.q_bigint between -9223372036854775807 and 9223372036854775807', + 't1.q_int between -2147483647 and 2147483647 and t2.q_int between -2147483647 and 2147483647', + 't1.q_smallint between -32767 and 32767 and t2.q_smallint between -32767 and 32767', + 't1.q_tinyint between -127 and 127 and t2.q_tinyint between -127 and 127 ','t1.q_float between -1.7E308 and 1.7E308 and t2.q_float between -1.7E308 and 1.7E308', + 't1.q_double between -1.7E308 and 1.7E308 and t2.q_double between -1.7E308 and 1.7E308', + 't1.q_bigint not between 9223372036854775807 and -9223372036854775807 and t2.q_bigint not between 9223372036854775807 and -9223372036854775807', + 't1.q_int not between 2147483647 and -2147483647 and t2.q_int not between 2147483647 and -2147483647', + 't1.q_smallint not between 32767 and -32767 and t2.q_smallint not between 32767 and -32767', + 't1.q_tinyint not between 127 and -127 and t2.q_tinyint not between 127 and -127 ','t1.q_float not between -1.7E308 and -1.7E308 and t2.q_float not between 1.7E308 and -1.7E308', + 't1.q_double not between 1.7E308 and -1.7E308 and t2.q_double not between 1.7E308 and -1.7E308'] + #TD-6201 ,'t1.q_bool between 0 and 1 or t2.q_bool between 0 and 1'] + #'t1.q_bool = true and t1.q_bool = false and t2.q_bool = true and t2.q_bool = false' , 't1.q_bool = 0 and t1.q_bool = 1 and t2.q_bool = 0 and t2.q_bool = 1' , + + self.q_u_or_where = ['(t1.q_binary like \'binary%\' or t1.q_binary = \'0\' or t2.q_binary like \'binary%\' or t2.q_binary = \'0\' )' , + '(t1.q_nchar like \'nchar%\' or t1.q_nchar = \'0\' or t2.q_nchar like \'nchar%\' or t2.q_nchar = \'0\' )' , '(t1.q_bool = true or t1.q_bool = false or t2.q_bool = true or t2.q_bool = false)' , + '(t1.q_bool in (0 , 1) or t2.q_bool in (0 , 1))' , '(t1.q_bool in ( true , false) or t2.q_bool in ( true , false))' , '(t1.q_bool = 0 or t1.q_bool = 1 or t2.q_bool = 0 or t2.q_bool = 1)' , + '(t1.q_bigint between -9223372036854775807 and 9223372036854775807 or t2.q_bigint between -9223372036854775807 and 9223372036854775807)', + '(t1.q_int between -2147483647 and 2147483647 or t2.q_int between -2147483647 and 2147483647)', + '(t1.q_smallint between -32767 and 32767 or t2.q_smallint between -32767 and 32767)', + '(t1.q_tinyint between -127 and 127 or t2.q_tinyint between -127 and 127 )','(t1.q_float between -1.7E308 and 1.7E308 or t2.q_float between -1.7E308 and 1.7E308)', + '(t1.q_double between -1.7E308 and 1.7E308 or t2.q_double between -1.7E308 and 1.7E308)'] + + # tag column where + self.t_where = ['ts < now +1s','t_bigint >= -9223372036854775807 and t_bigint <= 9223372036854775807','t_int <= 2147483647 and t_int >= -2147483647', + 't_smallint >= -32767 and t_smallint <= 32767','q_tinyint >= -127 and t_tinyint <= 127','t_float >= -1.7E308 and t_float <= 1.7E308', + 't_double >= -1.7E308 and t_double <= 1.7E308', 't_binary like \'binary%\' or t_binary = \'0\' ' , 't_nchar like \'nchar%\' or t_nchar = \'0\'' , + 't_bool = true or t_bool = false' , 't_bool in (0 , 1)' , 't_bool in ( true , false)' , 't_bool = 0 or t_bool = 1', + 't_bigint between -9223372036854775807 and 9223372036854775807',' t_int between -2147483647 and 2147483647','t_smallint between -32767 and 32767', + 't_tinyint between -127 and 127 ','t_float between -1.7E308 and 1.7E308','t_double between -1.7E308 and 1.7E308', + 't_binary match \'binary\' ','t_binary nmatch \'binarynchar\' ','t_nchar match \'nchar\' ','t_nchar nmatch \'binarynchar\' ', + 't_binary like \'binary%\' ','t_nchar like \'nchar%\' ','(t_binary like \'binary%\' or t_nchar = \'0\' ) ','(t_nchar like \'nchar%\' or t_binary = \'0\' ) ',] + #TD-6201,'t_bool between 0 and 1' + + # tag column where for test union,join | this is not support + self.t_u_where = ['t1.ts < now +1s' , 't2.ts < now +1s','t1.t_bigint >= -9223372036854775807 and t1.t_bigint <= 9223372036854775807 and t2.t_bigint >= -9223372036854775807 and t2.t_bigint <= 9223372036854775807', + 't1.t_int <= 2147483647 and t1.t_int >= -2147483647 and t2.t_int <= 2147483647 and t2.t_int >= -2147483647', + 't1.t_smallint >= -32767 and t1.t_smallint <= 32767 and t2.t_smallint >= -32767 and t2.t_smallint <= 32767', + 't1.t_tinyint >= -127 and t1.t_tinyint <= 127 and t2.t_tinyint >= -127 and t2.t_tinyint <= 127', + 't1.t_float >= -1.7E308 and t1.t_float <= 1.7E308 and t2.t_float >= -1.7E308 and t2.t_float <= 1.7E308', + 't1.t_double >= -1.7E308 and t1.t_double <= 1.7E308 and t2.t_double >= -1.7E308 and t2.t_double <= 1.7E308', + '(t1.t_binary like \'binary%\' or t1.t_binary = \'0\' or t2.t_binary like \'binary%\' or t2.t_binary = \'0\') ' , + '(t1.t_nchar like \'nchar%\' or t1.t_nchar = \'0\' or t2.t_nchar like \'nchar%\' or t2.t_nchar = \'0\' )' , '(t1.t_bool = true or t1.t_bool = false or t2.t_bool = true or t2.t_bool = false)' , + 't1.t_bool in (0 , 1) and t2.t_bool in (0 , 1)' , 't1.t_bool in ( true , false) and t2.t_bool in ( true , false)' , '(t1.t_bool = 0 or t1.t_bool = 1 or t2.t_bool = 0 or t2.t_bool = 1)', + 't1.t_bigint between -9223372036854775807 and 9223372036854775807 and t2.t_bigint between -9223372036854775807 and 9223372036854775807', + 't1.t_int between -2147483647 and 2147483647 and t2.t_int between -2147483647 and 2147483647', + 't1.t_smallint between -32767 and 32767 and t2.t_smallint between -32767 and 32767', + '(t1.t_tinyint between -127 and 127 and t2.t_tinyint between -127 and 127) ','t1.t_float between -1.7E308 and 1.7E308 and t2.t_float between -1.7E308 and 1.7E308', + '(t1.t_double between -1.7E308 and 1.7E308 and t2.t_double between -1.7E308 and 1.7E308)'] + #TD-6201,'t1.t_bool between 0 and 1 or t2.q_bool between 0 and 1'] + + self.t_u_or_where = ['(t1.t_binary like \'binary%\' or t1.t_binary = \'0\' or t2.t_binary like \'binary%\' or t2.t_binary = \'0\' )' , + '(t1.t_nchar like \'nchar%\' or t1.t_nchar = \'0\' or t2.t_nchar like \'nchar%\' or t2.t_nchar = \'0\' )' , '(t1.t_bool = true or t1.t_bool = false or t2.t_bool = true or t2.t_bool = false)' , + '(t1.t_bool in (0 , 1) or t2.t_bool in (0 , 1))' , '(t1.t_bool in ( true , false) or t2.t_bool in ( true , false))' , '(t1.t_bool = 0 or t1.t_bool = 1 or t2.t_bool = 0 or t2.t_bool = 1)', + '(t1.t_bigint between -9223372036854775807 and 9223372036854775807 or t2.t_bigint between -9223372036854775807 and 9223372036854775807)', + '(t1.t_int between -2147483647 and 2147483647 or t2.t_int between -2147483647 and 2147483647)', + '(t1.t_smallint between -32767 and 32767 or t2.t_smallint between -32767 and 32767)', + '(t1.t_tinyint between -127 and 127 or t2.t_tinyint between -127 and 127 )','(t1.t_float between -1.7E308 and 1.7E308 or t2.t_float between -1.7E308 and 1.7E308)', + '(t1.t_double between -1.7E308 and 1.7E308 or t2.t_double between -1.7E308 and 1.7E308)'] + + # self.t_u_where = ['t1.ts < now +1s'] # 超级表tag不支持,暂时注掉 + # self.t_u_or_where = ['(t1.q_bool in (0 , 1))'] #超级表tag不支持,暂时注掉 + + # regular and tag column where + self.qt_where = self.q_where + self.t_where + #self.qt_where = self.q_where #超级表tag不支持,暂时注掉 + self.qt_u_where = self.q_u_where + self.t_u_where + # now,qt_u_or_where is not support + self.qt_u_or_where = self.q_u_or_where + self.t_u_or_where + + # tag column where for test super join | this is support , 't1.t_bool = t2.t_bool ' ??? + self.t_join_where = ['t1.t_bigint = t2.t_bigint ', 't1.t_int = t2.t_int ', 't1.t_smallint = t2.t_smallint ', 't1.t_tinyint = t2.t_tinyint ', + 't1.t_float = t2.t_float ', 't1.t_double = t2.t_double ', 't1.t_binary = t2.t_binary ' , 't1.t_nchar = t2.t_nchar ' ] + #self.t_join_where = ['t1.ts = t2.ts'] # 超级表tag不支持,暂时注掉 + + # session && fill + self.session_where = ['session(ts,10a)' , 'session(ts,10s)', 'session(ts,10m)' , 'session(ts,10h)','session(ts,10d)' , 'session(ts,10w)'] + self.session_u_where = ['session(t1.ts,10a)' , 'session(t1.ts,10s)', 'session(t1.ts,10m)' , 'session(t1.ts,10h)','session(t1.ts,10d)' , 'session(t1.ts,10w)', + 'session(t2.ts,10a)' , 'session(t2.ts,10s)', 'session(t2.ts,10m)' , 'session(t2.ts,10h)','session(t2.ts,10d)' , 'session(t2.ts,10w)'] + + self.fill_where = ['FILL(NONE)','FILL(PREV)','FILL(NULL)','FILL(LINEAR)','FILL(NEXT)','FILL(VALUE, 1.23)'] + + self.state_window = ['STATE_WINDOW(q_tinyint)','STATE_WINDOW(q_bigint)','STATE_WINDOW(q_int)','STATE_WINDOW(q_bool)','STATE_WINDOW(q_smallint)'] + self.state_u_window = ['STATE_WINDOW(t1.q_tinyint)','STATE_WINDOW(t1.q_bigint)','STATE_WINDOW(t1.q_int)','STATE_WINDOW(t1.q_bool)','STATE_WINDOW(t1.q_smallint)', + 'STATE_WINDOW(t2.q_tinyint)','STATE_WINDOW(t2.q_bigint)','STATE_WINDOW(t2.q_int)','STATE_WINDOW(t2.q_bool)','STATE_WINDOW(t2.q_smallint)'] + + # order by where + self.order_where = ['order by ts' , 'order by ts asc'] + self.order_u_where = ['order by t1.ts' , 'order by t1.ts asc' , 'order by t2.ts' , 'order by t2.ts asc'] + self.order_desc_where = ['order by ts' , 'order by ts asc' , 'order by ts desc' ] + self.orders_desc_where = ['order by ts' , 'order by ts asc' , 'order by ts desc' , 'order by ts,loc' , 'order by ts,loc asc' , 'order by ts,loc desc'] + + self.group_where = ['group by tbname , loc' , 'group by tbname', 'group by tbname, t_bigint', 'group by tbname,t_int', 'group by tbname, t_smallint', 'group by tbname,t_tinyint', + 'group by tbname,t_float', 'group by tbname,t_double' , 'group by tbname,t_binary', 'group by tbname,t_nchar', 'group by tbname,t_bool' ,'group by tbname ,loc ,t_bigint', + 'group by tbname,t_binary ,t_nchar ,t_bool' , 'group by tbname,t_int ,t_smallint ,t_tinyint' , 'group by tbname,t_float ,t_double ' , + 'PARTITION BY tbname , loc' , 'PARTITION BY tbname', 'PARTITION BY tbname, t_bigint', 'PARTITION BY tbname,t_int', 'PARTITION BY tbname, t_smallint', 'PARTITION BY tbname,t_tinyint', + 'PARTITION BY tbname,t_float', 'PARTITION BY tbname,t_double' , 'PARTITION BY tbname,t_binary', 'PARTITION BY tbname,t_nchar', 'PARTITION BY tbname,t_bool' ,'PARTITION BY tbname ,loc ,t_bigint', + 'PARTITION BY tbname,t_binary ,t_nchar ,t_bool' , 'PARTITION BY tbname,t_int ,t_smallint ,t_tinyint' , 'PARTITION BY tbname,t_float ,t_double '] + self.group_where_j = ['group by t1.loc' , 'group by t1.t_bigint', 'group by t1.t_int', 'group by t1.t_smallint', 'group by t1.t_tinyint', + 'group by t1.t_float', 'group by t1.t_double' , 'group by t1.t_binary', 'group by t1.t_nchar', 'group by t1.t_bool' ,'group by t1.loc ,t1.t_bigint', + 'group by t1.t_binary ,t1.t_nchar ,t1.t_bool' , 'group by t1.t_int ,t1.t_smallint ,t1.t_tinyint' , 'group by t1.t_float ,t1.t_double ' , + 'PARTITION BY t1.loc' , 'PARTITION by t1.t_bigint', 'PARTITION by t1.t_int', 'PARTITION by t1.t_smallint', 'PARTITION by t1.t_tinyint', + 'PARTITION by t1.t_float', 'PARTITION by t1.t_double' , 'PARTITION by t1.t_binary', 'PARTITION by t1.t_nchar', 'PARTITION by t1.t_bool' ,'PARTITION BY t1.loc ,t1.t_bigint', + 'PARTITION by t1.t_binary ,t1.t_nchar ,t1.t_bool' , 'PARTITION by t1.t_int ,t1.t_smallint ,t1.t_tinyint' , 'PARTITION by t1.t_float ,t1.t_double ', + 'group by t2.loc' , 'group by t2.t_bigint', 'group by t2.t_int', 'group by t2.t_smallint', 'group by t2.t_tinyint', + 'group by t2.t_float', 'group by t2.t_double' , 'group by t2.t_binary', 'group by t2.t_nchar', 'group by t2.t_bool' ,'group by t2.loc ,t2.t_bigint', + 'group by t2.t_binary ,t2.t_nchar ,t2.t_bool' , 'group by t2.t_int ,t2.t_smallint ,t2.t_tinyint' , 'group by t2.t_float ,t2.t_double ' , + 'PARTITION BY t2.loc' , 'PARTITION by t2.t_bigint', 'PARTITION by t2.t_int', 'PARTITION by t2.t_smallint', 'PARTITION by t2.t_tinyint', + 'PARTITION by t2.t_float', 'PARTITION by t2.t_double' , 'PARTITION by t2.t_binary', 'PARTITION by t2.t_nchar', 'PARTITION by t2.t_bool' ,'PARTITION BY t2.loc ,t2.t_bigint', + 'PARTITION by t2.t_binary ,t2.t_nchar ,t2.t_bool' , 'PARTITION by t2.t_int ,t2.t_smallint ,t2.t_tinyint' , 'PARTITION by t2.t_float ,t2.t_double '] + + self.group_only_where = ['group by tbname , loc' , 'group by tbname', 'group by tbname, t_bigint', 'group by tbname,t_int', 'group by tbname, t_smallint', 'group by tbname,t_tinyint', + 'group by tbname,t_float', 'group by tbname,t_double' , 'group by tbname,t_binary', 'group by tbname,t_nchar', 'group by tbname,t_bool' ,'group by tbname ,loc ,t_bigint', + 'group by tbname,t_binary ,t_nchar ,t_bool' , 'group by tbname,t_int ,t_smallint ,t_tinyint' , 'group by tbname,t_float ,t_double ' ] + self.group_only_where_j = ['group by t1.loc' , 'group by t1.t_bigint', 'group by t1.t_int', 'group by t1.t_smallint', 'group by t1.t_tinyint', + 'group by t1.t_float', 'group by t1.t_double' , 'group by t1.t_binary', 'group by t1.t_nchar', 'group by t1.t_bool' ,'group by t1.loc ,t1.t_bigint', + 'group by t1.t_binary ,t1.t_nchar ,t1.t_bool' , 'group by t1.t_int ,t1.t_smallint ,t1.t_tinyint' , 'group by t1.t_float ,t1.t_double ' , + 'group by t2.loc' , 'group by t2.t_bigint', 'group by t2.t_int', 'group by t2.t_smallint', 'group by t2.t_tinyint', + 'group by t2.t_float', 'group by t2.t_double' , 'group by t2.t_binary', 'group by t2.t_nchar', 'group by t2.t_bool' ,'group by t2.loc ,t2.t_bigint', + 'group by t2.t_binary ,t2.t_nchar ,t2.t_bool' , 'group by t2.t_int ,t2.t_smallint ,t2.t_tinyint' , 'group by t2.t_float ,t2.t_double ' ] + + self.partiton_where = ['PARTITION BY tbname , loc' , 'PARTITION BY tbname', 'PARTITION BY tbname, t_bigint', 'PARTITION BY tbname,t_int', 'PARTITION BY tbname, t_smallint', 'PARTITION BY tbname,t_tinyint', + 'PARTITION BY tbname,t_float', 'PARTITION BY tbname,t_double' , 'PARTITION BY tbname,t_binary', 'PARTITION BY tbname,t_nchar', 'PARTITION BY tbname,t_bool' ,'PARTITION BY tbname ,loc ,t_bigint', + 'PARTITION BY tbname,t_binary ,t_nchar ,t_bool' , 'PARTITION BY tbname,t_int ,t_smallint ,t_tinyint' , 'PARTITION BY tbname,t_float ,t_double '] + self.partiton_where_j = ['PARTITION BY t1.loc' , 'PARTITION by t1.t_bigint', 'PARTITION by t1.t_int', 'PARTITION by t1.t_smallint', 'PARTITION by t1.t_tinyint', + 'PARTITION by t1.t_float', 'PARTITION by t1.t_double' , 'PARTITION by t1.t_binary', 'PARTITION by t1.t_nchar', 'PARTITION by t1.t_bool' ,'PARTITION BY t1.loc ,t1.t_bigint', + 'PARTITION by t1.t_binary ,t1.t_nchar ,t1.t_bool' , 'PARTITION by t1.t_int ,t1.t_smallint ,t1.t_tinyint' , 'PARTITION by t1.t_float ,t1.t_double ', + 'PARTITION BY t2.loc' , 'PARTITION by t2.t_bigint', 'PARTITION by t2.t_int', 'PARTITION by t2.t_smallint', 'PARTITION by t2.t_tinyint', + 'PARTITION by t2.t_float', 'PARTITION by t2.t_double' , 'PARTITION by t2.t_binary', 'PARTITION by t2.t_nchar', 'PARTITION by t2.t_bool' ,'PARTITION BY t2.loc ,t2.t_bigint', + 'PARTITION by t2.t_binary ,t2.t_nchar ,t2.t_bool' , 'PARTITION by t2.t_int ,t2.t_smallint ,t2.t_tinyint' , 'PARTITION by t2.t_float ,t2.t_double '] + + + self.group_where_regular = ['group by tbname ' , 'group by tbname', 'group by tbname, q_bigint', 'group by tbname,q_int', 'group by tbname, q_smallint', 'group by tbname,q_tinyint', + 'group by tbname,q_float', 'group by tbname,q_double' , 'group by tbname,q_binary', 'group by tbname,q_nchar', 'group by tbname,q_bool' ,'group by tbname ,q_bigint', + 'group by tbname,q_binary ,q_nchar ,q_bool' , 'group by tbname,q_int ,q_smallint ,q_tinyint' , 'group by tbname,q_float ,q_double ' , + 'PARTITION BY tbname ' , 'PARTITION BY tbname', 'PARTITION BY tbname, q_bigint', 'PARTITION BY tbname,q_int', 'PARTITION BY tbname, q_smallint', 'PARTITION BY tbname,q_tinyint', + 'PARTITION BY tbname,q_float', 'PARTITION BY tbname,q_double' , 'PARTITION BY tbname,q_binary', 'PARTITION BY tbname,q_nchar', 'PARTITION BY tbname,q_bool' ,'PARTITION BY tbname ,q_bigint', + 'PARTITION BY tbname,q_binary ,q_nchar ,q_bool' , 'PARTITION BY tbname,q_int ,q_smallint ,q_tinyint' , 'PARTITION BY tbname,q_float ,q_double '] + self.group_where_regular_j = ['group by t1.q_bigint', 'group by t1.q_int', 'group by t1.q_smallint', 'group by t1.q_tinyint', + 'group by t1.q_float', 'group by t1.q_double' , 'group by t1.q_binary', 'group by t1.q_nchar', 'group by t1.q_bool' ,'group by t1.q_bigint', + 'group by t1.q_binary ,t1.q_nchar ,t1.q_bool' , 'group by t1.q_int ,t1.q_smallint ,t1.q_tinyint' , 'group by t1.q_float ,t1.q_double ' , + 'PARTITION by t1.q_bigint', 'PARTITION by t1.q_int', 'PARTITION by t1.q_smallint', 'PARTITION by t1.q_tinyint', + 'PARTITION by t1.q_float', 'PARTITION by t1.q_double' , 'PARTITION by t1.q_binary', 'PARTITION by t1.q_nchar', 'PARTITION by t1.q_bool' ,'PARTITION BY t1.q_bigint', + 'PARTITION by t1.q_binary ,t1.q_nchar ,t1.q_bool' , 'PARTITION by t1.q_int ,t1.q_smallint ,t1.q_tinyint' , 'PARTITION by t1.q_float ,t1.q_double ', + 'group by t2.q_bigint', 'group by t2.q_int', 'group by t2.q_smallint', 'group by t2.q_tinyint', + 'group by t2.q_float', 'group by t2.q_double' , 'group by t2.q_binary', 'group by t2.q_nchar', 'group by t2.q_bool' ,'group by t2.q_bigint', + 'group by t2.q_binary ,t2.q_nchar ,t2.q_bool' , 'group by t2.q_int ,t2.q_smallint ,t2.q_tinyint' , 'group by t2.q_float ,t2.q_double ' , + 'PARTITION by t2.q_bigint', 'PARTITION by t2.q_int', 'PARTITION by t2.q_smallint', 'PARTITION by t2.q_tinyint', + 'PARTITION by t2.q_float', 'PARTITION by t2.q_double' , 'PARTITION by t2.q_binary', 'PARTITION by t2.q_nchar', 'PARTITION by t2.q_bool' ,'PARTITION BY t2.q_bigint', + 'PARTITION by t2.q_binary ,t2.q_nchar ,t2.q_bool' , 'PARTITION by t2.q_int ,t2.q_smallint ,t2.q_tinyint' , 'PARTITION by t2.q_float ,t2.q_double '] + + self.partiton_where_regular = ['PARTITION BY tbname ' , 'PARTITION BY tbname', 'PARTITION BY tbname, q_bigint', 'PARTITION BY tbname,q_int', 'PARTITION BY tbname, q_smallint', 'PARTITION BY tbname,q_tinyint', + 'PARTITION BY tbname,q_float', 'PARTITION BY tbname,q_double' , 'PARTITION BY tbname,q_binary', 'PARTITION BY tbname,q_nchar', 'PARTITION BY tbname,q_bool' ,'PARTITION BY tbname ,q_bigint', + 'PARTITION BY tbname,q_binary ,q_nchar ,q_bool' , 'PARTITION BY tbname,q_int ,q_smallint ,q_tinyint' , 'PARTITION BY tbname,q_float ,q_double '] + self.partiton_where_regular_j = ['PARTITION by t1.q_bigint', 'PARTITION by t1.q_int', 'PARTITION by t1.q_smallint', 'PARTITION by t1.q_tinyint', + 'PARTITION by t1.q_float', 'PARTITION by t1.q_double' , 'PARTITION by t1.q_binary', 'PARTITION by t1.q_nchar', 'PARTITION by t1.q_bool' ,'PARTITION BY t1.q_bigint', + 'PARTITION by t1.q_binary ,t1.q_nchar ,t1.q_bool' , 'PARTITION by t1.q_int ,t1.q_smallint ,t1.q_tinyint' , 'PARTITION by t1.q_float ,t1.q_double ', + 'PARTITION by t2.q_bigint', 'PARTITION by t2.q_int', 'PARTITION by t2.q_smallint', 'PARTITION by t2.q_tinyint', + 'PARTITION by t2.q_float', 'PARTITION by t2.q_double' , 'PARTITION by t2.q_binary', 'PARTITION by t2.q_nchar', 'PARTITION by t2.q_bool' ,'PARTITION BY t2.q_bigint', + 'PARTITION by t2.q_binary ,t2.q_nchar ,t2.q_bool' , 'PARTITION by t2.q_int ,t2.q_smallint ,t2.q_tinyint' , 'PARTITION by t2.q_float ,t2.q_double '] + + self.having_support = ['having count(q_int) > 0','having count(q_bigint) > 0','having count(q_smallint) > 0','having count(q_tinyint) > 0','having count(q_float) > 0','having count(q_double) > 0','having count(q_bool) > 0', + 'having avg(q_int) > 0','having avg(q_bigint) > 0','having avg(q_smallint) > 0','having avg(q_tinyint) > 0','having avg(q_float) > 0','having avg(q_double) > 0', + 'having sum(q_int) > 0','having sum(q_bigint) > 0','having sum(q_smallint) > 0','having sum(q_tinyint) > 0','having sum(q_float) > 0','having sum(q_double) > 0', + 'having STDDEV(q_int) > 0','having STDDEV(q_bigint) > 0','having STDDEV(q_smallint) > 0','having STDDEV(q_tinyint) > 0','having STDDEV(q_float) > 0','having STDDEV(q_double) > 0', + 'having TWA(q_int) > 0','having TWA(q_bigint) > 0','having TWA(q_smallint) > 0','having TWA(q_tinyint) > 0','having TWA(q_float) > 0','having TWA(q_double) > 0', + 'having IRATE(q_int) > 0','having IRATE(q_bigint) > 0','having IRATE(q_smallint) > 0','having IRATE(q_tinyint) > 0','having IRATE(q_float) > 0','having IRATE(q_double) > 0', + 'having MIN(q_int) > 0','having MIN(q_bigint) > 0','having MIN(q_smallint) > 0','having MIN(q_tinyint) > 0','having MIN(q_float) > 0','having MIN(q_double) > 0', + 'having MAX(q_int) > 0','having MAX(q_bigint) > 0','having MAX(q_smallint) > 0','having MAX(q_tinyint) > 0','having MAX(q_float) > 0','having MAX(q_double) > 0', + 'having FIRST(q_int) > 0','having FIRST(q_bigint) > 0','having FIRST(q_smallint) > 0','having FIRST(q_tinyint) > 0','having FIRST(q_float) > 0','having FIRST(q_double) > 0', + 'having LAST(q_int) > 0','having LAST(q_bigint) > 0','having LAST(q_smallint) > 0','having LAST(q_tinyint) > 0','having LAST(q_float) > 0','having LAST(q_double) > 0', + 'having APERCENTILE(q_int,10) > 0','having APERCENTILE(q_bigint,10) > 0','having APERCENTILE(q_smallint,10) > 0','having APERCENTILE(q_tinyint,10) > 0','having APERCENTILE(q_float,10) > 0','having APERCENTILE(q_double,10) > 0', + 'having count(q_int_null) > 0','having count(q_bigint_null) > 0','having count(q_smallint_null) > 0','having count(q_tinyint_null) > 0','having count(q_float_null) > 0','having count(q_double_null) > 0','having count(q_bool_null) > 0', + 'having avg(q_int_null) > 0','having avg(q_bigint_null) > 0','having avg(q_smallint_null) > 0','having avg(q_tinyint_null) > 0','having avg(q_float_null) > 0','having avg(q_double_null) > 0', + 'having sum(q_int_null) > 0','having sum(q_bigint_null) > 0','having sum(q_smallint_null) > 0','having sum(q_tinyint_null) > 0','having sum(q_float_null) > 0','having sum(q_double_null) > 0', + 'having STDDEV(q_int_null) > 0','having STDDEV(q_bigint_null) > 0','having STDDEV(q_smallint_null) > 0','having STDDEV(q_tinyint_null) > 0','having STDDEV(q_float_null) > 0','having STDDEV(q_double_null) > 0', + 'having TWA(q_int_null) > 0','having TWA(q_bigint_null) > 0','having TWA(q_smallint_null) > 0','having TWA(q_tinyint_null) > 0','having TWA(q_float_null) > 0','having TWA(q_double_null) > 0', + 'having IRATE(q_int_null) > 0','having IRATE(q_bigint_null) > 0','having IRATE(q_smallint_null) > 0','having IRATE(q_tinyint_null) > 0','having IRATE(q_float_null) > 0','having IRATE(q_double_null) > 0', + 'having MIN(q_int_null) > 0','having MIN(q_bigint_null) > 0','having MIN(q_smallint_null) > 0','having MIN(q_tinyint_null) > 0','having MIN(q_float_null) > 0','having MIN(q_double_null) > 0', + 'having MAX(q_int_null) > 0','having MAX(q_bigint_null) > 0','having MAX(q_smallint_null) > 0','having MAX(q_tinyint_null) > 0','having MAX(q_float_null) > 0','having MAX(q_double_null) > 0', + 'having FIRST(q_int_null) > 0','having FIRST(q_bigint_null) > 0','having FIRST(q_smallint_null) > 0','having FIRST(q_tinyint_null) > 0','having FIRST(q_float_null) > 0','having FIRST(q_double_null) > 0', + 'having LAST(q_int_null) > 0','having LAST(q_bigint_null) > 0','having LAST(q_smallint_null) > 0','having LAST(q_tinyint_null) > 0','having LAST(q_float_null) > 0','having LAST(q_double_null) > 0', + 'having APERCENTILE(q_int_null,10) > 0','having APERCENTILE(q_bigint_null,10) > 0','having APERCENTILE(q_smallint_null,10) > 0','having APERCENTILE(q_tinyint_null,10) > 0','having APERCENTILE(q_float_null,10) > 0','having APERCENTILE(q_double_null,10) > 0'] + self.having_not_support = ['having TOP(q_int,10) > 0','having TOP(q_bigint,10) > 0','having TOP(q_smallint,10) > 0','having TOP(q_tinyint,10) > 0','having TOP(q_float,10) > 0','having TOP(q_double,10) > 0','having TOP(q_bool,10) > 0', + 'having BOTTOM(q_int,10) > 0','having BOTTOM(q_bigint,10) > 0','having BOTTOM(q_smallint,10) > 0','having BOTTOM(q_tinyint,10) > 0','having BOTTOM(q_float,10) > 0','having BOTTOM(q_double,10) > 0','having BOTTOM(q_bool,10) > 0', + 'having LEASTSQUARES(q_int) > 0','having LEASTSQUARES(q_bigint) > 0','having LEASTSQUARES(q_smallint) > 0','having LEASTSQUARES(q_tinyint) > 0','having LEASTSQUARES(q_float) > 0','having LEASTSQUARES(q_double) > 0','having LEASTSQUARES(q_bool) > 0', + 'having FIRST(q_bool) > 0','having IRATE(q_bool) > 0','having PERCENTILE(q_bool,10) > 0','having avg(q_bool) > 0','having LAST_ROW(q_bool) > 0','having sum(q_bool) > 0','having STDDEV(q_bool) > 0','having APERCENTILE(q_bool,10) > 0','having TWA(q_bool) > 0','having LAST(q_bool) > 0', + 'having PERCENTILE(q_int,10) > 0','having PERCENTILE(q_bigint,10) > 0','having PERCENTILE(q_smallint,10) > 0','having PERCENTILE(q_tinyint,10) > 0','having PERCENTILE(q_float,10) > 0','having PERCENTILE(q_double,10) > 0', + 'having TOP(q_int_null,10) > 0','having TOP(q_bigint_null,10) > 0','having TOP(q_smallint_null,10) > 0','having TOP(q_tinyint_null,10) > 0','having TOP(q_float_null,10) > 0','having TOP(q_double_null,10) > 0','having TOP(q_bool_null,10) > 0', + 'having BOTTOM(q_int_null,10) > 0','having BOTTOM(q_bigint_null,10) > 0','having BOTTOM(q_smallint_null,10) > 0','having BOTTOM(q_tinyint_null,10) > 0','having BOTTOM(q_float_null,10) > 0','having BOTTOM(q_double_null,10) > 0','having BOTTOM(q_bool_null,10) > 0', + 'having LEASTSQUARES(q_int_null) > 0','having LEASTSQUARES(q_bigint_null) > 0','having LEASTSQUARES(q_smallint_null) > 0','having LEASTSQUARES(q_tinyint_null) > 0','having LEASTSQUARES(q_float_null) > 0','having LEASTSQUARES(q_double_null) > 0','having LEASTSQUARES(q_bool_null) > 0', + 'having FIRST(q_bool_null) > 0','having IRATE(q_bool_null) > 0','having PERCENTILE(q_bool_null,10) > 0','having avg(q_bool_null) > 0','having LAST_ROW(q_bool_null) > 0','having sum(q_bool_null) > 0','having STDDEV(q_bool_null) > 0','having APERCENTILE(q_bool_null,10) > 0','having TWA(q_bool_null) > 0','having LAST(q_bool_null) > 0', + 'having PERCENTILE(q_int_null,10) > 0','having PERCENTILE(q_bigint_null,10) > 0','having PERCENTILE(q_smallint_null,10) > 0','having PERCENTILE(q_tinyint_null,10) > 0','having PERCENTILE(q_float_null,10) > 0','having PERCENTILE(q_double_null,10) > 0'] + self.having_tagnot_support = ['having LAST_ROW(q_int) > 0','having LAST_ROW(q_bigint) > 0','having LAST_ROW(q_smallint) > 0','having LAST_ROW(q_tinyint) > 0','having LAST_ROW(q_float) > 0','having LAST_ROW(q_double) > 0', + 'having LAST_ROW(q_int_null) > 0','having LAST_ROW(q_bigint_null) > 0','having LAST_ROW(q_smallint_null) > 0','having LAST_ROW(q_tinyint_null) > 0','having LAST_ROW(q_float_null) > 0','having LAST_ROW(q_double_null) > 0'] + + self.having_support_j = ['having count(t1.q_int) > 0','having count(t1.q_bigint) > 0','having count(t1.q_smallint) > 0','having count(t1.q_tinyint) > 0','having count(t1.q_float) > 0','having count(t1.q_double) > 0','having count(t1.q_bool) > 0', + 'having avg(t1.q_int) > 0','having avg(t1.q_bigint) > 0','having avg(t1.q_smallint) > 0','having avg(t1.q_tinyint) > 0','having avg(t1.q_float) > 0','having avg(t1.q_double) > 0', + 'having sum(t1.q_int) > 0','having sum(t1.q_bigint) > 0','having sum(t1.q_smallint) > 0','having sum(t1.q_tinyint) > 0','having sum(t1.q_float) > 0','having sum(t1.q_double) > 0', + 'having STDDEV(t1.q_int) > 0','having STDDEV(t1.q_bigint) > 0','having STDDEV(t1.q_smallint) > 0','having STDDEV(t1.q_tinyint) > 0','having STDDEV(t1.q_float) > 0','having STDDEV(t1.q_double) > 0', + 'having TWA(t1.q_int) > 0','having TWA(t1.q_bigint) > 0','having TWA(t1.q_smallint) > 0','having TWA(t1.q_tinyint) > 0','having TWA(t1.q_float) > 0','having TWA(t1.q_double) > 0', + 'having IRATE(t1.q_int) > 0','having IRATE(t1.q_bigint) > 0','having IRATE(t1.q_smallint) > 0','having IRATE(t1.q_tinyint) > 0','having IRATE(t1.q_float) > 0','having IRATE(t1.q_double) > 0', + 'having MIN(t1.q_int) > 0','having MIN(t1.q_bigint) > 0','having MIN(t1.q_smallint) > 0','having MIN(t1.q_tinyint) > 0','having MIN(t1.q_float) > 0','having MIN(t1.q_double) > 0', + 'having MAX(t1.q_int) > 0','having MAX(t1.q_bigint) > 0','having MAX(t1.q_smallint) > 0','having MAX(t1.q_tinyint) > 0','having MAX(t1.q_float) > 0','having MAX(t1.q_double) > 0', + 'having FIRST(t1.q_int) > 0','having FIRST(t1.q_bigint) > 0','having FIRST(t1.q_smallint) > 0','having FIRST(t1.q_tinyint) > 0','having FIRST(t1.q_float) > 0','having FIRST(t1.q_double) > 0', + 'having LAST(t1.q_int) > 0','having LAST(t1.q_bigint) > 0','having LAST(t1.q_smallint) > 0','having LAST(t1.q_tinyint) > 0','having LAST(t1.q_float) > 0','having LAST(t1.q_double) > 0', + 'having APERCENTILE(t1.q_int,10) > 0','having APERCENTILE(t1.q_bigint,10) > 0','having APERCENTILE(t1.q_smallint,10) > 0','having APERCENTILE(t1.q_tinyint,10) > 0','having APERCENTILE(t1.q_float,10) > 0','having APERCENTILE(t1.q_double,10) > 0'] + + # limit offset where + self.limit_where = ['limit 1 offset 1' , 'limit 1' , 'limit 2 offset 1' , 'limit 2', 'limit 12 offset 1' , 'limit 20', 'limit 20 offset 10' , 'limit 200'] + self.limit1_where = ['limit 1 offset 1' , 'limit 1' ] + self.limit_u_where = ['limit 100 offset 10' , 'limit 50' , 'limit 100' , 'limit 10' ] + + # slimit soffset where + self.slimit_where = ['slimit 1 soffset 1' , 'slimit 1' , 'slimit 2 soffset 1' , 'slimit 2'] + self.slimit1_where = ['slimit 2 soffset 1' , 'slimit 1' ] + + # aggregate function include [all:count(*)\avg\sum\stddev ||regualr:twa\irate\leastsquares ||group by tbname:twa\irate\] + # select /*+ para_tables_sort() */function include [all: min\max\first(*)\last(*)\top\bottom\apercentile\last_row(*)(not with interval)\interp(*)(FILL) ||regualr: percentile] + # calculation function include [all:spread\+-*/ ||regualr:diff\derivative ||group by tbname:diff\derivative\] + # **_ns_** express is not support stable, therefore, separated from regular tables + # calc_select_all calc_select_regular calc_select_in_ts calc_select_fill calc_select_not_interval + # calc_aggregate_all calc_aggregate_regular calc_aggregate_groupbytbname + # calc_calculate_all calc_calculate_regular calc_calculate_groupbytbname + + # calc_select_all calc_select_regular calc_select_in_ts calc_select_fill calc_select_not_interval + # select /*+ para_tables_sort() */function include [all: min\max\first(*)\last(*)\top\bottom\apercentile\last_row(*)(not with interval)\interp(*)(FILL) ||regualr: percentile] + + self.calc_select_all = ['bottom(q_int,20)' , 'bottom(q_bigint,20)' , 'bottom(q_smallint,20)' , 'bottom(q_tinyint,20)' ,'bottom(q_float,20)' , 'bottom(q_double,20)' , + 'top(q_int,20)' , 'top(q_bigint,20)' , 'top(q_smallint,20)' ,'top(q_tinyint,20)' ,'top(q_float,20)' ,'top(q_double,20)' , + 'first(q_int)' , 'first(q_bigint)' , 'first(q_smallint)' , 'first(q_tinyint)' , 'first(q_float)' ,'first(q_double)' ,'first(q_binary)' ,'first(q_nchar)' ,'first(q_bool)' ,'first(q_ts)' , + 'last(q_int)' , 'last(q_bigint)' , 'last(q_smallint)' , 'last(q_tinyint)' , 'last(q_float)' ,'last(q_double)' , 'last(q_binary)' ,'last(q_nchar)' ,'last(q_bool)' ,'last(q_ts)' , + 'min(q_int)' , 'min(q_bigint)' , 'min(q_smallint)' , 'min(q_tinyint)' , 'min(q_float)' ,'min(q_double)' , + 'max(q_int)' , 'max(q_bigint)' , 'max(q_smallint)' , 'max(q_tinyint)' ,'max(q_float)' ,'max(q_double)' , + 'apercentile(q_int,20)' , 'apercentile(q_bigint,20)' ,'apercentile(q_smallint,20)' ,'apercentile(q_tinyint,20)' ,'apercentile(q_float,20)' ,'apercentile(q_double,20)' , + 'last_row(q_int)' , 'last_row(q_bigint)' , 'last_row(q_smallint)' , 'last_row(q_tinyint)' , 'last_row(q_float)' , + 'last_row(q_double)' , 'last_row(q_bool)' ,'last_row(q_binary)' ,'last_row(q_nchar)' ,'last_row(q_ts)', + 'bottom(q_int_null,20)' , 'bottom(q_bigint_null,20)' , 'bottom(q_smallint_null,20)' , 'bottom(q_tinyint_null,20)' ,'bottom(q_float_null,20)' , 'bottom(q_double_null,20)' , + 'top(q_int_null,20)' , 'top(q_bigint_null,20)' , 'top(q_smallint_null,20)' ,'top(q_tinyint_null,20)' ,'top(q_float_null,20)' ,'top(q_double_null,20)' , + 'first(q_int_null)' , 'first(q_bigint_null)' , 'first(q_smallint_null)' , 'first(q_tinyint_null)' , 'first(q_float_null)' ,'first(q_double_null)' ,'first(q_binary_null)' ,'first(q_nchar_null)' ,'first(q_bool_null)' ,'first(q_ts_null)' , + 'last(q_int_null)' , 'last(q_bigint_null)' , 'last(q_smallint_null)' , 'last(q_tinyint_null)' , 'last(q_float_null)' ,'last(q_double_null)' , 'last(q_binary_null)' ,'last(q_nchar_null)' ,'last(q_bool_null)' ,'last(q_ts_null)' , + 'min(q_int_null)' , 'min(q_bigint_null)' , 'min(q_smallint_null)' , 'min(q_tinyint_null)' , 'min(q_float_null)' ,'min(q_double_null)' , + 'max(q_int_null)' , 'max(q_bigint_null)' , 'max(q_smallint_null)' , 'max(q_tinyint_null)' ,'max(q_float_null)' ,'max(q_double_null)' , + 'last_row(q_int_null)' , 'last_row(q_bigint_null)' , 'last_row(q_smallint_null)' , 'last_row(q_tinyint_null)' , 'last_row(q_float_null)' , + 'last_row(q_double_null)' , 'last_row(q_bool_null)' ,'last_row(q_binary_null)' ,'last_row(q_nchar_null)' ,'last_row(q_ts_null)', + 'apercentile(q_int_null,20)' , 'apercentile(q_bigint_null,20)' ,'apercentile(q_smallint_null,20)' ,'apercentile(q_tinyint_null,20)' ,'apercentile(q_float_null,20)' ,'apercentile(q_double_null,20)' ,] + + self.calc_select_in_ts = ['bottom(q_int,20)' , 'bottom(q_bigint,20)' , 'bottom(q_smallint,20)' , 'bottom(q_tinyint,20)' ,'bottom(q_float,20)' , 'bottom(q_double,20)' , + 'top(q_int,20)' , 'top(q_bigint,20)' , 'top(q_smallint,20)' ,'top(q_tinyint,20)' ,'top(q_float,20)' ,'top(q_double,20)' , + 'bottom(q_int_null,20)' , 'bottom(q_bigint_null,20)' , 'bottom(q_smallint_null,20)' , 'bottom(q_tinyint_null,20)' ,'bottom(q_float_null,20)' , 'bottom(q_double_null,20)' , + 'top(q_int_null,20)' , 'top(q_bigint_null,20)' , 'top(q_smallint_null,20)' ,'top(q_tinyint_null,20)' ,'top(q_float_null,20)' ,'top(q_double_null,20)' , + 'first(q_int)' , 'first(q_bigint)' , 'first(q_smallint)' , 'first(q_tinyint)' , 'first(q_float)' ,'first(q_double)' ,'first(q_binary)' ,'first(q_nchar)' ,'first(q_bool)' ,'first(q_ts)' , + 'last(q_int)' , 'last(q_bigint)' , 'last(q_smallint)' , 'last(q_tinyint)' , 'last(q_float)' ,'last(q_double)' , 'last(q_binary)' ,'last(q_nchar)' ,'last(q_bool)' ,'last(q_ts)' , + 'first(q_int_null)' , 'first(q_bigint_null)' , 'first(q_smallint_null)' , 'first(q_tinyint_null)' , 'first(q_float_null)' ,'first(q_double_null)' ,'first(q_binary_null)' ,'first(q_nchar_null)' ,'first(q_bool_null)' ,'first(q_ts_null)' , + 'last(q_int_null)' , 'last(q_bigint_null)' , 'last(q_smallint_null)' , 'last(q_tinyint_null)' , 'last(q_float_null)' ,'last(q_double_null)' , 'last(q_binary_null)' ,'last(q_nchar_null)' ,'last(q_bool_null)' ,'last(q_ts_null)' ] + + self.calc_select_in = ['min(q_int)' , 'min(q_bigint)' , 'min(q_smallint)' , 'min(q_tinyint)' , 'min(q_float)' ,'min(q_double)' , + 'max(q_int)' , 'max(q_bigint)' , 'max(q_smallint)' , 'max(q_tinyint)' ,'max(q_float)' ,'max(q_double)' , + 'apercentile(q_int,20)' , 'apercentile(q_bigint,20)' ,'apercentile(q_smallint,20)' ,'apercentile(q_tinyint,20)' ,'apercentile(q_float,20)' ,'apercentile(q_double,20)' , + 'last_row(q_int)' , 'last_row(q_bigint)' , 'last_row(q_smallint)' , 'last_row(q_tinyint)' , 'last_row(q_float)' , + 'last_row(q_double)' , 'last_row(q_bool)' ,'last_row(q_binary)' ,'last_row(q_nchar)' ,'last_row(q_ts)', + 'min(q_int_null)' , 'min(q_bigint_null)' , 'min(q_smallint_null)' , 'min(q_tinyint_null)' , 'min(q_float_null)' ,'min(q_double_null)' , + 'max(q_int_null)' , 'max(q_bigint_null)' , 'max(q_smallint_null)' , 'max(q_tinyint_null)' ,'max(q_float_null)' ,'max(q_double_null)' , + 'apercentile(q_int_null,20)' , 'apercentile(q_bigint_null,20)' ,'apercentile(q_smallint_null,20)' ,'apercentile(q_tinyint_null,20)' ,'apercentile(q_float_null,20)' ,'apercentile(q_double_null,20)' , + 'last_row(q_int_null)' , 'last_row(q_bigint_null)' , 'last_row(q_smallint_null)' , 'last_row(q_tinyint_null)' , 'last_row(q_float_null)' , + 'last_row(q_double_null)' , 'last_row(q_bool_null)' ,'last_row(q_binary_null)' ,'last_row(q_nchar_null)' ,'last_row(q_ts_null)'] + + self.calc_select_not_support_ts = ['first(q_int)' , 'first(q_bigint)' , 'first(q_smallint)' , 'first(q_tinyint)' , 'first(q_float)' ,'first(q_double)' ,'first(q_binary)' ,'first(q_nchar)' ,'first(q_bool)' ,'first(q_ts)' , + 'last(q_int)' , 'last(q_bigint)' , 'last(q_smallint)' , 'last(q_tinyint)' , 'last(q_float)' ,'last(q_double)' , 'last(q_binary)' ,'last(q_nchar)' ,'last(q_bool)' ,'last(q_ts)' , + 'last_row(q_int)' , 'last_row(q_bigint)' , 'last_row(q_smallint)' , 'last_row(q_tinyint)' , 'last_row(q_float)' , + 'last_row(q_double)' , 'last_row(q_bool)' ,'last_row(q_binary)' ,'last_row(q_nchar)' ,'last_row(q_ts)', + 'apercentile(q_int,20)' , 'apercentile(q_bigint,20)' ,'apercentile(q_smallint,20)' ,'apercentile(q_tinyint,20)' ,'apercentile(q_float,20)' ,'apercentile(q_double,20)', + 'first(q_int_null)' , 'first(q_bigint_null)' , 'first(q_smallint_null)' , 'first(q_tinyint_null)' , 'first(q_float_null)' ,'first(q_double_null)' ,'first(q_binary_null)' ,'first(q_nchar_null)' ,'first(q_bool_null)' ,'first(q_ts_null)' , + 'last(q_int_null)' , 'last(q_bigint_null)' , 'last(q_smallint_null)' , 'last(q_tinyint_null)' , 'last(q_float_null)' ,'last(q_double_null)' , 'last(q_binary_null)' ,'last(q_nchar_null)' ,'last(q_bool_null)' ,'last(q_ts_null)' , + 'last_row(q_int_null)' , 'last_row(q_bigint_null)' , 'last_row(q_smallint_null)' , 'last_row(q_tinyint_null)' , 'last_row(q_float_null)' , + 'last_row(q_double_null)' , 'last_row(q_bool_null)' ,'last_row(q_binary_null)' ,'last_row(q_nchar_null)' ,'last_row(q_ts_null)', + 'apercentile(q_int_null,20)' , 'apercentile(q_bigint_null,20)' ,'apercentile(q_smallint_null,20)' ,'apercentile(q_tinyint_null,20)' ,'apercentile(q_float_null,20)' ,'apercentile(q_double_null,20)'] + + self.calc_select_support_ts = ['bottom(q_int,20)' , 'bottom(q_bigint,20)' , 'bottom(q_smallint,20)' , 'bottom(q_tinyint,20)' ,'bottom(q_float,20)' , 'bottom(q_double,20)' , + 'top(q_int,20)' , 'top(q_bigint,20)' , 'top(q_smallint,20)' ,'top(q_tinyint,20)' ,'top(q_float,20)' ,'top(q_double,20)' , + 'bottom(q_int_null,20)' , 'bottom(q_bigint_null,20)' , 'bottom(q_smallint_null,20)' , 'bottom(q_tinyint_null,20)' ,'bottom(q_float_null,20)' , 'bottom(q_double_null,20)' , + 'top(q_int_null,20)' , 'top(q_bigint_null,20)' , 'top(q_smallint_null,20)' ,'top(q_tinyint_null,20)' ,'top(q_float_null,20)' ,'top(q_double_null,20)' , + 'min(q_int)' , 'min(q_bigint)' , 'min(q_smallint)' , 'min(q_tinyint)' , 'min(q_float)' ,'min(q_double)' , + 'max(q_int)' , 'max(q_bigint)' , 'max(q_smallint)' , 'max(q_tinyint)' ,'max(q_float)' ,'max(q_double)' , + 'min(q_int_null)' , 'min(q_bigint_null)' , 'min(q_smallint_null)' , 'min(q_tinyint_null)' , 'min(q_float_null)' ,'min(q_double_null)' , + 'max(q_int_null)' , 'max(q_bigint_null)' , 'max(q_smallint_null)' , 'max(q_tinyint_null)' ,'max(q_float_null)' ,'max(q_double_null)'] + + self.calc_select_regular = [ 'PERCENTILE(q_int,10)' ,'PERCENTILE(q_bigint,20)' , 'PERCENTILE(q_smallint,30)' ,'PERCENTILE(q_tinyint,40)' ,'PERCENTILE(q_float,50)' ,'PERCENTILE(q_double,60)', + 'PERCENTILE(q_int_null,10)' ,'PERCENTILE(q_bigint_null,20)' , 'PERCENTILE(q_smallint_null,30)' ,'PERCENTILE(q_tinyint_null,40)' ,'PERCENTILE(q_float_null,50)' ,'PERCENTILE(q_double_null,60)'] + + + self.calc_select_fill = ['INTERP(q_int)' ,'INTERP(q_bigint)' ,'INTERP(q_smallint)' ,'INTERP(q_tinyint)', 'INTERP(q_float)' ,'INTERP(q_double)'] + self.interp_where = ['ts = now' , 'ts = \'2020-09-13 20:26:40.000\'' , 'ts = \'2020-09-13 20:26:40.009\'' ,'tbname in (\'table_1\') and ts = now' ,'tbname in (\'table_0\' ,\'table_1\',\'table_2\',\'table_3\',\'table_4\',\'table_5\') and ts = \'2020-09-13 20:26:40.000\'','tbname like \'table%\' and ts = \'2020-09-13 20:26:40.002\''] + + #two table join + self.calc_select_in_ts_j = ['bottom(t1.q_int,20)' , 'bottom(t1.q_bigint,20)' , 'bottom(t1.q_smallint,20)' , 'bottom(t1.q_tinyint,20)' ,'bottom(t1.q_float,20)' , 'bottom(t1.q_double,20)' , + 'top(t1.q_int,20)' , 'top(t1.q_bigint,20)' , 'top(t1.q_smallint,20)' ,'top(t1.q_tinyint,20)' ,'top(t1.q_float,20)' ,'top(t1.q_double,20)' , + 'first(t1.q_int)' , 'first(t1.q_bigint)' , 'first(t1.q_smallint)' , 'first(t1.q_tinyint)' , 'first(t1.q_float)' ,'first(t1.q_double)' ,'first(t1.q_binary)' ,'first(t1.q_nchar)' ,'first(t1.q_bool)' ,'first(t1.q_ts)' , + 'last(t1.q_int)' , 'last(t1.q_bigint)' , 'last(t1.q_smallint)' , 'last(t1.q_tinyint)' , 'last(t1.q_float)' ,'last(t1.q_double)' , 'last(t1.q_binary)' ,'last(t1.q_nchar)' ,'last(t1.q_bool)' ,'last(t1.q_ts)' , + 'bottom(t2.q_int,20)' , 'bottom(t2.q_bigint,20)' , 'bottom(t2.q_smallint,20)' , 'bottom(t2.q_tinyint,20)' ,'bottom(t2.q_float,20)' , 'bottom(t2.q_double,20)' , + 'top(t2.q_int,20)' , 'top(t2.q_bigint,20)' , 'top(t2.q_smallint,20)' ,'top(t2.q_tinyint,20)' ,'top(t2.q_float,20)' ,'top(t2.q_double,20)' , + 'first(t2.q_int)' , 'first(t2.q_bigint)' , 'first(t2.q_smallint)' , 'first(t2.q_tinyint)' , 'first(t2.q_float)' ,'first(t2.q_double)' ,'first(t2.q_binary)' ,'first(t2.q_nchar)' ,'first(t2.q_bool)' ,'first(t2.q_ts)' , + 'last(t2.q_int)' , 'last(t2.q_bigint)' , 'last(t2.q_smallint)' , 'last(t2.q_tinyint)' , 'last(t2.q_float)' ,'last(t2.q_double)' , 'last(t2.q_binary)' ,'last(t2.q_nchar)' ,'last(t2.q_bool)' ,'last(t2.q_ts)', + 'bottom(t1.q_int_null,20)' , 'bottom(t1.q_bigint_null,20)' , 'bottom(t1.q_smallint_null,20)' , 'bottom(t1.q_tinyint_null,20)' ,'bottom(t1.q_float_null,20)' , 'bottom(t1.q_double_null,20)' , + 'top(t1.q_int_null,20)' , 'top(t1.q_bigint_null,20)' , 'top(t1.q_smallint_null,20)' ,'top(t1.q_tinyint_null,20)' ,'top(t1.q_float_null,20)' ,'top(t1.q_double_null,20)' , + 'first(t1.q_int_null)' , 'first(t1.q_bigint_null)' , 'first(t1.q_smallint_null)' , 'first(t1.q_tinyint_null)' , 'first(t1.q_float_null)' ,'first(t1.q_double_null)' ,'first(t1.q_binary_null)' ,'first(t1.q_nchar_null))' ,'first(t1.q_bool_null)' ,'first(t1.q_ts_null)' , + 'last(t1.q_int_null)' , 'last(t1.q_bigint_null)' , 'last(t1.q_smallint_null)' , 'last(t1.q_tinyint_null)' , 'last(t1.q_float_null)' ,'last(t1.q_double_null)' , 'last(t1.q_binary_null)' ,'last(t1.q_nchar_null))' ,'last(t1.q_bool_null)' ,'last(t1.q_ts_null)' , + 'bottom(t2.q_int_null,20)' , 'bottom(t2.q_bigint_null,20)' , 'bottom(t2.q_smallint_null,20)' , 'bottom(t2.q_tinyint_null,20)' ,'bottom(t2.q_float_null,20)' , 'bottom(t2.q_double_null,20)' , + 'top(t2.q_int_null,20)' , 'top(t2.q_bigint_null,20)' , 'top(t2.q_smallint_null,20)' ,'top(t2.q_tinyint_null,20)' ,'top(t2.q_float_null,20)' ,'top(t2.q_double_null,20)' , + 'first(t2.q_int_null)' , 'first(t2.q_bigint_null)' , 'first(t2.q_smallint_null)' , 'first(t2.q_tinyint_null)' , 'first(t2.q_float_null)' ,'first(t2.q_double_null)' ,'first(t2.q_binary_null)' ,'first(t2.q_nchar_null))' ,'first(t2.q_bool_null)' ,'first(t2.q_ts_null)' , + 'last(t2.q_int_null)' , 'last(t2.q_bigint_null)' , 'last(t2.q_smallint_null)' , 'last(t2.q_tinyint_null)' , 'last(t2.q_float_null)' ,'last(t2.q_double_null)' , 'last(t2.q_binary_null)' ,'last(t2.q_nchar_null))' ,'last(t2.q_bool_null)' ,'last(t2.q_ts_null)'] + + self.calc_select_in_support_ts_j = ['bottom(t1.q_int,20)' , 'bottom(t1.q_bigint,20)' , 'bottom(t1.q_smallint,20)' , 'bottom(t1.q_tinyint,20)' ,'bottom(t1.q_float,20)' , 'bottom(t1.q_double,20)' , + 'top(t1.q_int,20)' , 'top(t1.q_bigint,20)' , 'top(t1.q_smallint,20)' ,'top(t1.q_tinyint,20)' ,'top(t1.q_float,20)' ,'top(t1.q_double,20)' , + 'min(t1.q_int)' , 'min(t1.q_bigint)' , 'min(t1.q_smallint)' , 'min(t1.q_tinyint)' , 'min(t1.q_float)' ,'min(t1.q_double)' , + 'max(t1.q_int)' , 'max(t1.q_bigint)' , 'max(t1.q_smallint)' , 'max(t1.q_tinyint)' ,'max(t1.q_float)' ,'max(t1.q_double)' , + 'bottom(t2.q_int,20)' , 'bottom(t2.q_bigint,20)' , 'bottom(t2.q_smallint,20)' , 'bottom(t2.q_tinyint,20)' ,'bottom(t2.q_float,20)' , 'bottom(t2.q_double,20)' , + 'top(t2.q_int,20)' , 'top(t2.q_bigint,20)' , 'top(t2.q_smallint,20)' ,'top(t2.q_tinyint,20)' ,'top(t2.q_float,20)' ,'top(t2.q_double,20)' , + 'min(t2.q_int)' , 'min(t2.q_bigint)' , 'min(t2.q_smallint)' , 'min(t2.q_tinyint)' , 'min(t2.q_float)' ,'min(t2.q_double)' , + 'max(t2.q_int)' , 'max(t2.q_bigint)' , 'max(t2.q_smallint)' , 'max(t2.q_tinyint)' ,'max(t2.q_float)' ,'max(t2.q_double)' , + 'bottom(t1.q_int_null,20)' , 'bottom(t1.q_bigint_null,20)' , 'bottom(t1.q_smallint_null,20)' , 'bottom(t1.q_tinyint_null,20)' ,'bottom(t1.q_float_null,20)' , 'bottom(t1.q_double_null,20)' , + 'top(t1.q_int_null,20)' , 'top(t1.q_bigint_null,20)' , 'top(t1.q_smallint_null,20)' ,'top(t1.q_tinyint_null,20)' ,'top(t1.q_float_null,20)' ,'top(t1.q_double_null,20)' , + 'bottom(t2.q_int_null,20)' , 'bottom(t2.q_bigint_null,20)' , 'bottom(t2.q_smallint_null,20)' , 'bottom(t2.q_tinyint_null,20)' ,'bottom(t2.q_float_null,20)' , 'bottom(t2.q_double_null,20)' , + 'top(t2.q_int_null,20)' , 'top(t2.q_bigint_null,20)' , 'top(t2.q_smallint_null,20)' ,'top(t2.q_tinyint_null,20)' ,'top(t2.q_float_null,20)' ,'top(t2.q_double_null,20)' , + 'min(t1.q_int_null)' , 'min(t1.q_bigint_null)' , 'min(t1.q_smallint_null)' , 'min(t1.q_tinyint_null)' , 'min(t1.q_float_null)' ,'min(t1.q_double_null)' , + 'max(t1.q_int_null)' , 'max(t1.q_bigint_null)' , 'max(t1.q_smallint_null)' , 'max(t1.q_tinyint_null)' ,'max(t1.q_float_null)' ,'max(t1.q_double_null)' , + 'min(t2.q_int_null)' , 'min(t2.q_bigint_null)' , 'min(t2.q_smallint_null)' , 'min(t2.q_tinyint_null)' , 'min(t2.q_float_null)' ,'min(t2.q_double_null)' , + 'max(t2.q_int_null)' , 'max(t2.q_bigint_null)' , 'max(t2.q_smallint_null)' , 'max(t2.q_tinyint_null)' ,'max(t2.q_float_null)' ,'max(t2.q_double_null)' ] + + self.calc_select_in_not_support_ts_j = ['apercentile(t1.q_int,20)' , 'apercentile(t1.q_bigint,20)' ,'apercentile(t1.q_smallint,20)' ,'apercentile(t1.q_tinyint,20)' ,'apercentile(t1.q_float,20)' ,'apercentile(t1.q_double,20)' , + 'apercentile(t1.q_int_null,20)' , 'apercentile(t1.q_bigint_null,20)' ,'apercentile(t1.q_smallint_null,20)' ,'apercentile(t1.q_tinyint_null,20)' ,'apercentile(t1.q_float_null,20)' ,'apercentile(t1.q_double_null,20)' , + 'last_row(t1.q_int)' , 'last_row(t1.q_bigint)' , 'last_row(t1.q_smallint)' , 'last_row(t1.q_tinyint)' , 'last_row(t1.q_float)' , + 'last_row(t1.q_double)' , 'last_row(t1.q_bool)' ,'last_row(t1.q_binary)' ,'last_row(t1.q_nchar)' ,'last_row(t1.q_ts)' , + 'last_row(t1.q_int_null)' , 'last_row(t1.q_bigint_null)' , 'last_row(t1.q_smallint_null)' , 'last_row(t1.q_tinyint_null)' , 'last_row(t1.q_float_null)' , + 'last_row(t1.q_double_null)' , 'last_row(t1.q_bool_null)' ,'last_row(t1.q_binary_null)' ,'last_row(t1.q_nchar_null)' ,'last_row(t1.q_ts_null)' , + 'apercentile(t2.q_int,20)' , 'apercentile(t2.q_bigint,20)' ,'apercentile(t2.q_smallint,20)' ,'apercentile(t2.q_tinyint,20)' ,'apercentile(t2.q_float,20)' ,'apercentile(t2.q_double,20)' , + 'apercentile(t2.q_int_null,20)' , 'apercentile(t2.q_bigint_null,20)' ,'apercentile(t2.q_smallint_null,20)' ,'apercentile(t2.q_tinyint_null,20)' ,'apercentile(t2.q_float_null,20)' ,'apercentile(t2.q_double_null,20)' , + 'last_row(t2.q_int)' , 'last_row(t2.q_bigint)' , 'last_row(t2.q_smallint)' , 'last_row(t2.q_tinyint)' , 'last_row(t2.q_float)' , + 'last_row(t2.q_double)' , 'last_row(t2.q_bool)' ,'last_row(t2.q_binary)' ,'last_row(t2.q_nchar)' ,'last_row(t2.q_ts)', + 'last_row(t2.q_int_null)' , 'last_row(t2.q_bigint_null)' , 'last_row(t2.q_smallint_null)' , 'last_row(t2.q_tinyint_null)' , 'last_row(t2.q_float_null)' , + 'last_row(t2.q_double_null)' , 'last_row(t2.q_bool_null)' ,'last_row(t2.q_binary_null)' ,'last_row(t2.q_nchar_null)' ,'last_row(t2.q_ts_null)'] + + self.calc_select_in_j = ['min(t1.q_int)' , 'min(t1.q_bigint)' , 'min(t1.q_smallint)' , 'min(t1.q_tinyint)' , 'min(t1.q_float)' ,'min(t1.q_double)' , + 'max(t1.q_int)' , 'max(t1.q_bigint)' , 'max(t1.q_smallint)' , 'max(t1.q_tinyint)' ,'max(t1.q_float)' ,'max(t1.q_double)' , + 'apercentile(t1.q_int,20)' , 'apercentile(t1.q_bigint,20)' ,'apercentile(t1.q_smallint,20)' ,'apercentile(t1.q_tinyint,20)' ,'apercentile(t1.q_float,20)' ,'apercentile(t1.q_double,20)' , + 'min(t1.q_int_null)' , 'min(t1.q_bigint_null)' , 'min(t1.q_smallint_null)' , 'min(t1.q_tinyint_null)' , 'min(t1.q_float_null)' ,'min(t1.q_double_null)' , + 'max(t1.q_int_null)' , 'max(t1.q_bigint_null)' , 'max(t1.q_smallint_null)' , 'max(t1.q_tinyint_null)' ,'max(t1.q_float_null)' ,'max(t1.q_double_null)' , + 'apercentile(t1.q_int_null,20)' , 'apercentile(t1.q_bigint_null,20)' ,'apercentile(t1.q_smallint_null,20)' ,'apercentile(t1.q_tinyint_null,20)' ,'apercentile(t1.q_float_null,20)' ,'apercentile(t1.q_double_null,20)' , + 'last_row(t1.q_int)' , 'last_row(t1.q_bigint)' , 'last_row(t1.q_smallint)' , 'last_row(t1.q_tinyint)' , 'last_row(t1.q_float)' , + 'last_row(t1.q_double)' , 'last_row(t1.q_bool)' ,'last_row(t1.q_binary)' ,'last_row(t1.q_nchar)' ,'last_row(t1.q_ts)' , + 'min(t2.q_int)' , 'min(t2.q_bigint)' , 'min(t2.q_smallint)' , 'min(t2.q_tinyint)' , 'min(t2.q_float)' ,'min(t2.q_double)' , + 'max(t2.q_int)' , 'max(t2.q_bigint)' , 'max(t2.q_smallint)' , 'max(t2.q_tinyint)' ,'max(t2.q_float)' ,'max(t2.q_double)' , + 'last_row(t1.q_int_null)' , 'last_row(t1.q_bigint_null)' , 'last_row(t1.q_smallint_null)' , 'last_row(t1.q_tinyint_null)' , 'last_row(t1.q_float_null)' , + 'last_row(t1.q_double_null)' , 'last_row(t1.q_bool_null)' ,'last_row(t1.q_binary_null)' ,'last_row(t1.q_nchar_null)' ,'last_row(t1.q_ts_null)' , + 'min(t2.q_int_null)' , 'min(t2.q_bigint_null)' , 'min(t2.q_smallint_null)' , 'min(t2.q_tinyint_null)' , 'min(t2.q_float_null)' ,'min(t2.q_double_null)' , + 'max(t2.q_int_null)' , 'max(t2.q_bigint_null)' , 'max(t2.q_smallint_null)' , 'max(t2.q_tinyint_null)' ,'max(t2.q_float_null)' ,'max(t2.q_double_null)' , + 'apercentile(t2.q_int,20)' , 'apercentile(t2.q_bigint,20)' ,'apercentile(t2.q_smallint,20)' ,'apercentile(t2.q_tinyint,20)' ,'apercentile(t2.q_float,20)' ,'apercentile(t2.q_double,20)' , + 'apercentile(t2.q_int_null,20)' , 'apercentile(t2.q_bigint_null,20)' ,'apercentile(t2.q_smallint_null,20)' ,'apercentile(t2.q_tinyint_null,20)' ,'apercentile(t2.q_float_null,20)' ,'apercentile(t2.q_double_null,20)' , + 'last_row(t2.q_int)' , 'last_row(t2.q_bigint)' , 'last_row(t2.q_smallint)' , 'last_row(t2.q_tinyint)' , 'last_row(t2.q_float)' , + 'last_row(t2.q_double)' , 'last_row(t2.q_bool)' ,'last_row(t2.q_binary)' ,'last_row(t2.q_nchar)' ,'last_row(t2.q_ts)', + 'last_row(t2.q_int_null)' , 'last_row(t2.q_bigint_null)' , 'last_row(t2.q_smallint_null)' , 'last_row(t2.q_tinyint_null)' , 'last_row(t2.q_float_null)' , + 'last_row(t2.q_double_null)' , 'last_row(t2.q_bool_null)' ,'last_row(t2.q_binary_null)' ,'last_row(t2.q_nchar_null)' ,'last_row(t2.q_ts_null)'] + self.calc_select_all_j = self.calc_select_in_ts_j + self.calc_select_in_j + + self.calc_select_regular_j = [ 'PERCENTILE(t1.q_int,10)' ,'PERCENTILE(t1.q_bigint,20)' , 'PERCENTILE(t1.q_smallint,30)' ,'PERCENTILE(t1.q_tinyint,40)' ,'PERCENTILE(t1.q_float,50)' ,'PERCENTILE(t1.q_double,60)' , + 'PERCENTILE(t2.q_int,10)' ,'PERCENTILE(t2.q_bigint,20)' , 'PERCENTILE(t2.q_smallint,30)' ,'PERCENTILE(t2.q_tinyint,40)' ,'PERCENTILE(t2.q_float,50)' ,'PERCENTILE(t2.q_double,60)', + 'PERCENTILE(t1.q_int_null,10)' ,'PERCENTILE(t1.q_bigint_null,20)' , 'PERCENTILE(t1.q_smallint_null,30)' ,'PERCENTILE(t1.q_tinyint_null,40)' ,'PERCENTILE(t1.q_float_null,50)' ,'PERCENTILE(t1.q_double_null,60)' , + 'PERCENTILE(t2.q_int_null,10)' ,'PERCENTILE(t2.q_bigint_null,20)' , 'PERCENTILE(t2.q_smallint_null,30)' ,'PERCENTILE(t2.q_tinyint_null,40)' ,'PERCENTILE(t2.q_float_null,50)' ,'PERCENTILE(t2.q_double_null,60)'] + + + self.calc_select_fill_j = ['INTERP(t1.q_int)' ,'INTERP(t1.q_bigint)' ,'INTERP(t1.q_smallint)' ,'INTERP(t1.q_tinyint)', 'INTERP(t1.q_float)' ,'INTERP(t1.q_double)' , + 'INTERP(t2.q_int)' ,'INTERP(t2.q_bigint)' ,'INTERP(t2.q_smallint)' ,'INTERP(t2.q_tinyint)', 'INTERP(t2.q_float)' ,'INTERP(t2.q_double)'] + self.interp_where_j = ['t1.ts = now' , 't1.ts = \'2020-09-13 20:26:40.000\'' , 't1.ts = \'2020-09-13 20:26:40.009\'' ,'t2.ts = now' , 't2.ts = \'2020-09-13 20:26:40.000\'' , 't2.ts = \'2020-09-13 20:26:40.009\'' , + 't1.tbname in (\'table_1\') and t1.ts = now' ,'t1.tbname in (\'table_0\' ,\'table_1\',\'table_2\',\'table_3\',\'table_4\',\'table_5\') and t1.ts = \'2020-09-13 20:26:40.000\'','t1.tbname like \'table%\' and t1.ts = \'2020-09-13 20:26:40.002\'', + 't2.tbname in (\'table_1\') and t2.ts = now' ,'t2.tbname in (\'table_0\' ,\'table_1\',\'table_2\',\'table_3\',\'table_4\',\'table_5\') and t2.ts = \'2020-09-13 20:26:40.000\'','t2.tbname like \'table%\' and t2.ts = \'2020-09-13 20:26:40.002\''] + + # calc_aggregate_all calc_aggregate_regular calc_aggregate_groupbytbname APERCENTILE\PERCENTILE + # aggregate function include [all:count(*)\avg\sum\stddev ||regualr:twa\irate\leastsquares ||group by tbname:twa\irate\] + self.calc_aggregate_all = ['count(*)' , 'count(q_int)' ,'count(q_bigint)' , 'count(q_smallint)' ,'count(q_tinyint)' ,'count(q_float)' , + 'count(q_double)' ,'count(q_binary)' ,'count(q_nchar)' ,'count(q_bool)' ,'count(q_ts)' , + 'avg(q_int)' ,'avg(q_bigint)' , 'avg(q_smallint)' ,'avg(q_tinyint)' ,'avg(q_float)' ,'avg(q_double)' , + 'sum(q_int)' ,'sum(q_bigint)' , 'sum(q_smallint)' ,'sum(q_tinyint)' ,'sum(q_float)' ,'sum(q_double)' , + 'STDDEV(q_int)' ,'STDDEV(q_bigint)' , 'STDDEV(q_smallint)' ,'STDDEV(q_tinyint)' ,'STDDEV(q_float)' ,'STDDEV(q_double)', + 'APERCENTILE(q_int,10)' ,'APERCENTILE(q_bigint,20)' , 'APERCENTILE(q_smallint,30)' ,'APERCENTILE(q_tinyint,40)' ,'APERCENTILE(q_float,50)' ,'APERCENTILE(q_double,60)', + 'count(q_int_null)' ,'count(q_bigint_null)' , 'count(q_smallint_null)' ,'count(q_tinyint_null)' ,'count(q_float_null)' , + 'count(q_double_null)' ,'count(q_binary_null)' ,'count(q_nchar_null)' ,'count(q_bool_null)' ,'count(q_ts_null)' , + 'avg(q_int_null)' ,'avg(q_bigint_null)' , 'avg(q_smallint_null)' ,'avg(q_tinyint_null)' ,'avg(q_float_null)' ,'avg(q_double_null)' , + 'sum(q_int_null)' ,'sum(q_bigint_null)' , 'sum(q_smallint_null)' ,'sum(q_tinyint_null)' ,'sum(q_float_null)' ,'sum(q_double_null)' , + 'STDDEV(q_int_null)' ,'STDDEV(q_bigint_null)' , 'STDDEV(q_smallint_null)' ,'STDDEV(q_tinyint_null)' ,'STDDEV(q_float_null)' ,'STDDEV(q_double_null)', + 'APERCENTILE(q_int_null,10)' ,'APERCENTILE(q_bigint_null,20)' , 'APERCENTILE(q_smallint_null,30)' ,'APERCENTILE(q_tinyint_null,40)' ,'APERCENTILE(q_float_null,50)' ,'APERCENTILE(q_double_null,60)'] + + self.calc_aggregate_regular = ['twa(q_int)' ,'twa(q_bigint)' , 'twa(q_smallint)' ,'twa(q_tinyint)' ,'twa (q_float)' ,'twa(q_double)' , + 'IRATE(q_int)' ,'IRATE(q_bigint)' , 'IRATE(q_smallint)' ,'IRATE(q_tinyint)' ,'IRATE (q_float)' ,'IRATE(q_double)' , + 'twa(q_int_null)' ,'twa(q_bigint_null)' , 'twa(q_smallint_null)' ,'twa(q_tinyint_null)' ,'twa (q_float_null)' ,'twa(q_double_null)' , + 'IRATE(q_int_null)' ,'IRATE(q_bigint_null)' , 'IRATE(q_smallint_null)' ,'IRATE(q_tinyint_null)' ,'IRATE (q_float_null)' ,'IRATE(q_double_null)' , + 'LEASTSQUARES(q_int,15,3)' , 'LEASTSQUARES(q_bigint,10,1)' , 'LEASTSQUARES(q_smallint,20,3)' ,'LEASTSQUARES(q_tinyint,10,4)' ,'LEASTSQUARES(q_float,6,4)' ,'LEASTSQUARES(q_double,3,1)' , + 'PERCENTILE(q_int,10)' ,'PERCENTILE(q_bigint,20)' , 'PERCENTILE(q_smallint,30)' ,'PERCENTILE(q_tinyint,40)' ,'PERCENTILE(q_float,50)' ,'PERCENTILE(q_double,60)', + 'LEASTSQUARES(q_int_null,15,3)' , 'LEASTSQUARES(q_bigint_null,10,1)' , 'LEASTSQUARES(q_smallint_null,20,3)' ,'LEASTSQUARES(q_tinyint_null,10,4)' ,'LEASTSQUARES(q_float_null,6,4)' ,'LEASTSQUARES(q_double_null,3,1)' , + 'PERCENTILE(q_int_null,10)' ,'PERCENTILE(q_bigint_null,20)' , 'PERCENTILE(q_smallint_null,30)' ,'PERCENTILE(q_tinyint_null,40)' ,'PERCENTILE(q_float_null,50)' ,'PERCENTILE(q_double_null,60)'] + + self.calc_aggregate_groupbytbname = ['twa(q_int)' ,'twa(q_bigint)' , 'twa(q_smallint)' ,'twa(q_tinyint)' ,'twa (q_float)' ,'twa(q_double)' , + 'IRATE(q_int)' ,'IRATE(q_bigint)' , 'IRATE(q_smallint)' ,'IRATE(q_tinyint)' ,'IRATE (q_float)' ,'IRATE(q_double)', + 'twa(q_int_null)' ,'twa(q_bigint_null)' , 'twa(q_smallint_null)' ,'twa(q_tinyint_null)' ,'twa (q_float_null)' ,'twa(q_double_null)' , + 'IRATE(q_int_null)' ,'IRATE(q_bigint_null)' , 'IRATE(q_smallint_null)' ,'IRATE(q_tinyint_null)' ,'IRATE (q_float_null)' ,'IRATE(q_double_null)'] + + #two table join + self.calc_aggregate_all_j = ['count(t1.*)' , 'count(t1.q_int)' ,'count(t1.q_bigint)' , 'count(t1.q_smallint)' ,'count(t1.q_tinyint)' ,'count(t1.q_float)' , + 'count(t1.q_double)' ,'count(t1.q_binary)' ,'count(t1.q_nchar)' ,'count(t1.q_bool)' ,'count(t1.q_ts)' , + 'avg(t1.q_int)' ,'avg(t1.q_bigint)' , 'avg(t1.q_smallint)' ,'avg(t1.q_tinyint)' ,'avg(t1.q_float)' ,'avg(t1.q_double)' , + 'sum(t1.q_int)' ,'sum(t1.q_bigint)' , 'sum(t1.q_smallint)' ,'sum(t1.q_tinyint)' ,'sum(t1.q_float)' ,'sum(t1.q_double)' , + 'STDDEV(t1.q_int)' ,'STDDEV(t1.q_bigint)' , 'STDDEV(t1.q_smallint)' ,'STDDEV(t1.q_tinyint)' ,'STDDEV(t1.q_float)' ,'STDDEV(t1.q_double)', + 'APERCENTILE(t1.q_int,10)' ,'APERCENTILE(t1.q_bigint,20)' , 'APERCENTILE(t1.q_smallint,30)' ,'APERCENTILE(t1.q_tinyint,40)' ,'APERCENTILE(t1.q_float,50)' ,'APERCENTILE(t1.q_double,60)' , + 'count(t1.q_int_null)' ,'count(t1.q_bigint_null)' , 'count(t1.q_smallint_null)' ,'count(t1.q_tinyint_null)' ,'count(t1.q_float_null)' , + 'count(t1.q_double_null)' ,'count(t1.q_binary_null)' ,'count(t1.q_nchar_null)' ,'count(t1.q_bool_null)' ,'count(t1.q_ts_null)' , + 'avg(t1.q_int_null)' ,'avg(t1.q_bigint_null)' , 'avg(t1.q_smallint_null)' ,'avg(t1.q_tinyint_null)' ,'avg(t1.q_float_null)' ,'avg(t1.q_double_null)' , + 'sum(t1.q_int_null)' ,'sum(t1.q_bigint_null)' , 'sum(t1.q_smallint_null)' ,'sum(t1.q_tinyint_null)' ,'sum(t1.q_float_null)' ,'sum(t1.q_double_null)' , + 'STDDEV(t1.q_int_null)' ,'STDDEV(t1.q_bigint_null)' , 'STDDEV(t1.q_smallint_null)' ,'STDDEV(t1.q_tinyint_null)' ,'STDDEV(t1.q_float_null)' ,'STDDEV(t1.q_double_null)', + 'APERCENTILE(t1.q_int_null,10)' ,'APERCENTILE(t1.q_bigint_null,20)' , 'APERCENTILE(t1.q_smallint_null,30)' ,'APERCENTILE(t1.q_tinyint_null,40)' ,'APERCENTILE(t1.q_float_null,50)' ,'APERCENTILE(t1.q_double,60)' , + 'count(t2.*)' , 'count(t2.q_int)' ,'count(t2.q_bigint)' , 'count(t2.q_smallint)' ,'count(t2.q_tinyint)' ,'count(t2.q_float)' , + 'count(t2.q_double)' ,'count(t2.q_binary)' ,'count(t2.q_nchar)' ,'count(t2.q_bool)' ,'count(t2.q_ts)' , + 'avg(t2.q_int)' ,'avg(t2.q_bigint)' , 'avg(t2.q_smallint)' ,'avg(t2.q_tinyint)' ,'avg(t2.q_float)' ,'avg(t2.q_double)' , + 'sum(t2.q_int)' ,'sum(t2.q_bigint)' , 'sum(t2.q_smallint)' ,'sum(t2.q_tinyint)' ,'sum(t2.q_float)' ,'sum(t2.q_double)' , + 'STDDEV(t2.q_int)' ,'STDDEV(t2.q_bigint)' , 'STDDEV(t2.q_smallint)' ,'STDDEV(t2.q_tinyint)' ,'STDDEV(t2.q_float)' ,'STDDEV(t2.q_double)', + 'APERCENTILE(t2.q_int,10)' ,'APERCENTILE(t2.q_bigint,20)' , 'APERCENTILE(t2.q_smallint,30)' ,'APERCENTILE(t2.q_tinyint,40)' ,'APERCENTILE(t2.q_float,50)' ,'APERCENTILE(t2.q_double,60)', + 'count(t2.q_int_null)' ,'count(t2.q_bigint_null)' , 'count(t2.q_smallint_null)' ,'count(t2.q_tinyint_null)' ,'count(t2.q_float_null)' , + 'count(t2.q_double_null)' ,'count(t2.q_binary_null)' ,'count(t2.q_nchar_null)' ,'count(t2.q_bool_null)' ,'count(t2.q_ts_null)' , + 'avg(t2.q_int_null)' ,'avg(t2.q_bigint_null)' , 'avg(t2.q_smallint_null)' ,'avg(t2.q_tinyint_null)' ,'avg(t2.q_float_null)' ,'avg(t2.q_double_null)' , + 'sum(t2.q_int_null)' ,'sum(t2.q_bigint_null)' , 'sum(t2.q_smallint_null)' ,'sum(t2.q_tinyint_null)' ,'sum(t2.q_float_null)' ,'sum(t2.q_double_null)' , + 'STDDEV(t2.q_int_null)' ,'STDDEV(t2.q_bigint_null)' , 'STDDEV(t2.q_smallint_null)' ,'STDDEV(t2.q_tinyint_null)' ,'STDDEV(t2.q_float_null)' ,'STDDEV(t2.q_double_null)', + 'APERCENTILE(t2.q_int_null,10)' ,'APERCENTILE(t2.q_bigint_null,20)' , 'APERCENTILE(t2.q_smallint_null,30)' ,'APERCENTILE(t2.q_tinyint_null,40)' ,'APERCENTILE(t2.q_float_null,50)' ,'APERCENTILE(t2.q_double,60)'] + + self.calc_aggregate_regular_j = ['twa(t1.q_int)' ,'twa(t1.q_bigint)' , 'twa(t1.q_smallint)' ,'twa(t1.q_tinyint)' ,'twa (t1.q_float)' ,'twa(t1.q_double)' , + 'IRATE(t1.q_int)' ,'IRATE(t1.q_bigint)' , 'IRATE(t1.q_smallint)' ,'IRATE(t1.q_tinyint)' ,'IRATE (t1.q_float)' ,'IRATE(t1.q_double)' , + 'LEASTSQUARES(t1.q_int,15,3)' , 'LEASTSQUARES(t1.q_bigint,10,1)' , 'LEASTSQUARES(t1.q_smallint,20,3)' ,'LEASTSQUARES(t1.q_tinyint,10,4)' ,'LEASTSQUARES(t1.q_float,6,4)' ,'LEASTSQUARES(t1.q_double,3,1)' , + 'twa(t2.q_int)' ,'twa(t2.q_bigint)' , 'twa(t2.q_smallint)' ,'twa(t2.q_tinyint)' ,'twa (t2.q_float)' ,'twa(t2.q_double)' , + 'IRATE(t2.q_int)' ,'IRATE(t2.q_bigint)' , 'IRATE(t2.q_smallint)' ,'IRATE(t2.q_tinyint)' ,'IRATE (t2.q_float)' ,'IRATE(t2.q_double)', + 'LEASTSQUARES(t2.q_int,15,3)' , 'LEASTSQUARES(t2.q_bigint,10,1)' , 'LEASTSQUARES(t2.q_smallint,20,3)' ,'LEASTSQUARES(t2.q_tinyint,10,4)' ,'LEASTSQUARES(t2.q_float,6,4)' ,'LEASTSQUARES(t2.q_double,3,1)' , + 'twa(t1.q_int_null)' ,'twa(t1.q_bigint_null)' , 'twa(t1.q_smallint_null)' ,'twa(t1.q_tinyint_null)' ,'twa (t1.q_float_null)' ,'twa(t1.q_double_null)' , + 'IRATE(t1.q_int_null)' ,'IRATE(t1.q_bigint_null)' , 'IRATE(t1.q_smallint_null)' ,'IRATE(t1.q_tinyint_null)' ,'IRATE (t1.q_float_null)' ,'IRATE(t1.q_double_null)' , + 'LEASTSQUARES(t1.q_int_null,15,3)' , 'LEASTSQUARES(t1.q_bigint_null,10,1)' , 'LEASTSQUARES(t1.q_smallint_null,20,3)' ,'LEASTSQUARES(t1.q_tinyint_null,10,4)' ,'LEASTSQUARES(t1.q_float_null,6,4)' ,'LEASTSQUARES(t1.q_double_null,3,1)' , + 'twa(t2.q_int_null)' ,'twa(t2.q_bigint_null)' , 'twa(t2.q_smallint_null)' ,'twa(t2.q_tinyint_null)' ,'twa (t2.q_float_null)' ,'twa(t2.q_double_null)' , + 'IRATE(t2.q_int_null)' ,'IRATE(t2.q_bigint_null)' , 'IRATE(t2.q_smallint_null)' ,'IRATE(t2.q_tinyint_null)' ,'IRATE (t2.q_float_null)' ,'IRATE(t2.q_double_null)', + 'LEASTSQUARES(t2.q_int_null,15,3)' , 'LEASTSQUARES(t2.q_bigint_null,10,1)' , 'LEASTSQUARES(t2.q_smallint_null,20,3)' ,'LEASTSQUARES(t2.q_tinyint_null,10,4)' ,'LEASTSQUARES(t2.q_float_null,6,4)' ,'LEASTSQUARES(t2.q_double_null,3,1)' ] + + self.calc_aggregate_groupbytbname_j = ['twa(t1.q_int)' ,'twa(t1.q_bigint)' , 'twa(t1.q_smallint)' ,'twa(t1.q_tinyint)' ,'twa (t1.q_float)' ,'twa(t1.q_double)' , + 'IRATE(t1.q_int)' ,'IRATE(t1.q_bigint)' , 'IRATE(t1.q_smallint)' ,'IRATE(t1.q_tinyint)' ,'IRATE (t1.q_float)' ,'IRATE(t1.q_double)' , + 'twa(t2.q_int)' ,'twa(t2.q_bigint)' , 'twa(t2.q_smallint)' ,'twa(t2.q_tinyint)' ,'twa (t2.q_float)' ,'twa(t2.q_double)' , + 'IRATE(t2.q_int)' ,'IRATE(t2.q_bigint)' , 'IRATE(t2.q_smallint)' ,'IRATE(t2.q_tinyint)' ,'IRATE (t2.q_float)' ,'IRATE(t2.q_double)' , + 'twa(t1.q_int_null)' ,'twa(t1.q_bigint_null)' , 'twa(t1.q_smallint_null)' ,'twa(t1.q_tinyint_null)' ,'twa (t1.q_float_null)' ,'twa(t1.q_double_null)' , + 'IRATE(t1.q_int_null)' ,'IRATE(t1.q_bigint_null)' , 'IRATE(t1.q_smallint_null)' ,'IRATE(t1.q_tinyint_null)' ,'IRATE (t1.q_float_null)' ,'IRATE(t1.q_double_null)' , + 'twa(t2.q_int_null)' ,'twa(t2.q_bigint_null)' , 'twa(t2.q_smallint_null)' ,'twa(t2.q_tinyint_null)' ,'twa (t2.q_float_null)' ,'twa(t2.q_double_null)' , + 'IRATE(t2.q_int_null)' ,'IRATE(t2.q_bigint_null)' , 'IRATE(t2.q_smallint_null)' ,'IRATE(t2.q_tinyint_null)' ,'IRATE (t2.q_float_null)' ,'IRATE(t2.q_double_null)' ] + + # calc_calculate_all calc_calculate_regular calc_calculate_groupbytbname + # calculation function include [all:spread\+-*/ ||regualr:diff\derivative ||group by tbname:diff\derivative\] + self.calc_calculate_all = ['SPREAD(ts)' , 'SPREAD(q_ts)' , 'SPREAD(q_int)' ,'SPREAD(q_bigint)' , 'SPREAD(q_smallint)' ,'SPREAD(q_tinyint)' ,'SPREAD(q_float)' ,'SPREAD(q_double)' , + '(SPREAD(q_int) + SPREAD(q_bigint))' , '(SPREAD(q_smallint) - SPREAD(q_float))', '(SPREAD(q_double) * SPREAD(q_tinyint))' , '(SPREAD(q_double) / SPREAD(q_float))', + 'SPREAD(q_ts_null)' , 'SPREAD(q_int_null)' ,'SPREAD(q_bigint_null)' , 'SPREAD(q_smallint_null)' ,'SPREAD(q_tinyint_null)' ,'SPREAD(q_float_null)' ,'SPREAD(q_double_null)' , + '(SPREAD(q_int_null) + SPREAD(q_bigint_null))' , '(SPREAD(q_smallint_null) - SPREAD(q_float_null))', '(SPREAD(q_double_null) * SPREAD(q_tinyint_null))' , '(SPREAD(q_double_null) / SPREAD(q_float_null))'] + self.calc_calculate_regular = ['DIFF(q_int)' ,'DIFF(q_bigint)' , 'DIFF(q_smallint)' ,'DIFF(q_tinyint)' ,'DIFF(q_float)' ,'DIFF(q_double)' , + 'DIFF(q_int,0)' ,'DIFF(q_bigint,0)' , 'DIFF(q_smallint,0)' ,'DIFF(q_tinyint,0)' ,'DIFF(q_float,0)' ,'DIFF(q_double,0)' , + 'DIFF(q_int,1)' ,'DIFF(q_bigint,1)' , 'DIFF(q_smallint,1)' ,'DIFF(q_tinyint,1)' ,'DIFF(q_float,1)' ,'DIFF(q_double,1)' , + 'DERIVATIVE(q_int,15s,0)' , 'DERIVATIVE(q_bigint,10s,1)' , 'DERIVATIVE(q_smallint,20s,0)' ,'DERIVATIVE(q_tinyint,10s,1)' ,'DERIVATIVE(q_float,6s,0)' ,'DERIVATIVE(q_double,3s,1)', + 'DIFF(q_int_null)' ,'DIFF(q_bigint_null)' , 'DIFF(q_smallint_null)' ,'DIFF(q_tinyint_null)' ,'DIFF(q_float_null)' ,'DIFF(q_double_null)' , + 'DIFF(q_int_null,0)' ,'DIFF(q_bigint_null,0)' , 'DIFF(q_smallint_null,0)' ,'DIFF(q_tinyint_null,0)' ,'DIFF(q_float_null,0)' ,'DIFF(q_double_null,0)' , + 'DIFF(q_int_null,1)' ,'DIFF(q_bigint_null,1)' , 'DIFF(q_smallint_null,1)' ,'DIFF(q_tinyint_null,1)' ,'DIFF(q_float_null,1)' ,'DIFF(q_double_null,1)' , + 'DERIVATIVE(q_int_null,15s,0)' , 'DERIVATIVE(q_bigint_null,10s,1)' , 'DERIVATIVE(q_smallint_null,20s,0)' ,'DERIVATIVE(q_tinyint_null,10s,1)' ,'DERIVATIVE(q_float_null,6s,0)' ,'DERIVATIVE(q_double_null,3s,1)'] + self.calc_calculate_groupbytbname = self.calc_calculate_regular + + #two table join + self.calc_calculate_all_j = ['SPREAD(t1.ts)' , 'SPREAD(t1.q_ts)' , 'SPREAD(t1.q_int)' ,'SPREAD(t1.q_bigint)' , 'SPREAD(t1.q_smallint)' ,'SPREAD(t1.q_tinyint)' ,'SPREAD(t1.q_float)' ,'SPREAD(t1.q_double)' , + 'SPREAD(t2.ts)' , 'SPREAD(t2.q_ts)' , 'SPREAD(t2.q_int)' ,'SPREAD(t2.q_bigint)' , 'SPREAD(t2.q_smallint)' ,'SPREAD(t2.q_tinyint)' ,'SPREAD(t2.q_float)' ,'SPREAD(t2.q_double)' , + '(SPREAD(t1.q_int) + SPREAD(t1.q_bigint))' , '(SPREAD(t1.q_tinyint) - SPREAD(t1.q_float))', '(SPREAD(t1.q_double) * SPREAD(t1.q_tinyint))' , '(SPREAD(t1.q_double) / SPREAD(t1.q_tinyint))', + '(SPREAD(t2.q_int) + SPREAD(t2.q_bigint))' , '(SPREAD(t2.q_smallint) - SPREAD(t2.q_float))', '(SPREAD(t2.q_double) * SPREAD(t2.q_tinyint))' , '(SPREAD(t2.q_double) / SPREAD(t2.q_tinyint))', + '(SPREAD(t1.q_int) + SPREAD(t1.q_smallint))' , '(SPREAD(t2.q_smallint) - SPREAD(t2.q_float))', '(SPREAD(t1.q_double) * SPREAD(t1.q_tinyint))' , '(SPREAD(t1.q_double) / SPREAD(t1.q_float))', + 'SPREAD(t1.q_ts_null)' , 'SPREAD(t1.q_int_null)' ,'SPREAD(t1.q_bigint_null)' , 'SPREAD(t1.q_smallint_null)' ,'SPREAD(t1.q_tinyint_null)' ,'SPREAD(t1.q_float_null)' ,'SPREAD(t1.q_double_null)' , + 'SPREAD(t2.q_ts_null)' , 'SPREAD(t2.q_int_null)' ,'SPREAD(t2.q_bigint_null)' , 'SPREAD(t2.q_smallint_null)' ,'SPREAD(t2.q_tinyint_null)' ,'SPREAD(t2.q_float_null)' ,'SPREAD(t2.q_double_null)' , + '(SPREAD(t1.q_int_null) + SPREAD(t1.q_bigint_null))' , '(SPREAD(t1.q_tinyint_null) - SPREAD(t1.q_float_null))', '(SPREAD(t1.q_double_null) * SPREAD(t1.q_tinyint_null))' , '(SPREAD(t1.q_double_null) / SPREAD(t1.q_tinyint_null))', + '(SPREAD(t2.q_int_null) + SPREAD(t2.q_bigint_null))' , '(SPREAD(t2.q_smallint_null) - SPREAD(t2.q_float_null))', '(SPREAD(t2.q_double_null) * SPREAD(t2.q_tinyint_null))' , '(SPREAD(t2.q_double_null) / SPREAD(t2.q_tinyint_null))', + '(SPREAD(t1.q_int_null) + SPREAD(t1.q_smallint_null))' , '(SPREAD(t2.q_smallint_null) - SPREAD(t2.q_float_null))', '(SPREAD(t1.q_double_null) * SPREAD(t1.q_tinyint_null))' , '(SPREAD(t1.q_double_null) / SPREAD(t1.q_float_null))'] + self.calc_calculate_regular_j = ['DIFF(t1.q_int)' ,'DIFF(t1.q_bigint)' , 'DIFF(t1.q_smallint)' ,'DIFF(t1.q_tinyint)' ,'DIFF(t1.q_float)' ,'DIFF(t1.q_double)' , + 'DIFF(t1.q_int,0)' ,'DIFF(t1.q_bigint,0)' , 'DIFF(t1.q_smallint,0)' ,'DIFF(t1.q_tinyint,0)' ,'DIFF(t1.q_float,0)' ,'DIFF(t1.q_double,0)' , + 'DIFF(t1.q_int,1)' ,'DIFF(t1.q_bigint,1)' , 'DIFF(t1.q_smallint,1)' ,'DIFF(t1.q_tinyint,1)' ,'DIFF(t1.q_float,1)' ,'DIFF(t1.q_double,1)' , + 'DERIVATIVE(t1.q_int,15s,0)' , 'DERIVATIVE(t1.q_bigint,10s,1)' , 'DERIVATIVE(t1.q_smallint,20s,0)' ,'DERIVATIVE(t1.q_tinyint,10s,1)' ,'DERIVATIVE(t1.q_float,6s,0)' ,'DERIVATIVE(t1.q_double,3s,1)' , + 'DIFF(t2.q_int)' ,'DIFF(t2.q_bigint)' , 'DIFF(t2.q_smallint)' ,'DIFF(t2.q_tinyint)' ,'DIFF(t2.q_float)' ,'DIFF(t2.q_double)' , + 'DIFF(t2.q_int,0)' ,'DIFF(t2.q_bigint,0)' , 'DIFF(t2.q_smallint,0)' ,'DIFF(t2.q_tinyint,0)' ,'DIFF(t2.q_float,0)' ,'DIFF(t2.q_double,0)' , + 'DIFF(t2.q_int,1)' ,'DIFF(t2.q_bigint,1)' , 'DIFF(t2.q_smallint,1)' ,'DIFF(t2.q_tinyint,1)' ,'DIFF(t2.q_float,1)' ,'DIFF(t2.q_double,1)' , + 'DERIVATIVE(t2.q_int,15s,0)' , 'DERIVATIVE(t2.q_bigint,10s,1)' , 'DERIVATIVE(t2.q_smallint,20s,0)' ,'DERIVATIVE(t2.q_tinyint,10s,1)' ,'DERIVATIVE(t2.q_float,6s,0)' ,'DERIVATIVE(t2.q_double,3s,1)' , + 'DIFF(t1.q_int_null)' ,'DIFF(t1.q_bigint_null)' , 'DIFF(t1.q_smallint_null)' ,'DIFF(t1.q_tinyint_null)' ,'DIFF(t1.q_float_null)' ,'DIFF(t1.q_double_null)' , + 'DIFF(t1.q_int_null,0)' ,'DIFF(t1.q_bigint_null,0)' , 'DIFF(t1.q_smallint_null,0)' ,'DIFF(t1.q_tinyint_null,0)' ,'DIFF(t1.q_float_null,0)' ,'DIFF(t1.q_double_null,0)' , + 'DIFF(t1.q_int_null,1)' ,'DIFF(t1.q_bigint_null,1)' , 'DIFF(t1.q_smallint_null,1)' ,'DIFF(t1.q_tinyint_null,1)' ,'DIFF(t1.q_float_null,1)' ,'DIFF(t1.q_double_null,1)' , + 'DERIVATIVE(t1.q_int_null,15s,0)' , 'DERIVATIVE(t1.q_bigint_null,10s,1)' , 'DERIVATIVE(t1.q_smallint_null,20s,0)' ,'DERIVATIVE(t1.q_tinyint_null,10s,1)' ,'DERIVATIVE(t1.q_float_null,6s,0)' ,'DERIVATIVE(t1.q_double_null,3s,1)' , + 'DIFF(t2.q_int_null)' ,'DIFF(t2.q_bigint_null)' , 'DIFF(t2.q_smallint_null)' ,'DIFF(t2.q_tinyint_null)' ,'DIFF(t2.q_float_null)' ,'DIFF(t2.q_double_null)' , + 'DIFF(t2.q_int_null,0)' ,'DIFF(t2.q_bigint_null,0)' , 'DIFF(t2.q_smallint_null,0)' ,'DIFF(t2.q_tinyint_null,0)' ,'DIFF(t2.q_float_null,0)' ,'DIFF(t2.q_double_null,0)' , + 'DIFF(t2.q_int_null,1)' ,'DIFF(t2.q_bigint_null,1)' , 'DIFF(t2.q_smallint_null,1)' ,'DIFF(t2.q_tinyint_null,1)' ,'DIFF(t2.q_float_null,1)' ,'DIFF(t2.q_double_null,1)' , + 'DERIVATIVE(t2.q_int_null,15s,0)' , 'DERIVATIVE(t2.q_bigint_null,10s,1)' , 'DERIVATIVE(t2.q_smallint_null,20s,0)' ,'DERIVATIVE(t2.q_tinyint_null,10s,1)' ,'DERIVATIVE(t2.q_float_null,6s,0)' ,'DERIVATIVE(t2.q_double_null,3s,1)'] + self.calc_calculate_groupbytbname_j = self.calc_calculate_regular_j + + #inter && calc_aggregate_all\calc_aggregate_regular\calc_select_all + self.interval_sliding = ['interval(4w) sliding(1w) ','interval(1w) sliding(1d) ','interval(1d) sliding(1h) ' , + 'interval(1h) sliding(1m) ','interval(1m) sliding(1s) ','interval(1s) sliding(10a) ', + 'interval(1y) ','interval(1n) ','interval(1w) ','interval(1d) ','interval(1h) ','interval(1m) ','interval(1s) ' ,'interval(10a)', + 'interval(1y,1n) ','interval(1n,1w) ','interval(1w,1d) ','interval(1d,1h) ','interval(1h,1m) ','interval(1m,1s) ','interval(1s,10a) ' ,'interval(100a,30a)'] + + self.conn1 = taos.connect(host="127.0.0.1", user="root", password="taosdata", config="/etc/taos/") + self.cur1 = self.conn1.cursor() + print(self.cur1) + self.cur1.execute("use %s ;" %self.db_nest) + sql = 'select /*+ para_tables_sort() */* from stable_1 limit 5;' + self.cur1.execute(sql) + + + def data_matrix_equal(self, sql1,row1_s,row1_e,col1_s,col1_e, sql2,row2_s,row2_e,col2_s,col2_e): + # ----row1_start----col1_start---- + # - - - - 是一个矩阵内的数据相等- - - + # - - - - - - - - - - - - - - - - + # ----row1_end------col1_end------ + self.sql1 = sql1 + list1 =[] + tdSql.query(sql1) + for i1 in range(row1_s-1,row1_e): + #print("iiii=%d"%i1) + for j1 in range(col1_s-1,col1_e): + #print("jjjj=%d"%j1) + #print("data=%s" %(tdSql.getData(i1,j1))) + list1.append(tdSql.getData(i1,j1)) + print("=====list1-------list1---=%s" %set(list1)) + + tdSql.execute("reset query cache;") + self.sql2 = sql2 + list2 =[] + tdSql.query(sql2) + for i2 in range(row2_s-1,row2_e): + #print("iiii222=%d"%i2) + for j2 in range(col2_s-1,col2_e): + #print("jjjj222=%d"%j2) + #print("data=%s" %(tdSql.getData(i2,j2))) + list2.append(tdSql.getData(i2,j2)) + print("=====list2-------list2---=%s" %set(list2)) + + if (list1 == list2) and len(list2)>0: + # print(("=====matrix===sql1.list1:'%s',sql2.list2:'%s'") %(list1,list2)) + tdLog.info(("===matrix===sql1:'%s' matrix_result = sql2:'%s' matrix_result") %(sql1,sql2)) + elif (set(list2)).issubset(set(list1)): + # 解决不同子表排列结果乱序 + # print(("=====list_issubset==matrix2in1-true===sql1.list1:'%s',sql2.list2:'%s'") %(list1,list2)) + tdLog.info(("===matrix_issubset===sql1:'%s' matrix_set_result = sql2:'%s' matrix_set_result") %(sql1,sql2)) + #elif abs(float(str(list1).replace("]","").replace("[","").replace("e+","")) - float(str(list2).replace("]","").replace("[","").replace("e+",""))) <= 0.0001: + elif abs(float(str(list1).replace("datetime.datetime","").replace("]","").replace("[","").replace("e+","").replace(", ","").replace("(","").replace(")","").replace("-","")) - float(str(list2).replace("datetime.datetime","").replace("]","").replace("[","").replace("e+","").replace(", ","").replace("(","").replace(")","").replace("-",""))) <= 0.0001: + print(("=====matrix_abs+e+===sql1.list1:'%s',sql2.list2:'%s'") %(list1,list2)) + print(("=====matrix_abs+e+replace_after===sql1.list1:'%s',sql2.list2:'%s'") %(float(str(list1).replace("datetime.datetime","").replace("]","").replace("[","").replace("e+","").replace(", ","").replace("(","").replace(")","").replace("-","")),float(str(list2).replace("datetime.datetime","").replace("]","").replace("[","").replace("e+","").replace(", ","").replace("(","").replace(")","").replace("-","")))) + tdLog.info(("===matrix_abs+e+===sql1:'%s' matrix_result = sql2:'%s' matrix_result") %(sql1,sql2)) + elif abs(float(str(list1).replace("datetime.datetime","").replace("]","").replace("[","").replace(", ","").replace("(","").replace(")","").replace("-","")) - float(str(list2).replace("datetime.datetime","").replace("]","").replace("[","").replace(", ","").replace("(","").replace(")","").replace("-",""))) <= 0.1: + #{datetime.datetime(2021, 8, 27, 1, 46, 40), -441.46841430664057}replace + print(("=====matrix_abs+replace===sql1.list1:'%s',sql2.list2:'%s'") %(list1,list2)) + print(("=====matrix_abs+replace_after===sql1.list1:'%s',sql2.list2:'%s'") %(float(str(list1).replace("datetime.datetime","").replace("]","").replace("[","").replace(", ","").replace("(","").replace(")","").replace("-","")),float(str(list1).replace("datetime.datetime","").replace("]","").replace("[","").replace(", ","").replace("(","").replace(")","").replace("-","")))) + tdLog.info(("===matrix_abs+replace===sql1:'%s' matrix_result = sql2:'%s' matrix_result") %(sql1,sql2)) + elif abs(float(str(list1).replace("datetime.datetime","").replace("]","").replace("[","").replace(", ","").replace("(","").replace(")","").replace("-","")) - float(str(list2).replace("datetime.datetime","").replace("]","").replace("[","").replace(", ","").replace("(","").replace(")","").replace("-",""))) <= 0.5: + print(("=====matrix_abs===sql1.list1:'%s',sql2.list2:'%s'") %(list1,list2)) + print(("=====matrix_abs===sql1.list1:'%s',sql2.list2:'%s'") %(float(str(list1).replace("datetime.datetime","").replace("]","").replace("[","").replace(", ","").replace("(","").replace(")","").replace("-","")),float(str(list2).replace("datetime.datetime","").replace("]","").replace("[","").replace(", ","").replace("(","").replace(")","").replace("-","")))) + tdLog.info(("===matrix_abs======sql1:'%s' matrix_result = sql2:'%s' matrix_result") %(sql1,sql2)) + else: + print(("=====matrix_error===sql1.list1:'%s',sql2.list2:'%s'") %(list1,list2)) + tdLog.info(("sql1:'%s' matrix_result != sql2:'%s' matrix_result") %(sql1,sql2)) + return tdSql.checkEqual(list1,list2) + + def restartDnodes(self): + pass + # tdDnodes.stop(1) + # tdDnodes.start(1) + + def dropandcreateDB_random(self,database,n): + ts = 1630000000000 + num_random = 100 + fake = Faker('zh_CN') + tdSql.execute('''drop database if exists %s ;''' %database) + tdSql.execute('''create database %s keep 36500;'''%database) + tdSql.execute('''use %s;'''%database) + + tdSql.execute('''create stable stable_1 (ts timestamp , q_int int , q_bigint bigint , q_smallint smallint , q_tinyint tinyint , q_float float , q_double double , q_bool bool , q_binary binary(100) , q_nchar nchar(100) , q_ts timestamp , \ + q_binary1 binary(100) , q_nchar1 nchar(100) ,q_binary2 binary(100) , q_nchar2 nchar(100) ,q_binary3 binary(100) , q_nchar3 nchar(100) ,q_binary4 binary(100) , q_nchar4 nchar(100) ,\ + q_binary5 binary(100) , q_nchar5 nchar(100) ,q_binary6 binary(100) , q_nchar6 nchar(100) ,q_binary7 binary(100) , q_nchar7 nchar(100) ,q_binary8 binary(100) , q_nchar8 nchar(100) ,\ + q_int_null int , q_bigint_null bigint , q_smallint_null smallint , q_tinyint_null tinyint, q_float_null float , q_double_null double , q_bool_null bool , q_binary_null binary(20) , q_nchar_null nchar(20) , q_ts_null timestamp) \ + tags(loc nchar(100) , t_int int , t_bigint bigint , t_smallint smallint , t_tinyint tinyint, t_bool bool , t_binary binary(100) , t_nchar nchar(100) ,t_float float , t_double double , t_ts timestamp);''') + tdSql.execute('''create stable stable_2 (ts timestamp , q_int int , q_bigint bigint , q_smallint smallint , q_tinyint tinyint , q_float float , q_double double , q_bool bool , q_binary binary(100) , q_nchar nchar(100) , q_ts timestamp , \ + q_binary1 binary(100) , q_nchar1 nchar(100) ,q_binary2 binary(100) , q_nchar2 nchar(100) ,q_binary3 binary(100) , q_nchar3 nchar(100) ,q_binary4 binary(100) , q_nchar4 nchar(100) ,\ + q_binary5 binary(100) , q_nchar5 nchar(100) ,q_binary6 binary(100) , q_nchar6 nchar(100) ,q_binary7 binary(100) , q_nchar7 nchar(100) ,q_binary8 binary(100) , q_nchar8 nchar(100) ,\ + q_int_null int , q_bigint_null bigint , q_smallint_null smallint , q_tinyint_null tinyint, q_float_null float , q_double_null double , q_bool_null bool , q_binary_null binary(20) , q_nchar_null nchar(20) , q_ts_null timestamp) \ + tags(loc nchar(100) , t_int int , t_bigint bigint , t_smallint smallint , t_tinyint tinyint, t_bool bool , t_binary binary(100) , t_nchar nchar(100) ,t_float float , t_double double , t_ts timestamp);''') + + tdSql.execute('''create stable stable_null_data (ts timestamp , q_int int , q_bigint bigint , q_smallint smallint , q_tinyint tinyint , q_float float , q_double double , q_bool bool , q_binary binary(100) , q_nchar nchar(100) , q_ts timestamp , \ + q_binary1 binary(100) , q_nchar1 nchar(100) ,q_binary2 binary(100) , q_nchar2 nchar(100) ,q_binary3 binary(100) , q_nchar3 nchar(100) ,q_binary4 binary(100) , q_nchar4 nchar(100) ,\ + q_binary5 binary(100) , q_nchar5 nchar(100) ,q_binary6 binary(100) , q_nchar6 nchar(100) ,q_binary7 binary(100) , q_nchar7 nchar(100) ,q_binary8 binary(100) , q_nchar8 nchar(100) ,\ + q_int_null int , q_bigint_null bigint , q_smallint_null smallint , q_tinyint_null tinyint, q_float_null float , q_double_null double , q_bool_null bool , q_binary_null binary(20) , q_nchar_null nchar(20) , q_ts_null timestamp) \ + tags(loc nchar(100) , t_int int , t_bigint bigint , t_smallint smallint , t_tinyint tinyint, t_bool bool , t_binary binary(100) , t_nchar nchar(100) ,t_float float , t_double double , t_ts timestamp);''') + + tdSql.execute('''create stable stable_null_childtable (ts timestamp , q_int int , q_bigint bigint , q_smallint smallint , q_tinyint tinyint , q_float float , q_double double , q_bool bool , q_binary binary(100) , q_nchar nchar(100) , q_ts timestamp , \ + q_binary1 binary(100) , q_nchar1 nchar(100) ,q_binary2 binary(100) , q_nchar2 nchar(100) ,q_binary3 binary(100) , q_nchar3 nchar(100) ,q_binary4 binary(100) , q_nchar4 nchar(100) ,\ + q_binary5 binary(100) , q_nchar5 nchar(100) ,q_binary6 binary(100) , q_nchar6 nchar(100) ,q_binary7 binary(100) , q_nchar7 nchar(100) ,q_binary8 binary(100) , q_nchar8 nchar(100) ,\ + q_int_null int , q_bigint_null bigint , q_smallint_null smallint , q_tinyint_null tinyint, q_float_null float , q_double_null double , q_bool_null bool , q_binary_null binary(20) , q_nchar_null nchar(20) , q_ts_null timestamp) \ + tags(loc nchar(100) , t_int int , t_bigint bigint , t_smallint smallint , t_tinyint tinyint, t_bool bool , t_binary binary(100) , t_nchar nchar(100) ,t_float float , t_double double , t_ts timestamp);''') + + #tdSql.execute('''create table stable_1_1 using stable_1 tags('stable_1_1', '0' , '0' , '0' , '0' , 0 , 'binary1' , 'nchar1' , '0' , '0' ,'0') ;''') + tdSql.execute('''create table stable_1_1 using stable_1 tags('stable_1_1', '%d' , '%d', '%d' , '%d' , 0 , 'binary1.%s' , 'nchar1.%s' , '%f', '%f' ,'%d') ;''' + %(fake.random_int(min=-2147483647, max=2147483647, step=1), fake.random_int(min=-9223372036854775807, max=9223372036854775807, step=1), + fake.random_int(min=-32767, max=32767, step=1) , fake.random_int(min=-127, max=127, step=1) , + fake.pystr() ,fake.pystr() ,fake.pyfloat(),fake.pyfloat(),fake.random_int(min=-2147483647, max=2147483647, step=1))) + tdSql.execute('''create table stable_1_2 using stable_1 tags('stable_1_2', '2147483647' , '9223372036854775807' , '32767' , '127' , 1 , 'binary2' , 'nchar2' , '2' , '22' , \'1999-09-09 09:09:09.090\') ;''') + tdSql.execute('''create table stable_1_3 using stable_1 tags('stable_1_3', '-2147483647' , '-9223372036854775807' , '-32767' , '-127' , false , 'binary3' , 'nchar3nchar3' , '-3.3' , '-33.33' , \'2099-09-09 09:09:09.090\') ;''') + #tdSql.execute('''create table stable_1_4 using stable_1 tags('stable_1_4', '0' , '0' , '0' , '0' , 0 , '0' , '0' , '0' , '0' ,'0') ;''') + tdSql.execute('''create table stable_1_4 using stable_1 tags('stable_1_4', '%d' , '%d', '%d' , '%d' , 1 , 'binary1.%s' , 'nchar1.%s' , '%f', '%f' ,'%d') ;''' + %(fake.random_int(min=-2147483647, max=2147483647, step=1), fake.random_int(min=-9223372036854775807, max=9223372036854775807, step=1), + fake.random_int(min=-32767, max=32767, step=1) , fake.random_int(min=-127, max=127, step=1) , + fake.pystr() ,fake.pystr() ,fake.pyfloat(),fake.pyfloat(),fake.random_int(min=-2147483647, max=2147483647, step=1))) + + # tdSql.execute('''create table stable_2_1 using stable_2 tags('stable_2_1' , '0' , '0' , '0' , '0' , 0 , 'binary21' , 'nchar21' , '0' , '0' ,'0') ;''') + # tdSql.execute('''create table stable_2_2 using stable_2 tags('stable_2_2' , '0' , '0' , '0' , '0' , 0 , '0' , '0' , '0' , '0' ,'0') ;''') + + # tdSql.execute('''create table stable_null_data_1 using stable_null_data tags('stable_null_data_1', '0' , '0' , '0' , '0' , 0 , '0' , '0' , '0' , '0' ,'0') ;''') + + tdSql.execute('''create table stable_2_1 using stable_2 tags('stable_2_1' , '0' , '0' , '0' , '0' , 0 , 'binary21' , 'nchar21' , '0' , '0' ,\'2099-09-09 09:09:09.090\') ;''') + tdSql.execute('''create table stable_2_2 using stable_2 tags('stable_2_2' , '%d' , '%d', '%d' , '%d' , 0 , 'binary2.%s' , 'nchar2.%s' , '%f', '%f' ,'%d') ;''' + %(fake.random_int(min=-2147483647, max=2147483647, step=1), fake.random_int(min=-9223372036854775807, max=9223372036854775807, step=1), + fake.random_int(min=-32767, max=32767, step=1) , fake.random_int(min=-127, max=127, step=1) , + fake.pystr() ,fake.pystr() ,fake.pyfloat(),fake.pyfloat(),fake.random_int(min=-2147483647, max=2147483647, step=1))) + + tdSql.execute('''create table stable_null_data_1 using stable_null_data tags('stable_null_data_1', '%d' , '%d', '%d' , '%d' , 1 , 'binary1.%s' , 'nchar1.%s' , '%f', '%f' ,'%d') ;''' + %(fake.random_int(min=-2147483647, max=2147483647, step=1), fake.random_int(min=-9223372036854775807, max=9223372036854775807, step=1), + fake.random_int(min=-32767, max=32767, step=1) , fake.random_int(min=-127, max=127, step=1) , + fake.pystr() ,fake.pystr() ,fake.pyfloat(),fake.pyfloat(),fake.random_int(min=-2147483647, max=2147483647, step=1))) + + #regular table + tdSql.execute('''create table regular_table_1 \ + (ts timestamp , q_int int , q_bigint bigint , q_smallint smallint , q_tinyint tinyint , q_float float , q_double double , q_bool bool , q_binary binary(100) , q_nchar nchar(100) , q_ts timestamp , \ + q_binary1 binary(100) , q_nchar1 nchar(100) ,q_binary2 binary(100) , q_nchar2 nchar(100) ,q_binary3 binary(100) , q_nchar3 nchar(100) ,q_binary4 binary(100) , q_nchar4 nchar(100) ,\ + q_binary5 binary(100) , q_nchar5 nchar(100) ,q_binary6 binary(100) , q_nchar6 nchar(100) ,q_binary7 binary(100) , q_nchar7 nchar(100) ,q_binary8 binary(100) , q_nchar8 nchar(100) ,\ + q_int_null int , q_bigint_null bigint , q_smallint_null smallint , q_tinyint_null tinyint, q_float_null float , q_double_null double , q_bool_null bool , q_binary_null binary(20) , q_nchar_null nchar(20) , q_ts_null timestamp) ;''') + tdSql.execute('''create table regular_table_2 \ + (ts timestamp , q_int int , q_bigint bigint , q_smallint smallint , q_tinyint tinyint , q_float float , q_double double , q_bool bool , q_binary binary(100) , q_nchar nchar(100) , q_ts timestamp , \ + q_binary1 binary(100) , q_nchar1 nchar(100) ,q_binary2 binary(100) , q_nchar2 nchar(100) ,q_binary3 binary(100) , q_nchar3 nchar(100) ,q_binary4 binary(100) , q_nchar4 nchar(100) ,\ + q_binary5 binary(100) , q_nchar5 nchar(100) ,q_binary6 binary(100) , q_nchar6 nchar(100) ,q_binary7 binary(100) , q_nchar7 nchar(100) ,q_binary8 binary(100) , q_nchar8 nchar(100) ,\ + q_int_null int , q_bigint_null bigint , q_smallint_null smallint , q_tinyint_null tinyint, q_float_null float , q_double_null double , q_bool_null bool , q_binary_null binary(20) , q_nchar_null nchar(20) , q_ts_null timestamp) ;''') + tdSql.execute('''create table regular_table_3 \ + (ts timestamp , q_int int , q_bigint bigint , q_smallint smallint , q_tinyint tinyint , q_float float , q_double double , q_bool bool , q_binary binary(100) , q_nchar nchar(100) , q_ts timestamp , \ + q_binary1 binary(100) , q_nchar1 nchar(100) ,q_binary2 binary(100) , q_nchar2 nchar(100) ,q_binary3 binary(100) , q_nchar3 nchar(100) ,q_binary4 binary(100) , q_nchar4 nchar(100) ,\ + q_binary5 binary(100) , q_nchar5 nchar(100) ,q_binary6 binary(100) , q_nchar6 nchar(100) ,q_binary7 binary(100) , q_nchar7 nchar(100) ,q_binary8 binary(100) , q_nchar8 nchar(100) ,\ + q_int_null int , q_bigint_null bigint , q_smallint_null smallint , q_tinyint_null tinyint, q_float_null float , q_double_null double , q_bool_null bool , q_binary_null binary(20) , q_nchar_null nchar(20) , q_ts_null timestamp) ;''') + + tdSql.execute('''create table regular_table_null \ + (ts timestamp , q_int int , q_bigint bigint , q_smallint smallint , q_tinyint tinyint , q_float float , q_double double , q_bool bool , q_binary binary(100) , q_nchar nchar(100) , q_ts timestamp , \ + q_binary1 binary(100) , q_nchar1 nchar(100) ,q_binary2 binary(100) , q_nchar2 nchar(100) ,q_binary3 binary(100) , q_nchar3 nchar(100) ,q_binary4 binary(100) , q_nchar4 nchar(100) ,\ + q_binary5 binary(100) , q_nchar5 nchar(100) ,q_binary6 binary(100) , q_nchar6 nchar(100) ,q_binary7 binary(100) , q_nchar7 nchar(100) ,q_binary8 binary(100) , q_nchar8 nchar(100) ,\ + q_int_null int , q_bigint_null bigint , q_smallint_null smallint , q_tinyint_null tinyint, q_float_null float , q_double_null double , q_bool_null bool , q_binary_null binary(20) , q_nchar_null nchar(20) , q_ts_null timestamp) ;''') + + + for i in range(1, num_random*n + 1): + tdSql.execute('''insert into stable_1_1 (ts , q_int , q_bigint , q_smallint , q_tinyint , q_float , q_double , q_bool , q_binary , q_nchar, q_ts,\ + q_binary1 , q_nchar1 , q_binary2 , q_nchar2 , q_binary3 , q_nchar3 , q_binary4 , q_nchar4 , q_binary5 , q_nchar5 , q_binary6 , q_nchar6 , q_binary7 , q_nchar7, q_binary8 , q_nchar8) \ + values(%d, %d, %d, %d, %d, %f, %f, 0, 'binary.%s', 'nchar.%s', %d, 'binary1.%s', 'nchar1.%s', 'binary2.%s', 'nchar2.%s', 'binary3.%s', 'nchar3.%s', \ + 'binary4.%s', 'nchar4.%s', 'binary5.%s', 'nchar5.%s', 'binary6.%s', 'nchar6.%s', 'binary7.%s', 'nchar7.%s', 'binary8.%s', 'nchar8.%s') ;''' + % (ts + i*1000, fake.random_int(min=-2147483647, max=2147483647, step=1), + fake.random_int(min=-9223372036854775807, max=9223372036854775807, step=1), + fake.random_int(min=-32767, max=32767, step=1) , fake.random_int(min=-127, max=127, step=1) , + fake.pyfloat() , fake.pyfloat() , fake.pystr() , fake.pystr() , ts + i , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , + fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr())) + tdSql.execute('''insert into regular_table_1 (ts , q_int , q_bigint , q_smallint , q_tinyint , q_float , q_double, q_bool , q_binary , q_nchar, q_ts,\ + q_binary1 , q_nchar1 , q_binary2 , q_nchar2 , q_binary3 , q_nchar3 , q_binary4 , q_nchar4 , q_binary5 , q_nchar5 , q_binary6 , q_nchar6 , q_binary7 , q_nchar7, q_binary8 , q_nchar8) \ + values(%d, %d, %d, %d, %d, %f, %f, 0, 'binary.%s', 'nchar.%s', %d, 'binary1.%s', 'nchar1.%s', 'binary2.%s', 'nchar2.%s', 'binary3.%s', 'nchar3.%s', \ + 'binary4.%s', 'nchar4.%s', 'binary5.%s', 'nchar5.%s', 'binary6.%s', 'nchar6.%s', 'binary7.%s', 'nchar7.%s', 'binary8.%s', 'nchar8.%s') ;''' + % (ts + i*1000, fake.random_int(min=-2147483647, max=2147483647, step=1) , + fake.random_int(min=-9223372036854775807, max=9223372036854775807, step=1) , + fake.random_int(min=-32767, max=32767, step=1) , fake.random_int(min=-127, max=127, step=1) , + fake.pyfloat() , fake.pyfloat() , fake.pystr() , fake.pystr() , ts + i, fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , + fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr())) + + tdSql.execute('''insert into stable_1_2 (ts , q_int , q_bigint , q_smallint , q_tinyint , q_float , q_double, q_bool , q_binary , q_nchar, q_ts,\ + q_binary1 , q_nchar1 , q_binary2 , q_nchar2 , q_binary3 , q_nchar3 , q_binary4 , q_nchar4 , q_binary5 , q_nchar5 , q_binary6 , q_nchar6 , q_binary7 , q_nchar7, q_binary8 , q_nchar8)\ + values(%d, %d, %d, %d, %d, %f, %f, 1, 'binary.%s', 'nchar.%s', %d, 'binary1.%s', 'nchar1.%s', 'binary2.%s', 'nchar2.%s', 'binary3.%s', 'nchar3.%s', \ + 'binary4.%s', 'nchar4.%s', 'binary5.%s', 'nchar5.%s', 'binary6.%s', 'nchar6.%s', 'binary7.%s', 'nchar7.%s', 'binary8.%s', 'nchar8.%s') ;''' + % (ts + i*1000*60*60*2, fake.random_int(min=0, max=2147483647, step=1), + fake.random_int(min=0, max=9223372036854775807, step=1), + fake.random_int(min=0, max=32767, step=1) , fake.random_int(min=0, max=127, step=1) , + fake.pyfloat() , fake.pyfloat() , fake.pystr() , fake.pystr() , ts + i, fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , + fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr())) + tdSql.execute('''insert into regular_table_2 (ts , q_int , q_bigint , q_smallint , q_tinyint , q_float , q_double, q_bool , q_binary , q_nchar, q_ts,\ + q_binary1 , q_nchar1 , q_binary2 , q_nchar2 , q_binary3 , q_nchar3 , q_binary4 , q_nchar4 , q_binary5 , q_nchar5 , q_binary6 , q_nchar6 , q_binary7 , q_nchar7, q_binary8 , q_nchar8) \ + values(%d, %d, %d, %d, %d, %f, %f, 1, 'binary.%s', 'nchar.%s', %d, 'binary1.%s', 'nchar1.%s', 'binary2.%s', 'nchar2.%s', 'binary3.%s', 'nchar3.%s', \ + 'binary4.%s', 'nchar4.%s', 'binary5.%s', 'nchar5.%s', 'binary6.%s', 'nchar6.%s', 'binary7.%s', 'nchar7.%s', 'binary8.%s', 'nchar8.%s') ;''' + % (ts + i*1000, fake.random_int(min=0, max=2147483647, step=1), + fake.random_int(min=0, max=9223372036854775807, step=1), + fake.random_int(min=0, max=32767, step=1) , fake.random_int(min=0, max=127, step=1) , + fake.pyfloat() , fake.pyfloat() , fake.pystr() , fake.pystr() , ts + i, fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , + fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr())) + + tdSql.execute('''insert into stable_1_2 (ts , q_int , q_bigint , q_smallint , q_tinyint , q_float , q_double, q_bool , q_binary , q_nchar, q_ts,\ + q_binary1 , q_nchar1 , q_binary2 , q_nchar2 , q_binary3 , q_nchar3 , q_binary4 , q_nchar4 , q_binary5 , q_nchar5 , q_binary6 , q_nchar6 , q_binary7 , q_nchar7, q_binary8 , q_nchar8) \ + values(%d, %d, %d, %d, %d, %f, %f, 1, 'binary.%s', 'nchar.%s', %d, 'binary1.%s', 'nchar1.%s', 'binary2.%s', 'nchar2.%s', 'binary3.%s', 'nchar3.%s', \ + 'binary4.%s', 'nchar4.%s', 'binary5.%s', 'nchar5.%s', 'binary6.%s', 'nchar6.%s', 'binary7.%s', 'nchar7.%s', 'binary8.%s', 'nchar8.%s') ;''' + % (ts + i*1000*60*60*2 +1, fake.random_int(min=-2147483647, max=0, step=1), + fake.random_int(min=-9223372036854775807, max=0, step=1), + fake.random_int(min=-32767, max=0, step=1) , fake.random_int(min=-127, max=0, step=1) , + fake.pyfloat() , fake.pyfloat() , fake.pystr() , fake.pystr() , ts + i +1, fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , + fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr())) + tdSql.execute('''insert into regular_table_2 (ts , q_int , q_bigint , q_smallint , q_tinyint , q_float , q_double, q_bool , q_binary , q_nchar, q_ts,\ + q_binary1 , q_nchar1 , q_binary2 , q_nchar2 , q_binary3 , q_nchar3 , q_binary4 , q_nchar4 , q_binary5 , q_nchar5 , q_binary6 , q_nchar6 , q_binary7 , q_nchar7, q_binary8 , q_nchar8) \ + values(%d, %d, %d, %d, %d, %f, %f, 1, 'binary.%s', 'nchar.%s', %d, 'binary1.%s', 'nchar1.%s', 'binary2.%s', 'nchar2.%s', 'binary3.%s', 'nchar3.%s', \ + 'binary4.%s', 'nchar4.%s', 'binary5.%s', 'nchar5.%s', 'binary6.%s', 'nchar6.%s', 'binary7.%s', 'nchar7.%s', 'binary8.%s', 'nchar8.%s') ;''' + % (ts + i*1000 +1, fake.random_int(min=-2147483647, max=0, step=1), + fake.random_int(min=-9223372036854775807, max=0, step=1), + fake.random_int(min=-32767, max=0, step=1) , fake.random_int(min=-127, max=0, step=1) , + fake.pyfloat() , fake.pyfloat() , fake.pystr() , fake.pystr() , ts + i +1, fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , + fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr())) + + tdSql.execute('''insert into stable_2_1 (ts , q_int , q_bigint , q_smallint , q_tinyint , q_float , q_double, q_bool , q_binary , q_nchar, q_ts,\ + q_binary1 , q_nchar1 , q_binary2 , q_nchar2 , q_binary3 , q_nchar3 , q_binary4 , q_nchar4 , q_binary5 , q_nchar5 , q_binary6 , q_nchar6 , q_binary7 , q_nchar7, q_binary8 , q_nchar8) \ + values(%d, %d, %d, %d, %d, %f, %f, 0, 'binary.%s', 'nchar.%s', %d, 'binary1.%s', 'nchar1.%s', 'binary2.%s', 'nchar2.%s', 'binary3.%s', 'nchar3.%s', \ + 'binary4.%s', 'nchar4.%s', 'binary5.%s', 'nchar5.%s', 'binary6.%s', 'nchar6.%s', 'binary7.%s', 'nchar7.%s', 'binary8.%s', 'nchar8.%s') ;''' + % (ts + i*1000*60*60*4, fake.random_int(min=-0, max=2147483647, step=1), + fake.random_int(min=-0, max=9223372036854775807, step=1), + fake.random_int(min=-0, max=32767, step=1) , fake.random_int(min=-0, max=127, step=1) , + fake.pyfloat() , fake.pyfloat() , fake.pystr() , fake.pystr() , ts + i, fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , + fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr())) + + tdSql.execute('''insert into stable_2_1 (ts , q_int , q_bigint , q_smallint , q_tinyint , q_float , q_double, q_bool , q_binary , q_nchar, q_ts,\ + q_binary1 , q_nchar1 , q_binary2 , q_nchar2 , q_binary3 , q_nchar3 , q_binary4 , q_nchar4 , q_binary5 , q_nchar5 , q_binary6 , q_nchar6 , q_binary7 , q_nchar7, q_binary8 , q_nchar8) \ + values(%d, %d, %d, %d, %d, %f, %f, 0, 'binary.%s', 'nchar.%s', %d, 'binary1.%s', 'nchar1.%s', 'binary2.%s', 'nchar2.%s', 'binary3.%s', 'nchar3.%s', \ + 'binary4.%s', 'nchar4.%s', 'binary5.%s', 'nchar5.%s', 'binary6.%s', 'nchar6.%s', 'binary7.%s', 'nchar7.%s', 'binary8.%s', 'nchar8.%s') ;''' + % (ts + i*1000*60*60*4 +1, fake.random_int(min=-0, max=2147483647, step=1), + fake.random_int(min=-0, max=9223372036854775807, step=1), + fake.random_int(min=-0, max=32767, step=1) , fake.random_int(min=-0, max=127, step=1) , + fake.pyfloat() , fake.pyfloat() , fake.pystr() , fake.pystr() , ts + i, fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , + fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr())) + + tdSql.execute('''insert into stable_2_1 (ts , q_int , q_bigint , q_smallint , q_tinyint , q_float , q_double, q_bool , q_binary , q_nchar, q_ts,\ + q_binary1 , q_nchar1 , q_binary2 , q_nchar2 , q_binary3 , q_nchar3 , q_binary4 , q_nchar4 , q_binary5 , q_nchar5 , q_binary6 , q_nchar6 , q_binary7 , q_nchar7, q_binary8 , q_nchar8) \ + values(%d, %d, %d, %d, %d, %f, %f, 0, 'binary.%s', 'nchar.%s', %d, 'binary1.%s', 'nchar1.%s', 'binary2.%s', 'nchar2.%s', 'binary3.%s', 'nchar3.%s', \ + 'binary4.%s', 'nchar4.%s', 'binary5.%s', 'nchar5.%s', 'binary6.%s', 'nchar6.%s', 'binary7.%s', 'nchar7.%s', 'binary8.%s', 'nchar8.%s') ;''' + % (ts + i*1000*60*60*4 +10, fake.random_int(min=-0, max=2147483647, step=1), + fake.random_int(min=-0, max=9223372036854775807, step=1), + fake.random_int(min=-0, max=32767, step=1) , fake.random_int(min=-0, max=127, step=1) , + fake.pyfloat() , fake.pyfloat() , fake.pystr() , fake.pystr() , ts + i, fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , + fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr() , fake.pystr())) + + tdSql.query("select /*+ para_tables_sort() */count(*) from stable_1;") + tdSql.checkData(0,0,3*num_random*n) + tdSql.query("select /*+ para_tables_sort() */count(*) from regular_table_1;") + tdSql.checkData(0,0,num_random*n) + + def explain_sql(self,sql): + # #执行sql解析 + sql = "explain " + sql + tdLog.info(sql) + tdSql.query(sql) + #pass + + def data_check(self,sql,mark='mark') : + tdLog.info("========mark==%s==="% mark); + try: + tdSql.query(sql,queryTimes=1) + self.explain_sql(sql) + except: + tdLog.info("sql is not support :=====%s; " %sql) + tdSql.error(sql) + + + def math_nest(self,mathlist): + + print("==========%s===start=============" %mathlist) + os.system("rm -rf %s/%s.sql" % (self.testcasePath,self.testcaseFilename)) + + self.dropandcreateDB_random("%s" %self.db_nest, 1) + + if (mathlist == ['ABS','SQRT']) or (mathlist == ['SIN','COS','TAN','ASIN','ACOS','ATAN']) or (mathlist == ['FLOOR','CEIL','ROUND']) \ + or (mathlist == ['CSUM']) : + math_functions = mathlist + fun_fix_column = ['(q_bigint)','(q_smallint)','(q_tinyint)','(q_int)','(q_float)','(q_double)','(q_bigint_null)','(q_smallint_null)','(q_tinyint_null)','(q_int_null)','(q_float_null)','(q_double_null)'] + fun_column_1 = random.sample(math_functions,1)+random.sample(fun_fix_column,1) + math_fun_1 = str(fun_column_1).replace("[","").replace("]","").replace("'","").replace(", ","") + fun_column_2 = random.sample(math_functions,1)+random.sample(fun_fix_column,1) + math_fun_2 = str(fun_column_2).replace("[","").replace("]","").replace("'","").replace(", ","") + + fun_fix_column_j = ['(t1.q_bigint)','(t1.q_smallint)','(t1.q_tinyint)','(t1.q_int)','(t1.q_float)','(t1.q_double)','(t1.q_bigint_null)','(t1.q_smallint_null)','(t1.q_tinyint_null)','(t1.q_int_null)','(t1.q_float_null)','(t1.q_double_null)', + '(t2.q_bigint)','(t2.q_smallint)','(t2.q_tinyint)','(t2.q_int)','(t2.q_float)','(t2.q_double)','(t2.q_bigint_null)','(t2.q_smallint_null)','(t2.q_tinyint_null)','(t2.q_int_null)','(t2.q_float_null)','(t2.q_double_null)'] + fun_column_join_1 = random.sample(math_functions,1)+random.sample(fun_fix_column_j,1) + math_fun_join_1 = str(fun_column_join_1).replace("[","").replace("]","").replace("'","").replace(", ","") + fun_column_join_2 = random.sample(math_functions,1)+random.sample(fun_fix_column_j,1) + math_fun_join_2 = str(fun_column_join_2).replace("[","").replace("]","").replace("'","").replace(", ","") + + elif (mathlist == ['UNIQUE']) or (mathlist == ['HYPERLOGLOG']) or (mathlist == ['MODE']) : + math_functions = mathlist + fun_fix_column = ['(q_bigint)','(q_smallint)','(q_tinyint)','(q_int)','(q_float)','(q_double)','(q_binary)','(q_nchar)','(q_bool)','(q_ts)', + '(q_bigint_null)','(q_smallint_null)','(q_tinyint_null)','(q_int_null)','(q_float_null)','(q_double_null)','(q_binary_null)','(q_nchar_null)','(q_bool_null)','(q_ts_null)'] + fun_column_1 = random.sample(math_functions,1)+random.sample(fun_fix_column,1) + math_fun_1 = str(fun_column_1).replace("[","").replace("]","").replace("'","").replace(", ","") + fun_column_2 = random.sample(math_functions,1)+random.sample(fun_fix_column,1) + math_fun_2 = str(fun_column_2).replace("[","").replace("]","").replace("'","").replace(", ","") + + fun_fix_column_j = ['(t1.q_bigint)','(t1.q_smallint)','(t1.q_tinyint)','(t1.q_int)','(t1.q_float)','(t1.q_double)','(t1.q_bigint_null)','(t1.q_smallint_null)','(t1.q_tinyint_null)','(t1.q_int_null)','(t1.q_float_null)','(t1.q_double_null)','(t1.q_ts)','(t1.q_ts_null)', + '(t2.q_bigint)','(t2.q_smallint)','(t2.q_tinyint)','(t2.q_int)','(t2.q_float)','(t2.q_double)','(t2.q_bigint_null)','(t2.q_smallint_null)','(t2.q_tinyint_null)','(t2.q_int_null)','(t2.q_float_null)','(t2.q_double_null)','(t2.q_ts)','(t2.q_ts_null)'] + fun_column_join_1 = random.sample(math_functions,1)+random.sample(fun_fix_column_j,1) + math_fun_join_1 = str(fun_column_join_1).replace("[","").replace("]","").replace("'","").replace(", ","") + fun_column_join_2 = random.sample(math_functions,1)+random.sample(fun_fix_column_j,1) + math_fun_join_2 = str(fun_column_join_2).replace("[","").replace("]","").replace("'","").replace(", ","") + + elif (mathlist == ['TAIL']): + math_functions = mathlist + num = random.randint(1, 100) + offset_rows = random.randint(0, 100) + fun_fix_column = ['(q_bigint,num)','(q_smallint,num)','(q_tinyint,num)','(q_int,num)','(q_float,num)','(q_double,num)','(q_binary,num)','(q_nchar,num)','(q_bool,num)','(q_ts,num)', + '(q_bigint_null,num)','(q_smallint_null,num)','(q_tinyint_null,num)','(q_int_null,num)','(q_float_null,num)','(q_double_null,num)','(q_binary_null,num)','(q_nchar_null,num)','(q_bool_null,num)','(q_ts_null,num)', + '(q_bigint,num,offset_rows)','(q_smallint,num,offset_rows)','(q_tinyint,num,offset_rows)','(q_int,num,offset_rows)','(q_float,num,offset_rows)','(q_double,num,offset_rows)','(q_binary,num,offset_rows)','(q_nchar,num,offset_rows)','(q_bool,num,offset_rows)','(q_ts,num,offset_rows)', + '(q_bigint_null,num,offset_rows)','(q_smallint_null,num,offset_rows)','(q_tinyint_null,num,offset_rows)','(q_int_null,num,offset_rows)','(q_float_null,num,offset_rows)','(q_double_null,num,offset_rows)','(q_binary_null,num,offset_rows)','(q_nchar_null,num,offset_rows)','(q_bool_null,num,offset_rows)','(q_ts_null,num,offset_rows)'] + fun_column_1 = random.sample(math_functions,1)+random.sample(fun_fix_column,1) + math_fun_1 = str(fun_column_1).replace("[","").replace("]","").replace("'","").replace(", ","").replace("num",str(num)).replace("offset_rows",str(offset_rows)) + fun_column_2 = random.sample(math_functions,1)+random.sample(fun_fix_column,1) + math_fun_2 = str(fun_column_2).replace("[","").replace("]","").replace("'","").replace(", ","").replace("num",str(num)).replace("offset_rows",str(offset_rows)) + + fun_fix_column_j = ['(t1.q_bigint,num)','(t1.q_smallint,num)','(t1.q_tinyint,num)','(t1.q_int,num)','(t1.q_float,num)','(t1.q_double,num)','(t1.q_binary,num)','(t1.q_nchar,num)','(t1.q_bool,num)','(t1.q_ts,num)', + '(t1.q_bigint_null,num)','(t1.q_smallint_null,num)','(t1.q_tinyint_null,num)','(t1.q_int_null,num)','(t1.q_float_null,num)','(t1.q_double_null,num)','(t1.q_binary_null,num)','(t1.q_nchar_null,num)','(t1.q_bool_null,num)','(t1.q_ts_null,num)', + '(t2.q_bigint,num)','(t2.q_smallint,num)','(t2.q_tinyint,num)','(t2.q_int,num)','(t2.q_float,num)','(t2.q_double,num)','(t2.q_binary,num)','(t2.q_nchar,num)','(t2.q_bool,num)','(t2.q_ts,num)', + '(t2.q_bigint_null,num)','(t2.q_smallint_null,num)','(t2.q_tinyint_null,num)','(t2.q_int_null,num)','(t2.q_float_null,num)','(t2.q_double_null,num)','(t2.q_binary_null,num)','(t2.q_nchar_null,num)','(t2.q_bool_null,num)','(t2.q_ts_null,num)', + '(t1.q_bigint,num,offset_rows)','(t1.q_smallint,num,offset_rows)','(t1.q_tinyint,num,offset_rows)','(t1.q_int,num,offset_rows)','(t1.q_float,num,offset_rows)','(t1.q_double,num,offset_rows)','(t1.q_binary,num,offset_rows)','(t1.q_nchar,num,offset_rows)','(t1.q_bool,num,offset_rows)','(t1.q_ts,num,offset_rows)', + '(t1.q_bigint_null,num,offset_rows)','(t1.q_smallint_null,num,offset_rows)','(t1.q_tinyint_null,num,offset_rows)','(t1.q_int_null,num,offset_rows)','(t1.q_float_null,num,offset_rows)','(t1.q_double_null,num,offset_rows)','(t1.q_binary_null,num,offset_rows)','(t1.q_nchar_null,num,offset_rows)','(t1.q_bool_null,num,offset_rows)','(t1.q_ts_null,num,offset_rows)', + '(t2.q_bigint,num,offset_rows)','(t2.q_smallint,num,offset_rows)','(t2.q_tinyint,num,offset_rows)','(t2.q_int,num,offset_rows)','(t2.q_float,num,offset_rows)','(t2.q_double,num,offset_rows)','(t2.q_binary,num,offset_rows)','(t2.q_nchar,num,offset_rows)','(t2.q_bool,num,offset_rows)','(t2.q_ts,num,offset_rows)', + '(t2.q_bigint_null,num,offset_rows)','(t2.q_smallint_null,num,offset_rows)','(t2.q_tinyint_null,num,offset_rows)','(t2.q_int_null,num,offset_rows)','(t2.q_float_null,num,offset_rows)','(t2.q_double_null,num,offset_rows)','(t2.q_binary_null,num,offset_rows)','(t2.q_nchar_null,num,offset_rows)','(t2.q_bool_null,num,offset_rows)','(t2.q_ts_null,num,offset_rows)'] + fun_column_join_1 = random.sample(math_functions,1)+random.sample(fun_fix_column_j,1) + math_fun_join_1 = str(fun_column_join_1).replace("[","").replace("]","").replace("'","").replace(", ","").replace("num",str(num)).replace("offset_rows",str(offset_rows)) + fun_column_join_2 = random.sample(math_functions,1)+random.sample(fun_fix_column_j,1) + math_fun_join_2 = str(fun_column_join_2).replace("[","").replace("]","").replace("'","").replace(", ","").replace("num",str(num)).replace("offset_rows",str(offset_rows)) + + elif (mathlist == ['POW','LOG']) or (mathlist == ['MAVG']) or (mathlist == ['SAMPLE']) : + math_functions = mathlist + num = random.randint(1, 1000) + fun_fix_column = ['(q_bigint,num)','(q_smallint,num)','(q_tinyint,num)','(q_int,num)','(q_float,num)','(q_double,num)', + '(q_bigint_null,num)','(q_smallint_null,num)','(q_tinyint_null,num)','(q_int_null,num)','(q_float_null,num)','(q_double_null,num)'] + fun_column_1 = random.sample(math_functions,1)+random.sample(fun_fix_column,1) + math_fun_1 = str(fun_column_1).replace("[","").replace("]","").replace("'","").replace(", ","").replace("num",str(num)) + fun_column_2 = random.sample(math_functions,1)+random.sample(fun_fix_column,1) + math_fun_2 = str(fun_column_2).replace("[","").replace("]","").replace("'","").replace(", ","").replace("num",str(num)) + + fun_fix_column_j = ['(t1.q_bigint,num)','(t1.q_smallint,num)','(t1.q_tinyint,num)','(t1.q_int,num)','(t1.q_float,num)','(t1.q_double,num)', + '(t1.q_bigint_null,num)','(t1.q_smallint_null,num)','(t1.q_tinyint_null,num)','(t1.q_int_null,num)','(t1.q_float_null,num)','(t1.q_double_null,num)', + '(t2.q_bigint,num)','(t2.q_smallint,num)','(t2.q_tinyint,num)','(t2.q_int,num)','(t2.q_float,num)','(t2.q_double,num)', + '(t2.q_bigint_null,num)','(t2.q_smallint_null,num)','(t2.q_tinyint_null,num)','(t2.q_int_null,num)','(t2.q_float_null,num)','(t2.q_double_null,num)'] + fun_column_join_1 = random.sample(math_functions,1)+random.sample(fun_fix_column_j,1) + math_fun_join_1 = str(fun_column_join_1).replace("[","").replace("]","").replace("'","").replace(", ","").replace("num",str(num)) + fun_column_join_2 = random.sample(math_functions,1)+random.sample(fun_fix_column_j,1) + math_fun_join_2 = str(fun_column_join_2).replace("[","").replace("]","").replace("'","").replace(", ","").replace("num",str(num)) + + elif (mathlist == ['statecount','stateduration']): + math_functions = mathlist + num = random.randint(-1000, 1000) + + operator = ['LT' , 'GT' ,'GE','NE','EQ'] + oper = str(random.sample(operator,1)).replace("[","").replace("]","")#.replace("'","") + + fun_fix_column = ['(q_bigint,oper,num,time)','(q_smallint,oper,num,time)','(q_tinyint,oper,num,time)','(q_int,oper,num,time)','(q_float,oper,num,time)','(q_double,oper,num,time)', + '(q_bigint_null,oper,num,time)','(q_smallint_null,oper,num,time)','(q_tinyint_null,oper,num,time)','(q_int_null,oper,num,time)','(q_float_null,oper,num,time)','(q_double_null,oper,num,time)'] + + hanshu_select1 = random.sample(math_functions,1) + fun_column_1 = random.sample(hanshu_select1,1)+random.sample(fun_fix_column,1) + math_fun_1 = str(fun_column_1).replace("[","").replace("]","").replace("'","").replace(", ","") + + if str(hanshu_select1).replace("[","").replace("]","").replace("'","") == 'statecount': + math_fun_1 = math_fun_1.replace("oper","%s" %oper).replace(",time","").replace("num",str(num)) + elif str(hanshu_select1).replace("[","").replace("]","").replace("'","") == 'stateduration': + timeunit = ['1s' , '1m' ,'1h'] + time = str(random.sample(timeunit,1)).replace("[","").replace("]","").replace("'","") + math_fun_1 = math_fun_1.replace("oper","%s" %oper).replace("time","%s" %time).replace("num",str(num)) + + hanshu_select2 = random.sample(math_functions,1) + fun_column_2 = random.sample(hanshu_select2,1)+random.sample(fun_fix_column,1) + math_fun_2 = str(fun_column_2).replace("[","").replace("]","").replace("'","").replace(", ","") + + if str(hanshu_select2).replace("[","").replace("]","").replace("'","") == 'statecount': + math_fun_2 = math_fun_2.replace("oper","%s" %oper).replace(",time","").replace("num",str(num)) + elif str(hanshu_select2).replace("[","").replace("]","").replace("'","") == 'stateduration': + timeunit = ['1s' , '1m' ,'1h'] + time = str(random.sample(timeunit,1)).replace("[","").replace("]","").replace("'","") + math_fun_2 = math_fun_2.replace("oper","%s" %oper).replace("time","%s" %time).replace("num",str(num)) + + fun_fix_column_j = ['(t1.q_bigint,oper,num,time)','(t1.q_smallint,oper,num,time)','(t1.q_tinyint,oper,num,time)','(t1.q_int,oper,num,time)','(t1.q_float,oper,num,time)','(t1.q_double,oper,num,time)', + '(t1.q_bigint_null,oper,num,time)','(t1.q_smallint_null,oper,num,time)','(t1.q_tinyint_null,oper,num,time)','(t1.q_int_null,oper,num,time)','(t1.q_float_null,oper,num,time)','(t1.q_double_null,oper,num,time)', + '(t2.q_bigint,oper,num,time)','(t2.q_smallint,oper,num,time)','(t2.q_tinyint,oper,num,time)','(t2.q_int,oper,num,time)','(t2.q_float,oper,num,time)','(t2.q_double,oper,num,time)', + '(t2.q_bigint_null,oper,num,time)','(t2.q_smallint_null,oper,num,time)','(t2.q_tinyint_null,oper,num,time)','(t2.q_int_null,oper,num,time)','(t2.q_float_null,oper,num,time)','(t2.q_double_null,oper,num,time)'] + + hanshu_select_join_1 = random.sample(math_functions,1) + fun_column_join_1 = random.sample(hanshu_select_join_1,1)+random.sample(fun_fix_column_j,1) + math_fun_join_1 = str(fun_column_join_1).replace("[","").replace("]","").replace("'","").replace(", ","") + + if str(hanshu_select_join_1).replace("[","").replace("]","").replace("'","") == 'statecount': + math_fun_join_1 = math_fun_join_1.replace("oper","%s" %oper).replace(",time","").replace("num",str(num)) + elif str(hanshu_select_join_1).replace("[","").replace("]","").replace("'","") == 'stateduration': + timeunit = ['1s' , '1m' ,'1h'] + time = str(random.sample(timeunit,1)).replace("[","").replace("]","").replace("'","") + math_fun_join_1 = math_fun_join_1.replace("oper","%s" %oper).replace("time","%s" %time).replace("num",str(num)) + + hanshu_select_join_2 = random.sample(math_functions,1) + fun_column_join_2 = random.sample(hanshu_select_join_2,1)+random.sample(fun_fix_column_j,1) + math_fun_join_2 = str(fun_column_join_2).replace("[","").replace("]","").replace("'","").replace(", ","") + + if str(hanshu_select_join_2).replace("[","").replace("]","").replace("'","") == 'statecount': + math_fun_join_2 = math_fun_join_2.replace("oper","%s" %oper).replace(",time","").replace("num",str(num)) + elif str(hanshu_select_join_2).replace("[","").replace("]","").replace("'","") == 'stateduration': + timeunit = ['1s' , '1m' ,'1h'] + time = str(random.sample(timeunit,1)).replace("[","").replace("]","").replace("'","") + math_fun_join_2 = math_fun_join_2.replace("oper","%s" %oper).replace("time","%s" %time).replace("num",str(num)) + + elif(mathlist == ['HISTOGRAM']) : + math_functions = mathlist + fun_fix_column = ['(q_bigint','(q_smallint','(q_tinyint','(q_int','(q_float','(q_double','(q_bigint_null','(q_smallint_null','(q_tinyint_null','(q_int_null','(q_float_null','(q_double_null'] + + fun_fix_column_j = ['(t1.q_bigint','(t1.q_smallint','(t1.q_tinyint','(t1.q_int','(t1.q_float','(t1.q_double','(t1.q_bigint_null','(t1.q_smallint_null','(t1.q_tinyint_null','(t1.q_int_null','(t1.q_float_null','(t1.q_double_null', + '(t2.q_bigint','(t2.q_smallint','(t2.q_tinyint','(t2.q_int','(t2.q_float','(t2.q_double','(t2.q_bigint_null','(t2.q_smallint_null','(t2.q_tinyint_null','(t2.q_int_null','(t2.q_float_null','(t2.q_double_null'] + + normalized = random.randint(0, 1) + + i = random.randint(1,3) + if i == 1: + bin_type = 'user_input' + bin_description = {-11111119395555977777} #9一会转译成, + fun_column_1 = [math_functions , random.sample(fun_fix_column,1), ',',"'%s'" %bin_type, ',',"'%s'" % bin_description, ',', "%d" %normalized,')'] + math_fun_1 = str(fun_column_1).replace("[","").replace("]","").replace("'","").replace(", ","").replace("{","[").replace("}","]").replace("9",",") + + fun_column_2 = [math_functions , random.sample(fun_fix_column,1), ',',"'%s'" %bin_type, ',',"'%s'" % bin_description, ',', "%d" %normalized,')'] + math_fun_2 = str(fun_column_2).replace("[","").replace("]","").replace("'","").replace(", ","").replace("{","[").replace("}","]").replace("9",",") + + fun_column_join_1 = [math_functions , random.sample(fun_fix_column_j,1), ',',"'%s'" %bin_type, ',',"'%s'" % bin_description, ',', "%d" %normalized,')'] + math_fun_join_1 = str(fun_column_join_1).replace("[","").replace("]","").replace("'","").replace(", ","").replace("{","[").replace("}","]").replace("9",",") + + fun_column_join_2 = [math_functions , random.sample(fun_fix_column_j,1), ',',"'%s'" %bin_type, ',',"'%s'" % bin_description, ',', "%d" %normalized,')'] + math_fun_join_2 = str(fun_column_join_2).replace("[","").replace("]","").replace("'","").replace(", ","").replace("{","[").replace("}","]").replace("9",",") + + elif i == 2: + bin_type = 'linear_bin' + true_false = random.randint(10, 11) + bin_description = {"ZstartZ": -333339, "ZwidthZ":559, "ZcountZ":59, "ZinfinityZ":'%d' %true_false} #Z一会转译成" ,9一会转译成 , + fun_column_1 = [math_functions , random.sample(fun_fix_column,1), ',',"'%s'" %bin_type, ',','%s' % bin_description, ',', "%d" %normalized,')'] + math_fun_1 = str(fun_column_1).replace("[","").replace("]","").replace("'","").replace(", ","").replace("9",",").replace("Z","\"").replace("10","false").replace("11","true").replace("\"{","'{").replace("}\"","}'") + + fun_column_2 = [math_functions , random.sample(fun_fix_column,1), ',',"'%s'" %bin_type, ',','%s' % bin_description, ',', "%d" %normalized,')'] + math_fun_2 = str(fun_column_2).replace("[","").replace("]","").replace("'","").replace(", ","").replace("9",",").replace("Z","\"").replace("10","false").replace("11","true").replace("\"{","'{").replace("}\"","}'") + + fun_column_join_1 = [math_functions , random.sample(fun_fix_column_j,1), ',',"'%s'" %bin_type, ',','%s' % bin_description, ',', "%d" %normalized,')'] + math_fun_join_1 = str(fun_column_join_1).replace("[","").replace("]","").replace("'","").replace(", ","").replace("9",",").replace("Z","\"").replace("10","false").replace("11","true").replace("\"{","'{").replace("}\"","}'") + + fun_column_join_2 = [math_functions , random.sample(fun_fix_column_j,1), ',',"'%s'" %bin_type, ',','%s' % bin_description, ',', "%d" %normalized,')'] + math_fun_join_2 = str(fun_column_join_2).replace("[","").replace("]","").replace("'","").replace(", ","").replace("9",",").replace("Z","\"").replace("10","false").replace("11","true").replace("\"{","'{").replace("}\"","}'") + + elif i == 3: + bin_type = 'log_bin' + true_false = random.randint(10, 11) + bin_description = {"ZstartZ": -333339, "ZfactorZ":559, "ZcountZ":59, "ZinfinityZ":'%d' %true_false} #Z一会转译成" ,9一会转译成 , + fun_column_1 = [math_functions , random.sample(fun_fix_column,1), ',',"'%s'" %bin_type, ',','%s' % bin_description, ',', "%d" %normalized,')'] + math_fun_1 = str(fun_column_1).replace("[","").replace("]","").replace("'","").replace(", ","").replace("9",",").replace("Z","\"").replace("10","false").replace("11","true").replace("\"{","'{").replace("}\"","}'") + + fun_column_2 = [math_functions , random.sample(fun_fix_column,1), ',',"'%s'" %bin_type, ',','%s' % bin_description, ',', "%d" %normalized,')'] + math_fun_2 = str(fun_column_2).replace("[","").replace("]","").replace("'","").replace(", ","").replace("9",",").replace("Z","\"").replace("10","false").replace("11","true").replace("\"{","'{").replace("}\"","}'") + + fun_column_join_1 = [math_functions , random.sample(fun_fix_column_j,1), ',',"'%s'" %bin_type, ',','%s' % bin_description, ',', "%d" %normalized,')'] + math_fun_join_1 = str(fun_column_join_1).replace("[","").replace("]","").replace("'","").replace(", ","").replace("9",",").replace("Z","\"").replace("10","false").replace("11","true").replace("\"{","'{").replace("}\"","}'") + + fun_column_join_2 = [math_functions , random.sample(fun_fix_column_j,1), ',',"'%s'" %bin_type, ',','%s' % bin_description, ',', "%d" %normalized,')'] + math_fun_join_2 = str(fun_column_join_2).replace("[","").replace("]","").replace("'","").replace(", ","").replace("9",",").replace("Z","\"").replace("10","false").replace("11","true").replace("\"{","'{").replace("}\"","}'") + + tdSql.query("select /*+ para_tables_sort() */1-1 as math_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (mathlist == ['SIN','COS','TAN','ASIN','ACOS','ATAN']) or (mathlist == ['ABS','SQRT']) \ + or (mathlist == ['POW','LOG']) or (mathlist == ['FLOOR','CEIL','ROUND']): + sql = "select /*+ para_tables_sort() */ ts1 , floor(asct1) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1, " % math_fun_1 + sql += "%s as asct2, " % math_fun_2 + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts as ts1 from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += ");" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + tdSql.checkRows(100) + self.cur1.execute(sql) + self.explain_sql(sql) + elif (mathlist == ['MAVG']) or (mathlist == ['SAMPLE']) or (mathlist == ['TAIL']) or (mathlist == ['CSUM']) or (mathlist == ['HISTOGRAM']) \ + or (mathlist == ['HYPERLOGLOG']) or (mathlist == ['UNIQUE']) or (mathlist == ['MODE']) or (mathlist == ['statecount','stateduration']) : + sql = "select /*+ para_tables_sort() */ count(asct1) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1 " % math_fun_1 + sql += " from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += ");" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-2 as math_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (mathlist == ['SIN','COS','TAN','ASIN','ACOS','ATAN']) or (mathlist == ['ABS','SQRT']) \ + or (mathlist == ['POW','LOG']) or (mathlist == ['FLOOR','CEIL','ROUND']) : + sql = "select /*+ para_tables_sort() */ ts , abs(asct1) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1, " % math_fun_1 + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts ts from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s )" % random.choice(self.order_where) + sql += "%s " % random.choice(self.unionall_or_union) + sql += "select /*+ para_tables_sort() */ ts , asct2 from ( select /*+ para_tables_sort() */ " + sql += "%s as asct2, " % math_fun_2 + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts ts from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ")" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + elif (mathlist == ['MAVG']) or (mathlist == ['SAMPLE']) or (mathlist == ['TAIL']) or (mathlist == ['CSUM']) or (mathlist == ['HISTOGRAM']) \ + or (mathlist == ['HYPERLOGLOG']) or (mathlist == ['UNIQUE']) or (mathlist == ['MODE']) or (mathlist == ['statecount','stateduration']): + sql = "select /*+ para_tables_sort() */ count(asct1) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1 " % math_fun_1 + sql += "from regular_table_1 where " + sql += "%s )" % random.choice(self.q_where) + sql += "%s " % random.choice(self.unionall_or_union) + sql += "select /*+ para_tables_sort() */ count(asct2) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct2 " % math_fun_2 + sql += " from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ")" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-3 as math_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (mathlist == ['SIN','COS','TAN','ASIN','ACOS','ATAN']) or (mathlist == ['ABS','SQRT']) \ + or (mathlist == ['POW','LOG']) or (mathlist == ['FLOOR','CEIL','ROUND']) : + sql = "select /*+ para_tables_sort() */ ts , min(asct1) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1, ts ," % math_fun_1 + sql += "%s as asct2, " % math_fun_2 + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s " % random.choice(self.q_select) + sql += " from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s select /*+ para_tables_sort() */" % random.choice(self.unionall_or_union) + sql += "%s as asct2, ts ," % math_fun_2 + sql += "%s as asct1, " % math_fun_1 + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s " % random.choice(self.q_select) + sql += " from regular_table_2 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += ")" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + elif (mathlist == ['MODE']) : + sql = "select /*+ para_tables_sort() */ count(asct1) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1," % math_fun_1 + sql += "%s as asct2 " % math_fun_2 + sql += " from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s select /*+ para_tables_sort() */" % random.choice(self.unionall_or_union) + sql += "%s as asct2 ," % math_fun_2 + sql += "%s as asct1 " % math_fun_1 + sql += " from regular_table_2 where " + sql += "%s " % random.choice(self.q_where) + sql += ")" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + elif (mathlist == ['MAVG']) or (mathlist == ['statecount','stateduration']) or (mathlist == ['CSUM']) or (mathlist == ['HISTOGRAM']) \ + or (mathlist == ['TAIL']) or (mathlist == ['HYPERLOGLOG']) or (mathlist == ['UNIQUE']) or (mathlist == ['SAMPLE']): + sql = "select /*+ para_tables_sort() */ count(asct1) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1 " % math_fun_1 + sql += " from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s select /*+ para_tables_sort() */" % random.choice(self.unionall_or_union) + sql += "%s as asct2 " % math_fun_2 + sql += " from regular_table_2 where " + sql += "%s " % random.choice(self.q_where) + sql += " order by asct1 asc " + sql += ")" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-4 as math_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (mathlist == ['SIN','COS','TAN','ASIN','ACOS','ATAN']) or (mathlist == ['ABS','SQRT']) \ + or (mathlist == ['POW','LOG']) or (mathlist == ['FLOOR','CEIL','ROUND']) : + sql = "select /*+ para_tables_sort() */ ts1,ts2 ,timediff(ts1,ts2), asct1 from ( select /*+ para_tables_sort() */ t1.ts as ts1," + sql += "%s as asct0, " % math_fun_join_1 + sql += "%s as asct1, " % math_fun_join_2 + sql += "%s as asct2, " % math_fun_join_1 + sql += "t1.%s as asct12, " % random.choice(self.q_select) + sql += "t2.%s as asct22, " % random.choice(self.q_select) + sql += "t2.ts as ts2 from regular_table_1 t1 , regular_table_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_where) + sql += "and %s " % random.choice(self.q_u_or_where) + sql += "%s " % random.choice(self.order_u_where) + sql += ");" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + tdSql.checkRows(100) + self.cur1.execute(sql) + self.explain_sql(sql) + elif (mathlist == ['MAVG']) or (mathlist == ['SAMPLE'])or (mathlist == ['TAIL']) or (mathlist == ['CSUM']) or (mathlist == ['HISTOGRAM']) \ + or (mathlist == ['HYPERLOGLOG']) or (mathlist == ['UNIQUE']) or (mathlist == ['MODE']) or (mathlist == ['statecount','stateduration']) : + sql = "select /*+ para_tables_sort() */ count(asct1) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1 " % math_fun_join_2 + sql += "from regular_table_1 t1 , regular_table_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_where) + sql += "and %s " % random.choice(self.q_u_or_where) + sql += ");" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-5 as math_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (mathlist == ['SIN','COS','TAN','ASIN','ACOS','ATAN']) or (mathlist == ['ABS','SQRT']) \ + or (mathlist == ['POW','LOG']) or (mathlist == ['FLOOR','CEIL','ROUND']) : + sql = "select /*+ para_tables_sort() */ ts ," + sql += "%s, " % math_fun_1 + sql += "%s as asct1, " % random.choice(self.q_select) + sql += "%s as asct2, " % random.choice(self.q_select) + sql += "%s " % math_fun_2 + sql += " from ( select /*+ para_tables_sort() */ * from regular_table_1 ) where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += " ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + tdSql.checkRows(100) + self.cur1.execute(sql) + self.explain_sql(sql) + elif (mathlist == ['MAVG']) or (mathlist == ['SAMPLE'])or (mathlist == ['TAIL']) or (mathlist == ['CSUM']) or (mathlist == ['HISTOGRAM']) \ + or (mathlist == ['HYPERLOGLOG']) or (mathlist == ['UNIQUE']) or (mathlist == ['MODE']) or (mathlist == ['statecount','stateduration']) : + sql = "select /*+ para_tables_sort() */" + sql += "%s " % math_fun_2 + sql += " from ( select /*+ para_tables_sort() */ * from regular_table_1 ) where " + sql += "%s " % random.choice(self.q_where) + sql += " ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-6 as math_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (mathlist == ['SIN','COS','TAN','ASIN','ACOS','ATAN']) or (mathlist == ['ABS','SQRT']) \ + or (mathlist == ['POW','LOG']) or (mathlist == ['FLOOR','CEIL','ROUND']) : + sql = "select /*+ para_tables_sort() */ ts1 ,timediff(ts1,ts2), max(asct1) from ( select /*+ para_tables_sort() */ t1.ts,t1.ts as ts1," + sql += "%s as asct0, " % math_fun_join_1 + sql += "%s as asct1, " % math_fun_join_2 + sql += "t1.%s as asct11, " % random.choice(self.q_select) + sql += "t2.%s as asct12, " % random.choice(self.q_select) + sql += "%s as asct13, " % math_fun_join_1 + sql += "t2.ts as ts2 from regular_table_1 t1 , regular_table_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_where) + sql += "and %s )" % random.choice(self.q_u_or_where) + sql += "%s " % random.choice(self.order_where) + sql += "%s " % random.choice(self.limit1_where) + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + elif (mathlist == ['MAVG']) or (mathlist == ['SAMPLE'])or (mathlist == ['TAIL']) or (mathlist == ['CSUM']) or (mathlist == ['HISTOGRAM']) \ + or (mathlist == ['HYPERLOGLOG']) or (mathlist == ['UNIQUE']) or (mathlist == ['MODE']) or (mathlist == ['statecount','stateduration']) : + sql = "select /*+ para_tables_sort() */ count(asct1) from ( select /*+ para_tables_sort() */" + sql += "%s as asct1 " % math_fun_join_2 + sql += "from regular_table_1 t1 , regular_table_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_where) + sql += "and %s )" % random.choice(self.q_u_or_where) + sql += "%s " % random.choice(self.limit1_where) + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-7 as math_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (mathlist == ['SIN','COS','TAN','ASIN','ACOS','ATAN']) or (mathlist == ['ABS','SQRT']) \ + or (mathlist == ['POW','LOG']) or (mathlist == ['FLOOR','CEIL','ROUND']) : + sql = "select /*+ para_tables_sort() */ ts1,ts2 , abs(asct1) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1, ts as ts1," % math_fun_1 + sql += "%s as asct2, " % math_fun_2 + sql += "%s, " % random.choice(self.q_select) + sql += "%s, " % random.choice(self.t_select) + sql += "ts as ts2 from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += "%s " % random.choice(self.order_where) + sql += ") ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + tdSql.checkRows(300) + self.cur1.execute(sql) + self.explain_sql(sql) + elif (mathlist == ['MAVG']) or (mathlist == ['SAMPLE']) or (mathlist == ['TAIL']) or (mathlist == ['CSUM']) or (mathlist == ['HISTOGRAM']) \ + or (mathlist == ['HYPERLOGLOG']) or (mathlist == ['UNIQUE']) or (mathlist == ['MODE']) or (mathlist == ['statecount','stateduration']) : + sql = "select /*+ para_tables_sort() */ count(asct1) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1 " % math_fun_1 + sql += "from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += ") ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-8 as math_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (mathlist == ['SIN','COS','TAN','ASIN','ACOS','ATAN']) or (mathlist == ['ABS','SQRT']) \ + or (mathlist == ['POW','LOG']) or (mathlist == ['FLOOR','CEIL','ROUND']) : + sql = "select /*+ para_tables_sort() */ ts1,floor(asct1),ts2 " + sql += "from ( select /*+ para_tables_sort() */ " + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s as asct1, ts as ts1," % math_fun_1 + sql += "%s as asct2, " % math_fun_2 + sql += "%s, " % random.choice(self.q_select) + sql += "%s, " % random.choice(self.t_select) + sql += "ts as ts2 from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += "%s " % random.choice(self.order_where) + sql += ") ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + tdSql.checkRows(300) + self.cur1.execute(sql) + self.explain_sql(sql) + elif (mathlist == ['MAVG']) or (mathlist == ['SAMPLE'])or (mathlist == ['TAIL']) or (mathlist == ['CSUM']) or (mathlist == ['HISTOGRAM']) \ + or (mathlist == ['HYPERLOGLOG']) or (mathlist == ['UNIQUE']) or (mathlist == ['MODE']) or (mathlist == ['statecount','stateduration']) : + sql = "select /*+ para_tables_sort() */ count(asct1) " + sql += "from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1 " % math_fun_1 + sql += " from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += ") ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-9 as math_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (mathlist == ['SIN','COS','TAN','ASIN','ACOS','ATAN']) or (mathlist == ['ABS','SQRT']) \ + or (mathlist == ['POW','LOG']) or (mathlist == ['FLOOR','CEIL','ROUND']) : + sql = "select /*+ para_tables_sort() */ ts1,ts2 ,timediff(ts1,ts2) , max(asct1) from ( select /*+ para_tables_sort() */ t1.ts as ts1," + sql += "%s, " % math_fun_join_1 + sql += "%s as asct1, " % math_fun_join_2 + sql += "t1.%s as asct11, " % random.choice(self.q_select) + sql += "t1.%s as asct12, " % random.choice(self.q_select) + sql += "t2.%s as asct21, " % random.choice(self.q_select) + sql += "t2.%s as asct22, " % random.choice(self.q_select) + sql += "t2.ts as ts2 from stable_1 t1 , stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.t_join_where) + sql += "and %s " % random.choice(self.t_u_where) + sql += "and %s " % random.choice(self.t_u_or_where) + sql += "%s " % random.choice(self.order_u_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ") ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + elif (mathlist == ['MAVG']) or (mathlist == ['SAMPLE'])or (mathlist == ['TAIL']) or (mathlist == ['CSUM']) or (mathlist == ['HISTOGRAM']) \ + or (mathlist == ['HYPERLOGLOG']) or (mathlist == ['UNIQUE']) or (mathlist == ['MODE']) or (mathlist == ['statecount','stateduration']) : + sql = "select /*+ para_tables_sort() */ count(asct1) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1 " % math_fun_join_2 + sql += "from stable_1 t1 , stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.t_join_where) + sql += "and %s " % random.choice(self.t_u_where) + sql += "and %s " % random.choice(self.t_u_or_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ") ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + self.restartDnodes() + tdSql.query("select /*+ para_tables_sort() */1-10 as math_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (mathlist == ['SIN','COS','TAN','ASIN','ACOS','ATAN']) or (mathlist == ['ABS','SQRT']) \ + or (mathlist == ['POW','LOG']) or (mathlist == ['FLOOR','CEIL','ROUND']) : + sql = "select /*+ para_tables_sort() */ ts , min(asct1) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1 ," % math_fun_1 + sql += "%s as asct2, " % math_fun_2 + sql += "%s, " % random.choice(self.s_r_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += ") %s " % random.choice(self.unionall_or_union) + sql += "select /*+ para_tables_sort() */ ts , max(asct2) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1 ," % math_fun_1 + sql += "%s as asct2, " % math_fun_2 + sql += "%s, " % random.choice(self.s_r_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts from stable_2 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ")" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + elif (mathlist == ['MAVG']) or (mathlist == ['SAMPLE'])or (mathlist == ['TAIL']) or (mathlist == ['CSUM']) or (mathlist == ['HISTOGRAM']) \ + or (mathlist == ['HYPERLOGLOG']) or (mathlist == ['UNIQUE']) or (mathlist == ['MODE']) or (mathlist == ['statecount','stateduration']) : + sql = "select /*+ para_tables_sort() */ count(asct1) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1 " % math_fun_1 + sql += " from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += ") %s " % random.choice(self.unionall_or_union) + sql += "select /*+ para_tables_sort() */ count(asct2) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct2 " % math_fun_2 + sql += "from stable_2 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ")" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + #3 inter union not support + tdSql.query("select /*+ para_tables_sort() */1-11 as math_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (mathlist == ['SIN','COS','TAN','ASIN','ACOS','ATAN']) or (mathlist == ['ABS','SQRT']) \ + or (mathlist == ['POW','LOG']) or (mathlist == ['FLOOR','CEIL','ROUND']) : + sql = "select /*+ para_tables_sort() */ min(asct1), max(asct2) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1, ts ," % math_fun_1 + sql += "%s as asct2, " % math_fun_2 + sql += "%s, " % random.choice(self.s_r_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts as t2ts from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += " %s " % random.choice(self.unionall_or_union) + sql += " select /*+ para_tables_sort() */ " + sql += "%s as asct1, ts as t1ts," % math_fun_1 + sql += "%s as asct2, " % math_fun_2 + sql += "%s, " % random.choice(self.s_r_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts as t2ts from stable_2 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ")" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + elif (mathlist == ['MAVG']) or (mathlist == ['SAMPLE'])or (mathlist == ['TAIL']) or (mathlist == ['CSUM']) or (mathlist == ['HISTOGRAM']) \ + or (mathlist == ['HYPERLOGLOG']) or (mathlist == ['UNIQUE']) or (mathlist == ['MODE']) or (mathlist == ['statecount','stateduration']) : + sql = "select /*+ para_tables_sort() */ count(asct1) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1 " % math_fun_1 + sql += " from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += " %s " % random.choice(self.unionall_or_union) + sql += " select /*+ para_tables_sort() */ " + sql += "%s as asct2 " % math_fun_2 + sql += " from stable_2 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ")" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-12 as math_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (mathlist == ['SIN','COS','TAN','ASIN','ACOS','ATAN']) or (mathlist == ['ABS','SQRT']) \ + or (mathlist == ['POW','LOG']) or (mathlist == ['FLOOR','CEIL','ROUND']) : + sql = "select /*+ para_tables_sort() */ ts1,ts2 ,timediff(ts1,ts2), max(asct1) from ( select /*+ para_tables_sort() */ t1.ts as ts1," + sql += "%s, " % math_fun_join_1 + sql += "%s as asct1, " % math_fun_join_2 + sql += "t1.%s as asct11, " % random.choice(self.q_select) + sql += "t1.%s as asct12, " % random.choice(self.q_select) + sql += "t2.%s as asct21, " % random.choice(self.q_select) + sql += "t2.%s as asct111, " % random.choice(self.q_select) + sql += "t2.ts as ts2 from stable_1 t1 , stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.t_u_where) + sql += "and %s " % random.choice(self.t_u_or_where) + sql += "%s " % random.choice(self.order_u_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ");" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + elif (mathlist == ['MAVG']) or (mathlist == ['SAMPLE'])or (mathlist == ['TAIL']) or (mathlist == ['CSUM']) or (mathlist == ['HISTOGRAM']) \ + or (mathlist == ['HYPERLOGLOG']) or (mathlist == ['UNIQUE']) or (mathlist == ['MODE']) or (mathlist == ['statecount','stateduration']) : + sql = "select /*+ para_tables_sort() */count(asct1) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1 " % math_fun_join_2 + sql += " from stable_1 t1 , stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.t_u_where) + sql += "and %s " % random.choice(self.t_u_or_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ");" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-13 as math_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (mathlist == ['SIN','COS','TAN','ASIN','ACOS','ATAN']) or (mathlist == ['ABS','SQRT']) \ + or (mathlist == ['POW','LOG']) or (mathlist == ['FLOOR','CEIL','ROUND']) : + sql = "select /*+ para_tables_sort() */ ts ," + sql += "%s as asct11, " % math_fun_1 + sql += "%s as asct12, " % random.choice(self.q_select) + sql += "%s as asct13, " % random.choice(self.q_select) + sql += "%s as asct14, " % math_fun_2 + sql += "%s as asct15 " % random.choice(self.t_select) + sql += " from ( select /*+ para_tables_sort() */ * from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += "%s " % random.choice(self.order_where) + sql += ") ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + tdSql.checkRows(300) + self.cur1.execute(sql) + self.explain_sql(sql) + elif (mathlist == ['MAVG']) or (mathlist == ['SAMPLE'])or (mathlist == ['TAIL']) or (mathlist == ['CSUM']) or (mathlist == ['HISTOGRAM']) \ + or (mathlist == ['HYPERLOGLOG']) or (mathlist == ['UNIQUE']) or (mathlist == ['MODE']) or (mathlist == ['statecount','stateduration']) : + sql = "select /*+ para_tables_sort() */ " + sql += "%s " % math_fun_2 + sql += "%s " % random.choice(self.t_select) + sql += " from ( select /*+ para_tables_sort() */ * from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += ") ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-14 as math_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (mathlist == ['SIN','COS','TAN','ASIN','ACOS','ATAN']) or (mathlist == ['ABS','SQRT']) \ + or (mathlist == ['POW','LOG']) or (mathlist == ['FLOOR','CEIL','ROUND']) : + sql = "select /*+ para_tables_sort() */ avg(asct1),count(asct2) from ( select /*+ para_tables_sort() */" + sql += "%s as asct1, " % math_fun_1 + sql += "%s as asct2" % math_fun_2 + sql += " from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.partiton_where) + sql += "%s " % random.choice(self.order_desc_where) + sql += "%s " % random.choice([self.limit_where[2] , self.limit_where[3]] ) + sql += " ) ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + elif (mathlist == ['MAVG']) or (mathlist == ['SAMPLE'])or (mathlist == ['TAIL']) or (mathlist == ['CSUM']) or (mathlist == ['HISTOGRAM']) \ + or (mathlist == ['HYPERLOGLOG']) or (mathlist == ['UNIQUE']) or (mathlist == ['MODE']) or (mathlist == ['statecount','stateduration']) : + sql = "select /*+ para_tables_sort() */ count(asct1) from ( select /*+ para_tables_sort() */" + sql += "%s as asct1 " % math_fun_1 + sql += " from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.partiton_where) + sql += "%s " % random.choice([self.limit_where[2] , self.limit_where[3]] ) + sql += " ) ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-15 as math_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (mathlist == ['SIN','COS','TAN','ASIN','ACOS','ATAN']) or (mathlist == ['ABS','SQRT']) \ + or (mathlist == ['POW','LOG']) or (mathlist == ['FLOOR','CEIL','ROUND']) : + sql = "select /*+ para_tables_sort() */ ts1,ts ,timediff(ts1,ts), max(asct1) from ( select /*+ para_tables_sort() */ t1.ts as ts1," + sql += "%s, " % math_fun_join_1 + sql += "%s as asct1, " % math_fun_join_2 + sql += "t1.%s as asct11, " % random.choice(self.q_select) + sql += "t1.%s as asct12, " % random.choice(self.q_select) + sql += "t2.%s as asct13, " % random.choice(self.q_select) + sql += "t2.%s as asct14, " % random.choice(self.q_select) + sql += "t2.ts from stable_1 t1, stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.t_join_where) + sql += " and %s " % random.choice(self.qt_u_or_where) + sql += "%s " % random.choice(self.partiton_where_j) + sql += "%s " % random.choice(self.slimit1_where) + sql += ") " + sql += "%s " % random.choice(self.order_desc_where) + sql += "%s ;" % random.choice(self.limit_u_where) + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + elif (mathlist == ['MAVG']) or (mathlist == ['SAMPLE']) or (mathlist == ['TAIL']) or (mathlist == ['CSUM']) or (mathlist == ['HISTOGRAM']) \ + or (mathlist == ['HYPERLOGLOG']) or (mathlist == ['UNIQUE']) or (mathlist == ['MODE']) or (mathlist == ['statecount','stateduration']) : + sql = "select /*+ para_tables_sort() */ count(asct1) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1 " % math_fun_join_2 + sql += "from stable_1 t1, stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.t_join_where) + sql += " and %s " % random.choice(self.qt_u_or_where) + sql += "%s " % random.choice(self.partiton_where_j) + sql += "%s " % random.choice(self.slimit1_where) + sql += ") " + sql += "%s ;" % random.choice(self.limit_u_where) + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + #taos -f sql + # startTime_taosf = time.time() + print("taos -f %s sql start!" %mathlist) + taos_cmd1 = "taos -f %s/%s.sql" % (self.testcasePath,self.testcaseFilename) + #_ = subprocess.check_output(taos_cmd1, shell=True).decode("utf-8") + _ = subprocess.check_output(taos_cmd1, shell=True) + print("taos -f %s sql over!" %mathlist) + # endTime_taosf = time.time() + # print("taos_f total time %ds" % (endTime_taos_f - startTime_taos_f)) + + print("=========%s====over=============" %mathlist) + + + def str_nest(self,strlist): + + print("==========%s===start=============" %strlist) + os.system("rm -rf %s/%s.sql" % (self.testcasePath,self.testcaseFilename)) + + self.dropandcreateDB_random("%s" %self.db_nest, 1) + + if (strlist == ['LTRIM','RTRIM','LOWER','UPPER']) or (strlist == ['LENGTH','CHAR_LENGTH']) \ + or (strlist == ['']): + str_functions = strlist + fun_fix_column = ['(q_nchar)','(q_binary)','(q_nchar_null)','(q_binary_null)'] + fun_column_1 = random.sample(str_functions,1)+random.sample(fun_fix_column,1) + str_fun_1 = str(fun_column_1).replace("[","").replace("]","").replace("'","").replace(", ","") + fun_column_2 = random.sample(str_functions,1)+random.sample(fun_fix_column,1) + str_fun_2 = str(fun_column_2).replace("[","").replace("]","").replace("'","").replace(", ","") + + fun_fix_column_j = ['(t1.q_nchar)','(t1.q_binary)','(t1.q_nchar_null)','(t1.q_binary_null)', + '(t2.q_nchar)','(t2.q_binary)','(t2.q_nchar_null)','(t2.q_binary_null)'] + fun_column_join_1 = random.sample(str_functions,1)+random.sample(fun_fix_column_j,1) + str_fun_join_1 = str(fun_column_join_1).replace("[","").replace("]","").replace("'","").replace(", ","") + fun_column_join_2 = random.sample(str_functions,1)+random.sample(fun_fix_column_j,1) + str_fun_join_2 = str(fun_column_join_2).replace("[","").replace("]","").replace("'","").replace(", ","") + + fun_fix_column_s = ['(q_nchar)','(q_binary)','(q_nchar_null)','(q_binary_null)','(loc)','(tbname)'] + fun_column_s_1 = random.sample(str_functions,1)+random.sample(fun_fix_column_s,1) + str_fun_s_1 = str(fun_column_s_1).replace("[","").replace("]","").replace("'","").replace(", ","") + fun_column_s_2 = random.sample(str_functions,1)+random.sample(fun_fix_column_s,1) + str_fun_s_2 = str(fun_column_s_2).replace("[","").replace("]","").replace("'","").replace(", ","") + + fun_fix_column_s_j = ['(t1.q_nchar)','(t1.q_binary)','(t1.q_nchar_null)','(t1.q_binary_null)','(t1.loc)','(t1.tbname)', + '(t2.q_nchar)','(t2.q_binary)','(t2.q_nchar_null)','(t2.q_binary_null)','(t2.loc)','(t2.tbname)'] + fun_column_join_s_1 = random.sample(str_functions,1)+random.sample(fun_fix_column_j,1) + str_fun_join_s_1 = str(fun_column_join_1).replace("[","").replace("]","").replace("'","").replace(", ","") + fun_column_join_s_2 = random.sample(str_functions,1)+random.sample(fun_fix_column_j,1) + str_fun_join_s_2 = str(fun_column_join_2).replace("[","").replace("]","").replace("'","").replace(", ","") + + elif (strlist == ['SUBSTR']) : + str_functions = strlist + pos = random.randint(1, 20) + sub_len = random.randint(1, 10) + fun_fix_column = ['(q_nchar,pos)','(q_binary,pos)','(q_nchar_null,pos)','(q_binary_null,pos)', + '(q_nchar,pos,sub_len)','(q_binary,pos,sub_len)','(q_nchar_null,pos,sub_len)','(q_binary_null,pos,sub_len)',] + fun_column_1 = random.sample(str_functions,1)+random.sample(fun_fix_column,1) + str_fun_1 = str(fun_column_1).replace("[","").replace("]","").replace("'","").replace(", ","").replace("pos",str(pos)).replace("sub_len",str(sub_len)) + fun_column_2 = random.sample(str_functions,1)+random.sample(fun_fix_column,1) + str_fun_2 = str(fun_column_2).replace("[","").replace("]","").replace("'","").replace(", ","").replace("pos",str(pos)).replace("sub_len",str(sub_len)) + + fun_fix_column_j = ['(t1.q_nchar,pos)','(t1.q_binary,pos)','(t1.q_nchar_null,pos)','(t1.q_binary_null,pos)', + '(t1.q_nchar,pos,sub_len)','(t1.q_binary,pos,sub_len)','(t1.q_nchar_null,pos,sub_len)','(t1.q_binary_null,pos,sub_len)', + '(t2.q_nchar,pos)','(t2.q_binary,pos)','(t2.q_nchar_null,pos)','(t2.q_binary_null,pos)', + '(t2.q_nchar,pos,sub_len)','(t2.q_binary,pos,sub_len)','(t2.q_nchar_null,pos,sub_len)','(t2.q_binary_null,pos,sub_len)'] + fun_column_join_1 = random.sample(str_functions,1)+random.sample(fun_fix_column_j,1) + str_fun_join_1 = str(fun_column_join_1).replace("[","").replace("]","").replace("'","").replace(", ","").replace("pos",str(pos)).replace("sub_len",str(sub_len)) + fun_column_join_2 = random.sample(str_functions,1)+random.sample(fun_fix_column_j,1) + str_fun_join_2 = str(fun_column_join_2).replace("[","").replace("]","").replace("'","").replace(", ","").replace("pos",str(pos)).replace("sub_len",str(sub_len)) + + fun_fix_column_s = ['(q_nchar,pos)','(q_binary,pos)','(q_nchar_null,pos)','(q_binary_null,pos)','(loc,pos)', + '(q_nchar,pos,sub_len)','(q_binary,pos,sub_len)','(q_nchar_null,pos,sub_len)','(q_binary_null,pos,sub_len)','(loc,pos,sub_len)',] + fun_column_s_1 = random.sample(str_functions,1)+random.sample(fun_fix_column_s,1) + str_fun_s_1 = str(fun_column_1).replace("[","").replace("]","").replace("'","").replace(", ","").replace("pos",str(pos)).replace("sub_len",str(sub_len)) + fun_column_s_2 = random.sample(str_functions,1)+random.sample(fun_fix_column_s,1) + str_fun_s_2 = str(fun_column_2).replace("[","").replace("]","").replace("'","").replace(", ","").replace("pos",str(pos)).replace("sub_len",str(sub_len)) + + fun_fix_column_s_j = ['(t1.q_nchar,pos)','(t1.q_binary,pos)','(t1.q_nchar_null,pos)','(t1.q_binary_null,pos)','(t1.loc,pos)', + '(t1.q_nchar,pos,sub_len)','(t1.q_binary,pos,sub_len)','(t1.q_nchar_null,pos,sub_len)','(t1.q_binary_null,pos,sub_len)','(t1.loc,pos,sub_len)', + '(t2.q_nchar,pos)','(t2.q_binary,pos)','(t2.q_nchar_null,pos)','(t2.q_binary_null,pos)','(t2.loc,pos)', + '(t2.q_nchar,pos,sub_len)','(t2.q_binary,pos,sub_len)','(t2.q_nchar_null,pos,sub_len)','(t2.q_binary_null,pos,sub_len)','(t2.loc,pos,sub_len)'] + fun_column_join_s_1 = random.sample(str_functions,1)+random.sample(fun_fix_column_s_j,1) + str_fun_join_s_1 = str(fun_column_join_1).replace("[","").replace("]","").replace("'","").replace(", ","").replace("pos",str(pos)).replace("sub_len",str(sub_len)) + fun_column_join_s_2 = random.sample(str_functions,1)+random.sample(fun_fix_column_s_j,1) + str_fun_join_s_2 = str(fun_column_join_2).replace("[","").replace("]","").replace("'","").replace(", ","").replace("pos",str(pos)).replace("sub_len",str(sub_len)) + + elif (strlist == ['CONCAT']) : + str_functions = strlist + i = random.randint(2,4) + fun_fix_column = ['q_nchar','q_nchar1','q_nchar2','q_nchar3','q_nchar4','q_nchar5','q_nchar6','q_nchar7','q_nchar8','q_nchar_null', + 'q_binary','q_binary1','q_binary2','q_binary3','q_binary4','q_binary5','q_binary6','q_binary7','q_binary8','q_binary_null'] + + column1 = str(random.sample(fun_fix_column,i)).replace("[","").replace("]","").replace("'","") + fun_column_1 = str(random.sample(str_functions,1))+'('+column1+')' + str_fun_1 = str(fun_column_1).replace("[","").replace("]","").replace("'","") + + column2 = str(random.sample(fun_fix_column,i)).replace("[","").replace("]","").replace("'","") + fun_column_2 = str(random.sample(str_functions,1))+'('+column2+')' + str_fun_2 = str(fun_column_2).replace("[","").replace("]","").replace("'","") + + fun_fix_column_j = ['(t1.q_nchar)','(t1.q_nchar1)','(t1.q_nchar2)','(t1.q_nchar3)','(t1.q_nchar4)','(t1.q_nchar5)','(t1.q_nchar6)','(t1.q_nchar7)','(t1.q_nchar8)','(t1.q_nchar_null)', + '(t2.q_nchar)','(t2.q_nchar1)','(t2.q_nchar2)','(t2.q_nchar3)','(t2.q_nchar4)','(t2.q_nchar5)','(t2.q_nchar6)','(t2.q_nchar7)','(t2.q_nchar8)','(t2.q_nchar_null)', + '(t1.q_binary)','(t1.q_binary1)','(t1.q_binary2)','(t1.q_binary3)','(t1.q_binary4)','(t1.q_binary5)','(t1.q_binary6)','(t1.q_binary7)','(t1.q_binary8)','(t1.q_binary_null)', + '(t2.q_binary)','(t2.q_binary1)','(t2.q_binary2)','(t2.q_binary3)','(t2.q_binary4)','(t2.q_binary5)','(t2.q_binary6)','(t2.q_binary7)','(t2.q_binary8)','(t2.q_binary_null)'] + + column_j1 = str(random.sample(fun_fix_column_j,i)).replace("[","").replace("]","").replace("'","") + fun_column_join_1 = str(random.sample(str_functions,1))+'('+column_j1+')' + str_fun_join_1 = str(fun_column_join_1).replace("[","").replace("]","").replace("'","") + + column_j2 = str(random.sample(fun_fix_column_j,i)).replace("[","").replace("]","").replace("'","") + fun_column_join_2 = str(random.sample(str_functions,1))+'('+column_j2+')' + str_fun_join_2 = str(fun_column_join_2).replace("[","").replace("]","").replace("'","") + + fun_fix_column_s = ['q_nchar','q_nchar1','q_nchar2','q_nchar3','q_nchar4','q_nchar5','q_nchar6','q_nchar7','q_nchar8','loc','q_nchar_null', + 'q_binary','q_binary1','q_binary2','q_binary3','q_binary4','q_binary5','q_binary6','q_binary7','q_binary8','q_binary_null'] + + column_s1 = str(random.sample(fun_fix_column,i)).replace("[","").replace("]","").replace("'","") + fun_column_s_1 = str(random.sample(str_functions,1))+'('+column_s1+')' + str_fun_s_1 = str(fun_column_s_1).replace("[","").replace("]","").replace("'","") + + column_s2 = str(random.sample(fun_fix_column,i)).replace("[","").replace("]","").replace("'","") + fun_column_s_2 = str(random.sample(str_functions,1))+'('+column_s2+')' + str_fun_s_2 = str(fun_column_s_2).replace("[","").replace("]","").replace("'","") + + fun_fix_column_s_j = ['(t1.q_nchar)','(t1.q_nchar1)','(t1.q_nchar2)','(t1.q_nchar3)','(t1.q_nchar4)','(t1.q_nchar5)','(t1.q_nchar6)','(t1.q_nchar7)','(t1.q_nchar8)','(t1.q_nchar_null)','(t1.loc)', + '(t2.q_nchar)','(t2.q_nchar1)','(t2.q_nchar2)','(t2.q_nchar3)','(t2.q_nchar4)','(t2.q_nchar5)','(t2.q_nchar6)','(t2.q_nchar7)','(t2.q_nchar8)','(t2.q_nchar_null)','(t2.loc)', + '(t1.q_binary)','(t1.q_binary1)','(t1.q_binary2)','(t1.q_binary3)','(t1.q_binary4)','(t1.q_binary5)','(t1.q_binary6)','(t1.q_binary7)','(t1.q_binary8)','(t1.q_binary_null)', + '(t2.q_binary)','(t2.q_binary1)','(t2.q_binary2)','(t2.q_binary3)','(t2.q_binary4)','(t2.q_binary5)','(t2.q_binary6)','(t2.q_binary7)','(t2.q_binary8)','(t2.q_binary_null)'] + + column_j_s1 = str(random.sample(fun_fix_column_s_j,i)).replace("[","").replace("]","").replace("'","") + fun_column_join_s_1 = str(random.sample(str_functions,1))+'('+column_j_s1+')' + str_fun_join_s_1 = str(fun_column_join_s_1).replace("[","").replace("]","").replace("'","") + + column_j_s2 = str(random.sample(fun_fix_column_s_j,i)).replace("[","").replace("]","").replace("'","") + fun_column_join_s_2 = str(random.sample(str_functions,1))+'('+column_j_s2+')' + str_fun_join_s_2 = str(fun_column_join_s_2).replace("[","").replace("]","").replace("'","") + + elif (strlist == ['CONCAT_WS']): + str_functions = strlist + i = random.randint(2,4) + fun_fix_column = ['q_nchar','q_nchar1','q_nchar2','q_nchar3','q_nchar4','q_nchar5','q_nchar6','q_nchar7','q_nchar8','q_nchar_null', + 'q_binary','q_binary1','q_binary2','q_binary3','q_binary4','q_binary5','q_binary6','q_binary7','q_binary8','q_binary_null'] + + separators = ['',' ','abc','123','!','@','#','$','%','^','&','*','(',')','-','_','+','=','{', + '[','}',']','|',';',':',',','.','<','>','?','/','~','`','taos','涛思'] + separator = str(random.sample(separators,i)).replace("[","").replace("]","") + + column1 = str(random.sample(fun_fix_column,i)).replace("[","").replace("]","").replace("'","") + fun_column_1 = str(random.sample(str_functions,1))+'('+'\"'+separator+'\",'+column1+')' + str_fun_1 = str(fun_column_1).replace("[","").replace("]","").replace("'","") + + column2 = str(random.sample(fun_fix_column,i)).replace("[","").replace("]","").replace("'","") + fun_column_2 = str(random.sample(str_functions,1))+'('+'\"'+separator+'\",'+column2+')' + str_fun_2 = str(fun_column_2).replace("[","").replace("]","").replace("'","") + + fun_fix_column_j = ['(t1.q_nchar)','(t1.q_nchar1)','(t1.q_nchar2)','(t1.q_nchar3)','(t1.q_nchar4)','(t1.q_nchar5)','(t1.q_nchar6)','(t1.q_nchar7)','(t1.q_nchar8)','(t1.q_nchar_null)', + '(t2.q_nchar)','(t2.q_nchar1)','(t2.q_nchar2)','(t2.q_nchar3)','(t2.q_nchar4)','(t2.q_nchar5)','(t2.q_nchar6)','(t2.q_nchar7)','(t2.q_nchar8)','(t2.q_nchar_null)', + '(t1.q_binary)','(t1.q_binary1)','(t1.q_binary2)','(t1.q_binary3)','(t1.q_binary4)','(t1.q_binary5)','(t1.q_binary6)','(t1.q_binary7)','(t1.q_binary8)','(t1.q_binary_null)', + '(t2.q_binary)','(t2.q_binary1)','(t2.q_binary2)','(t2.q_binary3)','(t2.q_binary4)','(t2.q_binary5)','(t2.q_binary6)','(t2.q_binary7)','(t2.q_binary8)','(t2.q_binary_null)'] + + column_j1 = str(random.sample(fun_fix_column_j,i)).replace("[","").replace("]","").replace("'","") + fun_column_join_1 = str(random.sample(str_functions,1))+'('+'\"'+separator+'\",'+column_j1+')' + str_fun_join_1 = str(fun_column_join_1).replace("[","").replace("]","").replace("'","") + + column_j2 = str(random.sample(fun_fix_column_j,i)).replace("[","").replace("]","").replace("'","") + fun_column_join_2 = str(random.sample(str_functions,1))+'('+'\"'+separator+'\",'+column_j2+')' + str_fun_join_2 = str(fun_column_join_2).replace("[","").replace("]","").replace("'","") + + fun_fix_column_s = ['q_nchar','q_nchar1','q_nchar2','q_nchar3','q_nchar4','q_nchar5','q_nchar6','q_nchar7','q_nchar8','loc','q_nchar_null', + 'q_binary','q_binary1','q_binary2','q_binary3','q_binary4','q_binary5','q_binary6','q_binary7','q_binary8','q_binary_null'] + + column_s1 = str(random.sample(fun_fix_column,i)).replace("[","").replace("]","").replace("'","") + fun_column_s_1 = str(random.sample(str_functions,1))+'('+'\"'+separator+'\",'+column_s1+')' + str_fun_s_1 = str(fun_column_s_1).replace("[","").replace("]","").replace("'","") + + column_s2 = str(random.sample(fun_fix_column,i)).replace("[","").replace("]","").replace("'","") + fun_column_s_2 = str(random.sample(str_functions,1))+'('+'\"'+separator+'\",'+column_s2+')' + str_fun_s_2 = str(fun_column_s_2).replace("[","").replace("]","").replace("'","") + + fun_fix_column_s_j = ['(t1.q_nchar)','(t1.q_nchar1)','(t1.q_nchar2)','(t1.q_nchar3)','(t1.q_nchar4)','(t1.q_nchar5)','(t1.q_nchar6)','(t1.q_nchar7)','(t1.q_nchar8)','(t1.q_nchar_null)','(t1.loc)', + '(t2.q_nchar)','(t2.q_nchar1)','(t2.q_nchar2)','(t2.q_nchar3)','(t2.q_nchar4)','(t2.q_nchar5)','(t2.q_nchar6)','(t2.q_nchar7)','(t2.q_nchar8)','(t2.q_nchar_null)','(t2.loc)', + '(t1.q_binary)','(t1.q_binary1)','(t1.q_binary2)','(t1.q_binary3)','(t1.q_binary4)','(t1.q_binary5)','(t1.q_binary6)','(t1.q_binary7)','(t1.q_binary8)','(t1.q_binary_null)', + '(t2.q_binary)','(t2.q_binary1)','(t2.q_binary2)','(t2.q_binary3)','(t2.q_binary4)','(t2.q_binary5)','(t2.q_binary6)','(t2.q_binary7)','(t2.q_binary8)','(t2.q_binary_null)'] + + column_j_s1 = str(random.sample(fun_fix_column_s_j,i)).replace("[","").replace("]","").replace("'","") + fun_column_join_s_1 = str(random.sample(str_functions,1))+'('+'\"'+separator+'\",'+column_j_s1+')' + str_fun_join_s_1 = str(fun_column_join_s_1).replace("[","").replace("]","").replace("'","") + + column_j_s2 = str(random.sample(fun_fix_column_s_j,i)).replace("[","").replace("]","").replace("'","") + fun_column_join_s_2 = str(random.sample(str_functions,1))+'('+'\"'+separator+'\",'+column_j_s2+')' + str_fun_join_s_2 = str(fun_column_join_s_2).replace("[","").replace("]","").replace("'","") + + + tdSql.query("select /*+ para_tables_sort() */1-1 as str_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (strlist == ['LTRIM','RTRIM','LOWER','UPPER']) or (strlist == ['SUBSTR']) or (strlist == ['CONCAT']) or (strlist == ['CONCAT_WS']) : + sql = "select /*+ para_tables_sort() */ t1s , LTRIM(asct1), LOWER(asct1), RTRIM(asct2), UPPER(asct2) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1, " % str_fun_1 + sql += "%s as asct2, " % str_fun_2 + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts as t1s from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += ");" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + tdSql.checkRows(100) + self.cur1.execute(sql) + self.explain_sql(sql) + elif (strlist == ['LENGTH','CHAR_LENGTH']): + sql = "select /*+ para_tables_sort() */ sum(asct1), min(asct1), max(asct2), avg(asct2) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1, " % str_fun_1 + sql += "%s as asct2, " % str_fun_2 + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += ");" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-2 as str_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (strlist == ['LTRIM','RTRIM','LOWER','UPPER']) or (strlist == ['SUBSTR']) or (strlist == ['CONCAT']) or (strlist == ['CONCAT_WS']) : + sql = "select /*+ para_tables_sort() */ ts , asct1 from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1, " % str_fun_1 + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts ts from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s )" % random.choice(self.order_where) + sql += "%s " % random.choice(self.unionall_or_union) + sql += "select /*+ para_tables_sort() */ ts , asct2 from ( select /*+ para_tables_sort() */ " + sql += "%s as asct2, " % str_fun_2 + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts ts from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + #sql += "%s " % random.choice(having_support) + sql += "%s " % random.choice(self.order_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ")" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + elif (strlist == ['LENGTH','CHAR_LENGTH']): + sql = "select /*+ para_tables_sort() */ sum(asct1), min(asct1) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1, " % str_fun_1 + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts ts from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s )" % random.choice(self.order_where) + sql += "%s " % random.choice(self.unionall_or_union) + sql += "select /*+ para_tables_sort() */ sum(asct2), min(asct2) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct2, " % str_fun_2 + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts ts from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ")" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-3 as str_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (strlist == ['LTRIM','RTRIM','LOWER','UPPER']) or (strlist == ['SUBSTR']) or (strlist == ['CONCAT']) or (strlist == ['CONCAT_WS']): + sql = "select /*+ para_tables_sort() */ ts , LTRIM(asct1), LOWER(asct1), RTRIM(asct2), UPPER(asct2) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1 ," % str_fun_1 + sql += "%s as asct2, " % str_fun_2 + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s select /*+ para_tables_sort() */" % random.choice(self.unionall_or_union) + sql += "%s as asct2 ," % str_fun_2 + sql += "%s as asct1, " % str_fun_1 + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts from regular_table_2 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += ")" + tdLog.info(sql) + tdLog.info(len(sql)) + #tdSql.query(sql) #'unexpected end of data' + # self.cur1.execute(sql) + # self.explain_sql(sql) + elif (strlist == ['LENGTH','CHAR_LENGTH']): + sql = "select /*+ para_tables_sort() */ sum(asct1), min(asct1), max(asct2), avg(asct2) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1 ," % str_fun_1 + sql += "%s as asct2, " % str_fun_2 + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s select /*+ para_tables_sort() */" % random.choice(self.unionall_or_union) + sql += "%s as asct2 ," % str_fun_2 + sql += "%s as asct1, " % str_fun_1 + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts from regular_table_2 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += ")" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-4 as str_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (strlist == ['LTRIM','RTRIM','LOWER','UPPER']) or (strlist == ['SUBSTR']) or (strlist == ['CONCAT']) or (strlist == ['CONCAT_WS']): + sql = "select /*+ para_tables_sort() */ ts1,ts2 ,timediff(ts1,ts2), LTRIM(asct1), LOWER(asct1), RTRIM(asct2), UPPER(asct2) from ( select /*+ para_tables_sort() */ t1.ts as ts1," + sql += "%s as asct2, " % str_fun_join_1 + sql += "%s as asct1, " % str_fun_join_2 + sql += "%s, " % str_fun_join_1 + sql += "t1.%s as asct11, " % random.choice(self.q_select) + sql += "t2.%s as asct12, " % random.choice(self.q_select) + sql += "t2.ts as ts2 from regular_table_1 t1 , regular_table_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_where) + sql += "and %s " % random.choice(self.q_u_or_where) + sql += "%s " % random.choice(self.order_u_where) + sql += ");" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + tdSql.checkRows(100) + self.cur1.execute(sql) + self.explain_sql(sql) + elif (strlist == ['LENGTH','CHAR_LENGTH']): + sql = "select /*+ para_tables_sort() */ sum(asct1), min(asct1), max(asct2), avg(asct2) from ( select /*+ para_tables_sort() */ t1.ts as ts1," + sql += "%s as asct2, " % str_fun_join_1 + sql += "%s as asct1, " % str_fun_join_2 + sql += "%s, " % str_fun_join_1 + sql += "t1.%s as asct21, " % random.choice(self.q_select) + sql += "t2.%s as asct22, " % random.choice(self.q_select) + sql += "t2.ts as ts2 from regular_table_1 t1 , regular_table_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_where) + sql += "and %s " % random.choice(self.q_u_or_where) + sql += "%s " % random.choice(self.order_u_where) + sql += ");" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-5 as str_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (strlist == ['LTRIM','RTRIM','LOWER','UPPER']) or (strlist == ['SUBSTR']) or (strlist == ['CONCAT']) or (strlist == ['CONCAT_WS']): + sql = "select /*+ para_tables_sort() */ ts ," + sql += "%s, " % str_fun_1 + sql += "%s as asct21, " % random.choice(self.q_select) + sql += "%s as asct22, " % random.choice(self.q_select) + sql += "%s " % str_fun_2 + sql += " from ( select /*+ para_tables_sort() */ * from regular_table_1 ) where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += " ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + tdSql.checkRows(100) + self.cur1.execute(sql) + self.explain_sql(sql) + elif (strlist == ['LENGTH','CHAR_LENGTH']): + sql = "select /*+ para_tables_sort() */ ts ," + sql += "%s, " % str_fun_1 + sql += "%s as asct22, " % random.choice(self.q_select) + sql += "%s as asct21, " % random.choice(self.q_select) + sql += "%s " % str_fun_2 + sql += " from ( select /*+ para_tables_sort() */ * from regular_table_1 ) where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += " ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + tdSql.checkRows(100) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-6 as str_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (strlist == ['LTRIM','RTRIM','LOWER','UPPER']) or (strlist == ['SUBSTR']) or (strlist == ['CONCAT']) or (strlist == ['CONCAT_WS']): + sql = "select /*+ para_tables_sort() */ ts1,ts ,timediff(ts1,ts), LTRIM(asct1), LOWER(asct1), RTRIM(asct2), UPPER(asct2) from ( select /*+ para_tables_sort() */ t1.ts as ts1," + sql += "%s as asct2, " % str_fun_join_1 + sql += "%s as asct1, " % str_fun_join_2 + sql += "t1.%s as asct22, " % random.choice(self.q_select) + sql += "t2.%s as asct21, " % random.choice(self.q_select) + sql += "%s, " % str_fun_join_1 + sql += "t2.ts from regular_table_1 t1 , regular_table_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_where) + sql += "and %s )" % random.choice(self.q_u_or_where) + sql += "%s " % random.choice(self.order_where) + sql += "%s " % random.choice(self.limit1_where) + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + elif (strlist == ['LENGTH','CHAR_LENGTH']): + sql = "select /*+ para_tables_sort() */ sum(asct1), min(asct1), max(asct2), avg(asct2) from ( select /*+ para_tables_sort() */ t1.ts as ts1," + sql += "%s as asct2, " % str_fun_join_1 + sql += "%s as asct1, " % str_fun_join_2 + sql += "t1.%s as asct22, " % random.choice(self.q_select) + sql += "t2.%s as asct21, " % random.choice(self.q_select) + sql += "%s, " % str_fun_join_1 + sql += "t2.ts as ts2 from regular_table_1 t1 , regular_table_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_where) + sql += "and %s )" % random.choice(self.q_u_or_where) + sql += "%s " % random.choice(self.limit1_where) + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-7 as str_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (strlist == ['LTRIM','RTRIM','LOWER','UPPER']) or (strlist == ['SUBSTR']) or (strlist == ['CONCAT']) or (strlist == ['CONCAT_WS']): + sql = "select /*+ para_tables_sort() */ t1s ,ts1, LTRIM(asct1), LOWER(asct1), RTRIM(asct2), UPPER(asct2) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1, ts as t1s," % str_fun_s_1 + sql += "%s as asct2, " % str_fun_s_2 + sql += "%s, " % random.choice(self.q_select) + sql += "%s, " % random.choice(self.t_select) + sql += "ts as ts1 from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += "%s " % random.choice(self.order_where) + sql += ") ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + tdSql.checkRows(300) + self.cur1.execute(sql) + self.explain_sql(sql) + elif (strlist == ['LENGTH','CHAR_LENGTH']): + sql = "select /*+ para_tables_sort() */ sum(asct1), min(asct1), max(asct2), avg(asct2) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1, ts as ts1," % str_fun_s_1 + sql += "%s as asct2, " % str_fun_s_2 + sql += "%s, " % random.choice(self.q_select) + sql += "%s, " % random.choice(self.t_select) + sql += "ts as t1s from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += "%s " % random.choice(self.order_where) + sql += ") ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-8 as str_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (strlist == ['LTRIM','RTRIM','LOWER','UPPER']) or (strlist == ['SUBSTR']) or (strlist == ['CONCAT']) or (strlist == ['CONCAT_WS']): + sql = "select /*+ para_tables_sort() */ ts1,st1, LTRIM(asct1), LOWER(asct1), RTRIM(asct2), UPPER(asct2) " + sql += "from ( select /*+ para_tables_sort() */ " + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s as asct1, ts as st1," % str_fun_s_1 + sql += "%s as asct2, " % str_fun_s_2 + sql += "%s, " % random.choice(self.q_select) + sql += "%s, " % random.choice(self.t_select) + sql += "ts as ts1 from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += "%s " % random.choice(self.order_where) + sql += ") ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + tdSql.checkRows(300) + self.cur1.execute(sql) + self.explain_sql(sql) + elif (strlist == ['LENGTH','CHAR_LENGTH']): + sql = "select /*+ para_tables_sort() */ sum(asct1), min(asct1), max(asct2), avg(asct2) " + sql += "from ( select /*+ para_tables_sort() */ " + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s as asct1, ts as ts1," % str_fun_s_1 + sql += "%s as asct2, " % str_fun_s_2 + sql += "%s, " % random.choice(self.q_select) + sql += "%s, " % random.choice(self.t_select) + sql += "ts as st1 from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += "%s " % random.choice(self.order_where) + sql += ") ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-9 as str_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (strlist == ['LTRIM','RTRIM','LOWER','UPPER']) or (strlist == ['SUBSTR']) or (strlist == ['CONCAT']) or (strlist == ['CONCAT_WS']): + sql = "select /*+ para_tables_sort() */ ts1,ts2 ,timediff(ts1,ts2), LTRIM(asct1), LOWER(asct1), RTRIM(asct2), UPPER(asct2) from ( select /*+ para_tables_sort() */ t1.ts as ts1," + sql += "%s as asct2, " % str_fun_join_s_1 + sql += "%s as asct1, " % str_fun_join_s_2 + sql += "t1.%s as asct21, " % random.choice(self.q_select) + sql += "t1.%s as asct22, " % random.choice(self.q_select) + sql += "t2.%s as asct23, " % random.choice(self.q_select) + sql += "t2.%s as asct24, " % random.choice(self.q_select) + sql += "t2.ts as ts2 from stable_1 t1 , stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.t_join_where) + sql += "and %s " % random.choice(self.t_u_where) + sql += "and %s " % random.choice(self.t_u_or_where) + sql += "%s " % random.choice(self.order_u_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ") ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + elif (strlist == ['LENGTH','CHAR_LENGTH']): + sql = "select /*+ para_tables_sort() */ sum(asct1), min(asct1), max(asct2), avg(asct2) from ( select /*+ para_tables_sort() */ t1.ts as ts1," + sql += "%s as asct2, " % str_fun_join_s_1 + sql += "%s as asct1, " % str_fun_join_s_2 + sql += "t1.%s as asct21, " % random.choice(self.q_select) + sql += "t1.%s as asct22, " % random.choice(self.q_select) + sql += "t2.%s as asct23, " % random.choice(self.q_select) + sql += "t2.%s as asct24, " % random.choice(self.q_select) + sql += "t2.ts as ts2 from stable_1 t1 , stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.t_join_where) + sql += "and %s " % random.choice(self.t_u_where) + sql += "and %s " % random.choice(self.t_u_or_where) + sql += "%s " % random.choice(self.order_u_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ") ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + self.restartDnodes() + tdSql.query("select /*+ para_tables_sort() */1-10 as str_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (strlist == ['LTRIM','RTRIM','LOWER','UPPER']) or (strlist == ['SUBSTR']) or (strlist == ['CONCAT']) or (strlist == ['CONCAT_WS']): + sql = "select /*+ para_tables_sort() */ ts , LTRIM(asct1), LOWER(asct1), RTRIM(asct2), UPPER(asct2) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1 ," % str_fun_s_1 + sql += "%s as asct2, " % str_fun_s_2 + sql += "%s, " % random.choice(self.s_r_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += ") %s " % random.choice(self.unionall_or_union) + sql += "select /*+ para_tables_sort() */ ts , LTRIM(asct1), LOWER(asct1), RTRIM(asct2), UPPER(asct2) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1 ," % str_fun_1 + sql += "%s as asct2, " % str_fun_2 + sql += "%s, " % random.choice(self.s_r_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts from stable_2 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ")" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + elif (strlist == ['LENGTH','CHAR_LENGTH']): + sql = "select /*+ para_tables_sort() */ sum(asct1), min(asct1), max(asct2), avg(asct2) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1 ," % str_fun_s_1 + sql += "%s as asct2, " % str_fun_s_2 + sql += "%s, " % random.choice(self.s_r_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += ") %s " % random.choice(self.unionall_or_union) + sql += "select /*+ para_tables_sort() */ sum(asct1), min(asct1), max(asct2), avg(asct2) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1 ," % str_fun_1 + sql += "%s as asct2, " % str_fun_2 + sql += "%s, " % random.choice(self.s_r_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts from stable_2 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ")" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + #3 inter union not support + tdSql.query("select /*+ para_tables_sort() */1-11 as str_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (strlist == ['LTRIM','RTRIM','LOWER','UPPER']) or (strlist == ['SUBSTR']) or (strlist == ['CONCAT']) or (strlist == ['CONCAT_WS']): + sql = "select /*+ para_tables_sort() */ ts , LTRIM(asct1), LOWER(asct1), RTRIM(asct2), UPPER(asct2) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1, ts ," % str_fun_s_1 + sql += "%s as asct2, " % str_fun_s_2 + sql += "%s, " % random.choice(self.s_r_select) + sql += "%s " % random.choice(self.q_select) + sql += " from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += " %s " % random.choice(self.unionall_or_union) + sql += " select /*+ para_tables_sort() */ " + sql += "%s as asct1, ts ," % str_fun_1 + sql += "%s as asct2, " % str_fun_2 + sql += "%s, " % random.choice(self.s_r_select) + sql += "%s " % random.choice(self.q_select) + sql += " from stable_2 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ")" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + elif (strlist == ['LENGTH','CHAR_LENGTH']): + sql = "select /*+ para_tables_sort() */sum(asct1), min(asct1), max(asct2), avg(asct2) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1 ," % str_fun_s_1 + sql += "%s as asct2, " % str_fun_s_2 + sql += "%s, " % random.choice(self.s_r_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += " %s " % random.choice(self.unionall_or_union) + sql += " select /*+ para_tables_sort() */ " + sql += "%s as asct1 ," % str_fun_1 + sql += "%s as asct2, " % str_fun_2 + sql += "%s, " % random.choice(self.s_r_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts from stable_2 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ")" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-12 as str_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (strlist == ['LTRIM','RTRIM','LOWER','UPPER']) or (strlist == ['SUBSTR']) or (strlist == ['CONCAT']) or (strlist == ['CONCAT_WS']): + sql = "select /*+ para_tables_sort() */ ts1,ts2 ,timediff(ts1,ts2), LTRIM(asct1), LOWER(asct1), RTRIM(asct2), UPPER(asct2) from ( select /*+ para_tables_sort() */ t1.ts as ts1," + sql += "%s as asct2, " % str_fun_join_s_1 + sql += "%s as asct1, " % str_fun_join_s_2 + sql += "t1.%s as asct21, " % random.choice(self.q_select) + sql += "t1.%s as asct22, " % random.choice(self.q_select) + sql += "t2.%s as asct23, " % random.choice(self.q_select) + sql += "t2.%s as asct24, " % random.choice(self.q_select) + sql += "t2.ts as ts2 from stable_1 t1 , stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.t_u_where) + sql += "and %s " % random.choice(self.t_u_or_where) + sql += "%s " % random.choice(self.order_u_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ");" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + elif (strlist == ['LENGTH','CHAR_LENGTH']): + sql = "select /*+ para_tables_sort() */ sum(asct1), min(asct1), max(asct2), avg(asct2) from ( select /*+ para_tables_sort() */ t1.ts as ts1," + sql += "%s as asct2, " % str_fun_join_s_1 + sql += "%s as asct1, " % str_fun_join_s_2 + sql += "t1.%s as asct21, " % random.choice(self.q_select) + sql += "t1.%s as asct22, " % random.choice(self.q_select) + sql += "t2.%s as asct23, " % random.choice(self.q_select) + sql += "t2.%s as asct24, " % random.choice(self.q_select) + sql += "t2.ts as ts2 from stable_1 t1 , stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.t_u_where) + sql += "and %s " % random.choice(self.t_u_or_where) + sql += "%s " % random.choice(self.order_u_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ");" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-13 as str_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (strlist == ['LTRIM','RTRIM','LOWER','UPPER']) or (strlist == ['SUBSTR']) or (strlist == ['CONCAT']) or (strlist == ['CONCAT_WS']): + sql = "select /*+ para_tables_sort() */ ts ," + sql += "%s as asct10, " % str_fun_1 + sql += "%s as asct1, " % random.choice(self.q_select) + sql += "%s as asct12, " % random.choice(self.q_select) + sql += "%s as asct13, " % str_fun_2 + sql += "%s as asct14 " % random.choice(self.t_select) + sql += " from ( select /*+ para_tables_sort() */ * from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += "%s " % random.choice(self.order_where) + sql += ") ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + tdSql.checkRows(300) + self.cur1.execute(sql) + self.explain_sql(sql) + elif (strlist == ['LENGTH','CHAR_LENGTH']): + sql = "select /*+ para_tables_sort() */ ts ," + sql += "%s as asct1, " % str_fun_1 + sql += "%s as asct11, " % random.choice(self.q_select) + sql += "%s as asct12, " % random.choice(self.q_select) + sql += "%s as asct13, " % str_fun_2 + sql += "%s as asct14 " % random.choice(self.t_select) + sql += " from ( select /*+ para_tables_sort() */ * from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += "%s " % random.choice(self.order_where) + sql += ") ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + tdSql.checkRows(300) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-14 as str_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (strlist == ['LTRIM','RTRIM','LOWER','UPPER']) or (strlist == ['SUBSTR']) or (strlist == ['CONCAT']) or (strlist == ['CONCAT_WS']): + sql = "select /*+ para_tables_sort() */ LTRIM(asct1), LOWER(asct1), RTRIM(asct2), UPPER(asct2) from ( select /*+ para_tables_sort() */" + sql += "%s as asct1, " % str_fun_s_1 + sql += "%s as asct2" % str_fun_s_2 + sql += " from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.partiton_where) + sql += "%s " % random.choice(self.order_desc_where) + sql += "%s " % random.choice([self.limit_where[2] , self.limit_where[3]] ) + sql += " ) ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + elif (strlist == ['LENGTH','CHAR_LENGTH']): + sql = "select /*+ para_tables_sort() */ sum(asct1), min(asct1), max(asct2), avg(asct2) from ( select /*+ para_tables_sort() */" + sql += "%s as asct1, " % str_fun_s_1 + sql += "%s as asct2" % str_fun_s_2 + sql += " from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.partiton_where) + sql += "%s " % random.choice(self.order_desc_where) + sql += "%s " % random.choice([self.limit_where[2] , self.limit_where[3]] ) + sql += " ) ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-15 as str_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (strlist == ['LTRIM','RTRIM','LOWER','UPPER']) or (strlist == ['SUBSTR']) or (strlist == ['CONCAT']) or (strlist == ['CONCAT_WS']): + sql = "select /*+ para_tables_sort() */ ts,ts2 ,timediff(ts,ts2), LTRIM(asct1), LOWER(asct1), RTRIM(asct2), UPPER(asct2) from ( select /*+ para_tables_sort() */ t1.ts ," + sql += "%s as asct2, " % str_fun_join_s_1 + sql += "%s as asct1, " % str_fun_join_s_2 + sql += "t1.%s as asct11, " % random.choice(self.q_select) + sql += "t1.%s as asct12, " % random.choice(self.q_select) + sql += "t2.%s as asct13, " % random.choice(self.q_select) + sql += "t2.%s as asct14, " % random.choice(self.q_select) + sql += "t2.ts as ts2 from stable_1 t1, stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.t_join_where) + sql += " and %s " % random.choice(self.qt_u_or_where) + sql += "%s " % random.choice(self.partiton_where_j) + sql += "%s " % random.choice(self.slimit1_where) + sql += ") " + sql += "%s " % random.choice(self.order_desc_where) + sql += "%s ;" % random.choice(self.limit_u_where) + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + elif (strlist == ['LENGTH','CHAR_LENGTH']): + sql = "select /*+ para_tables_sort() */ sum(asct1), min(asct1), max(asct2), avg(asct2) from ( select /*+ para_tables_sort() */ t1.ts as ts1," + sql += "%s as asct2, " % str_fun_join_s_1 + sql += "%s as asct1, " % str_fun_join_s_2 + sql += "t1.%s as asct11, " % random.choice(self.q_select) + sql += "t1.%s as asct12, " % random.choice(self.q_select) + sql += "t2.%s as asct13, " % random.choice(self.q_select) + sql += "t2.%s as asct14 " % random.choice(self.q_select) + sql += "from stable_1 t1, stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.t_join_where) + sql += " and %s " % random.choice(self.qt_u_or_where) + sql += "%s " % random.choice(self.partiton_where_j) + sql += "%s " % random.choice(self.slimit1_where) + sql += ") " + sql += "%s ;" % random.choice(self.limit_u_where) + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + #taos -f sql + startTime_taos_f = time.time() + print("taos -f %s sql start!" %strlist) + taos_cmd1 = "taos -f %s/%s.sql" % (self.testcasePath,self.testcaseFilename) + #_ = subprocess.check_output(taos_cmd1, shell=True).decode("utf-8") + _ = subprocess.check_output(taos_cmd1, shell=True) + print("taos -f %s sql over!" %strlist) + endTime_taos_f = time.time() + print("taos_f total time %ds" % (endTime_taos_f - startTime_taos_f)) + + print("=========%s====over=============" %strlist) + + def time_nest(self,timelist): + + print("==========%s===start=============" %timelist) + os.system("rm -rf %s/%s.sql" % (self.testcasePath,self.testcaseFilename)) + + self.dropandcreateDB_random("%s" %self.db_nest, 1) + + if (timelist == ['NOW','TODAY']) or (timelist == ['TIMEZONE']): + time_functions = timelist + fun_fix_column = ['()'] + fun_column_1 = random.sample(time_functions,1)+random.sample(fun_fix_column,1) + time_fun_1 = str(fun_column_1).replace("[","").replace("]","").replace("'","").replace(", ","") + fun_column_2 = random.sample(time_functions,1)+random.sample(fun_fix_column,1) + time_fun_2 = str(fun_column_2).replace("[","").replace("]","").replace("'","").replace(", ","") + + fun_fix_column_j = ['()'] + fun_column_join_1 = random.sample(time_functions,1)+random.sample(fun_fix_column_j,1) + time_fun_join_1 = str(fun_column_join_1).replace("[","").replace("]","").replace("'","").replace(", ","") + fun_column_join_2 = random.sample(time_functions,1)+random.sample(fun_fix_column_j,1) + time_fun_join_2 = str(fun_column_join_2).replace("[","").replace("]","").replace("'","").replace(", ","") + + elif (timelist == ['TIMETRUNCATE']): + time_functions = timelist + + t = time.time() + t_to_s = time.strftime('%Y-%m-%d %H:%M:%S', time.localtime(t)) + fun_fix_column = ['q_ts','ts','_c0','_C0','_rowts','1600000000000','1600000000000000','1600000000000000000', + '%d' %t, '%d000' %t, '%d000000' %t,'t_to_s'] + + timeunits = ['1a' ,'1s', '1m' ,'1h', '1d'] + timeunit = str(random.sample(timeunits,1)).replace("[","").replace("]","").replace("'","") + + column_1 = ['(%s,timeutil)'%(random.sample(fun_fix_column,1))] + fun_column_1 = random.sample(time_functions,1)+random.sample(column_1,1) + time_fun_1 = str(fun_column_1).replace("[","").replace("]","").replace("'","").replace(", ","").replace("\"","").replace("t_to_s","'t_to_s'") + time_fun_1 = str(time_fun_1).replace("timeutil","%s" %timeunit).replace("t_to_s","%s" %t_to_s) + + column_2 = ['(%s,timeutil)'%(random.sample(fun_fix_column,1))] + fun_column_2 = random.sample(time_functions,1)+random.sample(column_2,1) + time_fun_2 = str(fun_column_2).replace("[","").replace("]","").replace("'","").replace(", ","").replace("\"","").replace("t_to_s","'t_to_s'") + time_fun_2 = str(time_fun_2).replace("timeutil","%s" %timeunit).replace("t_to_s","%s" %t_to_s) + + + fun_fix_column_j = ['(t1.q_ts)','(t1.ts)', '(t2.q_ts)','(t2.ts)','(1600000000000)','(1600000000000000)','(1600000000000000000)', + '(%d)' %t, '(%d000)' %t, '(%d000000)' %t,'t_to_s'] + + column_j1 = ['(%s,timeutil)'%(random.sample(fun_fix_column_j,1))] + fun_column_join_1 = random.sample(time_functions,1)+random.sample(column_j1,1) + time_fun_join_1 = str(fun_column_join_1).replace("[","").replace("]","").replace("'","").replace(", ","").replace("\"","").replace("t_to_s","'t_to_s'") + time_fun_join_1 = str(time_fun_join_1).replace("timeutil","%s" %timeunit).replace("t_to_s","%s" %t_to_s) + + column_j2 = ['(%s,timeutil)'%(random.sample(fun_fix_column_j,1))] + fun_column_join_2 = random.sample(time_functions,1)+random.sample(column_j2,1) + time_fun_join_2 = str(fun_column_join_2).replace("[","").replace("]","").replace("'","").replace(", ","").replace("\"","").replace("t_to_s","'t_to_s'") + time_fun_join_2 = str(time_fun_join_2).replace("timeutil","%s" %timeunit).replace("t_to_s","%s" %t_to_s) + + elif (timelist == ['TO_ISO8601']): + time_functions = timelist + + t = time.time() + fun_fix_column = ['(now())','(ts)','(q_ts)','(_rowts)','(_c0)','(_C0)', + '(1600000000000)','(1600000000000000)','(1600000000000000000)', + '(%d)' %t, '(%d000)' %t, '(%d000000)' %t] + + fun_column_1 = random.sample(time_functions,1)+random.sample(fun_fix_column,1) + time_fun_1 = str(fun_column_1).replace("[","").replace("]","").replace("'","").replace(", ","") + + fun_column_2 = random.sample(time_functions,1)+random.sample(fun_fix_column,1) + time_fun_2 = str(fun_column_2).replace("[","").replace("]","").replace("'","").replace(", ","") + + fun_fix_column_j = ['(t1.q_ts)','(t1.ts)', '(t2.q_ts)','(t2.ts)','(1600000000000)','(1600000000000000)','(1600000000000000000)','(now())', + '(%d)' %t, '(%d000)' %t, '(%d000000)' %t] + + fun_column_join_1 = random.sample(time_functions,1)+random.sample(fun_fix_column_j,1) + time_fun_join_1 = str(fun_column_join_1).replace("[","").replace("]","").replace("'","").replace(", ","") + + fun_column_join_2 = random.sample(time_functions,1)+random.sample(fun_fix_column_j,1) + time_fun_join_2 = str(fun_column_join_2).replace("[","").replace("]","").replace("'","").replace(", ","") + + elif (timelist == ['TO_UNIXTIMESTAMP']): + time_functions = timelist + + t = time.time() + t_to_s = time.strftime('%Y-%m-%d %H:%M:%S', time.localtime(t)) + fun_fix_column = ['(q_nchar)','(q_nchar1)','(q_nchar2)','(q_nchar3)','(q_nchar4)','(q_nchar_null)','(q_binary)','(q_binary5)','(q_binary6)','(q_binary7)','(q_binary8)','(q_binary_null)','(t_to_s)'] + + fun_column_1 = random.sample(time_functions,1)+random.sample(fun_fix_column,1) + time_fun_1 = str(fun_column_1).replace("[","").replace("]","").replace("'","").replace(", ","").replace("t_to_s","'t_to_s'") + time_fun_1 = str(time_fun_1).replace("t_to_s","%s" %t_to_s) + + fun_column_2 = random.sample(time_functions,1)+random.sample(fun_fix_column,1) + time_fun_2 = str(fun_column_2).replace("[","").replace("]","").replace("'","").replace(", ","").replace("t_to_s","'t_to_s'") + time_fun_2 = str(time_fun_2).replace("t_to_s","%s" %t_to_s) + + fun_fix_column_j = ['(t1.q_nchar)','(t1.q_binary)', '(t2.q_nchar)','(t2.q_binary)','(t_to_s)'] + + fun_column_join_1 = random.sample(time_functions,1)+random.sample(fun_fix_column_j,1) + time_fun_join_1 = str(fun_column_join_1).replace("[","").replace("]","").replace("'","").replace(", ","").replace("t_to_s","'t_to_s'") + time_fun_join_1 = str(time_fun_join_1).replace("t_to_s","%s" %t_to_s) + + fun_column_join_2 = random.sample(time_functions,1)+random.sample(fun_fix_column_j,1) + time_fun_join_2 = str(fun_column_join_2).replace("[","").replace("]","").replace("'","").replace(", ","").replace("t_to_s","'t_to_s'") + time_fun_join_2 = str(time_fun_join_2).replace("t_to_s","%s" %t_to_s) + + elif (timelist == ['TIMEDIFF_1']): + time_functions = timelist + + t = time.time() + t_to_s = time.strftime('%Y-%m-%d %H:%M:%S', time.localtime(t)) + timeunits = [ '1a' ,'1s', '1m' ,'1h', '1d'] + timeunit = str(random.sample(timeunits,1)).replace("[","").replace("]","").replace("'","") + + fun_fix_column = ['q_ts','ts','_c0','_C0','_rowts','1600000000000','1600000000000000','1600000000000000000', + '%d' %t, '%d000' %t, '%d000000' %t,'t_to_s'] + + column_1,column_2 = random.sample(fun_fix_column,1),random.sample(fun_fix_column,1) + column_12 = ['(%s,%s,timeutil)'%(column_1,column_2)] + fun_column_1 = random.sample(time_functions,1)+random.sample(column_12,1) + time_fun_1 = str(fun_column_1).replace("[","").replace("]","").replace("'","").replace(", ","").replace("\"","").replace("t_to_s","'t_to_s'") + time_fun_1 = str(time_fun_1).replace("timeutil","%s" %timeunit).replace("t_to_s","%s" %t_to_s).replace("_1","") + + column_3,column_4 = random.sample(fun_fix_column,1),random.sample(fun_fix_column,1) + column_34 = ['(%s,%s,timeutil)'%(column_3,column_4)] + fun_column_2 = random.sample(time_functions,1)+random.sample(column_34,1) + time_fun_2 = str(fun_column_2).replace("[","").replace("]","").replace("'","").replace(", ","").replace("\"","").replace("t_to_s","'t_to_s'") + time_fun_2 = str(time_fun_2).replace("timeutil","%s" %timeunit).replace("t_to_s","%s" %t_to_s).replace("_1","") + + fun_fix_column_j = ['(t1.q_ts)','(t1.ts)', '(t2.q_ts)','(t2.ts)','1600000000000','1600000000000000','1600000000000000000', + '%d' %t, '%d000' %t, '%d000000' %t,'t_to_s'] + + column_j1,column_j2 = random.sample(fun_fix_column_j,1),random.sample(fun_fix_column_j,1) + column_j12 = ['(%s,%s,timeutil)'%(column_j1,column_j2)] + fun_column_join_1 = random.sample(time_functions,1)+random.sample(column_j12,1) + time_fun_join_1 = str(fun_column_join_1).replace("[","").replace("]","").replace("'","").replace(", ","").replace("\"","").replace("t_to_s","'t_to_s'") + time_fun_join_1 = str(time_fun_join_1).replace("timeutil","%s" %timeunit).replace("t_to_s","%s" %t_to_s).replace("_1","") + + column_j3,column_j4 = random.sample(fun_fix_column_j,1),random.sample(fun_fix_column_j,1) + column_j34 = ['(%s,%s,timeutil)'%(column_j3,column_j4)] + fun_column_join_2 = random.sample(time_functions,1)+random.sample(column_j34,1) + time_fun_join_2 = str(fun_column_join_2).replace("[","").replace("]","").replace("'","").replace(", ","").replace("\"","").replace("t_to_s","'t_to_s'") + time_fun_join_2 = str(time_fun_join_2).replace("timeutil","%s" %timeunit).replace("t_to_s","%s" %t_to_s).replace("_1","") + + elif (timelist == ['TIMEDIFF_2']): + time_functions = timelist + + t = time.time() + t_to_s = time.strftime('%Y-%m-%d %H:%M:%S', time.localtime(t)) + + fun_fix_column = ['q_ts','ts','_c0','_C0','_rowts','1600000000000','1600000000000000','1600000000000000000', + '%d' %t, '%d000' %t, '%d000000' %t,'t_to_s'] + + column_1,column_2 = random.sample(fun_fix_column,1),random.sample(fun_fix_column,1) + column_12 = ['(%s,%s)'%(column_1,column_2)] + fun_column_1 = random.sample(time_functions,1)+random.sample(column_12,1) + time_fun_1 = str(fun_column_1).replace("[","").replace("]","").replace("'","").replace(", ","").replace("\"","").replace("t_to_s","'t_to_s'") + time_fun_1 = str(time_fun_1).replace("t_to_s","%s" %t_to_s).replace("_2","") + + column_3,column_4 = random.sample(fun_fix_column,1),random.sample(fun_fix_column,1) + column_34 = ['(%s,%s)'%(column_3,column_4)] + fun_column_2 = random.sample(time_functions,1)+random.sample(column_34,1) + time_fun_2 = str(fun_column_2).replace("[","").replace("]","").replace("'","").replace(", ","").replace("\"","").replace("t_to_s","'t_to_s'") + time_fun_2 = str(time_fun_2).replace("t_to_s","%s" %t_to_s).replace("_2","") + + fun_fix_column_j = ['(t1.q_ts)','(t1.ts)', '(t2.q_ts)','(t2.ts)','1600000000000','1600000000000000','1600000000000000000', + '%d' %t, '%d000' %t, '%d000000' %t,'t_to_s'] + + column_j1,column_j2 = random.sample(fun_fix_column_j,1),random.sample(fun_fix_column_j,1) + column_j12 = ['(%s,%s)'%(column_j1,column_j2)] + fun_column_join_1 = random.sample(time_functions,1)+random.sample(column_j12,1) + time_fun_join_1 = str(fun_column_join_1).replace("[","").replace("]","").replace("'","").replace(", ","").replace("\"","").replace("t_to_s","'t_to_s'") + time_fun_join_1 = str(time_fun_join_1).replace("t_to_s","%s" %t_to_s).replace("_2","") + + column_j3,column_j4 = random.sample(fun_fix_column_j,1),random.sample(fun_fix_column_j,1) + column_j34 = ['(%s,%s)'%(column_j3,column_j4)] + fun_column_join_2 = random.sample(time_functions,1)+random.sample(column_j34,1) + time_fun_join_2 = str(fun_column_join_2).replace("[","").replace("]","").replace("'","").replace(", ","").replace("\"","").replace("t_to_s","'t_to_s'") + time_fun_join_2 = str(time_fun_join_2).replace("t_to_s","%s" %t_to_s).replace("_2","") + + elif (timelist == ['ELAPSED']): + time_functions = timelist + + fun_fix_column = ['(ts)','(_c0)','(_C0)','(_rowts)','(ts,time_unit)','(_c0,time_unit)','(_C0,time_unit)','(_rowts,time_unit)'] + + time_units = ['1s','1m','1h','1d','1a'] + time_unit1 = str(random.sample(time_units,1)).replace("[","").replace("]","").replace("'","") + time_unit2 = str(random.sample(time_units,1)).replace("[","").replace("]","").replace("'","") + + fun_column_1 = random.sample(time_functions,1)+random.sample(fun_fix_column,1) + time_fun_1 = str(fun_column_1).replace("[","").replace("]","").replace("'","").replace(", ","").replace("time_unit","%s" %time_unit1) + + fun_column_2 = random.sample(time_functions,1)+random.sample(fun_fix_column,1) + time_fun_2 = str(fun_column_2).replace("[","").replace("]","").replace("'","").replace(", ","").replace("time_unit","%s" %time_unit2) + + + fun_fix_column_j = ['(t1.ts)', '(t2.ts)','(t1.ts,time_unit)','(t1.ts,time_unit)','(t2.ts,time_unit)','(t2.ts,time_unit)'] + + fun_column_join_1 = random.sample(time_functions,1)+random.sample(fun_fix_column_j,1) + time_fun_join_1 = str(fun_column_join_1).replace("[","").replace("]","").replace("'","").replace(", ","").replace("time_unit","%s" %time_unit1) + + fun_column_join_2 = random.sample(time_functions,1)+random.sample(fun_fix_column_j,1) + time_fun_join_2 = str(fun_column_join_2).replace("[","").replace("]","").replace("'","").replace(", ","").replace("time_unit","%s" %time_unit2) + + + elif (timelist == ['CAST']) : + str_functions = timelist + #下面的4个是全的,这个只是1个 + i = random.randint(1,4) + if i ==1: + print('===========cast_1===========') + fun_fix_column = ['q_bool','q_bool_null','q_bigint','q_bigint_null','q_smallint','q_smallint_null', + 'q_tinyint','q_tinyint_null','q_int','q_int_null','q_float','q_float_null','q_double','q_double_null'] + type_names = ['BIGINT','BINARY(100)','TIMESTAMP','NCHAR(100)','BIGINT UNSIGNED'] + + type_name1 = str(random.sample(type_names,1)).replace("[","").replace("]","").replace("'","") + fun_column_1 = str(random.sample(str_functions,1))+'('+str(random.sample(fun_fix_column,1))+' AS '+type_name1+')' + time_fun_1 = str(fun_column_1).replace("[","").replace("]","").replace("'","") + + type_name2 = str(random.sample(type_names,1)).replace("[","").replace("]","").replace("'","") + fun_column_2 = str(random.sample(str_functions,1))+'('+str(random.sample(fun_fix_column,1))+' AS '+type_name2+')' + time_fun_2 = str(fun_column_2).replace("[","").replace("]","").replace("'","") + + fun_fix_column_j = ['t1.q_bool','t1.q_bool_null','t1.q_bigint','t1.q_bigint_null','t1.q_smallint','t1.q_smallint_null', + 't1.q_tinyint','t1.q_tinyint_null','t1.q_int','t1.q_int_null','t1.q_float','t1.q_float_null','t1.q_double','t1.q_double_null', + 't2.q_bool','t2.q_bool_null','t2.q_bigint','t2.q_bigint_null','t2.q_smallint','t2.q_smallint_null', + 't2.q_tinyint','t2.q_tinyint_null','t2.q_int','t2.q_int_null','t2.q_float','t2.q_float_null','t2.q_double','t2.q_double_null'] + + type_name_j1 = str(random.sample(type_names,1)).replace("[","").replace("]","").replace("'","") + fun_column_join_1 = str(random.sample(str_functions,1))+'('+str(random.sample(fun_fix_column_j,1))+' AS '+type_name_j1+')' + time_fun_join_1 = str(fun_column_join_1).replace("[","").replace("]","").replace("'","") + + type_name_j2 = str(random.sample(type_names,1)).replace("[","").replace("]","").replace("'","") + fun_column_join_2 = str(random.sample(str_functions,1))+'('+str(random.sample(fun_fix_column_j,1))+' AS '+type_name_j2+')' + time_fun_join_2 = str(fun_column_join_2).replace("[","").replace("]","").replace("'","") + + elif i==2: + print('===========cast_2===========') + fun_fix_column = ['q_binary','q_binary_null','q_binary1','q_binary2','q_binary3','q_binary4'] + type_names = ['BIGINT','BINARY(100)','NCHAR(100)','BIGINT UNSIGNED'] + + type_name1 = str(random.sample(type_names,1)).replace("[","").replace("]","").replace("'","") + fun_column_1 = str(random.sample(str_functions,1))+'('+str(random.sample(fun_fix_column,1))+' AS '+type_name1+')' + time_fun_1 = str(fun_column_1).replace("[","").replace("]","").replace("'","") + + type_name2 = str(random.sample(type_names,1)).replace("[","").replace("]","").replace("'","") + fun_column_2 = str(random.sample(str_functions,1))+'('+str(random.sample(fun_fix_column,1))+' AS '+type_name2+')' + time_fun_2 = str(fun_column_2).replace("[","").replace("]","").replace("'","") + + fun_fix_column_j = ['t1.q_binary','t1.q_binary_null','t1.q_binary1','t1.q_binary2','t1.q_binary3','t1.q_binary4', + 't2.q_binary','t2.q_binary_null','t2.q_binary1','t2.q_binary2','t2.q_binary3','t2.q_binary4'] + + type_name_j1 = str(random.sample(type_names,1)).replace("[","").replace("]","").replace("'","") + fun_column_join_1 = str(random.sample(str_functions,1))+'('+str(random.sample(fun_fix_column_j,1))+' AS '+type_name_j1+')' + time_fun_join_1 = str(fun_column_join_1).replace("[","").replace("]","").replace("'","") + + type_name_j2 = str(random.sample(type_names,1)).replace("[","").replace("]","").replace("'","") + fun_column_join_2 = str(random.sample(str_functions,1))+'('+str(random.sample(fun_fix_column_j,1))+' AS '+type_name_j2+')' + time_fun_join_2 = str(fun_column_join_2).replace("[","").replace("]","").replace("'","") + + elif i==3: + print('===========cast_3===========') + fun_fix_column = ['q_nchar','q_nchar_null','q_nchar5','q_nchar6','q_nchar7','q_nchar8'] + type_names = ['BIGINT','NCHAR(100)','BIGINT UNSIGNED'] + + type_name1 = str(random.sample(type_names,1)).replace("[","").replace("]","").replace("'","") + fun_column_1 = str(random.sample(str_functions,1))+'('+str(random.sample(fun_fix_column,1))+' AS '+type_name1+')' + time_fun_1 = str(fun_column_1).replace("[","").replace("]","").replace("'","") + + type_name2 = str(random.sample(type_names,1)).replace("[","").replace("]","").replace("'","") + fun_column_2 = str(random.sample(str_functions,1))+'('+str(random.sample(fun_fix_column,1))+' AS '+type_name2+')' + time_fun_2 = str(fun_column_2).replace("[","").replace("]","").replace("'","") + + fun_fix_column_j = ['t1.q_nchar','t1.q_nchar_null','t1.q_nchar5','t1.q_nchar6','t1.q_nchar7','t1.q_nchar8', + 't2.q_nchar','t2.q_nchar_null','t2.q_nchar5','t2.q_nchar6','t2.q_nchar7','t2.q_nchar8'] + + type_name_j1 = str(random.sample(type_names,1)).replace("[","").replace("]","").replace("'","") + fun_column_join_1 = str(random.sample(str_functions,1))+'('+str(random.sample(fun_fix_column_j,1))+' AS '+type_name_j1+')' + time_fun_join_1 = str(fun_column_join_1).replace("[","").replace("]","").replace("'","") + + type_name_j2 = str(random.sample(type_names,1)).replace("[","").replace("]","").replace("'","") + fun_column_join_2 = str(random.sample(str_functions,1))+'('+str(random.sample(fun_fix_column_j,1))+' AS '+type_name_j2+')' + time_fun_join_2 = str(fun_column_join_2).replace("[","").replace("]","").replace("'","") + + elif i==4: + print('===========cast_4===========') + fun_fix_column = ['q_ts','q_ts_null','_C0','_c0','ts','_rowts'] + type_names = ['BIGINT','TIMESTAMP','BIGINT UNSIGNED'] + + type_name1 = str(random.sample(type_names,1)).replace("[","").replace("]","").replace("'","") + fun_column_1 = str(random.sample(str_functions,1))+'('+str(random.sample(fun_fix_column,1))+' AS '+type_name1+')' + time_fun_1 = str(fun_column_1).replace("[","").replace("]","").replace("'","") + + type_name2 = str(random.sample(type_names,1)).replace("[","").replace("]","").replace("'","") + fun_column_2 = str(random.sample(str_functions,1))+'('+str(random.sample(fun_fix_column,1))+' AS '+type_name2+')' + time_fun_2 = str(fun_column_2).replace("[","").replace("]","").replace("'","") + + fun_fix_column_j = ['t1.q_ts','t1.q_ts_null','t1.ts','t2.q_ts','t2.q_ts_null','t2.ts'] + + type_name_j1 = str(random.sample(type_names,1)).replace("[","").replace("]","").replace("'","") + fun_column_join_1 = str(random.sample(str_functions,1))+'('+str(random.sample(fun_fix_column_j,1))+' AS '+type_name_j1+')' + time_fun_join_1 = str(fun_column_join_1).replace("[","").replace("]","").replace("'","") + + type_name_j2 = str(random.sample(type_names,1)).replace("[","").replace("]","").replace("'","") + fun_column_join_2 = str(random.sample(str_functions,1))+'('+str(random.sample(fun_fix_column_j,1))+' AS '+type_name_j2+')' + time_fun_join_2 = str(fun_column_join_2).replace("[","").replace("]","").replace("'","") + + elif (timelist == ['CAST_1']) : + str_functions = timelist + + print('===========cast_1===========') + fun_fix_column = ['q_bool','q_bool_null','q_bigint','q_bigint_null','q_smallint','q_smallint_null', + 'q_tinyint','q_tinyint_null','q_int','q_int_null','q_float','q_float_null','q_double','q_double_null'] + type_names = ['BIGINT','BINARY(100)','TIMESTAMP','NCHAR(100)','BIGINT UNSIGNED'] + + type_name1 = str(random.sample(type_names,1)).replace("[","").replace("]","").replace("'","") + fun_column_1 = str(random.sample(str_functions,1))+'('+str(random.sample(fun_fix_column,1))+' AS '+type_name1+')' + time_fun_1 = str(fun_column_1).replace("[","").replace("]","").replace("'","").replace("_1","") + + type_name2 = str(random.sample(type_names,1)).replace("[","").replace("]","").replace("'","") + fun_column_2 = str(random.sample(str_functions,1))+'('+str(random.sample(fun_fix_column,1))+' AS '+type_name2+')' + time_fun_2 = str(fun_column_2).replace("[","").replace("]","").replace("'","").replace("_1","") + + fun_fix_column_j = ['t1.q_bool','t1.q_bool_null','t1.q_bigint','t1.q_bigint_null','t1.q_smallint','t1.q_smallint_null', + 't1.q_tinyint','t1.q_tinyint_null','t1.q_int','t1.q_int_null','t1.q_float','t1.q_float_null','t1.q_double','t1.q_double_null', + 't2.q_bool','t2.q_bool_null','t2.q_bigint','t2.q_bigint_null','t2.q_smallint','t2.q_smallint_null', + 't2.q_tinyint','t2.q_tinyint_null','t2.q_int','t2.q_int_null','t2.q_float','t2.q_float_null','t2.q_double','t2.q_double_null'] + + type_name_j1 = str(random.sample(type_names,1)).replace("[","").replace("]","").replace("'","") + fun_column_join_1 = str(random.sample(str_functions,1))+'('+str(random.sample(fun_fix_column_j,1))+' AS '+type_name_j1+')' + time_fun_join_1 = str(fun_column_join_1).replace("[","").replace("]","").replace("'","").replace("_1","") + + type_name_j2 = str(random.sample(type_names,1)).replace("[","").replace("]","").replace("'","") + fun_column_join_2 = str(random.sample(str_functions,1))+'('+str(random.sample(fun_fix_column_j,1))+' AS '+type_name_j2+')' + time_fun_join_2 = str(fun_column_join_2).replace("[","").replace("]","").replace("'","").replace("_1","") + + elif (timelist == ['CAST_2']) : + str_functions = timelist + print('===========cast_2===========') + fun_fix_column = ['q_binary','q_binary_null','q_binary1','q_binary2','q_binary3','q_binary4'] + type_names = ['BIGINT','BINARY(100)','NCHAR(100)','BIGINT UNSIGNED'] + + type_name1 = str(random.sample(type_names,1)).replace("[","").replace("]","").replace("'","") + fun_column_1 = str(random.sample(str_functions,1))+'('+str(random.sample(fun_fix_column,1))+' AS '+type_name1+')' + time_fun_1 = str(fun_column_1).replace("[","").replace("]","").replace("'","").replace("_2","") + + type_name2 = str(random.sample(type_names,1)).replace("[","").replace("]","").replace("'","") + fun_column_2 = str(random.sample(str_functions,1))+'('+str(random.sample(fun_fix_column,1))+' AS '+type_name2+')' + time_fun_2 = str(fun_column_2).replace("[","").replace("]","").replace("'","").replace("_2","") + + fun_fix_column_j = ['t1.q_binary','t1.q_binary_null','t1.q_binary1','t1.q_binary2','t1.q_binary3','t1.q_binary4', + 't2.q_binary','t2.q_binary_null','t2.q_binary1','t2.q_binary2','t2.q_binary3','t2.q_binary4'] + + type_name_j1 = str(random.sample(type_names,1)).replace("[","").replace("]","").replace("'","") + fun_column_join_1 = str(random.sample(str_functions,1))+'('+str(random.sample(fun_fix_column_j,1))+' AS '+type_name_j1+')' + time_fun_join_1 = str(fun_column_join_1).replace("[","").replace("]","").replace("'","").replace("_2","") + + type_name_j2 = str(random.sample(type_names,1)).replace("[","").replace("]","").replace("'","") + fun_column_join_2 = str(random.sample(str_functions,1))+'('+str(random.sample(fun_fix_column_j,1))+' AS '+type_name_j2+')' + time_fun_join_2 = str(fun_column_join_2).replace("[","").replace("]","").replace("'","").replace("_2","") + + elif (timelist == ['CAST_3']) : + str_functions = timelist + print('===========cast_3===========') + fun_fix_column = ['q_nchar','q_nchar_null','q_nchar5','q_nchar6','q_nchar7','q_nchar8'] + type_names = ['BIGINT','NCHAR(100)','BIGINT UNSIGNED'] + + type_name1 = str(random.sample(type_names,1)).replace("[","").replace("]","").replace("'","") + fun_column_1 = str(random.sample(str_functions,1))+'('+str(random.sample(fun_fix_column,1))+' AS '+type_name1+')' + time_fun_1 = str(fun_column_1).replace("[","").replace("]","").replace("'","").replace("_3","") + + type_name2 = str(random.sample(type_names,1)).replace("[","").replace("]","").replace("'","") + fun_column_2 = str(random.sample(str_functions,1))+'('+str(random.sample(fun_fix_column,1))+' AS '+type_name2+')' + time_fun_2 = str(fun_column_2).replace("[","").replace("]","").replace("'","").replace("_3","") + + fun_fix_column_j = ['t1.q_nchar','t1.q_nchar_null','t1.q_nchar5','t1.q_nchar6','t1.q_nchar7','t1.q_nchar8', + 't2.q_nchar','t2.q_nchar_null','t2.q_nchar5','t2.q_nchar6','t2.q_nchar7','t2.q_nchar8'] + + type_name_j1 = str(random.sample(type_names,1)).replace("[","").replace("]","").replace("'","") + fun_column_join_1 = str(random.sample(str_functions,1))+'('+str(random.sample(fun_fix_column_j,1))+' AS '+type_name_j1+')' + time_fun_join_1 = str(fun_column_join_1).replace("[","").replace("]","").replace("'","").replace("_3","") + + type_name_j2 = str(random.sample(type_names,1)).replace("[","").replace("]","").replace("'","") + fun_column_join_2 = str(random.sample(str_functions,1))+'('+str(random.sample(fun_fix_column_j,1))+' AS '+type_name_j2+')' + time_fun_join_2 = str(fun_column_join_2).replace("[","").replace("]","").replace("'","").replace("_3","") + + elif (timelist == ['CAST_4']) : + str_functions = timelist + print('===========cast_4===========') + fun_fix_column = ['q_ts','q_ts_null','_C0','_c0','ts','_rowts'] + type_names = ['BIGINT','TIMESTAMP','BIGINT UNSIGNED'] + + type_name1 = str(random.sample(type_names,1)).replace("[","").replace("]","").replace("'","") + fun_column_1 = str(random.sample(str_functions,1))+'('+str(random.sample(fun_fix_column,1))+' AS '+type_name1+')' + time_fun_1 = str(fun_column_1).replace("[","").replace("]","").replace("'","").replace("_4","") + + type_name2 = str(random.sample(type_names,1)).replace("[","").replace("]","").replace("'","") + fun_column_2 = str(random.sample(str_functions,1))+'('+str(random.sample(fun_fix_column,1))+' AS '+type_name2+')' + time_fun_2 = str(fun_column_2).replace("[","").replace("]","").replace("'","").replace("_4","") + + fun_fix_column_j = ['t1.q_ts','t1.q_ts_null','t1.ts','t2.q_ts','t2.q_ts_null','t2.ts'] + + type_name_j1 = str(random.sample(type_names,1)).replace("[","").replace("]","").replace("'","") + fun_column_join_1 = str(random.sample(str_functions,1))+'('+str(random.sample(fun_fix_column_j,1))+' AS '+type_name_j1+')' + time_fun_join_1 = str(fun_column_join_1).replace("[","").replace("]","").replace("'","").replace("_4","") + + type_name_j2 = str(random.sample(type_names,1)).replace("[","").replace("]","").replace("'","") + fun_column_join_2 = str(random.sample(str_functions,1))+'('+str(random.sample(fun_fix_column_j,1))+' AS '+type_name_j2+')' + time_fun_join_2 = str(fun_column_join_2).replace("[","").replace("]","").replace("'","").replace("_4","") + + tdSql.query("select /*+ para_tables_sort() */1-1 as time_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (timelist == ['NOW','TODAY']) or (timelist == ['TIMETRUNCATE']) or (timelist == ['TO_ISO8601'])\ + or (timelist == ['TO_UNIXTIMESTAMP']) or (timelist == ['TIMEDIFF_1']) or (timelist == ['TIMEDIFF_2']): + sql = "select /*+ para_tables_sort() */ ts1 , timediff(asct1,now) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1, " % time_fun_1 + sql += "%s as asct2, " % time_fun_2 + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts as ts1 from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += ");" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + tdSql.checkRows(100) + self.cur1.execute(sql) + self.explain_sql(sql) + elif (timelist == ['TIMEZONE']) \ + or (timelist == ['CAST']) or (timelist == ['CAST_1']) or (timelist == ['CAST_2']) or (timelist == ['CAST_3']) or (timelist == ['CAST_4']): + sql = "select /*+ para_tables_sort() */ ts2 , asct1,now(),today(),timezone() from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1, " % time_fun_1 + sql += "%s as asct2, " % time_fun_2 + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts as ts2 from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += ");" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + tdSql.checkRows(100) + self.cur1.execute(sql) + self.explain_sql(sql) + elif (timelist == ['ELAPSED']) : + sql = "select /*+ para_tables_sort() */ max(asct1),now(),today(),timezone() from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1, " % time_fun_1 + sql += "%s as asct2 " % time_fun_2 + sql += "from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += ");" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-2 as time_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (timelist == ['NOW','TODAY']) or (timelist == ['TIMETRUNCATE']) or (timelist == ['TO_ISO8601'])\ + or (timelist == ['TO_UNIXTIMESTAMP']) or (timelist == ['TIMEDIFF_1']) or (timelist == ['TIMEDIFF_2']): + sql = "select /*+ para_tables_sort() */ ts , timediff(asct1,now),now(),today(),timezone() from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1, " % time_fun_1 + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts ts from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s )" % random.choice(self.order_where) + sql += "%s " % random.choice(self.unionall_or_union) + sql += "select /*+ para_tables_sort() */ ts , timediff(asct2,now),now(),today(),timezone() from ( select /*+ para_tables_sort() */ " + sql += "%s as asct2, " % time_fun_2 + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts ts from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + #sql += "%s " % random.choice(having_support) + sql += "%s " % random.choice(self.order_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ")" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + elif (timelist == ['TIMEZONE']) or (timelist == ['CAST']) or (timelist == ['CAST_1']) or (timelist == ['CAST_2']) or (timelist == ['CAST_3']) or (timelist == ['CAST_4']): + sql = "select /*+ para_tables_sort() */ ts , (asct1),now(),today(),timezone() from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1, " % time_fun_1 + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts ts from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s )" % random.choice(self.order_where) + sql += "%s " % random.choice(self.unionall_or_union) + sql += "select /*+ para_tables_sort() */ ts , asct2,now(),today(),timezone() from ( select /*+ para_tables_sort() */ " + sql += "%s as asct2, " % time_fun_2 + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts ts from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ")" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + elif (timelist == ['ELAPSED']) : + sql = "select /*+ para_tables_sort() */ min(asct1),now(),today(),timezone() from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1 " % time_fun_1 + sql += " from regular_table_1 where " + sql += "%s )" % random.choice(self.q_where) + sql += "%s " % random.choice(self.unionall_or_union) + sql += "select /*+ para_tables_sort() */ avg(asct2),now(),today(),timezone() from ( select /*+ para_tables_sort() */ " + sql += "%s as asct2 " % time_fun_2 + sql += " from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ")" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-3 as time_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (timelist == ['NOW','TODAY']) or (timelist == ['TIMETRUNCATE']) or (timelist == ['TO_ISO8601'])\ + or (timelist == ['TO_UNIXTIMESTAMP']) or (timelist == ['TIMEDIFF_1']) or (timelist == ['TIMEDIFF_2']): + sql = "select /*+ para_tables_sort() */ ts , timediff(asct1,now) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1, ts ," % time_fun_1 + sql += "%s as asct2, " % time_fun_2 + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s " % random.choice(self.q_select) + sql += "from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s select /*+ para_tables_sort() */" % random.choice(self.unionall_or_union) + sql += "%s as asct2, ts ," % time_fun_2 + sql += "%s as asct1, " % time_fun_1 + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s " % random.choice(self.q_select) + sql += "from regular_table_2 where " + sql += "%s " % random.choice(self.q_where) + sql += " order by asct1 desc " + sql += ")" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + elif (timelist == ['TIMEZONE']) or (timelist == ['CAST']) or (timelist == ['CAST_1']) or (timelist == ['CAST_2']) or (timelist == ['CAST_3']) or (timelist == ['CAST_4']): + sql = "select /*+ para_tables_sort() */ ts , (asct1),now(),today(),timezone() from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1, ts ," % time_fun_1 + sql += "%s as asct2, " % time_fun_2 + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s " % random.choice(self.q_select) + sql += " from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s select /*+ para_tables_sort() */" % random.choice(self.unionall_or_union) + sql += "%s as asct2, ts ," % time_fun_2 + sql += "%s as asct1, " % time_fun_1 + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s " % random.choice(self.q_select) + sql += "from regular_table_2 where " + sql += "%s " % random.choice(self.q_where) + sql += " order by asct1 desc " + sql += ")" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + elif (timelist == ['ELAPSED']) : + sql = "select /*+ para_tables_sort() */ abs(asct1),now(),today(),timezone() from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1," % time_fun_1 + sql += "%s as asct2 " % time_fun_2 + sql += "from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s select /*+ para_tables_sort() */" % random.choice(self.unionall_or_union) + sql += "%s as asct2," % time_fun_2 + sql += "%s as asct1 " % time_fun_1 + sql += "from regular_table_2 where " + sql += "%s " % random.choice(self.q_where) + sql += " order by asct1 asc " + sql += ")" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-4 as time_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (timelist == ['NOW','TODAY']) or (timelist == ['TIMETRUNCATE']) or (timelist == ['TO_ISO8601'])\ + or (timelist == ['TO_UNIXTIMESTAMP']) or (timelist == ['TIMEDIFF_1']) or (timelist == ['TIMEDIFF_2']): + sql = "select /*+ para_tables_sort() */ ts1,ts2 ,timediff(ts1,ts2), timediff(asct1,now) from ( select /*+ para_tables_sort() */ t1.ts as ts1," + sql += "%s as asct11, " % time_fun_join_1 + sql += "%s as asct1, " % time_fun_join_2 + sql += "%s as asct12, " % time_fun_join_1 + sql += "t1.%s as asct111, " % random.choice(self.q_select) + sql += "t2.%s as asct121, " % random.choice(self.q_select) + sql += "t2.ts as ts2 from regular_table_1 t1 , regular_table_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_where) + sql += "and %s " % random.choice(self.q_u_or_where) + sql += "%s " % random.choice(self.order_u_where) + sql += ");" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + tdSql.checkRows(100) + self.cur1.execute(sql) + self.explain_sql(sql) + elif (timelist == ['TIMEZONE']) or (timelist == ['CAST']) or (timelist == ['CAST_1']) or (timelist == ['CAST_2']) or (timelist == ['CAST_3']) or (timelist == ['CAST_4']): + sql = "select /*+ para_tables_sort() */ ts1,ts2 ,timediff(ts1,ts2), (asct1) from ( select /*+ para_tables_sort() */ t1.ts as ts1," + sql += "%s as asct10, " % time_fun_join_1 + sql += "%s as asct1, " % time_fun_join_2 + sql += "%s as asct11, " % time_fun_join_1 + sql += "t1.%s as asct12, " % random.choice(self.q_select) + sql += "t2.%s as asct13, " % random.choice(self.q_select) + sql += "t2.ts as ts2 from regular_table_1 t1 , regular_table_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_where) + sql += "and %s " % random.choice(self.q_u_or_where) + sql += "%s " % random.choice(self.order_u_where) + sql += ");" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + tdSql.checkRows(100) + self.cur1.execute(sql) + self.explain_sql(sql) + elif (timelist == ['ELAPSED']) : + sql = "select /*+ para_tables_sort() */ floor(asct1) from ( select /*+ para_tables_sort() */" + sql += "%s as asct10, " % time_fun_join_1 + sql += "%s as asct1, " % time_fun_join_2 + sql += "%s as asct11" % time_fun_join_1 + sql += " from regular_table_1 t1 , regular_table_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_where) + sql += "and %s " % random.choice(self.q_u_or_where) + sql += ");" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-5 as time_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (timelist == ['ELAPSED']) : + sql = "select /*+ para_tables_sort() */ now(),today(),timezone(), " + sql += "%s, " % time_fun_1 + sql += "%s " % time_fun_2 + sql += " from ( select /*+ para_tables_sort() */ * from regular_table_1 ) where " + sql += "%s " % random.choice(self.q_where) + sql += " ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + else: + sql = "select /*+ para_tables_sort() */ ts ,now(),today(),timezone(), " + sql += "%s as asct11, " % time_fun_1 + sql += "%s as asct12, " % random.choice(self.q_select) + sql += "%s as asct13, " % random.choice(self.q_select) + sql += "%s as asct14 " % time_fun_2 + sql += " from ( select /*+ para_tables_sort() */ * from regular_table_1 ) where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += " ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + tdSql.checkRows(100) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-6 as time_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (timelist == ['NOW','TODAY']) or (timelist == ['TIMETRUNCATE']) or (timelist == ['TO_ISO8601'])\ + or (timelist == ['TO_UNIXTIMESTAMP']) or (timelist == ['TIMEDIFF_1']) or (timelist == ['TIMEDIFF_2']): + sql = "select /*+ para_tables_sort() */ ts1,ts ,timediff(ts1,ts), timediff(asct1,now) from ( select /*+ para_tables_sort() */ t1.ts as ts1," + sql += "%s as asct121, " % time_fun_join_1 + sql += "%s as asct1, " % time_fun_join_2 + sql += "t1.%s as asct11, " % random.choice(self.q_select) + sql += "t2.%s as asct12, " % random.choice(self.q_select) + sql += "%s as asct13, " % time_fun_join_1 + sql += "t2.ts from regular_table_1 t1 , regular_table_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_where) + sql += "and %s )" % random.choice(self.q_u_or_where) + sql += "%s " % random.choice(self.order_where) + sql += "%s " % random.choice(self.limit1_where) + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + elif (timelist == ['TIMEZONE']) or (timelist == ['CAST']) or (timelist == ['CAST_1']) or (timelist == ['CAST_2']) or (timelist == ['CAST_3']) or (timelist == ['CAST_4']): + sql = "select /*+ para_tables_sort() */ ts1,ts ,timediff(ts1,ts), (asct1) from ( select /*+ para_tables_sort() */ t1.ts as ts1," + sql += "%s as asct121, " % time_fun_join_1 + sql += "%s as asct1, " % time_fun_join_2 + sql += "t1.%s as asct11, " % random.choice(self.q_select) + sql += "t2.%s as asct12, " % random.choice(self.q_select) + sql += "%s as asct13, " % time_fun_join_1 + sql += "t2.ts from regular_table_1 t1 , regular_table_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_where) + sql += "and %s )" % random.choice(self.q_u_or_where) + sql += "%s " % random.choice(self.order_where) + sql += "%s " % random.choice(self.limit1_where) + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + elif (timelist == ['ELAPSED']) : + sql = "select /*+ para_tables_sort() */ (asct1)*111 from ( select /*+ para_tables_sort() */ " + sql += "%s as asct121, " % time_fun_join_1 + sql += "%s as asct1, " % time_fun_join_2 + sql += "%s as asct122 " % time_fun_join_1 + sql += " from regular_table_1 t1 , regular_table_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_where) + sql += "and %s )" % random.choice(self.q_u_or_where) + sql += "%s " % random.choice(self.limit1_where) + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-7 as time_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (timelist == ['NOW','TODAY']) or (timelist == ['TIMETRUNCATE']) or (timelist == ['TO_ISO8601'])\ + or (timelist == ['TO_UNIXTIMESTAMP']) or (timelist == ['TIMEDIFF_1']) or (timelist == ['TIMEDIFF_2']): + sql = "select /*+ para_tables_sort() */ ts1,m1 , timediff(asct1,now) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1, ts as m1," % time_fun_1 + sql += "%s as asct2, " % time_fun_2 + sql += "%s as asct11, " % random.choice(self.q_select) + sql += "%s as asct12, " % random.choice(self.t_select) + sql += "ts as ts1 from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += "%s " % random.choice(self.order_where) + sql += ") ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + tdSql.checkRows(300) + self.cur1.execute(sql) + self.explain_sql(sql) + elif (timelist == ['TIMEZONE']) or (timelist == ['CAST']) or (timelist == ['CAST_1']) or (timelist == ['CAST_2']) or (timelist == ['CAST_3']) or (timelist == ['CAST_4']): + sql = "select /*+ para_tables_sort() */ tm1,tm2 , (asct1),now(),today(),timezone() from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1, ts as tm1," % time_fun_1 + sql += "%s as asct2, " % time_fun_2 + sql += "%s as asct11, " % random.choice(self.q_select) + sql += "%s as asct12, " % random.choice(self.t_select) + sql += "ts as tm2 from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += "%s " % random.choice(self.order_where) + sql += ") ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + tdSql.checkRows(300) + self.cur1.execute(sql) + self.explain_sql(sql) + elif (timelist == ['ELAPSED']) : + sql = "select /*+ para_tables_sort() */ (asct1)/asct2 ,now(),today(),timezone() from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1, " % time_fun_1 + sql += "%s as asct2 " % time_fun_2 + sql += "from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += ") ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-8 as time_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (timelist == ['NOW','TODAY']) or (timelist == ['TIMETRUNCATE']) or (timelist == ['TO_ISO8601'])\ + or (timelist == ['TO_UNIXTIMESTAMP']) or (timelist == ['TIMEDIFF_1']) or (timelist == ['TIMEDIFF_2']): + sql = "select /*+ para_tables_sort() */ tm1,tm2 , timediff(asct1,now) " + sql += "from ( select /*+ para_tables_sort() */ " + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s as asct1, ts as tm1," % time_fun_1 + sql += "%s as asct2, " % time_fun_2 + sql += "%s as asct11, " % random.choice(self.q_select) + sql += "%s as asct12, " % random.choice(self.t_select) + sql += "ts as tm2 from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += "%s " % random.choice(self.order_where) + sql += ") ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + tdSql.checkRows(300) + self.cur1.execute(sql) + self.explain_sql(sql) + elif (timelist == ['TIMEZONE']) or (timelist == ['CAST']) or (timelist == ['CAST_1']) or (timelist == ['CAST_2']) or (timelist == ['CAST_3']) or (timelist == ['CAST_4']): + sql = "select /*+ para_tables_sort() */ ts1,ts2 , (asct1),now(),today(),timezone() " + sql += "from ( select /*+ para_tables_sort() */ " + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s as asct1, ts as ts1," % time_fun_1 + sql += "%s as asct2, " % time_fun_2 + sql += "%s as asct11, " % random.choice(self.q_select) + sql += "%s as asct12, " % random.choice(self.t_select) + sql += "ts as ts2 from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += "%s " % random.choice(self.order_where) + sql += ") ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + tdSql.checkRows(300) + self.cur1.execute(sql) + self.explain_sql(sql) + elif (timelist == ['ELAPSED']) : + sql = "select /*+ para_tables_sort() */ floor(abs(asct1)),now(),today(),timezone() " + sql += "from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1, " % time_fun_1 + sql += "%s as asct2 " % time_fun_2 + sql += "from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += ") ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-9 as time_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (timelist == ['NOW','TODAY']) or (timelist == ['TIMETRUNCATE']) or (timelist == ['TO_ISO8601'])\ + or (timelist == ['TO_UNIXTIMESTAMP']) or (timelist == ['TIMEDIFF_1']) or (timelist == ['TIMEDIFF_2']): + sql = "select /*+ para_tables_sort() */ ts1,ts2 ,timediff(ts1,ts2), timediff(asct1,now) from ( select /*+ para_tables_sort() */ t1.ts as ts1," + sql += "%s as asct121, " % time_fun_join_1 + sql += "%s as asct1, " % time_fun_join_2 + sql += "t1.%s as asct11, " % random.choice(self.q_select) + sql += "t1.%s as asct12, " % random.choice(self.q_select) + sql += "t2.%s as asct13, " % random.choice(self.q_select) + sql += "t2.%s as asct14, " % random.choice(self.q_select) + sql += "t2.ts as ts2 from stable_1 t1 , stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.t_join_where) + sql += "and %s " % random.choice(self.t_u_where) + sql += "and %s " % random.choice(self.t_u_or_where) + sql += "%s " % random.choice(self.order_u_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ") ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + elif (timelist == ['TIMEZONE']) or (timelist == ['CAST']) or (timelist == ['CAST_1']) or (timelist == ['CAST_2']) or (timelist == ['CAST_3']) or (timelist == ['CAST_4']): + sql = "select /*+ para_tables_sort() */ ts1,ts2 ,timediff(ts1,ts2), asct1 from ( select /*+ para_tables_sort() */ t1.ts as ts1," + sql += "%s as asct121, " % time_fun_join_1 + sql += "%s as asct1, " % time_fun_join_2 + sql += "t1.%s as asct11, " % random.choice(self.q_select) + sql += "t1.%s as asct12, " % random.choice(self.q_select) + sql += "t2.%s as asct13, " % random.choice(self.q_select) + sql += "t2.%s as asct14, " % random.choice(self.q_select) + sql += "t2.ts as ts2 from stable_1 t1 , stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.t_join_where) + sql += "and %s " % random.choice(self.t_u_where) + sql += "and %s " % random.choice(self.t_u_or_where) + sql += "%s " % random.choice(self.order_u_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ") ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + elif (timelist == ['ELAPSED']) : + sql = "select /*+ para_tables_sort() */ min(asct1*110) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct121, " % time_fun_join_1 + sql += "%s as asct1 " % time_fun_join_2 + sql += "from stable_1 t1 , stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.t_join_where) + sql += "and %s " % random.choice(self.t_u_where) + sql += "and %s " % random.choice(self.t_u_or_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ") ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + self.restartDnodes() + tdSql.query("select /*+ para_tables_sort() */1-10 as time_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (timelist == ['NOW','TODAY']) or (timelist == ['TIMETRUNCATE']) or (timelist == ['TO_ISO8601'])\ + or (timelist == ['TO_UNIXTIMESTAMP']) or (timelist == ['TIMEDIFF_1']) or (timelist == ['TIMEDIFF_2']): + sql = "select /*+ para_tables_sort() */ ts , timediff(asct1,now) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1 ," % time_fun_1 + sql += "%s as asct2, " % time_fun_2 + sql += "%s as asct121, " % random.choice(self.s_r_select) + sql += "%s as asct122, " % random.choice(self.q_select) + sql += "ts from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += ") %s " % random.choice(self.unionall_or_union) + sql += "select /*+ para_tables_sort() */ ts , timediff(asct1,now) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1 ," % time_fun_1 + sql += "%s as asct2, " % time_fun_2 + sql += "%s as asct121, " % random.choice(self.s_r_select) + sql += "%s as asct122, " % random.choice(self.q_select) + sql += "ts from stable_2 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ")" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + elif (timelist == ['TIMEZONE']) or (timelist == ['CAST']) or (timelist == ['CAST_1']) or (timelist == ['CAST_2']) or (timelist == ['CAST_3']) or (timelist == ['CAST_4']): + sql = "select /*+ para_tables_sort() */ ts , (asct1),now(),today(),timezone() from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1 ," % time_fun_1 + sql += "%s as asct2, " % time_fun_2 + sql += "%s as asct121, " % random.choice(self.s_r_select) + sql += "%s as asct122, " % random.choice(self.q_select) + sql += "ts from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += ") %s " % random.choice(self.unionall_or_union) + sql += "select /*+ para_tables_sort() */ ts , (asct2),now(),today(),timezone() from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1 ," % time_fun_1 + sql += "%s as asct2, " % time_fun_2 + sql += "%s as asct121, " % random.choice(self.s_r_select) + sql += "%s as asct122, " % random.choice(self.q_select) + sql += "ts from stable_2 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ")" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + elif (timelist == ['ELAPSED']) : + sql = "select /*+ para_tables_sort() */ abs(asct1),now(),today(),timezone() from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1 ," % time_fun_1 + sql += "%s as asct2 " % time_fun_2 + sql += " from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += ") %s " % random.choice(self.unionall_or_union) + sql += "select /*+ para_tables_sort() */ max(asct2),now(),today(),timezone() from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1 ," % time_fun_1 + sql += "%s as asct2 " % time_fun_2 + sql += "from stable_2 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ")" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + #3 inter union not support + tdSql.query("select /*+ para_tables_sort() */1-11 as time_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (timelist == ['NOW','TODAY']) or (timelist == ['TIMETRUNCATE']) or (timelist == ['TO_ISO8601'])\ + or (timelist == ['TO_UNIXTIMESTAMP']) or (timelist == ['TIMEDIFF_1']) or (timelist == ['TIMEDIFF_2']): + sql = "select /*+ para_tables_sort() */ ts , timediff(asct1,now), timediff(now,asct2) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1 ," % time_fun_1 + sql += "%s as asct2, " % time_fun_2 + sql += "%s as asct121, " % random.choice(self.s_r_select) + sql += "%s as asct122, " % random.choice(self.q_select) + sql += "ts from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += " %s " % random.choice(self.unionall_or_union) + sql += " select /*+ para_tables_sort() */ " + sql += "%s as asct1 ," % time_fun_1 + sql += "%s as asct2, " % time_fun_2 + sql += "%s as asct121, " % random.choice(self.s_r_select) + sql += "%s as asct122, " % random.choice(self.q_select) + sql += "ts from stable_2 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ")" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + elif (timelist == ['TIMEZONE']) or (timelist == ['CAST']) or (timelist == ['CAST_1']) or (timelist == ['CAST_2']) or (timelist == ['CAST_3']) or (timelist == ['CAST_4']): + sql = "select /*+ para_tables_sort() */ ts , asct1,now(),now(),asct2 from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1 ," % time_fun_1 + sql += "%s as asct2, " % time_fun_2 + sql += "%s as asct121, " % random.choice(self.s_r_select) + sql += "%s as asct122, " % random.choice(self.q_select) + sql += "ts from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += " %s " % random.choice(self.unionall_or_union) + sql += " select /*+ para_tables_sort() */ " + sql += "%s as asct1 ," % time_fun_1 + sql += "%s as asct2, " % time_fun_2 + sql += "%s as asct121, " % random.choice(self.s_r_select) + sql += "%s as asct122, " % random.choice(self.q_select) + sql += "ts from stable_2 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ")" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + elif (timelist == ['ELAPSED']) : + sql = "select /*+ para_tables_sort() */ asct1+asct2,now(),today(),timezone() from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1, " % time_fun_1 + sql += "%s as asct2 " % time_fun_2 + sql += " from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += " %s " % random.choice(self.unionall_or_union) + sql += " select /*+ para_tables_sort() */ " + sql += "%s as asct1 ," % time_fun_1 + sql += "%s as asct2 " % time_fun_2 + sql += " from stable_2 where " + sql += "%s " % random.choice(self.q_where) + sql += "order by asct1 " + sql += "%s " % random.choice(self.limit1_where) + sql += ")" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-12 as time_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (timelist == ['NOW','TODAY']) or (timelist == ['TIMETRUNCATE']) or (timelist == ['TO_ISO8601'])\ + or (timelist == ['TO_UNIXTIMESTAMP']) or (timelist == ['TIMEDIFF_1']) or (timelist == ['TIMEDIFF_2']): + sql = "select /*+ para_tables_sort() */ ts1,ts2 ,timediff(ts1,ts2), timediff(asct1,now) from ( select /*+ para_tables_sort() */ t1.ts as ts1," + sql += "%s, " % time_fun_join_1 + sql += "%s as asct1, " % time_fun_join_2 + sql += "t1.%s as asct11, " % random.choice(self.q_select) + sql += "t1.%s as asct12, " % random.choice(self.q_select) + sql += "t2.%s as asct13, " % random.choice(self.q_select) + sql += "t2.%s as asct14, " % random.choice(self.q_select) + sql += "t2.ts as ts2 from stable_1 t1 , stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.t_u_where) + sql += "and %s " % random.choice(self.t_u_or_where) + sql += "%s " % random.choice(self.order_u_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ");" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + elif (timelist == ['TIMEZONE']) or (timelist == ['CAST']) or (timelist == ['CAST_1']) or (timelist == ['CAST_2']) or (timelist == ['CAST_3']) or (timelist == ['CAST_4']): + sql = "select /*+ para_tables_sort() */ ts1,ts2 ,timediff(ts1,ts2), asct1,now() from ( select /*+ para_tables_sort() */ t1.ts as ts1," + sql += "%s, " % time_fun_join_1 + sql += "%s as asct1, " % time_fun_join_2 + sql += "t1.%s as asct11, " % random.choice(self.q_select) + sql += "t1.%s as asct12, " % random.choice(self.q_select) + sql += "t2.%s as asct13, " % random.choice(self.q_select) + sql += "t2.%s as asct14, " % random.choice(self.q_select) + sql += "t2.ts as ts2 from stable_1 t1 , stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.t_u_where) + sql += "and %s " % random.choice(self.t_u_or_where) + sql += "%s " % random.choice(self.order_u_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ");" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + elif (timelist == ['ELAPSED']) : + sql = "select /*+ para_tables_sort() */min(floor(asct1)),now() from ( select /*+ para_tables_sort() */" + sql += "%s as asct121, " % time_fun_join_1 + sql += "%s as asct1 " % time_fun_join_2 + sql += " from stable_1 t1 , stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.t_u_where) + sql += "and %s " % random.choice(self.t_u_or_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ");" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-13 as time_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (timelist == ['NOW','TODAY']) or (timelist == ['TIMETRUNCATE']) or (timelist == ['TO_ISO8601'])\ + or (timelist == ['TO_UNIXTIMESTAMP']) or (timelist == ['TIMEDIFF_1']) or (timelist == ['TIMEDIFF_2']): + sql = "select /*+ para_tables_sort() */ ts , timediff(%s,now)," % time_fun_2 + sql += "%s as asct1, " % time_fun_1 + sql += "%s as asct11, " % random.choice(self.q_select) + sql += "%s as asct12, " % random.choice(self.q_select) + sql += "%s as asct13, " % time_fun_2 + sql += "%s as asct122 " % random.choice(self.t_select) + sql += " from ( select /*+ para_tables_sort() */ * from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += "%s " % random.choice(self.order_where) + sql += ") ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + tdSql.checkRows(300) + self.cur1.execute(sql) + self.explain_sql(sql) + elif (timelist == ['TIMEZONE']) or (timelist == ['CAST']) or (timelist == ['CAST_1']) or (timelist == ['CAST_2']) or (timelist == ['CAST_3']) or (timelist == ['CAST_4']): + sql = "select /*+ para_tables_sort() */ ts ,now(),today(),timezone(), " + sql += "%s as asct1, " % time_fun_1 + sql += "%s as asct11, " % random.choice(self.q_select) + sql += "%s as asct12, " % random.choice(self.q_select) + sql += "%s as asct13, " % time_fun_2 + sql += "%s as asct122 " % random.choice(self.t_select) + sql += " from ( select /*+ para_tables_sort() */ * from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += "%s " % random.choice(self.order_where) + sql += ") ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + tdSql.checkRows(300) + self.cur1.execute(sql) + self.explain_sql(sql) + elif (timelist == ['ELAPSED']) : + sql = "select /*+ para_tables_sort() */ now(),today(),timezone(), " + sql += "%s as asct1, " % time_fun_1 + sql += "%s as asct12 " % time_fun_2 + sql += " from ( select /*+ para_tables_sort() */ * from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += "%s " % random.choice(self.order_where) + sql += ") ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-14 as time_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (timelist == ['NOW','TODAY']) or (timelist == ['TIMETRUNCATE']) or (timelist == ['TO_ISO8601'])\ + or (timelist == ['TO_UNIXTIMESTAMP']) or (timelist == ['TIMEDIFF_1']) or (timelist == ['TIMEDIFF_2']): + sql = "select /*+ para_tables_sort() */ ts , timediff(asct1,now),timediff(now,asct2) from ( select /*+ para_tables_sort() */ts ts ," + sql += "%s as asct1, " % time_fun_1 + sql += "%s as asct2" % time_fun_2 + sql += " from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.partiton_where) + sql += "%s " % random.choice(self.order_desc_where) + sql += "%s " % random.choice([self.limit_where[2] , self.limit_where[3]] ) + sql += " ) ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + elif (timelist == ['TIMEZONE']) or (timelist == ['CAST']) or (timelist == ['CAST_1']) or (timelist == ['CAST_2']) or (timelist == ['CAST_3']) or (timelist == ['CAST_4']): + sql = "select /*+ para_tables_sort() */ ts , (asct1),now(),(now()),asct2 from ( select /*+ para_tables_sort() */ts ts ," + sql += "%s as asct1, " % time_fun_1 + sql += "%s as asct2" % time_fun_2 + sql += " from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.partiton_where) + sql += "%s " % random.choice(self.order_desc_where) + sql += "%s " % random.choice([self.limit_where[2] , self.limit_where[3]] ) + sql += " ) ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + elif (timelist == ['ELAPSED']) : + sql = "select /*+ para_tables_sort() */ (asct1)*asct2,now(),(now()) from ( select /*+ para_tables_sort() */" + sql += "%s as asct1, " % time_fun_1 + sql += "%s as asct2" % time_fun_2 + sql += " from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.partiton_where) + sql += "%s " % random.choice([self.limit_where[2] , self.limit_where[3]] ) + sql += " ) ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-15 as time_nest from stable_1 limit 1;") + for i in range(self.fornum): + if (timelist == ['NOW','TODAY']) or (timelist == ['TIMETRUNCATE']) or (timelist == ['TO_ISO8601'])\ + or (timelist == ['TO_UNIXTIMESTAMP']) or (timelist == ['TIMEDIFF_1']) or (timelist == ['TIMEDIFF_2']): + sql = "select /*+ para_tables_sort() */ ts1,ts ,timediff(ts1,ts), timediff(asct1,now),timediff(now,asct2) from ( select /*+ para_tables_sort() */ t1.ts as ts1," + sql += "%s as asct2, " % time_fun_join_1 + sql += "%s as asct1, " % time_fun_join_2 + sql += "t1.%s as asct11, " % random.choice(self.q_select) + sql += "t1.%s as asct12, " % random.choice(self.q_select) + sql += "t2.%s as asct13, " % random.choice(self.q_select) + sql += "t2.%s as asct14, " % random.choice(self.q_select) + sql += "t2.ts from stable_1 t1, stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.t_join_where) + sql += " and %s " % random.choice(self.qt_u_or_where) + sql += "%s " % random.choice(self.partiton_where_j) + sql += "%s " % random.choice(self.slimit1_where) + sql += ") " + sql += "%s " % random.choice(self.order_desc_where) + sql += "%s ;" % random.choice(self.limit_u_where) + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + elif (timelist == ['TIMEZONE']) or (timelist == ['CAST']) or (timelist == ['CAST_1']) or (timelist == ['CAST_2']) or (timelist == ['CAST_3']) or (timelist == ['CAST_4']): + sql = "select /*+ para_tables_sort() */ ts1,ts ,timediff(ts1,ts), asct1,(now()),(now()),asct2 ,now(),today(),timezone() from ( select /*+ para_tables_sort() */ t1.ts as ts1," + sql += "%s as asct2, " % time_fun_join_1 + sql += "%s as asct1, " % time_fun_join_2 + sql += "t1.%s as asct11, " % random.choice(self.q_select) + sql += "t1.%s as asct12, " % random.choice(self.q_select) + sql += "t2.%s as asct13, " % random.choice(self.q_select) + sql += "t2.%s as asct14, " % random.choice(self.q_select) + sql += "t2.ts from stable_1 t1, stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.t_join_where) + sql += " and %s " % random.choice(self.qt_u_or_where) + sql += "%s " % random.choice(self.partiton_where_j) + sql += "%s " % random.choice(self.slimit1_where) + sql += ") " + sql += "%s " % random.choice(self.order_desc_where) + sql += "%s ;" % random.choice(self.limit_u_where) + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + elif (timelist == ['ELAPSED']) : + sql = "select /*+ para_tables_sort() */ asct1,(now()),(now()),asct2 ,now(),today(),timezone() from ( select /*+ para_tables_sort() */" + sql += "%s as asct2, " % time_fun_join_1 + sql += "%s as asct1 " % time_fun_join_2 + sql += "from stable_1 t1, stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.t_join_where) + sql += " and %s " % random.choice(self.qt_u_or_where) + sql += "%s " % random.choice(self.partiton_where_j) + sql += "%s " % random.choice(self.slimit1_where) + sql += ") " + sql += "%s ;" % random.choice(self.limit_u_where) + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + #taos -f sql + startTime_taos_f = time.time() + print("taos -f %s sql start!" %timelist) + taos_cmd1 = "taos -f %s/%s.sql" % (self.testcasePath,self.testcaseFilename) + #_ = subprocess.check_output(taos_cmd1, shell=True).decode("utf-8") + _ = subprocess.check_output(taos_cmd1, shell=True) + print("taos -f %s sql over!" %timelist) + endTime_taos_f = time.time() + print("taos_f total time %ds" % (endTime_taos_f - startTime_taos_f)) + + print("=========%s====over=============" %timelist) + + def base_nest(self,baselist): + + print("==========%s===start=============" %baselist) + os.system("rm -rf %s/%s.sql" % (self.testcasePath,self.testcaseFilename)) + + self.dropandcreateDB_random("%s" %self.db_nest, 1) + + if (baselist == ['A']) or (baselist == ['S']) or (baselist == ['F']) \ + or (baselist == ['C']): + base_functions = baselist + fun_fix_column = ['(q_bigint)','(q_smallint)','(q_tinyint)','(q_int)','(q_float)','(q_double)','(q_bigint_null)','(q_smallint_null)','(q_tinyint_null)','(q_int_null)','(q_float_null)','(q_double_null)'] + fun_column_1 = random.sample(base_functions,1)+random.sample(fun_fix_column,1) + base_fun_1 = str(fun_column_1).replace("[","").replace("]","").replace("'","").replace(", ","") + fun_column_2 = random.sample(base_functions,1)+random.sample(fun_fix_column,1) + base_fun_2 = str(fun_column_2).replace("[","").replace("]","").replace("'","").replace(", ","") + + fun_fix_column_j = ['(t1.q_bigint)','(t1.q_smallint)','(t1.q_tinyint)','(t1.q_int)','(t1.q_float)','(t1.q_double)','(t1.q_bigint_null)','(t1.q_smallint_null)','(t1.q_tinyint_null)','(t1.q_int_null)','(t1.q_float_null)','(t1.q_double_null)', + '(t2.q_bigint)','(t2.q_smallint)','(t2.q_tinyint)','(t2.q_int)','(t2.q_float)','(t2.q_double)','(t2.q_bigint_null)','(t2.q_smallint_null)','(t2.q_tinyint_null)','(t2.q_int_null)','(t2.q_float_null)','(t2.q_double_null)'] + fun_column_join_1 = random.sample(base_functions,1)+random.sample(fun_fix_column_j,1) + base_fun_join_1 = str(fun_column_join_1).replace("[","").replace("]","").replace("'","").replace(", ","") + fun_column_join_2 = random.sample(base_functions,1)+random.sample(fun_fix_column_j,1) + base_fun_join_2 = str(fun_column_join_2).replace("[","").replace("]","").replace("'","").replace(", ","") + + elif (baselist == ['P']) or (baselist == ['M']) or (baselist == ['S'])or (baselist == ['T']): + base_functions = baselist + num = random.randint(0, 1000) + fun_fix_column = ['(q_bigint,num)','(q_smallint,num)','(q_tinyint,num)','(q_int,num)','(q_float,num)','(q_double,num)', + '(q_bigint_null,num)','(q_smallint_null,num)','(q_tinyint_null,num)','(q_int_null,num)','(q_float_null,num)','(q_double_null,num)'] + fun_column_1 = random.sample(base_functions,1)+random.sample(fun_fix_column,1) + base_fun_1 = str(fun_column_1).replace("[","").replace("]","").replace("'","").replace(", ","").replace("num",base(num)) + fun_column_2 = random.sample(base_functions,1)+random.sample(fun_fix_column,1) + base_fun_2 = str(fun_column_2).replace("[","").replace("]","").replace("'","").replace(", ","").replace("num",base(num)) + + fun_fix_column_j = ['(t1.q_bigint,num)','(t1.q_smallint,num)','(t1.q_tinyint,num)','(t1.q_int,num)','(t1.q_float,num)','(t1.q_double,num)', + '(t1.q_bigint_null,num)','(t1.q_smallint_null,num)','(t1.q_tinyint_null,num)','(t1.q_int_null,num)','(t1.q_float_null,num)','(t1.q_double_null,num)', + '(t2.q_bigint,num)','(t2.q_smallint,num)','(t2.q_tinyint,num)','(t2.q_int,num)','(t2.q_float,num)','(t2.q_double,num)', + '(t2.q_bigint_null,num)','(t2.q_smallint_null,num)','(t2.q_tinyint_null,num)','(t2.q_int_null,num)','(t2.q_float_null,num)','(t2.q_double_null,num)'] + fun_column_join_1 = random.sample(base_functions,1)+random.sample(fun_fix_column_j,1) + base_fun_join_1 = str(fun_column_join_1).replace("[","").replace("]","").replace("'","").replace(", ","").replace("num",base(num)) + fun_column_join_2 = random.sample(base_functions,1)+random.sample(fun_fix_column_j,1) + base_fun_join_2 = str(fun_column_join_2).replace("[","").replace("]","").replace("'","").replace(", ","").replace("num",base(num)) + + tdSql.query("select /*+ para_tables_sort() */1-1 as base_nest from stable_1 limit 1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ ts , floor(asct1) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1, " % base_fun_1 + sql += "%s as asct2, " % base_fun_2 + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += ");" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + tdSql.checkRows(100) + self.cur1.execute(sql) + + tdSql.query("select /*+ para_tables_sort() */1-2 as base_nest from stable_1 limit 1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ ts , abs(asct1) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1, " % base_fun_1 + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts ts from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s )" % random.choice(self.order_where) + sql += "%s " % random.choice(self.unionall_or_union) + sql += "select /*+ para_tables_sort() */ ts , asct2 from ( select /*+ para_tables_sort() */ " + sql += "%s as asct2, " % base_fun_2 + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts ts from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + #sql += "%s " % random.choice(having_support) + sql += "%s " % random.choice(self.order_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ")" + tdLog.info(sql) + tdLog.info(len(sql)) + #tdSql.query(sql) + #self.cur1.execute(sql) + + tdSql.query("select /*+ para_tables_sort() */1-3 as base_nest from stable_1 limit 1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ ts , min(asct1) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1, ts ," % base_fun_1 + sql += "%s as asct2, " % base_fun_2 + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s select /*+ para_tables_sort() */" % random.choice(self.unionall_or_union) + sql += "%s as asct2, ts ," % base_fun_2 + sql += "%s as asct1, " % base_fun_1 + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts from regular_table_2 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += ")" + tdLog.info(sql) + tdLog.info(len(sql)) + #tdSql.query(sql) + #self.cur1.execute(sql) + + tdSql.query("select /*+ para_tables_sort() */1-4 as base_nest from stable_1 limit 1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ ts1,ts2 ,timediff(ts1,ts2), asct1 from ( select /*+ para_tables_sort() */ t1.ts as ts1," + sql += "%s, " % base_fun_join_1 + sql += "%s as asct1, " % base_fun_join_2 + sql += "%s, " % base_fun_join_1 + sql += "t1.%s, " % random.choice(self.q_select) + sql += "t2.%s, " % random.choice(self.q_select) + sql += "t2.ts from regular_table_1 t1 , regular_table_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_where) + sql += "and %s " % random.choice(self.q_u_or_where) + sql += "%s " % random.choice(self.order_u_where) + sql += ");" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + tdSql.checkRows(100) + self.cur1.execute(sql) + + tdSql.query("select /*+ para_tables_sort() */1-5 as base_nest from stable_1 limit 1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ ts ," + sql += "%s, " % base_fun_1 + sql += "%s, " % random.choice(self.q_select) + sql += "%s, " % random.choice(self.q_select) + sql += "%s " % base_fun_2 + sql += " from ( select /*+ para_tables_sort() */ * from regular_table_1 ) where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += " ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + tdSql.checkRows(100) + self.cur1.execute(sql) + + tdSql.query("select /*+ para_tables_sort() */1-6 as base_nest from stable_1 limit 1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ ts , max(asct1) from ( select /*+ para_tables_sort() */ t1.ts as ts1," + sql += "%s, " % base_fun_join_1 + sql += "%s as asct1, " % base_fun_join_2 + sql += "t1.%s, " % random.choice(self.q_select) + sql += "t2.%s, " % random.choice(self.q_select) + sql += "%s, " % base_fun_join_1 + sql += "t2.ts from regular_table_1 t1 , regular_table_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_where) + sql += "and %s )" % random.choice(self.q_u_or_where) + sql += "%s " % random.choice(self.order_where) + sql += "%s " % random.choice(self.limit1_where) + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + + tdSql.query("select /*+ para_tables_sort() */1-7 as base_nest from stable_1 limit 1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ ts , abs(asct1) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1, ts ," % base_fun_1 + sql += "%s as asct2, " % base_fun_2 + sql += "%s, " % random.choice(self.q_select) + sql += "%s, " % random.choice(self.t_select) + sql += "ts from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += "%s " % random.choice(self.order_where) + sql += ") ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + tdSql.checkRows(300) + self.cur1.execute(sql) + + tdSql.query("select /*+ para_tables_sort() */1-8 as base_nest from stable_1 limit 1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ ts,floor(asct1) " + sql += "from ( select /*+ para_tables_sort() */ " + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s as asct1, ts ," % base_fun_1 + sql += "%s as asct2, " % base_fun_2 + sql += "%s, " % random.choice(self.q_select) + sql += "%s, " % random.choice(self.t_select) + sql += "ts from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += "%s " % random.choice(self.order_where) + sql += ") ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + tdSql.checkRows(300) + self.cur1.execute(sql) + + tdSql.query("select /*+ para_tables_sort() */1-9 as base_nest from stable_1 limit 1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ ts , max(asct1) from ( select /*+ para_tables_sort() */ t1.ts as ts1," + sql += "%s, " % base_fun_join_1 + sql += "%s as asct1, " % base_fun_join_2 + sql += "t1.%s, " % random.choice(self.q_select) + sql += "t1.%s, " % random.choice(self.q_select) + sql += "t2.%s, " % random.choice(self.q_select) + sql += "t2.%s, " % random.choice(self.q_select) + sql += "t2.ts from stable_1 t1 , stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.t_join_where) + sql += "and %s " % random.choice(self.t_u_where) + sql += "and %s " % random.choice(self.t_u_or_where) + sql += "%s " % random.choice(self.order_u_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ") ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + + self.restartDnodes() + tdSql.query("select /*+ para_tables_sort() */1-10 as base_nest from stable_1 limit 1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ ts , min(asct1) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1, ts ," % base_fun_1 + sql += "%s as asct2, " % base_fun_2 + sql += "%s, " % random.choice(self.s_r_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += ") %s " % random.choice(self.unionall_or_union) + sql += "select /*+ para_tables_sort() */ ts , max(asct2) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1, ts ," % base_fun_1 + sql += "%s as asct2, " % base_fun_2 + sql += "%s, " % random.choice(self.s_r_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts from stable_2 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ")" + tdLog.info(sql) + tdLog.info(len(sql)) + #tdSql.query(sql) + #self.cur1.execute(sql) + + #3 inter union not support + tdSql.query("select /*+ para_tables_sort() */1-11 as base_nest from stable_1 limit 1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ ts , min(asct1), max(asct2) from ( select /*+ para_tables_sort() */ " + sql += "%s as asct1, ts ," % base_fun_1 + sql += "%s as asct2, " % base_fun_2 + sql += "%s, " % random.choice(self.s_r_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + #sql += "%s " % random.choice(limit1_where) + sql += " %s " % random.choice(self.unionall_or_union) + sql += " select /*+ para_tables_sort() */ " + sql += "%s as asct1, ts ," % base_fun_1 + sql += "%s as asct2, " % base_fun_2 + sql += "%s, " % random.choice(self.s_r_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts from stable_2 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ")" + tdLog.info(sql) + tdLog.info(len(sql)) + #TD-15837 tdSql.query(sql) + # self.cur1.execute(sql) + + tdSql.query("select /*+ para_tables_sort() */1-12 as base_nest from stable_1 limit 1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ ts , max(asct1) from ( select /*+ para_tables_sort() */ t1.ts as ts1," + sql += "%s, " % base_fun_join_1 + sql += "%s as asct1, " % base_fun_join_2 + sql += "t1.%s, " % random.choice(self.q_select) + sql += "t1.%s, " % random.choice(self.q_select) + sql += "t2.%s, " % random.choice(self.q_select) + sql += "t2.%s, " % random.choice(self.q_select) + sql += "t2.ts from stable_1 t1 , stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.t_u_where) + sql += "and %s " % random.choice(self.t_u_or_where) + sql += "%s " % random.choice(self.order_u_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ");" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + + tdSql.query("select /*+ para_tables_sort() */1-13 as base_nest from stable_1 limit 1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ ts ," + sql += "%s, " % base_fun_1 + sql += "%s, " % random.choice(self.q_select) + sql += "%s, " % random.choice(self.q_select) + sql += "%s " % base_fun_2 + sql += "%s " % random.choice(self.t_select) + sql += " from ( select /*+ para_tables_sort() */ * from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += "%s " % random.choice(self.order_where) + sql += ") ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + tdSql.checkRows(300) + self.cur1.execute(sql) + + tdSql.query("select /*+ para_tables_sort() */1-14 as base_nest from stable_1 limit 1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ avg(asct1),count(asct2) from ( select /*+ para_tables_sort() */" + sql += "%s as asct1, " % base_fun_1 + sql += "%s as asct2" % base_fun_2 + sql += " from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.partiton_where) + sql += "%s " % random.choice(self.order_desc_where) + sql += "%s " % random.choice([self.limit_where[2] , self.limit_where[3]] ) + sql += " ) ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + + tdSql.query("select /*+ para_tables_sort() */1-15 as base_nest from stable_1 limit 1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ ts , max(asct1) from ( select /*+ para_tables_sort() */ t1.ts as ts1," + sql += "%s, " % base_fun_join_1 + sql += "%s as asct1, " % base_fun_join_2 + sql += "t1.%s, " % random.choice(self.q_select) + sql += "t1.%s, " % random.choice(self.q_select) + sql += "t2.%s, " % random.choice(self.q_select) + sql += "t2.%s " % random.choice(self.q_select) + sql += "from stable_1 t1, stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.t_join_where) + sql += " and %s " % random.choice(self.qt_u_or_where) + sql += "%s " % random.choice(self.partiton_where_j) + sql += "%s " % random.choice(self.slimit1_where) + sql += ") " + sql += "%s " % random.choice(self.order_desc_where) + sql += "%s ;" % random.choice(self.limit_u_where) + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + + #taos -f sql + startTime_taos_f = time.time() + print("taos -f %s sql start!" %baselist) + taos_cmd1 = "taos -f %s/%s.sql" % (self.testcasePath,self.testcaseFilename) + _ = subprocess.check_output(taos_cmd1, shell=True).decode("utf-8") + print("taos -f %s sql over!" %baselist) + endTime_taos_f = time.time() + print("taos_f total time %ds" % (endTime_taos_f - startTime_taos_f)) + + print("=========%s====over=============" %baselist) + + def function_before_26(self): + + print('=====================2.6 old function start ===========') + os.system("rm -rf %s/%s.sql" % (self.testcasePath,self.testcaseFilename)) + + self.dropandcreateDB_random("%s" %self.db_nest, 1) + + #1 select /*+ para_tables_sort() */* from (select /*+ para_tables_sort() */column form regular_table where <\>\in\and\or order by) + tdSql.query("select /*+ para_tables_sort() */1-1 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ tas from ( select /*+ para_tables_sort() */ " + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts as tas from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += ");" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql,queryTimes=1) + tdSql.checkRows(100) + self.cur1.execute(sql) + self.explain_sql(sql) + + #1 outer union not support + #self.dropandcreateDB_random("%s" %db, 1) + tdSql.query("select /*+ para_tables_sort() */1-2 from stable_1;") + for i in range(self.fornum): + #sql = "select /*+ para_tables_sort() */ ts , * from ( select /*+ para_tables_sort() */ " + sql = "select /*+ para_tables_sort() */ t1s from ( select /*+ para_tables_sort() */ " + sql += "%s, " % random.choice(self.s_r_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts as t1s from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += ") union " + sql += "select /*+ para_tables_sort() */ t2s from ( select /*+ para_tables_sort() */ " + sql += "%s, " % random.choice(self.s_r_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts as t2s from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += ")" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + tdSql.checkRows(100) + self.cur1.execute(sql) + self.explain_sql(sql) + + #self.dropandcreateDB_random("%s" %db, 1) + tdSql.query("select /*+ para_tables_sort() */1-2 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ ts from ( select /*+ para_tables_sort() */ " + sql += "%s, " % random.choice(self.s_r_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += ") union all " + sql += "select /*+ para_tables_sort() */ ts from ( select /*+ para_tables_sort() */ " + sql += "%s, " % random.choice(self.s_r_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += ")" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + tdSql.checkRows(200) + self.cur1.execute(sql) + self.explain_sql(sql) + + #1 inter union not support + tdSql.query("select /*+ para_tables_sort() */1-3 from stable_1;") + for i in range(self.fornum): + #sql = "select /*+ para_tables_sort() */ ts , * from ( select /*+ para_tables_sort() */ " + sql = "select /*+ para_tables_sort() */ ts from ( select /*+ para_tables_sort() */ " + sql += "%s, " % random.choice(self.s_r_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "" + sql += " union all select /*+ para_tables_sort() */ " + sql += "%s, " % random.choice(self.s_r_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts from regular_table_2 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += ")" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + tdSql.checkRows(300) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */1-3 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ ts from ( select /*+ para_tables_sort() */ " + sql += "%s, " % random.choice(self.s_r_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += " union all select /*+ para_tables_sort() */ " + sql += "%s, " % random.choice(self.s_r_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts from regular_table_2 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += ")" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + tdSql.checkRows(300) + self.cur1.execute(sql) + self.explain_sql(sql) + + #join:select /*+ para_tables_sort() */* from (select /*+ para_tables_sort() */column form regular_table1,regular_table2 where t1.ts=t2.ts and <\>\in\and\or order by) + #self.dropandcreateDB_random("%s" %db, 1) + tdSql.query("select /*+ para_tables_sort() */1-4 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */ t1.ts as t1ts," + sql += "t1.%s as t11, " % random.choice(self.q_select) + sql += "t1.%s as t12, " % random.choice(self.q_select) + sql += "t2.%s as t21, " % random.choice(self.q_select) + sql += "t2.%s as t22, " % random.choice(self.q_select) + sql += "t2.ts as t2ts from regular_table_1 t1 , regular_table_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_where) + sql += "and %s " % random.choice(self.q_u_or_where) + sql += "%s " % random.choice(self.order_u_where) + sql += ");" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + tdSql.checkRows(100) + self.cur1.execute(sql) + self.explain_sql(sql) + + + #2 select /*+ para_tables_sort() */column from (select /*+ para_tables_sort() */* form regular_table ) where <\>\in\and\or order by + #self.dropandcreateDB_random("%s" %db, 1) + tdSql.query("select /*+ para_tables_sort() */2-1 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ ts ," + sql += "%s, " % random.choice(self.s_r_select) + sql += "%s " % random.choice(self.q_select) + sql += " from ( select /*+ para_tables_sort() */ * from regular_table_1 ) where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += " ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + tdSql.checkRows(100) + self.cur1.execute(sql) + self.explain_sql(sql) + + #join: select /*+ para_tables_sort() */column from (select /*+ para_tables_sort() */column form regular_table1,regular_table2 )where t1.ts=t2.ts and <\>\in\and\or order by + #cross join not supported yet + tdSql.query("select /*+ para_tables_sort() */2-2 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ ts , * from ( select /*+ para_tables_sort() */ t1.ts ," + sql += "t1.%s, " % random.choice(self.q_select) + sql += "t1.%s, " % random.choice(self.q_select) + sql += "t2.%s, " % random.choice(self.q_select) + sql += "t2.%s, " % random.choice(self.q_select) + sql += "t2.ts from regular_table_1 t1 , regular_table_2 t2 ) where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_where) + sql += "%s " % random.choice(self.order_u_where) + #sql += ");" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.error(sql) + + #3 select /*+ para_tables_sort() */* from (select /*+ para_tables_sort() */column\tag form stable where <\>\in\and\or order by ) + #self.dropandcreateDB_random("%s" %db, 1) + tdSql.query("select /*+ para_tables_sort() */3-1 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */ " + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s, " % random.choice(self.q_select) + sql += "%s, " % random.choice(self.t_select) + sql += "ts from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += "%s " % random.choice(self.order_where) + sql += ") ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + tdSql.checkRows(300) + self.cur1.execute(sql) + self.explain_sql(sql) + tdSql.query("select /*+ para_tables_sort() */3-1 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ ts, " + sql += "%s " % random.choice(self.s_r_select) + sql += "from ( select /*+ para_tables_sort() */ " + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s, " % random.choice(self.q_select) + sql += "%s, " % random.choice(self.t_select) + sql += "ts from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += "%s " % random.choice(self.order_where) + sql += ") ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + tdSql.checkRows(300) + self.cur1.execute(sql) + self.explain_sql(sql) + + # select /*+ para_tables_sort() */ts,* from (select /*+ para_tables_sort() */column\tag form stable1,stable2 where t1.ts = t2.ts and <\>\in\and\or order by ) + #self.dropandcreateDB_random("%s" %db, 1) + tdSql.query("select /*+ para_tables_sort() */3-2 from stable_1;") + for i in range(self.fornum): + #sql = "select /*+ para_tables_sort() */ ts , * from ( select /*+ para_tables_sort() */ t1.ts as t1ts , " + sql = "select /*+ para_tables_sort() */ t1ts , t2ts from ( select /*+ para_tables_sort() */ t1.ts as t1ts , " + sql += "t1.%s as t11, " % random.choice(self.t_select) + sql += "t1.%s as t12, " % random.choice(self.q_select) + sql += "t2.%s as t13, " % random.choice(self.t_select) + sql += "t2.%s as t14, " % random.choice(self.q_select) + sql += "t2.ts as t2ts from stable_1 t1 , stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.t_join_where) + sql += "%s " % random.choice(self.order_u_where) + sql += ") ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + #3 outer union not support + self.restartDnodes() + tdSql.query("select /*+ para_tables_sort() */3-3 from stable_1;") + for i in range(self.fornum): + #sql = "select /*+ para_tables_sort() */ ts , * from ( select /*+ para_tables_sort() */ " + sql = "select /*+ para_tables_sort() */ ts1 from ( select /*+ para_tables_sort() */ " + sql += "%s, " % random.choice(self.s_r_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts as ts1 from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += ") union " + sql += "select /*+ para_tables_sort() */ ts2 from ( select /*+ para_tables_sort() */ " + sql += "%s, " % random.choice(self.s_r_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts as ts2 from stable_2 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += ")" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + tdSql.checkRows(500) + self.cur1.execute(sql) + self.explain_sql(sql) + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ ts1 from ( select /*+ para_tables_sort() */ " + sql += "%s, " % random.choice(self.s_r_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts as ts1 from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += ") union all " + sql += "select /*+ para_tables_sort() */ ts2 from ( select /*+ para_tables_sort() */ " + sql += "%s, " % random.choice(self.s_r_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts as ts2 from stable_2 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += ")" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + tdSql.checkRows(600) + self.cur1.execute(sql) + self.explain_sql(sql) + + #3 inter union not support + tdSql.query("select /*+ para_tables_sort() */3-4 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */ " + sql += "%s, " % random.choice(self.s_r_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += " %s " % random.choice(self.unionall_or_union) + sql += " select /*+ para_tables_sort() */ " + sql += "%s, " % random.choice(self.s_r_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts from stable_2 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += ")" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + #join:select /*+ para_tables_sort() */* from (select /*+ para_tables_sort() */column form stable1,stable2 where t1.ts=t2.ts and <\>\in\and\or order by) + tdSql.query("select /*+ para_tables_sort() */3-5 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */ t1.ts as t1ts," + sql += "t1.%s as t11, " % random.choice(self.q_select) + sql += "t1.%s as t12, " % random.choice(self.q_select) + sql += "t2.%s as t21, " % random.choice(self.q_select) + sql += "t2.%s as t22, " % random.choice(self.q_select) + sql += "t2.ts as t2ts from stable_1 t1 , stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.t_u_where) + sql += "%s " % random.choice(self.order_u_where) + sql += ");" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + tdSql.checkRows(100) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */3-6 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */ t1.ts as t1ts ," + sql += "t1.%s as t11, " % random.choice(self.q_select) + sql += "t1.%s as t12, " % random.choice(self.q_select) + sql += "t2.%s as t21, " % random.choice(self.q_select) + sql += "t2.%s as t22, " % random.choice(self.q_select) + sql += "t2.ts as t2ts from stable_1 t1 , stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.t_u_or_where) + sql += "%s " % random.choice(self.order_u_where) + sql += ");" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + tdSql.checkRows(100) + self.cur1.execute(sql) + self.explain_sql(sql) + + #4 select /*+ para_tables_sort() */column from (select /*+ para_tables_sort() */* form stable where <\>\in\and\or order by ) + #self.dropandcreateDB_random("%s" %db, 1) + tdSql.query("select /*+ para_tables_sort() */4-1 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ ts , " + sql += "%s as t11, " % random.choice(self.q_select) + sql += "%s as t12, " % random.choice(self.q_select) + sql += "%s " % random.choice(self.t_select) + sql += " from ( select /*+ para_tables_sort() */ * from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += "%s " % random.choice(self.order_where) + sql += ") ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + tdSql.checkRows(300) + self.cur1.execute(sql) + self.explain_sql(sql) + + #5 select /*+ para_tables_sort() */distinct column\tag from (select /*+ para_tables_sort() */* form stable where <\>\in\and\or order by limit offset ) + tdSql.query("select /*+ para_tables_sort() */5-1 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ " + sql += "%s " % random.choice(self.dqt_select) + sql += " from ( select /*+ para_tables_sort() */ * from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += "%s " % random.choice(self.order_where) + sql += ") ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + #5-1 select /*+ para_tables_sort() */distinct column\tag from (select /*+ para_tables_sort() */calc form stable where <\>\in\and\or order by limit offset ) + tdSql.query("select /*+ para_tables_sort() */5-2 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ distinct c5_1 " + sql += " from ( select /*+ para_tables_sort() */ " + sql += "%s " % random.choice(self.calc_select_in_ts) + sql += " as c5_1 from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += ") ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + #6-error select /*+ para_tables_sort() */* from (select /*+ para_tables_sort() */distinct(tag) form stable where <\>\in\and\or order by limit ) + tdSql.query("select /*+ para_tables_sort() */6-1 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */ " + sql += "%s " % random.choice(self.dt_select) + sql += " from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += "%s " % random.choice(self.order_desc_where) + sql += ") ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.error(sql) + tdSql.query("select /*+ para_tables_sort() */6-1 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */ " + sql += "%s " % random.choice(self.dt_select) + sql += " from stable_1 where " + sql += "%s ) ;" % random.choice(self.qt_where) + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + #7-error select /*+ para_tables_sort() */* from (select /*+ para_tables_sort() */distinct(tag) form stable where <\>\in\and\or order by limit ) + tdSql.query("select /*+ para_tables_sort() */7-1 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */ " + sql += "%s " % random.choice(self.dq_select) + sql += " from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += "%s " % random.choice(self.order_desc_where) + sql += "%s " % random.choice([self.limit_where[0] , self.limit_where[1]] ) + sql += ") ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.error(sql) + tdSql.query("select /*+ para_tables_sort() */7-1 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */ " + sql += "%s " % random.choice(self.dq_select) + sql += " from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += "%s " % random.choice([self.limit_where[0] , self.limit_where[1]] ) + sql += ") ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + tdSql.checkRows(1) + self.cur1.execute(sql) + self.explain_sql(sql) + + #calc_select,TWA/Diff/Derivative/Irate are not allowed to apply to super table directly + #8 select /*+ para_tables_sort() */* from (select /*+ para_tables_sort() */ts,calc form ragular_table where <\>\in\and\or order by ) + + # dcDB = self.dropandcreateDB_random("%s" %db, 1) + tdSql.query("select /*+ para_tables_sort() */8-1 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */ ts ," + sql += "%s " % random.choice(self.calc_select_support_ts) + sql += "from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ") ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + tdSql.query("select /*+ para_tables_sort() */8-1 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */ " + sql += "%s " % random.choice(self.calc_select_not_support_ts) + sql += "from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ") ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */ " + sql += "%s " % random.choice(self.calc_select_in_ts) + sql += "from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ") ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */8-2 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */t1.ts, " + sql += "%s " % random.choice(self.calc_select_in_support_ts_j) + sql += "from regular_table_1 t1, regular_table_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_where) + sql += "%s " % random.choice(self.order_u_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ") ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */ " + sql += "%s " % random.choice(self.calc_select_in_not_support_ts_j) + sql += "from regular_table_1 t1, regular_table_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ") ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + #9 select /*+ para_tables_sort() */* from (select /*+ para_tables_sort() */ts,calc form stable where <\>\in\and\or order by ) + # self.dropandcreateDB_random("%s" %db, 1) + tdSql.query("select /*+ para_tables_sort() */9-1 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */ " + sql += "%s " % random.choice(self.calc_select_not_support_ts) + sql += "from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ") ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + tdSql.query("select /*+ para_tables_sort() */9-2 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */ ts ," + sql += "%s " % random.choice(self.calc_select_support_ts) + sql += "from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += "%s " % random.choice(self.order_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ") ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */9-3 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */ " + sql += "%s " % random.choice(self.calc_select_in_not_support_ts_j) + sql += "from stable_1 t1, stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.t_join_where) + sql += " and %s " % random.choice(self.qt_u_or_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ") ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + tdSql.query("select /*+ para_tables_sort() */9-4 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */ t1.ts," + sql += "%s " % random.choice(self.calc_select_in_support_ts_j) + sql += "from stable_1 t1, stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.t_join_where) + sql += " and %s " % random.choice(self.qt_u_or_where) + sql += "%s " % random.choice(self.order_u_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ") ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + #10 select /*+ para_tables_sort() */calc from (select /*+ para_tables_sort() */* form regualr_table where <\>\in\and\or order by ) + tdSql.query("select /*+ para_tables_sort() */10-1 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ " + sql += "%s " % random.choice(self.calc_select_in_ts) + sql += "as calc10_1 from ( select /*+ para_tables_sort() */* from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_desc_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ") ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + #10-1 select /*+ para_tables_sort() */calc from (select /*+ para_tables_sort() */* form regualr_table where <\>\in\and\or order by ) + # rsDn = self.restartDnodes() + # self.dropandcreateDB_random("%s" %db, 1) + # rsDn = self.restartDnodes() + tdSql.query("select /*+ para_tables_sort() */10-2 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ " + sql += "%s " % random.choice(self.calc_select_all) + sql += "as calc10_2 from ( select /*+ para_tables_sort() */* from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_desc_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ") ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + #10-2 select /*+ para_tables_sort() */calc from (select /*+ para_tables_sort() */* form regualr_tables where <\>\in\and\or order by ) + tdSql.query("select /*+ para_tables_sort() */10-3 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ " + sql += "count(*) as calc10_3 " + sql += " from ( select /*+ para_tables_sort() */t1.ts as t11, t2.ts as t22 from regular_table_1 t1, regular_table_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_where) + sql += " and %s " % random.choice(self.q_u_or_where) + sql += "%s " % random.choice(self.order_u_where) + sql += "%s " % random.choice(self.limit_u_where) + sql += ") " + sql += "%s ;" % random.choice(self.limit1_where) + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */10-4 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ " + sql += "%s as calc10_4 " % random.choice(self.calc_select_all) + sql += " from ( select /*+ para_tables_sort() */* from regular_table_1 t1, regular_table_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_or_where) + sql += " and %s " % random.choice(self.q_u_or_where) + sql += "%s " % random.choice(self.order_u_where) + sql += "%s " % random.choice(self.limit_u_where) + sql += ") " + sql += "%s ;" % random.choice(self.limit_u_where) + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.error(sql) + + #11 select /*+ para_tables_sort() */calc from (select /*+ para_tables_sort() */* form stable where <\>\in\and\or order by limit ) + tdSql.query("select /*+ para_tables_sort() */11-1 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ " + sql += "%s " % random.choice(self.calc_select_in_ts) + sql += "as calc11_1 from ( select /*+ para_tables_sort() */* from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += "%s " % random.choice(self.order_desc_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ") ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + #11-1 select /*+ para_tables_sort() */calc from (select /*+ para_tables_sort() */* form stable where <\>\in\and\or order by limit ) + tdSql.query("select /*+ para_tables_sort() */11-2 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ " + sql += "%s " % random.choice(self.calc_select_all) + sql += "as calc11_1 from ( select /*+ para_tables_sort() */* from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += "%s " % random.choice(self.order_desc_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ") ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + #11-2 select /*+ para_tables_sort() */calc from (select /*+ para_tables_sort() */* form stables where <\>\in\and\or order by limit ) + tdSql.query("select /*+ para_tables_sort() */11-3 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ " + sql += "%s " % random.choice(self.calc_select_all) + sql += "as calc11_1 from ( select /*+ para_tables_sort() */* from stable_1 t1, stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.t_join_where) + sql += "%s " % random.choice(self.order_u_where) + sql += "%s " % random.choice(self.limit_u_where) + sql += ") " + sql += "%s ;" % random.choice(self.limit_u_where) + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.error(sql) + + tdSql.query("select /*+ para_tables_sort() */11-4 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ " + sql += "%s " % random.choice(self.calc_select_all) + sql += "as calc11_1 from ( select /*+ para_tables_sort() */* from stable_1 t1, stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.qt_u_or_where) + sql += "%s " % random.choice(self.order_u_where) + sql += "%s " % random.choice(self.limit_u_where) + sql += ") " + sql += "%s ;" % random.choice(self.limit_u_where) + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.error(sql) + + #12 select /*+ para_tables_sort() */calc-diff from (select /*+ para_tables_sort() */* form regualr_table where <\>\in\and\or order by limit ) + ##self.dropandcreateDB_random("%s" %db, 1) + tdSql.query("select /*+ para_tables_sort() */12-1 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ " + sql += "%s " % random.choice(self.calc_calculate_regular) + sql += " from ( select /*+ para_tables_sort() */* from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.order_desc_where) + sql += "%s " % random.choice([self.limit_where[2] , self.limit_where[3]] ) + sql += ") ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */12-2 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ " + sql += "%s " % random.choice(self.calc_calculate_regular) + sql += " from ( select /*+ para_tables_sort() */* from regular_table_1 t1, regular_table_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_where) + sql += "%s " % random.choice(self.order_u_where) + sql += "%s " % random.choice([self.limit_where[2] , self.limit_where[3]] ) + sql += ") ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.error(sql) + + tdSql.query("select /*+ para_tables_sort() */12-2.2 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ " + sql += "%s " % random.choice(self.calc_calculate_regular) + sql += " from ( select /*+ para_tables_sort() */* from regular_table_1 t1, regular_table_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_or_where) + sql += "%s " % random.choice(self.order_u_where) + sql += "%s " % random.choice([self.limit_where[2] , self.limit_where[3]] ) + sql += ") ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.error(sql) + + #12-1 select /*+ para_tables_sort() */calc-diff from (select /*+ para_tables_sort() */* form stable where <\>\in\and\or order by limit ) + tdSql.query("select /*+ para_tables_sort() */12-3 from stable_1;") + self.restartDnodes() + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */" + sql += "%s " % random.choice(self.calc_calculate_regular) + sql += " from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.partiton_where) + sql += ") " + sql += "%s " % random.choice([self.limit_where[2] , self.limit_where[3]] ) + sql += " ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */12-4 from stable_1;") + #join query does not support group by + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */" + sql += "%s " % random.choice(self.calc_calculate_regular_j) + sql += " from stable_1 t1, stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.t_join_where) + sql += "%s " % random.choice(self.partiton_where_j) + sql += ") " + sql += "%s " % random.choice([self.limit_where[2] , self.limit_where[3]] ) + sql += " ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */12-5 from stable_1;") + #join query does not support group by + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */" + sql += "%s " % random.choice(self.calc_calculate_regular_j) + sql += " from stable_1 t1, stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.qt_u_or_where) + sql += ") " + sql += "%s " % random.choice([self.limit_where[2] , self.limit_where[3]] ) + sql += " ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + + #13 select /*+ para_tables_sort() */calc-diff as diffns from (select /*+ para_tables_sort() */* form stable where <\>\in\and\or order by limit ) + tdSql.query("select /*+ para_tables_sort() */13-1 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ " + sql += "%s " % random.choice(self.calc_calculate_regular) + sql += " as calc13_1 from ( select /*+ para_tables_sort() */* from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += "%s " % random.choice(self.orders_desc_where) + sql += "%s " % random.choice([self.limit_where[2] , self.limit_where[3]] ) + sql += ") ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + #14 select /*+ para_tables_sort() */* from (select /*+ para_tables_sort() */calc_aggregate_alls as agg from stable where <\>\in\and\or group by order by slimit soffset ) + tdSql.query("select /*+ para_tables_sort() */14-1 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */" + sql += "%s as calc14_1, " % random.choice(self.calc_aggregate_all) + sql += "%s as calc14_2, " % random.choice(self.calc_aggregate_all) + sql += "%s " % random.choice(self.calc_aggregate_all) + sql += " as calc14_3 from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += "%s " % random.choice(self.group_where) + sql += ") ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + # error group by in out query + tdSql.query("select /*+ para_tables_sort() */14-2 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */" + sql += "%s as calc14_1, " % random.choice(self.calc_aggregate_all) + sql += "%s as calc14_2, " % random.choice(self.calc_aggregate_all) + sql += "%s " % random.choice(self.calc_aggregate_all) + sql += " as calc14_3 from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += "%s " % random.choice(self.partiton_where_regular) + sql += "%s " % random.choice(self.slimit1_where) + sql += ") " + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + #14-2 select /*+ para_tables_sort() */* from (select /*+ para_tables_sort() */calc_aggregate_all_js as agg from stables where <\>\in\and\or group by order by slimit soffset ) + tdSql.query("select /*+ para_tables_sort() */14-3 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */" + sql += "%s as calc14_1, " % random.choice(self.calc_aggregate_all_j) + sql += "%s as calc14_2, " % random.choice(self.calc_aggregate_all_j) + sql += "%s " % random.choice(self.calc_aggregate_all_j) + sql += " as calc14_3 from stable_1 t1, stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.t_join_where) + sql += "%s " % random.choice(self.partiton_where_j) + sql += "%s " % random.choice(self.slimit1_where) + sql += ") " + sql += "%s ;" % random.choice(self.limit_u_where) + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */14-4 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */" + sql += "%s as calc14_1, " % random.choice(self.calc_aggregate_all_j) + sql += "%s as calc14_2, " % random.choice(self.calc_aggregate_all_j) + sql += "%s " % random.choice(self.calc_aggregate_all_j) + sql += " as calc14_3 from stable_1 t1, stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.qt_u_or_where) + sql += "%s " % random.choice(self.partiton_where_j) + sql += "%s " % random.choice(self.slimit1_where) + sql += ") " + sql += "%s ;" % random.choice(self.limit_u_where) + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + #15 select /*+ para_tables_sort() */* from (select /*+ para_tables_sort() */calc_aggregate_regulars as agg from regular_table where <\>\in\and\or order by slimit soffset ) + tdSql.query("select /*+ para_tables_sort() */15-1 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */" + sql += "%s as calc15_1, " % random.choice(self.calc_aggregate_regular) + sql += "%s as calc15_2, " % random.choice(self.calc_aggregate_regular) + sql += "%s " % random.choice(self.calc_aggregate_regular) + sql += " as calc15_3 from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.group_where_regular) + sql += ") ;" + tdLog.info(sql) + tdLog.info(len(sql)) + self.data_check(sql,mark='15-1') + + tdSql.query("select /*+ para_tables_sort() */15-2 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */" + sql += "%s as calc15_2 " % random.choice(self.calc_aggregate_regular_j) + sql += "from regular_table_1 t1, regular_table_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_where) + sql += "%s " % random.choice(self.group_where_regular_j) + sql += "%s " % random.choice(self.limit_u_where) + sql += ") " + sql += "%s ;" % random.choice(self.limit_u_where) + tdLog.info(sql) + tdLog.info(len(sql)) + self.data_check(sql,mark='15-2') + + tdSql.query("select /*+ para_tables_sort() */15-2.2 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */" + sql += "%s as calc15_1, " % random.choice(self.calc_aggregate_regular_j) + sql += "%s as calc15_2, " % random.choice(self.calc_aggregate_regular_j) + sql += "%s " % random.choice(self.calc_aggregate_regular_j) + sql += " as calc15_3 from regular_table_1 t1, regular_table_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_or_where) + sql += "%s " % random.choice(self.group_where_regular_j) + sql += "%s " % random.choice(self.limit_u_where) + sql += ") " + sql += "%s ;" % random.choice(self.limit_u_where) + tdLog.info(sql) + tdLog.info(len(sql)) + self.data_check(sql,mark='15-2.2') + + self.restartDnodes() + tdSql.query("select /*+ para_tables_sort() */15-3 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */" + sql += "%s as calc15_1, " % random.choice(self.calc_aggregate_groupbytbname) + sql += "%s as calc15_2, " % random.choice(self.calc_aggregate_groupbytbname) + sql += "%s " % random.choice(self.calc_aggregate_groupbytbname) + sql += " as calc15_3 from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.group_only_where) + sql += "%s " % random.choice(self.having_support) + sql += ") " + sql += "order by calc15_1 " + sql += "%s " % random.choice(self.limit_where) + tdLog.info(sql) + tdLog.info(len(sql)) + self.data_check(sql,mark='15-3') + + tdSql.query("select /*+ para_tables_sort() */15-4 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */" + sql += "%s as calc15_1, " % random.choice(self.calc_aggregate_groupbytbname_j) + sql += "%s as calc15_2, " % random.choice(self.calc_aggregate_groupbytbname_j) + sql += "%s " % random.choice(self.calc_aggregate_groupbytbname_j) + sql += " as calc15_3 from stable_1 t1, stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.t_join_where) + sql += "%s " % random.choice(self.group_only_where_j) + sql += "%s " % random.choice(self.having_support_j) + sql += ") " + sql += "order by calc15_1 " + sql += "%s " % random.choice(self.limit_u_where) + tdLog.info(sql) + tdLog.info(len(sql)) + self.data_check(sql,mark='15-4') + + tdSql.query("select /*+ para_tables_sort() */15-4.2 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */" + sql += "%s as calc15_1, " % random.choice(self.calc_aggregate_groupbytbname_j) + sql += "%s as calc15_2, " % random.choice(self.calc_aggregate_groupbytbname_j) + sql += "%s " % random.choice(self.calc_aggregate_groupbytbname_j) + sql += " as calc15_3 from stable_1 t1, stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.qt_u_or_where) + sql += "%s " % random.choice(self.group_where_j) + sql += ") " + sql += "order by calc15_1 " + sql += "%s " % random.choice(self.limit_u_where) + tdLog.info(sql) + tdLog.info(len(sql)) + self.data_check(sql,mark='15-4.2') + + tdSql.query("select /*+ para_tables_sort() */15-5 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */" + sql += "%s as calc15_1, " % random.choice(self.calc_aggregate_groupbytbname) + sql += "%s as calc15_2, " % random.choice(self.calc_aggregate_groupbytbname) + sql += "%s " % random.choice(self.calc_aggregate_groupbytbname) + sql += " as calc15_3 from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.group_where) + sql += ") " + sql += "order by calc15_1 " + sql += "%s " % random.choice(self.limit_where) + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.data_check(sql,mark='15-5') + + #16 select /*+ para_tables_sort() */* from (select /*+ para_tables_sort() */calc_aggregate_regulars as agg from regular_table where <\>\in\and\or order by limit offset ) + #self.dropandcreateDB_random("%s" %db, 1) + tdSql.query("select /*+ para_tables_sort() */16-1 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */ " + sql += "%s as calc16_0 , " % random.choice(self.calc_calculate_all) + sql += "%s as calc16_1 , " % random.choice(self.calc_aggregate_all) + sql += "%s as calc16_2 " % random.choice(self.calc_select_in) + sql += " from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.group_where) + sql += ") " + sql += "order by calc16_0 " + sql += "%s " % random.choice(self.limit1_where) + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */16-2 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */ " + sql += "%s as calc16_0 " % random.choice(self.calc_calculate_all_j) + sql += ", %s as calc16_1 " % random.choice(self.calc_aggregate_all_j) + sql += " from stable_1 t1, stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.t_join_where) + sql += ") " + sql += "order by calc16_0 " + sql += "%s " % random.choice(self.limit1_where) + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */16-2.2 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */ " + sql += "%s as calc16_0 " % random.choice(self.calc_calculate_all_j) + sql += ", %s as calc16_1 " % random.choice(self.calc_aggregate_all_j) + sql += " from stable_1 t1, stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.qt_u_or_where) + sql += ") " + sql += "order by calc16_0 " + sql += "%s " % random.choice(self.limit1_where) + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */16-3 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */ " + sql += "%s as calc16_1 " % random.choice(self.calc_calculate_regular) + sql += " from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "limit 2 ) " + sql += "%s " % random.choice(self.limit1_where) + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */16-4 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */ " + sql += "%s as calc16_1 " % random.choice(self.calc_calculate_regular_j) + sql += " from regular_table_1 t1, regular_table_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_where) + sql += "limit 2 ) " + sql += "%s " % random.choice(self.limit1_where) + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */16-4.2 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */ " + sql += "%s as calc16_1 " % random.choice(self.calc_calculate_regular_j) + sql += " from regular_table_1 t1, regular_table_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_or_where) + sql += "limit 2 ) " + sql += "%s " % random.choice(self.limit1_where) + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */16-5 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */ " + sql += "%s as calc16_1 , " % random.choice(self.calc_calculate_all) + sql += "%s as calc16_2 , " % random.choice(self.calc_calculate_all) + sql += "%s as calc16_3 " % random.choice(self.calc_calculate_all) + sql += " from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.group_where) + sql += ") " + sql += "order by calc16_1 " + sql += "%s " % random.choice(self.limit1_where) + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */16-6 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */ " + sql += "%s as calc16_1 " % random.choice(self.calc_calculate_groupbytbname) + sql += " from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.partiton_where) + sql += "limit 2 ) " + sql += "%s " % random.choice(self.limit1_where) + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */16-7 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */ " + sql += "%s as calc16_1 " % random.choice(self.calc_calculate_groupbytbname_j) + sql += " from stable_1 t1, stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.t_join_where) + sql += "limit 2 ) " + sql += "%s " % random.choice(self.limit1_where) + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */16-8 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */ " + sql += "%s as calc16_1 " % random.choice(self.calc_calculate_groupbytbname_j) + sql += " from stable_1 t1, stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.qt_u_or_where) + sql += "limit 2 ) " + sql += "%s " % random.choice(self.limit1_where) + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + #17 select /*+ para_tables_sort() */apercentile from (select /*+ para_tables_sort() */calc_aggregate_alls form regualr_table or stable where <\>\in\and\or interval_sliding group by having order by limit offset )interval_sliding + #self.dropandcreateDB_random("%s" %db, 1) + tdSql.query("select /*+ para_tables_sort() */17-1 from stable_1;") + for i in range(self.fornum): + #this is having_support , but tag-select /*+ para_tables_sort() */cannot mix with last_row,other select /*+ para_tables_sort() */can + sql = "select /*+ para_tables_sort() */ apercentile(cal17_0, %d)/10 ,apercentile(cal17_1, %d)/1000 ,apercentile(cal17_2, %d)*10+%d from ( select /*+ para_tables_sort() */ " %(random.randint(0,100) , random.randint(0,100) , random.randint(0,100) ,random.randint(-1000,1000)) + sql += "%s as cal17_0 , " % random.choice(self.calc_calculate_all) + sql += "%s as cal17_1 ," % random.choice(self.calc_aggregate_all) + sql += "%s as cal17_2 " % random.choice(self.calc_aggregate_all) + sql += " from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += "%s " % random.choice(self.partiton_where) + sql += "%s " % random.choice(self.interval_sliding) + sql += "%s " % random.choice(self.limit1_where) + sql += ") " + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */17-2 from stable_1;") + for i in range(self.fornum): + #this is having_support , but tag-select /*+ para_tables_sort() */cannot mix with last_row,other select /*+ para_tables_sort() */can + sql = "select /*+ para_tables_sort() */ apercentile(cal17_0, %d)/10 ,apercentile(cal17_1, %d)/1000 ,apercentile(cal17_2, %d)*10+%d from ( select /*+ para_tables_sort() */ " %(random.randint(0,100) , random.randint(0,100) , random.randint(0,100) ,random.randint(-1000,1000)) + sql += "%s as cal17_0 , " % random.choice(self.calc_calculate_all_j) + sql += "%s as cal17_1 ," % random.choice(self.calc_aggregate_all_j) + sql += "%s as cal17_2 " % random.choice(self.calc_aggregate_all_j) + sql += " from stable_1 t1, stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.t_join_where) + sql += "%s " % random.choice(self.interval_sliding) + sql += "%s " % random.choice(self.limit_u_where) + sql += ") " + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */17-2.2 from stable_1;") + for i in range(self.fornum): + #this is having_support , but tag-select /*+ para_tables_sort() */cannot mix with last_row,other select /*+ para_tables_sort() */can + sql = "select /*+ para_tables_sort() */ apercentile(cal17_0, %d)/10 ,apercentile(cal17_1, %d)/1000 ,apercentile(cal17_2, %d)*10+%d from ( select /*+ para_tables_sort() */ " %(random.randint(0,100) , random.randint(0,100) , random.randint(0,100) ,random.randint(-1000,1000)) + sql += "%s as cal17_0 , " % random.choice(self.calc_calculate_all_j) + sql += "%s as cal17_1 ," % random.choice(self.calc_aggregate_all_j) + sql += "%s as cal17_2 " % random.choice(self.calc_aggregate_all_j) + sql += " from stable_1 t1, stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.qt_u_or_where) + sql += "%s " % random.choice(self.interval_sliding) + sql += "%s " % random.choice(self.limit_u_where) + sql += ") " + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + self.restartDnodes() + tdSql.query("select /*+ para_tables_sort() */17-3 from stable_1;") + for i in range(self.fornum): + #this is having_tagnot_support , because tag-select /*+ para_tables_sort() */cannot mix with last_row... + sql = "select /*+ para_tables_sort() */ apercentile(cal17_1, %d)/1000 ,apercentile(cal17_2, %d)*10+%d from ( select /*+ para_tables_sort() */ " %(random.randint(0,100) , random.randint(0,100) ,random.randint(-1000,1000)) + sql += "%s as cal17_1 ," % random.choice(self.calc_aggregate_all) + sql += "%s as cal17_2 " % random.choice(self.calc_aggregate_all) + sql += " from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.partiton_where) + sql += "%s " % random.choice(self.interval_sliding) + sql += "%s " % random.choice(self.limit1_where) + sql += ") " + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */17-4 from stable_1;") + for i in range(self.fornum): + #this is having_tagnot_support , because tag-select /*+ para_tables_sort() */cannot mix with last_row... + sql = "select /*+ para_tables_sort() */ apercentile(cal17_1, %d)/1000 ,apercentile(cal17_2, %d)*10+%d from ( select /*+ para_tables_sort() */ " %(random.randint(0,100) , random.randint(0,100) ,random.randint(-1000,1000)) + sql += "%s as cal17_1 ," % random.choice(self.calc_aggregate_all_j) + sql += "%s as cal17_2 " % random.choice(self.calc_aggregate_all_j) + sql += " from stable_1 t1, stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.t_join_where) + sql += "%s " % random.choice(self.interval_sliding) + sql += "%s " % random.choice(self.limit_u_where) + sql += ") " + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */17-4.2 from stable_1;") + for i in range(self.fornum): + #this is having_tagnot_support , because tag-select /*+ para_tables_sort() */cannot mix with last_row... + sql = "select /*+ para_tables_sort() */ apercentile(cal17_1, %d)/1000 ,apercentile(cal17_2, %d)*10+%d from ( select /*+ para_tables_sort() */ " %(random.randint(0,100) , random.randint(0,100) ,random.randint(-1000,1000)) + sql += "%s as cal17_1 ," % random.choice(self.calc_aggregate_all_j) + sql += "%s as cal17_2 " % random.choice(self.calc_aggregate_all_j) + sql += " from stable_1 t1, stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.qt_u_or_where) + sql += "%s " % random.choice(self.interval_sliding) + sql += "%s " % random.choice(self.limit_u_where) + sql += ") " + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */17-5 from stable_1;") + for i in range(self.fornum): + #having_not_support + sql = "select /*+ para_tables_sort() */ apercentile(cal17_1, %d)/1000 ,apercentile(cal17_2, %d)*10+%d from ( select /*+ para_tables_sort() */ " %(random.randint(0,100) , random.randint(0,100) ,random.randint(-1000,1000)) + sql += "%s as cal17_1 ," % random.choice(self.calc_aggregate_all) + sql += "%s as cal17_2 " % random.choice(self.calc_aggregate_all) + sql += " from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += "%s " % random.choice(self.partiton_where) + sql += "%s " % random.choice(self.interval_sliding) + sql += "%s " % random.choice(self.limit1_where) + sql += ") " + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */17-6 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ apercentile(cal17_1, %d)/1000 ,apercentile(cal17_2, %d)*10+%d from ( select /*+ para_tables_sort() */ " %(random.randint(0,100) , random.randint(0,100) ,random.randint(-1000,1000)) + sql += "%s as cal17_1 ," % random.choice(self.calc_aggregate_all) + sql += "%s as cal17_2 " % random.choice(self.calc_aggregate_all) + sql += " from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.interval_sliding) + sql += "%s " % random.choice(self.limit1_where) + sql += ") " + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */17-7 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ apercentile(cal17_1, %d)/1000 ,apercentile(cal17_2, %d)*10+%d from ( select /*+ para_tables_sort() */ " %(random.randint(0,100) , random.randint(0,100) ,random.randint(-1000,1000)) + sql += "%s as cal17_1 ," % random.choice(self.calc_aggregate_all_j) + sql += "%s as cal17_2 " % random.choice(self.calc_aggregate_all_j) + sql += " from stable_1_1 t1, stable_1_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_where) + sql += "%s " % random.choice(self.interval_sliding) + sql += "%s " % random.choice(self.limit1_where) + sql += ") " + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */17-7.2 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ apercentile(cal17_1, %d)/1000 ,apercentile(cal17_2, %d)*10+%d from ( select /*+ para_tables_sort() */ " %(random.randint(0,100) , random.randint(0,100) ,random.randint(-1000,1000)) + sql += "%s as cal17_1 ," % random.choice(self.calc_aggregate_all_j) + sql += "%s as cal17_2 " % random.choice(self.calc_aggregate_all_j) + sql += " from stable_1_1 t1, stable_1_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_or_where) + sql += "%s " % random.choice(self.interval_sliding) + sql += "%s " % random.choice(self.limit1_where) + sql += ") " + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + self.restartDnodes() + tdSql.query("select /*+ para_tables_sort() */17-8 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ apercentile(cal17_1, %d)/1000 ,apercentile(cal17_2, %d)*10+%d from ( select /*+ para_tables_sort() */ " %(random.randint(0,100) , random.randint(0,100) ,random.randint(-1000,1000)) + sql += "%s as cal17_1 ," % random.choice(self.calc_aggregate_all) + sql += "%s as cal17_2 " % random.choice(self.calc_aggregate_all) + sql += " from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.interval_sliding) + sql += "%s " % random.choice(self.limit1_where) + sql += ") " + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */17-9 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ apercentile(cal17_1, %d)/1000 ,apercentile(cal17_2, %d)*10+%d from ( select /*+ para_tables_sort() */ " %(random.randint(0,100) , random.randint(0,100) ,random.randint(-1000,1000)) + sql += "%s as cal17_1 ," % random.choice(self.calc_aggregate_all_j) + sql += "%s as cal17_2 " % random.choice(self.calc_aggregate_all_j) + sql += " from regular_table_1 t1, regular_table_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_where) + sql += "%s " % random.choice(self.interval_sliding) + sql += "%s " % random.choice(self.limit_u_where) + sql += ") " + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */17-10 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ apercentile(cal17_1, %d)/1000 ,apercentile(cal17_2, %d)*10+%d from ( select /*+ para_tables_sort() */ " %(random.randint(0,100) , random.randint(0,100) ,random.randint(-1000,1000)) + sql += "%s as cal17_1 ," % random.choice(self.calc_aggregate_all_j) + sql += "%s as cal17_2 " % random.choice(self.calc_aggregate_all_j) + sql += " from regular_table_1 t1, regular_table_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_or_where) + sql += "%s " % random.choice(self.interval_sliding) + sql += "%s " % random.choice(self.limit_u_where) + sql += ") " + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + #18 select /*+ para_tables_sort() */apercentile from (select /*+ para_tables_sort() */calc_aggregate_alls form regualr_table or stable where <\>\in\and\or session order by limit )interval_sliding + tdSql.query("select /*+ para_tables_sort() */18-1 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ apercentile(cal18_1, %d)/1000 ,apercentile(cal18_2, %d)*10+%d from ( select /*+ para_tables_sort() */ " %(random.randint(0,100) , random.randint(0,100) ,random.randint(-1000,1000)) + sql += "%s as cal18_1 ," % random.choice(self.calc_aggregate_all) + sql += "%s as cal18_2 " % random.choice(self.calc_aggregate_all) + sql += " from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.session_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ") " + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */18-2 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ apercentile(cal18_1, %d)/1000 ,apercentile(cal18_2, %d)*10+%d from ( select /*+ para_tables_sort() */ " %(random.randint(0,100) , random.randint(0,100) ,random.randint(-1000,1000)) + sql += "%s as cal18_1 ," % random.choice(self.calc_aggregate_all_j) + sql += "%s as cal18_2 " % random.choice(self.calc_aggregate_all_j) + sql += " from regular_table_1 t1, regular_table_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_where) + sql += "%s " % random.choice(self.session_u_where) + sql += "%s " % random.choice(self.limit_u_where) + sql += ") " + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */18-2.2 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ apercentile(cal18_1, %d)/1000 ,apercentile(cal18_2, %d)*10+%d from ( select /*+ para_tables_sort() */ " %(random.randint(0,100) , random.randint(0,100) ,random.randint(-1000,1000)) + sql += "%s as cal18_1 ," % random.choice(self.calc_aggregate_all_j) + sql += "%s as cal18_2 " % random.choice(self.calc_aggregate_all_j) + sql += " from regular_table_1 t1, regular_table_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_or_where) + sql += "%s " % random.choice(self.session_u_where) + sql += "%s " % random.choice(self.limit_u_where) + sql += ") " + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + self.restartDnodes() + tdSql.query("select /*+ para_tables_sort() */18-3 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ apercentile(cal18_1, %d)/1000 ,apercentile(cal18_2, %d)*10+%d from ( select /*+ para_tables_sort() */ " %(random.randint(0,100) , random.randint(0,100) ,random.randint(-1000,1000)) + sql += "%s as cal18_1 ," % random.choice(self.calc_aggregate_all) + sql += "%s as cal18_2 " % random.choice(self.calc_aggregate_all) + sql += " from stable_1_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.session_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ") " + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */18-4 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ apercentile(cal18_1, %d)/1000 ,apercentile(cal18_2, %d)*10+%d from ( select /*+ para_tables_sort() */ " %(random.randint(0,100) , random.randint(0,100) ,random.randint(-1000,1000)) + sql += "%s as cal18_1 ," % random.choice(self.calc_aggregate_all_j) + sql += "%s as cal18_2 " % random.choice(self.calc_aggregate_all_j) + sql += " from stable_1_1 t1, regular_table_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_where) + sql += "%s " % random.choice(self.session_u_where) + sql += "%s " % random.choice(self.limit_u_where) + sql += ") " + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */18-4.2 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ apercentile(cal18_1, %d)/1000 ,apercentile(cal18_2, %d)*10+%d from ( select /*+ para_tables_sort() */ " %(random.randint(0,100) , random.randint(0,100) ,random.randint(-1000,1000)) + sql += "%s as cal18_1 ," % random.choice(self.calc_aggregate_all_j) + sql += "%s as cal18_2 " % random.choice(self.calc_aggregate_all_j) + sql += " from stable_1_1 t1, regular_table_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_or_where) + sql += "%s " % random.choice(self.session_u_where) + sql += "%s " % random.choice(self.limit_u_where) + sql += ") " + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */18-5 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ apercentile(cal18_1, %d)/1000 ,apercentile(cal18_2, %d)*10+%d from ( select /*+ para_tables_sort() */ " %(random.randint(0,100) , random.randint(0,100) ,random.randint(-1000,1000)) + sql += "%s as cal18_1 ," % random.choice(self.calc_aggregate_all) + sql += "%s as cal18_2 " % random.choice(self.calc_aggregate_all) + sql += " from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.session_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ") " + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */18-6 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ apercentile(cal18_1, %d)/1000 ,apercentile(cal18_2, %d)*10+%d from ( select /*+ para_tables_sort() */ " %(random.randint(0,100) , random.randint(0,100) ,random.randint(-1000,1000)) + sql += "%s as cal18_1 ," % random.choice(self.calc_aggregate_all_j) + sql += "%s as cal18_2 " % random.choice(self.calc_aggregate_all_j) + sql += " from stable_1 t1, stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.t_join_where) + sql += "%s " % random.choice(self.session_u_where) + sql += "%s " % random.choice(self.limit_u_where) + sql += ") " + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */18-7 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ apercentile(cal18_1, %d)/1000 ,apercentile(cal18_2, %d)*10+%d from ( select /*+ para_tables_sort() */ " %(random.randint(0,100) , random.randint(0,100) ,random.randint(-1000,1000)) + sql += "%s as cal18_1 ," % random.choice(self.calc_aggregate_all_j) + sql += "%s as cal18_2 " % random.choice(self.calc_aggregate_all_j) + sql += " from stable_1 t1, stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.qt_u_or_where) + sql += "%s " % random.choice(self.session_u_where) + sql += "%s " % random.choice(self.limit_u_where) + sql += ") " + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + #19 select /*+ para_tables_sort() */apercentile from (select /*+ para_tables_sort() */calc_aggregate_alls form regualr_table or stable where <\>\in\and\or session order by limit )interval_sliding + #self.dropandcreateDB_random("%s" %db, 1) + tdSql.query("select /*+ para_tables_sort() */19-1 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ apercentile(cal19_1, %d)/1000 ,apercentile(cal19_2, %d)*10+%d from ( select /*+ para_tables_sort() */ " %(random.randint(0,100) , random.randint(0,100) ,random.randint(-1000,1000)) + sql += "%s as cal19_1 ," % random.choice(self.calc_aggregate_all) + sql += "%s as cal19_2 " % random.choice(self.calc_aggregate_all) + sql += " from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.state_window) + sql += "%s " % random.choice(self.limit1_where) + sql += ") " + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */19-2 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ apercentile(cal19_1, %d)/1000 ,apercentile(cal19_2, %d)*10+%d from ( select /*+ para_tables_sort() */ " %(random.randint(0,100) , random.randint(0,100) ,random.randint(-1000,1000)) + sql += "%s as cal19_1 ," % random.choice(self.calc_aggregate_all_j) + sql += "%s as cal19_2 " % random.choice(self.calc_aggregate_all_j) + sql += " from regular_table_1 t1, regular_table_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_where) + sql += "%s " % random.choice(self.state_u_window) + sql += "%s " % random.choice(self.limit_u_where) + sql += ") " + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */19-2.2 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ apercentile(cal19_1, %d)/1000 ,apercentile(cal19_2, %d)*10+%d from ( select /*+ para_tables_sort() */ " %(random.randint(0,100) , random.randint(0,100) ,random.randint(-1000,1000)) + sql += "%s as cal19_1 ," % random.choice(self.calc_aggregate_all_j) + sql += "%s as cal19_2 " % random.choice(self.calc_aggregate_all_j) + sql += " from regular_table_1 t1, regular_table_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_or_where) + sql += "%s " % random.choice(self.state_u_window) + sql += "%s " % random.choice(self.limit_u_where) + sql += ") " + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */19-3 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ apercentile(cal19_1, %d)/1000 ,apercentile(cal19_2, %d)*10+%d from ( select /*+ para_tables_sort() */ " %(random.randint(0,100) , random.randint(0,100) ,random.randint(-1000,1000)) + sql += "%s as cal19_1 ," % random.choice(self.calc_aggregate_all) + sql += "%s as cal19_2 " % random.choice(self.calc_aggregate_all) + sql += " from stable_1_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.state_window) + sql += "%s " % random.choice(self.limit1_where) + sql += ") " + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */19-4 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ apercentile(cal19_1, %d)/1000 ,apercentile(cal19_2, %d)*10+%d from ( select /*+ para_tables_sort() */ " %(random.randint(0,100) , random.randint(0,100) ,random.randint(-1000,1000)) + sql += "%s as cal19_1 ," % random.choice(self.calc_aggregate_all_j) + sql += "%s as cal19_2 " % random.choice(self.calc_aggregate_all_j) + sql += " from stable_1_1 t1, stable_1_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_where) + sql += "%s " % random.choice(self.limit_u_where) + sql += ") " + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */19-4.2 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ apercentile(cal19_1, %d)/1000 ,apercentile(cal19_2, %d)*10+%d from ( select /*+ para_tables_sort() */ " %(random.randint(0,100) , random.randint(0,100) ,random.randint(-1000,1000)) + sql += "%s as cal19_1 ," % random.choice(self.calc_aggregate_all_j) + sql += "%s as cal19_2 " % random.choice(self.calc_aggregate_all_j) + sql += " from stable_1_1 t1, stable_1_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_or_where) + sql += "%s " % random.choice(self.limit_u_where) + sql += ") " + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */19-5 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ apercentile(cal19_1, %d)/1000 ,apercentile(cal19_2, %d)*10+%d from ( select /*+ para_tables_sort() */ " %(random.randint(0,100) , random.randint(0,100) ,random.randint(-1000,1000)) + sql += "%s as cal19_1 ," % random.choice(self.calc_aggregate_all) + sql += "%s as cal19_2 " % random.choice(self.calc_aggregate_all) + sql += " from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += "%s " % random.choice(self.state_window) + sql += "%s " % random.choice(self.order_where) + sql += "%s " % random.choice(self.limit1_where) + sql += ") " + sql += "%s " % random.choice(self.interval_sliding) + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.error(sql) + + tdSql.query("select /*+ para_tables_sort() */19-6 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ apercentile(cal19_1, %d)/1000 ,apercentile(cal19_2, %d)*10+%d from ( select /*+ para_tables_sort() */ " %(random.randint(0,100) , random.randint(0,100) ,random.randint(-1000,1000)) + sql += "%s as cal19_1 ," % random.choice(self.calc_aggregate_all_j) + sql += "%s as cal19_2 " % random.choice(self.calc_aggregate_all_j) + sql += " from stable_1 t1 , stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.q_u_where) + sql += "%s " % random.choice(self.limit_u_where) + sql += ") " + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */19-7 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ apercentile(cal19_1, %d)/1000 ,apercentile(cal19_2, %d)*10+%d from ( select /*+ para_tables_sort() */ " %(random.randint(0,100) , random.randint(0,100) ,random.randint(-1000,1000)) + sql += "%s as cal19_1 ," % random.choice(self.calc_aggregate_all_j) + sql += "%s as cal19_2 " % random.choice(self.calc_aggregate_all_j) + sql += " from stable_1 t1 , stable_2 t2 where t1.ts = t2.ts and " + sql += "%s " % random.choice(self.qt_u_or_where) + sql += "%s " % random.choice(self.limit_u_where) + sql += ") " + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + #20 select /*+ para_tables_sort() */* from (select /*+ para_tables_sort() */calc_select_fills form regualr_table or stable where <\>\in\and\or fill_where group by order by limit offset ) + #self.dropandcreateDB_random("%s" %db, 1) + tdSql.query("select /*+ para_tables_sort() */20-1 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */ " + sql += "%s , " % random.choice(self.calc_select_fill) + sql += "%s ," % random.choice(self.calc_select_fill) + sql += "%s " % random.choice(self.calc_select_fill) + sql += " from stable_1 where " + sql += "%s " % random.choice(self.interp_where) + sql += "%s " % random.choice(self.fill_where) + sql += "%s " % random.choice(self.group_where) + sql += "%s " % random.choice(self.order_where) + sql += "%s " % random.choice(self.limit_where) + sql += ") " + tdLog.info(sql) + tdLog.info(len(sql)) + #interp不支持 tdSql.query(sql) + #self.cur1.execute(sql) + #self.explain_sql(sql) + + rsDn = self.restartDnodes() + tdSql.query("select /*+ para_tables_sort() */20-2 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */ " + sql += "%s , " % random.choice(self.calc_select_fill_j) + sql += "%s ," % random.choice(self.calc_select_fill_j) + sql += "%s " % random.choice(self.calc_select_fill_j) + sql += " from stable_1 t1 , stable_2 t2 where t1.ts = t2.ts and " + sql += "%s and " % random.choice(self.t_join_where) + sql += "%s " % random.choice(self.interp_where_j) + sql += "%s " % random.choice(self.order_u_where) + sql += "%s " % random.choice(self.limit_u_where) + sql += ") " + tdLog.info(sql) + tdLog.info(len(sql)) + #interp不支持 tdSql.query(sql) + #self.cur1.execute(sql) + #self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */20-2.2 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */ " + sql += "%s , " % random.choice(self.calc_select_fill_j) + sql += "%s ," % random.choice(self.calc_select_fill_j) + sql += "%s " % random.choice(self.calc_select_fill_j) + sql += " from stable_1 t1 , stable_2 t2 where t1.ts = t2.ts and " + sql += "%s and " % random.choice(self.qt_u_or_where) + sql += "%s " % random.choice(self.interp_where_j) + sql += "%s " % random.choice(self.order_u_where) + sql += "%s " % random.choice(self.limit_u_where) + sql += ") " + tdLog.info(sql) + tdLog.info(len(sql)) + #interp不支持 tdSql.query(sql) + #self.cur1.execute(sql) + #self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */20-3 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */ " + sql += "%s , " % random.choice(self.calc_select_fill) + sql += "%s ," % random.choice(self.calc_select_fill) + sql += "%s " % random.choice(self.calc_select_fill) + sql += " from stable_1 where " + sql += "%s " % self.interp_where[2] + sql += "%s " % random.choice(self.fill_where) + sql += "%s " % random.choice(self.order_where) + sql += "%s " % random.choice(self.limit_where) + sql += ") " + tdLog.info(sql) + tdLog.info(len(sql)) + #interp不支持 tdSql.query(sql) + #self.cur1.execute(sql) + #self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */20-4 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */ " + sql += "%s , " % random.choice(self.calc_select_fill_j) + sql += "%s ," % random.choice(self.calc_select_fill_j) + sql += "%s " % random.choice(self.calc_select_fill_j) + sql += " from stable_1 t1, table_1 t2 where t1.ts = t2.ts and " + #sql += "%s and " % random.choice(self.t_join_where) + sql += "%s " % self.interp_where_j[random.randint(0,5)] + sql += "%s " % random.choice(self.order_u_where) + sql += "%s " % random.choice(self.limit_u_where) + sql += ") " + tdLog.info(sql) + tdLog.info(len(sql)) + #interp不支持 tdSql.query(sql) + #self.cur1.execute(sql) + #self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */20-4.2 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */ " + sql += "%s , " % random.choice(self.calc_select_fill_j) + sql += "%s ," % random.choice(self.calc_select_fill_j) + sql += "%s " % random.choice(self.calc_select_fill_j) + sql += " from stable_1 t1, stable_1_1 t2 where t1.ts = t2.ts and " + sql += "%s and " % random.choice(self.qt_u_or_where) + sql += "%s " % self.interp_where_j[random.randint(0,5)] + sql += "%s " % random.choice(self.fill_where) + sql += "%s " % random.choice(self.order_u_where) + sql += "%s " % random.choice(self.limit_u_where) + sql += ") " + tdLog.info(sql) + tdLog.info(len(sql)) + ##interp不支持 tdSql.error(sql) + #self.cur1.execute(sql) + #self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */20-5 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */ " + sql += "%s , " % random.choice(self.calc_select_fill) + sql += "%s ," % random.choice(self.calc_select_fill) + sql += "%s " % random.choice(self.calc_select_fill) + sql += " from regular_table_1 where " + sql += "%s " % self.interp_where[1] + sql += "%s " % random.choice(self.fill_where) + sql += "%s " % random.choice(self.order_where) + sql += "%s " % random.choice(self.limit_where) + sql += ") " + tdLog.info(sql) + tdLog.info(len(sql)) + ##interp不支持 tdSql.query(sql) + #self.cur1.execute(sql) + #self.explain_sql(sql) + + tdSql.query("select /*+ para_tables_sort() */20-6 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */ " + sql += "%s , " % random.choice(self.calc_select_fill_j) + sql += "%s ," % random.choice(self.calc_select_fill_j) + sql += "%s " % random.choice(self.calc_select_fill_j) + sql += " from regular_table_1 t1, regular_table_2 t2 where t1.ts = t2.ts and " + #sql += "%s " % random.choice(self.interp_where_j) + sql += "%s " % self.interp_where_j[random.randint(0,5)] + sql += "%s " % random.choice(self.order_u_where) + sql += "%s " % random.choice(self.limit_u_where) + sql += ") " + tdLog.info(sql) + tdLog.info(len(sql)) + ##interp不支持 tdSql.query(sql) + #self.cur1.execute(sql) + #self.explain_sql(sql) + + #1 select /*+ para_tables_sort() */* from (select /*+ para_tables_sort() */* from (select /*+ para_tables_sort() */* form regular_table where <\>\in\and\or order by limit )) + tdSql.query("select /*+ para_tables_sort() */1-1 from stable_1;") + for i in range(self.fornum): + # sql_start = "select /*+ para_tables_sort() */ * from ( " + # sql_end = ")" + for_num = random.randint(1, 15); + sql = "select /*+ para_tables_sort() */ * from (" * for_num + sql += "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */* from ( select /*+ para_tables_sort() */ " + sql += "%s, " % random.choice(self.s_r_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts as ttt from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += ")) " + sql += ")" * for_num + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + sql2 = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */* from ( select /*+ para_tables_sort() */ " + sql2 += "%s, " % random.choice(self.s_r_select) + sql2 += "%s, " % random.choice(self.q_select) + sql2 += "ts as tin from regular_table_1 where " + sql2 += "%s " % random.choice(self.q_where) + sql2 += ")) " + tdLog.info(sql2) + tdLog.info(len(sql2)) + tdSql.query(sql2) + self.cur1.execute(sql2) + self.explain_sql(sql2) + + self.data_matrix_equal('%s' %sql ,1,10,1,1,'%s' %sql2 ,1,10,1,1) + self.data_matrix_equal('%s' %sql ,1,10,1,1,'%s' %sql ,1,10,3,3) + self.data_matrix_equal('%s' %sql ,1,10,3,3,'%s' %sql2 ,1,10,3,3) + + tdLog.info("=====1-1==over=========") + + for i in range(self.fornum): + for_num = random.randint(1, 15); + sql = "select /*+ para_tables_sort() */ ts2 from (" * for_num + sql += "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */* from ( select /*+ para_tables_sort() */ " + sql += "%s, " % random.choice(self.s_r_select) + sql += "%s, " % random.choice(self.q_select) + sql += "ts as ts2 from regular_table_1 where " + sql += "%s " % random.choice(self.q_where) + sql += ")) " + sql += ")" * for_num + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + sql2 = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */* from ( select /*+ para_tables_sort() */ " + sql2 += "%s, " % random.choice(self.s_r_select) + sql2 += "%s, " % random.choice(self.q_select) + sql2 += "ts as tt from regular_table_1 where " + sql2 += "%s " % random.choice(self.q_where) + sql2 += ")) " + tdLog.info(sql2) + tdLog.info(len(sql2)) + tdSql.query(sql2) + self.cur1.execute(sql2) + self.explain_sql(sql2) + + self.data_matrix_equal('%s' %sql ,1,10,1,1,'%s' %sql2 ,1,10,1,1) + tdLog.info("=====1-2==over=========") + + #2 select /*+ para_tables_sort() */* from (select /*+ para_tables_sort() */* from (select /*+ para_tables_sort() */* form stable where <\>\in\and\or order by limit )) + tdSql.query("select /*+ para_tables_sort() */2-1 from stable_1;") + for i in range(self.fornum): + for_num = random.randint(1, 15); + sql = "select /*+ para_tables_sort() */ * from (" * for_num + sql += "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */* from ( select /*+ para_tables_sort() */ " + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s, " % random.choice(self.qt_select) + sql += "ts as tss from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += ")) " + sql += ")" * for_num + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + sql2 = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */* from ( select /*+ para_tables_sort() */ " + sql2 += "%s, " % random.choice(self.s_s_select) + sql2 += "%s, " % random.choice(self.qt_select) + sql2 += "ts as tst from stable_1 where " + sql2 += "%s " % random.choice(self.q_where) + sql2 += ")) " + tdLog.info(sql2) + tdLog.info(len(sql2)) + tdSql.query(sql2) + self.cur1.execute(sql2) + self.explain_sql(sql2) + + self.data_matrix_equal('%s' %sql ,1,10,3,3,'%s' %sql2 ,1,10,3,3) + + tdLog.info("=====2-1==over=========") + + for i in range(self.fornum): + for_num = random.randint(1, 15); + sql = "select /*+ para_tables_sort() */ tsn from (" * for_num + sql += "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */* from ( select /*+ para_tables_sort() */ " + sql += "%s, " % random.choice(self.s_s_select) + sql += "%s, " % random.choice(self.qt_select) + sql += "ts as tsn from stable_1 where " + sql += "%s " % random.choice(self.q_where) + sql += ")) " + sql += ")" * for_num + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + sql2 = "select /*+ para_tables_sort() */ ts1 from ( select /*+ para_tables_sort() */* from ( select /*+ para_tables_sort() */ " + sql2 += "%s, " % random.choice(self.s_s_select) + sql2 += "%s, " % random.choice(self.qt_select) + sql2 += "ts as ts1 from stable_1 where " + sql2 += "%s " % random.choice(self.q_where) + sql2 += ")) " + tdLog.info(sql2) + tdLog.info(len(sql2)) + tdSql.query(sql2) + self.cur1.execute(sql2) + self.explain_sql(sql2) + + self.data_matrix_equal('%s' %sql ,1,10,1,1,'%s' %sql2 ,1,10,1,1) + tdLog.info("=====2-2==over=========") + + #3 select /*+ para_tables_sort() */ts ,calc from (select /*+ para_tables_sort() */* form stable where <\>\in\and\or order by limit ) + #self.dropandcreateDB_random("%s" %db, 1) + tdSql.query("select /*+ para_tables_sort() */3-1 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ " + sql += "%s " % random.choice(self.calc_calculate_regular) + sql += " from ( select /*+ para_tables_sort() */* from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += "%s " % random.choice(self.orders_desc_where) + sql += "%s " % random.choice(self.limit_where) + sql += ") ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + #4 select /*+ para_tables_sort() */* from (select /*+ para_tables_sort() */calc form stable where <\>\in\and\or order by limit ) + tdSql.query("select /*+ para_tables_sort() */4-1 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ * from ( select /*+ para_tables_sort() */ " + sql += "%s " % random.choice(self.calc_select_in_ts) + sql += "from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + #sql += "%s " % random.choice(self.order_desc_where) + sql += "%s " % random.choice(self.limit_where) + sql += ") ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + #5 select /*+ para_tables_sort() */ts ,tbname from (select /*+ para_tables_sort() */* form stable where <\>\in\and\or order by limit ) + tdSql.query("select /*+ para_tables_sort() */5-1 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */ ts , tbname , " + sql += "%s ," % random.choice(self.calc_calculate_regular) + sql += "%s ," % random.choice(self.dqt_select) + sql += "%s " % random.choice(self.qt_select) + sql += " from ( select /*+ para_tables_sort() */* from stable_1 where " + sql += "%s " % random.choice(self.qt_where) + sql += "%s " % random.choice(self.orders_desc_where) + sql += "%s " % random.choice(self.limit_where) + sql += ") ;" + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.error(sql) + + #6 + tdSql.query("select /*+ para_tables_sort() */6-1 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */count(*) from (select /*+ para_tables_sort() */avg(q_int)/1000 from stable_1); " + tdLog.info(sql) + tdLog.info(len(sql)) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + #special sql + tdSql.query("select /*+ para_tables_sort() */7-1 from stable_1;") + for i in range(self.fornum): + sql = "select /*+ para_tables_sort() */* from ( select /*+ para_tables_sort() */_block_dist() from stable_1);" + tdSql.error(sql) + sql = "select /*+ para_tables_sort() */_block_dist() from (select /*+ para_tables_sort() */* from stable_1);" + tdSql.error(sql) + sql = "select /*+ para_tables_sort() */* from (select /*+ para_tables_sort() */database());" + tdLog.info(sql) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + sql = "select /*+ para_tables_sort() */* from (select /*+ para_tables_sort() */client_version());" + tdLog.info(sql) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + sql = "select /*+ para_tables_sort() */* from (select /*+ para_tables_sort() */client_version() as version);" + tdLog.info(sql) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + sql = "select /*+ para_tables_sort() */* from (select /*+ para_tables_sort() */server_version());" + tdLog.info(sql) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + sql = "select /*+ para_tables_sort() */* from (select /*+ para_tables_sort() */server_version() as version);" + tdLog.info(sql) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + sql = "select /*+ para_tables_sort() */* from (select /*+ para_tables_sort() */server_status());" + tdLog.info(sql) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + sql = "select /*+ para_tables_sort() */* from (select /*+ para_tables_sort() */server_status() as status);" + tdLog.info(sql) + tdSql.query(sql) + self.cur1.execute(sql) + self.explain_sql(sql) + + #taos -f sql + startTime_taos_f = time.time() + print("taos -f sql start!") + taos_cmd1 = "taos -f %s/%s.sql" % (self.testcasePath,self.testcaseFilename) + #_ = subprocess.check_output(taos_cmd1, shell=True).decode("utf-8") + _ = subprocess.check_output(taos_cmd1, shell=True) + print("taos -f sql over!") + endTime_taos_f = time.time() + print("taos_f total time %ds" % (endTime_taos_f - startTime_taos_f)) + + print('=====================2.6 old function end ===========') + + + + def run(self): + tdSql.prepare() + + startTime = time.time() + + #self.function_before_26() + + self.math_nest(['UNIQUE']) + self.math_nest(['MODE']) + self.math_nest(['SAMPLE']) + + # self.math_nest(['ABS','SQRT']) + # self.math_nest(['SIN','COS','TAN','ASIN','ACOS','ATAN']) + # self.math_nest(['POW','LOG']) + # self.math_nest(['FLOOR','CEIL','ROUND']) + # self.math_nest(['MAVG']) + # self.math_nest(['HYPERLOGLOG']) + # self.math_nest(['TAIL']) + self.math_nest(['CSUM']) + self.math_nest(['statecount','stateduration']) + self.math_nest(['HISTOGRAM']) + + # self.str_nest(['LTRIM','RTRIM','LOWER','UPPER']) + # self.str_nest(['LENGTH','CHAR_LENGTH']) + # self.str_nest(['SUBSTR']) + # self.str_nest(['CONCAT']) + # self.str_nest(['CONCAT_WS']) + # self.time_nest(['CAST']) #放到time里起来弄 + # self.time_nest(['CAST_1']) + # self.time_nest(['CAST_2']) + # self.time_nest(['CAST_3']) + # self.time_nest(['CAST_4']) + + + + # self.time_nest(['NOW','TODAY']) + # self.time_nest(['TIMEZONE']) + # self.time_nest(['TIMETRUNCATE']) + # self.time_nest(['TO_ISO8601']) + # self.time_nest(['TO_UNIXTIMESTAMP']) + # self.time_nest(['ELAPSED']) + self.time_nest(['TIMEDIFF_1']) + self.time_nest(['TIMEDIFF_2']) + + + endTime = time.time() + print("total time %ds" % (endTime - startTime)) + + + + + def stop(self): + tdSql.close() + tdLog.success("%s successfully executed" % __file__) + + +tdCases.addWindows(__file__, TDTestCase()) +tdCases.addLinux(__file__, TDTestCase()) From b3c4fe489ff398ab188241c7356b44ac120234bb Mon Sep 17 00:00:00 2001 From: slzhou Date: Tue, 20 Feb 2024 13:22:35 +0800 Subject: [PATCH 26/41] feat: remove todo --- source/libs/executor/src/scanoperator.c | 6 ------ 1 file changed, 6 deletions(-) diff --git a/source/libs/executor/src/scanoperator.c b/source/libs/executor/src/scanoperator.c index dc9fceee84..cb83cb7599 100644 --- a/source/libs/executor/src/scanoperator.c +++ b/source/libs/executor/src/scanoperator.c @@ -3317,12 +3317,6 @@ _error: // table merge scan operator // table merge scan operator -// TODO: limit / duration optimization -// TODO: get block from tsdReader function, with task killed, func_data all filter out, skip, finish -// TODO: error processing, memory freeing -// TODO: add log for error and perf -// TODO: tsdb reader open/close dynamically -// TODO: blockdata deep cleanup static int32_t subTblRowCompareFn(const void* pLeft, const void* pRight, void* param) { int32_t left = *(int32_t*)pLeft; From af1a40a7fffc7feb0b4b9586cfcd3e391933a1bf Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Tue, 20 Feb 2024 12:09:30 +0000 Subject: [PATCH 27/41] fix duplicate uid --- source/libs/index/src/indexFilter.c | 26 ++++++++++++++++---------- 1 file changed, 16 insertions(+), 10 deletions(-) diff --git a/source/libs/index/src/indexFilter.c b/source/libs/index/src/indexFilter.c index 7ed36fbf9e..cb42e60c01 100644 --- a/source/libs/index/src/indexFilter.c +++ b/source/libs/index/src/indexFilter.c @@ -17,6 +17,7 @@ #include "index.h" #include "indexComm.h" #include "indexInt.h" +#include "indexUtil.h" #include "nodes.h" #include "querynodes.h" #include "scalar.h" @@ -77,15 +78,15 @@ typedef struct SIFParam { char dbName[TSDB_DB_NAME_LEN]; char colName[TSDB_COL_NAME_LEN * 2 + 4]; - SIndexMetaArg arg; + SIndexMetaArg arg; SMetaDataFilterAPI api; } SIFParam; typedef struct SIFCtx { - int32_t code; - SHashObj *pRes; /* element is SIFParam */ - bool noExec; // true: just iterate condition tree, and add hint to executor plan - SIndexMetaArg arg; + int32_t code; + SHashObj *pRes; /* element is SIFParam */ + bool noExec; // true: just iterate condition tree, and add hint to executor plan + SIndexMetaArg arg; SMetaDataFilterAPI *pAPI; } SIFCtx; @@ -669,6 +670,10 @@ static int32_t sifDoIndex(SIFParam *left, SIFParam *right, int8_t operType, SIFP if (sifSetFltParam(left, right, &typedata, ¶m) != 0) return -1; } ret = left->api.metaFilterTableIds(arg->metaEx, ¶m, output->result); + if (ret == 0) { + taosArraySort(output->result, uidCompare); + taosArrayRemoveDuplicate(output->result, uidCompare, NULL); + } } return ret; } @@ -875,8 +880,8 @@ static int32_t sifExecLogic(SLogicConditionNode *node, SIFCtx *ctx, SIFParam *ou } else if (node->condType == LOGIC_COND_TYPE_NOT) { // taosArrayAddAll(output->result, params[m].result); } - taosArraySort(output->result, idxUidCompare); - taosArrayRemoveDuplicate(output->result, idxUidCompare, NULL); + taosArraySort(output->result, uidCompare); + taosArrayRemoveDuplicate(output->result, uidCompare, NULL); } } else { for (int32_t m = 0; m < node->pParameterList->length; m++) { @@ -1016,7 +1021,7 @@ static int32_t sifCalculate(SNode *pNode, SIFParam *pDst) { return code; } -static int32_t sifGetFltHint(SNode *pNode, SIdxFltStatus *status, SMetaDataFilterAPI* pAPI) { +static int32_t sifGetFltHint(SNode *pNode, SIdxFltStatus *status, SMetaDataFilterAPI *pAPI) { int32_t code = TSDB_CODE_SUCCESS; if (pNode == NULL) { return TSDB_CODE_QRY_INVALID_INPUT; @@ -1054,7 +1059,8 @@ static int32_t sifGetFltHint(SNode *pNode, SIdxFltStatus *status, SMetaDataFilte return code; } -int32_t doFilterTag(SNode *pFilterNode, SIndexMetaArg *metaArg, SArray *result, SIdxFltStatus *status, SMetaDataFilterAPI* pAPI) { +int32_t doFilterTag(SNode *pFilterNode, SIndexMetaArg *metaArg, SArray *result, SIdxFltStatus *status, + SMetaDataFilterAPI *pAPI) { SIdxFltStatus st = idxGetFltStatus(pFilterNode, pAPI); if (st == SFLT_NOT_INDEX) { *status = st; @@ -1081,7 +1087,7 @@ int32_t doFilterTag(SNode *pFilterNode, SIndexMetaArg *metaArg, SArray *result, return TSDB_CODE_SUCCESS; } -SIdxFltStatus idxGetFltStatus(SNode *pFilterNode, SMetaDataFilterAPI* pAPI) { +SIdxFltStatus idxGetFltStatus(SNode *pFilterNode, SMetaDataFilterAPI *pAPI) { SIdxFltStatus st = SFLT_NOT_INDEX; if (pFilterNode == NULL) { return SFLT_NOT_INDEX; From 05c6c3815abe5173e1f0635904880f53ad44b3fc Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Tue, 20 Feb 2024 12:45:21 +0000 Subject: [PATCH 28/41] fix duplicate uid --- source/libs/index/src/index.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/libs/index/src/index.c b/source/libs/index/src/index.c index 3ed66956e8..f41b5525f3 100644 --- a/source/libs/index/src/index.c +++ b/source/libs/index/src/index.c @@ -459,7 +459,7 @@ static void idxInterRsltDestroy(SArray* results) { static int idxMergeFinalResults(SArray* in, EIndexOperatorType oType, SArray* out) { // refactor, merge interResults into fResults by oType - for (int i = 0; i < taosArrayGetSize(in); i--) { + for (int i = 0; i < taosArrayGetSize(in); i++) { SArray* t = taosArrayGetP(in, i); taosArraySort(t, uidCompare); taosArrayRemoveDuplicate(t, uidCompare, NULL); From 6b331594d9ff7151b73b8d9cdb6153765f120f5f Mon Sep 17 00:00:00 2001 From: slzhou Date: Wed, 21 Feb 2024 13:38:49 +0800 Subject: [PATCH 29/41] fix: change typo error --- source/libs/nodes/src/nodesCodeFuncs.c | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/source/libs/nodes/src/nodesCodeFuncs.c b/source/libs/nodes/src/nodesCodeFuncs.c index a9907295b9..689886c366 100644 --- a/source/libs/nodes/src/nodesCodeFuncs.c +++ b/source/libs/nodes/src/nodesCodeFuncs.c @@ -698,7 +698,7 @@ static const char* jkScanLogicPlanTagCond = "TagCond"; static const char* jkScanLogicPlanGroupTags = "GroupTags"; static const char* jkScanLogicPlanOnlyMetaCtbIdx = "OnlyMetaCtbIdx"; static const char* jkScanLogicPlanFilesetDelimited = "FilesetDelimited"; -static const char* jkScanLogicPlanparaTablesSort = "paraTablesSort"; +static const char* jkScanLogicPlanParaTablesSort = "ParaTablesSort"; static int32_t logicScanNodeToJson(const void* pObj, SJson* pJson) { const SScanLogicNode* pNode = (const SScanLogicNode*)pObj; @@ -747,7 +747,7 @@ static int32_t logicScanNodeToJson(const void* pObj, SJson* pJson) { code = tjsonAddBoolToObject(pJson, jkScanLogicPlanFilesetDelimited, pNode->filesetDelimited); } if (TSDB_CODE_SUCCESS == code) { - code = tjsonAddBoolToObject(pJson, jkScanLogicPlanFilesetDelimited, pNode->paraTablesSort); + code = tjsonAddBoolToObject(pJson, jkScanLogicPlanParaTablesSort, pNode->paraTablesSort); } return code; } @@ -800,7 +800,7 @@ static int32_t jsonToLogicScanNode(const SJson* pJson, void* pObj) { code = tjsonGetBoolValue(pJson, jkScanLogicPlanFilesetDelimited, &pNode->filesetDelimited); } if (TSDB_CODE_SUCCESS == code) { - code = tjsonGetBoolValue(pJson, jkScanLogicPlanFilesetDelimited, &pNode->paraTablesSort); + code = tjsonGetBoolValue(pJson, jkScanLogicPlanParaTablesSort, &pNode->paraTablesSort); } return code; } @@ -1895,7 +1895,7 @@ static const char* jkTableScanPhysiPlanAssignBlockUid = "AssignBlockUid"; static const char* jkTableScanPhysiPlanIgnoreUpdate = "IgnoreUpdate"; static const char* jkTableScanPhysiPlanFilesetDelimited = "FilesetDelimited"; static const char* jkTableScanPhysiPlanNeedCountEmptyTable = "NeedCountEmptyTable"; -static const char* jkTableScanPhysiPlanparaTablesSort = "paraTablesSort"; +static const char* jkTableScanPhysiPlanParaTablesSort = "ParaTablesSort"; static int32_t physiTableScanNodeToJson(const void* pObj, SJson* pJson) { const STableScanPhysiNode* pNode = (const STableScanPhysiNode*)pObj; @@ -1971,7 +1971,7 @@ static int32_t physiTableScanNodeToJson(const void* pObj, SJson* pJson) { code = tjsonAddBoolToObject(pJson, jkTableScanPhysiPlanNeedCountEmptyTable, pNode->needCountEmptyTable); } if (TSDB_CODE_SUCCESS == code) { - code = tjsonAddBoolToObject(pJson, jkTableScanPhysiPlanparaTablesSort, pNode->paraTablesSort); + code = tjsonAddBoolToObject(pJson, jkTableScanPhysiPlanParaTablesSort, pNode->paraTablesSort); } return code; } @@ -2050,7 +2050,7 @@ static int32_t jsonToPhysiTableScanNode(const SJson* pJson, void* pObj) { code = tjsonGetBoolValue(pJson, jkTableScanPhysiPlanNeedCountEmptyTable, &pNode->needCountEmptyTable); } if (TSDB_CODE_SUCCESS == code) { - code = tjsonGetBoolValue(pJson, jkTableScanPhysiPlanparaTablesSort, &pNode->paraTablesSort); + code = tjsonGetBoolValue(pJson, jkTableScanPhysiPlanParaTablesSort, &pNode->paraTablesSort); } return code; } From 35aed7653955340e9457b47e09e60753a944d5dd Mon Sep 17 00:00:00 2001 From: slzhou Date: Wed, 21 Feb 2024 13:39:15 +0800 Subject: [PATCH 30/41] docs: add para_tables_sort() hint --- docs/en/12-taos-sql/06-select.md | 6 ++++-- docs/zh/12-taos-sql/06-select.md | 5 +++-- 2 files changed, 7 insertions(+), 4 deletions(-) diff --git a/docs/en/12-taos-sql/06-select.md b/docs/en/12-taos-sql/06-select.md index 1fc6fb7e67..0cd2d06989 100755 --- a/docs/en/12-taos-sql/06-select.md +++ b/docs/en/12-taos-sql/06-select.md @@ -24,7 +24,7 @@ SELECT [hints] [DISTINCT] [TAGS] select_list hints: /*+ [hint([hint_param_list])] [hint([hint_param_list])] */ hint: - BATCH_SCAN | NO_BATCH_SCAN | SORT_FOR_GROUP + BATCH_SCAN | NO_BATCH_SCAN | SORT_FOR_GROUP | PARA_TABLES_SORT select_list: select_expr [, select_expr] ... @@ -87,12 +87,13 @@ Hints are a means of user control over query optimization for individual stateme The list of currently supported Hints is as follows: -| **Hint** | **Params** | **Comment** | **Scopt** | +| **Hint** | **Params** | **Comment** | **Scope** | | :-----------: | -------------- | -------------------------- | -----------------------------------| | BATCH_SCAN | None | Batch table scan | JOIN statment for stable | | NO_BATCH_SCAN | None | Sequential table scan | JOIN statment for stable | | SORT_FOR_GROUP| None | Use sort for partition, conflict with PARTITION_FIRST | With normal column in partition by list | | PARTITION_FIRST| None | Use Partition before aggregate, conflict with SORT_FOR_GROUP | With normal column in partition by list | +| PARA_TABLES_SORT| None | When sorting the supertable rows by timestamp, No temporary disk space is used | Sorting the supertable rows by timestamp | For example: @@ -100,6 +101,7 @@ For example: SELECT /*+ BATCH_SCAN() */ a.ts FROM stable1 a, stable2 b where a.tag0 = b.tag0 and a.ts = b.ts; SELECT /*+ SORT_FOR_GROUP() */ count(*), c1 FROM stable1 PARTITION BY c1; SELECT /*+ PARTITION_FIRST() */ count(*), c1 FROM stable1 PARTITION BY c1; +SELECT /*+ PARA_TABLES_SORT() */ * from stable1 order by ts; ``` ## Lists diff --git a/docs/zh/12-taos-sql/06-select.md b/docs/zh/12-taos-sql/06-select.md index 2a7dff6f7d..43b32efe32 100755 --- a/docs/zh/12-taos-sql/06-select.md +++ b/docs/zh/12-taos-sql/06-select.md @@ -24,7 +24,7 @@ SELECT [hints] [DISTINCT] [TAGS] select_list hints: /*+ [hint([hint_param_list])] [hint([hint_param_list])] */ hint: - BATCH_SCAN | NO_BATCH_SCAN | SORT_FOR_GROUP + BATCH_SCAN | NO_BATCH_SCAN | SORT_FOR_GROUP | PARA_TABLES_SORT select_list: select_expr [, select_expr] ... @@ -93,13 +93,14 @@ Hints 是用户控制单个语句查询优化的一种手段,当 Hint 不适 | NO_BATCH_SCAN | 无 | 采用顺序读表的方式 | 超级表 JOIN 语句 | | SORT_FOR_GROUP| 无 | 采用sort方式进行分组, 与PARTITION_FIRST冲突 | partition by 列表有普通列时 | | PARTITION_FIRST| 无 | 在聚合之前使用PARTITION计算分组, 与SORT_FOR_GROUP冲突 | partition by 列表有普通列时 | - +| PARA_TABLES_SORT| 无 | 超级表的数据按时间戳排序时, 不使用临时磁盘空间, 只使用内存 | 超级表的数据按时间戳排序时 | 举例: ```sql SELECT /*+ BATCH_SCAN() */ a.ts FROM stable1 a, stable2 b where a.tag0 = b.tag0 and a.ts = b.ts; SELECT /*+ SORT_FOR_GROUP() */ count(*), c1 FROM stable1 PARTITION BY c1; SELECT /*+ PARTITION_FIRST() */ count(*), c1 FROM stable1 PARTITION BY c1; +SELECT /*+ PARA_TABLES_SORT() */ * from stable1 order by ts; ``` ## 列表 From 6b716b1a7e8f0c103af44af7d76f08c9609b2293 Mon Sep 17 00:00:00 2001 From: wangmm0220 Date: Wed, 21 Feb 2024 13:54:43 +0800 Subject: [PATCH 31/41] fix:syntax error --- docs/zh/07-develop/07-tmq.mdx | 1089 +++++++++++++++++---------------- 1 file changed, 545 insertions(+), 544 deletions(-) diff --git a/docs/zh/07-develop/07-tmq.mdx b/docs/zh/07-develop/07-tmq.mdx index a852e71ff4..1476cd2170 100644 --- a/docs/zh/07-develop/07-tmq.mdx +++ b/docs/zh/07-develop/07-tmq.mdx @@ -91,220 +91,223 @@ TDengine 会为 WAL 文件自动创建索引以支持快速随机访问,并提 不同语言下, TMQ 订阅相关的 API 及数据结构如下(详细的接口说明可以参考连接器章节,注意consumer结构不是线程安全的,在一个线程使用consumer时,不要在另一个线程close这个consumer): - - + - ```c - typedef struct tmq_t tmq_t; - typedef struct tmq_conf_t tmq_conf_t; - typedef struct tmq_list_t tmq_list_t; +```c + typedef struct tmq_t tmq_t; + typedef struct tmq_conf_t tmq_conf_t; + typedef struct tmq_list_t tmq_list_t; - typedef void(tmq_commit_cb(tmq_t *tmq, int32_t code, void *param)); + typedef void(tmq_commit_cb(tmq_t *tmq, int32_t code, void *param)); - typedef enum tmq_conf_res_t { - TMQ_CONF_UNKNOWN = -2, - TMQ_CONF_INVALID = -1, - TMQ_CONF_OK = 0, - } tmq_conf_res_t; + typedef enum tmq_conf_res_t { + TMQ_CONF_UNKNOWN = -2, + TMQ_CONF_INVALID = -1, + TMQ_CONF_OK = 0, + } tmq_conf_res_t; - typedef struct tmq_topic_assignment { - int32_t vgId; - int64_t currentOffset; - int64_t begin; - int64_t end; - } tmq_topic_assignment; + typedef struct tmq_topic_assignment { + int32_t vgId; + int64_t currentOffset; + int64_t begin; + int64_t end; + } tmq_topic_assignment; - DLL_EXPORT tmq_conf_t *tmq_conf_new(); - DLL_EXPORT tmq_conf_res_t tmq_conf_set(tmq_conf_t *conf, const char *key, const char *value); - DLL_EXPORT void tmq_conf_destroy(tmq_conf_t *conf); - DLL_EXPORT void tmq_conf_set_auto_commit_cb(tmq_conf_t *conf, tmq_commit_cb *cb, void *param); + DLL_EXPORT tmq_conf_t *tmq_conf_new(); + DLL_EXPORT tmq_conf_res_t tmq_conf_set(tmq_conf_t *conf, const char *key, const char *value); + DLL_EXPORT void tmq_conf_destroy(tmq_conf_t *conf); + DLL_EXPORT void tmq_conf_set_auto_commit_cb(tmq_conf_t *conf, tmq_commit_cb *cb, void *param); - DLL_EXPORT tmq_list_t *tmq_list_new(); - DLL_EXPORT int32_t tmq_list_append(tmq_list_t *, const char *); - DLL_EXPORT void tmq_list_destroy(tmq_list_t *); - DLL_EXPORT int32_t tmq_list_get_size(const tmq_list_t *); - DLL_EXPORT char **tmq_list_to_c_array(const tmq_list_t *); + DLL_EXPORT tmq_list_t *tmq_list_new(); + DLL_EXPORT int32_t tmq_list_append(tmq_list_t *, const char *); + DLL_EXPORT void tmq_list_destroy(tmq_list_t *); + DLL_EXPORT int32_t tmq_list_get_size(const tmq_list_t *); + DLL_EXPORT char **tmq_list_to_c_array(const tmq_list_t *); - DLL_EXPORT tmq_t *tmq_consumer_new(tmq_conf_t *conf, char *errstr, int32_t errstrLen); - DLL_EXPORT int32_t tmq_subscribe(tmq_t *tmq, const tmq_list_t *topic_list); - DLL_EXPORT int32_t tmq_unsubscribe(tmq_t *tmq); - DLL_EXPORT int32_t tmq_subscription(tmq_t *tmq, tmq_list_t **topics); - DLL_EXPORT TAOS_RES *tmq_consumer_poll(tmq_t *tmq, int64_t timeout); - DLL_EXPORT int32_t tmq_consumer_close(tmq_t *tmq); - DLL_EXPORT int32_t tmq_commit_sync(tmq_t *tmq, const TAOS_RES *msg); - DLL_EXPORT void tmq_commit_async(tmq_t *tmq, const TAOS_RES *msg, tmq_commit_cb *cb, void *param); - DLL_EXPORT int32_t tmq_commit_offset_sync(tmq_t *tmq, const char *pTopicName, int32_t vgId, int64_t offset); - DLL_EXPORT void tmq_commit_offset_async(tmq_t *tmq, const char *pTopicName, int32_t vgId, int64_t offset, tmq_commit_cb *cb, void *param); - DLL_EXPORT int32_t tmq_get_topic_assignment(tmq_t *tmq, const char *pTopicName, tmq_topic_assignment **assignment,int32_t *numOfAssignment); - DLL_EXPORT void tmq_free_assignment(tmq_topic_assignment* pAssignment); - DLL_EXPORT int32_t tmq_offset_seek(tmq_t *tmq, const char *pTopicName, int32_t vgId, int64_t offset); - DLL_EXPORT int64_t tmq_position(tmq_t *tmq, const char *pTopicName, int32_t vgId); - DLL_EXPORT int64_t tmq_committed(tmq_t *tmq, const char *pTopicName, int32_t vgId); + DLL_EXPORT tmq_t *tmq_consumer_new(tmq_conf_t *conf, char *errstr, int32_t errstrLen); + DLL_EXPORT int32_t tmq_subscribe(tmq_t *tmq, const tmq_list_t *topic_list); + DLL_EXPORT int32_t tmq_unsubscribe(tmq_t *tmq); + DLL_EXPORT int32_t tmq_subscription(tmq_t *tmq, tmq_list_t **topics); + DLL_EXPORT TAOS_RES *tmq_consumer_poll(tmq_t *tmq, int64_t timeout); + DLL_EXPORT int32_t tmq_consumer_close(tmq_t *tmq); + DLL_EXPORT int32_t tmq_commit_sync(tmq_t *tmq, const TAOS_RES *msg); + DLL_EXPORT void tmq_commit_async(tmq_t *tmq, const TAOS_RES *msg, tmq_commit_cb *cb, void *param); + DLL_EXPORT int32_t tmq_commit_offset_sync(tmq_t *tmq, const char *pTopicName, int32_t vgId, int64_t offset); + DLL_EXPORT void tmq_commit_offset_async(tmq_t *tmq, const char *pTopicName, int32_t vgId, int64_t offset, tmq_commit_cb *cb, void *param); + DLL_EXPORT int32_t tmq_get_topic_assignment(tmq_t *tmq, const char *pTopicName, tmq_topic_assignment **assignment,int32_t *numOfAssignment); + DLL_EXPORT void tmq_free_assignment(tmq_topic_assignment* pAssignment); + DLL_EXPORT int32_t tmq_offset_seek(tmq_t *tmq, const char *pTopicName, int32_t vgId, int64_t offset); + DLL_EXPORT int64_t tmq_position(tmq_t *tmq, const char *pTopicName, int32_t vgId); + DLL_EXPORT int64_t tmq_committed(tmq_t *tmq, const char *pTopicName, int32_t vgId); - DLL_EXPORT const char *tmq_get_topic_name(TAOS_RES *res); - DLL_EXPORT const char *tmq_get_db_name(TAOS_RES *res); - DLL_EXPORT int32_t tmq_get_vgroup_id(TAOS_RES *res); - DLL_EXPORT int64_t tmq_get_vgroup_offset(TAOS_RES* res); - DLL_EXPORT const char *tmq_err2str(int32_t code); - ``` - - + DLL_EXPORT const char *tmq_get_topic_name(TAOS_RES *res); + DLL_EXPORT const char *tmq_get_db_name(TAOS_RES *res); + DLL_EXPORT int32_t tmq_get_vgroup_id(TAOS_RES *res); + DLL_EXPORT int64_t tmq_get_vgroup_offset(TAOS_RES* res); + DLL_EXPORT const char *tmq_err2str(int32_t code); +``` - ```java - void subscribe(Collection topics) throws SQLException; +下面介绍一下它们的具体用法(超级表和子表结构请参考“数据建模”一节),完整的示例代码请见下面 C 语言的示例代码。 - void unsubscribe() throws SQLException; + + - Set subscription() throws SQLException; +```java +void subscribe(Collection topics) throws SQLException; - ConsumerRecords poll(Duration timeout) throws SQLException; +void unsubscribe() throws SQLException; - Set assignment() throws SQLException; - long position(TopicPartition partition) throws SQLException; - Map position(String topic) throws SQLException; - Map beginningOffsets(String topic) throws SQLException; - Map endOffsets(String topic) throws SQLException; - Map committed(Set partitions) throws SQLException; +Set subscription() throws SQLException; - void seek(TopicPartition partition, long offset) throws SQLException; - void seekToBeginning(Collection partitions) throws SQLException; - void seekToEnd(Collection partitions) throws SQLException; +ConsumerRecords poll(Duration timeout) throws SQLException; - void commitSync() throws SQLException; - void commitSync(Map offsets) throws SQLException; +Set assignment() throws SQLException; +long position(TopicPartition partition) throws SQLException; +Map position(String topic) throws SQLException; +Map beginningOffsets(String topic) throws SQLException; +Map endOffsets(String topic) throws SQLException; +Map committed(Set partitions) throws SQLException; - void close() throws SQLException; - ``` +void seek(TopicPartition partition, long offset) throws SQLException; +void seekToBeginning(Collection partitions) throws SQLException; +void seekToEnd(Collection partitions) throws SQLException; - +void commitSync() throws SQLException; +void commitSync(Map offsets) throws SQLException; - +void close() throws SQLException; +``` - ```python - class Consumer: - def subscribe(self, topics): + + + + +```python +class Consumer: + def subscribe(self, topics): pass - def unsubscribe(self): + def unsubscribe(self): pass - def poll(self, timeout: float = 1.0): + def poll(self, timeout: float = 1.0): pass - def assignment(self): + def assignment(self): pass - def seek(self, partition): + def seek(self, partition): pass - def close(self): + def close(self): pass - def commit(self, message): + def commit(self, message): pass - ``` +``` - + - + - ```go - func NewConsumer(conf *tmq.ConfigMap) (*Consumer, error) +```go +func NewConsumer(conf *tmq.ConfigMap) (*Consumer, error) - // 出于兼容目的保留 rebalanceCb 参数,当前未使用 - func (c *Consumer) Subscribe(topic string, rebalanceCb RebalanceCb) error +// 出于兼容目的保留 rebalanceCb 参数,当前未使用 +func (c *Consumer) Subscribe(topic string, rebalanceCb RebalanceCb) error - // 出于兼容目的保留 rebalanceCb 参数,当前未使用 - func (c *Consumer) SubscribeTopics(topics []string, rebalanceCb RebalanceCb) error +// 出于兼容目的保留 rebalanceCb 参数,当前未使用 +func (c *Consumer) SubscribeTopics(topics []string, rebalanceCb RebalanceCb) error - func (c *Consumer) Poll(timeoutMs int) tmq.Event +func (c *Consumer) Poll(timeoutMs int) tmq.Event - // 出于兼容目的保留 tmq.TopicPartition 参数,当前未使用 - func (c *Consumer) Commit() ([]tmq.TopicPartition, error) +// 出于兼容目的保留 tmq.TopicPartition 参数,当前未使用 +func (c *Consumer) Commit() ([]tmq.TopicPartition, error) - func (c *Consumer) Unsubscribe() error +func (c *Consumer) Unsubscribe() error - func (c *Consumer) Close() error - ``` +func (c *Consumer) Close() error +``` - + - + - ```rust - impl TBuilder for TmqBuilder - fn from_dsn(dsn: D) -> Result - fn build(&self) -> Result +```rust +impl TBuilder for TmqBuilder + fn from_dsn(dsn: D) -> Result + fn build(&self) -> Result - impl AsAsyncConsumer for Consumer - async fn subscribe, I: IntoIterator + Send>( +impl AsAsyncConsumer for Consumer + async fn subscribe, I: IntoIterator + Send>( &mut self, topics: I, - ) -> Result<(), Self::Error>; - fn stream( + ) -> Result<(), Self::Error>; + fn stream( &self, - ) -> Pin< - Box< + ) -> Pin< + Box< dyn '_ + Send + futures::Stream< Item = Result<(Self::Offset, MessageSet), Self::Error>, >, - >, - >; - async fn commit(&self, offset: Self::Offset) -> Result<(), Self::Error>; + >, + >; + async fn commit(&self, offset: Self::Offset) -> Result<(), Self::Error>; - async fn unsubscribe(self); - ``` + async fn unsubscribe(self); +``` - 可在 上查看详细 API 说明。 +可在 上查看详细 API 说明。 - + - + - ```js - function TMQConsumer(config) - - function subscribe(topic) - - function consume(timeout) - - function subscription() - - function unsubscribe() - - function commit(msg) - - function close() - ``` +```js +function TMQConsumer(config) - +function subscribe(topic) - +function consume(timeout) - ```csharp - class ConsumerBuilder - - ConsumerBuilder(IEnumerable> config) - - public IConsumer Build() - - void Subscribe(IEnumerable topics) - - void Subscribe(string topic) - - ConsumeResult Consume(int millisecondsTimeout) - - List Subscription() - - void Unsubscribe() - - List Commit() - - void Close() - ``` - +function subscription() + +function unsubscribe() + +function commit(msg) + +function close() +``` + + + + + +```csharp +class ConsumerBuilder + +ConsumerBuilder(IEnumerable> config) + +public IConsumer Build() + +void Subscribe(IEnumerable topics) + +void Subscribe(string topic) + +ConsumeResult Consume(int millisecondsTimeout) + +List Subscription() + +void Unsubscribe() + +List Commit() + +void Close() +``` + + # 数据订阅示例 @@ -334,163 +337,164 @@ CREATE TOPIC topic_name AS SELECT ts, c1, c2, c3 FROM tmqdb.stb WHERE c1 > 1; 对于不同编程语言,其设置方式如下: + - - ```c - /* 根据需要,设置消费组 (group.id)、自动提交 (enable.auto.commit)、 - 自动提交时间间隔 (auto.commit.interval.ms)、用户名 (td.connect.user)、密码 (td.connect.pass) 等参数 */ - tmq_conf_t* conf = tmq_conf_new(); - tmq_conf_set(conf, "enable.auto.commit", "true"); - tmq_conf_set(conf, "auto.commit.interval.ms", "1000"); - tmq_conf_set(conf, "group.id", "cgrpName"); - tmq_conf_set(conf, "td.connect.user", "root"); - tmq_conf_set(conf, "td.connect.pass", "taosdata"); - tmq_conf_set(conf, "auto.offset.reset", "latest"); - tmq_conf_set(conf, "msg.with.table.name", "true"); - tmq_conf_set_auto_commit_cb(conf, tmq_commit_cb_print, NULL); +```c +/* 根据需要,设置消费组 (group.id)、自动提交 (enable.auto.commit)、 + 自动提交时间间隔 (auto.commit.interval.ms)、用户名 (td.connect.user)、密码 (td.connect.pass) 等参数 */ +tmq_conf_t* conf = tmq_conf_new(); +tmq_conf_set(conf, "enable.auto.commit", "true"); +tmq_conf_set(conf, "auto.commit.interval.ms", "1000"); +tmq_conf_set(conf, "group.id", "cgrpName"); +tmq_conf_set(conf, "td.connect.user", "root"); +tmq_conf_set(conf, "td.connect.pass", "taosdata"); +tmq_conf_set(conf, "auto.offset.reset", "latest"); +tmq_conf_set(conf, "msg.with.table.name", "true"); +tmq_conf_set_auto_commit_cb(conf, tmq_commit_cb_print, NULL); - tmq_t* tmq = tmq_consumer_new(conf, NULL, 0); - tmq_conf_destroy(conf); - ``` - +tmq_t* tmq = tmq_consumer_new(conf, NULL, 0); +tmq_conf_destroy(conf); +``` - + + - 对于 Java 程序,还可以使用如下配置项: - - | 参数名称 | 类型 | 参数说明 | - | ----------------------------- | ------ | ----------------------------------------------------------------------------------------------------------------------------- | - | `td.connect.type` | string | 连接类型,"jni" 指原生连接,"ws" 指 websocket 连接,默认值为 "jni" | - | `bootstrap.servers` | string | 连接地址,如 `localhost:6030` | - | `value.deserializer` | string | 值解析方法,使用此方法应实现 `com.taosdata.jdbc.tmq.Deserializer` 接口或继承 `com.taosdata.jdbc.tmq.ReferenceDeserializer` 类 | - | `value.deserializer.encoding` | string | 指定字符串解析的字符集 | | - - 需要注意:此处使用 `bootstrap.servers` 替代 `td.connect.ip` 和 `td.connect.port`,以提供与 Kafka 一致的接口。 - - ```java - Properties properties = new Properties(); - properties.setProperty("enable.auto.commit", "true"); - properties.setProperty("auto.commit.interval.ms", "1000"); - properties.setProperty("group.id", "cgrpName"); - properties.setProperty("bootstrap.servers", "127.0.0.1:6030"); - properties.setProperty("td.connect.user", "root"); - properties.setProperty("td.connect.pass", "taosdata"); - properties.setProperty("auto.offset.reset", "latest"); - properties.setProperty("msg.with.table.name", "true"); - properties.setProperty("value.deserializer", "com.taos.example.MetersDeserializer"); - - TaosConsumer consumer = new TaosConsumer<>(properties); - - /* value deserializer definition. */ - import com.taosdata.jdbc.tmq.ReferenceDeserializer; - - public class MetersDeserializer extends ReferenceDeserializer { - } - ``` - +对于 Java 程序,还可以使用如下配置项: - - - ```go - conf := &tmq.ConfigMap{ - "group.id": "test", - "auto.offset.reset": "latest", - "td.connect.ip": "127.0.0.1", - "td.connect.user": "root", - "td.connect.pass": "taosdata", - "td.connect.port": "6030", - "client.id": "test_tmq_c", - "enable.auto.commit": "false", - "msg.with.table.name": "true", - } - consumer, err := NewConsumer(conf) - ``` - - +| 参数名称 | 类型 | 参数说明 | +| ----------------------------- | ------ | ----------------------------------------------------------------------------------------------------------------------------- | +| `td.connect.type` | string | 连接类型,"jni" 指原生连接,"ws" 指 websocket 连接,默认值为 "jni" | +| `bootstrap.servers` | string | 连接地址,如 `localhost:6030` | +| `value.deserializer` | string | 值解析方法,使用此方法应实现 `com.taosdata.jdbc.tmq.Deserializer` 接口或继承 `com.taosdata.jdbc.tmq.ReferenceDeserializer` 类 | +| `value.deserializer.encoding` | string | 指定字符串解析的字符集 | | - - - ```rust - let mut dsn: Dsn = "taos://".parse()?; - dsn.set("group.id", "group1"); - dsn.set("client.id", "test"); - dsn.set("auto.offset.reset", "latest"); - - let tmq = TmqBuilder::from_dsn(dsn)?; - - let mut consumer = tmq.build()?; - ``` - - +需要注意:此处使用 `bootstrap.servers` 替代 `td.connect.ip` 和 `td.connect.port`,以提供与 Kafka 一致的接口。 - - - Python 语言下引入 `taos` 库的 `Consumer` 类,创建一个 Consumer 示例: - - ```python - from taos.tmq import Consumer - - # Syntax: `consumer = Consumer(configs)` - # - # Example: - consumer = Consumer( - { - "group.id": "local", - "client.id": "1", - "enable.auto.commit": "true", - "auto.commit.interval.ms": "1000", - "td.connect.ip": "127.0.0.1", - "td.connect.user": "root", - "td.connect.pass": "taosdata", - "auto.offset.reset": "latest", - "msg.with.table.name": "true", - } - ) - ``` - - +```java +Properties properties = new Properties(); +properties.setProperty("enable.auto.commit", "true"); +properties.setProperty("auto.commit.interval.ms", "1000"); +properties.setProperty("group.id", "cgrpName"); +properties.setProperty("bootstrap.servers", "127.0.0.1:6030"); +properties.setProperty("td.connect.user", "root"); +properties.setProperty("td.connect.pass", "taosdata"); +properties.setProperty("auto.offset.reset", "latest"); +properties.setProperty("msg.with.table.name", "true"); +properties.setProperty("value.deserializer", "com.taos.example.MetersDeserializer"); - - - ```js - // 根据需要,设置消费组 (group.id)、自动提交 (enable.auto.commit)、 - // 自动提交时间间隔 (auto.commit.interval.ms)、用户名 (td.connect.user)、密码 (td.connect.pass) 等参数 - - let consumer = taos.consumer({ - 'enable.auto.commit': 'true', - 'auto.commit.interval.ms','1000', - 'group.id': 'tg2', - 'td.connect.user': 'root', - 'td.connect.pass': 'taosdata', - 'auto.offset.reset','latest', - 'msg.with.table.name': 'true', - 'td.connect.ip','127.0.0.1', - 'td.connect.port','6030' - }); - ``` - - +TaosConsumer consumer = new TaosConsumer<>(properties); + +/* value deserializer definition. */ +import com.taosdata.jdbc.tmq.ReferenceDeserializer; + +public class MetersDeserializer extends ReferenceDeserializer { +} +``` + + + + + +```go +conf := &tmq.ConfigMap{ + "group.id": "test", + "auto.offset.reset": "latest", + "td.connect.ip": "127.0.0.1", + "td.connect.user": "root", + "td.connect.pass": "taosdata", + "td.connect.port": "6030", + "client.id": "test_tmq_c", + "enable.auto.commit": "false", + "msg.with.table.name": "true", +} +consumer, err := NewConsumer(conf) +``` + + + + + +```rust +let mut dsn: Dsn = "taos://".parse()?; +dsn.set("group.id", "group1"); +dsn.set("client.id", "test"); +dsn.set("auto.offset.reset", "latest"); + +let tmq = TmqBuilder::from_dsn(dsn)?; + +let mut consumer = tmq.build()?; +``` + + + + + +Python 语言下引入 `taos` 库的 `Consumer` 类,创建一个 Consumer 示例: + +```python +from taos.tmq import Consumer + +# Syntax: `consumer = Consumer(configs)` +# +# Example: +consumer = Consumer( + { + "group.id": "local", + "client.id": "1", + "enable.auto.commit": "true", + "auto.commit.interval.ms": "1000", + "td.connect.ip": "127.0.0.1", + "td.connect.user": "root", + "td.connect.pass": "taosdata", + "auto.offset.reset": "latest", + "msg.with.table.name": "true", + } +) +``` + + + + + +```js +// 根据需要,设置消费组 (group.id)、自动提交 (enable.auto.commit)、 +// 自动提交时间间隔 (auto.commit.interval.ms)、用户名 (td.connect.user)、密码 (td.connect.pass) 等参数 + +let consumer = taos.consumer({ + 'enable.auto.commit': 'true', + 'auto.commit.interval.ms','1000', + 'group.id': 'tg2', + 'td.connect.user': 'root', + 'td.connect.pass': 'taosdata', + 'auto.offset.reset','latest', + 'msg.with.table.name': 'true', + 'td.connect.ip','127.0.0.1', + 'td.connect.port','6030' + }); +``` + + + + + +```csharp +var cfg = new Dictionary() +{ + { "group.id", "group1" }, + { "auto.offset.reset", "latest" }, + { "td.connect.ip", "127.0.0.1" }, + { "td.connect.user", "root" }, + { "td.connect.pass", "taosdata" }, + { "td.connect.port", "6030" }, + { "client.id", "tmq_example" }, + { "enable.auto.commit", "true" }, + { "msg.with.table.name", "false" }, +}; +var consumer = new ConsumerBuilder>(cfg).Build(); +``` + + - - - ```csharp - var cfg = new Dictionary() - { - { "group.id", "group1" }, - { "auto.offset.reset", "latest" }, - { "td.connect.ip", "127.0.0.1" }, - { "td.connect.user", "root" }, - { "td.connect.pass", "taosdata" }, - { "td.connect.port", "6030" }, - { "client.id", "tmq_example" }, - { "enable.auto.commit", "true" }, - { "msg.with.table.name", "false" }, - }; - var consumer = new ConsumerBuilder>(cfg).Build(); - ``` - - - 上述配置中包括 consumer group ID,如果多个 consumer 指定的 consumer group ID 一样,则自动形成一个 consumer group,共享消费进度。 @@ -500,78 +504,77 @@ CREATE TOPIC topic_name AS SELECT ts, c1, c2, c3 FROM tmqdb.stb WHERE c1 > 1; 一个 consumer 支持同时订阅多个 topic。 + - - - ```c - // 创建订阅 topics 列表 - tmq_list_t* topicList = tmq_list_new(); - tmq_list_append(topicList, "topicName"); - // 启动订阅 - tmq_subscribe(tmq, topicList); - tmq_list_destroy(topicList); - - ``` - - - - - ```java - List topics = new ArrayList<>(); - topics.add("tmq_topic"); - consumer.subscribe(topics); - ``` - - - +```c +// 创建订阅 topics 列表 +tmq_list_t* topicList = tmq_list_new(); +tmq_list_append(topicList, "topicName"); +// 启动订阅 +tmq_subscribe(tmq, topicList); +tmq_list_destroy(topicList); + +``` - ```go - err = consumer.Subscribe("example_tmq_topic", nil) - if err != nil { - panic(err) - } - ``` + + - - +```java +List topics = new ArrayList<>(); +topics.add("tmq_topic"); +consumer.subscribe(topics); +``` - ```rust - consumer.subscribe(["tmq_meters"]).await?; - ``` + + - +```go +err = consumer.Subscribe("example_tmq_topic", nil) +if err != nil { + panic(err) +} +``` - + + - ```python - consumer.subscribe(['topic1', 'topic2']) - ``` +```rust +consumer.subscribe(["tmq_meters"]).await?; +``` - + - + - ```js - // 创建订阅 topics 列表 - let topics = ['topic_test'] +```python +consumer.subscribe(['topic1', 'topic2']) +``` - // 启动订阅 - consumer.subscribe(topics); - ``` + - + - +```js +// 创建订阅 topics 列表 +let topics = ['topic_test'] - ```csharp - // 创建订阅 topics 列表 - List topics = new List(); - topics.add("tmq_topic"); - // 启动订阅 - consumer.Subscribe(topics); - ``` +// 启动订阅 +consumer.subscribe(topics); +``` - + + + + +```csharp +// 创建订阅 topics 列表 +List topics = new List(); +topics.add("tmq_topic"); +// 启动订阅 +consumer.Subscribe(topics); +``` + + @@ -580,135 +583,134 @@ CREATE TOPIC topic_name AS SELECT ts, c1, c2, c3 FROM tmqdb.stb WHERE c1 > 1; 以下代码展示了不同语言下如何对 TMQ 消息进行消费。 + - +```c +// 消费数据 +while (running) { + TAOS_RES* msg = tmq_consumer_poll(tmq, timeOut); + msg_process(msg); +} +``` - ```c - // 消费数据 - while (running) { - TAOS_RES* msg = tmq_consumer_poll(tmq, timeOut); - msg_process(msg); - } - ``` +这里是一个 **while** 循环,每调用一次 tmq_consumer_poll(),获取一个消息,该消息与普通查询返回的结果集完全相同,可以使用相同的解析 API 完成消息内容的解析。 - 这里是一个 **while** 循环,每调用一次 tmq_consumer_poll(),获取一个消息,该消息与普通查询返回的结果集完全相同,可以使用相同的解析 API 完成消息内容的解析。 + + - - - - ```java - while(running){ - ConsumerRecords meters = consumer.poll(Duration.ofMillis(100)); - for (Meters meter : meters) { - processMsg(meter); - } - } - ``` - - +```java +while(running){ + ConsumerRecords meters = consumer.poll(Duration.ofMillis(100)); + for (Meters meter : meters) { + processMsg(meter); + } +} +``` - + - ```go - for { - ev := consumer.Poll(0) - if ev != nil { - switch e := ev.(type) { - case *tmqcommon.DataMessage: - fmt.Println(e.Value()) - case tmqcommon.Error: - fmt.Fprintf(os.Stderr, "%% Error: %v: %v\n", e.Code(), e) - panic(e) - } - consumer.Commit() - } - } - ``` + - +```go +for { + ev := consumer.Poll(0) + if ev != nil { + switch e := ev.(type) { + case *tmqcommon.DataMessage: + fmt.Println(e.Value()) + case tmqcommon.Error: + fmt.Fprintf(os.Stderr, "%% Error: %v: %v\n", e.Code(), e) + panic(e) + } + consumer.Commit() + } +} +``` - + - ```rust - { - let mut stream = consumer.stream(); + - while let Some((offset, message)) = stream.try_next().await? { - // get information from offset +```rust +{ + let mut stream = consumer.stream(); - // the topic - let topic = offset.topic(); - // the vgroup id, like partition id in kafka. - let vgroup_id = offset.vgroup_id(); - println!("* in vgroup id {vgroup_id} of topic {topic}\n"); + while let Some((offset, message)) = stream.try_next().await? { + // get information from offset - if let Some(data) = message.into_data() { + // the topic + let topic = offset.topic(); + // the vgroup id, like partition id in kafka. + let vgroup_id = offset.vgroup_id(); + println!("* in vgroup id {vgroup_id} of topic {topic}\n"); + + if let Some(data) = message.into_data() { while let Some(block) = data.fetch_raw_block().await? { - // one block for one table, get table name if needed - let name = block.table_name(); - let records: Vec = block.deserialize().try_collect()?; - println!( - "** table: {}, got {} records: {:#?}\n", - name.unwrap(), - records.len(), - records - ); + // one block for one table, get table name if needed + let name = block.table_name(); + let records: Vec = block.deserialize().try_collect()?; + println!( + "** table: {}, got {} records: {:#?}\n", + name.unwrap(), + records.len(), + records + ); + } } - } - consumer.commit(offset).await?; - } - } - ``` + consumer.commit(offset).await?; + } +} +``` - - + + - ```python - while True: - res = consumer.poll(100) - if not res: +```python +while True: + res = consumer.poll(100) + if not res: continue - err = res.error() - if err is not None: + err = res.error() + if err is not None: raise err - val = res.value() + val = res.value() - for block in val: + for block in val: print(block.fetchall()) - ``` +``` - + - + - ```js - while(true){ - msg = consumer.consume(200); - // process message(consumeResult) - console.log(msg.topicPartition); - console.log(msg.block); - console.log(msg.fields) - } - ``` +```js +while(true){ + msg = consumer.consume(200); + // process message(consumeResult) + console.log(msg.topicPartition); + console.log(msg.block); + console.log(msg.fields) +} +``` - + - + - ```csharp - // 消费数据 - while (true) - { - using (var result = consumer.Consume(500)) - { - if (result == null) continue; - ProcessMsg(result); - consumer.Commit(); - } - } - ``` +```csharp +// 消费数据 +while (true) +{ + using (var result = consumer.Consume(500)) + { + if (result == null) continue; + ProcessMsg(result); + consumer.Commit(); + } +} +``` - + @@ -717,80 +719,79 @@ CREATE TOPIC topic_name AS SELECT ts, c1, c2, c3 FROM tmqdb.stb WHERE c1 > 1; 消费结束后,应当取消订阅。 - - + - ```c - /* 取消订阅 */ - tmq_unsubscribe(tmq); +```c +/* 取消订阅 */ +tmq_unsubscribe(tmq); - /* 关闭消费者对象 */ - tmq_consumer_close(tmq); - ``` +/* 关闭消费者对象 */ +tmq_consumer_close(tmq); +``` - - + + - ```java - /* 取消订阅 */ - consumer.unsubscribe(); +```java +/* 取消订阅 */ +consumer.unsubscribe(); - /* 关闭消费 */ - consumer.close(); - ``` +/* 关闭消费 */ +consumer.close(); +``` - - - + - ```go - /* Unsubscribe */ - _ = consumer.Unsubscribe() + - /* Close consumer */ - _ = consumer.Close() - ``` +```go +/* Unsubscribe */ +_ = consumer.Unsubscribe() - - - +/* Close consumer */ +_ = consumer.Close() +``` - ```rust - consumer.unsubscribe().await; - ``` + - + - +```rust +consumer.unsubscribe().await; +``` - ```py - # 取消订阅 - consumer.unsubscribe() - # 关闭消费 - consumer.close() - ``` + - - + - ```js - consumer.unsubscribe(); - consumer.close(); - ``` +```py +# 取消订阅 +consumer.unsubscribe() +# 关闭消费 +consumer.close() +``` - + + - +```js +consumer.unsubscribe(); +consumer.close(); +``` - ```csharp - // 取消订阅 - consumer.Unsubscribe(); + - // 关闭消费 - consumer.Close(); - ``` + - +```csharp +// 取消订阅 +consumer.Unsubscribe(); + +// 关闭消费 +consumer.Close(); +``` + + @@ -800,41 +801,41 @@ CREATE TOPIC topic_name AS SELECT ts, c1, c2, c3 FROM tmqdb.stb WHERE c1 > 1; - - - + + + - - - - - - - - - - + + + + + + + + + + - - - + + + - - - + + + - - - + + + - - - + + + + + + + - - - - #订阅高级功能 @@ -854,4 +855,4 @@ CREATE TOPIC topic_name AS SELECT ts, c1, c2, c3 FROM tmqdb.stb WHERE c1 > 1; - 超级表和库订阅不支持回放 - enable.replay 参数,true表示开启订阅回放功能,false表示不开启订阅回放功能,默认不开启。 - 回放不支持进度保存,所以回放参数 enable.replay = true 时,auto commit 自动关闭 -- 因为数据回放本身需要处理时间,所以回放的精度存在几十ms的误差 \ No newline at end of file +- 因为数据回放本身需要处理时间,所以回放的精度存在几十ms的误差 From 2049fa688a37ed1119ed38cc27660a92988407db Mon Sep 17 00:00:00 2001 From: wangmm0220 Date: Wed, 21 Feb 2024 16:01:02 +0800 Subject: [PATCH 32/41] fix:add log --- source/dnode/mnode/impl/src/mndStream.c | 2 ++ source/dnode/mnode/impl/src/mndStreamHb.c | 2 +- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index 4e0c76f6de..9edac8fbde 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -1545,6 +1545,8 @@ static int32_t mndProcessPauseStreamReq(SRpcMsg *pReq) { } } + mInfo("stream:%s,%"PRId64 " start to pause stream", pauseReq.name, pStream->uid); + if (pStream->status == STREAM_STATUS__PAUSE) { sdbRelease(pMnode->pSdb, pStream); return 0; diff --git a/source/dnode/mnode/impl/src/mndStreamHb.c b/source/dnode/mnode/impl/src/mndStreamHb.c index 5de442951c..23f5c5d620 100644 --- a/source/dnode/mnode/impl/src/mndStreamHb.c +++ b/source/dnode/mnode/impl/src/mndStreamHb.c @@ -211,7 +211,7 @@ int32_t suspendAllStreams(SMnode *pMnode, SRpcHandleInfo* info){ }; tmsgPutToQueue(&pMnode->msgCb, WRITE_QUEUE, &rpcMsg); - mInfo("receive pause stream:%s, %s, %p, because grant expired", pStream->name, reqPause.name, reqPause.name); + mInfo("receive pause stream:%s, %s, %"PRId64 ", because grant expired", pStream->name, reqPause.name, pStream->uid); } sdbRelease(pSdb, pStream); From 9739b9feb6a6fd6f827ecf455d0a31763a3bc08d Mon Sep 17 00:00:00 2001 From: 54liuyao <54liuyao> Date: Wed, 21 Feb 2024 16:22:51 +0800 Subject: [PATCH 33/41] set group id of delete range --- source/libs/executor/src/streamcountwindowoperator.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/libs/executor/src/streamcountwindowoperator.c b/source/libs/executor/src/streamcountwindowoperator.c index 706b4c5a01..080f9d4e2b 100644 --- a/source/libs/executor/src/streamcountwindowoperator.c +++ b/source/libs/executor/src/streamcountwindowoperator.c @@ -164,7 +164,7 @@ void getCountWinRange(SStreamAggSupporter* pAggSup, const SSessionKey* pKey, ESt } else { pCur = pAggSup->stateStore.streamStateSessionSeekKeyCurrentNext(pAggSup->pState, pKey); } - SSessionKey tmpKey = {0}; + SSessionKey tmpKey = {.groupId = pKey->groupId, .win.ekey = INT64_MIN, .win.skey = INT64_MIN}; int32_t code = pAggSup->stateStore.streamStateSessionGetKVByCur(pCur, &tmpKey, NULL, 0); if (code != TSDB_CODE_SUCCESS) { pAggSup->stateStore.streamStateFreeCur(pCur); From 036b54926223d1353306875b81d67fc1b4fb4599 Mon Sep 17 00:00:00 2001 From: wangmm0220 Date: Wed, 21 Feb 2024 17:17:51 +0800 Subject: [PATCH 34/41] fix:syntax error --- docs/zh/07-develop/07-tmq.mdx | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/zh/07-develop/07-tmq.mdx b/docs/zh/07-develop/07-tmq.mdx index a852e71ff4..dc08f4e24d 100644 --- a/docs/zh/07-develop/07-tmq.mdx +++ b/docs/zh/07-develop/07-tmq.mdx @@ -20,13 +20,13 @@ import CDemo from "./_sub_c.mdx"; # 介绍 ## 主题 -与 kafka 一样,你需要定义 topic, TDengine 的 topic 有三种,可以是数据库,超级表,或者一个 `SELECT` 语句,具体的语法参见 [CREATE TOPIC](../../12-taos-sql/13-tmq)。与其他消息队列软件相比,这是 TDengine 数据订阅功能的最大的优势,它提供了更大的灵活性,数据的颗粒度可以由应用随时调整,而且数据的过滤与预处理交给 TDengine,而不是应用完成,有效的减少传输的数据量与应用的复杂度。 +与 kafka 一样,你需要定义 topic, TDengine 的 topic 有三种,可以是数据库,超级表,或者一个 `SELECT` 语句,具体的语法参见 [CREATE TOPIC](../../taos-sql/tmq)。与其他消息队列软件相比,这是 TDengine 数据订阅功能的最大的优势,它提供了更大的灵活性,数据的颗粒度可以由应用随时调整,而且数据的过滤与预处理交给 TDengine,而不是应用完成,有效的减少传输的数据量与应用的复杂度。 如下图,每个 topic 涉及到的数据表可能分布在多个 vnode(相当于 kafka 里的 partition) 上,每个 vnode 上的数据保存在 WAL(Write-Ahead-Log) 文件中,WAL 文件里的数据是顺序写入的(由于 WAL 文件中存储的不只有数据,还有元数据,写入消息等,所以数据的版本号不是连续的)。 ![img_5.png](img_5.png) -TDengine 会为 WAL 文件自动创建索引以支持快速随机访问,并提供了灵活可配置的文件切换与保留机制,用户可以按需指定 WAL 文件保留的时间以及大小(详见 [CREATE DATABASE](../../12-taos-sql/02-database) 语句,由于消费是通过 WAL 实现的,所以应该根据写入消费速度来确定 WAL 的保存时长)。通过以上方式将 WAL 改造成了一个保留事件到达顺序的、可持久化的存储引擎。 +TDengine 会为 WAL 文件自动创建索引以支持快速随机访问,并提供了灵活可配置的文件切换与保留机制,用户可以按需指定 WAL 文件保留的时间以及大小(详见 [CREATE DATABASE](../../taos-sql/database) 语句,由于消费是通过 WAL 实现的,所以应该根据写入消费速度来确定 WAL 的保存时长)。通过以上方式将 WAL 改造成了一个保留事件到达顺序的、可持久化的存储引擎。 对于 `SELECT` 语句形式的 topic,在消费时,TDengine 根据当前消费进度从 WAL 直接读取数据,并使用统一的查询引擎实现过滤、变换等操作,将数据推送给消费者。 @@ -66,7 +66,7 @@ TDengine 会为 WAL 文件自动创建索引以支持快速随机访问,并提 # 语法说明 -具体的语法参见 [数据订阅](../../12-taos-sql/13-tmq) +具体的语法参见 [数据订阅](../../taos-sql/tmq) # 消费参数 From b629c27691f3e0a8350076064d7e7901ae1183e0 Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Wed, 21 Feb 2024 10:53:21 +0000 Subject: [PATCH 35/41] fix invalid read --- source/libs/transport/src/transCli.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/libs/transport/src/transCli.c b/source/libs/transport/src/transCli.c index e2b69dd145..2370efa460 100644 --- a/source/libs/transport/src/transCli.c +++ b/source/libs/transport/src/transCli.c @@ -961,7 +961,7 @@ static void cliSendCb(uv_write_t* req, int status) { tTrace("%s conn %p send cost:%dus ", CONN_GET_INST_LABEL(pConn), pConn, (int)cost); } } - if (pMsg->msg.contLen == 0 && pMsg->msg.pCont != 0) { + if (pMsg != NULL && pMsg->msg.contLen == 0 && pMsg->msg.pCont != 0) { rpcFreeCont(pMsg->msg.pCont); pMsg->msg.pCont = 0; } From 705ae7d8274be5e74314ca4ad8ef70a4a54473e7 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 22 Feb 2024 09:04:57 +0800 Subject: [PATCH 36/41] test(stream): init mnode lock. --- source/dnode/mnode/impl/test/stream/stream.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/source/dnode/mnode/impl/test/stream/stream.cpp b/source/dnode/mnode/impl/test/stream/stream.cpp index 369053c7bb..1e163d1c8e 100644 --- a/source/dnode/mnode/impl/test/stream/stream.cpp +++ b/source/dnode/mnode/impl/test/stream/stream.cpp @@ -189,6 +189,7 @@ TEST_F(StreamTest, kill_checkpoint_trans) { opt.pWal = pMnode->pWal; pMnode->pSdb = sdbInit(&opt); + taosThreadMutexInit(&pMnode->syncMgmt.lock, NULL); } SVgroupChangeInfo info; From 4c258d7187c9e54b23ccd4a221f3e122f8d59d5b Mon Sep 17 00:00:00 2001 From: slzhou Date: Thu, 22 Feb 2024 09:27:53 +0800 Subject: [PATCH 37/41] docs: add implication of para_tables_sort --- docs/en/12-taos-sql/06-select.md | 2 +- docs/zh/12-taos-sql/06-select.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/12-taos-sql/06-select.md b/docs/en/12-taos-sql/06-select.md index 0cd2d06989..a2e6bca46c 100755 --- a/docs/en/12-taos-sql/06-select.md +++ b/docs/en/12-taos-sql/06-select.md @@ -93,7 +93,7 @@ The list of currently supported Hints is as follows: | NO_BATCH_SCAN | None | Sequential table scan | JOIN statment for stable | | SORT_FOR_GROUP| None | Use sort for partition, conflict with PARTITION_FIRST | With normal column in partition by list | | PARTITION_FIRST| None | Use Partition before aggregate, conflict with SORT_FOR_GROUP | With normal column in partition by list | -| PARA_TABLES_SORT| None | When sorting the supertable rows by timestamp, No temporary disk space is used | Sorting the supertable rows by timestamp | +| PARA_TABLES_SORT| None | When sorting the supertable rows by timestamp, No temporary disk space is used. When there are numerous tables, each with long rows, the corresponding algorithm associated with this prompt may consume a substantial amount of memory, potentially leading to an Out Of Memory (OOM) situation. | Sorting the supertable rows by timestamp | For example: diff --git a/docs/zh/12-taos-sql/06-select.md b/docs/zh/12-taos-sql/06-select.md index 43b32efe32..eec947ea23 100755 --- a/docs/zh/12-taos-sql/06-select.md +++ b/docs/zh/12-taos-sql/06-select.md @@ -93,7 +93,7 @@ Hints 是用户控制单个语句查询优化的一种手段,当 Hint 不适 | NO_BATCH_SCAN | 无 | 采用顺序读表的方式 | 超级表 JOIN 语句 | | SORT_FOR_GROUP| 无 | 采用sort方式进行分组, 与PARTITION_FIRST冲突 | partition by 列表有普通列时 | | PARTITION_FIRST| 无 | 在聚合之前使用PARTITION计算分组, 与SORT_FOR_GROUP冲突 | partition by 列表有普通列时 | -| PARA_TABLES_SORT| 无 | 超级表的数据按时间戳排序时, 不使用临时磁盘空间, 只使用内存 | 超级表的数据按时间戳排序时 | +| PARA_TABLES_SORT| 无 | 超级表的数据按时间戳排序时, 不使用临时磁盘空间, 只使用内存。当子表数量多, 行长比较大时候, 会使用大量内存, 可能发生OOM | 超级表的数据按时间戳排序时 | 举例: ```sql From 5c817d101a6ef510788fac05d1a181da01e18d13 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 22 Feb 2024 09:29:40 +0800 Subject: [PATCH 38/41] test(stream): add print info. --- source/dnode/mnode/impl/test/stream/stream.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/source/dnode/mnode/impl/test/stream/stream.cpp b/source/dnode/mnode/impl/test/stream/stream.cpp index 1e163d1c8e..c4449f613b 100644 --- a/source/dnode/mnode/impl/test/stream/stream.cpp +++ b/source/dnode/mnode/impl/test/stream/stream.cpp @@ -222,6 +222,8 @@ TEST_F(StreamTest, kill_checkpoint_trans) { taosArrayPush(stream.tasks, &pLevel); mndCreateStreamResetStatusTrans(pMnode, &stream); + std::cout<<"finish create reset status trans"<< std::endl; + tFreeStreamObj(&stream); sdbCleanup(pMnode->pSdb); taosMemoryFree(pMnode); From 637e6043682a6797d2eb4ea1051df72a4c035043 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 22 Feb 2024 09:38:44 +0800 Subject: [PATCH 39/41] test(stream): add print info. --- source/dnode/mnode/impl/test/stream/stream.cpp | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/source/dnode/mnode/impl/test/stream/stream.cpp b/source/dnode/mnode/impl/test/stream/stream.cpp index c4449f613b..fdc108cbe3 100644 --- a/source/dnode/mnode/impl/test/stream/stream.cpp +++ b/source/dnode/mnode/impl/test/stream/stream.cpp @@ -222,14 +222,21 @@ TEST_F(StreamTest, kill_checkpoint_trans) { taosArrayPush(stream.tasks, &pLevel); mndCreateStreamResetStatusTrans(pMnode, &stream); - std::cout<<"finish create reset status trans"<< std::endl; + std::cout << "finish create reset status trans" << std::endl; tFreeStreamObj(&stream); + + std::cout << "end1" << std::endl; sdbCleanup(pMnode->pSdb); + + std::cout << "end2" << std::endl; + taosMemoryFree(pMnode); taosArrayDestroy(info.pUpdateNodeList); taosHashCleanup(info.pDBMap); + + std::cout << "end3" << std::endl; } TEST_F(StreamTest, plan_Test) { From 045a30035f28c9ed5a638e44ba4b22adcaf51638 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 22 Feb 2024 09:52:47 +0800 Subject: [PATCH 40/41] test(stream): init lock. --- source/dnode/mnode/impl/test/stream/stream.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/source/dnode/mnode/impl/test/stream/stream.cpp b/source/dnode/mnode/impl/test/stream/stream.cpp index fdc108cbe3..7dfb944d55 100644 --- a/source/dnode/mnode/impl/test/stream/stream.cpp +++ b/source/dnode/mnode/impl/test/stream/stream.cpp @@ -216,6 +216,7 @@ TEST_F(StreamTest, kill_checkpoint_trans) { pTask->id.streamId = defStreamId; pTask->id.taskId = 1; pTask->exec.qmsg = (char*)taosMemoryMalloc(1); + taosThreadMutexInit(&pTask->lock, NULL); taosArrayPush(pLevel, &pTask); From dbd3b4e2b92334bd86df363f7b3fd9c59b859340 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 22 Feb 2024 10:02:25 +0800 Subject: [PATCH 41/41] test: remove print info. --- source/dnode/mnode/impl/test/stream/stream.cpp | 8 -------- 1 file changed, 8 deletions(-) diff --git a/source/dnode/mnode/impl/test/stream/stream.cpp b/source/dnode/mnode/impl/test/stream/stream.cpp index 7dfb944d55..8d106b1ede 100644 --- a/source/dnode/mnode/impl/test/stream/stream.cpp +++ b/source/dnode/mnode/impl/test/stream/stream.cpp @@ -223,21 +223,13 @@ TEST_F(StreamTest, kill_checkpoint_trans) { taosArrayPush(stream.tasks, &pLevel); mndCreateStreamResetStatusTrans(pMnode, &stream); - std::cout << "finish create reset status trans" << std::endl; - tFreeStreamObj(&stream); - - std::cout << "end1" << std::endl; sdbCleanup(pMnode->pSdb); - std::cout << "end2" << std::endl; - taosMemoryFree(pMnode); taosArrayDestroy(info.pUpdateNodeList); taosHashCleanup(info.pDBMap); - - std::cout << "end3" << std::endl; } TEST_F(StreamTest, plan_Test) {