diff --git a/include/common/tdatablock.h b/include/common/tdatablock.h index e8fe47a4621828a3c62fb94b1c516947791f7fdb..b6af1ee7a6c486e4cd307d3458286f61ce162174 100644 --- a/include/common/tdatablock.h +++ b/include/common/tdatablock.h @@ -232,7 +232,8 @@ void blockDebugShowData(const SArray* dataBlocks); int32_t buildSubmitReqFromDataBlock(SSubmitReq** pReq, const SArray* pDataBlocks, STSchema* pTSchema, int32_t vgId, tb_uid_t uid, tb_uid_t suid); -SSubmitReq* tdBlockToSubmit(const SArray* pBlocks, const STSchema* pSchema, bool createTb, int64_t suid, int32_t vgId); +SSubmitReq* tdBlockToSubmit(const SArray* pBlocks, const STSchema* pSchema, bool createTb, int64_t suid, + const char* stbFullName, int32_t vgId); static FORCE_INLINE int32_t blockGetEncodeSize(const SSDataBlock* pBlock) { return blockDataGetSerialMetaSize(pBlock) + blockDataGetSize(pBlock); diff --git a/include/common/tmsg.h b/include/common/tmsg.h index bf7ca50bdc4391c0d047c5d855acb5a305fafc79..e20b51aa6afa9fe41a85037256f751d587f1e6fe 100644 --- a/include/common/tmsg.h +++ b/include/common/tmsg.h @@ -1646,8 +1646,8 @@ _err: return NULL; } -// this message is sent from mnode to mnode(read thread to write thread), so there is no need for serialization or -// deserialization +// this message is sent from mnode to mnode(read thread to write thread), +// so there is no need for serialization or deserialization typedef struct { SHashObj* rebSubHash; // SHashObj } SMqDoRebalanceMsg; diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 55e8cf00500288add743fdddcc1b38ceefa855f1..ae25e1bffd8892a3014dfd18167d6e8a038d27b3 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -142,6 +142,7 @@ typedef void FTbSink(SStreamTask* pTask, void* vnode, int64_t ver, void* data); typedef struct { int64_t stbUid; + char stbFullName[TSDB_TABLE_FNAME_LEN]; SSchemaWrapper* pSchemaWrapper; // not applicable to encoder and decoder void* vnode; diff --git a/include/util/tdigest.h b/include/util/tdigest.h new file mode 100644 index 0000000000000000000000000000000000000000..f9b615318f5c33f0cf386653367ddfe36ae759f8 --- /dev/null +++ b/include/util/tdigest.h @@ -0,0 +1,71 @@ +/* + * Copyright (c) 2019 TAOS Data, Inc. + * + * This program is free software: you can use, redistribute, and/or modify + * it under the terms of the GNU Affero General Public License, version 3 + * or later ("AGPL"), as published by the Free Software Foundation. + * + * This program is distributed in the hope that it will be useful, but WITHOUT + * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or + * FITNESS FOR A PARTICULAR PURPOSE. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + */ + +/* + * include/tdigest.c + * + * Copyright (c) 2016, Usman Masood + */ + +#ifndef TDIGEST_H +#define TDIGEST_H + +#ifndef M_PI +#define M_PI 3.14159265358979323846264338327950288 /* pi */ +#endif + +#define DOUBLE_MAX 1.79e+308 + +#define ADDITION_CENTROID_NUM 2 +#define COMPRESSION 300 +#define GET_CENTROID(compression) (ceil(compression * M_PI / 2) + 1 + ADDITION_CENTROID_NUM) +#define GET_THRESHOLD(compression) (7.5 + 0.37 * compression - 2e-4 * pow(compression, 2)) +#define TDIGEST_SIZE(compression) (sizeof(TDigest) + sizeof(SCentroid)*GET_CENTROID(compression) + sizeof(SPt)*GET_THRESHOLD(compression)) + +typedef struct SCentroid { + double mean; + int64_t weight; +}SCentroid; + +typedef struct SPt { + double value; + int64_t weight; +}SPt; + +typedef struct TDigest { + double compression; + int32_t threshold; + int64_t size; + + int64_t total_weight; + double min; + double max; + + int32_t num_buffered_pts; + SPt *buffered_pts; + + int32_t num_centroids; + SCentroid *centroids; +}TDigest; + +TDigest *tdigestNewFrom(void* pBuf, int32_t compression); +void tdigestAdd(TDigest *t, double x, int64_t w); +void tdigestMerge(TDigest *t1, TDigest *t2); +double tdigestQuantile(TDigest *t, double q); +void tdigestCompress(TDigest *t); +void tdigestFreeFrom(TDigest *t); +void tdigestAutoFill(TDigest* t, int32_t compression); + +#endif /* TDIGEST_H */ diff --git a/source/common/src/tdatablock.c b/source/common/src/tdatablock.c index 61ac01e4fad52d325abe1e76834f2d843065c11b..52169226436b3aa75fe76ca073de756b7b227550 100644 --- a/source/common/src/tdatablock.c +++ b/source/common/src/tdatablock.c @@ -1630,7 +1630,7 @@ int32_t buildSubmitReqFromDataBlock(SSubmitReq** pReq, const SArray* pDataBlocks } SSubmitReq* tdBlockToSubmit(const SArray* pBlocks, const STSchema* pTSchema, bool createTb, int64_t suid, - int32_t vgId) { + const char* stbFullName, int32_t vgId) { SSubmitReq* ret = NULL; // cal size @@ -1646,10 +1646,12 @@ SSubmitReq* tdBlockToSubmit(const SArray* pBlocks, const STSchema* pTSchema, boo if (createTb) { SVCreateTbReq createTbReq = {0}; - createTbReq.name = "a"; + char* cname = taosMemoryCalloc(1, TSDB_TABLE_FNAME_LEN); + snprintf(cname, TSDB_TABLE_FNAME_LEN, "%s:%ld", stbFullName, pDataBlock->info.groupId); + createTbReq.name = cname; createTbReq.flags = 0; createTbReq.type = TSDB_CHILD_TABLE; - createTbReq.ctb.suid = htobe64(suid); + createTbReq.ctb.suid = suid; SKVRowBuilder kvRowBuilder = {0}; if (tdInitKVRowBuilder(&kvRowBuilder) < 0) { @@ -1662,6 +1664,7 @@ SSubmitReq* tdBlockToSubmit(const SArray* pBlocks, const STSchema* pTSchema, boo int32_t code; tEncodeSize(tEncodeSVCreateTbReq, &createTbReq, schemaLen, code); if (code < 0) return NULL; + taosMemoryFree(cname); } cap += sizeof(SSubmitBlk) + schemaLen + rows * maxLen; @@ -1697,7 +1700,9 @@ SSubmitReq* tdBlockToSubmit(const SArray* pBlocks, const STSchema* pTSchema, boo int32_t schemaLen = 0; if (createTb) { SVCreateTbReq createTbReq = {0}; - createTbReq.name = "a"; + char* cname = taosMemoryCalloc(1, TSDB_TABLE_FNAME_LEN); + snprintf(cname, TSDB_TABLE_FNAME_LEN, "%s:%ld", stbFullName, pDataBlock->info.groupId); + createTbReq.name = cname; createTbReq.flags = 0; createTbReq.type = TSDB_CHILD_TABLE; createTbReq.ctb.suid = suid; diff --git a/source/common/src/tmsg.c b/source/common/src/tmsg.c index c300790f5720dd5c2d4f5d8319815a19afc20905..19ba42332e80cc08aaedc966f38b506072fc9bd6 100644 --- a/source/common/src/tmsg.c +++ b/source/common/src/tmsg.c @@ -3702,6 +3702,7 @@ int32_t tSerializeSCMCreateStreamReq(void *buf, int32_t bufLen, const SCMCreateS if (tStartEncode(&encoder) < 0) return -1; if (tEncodeCStr(&encoder, pReq->name) < 0) return -1; + if (tEncodeCStr(&encoder, pReq->sourceDB) < 0) return -1; if (tEncodeCStr(&encoder, pReq->targetStbFullName) < 0) return -1; if (tEncodeI8(&encoder, pReq->igExists) < 0) return -1; if (tEncodeI32(&encoder, sqlLen) < 0) return -1; @@ -3727,6 +3728,7 @@ int32_t tDeserializeSCMCreateStreamReq(void *buf, int32_t bufLen, SCMCreateStrea if (tStartDecode(&decoder) < 0) return -1; if (tDecodeCStrTo(&decoder, pReq->name) < 0) return -1; + if (tDecodeCStrTo(&decoder, pReq->sourceDB) < 0) return -1; if (tDecodeCStrTo(&decoder, pReq->targetStbFullName) < 0) return -1; if (tDecodeI8(&decoder, &pReq->igExists) < 0) return -1; if (tDecodeI32(&decoder, &sqlLen) < 0) return -1; diff --git a/source/common/src/tname.c b/source/common/src/tname.c index 104dee261c9f64c7c8859228dcb0595f4b4df2c0..fd055135799a5e508ec535b43d46e9246c8d644e 100644 --- a/source/common/src/tname.c +++ b/source/common/src/tname.c @@ -127,7 +127,7 @@ int32_t tNameExtractFullName(const SName* name, char* dst) { size_t tnameLen = strlen(name->tname); if (tnameLen > 0) { - assert(name->type == TSDB_TABLE_NAME_T); + /*assert(name->type == TSDB_TABLE_NAME_T);*/ dst[len] = TS_PATH_DELIMITER[0]; memcpy(dst + len + 1, name->tname, tnameLen); @@ -314,9 +314,9 @@ void buildChildTableName(RandTableName* rName) { for (int j = 0; j < taosArrayGetSize(rName->tags); ++j) { SSmlKv* tagKv = taosArrayGetP(rName->tags, j); taosStringBuilderAppendStringLen(&sb, tagKv->key, tagKv->keyLen); - if(IS_VAR_DATA_TYPE(tagKv->type)){ + if (IS_VAR_DATA_TYPE(tagKv->type)) { taosStringBuilderAppendStringLen(&sb, tagKv->value, tagKv->length); - }else{ + } else { taosStringBuilderAppendStringLen(&sb, (char*)(&(tagKv->value)), tagKv->length); } } diff --git a/source/dnode/mnode/impl/src/mndScheduler.c b/source/dnode/mnode/impl/src/mndScheduler.c index 22a5f37334b4f18a422249afa9e870068e0e5f83..516e89af8ddcafe75035a16352afc3ce496e2bd3 100644 --- a/source/dnode/mnode/impl/src/mndScheduler.c +++ b/source/dnode/mnode/impl/src/mndScheduler.c @@ -206,6 +206,7 @@ int32_t mndAddShuffledSinkToStream(SMnode* pMnode, STrans* pTrans, SStreamObj* p } else { pTask->sinkType = TASK_SINK__TABLE; pTask->tbSink.stbUid = pStream->targetStbUid; + memcpy(pTask->tbSink.stbFullName, pStream->targetSTbName, TSDB_TABLE_FNAME_LEN); pTask->tbSink.pSchemaWrapper = tCloneSSchemaWrapper(&pStream->outputSchema); ASSERT(pTask->tbSink.pSchemaWrapper); } @@ -248,6 +249,7 @@ int32_t mndAddFixedSinkToStream(SMnode* pMnode, STrans* pTrans, SStreamObj* pStr } else { pTask->sinkType = TASK_SINK__TABLE; pTask->tbSink.stbUid = pStream->targetStbUid; + memcpy(pTask->tbSink.stbFullName, pStream->targetSTbName, TSDB_TABLE_FNAME_LEN); pTask->tbSink.pSchemaWrapper = tCloneSSchemaWrapper(&pStream->outputSchema); } @@ -325,6 +327,7 @@ int32_t mndScheduleStream(SMnode* pMnode, STrans* pTrans, SStreamObj* pStream) { } else { pTask->sinkType = TASK_SINK__TABLE; pTask->tbSink.stbUid = pStream->targetStbUid; + memcpy(pTask->tbSink.stbFullName, pStream->targetSTbName, TSDB_TABLE_FNAME_LEN); pTask->tbSink.pSchemaWrapper = tCloneSSchemaWrapper(&pStream->outputSchema); } #endif diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index 9de6138689b43b195ea004c6ba6a7fa489d4060f..cbef1facdcd5c1a680c90b3f11936316e12a2a4f 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -456,7 +456,7 @@ static int32_t mndProcessCreateStreamReq(SRpcMsg *pReq) { goto CREATE_STREAM_OVER; } - pDb = mndAcquireDbByStream(pMnode, createStreamReq.name); + pDb = mndAcquireDb(pMnode, createStreamReq.sourceDB); if (pDb == NULL) { terrno = TSDB_CODE_MND_DB_NOT_SELECTED; goto CREATE_STREAM_OVER; diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 0e8835357ac31612020ed792e32c5f44182dde2e..7bfb43bfb16cb016a5fc6eadc9d577a3dbd1d9eb 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -748,7 +748,8 @@ void tqTableSink(SStreamTask* pTask, void* vnode, int64_t ver, void* data) { SVnode* pVnode = (SVnode*)vnode; ASSERT(pTask->tbSink.pTSchema); - SSubmitReq* pReq = tdBlockToSubmit(pRes, pTask->tbSink.pTSchema, true, pTask->tbSink.stbUid, pVnode->config.vgId); + SSubmitReq* pReq = tdBlockToSubmit(pRes, pTask->tbSink.pTSchema, true, pTask->tbSink.stbUid, + pTask->tbSink.stbFullName, pVnode->config.vgId); /*tPrintFixedSchemaSubmitReq(pReq, pTask->tbSink.pTSchema);*/ // build write msg SRpcMsg msg = { diff --git a/source/libs/function/inc/builtinsimpl.h b/source/libs/function/inc/builtinsimpl.h index d041e08d35be44d97feb5ec09642f3543b002c44..a8eccb57e9d42a038cf9382305db143c40ff35b3 100644 --- a/source/libs/function/inc/builtinsimpl.h +++ b/source/libs/function/inc/builtinsimpl.h @@ -73,6 +73,11 @@ bool percentileFunctionSetup(SqlFunctionCtx *pCtx, SResultRowEntryInfo* pResultI int32_t percentileFunction(SqlFunctionCtx *pCtx); int32_t percentileFinalize(SqlFunctionCtx* pCtx, SSDataBlock* pBlock); +bool getApercentileFuncEnv(struct SFunctionNode* pFunc, SFuncExecEnv* pEnv); +bool apercentileFunctionSetup(SqlFunctionCtx *pCtx, SResultRowEntryInfo* pResultInfo); +int32_t apercentileFunction(SqlFunctionCtx *pCtx); +int32_t apercentileFinalize(SqlFunctionCtx* pCtx, SSDataBlock* pBlock); + bool getDiffFuncEnv(struct SFunctionNode* pFunc, SFuncExecEnv* pEnv); bool diffFunctionSetup(SqlFunctionCtx *pCtx, SResultRowEntryInfo* pResInfo); int32_t diffFunction(SqlFunctionCtx *pCtx); diff --git a/source/libs/function/src/builtins.c b/source/libs/function/src/builtins.c index 68ff7bdf3eff3a431328f1e3edcfa3066e6e947f..8914dca639c0c1c21bacb923860ef69a66e1deb8 100644 --- a/source/libs/function/src/builtins.c +++ b/source/libs/function/src/builtins.c @@ -760,19 +760,32 @@ static int32_t translateSubstr(SFunctionNode* pFunc, char* pErrBuf, int32_t len) return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName); } - SExprNode* pPara1 = (SExprNode*)nodesListGetNode(pFunc->pParameterList, 0); - uint8_t para2Type = ((SExprNode*)nodesListGetNode(pFunc->pParameterList, 1))->resType.type; - if (!IS_VAR_DATA_TYPE(pPara1->resType.type) || !IS_INTEGER_TYPE(para2Type)) { + SExprNode* pPara0 = (SExprNode*)nodesListGetNode(pFunc->pParameterList, 0); + SExprNode* p1 = (SExprNode*)nodesListGetNode(pFunc->pParameterList, 1); + + uint8_t para1Type = p1->resType.type; + if (!IS_VAR_DATA_TYPE(pPara0->resType.type) || !IS_INTEGER_TYPE(para1Type)) { return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName); } + + if (((SValueNode*)p1)->datum.i < 1) { + return invaildFuncParaValueErrMsg(pErrBuf, len, pFunc->functionName); + } + if (3 == numOfParams) { - uint8_t para3Type = ((SExprNode*)nodesListGetNode(pFunc->pParameterList, 1))->resType.type; - if (!IS_INTEGER_TYPE(para3Type)) { + SExprNode* p2 = (SExprNode*)nodesListGetNode(pFunc->pParameterList, 2); + uint8_t para2Type = p2->resType.type; + if (!IS_INTEGER_TYPE(para2Type)) { return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName); } + + int64_t v = ((SValueNode*)p1)->datum.i; + if (v < 0 || v > INT16_MAX) { + return invaildFuncParaValueErrMsg(pErrBuf, len, pFunc->functionName); + } } - pFunc->node.resType = (SDataType){.bytes = pPara1->resType.bytes, .type = pPara1->resType.type}; + pFunc->node.resType = (SDataType){.bytes = pPara0->resType.bytes, .type = pPara0->resType.type}; return TSDB_CODE_SUCCESS; } @@ -985,10 +998,10 @@ const SBuiltinFuncDefinition funcMgtBuiltins[] = { .type = FUNCTION_TYPE_APERCENTILE, .classification = FUNC_MGT_AGG_FUNC, .translateFunc = translateApercentile, - .getEnvFunc = getMinmaxFuncEnv, - .initFunc = minmaxFunctionSetup, - .processFunc = maxFunction, - .finalizeFunc = functionFinalize + .getEnvFunc = getApercentileFuncEnv, + .initFunc = apercentileFunctionSetup, + .processFunc = apercentileFunction, + .finalizeFunc = apercentileFinalize }, { .name = "top", diff --git a/source/libs/function/src/builtinsimpl.c b/source/libs/function/src/builtinsimpl.c index da842877dcdb9afcb2d32dbf47dada1864a35972..fe5737220b5cdeb57820562cc70d68956c05d339 100644 --- a/source/libs/function/src/builtinsimpl.c +++ b/source/libs/function/src/builtinsimpl.c @@ -20,6 +20,8 @@ #include "taggfunction.h" #include "tcompare.h" #include "tdatablock.h" +#include "tdigest.h" +#include "thistogram.h" #include "tpercentile.h" #define HISTOGRAM_MAX_BINS_NUM 1000 @@ -95,6 +97,19 @@ typedef struct SPercentileInfo { int64_t numOfElems; } SPercentileInfo; +typedef struct SAPercentileInfo { + double result; + int8_t algo; + SHistogramInfo *pHisto; + TDigest *pTDigest; +} SAPercentileInfo; + +typedef enum { + APERCT_ALGO_UNKNOWN = 0, + APERCT_ALGO_DEFAULT, + APERCT_ALGO_TDIGEST, +} EAPerctAlgoType; + typedef struct SDiffInfo { bool hasPrev; bool includeNull; @@ -1905,6 +1920,131 @@ int32_t percentileFinalize(SqlFunctionCtx* pCtx, SSDataBlock* pBlock) { return functionFinalize(pCtx, pBlock); } +bool getApercentileFuncEnv(SFunctionNode* pFunc, SFuncExecEnv* pEnv) { + int32_t bytesHist = (int32_t)(sizeof(SAPercentileInfo) + sizeof(SHistogramInfo) + sizeof(SHistBin) * (MAX_HISTOGRAM_BIN + 1)); + int32_t bytesDigest = (int32_t)(sizeof(SAPercentileInfo) + TDIGEST_SIZE(COMPRESSION)); + pEnv->calcMemSize = TMAX(bytesHist, bytesDigest); + return true; +} + +static int8_t getApercentileAlgo(char *algoStr) { + int8_t algoType; + if (strcasecmp(algoStr, "default") == 0) { + algoType = APERCT_ALGO_DEFAULT; + } else if (strcasecmp(algoStr, "t-digest") == 0) { + algoType = APERCT_ALGO_TDIGEST; + } else { + algoType = APERCT_ALGO_UNKNOWN; + } + + return algoType; +} + +static void buildHistogramInfo(SAPercentileInfo* pInfo) { + pInfo->pHisto = (SHistogramInfo*) ((char*) pInfo + sizeof(SAPercentileInfo)); + pInfo->pHisto->elems = (SHistBin*) ((char*)pInfo->pHisto + sizeof(SHistogramInfo)); +} + +bool apercentileFunctionSetup(SqlFunctionCtx* pCtx, SResultRowEntryInfo* pResultInfo) { + if (!functionSetup(pCtx, pResultInfo)) { + return false; + } + + SAPercentileInfo* pInfo = GET_ROWCELL_INTERBUF(pResultInfo); + if (pCtx->numOfParams == 2) { + pInfo->algo = APERCT_ALGO_DEFAULT; + } else if (pCtx->numOfParams == 3) { + pInfo->algo = getApercentileAlgo(pCtx->param[2].param.pz); + if (pInfo->algo == APERCT_ALGO_UNKNOWN) { + return false; + } + } + + char *tmp = (char *)pInfo + sizeof(SAPercentileInfo); + if (pInfo->algo == APERCT_ALGO_TDIGEST) { + pInfo->pTDigest = tdigestNewFrom(tmp, COMPRESSION); + } else { + buildHistogramInfo(pInfo); + pInfo->pHisto = tHistogramCreateFrom(tmp, MAX_HISTOGRAM_BIN); + } + + return true; +} + +int32_t apercentileFunction(SqlFunctionCtx* pCtx) { + int32_t notNullElems = 0; + SResultRowEntryInfo* pResInfo = GET_RES_INFO(pCtx); + + SInputColumnInfoData* pInput = &pCtx->input; + //SColumnDataAgg* pAgg = pInput->pColumnDataAgg[0]; + + SColumnInfoData* pCol = pInput->pData[0]; + int32_t type = pCol->info.type; + + SAPercentileInfo* pInfo = GET_ROWCELL_INTERBUF(pResInfo); + + int32_t start = pInput->startRowIndex; + if (pInfo->algo == APERCT_ALGO_TDIGEST) { + for (int32_t i = start; i < pInput->numOfRows + start; ++i) { + if (colDataIsNull_f(pCol->nullbitmap, i)) { + continue; + } + notNullElems += 1; + char* data = colDataGetData(pCol, i); + + double v = 0; // value + int64_t w = 1; // weigth + GET_TYPED_DATA(v, double, type, data); + tdigestAdd(pInfo->pTDigest, v, w); + } + } else { + for (int32_t i = start; i < pInput->numOfRows + start; ++i) { + if (colDataIsNull_f(pCol->nullbitmap, i)) { + continue; + } + notNullElems += 1; + char* data = colDataGetData(pCol, i); + + double v = 0; + GET_TYPED_DATA(v, double, type, data); + tHistogramAdd(&pInfo->pHisto, v); + } + } + + SET_VAL(pResInfo, notNullElems, 1); + return TSDB_CODE_SUCCESS; +} + +int32_t apercentileFinalize(SqlFunctionCtx* pCtx, SSDataBlock* pBlock) { + SVariant* pVal = &pCtx->param[1].param; + double percent = (pVal->nType == TSDB_DATA_TYPE_BIGINT) ? pVal->i : pVal->d; + + SResultRowEntryInfo* pResInfo = GET_RES_INFO(pCtx); + SAPercentileInfo* pInfo = (SAPercentileInfo*)GET_ROWCELL_INTERBUF(pResInfo); + + if (pInfo->algo == APERCT_ALGO_TDIGEST) { + if (pInfo->pTDigest->size > 0) { + pInfo->result = tdigestQuantile(pInfo->pTDigest, percent/100); + } else { // no need to free + //setNull(pCtx->pOutput, pCtx->outputType, pCtx->outputBytes); + return TSDB_CODE_SUCCESS; + } + } else { + if (pInfo->pHisto->numOfElems > 0) { + double ratio[] = {percent}; + double *res = tHistogramUniform(pInfo->pHisto, ratio, 1); + pInfo->result = *res; + //memcpy(pCtx->pOutput, res, sizeof(double)); + taosMemoryFree(res); + } else { // no need to free + //setNull(pCtx->pOutput, pCtx->outputType, pCtx->outputBytes); + return TSDB_CODE_SUCCESS; + } + } + + return functionFinalize(pCtx, pBlock); +} + bool getFirstLastFuncEnv(SFunctionNode* pFunc, SFuncExecEnv* pEnv) { SColumnNode* pNode = nodesListGetNode(pFunc->pParameterList, 0); pEnv->calcMemSize = pNode->node.resType.bytes + sizeof(int64_t); @@ -1917,8 +2057,6 @@ bool getSelectivityFuncEnv(SFunctionNode* pFunc, SFuncExecEnv* pEnv) { return true; } - - static FORCE_INLINE TSKEY getRowPTs(SColumnInfoData* pTsColInfo, int32_t rowIndex) { if (pTsColInfo == NULL) { return 0; diff --git a/source/libs/parser/src/parInsert.c b/source/libs/parser/src/parInsert.c index ad36e06117ae3929ab9efe30a9eb87317dd7e086..b5d97a80e5037ea3e3ad1797fc2dfe914b4e41a1 100644 --- a/source/libs/parser/src/parInsert.c +++ b/source/libs/parser/src/parInsert.c @@ -1120,12 +1120,12 @@ static int32_t parseInsertBody(SInsertParseContext* pCxt) { NEXT_TOKEN(pCxt->pSql, sToken); SName name; - createSName(&name, &tbnameToken, pCxt->pComCxt->acctId, pCxt->pComCxt->db, &pCxt->msg); - tNameExtractFullName(&name, tbFName); + CHECK_CODE(createSName(&name, &tbnameToken, pCxt->pComCxt->acctId, pCxt->pComCxt->db, &pCxt->msg)); + tNameExtractFullName(&name, tbFName); CHECK_CODE(taosHashPut(pCxt->pTableNameHashObj, tbFName, strlen(tbFName), &name, sizeof(SName))); - // USING cluase + // USING clause if (TK_USING == sToken.type) { CHECK_CODE(parseUsingClause(pCxt, &name, tbFName)); NEXT_TOKEN(pCxt->pSql, sToken); diff --git a/source/libs/parser/src/parTranslater.c b/source/libs/parser/src/parTranslater.c index 559fe3e85a7e37b1a811c757c965448a9b9bfabf..340153f5f0559a14997cd56bdb5a55b8cf674e56 100644 --- a/source/libs/parser/src/parTranslater.c +++ b/source/libs/parser/src/parTranslater.c @@ -3386,9 +3386,9 @@ static int32_t buildCreateStreamReq(STranslateContext* pCxt, SCreateStreamStmt* pReq->igExists = pStmt->ignoreExists; SName name; - // tNameSetDbName(&name, pCxt->pParseCxt->acctId, pStmt->streamName, strlen(pStmt->streamName)); - // tNameGetFullDbName(&name, pReq->name); - tNameExtractFullName(toName(pCxt->pParseCxt->acctId, pCxt->pParseCxt->db, pStmt->streamName, &name), pReq->name); + tNameSetDbName(&name, pCxt->pParseCxt->acctId, pStmt->streamName, strlen(pStmt->streamName)); + tNameGetFullDbName(&name, pReq->name); + // tNameExtractFullName(toName(pCxt->pParseCxt->acctId, pCxt->pParseCxt->db, pStmt->streamName, &name), pReq->name); if ('\0' != pStmt->targetTabName[0]) { strcpy(name.dbname, pStmt->targetDbName); diff --git a/source/libs/stream/src/tstream.c b/source/libs/stream/src/tstream.c index 933b37825fdad07cd25c63b885868c62f771bfdb..775a185da7b6304e2f9f03d248336db170a82f55 100644 --- a/source/libs/stream/src/tstream.c +++ b/source/libs/stream/src/tstream.c @@ -505,6 +505,7 @@ int32_t tEncodeSStreamTask(SEncoder* pEncoder, const SStreamTask* pTask) { if (pTask->sinkType == TASK_SINK__TABLE) { if (tEncodeI64(pEncoder, pTask->tbSink.stbUid) < 0) return -1; + if (tEncodeCStr(pEncoder, pTask->tbSink.stbFullName) < 0) return -1; if (tEncodeSSchemaWrapper(pEncoder, pTask->tbSink.pSchemaWrapper) < 0) return -1; } else if (pTask->sinkType == TASK_SINK__SMA) { if (tEncodeI64(pEncoder, pTask->smaSink.smaId) < 0) return -1; @@ -551,6 +552,7 @@ int32_t tDecodeSStreamTask(SDecoder* pDecoder, SStreamTask* pTask) { if (pTask->sinkType == TASK_SINK__TABLE) { if (tDecodeI64(pDecoder, &pTask->tbSink.stbUid) < 0) return -1; + if (tDecodeCStrTo(pDecoder, pTask->tbSink.stbFullName) < 0) return -1; pTask->tbSink.pSchemaWrapper = taosMemoryCalloc(1, sizeof(SSchemaWrapper)); if (pTask->tbSink.pSchemaWrapper == NULL) return -1; if (tDecodeSSchemaWrapper(pDecoder, pTask->tbSink.pSchemaWrapper) < 0) return -1; diff --git a/source/util/src/tdigest.c b/source/util/src/tdigest.c new file mode 100644 index 0000000000000000000000000000000000000000..56b113fd8f166aae397e05ef3fed40e4df00309a --- /dev/null +++ b/source/util/src/tdigest.c @@ -0,0 +1,319 @@ +/* + * Copyright (c) 2019 TAOS Data, Inc. + * + * This program is free software: you can use, redistribute, and/or modify + * it under the terms of the GNU Affero General Public License, version 3 + * or later ("AGPL"), as published by the Free Software Foundation. + * + * This program is distributed in the hope that it will be useful, but WITHOUT + * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or + * FITNESS FOR A PARTICULAR PURPOSE. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + */ + +/* + * src/tdigest.c + * + * Implementation of the t-digest data structure used to compute accurate percentiles. + * + * It is based on the MergingDigest implementation found at: + * https://github.com/tdunning/t-digest/blob/master/src/main/java/com/tdunning/math/stats/MergingDigest.java + * + * Copyright (c) 2016, Usman Masood + */ + +#include "os.h" +#include "osMath.h" +#include "tdigest.h" + +#define INTERPOLATE(x, x0, x1) (((x) - (x0)) / ((x1) - (x0))) +//#define INTEGRATED_LOCATION(compression, q) ((compression) * (asin(2 * (q) - 1) + M_PI / 2) / M_PI) +#define INTEGRATED_LOCATION(compression, q) ((compression) * (asin(2 * (double)(q) - 1)/M_PI + (double)1/2)) +#define FLOAT_EQ(f1, f2) (fabs((f1) - (f2)) <= FLT_EPSILON) + +typedef struct SMergeArgs { + TDigest *t; + SCentroid *centroids; + int32_t idx; + double weight_so_far; + double k1; + double min; + double max; +}SMergeArgs; + +void tdigestAutoFill(TDigest* t, int32_t compression) { + t->centroids = (SCentroid*)((char*)t + sizeof(TDigest)); + t->buffered_pts = (SPt*) ((char*)t + sizeof(TDigest) + sizeof(SCentroid) * (int32_t)GET_CENTROID(compression)); +} + +TDigest *tdigestNewFrom(void* pBuf, int32_t compression) { + memset(pBuf, 0, (size_t)TDIGEST_SIZE(compression)); + TDigest* t = (TDigest*)pBuf; + tdigestAutoFill(t, compression); + + t->compression = compression; + t->size = (int64_t)GET_CENTROID(compression); + t->threshold = (int32_t)GET_THRESHOLD(compression); + t->min = DOUBLE_MAX; + t->max = -DOUBLE_MAX; + + return t; +} + +static int32_t cmpCentroid(const void *a, const void *b) { + SCentroid *c1 = (SCentroid *) a; + SCentroid *c2 = (SCentroid *) b; + if (c1->mean < c2->mean) + return -1; + if (c1->mean > c2->mean) + return 1; + return 0; +} + + +static void mergeCentroid(SMergeArgs *args, SCentroid *merge) { + double k2; + SCentroid *c = &args->centroids[args->idx]; + + args->weight_so_far += merge->weight; + k2 = INTEGRATED_LOCATION(args->t->size, + args->weight_so_far / args->t->total_weight); + //idx++ + if(k2 - args->k1 > 1 && c->weight > 0) { + if(args->idx + 1 < args->t->size + && merge->mean != args->centroids[args->idx].mean) { + args->idx++; + } + args->k1 = k2; + } + + c = &args->centroids[args->idx]; + if(c->mean == merge->mean) { + c->weight += merge->weight; + } else { + c->weight += merge->weight; + c->mean += (merge->mean - c->mean) * merge->weight / c->weight; + + if (merge->weight > 0) { + args->min = TMIN(merge->mean, args->min); + args->max = TMAX(merge->mean, args->max); + } + } +} + +void tdigestCompress(TDigest *t) { + SCentroid *unmerged_centroids; + int64_t unmerged_weight = 0; + int32_t num_unmerged = t->num_buffered_pts; + int32_t i, j; + SMergeArgs args; + + if (t->num_buffered_pts <= 0) + return; + + unmerged_centroids = (SCentroid*)taosMemoryMalloc(sizeof(SCentroid) * t->num_buffered_pts); + for (i = 0; i < num_unmerged; i++) { + SPt *p = t->buffered_pts + i; + SCentroid *c = &unmerged_centroids[i]; + c->mean = p->value; + c->weight = p->weight; + unmerged_weight += c->weight; + } + t->num_buffered_pts = 0; + t->total_weight += unmerged_weight; + + qsort(unmerged_centroids, num_unmerged, sizeof(SCentroid), cmpCentroid); + memset(&args, 0, sizeof(SMergeArgs)); + args.centroids = (SCentroid*)taosMemoryMalloc((size_t)(sizeof(SCentroid) * t->size)); + memset(args.centroids, 0, (size_t)(sizeof(SCentroid) * t->size)); + + args.t = t; + args.min = DOUBLE_MAX; + args.max = -DOUBLE_MAX; + + i = 0; + j = 0; + while (i < num_unmerged && j < t->num_centroids) { + SCentroid *a = &unmerged_centroids[i]; + SCentroid *b = &t->centroids[j]; + + if (a->mean <= b->mean) { + mergeCentroid(&args, a); + assert(args.idx < t->size); + i++; + } else { + mergeCentroid(&args, b); + assert(args.idx < t->size); + j++; + } + } + + while (i < num_unmerged) { + mergeCentroid(&args, &unmerged_centroids[i++]); + assert(args.idx < t->size); + } + taosMemoryFree((void*)unmerged_centroids); + + while (j < t->num_centroids) { + mergeCentroid(&args, &t->centroids[j++]); + assert(args.idx < t->size); + } + + if (t->total_weight > 0) { + t->min = TMIN(t->min, args.min); + if (args.centroids[args.idx].weight <= 0) { + args.idx--; + } + t->num_centroids = args.idx + 1; + t->max = TMAX(t->max, args.max); + } + + memcpy(t->centroids, args.centroids, sizeof(SCentroid) * t->num_centroids); + taosMemoryFree((void*)args.centroids); +} + +void tdigestAdd(TDigest* t, double x, int64_t w) { + if (w == 0) + return; + + int32_t i = t->num_buffered_pts; + if(i > 0 && t->buffered_pts[i-1].value == x ) { + t->buffered_pts[i].weight = w; + } else { + t->buffered_pts[i].value = x; + t->buffered_pts[i].weight = w; + t->num_buffered_pts++; + } + + + if (t->num_buffered_pts >= t->threshold) + tdigestCompress(t); +} + +double tdigestCDF(TDigest *t, double x) { + if (t == NULL) + return 0; + + int32_t i; + double left, right; + int64_t weight_so_far; + SCentroid *a, *b, tmp; + + tdigestCompress(t); + if (t->num_centroids == 0) + return NAN; + if (x < t->min) + return 0; + if (x > t->max) + return 1; + if (t->num_centroids == 1) { + if (FLOAT_EQ(t->max, t->min)) + return 0.5; + + return INTERPOLATE(x, t->min, t->max); + } + + weight_so_far = 0; + a = b = &tmp; + b->mean = t->min; + b->weight = 0; + right = 0; + + for (i = 0; i < t->num_centroids; i++) { + SCentroid *c = &t->centroids[i]; + + left = b->mean - (a->mean + right); + a = b; + b = c; + right = (b->mean - a->mean) * a->weight / (a->weight + b->weight); + + if (x < a->mean + right) { + double cdf = (weight_so_far + + a->weight + * INTERPOLATE(x, a->mean - left, a->mean + right)) + / t->total_weight; + return TMAX(cdf, 0.0); + } + + weight_so_far += a->weight; + } + + left = b->mean - (a->mean + right); + a = b; + right = t->max - a->mean; + + if (x < a->mean + right) { + return (weight_so_far + a->weight * INTERPOLATE(x, a->mean - left, a->mean + right)) + / t->total_weight; + } + + return 1; +} + +double tdigestQuantile(TDigest *t, double q) { + if (t == NULL) + return 0; + + int32_t i; + double left, right, idx; + int64_t weight_so_far; + SCentroid *a, *b, tmp; + + tdigestCompress(t); + if (t->num_centroids == 0) + return NAN; + if (t->num_centroids == 1) + return t->centroids[0].mean; + if (FLOAT_EQ(q, 0.0)) + return t->min; + if (FLOAT_EQ(q, 1.0)) + return t->max; + + idx = q * t->total_weight; + weight_so_far = 0; + b = &tmp; + b->mean = t->min; + b->weight = 0; + right = t->min; + + for (i = 0; i < t->num_centroids; i++) { + SCentroid *c = &t->centroids[i]; + a = b; + left = right; + + b = c; + right = (b->weight * a->mean + a->weight * b->mean)/ (a->weight + b->weight); + if (idx < weight_so_far + a->weight) { + double p = (idx - weight_so_far) / a->weight; + return left * (1 - p) + right * p; + } + weight_so_far += a->weight; + } + + left = right; + a = b; + right = t->max; + + if (idx < weight_so_far + a->weight && a->weight != 0) { + double p = (idx - weight_so_far) / a->weight; + return left * (1 - p) + right * p; + } + + return t->max; +} + +void tdigestMerge(TDigest *t1, TDigest *t2) { + // SPoints + int32_t num_pts = t2->num_buffered_pts; + for(int32_t i = num_pts - 1; i >= 0; i--) { + SPt* p = t2->buffered_pts + i; + tdigestAdd(t1, p->value, p->weight); + t2->num_buffered_pts --; + } + // centroids + for (int32_t i = 0; i < t2->num_centroids; i++) { + tdigestAdd(t1, t2->centroids[i].mean, t2->centroids[i].weight); + } +} diff --git a/tests/system-test/7-tmq/tmqDnode.py b/tests/system-test/7-tmq/tmqDnode.py new file mode 100644 index 0000000000000000000000000000000000000000..4200b357a7e314720d9c5aeff8199a07dbcd45dd --- /dev/null +++ b/tests/system-test/7-tmq/tmqDnode.py @@ -0,0 +1,1457 @@ + +import taos +import sys +import time +import socket +import os +import threading +from enum import Enum + +from util.log import * +from util.sql import * +from util.cases import * +from util.dnodes import * + +class actionType(Enum): + CREATE_DATABASE = 0 + CREATE_STABLE = 1 + CREATE_CTABLE = 2 + INSERT_DATA = 3 + +class TDTestCase: + hostname = socket.gethostname() + #rpcDebugFlagVal = '143' + #clientCfgDict = {'serverPort': '', 'firstEp': '', 'secondEp':'', 'rpcDebugFlag':'135', 'fqdn':''} + #clientCfgDict["rpcDebugFlag"] = rpcDebugFlagVal + #updatecfgDict = {'clientCfg': {}, 'serverPort': '', 'firstEp': '', 'secondEp':'', 'rpcDebugFlag':'135', 'fqdn':''} + #updatecfgDict["rpcDebugFlag"] = rpcDebugFlagVal + #print ("===================: ", updatecfgDict) + + def init(self, conn, logSql): + tdLog.debug(f"start to excute {__file__}") + #tdSql.init(conn.cursor()) + tdSql.init(conn.cursor(), logSql) # output sql.txt file + + def getBuildPath(self): + selfPath = os.path.dirname(os.path.realpath(__file__)) + + if ("community" in selfPath): + projPath = selfPath[:selfPath.find("community")] + else: + projPath = selfPath[:selfPath.find("tests")] + + for root, dirs, files in os.walk(projPath): + if ("taosd" in files): + rootRealPath = os.path.dirname(os.path.realpath(root)) + if ("packaging" not in rootRealPath): + buildPath = root[:len(root) - len("/build/bin")] + break + return buildPath + + def newcur(self,cfg,host,port): + user = "root" + password = "taosdata" + con=taos.connect(host=host, user=user, password=password, config=cfg ,port=port) + cur=con.cursor() + print(cur) + return cur + + def initConsumerTable(self,cdbName='cdb'): + tdLog.info("create consume database, and consume info table, and consume result table") + tdSql.query("create database if not exists %s vgroups 1"%(cdbName)) + tdSql.query("drop table if exists %s.consumeinfo "%(cdbName)) + tdSql.query("drop table if exists %s.consumeresult "%(cdbName)) + + tdSql.query("create table %s.consumeinfo (ts timestamp, consumerid int, topiclist binary(1024), keylist binary(1024), expectmsgcnt bigint, ifcheckdata int, ifmanualcommit int)"%cdbName) + tdSql.query("create table %s.consumeresult (ts timestamp, consumerid int, consummsgcnt bigint, consumrowcnt bigint, checkresult int)"%cdbName) + + def initConsumerInfoTable(self,cdbName='cdb'): + tdLog.info("drop consumeinfo table") + tdSql.query("drop table if exists %s.consumeinfo "%(cdbName)) + tdSql.query("create table %s.consumeinfo (ts timestamp, consumerid int, topiclist binary(1024), keylist binary(1024), expectmsgcnt bigint, ifcheckdata int, ifmanualcommit int)"%cdbName) + + def insertConsumerInfo(self,consumerId, expectrowcnt,topicList,keyList,ifcheckdata,ifmanualcommit,cdbName='cdb'): + sql = "insert into %s.consumeinfo values "%cdbName + sql += "(now, %d, '%s', '%s', %d, %d, %d)"%(consumerId, topicList, keyList, expectrowcnt, ifcheckdata, ifmanualcommit) + tdLog.info("consume info sql: %s"%sql) + tdSql.query(sql) + + def selectConsumeResult(self,expectRows,cdbName='cdb'): + resultList=[] + while 1: + tdSql.query("select * from %s.consumeresult"%cdbName) + #tdLog.info("row: %d, %l64d, %l64d"%(tdSql.getData(0, 1),tdSql.getData(0, 2),tdSql.getData(0, 3)) + if tdSql.getRows() == expectRows: + break + else: + time.sleep(5) + + for i in range(expectRows): + tdLog.info ("consume id: %d, consume msgs: %d, consume rows: %d"%(tdSql.getData(i , 1), tdSql.getData(i , 2), tdSql.getData(i , 3))) + resultList.append(tdSql.getData(i , 3)) + + return resultList + + def startTmqSimProcess(self,buildPath,cfgPath,pollDelay,dbName,showMsg=1,showRow=1,cdbName='cdb',valgrind=0): + shellCmd = 'nohup ' + if valgrind == 1: + logFile = cfgPath + '/../log/valgrind-tmq.log' + shellCmd = 'nohup valgrind --log-file=' + logFile + shellCmd += '--tool=memcheck --leak-check=full --show-reachable=no --track-origins=yes --show-leak-kinds=all --num-callers=20 -v --workaround-gcc296-bugs=yes ' + + shellCmd += buildPath + '/build/bin/tmq_sim -c ' + cfgPath + shellCmd += " -y %d -d %s -g %d -r %d -w %s "%(pollDelay, dbName, showMsg, showRow, cdbName) + shellCmd += "> /dev/null 2>&1 &" + tdLog.info(shellCmd) + os.system(shellCmd) + + def create_database(self,tsql, dbName,dropFlag=1,vgroups=4,replica=1): + if dropFlag == 1: + tsql.execute("drop database if exists %s"%(dbName)) + + tsql.execute("create database if not exists %s vgroups %d replica %d"%(dbName, vgroups, replica)) + tdLog.debug("complete to create database %s"%(dbName)) + return + + def create_stable(self,tsql, dbName,stbName): + tsql.execute("create table if not exists %s.%s (ts timestamp, c1 bigint, c2 binary(16)) tags(t1 int)"%(dbName, stbName)) + tdLog.debug("complete to create %s.%s" %(dbName, stbName)) + return + + def create_ctables(self,tsql, dbName,stbName,ctbNum): + tsql.execute("use %s" %dbName) + pre_create = "create table" + sql = pre_create + #tdLog.debug("doing create one stable %s and %d child table in %s ..." %(stbname, count ,dbname)) + for i in range(ctbNum): + sql += " %s_%d using %s tags(%d)"%(stbName,i,stbName,i+1) + if (i > 0) and (i%100 == 0): + tsql.execute(sql) + sql = pre_create + if sql != pre_create: + tsql.execute(sql) + + tdLog.debug("complete to create %d child tables in %s.%s" %(ctbNum, dbName, stbName)) + return + + def insert_data_interlaceByMultiTbl(self,tsql,dbName,ctbPrefix,ctbNum,rowsPerTbl,batchNum,startTs=0): + tdLog.debug("start to insert data ............") + tsql.execute("use %s" %dbName) + pre_insert = "insert into " + sql = pre_insert + + if startTs == 0: + t = time.time() + startTs = int(round(t * 1000)) + + ctbDict = {} + for i in range(ctbNum): + ctbDict[i] = 0 + + #tdLog.debug("doing insert data into stable:%s rows:%d ..."%(stbName, allRows)) + rowsOfCtb = 0 + while rowsOfCtb < rowsPerTbl: + for i in range(ctbNum): + sql += " %s.%s_%d values "%(dbName,ctbPrefix,i) + for k in range(batchNum): + sql += "(%d, %d, 'tmqrow_%d') "%(startTs + ctbDict[i], ctbDict[i], ctbDict[i]) + ctbDict[i] += 1 + if (0 == ctbDict[i]%batchNum) or (ctbDict[i] == rowsPerTbl): + tsql.execute(sql) + sql = "insert into " + break + rowsOfCtb = ctbDict[0] + + tdLog.debug("insert data ............ [OK]") + return + + def insert_data(self,tsql,dbName,ctbPrefix,ctbNum,rowsPerTbl,batchNum,startTs=0): + tdLog.debug("start to insert data ............") + tsql.execute("use %s" %dbName) + pre_insert = "insert into " + sql = pre_insert + + if startTs == 0: + t = time.time() + startTs = int(round(t * 1000)) + + #tdLog.debug("doing insert data into stable:%s rows:%d ..."%(stbName, allRows)) + rowsOfSql = 0 + for i in range(ctbNum): + sql += " %s_%d values "%(ctbPrefix,i) + for j in range(rowsPerTbl): + sql += "(%d, %d, 'tmqrow_%d') "%(startTs + j, j, j) + rowsOfSql += 1 + if (j > 0) and ((rowsOfSql == batchNum) or (j == rowsPerTbl - 1)): + tsql.execute(sql) + rowsOfSql = 0 + if j < rowsPerTbl - 1: + sql = "insert into %s_%d values " %(ctbPrefix,i) + else: + sql = "insert into " + #end sql + if sql != pre_insert: + #print("insert sql:%s"%sql) + tsql.execute(sql) + tdLog.debug("insert data ............ [OK]") + return + + def insert_data_with_autoCreateTbl(self,tsql,dbName,stbName,ctbPrefix,ctbNum,rowsPerTbl,batchNum,startTs=0): + tdLog.debug("start to insert data wiht auto create child table ............") + tsql.execute("use %s" %dbName) + pre_insert = "insert into " + sql = pre_insert + + if startTs == 0: + t = time.time() + startTs = int(round(t * 1000)) + + #tdLog.debug("doing insert data into stable:%s rows:%d ..."%(stbName, allRows)) + rowsOfSql = 0 + for i in range(ctbNum): + sql += " %s.%s_%d using %s.%s tags (%d) values "%(dbName,ctbPrefix,i,dbName,stbName,i) + for j in range(rowsPerTbl): + sql += "(%d, %d, 'tmqrow_%d') "%(startTs + j, j, j) + rowsOfSql += 1 + if (j > 0) and ((rowsOfSql == batchNum) or (j == rowsPerTbl - 1)): + tsql.execute(sql) + rowsOfSql = 0 + if j < rowsPerTbl - 1: + sql = "insert into %s.%s_%d using %s.%s tags (%d) values " %(dbName,ctbPrefix,i,dbName,stbName,i) + else: + sql = "insert into " + #end sql + if sql != pre_insert: + #print("insert sql:%s"%sql) + tsql.execute(sql) + tdLog.debug("insert data ............ [OK]") + return + + def prepareEnv(self, **parameterDict): + # create new connector for my thread + tsql=self.newcur(parameterDict['cfg'], 'localhost', 6030) + + if parameterDict["actionType"] == actionType.CREATE_DATABASE: + self.create_database(tsql, parameterDict["dbName"]) + elif parameterDict["actionType"] == actionType.CREATE_STABLE: + self.create_stable(tsql, parameterDict["dbName"], parameterDict["stbName"]) + elif parameterDict["actionType"] == actionType.CREATE_CTABLE: + self.create_ctables(tsql, parameterDict["dbName"], parameterDict["stbName"], parameterDict["ctbNum"]) + elif parameterDict["actionType"] == actionType.INSERT_DATA: + self.insert_data(tsql, parameterDict["dbName"], parameterDict["stbName"], parameterDict["ctbNum"],\ + parameterDict["rowsPerTbl"],parameterDict["batchNum"]) + else: + tdLog.exit("not support's action: ", parameterDict["actionType"]) + + return + + def tmqCase1(self, cfgPath, buildPath): + tdLog.printNoPrefix("======== test case 1: ") + + self.initConsumerTable() + + # create and start thread + parameterDict = {'cfg': '', \ + 'actionType': 0, \ + 'dbName': 'db1', \ + 'dropFlag': 1, \ + 'vgroups': 4, \ + 'replica': 1, \ + 'stbName': 'stb1', \ + 'ctbNum': 10, \ + 'rowsPerTbl': 10000, \ + 'batchNum': 33, \ + 'startTs': 1640966400000} # 2022-01-01 00:00:00.000 + parameterDict['cfg'] = cfgPath + + self.create_database(tdSql, parameterDict["dbName"]) + self.create_stable(tdSql, parameterDict["dbName"], parameterDict["stbName"]) + self.create_ctables(tdSql, parameterDict["dbName"], parameterDict["stbName"], parameterDict["ctbNum"]) + self.insert_data_interlaceByMultiTbl(tdSql,parameterDict["dbName"],parameterDict["stbName"],parameterDict["ctbNum"],parameterDict["rowsPerTbl"],parameterDict["batchNum"]) + + tdLog.info("create topics from stb1") + topicFromStb1 = 'topic_stb1' + + tdSql.execute("create topic %s as select ts, c1, c2 from %s.%s" %(topicFromStb1, parameterDict['dbName'], parameterDict['stbName'])) + consumerId = 0 + expectrowcnt = parameterDict["rowsPerTbl"] * parameterDict["ctbNum"] + topicList = topicFromStb1 + ifcheckdata = 0 + ifManualCommit = 0 + keyList = 'group.id:cgrp1,\ + enable.auto.commit:false,\ + auto.commit.interval.ms:6000,\ + auto.offset.reset:earliest' + self.insertConsumerInfo(consumerId, expectrowcnt,topicList,keyList,ifcheckdata,ifManualCommit) + + tdLog.info("start consume processor") + pollDelay = 10 + showMsg = 1 + showRow = 1 + self.startTmqSimProcess(buildPath,cfgPath,pollDelay,parameterDict["dbName"],showMsg, showRow) + + print("================= restart dnode ===========================") + time.sleep(3) + tdDnodes.stop(1) + tdDnodes.start(1) + time.sleep(2) + + tdLog.info("insert process end, and start to check consume result") + expectRows = 1 + resultList = self.selectConsumeResult(expectRows) + totalConsumeRows = 0 + for i in range(expectRows): + totalConsumeRows += resultList[i] + + if totalConsumeRows != expectrowcnt: + tdLog.info("act consume rows: %d, expect consume rows: %d"%(totalConsumeRows, expectrowcnt)) + tdLog.exit("tmq consume rows error!") + + tdSql.query("drop topic %s"%topicFromStb1) + + tdLog.printNoPrefix("======== test case 1 end ...... ") + + def tmqCase2(self, cfgPath, buildPath): + tdLog.printNoPrefix("======== test case 2: ") + + self.initConsumerTable() + + # create and start thread + parameterDict = {'cfg': '', \ + 'actionType': 0, \ + 'dbName': 'db2', \ + 'dropFlag': 1, \ + 'vgroups': 4, \ + 'replica': 1, \ + 'stbName': 'stb1', \ + 'ctbNum': 10, \ + 'rowsPerTbl': 15000, \ + 'batchNum': 100, \ + 'startTs': 1640966400000} # 2022-01-01 00:00:00.000 + parameterDict['cfg'] = cfgPath + + parameterDict2 = {'cfg': '', \ + 'actionType': 0, \ + 'dbName': 'db2', \ + 'dropFlag': 1, \ + 'vgroups': 4, \ + 'replica': 1, \ + 'stbName': 'stb2', \ + 'ctbNum': 10, \ + 'rowsPerTbl': 16000, \ + 'batchNum': 100, \ + 'startTs': 1640966400000} # 2022-01-01 00:00:00.000 + parameterDict2['cfg'] = cfgPath + + self.create_database(tdSql, parameterDict["dbName"]) + self.create_stable(tdSql, parameterDict["dbName"], parameterDict["stbName"]) + self.create_stable(tdSql, parameterDict2["dbName"], parameterDict2["stbName"]) + + tdLog.info("create topics from stb1") + topicFromStb1 = 'topic_stb1' + + tdSql.execute("create topic %s as select ts, c1, c2 from %s.%s" %(topicFromStb1, parameterDict['dbName'], parameterDict['stbName'])) + consumerId = 0 + expectrowcnt = parameterDict["rowsPerTbl"] * parameterDict["ctbNum"] + topicList = topicFromStb1 + ifcheckdata = 0 + ifManualCommit = 0 + keyList = 'group.id:cgrp1,\ + enable.auto.commit:false,\ + auto.commit.interval.ms:6000,\ + auto.offset.reset:earliest' + self.insertConsumerInfo(consumerId, expectrowcnt,topicList,keyList,ifcheckdata,ifManualCommit) + + tdLog.info("start consume processor") + pollDelay = 0 + showMsg = 1 + showRow = 1 + self.startTmqSimProcess(buildPath,cfgPath,pollDelay,parameterDict["dbName"],showMsg, showRow) + + tdLog.info("start create child tables of stb1 and stb2") + parameterDict['actionType'] = actionType.CREATE_CTABLE + parameterDict2['actionType'] = actionType.CREATE_CTABLE + + prepareEnvThread = threading.Thread(target=self.prepareEnv, kwargs=parameterDict) + prepareEnvThread.start() + prepareEnvThread2 = threading.Thread(target=self.prepareEnv, kwargs=parameterDict2) + prepareEnvThread2.start() + + prepareEnvThread.join() + prepareEnvThread2.join() + + tdLog.info("start insert data into child tables of stb1 and stb2") + parameterDict['actionType'] = actionType.INSERT_DATA + parameterDict2['actionType'] = actionType.INSERT_DATA + + prepareEnvThread = threading.Thread(target=self.prepareEnv, kwargs=parameterDict) + prepareEnvThread.start() + prepareEnvThread2 = threading.Thread(target=self.prepareEnv, kwargs=parameterDict2) + prepareEnvThread2.start() + + prepareEnvThread.join() + prepareEnvThread2.join() + + print("================= restart dnode ===========================") + tdDnodes.stop(1) + tdDnodes.start(1) + time.sleep(2) + + tdLog.info("insert process end, and start to check consume result") + expectRows = 1 + resultList = self.selectConsumeResult(expectRows) + totalConsumeRows = 0 + for i in range(expectRows): + totalConsumeRows += resultList[i] + + if totalConsumeRows != expectrowcnt: + tdLog.info("act consume rows: %d, expect consume rows: %d"%(totalConsumeRows, expectrowcnt)) + tdLog.exit("tmq consume rows error!") + + tdSql.query("drop topic %s"%topicFromStb1) + + tdLog.printNoPrefix("======== test case 2 end ...... ") + + def tmqCase3(self, cfgPath, buildPath): + tdLog.printNoPrefix("======== test case 3: ") + + self.initConsumerTable() + + # create and start thread + parameterDict = {'cfg': '', \ + 'actionType': 0, \ + 'dbName': 'db3', \ + 'dropFlag': 1, \ + 'vgroups': 4, \ + 'replica': 1, \ + 'stbName': 'stb1', \ + 'ctbNum': 10, \ + 'rowsPerTbl': 50000, \ + 'batchNum': 13, \ + 'startTs': 1640966400000} # 2022-01-01 00:00:00.000 + parameterDict['cfg'] = cfgPath + + self.create_database(tdSql, parameterDict["dbName"]) + self.create_stable(tdSql, parameterDict["dbName"], parameterDict["stbName"]) + self.create_ctables(tdSql, parameterDict["dbName"], parameterDict["stbName"], parameterDict["ctbNum"]) + self.insert_data(tdSql,parameterDict["dbName"],parameterDict["stbName"],parameterDict["ctbNum"],parameterDict["rowsPerTbl"],parameterDict["batchNum"]) + + tdLog.info("create topics from stb1") + topicFromStb1 = 'topic_stb1' + + tdSql.execute("create topic %s as select ts, c1, c2 from %s.%s" %(topicFromStb1, parameterDict['dbName'], parameterDict['stbName'])) + consumerId = 0 + expectrowcnt = parameterDict["rowsPerTbl"] * parameterDict["ctbNum"] + topicList = topicFromStb1 + ifcheckdata = 0 + ifManualCommit = 0 + keyList = 'group.id:cgrp1,\ + enable.auto.commit:false,\ + auto.commit.interval.ms:6000,\ + auto.offset.reset:earliest' + self.insertConsumerInfo(consumerId, expectrowcnt,topicList,keyList,ifcheckdata,ifManualCommit) + + tdLog.info("start consume processor") + pollDelay = 5 + showMsg = 1 + showRow = 1 + self.startTmqSimProcess(buildPath,cfgPath,pollDelay,parameterDict["dbName"],showMsg, showRow) + + time.sleep(5) + tdLog.info("drop som child table of stb1") + dropTblNum = 4 + tdSql.query("drop table if exists %s.%s_1"%(parameterDict["dbName"], parameterDict["stbName"])) + tdSql.query("drop table if exists %s.%s_2"%(parameterDict["dbName"], parameterDict["stbName"])) + tdSql.query("drop table if exists %s.%s_3"%(parameterDict["dbName"], parameterDict["stbName"])) + tdSql.query("drop table if exists %s.%s_4"%(parameterDict["dbName"], parameterDict["stbName"])) + + tdLog.info("drop some child tables, then start to check consume result") + expectRows = 1 + resultList = self.selectConsumeResult(expectRows) + totalConsumeRows = 0 + for i in range(expectRows): + totalConsumeRows += resultList[i] + + remaindrowcnt = parameterDict["rowsPerTbl"] * (parameterDict["ctbNum"] - dropTblNum) + + if not (totalConsumeRows < expectrowcnt and totalConsumeRows > remaindrowcnt): + tdLog.info("act consume rows: %d, expect consume rows: between %d and %d"%(totalConsumeRows, remaindrowcnt, expectrowcnt)) + tdLog.exit("tmq consume rows error!") + + tdSql.query("drop topic %s"%topicFromStb1) + + tdLog.printNoPrefix("======== test case 3 end ...... ") + + def tmqCase4(self, cfgPath, buildPath): + tdLog.printNoPrefix("======== test case 4: ") + + self.initConsumerTable() + + # create and start thread + parameterDict = {'cfg': '', \ + 'actionType': 0, \ + 'dbName': 'db4', \ + 'dropFlag': 1, \ + 'vgroups': 4, \ + 'replica': 1, \ + 'stbName': 'stb1', \ + 'ctbNum': 10, \ + 'rowsPerTbl': 10000, \ + 'batchNum': 100, \ + 'startTs': 1640966400000} # 2022-01-01 00:00:00.000 + parameterDict['cfg'] = cfgPath + + self.create_database(tdSql, parameterDict["dbName"]) + self.create_stable(tdSql, parameterDict["dbName"], parameterDict["stbName"]) + self.create_ctables(tdSql, parameterDict["dbName"], parameterDict["stbName"], parameterDict["ctbNum"]) + self.insert_data(tdSql,\ + parameterDict["dbName"],\ + parameterDict["stbName"],\ + parameterDict["ctbNum"],\ + parameterDict["rowsPerTbl"],\ + parameterDict["batchNum"]) + + tdLog.info("create topics from stb1") + topicFromStb1 = 'topic_stb1' + + tdSql.execute("create topic %s as select ts, c1, c2 from %s.%s" %(topicFromStb1, parameterDict['dbName'], parameterDict['stbName'])) + consumerId = 0 + expectrowcnt = parameterDict["rowsPerTbl"] * parameterDict["ctbNum"] + topicList = topicFromStb1 + ifcheckdata = 0 + ifManualCommit = 1 + keyList = 'group.id:cgrp1,\ + enable.auto.commit:false,\ + auto.commit.interval.ms:6000,\ + auto.offset.reset:earliest' + self.insertConsumerInfo(consumerId, expectrowcnt/4,topicList,keyList,ifcheckdata,ifManualCommit) + + tdLog.info("start consume processor") + pollDelay = 5 + showMsg = 1 + showRow = 1 + self.startTmqSimProcess(buildPath,cfgPath,pollDelay,parameterDict["dbName"],showMsg, showRow) + + tdLog.info("start to check consume result") + expectRows = 1 + resultList = self.selectConsumeResult(expectRows) + totalConsumeRows = 0 + for i in range(expectRows): + totalConsumeRows += resultList[i] + + if totalConsumeRows != expectrowcnt/4: + tdLog.info("act consume rows: %d, expect consume rows: %d"%(totalConsumeRows, expectrowcnt/4)) + tdLog.exit("tmq consume rows error!") + + self.initConsumerInfoTable() + consumerId = 1 + self.insertConsumerInfo(consumerId, expectrowcnt,topicList,keyList,ifcheckdata,ifManualCommit) + + tdLog.info("again start consume processor") + self.startTmqSimProcess(buildPath,cfgPath,pollDelay,parameterDict["dbName"],showMsg, showRow) + + tdLog.info("again check consume result") + expectRows = 2 + resultList = self.selectConsumeResult(expectRows) + totalConsumeRows = 0 + for i in range(expectRows): + totalConsumeRows += resultList[i] + + if totalConsumeRows != expectrowcnt: + tdLog.info("act consume rows: %d, expect consume rows: %d"%(totalConsumeRows, expectrowcnt)) + tdLog.exit("tmq consume rows error!") + + tdSql.query("drop topic %s"%topicFromStb1) + + tdLog.printNoPrefix("======== test case 4 end ...... ") + + def tmqCase5(self, cfgPath, buildPath): + tdLog.printNoPrefix("======== test case 5: ") + + self.initConsumerTable() + + # create and start thread + parameterDict = {'cfg': '', \ + 'actionType': 0, \ + 'dbName': 'db5', \ + 'dropFlag': 1, \ + 'vgroups': 4, \ + 'replica': 1, \ + 'stbName': 'stb1', \ + 'ctbNum': 10, \ + 'rowsPerTbl': 10000, \ + 'batchNum': 100, \ + 'startTs': 1640966400000} # 2022-01-01 00:00:00.000 + parameterDict['cfg'] = cfgPath + + self.create_database(tdSql, parameterDict["dbName"]) + self.create_stable(tdSql, parameterDict["dbName"], parameterDict["stbName"]) + self.create_ctables(tdSql, parameterDict["dbName"], parameterDict["stbName"], parameterDict["ctbNum"]) + self.insert_data(tdSql,\ + parameterDict["dbName"],\ + parameterDict["stbName"],\ + parameterDict["ctbNum"],\ + parameterDict["rowsPerTbl"],\ + parameterDict["batchNum"]) + + tdLog.info("create topics from stb1") + topicFromStb1 = 'topic_stb1' + + tdSql.execute("create topic %s as select ts, c1, c2 from %s.%s" %(topicFromStb1, parameterDict['dbName'], parameterDict['stbName'])) + consumerId = 0 + expectrowcnt = parameterDict["rowsPerTbl"] * parameterDict["ctbNum"] + topicList = topicFromStb1 + ifcheckdata = 0 + ifManualCommit = 0 + keyList = 'group.id:cgrp1,\ + enable.auto.commit:false,\ + auto.commit.interval.ms:6000,\ + auto.offset.reset:earliest' + self.insertConsumerInfo(consumerId, expectrowcnt/4,topicList,keyList,ifcheckdata,ifManualCommit) + + tdLog.info("start consume processor") + pollDelay = 5 + showMsg = 1 + showRow = 1 + self.startTmqSimProcess(buildPath,cfgPath,pollDelay,parameterDict["dbName"],showMsg, showRow) + + tdLog.info("start to check consume result") + expectRows = 1 + resultList = self.selectConsumeResult(expectRows) + totalConsumeRows = 0 + for i in range(expectRows): + totalConsumeRows += resultList[i] + + if totalConsumeRows != expectrowcnt/4: + tdLog.info("act consume rows: %d, expect consume rows: %d"%(totalConsumeRows, expectrowcnt/4)) + tdLog.exit("tmq consume rows error!") + + self.initConsumerInfoTable() + consumerId = 1 + self.insertConsumerInfo(consumerId, expectrowcnt,topicList,keyList,ifcheckdata,ifManualCommit) + + tdLog.info("again start consume processor") + self.startTmqSimProcess(buildPath,cfgPath,pollDelay,parameterDict["dbName"],showMsg, showRow) + + tdLog.info("again check consume result") + expectRows = 2 + resultList = self.selectConsumeResult(expectRows) + totalConsumeRows = 0 + for i in range(expectRows): + totalConsumeRows += resultList[i] + + if totalConsumeRows != (expectrowcnt * (1 + 1/4)): + tdLog.info("act consume rows: %d, expect consume rows: %d"%(totalConsumeRows, expectrowcnt)) + tdLog.exit("tmq consume rows error!") + + tdSql.query("drop topic %s"%topicFromStb1) + + tdLog.printNoPrefix("======== test case 5 end ...... ") + + def tmqCase6(self, cfgPath, buildPath): + tdLog.printNoPrefix("======== test case 6: ") + + self.initConsumerTable() + + # create and start thread + parameterDict = {'cfg': '', \ + 'actionType': 0, \ + 'dbName': 'db6', \ + 'dropFlag': 1, \ + 'vgroups': 4, \ + 'replica': 1, \ + 'stbName': 'stb1', \ + 'ctbNum': 10, \ + 'rowsPerTbl': 10000, \ + 'batchNum': 100, \ + 'startTs': 1640966400000} # 2022-01-01 00:00:00.000 + parameterDict['cfg'] = cfgPath + + self.create_database(tdSql, parameterDict["dbName"]) + self.create_stable(tdSql, parameterDict["dbName"], parameterDict["stbName"]) + self.create_ctables(tdSql, parameterDict["dbName"], parameterDict["stbName"], parameterDict["ctbNum"]) + self.insert_data(tdSql,\ + parameterDict["dbName"],\ + parameterDict["stbName"],\ + parameterDict["ctbNum"],\ + parameterDict["rowsPerTbl"],\ + parameterDict["batchNum"]) + + tdLog.info("create topics from stb1") + topicFromStb1 = 'topic_stb1' + + tdSql.execute("create topic %s as select ts, c1, c2 from %s.%s" %(topicFromStb1, parameterDict['dbName'], parameterDict['stbName'])) + consumerId = 0 + expectrowcnt = parameterDict["rowsPerTbl"] * parameterDict["ctbNum"] + topicList = topicFromStb1 + ifcheckdata = 0 + ifManualCommit = 1 + keyList = 'group.id:cgrp1,\ + enable.auto.commit:false,\ + auto.commit.interval.ms:6000,\ + auto.offset.reset:earliest' + self.insertConsumerInfo(consumerId, expectrowcnt/4,topicList,keyList,ifcheckdata,ifManualCommit) + + tdLog.info("start consume processor") + pollDelay = 5 + showMsg = 1 + showRow = 1 + self.startTmqSimProcess(buildPath,cfgPath,pollDelay,parameterDict["dbName"],showMsg, showRow) + + tdLog.info("start to check consume result") + expectRows = 1 + resultList = self.selectConsumeResult(expectRows) + totalConsumeRows = 0 + for i in range(expectRows): + totalConsumeRows += resultList[i] + + if totalConsumeRows != expectrowcnt/4: + tdLog.info("act consume rows: %d, expect consume rows: %d"%(totalConsumeRows, expectrowcnt/4)) + tdLog.exit("tmq consume rows error!") + + self.initConsumerInfoTable() + consumerId = 1 + keyList = 'group.id:cgrp1,\ + enable.auto.commit:false,\ + auto.commit.interval.ms:6000,\ + auto.offset.reset:latest' + self.insertConsumerInfo(consumerId, expectrowcnt,topicList,keyList,ifcheckdata,ifManualCommit) + + tdLog.info("again start consume processor") + self.startTmqSimProcess(buildPath,cfgPath,pollDelay,parameterDict["dbName"],showMsg, showRow) + + tdLog.info("again check consume result") + expectRows = 2 + resultList = self.selectConsumeResult(expectRows) + totalConsumeRows = 0 + for i in range(expectRows): + totalConsumeRows += resultList[i] + + if totalConsumeRows != expectrowcnt: + tdLog.info("act consume rows: %d, expect consume rows: %d"%(totalConsumeRows, expectrowcnt)) + tdLog.exit("tmq consume rows error!") + + tdSql.query("drop topic %s"%topicFromStb1) + + tdLog.printNoPrefix("======== test case 6 end ...... ") + + def tmqCase7(self, cfgPath, buildPath): + tdLog.printNoPrefix("======== test case 7: ") + + self.initConsumerTable() + + # create and start thread + parameterDict = {'cfg': '', \ + 'actionType': 0, \ + 'dbName': 'db7', \ + 'dropFlag': 1, \ + 'vgroups': 4, \ + 'replica': 1, \ + 'stbName': 'stb1', \ + 'ctbNum': 10, \ + 'rowsPerTbl': 10000, \ + 'batchNum': 100, \ + 'startTs': 1640966400000} # 2022-01-01 00:00:00.000 + parameterDict['cfg'] = cfgPath + + self.create_database(tdSql, parameterDict["dbName"]) + self.create_stable(tdSql, parameterDict["dbName"], parameterDict["stbName"]) + self.create_ctables(tdSql, parameterDict["dbName"], parameterDict["stbName"], parameterDict["ctbNum"]) + self.insert_data(tdSql,\ + parameterDict["dbName"],\ + parameterDict["stbName"],\ + parameterDict["ctbNum"],\ + parameterDict["rowsPerTbl"],\ + parameterDict["batchNum"]) + + tdLog.info("create topics from stb1") + topicFromStb1 = 'topic_stb1' + + tdSql.execute("create topic %s as select ts, c1, c2 from %s.%s" %(topicFromStb1, parameterDict['dbName'], parameterDict['stbName'])) + consumerId = 0 + expectrowcnt = parameterDict["rowsPerTbl"] * parameterDict["ctbNum"] + topicList = topicFromStb1 + ifcheckdata = 0 + ifManualCommit = 1 + keyList = 'group.id:cgrp1,\ + enable.auto.commit:false,\ + auto.commit.interval.ms:6000,\ + auto.offset.reset:latest' + self.insertConsumerInfo(consumerId, expectrowcnt,topicList,keyList,ifcheckdata,ifManualCommit) + + tdLog.info("start consume processor") + pollDelay = 5 + showMsg = 1 + showRow = 1 + self.startTmqSimProcess(buildPath,cfgPath,pollDelay,parameterDict["dbName"],showMsg, showRow) + + tdLog.info("start to check consume result") + expectRows = 1 + resultList = self.selectConsumeResult(expectRows) + totalConsumeRows = 0 + for i in range(expectRows): + totalConsumeRows += resultList[i] + + if totalConsumeRows != 0: + tdLog.info("act consume rows: %d, expect consume rows: %d"%(totalConsumeRows, 0)) + tdLog.exit("tmq consume rows error!") + + self.initConsumerInfoTable() + consumerId = 1 + self.insertConsumerInfo(consumerId, expectrowcnt,topicList,keyList,ifcheckdata,ifManualCommit) + + tdLog.info("again start consume processor") + self.startTmqSimProcess(buildPath,cfgPath,pollDelay,parameterDict["dbName"],showMsg, showRow) + + tdLog.info("again check consume result") + expectRows = 2 + resultList = self.selectConsumeResult(expectRows) + totalConsumeRows = 0 + for i in range(expectRows): + totalConsumeRows += resultList[i] + + if totalConsumeRows != 0: + tdLog.info("act consume rows: %d, expect consume rows: %d"%(totalConsumeRows, 0)) + tdLog.exit("tmq consume rows error!") + + tdSql.query("drop topic %s"%topicFromStb1) + + tdLog.printNoPrefix("======== test case 7 end ...... ") + + def tmqCase8(self, cfgPath, buildPath): + tdLog.printNoPrefix("======== test case 8: ") + + self.initConsumerTable() + + # create and start thread + parameterDict = {'cfg': '', \ + 'actionType': 0, \ + 'dbName': 'db8', \ + 'dropFlag': 1, \ + 'vgroups': 4, \ + 'replica': 1, \ + 'stbName': 'stb1', \ + 'ctbNum': 10, \ + 'rowsPerTbl': 10000, \ + 'batchNum': 100, \ + 'startTs': 1640966400000} # 2022-01-01 00:00:00.000 + parameterDict['cfg'] = cfgPath + + self.create_database(tdSql, parameterDict["dbName"]) + self.create_stable(tdSql, parameterDict["dbName"], parameterDict["stbName"]) + self.create_ctables(tdSql, parameterDict["dbName"], parameterDict["stbName"], parameterDict["ctbNum"]) + self.insert_data(tdSql,\ + parameterDict["dbName"],\ + parameterDict["stbName"],\ + parameterDict["ctbNum"],\ + parameterDict["rowsPerTbl"],\ + parameterDict["batchNum"]) + + tdLog.info("create topics from stb1") + topicFromStb1 = 'topic_stb1' + + tdSql.execute("create topic %s as select ts, c1, c2 from %s.%s" %(topicFromStb1, parameterDict['dbName'], parameterDict['stbName'])) + consumerId = 0 + expectrowcnt = parameterDict["rowsPerTbl"] * parameterDict["ctbNum"] + topicList = topicFromStb1 + ifcheckdata = 0 + ifManualCommit = 1 + keyList = 'group.id:cgrp1,\ + enable.auto.commit:false,\ + auto.commit.interval.ms:6000,\ + auto.offset.reset:latest' + self.insertConsumerInfo(consumerId, expectrowcnt,topicList,keyList,ifcheckdata,ifManualCommit) + + tdLog.info("start consume 0 processor") + pollDelay = 10 + showMsg = 1 + showRow = 1 + self.startTmqSimProcess(buildPath,cfgPath,pollDelay,parameterDict["dbName"],showMsg, showRow) + + tdLog.info("start to check consume 0 result") + expectRows = 1 + resultList = self.selectConsumeResult(expectRows) + totalConsumeRows = 0 + for i in range(expectRows): + totalConsumeRows += resultList[i] + + if totalConsumeRows != 0: + tdLog.info("act consume rows: %d, expect consume rows: %d"%(totalConsumeRows, 0)) + tdLog.exit("tmq consume rows error!") + + tdLog.info("start consume 1 processor") + self.startTmqSimProcess(buildPath,cfgPath,pollDelay,parameterDict["dbName"],showMsg, showRow) + + tdLog.info("start one new thread to insert data") + parameterDict['actionType'] = actionType.INSERT_DATA + prepareEnvThread = threading.Thread(target=self.prepareEnv, kwargs=parameterDict) + prepareEnvThread.start() + prepareEnvThread.join() + + tdLog.info("start to check consume 0 and 1 result") + expectRows = 2 + resultList = self.selectConsumeResult(expectRows) + totalConsumeRows = 0 + for i in range(expectRows): + totalConsumeRows += resultList[i] + + if totalConsumeRows != expectrowcnt: + tdLog.info("act consume rows: %d, expect consume rows: %d"%(totalConsumeRows, expectrowcnt)) + tdLog.exit("tmq consume rows error!") + + tdLog.info("start consume 2 processor") + self.startTmqSimProcess(buildPath,cfgPath,pollDelay,parameterDict["dbName"],showMsg, showRow) + + tdLog.info("start one new thread to insert data") + parameterDict['actionType'] = actionType.INSERT_DATA + prepareEnvThread = threading.Thread(target=self.prepareEnv, kwargs=parameterDict) + prepareEnvThread.start() + prepareEnvThread.join() + + tdLog.info("start to check consume 0 and 1 and 2 result") + expectRows = 3 + resultList = self.selectConsumeResult(expectRows) + totalConsumeRows = 0 + for i in range(expectRows): + totalConsumeRows += resultList[i] + + if totalConsumeRows != expectrowcnt*2: + tdLog.info("act consume rows: %d, expect consume rows: %d"%(totalConsumeRows, expectrowcnt*2)) + tdLog.exit("tmq consume rows error!") + + tdSql.query("drop topic %s"%topicFromStb1) + + tdLog.printNoPrefix("======== test case 8 end ...... ") + + def tmqCase9(self, cfgPath, buildPath): + tdLog.printNoPrefix("======== test case 9: ") + + self.initConsumerTable() + + # create and start thread + parameterDict = {'cfg': '', \ + 'actionType': 0, \ + 'dbName': 'db9', \ + 'dropFlag': 1, \ + 'vgroups': 4, \ + 'replica': 1, \ + 'stbName': 'stb1', \ + 'ctbNum': 10, \ + 'rowsPerTbl': 10000, \ + 'batchNum': 100, \ + 'startTs': 1640966400000} # 2022-01-01 00:00:00.000 + parameterDict['cfg'] = cfgPath + + self.create_database(tdSql, parameterDict["dbName"]) + self.create_stable(tdSql, parameterDict["dbName"], parameterDict["stbName"]) + self.create_ctables(tdSql, parameterDict["dbName"], parameterDict["stbName"], parameterDict["ctbNum"]) + self.insert_data(tdSql,\ + parameterDict["dbName"],\ + parameterDict["stbName"],\ + parameterDict["ctbNum"],\ + parameterDict["rowsPerTbl"],\ + parameterDict["batchNum"]) + + tdLog.info("create topics from stb1") + topicFromStb1 = 'topic_stb1' + + tdSql.execute("create topic %s as select ts, c1, c2 from %s.%s" %(topicFromStb1, parameterDict['dbName'], parameterDict['stbName'])) + consumerId = 0 + expectrowcnt = parameterDict["rowsPerTbl"] * parameterDict["ctbNum"] + topicList = topicFromStb1 + ifcheckdata = 0 + ifManualCommit = 1 + keyList = 'group.id:cgrp1,\ + enable.auto.commit:false,\ + auto.commit.interval.ms:6000,\ + auto.offset.reset:latest' + self.insertConsumerInfo(consumerId, expectrowcnt,topicList,keyList,ifcheckdata,ifManualCommit) + + tdLog.info("start consume 0 processor") + pollDelay = 10 + showMsg = 1 + showRow = 1 + self.startTmqSimProcess(buildPath,cfgPath,pollDelay,parameterDict["dbName"],showMsg, showRow) + + tdLog.info("start to check consume 0 result") + expectRows = 1 + resultList = self.selectConsumeResult(expectRows) + totalConsumeRows = 0 + for i in range(expectRows): + totalConsumeRows += resultList[i] + + if totalConsumeRows != 0: + tdLog.info("act consume rows: %d, expect consume rows: %d"%(totalConsumeRows, 0)) + tdLog.exit("tmq consume rows error!") + + tdLog.info("start consume 1 processor") + self.initConsumerInfoTable() + consumerId = 1 + ifManualCommit = 0 + self.insertConsumerInfo(consumerId, expectrowcnt,topicList,keyList,ifcheckdata,ifManualCommit) + self.startTmqSimProcess(buildPath,cfgPath,pollDelay,parameterDict["dbName"],showMsg, showRow) + + tdLog.info("start one new thread to insert data") + parameterDict['actionType'] = actionType.INSERT_DATA + prepareEnvThread = threading.Thread(target=self.prepareEnv, kwargs=parameterDict) + prepareEnvThread.start() + prepareEnvThread.join() + + tdLog.info("start to check consume 0 and 1 result") + expectRows = 2 + resultList = self.selectConsumeResult(expectRows) + totalConsumeRows = 0 + for i in range(expectRows): + totalConsumeRows += resultList[i] + + if totalConsumeRows != expectrowcnt: + tdLog.info("act consume rows: %d, expect consume rows: %d"%(totalConsumeRows, expectrowcnt)) + tdLog.exit("tmq consume rows error!") + + tdLog.info("start consume 2 processor") + self.startTmqSimProcess(buildPath,cfgPath,pollDelay,parameterDict["dbName"],showMsg, showRow) + + tdLog.info("start one new thread to insert data") + parameterDict['actionType'] = actionType.INSERT_DATA + prepareEnvThread = threading.Thread(target=self.prepareEnv, kwargs=parameterDict) + prepareEnvThread.start() + prepareEnvThread.join() + + tdLog.info("start to check consume 0 and 1 and 2 result") + expectRows = 3 + resultList = self.selectConsumeResult(expectRows) + totalConsumeRows = 0 + for i in range(expectRows): + totalConsumeRows += resultList[i] + + if totalConsumeRows != expectrowcnt*2: + tdLog.info("act consume rows: %d, expect consume rows: %d"%(totalConsumeRows, expectrowcnt*2)) + tdLog.exit("tmq consume rows error!") + + tdSql.query("drop topic %s"%topicFromStb1) + + tdLog.printNoPrefix("======== test case 9 end ...... ") + + def tmqCase10(self, cfgPath, buildPath): + tdLog.printNoPrefix("======== test case 10: ") + + self.initConsumerTable() + + # create and start thread + parameterDict = {'cfg': '', \ + 'actionType': 0, \ + 'dbName': 'db10', \ + 'dropFlag': 1, \ + 'vgroups': 4, \ + 'replica': 1, \ + 'stbName': 'stb1', \ + 'ctbNum': 10, \ + 'rowsPerTbl': 10000, \ + 'batchNum': 100, \ + 'startTs': 1640966400000} # 2022-01-01 00:00:00.000 + parameterDict['cfg'] = cfgPath + + self.create_database(tdSql, parameterDict["dbName"]) + self.create_stable(tdSql, parameterDict["dbName"], parameterDict["stbName"]) + self.create_ctables(tdSql, parameterDict["dbName"], parameterDict["stbName"], parameterDict["ctbNum"]) + self.insert_data(tdSql,\ + parameterDict["dbName"],\ + parameterDict["stbName"],\ + parameterDict["ctbNum"],\ + parameterDict["rowsPerTbl"],\ + parameterDict["batchNum"]) + + tdLog.info("create topics from stb1") + topicFromStb1 = 'topic_stb1' + + tdSql.execute("create topic %s as select ts, c1, c2 from %s.%s" %(topicFromStb1, parameterDict['dbName'], parameterDict['stbName'])) + consumerId = 0 + expectrowcnt = parameterDict["rowsPerTbl"] * parameterDict["ctbNum"] + topicList = topicFromStb1 + ifcheckdata = 0 + ifManualCommit = 1 + keyList = 'group.id:cgrp1,\ + enable.auto.commit:false,\ + auto.commit.interval.ms:6000,\ + auto.offset.reset:latest' + self.insertConsumerInfo(consumerId, expectrowcnt,topicList,keyList,ifcheckdata,ifManualCommit) + + tdLog.info("start consume 0 processor") + pollDelay = 10 + showMsg = 1 + showRow = 1 + self.startTmqSimProcess(buildPath,cfgPath,pollDelay,parameterDict["dbName"],showMsg, showRow) + + tdLog.info("start to check consume 0 result") + expectRows = 1 + resultList = self.selectConsumeResult(expectRows) + totalConsumeRows = 0 + for i in range(expectRows): + totalConsumeRows += resultList[i] + + if totalConsumeRows != 0: + tdLog.info("act consume rows: %d, expect consume rows: %d"%(totalConsumeRows, 0)) + tdLog.exit("tmq consume rows error!") + + tdLog.info("start consume 1 processor") + self.initConsumerInfoTable() + consumerId = 1 + ifManualCommit = 1 + self.insertConsumerInfo(consumerId, expectrowcnt-10000,topicList,keyList,ifcheckdata,ifManualCommit) + self.startTmqSimProcess(buildPath,cfgPath,pollDelay,parameterDict["dbName"],showMsg, showRow) + + tdLog.info("start one new thread to insert data") + parameterDict['actionType'] = actionType.INSERT_DATA + prepareEnvThread = threading.Thread(target=self.prepareEnv, kwargs=parameterDict) + prepareEnvThread.start() + prepareEnvThread.join() + + tdLog.info("start to check consume 0 and 1 result") + expectRows = 2 + resultList = self.selectConsumeResult(expectRows) + totalConsumeRows = 0 + for i in range(expectRows): + totalConsumeRows += resultList[i] + + if totalConsumeRows != expectrowcnt-10000: + tdLog.info("act consume rows: %d, expect consume rows: %d"%(totalConsumeRows, expectrowcnt-10000)) + tdLog.exit("tmq consume rows error!") + + tdLog.info("start consume 2 processor") + self.initConsumerInfoTable() + consumerId = 2 + ifManualCommit = 1 + self.insertConsumerInfo(consumerId, expectrowcnt+10000,topicList,keyList,ifcheckdata,ifManualCommit) + self.startTmqSimProcess(buildPath,cfgPath,pollDelay,parameterDict["dbName"],showMsg, showRow) + + tdLog.info("start one new thread to insert data") + parameterDict['actionType'] = actionType.INSERT_DATA + prepareEnvThread = threading.Thread(target=self.prepareEnv, kwargs=parameterDict) + prepareEnvThread.start() + prepareEnvThread.join() + + tdLog.info("start to check consume 0 and 1 and 2 result") + expectRows = 3 + resultList = self.selectConsumeResult(expectRows) + totalConsumeRows = 0 + for i in range(expectRows): + totalConsumeRows += resultList[i] + + if totalConsumeRows != expectrowcnt*2: + tdLog.info("act consume rows: %d, expect consume rows: %d"%(totalConsumeRows, expectrowcnt*2)) + tdLog.exit("tmq consume rows error!") + + tdSql.query("drop topic %s"%topicFromStb1) + + tdLog.printNoPrefix("======== test case 10 end ...... ") + + def tmqCase11(self, cfgPath, buildPath): + tdLog.printNoPrefix("======== test case 11: ") + + self.initConsumerTable() + + # create and start thread + parameterDict = {'cfg': '', \ + 'actionType': 0, \ + 'dbName': 'db11', \ + 'dropFlag': 1, \ + 'vgroups': 4, \ + 'replica': 1, \ + 'stbName': 'stb1', \ + 'ctbNum': 10, \ + 'rowsPerTbl': 10000, \ + 'batchNum': 100, \ + 'startTs': 1640966400000} # 2022-01-01 00:00:00.000 + parameterDict['cfg'] = cfgPath + + self.create_database(tdSql, parameterDict["dbName"]) + self.create_stable(tdSql, parameterDict["dbName"], parameterDict["stbName"]) + self.create_ctables(tdSql, parameterDict["dbName"], parameterDict["stbName"], parameterDict["ctbNum"]) + self.insert_data(tdSql,\ + parameterDict["dbName"],\ + parameterDict["stbName"],\ + parameterDict["ctbNum"],\ + parameterDict["rowsPerTbl"],\ + parameterDict["batchNum"]) + + tdLog.info("create topics from stb1") + topicFromStb1 = 'topic_stb1' + + tdSql.execute("create topic %s as select ts, c1, c2 from %s.%s" %(topicFromStb1, parameterDict['dbName'], parameterDict['stbName'])) + consumerId = 0 + expectrowcnt = parameterDict["rowsPerTbl"] * parameterDict["ctbNum"] + topicList = topicFromStb1 + ifcheckdata = 0 + ifManualCommit = 1 + keyList = 'group.id:cgrp1,\ + enable.auto.commit:false,\ + auto.commit.interval.ms:6000,\ + auto.offset.reset:none' + self.insertConsumerInfo(consumerId, expectrowcnt/4,topicList,keyList,ifcheckdata,ifManualCommit) + + tdLog.info("start consume processor") + pollDelay = 5 + showMsg = 1 + showRow = 1 + self.startTmqSimProcess(buildPath,cfgPath,pollDelay,parameterDict["dbName"],showMsg, showRow) + + tdLog.info("start to check consume result") + expectRows = 1 + resultList = self.selectConsumeResult(expectRows) + totalConsumeRows = 0 + for i in range(expectRows): + totalConsumeRows += resultList[i] + + if totalConsumeRows != 0: + tdLog.info("act consume rows: %d, expect consume rows: %d"%(totalConsumeRows, 0)) + tdLog.exit("tmq consume rows error!") + + self.initConsumerInfoTable() + consumerId = 1 + keyList = 'group.id:cgrp1,\ + enable.auto.commit:false,\ + auto.commit.interval.ms:6000,\ + auto.offset.reset:none' + self.insertConsumerInfo(consumerId, expectrowcnt,topicList,keyList,ifcheckdata,ifManualCommit) + + tdLog.info("again start consume processor") + self.startTmqSimProcess(buildPath,cfgPath,pollDelay,parameterDict["dbName"],showMsg, showRow) + + tdLog.info("again check consume result") + expectRows = 2 + resultList = self.selectConsumeResult(expectRows) + totalConsumeRows = 0 + for i in range(expectRows): + totalConsumeRows += resultList[i] + + if totalConsumeRows != 0: + tdLog.info("act consume rows: %d, expect consume rows: %d"%(totalConsumeRows, 0)) + tdLog.exit("tmq consume rows error!") + + tdSql.query("drop topic %s"%topicFromStb1) + + tdLog.printNoPrefix("======== test case 11 end ...... ") + + def tmqCase12(self, cfgPath, buildPath): + tdLog.printNoPrefix("======== test case 12: ") + + self.initConsumerTable() + + # create and start thread + parameterDict = {'cfg': '', \ + 'actionType': 0, \ + 'dbName': 'db12', \ + 'dropFlag': 1, \ + 'vgroups': 4, \ + 'replica': 1, \ + 'stbName': 'stb1', \ + 'ctbNum': 10, \ + 'rowsPerTbl': 10000, \ + 'batchNum': 100, \ + 'startTs': 1640966400000} # 2022-01-01 00:00:00.000 + parameterDict['cfg'] = cfgPath + + self.create_database(tdSql, parameterDict["dbName"]) + self.create_stable(tdSql, parameterDict["dbName"], parameterDict["stbName"]) + self.create_ctables(tdSql, parameterDict["dbName"], parameterDict["stbName"], parameterDict["ctbNum"]) + self.insert_data(tdSql,\ + parameterDict["dbName"],\ + parameterDict["stbName"],\ + parameterDict["ctbNum"],\ + parameterDict["rowsPerTbl"],\ + parameterDict["batchNum"]) + + tdLog.info("create topics from stb1") + topicFromStb1 = 'topic_stb1' + + tdSql.execute("create topic %s as select ts, c1, c2 from %s.%s" %(topicFromStb1, parameterDict['dbName'], parameterDict['stbName'])) + consumerId = 0 + expectrowcnt = parameterDict["rowsPerTbl"] * parameterDict["ctbNum"] + topicList = topicFromStb1 + ifcheckdata = 0 + ifManualCommit = 0 + keyList = 'group.id:cgrp1,\ + enable.auto.commit:false,\ + auto.commit.interval.ms:6000,\ + auto.offset.reset:earliest' + self.insertConsumerInfo(consumerId, expectrowcnt/4,topicList,keyList,ifcheckdata,ifManualCommit) + + tdLog.info("start consume processor") + pollDelay = 5 + showMsg = 1 + showRow = 1 + self.startTmqSimProcess(buildPath,cfgPath,pollDelay,parameterDict["dbName"],showMsg, showRow) + + tdLog.info("start to check consume result") + expectRows = 1 + resultList = self.selectConsumeResult(expectRows) + totalConsumeRows = 0 + for i in range(expectRows): + totalConsumeRows += resultList[i] + + if totalConsumeRows != expectrowcnt/4: + tdLog.info("act consume rows: %d, expect consume rows: %d"%(totalConsumeRows, expectrowcnt/4)) + tdLog.exit("tmq consume rows error!") + + self.initConsumerInfoTable() + consumerId = 1 + keyList = 'group.id:cgrp1,\ + enable.auto.commit:false,\ + auto.commit.interval.ms:6000,\ + auto.offset.reset:none' + self.insertConsumerInfo(consumerId, expectrowcnt,topicList,keyList,ifcheckdata,ifManualCommit) + + tdLog.info("again start consume processor") + self.startTmqSimProcess(buildPath,cfgPath,pollDelay,parameterDict["dbName"],showMsg, showRow) + + tdLog.info("again check consume result") + expectRows = 2 + resultList = self.selectConsumeResult(expectRows) + totalConsumeRows = 0 + for i in range(expectRows): + totalConsumeRows += resultList[i] + + if totalConsumeRows != expectrowcnt/4: + tdLog.info("act consume rows: %d, expect consume rows: %d"%(totalConsumeRows, expectrowcnt/4)) + tdLog.exit("tmq consume rows error!") + + tdSql.query("drop topic %s"%topicFromStb1) + + tdLog.printNoPrefix("======== test case 12 end ...... ") + + def tmqCase13(self, cfgPath, buildPath): + tdLog.printNoPrefix("======== test case 13: ") + + self.initConsumerTable() + + # create and start thread + parameterDict = {'cfg': '', \ + 'actionType': 0, \ + 'dbName': 'db13', \ + 'dropFlag': 1, \ + 'vgroups': 4, \ + 'replica': 1, \ + 'stbName': 'stb1', \ + 'ctbNum': 10, \ + 'rowsPerTbl': 10000, \ + 'batchNum': 100, \ + 'startTs': 1640966400000} # 2022-01-01 00:00:00.000 + parameterDict['cfg'] = cfgPath + + self.create_database(tdSql, parameterDict["dbName"]) + self.create_stable(tdSql, parameterDict["dbName"], parameterDict["stbName"]) + self.create_ctables(tdSql, parameterDict["dbName"], parameterDict["stbName"], parameterDict["ctbNum"]) + self.insert_data(tdSql,\ + parameterDict["dbName"],\ + parameterDict["stbName"],\ + parameterDict["ctbNum"],\ + parameterDict["rowsPerTbl"],\ + parameterDict["batchNum"]) + + tdLog.info("create topics from stb1") + topicFromStb1 = 'topic_stb1' + + tdSql.execute("create topic %s as select ts, c1, c2 from %s.%s" %(topicFromStb1, parameterDict['dbName'], parameterDict['stbName'])) + consumerId = 0 + expectrowcnt = parameterDict["rowsPerTbl"] * parameterDict["ctbNum"] + topicList = topicFromStb1 + ifcheckdata = 0 + ifManualCommit = 1 + keyList = 'group.id:cgrp1,\ + enable.auto.commit:false,\ + auto.commit.interval.ms:6000,\ + auto.offset.reset:earliest' + self.insertConsumerInfo(consumerId, expectrowcnt/4,topicList,keyList,ifcheckdata,ifManualCommit) + + tdLog.info("start consume processor") + pollDelay = 5 + showMsg = 1 + showRow = 1 + self.startTmqSimProcess(buildPath,cfgPath,pollDelay,parameterDict["dbName"],showMsg, showRow) + + tdLog.info("start to check consume result") + expectRows = 1 + resultList = self.selectConsumeResult(expectRows) + totalConsumeRows = 0 + for i in range(expectRows): + totalConsumeRows += resultList[i] + + if totalConsumeRows != expectrowcnt/4: + tdLog.info("act consume rows: %d, expect consume rows: %d"%(totalConsumeRows, expectrowcnt/4)) + tdLog.exit("tmq consume rows error!") + + self.initConsumerInfoTable() + consumerId = 1 + ifManualCommit = 1 + keyList = 'group.id:cgrp1,\ + enable.auto.commit:false,\ + auto.commit.interval.ms:6000,\ + auto.offset.reset:none' + self.insertConsumerInfo(consumerId, expectrowcnt/2,topicList,keyList,ifcheckdata,ifManualCommit) + + tdLog.info("again start consume processor") + self.startTmqSimProcess(buildPath,cfgPath,pollDelay,parameterDict["dbName"],showMsg, showRow) + + tdLog.info("again check consume result") + expectRows = 2 + resultList = self.selectConsumeResult(expectRows) + totalConsumeRows = 0 + for i in range(expectRows): + totalConsumeRows += resultList[i] + + if totalConsumeRows != expectrowcnt*(1/2+1/4): + tdLog.info("act consume rows: %d, expect consume rows: %d"%(totalConsumeRows, expectrowcnt*(1/2+1/4))) + tdLog.exit("tmq consume rows error!") + + self.initConsumerInfoTable() + consumerId = 2 + ifManualCommit = 1 + keyList = 'group.id:cgrp1,\ + enable.auto.commit:false,\ + auto.commit.interval.ms:6000,\ + auto.offset.reset:none' + self.insertConsumerInfo(consumerId, expectrowcnt,topicList,keyList,ifcheckdata,ifManualCommit) + + tdLog.info("again start consume processor") + self.startTmqSimProcess(buildPath,cfgPath,pollDelay,parameterDict["dbName"],showMsg, showRow) + + tdLog.info("again check consume result") + expectRows = 3 + resultList = self.selectConsumeResult(expectRows) + totalConsumeRows = 0 + for i in range(expectRows): + totalConsumeRows += resultList[i] + + if totalConsumeRows != expectrowcnt: + tdLog.info("act consume rows: %d, expect consume rows: %d"%(totalConsumeRows, expectrowcnt)) + tdLog.exit("tmq consume rows error!") + + tdSql.query("drop topic %s"%topicFromStb1) + + tdLog.printNoPrefix("======== test case 13 end ...... ") + + def run(self): + tdSql.prepare() + + buildPath = self.getBuildPath() + if (buildPath == ""): + tdLog.exit("taosd not found!") + else: + tdLog.info("taosd found in %s" % buildPath) + cfgPath = buildPath + "/../sim/psim/cfg" + tdLog.info("cfgPath: %s" % cfgPath) + + # self.tmqCase1(cfgPath, buildPath) + self.tmqCase2(cfgPath, buildPath) + # self.tmqCase3(cfgPath, buildPath) + # self.tmqCase4(cfgPath, buildPath) + # self.tmqCase5(cfgPath, buildPath) + + def stop(self): + tdSql.close() + tdLog.success(f"{__file__} successfully executed") + +event = threading.Event() + +tdCases.addLinux(__file__, TDTestCase()) +tdCases.addWindows(__file__, TDTestCase()) diff --git a/tests/system-test/7-tmq/tmqModule.py b/tests/system-test/7-tmq/tmqModule.py new file mode 100644 index 0000000000000000000000000000000000000000..8e0d741040d4aa68cebe12cf20c25a81c4bd7ee2 --- /dev/null +++ b/tests/system-test/7-tmq/tmqModule.py @@ -0,0 +1,1446 @@ + +import taos +import sys +import time +import socket +import os +import threading +from enum import Enum + +from util.log import * +from util.sql import * +from util.cases import * +from util.dnodes import * + +class actionType(Enum): + CREATE_DATABASE = 0 + CREATE_STABLE = 1 + CREATE_CTABLE = 2 + INSERT_DATA = 3 + +class TDTestCase: + hostname = socket.gethostname() + #rpcDebugFlagVal = '143' + #clientCfgDict = {'serverPort': '', 'firstEp': '', 'secondEp':'', 'rpcDebugFlag':'135', 'fqdn':''} + #clientCfgDict["rpcDebugFlag"] = rpcDebugFlagVal + #updatecfgDict = {'clientCfg': {}, 'serverPort': '', 'firstEp': '', 'secondEp':'', 'rpcDebugFlag':'135', 'fqdn':''} + #updatecfgDict["rpcDebugFlag"] = rpcDebugFlagVal + #print ("===================: ", updatecfgDict) + + def init(self, conn, logSql): + tdLog.debug(f"start to excute {__file__}") + #tdSql.init(conn.cursor()) + tdSql.init(conn.cursor(), logSql) # output sql.txt file + + def getBuildPath(self): + selfPath = os.path.dirname(os.path.realpath(__file__)) + + if ("community" in selfPath): + projPath = selfPath[:selfPath.find("community")] + else: + projPath = selfPath[:selfPath.find("tests")] + + for root, dirs, files in os.walk(projPath): + if ("taosd" in files): + rootRealPath = os.path.dirname(os.path.realpath(root)) + if ("packaging" not in rootRealPath): + buildPath = root[:len(root) - len("/build/bin")] + break + return buildPath + + def newcur(self,cfg,host,port): + user = "root" + password = "taosdata" + con=taos.connect(host=host, user=user, password=password, config=cfg ,port=port) + cur=con.cursor() + print(cur) + return cur + + def initConsumerTable(self,cdbName='cdb'): + tdLog.info("create consume database, and consume info table, and consume result table") + tdSql.query("create database if not exists %s vgroups 1"%(cdbName)) + tdSql.query("drop table if exists %s.consumeinfo "%(cdbName)) + tdSql.query("drop table if exists %s.consumeresult "%(cdbName)) + + tdSql.query("create table %s.consumeinfo (ts timestamp, consumerid int, topiclist binary(1024), keylist binary(1024), expectmsgcnt bigint, ifcheckdata int, ifmanualcommit int)"%cdbName) + tdSql.query("create table %s.consumeresult (ts timestamp, consumerid int, consummsgcnt bigint, consumrowcnt bigint, checkresult int)"%cdbName) + + def initConsumerInfoTable(self,cdbName='cdb'): + tdLog.info("drop consumeinfo table") + tdSql.query("drop table if exists %s.consumeinfo "%(cdbName)) + tdSql.query("create table %s.consumeinfo (ts timestamp, consumerid int, topiclist binary(1024), keylist binary(1024), expectmsgcnt bigint, ifcheckdata int, ifmanualcommit int)"%cdbName) + + def insertConsumerInfo(self,consumerId, expectrowcnt,topicList,keyList,ifcheckdata,ifmanualcommit,cdbName='cdb'): + sql = "insert into %s.consumeinfo values "%cdbName + sql += "(now, %d, '%s', '%s', %d, %d, %d)"%(consumerId, topicList, keyList, expectrowcnt, ifcheckdata, ifmanualcommit) + tdLog.info("consume info sql: %s"%sql) + tdSql.query(sql) + + def selectConsumeResult(self,expectRows,cdbName='cdb'): + resultList=[] + while 1: + tdSql.query("select * from %s.consumeresult"%cdbName) + #tdLog.info("row: %d, %l64d, %l64d"%(tdSql.getData(0, 1),tdSql.getData(0, 2),tdSql.getData(0, 3)) + if tdSql.getRows() == expectRows: + break + else: + time.sleep(5) + + for i in range(expectRows): + tdLog.info ("consume id: %d, consume msgs: %d, consume rows: %d"%(tdSql.getData(i , 1), tdSql.getData(i , 2), tdSql.getData(i , 3))) + resultList.append(tdSql.getData(i , 3)) + + return resultList + + def startTmqSimProcess(self,buildPath,cfgPath,pollDelay,dbName,showMsg=1,showRow=1,cdbName='cdb',valgrind=0): + shellCmd = 'nohup ' + if valgrind == 1: + logFile = cfgPath + '/../log/valgrind-tmq.log' + shellCmd = 'nohup valgrind --log-file=' + logFile + shellCmd += '--tool=memcheck --leak-check=full --show-reachable=no --track-origins=yes --show-leak-kinds=all --num-callers=20 -v --workaround-gcc296-bugs=yes ' + + shellCmd += buildPath + '/build/bin/tmq_sim -c ' + cfgPath + shellCmd += " -y %d -d %s -g %d -r %d -w %s "%(pollDelay, dbName, showMsg, showRow, cdbName) + shellCmd += "> /dev/null 2>&1 &" + tdLog.info(shellCmd) + os.system(shellCmd) + + def create_database(self,tsql, dbName,dropFlag=1,vgroups=4,replica=1): + if dropFlag == 1: + tsql.execute("drop database if exists %s"%(dbName)) + + tsql.execute("create database if not exists %s vgroups %d replica %d"%(dbName, vgroups, replica)) + tdLog.debug("complete to create database %s"%(dbName)) + return + + def create_stable(self,tsql, dbName,stbName): + tsql.execute("create table if not exists %s.%s (ts timestamp, c1 bigint, c2 binary(16)) tags(t1 int)"%(dbName, stbName)) + tdLog.debug("complete to create %s.%s" %(dbName, stbName)) + return + + def create_ctables(self,tsql, dbName,stbName,ctbNum): + tsql.execute("use %s" %dbName) + pre_create = "create table" + sql = pre_create + #tdLog.debug("doing create one stable %s and %d child table in %s ..." %(stbname, count ,dbname)) + for i in range(ctbNum): + sql += " %s_%d using %s tags(%d)"%(stbName,i,stbName,i+1) + if (i > 0) and (i%100 == 0): + tsql.execute(sql) + sql = pre_create + if sql != pre_create: + tsql.execute(sql) + + tdLog.debug("complete to create %d child tables in %s.%s" %(ctbNum, dbName, stbName)) + return + + def insert_data_interlaceByMultiTbl(self,tsql,dbName,ctbPrefix,ctbNum,rowsPerTbl,batchNum,startTs=0): + tdLog.debug("start to insert data ............") + tsql.execute("use %s" %dbName) + pre_insert = "insert into " + sql = pre_insert + + if startTs == 0: + t = time.time() + startTs = int(round(t * 1000)) + + ctbDict = {} + for i in range(ctbNum): + ctbDict[i] = 0 + + #tdLog.debug("doing insert data into stable:%s rows:%d ..."%(stbName, allRows)) + rowsOfCtb = 0 + while rowsOfCtb < rowsPerTbl: + for i in range(ctbNum): + sql += " %s.%s_%d values "%(dbName,ctbPrefix,i) + for k in range(batchNum): + sql += "(%d, %d, 'tmqrow_%d') "%(startTs + ctbDict[i], ctbDict[i], ctbDict[i]) + ctbDict[i] += 1 + if (0 == ctbDict[i]%batchNum) or (ctbDict[i] == rowsPerTbl): + tsql.execute(sql) + sql = "insert into " + break + rowsOfCtb = ctbDict[0] + + tdLog.debug("insert data ............ [OK]") + return + + def insert_data(self,tsql,dbName,ctbPrefix,ctbNum,rowsPerTbl,batchNum,startTs=0): + tdLog.debug("start to insert data ............") + tsql.execute("use %s" %dbName) + pre_insert = "insert into " + sql = pre_insert + + if startTs == 0: + t = time.time() + startTs = int(round(t * 1000)) + + #tdLog.debug("doing insert data into stable:%s rows:%d ..."%(stbName, allRows)) + rowsOfSql = 0 + for i in range(ctbNum): + sql += " %s_%d values "%(ctbPrefix,i) + for j in range(rowsPerTbl): + sql += "(%d, %d, 'tmqrow_%d') "%(startTs + j, j, j) + rowsOfSql += 1 + if (j > 0) and ((rowsOfSql == batchNum) or (j == rowsPerTbl - 1)): + tsql.execute(sql) + rowsOfSql = 0 + if j < rowsPerTbl - 1: + sql = "insert into %s_%d values " %(ctbPrefix,i) + else: + sql = "insert into " + #end sql + if sql != pre_insert: + #print("insert sql:%s"%sql) + tsql.execute(sql) + tdLog.debug("insert data ............ [OK]") + return + + def insert_data_with_autoCreateTbl(self,tsql,dbName,stbName,ctbPrefix,ctbNum,rowsPerTbl,batchNum,startTs=0): + tdLog.debug("start to insert data wiht auto create child table ............") + tsql.execute("use %s" %dbName) + pre_insert = "insert into " + sql = pre_insert + + if startTs == 0: + t = time.time() + startTs = int(round(t * 1000)) + + #tdLog.debug("doing insert data into stable:%s rows:%d ..."%(stbName, allRows)) + rowsOfSql = 0 + for i in range(ctbNum): + sql += " %s.%s_%d using %s.%s tags (%d) values "%(dbName,ctbPrefix,i,dbName,stbName,i) + for j in range(rowsPerTbl): + sql += "(%d, %d, 'tmqrow_%d') "%(startTs + j, j, j) + rowsOfSql += 1 + if (j > 0) and ((rowsOfSql == batchNum) or (j == rowsPerTbl - 1)): + tsql.execute(sql) + rowsOfSql = 0 + if j < rowsPerTbl - 1: + sql = "insert into %s.%s_%d using %s.%s tags (%d) values " %(dbName,ctbPrefix,i,dbName,stbName,i) + else: + sql = "insert into " + #end sql + if sql != pre_insert: + #print("insert sql:%s"%sql) + tsql.execute(sql) + tdLog.debug("insert data ............ [OK]") + return + + def prepareEnv(self, **parameterDict): + # create new connector for my thread + tsql=self.newcur(parameterDict['cfg'], 'localhost', 6030) + + if parameterDict["actionType"] == actionType.CREATE_DATABASE: + self.create_database(tsql, parameterDict["dbName"]) + elif parameterDict["actionType"] == actionType.CREATE_STABLE: + self.create_stable(tsql, parameterDict["dbName"], parameterDict["stbName"]) + elif parameterDict["actionType"] == actionType.CREATE_CTABLE: + self.create_ctables(tsql, parameterDict["dbName"], parameterDict["stbName"], parameterDict["ctbNum"]) + elif parameterDict["actionType"] == actionType.INSERT_DATA: + self.insert_data(tsql, parameterDict["dbName"], parameterDict["stbName"], parameterDict["ctbNum"],\ + parameterDict["rowsPerTbl"],parameterDict["batchNum"]) + else: + tdLog.exit("not support's action: ", parameterDict["actionType"]) + + return + + def tmqCase1(self, cfgPath, buildPath): + tdLog.printNoPrefix("======== test case 1: ") + + self.initConsumerTable() + + # create and start thread + parameterDict = {'cfg': '', \ + 'actionType': 0, \ + 'dbName': 'db1', \ + 'dropFlag': 1, \ + 'vgroups': 4, \ + 'replica': 1, \ + 'stbName': 'stb1', \ + 'ctbNum': 10, \ + 'rowsPerTbl': 10000, \ + 'batchNum': 33, \ + 'startTs': 1640966400000} # 2022-01-01 00:00:00.000 + parameterDict['cfg'] = cfgPath + + self.create_database(tdSql, parameterDict["dbName"]) + self.create_stable(tdSql, parameterDict["dbName"], parameterDict["stbName"]) + self.create_ctables(tdSql, parameterDict["dbName"], parameterDict["stbName"], parameterDict["ctbNum"]) + self.insert_data_interlaceByMultiTbl(tdSql,parameterDict["dbName"],parameterDict["stbName"],parameterDict["ctbNum"],parameterDict["rowsPerTbl"],parameterDict["batchNum"]) + + tdLog.info("create topics from stb1") + topicFromStb1 = 'topic_stb1' + + tdSql.execute("create topic %s as select ts, c1, c2 from %s.%s" %(topicFromStb1, parameterDict['dbName'], parameterDict['stbName'])) + consumerId = 0 + expectrowcnt = parameterDict["rowsPerTbl"] * parameterDict["ctbNum"] + topicList = topicFromStb1 + ifcheckdata = 0 + ifManualCommit = 0 + keyList = 'group.id:cgrp1,\ + enable.auto.commit:false,\ + auto.commit.interval.ms:6000,\ + auto.offset.reset:earliest' + self.insertConsumerInfo(consumerId, expectrowcnt,topicList,keyList,ifcheckdata,ifManualCommit) + + tdLog.info("start consume processor") + pollDelay = 10 + showMsg = 1 + showRow = 1 + self.startTmqSimProcess(buildPath,cfgPath,pollDelay,parameterDict["dbName"],showMsg, showRow) + + tdLog.info("insert process end, and start to check consume result") + expectRows = 1 + resultList = self.selectConsumeResult(expectRows) + totalConsumeRows = 0 + for i in range(expectRows): + totalConsumeRows += resultList[i] + + if totalConsumeRows != expectrowcnt: + tdLog.info("act consume rows: %d, expect consume rows: %d"%(totalConsumeRows, expectrowcnt)) + tdLog.exit("tmq consume rows error!") + + tdSql.query("drop topic %s"%topicFromStb1) + + tdLog.printNoPrefix("======== test case 1 end ...... ") + + def tmqCase2(self, cfgPath, buildPath): + tdLog.printNoPrefix("======== test case 2: ") + + self.initConsumerTable() + + # create and start thread + parameterDict = {'cfg': '', \ + 'actionType': 0, \ + 'dbName': 'db2', \ + 'dropFlag': 1, \ + 'vgroups': 4, \ + 'replica': 1, \ + 'stbName': 'stb1', \ + 'ctbNum': 10, \ + 'rowsPerTbl': 10000, \ + 'batchNum': 100, \ + 'startTs': 1640966400000} # 2022-01-01 00:00:00.000 + parameterDict['cfg'] = cfgPath + + parameterDict2 = {'cfg': '', \ + 'actionType': 0, \ + 'dbName': 'db2', \ + 'dropFlag': 1, \ + 'vgroups': 4, \ + 'replica': 1, \ + 'stbName': 'stb2', \ + 'ctbNum': 10, \ + 'rowsPerTbl': 10000, \ + 'batchNum': 100, \ + 'startTs': 1640966400000} # 2022-01-01 00:00:00.000 + parameterDict2['cfg'] = cfgPath + + self.create_database(tdSql, parameterDict["dbName"]) + self.create_stable(tdSql, parameterDict["dbName"], parameterDict["stbName"]) + self.create_stable(tdSql, parameterDict2["dbName"], parameterDict2["stbName"]) + + tdLog.info("create topics from stb1") + topicFromStb1 = 'topic_stb1' + + tdSql.execute("create topic %s as select ts, c1, c2 from %s.%s" %(topicFromStb1, parameterDict['dbName'], parameterDict['stbName'])) + consumerId = 0 + expectrowcnt = parameterDict["rowsPerTbl"] * parameterDict["ctbNum"] + topicList = topicFromStb1 + ifcheckdata = 0 + ifManualCommit = 0 + keyList = 'group.id:cgrp1,\ + enable.auto.commit:false,\ + auto.commit.interval.ms:6000,\ + auto.offset.reset:earliest' + self.insertConsumerInfo(consumerId, expectrowcnt,topicList,keyList,ifcheckdata,ifManualCommit) + + tdLog.info("start consume processor") + pollDelay = 100 + showMsg = 1 + showRow = 1 + self.startTmqSimProcess(buildPath,cfgPath,pollDelay,parameterDict["dbName"],showMsg, showRow) + + tdLog.info("start create child tables of stb1 and stb2") + parameterDict['actionType'] = actionType.CREATE_CTABLE + parameterDict2['actionType'] = actionType.CREATE_CTABLE + + prepareEnvThread = threading.Thread(target=self.prepareEnv, kwargs=parameterDict) + prepareEnvThread.start() + prepareEnvThread2 = threading.Thread(target=self.prepareEnv, kwargs=parameterDict2) + prepareEnvThread2.start() + + prepareEnvThread.join() + prepareEnvThread2.join() + + tdLog.info("start insert data into child tables of stb1 and stb2") + parameterDict['actionType'] = actionType.INSERT_DATA + parameterDict2['actionType'] = actionType.INSERT_DATA + + prepareEnvThread = threading.Thread(target=self.prepareEnv, kwargs=parameterDict) + prepareEnvThread.start() + prepareEnvThread2 = threading.Thread(target=self.prepareEnv, kwargs=parameterDict2) + prepareEnvThread2.start() + + prepareEnvThread.join() + prepareEnvThread2.join() + + tdLog.info("insert process end, and start to check consume result") + expectRows = 1 + resultList = self.selectConsumeResult(expectRows) + totalConsumeRows = 0 + for i in range(expectRows): + totalConsumeRows += resultList[i] + + if totalConsumeRows != expectrowcnt: + tdLog.info("act consume rows: %d, expect consume rows: %d"%(totalConsumeRows, expectrowcnt)) + tdLog.exit("tmq consume rows error!") + + tdSql.query("drop topic %s"%topicFromStb1) + + tdLog.printNoPrefix("======== test case 2 end ...... ") + + def tmqCase3(self, cfgPath, buildPath): + tdLog.printNoPrefix("======== test case 3: ") + + self.initConsumerTable() + + # create and start thread + parameterDict = {'cfg': '', \ + 'actionType': 0, \ + 'dbName': 'db3', \ + 'dropFlag': 1, \ + 'vgroups': 4, \ + 'replica': 1, \ + 'stbName': 'stb1', \ + 'ctbNum': 10, \ + 'rowsPerTbl': 50000, \ + 'batchNum': 13, \ + 'startTs': 1640966400000} # 2022-01-01 00:00:00.000 + parameterDict['cfg'] = cfgPath + + self.create_database(tdSql, parameterDict["dbName"]) + self.create_stable(tdSql, parameterDict["dbName"], parameterDict["stbName"]) + self.create_ctables(tdSql, parameterDict["dbName"], parameterDict["stbName"], parameterDict["ctbNum"]) + self.insert_data(tdSql,parameterDict["dbName"],parameterDict["stbName"],parameterDict["ctbNum"],parameterDict["rowsPerTbl"],parameterDict["batchNum"]) + + tdLog.info("create topics from stb1") + topicFromStb1 = 'topic_stb1' + + tdSql.execute("create topic %s as select ts, c1, c2 from %s.%s" %(topicFromStb1, parameterDict['dbName'], parameterDict['stbName'])) + consumerId = 0 + expectrowcnt = parameterDict["rowsPerTbl"] * parameterDict["ctbNum"] + topicList = topicFromStb1 + ifcheckdata = 0 + ifManualCommit = 0 + keyList = 'group.id:cgrp1,\ + enable.auto.commit:false,\ + auto.commit.interval.ms:6000,\ + auto.offset.reset:earliest' + self.insertConsumerInfo(consumerId, expectrowcnt,topicList,keyList,ifcheckdata,ifManualCommit) + + tdLog.info("start consume processor") + pollDelay = 5 + showMsg = 1 + showRow = 1 + self.startTmqSimProcess(buildPath,cfgPath,pollDelay,parameterDict["dbName"],showMsg, showRow) + + time.sleep(5) + tdLog.info("drop som child table of stb1") + dropTblNum = 4 + tdSql.query("drop table if exists %s.%s_1"%(parameterDict["dbName"], parameterDict["stbName"])) + tdSql.query("drop table if exists %s.%s_2"%(parameterDict["dbName"], parameterDict["stbName"])) + tdSql.query("drop table if exists %s.%s_3"%(parameterDict["dbName"], parameterDict["stbName"])) + tdSql.query("drop table if exists %s.%s_4"%(parameterDict["dbName"], parameterDict["stbName"])) + + tdLog.info("drop some child tables, then start to check consume result") + expectRows = 1 + resultList = self.selectConsumeResult(expectRows) + totalConsumeRows = 0 + for i in range(expectRows): + totalConsumeRows += resultList[i] + + remaindrowcnt = parameterDict["rowsPerTbl"] * (parameterDict["ctbNum"] - dropTblNum) + + if not (totalConsumeRows < expectrowcnt and totalConsumeRows > remaindrowcnt): + tdLog.info("act consume rows: %d, expect consume rows: between %d and %d"%(totalConsumeRows, remaindrowcnt, expectrowcnt)) + tdLog.exit("tmq consume rows error!") + + tdSql.query("drop topic %s"%topicFromStb1) + + tdLog.printNoPrefix("======== test case 3 end ...... ") + + def tmqCase4(self, cfgPath, buildPath): + tdLog.printNoPrefix("======== test case 4: ") + + self.initConsumerTable() + + # create and start thread + parameterDict = {'cfg': '', \ + 'actionType': 0, \ + 'dbName': 'db4', \ + 'dropFlag': 1, \ + 'vgroups': 4, \ + 'replica': 1, \ + 'stbName': 'stb1', \ + 'ctbNum': 10, \ + 'rowsPerTbl': 10000, \ + 'batchNum': 100, \ + 'startTs': 1640966400000} # 2022-01-01 00:00:00.000 + parameterDict['cfg'] = cfgPath + + self.create_database(tdSql, parameterDict["dbName"]) + self.create_stable(tdSql, parameterDict["dbName"], parameterDict["stbName"]) + self.create_ctables(tdSql, parameterDict["dbName"], parameterDict["stbName"], parameterDict["ctbNum"]) + self.insert_data(tdSql,\ + parameterDict["dbName"],\ + parameterDict["stbName"],\ + parameterDict["ctbNum"],\ + parameterDict["rowsPerTbl"],\ + parameterDict["batchNum"]) + + tdLog.info("create topics from stb1") + topicFromStb1 = 'topic_stb1' + + tdSql.execute("create topic %s as select ts, c1, c2 from %s.%s" %(topicFromStb1, parameterDict['dbName'], parameterDict['stbName'])) + consumerId = 0 + expectrowcnt = parameterDict["rowsPerTbl"] * parameterDict["ctbNum"] + topicList = topicFromStb1 + ifcheckdata = 0 + ifManualCommit = 1 + keyList = 'group.id:cgrp1,\ + enable.auto.commit:false,\ + auto.commit.interval.ms:6000,\ + auto.offset.reset:earliest' + self.insertConsumerInfo(consumerId, expectrowcnt/4,topicList,keyList,ifcheckdata,ifManualCommit) + + tdLog.info("start consume processor") + pollDelay = 5 + showMsg = 1 + showRow = 1 + self.startTmqSimProcess(buildPath,cfgPath,pollDelay,parameterDict["dbName"],showMsg, showRow) + + tdLog.info("start to check consume result") + expectRows = 1 + resultList = self.selectConsumeResult(expectRows) + totalConsumeRows = 0 + for i in range(expectRows): + totalConsumeRows += resultList[i] + + if totalConsumeRows != expectrowcnt/4: + tdLog.info("act consume rows: %d, expect consume rows: %d"%(totalConsumeRows, expectrowcnt/4)) + tdLog.exit("tmq consume rows error!") + + self.initConsumerInfoTable() + consumerId = 1 + self.insertConsumerInfo(consumerId, expectrowcnt,topicList,keyList,ifcheckdata,ifManualCommit) + + tdLog.info("again start consume processor") + self.startTmqSimProcess(buildPath,cfgPath,pollDelay,parameterDict["dbName"],showMsg, showRow) + + tdLog.info("again check consume result") + expectRows = 2 + resultList = self.selectConsumeResult(expectRows) + totalConsumeRows = 0 + for i in range(expectRows): + totalConsumeRows += resultList[i] + + if totalConsumeRows != expectrowcnt: + tdLog.info("act consume rows: %d, expect consume rows: %d"%(totalConsumeRows, expectrowcnt)) + tdLog.exit("tmq consume rows error!") + + tdSql.query("drop topic %s"%topicFromStb1) + + tdLog.printNoPrefix("======== test case 4 end ...... ") + + def tmqCase5(self, cfgPath, buildPath): + tdLog.printNoPrefix("======== test case 5: ") + + self.initConsumerTable() + + # create and start thread + parameterDict = {'cfg': '', \ + 'actionType': 0, \ + 'dbName': 'db5', \ + 'dropFlag': 1, \ + 'vgroups': 4, \ + 'replica': 1, \ + 'stbName': 'stb1', \ + 'ctbNum': 10, \ + 'rowsPerTbl': 10000, \ + 'batchNum': 100, \ + 'startTs': 1640966400000} # 2022-01-01 00:00:00.000 + parameterDict['cfg'] = cfgPath + + self.create_database(tdSql, parameterDict["dbName"]) + self.create_stable(tdSql, parameterDict["dbName"], parameterDict["stbName"]) + self.create_ctables(tdSql, parameterDict["dbName"], parameterDict["stbName"], parameterDict["ctbNum"]) + self.insert_data(tdSql,\ + parameterDict["dbName"],\ + parameterDict["stbName"],\ + parameterDict["ctbNum"],\ + parameterDict["rowsPerTbl"],\ + parameterDict["batchNum"]) + + tdLog.info("create topics from stb1") + topicFromStb1 = 'topic_stb1' + + tdSql.execute("create topic %s as select ts, c1, c2 from %s.%s" %(topicFromStb1, parameterDict['dbName'], parameterDict['stbName'])) + consumerId = 0 + expectrowcnt = parameterDict["rowsPerTbl"] * parameterDict["ctbNum"] + topicList = topicFromStb1 + ifcheckdata = 0 + ifManualCommit = 0 + keyList = 'group.id:cgrp1,\ + enable.auto.commit:false,\ + auto.commit.interval.ms:6000,\ + auto.offset.reset:earliest' + self.insertConsumerInfo(consumerId, expectrowcnt/4,topicList,keyList,ifcheckdata,ifManualCommit) + + tdLog.info("start consume processor") + pollDelay = 5 + showMsg = 1 + showRow = 1 + self.startTmqSimProcess(buildPath,cfgPath,pollDelay,parameterDict["dbName"],showMsg, showRow) + + tdLog.info("start to check consume result") + expectRows = 1 + resultList = self.selectConsumeResult(expectRows) + totalConsumeRows = 0 + for i in range(expectRows): + totalConsumeRows += resultList[i] + + if totalConsumeRows != expectrowcnt/4: + tdLog.info("act consume rows: %d, expect consume rows: %d"%(totalConsumeRows, expectrowcnt/4)) + tdLog.exit("tmq consume rows error!") + + self.initConsumerInfoTable() + consumerId = 1 + self.insertConsumerInfo(consumerId, expectrowcnt,topicList,keyList,ifcheckdata,ifManualCommit) + + tdLog.info("again start consume processor") + self.startTmqSimProcess(buildPath,cfgPath,pollDelay,parameterDict["dbName"],showMsg, showRow) + + tdLog.info("again check consume result") + expectRows = 2 + resultList = self.selectConsumeResult(expectRows) + totalConsumeRows = 0 + for i in range(expectRows): + totalConsumeRows += resultList[i] + + if totalConsumeRows != (expectrowcnt * (1 + 1/4)): + tdLog.info("act consume rows: %d, expect consume rows: %d"%(totalConsumeRows, expectrowcnt)) + tdLog.exit("tmq consume rows error!") + + tdSql.query("drop topic %s"%topicFromStb1) + + tdLog.printNoPrefix("======== test case 5 end ...... ") + + def tmqCase6(self, cfgPath, buildPath): + tdLog.printNoPrefix("======== test case 6: ") + + self.initConsumerTable() + + # create and start thread + parameterDict = {'cfg': '', \ + 'actionType': 0, \ + 'dbName': 'db6', \ + 'dropFlag': 1, \ + 'vgroups': 4, \ + 'replica': 1, \ + 'stbName': 'stb1', \ + 'ctbNum': 10, \ + 'rowsPerTbl': 10000, \ + 'batchNum': 100, \ + 'startTs': 1640966400000} # 2022-01-01 00:00:00.000 + parameterDict['cfg'] = cfgPath + + self.create_database(tdSql, parameterDict["dbName"]) + self.create_stable(tdSql, parameterDict["dbName"], parameterDict["stbName"]) + self.create_ctables(tdSql, parameterDict["dbName"], parameterDict["stbName"], parameterDict["ctbNum"]) + self.insert_data(tdSql,\ + parameterDict["dbName"],\ + parameterDict["stbName"],\ + parameterDict["ctbNum"],\ + parameterDict["rowsPerTbl"],\ + parameterDict["batchNum"]) + + tdLog.info("create topics from stb1") + topicFromStb1 = 'topic_stb1' + + tdSql.execute("create topic %s as select ts, c1, c2 from %s.%s" %(topicFromStb1, parameterDict['dbName'], parameterDict['stbName'])) + consumerId = 0 + expectrowcnt = parameterDict["rowsPerTbl"] * parameterDict["ctbNum"] + topicList = topicFromStb1 + ifcheckdata = 0 + ifManualCommit = 1 + keyList = 'group.id:cgrp1,\ + enable.auto.commit:false,\ + auto.commit.interval.ms:6000,\ + auto.offset.reset:earliest' + self.insertConsumerInfo(consumerId, expectrowcnt/4,topicList,keyList,ifcheckdata,ifManualCommit) + + tdLog.info("start consume processor") + pollDelay = 5 + showMsg = 1 + showRow = 1 + self.startTmqSimProcess(buildPath,cfgPath,pollDelay,parameterDict["dbName"],showMsg, showRow) + + tdLog.info("start to check consume result") + expectRows = 1 + resultList = self.selectConsumeResult(expectRows) + totalConsumeRows = 0 + for i in range(expectRows): + totalConsumeRows += resultList[i] + + if totalConsumeRows != expectrowcnt/4: + tdLog.info("act consume rows: %d, expect consume rows: %d"%(totalConsumeRows, expectrowcnt/4)) + tdLog.exit("tmq consume rows error!") + + self.initConsumerInfoTable() + consumerId = 1 + keyList = 'group.id:cgrp1,\ + enable.auto.commit:false,\ + auto.commit.interval.ms:6000,\ + auto.offset.reset:latest' + self.insertConsumerInfo(consumerId, expectrowcnt,topicList,keyList,ifcheckdata,ifManualCommit) + + tdLog.info("again start consume processor") + self.startTmqSimProcess(buildPath,cfgPath,pollDelay,parameterDict["dbName"],showMsg, showRow) + + tdLog.info("again check consume result") + expectRows = 2 + resultList = self.selectConsumeResult(expectRows) + totalConsumeRows = 0 + for i in range(expectRows): + totalConsumeRows += resultList[i] + + if totalConsumeRows != expectrowcnt: + tdLog.info("act consume rows: %d, expect consume rows: %d"%(totalConsumeRows, expectrowcnt)) + tdLog.exit("tmq consume rows error!") + + tdSql.query("drop topic %s"%topicFromStb1) + + tdLog.printNoPrefix("======== test case 6 end ...... ") + + def tmqCase7(self, cfgPath, buildPath): + tdLog.printNoPrefix("======== test case 7: ") + + self.initConsumerTable() + + # create and start thread + parameterDict = {'cfg': '', \ + 'actionType': 0, \ + 'dbName': 'db7', \ + 'dropFlag': 1, \ + 'vgroups': 4, \ + 'replica': 1, \ + 'stbName': 'stb1', \ + 'ctbNum': 10, \ + 'rowsPerTbl': 10000, \ + 'batchNum': 100, \ + 'startTs': 1640966400000} # 2022-01-01 00:00:00.000 + parameterDict['cfg'] = cfgPath + + self.create_database(tdSql, parameterDict["dbName"]) + self.create_stable(tdSql, parameterDict["dbName"], parameterDict["stbName"]) + self.create_ctables(tdSql, parameterDict["dbName"], parameterDict["stbName"], parameterDict["ctbNum"]) + self.insert_data(tdSql,\ + parameterDict["dbName"],\ + parameterDict["stbName"],\ + parameterDict["ctbNum"],\ + parameterDict["rowsPerTbl"],\ + parameterDict["batchNum"]) + + tdLog.info("create topics from stb1") + topicFromStb1 = 'topic_stb1' + + tdSql.execute("create topic %s as select ts, c1, c2 from %s.%s" %(topicFromStb1, parameterDict['dbName'], parameterDict['stbName'])) + consumerId = 0 + expectrowcnt = parameterDict["rowsPerTbl"] * parameterDict["ctbNum"] + topicList = topicFromStb1 + ifcheckdata = 0 + ifManualCommit = 1 + keyList = 'group.id:cgrp1,\ + enable.auto.commit:false,\ + auto.commit.interval.ms:6000,\ + auto.offset.reset:latest' + self.insertConsumerInfo(consumerId, expectrowcnt,topicList,keyList,ifcheckdata,ifManualCommit) + + tdLog.info("start consume processor") + pollDelay = 5 + showMsg = 1 + showRow = 1 + self.startTmqSimProcess(buildPath,cfgPath,pollDelay,parameterDict["dbName"],showMsg, showRow) + + tdLog.info("start to check consume result") + expectRows = 1 + resultList = self.selectConsumeResult(expectRows) + totalConsumeRows = 0 + for i in range(expectRows): + totalConsumeRows += resultList[i] + + if totalConsumeRows != 0: + tdLog.info("act consume rows: %d, expect consume rows: %d"%(totalConsumeRows, 0)) + tdLog.exit("tmq consume rows error!") + + self.initConsumerInfoTable() + consumerId = 1 + self.insertConsumerInfo(consumerId, expectrowcnt,topicList,keyList,ifcheckdata,ifManualCommit) + + tdLog.info("again start consume processor") + self.startTmqSimProcess(buildPath,cfgPath,pollDelay,parameterDict["dbName"],showMsg, showRow) + + tdLog.info("again check consume result") + expectRows = 2 + resultList = self.selectConsumeResult(expectRows) + totalConsumeRows = 0 + for i in range(expectRows): + totalConsumeRows += resultList[i] + + if totalConsumeRows != 0: + tdLog.info("act consume rows: %d, expect consume rows: %d"%(totalConsumeRows, 0)) + tdLog.exit("tmq consume rows error!") + + tdSql.query("drop topic %s"%topicFromStb1) + + tdLog.printNoPrefix("======== test case 7 end ...... ") + + def tmqCase8(self, cfgPath, buildPath): + tdLog.printNoPrefix("======== test case 8: ") + + self.initConsumerTable() + + # create and start thread + parameterDict = {'cfg': '', \ + 'actionType': 0, \ + 'dbName': 'db8', \ + 'dropFlag': 1, \ + 'vgroups': 4, \ + 'replica': 1, \ + 'stbName': 'stb1', \ + 'ctbNum': 10, \ + 'rowsPerTbl': 10000, \ + 'batchNum': 100, \ + 'startTs': 1640966400000} # 2022-01-01 00:00:00.000 + parameterDict['cfg'] = cfgPath + + self.create_database(tdSql, parameterDict["dbName"]) + self.create_stable(tdSql, parameterDict["dbName"], parameterDict["stbName"]) + self.create_ctables(tdSql, parameterDict["dbName"], parameterDict["stbName"], parameterDict["ctbNum"]) + self.insert_data(tdSql,\ + parameterDict["dbName"],\ + parameterDict["stbName"],\ + parameterDict["ctbNum"],\ + parameterDict["rowsPerTbl"],\ + parameterDict["batchNum"]) + + tdLog.info("create topics from stb1") + topicFromStb1 = 'topic_stb1' + + tdSql.execute("create topic %s as select ts, c1, c2 from %s.%s" %(topicFromStb1, parameterDict['dbName'], parameterDict['stbName'])) + consumerId = 0 + expectrowcnt = parameterDict["rowsPerTbl"] * parameterDict["ctbNum"] + topicList = topicFromStb1 + ifcheckdata = 0 + ifManualCommit = 1 + keyList = 'group.id:cgrp1,\ + enable.auto.commit:false,\ + auto.commit.interval.ms:6000,\ + auto.offset.reset:latest' + self.insertConsumerInfo(consumerId, expectrowcnt,topicList,keyList,ifcheckdata,ifManualCommit) + + tdLog.info("start consume 0 processor") + pollDelay = 10 + showMsg = 1 + showRow = 1 + self.startTmqSimProcess(buildPath,cfgPath,pollDelay,parameterDict["dbName"],showMsg, showRow) + + tdLog.info("start to check consume 0 result") + expectRows = 1 + resultList = self.selectConsumeResult(expectRows) + totalConsumeRows = 0 + for i in range(expectRows): + totalConsumeRows += resultList[i] + + if totalConsumeRows != 0: + tdLog.info("act consume rows: %d, expect consume rows: %d"%(totalConsumeRows, 0)) + tdLog.exit("tmq consume rows error!") + + tdLog.info("start consume 1 processor") + self.startTmqSimProcess(buildPath,cfgPath,pollDelay,parameterDict["dbName"],showMsg, showRow) + + tdLog.info("start one new thread to insert data") + parameterDict['actionType'] = actionType.INSERT_DATA + prepareEnvThread = threading.Thread(target=self.prepareEnv, kwargs=parameterDict) + prepareEnvThread.start() + prepareEnvThread.join() + + tdLog.info("start to check consume 0 and 1 result") + expectRows = 2 + resultList = self.selectConsumeResult(expectRows) + totalConsumeRows = 0 + for i in range(expectRows): + totalConsumeRows += resultList[i] + + if totalConsumeRows != expectrowcnt: + tdLog.info("act consume rows: %d, expect consume rows: %d"%(totalConsumeRows, expectrowcnt)) + tdLog.exit("tmq consume rows error!") + + tdLog.info("start consume 2 processor") + self.startTmqSimProcess(buildPath,cfgPath,pollDelay,parameterDict["dbName"],showMsg, showRow) + + tdLog.info("start one new thread to insert data") + parameterDict['actionType'] = actionType.INSERT_DATA + prepareEnvThread = threading.Thread(target=self.prepareEnv, kwargs=parameterDict) + prepareEnvThread.start() + prepareEnvThread.join() + + tdLog.info("start to check consume 0 and 1 and 2 result") + expectRows = 3 + resultList = self.selectConsumeResult(expectRows) + totalConsumeRows = 0 + for i in range(expectRows): + totalConsumeRows += resultList[i] + + if totalConsumeRows != expectrowcnt*2: + tdLog.info("act consume rows: %d, expect consume rows: %d"%(totalConsumeRows, expectrowcnt*2)) + tdLog.exit("tmq consume rows error!") + + tdSql.query("drop topic %s"%topicFromStb1) + + tdLog.printNoPrefix("======== test case 8 end ...... ") + + def tmqCase9(self, cfgPath, buildPath): + tdLog.printNoPrefix("======== test case 9: ") + + self.initConsumerTable() + + # create and start thread + parameterDict = {'cfg': '', \ + 'actionType': 0, \ + 'dbName': 'db9', \ + 'dropFlag': 1, \ + 'vgroups': 4, \ + 'replica': 1, \ + 'stbName': 'stb1', \ + 'ctbNum': 10, \ + 'rowsPerTbl': 10000, \ + 'batchNum': 100, \ + 'startTs': 1640966400000} # 2022-01-01 00:00:00.000 + parameterDict['cfg'] = cfgPath + + self.create_database(tdSql, parameterDict["dbName"]) + self.create_stable(tdSql, parameterDict["dbName"], parameterDict["stbName"]) + self.create_ctables(tdSql, parameterDict["dbName"], parameterDict["stbName"], parameterDict["ctbNum"]) + self.insert_data(tdSql,\ + parameterDict["dbName"],\ + parameterDict["stbName"],\ + parameterDict["ctbNum"],\ + parameterDict["rowsPerTbl"],\ + parameterDict["batchNum"]) + + tdLog.info("create topics from stb1") + topicFromStb1 = 'topic_stb1' + + tdSql.execute("create topic %s as select ts, c1, c2 from %s.%s" %(topicFromStb1, parameterDict['dbName'], parameterDict['stbName'])) + consumerId = 0 + expectrowcnt = parameterDict["rowsPerTbl"] * parameterDict["ctbNum"] + topicList = topicFromStb1 + ifcheckdata = 0 + ifManualCommit = 1 + keyList = 'group.id:cgrp1,\ + enable.auto.commit:false,\ + auto.commit.interval.ms:6000,\ + auto.offset.reset:latest' + self.insertConsumerInfo(consumerId, expectrowcnt,topicList,keyList,ifcheckdata,ifManualCommit) + + tdLog.info("start consume 0 processor") + pollDelay = 10 + showMsg = 1 + showRow = 1 + self.startTmqSimProcess(buildPath,cfgPath,pollDelay,parameterDict["dbName"],showMsg, showRow) + + tdLog.info("start to check consume 0 result") + expectRows = 1 + resultList = self.selectConsumeResult(expectRows) + totalConsumeRows = 0 + for i in range(expectRows): + totalConsumeRows += resultList[i] + + if totalConsumeRows != 0: + tdLog.info("act consume rows: %d, expect consume rows: %d"%(totalConsumeRows, 0)) + tdLog.exit("tmq consume rows error!") + + tdLog.info("start consume 1 processor") + self.initConsumerInfoTable() + consumerId = 1 + ifManualCommit = 0 + self.insertConsumerInfo(consumerId, expectrowcnt,topicList,keyList,ifcheckdata,ifManualCommit) + self.startTmqSimProcess(buildPath,cfgPath,pollDelay,parameterDict["dbName"],showMsg, showRow) + + tdLog.info("start one new thread to insert data") + parameterDict['actionType'] = actionType.INSERT_DATA + prepareEnvThread = threading.Thread(target=self.prepareEnv, kwargs=parameterDict) + prepareEnvThread.start() + prepareEnvThread.join() + + tdLog.info("start to check consume 0 and 1 result") + expectRows = 2 + resultList = self.selectConsumeResult(expectRows) + totalConsumeRows = 0 + for i in range(expectRows): + totalConsumeRows += resultList[i] + + if totalConsumeRows != expectrowcnt: + tdLog.info("act consume rows: %d, expect consume rows: %d"%(totalConsumeRows, expectrowcnt)) + tdLog.exit("tmq consume rows error!") + + tdLog.info("start consume 2 processor") + self.startTmqSimProcess(buildPath,cfgPath,pollDelay,parameterDict["dbName"],showMsg, showRow) + + tdLog.info("start one new thread to insert data") + parameterDict['actionType'] = actionType.INSERT_DATA + prepareEnvThread = threading.Thread(target=self.prepareEnv, kwargs=parameterDict) + prepareEnvThread.start() + prepareEnvThread.join() + + tdLog.info("start to check consume 0 and 1 and 2 result") + expectRows = 3 + resultList = self.selectConsumeResult(expectRows) + totalConsumeRows = 0 + for i in range(expectRows): + totalConsumeRows += resultList[i] + + if totalConsumeRows != expectrowcnt*2: + tdLog.info("act consume rows: %d, expect consume rows: %d"%(totalConsumeRows, expectrowcnt*2)) + tdLog.exit("tmq consume rows error!") + + tdSql.query("drop topic %s"%topicFromStb1) + + tdLog.printNoPrefix("======== test case 9 end ...... ") + + def tmqCase10(self, cfgPath, buildPath): + tdLog.printNoPrefix("======== test case 10: ") + + self.initConsumerTable() + + # create and start thread + parameterDict = {'cfg': '', \ + 'actionType': 0, \ + 'dbName': 'db10', \ + 'dropFlag': 1, \ + 'vgroups': 4, \ + 'replica': 1, \ + 'stbName': 'stb1', \ + 'ctbNum': 10, \ + 'rowsPerTbl': 10000, \ + 'batchNum': 100, \ + 'startTs': 1640966400000} # 2022-01-01 00:00:00.000 + parameterDict['cfg'] = cfgPath + + self.create_database(tdSql, parameterDict["dbName"]) + self.create_stable(tdSql, parameterDict["dbName"], parameterDict["stbName"]) + self.create_ctables(tdSql, parameterDict["dbName"], parameterDict["stbName"], parameterDict["ctbNum"]) + self.insert_data(tdSql,\ + parameterDict["dbName"],\ + parameterDict["stbName"],\ + parameterDict["ctbNum"],\ + parameterDict["rowsPerTbl"],\ + parameterDict["batchNum"]) + + tdLog.info("create topics from stb1") + topicFromStb1 = 'topic_stb1' + + tdSql.execute("create topic %s as select ts, c1, c2 from %s.%s" %(topicFromStb1, parameterDict['dbName'], parameterDict['stbName'])) + consumerId = 0 + expectrowcnt = parameterDict["rowsPerTbl"] * parameterDict["ctbNum"] + topicList = topicFromStb1 + ifcheckdata = 0 + ifManualCommit = 1 + keyList = 'group.id:cgrp1,\ + enable.auto.commit:false,\ + auto.commit.interval.ms:6000,\ + auto.offset.reset:latest' + self.insertConsumerInfo(consumerId, expectrowcnt,topicList,keyList,ifcheckdata,ifManualCommit) + + tdLog.info("start consume 0 processor") + pollDelay = 10 + showMsg = 1 + showRow = 1 + self.startTmqSimProcess(buildPath,cfgPath,pollDelay,parameterDict["dbName"],showMsg, showRow) + + tdLog.info("start to check consume 0 result") + expectRows = 1 + resultList = self.selectConsumeResult(expectRows) + totalConsumeRows = 0 + for i in range(expectRows): + totalConsumeRows += resultList[i] + + if totalConsumeRows != 0: + tdLog.info("act consume rows: %d, expect consume rows: %d"%(totalConsumeRows, 0)) + tdLog.exit("tmq consume rows error!") + + tdLog.info("start consume 1 processor") + self.initConsumerInfoTable() + consumerId = 1 + ifManualCommit = 1 + self.insertConsumerInfo(consumerId, expectrowcnt-10000,topicList,keyList,ifcheckdata,ifManualCommit) + self.startTmqSimProcess(buildPath,cfgPath,pollDelay,parameterDict["dbName"],showMsg, showRow) + + tdLog.info("start one new thread to insert data") + parameterDict['actionType'] = actionType.INSERT_DATA + prepareEnvThread = threading.Thread(target=self.prepareEnv, kwargs=parameterDict) + prepareEnvThread.start() + prepareEnvThread.join() + + tdLog.info("start to check consume 0 and 1 result") + expectRows = 2 + resultList = self.selectConsumeResult(expectRows) + totalConsumeRows = 0 + for i in range(expectRows): + totalConsumeRows += resultList[i] + + if totalConsumeRows != expectrowcnt-10000: + tdLog.info("act consume rows: %d, expect consume rows: %d"%(totalConsumeRows, expectrowcnt-10000)) + tdLog.exit("tmq consume rows error!") + + tdLog.info("start consume 2 processor") + self.initConsumerInfoTable() + consumerId = 2 + ifManualCommit = 1 + self.insertConsumerInfo(consumerId, expectrowcnt+10000,topicList,keyList,ifcheckdata,ifManualCommit) + self.startTmqSimProcess(buildPath,cfgPath,pollDelay,parameterDict["dbName"],showMsg, showRow) + + tdLog.info("start one new thread to insert data") + parameterDict['actionType'] = actionType.INSERT_DATA + prepareEnvThread = threading.Thread(target=self.prepareEnv, kwargs=parameterDict) + prepareEnvThread.start() + prepareEnvThread.join() + + tdLog.info("start to check consume 0 and 1 and 2 result") + expectRows = 3 + resultList = self.selectConsumeResult(expectRows) + totalConsumeRows = 0 + for i in range(expectRows): + totalConsumeRows += resultList[i] + + if totalConsumeRows != expectrowcnt*2: + tdLog.info("act consume rows: %d, expect consume rows: %d"%(totalConsumeRows, expectrowcnt*2)) + tdLog.exit("tmq consume rows error!") + + tdSql.query("drop topic %s"%topicFromStb1) + + tdLog.printNoPrefix("======== test case 10 end ...... ") + + def tmqCase11(self, cfgPath, buildPath): + tdLog.printNoPrefix("======== test case 11: ") + + self.initConsumerTable() + + # create and start thread + parameterDict = {'cfg': '', \ + 'actionType': 0, \ + 'dbName': 'db11', \ + 'dropFlag': 1, \ + 'vgroups': 4, \ + 'replica': 1, \ + 'stbName': 'stb1', \ + 'ctbNum': 10, \ + 'rowsPerTbl': 10000, \ + 'batchNum': 100, \ + 'startTs': 1640966400000} # 2022-01-01 00:00:00.000 + parameterDict['cfg'] = cfgPath + + self.create_database(tdSql, parameterDict["dbName"]) + self.create_stable(tdSql, parameterDict["dbName"], parameterDict["stbName"]) + self.create_ctables(tdSql, parameterDict["dbName"], parameterDict["stbName"], parameterDict["ctbNum"]) + self.insert_data(tdSql,\ + parameterDict["dbName"],\ + parameterDict["stbName"],\ + parameterDict["ctbNum"],\ + parameterDict["rowsPerTbl"],\ + parameterDict["batchNum"]) + + tdLog.info("create topics from stb1") + topicFromStb1 = 'topic_stb1' + + tdSql.execute("create topic %s as select ts, c1, c2 from %s.%s" %(topicFromStb1, parameterDict['dbName'], parameterDict['stbName'])) + consumerId = 0 + expectrowcnt = parameterDict["rowsPerTbl"] * parameterDict["ctbNum"] + topicList = topicFromStb1 + ifcheckdata = 0 + ifManualCommit = 1 + keyList = 'group.id:cgrp1,\ + enable.auto.commit:false,\ + auto.commit.interval.ms:6000,\ + auto.offset.reset:none' + self.insertConsumerInfo(consumerId, expectrowcnt/4,topicList,keyList,ifcheckdata,ifManualCommit) + + tdLog.info("start consume processor") + pollDelay = 5 + showMsg = 1 + showRow = 1 + self.startTmqSimProcess(buildPath,cfgPath,pollDelay,parameterDict["dbName"],showMsg, showRow) + + tdLog.info("start to check consume result") + expectRows = 1 + resultList = self.selectConsumeResult(expectRows) + totalConsumeRows = 0 + for i in range(expectRows): + totalConsumeRows += resultList[i] + + if totalConsumeRows != 0: + tdLog.info("act consume rows: %d, expect consume rows: %d"%(totalConsumeRows, 0)) + tdLog.exit("tmq consume rows error!") + + self.initConsumerInfoTable() + consumerId = 1 + keyList = 'group.id:cgrp1,\ + enable.auto.commit:false,\ + auto.commit.interval.ms:6000,\ + auto.offset.reset:none' + self.insertConsumerInfo(consumerId, expectrowcnt,topicList,keyList,ifcheckdata,ifManualCommit) + + tdLog.info("again start consume processor") + self.startTmqSimProcess(buildPath,cfgPath,pollDelay,parameterDict["dbName"],showMsg, showRow) + + tdLog.info("again check consume result") + expectRows = 2 + resultList = self.selectConsumeResult(expectRows) + totalConsumeRows = 0 + for i in range(expectRows): + totalConsumeRows += resultList[i] + + if totalConsumeRows != 0: + tdLog.info("act consume rows: %d, expect consume rows: %d"%(totalConsumeRows, 0)) + tdLog.exit("tmq consume rows error!") + + tdSql.query("drop topic %s"%topicFromStb1) + + tdLog.printNoPrefix("======== test case 11 end ...... ") + + def tmqCase12(self, cfgPath, buildPath): + tdLog.printNoPrefix("======== test case 12: ") + + self.initConsumerTable() + + # create and start thread + parameterDict = {'cfg': '', \ + 'actionType': 0, \ + 'dbName': 'db12', \ + 'dropFlag': 1, \ + 'vgroups': 4, \ + 'replica': 1, \ + 'stbName': 'stb1', \ + 'ctbNum': 10, \ + 'rowsPerTbl': 10000, \ + 'batchNum': 100, \ + 'startTs': 1640966400000} # 2022-01-01 00:00:00.000 + parameterDict['cfg'] = cfgPath + + self.create_database(tdSql, parameterDict["dbName"]) + self.create_stable(tdSql, parameterDict["dbName"], parameterDict["stbName"]) + self.create_ctables(tdSql, parameterDict["dbName"], parameterDict["stbName"], parameterDict["ctbNum"]) + self.insert_data(tdSql,\ + parameterDict["dbName"],\ + parameterDict["stbName"],\ + parameterDict["ctbNum"],\ + parameterDict["rowsPerTbl"],\ + parameterDict["batchNum"]) + + tdLog.info("create topics from stb1") + topicFromStb1 = 'topic_stb1' + + tdSql.execute("create topic %s as select ts, c1, c2 from %s.%s" %(topicFromStb1, parameterDict['dbName'], parameterDict['stbName'])) + consumerId = 0 + expectrowcnt = parameterDict["rowsPerTbl"] * parameterDict["ctbNum"] + topicList = topicFromStb1 + ifcheckdata = 0 + ifManualCommit = 0 + keyList = 'group.id:cgrp1,\ + enable.auto.commit:false,\ + auto.commit.interval.ms:6000,\ + auto.offset.reset:earliest' + self.insertConsumerInfo(consumerId, expectrowcnt/4,topicList,keyList,ifcheckdata,ifManualCommit) + + tdLog.info("start consume processor") + pollDelay = 5 + showMsg = 1 + showRow = 1 + self.startTmqSimProcess(buildPath,cfgPath,pollDelay,parameterDict["dbName"],showMsg, showRow) + + tdLog.info("start to check consume result") + expectRows = 1 + resultList = self.selectConsumeResult(expectRows) + totalConsumeRows = 0 + for i in range(expectRows): + totalConsumeRows += resultList[i] + + if totalConsumeRows != expectrowcnt/4: + tdLog.info("act consume rows: %d, expect consume rows: %d"%(totalConsumeRows, expectrowcnt/4)) + tdLog.exit("tmq consume rows error!") + + self.initConsumerInfoTable() + consumerId = 1 + keyList = 'group.id:cgrp1,\ + enable.auto.commit:false,\ + auto.commit.interval.ms:6000,\ + auto.offset.reset:none' + self.insertConsumerInfo(consumerId, expectrowcnt,topicList,keyList,ifcheckdata,ifManualCommit) + + tdLog.info("again start consume processor") + self.startTmqSimProcess(buildPath,cfgPath,pollDelay,parameterDict["dbName"],showMsg, showRow) + + tdLog.info("again check consume result") + expectRows = 2 + resultList = self.selectConsumeResult(expectRows) + totalConsumeRows = 0 + for i in range(expectRows): + totalConsumeRows += resultList[i] + + if totalConsumeRows != expectrowcnt/4: + tdLog.info("act consume rows: %d, expect consume rows: %d"%(totalConsumeRows, expectrowcnt/4)) + tdLog.exit("tmq consume rows error!") + + tdSql.query("drop topic %s"%topicFromStb1) + + tdLog.printNoPrefix("======== test case 12 end ...... ") + + def tmqCase13(self, cfgPath, buildPath): + tdLog.printNoPrefix("======== test case 13: ") + + self.initConsumerTable() + + # create and start thread + parameterDict = {'cfg': '', \ + 'actionType': 0, \ + 'dbName': 'db13', \ + 'dropFlag': 1, \ + 'vgroups': 4, \ + 'replica': 1, \ + 'stbName': 'stb1', \ + 'ctbNum': 10, \ + 'rowsPerTbl': 10000, \ + 'batchNum': 100, \ + 'startTs': 1640966400000} # 2022-01-01 00:00:00.000 + parameterDict['cfg'] = cfgPath + + self.create_database(tdSql, parameterDict["dbName"]) + self.create_stable(tdSql, parameterDict["dbName"], parameterDict["stbName"]) + self.create_ctables(tdSql, parameterDict["dbName"], parameterDict["stbName"], parameterDict["ctbNum"]) + self.insert_data(tdSql,\ + parameterDict["dbName"],\ + parameterDict["stbName"],\ + parameterDict["ctbNum"],\ + parameterDict["rowsPerTbl"],\ + parameterDict["batchNum"]) + + tdLog.info("create topics from stb1") + topicFromStb1 = 'topic_stb1' + + tdSql.execute("create topic %s as select ts, c1, c2 from %s.%s" %(topicFromStb1, parameterDict['dbName'], parameterDict['stbName'])) + consumerId = 0 + expectrowcnt = parameterDict["rowsPerTbl"] * parameterDict["ctbNum"] + topicList = topicFromStb1 + ifcheckdata = 0 + ifManualCommit = 1 + keyList = 'group.id:cgrp1,\ + enable.auto.commit:false,\ + auto.commit.interval.ms:6000,\ + auto.offset.reset:earliest' + self.insertConsumerInfo(consumerId, expectrowcnt/4,topicList,keyList,ifcheckdata,ifManualCommit) + + tdLog.info("start consume processor") + pollDelay = 5 + showMsg = 1 + showRow = 1 + self.startTmqSimProcess(buildPath,cfgPath,pollDelay,parameterDict["dbName"],showMsg, showRow) + + tdLog.info("start to check consume result") + expectRows = 1 + resultList = self.selectConsumeResult(expectRows) + totalConsumeRows = 0 + for i in range(expectRows): + totalConsumeRows += resultList[i] + + if totalConsumeRows != expectrowcnt/4: + tdLog.info("act consume rows: %d, expect consume rows: %d"%(totalConsumeRows, expectrowcnt/4)) + tdLog.exit("tmq consume rows error!") + + self.initConsumerInfoTable() + consumerId = 1 + ifManualCommit = 1 + keyList = 'group.id:cgrp1,\ + enable.auto.commit:false,\ + auto.commit.interval.ms:6000,\ + auto.offset.reset:none' + self.insertConsumerInfo(consumerId, expectrowcnt/2,topicList,keyList,ifcheckdata,ifManualCommit) + + tdLog.info("again start consume processor") + self.startTmqSimProcess(buildPath,cfgPath,pollDelay,parameterDict["dbName"],showMsg, showRow) + + tdLog.info("again check consume result") + expectRows = 2 + resultList = self.selectConsumeResult(expectRows) + totalConsumeRows = 0 + for i in range(expectRows): + totalConsumeRows += resultList[i] + + if totalConsumeRows != expectrowcnt*(1/2+1/4): + tdLog.info("act consume rows: %d, expect consume rows: %d"%(totalConsumeRows, expectrowcnt*(1/2+1/4))) + tdLog.exit("tmq consume rows error!") + + self.initConsumerInfoTable() + consumerId = 2 + ifManualCommit = 1 + keyList = 'group.id:cgrp1,\ + enable.auto.commit:false,\ + auto.commit.interval.ms:6000,\ + auto.offset.reset:none' + self.insertConsumerInfo(consumerId, expectrowcnt,topicList,keyList,ifcheckdata,ifManualCommit) + + tdLog.info("again start consume processor") + self.startTmqSimProcess(buildPath,cfgPath,pollDelay,parameterDict["dbName"],showMsg, showRow) + + tdLog.info("again check consume result") + expectRows = 3 + resultList = self.selectConsumeResult(expectRows) + totalConsumeRows = 0 + for i in range(expectRows): + totalConsumeRows += resultList[i] + + if totalConsumeRows != expectrowcnt: + tdLog.info("act consume rows: %d, expect consume rows: %d"%(totalConsumeRows, expectrowcnt)) + tdLog.exit("tmq consume rows error!") + + tdSql.query("drop topic %s"%topicFromStb1) + + tdLog.printNoPrefix("======== test case 13 end ...... ") + + def run(self): + tdSql.prepare() + + buildPath = self.getBuildPath() + if (buildPath == ""): + tdLog.exit("taosd not found!") + else: + tdLog.info("taosd found in %s" % buildPath) + cfgPath = buildPath + "/../sim/psim/cfg" + tdLog.info("cfgPath: %s" % cfgPath) + + self.tmqCase1(cfgPath, buildPath) + # self.tmqCase2(cfgPath, buildPath) + # self.tmqCase3(cfgPath, buildPath) + # self.tmqCase4(cfgPath, buildPath) + # self.tmqCase5(cfgPath, buildPath) + + def stop(self): + tdSql.close() + tdLog.success(f"{__file__} successfully executed") + +event = threading.Event() + +tdCases.addLinux(__file__, TDTestCase()) +tdCases.addWindows(__file__, TDTestCase()) diff --git a/tests/system-test/fulltest.sh b/tests/system-test/fulltest.sh index 37dbc3cc133f3d32dbdd7038cefa881e4eb2513b..1724abcfda16d63d55d5ae81074ba95b73df22ce 100755 --- a/tests/system-test/fulltest.sh +++ b/tests/system-test/fulltest.sh @@ -23,10 +23,10 @@ python3 ./test.py -f 2-query/length.py python3 ./test.py -f 2-query/char_length.py python3 ./test.py -f 2-query/upper.py python3 ./test.py -f 2-query/lower.py -python3 ./test.py -f 2-query/join.py +#python3 ./test.py -f 2-query/join.py python3 ./test.py -f 2-query/cast.py -python3 ./test.py -f 2-query/concat.py -python3 ./test.py -f 2-query/concat_ws.py +#python3 ./test.py -f 2-query/concat.py +#python3 ./test.py -f 2-query/concat_ws.py python3 ./test.py -f 2-query/check_tsdb.py # python3 ./test.py -f 2-query/union.py # python3 ./test.py -f 2-query/union2.py @@ -50,6 +50,7 @@ python3 ./test.py -f 2-query/Timediff.py python3 ./test.py -f 2-query/top.py python3 ./test.py -f 2-query/bottom.py + python3 ./test.py -f 2-query/abs.py python3 ./test.py -f 2-query/ceil.py python3 ./test.py -f 2-query/floor.py @@ -64,8 +65,7 @@ python3 ./test.py -f 2-query/arcsin.py python3 ./test.py -f 2-query/arccos.py python3 ./test.py -f 2-query/arctan.py python3 ./test.py -f 2-query/query_cols_tags_and_or.py -python3 ./test.py -f 2-query/nestedQuery.py - +#python3 ./test.py -f 2-query/nestedQuery.py python3 ./test.py -f 7-tmq/basic5.py python3 ./test.py -f 7-tmq/subscribeDb.py diff --git a/tools/taos-tools b/tools/taos-tools index a8bb88c9056735919fc50bf9b12d9562f17e844f..4d83d8c62973506f760bcaa3a33f4665ed9046d0 160000 --- a/tools/taos-tools +++ b/tools/taos-tools @@ -1 +1 @@ -Subproject commit a8bb88c9056735919fc50bf9b12d9562f17e844f +Subproject commit 4d83d8c62973506f760bcaa3a33f4665ed9046d0