Merge branch '3.0' into fix/xsren/TD-28204/whereOptimizeConditional

This commit is contained in:
xinsheng Ren 2024-01-16 14:28:37 +08:00 committed by GitHub
commit ae5f540692
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
61 changed files with 2343 additions and 701 deletions

View File

@ -189,7 +189,8 @@ typedef struct TsdReader {
typedef struct SStoreCacheReader { typedef struct SStoreCacheReader {
int32_t (*openReader)(void *pVnode, int32_t type, void *pTableIdList, int32_t numOfTables, int32_t numOfCols, int32_t (*openReader)(void *pVnode, int32_t type, void *pTableIdList, int32_t numOfTables, int32_t numOfCols,
SArray *pCidList, int32_t *pSlotIds, uint64_t suid, void **pReader, const char *idstr); SArray *pCidList, int32_t *pSlotIds, uint64_t suid, void **pReader, const char *idstr,
SArray *pFuncTypeList);
void *(*closeReader)(void *pReader); void *(*closeReader)(void *pReader);
int32_t (*retrieveRows)(void *pReader, SSDataBlock *pResBlock, const int32_t *slotIds, const int32_t *dstSlotIds, int32_t (*retrieveRows)(void *pReader, SSDataBlock *pResBlock, const int32_t *slotIds, const int32_t *dstSlotIds,
SArray *pTableUidList); SArray *pTableUidList);

View File

@ -149,6 +149,8 @@ void nodesRewriteExprPostOrder(SNode** pNode, FNodeRewriter rewriter, void* pCon
void nodesRewriteExprsPostOrder(SNodeList* pList, FNodeRewriter rewriter, void* pContext); void nodesRewriteExprsPostOrder(SNodeList* pList, FNodeRewriter rewriter, void* pContext);
bool nodesEqualNode(const SNode* a, const SNode* b); bool nodesEqualNode(const SNode* a, const SNode* b);
bool nodeListNodeEqual(const SNodeList* a, const SNode* b);
bool nodesMatchNode(const SNode* pSub, const SNode* pNode); bool nodesMatchNode(const SNode* pSub, const SNode* pNode);
SNode* nodesCloneNode(const SNode* pNode); SNode* nodesCloneNode(const SNode* pNode);

View File

@ -120,6 +120,7 @@ typedef struct SScanLogicNode {
bool onlyMetaCtbIdx; // for tag scan with no tbname bool onlyMetaCtbIdx; // for tag scan with no tbname
bool filesetDelimited; // returned blocks delimited by fileset bool filesetDelimited; // returned blocks delimited by fileset
bool isCountByTag; // true if selectstmt hasCountFunc & part by tag/tbname bool isCountByTag; // true if selectstmt hasCountFunc & part by tag/tbname
SArray* pFuncTypes; // for last, last_row
} SScanLogicNode; } SScanLogicNode;
typedef struct SJoinLogicNode { typedef struct SJoinLogicNode {
@ -401,6 +402,7 @@ typedef struct SLastRowScanPhysiNode {
bool groupSort; bool groupSort;
bool ignoreNull; bool ignoreNull;
SNodeList* pTargets; SNodeList* pTargets;
SArray* pFuncTypes;
} SLastRowScanPhysiNode; } SLastRowScanPhysiNode;
typedef SLastRowScanPhysiNode STableCountScanPhysiNode; typedef SLastRowScanPhysiNode STableCountScanPhysiNode;

View File

@ -89,6 +89,7 @@ typedef struct SColumnNode {
typedef struct SColumnRefNode { typedef struct SColumnRefNode {
ENodeType type; ENodeType type;
SDataType resType;
char colName[TSDB_COL_NAME_LEN]; char colName[TSDB_COL_NAME_LEN];
} SColumnRefNode; } SColumnRefNode;
@ -532,6 +533,7 @@ int32_t nodesCollectColumnsFromNode(SNode* node, const char* pTableAlias, EColle
typedef bool (*FFuncClassifier)(int32_t funcId); typedef bool (*FFuncClassifier)(int32_t funcId);
int32_t nodesCollectFuncs(SSelectStmt* pSelect, ESqlClause clause, char* tableAlias, FFuncClassifier classifier, SNodeList** pFuncs); int32_t nodesCollectFuncs(SSelectStmt* pSelect, ESqlClause clause, char* tableAlias, FFuncClassifier classifier, SNodeList** pFuncs);
int32_t nodesCollectSelectFuncs(SSelectStmt* pSelect, ESqlClause clause, char* tableAlias, FFuncClassifier classifier, SNodeList* pFuncs);
int32_t nodesCollectSpecialNodes(SSelectStmt* pSelect, ESqlClause clause, ENodeType type, SNodeList** pNodes); int32_t nodesCollectSpecialNodes(SSelectStmt* pSelect, ESqlClause clause, ENodeType type, SNodeList** pNodes);

View File

@ -801,7 +801,7 @@ int32_t streamTaskCheckStatus(SStreamTask* pTask, int32_t upstreamTaskId, int32_
int64_t* oldStage); int64_t* oldStage);
int32_t streamTaskUpdateEpsetInfo(SStreamTask* pTask, SArray* pNodeList); int32_t streamTaskUpdateEpsetInfo(SStreamTask* pTask, SArray* pNodeList);
void streamTaskResetUpstreamStageInfo(SStreamTask* pTask); void streamTaskResetUpstreamStageInfo(SStreamTask* pTask);
bool streamTaskAllUpstreamClosed(SStreamTask* pTask); bool streamTaskIsAllUpstreamClosed(SStreamTask* pTask);
bool streamTaskSetSchedStatusWait(SStreamTask* pTask); bool streamTaskSetSchedStatusWait(SStreamTask* pTask);
int8_t streamTaskSetSchedStatusActive(SStreamTask* pTask); int8_t streamTaskSetSchedStatusActive(SStreamTask* pTask);
int8_t streamTaskSetSchedStatusInactive(SStreamTask* pTask); int8_t streamTaskSetSchedStatusInactive(SStreamTask* pTask);
@ -826,8 +826,7 @@ int32_t streamQueueGetNumOfItems(const SStreamQueue* pQueue);
// common // common
int32_t streamRestoreParam(SStreamTask* pTask); int32_t streamRestoreParam(SStreamTask* pTask);
int32_t streamResetParamForScanHistory(SStreamTask* pTask); void streamTaskPause(SStreamMeta* pMeta, SStreamTask* pTask);
void streamTaskPause(SStreamTask* pTask, SStreamMeta* pMeta);
void streamTaskResume(SStreamTask* pTask); void streamTaskResume(SStreamTask* pTask);
int32_t streamTaskSetUpstreamInfo(SStreamTask* pTask, const SStreamTask* pUpstreamTask); int32_t streamTaskSetUpstreamInfo(SStreamTask* pTask, const SStreamTask* pUpstreamTask);
void streamTaskUpdateUpstreamInfo(SStreamTask* pTask, int32_t nodeId, const SEpSet* pEpSet); void streamTaskUpdateUpstreamInfo(SStreamTask* pTask, int32_t nodeId, const SEpSet* pEpSet);
@ -838,6 +837,7 @@ int32_t streamTaskReloadState(SStreamTask* pTask);
void streamTaskCloseUpstreamInput(SStreamTask* pTask, int32_t taskId); void streamTaskCloseUpstreamInput(SStreamTask* pTask, int32_t taskId);
void streamTaskOpenAllUpstreamInput(SStreamTask* pTask); void streamTaskOpenAllUpstreamInput(SStreamTask* pTask);
int32_t streamTaskSetDb(SStreamMeta* pMeta, void* pTask, char* key); int32_t streamTaskSetDb(SStreamMeta* pMeta, void* pTask, char* key);
bool streamTaskIsSinkTask(const SStreamTask* pTask);
void streamTaskStatusInit(STaskStatusEntry* pEntry, const SStreamTask* pTask); void streamTaskStatusInit(STaskStatusEntry* pEntry, const SStreamTask* pTask);
void streamTaskStatusCopy(STaskStatusEntry* pDst, const STaskStatusEntry* pSrc); void streamTaskStatusCopy(STaskStatusEntry* pDst, const STaskStatusEntry* pSrc);
@ -885,6 +885,7 @@ int32_t streamMetaLoadAllTasks(SStreamMeta* pMeta);
int32_t streamMetaStartAllTasks(SStreamMeta* pMeta); int32_t streamMetaStartAllTasks(SStreamMeta* pMeta);
int32_t streamMetaStopAllTasks(SStreamMeta* pMeta); int32_t streamMetaStopAllTasks(SStreamMeta* pMeta);
int32_t streamMetaStartOneTask(SStreamMeta* pMeta, int64_t streamId, int32_t taskId); int32_t streamMetaStartOneTask(SStreamMeta* pMeta, int64_t streamId, int32_t taskId);
bool streamMetaAllTasksReady(const SStreamMeta* pMeta);
// checkpoint // checkpoint
int32_t streamProcessCheckpointSourceReq(SStreamTask* pTask, SStreamCheckpointSourceReq* pReq); int32_t streamProcessCheckpointSourceReq(SStreamTask* pTask, SStreamCheckpointSourceReq* pReq);

View File

@ -117,6 +117,8 @@ int32_t taosCompressFile(char *srcFileName, char *destFileName);
int32_t taosSetFileHandlesLimit(); int32_t taosSetFileHandlesLimit();
int32_t taosLinkFile(char *src, char *dst);
#ifdef __cplusplus #ifdef __cplusplus
} }
#endif #endif

View File

@ -57,10 +57,6 @@ void taos_cleanup(void) {
tscStopCrashReport(); tscStopCrashReport();
int32_t id = clientReqRefPool;
clientReqRefPool = -1;
taosCloseRef(id);
hbMgrCleanUp(); hbMgrCleanUp();
catalogDestroy(); catalogDestroy();
@ -70,6 +66,12 @@ void taos_cleanup(void) {
qCleanupKeywordsTable(); qCleanupKeywordsTable();
nodesDestroyAllocatorSet(); nodesDestroyAllocatorSet();
cleanupTaskQueue();
int32_t id = clientReqRefPool;
clientReqRefPool = -1;
taosCloseRef(id);
id = clientConnRefPool; id = clientConnRefPool;
clientConnRefPool = -1; clientConnRefPool = -1;
taosCloseRef(id); taosCloseRef(id);
@ -77,8 +79,6 @@ void taos_cleanup(void) {
rpcCleanup(); rpcCleanup();
tscDebug("rpc cleanup"); tscDebug("rpc cleanup");
cleanupTaskQueue();
taosConvDestroy(); taosConvDestroy();
tscInfo("all local resources released"); tscInfo("all local resources released");

View File

@ -687,6 +687,7 @@ _OVER:
static int32_t extractNodeEpset(SMnode *pMnode, SEpSet *pEpSet, bool *hasEpset, int32_t taskId, int32_t nodeId) { static int32_t extractNodeEpset(SMnode *pMnode, SEpSet *pEpSet, bool *hasEpset, int32_t taskId, int32_t nodeId) {
*hasEpset = false; *hasEpset = false;
pEpSet->numOfEps = 0;
if (nodeId == SNODE_HANDLE) { if (nodeId == SNODE_HANDLE) {
SSnodeObj *pObj = NULL; SSnodeObj *pObj = NULL;
void *pIter = NULL; void *pIter = NULL;
@ -1823,7 +1824,8 @@ static int32_t mndPauseStreamTask(SMnode *pMnode, STrans *pTrans, SStreamTask *p
pReq->taskId = pTask->id.taskId; pReq->taskId = pTask->id.taskId;
pReq->streamId = pTask->id.streamId; pReq->streamId = pTask->id.streamId;
SEpSet epset; SEpSet epset = {0};
mDebug("pause node:%d, epset:%d", pTask->info.nodeId, epset.numOfEps);
bool hasEpset = false; bool hasEpset = false;
int32_t code = extractNodeEpset(pMnode, &epset, &hasEpset, pTask->id.taskId, pTask->info.nodeId); int32_t code = extractNodeEpset(pMnode, &epset, &hasEpset, pTask->id.taskId, pTask->info.nodeId);
if (code != TSDB_CODE_SUCCESS) { if (code != TSDB_CODE_SUCCESS) {
@ -1869,12 +1871,14 @@ int32_t mndPauseAllStreamTasks(SMnode *pMnode, STrans *pTrans, SStreamObj *pStre
return 0; return 0;
} }
static int32_t mndPersistStreamLog(STrans *pTrans, const SStreamObj *pStream, int8_t status) { static int32_t mndPersistStreamLog(STrans *pTrans, SStreamObj *pStream, int8_t status) {
SStreamObj streamObj = {0}; // SStreamObj streamObj = {0};
memcpy(streamObj.name, pStream->name, TSDB_STREAM_FNAME_LEN); // memcpy(streamObj.name, pStream->name, TSDB_STREAM_FNAME_LEN);
streamObj.status = status; taosWLockLatch(&pStream->lock);
pStream->status = status;
SSdbRaw *pCommitRaw = mndStreamActionEncode(pStream);
SSdbRaw *pCommitRaw = mndStreamActionEncode(&streamObj); taosWUnLockLatch(&pStream->lock);
if (pCommitRaw == NULL) return -1; if (pCommitRaw == NULL) return -1;
if (mndTransAppendCommitlog(pTrans, pCommitRaw) != 0) { if (mndTransAppendCommitlog(pTrans, pCommitRaw) != 0) {
mError("stream trans:%d, failed to append commit log since %s", pTrans->id, terrstr()); mError("stream trans:%d, failed to append commit log since %s", pTrans->id, terrstr());
@ -1989,7 +1993,7 @@ static int32_t mndResumeStreamTask(STrans *pTrans, SMnode *pMnode, SStreamTask *
pReq->streamId = pTask->id.streamId; pReq->streamId = pTask->id.streamId;
pReq->igUntreated = igUntreated; pReq->igUntreated = igUntreated;
SEpSet epset; SEpSet epset = {0};
bool hasEpset = false; bool hasEpset = false;
int32_t code = extractNodeEpset(pMnode, &epset, &hasEpset, pTask->id.taskId, pTask->info.nodeId); int32_t code = extractNodeEpset(pMnode, &epset, &hasEpset, pTask->id.taskId, pTask->info.nodeId);
if (code != TSDB_CODE_SUCCESS) { if (code != TSDB_CODE_SUCCESS) {
@ -2376,7 +2380,6 @@ static int32_t mndProcessVgroupChange(SMnode *pMnode, SVgroupChangeInfo *pChange
} }
} }
while (1) { while (1) {
pIter = sdbFetch(pSdb, SDB_STREAM, pIter, (void **)&pStream); pIter = sdbFetch(pSdb, SDB_STREAM, pIter, (void **)&pStream);
if (pIter == NULL) { if (pIter == NULL) {
@ -2790,7 +2793,7 @@ int32_t createStreamResetStatusTrans(SMnode *pMnode, SStreamObj *pStream) {
pReq->taskId = pTask->id.taskId; pReq->taskId = pTask->id.taskId;
pReq->streamId = pTask->id.streamId; pReq->streamId = pTask->id.streamId;
SEpSet epset; SEpSet epset = {0};
bool hasEpset = false; bool hasEpset = false;
int32_t code = extractNodeEpset(pMnode, &epset, &hasEpset, pTask->id.taskId, pTask->info.nodeId); int32_t code = extractNodeEpset(pMnode, &epset, &hasEpset, pTask->id.taskId, pTask->info.nodeId);
if (code != TSDB_CODE_SUCCESS) { if (code != TSDB_CODE_SUCCESS) {
@ -2933,8 +2936,8 @@ static SStreamTask *mndGetStreamTask(STaskId *pId, SStreamObj *pStream) {
// //
// if (numOfReady > 0) { // if (numOfReady > 0) {
// mDebug("stream:0x%" PRIx64 // mDebug("stream:0x%" PRIx64
// " %d tasks are ready, %d tasks in stream-scan-history for more than 50s, drop related fill-history task", // " %d tasks are ready, %d tasks in stream-scan-history for more than 50s, drop related fill-history
// pTaskEntry->id.streamId, numOfReady, numOfTotal - numOfReady); // task", pTaskEntry->id.streamId, numOfReady, numOfTotal - numOfReady);
// return true; // return true;
// } else { // } else {
// return false; // return false;
@ -2973,7 +2976,7 @@ static int32_t mndDropRelatedFillhistoryTask(SMnode *pMnode, STaskStatusEntry *p
mDebug("build and send drop related fill-history task for task:0x%x", pTask->id.taskId); mDebug("build and send drop related fill-history task for task:0x%x", pTask->id.taskId);
SEpSet epset; SEpSet epset = {0};
bool hasEpset = false; bool hasEpset = false;
int32_t code = extractNodeEpset(pMnode, &epset, &hasEpset, pTask->id.taskId, pTask->info.nodeId); int32_t code = extractNodeEpset(pMnode, &epset, &hasEpset, pTask->id.taskId, pTask->info.nodeId);
if (code != TSDB_CODE_SUCCESS) { if (code != TSDB_CODE_SUCCESS) {

View File

@ -175,7 +175,8 @@ void tsdbReaderSetNotifyCb(STsdbReader* pReader, TsdReaderNotifyCbFn not
int32_t tsdbReuseCacherowsReader(void *pReader, void *pTableIdList, int32_t numOfTables); int32_t tsdbReuseCacherowsReader(void *pReader, void *pTableIdList, int32_t numOfTables);
int32_t tsdbCacherowsReaderOpen(void *pVnode, int32_t type, void *pTableIdList, int32_t numOfTables, int32_t numOfCols, int32_t tsdbCacherowsReaderOpen(void *pVnode, int32_t type, void *pTableIdList, int32_t numOfTables, int32_t numOfCols,
SArray *pCidList, int32_t *pSlotIds, uint64_t suid, void **pReader, const char *idstr); SArray *pCidList, int32_t *pSlotIds, uint64_t suid, void **pReader, const char *idstr,
SArray* pFuncTypeList);
int32_t tsdbRetrieveCacheRows(void *pReader, SSDataBlock *pResBlock, const int32_t *slotIds, const int32_t *dstSlotIds, int32_t tsdbRetrieveCacheRows(void *pReader, SSDataBlock *pResBlock, const int32_t *slotIds, const int32_t *dstSlotIds,
SArray *pTableUids); SArray *pTableUids);
void *tsdbCacherowsReaderClose(void *pReader); void *tsdbCacherowsReaderClose(void *pReader);

View File

@ -742,8 +742,6 @@ int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t nextProcessVer) {
return code; return code;
} }
streamTaskOpenAllUpstreamInput(pTask);
if (pTask->info.taskLevel == TASK_LEVEL__SOURCE) { if (pTask->info.taskLevel == TASK_LEVEL__SOURCE) {
STaskId taskId = {0}; STaskId taskId = {0};
if (pTask->info.fillHistory) { if (pTask->info.fillHistory) {
@ -1126,6 +1124,19 @@ int32_t tqProcessTaskCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg* pRsp)
pRsp->info.handle = NULL; pRsp->info.handle = NULL;
SStreamCheckpointSourceReq req = {0}; SStreamCheckpointSourceReq req = {0};
SDecoder decoder;
tDecoderInit(&decoder, (uint8_t*)msg, len);
if (tDecodeStreamCheckpointSourceReq(&decoder, &req) < 0) {
code = TSDB_CODE_MSG_DECODE_ERROR;
tDecoderClear(&decoder);
tqError("vgId:%d failed to decode checkpoint-source msg, code:%s", vgId, tstrerror(code));
SRpcMsg rsp = {0};
buildCheckpointSourceRsp(&req, &pMsg->info, &rsp, 0);
tmsgSendRsp(&rsp); // error occurs
return code;
}
tDecoderClear(&decoder);
if (!vnodeIsRoleLeader(pTq->pVnode)) { if (!vnodeIsRoleLeader(pTq->pVnode)) {
tqDebug("vgId:%d not leader, ignore checkpoint-source msg, s-task:0x%x", vgId, req.taskId); tqDebug("vgId:%d not leader, ignore checkpoint-source msg, s-task:0x%x", vgId, req.taskId);
SRpcMsg rsp = {0}; SRpcMsg rsp = {0};
@ -1142,19 +1153,6 @@ int32_t tqProcessTaskCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg* pRsp)
return TSDB_CODE_SUCCESS; return TSDB_CODE_SUCCESS;
} }
SDecoder decoder;
tDecoderInit(&decoder, (uint8_t*)msg, len);
if (tDecodeStreamCheckpointSourceReq(&decoder, &req) < 0) {
code = TSDB_CODE_MSG_DECODE_ERROR;
tDecoderClear(&decoder);
tqError("vgId:%d failed to decode checkpoint-source msg, code:%s", vgId, tstrerror(code));
SRpcMsg rsp = {0};
buildCheckpointSourceRsp(&req, &pMsg->info, &rsp, 0);
tmsgSendRsp(&rsp); // error occurs
return code;
}
tDecoderClear(&decoder);
SStreamTask* pTask = streamMetaAcquireTask(pMeta, req.streamId, req.taskId); SStreamTask* pTask = streamMetaAcquireTask(pMeta, req.streamId, req.taskId);
if (pTask == NULL) { if (pTask == NULL) {
tqError("vgId:%d failed to find s-task:0x%x, ignore checkpoint msg. it may have been destroyed already", vgId, tqError("vgId:%d failed to find s-task:0x%x, ignore checkpoint msg. it may have been destroyed already", vgId,

View File

@ -76,33 +76,6 @@ int32_t tqStreamOneTaskStartAsync(SStreamMeta* pMeta, SMsgCb* cb, int64_t stream
return 0; return 0;
} }
int32_t tqUpdateNodeEpsetAsync(SStreamMeta* pMeta, SMsgCb* cb, int64_t streamId, int32_t taskId) {
int32_t vgId = pMeta->vgId;
int32_t numOfTasks = taosArrayGetSize(pMeta->pTaskList);
if (numOfTasks == 0) {
tqDebug("vgId:%d no stream tasks existed to run", vgId);
return 0;
}
SStreamTaskRunReq* pRunReq = rpcMallocCont(sizeof(SStreamTaskRunReq));
if (pRunReq == NULL) {
terrno = TSDB_CODE_OUT_OF_MEMORY;
tqError("vgId:%d failed to create msg to start task:0x%x, code:%s", vgId, taskId, terrstr());
return -1;
}
tqDebug("vgId:%d update s-task:0x%x nodeEpset async", vgId, taskId);
pRunReq->head.vgId = vgId;
pRunReq->streamId = streamId;
pRunReq->taskId = taskId;
pRunReq->reqType = STREAM_EXEC_T_UPDATE_TASK_EPSET;
SRpcMsg msg = {.msgType = TDMT_STREAM_TASK_RUN, .pCont = pRunReq, .contLen = sizeof(SStreamTaskRunReq)};
tmsgPutToQueue(cb, STREAM_QUEUE, &msg);
return 0;
}
int32_t tqStreamTaskProcessUpdateReq(SStreamMeta* pMeta, SMsgCb* cb, SRpcMsg* pMsg, bool restored) { int32_t tqStreamTaskProcessUpdateReq(SStreamMeta* pMeta, SMsgCb* cb, SRpcMsg* pMsg, bool restored) {
int32_t vgId = pMeta->vgId; int32_t vgId = pMeta->vgId;
char* msg = POINTER_SHIFT(pMsg->pCont, sizeof(SMsgHead)); char* msg = POINTER_SHIFT(pMsg->pCont, sizeof(SMsgHead));
@ -728,10 +701,6 @@ int32_t tqStreamTaskResetStatus(SStreamMeta* pMeta, int32_t* numOfTasks) {
STaskId id = {.streamId = pTaskId->streamId, .taskId = pTaskId->taskId}; STaskId id = {.streamId = pTaskId->streamId, .taskId = pTaskId->taskId};
SStreamTask** pTask = taosHashGet(pMeta->pTasksMap, &id, sizeof(id)); SStreamTask** pTask = taosHashGet(pMeta->pTasksMap, &id, sizeof(id));
streamTaskResetStatus(*pTask); streamTaskResetStatus(*pTask);
// if ((*pTask)->info.fillHistory == 1) {
// streamResetParamForScanHistory(*pTask);
// }
} }
return 0; return 0;
@ -844,8 +813,8 @@ int32_t tqStreamTaskProcessRunReq(SStreamMeta* pMeta, SRpcMsg* pMsg, bool isLead
if (pTask != NULL) { // even in halt status, the data in inputQ must be processed if (pTask != NULL) { // even in halt status, the data in inputQ must be processed
char* p = NULL; char* p = NULL;
if (streamTaskReadyToRun(pTask, &p)) { if (streamTaskReadyToRun(pTask, &p)) {
tqDebug("vgId:%d s-task:%s start to process block from inputQ, next checked ver:%" PRId64, vgId, pTask->id.idStr, tqDebug("vgId:%d s-task:%s status:%s start to process block from inputQ, next checked ver:%" PRId64, vgId, pTask->id.idStr,
pTask->chkInfo.nextProcessVer); p, pTask->chkInfo.nextProcessVer);
streamExecTask(pTask); streamExecTask(pTask);
} else { } else {
int8_t status = streamTaskSetSchedStatusInactive(pTask); int8_t status = streamTaskSetSchedStatusInactive(pTask);
@ -871,16 +840,24 @@ int32_t tqStartTaskCompleteCallback(SStreamMeta* pMeta) {
tqDebug("vgId:%d already in start tasks procedure in other thread, restartCounter:%d, do nothing", vgId, tqDebug("vgId:%d already in start tasks procedure in other thread, restartCounter:%d, do nothing", vgId,
pMeta->startInfo.restartCount); pMeta->startInfo.restartCount);
} else { // not in starting procedure } else { // not in starting procedure
if (pStartInfo->restartCount > 0) { bool allReady = streamMetaAllTasksReady(pMeta);
if ((pStartInfo->restartCount > 0) && (!allReady)) {
// if all tasks are ready now, do NOT restart again, and reset the value of pStartInfo->restartCount
pStartInfo->restartCount -= 1; pStartInfo->restartCount -= 1;
tqDebug("vgId:%d role:%d need to restart all tasks again, restartCounter:%d", vgId, pMeta->role, tqDebug("vgId:%d role:%d need to restart all tasks again, restartCounter:%d", vgId, pMeta->role,
pStartInfo->restartCount); pStartInfo->restartCount);
streamMetaWUnLock(pMeta); streamMetaWUnLock(pMeta);
restartStreamTasks(pMeta, (pMeta->role == NODE_ROLE_LEADER)); restartStreamTasks(pMeta, (pMeta->role == NODE_ROLE_LEADER));
return TSDB_CODE_SUCCESS; return TSDB_CODE_SUCCESS;
} else { } else {
tqDebug("vgId:%d start all tasks completed in callbackFn", pMeta->vgId); if (pStartInfo->restartCount == 0) {
tqDebug("vgId:%d start all tasks completed in callbackFn, restartCount is 0", pMeta->vgId);
} else if (allReady) {
pStartInfo->restartCount = 0;
tqDebug("vgId:%d all tasks are ready, reset restartCounter 0, not restart tasks", vgId);
}
} }
} }
@ -922,7 +899,7 @@ int32_t tqStreamTaskProcessTaskPauseReq(SStreamMeta* pMeta, char* pMsg){
} }
tqDebug("s-task:%s receive pause msg from mnode", pTask->id.idStr); tqDebug("s-task:%s receive pause msg from mnode", pTask->id.idStr);
streamTaskPause(pTask, pMeta); streamTaskPause(pMeta, pTask);
SStreamTask* pHistoryTask = NULL; SStreamTask* pHistoryTask = NULL;
if (HAS_RELATED_FILLHISTORY_TASK(pTask)) { if (HAS_RELATED_FILLHISTORY_TASK(pTask)) {
@ -939,7 +916,7 @@ int32_t tqStreamTaskProcessTaskPauseReq(SStreamMeta* pMeta, char* pMsg){
tqDebug("s-task:%s fill-history task handle paused along with related stream task", pHistoryTask->id.idStr); tqDebug("s-task:%s fill-history task handle paused along with related stream task", pHistoryTask->id.idStr);
streamTaskPause(pHistoryTask, pMeta); streamTaskPause(pMeta, pHistoryTask);
streamMetaReleaseTask(pMeta, pHistoryTask); streamMetaReleaseTask(pMeta, pHistoryTask);
} }

View File

@ -13,6 +13,7 @@
* along with this program. If not, see <http://www.gnu.org/licenses/>. * along with this program. If not, see <http://www.gnu.org/licenses/>.
*/ */
#include "cos.h" #include "cos.h"
#include "functionMgt.h"
#include "tsdb.h" #include "tsdb.h"
#include "tsdbDataFileRW.h" #include "tsdbDataFileRW.h"
#include "tsdbReadUtil.h" #include "tsdbReadUtil.h"
@ -894,19 +895,56 @@ static int32_t tsdbCacheLoadFromRaw(STsdb *pTsdb, tb_uid_t uid, SArray *pLastArr
} }
int num_keys = TARRAY_SIZE(remainCols); int num_keys = TARRAY_SIZE(remainCols);
int16_t *aCols = taosMemoryMalloc(num_keys * sizeof(int16_t));
int16_t *slotIds = taosMemoryMalloc(num_keys * sizeof(int16_t)); int16_t *slotIds = taosMemoryMalloc(num_keys * sizeof(int16_t));
int16_t *lastColIds = taosMemoryMalloc(num_keys * sizeof(int16_t));
int16_t *lastSlotIds = taosMemoryMalloc(num_keys * sizeof(int16_t));
int16_t *lastrowColIds = taosMemoryMalloc(num_keys * sizeof(int16_t));
int16_t *lastrowSlotIds = taosMemoryMalloc(num_keys * sizeof(int16_t));
SArray* lastTmpColArray = NULL;
SArray* lastTmpIndexArray = NULL;
SArray* lastrowTmpColArray = NULL;
SArray* lastrowTmpIndexArray = NULL;
int lastIndex = 0;
int lastrowIndex = 0;
for (int i = 0; i < num_keys; ++i) { for (int i = 0; i < num_keys; ++i) {
SIdxKey *idxKey = taosArrayGet(remainCols, i); SIdxKey *idxKey = taosArrayGet(remainCols, i);
aCols[i] = idxKey->key.cid;
slotIds[i] = pr->pSlotIds[idxKey->idx]; slotIds[i] = pr->pSlotIds[idxKey->idx];
if (idxKey->key.ltype == CACHESCAN_RETRIEVE_LAST >> 3) {
if(NULL == lastTmpIndexArray) {
lastTmpIndexArray = taosArrayInit(num_keys, sizeof(int32_t));
}
taosArrayPush(lastTmpIndexArray, &(i));
lastColIds[lastIndex] = idxKey->key.cid;
lastSlotIds[lastIndex] = pr->pSlotIds[idxKey->idx];
lastIndex++;
} else {
if(NULL == lastrowTmpIndexArray) {
lastrowTmpIndexArray = taosArrayInit(num_keys, sizeof(int32_t));
}
taosArrayPush(lastrowTmpIndexArray, &(i));
lastrowColIds[lastrowIndex] = idxKey->key.cid;
lastrowSlotIds[lastrowIndex] = pr->pSlotIds[idxKey->idx];
lastrowIndex++;
}
} }
if (ltype) { pTmpColArray = taosArrayInit(lastIndex + lastrowIndex, sizeof(SLastCol));
mergeLastCid(uid, pTsdb, &pTmpColArray, pr, aCols, num_keys, slotIds);
} else { if(lastTmpIndexArray != NULL) {
mergeLastRowCid(uid, pTsdb, &pTmpColArray, pr, aCols, num_keys, slotIds); mergeLastCid(uid, pTsdb, &lastTmpColArray, pr, lastColIds, lastIndex, lastSlotIds);
for(int i = 0; i < taosArrayGetSize(lastTmpColArray); i++) {
taosArrayInsert(pTmpColArray, *(int32_t*)taosArrayGet(lastTmpIndexArray, i), taosArrayGet(lastTmpColArray, i));
}
}
if(lastrowTmpIndexArray != NULL) {
mergeLastCid(uid, pTsdb, &lastrowTmpColArray, pr, lastrowColIds, lastrowIndex, lastrowSlotIds);
for(int i = 0; i < taosArrayGetSize(lastrowTmpColArray); i++) {
taosArrayInsert(pTmpColArray, *(int32_t*)taosArrayGet(lastrowTmpIndexArray, i), taosArrayGet(lastrowTmpColArray, i));
}
} }
SLRUCache *pCache = pTsdb->lruCache; SLRUCache *pCache = pTsdb->lruCache;
@ -965,9 +1003,18 @@ static int32_t tsdbCacheLoadFromRaw(STsdb *pTsdb, tb_uid_t uid, SArray *pLastArr
rocksMayWrite(pTsdb, false, true, false); rocksMayWrite(pTsdb, false, true, false);
} }
taosArrayDestroy(lastrowTmpIndexArray);
taosArrayDestroy(lastrowTmpColArray);
taosArrayDestroy(lastTmpIndexArray);
taosArrayDestroy(lastTmpColArray);
taosMemoryFree(lastColIds);
taosMemoryFree(lastSlotIds);
taosMemoryFree(lastrowColIds);
taosMemoryFree(lastrowSlotIds);
taosArrayDestroy(pTmpColArray); taosArrayDestroy(pTmpColArray);
taosMemoryFree(aCols);
taosMemoryFree(slotIds); taosMemoryFree(slotIds);
return code; return code;
@ -1057,6 +1104,15 @@ int32_t tsdbCacheGetBatch(STsdb *pTsdb, tb_uid_t uid, SArray *pLastArray, SCache
int16_t cid = ((int16_t *)TARRAY_DATA(pCidList))[i]; int16_t cid = ((int16_t *)TARRAY_DATA(pCidList))[i];
SLastKey *key = &(SLastKey){.ltype = ltype, .uid = uid, .cid = cid}; SLastKey *key = &(SLastKey){.ltype = ltype, .uid = uid, .cid = cid};
// for select last_row, last case
int32_t funcType = FUNCTION_TYPE_CACHE_LAST;
if (pr->pFuncTypeList != NULL && taosArrayGetSize(pr->pFuncTypeList) > i) {
funcType = ((int32_t *)TARRAY_DATA(pr->pFuncTypeList))[i];
}
if (((pr->type & CACHESCAN_RETRIEVE_LAST) == CACHESCAN_RETRIEVE_LAST) && FUNCTION_TYPE_CACHE_LAST_ROW == funcType) {
int8_t tempType = CACHESCAN_RETRIEVE_LAST_ROW | (pr->type ^ CACHESCAN_RETRIEVE_LAST);
key->ltype = (tempType & CACHESCAN_RETRIEVE_LAST) >> 3;
}
LRUHandle *h = taosLRUCacheLookup(pCache, key, ROCKS_KEY_LEN); LRUHandle *h = taosLRUCacheLookup(pCache, key, ROCKS_KEY_LEN);
if (h) { if (h) {

View File

@ -13,6 +13,7 @@
* along with this program. If not, see <http://www.gnu.org/licenses/>. * along with this program. If not, see <http://www.gnu.org/licenses/>.
*/ */
#include "functionMgt.h"
#include "taoserror.h" #include "taoserror.h"
#include "tarray.h" #include "tarray.h"
#include "tcommon.h" #include "tcommon.h"
@ -33,31 +34,69 @@ static void setFirstLastResColToNull(SColumnInfoData* pCol, int32_t row) {
taosMemoryFree(buf); taosMemoryFree(buf);
} }
static void saveOneRowForLastRaw(SLastCol* pColVal, SCacheRowsReader* pReader, const int32_t slotId,
SColumnInfoData* pColInfoData, int32_t numOfRows) {
SColVal* pVal = &pColVal->colVal;
// allNullRow = false;
if (IS_VAR_DATA_TYPE(pColVal->colVal.type)) {
if (!COL_VAL_IS_VALUE(&pColVal->colVal)) {
colDataSetNULL(pColInfoData, numOfRows);
} else {
varDataSetLen(pReader->transferBuf[slotId], pVal->value.nData);
memcpy(varDataVal(pReader->transferBuf[slotId]), pVal->value.pData, pVal->value.nData);
colDataSetVal(pColInfoData, numOfRows, pReader->transferBuf[slotId], false);
}
} else {
colDataSetVal(pColInfoData, numOfRows, (const char*)&pVal->value.val, !COL_VAL_IS_VALUE(pVal));
}
return;
}
static int32_t saveOneRow(SArray* pRow, SSDataBlock* pBlock, SCacheRowsReader* pReader, const int32_t* slotIds, static int32_t saveOneRow(SArray* pRow, SSDataBlock* pBlock, SCacheRowsReader* pReader, const int32_t* slotIds,
const int32_t* dstSlotIds, void** pRes, const char* idStr) { const int32_t* dstSlotIds, void** pRes, const char* idStr) {
int32_t numOfRows = pBlock->info.rows; int32_t numOfRows = pBlock->info.rows;
// bool allNullRow = true; // bool allNullRow = true;
if (HASTYPE(pReader->type, CACHESCAN_RETRIEVE_LAST)) { if (HASTYPE(pReader->type, CACHESCAN_RETRIEVE_LAST)) {
uint64_t ts = TSKEY_MIN; uint64_t ts = TSKEY_MIN;
SFirstLastRes* p = NULL; SFirstLastRes* p = NULL;
col_id_t colId = -1; col_id_t colId = -1;
SArray* funcTypeBlockArray = taosArrayInit(pReader->numOfCols, sizeof(int32_t));
for (int32_t i = 0; i < pReader->numOfCols; ++i) { for (int32_t i = 0; i < pReader->numOfCols; ++i) {
SColumnInfoData* pColInfoData = taosArrayGet(pBlock->pDataBlock, dstSlotIds[i]); SColumnInfoData* pColInfoData = taosArrayGet(pBlock->pDataBlock, dstSlotIds[i]);
int32_t funcType = FUNCTION_TYPE_CACHE_LAST;
if (pReader->pFuncTypeList != NULL && taosArrayGetSize(pReader->pFuncTypeList) > i) {
funcType = *(int32_t*)taosArrayGet(pReader->pFuncTypeList, i);
}
taosArrayInsert(funcTypeBlockArray, dstSlotIds[i], taosArrayGet(pReader->pFuncTypeList, i));
if (slotIds[i] == -1) { if (slotIds[i] == -1) {
if (FUNCTION_TYPE_CACHE_LAST_ROW == funcType) {
colDataSetNULL(pColInfoData, numOfRows);
continue;
}
setFirstLastResColToNull(pColInfoData, numOfRows); setFirstLastResColToNull(pColInfoData, numOfRows);
continue; continue;
} }
int32_t slotId = slotIds[i]; int32_t slotId = slotIds[i];
SLastCol* pColVal = (SLastCol*)taosArrayGet(pRow, i); SLastCol* pColVal = (SLastCol*)taosArrayGet(pRow, i);
colId = pColVal->colVal.cid; colId = pColVal->colVal.cid;
if (FUNCTION_TYPE_CACHE_LAST_ROW == funcType) {
saveOneRowForLastRaw(pColVal, pReader, slotId, pColInfoData, numOfRows);
continue;
}
p = (SFirstLastRes*)varDataVal(pRes[i]); p = (SFirstLastRes*)varDataVal(pRes[i]);
p->ts = pColVal->ts; p->ts = pColVal->ts;
ts = p->ts; ts = p->ts;
p->isNull = !COL_VAL_IS_VALUE(&pColVal->colVal); p->isNull = !COL_VAL_IS_VALUE(&pColVal->colVal);
// allNullRow = p->isNull & allNullRow; // allNullRow = p->isNull & allNullRow;
if (!p->isNull) { if (!p->isNull) {
if (IS_VAR_DATA_TYPE(pColVal->colVal.type)) { if (IS_VAR_DATA_TYPE(pColVal->colVal.type)) {
varDataSetLen(p->buf, pColVal->colVal.value.nData); varDataSetLen(p->buf, pColVal->colVal.value.nData);
@ -77,6 +116,13 @@ static int32_t saveOneRow(SArray* pRow, SSDataBlock* pBlock, SCacheRowsReader* p
} }
for (int32_t idx = 0; idx < taosArrayGetSize(pBlock->pDataBlock); ++idx) { for (int32_t idx = 0; idx < taosArrayGetSize(pBlock->pDataBlock); ++idx) {
SColumnInfoData* pCol = taosArrayGet(pBlock->pDataBlock, idx); SColumnInfoData* pCol = taosArrayGet(pBlock->pDataBlock, idx);
if (idx < funcTypeBlockArray->size) {
int32_t funcType = *(int32_t*)taosArrayGet(funcTypeBlockArray, idx);
if (FUNCTION_TYPE_CACHE_LAST_ROW == funcType) {
continue;
}
}
if (pCol->info.colId == PRIMARYKEY_TIMESTAMP_COL_ID && pCol->info.type == TSDB_DATA_TYPE_TIMESTAMP) { if (pCol->info.colId == PRIMARYKEY_TIMESTAMP_COL_ID && pCol->info.type == TSDB_DATA_TYPE_TIMESTAMP) {
if (ts == TSKEY_MIN) { if (ts == TSKEY_MIN) {
colDataSetNULL(pCol, numOfRows); colDataSetNULL(pCol, numOfRows);
@ -95,6 +141,7 @@ static int32_t saveOneRow(SArray* pRow, SSDataBlock* pBlock, SCacheRowsReader* p
// pBlock->info.rows += allNullRow ? 0 : 1; // pBlock->info.rows += allNullRow ? 0 : 1;
++pBlock->info.rows; ++pBlock->info.rows;
taosArrayDestroy(funcTypeBlockArray);
} else if (HASTYPE(pReader->type, CACHESCAN_RETRIEVE_LAST_ROW)) { } else if (HASTYPE(pReader->type, CACHESCAN_RETRIEVE_LAST_ROW)) {
for (int32_t i = 0; i < pReader->numOfCols; ++i) { for (int32_t i = 0; i < pReader->numOfCols; ++i) {
SColumnInfoData* pColInfoData = taosArrayGet(pBlock->pDataBlock, dstSlotIds[i]); SColumnInfoData* pColInfoData = taosArrayGet(pBlock->pDataBlock, dstSlotIds[i]);
@ -105,21 +152,8 @@ static int32_t saveOneRow(SArray* pRow, SSDataBlock* pBlock, SCacheRowsReader* p
continue; continue;
} }
SLastCol* pColVal = (SLastCol*)taosArrayGet(pRow, i); SLastCol* pColVal = (SLastCol*)taosArrayGet(pRow, i);
SColVal* pVal = &pColVal->colVal;
// allNullRow = false; saveOneRowForLastRaw(pColVal, pReader, slotId, pColInfoData, numOfRows);
if (IS_VAR_DATA_TYPE(pColVal->colVal.type)) {
if (!COL_VAL_IS_VALUE(&pColVal->colVal)) {
colDataSetNULL(pColInfoData, numOfRows);
} else {
varDataSetLen(pReader->transferBuf[slotId], pVal->value.nData);
memcpy(varDataVal(pReader->transferBuf[slotId]), pVal->value.pData, pVal->value.nData);
colDataSetVal(pColInfoData, numOfRows, pReader->transferBuf[slotId], false);
}
} else {
colDataSetVal(pColInfoData, numOfRows, (const char*)&pVal->value.val, !COL_VAL_IS_VALUE(pVal));
}
} }
// pBlock->info.rows += allNullRow ? 0 : 1; // pBlock->info.rows += allNullRow ? 0 : 1;
@ -175,7 +209,8 @@ int32_t tsdbReuseCacherowsReader(void* reader, void* pTableIdList, int32_t numOf
} }
int32_t tsdbCacherowsReaderOpen(void* pVnode, int32_t type, void* pTableIdList, int32_t numOfTables, int32_t numOfCols, int32_t tsdbCacherowsReaderOpen(void* pVnode, int32_t type, void* pTableIdList, int32_t numOfTables, int32_t numOfCols,
SArray* pCidList, int32_t* pSlotIds, uint64_t suid, void** pReader, const char* idstr) { SArray* pCidList, int32_t* pSlotIds, uint64_t suid, void** pReader, const char* idstr,
SArray* pFuncTypeList) {
*pReader = NULL; *pReader = NULL;
SCacheRowsReader* p = taosMemoryCalloc(1, sizeof(SCacheRowsReader)); SCacheRowsReader* p = taosMemoryCalloc(1, sizeof(SCacheRowsReader));
if (p == NULL) { if (p == NULL) {
@ -190,6 +225,7 @@ int32_t tsdbCacherowsReaderOpen(void* pVnode, int32_t type, void* pTableIdList,
p->numOfCols = numOfCols; p->numOfCols = numOfCols;
p->pCidList = pCidList; p->pCidList = pCidList;
p->pSlotIds = pSlotIds; p->pSlotIds = pSlotIds;
p->pFuncTypeList = pFuncTypeList;
if (numOfTables == 0) { if (numOfTables == 0) {
*pReader = p; *pReader = p;
@ -391,8 +427,11 @@ int32_t tsdbRetrieveCacheRows(void* pReader, SSDataBlock* pResBlock, const int32
if (!COL_VAL_IS_VALUE(&p->colVal)) { if (!COL_VAL_IS_VALUE(&p->colVal)) {
hasNotNullRow = false; hasNotNullRow = false;
} }
// For all of cols is null, the last null col of last table will be save
if (i != pr->numOfTables - 1 || k != pr->numOfCols - 1 || hasRes) {
continue; continue;
} }
}
hasRes = true; hasRes = true;
p->ts = pColVal->ts; p->ts = pColVal->ts;

View File

@ -348,6 +348,7 @@ typedef struct SCacheRowsReader {
STsdbReadSnap* pReadSnap; STsdbReadSnap* pReadSnap;
char* idstr; char* idstr;
int64_t lastTs; int64_t lastTs;
SArray* pFuncTypeList;
} SCacheRowsReader; } SCacheRowsReader;
int32_t tsdbCacheGetBatch(STsdb* pTsdb, tb_uid_t uid, SArray* pLastArray, SCacheRowsReader* pr, int8_t ltype); int32_t tsdbCacheGetBatch(STsdb* pTsdb, tb_uid_t uid, SArray* pLastArray, SCacheRowsReader* pr, int8_t ltype);

View File

@ -82,6 +82,7 @@ typedef struct SColMatchItem {
int32_t dstSlotId; int32_t dstSlotId;
bool needOutput; bool needOutput;
SDataType dataType; SDataType dataType;
int32_t funcType;
} SColMatchItem; } SColMatchItem;
typedef struct SColMatchInfo { typedef struct SColMatchInfo {

View File

@ -273,6 +273,7 @@ SSDataBlock* getAggregateResult(SOperatorInfo* pOperator) {
} }
int32_t doAggregateImpl(SOperatorInfo* pOperator, SqlFunctionCtx* pCtx) { int32_t doAggregateImpl(SOperatorInfo* pOperator, SqlFunctionCtx* pCtx) {
int32_t code = TSDB_CODE_SUCCESS;
for (int32_t k = 0; k < pOperator->exprSupp.numOfExprs; ++k) { for (int32_t k = 0; k < pOperator->exprSupp.numOfExprs; ++k) {
if (functionNeedToExecute(&pCtx[k])) { if (functionNeedToExecute(&pCtx[k])) {
// todo add a dummy funtion to avoid process check // todo add a dummy funtion to avoid process check
@ -280,7 +281,13 @@ int32_t doAggregateImpl(SOperatorInfo* pOperator, SqlFunctionCtx* pCtx) {
continue; continue;
} }
int32_t code = pCtx[k].fpSet.process(&pCtx[k]); if ((&pCtx[k])->input.pData[0] == NULL) {
code = TSDB_CODE_QRY_EXECUTOR_INTERNAL_ERROR;
qError("%s aggregate function error happens, input data is NULL.", GET_TASKID(pOperator->pTaskInfo));
} else {
code = pCtx[k].fpSet.process(&pCtx[k]);
}
if (code != TSDB_CODE_SUCCESS) { if (code != TSDB_CODE_SUCCESS) {
qError("%s aggregate function error happens, code: %s", GET_TASKID(pOperator->pTaskInfo), tstrerror(code)); qError("%s aggregate function error happens, code: %s", GET_TASKID(pOperator->pTaskInfo), tstrerror(code));
return code; return code;
@ -562,7 +569,12 @@ void applyAggFunctionOnPartialTuples(SExecTaskInfo* taskInfo, SqlFunctionCtx* pC
} else { } else {
int32_t code = TSDB_CODE_SUCCESS; int32_t code = TSDB_CODE_SUCCESS;
if (functionNeedToExecute(&pCtx[k]) && pCtx[k].fpSet.process != NULL) { if (functionNeedToExecute(&pCtx[k]) && pCtx[k].fpSet.process != NULL) {
if ((&pCtx[k])->input.pData[0] == NULL) {
code = TSDB_CODE_QRY_EXECUTOR_INTERNAL_ERROR;
qError("%s apply functions error, input data is NULL.", GET_TASKID(taskInfo));
} else {
code = pCtx[k].fpSet.process(&pCtx[k]); code = pCtx[k].fpSet.process(&pCtx[k]);
}
if (code != TSDB_CODE_SUCCESS) { if (code != TSDB_CODE_SUCCESS) {
qError("%s apply functions error, code: %s", GET_TASKID(taskInfo), tstrerror(code)); qError("%s apply functions error, code: %s", GET_TASKID(taskInfo), tstrerror(code));

View File

@ -21,6 +21,7 @@
#include "tmsg.h" #include "tmsg.h"
#include "executorInt.h" #include "executorInt.h"
#include "functionMgt.h"
#include "operator.h" #include "operator.h"
#include "querytask.h" #include "querytask.h"
#include "tcompare.h" #include "tcompare.h"
@ -44,6 +45,7 @@ typedef struct SCacheRowsScanInfo {
SArray* pCidList; SArray* pCidList;
int32_t indexOfBufferedRes; int32_t indexOfBufferedRes;
STableListInfo* pTableList; STableListInfo* pTableList;
SArray* pFuncTypeList;
} SCacheRowsScanInfo; } SCacheRowsScanInfo;
static SSDataBlock* doScanCache(SOperatorInfo* pOperator); static SSDataBlock* doScanCache(SOperatorInfo* pOperator);
@ -105,9 +107,15 @@ SOperatorInfo* createCacherowsScanOperator(SLastRowScanPhysiNode* pScanNode, SRe
} }
SArray* pCidList = taosArrayInit(numOfCols, sizeof(int16_t)); SArray* pCidList = taosArrayInit(numOfCols, sizeof(int16_t));
pInfo->pFuncTypeList = taosArrayInit(taosArrayGetSize(pScanNode->pFuncTypes), sizeof(int32_t));
taosArrayAddAll(pInfo->pFuncTypeList, pScanNode->pFuncTypes);
for (int i = 0; i < TARRAY_SIZE(pInfo->matchInfo.pList); ++i) { for (int i = 0; i < TARRAY_SIZE(pInfo->matchInfo.pList); ++i) {
SColMatchItem* pColInfo = taosArrayGet(pInfo->matchInfo.pList, i); SColMatchItem* pColInfo = taosArrayGet(pInfo->matchInfo.pList, i);
taosArrayPush(pCidList, &pColInfo->colId); taosArrayPush(pCidList, &pColInfo->colId);
if (pInfo->pFuncTypeList != NULL && taosArrayGetSize(pInfo->pFuncTypeList) > i) {
pColInfo->funcType = *(int32_t*)taosArrayGet(pInfo->pFuncTypeList, i);
}
} }
pInfo->pCidList = pCidList; pInfo->pCidList = pCidList;
@ -132,7 +140,7 @@ SOperatorInfo* createCacherowsScanOperator(SLastRowScanPhysiNode* pScanNode, SRe
uint64_t suid = tableListGetSuid(pTableListInfo); uint64_t suid = tableListGetSuid(pTableListInfo);
code = pInfo->readHandle.api.cacheFn.openReader(pInfo->readHandle.vnode, pInfo->retrieveType, pList, totalTables, code = pInfo->readHandle.api.cacheFn.openReader(pInfo->readHandle.vnode, pInfo->retrieveType, pList, totalTables,
taosArrayGetSize(pInfo->matchInfo.pList), pCidList, pInfo->pSlotIds, taosArrayGetSize(pInfo->matchInfo.pList), pCidList, pInfo->pSlotIds,
suid, &pInfo->pLastrowReader, pTaskInfo->id.str); suid, &pInfo->pLastrowReader, pTaskInfo->id.str, pScanNode->pFuncTypes);
if (code != TSDB_CODE_SUCCESS) { if (code != TSDB_CODE_SUCCESS) {
goto _error; goto _error;
} }
@ -274,7 +282,7 @@ SSDataBlock* doScanCache(SOperatorInfo* pOperator) {
if (NULL == pInfo->pLastrowReader) { if (NULL == pInfo->pLastrowReader) {
code = pInfo->readHandle.api.cacheFn.openReader(pInfo->readHandle.vnode, pInfo->retrieveType, pList, num, code = pInfo->readHandle.api.cacheFn.openReader(pInfo->readHandle.vnode, pInfo->retrieveType, pList, num,
taosArrayGetSize(pInfo->matchInfo.pList), pInfo->pCidList, pInfo->pSlotIds, suid, &pInfo->pLastrowReader, taosArrayGetSize(pInfo->matchInfo.pList), pInfo->pCidList, pInfo->pSlotIds, suid, &pInfo->pLastrowReader,
pTaskInfo->id.str); pTaskInfo->id.str, pInfo->pFuncTypeList);
if (code != TSDB_CODE_SUCCESS) { if (code != TSDB_CODE_SUCCESS) {
pInfo->currentGroupIndex += 1; pInfo->currentGroupIndex += 1;
taosArrayClear(pInfo->pUidList); taosArrayClear(pInfo->pUidList);
@ -333,6 +341,7 @@ void destroyCacheScanOperator(void* param) {
taosMemoryFree(pInfo->pSlotIds); taosMemoryFree(pInfo->pSlotIds);
taosMemoryFree(pInfo->pDstSlotIds); taosMemoryFree(pInfo->pDstSlotIds);
taosArrayDestroy(pInfo->pCidList); taosArrayDestroy(pInfo->pCidList);
taosArrayDestroy(pInfo->pFuncTypeList);
taosArrayDestroy(pInfo->pUidList); taosArrayDestroy(pInfo->pUidList);
taosArrayDestroy(pInfo->matchInfo.pList); taosArrayDestroy(pInfo->matchInfo.pList);
tableListDestroy(pInfo->pTableList); tableListDestroy(pInfo->pTableList);
@ -405,6 +414,8 @@ int32_t removeRedundantTsCol(SLastRowScanPhysiNode* pScanNode, SColMatchInfo* pC
SSlotDescNode* pDesc = (SSlotDescNode*)nodesListGetNode(pList, slotId); SSlotDescNode* pDesc = (SSlotDescNode*)nodesListGetNode(pList, slotId);
if (pDesc->dataType.type != TSDB_DATA_TYPE_TIMESTAMP) { if (pDesc->dataType.type != TSDB_DATA_TYPE_TIMESTAMP) {
taosArrayPush(pMatchInfo, pColInfo); taosArrayPush(pMatchInfo, pColInfo);
} else if (FUNCTION_TYPE_CACHE_LAST_ROW == pColInfo->funcType){
taosArrayPush(pMatchInfo, pColInfo);
} }
} }

View File

@ -1343,7 +1343,6 @@ int32_t extractColMatchInfo(SNodeList* pNodeList, SDataBlockDescNode* pOutputNod
c.colId = pColNode->colId; c.colId = pColNode->colId;
c.srcSlotId = pColNode->slotId; c.srcSlotId = pColNode->slotId;
c.dstSlotId = pNode->slotId; c.dstSlotId = pNode->slotId;
c.dataType = pColNode->node.resType;
taosArrayPush(pList, &c); taosArrayPush(pList, &c);
} }
} }

View File

@ -622,6 +622,10 @@ int32_t qExecTaskOpt(qTaskInfo_t tinfo, SArray* pResList, uint64_t* useconds, bo
pRes = pTaskInfo->pRoot->fpSet.getNextFn(pTaskInfo->pRoot); pRes = pTaskInfo->pRoot->fpSet.getNextFn(pTaskInfo->pRoot);
} }
if(pRes == NULL) {
st = taosGetTimestampUs();
}
int32_t rowsThreshold = pTaskInfo->pSubplan->rowsThreshold; int32_t rowsThreshold = pTaskInfo->pSubplan->rowsThreshold;
if (!pTaskInfo->pSubplan->dynamicRowThreshold || 4096 <= pTaskInfo->pSubplan->rowsThreshold) { if (!pTaskInfo->pSubplan->dynamicRowThreshold || 4096 <= pTaskInfo->pSubplan->rowsThreshold) {
rowsThreshold = 4096; rowsThreshold = 4096;

View File

@ -2893,6 +2893,14 @@ SOperatorInfo* createStreamSessionAggOperatorInfo(SOperatorInfo* downstream, SPh
goto _error; goto _error;
} }
pInfo->twAggSup = (STimeWindowAggSupp){
.waterMark = pSessionNode->window.watermark,
.calTrigger = pSessionNode->window.triggerType,
.maxTs = INT64_MIN,
.minTs = INT64_MAX,
.deleteMark = getDeleteMark(&pSessionNode->window, 0),
};
code = initStreamAggSupporter(&pInfo->streamAggSup, pExpSup, numOfCols, pSessionNode->gap, code = initStreamAggSupporter(&pInfo->streamAggSup, pExpSup, numOfCols, pSessionNode->gap,
pTaskInfo->streamInfo.pState, 0, 0, &pTaskInfo->storageAPI.stateStore, pHandle, pTaskInfo->streamInfo.pState, 0, 0, &pTaskInfo->storageAPI.stateStore, pHandle,
&pInfo->twAggSup, GET_TASKID(pTaskInfo), &pTaskInfo->storageAPI); &pInfo->twAggSup, GET_TASKID(pTaskInfo), &pTaskInfo->storageAPI);
@ -2900,13 +2908,6 @@ SOperatorInfo* createStreamSessionAggOperatorInfo(SOperatorInfo* downstream, SPh
goto _error; goto _error;
} }
pInfo->twAggSup = (STimeWindowAggSupp){
.waterMark = pSessionNode->window.watermark,
.calTrigger = pSessionNode->window.triggerType,
.maxTs = INT64_MIN,
.minTs = INT64_MAX,
};
initExecTimeWindowInfo(&pInfo->twAggSup.timeWindowData, &pTaskInfo->window); initExecTimeWindowInfo(&pInfo->twAggSup.timeWindowData, &pTaskInfo->window);
pInfo->primaryTsIndex = ((SColumnNode*)pSessionNode->window.pTspk)->slotId; pInfo->primaryTsIndex = ((SColumnNode*)pSessionNode->window.pTspk)->slotId;
@ -3775,6 +3776,7 @@ SOperatorInfo* createStreamStateAggOperatorInfo(SOperatorInfo* downstream, SPhys
.calTrigger = pStateNode->window.triggerType, .calTrigger = pStateNode->window.triggerType,
.maxTs = INT64_MIN, .maxTs = INT64_MIN,
.minTs = INT64_MAX, .minTs = INT64_MAX,
.deleteMark = getDeleteMark(&pStateNode->window, 0),
}; };
initExecTimeWindowInfo(&pInfo->twAggSup.timeWindowData, &pTaskInfo->window); initExecTimeWindowInfo(&pInfo->twAggSup.timeWindowData, &pTaskInfo->window);

View File

@ -258,13 +258,24 @@ static int32_t addDbPrecisonParam(SNodeList** pList, uint8_t precision) {
return TSDB_CODE_SUCCESS; return TSDB_CODE_SUCCESS;
} }
static SDataType* getSDataTypeFromNode(SNode* pNode) {
if (pNode == NULL) return NULL;
if (nodesIsExprNode(pNode)) {
return &((SExprNode*)pNode)->resType;
} else if (QUERY_NODE_COLUMN_REF == pNode->type) {
return &((SColumnRefNode*)pNode)->resType;
} else {
return NULL;
}
}
// There is only one parameter of numeric type, and the return type is parameter type // There is only one parameter of numeric type, and the return type is parameter type
static int32_t translateInOutNum(SFunctionNode* pFunc, char* pErrBuf, int32_t len) { static int32_t translateInOutNum(SFunctionNode* pFunc, char* pErrBuf, int32_t len) {
if (1 != LIST_LENGTH(pFunc->pParameterList)) { if (1 != LIST_LENGTH(pFunc->pParameterList)) {
return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName);
} }
uint8_t paraType = ((SExprNode*)nodesListGetNode(pFunc->pParameterList, 0))->resType.type; uint8_t paraType = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 0))->type;
if (!IS_NUMERIC_TYPE(paraType) && !IS_NULL_TYPE(paraType)) { if (!IS_NUMERIC_TYPE(paraType) && !IS_NULL_TYPE(paraType)) {
return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName);
} else if (IS_NULL_TYPE(paraType)) { } else if (IS_NULL_TYPE(paraType)) {
@ -281,7 +292,7 @@ static int32_t translateInNumOutDou(SFunctionNode* pFunc, char* pErrBuf, int32_t
return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName);
} }
uint8_t paraType = ((SExprNode*)nodesListGetNode(pFunc->pParameterList, 0))->resType.type; uint8_t paraType = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 0))->type;
if (!IS_NUMERIC_TYPE(paraType) && !IS_NULL_TYPE(paraType)) { if (!IS_NUMERIC_TYPE(paraType) && !IS_NULL_TYPE(paraType)) {
return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName);
} }
@ -296,8 +307,8 @@ static int32_t translateIn2NumOutDou(SFunctionNode* pFunc, char* pErrBuf, int32_
return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName);
} }
uint8_t para1Type = ((SExprNode*)nodesListGetNode(pFunc->pParameterList, 0))->resType.type; uint8_t para1Type = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 0))->type;
uint8_t para2Type = ((SExprNode*)nodesListGetNode(pFunc->pParameterList, 1))->resType.type; uint8_t para2Type = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 1))->type;
if ((!IS_NUMERIC_TYPE(para1Type) && !IS_NULL_TYPE(para1Type)) || if ((!IS_NUMERIC_TYPE(para1Type) && !IS_NULL_TYPE(para1Type)) ||
(!IS_NUMERIC_TYPE(para2Type) && !IS_NULL_TYPE(para2Type))) { (!IS_NUMERIC_TYPE(para2Type) && !IS_NULL_TYPE(para2Type))) {
return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName);
@ -313,12 +324,12 @@ static int32_t translateInOutStr(SFunctionNode* pFunc, char* pErrBuf, int32_t le
return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName);
} }
SExprNode* pPara1 = (SExprNode*)nodesListGetNode(pFunc->pParameterList, 0); SDataType* pRestType = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 0));
if (TSDB_DATA_TYPE_VARBINARY == pPara1->resType.type || !IS_STR_DATA_TYPE(pPara1->resType.type)) { if (TSDB_DATA_TYPE_VARBINARY == pRestType->type || !IS_STR_DATA_TYPE(pRestType->type)) {
return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName);
} }
pFunc->node.resType = (SDataType){.bytes = pPara1->resType.bytes, .type = pPara1->resType.type}; pFunc->node.resType = (SDataType){.bytes = pRestType->bytes, .type = pRestType->type};
return TSDB_CODE_SUCCESS; return TSDB_CODE_SUCCESS;
} }
@ -327,8 +338,8 @@ static int32_t translateTrimStr(SFunctionNode* pFunc, char* pErrBuf, int32_t len
return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName);
} }
SExprNode* pPara1 = (SExprNode*)nodesListGetNode(pFunc->pParameterList, 0); SDataType* pRestType1 = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 0));
if (TSDB_DATA_TYPE_VARBINARY == pPara1->resType.type || !IS_STR_DATA_TYPE(pPara1->resType.type)) { if (TSDB_DATA_TYPE_VARBINARY == pRestType1->type || !IS_STR_DATA_TYPE(pRestType1->type)) {
return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName);
} }
@ -342,8 +353,8 @@ static int32_t translateTrimStr(SFunctionNode* pFunc, char* pErrBuf, int32_t len
numOfSpaces = countTrailingSpaces(pValue, isLtrim); numOfSpaces = countTrailingSpaces(pValue, isLtrim);
} }
int32_t resBytes = pPara1->resType.bytes - numOfSpaces; int32_t resBytes = pRestType1->bytes - numOfSpaces;
pFunc->node.resType = (SDataType){.bytes = resBytes, .type = pPara1->resType.type}; pFunc->node.resType = (SDataType){.bytes = resBytes, .type = pRestType1->type};
return TSDB_CODE_SUCCESS; return TSDB_CODE_SUCCESS;
} }
@ -361,13 +372,13 @@ static int32_t translateLogarithm(SFunctionNode* pFunc, char* pErrBuf, int32_t l
return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName);
} }
uint8_t para1Type = ((SExprNode*)nodesListGetNode(pFunc->pParameterList, 0))->resType.type; uint8_t para1Type = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 0))->type;
if (!IS_NUMERIC_TYPE(para1Type) && !IS_NULL_TYPE(para1Type)) { if (!IS_NUMERIC_TYPE(para1Type) && !IS_NULL_TYPE(para1Type)) {
return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName);
} }
if (2 == numOfParams) { if (2 == numOfParams) {
uint8_t para2Type = ((SExprNode*)nodesListGetNode(pFunc->pParameterList, 1))->resType.type; uint8_t para2Type = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 1))->type;
if (!IS_NUMERIC_TYPE(para2Type) && !IS_NULL_TYPE(para2Type)) { if (!IS_NUMERIC_TYPE(para2Type) && !IS_NULL_TYPE(para2Type)) {
return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName);
} }
@ -390,7 +401,7 @@ static int32_t translateSum(SFunctionNode* pFunc, char* pErrBuf, int32_t len) {
return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName);
} }
uint8_t paraType = ((SExprNode*)nodesListGetNode(pFunc->pParameterList, 0))->resType.type; uint8_t paraType = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 0))->type;
if (!IS_NUMERIC_TYPE(paraType) && !IS_NULL_TYPE(paraType)) { if (!IS_NUMERIC_TYPE(paraType) && !IS_NULL_TYPE(paraType)) {
return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName);
} }
@ -413,7 +424,7 @@ static int32_t translateAvgPartial(SFunctionNode* pFunc, char* pErrBuf, int32_t
return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName);
} }
uint8_t paraType = ((SExprNode*)nodesListGetNode(pFunc->pParameterList, 0))->resType.type; uint8_t paraType = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 0))->type;
if (!IS_NUMERIC_TYPE(paraType) && !IS_NULL_TYPE(paraType)) { if (!IS_NUMERIC_TYPE(paraType) && !IS_NULL_TYPE(paraType)) {
return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName);
} }
@ -427,7 +438,7 @@ static int32_t translateAvgMerge(SFunctionNode* pFunc, char* pErrBuf, int32_t le
return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName);
} }
uint8_t paraType = ((SExprNode*)nodesListGetNode(pFunc->pParameterList, 0))->resType.type; uint8_t paraType = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 0))->type;
if (TSDB_DATA_TYPE_BINARY != paraType) { if (TSDB_DATA_TYPE_BINARY != paraType) {
return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName);
} }
@ -442,7 +453,7 @@ static int32_t translateStddevPartial(SFunctionNode* pFunc, char* pErrBuf, int32
return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName);
} }
uint8_t paraType = ((SExprNode*)nodesListGetNode(pFunc->pParameterList, 0))->resType.type; uint8_t paraType = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 0))->type;
if (!IS_NUMERIC_TYPE(paraType) && !IS_NULL_TYPE(paraType)) { if (!IS_NUMERIC_TYPE(paraType) && !IS_NULL_TYPE(paraType)) {
return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName);
} }
@ -456,7 +467,7 @@ static int32_t translateStddevMerge(SFunctionNode* pFunc, char* pErrBuf, int32_t
return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName);
} }
uint8_t paraType = ((SExprNode*)nodesListGetNode(pFunc->pParameterList, 0))->resType.type; uint8_t paraType = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 0))->type;
if (TSDB_DATA_TYPE_BINARY != paraType) { if (TSDB_DATA_TYPE_BINARY != paraType) {
return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName);
} }
@ -515,7 +526,7 @@ static int32_t translatePercentile(SFunctionNode* pFunc, char* pErrBuf, int32_t
return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName);
} }
uint8_t para1Type = ((SExprNode*)nodesListGetNode(pFunc->pParameterList, 0))->resType.type; uint8_t para1Type = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 0))->type;
if (!IS_NUMERIC_TYPE(para1Type)) { if (!IS_NUMERIC_TYPE(para1Type)) {
return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName);
} }
@ -524,7 +535,7 @@ static int32_t translatePercentile(SFunctionNode* pFunc, char* pErrBuf, int32_t
SValueNode* pValue = (SValueNode*)nodesListGetNode(pFunc->pParameterList, i); SValueNode* pValue = (SValueNode*)nodesListGetNode(pFunc->pParameterList, i);
pValue->notReserved = true; pValue->notReserved = true;
uint8_t paraType = ((SExprNode*)nodesListGetNode(pFunc->pParameterList, i))->resType.type; uint8_t paraType = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, i))->type;
if (!IS_NUMERIC_TYPE(paraType)) { if (!IS_NUMERIC_TYPE(paraType)) {
return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName);
} }
@ -577,15 +588,15 @@ static int32_t translateApercentile(SFunctionNode* pFunc, char* pErrBuf, int32_t
pValue->notReserved = true; pValue->notReserved = true;
uint8_t para1Type = ((SExprNode*)nodesListGetNode(pFunc->pParameterList, 0))->resType.type; uint8_t para1Type = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 0))->type;
uint8_t para2Type = ((SExprNode*)nodesListGetNode(pFunc->pParameterList, 1))->resType.type; uint8_t para2Type = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 1))->type;
if (!IS_NUMERIC_TYPE(para1Type) || !IS_INTEGER_TYPE(para2Type)) { if (!IS_NUMERIC_TYPE(para1Type) || !IS_INTEGER_TYPE(para2Type)) {
return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName);
} }
// param2 // param2
if (3 == numOfParams) { if (3 == numOfParams) {
uint8_t para3Type = ((SExprNode*)nodesListGetNode(pFunc->pParameterList, 2))->resType.type; uint8_t para3Type = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 2))->type;
if (!IS_STR_DATA_TYPE(para3Type)) { if (!IS_STR_DATA_TYPE(para3Type)) {
return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName);
} }
@ -624,15 +635,15 @@ static int32_t translateApercentileImpl(SFunctionNode* pFunc, char* pErrBuf, int
pValue->notReserved = true; pValue->notReserved = true;
uint8_t para1Type = ((SExprNode*)nodesListGetNode(pFunc->pParameterList, 0))->resType.type; uint8_t para1Type = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 0))->type;
uint8_t para2Type = ((SExprNode*)nodesListGetNode(pFunc->pParameterList, 1))->resType.type; uint8_t para2Type = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 1))->type;
if (!IS_NUMERIC_TYPE(para1Type) || !IS_INTEGER_TYPE(para2Type)) { if (!IS_NUMERIC_TYPE(para1Type) || !IS_INTEGER_TYPE(para2Type)) {
return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName);
} }
// param2 // param2
if (3 == numOfParams) { if (3 == numOfParams) {
uint8_t para3Type = ((SExprNode*)nodesListGetNode(pFunc->pParameterList, 2))->resType.type; uint8_t para3Type = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 2))->type;
if (!IS_STR_DATA_TYPE(para3Type)) { if (!IS_STR_DATA_TYPE(para3Type)) {
return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName);
} }
@ -654,14 +665,14 @@ static int32_t translateApercentileImpl(SFunctionNode* pFunc, char* pErrBuf, int
if (3 != numOfParams && 2 != numOfParams) { if (3 != numOfParams && 2 != numOfParams) {
return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName);
} }
uint8_t para1Type = ((SExprNode*)nodesListGetNode(pFunc->pParameterList, 0))->resType.type; uint8_t para1Type = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 0))->type;
uint8_t para2Type = ((SExprNode*)nodesListGetNode(pFunc->pParameterList, 1))->resType.type; uint8_t para2Type = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 1))->type;
if (TSDB_DATA_TYPE_BINARY != para1Type || !IS_INTEGER_TYPE(para2Type)) { if (TSDB_DATA_TYPE_BINARY != para1Type || !IS_INTEGER_TYPE(para2Type)) {
return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName);
} }
if (3 == numOfParams) { if (3 == numOfParams) {
uint8_t para3Type = ((SExprNode*)nodesListGetNode(pFunc->pParameterList, 2))->resType.type; uint8_t para3Type = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 2))->type;
if (!IS_STR_DATA_TYPE(para3Type)) { if (!IS_STR_DATA_TYPE(para3Type)) {
return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName);
} }
@ -715,8 +726,8 @@ static int32_t translateTopBot(SFunctionNode* pFunc, char* pErrBuf, int32_t len)
return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName);
} }
uint8_t para1Type = ((SExprNode*)nodesListGetNode(pFunc->pParameterList, 0))->resType.type; uint8_t para1Type = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 0))->type;
uint8_t para2Type = ((SExprNode*)nodesListGetNode(pFunc->pParameterList, 1))->resType.type; uint8_t para2Type = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 1))->type;
if (!IS_NUMERIC_TYPE(para1Type) || !IS_INTEGER_TYPE(para2Type)) { if (!IS_NUMERIC_TYPE(para1Type) || !IS_INTEGER_TYPE(para2Type)) {
return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName);
} }
@ -739,7 +750,7 @@ static int32_t translateTopBot(SFunctionNode* pFunc, char* pErrBuf, int32_t len)
pValue->notReserved = true; pValue->notReserved = true;
// set result type // set result type
SDataType* pType = &((SExprNode*)nodesListGetNode(pFunc->pParameterList, 0))->resType; SDataType* pType = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 0));
pFunc->node.resType = (SDataType){.bytes = pType->bytes, .type = pType->type}; pFunc->node.resType = (SDataType){.bytes = pType->bytes, .type = pType->type};
return TSDB_CODE_SUCCESS; return TSDB_CODE_SUCCESS;
} }
@ -769,7 +780,7 @@ static int32_t translateSpread(SFunctionNode* pFunc, char* pErrBuf, int32_t len)
return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName);
} }
uint8_t paraType = ((SExprNode*)nodesListGetNode(pFunc->pParameterList, 0))->resType.type; uint8_t paraType = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 0))->type;
if (!IS_NUMERIC_TYPE(paraType) && !IS_TIMESTAMP_TYPE(paraType)) { if (!IS_NUMERIC_TYPE(paraType) && !IS_TIMESTAMP_TYPE(paraType)) {
return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName);
} }
@ -783,7 +794,7 @@ static int32_t translateSpreadImpl(SFunctionNode* pFunc, char* pErrBuf, int32_t
return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName);
} }
uint8_t paraType = ((SExprNode*)nodesListGetNode(pFunc->pParameterList, 0))->resType.type; uint8_t paraType = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 0))->type;
if (isPartial) { if (isPartial) {
if (!IS_NUMERIC_TYPE(paraType) && !IS_TIMESTAMP_TYPE(paraType)) { if (!IS_NUMERIC_TYPE(paraType) && !IS_TIMESTAMP_TYPE(paraType)) {
return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName);
@ -859,7 +870,7 @@ static int32_t translateElapsedImpl(SFunctionNode* pFunc, char* pErrBuf, int32_t
return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName);
} }
uint8_t paraType = ((SExprNode*)nodesListGetNode(pFunc->pParameterList, 0))->resType.type; uint8_t paraType = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 0))->type;
if (!IS_TIMESTAMP_TYPE(paraType)) { if (!IS_TIMESTAMP_TYPE(paraType)) {
return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName);
} }
@ -867,6 +878,7 @@ static int32_t translateElapsedImpl(SFunctionNode* pFunc, char* pErrBuf, int32_t
// param1 // param1
if (2 == numOfParams) { if (2 == numOfParams) {
SNode* pParamNode1 = nodesListGetNode(pFunc->pParameterList, 1); SNode* pParamNode1 = nodesListGetNode(pFunc->pParameterList, 1);
if (QUERY_NODE_VALUE != nodeType(pParamNode1)) { if (QUERY_NODE_VALUE != nodeType(pParamNode1)) {
return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName);
} }
@ -875,7 +887,7 @@ static int32_t translateElapsedImpl(SFunctionNode* pFunc, char* pErrBuf, int32_t
pValue->notReserved = true; pValue->notReserved = true;
paraType = ((SExprNode*)nodesListGetNode(pFunc->pParameterList, 1))->resType.type; paraType = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 1))->type;
if (!IS_INTEGER_TYPE(paraType)) { if (!IS_INTEGER_TYPE(paraType)) {
return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName);
} }
@ -893,7 +905,7 @@ static int32_t translateElapsedImpl(SFunctionNode* pFunc, char* pErrBuf, int32_t
return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName);
} }
uint8_t paraType = ((SExprNode*)nodesListGetNode(pFunc->pParameterList, 0))->resType.type; uint8_t paraType = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 0))->type;
if (TSDB_DATA_TYPE_BINARY != paraType) { if (TSDB_DATA_TYPE_BINARY != paraType) {
return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName);
} }
@ -934,7 +946,7 @@ static int32_t translateLeastSQR(SFunctionNode* pFunc, char* pErrBuf, int32_t le
pValue->notReserved = true; pValue->notReserved = true;
} }
uint8_t colType = ((SExprNode*)nodesListGetNode(pFunc->pParameterList, i))->resType.type; uint8_t colType = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, i))->type;
if (!IS_NUMERIC_TYPE(colType)) { if (!IS_NUMERIC_TYPE(colType)) {
return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName);
} }
@ -1128,15 +1140,15 @@ static int32_t translateHistogram(SFunctionNode* pFunc, char* pErrBuf, int32_t l
return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName);
} }
uint8_t colType = ((SExprNode*)nodesListGetNode(pFunc->pParameterList, 0))->resType.type; uint8_t colType = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 0))->type;
if (!IS_NUMERIC_TYPE(colType)) { if (!IS_NUMERIC_TYPE(colType)) {
return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName);
} }
// param1 ~ param3 // param1 ~ param3
if (((SExprNode*)nodesListGetNode(pFunc->pParameterList, 1))->resType.type != TSDB_DATA_TYPE_BINARY || if (getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 1))->type != TSDB_DATA_TYPE_BINARY ||
((SExprNode*)nodesListGetNode(pFunc->pParameterList, 2))->resType.type != TSDB_DATA_TYPE_BINARY || getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 2))->type != TSDB_DATA_TYPE_BINARY ||
!IS_INTEGER_TYPE(((SExprNode*)nodesListGetNode(pFunc->pParameterList, 3))->resType.type)) { !IS_INTEGER_TYPE(getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 3))->type)) {
return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName);
} }
@ -1186,15 +1198,15 @@ static int32_t translateHistogramImpl(SFunctionNode* pFunc, char* pErrBuf, int32
return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName);
} }
uint8_t colType = ((SExprNode*)nodesListGetNode(pFunc->pParameterList, 0))->resType.type; uint8_t colType = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 0))->type;
if (!IS_NUMERIC_TYPE(colType)) { if (!IS_NUMERIC_TYPE(colType)) {
return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName);
} }
// param1 ~ param3 // param1 ~ param3
if (((SExprNode*)nodesListGetNode(pFunc->pParameterList, 1))->resType.type != TSDB_DATA_TYPE_BINARY || if (getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 1))->type != TSDB_DATA_TYPE_BINARY ||
((SExprNode*)nodesListGetNode(pFunc->pParameterList, 2))->resType.type != TSDB_DATA_TYPE_BINARY || getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 2))->type != TSDB_DATA_TYPE_BINARY ||
!IS_INTEGER_TYPE(((SExprNode*)nodesListGetNode(pFunc->pParameterList, 3))->resType.type)) { !IS_INTEGER_TYPE(getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 3))->type)) {
return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName);
} }
@ -1240,7 +1252,7 @@ static int32_t translateHistogramImpl(SFunctionNode* pFunc, char* pErrBuf, int32
return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName);
} }
if (((SExprNode*)nodesListGetNode(pFunc->pParameterList, 0))->resType.type != TSDB_DATA_TYPE_BINARY) { if (getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 0))->type != TSDB_DATA_TYPE_BINARY) {
return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName);
} }
@ -1305,7 +1317,7 @@ static int32_t translateStateCount(SFunctionNode* pFunc, char* pErrBuf, int32_t
return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName);
} }
uint8_t colType = ((SExprNode*)nodesListGetNode(pFunc->pParameterList, 0))->resType.type; uint8_t colType = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 0))->type;
if (!IS_NUMERIC_TYPE(colType)) { if (!IS_NUMERIC_TYPE(colType)) {
return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName);
} }
@ -1328,9 +1340,9 @@ static int32_t translateStateCount(SFunctionNode* pFunc, char* pErrBuf, int32_t
pValue->notReserved = true; pValue->notReserved = true;
} }
if (((SExprNode*)nodesListGetNode(pFunc->pParameterList, 1))->resType.type != TSDB_DATA_TYPE_BINARY || if (getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 1))->type != TSDB_DATA_TYPE_BINARY ||
(((SExprNode*)nodesListGetNode(pFunc->pParameterList, 2))->resType.type != TSDB_DATA_TYPE_BIGINT && (getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 2))->type != TSDB_DATA_TYPE_BIGINT &&
((SExprNode*)nodesListGetNode(pFunc->pParameterList, 2))->resType.type != TSDB_DATA_TYPE_DOUBLE)) { getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 2))->type != TSDB_DATA_TYPE_DOUBLE)) {
return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName);
} }
@ -1345,7 +1357,7 @@ static int32_t translateStateDuration(SFunctionNode* pFunc, char* pErrBuf, int32
return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName);
} }
uint8_t colType = ((SExprNode*)nodesListGetNode(pFunc->pParameterList, 0))->resType.type; uint8_t colType = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 0))->type;
if (!IS_NUMERIC_TYPE(colType)) { if (!IS_NUMERIC_TYPE(colType)) {
return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName);
} }
@ -1371,14 +1383,14 @@ static int32_t translateStateDuration(SFunctionNode* pFunc, char* pErrBuf, int32
pValue->notReserved = true; pValue->notReserved = true;
} }
if (((SExprNode*)nodesListGetNode(pFunc->pParameterList, 1))->resType.type != TSDB_DATA_TYPE_BINARY || if (getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 1))->type != TSDB_DATA_TYPE_BINARY ||
(((SExprNode*)nodesListGetNode(pFunc->pParameterList, 2))->resType.type != TSDB_DATA_TYPE_BIGINT && (getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 2))->type != TSDB_DATA_TYPE_BIGINT &&
((SExprNode*)nodesListGetNode(pFunc->pParameterList, 2))->resType.type != TSDB_DATA_TYPE_DOUBLE)) { getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 2))->type != TSDB_DATA_TYPE_DOUBLE)) {
return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName);
} }
if (numOfParams == 4 && if (numOfParams == 4 &&
((SExprNode*)nodesListGetNode(pFunc->pParameterList, 3))->resType.type != TSDB_DATA_TYPE_BIGINT) { getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 3))->type != TSDB_DATA_TYPE_BIGINT) {
return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName);
} }
@ -1406,7 +1418,7 @@ static int32_t translateCsum(SFunctionNode* pFunc, char* pErrBuf, int32_t len) {
return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName);
} }
uint8_t colType = ((SExprNode*)nodesListGetNode(pFunc->pParameterList, 0))->resType.type; uint8_t colType = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 0))->type;
uint8_t resType; uint8_t resType;
if (!IS_NUMERIC_TYPE(colType)) { if (!IS_NUMERIC_TYPE(colType)) {
return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName);
@ -1433,8 +1445,7 @@ static int32_t translateMavg(SFunctionNode* pFunc, char* pErrBuf, int32_t len) {
return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName);
} }
uint8_t colType = ((SExprNode*)nodesListGetNode(pFunc->pParameterList, 0))->resType.type; uint8_t colType = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 0))->type;
// param1 // param1
SNode* pParamNode1 = nodesListGetNode(pFunc->pParameterList, 1); SNode* pParamNode1 = nodesListGetNode(pFunc->pParameterList, 1);
if (QUERY_NODE_VALUE != nodeType(pParamNode1)) { if (QUERY_NODE_VALUE != nodeType(pParamNode1)) {
@ -1448,7 +1459,7 @@ static int32_t translateMavg(SFunctionNode* pFunc, char* pErrBuf, int32_t len) {
pValue->notReserved = true; pValue->notReserved = true;
uint8_t paraType = ((SExprNode*)nodesListGetNode(pFunc->pParameterList, 1))->resType.type; uint8_t paraType = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 1))->type;
if (!IS_NUMERIC_TYPE(colType) || !IS_INTEGER_TYPE(paraType)) { if (!IS_NUMERIC_TYPE(colType) || !IS_INTEGER_TYPE(paraType)) {
return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName);
} }
@ -1462,8 +1473,8 @@ static int32_t translateSample(SFunctionNode* pFunc, char* pErrBuf, int32_t len)
return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName);
} }
SExprNode* pCol = (SExprNode*)nodesListGetNode(pFunc->pParameterList, 0); SDataType* pSDataType = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 0));
uint8_t colType = pCol->resType.type; uint8_t colType = pSDataType->type;
// param1 // param1
SNode* pParamNode1 = nodesListGetNode(pFunc->pParameterList, 1); SNode* pParamNode1 = nodesListGetNode(pFunc->pParameterList, 1);
@ -1478,14 +1489,14 @@ static int32_t translateSample(SFunctionNode* pFunc, char* pErrBuf, int32_t len)
pValue->notReserved = true; pValue->notReserved = true;
uint8_t paraType = ((SExprNode*)nodesListGetNode(pFunc->pParameterList, 1))->resType.type; uint8_t paraType = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 1))->type;
if (!IS_INTEGER_TYPE(paraType)) { if (!IS_INTEGER_TYPE(paraType)) {
return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName);
} }
// set result type // set result type
if (IS_STR_DATA_TYPE(colType)) { if (IS_STR_DATA_TYPE(colType)) {
pFunc->node.resType = (SDataType){.bytes = pCol->resType.bytes, .type = colType}; pFunc->node.resType = (SDataType){.bytes = pSDataType->bytes, .type = colType};
} else { } else {
pFunc->node.resType = (SDataType){.bytes = tDataTypes[colType].bytes, .type = colType}; pFunc->node.resType = (SDataType){.bytes = tDataTypes[colType].bytes, .type = colType};
} }
@ -1499,8 +1510,8 @@ static int32_t translateTail(SFunctionNode* pFunc, char* pErrBuf, int32_t len) {
return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName);
} }
SExprNode* pCol = (SExprNode*)nodesListGetNode(pFunc->pParameterList, 0); SDataType* pSDataType = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 0));
uint8_t colType = pCol->resType.type; uint8_t colType = pSDataType->type;
// param1 & param2 // param1 & param2
for (int32_t i = 1; i < numOfParams; ++i) { for (int32_t i = 1; i < numOfParams; ++i) {
@ -1520,7 +1531,7 @@ static int32_t translateTail(SFunctionNode* pFunc, char* pErrBuf, int32_t len) {
pValue->notReserved = true; pValue->notReserved = true;
uint8_t paraType = ((SExprNode*)nodesListGetNode(pFunc->pParameterList, i))->resType.type; uint8_t paraType = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, i))->type;
if (!IS_INTEGER_TYPE(paraType)) { if (!IS_INTEGER_TYPE(paraType)) {
return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName);
} }
@ -1528,7 +1539,7 @@ static int32_t translateTail(SFunctionNode* pFunc, char* pErrBuf, int32_t len) {
// set result type // set result type
if (IS_STR_DATA_TYPE(colType)) { if (IS_STR_DATA_TYPE(colType)) {
pFunc->node.resType = (SDataType){.bytes = pCol->resType.bytes, .type = colType}; pFunc->node.resType = (SDataType){.bytes = pSDataType->bytes, .type = colType};
} else { } else {
pFunc->node.resType = (SDataType){.bytes = tDataTypes[colType].bytes, .type = colType}; pFunc->node.resType = (SDataType){.bytes = tDataTypes[colType].bytes, .type = colType};
} }
@ -1540,7 +1551,7 @@ static int32_t translateDerivative(SFunctionNode* pFunc, char* pErrBuf, int32_t
return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName);
} }
uint8_t colType = ((SExprNode*)nodesListGetNode(pFunc->pParameterList, 0))->resType.type; uint8_t colType = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 0))->type;
// param1 // param1
SNode* pParamNode1 = nodesListGetNode(pFunc->pParameterList, 1); SNode* pParamNode1 = nodesListGetNode(pFunc->pParameterList, 1);
@ -1582,7 +1593,7 @@ static int32_t translateIrate(SFunctionNode* pFunc, char* pErrBuf, int32_t len)
return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName);
} }
uint8_t colType = ((SExprNode*)nodesListGetNode(pFunc->pParameterList, 0))->resType.type; uint8_t colType = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 0))->type;
if (!IS_NUMERIC_TYPE(colType)) { if (!IS_NUMERIC_TYPE(colType)) {
return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName);
@ -1600,7 +1611,7 @@ static int32_t translateIrate(SFunctionNode* pFunc, char* pErrBuf, int32_t len)
} }
static int32_t translateIrateImpl(SFunctionNode* pFunc, char* pErrBuf, int32_t len, bool isPartial) { static int32_t translateIrateImpl(SFunctionNode* pFunc, char* pErrBuf, int32_t len, bool isPartial) {
uint8_t colType = ((SExprNode*)nodesListGetNode(pFunc->pParameterList, 0))->resType.type; uint8_t colType = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 0))->type;
if (isPartial) { if (isPartial) {
if (3 != LIST_LENGTH(pFunc->pParameterList)) { if (3 != LIST_LENGTH(pFunc->pParameterList)) {
return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName);
@ -1647,14 +1658,14 @@ static int32_t translateInterp(SFunctionNode* pFunc, char* pErrBuf, int32_t len)
} }
uint8_t nodeType = nodeType(nodesListGetNode(pFunc->pParameterList, 0)); uint8_t nodeType = nodeType(nodesListGetNode(pFunc->pParameterList, 0));
uint8_t paraType = ((SExprNode*)nodesListGetNode(pFunc->pParameterList, 0))->resType.type; uint8_t paraType = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 0))->type;
if ((!IS_NUMERIC_TYPE(paraType) && !IS_BOOLEAN_TYPE(paraType)) || QUERY_NODE_VALUE == nodeType) { if ((!IS_NUMERIC_TYPE(paraType) && !IS_BOOLEAN_TYPE(paraType)) || QUERY_NODE_VALUE == nodeType) {
return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName);
} }
if (2 == numOfParams) { if (2 == numOfParams) {
nodeType = nodeType(nodesListGetNode(pFunc->pParameterList, 1)); nodeType = nodeType(nodesListGetNode(pFunc->pParameterList, 1));
paraType = ((SExprNode*)nodesListGetNode(pFunc->pParameterList, 1))->resType.type; paraType = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 1))->type;
if (!IS_INTEGER_TYPE(paraType) || QUERY_NODE_VALUE != nodeType) { if (!IS_INTEGER_TYPE(paraType) || QUERY_NODE_VALUE != nodeType) {
return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName);
} }
@ -1728,26 +1739,26 @@ static int32_t translateFirstLast(SFunctionNode* pFunc, char* pErrBuf, int32_t l
for (int32_t i = 0; i < numOfParams; ++i) { for (int32_t i = 0; i < numOfParams; ++i) {
uint8_t nodeType = nodeType(nodesListGetNode(pFunc->pParameterList, i)); uint8_t nodeType = nodeType(nodesListGetNode(pFunc->pParameterList, i));
uint8_t paraType = ((SExprNode*)nodesListGetNode(pFunc->pParameterList, i))->resType.type; uint8_t paraType = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, i))->type;
if (IS_NULL_TYPE(paraType) && QUERY_NODE_VALUE == nodeType) { if (IS_NULL_TYPE(paraType) && QUERY_NODE_VALUE == nodeType) {
return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName);
} }
} }
pFunc->node.resType = ((SExprNode*)nodesListGetNode(pFunc->pParameterList, 0))->resType; pFunc->node.resType = *getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 0));
return TSDB_CODE_SUCCESS; return TSDB_CODE_SUCCESS;
} }
static int32_t translateFirstLastImpl(SFunctionNode* pFunc, char* pErrBuf, int32_t len, bool isPartial) { static int32_t translateFirstLastImpl(SFunctionNode* pFunc, char* pErrBuf, int32_t len, bool isPartial) {
// first(col_list) will be rewritten as first(col) // first(col_list) will be rewritten as first(col)
SNode* pPara = nodesListGetNode(pFunc->pParameterList, 0); SNode* pPara = nodesListGetNode(pFunc->pParameterList, 0);
uint8_t paraType = ((SExprNode*)pPara)->resType.type; uint8_t paraType = getSDataTypeFromNode(pPara)->type;
int32_t paraBytes = ((SExprNode*)pPara)->resType.bytes; int32_t paraBytes = getSDataTypeFromNode(pPara)->bytes;
if (isPartial) { if (isPartial) {
int32_t numOfParams = LIST_LENGTH(pFunc->pParameterList); int32_t numOfParams = LIST_LENGTH(pFunc->pParameterList);
for (int32_t i = 0; i < numOfParams; ++i) { for (int32_t i = 0; i < numOfParams; ++i) {
uint8_t nodeType = nodeType(nodesListGetNode(pFunc->pParameterList, i)); uint8_t nodeType = nodeType(nodesListGetNode(pFunc->pParameterList, i));
uint8_t pType = ((SExprNode*)nodesListGetNode(pFunc->pParameterList, i))->resType.type; uint8_t pType = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, i))->type;
if (IS_NULL_TYPE(pType) && QUERY_NODE_VALUE == nodeType) { if (IS_NULL_TYPE(pType) && QUERY_NODE_VALUE == nodeType) {
return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName);
} }
@ -1802,7 +1813,7 @@ static int32_t translateDiff(SFunctionNode* pFunc, char* pErrBuf, int32_t len) {
return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName);
} }
uint8_t colType = ((SExprNode*)nodesListGetNode(pFunc->pParameterList, 0))->resType.type; uint8_t colType = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 0))->type;
if (!IS_INTEGER_TYPE(colType) && !IS_FLOAT_TYPE(colType) && TSDB_DATA_TYPE_BOOL != colType && if (!IS_INTEGER_TYPE(colType) && !IS_FLOAT_TYPE(colType) && TSDB_DATA_TYPE_BOOL != colType &&
!IS_TIMESTAMP_TYPE(colType)) { !IS_TIMESTAMP_TYPE(colType)) {
return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName);
@ -1810,7 +1821,7 @@ static int32_t translateDiff(SFunctionNode* pFunc, char* pErrBuf, int32_t len) {
// param1 // param1
if (numOfParams == 2) { if (numOfParams == 2) {
uint8_t paraType = ((SExprNode*)nodesListGetNode(pFunc->pParameterList, 1))->resType.type; uint8_t paraType = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 1))->type;
if (!IS_INTEGER_TYPE(paraType)) { if (!IS_INTEGER_TYPE(paraType)) {
return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName);
} }
@ -1854,7 +1865,7 @@ static int32_t translateLength(SFunctionNode* pFunc, char* pErrBuf, int32_t len)
return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName);
} }
if (!IS_STR_DATA_TYPE(((SExprNode*)nodesListGetNode(pFunc->pParameterList, 0))->resType.type)) { if (!IS_STR_DATA_TYPE(getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 0))->type)) {
return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName);
} }
@ -1885,7 +1896,7 @@ static int32_t translateConcatImpl(SFunctionNode* pFunc, char* pErrBuf, int32_t
/* For concat/concat_ws function, if params have NCHAR type, promote the final result to NCHAR */ /* For concat/concat_ws function, if params have NCHAR type, promote the final result to NCHAR */
for (int32_t i = 0; i < numOfParams; ++i) { for (int32_t i = 0; i < numOfParams; ++i) {
SNode* pPara = nodesListGetNode(pFunc->pParameterList, i); SNode* pPara = nodesListGetNode(pFunc->pParameterList, i);
uint8_t paraType = ((SExprNode*)pPara)->resType.type; uint8_t paraType = getSDataTypeFromNode(pPara)->type;
if (TSDB_DATA_TYPE_VARBINARY == paraType) { if (TSDB_DATA_TYPE_VARBINARY == paraType) {
return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName);
} }
@ -1900,8 +1911,8 @@ static int32_t translateConcatImpl(SFunctionNode* pFunc, char* pErrBuf, int32_t
for (int32_t i = 0; i < numOfParams; ++i) { for (int32_t i = 0; i < numOfParams; ++i) {
SNode* pPara = nodesListGetNode(pFunc->pParameterList, i); SNode* pPara = nodesListGetNode(pFunc->pParameterList, i);
uint8_t paraType = ((SExprNode*)pPara)->resType.type; uint8_t paraType = getSDataTypeFromNode(pPara)->type;
int32_t paraBytes = ((SExprNode*)pPara)->resType.bytes; int32_t paraBytes = getSDataTypeFromNode(pPara)->bytes;
int32_t factor = 1; int32_t factor = 1;
if (IS_NULL_TYPE(paraType)) { if (IS_NULL_TYPE(paraType)) {
resultType = TSDB_DATA_TYPE_VARCHAR; resultType = TSDB_DATA_TYPE_VARCHAR;
@ -2009,7 +2020,7 @@ static int32_t translateToIso8601(SFunctionNode* pFunc, char* pErrBuf, int32_t l
} }
// param0 // param0
uint8_t paraType = ((SExprNode*)nodesListGetNode(pFunc->pParameterList, 0))->resType.type; uint8_t paraType = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 0))->type;
if (!IS_INTEGER_TYPE(paraType) && !IS_TIMESTAMP_TYPE(paraType)) { if (!IS_INTEGER_TYPE(paraType) && !IS_TIMESTAMP_TYPE(paraType)) {
return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName);
} }
@ -2050,13 +2061,13 @@ static int32_t translateToUnixtimestamp(SFunctionNode* pFunc, char* pErrBuf, int
return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName);
} }
uint8_t para1Type = ((SExprNode*)nodesListGetNode(pFunc->pParameterList, 0))->resType.type; uint8_t para1Type = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 0))->type;
if (para1Type == TSDB_DATA_TYPE_VARBINARY || !IS_STR_DATA_TYPE(para1Type)) { if (para1Type == TSDB_DATA_TYPE_VARBINARY || !IS_STR_DATA_TYPE(para1Type)) {
return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName);
} }
if (2 == numOfParams) { if (2 == numOfParams) {
uint8_t para2Type = ((SExprNode*)nodesListGetNode(pFunc->pParameterList, 1))->resType.type; uint8_t para2Type = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 1))->type;
if (!IS_INTEGER_TYPE(para2Type)) { if (!IS_INTEGER_TYPE(para2Type)) {
return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName);
} }
@ -2087,8 +2098,8 @@ static int32_t translateToTimestamp(SFunctionNode* pFunc, char* pErrBuf, int32_t
if (LIST_LENGTH(pFunc->pParameterList) != 2) { if (LIST_LENGTH(pFunc->pParameterList) != 2) {
return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName);
} }
uint8_t para1Type = ((SExprNode*)nodesListGetNode(pFunc->pParameterList, 0))->resType.type; uint8_t para1Type = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 0))->type;
uint8_t para2Type = ((SExprNode*)nodesListGetNode(pFunc->pParameterList, 1))->resType.type; uint8_t para2Type = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 1))->type;
if (!IS_STR_DATA_TYPE(para1Type) || !IS_STR_DATA_TYPE(para2Type)) { if (!IS_STR_DATA_TYPE(para1Type) || !IS_STR_DATA_TYPE(para2Type)) {
return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName);
} }
@ -2101,8 +2112,8 @@ static int32_t translateToChar(SFunctionNode* pFunc, char* pErrBuf, int32_t len)
if (LIST_LENGTH(pFunc->pParameterList) != 2) { if (LIST_LENGTH(pFunc->pParameterList) != 2) {
return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName);
} }
uint8_t para1Type = ((SExprNode*)nodesListGetNode(pFunc->pParameterList, 0))->resType.type; uint8_t para1Type = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 0))->type;
uint8_t para2Type = ((SExprNode*)nodesListGetNode(pFunc->pParameterList, 1))->resType.type; uint8_t para2Type = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 1))->type;
// currently only support to_char(timestamp, str) // currently only support to_char(timestamp, str)
if (!IS_STR_DATA_TYPE(para2Type) || !IS_TIMESTAMP_TYPE(para1Type)) { if (!IS_STR_DATA_TYPE(para2Type) || !IS_TIMESTAMP_TYPE(para1Type)) {
return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName);
@ -2117,8 +2128,8 @@ static int32_t translateTimeTruncate(SFunctionNode* pFunc, char* pErrBuf, int32_
return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName);
} }
uint8_t para1Type = ((SExprNode*)nodesListGetNode(pFunc->pParameterList, 0))->resType.type; uint8_t para1Type = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 0))->type;
uint8_t para2Type = ((SExprNode*)nodesListGetNode(pFunc->pParameterList, 1))->resType.type; uint8_t para2Type = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 1))->type;
if ((!IS_STR_DATA_TYPE(para1Type) && !IS_INTEGER_TYPE(para1Type) && !IS_TIMESTAMP_TYPE(para1Type)) || if ((!IS_STR_DATA_TYPE(para1Type) && !IS_INTEGER_TYPE(para1Type) && !IS_TIMESTAMP_TYPE(para1Type)) ||
!IS_INTEGER_TYPE(para2Type)) { !IS_INTEGER_TYPE(para2Type)) {
return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName);
@ -2136,7 +2147,7 @@ static int32_t translateTimeTruncate(SFunctionNode* pFunc, char* pErrBuf, int32_
} }
if (3 == numOfParams) { if (3 == numOfParams) {
uint8_t para3Type = ((SExprNode*)nodesListGetNode(pFunc->pParameterList, 2))->resType.type; uint8_t para3Type = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 2))->type;
if (!IS_INTEGER_TYPE(para3Type)) { if (!IS_INTEGER_TYPE(para3Type)) {
return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName);
} }
@ -2171,14 +2182,14 @@ static int32_t translateTimeDiff(SFunctionNode* pFunc, char* pErrBuf, int32_t le
} }
for (int32_t i = 0; i < 2; ++i) { for (int32_t i = 0; i < 2; ++i) {
uint8_t paraType = ((SExprNode*)nodesListGetNode(pFunc->pParameterList, i))->resType.type; uint8_t paraType = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, i))->type;
if (!IS_STR_DATA_TYPE(paraType) && !IS_INTEGER_TYPE(paraType) && !IS_TIMESTAMP_TYPE(paraType)) { if (!IS_STR_DATA_TYPE(paraType) && !IS_INTEGER_TYPE(paraType) && !IS_TIMESTAMP_TYPE(paraType)) {
return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName);
} }
} }
if (3 == numOfParams) { if (3 == numOfParams) {
if (!IS_INTEGER_TYPE(((SExprNode*)nodesListGetNode(pFunc->pParameterList, 2))->resType.type)) { if (!IS_INTEGER_TYPE(getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 2))->type)) {
return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName);
} }
} }
@ -2226,7 +2237,7 @@ static int32_t translateInStrOutGeom(SFunctionNode* pFunc, char* pErrBuf, int32_
return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName);
} }
uint8_t para1Type = ((SExprNode*)nodesListGetNode(pFunc->pParameterList, 0))->resType.type; uint8_t para1Type = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 0))->type;
if (!IS_STR_DATA_TYPE(para1Type) && !IS_NULL_TYPE(para1Type)) { if (!IS_STR_DATA_TYPE(para1Type) && !IS_NULL_TYPE(para1Type)) {
return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName);
} }
@ -2241,7 +2252,7 @@ static int32_t translateInGeomOutStr(SFunctionNode* pFunc, char* pErrBuf, int32_
return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName);
} }
uint8_t para1Type = ((SExprNode*)nodesListGetNode(pFunc->pParameterList, 0))->resType.type; uint8_t para1Type = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 0))->type;
if (para1Type != TSDB_DATA_TYPE_GEOMETRY && !IS_NULL_TYPE(para1Type)) { if (para1Type != TSDB_DATA_TYPE_GEOMETRY && !IS_NULL_TYPE(para1Type)) {
return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName);
} }
@ -2256,8 +2267,8 @@ static int32_t translateIn2NumOutGeom(SFunctionNode* pFunc, char* pErrBuf, int32
return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName);
} }
uint8_t para1Type = ((SExprNode*)nodesListGetNode(pFunc->pParameterList, 0))->resType.type; uint8_t para1Type = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 0))->type;
uint8_t para2Type = ((SExprNode*)nodesListGetNode(pFunc->pParameterList, 1))->resType.type; uint8_t para2Type = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 1))->type;
if ((!IS_NUMERIC_TYPE(para1Type) && !IS_NULL_TYPE(para1Type)) || if ((!IS_NUMERIC_TYPE(para1Type) && !IS_NULL_TYPE(para1Type)) ||
(!IS_NUMERIC_TYPE(para2Type) && !IS_NULL_TYPE(para2Type))) { (!IS_NUMERIC_TYPE(para2Type) && !IS_NULL_TYPE(para2Type))) {
return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName);
@ -2273,8 +2284,8 @@ static int32_t translateIn2GeomOutBool(SFunctionNode* pFunc, char* pErrBuf, int3
return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName);
} }
uint8_t para1Type = ((SExprNode*)nodesListGetNode(pFunc->pParameterList, 0))->resType.type; uint8_t para1Type = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 0))->type;
uint8_t para2Type = ((SExprNode*)nodesListGetNode(pFunc->pParameterList, 1))->resType.type; uint8_t para2Type = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 1))->type;
if ((para1Type != TSDB_DATA_TYPE_GEOMETRY && !IS_NULL_TYPE(para1Type)) || if ((para1Type != TSDB_DATA_TYPE_GEOMETRY && !IS_NULL_TYPE(para1Type)) ||
(para2Type != TSDB_DATA_TYPE_GEOMETRY && !IS_NULL_TYPE(para2Type))) { (para2Type != TSDB_DATA_TYPE_GEOMETRY && !IS_NULL_TYPE(para2Type))) {
return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName); return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName);

