From d3f54e678f15233bce75fb5f77d4acffda543f30 Mon Sep 17 00:00:00 2001 From: wangmm0220 Date: Wed, 20 Sep 2023 18:27:43 +0800 Subject: [PATCH] feat:support vnode split in tmq --- source/dnode/mnode/impl/inc/mndSubscribe.h | 8 +- source/dnode/mnode/impl/src/mndConsumer.c | 34 +- source/dnode/mnode/impl/src/mndSubscribe.c | 116 +++++-- tests/system-test/7-tmq/tmqVnodeSplit.py | 329 +++++++++++++++++++ tests/system-test/7-tmq/tmqVnodeTransform.py | 4 +- 5 files changed, 454 insertions(+), 37 deletions(-) create mode 100644 tests/system-test/7-tmq/tmqVnodeSplit.py diff --git a/source/dnode/mnode/impl/inc/mndSubscribe.h b/source/dnode/mnode/impl/inc/mndSubscribe.h index ba4328b8fe..10864da5fb 100644 --- a/source/dnode/mnode/impl/inc/mndSubscribe.h +++ b/source/dnode/mnode/impl/inc/mndSubscribe.h @@ -32,11 +32,11 @@ void mndReleaseSubscribe(SMnode *pMnode, SMqSubscribeObj *pSub); int32_t mndMakeSubscribeKey(char *key, const char *cgroup, const char *topicName); -static FORCE_INLINE int32_t mndMakePartitionKey(char *key, const char *cgroup, const char *topicName, int32_t vgId) { - return snprintf(key, TSDB_PARTITION_KEY_LEN, "%d:%s:%s", vgId, cgroup, topicName); -} +//static FORCE_INLINE int32_t mndMakePartitionKey(char *key, const char *cgroup, const char *topicName, int32_t vgId) { +// return snprintf(key, TSDB_PARTITION_KEY_LEN, "%d:%s:%s", vgId, cgroup, topicName); +//} -int32_t mndDropSubByDB(SMnode *pMnode, STrans *pTrans, SDbObj *pDb); +//int32_t mndDropSubByDB(SMnode *pMnode, STrans *pTrans, SDbObj *pDb); int32_t mndDropSubByTopic(SMnode *pMnode, STrans *pTrans, const char *topic); int32_t mndSetDropSubCommitLogs(SMnode *pMnode, STrans *pTrans, SMqSubscribeObj *pSub); diff --git a/source/dnode/mnode/impl/src/mndConsumer.c b/source/dnode/mnode/impl/src/mndConsumer.c index c1494fd0d0..30ed4ed07f 100644 --- a/source/dnode/mnode/impl/src/mndConsumer.c +++ b/source/dnode/mnode/impl/src/mndConsumer.c @@ -311,6 +311,34 @@ static int32_t mndProcessMqTimerMsg(SRpcMsg *pMsg) { taosArrayPush(pRebSub->removedConsumers, &pConsumer->consumerId); } taosRUnLockLatch(&pConsumer->lock); + }else{ + int32_t newTopicNum = taosArrayGetSize(pConsumer->currentTopics); + for (int32_t i = 0; i < newTopicNum; i++) { + char * topic = taosArrayGetP(pConsumer->currentTopics, i); + SMqSubscribeObj *pSub = mndAcquireSubscribe(pMnode, pConsumer->cgroup, topic); + if (pSub == NULL) { + continue; + } + taosRLockLatch(&pSub->lock); + + // 2.2 iterate all vg assigned to the consumer of that topic + SMqConsumerEp *pConsumerEp = taosHashGet(pSub->consumerHash, &pConsumer->consumerId, sizeof(int64_t)); + int32_t vgNum = taosArrayGetSize(pConsumerEp->vgs); + + for (int32_t j = 0; j < vgNum; j++) { + SMqVgEp *pVgEp = taosArrayGetP(pConsumerEp->vgs, j); + SVgObj * pVgroup = mndAcquireVgroup(pMnode, pVgEp->vgId); + if (!pVgroup) { + char key[TSDB_SUBSCRIBE_KEY_LEN]; + mndMakeSubscribeKey(key, pConsumer->cgroup, topic); + mndGetOrCreateRebSub(pRebMsg->rebSubHash, key); + mInfo("vnode splitted, rebalance will be triggered"); + } + mndReleaseVgroup(pMnode, pVgroup); + } + taosRUnLockLatch(&pSub->lock); + mndReleaseSubscribe(pMnode, pSub); + } } } else if (status == MQ_CONSUMER_STATUS_LOST) { if (hbStatus > MND_CONSUMER_LOST_CLEAR_THRESHOLD) { // clear consumer if lost a day @@ -343,7 +371,7 @@ static int32_t mndProcessMqTimerMsg(SRpcMsg *pMsg) { } if (taosHashGetSize(pRebMsg->rebSubHash) != 0) { - mInfo("mq rebalance will be triggered"); + mInfo("mq rebalance will be triggered"); SRpcMsg rpcMsg = { .msgType = TDMT_MND_TMQ_DO_REBALANCE, .pCont = pRebMsg, @@ -548,8 +576,8 @@ static int32_t mndProcessAskEpReq(SRpcMsg *pMsg) { for (int32_t j = 0; j < vgNum; j++) { SMqVgEp *pVgEp = taosArrayGetP(pConsumerEp->vgs, j); - char offsetKey[TSDB_PARTITION_KEY_LEN]; - mndMakePartitionKey(offsetKey, pConsumer->cgroup, topic, pVgEp->vgId); +// char offsetKey[TSDB_PARTITION_KEY_LEN]; +// mndMakePartitionKey(offsetKey, pConsumer->cgroup, topic, pVgEp->vgId); if(epoch == -1){ SVgObj *pVgroup = mndAcquireVgroup(pMnode, pVgEp->vgId); diff --git a/source/dnode/mnode/impl/src/mndSubscribe.c b/source/dnode/mnode/impl/src/mndSubscribe.c index 93bafb34e1..b6ce8c17e6 100644 --- a/source/dnode/mnode/impl/src/mndSubscribe.c +++ b/source/dnode/mnode/impl/src/mndSubscribe.c @@ -361,8 +361,68 @@ static void transferVgroupsForConsumers(SMqRebOutputObj *pOutput, SHashObj *pHas } } +static int32_t processRemoveAddVgs(SMnode *pMnode, SMqRebOutputObj *pOutput){ + int32_t totalVgNum = 0; + SVgObj* pVgroup = NULL; + void* pIter = NULL; + SArray* newVgs = taosArrayInit(0, POINTER_BYTES); + while (1) { + pIter = sdbFetch(pMnode->pSdb, SDB_VGROUP, pIter, (void**)&pVgroup); + if (pIter == NULL) { + break; + } + + if (!mndVgroupInDb(pVgroup, pOutput->pSub->dbUid)) { + sdbRelease(pMnode->pSdb, pVgroup); + continue; + } + + totalVgNum++; + SMqVgEp* pVgEp = taosMemoryMalloc(sizeof(SMqVgEp)); + pVgEp->epSet = mndGetVgroupEpset(pMnode, pVgroup); + pVgEp->vgId = pVgroup->vgId; + taosArrayPush(newVgs, &pVgEp); + sdbRelease(pMnode->pSdb, pVgroup); + } + + pIter = NULL; + while (1) { + pIter = taosHashIterate(pOutput->pSub->consumerHash, pIter); + if (pIter == NULL) break; + SMqConsumerEp *pConsumerEp = (SMqConsumerEp *)pIter; + + int32_t j = 0; + while (j < taosArrayGetSize(pConsumerEp->vgs)) { + SMqVgEp *pVgEp = taosArrayGetP(pConsumerEp->vgs, j); + bool find = false; + for(int32_t k = 0; k < taosArrayGetSize(newVgs); k++){ + SMqVgEp *pnewVgEp = taosArrayGetP(newVgs, k); + if(pVgEp->vgId == pnewVgEp->vgId){ + taosArrayRemove(newVgs, k); + find = true; + break; + } + } + if(!find){ + mInfo("processRemoveAddVgs old vgId:%d", pVgEp->vgId); + tDeleteSMqVgEp(pVgEp); + taosArrayRemove(pConsumerEp->vgs, j); + continue; + } + j++; + } + } + + if(taosArrayGetSize(pOutput->pSub->unassignedVgs) == 0 && taosArrayGetSize(newVgs) != 0){ + taosArrayAddAll(pOutput->pSub->unassignedVgs, newVgs); + mInfo("processRemoveAddVgs add new vg num:%d", (int)taosArrayGetSize(newVgs)); + } + taosArrayDestroy(newVgs); + return totalVgNum; +} + static int32_t mndDoRebalance(SMnode *pMnode, const SMqRebInputObj *pInput, SMqRebOutputObj *pOutput) { - int32_t totalVgNum = pOutput->pSub->vgNum; + int32_t totalVgNum = processRemoveAddVgs(pMnode, pOutput); const char *pSubKey = pOutput->pSub->key; int32_t numOfRemoved = taosArrayGetSize(pInput->pRebInfo->removedConsumers); @@ -1093,33 +1153,33 @@ int32_t mndSetDropSubCommitLogs(SMnode *pMnode, STrans *pTrans, SMqSubscribeObj return 0; } -int32_t mndDropSubByDB(SMnode *pMnode, STrans *pTrans, SDbObj *pDb) { - int32_t code = 0; - SSdb *pSdb = pMnode->pSdb; - - void *pIter = NULL; - SMqSubscribeObj *pSub = NULL; - while (1) { - pIter = sdbFetch(pSdb, SDB_SUBSCRIBE, pIter, (void **)&pSub); - if (pIter == NULL) break; - - if (pSub->dbUid != pDb->uid) { - sdbRelease(pSdb, pSub); - continue; - } - - if (mndSetDropSubCommitLogs(pMnode, pTrans, pSub) < 0) { - sdbRelease(pSdb, pSub); - sdbCancelFetch(pSdb, pIter); - code = -1; - break; - } - - sdbRelease(pSdb, pSub); - } - - return code; -} +//int32_t mndDropSubByDB(SMnode *pMnode, STrans *pTrans, SDbObj *pDb) { +// int32_t code = 0; +// SSdb *pSdb = pMnode->pSdb; +// +// void *pIter = NULL; +// SMqSubscribeObj *pSub = NULL; +// while (1) { +// pIter = sdbFetch(pSdb, SDB_SUBSCRIBE, pIter, (void **)&pSub); +// if (pIter == NULL) break; +// +// if (pSub->dbUid != pDb->uid) { +// sdbRelease(pSdb, pSub); +// continue; +// } +// +// if (mndSetDropSubCommitLogs(pMnode, pTrans, pSub) < 0) { +// sdbRelease(pSdb, pSub); +// sdbCancelFetch(pSdb, pIter); +// code = -1; +// break; +// } +// +// sdbRelease(pSdb, pSub); +// } +// +// return code; +//} int32_t mndDropSubByTopic(SMnode *pMnode, STrans *pTrans, const char *topicName) { SSdb *pSdb = pMnode->pSdb; diff --git a/tests/system-test/7-tmq/tmqVnodeSplit.py b/tests/system-test/7-tmq/tmqVnodeSplit.py new file mode 100644 index 0000000000..ee4dc32e40 --- /dev/null +++ b/tests/system-test/7-tmq/tmqVnodeSplit.py @@ -0,0 +1,329 @@ + +import taos +import sys +import time +import socket +import os +import threading +import math + +from util.log import * +from util.sql import * +from util.cases import * +from util.dnodes import * +from util.common import * +from util.cluster import * +sys.path.append("./7-tmq") +from tmqCommon import * + +class TDTestCase: + def __init__(self): + self.vgroups = 1 + self.ctbNum = 10 + self.rowsPerTbl = 10000 + + def init(self, conn, logSql, replicaVar=1): + self.replicaVar = int(replicaVar) + tdLog.debug(f"start to excute {__file__}") + tdSql.init(conn.cursor(), False) + + def getDataPath(self): + selfPath = os.path.dirname(os.path.realpath(__file__)) + + return selfPath + '/../../../sim/dnode%d/data/vnode/vnode%d/wal/*'; + + def prepareTestEnv(self): + tdLog.printNoPrefix("======== prepare test env include database, stable, ctables, and insert data: ") + paraDict = {'dbName': 'dbt', + 'dropFlag': 1, + 'event': '', + 'vgroups': 1, + 'stbName': 'stb', + 'colPrefix': 'c', + 'tagPrefix': 't', + 'colSchema': [{'type': 'INT', 'count':1},{'type': 'BIGINT', 'count':1},{'type': 'DOUBLE', 'count':1},{'type': 'BINARY', 'len':32, 'count':1},{'type': 'NCHAR', 'len':32, 'count':1},{'type': 'TIMESTAMP', 'count':1}], + 'tagSchema': [{'type': 'INT', 'count':1},{'type': 'BIGINT', 'count':1},{'type': 'DOUBLE', 'count':1},{'type': 'BINARY', 'len':32, 'count':1},{'type': 'NCHAR', 'len':32, 'count':1}], + 'ctbPrefix': 'ctb', + 'ctbStartIdx': 0, + 'ctbNum': 10, + 'rowsPerTbl': 10000, + 'batchNum': 10, + 'startTs': 1640966400000, # 2022-01-01 00:00:00.000 + 'pollDelay': 60, + 'showMsg': 1, + 'showRow': 1, + 'snapshot': 0} + + paraDict['vgroups'] = self.vgroups + paraDict['ctbNum'] = self.ctbNum + paraDict['rowsPerTbl'] = self.rowsPerTbl + + tmqCom.initConsumerTable() + tdCom.create_database(tdSql, paraDict["dbName"],paraDict["dropFlag"], wal_retention_period=36000,vgroups=paraDict["vgroups"],replica=self.replicaVar) + tdLog.info("create stb") + tmqCom.create_stable(tdSql, dbName=paraDict["dbName"],stbName=paraDict["stbName"]) + # tdLog.info("create ctb") + # tmqCom.create_ctable(tdSql, dbName=paraDict["dbName"],stbName=paraDict["stbName"],ctbPrefix=paraDict['ctbPrefix'], + # ctbNum=paraDict["ctbNum"],ctbStartIdx=paraDict['ctbStartIdx']) + # tdLog.info("insert data") + # tmqCom.insert_data_interlaceByMultiTbl(tsql=tdSql,dbName=paraDict["dbName"],ctbPrefix=paraDict["ctbPrefix"], + # ctbNum=paraDict["ctbNum"],rowsPerTbl=paraDict["rowsPerTbl"],batchNum=paraDict["batchNum"], + # startTs=paraDict["startTs"],ctbStartIdx=paraDict['ctbStartIdx']) + + # tdLog.info("restart taosd to ensure that the data falls into the disk") + # tdDnodes.stop(1) + # tdDnodes.start(1) + # tdSql.query("flush database %s"%(paraDict['dbName'])) + return + + def restartAndRemoveWal(self): + tdDnodes = cluster.dnodes + tdSql.query("select * from information_schema.ins_vnodes") + for result in tdSql.queryResult: + if result[2] == 'dbt': + tdLog.debug("dnode is %d"%(result[0])) + dnodeId = result[0] + vnodeId = result[1] + + tdDnodes[dnodeId - 1].stoptaosd() + time.sleep(1) + dataPath = self.getDataPath() + dataPath = dataPath%(dnodeId,vnodeId) + os.system('rm -rf ' + dataPath) + tdLog.debug("dataPath:%s"%dataPath) + tdDnodes[dnodeId - 1].starttaosd() + time.sleep(1) + break + tdLog.debug("restart dnode ok") + + def redistributeVgroups(self): + tdSql.query("select * from information_schema.ins_vnodes") + vnodeId = 0 + for result in tdSql.queryResult: + if result[2] == 'dbt': + vnodeId = result[1] + tdLog.debug("vnode is %d"%(vnodeId)) + break + splitSql = "split vgroup %d" %(vnodeId) + tdLog.debug("splitSql:%s"%(splitSql)) + tdSql.query(splitSql) + tdLog.debug("splitSql ok") + + def tmqCase1(self): + tdLog.printNoPrefix("======== test case 1: ") + paraDict = {'dbName': 'dbt', + 'dropFlag': 1, + 'event': '', + 'vgroups': 1, + 'stbName': 'stb', + 'colPrefix': 'c', + 'tagPrefix': 't', + 'colSchema': [{'type': 'INT', 'count':1},{'type': 'BIGINT', 'count':1},{'type': 'DOUBLE', 'count':1},{'type': 'BINARY', 'len':32, 'count':1},{'type': 'NCHAR', 'len':32, 'count':1},{'type': 'TIMESTAMP', 'count':1}], + 'tagSchema': [{'type': 'INT', 'count':1},{'type': 'BIGINT', 'count':1},{'type': 'DOUBLE', 'count':1},{'type': 'BINARY', 'len':32, 'count':1},{'type': 'NCHAR', 'len':32, 'count':1}], + 'ctbPrefix': 'ctb', + 'ctbStartIdx': 0, + 'ctbNum': 10, + 'rowsPerTbl': 10000, + 'batchNum': 10, + 'startTs': 1640966400000, # 2022-01-01 00:00:00.000 + 'pollDelay': 60, + 'showMsg': 1, + 'showRow': 1, + 'snapshot': 0} + + paraDict['vgroups'] = self.vgroups + paraDict['ctbNum'] = self.ctbNum + paraDict['rowsPerTbl'] = self.rowsPerTbl + + topicNameList = ['topic1'] + # expectRowsList = [] + tmqCom.initConsumerTable() + + tdLog.info("create topics from stb with filter") + queryString = "select * from %s.%s"%(paraDict['dbName'], paraDict['stbName']) + # sqlString = "create topic %s as stable %s" %(topicNameList[0], paraDict['stbName']) + sqlString = "create topic %s as %s" %(topicNameList[0], queryString) + tdLog.info("create topic sql: %s"%sqlString) + tdSql.execute(sqlString) + # tdSql.query(queryString) + # expectRowsList.append(tdSql.getRows()) + + # init consume info, and start tmq_sim, then check consume result + tdLog.info("insert consume info to consume processor") + consumerId = 0 + expectrowcnt = paraDict["rowsPerTbl"] * paraDict["ctbNum"] * 2 + topicList = topicNameList[0] + ifcheckdata = 1 + ifManualCommit = 1 + keyList = 'group.id:cgrp1, enable.auto.commit:true, auto.commit.interval.ms:200, auto.offset.reset:earliest' + tmqCom.insertConsumerInfo(consumerId, expectrowcnt,topicList,keyList,ifcheckdata,ifManualCommit) + + tdLog.info("start consume processor") + tmqCom.startTmqSimProcess(pollDelay=paraDict['pollDelay'],dbName=paraDict["dbName"],showMsg=paraDict['showMsg'], showRow=paraDict['showRow'],snapshot=paraDict['snapshot']) + tdLog.info("wait the consume result") + + tdLog.info("create ctb1") + tmqCom.create_ctable(tdSql, dbName=paraDict["dbName"],stbName=paraDict["stbName"],ctbPrefix=paraDict['ctbPrefix'], + ctbNum=paraDict["ctbNum"],ctbStartIdx=paraDict['ctbStartIdx']) + tdLog.info("insert data") + pInsertThread = tmqCom.asyncInsertDataByInterlace(paraDict) + + tmqCom.getStartConsumeNotifyFromTmqsim() + tmqCom.getStartCommitNotifyFromTmqsim() + + #restart dnode & remove wal + self.restartAndRemoveWal() + + # redistribute vgroup + self.redistributeVgroups(); + + tdLog.info("create ctb2") + paraDict['ctbPrefix'] = "ctbn" + tmqCom.create_ctable(tdSql, dbName=paraDict["dbName"],stbName=paraDict["stbName"],ctbPrefix=paraDict['ctbPrefix'], + ctbNum=paraDict["ctbNum"],ctbStartIdx=paraDict['ctbStartIdx']) + tdLog.info("insert data") + pInsertThread1 = tmqCom.asyncInsertDataByInterlace(paraDict) + pInsertThread.join() + pInsertThread1.join() + + expectRows = 1 + resultList = tmqCom.selectConsumeResult(expectRows) + + if expectrowcnt / 2 >= resultList[0]: + tdLog.info("expect consume rows: %d, act consume rows: %d"%(expectrowcnt / 2, resultList[0])) + tdLog.exit("%d tmq consume rows error!"%consumerId) + + # tmqCom.checkFileContent(consumerId, queryString) + + time.sleep(10) + for i in range(len(topicNameList)): + tdSql.query("drop topic %s"%topicNameList[i]) + + tdLog.printNoPrefix("======== test case 1 end ...... ") + + def tmqCase2(self): + tdLog.printNoPrefix("======== test case 2: ") + paraDict = {'dbName':'dbt'} + + ntbName = "ntb" + + topicNameList = ['topic2'] + tmqCom.initConsumerTable() + + sqlString = "create table %s.%s(ts timestamp, i nchar(8))" %(paraDict['dbName'], ntbName) + tdLog.info("create nomal table sql: %s"%sqlString) + tdSql.execute(sqlString) + + tdLog.info("create topics from nomal table") + queryString = "select * from %s.%s"%(paraDict['dbName'], ntbName) + sqlString = "create topic %s as %s" %(topicNameList[0], queryString) + tdLog.info("create topic sql: %s"%sqlString) + tdSql.execute(sqlString) + tdSql.query("flush database %s"%(paraDict['dbName'])) + #restart dnode & remove wal + self.restartAndRemoveWal() + + # redistribute vgroup + self.redistributeVgroups(); + + sqlString = "alter table %s.%s modify column i nchar(16)" %(paraDict['dbName'], ntbName) + tdLog.info("alter table sql: %s"%sqlString) + tdSql.error(sqlString) + + time.sleep(1) + for i in range(len(topicNameList)): + tdSql.query("drop topic %s"%topicNameList[i]) + + tdLog.printNoPrefix("======== test case 2 end ...... ") + + def tmqCase3(self): + tdLog.printNoPrefix("======== test case 3: ") + paraDict = {'dbName': 'dbt', + 'dropFlag': 1, + 'event': '', + 'vgroups': 1, + 'stbName': 'stbn', + 'colPrefix': 'c', + 'tagPrefix': 't', + 'colSchema': [{'type': 'INT', 'count':1},{'type': 'BIGINT', 'count':1},{'type': 'DOUBLE', 'count':1},{'type': 'BINARY', 'len':32, 'count':1},{'type': 'NCHAR', 'len':32, 'count':1},{'type': 'TIMESTAMP', 'count':1}], + 'tagSchema': [{'type': 'INT', 'count':1},{'type': 'BIGINT', 'count':1},{'type': 'DOUBLE', 'count':1},{'type': 'BINARY', 'len':32, 'count':1},{'type': 'NCHAR', 'len':32, 'count':1}], + 'ctbPrefix': 'ctb', + 'ctbStartIdx': 0, + 'ctbNum': 10, + 'rowsPerTbl': 10000, + 'batchNum': 10, + 'startTs': 1640966400000, # 2022-01-01 00:00:00.000 + 'pollDelay': 2, + 'showMsg': 1, + 'showRow': 1, + 'snapshot': 0} + + paraDict['vgroups'] = self.vgroups + paraDict['ctbNum'] = self.ctbNum + paraDict['rowsPerTbl'] = self.rowsPerTbl + + topicNameList = ['topic3'] + tmqCom.initConsumerTable() + + tdLog.info("create stb") + tmqCom.create_stable(tdSql, dbName=paraDict["dbName"],stbName=paraDict["stbName"]) + + tdLog.info("create ctb") + tmqCom.create_ctable(tdSql, dbName=paraDict["dbName"],stbName=paraDict["stbName"],ctbPrefix=paraDict['ctbPrefix'], + ctbNum=paraDict["ctbNum"],ctbStartIdx=paraDict['ctbStartIdx']) + tdLog.info("insert data") + tmqCom.insert_data_interlaceByMultiTbl(tsql=tdSql,dbName=paraDict["dbName"],ctbPrefix=paraDict["ctbPrefix"], + ctbNum=paraDict["ctbNum"],rowsPerTbl=paraDict["rowsPerTbl"],batchNum=paraDict["batchNum"], + startTs=paraDict["startTs"],ctbStartIdx=paraDict['ctbStartIdx']) + + tdLog.info("create topics from stb with filter") + queryString = "select * from %s.%s"%(paraDict['dbName'], paraDict['stbName']) + sqlString = "create topic %s as %s" %(topicNameList[0], queryString) + tdLog.info("create topic sql: %s"%sqlString) + tdSql.execute(sqlString) + + # init consume info, and start tmq_sim, then check consume result + tdLog.info("insert consume info to consume processor") + consumerId = 0 + expectrowcnt = paraDict["rowsPerTbl"] * paraDict["ctbNum"] + topicList = topicNameList[0] + ifcheckdata = 1 + ifManualCommit = 1 + keyList = 'group.id:cgrp1, enable.auto.commit:true, auto.commit.interval.ms:200, auto.offset.reset:earliest' + tmqCom.insertConsumerInfo(consumerId, expectrowcnt,topicList,keyList,ifcheckdata,ifManualCommit) + + tdLog.info("start consume processor") + tmqCom.startTmqSimProcess(pollDelay=paraDict['pollDelay'],dbName=paraDict["dbName"],showMsg=paraDict['showMsg'], showRow=paraDict['showRow'],snapshot=paraDict['snapshot']) + tdLog.info("wait the consume result") + + time.sleep(5) + #restart dnode & remove wal + self.restartAndRemoveWal() + + # redistribute vgroup + self.redistributeVgroups(); + + tdLog.info("start consume processor") + tmqCom.startTmqSimProcess(pollDelay=paraDict['pollDelay'],dbName=paraDict["dbName"],showMsg=paraDict['showMsg'], showRow=paraDict['showRow'],snapshot=paraDict['snapshot']) + tdLog.info("wait the consume result") + + time.sleep(10) + for i in range(len(topicNameList)): + tdSql.query("drop topic %s"%topicNameList[i]) + + tdLog.printNoPrefix("======== test case 3 end ...... ") + + def run(self): + + tdSql.prepare() + self.prepareTestEnv() + self.tmqCase1() + + 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/tmqVnodeTransform.py b/tests/system-test/7-tmq/tmqVnodeTransform.py index 8db9ce0e13..8b2524a6bc 100644 --- a/tests/system-test/7-tmq/tmqVnodeTransform.py +++ b/tests/system-test/7-tmq/tmqVnodeTransform.py @@ -324,8 +324,8 @@ class TDTestCase: tdSql.prepare() self.prepareTestEnv() self.tmqCase1() - # self.tmqCase2() - # self.tmqCase3() + self.tmqCase2() + self.tmqCase3() def stop(self): tdSql.close()