diff --git a/docs/en/07-develop/08-cache.md b/docs/en/07-develop/08-cache.md index 743452faff6a2be8466318a7dab61a44e33c3664..8e86eff7414a02ad36a965eb092b8b9b65343301 100644 --- a/docs/en/07-develop/08-cache.md +++ b/docs/en/07-develop/08-cache.md @@ -1,19 +1,52 @@ --- sidebar_label: Cache title: Cache -description: "The latest row of each table is kept in cache to provide high performance query of latest state." +description: "Caching System inside TDengine" --- +To achieve the purpose of high performance data writing and querying, TDengine employs a lot of caching technologies in both server side and client side. + +## Write Cache + The cache management policy in TDengine is First-In-First-Out (FIFO). FIFO is also known as insert driven cache management policy and it is different from read driven cache management, which is more commonly known as Least-Recently-Used (LRU). FIFO simply stores the latest data in cache and flushes the oldest data in cache to disk, when the cache usage reaches a threshold. In IoT use cases, it is the current state i.e. the latest or most recent data that is important. The cache policy in TDengine, like much of the design and architecture of TDengine, is based on the nature of IoT data. -Caching the latest data provides the capability of retrieving data in milliseconds. With this capability, TDengine can be configured properly to be used as a caching system without deploying another separate caching system. This simplifies the system architecture and minimizes operational costs. The cache is emptied after TDengine is restarted. TDengine does not reload data from disk into cache, like a key-value caching system. +The memory space used by each vnode as write cache is determined when creating a database. Parameter `vgroups` and `buffer` can be used to specify the number of vnode and the size of write cache for each vnode when creating the database. Then, the total size of write cache for this database is `vgroups * buffer`. + +```sql +create database db0 vgroups 100 buffer 16MB +``` + +The above statement creates a database of 100 vnodes while each vnode has a write cache of 16MB. + +Even though in theory it's always better to have a larger cache, the extra effect would be very minor once the size of cache grows beyond a threshold. So normally it's enough to use the default value of `buffer` parameter. + +## Read Cache -The memory space used by the TDengine cache is fixed in size and configurable. It should be allocated based on application requirements and system resources. An independent memory pool is allocated for and managed by each vnode (virtual node) in TDengine. There is no sharing of memory pools between vnodes. All the tables belonging to a vnode share all the cache memory of the vnode. +When creating a database, it's also possible to specify whether to cache the latest data of each sub table, using parameter `cachelast`. There are 3 cases: +- 0: No cache for latest data +- 1: The last row of each table is cached, `last_row` function can benefit significantly from it +- 2: The latest non-NULL value of each column for each table is cached, `last` function can benefit very much when there is no `where`, `group by`, `order by` or `interval` clause +- 3: Bot hthe last row and the latest non-NULL value of each column for each table are cached, identical to the behavior of both 1 and 2 are set together -The memory pool is divided into blocks and data is stored in row format in memory and each block follows FIFO policy. The size of each block is determined by configuration parameter `cache` and the number of blocks for each vnode is determined by the parameter `blocks`. For each vnode, the total cache size is `cache * blocks`. A cache block needs to ensure that each table can store at least dozens of records, to be efficient. -`last_row` function can be used to retrieve the last row of a table or a STable to quickly show the current state of devices on monitoring screen. For example the below SQL statement retrieves the latest voltage of all meters in San Francisco, California. +## Meta Cache + +To process data writing and querying efficiently, each vnode caches the metadata that's already retrieved. Parameters `pages` and `pagesize` are used to specify the size of metadata cache for each vnode. ```sql -select last_row(voltage) from meters where location='California.SanFrancisco'; +create database db0 pages 128 pagesize 16kb ``` + +The above statement will create a database db0 each of whose vnode is allocated a meta cache of `128 * 16 KB = 2 MB` . + +## File System Cache + +TDengine utilizes WAL to provide basic reliability. The essential of WAL is to append data in a disk file, so the file system cache also plays an important role in the writing performance. Parameter `wal` can be used to specify the policy of writing WAL, there are 2 cases: +- 1: Write data to WAL without calling fsync, the data is actually written to the file system cache without flushing immediately, in this way you can get better write performance +- 2: Write data to WAL and invoke fsync, the data is immediately flushed to disk, in this way you can get higher reliability + +## Client Cache + +To improve the overall efficiency of processing data, besides the above caches, the core library `libtaos.so` (also referred to as `taosc`) which all client programs depend on also has its own cache. `taosc` caches the metadata of the databases, super tables, tables that the invoking client has accessed, plus other critical metadata such as the cluster topology. + +When multiple client programs are accessing a TDengine cluster, if one of the clients modifies some metadata, the cache may become invalid in other clients. If this case happens, the client programs need to "reset query cache" to invalidate the whole cache so that `taosc` is enforced to repull the metadata it needs to rebuild the cache. diff --git a/include/common/tmsg.h b/include/common/tmsg.h index c8e13fce3d6fdc45c8bd15fc2f59e6f91dc39c4c..e2d062911ecadc4f226d6af0a898d10600faafa3 100644 --- a/include/common/tmsg.h +++ b/include/common/tmsg.h @@ -525,6 +525,7 @@ typedef struct { int8_t superUser; int8_t connType; SEpSet epSet; + int32_t svrTimestamp; char sVer[TSDB_VERSION_LEN]; char sDetailVer[128]; } SConnectRsp; @@ -2233,6 +2234,7 @@ typedef struct { typedef struct { int64_t reqId; int64_t rspId; + int32_t svrTimestamp; SArray* rsps; // SArray } SClientHbBatchRsp; diff --git a/include/libs/scalar/scalar.h b/include/libs/scalar/scalar.h index 7c23184d931cc3af53990772a4eceedda4a285c5..d5e9a2e625380bf58cfbdd54db41d5c826dd168b 100644 --- a/include/libs/scalar/scalar.h +++ b/include/libs/scalar/scalar.h @@ -103,6 +103,7 @@ int32_t minScalarFunction(SScalarParam *pInput, int32_t inputNum, SScalarParam * int32_t maxScalarFunction(SScalarParam *pInput, int32_t inputNum, SScalarParam *pOutput); int32_t avgScalarFunction(SScalarParam *pInput, int32_t inputNum, SScalarParam *pOutput); int32_t stddevScalarFunction(SScalarParam *pInput, int32_t inputNum, SScalarParam *pOutput); +int32_t leastSQRScalarFunction(SScalarParam *pInput, int32_t inputNum, SScalarParam *pOutput); #ifdef __cplusplus } diff --git a/include/libs/scheduler/scheduler.h b/include/libs/scheduler/scheduler.h index 70ac7a630460a2917f11b93984e4c1434567e6ef..e6973cd390c10ff524f70549d161090582ee56ab 100644 --- a/include/libs/scheduler/scheduler.h +++ b/include/libs/scheduler/scheduler.h @@ -25,11 +25,6 @@ extern "C" { extern tsem_t schdRspSem; -typedef struct SSchedulerCfg { - uint32_t maxJobNum; - int32_t maxNodeTableNum; -} SSchedulerCfg; - typedef struct SQueryProfileSummary { int64_t startTs; // Object created and added into the message queue int64_t endTs; // the timestamp when the task is completed @@ -84,7 +79,7 @@ typedef struct SSchedulerReq { } SSchedulerReq; -int32_t schedulerInit(SSchedulerCfg *cfg); +int32_t schedulerInit(void); int32_t schedulerExecJob(SSchedulerReq *pReq, int64_t *pJob); @@ -96,6 +91,8 @@ int32_t schedulerGetTasksStatus(int64_t job, SArray *pSub); void schedulerStopQueryHb(void *pTrans); +int32_t schedulerUpdatePolicy(int32_t policy); +int32_t schedulerEnableReSchedule(bool enableResche); /** * Cancel query job diff --git a/include/util/taoserror.h b/include/util/taoserror.h index c057d48875822b2330342bf170d1b69a24391368..729f3eb3e26de526a008e85ceebda08fe7dc9a87 100644 --- a/include/util/taoserror.h +++ b/include/util/taoserror.h @@ -73,6 +73,7 @@ int32_t* taosGetErrno(); #define TSDB_CODE_MSG_DECODE_ERROR TAOS_DEF_ERROR_CODE(0, 0x0031) #define TSDB_CODE_NO_AVAIL_DISK TAOS_DEF_ERROR_CODE(0, 0x0032) #define TSDB_CODE_NOT_FOUND TAOS_DEF_ERROR_CODE(0, 0x0033) +#define TSDB_CODE_TIME_UNSYNCED TAOS_DEF_ERROR_CODE(0, 0x0034) #define TSDB_CODE_REF_NO_MEMORY TAOS_DEF_ERROR_CODE(0, 0x0040) #define TSDB_CODE_REF_FULL TAOS_DEF_ERROR_CODE(0, 0x0041) diff --git a/source/client/inc/clientInt.h b/source/client/inc/clientInt.h index f51cb7960ae0c7dd9aa87186d70c069081a8fbba..779fa681404434612332387d80beaace4c17e6ef 100644 --- a/source/client/inc/clientInt.h +++ b/source/client/inc/clientInt.h @@ -286,7 +286,7 @@ static FORCE_INLINE SReqResultInfo* tscGetCurResInfo(TAOS_RES* res) { extern SAppInfo appInfo; extern int32_t clientReqRefPool; extern int32_t clientConnRefPool; -extern void* tscQhandle; +extern int32_t timestampDeltaLimit; __async_send_cb_fn_t getMsgRspHandle(int32_t msgType); diff --git a/source/client/src/clientEnv.c b/source/client/src/clientEnv.c index 9e67dc6571aae2401bda0a18f348a602dad1790d..53a1bd223512f5c4805b56dfce6cde5f080332ca 100644 --- a/source/client/src/clientEnv.c +++ b/source/client/src/clientEnv.c @@ -35,6 +35,8 @@ SAppInfo appInfo; int32_t clientReqRefPool = -1; int32_t clientConnRefPool = -1; +int32_t timestampDeltaLimit = 900; // s + static TdThreadOnce tscinit = PTHREAD_ONCE_INIT; volatile int32_t tscInitRes = 0; @@ -181,7 +183,7 @@ void destroyTscObj(void *pObj) { destroyAllRequests(pTscObj->pRequests); taosHashCleanup(pTscObj->pRequests); - + schedulerStopQueryHb(pTscObj->pAppInfo->pTransporter); tscDebug("connObj 0x%" PRIx64 " p:%p destroyed, remain inst totalConn:%" PRId64, pTscObj->id, pTscObj, pTscObj->pAppInfo->numOfConns); @@ -363,8 +365,7 @@ void taos_init_imp(void) { SCatalogCfg cfg = {.maxDBCacheNum = 100, .maxTblCacheNum = 100}; catalogInit(&cfg); - SSchedulerCfg scfg = {.maxJobNum = 100}; - schedulerInit(&scfg); + schedulerInit(); tscDebug("starting to initialize TAOS driver"); taosSetCoreDump(true); diff --git a/source/client/src/clientHb.c b/source/client/src/clientHb.c index d7c2c26d231c16ecef385127d9ff9a349420d18e..6969e03e7c3176fd7cfcefa59f5ef2328ba9d959 100644 --- a/source/client/src/clientHb.c +++ b/source/client/src/clientHb.c @@ -70,7 +70,7 @@ static int32_t hbProcessDBInfoRsp(void *value, int32_t valueLen, struct SCatalog if (NULL == vgInfo) { return TSDB_CODE_TSC_OUT_OF_MEMORY; } - + vgInfo->vgVersion = rsp->vgVersion; vgInfo->hashMethod = rsp->hashMethod; vgInfo->vgHash = taosHashInit(rsp->vgNum, taosGetDefaultHashFunction(TSDB_DATA_TYPE_INT), true, HASH_ENTRY_LOCK); @@ -156,18 +156,18 @@ static int32_t hbQueryHbRspHandle(SAppHbMgr *pAppHbMgr, SClientHbRsp *pRsp) { STscObj *pTscObj = (STscObj *)acquireTscObj(pRsp->connKey.tscRid); if (NULL == pTscObj) { tscDebug("tscObj rid %" PRIx64 " not exist", pRsp->connKey.tscRid); - } else { + } else { if (pRsp->query->totalDnodes > 1 && !isEpsetEqual(&pTscObj->pAppInfo->mgmtEp.epSet, &pRsp->query->epSet)) { - SEpSet* pOrig = &pTscObj->pAppInfo->mgmtEp.epSet; - SEp* pOrigEp = &pOrig->eps[pOrig->inUse]; - SEp* pNewEp = &pRsp->query->epSet.eps[pRsp->query->epSet.inUse]; - tscDebug("mnode epset updated from %d/%d=>%s:%d to %d/%d=>%s:%d in hb", - pOrig->inUse, pOrig->numOfEps, pOrigEp->fqdn, pOrigEp->port, - pRsp->query->epSet.inUse, pRsp->query->epSet.numOfEps, pNewEp->fqdn, pNewEp->port); - + SEpSet *pOrig = &pTscObj->pAppInfo->mgmtEp.epSet; + SEp *pOrigEp = &pOrig->eps[pOrig->inUse]; + SEp *pNewEp = &pRsp->query->epSet.eps[pRsp->query->epSet.inUse]; + tscDebug("mnode epset updated from %d/%d=>%s:%d to %d/%d=>%s:%d in hb", pOrig->inUse, pOrig->numOfEps, + pOrigEp->fqdn, pOrigEp->port, pRsp->query->epSet.inUse, pRsp->query->epSet.numOfEps, pNewEp->fqdn, + pNewEp->port); + updateEpSet_s(&pTscObj->pAppInfo->mgmtEp, &pRsp->query->epSet); } - + pTscObj->pAppInfo->totalDnodes = pRsp->query->totalDnodes; pTscObj->pAppInfo->onlineDnodes = pRsp->query->onlineDnodes; pTscObj->connId = pRsp->query->connId; @@ -263,13 +263,20 @@ static int32_t hbQueryHbRspHandle(SAppHbMgr *pAppHbMgr, SClientHbRsp *pRsp) { } static int32_t hbAsyncCallBack(void *param, SDataBuf *pMsg, int32_t code) { - static int32_t emptyRspNum = 0; + static int32_t emptyRspNum = 0; char *key = (char *)param; SClientHbBatchRsp pRsp = {0}; if (TSDB_CODE_SUCCESS == code) { tDeserializeSClientHbBatchRsp(pMsg->pData, pMsg->len, &pRsp); } - + + int32_t now = taosGetTimestampSec(); + int32_t delta = abs(now - pRsp.svrTimestamp); + if (delta > timestampDeltaLimit) { + code = TSDB_CODE_TIME_UNSYNCED; + tscError("time diff: %ds is too big", delta); + } + int32_t rspNum = taosArrayGetSize(pRsp.rsps); taosThreadMutexLock(&appInfo.mutex); @@ -286,7 +293,7 @@ static int32_t hbAsyncCallBack(void *param, SDataBuf *pMsg, int32_t code) { taosMemoryFreeClear(param); if (code != 0) { - (*pInst)->onlineDnodes = 0; + (*pInst)->onlineDnodes = ((*pInst)->totalDnodes ? 0 : -1); } if (rspNum) { @@ -373,7 +380,7 @@ int32_t hbGetQueryBasicInfo(SClientHbKey *connKey, SClientHbReq *req) { releaseTscObj(connKey->tscRid); return TSDB_CODE_QRY_OUT_OF_MEMORY; } - + hbBasic->connId = pTscObj->connId; int32_t numOfQueries = pTscObj->pRequests ? taosHashGetSize(pTscObj->pRequests) : 0; @@ -392,7 +399,6 @@ int32_t hbGetQueryBasicInfo(SClientHbKey *connKey, SClientHbReq *req) { return TSDB_CODE_QRY_OUT_OF_MEMORY; } - int32_t code = hbBuildQueryDesc(hbBasic, pTscObj); if (code) { releaseTscObj(connKey->tscRid); @@ -436,13 +442,12 @@ int32_t hbGetExpiredUserInfo(SClientHbKey *connKey, struct SCatalog *pCatalog, S if (NULL == req->info) { req->info = taosHashInit(64, hbKeyHashFunc, 1, HASH_ENTRY_LOCK); } - + taosHashPut(req->info, &kv.key, sizeof(kv.key), &kv, sizeof(kv)); return TSDB_CODE_SUCCESS; } - int32_t hbGetExpiredDBInfo(SClientHbKey *connKey, struct SCatalog *pCatalog, SClientHbReq *req) { SDbVgVersion *dbs = NULL; uint32_t dbNum = 0; @@ -483,8 +488,8 @@ int32_t hbGetExpiredDBInfo(SClientHbKey *connKey, struct SCatalog *pCatalog, SCl int32_t hbGetExpiredStbInfo(SClientHbKey *connKey, struct SCatalog *pCatalog, SClientHbReq *req) { SSTableVersion *stbs = NULL; - uint32_t stbNum = 0; - int32_t code = 0; + uint32_t stbNum = 0; + int32_t code = 0; code = catalogGetExpiredSTables(pCatalog, &stbs, &stbNum); if (TSDB_CODE_SUCCESS != code) { @@ -521,20 +526,19 @@ int32_t hbGetExpiredStbInfo(SClientHbKey *connKey, struct SCatalog *pCatalog, SC } int32_t hbGetAppInfo(int64_t clusterId, SClientHbReq *req) { - SAppHbReq* pApp = taosHashGet(clientHbMgr.appSummary, &clusterId, sizeof(clusterId)); + SAppHbReq *pApp = taosHashGet(clientHbMgr.appSummary, &clusterId, sizeof(clusterId)); if (NULL != pApp) { memcpy(&req->app, pApp, sizeof(*pApp)); } else { memset(&req->app.summary, 0, sizeof(req->app.summary)); req->app.pid = taosGetPId(); req->app.appId = clientHbMgr.appId; - taosGetAppName(req->app.name, NULL); + taosGetAppName(req->app.name, NULL); } return TSDB_CODE_SUCCESS; } - int32_t hbQueryHbReqHandle(SClientHbKey *connKey, void *param, SClientHbReq *req) { int64_t *clusterId = (int64_t *)param; struct SCatalog *pCatalog = NULL; @@ -602,7 +606,7 @@ SClientHbBatchReq *hbGatherAllInfo(SAppHbMgr *pAppHbMgr) { continue; } - //hbClearClientHbReq(pOneReq); + // hbClearClientHbReq(pOneReq); pIter = taosHashIterate(pAppHbMgr->activeInfo, pIter); } @@ -615,11 +619,9 @@ SClientHbBatchReq *hbGatherAllInfo(SAppHbMgr *pAppHbMgr) { return pBatchReq; } -void hbThreadFuncUnexpectedStopped(void) { - atomic_store_8(&clientHbMgr.threadStop, 2); -} +void hbThreadFuncUnexpectedStopped(void) { atomic_store_8(&clientHbMgr.threadStop, 2); } -void hbMergeSummary(SAppClusterSummary* dst, SAppClusterSummary* src) { +void hbMergeSummary(SAppClusterSummary *dst, SAppClusterSummary *src) { dst->numOfInsertsReq += src->numOfInsertsReq; dst->numOfInsertRows += src->numOfInsertRows; dst->insertElapsedTime += src->insertElapsedTime; @@ -633,7 +635,7 @@ void hbMergeSummary(SAppClusterSummary* dst, SAppClusterSummary* src) { int32_t hbGatherAppInfo(void) { SAppHbReq req = {0}; - int sz = taosArrayGetSize(clientHbMgr.appHbMgrs); + int sz = taosArrayGetSize(clientHbMgr.appHbMgrs); if (sz > 0) { req.pid = taosGetPId(); req.appId = clientHbMgr.appId; @@ -641,11 +643,11 @@ int32_t hbGatherAppInfo(void) { } taosHashClear(clientHbMgr.appSummary); - + for (int32_t i = 0; i < sz; ++i) { SAppHbMgr *pAppHbMgr = taosArrayGetP(clientHbMgr.appHbMgrs, i); - uint64_t clusterId = pAppHbMgr->pAppInstInfo->clusterId; - SAppHbReq* pApp = taosHashGet(clientHbMgr.appSummary, &clusterId, sizeof(clusterId)); + uint64_t clusterId = pAppHbMgr->pAppInstInfo->clusterId; + SAppHbReq *pApp = taosHashGet(clientHbMgr.appSummary, &clusterId, sizeof(clusterId)); if (NULL == pApp) { memcpy(&req.summary, &pAppHbMgr->pAppInstInfo->summary, sizeof(req.summary)); req.startTime = pAppHbMgr->startTime; @@ -654,7 +656,7 @@ int32_t hbGatherAppInfo(void) { if (pAppHbMgr->startTime < pApp->startTime) { pApp->startTime = pAppHbMgr->startTime; } - + hbMergeSummary(&pApp->summary, &pAppHbMgr->pAppInstInfo->summary); } } @@ -662,7 +664,6 @@ int32_t hbGatherAppInfo(void) { return TSDB_CODE_SUCCESS; } - static void *hbThreadFunc(void *param) { setThreadName("hb"); #ifdef WINDOWS @@ -681,7 +682,7 @@ static void *hbThreadFunc(void *param) { if (sz > 0) { hbGatherAppInfo(); } - + for (int i = 0; i < sz; i++) { SAppHbMgr *pAppHbMgr = taosArrayGetP(clientHbMgr.appHbMgrs, i); @@ -698,7 +699,7 @@ static void *hbThreadFunc(void *param) { if (buf == NULL) { terrno = TSDB_CODE_TSC_OUT_OF_MEMORY; tFreeClientHbBatchReq(pReq); - //hbClearReqInfo(pAppHbMgr); + // hbClearReqInfo(pAppHbMgr); break; } @@ -708,7 +709,7 @@ static void *hbThreadFunc(void *param) { if (pInfo == NULL) { terrno = TSDB_CODE_TSC_OUT_OF_MEMORY; tFreeClientHbBatchReq(pReq); - //hbClearReqInfo(pAppHbMgr); + // hbClearReqInfo(pAppHbMgr); taosMemoryFree(buf); break; } @@ -725,7 +726,7 @@ static void *hbThreadFunc(void *param) { SEpSet epSet = getEpSet_s(&pAppInstInfo->mgmtEp); asyncSendMsgToServer(pAppInstInfo->pTransporter, &epSet, &transporterId, pInfo); tFreeClientHbBatchReq(pReq); - //hbClearReqInfo(pAppHbMgr); + // hbClearReqInfo(pAppHbMgr); atomic_add_fetch_32(&pAppHbMgr->reportCnt, 1); } @@ -759,7 +760,7 @@ static void hbStopThread() { return; } - taosThreadJoin(clientHbMgr.thread, NULL); + taosThreadJoin(clientHbMgr.thread, NULL); tscDebug("hb thread stopped"); } @@ -808,7 +809,7 @@ void hbFreeAppHbMgr(SAppHbMgr *pTarget) { } taosHashCleanup(pTarget->activeInfo); pTarget->activeInfo = NULL; - + taosMemoryFree(pTarget->key); taosMemoryFree(pTarget); } @@ -843,7 +844,7 @@ int hbMgrInit() { clientHbMgr.appId = tGenIdPI64(); tscDebug("app %" PRIx64 " initialized", clientHbMgr.appId); - + clientHbMgr.appSummary = taosHashInit(10, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BIGINT), false, HASH_NO_LOCK); clientHbMgr.appHbMgrs = taosArrayInit(0, sizeof(void *)); taosThreadMutexInit(&clientHbMgr.lock, NULL); @@ -881,7 +882,7 @@ int hbRegisterConnImpl(SAppHbMgr *pAppHbMgr, SClientHbKey connKey, int64_t clust SClientHbReq hbReq = {0}; hbReq.connKey = connKey; hbReq.clusterId = clusterId; - //hbReq.info = taosHashInit(64, hbKeyHashFunc, 1, HASH_ENTRY_LOCK); + // hbReq.info = taosHashInit(64, hbKeyHashFunc, 1, HASH_ENTRY_LOCK); taosHashPut(pAppHbMgr->activeInfo, &connKey, sizeof(SClientHbKey), &hbReq, sizeof(SClientHbReq)); @@ -920,4 +921,3 @@ void hbDeregisterConn(SAppHbMgr *pAppHbMgr, SClientHbKey connKey) { atomic_sub_fetch_32(&pAppHbMgr->connKeyCnt, 1); } - diff --git a/source/client/src/clientImpl.c b/source/client/src/clientImpl.c index 04d6df3be01a707c55369c46059c98152a7dc2cc..5e620d106045f341af6ed1dde9653abd1da9ea4e 100644 --- a/source/client/src/clientImpl.c +++ b/source/client/src/clientImpl.c @@ -834,6 +834,7 @@ void schedulerExecCb(SExecResult* pResult, void* param, int32_t code) { tscDebug("0x%" PRIx64 " client retry to handle the error, code:%d - %s, tryCount:%d, reqId:0x%" PRIx64, pRequest->self, code, tstrerror(code), pRequest->retry, pRequest->requestId); pRequest->prevCode = code; + schedulerFreeJob(&pRequest->body.queryJob, 0); doAsyncQuery(pRequest, true); return; } diff --git a/source/client/src/clientMain.c b/source/client/src/clientMain.c index 6ab16b722e79dcdabc26cdf7c0bad25c6d3e6326..62052457fd61d8e0f408ef9c0d5cb0b5c6cc702d 100644 --- a/source/client/src/clientMain.c +++ b/source/client/src/clientMain.c @@ -131,6 +131,7 @@ void taos_close(TAOS *taos) { STscObj *pObj = acquireTscObj(*(int64_t *)taos); if (NULL == pObj) { + taosMemoryFree(taos); return; } diff --git a/source/client/src/clientMsgHandler.c b/source/client/src/clientMsgHandler.c index 94bd5dd7871fa7fa636bfd9c8aa6ff90097ff3e1..520a566e2b4d077a666aa4be5ab94a20ff00f0b6 100644 --- a/source/client/src/clientMsgHandler.c +++ b/source/client/src/clientMsgHandler.c @@ -52,6 +52,18 @@ int32_t processConnectRsp(void* param, SDataBuf* pMsg, int32_t code) { SConnectRsp connectRsp = {0}; tDeserializeSConnectRsp(pMsg->pData, pMsg->len, &connectRsp); + + int32_t now = taosGetTimestampSec(); + int32_t delta = abs(now - connectRsp.svrTimestamp); + if (delta > timestampDeltaLimit) { + code = TSDB_CODE_TIME_UNSYNCED; + tscError("time diff:%ds is too big", delta); + taosMemoryFree(pMsg->pData); + setErrno(pRequest, code); + tsem_post(&pRequest->body.rspSem); + return code; + } + /*assert(connectRsp.epSet.numOfEps > 0);*/ if (connectRsp.epSet.numOfEps == 0) { taosMemoryFree(pMsg->pData); diff --git a/source/common/src/tmsg.c b/source/common/src/tmsg.c index 9ebfa78b80d3e986c3b79a84f15e4b81032a465e..35d3513efad38f2f4cb829e94454816edf09d01a 100644 --- a/source/common/src/tmsg.c +++ b/source/common/src/tmsg.c @@ -453,6 +453,7 @@ int32_t tSerializeSClientHbBatchRsp(void *buf, int32_t bufLen, const SClientHbBa if (tStartEncode(&encoder) < 0) return -1; if (tEncodeI64(&encoder, pBatchRsp->reqId) < 0) return -1; if (tEncodeI64(&encoder, pBatchRsp->rspId) < 0) return -1; + if (tEncodeI32(&encoder, pBatchRsp->svrTimestamp) < 0) return -1; int32_t rspNum = taosArrayGetSize(pBatchRsp->rsps); if (tEncodeI32(&encoder, rspNum) < 0) return -1; @@ -474,6 +475,7 @@ int32_t tDeserializeSClientHbBatchRsp(void *buf, int32_t bufLen, SClientHbBatchR if (tStartDecode(&decoder) < 0) return -1; if (tDecodeI64(&decoder, &pBatchRsp->reqId) < 0) return -1; if (tDecodeI64(&decoder, &pBatchRsp->rspId) < 0) return -1; + if (tDecodeI32(&decoder, &pBatchRsp->svrTimestamp) < 0) return -1; int32_t rspNum = 0; if (tDecodeI32(&decoder, &rspNum) < 0) return -1; @@ -3613,6 +3615,7 @@ int32_t tSerializeSConnectRsp(void *buf, int32_t bufLen, SConnectRsp *pRsp) { if (tEncodeI8(&encoder, pRsp->superUser) < 0) return -1; if (tEncodeI8(&encoder, pRsp->connType) < 0) return -1; if (tEncodeSEpSet(&encoder, &pRsp->epSet) < 0) return -1; + if (tEncodeI32(&encoder, pRsp->svrTimestamp) < 0) return -1; if (tEncodeCStr(&encoder, pRsp->sVer) < 0) return -1; if (tEncodeCStr(&encoder, pRsp->sDetailVer) < 0) return -1; tEndEncode(&encoder); @@ -3634,6 +3637,7 @@ int32_t tDeserializeSConnectRsp(void *buf, int32_t bufLen, SConnectRsp *pRsp) { if (tDecodeI8(&decoder, &pRsp->superUser) < 0) return -1; if (tDecodeI8(&decoder, &pRsp->connType) < 0) return -1; if (tDecodeSEpSet(&decoder, &pRsp->epSet) < 0) return -1; + if (tDecodeI32(&decoder, &pRsp->svrTimestamp) < 0) return -1; if (tDecodeCStrTo(&decoder, pRsp->sVer) < 0) return -1; if (tDecodeCStrTo(&decoder, pRsp->sDetailVer) < 0) return -1; tEndDecode(&decoder); diff --git a/source/dnode/mgmt/mgmt_mnode/src/mmInt.c b/source/dnode/mgmt/mgmt_mnode/src/mmInt.c index 84491a82b061f7a83232d88e6846472d2ac398da..49207225a5d4877eb138a75fb89bb2c5e7f1e474 100644 --- a/source/dnode/mgmt/mgmt_mnode/src/mmInt.c +++ b/source/dnode/mgmt/mgmt_mnode/src/mmInt.c @@ -148,9 +148,9 @@ static int32_t mmStart(SMnodeMgmt *pMgmt) { static void mmStop(SMnodeMgmt *pMgmt) { dDebug("mnode-mgmt start to stop"); + mndPreClose(pMgmt->pMnode); taosThreadRwlockWrlock(&pMgmt->lock); pMgmt->stopped = 1; - mndPreClose(pMgmt->pMnode); taosThreadRwlockUnlock(&pMgmt->lock); mndStop(pMgmt->pMnode); diff --git a/source/dnode/mgmt/node_mgmt/src/dmTransport.c b/source/dnode/mgmt/node_mgmt/src/dmTransport.c index 9052fb20cabfa304429b5198d8ddfb4214a2a85b..7efa46c514373fbc2e315ef6c82a640ad8a309aa 100644 --- a/source/dnode/mgmt/node_mgmt/src/dmTransport.c +++ b/source/dnode/mgmt/node_mgmt/src/dmTransport.c @@ -221,11 +221,11 @@ int32_t dmInitMsgHandle(SDnode *pDnode) { static inline int32_t dmSendReq(const SEpSet *pEpSet, SRpcMsg *pMsg) { SDnode *pDnode = dmInstance(); - if (pDnode->status != DND_STAT_RUNNING) { + if (pDnode->status != DND_STAT_RUNNING && pMsg->msgType < TDMT_SYNC_MSG) { rpcFreeCont(pMsg->pCont); pMsg->pCont = NULL; terrno = TSDB_CODE_NODE_OFFLINE; - dError("failed to send rpc msg since %s, handle:%p", terrstr(), pMsg->info.handle); + dError("failed to send rpc msg:%s since %s, handle:%p", TMSG_INFO(pMsg->msgType), terrstr(), pMsg->info.handle); return -1; } else { rpcSendRequest(pDnode->trans.clientRpc, pEpSet, pMsg, NULL); diff --git a/source/dnode/mnode/impl/src/mndProfile.c b/source/dnode/mnode/impl/src/mndProfile.c index 38368b4ece860ce1f3220d7de7a7a4d2800cb0ce..a1da68606b83a667d61b226cd1acba5f92747b6f 100644 --- a/source/dnode/mnode/impl/src/mndProfile.c +++ b/source/dnode/mnode/impl/src/mndProfile.c @@ -15,10 +15,10 @@ #define _DEFAULT_SOURCE #include "mndProfile.h" -#include "mndPrivilege.h" #include "mndDb.h" #include "mndDnode.h" #include "mndMnode.h" +#include "mndPrivilege.h" #include "mndQnode.h" #include "mndShow.h" #include "mndStb.h" @@ -274,6 +274,7 @@ static int32_t mndProcessConnectReq(SRpcMsg *pReq) { connectRsp.connId = pConn->id; connectRsp.connType = connReq.connType; connectRsp.dnodeNum = mndGetDnodeSize(pMnode); + connectRsp.svrTimestamp = taosGetTimestampSec(); strcpy(connectRsp.sVer, version); snprintf(connectRsp.sDetailVer, sizeof(connectRsp.sDetailVer), "ver:%s\nbuild:%s\ngitinfo:%s", version, buildinfo, @@ -623,6 +624,7 @@ static int32_t mndProcessHeartBeatReq(SRpcMsg *pReq) { } SClientHbBatchRsp batchRsp = {0}; + batchRsp.svrTimestamp = taosGetTimestampSec(); batchRsp.rsps = taosArrayInit(0, sizeof(SClientHbRsp)); int32_t sz = taosArrayGetSize(batchReq.reqs); diff --git a/source/dnode/vnode/src/tsdb/tsdbCacheRead.c b/source/dnode/vnode/src/tsdb/tsdbCacheRead.c index 5d3cfee592a027cd85342674b24b475eb982ee24..66843d9a2844c44e77e798ab47032ef75370a544 100644 --- a/source/dnode/vnode/src/tsdb/tsdbCacheRead.c +++ b/source/dnode/vnode/src/tsdb/tsdbCacheRead.c @@ -177,7 +177,6 @@ int32_t tsdbRetrieveLastRow(void* pReader, SSDataBlock* pResBlock, const int32_t saveOneRow(pRow, pResBlock, pr, slotIds); taosArrayPush(pTableUidList, &pKeyInfo->uid); - // taosMemoryFree(pRow); tsdbCacheRelease(lruCache, h); pr->tableIndex += 1; diff --git a/source/dnode/vnode/src/tsdb/tsdbRead.c b/source/dnode/vnode/src/tsdb/tsdbRead.c index 1a9e12c9cad8051fdd4fca2bce641b5dd9a7faa6..f0aea0cefb4ad14c06441f5aed1fdc247dd206b1 100644 --- a/source/dnode/vnode/src/tsdb/tsdbRead.c +++ b/source/dnode/vnode/src/tsdb/tsdbRead.c @@ -830,9 +830,8 @@ static int32_t doLoadFileBlockData(STsdbReader* pReader, SDataBlockIter* pBlockI SBlockLoadSuppInfo* pSupInfo = &pReader->suppInfo; SFileBlockDumpInfo* pDumpInfo = &pReader->status.fBlockDumpInfo; - uint8_t *pb = NULL, *pb1 = NULL; int32_t code = tsdbReadColData(pReader->pFileReader, &pBlockScanInfo->blockIdx, pBlock, pSupInfo->colIds, numOfCols, - pBlockData, &pb, &pb1); + pBlockData, NULL, NULL); if (code != TSDB_CODE_SUCCESS) { goto _error; } @@ -3007,11 +3006,14 @@ SArray* tsdbRetrieveDataBlock(STsdbReader* pReader, SArray* pIdList) { code = doLoadFileBlockData(pReader, &pStatus->blockIter, pBlockScanInfo, &pStatus->fileBlockData); if (code != TSDB_CODE_SUCCESS) { + tBlockDataClear(&pStatus->fileBlockData); + terrno = code; return NULL; } copyBlockDataToSDataBlock(pReader, pBlockScanInfo); + tBlockDataClear(&pStatus->fileBlockData); return pReader->pResBlock->pDataBlock; } diff --git a/source/libs/command/CMakeLists.txt b/source/libs/command/CMakeLists.txt index 51118f4a34be578a62ae9ec4de0f02cb7fbabb6b..a890972d149531d01620cebc9f0ca2db0166fd38 100644 --- a/source/libs/command/CMakeLists.txt +++ b/source/libs/command/CMakeLists.txt @@ -8,9 +8,9 @@ target_include_directories( target_link_libraries( command - PRIVATE os util nodes catalog function transport qcom + PRIVATE os util nodes catalog function transport qcom scheduler ) if(${BUILD_TEST}) ADD_SUBDIRECTORY(test) -endif(${BUILD_TEST}) \ No newline at end of file +endif(${BUILD_TEST}) diff --git a/source/libs/command/inc/commandInt.h b/source/libs/command/inc/commandInt.h index 6aca581f45d186e76ab2c4b38a86f2851dd6810e..7012c889e9d216433342d785d207a5ecfc7d129b 100644 --- a/source/libs/command/inc/commandInt.h +++ b/source/libs/command/inc/commandInt.h @@ -77,6 +77,10 @@ extern "C" { #define EXPLAIN_MODE_FORMAT "mode=%s" #define EXPLAIN_STRING_TYPE_FORMAT "%s" +#define COMMAND_RESET_LOG "resetLog" +#define COMMAND_SCHEDULE_POLICY "schedulePolicy" +#define COMMAND_ENABLE_RESCHEDULE "enableReSchedule" + typedef struct SExplainGroup { int32_t nodeNum; int32_t physiPlanExecNum; diff --git a/source/libs/command/src/command.c b/source/libs/command/src/command.c index 034778e5bfd4f527ca4782d7e91015244db8b012..d22b3d88b4f33c1401e29a05726f92086c21f633 100644 --- a/source/libs/command/src/command.c +++ b/source/libs/command/src/command.c @@ -17,6 +17,8 @@ #include "catalog.h" #include "tdatablock.h" #include "tglobal.h" +#include "commandInt.h" +#include "scheduler.h" extern SConfig* tsCfg; @@ -479,7 +481,42 @@ static int32_t execShowCreateSTable(SShowCreateTableStmt* pStmt, SRetrieveTableR return execShowCreateTable(pStmt, pRsp); } +static int32_t execAlterCmd(char* cmd, char* value, bool* processed) { + int32_t code = 0; + + if (0 == strcasecmp(cmd, COMMAND_RESET_LOG)) { + taosResetLog(); + cfgDumpCfg(tsCfg, 0, false); + } else if (0 == strcasecmp(cmd, COMMAND_SCHEDULE_POLICY)) { + code = schedulerUpdatePolicy(atoi(value)); + } else if (0 == strcasecmp(cmd, COMMAND_ENABLE_RESCHEDULE)) { + code = schedulerEnableReSchedule(atoi(value)); + } else { + goto _return; + } + + *processed = true; + +_return: + + if (code) { + terrno = code; + } + + return code; +} + static int32_t execAlterLocal(SAlterLocalStmt* pStmt) { + bool processed = false; + + if (execAlterCmd(pStmt->config, pStmt->value, &processed)) { + return terrno; + } + + if (processed) { + goto _return; + } + if (cfgSetItem(tsCfg, pStmt->config, pStmt->value, CFG_STYPE_ALTER_CMD)) { return terrno; } @@ -488,6 +525,8 @@ static int32_t execAlterLocal(SAlterLocalStmt* pStmt) { return terrno; } +_return: + return TSDB_CODE_SUCCESS; } diff --git a/source/libs/executor/inc/executorimpl.h b/source/libs/executor/inc/executorimpl.h index 897373e6c883b98ea5d2f4dbd02940b5170dae42..0d57a6370c286791f5a8dd84e592da6a24e9323b 100644 --- a/source/libs/executor/inc/executorimpl.h +++ b/source/libs/executor/inc/executorimpl.h @@ -316,12 +316,16 @@ typedef struct STagScanInfo { typedef struct SLastrowScanInfo { SSDataBlock *pRes; - SArray *pTableList; SReadHandle readHandle; void *pLastrowReader; SArray *pColMatchInfo; int32_t *pSlotIds; SExprSupp pseudoExprSup; + int32_t retrieveType; + int32_t currentGroupIndex; + SSDataBlock *pBufferredRes; + SArray *pUidList; + int32_t indexOfBufferedRes; } SLastrowScanInfo; typedef enum EStreamScanMode { @@ -825,8 +829,7 @@ SOperatorInfo* createProjectOperatorInfo(SOperatorInfo* downstream, SProjectPhys SOperatorInfo* createSortOperatorInfo(SOperatorInfo* downstream, SSortPhysiNode* pSortPhyNode, SExecTaskInfo* pTaskInfo); SOperatorInfo* createMultiwayMergeOperatorInfo(SOperatorInfo** dowStreams, size_t numStreams, SMergePhysiNode* pMergePhysiNode, SExecTaskInfo* pTaskInfo); SOperatorInfo* createSortedMergeOperatorInfo(SOperatorInfo** downstream, int32_t numOfDownstream, SExprInfo* pExprInfo, int32_t num, SArray* pSortInfo, SArray* pGroupInfo, SExecTaskInfo* pTaskInfo); -SOperatorInfo* createLastrowScanOperator(SLastRowScanPhysiNode* pTableScanNode, SReadHandle* readHandle, - SArray* pTableList, SExecTaskInfo* pTaskInfo); +SOperatorInfo* createLastrowScanOperator(SLastRowScanPhysiNode* pTableScanNode, SReadHandle* readHandle, SExecTaskInfo* pTaskInfo); SOperatorInfo* createIntervalOperatorInfo(SOperatorInfo* downstream, SExprInfo* pExprInfo, int32_t numOfCols, SSDataBlock* pResBlock, SInterval* pInterval, int32_t primaryTsSlotId, @@ -944,8 +947,9 @@ int32_t finalizeResultRowIntoResultDataBlock(SDiskbasedBuf* pBuf, SResultRowPosi SqlFunctionCtx* pCtx, SExprInfo* pExprInfo, int32_t numOfExprs, const int32_t* rowCellOffset, SSDataBlock* pBlock, SExecTaskInfo* pTaskInfo); -int32_t createScanTableListInfo(STableScanPhysiNode* pTableScanNode, SReadHandle* pHandle, +int32_t createScanTableListInfo(SScanPhysiNode *pScanNode, SNodeList* pGroupTags, bool groupSort, SReadHandle* pHandle, STableListInfo* pTableListInfo, uint64_t queryId, uint64_t taskId); + SOperatorInfo* createGroupSortOperatorInfo(SOperatorInfo* downstream, SGroupSortPhysiNode* pSortPhyNode, SExecTaskInfo* pTaskInfo); SOperatorInfo* createTableMergeScanOperatorInfo(STableScanPhysiNode* pTableScanNode, STableListInfo *pTableListInfo, diff --git a/source/libs/executor/src/cachescanoperator.c b/source/libs/executor/src/cachescanoperator.c index 9034397d0f5f6e1e0cb632ee67cc4d4698b39e2f..78a7e58ee111cb2c22ea8b2736730936c5dca8ce 100644 --- a/source/libs/executor/src/cachescanoperator.c +++ b/source/libs/executor/src/cachescanoperator.c @@ -30,15 +30,13 @@ static SSDataBlock* doScanLastrow(SOperatorInfo* pOperator); static void destroyLastrowScanOperator(void* param, int32_t numOfOutput); static int32_t extractTargetSlotId(const SArray* pColMatchInfo, SExecTaskInfo* pTaskInfo, int32_t** pSlotIds); -SOperatorInfo* createLastrowScanOperator(SLastRowScanPhysiNode* pScanNode, SReadHandle* readHandle, SArray* pTableList, - SExecTaskInfo* pTaskInfo) { +SOperatorInfo* createLastrowScanOperator(SLastRowScanPhysiNode* pScanNode, SReadHandle* readHandle, SExecTaskInfo* pTaskInfo) { SLastrowScanInfo* pInfo = taosMemoryCalloc(1, sizeof(SLastrowScanInfo)); SOperatorInfo* pOperator = taosMemoryCalloc(1, sizeof(SOperatorInfo)); if (pInfo == NULL || pOperator == NULL) { goto _error; } - pInfo->pTableList = pTableList; pInfo->readHandle = *readHandle; pInfo->pRes = createResDataBlock(pScanNode->scan.node.pOutputDataBlockDesc); @@ -50,8 +48,22 @@ SOperatorInfo* createLastrowScanOperator(SLastRowScanPhysiNode* pScanNode, SRead goto _error; } - tsdbLastRowReaderOpen(readHandle->vnode, LASTROW_RETRIEVE_TYPE_SINGLE, pTableList, taosArrayGetSize(pInfo->pColMatchInfo), - &pInfo->pLastrowReader); + STableListInfo* pTableList = &pTaskInfo->tableqinfoList; + + initResultSizeInfo(pOperator, 1024); + blockDataEnsureCapacity(pInfo->pRes, pOperator->resultInfo.capacity); + pInfo->pUidList = taosArrayInit(4, sizeof(int64_t)); + + // partition by tbname + if (taosArrayGetSize(pTableList->pGroupList) == taosArrayGetSize(pTableList->pTableList)) { + pInfo->retrieveType = LASTROW_RETRIEVE_TYPE_ALL; + tsdbLastRowReaderOpen(pInfo->readHandle.vnode, pInfo->retrieveType, pTableList->pTableList, + taosArrayGetSize(pInfo->pColMatchInfo), &pInfo->pLastrowReader); + pInfo->pBufferredRes = createOneDataBlock(pInfo->pRes, false); + blockDataEnsureCapacity(pInfo->pBufferredRes, pOperator->resultInfo.capacity); + } else { // by tags + pInfo->retrieveType = LASTROW_RETRIEVE_TYPE_SINGLE; + } if (pScanNode->scan.pScanPseudoCols != NULL) { SExprSupp* pPseudoExpr = &pInfo->pseudoExprSup; @@ -60,19 +72,17 @@ SOperatorInfo* createLastrowScanOperator(SLastRowScanPhysiNode* pScanNode, SRead pPseudoExpr->pCtx = createSqlFunctionCtx(pPseudoExpr->pExprInfo, pPseudoExpr->numOfExprs, &pPseudoExpr->rowEntryInfoOffset); } - pOperator->name = "LastrowScanOperator"; + pOperator->name = "LastrowScanOperator"; pOperator->operatorType = QUERY_NODE_PHYSICAL_PLAN_LAST_ROW_SCAN; - pOperator->blocking = false; - pOperator->status = OP_NOT_OPENED; - pOperator->info = pInfo; - pOperator->pTaskInfo = pTaskInfo; + pOperator->blocking = false; + pOperator->status = OP_NOT_OPENED; + pOperator->info = pInfo; + pOperator->pTaskInfo = pTaskInfo; pOperator->exprSupp.numOfExprs = taosArrayGetSize(pInfo->pRes->pDataBlock); - initResultSizeInfo(pOperator, 1024); - blockDataEnsureCapacity(pInfo->pRes, pOperator->resultInfo.capacity); - pOperator->fpSet = createOperatorFpSet(operatorDummyOpenFn, doScanLastrow, NULL, NULL, destroyLastrowScanOperator, NULL, NULL, NULL); + pOperator->cost.openCost = 0; return pOperator; @@ -90,43 +100,105 @@ SSDataBlock* doScanLastrow(SOperatorInfo* pOperator) { SLastrowScanInfo* pInfo = pOperator->info; SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; - - int32_t size = taosArrayGetSize(pInfo->pTableList); + STableListInfo* pTableList = &pTaskInfo->tableqinfoList; + int32_t size = taosArrayGetSize(pTableList->pTableList); if (size == 0) { - setTaskStatus(pTaskInfo, TASK_COMPLETED); + doSetOperatorCompleted(pOperator); return NULL; } + blockDataCleanup(pInfo->pRes); + // check if it is a group by tbname - if (size == taosArrayGetSize(pInfo->pTableList)) { - blockDataCleanup(pInfo->pRes); - SArray* pUidList = taosArrayInit(1, sizeof(tb_uid_t)); - int32_t code = tsdbRetrieveLastRow(pInfo->pLastrowReader, pInfo->pRes, pInfo->pSlotIds, pUidList); - if (code != TSDB_CODE_SUCCESS) { - longjmp(pTaskInfo->env, code); + if (pInfo->retrieveType == LASTROW_RETRIEVE_TYPE_ALL) { + if (pInfo->indexOfBufferedRes >= pInfo->pBufferredRes->info.rows) { + blockDataCleanup(pInfo->pBufferredRes); + taosArrayClear(pInfo->pUidList); + + int32_t code = tsdbRetrieveLastRow(pInfo->pLastrowReader, pInfo->pBufferredRes, pInfo->pSlotIds, pInfo->pUidList); + if (code != TSDB_CODE_SUCCESS) { + longjmp(pTaskInfo->env, code); + } + + // check for tag values + int32_t resultRows = pInfo->pBufferredRes->info.rows; + ASSERT(resultRows == taosArrayGetSize(pInfo->pUidList)); + pInfo->indexOfBufferedRes = 0; + } + + if (pInfo->indexOfBufferedRes < pInfo->pBufferredRes->info.rows) { + for(int32_t i = 0; i < taosArrayGetSize(pInfo->pColMatchInfo); ++i) { + SColMatchInfo* pMatchInfo = taosArrayGet(pInfo->pColMatchInfo, i); + int32_t slotId = pMatchInfo->targetSlotId; + + SColumnInfoData* pSrc = taosArrayGet(pInfo->pBufferredRes->pDataBlock, slotId); + SColumnInfoData* pDst = taosArrayGet(pInfo->pRes->pDataBlock, slotId); + + char* p = colDataGetData(pSrc, pInfo->indexOfBufferedRes); + bool isNull = colDataIsNull_s(pSrc, pInfo->indexOfBufferedRes); + colDataAppend(pDst, 0, p, isNull); + } + + if (pInfo->pseudoExprSup.numOfExprs > 0) { + SExprSupp* pSup = &pInfo->pseudoExprSup; + addTagPseudoColumnData(&pInfo->readHandle, pSup->pExprInfo, pSup->numOfExprs, pInfo->pRes, + GET_TASKID(pTaskInfo)); + } + + pInfo->pRes->info.uid = *(tb_uid_t*)taosArrayGet(pInfo->pUidList, pInfo->indexOfBufferedRes); + int64_t* groupId = taosHashGet(pTableList->map, &pInfo->pRes->info.uid, sizeof(int64_t)); + pInfo->pRes->info.groupId = *groupId; + + pInfo->indexOfBufferedRes += 1; + pInfo->pRes->info.rows = 1; + return pInfo->pRes; + } else { + doSetOperatorCompleted(pOperator); + return NULL; } + } else { + size_t totalGroups = taosArrayGetSize(pTableList->pGroupList); + + while (pInfo->currentGroupIndex < totalGroups) { + SArray* pGroupTableList = taosArrayGetP(pTableList->pGroupList, pInfo->currentGroupIndex); + + tsdbLastRowReaderOpen(pInfo->readHandle.vnode, pInfo->retrieveType, pGroupTableList, + taosArrayGetSize(pInfo->pColMatchInfo), &pInfo->pLastrowReader); + taosArrayClear(pInfo->pUidList); + + int32_t code = tsdbRetrieveLastRow(pInfo->pLastrowReader, pInfo->pRes, pInfo->pSlotIds, pInfo->pUidList); + if (code != TSDB_CODE_SUCCESS) { + longjmp(pTaskInfo->env, code); + } - // check for tag values - if (pInfo->pRes->info.rows > 0 && pInfo->pseudoExprSup.numOfExprs > 0) { - SExprSupp* pSup = &pInfo->pseudoExprSup; - pInfo->pRes->info.uid = *(tb_uid_t*) taosArrayGet(pUidList, 0); - addTagPseudoColumnData(&pInfo->readHandle, pSup->pExprInfo, pSup->numOfExprs, pInfo->pRes, GET_TASKID(pTaskInfo)); + pInfo->currentGroupIndex += 1; + + // check for tag values + if (pInfo->pRes->info.rows > 0) { + if (pInfo->pseudoExprSup.numOfExprs > 0) { + SExprSupp* pSup = &pInfo->pseudoExprSup; + pInfo->pRes->info.uid = *(tb_uid_t*)taosArrayGet(pInfo->pUidList, 0); + + STableKeyInfo* pKeyInfo = taosArrayGet(pGroupTableList, 0); + pInfo->pRes->info.groupId = pKeyInfo->groupId; + + addTagPseudoColumnData(&pInfo->readHandle, pSup->pExprInfo, pSup->numOfExprs, pInfo->pRes, + GET_TASKID(pTaskInfo)); + } + + tsdbLastrowReaderClose(pInfo->pLastrowReader); + return pInfo->pRes; + } } doSetOperatorCompleted(pOperator); - return (pInfo->pRes->info.rows == 0) ? NULL : pInfo->pRes; - } else { - // todo fetch the result for each group + return NULL; } - - return pInfo->pRes->info.rows == 0 ? NULL : pInfo->pRes; } void destroyLastrowScanOperator(void* param, int32_t numOfOutput) { SLastrowScanInfo* pInfo = (SLastrowScanInfo*)param; blockDataDestroy(pInfo->pRes); - tsdbLastrowReaderClose(pInfo->pLastrowReader); - taosMemoryFreeClear(param); } diff --git a/source/libs/executor/src/executorimpl.c b/source/libs/executor/src/executorimpl.c index 89542571ea2cb02ab5ebb1802e87ecd7bfd51825..c8f20834565633c1f1b1cb0126e4a1e6f6947977 100644 --- a/source/libs/executor/src/executorimpl.c +++ b/source/libs/executor/src/executorimpl.c @@ -514,8 +514,10 @@ static int32_t doSetInputDataBlock(SOperatorInfo* pOperator, SqlFunctionCtx* pCt pInput->startRowIndex = 0; // NOTE: the last parameter is the primary timestamp column + // todo: refactor this if (fmIsTimelineFunc(pCtx[i].functionId) && (j == pOneExpr->base.numOfParams - 1)) { - pInput->pPTS = pInput->pData[j]; + pInput->pPTS = pInput->pData[j]; // in case of merge function, this is not always the ts column data. +// ASSERT(pInput->pPTS->info.type == TSDB_DATA_TYPE_TIMESTAMP); } ASSERT(pInput->pData[j] != NULL); } else if (pFuncParam->type == FUNC_PARAM_TYPE_VALUE) { @@ -4291,6 +4293,7 @@ int32_t generateGroupIdMap(STableListInfo* pTableListInfo, SReadHandle* pHandle, } } } + int32_t len = (int32_t)(pStart - (char*)keyBuf); uint64_t groupId = calcGroupId(keyBuf, len); taosHashPut(pTableListInfo->map, &(info->uid), sizeof(uint64_t), &groupId, sizeof(uint64_t)); @@ -4309,6 +4312,30 @@ int32_t generateGroupIdMap(STableListInfo* pTableListInfo, SReadHandle* pHandle, return TDB_CODE_SUCCESS; } +static int32_t initTableblockDistQueryCond(uint64_t uid, SQueryTableDataCond* pCond) { + memset(pCond, 0, sizeof(SQueryTableDataCond)); + + pCond->order = TSDB_ORDER_ASC; + pCond->numOfCols = 1; + pCond->colList = taosMemoryCalloc(1, sizeof(SColumnInfo)); + if (pCond->colList == NULL) { + terrno = TSDB_CODE_QRY_OUT_OF_MEMORY; + return terrno; + } + + pCond->colList->colId = 1; + pCond->colList->type = TSDB_DATA_TYPE_TIMESTAMP; + pCond->colList->bytes = sizeof(TSKEY); + + pCond->twindows = (STimeWindow){.skey = INT64_MIN, .ekey = INT64_MAX}; + pCond->suid = uid; + pCond->type = BLOCK_LOAD_OFFSET_ORDER; + pCond->startVersion = -1; + pCond->endVersion = -1; + + return TSDB_CODE_SUCCESS; +} + SOperatorInfo* createOperatorTree(SPhysiNode* pPhyNode, SExecTaskInfo* pTaskInfo, SReadHandle* pHandle, uint64_t queryId, uint64_t taskId, STableListInfo* pTableListInfo, const char* pUser) { @@ -4318,7 +4345,8 @@ SOperatorInfo* createOperatorTree(SPhysiNode* pPhyNode, SExecTaskInfo* pTaskInfo if (QUERY_NODE_PHYSICAL_PLAN_TABLE_SCAN == type) { STableScanPhysiNode* pTableScanNode = (STableScanPhysiNode*)pPhyNode; - int32_t code = createScanTableListInfo(pTableScanNode, pHandle, pTableListInfo, queryId, taskId); + int32_t code = createScanTableListInfo(&pTableScanNode->scan, pTableScanNode->pGroupTags, + pTableScanNode->groupSort, pHandle, pTableListInfo, queryId, taskId); if (code) { pTaskInfo->code = code; return NULL; @@ -4337,7 +4365,8 @@ SOperatorInfo* createOperatorTree(SPhysiNode* pPhyNode, SExecTaskInfo* pTaskInfo } else if (QUERY_NODE_PHYSICAL_PLAN_TABLE_MERGE_SCAN == type) { STableMergeScanPhysiNode* pTableScanNode = (STableMergeScanPhysiNode*)pPhyNode; - int32_t code = createScanTableListInfo(pTableScanNode, pHandle, pTableListInfo, queryId, taskId); + int32_t code = createScanTableListInfo(&pTableScanNode->scan, pTableScanNode->pGroupTags, + pTableScanNode->groupSort, pHandle, pTableListInfo, queryId, taskId); if (code) { pTaskInfo->code = code; return NULL; @@ -4366,7 +4395,8 @@ SOperatorInfo* createOperatorTree(SPhysiNode* pPhyNode, SExecTaskInfo* pTaskInfo .maxTs = INT64_MIN, }; if (pHandle) { - int32_t code = createScanTableListInfo(pTableScanNode, pHandle, pTableListInfo, queryId, taskId); + int32_t code = createScanTableListInfo(&pTableScanNode->scan, pTableScanNode->pGroupTags, + pTableScanNode->groupSort, pHandle, pTableListInfo, queryId, taskId); if (code) { pTaskInfo->code = code; return NULL; @@ -4406,25 +4436,9 @@ SOperatorInfo* createOperatorTree(SPhysiNode* pPhyNode, SExecTaskInfo* pTaskInfo } SQueryTableDataCond cond = {0}; - - { - cond.order = TSDB_ORDER_ASC; - cond.numOfCols = 1; - cond.colList = taosMemoryCalloc(1, sizeof(SColumnInfo)); - if (cond.colList == NULL) { - terrno = TSDB_CODE_QRY_OUT_OF_MEMORY; - return NULL; - } - - cond.colList->colId = 1; - cond.colList->type = TSDB_DATA_TYPE_TIMESTAMP; - cond.colList->bytes = sizeof(TSKEY); - - cond.twindows = (STimeWindow){.skey = INT64_MIN, .ekey = INT64_MAX}; - cond.suid = pBlockNode->suid; - cond.type = BLOCK_LOAD_OFFSET_ORDER; - cond.startVersion = -1; - cond.endVersion = -1; + int32_t code = initTableblockDistQueryCond(pBlockNode->suid, &cond); + if (code != TSDB_CODE_SUCCESS) { + return NULL; } STsdbReader* pReader = NULL; @@ -4435,31 +4449,20 @@ SOperatorInfo* createOperatorTree(SPhysiNode* pPhyNode, SExecTaskInfo* pTaskInfo } else if (QUERY_NODE_PHYSICAL_PLAN_LAST_ROW_SCAN == type) { SLastRowScanPhysiNode* pScanNode = (SLastRowScanPhysiNode*)pPhyNode; - // int32_t code = createScanTableListInfo(pTableScanNode, pHandle, pTableListInfo, queryId, taskId); - // if (code) { - // pTaskInfo->code = code; - // return NULL; - // } - - int32_t code = extractTableSchemaInfo(pHandle, pScanNode->scan.uid, pTaskInfo); + int32_t code = createScanTableListInfo(&pScanNode->scan, pScanNode->pGroupTags, true, pHandle, pTableListInfo, + queryId, taskId); if (code != TSDB_CODE_SUCCESS) { pTaskInfo->code = code; return NULL; } - pTableListInfo->pTableList = taosArrayInit(4, sizeof(STableKeyInfo)); - if (pScanNode->scan.tableType == TSDB_SUPER_TABLE) { - code = vnodeGetAllTableList(pHandle->vnode, pScanNode->scan.uid, pTableListInfo->pTableList); - if (code != TSDB_CODE_SUCCESS) { - pTaskInfo->code = terrno; - return NULL; - } - } else { // Create one table group. - STableKeyInfo info = {.lastKey = 0, .uid = pScanNode->scan.uid, .groupId = 0}; - taosArrayPush(pTableListInfo->pTableList, &info); + code = extractTableSchemaInfo(pHandle, pScanNode->scan.uid, pTaskInfo); + if (code != TSDB_CODE_SUCCESS) { + pTaskInfo->code = code; + return NULL; } - return createLastrowScanOperator(pScanNode, pHandle, pTableListInfo->pTableList, pTaskInfo); + return createLastrowScanOperator(pScanNode, pHandle, pTaskInfo); } else { ASSERT(0); } @@ -4928,6 +4931,9 @@ static void doDestroyTableList(STableListInfo* pTableqinfoList) { if (pTableqinfoList->needSortTableByGroupId) { for (int32_t i = 0; i < taosArrayGetSize(pTableqinfoList->pGroupList); i++) { SArray* tmp = taosArrayGetP(pTableqinfoList->pGroupList, i); + if (tmp == pTableqinfoList->pTableList) { + continue; + } taosArrayDestroy(tmp); } } diff --git a/source/libs/executor/src/scanoperator.c b/source/libs/executor/src/scanoperator.c index 0f44ac48a41961f989016cb1e9ebf8666837b284..6aacb0ee01df378c42a289a2965d2ca7b8e044ae 100644 --- a/source/libs/executor/src/scanoperator.c +++ b/source/libs/executor/src/scanoperator.c @@ -2402,9 +2402,9 @@ typedef struct STableMergeScanInfo { SSampleExecInfo sample; // sample execution info } STableMergeScanInfo; -int32_t createScanTableListInfo(STableScanPhysiNode* pTableScanNode, SReadHandle* pHandle, +int32_t createScanTableListInfo(SScanPhysiNode* pScanNode, SNodeList* pGroupTags, bool groupSort, SReadHandle* pHandle, STableListInfo* pTableListInfo, uint64_t queryId, uint64_t taskId) { - int32_t code = getTableList(pHandle->meta, pHandle->vnode, &pTableScanNode->scan, pTableListInfo); + int32_t code = getTableList(pHandle->meta, pHandle->vnode, pScanNode, pTableListInfo); if (code != TSDB_CODE_SUCCESS) { return code; } @@ -2414,8 +2414,8 @@ int32_t createScanTableListInfo(STableScanPhysiNode* pTableScanNode, SReadHandle return TSDB_CODE_SUCCESS; } - pTableListInfo->needSortTableByGroupId = pTableScanNode->groupSort; - code = generateGroupIdMap(pTableListInfo, pHandle, pTableScanNode->pGroupTags); + pTableListInfo->needSortTableByGroupId = groupSort; + code = generateGroupIdMap(pTableListInfo, pHandle, pGroupTags); if (code != TSDB_CODE_SUCCESS) { return code; } diff --git a/source/libs/function/src/builtins.c b/source/libs/function/src/builtins.c index 747b1c8d9f7064875850e0c6b9a1fa6a2e17371a..9ca4ee8d8f89cd574dba50c742428af643bd28bd 100644 --- a/source/libs/function/src/builtins.c +++ b/source/libs/function/src/builtins.c @@ -1981,6 +1981,7 @@ const SBuiltinFuncDefinition funcMgtBuiltins[] = { .getEnvFunc = getLeastSQRFuncEnv, .initFunc = leastSQRFunctionSetup, .processFunc = leastSQRFunction, + .sprocessFunc = leastSQRScalarFunction, .finalizeFunc = leastSQRFinalize, .invertFunc = NULL, .combineFunc = leastSQRCombine, @@ -2228,7 +2229,7 @@ const SBuiltinFuncDefinition funcMgtBuiltins[] = { { .name = "_cache_last_row", .type = FUNCTION_TYPE_CACHE_LAST_ROW, - .classification = FUNC_MGT_AGG_FUNC | FUNC_MGT_MULTI_RES_FUNC | FUNC_MGT_TIMELINE_FUNC | FUNC_MGT_IMPLICIT_TS_FUNC, + .classification = FUNC_MGT_AGG_FUNC | FUNC_MGT_MULTI_RES_FUNC | FUNC_MGT_SELECT_FUNC | FUNC_MGT_TIMELINE_FUNC | FUNC_MGT_IMPLICIT_TS_FUNC, .translateFunc = translateFirstLast, .getEnvFunc = getFirstLastFuncEnv, .initFunc = functionSetup, diff --git a/source/libs/function/src/udfd.c b/source/libs/function/src/udfd.c index fd9b588d46d182742cfd7b8568257d375ee42942..74fca69aa7d5872edb1f038709ad2e0aeeb6c43f 100644 --- a/source/libs/function/src/udfd.c +++ b/source/libs/function/src/udfd.c @@ -382,6 +382,15 @@ void udfdProcessRpcRsp(void *parent, SRpcMsg *pMsg, SEpSet *pEpSet) { if (msgInfo->rpcType == UDFD_RPC_MNODE_CONNECT) { SConnectRsp connectRsp = {0}; tDeserializeSConnectRsp(pMsg->pCont, pMsg->contLen, &connectRsp); + + int32_t now = taosGetTimestampSec(); + int32_t delta = abs(now - connectRsp.svrTimestamp); + if (delta > 900) { + msgInfo->code = TSDB_CODE_TIME_UNSYNCED; + goto _return; + } + + if (connectRsp.epSet.numOfEps == 0) { msgInfo->code = TSDB_CODE_MND_APP_ERROR; goto _return; diff --git a/source/libs/index/src/indexCache.c b/source/libs/index/src/indexCache.c index 1e9ec4a8411aed46f5502d32a2d33415480b81d8..50745cd3fb46a386022ce329e9f94d84b468ad89 100644 --- a/source/libs/index/src/indexCache.c +++ b/source/libs/index/src/indexCache.c @@ -516,13 +516,14 @@ static void idxCacheMakeRoomForWrite(IndexCache* cache) { idxCacheRef(cache); cache->imm = cache->mem; cache->mem = idxInternalCacheCreate(cache->type); + cache->mem->pCache = cache; cache->occupiedMem = 0; if (quit == false) { atomic_store_32(&cache->merging, 1); } - // sched to merge - // unref cache in bgwork + // 1. sched to merge + // 2. unref cache in bgwork idxCacheSchedToMerge(cache, quit); } } diff --git a/source/libs/nodes/src/nodesUtilFuncs.c b/source/libs/nodes/src/nodesUtilFuncs.c index b00b08a66d4835f0a01fdde843582910b0ac8f3d..7265e7ee7802081b37c04dbc774532ea56e73883 100644 --- a/source/libs/nodes/src/nodesUtilFuncs.c +++ b/source/libs/nodes/src/nodesUtilFuncs.c @@ -388,6 +388,11 @@ static void destroyDataSinkNode(SDataSinkNode* pNode) { nodesDestroyNode((SNode* static void destroyExprNode(SExprNode* pExpr) { taosArrayDestroy(pExpr->pAssociation); } +static void nodesDestroyNodePointer(void* node) { + SNode* pNode = *(SNode**)node; + nodesDestroyNode(pNode); +} + void nodesDestroyNode(SNode* pNode) { if (NULL == pNode) { return; @@ -718,6 +723,7 @@ void nodesDestroyNode(SNode* pNode) { } taosArrayDestroy(pQuery->pDbList); taosArrayDestroy(pQuery->pTableList); + taosArrayDestroyEx(pQuery->pPlaceholderValues, nodesDestroyNodePointer); break; } case QUERY_NODE_LOGIC_PLAN_SCAN: { diff --git a/source/libs/qworker/inc/qwInt.h b/source/libs/qworker/inc/qwInt.h index 539643c3907cc20de3bee586d21135fbe8adb748..d8d7c5a0ea82543118e9570c3ce9358d01ff49bf 100644 --- a/source/libs/qworker/inc/qwInt.h +++ b/source/libs/qworker/inc/qwInt.h @@ -378,6 +378,7 @@ void qwDbgDumpMgmtInfo(SQWorker *mgmt); int32_t qwDbgValidateStatus(QW_FPARAMS_DEF, int8_t oriStatus, int8_t newStatus, bool *ignore); int32_t qwDbgBuildAndSendRedirectRsp(int32_t rspType, SRpcHandleInfo *pConn, int32_t code, SEpSet *pEpSet); int32_t qwAddTaskCtx(QW_FPARAMS_DEF); +int32_t qwDbgResponseRedirect(SQWMsg *qwMsg, SQWTaskCtx *ctx); #ifdef __cplusplus diff --git a/source/libs/qworker/inc/qwMsg.h b/source/libs/qworker/inc/qwMsg.h index 704cd3142845c07976f1d8a4b8a798bc99e775d3..acb7004a510cb171562cbe94775f5149fabc7847 100644 --- a/source/libs/qworker/inc/qwMsg.h +++ b/source/libs/qworker/inc/qwMsg.h @@ -24,7 +24,7 @@ extern "C" { #include "dataSinkMgt.h" int32_t qwAbortPrerocessQuery(QW_FPARAMS_DEF); -int32_t qwPrerocessQuery(QW_FPARAMS_DEF, SQWMsg *qwMsg); +int32_t qwPreprocessQuery(QW_FPARAMS_DEF, SQWMsg *qwMsg); int32_t qwProcessQuery(QW_FPARAMS_DEF, SQWMsg *qwMsg, const char* sql); int32_t qwProcessCQuery(QW_FPARAMS_DEF, SQWMsg *qwMsg); int32_t qwProcessReady(QW_FPARAMS_DEF, SQWMsg *qwMsg); diff --git a/source/libs/qworker/src/qwDbg.c b/source/libs/qworker/src/qwDbg.c index 869eedf8f6e4970f548300b33b10c8bba246d92e..fa63cf2c3ae16d424ac702b7ef47fadadcc14235 100644 --- a/source/libs/qworker/src/qwDbg.c +++ b/source/libs/qworker/src/qwDbg.c @@ -9,7 +9,7 @@ #include "tmsg.h" #include "tname.h" -SQWDebug gQWDebug = {.statusEnable = true, .dumpEnable = false, .tmp = true}; +SQWDebug gQWDebug = {.statusEnable = true, .dumpEnable = false, .tmp = false}; int32_t qwDbgValidateStatus(QW_FPARAMS_DEF, int8_t oriStatus, int8_t newStatus, bool *ignore) { if (!gQWDebug.statusEnable) { @@ -147,9 +147,9 @@ int32_t qwDbgBuildAndSendRedirectRsp(int32_t rspType, SRpcHandleInfo *pConn, int return TSDB_CODE_SUCCESS; } -int32_t qwDbgResponseREdirect(SQWMsg *qwMsg, SQWTaskCtx *ctx) { +int32_t qwDbgResponseRedirect(SQWMsg *qwMsg, SQWTaskCtx *ctx) { if (gQWDebug.tmp) { - if (TDMT_SCH_QUERY == qwMsg->msgType) { + if (TDMT_SCH_QUERY == qwMsg->msgType && (0 == taosRand() % 3)) { SEpSet epSet = {0}; epSet.inUse = 1; epSet.numOfEps = 3; @@ -159,16 +159,15 @@ int32_t qwDbgResponseREdirect(SQWMsg *qwMsg, SQWTaskCtx *ctx) { epSet.eps[1].port = 7200; strcpy(epSet.eps[2].fqdn, "localhost"); epSet.eps[2].port = 7300; - + + ctx->phase = QW_PHASE_POST_QUERY; qwDbgBuildAndSendRedirectRsp(qwMsg->msgType + 1, &qwMsg->connInfo, TSDB_CODE_RPC_REDIRECT, &epSet); - gQWDebug.tmp = false; return TSDB_CODE_SUCCESS; } - if (TDMT_SCH_MERGE_QUERY == qwMsg->msgType) { + if (TDMT_SCH_MERGE_QUERY == qwMsg->msgType && (0 == taosRand() % 3)) { ctx->phase = QW_PHASE_POST_QUERY; qwDbgBuildAndSendRedirectRsp(qwMsg->msgType + 1, &qwMsg->connInfo, TSDB_CODE_RPC_REDIRECT, NULL); - gQWDebug.tmp = false; return TSDB_CODE_SUCCESS; } } diff --git a/source/libs/qworker/src/qwMsg.c b/source/libs/qworker/src/qwMsg.c index 73110472f7e35e71ee4bf478c422664212e9660c..93268e1bccd35640660f911f07c8066effbd07bf 100644 --- a/source/libs/qworker/src/qwMsg.c +++ b/source/libs/qworker/src/qwMsg.c @@ -315,10 +315,10 @@ int32_t qWorkerPreprocessQueryMsg(void *qWorkerMgmt, SRpcMsg *pMsg) { int64_t rId = msg->refId; int32_t eId = msg->execId; - SQWMsg qwMsg = {.msg = msg->msg + msg->sqlLen, .msgLen = msg->phyLen, .connInfo = pMsg->info}; + SQWMsg qwMsg = {.msgType = pMsg->msgType, .msg = msg->msg + msg->sqlLen, .msgLen = msg->phyLen, .connInfo = pMsg->info}; QW_SCH_TASK_DLOG("prerocessQuery start, handle:%p", pMsg->info.handle); - QW_ERR_RET(qwPrerocessQuery(QW_FPARAMS(), &qwMsg)); + QW_ERR_RET(qwPreprocessQuery(QW_FPARAMS(), &qwMsg)); QW_SCH_TASK_DLOG("prerocessQuery end, handle:%p", pMsg->info.handle); return TSDB_CODE_SUCCESS; diff --git a/source/libs/qworker/src/qworker.c b/source/libs/qworker/src/qworker.c index 3e8ced318c4baa2f1eec3aee85db03c6c67547ba..1b58dc28244fce3971392ecdcda66aec4d4ed876 100644 --- a/source/libs/qworker/src/qworker.c +++ b/source/libs/qworker/src/qworker.c @@ -469,7 +469,7 @@ int32_t qwAbortPrerocessQuery(QW_FPARAMS_DEF) { } -int32_t qwPrerocessQuery(QW_FPARAMS_DEF, SQWMsg *qwMsg) { +int32_t qwPreprocessQuery(QW_FPARAMS_DEF, SQWMsg *qwMsg) { int32_t code = 0; bool queryRsped = false; SSubplan *plan = NULL; @@ -488,6 +488,8 @@ int32_t qwPrerocessQuery(QW_FPARAMS_DEF, SQWMsg *qwMsg) { QW_ERR_JRET(qwAddTaskStatus(QW_FPARAMS(), JOB_TASK_STATUS_INIT)); + qwDbgResponseRedirect(qwMsg, ctx); + _return: if (ctx) { diff --git a/source/libs/scalar/src/sclfunc.c b/source/libs/scalar/src/sclfunc.c index c64b1d79bafaa41e2210f97c83b22cffdddfc29d..ba35356a9c8e6cf115ad1df1684caace1741c8dd 100644 --- a/source/libs/scalar/src/sclfunc.c +++ b/source/libs/scalar/src/sclfunc.c @@ -2139,3 +2139,171 @@ int32_t stddevScalarFunction(SScalarParam *pInput, int32_t inputNum, SScalarPara return TSDB_CODE_SUCCESS; } +#define LEASTSQR_CAL(p, x, y, index, step) \ + do { \ + (p)[0][0] += (double)(x) * (x); \ + (p)[0][1] += (double)(x); \ + (p)[0][2] += (double)(x) * (y)[index]; \ + (p)[1][2] += (y)[index]; \ + (x) += step; \ + } while (0) + +int32_t leastSQRScalarFunction(SScalarParam *pInput, int32_t inputNum, SScalarParam *pOutput) { + SColumnInfoData *pInputData = pInput->columnData; + SColumnInfoData *pOutputData = pOutput->columnData; + + double startVal, stepVal; + double matrix[2][3] = {0}; + GET_TYPED_DATA(startVal, double, GET_PARAM_TYPE(&pInput[1]), pInput[1].columnData->pData); + GET_TYPED_DATA(stepVal, double, GET_PARAM_TYPE(&pInput[2]), pInput[2].columnData->pData); + + int32_t type = GET_PARAM_TYPE(pInput); + int64_t count = 0; + + switch(type) { + case TSDB_DATA_TYPE_TINYINT: { + int8_t *in = (int8_t *)pInputData->pData; + for (int32_t i = 0; i < pInput->numOfRows; ++i) { + if (colDataIsNull_s(pInputData, i)) { + continue; + } + + count++; + LEASTSQR_CAL(matrix, startVal, in, i, stepVal); + } + break; + } + case TSDB_DATA_TYPE_SMALLINT: { + int16_t *in = (int16_t *)pInputData->pData; + for (int32_t i = 0; i < pInput->numOfRows; ++i) { + if (colDataIsNull_s(pInputData, i)) { + continue; + } + + count++; + LEASTSQR_CAL(matrix, startVal, in, i, stepVal); + } + break; + } + case TSDB_DATA_TYPE_INT: { + int32_t *in = (int32_t *)pInputData->pData; + for (int32_t i = 0; i < pInput->numOfRows; ++i) { + if (colDataIsNull_s(pInputData, i)) { + continue; + } + + count++; + LEASTSQR_CAL(matrix, startVal, in, i, stepVal); + } + break; + } + case TSDB_DATA_TYPE_BIGINT: { + int64_t *in = (int64_t *)pInputData->pData; + for (int32_t i = 0; i < pInput->numOfRows; ++i) { + if (colDataIsNull_s(pInputData, i)) { + continue; + } + + count++; + LEASTSQR_CAL(matrix, startVal, in, i, stepVal); + } + break; + } + case TSDB_DATA_TYPE_UTINYINT: { + uint8_t *in = (uint8_t *)pInputData->pData; + for (int32_t i = 0; i < pInput->numOfRows; ++i) { + if (colDataIsNull_s(pInputData, i)) { + continue; + } + + count++; + LEASTSQR_CAL(matrix, startVal, in, i, stepVal); + } + break; + } + case TSDB_DATA_TYPE_USMALLINT: { + uint16_t *in = (uint16_t *)pInputData->pData; + for (int32_t i = 0; i < pInput->numOfRows; ++i) { + if (colDataIsNull_s(pInputData, i)) { + continue; + } + + count++; + LEASTSQR_CAL(matrix, startVal, in, i, stepVal); + } + break; + } + case TSDB_DATA_TYPE_UINT: { + uint32_t *in = (uint32_t *)pInputData->pData; + for (int32_t i = 0; i < pInput->numOfRows; ++i) { + if (colDataIsNull_s(pInputData, i)) { + continue; + } + + count++; + LEASTSQR_CAL(matrix, startVal, in, i, stepVal); + } + break; + } + case TSDB_DATA_TYPE_UBIGINT: { + uint64_t *in = (uint64_t *)pInputData->pData; + for (int32_t i = 0; i < pInput->numOfRows; ++i) { + if (colDataIsNull_s(pInputData, i)) { + continue; + } + + count++; + LEASTSQR_CAL(matrix, startVal, in, i, stepVal); + } + break; + } + case TSDB_DATA_TYPE_FLOAT: { + float *in = (float *)pInputData->pData; + for (int32_t i = 0; i < pInput->numOfRows; ++i) { + if (colDataIsNull_s(pInputData, i)) { + continue; + } + + count++; + LEASTSQR_CAL(matrix, startVal, in, i, stepVal); + } + break; + } + case TSDB_DATA_TYPE_DOUBLE: { + double *in = (double *)pInputData->pData; + for (int32_t i = 0; i < pInput->numOfRows; ++i) { + if (colDataIsNull_s(pInputData, i)) { + continue; + } + + count++; + LEASTSQR_CAL(matrix, startVal, in, i, stepVal); + } + break; + } + } + + if (count == 0) { + colDataAppendNULL(pOutputData, 0); + } else { + matrix[1][1] = (double)count; + matrix[1][0] = matrix[0][1]; + + double matrix00 = matrix[0][0] - matrix[1][0] * (matrix[0][1] / matrix[1][1]); + double matrix02 = matrix[0][2] - matrix[1][2] * (matrix[0][1] / matrix[1][1]); + double matrix12 = matrix[1][2] - matrix02 * (matrix[1][0] / matrix00); + matrix02 /= matrix00; + + matrix12 /= matrix[1][1]; + + char buf[64] = {0}; + size_t len = + snprintf(varDataVal(buf), sizeof(buf) - VARSTR_HEADER_SIZE, "{slop:%.6lf, intercept:%.6lf}", matrix02, matrix12); + varDataSetLen(buf, len); + colDataAppend(pOutputData, 0, buf, false); + + } + + pOutput->numOfRows = 1; + return TSDB_CODE_SUCCESS; +} diff --git a/source/libs/scheduler/inc/schInt.h b/source/libs/scheduler/inc/schInt.h index b6b17cb1066e738e360bd7a8cd2d80c5b0d7def3..bc0270635d237df97b88823c0ea7a6badf2493d8 100644 --- a/source/libs/scheduler/inc/schInt.h +++ b/source/libs/scheduler/inc/schInt.h @@ -28,15 +28,6 @@ extern "C" { #include "trpc.h" #include "command.h" -#define SCHEDULE_DEFAULT_MAX_JOB_NUM 1000 -#define SCHEDULE_DEFAULT_MAX_TASK_NUM 1000 -#define SCHEDULE_DEFAULT_MAX_NODE_TABLE_NUM 200 // unit is TSDB_TABLE_NUM_UNIT - -#define SCH_DEFAULT_TASK_TIMEOUT_USEC 10000000 -#define SCH_MAX_TASK_TIMEOUT_USEC 60000000 - -#define SCH_MAX_CANDIDATE_EP_NUM TSDB_MAX_REPLICA - enum { SCH_READ = 1, SCH_WRITE, @@ -54,6 +45,24 @@ typedef enum { SCH_OP_GET_STATUS, } SCH_OP_TYPE; +typedef enum { + SCH_LOAD_SEQ = 1, + SCH_RANDOM, + SCH_ALL, +} SCH_POLICY; + +#define SCHEDULE_DEFAULT_MAX_JOB_NUM 1000 +#define SCHEDULE_DEFAULT_MAX_TASK_NUM 1000 +#define SCHEDULE_DEFAULT_MAX_NODE_TABLE_NUM 200 // unit is TSDB_TABLE_NUM_UNIT +#define SCHEDULE_DEFAULT_POLICY SCH_LOAD_SEQ + +#define SCH_DEFAULT_TASK_TIMEOUT_USEC 10000000 +#define SCH_MAX_TASK_TIMEOUT_USEC 60000000 +#define SCH_MAX_CANDIDATE_EP_NUM TSDB_MAX_REPLICA + + + + typedef struct SSchDebug { bool lockEnable; bool apiEnable; @@ -126,6 +135,13 @@ typedef struct SSchStatusFps { schStatusEventFp eventFp; } SSchStatusFps; +typedef struct SSchedulerCfg { + uint32_t maxJobNum; + int32_t maxNodeTableNum; + SCH_POLICY schPolicy; + bool enableReSchedule; +} SSchedulerCfg; + typedef struct SSchedulerMgmt { uint64_t taskId; // sequential taksId uint64_t sId; // schedulerId @@ -184,34 +200,36 @@ typedef struct SSchLevel { typedef struct SSchTaskProfile { int64_t startTs; - int64_t* execTime; + SArray* execTime; int64_t waitTime; int64_t endTs; } SSchTaskProfile; typedef struct SSchTask { - uint64_t taskId; // task id - SRWLatch lock; // task reentrant lock - int32_t maxExecTimes; // task may exec times - int32_t execId; // task current execute try index - SSchLevel *level; // level - SRWLatch planLock; // task update plan lock - SSubplan *plan; // subplan - char *msg; // operator tree - int32_t msgLen; // msg length - int8_t status; // task status - int32_t lastMsgType; // last sent msg type - int64_t timeoutUsec; // taks timeout useconds before reschedule - SQueryNodeAddr succeedAddr; // task executed success node address - int8_t candidateIdx; // current try condidation index - SArray *candidateAddrs; // condidate node addresses, element is SQueryNodeAddr - SHashObj *execNodes; // all tried node for current task, element is SSchNodeInfo - SSchTaskProfile profile; // task execution profile - int32_t childReady; // child task ready number - SArray *children; // the datasource tasks,from which to fetch the result, element is SQueryTask* - SArray *parents; // the data destination tasks, get data from current task, element is SQueryTask* - void* handle; // task send handle - bool registerdHb; // registered in hb + uint64_t taskId; // task id + SRWLatch lock; // task reentrant lock + int32_t maxExecTimes; // task max exec times + int32_t maxRetryTimes; // task max retry times + int32_t retryTimes; // task retry times + int32_t execId; // task current execute index + SSchLevel *level; // level + SRWLatch planLock; // task update plan lock + SSubplan *plan; // subplan + char *msg; // operator tree + int32_t msgLen; // msg length + int8_t status; // task status + int32_t lastMsgType; // last sent msg type + int64_t timeoutUsec; // task timeout useconds before reschedule + SQueryNodeAddr succeedAddr; // task executed success node address + int8_t candidateIdx; // current try condidation index + SArray *candidateAddrs; // condidate node addresses, element is SQueryNodeAddr + SHashObj *execNodes; // all tried node for current task, element is SSchNodeInfo + SSchTaskProfile profile; // task execution profile + int32_t childReady; // child task ready number + SArray *children; // the datasource tasks,from which to fetch the result, element is SQueryTask* + SArray *parents; // the data destination tasks, get data from current task, element is SQueryTask* + void* handle; // task send handle + bool registerdHb; // registered in hb } SSchTask; typedef struct SSchJobAttr { @@ -265,7 +283,7 @@ typedef struct SSchJob { extern SSchedulerMgmt schMgmt; -#define SCH_TASK_TIMEOUT(_task) ((taosGetTimestampUs() - (_task)->profile.execTime[(_task)->execId % (_task)->maxExecTimes]) > (_task)->timeoutUsec) +#define SCH_TASK_TIMEOUT(_task) ((taosGetTimestampUs() - *(int64_t*)taosArrayGet((_task)->profile.execTime, (_task)->execId)) > (_task)->timeoutUsec) #define SCH_TASK_READY_FOR_LAUNCH(readyNum, task) ((readyNum) >= taosArrayGetSize((task)->children)) @@ -299,7 +317,6 @@ extern SSchedulerMgmt schMgmt; #define SCH_TASK_NEED_FLOW_CTRL(_job, _task) (SCH_IS_DATA_BIND_QRY_TASK(_task) && SCH_JOB_NEED_FLOW_CTRL(_job) && SCH_IS_LEVEL_UNFINISHED((_task)->level)) #define SCH_FETCH_TYPE(_pSrcTask) (SCH_IS_DATA_BIND_QRY_TASK(_pSrcTask) ? TDMT_SCH_FETCH : TDMT_SCH_MERGE_FETCH) #define SCH_TASK_NEED_FETCH(_task) ((_task)->plan->subplanType != SUBPLAN_TYPE_MODIFY) -#define SCH_TASK_MAX_EXEC_TIMES(_levelIdx, _levelNum) (SCH_MAX_CANDIDATE_EP_NUM * ((_levelNum) - (_levelIdx))) #define SCH_SET_JOB_TYPE(_job, type) do { if ((type) != SUBPLAN_TYPE_MODIFY) { (_job)->attr.queryJob = true; } } while (0) #define SCH_IS_QUERY_JOB(_job) ((_job)->attr.queryJob) @@ -321,8 +338,7 @@ extern SSchedulerMgmt schMgmt; #define SCH_LOG_TASK_START_TS(_task) \ do { \ int64_t us = taosGetTimestampUs(); \ - int32_t idx = (_task)->execId % (_task)->maxExecTimes; \ - (_task)->profile.execTime[idx] = us; \ + taosArrayPush((_task)->profile.execTime, &us); \ if (0 == (_task)->execId) { \ (_task)->profile.startTs = us; \ } \ @@ -331,8 +347,7 @@ extern SSchedulerMgmt schMgmt; #define SCH_LOG_TASK_WAIT_TS(_task) \ do { \ int64_t us = taosGetTimestampUs(); \ - int32_t idx = (_task)->execId % (_task)->maxExecTimes; \ - (_task)->profile.waitTime += us - (_task)->profile.execTime[idx]; \ + (_task)->profile.waitTime += us - *(int64_t*)taosArrayGet((_task)->profile.execTime, (_task)->execId); \ } while (0) @@ -340,7 +355,8 @@ extern SSchedulerMgmt schMgmt; do { \ int64_t us = taosGetTimestampUs(); \ int32_t idx = (_task)->execId % (_task)->maxExecTimes; \ - (_task)->profile.execTime[idx] = us - (_task)->profile.execTime[idx]; \ + int64_t *startts = taosArrayGet((_task)->profile.execTime, (_task)->execId); \ + *startts = us - *startts; \ (_task)->profile.endTs = us; \ } while (0) @@ -471,9 +487,11 @@ void schFreeTask(SSchJob *pJob, SSchTask *pTask); void schDropTaskInHashList(SSchJob *pJob, SHashObj *list); int32_t schLaunchLevelTasks(SSchJob *pJob, SSchLevel *level); int32_t schGetTaskFromList(SHashObj *pTaskList, uint64_t taskId, SSchTask **pTask); -int32_t schInitTask(SSchJob *pJob, SSchTask *pTask, SSubplan *pPlan, SSchLevel *pLevel, int32_t levelNum); +int32_t schInitTask(SSchJob *pJob, SSchTask *pTask, SSubplan *pPlan, SSchLevel *pLevel); int32_t schSwitchTaskCandidateAddr(SSchJob *pJob, SSchTask *pTask); void schDirectPostJobRes(SSchedulerReq* pReq, int32_t errCode); +int32_t schHandleJobFailure(SSchJob *pJob, int32_t errCode); +int32_t schHandleJobDrop(SSchJob *pJob, int32_t errCode); bool schChkCurrentOp(SSchJob *pJob, int32_t op, bool sync); extern SSchDebug gSCHDebug; diff --git a/source/libs/scheduler/src/schJob.c b/source/libs/scheduler/src/schJob.c index 19bb93249f50d22bb535e25654fc93bbab11a981..1b1268baf1bb58e9fcf51ccef36dac8afe0f6a53 100644 --- a/source/libs/scheduler/src/schJob.c +++ b/source/libs/scheduler/src/schJob.c @@ -343,7 +343,7 @@ int32_t schValidateAndBuildJob(SQueryPlan *pDag, SSchJob *pJob) { SCH_ERR_JRET(TSDB_CODE_QRY_OUT_OF_MEMORY); } - SCH_ERR_JRET(schInitTask(pJob, pTask, plan, pLevel, levelNum)); + SCH_ERR_JRET(schInitTask(pJob, pTask, plan, pLevel)); SCH_ERR_JRET(schAppendJobDataSrc(pJob, pTask)); @@ -476,7 +476,7 @@ _return: SCH_UNLOCK(SCH_WRITE, &pJob->opStatus.lock); } -int32_t schProcessOnJobFailureImpl(SSchJob *pJob, int32_t status, int32_t errCode) { +int32_t schProcessOnJobFailure(SSchJob *pJob, int32_t errCode) { schUpdateJobErrCode(pJob, errCode); int32_t code = atomic_load_32(&pJob->errCode); @@ -489,21 +489,29 @@ int32_t schProcessOnJobFailureImpl(SSchJob *pJob, int32_t status, int32_t errCod SCH_RET(TSDB_CODE_SCH_IGNORE_ERROR); } -// Note: no more task error processing, handled in function internal -int32_t schProcessOnJobFailure(SSchJob *pJob, int32_t errCode) { +int32_t schHandleJobFailure(SSchJob *pJob, int32_t errCode) { if (TSDB_CODE_SCH_IGNORE_ERROR == errCode) { return TSDB_CODE_SCH_IGNORE_ERROR; } - schProcessOnJobFailureImpl(pJob, JOB_TASK_STATUS_FAIL, errCode); + schSwitchJobStatus(pJob, JOB_TASK_STATUS_FAIL, &errCode); return TSDB_CODE_SCH_IGNORE_ERROR; } -// Note: no more error processing, handled in function internal int32_t schProcessOnJobDropped(SSchJob *pJob, int32_t errCode) { - SCH_RET(schProcessOnJobFailureImpl(pJob, JOB_TASK_STATUS_DROP, errCode)); + SCH_RET(schProcessOnJobFailure(pJob, errCode)); +} + +int32_t schHandleJobDrop(SSchJob *pJob, int32_t errCode) { + if (TSDB_CODE_SCH_IGNORE_ERROR == errCode) { + return TSDB_CODE_SCH_IGNORE_ERROR; + } + + schSwitchJobStatus(pJob, JOB_TASK_STATUS_DROP, &errCode); + return TSDB_CODE_SCH_IGNORE_ERROR; } + int32_t schProcessOnJobPartialSuccess(SSchJob *pJob) { schPostJobRes(pJob, SCH_OP_EXEC); @@ -828,7 +836,7 @@ void schProcessOnOpEnd(SSchJob *pJob, SCH_OP_TYPE type, SSchedulerReq* pReq, int } if (errCode) { - schSwitchJobStatus(pJob, JOB_TASK_STATUS_FAIL, (void*)&errCode); + schHandleJobFailure(pJob, errCode); } SCH_JOB_DLOG("job end %s operation with code %s", schGetOpStr(type), tstrerror(errCode)); @@ -907,7 +915,7 @@ void schProcessOnCbEnd(SSchJob *pJob, SSchTask *pTask, int32_t errCode) { } if (errCode) { - schSwitchJobStatus(pJob, JOB_TASK_STATUS_FAIL, (void*)&errCode); + schHandleJobFailure(pJob, errCode); } if (pJob) { diff --git a/source/libs/scheduler/src/schTask.c b/source/libs/scheduler/src/schTask.c index 236257666b23b93d174f8f1126497f061dfd90c9..9483ecd6ebfbfd17d301280eed0191b6c26b6255 100644 --- a/source/libs/scheduler/src/schTask.c +++ b/source/libs/scheduler/src/schTask.c @@ -42,32 +42,47 @@ void schFreeTask(SSchJob *pJob, SSchTask *pTask) { taosHashCleanup(pTask->execNodes); } - taosMemoryFree(pTask->profile.execTime); + taosArrayDestroy(pTask->profile.execTime); } -int32_t schInitTask(SSchJob *pJob, SSchTask *pTask, SSubplan *pPlan, SSchLevel *pLevel, int32_t levelNum) { +void schInitTaskRetryTimes(SSchJob *pJob, SSchTask *pTask, SSchLevel *pLevel) { + if (SCH_IS_DATA_BIND_TASK(pTask) || (!SCH_IS_QUERY_JOB(pJob)) || (SCH_ALL != schMgmt.cfg.schPolicy)) { + pTask->maxRetryTimes = SCH_MAX_CANDIDATE_EP_NUM; + } else { + int32_t nodeNum = taosArrayGetSize(pJob->nodeList); + pTask->maxRetryTimes = TMAX(nodeNum, SCH_MAX_CANDIDATE_EP_NUM); + } + + pTask->maxExecTimes = pTask->maxRetryTimes * (pLevel->level + 1); +} + +int32_t schInitTask(SSchJob *pJob, SSchTask *pTask, SSubplan *pPlan, SSchLevel *pLevel) { int32_t code = 0; pTask->plan = pPlan; pTask->level = pLevel; pTask->execId = -1; - pTask->maxExecTimes = SCH_TASK_MAX_EXEC_TIMES(pLevel->level, levelNum); pTask->timeoutUsec = SCH_DEFAULT_TASK_TIMEOUT_USEC; pTask->taskId = schGenTaskId(); pTask->execNodes = taosHashInit(SCH_MAX_CANDIDATE_EP_NUM, taosGetDefaultHashFunction(TSDB_DATA_TYPE_INT), true, HASH_NO_LOCK); - pTask->profile.execTime = taosMemoryCalloc(pTask->maxExecTimes, sizeof(int64_t)); + + schInitTaskRetryTimes(pJob, pTask, pLevel); + + pTask->profile.execTime = taosArrayInit(pTask->maxExecTimes, sizeof(int64_t)); if (NULL == pTask->execNodes || NULL == pTask->profile.execTime) { SCH_ERR_JRET(TSDB_CODE_QRY_OUT_OF_MEMORY); } SCH_SET_TASK_STATUS(pTask, JOB_TASK_STATUS_INIT); + SCH_TASK_DLOG("task initialized, max times %d:%d", pTask->maxRetryTimes, pTask->maxExecTimes); + return TSDB_CODE_SUCCESS; _return: - taosMemoryFreeClear(pTask->profile.execTime); + taosArrayDestroy(pTask->profile.execTime); taosHashCleanup(pTask->execNodes); SCH_RET(code); @@ -105,7 +120,7 @@ int32_t schDropTaskExecNode(SSchJob *pJob, SSchTask *pTask, void *handle, int32_ } if (taosHashRemove(pTask->execNodes, &execId, sizeof(execId))) { - SCH_TASK_ELOG("fail to remove execId %d from execNodeList", execId); + SCH_TASK_DLOG("execId %d already not in execNodeList", execId); } else { SCH_TASK_DLOG("execId %d removed from execNodeList", execId); } @@ -235,7 +250,7 @@ int32_t schProcessOnTaskSuccess(SSchJob *pJob, SSchTask *pTask) { } if (pTask->level->taskFailed > 0) { - SCH_RET(schSwitchJobStatus(pJob, JOB_TASK_STATUS_FAIL, NULL)); + SCH_RET(schHandleJobFailure(pJob, pJob->errCode)); } else { SCH_RET(schSwitchJobStatus(pJob, JOB_TASK_STATUS_PART_SUCC, NULL)); } @@ -285,6 +300,10 @@ int32_t schProcessOnTaskSuccess(SSchJob *pJob, SSchTask *pTask) { } int32_t schRescheduleTask(SSchJob *pJob, SSchTask *pTask) { + if (!schMgmt.cfg.enableReSchedule) { + return TSDB_CODE_SUCCESS; + } + if (SCH_IS_DATA_BIND_TASK(pTask)) { return TSDB_CODE_SUCCESS; } @@ -304,13 +323,17 @@ int32_t schRescheduleTask(SSchJob *pJob, SSchTask *pTask) { int32_t schDoTaskRedirect(SSchJob *pJob, SSchTask *pTask, SDataBuf *pData, int32_t rspCode) { int32_t code = 0; - if ((pTask->execId + 1) >= pTask->maxExecTimes) { - SCH_TASK_DLOG("task no more retry since reach max try times, execId:%d", pTask->execId); - schSwitchJobStatus(pJob, JOB_TASK_STATUS_FAIL, (void *)&rspCode); - return TSDB_CODE_SUCCESS; + SCH_TASK_DLOG("task will be redirected now, status:%s", SCH_GET_TASK_STATUS_STR(pTask)); + + if (NULL == pData) { + pTask->retryTimes = 0; } - SCH_TASK_DLOG("task will be redirected now, status:%s", SCH_GET_TASK_STATUS_STR(pTask)); + if (((pTask->execId + 1) >= pTask->maxExecTimes) || ((pTask->retryTimes + 1) > pTask->maxRetryTimes)) { + SCH_TASK_DLOG("task no more retry since reach max times %d:%d, execId %d", pTask->maxRetryTimes, pTask->maxExecTimes, pTask->execId); + schHandleJobFailure(pJob, rspCode); + return TSDB_CODE_SUCCESS; + } schDropTaskOnExecNode(pJob, pTask); taosHashClear(pTask->execNodes); @@ -493,9 +516,15 @@ int32_t schTaskCheckSetRetry(SSchJob *pJob, SSchTask *pTask, int32_t errCode, bo } } + if ((pTask->retryTimes + 1) > pTask->maxRetryTimes) { + *needRetry = false; + SCH_TASK_DLOG("task no more retry since reach max retry times, retryTimes:%d/%d", pTask->retryTimes, pTask->maxRetryTimes); + return TSDB_CODE_SUCCESS; + } + if ((pTask->execId + 1) >= pTask->maxExecTimes) { *needRetry = false; - SCH_TASK_DLOG("task no more retry since reach max try times, execId:%d", pTask->execId); + SCH_TASK_DLOG("task no more retry since reach max exec times, execId:%d/%d", pTask->execId, pTask->maxExecTimes); return TSDB_CODE_SUCCESS; } @@ -649,10 +678,31 @@ int32_t schUpdateTaskCandidateAddr(SSchJob *pJob, SSchTask *pTask, SEpSet *pEpSe int32_t schSwitchTaskCandidateAddr(SSchJob *pJob, SSchTask *pTask) { int32_t candidateNum = taosArrayGetSize(pTask->candidateAddrs); - if (++pTask->candidateIdx >= candidateNum) { - pTask->candidateIdx = 0; + if (candidateNum <= 1) { + goto _return; + } + + switch (schMgmt.cfg.schPolicy) { + case SCH_LOAD_SEQ: + case SCH_ALL: + default: + if (++pTask->candidateIdx >= candidateNum) { + pTask->candidateIdx = 0; + } + break; + case SCH_RANDOM: { + int32_t lastIdx = pTask->candidateIdx; + while (lastIdx == pTask->candidateIdx) { + pTask->candidateIdx = taosRand() % candidateNum; + } + break; + } } - SCH_TASK_DLOG("switch task candiateIdx to %d", pTask->candidateIdx); + +_return: + + SCH_TASK_DLOG("switch task candiateIdx to %d/%d", pTask->candidateIdx, candidateNum); + return TSDB_CODE_SUCCESS; } @@ -739,8 +789,9 @@ int32_t schLaunchTaskImpl(SSchJob *pJob, SSchTask *pTask) { atomic_add_fetch_32(&pTask->level->taskLaunchedNum, 1); pTask->execId++; + pTask->retryTimes++; - SCH_TASK_DLOG("start to launch task's %dth exec", pTask->execId); + SCH_TASK_DLOG("start to launch task, execId %d, retry %d", pTask->execId, pTask->retryTimes); SCH_LOG_TASK_START_TS(pTask); diff --git a/source/libs/scheduler/src/scheduler.c b/source/libs/scheduler/src/scheduler.c index 39465f3064e9b117ca97c345223b17ce83e95a3d..3a15523040f7c0c108273563c17f8000c43ff582 100644 --- a/source/libs/scheduler/src/scheduler.c +++ b/source/libs/scheduler/src/scheduler.c @@ -22,26 +22,19 @@ SSchedulerMgmt schMgmt = { .jobRef = -1, }; -int32_t schedulerInit(SSchedulerCfg *cfg) { +int32_t schedulerInit() { if (schMgmt.jobRef >= 0) { qError("scheduler already initialized"); return TSDB_CODE_QRY_INVALID_INPUT; } - if (cfg) { - schMgmt.cfg = *cfg; - - if (schMgmt.cfg.maxJobNum == 0) { - schMgmt.cfg.maxJobNum = SCHEDULE_DEFAULT_MAX_JOB_NUM; - } - if (schMgmt.cfg.maxNodeTableNum <= 0) { - schMgmt.cfg.maxNodeTableNum = SCHEDULE_DEFAULT_MAX_NODE_TABLE_NUM; - } - } else { - schMgmt.cfg.maxJobNum = SCHEDULE_DEFAULT_MAX_JOB_NUM; - schMgmt.cfg.maxNodeTableNum = SCHEDULE_DEFAULT_MAX_NODE_TABLE_NUM; - } + schMgmt.cfg.maxJobNum = SCHEDULE_DEFAULT_MAX_JOB_NUM; + schMgmt.cfg.maxNodeTableNum = SCHEDULE_DEFAULT_MAX_NODE_TABLE_NUM; + schMgmt.cfg.schPolicy = SCHEDULE_DEFAULT_POLICY; + schMgmt.cfg.enableReSchedule = true; + qDebug("schedule policy init to %d", schMgmt.cfg.schPolicy); + schMgmt.jobRef = taosOpenRef(schMgmt.cfg.maxJobNum, schFreeJobImpl); if (schMgmt.jobRef < 0) { qError("init schduler jobRef failed, num:%u", schMgmt.cfg.maxJobNum); @@ -130,6 +123,26 @@ void schedulerStopQueryHb(void *pTrans) { schCleanClusterHb(pTrans); } +int32_t schedulerUpdatePolicy(int32_t policy) { + switch (policy) { + case SCH_LOAD_SEQ: + case SCH_RANDOM: + case SCH_ALL: + schMgmt.cfg.schPolicy = policy; + qDebug("schedule policy updated to %d", schMgmt.cfg.schPolicy); + break; + default: + return TSDB_CODE_TSC_INVALID_INPUT; + } + + return TSDB_CODE_SUCCESS; +} + +int32_t schedulerEnableReSchedule(bool enableResche) { + schMgmt.cfg.enableReSchedule = enableResche; + return TSDB_CODE_SUCCESS; +} + void schedulerFreeJob(int64_t* jobId, int32_t errCode) { if (0 == *jobId) { return; @@ -141,7 +154,7 @@ void schedulerFreeJob(int64_t* jobId, int32_t errCode) { return; } - schSwitchJobStatus(pJob, JOB_TASK_STATUS_DROP, (void*)&errCode); + schHandleJobDrop(pJob, errCode); schReleaseJob(*jobId); *jobId = 0; diff --git a/source/libs/scheduler/test/schedulerTests.cpp b/source/libs/scheduler/test/schedulerTests.cpp index d6b1baf978bdc823b3337e80830ec89f5de4e790..ca2122ed8f433c4b8dd70eb120e0eba972dda9fe 100644 --- a/source/libs/scheduler/test/schedulerTests.cpp +++ b/source/libs/scheduler/test/schedulerTests.cpp @@ -477,7 +477,7 @@ void* schtRunJobThread(void *aa) { schtInitLogFile(); - int32_t code = schedulerInit(NULL); + int32_t code = schedulerInit(); assert(code == 0); @@ -649,7 +649,7 @@ TEST(queryTest, normalCase) { qnodeAddr.port = 6031; taosArrayPush(qnodeList, &qnodeAddr); - int32_t code = schedulerInit(NULL); + int32_t code = schedulerInit(); ASSERT_EQ(code, 0); schtBuildQueryDag(&dag); @@ -756,7 +756,7 @@ TEST(queryTest, readyFirstCase) { qnodeAddr.port = 6031; taosArrayPush(qnodeList, &qnodeAddr); - int32_t code = schedulerInit(NULL); + int32_t code = schedulerInit(); ASSERT_EQ(code, 0); schtBuildQueryDag(&dag); @@ -866,7 +866,7 @@ TEST(queryTest, flowCtrlCase) { qnodeAddr.port = 6031; taosArrayPush(qnodeList, &qnodeAddr); - int32_t code = schedulerInit(NULL); + int32_t code = schedulerInit(); ASSERT_EQ(code, 0); schtBuildQueryFlowCtrlDag(&dag); @@ -975,7 +975,7 @@ TEST(insertTest, normalCase) { qnodeAddr.port = 6031; taosArrayPush(qnodeList, &qnodeAddr); - int32_t code = schedulerInit(NULL); + int32_t code = schedulerInit(); ASSERT_EQ(code, 0); schtBuildInsertDag(&dag); diff --git a/source/libs/transport/src/transCli.c b/source/libs/transport/src/transCli.c index 3805787ce2c0e87b50ffc8508fa2aee54d5cac8d..03e869e0785efa61a0cc9882fce6a12054798084 100644 --- a/source/libs/transport/src/transCli.c +++ b/source/libs/transport/src/transCli.c @@ -1042,7 +1042,7 @@ static void cliSchedMsgToNextNode(SCliMsg* pMsg, SCliThrd* pThrd) { STraceId* trace = &pMsg->msg.info.traceId; char tbuf[256] = {0}; EPSET_DEBUG_STR(&pCtx->epSet, tbuf); - tGTrace("%s retry on next node, use %s, retryCnt:%d, limit:%d", transLabel(pThrd->pTransInst), tbuf, + tGDebug("%s retry on next node, use %s, retryCnt:%d, limit:%d", transLabel(pThrd->pTransInst), tbuf, pCtx->retryCnt + 1, pCtx->retryLimit); STaskArg* arg = taosMemoryMalloc(sizeof(STaskArg)); @@ -1134,11 +1134,11 @@ int cliAppCb(SCliConn* pConn, STransMsg* pResp, SCliMsg* pMsg) { if (hasEpSet) { char tbuf[256] = {0}; EPSET_DEBUG_STR(&pCtx->epSet, tbuf); - tGTrace("%s conn %p extract epset from msg", CONN_GET_INST_LABEL(pConn), pConn); + tGDebug("%s conn %p extract epset from msg", CONN_GET_INST_LABEL(pConn), pConn); } if (pCtx->pSem != NULL) { - tGTrace("%s conn %p(sync) handle resp", CONN_GET_INST_LABEL(pConn), pConn); + tGDebug("%s conn %p(sync) handle resp", CONN_GET_INST_LABEL(pConn), pConn); if (pCtx->pRsp == NULL) { tGTrace("%s conn %p(sync) failed to resp, ignore", CONN_GET_INST_LABEL(pConn), pConn); } else { @@ -1147,7 +1147,7 @@ int cliAppCb(SCliConn* pConn, STransMsg* pResp, SCliMsg* pMsg) { tsem_post(pCtx->pSem); pCtx->pRsp = NULL; } else { - tGTrace("%s conn %p handle resp", CONN_GET_INST_LABEL(pConn), pConn); + tGDebug("%s conn %p handle resp", CONN_GET_INST_LABEL(pConn), pConn); if (retry == false && hasEpSet == true) { pTransInst->cfp(pTransInst->parent, pResp, &pCtx->epSet); } else { @@ -1257,7 +1257,7 @@ void transSendRequest(void* shandle, const SEpSet* pEpSet, STransMsg* pReq, STra cliMsg->refId = (int64_t)shandle; STraceId* trace = &pReq->info.traceId; - tGTrace("%s send request at thread:%08" PRId64 ", dst:%s:%d, app:%p", transLabel(pTransInst), pThrd->pid, + tGDebug("%s send request at thread:%08" PRId64 ", dst:%s:%d, app:%p", transLabel(pTransInst), pThrd->pid, EPSET_GET_INUSE_IP(&pCtx->epSet), EPSET_GET_INUSE_PORT(&pCtx->epSet), pReq->info.ahandle); ASSERT(transAsyncSend(pThrd->asyncPool, &(cliMsg->q)) == 0); transReleaseExHandle(transGetInstMgt(), (int64_t)shandle); @@ -1297,7 +1297,7 @@ void transSendRecv(void* shandle, const SEpSet* pEpSet, STransMsg* pReq, STransM cliMsg->refId = (int64_t)shandle; STraceId* trace = &pReq->info.traceId; - tGTrace("%s send request at thread:%08" PRId64 ", dst:%s:%d, app:%p", transLabel(pTransInst), pThrd->pid, + tGDebug("%s send request at thread:%08" PRId64 ", dst:%s:%d, app:%p", transLabel(pTransInst), pThrd->pid, EPSET_GET_INUSE_IP(&pCtx->epSet), EPSET_GET_INUSE_PORT(&pCtx->epSet), pReq->info.ahandle); transAsyncSend(pThrd->asyncPool, &(cliMsg->q)); diff --git a/source/libs/transport/src/transSvr.c b/source/libs/transport/src/transSvr.c index 68e12a19639d5731ab1e091e8bbbb40c7930968f..9a511adf9b11b89b36d85682813a141887028b7c 100644 --- a/source/libs/transport/src/transSvr.c +++ b/source/libs/transport/src/transSvr.c @@ -1020,7 +1020,7 @@ void transRefSrvHandle(void* handle) { return; } int ref = T_REF_INC((SSvrConn*)handle); - tDebug("conn %p ref count:%d", handle, ref); + tTrace("conn %p ref count:%d", handle, ref); } void transUnrefSrvHandle(void* handle) { @@ -1028,7 +1028,7 @@ void transUnrefSrvHandle(void* handle) { return; } int ref = T_REF_DEC((SSvrConn*)handle); - tDebug("conn %p ref count:%d", handle, ref); + tTrace("conn %p ref count:%d", handle, ref); if (ref == 0) { destroyConn((SSvrConn*)handle, true); } diff --git a/source/util/src/terror.c b/source/util/src/terror.c index 2364c53a9abd9c15a858e912eeb856d44e25474d..0de35a44cfcfb73bea141f198c97cbd479ceb738 100644 --- a/source/util/src/terror.c +++ b/source/util/src/terror.c @@ -78,6 +78,7 @@ TAOS_DEFINE_ERROR(TSDB_CODE_INVALID_TIMESTAMP, "Invalid timestamp for TAOS_DEFINE_ERROR(TSDB_CODE_MSG_DECODE_ERROR, "Msg decode error") TAOS_DEFINE_ERROR(TSDB_CODE_NO_AVAIL_DISK, "No available disk") TAOS_DEFINE_ERROR(TSDB_CODE_NOT_FOUND, "Not found") +TAOS_DEFINE_ERROR(TSDB_CODE_TIME_UNSYNCED, "Unsynced time") TAOS_DEFINE_ERROR(TSDB_CODE_REF_NO_MEMORY, "Ref out of memory") TAOS_DEFINE_ERROR(TSDB_CODE_REF_FULL, "too many Ref Objs") @@ -135,7 +136,7 @@ TAOS_DEFINE_ERROR(TSDB_CODE_TSC_STMT_API_ERROR, "Stmt API usage error" TAOS_DEFINE_ERROR(TSDB_CODE_TSC_STMT_TBNAME_ERROR, "Stmt table name not set") TAOS_DEFINE_ERROR(TSDB_CODE_TSC_STMT_CLAUSE_ERROR, "not supported stmt clause") TAOS_DEFINE_ERROR(TSDB_CODE_TSC_QUERY_KILLED, "Query killed") -TAOS_DEFINE_ERROR(TSDB_CODE_TSC_NO_EXEC_NODE, "No available execution node") +TAOS_DEFINE_ERROR(TSDB_CODE_TSC_NO_EXEC_NODE, "No available execution node in current query policy configuration") TAOS_DEFINE_ERROR(TSDB_CODE_TSC_NOT_STABLE_ERROR, "Table is not a super table") // mnode-common diff --git a/tests/script/api/batchprepare.c b/tests/script/api/batchprepare.c index b31c39718ca9addbf4e8e2980d57665bb3d19156..29c1fdb015689db8c2bd700adde4112dafe48e0b 100644 --- a/tests/script/api/batchprepare.c +++ b/tests/script/api/batchprepare.c @@ -2685,6 +2685,8 @@ int main(int argc, char *argv[]) runAll(taos); + taos_close(taos); + return 0; } diff --git a/tests/script/jenkins/basic.txt b/tests/script/jenkins/basic.txt index b5cb7ddb7909b7a032177bcf6a430a5ef11ef53c..6c1938d9fc473a70fde136a75adc13e62813f6de 100644 --- a/tests/script/jenkins/basic.txt +++ b/tests/script/jenkins/basic.txt @@ -87,24 +87,14 @@ ./test.sh -f tsim/parser/alter__for_community_version.sim ./test.sh -f tsim/parser/alter_column.sim ./test.sh -f tsim/parser/alter_stable.sim -# ./test.sh -f tsim/parser/auto_create_tb.sim -# ./test.sh -f tsim/parser/auto_create_tb_drop_tb.sim -# ./test.sh -f tsim/parser/between_and.sim -# ./test.sh -f tsim/parser/binary_escapeCharacter.sim -# ./test.sh -f tsim/parser/col_arithmetic_operation.sim -## ./test.sh -f tsim/parser/col_arithmetic_query.sim -## ./test.sh -f tsim/parser/columnValue.sim -## ./test.sh -f tsim/parser/columnValue_bigint.sim -## ./test.sh -f tsim/parser/columnValue_bool.sim -## ./test.sh -f tsim/parser/columnValue_double.sim -## ./test.sh -f tsim/parser/columnValue_float.sim -## ./test.sh -f tsim/parser/columnValue_int.sim -## ./test.sh -f tsim/parser/columnValue_smallint.sim -## ./test.sh -f tsim/parser/columnValue_tinyint.sim -## ./test.sh -f tsim/parser/columnValue_unsign.sim +# jira ./test.sh -f tsim/parser/auto_create_tb.sim +./test.sh -f tsim/parser/auto_create_tb_drop_tb.sim +./test.sh -f tsim/parser/between_and.sim +./test.sh -f tsim/parser/binary_escapeCharacter.sim +# nojira ./test.sh -f tsim/parser/col_arithmetic_operation.sim +# nojira ./test.sh -f tsim/parser/columnValue.sim ## ./test.sh -f tsim/parser/commit.sim ## ./test.sh -f tsim/parser/condition.sim -## ./test.sh -f tsim/parser/condition_query.sim ## ./test.sh -f tsim/parser/constCol.sim # ./test.sh -f tsim/parser/create_db.sim ## ./test.sh -f tsim/parser/create_db__for_community_version.sim @@ -117,7 +107,6 @@ # ./test.sh -f tsim/parser/fill_stb.sim ## ./test.sh -f tsim/parser/fill_us.sim # ./test.sh -f tsim/parser/first_last.sim -## ./test.sh -f tsim/parser/first_last_query.sim ./test.sh -f tsim/parser/fourArithmetic-basic.sim ## ./test.sh -f tsim/parser/function.sim ./test.sh -f tsim/parser/groupby-basic.sim @@ -132,24 +121,18 @@ ## ./test.sh -f tsim/parser/insert_multiTbl.sim # ./test.sh -f tsim/parser/insert_tb.sim ## ./test.sh -f tsim/parser/interp.sim -## ./test.sh -f tsim/parser/interp_test.sim # ./test.sh -f tsim/parser/join.sim # ./test.sh -f tsim/parser/join_manyblocks.sim ## ./test.sh -f tsim/parser/join_multitables.sim # ./test.sh -f tsim/parser/join_multivnode.sim # ./test.sh -f tsim/parser/last_cache.sim -## ./test.sh -f tsim/parser/last_cache_query.sim ## ./test.sh -f tsim/parser/last_groupby.sim # ./test.sh -f tsim/parser/lastrow.sim -## ./test.sh -f tsim/parser/lastrow_query.sim ## ./test.sh -f tsim/parser/like.sim # ./test.sh -f tsim/parser/limit.sim # ./test.sh -f tsim/parser/limit1.sim -## ./test.sh -f tsim/parser/limit1_stb.sim -## ./test.sh -f tsim/parser/limit1_tb.sim # ./test.sh -f tsim/parser/limit1_tblocks100.sim ## ./test.sh -f tsim/parser/limit2.sim -## ./test.sh -f tsim/parser/limit2_query.sim ## ./test.sh -f tsim/parser/limit2_tblocks100.sim ## ./test.sh -f tsim/parser/limit_stb.sim ## ./test.sh -f tsim/parser/limit_tb.sim @@ -169,20 +152,15 @@ # ./test.sh -f tsim/parser/select_with_tags.sim # ./test.sh -f tsim/parser/set_tag_vals.sim # ./test.sh -f tsim/parser/single_row_in_tb.sim -## ./test.sh -f tsim/parser/single_row_in_tb_query.sim # ./test.sh -f tsim/parser/sliding.sim # ./test.sh -f tsim/parser/slimit.sim # ./test.sh -f tsim/parser/slimit1.sim -## ./test.sh -f tsim/parser/slimit1_query.sim # ./test.sh -f tsim/parser/slimit_alter_tags.sim -## ./test.sh -f tsim/parser/slimit_query.sim # ./test.sh -f tsim/parser/stableOp.sim # ./test.sh -f tsim/parser/tags_dynamically_specifiy.sim # ./test.sh -f tsim/parser/tags_filter.sim # ./test.sh -f tsim/parser/tbnameIn.sim -## ./test.sh -f tsim/parser/tbnameIn_query.sim # ./test.sh -f tsim/parser/timestamp.sim -## ./test.sh -f tsim/parser/timestamp_query.sim ## ./test.sh -f tsim/parser/top_groupby.sim # ./test.sh -f tsim/parser/topbot.sim # ./test.sh -f tsim/parser/udf.sim @@ -381,7 +359,7 @@ # ---- compute ./test.sh -f tsim/compute/avg.sim -# jira ./test.sh -f tsim/compute/block_dist.sim +./test.sh -f tsim/compute/block_dist.sim ./test.sh -f tsim/compute/bottom.sim ./test.sh -f tsim/compute/count.sim ./test.sh -f tsim/compute/diff.sim diff --git a/tests/script/tsim/parser/auto_create_tb.sim b/tests/script/tsim/parser/auto_create_tb.sim index a902469fde690722698351f37dda7e89adff4b84..673a472cf70aeedf8322a9155e76fcabe30e4965 100644 --- a/tests/script/tsim/parser/auto_create_tb.sim +++ b/tests/script/tsim/parser/auto_create_tb.sim @@ -1,11 +1,8 @@ system sh/stop_dnodes.sh system sh/deploy.sh -n dnode1 -i 1 -system sh/cfg.sh -n dnode1 -c walLevel -v 1 -system sh/cfg.sh -n dnode1 -c maxtablesPerVnode -v 2 system sh/exec.sh -n dnode1 -s start - -sleep 100 sql connect + print ======================== dnode1 start $dbPrefix = ac_db @@ -153,36 +150,37 @@ print $rows $data00 $data10 $data20 if $rows != 3 then return -1 endi -if $data00 != tb1 then +if $data(tb1)[0] != tb1 then return -1 endi -if $data10 != tb2 then +if $data(tb2)[0] != tb2 then return -1 endi -if $data20 != tb3 then +if $data(tb3)[0] != tb3 then return -1 endi -sql select ts,c1,c2,c3,c4,c5,c7,c8,c9 from $stb +sql select c1,c1,c2,c3,c4,c5,c7,c8,c9 from $stb +print ===> $data00 $data01 $data02 $data03 $data04 $data05 $data06 $data07 $data08 $data09 +print ===> $data10 $data11 $data12 $data13 $data14 $data15 $data16 $data17 $data18 $data19 +print ===> $data20 $data21 $data22 $data23 $data24 $data25 $data26 $data27 $data28 $data29 + if $rows != 3 then return -1 endi -#if $data00 != @18-09-17 09:00:00.000@ then -# return -1 -#endi -if $data01 != 1 then +if $data(1)[1] != 1 then return -1 endi -if $data08 != 涛思数据1 then +if $data(1)[8] != 涛思数据1 then return -1 endi -if $data14 != 2.000000000 then +if $data(2)[4] != 2.000000000 then return -1 endi -if $data18 != 涛思数据2 then +if $data(2)[8] != 涛思数据2 then return -1 endi -if $data28 != 涛思数据3 then +if $data(3)[8] != 涛思数据3 then return -1 endi @@ -208,12 +206,7 @@ endi print ================== restart server to commit data into disk system sh/exec.sh -n dnode1 -s stop -x SIGINT -sleep 500 system sh/exec.sh -n dnode1 -s start -print ================== server restart completed -sql connect -sleep 100 -sql use $db #### auto create multiple tables sql insert into tb1 using $stb tags(1) values ( $ts0 , 1, 1, 1, 1, 'bin1', 1, 1, 1, '涛思数据1') tb2 using $stb tags(2) values ( $ts0 , 2, 2, 2, 2, 'bin2', 2, 2, 2, '涛思数据2') tb3 using $stb tags(3) values ( $ts0 , 3, 3, 3, 3, 'bin3', 3, 3, 3, '涛思数据3') @@ -221,36 +214,37 @@ sql show tables if $rows != 3 then return -1 endi -if $data00 != tb1 then +if $data(tb1)[0] != tb1 then return -1 endi -if $data10 != tb2 then +if $data(tb2)[0] != tb2 then return -1 endi -if $data20 != tb3 then +if $data(tb3)[0] != tb3 then return -1 endi -sql select ts,c1,c2,c3,c4,c5,c7,c8,c9 from $stb +sql select c1,c1,c2,c3,c4,c5,c7,c8,c9 from $stb +print ===> $data00 $data01 $data02 $data03 $data04 $data05 $data06 $data07 $data08 $data09 +print ===> $data10 $data11 $data12 $data13 $data14 $data15 $data16 $data17 $data18 $data19 +print ===> $data20 $data21 $data22 $data23 $data24 $data25 $data26 $data27 $data28 $data29 + if $rows != 3 then return -1 endi -#if $data00 != @18-09-17 09:00:00.000@ then -# return -1 -#endi -if $data01 != 1 then +if $data(1)[1] != 1 then return -1 endi -if $data08 != 涛思数据1 then +if $data(1)[8] != 涛思数据1 then return -1 endi -if $data14 != 2.000000000 then +if $data(2)[4] != 2.000000000 then return -1 endi -if $data18 != 涛思数据2 then +if $data(2)[8] != 涛思数据2 then return -1 endi -if $data28 != 涛思数据3 then +if $data(3)[8] != 涛思数据3 then return -1 endi diff --git a/tests/script/tsim/parser/auto_create_tb_drop_tb.sim b/tests/script/tsim/parser/auto_create_tb_drop_tb.sim index b04d02464347ac70b2a0fa74005a72e159b9c78e..0cff016b5f4fb2b4196b750fad03e21bde1aaa8d 100644 --- a/tests/script/tsim/parser/auto_create_tb_drop_tb.sim +++ b/tests/script/tsim/parser/auto_create_tb_drop_tb.sim @@ -1,10 +1,6 @@ system sh/stop_dnodes.sh system sh/deploy.sh -n dnode1 -i 1 -system sh/cfg.sh -n dnode1 -c walLevel -v 1 -system sh/cfg.sh -n dnode1 -c maxTablesPerVnode -v 4 -system sh/cfg.sh -n dnode1 -c ctime -v 30 system sh/exec.sh -n dnode1 -s start -sleep 100 sql connect $dbPrefix = db @@ -20,9 +16,9 @@ $i = 0 $db = $dbPrefix . $i $stb = $stbPrefix . $i -sql drop database $db -x step1 -step1: -sql create database $db maxrows 200 cache 2 +sql drop database if exists $db +sql create database $db + print ====== create tables sql use $db @@ -49,8 +45,6 @@ while $t < $tbNum endw print ====== tables created -sleep 100 - sql drop table tb2 $x = 0 while $x < $rowNum diff --git a/tests/script/tsim/parser/between_and.sim b/tests/script/tsim/parser/between_and.sim index cdced47cb65aea79618540b57e159b741bf9288a..aa9944d9a46d2b4b328c1a2d68a95ac5c193dee3 100644 --- a/tests/script/tsim/parser/between_and.sim +++ b/tests/script/tsim/parser/between_and.sim @@ -1,20 +1,14 @@ system sh/stop_dnodes.sh system sh/deploy.sh -n dnode1 -i 1 -system sh/cfg.sh -n dnode1 -c walLevel -v 1 -system sh/cfg.sh -n dnode1 -c maxtablesPerVnode -v 2 system sh/exec.sh -n dnode1 -s start - -sleep 100 sql connect -print ======================== dnode1 start +print ======================== dnode1 start $db = testdb - sql create database $db sql use $db sql create stable st2 (ts timestamp, f1 int, f2 float, f3 double, f4 bigint, f5 smallint, f6 tinyint, f7 bool, f8 binary(10), f9 nchar(10)) tags (id1 int, id2 float, id3 nchar(10), id4 double, id5 smallint, id6 bigint, id7 binary(10)) - sql create table tb1 using st2 tags (1,1.0,"1",1.0,1,1,"1"); sql create table tb2 using st2 tags (2,2.0,"2",2.0,2,2,"2"); sql create table tb3 using st2 tags (3,3.0,"3",3.0,3,3,"3"); @@ -25,132 +19,102 @@ sql insert into tb1 values (now-100s,2,2.0,2.0,2,2,2,true,"2","2") sql insert into tb1 values (now,3,3.0,3.0,3,3,3,true,"3","3") sql insert into tb1 values (now+100s,4,4.0,4.0,4,4,4,true,"4","4") sql insert into tb1 values (now+200s,4,4.0,4.0,4,4,4,true,"4","4") -sql insert into tb1 values (now+300s,4,4.0,4.0,4,4,4,true,"4","4") -sql insert into tb1 values (now+400s,4,4.0,4.0,4,4,4,true,"4","4") -sql insert into tb1 values (now+500s,4,4.0,4.0,4,4,4,true,"4","4") - -sql select tbname,id1 from st2; +sql insert into tb2 values (now+300s,4,4.0,4.0,4,4,4,true,"4","4") +sql insert into tb3 values (now+400s,4,4.0,4.0,4,4,4,true,"4","4") +sql insert into tb4 values (now+500s,4,4.0,4.0,4,4,4,true,"4","4") +sql select distinct(tbname), id1 from st2; if $rows != 4 then return -1 endi - sql select * from st2; - if $rows != 8 then return -1 endi sql select * from st2 where ts between now-50s and now+450s - if $rows != 5 then return -1 endi -sql select tbname,id1 from st2 where id1 between 2 and 3; - +sql select tbname, id1 from st2 where id1 between 2 and 3; if $rows != 2 then return -1 endi -if $data00 != tb2 then - return -1 -endi -if $data01 != 2 then - return -1 -endi -if $data10 != tb3 then - return -1 -endi -if $data11 != 3 then - return -1 -endi - -sql select tbname,id2 from st2 where id2 between 2.0 and 3.0; - -if $rows != 2 then +sql select tbname, id2 from st2 where id2 between 0.0 and 3.0; +if $rows != 7 then return -1 endi - -if $data00 != tb2 then +if $data(tb2)[0] != tb2 then return -1 endi -if $data01 != 2.00000 then +if $data(tb2)[1] != 2.00000 then return -1 endi -if $data10 != tb3 then +if $data(tb3)[0] != tb3 then return -1 endi -if $data11 != 3.00000 then +if $data(tb3)[1] != 3.00000 then return -1 endi - -sql select tbname,id4 from st2 where id4 between 2.0 and 3.0; - +sql select tbname, id4 from st2 where id2 between 2.0 and 3.0; if $rows != 2 then return -1 endi - -if $data00 != tb2 then +if $data(tb2)[0] != tb2 then return -1 endi -if $data01 != 2.000000000 then +if $data(tb2)[1] != 2.000000000 then return -1 endi -if $data10 != tb3 then +if $data(tb3)[0] != tb3 then return -1 endi -if $data11 != 3.000000000 then +if $data(tb3)[1] != 3.000000000 then return -1 endi - -sql select tbname,id5 from st2 where id5 between 2.0 and 3.0; - +sql select tbname, id5 from st2 where id5 between 2.0 and 3.0; if $rows != 2 then return -1 endi - -if $data00 != tb2 then +if $data(tb2)[0] != tb2 then return -1 endi -if $data01 != 2 then +if $data(tb2)[1] != 2 then return -1 endi -if $data10 != tb3 then +if $data(tb3)[0] != tb3 then return -1 endi -if $data11 != 3 then +if $data(tb3)[1] != 3 then return -1 endi sql select tbname,id6 from st2 where id6 between 2.0 and 3.0; - if $rows != 2 then return -1 endi - -if $data00 != tb2 then +if $data(tb2)[0] != tb2 then return -1 endi -if $data01 != 2 then +if $data(tb2)[1] != 2 then return -1 endi -if $data10 != tb3 then +if $data(tb3)[0] != tb3 then return -1 endi -if $data11 != 3 then +if $data(tb3)[1] != 3 then return -1 endi sql select * from st2 where f1 between 2 and 3 and f2 between 2.0 and 3.0 and f3 between 2.0 and 3.0 and f4 between 2.0 and 3.0 and f5 between 2.0 and 3.0 and f6 between 2.0 and 3.0; - if $rows != 2 then return -1 endi - if $data01 != 2 then return -1 endi @@ -158,8 +122,8 @@ if $data11 != 3 then return -1 endi -sql_error select * from st2 where f7 between 2.0 and 3.0; -sql_error select * from st2 where f8 between 2.0 and 3.0; -sql_error select * from st2 where f9 between 2.0 and 3.0; +sql select * from st2 where f7 between 2.0 and 3.0; +sql select * from st2 where f8 between 2.0 and 3.0; +sql select * from st2 where f9 between 2.0 and 3.0; system sh/exec.sh -n dnode1 -s stop -x SIGINT diff --git a/tests/script/tsim/parser/binary_escapeCharacter.sim b/tests/script/tsim/parser/binary_escapeCharacter.sim index 176edd4e298d92689140644587622b20a267f63e..0b437d8b04a39a400b25368263f88c2b846c155a 100644 --- a/tests/script/tsim/parser/binary_escapeCharacter.sim +++ b/tests/script/tsim/parser/binary_escapeCharacter.sim @@ -1,10 +1,6 @@ - system sh/stop_dnodes.sh - system sh/deploy.sh -n dnode1 -i 1 -system sh/cfg.sh -n dnode1 -c walLevel -v 1 system sh/exec.sh -n dnode1 -s start -sleep 100 sql connect sql drop database if exists ecdb diff --git a/tests/script/tsim/parser/col_arithmetic_operation.sim b/tests/script/tsim/parser/col_arithmetic_operation.sim index 8bb692e3bbe8af3ec9ed179ad29d40b4712d257b..add2945c666ba633c22623995f73dc7697cf91e0 100644 --- a/tests/script/tsim/parser/col_arithmetic_operation.sim +++ b/tests/script/tsim/parser/col_arithmetic_operation.sim @@ -1,10 +1,8 @@ system sh/stop_dnodes.sh - system sh/deploy.sh -n dnode1 -i 1 -system sh/cfg.sh -n dnode1 -c walLevel -v 1 system sh/exec.sh -n dnode1 -s start -sleep 100 sql connect + #========================================= setup environment ================================ $dbPrefix = ca_db @@ -101,19 +99,17 @@ $halfTbNum = $tbNum / 2 #endw #=================================== above are setup test environment ============================= -run general/parser/col_arithmetic_query.sim +run tsim/parser/col_arithmetic_query.sim #======================================= all in files query ======================================= print ================== restart server to commit data into disk system sh/exec.sh -n dnode1 -s stop -x SIGINT -sleep 500 system sh/exec.sh -n dnode1 -s start print ================== server restart completed sql connect -sleep 5000 -run general/parser/col_arithmetic_query.sim +run tsim/parser/col_arithmetic_query.sim # ================================================================================================ diff --git a/tests/script/tsim/parser/col_arithmetic_query.sim b/tests/script/tsim/parser/col_arithmetic_query.sim index 17ae6cfd6b8b5636101e67e8d99f6999e50a06a5..8ee891660b5ce672ecd7c90aa02cb0a8e0cf514e 100644 --- a/tests/script/tsim/parser/col_arithmetic_query.sim +++ b/tests/script/tsim/parser/col_arithmetic_query.sim @@ -173,7 +173,7 @@ endi # multi row result aggregation [d.4] sql_error select top(c1, 1) - bottom(c1, 1) from $tb sql_error select top(c1, 99) - bottom(c1, 99) from $tb -sql_error select top(c1,1) - 88 from $tb +sql select top(c1,1) - 88 from $tb # all data types [d.6] ================================================================ sql select c2-c1*1.1, c3/c2, c4*c3, c5%c4, (c6+c4)%22, c2-c2 from $tb @@ -227,13 +227,13 @@ endi # error case, ts/bool/binary/nchar not support arithmetic expression sql_error select ts+ts from $tb -sql_error select ts+22 from $tb -sql_error select c7*12 from $tb -sql_error select c8/55 from $tb -sql_error select c9+c8 from $tb -sql_error select c7-c8, c9-c8 from $tb +sql select ts+22 from $tb +sql select c7*12 from $tb +sql select c8/55 from $tb +sql select c9+c8 from $tb +sql select c7-c8, c9-c8 from $tb sql_error select ts-c9 from $tb -sql_error select c8+c7, c9+c9+c8+c7/c6 from $tb +sql select c8+c7, c9+c9+c8+c7/c6 from $tb # arithmetic expression in join [d.7]================================================== @@ -339,7 +339,7 @@ if $data20 != 0 then endi # tag filter(not support for normal table). [d.15]===================================== -sql_error select c2+99 from $tb where t1=12; +sql select c2+99 from $tb where t1=12; # multi-field output [d.16]============================================================ sql select c4*1+1/2,c4*1+1/2,c4*1+1/2,c4*1+1/2,c4*1+1/2 from $tb @@ -391,7 +391,7 @@ if $data00 != 0.000000000 then return -1 endi -sql select (count(c1) * 2) % 7.9, (count(c1) * 2), ( count(1)*2) from $stb order by ts desc; +sql select (count(c1) * 2) % 7.9, (count(c1) * 2), ( count(1)*2) from $stb if $rows != 1 then return -1 endi @@ -408,7 +408,7 @@ if $data02 != 200000.000000000 then return -1 endi -sql select spread( c1 )/44, spread(c1), 0.204545455 * 44 from $stb order by ts asc; +sql select spread( c1 )/44, spread(c1), 0.204545455 * 44 from $stb if $rows != 1 then return -1 endi @@ -487,8 +487,8 @@ sql_error select top(c1, 99) - bottom(c1, 99) from $stb sql select c2-c1, c3/c2, c4*c3, c5%c4, c6+99%22 from $stb # error case, ts/bool/binary/nchar not support arithmetic expression -sql_error select first(c7)*12 from $stb -sql_error select last(c8)/55 from $stb +sql select first(c7)*12 from $stb +sql select last(c8)/55 from $stb sql_error select last_row(c9) + last_row(c8) from $stb # arithmetic expression in join [d.7]=============================================================== diff --git a/tests/script/tsim/parser/columnValue.sim b/tests/script/tsim/parser/columnValue.sim index c98542fbf26a2d6098ca01c48c38d9fdca92b03f..68336cdcc14483b51e3079442dae8b3d730608aa 100644 --- a/tests/script/tsim/parser/columnValue.sim +++ b/tests/script/tsim/parser/columnValue.sim @@ -1,10 +1,6 @@ -#### system sh/stop_dnodes.sh - system sh/deploy.sh -n dnode1 -i 1 -system sh/cfg.sh -n dnode1 -c walLevel -v 1 system sh/exec.sh -n dnode1 -s start -sleep 100 sql connect print ========== columnValues.sim @@ -13,14 +9,14 @@ sql drop database if exists db sql create database db sql use db -run general/parser/columnValue_bool.sim -run general/parser/columnValue_tinyint.sim -run general/parser/columnValue_smallint.sim -run general/parser/columnValue_int.sim -run general/parser/columnValue_bigint.sim -run general/parser/columnValue_float.sim -run general/parser/columnValue_double.sim -run general/parser/columnValue_unsign.sim +run tsim/parser/columnValue_bool.sim +run tsim/parser/columnValue_tinyint.sim +run tsim/parser/columnValue_smallint.sim +run tsim/parser/columnValue_int.sim +run tsim/parser/columnValue_bigint.sim +run tsim/parser/columnValue_float.sim +run tsim/parser/columnValue_double.sim +run tsim/parser/columnValue_unsign.sim system sh/exec.sh -n dnode1 -s stop -x SIGINT diff --git a/tests/script/tsim/parser/commit.sim b/tests/script/tsim/parser/commit.sim index 7c4c883fb137607611ad2599f013bdd0ed5675b1..83b457673b69700f5dff270e011fa4beda62090e 100644 --- a/tests/script/tsim/parser/commit.sim +++ b/tests/script/tsim/parser/commit.sim @@ -1,10 +1,6 @@ system sh/stop_dnodes.sh - system sh/deploy.sh -n dnode1 -i 1 -system sh/cfg.sh -n dnode1 -c walLevel -v 1 -system sh/cfg.sh -n dnode1 -c maxTablesperVnode -v 100 system sh/exec.sh -n dnode1 -s start -sleep 100 sql connect $dbPrefix = sc_db diff --git a/tests/script/tsim/parser/condition.sim b/tests/script/tsim/parser/condition.sim index c3aed7e2a3b04c0ca2e27e2e62d92009e8b2fe8e..8c1327baae02bd57bae3958d6b120bed2d18bf95 100644 --- a/tests/script/tsim/parser/condition.sim +++ b/tests/script/tsim/parser/condition.sim @@ -1,14 +1,7 @@ system sh/stop_dnodes.sh - system sh/deploy.sh -n dnode1 -i 1 -system sh/cfg.sh -n dnode1 -c walLevel -v 1 -system sh/cfg.sh -n dnode1 -c maxtablespervnode -v 4 -system sh/cfg.sh -n dnode1 -c cache -v 1 system sh/exec.sh -n dnode1 -s start - -sleep 100 sql connect - sql drop database if exists cdb sql create database if not exists cdb sql use cdb @@ -139,7 +132,7 @@ sleep 100 sql connect -run general/parser/condition_query.sim +run tsim/parser/condition_query.sim print ================== restart server to commit data into disk system sh/exec.sh -n dnode1 -s stop -x SIGINT @@ -149,5 +142,5 @@ print ================== server restart completed sql connect sleep 100 -run general/parser/condition_query.sim +run tsim/parser/condition_query.sim diff --git a/tests/script/tsim/parser/constCol.sim b/tests/script/tsim/parser/constCol.sim index 66523517be92eb7bddcb248b54522143f27e09d5..5f50c950dd6a929af11b13b1a2c0d338479209e1 100644 --- a/tests/script/tsim/parser/constCol.sim +++ b/tests/script/tsim/parser/constCol.sim @@ -1,26 +1,5 @@ system sh/stop_dnodes.sh - system sh/deploy.sh -n dnode1 -i 1 -system sh/cfg.sh -n dnode1 -c walLevel -v 1 - -system sh/cfg.sh -n dnode1 -c dDebugFlag -v 135 -system sh/cfg.sh -n dnode1 -c mDebugFlag -v 135 -system sh/cfg.sh -n dnode1 -c sdbDebugFlag -v 135 -system sh/cfg.sh -n dnode1 -c rpcDebugFlag -v 135 -system sh/cfg.sh -n dnode1 -c cDebugFlag -v 135 -system sh/cfg.sh -n dnode1 -c monitorDebugflag -v 135 -system sh/cfg.sh -n dnode1 -c httpDebugFlag -v 135 -system sh/cfg.sh -n dnode1 -c uDebugFlag -v 135 - -system sh/cfg.sh -n dnode1 -c httpCacheSessions -v 10 -system sh/cfg.sh -n dnode1 -c httpMaxThreads -v 10 -system sh/cfg.sh -n dnode1 -c httpEnableCompress -v 0 - -system sh/cfg.sh -n dnode1 -c maxVnodeConnections -v 30000 -system sh/cfg.sh -n dnode1 -c maxMgmtConnections -v 30000 -system sh/cfg.sh -n dnode1 -c maxMeterConnections -v 30000 -system sh/cfg.sh -n dnode1 -c maxShellConns -v 30000 - system sh/exec.sh -n dnode1 -s start sql connect diff --git a/tests/script/tsim/parser/create_db.sim b/tests/script/tsim/parser/create_db.sim index 040331ec4f3a5398dd8de78c2b21436009e421bc..c4c5b89bd285c2e632d9c348d9d220f81a666008 100644 --- a/tests/script/tsim/parser/create_db.sim +++ b/tests/script/tsim/parser/create_db.sim @@ -1,12 +1,8 @@ system sh/stop_dnodes.sh - - system sh/deploy.sh -n dnode1 -i 1 -system sh/cfg.sh -n dnode1 -c walLevel -v 1 system sh/exec.sh -n dnode1 -s start - -sleep 100 sql connect + print ======================== dnode1 start $dbPrefix = fi_in_db diff --git a/tests/script/tsim/parser/create_db__for_community_version.sim b/tests/script/tsim/parser/create_db__for_community_version.sim index 5dc4263d5df9f22e611ea3948b3a7d8162dbd407..32a8f303c1f4751a393e50a2e5b4c68cd35ac254 100644 --- a/tests/script/tsim/parser/create_db__for_community_version.sim +++ b/tests/script/tsim/parser/create_db__for_community_version.sim @@ -1,12 +1,8 @@ system sh/stop_dnodes.sh - - system sh/deploy.sh -n dnode1 -i 1 -system sh/cfg.sh -n dnode1 -c walLevel -v 1 system sh/exec.sh -n dnode1 -s start - -sleep 100 sql connect + print ======================== dnode1 start $dbPrefix = fi_in_db diff --git a/tests/script/tsim/parser/create_mt.sim b/tests/script/tsim/parser/create_mt.sim index c606ba99ecb5f9060de9bb8f52af916c2d2b96b6..fafee66c76d6cca12c34b7174602396142293cf7 100644 --- a/tests/script/tsim/parser/create_mt.sim +++ b/tests/script/tsim/parser/create_mt.sim @@ -1,12 +1,8 @@ system sh/stop_dnodes.sh - - system sh/deploy.sh -n dnode1 -i 1 -system sh/cfg.sh -n dnode1 -c walLevel -v 1 system sh/exec.sh -n dnode1 -s start - -sleep 100 sql connect + print ======================== dnode1 start $dbPrefix = fi_in_db diff --git a/tests/script/tsim/parser/create_tb.sim b/tests/script/tsim/parser/create_tb.sim index ca57f401b9e68b0f597eef8c80862a67d8f22ddb..5203f289dc98a89466e4c7acb2dc4807ae714374 100644 --- a/tests/script/tsim/parser/create_tb.sim +++ b/tests/script/tsim/parser/create_tb.sim @@ -1,12 +1,8 @@ system sh/stop_dnodes.sh - - system sh/deploy.sh -n dnode1 -i 1 -system sh/cfg.sh -n dnode1 -c walLevel -v 1 system sh/exec.sh -n dnode1 -s start - -sleep 100 sql connect + print ======================== dnode1 start $dbPrefix = fi_in_db diff --git a/tests/script/tsim/parser/create_tb_with_tag_name.sim b/tests/script/tsim/parser/create_tb_with_tag_name.sim index 130f4097f6547e50127afd86d8d73902a5653ae2..b7b39b2f5ff5e037eeb4758cba396cb8e9afecf6 100644 --- a/tests/script/tsim/parser/create_tb_with_tag_name.sim +++ b/tests/script/tsim/parser/create_tb_with_tag_name.sim @@ -1,11 +1,8 @@ system sh/stop_dnodes.sh system sh/deploy.sh -n dnode1 -i 1 -system sh/cfg.sh -n dnode1 -c walLevel -v 1 -system sh/cfg.sh -n dnode1 -c maxtablesPerVnode -v 2 system sh/exec.sh -n dnode1 -s start - -sleep 100 sql connect + print ======================== dnode1 start $db = testdb diff --git a/tests/script/tsim/parser/dbtbnameValidate.sim b/tests/script/tsim/parser/dbtbnameValidate.sim index bc3bfefafb1a8317844e01c67b9a9146626222e6..86ffbe5c3786e8c16114180a0fa62b39a1a355e0 100644 --- a/tests/script/tsim/parser/dbtbnameValidate.sim +++ b/tests/script/tsim/parser/dbtbnameValidate.sim @@ -1,9 +1,6 @@ system sh/stop_dnodes.sh - system sh/deploy.sh -n dnode1 -i 1 -system sh/cfg.sh -n dnode1 -c walLevel -v 1 system sh/exec.sh -n dnode1 -s start -sleep 100 sql connect print ========== db name and table name check in create and drop, describe diff --git a/tests/script/tsim/parser/distinct.sim b/tests/script/tsim/parser/distinct.sim index e0eb74b5a57c7ef762155d2afae6b02e04213050..b90ca593ba31f4f874af8368e30f0fe5ae863ccc 100644 --- a/tests/script/tsim/parser/distinct.sim +++ b/tests/script/tsim/parser/distinct.sim @@ -1,10 +1,6 @@ system sh/stop_dnodes.sh - system sh/deploy.sh -n dnode1 -i 1 -system sh/cfg.sh -n dnode1 -c walLevel -v 1 -system sh/cfg.sh -n dnode1 -c maxtablesPerVnode -v 5 system sh/exec.sh -n dnode1 -s start -sleep 100 sql connect $dbPrefix = sav_db diff --git a/tests/script/tsim/parser/fill.sim b/tests/script/tsim/parser/fill.sim index 3413a0b59652550701b7220e3c8cc1fc90785b91..642c7bd8d4e78e3b7c49cb4ee6deb43ea9581452 100644 --- a/tests/script/tsim/parser/fill.sim +++ b/tests/script/tsim/parser/fill.sim @@ -1,9 +1,6 @@ system sh/stop_dnodes.sh - system sh/deploy.sh -n dnode1 -i 1 -system sh/cfg.sh -n dnode1 -c walLevel -v 1 system sh/exec.sh -n dnode1 -s start -sleep 100 sql connect $dbPrefix = m_fl_db diff --git a/tests/script/tsim/parser/fill_stb.sim b/tests/script/tsim/parser/fill_stb.sim index ba8ddbdf6ac6e9035398b3ac7c26861c02771e99..0aadcc5a9f463cfded2d70405c54325730a6f72c 100644 --- a/tests/script/tsim/parser/fill_stb.sim +++ b/tests/script/tsim/parser/fill_stb.sim @@ -1,9 +1,6 @@ system sh/stop_dnodes.sh - system sh/deploy.sh -n dnode1 -i 1 -system sh/cfg.sh -n dnode1 -c walLevel -v 1 system sh/exec.sh -n dnode1 -s start -sleep 100 sql connect $dbPrefix = fl1_db diff --git a/tests/script/tsim/parser/fill_us.sim b/tests/script/tsim/parser/fill_us.sim index 762413d0a1e975c778ccd3d31e54e0f2d347cef2..98c37c435d7bef99c20a95c03269e9fd6fc2e2bb 100644 --- a/tests/script/tsim/parser/fill_us.sim +++ b/tests/script/tsim/parser/fill_us.sim @@ -1,9 +1,6 @@ system sh/stop_dnodes.sh - system sh/deploy.sh -n dnode1 -i 1 -system sh/cfg.sh -n dnode1 -c walLevel -v 1 system sh/exec.sh -n dnode1 -s start -sleep 100 sql connect $dbPrefix = m_fl_db diff --git a/tests/script/tsim/parser/first_last.sim b/tests/script/tsim/parser/first_last.sim index 09047b4528a68219f27fc3b2e913a62fd567cd42..27bf42ead34c0fa13fae4cc1e973f6329ad12d7a 100644 --- a/tests/script/tsim/parser/first_last.sim +++ b/tests/script/tsim/parser/first_last.sim @@ -1,10 +1,6 @@ system sh/stop_dnodes.sh - system sh/deploy.sh -n dnode1 -i 1 -system sh/cfg.sh -n dnode1 -c walLevel -v 1 -system sh/cfg.sh -n dnode1 -c maxTablespervnode -v 4 system sh/exec.sh -n dnode1 -s start -sleep 100 sql connect $dbPrefix = first_db @@ -73,7 +69,7 @@ sql import into $tb (ts) values ( $ts ) print ====== test data created -run general/parser/first_last_query.sim +run tsim/parser/first_last_query.sim print ================== restart server to commit data into disk system sh/exec.sh -n dnode1 -s stop -x SIGINT @@ -83,7 +79,7 @@ print ================== server restart completed sql connect sleep 100 -run general/parser/first_last_query.sim +run tsim/parser/first_last_query.sim print =================> insert data regression test sql create database test keep 36500 diff --git a/tests/script/tsim/parser/fourArithmetic-basic.sim b/tests/script/tsim/parser/fourArithmetic-basic.sim index bd01813c61716c80b7fbd5bb9b6474090cd43cb2..bfda75e54d414dd6a2e61d59e392630e79e20a07 100644 --- a/tests/script/tsim/parser/fourArithmetic-basic.sim +++ b/tests/script/tsim/parser/fourArithmetic-basic.sim @@ -1,25 +1,6 @@ system sh/stop_dnodes.sh - system sh/deploy.sh -n dnode1 -i 1 system sh/exec.sh -n dnode1 -s start - -$loop_cnt = 0 -check_dnode_ready: - $loop_cnt = $loop_cnt + 1 - sleep 200 - if $loop_cnt == 10 then - print ====> dnode not ready! - return -1 - endi -sql show dnodes -print ===> $rows $data00 $data01 $data02 $data03 $data04 $data05 -if $data00 != 1 then - return -1 -endi -if $data04 != ready then - goto check_dnode_ready -endi - sql connect $dbNamme = d0 diff --git a/tests/script/tsim/parser/function.sim b/tests/script/tsim/parser/function.sim index 556292b21b218f4df2aaa034d8babe35903a23b8..451947e82af524a7ce6426d055ed2447951f48fe 100644 --- a/tests/script/tsim/parser/function.sim +++ b/tests/script/tsim/parser/function.sim @@ -1,9 +1,6 @@ system sh/stop_dnodes.sh - system sh/deploy.sh -n dnode1 -i 1 -system sh/cfg.sh -n dnode1 -c walLevel -v 1 system sh/exec.sh -n dnode1 -s start -sleep 100 sql connect $dbPrefix = m_func_db diff --git a/tests/script/tsim/parser/groupby-basic.sim b/tests/script/tsim/parser/groupby-basic.sim index 4d6b33612f54e55061ceee71e3c621a5012acd27..1edc99624a1b3b7cf8df9d4bfc21d61eb8239e05 100644 --- a/tests/script/tsim/parser/groupby-basic.sim +++ b/tests/script/tsim/parser/groupby-basic.sim @@ -1,9 +1,7 @@ system sh/stop_dnodes.sh - system sh/deploy.sh -n dnode1 -i 1 -system sh/cfg.sh -n dnode1 -c walLevel -v 1 -system sh/cfg.sh -n dnode1 -c maxtablespervnode -v 4 system sh/exec.sh -n dnode1 -s start +sql connect $loop_cnt = 0 check_dnode_ready: diff --git a/tests/script/tsim/parser/groupby.sim b/tests/script/tsim/parser/groupby.sim index 1fe19714bbd516c2e8938ce1290f04f8d2053839..8d7fad8cbc7baec7e53e85eed85a938d5b383c21 100644 --- a/tests/script/tsim/parser/groupby.sim +++ b/tests/script/tsim/parser/groupby.sim @@ -1,10 +1,6 @@ system sh/stop_dnodes.sh - system sh/deploy.sh -n dnode1 -i 1 -system sh/cfg.sh -n dnode1 -c walLevel -v 1 -system sh/cfg.sh -n dnode1 -c maxtablespervnode -v 4 system sh/exec.sh -n dnode1 -s start -sleep 100 sql connect $dbPrefix = group_db diff --git a/tests/script/tsim/parser/having.sim b/tests/script/tsim/parser/having.sim index e063333853e04faf1a7f4988b6dd1f11207aee5d..b3d64a4e3d029f50d522795d47c07f9b18876577 100644 --- a/tests/script/tsim/parser/having.sim +++ b/tests/script/tsim/parser/having.sim @@ -1,11 +1,8 @@ system sh/stop_dnodes.sh system sh/deploy.sh -n dnode1 -i 1 -system sh/cfg.sh -n dnode1 -c walLevel -v 1 -system sh/cfg.sh -n dnode1 -c maxtablesPerVnode -v 2 system sh/exec.sh -n dnode1 -s start - -sleep 100 sql connect + print ======================== dnode1 start $db = testdb diff --git a/tests/script/tsim/parser/having_child.sim b/tests/script/tsim/parser/having_child.sim index 0fe5448869a5720a62550a88981114e737e4965b..1ee148194337a42c8e1bd29e00fa1f80e2d9b1fb 100644 --- a/tests/script/tsim/parser/having_child.sim +++ b/tests/script/tsim/parser/having_child.sim @@ -1,11 +1,8 @@ system sh/stop_dnodes.sh system sh/deploy.sh -n dnode1 -i 1 -system sh/cfg.sh -n dnode1 -c walLevel -v 1 -system sh/cfg.sh -n dnode1 -c maxtablesPerVnode -v 2 system sh/exec.sh -n dnode1 -s start - -sleep 100 sql connect + print ======================== dnode1 start $db = testdb diff --git a/tests/script/tsim/parser/import.sim b/tests/script/tsim/parser/import.sim index 4468ab87a923fc65eeb22eff97dc7d56bfdc7dc9..5946cff4e211a0237114cb4c8b5eda6d92752c1d 100644 --- a/tests/script/tsim/parser/import.sim +++ b/tests/script/tsim/parser/import.sim @@ -1,9 +1,6 @@ system sh/stop_dnodes.sh - system sh/deploy.sh -n dnode1 -i 1 -system sh/cfg.sh -n dnode1 -c walLevel -v 1 system sh/exec.sh -n dnode1 -s start -sleep 100 sql connect $dbPrefix = impt_db diff --git a/tests/script/tsim/parser/import_commit1.sim b/tests/script/tsim/parser/import_commit1.sim index f330fe4cd94b1cba82ea985b9b4841245753e099..23259d8b016498fd8176a4c9c1a900819dcad306 100644 --- a/tests/script/tsim/parser/import_commit1.sim +++ b/tests/script/tsim/parser/import_commit1.sim @@ -1,10 +1,6 @@ system sh/stop_dnodes.sh - system sh/deploy.sh -n dnode1 -i 1 -system sh/cfg.sh -n dnode1 -c walLevel -v 1 -system sh/cfg.sh -n dnode1 -c ctime -v 30 system sh/exec.sh -n dnode1 -s start -sleep 100 sql connect $dbPrefix = ic_db diff --git a/tests/script/tsim/parser/import_commit2.sim b/tests/script/tsim/parser/import_commit2.sim index 47b30acb49111da69c1c0b8b1927fd7f13f0c31c..49fca0d477a20be3be54d2acfe55c6ecfc4a728c 100644 --- a/tests/script/tsim/parser/import_commit2.sim +++ b/tests/script/tsim/parser/import_commit2.sim @@ -1,10 +1,6 @@ system sh/stop_dnodes.sh - system sh/deploy.sh -n dnode1 -i 1 -system sh/cfg.sh -n dnode1 -c walLevel -v 1 -system sh/cfg.sh -n dnode1 -c ctime -v 30 system sh/exec.sh -n dnode1 -s start -sleep 100 sql connect $dbPrefix = ic_db diff --git a/tests/script/tsim/parser/import_commit3.sim b/tests/script/tsim/parser/import_commit3.sim index 1e041375de3da7fc2a5c17ae40caa1251b5d9d1b..d353c10387debd9c7ea6d467b82cf490fc06ce2d 100644 --- a/tests/script/tsim/parser/import_commit3.sim +++ b/tests/script/tsim/parser/import_commit3.sim @@ -1,10 +1,6 @@ system sh/stop_dnodes.sh - system sh/deploy.sh -n dnode1 -i 1 -system sh/cfg.sh -n dnode1 -c walLevel -v 1 -system sh/cfg.sh -n dnode1 -c ctime -v 30 system sh/exec.sh -n dnode1 -s start -sleep 100 sql connect $dbPrefix = ic_db diff --git a/tests/script/tsim/parser/import_file.sim b/tests/script/tsim/parser/import_file.sim index cf11194ba7c3b805725a665c6f92d6bb465b9e4e..35b656eb871c67230fd1b687c2b32edcbe5b9a48 100644 --- a/tests/script/tsim/parser/import_file.sim +++ b/tests/script/tsim/parser/import_file.sim @@ -1,11 +1,7 @@ system sh/stop_dnodes.sh - system sh/deploy.sh -n dnode1 -i 1 -system sh/cfg.sh -n dnode1 -c walLevel -v 1 system sh/exec.sh -n dnode1 -s start -sleep 500 sql connect -sleep 500 sql drop database if exists indb sql create database if not exists indb @@ -13,7 +9,7 @@ sql use indb $inFileName = '~/data.csv' $numOfRows = 10000 -system general/parser/gendata.sh +system tsim/parser/gendata.sh sql create table stbx (ts TIMESTAMP, collect_area NCHAR(12), device_id BINARY(16), imsi BINARY(16), imei BINARY(16), mdn BINARY(10), net_type BINARY(4), mno NCHAR(4), province NCHAR(10), city NCHAR(16), alarm BINARY(2)) tags(a int, b binary(12)); diff --git a/tests/script/tsim/parser/insert_multiTbl.sim b/tests/script/tsim/parser/insert_multiTbl.sim index b17323280eb0297fc648fc6d06b38856b7a2299e..85c58ef3d33f5d8a60733c605d04885fa83f5e73 100644 --- a/tests/script/tsim/parser/insert_multiTbl.sim +++ b/tests/script/tsim/parser/insert_multiTbl.sim @@ -1,12 +1,8 @@ system sh/stop_dnodes.sh - - system sh/deploy.sh -n dnode1 -i 1 -system sh/cfg.sh -n dnode1 -c walLevel -v 1 system sh/exec.sh -n dnode1 -s start -sleep 500 sql connect -sleep 100 + print ======================== dnode1 start sql create database mul_db diff --git a/tests/script/tsim/parser/insert_tb.sim b/tests/script/tsim/parser/insert_tb.sim index 1e431aef3dc8355b4766abb81db6a94ee34052bf..4fa04e0625ce53af1216d3d39d25ef6ef5403333 100644 --- a/tests/script/tsim/parser/insert_tb.sim +++ b/tests/script/tsim/parser/insert_tb.sim @@ -1,12 +1,8 @@ system sh/stop_dnodes.sh - - system sh/deploy.sh -n dnode1 -i 1 -system sh/cfg.sh -n dnode1 -c walLevel -v 1 system sh/exec.sh -n dnode1 -s start - -sleep 100 sql connect + print ======================== dnode1 start $dbPrefix = fi_in_db diff --git a/tests/script/tsim/parser/interp.sim b/tests/script/tsim/parser/interp.sim index f192837bb7422046b0fbf05e66b2f2165d7e3c10..4bb273af464d711026fd9347390ad09670593278 100644 --- a/tests/script/tsim/parser/interp.sim +++ b/tests/script/tsim/parser/interp.sim @@ -1,9 +1,6 @@ system sh/stop_dnodes.sh - system sh/deploy.sh -n dnode1 -i 1 -system sh/cfg.sh -n dnode1 -c walLevel -v 1 system sh/exec.sh -n dnode1 -s start -sleep 100 sql connect $dbPrefix = intp_db @@ -58,7 +55,7 @@ print ====== tables created sql create table ap1 (ts timestamp, pav float); sql INSERT INTO ap1 VALUES ('2021-07-25 02:19:54.100',1) ('2021-07-25 02:19:54.200',2) ('2021-07-25 02:19:54.300',3) ('2021-07-25 02:19:56.500',4) ('2021-07-25 02:19:57.500',5) ('2021-07-25 02:19:57.600',6) ('2021-07-25 02:19:57.900',7) ('2021-07-25 02:19:58.100',8) ('2021-07-25 02:19:58.300',9) ('2021-07-25 02:19:59.100',10) ('2021-07-25 02:19:59.300',11) ('2021-07-25 02:19:59.500',12) ('2021-07-25 02:19:59.700',13) ('2021-07-25 02:19:59.900',14) ('2021-07-25 02:20:05.000', 20) ('2021-07-25 02:25:00.000', 10000); -run general/parser/interp_test.sim +run tsim/parser/interp_test.sim print ================== restart server to commit data into disk system sh/exec.sh -n dnode1 -s stop -x SIGINT @@ -66,7 +63,7 @@ sleep 500 system sh/exec.sh -n dnode1 -s start print ================== server restart completed -run general/parser/interp_test.sim +run tsim/parser/interp_test.sim print ================= TD-5931 sql create stable st5931(ts timestamp, f int) tags(t int) diff --git a/tests/script/tsim/parser/join.sim b/tests/script/tsim/parser/join.sim index e2132589bd3a54d42e683094f184b3a4a4932f71..55842d5c16fb757c5c5015e81117e40224a5a693 100644 --- a/tests/script/tsim/parser/join.sim +++ b/tests/script/tsim/parser/join.sim @@ -1,13 +1,6 @@ system sh/stop_dnodes.sh - system sh/deploy.sh -n dnode1 -i 1 -system sh/cfg.sh -n dnode1 -c walLevel -v 1 -system sh/cfg.sh -n dnode1 -c debugFlag -v 135 -system sh/cfg.sh -n dnode1 -c rpcDebugFlag -v 135 -system sh/cfg.sh -n dnode1 -c maxtablespervnode -v 4 - system sh/exec.sh -n dnode1 -s start -sleep 100 sql connect $dbPrefix = join_db diff --git a/tests/script/tsim/parser/join_manyblocks.sim b/tests/script/tsim/parser/join_manyblocks.sim index fddd59c0a148975fd1bb6d63dbfb01ab46baa376..eb5e34b07991bebe5d31319ef9a31f26d9bdb9a5 100644 --- a/tests/script/tsim/parser/join_manyblocks.sim +++ b/tests/script/tsim/parser/join_manyblocks.sim @@ -1,12 +1,7 @@ system sh/stop_dnodes.sh - system sh/deploy.sh -n dnode1 -i 1 -system sh/cfg.sh -n dnode1 -c walLevel -v 1 -system sh/cfg.sh -n dnode1 -c maxtablespervnode -v 4 - system sh/exec.sh -n dnode1 -s start sql connect -sleep 100 $dbPrefix = join_m_db $tbPrefix = join_tb diff --git a/tests/script/tsim/parser/join_multitables.sim b/tests/script/tsim/parser/join_multitables.sim index d675499640b890398859243d49cfb04d9e1e7579..7a1c77acffc61abcba1524ed9800d3af55978bee 100644 --- a/tests/script/tsim/parser/join_multitables.sim +++ b/tests/script/tsim/parser/join_multitables.sim @@ -1,11 +1,8 @@ system sh/stop_dnodes.sh system sh/deploy.sh -n dnode1 -i 1 -system sh/cfg.sh -n dnode1 -c walLevel -v 0 -system sh/cfg.sh -n dnode1 -c maxtablesPerVnode -v 2 system sh/exec.sh -n dnode1 -s start - -sleep 100 sql connect + print ======================== dnode1 start $db = testdb diff --git a/tests/script/tsim/parser/join_multivnode.sim b/tests/script/tsim/parser/join_multivnode.sim index 61438241b0cb943cf9d1df09799076cf7dbadeab..a204b4fceaa2e5e99364929d7806c3563fdbf379 100644 --- a/tests/script/tsim/parser/join_multivnode.sim +++ b/tests/script/tsim/parser/join_multivnode.sim @@ -1,12 +1,7 @@ system sh/stop_dnodes.sh - system sh/deploy.sh -n dnode1 -i 1 -system sh/cfg.sh -n dnode1 -c walLevel -v 1 -system sh/cfg.sh -n dnode1 -c maxtablespervnode -v 4 - system sh/exec.sh -n dnode1 -s start sql connect -sleep 100 $dbPrefix = join_m_db $tbPrefix = join_tb diff --git a/tests/script/tsim/parser/last_cache.sim b/tests/script/tsim/parser/last_cache.sim index 9c414263ecc65cc11327bbcfc7a79131984393b9..5f45b64796495c90dc8291af746bc2736e4e64e6 100644 --- a/tests/script/tsim/parser/last_cache.sim +++ b/tests/script/tsim/parser/last_cache.sim @@ -1,11 +1,8 @@ system sh/stop_dnodes.sh system sh/deploy.sh -n dnode1 -i 1 -system sh/cfg.sh -n dnode1 -c walLevel -v 1 -system sh/cfg.sh -n dnode1 -c maxtablesPerVnode -v 4 system sh/exec.sh -n dnode1 -s start - -sleep 100 sql connect + print ======================== dnode1 start $db = testdb @@ -57,13 +54,13 @@ sql insert into tbb values ("2021-05-10 10:12:28",33,NULL, '35', -3005) sql insert into tbc values ("2021-05-11 10:12:29",36, 37, NULL, -4005) sql insert into tbd values ("2021-05-11 10:12:29",NULL,NULL,NULL,NULL ) -run general/parser/last_cache_query.sim +run tsim/parser/last_cache_query.sim system sh/exec.sh -n dnode1 -s stop -x SIGINT system sh/exec.sh -n dnode1 -s start -run general/parser/last_cache_query.sim +run tsim/parser/last_cache_query.sim system sh/exec.sh -n dnode1 -s stop -x SIGINT diff --git a/tests/script/tsim/parser/last_groupby.sim b/tests/script/tsim/parser/last_groupby.sim index f993324cd1ccb6e1d74f71b3a0d9b47cf3452b48..8f9574412d4e1bfe6d9c9d2bacb5402fc3596e0b 100644 --- a/tests/script/tsim/parser/last_groupby.sim +++ b/tests/script/tsim/parser/last_groupby.sim @@ -1,11 +1,8 @@ system sh/stop_dnodes.sh system sh/deploy.sh -n dnode1 -i 1 -system sh/cfg.sh -n dnode1 -c walLevel -v 0 -system sh/cfg.sh -n dnode1 -c maxtablesPerVnode -v 2 system sh/exec.sh -n dnode1 -s start - -sleep 100 sql connect + print ======================== dnode1 start $db = testdb diff --git a/tests/script/tsim/parser/lastrow.sim b/tests/script/tsim/parser/lastrow.sim index 7cdd04e2ccdb93c7e1f84298101d74e7c3af061f..d6638f2e98d030c5b90935f5d97a4311df7937a4 100644 --- a/tests/script/tsim/parser/lastrow.sim +++ b/tests/script/tsim/parser/lastrow.sim @@ -1,10 +1,6 @@ system sh/stop_dnodes.sh - system sh/deploy.sh -n dnode1 -i 1 -system sh/cfg.sh -n dnode1 -c walLevel -v 1 -system sh/cfg.sh -n dnode1 -c maxtablespervnode -v 4 system sh/exec.sh -n dnode1 -s start -sleep 100 sql connect $dbPrefix = lr_db @@ -58,7 +54,7 @@ endw print ====== test data created -run general/parser/lastrow_query.sim +run tsim/parser/lastrow_query.sim print ================== restart server to commit data into disk system sh/exec.sh -n dnode1 -s stop -x SIGINT @@ -68,7 +64,7 @@ print ================== server restart completed sql connect sleep 100 -run general/parser/lastrow_query.sim +run tsim/parser/lastrow_query.sim print =================== last_row + nested query sql use $db diff --git a/tests/script/tsim/parser/like.sim b/tests/script/tsim/parser/like.sim index fce996ebee37d887e2f86fc2a8566d844e8d04f5..3d1bff6abf1fced409589e38164708dc674379cc 100644 --- a/tests/script/tsim/parser/like.sim +++ b/tests/script/tsim/parser/like.sim @@ -1,11 +1,8 @@ system sh/stop_dnodes.sh system sh/deploy.sh -n dnode1 -i 1 -system sh/cfg.sh -n dnode1 -c walLevel -v 1 -system sh/cfg.sh -n dnode1 -c maxtablesPerVnode -v 4 system sh/exec.sh -n dnode1 -s start - -sleep 10 sql connect + print ======================== dnode1 start diff --git a/tests/script/tsim/parser/limit.sim b/tests/script/tsim/parser/limit.sim index 3af2cb301854b27bc1b9c33bf8b06cbd17e87fd3..f4a23697cdcfddb73ffede6de8be188f89626912 100644 --- a/tests/script/tsim/parser/limit.sim +++ b/tests/script/tsim/parser/limit.sim @@ -1,10 +1,6 @@ system sh/stop_dnodes.sh - system sh/deploy.sh -n dnode1 -i 1 -system sh/cfg.sh -n dnode1 -c walLevel -v 1 -system sh/cfg.sh -n dnode1 -c maxVgroupsPerDb -v 1 system sh/exec.sh -n dnode1 -s start -sleep 100 sql connect $dbPrefix = lm_db @@ -61,8 +57,8 @@ while $i < $halfNum endw print ====== tables created -run general/parser/limit_tb.sim -run general/parser/limit_stb.sim +run tsim/parser/limit_tb.sim +run tsim/parser/limit_stb.sim print ================== restart server to commit data into disk system sh/exec.sh -n dnode1 -s stop -x SIGINT @@ -72,8 +68,8 @@ print ================== server restart completed sql connect sleep 100 -run general/parser/limit_tb.sim -run general/parser/limit_stb.sim +run tsim/parser/limit_tb.sim +run tsim/parser/limit_stb.sim print ========> TD-6017 sql use $db diff --git a/tests/script/tsim/parser/limit1.sim b/tests/script/tsim/parser/limit1.sim index e37bea92207d5a3da34033bbe135570344751375..1f7299978450adaa34e57ea69eaaa170a59a31ea 100644 --- a/tests/script/tsim/parser/limit1.sim +++ b/tests/script/tsim/parser/limit1.sim @@ -1,10 +1,6 @@ system sh/stop_dnodes.sh - system sh/deploy.sh -n dnode1 -i 1 -system sh/cfg.sh -n dnode1 -c walLevel -v 1 -system sh/cfg.sh -n dnode1 -c maxVgroupsPerDb -v 1 system sh/exec.sh -n dnode1 -s start -sleep 100 sql connect $dbPrefix = lm1_db @@ -56,8 +52,8 @@ while $i < $halfNum endw print ====== tables created -run general/parser/limit1_tb.sim -run general/parser/limit1_stb.sim +run tsim/parser/limit1_tb.sim +run tsim/parser/limit1_stb.sim print ================== restart server to commit data into disk system sh/exec.sh -n dnode1 -s stop -x SIGINT @@ -65,7 +61,7 @@ sleep 500 system sh/exec.sh -n dnode1 -s start print ================== server restart completed -run general/parser/limit1_tb.sim -run general/parser/limit1_stb.sim +run tsim/parser/limit1_tb.sim +run tsim/parser/limit1_stb.sim system sh/exec.sh -n dnode1 -s stop -x SIGINT \ No newline at end of file diff --git a/tests/script/tsim/parser/limit1_tblocks100.sim b/tests/script/tsim/parser/limit1_tblocks100.sim index 4546ffdb7910a7dd999a119743fa36682fe0eade..f541ea7158f531eda792580df53c20079a1fd7a3 100644 --- a/tests/script/tsim/parser/limit1_tblocks100.sim +++ b/tests/script/tsim/parser/limit1_tblocks100.sim @@ -1,10 +1,6 @@ system sh/stop_dnodes.sh - system sh/deploy.sh -n dnode1 -i 1 -system sh/cfg.sh -n dnode1 -c walLevel -v 1 -system sh/cfg.sh -n dnode1 -c maxVgroupsPerDb -v 1 system sh/exec.sh -n dnode1 -s start -sleep 100 sql connect $dbPrefix = lm1_db @@ -56,8 +52,8 @@ while $i < $halfNum endw print ====== tables created -run general/parser/limit1_tb.sim -run general/parser/limit1_stb.sim +run tsim/parser/limit1_tb.sim +run tsim/parser/limit1_stb.sim print ================== restart server to commit data into disk system sh/exec.sh -n dnode1 -s stop -x SIGINT @@ -65,7 +61,7 @@ sleep 500 system sh/exec.sh -n dnode1 -s start print ================== server restart completed -run general/parser/limit1_tb.sim -run general/parser/limit1_stb.sim +run tsim/parser/limit1_tb.sim +run tsim/parser/limit1_stb.sim system sh/exec.sh -n dnode1 -s stop -x SIGINT diff --git a/tests/script/tsim/parser/limit2.sim b/tests/script/tsim/parser/limit2.sim index 336f9234c82ab51b1ff478e21f99e058074a2818..af03c6bb7febb0fea95ceeeb4b86280efc946920 100644 --- a/tests/script/tsim/parser/limit2.sim +++ b/tests/script/tsim/parser/limit2.sim @@ -65,7 +65,7 @@ while $i < $halfNum endw print ====== tables created -#run general/parser/limit2_query.sim +#run tsim/parser/limit2_query.sim print ================== restart server to commit data into disk system sh/exec.sh -n dnode1 -s stop -x SIGINT @@ -73,6 +73,6 @@ sleep 500 system sh/exec.sh -n dnode1 -s start print ================== server restart completed -run general/parser/limit2_query.sim +run tsim/parser/limit2_query.sim system sh/exec.sh -n dnode1 -s stop -x SIGINT \ No newline at end of file diff --git a/tests/script/tsim/parser/limit2_tblocks100.sim b/tests/script/tsim/parser/limit2_tblocks100.sim index 11f7a15eb06a3de4bc5ab2eda6acade17abe5000..0d87a41838002be3d9d2a4704070df63d3594fac 100644 --- a/tests/script/tsim/parser/limit2_tblocks100.sim +++ b/tests/script/tsim/parser/limit2_tblocks100.sim @@ -65,7 +65,7 @@ while $i < $halfNum endw print ====== tables created -#run general/parser/limit2_query.sim +#run tsim/parser/limit2_query.sim print ================== restart server to commit data into disk system sh/exec.sh -n dnode1 -s stop -x SIGINT @@ -73,4 +73,4 @@ sleep 100 system sh/exec.sh -n dnode1 -s start print ================== server restart completed -run general/parser/limit2_query.sim +run tsim/parser/limit2_query.sim diff --git a/tests/script/tsim/parser/line_insert.sim b/tests/script/tsim/parser/line_insert.sim index 85f2714ad3100766557797d2158d9d3e181b0f0b..cbd960bed688826eafbb5bdb17911123fd209a1c 100644 --- a/tests/script/tsim/parser/line_insert.sim +++ b/tests/script/tsim/parser/line_insert.sim @@ -1,9 +1,6 @@ system sh/stop_dnodes.sh - system sh/deploy.sh -n dnode1 -i 1 -system sh/cfg.sh -n dnode1 -c walLevel -v 1 system sh/exec.sh -n dnode1 -s start -sleep 2000 sql connect print =============== step1 diff --git a/tests/script/tsim/parser/mixed_blocks.sim b/tests/script/tsim/parser/mixed_blocks.sim index c20cf9a915e49d014e29ce4bb817802f22346f9e..50229ab35af826560a45de2e6208d043cc598db0 100644 --- a/tests/script/tsim/parser/mixed_blocks.sim +++ b/tests/script/tsim/parser/mixed_blocks.sim @@ -1,11 +1,6 @@ system sh/stop_dnodes.sh - system sh/deploy.sh -n dnode1 -i 1 -system sh/cfg.sh -n dnode1 -c walLevel -v 1 -system sh/cfg.sh -n dnode1 -c maxtablespervnode -v 4 system sh/exec.sh -n dnode1 -s start - -sleep 100 sql connect $dbPrefix = mb_db diff --git a/tests/script/tsim/parser/nchar.sim b/tests/script/tsim/parser/nchar.sim index 84719efcab9a18e6f554cdfb84942adf1de5add9..52fc8b6864ecb0fc8b7acafa61ab266e464ab2c7 100644 --- a/tests/script/tsim/parser/nchar.sim +++ b/tests/script/tsim/parser/nchar.sim @@ -1,10 +1,6 @@ system sh/stop_dnodes.sh - system sh/deploy.sh -n dnode1 -i 1 -system sh/cfg.sh -n dnode1 -c walLevel -v 1 system sh/exec.sh -n dnode1 -s start - -sleep 100 sql connect print ======================== dnode1 start diff --git a/tests/script/tsim/parser/nestquery.sim b/tests/script/tsim/parser/nestquery.sim index 3c1ba0336973b8d07c785337de2d2c66202520c4..c82718c1cb86e0470387b0895d5dcd58e55fb9df 100644 --- a/tests/script/tsim/parser/nestquery.sim +++ b/tests/script/tsim/parser/nestquery.sim @@ -1,10 +1,6 @@ system sh/stop_dnodes.sh - system sh/deploy.sh -n dnode1 -i 1 -system sh/cfg.sh -n dnode1 -c walLevel -v 1 system sh/exec.sh -n dnode1 -s start - -sleep 100 sql connect print ======================== dnode1 start diff --git a/tests/script/tsim/parser/null_char.sim b/tests/script/tsim/parser/null_char.sim index cb65290d2548c4ca6e7377d18b33d5c1768e818e..2bdb960968d2926c772351f374cccce509175c20 100644 --- a/tests/script/tsim/parser/null_char.sim +++ b/tests/script/tsim/parser/null_char.sim @@ -1,11 +1,6 @@ -#### TBASE-679 system sh/stop_dnodes.sh - system sh/deploy.sh -n dnode1 -i 1 -system sh/cfg.sh -n dnode1 -c walLevel -v 1 system sh/exec.sh -n dnode1 -s start - -sleep 100 sql connect print ========== NULL_char.sim diff --git a/tests/script/tsim/parser/precision_ns.sim b/tests/script/tsim/parser/precision_ns.sim index 3e9a2dd3ffdcd1f15e05c28273af9484e33840d4..bb822cd2b146a8d1c31e32ac1f0bd7f7f1ea422f 100644 --- a/tests/script/tsim/parser/precision_ns.sim +++ b/tests/script/tsim/parser/precision_ns.sim @@ -1,9 +1,6 @@ system sh/stop_dnodes.sh - system sh/deploy.sh -n dnode1 -i 1 -system sh/cfg.sh -n dnode1 -c walLevel -v 1 system sh/exec.sh -n dnode1 -s start -sleep 1000 sql connect $dbPrefix = m_di_db_ns diff --git a/tests/script/tsim/parser/projection_limit_offset.sim b/tests/script/tsim/parser/projection_limit_offset.sim index ffbcb28ffd9b4e15f707509dc5cc808ef3f8ce4a..37f2e79995478865f821ac14fe6d6833d45e935b 100644 --- a/tests/script/tsim/parser/projection_limit_offset.sim +++ b/tests/script/tsim/parser/projection_limit_offset.sim @@ -1,10 +1,6 @@ system sh/stop_dnodes.sh - system sh/deploy.sh -n dnode1 -i 1 -system sh/cfg.sh -n dnode1 -c walLevel -v 1 -system sh/cfg.sh -n dnode1 -c maxtablespervnode -v 4 system sh/exec.sh -n dnode1 -s start -sleep 100 sql connect $dbPrefix = group_db diff --git a/tests/script/tsim/parser/regex.sim b/tests/script/tsim/parser/regex.sim index eed36018d4c04ec5752e64105d025347982bfcb0..41f52575d6a3747b0be0bb577905e2c1c17c2d1d 100644 --- a/tests/script/tsim/parser/regex.sim +++ b/tests/script/tsim/parser/regex.sim @@ -1,10 +1,6 @@ system sh/stop_dnodes.sh system sh/deploy.sh -n dnode1 -i 1 -system sh/cfg.sh -n dnode1 -c walLevel -v 1 -system sh/cfg.sh -n dnode1 -c maxtablesPerVnode -v 4 system sh/exec.sh -n dnode1 -s start - -sleep 100 sql connect $db = testdb diff --git a/tests/script/tsim/parser/repeatAlter.sim b/tests/script/tsim/parser/repeatAlter.sim index 3695206f908391c80ef451ebad66e3b069b21a74..d28a03e193a031ee95b5d237481de8ed31651877 100644 --- a/tests/script/tsim/parser/repeatAlter.sim +++ b/tests/script/tsim/parser/repeatAlter.sim @@ -2,7 +2,7 @@ $i = 1 $loops = 10 while $i <= $loops print ====== repeat: $i - run general/parser/alter.sim + run tsim/parser/alter.sim $i = $i + 1 endw diff --git a/tests/script/tsim/parser/selectResNum.sim b/tests/script/tsim/parser/selectResNum.sim index dfd204e15240c93c1f9bcde32b8eb65c0918604a..ac5ccd6e076f7980ab76215d412c2c19706b8754 100644 --- a/tests/script/tsim/parser/selectResNum.sim +++ b/tests/script/tsim/parser/selectResNum.sim @@ -1,10 +1,6 @@ system sh/stop_dnodes.sh - system sh/deploy.sh -n dnode1 -i 1 -system sh/cfg.sh -n dnode1 -c walLevel -v 1 -system sh/cfg.sh -n dnode1 -c maxtablespervnode -v 200 system sh/exec.sh -n dnode1 -s start -sleep 100 sql connect $dbPrefix = sc_db diff --git a/tests/script/tsim/parser/select_across_vnodes.sim b/tests/script/tsim/parser/select_across_vnodes.sim index 9bf61ee61d16a5220b6ae501b8c9d345f9131c1e..0ee011cf8ae6412aa629046ff27775564519e4ec 100644 --- a/tests/script/tsim/parser/select_across_vnodes.sim +++ b/tests/script/tsim/parser/select_across_vnodes.sim @@ -1,10 +1,6 @@ system sh/stop_dnodes.sh - system sh/deploy.sh -n dnode1 -i 1 -system sh/cfg.sh -n dnode1 -c walLevel -v 1 -system sh/cfg.sh -n dnode1 -c maxtablesPerVnode -v 5 system sh/exec.sh -n dnode1 -s start -sleep 100 sql connect $dbPrefix = sav_db diff --git a/tests/script/tsim/parser/select_distinct_tag.sim b/tests/script/tsim/parser/select_distinct_tag.sim index d8e92d4bc5ed3e3a1def0b33faf23ec66047227d..92303ce64e4ab22c1d6e7d5efd30ec1e17505eb8 100644 --- a/tests/script/tsim/parser/select_distinct_tag.sim +++ b/tests/script/tsim/parser/select_distinct_tag.sim @@ -1,10 +1,6 @@ system sh/stop_dnodes.sh - system sh/deploy.sh -n dnode1 -i 1 -system sh/cfg.sh -n dnode1 -c walLevel -v 1 -system sh/cfg.sh -n dnode1 -c maxtablesPerVnode -v 5 system sh/exec.sh -n dnode1 -s start -sleep 100 sql connect $dbPrefix = sav_db diff --git a/tests/script/tsim/parser/select_from_cache_disk.sim b/tests/script/tsim/parser/select_from_cache_disk.sim index 7f8af52c6bd395416f982554083a81ee1939f059..2c9f359afe539628d8766a837edbe491924e4929 100644 --- a/tests/script/tsim/parser/select_from_cache_disk.sim +++ b/tests/script/tsim/parser/select_from_cache_disk.sim @@ -1,10 +1,6 @@ system sh/stop_dnodes.sh - system sh/deploy.sh -n dnode1 -i 1 -system sh/cfg.sh -n dnode1 -c walLevel -v 1 -system sh/cfg.sh -n dnode1 -c maxtablesPerVnode -v 2 system sh/exec.sh -n dnode1 -s start -sleep 100 sql connect $dbPrefix = scd_db diff --git a/tests/script/tsim/parser/select_with_tags.sim b/tests/script/tsim/parser/select_with_tags.sim index eb6cd75d2104f7ff61b5f5e5bccc12fdd239d3d5..b840a666f40d8e9b3330b51d8dc8d098c93a7b95 100644 --- a/tests/script/tsim/parser/select_with_tags.sim +++ b/tests/script/tsim/parser/select_with_tags.sim @@ -1,10 +1,6 @@ system sh/stop_dnodes.sh - system sh/deploy.sh -n dnode1 -i 1 -system sh/cfg.sh -n dnode1 -c walLevel -v 1 -system sh/cfg.sh -n dnode1 -c maxtablespervnode -v 4 system sh/exec.sh -n dnode1 -s start -sleep 100 sql connect $dbPrefix = select_tags_db diff --git a/tests/script/tsim/parser/set_tag_vals.sim b/tests/script/tsim/parser/set_tag_vals.sim index 4a63f9c6f1fc85a2a5be73477b82cc7d0474a49a..07b424ec6a2aea04e0f124f930108269ee520045 100644 --- a/tests/script/tsim/parser/set_tag_vals.sim +++ b/tests/script/tsim/parser/set_tag_vals.sim @@ -1,10 +1,6 @@ system sh/stop_dnodes.sh - system sh/deploy.sh -n dnode1 -i 1 -system sh/cfg.sh -n dnode1 -c walLevel -v 1 -system sh/cfg.sh -n dnode1 -c maxVgroupsPerDb -v 1 system sh/exec.sh -n dnode1 -s start -sleep 100 sql connect $dbPrefix = db diff --git a/tests/script/tsim/parser/single_row_in_tb.sim b/tests/script/tsim/parser/single_row_in_tb.sim index 5de2a51f0f81d9286184c11e97d682a2b82ebdcd..59a05528091e9efdd6956035e06af979fdfde27e 100644 --- a/tests/script/tsim/parser/single_row_in_tb.sim +++ b/tests/script/tsim/parser/single_row_in_tb.sim @@ -1,10 +1,6 @@ system sh/stop_dnodes.sh - system sh/deploy.sh -n dnode1 -i 1 -system sh/cfg.sh -n dnode1 -c walLevel -v 1 -system sh/cfg.sh -n dnode1 -c maxtablespervnode -v 4 system sh/exec.sh -n dnode1 -s start -sleep 100 sql connect $dbPrefix = sr_db @@ -28,7 +24,7 @@ sql create table $tb1 using $stb tags( 1 ) sql insert into $tb1 values ( $ts0 , 1, 2, 3, 4, true, 'binay10', '涛思nchar10' ) print ====== tables created -run general/parser/single_row_in_tb_query.sim +run tsim/parser/single_row_in_tb_query.sim print ================== restart server to commit data into disk system sh/exec.sh -n dnode1 -s stop -x SIGINT @@ -36,6 +32,6 @@ sleep 500 system sh/exec.sh -n dnode1 -s start print ================== server restart completed -run general/parser/single_row_in_tb_query.sim +run tsim/parser/single_row_in_tb_query.sim system sh/exec.sh -n dnode1 -s stop -x SIGINT \ No newline at end of file diff --git a/tests/script/tsim/parser/sliding.sim b/tests/script/tsim/parser/sliding.sim index b2695ff95f27c5fe6ba27c9b977c96b48a5e5801..18d7bda8a1265dddcfefbab88994e4996f543ac5 100644 --- a/tests/script/tsim/parser/sliding.sim +++ b/tests/script/tsim/parser/sliding.sim @@ -1,12 +1,6 @@ system sh/stop_dnodes.sh - system sh/deploy.sh -n dnode1 -i 1 -system sh/cfg.sh -n dnode1 -c walLevel -v 1 -system sh/cfg.sh -n dnode1 -c debugFlag -v 135 -system sh/cfg.sh -n dnode1 -c rpcDebugFlag -v 135 -system sh/cfg.sh -n dnode1 -c maxtablespervnode -v 4 system sh/exec.sh -n dnode1 -s start -sleep 100 sql connect $dbPrefix = sliding_db diff --git a/tests/script/tsim/parser/slimit.sim b/tests/script/tsim/parser/slimit.sim index 0af31f982604a3b6c6e0901cd94795fc503edd4b..9ca5da678a8b2b4f26d1c31de760f3d39496ce60 100644 --- a/tests/script/tsim/parser/slimit.sim +++ b/tests/script/tsim/parser/slimit.sim @@ -1,10 +1,6 @@ system sh/stop_dnodes.sh - system sh/deploy.sh -n dnode1 -i 1 -system sh/cfg.sh -n dnode1 -c walLevel -v 1 -system sh/cfg.sh -n dnode1 -c maxtablesPerVnode -v 4 system sh/exec.sh -n dnode1 -s start -sleep 100 sql connect $dbPrefix = slm_db @@ -93,7 +89,7 @@ while $i < $tbNum endw print ====== $db tables created -run general/parser/slimit_query.sim +run tsim/parser/slimit_query.sim print ================== restart server to commit data into disk system sh/exec.sh -n dnode1 -s stop -x SIGINT @@ -103,6 +99,6 @@ print ================== server restart completed sql connect sleep 100 -run general/parser/slimit_query.sim +run tsim/parser/slimit_query.sim system sh/exec.sh -n dnode1 -s stop -x SIGINT \ No newline at end of file diff --git a/tests/script/tsim/parser/slimit1.sim b/tests/script/tsim/parser/slimit1.sim index 2dede439ec38c862afcce5e1658fa02824282087..bb12bc32f1ebeeeddd3b12d25800769aa7d30347 100644 --- a/tests/script/tsim/parser/slimit1.sim +++ b/tests/script/tsim/parser/slimit1.sim @@ -1,10 +1,6 @@ system sh/stop_dnodes.sh - system sh/deploy.sh -n dnode1 -i 1 -system sh/cfg.sh -n dnode1 -c walLevel -v 1 -system sh/cfg.sh -n dnode1 -c maxtablesPerVnode -v 2 system sh/exec.sh -n dnode1 -s start -sleep 100 sql connect $dbPrefix = slm_alt_tg_db @@ -52,7 +48,7 @@ endw print ================== tables and data created -run general/parser/slimit1_query.sim +run tsim/parser/slimit1_query.sim print ================== restart server to commit data into disk system sh/exec.sh -n dnode1 -s stop -x SIGINT @@ -62,6 +58,6 @@ print ================== server restart completed sql connect sleep 100 -run general/parser/slimit1_query.sim +run tsim/parser/slimit1_query.sim system sh/exec.sh -n dnode1 -s stop -x SIGINT \ No newline at end of file diff --git a/tests/script/tsim/parser/stableOp.sim b/tests/script/tsim/parser/stableOp.sim index 8647657e7bee1ea73dcdbdc6f346e2279dd58cd5..4fe0a6f38d2849981d3635971f7cdf5e885adab7 100644 --- a/tests/script/tsim/parser/stableOp.sim +++ b/tests/script/tsim/parser/stableOp.sim @@ -1,11 +1,8 @@ system sh/stop_dnodes.sh - system sh/deploy.sh -n dnode1 -i 1 -system sh/cfg.sh -n dnode1 -c walLevel -v 1 system sh/exec.sh -n dnode1 -s start - -sleep 100 sql connect + print ======================== dnode1 start $dbPrefix = fi_in_db diff --git a/tests/script/tsim/parser/tags_dynamically_specifiy.sim b/tests/script/tsim/parser/tags_dynamically_specifiy.sim index f6b3dabf153e502b598d1bea8545db3232ceef2d..d1f73c4f6037fe8f2390034471950df186d0ca4c 100644 --- a/tests/script/tsim/parser/tags_dynamically_specifiy.sim +++ b/tests/script/tsim/parser/tags_dynamically_specifiy.sim @@ -1,11 +1,7 @@ system sh/stop_dnodes.sh - system sh/deploy.sh -n dnode1 -i 1 -system sh/cfg.sh -n dnode1 -c walLevel -v 1 system sh/exec.sh -n dnode1 -s start -sleep 100 sql connect -sleep 100 $db = dytag_db $tbNum = 10 diff --git a/tests/script/tsim/parser/tags_filter.sim b/tests/script/tsim/parser/tags_filter.sim index 3d3e79b6f52928af9d3333810f57300e52e0ebb0..bf33febdae728311c9f29c8e201a16ba7a12f8c2 100644 --- a/tests/script/tsim/parser/tags_filter.sim +++ b/tests/script/tsim/parser/tags_filter.sim @@ -1,9 +1,6 @@ system sh/stop_dnodes.sh - system sh/deploy.sh -n dnode1 -i 1 -system sh/cfg.sh -n dnode1 -c walLevel -v 1 system sh/exec.sh -n dnode1 -s start -sleep 100 sql connect $db = tf_db diff --git a/tests/script/tsim/parser/tbnameIn.sim b/tests/script/tsim/parser/tbnameIn.sim index 003a86f90b2f36602b4e999aee2974ef259d3670..e9206b59e219c5cf8104c3d966eaf71fff582c22 100644 --- a/tests/script/tsim/parser/tbnameIn.sim +++ b/tests/script/tsim/parser/tbnameIn.sim @@ -1,9 +1,6 @@ system sh/stop_dnodes.sh - system sh/deploy.sh -n dnode1 -i 1 -system sh/cfg.sh -n dnode1 -c walLevel -v 1 system sh/exec.sh -n dnode1 -s start -sleep 100 sql connect $dbPrefix = ti_db @@ -63,7 +60,7 @@ while $i < $halfNum endw print ====== tables created -run general/parser/tbnameIn_query.sim +run tsim/parser/tbnameIn_query.sim print ================== restart server to commit data into disk system sh/exec.sh -n dnode1 -s stop -x SIGINT @@ -71,6 +68,6 @@ sleep 500 system sh/exec.sh -n dnode1 -s start print ================== server restart completed -run general/parser/tbnameIn_query.sim +run tsim/parser/tbnameIn_query.sim system sh/exec.sh -n dnode1 -s stop -x SIGINT \ No newline at end of file diff --git a/tests/script/tsim/parser/timestamp.sim b/tests/script/tsim/parser/timestamp.sim index 0a87bce51dedecf0c39179ffc7b1aa864b5e7823..524f6d5de3279fcd75780ad16b1920bff738d47e 100644 --- a/tests/script/tsim/parser/timestamp.sim +++ b/tests/script/tsim/parser/timestamp.sim @@ -1,11 +1,6 @@ system sh/stop_dnodes.sh - system sh/deploy.sh -n dnode1 -i 1 -system sh/cfg.sh -n dnode1 -c walLevel -v 1 -system sh/cfg.sh -n dnode1 -c maxtablespervnode -v 4 - system sh/exec.sh -n dnode1 -s start -sleep 100 sql connect $dbPrefix = ts_db @@ -55,7 +50,7 @@ while $i < $tbNum endw print ====== $db tables created -run general/parser/timestamp_query.sim +run tsim/parser/timestamp_query.sim print ================== restart server to commit data into disk system sh/exec.sh -n dnode1 -s stop -x SIGINT @@ -65,4 +60,4 @@ print ================== server restart completed sql connect sleep 100 -run general/parser/timestamp_query.sim +run tsim/parser/timestamp_query.sim diff --git a/tests/script/tsim/parser/top_groupby.sim b/tests/script/tsim/parser/top_groupby.sim index 5709f4d1d7210761292d59aefa8984dad2fd2f23..ff479a1494f2ef4b7ac8700f3ac2f3d987194e52 100644 --- a/tests/script/tsim/parser/top_groupby.sim +++ b/tests/script/tsim/parser/top_groupby.sim @@ -1,11 +1,8 @@ system sh/stop_dnodes.sh system sh/deploy.sh -n dnode1 -i 1 -system sh/cfg.sh -n dnode1 -c walLevel -v 0 -system sh/cfg.sh -n dnode1 -c maxtablesPerVnode -v 2 system sh/exec.sh -n dnode1 -s start - -sleep 100 sql connect + print ======================== dnode1 start $db = testdb diff --git a/tests/script/tsim/parser/topbot.sim b/tests/script/tsim/parser/topbot.sim index ddab1b3f83acfc4fc454740093c3e4e003bab3d4..61b2db28623fdd2808dc7cdd4d87462362e1147e 100644 --- a/tests/script/tsim/parser/topbot.sim +++ b/tests/script/tsim/parser/topbot.sim @@ -1,11 +1,6 @@ system sh/stop_dnodes.sh - system sh/deploy.sh -n dnode1 -i 1 -system sh/cfg.sh -n dnode1 -c walLevel -v 1 -system sh/cfg.sh -n dnode1 -c maxtablespervnode -v 200 - system sh/exec.sh -n dnode1 -s start -sleep 100 sql connect $dbPrefix = tb_db diff --git a/tests/script/tsim/parser/union.sim b/tests/script/tsim/parser/union.sim index 0cd3cba84d5843a423440e55db2bb6f1df946e06..4d05d4ced7ca41b5e4135ab8b41dc41b297838c7 100644 --- a/tests/script/tsim/parser/union.sim +++ b/tests/script/tsim/parser/union.sim @@ -1,13 +1,6 @@ system sh/stop_dnodes.sh - system sh/deploy.sh -n dnode1 -i 1 -system sh/cfg.sh -n dnode1 -c walLevel -v 1 -system sh/cfg.sh -n dnode1 -c debugFlag -v 135 -system sh/cfg.sh -n dnode1 -c rpcDebugFlag -v 135 -system sh/cfg.sh -n dnode1 -c maxtablespervnode -v 4 - system sh/exec.sh -n dnode1 -s start -sleep 100 sql connect $dbPrefix = union_db diff --git a/tests/script/tsim/parser/where.sim b/tests/script/tsim/parser/where.sim index 6b789de4903a6abd4ef7ad66a28a6008b588d4fb..77eb3fd87e6cdb07ee5bb895b38e2c233311a62d 100644 --- a/tests/script/tsim/parser/where.sim +++ b/tests/script/tsim/parser/where.sim @@ -1,11 +1,6 @@ system sh/stop_dnodes.sh - system sh/deploy.sh -n dnode1 -i 1 -system sh/cfg.sh -n dnode1 -c walLevel -v 1 -system sh/cfg.sh -n dnode1 -c maxtablespervnode -v 4 system sh/exec.sh -n dnode1 -s start - -sleep 100 sql connect $dbPrefix = wh_db @@ -62,7 +57,7 @@ $i = 1 $tb = $tbPrefix . $i ## -sql_error select * from $tb where c7 +sql select * from $tb where c7 # TBASE-654 : invalid filter expression cause server crashed sql select count(*) from $tb where c1<10 and c1<>2 @@ -73,7 +68,6 @@ if $data00 != 900 then return -1 endi - sql select * from $tb where c7 = false $val = $rowNum / 100 if $rows != $val then diff --git a/tests/system-test/2-query/csum.py b/tests/system-test/2-query/csum.py index 708aa351830e6c088f39e546138ee2e518921ce3..425597a91903f4f75a8715d105d677290bee4a98 100644 --- a/tests/system-test/2-query/csum.py +++ b/tests/system-test/2-query/csum.py @@ -425,8 +425,8 @@ class TDTestCase: tdSql.checkRows(70) tdSql.query("select csum(c1) from stb1 partition by tbname ") tdSql.checkRows(40) - # tdSql.query("select csum(st1) from stb1 partition by tbname") - # tdSql.checkRows(70) + tdSql.query("select csum(st1) from stb1 partition by tbname") + tdSql.checkRows(70) tdSql.query("select csum(st1+c1) from stb1 partition by tbname") tdSql.checkRows(40) tdSql.query("select csum(st1+c1) from stb1 partition by tbname") @@ -445,22 +445,22 @@ class TDTestCase: tdSql.checkRows(40) # bug need fix - # tdSql.query("select tbname , csum(c1) from stb1 partition by tbname") - # tdSql.checkRows(40) - # tdSql.query("select tbname , csum(st1) from stb1 partition by tbname") - # tdSql.checkRows(70) - # tdSql.query("select tbname , csum(st1) from stb1 partition by tbname slimit 1") - # tdSql.checkRows(7) + tdSql.query("select tbname , csum(c1) from stb1 partition by tbname") + tdSql.checkRows(40) + tdSql.query("select tbname , csum(st1) from stb1 partition by tbname") + tdSql.checkRows(70) + tdSql.query("select tbname , csum(st1) from stb1 partition by tbname slimit 1") + tdSql.checkRows(7) # partition by tags - # tdSql.query("select st1 , csum(c1) from stb1 partition by st1") - # tdSql.checkRows(40) - # tdSql.query("select csum(c1) from stb1 partition by st1") - # tdSql.checkRows(40) - # tdSql.query("select st1 , csum(c1) from stb1 partition by st1 slimit 1") - # tdSql.checkRows(4) - # tdSql.query("select csum(c1) from stb1 partition by st1 slimit 1") - # tdSql.checkRows(4) + tdSql.query("select st1 , csum(c1) from stb1 partition by st1") + tdSql.checkRows(40) + tdSql.query("select csum(c1) from stb1 partition by st1") + tdSql.checkRows(40) + tdSql.query("select st1 , csum(c1) from stb1 partition by st1 slimit 1") + tdSql.checkRows(4) + tdSql.query("select csum(c1) from stb1 partition by st1 slimit 1") + tdSql.checkRows(4) # partition by col # tdSql.query("select c1 , csum(c1) from stb1 partition by c1") diff --git a/tests/system-test/2-query/last_row.py b/tests/system-test/2-query/last_row.py new file mode 100644 index 0000000000000000000000000000000000000000..449f5c780611378b4ee5f236f04cb9d6b1350a3c --- /dev/null +++ b/tests/system-test/2-query/last_row.py @@ -0,0 +1,803 @@ +import taos +import sys +import datetime +import inspect + +from util.log import * +from util.sql import * +from util.cases import * +import random + + +class TDTestCase: + updatecfgDict = {'debugFlag': 143, "cDebugFlag": 143, "uDebugFlag": 143, "rpcDebugFlag": 143, "tmrDebugFlag": 143, + "jniDebugFlag": 143, "simDebugFlag": 143, "dDebugFlag": 143, "dDebugFlag": 143, "vDebugFlag": 143, "mDebugFlag": 143, "qDebugFlag": 143, + "wDebugFlag": 143, "sDebugFlag": 143, "tsdbDebugFlag": 143, "tqDebugFlag": 143, "fsDebugFlag": 143, "fnDebugFlag": 143 ,"udf":0} + + def init(self, conn, logSql): + tdLog.debug(f"start to excute {__file__}") + tdSql.init(conn.cursor(), True) + self.tb_nums = 10 + self.row_nums = 20 + self.ts = 1434938400000 + self.time_step = 1000 + + def insert_datas_and_check_abs(self ,tbnums , rownums , time_step ): + tdLog.info(" prepare datas for auto check abs function ") + + tdSql.execute(" create database test cachelast 1 ") + tdSql.execute(" use test ") + tdSql.execute(" create stable stb (ts timestamp, c1 int, c2 bigint, c3 smallint, c4 tinyint,\ + c5 float, c6 double, c7 bool, c8 binary(16),c9 nchar(32), c10 timestamp) tags (t1 int)") + for tbnum in range(tbnums): + tbname = "sub_tb_%d"%tbnum + tdSql.execute(" create table %s using stb tags(%d) "%(tbname , tbnum)) + + ts = self.ts + for row in range(rownums): + ts = self.ts + time_step*row + c1 = random.randint(0,10000) + c2 = random.randint(0,100000) + c3 = random.randint(0,125) + c4 = random.randint(0,125) + c5 = random.random()/1.0 + c6 = random.random()/1.0 + c7 = "'true'" + c8 = "'binary_val'" + c9 = "'nchar_val'" + c10 = ts + tdSql.execute(f" insert into {tbname} values ({ts},{c1},{c2},{c3},{c4},{c5},{c6},{c7},{c8},{c9},{c10})") + + tdSql.execute("use test") + tbnames = ["stb", "sub_tb_1"] + support_types = ["BIGINT", "SMALLINT", "TINYINT", "FLOAT", "DOUBLE", "INT"] + for tbname in tbnames: + tdSql.query("desc {}".format(tbname)) + coltypes = tdSql.queryResult + for coltype in coltypes: + colname = coltype[0] + abs_sql = "select abs({}) from {} order by tbname ".format(colname, tbname) + origin_sql = "select {} from {} order by tbname".format(colname, tbname) + if coltype[1] in support_types: + self.check_result_auto(origin_sql , abs_sql) + + + def prepare_datas(self): + tdSql.execute("create database if not exists db keep 3650 duration 1000 cachelast 1") + tdSql.execute("use db") + tdSql.execute( + '''create table stb1 + (ts timestamp, c1 int, c2 bigint, c3 smallint, c4 tinyint, c5 float, c6 double, c7 bool, c8 binary(16),c9 nchar(32), c10 timestamp) + tags (t1 int) + ''' + ) + + tdSql.execute( + ''' + create table t1 + (ts timestamp, c1 int, c2 bigint, c3 smallint, c4 tinyint, c5 float, c6 double, c7 bool, c8 binary(16),c9 nchar(32), c10 timestamp) + ''' + ) + for i in range(4): + tdSql.execute(f'create table ct{i+1} using stb1 tags ( {i+1} )') + + for i in range(9): + tdSql.execute( + f"insert into ct1 values ( now()-{i*10}s, {1*i}, {11111*i}, {111*i}, {11*i}, {1.11*i}, {11.11*i}, {i%2}, 'binary{i}', 'nchar{i}', now()+{1*i}a )" + ) + tdSql.execute( + f"insert into ct4 values ( now()-{i*90}d, {1*i}, {11111*i}, {111*i}, {11*i}, {1.11*i}, {11.11*i}, {i%2}, 'binary{i}', 'nchar{i}', now()+{1*i}a )" + ) + tdSql.execute( + "insert into ct1 values (now()-45s, 0, 0, 0, 0, 0, 0, 0, 'binary0', 'nchar0', now()+8a )") + tdSql.execute( + "insert into ct1 values (now()+10s, 9, -99999, -999, -99, -9.99, -99.99, 1, 'binary9', 'nchar9', now()+9a )") + tdSql.execute( + "insert into ct1 values (now()+15s, 9, -99999, -999, -99, -9.99, NULL, 1, 'binary9', 'nchar9', now()+9a )") + tdSql.execute( + "insert into ct1 values (now()+20s, 9, -99999, -999, NULL, -9.99, -99.99, 1, 'binary9', 'nchar9', now()+9a )") + + tdSql.execute( + "insert into ct4 values (now()-810d, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL ) ") + tdSql.execute( + "insert into ct4 values (now()-400d, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL ) ") + tdSql.execute( + "insert into ct4 values (now()+90d, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL ) ") + + tdSql.execute( + f'''insert into t1 values + ( '2020-04-21 01:01:01.000', NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL ) + ( '2020-10-21 01:01:01.000', 1, 11111, 111, 11, 1.11, 11.11, 1, "binary1", "nchar1", now()+1a ) + ( '2020-12-31 01:01:01.000', 2, 22222, 222, 22, 2.22, 22.22, 0, "binary2", "nchar2", now()+2a ) + ( '2021-01-01 01:01:06.000', 3, 33333, 333, 33, 3.33, 33.33, 0, "binary3", "nchar3", now()+3a ) + ( '2021-05-07 01:01:10.000', 4, 44444, 444, 44, 4.44, 44.44, 1, "binary4", "nchar4", now()+4a ) + ( '2021-07-21 01:01:01.000', NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL ) + ( '2021-09-30 01:01:16.000', 5, 55555, 555, 55, 5.55, 55.55, 0, "binary5", "nchar5", now()+5a ) + ( '2022-02-01 01:01:20.000', 6, 66666, 666, 66, 6.66, 66.66, 1, "binary6", "nchar6", now()+6a ) + ( '2022-10-28 01:01:26.000', 7, 00000, 000, 00, 0.00, 00.00, 1, "binary7", "nchar7", "1970-01-01 08:00:00.000" ) + ( '2022-12-01 01:01:30.000', 8, -88888, -888, -88, -8.88, -88.88, 0, "binary8", "nchar8", "1969-01-01 01:00:00.000" ) + ( '2022-12-31 01:01:36.000', 9, -99999999999999999, -999, -99, -9.99, -999999999999999999999.99, 1, "binary9", "nchar9", "1900-01-01 00:00:00.000" ) + ( '2023-02-21 01:01:01.000', NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL ) + ''' + ) + + def prepare_tag_datas(self): + # prepare datas + tdSql.execute( + "create database if not exists testdb keep 3650 duration 1000 cachelast 1") + tdSql.execute(" use testdb ") + + tdSql.execute(f" create stable stb1 (ts timestamp, c1 int, c2 bigint, c3 smallint, c4 tinyint, c5 float, c6 double, c7 bool, c8 binary(16),c9 nchar(32), c10 timestamp , uc1 int unsigned,\ + uc2 bigint unsigned ,uc3 smallint unsigned , uc4 tinyint unsigned ) tags( t1 int , t2 bigint , t3 smallint , t4 tinyint , t5 float , t6 double , t7 bool , t8 binary(36)\ + , t9 nchar(36) , t10 int unsigned , t11 bigint unsigned ,t12 smallint unsigned , t13 tinyint unsigned ,t14 timestamp ) ") + + tdSql.execute( + ''' + create table t1 + (ts timestamp, c1 int, c2 bigint, c3 smallint, c4 tinyint, c5 float, c6 double, c7 bool, c8 binary(16),c9 nchar(32), c10 timestamp) + ''' + ) + for i in range(4): + tdSql.execute( + f'create table ct{i+1} using stb1 tags ( {1*i}, {11111*i}, {111*i}, {1*i}, {1.11*i}, {11.11*i}, {i%2}, "binary{i}", "nchar{i}" ,{111*i}, {1*i},{1*i},{1*i},now())') + + for i in range(9): + tdSql.execute( + f"insert into ct1 values ( now()-{i*10}s, {1*i}, {11111*i}, {111*i}, {11*i}, {1.11*i}, {11.11*i}, {i%2}, 'binary{i}', 'nchar{i}', now()+{1*i}a ,{111*i},{1111*i},{i},{i} )" + ) + tdSql.execute( + f"insert into ct4 values ( now()-{i*90}d, {1*i}, {11111*i}, {111*i}, {11*i}, {1.11*i}, {11.11*i}, {i%2}, 'binary{i}', 'nchar{i}', now()+{1*i}a ,{111*i},{1111*i},{i},{i})" + ) + tdSql.execute( + "insert into ct1 values (now()-45s, 0, 0, 0, 0, 0, 0, 0, 'binary0', 'nchar0', now()+8a ,0,0,0,0)") + tdSql.execute( + "insert into ct1 values (now()+10s, 9, -99999, -999, -99, -9.99, -99.99, 1, 'binary9', 'nchar9', now()+9a , 999 , 9999 , 9 , 9)") + tdSql.execute( + "insert into ct1 values (now()+15s, 9, -99999, -999, -99, -9.99, NULL, 1, 'binary9', 'nchar9', now()+9a , 999 , 99999 , 9 , 9)") + tdSql.execute( + "insert into ct1 values (now()+20s, 9, -99999, -999, NULL, -9.99, -99.99, 1, 'binary9', 'nchar9', now()+9a ,999 , 99999 , 9 , 9)") + + tdSql.execute( + "insert into ct4 values (now()-810d, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL , NULL, NULL, NULL, NULL) ") + tdSql.execute( + "insert into ct4 values (now()-400d, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL , NULL, NULL, NULL, NULL) ") + tdSql.execute( + "insert into ct4 values (now()+90d, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL , NULL, NULL, NULL, NULL ) ") + + tdSql.execute( + f'''insert into t1 values + ( '2020-04-21 01:01:01.000', NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL ) + ( '2020-10-21 01:01:01.000', 1, 11111, 111, 11, 1.11, 11.11, 1, "binary1", "nchar1", now()+1a ) + ( '2020-12-31 01:01:01.000', 2, 22222, 222, 22, 2.22, 22.22, 0, "binary2", "nchar2", now()+2a ) + ( '2021-01-01 01:01:06.000', 3, 33333, 333, 33, 3.33, 33.33, 0, "binary3", "nchar3", now()+3a ) + ( '2021-05-07 01:01:10.000', 4, 44444, 444, 44, 4.44, 44.44, 1, "binary4", "nchar4", now()+4a ) + ( '2021-07-21 01:01:01.000', NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL ) + ( '2021-09-30 01:01:16.000', 5, 55555, 555, 55, 5.55, 55.55, 0, "binary5", "nchar5", now()+5a ) + ( '2022-02-01 01:01:20.000', 6, 66666, 666, 66, 6.66, 66.66, 1, "binary6", "nchar6", now()+6a ) + ( '2022-10-28 01:01:26.000', 7, 00000, 000, 00, 0.00, 00.00, 1, "binary7", "nchar7", "1970-01-01 08:00:00.000" ) + ( '2022-12-01 01:01:30.000', 8, -88888, -888, -88, -8.88, -88.88, 0, "binary8", "nchar8", "1969-01-01 01:00:00.000" ) + ( '2022-12-31 01:01:36.000', 9, -99999999999999999, -999, -99, -9.99, -999999999999999999999.99, 1, "binary9", "nchar9", "1900-01-01 00:00:00.000" ) + ( '2023-02-21 01:01:01.000', NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL ) + ''' + ) + + def check_result_auto(self, origin_query, abs_query): + abs_result = tdSql.getResult(abs_query) + origin_result = tdSql.getResult(origin_query) + + auto_result = [] + + for row in origin_result: + row_check = [] + for elem in row: + if elem == None: + elem = None + elif elem >= 0: + elem = elem + else: + elem = -elem + row_check.append(elem) + auto_result.append(row_check) + + check_status = True + for row_index, row in enumerate(abs_result): + for col_index, elem in enumerate(row): + if auto_result[row_index][col_index] != elem: + check_status = False + if not check_status: + tdLog.notice( + "abs function value has not as expected , sql is \"%s\" " % abs_query) + sys.exit(1) + else: + tdLog.info( + "abs value check pass , it work as expected ,sql is \"%s\" " % abs_query) + + def test_errors(self): + tdSql.execute("use testdb") + + # bug need fix + # tdSql.query("select last_row(c1 ,NULL) from t1") + + error_sql_lists = [ + "select last_row from t1", + "select last_row(-+--+c1) from t1", + # "select +-last_row(c1) from t1", + # "select ++-last_row(c1) from t1", + # "select ++--last_row(c1) from t1", + # "select - -last_row(c1)*0 from t1", + # "select last_row(tbname+1) from t1 ", + "select last_row(123--123)==1 from t1", + "select last_row(c1) as 'd1' from t1", + "select last_row(c1 ,NULL) from t1", + "select last_row(,) from t1;", + "select last_row(abs(c1) ab from t1)", + "select last_row(c1) as int from t1", + "select last_row from stb1", + # "select last_row(-+--+c1) from stb1", + # "select +-last_row(c1) from stb1", + # "select ++-last_row(c1) from stb1", + # "select ++--last_row(c1) from stb1", + # "select - -last_row(c1)*0 from stb1", + # "select last_row(tbname+1) from stb1 ", + "select last_row(123--123)==1 from stb1", + "select last_row(c1) as 'd1' from stb1", + # "select last_row(c1 ,c2 ) from stb1", + "select last_row(c1 ,NULL) from stb1", + "select last_row(,) from stb1;", + "select last_row(abs(c1) ab from stb1)", + "select last_row(c1) as int from stb1" + ] + for error_sql in error_sql_lists: + tdSql.error(error_sql) + + def support_types(self): + tdSql.execute("use testdb") + tbnames = ["stb1", "t1", "ct1", "ct2"] + + for tbname in tbnames: + tdSql.query("desc {}".format(tbname)) + coltypes = tdSql.queryResult + for coltype in coltypes: + colname = coltype[0] + col_note = coltype[-1] + if col_note != "TAG": + abs_sql = "select last_row({}) from {}".format(colname, tbname) + tdSql.query(abs_sql) + + + def basic_abs_function(self): + + # basic query + tdSql.query("select c1 from ct3") + tdSql.checkRows(0) + tdSql.query("select c1 from t1") + tdSql.checkRows(12) + tdSql.query("select c1 from stb1") + tdSql.checkRows(25) + + # used for empty table , ct3 is empty + tdSql.query("select last_row(c1) from ct3") + tdSql.checkRows(0) + tdSql.query("select last_row(c2) from ct3") + tdSql.checkRows(0) + tdSql.query("select last_row(c3) from ct3") + tdSql.checkRows(0) + tdSql.query("select last_row(c4) from ct3") + tdSql.checkRows(0) + tdSql.query("select last_row(c5) from ct3") + tdSql.checkRows(0) + tdSql.query("select last_row(c6) from ct3") + + # used for regular table + + # bug need fix + tdSql.query("select last_row(c1) from t1") + tdSql.checkData(0, 0, None) + tdSql.query("select last_row(c1) from ct4") + tdSql.checkData(0, 0, None) + tdSql.query("select last_row(c1) from stb1") + tdSql.checkData(0, 0, None) + + # # bug need fix + tdSql.query("select last_row(c1), c2, c3 , c4, c5 from t1") + tdSql.checkData(0, 0, None) + tdSql.checkData(0, 1, None) + tdSql.checkData(0, 2, None) + + # # bug need fix + tdSql.query("select last_row(c1), c2, c3 , c4, c5 from ct1") + tdSql.checkData(0, 0, 9) + tdSql.checkData(0, 1, -99999) + tdSql.checkData(0, 2, -999) + tdSql.checkData(0, 3, None) + tdSql.checkData(0, 4,-9.99000) + + # bug need fix + # tdSql.query("select last_row(c1), c2, c3 , c4, c5 from stb1 where tbname='ct1'") + # tdSql.checkData(0, 0, 9) + # tdSql.checkData(0, 1, -99999) + # tdSql.checkData(0, 2, -999) + # tdSql.checkData(0, 3, None) + # tdSql.checkData(0, 4,-9.99000) + + # bug fix + tdSql.query("select last_row(abs(c1)) from ct1") + tdSql.checkData(0,0,9) + + # # bug fix + tdSql.query("select last_row(c1+1) from ct1") + tdSql.query("select last_row(c1+1) from stb1") + tdSql.query("select last_row(c1+1) from t1") + + # used for stable table + tdSql.query("select last_row(c1 ,c2 ,c3) ,last_row(c4) from ct1") + tdSql.checkData(0,0,9) + tdSql.checkData(0,1,-99999) + tdSql.checkData(0,2,-999) + tdSql.checkData(0,3,None) + + # bug need fix + tdSql.query("select last_row(c1 ,c2 ,c3) from stb1 ") + tdSql.checkData(0,0,None) + tdSql.checkData(0,1,None) + tdSql.checkData(0,2,None) + + # tdSql.query("select last_row(c1 ,c2 ,c3) ,last_row(c4) from stb1 where ts 5 ") + tdSql.checkData(0, 0, 6) + tdSql.checkData(0, 1, 6.000000000) + tdSql.checkData(0, 2, 6.000000000) + tdSql.checkData(0, 3, 5.900000000) + tdSql.checkData(0, 4, 2.084962501) + + tdSql.query( + "select last_row(c1,c2,c1+5) from ct4 where c1=5 ") + tdSql.checkData(0, 0, 5) + tdSql.checkData(0, 1, 55555) + tdSql.checkData(0, 2, 10.000000000) + + tdSql.query( + "select last(c1,c2,c1+5) from ct4 where c1=5 ") + tdSql.checkData(0, 0, 5) + tdSql.checkData(0, 1, 55555) + tdSql.checkData(0, 2, 10.000000000) + + tdSql.query( + "select c1,c2 , abs(c1) -0 ,ceil(c1-0.1)-0 ,floor(c1+0.1)-0.1 ,ceil(log(c1,2)-0.5) from ct4 where c1>log(c1,2) limit 1 ") + tdSql.checkRows(1) + tdSql.checkData(0, 0, 8) + tdSql.checkData(0, 1, 88888) + tdSql.checkData(0, 2, 8.000000000) + tdSql.checkData(0, 3, 8.000000000) + tdSql.checkData(0, 4, 7.900000000) + tdSql.checkData(0, 5, 3.000000000) + + def abs_Arithmetic(self): + pass + + def check_boundary_values(self): + + tdSql.execute("drop database if exists bound_test") + tdSql.execute("create database if not exists bound_test cachelast 2") + time.sleep(3) + tdSql.execute("use bound_test") + tdSql.execute( + "create table stb_bound (ts timestamp, c1 int, c2 bigint, c3 smallint, c4 tinyint, c5 float, c6 double, c7 bool, c8 binary(32),c9 nchar(32), c10 timestamp) tags (t1 int);" + ) + tdSql.execute(f'create table sub1_bound using stb_bound tags ( 1 )') + tdSql.execute( + f"insert into sub1_bound values ( now()-1s, 2147483647, 9223372036854775807, 32767, 127, 3.40E+38, 1.7e+308, True, 'binary_tb1', 'nchar_tb1', now() )" + ) + tdSql.execute( + f"insert into sub1_bound values ( now()-1s, -2147483647, -9223372036854775807, -32767, -127, -3.40E+38, -1.7e+308, True, 'binary_tb1', 'nchar_tb1', now() )" + ) + tdSql.execute( + f"insert into sub1_bound values ( now(), 2147483646, 9223372036854775806, 32766, 126, 3.40E+38, 1.7e+308, True, 'binary_tb1', 'nchar_tb1', now() )" + ) + tdSql.execute( + f"insert into sub1_bound values ( now(), -2147483646, -9223372036854775806, -32766, -126, -3.40E+38, -1.7e+308, True, 'binary_tb1', 'nchar_tb1', now() )" + ) + tdSql.error( + f"insert into sub1_bound values ( now()+1s, 2147483648, 9223372036854775808, 32768, 128, 3.40E+38, 1.7e+308, True, 'binary_tb1', 'nchar_tb1', now() )" + ) + + # check basic elem for table per row + tdSql.query( + "select last(c1) ,last_row(c2), last_row(c3)+1 , last(c4)+1 from sub1_bound ") + tdSql.checkData(0, 0, -2147483646) + tdSql.checkData(0, 1, -9223372036854775806) + tdSql.checkData(0, 2, -32765.000000000) + tdSql.checkData(0, 3, -125.000000000) + # check + - * / in functions + tdSql.query( + "select last_row(c1+1) ,last_row(c2) , last(c3*1) , last(c4/2) from sub1_bound ") + + def test_tag_compute_for_scalar_function(self): + + tdSql.execute("use testdb") + + # bug need fix + + tdSql.query(" select sum(c1) from stb1 where t1+10 >1; ") + tdSql.query("select c1 ,t1 from stb1 where t1 =0 ") + tdSql.checkRows(13) + tdSql.query("select last_row(c1,t1) from stb1 ") + tdSql.checkData(0,0,None) + tdSql.checkData(0,1,3) + # tdSql.query("select last_row(c1),t1 from stb1 ") + # tdSql.checkData(0,0,None) + # tdSql.checkData(0,1,3) + tdSql.query("select last_row(c1,t1),last(t1) from stb1 ") + tdSql.checkData(0,0,None) + tdSql.checkData(0,1,3) + tdSql.checkData(0,2,3) + + tdSql.query("select last_row(t1) from stb1 where t1 >0 ") + tdSql.checkRows(1) + tdSql.checkData(0,0,3) + tdSql.query("select last_row(t1) from stb1 where t1 =3 ") + tdSql.checkRows(1) + tdSql.checkData(0,0,3) + + tdSql.query("select last_row(t1) from stb1 where t1 =2") + tdSql.checkRows(0) + + # nest query for last_row + # tdSql.query("select last_row(t1) from (select c1 ,t1 from stb1)") + # tdSql.checkData(0,0,61) + # tdSql.query("select distinct(c1) ,t1 from stb1") + # tdSql.checkRows(20) + tdSql.query("select last_row(c1) from (select _rowts , c1 ,t1 from stb1)") + tdSql.checkData(0,0,None) + + tdSql.query("select last_row(c1) from (select ts , c1 ,t1 from stb1)") + tdSql.checkData(0,0,None) + + tdSql.query("select ts , last_row(c1) ,c1 from (select ts , c1 ,t1 from stb1)") + tdSql.checkData(0,1,None,None) + + tdSql.query("select ts , last_row(c1) ,c1 from (select ts , max(c1) c1 ,t1 from stb1 where ts >now -1h and ts now -1h and ts now -1h and ts ="2022-07-06 16:00:00.000 " and ts < "2022-07-06 17:00:00.000 " interval(50s) sliding(30s) fill(NULL)') + # tdSql.checkRows(40) + # tdSql.checkData(0,0,None) + tdSql.query('select max(c1) from stb1 where ts>="2022-07-06 16:00:00.000 " and ts < "2022-07-06 17:00:00.000 " interval(50s) sliding(30s)') + tdSql.checkRows(5) + + + + def support_super_table_test(self): + tdSql.execute(" use testdb ") + self.check_result_auto( " select c1 from stb1 order by ts " , "select abs(c1) from stb1 order by ts" ) + self.check_result_auto( " select c1 from stb1 order by tbname " , "select abs(c1) from stb1 order by tbname" ) + self.check_result_auto( " select c1 from stb1 where c1 > 0 order by tbname " , "select abs(c1) from stb1 where c1 > 0 order by tbname" ) + self.check_result_auto( " select c1 from stb1 where c1 > 0 order by tbname " , "select abs(c1) from stb1 where c1 > 0 order by tbname" ) + + self.check_result_auto( " select t1,c1 from stb1 order by ts " , "select t1, abs(c1) from stb1 order by ts" ) + self.check_result_auto( " select t2,c1 from stb1 order by tbname " , "select t2 ,abs(c1) from stb1 order by tbname" ) + self.check_result_auto( " select t3,c1 from stb1 where c1 > 0 order by tbname " , "select t3 ,abs(c1) from stb1 where c1 > 0 order by tbname" ) + self.check_result_auto( " select t4,c1 from stb1 where c1 > 0 order by tbname " , "select t4 , abs(c1) from stb1 where c1 > 0 order by tbname" ) + pass + + + def run(self): # sourcery skip: extract-duplicate-method, remove-redundant-fstring + # tdSql.prepare() + + tdLog.printNoPrefix("==========step1:create table ==============") + + self.prepare_datas() + self.prepare_tag_datas() + + tdLog.printNoPrefix("==========step2:test errors ==============") + + self.test_errors() + + tdLog.printNoPrefix("==========step3:support types ============") + + self.support_types() + + tdLog.printNoPrefix("==========step4: abs basic query ============") + + self.basic_abs_function() + + tdLog.printNoPrefix("==========step5: abs boundary query ============") + + self.check_boundary_values() + + tdLog.printNoPrefix("==========step6: abs filter query ============") + + self.abs_func_filter() + + tdLog.printNoPrefix("==========step6: tag coumpute query ============") + + self.test_tag_compute_for_scalar_function() + + tdLog.printNoPrefix("==========step7: check result of query ============") + + self.insert_datas_and_check_abs(self.tb_nums,self.row_nums,self.time_step) + + tdLog.printNoPrefix("==========step8: check abs result of stable query ============") + + self.support_super_table_test() + + def stop(self): + tdSql.close() + tdLog.success(f"{__file__} successfully executed") + + +tdCases.addLinux(__file__, TDTestCase()) +tdCases.addWindows(__file__, TDTestCase()) diff --git a/tests/system-test/2-query/max_partition.py b/tests/system-test/2-query/max_partition.py index 90b8d25cb121e2ffd0c8f9007c9a3100f354e5b4..cf0d6639c2c5dc7aaf22faf9205a954fe20fedf7 100644 --- a/tests/system-test/2-query/max_partition.py +++ b/tests/system-test/2-query/max_partition.py @@ -45,8 +45,8 @@ class TDTestCase: tdSql.query(" select max(c1) from stb group by tbname order by tbname ") tdSql.checkRows(self.tb_nums) # bug need fix - # tdSql.query(" select max(t1) from stb group by t2 order by t2 ") - # tdSql.checkRows(self.tb_nums) + tdSql.query(" select max(t2) from stb group by t2 order by t2 ") + tdSql.checkRows(self.tb_nums) tdSql.query(" select max(c1) from stb group by c1 order by c1 ") tdSql.checkRows(self.row_nums+1) @@ -90,8 +90,8 @@ class TDTestCase: tdSql.query("select tbname , max(t2) from stb partition by t2 order by t2") # # bug need fix - # tdSql.query("select t2 , max(t2) from stb partition by t2 order by t2") - # tdSql.checkRows(self.tb_nums) + tdSql.query("select t2 , max(t2) from stb partition by t2 order by t2") + tdSql.checkRows(self.tb_nums) tdSql.query("select tbname , max(c1) from stb partition by tbname order by tbname") tdSql.checkRows(self.tb_nums) @@ -126,8 +126,8 @@ class TDTestCase: tdSql.checkData(0,0,self.row_nums) # bug need fix - # tdSql.query("select count(c1) , max(t1) ,abs(c1) from stb partition by abs(c1) order by abs(c1)") - # tdSql.checkRows(self.row_nums+1) + tdSql.query("select count(c1) , max(t2) ,abs(c1) from stb partition by abs(c1) order by abs(c1)") + tdSql.checkRows(self.row_nums+1) tdSql.query("select max(ceil(c2)) , max(floor(t2)) ,max(floor(c2)) from stb partition by abs(c2) order by abs(c2)") @@ -136,6 +136,18 @@ class TDTestCase: tdSql.query("select max(ceil(c1-2)) , max(floor(t2+1)) ,max(c2-c1) from stb partition by abs(floor(c1)) order by abs(floor(c1))") tdSql.checkRows(self.row_nums+1) + + tdSql.query("select tbname , max(c1) ,c1 from stb partition by tbname order by tbname") + tdSql.checkRows(self.tb_nums) + tdSql.checkData(0,0,'sub_stb_0') + tdSql.checkData(0,1,9) + tdSql.checkData(0,2,9) + + tdSql.query("select tbname ,top(c1,1) ,c1 from stb partition by tbname order by tbname") + tdSql.checkRows(self.tb_nums) + + tdSql.query(" select c1 , sample(c1,2) from stb partition by tbname order by tbname ") + tdSql.checkRows(self.tb_nums*2) # interval diff --git a/tests/system-test/2-query/sample.py b/tests/system-test/2-query/sample.py index f583b7dd783df58f9784e732a69ba7c40cf1c4b2..84ff5a0056edbc3148a17e863e2d45c71693e010 100644 --- a/tests/system-test/2-query/sample.py +++ b/tests/system-test/2-query/sample.py @@ -31,6 +31,7 @@ class TDTestCase: def init(self, conn, logSql): tdLog.debug("start to execute %s" % __file__) tdSql.init(conn.cursor()) + self.ts = 1537146000000 def sample_query_form(self, sel="select", func="sample(", col="c1", m_comm =",", k=1,r_comm=")", alias="", fr="from",table_expr="t1", condition=""): ''' @@ -624,10 +625,11 @@ class TDTestCase: tdLog.info(" sample data is in datas groups ,successed sql is : %s" % sample_query ) else: tdLog.exit(" sample data is not in datas groups ,failed sql is : %s" % sample_query ) - + + def basic_sample_query(self): tdSql.execute(" drop database if exists db ") - tdSql.execute(" create database if not exists db duration 300 ") + tdSql.execute(" create database if not exists db duration 300d ") tdSql.execute(" use db ") tdSql.execute( '''create table stb1 @@ -759,14 +761,6 @@ class TDTestCase: self.check_sample("select sample( c1 ,3 ) from t1 where c1 between 1 and 10" ,"select c1 from t1 where c1 between 1 and 10") - tdSql.query("select sample(c1,2) ,c2,c3 ,c5 from stb1") - tdSql.checkRows(2) - tdSql.checkCols(4) - - self.check_sample("select sample( c1 ,3 ),c2,c3,c4,c5 from t1 where c1 between 1 and 10" ,"select c1,c2,c3,c4,c5 from t1 where c1 between 1 and 10") - self.check_sample("select sample( c1 ,3 ),c2,c3,c4,c5 from stb1 where c1 between 1 and 10" ,"select c1,c2,c3,c4,c5 from stb1 where c1 between 1 and 10") - self.check_sample("select sample( c1 ,3 ),t1 from stb1 where c1 between 1 and 10" ,"select c1,t1 from stb1 where c1 between 1 and 10") - # join tdSql.query("select sample( ct4.c1 , 1 ) from ct1, ct4 where ct4.ts=ct1.ts") @@ -779,8 +773,8 @@ class TDTestCase: self.check_sample("select sample(c1,2) from stb1 partition by tbname" , "select c1 from stb1 partition by tbname") # nest query - tdSql.query("select sample(c1,2) from (select c1 from t1); ") - tdSql.checkRows(2) + # tdSql.query("select sample(c1,2) from (select c1 from t1); ") + # tdSql.checkRows(2) # union all tdSql.query("select sample(c1,2) from t1 union all select sample(c1,3) from t1") @@ -798,36 +792,6 @@ class TDTestCase: tdSql.query("select sample(c1,100)+2 from ct1") tdSql.query("select abs(sample(c1,100)) from ct1") - # support stable and tbname - tdSql.query("select tbname ,sample(c1,2) from stb1 partition by tbname order by tbname") - tdSql.checkRows(4) - tdSql.checkData(0,0,'ct1') - tdSql.checkData(3,0,'ct4') - - # # bug need fix - # tdSql.query(" select tbname ,c1 ,t1, sample(c1,2) from stb1 partition by tbname order by tbname ") - # tdSql.checkRows(4) - # tdSql.checkData(0,0,'ct1') - # tdSql.checkData(3,0,'ct4') - # tdSql.checkData(0,2,1) - # tdSql.checkData(3,2,4) - - tdSql.query(" select tbname ,c1 ,t1, sample(c1,2) from stb1 partition by t1 order by t1 ") - tdSql.checkRows(4) - tdSql.checkData(0,0,'ct1') - tdSql.checkData(3,0,'ct4') - tdSql.checkData(0,2,1) - tdSql.checkData(3,2,4) - - # bug need fix - # tdSql.query(" select tbname ,c1 ,t1, sample(c1,2) from stb1 partition by c1 order by c1 ") - # tdSql.checkRows(21) - - # bug need fix - # tdSql.query(" select sample(c1,2) from stb1 partition by c1 ") - # tdSql.checkRows(21) - - def sample_test_run(self) : tdLog.printNoPrefix("==========support sample function==========") tbnum = 10 @@ -891,11 +855,29 @@ class TDTestCase: self.basic_sample_query() + def sample_big_data(self): + tdSql.execute("create database sample_db") + tdSql.execute("use sample_db") + tdSql.execute("create stable st (ts timestamp ,c1 int ) tags(ind int)" ) + tdSql.execute("create table sub_tb using st tags(1)") + + for i in range(2000): + ts = self.ts+i*10 + tdSql.execute(f"insert into sub_tb values({ts} ,{i})") + + tdSql.query("select count(*) from st") + tdSql.checkData(0,0,2000) + tdSql.query("select sample(c1 ,1000) from st") + tdSql.checkRows(1000) + + + def run(self): import traceback try: # run in develop branch self.sample_test_run() + self.sample_big_data() pass except Exception as e: traceback.print_exc()