diff --git a/docs/en/12-taos-sql/14-stream.md b/docs/en/12-taos-sql/14-stream.md index 6f2343d347..e1bf18c854 100644 --- a/docs/en/12-taos-sql/14-stream.md +++ b/docs/en/12-taos-sql/14-stream.md @@ -41,16 +41,28 @@ window_clause: { SESSION(ts_col, tol_val) | STATE_WINDOW(col) | INTERVAL(interval_val [, interval_offset]) [SLIDING (sliding_val)] + | EVENT_WINDOW START WITH start_trigger_condition END WITH end_trigger_condition + | COUNT_WINDOW(count_val[, sliding_val]) } ``` `SESSION` indicates a session window, and `tol_val` indicates the maximum range of the time interval. If the time interval between two continuous rows are within the time interval specified by `tol_val` they belong to the same session window; otherwise a new session window is started automatically. +`EVENT_WINDOW` is determined according to the window start condition and the window close condition. The window is started when `start_trigger_condition` is evaluated to true, the window is closed when `end_trigger_condition` is evaluated to true. `start_trigger_condition` and `end_trigger_condition` can be any conditional expressions supported by TDengine and can include multiple columns. + +`COUNT_WINDOW` is a counting window that is divided by a fixed number of data rows.`count_val`: A constant, which is a positive integer and must be greater than or equal to 2. The maximum value is 2147483648. `count_val` represents the maximum number of data rows contained in each `COUNT_WINDOW`. When the total number of data rows cannot be divided by `count_val`, the number of rows in the last window will be less than `count_val`. `sliding_val`: is a constant that represents the number of window slides, similar to `SLIDING` in `INTERVAL`. + For example, the following SQL statement creates a stream and automatically creates a supertable named `avg_vol`. The stream has a 1 minute time window that slides forward in 30 second intervals to calculate the average voltage of the meters supertable. ```sql CREATE STREAM avg_vol_s INTO avg_vol AS SELECT _wstart, count(*), avg(voltage) FROM meters PARTITION BY tbname INTERVAL(1m) SLIDING(30s); + +CREATE STREAM streams0 INTO streamt0 AS +SELECT _wstart, count(*), avg(voltage) from meters PARTITION BY tbname EVENT_WINDOW START WITH voltage < 0 END WITH voltage > 9; + +CREATE STREAM streams1 IGNORE EXPIRED 1 WATERMARK 100s INTO streamt1 AS +SELECT _wstart, count(*), avg(voltage) from meters PARTITION BY tbname COUNT_WINDOW(10); ``` ## Partitions of Stream diff --git a/docs/zh/12-taos-sql/14-stream.md b/docs/zh/12-taos-sql/14-stream.md index 979fc436b9..8868b728f8 100644 --- a/docs/zh/12-taos-sql/14-stream.md +++ b/docs/zh/12-taos-sql/14-stream.md @@ -49,10 +49,14 @@ window_clause: { SESSION(ts_col, tol_val) | STATE_WINDOW(col) | INTERVAL(interval_val [, interval_offset]) [SLIDING (sliding_val)] + | EVENT_WINDOW START WITH start_trigger_condition END WITH end_trigger_condition + | COUNT_WINDOW(count_val[, sliding_val]) } ``` 其中,SESSION 是会话窗口,tol_val 是时间间隔的最大范围。在 tol_val 时间间隔范围内的数据都属于同一个窗口,如果连续的两条数据的时间超过 tol_val,则自动开启下一个窗口。 +EVENT_WINDOW 是事件窗口,根据开始条件和结束条件来划定窗口。当 start_trigger_condition 满足时则窗口开始,直到 end_trigger_condition 满足时窗口关闭。 start_trigger_condition 和 end_trigger_condition 可以是任意 TDengine 支持的条件表达式,且可以包含不同的列。 +COUNT_WINDOW 是计数窗口,按固定的数据行数来划分窗口。 count_val 是常量,是正整数,必须大于等于2,小于2147483648。 count_val 表示每个 COUNT_WINDOW 包含的最大数据行数,总数据行数不能整除 count_val 时,最后一个窗口的行数会小于 count_val 。 sliding_val 是常量,表示窗口滑动的数量,类似于 INTERVAL 的 SLIDING 。 窗口的定义与时序数据特色查询中的定义完全相同,详见 [TDengine 特色查询](../distinguished) @@ -61,6 +65,12 @@ window_clause: { ```sql CREATE STREAM avg_vol_s INTO avg_vol AS SELECT _wstart, count(*), avg(voltage) FROM meters PARTITION BY tbname INTERVAL(1m) SLIDING(30s); + +CREATE STREAM streams0 INTO streamt0 AS +SELECT _wstart, count(*), avg(voltage) from meters PARTITION BY tbname EVENT_WINDOW START WITH voltage < 0 END WITH voltage > 9; + +CREATE STREAM streams1 IGNORE EXPIRED 1 WATERMARK 100s INTO streamt1 AS +SELECT _wstart, count(*), avg(voltage) from meters PARTITION BY tbname COUNT_WINDOW(10); ``` ## 流式计算的 partition diff --git a/source/client/inc/clientInt.h b/source/client/inc/clientInt.h index 989c6614a6..59eee6fd9d 100644 --- a/source/client/inc/clientInt.h +++ b/source/client/inc/clientInt.h @@ -354,6 +354,7 @@ SRequestObj* acquireRequest(int64_t rid); int32_t releaseRequest(int64_t rid); int32_t removeRequest(int64_t rid); void doDestroyRequest(void* p); +int64_t removeFromMostPrevReq(SRequestObj* pRequest); char* getDbOfConnection(STscObj* pObj); void setConnectionDB(STscObj* pTscObj, const char* db); diff --git a/source/client/src/clientEnv.c b/source/client/src/clientEnv.c index 1df50a51da..6c20813118 100644 --- a/source/client/src/clientEnv.c +++ b/source/client/src/clientEnv.c @@ -385,6 +385,33 @@ int32_t releaseRequest(int64_t rid) { return taosReleaseRef(clientReqRefPool, ri int32_t removeRequest(int64_t rid) { return taosRemoveRef(clientReqRefPool, rid); } +/// return the most previous req ref id +int64_t removeFromMostPrevReq(SRequestObj* pRequest) { + int64_t mostPrevReqRefId = pRequest->self; + SRequestObj* pTmp = pRequest; + while (pTmp->relation.prevRefId) { + pTmp = acquireRequest(pTmp->relation.prevRefId); + if (pTmp) { + mostPrevReqRefId = pTmp->self; + releaseRequest(mostPrevReqRefId); + } else { + break; + } + } + removeRequest(mostPrevReqRefId); + return mostPrevReqRefId; +} + +void destroyNextReq(int64_t nextRefId) { + if (nextRefId) { + SRequestObj* pObj = acquireRequest(nextRefId); + if (pObj) { + releaseRequest(nextRefId); + releaseRequest(nextRefId); + } + } +} + void destroySubRequests(SRequestObj *pRequest) { int32_t reqIdx = -1; SRequestObj *pReqList[16] = {NULL}; @@ -435,7 +462,7 @@ void doDestroyRequest(void *p) { uint64_t reqId = pRequest->requestId; tscTrace("begin to destroy request %" PRIx64 " p:%p", reqId, pRequest); - destroySubRequests(pRequest); + int64_t nextReqRefId = pRequest->relation.nextRefId; taosHashRemove(pRequest->pTscObj->pRequests, &pRequest->self, sizeof(pRequest->self)); @@ -471,6 +498,7 @@ void doDestroyRequest(void *p) { taosMemoryFreeClear(pRequest->sqlstr); taosMemoryFree(pRequest); tscTrace("end to destroy request %" PRIx64 " p:%p", reqId, pRequest); + destroyNextReq(nextReqRefId); } void destroyRequest(SRequestObj *pRequest) { @@ -479,7 +507,7 @@ void destroyRequest(SRequestObj *pRequest) { } taos_stop_query(pRequest); - removeRequest(pRequest->self); + removeFromMostPrevReq(pRequest); } void taosStopQueryImpl(SRequestObj *pRequest) { diff --git a/source/client/src/clientMain.c b/source/client/src/clientMain.c index 275ca0d2aa..e9379946b1 100644 --- a/source/client/src/clientMain.c +++ b/source/client/src/clientMain.c @@ -1254,54 +1254,34 @@ void doAsyncQuery(SRequestObj *pRequest, bool updateMetaForce) { } void restartAsyncQuery(SRequestObj *pRequest, int32_t code) { - int32_t reqIdx = 0; - SRequestObj *pReqList[16] = {NULL}; - SRequestObj *pUserReq = NULL; - pReqList[0] = pRequest; - uint64_t tmpRefId = 0; - SRequestObj *pTmp = pRequest; - while (pTmp->relation.prevRefId) { - tmpRefId = pTmp->relation.prevRefId; - pTmp = acquireRequest(tmpRefId); - if (pTmp) { - pReqList[++reqIdx] = pTmp; - releaseRequest(tmpRefId); - } else { - tscError("prev req ref 0x%" PRIx64 " is not there", tmpRefId); + tscInfo("restart request: %s p: %p", pRequest->sqlstr, pRequest); + SRequestObj* pUserReq = pRequest; + acquireRequest(pRequest->self); + while (pUserReq) { + if (pUserReq->self == pUserReq->relation.userRefId || pUserReq->relation.userRefId == 0) { break; - } - } - - tmpRefId = pRequest->relation.nextRefId; - while (tmpRefId) { - pTmp = acquireRequest(tmpRefId); - if (pTmp) { - tmpRefId = pTmp->relation.nextRefId; - removeRequest(pTmp->self); - releaseRequest(pTmp->self); } else { - tscError("next req ref 0x%" PRIx64 " is not there", tmpRefId); - break; + int64_t nextRefId = pUserReq->relation.nextRefId; + releaseRequest(pUserReq->self); + if (nextRefId) { + pUserReq = acquireRequest(nextRefId); + } } } - - for (int32_t i = reqIdx; i >= 0; i--) { - destroyCtxInRequest(pReqList[i]); - if (pReqList[i]->relation.userRefId == pReqList[i]->self || 0 == pReqList[i]->relation.userRefId) { - pUserReq = pReqList[i]; - } else { - removeRequest(pReqList[i]->self); - } - } - + bool hasSubRequest = pUserReq != pRequest || pRequest->relation.prevRefId != 0; if (pUserReq) { + destroyCtxInRequest(pUserReq); pUserReq->prevCode = code; memset(&pUserReq->relation, 0, sizeof(pUserReq->relation)); } else { - tscError("user req is missing"); + tscError("User req is missing"); + removeFromMostPrevReq(pRequest); return; } - + if (hasSubRequest) + removeFromMostPrevReq(pRequest); + else + releaseRequest(pUserReq->self); doAsyncQuery(pUserReq, true); } diff --git a/tests/parallel_test/cases.task b/tests/parallel_test/cases.task index 6975734226..c32790022a 100644 --- a/tests/parallel_test/cases.task +++ b/tests/parallel_test/cases.task @@ -238,7 +238,8 @@ ,,y,system-test,./pytest.sh python3 ./test.py -f 7-tmq/tmqSubscribeStb-r3.py -N 5 ,,y,system-test,./pytest.sh python3 ./test.py -f 7-tmq/tmq3mnodeSwitch.py -N 6 -M 3 -i True ,,y,system-test,./pytest.sh python3 ./test.py -f 7-tmq/tmq3mnodeSwitch.py -N 6 -M 3 -n 3 -i True -,,y,system-test,./pytest.sh python3 test.py -f 7-tmq/tmqVnodeTransform.py -N 2 -n 1 +,,y,system-test,./pytest.sh python3 test.py -f 7-tmq/tmqVnodeTransform-db-removewal.py -N 2 -n 1 +,,y,system-test,./pytest.sh python3 test.py -f 7-tmq/tmqVnodeTransform-stb-removewal.py -N 6 -n 3 ,,y,system-test,./pytest.sh python3 test.py -f 7-tmq/tmqVnodeTransform-stb.py -N 2 -n 1 ,,y,system-test,./pytest.sh python3 test.py -f 7-tmq/tmqVnodeTransform-stb.py -N 6 -n 3 #,,y,system-test,./pytest.sh python3 test.py -f 7-tmq/tmqVnodeTransform-db.py -N 6 -n 3 diff --git a/tests/system-test/7-tmq/tmqVnodeTransform.py b/tests/system-test/7-tmq/tmqVnodeTransform-db-removewal.py similarity index 78% rename from tests/system-test/7-tmq/tmqVnodeTransform.py rename to tests/system-test/7-tmq/tmqVnodeTransform-db-removewal.py index c2b002ead6..a853489c3f 100644 --- a/tests/system-test/7-tmq/tmqVnodeTransform.py +++ b/tests/system-test/7-tmq/tmqVnodeTransform-db-removewal.py @@ -122,135 +122,7 @@ class TDTestCase: tdLog.debug(f"redistributeSql:{redistributeSql}") tdSql.query(redistributeSql) tdLog.debug("redistributeSql 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': 1000, - '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) - expectRows = 0 - resultList = tmqCom.selectConsumeResult(expectRows) - 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', @@ -330,12 +202,90 @@ class TDTestCase: tdLog.printNoPrefix("======== test case 3 end ...... ") + def tmqCaseDbname(self): + tdLog.printNoPrefix("======== test case 4 subscrib Dbname start: ") + 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': 1000, + 'batchNum': 10, + 'startTs': 1640966400000, # 2022-01-01 00:00:00.000 + 'pollDelay': 10, + 'showMsg': 1, + 'showRow': 1, + 'snapshot': 0} + + paraDict['vgroups'] = self.vgroups + paraDict['ctbNum'] = self.ctbNum + paraDict['rowsPerTbl'] = self.rowsPerTbl + + topicNameList = ['topic4'] + 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 database ") + queryString = "database %s "%(paraDict['dbName']) + 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(1) + # 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") + expectRows = 2 + resultList = tmqCom.selectConsumeResult(expectRows) + + time.sleep(6) + for i in range(len(topicNameList)): + tdSql.query("drop topic %s"%topicNameList[i]) + + tdLog.printNoPrefix("======== test case 4 subscrib Dbname end ...... ") + def run(self): - self.prepareTestEnv() - self.tmqCase1() - self.tmqCase2() self.prepareTestEnv() self.tmqCase3() + self.prepareTestEnv() + self.tmqCaseDbname() def stop(self): tdSql.close() diff --git a/tests/system-test/7-tmq/tmqVnodeTransform-stb-removewal.py b/tests/system-test/7-tmq/tmqVnodeTransform-stb-removewal.py new file mode 100644 index 0000000000..40879d5c66 --- /dev/null +++ b/tests/system-test/7-tmq/tmqVnodeTransform-stb-removewal.py @@ -0,0 +1,266 @@ + +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 = 1000 + + def init(self, conn, logSql, replicaVar=1): + self.replicaVar = int(replicaVar) + tdLog.debug(f"start to excute {__file__}") + tdSql.init(conn.cursor(), True) + + def getDataPath(self): + selfPath = tdCom.getBuildPath() + + 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': 1000, + '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 + + tdCom.drop_all_db() + 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): + dnodesList = [] + tdSql.query("show dnodes") + for result in tdSql.queryResult: + dnodesList.append(result[0]) + print("dnodeList:",dnodesList) + tdSql.query("select * from information_schema.ins_vnodes") + vnodeId = 0 + for result in tdSql.queryResult: + if result[2] == 'dbt': + tdLog.debug("dnode is %d"%(result[0])) + dnodesList.remove(result[0]) + vnodeId = result[1] + print("its all data",dnodesList) + # if self.replicaVar == 1: + # redistributeSql = "redistribute vgroup %d dnode %d" %(vnodeId, dnodesList[0]) + # else: + redistributeSql = f"redistribute vgroup {vnodeId} " + for vgdnode in dnodesList: + redistributeSql += f"dnode {vgdnode} " + print(redistributeSql) + + tdLog.debug(f"redistributeSql:{redistributeSql}") + tdSql.query(redistributeSql) + tdLog.debug("redistributeSql 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': 1000, + '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) + expectRows = 0 + resultList = tmqCom.selectConsumeResult(expectRows) + time.sleep(1) + for i in range(len(topicNameList)): + tdSql.query("drop topic %s"%topicNameList[i]) + + tdLog.printNoPrefix("======== test case 2 end ...... ") + + def run(self): + self.prepareTestEnv() + self.tmqCase1() + self.tmqCase2() + + def stop(self): + tdSql.close() + tdLog.success(f"{__file__} successfully executed") + +event = threading.Event() + +tdCases.addLinux(__file__, TDTestCase()) +tdCases.addWindows(__file__, TDTestCase())