View File

@ -832,6 +832,7 @@ int32_t minmaxFunctionFinalize(SqlFunctionCtx* pCtx, SSDataBlock* pBlock) {
return code; return code;
} }
#ifdef BUILD_NO_CALL
int32_t setNullSelectivityValue(SqlFunctionCtx* pCtx, SSDataBlock* pBlock, int32_t rowIndex) { int32_t setNullSelectivityValue(SqlFunctionCtx* pCtx, SSDataBlock* pBlock, int32_t rowIndex) {
if (pCtx->subsidiaries.num <= 0) { if (pCtx->subsidiaries.num <= 0) {
return TSDB_CODE_SUCCESS; return TSDB_CODE_SUCCESS;
@ -847,6 +848,7 @@ int32_t setNullSelectivityValue(SqlFunctionCtx* pCtx, SSDataBlock* pBlock, int32
return TSDB_CODE_SUCCESS; return TSDB_CODE_SUCCESS;
} }
#endif
int32_t setSelectivityValue(SqlFunctionCtx* pCtx, SSDataBlock* pBlock, const STuplePos* pTuplePos, int32_t rowIndex) { int32_t setSelectivityValue(SqlFunctionCtx* pCtx, SSDataBlock* pBlock, const STuplePos* pTuplePos, int32_t rowIndex) {
if (pCtx->subsidiaries.num <= 0) { if (pCtx->subsidiaries.num <= 0) {
@ -2125,7 +2127,7 @@ bool getGroupKeyFuncEnv(SFunctionNode* pFunc, SFuncExecEnv* pEnv) {
} }
static FORCE_INLINE TSKEY getRowPTs(SColumnInfoData* pTsColInfo, int32_t rowIndex) { static FORCE_INLINE TSKEY getRowPTs(SColumnInfoData* pTsColInfo, int32_t rowIndex) {
if (pTsColInfo == NULL) { if (pTsColInfo == NULL || pTsColInfo->pData == NULL) {
return 0; return 0;
} }

View File

@ -241,6 +241,29 @@ static SVgroupsInfo* vgroupsInfoClone(const SVgroupsInfo* pSrc) {
return pDst; return pDst;
} }
static SArray* functParamClone(const SArray* pSrc) {
int32_t len = sizeof(SArray) + pSrc->capacity * pSrc->elemSize;
SArray* pDst = taosArrayInit(pSrc->capacity, pSrc->elemSize);
if (NULL == pDst) {
return NULL;
}
for (int i = 0; i < TARRAY_SIZE(pSrc); ++i) {
SFunctParam* pFunctParam = taosArrayGet(pSrc, i);
SFunctParam* pNewFunctParam = (SFunctParam*)taosArrayPush(pDst, pFunctParam);
if (NULL == pNewFunctParam) {
return NULL;
}
pNewFunctParam->type = pFunctParam->type;
pNewFunctParam->pCol = taosMemoryCalloc(1, sizeof(SColumn));
memcpy(pNewFunctParam->pCol, pFunctParam->pCol, sizeof(SColumn));
}
return pDst;
}
static int32_t realTableNodeCopy(const SRealTableNode* pSrc, SRealTableNode* pDst) { static int32_t realTableNodeCopy(const SRealTableNode* pSrc, SRealTableNode* pDst) {
COPY_BASE_OBJECT_FIELD(table, tableNodeCopy); COPY_BASE_OBJECT_FIELD(table, tableNodeCopy);
CLONE_OBJECT_FIELD(pMeta, tableMetaClone); CLONE_OBJECT_FIELD(pMeta, tableMetaClone);
@ -425,6 +448,7 @@ static int32_t logicScanCopy(const SScanLogicNode* pSrc, SScanLogicNode* pDst) {
COPY_SCALAR_FIELD(onlyMetaCtbIdx); COPY_SCALAR_FIELD(onlyMetaCtbIdx);
COPY_SCALAR_FIELD(filesetDelimited); COPY_SCALAR_FIELD(filesetDelimited);
COPY_SCALAR_FIELD(isCountByTag); COPY_SCALAR_FIELD(isCountByTag);
CLONE_OBJECT_FIELD(pFuncTypes, functParamClone);
return TSDB_CODE_SUCCESS; return TSDB_CODE_SUCCESS;
} }

View File

@ -1784,6 +1784,24 @@ static int32_t jsonToPhysiTagScanNode(const SJson* pJson, void* pObj) {
static const char* jkLastRowScanPhysiPlanGroupTags = "GroupTags"; static const char* jkLastRowScanPhysiPlanGroupTags = "GroupTags";
static const char* jkLastRowScanPhysiPlanGroupSort = "GroupSort"; static const char* jkLastRowScanPhysiPlanGroupSort = "GroupSort";
static const char* jkLastRowScanPhysiPlanTargets = "Targets"; static const char* jkLastRowScanPhysiPlanTargets = "Targets";
static const char* jkLastRowScanPhysiPlanFuncType = "FuncType";
static const char* jkLastRowScanPhysiPlanFuncTypes = "FuncTypes";
static int32_t funcTypeToJson(const void* pObj, SJson* pJson) {
const int32_t* pNode = (const int32_t*)pObj;
int32_t code = tjsonAddIntegerToObject(pJson, jkLastRowScanPhysiPlanFuncType, *pNode);
return code;
}
static int32_t jsonToFuncType(const SJson* pJson, void* pObj) {
int32_t* pNode = (int32_t*)pObj;
int32_t code = tjsonGetIntValue(pJson, jkLastRowScanPhysiPlanFuncType, pNode);
return code;
}
static int32_t physiLastRowScanNodeToJson(const void* pObj, SJson* pJson) { static int32_t physiLastRowScanNodeToJson(const void* pObj, SJson* pJson) {
const SLastRowScanPhysiNode* pNode = (const SLastRowScanPhysiNode*)pObj; const SLastRowScanPhysiNode* pNode = (const SLastRowScanPhysiNode*)pObj;
@ -1798,6 +1816,9 @@ static int32_t physiLastRowScanNodeToJson(const void* pObj, SJson* pJson) {
if (TSDB_CODE_SUCCESS == code) { if (TSDB_CODE_SUCCESS == code) {
code = nodeListToJson(pJson, jkLastRowScanPhysiPlanTargets, pNode->pTargets); code = nodeListToJson(pJson, jkLastRowScanPhysiPlanTargets, pNode->pTargets);
} }
if (TSDB_CODE_SUCCESS == code) {
code = tjsonAddTArray(pJson, jkLastRowScanPhysiPlanFuncTypes, funcTypeToJson, pNode->pFuncTypes);
}
return code; return code;
} }
@ -1815,6 +1836,9 @@ static int32_t jsonToPhysiLastRowScanNode(const SJson* pJson, void* pObj) {
if (TSDB_CODE_SUCCESS == code) { if (TSDB_CODE_SUCCESS == code) {
code = jsonToNodeList(pJson, jkLastRowScanPhysiPlanTargets, &pNode->pTargets); code = jsonToNodeList(pJson, jkLastRowScanPhysiPlanTargets, &pNode->pTargets);
} }
if (TSDB_CODE_SUCCESS == code) {
code = tjsonToTArray(pJson, jkLastRowScanPhysiPlanFuncTypes, jsonToFuncType, &pNode->pFuncTypes, sizeof(int32_t));
}
return code; return code;
} }

View File

@ -194,3 +194,21 @@ bool nodesEqualNode(const SNode* a, const SNode* b) {
return false; return false;
} }
bool nodeListNodeEqual(const SNodeList* a, const SNode* b) {
if (NULL == a || NULL == b) {
return false;
}
if (LIST_LENGTH(a) < 1) {
return false;
}
SNode *na;
FOREACH(na, a) {
if (nodesEqualNode(na, b)) {
return true;
}
}
return false;
}

View File

@ -65,10 +65,14 @@ typedef int32_t (*FSetObject)(STlv* pTlv, void* pObj);
static int32_t nodeToMsg(const void* pObj, STlvEncoder* pEncoder); static int32_t nodeToMsg(const void* pObj, STlvEncoder* pEncoder);
static int32_t nodeListToMsg(const void* pObj, STlvEncoder* pEncoder); static int32_t nodeListToMsg(const void* pObj, STlvEncoder* pEncoder);
static int32_t SArrayToMsg(const void* pObj, STlvEncoder* pEncoder);
static int32_t msgToNode(STlvDecoder* pDecoder, void** pObj); static int32_t msgToNode(STlvDecoder* pDecoder, void** pObj);
static int32_t msgToNodeFromTlv(STlv* pTlv, void** pObj); static int32_t msgToNodeFromTlv(STlv* pTlv, void** pObj);
static int32_t msgToNodeList(STlvDecoder* pDecoder, void** pObj); static int32_t msgToNodeList(STlvDecoder* pDecoder, void** pObj);
static int32_t msgToNodeListFromTlv(STlv* pTlv, void** pObj); static int32_t msgToNodeListFromTlv(STlv* pTlv, void** pObj);
static int32_t msgToSArray(STlv* pTlv, void** pObj);
static int32_t initTlvEncoder(STlvEncoder* pEncoder) { static int32_t initTlvEncoder(STlvEncoder* pEncoder) {
pEncoder->allocSize = NODES_MSG_DEFAULT_LEN; pEncoder->allocSize = NODES_MSG_DEFAULT_LEN;
@ -2053,7 +2057,8 @@ enum {
PHY_LAST_ROW_SCAN_CODE_GROUP_TAGS, PHY_LAST_ROW_SCAN_CODE_GROUP_TAGS,
PHY_LAST_ROW_SCAN_CODE_GROUP_SORT, PHY_LAST_ROW_SCAN_CODE_GROUP_SORT,
PHY_LAST_ROW_SCAN_CODE_IGNULL, PHY_LAST_ROW_SCAN_CODE_IGNULL,
PHY_LAST_ROW_SCAN_CODE_TARGETS PHY_LAST_ROW_SCAN_CODE_TARGETS,
PHY_LAST_ROW_SCAN_CODE_FUNCTYPES
}; };
static int32_t physiLastRowScanNodeToMsg(const void* pObj, STlvEncoder* pEncoder) { static int32_t physiLastRowScanNodeToMsg(const void* pObj, STlvEncoder* pEncoder) {
@ -2072,6 +2077,9 @@ static int32_t physiLastRowScanNodeToMsg(const void* pObj, STlvEncoder* pEncoder
if (TSDB_CODE_SUCCESS == code) { if (TSDB_CODE_SUCCESS == code) {
code = tlvEncodeObj(pEncoder, PHY_LAST_ROW_SCAN_CODE_TARGETS, nodeListToMsg, pNode->pTargets); code = tlvEncodeObj(pEncoder, PHY_LAST_ROW_SCAN_CODE_TARGETS, nodeListToMsg, pNode->pTargets);
} }
if (TSDB_CODE_SUCCESS == code) {
code = tlvEncodeObj(pEncoder, PHY_LAST_ROW_SCAN_CODE_FUNCTYPES, SArrayToMsg, pNode->pFuncTypes);
}
return code; return code;
} }
@ -2098,6 +2106,10 @@ static int32_t msgToPhysiLastRowScanNode(STlvDecoder* pDecoder, void* pObj) {
case PHY_LAST_ROW_SCAN_CODE_TARGETS: case PHY_LAST_ROW_SCAN_CODE_TARGETS:
code = msgToNodeListFromTlv(pTlv, (void**)&pNode->pTargets); code = msgToNodeListFromTlv(pTlv, (void**)&pNode->pTargets);
break; break;
case PHY_LAST_ROW_SCAN_CODE_FUNCTYPES:
code = msgToSArray(pTlv, (void**)&pNode->pFuncTypes);
break;
default: default:
break; break;
} }
@ -4391,6 +4403,31 @@ static int32_t nodeListToMsg(const void* pObj, STlvEncoder* pEncoder) {
return TSDB_CODE_SUCCESS; return TSDB_CODE_SUCCESS;
} }
enum {
SARRAY_CODE_CAPACITY = 1,
SARRAY_CODE_ELEMSIZE,
SARRAY_CODE_SIZE,
SARRAY_CODE_PDATA
};
static int32_t SArrayToMsg(const void* pObj, STlvEncoder* pEncoder) {
const SArray* pArray = (const SArray*)pObj;
int32_t code = TSDB_CODE_SUCCESS;
if (TSDB_CODE_SUCCESS == code) {
code = tlvEncodeI32(pEncoder, SARRAY_CODE_CAPACITY, pArray->capacity);
}
if (TSDB_CODE_SUCCESS == code) {
code = tlvEncodeI32(pEncoder, SARRAY_CODE_ELEMSIZE, pArray->elemSize);
}
if (TSDB_CODE_SUCCESS == code) {
code = tlvEncodeI32(pEncoder, SARRAY_CODE_SIZE, pArray->size);
}
if (TSDB_CODE_SUCCESS == code && pArray->capacity * pArray->elemSize > 0 && pArray->pData != NULL) {
code = tlvEncodeBinary(pEncoder, SARRAY_CODE_PDATA, pArray->pData, pArray->capacity * pArray->elemSize);
}
return code;
}
static int32_t msgToNodeList(STlvDecoder* pDecoder, void** pObj) { static int32_t msgToNodeList(STlvDecoder* pDecoder, void** pObj) {
SNodeList* pList = nodesMakeList(); SNodeList* pList = nodesMakeList();
@ -4411,6 +4448,67 @@ static int32_t msgToNodeList(STlvDecoder* pDecoder, void** pObj) {
return code; return code;
} }
static int32_t msgToSArray(STlv* pTlv, void** pObj){
SArray* pArray = NULL;
uint32_t capacity = 0;
uint32_t elemSize = 0;
uint32_t actualSize;
int32_t decodeFieldNum = 0;;
int32_t code = TSDB_CODE_SUCCESS;
STlvDecoder decoder = {.bufSize = pTlv->len, .offset = 0, .pBuf = pTlv->value};
STlv* pTlvTemp = NULL;
STlv* pDataTlv = NULL;
tlvForEach(&decoder, pTlvTemp, code) {
switch (pTlvTemp->type) {
case SARRAY_CODE_CAPACITY:
code = tlvDecodeI32(pTlvTemp, &capacity);
break;
case SARRAY_CODE_ELEMSIZE:
code = tlvDecodeI32(pTlvTemp, &elemSize);
break;
case SARRAY_CODE_SIZE:
code = tlvDecodeI32(pTlvTemp, &actualSize);
break;
case SARRAY_CODE_PDATA:
if (decodeFieldNum < 3) {
pDataTlv = pTlvTemp;
break;
}
pArray = taosArrayInit(capacity, elemSize);
if (NULL == pArray) {
return TSDB_CODE_OUT_OF_MEMORY;
}
pArray->size = actualSize;
if (TSDB_CODE_SUCCESS != code || pTlvTemp == NULL) {
taosArrayDestroy(pArray);
return TSDB_CODE_OUT_OF_MEMORY;
}
code = tlvDecodeBinary(pTlvTemp, pArray->pData);
break;
default:
break;
}
decodeFieldNum++;
}
if (pDataTlv != NULL) {
pArray = taosArrayInit(capacity, elemSize);
if (NULL == pArray) {
return TSDB_CODE_OUT_OF_MEMORY;
}
pArray->size = actualSize;
if (TSDB_CODE_SUCCESS != code || pTlvTemp == NULL) {
taosArrayDestroy(pArray);
return TSDB_CODE_OUT_OF_MEMORY;
}
code = tlvDecodeBinary(pDataTlv, pArray->pData);
}
*pObj = pArray;
return code;
}
static int32_t msgToNodeListFromTlv(STlv* pTlv, void** pObj) { static int32_t msgToNodeListFromTlv(STlv* pTlv, void** pObj) {
STlvDecoder decoder = {.bufSize = pTlv->len, .offset = 0, .pBuf = pTlv->value}; STlvDecoder decoder = {.bufSize = pTlv->len, .offset = 0, .pBuf = pTlv->value};
return msgToNodeList(&decoder, pObj); return msgToNodeList(&decoder, pObj);

View File

@ -295,7 +295,7 @@ SNode* nodesMakeNode(ENodeType type) {
case QUERY_NODE_LEFT_VALUE: case QUERY_NODE_LEFT_VALUE:
return makeNode(type, sizeof(SLeftValueNode)); return makeNode(type, sizeof(SLeftValueNode));
case QUERY_NODE_COLUMN_REF: case QUERY_NODE_COLUMN_REF:
return makeNode(type, sizeof(SColumnDefNode)); return makeNode(type, sizeof(SColumnRefNode));
case QUERY_NODE_WHEN_THEN: case QUERY_NODE_WHEN_THEN:
return makeNode(type, sizeof(SWhenThenNode)); return makeNode(type, sizeof(SWhenThenNode));
case QUERY_NODE_CASE_WHEN: case QUERY_NODE_CASE_WHEN:
@ -674,6 +674,8 @@ static void destroyTableCfg(STableCfg* pCfg) {
static void destroySmaIndex(void* pIndex) { taosMemoryFree(((STableIndexInfo*)pIndex)->expr); } static void destroySmaIndex(void* pIndex) { taosMemoryFree(((STableIndexInfo*)pIndex)->expr); }
static void destroyFuncParam(void* pValue) { taosMemoryFree(((SFunctParam*)pValue)->pCol); }
static void destroyHintValue(EHintOption option, void* value) { static void destroyHintValue(EHintOption option, void* value) {
switch (option) { switch (option) {
default: default:
@ -1173,6 +1175,7 @@ void nodesDestroyNode(SNode* pNode) {
nodesDestroyList(pLogicNode->pGroupTags); nodesDestroyList(pLogicNode->pGroupTags);
nodesDestroyList(pLogicNode->pTags); nodesDestroyList(pLogicNode->pTags);
nodesDestroyNode(pLogicNode->pSubtable); nodesDestroyNode(pLogicNode->pSubtable);
taosArrayDestroyEx(pLogicNode->pFuncTypes, destroyFuncParam);
break; break;
} }
case QUERY_NODE_LOGIC_PLAN_JOIN: { case QUERY_NODE_LOGIC_PLAN_JOIN: {
@ -1300,6 +1303,7 @@ void nodesDestroyNode(SNode* pNode) {
destroyScanPhysiNode((SScanPhysiNode*)pNode); destroyScanPhysiNode((SScanPhysiNode*)pNode);
nodesDestroyList(pPhyNode->pGroupTags); nodesDestroyList(pPhyNode->pGroupTags);
nodesDestroyList(pPhyNode->pTargets); nodesDestroyList(pPhyNode->pTargets);
taosArrayDestroy(pPhyNode->pFuncTypes);
break; break;
} }
case QUERY_NODE_PHYSICAL_PLAN_TABLE_SCAN: case QUERY_NODE_PHYSICAL_PLAN_TABLE_SCAN:
@ -2111,6 +2115,7 @@ static EDealRes collectFuncs(SNode* pNode, void* pContext) {
FOREACH(pn, pCxt->pFuncs) { FOREACH(pn, pCxt->pFuncs) {
if (nodesEqualNode(pn, pNode)) { if (nodesEqualNode(pn, pNode)) {
bFound = true; bFound = true;
break;
} }
} }
if (!bFound) { if (!bFound) {
@ -2133,6 +2138,20 @@ static int32_t funcNodeEqual(const void* pLeft, const void* pRight, size_t len)
return nodesEqualNode(*(const SNode**)pLeft, *(const SNode**)pRight) ? 0 : 1; return nodesEqualNode(*(const SNode**)pLeft, *(const SNode**)pRight) ? 0 : 1;
} }
int32_t nodesCollectSelectFuncs(SSelectStmt* pSelect, ESqlClause clause, char* tableAlias, FFuncClassifier classifier, SNodeList* pFuncs) {
if (NULL == pSelect || NULL == pFuncs) {
return TSDB_CODE_FAILED;
}
SCollectFuncsCxt cxt = {.errCode = TSDB_CODE_SUCCESS,
.classifier = classifier,
.tableAlias = tableAlias,
.pFuncs = pFuncs};
nodesWalkSelectStmt(pSelect, clause, collectFuncs, &cxt);
return cxt.errCode;
}
int32_t nodesCollectFuncs(SSelectStmt* pSelect, ESqlClause clause, char* tableAlias, FFuncClassifier classifier, SNodeList** pFuncs) { int32_t nodesCollectFuncs(SSelectStmt* pSelect, ESqlClause clause, char* tableAlias, FFuncClassifier classifier, SNodeList** pFuncs) {
if (NULL == pSelect || NULL == pFuncs) { if (NULL == pSelect || NULL == pFuncs) {
return TSDB_CODE_FAILED; return TSDB_CODE_FAILED;

View File

@ -1088,14 +1088,13 @@ static EDealRes translateColumnUseAlias(STranslateContext* pCxt, SColumnNode** p
FOREACH(pNode, pProjectionList) { FOREACH(pNode, pProjectionList) {
SExprNode* pExpr = (SExprNode*)pNode; SExprNode* pExpr = (SExprNode*)pNode;
if (0 == strcmp((*pCol)->colName, pExpr->userAlias)) { if (0 == strcmp((*pCol)->colName, pExpr->userAlias)) {
SColumnRefNode* pColRef = (SColumnRefNode*)nodesMakeNode(QUERY_NODE_COLUMN_REF); SNode* pNew = nodesCloneNode(pNode);
if (NULL == pColRef) { if (NULL == pNew) {
pCxt->errCode = TSDB_CODE_OUT_OF_MEMORY; pCxt->errCode = TSDB_CODE_OUT_OF_MEMORY;
return DEAL_RES_ERROR; return DEAL_RES_ERROR;
} }
strcpy(pColRef->colName, pExpr->aliasName);
nodesDestroyNode(*(SNode**)pCol); nodesDestroyNode(*(SNode**)pCol);
*(SNode**)pCol = (SNode*)pColRef; *(SNode**)pCol = (SNode*)pNew;
*pFound = true; *pFound = true;
return DEAL_RES_CONTINUE; return DEAL_RES_CONTINUE;
} }
@ -2752,6 +2751,23 @@ static int32_t checkIsEmptyResult(STranslateContext* pCxt, SSelectStmt* pSelect)
return TSDB_CODE_SUCCESS; return TSDB_CODE_SUCCESS;
} }
static int32_t resetSelectFuncNumWithoutDup(SSelectStmt* pSelect) {
if (pSelect->selectFuncNum <= 1) return TSDB_CODE_SUCCESS;
pSelect->selectFuncNum = 0;
SNodeList* pNodeList = nodesMakeList();
int32_t code = nodesCollectSelectFuncs(pSelect, SQL_CLAUSE_FROM, NULL, fmIsSelectFunc, pNodeList);
if (TSDB_CODE_SUCCESS != code) {
nodesDestroyList(pNodeList);
return code;
}
SNode* pNode = NULL;
FOREACH(pNode, pNodeList) {
pSelect->selectFuncNum = calcSelectFuncNum((SFunctionNode*)pNode, pSelect->selectFuncNum);
}
nodesDestroyList(pNodeList);
return TSDB_CODE_SUCCESS;
}
static int32_t checkAggColCoexist(STranslateContext* pCxt, SSelectStmt* pSelect) { static int32_t checkAggColCoexist(STranslateContext* pCxt, SSelectStmt* pSelect) {
if (NULL != pSelect->pGroupByList || NULL != pSelect->pWindow || if (NULL != pSelect->pGroupByList || NULL != pSelect->pWindow ||
(!pSelect->hasAggFuncs && !pSelect->hasIndefiniteRowsFunc && !pSelect->hasInterpFunc)) { (!pSelect->hasAggFuncs && !pSelect->hasIndefiniteRowsFunc && !pSelect->hasInterpFunc)) {
@ -2765,7 +2781,8 @@ static int32_t checkAggColCoexist(STranslateContext* pCxt, SSelectStmt* pSelect)
if (!pSelect->isDistinct) { if (!pSelect->isDistinct) {
nodesRewriteExprs(pSelect->pOrderByList, doCheckAggColCoexist, &cxt); nodesRewriteExprs(pSelect->pOrderByList, doCheckAggColCoexist, &cxt);
} }
if (1 == pSelect->selectFuncNum && !pSelect->hasOtherVectorFunc) { if (((!cxt.existCol && 0 < pSelect->selectFuncNum) || (cxt.existCol && 1 == pSelect->selectFuncNum) )
&& !pSelect->hasOtherVectorFunc) {
return rewriteColsToSelectValFunc(pCxt, pSelect); return rewriteColsToSelectValFunc(pCxt, pSelect);
} }
if (cxt.existCol) { if (cxt.existCol) {
@ -3462,13 +3479,7 @@ static int32_t translateOrderByPosition(STranslateContext* pCxt, SNodeList* pPro
} else if (0 == pos || pos > LIST_LENGTH(pProjectionList)) { } else if (0 == pos || pos > LIST_LENGTH(pProjectionList)) {
return generateSyntaxErrMsg(&pCxt->msgBuf, TSDB_CODE_PAR_WRONG_NUMBER_OF_SELECT); return generateSyntaxErrMsg(&pCxt->msgBuf, TSDB_CODE_PAR_WRONG_NUMBER_OF_SELECT);
} else { } else {
SColumnRefNode* pCol = (SColumnRefNode*)nodesMakeNode(QUERY_NODE_COLUMN_REF); // No longer using SColumnRefNode, processing in replaceOrderByAliasImpl function
if (NULL == pCol) {
return generateSyntaxErrMsg(&pCxt->msgBuf, TSDB_CODE_OUT_OF_MEMORY);
}
strcpy(pCol->colName, ((SExprNode*)nodesListGetNode(pProjectionList, pos - 1))->aliasName);
((SOrderByExprNode*)pNode)->pExpr = (SNode*)pCol;
nodesDestroyNode(pExpr);
} }
} else { } else {
*pOther = true; *pOther = true;
@ -4488,12 +4499,13 @@ typedef struct SReplaceOrderByAliasCxt {
static EDealRes replaceOrderByAliasImpl(SNode** pNode, void* pContext) { static EDealRes replaceOrderByAliasImpl(SNode** pNode, void* pContext) {
SReplaceOrderByAliasCxt* pCxt = pContext; SReplaceOrderByAliasCxt* pCxt = pContext;
if (QUERY_NODE_COLUMN_REF == nodeType(*pNode)) {
SNodeList* pProjectionList = pCxt->pProjectionList; SNodeList* pProjectionList = pCxt->pProjectionList;
SNode* pProject = NULL; SNode* pProject = NULL;
if (QUERY_NODE_COLUMN == nodeType(*pNode)) {
FOREACH(pProject, pProjectionList) { FOREACH(pProject, pProjectionList) {
SExprNode* pExpr = (SExprNode*)pProject; SExprNode* pExpr = (SExprNode*)pProject;
if (0 == strcmp(((SColumnRefNode*)*pNode)->colName, pExpr->aliasName)) { if (0 == strcmp(((SColumnRefNode*)*pNode)->colName, pExpr->userAlias)
&& nodeType(*pNode) == nodeType(pProject)) {
SNode* pNew = nodesCloneNode(pProject); SNode* pNew = nodesCloneNode(pProject);
if (NULL == pNew) { if (NULL == pNew) {
pCxt->pTranslateCxt->errCode = TSDB_CODE_OUT_OF_MEMORY; pCxt->pTranslateCxt->errCode = TSDB_CODE_OUT_OF_MEMORY;
@ -4505,7 +4517,29 @@ static EDealRes replaceOrderByAliasImpl(SNode** pNode, void* pContext) {
return DEAL_RES_CONTINUE; return DEAL_RES_CONTINUE;
} }
} }
} else if (QUERY_NODE_ORDER_BY_EXPR == nodeType(*pNode)) {
STranslateContext* pTransCxt = pCxt->pTranslateCxt;
SNode* pExpr = ((SOrderByExprNode*)*pNode)->pExpr;
if (QUERY_NODE_VALUE == nodeType(pExpr)) {
SValueNode* pVal = (SValueNode*)pExpr;
if (DEAL_RES_ERROR == translateValue(pTransCxt, pVal)) {
return pTransCxt->errCode;
} }
int32_t pos = getPositionValue(pVal);
if ( 0 < pos && pos <= LIST_LENGTH(pProjectionList)) {
SNode* pNew = nodesCloneNode(nodesListGetNode(pProjectionList, pos - 1));
if (NULL == pNew) {
pCxt->pTranslateCxt->errCode = TSDB_CODE_OUT_OF_MEMORY;
return DEAL_RES_ERROR;
}
((SExprNode*)pNew)->orderAlias = true;
((SOrderByExprNode*)*pNode)->pExpr = pNew;
nodesDestroyNode(pExpr);
return DEAL_RES_CONTINUE;
}
}
}
return DEAL_RES_CONTINUE; return DEAL_RES_CONTINUE;
} }
@ -4581,6 +4615,7 @@ static int32_t translateSelectFrom(STranslateContext* pCxt, SSelectStmt* pSelect
code = checkIsEmptyResult(pCxt, pSelect); code = checkIsEmptyResult(pCxt, pSelect);
} }
if (TSDB_CODE_SUCCESS == code) { if (TSDB_CODE_SUCCESS == code) {
resetSelectFuncNumWithoutDup(pSelect);
code = checkAggColCoexist(pCxt, pSelect); code = checkAggColCoexist(pCxt, pSelect);
} }
if (TSDB_CODE_SUCCESS == code) { if (TSDB_CODE_SUCCESS == code) {

View File

@ -413,6 +413,28 @@ TEST_F(ParserSelectTest, semanticCheck) {
run("SELECT c1 FROM t1 order by COUNT(*)", TSDB_CODE_PAR_NOT_SINGLE_GROUP); run("SELECT c1 FROM t1 order by COUNT(*)", TSDB_CODE_PAR_NOT_SINGLE_GROUP);
run("SELECT COUNT(*) FROM t1 order by COUNT(*)");
run("SELECT COUNT(*) FROM t1 order by last(c2)");
run("SELECT c1 FROM t1 order by last(ts)");
run("SELECT ts FROM t1 order by last(ts)");
run("SELECT c2 FROM t1 order by last(ts)");
run("SELECT * FROM t1 order by last(ts)");
run("SELECT last(ts) FROM t1 order by last(ts)");
run("SELECT last(ts), ts, c1 FROM t1 order by last(ts)");
run("SELECT ts, last(ts) FROM t1 order by last(ts)");
run("SELECT first(ts), c2 FROM t1 order by last(c1)", TSDB_CODE_PAR_NOT_SINGLE_GROUP);
run("SELECT c1 FROM t1 order by concat(c2, 'abc')");
// TSDB_CODE_PAR_NOT_SELECTED_EXPRESSION // TSDB_CODE_PAR_NOT_SELECTED_EXPRESSION
run("SELECT distinct c1, c2 FROM t1 WHERE c1 > 0 order by ts", TSDB_CODE_PAR_NOT_SELECTED_EXPRESSION); run("SELECT distinct c1, c2 FROM t1 WHERE c1 > 0 order by ts", TSDB_CODE_PAR_NOT_SELECTED_EXPRESSION);

View File

@ -2501,17 +2501,30 @@ static bool lastRowScanOptCheckColNum(int32_t lastColNum, col_id_t lastColId,
return true; return true;
} }
static bool lastRowScanOptCheckFuncList(SLogicNode* pNode, bool* hasOtherFunc) { static bool isNeedSplitCacheLastFunc(SFunctionNode* pFunc, SScanLogicNode* pScan) {
int32_t funcType = pFunc->funcType;
if ((FUNCTION_TYPE_LAST_ROW != funcType || (FUNCTION_TYPE_LAST_ROW == funcType && TSDB_CACHE_MODEL_LAST_VALUE == pScan->cacheLastMode)) &&
(FUNCTION_TYPE_LAST != funcType || (FUNCTION_TYPE_LAST == funcType && (TSDB_CACHE_MODEL_LAST_ROW == pScan->cacheLastMode ||
QUERY_NODE_OPERATOR == nodeType(nodesListGetNode(pFunc->pParameterList, 0)) || QUERY_NODE_VALUE == nodeType(nodesListGetNode(pFunc->pParameterList, 0))))) &&
FUNCTION_TYPE_SELECT_VALUE != funcType && FUNCTION_TYPE_GROUP_KEY != funcType) {
return true;
}
return false;
}
static bool lastRowScanOptCheckFuncList(SLogicNode* pNode, int8_t cacheLastModel, bool* hasOtherFunc) {
bool hasNonPKSelectFunc = false; bool hasNonPKSelectFunc = false;
SNode* pFunc = NULL; SNode* pFunc = NULL;
int32_t lastColNum = 0, selectNonPKColNum = 0; int32_t lastColNum = 0, selectNonPKColNum = 0;
col_id_t lastColId = -1, selectNonPKColId = -1; col_id_t lastColId = -1, selectNonPKColId = -1;
SScanLogicNode* pScan = (SScanLogicNode*)nodesListGetNode(((SAggLogicNode*)pNode)->node.pChildren, 0);
uint32_t needSplitFuncCount = 0;
FOREACH(pFunc, ((SAggLogicNode*)pNode)->pAggFuncs) { FOREACH(pFunc, ((SAggLogicNode*)pNode)->pAggFuncs) {
SFunctionNode* pAggFunc = (SFunctionNode*)pFunc; SFunctionNode* pAggFunc = (SFunctionNode*)pFunc;
SNode* pParam = nodesListGetNode(pAggFunc->pParameterList, 0);
if (FUNCTION_TYPE_LAST == pAggFunc->funcType) { if (FUNCTION_TYPE_LAST == pAggFunc->funcType) {
SNode* pPar = nodesListGetNode(pAggFunc->pParameterList, 0); if (QUERY_NODE_COLUMN == nodeType(pParam)) {
if (QUERY_NODE_COLUMN == nodeType(pPar)) { SColumnNode* pCol = (SColumnNode*)pParam;
SColumnNode* pCol = (SColumnNode*)pPar;
if (pCol->colType != COLUMN_TYPE_COLUMN) { if (pCol->colType != COLUMN_TYPE_COLUMN) {
return false; return false;
} }
@ -2520,13 +2533,18 @@ static bool lastRowScanOptCheckFuncList(SLogicNode* pNode, bool* hasOtherFunc) {
lastColNum++; lastColNum++;
} }
} }
if (QUERY_NODE_VALUE == nodeType(nodesListGetNode(pAggFunc->pParameterList, 0))) { else if (QUERY_NODE_VALUE == nodeType(pParam) || QUERY_NODE_OPERATOR == nodeType(pParam)) {
needSplitFuncCount++;
*hasOtherFunc = true;
}
if (!lastRowScanOptCheckColNum(lastColNum, lastColId, selectNonPKColNum, selectNonPKColId)) {
return false; return false;
} }
if (!lastRowScanOptCheckColNum(lastColNum, lastColId, selectNonPKColNum, selectNonPKColId)) if (TSDB_CACHE_MODEL_LAST_ROW == cacheLastModel) {
return false; needSplitFuncCount++;
*hasOtherFunc = true;
}
} else if (FUNCTION_TYPE_SELECT_VALUE == pAggFunc->funcType) { } else if (FUNCTION_TYPE_SELECT_VALUE == pAggFunc->funcType) {
SNode* pParam = nodesListGetNode(pAggFunc->pParameterList, 0);
if (QUERY_NODE_COLUMN == nodeType(pParam)) { if (QUERY_NODE_COLUMN == nodeType(pParam)) {
SColumnNode* pCol = (SColumnNode*)pParam; SColumnNode* pCol = (SColumnNode*)pParam;
if (COLUMN_TYPE_COLUMN == pCol->colType && PRIMARYKEY_TIMESTAMP_COL_ID != pCol->colId) { if (COLUMN_TYPE_COLUMN == pCol->colType && PRIMARYKEY_TIMESTAMP_COL_ID != pCol->colId) {
@ -2548,15 +2566,21 @@ static bool lastRowScanOptCheckFuncList(SLogicNode* pNode, bool* hasOtherFunc) {
} }
} else if (FUNCTION_TYPE_LAST_ROW != pAggFunc->funcType) { } else if (FUNCTION_TYPE_LAST_ROW != pAggFunc->funcType) {
*hasOtherFunc = true; *hasOtherFunc = true;
needSplitFuncCount++;
} else if (FUNCTION_TYPE_LAST_ROW == pAggFunc->funcType && TSDB_CACHE_MODEL_LAST_VALUE == cacheLastModel) {
*hasOtherFunc = true;
needSplitFuncCount++;
} }
} }
if (needSplitFuncCount >= ((SAggLogicNode*)pNode)->pAggFuncs->length) {
return false;
}
return true; return true;
} }
static bool lastRowScanOptCheckLastCache(SAggLogicNode* pAgg, SScanLogicNode* pScan) { static bool lastRowScanOptCheckLastCache(SAggLogicNode* pAgg, SScanLogicNode* pScan) {
// Only one of LAST and LASTROW can appear if ((pAgg->hasLastRow == pAgg->hasLast && !pAgg->hasLastRow) || (!pAgg->hasLast && !pAgg->hasLastRow) || NULL != pAgg->pGroupKeys || NULL != pScan->node.pConditions ||
if (pAgg->hasLastRow == pAgg->hasLast || (!pAgg->hasLast && !pAgg->hasLastRow) || NULL != pAgg->pGroupKeys || NULL != pScan->node.pConditions ||
!hasSuitableCache(pScan->cacheLastMode, pAgg->hasLastRow, pAgg->hasLast) || !hasSuitableCache(pScan->cacheLastMode, pAgg->hasLastRow, pAgg->hasLast) ||
IS_TSWINDOW_SPECIFIED(pScan->scanRange)) { IS_TSWINDOW_SPECIFIED(pScan->scanRange)) {
return false; return false;
@ -2578,7 +2602,7 @@ static bool lastRowScanOptMayBeOptimized(SLogicNode* pNode) {
} }
bool hasOtherFunc = false; bool hasOtherFunc = false;
if (!lastRowScanOptCheckFuncList(pNode, &hasOtherFunc)) { if (!lastRowScanOptCheckFuncList(pNode, pScan->cacheLastMode, &hasOtherFunc)) {
return false; return false;
} }
@ -2593,6 +2617,7 @@ typedef struct SLastRowScanOptSetColDataTypeCxt {
bool doAgg; bool doAgg;
SNodeList* pLastCols; SNodeList* pLastCols;
SNodeList* pOtherCols; SNodeList* pOtherCols;
int32_t funcType;
} SLastRowScanOptSetColDataTypeCxt; } SLastRowScanOptSetColDataTypeCxt;
static EDealRes lastRowScanOptSetColDataType(SNode* pNode, void* pContext) { static EDealRes lastRowScanOptSetColDataType(SNode* pNode, void* pContext) {
@ -2615,7 +2640,7 @@ static EDealRes lastRowScanOptSetColDataType(SNode* pNode, void* pContext) {
return DEAL_RES_CONTINUE; return DEAL_RES_CONTINUE;
} }
static void lastRowScanOptSetLastTargets(SNodeList* pTargets, SNodeList* pLastCols, bool erase) { static void lastRowScanOptSetLastTargets(SNodeList* pTargets, SNodeList* pLastCols, SNodeList* pLastRowCols, bool erase) {
SNode* pTarget = NULL; SNode* pTarget = NULL;
WHERE_EACH(pTarget, pTargets) { WHERE_EACH(pTarget, pTargets) {
bool found = false; bool found = false;
@ -2627,6 +2652,10 @@ static void lastRowScanOptSetLastTargets(SNodeList* pTargets, SNodeList* pLastCo
break; break;
} }
} }
if (!found && nodeListNodeEqual(pLastRowCols, pTarget)) {
found = true;
}
if (!found && erase) { if (!found && erase) {
ERASE_NODE(pTargets); ERASE_NODE(pTargets);
continue; continue;
@ -2635,7 +2664,7 @@ static void lastRowScanOptSetLastTargets(SNodeList* pTargets, SNodeList* pLastCo
} }
} }
static void lastRowScanOptRemoveUslessTargets(SNodeList* pTargets, SNodeList* pList1, SNodeList* pList2) { static void lastRowScanOptRemoveUslessTargets(SNodeList* pTargets, SNodeList* pList1, SNodeList* pList2, SNodeList* pList3) {
SNode* pTarget = NULL; SNode* pTarget = NULL;
WHERE_EACH(pTarget, pTargets) { WHERE_EACH(pTarget, pTargets) {
bool found = false; bool found = false;
@ -2654,6 +2683,11 @@ static void lastRowScanOptRemoveUslessTargets(SNodeList* pTargets, SNodeList* pL
} }
} }
} }
if (!found && nodeListNodeEqual(pList3, pTarget)) {
found = true;
}
if (!found) { if (!found) {
ERASE_NODE(pTargets); ERASE_NODE(pTargets);
continue; continue;
@ -2662,6 +2696,33 @@ static void lastRowScanOptRemoveUslessTargets(SNodeList* pTargets, SNodeList* pL
} }
} }
static int32_t lastRowScanBuildFuncTypes(SScanLogicNode* pScan, SColumnNode* pColNode, int32_t funcType) {
SFunctParam* pFuncTypeParam = taosMemoryCalloc(1, sizeof(SFunctParam));
if (NULL == pFuncTypeParam) {
return TSDB_CODE_OUT_OF_MEMORY;
}
pFuncTypeParam->type = funcType;
if (NULL == pScan->pFuncTypes) {
pScan->pFuncTypes = taosArrayInit(pScan->pScanCols->length, sizeof(SFunctParam));
if (NULL == pScan->pFuncTypes) {
taosMemoryFree(pFuncTypeParam);
return TSDB_CODE_OUT_OF_MEMORY;
}
}
pFuncTypeParam->pCol = taosMemoryCalloc(1, sizeof(SColumn));
if (NULL == pFuncTypeParam->pCol) {
taosMemoryFree(pFuncTypeParam);
return TSDB_CODE_OUT_OF_MEMORY;
}
pFuncTypeParam->pCol->colId = pColNode->colId;
strcpy(pFuncTypeParam->pCol->name, pColNode->colName);
taosArrayPush(pScan->pFuncTypes, pFuncTypeParam);
taosMemoryFree(pFuncTypeParam);
return TSDB_CODE_SUCCESS;
}
static int32_t lastRowScanOptimize(SOptimizeContext* pCxt, SLogicSubplan* pLogicSubplan) { static int32_t lastRowScanOptimize(SOptimizeContext* pCxt, SLogicSubplan* pLogicSubplan) {
SAggLogicNode* pAgg = (SAggLogicNode*)optFindPossibleNode(pLogicSubplan->pNode, lastRowScanOptMayBeOptimized); SAggLogicNode* pAgg = (SAggLogicNode*)optFindPossibleNode(pLogicSubplan->pNode, lastRowScanOptMayBeOptimized);
@ -2673,10 +2734,16 @@ static int32_t lastRowScanOptimize(SOptimizeContext* pCxt, SLogicSubplan* pLogic
SNode* pNode = NULL; SNode* pNode = NULL;
SColumnNode* pPKTsCol = NULL; SColumnNode* pPKTsCol = NULL;
SColumnNode* pNonPKCol = NULL; SColumnNode* pNonPKCol = NULL;
SScanLogicNode* pScan = (SScanLogicNode*)nodesListGetNode(pAgg->node.pChildren, 0);
pScan->scanType = SCAN_TYPE_LAST_ROW;
pScan->igLastNull = pAgg->hasLast ? true : false;
SArray* isDuplicateCol = taosArrayInit(pScan->pScanCols->length, sizeof(bool));
SNodeList* pLastRowCols = NULL;
FOREACH(pNode, pAgg->pAggFuncs) { FOREACH(pNode, pAgg->pAggFuncs) {
SFunctionNode* pFunc = (SFunctionNode*)pNode; SFunctionNode* pFunc = (SFunctionNode*)pNode;
int32_t funcType = pFunc->funcType; int32_t funcType = pFunc->funcType;
SNode* pParamNode = nodesListGetNode(pFunc->pParameterList, 0);
if (FUNCTION_TYPE_LAST_ROW == funcType || FUNCTION_TYPE_LAST == funcType) { if (FUNCTION_TYPE_LAST_ROW == funcType || FUNCTION_TYPE_LAST == funcType) {
int32_t len = snprintf(pFunc->functionName, sizeof(pFunc->functionName), int32_t len = snprintf(pFunc->functionName, sizeof(pFunc->functionName),
FUNCTION_TYPE_LAST_ROW == funcType ? "_cache_last_row" : "_cache_last"); FUNCTION_TYPE_LAST_ROW == funcType ? "_cache_last_row" : "_cache_last");
@ -2686,6 +2753,61 @@ static int32_t lastRowScanOptimize(SOptimizeContext* pCxt, SLogicSubplan* pLogic
nodesClearList(cxt.pLastCols); nodesClearList(cxt.pLastCols);
return code; return code;
} }
cxt.funcType = pFunc->funcType;
// add duplicate cols which be removed for both last_row, last
if (pAgg->hasLast && pAgg->hasLastRow) {
if (QUERY_NODE_COLUMN == nodeType(pParamNode)) {
SNode* pColNode = NULL;
int i = 0;
FOREACH(pColNode, pScan->pScanCols) {
bool isDup = false;
bool* isDuplicate = taosArrayGet(isDuplicateCol, i);
if (NULL == isDuplicate) {
taosArrayInsert(isDuplicateCol, i, &isDup);
isDuplicate = taosArrayGet(isDuplicateCol, i);
}
i++;
if (nodesEqualNode(pParamNode, pColNode)) {
if (*isDuplicate) {
if (0 == strncmp(((SColumnNode*)pColNode)->colName, "#dup_col.", 9)) {
continue;
}
SNode* newColNode = nodesCloneNode(pColNode);
sprintf(((SColumnNode*)newColNode)->colName, "#dup_col.%p", newColNode);
sprintf(((SColumnNode*)pParamNode)->colName, "#dup_col.%p", newColNode);
nodesListAppend(pScan->pScanCols, newColNode);
isDup = true;
taosArrayInsert(isDuplicateCol, pScan->pScanCols->length, &isDup);
nodesListAppend(pScan->node.pTargets, nodesCloneNode(newColNode));
if (funcType != FUNCTION_TYPE_LAST) {
nodesListMakeAppend(&pLastRowCols, nodesCloneNode(newColNode));
}
lastRowScanBuildFuncTypes(pScan, (SColumnNode*)newColNode, pFunc->funcType);
} else {
isDup = true;
*isDuplicate = isDup;
if (funcType != FUNCTION_TYPE_LAST && !nodeListNodeEqual(cxt.pLastCols, pColNode)) {
nodesListMakeAppend(&pLastRowCols, nodesCloneNode(pColNode));
}
lastRowScanBuildFuncTypes(pScan, (SColumnNode*)pColNode, pFunc->funcType);
}
continue;
}else if (nodeListNodeEqual(pFunc->pParameterList, pColNode)) {
if (funcType != FUNCTION_TYPE_LAST && ((SColumnNode*)pColNode)->colId == PRIMARYKEY_TIMESTAMP_COL_ID &&
!nodeListNodeEqual(pLastRowCols, pColNode)) {
nodesListMakeAppend(&pLastRowCols, nodesCloneNode(pColNode));
lastRowScanBuildFuncTypes(pScan, (SColumnNode*)pColNode, pFunc->funcType);
isDup = true;
*isDuplicate = isDup;
}
}
}
}
}
if (FUNCTION_TYPE_LAST == funcType) { if (FUNCTION_TYPE_LAST == funcType) {
nodesWalkExpr(nodesListGetNode(pFunc->pParameterList, 0), lastRowScanOptSetColDataType, &cxt); nodesWalkExpr(nodesListGetNode(pFunc->pParameterList, 0), lastRowScanOptSetColDataType, &cxt);
nodesListErase(pFunc->pParameterList, nodesListGetCell(pFunc->pParameterList, 1)); nodesListErase(pFunc->pParameterList, nodesListGetCell(pFunc->pParameterList, 1));
@ -2707,15 +2829,13 @@ static int32_t lastRowScanOptimize(SOptimizeContext* pCxt, SLogicSubplan* pLogic
} }
} }
SScanLogicNode* pScan = (SScanLogicNode*)nodesListGetNode(pAgg->node.pChildren, 0);
pScan->scanType = SCAN_TYPE_LAST_ROW;
pScan->igLastNull = pAgg->hasLast ? true : false;
if (NULL != cxt.pLastCols) { if (NULL != cxt.pLastCols) {
cxt.doAgg = false; cxt.doAgg = false;
lastRowScanOptSetLastTargets(pScan->pScanCols, cxt.pLastCols, true); cxt.funcType = FUNCTION_TYPE_CACHE_LAST;
lastRowScanOptSetLastTargets(pScan->pScanCols, cxt.pLastCols, pLastRowCols, true);
nodesWalkExprs(pScan->pScanPseudoCols, lastRowScanOptSetColDataType, &cxt); nodesWalkExprs(pScan->pScanPseudoCols, lastRowScanOptSetColDataType, &cxt);
lastRowScanOptSetLastTargets(pScan->node.pTargets, cxt.pLastCols, false); lastRowScanOptSetLastTargets(pScan->node.pTargets, cxt.pLastCols, pLastRowCols, false);
lastRowScanOptRemoveUslessTargets(pScan->node.pTargets, cxt.pLastCols, cxt.pOtherCols); lastRowScanOptRemoveUslessTargets(pScan->node.pTargets, cxt.pLastCols, cxt.pOtherCols, pLastRowCols);
if (pPKTsCol && pScan->node.pTargets->length == 1) { if (pPKTsCol && pScan->node.pTargets->length == 1) {
// when select last(ts),ts from ..., we add another ts to targets // when select last(ts),ts from ..., we add another ts to targets
sprintf(pPKTsCol->colName, "#sel_val.%p", pPKTsCol); sprintf(pPKTsCol->colName, "#sel_val.%p", pPKTsCol);
@ -2728,10 +2848,12 @@ static int32_t lastRowScanOptimize(SOptimizeContext* pCxt, SLogicSubplan* pLogic
} }
nodesClearList(cxt.pLastCols); nodesClearList(cxt.pLastCols);
} }
pAgg->hasLastRow = false; pAgg->hasLastRow = false;
pAgg->hasLast = false; pAgg->hasLast = false;
pCxt->optimized = true; pCxt->optimized = true;
taosArrayDestroy(isDuplicateCol);
return TSDB_CODE_SUCCESS; return TSDB_CODE_SUCCESS;
} }
@ -2749,7 +2871,7 @@ static bool splitCacheLastFuncOptMayBeOptimized(SLogicNode* pNode) {
} }
bool hasOtherFunc = false; bool hasOtherFunc = false;
if (!lastRowScanOptCheckFuncList(pNode, &hasOtherFunc)) { if (!lastRowScanOptCheckFuncList(pNode, pScan->cacheLastMode, &hasOtherFunc)) {
return false; return false;
} }
@ -2770,6 +2892,16 @@ static int32_t splitCacheLastFuncOptCreateAggLogicNode(SAggLogicNode** pNewAgg,
pNew->hasLastRow = false; pNew->hasLastRow = false;
pNew->hasLast = false; pNew->hasLast = false;
SNode* pFuncNode = NULL;
FOREACH(pFuncNode, pFunc) {
SFunctionNode* pFunc = (SFunctionNode*)pFuncNode;
if (FUNCTION_TYPE_LAST_ROW == pFunc->funcType) {
pNew->hasLastRow = true;
} else if (FUNCTION_TYPE_LAST == pFunc->funcType) {
pNew->hasLast = true;
}
}
pNew->hasTimeLineFunc = pAgg->hasTimeLineFunc; pNew->hasTimeLineFunc = pAgg->hasTimeLineFunc;
pNew->hasGroupKeyOptimized = false; pNew->hasGroupKeyOptimized = false;
pNew->onlyHasKeepOrderFunc = pAgg->onlyHasKeepOrderFunc; pNew->onlyHasKeepOrderFunc = pAgg->onlyHasKeepOrderFunc;
@ -2894,21 +3026,31 @@ static int32_t splitCacheLastFuncOptimize(SOptimizeContext* pCxt, SLogicSubplan*
if (NULL == pAgg) { if (NULL == pAgg) {
return TSDB_CODE_SUCCESS; return TSDB_CODE_SUCCESS;
} }
SScanLogicNode* pScan = (SScanLogicNode*)nodesListGetNode(pAgg->node.pChildren, 0);
SNode* pNode = NULL; SNode* pNode = NULL;
SNodeList* pAggFuncList = NULL; SNodeList* pAggFuncList = NULL;
{ {
bool hasLast = false;
bool hasLastRow = false;
WHERE_EACH(pNode, pAgg->pAggFuncs) { WHERE_EACH(pNode, pAgg->pAggFuncs) {
SFunctionNode* pFunc = (SFunctionNode*)pNode; SFunctionNode* pFunc = (SFunctionNode*)pNode;
int32_t funcType = pFunc->funcType; int32_t funcType = pFunc->funcType;
if (FUNCTION_TYPE_LAST_ROW != funcType && FUNCTION_TYPE_LAST != funcType &&
FUNCTION_TYPE_SELECT_VALUE != funcType && FUNCTION_TYPE_GROUP_KEY != funcType) { if (isNeedSplitCacheLastFunc(pFunc, pScan)) {
nodesListMakeStrictAppend(&pAggFuncList, nodesCloneNode(pNode)); nodesListMakeStrictAppend(&pAggFuncList, nodesCloneNode(pNode));
ERASE_NODE(pAgg->pAggFuncs); ERASE_NODE(pAgg->pAggFuncs);
continue; continue;
} }
if (FUNCTION_TYPE_LAST_ROW == funcType ) {
hasLastRow = true;
} else if (FUNCTION_TYPE_LAST == funcType) {
hasLast = true;
}
WHERE_NEXT; WHERE_NEXT;
} }
pAgg->hasLast = hasLast;
pAgg->hasLastRow = hasLastRow;
} }
if (NULL == pAggFuncList) { if (NULL == pAggFuncList) {

View File

@ -562,9 +562,36 @@ static int32_t createLastRowScanPhysiNode(SPhysiPlanContext* pCxt, SSubplan* pSu
pScan->groupSort = pScanLogicNode->groupSort; pScan->groupSort = pScanLogicNode->groupSort;
pScan->ignoreNull = pScanLogicNode->igLastNull; pScan->ignoreNull = pScanLogicNode->igLastNull;
vgroupInfoToNodeAddr(pScanLogicNode->pVgroupList->vgroups, &pSubplan->execNode); vgroupInfoToNodeAddr(pScanLogicNode->pVgroupList->vgroups, &pSubplan->execNode);
return createScanPhysiNodeFinalize(pCxt, pSubplan, pScanLogicNode, (SScanPhysiNode*)pScan, pPhyNode); int32_t code = createScanPhysiNodeFinalize(pCxt, pSubplan, pScanLogicNode, (SScanPhysiNode*)pScan, pPhyNode);
if (TSDB_CODE_SUCCESS == code && pScanLogicNode->pFuncTypes != NULL) {
pScan->pFuncTypes = taosArrayInit(taosArrayGetSize(pScanLogicNode->pFuncTypes), sizeof(int32_t));
if (NULL == pScan->pFuncTypes) {
return TSDB_CODE_OUT_OF_MEMORY;
}
SNode* pTargetNode = NULL;
int funcTypeIndex = 0;
FOREACH(pTargetNode, ((SScanPhysiNode*)pScan)->pScanCols) {
if (((STargetNode*)pTargetNode)->pExpr->type != QUERY_NODE_COLUMN) {
continue;
}
SColumnNode* pColNode = (SColumnNode*)((STargetNode*)pTargetNode)->pExpr;
for (int i = 0; i < TARRAY_SIZE(pScanLogicNode->pFuncTypes); ++i) {
SFunctParam* pFunctParam = taosArrayGet(pScanLogicNode->pFuncTypes, i);
if (pColNode->colId == pFunctParam->pCol->colId &&
0 == strncmp(pColNode->colName, pFunctParam->pCol->name, strlen(pColNode->colName))) {
taosArrayInsert(pScan->pFuncTypes, funcTypeIndex, &pFunctParam->type);
break;
}
}
funcTypeIndex++;
}
}
return code;
} }
static int32_t createTableCountScanPhysiNode(SPhysiPlanContext* pCxt, SSubplan* pSubplan, static int32_t createTableCountScanPhysiNode(SPhysiPlanContext* pCxt, SSubplan* pSubplan,

View File

@ -120,7 +120,7 @@ int32_t streamTaskSendCheckpointSourceRsp(SStreamTask* pTask);
void streamTaskSetCheckpointFailedId(SStreamTask* pTask); void streamTaskSetCheckpointFailedId(SStreamTask* pTask);
int32_t streamTaskGetNumOfDownstream(const SStreamTask* pTask); int32_t streamTaskGetNumOfDownstream(const SStreamTask* pTask);
int32_t streamTaskInitTokenBucket(STokenBucket* pBucket, int32_t numCap, int32_t numRate, float quotaRate, const char*); int32_t streamTaskInitTokenBucket(STokenBucket* pBucket, int32_t numCap, int32_t numRate, float quotaRate, const char*);
STaskId streamTaskExtractKey(const SStreamTask* pTask); STaskId streamTaskGetTaskId(const SStreamTask* pTask);
void streamTaskInitForLaunchHTask(SHistoryTaskInfo* pInfo); void streamTaskInitForLaunchHTask(SHistoryTaskInfo* pInfo);
void streamTaskSetRetryInfoForLaunch(SHistoryTaskInfo* pInfo); void streamTaskSetRetryInfoForLaunch(SHistoryTaskInfo* pInfo);
int32_t streamTaskBuildScanhistoryRspMsg(SStreamTask* pTask, SStreamScanHistoryFinishReq* pReq, void** pBuffer, int32_t streamTaskBuildScanhistoryRspMsg(SStreamTask* pTask, SStreamScanHistoryFinishReq* pReq, void** pBuffer,

View File

@ -300,28 +300,6 @@ int32_t streamProcessRetrieveReq(SStreamTask* pTask, SStreamRetrieveReq* pReq, S
void streamTaskInputFail(SStreamTask* pTask) { atomic_store_8(&pTask->inputq.status, TASK_INPUT_STATUS__FAILED); } void streamTaskInputFail(SStreamTask* pTask) { atomic_store_8(&pTask->inputq.status, TASK_INPUT_STATUS__FAILED); }
void streamTaskOpenAllUpstreamInput(SStreamTask* pTask) {
int32_t num = taosArrayGetSize(pTask->upstreamInfo.pList);
if (num == 0) {
return;
}
for (int32_t i = 0; i < num; ++i) {
SStreamChildEpInfo* pInfo = taosArrayGetP(pTask->upstreamInfo.pList, i);
pInfo->dataAllowed = true;
}
pTask->upstreamInfo.numOfClosed = 0;
stDebug("s-task:%s opening up inputQ from upstream tasks", pTask->id.idStr);
}
void streamTaskCloseUpstreamInput(SStreamTask* pTask, int32_t taskId) {
SStreamChildEpInfo* pInfo = streamTaskGetUpstreamTaskEpInfo(pTask, taskId);
if (pInfo != NULL) {
pInfo->dataAllowed = false;
}
}
SStreamChildEpInfo* streamTaskGetUpstreamTaskEpInfo(SStreamTask* pTask, int32_t taskId) { SStreamChildEpInfo* streamTaskGetUpstreamTaskEpInfo(SStreamTask* pTask, int32_t taskId) {
int32_t num = taosArrayGetSize(pTask->upstreamInfo.pList); int32_t num = taosArrayGetSize(pTask->upstreamInfo.pList);
for (int32_t i = 0; i < num; ++i) { for (int32_t i = 0; i < num; ++i) {

View File

@ -41,6 +41,8 @@ void destroyRocksdbCfInst(RocksdbCfInst* inst);
int32_t getCfIdx(const char* cfName); int32_t getCfIdx(const char* cfName);
STaskDbWrapper* taskDbOpenImpl(char* key, char* statePath, char* dbPath); STaskDbWrapper* taskDbOpenImpl(char* key, char* statePath, char* dbPath);
int32_t backendCopyFiles(char* src, char* dst);
void destroyCompactFilteFactory(void* arg); void destroyCompactFilteFactory(void* arg);
void destroyCompactFilte(void* arg); void destroyCompactFilte(void* arg);
const char* compactFilteFactoryName(void* arg); const char* compactFilteFactoryName(void* arg);
@ -218,7 +220,7 @@ int32_t rebuildDirFromCheckpoint(const char* path, int64_t chkpId, char** dst) {
taosRemoveDir(state); taosRemoveDir(state);
} }
taosMkDir(state); taosMkDir(state);
code = copyFiles(chkp, state); code = backendCopyFiles(chkp, state);
stInfo("copy snap file from %s to %s", chkp, state); stInfo("copy snap file from %s to %s", chkp, state);
if (code != 0) { if (code != 0) {
stError("failed to restart stream backend from %s, reason: %s", chkp, tstrerror(TAOS_SYSTEM_ERROR(errno))); stError("failed to restart stream backend from %s, reason: %s", chkp, tstrerror(TAOS_SYSTEM_ERROR(errno)));
@ -334,7 +336,7 @@ int32_t rebuildFromRemoteChkp_rsync(char* key, char* chkpPath, int64_t chkpId, c
if (code != 0) { if (code != 0) {
return code; return code;
} }
code = copyFiles(chkpPath, defaultPath); code = backendCopyFiles(chkpPath, defaultPath);
return code; return code;
} }
@ -359,7 +361,7 @@ int32_t rebuildFromRemoteChkp_s3(char* key, char* chkpPath, int64_t chkpId, char
if (code == 0) { if (code == 0) {
taosMkDir(defaultPath); taosMkDir(defaultPath);
code = copyFiles(chkpPath, defaultPath); code = backendCopyFiles(chkpPath, defaultPath);
} }
if (code != 0) { if (code != 0) {
@ -382,6 +384,121 @@ int32_t rebuildFromRemoteChkp(char* key, char* chkpPath, int64_t chkpId, char* d
return -1; return -1;
} }
int32_t copyFiles_create(char* src, char* dst, int8_t type) {
// create and copy file
int32_t err = taosCopyFile(src, dst);
if (errno == EXDEV || errno == ENOTSUP) {
errno = 0;
return 0;
}
return 0;
}
int32_t copyFiles_hardlink(char* src, char* dst, int8_t type) {
// same fs and hard link
return taosLinkFile(src, dst);
}
int32_t backendFileCopyFilesImpl(char* src, char* dst) {
const char* current = "CURRENT";
size_t currLen = strlen(current);
int32_t code = 0;
int32_t sLen = strlen(src);
int32_t dLen = strlen(dst);
char* srcName = taosMemoryCalloc(1, sLen + 64);
char* dstName = taosMemoryCalloc(1, dLen + 64);
// copy file to dst
TdDirPtr pDir = taosOpenDir(src);
if (pDir == NULL) {
taosMemoryFree(srcName);
taosMemoryFree(dstName);
errno = 0;
return -1;
}
TdDirEntryPtr de = NULL;
while ((de = taosReadDir(pDir)) != NULL) {
char* name = taosGetDirEntryName(de);
if (strcmp(name, ".") == 0 || strcmp(name, "..") == 0) continue;
sprintf(srcName, "%s%s%s", src, TD_DIRSEP, name);
sprintf(dstName, "%s%s%s", dst, TD_DIRSEP, name);
if (strncmp(name, current, strlen(name) <= currLen ? strlen(name) : currLen) == 0) {
code = copyFiles_create(srcName, dstName, 0);
if (code != 0) {
stError("failed to copy file, detail: %s to %s reason: %s", srcName, dstName,
tstrerror(TAOS_SYSTEM_ERROR(code)));
goto _ERROR;
}
} else {
code = copyFiles_hardlink(srcName, dstName, 0);
if (code != 0) {
stError("failed to hard line file, detail: %s to %s, reason: %s", srcName, dstName,
tstrerror(TAOS_SYSTEM_ERROR(code)));
goto _ERROR;
}
}
memset(srcName, 0, sLen + 64);
memset(dstName, 0, dLen + 64);
}
taosMemoryFreeClear(srcName);
taosMemoryFreeClear(dstName);
taosCloseDir(&pDir);
errno = 0;
return 0;
_ERROR:
taosMemoryFreeClear(srcName);
taosMemoryFreeClear(dstName);
taosCloseDir(&pDir);
errno = 0;
return -1;
}
int32_t backendCopyFiles(char* src, char* dst) {
return backendFileCopyFilesImpl(src, dst);
// // opt later, just hard link
// int32_t sLen = strlen(src);
// int32_t dLen = strlen(dst);
// char* srcName = taosMemoryCalloc(1, sLen + 64);
// char* dstName = taosMemoryCalloc(1, dLen + 64);
// TdDirPtr pDir = taosOpenDir(src);
// if (pDir == NULL) {
// taosMemoryFree(srcName);
// taosMemoryFree(dstName);
// return -1;
// }
// TdDirEntryPtr de = NULL;
// while ((de = taosReadDir(pDir)) != NULL) {
// char* name = taosGetDirEntryName(de);
// if (strcmp(name, ".") == 0 || strcmp(name, "..") == 0) continue;
// sprintf(srcName, "%s%s%s", src, TD_DIRSEP, name);
// sprintf(dstName, "%s%s%s", dst, TD_DIRSEP, name);
// // if (!taosDirEntryIsDir(de)) {
// // // code = taosCopyFile(srcName, dstName);
// // if (code == -1) {
// // goto _err;
// // }
// // }
// return backendFileCopyFilesImpl(src, dst);
// memset(srcName, 0, sLen + 64);
// memset(dstName, 0, dLen + 64);
// }
// _err:
// taosMemoryFreeClear(srcName);
// taosMemoryFreeClear(dstName);
// taosCloseDir(&pDir);
// return code >= 0 ? 0 : -1;
// return 0;
}
int32_t rebuildFromLocalChkp(char* key, char* chkpPath, int64_t chkpId, char* defaultPath) { int32_t rebuildFromLocalChkp(char* key, char* chkpPath, int64_t chkpId, char* defaultPath) {
int32_t code = -1; int32_t code = -1;
int32_t len = strlen(defaultPath) + 32; int32_t len = strlen(defaultPath) + 32;
@ -396,7 +513,7 @@ int32_t rebuildFromLocalChkp(char* key, char* chkpPath, int64_t chkpId, char* de
taosRemoveDir(tmp); taosRemoveDir(tmp);
} }
taosMkDir(defaultPath); taosMkDir(defaultPath);
code = copyFiles(chkpPath, defaultPath); code = backendCopyFiles(chkpPath, defaultPath);
if (code != 0) { if (code != 0) {
stError("failed to restart stream backend from %s, reason: %s", chkpPath, tstrerror(TAOS_SYSTEM_ERROR(errno))); stError("failed to restart stream backend from %s, reason: %s", chkpPath, tstrerror(TAOS_SYSTEM_ERROR(errno)));
} else { } else {

View File

@ -185,6 +185,11 @@ int32_t streamProcessCheckpointBlock(SStreamTask* pTask, SStreamDataBlock* pBloc
int64_t checkpointId = pDataBlock->info.version; int64_t checkpointId = pDataBlock->info.version;
const char* id = pTask->id.idStr; const char* id = pTask->id.idStr;
int32_t code = TSDB_CODE_SUCCESS; int32_t code = TSDB_CODE_SUCCESS;
int32_t vgId = pTask->pMeta->vgId;
stDebug("s-task:%s vgId:%d start to handle the checkpoint block, checkpointId:%" PRId64 " ver:%" PRId64
", current checkpointingId:%" PRId64,
id, vgId, pTask->chkInfo.checkpointId, pTask->chkInfo.checkpointVer, checkpointId);
// set task status // set task status
if (streamTaskGetStatus(pTask)->state != TASK_STATUS__CK) { if (streamTaskGetStatus(pTask)->state != TASK_STATUS__CK) {
@ -330,7 +335,7 @@ int32_t streamSaveTaskCheckpointInfo(SStreamTask* p, int64_t checkpointId) {
vgId, id, p->info.taskLevel, checkpointId, pCKInfo->checkpointVer, pCKInfo->nextProcessVer, pStatus->name); vgId, id, p->info.taskLevel, checkpointId, pCKInfo->checkpointVer, pCKInfo->nextProcessVer, pStatus->name);
// save the task if not sink task // save the task if not sink task
if (p->info.taskLevel < TASK_LEVEL__SINK) { if (p->info.taskLevel <= TASK_LEVEL__SINK) {
streamMetaWLock(pMeta); streamMetaWLock(pMeta);
code = streamMetaSaveTask(pMeta, p); code = streamMetaSaveTask(pMeta, p);
@ -455,7 +460,7 @@ int32_t streamTaskBuildCheckpoint(SStreamTask* pTask) {
// sink task do not need to save the status, and generated the checkpoint // sink task do not need to save the status, and generated the checkpoint
if (pTask->info.taskLevel != TASK_LEVEL__SINK) { if (pTask->info.taskLevel != TASK_LEVEL__SINK) {
stDebug("s-task:%s level:%d start gen checkpoint", id, pTask->info.taskLevel); stDebug("s-task:%s level:%d start gen checkpoint, checkpointId:%" PRId64, id, pTask->info.taskLevel, ckId);
code = streamBackendDoCheckpoint(pTask->pBackend, ckId); code = streamBackendDoCheckpoint(pTask->pBackend, ckId);
if (code != TSDB_CODE_SUCCESS) { if (code != TSDB_CODE_SUCCESS) {
stError("s-task:%s gen checkpoint:%" PRId64 " failed, code:%s", id, ckId, tstrerror(terrno)); stError("s-task:%s gen checkpoint:%" PRId64 " failed, code:%s", id, ckId, tstrerror(terrno));

View File

@ -713,8 +713,14 @@ bool streamTaskReadyToRun(const SStreamTask* pTask, char** pStatus) {
*pStatus = pState->name; *pStatus = pState->name;
} }
// pause & halt will still run for sink tasks.
if (streamTaskIsSinkTask(pTask)) {
return (st == TASK_STATUS__READY || st == TASK_STATUS__SCAN_HISTORY || st == TASK_STATUS__CK ||
st == TASK_STATUS__PAUSE || st == TASK_STATUS__HALT);
} else {
return (st == TASK_STATUS__READY || st == TASK_STATUS__SCAN_HISTORY || st == TASK_STATUS__CK); return (st == TASK_STATUS__READY || st == TASK_STATUS__SCAN_HISTORY || st == TASK_STATUS__CK);
} }
}
static void doStreamExecTaskHelper(void* param, void* tmrId) { static void doStreamExecTaskHelper(void* param, void* tmrId) {
SStreamTask* pTask = (SStreamTask*)param; SStreamTask* pTask = (SStreamTask*)param;

View File

@ -47,6 +47,12 @@ struct SMetaHbInfo {
int64_t hbStart; int64_t hbStart;
}; };
typedef struct STaskInitTs {
int64_t start;
int64_t end;
bool success;
} STaskInitTs;
SMetaRefMgt gMetaRefMgt; SMetaRefMgt gMetaRefMgt;
void metaRefMgtInit(); void metaRefMgtInit();
@ -581,7 +587,7 @@ int32_t streamMetaRemoveTask(SStreamMeta* pMeta, STaskId* pTaskId) {
int32_t streamMetaRegisterTask(SStreamMeta* pMeta, int64_t ver, SStreamTask* pTask, bool* pAdded) { int32_t streamMetaRegisterTask(SStreamMeta* pMeta, int64_t ver, SStreamTask* pTask, bool* pAdded) {
*pAdded = false; *pAdded = false;
STaskId id = streamTaskExtractKey(pTask); STaskId id = streamTaskGetTaskId(pTask);
void* p = taosHashGet(pMeta->pTasksMap, &id, sizeof(id)); void* p = taosHashGet(pMeta->pTasksMap, &id, sizeof(id));
if (p != NULL) { if (p != NULL) {
return 0; return 0;
@ -871,7 +877,7 @@ int32_t streamMetaLoadAllTasks(SStreamMeta* pMeta) {
int32_t taskId = pTask->id.taskId; int32_t taskId = pTask->id.taskId;
tFreeStreamTask(pTask); tFreeStreamTask(pTask);
STaskId id = streamTaskExtractKey(pTask); STaskId id = streamTaskGetTaskId(pTask);
taosArrayPush(pRecycleList, &id); taosArrayPush(pRecycleList, &id);
int32_t total = taosArrayGetSize(pRecycleList); int32_t total = taosArrayGetSize(pRecycleList);
@ -1302,28 +1308,28 @@ void streamMetaResetStartInfo(STaskStartInfo* pStartInfo) {
} }
void streamMetaRLock(SStreamMeta* pMeta) { void streamMetaRLock(SStreamMeta* pMeta) {
stTrace("vgId:%d meta-rlock", pMeta->vgId); // stTrace("vgId:%d meta-rlock", pMeta->vgId);
taosThreadRwlockRdlock(&pMeta->lock); taosThreadRwlockRdlock(&pMeta->lock);
} }
void streamMetaRUnLock(SStreamMeta* pMeta) { void streamMetaRUnLock(SStreamMeta* pMeta) {
stTrace("vgId:%d meta-runlock", pMeta->vgId); // stTrace("vgId:%d meta-runlock", pMeta->vgId);
int32_t code = taosThreadRwlockUnlock(&pMeta->lock); int32_t code = taosThreadRwlockUnlock(&pMeta->lock);
if (code != TSDB_CODE_SUCCESS) { if (code != TSDB_CODE_SUCCESS) {
stError("vgId:%d meta-runlock failed, code:%d", pMeta->vgId, code); stError("vgId:%d meta-runlock failed, code:%d", pMeta->vgId, code);
} else { } else {
stDebug("vgId:%d meta-runlock completed", pMeta->vgId); // stDebug("vgId:%d meta-runlock completed", pMeta->vgId);
} }
} }
void streamMetaWLock(SStreamMeta* pMeta) { void streamMetaWLock(SStreamMeta* pMeta) {
stTrace("vgId:%d meta-wlock", pMeta->vgId); // stTrace("vgId:%d meta-wlock", pMeta->vgId);
taosThreadRwlockWrlock(&pMeta->lock); taosThreadRwlockWrlock(&pMeta->lock);
stTrace("vgId:%d meta-wlock completed", pMeta->vgId); // stTrace("vgId:%d meta-wlock completed", pMeta->vgId);
} }
void streamMetaWUnLock(SStreamMeta* pMeta) { void streamMetaWUnLock(SStreamMeta* pMeta) {
stTrace("vgId:%d meta-wunlock", pMeta->vgId); // stTrace("vgId:%d meta-wunlock", pMeta->vgId);
taosThreadRwlockUnlock(&pMeta->lock); taosThreadRwlockUnlock(&pMeta->lock);
} }
@ -1407,37 +1413,6 @@ void streamMetaUpdateStageRole(SStreamMeta* pMeta, int64_t stage, bool isLeader)
} }
} }
int32_t streamMetaStopAllTasks(SStreamMeta* pMeta) {
streamMetaRLock(pMeta);
int32_t num = taosArrayGetSize(pMeta->pTaskList);
stDebug("vgId:%d stop all %d stream task(s)", pMeta->vgId, num);
if (num == 0) {
streamMetaRUnLock(pMeta);
return TSDB_CODE_SUCCESS;
}
// send hb msg to mnode before closing all tasks.
SArray* pTaskList = streamMetaSendMsgBeforeCloseTasks(pMeta);
int32_t numOfTasks = taosArrayGetSize(pTaskList);
for (int32_t i = 0; i < numOfTasks; ++i) {
SStreamTaskId* pTaskId = taosArrayGet(pTaskList, i);
SStreamTask* pTask = streamMetaAcquireTaskNoLock(pMeta, pTaskId->streamId, pTaskId->taskId);
if (pTask == NULL) {
continue;
}
streamTaskStop(pTask);
streamMetaReleaseTask(pMeta, pTask);
}
taosArrayDestroy(pTaskList);
streamMetaRUnLock(pMeta);
return 0;
}
int32_t streamMetaStartAllTasks(SStreamMeta* pMeta) { int32_t streamMetaStartAllTasks(SStreamMeta* pMeta) {
int32_t code = TSDB_CODE_SUCCESS; int32_t code = TSDB_CODE_SUCCESS;
int32_t vgId = pMeta->vgId; int32_t vgId = pMeta->vgId;
@ -1512,6 +1487,54 @@ int32_t streamMetaStartAllTasks(SStreamMeta* pMeta) {
return code; return code;
} }
int32_t streamMetaStopAllTasks(SStreamMeta* pMeta) {
streamMetaRLock(pMeta);
int32_t num = taosArrayGetSize(pMeta->pTaskList);
stDebug("vgId:%d stop all %d stream task(s)", pMeta->vgId, num);
if (num == 0) {
streamMetaRUnLock(pMeta);
return TSDB_CODE_SUCCESS;
}
// send hb msg to mnode before closing all tasks.
SArray* pTaskList = streamMetaSendMsgBeforeCloseTasks(pMeta);
int32_t numOfTasks = taosArrayGetSize(pTaskList);
for (int32_t i = 0; i < numOfTasks; ++i) {
SStreamTaskId* pTaskId = taosArrayGet(pTaskList, i);
SStreamTask* pTask = streamMetaAcquireTaskNoLock(pMeta, pTaskId->streamId, pTaskId->taskId);
if (pTask == NULL) {
continue;
}
streamTaskStop(pTask);
streamMetaReleaseTask(pMeta, pTask);
}
taosArrayDestroy(pTaskList);
streamMetaRUnLock(pMeta);
return 0;
}
bool streamMetaAllTasksReady(const SStreamMeta* pMeta) {
int32_t num = taosArrayGetSize(pMeta->pTaskList);
for(int32_t i = 0; i < num; ++i) {
STaskId* pTaskId = taosArrayGet(pMeta->pTaskList, i);
SStreamTask** ppTask = taosHashGet(pMeta->pTasksMap, pTaskId, sizeof(*pTaskId));
if (ppTask == NULL) {
continue;
}
if ((*ppTask)->status.downstreamReady == 0) {
return false;
}
}
return true;
}
int32_t streamMetaStartOneTask(SStreamMeta* pMeta, int64_t streamId, int32_t taskId) { int32_t streamMetaStartOneTask(SStreamMeta* pMeta, int64_t streamId, int32_t taskId) {
int32_t vgId = pMeta->vgId; int32_t vgId = pMeta->vgId;
stInfo("vgId:%d start to task:0x%x by checking downstream status", vgId, taskId); stInfo("vgId:%d start to task:0x%x by checking downstream status", vgId, taskId);
@ -1548,3 +1571,73 @@ int32_t streamMetaStartOneTask(SStreamMeta* pMeta, int64_t streamId, int32_t tas
streamMetaReleaseTask(pMeta, pTask); streamMetaReleaseTask(pMeta, pTask);
return ret; return ret;
} }
static void displayStatusInfo(SStreamMeta* pMeta, SHashObj* pTaskSet, bool succ) {
int32_t vgId = pMeta->vgId;
void* pIter = NULL;
size_t keyLen = 0;
stInfo("vgId:%d %d tasks check-downstream completed %s", vgId, taosHashGetSize(pTaskSet),
succ ? "success" : "failed");
while ((pIter = taosHashIterate(pTaskSet, pIter)) != NULL) {
STaskInitTs* pInfo = pIter;
void* key = taosHashGetKey(pIter, &keyLen);
SStreamTask** pTask1 = taosHashGet(pMeta->pTasksMap, key, sizeof(STaskId));
if (pTask1 == NULL) {
stInfo("s-task:0x%x is dropped already, %s", (int32_t)((STaskId*)key)->taskId, succ ? "success" : "failed");
} else {
stInfo("s-task:%s level:%d vgId:%d, init:%" PRId64 ", initEnd:%" PRId64 ", %s", (*pTask1)->id.idStr,
(*pTask1)->info.taskLevel, vgId, pInfo->start, pInfo->end, pInfo->success ? "success" : "failed");
}
}
}
int32_t streamMetaAddTaskLaunchResult(SStreamMeta* pMeta, int64_t streamId, int32_t taskId, int64_t startTs,
int64_t endTs, bool ready) {
STaskStartInfo* pStartInfo = &pMeta->startInfo;
STaskId id = {.streamId = streamId, .taskId = taskId};
streamMetaWLock(pMeta);
if (pStartInfo->taskStarting != 1) {
int64_t el = endTs - startTs;
qDebug("vgId:%d not start all task(s), not record status, s-task:0x%x launch succ:%d elapsed time:%" PRId64 "ms",
pMeta->vgId, taskId, ready, el);
streamMetaWUnLock(pMeta);
return 0;
}
SHashObj* pDst = ready ? pStartInfo->pReadyTaskSet : pStartInfo->pFailedTaskSet;
STaskInitTs initTs = {.start = startTs, .end = endTs, .success = ready};
taosHashPut(pDst, &id, sizeof(id), &initTs, sizeof(STaskInitTs));
int32_t numOfTotal = streamMetaGetNumOfTasks(pMeta);
int32_t numOfRecv = taosHashGetSize(pStartInfo->pReadyTaskSet) + taosHashGetSize(pStartInfo->pFailedTaskSet);
if (numOfRecv == numOfTotal) {
pStartInfo->readyTs = taosGetTimestampMs();
pStartInfo->elapsedTime = (pStartInfo->startTs != 0) ? pStartInfo->readyTs - pStartInfo->startTs : 0;
stDebug("vgId:%d all %d task(s) check downstream completed, last completed task:0x%x (succ:%d) startTs:%" PRId64
", readyTs:%" PRId64 " total elapsed time:%.2fs",
pMeta->vgId, numOfTotal, taskId, ready, pStartInfo->startTs, pStartInfo->readyTs,
pStartInfo->elapsedTime / 1000.0);
// print the initialization elapsed time and info
displayStatusInfo(pMeta, pStartInfo->pReadyTaskSet, true);
displayStatusInfo(pMeta, pStartInfo->pFailedTaskSet, false);
streamMetaResetStartInfo(pStartInfo);
streamMetaWUnLock(pMeta);
pStartInfo->completeFn(pMeta);
} else {
streamMetaWUnLock(pMeta);
stDebug("vgId:%d recv check downstream results, s-task:0x%x succ:%d, received:%d, total:%d", pMeta->vgId, taskId,
ready, numOfRecv, numOfTotal);
}
return TSDB_CODE_SUCCESS;
}

View File

@ -154,7 +154,6 @@ int32_t streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueItem** pInpu
*numOfBlocks = 0; *numOfBlocks = 0;
*blockSize = 0; *blockSize = 0;
// todo remove it
// no available token in bucket for sink task, let's wait for a little bit // no available token in bucket for sink task, let's wait for a little bit
if (taskLevel == TASK_LEVEL__SINK && (!streamTaskExtractAvailableToken(pTask->outputInfo.pTokenBucket, pTask->id.idStr))) { if (taskLevel == TASK_LEVEL__SINK && (!streamTaskExtractAvailableToken(pTask->outputInfo.pTokenBucket, pTask->id.idStr))) {
stDebug("s-task:%s no available token in bucket for sink data, wait for 10ms", id); stDebug("s-task:%s no available token in bucket for sink data, wait for 10ms", id);

View File

@ -35,12 +35,6 @@ typedef struct STaskRecheckInfo {
void* checkTimer; void* checkTimer;
} STaskRecheckInfo; } STaskRecheckInfo;
typedef struct STaskInitTs {
int64_t start;
int64_t end;
bool success;
} STaskInitTs;
static int32_t streamSetParamForScanHistory(SStreamTask* pTask); static int32_t streamSetParamForScanHistory(SStreamTask* pTask);
static void streamTaskSetRangeStreamCalc(SStreamTask* pTask); static void streamTaskSetRangeStreamCalc(SStreamTask* pTask);
static int32_t initScanHistoryReq(SStreamTask* pTask, SStreamScanHistoryReq* pReq, int8_t igUntreated); static int32_t initScanHistoryReq(SStreamTask* pTask, SStreamScanHistoryReq* pReq, int8_t igUntreated);
@ -546,15 +540,6 @@ int32_t streamSetParamForScanHistory(SStreamTask* pTask) {
return qSetStreamOperatorOptionForScanHistory(pTask->exec.pExecutor); return qSetStreamOperatorOptionForScanHistory(pTask->exec.pExecutor);
} }
int32_t streamResetParamForScanHistory(SStreamTask* pTask) {
stDebug("s-task:%s reset operator option for scan-history data", pTask->id.idStr);
if (pTask->exec.pExecutor != NULL) {
return qResetStreamOperatorOptionForScanHistory(pTask->exec.pExecutor);
} else {
return TSDB_CODE_SUCCESS;
}
}
int32_t streamRestoreParam(SStreamTask* pTask) { int32_t streamRestoreParam(SStreamTask* pTask) {
stDebug("s-task:%s restore operator param after scan-history", pTask->id.idStr); stDebug("s-task:%s restore operator param after scan-history", pTask->id.idStr);
return qRestoreStreamOperatorOption(pTask->exec.pExecutor); return qRestoreStreamOperatorOption(pTask->exec.pExecutor);
@ -1134,97 +1119,3 @@ void streamTaskSetRangeStreamCalc(SStreamTask* pTask) {
} }
} }
void streamTaskPause(SStreamTask* pTask, SStreamMeta* pMeta) {
streamTaskHandleEvent(pTask->status.pSM, TASK_EVENT_PAUSE);
int32_t num = atomic_add_fetch_32(&pMeta->numOfPausedTasks, 1);
stInfo("vgId:%d s-task:%s pause stream task. pause task num:%d", pMeta->vgId, pTask->id.idStr, num);
// in case of fill-history task, stop the tsdb file scan operation.
if (pTask->info.fillHistory == 1) {
void* pExecutor = pTask->exec.pExecutor;
qKillTask(pExecutor, TSDB_CODE_SUCCESS);
}
stDebug("vgId:%d s-task:%s set pause flag and pause task", pMeta->vgId, pTask->id.idStr);
}
void streamTaskResume(SStreamTask* pTask) {
SStreamTaskState prevState = *streamTaskGetStatus(pTask);
SStreamMeta* pMeta = pTask->pMeta;
if (prevState.state == TASK_STATUS__PAUSE || prevState.state == TASK_STATUS__HALT) {
streamTaskRestoreStatus(pTask);
char* pNew = streamTaskGetStatus(pTask)->name;
if (prevState.state == TASK_STATUS__PAUSE) {
int32_t num = atomic_sub_fetch_32(&pMeta->numOfPausedTasks, 1);
stInfo("s-task:%s status:%s resume from %s, paused task(s):%d", pTask->id.idStr, pNew, prevState.name, num);
} else {
stInfo("s-task:%s status:%s resume from %s", pTask->id.idStr, pNew, prevState.name);
}
} else {
stDebug("s-task:%s status:%s not in pause/halt status, no need to resume", pTask->id.idStr, prevState.name);
}
}
static void displayStatusInfo(SStreamMeta* pMeta, SHashObj* pTaskSet, bool succ) {
int32_t vgId = pMeta->vgId;
void* pIter = NULL;
size_t keyLen = 0;
stInfo("vgId:%d %d tasks check-downstream completed %s", vgId, taosHashGetSize(pTaskSet),
succ ? "success" : "failed");
while ((pIter = taosHashIterate(pTaskSet, pIter)) != NULL) {
STaskInitTs* pInfo = pIter;
void* key = taosHashGetKey(pIter, &keyLen);
SStreamTask** pTask1 = taosHashGet(pMeta->pTasksMap, key, sizeof(STaskId));
if (pTask1 == NULL) {
stInfo("s-task:0x%x is dropped already, %s", (int32_t)((STaskId*)key)->taskId, succ ? "success" : "failed");
} else {
stInfo("s-task:%s level:%d vgId:%d, init:%" PRId64 ", initEnd:%" PRId64 ", %s", (*pTask1)->id.idStr,
(*pTask1)->info.taskLevel, vgId, pInfo->start, pInfo->end, pInfo->success ? "success" : "failed");
}
}
}
int32_t streamMetaAddTaskLaunchResult(SStreamMeta* pMeta, int64_t streamId, int32_t taskId, int64_t startTs,
int64_t endTs, bool ready) {
STaskStartInfo* pStartInfo = &pMeta->startInfo;
STaskId id = {.streamId = streamId, .taskId = taskId};
streamMetaWLock(pMeta);
SHashObj* pDst = ready ? pStartInfo->pReadyTaskSet : pStartInfo->pFailedTaskSet;
STaskInitTs initTs = {.start = startTs, .end = endTs, .success = ready};
taosHashPut(pDst, &id, sizeof(id), &initTs, sizeof(STaskInitTs));
int32_t numOfTotal = streamMetaGetNumOfTasks(pMeta);
int32_t numOfRecv = taosHashGetSize(pStartInfo->pReadyTaskSet) + taosHashGetSize(pStartInfo->pFailedTaskSet);
if (numOfRecv == numOfTotal) {
pStartInfo->readyTs = taosGetTimestampMs();
pStartInfo->elapsedTime = (pStartInfo->startTs != 0) ? pStartInfo->readyTs - pStartInfo->startTs : 0;
stDebug("vgId:%d all %d task(s) check downstream completed, last completed task:0x%x (succ:%d) startTs:%" PRId64
", readyTs:%" PRId64 " total elapsed time:%.2fs",
pMeta->vgId, numOfTotal, taskId, ready, pStartInfo->startTs, pStartInfo->readyTs,
pStartInfo->elapsedTime / 1000.0);
// print the initialization elapsed time and info
displayStatusInfo(pMeta, pStartInfo->pReadyTaskSet, true);
displayStatusInfo(pMeta, pStartInfo->pFailedTaskSet, false);
streamMetaResetStartInfo(pStartInfo);
streamMetaWUnLock(pMeta);
pStartInfo->completeFn(pMeta);
} else {
streamMetaWUnLock(pMeta);
stDebug("vgId:%d recv check downstream results, s-task:0x%x succ:%d, received:%d, total:%d", pMeta->vgId, taskId,
ready, numOfRecv, numOfTotal);
}
return TSDB_CODE_SUCCESS;
}

View File

@ -435,7 +435,6 @@ int32_t streamStateAddIfNotExist(SStreamState* pState, const SWinKey* key, void*
int32_t streamStateReleaseBuf(SStreamState* pState, void* pVal, bool used) { int32_t streamStateReleaseBuf(SStreamState* pState, void* pVal, bool used) {
// todo refactor // todo refactor
stDebug("streamStateReleaseBuf");
if (!pVal) { if (!pVal) {
return 0; return 0;
} }

View File

@ -30,6 +30,55 @@ static int32_t addToTaskset(SArray* pArray, SStreamTask* pTask) {
return 0; return 0;
} }
static int32_t doUpdateTaskEpset(SStreamTask* pTask, int32_t nodeId, SEpSet* pEpSet) {
char buf[512] = {0};
if (pTask->info.nodeId == nodeId) { // execution task should be moved away
epsetAssign(&pTask->info.epSet, pEpSet);
EPSET_TO_STR(pEpSet, buf)
stDebug("s-task:0x%x (vgId:%d) self node epset is updated %s", pTask->id.taskId, nodeId, buf);
}
// check for the dispath info and the upstream task info
int32_t level = pTask->info.taskLevel;
if (level == TASK_LEVEL__SOURCE) {
streamTaskUpdateDownstreamInfo(pTask, nodeId, pEpSet);
} else if (level == TASK_LEVEL__AGG) {
streamTaskUpdateUpstreamInfo(pTask, nodeId, pEpSet);
streamTaskUpdateDownstreamInfo(pTask, nodeId, pEpSet);
} else { // TASK_LEVEL__SINK
streamTaskUpdateUpstreamInfo(pTask, nodeId, pEpSet);
}
return 0;
}
static void freeItem(void* p) {
SStreamContinueExecInfo* pInfo = p;
rpcFreeCont(pInfo->msg.pCont);
}
static void freeUpstreamItem(void* p) {
SStreamChildEpInfo** pInfo = p;
taosMemoryFree(*pInfo);
}
static SStreamChildEpInfo* createStreamTaskEpInfo(const SStreamTask* pTask) {
SStreamChildEpInfo* pEpInfo = taosMemoryMalloc(sizeof(SStreamChildEpInfo));
if (pEpInfo == NULL) {
terrno = TSDB_CODE_OUT_OF_MEMORY;
return NULL;
}
pEpInfo->childId = pTask->info.selfChildId;
pEpInfo->epSet = pTask->info.epSet;
pEpInfo->nodeId = pTask->info.nodeId;
pEpInfo->taskId = pTask->id.taskId;
pEpInfo->stage = -1;
return pEpInfo;
}
SStreamTask* tNewStreamTask(int64_t streamId, int8_t taskLevel, bool fillHistory, int64_t triggerParam, SStreamTask* tNewStreamTask(int64_t streamId, int8_t taskLevel, bool fillHistory, int64_t triggerParam,
SArray* pTaskList, bool hasFillhistory) { SArray* pTaskList, bool hasFillhistory) {
SStreamTask* pTask = (SStreamTask*)taosMemoryCalloc(1, sizeof(SStreamTask)); SStreamTask* pTask = (SStreamTask*)taosMemoryCalloc(1, sizeof(SStreamTask));
@ -291,16 +340,6 @@ int32_t tDecodeStreamTaskId(SDecoder* pDecoder, STaskId* pTaskId) {
return 0; return 0;
} }
static void freeItem(void* p) {
SStreamContinueExecInfo* pInfo = p;
rpcFreeCont(pInfo->msg.pCont);
}
static void freeUpstreamItem(void* p) {
SStreamChildEpInfo** pInfo = p;
taosMemoryFree(*pInfo);
}
void tFreeStreamTask(SStreamTask* pTask) { void tFreeStreamTask(SStreamTask* pTask) {
char* p = NULL; char* p = NULL;
int32_t taskId = pTask->id.taskId; int32_t taskId = pTask->id.taskId;
@ -475,14 +514,6 @@ int32_t streamTaskInit(SStreamTask* pTask, SStreamMeta* pMeta, SMsgCb* pMsgCb, i
} }
taosThreadMutexInit(&pTask->lock, &attr); taosThreadMutexInit(&pTask->lock, &attr);
// if (pTask->info.fillHistory == 1) {
// //
// } else {
// }
// if (streamTaskSetDb(pMeta, pTask) != 0) {
// return -1;
// }
streamTaskOpenAllUpstreamInput(pTask); streamTaskOpenAllUpstreamInput(pTask);
pTask->outputInfo.pDownstreamUpdateList = taosArrayInit(4, sizeof(SDownstreamTaskEpset)); pTask->outputInfo.pDownstreamUpdateList = taosArrayInit(4, sizeof(SDownstreamTaskEpset));
@ -509,22 +540,6 @@ int32_t streamTaskGetNumOfDownstream(const SStreamTask* pTask) {
} }
} }
static SStreamChildEpInfo* createStreamTaskEpInfo(const SStreamTask* pTask) {
SStreamChildEpInfo* pEpInfo = taosMemoryMalloc(sizeof(SStreamChildEpInfo));
if (pEpInfo == NULL) {
terrno = TSDB_CODE_OUT_OF_MEMORY;
return NULL;
}
pEpInfo->childId = pTask->info.selfChildId;
pEpInfo->epSet = pTask->info.epSet;
pEpInfo->nodeId = pTask->info.nodeId;
pEpInfo->taskId = pTask->id.taskId;
pEpInfo->stage = -1;
return pEpInfo;
}
int32_t streamTaskSetUpstreamInfo(SStreamTask* pTask, const SStreamTask* pUpstreamTask) { int32_t streamTaskSetUpstreamInfo(SStreamTask* pTask, const SStreamTask* pUpstreamTask) {
SStreamChildEpInfo* pEpInfo = createStreamTaskEpInfo(pUpstreamTask); SStreamChildEpInfo* pEpInfo = createStreamTaskEpInfo(pUpstreamTask);
if (pEpInfo == NULL) { if (pEpInfo == NULL) {
@ -622,29 +637,6 @@ int32_t streamTaskStop(SStreamTask* pTask) {
return 0; return 0;
} }
int32_t doUpdateTaskEpset(SStreamTask* pTask, int32_t nodeId, SEpSet* pEpSet) {
char buf[512] = {0};
if (pTask->info.nodeId == nodeId) { // execution task should be moved away
epsetAssign(&pTask->info.epSet, pEpSet);
EPSET_TO_STR(pEpSet, buf)
stDebug("s-task:0x%x (vgId:%d) self node epset is updated %s", pTask->id.taskId, nodeId, buf);
}
// check for the dispath info and the upstream task info
int32_t level = pTask->info.taskLevel;
if (level == TASK_LEVEL__SOURCE) {
streamTaskUpdateDownstreamInfo(pTask, nodeId, pEpSet);
} else if (level == TASK_LEVEL__AGG) {
streamTaskUpdateUpstreamInfo(pTask, nodeId, pEpSet);
streamTaskUpdateDownstreamInfo(pTask, nodeId, pEpSet);
} else { // TASK_LEVEL__SINK
streamTaskUpdateUpstreamInfo(pTask, nodeId, pEpSet);
}
return 0;
}
int32_t streamTaskUpdateEpsetInfo(SStreamTask* pTask, SArray* pNodeList) { int32_t streamTaskUpdateEpsetInfo(SStreamTask* pTask, SArray* pNodeList) {
STaskExecStatisInfo* p = &pTask->execInfo; STaskExecStatisInfo* p = &pTask->execInfo;
@ -677,7 +669,29 @@ void streamTaskResetUpstreamStageInfo(SStreamTask* pTask) {
stDebug("s-task:%s reset all upstream tasks stage info", pTask->id.idStr); stDebug("s-task:%s reset all upstream tasks stage info", pTask->id.idStr);
} }
bool streamTaskAllUpstreamClosed(SStreamTask* pTask) { void streamTaskOpenAllUpstreamInput(SStreamTask* pTask) {
int32_t num = taosArrayGetSize(pTask->upstreamInfo.pList);
if (num == 0) {
return;
}
for (int32_t i = 0; i < num; ++i) {
SStreamChildEpInfo* pInfo = taosArrayGetP(pTask->upstreamInfo.pList, i);
pInfo->dataAllowed = true;
}
pTask->upstreamInfo.numOfClosed = 0;
stDebug("s-task:%s opening up inputQ for %d upstream tasks", pTask->id.idStr, num);
}
void streamTaskCloseUpstreamInput(SStreamTask* pTask, int32_t taskId) {
SStreamChildEpInfo* pInfo = streamTaskGetUpstreamTaskEpInfo(pTask, taskId);
if (pInfo != NULL) {
pInfo->dataAllowed = false;
}
}
bool streamTaskIsAllUpstreamClosed(SStreamTask* pTask) {
return pTask->upstreamInfo.numOfClosed == taosArrayGetSize(pTask->upstreamInfo.pList); return pTask->upstreamInfo.numOfClosed == taosArrayGetSize(pTask->upstreamInfo.pList);
} }
@ -760,7 +774,7 @@ int32_t streamBuildAndSendDropTaskMsg(SMsgCb* pMsgCb, int32_t vgId, SStreamTaskI
return code; return code;
} }
STaskId streamTaskExtractKey(const SStreamTask* pTask) { STaskId streamTaskGetTaskId(const SStreamTask* pTask) {
STaskId id = {.streamId = pTask->id.streamId, .taskId = pTask->id.taskId}; STaskId id = {.streamId = pTask->id.streamId, .taskId = pTask->id.taskId};
return id; return id;
} }
@ -801,3 +815,40 @@ void streamTaskStatusCopy(STaskStatusEntry* pDst, const STaskStatusEntry* pSrc)
pDst->chkpointTransId = pSrc->chkpointTransId; pDst->chkpointTransId = pSrc->chkpointTransId;
} }
void streamTaskPause(SStreamMeta* pMeta, SStreamTask* pTask) {
streamTaskHandleEvent(pTask->status.pSM, TASK_EVENT_PAUSE);
int32_t num = atomic_add_fetch_32(&pMeta->numOfPausedTasks, 1);
stInfo("vgId:%d s-task:%s pause stream task. pause task num:%d", pMeta->vgId, pTask->id.idStr, num);
// in case of fill-history task, stop the tsdb file scan operation.
if (pTask->info.fillHistory == 1) {
void* pExecutor = pTask->exec.pExecutor;
qKillTask(pExecutor, TSDB_CODE_SUCCESS);
}
stDebug("vgId:%d s-task:%s set pause flag and pause task", pMeta->vgId, pTask->id.idStr);
}
void streamTaskResume(SStreamTask* pTask) {
SStreamTaskState prevState = *streamTaskGetStatus(pTask);
SStreamMeta* pMeta = pTask->pMeta;
if (prevState.state == TASK_STATUS__PAUSE || prevState.state == TASK_STATUS__HALT) {
streamTaskRestoreStatus(pTask);
char* pNew = streamTaskGetStatus(pTask)->name;
if (prevState.state == TASK_STATUS__PAUSE) {
int32_t num = atomic_sub_fetch_32(&pMeta->numOfPausedTasks, 1);
stInfo("s-task:%s status:%s resume from %s, paused task(s):%d", pTask->id.idStr, pNew, prevState.name, num);
} else {
stInfo("s-task:%s status:%s resume from %s", pTask->id.idStr, pNew, prevState.name);
}
} else {
stDebug("s-task:%s status:%s not in pause/halt status, no need to resume", pTask->id.idStr, prevState.name);
}
}
bool streamTaskIsSinkTask(const SStreamTask* pTask) {
return pTask->info.taskLevel == TASK_LEVEL__SINK;
}

View File

@ -527,8 +527,7 @@ bool hasRowBuff(SStreamFileState* pFileState, void* pKey, int32_t keyLen) {
} }
SStreamSnapshot* getSnapshot(SStreamFileState* pFileState) { SStreamSnapshot* getSnapshot(SStreamFileState* pFileState) {
int64_t mark = (INT64_MIN + pFileState->deleteMark >= pFileState->maxTs) ? INT64_MIN int64_t mark = (pFileState->deleteMark == INT64_MAX) ? INT64_MIN : pFileState->maxTs - pFileState->deleteMark;
: pFileState->maxTs - pFileState->deleteMark;
clearExpiredRowBuff(pFileState, mark, false); clearExpiredRowBuff(pFileState, mark, false);
return pFileState->usedBuffs; return pFileState->usedBuffs;
} }

View File

@ -18,8 +18,8 @@
#include "zlib.h" #include "zlib.h"
#ifdef WINDOWS #ifdef WINDOWS
#include <io.h>
#include <WinBase.h> #include <WinBase.h>
#include <io.h>
#include <ktmw32.h> #include <ktmw32.h>
#include <windows.h> #include <windows.h>
#define F_OK 0 #define F_OK 0
@ -1386,3 +1386,15 @@ int32_t taosSetFileHandlesLimit() {
#endif #endif
return 0; return 0;
} }
int32_t taosLinkFile(char *src, char *dst) {
#ifndef WINDOWS
if (link(src, dst) != 0) {
if (errno == EXDEV || errno == ENOTSUP) {
return -1;
}
return errno;
}
#endif
return 0;
}

View File

@ -6,7 +6,8 @@
"user": "root", "user": "root",
"password": "taosdata", "password": "taosdata",
"connection_pool_size": 8, "connection_pool_size": 8,
"num_of_records_per_req": 2000, "num_of_records_per_req": 3000,
"prepared_rand": 3000,
"thread_count": 2, "thread_count": 2,
"create_table_thread_count": 1, "create_table_thread_count": 1,
"confirm_parameter_prompt": "no", "confirm_parameter_prompt": "no",

View File

@ -28,7 +28,9 @@ from frame import *
class TDTestCase(TBase): class TDTestCase(TBase):
updatecfgDict = {
"countAlwaysReturnValue" : "0"
}
def insertData(self): def insertData(self):
tdLog.info(f"insert data.") tdLog.info(f"insert data.")
@ -42,6 +44,10 @@ class TDTestCase(TBase):
self.insert_rows = 100000 self.insert_rows = 100000
self.timestamp_step = 10000 self.timestamp_step = 10000
# create count check table
sql = f"create table {self.db}.ta(ts timestamp, age int) tags(area int)"
tdSql.execute(sql)
def doAction(self): def doAction(self):
tdLog.info(f"do action.") tdLog.info(f"do action.")
self.flushDb() self.flushDb()
@ -64,7 +70,10 @@ class TDTestCase(TBase):
selid = random.choice(vgids) selid = random.choice(vgids)
self.balanceVGroupLeaderOn(selid) self.balanceVGroupLeaderOn(selid)
# check count always return value
sql = f"select count(*) from {self.db}.ta"
tdSql.query(sql)
tdSql.checkRows(0) # countAlwaysReturnValue is false
# run # run
def run(self): def run(self):

View File

@ -75,7 +75,6 @@ class TDTestCase(TBase):
# others # others
etool.exeBinFile("taos", f'-N 200 -l 2048 -s "{sql}" ', wait=False) etool.exeBinFile("taos", f'-N 200 -l 2048 -s "{sql}" ', wait=False)
etool.exeBinFile("taos", f'-n server', wait=False)
def doTaosd(self): def doTaosd(self):
@ -86,11 +85,11 @@ class TDTestCase(TBase):
# -s # -s
sdb = "./sdb.json" sdb = "./sdb.json"
eos.delFile(sdb) eos.delFile(sdb)
etool.runBinFile("taosd", f"-s -c {cfg}") etool.exeBinFile("taosd", f"-s -c {cfg}")
self.checkFileExist(sdb)
# -C # -C
etool.runBinFile("taosd", "-C") etool.exeBinFile("taosd", "-C")
# -k # -k
rets = etool.runBinFile("taosd", "-C") rets = etool.runBinFile("taosd", "-C")
self.checkListNotEmpty(rets) self.checkListNotEmpty(rets)

View File

@ -0,0 +1,61 @@
{
"filetype": "insert",
"cfgdir": "/etc/taos",
"host": "127.0.0.1",
"port": 6030,
"user": "root",
"password": "taosdata",
"connection_pool_size": 8,
"num_of_records_per_req": 4000,
"prepared_rand": 1000,
"thread_count": 3,
"create_table_thread_count": 1,
"confirm_parameter_prompt": "no",
"databases": [
{
"dbinfo": {
"name": "db",
"drop": "yes",
"vgroups": 3,
"replica": 3,
"duration":"3d",
"wal_retention_period": 1,
"wal_retention_size": 1,
"stt_trigger": 1
},
"super_tables": [
{
"name": "stb",
"child_table_exists": "no",
"childtable_count": 6,
"insert_rows": 100000,
"childtable_prefix": "d",
"insert_mode": "taosc",
"timestamp_step": 30000,
"start_timestamp":"2023-10-01 10:00:00",
"columns": [
{ "type": "bool", "name": "bc"},
{ "type": "float", "name": "fc" },
{ "type": "double", "name": "dc"},
{ "type": "tinyint", "name": "ti"},
{ "type": "smallint", "name": "si" },
{ "type": "int", "name": "ic" },
{ "type": "bigint", "name": "bi" },
{ "type": "utinyint", "name": "uti"},
{ "type": "usmallint", "name": "usi"},
{ "type": "uint", "name": "ui" },
{ "type": "ubigint", "name": "ubi"},
{ "type": "binary", "name": "bin", "len": 8},
{ "type": "nchar", "name": "nch", "len": 16}
],
"tags": [
{"type": "tinyint", "name": "groupid","max": 10,"min": 1},
{"name": "location","type": "binary", "len": 16, "values":
["San Francisco", "Los Angles", "San Diego", "San Jose", "Palo Alto", "Campbell", "Mountain View","Sunnyvale", "Santa Clara", "Cupertino"]
}
]
}
]
}
]
}

View File

@ -0,0 +1,79 @@
###################################################################
# 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 -*-
import sys
import time
import random
import taos
import frame
import frame.etool
from frame.log import *
from frame.cases import *
from frame.sql import *
from frame.caseBase import *
from frame import *
class TDTestCase(TBase):
updatecfgDict = {
"keepColumnName" : "1",
"ttlChangeOnWrite" : "1",
"querySmaOptimize": "1"
}
def insertData(self):
tdLog.info(f"insert data.")
# taosBenchmark run
jfile = etool.curFile(__file__, "query_basic.json")
etool.benchMark(json=jfile)
tdSql.execute(f"use {self.db}")
# set insert data information
self.childtable_count = 6
self.insert_rows = 100000
self.timestamp_step = 30000
def doQuery(self):
tdLog.info(f"do query.")
# top bottom
sql = f"select top(uti, 5) from {self.stb} "
tdSql.execute(sql)
# run
def run(self):
tdLog.debug(f"start to excute {__file__}")
# insert data
self.insertData()
# check insert data correct
self.checkInsertCorrect()
# check
self.checkConsistency("usi")
# do action
self.doQuery()
tdLog.success(f"{__file__} successfully executed")
tdCases.addLinux(__file__, TDTestCase())
tdCases.addWindows(__file__, TDTestCase())

View File

@ -7,6 +7,7 @@
"password": "taosdata", "password": "taosdata",
"connection_pool_size": 8, "connection_pool_size": 8,
"num_of_records_per_req": 2000, "num_of_records_per_req": 2000,
"prepared_rand": 1000,
"thread_count": 2, "thread_count": 2,
"create_table_thread_count": 1, "create_table_thread_count": 1,
"confirm_parameter_prompt": "no", "confirm_parameter_prompt": "no",

View File

@ -14,11 +14,12 @@ import time
# Auto Gen class # Auto Gen class
# #
class AutoGen: class AutoGen:
def __init__(self): def __init__(self, fillOne=False):
self.ts = 1600000000000 self.ts = 1600000000000
self.batch_size = 100 self.batch_size = 100
seed = time.time() % 10000 seed = time.time() % 10000
random.seed(seed) random.seed(seed)
self.fillOne = fillOne
# set start ts # set start ts
def set_start_ts(self, ts): def set_start_ts(self, ts):
@ -87,6 +88,23 @@ class AutoGen:
return datas return datas
# fill one data
def fillone_data(self, i, marr):
datas = ""
for c in marr:
if datas != "":
datas += ","
if c == 0:
datas += "%d" % (self.ts + i)
elif c == 12 or c == 13: # binary
datas += '"1"'
else:
datas += '1'
return datas
# generate specail wide random string # generate specail wide random string
def random_string(self, count): def random_string(self, count):
letters = string.ascii_letters letters = string.ascii_letters
@ -96,7 +114,6 @@ class AutoGen:
def create_db(self, dbname, vgroups = 2, replica = 1): def create_db(self, dbname, vgroups = 2, replica = 1):
self.dbname = dbname self.dbname = dbname
tdSql.execute(f'create database {dbname} vgroups {vgroups} replica {replica}') tdSql.execute(f'create database {dbname} vgroups {vgroups} replica {replica}')
tdSql.execute(f'use {dbname}')
# create table or stable # create table or stable
def create_stable(self, stbname, tag_cnt, column_cnt, binary_len, nchar_len): def create_stable(self, stbname, tag_cnt, column_cnt, binary_len, nchar_len):
@ -106,7 +123,7 @@ class AutoGen:
self.mtags, tags = self.gen_columns_sql("t", tag_cnt, binary_len, nchar_len) self.mtags, tags = self.gen_columns_sql("t", tag_cnt, binary_len, nchar_len)
self.mcols, cols = self.gen_columns_sql("c", column_cnt - 1, binary_len, nchar_len) self.mcols, cols = self.gen_columns_sql("c", column_cnt - 1, binary_len, nchar_len)
sql = f"create table {stbname} (ts timestamp, {cols}) tags({tags})" sql = f"create table {self.dbname}.{stbname} (ts timestamp, {cols}) tags({tags})"
tdSql.execute(sql) tdSql.execute(sql)
# create child table # create child table
@ -115,7 +132,7 @@ class AutoGen:
self.child_name = prename self.child_name = prename
for i in range(cnt): for i in range(cnt):
tags_data = self.gen_data(i, self.mtags) tags_data = self.gen_data(i, self.mtags)
sql = f"create table {prename}{i} using {stbname} tags({tags_data})" sql = f"create table {self.dbname}.{prename}{i} using {self.dbname}.{stbname} tags({tags_data})"
tdSql.execute(sql) tdSql.execute(sql)
tdLog.info(f"create child tables {cnt} ok") tdLog.info(f"create child tables {cnt} ok")
@ -127,17 +144,20 @@ class AutoGen:
# loop do # loop do
for i in range(cnt): for i in range(cnt):
if self.fillOne :
value = self.fillone_data(i, self.mcols)
else:
value = self.gen_data(i, self.mcols) value = self.gen_data(i, self.mcols)
ts += step ts += step
values += f"({ts},{value}) " values += f"({ts},{value}) "
if batch_size == 1 or (i > 0 and i % batch_size == 0) : if batch_size == 1 or (i > 0 and i % batch_size == 0) :
sql = f"insert into {child_name} values {values}" sql = f"insert into {self.dbname}.{child_name} values {values}"
tdSql.execute(sql) tdSql.execute(sql)
values = "" values = ""
# end batch # end batch
if values != "": if values != "":
sql = f"insert into {child_name} values {values}" sql = f"insert into {self.dbname}.{child_name} values {values}"
tdSql.execute(sql) tdSql.execute(sql)
tdLog.info(f" insert data i={i}") tdLog.info(f" insert data i={i}")
values = "" values = ""
@ -159,5 +179,3 @@ class AutoGen:
self.insert_data_child(name, cnt, self.batch_size, 0) self.insert_data_child(name, cnt, self.batch_size, 0)
tdLog.info(f" insert same timestamp ok, child table={self.child_cnt} insert rows={cnt}") tdLog.info(f" insert same timestamp ok, child table={self.child_cnt} insert rows={cnt}")

View File

@ -135,8 +135,9 @@ class TBase:
tdSql.checkAgg(sql, self.childtable_count) tdSql.checkAgg(sql, self.childtable_count)
# check step # check step
sql = f"select count(*) from (select diff(ts) as dif from {self.stb} partition by tbname) where dif != {self.timestamp_step}" sql = f"select * from (select diff(ts) as dif from {self.stb} partition by tbname) where dif != {self.timestamp_step}"
tdSql.checkAgg(sql, 0) tdSql.query(sql)
tdSql.checkRows(0)
# save agg result # save agg result
def snapshotAgg(self): def snapshotAgg(self):
@ -156,6 +157,31 @@ class TBase:
tdSql.checkAgg(self.sqlFirst, self.first) tdSql.checkAgg(self.sqlFirst, self.first)
tdSql.checkAgg(self.sqlLast, self.last) tdSql.checkAgg(self.sqlLast, self.last)
# self check
def checkConsistency(self, col):
# top with max
sql = f"select max({col}) from {self.stb}"
expect = tdSql.getFirstValue(sql)
sql = f"select top({col}, 5) from {self.stb}"
tdSql.checkFirstValue(sql, expect)
#bottom with min
sql = f"select min({col}) from {self.stb}"
expect = tdSql.getFirstValue(sql)
sql = f"select bottom({col}, 5) from {self.stb}"
tdSql.checkFirstValue(sql, expect)
# order by asc limit 1 with first
sql = f"select last({col}) from {self.stb}"
expect = tdSql.getFirstValue(sql)
sql = f"select {col} from {self.stb} order by _c0 desc limit 1"
tdSql.checkFirstValue(sql, expect)
# order by desc limit 1 with last
sql = f"select first({col}) from {self.stb}"
expect = tdSql.getFirstValue(sql)
sql = f"select {col} from {self.stb} order by _c0 asc limit 1"
tdSql.checkFirstValue(sql, expect)
# #
# get db information # get db information

View File

@ -550,6 +550,12 @@ class TDSql:
self.query(sql) self.query(sql)
return self.getData(0, 0) return self.getData(0, 0)
# expect first value
def checkFirstValue(self, sql, expect):
self.query(sql)
self.checkData(0, 0, expect)
def get_times(self, time_str, precision="ms"): def get_times(self, time_str, precision="ms"):
caller = inspect.getframeinfo(inspect.stack()[1][0]) caller = inspect.getframeinfo(inspect.stack()[1][0])
if time_str[-1] not in TAOS_TIME_INIT: if time_str[-1] not in TAOS_TIME_INIT:

View File

@ -34,7 +34,7 @@
,,y,system-test,./pytest.sh python3 ./test.py -f 8-stream/pause_resume_test.py ,,y,system-test,./pytest.sh python3 ./test.py -f 8-stream/pause_resume_test.py
#,,n,system-test,python3 ./test.py -f 8-stream/vnode_restart.py -N 4 #,,n,system-test,python3 ./test.py -f 8-stream/vnode_restart.py -N 4
#,,n,system-test,python3 ./test.py -f 8-stream/snode_restart.py -N 4 #,,n,system-test,python3 ./test.py -f 8-stream/snode_restart.py -N 4
#,,n,system-test,python3 ./test.py -f 8-stream/snode_restart_with_checkpoint.py -N 4 ,,n,system-test,python3 ./test.py -f 8-stream/snode_restart_with_checkpoint.py -N 4
,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/tbname_vgroup.py ,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/tbname_vgroup.py
,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/count_interval.py ,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/count_interval.py
@ -422,6 +422,11 @@ e
,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/last_row.py -R ,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/last_row.py -R
,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/last.py ,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/last.py
,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/last.py -R ,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/last.py -R
,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/last_and_last_row.py
,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/last_and_last_row.py -R
,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/last_and_last_row.py -Q 2
,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/last_and_last_row.py -Q 3
,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/last_and_last_row.py -Q 4
,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/leastsquares.py ,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/leastsquares.py
,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/leastsquares.py -R ,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/leastsquares.py -R
,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/length.py ,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/length.py

View File

@ -14,11 +14,12 @@ import time
# Auto Gen class # Auto Gen class
# #
class AutoGen: class AutoGen:
def __init__(self): def __init__(self, fillOne=False):
self.ts = 1600000000000 self.ts = 1600000000000
self.batch_size = 100 self.batch_size = 100
seed = time.time() % 10000 seed = time.time() % 10000
random.seed(seed) random.seed(seed)
self.fillOne = fillOne
# set start ts # set start ts
def set_start_ts(self, ts): def set_start_ts(self, ts):
@ -87,6 +88,23 @@ class AutoGen:
return datas return datas
# fill one data
def fillone_data(self, i, marr):
datas = ""
for c in marr:
if datas != "":
datas += ","
if c == 0:
datas += "%d" % (self.ts + i)
elif c == 12 or c == 13: # binary
datas += '"1"'
else:
datas += '1'
return datas
# generate specail wide random string # generate specail wide random string
def random_string(self, count): def random_string(self, count):
letters = string.ascii_letters letters = string.ascii_letters
@ -96,7 +114,7 @@ class AutoGen:
def create_db(self, dbname, vgroups = 2, replica = 1): def create_db(self, dbname, vgroups = 2, replica = 1):
self.dbname = dbname self.dbname = dbname
tdSql.execute(f'create database {dbname} vgroups {vgroups} replica {replica}') tdSql.execute(f'create database {dbname} vgroups {vgroups} replica {replica}')
tdSql.execute(f'use {dbname}') tdSql.execute(f"use {dbname}")
# create table or stable # create table or stable
def create_stable(self, stbname, tag_cnt, column_cnt, binary_len, nchar_len): def create_stable(self, stbname, tag_cnt, column_cnt, binary_len, nchar_len):
@ -106,7 +124,7 @@ class AutoGen:
self.mtags, tags = self.gen_columns_sql("t", tag_cnt, binary_len, nchar_len) self.mtags, tags = self.gen_columns_sql("t", tag_cnt, binary_len, nchar_len)
self.mcols, cols = self.gen_columns_sql("c", column_cnt - 1, binary_len, nchar_len) self.mcols, cols = self.gen_columns_sql("c", column_cnt - 1, binary_len, nchar_len)
sql = f"create table {stbname} (ts timestamp, {cols}) tags({tags})" sql = f"create table {self.dbname}.{stbname} (ts timestamp, {cols}) tags({tags})"
tdSql.execute(sql) tdSql.execute(sql)
# create child table # create child table
@ -115,7 +133,7 @@ class AutoGen:
self.child_name = prename self.child_name = prename
for i in range(cnt): for i in range(cnt):
tags_data = self.gen_data(i, self.mtags) tags_data = self.gen_data(i, self.mtags)
sql = f"create table {prename}{i} using {stbname} tags({tags_data})" sql = f"create table {self.dbname}.{prename}{i} using {self.dbname}.{stbname} tags({tags_data})"
tdSql.execute(sql) tdSql.execute(sql)
tdLog.info(f"create child tables {cnt} ok") tdLog.info(f"create child tables {cnt} ok")
@ -127,17 +145,20 @@ class AutoGen:
# loop do # loop do
for i in range(cnt): for i in range(cnt):
if self.fillOne :
value = self.fillone_data(i, self.mcols)
else:
value = self.gen_data(i, self.mcols) value = self.gen_data(i, self.mcols)
ts += step ts += step
values += f"({ts},{value}) " values += f"({ts},{value}) "
if batch_size == 1 or (i > 0 and i % batch_size == 0) : if batch_size == 1 or (i > 0 and i % batch_size == 0) :
sql = f"insert into {child_name} values {values}" sql = f"insert into {self.dbname}.{child_name} values {values}"
tdSql.execute(sql) tdSql.execute(sql)
values = "" values = ""
# end batch # end batch
if values != "": if values != "":
sql = f"insert into {child_name} values {values}" sql = f"insert into {self.dbname}.{child_name} values {values}"
tdSql.execute(sql) tdSql.execute(sql)
tdLog.info(f" insert data i={i}") tdLog.info(f" insert data i={i}")
values = "" values = ""

View File

@ -16,9 +16,9 @@ if [[ "$TD_OS" == "Alpine" ]]; then
exit 0 exit 0
fi fi
unset LD_PRELOAD unset LD_PRELOAD
SCRIPT_DIR=`dirname $0` SCRIPT_DIR=$(dirname $0)
cd $SCRIPT_DIR/../ cd $SCRIPT_DIR/../
SCRIPT_DIR=`pwd` SCRIPT_DIR=$(pwd)
IN_TDINTERNAL="community" IN_TDINTERNAL="community"
if [[ "$SCRIPT_DIR" == *"$IN_TDINTERNAL"* ]]; then if [[ "$SCRIPT_DIR" == *"$IN_TDINTERNAL"* ]]; then
@ -27,18 +27,16 @@ else
cd ../../ cd ../../
fi fi
TAOS_DIR=`pwd` TAOS_DIR=$(pwd)
LOG_DIR=$TAOS_DIR/sim/asan LOG_DIR=$TAOS_DIR/sim/asan
error_num=$(cat ${LOG_DIR}/*.asan | grep "ERROR" | wc -l)
archOs=$(arch)
error_num=`cat ${LOG_DIR}/*.asan | grep "ERROR" | wc -l`
archOs=`arch`
if [[ $archOs =~ "aarch64" ]]; then if [[ $archOs =~ "aarch64" ]]; then
echo "arm64 check mem leak" echo "arm64 check mem leak"
memory_leak=`cat ${LOG_DIR}/*.asan | grep "Direct leak" | grep -v "Direct leak of 32 byte"| wc -l` memory_leak=$(cat ${LOG_DIR}/*.asan | grep "Direct leak" | grep -v "Direct leak of 32 byte" | wc -l)
memory_count=`cat ${LOG_DIR}/*.asan | grep "Direct leak of 32 byte"| wc -l` memory_count=$(cat ${LOG_DIR}/*.asan | grep "Direct leak of 32 byte" | wc -l)
if [ $memory_count -eq $error_num ] && [ $memory_leak -eq 0 ]; then if [ $memory_count -eq $error_num ] && [ $memory_leak -eq 0 ]; then
echo "reset error_num to 0, ignore: __cxa_thread_atexit_impl leak" echo "reset error_num to 0, ignore: __cxa_thread_atexit_impl leak"
@ -46,12 +44,11 @@ if [[ $archOs =~ "aarch64" ]]; then
fi fi
else else
echo "os check mem leak" echo "os check mem leak"
memory_leak=`cat ${LOG_DIR}/*.asan | grep "Direct leak" | wc -l` memory_leak=$(cat ${LOG_DIR}/*.asan | grep "Direct leak" | wc -l)
fi fi
indirect_leak=$(cat ${LOG_DIR}/*.asan | grep "Indirect leak" | wc -l)
indirect_leak=`cat ${LOG_DIR}/*.asan | grep "Indirect leak" | wc -l` python_error=$(cat ${LOG_DIR}/*.info | grep -w "stack" | wc -l)
python_error=`cat ${LOG_DIR}/*.info | grep -w "stack" | wc -l`
# ignore # ignore
@ -75,7 +72,7 @@ python_error=`cat ${LOG_DIR}/*.info | grep -w "stack" | wc -l`
#0 0x7f2d64f5a808 in __interceptor_malloc ../../../../src/libsanitizer/asan/asan_malloc_linux.cc:144 #0 0x7f2d64f5a808 in __interceptor_malloc ../../../../src/libsanitizer/asan/asan_malloc_linux.cc:144
#1 0x7f2d63fcf459 in strerror /build/glibc-SzIz7B/glibc-2.31/string/strerror.c:38 #1 0x7f2d63fcf459 in strerror /build/glibc-SzIz7B/glibc-2.31/string/strerror.c:38
runtime_error=`cat ${LOG_DIR}/*.asan | grep "runtime error" | grep -v "trees.c:873" | grep -v "sclfunc.c.*outside the range of representable values of type"| grep -v "signed integer overflow" |grep -v "strerror.c"| grep -v "asan_malloc_linux.cc" |grep -v "strerror.c"|wc -l` runtime_error=$(cat ${LOG_DIR}/*.asan | grep "runtime error" | grep -v "trees.c:873" | grep -v "sclfunc.c.*outside the range of representable values of type" | grep -v "signed integer overflow" | grep -v "strerror.c" | grep -v "asan_malloc_linux.cc" | grep -v "strerror.c" | wc -l)
echo -e "\033[44;32;1m"asan error_num: $error_num"\033[0m" echo -e "\033[44;32;1m"asan error_num: $error_num"\033[0m"
echo -e "\033[44;32;1m"asan memory_leak: $memory_leak"\033[0m" echo -e "\033[44;32;1m"asan memory_leak: $memory_leak"\033[0m"

View File

@ -55,7 +55,7 @@ if $rows != 1 then
return -1 return -1
endi endi
sql explain select count(*), last_row(f1), last(f1) from sta; sql explain select count(*), last_row(f1), last(f1) from sta;
if $data00 != @-> Aggragate (functions=3 width=24 input_order=desc )@ then if $data00 != @-> Merge (columns=3 width=24 input_order=unknown output_order=unknown mode=column)@ then
return -1 return -1
endi endi
sql_error select count(*), last_row(f1), min(f1), f1 from sta; sql_error select count(*), last_row(f1), min(f1), f1 from sta;

View File

@ -0,0 +1,660 @@
import datetime
import sys
from util.log import *
from util.cases import *
from util.sql import *
from util.dnodes import tdDnodes
from math import inf
class TDTestCase:
def init(self, conn, logSql, replicaVer=1):
tdLog.debug("start to execute %s" % __file__)
tdSql.init(conn.cursor(), True)
def check_explain_res_has_row(self, plan_str_expect: str, rows, sql):
plan_found = False
for row in rows:
if str(row).find(plan_str_expect) >= 0:
tdLog.debug("plan: [%s] found in: [%s]" % (plan_str_expect, str(row)))
plan_found = True
break
if not plan_found:
tdLog.exit("plan: %s not found in res: [%s] in sql: %s" % (plan_str_expect, str(rows), sql))
def check_explain_res_no_row(self, plan_str_not_expect: str, res, sql):
for row in res:
if str(row).find(plan_str_not_expect) >= 0:
tdLog.exit('plan: [%s] found in: [%s] for sql: %s' % (plan_str_not_expect, str(row), sql))
def explain_sql(self, sql: str):
sql = "explain " + sql
tdSql.query(sql, queryTimes=1)
return tdSql.queryResult
def explain_and_check_res(self, sqls, hasLastRowScanRes):
for sql, has_last in zip(sqls, hasLastRowScanRes):
res = self.explain_sql(sql)
if has_last == 1:
self.check_explain_res_has_row("Last Row Scan", res, sql)
else:
self.check_explain_res_no_row("Last Row Scan", res, sql)
def none_model_test(self):
tdSql.execute("drop database if exists last_test_none_model ;")
tdSql.execute("create database last_test_none_model cachemodel 'none';")
tdSql.execute("use last_test_none_model;")
tdSql.execute("create stable last_test_none_model.st(ts timestamp, id int) tags(tid int);")
tdSql.execute("create table last_test_none_model.test_t1 using last_test_none_model.st tags(1);")
tdSql.execute("create table last_test_none_model.test_t2 using last_test_none_model.st tags(2);")
tdSql.execute("create table last_test_none_model.test_t3 using last_test_none_model.st tags(3);")
tdSql.execute("create table last_test_none_model.test_t4 using last_test_none_model.st tags(4);")
maxRange = 100
# 2023-11-13 00:00:00.000
startTs = 1699804800000
for i in range(maxRange):
insertSqlString = "insert into last_test_none_model.test_t1 values(%d, %d);" % (startTs + i, i)
tdSql.execute(insertSqlString)
last_ts = startTs + maxRange
tdSql.execute("insert into last_test_none_model.test_t1 (ts) values(%d)" % (last_ts))
sql = f'select last_row(ts), last(*) from last_test_none_model.test_t1;'
tdSql.query(sql)
tdSql.checkRows(1)
tdSql.checkData(0, 0, last_ts)
tdSql.checkData(0, 1, last_ts)
tdSql.checkData(0, 2, maxRange - 1)
explain_res = self.explain_sql(sql)
self.check_explain_res_no_row("Last Row Scan", explain_res, sql)
sql = f'select last_row(ts), last(ts), last_row(id), last(id) from last_test_none_model.test_t1;'
tdSql.query(sql)
tdSql.checkRows(1)
tdSql.checkData(0, 0, last_ts)
tdSql.checkData(0, 1, last_ts)
tdSql.checkData(0, 2, None)
tdSql.checkData(0, 3, maxRange - 1)
explain_res = self.explain_sql(sql)
self.check_explain_res_no_row("Last Row Scan", explain_res, sql)
sql = f'select last(*), last_row(ts), count(*) from last_test_none_model.test_t1;'
tdSql.query(sql)
tdSql.checkRows(1)
tdSql.checkData(0, 0, last_ts)
tdSql.checkData(0, 1, maxRange - 1)
tdSql.checkData(0, 2, last_ts)
tdSql.checkData(0, 3, maxRange + 1)
explain_res = self.explain_sql(sql)
self.check_explain_res_no_row("Last Row Scan", explain_res, sql)
tdSql.error(f'select last(*), last_row(ts), ts from last_test_none_model.test_t1;')
sql = f'select last_row(ts), last(ts), count(*) , last_row(id), last(id), last(*) from last_test_none_model.test_t1;'
tdSql.query(sql)
tdSql.checkRows(1)
tdSql.checkData(0, 0, last_ts)
tdSql.checkData(0, 1, last_ts)
tdSql.checkData(0, 2, maxRange + 1)
tdSql.checkData(0, 3, None)
tdSql.checkData(0, 4, maxRange - 1)
tdSql.checkData(0, 5, last_ts)
tdSql.checkData(0, 6, maxRange - 1)
startTs2 = startTs + 86400000
for i in range(maxRange):
i = i + 2 * maxRange
insertSqlString = "insert into last_test_none_model.test_t2 values(%d, %d);" % (startTs2 + i, i)
tdSql.execute(insertSqlString)
last_ts2 = startTs2 + maxRange
startTs3 = startTs + 2 * 86400000
for i in range(maxRange):
i = i + 3 * maxRange
insertSqlString = "insert into last_test_none_model.test_t3 values(%d, %d);" % (startTs3 + i, i)
tdSql.execute(insertSqlString)
last_ts3 = startTs3 + 4 * maxRange - 1
startTs4 = startTs + 3 * 86400000
for i in range(maxRange):
i = i + 4 * maxRange
insertSqlString = "insert into last_test_none_model.test_t4 values(%d, %d);" % (startTs4 + i, i)
tdSql.execute(insertSqlString)
last_ts4 = startTs4 + 5 * maxRange - 1
sql = f'select last_row(ts), last(*) from last_test_none_model.st;'
tdSql.query(sql)
tdSql.checkRows(1)
tdSql.checkData(0, 0, last_ts4)
tdSql.checkData(0, 1, last_ts4)
tdSql.checkData(0, 2, 5 * maxRange - 1)
explain_res = self.explain_sql(sql)
self.check_explain_res_no_row("Last Row Scan", explain_res, sql)
sql = f'select last_row(ts), last(ts), last_row(id), last(id) from last_test_none_model.st;'
tdSql.query(sql)
tdSql.checkRows(1)
tdSql.checkData(0, 0, last_ts4)
tdSql.checkData(0, 1, last_ts4)
tdSql.checkData(0, 2, 5 * maxRange - 1)
tdSql.checkData(0, 3, 5 * maxRange - 1)
explain_res = self.explain_sql(sql)
self.check_explain_res_no_row("Last Row Scan", explain_res, sql)
sql = f'select last(*), last_row(ts), count(*) from last_test_none_model.st;'
tdSql.query(sql)
tdSql.checkRows(1)
tdSql.checkData(0, 0, last_ts4)
tdSql.checkData(0, 1, 5 * maxRange - 1)
tdSql.checkData(0, 2, last_ts4)
tdSql.checkData(0, 3, 4 * maxRange + 1)
explain_res = self.explain_sql(sql)
self.check_explain_res_no_row("Last Row Scan", explain_res, sql)
tdSql.error(f'select last(*), last_row(ts), ts from last_test_none_model.st;')
sql = f'select last_row(ts), last(ts), count(*) , last_row(id), last(id), last(*) from last_test_none_model.st;'
tdSql.query(sql)
tdSql.checkRows(1)
tdSql.checkData(0, 0, last_ts4)
tdSql.checkData(0, 1, last_ts4)
tdSql.checkData(0, 2, 4 * maxRange + 1)
tdSql.checkData(0, 3, 5 * maxRange - 1)
tdSql.checkData(0, 4, 5 * maxRange - 1)
tdSql.checkData(0, 5, last_ts4)
tdSql.checkData(0, 6, 5 * maxRange - 1)
sql = f'select last_row(1), last(2), count(*) , last_row(id), last(id), last(*) from last_test_none_model.st;'
tdSql.query(sql)
tdSql.checkRows(1)
tdSql.checkData(0, 0, 1)
tdSql.checkData(0, 1, 2)
tdSql.checkData(0, 2, 4 * maxRange + 1)
tdSql.checkData(0, 3, 5 * maxRange - 1)
tdSql.checkData(0, 4, 5 * maxRange - 1)
tdSql.checkData(0, 5, last_ts4)
tdSql.checkData(0, 6, 5 * maxRange - 1)
tdSql.execute("drop table if exists last_test_none_model.test_t4 ;")
tdSql.execute("drop table if exists last_test_none_model.test_t3 ;")
tdSql.execute("drop table if exists last_test_none_model.test_t2 ;")
tdSql.execute("drop table if exists last_test_none_model.test_t1 ;")
tdSql.execute("drop stable if exists last_test_none_model.st;")
tdSql.execute("drop database if exists last_test_none_model;")
def last_value_model_test(self):
tdSql.execute("create database last_test_last_value_model cachemodel 'last_value' ;")
tdSql.execute("use last_test_last_value_model;")
tdSql.execute("create stable last_test_last_value_model.st(ts timestamp, id int) tags(tid int);")
tdSql.execute("create table last_test_last_value_model.test_t1 using last_test_last_value_model.st tags(1);")
tdSql.execute("create table last_test_last_value_model.test_t2 using last_test_last_value_model.st tags(2);")
tdSql.execute("create table last_test_last_value_model.test_t3 using last_test_last_value_model.st tags(3);")
tdSql.execute("create table last_test_last_value_model.test_t4 using last_test_last_value_model.st tags(4);")
maxRange = 100
# 2023-11-13 00:00:00.000
startTs = 1699804800000
for i in range(maxRange):
insertSqlString = "insert into last_test_last_value_model.test_t1 values(%d, %d);" % (startTs + i, i)
tdSql.execute(insertSqlString)
last_ts = startTs + maxRange
tdSql.execute("insert into last_test_last_value_model.test_t1 (ts) values(%d)" % (last_ts))
sql = f'select last_row(ts), last(*) from last_test_last_value_model.test_t1;'
tdSql.query(sql)
tdSql.checkRows(1)
tdSql.checkData(0, 0, last_ts)
tdSql.checkData(0, 1, last_ts)
tdSql.checkData(0, 2, maxRange - 1)
explain_res = self.explain_sql(sql)
self.check_explain_res_has_row("Last Row Scan", explain_res, sql)
self.check_explain_res_has_row("Table Scan", explain_res, sql)
sql = f'select last_row(ts), last(ts), last_row(id), last(id) from last_test_last_value_model.test_t1;'
tdSql.query(sql)
tdSql.checkRows(1)
tdSql.checkData(0, 0, last_ts)
tdSql.checkData(0, 1, last_ts)
tdSql.checkData(0, 2, None)
tdSql.checkData(0, 3, maxRange - 1)
explain_res = self.explain_sql(sql)
self.check_explain_res_has_row("Last Row Scan", explain_res, sql)
self.check_explain_res_has_row("Table Scan", explain_res, sql)
sql = f'select last(*), last_row(ts), count(*) from last_test_last_value_model.test_t1;'
tdSql.query(sql)
tdSql.checkRows(1)
tdSql.checkData(0, 0, last_ts)
tdSql.checkData(0, 1, maxRange - 1)
tdSql.checkData(0, 2, last_ts)
tdSql.checkData(0, 3, maxRange + 1)
explain_res = self.explain_sql(sql)
self.check_explain_res_has_row("Last Row Scan", explain_res, sql)
self.check_explain_res_has_row("Table Scan", explain_res, sql)
sql = f'select last_row(ts), last(ts), count(*) , last_row(id), last(id), last(*) from last_test_last_value_model.test_t1;'
tdSql.query(sql)
tdSql.checkRows(1)
tdSql.checkData(0, 0, last_ts)
tdSql.checkData(0, 1, last_ts)
tdSql.checkData(0, 2, maxRange + 1)
tdSql.checkData(0, 3, None)
tdSql.checkData(0, 4, maxRange - 1)
tdSql.checkData(0, 5, last_ts)
tdSql.checkData(0, 6, maxRange - 1)
startTs2 = startTs + 86400000
for i in range(maxRange):
i = i + 2 * maxRange
insertSqlString = "insert into last_test_last_value_model.test_t2 values(%d, %d);" % (startTs2 + i, i)
tdSql.execute(insertSqlString)
last_ts2 = startTs2 + maxRange
startTs3 = startTs + 2 * 86400000
for i in range(maxRange):
i = i + 3 * maxRange
insertSqlString = "insert into last_test_last_value_model.test_t3 values(%d, %d);" % (startTs3 + i, i)
tdSql.execute(insertSqlString)
last_ts3 = startTs3 + 4 * maxRange - 1
startTs4 = startTs + 3 * 86400000
for i in range(maxRange):
i = i + 4 * maxRange
insertSqlString = "insert into last_test_last_value_model.test_t4 values(%d, %d);" % (startTs4 + i, i)
tdSql.execute(insertSqlString)
last_ts4 = startTs4 + 5 * maxRange - 1
sql = f'select last_row(ts), last(*) from last_test_last_value_model.st;'
tdSql.query(sql)
tdSql.checkRows(1)
tdSql.checkData(0, 0, last_ts4)
tdSql.checkData(0, 1, last_ts4)
tdSql.checkData(0, 2, 5 * maxRange - 1)
explain_res = self.explain_sql(sql)
self.check_explain_res_has_row("Last Row Scan", explain_res, sql)
self.check_explain_res_has_row("Table Scan", explain_res, sql)
sql = f'select last_row(ts), last(ts), last_row(id), last(id) from last_test_last_value_model.st;'
tdSql.query(sql)
tdSql.checkRows(1)
tdSql.checkData(0, 0, last_ts4)
tdSql.checkData(0, 1, last_ts4)
tdSql.checkData(0, 2, 5 * maxRange - 1)
tdSql.checkData(0, 3, 5 * maxRange - 1)
explain_res = self.explain_sql(sql)
self.check_explain_res_has_row("Last Row Scan", explain_res, sql)
self.check_explain_res_has_row("Table Scan", explain_res, sql)
sql = f'select last(*), last_row(ts), count(*) from last_test_last_value_model.st;'
tdSql.query(sql)
tdSql.checkRows(1)
tdSql.checkData(0, 0, last_ts4)
tdSql.checkData(0, 1, 5 * maxRange - 1)
tdSql.checkData(0, 2, last_ts4)
tdSql.checkData(0, 3, 4 * maxRange + 1)
explain_res = self.explain_sql(sql)
self.check_explain_res_has_row("Last Row Scan", explain_res, sql)
self.check_explain_res_has_row("Table Scan", explain_res, sql)
tdSql.error(f'select last(*), last_row(ts), ts from last_test_last_value_model.st;')
sql = f'select last_row(ts), last(ts), count(*) , last_row(id), last(id), last(*) from last_test_last_value_model.st;'
tdSql.query(sql)
tdSql.checkRows(1)
tdSql.checkData(0, 0, last_ts4)
tdSql.checkData(0, 1, last_ts4)
tdSql.checkData(0, 2, 4 * maxRange + 1)
tdSql.checkData(0, 3, 5 * maxRange - 1)
tdSql.checkData(0, 4, 5 * maxRange - 1)
tdSql.checkData(0, 5, last_ts4)
tdSql.checkData(0, 6, 5 * maxRange - 1)
sql = f'select last_row(1), last(2), count(*) , last_row(id), last(id), last(*) from last_test_last_value_model.st;'
tdSql.query(sql)
tdSql.checkRows(1)
tdSql.checkData(0, 0, 1)
tdSql.checkData(0, 1, 2)
tdSql.checkData(0, 2, 4 * maxRange + 1)
tdSql.checkData(0, 3, 5 * maxRange - 1)
tdSql.checkData(0, 4, 5 * maxRange - 1)
tdSql.checkData(0, 5, last_ts4)
tdSql.checkData(0, 6, 5 * maxRange - 1)
tdSql.execute("drop table if exists last_test_last_value_model.test_t4 ;")
tdSql.execute("drop table if exists last_test_last_value_model.test_t3 ;")
tdSql.execute("drop table if exists last_test_last_value_model.test_t2 ;")
tdSql.execute("drop table if exists last_test_last_value_model.test_t1 ;")
tdSql.execute("drop stable if exists last_test_last_value_model.st;")
tdSql.execute("drop database if exists last_test_last_value_model;")
def last_row_model_test(self):
tdSql.execute("create database last_test_last_row_model cachemodel 'last_row';")
tdSql.execute("use last_test_last_row_model;")
tdSql.execute("create stable last_test_last_row_model.st(ts timestamp, id int) tags(tid int);")
tdSql.execute("create table last_test_last_row_model.test_t1 using last_test_last_row_model.st tags(1);")
tdSql.execute("create table last_test_last_row_model.test_t2 using last_test_last_row_model.st tags(2);")
tdSql.execute("create table last_test_last_row_model.test_t3 using last_test_last_row_model.st tags(3);")
tdSql.execute("create table last_test_last_row_model.test_t4 using last_test_last_row_model.st tags(4);")
maxRange = 100
# 2023-11-13 00:00:00.000
startTs = 1699804800000
for i in range(maxRange):
insertSqlString = "insert into last_test_last_row_model.test_t1 values(%d, %d);" % (startTs + i, i)
tdSql.execute(insertSqlString)
last_ts = startTs + maxRange
tdSql.execute("insert into last_test_last_row_model.test_t1 (ts) values(%d)" % (last_ts))
sql = f'select last_row(ts), last(*) from last_test_last_row_model.test_t1;'
tdSql.query(sql)
tdSql.checkRows(1)
tdSql.checkData(0, 0, last_ts)
tdSql.checkData(0, 1, last_ts)
tdSql.checkData(0, 2, maxRange - 1)
explain_res = self.explain_sql(sql)
self.check_explain_res_has_row("Last Row Scan", explain_res, sql)
self.check_explain_res_has_row("Table Scan", explain_res, sql)
sql = f'select last_row(ts), last(ts), last_row(id), last(id) from last_test_last_row_model.test_t1;'
tdSql.query(sql)
tdSql.checkRows(1)
tdSql.checkData(0, 0, last_ts)
tdSql.checkData(0, 1, last_ts)
tdSql.checkData(0, 2, None)
tdSql.checkData(0, 3, maxRange - 1)
explain_res = self.explain_sql(sql)
self.check_explain_res_has_row("Last Row Scan", explain_res, sql)
self.check_explain_res_has_row("Table Scan", explain_res, sql)
sql = f'select last(*), last_row(ts), count(*) from last_test_last_row_model.test_t1;'
tdSql.query(sql)
tdSql.checkRows(1)
tdSql.checkData(0, 0, last_ts)
tdSql.checkData(0, 1, maxRange - 1)
tdSql.checkData(0, 2, last_ts)
tdSql.checkData(0, 3, maxRange + 1)
explain_res = self.explain_sql(sql)
self.check_explain_res_has_row("Last Row Scan", explain_res, sql)
self.check_explain_res_has_row("Table Scan", explain_res, sql)
sql = f'select last_row(ts), last(ts), count(*) , last_row(id), last(id), last(*) from last_test_last_row_model.test_t1;'
tdSql.query(sql)
tdSql.checkRows(1)
tdSql.checkData(0, 0, last_ts)
tdSql.checkData(0, 1, last_ts)
tdSql.checkData(0, 2, maxRange + 1)
tdSql.checkData(0, 3, None)
tdSql.checkData(0, 4, maxRange - 1)
tdSql.checkData(0, 5, last_ts)
tdSql.checkData(0, 6, maxRange - 1)
startTs2 = startTs + 86400000
for i in range(maxRange):
i = i + 2 * maxRange
insertSqlString = "insert into last_test_last_row_model.test_t2 values(%d, %d);" % (startTs2 + i, i)
tdSql.execute(insertSqlString)
last_ts2 = startTs2 + maxRange
startTs3 = startTs + 2 * 86400000
for i in range(maxRange):
i = i + 3 * maxRange
insertSqlString = "insert into last_test_last_row_model.test_t3 values(%d, %d);" % (startTs3 + i, i)
tdSql.execute(insertSqlString)
last_ts3 = startTs3 + 4 * maxRange - 1
startTs4 = startTs + 3 * 86400000
for i in range(maxRange):
i = i + 4 * maxRange
insertSqlString = "insert into last_test_last_row_model.test_t4 values(%d, %d);" % (startTs4 + i, i)
tdSql.execute(insertSqlString)
last_ts4 = startTs4 + 5 * maxRange - 1
sql = f'select last_row(ts), last(*) from last_test_last_row_model.st;'
tdSql.query(sql)
tdSql.checkRows(1)
tdSql.checkData(0, 0, last_ts4)
tdSql.checkData(0, 1, last_ts4)
tdSql.checkData(0, 2, 5 * maxRange - 1)
explain_res = self.explain_sql(sql)
self.check_explain_res_has_row("Last Row Scan", explain_res, sql)
self.check_explain_res_has_row("Table Scan", explain_res, sql)
sql = f'select last_row(ts), last(ts), last_row(id), last(id) from last_test_last_row_model.st;'
tdSql.query(sql)
tdSql.checkRows(1)
tdSql.checkData(0, 0, last_ts4)
tdSql.checkData(0, 1, last_ts4)
tdSql.checkData(0, 2, 5 * maxRange - 1)
tdSql.checkData(0, 3, 5 * maxRange - 1)
explain_res = self.explain_sql(sql)
self.check_explain_res_has_row("Last Row Scan", explain_res, sql)
self.check_explain_res_has_row("Table Scan", explain_res, sql)
sql = f'select last(*), last_row(ts), count(*) from last_test_last_row_model.st;'
tdSql.query(sql)
tdSql.checkRows(1)
tdSql.checkData(0, 0, last_ts4)
tdSql.checkData(0, 1, 5 * maxRange - 1)
tdSql.checkData(0, 2, last_ts4)
tdSql.checkData(0, 3, 4 * maxRange + 1)
explain_res = self.explain_sql(sql)
self.check_explain_res_has_row("Last Row Scan", explain_res, sql)
self.check_explain_res_has_row("Table Scan", explain_res, sql)
tdSql.error(f'select last(*), last_row(ts), ts from last_test_last_row_model.st;')
sql = f'select last_row(ts), last(ts), count(*) , last_row(id), last(id), last(*) from last_test_last_row_model.st;'
tdSql.query(sql)
tdSql.checkRows(1)
tdSql.checkData(0, 0, last_ts4)
tdSql.checkData(0, 1, last_ts4)
tdSql.checkData(0, 2, 4 * maxRange + 1)
tdSql.checkData(0, 3, 5 * maxRange - 1)
tdSql.checkData(0, 4, 5 * maxRange - 1)
tdSql.checkData(0, 5, last_ts4)
tdSql.checkData(0, 6, 5 * maxRange - 1)
sql = f'select last_row(1), last(2), count(*) , last_row(id), last(id), last(*) from last_test_last_row_model.st;'
tdSql.query(sql)
tdSql.checkRows(1)
tdSql.checkData(0, 0, 1)
tdSql.checkData(0, 1, 2)
tdSql.checkData(0, 2, 4 * maxRange + 1)
tdSql.checkData(0, 3, 5 * maxRange - 1)
tdSql.checkData(0, 4, 5 * maxRange - 1)
tdSql.checkData(0, 5, last_ts4)
tdSql.checkData(0, 6, 5 * maxRange - 1)
tdSql.execute("drop table if exists last_test_last_row_model.test_t4 ;")
tdSql.execute("drop table if exists last_test_last_row_model.test_t3 ;")
tdSql.execute("drop table if exists last_test_last_row_model.test_t2 ;")
tdSql.execute("drop table if exists last_test_last_row_model.test_t1 ;")
tdSql.execute("drop stable if exists last_test_last_row_model.st;")
tdSql.execute("drop database if exists last_test_last_row_model;")
def both_model_test(self):
tdSql.execute("create database last_test_both_model cachemodel 'both';")
tdSql.execute("use last_test_both_model;")
tdSql.execute("create stable last_test_both_model.st(ts timestamp, id int) tags(tid int);")
tdSql.execute("create table last_test_both_model.test_t1 using last_test_both_model.st tags(1);")
tdSql.execute("create table last_test_both_model.test_t2 using last_test_both_model.st tags(2);")
tdSql.execute("create table last_test_both_model.test_t3 using last_test_both_model.st tags(3);")
tdSql.execute("create table last_test_both_model.test_t4 using last_test_both_model.st tags(4);")
maxRange = 100
# 2023-11-13 00:00:00.000
startTs = 1699804800000
for i in range(maxRange):
insertSqlString = "insert into last_test_both_model.test_t1 values(%d, %d);" % (startTs + i, i)
tdSql.execute(insertSqlString)
last_ts = startTs + maxRange
tdSql.execute("insert into last_test_both_model.test_t1 (ts) values(%d)" % (last_ts))
sql = f'select last_row(ts), last(*) from last_test_both_model.test_t1;'
tdSql.query(sql)
tdSql.checkRows(1)
tdSql.checkData(0, 0, last_ts)
tdSql.checkData(0, 1, last_ts)
tdSql.checkData(0, 2, maxRange - 1)
explain_res = self.explain_sql(sql)
self.check_explain_res_has_row("Last Row Scan", explain_res, sql)
self.check_explain_res_no_row("Table Scan", explain_res, sql)
sql = f'select last_row(ts), last(ts), last_row(id), last(id) from last_test_both_model.test_t1;'
tdSql.query(sql)
tdSql.checkRows(1)
tdSql.checkData(0, 0, last_ts)
tdSql.checkData(0, 1, last_ts)
tdSql.checkData(0, 2, None)
tdSql.checkData(0, 3, maxRange - 1)
explain_res = self.explain_sql(sql)
self.check_explain_res_has_row("Last Row Scan", explain_res, sql)
self.check_explain_res_no_row("Table Scan", explain_res, sql)
sql = f'select last(*), last_row(ts), count(*) from last_test_both_model.test_t1;'
tdSql.query(sql)
tdSql.checkRows(1)
tdSql.checkData(0, 0, last_ts)
tdSql.checkData(0, 1, maxRange - 1)
tdSql.checkData(0, 2, last_ts)
tdSql.checkData(0, 3, maxRange + 1)
explain_res = self.explain_sql(sql)
self.check_explain_res_has_row("Last Row Scan", explain_res, sql)
sql = f'select last_row(ts), last(ts), count(*) , last_row(id), last(id), last(*) from last_test_both_model.test_t1;'
tdSql.query(sql)
tdSql.checkRows(1)
tdSql.checkData(0, 0, last_ts)
tdSql.checkData(0, 1, last_ts)
tdSql.checkData(0, 2, maxRange + 1)
tdSql.checkData(0, 3, None)
tdSql.checkData(0, 4, maxRange - 1)
tdSql.checkData(0, 5, last_ts)
tdSql.checkData(0, 6, maxRange - 1)
tdSql.error(f'select last(*), last_row(ts), ts from last_test_both_model.test_t1;')
startTs2 = startTs + 86400000
for i in range(maxRange):
i = i + 2 * maxRange
insertSqlString = "insert into last_test_both_model.test_t2 values(%d, %d);" % (startTs2 + i, i)
tdSql.execute(insertSqlString)
last_ts2 = startTs2 + maxRange
startTs3 = startTs + 2 * 86400000
for i in range(maxRange):
i = i + 3 * maxRange
insertSqlString = "insert into last_test_both_model.test_t3 values(%d, %d);" % (startTs3 + i, i)
tdSql.execute(insertSqlString)
last_ts3 = startTs3 + 4 * maxRange - 1
startTs4 = startTs + 3 * 86400000
for i in range(maxRange):
i = i + 4 * maxRange
insertSqlString = "insert into last_test_both_model.test_t4 values(%d, %d);" % (startTs4 + i, i)
tdSql.execute(insertSqlString)
last_ts4 = startTs4 + 5 * maxRange - 1
sql = f'select last_row(ts), last(*) from last_test_both_model.st;'
tdSql.query(sql)
tdSql.checkRows(1)
tdSql.checkData(0, 0, last_ts4)
tdSql.checkData(0, 1, last_ts4)
tdSql.checkData(0, 2, 5 * maxRange - 1)
explain_res = self.explain_sql(sql)
self.check_explain_res_has_row("Last Row Scan", explain_res, sql)
self.check_explain_res_no_row("Table Scan", explain_res, sql)
sql = f'select last_row(ts), last(ts), last_row(id), last(id) from last_test_both_model.st;'
tdSql.query(sql)
tdSql.checkRows(1)
tdSql.checkData(0, 0, last_ts4)
tdSql.checkData(0, 1, last_ts4)
tdSql.checkData(0, 2, 5 * maxRange - 1)
tdSql.checkData(0, 3, 5 * maxRange - 1)
explain_res = self.explain_sql(sql)
self.check_explain_res_has_row("Last Row Scan", explain_res, sql)
self.check_explain_res_no_row("Table Scan", explain_res, sql)
sql = f'select last(*), last_row(ts), count(*) from last_test_both_model.st;'
tdSql.query(sql)
tdSql.checkRows(1)
tdSql.checkData(0, 0, last_ts4)
tdSql.checkData(0, 1, 5 * maxRange - 1)
#tdSql.checkData(0, 2, last_ts4)
tdSql.checkData(0, 3, 4 * maxRange + 1)
explain_res = self.explain_sql(sql)
self.check_explain_res_has_row("Last Row Scan", explain_res, sql)
tdSql.error(f'select last(*), last_row(ts), ts from last_test_both_model.st;')
sql = f'select last_row(ts), last(ts), count(*) , last_row(id), last(id), last(*) from last_test_both_model.st;'
tdSql.query(sql)
tdSql.checkRows(1)
tdSql.checkData(0, 0, last_ts4)
tdSql.checkData(0, 1, last_ts4)
tdSql.checkData(0, 2, 4 * maxRange + 1)
tdSql.checkData(0, 3, 5 * maxRange - 1)
tdSql.checkData(0, 4, 5 * maxRange - 1)
tdSql.checkData(0, 5, last_ts4)
tdSql.checkData(0, 6, 5 * maxRange - 1)
sql = f'select last_row(1), last(2), count(*) , last_row(id), last(id), last(*) from last_test_both_model.st;'
tdSql.query(sql)
tdSql.checkRows(1)
tdSql.checkData(0, 0, 1)
tdSql.checkData(0, 1, 2)
tdSql.checkData(0, 2, 4 * maxRange + 1)
tdSql.checkData(0, 3, 5 * maxRange - 1)
tdSql.checkData(0, 4, 5 * maxRange - 1)
tdSql.checkData(0, 5, last_ts4)
tdSql.checkData(0, 6, 5 * maxRange - 1)
tdSql.execute("drop table if exists last_test_both_model.test_t4 ;")
tdSql.execute("drop table if exists last_test_both_model.test_t3 ;")
tdSql.execute("drop table if exists last_test_both_model.test_t2 ;")
tdSql.execute("drop table if exists last_test_both_model.test_t1 ;")
tdSql.execute("drop stable if exists last_test_both_model.st;")
tdSql.execute("drop database if exists last_test_both_model;")
def run(self):
self.none_model_test()
self.last_value_model_test()
self.last_row_model_test()
self.both_model_test()
def stop(self):
tdSql.close()
tdLog.success("%s successfully executed" % __file__)
tdCases.addWindows(__file__, TDTestCase())
tdCases.addLinux(__file__, TDTestCase())

View File

@ -250,7 +250,7 @@ class TDTestCase:
"last_row(c1), last(c1)", "last_row(c1), last(c1)",
"last_row(c1), c1,c3, ts" "last_row(c1), c1,c3, ts"
] ]
has_last_row_scan_res = [0,0,1] has_last_row_scan_res = [1,1,1]
sqls = self.format_sqls(sql_template, select_items) sqls = self.format_sqls(sql_template, select_items)
self.explain_and_check_res(sqls, has_last_row_scan_res) self.explain_and_check_res(sqls, has_last_row_scan_res)
#res_expect = [None, None, [999, 999, 499, "2018-11-25 19:30:00.000"]] #res_expect = [None, None, [999, 999, 499, "2018-11-25 19:30:00.000"]]
@ -387,7 +387,7 @@ class TDTestCase:
tdSql.query('select last(c1) from meters partition by t1') tdSql.query('select last(c1) from meters partition by t1')
print(str(tdSql.queryResult)) print(str(tdSql.queryResult))
tdSql.checkCols(1) tdSql.checkCols(1)
tdSql.checkRows(2) tdSql.checkRows(5)
p = subprocess.run(["taos", '-s', "alter table test.meters drop column c1; alter table test.meters add column c2 int"]) p = subprocess.run(["taos", '-s', "alter table test.meters drop column c1; alter table test.meters add column c2 int"])
p.check_returncode() p.check_returncode()
tdSql.query_success_failed('select last(c1) from meters partition by t1', queryTimes=10, expectErrInfo="Invalid column name: c1") tdSql.query_success_failed('select last(c1) from meters partition by t1', queryTimes=10, expectErrInfo="Invalid column name: c1")

View File

@ -1314,36 +1314,36 @@ class TDTestCase:
#stables #stables
tdSql.query(f"insert into nested.stable_1 (ts,tbname,q_int) values(now,'stable_1_1',1) \ tdSql.query(f"insert into nested.stable_1 (ts,tbname,q_int) values({ts},'stable_1_1',1) \
nested.stable_1 (ts,tbname,q_bigint) values(now+1a,'stable_1_1',1)\ nested.stable_1 (ts,tbname,q_bigint) values({ts}+1a,'stable_1_1',1)\
nested.stable_1 (ts,tbname,q_smallint) values(now+2a,'stable_1_1',1)\ nested.stable_1 (ts,tbname,q_smallint) values({ts}+2a,'stable_1_1',1)\
nested.stable_1 (ts,tbname,q_tinyint) values(now+3a,'stable_1_1',1)\ nested.stable_1 (ts,tbname,q_tinyint) values({ts}+3a,'stable_1_1',1)\
nested.stable_1 (ts,tbname,q_float) values(now+4a,'stable_1_1',1)\ nested.stable_1 (ts,tbname,q_float) values({ts}+4a,'stable_1_1',1)\
nested.stable_1 (ts,tbname,q_double) values(now+5a,'stable_1_1',1)\ nested.stable_1 (ts,tbname,q_double) values({ts}+5a,'stable_1_1',1)\
nested.stable_1 (ts,tbname,q_bool) values(now+6a,'stable_1_1',1)\ nested.stable_1 (ts,tbname,q_bool) values({ts}+6a,'stable_1_1',1)\
nested.stable_1 (ts,tbname,q_binary) values(now+7a,'stable_1_1',1)\ nested.stable_1 (ts,tbname,q_binary) values({ts}+7a,'stable_1_1',1)\
nested.stable_1 (ts,tbname,q_nchar) values(now+8a,'stable_1_1',1)\ nested.stable_1 (ts,tbname,q_nchar) values({ts}+8a,'stable_1_1',1)\
nested.stable_1 (ts,tbname,q_ts) values(now+9a,'stable_1_1',1)\ nested.stable_1 (ts,tbname,q_ts) values({ts}+9a,'stable_1_1',1)\
nested.stable_null_data (ts,tbname,q_int) values(now,'stable_null_data_1',1) \ nested.stable_null_data (ts,tbname,q_int) values({ts},'stable_null_data_1',1) \
nested.stable_null_data (ts,tbname,q_bigint) values(now+1a,'stable_null_data_1',1)\ nested.stable_null_data (ts,tbname,q_bigint) values({ts}+1a,'stable_null_data_1',1)\
nested.stable_null_data (ts,tbname,q_smallint) values(now+2a,'stable_null_data_1',1)\ nested.stable_null_data (ts,tbname,q_smallint) values({ts}+2a,'stable_null_data_1',1)\
nested.stable_null_data (ts,tbname,q_tinyint) values(now+3a,'stable_null_data_1',1)\ nested.stable_null_data (ts,tbname,q_tinyint) values({ts}+3a,'stable_null_data_1',1)\
nested.stable_null_data (ts,tbname,q_float) values(now+4a,'stable_null_data_1',1)\ nested.stable_null_data (ts,tbname,q_float) values({ts}+4a,'stable_null_data_1',1)\
nested.stable_null_data (ts,tbname,q_double) values(now+5a,'stable_null_data_1',1)\ nested.stable_null_data (ts,tbname,q_double) values({ts}+5a,'stable_null_data_1',1)\
nested.stable_null_data (ts,tbname,q_bool) values(now+6a,'stable_null_data_1',1)\ nested.stable_null_data (ts,tbname,q_bool) values({ts}+6a,'stable_null_data_1',1)\
nested.stable_null_data (ts,tbname,q_binary) values(now+7a,'stable_null_data_1',1)\ nested.stable_null_data (ts,tbname,q_binary) values({ts}+7a,'stable_null_data_1',1)\
nested.stable_null_data (ts,tbname,q_nchar) values(now+8a,'stable_null_data_1',1)\ nested.stable_null_data (ts,tbname,q_nchar) values({ts}+8a,'stable_null_data_1',1)\
nested.stable_null_data (ts,tbname,q_ts) values(now+9a,'stable_null_data_1',1)\ nested.stable_null_data (ts,tbname,q_ts) values({ts}+9a,'stable_null_data_1',1)\
nested.stable_null_childtable (ts,tbname,q_int) values(now,'stable_null_childtable_1',1) \ nested.stable_null_childtable (ts,tbname,q_int) values({ts},'stable_null_childtable_1',1) \
nested.stable_null_childtable (ts,tbname,q_bigint) values(now+1a,'stable_null_childtable_1',1)\ nested.stable_null_childtable (ts,tbname,q_bigint) values({ts}+1a,'stable_null_childtable_1',1)\
nested.stable_null_childtable (ts,tbname,q_smallint) values(now+2a,'stable_null_childtable_1',1)\ nested.stable_null_childtable (ts,tbname,q_smallint) values({ts}+2a,'stable_null_childtable_1',1)\
nested.stable_null_childtable (ts,tbname,q_tinyint) values(now+3a,'stable_null_childtable_1',1)\ nested.stable_null_childtable (ts,tbname,q_tinyint) values({ts}+3a,'stable_null_childtable_1',1)\
nested.stable_null_childtable (ts,tbname,q_float) values(now+4a,'stable_null_childtable_1',1)\ nested.stable_null_childtable (ts,tbname,q_float) values({ts}+4a,'stable_null_childtable_1',1)\
nested.stable_null_childtable (ts,tbname,q_double) values(now+5a,'stable_null_childtable_1',1)\ nested.stable_null_childtable (ts,tbname,q_double) values({ts}+5a,'stable_null_childtable_1',1)\
nested.stable_null_childtable (ts,tbname,q_bool) values(now+6a,'stable_null_childtable_1',1)\ nested.stable_null_childtable (ts,tbname,q_bool) values({ts}+6a,'stable_null_childtable_1',1)\
nested.stable_null_childtable (ts,tbname,q_binary) values(now+7a,'stable_null_childtable_1',1)\ nested.stable_null_childtable (ts,tbname,q_binary) values({ts}+7a,'stable_null_childtable_1',1)\
nested.stable_null_childtable (ts,tbname,q_nchar) values(now+8a,'stable_null_childtable_1',1)\ nested.stable_null_childtable (ts,tbname,q_nchar) values({ts}+8a,'stable_null_childtable_1',1)\
nested.stable_null_childtable (ts,tbname,q_ts) values(now+9a,'stable_null_childtable_1',1);") nested.stable_null_childtable (ts,tbname,q_ts) values({ts}+9a,'stable_null_childtable_1',1);")
tdSql.query(f"select tbname,count(*) from nested.stable_1 group by tbname order by tbname;") tdSql.query(f"select tbname,count(*) from nested.stable_1 group by tbname order by tbname;")
tdSql.checkRows(6) tdSql.checkRows(6)
@ -1360,9 +1360,9 @@ class TDTestCase:
#test special character #test special character
tdSql.query(f"insert into nested.stable_1 (ts,tbname,q_int) values(now+10a,'!@!@$$^$',1) \ tdSql.query(f"insert into nested.stable_1 (ts,tbname,q_int) values({ts}+10a,'!@!@$$^$',1) \
nested.stable_null_data (ts,tbname,q_int) values(now+10a,'%^$^&^&',1) \ nested.stable_null_data (ts,tbname,q_int) values({ts}+10a,'%^$^&^&',1) \
nested.stable_null_childtable (ts,tbname,q_int) values(now+10a,'$^%$%^&',1);") nested.stable_null_childtable (ts,tbname,q_int) values({ts}+10a,'$^%$%^&',1);")
tdSql.query(f"select tbname,count(*) from nested.stable_1 group by tbname order by tbname;") tdSql.query(f"select tbname,count(*) from nested.stable_1 group by tbname order by tbname;")
tdSql.checkRows(7) tdSql.checkRows(7)
@ -1477,9 +1477,9 @@ class TDTestCase:
#test stable #test stable
tdSql.error(f"insert into nested.stable_1 (ts,tbname,q_int) values(now,'stable_1',1) \ tdSql.error(f"insert into nested.stable_1 (ts,tbname,q_int) values({ts},'stable_1',1) \
nested.stable_null_data (ts,tbname,q_int) values(now,'stable_null_data',1) \ nested.stable_null_data (ts,tbname,q_int) values({ts},'stable_null_data',1) \
nested.stable_null_childtable (ts,tbname,q_int) values(now,'stable_null_childtable',1);") nested.stable_null_childtable (ts,tbname,q_int) values({ts},'stable_null_childtable',1);")
def stop(self): def stop(self):

View File

@ -276,6 +276,33 @@ class TDTestCase:
sql2 = "select count(*) as a, count(c2) as b, max(c2) as c, min(c2) as d, sum(c2) as e from st;" sql2 = "select count(*) as a, count(c2) as b, max(c2) as c, min(c2) as d, sum(c2) as e from st;"
self.queryResultSame(sql1, sql2) self.queryResultSame(sql1, sql2)
def queryOrderByAgg(self):
tdSql.query("SELECT COUNT(*) FROM t1 order by COUNT(*)")
tdSql.query("SELECT COUNT(*) FROM t1 order by last(c2)")
tdSql.query("SELECT c1 FROM t1 order by last(ts)")
tdSql.query("SELECT ts FROM t1 order by last(ts)")
tdSql.query("SELECT last(ts), ts, c1 FROM t1 order by 2")
tdSql.query("SELECT ts, last(ts) FROM t1 order by last(ts)")
tdSql.query(f"SELECT * FROM t1 order by last(ts)")
tdSql.query(f"SELECT last(ts) as t2, ts FROM t1 order by 1")
tdSql.checkRows(1)
tdSql.query(f"SELECT last(ts), ts FROM t1 order by last(ts)")
tdSql.checkRows(1)
tdSql.error(f"SELECT first(ts), ts FROM t1 order by last(ts)")
tdSql.error(f"SELECT last(ts) as t2, ts FROM t1 order by last(t2)")
# run # run
def run(self): def run(self):
# prepare env # prepare env
@ -287,6 +314,9 @@ class TDTestCase:
# advance # advance
self.queryAdvance() self.queryAdvance()
# agg
self.queryOrderByAgg()
# stop # stop
def stop(self): def stop(self):

View File

@ -2,6 +2,7 @@ from util.log import *
from util.sql import * from util.sql import *
from util.cases import * from util.cases import *
from util.dnodes import * from util.dnodes import *
from util.autogen import *
INT_COL = "c1" INT_COL = "c1"
@ -23,11 +24,11 @@ TS_TYPE_COL = [TS_COL]
DBNAME = "db" DBNAME = "db"
class TDTestCase: class TDTestCase:
def init(self, conn, logSql, replicaVar=1): def init(self, conn, logSql, replicaVar=1):
self.replicaVar = int(replicaVar) self.replicaVar = int(replicaVar)
tdLog.debug(f"start to excute {__file__}") tdLog.debug(f"start to excute {__file__}")
tdSql.init(conn.cursor()) tdSql.init(conn.cursor())
self.autoGen = AutoGen(True)
def __sum_condition(self): def __sum_condition(self):
sum_condition = [] sum_condition = []
@ -207,9 +208,51 @@ class TDTestCase:
''' '''
) )
def testAllTypes(self):
# create stable and insert
tdLog.info("test all types")
dbname = "sumdb"
stbname = "stb"
colnum = 16
self.autoGen.set_batch_size(1000)
self.autoGen.create_db(dbname)
self.autoGen.create_stable(stbname, 16, colnum, 8, 16)
self.autoGen.create_child(stbname, "d", 4)
self.autoGen.insert_data(10000)
# check correct
i = 0
for c in self.autoGen.mcols:
if c in [0, 11, 12, 13]:
i += 1
continue
# query
col = f"c{i}"
sql = f"select count({col}), sum({col}), avg({col}), max({col}), min({col}), stddev({col}), leastsquares({col},1,9) from {dbname}.{stbname}"
tdSql.query(sql)
# sum
tdSql.checkData(0, 0, 4*10000, True)
# sum
tdSql.checkData(0, 1, 4*10000, True)
# avg
tdSql.checkData(0, 2, 1, True)
# max
tdSql.checkData(0, 3, 1, True)
# min
tdSql.checkData(0, 4, 1, True)
# stddev
tdSql.checkData(0, 5, 0, True)
i += 1
def run(self): def run(self):
tdSql.prepare() tdSql.prepare()
self.testAllTypes()
tdLog.printNoPrefix("==========step1:create table") tdLog.printNoPrefix("==========step1:create table")
self.__create_tb() self.__create_tb()