提交 d0e83605 编写于 作者: S Shengliang Guan

Merge remote-tracking branch 'origin/3.0' into fix/tsim

......@@ -64,6 +64,7 @@ extern int32_t tsNumOfMnodeQueryThreads;
extern int32_t tsNumOfMnodeFetchThreads;
extern int32_t tsNumOfMnodeReadThreads;
extern int32_t tsNumOfVnodeQueryThreads;
extern int32_t tsNumOfVnodeStreamThreads;
extern int32_t tsNumOfVnodeFetchThreads;
extern int32_t tsNumOfVnodeWriteThreads;
extern int32_t tsNumOfVnodeSyncThreads;
......
......@@ -34,6 +34,7 @@ typedef enum {
WRITE_QUEUE,
APPLY_QUEUE,
SYNC_QUEUE,
STREAM_QUEUE,
QUEUE_MAX,
} EQueueType;
......
......@@ -186,9 +186,12 @@ int32_t qStreamExtractOffset(qTaskInfo_t tinfo, STqOffsetVal* pOffset);
void* qStreamExtractMetaMsg(qTaskInfo_t tinfo);
void* qExtractReaderFromStreamScanner(void* scanner);
void* qExtractReaderFromStreamScanner(void* scanner);
int32_t qExtractStreamScanner(qTaskInfo_t tinfo, void** scanner);
int32_t qStreamInput(qTaskInfo_t tinfo, void* pItem);
#ifdef __cplusplus
}
#endif
......
......@@ -13,6 +13,7 @@
* along with this program. If not, see <http://www.gnu.org/licenses/>.
*/
#include "executor.h"
#include "os.h"
#include "query.h"
#include "tdatablock.h"
......@@ -120,7 +121,6 @@ static FORCE_INLINE void* streamQueueCurItem(SStreamQueue* queue) { return queue
static FORCE_INLINE void* streamQueueNextItem(SStreamQueue* queue) {
int8_t dequeueFlag = atomic_exchange_8(&queue->status, STREAM_QUEUE__PROCESSING);
if (dequeueFlag == STREAM_QUEUE__FAILED) {
ASSERT(0);
ASSERT(queue->qItem != NULL);
return streamQueueCurItem(queue);
} else {
......@@ -307,14 +307,18 @@ static FORCE_INLINE int32_t streamTaskInput(SStreamTask* pTask, SStreamQueueItem
atomic_store_8(&pTask->inputStatus, TASK_INPUT_STATUS__FAILED);
return -1;
}
qInfo("task %d %p submit enqueue %p %p %p", pTask->taskId, pTask, pItem, pSubmitClone, pSubmitClone->data);
qDebug("task %d %p submit enqueue %p %p %p", pTask->taskId, pTask, pItem, pSubmitClone, pSubmitClone->data);
taosWriteQitem(pTask->inputQueue->queue, pSubmitClone);
// qStreamInput(pTask->exec.executor, pSubmitClone);
} else if (pItem->type == STREAM_INPUT__DATA_BLOCK || pItem->type == STREAM_INPUT__DATA_RETRIEVE) {
taosWriteQitem(pTask->inputQueue->queue, pItem);
// qStreamInput(pTask->exec.executor, pItem);
} else if (pItem->type == STREAM_INPUT__CHECKPOINT) {
taosWriteQitem(pTask->inputQueue->queue, pItem);
// qStreamInput(pTask->exec.executor, pItem);
} else if (pItem->type == STREAM_INPUT__TRIGGER) {
taosWriteQitem(pTask->inputQueue->queue, pItem);
// qStreamInput(pTask->exec.executor, pItem);
}
if (pItem->type != STREAM_INPUT__TRIGGER && pItem->type != STREAM_INPUT__CHECKPOINT && pTask->triggerParam != 0) {
......
......@@ -26,6 +26,8 @@ extern "C" {
extern bool gRaftDetailLog;
#define SYNC_RESP_TTL_MS 5000
#define SYNC_MAX_BATCH_SIZE 500
#define SYNC_INDEX_BEGIN 0
#define SYNC_INDEX_INVALID -1
......
......@@ -55,6 +55,7 @@ int32_t tsNumOfMnodeQueryThreads = 2;
int32_t tsNumOfMnodeFetchThreads = 1;
int32_t tsNumOfMnodeReadThreads = 1;
int32_t tsNumOfVnodeQueryThreads = 2;
int32_t tsNumOfVnodeStreamThreads = 2;
int32_t tsNumOfVnodeFetchThreads = 4;
int32_t tsNumOfVnodeWriteThreads = 2;
int32_t tsNumOfVnodeSyncThreads = 2;
......@@ -412,6 +413,10 @@ static int32_t taosAddServerCfg(SConfig *pCfg) {
tsNumOfVnodeQueryThreads = TMAX(tsNumOfVnodeQueryThreads, 2);
if (cfgAddInt32(pCfg, "numOfVnodeQueryThreads", tsNumOfVnodeQueryThreads, 1, 1024, 0) != 0) return -1;
tsNumOfVnodeStreamThreads = tsNumOfCores / 4;
tsNumOfVnodeStreamThreads = TMAX(tsNumOfVnodeStreamThreads, 4);
if (cfgAddInt32(pCfg, "numOfVnodeStreamThreads", tsNumOfVnodeStreamThreads, 1, 1024, 0) != 0) return -1;
tsNumOfVnodeFetchThreads = tsNumOfCores / 4;
tsNumOfVnodeFetchThreads = TMAX(tsNumOfVnodeFetchThreads, 4);
if (cfgAddInt32(pCfg, "numOfVnodeFetchThreads", tsNumOfVnodeFetchThreads, 1, 1024, 0) != 0) return -1;
......@@ -587,6 +592,7 @@ static int32_t taosSetServerCfg(SConfig *pCfg) {
tsNumOfMnodeQueryThreads = cfgGetItem(pCfg, "numOfMnodeQueryThreads")->i32;
tsNumOfMnodeReadThreads = cfgGetItem(pCfg, "numOfMnodeReadThreads")->i32;
tsNumOfVnodeQueryThreads = cfgGetItem(pCfg, "numOfVnodeQueryThreads")->i32;
tsNumOfVnodeStreamThreads = cfgGetItem(pCfg, "numOfVnodeStreamThreads")->i32;
tsNumOfVnodeFetchThreads = cfgGetItem(pCfg, "numOfVnodeFetchThreads")->i32;
tsNumOfVnodeWriteThreads = cfgGetItem(pCfg, "numOfVnodeWriteThreads")->i32;
tsNumOfVnodeSyncThreads = cfgGetItem(pCfg, "numOfVnodeSyncThreads")->i32;
......
......@@ -31,6 +31,7 @@ typedef struct SVnodeMgmt {
const char *path;
const char *name;
SQWorkerPool queryPool;
SQWorkerPool streamPool;
SWWorkerPool fetchPool;
SWWorkerPool syncPool;
SWWorkerPool writePool;
......@@ -61,6 +62,7 @@ typedef struct {
STaosQueue *pSyncQ;
STaosQueue *pApplyQ;
STaosQueue *pQueryQ;
STaosQueue *pStreamQ;
STaosQueue *pFetchQ;
} SVnodeObj;
......@@ -105,6 +107,7 @@ int32_t vmPutMsgToWriteQueue(SVnodeMgmt *pMgmt, SRpcMsg *pMsg);
int32_t vmPutMsgToSyncQueue(SVnodeMgmt *pMgmt, SRpcMsg *pMsg);
int32_t vmPutMsgToQueryQueue(SVnodeMgmt *pMgmt, SRpcMsg *pMsg);
int32_t vmPutMsgToFetchQueue(SVnodeMgmt *pMgmt, SRpcMsg *pMsg);
int32_t vmPutMsgToStreamQueue(SVnodeMgmt *pMgmt, SRpcMsg *pMsg);
int32_t vmPutMsgToMergeQueue(SVnodeMgmt *pMgmt, SRpcMsg *pMsg);
int32_t vmPutMsgToMgmtQueue(SVnodeMgmt *pMgmt, SRpcMsg *pMsg);
int32_t vmPutMsgToMonitorQueue(SVnodeMgmt *pMgmt, SRpcMsg *pMsg);
......
......@@ -357,16 +357,16 @@ SArray *vmGetMsgHandles() {
if (dmSetMgmtHandle(pArray, TDMT_VND_COMMIT, vmPutMsgToWriteQueue, 0) == NULL) goto _OVER;
if (dmSetMgmtHandle(pArray, TDMT_SCH_QUERY_HEARTBEAT, vmPutMsgToFetchQueue, 0) == NULL) goto _OVER;
if (dmSetMgmtHandle(pArray, TDMT_VND_STREAM_TRIGGER, vmPutMsgToFetchQueue, 0) == NULL) goto _OVER;
if (dmSetMgmtHandle(pArray, TDMT_VND_STREAM_TRIGGER, vmPutMsgToStreamQueue, 0) == NULL) goto _OVER;
if (dmSetMgmtHandle(pArray, TDMT_STREAM_TASK_DROP, vmPutMsgToWriteQueue, 0) == NULL) goto _OVER;
if (dmSetMgmtHandle(pArray, TDMT_STREAM_TASK_DEPLOY, vmPutMsgToWriteQueue, 0) == NULL) goto _OVER;
if (dmSetMgmtHandle(pArray, TDMT_STREAM_TASK_RUN, vmPutMsgToFetchQueue, 0) == NULL) goto _OVER;
if (dmSetMgmtHandle(pArray, TDMT_STREAM_TASK_DISPATCH, vmPutMsgToFetchQueue, 0) == NULL) goto _OVER;
if (dmSetMgmtHandle(pArray, TDMT_STREAM_TASK_DISPATCH_RSP, vmPutMsgToFetchQueue, 0) == NULL) goto _OVER;
if (dmSetMgmtHandle(pArray, TDMT_STREAM_TASK_RECOVER, vmPutMsgToFetchQueue, 0) == NULL) goto _OVER;
if (dmSetMgmtHandle(pArray, TDMT_STREAM_TASK_RECOVER_RSP, vmPutMsgToFetchQueue, 0) == NULL) goto _OVER;
if (dmSetMgmtHandle(pArray, TDMT_STREAM_RETRIEVE, vmPutMsgToFetchQueue, 0) == NULL) goto _OVER;
if (dmSetMgmtHandle(pArray, TDMT_STREAM_RETRIEVE_RSP, vmPutMsgToFetchQueue, 0) == NULL) goto _OVER;
if (dmSetMgmtHandle(pArray, TDMT_STREAM_TASK_RUN, vmPutMsgToStreamQueue, 0) == NULL) goto _OVER;
if (dmSetMgmtHandle(pArray, TDMT_STREAM_TASK_DISPATCH, vmPutMsgToStreamQueue, 0) == NULL) goto _OVER;
if (dmSetMgmtHandle(pArray, TDMT_STREAM_TASK_DISPATCH_RSP, vmPutMsgToStreamQueue, 0) == NULL) goto _OVER;
if (dmSetMgmtHandle(pArray, TDMT_STREAM_TASK_RECOVER, vmPutMsgToStreamQueue, 0) == NULL) goto _OVER;
if (dmSetMgmtHandle(pArray, TDMT_STREAM_TASK_RECOVER_RSP, vmPutMsgToStreamQueue, 0) == NULL) goto _OVER;
if (dmSetMgmtHandle(pArray, TDMT_STREAM_RETRIEVE, vmPutMsgToStreamQueue, 0) == NULL) goto _OVER;
if (dmSetMgmtHandle(pArray, TDMT_STREAM_RETRIEVE_RSP, vmPutMsgToStreamQueue, 0) == NULL) goto _OVER;
if (dmSetMgmtHandle(pArray, TDMT_VND_ALTER_REPLICA, vmPutMsgToWriteQueue, 0) == NULL) goto _OVER;
if (dmSetMgmtHandle(pArray, TDMT_VND_ALTER_CONFIG, vmPutMsgToWriteQueue, 0) == NULL) goto _OVER;
......
......@@ -92,6 +92,7 @@ void vmCloseVnode(SVnodeMgmt *pMgmt, SVnodeObj *pVnode) {
while (!taosQueueEmpty(pVnode->pApplyQ)) taosMsleep(10);
while (!taosQueueEmpty(pVnode->pQueryQ)) taosMsleep(10);
while (!taosQueueEmpty(pVnode->pFetchQ)) taosMsleep(10);
while (!taosQueueEmpty(pVnode->pStreamQ)) taosMsleep(10);
dTrace("vgId:%d, vnode queue is empty", pVnode->vgId);
vmFreeQueue(pMgmt, pVnode);
......
......@@ -81,6 +81,23 @@ static void vmProcessQueryQueue(SQueueInfo *pInfo, SRpcMsg *pMsg) {
taosFreeQitem(pMsg);
}
static void vmProcessStreamQueue(SQueueInfo *pInfo, SRpcMsg *pMsg) {
SVnodeObj *pVnode = pInfo->ahandle;
const STraceId *trace = &pMsg->info.traceId;
dGTrace("vgId:%d, msg:%p get from vnode-stream queue", pVnode->vgId, pMsg);
int32_t code = vnodeProcessFetchMsg(pVnode->pImpl, pMsg, pInfo);
if (code != 0) {
if (terrno != 0) code = terrno;
dGError("vgId:%d, msg:%p failed to stream since %s", pVnode->vgId, pMsg, terrstr());
vmSendRsp(pMsg, code);
}
dGTrace("vgId:%d, msg:%p is freed, code:0x%x", pVnode->vgId, pMsg, code);
rpcFreeCont(pMsg->pCont);
taosFreeQitem(pMsg);
}
static void vmProcessFetchQueue(SQueueInfo *pInfo, STaosQall *qall, int32_t numOfMsgs) {
SVnodeObj *pVnode = pInfo->ahandle;
SRpcMsg *pMsg = NULL;
......@@ -140,6 +157,10 @@ static int32_t vmPutMsgToQueue(SVnodeMgmt *pMgmt, SRpcMsg *pMsg, EQueueType qtyp
dGTrace("vgId:%d, msg:%p put into vnode-query queue", pVnode->vgId, pMsg);
taosWriteQitem(pVnode->pQueryQ, pMsg);
break;
case STREAM_QUEUE:
dGTrace("vgId:%d, msg:%p put into vnode-stream queue", pVnode->vgId, pMsg);
taosWriteQitem(pVnode->pStreamQ, pMsg);
break;
case FETCH_QUEUE:
dGTrace("vgId:%d, msg:%p put into vnode-fetch queue", pVnode->vgId, pMsg);
taosWriteQitem(pVnode->pFetchQ, pMsg);
......@@ -174,6 +195,8 @@ int32_t vmPutMsgToQueryQueue(SVnodeMgmt *pMgmt, SRpcMsg *pMsg) { return vmPutMsg
int32_t vmPutMsgToFetchQueue(SVnodeMgmt *pMgmt, SRpcMsg *pMsg) { return vmPutMsgToQueue(pMgmt, pMsg, FETCH_QUEUE); }
int32_t vmPutMsgToStreamQueue(SVnodeMgmt *pMgmt, SRpcMsg *pMsg) { return vmPutMsgToQueue(pMgmt, pMsg, STREAM_QUEUE); }
int32_t vmPutMsgToMgmtQueue(SVnodeMgmt *pMgmt, SRpcMsg *pMsg) {
const STraceId *trace = &pMsg->info.traceId;
dGTrace("msg:%p, put into vnode-mgmt queue", pMsg);
......@@ -234,6 +257,9 @@ int32_t vmGetQueueSize(SVnodeMgmt *pMgmt, int32_t vgId, EQueueType qtype) {
case FETCH_QUEUE:
size = taosQueueItemSize(pVnode->pFetchQ);
break;
case STREAM_QUEUE:
size = taosQueueItemSize(pVnode->pStreamQ);
break;
default:
break;
}
......@@ -247,10 +273,11 @@ int32_t vmAllocQueue(SVnodeMgmt *pMgmt, SVnodeObj *pVnode) {
pVnode->pSyncQ = tWWorkerAllocQueue(&pMgmt->syncPool, pVnode, (FItems)vmProcessSyncQueue);
pVnode->pApplyQ = tWWorkerAllocQueue(&pMgmt->applyPool, pVnode->pImpl, (FItems)vnodeApplyWriteMsg);
pVnode->pQueryQ = tQWorkerAllocQueue(&pMgmt->queryPool, pVnode, (FItem)vmProcessQueryQueue);
pVnode->pStreamQ = tQWorkerAllocQueue(&pMgmt->streamPool, pVnode, (FItem)vmProcessStreamQueue);
pVnode->pFetchQ = tWWorkerAllocQueue(&pMgmt->fetchPool, pVnode, (FItems)vmProcessFetchQueue);
if (pVnode->pWriteQ == NULL || pVnode->pSyncQ == NULL || pVnode->pApplyQ == NULL || pVnode->pQueryQ == NULL ||
pVnode->pFetchQ == NULL) {
pVnode->pStreamQ == NULL || pVnode->pFetchQ == NULL) {
terrno = TSDB_CODE_OUT_OF_MEMORY;
return -1;
}
......@@ -259,6 +286,7 @@ int32_t vmAllocQueue(SVnodeMgmt *pMgmt, SVnodeObj *pVnode) {
dDebug("vgId:%d, sync-queue:%p is alloced", pVnode->vgId, pVnode->pSyncQ);
dDebug("vgId:%d, apply-queue:%p is alloced", pVnode->vgId, pVnode->pApplyQ);
dDebug("vgId:%d, query-queue:%p is alloced", pVnode->vgId, pVnode->pQueryQ);
dDebug("vgId:%d, stream-queue:%p is alloced", pVnode->vgId, pVnode->pStreamQ);
dDebug("vgId:%d, fetch-queue:%p is alloced", pVnode->vgId, pVnode->pFetchQ);
return 0;
}
......@@ -268,11 +296,13 @@ void vmFreeQueue(SVnodeMgmt *pMgmt, SVnodeObj *pVnode) {
tWWorkerFreeQueue(&pMgmt->applyPool, pVnode->pApplyQ);
tWWorkerFreeQueue(&pMgmt->syncPool, pVnode->pSyncQ);
tQWorkerFreeQueue(&pMgmt->queryPool, pVnode->pQueryQ);
tQWorkerFreeQueue(&pMgmt->streamPool, pVnode->pStreamQ);
tWWorkerFreeQueue(&pMgmt->fetchPool, pVnode->pFetchQ);
pVnode->pWriteQ = NULL;
pVnode->pSyncQ = NULL;
pVnode->pApplyQ = NULL;
pVnode->pQueryQ = NULL;
pVnode->pStreamQ = NULL;
pVnode->pFetchQ = NULL;
dDebug("vgId:%d, queue is freed", pVnode->vgId);
}
......@@ -284,6 +314,12 @@ int32_t vmStartWorker(SVnodeMgmt *pMgmt) {
pQPool->max = tsNumOfVnodeQueryThreads;
if (tQWorkerInit(pQPool) != 0) return -1;
SQWorkerPool *pStreamPool = &pMgmt->streamPool;
pStreamPool->name = "vnode-stream";
pStreamPool->min = tsNumOfVnodeStreamThreads;
pStreamPool->max = tsNumOfVnodeStreamThreads;
if (tQWorkerInit(pStreamPool) != 0) return -1;
SWWorkerPool *pFPool = &pMgmt->fetchPool;
pFPool->name = "vnode-fetch";
pFPool->max = tsNumOfVnodeFetchThreads;
......@@ -333,6 +369,7 @@ void vmStopWorker(SVnodeMgmt *pMgmt) {
tWWorkerCleanup(&pMgmt->applyPool);
tWWorkerCleanup(&pMgmt->syncPool);
tQWorkerCleanup(&pMgmt->queryPool);
tQWorkerCleanup(&pMgmt->streamPool);
tWWorkerCleanup(&pMgmt->fetchPool);
dDebug("vnode workers are closed");
}
......@@ -218,6 +218,7 @@ bool mndIsMnode(SMnode *pMnode, int32_t dnodeId) {
}
void mndGetMnodeEpSet(SMnode *pMnode, SEpSet *pEpSet) {
#if 0
SSdb *pSdb = pMnode->pSdb;
int32_t totalMnodes = sdbGetSize(pSdb, SDB_MNODE);
void *pIter = NULL;
......@@ -237,6 +238,9 @@ void mndGetMnodeEpSet(SMnode *pMnode, SEpSet *pEpSet) {
addEpIntoEpSet(pEpSet, pObj->pDnode->fqdn, pObj->pDnode->port);
sdbRelease(pSdb, pObj);
}
#else
syncGetRetryEpSet(pMnode->syncMgmt.sync, pEpSet);
#endif
}
static int32_t mndSetCreateMnodeRedoLogs(SMnode *pMnode, STrans *pTrans, SMnodeObj *pObj) {
......
......@@ -22,11 +22,11 @@ typedef struct SLastrowReader {
SVnode* pVnode;
STSchema* pSchema;
uint64_t uid;
char** transferBuf; // todo remove it soon
int32_t numOfCols;
int32_t type;
int32_t tableIndex; // currently returned result tables
SArray* pTableList; // table id list
char** transferBuf; // todo remove it soon
int32_t numOfCols;
int32_t type;
int32_t tableIndex; // currently returned result tables
SArray* pTableList; // table id list
} SLastrowReader;
static void saveOneRow(STSRow* pRow, SSDataBlock* pBlock, SLastrowReader* pReader, const int32_t* slotIds) {
......@@ -94,12 +94,15 @@ int32_t tsdbLastRowReaderOpen(void* pVnode, int32_t type, SArray* pTableIdList,
int32_t tsdbLastrowReaderClose(void* pReader) {
SLastrowReader* p = pReader;
for (int32_t i = 0; i < p->pSchema->numOfCols; ++i) {
taosMemoryFreeClear(p->transferBuf[i]);
if (p->pSchema != NULL) {
for (int32_t i = 0; i < p->pSchema->numOfCols; ++i) {
taosMemoryFreeClear(p->transferBuf[i]);
}
taosMemoryFree(p->transferBuf);
taosMemoryFree(p->pSchema);
}
taosMemoryFree(p->pSchema);
taosMemoryFree(p->transferBuf);
taosMemoryFree(pReader);
return TSDB_CODE_SUCCESS;
}
......
......@@ -427,13 +427,22 @@ static void vnodeSyncCommitMsg(SSyncFSM *pFsm, const SRpcMsg *pMsg, SFsmCbMeta c
syncGetVgId(pVnode->sync), pFsm, cbMeta.index, cbMeta.isWeak, cbMeta.code, cbMeta.state,
syncUtilState2String(cbMeta.state), pMsg->msgType, TMSG_INFO(pMsg->msgType));
SRpcMsg rpcMsg = {.msgType = pMsg->msgType, .contLen = pMsg->contLen};
rpcMsg.pCont = rpcMallocCont(rpcMsg.contLen);
memcpy(rpcMsg.pCont, pMsg->pCont, pMsg->contLen);
syncGetAndDelRespRpc(pVnode->sync, cbMeta.seqNum, &rpcMsg.info);
rpcMsg.info.conn.applyIndex = cbMeta.index;
rpcMsg.info.conn.applyTerm = cbMeta.term;
tmsgPutToQueue(&pVnode->msgCb, APPLY_QUEUE, &rpcMsg);
if (cbMeta.code == 0) {
SRpcMsg rpcMsg = {.msgType = pMsg->msgType, .contLen = pMsg->contLen};
rpcMsg.pCont = rpcMallocCont(rpcMsg.contLen);
memcpy(rpcMsg.pCont, pMsg->pCont, pMsg->contLen);
syncGetAndDelRespRpc(pVnode->sync, cbMeta.seqNum, &rpcMsg.info);
rpcMsg.info.conn.applyIndex = cbMeta.index;
rpcMsg.info.conn.applyTerm = cbMeta.term;
tmsgPutToQueue(&pVnode->msgCb, APPLY_QUEUE, &rpcMsg);
} else {
SRpcMsg rsp = {.code = cbMeta.code, .info = pMsg->info};
vError("vgId:%d, sync commit error, msgtype:%d,%s, error:0x%X, errmsg:%s", syncGetVgId(pVnode->sync), pMsg->msgType,
TMSG_INFO(pMsg->msgType), cbMeta.code, tstrerror(cbMeta.code));
if (rsp.info.handle != NULL) {
tmsgSendRsp(&rsp);
}
}
}
static void vnodeSyncPreCommitMsg(SSyncFSM *pFsm, const SRpcMsg *pMsg, SFsmCbMeta cbMeta) {
......
......@@ -8,7 +8,7 @@ add_library(executor STATIC ${EXECUTOR_SRC})
# )
target_link_libraries(executor
PRIVATE os util common function parser planner qcom vnode scalar nodes index
PRIVATE os util common function parser planner qcom vnode scalar nodes index stream
)
target_include_directories(
......
......@@ -39,6 +39,7 @@ extern "C" {
#include "tmsg.h"
#include "tpagedbuf.h"
#include "tstreamUpdate.h"
#include "tstream.h"
#include "vnode.h"
#include "executorInt.h"
......@@ -139,12 +140,14 @@ typedef struct STaskIdInfo {
} STaskIdInfo;
typedef struct {
//TODO remove prepareStatus
STqOffsetVal prepareStatus; // for tmq
STqOffsetVal lastStatus; // for tmq
void* metaBlk; // for tmq fetching meta
SSDataBlock* pullOverBlk; // for streaming
SWalFilterCond cond;
int64_t lastScanUid;
SStreamQueue* inputQueue;
} SStreamTaskInfo;
typedef struct SExecTaskInfo {
......
......@@ -60,8 +60,6 @@ static int32_t doSetStreamBlock(SOperatorInfo* pOperator, void* input, size_t nu
taosArrayAddAll(p->pDataBlock, pDataBlock->pDataBlock);
taosArrayPush(pInfo->pBlockLists, &p);
}
/*} else if (type == STREAM_INPUT__TABLE_SCAN) {*/
/*ASSERT(pInfo->blockType == STREAM_INPUT__TABLE_SCAN);*/
} else {
ASSERT(0);
}
......
......@@ -44,6 +44,13 @@ int32_t qCreateExecTask(SReadHandle* readHandle, int32_t vgId, uint64_t taskId,
goto _error;
}
if (model == OPTR_EXEC_MODEL_STREAM) {
(*pTask)->streamInfo.inputQueue = streamQueueOpen();
if ((*pTask)->streamInfo.inputQueue == NULL) {
goto _error;
}
}
SDataSinkMgtCfg cfg = {.maxDataBlockNum = 1000, .maxDataBlockNumPerQuery = 100};
code = dsDataSinkMgtInit(&cfg);
if (code != TSDB_CODE_SUCCESS) {
......@@ -252,6 +259,13 @@ int32_t qExtractStreamScanner(qTaskInfo_t tinfo, void** scanner) {
}
}
int32_t qStreamInput(qTaskInfo_t tinfo, void* pItem) {
SExecTaskInfo* pTaskInfo = (SExecTaskInfo*)tinfo;
ASSERT(pTaskInfo->execModel == OPTR_EXEC_MODEL_STREAM);
taosWriteQitem(pTaskInfo->streamInfo.inputQueue->queue, pItem);
return 0;
}
void* qExtractReaderFromStreamScanner(void* scanner) {
SStreamScanInfo* pInfo = scanner;
return (void*)pInfo->tqReader;
......
......@@ -1202,15 +1202,6 @@ static int32_t setBlockIntoRes(SStreamScanInfo* pInfo, const SSDataBlock* pBlock
taosArrayDestroy(pBlock->pDataBlock);
ASSERT(pInfo->pRes->pDataBlock != NULL);
#if 0
if (pInfo->pRes->pDataBlock == NULL) {
// TODO add log
updateInfoDestoryColseWinSBF(pInfo->pUpdateInfo);
pOperator->status = OP_EXEC_DONE;
pTaskInfo->code = terrno;
return -1;
}
#endif
// currently only the tbname pseudo column
if (pInfo->numOfPseudoExpr > 0) {
......@@ -1231,11 +1222,6 @@ static SSDataBlock* doStreamScan(SOperatorInfo* pOperator) {
SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo;
SStreamScanInfo* pInfo = pOperator->info;
/*pTaskInfo->code = pOperator->fpSet._openFn(pOperator);*/
/*if (pTaskInfo->code != TSDB_CODE_SUCCESS || pOperator->status == OP_EXEC_DONE) {*/
/*return NULL;*/
/*}*/
qDebug("stream scan called");
if (pTaskInfo->streamInfo.prepareStatus.type == TMQ_OFFSET__LOG) {
while (1) {
......@@ -1425,15 +1411,6 @@ static SSDataBlock* doStreamScan(SOperatorInfo* pOperator) {
qDebug("scan rows: %d", pBlockInfo->rows);
return (pBlockInfo->rows == 0) ? NULL : pInfo->pRes;
#if 0
} else if (pInfo->blockType == STREAM_INPUT__TABLE_SCAN) {
ASSERT(0);
// check reader last status
// if not match, reset status
SSDataBlock* pResult = doTableScan(pInfo->pTableScanOp);
return pResult && pResult->info.rows > 0 ? pResult : NULL;
#endif
} else {
ASSERT(0);
return NULL;
......
......@@ -1341,9 +1341,9 @@ static int32_t closeIntervalWindow(SHashObj* pHashMap, STimeWindowAggSupp* pSup,
uint64_t groupId = *(uint64_t*)key;
ASSERT(keyLen == GET_RES_WINDOW_KEY_LEN(sizeof(TSKEY)));
TSKEY ts = *(int64_t*)((char*)key + sizeof(uint64_t));
SResultRowInfo dumyInfo;
dumyInfo.cur.pageId = -1;
STimeWindow win = getActiveTimeWindow(NULL, &dumyInfo, ts, pInterval, TSDB_ORDER_ASC);
STimeWindow win;
win.skey = ts;
win.ekey = taosTimeAdd(win.skey, pInterval->interval, pInterval->intervalUnit, pInterval->precision) - 1;
SWinRes winRe = {
.ts = win.skey,
.groupId = groupId,
......
......@@ -13,4 +13,4 @@ target_link_libraries(qworker
if(${BUILD_TEST})
ADD_SUBDIRECTORY(test)
endif(${BUILD_TEST})
\ No newline at end of file
endif(${BUILD_TEST})
......@@ -42,6 +42,9 @@ int32_t streamBroadcastToChildren(SStreamTask* pTask, const SSDataBlock* pBlock)
int32_t tEncodeStreamRetrieveReq(SEncoder* pEncoder, const SStreamRetrieveReq* pReq);
int32_t streamAppendQueueItem(SStreamQueueItem* dst, SStreamQueueItem* elem);
void streamFreeQitem(SStreamQueueItem* data);
#ifdef __cplusplus
}
#endif
......
......@@ -97,7 +97,7 @@ int32_t streamLaunchByWrite(SStreamTask* pTask, int32_t vgId) {
.pCont = pRunReq,
.contLen = sizeof(SStreamTaskRunReq),
};
tmsgPutToQueue(pTask->pMsgCb, FETCH_QUEUE, &msg);
tmsgPutToQueue(pTask->pMsgCb, STREAM_QUEUE, &msg);
}
return 0;
}
......
......@@ -97,3 +97,29 @@ void streamDataSubmitRefDec(SStreamDataSubmit* pDataSubmit) {
taosMemoryFree(pDataSubmit->dataRef);
}
}
int32_t streamAppendQueueItem(SStreamQueueItem* dst, SStreamQueueItem* elem) {
ASSERT(elem);
if (dst->type == elem->type && dst->type == STREAM_INPUT__DATA_BLOCK) {
SStreamDataBlock* pBlock = (SStreamDataBlock*)dst;
SStreamDataBlock* pBlockSrc = (SStreamDataBlock*)elem;
taosArrayAddAll(pBlock->blocks, pBlockSrc->blocks);
return 0;
} else {
return -1;
}
}
void streamFreeQitem(SStreamQueueItem* data) {
int8_t type = data->type;
if (type == STREAM_INPUT__TRIGGER) {
blockDataDestroy(((SStreamTrigger*)data)->pBlock);
taosFreeQitem(data);
} else if (type == STREAM_INPUT__DATA_BLOCK || type == STREAM_INPUT__DATA_RETRIEVE) {
taosArrayDestroyEx(((SStreamDataBlock*)data)->blocks, (FDelete)tDeleteSSDataBlock);
taosFreeQitem(data);
} else if (type == STREAM_INPUT__DATA_SUBMIT) {
streamDataSubmitRefDec((SStreamDataSubmit*)data);
taosFreeQitem(data);
}
}
......@@ -251,8 +251,8 @@ int32_t streamBuildDispatchMsg(SStreamTask* pTask, const SStreamDataBlock* data,
ASSERT(vgId > 0 || vgId == SNODE_HANDLE);
req.taskId = downstreamTaskId;
qInfo("dispatch from task %d (child id %d) to down stream task %d in vnode %d", pTask->taskId, pTask->selfChildId,
downstreamTaskId, vgId);
qDebug("dispatch from task %d (child id %d) to down stream task %d in vnode %d", pTask->taskId, pTask->selfChildId,
downstreamTaskId, vgId);
// serialize
int32_t tlen;
......@@ -298,6 +298,7 @@ int32_t streamDispatch(SStreamTask* pTask, SMsgCb* pMsgCb) {
SStreamDataBlock* pBlock = streamQueueNextItem(pTask->outputQueue);
if (pBlock == NULL) {
qDebug("stream stop dispatching since no output: task %d", pTask->taskId);
atomic_store_8(&pTask->outputStatus, TASK_OUTPUT_STATUS__NORMAL);
return 0;
}
......
......@@ -75,10 +75,35 @@ static int32_t streamTaskExecImpl(SStreamTask* pTask, void* data, SArray* pRes)
static SArray* streamExecForQall(SStreamTask* pTask, SArray* pRes) {
while (1) {
void* data = streamQueueNextItem(pTask->inputQueue);
int32_t cnt = 0;
void* data = NULL;
while (1) {
SStreamQueueItem* qItem = streamQueueNextItem(pTask->inputQueue);
if (qItem == NULL) {
qDebug("stream exec over, queue empty");
break;
}
if (data == NULL) {
data = qItem;
streamQueueProcessSuccess(pTask->inputQueue);
continue;
} else {
if (streamAppendQueueItem(data, qItem) < 0) {
streamQueueProcessFail(pTask->inputQueue);
break;
} else {
cnt++;
streamQueueProcessSuccess(pTask->inputQueue);
taosArrayDestroy(((SStreamDataBlock*)qItem)->blocks);
taosFreeQitem(qItem);
}
}
}
if (data == NULL) break;
qDebug("stream task %d exec begin, batch msg: %d", pTask->taskId, cnt);
streamTaskExecImpl(pTask, data, pRes);
qDebug("stream task %d exec end", pTask->taskId);
if (pTask->taskStatus == TASK_STATUS__DROPPING) {
taosArrayDestroyEx(pRes, (FDelete)tDeleteSSDataBlock);
......@@ -95,27 +120,16 @@ static SArray* streamExecForQall(SStreamTask* pTask, SArray* pRes) {
qRes->type = STREAM_INPUT__DATA_BLOCK;
qRes->blocks = pRes;
if (streamTaskOutput(pTask, qRes) < 0) {
streamQueueProcessFail(pTask->inputQueue);
/*streamQueueProcessFail(pTask->inputQueue);*/
taosArrayDestroyEx(pRes, (FDelete)tDeleteSSDataBlock);
taosFreeQitem(qRes);
return NULL;
}
streamQueueProcessSuccess(pTask->inputQueue);
/*streamQueueProcessSuccess(pTask->inputQueue);*/
pRes = taosArrayInit(0, sizeof(SSDataBlock));
}
int8_t type = ((SStreamQueueItem*)data)->type;
if (type == STREAM_INPUT__TRIGGER) {
blockDataDestroy(((SStreamTrigger*)data)->pBlock);
taosFreeQitem(data);
} else if (type == STREAM_INPUT__DATA_BLOCK || type == STREAM_INPUT__DATA_RETRIEVE) {
taosArrayDestroyEx(((SStreamDataBlock*)data)->blocks, (FDelete)tDeleteSSDataBlock);
taosFreeQitem(data);
} else if (type == STREAM_INPUT__DATA_SUBMIT) {
ASSERT(pTask->isDataScan);
streamDataSubmitRefDec((SStreamDataSubmit*)data);
taosFreeQitem(data);
}
streamFreeQitem(data);
}
return pRes;
}
......@@ -129,6 +143,7 @@ int32_t streamExec(SStreamTask* pTask, SMsgCb* pMsgCb) {
atomic_val_compare_exchange_8(&pTask->execStatus, TASK_EXEC_STATUS__IDLE, TASK_EXEC_STATUS__EXECUTING);
if (execStatus == TASK_EXEC_STATUS__IDLE) {
// first run
qDebug("stream exec, enter exec status");
pRes = streamExecForQall(pTask, pRes);
if (pRes == NULL) goto FAIL;
......@@ -136,11 +151,13 @@ int32_t streamExec(SStreamTask* pTask, SMsgCb* pMsgCb) {
atomic_store_8(&pTask->execStatus, TASK_EXEC_STATUS__CLOSING);
// second run, make sure inputQ and qall are cleared
qDebug("stream exec, enter closing status");
pRes = streamExecForQall(pTask, pRes);
if (pRes == NULL) goto FAIL;
taosArrayDestroyEx(pRes, (FDelete)tDeleteSSDataBlock);
atomic_store_8(&pTask->execStatus, TASK_EXEC_STATUS__IDLE);
qDebug("stream exec, return result");
return 0;
} else if (execStatus == TASK_EXEC_STATUS__CLOSING) {
continue;
......
......@@ -26,6 +26,7 @@ extern "C" {
#include "syncInt.h"
#include "syncMessage.h"
#include "taosdef.h"
#include "tskiplist.h"
typedef struct SSyncRaftEntry {
uint32_t bytes;
......@@ -58,29 +59,52 @@ void syncEntryLog(const SSyncRaftEntry* pObj);
void syncEntryLog2(char* s, const SSyncRaftEntry* pObj);
//-----------------------------------
typedef struct SRaftEntryCache {
typedef struct SRaftEntryHashCache {
SHashObj* pEntryHash;
int32_t maxCount;
int32_t currentCount;
TdThreadMutex mutex;
SSyncNode* pSyncNode;
} SRaftEntryHashCache;
SRaftEntryHashCache* raftCacheCreate(SSyncNode* pSyncNode, int32_t maxCount);
void raftCacheDestroy(SRaftEntryHashCache* pCache);
int32_t raftCachePutEntry(struct SRaftEntryHashCache* pCache, SSyncRaftEntry* pEntry);
int32_t raftCacheGetEntry(struct SRaftEntryHashCache* pCache, SyncIndex index, SSyncRaftEntry** ppEntry);
int32_t raftCacheGetEntryP(struct SRaftEntryHashCache* pCache, SyncIndex index, SSyncRaftEntry** ppEntry);
int32_t raftCacheDelEntry(struct SRaftEntryHashCache* pCache, SyncIndex index);
int32_t raftCacheGetAndDel(struct SRaftEntryHashCache* pCache, SyncIndex index, SSyncRaftEntry** ppEntry);
int32_t raftCacheClear(struct SRaftEntryHashCache* pCache);
cJSON* raftCache2Json(SRaftEntryHashCache* pObj);
char* raftCache2Str(SRaftEntryHashCache* pObj);
void raftCachePrint(SRaftEntryHashCache* pObj);
void raftCachePrint2(char* s, SRaftEntryHashCache* pObj);
void raftCacheLog(SRaftEntryHashCache* pObj);
void raftCacheLog2(char* s, SRaftEntryHashCache* pObj);
//-----------------------------------
typedef struct SRaftEntryCache {
SSkipList* pSkipList;
int32_t maxCount;
int32_t currentCount;
TdThreadMutex mutex;
SSyncNode* pSyncNode;
} SRaftEntryCache;
SRaftEntryCache* raftCacheCreate(SSyncNode* pSyncNode, int32_t maxCount);
void raftCacheDestroy(SRaftEntryCache* pCache);
int32_t raftCachePutEntry(struct SRaftEntryCache* pCache, SSyncRaftEntry* pEntry);
int32_t raftCacheGetEntry(struct SRaftEntryCache* pCache, SyncIndex index, SSyncRaftEntry** ppEntry);
int32_t raftCacheGetEntryP(struct SRaftEntryCache* pCache, SyncIndex index, SSyncRaftEntry** ppEntry);
int32_t raftCacheDelEntry(struct SRaftEntryCache* pCache, SyncIndex index);
int32_t raftCacheGetAndDel(struct SRaftEntryCache* pCache, SyncIndex index, SSyncRaftEntry** ppEntry);
int32_t raftCacheClear(struct SRaftEntryCache* pCache);
SRaftEntryCache* raftEntryCacheCreate(SSyncNode* pSyncNode, int32_t maxCount);
void raftEntryCacheDestroy(SRaftEntryCache* pCache);
int32_t raftEntryCachePutEntry(struct SRaftEntryCache* pCache, SSyncRaftEntry* pEntry);
int32_t raftEntryCacheGetEntry(struct SRaftEntryCache* pCache, SyncIndex index, SSyncRaftEntry** ppEntry);
int32_t raftEntryCacheGetEntryP(struct SRaftEntryCache* pCache, SyncIndex index, SSyncRaftEntry** ppEntry);
int32_t raftEntryCacheClear(struct SRaftEntryCache* pCache, int32_t count);
cJSON* raftCache2Json(SRaftEntryCache* pObj);
char* raftCache2Str(SRaftEntryCache* pObj);
void raftCachePrint(SRaftEntryCache* pObj);
void raftCachePrint2(char* s, SRaftEntryCache* pObj);
void raftCacheLog(SRaftEntryCache* pObj);
void raftCacheLog2(char* s, SRaftEntryCache* pObj);
cJSON* raftEntryCache2Json(SRaftEntryCache* pObj);
char* raftEntryCache2Str(SRaftEntryCache* pObj);
void raftEntryCachePrint(SRaftEntryCache* pObj);
void raftEntryCachePrint2(char* s, SRaftEntryCache* pObj);
void raftEntryCacheLog(SRaftEntryCache* pObj);
void raftEntryCacheLog2(char* s, SRaftEntryCache* pObj);
#ifdef __cplusplus
}
......
......@@ -1055,19 +1055,12 @@ SSyncNode* syncNodeOpen(const SSyncInfo* pOldSyncInfo) {
}
// tools
pSyncNode->pSyncRespMgr = syncRespMgrCreate(pSyncNode, 0);
pSyncNode->pSyncRespMgr = syncRespMgrCreate(pSyncNode, SYNC_RESP_TTL_MS);
ASSERT(pSyncNode->pSyncRespMgr != NULL);
// restore state
pSyncNode->restoreFinish = false;
// pSyncNode->pSnapshot = NULL;
// if (pSyncNode->pFsm->FpGetSnapshotInfo != NULL) {
// pSyncNode->pSnapshot = taosMemoryMalloc(sizeof(SSnapshot));
// pSyncNode->pFsm->FpGetSnapshotInfo(pSyncNode->pFsm, pSyncNode->pSnapshot);
// }
// tsem_init(&(pSyncNode->restoreSem), 0, 0);
// snapshot senders
for (int i = 0; i < TSDB_MAX_REPLICA; ++i) {
SSyncSnapshotSender* pSender = snapshotSenderCreate(pSyncNode, i);
......
......@@ -198,8 +198,8 @@ void syncEntryLog2(char* s, const SSyncRaftEntry* pObj) {
}
//-----------------------------------
SRaftEntryCache* raftCacheCreate(SSyncNode* pSyncNode, int32_t maxCount) {
SRaftEntryCache* pCache = taosMemoryMalloc(sizeof(SRaftEntryCache));
SRaftEntryHashCache* raftCacheCreate(SSyncNode* pSyncNode, int32_t maxCount) {
SRaftEntryHashCache* pCache = taosMemoryMalloc(sizeof(SRaftEntryHashCache));
if (pCache == NULL) {
sError("vgId:%d raft cache create error", pSyncNode->vgId);
return NULL;
......@@ -220,7 +220,7 @@ SRaftEntryCache* raftCacheCreate(SSyncNode* pSyncNode, int32_t maxCount) {
return pCache;
}
void raftCacheDestroy(SRaftEntryCache* pCache) {
void raftCacheDestroy(SRaftEntryHashCache* pCache) {
if (pCache != NULL) {
taosThreadMutexLock(&(pCache->mutex));
taosHashCleanup(pCache->pEntryHash);
......@@ -233,7 +233,7 @@ void raftCacheDestroy(SRaftEntryCache* pCache) {
// success, return 1
// max count, return 0
// error, return -1
int32_t raftCachePutEntry(struct SRaftEntryCache* pCache, SSyncRaftEntry* pEntry) {
int32_t raftCachePutEntry(struct SRaftEntryHashCache* pCache, SSyncRaftEntry* pEntry) {
taosThreadMutexLock(&(pCache->mutex));
if (pCache->currentCount >= pCache->maxCount) {
......@@ -259,7 +259,7 @@ int32_t raftCachePutEntry(struct SRaftEntryCache* pCache, SSyncRaftEntry* pEntry
// success, return 0
// error, return -1
// not exist, return -1, terrno = TSDB_CODE_WAL_LOG_NOT_EXIST
int32_t raftCacheGetEntry(struct SRaftEntryCache* pCache, SyncIndex index, SSyncRaftEntry** ppEntry) {
int32_t raftCacheGetEntry(struct SRaftEntryHashCache* pCache, SyncIndex index, SSyncRaftEntry** ppEntry) {
if (ppEntry == NULL) {
return -1;
}
......@@ -292,7 +292,7 @@ int32_t raftCacheGetEntry(struct SRaftEntryCache* pCache, SyncIndex index, SSync
// success, return 0
// error, return -1
// not exist, return -1, terrno = TSDB_CODE_WAL_LOG_NOT_EXIST
int32_t raftCacheGetEntryP(struct SRaftEntryCache* pCache, SyncIndex index, SSyncRaftEntry** ppEntry) {
int32_t raftCacheGetEntryP(struct SRaftEntryHashCache* pCache, SyncIndex index, SSyncRaftEntry** ppEntry) {
if (ppEntry == NULL) {
return -1;
}
......@@ -321,7 +321,7 @@ int32_t raftCacheGetEntryP(struct SRaftEntryCache* pCache, SyncIndex index, SSyn
return -1;
}
int32_t raftCacheDelEntry(struct SRaftEntryCache* pCache, SyncIndex index) {
int32_t raftCacheDelEntry(struct SRaftEntryHashCache* pCache, SyncIndex index) {
taosThreadMutexLock(&(pCache->mutex));
taosHashRemove(pCache->pEntryHash, &index, sizeof(index));
--(pCache->currentCount);
......@@ -329,7 +329,7 @@ int32_t raftCacheDelEntry(struct SRaftEntryCache* pCache, SyncIndex index) {
return 0;
}
int32_t raftCacheGetAndDel(struct SRaftEntryCache* pCache, SyncIndex index, SSyncRaftEntry** ppEntry) {
int32_t raftCacheGetAndDel(struct SRaftEntryHashCache* pCache, SyncIndex index, SSyncRaftEntry** ppEntry) {
if (ppEntry == NULL) {
return -1;
}
......@@ -362,7 +362,7 @@ int32_t raftCacheGetAndDel(struct SRaftEntryCache* pCache, SyncIndex index, SSyn
return -1;
}
int32_t raftCacheClear(struct SRaftEntryCache* pCache) {
int32_t raftCacheClear(struct SRaftEntryHashCache* pCache) {
taosThreadMutexLock(&(pCache->mutex));
taosHashClear(pCache->pEntryHash);
pCache->currentCount = 0;
......@@ -371,7 +371,7 @@ int32_t raftCacheClear(struct SRaftEntryCache* pCache) {
}
//-----------------------------------
cJSON* raftCache2Json(SRaftEntryCache* pCache) {
cJSON* raftCache2Json(SRaftEntryHashCache* pCache) {
char u64buf[128] = {0};
cJSON* pRoot = cJSON_CreateObject();
......@@ -402,41 +402,283 @@ cJSON* raftCache2Json(SRaftEntryCache* pCache) {
}
cJSON* pJson = cJSON_CreateObject();
cJSON_AddItemToObject(pJson, "SRaftEntryCache", pRoot);
cJSON_AddItemToObject(pJson, "SRaftEntryHashCache", pRoot);
return pJson;
}
char* raftCache2Str(SRaftEntryCache* pCache) {
char* raftCache2Str(SRaftEntryHashCache* pCache) {
cJSON* pJson = raftCache2Json(pCache);
char* serialized = cJSON_Print(pJson);
cJSON_Delete(pJson);
return serialized;
}
void raftCachePrint(SRaftEntryCache* pCache) {
void raftCachePrint(SRaftEntryHashCache* pCache) {
char* serialized = raftCache2Str(pCache);
printf("raftCachePrint | len:%" PRIu64 " | %s \n", strlen(serialized), serialized);
fflush(NULL);
taosMemoryFree(serialized);
}
void raftCachePrint2(char* s, SRaftEntryCache* pCache) {
void raftCachePrint2(char* s, SRaftEntryHashCache* pCache) {
char* serialized = raftCache2Str(pCache);
printf("raftCachePrint2 | len:%" PRIu64 " | %s | %s \n", strlen(serialized), s, serialized);
fflush(NULL);
taosMemoryFree(serialized);
}
void raftCacheLog(SRaftEntryCache* pCache) {
void raftCacheLog(SRaftEntryHashCache* pCache) {
char* serialized = raftCache2Str(pCache);
sTrace("raftCacheLog | len:%" PRIu64 " | %s", strlen(serialized), serialized);
taosMemoryFree(serialized);
}
void raftCacheLog2(char* s, SRaftEntryCache* pCache) {
void raftCacheLog2(char* s, SRaftEntryHashCache* pCache) {
if (gRaftDetailLog) {
char* serialized = raftCache2Str(pCache);
sTraceLong("raftCacheLog2 | len:%" PRIu64 " | %s | %s", strlen(serialized), s, serialized);
taosMemoryFree(serialized);
}
}
//-----------------------------------
static char* keyFn(const void* pData) {
SSyncRaftEntry* pEntry = (SSyncRaftEntry*)pData;
return (char*)(&(pEntry->index));
}
static int cmpFn(const void* p1, const void* p2) { return memcmp(p1, p2, sizeof(SyncIndex)); }
SRaftEntryCache* raftEntryCacheCreate(SSyncNode* pSyncNode, int32_t maxCount) {
SRaftEntryCache* pCache = taosMemoryMalloc(sizeof(SRaftEntryCache));
if (pCache == NULL) {
sError("vgId:%d raft cache create error", pSyncNode->vgId);
return NULL;
}
pCache->pSkipList =
tSkipListCreate(MAX_SKIP_LIST_LEVEL, TSDB_DATA_TYPE_BINARY, sizeof(SyncIndex), cmpFn, SL_ALLOW_DUP_KEY, keyFn);
if (pCache->pSkipList == NULL) {
sError("vgId:%d raft cache create hash error", pSyncNode->vgId);
return NULL;
}
taosThreadMutexInit(&(pCache->mutex), NULL);
pCache->maxCount = maxCount;
pCache->currentCount = 0;
pCache->pSyncNode = pSyncNode;
return pCache;
}
void raftEntryCacheDestroy(SRaftEntryCache* pCache) {
if (pCache != NULL) {
taosThreadMutexLock(&(pCache->mutex));
tSkipListDestroy(pCache->pSkipList);
taosThreadMutexUnlock(&(pCache->mutex));
taosThreadMutexDestroy(&(pCache->mutex));
taosMemoryFree(pCache);
}
}
// success, return 1
// max count, return 0
// error, return -1
int32_t raftEntryCachePutEntry(struct SRaftEntryCache* pCache, SSyncRaftEntry* pEntry) {
taosThreadMutexLock(&(pCache->mutex));
if (pCache->currentCount >= pCache->maxCount) {
taosThreadMutexUnlock(&(pCache->mutex));
return 0;
}
SSkipListNode* pSkipListNode = tSkipListPut(pCache->pSkipList, pEntry);
ASSERT(pSkipListNode != NULL);
++(pCache->currentCount);
do {
char eventLog[128];
snprintf(eventLog, sizeof(eventLog), "raft cache add, type:%s,%d, type2:%s,%d, index:%" PRId64 ", bytes:%d",
TMSG_INFO(pEntry->msgType), pEntry->msgType, TMSG_INFO(pEntry->originalRpcType), pEntry->originalRpcType,
pEntry->index, pEntry->bytes);
syncNodeEventLog(pCache->pSyncNode, eventLog);
} while (0);
taosThreadMutexUnlock(&(pCache->mutex));
return 1;
}
// find one, return 1
// not found, return 0
// error, return -1
int32_t raftEntryCacheGetEntry(struct SRaftEntryCache* pCache, SyncIndex index, SSyncRaftEntry** ppEntry) {
ASSERT(ppEntry != NULL);
SSyncRaftEntry* pEntry = NULL;
int32_t code = raftEntryCacheGetEntryP(pCache, index, &pEntry);
if (code == 1) {
*ppEntry = taosMemoryMalloc(pEntry->bytes);
memcpy(*ppEntry, pEntry, pEntry->bytes);
} else {
*ppEntry = NULL;
}
return code;
}
// find one, return 1
// not found, return 0
// error, return -1
int32_t raftEntryCacheGetEntryP(struct SRaftEntryCache* pCache, SyncIndex index, SSyncRaftEntry** ppEntry) {
taosThreadMutexLock(&(pCache->mutex));
SyncIndex index2 = index;
int32_t code = 0;
SArray* entryPArray = tSkipListGet(pCache->pSkipList, (char*)(&index2));
int32_t arraySize = taosArrayGetSize(entryPArray);
if (arraySize == 1) {
SSkipListNode** ppNode = (SSkipListNode**)taosArrayGet(entryPArray, 0);
ASSERT(*ppNode != NULL);
*ppEntry = (SSyncRaftEntry*)SL_GET_NODE_DATA(*ppNode);
code = 1;
} else if (arraySize == 0) {
code = 0;
} else {
ASSERT(0);
code = -1;
}
taosArrayDestroy(entryPArray);
taosThreadMutexUnlock(&(pCache->mutex));
return code;
}
// count = -1, clear all
// count >= 0, clear count
// return -1, error
// return delete count
int32_t raftEntryCacheClear(struct SRaftEntryCache* pCache, int32_t count) {
taosThreadMutexLock(&(pCache->mutex));
int32_t returnCnt = 0;
if (count == -1) {
// clear all
SSkipListIterator* pIter = tSkipListCreateIter(pCache->pSkipList);
while (tSkipListIterNext(pIter)) {
SSkipListNode* pNode = tSkipListIterGet(pIter);
ASSERT(pNode != NULL);
SSyncRaftEntry* pEntry = (SSyncRaftEntry*)SL_GET_NODE_DATA(pNode);
syncEntryDestory(pEntry);
++returnCnt;
}
tSkipListDestroyIter(pIter);
tSkipListDestroy(pCache->pSkipList);
pCache->pSkipList =
tSkipListCreate(MAX_SKIP_LIST_LEVEL, TSDB_DATA_TYPE_BINARY, sizeof(SyncIndex), cmpFn, SL_ALLOW_DUP_KEY, keyFn);
ASSERT(pCache->pSkipList != NULL);
} else {
// clear count
int i = 0;
SSkipListIterator* pIter = tSkipListCreateIter(pCache->pSkipList);
SArray* delNodeArray = taosArrayInit(0, sizeof(SSkipListNode*));
// free entry
while (tSkipListIterNext(pIter)) {
SSkipListNode* pNode = tSkipListIterGet(pIter);
ASSERT(pNode != NULL);
if (i++ >= count) {
break;
}
// sDebug("push pNode:%p", pNode);
taosArrayPush(delNodeArray, &pNode);
++returnCnt;
SSyncRaftEntry* pEntry = (SSyncRaftEntry*)SL_GET_NODE_DATA(pNode);
syncEntryDestory(pEntry);
}
tSkipListDestroyIter(pIter);
// delete skiplist node
int32_t arraySize = taosArrayGetSize(delNodeArray);
for (int32_t i = 0; i < arraySize; ++i) {
SSkipListNode** ppNode = taosArrayGet(delNodeArray, i);
// sDebug("get pNode:%p", *ppNode);
tSkipListRemoveNode(pCache->pSkipList, *ppNode);
}
taosArrayDestroy(delNodeArray);
}
pCache->currentCount -= returnCnt;
taosThreadMutexUnlock(&(pCache->mutex));
return returnCnt;
}
cJSON* raftEntryCache2Json(SRaftEntryCache* pCache) {
char u64buf[128] = {0};
cJSON* pRoot = cJSON_CreateObject();
if (pCache != NULL) {
taosThreadMutexLock(&(pCache->mutex));
snprintf(u64buf, sizeof(u64buf), "%p", pCache->pSyncNode);
cJSON_AddStringToObject(pRoot, "pSyncNode", u64buf);
cJSON_AddNumberToObject(pRoot, "currentCount", pCache->currentCount);
cJSON_AddNumberToObject(pRoot, "maxCount", pCache->maxCount);
cJSON* pEntries = cJSON_CreateArray();
cJSON_AddItemToObject(pRoot, "entries", pEntries);
SSkipListIterator* pIter = tSkipListCreateIter(pCache->pSkipList);
while (tSkipListIterNext(pIter)) {
SSkipListNode* pNode = tSkipListIterGet(pIter);
ASSERT(pNode != NULL);
SSyncRaftEntry* pEntry = (SSyncRaftEntry*)SL_GET_NODE_DATA(pNode);
cJSON_AddItemToArray(pEntries, syncEntry2Json(pEntry));
}
tSkipListDestroyIter(pIter);
taosThreadMutexUnlock(&(pCache->mutex));
}
cJSON* pJson = cJSON_CreateObject();
cJSON_AddItemToObject(pJson, "SRaftEntryCache", pRoot);
return pJson;
}
char* raftEntryCache2Str(SRaftEntryCache* pObj) {
cJSON* pJson = raftEntryCache2Json(pObj);
char* serialized = cJSON_Print(pJson);
cJSON_Delete(pJson);
return serialized;
}
void raftEntryCachePrint(SRaftEntryCache* pObj) {
char* serialized = raftEntryCache2Str(pObj);
printf("raftEntryCachePrint | len:%" PRIu64 " | %s \n", strlen(serialized), serialized);
fflush(NULL);
taosMemoryFree(serialized);
}
void raftEntryCachePrint2(char* s, SRaftEntryCache* pObj) {
char* serialized = raftEntryCache2Str(pObj);
printf("raftEntryCachePrint2 | len:%" PRIu64 " | %s | %s \n", strlen(serialized), s, serialized);
fflush(NULL);
taosMemoryFree(serialized);
}
void raftEntryCacheLog(SRaftEntryCache* pObj) {
char* serialized = raftEntryCache2Str(pObj);
sTrace("raftEntryCacheLog | len:%" PRIu64 " | %s", strlen(serialized), serialized);
taosMemoryFree(serialized);
}
void raftEntryCacheLog2(char* s, SRaftEntryCache* pObj) {
if (gRaftDetailLog) {
char* serialized = raftEntryCache2Str(pObj);
sTraceLong("raftEntryCacheLog2 | len:%" PRIu64 " | %s | %s", strlen(serialized), s, serialized);
taosMemoryFree(serialized);
}
}
\ No newline at end of file
......@@ -122,54 +122,45 @@ void syncRespCleanByTTL(SSyncRespMgr *pObj, int64_t ttl) {
int cnt = 0;
SSyncNode *pSyncNode = pObj->data;
SArray *delIndexArray = taosArrayInit(0, sizeof(SyncIndex));
SArray *delIndexArray = taosArrayInit(0, sizeof(uint64_t));
ASSERT(delIndexArray != NULL);
while (pStub) {
size_t len;
void * key = taosHashGetKey(pStub, &len);
SyncIndex *pIndex = (SyncIndex *)key;
size_t len;
void * key = taosHashGetKey(pStub, &len);
uint64_t *pSeqNum = (uint64_t *)key;
int64_t nowMS = taosGetTimestampMs();
if (nowMS - pStub->createTime > ttl) {
taosArrayPush(delIndexArray, pIndex);
taosArrayPush(delIndexArray, pSeqNum);
cnt++;
SSyncRaftEntry *pEntry = NULL;
int32_t code = 0;
if (pSyncNode->pLogStore != NULL) {
code = pSyncNode->pLogStore->syncLogGetEntry(pSyncNode->pLogStore, *pIndex, &pEntry);
if (code == 0 && pEntry != NULL) {
SFsmCbMeta cbMeta = {0};
cbMeta.index = pEntry->index;
cbMeta.lastConfigIndex = syncNodeGetSnapshotConfigIndex(pSyncNode, cbMeta.index);
cbMeta.isWeak = pEntry->isWeak;
cbMeta.code = TSDB_CODE_SYN_TIMEOUT;
cbMeta.state = pSyncNode->state;
cbMeta.seqNum = pEntry->seqNum;
cbMeta.term = pEntry->term;
cbMeta.currentTerm = pSyncNode->pRaftStore->currentTerm;
cbMeta.flag = 0;
SRpcMsg rpcMsg = pStub->rpcMsg;
rpcMsg.pCont = rpcMallocCont(pEntry->dataLen);
memcpy(rpcMsg.pCont, pEntry->data, pEntry->dataLen);
pSyncNode->pFsm->FpCommitCb(pSyncNode->pFsm, &rpcMsg, cbMeta);
syncEntryDestory(pEntry);
}
}
SFsmCbMeta cbMeta = {0};
cbMeta.index = SYNC_INDEX_INVALID;
cbMeta.lastConfigIndex = SYNC_INDEX_INVALID;
cbMeta.isWeak = false;
cbMeta.code = TSDB_CODE_SYN_TIMEOUT;
cbMeta.state = pSyncNode->state;
cbMeta.seqNum = *pSeqNum;
cbMeta.term = SYNC_TERM_INVALID;
cbMeta.currentTerm = pSyncNode->pRaftStore->currentTerm;
cbMeta.flag = 0;
pStub->rpcMsg.pCont = NULL;
pStub->rpcMsg.contLen = 0;
pSyncNode->pFsm->FpCommitCb(pSyncNode->pFsm, &(pStub->rpcMsg), cbMeta);
}
pStub = (SRespStub *)taosHashIterate(pObj->pRespHash, pStub);
}
int32_t arraySize = taosArrayGetSize(delIndexArray);
sDebug("vgId:%d, resp clean by ttl, cnt:%d, array-size:%d", pSyncNode->vgId, cnt, arraySize);
sDebug("vgId:%d, resp mgr clean by ttl, cnt:%d, array-size:%d", pSyncNode->vgId, cnt, arraySize);
for (int32_t i = 0; i < arraySize; ++i) {
SyncIndex *pIndex = taosArrayGet(delIndexArray, i);
taosHashRemove(pObj->pRespHash, pIndex, sizeof(SyncIndex));
uint64_t *pSeqNum = taosArrayGet(delIndexArray, i);
taosHashRemove(pObj->pRespHash, pSeqNum, sizeof(uint64_t));
sDebug("vgId:%d, resp mgr clean by ttl, seq:%d", pSyncNode->vgId, *pSeqNum);
}
taosArrayDestroy(delIndexArray);
}
......@@ -16,9 +16,14 @@
#include "syncTimeout.h"
#include "syncElection.h"
#include "syncReplication.h"
#include "syncRespMgr.h"
int32_t syncNodeTimerRoutine(SSyncNode* ths) {
syncNodeEventLog(ths, "timer routines ... ");
if (ths->vgId != 1) {
syncRespClean(ths->pSyncRespMgr);
}
return 0;
}
......
......@@ -18,6 +18,7 @@ add_executable(syncIndexMgrTest "")
add_executable(syncLogStoreTest "")
add_executable(syncEntryTest "")
add_executable(syncEntryCacheTest "")
add_executable(syncHashCacheTest "")
add_executable(syncRequestVoteTest "")
add_executable(syncRequestVoteReplyTest "")
add_executable(syncAppendEntriesTest "")
......@@ -137,6 +138,10 @@ target_sources(syncEntryCacheTest
PRIVATE
"syncEntryCacheTest.cpp"
)
target_sources(syncHashCacheTest
PRIVATE
"syncHashCacheTest.cpp"
)
target_sources(syncRequestVoteTest
PRIVATE
"syncRequestVoteTest.cpp"
......@@ -387,6 +392,11 @@ target_include_directories(syncEntryCacheTest
"${TD_SOURCE_DIR}/include/libs/sync"
"${CMAKE_CURRENT_SOURCE_DIR}/../inc"
)
target_include_directories(syncHashCacheTest
PUBLIC
"${TD_SOURCE_DIR}/include/libs/sync"
"${CMAKE_CURRENT_SOURCE_DIR}/../inc"
)
target_include_directories(syncRequestVoteTest
PUBLIC
"${TD_SOURCE_DIR}/include/libs/sync"
......@@ -654,6 +664,10 @@ target_link_libraries(syncEntryCacheTest
sync
gtest_main
)
target_link_libraries(syncHashCacheTest
sync
gtest_main
)
target_link_libraries(syncRequestVoteTest
sync
gtest_main
......
......@@ -43,222 +43,82 @@ SRaftEntryCache* createCache(int maxCount) {
SSyncNode* pSyncNode = createFakeNode();
ASSERT(pSyncNode != NULL);
SRaftEntryCache* pCache = raftCacheCreate(pSyncNode, maxCount);
SRaftEntryCache* pCache = raftEntryCacheCreate(pSyncNode, maxCount);
ASSERT(pCache != NULL);
return pCache;
}
void test1() {
int32_t code = 0;
int32_t code = 0;
SRaftEntryCache* pCache = createCache(5);
for (int i = 0; i < 5; ++i) {
for (int i = 0; i < 10; ++i) {
SSyncRaftEntry* pEntry = createEntry(i);
code = raftCachePutEntry(pCache, pEntry);
ASSERT(code == 1);
syncEntryDestory(pEntry);
code = raftEntryCachePutEntry(pCache, pEntry);
sTrace("put entry code:%d, pEntry:%p", code, pEntry);
}
raftCacheLog2((char*)"==test1 write 5 entries==", pCache);
raftEntryCacheLog2((char*)"==test1 write 5 entries==", pCache);
SyncIndex index;
index = 1;
code = raftCacheDelEntry(pCache, index);
ASSERT(code == 0);
index = 3;
code = raftCacheDelEntry(pCache, index);
ASSERT(code == 0);
raftCacheLog2((char*)"==test1 delete 1,3==", pCache);
raftEntryCacheClear(pCache, 3);
raftEntryCacheLog2((char*)"==test1 evict 3 entries==", pCache);
code = raftCacheClear(pCache);
ASSERT(code == 0);
raftCacheLog2((char*)"==clear all==", pCache);
raftEntryCacheClear(pCache, -1);
raftEntryCacheLog2((char*)"==test1 evict -1(all) entries==", pCache);
}
void test2() {
int32_t code = 0;
int32_t code = 0;
SRaftEntryCache* pCache = createCache(5);
for (int i = 0; i < 5; ++i) {
for (int i = 0; i < 10; ++i) {
SSyncRaftEntry* pEntry = createEntry(i);
code = raftCachePutEntry(pCache, pEntry);
ASSERT(code == 1);
syncEntryDestory(pEntry);
code = raftEntryCachePutEntry(pCache, pEntry);
sTrace("put entry code:%d, pEntry:%p", code, pEntry);
}
raftCacheLog2((char*)"==test2 write 5 entries==", pCache);
raftEntryCacheLog2((char*)"==test1 write 5 entries==", pCache);
SyncIndex index;
index = 1;
SSyncRaftEntry* pEntry;
code = raftCacheGetEntry(pCache, index, &pEntry);
ASSERT(code == 0);
syncEntryDestory(pEntry);
syncEntryLog2((char*)"==test2 get entry 1==", pEntry);
SyncIndex index = 2;
SSyncRaftEntry* pEntry = NULL;
index = 2;
code = raftCacheGetEntryP(pCache, index, &pEntry);
ASSERT(code == 0);
code = raftEntryCacheGetEntryP(pCache, index, &pEntry);
ASSERT(code == 1 && index == pEntry->index);
sTrace("get entry:%p for %ld", pEntry, index);
syncEntryLog2((char*)"==test2 get entry pointer 2==", pEntry);
code = raftEntryCacheGetEntry(pCache, index, &pEntry);
ASSERT(code == 1 && index == pEntry->index);
sTrace("get entry:%p for %ld", pEntry, index);
syncEntryLog2((char*)"==test2 get entry 2==", pEntry);
syncEntryDestory(pEntry);
// not found
index = 8;
code = raftCacheGetEntry(pCache, index, &pEntry);
ASSERT(code == -1 && terrno == TSDB_CODE_WAL_LOG_NOT_EXIST);
code = raftEntryCacheGetEntry(pCache, index, &pEntry);
ASSERT(code == 0);
sTrace("get entry:%p for %ld", pEntry, index);
sTrace("==test2 get entry 8 not found==");
// not found
index = 9;
code = raftCacheGetEntryP(pCache, index, &pEntry);
ASSERT(code == -1 && terrno == TSDB_CODE_WAL_LOG_NOT_EXIST);
sTrace("==test2 get entry pointer 9 not found==");
code = raftEntryCacheGetEntry(pCache, index, &pEntry);
ASSERT(code == 0);
sTrace("get entry:%p for %ld", pEntry, index);
sTrace("==test2 get entry 9 not found==");
}
void test3() {
int32_t code = 0;
SRaftEntryCache* pCache = createCache(5);
for (int i = 0; i < 5; ++i) {
SSyncRaftEntry* pEntry = createEntry(i);
code = raftCachePutEntry(pCache, pEntry);
ASSERT(code == 1);
syncEntryDestory(pEntry);
}
for (int i = 6; i < 10; ++i) {
SSyncRaftEntry* pEntry = createEntry(i);
code = raftCachePutEntry(pCache, pEntry);
ASSERT(code == 0);
syncEntryDestory(pEntry);
}
raftCacheLog2((char*)"==test3 write 10 entries, max count is 5==", pCache);
}
void test4() {
int32_t code = 0;
SRaftEntryCache* pCache = createCache(5);
for (int i = 0; i < 5; ++i) {
int32_t code = 0;
SRaftEntryCache* pCache = createCache(20);
for (int i = 0; i <= 4; ++i) {
SSyncRaftEntry* pEntry = createEntry(i);
code = raftCachePutEntry(pCache, pEntry);
ASSERT(code == 1);
syncEntryDestory(pEntry);
}
raftCacheLog2((char*)"==test4 write 5 entries==", pCache);
SyncIndex index;
index = 3;
SSyncRaftEntry* pEntry;
code = raftCacheGetAndDel(pCache, index, &pEntry);
ASSERT(code == 0);
syncEntryLog2((char*)"==test4 get-and-del entry 3==", pEntry);
raftCacheLog2((char*)"==test4 after get-and-del entry 3==", pCache);
}
static char* keyFn(const void* pData) {
SSyncRaftEntry* pEntry = (SSyncRaftEntry*)pData;
return (char*)(&(pEntry->index));
}
static int cmpFn(const void* p1, const void* p2) { return memcmp(p1, p2, sizeof(SyncIndex)); }
void printSkipList(SSkipList* pSkipList) {
ASSERT(pSkipList != NULL);
SSkipListIterator* pIter = tSkipListCreateIter(pSkipList);
while (tSkipListIterNext(pIter)) {
SSkipListNode* pNode = tSkipListIterGet(pIter);
ASSERT(pNode != NULL);
SSyncRaftEntry* pEntry = (SSyncRaftEntry*)SL_GET_NODE_DATA(pNode);
syncEntryPrint2((char*)"", pEntry);
}
}
void delSkipListFirst(SSkipList* pSkipList, int n) {
ASSERT(pSkipList != NULL);
sTrace("delete first %d -------------", n);
SSkipListIterator* pIter = tSkipListCreateIter(pSkipList);
for (int i = 0; i < n; ++i) {
tSkipListIterNext(pIter);
SSkipListNode* pNode = tSkipListIterGet(pIter);
tSkipListRemoveNode(pSkipList, pNode);
code = raftEntryCachePutEntry(pCache, pEntry);
sTrace("put entry code:%d, pEntry:%p", code, pEntry);
}
}
SSyncRaftEntry* getLogEntry2(SSkipList* pSkipList, SyncIndex index) {
SyncIndex index2 = index;
SSyncRaftEntry *pEntry = NULL;
int arraySize = 0;
SArray* entryPArray = tSkipListGet(pSkipList, (char*)(&index2));
arraySize = taosArrayGetSize(entryPArray);
if (arraySize > 0) {
SSkipListNode** ppNode = (SSkipListNode**)taosArrayGet(entryPArray, 0);
ASSERT(*ppNode != NULL);
pEntry = (SSyncRaftEntry*)SL_GET_NODE_DATA(*ppNode);
}
taosArrayDestroy(entryPArray);
sTrace("get index2: %ld, arraySize:%d -------------", index, arraySize);
syncEntryLog2((char*)"getLogEntry2", pEntry);
return pEntry;
}
SSyncRaftEntry* getLogEntry(SSkipList* pSkipList, SyncIndex index) {
sTrace("get index: %ld -------------", index);
SyncIndex index2 = index;
SSyncRaftEntry *pEntry = NULL;
SSkipListIterator* pIter = tSkipListCreateIterFromVal(pSkipList, (const char *)&index2, TSDB_DATA_TYPE_BINARY, TSDB_ORDER_ASC);
if (tSkipListIterNext(pIter)) {
SSkipListNode* pNode = tSkipListIterGet(pIter);
ASSERT(pNode != NULL);
pEntry = (SSyncRaftEntry*)SL_GET_NODE_DATA(pNode);
}
syncEntryLog2((char*)"getLogEntry", pEntry);
return pEntry;
}
void test5() {
SSkipList* pSkipList =
tSkipListCreate(MAX_SKIP_LIST_LEVEL, TSDB_DATA_TYPE_BINARY, sizeof(SyncIndex), cmpFn, SL_ALLOW_DUP_KEY, keyFn);
ASSERT(pSkipList != NULL);
sTrace("insert 9 - 5");
for (int i = 9; i >= 5; --i) {
SSyncRaftEntry* pEntry = createEntry(i);
SSkipListNode* pSkipListNode = tSkipListPut(pSkipList, pEntry);
code = raftEntryCachePutEntry(pCache, pEntry);
sTrace("put entry code:%d, pEntry:%p", code, pEntry);
}
sTrace("insert 0 - 4");
for (int i = 0; i <= 4; ++i) {
SSyncRaftEntry* pEntry = createEntry(i);
SSkipListNode* pSkipListNode = tSkipListPut(pSkipList, pEntry);
}
sTrace("insert 7 7 7 7 7");
for (int i = 0; i <= 4; ++i) {
SSyncRaftEntry* pEntry = createEntry(7);
SSkipListNode* pSkipListNode = tSkipListPut(pSkipList, pEntry);
}
sTrace("print: -------------");
printSkipList(pSkipList);
delSkipListFirst(pSkipList, 3);
sTrace("print: -------------");
printSkipList(pSkipList);
getLogEntry(pSkipList, 2);
getLogEntry(pSkipList, 5);
getLogEntry(pSkipList, 7);
getLogEntry(pSkipList, 7);
getLogEntry2(pSkipList, 2);
getLogEntry2(pSkipList, 5);
getLogEntry2(pSkipList, 7);
getLogEntry2(pSkipList, 7);
tSkipListDestroy(pSkipList);
raftEntryCacheLog2((char*)"==test3 write 10 entries==", pCache);
}
int main(int argc, char** argv) {
......@@ -266,14 +126,9 @@ int main(int argc, char** argv) {
tsAsyncLog = 0;
sDebugFlag = DEBUG_TRACE + DEBUG_SCREEN + DEBUG_FILE + DEBUG_DEBUG;
/*
test1();
test2();
test3();
test4();
*/
test5();
test1();
test2();
test3();
return 0;
}
#include <stdio.h>
#include "syncEnv.h"
#include "syncIO.h"
#include "syncInt.h"
#include "syncRaftLog.h"
#include "syncRaftStore.h"
#include "syncUtil.h"
#include "tskiplist.h"
void logTest() {
sTrace("--- sync log test: trace");
sDebug("--- sync log test: debug");
sInfo("--- sync log test: info");
sWarn("--- sync log test: warn");
sError("--- sync log test: error");
sFatal("--- sync log test: fatal");
}
SSyncRaftEntry* createEntry(int i) {
int32_t dataLen = 20;
SSyncRaftEntry* pEntry = syncEntryBuild(dataLen);
assert(pEntry != NULL);
pEntry->msgType = 88;
pEntry->originalRpcType = 99;
pEntry->seqNum = 3;
pEntry->isWeak = true;
pEntry->term = 100 + i;
pEntry->index = i;
snprintf(pEntry->data, dataLen, "value%d", i);
return pEntry;
}
SSyncNode* createFakeNode() {
SSyncNode* pSyncNode = (SSyncNode*)taosMemoryMalloc(sizeof(SSyncNode));
ASSERT(pSyncNode != NULL);
memset(pSyncNode, 0, sizeof(SSyncNode));
return pSyncNode;
}
SRaftEntryHashCache* createCache(int maxCount) {
SSyncNode* pSyncNode = createFakeNode();
ASSERT(pSyncNode != NULL);
SRaftEntryHashCache* pCache = raftCacheCreate(pSyncNode, maxCount);
ASSERT(pCache != NULL);
return pCache;
}
void test1() {
int32_t code = 0;
SRaftEntryHashCache* pCache = createCache(5);
for (int i = 0; i < 5; ++i) {
SSyncRaftEntry* pEntry = createEntry(i);
code = raftCachePutEntry(pCache, pEntry);
ASSERT(code == 1);
syncEntryDestory(pEntry);
}
raftCacheLog2((char*)"==test1 write 5 entries==", pCache);
SyncIndex index;
index = 1;
code = raftCacheDelEntry(pCache, index);
ASSERT(code == 0);
index = 3;
code = raftCacheDelEntry(pCache, index);
ASSERT(code == 0);
raftCacheLog2((char*)"==test1 delete 1,3==", pCache);
code = raftCacheClear(pCache);
ASSERT(code == 0);
raftCacheLog2((char*)"==clear all==", pCache);
}
void test2() {
int32_t code = 0;
SRaftEntryHashCache* pCache = createCache(5);
for (int i = 0; i < 5; ++i) {
SSyncRaftEntry* pEntry = createEntry(i);
code = raftCachePutEntry(pCache, pEntry);
ASSERT(code == 1);
syncEntryDestory(pEntry);
}
raftCacheLog2((char*)"==test2 write 5 entries==", pCache);
SyncIndex index;
index = 1;
SSyncRaftEntry* pEntry;
code = raftCacheGetEntry(pCache, index, &pEntry);
ASSERT(code == 0);
syncEntryDestory(pEntry);
syncEntryLog2((char*)"==test2 get entry 1==", pEntry);
index = 2;
code = raftCacheGetEntryP(pCache, index, &pEntry);
ASSERT(code == 0);
syncEntryLog2((char*)"==test2 get entry pointer 2==", pEntry);
// not found
index = 8;
code = raftCacheGetEntry(pCache, index, &pEntry);
ASSERT(code == -1 && terrno == TSDB_CODE_WAL_LOG_NOT_EXIST);
sTrace("==test2 get entry 8 not found==");
// not found
index = 9;
code = raftCacheGetEntryP(pCache, index, &pEntry);
ASSERT(code == -1 && terrno == TSDB_CODE_WAL_LOG_NOT_EXIST);
sTrace("==test2 get entry pointer 9 not found==");
}
void test3() {
int32_t code = 0;
SRaftEntryHashCache* pCache = createCache(5);
for (int i = 0; i < 5; ++i) {
SSyncRaftEntry* pEntry = createEntry(i);
code = raftCachePutEntry(pCache, pEntry);
ASSERT(code == 1);
syncEntryDestory(pEntry);
}
for (int i = 6; i < 10; ++i) {
SSyncRaftEntry* pEntry = createEntry(i);
code = raftCachePutEntry(pCache, pEntry);
ASSERT(code == 0);
syncEntryDestory(pEntry);
}
raftCacheLog2((char*)"==test3 write 10 entries, max count is 5==", pCache);
}
void test4() {
int32_t code = 0;
SRaftEntryHashCache* pCache = createCache(5);
for (int i = 0; i < 5; ++i) {
SSyncRaftEntry* pEntry = createEntry(i);
code = raftCachePutEntry(pCache, pEntry);
ASSERT(code == 1);
syncEntryDestory(pEntry);
}
raftCacheLog2((char*)"==test4 write 5 entries==", pCache);
SyncIndex index;
index = 3;
SSyncRaftEntry* pEntry;
code = raftCacheGetAndDel(pCache, index, &pEntry);
ASSERT(code == 0);
syncEntryLog2((char*)"==test4 get-and-del entry 3==", pEntry);
raftCacheLog2((char*)"==test4 after get-and-del entry 3==", pCache);
}
static char* keyFn(const void* pData) {
SSyncRaftEntry* pEntry = (SSyncRaftEntry*)pData;
return (char*)(&(pEntry->index));
}
static int cmpFn(const void* p1, const void* p2) { return memcmp(p1, p2, sizeof(SyncIndex)); }
void printSkipList(SSkipList* pSkipList) {
ASSERT(pSkipList != NULL);
SSkipListIterator* pIter = tSkipListCreateIter(pSkipList);
while (tSkipListIterNext(pIter)) {
SSkipListNode* pNode = tSkipListIterGet(pIter);
ASSERT(pNode != NULL);
SSyncRaftEntry* pEntry = (SSyncRaftEntry*)SL_GET_NODE_DATA(pNode);
syncEntryPrint2((char*)"", pEntry);
}
}
void delSkipListFirst(SSkipList* pSkipList, int n) {
ASSERT(pSkipList != NULL);
sTrace("delete first %d -------------", n);
SSkipListIterator* pIter = tSkipListCreateIter(pSkipList);
for (int i = 0; i < n; ++i) {
tSkipListIterNext(pIter);
SSkipListNode* pNode = tSkipListIterGet(pIter);
tSkipListRemoveNode(pSkipList, pNode);
}
}
SSyncRaftEntry* getLogEntry2(SSkipList* pSkipList, SyncIndex index) {
SyncIndex index2 = index;
SSyncRaftEntry* pEntry = NULL;
int arraySize = 0;
SArray* entryPArray = tSkipListGet(pSkipList, (char*)(&index2));
arraySize = taosArrayGetSize(entryPArray);
if (arraySize > 0) {
SSkipListNode** ppNode = (SSkipListNode**)taosArrayGet(entryPArray, 0);
ASSERT(*ppNode != NULL);
pEntry = (SSyncRaftEntry*)SL_GET_NODE_DATA(*ppNode);
}
taosArrayDestroy(entryPArray);
sTrace("get index2: %ld, arraySize:%d -------------", index, arraySize);
syncEntryLog2((char*)"getLogEntry2", pEntry);
return pEntry;
}
SSyncRaftEntry* getLogEntry(SSkipList* pSkipList, SyncIndex index) {
sTrace("get index: %ld -------------", index);
SyncIndex index2 = index;
SSyncRaftEntry* pEntry = NULL;
SSkipListIterator* pIter =
tSkipListCreateIterFromVal(pSkipList, (const char*)&index2, TSDB_DATA_TYPE_BINARY, TSDB_ORDER_ASC);
if (tSkipListIterNext(pIter)) {
SSkipListNode* pNode = tSkipListIterGet(pIter);
ASSERT(pNode != NULL);
pEntry = (SSyncRaftEntry*)SL_GET_NODE_DATA(pNode);
}
syncEntryLog2((char*)"getLogEntry", pEntry);
return pEntry;
}
void test5() {
SSkipList* pSkipList =
tSkipListCreate(MAX_SKIP_LIST_LEVEL, TSDB_DATA_TYPE_BINARY, sizeof(SyncIndex), cmpFn, SL_ALLOW_DUP_KEY, keyFn);
ASSERT(pSkipList != NULL);
sTrace("insert 9 - 5");
for (int i = 9; i >= 5; --i) {
SSyncRaftEntry* pEntry = createEntry(i);
SSkipListNode* pSkipListNode = tSkipListPut(pSkipList, pEntry);
}
sTrace("insert 0 - 4");
for (int i = 0; i <= 4; ++i) {
SSyncRaftEntry* pEntry = createEntry(i);
SSkipListNode* pSkipListNode = tSkipListPut(pSkipList, pEntry);
}
sTrace("insert 7 7 7 7 7");
for (int i = 0; i <= 4; ++i) {
SSyncRaftEntry* pEntry = createEntry(7);
SSkipListNode* pSkipListNode = tSkipListPut(pSkipList, pEntry);
}
sTrace("print: -------------");
printSkipList(pSkipList);
delSkipListFirst(pSkipList, 3);
sTrace("print: -------------");
printSkipList(pSkipList);
getLogEntry(pSkipList, 2);
getLogEntry(pSkipList, 5);
getLogEntry(pSkipList, 7);
getLogEntry(pSkipList, 7);
getLogEntry2(pSkipList, 2);
getLogEntry2(pSkipList, 5);
getLogEntry2(pSkipList, 7);
getLogEntry2(pSkipList, 7);
tSkipListDestroy(pSkipList);
}
int main(int argc, char** argv) {
gRaftDetailLog = true;
tsAsyncLog = 0;
sDebugFlag = DEBUG_TRACE + DEBUG_SCREEN + DEBUG_FILE + DEBUG_DEBUG;
/*
test1();
test2();
test3();
test4();
*/
test5();
return 0;
}
......@@ -41,7 +41,7 @@ class TDSetSql:
create_stb_sql = f'create table {stbname} ({column_sql[:-1]}) tags({tag_sql[:-1]})'
return create_stb_sql
def set_insertsql(self,column_dict,tbname,binary_str,nchar_str):
def set_insertsql(self,column_dict,tbname,binary_str=None,nchar_str=None):
sql = ''
for k, v in column_dict.items():
if v.lower() == 'timestamp' or v.lower() == 'tinyint' or v.lower() == 'smallint' or v.lower() == 'int' or v.lower() == 'bigint' or \
......
###################################################################
# Copyright (c) 2016 by TAOS Technologies, Inc.
# All rights reserved.
#
# This file is proprietary and confidential to TAOS Technologies.
# No part of this file may be reproduced, stored, transmitted,
# disclosed or used in any form or by any means other than as
# expressly provided by the written permission from Jianhui Tao
#
###################################################################
# -*- coding: utf-8 -*-
from util.log import *
from util.cases import *
from util.sql import *
import subprocess
from util.common import *
class TDTestCase:
def init(self, conn, logSql):
tdLog.debug("start to execute %s" % __file__)
tdSql.init(conn.cursor())
self.dbname = 'db'
def get_database_info(self):
tdSql.query('select database()')
tdSql.checkData(0,0,None)
tdSql.execute(f'create database if not exists {self.dbname}')
tdSql.execute(f'use {self.dbname}')
tdSql.query('select database()')
tdSql.checkData(0,0,self.dbname)
tdSql.execute(f'drop database {self.dbname}')
def check_version(self):
taos_list = ['server','client']
for i in taos_list:
tdSql.query(f'select {i}_version()')
version_info = str(subprocess.run('cat ../../source/util/src/version.c |grep "char version"', shell=True,capture_output=True).stdout.decode('utf8')).split('"')[1]
tdSql.checkData(0,0,version_info)
def get_server_status(self):
tdSql.query('select server_status()')
tdSql.checkData(0,0,1)
tdDnodes.stoptaosd(1)
tdSql.query('select server_status()')
print(tdSql.queryResult)
def run(self):
self.get_database_info()
self.check_version()
# self.get_server_status()
def stop(self):
tdSql.close()
tdLog.success("%s successfully executed" % __file__)
tdCases.addWindows(__file__, TDTestCase())
tdCases.addLinux(__file__, TDTestCase())
\ No newline at end of file
......@@ -13,11 +13,11 @@ class TDTestCase:
tdLog.debug(f"start to excute {__file__}")
tdSql.init(conn.cursor())
# name of normal table
self.ntbname = 'ntb'
self.ntbname = 'ntb'
# name of stable
self.stbname = 'stb'
self.stbname = 'stb'
# structure of column
self.column_dict = {
self.column_dict = {
'ts':'timestamp',
'c1':'int',
'c2':'float',
......@@ -25,13 +25,13 @@ class TDTestCase:
'c4':'nchar(20)'
}
# structure of tag
self.tag_dict = {
self.tag_dict = {
't0':'int'
}
# number of child tables
self.tbnum = 2
self.tbnum = 2
# values of tag,the number of values should equal to tbnum
self.tag_values = [
self.tag_values = [
f'10',
f'100'
]
......@@ -42,7 +42,7 @@ class TDTestCase:
]
self.error_param = [1,'now()']
def run(self): # sourcery skip: extract-duplicate-method
tdSql.prepare()
tdLog.printNoPrefix("==========step1:create tables==========")
......@@ -93,11 +93,11 @@ class TDTestCase:
tdSql.query("select ts from ntb where to_unixtimestamp('1970-01-01T08:00:00+08:00')=0")
tdSql.checkRows(3)
def stop(self):
tdSql.close()
tdLog.success(f"{__file__} successfully executed")
tdCases.addLinux(__file__, TDTestCase())
tdCases.addWindows(__file__, TDTestCase())
\ No newline at end of file
tdCases.addWindows(__file__, TDTestCase())
......@@ -9,14 +9,14 @@ from util.cases import *
class TDTestCase:
updatecfgDict = {'debugFlag': 143 ,"cDebugFlag":143,"uDebugFlag":143 ,"rpcDebugFlag":143 , "tmrDebugFlag":143 ,
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 ,"udfDebugFlag":143}
def init(self, conn, powSql):
tdLog.debug(f"start to excute {__file__}")
tdSql.init(conn.cursor())
self.PI =3.1415926
def prepare_datas(self):
tdSql.execute(
'''create table stb1
......@@ -24,7 +24,7 @@ class TDTestCase:
tags (t1 int)
'''
)
tdSql.execute(
'''
create table t1
......@@ -66,14 +66,14 @@ class TDTestCase:
( '2023-02-21 01:01:01.000', NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL )
'''
)
def check_result_auto_acos(self ,origin_query , pow_query):
pow_result = tdSql.getResult(pow_query)
origin_result = tdSql.getResult(origin_query)
auto_result =[]
for row in origin_result:
row_check = []
for elem in row:
......@@ -93,7 +93,7 @@ class TDTestCase:
if auto_result[row_index][col_index] == None and not (auto_result[row_index][col_index] == None and elem == None):
check_status = False
elif auto_result[row_index][col_index] != None and (auto_result[row_index][col_index] - elem > 0.00000001):
check_status = False
check_status = False
else:
pass
if not check_status:
......@@ -101,7 +101,7 @@ class TDTestCase:
sys.exit(1)
else:
tdLog.info("acos value check pass , it work as expected ,sql is \"%s\" "%pow_query )
def test_errors(self):
error_sql_lists = [
"select acos from t1",
......@@ -135,42 +135,42 @@ class TDTestCase:
]
for error_sql in error_sql_lists:
tdSql.error(error_sql)
def support_types(self):
type_error_sql_lists = [
"select acos(ts) from t1" ,
"select acos(ts) from t1" ,
"select acos(c7) from t1",
"select acos(c8) from t1",
"select acos(c9) from t1",
"select acos(ts) from ct1" ,
"select acos(ts) from ct1" ,
"select acos(c7) from ct1",
"select acos(c8) from ct1",
"select acos(c9) from ct1",
"select acos(ts) from ct3" ,
"select acos(ts) from ct3" ,
"select acos(c7) from ct3",
"select acos(c8) from ct3",
"select acos(c9) from ct3",
"select acos(ts) from ct4" ,
"select acos(ts) from ct4" ,
"select acos(c7) from ct4",
"select acos(c8) from ct4",
"select acos(c9) from ct4",
"select acos(ts) from stb1" ,
"select acos(ts) from stb1" ,
"select acos(c7) from stb1",
"select acos(c8) from stb1",
"select acos(c9) from stb1" ,
"select acos(ts) from stbbb1" ,
"select acos(ts) from stbbb1" ,
"select acos(c7) from stbbb1",
"select acos(ts) from tbname",
"select acos(c9) from tbname"
]
for type_sql in type_error_sql_lists:
tdSql.error(type_sql)
type_sql_lists = [
"select acos(c1) from t1",
"select acos(c2) from t1",
......@@ -200,16 +200,16 @@ class TDTestCase:
"select acos(c5) from stb1",
"select acos(c6) from stb1",
"select acos(c6) as alisb from stb1",
"select acos(c6) alisb from stb1",
"select acos(c6) as alisb from stb1",
"select acos(c6) alisb from stb1",
]
for type_sql in type_sql_lists:
tdSql.query(type_sql)
def basic_acos_function(self):
# basic query
# basic query
tdSql.query("select c1 from ct3")
tdSql.checkRows(0)
tdSql.query("select c1 from t1")
......@@ -250,7 +250,7 @@ class TDTestCase:
tdSql.checkData(5, 5, None)
self.check_result_auto_acos( "select abs(c1), abs(c2), abs(c3) , abs(c4), abs(c5) from t1", "select acos(abs(c1)), acos(abs(c2)) ,acos(abs(c3)), acos(abs(c4)), acos(abs(c5)) from t1")
# used for sub table
tdSql.query("select c2 ,acos(c2) from ct1")
tdSql.checkData(0, 1, None)
......@@ -266,7 +266,7 @@ class TDTestCase:
tdSql.checkData(5 , 2, None)
self.check_result_auto_acos( "select c1, c2, c3 , c4, c5 from ct1", "select acos(c1), acos(c2) ,acos(c3), acos(c4), acos(c5) from ct1")
# nest query for acos functions
tdSql.query("select c4 , acos(c4) ,acos(acos(c4)) , acos(acos(acos(c4))) from ct1;")
tdSql.checkData(0 , 0 , 88)
......@@ -284,21 +284,21 @@ class TDTestCase:
tdSql.checkData(11 , 2 , None)
tdSql.checkData(11 , 3 , None)
# used for stable table
# used for stable table
tdSql.query("select acos(c1) from stb1")
tdSql.checkRows(25)
# used for not exists table
tdSql.error("select acos(c1) from stbbb1")
tdSql.error("select acos(c1) from tbname")
tdSql.error("select acos(c1) from ct5")
# mix with common col
# mix with common col
tdSql.query("select c1, acos(c1) from ct1")
tdSql.query("select c2, acos(c2) from ct4")
# mix with common functions
tdSql.query("select c1, acos(c1),acos(c1), acos(acos(c1)) from ct4 ")
......@@ -306,7 +306,7 @@ class TDTestCase:
tdSql.checkData(0 , 1 ,None)
tdSql.checkData(0 , 2 ,None)
tdSql.checkData(0 , 3 ,None)
tdSql.checkData(3 , 0 , 6)
tdSql.checkData(3 , 1 ,None)
tdSql.checkData(3 , 2 ,None)
......@@ -327,8 +327,8 @@ class TDTestCase:
tdSql.query("select max(c5), count(c5) from stb1")
tdSql.query("select max(c5), count(c5) from ct1")
# # bug fix for compute
# # bug fix for compute
tdSql.query("select c1, acos(c1) -0 ,acos(c1-4)-0 from ct4 ")
tdSql.checkData(0, 0, None)
tdSql.checkData(0, 1, None)
......@@ -397,10 +397,10 @@ class TDTestCase:
tdSql.checkData(0,3,0.000000000)
tdSql.checkData(0,4,-0.100000000)
tdSql.checkData(0,5,2)
def pow_Arithmetic(self):
pass
def check_boundary_values(self):
PI=3.1415926
......@@ -429,11 +429,11 @@ class TDTestCase:
f"insert into sub1_bound values ( now()+1s, 2147483648, 9223372036854775808, 32768, 128, 3.40E+38, 1.7e+308, True, 'binary_tb1', 'nchar_tb1', now() )"
)
self.check_result_auto_acos( "select abs(c1), abs(c2), abs(c3) , abs(c4), abs(c5) from sub1_bound ", "select acos(abs(c1)), acos(abs(c2)) ,acos(abs(c3)), acos(abs(c4)), acos(abs(c5)) from sub1_bound")
self.check_result_auto_acos( "select c1, c2, c3 , c3, c2 ,c1 from sub1_bound ", "select acos(c1), acos(c2) ,acos(c3), acos(c3), acos(c2) ,acos(c1) from sub1_bound")
self.check_result_auto_acos("select abs(abs(abs(abs(abs(abs(abs(abs(abs(c1))))))))) nest_col_func from sub1_bound" , "select acos(abs(c1)) from sub1_bound" )
# check basic elem for table per row
tdSql.query("select acos(abs(c1)) ,acos(abs(c2)) , acos(abs(c3)) , acos(abs(c4)), acos(abs(c5)), acos(abs(c6)) from sub1_bound ")
tdSql.checkData(0,0,None)
......@@ -492,41 +492,41 @@ class TDTestCase:
self.check_result_auto_acos( " select t1,c5 from stb1 where c1 > 0 order by tbname " , "select acos(t1) ,acos(c5) from stb1 where c1 > 0 order by tbname" )
self.check_result_auto_acos( " select t1,c5 from stb1 where c1 > 0 order by tbname " , "select acos(t1) , acos(c5) 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()
tdLog.printNoPrefix("==========step2:test errors ==============")
tdLog.printNoPrefix("==========step2:test errors ==============")
self.test_errors()
tdLog.printNoPrefix("==========step3:support types ============")
tdLog.printNoPrefix("==========step3:support types ============")
self.support_types()
tdLog.printNoPrefix("==========step4: acos basic query ============")
tdLog.printNoPrefix("==========step4: acos basic query ============")
self.basic_acos_function()
tdLog.printNoPrefix("==========step5: big number acos query ============")
tdLog.printNoPrefix("==========step5: big number acos query ============")
self.test_big_number()
tdLog.printNoPrefix("==========step6: acos boundary query ============")
tdLog.printNoPrefix("==========step6: acos boundary query ============")
self.check_boundary_values()
tdLog.printNoPrefix("==========step7: acos filter query ============")
tdLog.printNoPrefix("==========step7: acos filter query ============")
self.abs_func_filter()
tdLog.printNoPrefix("==========step7: acos filter query ============")
tdLog.printNoPrefix("==========step7: acos filter query ============")
self.abs_func_filter()
......
......@@ -9,14 +9,14 @@ from util.cases import *
class TDTestCase:
updatecfgDict = {'debugFlag': 143 ,"cDebugFlag":143,"uDebugFlag":143 ,"rpcDebugFlag":143 , "tmrDebugFlag":143 ,
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 ,"udfDebugFlag":143}
def init(self, conn, powSql):
tdLog.debug(f"start to excute {__file__}")
tdSql.init(conn.cursor())
self.PI =3.1415926
def prepare_datas(self):
tdSql.execute(
'''create table stb1
......@@ -24,7 +24,7 @@ class TDTestCase:
tags (t1 int)
'''
)
tdSql.execute(
'''
create table t1
......@@ -66,14 +66,14 @@ class TDTestCase:
( '2023-02-21 01:01:01.000', NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL )
'''
)
def check_result_auto_asin(self ,origin_query , pow_query):
pow_result = tdSql.getResult(pow_query)
origin_result = tdSql.getResult(origin_query)
auto_result =[]
for row in origin_result:
row_check = []
for elem in row:
......@@ -93,7 +93,7 @@ class TDTestCase:
if auto_result[row_index][col_index] == None and not (auto_result[row_index][col_index] == None and elem == None):
check_status = False
elif auto_result[row_index][col_index] != None and (auto_result[row_index][col_index] - elem > 0.00000001):
check_status = False
check_status = False
else:
pass
if not check_status:
......@@ -101,7 +101,7 @@ class TDTestCase:
sys.exit(1)
else:
tdLog.info("asin value check pass , it work as expected ,sql is \"%s\" "%pow_query )
def test_errors(self):
error_sql_lists = [
"select asin from t1",
......@@ -135,42 +135,42 @@ class TDTestCase:
]
for error_sql in error_sql_lists:
tdSql.error(error_sql)
def support_types(self):
type_error_sql_lists = [
"select asin(ts) from t1" ,
"select asin(ts) from t1" ,
"select asin(c7) from t1",
"select asin(c8) from t1",
"select asin(c9) from t1",
"select asin(ts) from ct1" ,
"select asin(ts) from ct1" ,
"select asin(c7) from ct1",
"select asin(c8) from ct1",
"select asin(c9) from ct1",
"select asin(ts) from ct3" ,
"select asin(ts) from ct3" ,
"select asin(c7) from ct3",
"select asin(c8) from ct3",
"select asin(c9) from ct3",
"select asin(ts) from ct4" ,
"select asin(ts) from ct4" ,
"select asin(c7) from ct4",
"select asin(c8) from ct4",
"select asin(c9) from ct4",
"select asin(ts) from stb1" ,
"select asin(ts) from stb1" ,
"select asin(c7) from stb1",
"select asin(c8) from stb1",
"select asin(c9) from stb1" ,
"select asin(ts) from stbbb1" ,
"select asin(ts) from stbbb1" ,
"select asin(c7) from stbbb1",
"select asin(ts) from tbname",
"select asin(c9) from tbname"
]
for type_sql in type_error_sql_lists:
tdSql.error(type_sql)
type_sql_lists = [
"select asin(c1) from t1",
"select asin(c2) from t1",
......@@ -200,16 +200,16 @@ class TDTestCase:
"select asin(c5) from stb1",
"select asin(c6) from stb1",
"select asin(c6) as alisb from stb1",
"select asin(c6) alisb from stb1",
"select asin(c6) as alisb from stb1",
"select asin(c6) alisb from stb1",
]
for type_sql in type_sql_lists:
tdSql.query(type_sql)
def basic_asin_function(self):
# basic query
# basic query
tdSql.query("select c1 from ct3")
tdSql.checkRows(0)
tdSql.query("select c1 from t1")
......@@ -250,7 +250,7 @@ class TDTestCase:
tdSql.checkData(5, 5, None)
self.check_result_auto_asin( "select abs(c1), abs(c2), abs(c3) , abs(c4), abs(c5) from t1", "select asin(abs(c1)), asin(abs(c2)) ,asin(abs(c3)), asin(abs(c4)), asin(abs(c5)) from t1")
# used for sub table
tdSql.query("select c2 ,asin(c2) from ct1")
tdSql.checkData(0, 1, None)
......@@ -266,7 +266,7 @@ class TDTestCase:
tdSql.checkData(5 , 2, None)
self.check_result_auto_asin( "select c1, c2, c3 , c4, c5 from ct1", "select asin(c1), asin(c2) ,asin(c3), asin(c4), asin(c5) from ct1")
# nest query for asin functions
tdSql.query("select c4 , asin(c4) ,asin(asin(c4)) , asin(asin(asin(c4))) from ct1;")
tdSql.checkData(0 , 0 , 88)
......@@ -284,21 +284,21 @@ class TDTestCase:
tdSql.checkData(11 , 2 , None)
tdSql.checkData(11 , 3 , None)
# used for stable table
# used for stable table
tdSql.query("select asin(c1) from stb1")
tdSql.checkRows(25)
# used for not exists table
tdSql.error("select asin(c1) from stbbb1")
tdSql.error("select asin(c1) from tbname")
tdSql.error("select asin(c1) from ct5")
# mix with common col
# mix with common col
tdSql.query("select c1, asin(c1) from ct1")
tdSql.query("select c2, asin(c2) from ct4")
# mix with common functions
tdSql.query("select c1, asin(c1),asin(c1), asin(asin(c1)) from ct4 ")
......@@ -306,7 +306,7 @@ class TDTestCase:
tdSql.checkData(0 , 1 ,None)
tdSql.checkData(0 , 2 ,None)
tdSql.checkData(0 , 3 ,None)
tdSql.checkData(3 , 0 , 6)
tdSql.checkData(3 , 1 ,None)
tdSql.checkData(3 , 2 ,None)
......@@ -327,8 +327,8 @@ class TDTestCase:
tdSql.query("select max(c5), count(c5) from stb1")
tdSql.query("select max(c5), count(c5) from ct1")
# # bug fix for compute
# # bug fix for compute
tdSql.query("select c1, asin(c1) -0 ,asin(c1-4)-0 from ct4 ")
tdSql.checkData(0, 0, None)
tdSql.checkData(0, 1, None)
......@@ -397,10 +397,10 @@ class TDTestCase:
tdSql.checkData(0,3,1.000000000)
tdSql.checkData(0,4,0.900000000)
tdSql.checkData(0,5,2)
def pow_Arithmetic(self):
pass
def check_boundary_values(self):
PI=3.1415926
......@@ -429,11 +429,11 @@ class TDTestCase:
f"insert into sub1_bound values ( now()+1s, 2147483648, 9223372036854775808, 32768, 128, 3.40E+38, 1.7e+308, True, 'binary_tb1', 'nchar_tb1', now() )"
)
self.check_result_auto_asin( "select abs(c1), abs(c2), abs(c3) , abs(c4), abs(c5) from sub1_bound ", "select asin(abs(c1)), asin(abs(c2)) ,asin(abs(c3)), asin(abs(c4)), asin(abs(c5)) from sub1_bound")
self.check_result_auto_asin( "select c1, c2, c3 , c3, c2 ,c1 from sub1_bound ", "select asin(c1), asin(c2) ,asin(c3), asin(c3), asin(c2) ,asin(c1) from sub1_bound")
self.check_result_auto_asin("select abs(abs(abs(abs(abs(abs(abs(abs(abs(c1))))))))) nest_col_func from sub1_bound" , "select asin(abs(c1)) from sub1_bound" )
# check basic elem for table per row
tdSql.query("select asin(abs(c1)) ,asin(abs(c2)) , asin(abs(c3)) , asin(abs(c4)), asin(abs(c5)), asin(abs(c6)) from sub1_bound ")
tdSql.checkData(0,0,None)
......@@ -492,37 +492,37 @@ class TDTestCase:
self.check_result_auto_asin( " select t1,c5 from stb1 where c1 > 0 order by tbname " , "select asin(t1) ,asin(c5) from stb1 where c1 > 0 order by tbname" )
self.check_result_auto_asin( " select t1,c5 from stb1 where c1 > 0 order by tbname " , "select asin(t1) , asin(c5) 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()
tdLog.printNoPrefix("==========step2:test errors ==============")
tdLog.printNoPrefix("==========step2:test errors ==============")
self.test_errors()
tdLog.printNoPrefix("==========step3:support types ============")
tdLog.printNoPrefix("==========step3:support types ============")
self.support_types()
tdLog.printNoPrefix("==========step4: asin basic query ============")
tdLog.printNoPrefix("==========step4: asin basic query ============")
self.basic_asin_function()
tdLog.printNoPrefix("==========step5: big number asin query ============")
tdLog.printNoPrefix("==========step5: big number asin query ============")
self.test_big_number()
tdLog.printNoPrefix("==========step6: asin boundary query ============")
tdLog.printNoPrefix("==========step6: asin boundary query ============")
self.check_boundary_values()
tdLog.printNoPrefix("==========step7: asin filter query ============")
tdLog.printNoPrefix("==========step7: asin filter query ============")
self.abs_func_filter()
......
......@@ -9,13 +9,13 @@ from util.cases import *
class TDTestCase:
updatecfgDict = {'debugFlag': 143 ,"cDebugFlag":143,"uDebugFlag":143 ,"rpcDebugFlag":143 , "tmrDebugFlag":143 ,
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 ,"udfDebugFlag":143}
def init(self, conn, powSql):
tdLog.debug(f"start to excute {__file__}")
tdSql.init(conn.cursor())
def prepare_datas(self):
tdSql.execute(
'''create table stb1
......@@ -23,7 +23,7 @@ class TDTestCase:
tags (t1 int)
'''
)
tdSql.execute(
'''
create table t1
......@@ -65,14 +65,14 @@ class TDTestCase:
( '2023-02-21 01:01:01.000', NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL )
'''
)
def check_result_auto_atan(self ,origin_query , pow_query):
pow_result = tdSql.getResult(pow_query)
origin_result = tdSql.getResult(origin_query)
auto_result =[]
for row in origin_result:
row_check = []
for elem in row:
......@@ -90,7 +90,7 @@ class TDTestCase:
if auto_result[row_index][col_index] == None and not (auto_result[row_index][col_index] == None and elem == None):
check_status = False
elif auto_result[row_index][col_index] != None and (auto_result[row_index][col_index] - elem > 0.00000001):
check_status = False
check_status = False
else:
pass
if not check_status:
......@@ -98,7 +98,7 @@ class TDTestCase:
sys.exit(1)
else:
tdLog.info("atan value check pass , it work as expected ,sql is \"%s\" "%pow_query )
def test_errors(self):
error_sql_lists = [
"select atan from t1",
......@@ -132,42 +132,42 @@ class TDTestCase:
]
for error_sql in error_sql_lists:
tdSql.error(error_sql)
def support_types(self):
type_error_sql_lists = [
"select atan(ts) from t1" ,
"select atan(ts) from t1" ,
"select atan(c7) from t1",
"select atan(c8) from t1",
"select atan(c9) from t1",
"select atan(ts) from ct1" ,
"select atan(ts) from ct1" ,
"select atan(c7) from ct1",
"select atan(c8) from ct1",
"select atan(c9) from ct1",
"select atan(ts) from ct3" ,
"select atan(ts) from ct3" ,
"select atan(c7) from ct3",
"select atan(c8) from ct3",
"select atan(c9) from ct3",
"select atan(ts) from ct4" ,
"select atan(ts) from ct4" ,
"select atan(c7) from ct4",
"select atan(c8) from ct4",
"select atan(c9) from ct4",
"select atan(ts) from stb1" ,
"select atan(ts) from stb1" ,
"select atan(c7) from stb1",
"select atan(c8) from stb1",
"select atan(c9) from stb1" ,
"select atan(ts) from stbbb1" ,
"select atan(ts) from stbbb1" ,
"select atan(c7) from stbbb1",
"select atan(ts) from tbname",
"select atan(c9) from tbname"
]
for type_sql in type_error_sql_lists:
tdSql.error(type_sql)
type_sql_lists = [
"select atan(c1) from t1",
"select atan(c2) from t1",
......@@ -197,16 +197,16 @@ class TDTestCase:
"select atan(c5) from stb1",
"select atan(c6) from stb1",
"select atan(c6) as alisb from stb1",
"select atan(c6) alisb from stb1",
"select atan(c6) as alisb from stb1",
"select atan(c6) alisb from stb1",
]
for type_sql in type_sql_lists:
tdSql.query(type_sql)
def basic_atan_function(self):
# basic query
# basic query
tdSql.query("select c1 from ct3")
tdSql.checkRows(0)
tdSql.query("select c1 from t1")
......@@ -247,7 +247,7 @@ class TDTestCase:
tdSql.checkData(5, 5, None)
self.check_result_auto_atan( "select abs(c1), abs(c2), abs(c3) , abs(c4), abs(c5) from t1", "select atan(abs(c1)), atan(abs(c2)) ,atan(abs(c3)), atan(abs(c4)), atan(abs(c5)) from t1")
# used for sub table
tdSql.query("select c2 ,atan(c2) from ct1")
tdSql.checkData(0, 1, 1.570785077)
......@@ -263,7 +263,7 @@ class TDTestCase:
tdSql.checkData(5 , 2, None)
self.check_result_auto_atan( "select c1, c2, c3 , c4, c5 from ct1", "select atan(c1), atan(c2) ,atan(c3), atan(c4), atan(c5) from ct1")
# nest query for atan functions
tdSql.query("select c4 , atan(c4) ,atan(atan(c4)) , atan(atan(atan(c4))) from ct1;")
tdSql.checkData(0 , 0 , 88)
......@@ -281,21 +281,21 @@ class TDTestCase:
tdSql.checkData(11 , 2 , -1.000958403)
tdSql.checkData(11 , 3 , -0.785877135)
# used for stable table
# used for stable table
tdSql.query("select atan(c1) from stb1")
tdSql.checkRows(25)
# used for not exists table
tdSql.error("select atan(c1) from stbbb1")
tdSql.error("select atan(c1) from tbname")
tdSql.error("select atan(c1) from ct5")
# mix with common col
# mix with common col
tdSql.query("select c1, atan(c1) from ct1")
tdSql.query("select c2, atan(c2) from ct4")
# mix with common functions
tdSql.query("select c1, atan(c1),atan(c1), atan(atan(c1)) from ct4 ")
......@@ -303,7 +303,7 @@ class TDTestCase:
tdSql.checkData(0 , 1 ,None)
tdSql.checkData(0 , 2 ,None)
tdSql.checkData(0 , 3 ,None)
tdSql.checkData(3 , 0 , 6)
tdSql.checkData(3 , 1 ,1.405647649)
tdSql.checkData(3 , 2 ,1.405647649)
......@@ -324,8 +324,8 @@ class TDTestCase:
tdSql.query("select max(c5), count(c5) from stb1")
tdSql.query("select max(c5), count(c5) from ct1")
# # bug fix for compute
# # bug fix for compute
tdSql.query("select c1, atan(c1) -0 ,atan(c1-4)-0 from ct4 ")
tdSql.checkData(0, 0, None)
tdSql.checkData(0, 1, None)
......@@ -394,10 +394,10 @@ class TDTestCase:
tdSql.checkData(0,3,0.000000000)
tdSql.checkData(0,4,-0.100000000)
tdSql.checkData(0,5,0.000000000)
def pow_Arithmetic(self):
pass
def check_boundary_values(self):
PI=3.1415926
......@@ -426,11 +426,11 @@ class TDTestCase:
f"insert into sub1_bound values ( now()+1s, 2147483648, 9223372036854775808, 32768, 128, 3.40E+38, 1.7e+308, True, 'binary_tb1', 'nchar_tb1', now() )"
)
self.check_result_auto_atan( "select abs(c1), abs(c2), abs(c3) , abs(c4), abs(c5) from sub1_bound ", "select atan(abs(c1)), atan(abs(c2)) ,atan(abs(c3)), atan(abs(c4)), atan(abs(c5)) from sub1_bound")
self.check_result_auto_atan( "select c1, c2, c3 , c3, c2 ,c1 from sub1_bound ", "select atan(c1), atan(c2) ,atan(c3), atan(c3), atan(c2) ,atan(c1) from sub1_bound")
self.check_result_auto_atan("select abs(abs(abs(abs(abs(abs(abs(abs(abs(c1))))))))) nest_col_func from sub1_bound" , "select atan(abs(c1)) from sub1_bound" )
# check basic elem for table per row
tdSql.query("select atan(abs(c1)) ,atan(abs(c2)) , atan(abs(c3)) , atan(abs(c4)), atan(abs(c5)), atan(abs(c6)) from sub1_bound ")
tdSql.checkData(0,0,math.atan(2147483647))
......@@ -490,36 +490,36 @@ class TDTestCase:
self.check_result_auto_atan( " select t1,c5 from stb1 where c1 > 0 order by tbname " , "select atan(t1) ,atan(c5) from stb1 where c1 > 0 order by tbname" )
self.check_result_auto_atan( " select t1,c5 from stb1 where c1 > 0 order by tbname " , "select atan(t1) , atan(c5) 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()
tdLog.printNoPrefix("==========step2:test errors ==============")
tdLog.printNoPrefix("==========step2:test errors ==============")
self.test_errors()
tdLog.printNoPrefix("==========step3:support types ============")
tdLog.printNoPrefix("==========step3:support types ============")
self.support_types()
tdLog.printNoPrefix("==========step4: atan basic query ============")
tdLog.printNoPrefix("==========step4: atan basic query ============")
self.basic_atan_function()
tdLog.printNoPrefix("==========step5: big number atan query ============")
tdLog.printNoPrefix("==========step5: big number atan query ============")
self.test_big_number()
tdLog.printNoPrefix("==========step6: atan boundary query ============")
tdLog.printNoPrefix("==========step6: atan boundary query ============")
self.check_boundary_values()
tdLog.printNoPrefix("==========step7: atan filter query ============")
tdLog.printNoPrefix("==========step7: atan filter query ============")
self.abs_func_filter()
......@@ -527,7 +527,7 @@ class TDTestCase:
self.support_super_table_test()
def stop(self):
tdSql.close()
......
from util.log import *
from util.sql import *
from util.cases import *
from util.sqlset import *
class TDTestCase:
def init(self, conn, logSql):
tdLog.debug("start to execute %s" % __file__)
tdSql.init(conn.cursor())
tdSql.init(conn.cursor(),logSql)
self.setsql = TDSetSql()
self.rowNum = 10
self.ts = 1537146000000
def run(self):
self.ntbname = 'ntb'
self.stbname = 'stb'
self.column_dict = {
'ts':'timestamp',
'c1':'int',
'c2':'float',
'c3':'double',
'c4':'timestamp'
}
self.tag_dict = {
't0':'int'
}
# The number of tag_values should be same as tbnum
self.tbnum = 2
self.tag_values = [
f'10',
f'100'
]
def query_stb(self,k,stbname,tbnum,rownum):
tdSql.query(f'select count({k}) from {stbname}')
tdSql.checkEqual(tdSql.queryResult[0][0],tbnum*rownum)
tdSql.query(f'select count({k}) from {stbname} where ts <={self.ts+self.rowNum-1}')
tdSql.checkEqual(tdSql.queryResult[0][0],tbnum*rownum)
tdSql.query(f'select count({k}) from {stbname} where ts <={self.ts+self.rowNum-2}')
tdSql.checkEqual(tdSql.queryResult[0][0],tbnum*(rownum-1))
def query_ctb(self,k,i,stbname,rownum):
tdSql.query(f'select count({k}) from {stbname}_{i}')
tdSql.checkEqual(tdSql.queryResult[0][0],rownum)
tdSql.query(f'select count({k}) from {stbname}_{i} where ts <={self.ts+self.rowNum-1}')
tdSql.checkEqual(tdSql.queryResult[0][0],rownum)
tdSql.query(f'select count({k}) from {stbname}_{i} where ts <={self.ts+self.rowNum-2}')
tdSql.checkEqual(tdSql.queryResult[0][0],rownum-1)
def query_ntb(self,k,ntbname,rownum):
tdSql.query(f'select count({k}) from {ntbname}')
tdSql.checkEqual(tdSql.queryResult[0][0],rownum)
tdSql.query(f'select count({k}) from {ntbname} where ts <={self.ts+self.rowNum-1}')
tdSql.checkEqual(tdSql.queryResult[0][0],rownum)
tdSql.query(f'select count({k}) from {ntbname} where ts <={self.ts+self.rowNum-2}')
tdSql.checkEqual(tdSql.queryResult[0][0],rownum-1)
def count_query_stb(self,column_dict,tag_dict,stbname,tbnum,rownum):
tdSql.query(f'select count(tbname) from {stbname}')
tdSql.checkEqual(tdSql.queryResult[0][0],tbnum*rownum)
tdSql.query(f'SELECT count(*) from (select distinct tbname from {stbname})')
tdSql.checkEqual(tdSql.queryResult[0][0],tbnum)
for k in column_dict.keys():
self.query_stb(k,stbname,tbnum,rownum)
for k in tag_dict.keys():
self.query_stb(k,stbname,tbnum,rownum)
def count_query_ctb(self,column_dict,tag_dict,stbname,tbnum,rownum):
for i in range(tbnum):
tdSql.query(f'select count(tbname) from {stbname}_{i}')
tdSql.checkEqual(tdSql.queryResult[0][0],rownum)
for k in column_dict.keys():
self.query_ctb(k,i,stbname,rownum)
for k in tag_dict.keys():
self.query_ctb(k,i,stbname,rownum)
def count_query_ntb(self,column_dict,ntbname,rownum):
tdSql.query(f'select count(tbname) from {ntbname}')
tdSql.checkEqual(tdSql.queryResult[0][0],rownum)
for k in column_dict.keys():
self.query_ntb(k,ntbname,rownum)
def insert_data(self,column_dict,tbname,row_num):
insert_sql = self.setsql.set_insertsql(column_dict,tbname)
for i in range(row_num):
insert_list = []
self.setsql.insert_values(column_dict,i,insert_sql,insert_list,self.ts)
def check_ntb(self):
tdSql.prepare()
tdSql.execute('''create table stb(ts timestamp, col1 tinyint, col2 smallint, col3 int, col4 bigint, col5 float, col6 double,
col7 bool, col8 binary(20), col9 nchar(20), col11 tinyint unsigned, col12 smallint unsigned, col13 int unsigned, col14 bigint unsigned) tags(loc nchar(20))''')
tdSql.execute("create table stb_1 using stb tags('beijing')")
tdSql.execute("create table stb_2 using stb tags('shanghai')")
tdSql.execute('''create table ntb(ts timestamp, col1 tinyint, col2 smallint, col3 int, col4 bigint, col5 float, col6 double,
col7 bool, col8 binary(20), col9 nchar(20), col11 tinyint unsigned, col12 smallint unsigned, col13 int unsigned, col14 bigint unsigned)''')
for i in range(self.rowNum):
tdSql.execute("insert into stb_1 values(%d, %d, %d, %d, %d, %f, %f, %d, 'taosdata%d', '涛思数据%d', %d, %d, %d, %d)"
% (self.ts + i, i + 1, i + 1, i + 1, i + 1, i + 0.1, i + 0.1, i % 2, i + 1, i + 1, i + 1, i + 1, i + 1, i + 1))
tdSql.execute("insert into stb_2 values(%d, %d, %d, %d, %d, %f, %f, %d, 'taosdata%d', '涛思数据%d', %d, %d, %d, %d)"
% (self.ts + i, i + 1, i + 1, i + 1, i + 1, i + 0.1, i + 0.1, i % 2, i + 1, i + 1, i + 1, i + 1, i + 1, i + 1))
for i in range(self.rowNum):
tdSql.execute("insert into ntb values(%d, %d, %d, %d, %d, %f, %f, %d, 'taosdata%d', '涛思数据%d', %d, %d, %d, %d)"
% (self.ts + i, i + 1, i + 1, i + 1, i + 1, i + 0.1, i + 0.1, i % 2, i + 1, i + 1, i + 1, i + 1, i + 1, i + 1))
tdSql.query("select count(*) from stb")
tdSql.checkData(0,0,20)
tdSql.query("select count(*) from db.stb")
tdSql.checkData(0,0,20)
tdSql.query("select count(ts) from stb")
tdSql.checkData(0,0,20)
tdSql.query("select count(ts) from db.stb")
tdSql.checkData(0,0,20)
tdSql.query("select count(col1) from stb")
tdSql.checkData(0,0,20)
tdSql.query("select count(col1) from db.stb")
tdSql.checkData(0,0,20)
tdSql.query("select count(col2) from stb")
tdSql.checkData(0,0,20)
tdSql.query("select count(col2) from db.stb")
tdSql.checkData(0,0,20)
tdSql.query("select count(col3) from stb")
tdSql.checkData(0,0,20)
tdSql.query("select count(col3) from db.stb")
tdSql.checkData(0,0,20)
tdSql.query("select count(col4) from stb")
tdSql.checkData(0,0,20)
tdSql.query("select count(col4) from db.stb")
tdSql.checkData(0,0,20)
tdSql.query("select count(col5) from stb")
tdSql.checkData(0,0,20)
tdSql.query("select count(col5) from db.stb")
tdSql.checkData(0,0,20)
tdSql.query("select count(col6) from stb")
tdSql.checkData(0,0,20)
tdSql.query("select count(col6) from db.stb")
tdSql.checkData(0,0,20)
tdSql.query("select count(col7) from stb")
tdSql.checkData(0,0,20)
tdSql.query("select count(col7) from db.stb")
tdSql.checkData(0,0,20)
tdSql.query("select count(col8) from stb")
tdSql.checkData(0,0,20)
tdSql.query("select count(col8) from db.stb")
tdSql.checkData(0,0,20)
tdSql.query("select count(col9) from stb")
tdSql.checkData(0,0,20)
tdSql.query("select count(col9) from db.stb")
tdSql.checkData(0,0,20)
tdSql.query("select count(col11) from stb")
tdSql.checkData(0,0,20)
tdSql.query("select count(col11) from db.stb")
tdSql.checkData(0,0,20)
tdSql.query("select count(col12) from stb")
tdSql.checkData(0,0,20)
tdSql.query("select count(col12) from db.stb")
tdSql.checkData(0,0,20)
tdSql.query("select count(col13) from stb")
tdSql.checkData(0,0,20)
tdSql.query("select count(col13) from db.stb")
tdSql.checkData(0,0,20)
tdSql.query("select count(col14) from stb")
tdSql.checkData(0,0,20)
tdSql.query("select count(col14) from db.stb")
tdSql.checkData(0,0,20)
tdSql.query("select count(ts) from db.stb_1")
tdSql.checkData(0,0,10)
tdSql.query("select count(ts) from db.stb_1")
tdSql.checkData(0,0,10)
tdSql.query("select count(col1) from stb_1")
tdSql.checkData(0,0,10)
tdSql.query("select count(col1) from db.stb_1")
tdSql.checkData(0,0,10)
tdSql.query("select count(col2) from stb_1")
tdSql.checkData(0,0,10)
tdSql.query("select count(col2) from db.stb_1")
tdSql.checkData(0,0,10)
tdSql.query("select count(col3) from stb_1")
tdSql.checkData(0,0,10)
tdSql.query("select count(col3) from db.stb_1")
tdSql.checkData(0,0,10)
tdSql.query("select count(col4) from stb_1")
tdSql.checkData(0,0,10)
tdSql.query("select count(col4) from db.stb_1")
tdSql.checkData(0,0,10)
tdSql.query("select count(col5) from stb_1")
tdSql.checkData(0,0,10)
tdSql.query("select count(col5) from db.stb_1")
tdSql.checkData(0,0,10)
tdSql.query("select count(col6) from stb_1")
tdSql.checkData(0,0,10)
tdSql.query("select count(col6) from db.stb_1")
tdSql.checkData(0,0,10)
tdSql.query("select count(col7) from stb_1")
tdSql.checkData(0,0,10)
tdSql.query("select count(col7) from db.stb_1")
tdSql.checkData(0,0,10)
tdSql.query("select count(col8) from stb_1")
tdSql.checkData(0,0,10)
tdSql.query("select count(col8) from db.stb_1")
tdSql.checkData(0,0,10)
tdSql.query("select count(col9) from stb_1")
tdSql.checkData(0,0,10)
tdSql.query("select count(col9) from db.stb_1")
tdSql.checkData(0,0,10)
tdSql.query("select count(col11) from stb_1")
tdSql.checkData(0,0,10)
tdSql.query("select count(col11) from db.stb_1")
tdSql.checkData(0,0,10)
tdSql.query("select count(col12) from stb_1")
tdSql.checkData(0,0,10)
tdSql.query("select count(col12) from db.stb_1")
tdSql.checkData(0,0,10)
tdSql.query("select count(col13) from stb_1")
tdSql.checkData(0,0,10)
tdSql.query("select count(col13) from db.stb_1")
tdSql.checkData(0,0,10)
tdSql.query("select count(col14) from stb_1")
tdSql.checkData(0,0,10)
tdSql.query("select count(col14) from db.stb_1")
tdSql.checkData(0,0,10)
tdSql.query("select count(col1) from stb_1 group by col7")
tdSql.checkRows(2)
tdSql.execute("insert into stb_1 values(now,null,null,null,null,null,null,null,null,null,null,null,null,null)")
tdSql.query("select count(col1) from stb_1")
tdSql.checkData(0,0,10)
tdSql.query("select count(col1),count(ts) from stb_1")
tdSql.checkData(0,0,10)
tdSql.checkData(0,1,11)
tdSql.query("select count(col1) from db.stb_1")
tdSql.checkData(0,0,10)
tdSql.query("select count(col1),count(ts) from db.stb_1")
tdSql.checkData(0,0,10)
tdSql.checkData(0,1,11)
tdSql.query("select count(col1) from stb")
tdSql.checkData(0,0,20)
tdSql.query("select count(col1),count(ts) from stb")
tdSql.checkData(0,0,20)
tdSql.checkData(0,1,21)
tdSql.query("select count(col1) from db.stb")
tdSql.checkData(0,0,20)
tdSql.query("select count(col1),count(ts) from db.stb")
tdSql.checkData(0,0,20)
tdSql.checkData(0,1,21)
tdSql.query("select count(col1) from stb_1 group by col7")
tdSql.checkRows(3)
tdSql.query("select count(col1) from stb_2 group by col7")
tdSql.checkRows(2)
tdSql.query("select count(col1) from stb group by col7")
tdSql.checkRows(3)
tdSql.execute(self.setsql.set_create_normaltable_sql(self.ntbname,self.column_dict))
tdSql.query(f'select count(tbname) from {self.ntbname}')
tdSql.checkRows(0)
tdSql.execute('flush database db')
tdSql.query(f'select count(tbname) from {self.ntbname}')
tdSql.checkRows(0)
self.insert_data(self.column_dict,self.ntbname,self.rowNum)
self.count_query_ntb(self.column_dict,self.ntbname,self.rowNum)
tdSql.execute('flush database db')
self.count_query_ntb(self.column_dict,self.ntbname,self.rowNum)
tdSql.execute('drop database db')
def check_stb(self):
tdSql.prepare()
tdSql.execute(self.setsql.set_create_stable_sql(self.stbname,self.column_dict,self.tag_dict))
for i in range(self.tbnum):
tdSql.execute(f'create table {self.stbname}_{i} using {self.stbname} tags({self.tag_values[i]})')
#!TODO
# tdSql.query(f'SELECT count(*) from (select distinct tbname from {self.stbname})')
# tdSql.checkEqual(tdSql.queryResult[0][0],self.tbnum)
tdSql.query(f'select count(tbname) from {self.stbname}')
tdSql.checkRows(0)
tdSql.execute('flush database db')
tdSql.query(f'select count(tbname) from {self.stbname}')
tdSql.checkRows(0)
#!TODO
# tdSql.query(f'SELECT count(*) from (select distinct tbname from {self.stbname})')
# tdSql.checkEqual(tdSql.queryResult[0][0],self.tbnum)
for i in range(self.tbnum):
self.insert_data(self.column_dict,f'{self.stbname}_{i}',self.rowNum)
self.count_query_stb(self.column_dict,self.tag_dict,self.stbname,self.tbnum,self.rowNum)
self.count_query_ctb(self.column_dict,self.tag_dict,self.stbname,self.tbnum,self.rowNum)
tdSql.execute('flush database db')
self.count_query_stb(self.column_dict,self.tag_dict,self.stbname,self.tbnum,self.rowNum)
self.count_query_ctb(self.column_dict,self.tag_dict,self.stbname,self.tbnum,self.rowNum)
tdSql.execute('drop database db')
def run(self):
self.check_stb()
self.check_ntb()
def stop(self):
tdSql.close()
tdLog.success("%s successfully executed" % __file__)
......
......@@ -50,7 +50,7 @@ class TDTestCase:
tdSql.checkData(0, 0, None)
tdSql.checkData(1, 0, None)
tdSql.execute('''create table stb(ts timestamp, col1 tinyint, col2 smallint, col3 int, col4 bigint, col5 float, col6 double,
tdSql.execute('''create table stb(ts timestamp, col1 tinyint, col2 smallint, col3 int, col4 bigint, col5 float, col6 double,
col7 bool, col8 binary(20), col9 nchar(20), col11 tinyint unsigned, col12 smallint unsigned, col13 int unsigned, col14 bigint unsigned) tags(loc nchar(20))''')
tdSql.execute("create table stb_1 using stb tags('beijing')")
tdSql.execute(
......@@ -115,7 +115,7 @@ class TDTestCase:
tdSql.query("select diff(col6) from stb_1")
tdSql.checkRows(10)
tdSql.execute('''create table stb1(ts timestamp, col1 tinyint, col2 smallint, col3 int, col4 bigint, col5 float, col6 double,
tdSql.execute('''create table stb1(ts timestamp, col1 tinyint, col2 smallint, col3 int, col4 bigint, col5 float, col6 double,
col7 bool, col8 binary(20), col9 nchar(20), col11 tinyint unsigned, col12 smallint unsigned, col13 int unsigned, col14 bigint unsigned) tags(loc nchar(20))''')
tdSql.execute("create table stb1_1 using stb tags('shanghai')")
......
......@@ -894,7 +894,7 @@ class TDTestCase:
tdSql.query(sql_common)
results= query_datas[0]
if operator == "+":
for data in query_datas[1:]:
for data in query_datas[1:]:
results += data
tdSql.checkData(0,0,results)
......
......@@ -45,7 +45,7 @@ class TDTestCase:
case1: limit offset base function test
case2: offset return valid
'''
return
return
def getBuildPath(self):
selfPath = os.path.dirname(os.path.realpath(__file__))
......@@ -71,7 +71,7 @@ class TDTestCase:
# self.create_tables();
self.ts = 1500000000000
# stop
# stop
def stop(self):
tdSql.close()
tdLog.success("%s successfully executed" % __file__)
......@@ -82,7 +82,7 @@ class TDTestCase:
def newcur(self,host,cfg):
user = "root"
password = "taosdata"
port =6030
port =6030
con=taos.connect(host=host, user=user, password=password, config=cfg ,port=port)
cur=con.cursor()
print(cur)
......@@ -92,7 +92,7 @@ class TDTestCase:
def create_tables(self,host,dbname,stbname,count):
buildPath = self.getBuildPath()
config = buildPath+ "../sim/dnode1/cfg/"
tsql=self.newcur(host,config)
tsql.execute("use %s" %dbname)
......@@ -111,7 +111,7 @@ class TDTestCase:
tsql.execute(sql)
sql = pre_create
# print(time.time())
# end sql
# end sql
if sql != pre_create:
# print(sql)
tsql.execute(sql)
......@@ -124,7 +124,7 @@ class TDTestCase:
def mutiThread_create_tables(self,host,dbname,stbname,vgroups,threadNumbers,childcount):
buildPath = self.getBuildPath()
config = buildPath+ "../sim/dnode1/cfg/"
tsql=self.newcur(host,config)
tdLog.debug("create database %s"%dbname)
tsql.execute("drop database if exists %s"%dbname)
......@@ -134,7 +134,7 @@ class TDTestCase:
threads = []
for i in range(threadNumbers):
tsql.execute("create stable %s%d(ts timestamp, c1 int, c2 binary(10)) tags(t1 int)"%(stbname,i))
threads.append(thd.Thread(target=self.create_tables, args=(host, dbname, stbname+"%d"%i, count,)))
threads.append(thd.Thread(target=self.create_tables, args=(host, dbname, stbname+"%d"%i, count,)))
start_time = time.time()
for tr in threads:
tr.start()
......@@ -144,7 +144,7 @@ class TDTestCase:
spendTime=end_time-start_time
speedCreate=threadNumbers*count/spendTime
tdLog.debug("spent %.2fs to create %d stable and %d table, create speed is %.2f table/s... [OK]"% (spendTime,threadNumbers,threadNumbers*count,speedCreate))
return
# def create_tables(self,host,dbname,stbname,vgroups,tcountStart,tcountStop):
......@@ -171,7 +171,7 @@ class TDTestCase:
# print(sql)
tsql.execute(sql)
sql = "insert into %s_%d values " %(stbname,i)
# end sql
# end sql
if sql != pre_insert:
# print(sql)
print(len(sql))
......@@ -186,7 +186,7 @@ class TDTestCase:
def mutiThread_insert_data(self, host, dbname, stbname, threadNumbers, chilCount, ts_start, childrowcount):
buildPath = self.getBuildPath()
config = buildPath+ "../sim/dnode1/cfg/"
tsql=self.newcur(host,config)
tdLog.debug("ready to inser data")
......@@ -195,7 +195,7 @@ class TDTestCase:
threads = []
for i in range(threadNumbers):
# tsql.execute("create stable %s%d(ts timestamp, c1 int, c2 binary(10)) tags(t1 int)"%(stbname,i))
threads.append(thd.Thread(target=self.insert_data, args=(host, dbname, stbname+"%d"%i, chilCount, ts_start, childrowcount,)))
threads.append(thd.Thread(target=self.insert_data, args=(host, dbname, stbname+"%d"%i, chilCount, ts_start, childrowcount,)))
start_time = time.time()
for tr in threads:
tr.start()
......@@ -226,10 +226,10 @@ class TDTestCase:
tdLog.info("taosd found in %s" % buildPath)
taosBenchbin = buildPath+ "/build/bin/taosBenchmark"
os.system("%s -f %s -y " %(taosBenchbin,jsonFile))
return
def taosBenchCreate(self,host,dropdb,dbname,stbname,vgroups,processNumbers,count):
# count=50000
buildPath = self.getBuildPath()
config = buildPath+ "../sim/dnode1/cfg/"
......@@ -243,7 +243,7 @@ class TDTestCase:
# tsql.getResult("show databases")
# print(tdSql.queryResult)
tsql.execute("use %s" %dbname)
threads = []
for i in range(processNumbers):
jsonfile="1-insert/Vgroups%d%d.json"%(vgroups,i)
......@@ -254,7 +254,7 @@ class TDTestCase:
os.system("sed -i 's/\"childtable_count\": 10000,/\"childtable_count\": %d,/g' %s "%(count,jsonfile))
os.system("sed -i 's/\"name\": \"stb1\",/\"name\": \"%s%d\",/g' %s "%(stbname,i,jsonfile))
os.system("sed -i 's/\"childtable_prefix\": \"stb1_\",/\"childtable_prefix\": \"%s%d_\",/g' %s "%(stbname,i,jsonfile))
threads.append(mp.Process(target=self.taosBench, args=("%s"%jsonfile,)))
threads.append(mp.Process(target=self.taosBench, args=("%s"%jsonfile,)))
start_time = time.time()
for tr in threads:
tr.start()
......@@ -276,8 +276,8 @@ class TDTestCase:
for i in range(stableCount):
tdSql.query("select count(*) from %s%d"%(stbname,i))
tdSql.checkData(0,0,rowsPerSTable)
return
return
# test case : Switch back and forth among the three queryPolicy(1\2\3)
def test_case1(self):
self.taosBenchCreate("127.0.0.1","no","db1", "stb1", 1, 2, 1*10)
......@@ -303,7 +303,7 @@ class TDTestCase:
tdSql.execute("reset query cache")
tdSql.query("select max(c1) from stb10;")
tdSql.checkData(0, 0, "%s"%maxQnode)
tdSql.query("select min(c1) from stb11;")
tdSql.query("select min(c1) from stb11;")
tdSql.checkData(0, 0, "%s"%minQnode)
tdSql.query("select c0,c1 from stb11_1 where (c0>1000) union select c0,c1 from stb11_1 where c0>2000;")
unionVnode=tdSql.queryResult
......@@ -352,8 +352,8 @@ class TDTestCase:
tdSql.execute("reset query cache")
tdSql.query("select max(c1) from stb10;")
assert maxQnode==tdSql.getData(0,0)
tdSql.query("select min(c1) from stb11;")
assert minQnode==tdSql.getData(0,0)
tdSql.query("select min(c1) from stb11;")
assert minQnode==tdSql.getData(0,0)
tdSql.error("select c0,c1 from stb11_1 where (c0>1000) union select c0,c1 from stb11_1 where c0>2000;")
tdSql.error("select c0,c1 from stb11_1 where (c0>1000) union all select c0,c1 from stb11_1 where c0>2000;")
......@@ -417,7 +417,7 @@ class TDTestCase:
tdLog.exit("alter queryPolicy to %d failed"%queryPolicy)
tdSql.execute("use db1;")
tdSql.error("select max(c1) from stb10;")
tdSql.error("select min(c1) from stb11;")
tdSql.error("select min(c1) from stb11;")
tdSql.error("select c0,c1 from stb11_1 where (c0>1000) union select c0,c1 from stb11_1 where c0>2000;")
tdSql.error("select c0,c1 from stb11_1 where (c0>1000) union all select c0,c1 from stb11_1 where c0>2000;")
......@@ -452,20 +452,20 @@ class TDTestCase:
tdSql.execute("reset query cache")
tdSql.error("select max(c1) from stb10;")
tdSql.error("select min(c1) from stb11;")
tdSql.error("select min(c1) from stb11;")
tdSql.error("select c0,c1 from stb11_1 where (c0>1000) union select c0,c1 from stb11_1 where c0>2000;")
tdSql.error("select c0,c1 from stb11_1 where (c0>1000) union all select c0,c1 from stb11_1 where c0>2000;")
# run case
# run case
def run(self):
# test qnode
tdLog.debug(" test_case1 ............ [start]")
tdLog.debug(" test_case1 ............ [start]")
self.test_case1()
tdLog.debug(" test_case1 ............ [OK]")
tdLog.debug(" test_case2 ............ [start]")
tdLog.debug(" test_case2 ............ [start]")
self.test_case2()
tdLog.debug(" test_case2 ............ [OK]")
tdLog.debug(" test_case3 ............ [start]")
tdLog.debug(" test_case3 ............ [start]")
self.test_case3()
tdLog.debug(" test_case3 ............ [OK]")
......@@ -475,7 +475,7 @@ class TDTestCase:
tdSql.close()
tdLog.success(f"{__file__} successfully executed")
return
return
#
# add case with filename
#
......
......@@ -11,14 +11,14 @@ from util.sql import *
from util.cases import *
class TDTestCase:
updatecfgDict = {'debugFlag': 143 ,"cDebugFlag":143,"uDebugFlag":143 ,"rpcDebugFlag":143 , "tmrDebugFlag":143 ,
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 ,"udfDebugFlag":143}
def init(self, conn, logSql):
tdLog.debug(f"start to excute {__file__}")
tdSql.init(conn.cursor())
def prepare_datas(self):
tdSql.execute(
'''create table stb1
......@@ -26,7 +26,7 @@ class TDTestCase:
tags (t1 int)
'''
)
tdSql.execute(
'''
create table t1
......@@ -68,7 +68,7 @@ class TDTestCase:
( '2023-02-21 01:01:01.000', NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL )
'''
)
def test_errors(self):
error_sql_lists = [
"select unique from t1",
......@@ -119,40 +119,40 @@ class TDTestCase:
"select unique(c1) , diff(c1) from stb1 partition by tbname",
#"select unique(c1) , abs(c1) from stb1 partition by tbname", # support
#"select unique(c1) , c1 from stb1 partition by tbname" # support
]
for error_sql in error_sql_lists:
tdSql.error(error_sql)
pass
def support_types(self):
other_no_value_types = [
"select unique(ts) from t1" ,
"select unique(ts) from t1" ,
"select unique(c7) from t1",
"select unique(c8) from t1",
"select unique(c9) from t1",
"select unique(ts) from ct1" ,
"select unique(ts) from ct1" ,
"select unique(c7) from ct1",
"select unique(c8) from ct1",
"select unique(c9) from ct1",
"select unique(ts) from ct3" ,
"select unique(ts) from ct3" ,
"select unique(c7) from ct3",
"select unique(c8) from ct3",
"select unique(c9) from ct3",
"select unique(ts) from ct4" ,
"select unique(ts) from ct4" ,
"select unique(c7) from ct4",
"select unique(c8) from ct4",
"select unique(c9) from ct4",
"select unique(ts) from stb1 partition by tbname" ,
"select unique(ts) from stb1 partition by tbname" ,
"select unique(c7) from stb1 partition by tbname",
"select unique(c8) from stb1 partition by tbname",
"select unique(c9) from stb1 partition by tbname"
"select unique(c9) from stb1 partition by tbname"
]
for type_sql in other_no_value_types:
tdSql.query(type_sql)
tdLog.info("support type ok , sql is : %s"%type_sql)
type_sql_lists = [
"select unique(c1) from t1",
"select unique(c2) from t1",
......@@ -182,8 +182,8 @@ class TDTestCase:
"select unique(c5) from stb1 partition by tbname",
"select unique(c6) from stb1 partition by tbname",
"select unique(c6) as alisb from stb1 partition by tbname",
"select unique(c6) alisb from stb1 partition by tbname",
"select unique(c6) as alisb from stb1 partition by tbname",
"select unique(c6) alisb from stb1 partition by tbname",
]
for type_sql in type_sql_lists:
......@@ -194,18 +194,18 @@ class TDTestCase:
origin_sql = unique_sql.replace("unique(","").replace(")","")
tdSql.query(unique_sql)
unique_result = tdSql.queryResult
unique_datas = []
for elem in unique_result:
unique_datas.append(elem[0])
unique_datas.sort(key=lambda x: (x is None, x))
tdSql.query(origin_sql)
origin_result = tdSql.queryResult
origin_datas = []
for elem in origin_result:
origin_datas.append(elem[0])
pre_unique = []
for elem in origin_datas:
if elem in pre_unique:
......@@ -221,7 +221,7 @@ class TDTestCase:
def basic_unique_function(self):
# basic query
# basic query
tdSql.query("select c1 from ct3")
tdSql.checkRows(0)
tdSql.query("select c1 from t1")
......@@ -242,19 +242,19 @@ class TDTestCase:
tdSql.checkRows(0)
tdSql.query("select unique(c6) from ct3")
# will support _rowts mix with
# will support _rowts mix with
# tdSql.query("select unique(c6),_rowts from ct3")
# auto check for t1 table
# used for regular table
tdSql.query("select unique(c1) from t1")
tdSql.query("desc t1")
col_lists_rows = tdSql.queryResult
col_lists = []
for col_name in col_lists_rows:
col_lists.append(col_name[0])
for col in col_lists:
self.check_unique_table(f"select unique({col}) from t1")
......@@ -269,17 +269,17 @@ class TDTestCase:
#tdSql.error("select unique(c1),tbname from ct1") #support
#tdSql.error("select unique(c1),t1 from ct1") #support
# unique with common col
# unique with common col
#tdSql.error("select unique(c1) ,ts from ct1")
#tdSql.error("select unique(c1) ,c1 from ct1")
# unique with scalar function
# unique with scalar function
#tdSql.error("select unique(c1) ,abs(c1) from ct1")
tdSql.error("select unique(c1) , unique(c2) from ct1")
#tdSql.error("select unique(c1) , abs(c2)+2 from ct1")
# unique with aggregate function
# unique with aggregate function
tdSql.error("select unique(c1) ,sum(c1) from ct1")
tdSql.error("select unique(c1) ,max(c1) from ct1")
tdSql.error("select unique(c1) ,csum(c1) from ct1")
......@@ -306,7 +306,7 @@ class TDTestCase:
tdSql.checkData(7, 0, 1)
tdSql.checkData(8, 0, 0)
# unique with union all
# unique with union all
tdSql.query("select unique(c1) from ct4 union all select c1 from ct1")
tdSql.checkRows(23)
tdSql.query("select unique(c1) from ct4 union all select distinct(c1) from ct4")
......@@ -314,8 +314,8 @@ class TDTestCase:
tdSql.query("select unique(c2) from ct4 union all select abs(c2)/2 from ct4")
tdSql.checkRows(22)
# unique with join
# prepare join datas with same ts
# unique with join
# prepare join datas with same ts
tdSql.execute(" use db ")
tdSql.execute(" create stable st1 (ts timestamp , num int) tags(ind int)")
......@@ -371,7 +371,7 @@ class TDTestCase:
tdSql.checkRows(10)
tdSql.checkData(0, 0, None)
tdSql.checkData(1, 0, -7.000000000)
# bug for stable
#partition by tbname
......@@ -380,8 +380,8 @@ class TDTestCase:
# tdSql.query(" select unique(c1) from stb1 partition by tbname ")
# tdSql.checkRows(21)
# group by
# group by
tdSql.error("select unique(c1) from ct1 group by c1")
tdSql.error("select unique(c1) from ct1 group by tbname")
......@@ -393,7 +393,7 @@ class TDTestCase:
tdSql.checkRows(4)
# bug need fix
# bug need fix
# tdSql.query("select tbname , tail(c1,2) from stb1 partition by tbname")
# tdSql.checkRows(4)
......@@ -411,7 +411,7 @@ class TDTestCase:
tdSql.checkRows(4)
# # bug need fix
# # bug need fix
# tdSql.query(" select tbname , unique(c1) from stb1 where t1 = 0 partition by tbname ")
# tdSql.checkRows(2)
# tdSql.query(" select tbname , unique(c1) from stb1 where t1 = 0 partition by tbname order by tbname ")
......@@ -430,7 +430,7 @@ class TDTestCase:
tdSql.query(" select unique(t1) from stb1 partition by tbname ")
tdSql.checkRows(2)
# nest query
# nest query
tdSql.query(" select unique(c1) from (select _rowts , t1 ,c1 , tbname from stb1 ) ")
tdSql.checkRows(11)
tdSql.checkData(0,0,6)
......@@ -439,7 +439,7 @@ class TDTestCase:
tdSql.checkRows(2)
tdSql.checkData(0,0,4)
tdSql.checkData(1,0,1)
def check_boundary_values(self):
tdSql.execute("drop database if exists bound_test")
......@@ -467,11 +467,11 @@ class TDTestCase:
tdSql.execute(
f"insert into sub1_bound values ( now(), -2147483643, -9223372036854775803, -32763, -123, -3.39E+38, -1.69e+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() )"
)
tdSql.query("select unique(c2) from sub1_bound order by 1 desc")
tdSql.checkRows(5)
tdSql.checkData(0,0,9223372036854775807)
......@@ -480,22 +480,22 @@ class TDTestCase:
tdSql.prepare()
tdLog.printNoPrefix("==========step1:create table ==============")
self.prepare_datas()
tdLog.printNoPrefix("==========step2:test errors ==============")
tdLog.printNoPrefix("==========step2:test errors ==============")
self.test_errors()
tdLog.printNoPrefix("==========step3:support types ============")
tdLog.printNoPrefix("==========step3:support types ============")
self.support_types()
tdLog.printNoPrefix("==========step4: floor basic query ============")
tdLog.printNoPrefix("==========step4: floor basic query ============")
self.basic_unique_function()
tdLog.printNoPrefix("==========step5: floor boundary query ============")
tdLog.printNoPrefix("==========step5: floor boundary query ============")
self.check_boundary_values()
......
......@@ -371,7 +371,7 @@ class TMQCom:
elif (i % 3 == 0):
tagBinaryValue = 'changsha'
sql += " %s.%s_%d using %s.%s tags (%d, %d, %d, '%s', '%s') values "%(dbName,ctbPrefix,i+ctbStartIdx,dbName,stbName,i+ctbStartIdx,i+ctbStartIdx,i+ctbStartIdx,tagBinaryValue,tagBinaryValue)
sql += " %s.%s%d using %s.%s tags (%d, %d, %d, '%s', '%s') values "%(dbName,ctbPrefix,i+ctbStartIdx,dbName,stbName,i+ctbStartIdx,i+ctbStartIdx,i+ctbStartIdx,tagBinaryValue,tagBinaryValue)
for j in range(rowsPerTbl):
sql += "(%d, %d, %d, %d, 'binary_%d', 'nchar_%d', now) "%(startTs+j, j,j, j,i+ctbStartIdx,rowsBatched)
rowsBatched += 1
......@@ -379,7 +379,7 @@ class TMQCom:
tsql.execute(sql)
rowsBatched = 0
if j < rowsPerTbl - 1:
sql = "insert into %s.%s_%d using %s.%s tags (%d, %d, %d, '%s', '%s') values " %(dbName,ctbPrefix,i+ctbStartIdx,dbName,stbName,i+ctbStartIdx,i+ctbStartIdx,i+ctbStartIdx,tagBinaryValue,tagBinaryValue)
sql = "insert into %s.%s%d using %s.%s tags (%d, %d, %d, '%s', '%s') values " %(dbName,ctbPrefix,i+ctbStartIdx,dbName,stbName,i+ctbStartIdx,i+ctbStartIdx,i+ctbStartIdx,tagBinaryValue,tagBinaryValue)
else:
sql = "insert into "
#end sql
......
......@@ -17,7 +17,7 @@ from tmqCommon import *
class TDTestCase:
def __init__(self):
self.snapshot = 0
self.vgroups = 2
self.vgroups = 4
self.ctbNum = 1
self.rowsPerTbl = 100000
......@@ -235,18 +235,18 @@ class TDTestCase:
def run(self):
tdSql.prepare()
self.prepareTestEnv()
tdLog.printNoPrefix("=============================================")
tdLog.printNoPrefix("======== snapshot is 0: only consume from wal")
self.tmqCase1()
self.tmqCase2()
# self.prepareTestEnv()
# tdLog.printNoPrefix("=============================================")
# tdLog.printNoPrefix("======== snapshot is 0: only consume from wal")
# self.tmqCase1()
# self.tmqCase2()
self.prepareTestEnv()
tdLog.printNoPrefix("====================================================================")
tdLog.printNoPrefix("======== snapshot is 1: firstly consume from tsbs, and then from wal")
self.snapshot = 1
self.tmqCase1()
self.tmqCase2()
# self.tmqCase2()
def stop(self):
......
......@@ -17,9 +17,10 @@ from tmqCommon import *
class TDTestCase:
def __init__(self):
self.snapshot = 0
self.vgroups = 2
self.ctbNum = 1
self.rowsPerTbl = 100000
self.vgroups = 4
self.ctbNum = 100
self.rowsPerTbl = 1000
self.autoCtbPrefix = 'aCtb'
def init(self, conn, logSql):
tdLog.debug(f"start to excute {__file__}")
......@@ -38,9 +39,9 @@ class TDTestCase:
'tagSchema': [{'type': 'INT', 'count':1},{'type': 'BIGINT', 'count':1},{'type': 'DOUBLE', 'count':1},{'type': 'BINARY', 'len':32, 'count':1},{'type': 'NCHAR', 'len':32, 'count':1}],
'ctbPrefix': 'ctb',
'ctbStartIdx': 0,
'ctbNum': 1,
'rowsPerTbl': 100000,
'batchNum': 1200,
'ctbNum': 1000,
'rowsPerTbl': 1000,
'batchNum': 10000,
'startTs': 1640966400000, # 2022-01-01 00:00:00.000
'pollDelay': 3,
'showMsg': 1,
......@@ -62,9 +63,9 @@ class TDTestCase:
tmqCom.insert_data_interlaceByMultiTbl(tsql=tdSql,dbName=paraDict["dbName"],ctbPrefix=paraDict["ctbPrefix"],
ctbNum=paraDict["ctbNum"],rowsPerTbl=paraDict["rowsPerTbl"],batchNum=paraDict["batchNum"],
startTs=paraDict["startTs"],ctbStartIdx=paraDict['ctbStartIdx'])
# tmqCom.insert_data_with_autoCreateTbl(tsql=tdSql,dbName=paraDict["dbName"],stbName=paraDict["stbName"],ctbPrefix="ctbx",
# ctbNum=paraDict["ctbNum"],rowsPerTbl=paraDict["rowsPerTbl"],batchNum=paraDict["batchNum"],
# startTs=paraDict["startTs"],ctbStartIdx=paraDict['ctbStartIdx'])
tmqCom.insert_data_with_autoCreateTbl(tsql=tdSql,dbName=paraDict["dbName"],stbName=paraDict["stbName"],ctbPrefix=self.autoCtbPrefix,
ctbNum=paraDict["ctbNum"],rowsPerTbl=paraDict["rowsPerTbl"],batchNum=paraDict["batchNum"],
startTs=paraDict["startTs"],ctbStartIdx=paraDict['ctbStartIdx'])
# tdLog.info("restart taosd to ensure that the data falls into the disk")
# tdSql.query("flush database %s"%(paraDict['dbName']))
......@@ -84,9 +85,9 @@ class TDTestCase:
'tagSchema': [{'type': 'INT', 'count':1},{'type': 'BIGINT', 'count':1},{'type': 'DOUBLE', 'count':1},{'type': 'BINARY', 'len':32, 'count':1},{'type': 'NCHAR', 'len':32, 'count':1}],
'ctbPrefix': 'ctb',
'ctbStartIdx': 0,
'ctbNum': 1,
'rowsPerTbl': 100000,
'batchNum': 3000,
'ctbNum': 1000,
'rowsPerTbl': 1000,
'batchNum': 10000,
'startTs': 1640966400000, # 2022-01-01 00:00:00.000
'pollDelay': 5,
'showMsg': 1,
......@@ -98,10 +99,11 @@ class TDTestCase:
paraDict['rowsPerTbl'] = self.rowsPerTbl
# update to half tables
paraDict['ctbNum'] = int(self.ctbNum/2)
paraDict['rowsPerTbl'] = int(self.rowsPerTbl / 2)
# tmqCom.insert_data_with_autoCreateTbl(tsql=tdSql,dbName=paraDict["dbName"],stbName=paraDict["stbName"],ctbPrefix="ctbx",
# ctbNum=paraDict["ctbNum"],rowsPerTbl=paraDict["rowsPerTbl"],batchNum=paraDict["batchNum"],
# startTs=paraDict["startTs"],ctbStartIdx=paraDict['ctbStartIdx'])
tmqCom.insert_data_with_autoCreateTbl(tsql=tdSql,dbName=paraDict["dbName"],stbName=paraDict["stbName"],ctbPrefix=self.autoCtbPrefix,
ctbNum=paraDict["ctbNum"],rowsPerTbl=paraDict["rowsPerTbl"],batchNum=paraDict["batchNum"],
startTs=paraDict["startTs"],ctbStartIdx=paraDict['ctbStartIdx'])
tmqCom.insert_data_interlaceByMultiTbl(tsql=tdSql,dbName=paraDict["dbName"],ctbPrefix=paraDict["ctbPrefix"],
ctbNum=paraDict["ctbNum"],rowsPerTbl=paraDict["rowsPerTbl"],batchNum=paraDict["batchNum"],
startTs=paraDict["startTs"],ctbStartIdx=paraDict['ctbStartIdx'])
......@@ -113,10 +115,14 @@ class TDTestCase:
tdLog.info("create topic sql: %s"%sqlString)
tdSql.execute(sqlString)
# paraDict['ctbNum'] = self.ctbNum
paraDict['ctbNum'] = self.ctbNum
paraDict['rowsPerTbl'] = self.rowsPerTbl
consumerId = 0
expectrowcnt = int(paraDict["rowsPerTbl"] * paraDict["ctbNum"] * 3/2)
if self.snapshot == 0:
expectrowcnt = int(paraDict["rowsPerTbl"] * paraDict["ctbNum"] * (2 + 1/2*1/2*2))
elif self.snapshot == 1:
expectrowcnt = int(paraDict["rowsPerTbl"] * paraDict["ctbNum"] * (2))
topicList = topicFromStb1
ifcheckdata = 1
ifManualCommit = 1
......@@ -129,7 +135,7 @@ class TDTestCase:
tdLog.info("start consume processor")
tmqCom.startTmqSimProcess(pollDelay=paraDict['pollDelay'],dbName=paraDict["dbName"],showMsg=paraDict['showMsg'], showRow=paraDict['showRow'],snapshot=paraDict['snapshot'])
tdLog.info("insert process end, and start to check consume result")
tdLog.info("start to check consume result")
expectRows = 1
resultList = tmqCom.selectConsumeResult(expectRows)
totalConsumeRows = 0
......@@ -143,7 +149,7 @@ class TDTestCase:
if totalConsumeRows != expectrowcnt:
tdLog.exit("tmq consume rows error!")
tmqCom.checkFileContent(consumerId, queryString)
# tmqCom.checkFileContent(consumerId, queryString)
tdSql.query("drop topic %s"%topicFromStb1)
tdLog.printNoPrefix("======== test case 1 end ...... ")
......@@ -161,9 +167,9 @@ class TDTestCase:
'tagSchema': [{'type': 'INT', 'count':1},{'type': 'BIGINT', 'count':1},{'type': 'DOUBLE', 'count':1},{'type': 'BINARY', 'len':32, 'count':1},{'type': 'NCHAR', 'len':32, 'count':1}],
'ctbPrefix': 'ctb',
'ctbStartIdx': 0,
'ctbNum': 1,
'rowsPerTbl': 10000,
'batchNum': 5000,
'ctbNum': 1000,
'rowsPerTbl': 1000,
'batchNum': 10000,
'startTs': 1640966400000, # 2022-01-01 00:00:00.000
'pollDelay': 5,
'showMsg': 1,
......@@ -179,14 +185,20 @@ class TDTestCase:
tdSql.query("flush database %s"%(paraDict['dbName']))
# update to half tables
paraDict['startTs'] = paraDict['startTs'] + int(self.rowsPerTbl / 2)
paraDict['ctbNum'] = int(self.ctbNum/2)
paraDict['rowsPerTbl'] = int(self.rowsPerTbl / 2)
tmqCom.insert_data_with_autoCreateTbl(tsql=tdSql,dbName=paraDict["dbName"],stbName=paraDict["stbName"],ctbPrefix=paraDict["ctbPrefix"],
paraDict['startTs'] = paraDict['startTs'] + int(self.rowsPerTbl / 2)
tmqCom.insert_data_with_autoCreateTbl(tsql=tdSql,dbName=paraDict["dbName"],stbName=paraDict["stbName"],ctbPrefix=self.autoCtbPrefix,
ctbNum=paraDict["ctbNum"],rowsPerTbl=paraDict["rowsPerTbl"],batchNum=paraDict["batchNum"],
startTs=paraDict["startTs"],ctbStartIdx=paraDict['ctbStartIdx'])
# tmqCom.insert_data_interlaceByMultiTbl(tsql=tdSql,dbName=paraDict["dbName"],ctbPrefix=paraDict["ctbPrefix"],
# ctbNum=paraDict["ctbNum"],rowsPerTbl=paraDict["rowsPerTbl"],batchNum=paraDict["batchNum"],
# startTs=paraDict["startTs"],ctbStartIdx=paraDict['ctbStartIdx'])
startTs=paraDict["startTs"],ctbStartIdx=paraDict['ctbStartIdx']+int(self.ctbNum/2))
tmqCom.insert_data_with_autoCreateTbl(tsql=tdSql,dbName=paraDict["dbName"],stbName=paraDict["stbName"],ctbPrefix="aCtby",
ctbNum=paraDict["ctbNum"],rowsPerTbl=paraDict["rowsPerTbl"],batchNum=paraDict["batchNum"],
startTs=paraDict["startTs"],ctbStartIdx=paraDict['ctbStartIdx']+int(self.ctbNum/2))
tmqCom.insert_data_interlaceByMultiTbl(tsql=tdSql,dbName=paraDict["dbName"],ctbPrefix=paraDict["ctbPrefix"],
ctbNum=paraDict["ctbNum"],rowsPerTbl=paraDict["rowsPerTbl"],batchNum=paraDict["batchNum"],
startTs=paraDict["startTs"],ctbStartIdx=paraDict['ctbStartIdx']+int(self.ctbNum/2))
tmqCom.initConsumerTable()
tdLog.info("create topics from stb1")
......@@ -197,9 +209,14 @@ class TDTestCase:
tdSql.execute(sqlString)
# paraDict['ctbNum'] = self.ctbNum
paraDict['ctbNum'] = self.ctbNum
paraDict['rowsPerTbl'] = self.rowsPerTbl
consumerId = 1
expectrowcnt = int(paraDict["rowsPerTbl"] * paraDict["ctbNum"] * 2)
if self.snapshot == 0:
expectrowcnt = int(paraDict["rowsPerTbl"] * paraDict["ctbNum"] * (2 + 1/2*1/2*2 + 1/2*1/2))
elif self.snapshot == 1:
expectrowcnt = int(paraDict["rowsPerTbl"] * paraDict["ctbNum"] * (2 + 1/2*1/2))
topicList = topicFromStb1
ifcheckdata = 1
ifManualCommit = 1
......@@ -212,7 +229,7 @@ class TDTestCase:
tdLog.info("start consume processor")
tmqCom.startTmqSimProcess(pollDelay=paraDict['pollDelay'],dbName=paraDict["dbName"],showMsg=paraDict['showMsg'], showRow=paraDict['showRow'],snapshot=paraDict['snapshot'])
tdLog.info("insert process end, and start to check consume result")
tdLog.info("start to check consume result")
expectRows = 1
resultList = tmqCom.selectConsumeResult(expectRows)
totalConsumeRows = 0
......
......@@ -11,7 +11,7 @@ python3 ./test.py -f 0-others/udfTest.py
python3 ./test.py -f 0-others/udf_create.py
python3 ./test.py -f 0-others/udf_restart_taosd.py
python3 ./test.py -f 0-others/cachelast.py
python3 ./test.py -f 0-others/sysinfo.py
python3 ./test.py -f 0-others/user_control.py
python3 ./test.py -f 0-others/fsync.py
......@@ -178,6 +178,8 @@ python3 ./test.py -f 7-tmq/tmqConsFromTsdb1-mutilVg-mutilCtb-funcNFilter.py
python3 ./test.py -f 7-tmq/tmqConsFromTsdb1-mutilVg-mutilCtb.py
python3 ./test.py -f 7-tmq/tmqAutoCreateTbl.py
#python3 ./test.py -f 7-tmq/tmqDnodeRestart.py
#python3 ./test.py -f 7-tmq/tmqUpdate-1ctb.py
python3 ./test.py -f 7-tmq/tmqUpdate-multiCtb.py
#------------querPolicy 2-----------
......@@ -353,4 +355,4 @@ python3 ./test.py -f 2-query/twa.py -Q 3
python3 ./test.py -f 2-query/irate.py -Q 3
python3 ./test.py -f 2-query/function_null.py -Q 3
python3 ./test.py -f 2-query/count_partition.py -Q 3
python3 ./test.py -f 2-query/max_partition.py -Q 3
\ No newline at end of file
python3 ./test.py -f 2-query/max_partition.py -Q 3
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册