From 42985227929c970e407832588b712ef33c5a8097 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sun, 26 Jan 2025 18:19:37 +0800 Subject: [PATCH 01/70] enh(stream): add checkpoint queue for source tasks. --- include/libs/stream/tstream.h | 1 + source/libs/stream/inc/streamInt.h | 2 + source/libs/stream/src/streamQueue.c | 136 ++++++++++++++++++++++----- 3 files changed, 113 insertions(+), 26 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index b7d560e060..5b418e6e85 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -136,6 +136,7 @@ enum { enum { STREAM_QUEUE__SUCESS = 1, STREAM_QUEUE__FAILED, + STREAM_QUEUE__CHKPTFAILED, STREAM_QUEUE__PROCESSING, }; diff --git a/source/libs/stream/inc/streamInt.h b/source/libs/stream/inc/streamInt.h index d9778a6a05..0e5aaac58c 100644 --- a/source/libs/stream/inc/streamInt.h +++ b/source/libs/stream/inc/streamInt.h @@ -144,6 +144,8 @@ struct SStreamQueue { STaosQall* qall; void* qItem; int8_t status; + STaosQueue* pChkptQueue; + void* qChkptItem; }; struct SStreamQueueItem { diff --git a/source/libs/stream/src/streamQueue.c b/source/libs/stream/src/streamQueue.c index 401aa7530d..2a1332c0c4 100644 --- a/source/libs/stream/src/streamQueue.c +++ b/source/libs/stream/src/streamQueue.c @@ -47,7 +47,9 @@ static void streamQueueCleanup(SStreamQueue* pQueue) { int32_t streamQueueOpen(int64_t cap, SStreamQueue** pQ) { *pQ = NULL; + int32_t code = 0; + int32_t lino = 0; SStreamQueue* pQueue = taosMemoryCalloc(1, sizeof(SStreamQueue)); if (pQueue == NULL) { @@ -55,24 +57,26 @@ int32_t streamQueueOpen(int64_t cap, SStreamQueue** pQ) { } code = taosOpenQueue(&pQueue->pQueue); - if (code) { - taosMemoryFreeClear(pQueue); - return code; - } + TSDB_CHECK_CODE(code, lino, _error); code = taosAllocateQall(&pQueue->qall); - if (code) { - taosCloseQueue(pQueue->pQueue); - taosMemoryFree(pQueue); - return code; - } + TSDB_CHECK_CODE(code, lino, _error); + + code = taosOpenQueue(&pQueue->pChkptQueue); + TSDB_CHECK_CODE(code, lino, _error); pQueue->status = STREAM_QUEUE__SUCESS; + taosSetQueueCapacity(pQueue->pQueue, cap); taosSetQueueMemoryCapacity(pQueue->pQueue, cap * 1024); *pQ = pQueue; return code; + +_error: + streamQueueClose(pQueue, 0); + stError("failed to open stream queue at line:%d, code:%s", lino, tstrerror(code)); + return code; } void streamQueueClose(SStreamQueue* pQueue, int32_t taskId) { @@ -82,6 +86,11 @@ void streamQueueClose(SStreamQueue* pQueue, int32_t taskId) { taosFreeQall(pQueue->qall); taosCloseQueue(pQueue->pQueue); + pQueue->pQueue = NULL; + + taosCloseQueue(pQueue->pChkptQueue); + pQueue->pChkptQueue = NULL; + taosMemoryFree(pQueue); } @@ -94,6 +103,46 @@ void streamQueueNextItem(SStreamQueue* pQueue, SStreamQueueItem** pItem) { } else { pQueue->qItem = NULL; (void) taosGetQitem(pQueue->qall, &pQueue->qItem); + + if (pQueue->qItem == NULL) { + (void) taosReadAllQitems(pQueue->pQueue, pQueue->qall); + (void) taosGetQitem(pQueue->qall, &pQueue->qItem); + } + + *pItem = streamQueueCurItem(pQueue); + } +} + +void streamQueueNextItemInSourceQ(SStreamQueue* pQueue, SStreamQueueItem** pItem, ETaskStatus status) { + *pItem = NULL; + int8_t flag = atomic_exchange_8(&pQueue->status, STREAM_QUEUE__PROCESSING); + + if (flag == STREAM_QUEUE__CHKPTFAILED) { + *pItem = pQueue->qChkptItem; + } else { + pQueue->qChkptItem = NULL; + taosReadQitem(pQueue->pChkptQueue, (void**) &pQueue->qChkptItem); + if (pQueue->qChkptItem != NULL) { + stDebug("read data from checkpoint queue, status:%d", status); + + *pItem = pQueue->qChkptItem; + return; + } + + // if in checkpoint status, not read data from ordinary input q. + if (status == TASK_STATUS__CK) { + stDebug("in checkpoint status, not ready data in normal queue"); + return; + } + } + + // let's try the ordinary input q + if (flag == STREAM_QUEUE__FAILED) { + *pItem = streamQueueCurItem(pQueue); + } else { + pQueue->qItem = NULL; + (void) taosGetQitem(pQueue->qall, &pQueue->qItem); + if (pQueue->qItem == NULL) { (void) taosReadAllQitems(pQueue->pQueue, pQueue->qall); (void) taosGetQitem(pQueue->qall, &pQueue->qItem); @@ -110,6 +159,7 @@ void streamQueueProcessSuccess(SStreamQueue* queue) { } queue->qItem = NULL; + queue->qChkptItem = NULL; atomic_store_8(&queue->status, STREAM_QUEUE__SUCESS); } @@ -121,6 +171,14 @@ void streamQueueProcessFail(SStreamQueue* queue) { atomic_store_8(&queue->status, STREAM_QUEUE__FAILED); } +void streamQueueGetSourceChkptFailed(SStreamQueue* pQueue) { + if (atomic_load_8(&pQueue->status) != STREAM_QUEUE__PROCESSING) { + stError("invalid queue status:%d, expect:%d", atomic_load_8(&pQueue->status), STREAM_QUEUE__PROCESSING); + return; + } + atomic_store_8(&pQueue->status, STREAM_QUEUE__CHKPTFAILED); +} + bool streamQueueIsFull(const SStreamQueue* pQueue) { int32_t numOfItems = streamQueueGetNumOfItems(pQueue); if (numOfItems >= STREAM_TASK_QUEUE_CAPACITY) { @@ -175,8 +233,9 @@ const char* streamQueueItemGetTypeStr(int32_t type) { EExtractDataCode streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueItem** pInput, int32_t* numOfBlocks, int32_t* blockSize) { - const char* id = pTask->id.idStr; - int32_t taskLevel = pTask->info.taskLevel; + const char* id = pTask->id.idStr; + int32_t taskLevel = pTask->info.taskLevel; + SStreamQueue* pQueue = pTask->inputq.queue; *pInput = NULL; *numOfBlocks = 0; @@ -189,13 +248,19 @@ EExtractDataCode streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueIte } while (1) { - if (streamTaskShouldPause(pTask) || streamTaskShouldStop(pTask)) { - stDebug("s-task:%s task should pause, extract input blocks:%d", id, *numOfBlocks); + ETaskStatus status = streamTaskGetStatus(pTask).state; + if (status == TASK_STATUS__PAUSE || status == TASK_STATUS__STOP || status == TASK_STATUS__STOP) { + stDebug("s-task:%s task should pause/stop, extract input blocks:%d", id, *numOfBlocks); return EXEC_CONTINUE; } SStreamQueueItem* qItem = NULL; - streamQueueNextItem(pTask->inputq.queue, (SStreamQueueItem**)&qItem); + if (taskLevel == TASK_LEVEL__SOURCE) { + streamQueueNextItemInSourceQ(pQueue, &qItem, status); + } else { + streamQueueNextItem(pQueue, &qItem); + } + if (qItem == NULL) { // restore the token to bucket if (*numOfBlocks > 0) { @@ -225,14 +290,19 @@ EExtractDataCode streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueIte *numOfBlocks = 1; *pInput = qItem; return EXEC_CONTINUE; - } else { // previous existed blocks needs to be handle, before handle the checkpoint msg block + } else { // previous existed blocks needs to be handled, before handle the checkpoint msg block stDebug("s-task:%s %s msg extracted, handle previous blocks, numOfBlocks:%d", id, p, *numOfBlocks); *blockSize = streamQueueItemGetSize(*pInput); if (taskLevel == TASK_LEVEL__SINK) { streamTaskConsumeQuota(pTask->outputInfo.pTokenBucket, *blockSize); } - streamQueueProcessFail(pTask->inputq.queue); + if ((type == STREAM_INPUT__CHECKPOINT_TRIGGER || type == STREAM_INPUT__CHECKPOINT) && + (taskLevel == TASK_LEVEL__SOURCE)) { + streamQueueGetSourceChkptFailed(pQueue); + } else { + streamQueueProcessFail(pQueue); + } return EXEC_CONTINUE; } } else { @@ -252,7 +322,7 @@ EExtractDataCode streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueIte streamTaskConsumeQuota(pTask->outputInfo.pTokenBucket, *blockSize); } - streamQueueProcessFail(pTask->inputq.queue); + streamQueueProcessFail(pQueue); return EXEC_CONTINUE; } @@ -260,7 +330,7 @@ EExtractDataCode streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueIte } *numOfBlocks += 1; - streamQueueProcessSuccess(pTask->inputq.queue); + streamQueueProcessSuccess(pQueue); if (*numOfBlocks >= MAX_STREAM_EXEC_BATCH_NUM) { stDebug("s-task:%s batch size limit:%d reached, start to process blocks", id, MAX_STREAM_EXEC_BATCH_NUM); @@ -279,6 +349,7 @@ EExtractDataCode streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueIte int32_t streamTaskPutDataIntoInputQ(SStreamTask* pTask, SStreamQueueItem* pItem) { int8_t type = pItem->type; STaosQueue* pQueue = pTask->inputq.queue->pQueue; + int32_t level = pTask->info.taskLevel; int32_t total = streamQueueGetNumOfItems(pTask->inputq.queue) + 1; if (type == STREAM_INPUT__DATA_SUBMIT) { @@ -326,15 +397,28 @@ int32_t streamTaskPutDataIntoInputQ(SStreamTask* pTask, SStreamQueueItem* pItem) stDebug("s-task:%s blockdata enqueue, total in queue:%d, size:%.2fMiB", pTask->id.idStr, total, size); } else if (type == STREAM_INPUT__CHECKPOINT || type == STREAM_INPUT__CHECKPOINT_TRIGGER || type == STREAM_INPUT__TRANS_STATE || type == STREAM_INPUT__DATA_RETRIEVE) { - int32_t code = taosWriteQitem(pQueue, pItem); - if (code != TSDB_CODE_SUCCESS) { - streamFreeQitem(pItem); - return code; - } - double size = SIZE_IN_MiB(taosQueueMemorySize(pQueue)); - stDebug("s-task:%s level:%d %s blockdata enqueue, total in queue:%d, size:%.2fMiB", pTask->id.idStr, - pTask->info.taskLevel, streamQueueItemGetTypeStr(type), total, size); + int32_t code = 0; + if ((type == STREAM_INPUT__CHECKPOINT_TRIGGER || type == STREAM_INPUT__CHECKPOINT) && level == TASK_LEVEL__SOURCE) { + STaosQueue* pChkptQ = pTask->inputq.queue->pChkptQueue; + code = taosWriteQitem(pChkptQ, pItem); + + double size = SIZE_IN_MiB(taosQueueMemorySize(pChkptQ)); + int32_t num = taosQueueItemSize(pChkptQ); + + stDebug("s-task:%s level:%d %s checkpoint enqueue ctrl queue, total in queue:%d, size:%.2fMiB, data queue:%d", + pTask->id.idStr, pTask->info.taskLevel, streamQueueItemGetTypeStr(type), num, size, (total - 1)); + } else { + code = taosWriteQitem(pQueue, pItem); + if (code != TSDB_CODE_SUCCESS) { + streamFreeQitem(pItem); + return code; + } + + double size = SIZE_IN_MiB(taosQueueMemorySize(pQueue)); + stDebug("s-task:%s level:%d %s blockdata enqueue, total in queue:%d, size:%.2fMiB", pTask->id.idStr, + pTask->info.taskLevel, streamQueueItemGetTypeStr(type), total, size); + } } else if (type == STREAM_INPUT__GET_RES) { // use the default memory limit, refactor later. int32_t code = taosWriteQitem(pQueue, pItem); From 189a7f44abbcc4d0752b00d58efaa1686e452fb8 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sun, 26 Jan 2025 21:50:51 +0800 Subject: [PATCH 02/70] refactor(stream): do some internal refactor. --- source/libs/stream/src/streamQueue.c | 52 +++++++++++++++------------- 1 file changed, 28 insertions(+), 24 deletions(-) diff --git a/source/libs/stream/src/streamQueue.c b/source/libs/stream/src/streamQueue.c index 2a1332c0c4..6a9a1ac880 100644 --- a/source/libs/stream/src/streamQueue.c +++ b/source/libs/stream/src/streamQueue.c @@ -119,37 +119,41 @@ void streamQueueNextItemInSourceQ(SStreamQueue* pQueue, SStreamQueueItem** pItem if (flag == STREAM_QUEUE__CHKPTFAILED) { *pItem = pQueue->qChkptItem; - } else { - pQueue->qChkptItem = NULL; - taosReadQitem(pQueue->pChkptQueue, (void**) &pQueue->qChkptItem); - if (pQueue->qChkptItem != NULL) { - stDebug("read data from checkpoint queue, status:%d", status); + ASSERT(status != TASK_STATUS__CK && pQueue->qItem == NULL); + return; + } - *pItem = pQueue->qChkptItem; - return; - } + if (flag == STREAM_QUEUE__FAILED) { + *pItem = pQueue->qItem; + ASSERT(status != TASK_STATUS__CK && pQueue->qChkptItem == NULL); + return; + } - // if in checkpoint status, not read data from ordinary input q. - if (status == TASK_STATUS__CK) { - stDebug("in checkpoint status, not ready data in normal queue"); - return; - } + pQueue->qChkptItem = NULL; + taosReadQitem(pQueue->pChkptQueue, (void**)&pQueue->qChkptItem); + if (pQueue->qChkptItem != NULL) { + stDebug("read data from checkpoint queue, status:%d", status); + + *pItem = pQueue->qChkptItem; + return; + } + + // if in checkpoint status, not read data from ordinary input q. + if (status == TASK_STATUS__CK) { + stDebug("in checkpoint status, not ready data in normal queue"); + return; } // let's try the ordinary input q - if (flag == STREAM_QUEUE__FAILED) { - *pItem = streamQueueCurItem(pQueue); - } else { - pQueue->qItem = NULL; - (void) taosGetQitem(pQueue->qall, &pQueue->qItem); + pQueue->qItem = NULL; + (void)taosGetQitem(pQueue->qall, &pQueue->qItem); - if (pQueue->qItem == NULL) { - (void) taosReadAllQitems(pQueue->pQueue, pQueue->qall); - (void) taosGetQitem(pQueue->qall, &pQueue->qItem); - } - - *pItem = streamQueueCurItem(pQueue); + if (pQueue->qItem == NULL) { + (void)taosReadAllQitems(pQueue->pQueue, pQueue->qall); + (void)taosGetQitem(pQueue->qall, &pQueue->qItem); } + + *pItem = streamQueueCurItem(pQueue); } void streamQueueProcessSuccess(SStreamQueue* queue) { From 256d497fd9425da39ac439acf7461f32c8e4e22c Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sun, 26 Jan 2025 22:57:06 +0800 Subject: [PATCH 03/70] fix(stream): fix error in deciding exec should quit or not. --- source/libs/stream/src/streamExec.c | 28 +++++++++++++++++++++++++--- source/libs/stream/src/streamQueue.c | 9 ++++----- 2 files changed, 29 insertions(+), 8 deletions(-) diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index 4e9e236507..9a991752c9 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -910,8 +910,31 @@ bool streamTaskReadyToRun(const SStreamTask* pTask, char** pStatus) { } } +static bool shouldNotCont(SStreamTask* pTask) { + int32_t level = pTask->info.taskLevel; + SStreamQueue* pQueue = pTask->inputq.queue; + ETaskStatus status = streamTaskGetStatus(pTask).state; + + // 1. task should jump out + bool quit = (status == TASK_STATUS__STOP) || (status == TASK_STATUS__PAUSE) || (status == TASK_STATUS__DROPPING); + + // 2. checkpoint procedure, the source task's checkpoint queue is empty, not read from ordinary queue + bool notCkCont = + (taosQueueItemSize(pQueue->pChkptQueue) == 0) && (level == TASK_LEVEL__SOURCE) && (status == TASK_STATUS__CK); + + // 3. no data in ordinary queue + int32_t numOfItems = streamQueueGetNumOfItems(pQueue); + + if ((numOfItems == 0) || quit || notCkCont) { + return true; + } else { + return false; + } +} + int32_t streamResumeTask(SStreamTask* pTask) { const char* id = pTask->id.idStr; + int32_t level = pTask->info.taskLevel; int32_t code = 0; if (pTask->status.schedStatus != TASK_SCHED_STATUS__ACTIVE) { @@ -924,11 +947,10 @@ int32_t streamResumeTask(SStreamTask* pTask) { if (code) { stError("s-task:%s failed to exec stream task, code:%s, continue", id, tstrerror(code)); } - // check if continue + streamMutexLock(&pTask->lock); - int32_t numOfItems = streamQueueGetNumOfItems(pTask->inputq.queue); - if ((numOfItems == 0) || streamTaskShouldStop(pTask) || streamTaskShouldPause(pTask)) { + if (shouldNotCont(pTask)) { atomic_store_8(&pTask->status.schedStatus, TASK_SCHED_STATUS__INACTIVE); streamTaskClearSchedIdleInfo(pTask); streamMutexUnlock(&pTask->lock); diff --git a/source/libs/stream/src/streamQueue.c b/source/libs/stream/src/streamQueue.c index 6a9a1ac880..3d0d01ce0e 100644 --- a/source/libs/stream/src/streamQueue.c +++ b/source/libs/stream/src/streamQueue.c @@ -113,7 +113,7 @@ void streamQueueNextItem(SStreamQueue* pQueue, SStreamQueueItem** pItem) { } } -void streamQueueNextItemInSourceQ(SStreamQueue* pQueue, SStreamQueueItem** pItem, ETaskStatus status) { +void streamQueueNextItemInSourceQ(SStreamQueue* pQueue, SStreamQueueItem** pItem, ETaskStatus status, const char* id) { *pItem = NULL; int8_t flag = atomic_exchange_8(&pQueue->status, STREAM_QUEUE__PROCESSING); @@ -132,15 +132,14 @@ void streamQueueNextItemInSourceQ(SStreamQueue* pQueue, SStreamQueueItem** pItem pQueue->qChkptItem = NULL; taosReadQitem(pQueue->pChkptQueue, (void**)&pQueue->qChkptItem); if (pQueue->qChkptItem != NULL) { - stDebug("read data from checkpoint queue, status:%d", status); - + stDebug("s-task:%s read data from checkpoint queue, status:%d", id, status); *pItem = pQueue->qChkptItem; return; } // if in checkpoint status, not read data from ordinary input q. if (status == TASK_STATUS__CK) { - stDebug("in checkpoint status, not ready data in normal queue"); + stDebug("s-task:%s in checkpoint status, not read data in block queue, status:%d", id, status); return; } @@ -260,7 +259,7 @@ EExtractDataCode streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueIte SStreamQueueItem* qItem = NULL; if (taskLevel == TASK_LEVEL__SOURCE) { - streamQueueNextItemInSourceQ(pQueue, &qItem, status); + streamQueueNextItemInSourceQ(pQueue, &qItem, status, id); } else { streamQueueNextItem(pQueue, &qItem); } From 55f03f21871f32eb440745dd1c6efc6c12002fd3 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 27 Jan 2025 00:50:36 +0800 Subject: [PATCH 04/70] fix(stream): try starting task even the inputQ is full. --- source/dnode/vnode/src/tq/tqStreamTask.c | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/source/dnode/vnode/src/tq/tqStreamTask.c b/source/dnode/vnode/src/tq/tqStreamTask.c index 9ea84830f1..d98ce58161 100644 --- a/source/dnode/vnode/src/tq/tqStreamTask.c +++ b/source/dnode/vnode/src/tq/tqStreamTask.c @@ -273,9 +273,10 @@ bool taskReadyForDataFromWal(SStreamTask* pTask) { return false; } - // check if input queue is full or not + // check whether input queue is full or not if (streamQueueIsFull(pTask->inputq.queue)) { - tqTrace("s-task:%s input queue is full, do nothing", pTask->id.idStr); + tqTrace("s-task:%s input queue is full, launch task without scanning wal", pTask->id.idStr); + streamTrySchedExec(pTask); return false; } From b27a7f275e54d13a7bd51b5dd37ba323687e473b Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 28 Jan 2025 01:38:16 +0800 Subject: [PATCH 05/70] refactor: update some logs. --- source/libs/stream/src/streamDispatch.c | 2 ++ 1 file changed, 2 insertions(+) diff --git a/source/libs/stream/src/streamDispatch.c b/source/libs/stream/src/streamDispatch.c index b7039d372d..b093f808c0 100644 --- a/source/libs/stream/src/streamDispatch.c +++ b/source/libs/stream/src/streamDispatch.c @@ -1098,6 +1098,8 @@ static void chkptReadyMsgSendMonitorFn(void* param, void* tmrId) { pActiveInfo = pTask->chkInfo.pActiveInfo; pTmrInfo = &pActiveInfo->chkptReadyMsgTmr; + stDebug("s-task:%s acquire task, refId:%" PRId64, id, taskRefId); + // check the status every 100ms if (streamTaskShouldStop(pTask)) { streamCleanBeforeQuitTmr(pTmrInfo, param); From 8636856ef6aa2f09384408c0a0bd712c81cd21ce Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 5 Feb 2025 10:59:29 +0800 Subject: [PATCH 06/70] fix(stream): fix error in the check of continuing execution condition. --- source/libs/stream/src/streamExec.c | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index 9a991752c9..a492ba8070 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -919,16 +919,20 @@ static bool shouldNotCont(SStreamTask* pTask) { bool quit = (status == TASK_STATUS__STOP) || (status == TASK_STATUS__PAUSE) || (status == TASK_STATUS__DROPPING); // 2. checkpoint procedure, the source task's checkpoint queue is empty, not read from ordinary queue - bool notCkCont = - (taosQueueItemSize(pQueue->pChkptQueue) == 0) && (level == TASK_LEVEL__SOURCE) && (status == TASK_STATUS__CK); + bool emptyCkQueue = (taosQueueItemSize(pQueue->pChkptQueue) == 0) && (level == TASK_LEVEL__SOURCE); // 3. no data in ordinary queue - int32_t numOfItems = streamQueueGetNumOfItems(pQueue); + int32_t emptyBlockQueue = (streamQueueGetNumOfItems(pQueue) == 0); - if ((numOfItems == 0) || quit || notCkCont) { + if (quit) { return true; } else { - return false; + if (status == TASK_STATUS__CK) { + // in checkpoint procedure, we only check whether the controller queue is empty or not + return emptyCkQueue; + } else { // otherwise, if the block queue is empty, not continue. + return emptyBlockQueue; + } } } From 1c0a0443c5492b8349c2d3665e7d599aec202276 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 5 Feb 2025 13:35:23 +0800 Subject: [PATCH 07/70] fix(stream): fix error in the check of continuing execution condition. --- source/libs/stream/src/streamExec.c | 8 ++++---- source/libs/stream/src/streamQueue.c | 2 +- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index a492ba8070..b02273a6f6 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -919,19 +919,19 @@ static bool shouldNotCont(SStreamTask* pTask) { bool quit = (status == TASK_STATUS__STOP) || (status == TASK_STATUS__PAUSE) || (status == TASK_STATUS__DROPPING); // 2. checkpoint procedure, the source task's checkpoint queue is empty, not read from ordinary queue - bool emptyCkQueue = (taosQueueItemSize(pQueue->pChkptQueue) == 0) && (level == TASK_LEVEL__SOURCE); + bool emptyCkQueue = (taosQueueItemSize(pQueue->pChkptQueue) == 0); // 3. no data in ordinary queue - int32_t emptyBlockQueue = (streamQueueGetNumOfItems(pQueue) == 0); + bool emptyBlockQueue = (streamQueueGetNumOfItems(pQueue) == 0); if (quit) { return true; } else { - if (status == TASK_STATUS__CK) { + if (status == TASK_STATUS__CK && level == TASK_LEVEL__SOURCE) { // in checkpoint procedure, we only check whether the controller queue is empty or not return emptyCkQueue; } else { // otherwise, if the block queue is empty, not continue. - return emptyBlockQueue; + return emptyBlockQueue && emptyCkQueue; } } } diff --git a/source/libs/stream/src/streamQueue.c b/source/libs/stream/src/streamQueue.c index 3d0d01ce0e..f68dd1452f 100644 --- a/source/libs/stream/src/streamQueue.c +++ b/source/libs/stream/src/streamQueue.c @@ -402,7 +402,7 @@ int32_t streamTaskPutDataIntoInputQ(SStreamTask* pTask, SStreamQueueItem* pItem) type == STREAM_INPUT__TRANS_STATE || type == STREAM_INPUT__DATA_RETRIEVE) { int32_t code = 0; - if ((type == STREAM_INPUT__CHECKPOINT_TRIGGER || type == STREAM_INPUT__CHECKPOINT) && level == TASK_LEVEL__SOURCE) { + if ((type == STREAM_INPUT__CHECKPOINT_TRIGGER || type == STREAM_INPUT__CHECKPOINT) && (level == TASK_LEVEL__SOURCE)) { STaosQueue* pChkptQ = pTask->inputq.queue->pChkptQueue; code = taosWriteQitem(pChkptQ, pItem); From 5f9ce8826da65c3dddcd160e5d9c5a2f79476bd8 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sun, 9 Feb 2025 22:29:41 +0800 Subject: [PATCH 08/70] fix(stream): remove invalid assert. --- source/libs/stream/src/streamQueue.c | 2 -- 1 file changed, 2 deletions(-) diff --git a/source/libs/stream/src/streamQueue.c b/source/libs/stream/src/streamQueue.c index f68dd1452f..c7b0bc8f11 100644 --- a/source/libs/stream/src/streamQueue.c +++ b/source/libs/stream/src/streamQueue.c @@ -119,13 +119,11 @@ void streamQueueNextItemInSourceQ(SStreamQueue* pQueue, SStreamQueueItem** pItem if (flag == STREAM_QUEUE__CHKPTFAILED) { *pItem = pQueue->qChkptItem; - ASSERT(status != TASK_STATUS__CK && pQueue->qItem == NULL); return; } if (flag == STREAM_QUEUE__FAILED) { *pItem = pQueue->qItem; - ASSERT(status != TASK_STATUS__CK && pQueue->qChkptItem == NULL); return; } From 6766e6db5398364ec24801079b4c496433dc1cd1 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 11 Feb 2025 00:57:18 +0800 Subject: [PATCH 09/70] refactor(stream): add checkpoint queue to handle chkpt. --- include/common/tmsgcb.h | 1 + include/common/tmsgdef.h | 1 + include/libs/stream/tstream.h | 4 +- source/dnode/mgmt/mgmt_vnode/inc/vmInt.h | 3 ++ source/dnode/mgmt/mgmt_vnode/src/vmHandle.c | 1 + source/dnode/mgmt/mgmt_vnode/src/vmInt.c | 3 ++ source/dnode/mgmt/mgmt_vnode/src/vmWorker.c | 50 ++++++++++++++++++++- source/dnode/snode/src/snode.c | 2 +- source/dnode/vnode/inc/vnode.h | 1 + source/dnode/vnode/src/tq/tqStreamTask.c | 10 ++--- source/dnode/vnode/src/tqCommon/tqCommon.c | 6 +-- source/dnode/vnode/src/vnd/vnodeSvr.c | 18 ++++++++ source/dnode/vnode/src/vnd/vnodeSync.c | 2 +- source/libs/stream/src/streamCheckStatus.c | 2 +- source/libs/stream/src/streamCheckpoint.c | 2 +- source/libs/stream/src/streamDispatch.c | 4 +- source/libs/stream/src/streamSched.c | 35 ++++++++++----- source/libs/stream/src/streamTask.c | 2 +- 18 files changed, 119 insertions(+), 28 deletions(-) diff --git a/include/common/tmsgcb.h b/include/common/tmsgcb.h index 2847f4278a..3ce03d292f 100644 --- a/include/common/tmsgcb.h +++ b/include/common/tmsgcb.h @@ -39,6 +39,7 @@ typedef enum { ARB_QUEUE, STREAM_CTRL_QUEUE, STREAM_LONG_EXEC_QUEUE, + STREAM_CHKPT_QUEUE, QUEUE_MAX, } EQueueType; diff --git a/include/common/tmsgdef.h b/include/common/tmsgdef.h index 8bdc9a9346..3ab7044e83 100644 --- a/include/common/tmsgdef.h +++ b/include/common/tmsgdef.h @@ -354,6 +354,7 @@ TD_DEF_MSG_TYPE(TDMT_STREAM_DROP, "stream-drop", NULL, NULL) TD_DEF_MSG_TYPE(TDMT_STREAM_RETRIEVE_TRIGGER, "stream-retri-trigger", NULL, NULL) TD_DEF_MSG_TYPE(TDMT_STREAM_CONSEN_CHKPT, "stream-consen-chkpt", NULL, NULL) + TD_DEF_MSG_TYPE(TDMT_STREAM_CHKPT_EXEC, "stream-exec-chkpt", NULL, NULL) TD_CLOSE_MSG_SEG(TDMT_STREAM_MSG) TD_NEW_MSG_SEG(TDMT_MON_MSG) //5 << 8 diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 5b418e6e85..3262130bd4 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -684,8 +684,8 @@ int32_t streamTaskClearHTaskAttr(SStreamTask* pTask, int32_t clearRelHalt); int32_t streamExecTask(SStreamTask* pTask); int32_t streamResumeTask(SStreamTask* pTask); -int32_t streamTrySchedExec(SStreamTask* pTask); -int32_t streamTaskSchedTask(SMsgCb* pMsgCb, int32_t vgId, int64_t streamId, int32_t taskId, int32_t execType); +int32_t streamTrySchedExec(SStreamTask* pTask, bool chkptExec); +int32_t streamTaskSchedTask(SMsgCb* pMsgCb, int32_t vgId, int64_t streamId, int32_t taskId, int32_t execType, bool chkptExec); void streamTaskResumeInFuture(SStreamTask* pTask); void streamTaskClearSchedIdleInfo(SStreamTask* pTask); void streamTaskSetIdleInfo(SStreamTask* pTask, int32_t idleTime); diff --git a/source/dnode/mgmt/mgmt_vnode/inc/vmInt.h b/source/dnode/mgmt/mgmt_vnode/inc/vmInt.h index 9b4c11d6ae..583883590a 100644 --- a/source/dnode/mgmt/mgmt_vnode/inc/vmInt.h +++ b/source/dnode/mgmt/mgmt_vnode/inc/vmInt.h @@ -34,6 +34,7 @@ typedef struct SVnodeMgmt { SAutoQWorkerPool streamPool; SAutoQWorkerPool streamLongExecPool; SWWorkerPool streamCtrlPool; + SWWorkerPool streamChkPool; SWWorkerPool fetchPool; SSingleWorker mgmtWorker; SSingleWorker mgmtMultiWorker; @@ -77,6 +78,7 @@ typedef struct { STaosQueue *pStreamQ; STaosQueue *pStreamCtrlQ; STaosQueue *pStreamLongExecQ; + STaosQueue *pStreamChkQ; STaosQueue *pFetchQ; STaosQueue *pMultiMgmQ; } SVnodeObj; @@ -141,6 +143,7 @@ int32_t vmPutMsgToStreamQueue(SVnodeMgmt *pMgmt, SRpcMsg *pMsg); int32_t vmPutMsgToStreamCtrlQueue(SVnodeMgmt *pMgmt, SRpcMsg *pMsg); int32_t vmPutMsgToStreamLongExecQueue(SVnodeMgmt *pMgmt, SRpcMsg *pMsg); +int32_t vmPutMsgToStreamChkQueue(SVnodeMgmt *pMgmt, SRpcMsg *pMsg); int32_t vmPutMsgToMergeQueue(SVnodeMgmt *pMgmt, SRpcMsg *pMsg); int32_t vmPutMsgToMgmtQueue(SVnodeMgmt *pMgmt, SRpcMsg *pMsg); int32_t vmPutMsgToMultiMgmtQueue(SVnodeMgmt *pMgmt, SRpcMsg *pMsg); diff --git a/source/dnode/mgmt/mgmt_vnode/src/vmHandle.c b/source/dnode/mgmt/mgmt_vnode/src/vmHandle.c index 1dea7d3cad..7ead7f8e8d 100644 --- a/source/dnode/mgmt/mgmt_vnode/src/vmHandle.c +++ b/source/dnode/mgmt/mgmt_vnode/src/vmHandle.c @@ -1019,6 +1019,7 @@ SArray *vmGetMsgHandles() { if (dmSetMgmtHandle(pArray, TDMT_MND_STREAM_CHKPT_REPORT_RSP, vmPutMsgToStreamCtrlQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_VND_STREAM_SCAN_HISTORY, vmPutMsgToStreamLongExecQueue, 0) == NULL) goto _OVER; + if (dmSetMgmtHandle(pArray, TDMT_STREAM_CHKPT_EXEC, vmPutMsgToStreamChkQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_VND_GET_STREAM_PROGRESS, vmPutMsgToStreamQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_STREAM_RETRIEVE, vmPutMsgToStreamQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_STREAM_RETRIEVE_RSP, vmPutMsgToStreamQueue, 0) == NULL) goto _OVER; diff --git a/source/dnode/mgmt/mgmt_vnode/src/vmInt.c b/source/dnode/mgmt/mgmt_vnode/src/vmInt.c index 6f30977e10..e872e1f972 100644 --- a/source/dnode/mgmt/mgmt_vnode/src/vmInt.c +++ b/source/dnode/mgmt/mgmt_vnode/src/vmInt.c @@ -407,6 +407,9 @@ void vmCloseVnode(SVnodeMgmt *pMgmt, SVnodeObj *pVnode, bool commitAndRemoveWal, pVnode->pStreamLongExecQ, taosQueueItemSize(pVnode->pStreamLongExecQ)); while (!taosQueueEmpty(pVnode->pStreamLongExecQ)) taosMsleep(50); + dInfo("vgId:%d, wait for vnode stream chkpt queue:%p is empty", pVnode->vgId, pVnode->pStreamChkQ); + while (!taosQueueEmpty(pVnode->pStreamChkQ)) taosMsleep(10); + dInfo("vgId:%d, all vnode queues is empty", pVnode->vgId); dInfo("vgId:%d, post close", pVnode->vgId); diff --git a/source/dnode/mgmt/mgmt_vnode/src/vmWorker.c b/source/dnode/mgmt/mgmt_vnode/src/vmWorker.c index 5acd06bbda..41f3c64e7d 100644 --- a/source/dnode/mgmt/mgmt_vnode/src/vmWorker.c +++ b/source/dnode/mgmt/mgmt_vnode/src/vmWorker.c @@ -165,6 +165,34 @@ static void vmProcessStreamCtrlQueue(SQueueInfo *pInfo, STaosQall* pQall, int32_ } } +static void vmProcessStreamChkptQueue(SQueueInfo *pInfo, STaosQall* pQall, int32_t numOfItems) { + SVnodeObj *pVnode = pInfo->ahandle; + void *pItem = NULL; + int32_t code = 0; + + while (1) { + if (taosGetQitem(pQall, &pItem) == 0) { + break; + } + + SRpcMsg *pMsg = pItem; + const STraceId *trace = &pMsg->info.traceId; + + dGTrace("vgId:%d, msg:%p get from vnode-stream-chkpt queue", pVnode->vgId, pMsg); + code = vnodeProcessStreamChkptMsg(pVnode->pImpl, pMsg, pInfo); + if (code != 0) { + terrno = code; + dGError("vgId:%d, msg:%p failed to process stream chkpt msg %s since %s", pVnode->vgId, pMsg, + TMSG_INFO(pMsg->msgType), tstrerror(code)); + vmSendRsp(pMsg, code); + } + + dGTrace("vgId:%d, msg:%p is freed, code:0x%x", pVnode->vgId, pMsg, code); + rpcFreeCont(pMsg->pCont); + taosFreeQitem(pMsg); + } +} + static void vmProcessStreamLongExecQueue(SQueueInfo *pInfo, SRpcMsg *pMsg) { SVnodeObj *pVnode = pInfo->ahandle; const STraceId *trace = &pMsg->info.traceId; @@ -301,6 +329,10 @@ static int32_t vmPutMsgToQueue(SVnodeMgmt *pMgmt, SRpcMsg *pMsg, EQueueType qtyp dGTrace("vgId:%d, msg:%p put into vnode-stream-long-exec queue", pVnode->vgId, pMsg); code = taosWriteQitem(pVnode->pStreamLongExecQ, pMsg); break; + case STREAM_CHKPT_QUEUE: + dGTrace("vgId:%d, msg:%p put into vnode-stream-chkpt queue", pVnode->vgId, pMsg); + code = taosWriteQitem(pVnode->pStreamChkQ, pMsg); + break; case FETCH_QUEUE: dGTrace("vgId:%d, msg:%p put into vnode-fetch queue", pVnode->vgId, pMsg); code = taosWriteQitem(pVnode->pFetchQ, pMsg); @@ -361,6 +393,8 @@ int32_t vmPutMsgToStreamCtrlQueue(SVnodeMgmt *pMgmt, SRpcMsg *pMsg) { return vmP int32_t vmPutMsgToStreamLongExecQueue(SVnodeMgmt *pMgmt, SRpcMsg *pMsg) { return vmPutMsgToQueue(pMgmt, pMsg, STREAM_LONG_EXEC_QUEUE); } +int32_t vmPutMsgToStreamChkQueue(SVnodeMgmt *pMgmt, SRpcMsg *pMsg) { return vmPutMsgToQueue(pMgmt, pMsg, STREAM_CHKPT_QUEUE); } + int32_t vmPutMsgToMultiMgmtQueue(SVnodeMgmt *pMgmt, SRpcMsg *pMsg) { const STraceId *trace = &pMsg->info.traceId; dGTrace("msg:%p, put into vnode-multi-mgmt queue", pMsg); @@ -439,6 +473,8 @@ int32_t vmGetQueueSize(SVnodeMgmt *pMgmt, int32_t vgId, EQueueType qtype) { case STREAM_LONG_EXEC_QUEUE: size = taosQueueItemSize(pVnode->pStreamLongExecQ); break; + case STREAM_CHKPT_QUEUE: + size = taosQueueItemSize(pVnode->pStreamChkQ); default: break; } @@ -487,10 +523,11 @@ int32_t vmAllocQueue(SVnodeMgmt *pMgmt, SVnodeObj *pVnode) { pVnode->pStreamQ = tAutoQWorkerAllocQueue(&pMgmt->streamPool, pVnode, (FItem)vmProcessStreamQueue, 2); pVnode->pStreamCtrlQ = tWWorkerAllocQueue(&pMgmt->streamCtrlPool, pVnode, (FItems)vmProcessStreamCtrlQueue); pVnode->pStreamLongExecQ = tAutoQWorkerAllocQueue(&pMgmt->streamLongExecPool, pVnode, (FItem)vmProcessStreamLongExecQueue, 1); + pVnode->pStreamChkQ = tWWorkerAllocQueue(&pMgmt->streamChkPool, pVnode, (FItems)vmProcessStreamChkptQueue); if (pVnode->pWriteW.queue == NULL || pVnode->pSyncW.queue == NULL || pVnode->pSyncRdW.queue == NULL || pVnode->pApplyW.queue == NULL || pVnode->pQueryQ == NULL || pVnode->pStreamQ == NULL || pVnode->pFetchQ == NULL - || pVnode->pStreamCtrlQ == NULL || pVnode->pStreamLongExecQ == NULL) { + || pVnode->pStreamCtrlQ == NULL || pVnode->pStreamLongExecQ == NULL || pVnode->pStreamChkQ == NULL) { return TSDB_CODE_OUT_OF_MEMORY; } @@ -509,6 +546,8 @@ int32_t vmAllocQueue(SVnodeMgmt *pMgmt, SVnodeObj *pVnode) { dInfo("vgId:%d, stream-long-exec-queue:%p is alloced", pVnode->vgId, pVnode->pStreamLongExecQ); dInfo("vgId:%d, stream-ctrl-queue:%p is alloced, thread:%08" PRId64, pVnode->vgId, pVnode->pStreamCtrlQ, taosQueueGetThreadId(pVnode->pStreamCtrlQ)); + dInfo("vgId:%d, stream-chk-queue:%p is alloced, thread:%08" PRId64, pVnode->vgId, pVnode->pStreamChkQ, + taosQueueGetThreadId(pVnode->pStreamChkQ)); return 0; } @@ -517,6 +556,7 @@ void vmFreeQueue(SVnodeMgmt *pMgmt, SVnodeObj *pVnode) { tAutoQWorkerFreeQueue(&pMgmt->streamPool, pVnode->pStreamQ); tAutoQWorkerFreeQueue(&pMgmt->streamLongExecPool, pVnode->pStreamLongExecQ); tWWorkerFreeQueue(&pMgmt->streamCtrlPool, pVnode->pStreamCtrlQ); + tWWorkerFreeQueue(&pMgmt->streamChkPool, pVnode->pStreamChkQ); tWWorkerFreeQueue(&pMgmt->fetchPool, pVnode->pFetchQ); pVnode->pQueryQ = NULL; pVnode->pFetchQ = NULL; @@ -525,6 +565,8 @@ void vmFreeQueue(SVnodeMgmt *pMgmt, SVnodeObj *pVnode) { pVnode->pStreamCtrlQ = NULL; pVnode->pStreamLongExecQ = NULL; + pVnode->pStreamChkQ = NULL; + pVnode->pFetchQ = NULL; dDebug("vgId:%d, queue is freed", pVnode->vgId); } @@ -554,6 +596,11 @@ int32_t vmStartWorker(SVnodeMgmt *pMgmt) { pStreamCtrlPool->max = 1; if ((code = tWWorkerInit(pStreamCtrlPool)) != 0) return code; + SWWorkerPool *pStreamChkPool = &pMgmt->streamChkPool; + pStreamChkPool->name = "vnode-stream-chkpt"; + pStreamChkPool->max = 1; + if ((code = tWWorkerInit(pStreamChkPool)) != 0) return code; + SWWorkerPool *pFPool = &pMgmt->fetchPool; pFPool->name = "vnode-fetch"; pFPool->max = tsNumOfVnodeFetchThreads; @@ -587,6 +634,7 @@ void vmStopWorker(SVnodeMgmt *pMgmt) { tAutoQWorkerCleanup(&pMgmt->streamPool); tAutoQWorkerCleanup(&pMgmt->streamLongExecPool); tWWorkerCleanup(&pMgmt->streamCtrlPool); + tWWorkerCleanup(&pMgmt->streamChkPool); tWWorkerCleanup(&pMgmt->fetchPool); dDebug("vnode workers are closed"); } diff --git a/source/dnode/snode/src/snode.c b/source/dnode/snode/src/snode.c index 6eee8c510b..12609e2c06 100644 --- a/source/dnode/snode/src/snode.c +++ b/source/dnode/snode/src/snode.c @@ -92,7 +92,7 @@ FAIL: } int32_t sndInit(SSnode *pSnode) { - if (streamTaskSchedTask(&pSnode->msgCb, pSnode->pMeta->vgId, 0, 0, STREAM_EXEC_T_START_ALL_TASKS) != 0) { + if (streamTaskSchedTask(&pSnode->msgCb, pSnode->pMeta->vgId, 0, 0, STREAM_EXEC_T_START_ALL_TASKS, false) != 0) { sndError("failed to start all tasks"); } return 0; diff --git a/source/dnode/vnode/inc/vnode.h b/source/dnode/vnode/inc/vnode.h index 871a8c06e1..ddd9878eba 100644 --- a/source/dnode/vnode/inc/vnode.h +++ b/source/dnode/vnode/inc/vnode.h @@ -114,6 +114,7 @@ int32_t vnodeProcessFetchMsg(SVnode *pVnode, SRpcMsg *pMsg, SQueueInfo *pInfo); int32_t vnodeProcessStreamMsg(SVnode *pVnode, SRpcMsg *pMsg, SQueueInfo *pInfo); int32_t vnodeProcessStreamCtrlMsg(SVnode *pVnode, SRpcMsg *pMsg, SQueueInfo *pInfo); int32_t vnodeProcessStreamLongExecMsg(SVnode *pVnode, SRpcMsg *pMsg, SQueueInfo *pInfo); +int32_t vnodeProcessStreamChkptMsg(SVnode *pVnode, SRpcMsg *pMsg, SQueueInfo *pInfo); void vnodeProposeWriteMsg(SQueueInfo *pInfo, STaosQall *qall, int32_t numOfMsgs); void vnodeApplyWriteMsg(SQueueInfo *pInfo, STaosQall *qall, int32_t numOfMsgs); void vnodeProposeCommitOnNeed(SVnode *pVnode, bool atExit); diff --git a/source/dnode/vnode/src/tq/tqStreamTask.c b/source/dnode/vnode/src/tq/tqStreamTask.c index d98ce58161..645dbb2a1a 100644 --- a/source/dnode/vnode/src/tq/tqStreamTask.c +++ b/source/dnode/vnode/src/tq/tqStreamTask.c @@ -109,7 +109,7 @@ static void doStartScanWal(void* param, void* tmrId) { taosMsleep(10000); #endif - code = streamTaskSchedTask(&pParam->msgCb, vgId, 0, 0, STREAM_EXEC_T_EXTRACT_WAL_DATA); + code = streamTaskSchedTask(&pParam->msgCb, vgId, 0, 0, STREAM_EXEC_T_EXTRACT_WAL_DATA, false); if (code) { tqError("vgId:%d failed sched task to scan wal, code:%s", vgId, tstrerror(code)); } @@ -170,7 +170,7 @@ int32_t tqScanWalAsync(STQ* pTq, bool ckPause) { int32_t tqStopStreamTasksAsync(STQ* pTq) { SStreamMeta* pMeta = pTq->pStreamMeta; int32_t vgId = pMeta->vgId; - return streamTaskSchedTask(&pTq->pVnode->msgCb, vgId, 0, 0, STREAM_EXEC_T_STOP_ALL_TASKS); + return streamTaskSchedTask(&pTq->pVnode->msgCb, vgId, 0, 0, STREAM_EXEC_T_STOP_ALL_TASKS, false); } int32_t setWalReaderStartOffset(SStreamTask* pTask, int32_t vgId) { @@ -276,7 +276,7 @@ bool taskReadyForDataFromWal(SStreamTask* pTask) { // check whether input queue is full or not if (streamQueueIsFull(pTask->inputq.queue)) { tqTrace("s-task:%s input queue is full, launch task without scanning wal", pTask->id.idStr); - streamTrySchedExec(pTask); + streamTrySchedExec(pTask, false); return false; } @@ -412,7 +412,7 @@ int32_t doScanWalForAllTasks(SStreamMeta* pStreamMeta) { streamMutexUnlock(&pTask->lock); if ((numOfItems > 0) || hasNewData) { - code = streamTrySchedExec(pTask); + code = streamTrySchedExec(pTask, false); if (code != TSDB_CODE_SUCCESS) { streamMetaReleaseTask(pStreamMeta, pTask); taosArrayDestroy(pTaskList); @@ -465,7 +465,7 @@ int32_t doScanWalAsync(STQ* pTq, bool ckPause) { tqDebug("vgId:%d create msg to start wal scan to launch stream tasks, numOfTasks:%d, vnd restored:%d", vgId, numOfTasks, alreadyRestored); - return streamTaskSchedTask(&pTq->pVnode->msgCb, vgId, 0, 0, STREAM_EXEC_T_EXTRACT_WAL_DATA); + return streamTaskSchedTask(&pTq->pVnode->msgCb, vgId, 0, 0, STREAM_EXEC_T_EXTRACT_WAL_DATA, false); } void streamMetaFreeTQDuringScanWalError(STQ* pTq) { diff --git a/source/dnode/vnode/src/tqCommon/tqCommon.c b/source/dnode/vnode/src/tqCommon/tqCommon.c index 76f0fd6bc6..0b206110b3 100644 --- a/source/dnode/vnode/src/tqCommon/tqCommon.c +++ b/source/dnode/vnode/src/tqCommon/tqCommon.c @@ -123,7 +123,7 @@ int32_t tqStreamTaskStartAsync(SStreamMeta* pMeta, SMsgCb* cb, bool restart) { tqDebug("vgId:%d start all %d stream task(s) async", vgId, numOfTasks); int32_t type = restart ? STREAM_EXEC_T_RESTART_ALL_TASKS : STREAM_EXEC_T_START_ALL_TASKS; - return streamTaskSchedTask(cb, vgId, 0, 0, type); + return streamTaskSchedTask(cb, vgId, 0, 0, type, false); } int32_t tqStreamStartOneTaskAsync(SStreamMeta* pMeta, SMsgCb* cb, int64_t streamId, int32_t taskId) { @@ -135,7 +135,7 @@ int32_t tqStreamStartOneTaskAsync(SStreamMeta* pMeta, SMsgCb* cb, int64_t stream } tqDebug("vgId:%d start task:0x%x async", vgId, taskId); - return streamTaskSchedTask(cb, vgId, streamId, taskId, STREAM_EXEC_T_START_ONE_TASK); + return streamTaskSchedTask(cb, vgId, streamId, taskId, STREAM_EXEC_T_START_ONE_TASK, false); } // this is to process request from transaction, always return true. @@ -1177,7 +1177,7 @@ static int32_t tqProcessTaskResumeImpl(void* handle, SStreamTask* pTask, int64_t } else if (level == TASK_LEVEL__SOURCE && (streamQueueGetNumOfItems(pTask->inputq.queue) == 0)) { code = tqScanWalAsync((STQ*)handle, false); } else { - code = streamTrySchedExec(pTask); + code = streamTrySchedExec(pTask, false); } } diff --git a/source/dnode/vnode/src/vnd/vnodeSvr.c b/source/dnode/vnode/src/vnd/vnodeSvr.c index f37dd94106..aa55885f15 100644 --- a/source/dnode/vnode/src/vnd/vnodeSvr.c +++ b/source/dnode/vnode/src/vnd/vnodeSvr.c @@ -1005,6 +1005,24 @@ int32_t vnodeProcessStreamLongExecMsg(SVnode *pVnode, SRpcMsg *pMsg, SQueueInfo } } +int32_t vnodeProcessStreamChkptMsg(SVnode *pVnode, SRpcMsg *pMsg, SQueueInfo *pInfo) { + vTrace("vgId:%d, msg:%p in stream chkpt queue is processing", pVnode->config.vgId, pMsg); + if ((pMsg->msgType == TDMT_SCH_FETCH || pMsg->msgType == TDMT_VND_TABLE_META || pMsg->msgType == TDMT_VND_TABLE_CFG || + pMsg->msgType == TDMT_VND_BATCH_META) && + !syncIsReadyForRead(pVnode->sync)) { + vnodeRedirectRpcMsg(pVnode, pMsg, terrno); + return 0; + } + + switch (pMsg->msgType) { + case TDMT_STREAM_CHKPT_EXEC: + return tqProcessTaskRunReq(pVnode->pTq, pMsg); + default: + vError("unknown msg type:%d in stream chkpt queue", pMsg->msgType); + return TSDB_CODE_APP_ERROR; + } +} + void smaHandleRes(void *pVnode, int64_t smaId, const SArray *data) { int32_t code = tdProcessTSmaInsert(((SVnode *)pVnode)->pSma, smaId, (const char *)data); if (code) { diff --git a/source/dnode/vnode/src/vnd/vnodeSync.c b/source/dnode/vnode/src/vnd/vnodeSync.c index 068f4dec3d..74ffb69716 100644 --- a/source/dnode/vnode/src/vnd/vnodeSync.c +++ b/source/dnode/vnode/src/vnd/vnodeSync.c @@ -594,7 +594,7 @@ static void vnodeRestoreFinish(const SSyncFSM *pFsm, const SyncIndex commitIdx) streamMetaWUnLock(pMeta); tqInfo("vgId:%d stream task already loaded, start them", vgId); - int32_t code = streamTaskSchedTask(&pVnode->msgCb, TD_VID(pVnode), 0, 0, STREAM_EXEC_T_START_ALL_TASKS); + int32_t code = streamTaskSchedTask(&pVnode->msgCb, TD_VID(pVnode), 0, 0, STREAM_EXEC_T_START_ALL_TASKS, false); if (code != 0) { tqError("vgId:%d failed to sched stream task, code:%s", vgId, tstrerror(code)); } diff --git a/source/libs/stream/src/streamCheckStatus.c b/source/libs/stream/src/streamCheckStatus.c index f880526541..2185df5ec0 100644 --- a/source/libs/stream/src/streamCheckStatus.c +++ b/source/libs/stream/src/streamCheckStatus.c @@ -717,7 +717,7 @@ void handleNotReadyDownstreamTask(SStreamTask* pTask, SArray* pNotReadyList) { // The restart of all tasks requires that all tasks should not have active timer for now. Therefore, the execution // of restart in timer thread will result in a deadlock. int32_t addDownstreamFailedStatusResultAsync(SMsgCb* pMsgCb, int32_t vgId, int64_t streamId, int32_t taskId) { - return streamTaskSchedTask(pMsgCb, vgId, streamId, taskId, STREAM_EXEC_T_ADD_FAILED_TASK); + return streamTaskSchedTask(pMsgCb, vgId, streamId, taskId, STREAM_EXEC_T_ADD_FAILED_TASK, false); } static void doCleanup(SStreamTask* pTask, SArray* pNotReadyList, SArray* pTimeoutList, void* param) { diff --git a/source/libs/stream/src/streamCheckpoint.c b/source/libs/stream/src/streamCheckpoint.c index a43cdd0b85..020a02db20 100644 --- a/source/libs/stream/src/streamCheckpoint.c +++ b/source/libs/stream/src/streamCheckpoint.c @@ -93,7 +93,7 @@ int32_t appendCheckpointIntoInputQ(SStreamTask* pTask, int32_t checkpointType, i return TSDB_CODE_OUT_OF_MEMORY; } - return streamTrySchedExec(pTask); + return streamTrySchedExec(pTask, true); } int32_t streamProcessCheckpointSourceReq(SStreamTask* pTask, SStreamCheckpointSourceReq* pReq) { diff --git a/source/libs/stream/src/streamDispatch.c b/source/libs/stream/src/streamDispatch.c index b093f808c0..6cb68b7604 100644 --- a/source/libs/stream/src/streamDispatch.c +++ b/source/libs/stream/src/streamDispatch.c @@ -1834,6 +1834,7 @@ int32_t streamProcessDispatchMsg(SStreamTask* pTask, SStreamDispatchReq* pReq, S int32_t status = 0; SStreamMeta* pMeta = pTask->pMeta; const char* id = pTask->id.idStr; + bool chkptMsg = false; stDebug("s-task:%s receive dispatch msg from taskId:0x%x(vgId:%d), msgLen:%" PRId64 ", msgId:%d", id, pReq->upstreamTaskId, pReq->upstreamNodeId, pReq->totalLen, pReq->msgId); @@ -1863,6 +1864,7 @@ int32_t streamProcessDispatchMsg(SStreamTask* pTask, SStreamDispatchReq* pReq, S // blocked. Note that there is no race condition here. if (pReq->type == STREAM_INPUT__CHECKPOINT_TRIGGER) { streamTaskCloseUpstreamInput(pTask, pReq->upstreamTaskId); + chkptMsg = true; stDebug("s-task:%s close inputQ for upstream:0x%x, msgId:%d", id, pReq->upstreamTaskId, pReq->msgId); } else if (pReq->type == STREAM_INPUT__TRANS_STATE) { stDebug("s-task:%s recv trans-state msgId:%d from upstream:0x%x", id, pReq->msgId, pReq->upstreamTaskId); @@ -1890,5 +1892,5 @@ int32_t streamProcessDispatchMsg(SStreamTask* pTask, SStreamDispatchReq* pReq, S tmsgSendRsp(pRsp); } - return streamTrySchedExec(pTask); + return streamTrySchedExec(pTask, chkptMsg); } diff --git a/source/libs/stream/src/streamSched.c b/source/libs/stream/src/streamSched.c index 1f76f349ae..7d3d5dcc49 100644 --- a/source/libs/stream/src/streamSched.c +++ b/source/libs/stream/src/streamSched.c @@ -93,9 +93,9 @@ void streamSetupScheduleTrigger(SStreamTask* pTask) { } } -int32_t streamTrySchedExec(SStreamTask* pTask) { +int32_t streamTrySchedExec(SStreamTask* pTask, bool chkptQueue) { if (streamTaskSetSchedStatusWait(pTask)) { - return streamTaskSchedTask(pTask->pMsgCb, pTask->info.nodeId, pTask->id.streamId, pTask->id.taskId, 0); + return streamTaskSchedTask(pTask->pMsgCb, pTask->info.nodeId, pTask->id.streamId, pTask->id.taskId, 0, chkptQueue); } else { stTrace("s-task:%s not launch task since sched status:%d", pTask->id.idStr, pTask->status.schedStatus); } @@ -103,7 +103,7 @@ int32_t streamTrySchedExec(SStreamTask* pTask) { return 0; } -int32_t streamTaskSchedTask(SMsgCb* pMsgCb, int32_t vgId, int64_t streamId, int32_t taskId, int32_t execType) { +int32_t streamTaskSchedTask(SMsgCb* pMsgCb, int32_t vgId, int64_t streamId, int32_t taskId, int32_t execType, bool chkptExec) { int32_t code = 0; int32_t tlen = 0; @@ -142,10 +142,18 @@ int32_t streamTaskSchedTask(SMsgCb* pMsgCb, int32_t vgId, int64_t streamId, int3 stDebug("vgId:%d create msg to exec, type:%d, %s", vgId, execType, streamTaskGetExecType(execType)); } - SRpcMsg msg = {.msgType = TDMT_STREAM_TASK_RUN, .pCont = buf, .contLen = tlen + sizeof(SMsgHead)}; - code = tmsgPutToQueue(pMsgCb, STREAM_QUEUE, &msg); - if (code) { - stError("vgId:%d failed to put msg into stream queue, code:%s, %x", vgId, tstrerror(code), taskId); + if (chkptExec) { + SRpcMsg msg = {.msgType = TDMT_STREAM_CHKPT_EXEC, .pCont = buf, .contLen = tlen + sizeof(SMsgHead)}; + code = tmsgPutToQueue(pMsgCb, STREAM_CHKPT_QUEUE, &msg); + if (code) { + stError("vgId:%d failed to put msg into stream chkpt queue, code:%s, %x", vgId, tstrerror(code), taskId); + } + } else { + SRpcMsg msg = {.msgType = TDMT_STREAM_TASK_RUN, .pCont = buf, .contLen = tlen + sizeof(SMsgHead)}; + code = tmsgPutToQueue(pMsgCb, STREAM_QUEUE, &msg); + if (code) { + stError("vgId:%d failed to put msg into stream queue, code:%s, %x", vgId, tstrerror(code), taskId); + } } return code; } @@ -191,12 +199,17 @@ void streamTaskResumeHelper(void* param, void* tmrId) { return; } - code = streamTaskSchedTask(pTask->pMsgCb, pTask->info.nodeId, pId->streamId, pId->taskId, STREAM_EXEC_T_RESUME_TASK); + code = streamTaskSchedTask(pTask->pMsgCb, pTask->info.nodeId, pId->streamId, pId->taskId, STREAM_EXEC_T_RESUME_TASK, + (p.state == TASK_STATUS__CK)); if (code) { stError("s-task:%s sched task failed, code:%s", pId->idStr, tstrerror(code)); } else { - stDebug("trigger to resume s-task:%s after idled for %dms", pId->idStr, pTask->status.schedIdleTime); - + if (p.state == TASK_STATUS__CK) { + stDebug("trigger to resume s-task:%s in stream chkpt queue after idled for %dms", pId->idStr, + pTask->status.schedIdleTime); + } else { + stDebug("trigger to resume s-task:%s after idled for %dms", pId->idStr, pTask->status.schedIdleTime); + } // release the task ref count streamTaskClearSchedIdleInfo(pTask); } @@ -339,7 +352,7 @@ void streamTaskSchedHelper(void* param, void* tmrId) { } } - code = streamTrySchedExec(pTask); + code = streamTrySchedExec(pTask, false); if (code != TSDB_CODE_SUCCESS) { stError("s-task:%s failed to sched to run, wait for next time", pTask->id.idStr); } diff --git a/source/libs/stream/src/streamTask.c b/source/libs/stream/src/streamTask.c index e4e8a37b37..edada9a050 100644 --- a/source/libs/stream/src/streamTask.c +++ b/source/libs/stream/src/streamTask.c @@ -1185,7 +1185,7 @@ int32_t streamProcessRetrieveReq(SStreamTask* pTask, SStreamRetrieveReq* pReq) { if (code != 0) { return code; } - return streamTrySchedExec(pTask); + return streamTrySchedExec(pTask, false); } void streamTaskSetRemoveBackendFiles(SStreamTask* pTask) { pTask->status.removeBackendFiles = true; } From cc62216c14c9771aead13263a3b6203d1abc1995 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 11 Feb 2025 13:45:58 +0800 Subject: [PATCH 10/70] refactor(stream): add some logs. --- source/libs/stream/src/streamSched.c | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/source/libs/stream/src/streamSched.c b/source/libs/stream/src/streamSched.c index 7d3d5dcc49..367e54d1a1 100644 --- a/source/libs/stream/src/streamSched.c +++ b/source/libs/stream/src/streamSched.c @@ -97,7 +97,11 @@ int32_t streamTrySchedExec(SStreamTask* pTask, bool chkptQueue) { if (streamTaskSetSchedStatusWait(pTask)) { return streamTaskSchedTask(pTask->pMsgCb, pTask->info.nodeId, pTask->id.streamId, pTask->id.taskId, 0, chkptQueue); } else { - stTrace("s-task:%s not launch task since sched status:%d", pTask->id.idStr, pTask->status.schedStatus); + if (chkptQueue) { + stWarn("s-task:%s not launch task in chkpt queue, may delay checkpoint procedure", pTask->id.idStr); + } else { + stTrace("s-task:%s not launch task since sched status:%d", pTask->id.idStr, pTask->status.schedStatus); + } } return 0; From 0600fcceeb4fd79b482bba99fd1c61d622a1c320 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 11 Feb 2025 21:47:39 +0800 Subject: [PATCH 11/70] refactor(stream): idle for 1s when exec for 5s. --- source/libs/stream/src/streamExec.c | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index b02273a6f6..8d0909b9b9 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -778,6 +778,7 @@ static int32_t doStreamExecTask(SStreamTask* pTask) { // merge multiple input data if possible in the input queue. stDebug("s-task:%s start to extract data block from inputQ", id); + int64_t st = taosGetTimestampMs(); while (1) { int32_t blockSize = 0; @@ -807,7 +808,7 @@ static int32_t doStreamExecTask(SStreamTask* pTask) { return 0; } - int64_t st = taosGetTimestampMs(); + EExtractDataCode ret = streamTaskGetDataFromInputQ(pTask, &pInput, &numOfBlocks, &blockSize); if (ret == EXEC_AFTER_IDLE) { From 3f70661dcf4d500114048cde9277d4a06cc5fa32 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 20 Feb 2025 17:39:46 +0800 Subject: [PATCH 12/70] refactor(stream): disable error injection. --- source/libs/stream/src/streamExec.c | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index edde499a50..54dce4947a 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -824,10 +824,10 @@ static int32_t doStreamExecTask(SStreamTask* pTask) { // dispatch checkpoint msg to all downstream tasks int32_t type = pInput->type; if (type == STREAM_INPUT__CHECKPOINT_TRIGGER) { - +#if 0 // Injection error: for automatic kill long trans test taosMsleep(50*1000); - +#endif code = streamProcessCheckpointTriggerBlock(pTask, (SStreamDataBlock*)pInput); if (code != 0) { stError("s-task:%s failed to process checkpoint-trigger block, code:%s", pTask->id.idStr, tstrerror(code)); From 45e027d612b22a7150ab3f194b5e390d646638cc Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Fri, 21 Feb 2025 10:00:55 +0800 Subject: [PATCH 13/70] fix(stream): update the checkpoint info for follower anyway. --- source/libs/executor/src/executor.c | 2 +- source/libs/stream/src/streamCheckpoint.c | 107 ++++++++++++++-------- 2 files changed, 69 insertions(+), 40 deletions(-) diff --git a/source/libs/executor/src/executor.c b/source/libs/executor/src/executor.c index 1aa114a02a..b6044eaacf 100644 --- a/source/libs/executor/src/executor.c +++ b/source/libs/executor/src/executor.c @@ -608,7 +608,7 @@ void qUpdateOperatorParam(qTaskInfo_t tinfo, void* pParam) { } int32_t qExecutorInit(void) { - taosThreadOnce(&initPoolOnce, initRefPool); + (void) taosThreadOnce(&initPoolOnce, initRefPool); return TSDB_CODE_SUCCESS; } diff --git a/source/libs/stream/src/streamCheckpoint.c b/source/libs/stream/src/streamCheckpoint.c index 020a02db20..0a43baced8 100644 --- a/source/libs/stream/src/streamCheckpoint.c +++ b/source/libs/stream/src/streamCheckpoint.c @@ -595,68 +595,71 @@ void streamTaskClearCheckInfo(SStreamTask* pTask, bool clearChkpReadyMsg) { pTask->id.idStr, pInfo->failedId, pTask->chkInfo.checkpointId); } -int32_t streamTaskUpdateTaskCheckpointInfo(SStreamTask* pTask, bool restored, SVUpdateCheckpointInfoReq* pReq) { +// The checkpointInfo can be updated in the following three cases: +// 1. follower tasks; 2. leader task with status of TASK_STATUS__CK; 3. restore not completed +static int32_t doUpdateCheckpointInfoCheck(SStreamTask* pTask, bool restored, SVUpdateCheckpointInfoReq* pReq, + bool* pContinue) { SStreamMeta* pMeta = pTask->pMeta; int32_t vgId = pMeta->vgId; int32_t code = 0; const char* id = pTask->id.idStr; SCheckpointInfo* pInfo = &pTask->chkInfo; - streamMutexLock(&pTask->lock); + *pContinue = true; // not update the checkpoint info if the checkpointId is less than the failed checkpointId if (pReq->checkpointId < pInfo->pActiveInfo->failedId) { stWarn("s-task:%s vgId:%d not update the checkpoint-info, since update checkpointId:%" PRId64 - " is less than the failed checkpointId:%" PRId64 ", discard the update info", + " is less than the failed checkpointId:%" PRId64 ", discard", id, vgId, pReq->checkpointId, pInfo->pActiveInfo->failedId); - streamMutexUnlock(&pTask->lock); - // always return true + *pContinue = false; return TSDB_CODE_SUCCESS; } + // it's an expired checkpointInfo update msg, we still try to drop the required drop fill-history task. if (pReq->checkpointId <= pInfo->checkpointId) { stDebug("s-task:%s vgId:%d latest checkpointId:%" PRId64 " Ver:%" PRId64 " no need to update checkpoint info, updated checkpointId:%" PRId64 " Ver:%" PRId64 " transId:%d ignored", id, vgId, pInfo->checkpointId, pInfo->checkpointVer, pReq->checkpointId, pReq->checkpointVer, pReq->transId); - streamMutexUnlock(&pTask->lock); - { // destroy the related fill-history tasks - // drop task should not in the meta-lock, and drop the related fill-history task now - if (pReq->dropRelHTask) { - code = streamMetaUnregisterTask(pMeta, pReq->hStreamId, pReq->hTaskId); - int32_t numOfTasks = streamMetaGetNumOfTasks(pMeta); - stDebug("s-task:%s vgId:%d related fill-history task:0x%x dropped in update checkpointInfo, remain tasks:%d", - id, vgId, pReq->taskId, numOfTasks); - } + { // destroy the related fill-history tasks + if (pReq->dropRelHTask) { + code = streamMetaUnregisterTask(pMeta, pReq->hStreamId, pReq->hTaskId); - if (pReq->dropRelHTask) { - code = streamMetaCommit(pMeta); - } - } + int32_t numOfTasks = streamMetaGetNumOfTasks(pMeta); + stDebug("s-task:%s vgId:%d related fill-history task:0x%x dropped in update checkpointInfo, remain tasks:%d", + id, vgId, pReq->taskId, numOfTasks); + //todo: task may not exist, commit anyway, optimize this later + code = streamMetaCommit(pMeta); + } + } + + *pContinue = false; // always return true return TSDB_CODE_SUCCESS; } - SStreamTaskState pStatus = streamTaskGetStatus(pTask); + SStreamTaskState status = streamTaskGetStatus(pTask); if (!restored) { // during restore procedure, do update checkpoint-info stDebug("s-task:%s vgId:%d status:%s update the checkpoint-info during restore, checkpointId:%" PRId64 "->%" PRId64 " checkpointVer:%" PRId64 "->%" PRId64 " checkpointTs:%" PRId64 "->%" PRId64, - id, vgId, pStatus.name, pInfo->checkpointId, pReq->checkpointId, pInfo->checkpointVer, pReq->checkpointVer, + id, vgId, status.name, pInfo->checkpointId, pReq->checkpointId, pInfo->checkpointVer, pReq->checkpointVer, pInfo->checkpointTime, pReq->checkpointTs); } else { // not in restore status, must be in checkpoint status - if ((pStatus.state == TASK_STATUS__CK) || (pMeta->role == NODE_ROLE_FOLLOWER)) { - stDebug("s-task:%s vgId:%d status:%s role:%d start to update the checkpoint-info, checkpointId:%" PRId64 "->%" PRId64 - " checkpointVer:%" PRId64 "->%" PRId64 " checkpointTs:%" PRId64 "->%" PRId64, - id, vgId, pStatus.name, pMeta->role, pInfo->checkpointId, pReq->checkpointId, pInfo->checkpointVer, + if (((status.state == TASK_STATUS__CK) && (pMeta->role == NODE_ROLE_LEADER)) || + (pMeta->role == NODE_ROLE_FOLLOWER)) { + stDebug("s-task:%s vgId:%d status:%s role:%d start to update the checkpoint-info, checkpointId:%" PRId64 + "->%" PRId64 " checkpointVer:%" PRId64 "->%" PRId64 " checkpointTs:%" PRId64 "->%" PRId64, + id, vgId, status.name, pMeta->role, pInfo->checkpointId, pReq->checkpointId, pInfo->checkpointVer, pReq->checkpointVer, pInfo->checkpointTime, pReq->checkpointTs); } else { stDebug("s-task:%s vgId:%d status:%s NOT update the checkpoint-info, checkpointId:%" PRId64 "->%" PRId64 " checkpointVer:%" PRId64 "->%" PRId64, - id, vgId, pStatus.name, pInfo->checkpointId, pReq->checkpointId, pInfo->checkpointVer, + id, vgId, status.name, pInfo->checkpointId, pReq->checkpointId, pInfo->checkpointVer, pReq->checkpointVer); } } @@ -665,14 +668,48 @@ int32_t streamTaskUpdateTaskCheckpointInfo(SStreamTask* pTask, bool restored, SV pInfo->processedVer <= pReq->checkpointVer); if (!valid) { - stFatal("s-task:%s invalid checkpointId update info recv, current checkpointId:%" PRId64 " checkpointVer:%" PRId64 - " processedVer:%" PRId64 " req checkpointId:%" PRId64 " checkpointVer:%" PRId64 " discard it", - id, pInfo->checkpointId, pInfo->checkpointVer, pInfo->processedVer, pReq->checkpointId, - pReq->checkpointVer); - streamMutexUnlock(&pTask->lock); - return TSDB_CODE_STREAM_INTERNAL_ERROR; + // invalid update checkpoint info for leader, since the processedVer is greater than the checkpointVer + // It is possible for follower tasks that the processedVer is greater than the checkpointVer, and the processed info + // in follower tasks will be discarded, since the leader/follower switch happens before the checkpoint of the + // processedVer being generated. + if (pMeta->role == NODE_ROLE_LEADER) { + + stFatal("s-task:%s checkpointId update info recv, current checkpointId:%" PRId64 " checkpointVer:%" PRId64 + " processedVer:%" PRId64 " req checkpointId:%" PRId64 " checkpointVer:%" PRId64 " discard it", + id, pInfo->checkpointId, pInfo->checkpointVer, pInfo->processedVer, pReq->checkpointId, + pReq->checkpointVer); + + *pContinue = false; + return TSDB_CODE_STREAM_INTERNAL_ERROR; + } else { + stInfo("s-task:%s vgId:%d follower recv checkpointId update info, current checkpointId:%" PRId64 + " checkpointVer:%" PRId64 " processedVer:%" PRId64 " req checkpointId:%" PRId64 " checkpointVer:%" PRId64, + id, pMeta->vgId, pInfo->checkpointId, pInfo->checkpointVer, pInfo->processedVer, pReq->checkpointId, + pReq->checkpointVer); + } } + return TSDB_CODE_SUCCESS; +} + +int32_t streamTaskUpdateTaskCheckpointInfo(SStreamTask* pTask, bool restored, SVUpdateCheckpointInfoReq* pReq) { + SStreamMeta* pMeta = pTask->pMeta; + int32_t vgId = pMeta->vgId; + int32_t code = 0; + const char* id = pTask->id.idStr; + SCheckpointInfo* pInfo = &pTask->chkInfo; + bool continueUpdate = true; + + streamMutexLock(&pTask->lock); + code = doUpdateCheckpointInfoCheck(pTask, restored, pReq, &continueUpdate); + + if (!continueUpdate) { + streamMutexUnlock(&pTask->lock); + return code; + } + + SStreamTaskState pStatus = streamTaskGetStatus(pTask); + // update only it is in checkpoint status, or during restore procedure. if ((pStatus.state == TASK_STATUS__CK) || (!restored) || (pMeta->role == NODE_ROLE_FOLLOWER)) { pInfo->checkpointId = pReq->checkpointId; @@ -1537,14 +1574,6 @@ int32_t deleteCheckpointFile(const char* id, const char* name) { int32_t streamTaskSendNegotiateChkptIdMsg(SStreamTask* pTask) { streamMutexLock(&pTask->lock); ETaskStatus p = streamTaskGetStatus(pTask).state; - // if (pInfo->alreadySendChkptId == true) { - // stDebug("s-task:%s already start to consensus-checkpointId, not start again before it completed", id); - // streamMutexUnlock(&pTask->lock); - // return TSDB_CODE_SUCCESS; - // } else { - // pInfo->alreadySendChkptId = true; - // } - // streamTaskSetReqConsenChkptId(pTask, taosGetTimestampMs()); streamMutexUnlock(&pTask->lock); From 0f89421b62b04432c875ec0e5f05cb5de9819925 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sat, 22 Feb 2025 12:33:32 +0800 Subject: [PATCH 14/70] refactor(stream): scan wal is driven by time, instead of insert events. --- source/dnode/vnode/src/inc/vnodeInt.h | 2 +- source/dnode/vnode/src/tq/tq.c | 24 +- source/dnode/vnode/src/tq/tqPush.c | 14 -- source/dnode/vnode/src/tq/tqStreamTask.c | 242 +++++++++++---------- source/dnode/vnode/src/tq/tqUtil.c | 4 + source/dnode/vnode/src/tqCommon/tqCommon.c | 10 +- source/libs/stream/src/streamHb.c | 2 +- source/libs/stream/src/streamMeta.c | 2 +- 8 files changed, 156 insertions(+), 144 deletions(-) diff --git a/source/dnode/vnode/src/inc/vnodeInt.h b/source/dnode/vnode/src/inc/vnodeInt.h index 5bf0a9b199..a76fc17377 100644 --- a/source/dnode/vnode/src/inc/vnodeInt.h +++ b/source/dnode/vnode/src/inc/vnodeInt.h @@ -239,7 +239,7 @@ void tqClose(STQ*); int tqPushMsg(STQ*, tmsg_t msgType); int tqRegisterPushHandle(STQ* pTq, void* handle, SRpcMsg* pMsg); void tqUnregisterPushHandle(STQ* pTq, void* pHandle); -int tqScanWalAsync(STQ* pTq, bool ckPause); +int tqScanWalAsync(STQ* pTq); int32_t tqStopStreamTasksAsync(STQ* pTq); int32_t tqProcessTaskCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg* pRsp); int32_t tqProcessTaskCheckpointReadyMsg(STQ* pTq, SRpcMsg* pMsg); diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 119edb47bc..fc173eb691 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -920,12 +920,12 @@ static void doStartFillhistoryStep2(SStreamTask* pTask, SStreamTask* pStreamTask // now the fill-history task starts to scan data from wal files. code = streamTaskHandleEvent(pTask->status.pSM, TASK_EVENT_SCANHIST_DONE); - if (code == TSDB_CODE_SUCCESS) { - code = tqScanWalAsync(pTq, false); - if (code) { - tqError("vgId:%d failed to start scan wal file, code:%s", vgId, tstrerror(code)); - } - } +// if (code == TSDB_CODE_SUCCESS) { +// code = tqScanWalAsync(pTq, false); +// if (code) { +// tqError("vgId:%d failed to start scan wal file, code:%s", vgId, tstrerror(code)); +// } +// } } } @@ -1122,12 +1122,12 @@ int32_t tqProcessTaskRunReq(STQ* pTq, SRpcMsg* pMsg) { } // let's continue scan data in the wal files - if (req.reqType >= 0 || req.reqType == STREAM_EXEC_T_RESUME_TASK) { - code = tqScanWalAsync(pTq, false); // it's ok to failed - if (code) { - tqError("vgId:%d failed to start scan wal file, code:%s", pTq->pStreamMeta->vgId, tstrerror(code)); - } - } +// if (req.reqType >= 0 || req.reqType == STREAM_EXEC_T_RESUME_TASK) { +// code = tqScanWalAsync(pTq, false); // it's ok to failed +// if (code) { +// tqError("vgId:%d failed to start scan wal file, code:%s", pTq->pStreamMeta->vgId, tstrerror(code)); +// } +// } return code; } diff --git a/source/dnode/vnode/src/tq/tqPush.c b/source/dnode/vnode/src/tq/tqPush.c index 2b2667773a..fc83343c99 100644 --- a/source/dnode/vnode/src/tq/tqPush.c +++ b/source/dnode/vnode/src/tq/tqPush.c @@ -49,20 +49,6 @@ int32_t tqPushMsg(STQ* pTq, tmsg_t msgType) { } } - streamMetaRLock(pTq->pStreamMeta); - int32_t numOfTasks = streamMetaGetNumOfTasks(pTq->pStreamMeta); - streamMetaRUnLock(pTq->pStreamMeta); - -// tqTrace("vgId:%d handle submit, restore:%d, numOfTasks:%d", TD_VID(pTq->pVnode), pTq->pVnode->restored, numOfTasks); - - // push data for stream processing: - // 1. the vnode has already been restored. - // 2. the vnode should be the leader. - // 3. the stream is not suspended yet. - if ((!tsDisableStream) && (numOfTasks > 0)) { - code = tqScanWalAsync(pTq, true); - } - return code; } diff --git a/source/dnode/vnode/src/tq/tqStreamTask.c b/source/dnode/vnode/src/tq/tqStreamTask.c index 645dbb2a1a..f6f1010a67 100644 --- a/source/dnode/vnode/src/tq/tqStreamTask.c +++ b/source/dnode/vnode/src/tq/tqStreamTask.c @@ -17,22 +17,20 @@ #include "vnd.h" #define MAX_REPEAT_SCAN_THRESHOLD 3 -#define SCAN_WAL_IDLE_DURATION 500 // idle for 500ms to do next wal scan +#define SCAN_WAL_IDLE_DURATION 500 // idle for 500ms to do next wal scan typedef struct SBuildScanWalMsgParam { int64_t metaId; - int32_t numOfTasks; - int8_t restored; +// int8_t restored; SMsgCb msgCb; } SBuildScanWalMsgParam; -static int32_t doScanWalForAllTasks(SStreamMeta* pStreamMeta); +static int32_t doScanWalForAllTasks(SStreamMeta* pStreamMeta, int32_t* pNumOfTasks); static int32_t setWalReaderStartOffset(SStreamTask* pTask, int32_t vgId); static bool handleFillhistoryScanComplete(SStreamTask* pTask, int64_t ver); static bool taskReadyForDataFromWal(SStreamTask* pTask); static int32_t doPutDataIntoInputQ(SStreamTask* pTask, int64_t maxVer, int32_t* numOfItems, bool* pSucc); -static int32_t tqScanWalInFuture(STQ* pTq, int32_t numOfTasks, int32_t idleDuration); -static int32_t doScanWalAsync(STQ* pTq, bool ckPause); +static int32_t tqScanWalInFuture(STQ* pTq); // extract data blocks(submit/delete) from WAL, and add them into the input queue for all the sources tasks. int32_t tqScanWal(STQ* pTq) { @@ -40,44 +38,38 @@ int32_t tqScanWal(STQ* pTq) { int32_t vgId = pMeta->vgId; int64_t st = taosGetTimestampMs(); int32_t numOfTasks = 0; + int64_t el = 0; + int32_t code = 0; - tqDebug("vgId:%d continue to check if data in wal are available, scanCounter:%d", vgId, pMeta->scanInfo.scanCounter); - - // check all tasks - int32_t code = doScanWalForAllTasks(pMeta); - if (code) { - tqError("vgId:%d failed to start all tasks, try next time, code:%s", vgId, tstrerror(code)); + int32_t old = atomic_val_compare_exchange_32(&pMeta->scanInfo.scanCounter, 0, 1); + if (old == 0) { + tqDebug("vgId:%d try to scan wal to extract data", vgId); + } else { + tqDebug("vgId:%d already in wal scan, abort", vgId); return code; } - streamMetaWLock(pMeta); - int32_t times = (--pMeta->scanInfo.scanCounter); - if (times < 0) { - tqError("vgId:%d invalid scan counter:%d, reset to 0", vgId, times); - times = 0; - } - - numOfTasks = taosArrayGetSize(pMeta->pTaskList); - streamMetaWUnLock(pMeta); - - int64_t el = (taosGetTimestampMs() - st); - tqDebug("vgId:%d scan wal for stream tasks completed, elapsed time:%" PRId64 " ms", vgId, el); - - if (times > 0) { - tqDebug("vgId:%d scan wal for stream tasks for %d times in %dms", vgId, times, SCAN_WAL_IDLE_DURATION); - code = tqScanWalInFuture(pTq, numOfTasks, SCAN_WAL_IDLE_DURATION); - if (code) { - tqError("vgId:%d sched scan wal in %dms failed, ignore this failure", vgId, SCAN_WAL_IDLE_DURATION); - } + // check all tasks + code = doScanWalForAllTasks(pMeta, &numOfTasks); + + el = (taosGetTimestampMs() - st); + if (code) { + tqError("vgId:%d failed to scan wal for all tasks, try next time, elapsed time:%" PRId64 "ms code:%s", vgId, el, + tstrerror(code)); + } else { + tqDebug("vgId:%d scan wal for stream tasks completed, elapsed time:%" PRId64 "ms, next scan start in %dms", vgId, + el, SCAN_WAL_IDLE_DURATION); } + atomic_store_32(&pMeta->scanInfo.scanCounter, 0); return code; } static void doStartScanWal(void* param, void* tmrId) { - int32_t vgId = 0; - int32_t code = 0; - + int32_t vgId = 0; + int32_t code = 0; + SVnode* pVnode = NULL; + int32_t numOfTasks = 0; SBuildScanWalMsgParam* pParam = (SBuildScanWalMsgParam*)param; SStreamMeta* pMeta = taosAcquireRef(streamMetaRefPool, pParam->metaId); @@ -102,71 +94,93 @@ static void doStartScanWal(void* param, void* tmrId) { vgId = pMeta->vgId; - tqDebug("vgId:%d create msg to start wal scan, numOfTasks:%d, vnd restored:%d", vgId, pParam->numOfTasks, - pParam->restored); -#if 0 - // wait for the vnode is freed, and invalid read may occur. + if (pMeta->role == NODE_ROLE_FOLLOWER) { + tqDebug("vgId:%d not leader, role:%d not scan wal anymore", vgId, pMeta->role); + + code = taosReleaseRef(streamMetaRefPool, pParam->metaId); + if (code == TSDB_CODE_SUCCESS) { + tqDebug("vgId:%d jump out of scan wal timer since not leader", vgId); + } else { + tqError("vgId:%d failed to release ref for streamMeta, rid:%" PRId64 " code:%s", vgId, pParam->metaId, + tstrerror(code)); + } + + taosMemFree(pParam); + return; + } + + if (pMeta->startInfo.startAllTasks) { + tqTrace("vgId:%d in restart procedure, not ready to scan wal", vgId); + goto _end; + } + + streamMetaRLock(pMeta); + numOfTasks = taosArrayGetSize(pMeta->pTaskList); + streamMetaRUnLock(pMeta); + + if (numOfTasks == 0) { + goto _end; + } + + tqTrace("vgId:%d create msg to start wal scan, numOfTasks:%d", vgId, numOfTasks); + + // #if 0 + // wait for the vnode is freed, and invalid read may occur. taosMsleep(10000); -#endif + // #endif code = streamTaskSchedTask(&pParam->msgCb, vgId, 0, 0, STREAM_EXEC_T_EXTRACT_WAL_DATA, false); if (code) { tqError("vgId:%d failed sched task to scan wal, code:%s", vgId, tstrerror(code)); } +_end: + streamTmrStart(doStartScanWal, SCAN_WAL_IDLE_DURATION, pParam, &pMeta->scanInfo.scanTimer, &pMeta->scanInfo.scanTimer, + vgId, "scan-wal"); + code = taosReleaseRef(streamMetaRefPool, pParam->metaId); if (code) { tqError("vgId:%d failed to release ref for streamMeta, rid:%" PRId64 " code:%s", vgId, pParam->metaId, tstrerror(code)); } - - taosMemoryFree(pParam); } -int32_t tqScanWalInFuture(STQ* pTq, int32_t numOfTasks, int32_t idleDuration) { +int32_t tqScanWalAsync(STQ* pTq) { SStreamMeta* pMeta = pTq->pStreamMeta; int32_t code = 0; int32_t vgId = TD_VID(pTq->pVnode); tmr_h pTimer = NULL; SBuildScanWalMsgParam* pParam = NULL; + // 1. the vnode should be the leader. + // 2. the stream isn't disabled + if ((pMeta->role == NODE_ROLE_FOLLOWER) || tsDisableStream) { + tqInfo("vgId:%d follower node or stream disabled, not scan wal", vgId); + return TSDB_CODE_SUCCESS; + } + pParam = taosMemoryMalloc(sizeof(SBuildScanWalMsgParam)); if (pParam == NULL) { + tqError("vgId:%d failed to start scan wal, stream not executes, code:%s", vgId, tstrerror(code)); return terrno; } pParam->metaId = pMeta->rid; - pParam->numOfTasks = numOfTasks; - pParam->restored = pTq->pVnode->restored; pParam->msgCb = pTq->pVnode->msgCb; code = streamTimerGetInstance(&pTimer); if (code) { - tqError("vgId:%d failed to get tmr ctrl during sched scan wal", vgId); + tqFatal("vgId:%d failed to get tmr ctrl during sched scan wal", vgId); taosMemoryFree(pParam); } else { - streamTmrStart(doStartScanWal, idleDuration, pParam, pTimer, &pMeta->scanInfo.scanTimer, vgId, "scan-wal-fut"); + // todo: start in 1sec for the first time + streamTmrStart(doStartScanWal, 1000, pParam, pTimer, &pMeta->scanInfo.scanTimer, vgId, + "scan-wal"); } return code; } -int32_t tqScanWalAsync(STQ* pTq, bool ckPause) { - SStreamMeta* pMeta = pTq->pStreamMeta; - bool alreadyRestored = pTq->pVnode->restored; - int32_t code = 0; - - // do not launch the stream tasks, if it is a follower or not restored vnode. - if (!(vnodeIsRoleLeader(pTq->pVnode) && alreadyRestored)) { - return TSDB_CODE_SUCCESS; - } - - streamMetaWLock(pMeta); - code = doScanWalAsync(pTq, ckPause); - streamMetaWUnLock(pMeta); - return code; -} - int32_t tqStopStreamTasksAsync(STQ* pTq) { SStreamMeta* pMeta = pTq->pStreamMeta; int32_t vgId = pMeta->vgId; @@ -348,13 +362,10 @@ int32_t doPutDataIntoInputQ(SStreamTask* pTask, int64_t maxVer, int32_t* numOfIt return code; } -int32_t doScanWalForAllTasks(SStreamMeta* pStreamMeta) { +int32_t doScanWalForAllTasks(SStreamMeta* pStreamMeta, int32_t* pNumOfTasks) { int32_t vgId = pStreamMeta->vgId; SArray* pTaskList = NULL; - int32_t numOfTasks = taosArrayGetSize(pStreamMeta->pTaskList); - if (numOfTasks == 0) { - return TSDB_CODE_SUCCESS; - } + int32_t numOfTasks = 0; // clone the task list, to avoid the task update during scan wal files streamMetaWLock(pStreamMeta); @@ -365,10 +376,13 @@ int32_t doScanWalForAllTasks(SStreamMeta* pStreamMeta) { return terrno; } - tqDebug("vgId:%d start to check wal to extract new submit block for %d tasks", vgId, numOfTasks); - // update the new task number numOfTasks = taosArrayGetSize(pTaskList); + if (pNumOfTasks != NULL) { + *pNumOfTasks = numOfTasks; + } + + tqDebug("vgId:%d start to check wal to extract new submit block for %d tasks", vgId, numOfTasks); for (int32_t i = 0; i < numOfTasks; ++i) { STaskId* pTaskId = taosArrayGet(pTaskList, i); @@ -427,51 +441,59 @@ int32_t doScanWalForAllTasks(SStreamMeta* pStreamMeta) { return TSDB_CODE_SUCCESS; } -int32_t doScanWalAsync(STQ* pTq, bool ckPause) { - SStreamMeta* pMeta = pTq->pStreamMeta; - bool alreadyRestored = pTq->pVnode->restored; - 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; - } - - if (pMeta->startInfo.startAllTasks) { - tqTrace("vgId:%d in restart procedure, not scan wal", vgId); - return 0; - } - - pMeta->scanInfo.scanCounter += 1; - if (pMeta->scanInfo.scanCounter > MAX_REPEAT_SCAN_THRESHOLD) { - pMeta->scanInfo.scanCounter = MAX_REPEAT_SCAN_THRESHOLD; - } - - if (pMeta->scanInfo.scanCounter > 1) { - tqDebug("vgId:%d wal read task has been launched, remain scan times:%d", vgId, pMeta->scanInfo.scanCounter); - return 0; - } - - int32_t numOfPauseTasks = pMeta->numOfPausedTasks; - if (ckPause && numOfTasks == numOfPauseTasks) { - tqDebug("vgId:%d ignore all submit, all streams had been paused, reset the walScanCounter", vgId); - - // reset the counter value, since we do not launch the scan wal operation. - pMeta->scanInfo.scanCounter = 0; - return 0; - } - - tqDebug("vgId:%d create msg to start wal scan to launch stream tasks, numOfTasks:%d, vnd restored:%d", vgId, - numOfTasks, alreadyRestored); - - return streamTaskSchedTask(&pTq->pVnode->msgCb, vgId, 0, 0, STREAM_EXEC_T_EXTRACT_WAL_DATA, false); -} +// int32_t doScanWalAsync(STQ* pTq) { +// SStreamMeta* pMeta = pTq->pStreamMeta; +// bool alreadyRestored = pTq->pVnode->restored; +// int32_t vgId = pMeta->vgId; +// int32_t code = 0; +// int32_t numOfTasks = taosArrayGetSize(pMeta->pTaskList); +// +// // if (numOfTasks == 0) { +//// tqDebug("vgId:%d no stream tasks existed to run", vgId); +//// return 0; +//// } +// +//// if (pMeta->startInfo.startAllTasks) { +//// tqTrace("vgId:%d in restart procedure, not scan wal", vgId); +//// return 0; +//// } +// +//// pMeta->scanInfo.scanCounter += 1; +//// if (pMeta->scanInfo.scanCounter > MAX_REPEAT_SCAN_THRESHOLD) { +//// pMeta->scanInfo.scanCounter = MAX_REPEAT_SCAN_THRESHOLD; +//// } +// +//// if (pMeta->scanInfo.scanCounter > 1) { +//// tqDebug("vgId:%d wal read task has been launched, remain scan times:%d", vgId, pMeta->scanInfo.scanCounter); +//// return 0; +//// } +// +//// int32_t numOfPauseTasks = pMeta->numOfPausedTasks; +//// if (ckPause && numOfTasks == numOfPauseTasks) { +//// tqDebug("vgId:%d ignore all submit, all streams had been paused, reset the walScanCounter", vgId); +//// +//// reset the counter value, since we do not launch the scan wal operation. +//// pMeta->scanInfo.scanCounter = 0; +//// return 0; +//// } +// +//// tqDebug("vgId:%d create msg to start wal scan to launch stream tasks, numOfTasks:%d, vnd restored:%d", vgId, +//// numOfTasks, alreadyRestored); +//// +//// code = streamTaskSchedTask(&pTq->pVnode->msgCb, vgId, 0, 0, STREAM_EXEC_T_EXTRACT_WAL_DATA); +//// if (code) { +//// tqError("vgId:%d failed create msg to scan data in wal, retry in %dms", vgId, SCAN_WAL_IDLE_DURATION); +//// } +// +// code = tqScanWalInFuture(pTq, numOfTasks, SCAN_WAL_IDLE_DURATION); +// if (code) { +// tqError("vgId:%d sched scan wal in %dms failed, ignore this failure", vgId, SCAN_WAL_IDLE_DURATION); +// } +//} void streamMetaFreeTQDuringScanWalError(STQ* pTq) { SBuildScanWalMsgParam* p = taosMemoryCalloc(1, sizeof(SBuildScanWalMsgParam)); p->metaId = pTq->pStreamMeta->rid; - p->numOfTasks = 0; doStartScanWal(p, 0); } \ No newline at end of file diff --git a/source/dnode/vnode/src/tq/tqUtil.c b/source/dnode/vnode/src/tq/tqUtil.c index 197a45cdb9..6f001981fb 100644 --- a/source/dnode/vnode/src/tq/tqUtil.c +++ b/source/dnode/vnode/src/tq/tqUtil.c @@ -47,6 +47,10 @@ END: void tqUpdateNodeStage(STQ* pTq, bool isLeader) { SSyncState state = syncGetState(pTq->pVnode->sync); streamMetaUpdateStageRole(pTq->pStreamMeta, state.term, isLeader); + + if (isLeader) { + tqScanWalAsync(pTq); + } } static int32_t tqInitTaosxRsp(SMqDataRsp* pRsp, STqOffsetVal pOffset) { diff --git a/source/dnode/vnode/src/tqCommon/tqCommon.c b/source/dnode/vnode/src/tqCommon/tqCommon.c index 0b206110b3..885193fec1 100644 --- a/source/dnode/vnode/src/tqCommon/tqCommon.c +++ b/source/dnode/vnode/src/tqCommon/tqCommon.c @@ -943,10 +943,10 @@ int32_t tqStartTaskCompleteCallback(SStreamMeta* pMeta) { streamMetaWUnLock(pMeta); - if (scanWal && (vgId != SNODE_HANDLE)) { - tqDebug("vgId:%d start scan wal for executing tasks", vgId); - code = tqScanWalAsync(pMeta->ahandle, true); - } +// if (scanWal && (vgId != SNODE_HANDLE)) { +// tqDebug("vgId:%d start scan wal for executing tasks", vgId); +// code = tqScanWalAsync(pMeta->ahandle, true); +// } return code; } @@ -1175,7 +1175,7 @@ static int32_t tqProcessTaskResumeImpl(void* handle, SStreamTask* pTask, int64_t pTask->hTaskInfo.operatorOpen = false; code = streamStartScanHistoryAsync(pTask, igUntreated); } else if (level == TASK_LEVEL__SOURCE && (streamQueueGetNumOfItems(pTask->inputq.queue) == 0)) { - code = tqScanWalAsync((STQ*)handle, false); +// code = tqScanWalAsync((STQ*)handle, false); } else { code = streamTrySchedExec(pTask, false); } diff --git a/source/libs/stream/src/streamHb.c b/source/libs/stream/src/streamHb.c index 7c157bb05e..36a6488709 100644 --- a/source/libs/stream/src/streamHb.c +++ b/source/libs/stream/src/streamHb.c @@ -341,7 +341,7 @@ void streamMetaHbToMnode(void* param, void* tmrId) { if (code == TSDB_CODE_SUCCESS) { stInfo("vgId:%d role:%d not leader not send hb to mnode", vgId, role); } else { - stError("vgId:%d role:%d not leader not send hb to mnodefailed to release the meta rid:%" PRId64, vgId, role, rid); + stError("vgId:%d role:%d not leader not send hb to mnode, failed to release meta rid:%" PRId64, vgId, role, rid); } // taosMemoryFree(param); return; diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index c5a5c70115..9b8df5bb94 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -1348,7 +1348,7 @@ void streamMetaUpdateStageRole(SStreamMeta* pMeta, int64_t stage, bool isLeader) // mark the sign to send msg before close all tasks // 1. for leader vnode, always send msg before closing - // 2. for follower vnode, if it's is changed from leader, also sending msg before closing. + // 2. for follower vnode, if it's changed from leader, also sending msg before closing. if (pMeta->role == NODE_ROLE_LEADER) { pMeta->sendMsgBeforeClosing = true; } From 0ec8e1e2028c01fb567b35b065e19f52a9b1d2f2 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sat, 22 Feb 2025 12:35:37 +0800 Subject: [PATCH 15/70] refactor(stream): remove comments. --- source/dnode/vnode/src/tq/tqStreamTask.c | 50 ------------------------ 1 file changed, 50 deletions(-) diff --git a/source/dnode/vnode/src/tq/tqStreamTask.c b/source/dnode/vnode/src/tq/tqStreamTask.c index f6f1010a67..e879fb59d6 100644 --- a/source/dnode/vnode/src/tq/tqStreamTask.c +++ b/source/dnode/vnode/src/tq/tqStreamTask.c @@ -441,56 +441,6 @@ int32_t doScanWalForAllTasks(SStreamMeta* pStreamMeta, int32_t* pNumOfTasks) { return TSDB_CODE_SUCCESS; } -// int32_t doScanWalAsync(STQ* pTq) { -// SStreamMeta* pMeta = pTq->pStreamMeta; -// bool alreadyRestored = pTq->pVnode->restored; -// int32_t vgId = pMeta->vgId; -// int32_t code = 0; -// int32_t numOfTasks = taosArrayGetSize(pMeta->pTaskList); -// -// // if (numOfTasks == 0) { -//// tqDebug("vgId:%d no stream tasks existed to run", vgId); -//// return 0; -//// } -// -//// if (pMeta->startInfo.startAllTasks) { -//// tqTrace("vgId:%d in restart procedure, not scan wal", vgId); -//// return 0; -//// } -// -//// pMeta->scanInfo.scanCounter += 1; -//// if (pMeta->scanInfo.scanCounter > MAX_REPEAT_SCAN_THRESHOLD) { -//// pMeta->scanInfo.scanCounter = MAX_REPEAT_SCAN_THRESHOLD; -//// } -// -//// if (pMeta->scanInfo.scanCounter > 1) { -//// tqDebug("vgId:%d wal read task has been launched, remain scan times:%d", vgId, pMeta->scanInfo.scanCounter); -//// return 0; -//// } -// -//// int32_t numOfPauseTasks = pMeta->numOfPausedTasks; -//// if (ckPause && numOfTasks == numOfPauseTasks) { -//// tqDebug("vgId:%d ignore all submit, all streams had been paused, reset the walScanCounter", vgId); -//// -//// reset the counter value, since we do not launch the scan wal operation. -//// pMeta->scanInfo.scanCounter = 0; -//// return 0; -//// } -// -//// tqDebug("vgId:%d create msg to start wal scan to launch stream tasks, numOfTasks:%d, vnd restored:%d", vgId, -//// numOfTasks, alreadyRestored); -//// -//// code = streamTaskSchedTask(&pTq->pVnode->msgCb, vgId, 0, 0, STREAM_EXEC_T_EXTRACT_WAL_DATA); -//// if (code) { -//// tqError("vgId:%d failed create msg to scan data in wal, retry in %dms", vgId, SCAN_WAL_IDLE_DURATION); -//// } -// -// code = tqScanWalInFuture(pTq, numOfTasks, SCAN_WAL_IDLE_DURATION); -// if (code) { -// tqError("vgId:%d sched scan wal in %dms failed, ignore this failure", vgId, SCAN_WAL_IDLE_DURATION); -// } -//} - void streamMetaFreeTQDuringScanWalError(STQ* pTq) { SBuildScanWalMsgParam* p = taosMemoryCalloc(1, sizeof(SBuildScanWalMsgParam)); p->metaId = pTq->pStreamMeta->rid; From a9b5d0fbf061dd7b9303a8704e801ee7c74c4831 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sat, 22 Feb 2025 12:33:32 +0800 Subject: [PATCH 16/70] fix(stream): update function return value. --- source/dnode/vnode/src/inc/vnodeInt.h | 2 +- source/dnode/vnode/src/tq/tqStreamTask.c | 4 +--- source/dnode/vnode/src/tq/tqUtil.c | 2 +- 3 files changed, 3 insertions(+), 5 deletions(-) diff --git a/source/dnode/vnode/src/inc/vnodeInt.h b/source/dnode/vnode/src/inc/vnodeInt.h index a76fc17377..6b3cf47d13 100644 --- a/source/dnode/vnode/src/inc/vnodeInt.h +++ b/source/dnode/vnode/src/inc/vnodeInt.h @@ -239,7 +239,7 @@ void tqClose(STQ*); int tqPushMsg(STQ*, tmsg_t msgType); int tqRegisterPushHandle(STQ* pTq, void* handle, SRpcMsg* pMsg); void tqUnregisterPushHandle(STQ* pTq, void* pHandle); -int tqScanWalAsync(STQ* pTq); +void tqScanWalAsync(STQ* pTq); int32_t tqStopStreamTasksAsync(STQ* pTq); int32_t tqProcessTaskCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg* pRsp); int32_t tqProcessTaskCheckpointReadyMsg(STQ* pTq, SRpcMsg* pMsg); diff --git a/source/dnode/vnode/src/tq/tqStreamTask.c b/source/dnode/vnode/src/tq/tqStreamTask.c index e879fb59d6..0167ffc017 100644 --- a/source/dnode/vnode/src/tq/tqStreamTask.c +++ b/source/dnode/vnode/src/tq/tqStreamTask.c @@ -145,7 +145,7 @@ _end: } } -int32_t tqScanWalAsync(STQ* pTq) { +void tqScanWalAsync(STQ* pTq) { SStreamMeta* pMeta = pTq->pStreamMeta; int32_t code = 0; int32_t vgId = TD_VID(pTq->pVnode); @@ -177,8 +177,6 @@ int32_t tqScanWalAsync(STQ* pTq) { streamTmrStart(doStartScanWal, 1000, pParam, pTimer, &pMeta->scanInfo.scanTimer, vgId, "scan-wal"); } - - return code; } int32_t tqStopStreamTasksAsync(STQ* pTq) { diff --git a/source/dnode/vnode/src/tq/tqUtil.c b/source/dnode/vnode/src/tq/tqUtil.c index 6f001981fb..217859394a 100644 --- a/source/dnode/vnode/src/tq/tqUtil.c +++ b/source/dnode/vnode/src/tq/tqUtil.c @@ -49,7 +49,7 @@ void tqUpdateNodeStage(STQ* pTq, bool isLeader) { streamMetaUpdateStageRole(pTq->pStreamMeta, state.term, isLeader); if (isLeader) { - tqScanWalAsync(pTq); + int32_t code = tqScanWalAsync(pTq); } } From 4d62c60aaf96dcb48297ecf925222d8de63d686a Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sat, 22 Feb 2025 14:16:46 +0800 Subject: [PATCH 17/70] fix(stream): fix syntax error. --- source/dnode/vnode/src/tq/tqStreamTask.c | 4 ++-- source/dnode/vnode/src/tq/tqUtil.c | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/source/dnode/vnode/src/tq/tqStreamTask.c b/source/dnode/vnode/src/tq/tqStreamTask.c index 0167ffc017..07a0888802 100644 --- a/source/dnode/vnode/src/tq/tqStreamTask.c +++ b/source/dnode/vnode/src/tq/tqStreamTask.c @@ -156,13 +156,13 @@ void tqScanWalAsync(STQ* pTq) { // 2. the stream isn't disabled if ((pMeta->role == NODE_ROLE_FOLLOWER) || tsDisableStream) { tqInfo("vgId:%d follower node or stream disabled, not scan wal", vgId); - return TSDB_CODE_SUCCESS; + return; } pParam = taosMemoryMalloc(sizeof(SBuildScanWalMsgParam)); if (pParam == NULL) { tqError("vgId:%d failed to start scan wal, stream not executes, code:%s", vgId, tstrerror(code)); - return terrno; + return; } pParam->metaId = pMeta->rid; diff --git a/source/dnode/vnode/src/tq/tqUtil.c b/source/dnode/vnode/src/tq/tqUtil.c index 217859394a..6f001981fb 100644 --- a/source/dnode/vnode/src/tq/tqUtil.c +++ b/source/dnode/vnode/src/tq/tqUtil.c @@ -49,7 +49,7 @@ void tqUpdateNodeStage(STQ* pTq, bool isLeader) { streamMetaUpdateStageRole(pTq->pStreamMeta, state.term, isLeader); if (isLeader) { - int32_t code = tqScanWalAsync(pTq); + tqScanWalAsync(pTq); } } From 56f1c800180b87e1d16b0e4137e3e36deb041cea Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sat, 22 Feb 2025 14:18:44 +0800 Subject: [PATCH 18/70] fix(stream): disable error injection. --- source/dnode/vnode/src/tq/tqStreamTask.c | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/source/dnode/vnode/src/tq/tqStreamTask.c b/source/dnode/vnode/src/tq/tqStreamTask.c index 07a0888802..3934376427 100644 --- a/source/dnode/vnode/src/tq/tqStreamTask.c +++ b/source/dnode/vnode/src/tq/tqStreamTask.c @@ -124,10 +124,10 @@ static void doStartScanWal(void* param, void* tmrId) { tqTrace("vgId:%d create msg to start wal scan, numOfTasks:%d", vgId, numOfTasks); - // #if 0 +#if 0 // wait for the vnode is freed, and invalid read may occur. taosMsleep(10000); - // #endif +#endif code = streamTaskSchedTask(&pParam->msgCb, vgId, 0, 0, STREAM_EXEC_T_EXTRACT_WAL_DATA, false); if (code) { From 9018970135ef351411d0755bb3a6d4e26273bb3c Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sun, 23 Feb 2025 01:38:05 +0800 Subject: [PATCH 19/70] refactor: do some internal refactor. --- include/libs/stream/tstream.h | 4 +- source/dnode/vnode/src/tq/tq.c | 19 ++----- source/dnode/vnode/src/tq/tqStreamTask.c | 66 ++++++++++++++++++------ source/libs/stream/src/streamHb.c | 4 +- source/libs/stream/src/streamMeta.c | 10 +++- 5 files changed, 68 insertions(+), 35 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 3262130bd4..f7d42794be 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -485,8 +485,10 @@ typedef struct STaskUpdateInfo { } STaskUpdateInfo; typedef struct SScanWalInfo { - int32_t scanCounter; + int32_t scanSentinel; tmr_h scanTimer; + int64_t lastScanTs; + int32_t tickCounter; } SScanWalInfo; typedef struct SFatalErrInfo { diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index fc173eb691..63727e5c45 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1113,23 +1113,14 @@ int32_t tqProcessTaskRunReq(STQ* pTq, SRpcMsg* pMsg) { // extracted submit data from wal files for all tasks if (req.reqType == STREAM_EXEC_T_EXTRACT_WAL_DATA) { return tqScanWal(pTq); - } + } else { + code = tqStreamTaskProcessRunReq(pTq->pStreamMeta, pMsg, vnodeIsRoleLeader(pTq->pVnode)); + if (code) { + tqError("vgId:%d failed to create task run req, code:%s", TD_VID(pTq->pVnode), tstrerror(code)); + } - code = tqStreamTaskProcessRunReq(pTq->pStreamMeta, pMsg, vnodeIsRoleLeader(pTq->pVnode)); - if (code) { - tqError("vgId:%d failed to create task run req, code:%s", TD_VID(pTq->pVnode), tstrerror(code)); return code; } - - // let's continue scan data in the wal files -// if (req.reqType >= 0 || req.reqType == STREAM_EXEC_T_RESUME_TASK) { -// code = tqScanWalAsync(pTq, false); // it's ok to failed -// if (code) { -// tqError("vgId:%d failed to start scan wal file, code:%s", pTq->pStreamMeta->vgId, tstrerror(code)); -// } -// } - - return code; } int32_t tqProcessTaskDispatchReq(STQ* pTq, SRpcMsg* pMsg) { diff --git a/source/dnode/vnode/src/tq/tqStreamTask.c b/source/dnode/vnode/src/tq/tqStreamTask.c index 3934376427..4f56acea8b 100644 --- a/source/dnode/vnode/src/tq/tqStreamTask.c +++ b/source/dnode/vnode/src/tq/tqStreamTask.c @@ -16,12 +16,11 @@ #include "tq.h" #include "vnd.h" -#define MAX_REPEAT_SCAN_THRESHOLD 3 -#define SCAN_WAL_IDLE_DURATION 500 // idle for 500ms to do next wal scan +#define SCAN_WAL_IDLE_DURATION 250 // idle for 500ms to do next wal scan +#define SCAN_WAL_WAIT_COUNT 2 typedef struct SBuildScanWalMsgParam { int64_t metaId; -// int8_t restored; SMsgCb msgCb; } SBuildScanWalMsgParam; @@ -30,7 +29,6 @@ static int32_t setWalReaderStartOffset(SStreamTask* pTask, int32_t vgId); static bool handleFillhistoryScanComplete(SStreamTask* pTask, int64_t ver); static bool taskReadyForDataFromWal(SStreamTask* pTask); static int32_t doPutDataIntoInputQ(SStreamTask* pTask, int64_t maxVer, int32_t* numOfItems, bool* pSucc); -static int32_t tqScanWalInFuture(STQ* pTq); // extract data blocks(submit/delete) from WAL, and add them into the input queue for all the sources tasks. int32_t tqScanWal(STQ* pTq) { @@ -41,7 +39,7 @@ int32_t tqScanWal(STQ* pTq) { int64_t el = 0; int32_t code = 0; - int32_t old = atomic_val_compare_exchange_32(&pMeta->scanInfo.scanCounter, 0, 1); + int32_t old = atomic_val_compare_exchange_32(&pMeta->scanInfo.scanSentinel, 0, 1); if (old == 0) { tqDebug("vgId:%d try to scan wal to extract data", vgId); } else { @@ -49,27 +47,44 @@ int32_t tqScanWal(STQ* pTq) { return code; } + // the scan wal interval less than 200, not scan, actually. + if ((pMeta->scanInfo.lastScanTs > st) && (pMeta->scanInfo.lastScanTs - st < 200)) { + tqDebug("vgId:%d scan wal less than 200ms, do nothing", vgId); + atomic_store_32(&pMeta->scanInfo.scanSentinel, 0); + return code; + } + // check all tasks code = doScanWalForAllTasks(pMeta, &numOfTasks); - el = (taosGetTimestampMs() - st); + pMeta->scanInfo.lastScanTs = taosGetTimestampMs(); + el = (pMeta->scanInfo.lastScanTs - st); + if (code) { tqError("vgId:%d failed to scan wal for all tasks, try next time, elapsed time:%" PRId64 "ms code:%s", vgId, el, tstrerror(code)); } else { - tqDebug("vgId:%d scan wal for stream tasks completed, elapsed time:%" PRId64 "ms, next scan start in %dms", vgId, - el, SCAN_WAL_IDLE_DURATION); + tqDebug("vgId:%d scan wal for stream tasks completed, elapsed time:%" PRId64 "ms", vgId, el); } - atomic_store_32(&pMeta->scanInfo.scanCounter, 0); + atomic_store_32(&pMeta->scanInfo.scanSentinel, 0); return code; } +static bool waitEnoughDuration(SStreamMeta* pMeta) { + if ((++pMeta->scanInfo.tickCounter) >= SCAN_WAL_WAIT_COUNT) { + pMeta->scanInfo.tickCounter = 0; + return true; + } + + return false; +} + static void doStartScanWal(void* param, void* tmrId) { int32_t vgId = 0; int32_t code = 0; - SVnode* pVnode = NULL; int32_t numOfTasks = 0; + tmr_h pTimer = NULL; SBuildScanWalMsgParam* pParam = (SBuildScanWalMsgParam*)param; SStreamMeta* pMeta = taosAcquireRef(streamMetaRefPool, pParam->metaId); @@ -79,10 +94,12 @@ static void doStartScanWal(void* param, void* tmrId) { return; } + vgId = pMeta->vgId; + if (pMeta->closeFlag) { code = taosReleaseRef(streamMetaRefPool, pParam->metaId); if (code == TSDB_CODE_SUCCESS) { - tqDebug("vgId:%d jump out of scan wal timer since closed", vgId); + tqInfo("vgId:%d jump out of scan wal timer since closed", vgId); } else { tqError("vgId:%d failed to release ref for streamMeta, rid:%" PRId64 " code:%s", vgId, pParam->metaId, tstrerror(code)); @@ -92,9 +109,7 @@ static void doStartScanWal(void* param, void* tmrId) { return; } - vgId = pMeta->vgId; - - if (pMeta->role == NODE_ROLE_FOLLOWER) { + if (pMeta->role != NODE_ROLE_LEADER) { tqDebug("vgId:%d not leader, role:%d not scan wal anymore", vgId, pMeta->role); code = taosReleaseRef(streamMetaRefPool, pParam->metaId); @@ -114,6 +129,24 @@ static void doStartScanWal(void* param, void* tmrId) { goto _end; } + code = streamTimerGetInstance(&pTimer); + if (code) { + tqFatal("vgId:%d failed to get tmr ctrl during sched scan wal, not scan wal, code:%s", vgId, tstrerror(code)); + taosMemoryFree(pParam); + return; + } + + if (!waitEnoughDuration(pMeta)) { + streamTmrStart(doStartScanWal, SCAN_WAL_IDLE_DURATION, pParam, pTimer, &pMeta->scanInfo.scanTimer, vgId, + "scan-wal"); + code = taosReleaseRef(streamMetaRefPool, pParam->metaId); + if (code) { + tqError("vgId:%d failed to release ref for streamMeta, rid:%" PRId64 " code:%s", vgId, pParam->metaId, + tstrerror(code)); + } + return; + } + streamMetaRLock(pMeta); numOfTasks = taosArrayGetSize(pMeta->pTaskList); streamMetaRUnLock(pMeta); @@ -135,8 +168,9 @@ static void doStartScanWal(void* param, void* tmrId) { } _end: - streamTmrStart(doStartScanWal, SCAN_WAL_IDLE_DURATION, pParam, &pMeta->scanInfo.scanTimer, &pMeta->scanInfo.scanTimer, - vgId, "scan-wal"); + + streamTmrStart(doStartScanWal, SCAN_WAL_IDLE_DURATION, pParam, pTimer, &pMeta->scanInfo.scanTimer, vgId, "scan-wal"); + tqDebug("vgId:%d scan-wal will start in %dms", vgId, SCAN_WAL_IDLE_DURATION*SCAN_WAL_WAIT_COUNT); code = taosReleaseRef(streamMetaRefPool, pParam->metaId); if (code) { diff --git a/source/libs/stream/src/streamHb.c b/source/libs/stream/src/streamHb.c index 36a6488709..90ebd47ac6 100644 --- a/source/libs/stream/src/streamHb.c +++ b/source/libs/stream/src/streamHb.c @@ -327,7 +327,7 @@ void streamMetaHbToMnode(void* param, void* tmrId) { pMeta->pHbInfo->hbStart = 0; code = taosReleaseRef(streamMetaRefPool, rid); if (code == TSDB_CODE_SUCCESS) { - stDebug("vgId:%d jump out of meta timer", vgId); + stInfo("vgId:%d jump out of meta timer since closed", vgId); } else { stError("vgId:%d jump out of meta timer, failed to release the meta rid:%" PRId64, vgId, rid); } @@ -413,7 +413,7 @@ void destroyMetaHbInfo(SMetaHbInfo* pInfo) { void streamMetaWaitForHbTmrQuit(SStreamMeta* pMeta) { // wait for the stream meta hb function stopping if (pMeta->role == NODE_ROLE_LEADER) { - taosMsleep(2 * META_HB_CHECK_INTERVAL); + taosMsleep(3 * META_HB_CHECK_INTERVAL); stDebug("vgId:%d wait for meta to stop timer", pMeta->vgId); } } diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 9b8df5bb94..f724e46c49 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -427,7 +427,7 @@ int32_t streamMetaOpen(const char* path, void* ahandle, FTaskBuild buildTaskFn, pMeta->pTaskList = taosArrayInit(4, sizeof(SStreamTaskId)); TSDB_CHECK_NULL(pMeta->pTaskList, code, lino, _err, terrno); - pMeta->scanInfo.scanCounter = 0; + pMeta->scanInfo.scanSentinel = 0; pMeta->vgId = vgId; pMeta->ahandle = ahandle; pMeta->buildTaskFn = buildTaskFn; @@ -1241,8 +1241,8 @@ void streamMetaNotifyClose(SStreamMeta* pMeta) { vgId, (pMeta->role == NODE_ROLE_LEADER), startTs, sendCount); // wait for the stream meta hb function stopping - streamMetaWaitForHbTmrQuit(pMeta); pMeta->closeFlag = true; + streamMetaWaitForHbTmrQuit(pMeta); stDebug("vgId:%d start to check all tasks for closing", vgId); int64_t st = taosGetTimestampMs(); @@ -1281,6 +1281,12 @@ void streamMetaNotifyClose(SStreamMeta* pMeta) { double el = (taosGetTimestampMs() - st) / 1000.0; stDebug("vgId:%d stop all %d task(s) completed, elapsed time:%.2f Sec.", pMeta->vgId, numOfTasks, el); + + if (pMeta->scanInfo.scanTimer != NULL) { + streamTmrStop(pMeta->scanInfo.scanTimer); + pMeta->scanInfo.scanTimer = NULL; + } + streamMetaRUnLock(pMeta); } From f1c4d2734fecd3cc6349f25e868d9afabb07c139 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sun, 23 Feb 2025 10:43:56 +0800 Subject: [PATCH 20/70] fix(stream): follower nodes not restart tasks. --- include/dnode/vnode/tqCommon.h | 2 +- source/dnode/snode/src/snode.c | 2 +- source/dnode/vnode/src/tq/tq.c | 3 ++- source/dnode/vnode/src/tqCommon/tqCommon.c | 31 ++++++++++++++-------- source/dnode/vnode/src/vnd/vnodeSvr.c | 4 ++- 5 files changed, 27 insertions(+), 15 deletions(-) diff --git a/include/dnode/vnode/tqCommon.h b/include/dnode/vnode/tqCommon.h index 4d5e18520c..d4ca0aba62 100644 --- a/include/dnode/vnode/tqCommon.h +++ b/include/dnode/vnode/tqCommon.h @@ -19,7 +19,7 @@ // message process int32_t tqStreamTaskStartAsync(SStreamMeta* pMeta, SMsgCb* cb, bool restart); int32_t tqStreamStartOneTaskAsync(SStreamMeta* pMeta, SMsgCb* cb, int64_t streamId, int32_t taskId); -int32_t tqStreamTaskProcessUpdateReq(SStreamMeta* pMeta, SMsgCb* cb, SRpcMsg* pMsg, bool restored); +int32_t tqStreamTaskProcessUpdateReq(SStreamMeta* pMeta, SMsgCb* cb, SRpcMsg* pMsg, bool restored, bool isLeader); int32_t tqStreamTaskProcessDispatchReq(SStreamMeta* pMeta, SRpcMsg* pMsg); int32_t tqStreamTaskProcessDispatchRsp(SStreamMeta* pMeta, SRpcMsg* pMsg); int32_t tqStreamTaskProcessRetrieveReq(SStreamMeta* pMeta, SRpcMsg* pMsg); diff --git a/source/dnode/snode/src/snode.c b/source/dnode/snode/src/snode.c index 12609e2c06..b37232aba3 100644 --- a/source/dnode/snode/src/snode.c +++ b/source/dnode/snode/src/snode.c @@ -157,7 +157,7 @@ int32_t sndProcessWriteMsg(SSnode *pSnode, SRpcMsg *pMsg, SRpcMsg *pRsp) { case TDMT_STREAM_TASK_DROP: return tqStreamTaskProcessDropReq(pSnode->pMeta, pMsg->pCont, pMsg->contLen); case TDMT_VND_STREAM_TASK_UPDATE: - return tqStreamTaskProcessUpdateReq(pSnode->pMeta, &pSnode->msgCb, pMsg, true); + return tqStreamTaskProcessUpdateReq(pSnode->pMeta, &pSnode->msgCb, pMsg, true, true); case TDMT_VND_STREAM_TASK_RESET: return tqStreamTaskProcessTaskResetReq(pSnode->pMeta, pMsg->pCont); case TDMT_STREAM_TASK_PAUSE: diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 63727e5c45..e1298b11ea 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -1364,7 +1364,8 @@ int32_t tqProcessTaskCheckpointReadyMsg(STQ* pTq, SRpcMsg* pMsg) { } int32_t tqProcessTaskUpdateReq(STQ* pTq, SRpcMsg* pMsg) { - return tqStreamTaskProcessUpdateReq(pTq->pStreamMeta, &pTq->pVnode->msgCb, pMsg, pTq->pVnode->restored); + return tqStreamTaskProcessUpdateReq(pTq->pStreamMeta, &pTq->pVnode->msgCb, pMsg, + pTq->pVnode->restored, (pTq->pStreamMeta->role == NODE_ROLE_LEADER)); } int32_t tqProcessTaskResetReq(STQ* pTq, SRpcMsg* pMsg) { diff --git a/source/dnode/vnode/src/tqCommon/tqCommon.c b/source/dnode/vnode/src/tqCommon/tqCommon.c index 885193fec1..05a66b4770 100644 --- a/source/dnode/vnode/src/tqCommon/tqCommon.c +++ b/source/dnode/vnode/src/tqCommon/tqCommon.c @@ -139,7 +139,7 @@ int32_t tqStreamStartOneTaskAsync(SStreamMeta* pMeta, SMsgCb* cb, int64_t stream } // this is to process request from transaction, always return true. -int32_t tqStreamTaskProcessUpdateReq(SStreamMeta* pMeta, SMsgCb* cb, SRpcMsg* pMsg, bool restored) { +int32_t tqStreamTaskProcessUpdateReq(SStreamMeta* pMeta, SMsgCb* cb, SRpcMsg* pMsg, bool restored, bool isLeader) { int32_t vgId = pMeta->vgId; char* msg = POINTER_SHIFT(pMsg->pCont, sizeof(SMsgHead)); int32_t len = pMsg->contLen - sizeof(SMsgHead); @@ -298,14 +298,19 @@ int32_t tqStreamTaskProcessUpdateReq(SStreamMeta* pMeta, SMsgCb* cb, SRpcMsg* pM int32_t numOfTasks = streamMetaGetNumOfTasks(pMeta); int32_t updateTasks = taosHashGetSize(pMeta->updateInfo.pTasks); - if (restored) { + if (restored && isLeader) { tqDebug("vgId:%d s-task:0x%x update epset transId:%d, set the restart flag", vgId, req.taskId, req.transId); pMeta->startInfo.tasksWillRestart = 1; } if (updateTasks < numOfTasks) { - tqDebug("vgId:%d closed tasks:%d, unclosed:%d, all tasks will be started when nodeEp update completed", vgId, - updateTasks, (numOfTasks - updateTasks)); + if (isLeader) { + tqDebug("vgId:%d closed tasks:%d, unclosed:%d, all tasks will be started when nodeEp update completed", vgId, + updateTasks, (numOfTasks - updateTasks)); + } else { + tqDebug("vgId:%d closed tasks:%d, unclosed:%d, follower not restart tasks", vgId, updateTasks, + (numOfTasks - updateTasks)); + } } else { if ((code = streamMetaCommit(pMeta)) < 0) { // always return true @@ -316,17 +321,21 @@ int32_t tqStreamTaskProcessUpdateReq(SStreamMeta* pMeta, SMsgCb* cb, SRpcMsg* pM streamMetaClearSetUpdateTaskListComplete(pMeta); - if (!restored) { - tqDebug("vgId:%d vnode restore not completed, not start all tasks", vgId); - } else { - tqDebug("vgId:%d all %d task(s) nodeEp updated and closed, transId:%d", vgId, numOfTasks, req.transId); + if (isLeader) { + if (!restored) { + tqDebug("vgId:%d vnode restore not completed, not start all tasks", vgId); + } else { + tqDebug("vgId:%d all %d task(s) nodeEp updated and closed, transId:%d", vgId, numOfTasks, req.transId); #if 0 taosMSleep(5000);// for test purpose, to trigger the leader election #endif - code = tqStreamTaskStartAsync(pMeta, cb, true); - if (code) { - tqError("vgId:%d async start all tasks, failed, code:%s", vgId, tstrerror(code)); + code = tqStreamTaskStartAsync(pMeta, cb, true); + if (code) { + tqError("vgId:%d async start all tasks, failed, code:%s", vgId, tstrerror(code)); + } } + } else { + tqDebug("vgId:%d follower nodes not restart tasks", vgId); } } diff --git a/source/dnode/vnode/src/vnd/vnodeSvr.c b/source/dnode/vnode/src/vnd/vnodeSvr.c index aa55885f15..7c57bb36a7 100644 --- a/source/dnode/vnode/src/vnd/vnodeSvr.c +++ b/source/dnode/vnode/src/vnd/vnodeSvr.c @@ -931,7 +931,9 @@ int32_t vnodeProcessFetchMsg(SVnode *pVnode, SRpcMsg *pMsg, SQueueInfo *pInfo) { int32_t vnodeProcessStreamMsg(SVnode *pVnode, SRpcMsg *pMsg, SQueueInfo *pInfo) { vTrace("vgId:%d, msg:%p in stream queue is processing", pVnode->config.vgId, pMsg); - if (!syncIsReadyForRead(pVnode->sync)) { + if ((pMsg->msgType == TDMT_SCH_FETCH || pMsg->msgType == TDMT_VND_TABLE_META || pMsg->msgType == TDMT_VND_TABLE_CFG || + pMsg->msgType == TDMT_VND_BATCH_META) && + !syncIsReadyForRead(pVnode->sync)) { vnodeRedirectRpcMsg(pVnode, pMsg, terrno); return 0; } From 2fdb31a41a116ccc686071661e962fbf13434191 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sun, 23 Feb 2025 03:42:44 +0800 Subject: [PATCH 21/70] fix(stream): adjust scan timer --- source/dnode/vnode/src/tq/tqStreamTask.c | 3 +-- source/libs/stream/src/streamMeta.c | 4 ++-- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/source/dnode/vnode/src/tq/tqStreamTask.c b/source/dnode/vnode/src/tq/tqStreamTask.c index 4f56acea8b..86d3e40fd5 100644 --- a/source/dnode/vnode/src/tq/tqStreamTask.c +++ b/source/dnode/vnode/src/tq/tqStreamTask.c @@ -207,8 +207,7 @@ void tqScanWalAsync(STQ* pTq) { tqFatal("vgId:%d failed to get tmr ctrl during sched scan wal", vgId); taosMemoryFree(pParam); } else { - // todo: start in 1sec for the first time - streamTmrStart(doStartScanWal, 1000, pParam, pTimer, &pMeta->scanInfo.scanTimer, vgId, + streamTmrStart(doStartScanWal, SCAN_WAL_IDLE_DURATION, pParam, pTimer, &pMeta->scanInfo.scanTimer, vgId, "scan-wal"); } } diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index f724e46c49..f24f800d9a 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -1364,11 +1364,11 @@ void streamMetaUpdateStageRole(SStreamMeta* pMeta, int64_t stage, bool isLeader) if (isLeader) { stInfo("vgId:%d update meta stage:%" PRId64 ", prev:%" PRId64 " leader:%d, start to send Hb, rid:%" PRId64, - pMeta->vgId, prevStage, stage, isLeader, pMeta->rid); + pMeta->vgId, stage, prevStage, isLeader, pMeta->rid); streamMetaStartHb(pMeta); } else { stInfo("vgId:%d update meta stage:%" PRId64 " prev:%" PRId64 " leader:%d sendMsg beforeClosing:%d", pMeta->vgId, - prevStage, stage, isLeader, pMeta->sendMsgBeforeClosing); + stage, prevStage, isLeader, pMeta->sendMsgBeforeClosing); } } From 8c0ead7c6113800c532b0b55a6148c100ce01e99 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sun, 23 Feb 2025 11:03:49 +0800 Subject: [PATCH 22/70] refactor: add some comments. --- source/dnode/vnode/src/vnd/vnodeSvr.c | 2 ++ 1 file changed, 2 insertions(+) diff --git a/source/dnode/vnode/src/vnd/vnodeSvr.c b/source/dnode/vnode/src/vnd/vnodeSvr.c index 7c57bb36a7..8ee5f09e77 100644 --- a/source/dnode/vnode/src/vnd/vnodeSvr.c +++ b/source/dnode/vnode/src/vnd/vnodeSvr.c @@ -931,6 +931,8 @@ int32_t vnodeProcessFetchMsg(SVnode *pVnode, SRpcMsg *pMsg, SQueueInfo *pInfo) { int32_t vnodeProcessStreamMsg(SVnode *pVnode, SRpcMsg *pMsg, SQueueInfo *pInfo) { vTrace("vgId:%d, msg:%p in stream queue is processing", pVnode->config.vgId, pMsg); + + // todo: NOTE: some command needs to run on follower, such as, stop_all_tasks if ((pMsg->msgType == TDMT_SCH_FETCH || pMsg->msgType == TDMT_VND_TABLE_META || pMsg->msgType == TDMT_VND_TABLE_CFG || pMsg->msgType == TDMT_VND_BATCH_META) && !syncIsReadyForRead(pVnode->sync)) { From a98d839c01cf656175b5522192ad8ff787f0d630 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sun, 23 Feb 2025 15:36:26 +0800 Subject: [PATCH 23/70] fix(stream): reset value. --- source/libs/stream/src/streamMeta.c | 3 +++ 1 file changed, 3 insertions(+) diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index f24f800d9a..f50f3da71a 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -428,6 +428,9 @@ int32_t streamMetaOpen(const char* path, void* ahandle, FTaskBuild buildTaskFn, TSDB_CHECK_NULL(pMeta->pTaskList, code, lino, _err, terrno); pMeta->scanInfo.scanSentinel = 0; + pMeta->scanInfo.lastScanTs = 0; + pMeta->scanInfo.tickCounter = 0; + pMeta->vgId = vgId; pMeta->ahandle = ahandle; pMeta->buildTaskFn = buildTaskFn; From 4ff74ab2ea42f4ac05dd6f9123a1cd716ba28d00 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sun, 23 Feb 2025 15:37:59 +0800 Subject: [PATCH 24/70] fix(stream): adjust log. --- source/dnode/vnode/src/tq/tqStreamTask.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/dnode/vnode/src/tq/tqStreamTask.c b/source/dnode/vnode/src/tq/tqStreamTask.c index 86d3e40fd5..737d539fe0 100644 --- a/source/dnode/vnode/src/tq/tqStreamTask.c +++ b/source/dnode/vnode/src/tq/tqStreamTask.c @@ -125,7 +125,7 @@ static void doStartScanWal(void* param, void* tmrId) { } if (pMeta->startInfo.startAllTasks) { - tqTrace("vgId:%d in restart procedure, not ready to scan wal", vgId); + tqDebug("vgId:%d in restart procedure, not ready to scan wal", vgId); goto _end; } From f4c3fdedbdf744aff0658b6a14b704aa1ef74bc9 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sun, 23 Feb 2025 16:18:19 +0800 Subject: [PATCH 25/70] fix(stream): add some logs. --- source/dnode/vnode/src/tq/tqStreamTask.c | 2 ++ 1 file changed, 2 insertions(+) diff --git a/source/dnode/vnode/src/tq/tqStreamTask.c b/source/dnode/vnode/src/tq/tqStreamTask.c index 737d539fe0..13994c3938 100644 --- a/source/dnode/vnode/src/tq/tqStreamTask.c +++ b/source/dnode/vnode/src/tq/tqStreamTask.c @@ -87,6 +87,8 @@ static void doStartScanWal(void* param, void* tmrId) { tmr_h pTimer = NULL; SBuildScanWalMsgParam* pParam = (SBuildScanWalMsgParam*)param; + tqDebug("start to do scan wal in tmr, metaRid:%" PRId64, pParam->metaId); + SStreamMeta* pMeta = taosAcquireRef(streamMetaRefPool, pParam->metaId); if (pMeta == NULL) { tqError("metaRid:%" PRId64 " not valid now, stream meta has been freed", pParam->metaId); From b4de2d16b6520fa99a4629913bbc5b227511ed8c Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sun, 23 Feb 2025 16:18:49 +0800 Subject: [PATCH 26/70] fix(stream): add some logs. --- source/dnode/vnode/src/tq/tqStreamTask.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/dnode/vnode/src/tq/tqStreamTask.c b/source/dnode/vnode/src/tq/tqStreamTask.c index 13994c3938..25c6ef2cf5 100644 --- a/source/dnode/vnode/src/tq/tqStreamTask.c +++ b/source/dnode/vnode/src/tq/tqStreamTask.c @@ -157,7 +157,7 @@ static void doStartScanWal(void* param, void* tmrId) { goto _end; } - tqTrace("vgId:%d create msg to start wal scan, numOfTasks:%d", vgId, numOfTasks); + tqDebug("vgId:%d create msg to start wal scan, numOfTasks:%d", vgId, numOfTasks); #if 0 // wait for the vnode is freed, and invalid read may occur. From 787590b6c97dd95aba7d5954a2c1445093a8a575 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sun, 23 Feb 2025 16:25:17 +0800 Subject: [PATCH 27/70] fix(stream): adjust init tmr position. --- source/dnode/vnode/src/tq/tqStreamTask.c | 13 ++++++------- 1 file changed, 6 insertions(+), 7 deletions(-) diff --git a/source/dnode/vnode/src/tq/tqStreamTask.c b/source/dnode/vnode/src/tq/tqStreamTask.c index 25c6ef2cf5..bb1bedd022 100644 --- a/source/dnode/vnode/src/tq/tqStreamTask.c +++ b/source/dnode/vnode/src/tq/tqStreamTask.c @@ -97,6 +97,12 @@ static void doStartScanWal(void* param, void* tmrId) { } vgId = pMeta->vgId; + code = streamTimerGetInstance(&pTimer); + if (code) { + tqFatal("vgId:%d failed to get tmr ctrl during sched scan wal, not scan wal, code:%s", vgId, tstrerror(code)); + taosMemoryFree(pParam); + return; + } if (pMeta->closeFlag) { code = taosReleaseRef(streamMetaRefPool, pParam->metaId); @@ -131,13 +137,6 @@ static void doStartScanWal(void* param, void* tmrId) { goto _end; } - code = streamTimerGetInstance(&pTimer); - if (code) { - tqFatal("vgId:%d failed to get tmr ctrl during sched scan wal, not scan wal, code:%s", vgId, tstrerror(code)); - taosMemoryFree(pParam); - return; - } - if (!waitEnoughDuration(pMeta)) { streamTmrStart(doStartScanWal, SCAN_WAL_IDLE_DURATION, pParam, pTimer, &pMeta->scanInfo.scanTimer, vgId, "scan-wal"); From c7d9a1cd733647630e6ecdd4d22e1666e185feb0 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 24 Feb 2025 13:48:21 +0800 Subject: [PATCH 28/70] fix(stream): fix memory leak. --- source/dnode/mnode/impl/src/mndStream.c | 2 ++ 1 file changed, 2 insertions(+) diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index e14cc230e3..55974eb9c7 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -1298,6 +1298,8 @@ static int32_t mndProcessStreamCheckpoint(SRpcMsg *pReq) { return TSDB_CODE_SUCCESS; } + taosArrayDestroy(pLongChkpts); + while ((pIter = sdbFetch(pSdb, SDB_STREAM, pIter, (void **)&pStream)) != NULL) { int64_t duration = now - pStream->checkpointFreq; if (duration < tsStreamCheckpointInterval * 1000) { From e46a063c2f63bc483264dc9a9ed9024499956bd8 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 25 Feb 2025 14:29:49 +0800 Subject: [PATCH 29/70] fix(stream): fix memory leak in checkpt queue. --- source/libs/stream/src/streamQueue.c | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/source/libs/stream/src/streamQueue.c b/source/libs/stream/src/streamQueue.c index c7b0bc8f11..300b3bdb4d 100644 --- a/source/libs/stream/src/streamQueue.c +++ b/source/libs/stream/src/streamQueue.c @@ -32,11 +32,12 @@ typedef struct SQueueReader { static bool streamTaskExtractAvailableToken(STokenBucket* pBucket, const char* id); static void streamTaskPutbackToken(STokenBucket* pBucket); static void streamTaskConsumeQuota(STokenBucket* pBucket, int32_t bytes); +static void streamQueueNextItemInSourceQ(SStreamQueue* pQueue, SStreamQueueItem** pItem, ETaskStatus status, const char* id); static void streamQueueCleanup(SStreamQueue* pQueue) { SStreamQueueItem* qItem = NULL; while (1) { - streamQueueNextItem(pQueue, &qItem); + streamQueueNextItemInSourceQ(pQueue, &qItem, TASK_STATUS__READY, NULL); if (qItem == NULL) { break; } @@ -250,7 +251,7 @@ EExtractDataCode streamTaskGetDataFromInputQ(SStreamTask* pTask, SStreamQueueIte while (1) { ETaskStatus status = streamTaskGetStatus(pTask).state; - if (status == TASK_STATUS__PAUSE || status == TASK_STATUS__STOP || status == TASK_STATUS__STOP) { + if (status == TASK_STATUS__PAUSE || status == TASK_STATUS__STOP) { stDebug("s-task:%s task should pause/stop, extract input blocks:%d", id, *numOfBlocks); return EXEC_CONTINUE; } From 7ff3a5968a61b264c4076a3ce4967e8f20cee06b Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 26 Feb 2025 00:13:44 +0800 Subject: [PATCH 30/70] fix(stream): fix memory leak. --- source/libs/stream/src/streamCheckpoint.c | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/source/libs/stream/src/streamCheckpoint.c b/source/libs/stream/src/streamCheckpoint.c index 0a43baced8..a0237222c0 100644 --- a/source/libs/stream/src/streamCheckpoint.c +++ b/source/libs/stream/src/streamCheckpoint.c @@ -1581,6 +1581,11 @@ int32_t streamTaskSendNegotiateChkptIdMsg(SStreamTask* pTask) { streamFreeTaskState(pTask, p); pTask->pBackend = NULL; } + + if (pTask->exec.pExecutor != NULL) { + qDestroyTask(pTask->exec.pExecutor); + pTask->exec.pExecutor = NULL; + } return 0; } From 45dfa0f12eb6d5c8ff69571d95c3df2d881e964a Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 27 Feb 2025 14:51:26 +0800 Subject: [PATCH 31/70] refactor(stream): create task epset update trans out of lock, to avoid blocking mnode-read thread. --- source/dnode/mnode/impl/src/mndStream.c | 134 ++++++++++++++---------- 1 file changed, 80 insertions(+), 54 deletions(-) diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index 55974eb9c7..220319122c 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -1901,15 +1901,16 @@ static int32_t mndProcessResumeStreamReq(SRpcMsg *pReq) { return TSDB_CODE_ACTION_IN_PROGRESS; } -static int32_t mndProcessVgroupChange(SMnode *pMnode, SVgroupChangeInfo *pChangeInfo, bool includeAllNodes) { +static int32_t mndProcessVgroupChange(SMnode *pMnode, SVgroupChangeInfo *pChangeInfo, bool includeAllNodes, STrans** pUpdateTrans) { SSdb *pSdb = pMnode->pSdb; - SStreamObj *pStream = NULL; void *pIter = NULL; STrans *pTrans = NULL; int32_t code = 0; + *pUpdateTrans = NULL; // conflict check for nodeUpdate trans, here we randomly chose one stream to add into the trans pool while (1) { + SStreamObj *pStream = NULL; pIter = sdbFetch(pSdb, SDB_STREAM, pIter, (void **)&pStream); if (pIter == NULL) { break; @@ -1926,6 +1927,7 @@ static int32_t mndProcessVgroupChange(SMnode *pMnode, SVgroupChangeInfo *pChange } while (1) { + SStreamObj *pStream = NULL; pIter = sdbFetch(pSdb, SDB_STREAM, pIter, (void **)&pStream); if (pIter == NULL) { break; @@ -1946,7 +1948,7 @@ static int32_t mndProcessVgroupChange(SMnode *pMnode, SVgroupChangeInfo *pChange void *p1 = taosHashGet(pChangeInfo->pDBMap, pStream->targetDb, strlen(pStream->targetDb)); void *p2 = taosHashGet(pChangeInfo->pDBMap, pStream->sourceDb, strlen(pStream->sourceDb)); if (p1 == NULL && p2 == NULL) { - mDebug("stream:0x%" PRIx64 " %s not involved nodeUpdate, ignore", pStream->uid, pStream->name); + mDebug("stream:0x%" PRIx64 " %s not involved in nodeUpdate, ignore", pStream->uid, pStream->name); sdbRelease(pSdb, pStream); continue; } @@ -1981,20 +1983,7 @@ static int32_t mndProcessVgroupChange(SMnode *pMnode, SVgroupChangeInfo *pChange } // no need to build the trans to handle the vgroup update - if (pTrans == NULL) { - return 0; - } - - code = mndTransPrepare(pMnode, pTrans); - if (code != TSDB_CODE_SUCCESS && code != TSDB_CODE_ACTION_IN_PROGRESS) { - mError("trans:%d, failed to prepare update stream trans since %s", pTrans->id, tstrerror(code)); - sdbRelease(pMnode->pSdb, pStream); - mndTransDrop(pTrans); - return code; - } - - sdbRelease(pMnode->pSdb, pStream); - mndTransDrop(pTrans); + *pUpdateTrans = pTrans; return code; } @@ -2076,7 +2065,7 @@ static int32_t refreshNodeListFromExistedStreams(SMnode *pMnode, SArray *pNodeLi taosHashCleanup(pHash); - mDebug("numOfNodes:%d for stream after extract nodeInfo from stream", (int32_t)taosArrayGetSize(pNodeList)); + mDebug("numOfvNodes:%d get after extracting nodeInfo from all streams", (int32_t)taosArrayGetSize(pNodeList)); return code; } @@ -2100,14 +2089,49 @@ static void addAllDbsIntoHashmap(SHashObj *pDBMap, SSdb *pSdb) { } } +static int32_t doProcessNodeCheckHelp(SArray *pNodeSnapshot, SMnode *pMnode, SVgroupChangeInfo *pChangeInfo, + bool *pUpdateAllVgroups) { + int32_t code = removeExpiredNodeEntryAndTaskInBuf(pNodeSnapshot); + if (code) { + mDebug("failed to remove expired node entry in buf, code:%s", tstrerror(code)); + return code; + } + + code = mndFindChangedNodeInfo(pMnode, execInfo.pNodeList, pNodeSnapshot, pChangeInfo); + if (code) { + mDebug("failed to find changed vnode(s) during vnode(s) check, code:%s", tstrerror(code)); + return code; + } + + { + if (execInfo.role == NODE_ROLE_LEADER && execInfo.switchFromFollower) { + mInfo("rollback all stream due to mnode leader/follower switch by using nodeUpdate trans"); + *pUpdateAllVgroups = true; + execInfo.switchFromFollower = false; // reset the flag + addAllDbsIntoHashmap(pChangeInfo->pDBMap, pMnode->pSdb); + } + } + + if (taosArrayGetSize(pChangeInfo->pUpdateNodeList) > 0 || (*pUpdateAllVgroups)) { + // kill current active checkpoint transaction, since the transaction is vnode wide. + killAllCheckpointTrans(pMnode, pChangeInfo); + } else { + mDebug("no update found in vnode(s) list"); + } + + return code; +} + // this function runs by only one thread, so it is not multi-thread safe static int32_t mndProcessNodeCheckReq(SRpcMsg *pMsg) { - int32_t code = 0; - bool allReady = true; - SArray *pNodeSnapshot = NULL; - SMnode *pMnode = pMsg->info.node; - int64_t ts = taosGetTimestampSec(); - bool updateAllVgroups = false; + int32_t code = 0; + bool allReady = true; + SArray *pNodeSnapshot = NULL; + SMnode *pMnode = pMsg->info.node; + int64_t tsms = taosGetTimestampMs(); + int64_t ts = tsms / 1000; + bool updateAllVgroups = false; + SVgroupChangeInfo changeInfo = {0}; int32_t old = atomic_val_compare_exchange_32(&mndNodeCheckSentinel, 0, 1); if (old != 0) { @@ -2115,7 +2139,7 @@ static int32_t mndProcessNodeCheckReq(SRpcMsg *pMsg) { return 0; } - mDebug("start to do node changing check"); + mDebug("start to do node changing check, ts:%" PRId64, tsms); streamMutexLock(&execInfo.lock); int32_t numOfNodes = extractStreamNodeList(pMnode); @@ -2141,58 +2165,60 @@ static int32_t mndProcessNodeCheckReq(SRpcMsg *pMsg) { } streamMutexLock(&execInfo.lock); + code = doProcessNodeCheckHelp(pNodeSnapshot, pMnode, &changeInfo, &updateAllVgroups); + streamMutexUnlock(&execInfo.lock); - code = removeExpiredNodeEntryAndTaskInBuf(pNodeSnapshot); if (code) { goto _end; } - SVgroupChangeInfo changeInfo = {0}; - code = mndFindChangedNodeInfo(pMnode, execInfo.pNodeList, pNodeSnapshot, &changeInfo); - if (code) { - goto _end; - } - - { - if (execInfo.role == NODE_ROLE_LEADER && execInfo.switchFromFollower) { - mInfo("rollback all stream due to mnode leader/follower switch by using nodeUpdate trans"); - updateAllVgroups = true; - execInfo.switchFromFollower = false; // reset the flag - addAllDbsIntoHashmap(changeInfo.pDBMap, pMnode->pSdb); - } - } - if (taosArrayGetSize(changeInfo.pUpdateNodeList) > 0 || updateAllVgroups) { - // kill current active checkpoint transaction, since the transaction is vnode wide. - killAllCheckpointTrans(pMnode, &changeInfo); - code = mndProcessVgroupChange(pMnode, &changeInfo, updateAllVgroups); + mDebug("vnode(s) change detected, build trans to update stream task epsets"); + + STrans *pTrans = NULL; + + streamMutexLock(&execInfo.lock); + code = mndProcessVgroupChange(pMnode, &changeInfo, updateAllVgroups, &pTrans); + streamMutexUnlock(&execInfo.lock); + + // NOTE: sync trans out of lock + if (code == 0 && pTrans != NULL) { + code = mndTransPrepare(pMnode, pTrans); + if (code != TSDB_CODE_SUCCESS && code != TSDB_CODE_ACTION_IN_PROGRESS) { + mError("trans:%d, failed to prepare update stream trans since %s", pTrans->id, tstrerror(code)); + } + + mndTransDrop(pTrans); + } // keep the new vnode snapshot if success if (code == TSDB_CODE_SUCCESS || code == TSDB_CODE_ACTION_IN_PROGRESS) { + streamMutexLock(&execInfo.lock); + code = refreshNodeListFromExistedStreams(pMnode, execInfo.pNodeList); + int32_t num = (int)taosArrayGetSize(execInfo.pNodeList); + if (code == 0) { + execInfo.ts = ts; + mDebug("create trans successfully, update cached node list, numOfNodes:%d", num); + } + + streamMutexUnlock(&execInfo.lock); + if (code) { mError("failed to extract node list from stream, code:%s", tstrerror(code)); goto _end; } - - execInfo.ts = ts; - mDebug("create trans successfully, update cached node list, numOfNodes:%d", - (int)taosArrayGetSize(execInfo.pNodeList)); - } else { - mError("unexpected code during create nodeUpdate trans, code:%s", tstrerror(code)); } - } else { - mDebug("no update found in nodeList"); } mndDestroyVgroupChangeInfo(&changeInfo); _end: - streamMutexUnlock(&execInfo.lock); taosArrayDestroy(pNodeSnapshot); - mDebug("end to do stream task node change checking"); + mDebug("end to do stream task node change checking, elapsed time:%" PRId64 "ms", taosGetTimestampMs() - tsms); atomic_store_32(&mndNodeCheckSentinel, 0); + return 0; } From 77a49c18547ef2a2470b8d23d70d5938e1da7f53 Mon Sep 17 00:00:00 2001 From: dmchen Date: Wed, 26 Feb 2025 16:55:06 +0800 Subject: [PATCH 32/70] fix/remove-sync-heartbeat-lock --- source/libs/sync/inc/syncPipeline.h | 1 + source/libs/sync/src/syncPipeline.c | 16 +++++++++++++--- 2 files changed, 14 insertions(+), 3 deletions(-) diff --git a/source/libs/sync/inc/syncPipeline.h b/source/libs/sync/inc/syncPipeline.h index eeb24d2f16..147f8a67ae 100644 --- a/source/libs/sync/inc/syncPipeline.h +++ b/source/libs/sync/inc/syncPipeline.h @@ -40,6 +40,7 @@ typedef struct SSyncLogReplMgr { int32_t retryBackoff; int32_t peerId; int32_t sendCount; + TdThreadMutex mutex; } SSyncLogReplMgr; typedef struct SSyncLogBufEntry { diff --git a/source/libs/sync/src/syncPipeline.c b/source/libs/sync/src/syncPipeline.c index 18252db9ee..ceaba1e643 100644 --- a/source/libs/sync/src/syncPipeline.c +++ b/source/libs/sync/src/syncPipeline.c @@ -1140,26 +1140,29 @@ int32_t syncLogReplRecover(SSyncLogReplMgr* pMgr, SSyncNode* pNode, SyncAppendEn int32_t syncLogReplProcessHeartbeatReply(SSyncLogReplMgr* pMgr, SSyncNode* pNode, SyncHeartbeatReply* pMsg) { SSyncLogBuffer* pBuf = pNode->pLogBuf; - (void)taosThreadMutexLock(&pBuf->mutex); + (void)taosThreadMutexLock(&pMgr->mutex); if (pMsg->startTime != 0 && pMsg->startTime != pMgr->peerStartTime) { sInfo("vgId:%d, reset sync log repl in heartbeat. peer:%" PRIx64 ", start time:%" PRId64 ", old:%" PRId64 "", pNode->vgId, pMsg->srcId.addr, pMsg->startTime, pMgr->peerStartTime); syncLogReplReset(pMgr); pMgr->peerStartTime = pMsg->startTime; } - (void)taosThreadMutexUnlock(&pBuf->mutex); + (void)taosThreadMutexUnlock(&pMgr->mutex); return 0; } int32_t syncLogReplProcessReply(SSyncLogReplMgr* pMgr, SSyncNode* pNode, SyncAppendEntriesReply* pMsg) { SSyncLogBuffer* pBuf = pNode->pLogBuf; - (void)taosThreadMutexLock(&pBuf->mutex); + (void)taosThreadMutexLock(&pMgr->mutex); if (pMsg->startTime != pMgr->peerStartTime) { sInfo("vgId:%d, reset sync log repl in appendlog reply. peer:%" PRIx64 ", start time:%" PRId64 ", old:%" PRId64, pNode->vgId, pMsg->srcId.addr, pMsg->startTime, pMgr->peerStartTime); syncLogReplReset(pMgr); pMgr->peerStartTime = pMsg->startTime; } + (void)taosThreadMutexUnlock(&pMgr->mutex); + + (void)taosThreadMutexLock(&pBuf->mutex); int32_t code = 0; if (pMgr->restored) { @@ -1324,6 +1327,12 @@ SSyncLogReplMgr* syncLogReplCreate() { return NULL; } + int32_t code = taosThreadMutexInit(&pMgr->mutex, NULL); + if (code) { + terrno = code; + return NULL; + } + return pMgr; } @@ -1331,6 +1340,7 @@ void syncLogReplDestroy(SSyncLogReplMgr* pMgr) { if (pMgr == NULL) { return; } + taosThreadMutexDestroy(&pMgr->mutex); taosMemoryFree(pMgr); return; } From 2a037f7ef5e70fdc6f1fc8ced487183f04d79154 Mon Sep 17 00:00:00 2001 From: dmchen Date: Wed, 26 Feb 2025 18:41:17 +0800 Subject: [PATCH 33/70] fix/remove-sync-heartbeat-lock-fix-check --- source/libs/sync/src/syncPipeline.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/libs/sync/src/syncPipeline.c b/source/libs/sync/src/syncPipeline.c index ceaba1e643..8d81a03344 100644 --- a/source/libs/sync/src/syncPipeline.c +++ b/source/libs/sync/src/syncPipeline.c @@ -1340,7 +1340,7 @@ void syncLogReplDestroy(SSyncLogReplMgr* pMgr) { if (pMgr == NULL) { return; } - taosThreadMutexDestroy(&pMgr->mutex); + (void)taosThreadMutexDestroy(&pMgr->mutex); taosMemoryFree(pMgr); return; } From 55f58d743f390eafdb2c1a0288d91b80d45d8227 Mon Sep 17 00:00:00 2001 From: dmchen Date: Thu, 27 Feb 2025 11:02:18 +0800 Subject: [PATCH 34/70] fix/move-status-to-new-queue --- include/common/tmsgcb.h | 1 + source/dnode/mgmt/mgmt_dnode/src/dmHandle.c | 7 ++++--- source/dnode/mgmt/mgmt_mnode/inc/mmInt.h | 2 ++ source/dnode/mgmt/mgmt_mnode/src/mmHandle.c | 2 +- source/dnode/mgmt/mgmt_mnode/src/mmWorker.c | 20 +++++++++++++++++++ source/dnode/mgmt/mgmt_qnode/src/qmWorker.c | 1 + source/dnode/mnode/impl/test/trans/trans2.cpp | 3 ++- source/libs/sync/inc/syncUtil.h | 4 ++-- 8 files changed, 33 insertions(+), 7 deletions(-) diff --git a/include/common/tmsgcb.h b/include/common/tmsgcb.h index 3ce03d292f..4af223b537 100644 --- a/include/common/tmsgcb.h +++ b/include/common/tmsgcb.h @@ -31,6 +31,7 @@ typedef enum { QUERY_QUEUE, FETCH_QUEUE, READ_QUEUE, + STATUS_QUEUE, WRITE_QUEUE, APPLY_QUEUE, SYNC_QUEUE, diff --git a/source/dnode/mgmt/mgmt_dnode/src/dmHandle.c b/source/dnode/mgmt/mgmt_dnode/src/dmHandle.c index 637713d2f9..ef0106cdbb 100644 --- a/source/dnode/mgmt/mgmt_dnode/src/dmHandle.c +++ b/source/dnode/mgmt/mgmt_dnode/src/dmHandle.c @@ -267,7 +267,7 @@ void dmSendStatusReq(SDnodeMgmt *pMgmt) { code = rpcSendRecvWithTimeout(pMgmt->msgCb.statusRpc, &epSet, &rpcMsg, &rpcRsp, &epUpdated, tsStatusInterval * 5 * 1000); if (code != 0) { - dError("failed to send status req since %s", tstrerror(code)); + dError("failed to SendRecv with timeout %d status req since %s", tsStatusInterval * 5 * 1000, tstrerror(code)); return; } @@ -275,7 +275,8 @@ void dmSendStatusReq(SDnodeMgmt *pMgmt) { dmRotateMnodeEpSet(pMgmt->pData); char tbuf[512]; dmEpSetToStr(tbuf, sizeof(tbuf), &epSet); - dError("failed to send status req since %s, epSet:%s, inUse:%d", tstrerror(rpcRsp.code), tbuf, epSet.inUse); + dInfo("Rotate mnode ep set since failed to SendRecv status req %s, epSet:%s, inUse:%d", tstrerror(rpcRsp.code), + tbuf, epSet.inUse); } else { if (epUpdated == 1) { dmSetMnodeEpSet(pMgmt->pData, &epSet); @@ -403,7 +404,7 @@ void dmSendConfigReq(SDnodeMgmt *pMgmt) { code = rpcSendRecvWithTimeout(pMgmt->msgCb.statusRpc, &epSet, &rpcMsg, &rpcRsp, &epUpdated, tsStatusInterval * 5 * 1000); if (code != 0) { - dError("failed to send status req since %s", tstrerror(code)); + dError("failed to SendRecv config req with timeout %d since %s", tsStatusInterval * 5 * 1000, tstrerror(code)); return; } if (rpcRsp.code != 0) { diff --git a/source/dnode/mgmt/mgmt_mnode/inc/mmInt.h b/source/dnode/mgmt/mgmt_mnode/inc/mmInt.h index bc2977fe31..2d7aa35c13 100644 --- a/source/dnode/mgmt/mgmt_mnode/inc/mmInt.h +++ b/source/dnode/mgmt/mgmt_mnode/inc/mmInt.h @@ -32,6 +32,7 @@ typedef struct SMnodeMgmt { SSingleWorker queryWorker; SSingleWorker fetchWorker; SSingleWorker readWorker; + SSingleWorker statusWorker; SSingleWorker writeWorker; SSingleWorker arbWorker; SSingleWorker syncWorker; @@ -58,6 +59,7 @@ int32_t mmPutMsgToArbQueue(SMnodeMgmt *pMgmt, SRpcMsg *pMsg); int32_t mmPutMsgToSyncQueue(SMnodeMgmt *pMgmt, SRpcMsg *pMsg); int32_t mmPutMsgToSyncRdQueue(SMnodeMgmt *pMgmt, SRpcMsg *pMsg); int32_t mmPutMsgToReadQueue(SMnodeMgmt *pMgmt, SRpcMsg *pMsg); +int32_t mmPutMsgToStatusQueue(SMnodeMgmt *pMgmt, SRpcMsg *pMsg); int32_t mmPutMsgToQueryQueue(SMnodeMgmt *pMgmt, SRpcMsg *pMsg); int32_t mmPutMsgToFetchQueue(SMnodeMgmt *pMgmt, SRpcMsg *pMsg); int32_t mmPutMsgToQueue(SMnodeMgmt *pMgmt, EQueueType qtype, SRpcMsg *pRpc); diff --git a/source/dnode/mgmt/mgmt_mnode/src/mmHandle.c b/source/dnode/mgmt/mgmt_mnode/src/mmHandle.c index 80ef0d31de..0031042497 100644 --- a/source/dnode/mgmt/mgmt_mnode/src/mmHandle.c +++ b/source/dnode/mgmt/mgmt_mnode/src/mmHandle.c @@ -202,7 +202,7 @@ SArray *mmGetMsgHandles() { if (dmSetMgmtHandle(pArray, TDMT_MND_KILL_QUERY, mmPutMsgToWriteQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_MND_KILL_CONN, mmPutMsgToWriteQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_MND_HEARTBEAT, mmPutMsgToReadQueue, 0) == NULL) goto _OVER; - if (dmSetMgmtHandle(pArray, TDMT_MND_STATUS, mmPutMsgToReadQueue, 0) == NULL) goto _OVER; + if (dmSetMgmtHandle(pArray, TDMT_MND_STATUS, mmPutMsgToStatusQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_MND_CONFIG, mmPutMsgToWriteQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_MND_NOTIFY, mmPutMsgToReadQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_MND_SYSTABLE_RETRIEVE, mmPutMsgToReadQueue, 0) == NULL) goto _OVER; diff --git a/source/dnode/mgmt/mgmt_mnode/src/mmWorker.c b/source/dnode/mgmt/mgmt_mnode/src/mmWorker.c index cfdc09f29e..cb82ddae2e 100644 --- a/source/dnode/mgmt/mgmt_mnode/src/mmWorker.c +++ b/source/dnode/mgmt/mgmt_mnode/src/mmWorker.c @@ -133,6 +133,10 @@ int32_t mmPutMsgToReadQueue(SMnodeMgmt *pMgmt, SRpcMsg *pMsg) { return mmPutMsgToWorker(pMgmt, &pMgmt->readWorker, pMsg); } +int32_t mmPutMsgToStatusQueue(SMnodeMgmt *pMgmt, SRpcMsg *pMsg) { + return mmPutMsgToWorker(pMgmt, &pMgmt->statusWorker, pMsg); +} + int32_t mmPutMsgToQueryQueue(SMnodeMgmt *pMgmt, SRpcMsg *pMsg) { int32_t code = 0; if (NULL == pMgmt->pMnode) { @@ -172,6 +176,9 @@ int32_t mmPutMsgToQueue(SMnodeMgmt *pMgmt, EQueueType qtype, SRpcMsg *pRpc) { case READ_QUEUE: pWorker = &pMgmt->readWorker; break; + case STATUS_QUEUE: + pWorker = &pMgmt->statusWorker; + break; case ARB_QUEUE: pWorker = &pMgmt->arbWorker; break; @@ -246,6 +253,18 @@ int32_t mmStartWorker(SMnodeMgmt *pMgmt) { return code; } + SSingleWorkerCfg stautsCfg = { + .min = 1, + .max = 1, + .name = "mnode-status", + .fp = (FItem)mmProcessRpcMsg, + .param = pMgmt, + }; + if ((code = tSingleWorkerInit(&pMgmt->statusWorker, &stautsCfg)) != 0) { + dError("failed to start mnode-status worker since %s", tstrerror(code)); + return code; + } + SSingleWorkerCfg wCfg = { .min = 1, .max = 1, @@ -304,6 +323,7 @@ void mmStopWorker(SMnodeMgmt *pMgmt) { tSingleWorkerCleanup(&pMgmt->queryWorker); tSingleWorkerCleanup(&pMgmt->fetchWorker); tSingleWorkerCleanup(&pMgmt->readWorker); + tSingleWorkerCleanup(&pMgmt->statusWorker); tSingleWorkerCleanup(&pMgmt->writeWorker); tSingleWorkerCleanup(&pMgmt->arbWorker); tSingleWorkerCleanup(&pMgmt->syncWorker); diff --git a/source/dnode/mgmt/mgmt_qnode/src/qmWorker.c b/source/dnode/mgmt/mgmt_qnode/src/qmWorker.c index ab5b70079a..950da46a40 100644 --- a/source/dnode/mgmt/mgmt_qnode/src/qmWorker.c +++ b/source/dnode/mgmt/mgmt_qnode/src/qmWorker.c @@ -71,6 +71,7 @@ int32_t qmPutRpcMsgToQueue(SQnodeMgmt *pMgmt, EQueueType qtype, SRpcMsg *pRpc) { code = taosWriteQitem(pMgmt->queryWorker.queue, pMsg); return code; case READ_QUEUE: + case STATUS_QUEUE: case FETCH_QUEUE: dTrace("msg:%p, is created and will put into qnode-fetch queue, len:%d", pMsg, pRpc->contLen); code = taosWriteQitem(pMgmt->fetchWorker.queue, pMsg); diff --git a/source/dnode/mnode/impl/test/trans/trans2.cpp b/source/dnode/mnode/impl/test/trans/trans2.cpp index b73f07c778..e520fce111 100644 --- a/source/dnode/mnode/impl/test/trans/trans2.cpp +++ b/source/dnode/mnode/impl/test/trans/trans2.cpp @@ -65,7 +65,8 @@ class MndTestTrans2 : public ::testing::Test { msgCb.sendRspFp = sendRsp; msgCb.queueFps[SYNC_QUEUE] = putToQueue; msgCb.queueFps[WRITE_QUEUE] = putToQueue; - msgCb.queueFps[READ_QUEUE] = putToQueue; + msgCb.queueFps[READ_QUEUE] = putToQueue; + msgCb.queueFps[STATUS_QUEUE] = putToQueue; msgCb.mgmt = (SMgmtWrapper *)(&msgCb); // hack tmsgSetDefault(&msgCb); diff --git a/source/libs/sync/inc/syncUtil.h b/source/libs/sync/inc/syncUtil.h index 7b71491f47..e7110c0964 100644 --- a/source/libs/sync/inc/syncUtil.h +++ b/source/libs/sync/inc/syncUtil.h @@ -50,8 +50,8 @@ extern "C" { #define sNError(pNode, ...) if (sDebugFlag & DEBUG_ERROR) { syncPrintNodeLog("SYN ERROR ", DEBUG_ERROR, 255, true, pNode, __VA_ARGS__); } #define sNWarn(pNode, ...) if (sDebugFlag & DEBUG_WARN) { syncPrintNodeLog("SYN WARN ", DEBUG_WARN, 255, true, pNode, __VA_ARGS__); } #define sNInfo(pNode, ...) if (sDebugFlag & DEBUG_INFO) { syncPrintNodeLog("SYN ", DEBUG_INFO, 255, true, pNode, __VA_ARGS__); } -#define sNDebug(pNode, ...) if (sDebugFlag & DEBUG_DEBUG) { syncPrintNodeLog("SYN ", DEBUG_DEBUG, sDebugFlag, false, pNode, __VA_ARGS__); } -#define sNTrace(pNode, ...) if (sDebugFlag & DEBUG_TRACE) { syncPrintNodeLog("SYN ", DEBUG_TRACE, sDebugFlag, false, pNode, __VA_ARGS__); } +#define sNDebug(pNode, ...) if (sDebugFlag & DEBUG_DEBUG) { syncPrintNodeLog("SYN ", DEBUG_DEBUG, sDebugFlag, true, pNode, __VA_ARGS__); } +#define sNTrace(pNode, ...) if (sDebugFlag & DEBUG_TRACE) { syncPrintNodeLog("SYN ", DEBUG_TRACE, sDebugFlag, true, pNode, __VA_ARGS__); } #define sSFatal(pSender, ...) if (sDebugFlag & DEBUG_FATAL) { syncPrintSnapshotSenderLog("SYN FATAL ", DEBUG_FATAL, 255, pSender, __VA_ARGS__); } #define sSError(pSender, ...) if (sDebugFlag & DEBUG_ERROR) { syncPrintSnapshotSenderLog("SYN ERROR ", DEBUG_ERROR, 255, pSender, __VA_ARGS__); } From d435bbae1de61ae4966b25586cba30b2e476e05a Mon Sep 17 00:00:00 2001 From: dmchen Date: Thu, 27 Feb 2025 15:22:33 +0800 Subject: [PATCH 35/70] fix:add-status-timestamp --- include/common/tmsg.h | 1 + include/common/ttime.h | 2 +- source/common/src/msg/tmsg.c | 6 +++++ source/common/src/ttime.c | 28 +++++++++++++++++++++ source/dnode/mgmt/mgmt_dnode/src/dmHandle.c | 1 + source/dnode/mnode/impl/src/mndDnode.c | 8 ++++-- source/libs/sync/src/syncUtil.c | 2 +- 7 files changed, 44 insertions(+), 4 deletions(-) diff --git a/include/common/tmsg.h b/include/common/tmsg.h index 0b6a8b3f1b..38fdc52513 100644 --- a/include/common/tmsg.h +++ b/include/common/tmsg.h @@ -1839,6 +1839,7 @@ typedef struct { int32_t statusSeq; int64_t ipWhiteVer; int64_t analVer; + int64_t timestamp; } SStatusReq; int32_t tSerializeSStatusReq(void* buf, int32_t bufLen, SStatusReq* pReq); diff --git a/include/common/ttime.h b/include/common/ttime.h index 47875752d5..77a3041b2b 100644 --- a/include/common/ttime.h +++ b/include/common/ttime.h @@ -82,7 +82,7 @@ int32_t convertStringToTimestamp(int16_t type, char* inputData, int64_t timePrec int32_t getDuration(int64_t val, char unit, int64_t* result, int32_t timePrecision); int32_t taosFormatUtcTime(char* buf, int32_t bufLen, int64_t ts, int32_t precision); - +char* formatTimestampLocal(char* buf, int64_t val, int precision); struct STm { struct tm tm; int64_t fsec; // in NANOSECOND diff --git a/source/common/src/msg/tmsg.c b/source/common/src/msg/tmsg.c index f1ebcb3f42..cc635885c0 100644 --- a/source/common/src/msg/tmsg.c +++ b/source/common/src/msg/tmsg.c @@ -1471,6 +1471,8 @@ int32_t tSerializeSStatusReq(void *buf, int32_t bufLen, SStatusReq *pReq) { TAOS_CHECK_EXIT(tEncodeI64(&encoder, pReq->analVer)); TAOS_CHECK_EXIT(tSerializeSMonitorParas(&encoder, &pReq->clusterCfg.monitorParas)); + TAOS_CHECK_EXIT(tEncodeI64(&encoder, pReq->timestamp)); + tEndEncode(&encoder); _exit: @@ -1600,6 +1602,10 @@ int32_t tDeserializeSStatusReq(void *buf, int32_t bufLen, SStatusReq *pReq) { TAOS_CHECK_EXIT(tDeserializeSMonitorParas(&decoder, &pReq->clusterCfg.monitorParas)); } + if (!tDecodeIsEnd(&decoder)) { + TAOS_CHECK_EXIT(tDecodeI64(&decoder, &pReq->timestamp)); + } + tEndDecode(&decoder); _exit: diff --git a/source/common/src/ttime.c b/source/common/src/ttime.c index 9746fea034..22ab106bb3 100644 --- a/source/common/src/ttime.c +++ b/source/common/src/ttime.c @@ -1000,6 +1000,34 @@ int32_t taosFormatUtcTime(char* buf, int32_t bufLen, int64_t t, int32_t precisio TAOS_RETURN(TSDB_CODE_SUCCESS); } +char* formatTimestampLocal(char* buf, int64_t val, int precision) { + time_t tt; + if (precision == TSDB_TIME_PRECISION_MICRO) { + tt = (time_t)(val / 1000000); + } + if (precision == TSDB_TIME_PRECISION_NANO) { + tt = (time_t)(val / 1000000000); + } else { + tt = (time_t)(val / 1000); + } + + struct tm tm; + if (taosLocalTime(&tt, &tm, NULL, 0, NULL) == NULL) { + return NULL; + } + size_t pos = taosStrfTime(buf, 32, "%Y-%m-%d %H:%M:%S", &tm); + + if (precision == TSDB_TIME_PRECISION_MICRO) { + sprintf(buf + pos, ".%06d", (int)(val % 1000000)); + } else if (precision == TSDB_TIME_PRECISION_NANO) { + sprintf(buf + pos, ".%09d", (int)(val % 1000000000)); + } else { + sprintf(buf + pos, ".%03d", (int)(val % 1000)); + } + + return buf; +} + int32_t taosTs2Tm(int64_t ts, int32_t precision, struct STm* tm, timezone_t tz) { tm->fsec = ts % TICK_PER_SECOND[precision] * (TICK_PER_SECOND[TSDB_TIME_PRECISION_NANO] / TICK_PER_SECOND[precision]); time_t t = ts / TICK_PER_SECOND[precision]; diff --git a/source/dnode/mgmt/mgmt_dnode/src/dmHandle.c b/source/dnode/mgmt/mgmt_dnode/src/dmHandle.c index ef0106cdbb..653a54592a 100644 --- a/source/dnode/mgmt/mgmt_dnode/src/dmHandle.c +++ b/source/dnode/mgmt/mgmt_dnode/src/dmHandle.c @@ -167,6 +167,7 @@ static void dmProcessStatusRsp(SDnodeMgmt *pMgmt, SRpcMsg *pRsp) { void dmSendStatusReq(SDnodeMgmt *pMgmt) { int32_t code = 0; SStatusReq req = {0}; + req.timestamp = taosGetTimestampMs(); dDebug("send status req to mnode, statusSeq:%d, begin to mgnt lock", pMgmt->statusSeq); (void)taosThreadRwlockRdlock(&pMgmt->pData->lock); diff --git a/source/dnode/mnode/impl/src/mndDnode.c b/source/dnode/mnode/impl/src/mndDnode.c index ca119191eb..dac38dc4db 100644 --- a/source/dnode/mnode/impl/src/mndDnode.c +++ b/source/dnode/mnode/impl/src/mndDnode.c @@ -756,8 +756,12 @@ static int32_t mndProcessStatusReq(SRpcMsg *pReq) { bool needCheck = !online || dnodeChanged || reboot || supportVnodesChanged || analVerChanged || pMnode->ipWhiteVer != statusReq.ipWhiteVer || encryptKeyChanged || enableWhiteListChanged; const STraceId *trace = &pReq->info.traceId; - mGTrace("dnode:%d, status received, accessTimes:%d check:%d online:%d reboot:%d changed:%d statusSeq:%d", pDnode->id, - pDnode->accessTimes, needCheck, online, reboot, dnodeChanged, statusReq.statusSeq); + char timestamp[TD_TIME_STR_LEN] = {0}; + if (mDebugFlag & DEBUG_TRACE) (void)formatTimestampLocal(timestamp, statusReq.timestamp, TSDB_TIME_PRECISION_MILLI); + mGTrace( + "dnode:%d, status received, accessTimes:%d check:%d online:%d reboot:%d changed:%d statusSeq:%d " + "timestamp:%s", + pDnode->id, pDnode->accessTimes, needCheck, online, reboot, dnodeChanged, statusReq.statusSeq, timestamp); if (reboot) { tsGrantHBInterval = GRANT_HEART_BEAT_MIN; diff --git a/source/libs/sync/src/syncUtil.c b/source/libs/sync/src/syncUtil.c index 38fd660cfd..5a2571a291 100644 --- a/source/libs/sync/src/syncUtil.c +++ b/source/libs/sync/src/syncUtil.c @@ -116,7 +116,7 @@ static void syncPrintTime(bool formatTime, int32_t* len, int64_t tsMs, int32_t i if (formatTime) { char pBuf[TD_TIME_STR_LEN] = {0}; if (tsMs > 0) { - if (taosFormatUtcTime(pBuf, TD_TIME_STR_LEN, tsMs, TSDB_TIME_PRECISION_MILLI) != 0) { + if (formatTimestampLocal(pBuf, tsMs, TSDB_TIME_PRECISION_MILLI) == NULL) { pBuf[0] = '\0'; } } From 2d4697d32f9c5a214d77f626748cfda0523f993f Mon Sep 17 00:00:00 2001 From: dmchen Date: Thu, 27 Feb 2025 19:05:06 +0800 Subject: [PATCH 36/70] fix:decouple get mload --- source/dnode/mgmt/mgmt_dnode/src/dmHandle.c | 23 ++++++++++++++------- source/dnode/mnode/impl/src/mndMain.c | 1 + 2 files changed, 17 insertions(+), 7 deletions(-) diff --git a/source/dnode/mgmt/mgmt_dnode/src/dmHandle.c b/source/dnode/mgmt/mgmt_dnode/src/dmHandle.c index 653a54592a..49f634443f 100644 --- a/source/dnode/mgmt/mgmt_dnode/src/dmHandle.c +++ b/source/dnode/mgmt/mgmt_dnode/src/dmHandle.c @@ -25,6 +25,7 @@ extern SConfig *tsCfg; SMonVloadInfo tsVinfo = {0}; +SMnodeLoad tsMLoad = {0}; static void dmUpdateDnodeCfg(SDnodeMgmt *pMgmt, SDnodeCfg *pCfg) { int32_t code = 0; @@ -208,23 +209,22 @@ void dmSendStatusReq(SDnodeMgmt *pMgmt) { memcpy(req.clusterCfg.charset, tsCharset, TD_LOCALE_LEN); (void)taosThreadRwlockUnlock(&pMgmt->pData->lock); - dDebug("send status req to mnode, statusSeq:%d, begin to get vnode loads", pMgmt->statusSeq); + dDebug("send status req to mnode, statusSeq:%d, begin to get vnode and loads", pMgmt->statusSeq); if (taosThreadMutexLock(&pMgmt->pData->statusInfolock) != 0) { dError("failed to lock status info lock"); return; } + req.pVloads = tsVinfo.pVloads; tsVinfo.pVloads = NULL; + + req.mload = tsMLoad; + if (taosThreadMutexUnlock(&pMgmt->pData->statusInfolock) != 0) { dError("failed to unlock status info lock"); return; } - dDebug("send status req to mnode, statusSeq:%d, begin to get mnode loads", pMgmt->statusSeq); - SMonMloadInfo minfo = {0}; - (*pMgmt->getMnodeLoadsFp)(&minfo); - req.mload = minfo.load; - dDebug("send status req to mnode, statusSeq:%d, begin to get qnode loads", pMgmt->statusSeq); (*pMgmt->getQnodeLoadsFp)(&req.qload); @@ -418,12 +418,18 @@ void dmSendConfigReq(SDnodeMgmt *pMgmt) { void dmUpdateStatusInfo(SDnodeMgmt *pMgmt) { SMonVloadInfo vinfo = {0}; dDebug("begin to get vnode loads"); - (*pMgmt->getVnodeLoadsFp)(&vinfo); + (*pMgmt->getVnodeLoadsFp)(&vinfo); // dmGetVnodeLoads + + dDebug("begin to get mnode loads"); + SMonMloadInfo minfo = {0}; + (*pMgmt->getMnodeLoadsFp)(&minfo); // dmGetMnodeLoads + dDebug("begin to lock status info"); if (taosThreadMutexLock(&pMgmt->pData->statusInfolock) != 0) { dError("failed to lock status info lock"); return; } + if (tsVinfo.pVloads == NULL) { tsVinfo.pVloads = vinfo.pVloads; vinfo.pVloads = NULL; @@ -431,6 +437,9 @@ void dmUpdateStatusInfo(SDnodeMgmt *pMgmt) { taosArrayDestroy(vinfo.pVloads); vinfo.pVloads = NULL; } + + tsMLoad = minfo.load; + if (taosThreadMutexUnlock(&pMgmt->pData->statusInfolock) != 0) { dError("failed to unlock status info lock"); return; diff --git a/source/dnode/mnode/impl/src/mndMain.c b/source/dnode/mnode/impl/src/mndMain.c index b241af5adb..09314c9e63 100644 --- a/source/dnode/mnode/impl/src/mndMain.c +++ b/source/dnode/mnode/impl/src/mndMain.c @@ -1174,6 +1174,7 @@ int32_t mndGetMonitorInfo(SMnode *pMnode, SMonClusterInfo *pClusterInfo, SMonVgr } int32_t mndGetLoad(SMnode *pMnode, SMnodeLoad *pLoad) { + mTrace("mnode get load"); SSyncState state = syncGetState(pMnode->syncMgmt.sync); pLoad->syncState = state.state; pLoad->syncRestore = state.restored; From 8c671f51fbe77f0f2bd00dd4ad5c369306714836 Mon Sep 17 00:00:00 2001 From: dmchen Date: Wed, 26 Feb 2025 16:55:06 +0800 Subject: [PATCH 37/70] refactor(stream): move the generate checkpoint procedure out of lock, to avoid blocking heartbeat, and resulting in leader/follower switch. --- source/libs/stream/src/streamCheckpoint.c | 5 ++++- source/libs/stream/src/streamExec.c | 4 +++- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/source/libs/stream/src/streamCheckpoint.c b/source/libs/stream/src/streamCheckpoint.c index a0237222c0..0d951a3ff1 100644 --- a/source/libs/stream/src/streamCheckpoint.c +++ b/source/libs/stream/src/streamCheckpoint.c @@ -910,9 +910,12 @@ int32_t streamTaskBuildCheckpoint(SStreamTask* pTask) { int64_t startTs = pTask->chkInfo.startTs; int64_t ckId = pTask->chkInfo.pActiveInfo->activeId; const char* id = pTask->id.idStr; - bool dropRelHTask = (streamTaskGetPrevStatus(pTask) == TASK_STATUS__HALT); SStreamMeta* pMeta = pTask->pMeta; + streamMutexLock(&pTask->lock); + bool dropRelHTask = (streamTaskGetPrevStatus(pTask) == TASK_STATUS__HALT); + streamMutexUnlock(&pTask->lock); + // sink task does not need to save the status, and generated the checkpoint if (pTask->info.taskLevel != TASK_LEVEL__SINK) { stDebug("s-task:%s level:%d start gen checkpoint, checkpointId:%" PRId64, id, pTask->info.taskLevel, ckId); diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index 54dce4947a..1015917f61 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -698,6 +698,8 @@ static int32_t doHandleChkptBlock(SStreamTask* pTask) { streamMutexLock(&pTask->lock); SStreamTaskState pState = streamTaskGetStatus(pTask); + streamMutexUnlock(&pTask->lock); + if (pState.state == TASK_STATUS__CK) { // todo other thread may change the status stDebug("s-task:%s checkpoint block received, set status:%s", id, pState.name); code = streamTaskBuildCheckpoint(pTask); // ignore this error msg, and continue @@ -715,7 +717,7 @@ static int32_t doHandleChkptBlock(SStreamTask* pTask) { } } - streamMutexUnlock(&pTask->lock); +// streamMutexUnlock(&pTask->lock); return code; } From 45cdd871dbe4fb022171421e652a30900eb8b415 Mon Sep 17 00:00:00 2001 From: dmchen Date: Fri, 28 Feb 2025 11:52:27 +0800 Subject: [PATCH 38/70] fix/remove-get-queue-size-when-print-heartbeat-log --- source/dnode/mgmt/mgmt_vnode/inc/vmInt.h | 2 +- source/dnode/mgmt/mgmt_vnode/src/vmFile.c | 18 +++++------ source/dnode/mgmt/mgmt_vnode/src/vmHandle.c | 12 ++++---- source/dnode/mgmt/mgmt_vnode/src/vmInt.c | 34 ++++++++++----------- source/libs/sync/src/syncUtil.c | 30 +++++++++++------- source/libs/transport/src/tmsgcb.c | 2 +- 6 files changed, 53 insertions(+), 45 deletions(-) diff --git a/source/dnode/mgmt/mgmt_vnode/inc/vmInt.h b/source/dnode/mgmt/mgmt_vnode/inc/vmInt.h index 583883590a..cb41bc0ea7 100644 --- a/source/dnode/mgmt/mgmt_vnode/inc/vmInt.h +++ b/source/dnode/mgmt/mgmt_vnode/inc/vmInt.h @@ -41,7 +41,7 @@ typedef struct SVnodeMgmt { SHashObj *runngingHash; SHashObj *closedHash; SHashObj *creatingHash; - TdThreadRwlock lock; + TdThreadRwlock hashLock; TdThreadMutex mutex; SVnodesStat state; STfs *pTfs; diff --git a/source/dnode/mgmt/mgmt_vnode/src/vmFile.c b/source/dnode/mgmt/mgmt_vnode/src/vmFile.c index dbef048c23..cb14155b1c 100644 --- a/source/dnode/mgmt/mgmt_vnode/src/vmFile.c +++ b/source/dnode/mgmt/mgmt_vnode/src/vmFile.c @@ -20,7 +20,7 @@ #define MAX_CONTENT_LEN 2 * 1024 * 1024 int32_t vmGetAllVnodeListFromHash(SVnodeMgmt *pMgmt, int32_t *numOfVnodes, SVnodeObj ***ppVnodes) { - (void)taosThreadRwlockRdlock(&pMgmt->lock); + (void)taosThreadRwlockRdlock(&pMgmt->hashLock); int32_t num = 0; int32_t size = taosHashGetSize(pMgmt->runngingHash); @@ -28,7 +28,7 @@ int32_t vmGetAllVnodeListFromHash(SVnodeMgmt *pMgmt, int32_t *numOfVnodes, SVnod size += closedSize; SVnodeObj **pVnodes = taosMemoryCalloc(size, sizeof(SVnodeObj *)); if (pVnodes == NULL) { - (void)taosThreadRwlockUnlock(&pMgmt->lock); + (void)taosThreadRwlockUnlock(&pMgmt->hashLock); return terrno; } @@ -60,7 +60,7 @@ int32_t vmGetAllVnodeListFromHash(SVnodeMgmt *pMgmt, int32_t *numOfVnodes, SVnod } } - (void)taosThreadRwlockUnlock(&pMgmt->lock); + (void)taosThreadRwlockUnlock(&pMgmt->hashLock); *numOfVnodes = num; *ppVnodes = pVnodes; @@ -68,7 +68,7 @@ int32_t vmGetAllVnodeListFromHash(SVnodeMgmt *pMgmt, int32_t *numOfVnodes, SVnod } int32_t vmGetAllVnodeListFromHashWithCreating(SVnodeMgmt *pMgmt, int32_t *numOfVnodes, SVnodeObj ***ppVnodes) { - (void)taosThreadRwlockRdlock(&pMgmt->lock); + (void)taosThreadRwlockRdlock(&pMgmt->hashLock); int32_t num = 0; int32_t size = taosHashGetSize(pMgmt->runngingHash); @@ -76,7 +76,7 @@ int32_t vmGetAllVnodeListFromHashWithCreating(SVnodeMgmt *pMgmt, int32_t *numOfV size += creatingSize; SVnodeObj **pVnodes = taosMemoryCalloc(size, sizeof(SVnodeObj *)); if (pVnodes == NULL) { - (void)taosThreadRwlockUnlock(&pMgmt->lock); + (void)taosThreadRwlockUnlock(&pMgmt->hashLock); return terrno; } @@ -107,7 +107,7 @@ int32_t vmGetAllVnodeListFromHashWithCreating(SVnodeMgmt *pMgmt, int32_t *numOfV taosHashCancelIterate(pMgmt->creatingHash, pIter); } } - (void)taosThreadRwlockUnlock(&pMgmt->lock); + (void)taosThreadRwlockUnlock(&pMgmt->hashLock); *numOfVnodes = num; *ppVnodes = pVnodes; @@ -116,13 +116,13 @@ int32_t vmGetAllVnodeListFromHashWithCreating(SVnodeMgmt *pMgmt, int32_t *numOfV } int32_t vmGetVnodeListFromHash(SVnodeMgmt *pMgmt, int32_t *numOfVnodes, SVnodeObj ***ppVnodes) { - (void)taosThreadRwlockRdlock(&pMgmt->lock); + (void)taosThreadRwlockRdlock(&pMgmt->hashLock); int32_t num = 0; int32_t size = taosHashGetSize(pMgmt->runngingHash); SVnodeObj **pVnodes = taosMemoryCalloc(size, sizeof(SVnodeObj *)); if (pVnodes == NULL) { - (void)taosThreadRwlockUnlock(&pMgmt->lock); + (void)taosThreadRwlockUnlock(&pMgmt->hashLock); return terrno; } @@ -140,7 +140,7 @@ int32_t vmGetVnodeListFromHash(SVnodeMgmt *pMgmt, int32_t *numOfVnodes, SVnodeOb } } - (void)taosThreadRwlockUnlock(&pMgmt->lock); + (void)taosThreadRwlockUnlock(&pMgmt->hashLock); *numOfVnodes = num; *ppVnodes = pVnodes; diff --git a/source/dnode/mgmt/mgmt_vnode/src/vmHandle.c b/source/dnode/mgmt/mgmt_vnode/src/vmHandle.c index 7ead7f8e8d..e5de6e10ae 100644 --- a/source/dnode/mgmt/mgmt_vnode/src/vmHandle.c +++ b/source/dnode/mgmt/mgmt_vnode/src/vmHandle.c @@ -25,7 +25,7 @@ void vmGetVnodeLoads(SVnodeMgmt *pMgmt, SMonVloadInfo *pInfo, bool isReset) { tfsUpdateSize(pMgmt->pTfs); - (void)taosThreadRwlockRdlock(&pMgmt->lock); + (void)taosThreadRwlockRdlock(&pMgmt->hashLock); void *pIter = taosHashIterate(pMgmt->runngingHash, NULL); while (pIter) { @@ -46,14 +46,14 @@ void vmGetVnodeLoads(SVnodeMgmt *pMgmt, SMonVloadInfo *pInfo, bool isReset) { pIter = taosHashIterate(pMgmt->runngingHash, pIter); } - (void)taosThreadRwlockUnlock(&pMgmt->lock); + (void)taosThreadRwlockUnlock(&pMgmt->hashLock); } void vmGetVnodeLoadsLite(SVnodeMgmt *pMgmt, SMonVloadInfo *pInfo) { pInfo->pVloads = taosArrayInit(pMgmt->state.totalVnodes, sizeof(SVnodeLoadLite)); if (!pInfo->pVloads) return; - (void)taosThreadRwlockRdlock(&pMgmt->lock); + (void)taosThreadRwlockRdlock(&pMgmt->hashLock); void *pIter = taosHashIterate(pMgmt->runngingHash, NULL); while (pIter) { @@ -74,7 +74,7 @@ void vmGetVnodeLoadsLite(SVnodeMgmt *pMgmt, SMonVloadInfo *pInfo) { pIter = taosHashIterate(pMgmt->runngingHash, pIter); } - (void)taosThreadRwlockUnlock(&pMgmt->lock); + (void)taosThreadRwlockUnlock(&pMgmt->hashLock); } void vmGetMonitorInfo(SVnodeMgmt *pMgmt, SMonVmInfo *pInfo) { @@ -137,7 +137,7 @@ void vmCleanExpriedSamples(SVnodeMgmt *pMgmt) { dError("failed to get vgroup ids"); return; } - (void)taosThreadRwlockRdlock(&pMgmt->lock); + (void)taosThreadRwlockRdlock(&pMgmt->hashLock); for (int i = 0; i < list_size; i++) { int32_t vgroup_id = vgroup_ids[i]; void *vnode = taosHashGet(pMgmt->runngingHash, &vgroup_id, sizeof(int32_t)); @@ -148,7 +148,7 @@ void vmCleanExpriedSamples(SVnodeMgmt *pMgmt) { } } } - (void)taosThreadRwlockUnlock(&pMgmt->lock); + (void)taosThreadRwlockUnlock(&pMgmt->hashLock); if (vgroup_ids) taosMemoryFree(vgroup_ids); if (keys) taosMemoryFree(keys); return; diff --git a/source/dnode/mgmt/mgmt_vnode/src/vmInt.c b/source/dnode/mgmt/mgmt_vnode/src/vmInt.c index e872e1f972..5da3b2ce9a 100644 --- a/source/dnode/mgmt/mgmt_vnode/src/vmInt.c +++ b/source/dnode/mgmt/mgmt_vnode/src/vmInt.c @@ -24,12 +24,12 @@ int32_t vmGetPrimaryDisk(SVnodeMgmt *pMgmt, int32_t vgId) { int32_t diskId = -1; SVnodeObj *pVnode = NULL; - (void)taosThreadRwlockRdlock(&pMgmt->lock); + (void)taosThreadRwlockRdlock(&pMgmt->hashLock); int32_t r = taosHashGetDup(pMgmt->runngingHash, &vgId, sizeof(int32_t), (void *)&pVnode); if (pVnode != NULL) { diskId = pVnode->diskPrimary; } - (void)taosThreadRwlockUnlock(&pMgmt->lock); + (void)taosThreadRwlockUnlock(&pMgmt->hashLock); return diskId; } @@ -62,7 +62,7 @@ static int32_t vmRegisterCreatingState(SVnodeMgmt *pMgmt, int32_t vgId, int32_t pCreatingVnode->vgId = vgId; pCreatingVnode->diskPrimary = diskId; - code = taosThreadRwlockWrlock(&pMgmt->lock); + code = taosThreadRwlockWrlock(&pMgmt->hashLock); if (code != 0) { taosMemoryFree(pCreatingVnode); return code; @@ -75,7 +75,7 @@ static int32_t vmRegisterCreatingState(SVnodeMgmt *pMgmt, int32_t vgId, int32_t taosMemoryFree(pCreatingVnode); } - int32_t r = taosThreadRwlockUnlock(&pMgmt->lock); + int32_t r = taosThreadRwlockUnlock(&pMgmt->hashLock); if (r != 0) { dError("vgId:%d, failed to unlock since %s", vgId, tstrerror(r)); } @@ -86,7 +86,7 @@ static int32_t vmRegisterCreatingState(SVnodeMgmt *pMgmt, int32_t vgId, int32_t static void vmUnRegisterCreatingState(SVnodeMgmt *pMgmt, int32_t vgId) { SVnodeObj *pOld = NULL; - (void)taosThreadRwlockWrlock(&pMgmt->lock); + (void)taosThreadRwlockWrlock(&pMgmt->hashLock); int32_t r = taosHashGetDup(pMgmt->creatingHash, &vgId, sizeof(int32_t), (void *)&pOld); if (r != 0) { dError("vgId:%d, failed to get vnode from creating Hash", vgId); @@ -96,7 +96,7 @@ static void vmUnRegisterCreatingState(SVnodeMgmt *pMgmt, int32_t vgId) { if (r != 0) { dError("vgId:%d, failed to remove vnode from creatingHash", vgId); } - (void)taosThreadRwlockUnlock(&pMgmt->lock); + (void)taosThreadRwlockUnlock(&pMgmt->hashLock); if (pOld) { dTrace("vgId:%d, free vnode pOld:%p", vgId, &pOld); @@ -205,7 +205,7 @@ void vmCleanPrimaryDisk(SVnodeMgmt *pMgmt, int32_t vgId) { vmUnRegisterCreatingS SVnodeObj *vmAcquireVnodeImpl(SVnodeMgmt *pMgmt, int32_t vgId, bool strict) { SVnodeObj *pVnode = NULL; - (void)taosThreadRwlockRdlock(&pMgmt->lock); + (void)taosThreadRwlockRdlock(&pMgmt->hashLock); int32_t r = taosHashGetDup(pMgmt->runngingHash, &vgId, sizeof(int32_t), (void *)&pVnode); if (pVnode == NULL || strict && (pVnode->dropped || pVnode->failed)) { terrno = TSDB_CODE_VND_INVALID_VGROUP_ID; @@ -214,7 +214,7 @@ SVnodeObj *vmAcquireVnodeImpl(SVnodeMgmt *pMgmt, int32_t vgId, bool strict) { int32_t refCount = atomic_add_fetch_32(&pVnode->refCount, 1); dTrace("vgId:%d, acquire vnode, vnode:%p, ref:%d", pVnode->vgId, pVnode, refCount); } - (void)taosThreadRwlockUnlock(&pMgmt->lock); + (void)taosThreadRwlockUnlock(&pMgmt->hashLock); return pVnode; } @@ -334,10 +334,10 @@ int32_t vmOpenVnode(SVnodeMgmt *pMgmt, SWrapperCfg *pCfg, SVnode *pImpl) { pVnode->failed = 1; } - (void)taosThreadRwlockWrlock(&pMgmt->lock); + (void)taosThreadRwlockWrlock(&pMgmt->hashLock); int32_t code = vmRegisterRunningState(pMgmt, pVnode); vmUnRegisterClosedState(pMgmt, pVnode); - (void)taosThreadRwlockUnlock(&pMgmt->lock); + (void)taosThreadRwlockUnlock(&pMgmt->hashLock); return code; } @@ -350,15 +350,15 @@ void vmCloseVnode(SVnodeMgmt *pMgmt, SVnodeObj *pVnode, bool commitAndRemoveWal, vnodeProposeCommitOnNeed(pVnode->pImpl, atExit); } - (void)taosThreadRwlockWrlock(&pMgmt->lock); + (void)taosThreadRwlockWrlock(&pMgmt->hashLock); vmUnRegisterRunningState(pMgmt, pVnode->vgId); if (keepClosed) { if (vmRegisterClosedState(pMgmt, pVnode) != 0) { - (void)taosThreadRwlockUnlock(&pMgmt->lock); + (void)taosThreadRwlockUnlock(&pMgmt->hashLock); return; }; } - (void)taosThreadRwlockUnlock(&pMgmt->lock); + (void)taosThreadRwlockUnlock(&pMgmt->hashLock); vmReleaseVnode(pMgmt, pVnode); @@ -457,14 +457,14 @@ _closed: void vmCloseFailedVnode(SVnodeMgmt *pMgmt, int32_t vgId) { int32_t r = 0; - r = taosThreadRwlockWrlock(&pMgmt->lock); + r = taosThreadRwlockWrlock(&pMgmt->hashLock); if (r != 0) { dError("vgId:%d, failed to lock since %s", vgId, tstrerror(r)); } if (r == 0) { vmUnRegisterRunningState(pMgmt, vgId); } - r = taosThreadRwlockUnlock(&pMgmt->lock); + r = taosThreadRwlockUnlock(&pMgmt->hashLock); if (r != 0) { dError("vgId:%d, failed to unlock since %s", vgId, tstrerror(r)); } @@ -799,7 +799,7 @@ static void vmCleanup(SVnodeMgmt *pMgmt) { vmCloseVnodes(pMgmt); vmStopWorker(pMgmt); vnodeCleanup(); - (void)taosThreadRwlockDestroy(&pMgmt->lock); + (void)taosThreadRwlockDestroy(&pMgmt->hashLock); (void)taosThreadMutexDestroy(&pMgmt->mutex); (void)taosThreadMutexDestroy(&pMgmt->fileLock); taosMemoryFree(pMgmt); @@ -887,7 +887,7 @@ static int32_t vmInit(SMgmtInputOpt *pInput, SMgmtOutputOpt *pOutput) { pMgmt->msgCb.qsizeFp = (GetQueueSizeFp)vmGetQueueSize; pMgmt->msgCb.mgmt = pMgmt; - code = taosThreadRwlockInit(&pMgmt->lock, NULL); + code = taosThreadRwlockInit(&pMgmt->hashLock, NULL); if (code != 0) { code = TAOS_SYSTEM_ERROR(errno); goto _OVER; diff --git a/source/libs/sync/src/syncUtil.c b/source/libs/sync/src/syncUtil.c index 5a2571a291..44e8d97aca 100644 --- a/source/libs/sync/src/syncUtil.c +++ b/source/libs/sync/src/syncUtil.c @@ -215,7 +215,7 @@ void syncPrintNodeLog(const char* flags, ELogLevel level, int32_t dflag, bool fo SSnapshot snapshot = {.data = NULL, .lastApplyIndex = -1, .lastApplyTerm = 0}; if (pNode->pFsm != NULL && pNode->pFsm->FpGetSnapshotInfo != NULL) { - (void)pNode->pFsm->FpGetSnapshotInfo(pNode->pFsm, &snapshot); + (void)pNode->pFsm->FpGetSnapshotInfo(pNode->pFsm, &snapshot); // vnodeSyncGetSnapshotInfo } SyncIndex logLastIndex = SYNC_INDEX_INVALID; @@ -253,13 +253,15 @@ void syncPrintNodeLog(const char* flags, ELogLevel level, int32_t dflag, bool fo va_end(argpointer); int32_t aqItems = 0; + /* if (pNode != NULL && pNode->pFsm != NULL && pNode->pFsm->FpApplyQueueItems != NULL) { - aqItems = pNode->pFsm->FpApplyQueueItems(pNode->pFsm); + aqItems = pNode->pFsm->FpApplyQueueItems(pNode->pFsm); // vnodeApplyQueueItems } + */ // restore error code terrno = errCode; - SyncIndex appliedIndex = pNode->pFsm->FpAppliedIndexCb(pNode->pFsm); + SyncIndex appliedIndex = pNode->pFsm->FpAppliedIndexCb(pNode->pFsm); // vnodeSyncAppliedIndex if (pNode != NULL) { taosPrintLog( @@ -426,19 +428,25 @@ void syncLogSendHeartbeat(SSyncNode* pSyncNode, const SyncHeartbeat* pMsg, bool } void syncLogRecvHeartbeat(SSyncNode* pSyncNode, const SyncHeartbeat* pMsg, int64_t timeDiff, const char* s) { + char pBuf[TD_TIME_STR_LEN] = {0}; + if (pMsg->timeStamp > 0) { + if (formatTimestampLocal(pBuf, pMsg->timeStamp, TSDB_TIME_PRECISION_MILLI) == NULL) { + pBuf[0] = '\0'; + } + } if (timeDiff > SYNC_HEARTBEAT_SLOW_MS) { pSyncNode->hbSlowNum++; sNTrace(pSyncNode, - "recv sync-heartbeat from dnode:%d slow {term:%" PRId64 ", commit-index:%" PRId64 ", min-match:%" PRId64 - ", ts:%" PRId64 "}, QID:%s, net elapsed:%" PRId64, - DID(&pMsg->srcId), pMsg->term, pMsg->commitIndex, pMsg->minMatchIndex, pMsg->timeStamp, s, timeDiff); + "recv sync-heartbeat from dnode:%d slow(%d ms) {term:%" PRId64 ", commit-index:%" PRId64 + ", min-match:%" PRId64 ", ts:%s}, QID:%s, net elapsed:%" PRId64 "ms", + DID(&pMsg->srcId), SYNC_HEARTBEAT_SLOW_MS, pMsg->term, pMsg->commitIndex, pMsg->minMatchIndex, pBuf, s, timeDiff); + } else { + sNTrace(pSyncNode, + "recv sync-heartbeat from dnode:%d {term:%" PRId64 ", commit-index:%" PRId64 ", min-match:%" PRId64 + ", ts:%s}, QID:%s, net elapsed:%" PRId64 "ms", + DID(&pMsg->srcId), pMsg->term, pMsg->commitIndex, pMsg->minMatchIndex, pBuf, s, timeDiff); } - - sNTrace(pSyncNode, - "recv sync-heartbeat from dnode:%d {term:%" PRId64 ", commit-index:%" PRId64 ", min-match:%" PRId64 - ", ts:%" PRId64 "}, QID:%s, net elapsed:%" PRId64, - DID(&pMsg->srcId), pMsg->term, pMsg->commitIndex, pMsg->minMatchIndex, pMsg->timeStamp, s, timeDiff); } void syncLogSendHeartbeatReply(SSyncNode* pSyncNode, const SyncHeartbeatReply* pMsg, const char* s) { diff --git a/source/libs/transport/src/tmsgcb.c b/source/libs/transport/src/tmsgcb.c index e87011f097..0632e21d20 100644 --- a/source/libs/transport/src/tmsgcb.c +++ b/source/libs/transport/src/tmsgcb.c @@ -37,7 +37,7 @@ int32_t tmsgPutToQueue(const SMsgCb* msgcb, EQueueType qtype, SRpcMsg* pMsg) { } int32_t tmsgGetQueueSize(const SMsgCb* msgcb, int32_t vgId, EQueueType qtype) { - return (*msgcb->qsizeFp)(msgcb->mgmt, vgId, qtype); + return (*msgcb->qsizeFp)(msgcb->mgmt, vgId, qtype); // vmGetQueueSize } int32_t tmsgSendReq(const SEpSet* epSet, SRpcMsg* pMsg) { From 0945ddc63f534976a3439cc852a8378a31f532d2 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Fri, 28 Feb 2025 16:05:39 +0800 Subject: [PATCH 39/70] refactor(stream): not generated hb when trying to restart tasks. --- source/libs/stream/src/streamHb.c | 16 ++++++++++------ 1 file changed, 10 insertions(+), 6 deletions(-) diff --git a/source/libs/stream/src/streamHb.c b/source/libs/stream/src/streamHb.c index 90ebd47ac6..312ed85764 100644 --- a/source/libs/stream/src/streamHb.c +++ b/source/libs/stream/src/streamHb.c @@ -363,14 +363,18 @@ void streamMetaHbToMnode(void* param, void* tmrId) { pMeta->pHbInfo->hbStart = taosGetTimestampMs(); } - streamMetaRLock(pMeta); - code = streamMetaSendHbHelper(pMeta); - if (code) { - stError("vgId:%d failed to send hmMsg to mnode, try again in 5s, code:%s", pMeta->vgId, tstrerror(code)); + // NOTE: stream task in restart procedure. not generate the hb now, try to acquire the lock may cause stuck this timer. + if (pMeta->startInfo.startAllTasks == 1) { + stWarn("vgId:%d in restart all task procedure, not try to genbrate the hb now, try again in 5s", pMeta->vgId); + } else { + streamMetaRLock(pMeta); + code = streamMetaSendHbHelper(pMeta); + if (code) { + stError("vgId:%d failed to send hmMsg to mnode, try again in 5s, code:%s", pMeta->vgId, tstrerror(code)); + } + streamMetaRUnLock(pMeta); } - streamMetaRUnLock(pMeta); - streamTmrStart(streamMetaHbToMnode, META_HB_CHECK_INTERVAL, param, streamTimer, &pMeta->pHbInfo->hbTmr, pMeta->vgId, "meta-hb-tmr"); From 7eda6747fcc16fdbb25cb94f84ceb5cee4ad697d Mon Sep 17 00:00:00 2001 From: dmchen Date: Fri, 28 Feb 2025 16:02:18 +0800 Subject: [PATCH 40/70] fix:remove-lock-from-heartbeat-log --- source/libs/sync/inc/syncUtil.h | 9 +++ source/libs/sync/src/syncUtil.c | 133 +++++++++++++++++++++++++------- 2 files changed, 115 insertions(+), 27 deletions(-) diff --git a/source/libs/sync/inc/syncUtil.h b/source/libs/sync/inc/syncUtil.h index e7110c0964..95ea75731c 100644 --- a/source/libs/sync/inc/syncUtil.h +++ b/source/libs/sync/inc/syncUtil.h @@ -53,6 +53,13 @@ extern "C" { #define sNDebug(pNode, ...) if (sDebugFlag & DEBUG_DEBUG) { syncPrintNodeLog("SYN ", DEBUG_DEBUG, sDebugFlag, true, pNode, __VA_ARGS__); } #define sNTrace(pNode, ...) if (sDebugFlag & DEBUG_TRACE) { syncPrintNodeLog("SYN ", DEBUG_TRACE, sDebugFlag, true, pNode, __VA_ARGS__); } +#define sHFatal(pNode, ...) if (sDebugFlag & DEBUG_FATAL) { syncPrintHbLog("SYN FATAL ", DEBUG_FATAL, 255, true, pNode, __VA_ARGS__); } +#define sHError(pNode, ...) if (sDebugFlag & DEBUG_ERROR) { syncPrintHbLog("SYN ERROR ", DEBUG_ERROR, 255, true, pNode, __VA_ARGS__); } +#define sHWarn(pNode, ...) if (sDebugFlag & DEBUG_WARN) { syncPrintHbLog("SYN WARN ", DEBUG_WARN, 255, true, pNode, __VA_ARGS__); } +#define sHInfo(pNode, ...) if (sDebugFlag & DEBUG_INFO) { syncPrintHbLog("SYN ", DEBUG_INFO, 255, true, pNode, __VA_ARGS__); } +#define sHDebug(pNode, ...) if (sDebugFlag & DEBUG_DEBUG) { syncPrintHbLog("SYN ", DEBUG_DEBUG, sDebugFlag, true, pNode, __VA_ARGS__); } +#define sHTrace(pNode, ...) if (sDebugFlag & DEBUG_TRACE) { syncPrintHbLog("SYN ", DEBUG_TRACE, sDebugFlag, true, pNode, __VA_ARGS__); } + #define sSFatal(pSender, ...) if (sDebugFlag & DEBUG_FATAL) { syncPrintSnapshotSenderLog("SYN FATAL ", DEBUG_FATAL, 255, pSender, __VA_ARGS__); } #define sSError(pSender, ...) if (sDebugFlag & DEBUG_ERROR) { syncPrintSnapshotSenderLog("SYN ERROR ", DEBUG_ERROR, 255, pSender, __VA_ARGS__); } #define sSWarn(pSender, ...) if (sDebugFlag & DEBUG_WARN) { syncPrintSnapshotSenderLog("SYN WARN ", DEBUG_WARN, 255, pSender, __VA_ARGS__); } @@ -87,6 +94,8 @@ void syncUtilGenerateArbToken(int32_t nodeId, int32_t groupId, char* buf); void syncPrintNodeLog(const char* flags, ELogLevel level, int32_t dflag, bool formatTime, SSyncNode* pNode, const char* format, ...); +void syncPrintHbLog(const char* flags, ELogLevel level, int32_t dflag, bool formatTime, SSyncNode* pNode, + const char* format, ...); void syncPrintSnapshotSenderLog(const char* flags, ELogLevel level, int32_t dflag, SSyncSnapshotSender* pSender, const char* format, ...); void syncPrintSnapshotReceiverLog(const char* flags, ELogLevel level, int32_t dflag, SSyncSnapshotReceiver* pReceiver, diff --git a/source/libs/sync/src/syncUtil.c b/source/libs/sync/src/syncUtil.c index 44e8d97aca..435586041a 100644 --- a/source/libs/sync/src/syncUtil.c +++ b/source/libs/sync/src/syncUtil.c @@ -253,11 +253,9 @@ void syncPrintNodeLog(const char* flags, ELogLevel level, int32_t dflag, bool fo va_end(argpointer); int32_t aqItems = 0; - /* if (pNode != NULL && pNode->pFsm != NULL && pNode->pFsm->FpApplyQueueItems != NULL) { aqItems = pNode->pFsm->FpApplyQueueItems(pNode->pFsm); // vnodeApplyQueueItems } - */ // restore error code terrno = errCode; @@ -272,15 +270,71 @@ void syncPrintNodeLog(const char* flags, ELogLevel level, int32_t dflag, bool fo ", elect-times:%d, as-leader-times:%d, as-assigned-leader-times:%d, cfg-ch-times:%d, hb-slow:%d, hbr-slow:%d, " "aq-items:%d, snaping:%" PRId64 ", replicas:%d, last-cfg:%" PRId64 ", chging:%d, restore:%d, quorum:%d, elect-lc-timer:%" PRId64 ", hb:%" PRId64 - ", buffer:%s, repl-mgrs:%s, members:%s, send hb:%s, recv hb:%s, recv hb-reply:%s, arb-token:%s, msg[sent:%d, recv:%d, slow-recev:%d]", + ", buffer:%s, repl-mgrs:%s, members:%s, send hb:%s, recv hb:%s, recv hb-reply:%s, arb-token:%s, msg[sent:%d, " + "recv:%d, slow-recv:%d]", pNode->vgId, eventLog, syncStr(pNode->state), currentTerm, pNode->commitIndex, pNode->assignedCommitIndex, appliedIndex, logBeginIndex, logLastIndex, pNode->minMatchIndex, snapshot.lastApplyIndex, snapshot.lastApplyTerm, pNode->electNum, pNode->becomeLeaderNum, pNode->becomeAssignedLeaderNum, pNode->configChangeNum, pNode->hbSlowNum, pNode->hbrSlowNum, aqItems, pNode->snapshottingIndex, pNode->replicaNum, pNode->raftCfg.lastConfigIndex, pNode->changing, pNode->restoreFinish, syncNodeDynamicQuorum(pNode), pNode->electTimerLogicClock, pNode->heartbeatTimerLogicClockUser, bufferStatesStr, - replMgrStatesStr, cfgStr, sentHbTimeStr, hbTimeStr, hbrTimeStr, pNode->arbToken, pNode->sendCount, pNode->recvCount, - pNode->slowCount); + replMgrStatesStr, cfgStr, sentHbTimeStr, hbTimeStr, hbrTimeStr, pNode->arbToken, pNode->sendCount, + pNode->recvCount, pNode->slowCount); + } +} + +void syncPrintHbLog(const char* flags, ELogLevel level, int32_t dflag, bool formatTime, SSyncNode* pNode, + const char* format, ...) { + if (pNode == NULL || pNode->pLogStore == NULL) return; + int64_t currentTerm = raftStoreGetTerm(pNode); + + // save error code, otherwise it will be overwritten + int32_t errCode = terrno; + + int32_t cacheHit = pNode->pLogStore->cacheHit; + int32_t cacheMiss = pNode->pLogStore->cacheMiss; + + char cfgStr[1024] = ""; + syncCfg2SimpleStr(&pNode->raftCfg.cfg, cfgStr, sizeof(cfgStr)); + + char replMgrStatesStr[1024] = ""; + syncLogReplStates2Str(pNode, replMgrStatesStr, sizeof(replMgrStatesStr)); + + char bufferStatesStr[256] = ""; + syncLogBufferStates2Str(pNode, bufferStatesStr, sizeof(bufferStatesStr)); + + char hbrTimeStr[256] = ""; + syncHearbeatReplyTime2Str(pNode, hbrTimeStr, sizeof(hbrTimeStr), formatTime); + + char hbTimeStr[256] = ""; + syncHearbeatTime2Str(pNode, hbTimeStr, sizeof(hbTimeStr), formatTime); + + char sentHbTimeStr[512] = ""; + syncSentHearbeatTime2Str(pNode, sentHbTimeStr, sizeof(sentHbTimeStr), formatTime); + + char eventLog[512]; // {0}; + va_list argpointer; + va_start(argpointer, format); + int32_t writeLen = vsnprintf(eventLog, sizeof(eventLog), format, argpointer); + va_end(argpointer); + + terrno = errCode; + + if (pNode != NULL) { + taosPrintLog( + flags, level, dflag, + "vgId:%d, %s, sync:%s, term:%" PRIu64 ", commit-index:%" PRId64 ", assigned-index:%" PRId64 ", min:%" PRId64 + ", elect-times:%d, as-leader-times:%d, as-assigned-leader-times:%d, cfg-ch-times:%d, hb-slow:%d, hbr-slow:%d, " + ", snaping:%" PRId64 ", replicas:%d, last-cfg:%" PRId64 + ", chging:%d, restore:%d, quorum:%d, elect-lc-timer:%" PRId64 ", hb:%" PRId64 + ", buffer:%s, repl-mgrs:%s, members:%s, send hb:%s, recv hb:%s, recv hb-reply:%s, arb-token:%s, msg[sent:%d, " + "recv:%d, slow-recv:%d]", + pNode->vgId, eventLog, syncStr(pNode->state), currentTerm, pNode->commitIndex, pNode->assignedCommitIndex, + pNode->minMatchIndex, pNode->electNum, pNode->becomeLeaderNum, pNode->becomeAssignedLeaderNum, + pNode->configChangeNum, pNode->hbSlowNum, pNode->hbrSlowNum, pNode->snapshottingIndex, pNode->replicaNum, + pNode->raftCfg.lastConfigIndex, pNode->changing, pNode->restoreFinish, syncNodeDynamicQuorum(pNode), + pNode->electTimerLogicClock, pNode->heartbeatTimerLogicClockUser, bufferStatesStr, replMgrStatesStr, cfgStr, + sentHbTimeStr, hbTimeStr, hbrTimeStr, pNode->arbToken, pNode->sendCount, pNode->recvCount, pNode->slowCount); } } @@ -414,12 +468,12 @@ void syncLogRecvAppendEntriesReply(SSyncNode* pSyncNode, const SyncAppendEntries void syncLogSendHeartbeat(SSyncNode* pSyncNode, const SyncHeartbeat* pMsg, bool printX, int64_t timerElapsed, int64_t execTime) { if (printX) { - sNTrace(pSyncNode, + sHTrace(pSyncNode, "send sync-heartbeat to dnode:%d {term:%" PRId64 ", commit-index:%" PRId64 ", min-match:%" PRId64 ", ts:%" PRId64 "}, x", DID(&pMsg->destId), pMsg->term, pMsg->commitIndex, pMsg->minMatchIndex, pMsg->timeStamp); } else { - sNTrace(pSyncNode, + sHTrace(pSyncNode, "send sync-heartbeat to dnode:%d {term:%" PRId64 ", commit-index:%" PRId64 ", min-match:%" PRId64 ", ts:%" PRId64 "}, timer-elapsed:%" PRId64 ", next-exec:%" PRId64, DID(&pMsg->destId), pMsg->term, pMsg->commitIndex, pMsg->minMatchIndex, pMsg->timeStamp, timerElapsed, @@ -428,29 +482,39 @@ void syncLogSendHeartbeat(SSyncNode* pSyncNode, const SyncHeartbeat* pMsg, bool } void syncLogRecvHeartbeat(SSyncNode* pSyncNode, const SyncHeartbeat* pMsg, int64_t timeDiff, const char* s) { - char pBuf[TD_TIME_STR_LEN] = {0}; - if (pMsg->timeStamp > 0) { - if (formatTimestampLocal(pBuf, pMsg->timeStamp, TSDB_TIME_PRECISION_MILLI) == NULL) { - pBuf[0] = '\0'; - } - } if (timeDiff > SYNC_HEARTBEAT_SLOW_MS) { pSyncNode->hbSlowNum++; - sNTrace(pSyncNode, + char pBuf[TD_TIME_STR_LEN] = {0}; + if (pMsg->timeStamp > 0) { + if (formatTimestampLocal(pBuf, pMsg->timeStamp, TSDB_TIME_PRECISION_MILLI) == NULL) { + pBuf[0] = '\0'; + } + } + + sHError(pSyncNode, "recv sync-heartbeat from dnode:%d slow(%d ms) {term:%" PRId64 ", commit-index:%" PRId64 ", min-match:%" PRId64 ", ts:%s}, QID:%s, net elapsed:%" PRId64 "ms", - DID(&pMsg->srcId), SYNC_HEARTBEAT_SLOW_MS, pMsg->term, pMsg->commitIndex, pMsg->minMatchIndex, pBuf, s, timeDiff); + DID(&pMsg->srcId), SYNC_HEARTBEAT_SLOW_MS, pMsg->term, pMsg->commitIndex, pMsg->minMatchIndex, pBuf, s, + timeDiff); } else { - sNTrace(pSyncNode, - "recv sync-heartbeat from dnode:%d {term:%" PRId64 ", commit-index:%" PRId64 ", min-match:%" PRId64 - ", ts:%s}, QID:%s, net elapsed:%" PRId64 "ms", - DID(&pMsg->srcId), pMsg->term, pMsg->commitIndex, pMsg->minMatchIndex, pBuf, s, timeDiff); + if (sDebugFlag & DEBUG_TRACE) { + char pBuf[TD_TIME_STR_LEN] = {0}; + if (pMsg->timeStamp > 0) { + if (formatTimestampLocal(pBuf, pMsg->timeStamp, TSDB_TIME_PRECISION_MILLI) == NULL) { + pBuf[0] = '\0'; + } + } + sHTrace(pSyncNode, + "recv sync-heartbeat from dnode:%d {term:%" PRId64 ", commit-index:%" PRId64 ", min-match:%" PRId64 + ", ts:%s}, QID:%s, net elapsed:%" PRId64 "ms", + DID(&pMsg->srcId), pMsg->term, pMsg->commitIndex, pMsg->minMatchIndex, pBuf, s, timeDiff); + } } } void syncLogSendHeartbeatReply(SSyncNode* pSyncNode, const SyncHeartbeatReply* pMsg, const char* s) { - sNTrace(pSyncNode, "send sync-heartbeat-reply from dnode:%d {term:%" PRId64 ", ts:%" PRId64 "}, %s", + sHTrace(pSyncNode, "send sync-heartbeat-reply from dnode:%d {term:%" PRId64 ", ts:%" PRId64 "}, %s", DID(&pMsg->destId), pMsg->term, pMsg->timeStamp, s); } @@ -458,14 +522,29 @@ void syncLogRecvHeartbeatReply(SSyncNode* pSyncNode, const SyncHeartbeatReply* p if (timeDiff > SYNC_HEARTBEAT_REPLY_SLOW_MS) { pSyncNode->hbrSlowNum++; - sNTrace(pSyncNode, - "recv sync-heartbeat-reply from dnode:%d slow {term:%" PRId64 ", ts:%" PRId64 "}, %s, net elapsed:%" PRId64, - DID(&pMsg->srcId), pMsg->term, pMsg->timeStamp, s, timeDiff); - } + char pBuf[TD_TIME_STR_LEN] = {0}; + if (pMsg->timeStamp > 0) { + if (formatTimestampLocal(pBuf, pMsg->timeStamp, TSDB_TIME_PRECISION_MILLI) == NULL) { + pBuf[0] = '\0'; + } + } - sNTrace(pSyncNode, - "recv sync-heartbeat-reply from dnode:%d {term:%" PRId64 ", ts:%" PRId64 "}, %s, net elapsed:%" PRId64, - DID(&pMsg->srcId), pMsg->term, pMsg->timeStamp, s, timeDiff); + sHError(pSyncNode, + "recv sync-heartbeat-reply from dnode:%d slow(%d ms) {term:%" PRId64 ", ts:%s}, %s, net elapsed:%" PRId64, + DID(&pMsg->srcId), SYNC_HEARTBEAT_REPLY_SLOW_MS, pMsg->term, pBuf, s, timeDiff); + } else { + if (sDebugFlag & DEBUG_TRACE) { + char pBuf[TD_TIME_STR_LEN] = {0}; + if (pMsg->timeStamp > 0) { + if (formatTimestampLocal(pBuf, pMsg->timeStamp, TSDB_TIME_PRECISION_MILLI) == NULL) { + pBuf[0] = '\0'; + } + } + sHTrace(pSyncNode, + "recv sync-heartbeat-reply from dnode:%d {term:%" PRId64 ", ts:%" PRId64 "}, %s, net elapsed:%" PRId64, + DID(&pMsg->srcId), pMsg->term, pMsg->timeStamp, s, timeDiff); + } + } } void syncLogSendSyncSnapshotSend(SSyncNode* pSyncNode, const SyncSnapshotSend* pMsg, const char* s) { From 8c3035cf24adbb951fd5f8bc8b84b9906c133b42 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Fri, 28 Feb 2025 17:46:54 +0800 Subject: [PATCH 41/70] fix(stream): avoid lock in timer to avoid sync operation blocks. --- include/libs/stream/tstream.h | 1 + source/dnode/vnode/src/tq/tqStreamTask.c | 11 +++++++---- source/libs/stream/src/streamUtil.c | 9 +++++++++ 3 files changed, 17 insertions(+), 4 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index f7d42794be..a458db8e68 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -789,6 +789,7 @@ void streamMetaClearSetUpdateTaskListComplete(SStreamMeta* pMeta); bool streamMetaInitUpdateTaskList(SStreamMeta* pMeta, int32_t transId); void streamMetaRLock(SStreamMeta* pMeta); +int32_t streamMetaTryRlock(SStreamMeta* pMeta); void streamMetaRUnLock(SStreamMeta* pMeta); void streamMetaWLock(SStreamMeta* pMeta); void streamMetaWUnLock(SStreamMeta* pMeta); diff --git a/source/dnode/vnode/src/tq/tqStreamTask.c b/source/dnode/vnode/src/tq/tqStreamTask.c index bb1bedd022..f37f77ef2a 100644 --- a/source/dnode/vnode/src/tq/tqStreamTask.c +++ b/source/dnode/vnode/src/tq/tqStreamTask.c @@ -148,9 +148,13 @@ static void doStartScanWal(void* param, void* tmrId) { return; } - streamMetaRLock(pMeta); - numOfTasks = taosArrayGetSize(pMeta->pTaskList); - streamMetaRUnLock(pMeta); + code = streamMetaTryRlock(pMeta); + if (code == 0) { + numOfTasks = taosArrayGetSize(pMeta->pTaskList); + streamMetaRUnLock(pMeta); + } else { + numOfTasks = 0; + } if (numOfTasks == 0) { goto _end; @@ -169,7 +173,6 @@ static void doStartScanWal(void* param, void* tmrId) { } _end: - streamTmrStart(doStartScanWal, SCAN_WAL_IDLE_DURATION, pParam, pTimer, &pMeta->scanInfo.scanTimer, vgId, "scan-wal"); tqDebug("vgId:%d scan-wal will start in %dms", vgId, SCAN_WAL_IDLE_DURATION*SCAN_WAL_WAIT_COUNT); diff --git a/source/libs/stream/src/streamUtil.c b/source/libs/stream/src/streamUtil.c index 4c481e6041..11e291c876 100644 --- a/source/libs/stream/src/streamUtil.c +++ b/source/libs/stream/src/streamUtil.c @@ -54,6 +54,15 @@ void streamMetaRUnLock(SStreamMeta* pMeta) { } } +int32_t streamMetaTryRlock(SStreamMeta* pMeta) { + int32_t code = taosThreadRwlockTryRdlock(&pMeta->lock); + if (code) { + stError("vgId:%d try meta-rlock failed, code:%s", pMeta->vgId, tstrerror(code)); + } + + return code; +} + void streamMetaWLock(SStreamMeta* pMeta) { // stTrace("vgId:%d meta-wlock", pMeta->vgId); int32_t code = taosThreadRwlockWrlock(&pMeta->lock); From a822be3866278d05436e435ca037c00777daf6b3 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Fri, 28 Feb 2025 18:49:48 +0800 Subject: [PATCH 42/70] refactor(stream): try to lock during build stream hb. --- source/libs/stream/src/streamHb.c | 23 +++++++++++++++-------- 1 file changed, 15 insertions(+), 8 deletions(-) diff --git a/source/libs/stream/src/streamHb.c b/source/libs/stream/src/streamHb.c index 312ed85764..ca5b6630fd 100644 --- a/source/libs/stream/src/streamHb.c +++ b/source/libs/stream/src/streamHb.c @@ -364,15 +364,22 @@ void streamMetaHbToMnode(void* param, void* tmrId) { } // NOTE: stream task in restart procedure. not generate the hb now, try to acquire the lock may cause stuck this timer. - if (pMeta->startInfo.startAllTasks == 1) { - stWarn("vgId:%d in restart all task procedure, not try to genbrate the hb now, try again in 5s", pMeta->vgId); - } else { - streamMetaRLock(pMeta); - code = streamMetaSendHbHelper(pMeta); - if (code) { - stError("vgId:%d failed to send hmMsg to mnode, try again in 5s, code:%s", pMeta->vgId, tstrerror(code)); + int32_t count = 30; + bool send = false; + while ((--count) >= 0) { + int32_t ret = streamMetaTryRlock(pMeta); + if (ret != 0) { + taosMsleep(10); + } else { + send = true; + code = streamMetaSendHbHelper(pMeta); + streamMetaRUnLock(pMeta); + break; } - streamMetaRUnLock(pMeta); + } + + if (!send) { + stError("vgId:%d failed to send hmMsg to mnode, retry again in 5s, code:%s", pMeta->vgId, tstrerror(code)); } streamTmrStart(streamMetaHbToMnode, META_HB_CHECK_INTERVAL, param, streamTimer, &pMeta->pHbInfo->hbTmr, pMeta->vgId, From e831bf7845b62beb268cedc9e27acf151f14fb9f Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Fri, 28 Feb 2025 19:38:15 +0800 Subject: [PATCH 43/70] fix(stream): fix race condition --- source/libs/stream/src/streamCheckpoint.c | 3 +++ 1 file changed, 3 insertions(+) diff --git a/source/libs/stream/src/streamCheckpoint.c b/source/libs/stream/src/streamCheckpoint.c index 0d951a3ff1..5880783c35 100644 --- a/source/libs/stream/src/streamCheckpoint.c +++ b/source/libs/stream/src/streamCheckpoint.c @@ -1585,10 +1585,13 @@ int32_t streamTaskSendNegotiateChkptIdMsg(SStreamTask* pTask) { pTask->pBackend = NULL; } + streamMetaWLock(pTask->pMeta); if (pTask->exec.pExecutor != NULL) { qDestroyTask(pTask->exec.pExecutor); pTask->exec.pExecutor = NULL; } + streamMetaWUnLock(pTask->pMeta); + return 0; } From c9f9540530ed11c6c0f98b3c88cfe4ab1828a620 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sat, 1 Mar 2025 15:33:31 +0800 Subject: [PATCH 44/70] fix(stream): fix memory leak. --- source/dnode/mnode/impl/src/mndConfig.c | 2 ++ 1 file changed, 2 insertions(+) diff --git a/source/dnode/mnode/impl/src/mndConfig.c b/source/dnode/mnode/impl/src/mndConfig.c index 099fff7aee..1e69ae2b5a 100644 --- a/source/dnode/mnode/impl/src/mndConfig.c +++ b/source/dnode/mnode/impl/src/mndConfig.c @@ -299,6 +299,8 @@ _OVER: } sdbRelease(pMnode->pSdb, vObj); cfgArrayCleanUp(array); + + tFreeSConfigReq(&configReq); return code; } From 5cfdfc74d9a161e8c30f55b9ca2f6b11732ed7bd Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 6 Mar 2025 18:47:10 +0800 Subject: [PATCH 45/70] fix(stream): remove duplicated code. --- include/libs/stream/tstream.h | 1 - 1 file changed, 1 deletion(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index e976c4e0ee..08ee3bfa00 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -136,7 +136,6 @@ enum { enum { STREAM_QUEUE__SUCESS = 1, STREAM_QUEUE__FAILED, - STREAM_QUEUE__CHKPTFAILED, STREAM_QUEUE__PROCESSING, STREAM_QUEUE__CHKPTFAILED, }; From c34df137520e1c25c7d3e058d4c45171eeba3e13 Mon Sep 17 00:00:00 2001 From: Feng Chao Date: Mon, 10 Mar 2025 11:19:31 +0800 Subject: [PATCH 46/70] Add tdgpt ci and anode service restart workflow --- .github/workflows/tdgpt-ci.yml | 49 ++++++++++++++++++++++ .github/workflows/tdgpt-update-service.yml | 36 ++++++++++++++++ 2 files changed, 85 insertions(+) create mode 100644 .github/workflows/tdgpt-ci.yml create mode 100644 .github/workflows/tdgpt-update-service.yml diff --git a/.github/workflows/tdgpt-ci.yml b/.github/workflows/tdgpt-ci.yml new file mode 100644 index 0000000000..ece06d17a1 --- /dev/null +++ b/.github/workflows/tdgpt-ci.yml @@ -0,0 +1,49 @@ +name: TDGpt CI Pipeline + +on: + pull_request: + branches: + - '3.0' + paths: + - 'tools/tdgpt/*' + +jobs: + build: + runs-on: ubuntu-latest + strategy: + fail-fast: false + matrix: + python-version: ["3.10"] + + steps: + - uses: actions/checkout@v4 + + - name: Set up Python ${{ matrix.python-version }} + uses: actions/setup-python@v3 + with: + python-version: ${{ matrix.python-version }} + cache: 'pip' + + - name: Install dependencies + run: | + python -m pip install --upgrade pip + python -m pip install flake8 pytest pylint + if [ -f requirements.txt ]; then pip install -r requirements.txt; fi + + - name: Checking the code with pylint + working-directory: ${{ github.workspace }}/tools/tdgpt + run: | + pylint $(git ls-files '*.py') --exit-zero + + - name: Checking the code with flake8 + working-directory: ${{ github.workspace }}/tools/tdgpt + run: | + # stop the build if there are Python syntax errors or undefined names + flake8 . --count --select=E9,F63,F7,F82 --show-source --statistics + # exit-zero treats all errors as warnings. The GitHub editor is 127 chars wide + flake8 . --count --exit-zero --max-complexity=10 --max-line-length=127 --statistics + + - name: Run test cases with pytest + working-directory: ${{ github.workspace }}/tools/tdgpt + run: | + pytest diff --git a/.github/workflows/tdgpt-update-service.yml b/.github/workflows/tdgpt-update-service.yml new file mode 100644 index 0000000000..b45e6d1ffc --- /dev/null +++ b/.github/workflows/tdgpt-update-service.yml @@ -0,0 +1,36 @@ +name: TDGpt Update Service + +on: + schedule: + - cron: '30 00 * * *' + +env: + WKC: "/root/TDengine" + +jobs: + update-service: + runs-on: + group: CI + labels: [self-hosted, Linux, X64, tdgpt-anode-service] + steps: + - name: Update TDengine codes + run: | + set -euo pipefail + cd ${{ env.WKC }} + git checkout 3.0 + + - name: Package the TDGpt Anode Service + run: | + set -euo pipefail + cd ${{ env.WKC }}/tools/tdgpt/script && ./release.sh + + - name: Reinstall and restart the TDGpt Anode Service + run: | + set -euo pipefail + cd ${{ env.WKC }}/tools/tdgpt/release + if [[ -f "TDengine-enterprise-anode-1.0.1.tar.gz" ]]; then + tar -xzf TDengine-enterprise-anode-1.0.1.tar.gz + cd TDengine-enterprise-anode-1.0.1 + ./install.sh + fi + systemctl restart taosanoded From 438bee4e62d5b4ca7aef2819e66e31c8ceb90065 Mon Sep 17 00:00:00 2001 From: Feng Chao Date: Mon, 10 Mar 2025 13:41:43 +0800 Subject: [PATCH 47/70] ci: Update tdgpt-update-service.yml Update trigger time to verify the workflow --- .github/workflows/tdgpt-update-service.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/tdgpt-update-service.yml b/.github/workflows/tdgpt-update-service.yml index b45e6d1ffc..fe29501b43 100644 --- a/.github/workflows/tdgpt-update-service.yml +++ b/.github/workflows/tdgpt-update-service.yml @@ -2,7 +2,7 @@ name: TDGpt Update Service on: schedule: - - cron: '30 00 * * *' + - cron: '45 13 * * *' env: WKC: "/root/TDengine" From 425d7fb8c7785252a52054da0898279cc2a112ee Mon Sep 17 00:00:00 2001 From: Feng Chao Date: Mon, 10 Mar 2025 13:47:18 +0800 Subject: [PATCH 48/70] ci: Update tdgpt-update-service.yml add push trigger condition to verify --- .github/workflows/tdgpt-update-service.yml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/.github/workflows/tdgpt-update-service.yml b/.github/workflows/tdgpt-update-service.yml index fe29501b43..54af810dbb 100644 --- a/.github/workflows/tdgpt-update-service.yml +++ b/.github/workflows/tdgpt-update-service.yml @@ -1,8 +1,9 @@ name: TDGpt Update Service on: + push: schedule: - - cron: '45 13 * * *' + - cron: '30 00 * * *' env: WKC: "/root/TDengine" From 7de0dd67c14e53b1b66f802684fdce8fc7d25a35 Mon Sep 17 00:00:00 2001 From: Feng Chao Date: Mon, 10 Mar 2025 13:48:53 +0800 Subject: [PATCH 49/70] ci: Update tdgpt-update-service.yml Remove push trigger condition after verification --- .github/workflows/tdgpt-update-service.yml | 1 - 1 file changed, 1 deletion(-) diff --git a/.github/workflows/tdgpt-update-service.yml b/.github/workflows/tdgpt-update-service.yml index 54af810dbb..b45e6d1ffc 100644 --- a/.github/workflows/tdgpt-update-service.yml +++ b/.github/workflows/tdgpt-update-service.yml @@ -1,7 +1,6 @@ name: TDGpt Update Service on: - push: schedule: - cron: '30 00 * * *' From ae9eac69c0d098e865dd0d8295786fbf008c1d82 Mon Sep 17 00:00:00 2001 From: Feng Chao Date: Mon, 10 Mar 2025 13:51:15 +0800 Subject: [PATCH 50/70] ci: udpate ignore and add tdgpt requirements.txt to verify the tdgpt ci workflow --- .github/workflows/tdgpt-ci.yml | 2 ++ .gitignore | 2 +- tools/tdgpt/requirements.txt | 54 ++++++++++++++++++++++++++++++++++ 3 files changed, 57 insertions(+), 1 deletion(-) create mode 100644 tools/tdgpt/requirements.txt diff --git a/.github/workflows/tdgpt-ci.yml b/.github/workflows/tdgpt-ci.yml index ece06d17a1..abe446f9c2 100644 --- a/.github/workflows/tdgpt-ci.yml +++ b/.github/workflows/tdgpt-ci.yml @@ -1,6 +1,7 @@ name: TDGpt CI Pipeline on: + push: pull_request: branches: - '3.0' @@ -25,6 +26,7 @@ jobs: cache: 'pip' - name: Install dependencies + working-directory: ${{ github.workspace }}/tools/tdgpt run: | python -m pip install --upgrade pip python -m pip install flake8 pytest pylint diff --git a/.gitignore b/.gitignore index ff27b53139..25263ec25e 100644 --- a/.gitignore +++ b/.gitignore @@ -140,7 +140,7 @@ tags *CMakeCache* *CMakeFiles* .history/ -*.txt +#*.txt *.tcl *.pc contrib/geos diff --git a/tools/tdgpt/requirements.txt b/tools/tdgpt/requirements.txt new file mode 100644 index 0000000000..ad4c42d7cf --- /dev/null +++ b/tools/tdgpt/requirements.txt @@ -0,0 +1,54 @@ +absl-py==2.1.0 +blinker==1.8.2 +click==8.1.7 +contourpy==1.3.0 +cycler==0.12.1 +Cython==3.0.11 +filelock==3.13.1 +Flask==3.0.3 +fonttools==4.54.1 +fsspec==2024.2.0 +h5py==3.12.1 +itsdangerous==2.2.0 +Jinja2==3.1.4 +joblib==1.4.2 +keras==3.6.0 +kiwisolver==1.4.7 +markdown-it-py==3.0.0 +MarkupSafe==3.0.1 +matplotlib==3.9.2 +mdurl==0.1.2 +ml_dtypes==0.5.0 +mpmath==1.3.0 +namex==0.0.8 +networkx==3.2.1 +numpy==1.26.4 +optree==0.13.0 +outlier-utils==0.0.5 +packaging==24.1 +pandas==1.5.0 +patsy==0.5.6 +pillow==10.4.0 +pmdarima==2.0.4 +pyculiarity==0.0.7 +Pygments==2.18.0 +pyparsing==3.1.4 +python-dateutil==2.9.0.post0 +pytz==2024.2 +rich==13.9.2 +rstl==0.1.3 +scikit-learn==1.5.2 +scipy==1.14.1 +six==1.16.0 +statsmodels==0.14.4 +sympy==1.12 +threadpoolctl==3.5.0 +--find-links https://download.pytorch.org/whl/torch/ +torch==2.4.1+cpu +typing_extensions==4.9.0 +urllib3==2.2.3 +uWSGI==2.0.27 +Werkzeug==3.0.4 +Flask-Testing==0.8.1 +xlsxwriter==3.2.1 +taospy==2.7.16 \ No newline at end of file From 46dcdf36b75bbe54cc874bc07f8cda9e73845343 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 10 Mar 2025 19:42:03 +0800 Subject: [PATCH 51/70] fix(stream): check return value. --- source/libs/stream/src/streamQueue.c | 15 ++++++++++++--- 1 file changed, 12 insertions(+), 3 deletions(-) diff --git a/source/libs/stream/src/streamQueue.c b/source/libs/stream/src/streamQueue.c index f338202c5b..5a507358ee 100644 --- a/source/libs/stream/src/streamQueue.c +++ b/source/libs/stream/src/streamQueue.c @@ -144,11 +144,20 @@ void streamQueueNextItemInSourceQ(SStreamQueue* pQueue, SStreamQueueItem** pItem // let's try the ordinary input q pQueue->qItem = NULL; - (void)taosGetQitem(pQueue->qall, &pQueue->qItem); + int32_t code = taosGetQitem(pQueue->qall, &pQueue->qItem); + if (code) { + stError("s-task:%s failed to extract data from inputQ, code:%s", id, tstrerror(code)); + } if (pQueue->qItem == NULL) { - (void)taosReadAllQitems(pQueue->pQueue, pQueue->qall); - (void)taosGetQitem(pQueue->qall, &pQueue->qItem); + code = taosReadAllQitems(pQueue->pQueue, pQueue->qall); + if (code) { + stError("s-task:%s failed to read qitem into qall, code:%s", id, tstrerror(code)); + } + code = taosGetQitem(pQueue->qall, &pQueue->qItem); + if (code) { + stError("s-task:%s failed to extract data from inputQ(qall), code:%s", id, tstrerror(code)); + } } *pItem = streamQueueCurItem(pQueue); From 490371055ff53fefe3b7d4ee0e3262d2647192a1 Mon Sep 17 00:00:00 2001 From: Feng Chao Date: Tue, 11 Mar 2025 08:31:38 +0800 Subject: [PATCH 52/70] ci: Update .gitignore Remove the modification for debuging --- .gitignore | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.gitignore b/.gitignore index 25263ec25e..ff27b53139 100644 --- a/.gitignore +++ b/.gitignore @@ -140,7 +140,7 @@ tags *CMakeCache* *CMakeFiles* .history/ -#*.txt +*.txt *.tcl *.pc contrib/geos From 9e248bcf91a4833a8b55ba563f4e8001e4c8ecf1 Mon Sep 17 00:00:00 2001 From: Feng Chao Date: Tue, 11 Mar 2025 09:47:32 +0800 Subject: [PATCH 53/70] Update .github/workflows/tdgpt-ci.yml Co-authored-by: WANG Xu --- .github/workflows/tdgpt-ci.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/tdgpt-ci.yml b/.github/workflows/tdgpt-ci.yml index c8fbaccd7d..961734a419 100644 --- a/.github/workflows/tdgpt-ci.yml +++ b/.github/workflows/tdgpt-ci.yml @@ -5,7 +5,7 @@ on: branches: - '3.0' paths: - - 'tools/tdgpt/*' + - 'tools/tdgpt/**' jobs: build: From cef8c45eb2375da6fbd60c16ad2f8693510ec1c2 Mon Sep 17 00:00:00 2001 From: Feng Chao Date: Tue, 11 Mar 2025 09:57:50 +0800 Subject: [PATCH 54/70] ci: Update tdgpt-ci.yml Adjust the working-directory as job level --- .github/workflows/tdgpt-ci.yml | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/.github/workflows/tdgpt-ci.yml b/.github/workflows/tdgpt-ci.yml index 961734a419..02c1c5889f 100644 --- a/.github/workflows/tdgpt-ci.yml +++ b/.github/workflows/tdgpt-ci.yml @@ -10,6 +10,7 @@ on: jobs: build: runs-on: ubuntu-latest + working-directory: ${{ github.workspace }}/tools/tdgpt strategy: fail-fast: false matrix: @@ -25,19 +26,16 @@ jobs: cache: 'pip' - name: Install dependencies - working-directory: ${{ github.workspace }}/tools/tdgpt run: | python -m pip install --upgrade pip python -m pip install flake8 pytest pylint if [ -f requirements.txt ]; then pip install -r requirements.txt; fi - name: Checking the code with pylint - working-directory: ${{ github.workspace }}/tools/tdgpt run: | pylint $(git ls-files '*.py') --exit-zero - name: Checking the code with flake8 - working-directory: ${{ github.workspace }}/tools/tdgpt run: | # stop the build if there are Python syntax errors or undefined names flake8 . --count --select=E9,F63,F7,F82 --show-source --statistics @@ -45,6 +43,5 @@ jobs: flake8 . --count --exit-zero --max-complexity=10 --max-line-length=127 --statistics - name: Run test cases with pytest - working-directory: ${{ github.workspace }}/tools/tdgpt run: | pytest From 051c34e2f1f478e68feb21ff33bf9b6957113f5c Mon Sep 17 00:00:00 2001 From: 54liuyao <54liuyao@163.com> Date: Mon, 10 Mar 2025 16:28:28 +0800 Subject: [PATCH 55/70] fix(stream): fix stream mem leak --- .../executor/src/streamcountwindowoperator.c | 3 +- .../executor/src/streameventwindowoperator.c | 3 +- .../executor/src/streamtimesliceoperator.c | 3 +- .../executor/src/streamtimewindowoperator.c | 16 ++++++---- source/libs/stream/src/tstreamFileState.c | 31 ++++++++++++------- 5 files changed, 35 insertions(+), 21 deletions(-) diff --git a/source/libs/executor/src/streamcountwindowoperator.c b/source/libs/executor/src/streamcountwindowoperator.c index 63ff2fa92b..37466aac8d 100644 --- a/source/libs/executor/src/streamcountwindowoperator.c +++ b/source/libs/executor/src/streamcountwindowoperator.c @@ -57,11 +57,12 @@ void destroyStreamCountAggOperatorInfo(void* param) { } destroyStreamBasicInfo(&pInfo->basic); - destroyStreamAggSupporter(&pInfo->streamAggSup); + cleanupExprSupp(&pInfo->scalarSupp); clearGroupResInfo(&pInfo->groupResInfo); taosArrayDestroyP(pInfo->pUpdated, destroyFlusedPos); pInfo->pUpdated = NULL; + destroyStreamAggSupporter(&pInfo->streamAggSup); colDataDestroy(&pInfo->twAggSup.timeWindowData); blockDataDestroy(pInfo->pDelRes); diff --git a/source/libs/executor/src/streameventwindowoperator.c b/source/libs/executor/src/streameventwindowoperator.c index eacb2fcfc8..f25f711783 100644 --- a/source/libs/executor/src/streameventwindowoperator.c +++ b/source/libs/executor/src/streameventwindowoperator.c @@ -56,10 +56,11 @@ void destroyStreamEventOperatorInfo(void* param) { } destroyStreamBasicInfo(&pInfo->basic); - destroyStreamAggSupporter(&pInfo->streamAggSup); + clearGroupResInfo(&pInfo->groupResInfo); taosArrayDestroyP(pInfo->pUpdated, destroyFlusedPos); pInfo->pUpdated = NULL; + destroyStreamAggSupporter(&pInfo->streamAggSup); cleanupExprSupp(&pInfo->scalarSupp); if (pInfo->pChildren != NULL) { diff --git a/source/libs/executor/src/streamtimesliceoperator.c b/source/libs/executor/src/streamtimesliceoperator.c index 681e07f452..f511c5ab4f 100644 --- a/source/libs/executor/src/streamtimesliceoperator.c +++ b/source/libs/executor/src/streamtimesliceoperator.c @@ -151,7 +151,7 @@ void destroyStreamTimeSliceOperatorInfo(void* param) { pInfo->pOperator = NULL; } colDataDestroy(&pInfo->twAggSup.timeWindowData); - destroyStreamAggSupporter(&pInfo->streamAggSup); + resetPrevAndNextWindow(pInfo->pFillSup); destroyStreamFillSupporter(pInfo->pFillSup); destroyStreamFillInfo(pInfo->pFillInfo); @@ -179,6 +179,7 @@ void destroyStreamTimeSliceOperatorInfo(void* param) { taosArrayDestroy(pInfo->historyWins); taosArrayDestroy(pInfo->pCloseTs); + destroyStreamAggSupporter(&pInfo->streamAggSup); taosMemoryFreeClear(param); } diff --git a/source/libs/executor/src/streamtimewindowoperator.c b/source/libs/executor/src/streamtimewindowoperator.c index 33efd0cfb1..c15a9f9224 100644 --- a/source/libs/executor/src/streamtimewindowoperator.c +++ b/source/libs/executor/src/streamtimewindowoperator.c @@ -462,7 +462,7 @@ _end: void destroyFlusedPos(void* pRes) { SRowBuffPos* pPos = (SRowBuffPos*)pRes; - if (!pPos->needFree && !pPos->pRowBuff) { + if (pPos->needFree && !pPos->pRowBuff) { taosMemoryFreeClear(pPos->pKey); taosMemoryFree(pPos); } @@ -475,9 +475,11 @@ void destroyFlusedppPos(void* ppRes) { void clearGroupResInfo(SGroupResInfo* pGroupResInfo) { int32_t size = taosArrayGetSize(pGroupResInfo->pRows); - for (int32_t i = pGroupResInfo->index; i < size; i++) { - void* pPos = taosArrayGetP(pGroupResInfo->pRows, i); - destroyFlusedPos(pPos); + if (pGroupResInfo->index >= 0 && pGroupResInfo->index < size) { + for (int32_t i = pGroupResInfo->index; i < size; i++) { + void* pPos = taosArrayGetP(pGroupResInfo->pRows, i); + destroyFlusedPos(pPos); + } } pGroupResInfo->freeItem = false; taosArrayDestroy(pGroupResInfo->pRows); @@ -2204,11 +2206,12 @@ void destroyStreamSessionAggOperatorInfo(void* param) { } destroyStreamBasicInfo(&pInfo->basic); - destroyStreamAggSupporter(&pInfo->streamAggSup); + cleanupExprSupp(&pInfo->scalarSupp); clearGroupResInfo(&pInfo->groupResInfo); taosArrayDestroyP(pInfo->pUpdated, destroyFlusedPos); pInfo->pUpdated = NULL; + destroyStreamAggSupporter(&pInfo->streamAggSup); if (pInfo->pChildren != NULL) { int32_t size = taosArrayGetSize(pInfo->pChildren); @@ -4442,10 +4445,11 @@ void destroyStreamStateOperatorInfo(void* param) { } destroyStreamBasicInfo(&pInfo->basic); - destroyStreamAggSupporter(&pInfo->streamAggSup); + clearGroupResInfo(&pInfo->groupResInfo); taosArrayDestroyP(pInfo->pUpdated, destroyFlusedPos); pInfo->pUpdated = NULL; + destroyStreamAggSupporter(&pInfo->streamAggSup); cleanupExprSupp(&pInfo->scalarSupp); if (pInfo->pChildren != NULL) { diff --git a/source/libs/stream/src/tstreamFileState.c b/source/libs/stream/src/tstreamFileState.c index d6dfde1ee6..0255ac01b5 100644 --- a/source/libs/stream/src/tstreamFileState.c +++ b/source/libs/stream/src/tstreamFileState.c @@ -394,14 +394,17 @@ int32_t clearFlushedRowBuff(SStreamFileState* pFileState, SStreamSnapshot* pFlus pFileState->flushMark = TMAX(pFileState->flushMark, pFileState->getTs(pPos->pKey)); pFileState->stateBuffRemoveByPosFn(pFileState, pPos); - SListNode* tmp = tdListPopNode(pFileState->usedBuffs, pNode); - taosMemoryFreeClear(tmp); + if (pPos->beUsed == false) { + SListNode* tmp = tdListPopNode(pFileState->usedBuffs, pNode); + taosMemoryFreeClear(tmp); + } if (pPos->pRowBuff) { i++; } } } } + qDebug("clear flushed row buff. %d rows to disk. is all:%d", listNEles(pFlushList), all); _end: if (code != TSDB_CODE_SUCCESS) { @@ -433,7 +436,6 @@ int32_t popUsedBuffs(SStreamFileState* pFileState, SStreamSnapshot* pFlushList, SRowBuffPos* pPos = *(SRowBuffPos**)pNode->data; if (pPos->beUsed == used) { if (used && !pPos->pRowBuff) { - QUERY_CHECK_CONDITION((pPos->needFree == true), code, lino, _end, TSDB_CODE_QRY_EXECUTOR_INTERNAL_ERROR); continue; } code = tdListAppend(pFlushList, &pPos); @@ -441,8 +443,10 @@ int32_t popUsedBuffs(SStreamFileState* pFileState, SStreamSnapshot* pFlushList, pFileState->flushMark = TMAX(pFileState->flushMark, pFileState->getTs(pPos->pKey)); pFileState->stateBuffRemoveByPosFn(pFileState, pPos); - SListNode* tmp = tdListPopNode(pFileState->usedBuffs, pNode); - taosMemoryFreeClear(tmp); + if (pPos->beUsed == false) { + SListNode* tmp = tdListPopNode(pFileState->usedBuffs, pNode); + taosMemoryFreeClear(tmp); + } if (pPos->pRowBuff) { i++; } @@ -511,9 +515,12 @@ int32_t clearRowBuff(SStreamFileState* pFileState) { if (pFileState->deleteMark != INT64_MAX) { clearExpiredRowBuff(pFileState, pFileState->maxTs - pFileState->deleteMark, false); } - if (isListEmpty(pFileState->freeBuffs)) { - return flushRowBuff(pFileState); - } + do { + int32_t code = flushRowBuff(pFileState); + if (code != TSDB_CODE_SUCCESS) { + return code; + } + } while (isListEmpty(pFileState->freeBuffs) && pFileState->curRowCount == pFileState->maxRowCount); return TSDB_CODE_SUCCESS; } @@ -756,10 +763,10 @@ int32_t getRowBuffByPos(SStreamFileState* pFileState, SRowBuffPos* pPos, void** QUERY_CHECK_CODE(code, lino, _end); (*pVal) = pPos->pRowBuff; - if (!pPos->needFree) { - code = tdListPrepend(pFileState->usedBuffs, &pPos); - QUERY_CHECK_CODE(code, lino, _end); - } + // if (!pPos->needFree) { + // code = tdListPrepend(pFileState->usedBuffs, &pPos); + // QUERY_CHECK_CODE(code, lino, _end); + // } _end: if (code != TSDB_CODE_SUCCESS) { From 36183aa36bb69db1a0c0d04e1ad5e4082855ac0f Mon Sep 17 00:00:00 2001 From: factosea <285808407@qq.com> Date: Wed, 5 Mar 2025 18:28:48 +0800 Subject: [PATCH 56/70] feat: greatest func --- include/common/tglobal.h | 1 + include/common/ttypes.h | 3 + include/libs/function/functionMgt.h | 2 + include/libs/scalar/filter.h | 2 + include/libs/scalar/scalar.h | 3 + source/common/src/tglobal.c | 10 ++- source/libs/function/src/builtins.c | 66 ++++++++++++++ source/libs/scalar/src/sclfunc.c | 133 ++++++++++++++++++++++++++++ source/libs/scalar/src/sclvector.c | 4 +- tools/shell/src/shellEngine.c | 2 +- 10 files changed, 222 insertions(+), 4 deletions(-) diff --git a/include/common/tglobal.h b/include/common/tglobal.h index 8acff4e04b..da62966493 100644 --- a/include/common/tglobal.h +++ b/include/common/tglobal.h @@ -299,6 +299,7 @@ extern bool tsStreamCoverage; extern int8_t tsS3EpNum; extern int32_t tsStreamNotifyMessageSize; extern int32_t tsStreamNotifyFrameSize; +extern bool tsTransToStrWhenMixTypeInLeast; extern bool tsExperimental; // #define NEEDTO_COMPRESSS_MSG(size) (tsCompressMsgSize != -1 && (size) > tsCompressMsgSize) diff --git a/include/common/ttypes.h b/include/common/ttypes.h index d0cddd87a9..95fe14e572 100644 --- a/include/common/ttypes.h +++ b/include/common/ttypes.h @@ -276,6 +276,9 @@ typedef struct { #define IS_STR_DATA_TYPE(t) \ (((t) == TSDB_DATA_TYPE_VARCHAR) || ((t) == TSDB_DATA_TYPE_VARBINARY) || ((t) == TSDB_DATA_TYPE_NCHAR)) +#define IS_COMPARE_STR_DATA_TYPE(t) \ + (((t) == TSDB_DATA_TYPE_VARCHAR) || ((t) == TSDB_DATA_TYPE_NCHAR)) + #define IS_VALID_TINYINT(_t) ((_t) >= INT8_MIN && (_t) <= INT8_MAX) #define IS_VALID_SMALLINT(_t) ((_t) >= INT16_MIN && (_t) <= INT16_MAX) #define IS_VALID_INT(_t) ((_t) >= INT32_MIN && (_t) <= INT32_MAX) diff --git a/include/libs/function/functionMgt.h b/include/libs/function/functionMgt.h index 41b5d76371..1949081993 100644 --- a/include/libs/function/functionMgt.h +++ b/include/libs/function/functionMgt.h @@ -90,6 +90,8 @@ typedef enum EFunctionType { FUNCTION_TYPE_DEGREES, FUNCTION_TYPE_RADIANS, FUNCTION_TYPE_TRUNCATE, + FUNCTION_TYPE_GREATEST, + FUNCTION_TYPE_LEAST, // string function FUNCTION_TYPE_LENGTH = 1500, diff --git a/include/libs/scalar/filter.h b/include/libs/scalar/filter.h index a98bf29afb..99f1759a7a 100644 --- a/include/libs/scalar/filter.h +++ b/include/libs/scalar/filter.h @@ -66,6 +66,8 @@ int32_t filterPartitionCond(SNode **pCondition, SNode **pPrimaryKeyCond, SNode * SNode **pOtherCond); int32_t filterIsMultiTableColsCond(SNode *pCond, bool *res); EConditionType filterClassifyCondition(SNode *pNode); +int32_t filterGetCompFunc(__compar_fn_t *func, int32_t type, int32_t optr); +bool filterDoCompare(__compar_fn_t func, uint8_t optr, void *left, void *right); #ifdef __cplusplus } diff --git a/include/libs/scalar/scalar.h b/include/libs/scalar/scalar.h index 67fd954ad7..d1dda544ae 100644 --- a/include/libs/scalar/scalar.h +++ b/include/libs/scalar/scalar.h @@ -44,6 +44,7 @@ int32_t scalarGenerateSetFromList(void **data, void *pNode, uint32_t type, int8_ int32_t vectorGetConvertType(int32_t type1, int32_t type2); int32_t vectorConvertSingleColImpl(const SScalarParam *pIn, SScalarParam *pOut, int32_t *overflow, int32_t startIndex, int32_t numOfRows); +int32_t vectorConvertSingleCol(SScalarParam *input, SScalarParam *output, int32_t type, int32_t startIndex, int32_t numOfRows); /* Math functions */ int32_t absFunction(SScalarParam *pInput, int32_t inputNum, SScalarParam *pOutput); @@ -71,6 +72,8 @@ int32_t signFunction(SScalarParam *pInput, int32_t inputNum, SScalarParam *pOutp int32_t degreesFunction(SScalarParam *pInput, int32_t inputNum, SScalarParam *pOutput); int32_t radiansFunction(SScalarParam *pInput, int32_t inputNum, SScalarParam *pOutput); int32_t randFunction(SScalarParam *pInput, int32_t inputNum, SScalarParam *pOutput); +int32_t greatestFunction(SScalarParam *pInput, int32_t inputNum, SScalarParam *pOutput); +int32_t leastFunction(SScalarParam *pInput, int32_t inputNum, SScalarParam *pOutput); /* String functions */ int32_t lengthFunction(SScalarParam *pInput, int32_t inputNum, SScalarParam *pOutput); diff --git a/source/common/src/tglobal.c b/source/common/src/tglobal.c index a3c8f4fd52..235843c041 100644 --- a/source/common/src/tglobal.c +++ b/source/common/src/tglobal.c @@ -130,6 +130,8 @@ uint32_t tsEncryptionKeyChksum = 0; int8_t tsEncryptionKeyStat = ENCRYPT_KEY_STAT_UNSET; int8_t tsGrant = 1; +bool tsTransToStrWhenMixTypeInLeast = true; + // monitor bool tsEnableMonitor = true; int32_t tsMonitorInterval = 30; @@ -746,6 +748,8 @@ static int32_t taosAddClientCfg(SConfig *pCfg) { TAOS_CHECK_RETURN( cfgAddBool(pCfg, "streamCoverage", tsStreamCoverage, CFG_DYN_CLIENT, CFG_DYN_CLIENT, CFG_CATEGORY_LOCAL)); + + TAOS_CHECK_RETURN(cfgAddBool(pCfg, "transToStrWhenMixTypeInLeast", tsTransToStrWhenMixTypeInLeast, CFG_SCOPE_CLIENT, CFG_DYN_CLIENT,CFG_CATEGORY_LOCAL)); TAOS_RETURN(TSDB_CODE_SUCCESS); } @@ -1480,6 +1484,9 @@ static int32_t taosSetClientCfg(SConfig *pCfg) { TAOS_CHECK_GET_CFG_ITEM(pCfg, pItem, "streamCoverage"); tsStreamCoverage = pItem->bval; + TAOS_CHECK_GET_CFG_ITEM(pCfg, pItem, "transToStrWhenMixTypeInLeast"); + tsTransToStrWhenMixTypeInLeast = pItem->bval; + TAOS_RETURN(TSDB_CODE_SUCCESS); } @@ -2783,7 +2790,8 @@ static int32_t taosCfgDynamicOptionsForClient(SConfig *pCfg, const char *name) { {"numOfRpcSessions", &tsNumOfRpcSessions}, {"bypassFlag", &tsBypassFlag}, {"safetyCheckLevel", &tsSafetyCheckLevel}, - {"streamCoverage", &tsStreamCoverage}}; + {"streamCoverage", &tsStreamCoverage}, + {"transToStrWhenMixTypeInLeast", &tsTransToStrWhenMixTypeInLeast}}; if ((code = taosCfgSetOption(debugOptions, tListLen(debugOptions), pItem, true)) != TSDB_CODE_SUCCESS) { code = taosCfgSetOption(options, tListLen(options), pItem, false); diff --git a/source/libs/function/src/builtins.c b/source/libs/function/src/builtins.c index be3f0d362b..a63596d0d2 100644 --- a/source/libs/function/src/builtins.c +++ b/source/libs/function/src/builtins.c @@ -22,6 +22,9 @@ #include "tanalytics.h" #include "taoserror.h" #include "ttime.h" +#include "functionMgt.h" +#include "ttypes.h" +#include "tglobal.h" static int32_t buildFuncErrMsg(char* pErrBuf, int32_t len, int32_t errCode, const char* pFormat, ...) { va_list vArgList; @@ -1745,6 +1748,49 @@ static int32_t translateHistogramPartial(SFunctionNode* pFunc, char* pErrBuf, in return TSDB_CODE_SUCCESS; } +static int32_t translateGreatestleast(SFunctionNode* pFunc, char* pErrBuf, int32_t len) { + if (LIST_LENGTH(pFunc->pParameterList) < 2) { + return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName); + } + + bool mixTypeToStrings = tsTransToStrWhenMixTypeInLeast; + + SDataType res = {.type = 0}; + for (int32_t i = 0; i < LIST_LENGTH(pFunc->pParameterList); i++) { + SDataType* para = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, i)); + + if (IS_NULL_TYPE(para->type)) { + res.type = TSDB_DATA_TYPE_NULL; + break; + } else if (IS_MATHABLE_TYPE(para->type)) { + if(res.type == 0) { + res.type = para->type; + res.bytes = para->bytes; + } else if(IS_MATHABLE_TYPE(res.type) || !mixTypeToStrings) { + int32_t resType = vectorGetConvertType(res.type, para->type); + res.type = resType == 0 ? res.type : resType; + res.bytes = tDataTypes[resType].bytes; + } + } else if (IS_COMPARE_STR_DATA_TYPE(para->type)) { + if(res.type == 0) { + res.type = para->type; + res.bytes = para->bytes; + } else if(IS_COMPARE_STR_DATA_TYPE(res.type)) { + int32_t resType = vectorGetConvertType(res.type, para->type); + res.type = resType == 0 ? res.type : resType; + res.bytes = TMAX(res.bytes, para->bytes); + } else if(mixTypeToStrings) { // res.type is mathable type + res.type = para->type; + res.bytes = para->bytes; + } + } else { + return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName); + } + } + pFunc->node.resType = res; + return TSDB_CODE_SUCCESS; +} + // clang-format off const SBuiltinFuncDefinition funcMgtBuiltins[] = { { @@ -5656,6 +5702,26 @@ const SBuiltinFuncDefinition funcMgtBuiltins[] = { .name = "cols", .translateFunc = invalidColsFunction, }, + { + .name = "greatest", + .type = FUNCTION_TYPE_GREATEST, + .classification = FUNC_MGT_SCALAR_FUNC, + .translateFunc = translateGreatestleast, + .getEnvFunc = NULL, + .initFunc = NULL, + .sprocessFunc = greatestFunction, + .finalizeFunc = NULL + }, + { + .name = "least", + .type = FUNCTION_TYPE_LEAST, + .classification = FUNC_MGT_SCALAR_FUNC, + .translateFunc = translateGreatestleast, + .getEnvFunc = NULL, + .initFunc = NULL, + .sprocessFunc = leastFunction, + .finalizeFunc = NULL + }, }; // clang-format on diff --git a/source/libs/scalar/src/sclfunc.c b/source/libs/scalar/src/sclfunc.c index f3b56da372..3ce310a261 100644 --- a/source/libs/scalar/src/sclfunc.c +++ b/source/libs/scalar/src/sclfunc.c @@ -1,11 +1,14 @@ +#include #include "cJSON.h" #include "function.h" #include "scalar.h" #include "sclInt.h" #include "sclvector.h" #include "tdatablock.h" +#include "tdef.h" #include "tjson.h" #include "ttime.h" +#include "filter.h" typedef float (*_float_fn)(float); typedef float (*_float_fn_2)(float, float); @@ -4403,3 +4406,133 @@ int32_t modeScalarFunction(SScalarParam *pInput, int32_t inputNum, SScalarParam return selectScalarFunction(pInput, inputNum, pOutput); } +typedef struct SCovertScarlarParam { + SScalarParam covertParam; + SScalarParam *param; + bool converted; +} SCovertScarlarParam; + +void freeSCovertScarlarParams(SCovertScarlarParam *pCovertParams, int32_t num) { + if (pCovertParams == NULL) { + return; + } + for (int32_t i = 0; i < num; i++) { + if (pCovertParams[i].converted) { + sclFreeParam(pCovertParams[i].param); + } + } + taosMemoryFree(pCovertParams); +} + +static int32_t vectorCompareAndSelect(SCovertScarlarParam *pParams, int32_t numOfRows, int numOfCols, + int32_t *resultColIndex, EOperatorType optr) { + int32_t code = TSDB_CODE_SUCCESS; + int32_t type = GET_PARAM_TYPE(pParams[0].param); + + __compar_fn_t fp = NULL; + code = filterGetCompFunc(&fp, type, optr); + if(code != TSDB_CODE_SUCCESS) { + qError("failed to get compare function, func:%s type:%d, optr:%d", __FUNCTION__, type, optr); + return code; + } + + for (int32_t i = 0; i < numOfRows; i++) { + int selectIndex = 0; + if (colDataIsNull_s(pParams[selectIndex].param->columnData, i)) { + resultColIndex[i] = -1; + continue; + } + for (int32_t j = 1; j < numOfCols; j++) { + if (colDataIsNull_s(pParams[j].param->columnData, i)) { + resultColIndex[i] = -1; + break; + } else { + int32_t leftRowNo = pParams[selectIndex].param->numOfRows == 1 ? 0 : i; + int32_t rightRowNo = pParams[j].param->numOfRows == 1 ? 0 : i; + char *pLeftData = colDataGetData(pParams[selectIndex].param->columnData, leftRowNo); + char *pRightData = colDataGetData(pParams[j].param->columnData, rightRowNo); + bool pRes = filterDoCompare(fp, optr, pLeftData, pRightData); + if (!pRes) { + selectIndex = j; + } + } + resultColIndex[i] = selectIndex; + } + } + + return code; +} + +static int32_t greatestLeastImpl(SScalarParam *pInput, int32_t inputNum, SScalarParam *pOutput, EOperatorType order) { + int32_t code = TSDB_CODE_SUCCESS; + SColumnInfoData *pOutputData = pOutput[0].columnData; + int16_t outputType = GET_PARAM_TYPE(&pOutput[0]); + int64_t outputLen = GET_PARAM_BYTES(&pOutput[0]); + + SCovertScarlarParam *pCovertParams = NULL; + int32_t *resultColIndex = NULL; + + int32_t numOfRows = 0; + bool IsNullType = false; + // If any column is NULL type, the output is NULL type + for (int32_t i = 0; i < inputNum; i++) { + if (numOfRows != 0 && numOfRows != pInput[i].numOfRows && pInput[i].numOfRows != 1 && numOfRows != 1) { + qError("input rows not match, func:%s, rows:%d, %d", __FUNCTION__, numOfRows, pInput[i].numOfRows); + code = TSDB_CODE_TSC_INTERNAL_ERROR; + goto _return; + } + numOfRows = TMAX(numOfRows, pInput[i].numOfRows); + IsNullType |= IS_NULL_TYPE(GET_PARAM_TYPE(&pInput[i])); + } + + if (IsNullType) { + colDataSetNNULL(pOutputData, 0, numOfRows); + pOutput->numOfRows = numOfRows; + return TSDB_CODE_SUCCESS; + } + pCovertParams = taosMemoryMalloc(inputNum * sizeof(SCovertScarlarParam)); + for (int32_t j = 0; j < inputNum; j++) { + SScalarParam *pParam = &pInput[j]; + int16_t oldType = GET_PARAM_TYPE(&pInput[j]); + if (oldType != outputType) { + pCovertParams[j].covertParam = (SScalarParam){0}; + setTzCharset(&pCovertParams[j].covertParam, pParam->tz, pParam->charsetCxt); + SCL_ERR_JRET(vectorConvertSingleCol(pParam, &pCovertParams[j].covertParam, outputType, 0, pParam->numOfRows)); + pCovertParams[j].param = &pCovertParams[j].covertParam; + pCovertParams[j].converted = true; + } else { + pCovertParams[j].param = pParam; + pCovertParams[j].converted = false; + } + } + + resultColIndex = taosMemoryCalloc(numOfRows, sizeof(int32_t)); + SCL_ERR_JRET(vectorCompareAndSelect(pCovertParams, numOfRows, inputNum, resultColIndex, order)); + + for (int32_t i = 0; i < numOfRows; i++) { + int32_t index = resultColIndex[i]; + if (index == -1) { + colDataSetNULL(pOutputData, i); + continue; + } + int32_t rowNo = pCovertParams[index].param->numOfRows == 1 ? 0 : i; + char *data = colDataGetData(pCovertParams[index].param->columnData, rowNo); + SCL_ERR_JRET(colDataSetVal(pOutputData, i, data, false)); + } + + pOutput->numOfRows = numOfRows; + +_return: + freeSCovertScarlarParams(pCovertParams, inputNum); + taosMemoryFree(resultColIndex); + return code; +} + +int32_t greatestFunction(SScalarParam *pInput, int32_t inputNum, SScalarParam *pOutput) { + return greatestLeastImpl(pInput, inputNum, pOutput, OP_TYPE_GREATER_THAN); +} + +int32_t leastFunction(SScalarParam *pInput, int32_t inputNum, SScalarParam *pOutput) { + return greatestLeastImpl(pInput, inputNum, pOutput, OP_TYPE_LOWER_THAN); +} + diff --git a/source/libs/scalar/src/sclvector.c b/source/libs/scalar/src/sclvector.c index 14dae1226d..ff45baa1e0 100644 --- a/source/libs/scalar/src/sclvector.c +++ b/source/libs/scalar/src/sclvector.c @@ -996,7 +996,7 @@ int32_t vectorConvertSingleColImpl(const SScalarParam *pIn, SScalarParam *pOut, } int8_t gConvertTypes[TSDB_DATA_TYPE_MAX][TSDB_DATA_TYPE_MAX] = { - /*NULL BOOL TINY SMAL INT BIG FLOA DOUB VARC TIME NCHA UTIN USMA UINT UBIG JSON VARB DECI BLOB MEDB GEOM*/ + /* NULL BOOL TINY SMAL INT BIG FLOA DOUB VARC TIME NCHA UTIN USMA UINT UBIG JSON VARB DECI BLOB MEDB GEOM*/ /*NULL*/ 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, /*BOOL*/ 0, 0, 2, 3, 4, 5, 6, 7, 5, 9, 5, 11, 12, 13, 14, 0, -1, 0, 0, 0, -1, /*TINY*/ 0, 0, 0, 3, 4, 5, 6, 7, 5, 9, 5, 3, 4, 5, 7, 0, -1, 0, 0, 0, -1, @@ -1021,7 +1021,7 @@ int8_t gConvertTypes[TSDB_DATA_TYPE_MAX][TSDB_DATA_TYPE_MAX] = { }; int8_t gDisplyTypes[TSDB_DATA_TYPE_MAX][TSDB_DATA_TYPE_MAX] = { - /*NULL BOOL TINY SMAL INT BIGI FLOA DOUB VARC TIM NCHA UTIN USMA UINT UBIG JSON VARB DECI BLOB MEDB GEOM*/ + /* NULL BOOL TINY SMAL INT BIGI FLOA DOUB VARC TIM NCHA UTIN USMA UINT UBIG JSON VARB DECI BLOB MEDB GEOM*/ /*NULL*/ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, -1, -1, -1, 20, /*BOOL*/ 0, 1, 2, 3, 4, 5, 6, 7, 8, 5, 10, 11, 12, 13, 14, -1, -1, -1, -1, -1, -1, /*TINY*/ 0, 0, 2, 3, 4, 5, 8, 8, 8, 5, 10, 3, 4, 5, 8, -1, -1, -1, -1, -1, -1, diff --git a/tools/shell/src/shellEngine.c b/tools/shell/src/shellEngine.c index a3e542a768..2a405d8f38 100644 --- a/tools/shell/src/shellEngine.c +++ b/tools/shell/src/shellEngine.c @@ -672,7 +672,7 @@ void shellPrintField(const char *val, TAOS_FIELD *field, int32_t width, int32_t printf("%*.7e", width, GET_FLOAT_VAL(val)); } else { n = snprintf(buf, LENGTH, "%*.*g", width, FLT_DIG, GET_FLOAT_VAL(val)); - if (n > SHELL_FLOAT_WIDTH) { + if (n > width) { printf("%*.7e", width, GET_FLOAT_VAL(val)); } else { printf("%s", buf); From b3ef8b2f44eb6bfeb2d3f4ad50e989fab083a697 Mon Sep 17 00:00:00 2001 From: factosea <285808407@qq.com> Date: Thu, 6 Mar 2025 10:23:01 +0800 Subject: [PATCH 57/70] fix: print folat when title name is too long --- tests/army/query/function/ans/ascii.csv | 2 +- tests/army/query/function/ans/char_length.csv | 2 +- tests/army/query/function/ans/max.csv | 4 +- tests/army/query/function/ans/min.csv | 4 +- tests/army/query/function/ans/mod.csv | 44 +++++------ tests/army/query/function/ans/pi.csv | 2 +- tests/army/query/function/ans/position.csv | 2 +- tests/army/query/function/ans/rand.csv | 4 +- tests/army/query/function/ans/round.csv | 26 +++---- tests/army/query/function/ans/sign.csv | 42 +++++------ tests/army/query/function/ans/stddev.csv | 6 +- tests/army/query/function/ans/trunc.csv | 74 +++++++++---------- tests/army/query/function/ans/varpop.csv | 2 +- tools/shell/src/shellEngine.c | 16 +--- 14 files changed, 111 insertions(+), 119 deletions(-) diff --git a/tests/army/query/function/ans/ascii.csv b/tests/army/query/function/ans/ascii.csv index 8d186f63e3..0e67acc9d1 100644 --- a/tests/army/query/function/ans/ascii.csv +++ b/tests/army/query/function/ans/ascii.csv @@ -51,7 +51,7 @@ taos> select ASCII('hello') + 1 from ts_4893.meters limit 1 taos> select ASCII('hello') + ASCII('hello') from ts_4893.meters limit 1 ascii('hello') + ascii('hello') | ================================== - 2.080000000000000e+02 | + 208 | taos> select ASCII(nch1) from ts_4893.meters order by ts limit 5 ascii(nch1) | diff --git a/tests/army/query/function/ans/char_length.csv b/tests/army/query/function/ans/char_length.csv index 4b79a42bfa..a8352357cf 100644 --- a/tests/army/query/function/ans/char_length.csv +++ b/tests/army/query/function/ans/char_length.csv @@ -51,7 +51,7 @@ taos> select CHAR_LENGTH('hello') + 1 from ts_4893.meters limit 1 taos> select CHAR_LENGTH('hello') + CHAR_LENGTH('hello') from ts_4893.meters limit 1 char_length('hello') + char_length('hello') | ============================================== - 1.000000000000000e+01 | + 10 | taos> select CHAR_LENGTH(nch1) from ts_4893.meters order by ts limit 5 char_length(nch1) | diff --git a/tests/army/query/function/ans/max.csv b/tests/army/query/function/ans/max.csv index 40f8b2ac23..e13a26847b 100644 --- a/tests/army/query/function/ans/max.csv +++ b/tests/army/query/function/ans/max.csv @@ -576,7 +576,7 @@ taos> select max(total_voltage) from (select sum(voltage) as total_voltage from taos> select round(max(current), 2) from ts_4893.meters round(max(current), 2) | ========================= - 1.2000000e+01 | + 12 | taos> select pow(max(current), 2) from ts_4893.meters pow(max(current), 2) | @@ -651,7 +651,7 @@ taos> select max(cast(10000000000 as bigint unsigned)) taos> select max(cast(1.1 as float)) max(cast(1.1 as float)) | ========================== - 1.1000000e+00 | + 1.1 | taos> select max(cast(1.1 as double)) max(cast(1.1 as double)) | diff --git a/tests/army/query/function/ans/min.csv b/tests/army/query/function/ans/min.csv index d77744a4f9..6426276446 100644 --- a/tests/army/query/function/ans/min.csv +++ b/tests/army/query/function/ans/min.csv @@ -576,7 +576,7 @@ taos> select min(total_voltage) from (select sum(voltage) as total_voltage from taos> select round(min(current), 2) from ts_4893.meters round(min(current), 2) | ========================= - 8.0000000e+00 | + 8 | taos> select pow(min(current), 2) from ts_4893.meters pow(min(current), 2) | @@ -651,7 +651,7 @@ taos> select min(cast(10000000000 as bigint unsigned)) taos> select min(cast(1.1 as float)) min(cast(1.1 as float)) | ========================== - 1.1000000e+00 | + 1.1 | taos> select min(cast(1.1 as double)) min(cast(1.1 as double)) | diff --git a/tests/army/query/function/ans/mod.csv b/tests/army/query/function/ans/mod.csv index 9d2232f49d..05770289db 100644 --- a/tests/army/query/function/ans/mod.csv +++ b/tests/army/query/function/ans/mod.csv @@ -42,26 +42,26 @@ taos> select MOD(10.55, 1) + 1 taos> select MOD(MOD(MOD(MOD(MOD(MOD(MOD(123.123456789, 9), 8), 7), 6), 5), 4), 3) mod(mod(mod(mod(mod(mod(mod(123.123456789, 9), 8), 7), 6), 5), 4 | =================================================================== - 1.234567890000022e-01 | + 0.12345678900000 | taos> select MOD(MOD(MOD(MOD(MOD(MOD(MOD(123456789.123456789, -1), -2), -3), -4), -5), -6), -7) mod(mod(mod(mod(mod(mod(mod(123456789.123456789, -1), -2), -3), | =================================================================== - 1.234567910432816e-01 | + 0.12345679104328 | taos> select MOD(87654321.123456789, id + 1) from ts_4893.meters order by ts limit 10 mod(87654321.123456789, id + 1) | ================================== - 1.234567910432816e-01 | - 1.123456791043282e+00 | - 1.234567910432816e-01 | - 1.123456791043282e+00 | - 1.123456791043282e+00 | - 3.123456791043282e+00 | - 6.123456791043282e+00 | - 1.123456791043282e+00 | - 1.234567910432816e-01 | - 1.123456791043282e+00 | + 0.123456791043282 | + 1.12345679104328 | + 0.123456791043282 | + 1.12345679104328 | + 1.12345679104328 | + 3.12345679104328 | + 6.12345679104328 | + 1.12345679104328 | + 0.123456791043282 | + 1.12345679104328 | taos> select MOD(current, id + 1) from ts_4893.meters order by ts limit 10 mod(current, id + 1) | @@ -94,16 +94,16 @@ taos> select MOD(current, 1) from ts_4893.meters order by ts limit 10 taos> select MOD(sqrt(current), abs(id + 1)) from ts_4893.meters order by ts limit 10 mod(sqrt(current), abs(id + 1)) | ================================== - 2.634337159700784e-01 | - 9.281394021770111e-01 | - 1.296964830944782e-01 | - 3.351566768190027e+00 | - 3.272002495118848e+00 | - 2.916847677517688e+00 | - 3.097741066924800e+00 | - 3.310891102586806e+00 | - 3.350522322288470e+00 | - 3.215120509901375e+00 | + 0.263433715970078 | + 0.928139402177011 | + 0.129696483094478 | + 3.35156676819003 | + 3.27200249511885 | + 2.91684767751769 | + 3.0977410669248 | + 3.31089110258681 | + 3.35052232228847 | + 3.21512050990137 | taos> select mod(10, -3) mod(10, -3) | diff --git a/tests/army/query/function/ans/pi.csv b/tests/army/query/function/ans/pi.csv index fb7d662ca3..a364f472c6 100644 --- a/tests/army/query/function/ans/pi.csv +++ b/tests/army/query/function/ans/pi.csv @@ -190,5 +190,5 @@ taos> select voltage / pi() from ts_4893.meters limit 1 taos> select id, case when voltage > 100 then pi() else pi() / 2 end from ts_4893.meters limit 1 id | case when voltage > 100 then pi() else pi() / 2 end | ==================================================================== - 0 | 3.141592653589793e+00 | + 0 | 3.14159265358979 | diff --git a/tests/army/query/function/ans/position.csv b/tests/army/query/function/ans/position.csv index 1547564322..8e082ef574 100644 --- a/tests/army/query/function/ans/position.csv +++ b/tests/army/query/function/ans/position.csv @@ -201,7 +201,7 @@ taos> select ABS(POSITION('aaa' IN 'aaaaaaaaa')) taos> select POW(POSITION('aaa' IN 'aaaaaaaaa'), 2) pow(position('aaa' in 'aaaaaaaaa'), 2) | ========================================= - 1.000000000000000e+00 | + 1 | taos> select position('t' in 'taos') position('t' in 'taos') | diff --git a/tests/army/query/function/ans/rand.csv b/tests/army/query/function/ans/rand.csv index 2e87f3404f..5685bd901e 100644 --- a/tests/army/query/function/ans/rand.csv +++ b/tests/army/query/function/ans/rand.csv @@ -50,12 +50,12 @@ taos> select rand(-1) taos> select rand(12345678901234567890) rand(12345678901234567890) | ============================= - 2.008294813338805e-01 | + 0.20082948133388 | taos> select rand(-12345678901234567890) rand(-12345678901234567890) | ============================== - 8.401877171547095e-01 | + 0.84018771715471 | taos> select rand(12345), rand(12345) rand(12345) | rand(12345) | diff --git a/tests/army/query/function/ans/round.csv b/tests/army/query/function/ans/round.csv index 2c185abd73..023c1fd43d 100644 --- a/tests/army/query/function/ans/round.csv +++ b/tests/army/query/function/ans/round.csv @@ -67,12 +67,12 @@ taos> select ROUND(10.55, 1) + 1 taos> select ROUND(ROUND(ROUND(ROUND(ROUND(ROUND(ROUND(123.123456789, 9), 8), 7), 6), 5), 4)) round(round(round(round(round(round(round(123.123456789, 9), 8), | =================================================================== - 1.230000000000000e+02 | + 12 | taos> select ROUND(ROUND(ROUND(ROUND(ROUND(ROUND(ROUND(123456789.123456789, -1), -2), -3), -4), -5), -6)) round(round(round(round(round(round(round(123456789.123456789, - | =================================================================== - 1.230000000000000e+08 | + 12300000 | taos> select ROUND(current) from ts_4893.meters order by ts limit 20 round(current) | @@ -101,16 +101,16 @@ taos> select ROUND(current) from ts_4893.meters order by ts limit 20 taos> select ROUND(87654321.123456789, id) from ts_4893.meters order by ts limit 10 round(87654321.123456789, id) | ================================ - 8.765432100000000e+07 | - 8.765432109999999e+07 | - 8.765432112000000e+07 | - 8.765432112300000e+07 | - 8.765432112350000e+07 | - 8.765432112345999e+07 | - 8.765432112345700e+07 | - 8.765432112345681e+07 | - 8.765432112345679e+07 | - 8.765432112345679e+07 | + 87654321 | + 87654321.1 | + 87654321.12 | + 87654321.123 | + 87654321.1235 | + 87654321.12346 | + 87654321.123457 | + 87654321.1234568 | + 87654321.1234568 | + 87654321.1234568 | taos> select ROUND(current, id) from ts_4893.meters order by ts limit 10 round(current, id) | @@ -286,7 +286,7 @@ taos> select round(voltage, -1) from ts_4893.meters limit 1 taos> select round(current * voltage, 2) from ts_4893.meters limit 1 round(current * voltage, 2) | ============================== - 2.353650000000000e+03 | + 2353.65 | taos> select round(abs(voltage), 2) from ts_4893.meters limit 1 round(abs(voltage), 2) | diff --git a/tests/army/query/function/ans/sign.csv b/tests/army/query/function/ans/sign.csv index f8fc961c2b..700e88fde6 100644 --- a/tests/army/query/function/ans/sign.csv +++ b/tests/army/query/function/ans/sign.csv @@ -164,7 +164,7 @@ taos> select sign(cast(1 as bigint unsigned)) taos> select sign(cast(1 as float)) sign(cast(1 as float)) | ========================= - 1.0000000e+00 | + 1 | taos> select sign(cast(1 as double)) sign(cast(1 as double)) | @@ -316,30 +316,30 @@ taos> select sign(current) from ts_4893.meters order by ts limit 10 taos> select sign(cast(current as float)) from ts_4893.d0 order by ts limit 10 sign(cast(current as float)) | =============================== - 1.0000000e+00 | - 1.0000000e+00 | - 1.0000000e+00 | - 1.0000000e+00 | - 1.0000000e+00 | - 1.0000000e+00 | - 1.0000000e+00 | - 1.0000000e+00 | - 1.0000000e+00 | - 1.0000000e+00 | + 1 | + 1 | + 1 | + 1 | + 1 | + 1 | + 1 | + 1 | + 1 | + 1 | taos> select sign(cast(current as float)) from ts_4893.meters order by ts limit 10 sign(cast(current as float)) | =============================== - 1.0000000e+00 | - 1.0000000e+00 | - 1.0000000e+00 | - 1.0000000e+00 | - 1.0000000e+00 | - 1.0000000e+00 | - 1.0000000e+00 | - 1.0000000e+00 | - 1.0000000e+00 | - 1.0000000e+00 | + 1 | + 1 | + 1 | + 1 | + 1 | + 1 | + 1 | + 1 | + 1 | + 1 | taos> select sign(null) sign(null) | diff --git a/tests/army/query/function/ans/stddev.csv b/tests/army/query/function/ans/stddev.csv index c0f93a9bcd..02f83a7742 100644 --- a/tests/army/query/function/ans/stddev.csv +++ b/tests/army/query/function/ans/stddev.csv @@ -109,17 +109,17 @@ taos> select stddev_pop(total_voltage) from (select sum(voltage) as total_voltag taos> select round(stddev_pop(current), 2) from ts_4893.meters round(stddev_pop(current), 2) | ================================ - 1.150000000000000e+00 | + 1.15 | taos> select pow(stddev_pop(current), 2) from ts_4893.meters pow(stddev_pop(current), 2) | ============================== - 1.332500071133751e+00 | + 1.33250007113375 | taos> select log(stddev_pop(voltage) + 1) from ts_4893.meters log(stddev_pop(voltage) + 1) | =============================== - 1.354922290183882e+00 | + 1.35492229018388 | taos> select groupid, stddev_pop(voltage) from ts_4893.meters group by groupid order by groupid groupid | stddev_pop(voltage) | diff --git a/tests/army/query/function/ans/trunc.csv b/tests/army/query/function/ans/trunc.csv index 15411afbf3..77b6127cba 100644 --- a/tests/army/query/function/ans/trunc.csv +++ b/tests/army/query/function/ans/trunc.csv @@ -47,40 +47,40 @@ taos> select TRUNCATE(10.55, 1) + 1 taos> select TRUNCATE(TRUNCATE(TRUNCATE(TRUNCATE(TRUNCATE(TRUNCATE(TRUNCATE(123.123456789, 9), 8), 7), 6), 5), 4), 3) truncate(truncate(truncate(truncate(truncate(truncate(truncate(1 | =================================================================== - 1.231230000000000e+02 | + 123.12 | taos> select TRUNCATE(TRUNCATE(TRUNCATE(TRUNCATE(TRUNCATE(TRUNCATE(TRUNCATE(123456789.123456789, -1), -2), -3), -4), -5), -6), -7) truncate(truncate(truncate(truncate(truncate(truncate(truncate(1 | =================================================================== - 1.200000000000000e+08 | + 12000000 | taos> select TRUNCATE(87654321.123456789, id) from ts_4893.meters order by ts limit 10 truncate(87654321.123456789, id) | =================================== - 8.765432100000000e+07 | - 8.765432109999999e+07 | - 8.765432112000000e+07 | - 8.765432112300000e+07 | - 8.765432112340000e+07 | - 8.765432112345000e+07 | - 8.765432112345600e+07 | - 8.765432112345670e+07 | - 8.765432112345679e+07 | - 8.765432112345679e+07 | + 87654321 | + 87654321.1 | + 87654321.12 | + 87654321.123 | + 87654321.1234 | + 87654321.12345 | + 87654321.123456 | + 87654321.1234567 | + 87654321.1234568 | + 87654321.1234568 | taos> select TRUNCATE(current, id) from ts_4893.meters order by ts limit 10 truncate(current, id) | ======================== - 1.0000000e+01 | - 8.5000000e+00 | - 9.7900000e+00 | - 1.1233000e+01 | - 1.0706000e+01 | - 8.5080004e+00 | - 9.5959997e+00 | - 1.0962000e+01 | - 1.1226000e+01 | - 1.0337000e+01 | + 10 | + 8.5 | + 9.79 | + 11.233 | + 10.706 | + 8.508 | + 9.596 | + 10.962 | + 11.226 | + 10.337 | taos> select TRUNCATE(current, 1) from ts_4893.meters order by ts limit 10 truncate(current, 1) | @@ -144,26 +144,26 @@ taos> select TRUNC(10.55, 1) + 1 taos> select TRUNC(TRUNC(TRUNC(TRUNC(TRUNC(TRUNC(TRUNC(123.123456789, 9), 8), 7), 6), 5), 4), 3) trunc(trunc(trunc(trunc(trunc(trunc(trunc(123.123456789, 9), 8), | =================================================================== - 1.231230000000000e+02 | + 123.12 | taos> select TRUNC(TRUNC(TRUNC(TRUNC(TRUNC(TRUNC(TRUNC(123456789.123456789, -1), -2), -3), -4), -5), -6), -7) trunc(trunc(trunc(trunc(trunc(trunc(trunc(123456789.123456789, - | =================================================================== - 1.200000000000000e+08 | + 12000000 | taos> select TRUNC(87654321.123456789, id) from ts_4893.meters order by ts limit 10 trunc(87654321.123456789, id) | ================================ - 8.765432100000000e+07 | - 8.765432109999999e+07 | - 8.765432112000000e+07 | - 8.765432112300000e+07 | - 8.765432112340000e+07 | - 8.765432112345000e+07 | - 8.765432112345600e+07 | - 8.765432112345670e+07 | - 8.765432112345679e+07 | - 8.765432112345679e+07 | + 87654321 | + 87654321.1 | + 87654321.12 | + 87654321.123 | + 87654321.1234 | + 87654321.12345 | + 87654321.123456 | + 87654321.1234567 | + 87654321.1234568 | + 87654321.1234568 | taos> select TRUNC(current, id) from ts_4893.meters order by ts limit 10 trunc(current, id) | @@ -289,7 +289,7 @@ taos> select truncate(100.9876, 2) taos> select truncate(99999999999999.9999, 2) truncate(99999999999999.9999, 2) | =================================== - 1.000000000000000e+14 | + 100000000000000 | taos> select truncate(-5.678, 2) truncate(-5.678, 2) | @@ -314,7 +314,7 @@ taos> select truncate(phase, 3) from ts_4893.meters limit 1 taos> select truncate(voltage + current, 2) from ts_4893.meters limit 1 truncate(voltage + current, 2) | ================================= - 2.316400000000000e+02 | + 231.64 | taos> select truncate(voltage, -1) from ts_4893.meters limit 1 truncate(voltage, -1) | @@ -329,7 +329,7 @@ taos> select round(truncate(voltage, 1), 2) from ts_4893.meters limit 1 taos> select truncate(abs(current), 1) from ts_4893.meters limit 1 truncate(abs(current), 1) | ============================ - 1.0600000e+01 | + 10.6 | taos> select truncate(exp(phase), 2) from ts_4893.meters limit 1 truncate(exp(phase), 2) | diff --git a/tests/army/query/function/ans/varpop.csv b/tests/army/query/function/ans/varpop.csv index 829996d978..ec5d479f3a 100644 --- a/tests/army/query/function/ans/varpop.csv +++ b/tests/army/query/function/ans/varpop.csv @@ -99,7 +99,7 @@ taos> select var_pop(total_voltage) from (select sum(voltage) as total_voltage f taos> select round(var_pop(current), 2) from ts_4893.meters round(var_pop(current), 2) | ============================= - 1.330000000000000e+00 | + 1.33 | taos> select pow(var_pop(current), 2) from ts_4893.meters pow(var_pop(current), 2) | diff --git a/tools/shell/src/shellEngine.c b/tools/shell/src/shellEngine.c index 2a405d8f38..b1e21e8f62 100644 --- a/tools/shell/src/shellEngine.c +++ b/tools/shell/src/shellEngine.c @@ -671,12 +671,8 @@ void shellPrintField(const char *val, TAOS_FIELD *field, int32_t width, int32_t if (tsEnableScience) { printf("%*.7e", width, GET_FLOAT_VAL(val)); } else { - n = snprintf(buf, LENGTH, "%*.*g", width, FLT_DIG, GET_FLOAT_VAL(val)); - if (n > width) { - printf("%*.7e", width, GET_FLOAT_VAL(val)); - } else { - printf("%s", buf); - } + snprintf(buf, LENGTH, "%*.*g", width, FLT_DIG, GET_FLOAT_VAL(val)); + printf("%s", buf); } break; case TSDB_DATA_TYPE_DOUBLE: @@ -684,12 +680,8 @@ void shellPrintField(const char *val, TAOS_FIELD *field, int32_t width, int32_t snprintf(buf, LENGTH, "%*.15e", width, GET_DOUBLE_VAL(val)); printf("%s", buf); } else { - n = snprintf(buf, LENGTH, "%*.*g", width, DBL_DIG, GET_DOUBLE_VAL(val)); - if (n > SHELL_DOUBLE_WIDTH) { - printf("%*.15e", width, GET_DOUBLE_VAL(val)); - } else { - printf("%*s", width, buf); - } + snprintf(buf, LENGTH, "%*.*g", width, DBL_DIG, GET_DOUBLE_VAL(val)); + printf("%*s", width, buf); } break; case TSDB_DATA_TYPE_VARBINARY: { From 8f5134913c6d9acc2a1dfc47c01767de0f76ce7b Mon Sep 17 00:00:00 2001 From: factosea <285808407@qq.com> Date: Fri, 7 Mar 2025 16:19:28 +0800 Subject: [PATCH 58/70] feat: test case --- source/libs/function/src/builtins.c | 41 +- source/libs/scalar/src/sclfunc.c | 5 +- tests/army/query/function/ans/greatest.csv | 1409 ++++++++++++++++++++ tests/army/query/function/ans/least.csv | 1389 +++++++++++++++++++ tests/army/query/function/ans/mod.csv | 4 +- tests/army/query/function/ans/round.csv | 4 +- tests/army/query/function/ans/trunc.csv | 8 +- tests/army/query/function/in/greatest.in | 232 ++++ tests/army/query/function/in/least.in | 228 ++++ tests/army/query/function/test_function.py | 183 ++- tools/shell/src/shellEngine.c | 2 + 11 files changed, 3452 insertions(+), 53 deletions(-) create mode 100644 tests/army/query/function/ans/greatest.csv create mode 100644 tests/army/query/function/ans/least.csv create mode 100644 tests/army/query/function/in/greatest.in create mode 100644 tests/army/query/function/in/least.in diff --git a/source/libs/function/src/builtins.c b/source/libs/function/src/builtins.c index a63596d0d2..94ccf799bb 100644 --- a/source/libs/function/src/builtins.c +++ b/source/libs/function/src/builtins.c @@ -1748,6 +1748,7 @@ static int32_t translateHistogramPartial(SFunctionNode* pFunc, char* pErrBuf, in return TSDB_CODE_SUCCESS; } +#define NUMERIC_TO_STRINGS_LEN 25 static int32_t translateGreatestleast(SFunctionNode* pFunc, char* pErrBuf, int32_t len) { if (LIST_LENGTH(pFunc->pParameterList) < 2) { return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName); @@ -1756,32 +1757,48 @@ static int32_t translateGreatestleast(SFunctionNode* pFunc, char* pErrBuf, int32 bool mixTypeToStrings = tsTransToStrWhenMixTypeInLeast; SDataType res = {.type = 0}; + bool resInit = false; for (int32_t i = 0; i < LIST_LENGTH(pFunc->pParameterList); i++) { SDataType* para = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, i)); if (IS_NULL_TYPE(para->type)) { res.type = TSDB_DATA_TYPE_NULL; + res.bytes = tDataTypes[TSDB_DATA_TYPE_NULL].bytes; break; - } else if (IS_MATHABLE_TYPE(para->type)) { - if(res.type == 0) { - res.type = para->type; - res.bytes = para->bytes; - } else if(IS_MATHABLE_TYPE(res.type) || !mixTypeToStrings) { + } + + if (!resInit) { + res.type = para->type; + res.bytes = para->bytes; + resInit = true; + continue; + } + + if (IS_MATHABLE_TYPE(para->type)) { + if (res.type == para->type) { + continue; + } else if (IS_MATHABLE_TYPE(res.type) || !mixTypeToStrings) { int32_t resType = vectorGetConvertType(res.type, para->type); res.type = resType == 0 ? res.type : resType; - res.bytes = tDataTypes[resType].bytes; + res.bytes = tDataTypes[resType].bytes; + } else { + // last res is strings, para is numeric and mixTypeToStrings is true + res.bytes = TMAX(res.bytes, NUMERIC_TO_STRINGS_LEN); } } else if (IS_COMPARE_STR_DATA_TYPE(para->type)) { - if(res.type == 0) { - res.type = para->type; - res.bytes = para->bytes; - } else if(IS_COMPARE_STR_DATA_TYPE(res.type)) { + if (IS_COMPARE_STR_DATA_TYPE(res.type)) { int32_t resType = vectorGetConvertType(res.type, para->type); res.type = resType == 0 ? res.type : resType; res.bytes = TMAX(res.bytes, para->bytes); - } else if(mixTypeToStrings) { // res.type is mathable type + } else if (mixTypeToStrings) { + // last res is numeric, para is string, and mixTypeToStrings is true res.type = para->type; - res.bytes = para->bytes; + res.bytes = TMAX(para->bytes, NUMERIC_TO_STRINGS_LEN); + } else { + // last res is numeric, para is string, and mixTypeToStrings is false + int32_t resType = vectorGetConvertType(res.type, para->type); + res.type = resType == 0 ? res.type : resType; + res.bytes = tDataTypes[resType].bytes; } } else { return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName); diff --git a/source/libs/scalar/src/sclfunc.c b/source/libs/scalar/src/sclfunc.c index 3ce310a261..280a469153 100644 --- a/source/libs/scalar/src/sclfunc.c +++ b/source/libs/scalar/src/sclfunc.c @@ -4473,9 +4473,12 @@ static int32_t greatestLeastImpl(SScalarParam *pInput, int32_t inputNum, SScalar int32_t *resultColIndex = NULL; int32_t numOfRows = 0; - bool IsNullType = false; + bool IsNullType = outputType == TSDB_DATA_TYPE_NULL ? true : false; // If any column is NULL type, the output is NULL type for (int32_t i = 0; i < inputNum; i++) { + if (IsNullType) { + break; + } if (numOfRows != 0 && numOfRows != pInput[i].numOfRows && pInput[i].numOfRows != 1 && numOfRows != 1) { qError("input rows not match, func:%s, rows:%d, %d", __FUNCTION__, numOfRows, pInput[i].numOfRows); code = TSDB_CODE_TSC_INTERNAL_ERROR; diff --git a/tests/army/query/function/ans/greatest.csv b/tests/army/query/function/ans/greatest.csv new file mode 100644 index 0000000000..ced28474f8 --- /dev/null +++ b/tests/army/query/function/ans/greatest.csv @@ -0,0 +1,1409 @@ + +taos> alter local 'transToStrWhenMixTypeInLeast' '1'; + +taos> select GREATEST(1,2,3,4,5,6,7,8,9,10); + greatest(1,2,3,4,5,6,7,8,9,10) | +================================= + 10 | + +taos> select GREATEST(1,1.1,2.23,3.4444,5.66666666,6.21241241,7.999999999999); + greatest(1,1.1,2.23,3.4444,5.66666666,6.21241241,7.999999999999) | +=================================================================== + 7.999999999999 | + +taos> select GREATEST(1,'2',3.3,4.4,5); + greatest(1,'2',3.3,4.4,5) | +============================ + 5 | + +taos> select GREATEST(121,'18'); + greatest(121,'18') | +============================ + 18 | + +taos> select GREATEST(18888,'18'); + greatest(18888,'18') | +============================ + 18888 | + +taos> select GREATEST(1,2,3,4,5,'5.1'); + greatest(1,2,3,4,5,'5.1') | +============================ + 5.1 | + +taos> select GREATEST('1','2','3','4',5); + greatest('1','2','3','4',5) | +============================== + 5 | + +taos> select GREATEST('1','2','3','4','5'); + greatest('1','2','3','4','5') | +================================ + 5 | + +taos> select GREATEST(1,2,3,4,5,6,7,'a','b','一','二','三'); + greatest(1,2,3,4,5,6,7,'a','b','一','二','三') | +==================================================== + 二 | + +taos> select GREATEST(1,2,3,4,5,6,7,'a','b','c','1','2','1231213'); + greatest(1,2,3,4,5,6,7,'a','b','c','1','2','1231213') | +======================================================== + c | + +taos> select GREATEST(1,2,3,4,5,6,7,'a','b','c','1','2','1231213','1231213.123123'); + greatest(1,2,3,4,5,6,7,'a','b','c','1','2','1231213','1231213.12 | +=================================================================== + c | + +taos> select GREATEST(cast(0 as bool), cast(1 as tinyint), cast(2 as smallint), cast(3 as int), cast(4 as bigint), cast(5 as float), cast(6 as double), cast(8 as varchar(20)), cast(9 as nchar(20))); + greatest(cast(0 as bool), cast(1 as tinyint), cast(2 as smallint | +=================================================================== + 9 | + +taos> select GREATEST(cast(0 as bool), cast(1 as tinyint unsigned), cast(2 as smallint unsigned), cast(3 as int unsigned), cast(4 as bigint unsigned), cast(5 as float), cast(6 as double), cast(8 as varchar(20)), cast(9 as nchar(20))); + greatest(cast(0 as bool), cast(1 as tinyint unsigned), cast(2 as | +=================================================================== + 9 | + +taos> select GREATEST(cast(0 as bool), cast(1 as tinyint), cast(2 as smallint), cast(3 as int), cast(4 as bigint), cast(5 as float), cast(6 as double), cast(8 as varchar(20)), cast(9 as nchar(20)), cast(0 as bool), cast(1 as tinyint unsigned), cast(2 as smallint unsigned), cast(3 as int unsigned), cast(4 as bigint unsigned), cast(5 as float), cast(6 as double)); + greatest(cast(0 as bool), cast(1 as tinyint), cast(2 as smallint | +=================================================================== + 9 | + +taos> select GREATEST(cast(100 as tinyint), cast(101 as tinyint)); + greatest(cast(100 as tinyint), cast(101 as tinyint)) | +======================================================= + 101 | + +taos> select GREATEST(cast(100 as tinyint), cast(101 as smallint)); + greatest(cast(100 as tinyint), cast(101 as smallint)) | +======================================================== + 101 | + +taos> select GREATEST(cast(100 as tinyint), cast(101 as int)); + greatest(cast(100 as tinyint), cast(101 as int)) | +=================================================== + 101 | + +taos> select GREATEST(cast(100 as tinyint), cast(101 as bigint)); + greatest(cast(100 as tinyint), cast(101 as bigint)) | +====================================================== + 101 | + +taos> select GREATEST(cast(100 as tinyint), cast(101 as float)); + greatest(cast(100 as tinyint), cast(101 as float)) | +===================================================== + 101 | + +taos> select GREATEST(cast(100 as tinyint), cast(101 as double)); + greatest(cast(100 as tinyint), cast(101 as double)) | +====================================================== + 101 | + +taos> select GREATEST(cast(100 as tinyint), cast(101 as varchar(20))); + greatest(cast(100 as tinyint), cast(101 as varchar(20))) | +=========================================================== + 101 | + +taos> select GREATEST(cast(100 as tinyint), cast(101 as nchar(20))); + greatest(cast(100 as tinyint), cast(101 as nchar(20))) | +========================================================= + 101 | + +taos> select GREATEST(cast(101 as tinyint), cast(100 as tinyint)); + greatest(cast(101 as tinyint), cast(100 as tinyint)) | +======================================================= + 101 | + +taos> select GREATEST(cast(101 as tinyint), cast(100 as smallint)); + greatest(cast(101 as tinyint), cast(100 as smallint)) | +======================================================== + 101 | + +taos> select GREATEST(cast(101 as tinyint), cast(100 as int)); + greatest(cast(101 as tinyint), cast(100 as int)) | +=================================================== + 101 | + +taos> select GREATEST(cast(101 as tinyint), cast(100 as bigint)); + greatest(cast(101 as tinyint), cast(100 as bigint)) | +====================================================== + 101 | + +taos> select GREATEST(cast(101 as tinyint), cast(100 as float)); + greatest(cast(101 as tinyint), cast(100 as float)) | +===================================================== + 101 | + +taos> select GREATEST(cast(101 as tinyint), cast(100 as double)); + greatest(cast(101 as tinyint), cast(100 as double)) | +====================================================== + 101 | + +taos> select GREATEST(cast(101 as tinyint), cast(100 as varchar(20))); + greatest(cast(101 as tinyint), cast(100 as varchar(20))) | +=========================================================== + 101 | + +taos> select GREATEST(cast(101 as tinyint), cast(100 as nchar(20))); + greatest(cast(101 as tinyint), cast(100 as nchar(20))) | +========================================================= + 101 | + +taos> select GREATEST(cast(1000 as smallint), cast(1001 as smallint)); + greatest(cast(1000 as smallint), cast(1001 as smallint)) | +=========================================================== + 1001 | + +taos> select GREATEST(cast(1000 as smallint), cast(1001 as int)); + greatest(cast(1000 as smallint), cast(1001 as int)) | +====================================================== + 1001 | + +taos> select GREATEST(cast(1000 as smallint), cast(1001 as bigint)); + greatest(cast(1000 as smallint), cast(1001 as bigint)) | +========================================================= + 1001 | + +taos> select GREATEST(cast(1000 as smallint), cast(1001 as float)); + greatest(cast(1000 as smallint), cast(1001 as float)) | +======================================================== + 1001 | + +taos> select GREATEST(cast(1000 as smallint), cast(1001 as double)); + greatest(cast(1000 as smallint), cast(1001 as double)) | +========================================================= + 1001 | + +taos> select GREATEST(cast(1000 as smallint), cast(1001 as varchar(20))); + greatest(cast(1000 as smallint), cast(1001 as varchar(20))) | +============================================================== + 1001 | + +taos> select GREATEST(cast(1000 as smallint), cast(1001 as nchar(20))); + greatest(cast(1000 as smallint), cast(1001 as nchar(20))) | +============================================================ + 1001 | + +taos> select GREATEST(cast(1001 as smallint), cast(1000 as smallint)); + greatest(cast(1001 as smallint), cast(1000 as smallint)) | +=========================================================== + 1001 | + +taos> select GREATEST(cast(1001 as smallint), cast(1000 as int)); + greatest(cast(1001 as smallint), cast(1000 as int)) | +====================================================== + 1001 | + +taos> select GREATEST(cast(1001 as smallint), cast(1000 as bigint)); + greatest(cast(1001 as smallint), cast(1000 as bigint)) | +========================================================= + 1001 | + +taos> select GREATEST(cast(1001 as smallint), cast(1000 as float)); + greatest(cast(1001 as smallint), cast(1000 as float)) | +======================================================== + 1001 | + +taos> select GREATEST(cast(1001 as smallint), cast(1000 as double)); + greatest(cast(1001 as smallint), cast(1000 as double)) | +========================================================= + 1001 | + +taos> select GREATEST(cast(1001 as smallint), cast(1000 as varchar(20))); + greatest(cast(1001 as smallint), cast(1000 as varchar(20))) | +============================================================== + 1001 | + +taos> select GREATEST(cast(1001 as smallint), cast(1000 as nchar(20))); + greatest(cast(1001 as smallint), cast(1000 as nchar(20))) | +============================================================ + 1001 | + +taos> select GREATEST(cast(1000000 as int), cast(1000001 as int)); + greatest(cast(1000000 as int), cast(1000001 as int)) | +======================================================= + 1000001 | + +taos> select GREATEST(cast(1000000 as int), cast(1000001 as bigint)); + greatest(cast(1000000 as int), cast(1000001 as bigint)) | +========================================================== + 1000001 | + +taos> select GREATEST(cast(1000000 as int), cast(1000001 as float)); + greatest(cast(1000000 as int), cast(1000001 as float)) | +========================================================= + 1e+06 | + +taos> select GREATEST(cast(1000000 as int), cast(1000001 as double)); + greatest(cast(1000000 as int), cast(1000001 as double)) | +========================================================== + 1000001 | + +taos> select GREATEST(cast(1000000 as int), cast(1000001 as varchar(20))); + greatest(cast(1000000 as int), cast(1000001 as varchar(20))) | +=============================================================== + 1000001 | + +taos> select GREATEST(cast(1000000 as int), cast(1000001 as nchar(20))); + greatest(cast(1000000 as int), cast(1000001 as nchar(20))) | +============================================================= + 1000001 | + +taos> select GREATEST(cast(1000001 as int), cast(1000000 as int)); + greatest(cast(1000001 as int), cast(1000000 as int)) | +======================================================= + 1000001 | + +taos> select GREATEST(cast(1000001 as int), cast(1000000 as bigint)); + greatest(cast(1000001 as int), cast(1000000 as bigint)) | +========================================================== + 1000001 | + +taos> select GREATEST(cast(1000001 as int), cast(1000000 as float)); + greatest(cast(1000001 as int), cast(1000000 as float)) | +========================================================= + 1e+06 | + +taos> select GREATEST(cast(1000001 as int), cast(1000000 as double)); + greatest(cast(1000001 as int), cast(1000000 as double)) | +========================================================== + 1000001 | + +taos> select GREATEST(cast(1000001 as int), cast(1000000 as varchar(20))); + greatest(cast(1000001 as int), cast(1000000 as varchar(20))) | +=============================================================== + 1000001 | + +taos> select GREATEST(cast(1000001 as int), cast(1000000 as nchar(20))); + greatest(cast(1000001 as int), cast(1000000 as nchar(20))) | +============================================================= + 1000001 | + +taos> select GREATEST(cast(1000000000 as bigint), cast(1000000001 as bigint)); + greatest(cast(1000000000 as bigint), cast(1000000001 as bigint)) | +=================================================================== + 1000000001 | + +taos> select GREATEST(cast(1000000000 as bigint), cast(1000000001 as float)); + greatest(cast(1000000000 as bigint), cast(1000000001 as float)) | +================================================================== + 1e+09 | + +taos> select GREATEST(cast(1000000000 as bigint), cast(1000000001 as double)); + greatest(cast(1000000000 as bigint), cast(1000000001 as double)) | +=================================================================== + 1000000001 | + +taos> select GREATEST(cast(1000000000 as bigint), cast(1000000001 as varchar(20))); + greatest(cast(1000000000 as bigint), cast(1000000001 as varchar( | +=================================================================== + 1000000001 | + +taos> select GREATEST(cast(1000000000 as bigint), cast(1000000001 as nchar(20))); + greatest(cast(1000000000 as bigint), cast(1000000001 as nchar(20 | +=================================================================== + 1000000001 | + +taos> select GREATEST(cast(1000000001 as bigint), cast(1000000000 as bigint)); + greatest(cast(1000000001 as bigint), cast(1000000000 as bigint)) | +=================================================================== + 1000000001 | + +taos> select GREATEST(cast(1000000001 as bigint), cast(1000000000 as float)); + greatest(cast(1000000001 as bigint), cast(1000000000 as float)) | +================================================================== + 1e+09 | + +taos> select GREATEST(cast(1000000001 as bigint), cast(1000000000 as double)); + greatest(cast(1000000001 as bigint), cast(1000000000 as double)) | +=================================================================== + 1000000001 | + +taos> select GREATEST(cast(1000000001 as bigint), cast(1000000000 as varchar(20))); + greatest(cast(1000000001 as bigint), cast(1000000000 as varchar( | +=================================================================== + 1000000001 | + +taos> select GREATEST(cast(1000000001 as bigint), cast(1000000000 as nchar(20))); + greatest(cast(1000000001 as bigint), cast(1000000000 as nchar(20 | +=================================================================== + 1000000001 | + +taos> select GREATEST(cast(100000.1111111 as float), cast(100001.1111111 as float)); + greatest(cast(100000.1111111 as float), cast(100001.1111111 as f | +=================================================================== + 100001 | + +taos> select GREATEST(cast(100000.1111111 as float), cast(100001.1111111 as double)); + greatest(cast(100000.1111111 as float), cast(100001.1111111 as d | +=================================================================== + 100001.1111111 | + +taos> select GREATEST(cast(100000.1111111 as float), cast(100001.1111111 as timestamp)); + greatest(cast(100000.1111111 as float), cast(100001.1111111 as t | +=================================================================== + 100001 | + +taos> select GREATEST(cast(100000.1111111 as float), cast(100001.1111111 as varchar(20))); + greatest(cast(100000.1111111 as float), cast(100001.1111111 as v | +=================================================================== + 100001.1111111 | + +taos> select GREATEST(cast(100000.1111111 as float), cast(100001.1111111 as nchar(20))); + greatest(cast(100000.1111111 as float), cast(100001.1111111 as n | +=================================================================== + 100001.1111111 | + +taos> select GREATEST(cast(100001.1111111 as float), cast(100000.1111111 as float)); + greatest(cast(100001.1111111 as float), cast(100000.1111111 as f | +=================================================================== + 100001 | + +taos> select GREATEST(cast(100001.1111111 as float), cast(100000.1111111 as double)); + greatest(cast(100001.1111111 as float), cast(100000.1111111 as d | +=================================================================== + 100001.109375 | + +taos> select GREATEST(cast(100001.1111111 as float), cast(100000.1111111 as timestamp)); + greatest(cast(100001.1111111 as float), cast(100000.1111111 as t | +=================================================================== + 100001 | + +taos> select GREATEST(cast(100001.1111111 as float), cast(100000.1111111 as varchar(20))); + greatest(cast(100001.1111111 as float), cast(100000.1111111 as v | +=================================================================== + 100001.109375 | + +taos> select GREATEST(cast(100001.1111111 as float), cast(100000.1111111 as nchar(20))); + greatest(cast(100001.1111111 as float), cast(100000.1111111 as n | +=================================================================== + 100001.109375 | + +taos> select GREATEST(cast(100000.1111111 as double), cast(100001.1111111 as double)); + greatest(cast(100000.1111111 as double), cast(100001.1111111 as | +=================================================================== + 100001.1111111 | + +taos> select GREATEST(cast(100000.1111111 as double), cast(100001.1111111 as timestamp)); + greatest(cast(100000.1111111 as double), cast(100001.1111111 as | +=================================================================== + 100001 | + +taos> select GREATEST(cast(100000.1111111 as double), cast(100001.1111111 as varchar(20))); + greatest(cast(100000.1111111 as double), cast(100001.1111111 as | +=================================================================== + 100001.1111111 | + +taos> select GREATEST(cast(100000.1111111 as double), cast(100001.1111111 as nchar(20))); + greatest(cast(100000.1111111 as double), cast(100001.1111111 as | +=================================================================== + 100001.1111111 | + +taos> select GREATEST(cast(100001.1111111 as double), cast(100000.1111111 as double)); + greatest(cast(100001.1111111 as double), cast(100000.1111111 as | +=================================================================== + 100001.1111111 | + +taos> select GREATEST(cast(100001.1111111 as double), cast(100000.1111111 as timestamp)); + greatest(cast(100001.1111111 as double), cast(100000.1111111 as | +=================================================================== + 100001.1111111 | + +taos> select GREATEST(cast(100001.1111111 as double), cast(100000.1111111 as varchar(20))); + greatest(cast(100001.1111111 as double), cast(100000.1111111 as | +=================================================================== + 100001.111111 | + +taos> select GREATEST(cast(100001.1111111 as double), cast(100000.1111111 as nchar(20))); + greatest(cast(100001.1111111 as double), cast(100000.1111111 as | +=================================================================== + 100001.111111 | + +taos> select GREATEST(cast('中文测试' as varchar(20)), cast('中文测试一' as varchar(20))); + greatest(cast('中文测试' as varchar(20)), cast('中文测试 | +=================================================================== + 中文测试一 | + +taos> select GREATEST(cast('中文测试' as varchar(20)), cast('中文测试一' as nchar(20))); + greatest(cast('中文测试' as varchar(20)), cast('中文测试 | +=================================================================== + 中文测试一 | + +taos> select GREATEST(cast('中文测试一' as varchar(20)), cast('中文测试' as varchar(20))); + greatest(cast('中文测试一' as varchar(20)), cast('中文测 | +=================================================================== + 中文测试一 | + +taos> select GREATEST(cast('中文测试一' as varchar(20)), cast('中文测试' as nchar(20))); + greatest(cast('中文测试一' as varchar(20)), cast('中文测 | +=================================================================== + 中文测试一 | + +taos> select GREATEST(cast('abc123abc' as varchar(20)), cast('abc124abc' as varchar(20))); + greatest(cast('abc123abc' as varchar(20)), cast('abc124abc' as v | +=================================================================== + abc124abc | + +taos> select GREATEST(cast('abc123abc' as varchar(20)), cast('abc124abc' as nchar(20))); + greatest(cast('abc123abc' as varchar(20)), cast('abc124abc' as n | +=================================================================== + abc124abc | + +taos> select GREATEST(cast('abc124abc' as varchar(20)), cast('abc123abc' as varchar(20))); + greatest(cast('abc124abc' as varchar(20)), cast('abc123abc' as v | +=================================================================== + abc124abc | + +taos> select GREATEST(cast('abc124abc' as varchar(20)), cast('abc123abc' as nchar(20))); + greatest(cast('abc124abc' as varchar(20)), cast('abc123abc' as n | +=================================================================== + abc124abc | + +taos> select GREATEST(cast('abc123abc' as nchar(20)), cast('abc124abc' as nchar(20))); + greatest(cast('abc123abc' as nchar(20)), cast('abc124abc' as nch | +=================================================================== + abc124abc | + +taos> select GREATEST(cast(100 as tinyint), cast(101 as float), cast(102 as varchar(20))); + greatest(cast(100 as tinyint), cast(101 as float), cast(102 as v | +=================================================================== + 102 | + +taos> select GREATEST(cast(100 as float), cast(101 as tinyint), cast(102 as varchar(20))); + greatest(cast(100 as float), cast(101 as tinyint), cast(102 as v | +=================================================================== + 102 | + +taos> select GREATEST(cast(100 as float), cast(101 as varchar(20)), cast(102 as tinyint)); + greatest(cast(100 as float), cast(101 as varchar(20)), cast(102 | +=================================================================== + 102 | + +taos> select GREATEST(cast(100 as varchar(20)), cast(101 as float), cast(102 as tinyint)); + greatest(cast(100 as varchar(20)), cast(101 as float), cast(102 | +=================================================================== + 102 | + +taos> select GREATEST('a','b','c','d','e','f','g','h','1231','15155'); + greatest('a','b','c','d','e','f','g','h','1231','15155') | +=========================================================== + h | + +taos> select GREATEST(current, voltage, phase, id, nch1, nch2, var1, var2) from ts_4893.meters order by ts limit 10; + greatest(current, voltage, phase, id, nch1, nch2, var1, var2) | +================================================================ + 四 | + 三a | + 四 | + 一 | + 一二三四五六七八九十 | + 一二三 | + prision | + 一二三四五六七八九十 | + prision | + 一 | + +taos> select GREATEST(current, voltage, phase, id) from ts_4893.meters order by ts limit 10; + greatest(current, voltage, phase, id) | +======================================== + 221 | + 220 | + 215 | + 216 | + 219 | + 221 | + 215 | + 217 | + 216 | + 223 | + +taos> select GREATEST(nch1, nch2, var1, var2) from ts_4893.meters order by ts limit 10; + greatest(nch1, nch2, var1, var2) | +=================================== + 四 | + 三a | + 四 | + 一 | + 一二三四五六七八九十 | + 一二三 | + prision | + 一二三四五六七八九十 | + prision | + 一 | + +taos> select GREATEST(221, voltage) from ts_4893.meters order by ts limit 10; + greatest(221, voltage) | +========================= + 221 | + 221 | + 221 | + 221 | + 221 | + 221 | + 221 | + 221 | + 221 | + 223 | + +taos> select GREATEST(5, id) from ts_4893.meters order by ts limit 10; + greatest(5, id) | +======================== + 5 | + 5 | + 5 | + 5 | + 5 | + 5 | + 6 | + 7 | + 8 | + 9 | + +taos> select GREATEST('r', nch1) from ts_4893.meters order by ts limit 10; + greatest('r', nch1) | +================================= + r | + 一二三四五六七八九十 | + update | + r | + r | + r | + r | + 一二三四五六七八九十 | + r | + r | + +taos> select GREATEST('r', nch1, nch2) from ts_4893.meters order by ts limit 10; + greatest('r', nch1, nch2) | +================================= + 四 | + 一二三四五六七八九十 | + update | + 一 | + r | + 一二三 | + r | + 一二三四五六七八九十 | + r | + r | + +taos> select GREATEST('r', var1) from ts_4893.meters order by ts limit 10; + greatest('r', var1) | +================================= + r | + r | + r | + r | + 一二三四五六七八九十 | + update | + r | + r | + r | + r | + +taos> select GREATEST('r', var1, var2) from ts_4893.meters order by ts limit 10; + greatest('r', var1, var2) | +================================= + r | + 三a | + 四 | + r | + 一二三四五六七八九十 | + update | + r | + r | + r | + 一 | + +taos> select GREATEST('二中文测试', nch1) from ts_4893.meters order by ts limit 10; + greatest('二中文测试', nch1) | +==================================== + 二中文测试 | + 二中文测试 | + 二中文测试 | + 二中文测试 | + 二中文测试 | + 二中文测试 | + 二中文测试 | + 二中文测试 | + 二中文测试 | + 二中文测试 | + +taos> select GREATEST('二中文测试', nch1, nch2) from ts_4893.meters order by ts limit 10; + greatest('二中文测试', nch1, nch2) | +========================================== + 四 | + 二中文测试 | + 二中文测试 | + 二中文测试 | + 二中文测试 | + 二中文测试 | + 二中文测试 | + 二中文测试 | + 二中文测试 | + 二中文测试 | + +taos> select GREATEST('二中文测试', var1) from ts_4893.meters order by ts limit 10; + greatest('二中文测试', var1) | +==================================== + 二中文测试 | + 二中文测试 | + 二中文测试 | + 二中文测试 | + 二中文测试 | + 二中文测试 | + 二中文测试 | + 二中文测试 | + 二中文测试 | + 二中文测试 | + +taos> select GREATEST('二中文测试', var1, var2) from ts_4893.meters order by ts limit 10; + greatest('二中文测试', var1, var2) | +========================================== + 二中文测试 | + 二中文测试 | + 四 | + 二中文测试 | + 二中文测试 | + 二中文测试 | + 二中文测试 | + 二中文测试 | + 二中文测试 | + 二中文测试 | + +taos> select GREATEST('23', 3443434343434343); + greatest('23', 3443434343434343) | +=================================== + 3443434343434343 | + +taos> select GREATEST(co, 3443434343434343) from ts_4893.n1; + greatest(co, 3443434343434343) | +================================= + 3443434343434343 | + 3443434343434343 | + 3443434343434343 | + +taos> select GREATEST('23', 3443434343434343) from ts_4893.n1; + greatest('23', 3443434343434343) | +=================================== + 3443434343434343 | + 3443434343434343 | + 3443434343434343 | + +taos> select GREATEST('23', 1443434343434343) from ts_4893.n1; + greatest('23', 1443434343434343) | +=================================== + 23 | + 23 | + 23 | + +taos> select GREATEST(current, voltage) from ts_4893.n1; + greatest(current, voltage) | +============================= + NULL | + NULL | + 5 | + +taos> select GREATEST(current, voltage, '15') from ts_4893.n1; + greatest(current, voltage, '15') | +=================================== + NULL | + NULL | + 5.000000 | + +taos> alter local 'transToStrWhenMixTypeInLeast' '0'; + +taos> select GREATEST(1,'2',3.3,4.4,5); + greatest(1,'2',3.3,4.4,5) | +============================ + 5 | + +taos> select GREATEST(1,2,3,4,5,'5.1'); + greatest(1,2,3,4,5,'5.1') | +============================ + 5 | + +taos> select GREATEST(121,'18'); + greatest(121,'18') | +======================== + 121 | + +taos> select GREATEST('1','2','3','4','5'); + greatest('1','2','3','4','5') | +================================ + 5 | + +taos> select GREATEST(1,2,3,4,5,6,7,'a','b','一','二','三'); + greatest(1,2,3,4,5,6,7,'a','b','一','二','三') | +==================================================== + 7 | + +taos> select GREATEST(1,2,3,4,5,6,7,'a','b','c','1','2','1231213'); + greatest(1,2,3,4,5,6,7,'a','b','c','1','2','1231213') | +======================================================== + 1231213 | + +taos> select GREATEST(1,2,3,4,5,6,7,'a','b','c','1','2','1231213','1231213.123123'); + greatest(1,2,3,4,5,6,7,'a','b','c','1','2','1231213','1231213.12 | +=================================================================== + 1231213 | + +taos> select GREATEST(cast(0 as bool), cast(1 as tinyint), cast(2 as smallint), cast(3 as int), cast(4 as bigint), cast(5 as float), cast(6 as double), cast(8 as varchar(20)), cast(9 as nchar(20))); + greatest(cast(0 as bool), cast(1 as tinyint), cast(2 as smallint | +=================================================================== + 9 | + +taos> select GREATEST(cast(0 as bool), cast(1 as tinyint unsigned), cast(2 as smallint unsigned), cast(3 as int unsigned), cast(4 as bigint unsigned), cast(5 as float), cast(6 as double), cast(8 as varchar(20)), cast(9 as nchar(20))); + greatest(cast(0 as bool), cast(1 as tinyint unsigned), cast(2 as | +=================================================================== + 9 | + +taos> select GREATEST(cast(0 as bool), cast(1 as tinyint), cast(2 as smallint), cast(3 as int), cast(4 as bigint), cast(5 as float), cast(6 as double), cast(8 as varchar(20)), cast(9 as nchar(20)), cast(0 as bool), cast(1 as tinyint unsigned), cast(2 as smallint unsigned), cast(3 as int unsigned), cast(4 as bigint unsigned), cast(5 as float), cast(6 as double)); + greatest(cast(0 as bool), cast(1 as tinyint), cast(2 as smallint | +=================================================================== + 9 | + +taos> select GREATEST(cast(100 as tinyint), cast(101 as tinyint)); + greatest(cast(100 as tinyint), cast(101 as tinyint)) | +======================================================= + 101 | + +taos> select GREATEST(cast(100 as tinyint), cast(101 as smallint)); + greatest(cast(100 as tinyint), cast(101 as smallint)) | +======================================================== + 101 | + +taos> select GREATEST(cast(100 as tinyint), cast(101 as int)); + greatest(cast(100 as tinyint), cast(101 as int)) | +=================================================== + 101 | + +taos> select GREATEST(cast(100 as tinyint), cast(101 as bigint)); + greatest(cast(100 as tinyint), cast(101 as bigint)) | +====================================================== + 101 | + +taos> select GREATEST(cast(100 as tinyint), cast(101 as float)); + greatest(cast(100 as tinyint), cast(101 as float)) | +===================================================== + 101 | + +taos> select GREATEST(cast(100 as tinyint), cast(101 as double)); + greatest(cast(100 as tinyint), cast(101 as double)) | +====================================================== + 101 | + +taos> select GREATEST(cast(100 as tinyint), cast(101 as varchar(20))); + greatest(cast(100 as tinyint), cast(101 as varchar(20))) | +=========================================================== + 101 | + +taos> select GREATEST(cast(100 as tinyint), cast(101 as nchar(20))); + greatest(cast(100 as tinyint), cast(101 as nchar(20))) | +========================================================= + 101 | + +taos> select GREATEST(cast(101 as tinyint), cast(100 as tinyint)); + greatest(cast(101 as tinyint), cast(100 as tinyint)) | +======================================================= + 101 | + +taos> select GREATEST(cast(101 as tinyint), cast(100 as smallint)); + greatest(cast(101 as tinyint), cast(100 as smallint)) | +======================================================== + 101 | + +taos> select GREATEST(cast(101 as tinyint), cast(100 as int)); + greatest(cast(101 as tinyint), cast(100 as int)) | +=================================================== + 101 | + +taos> select GREATEST(cast(101 as tinyint), cast(100 as bigint)); + greatest(cast(101 as tinyint), cast(100 as bigint)) | +====================================================== + 101 | + +taos> select GREATEST(cast(101 as tinyint), cast(100 as float)); + greatest(cast(101 as tinyint), cast(100 as float)) | +===================================================== + 101 | + +taos> select GREATEST(cast(101 as tinyint), cast(100 as double)); + greatest(cast(101 as tinyint), cast(100 as double)) | +====================================================== + 101 | + +taos> select GREATEST(cast(101 as tinyint), cast(100 as varchar(20))); + greatest(cast(101 as tinyint), cast(100 as varchar(20))) | +=========================================================== + 101 | + +taos> select GREATEST(cast(101 as tinyint), cast(100 as nchar(20))); + greatest(cast(101 as tinyint), cast(100 as nchar(20))) | +========================================================= + 101 | + +taos> select GREATEST(cast(1000 as smallint), cast(1001 as smallint)); + greatest(cast(1000 as smallint), cast(1001 as smallint)) | +=========================================================== + 1001 | + +taos> select GREATEST(cast(1000 as smallint), cast(1001 as int)); + greatest(cast(1000 as smallint), cast(1001 as int)) | +====================================================== + 1001 | + +taos> select GREATEST(cast(1000 as smallint), cast(1001 as bigint)); + greatest(cast(1000 as smallint), cast(1001 as bigint)) | +========================================================= + 1001 | + +taos> select GREATEST(cast(1000 as smallint), cast(1001 as float)); + greatest(cast(1000 as smallint), cast(1001 as float)) | +======================================================== + 1001 | + +taos> select GREATEST(cast(1000 as smallint), cast(1001 as double)); + greatest(cast(1000 as smallint), cast(1001 as double)) | +========================================================= + 1001 | + +taos> select GREATEST(cast(1000 as smallint), cast(1001 as varchar(20))); + greatest(cast(1000 as smallint), cast(1001 as varchar(20))) | +============================================================== + 1001 | + +taos> select GREATEST(cast(1000 as smallint), cast(1001 as nchar(20))); + greatest(cast(1000 as smallint), cast(1001 as nchar(20))) | +============================================================ + 1001 | + +taos> select GREATEST(cast(1001 as smallint), cast(1000 as smallint)); + greatest(cast(1001 as smallint), cast(1000 as smallint)) | +=========================================================== + 1001 | + +taos> select GREATEST(cast(1001 as smallint), cast(1000 as int)); + greatest(cast(1001 as smallint), cast(1000 as int)) | +====================================================== + 1001 | + +taos> select GREATEST(cast(1001 as smallint), cast(1000 as bigint)); + greatest(cast(1001 as smallint), cast(1000 as bigint)) | +========================================================= + 1001 | + +taos> select GREATEST(cast(1001 as smallint), cast(1000 as float)); + greatest(cast(1001 as smallint), cast(1000 as float)) | +======================================================== + 1001 | + +taos> select GREATEST(cast(1001 as smallint), cast(1000 as double)); + greatest(cast(1001 as smallint), cast(1000 as double)) | +========================================================= + 1001 | + +taos> select GREATEST(cast(1001 as smallint), cast(1000 as varchar(20))); + greatest(cast(1001 as smallint), cast(1000 as varchar(20))) | +============================================================== + 1001 | + +taos> select GREATEST(cast(1001 as smallint), cast(1000 as nchar(20))); + greatest(cast(1001 as smallint), cast(1000 as nchar(20))) | +============================================================ + 1001 | + +taos> select GREATEST(cast(1000000 as int), cast(1000001 as int)); + greatest(cast(1000000 as int), cast(1000001 as int)) | +======================================================= + 1000001 | + +taos> select GREATEST(cast(1000000 as int), cast(1000001 as bigint)); + greatest(cast(1000000 as int), cast(1000001 as bigint)) | +========================================================== + 1000001 | + +taos> select GREATEST(cast(1000000 as int), cast(1000001 as float)); + greatest(cast(1000000 as int), cast(1000001 as float)) | +========================================================= + 1e+06 | + +taos> select GREATEST(cast(1000000 as int), cast(1000001 as double)); + greatest(cast(1000000 as int), cast(1000001 as double)) | +========================================================== + 1000001 | + +taos> select GREATEST(cast(1000000 as int), cast(1000001 as varchar(20))); + greatest(cast(1000000 as int), cast(1000001 as varchar(20))) | +=============================================================== + 1000001 | + +taos> select GREATEST(cast(1000000 as int), cast(1000001 as nchar(20))); + greatest(cast(1000000 as int), cast(1000001 as nchar(20))) | +============================================================= + 1000001 | + +taos> select GREATEST(cast(1000001 as int), cast(1000000 as int)); + greatest(cast(1000001 as int), cast(1000000 as int)) | +======================================================= + 1000001 | + +taos> select GREATEST(cast(1000001 as int), cast(1000000 as bigint)); + greatest(cast(1000001 as int), cast(1000000 as bigint)) | +========================================================== + 1000001 | + +taos> select GREATEST(cast(1000001 as int), cast(1000000 as float)); + greatest(cast(1000001 as int), cast(1000000 as float)) | +========================================================= + 1e+06 | + +taos> select GREATEST(cast(1000001 as int), cast(1000000 as double)); + greatest(cast(1000001 as int), cast(1000000 as double)) | +========================================================== + 1000001 | + +taos> select GREATEST(cast(1000001 as int), cast(1000000 as varchar(20))); + greatest(cast(1000001 as int), cast(1000000 as varchar(20))) | +=============================================================== + 1000001 | + +taos> select GREATEST(cast(1000001 as int), cast(1000000 as nchar(20))); + greatest(cast(1000001 as int), cast(1000000 as nchar(20))) | +============================================================= + 1000001 | + +taos> select GREATEST(cast(1000000000 as bigint), cast(1000000001 as bigint)); + greatest(cast(1000000000 as bigint), cast(1000000001 as bigint)) | +=================================================================== + 1000000001 | + +taos> select GREATEST(cast(1000000000 as bigint), cast(1000000001 as float)); + greatest(cast(1000000000 as bigint), cast(1000000001 as float)) | +================================================================== + 1e+09 | + +taos> select GREATEST(cast(1000000000 as bigint), cast(1000000001 as double)); + greatest(cast(1000000000 as bigint), cast(1000000001 as double)) | +=================================================================== + 1000000001 | + +taos> select GREATEST(cast(1000000000 as bigint), cast(1000000001 as varchar(20))); + greatest(cast(1000000000 as bigint), cast(1000000001 as varchar( | +=================================================================== + 1000000001 | + +taos> select GREATEST(cast(1000000000 as bigint), cast(1000000001 as nchar(20))); + greatest(cast(1000000000 as bigint), cast(1000000001 as nchar(20 | +=================================================================== + 1000000001 | + +taos> select GREATEST(cast(1000000001 as bigint), cast(1000000000 as bigint)); + greatest(cast(1000000001 as bigint), cast(1000000000 as bigint)) | +=================================================================== + 1000000001 | + +taos> select GREATEST(cast(1000000001 as bigint), cast(1000000000 as float)); + greatest(cast(1000000001 as bigint), cast(1000000000 as float)) | +================================================================== + 1e+09 | + +taos> select GREATEST(cast(1000000001 as bigint), cast(1000000000 as double)); + greatest(cast(1000000001 as bigint), cast(1000000000 as double)) | +=================================================================== + 1000000001 | + +taos> select GREATEST(cast(1000000001 as bigint), cast(1000000000 as varchar(20))); + greatest(cast(1000000001 as bigint), cast(1000000000 as varchar( | +=================================================================== + 1000000001 | + +taos> select GREATEST(cast(1000000001 as bigint), cast(1000000000 as nchar(20))); + greatest(cast(1000000001 as bigint), cast(1000000000 as nchar(20 | +=================================================================== + 1000000001 | + +taos> select GREATEST(cast(100000.1111111 as float), cast(100001.1111111 as float)); + greatest(cast(100000.1111111 as float), cast(100001.1111111 as f | +=================================================================== + 100001 | + +taos> select GREATEST(cast(100000.1111111 as float), cast(100001.1111111 as double)); + greatest(cast(100000.1111111 as float), cast(100001.1111111 as d | +=================================================================== + 100001.1111111 | + +taos> select GREATEST(cast(100000.1111111 as float), cast(100001.1111111 as timestamp)); + greatest(cast(100000.1111111 as float), cast(100001.1111111 as t | +=================================================================== + 100001 | + +taos> select GREATEST(cast(100000.1111111 as float), cast(100001.1111111 as varchar(20))); + greatest(cast(100000.1111111 as float), cast(100001.1111111 as v | +=================================================================== + 100001 | + +taos> select GREATEST(cast(100000.1111111 as float), cast(100001.1111111 as nchar(20))); + greatest(cast(100000.1111111 as float), cast(100001.1111111 as n | +=================================================================== + 100001 | + +taos> select GREATEST(cast(100001.1111111 as float), cast(100000.1111111 as float)); + greatest(cast(100001.1111111 as float), cast(100000.1111111 as f | +=================================================================== + 100001 | + +taos> select GREATEST(cast(100001.1111111 as float), cast(100000.1111111 as double)); + greatest(cast(100001.1111111 as float), cast(100000.1111111 as d | +=================================================================== + 100001.109375 | + +taos> select GREATEST(cast(100001.1111111 as float), cast(100000.1111111 as timestamp)); + greatest(cast(100001.1111111 as float), cast(100000.1111111 as t | +=================================================================== + 100001 | + +taos> select GREATEST(cast(100001.1111111 as float), cast(100000.1111111 as varchar(20))); + greatest(cast(100001.1111111 as float), cast(100000.1111111 as v | +=================================================================== + 100001 | + +taos> select GREATEST(cast(100001.1111111 as float), cast(100000.1111111 as nchar(20))); + greatest(cast(100001.1111111 as float), cast(100000.1111111 as n | +=================================================================== + 100001 | + +taos> select GREATEST(cast(100000.1111111 as double), cast(100001.1111111 as double)); + greatest(cast(100000.1111111 as double), cast(100001.1111111 as | +=================================================================== + 100001.1111111 | + +taos> select GREATEST(cast(100000.1111111 as double), cast(100001.1111111 as timestamp)); + greatest(cast(100000.1111111 as double), cast(100001.1111111 as | +=================================================================== + 100001 | + +taos> select GREATEST(cast(100000.1111111 as double), cast(100001.1111111 as varchar(20))); + greatest(cast(100000.1111111 as double), cast(100001.1111111 as | +=================================================================== + 100001.1111111 | + +taos> select GREATEST(cast(100000.1111111 as double), cast(100001.1111111 as nchar(20))); + greatest(cast(100000.1111111 as double), cast(100001.1111111 as | +=================================================================== + 100001.1111111 | + +taos> select GREATEST(cast(100001.1111111 as double), cast(100000.1111111 as double)); + greatest(cast(100001.1111111 as double), cast(100000.1111111 as | +=================================================================== + 100001.1111111 | + +taos> select GREATEST(cast(100001.1111111 as double), cast(100000.1111111 as timestamp)); + greatest(cast(100001.1111111 as double), cast(100000.1111111 as | +=================================================================== + 100001.1111111 | + +taos> select GREATEST(cast(100001.1111111 as double), cast(100000.1111111 as varchar(20))); + greatest(cast(100001.1111111 as double), cast(100000.1111111 as | +=================================================================== + 100001.1111111 | + +taos> select GREATEST(cast(100001.1111111 as double), cast(100000.1111111 as nchar(20))); + greatest(cast(100001.1111111 as double), cast(100000.1111111 as | +=================================================================== + 100001.1111111 | + +taos> select GREATEST(cast('中文测试' as varchar(20)), cast('中文测试一' as varchar(20))); + greatest(cast('中文测试' as varchar(20)), cast('中文测试 | +=================================================================== + 中文测试一 | + +taos> select GREATEST(cast('中文测试' as varchar(20)), cast('中文测试一' as nchar(20))); + greatest(cast('中文测试' as varchar(20)), cast('中文测试 | +=================================================================== + 中文测试一 | + +taos> select GREATEST(cast('中文测试一' as varchar(20)), cast('中文测试' as varchar(20))); + greatest(cast('中文测试一' as varchar(20)), cast('中文测 | +=================================================================== + 中文测试一 | + +taos> select GREATEST(cast('中文测试一' as varchar(20)), cast('中文测试' as nchar(20))); + greatest(cast('中文测试一' as varchar(20)), cast('中文测 | +=================================================================== + 中文测试一 | + +taos> select GREATEST(cast('abc123abc' as varchar(20)), cast('abc124abc' as varchar(20))); + greatest(cast('abc123abc' as varchar(20)), cast('abc124abc' as v | +=================================================================== + abc124abc | + +taos> select GREATEST(cast('abc123abc' as varchar(20)), cast('abc124abc' as nchar(20))); + greatest(cast('abc123abc' as varchar(20)), cast('abc124abc' as n | +=================================================================== + abc124abc | + +taos> select GREATEST(cast('abc124abc' as varchar(20)), cast('abc123abc' as varchar(20))); + greatest(cast('abc124abc' as varchar(20)), cast('abc123abc' as v | +=================================================================== + abc124abc | + +taos> select GREATEST(cast('abc124abc' as varchar(20)), cast('abc123abc' as nchar(20))); + greatest(cast('abc124abc' as varchar(20)), cast('abc123abc' as n | +=================================================================== + abc124abc | + +taos> select GREATEST(cast('abc123abc' as nchar(20)), cast('abc124abc' as nchar(20))); + greatest(cast('abc123abc' as nchar(20)), cast('abc124abc' as nch | +=================================================================== + abc124abc | + +taos> select GREATEST(cast(100 as tinyint), cast(101 as float), cast(102 as varchar(20))); + greatest(cast(100 as tinyint), cast(101 as float), cast(102 as v | +=================================================================== + 102 | + +taos> select GREATEST(cast(100 as float), cast(101 as tinyint), cast(102 as varchar(20))); + greatest(cast(100 as float), cast(101 as tinyint), cast(102 as v | +=================================================================== + 102 | + +taos> select GREATEST(cast(100 as float), cast(101 as varchar(20)), cast(102 as tinyint)); + greatest(cast(100 as float), cast(101 as varchar(20)), cast(102 | +=================================================================== + 102 | + +taos> select GREATEST(cast(100 as varchar(20)), cast(101 as float), cast(102 as tinyint)); + greatest(cast(100 as varchar(20)), cast(101 as float), cast(102 | +=================================================================== + 102 | + +taos> select GREATEST('a','b','c','d','e','f','g','h','1231','15155'); + greatest('a','b','c','d','e','f','g','h','1231','15155') | +=========================================================== + h | + +taos> select GREATEST(current, voltage, phase, id, nch1, nch2, var1, var2) from ts_4893.meters order by ts limit 10; + greatest(current, voltage, phase, id, nch1, nch2, var1, var2) | +================================================================ + 221 | + 220 | + 215 | + 216 | + 219 | + 221 | + 215 | + 217 | + 216 | + 223 | + +taos> select GREATEST(current, voltage, phase, id) from ts_4893.meters order by ts limit 10; + greatest(current, voltage, phase, id) | +======================================== + 221 | + 220 | + 215 | + 216 | + 219 | + 221 | + 215 | + 217 | + 216 | + 223 | + +taos> select GREATEST(nch1, nch2, var1, var2) from ts_4893.meters order by ts limit 10; + greatest(nch1, nch2, var1, var2) | +=================================== + 四 | + 三a | + 四 | + 一 | + 一二三四五六七八九十 | + 一二三 | + prision | + 一二三四五六七八九十 | + prision | + 一 | + +taos> select GREATEST(221, voltage) from ts_4893.meters order by ts limit 10; + greatest(221, voltage) | +========================= + 221 | + 221 | + 221 | + 221 | + 221 | + 221 | + 221 | + 221 | + 221 | + 223 | + +taos> select GREATEST(5, id) from ts_4893.meters order by ts limit 10; + greatest(5, id) | +======================== + 5 | + 5 | + 5 | + 5 | + 5 | + 5 | + 6 | + 7 | + 8 | + 9 | + +taos> select GREATEST('r', nch1) from ts_4893.meters order by ts limit 10; + greatest('r', nch1) | +================================= + r | + 一二三四五六七八九十 | + update | + r | + r | + r | + r | + 一二三四五六七八九十 | + r | + r | + +taos> select GREATEST('r', nch1, nch2) from ts_4893.meters order by ts limit 10; + greatest('r', nch1, nch2) | +================================= + 四 | + 一二三四五六七八九十 | + update | + 一 | + r | + 一二三 | + r | + 一二三四五六七八九十 | + r | + r | + +taos> select GREATEST('r', var1) from ts_4893.meters order by ts limit 10; + greatest('r', var1) | +================================= + r | + r | + r | + r | + 一二三四五六七八九十 | + update | + r | + r | + r | + r | + +taos> select GREATEST('r', var1, var2) from ts_4893.meters order by ts limit 10; + greatest('r', var1, var2) | +================================= + r | + 三a | + 四 | + r | + 一二三四五六七八九十 | + update | + r | + r | + r | + 一 | + +taos> select GREATEST('二中文测试', nch1) from ts_4893.meters order by ts limit 10; + greatest('二中文测试', nch1) | +==================================== + 二中文测试 | + 二中文测试 | + 二中文测试 | + 二中文测试 | + 二中文测试 | + 二中文测试 | + 二中文测试 | + 二中文测试 | + 二中文测试 | + 二中文测试 | + +taos> select GREATEST('二中文测试', nch1, nch2) from ts_4893.meters order by ts limit 10; + greatest('二中文测试', nch1, nch2) | +========================================== + 四 | + 二中文测试 | + 二中文测试 | + 二中文测试 | + 二中文测试 | + 二中文测试 | + 二中文测试 | + 二中文测试 | + 二中文测试 | + 二中文测试 | + +taos> select GREATEST('二中文测试', var1) from ts_4893.meters order by ts limit 10; + greatest('二中文测试', var1) | +==================================== + 二中文测试 | + 二中文测试 | + 二中文测试 | + 二中文测试 | + 二中文测试 | + 二中文测试 | + 二中文测试 | + 二中文测试 | + 二中文测试 | + 二中文测试 | + +taos> select GREATEST('二中文测试', var1, var2) from ts_4893.meters order by ts limit 10; + greatest('二中文测试', var1, var2) | +========================================== + 二中文测试 | + 二中文测试 | + 四 | + 二中文测试 | + 二中文测试 | + 二中文测试 | + 二中文测试 | + 二中文测试 | + 二中文测试 | + 二中文测试 | + +taos> select GREATEST('23', 3443434343434343); + greatest('23', 3443434343434343) | +=================================== + 3443434343434343 | + +taos> select GREATEST(co, 3443434343434343) from ts_4893.n1; + greatest(co, 3443434343434343) | +================================= + 3443434343434343 | + 3443434343434343 | + 3443434343434343 | + +taos> select GREATEST('23', 1443434343434343) from ts_4893.n1; + greatest('23', 1443434343434343) | +=================================== + 1443434343434343 | + 1443434343434343 | + 1443434343434343 | + +taos> select GREATEST('23', 3443434343434343) from ts_4893.n1 + greatest('23', 3443434343434343) | +=================================== + 3443434343434343 | + 3443434343434343 | + 3443434343434343 | + +taos> select GREATEST(current, voltage) from ts_4893.n1; + greatest(current, voltage) | +============================= + NULL | + NULL | + 5 | + +taos> select GREATEST(current, voltage, '15') from ts_4893.n1; + greatest(current, voltage, '15') | +=================================== + NULL | + NULL | + 15 | + diff --git a/tests/army/query/function/ans/least.csv b/tests/army/query/function/ans/least.csv new file mode 100644 index 0000000000..e1ee1095ce --- /dev/null +++ b/tests/army/query/function/ans/least.csv @@ -0,0 +1,1389 @@ + +taos> alter local 'transToStrWhenMixTypeInLeast' '1'; + +taos> select LEAST(1,2,3,4,5,6,7,8,9,10); + least(1,2,3,4,5,6,7,8,9,10) | +============================== + 1 | + +taos> select LEAST(1,1.1,2.23,3.4444,5.66666666,6.21241241,7.999999999999); + least(1,1.1,2.23,3.4444,5.66666666,6.21241241,7.999999999999) | +================================================================ + 1 | + +taos> select LEAST(1,'2',3.3,4.4,5); + least(1,'2',3.3,4.4,5) | +============================ + 1 | + +taos> select LEAST(1,2,3,4,5,'5.1'); + least(1,2,3,4,5,'5.1') | +============================ + 1 | + +taos> select LEAST('1','2','3','4',5); + least('1','2','3','4',5) | +============================ + 1 | + +taos> select LEAST('1','2','3','4','5'); + least('1','2','3','4','5') | +============================= + 1 | + +taos> select LEAST(1,2,3,4,5,6,7,'a','b','一','二','三'); + least(1,2,3,4,5,6,7,'a','b','一','二','三') | +================================================= + 1 | + +taos> select LEAST(1,2,3,4,5,6,7,'a','b','c','1','2','1231213'); + least(1,2,3,4,5,6,7,'a','b','c','1','2','1231213') | +===================================================== + 1 | + +taos> select LEAST(1,2,3,4,5,6,7,'a','b','c','1','2','1231213','1231213.123123'); + least(1,2,3,4,5,6,7,'a','b','c','1','2','1231213','1231213.12312 | +=================================================================== + 1 | + +taos> select LEAST(cast(0 as bool), cast(1 as tinyint), cast(2 as smallint), cast(3 as int), cast(4 as bigint), cast(5 as float), cast(6 as double), cast(8 as varchar(20)), cast(9 as nchar(20))); + least(cast(0 as bool), cast(1 as tinyint), cast(2 as smallint), | +=================================================================== + 0 | + +taos> select LEAST(cast(0 as bool), cast(1 as tinyint unsigned), cast(2 as smallint unsigned), cast(3 as int unsigned), cast(4 as bigint unsigned), cast(5 as float), cast(6 as double), cast(8 as varchar(20)), cast(9 as nchar(20))); + least(cast(0 as bool), cast(1 as tinyint unsigned), cast(2 as sm | +=================================================================== + 0 | + +taos> select LEAST(cast(0 as bool), cast(1 as tinyint), cast(2 as smallint), cast(3 as int), cast(4 as bigint), cast(5 as float), cast(6 as double), cast(8 as varchar(20)), cast(9 as nchar(20)), cast(0 as bool), cast(1 as tinyint unsigned), cast(2 as smallint unsigned), cast(3 as int unsigned), cast(4 as bigint unsigned), cast(5 as float), cast(6 as double)); + least(cast(0 as bool), cast(1 as tinyint), cast(2 as smallint), | +=================================================================== + 0 | + +taos> select LEAST(cast(100 as tinyint), cast(101 as tinyint)); + least(cast(100 as tinyint), cast(101 as tinyint)) | +==================================================== + 100 | + +taos> select LEAST(cast(100 as tinyint), cast(101 as smallint)); + least(cast(100 as tinyint), cast(101 as smallint)) | +===================================================== + 100 | + +taos> select LEAST(cast(100 as tinyint), cast(101 as int)); + least(cast(100 as tinyint), cast(101 as int)) | +================================================ + 100 | + +taos> select LEAST(cast(100 as tinyint), cast(101 as bigint)); + least(cast(100 as tinyint), cast(101 as bigint)) | +=================================================== + 100 | + +taos> select LEAST(cast(100 as tinyint), cast(101 as float)); + least(cast(100 as tinyint), cast(101 as float)) | +================================================== + 100 | + +taos> select LEAST(cast(100 as tinyint), cast(101 as double)); + least(cast(100 as tinyint), cast(101 as double)) | +=================================================== + 100 | + +taos> select LEAST(cast(100 as tinyint), cast(101 as varchar(20))); + least(cast(100 as tinyint), cast(101 as varchar(20))) | +======================================================== + 100 | + +taos> select LEAST(cast(100 as tinyint), cast(101 as nchar(20))); + least(cast(100 as tinyint), cast(101 as nchar(20))) | +====================================================== + 100 | + +taos> select LEAST(cast(101 as tinyint), cast(100 as tinyint)); + least(cast(101 as tinyint), cast(100 as tinyint)) | +==================================================== + 100 | + +taos> select LEAST(cast(101 as tinyint), cast(100 as smallint)); + least(cast(101 as tinyint), cast(100 as smallint)) | +===================================================== + 100 | + +taos> select LEAST(cast(101 as tinyint), cast(100 as int)); + least(cast(101 as tinyint), cast(100 as int)) | +================================================ + 100 | + +taos> select LEAST(cast(101 as tinyint), cast(100 as bigint)); + least(cast(101 as tinyint), cast(100 as bigint)) | +=================================================== + 100 | + +taos> select LEAST(cast(101 as tinyint), cast(100 as float)); + least(cast(101 as tinyint), cast(100 as float)) | +================================================== + 100 | + +taos> select LEAST(cast(101 as tinyint), cast(100 as double)); + least(cast(101 as tinyint), cast(100 as double)) | +=================================================== + 100 | + +taos> select LEAST(cast(101 as tinyint), cast(100 as varchar(20))); + least(cast(101 as tinyint), cast(100 as varchar(20))) | +======================================================== + 100 | + +taos> select LEAST(cast(101 as tinyint), cast(100 as nchar(20))); + least(cast(101 as tinyint), cast(100 as nchar(20))) | +====================================================== + 100 | + +taos> select LEAST(cast(1000 as smallint), cast(1001 as smallint)); + least(cast(1000 as smallint), cast(1001 as smallint)) | +======================================================== + 1000 | + +taos> select LEAST(cast(1000 as smallint), cast(1001 as int)); + least(cast(1000 as smallint), cast(1001 as int)) | +=================================================== + 1000 | + +taos> select LEAST(cast(1000 as smallint), cast(1001 as bigint)); + least(cast(1000 as smallint), cast(1001 as bigint)) | +====================================================== + 1000 | + +taos> select LEAST(cast(1000 as smallint), cast(1001 as float)); + least(cast(1000 as smallint), cast(1001 as float)) | +===================================================== + 1000 | + +taos> select LEAST(cast(1000 as smallint), cast(1001 as double)); + least(cast(1000 as smallint), cast(1001 as double)) | +====================================================== + 1000 | + +taos> select LEAST(cast(1000 as smallint), cast(1001 as varchar(20))); + least(cast(1000 as smallint), cast(1001 as varchar(20))) | +=========================================================== + 1000 | + +taos> select LEAST(cast(1000 as smallint), cast(1001 as nchar(20))); + least(cast(1000 as smallint), cast(1001 as nchar(20))) | +========================================================= + 1000 | + +taos> select LEAST(cast(1001 as smallint), cast(1000 as smallint)); + least(cast(1001 as smallint), cast(1000 as smallint)) | +======================================================== + 1000 | + +taos> select LEAST(cast(1001 as smallint), cast(1000 as int)); + least(cast(1001 as smallint), cast(1000 as int)) | +=================================================== + 1000 | + +taos> select LEAST(cast(1001 as smallint), cast(1000 as bigint)); + least(cast(1001 as smallint), cast(1000 as bigint)) | +====================================================== + 1000 | + +taos> select LEAST(cast(1001 as smallint), cast(1000 as float)); + least(cast(1001 as smallint), cast(1000 as float)) | +===================================================== + 1000 | + +taos> select LEAST(cast(1001 as smallint), cast(1000 as double)); + least(cast(1001 as smallint), cast(1000 as double)) | +====================================================== + 1000 | + +taos> select LEAST(cast(1001 as smallint), cast(1000 as varchar(20))); + least(cast(1001 as smallint), cast(1000 as varchar(20))) | +=========================================================== + 1000 | + +taos> select LEAST(cast(1001 as smallint), cast(1000 as nchar(20))); + least(cast(1001 as smallint), cast(1000 as nchar(20))) | +========================================================= + 1000 | + +taos> select LEAST(cast(1000000 as int), cast(1000001 as int)); + least(cast(1000000 as int), cast(1000001 as int)) | +==================================================== + 1000000 | + +taos> select LEAST(cast(1000000 as int), cast(1000001 as bigint)); + least(cast(1000000 as int), cast(1000001 as bigint)) | +======================================================= + 1000000 | + +taos> select LEAST(cast(1000000 as int), cast(1000001 as float)); + least(cast(1000000 as int), cast(1000001 as float)) | +====================================================== + 1e+06 | + +taos> select LEAST(cast(1000000 as int), cast(1000001 as double)); + least(cast(1000000 as int), cast(1000001 as double)) | +======================================================= + 1000000 | + +taos> select LEAST(cast(1000000 as int), cast(1000001 as varchar(20))); + least(cast(1000000 as int), cast(1000001 as varchar(20))) | +============================================================ + 1000000 | + +taos> select LEAST(cast(1000000 as int), cast(1000001 as nchar(20))); + least(cast(1000000 as int), cast(1000001 as nchar(20))) | +========================================================== + 1000000 | + +taos> select LEAST(cast(1000001 as int), cast(1000000 as int)); + least(cast(1000001 as int), cast(1000000 as int)) | +==================================================== + 1000000 | + +taos> select LEAST(cast(1000001 as int), cast(1000000 as bigint)); + least(cast(1000001 as int), cast(1000000 as bigint)) | +======================================================= + 1000000 | + +taos> select LEAST(cast(1000001 as int), cast(1000000 as float)); + least(cast(1000001 as int), cast(1000000 as float)) | +====================================================== + 1e+06 | + +taos> select LEAST(cast(1000001 as int), cast(1000000 as double)); + least(cast(1000001 as int), cast(1000000 as double)) | +======================================================= + 1000000 | + +taos> select LEAST(cast(1000001 as int), cast(1000000 as varchar(20))); + least(cast(1000001 as int), cast(1000000 as varchar(20))) | +============================================================ + 1000000 | + +taos> select LEAST(cast(1000001 as int), cast(1000000 as nchar(20))); + least(cast(1000001 as int), cast(1000000 as nchar(20))) | +========================================================== + 1000000 | + +taos> select LEAST(cast(1000000000 as bigint), cast(1000000001 as bigint)); + least(cast(1000000000 as bigint), cast(1000000001 as bigint)) | +================================================================ + 1000000000 | + +taos> select LEAST(cast(1000000000 as bigint), cast(1000000001 as float)); + least(cast(1000000000 as bigint), cast(1000000001 as float)) | +=============================================================== + 1e+09 | + +taos> select LEAST(cast(1000000000 as bigint), cast(1000000001 as double)); + least(cast(1000000000 as bigint), cast(1000000001 as double)) | +================================================================ + 1000000000 | + +taos> select LEAST(cast(1000000000 as bigint), cast(1000000001 as varchar(20))); + least(cast(1000000000 as bigint), cast(1000000001 as varchar(20) | +=================================================================== + 1000000000 | + +taos> select LEAST(cast(1000000000 as bigint), cast(1000000001 as nchar(20))); + least(cast(1000000000 as bigint), cast(1000000001 as nchar(20))) | +=================================================================== + 1000000000 | + +taos> select LEAST(cast(1000000001 as bigint), cast(1000000000 as bigint)); + least(cast(1000000001 as bigint), cast(1000000000 as bigint)) | +================================================================ + 1000000000 | + +taos> select LEAST(cast(1000000001 as bigint), cast(1000000000 as float)); + least(cast(1000000001 as bigint), cast(1000000000 as float)) | +=============================================================== + 1e+09 | + +taos> select LEAST(cast(1000000001 as bigint), cast(1000000000 as double)); + least(cast(1000000001 as bigint), cast(1000000000 as double)) | +================================================================ + 1000000000 | + +taos> select LEAST(cast(1000000001 as bigint), cast(1000000000 as varchar(20))); + least(cast(1000000001 as bigint), cast(1000000000 as varchar(20) | +=================================================================== + 1000000000 | + +taos> select LEAST(cast(1000000001 as bigint), cast(1000000000 as nchar(20))); + least(cast(1000000001 as bigint), cast(1000000000 as nchar(20))) | +=================================================================== + 1000000000 | + +taos> select LEAST(cast(100000.1111111 as float), cast(100001.1111111 as float)); + least(cast(100000.1111111 as float), cast(100001.1111111 as floa | +=================================================================== + 100000 | + +taos> select LEAST(cast(100000.1111111 as float), cast(100001.1111111 as double)); + least(cast(100000.1111111 as float), cast(100001.1111111 as doub | +=================================================================== + 100000.109375 | + +taos> select LEAST(cast(100000.1111111 as float), cast(100001.1111111 as timestamp)); + least(cast(100000.1111111 as float), cast(100001.1111111 as time | +=================================================================== + 100000 | + +taos> select LEAST(cast(100000.1111111 as float), cast(100001.1111111 as varchar(20))); + least(cast(100000.1111111 as float), cast(100001.1111111 as varc | +=================================================================== + 100000.109375 | + +taos> select LEAST(cast(100000.1111111 as float), cast(100001.1111111 as nchar(20))); + least(cast(100000.1111111 as float), cast(100001.1111111 as ncha | +=================================================================== + 100000.109375 | + +taos> select LEAST(cast(100001.1111111 as float), cast(100000.1111111 as float)); + least(cast(100001.1111111 as float), cast(100000.1111111 as floa | +=================================================================== + 100000 | + +taos> select LEAST(cast(100001.1111111 as float), cast(100000.1111111 as double)); + least(cast(100001.1111111 as float), cast(100000.1111111 as doub | +=================================================================== + 100000.1111111 | + +taos> select LEAST(cast(100001.1111111 as float), cast(100000.1111111 as timestamp)); + least(cast(100001.1111111 as float), cast(100000.1111111 as time | +=================================================================== + 100000 | + +taos> select LEAST(cast(100001.1111111 as float), cast(100000.1111111 as varchar(20))); + least(cast(100001.1111111 as float), cast(100000.1111111 as varc | +=================================================================== + 100000.1111111 | + +taos> select LEAST(cast(100001.1111111 as float), cast(100000.1111111 as nchar(20))); + least(cast(100001.1111111 as float), cast(100000.1111111 as ncha | +=================================================================== + 100000.1111111 | + +taos> select LEAST(cast(100000.1111111 as double), cast(100001.1111111 as double)); + least(cast(100000.1111111 as double), cast(100001.1111111 as dou | +=================================================================== + 100000.1111111 | + +taos> select LEAST(cast(100000.1111111 as double), cast(100001.1111111 as timestamp)); + least(cast(100000.1111111 as double), cast(100001.1111111 as tim | +=================================================================== + 100000.1111111 | + +taos> select LEAST(cast(100000.1111111 as double), cast(100001.1111111 as varchar(20))); + least(cast(100000.1111111 as double), cast(100001.1111111 as var | +=================================================================== + 100000.111111 | + +taos> select LEAST(cast(100000.1111111 as double), cast(100001.1111111 as nchar(20))); + least(cast(100000.1111111 as double), cast(100001.1111111 as nch | +=================================================================== + 100000.111111 | + +taos> select LEAST(cast(100001.1111111 as double), cast(100000.1111111 as double)); + least(cast(100001.1111111 as double), cast(100000.1111111 as dou | +=================================================================== + 100000.1111111 | + +taos> select LEAST(cast(100001.1111111 as double), cast(100000.1111111 as timestamp)); + least(cast(100001.1111111 as double), cast(100000.1111111 as tim | +=================================================================== + 100000 | + +taos> select LEAST(cast(100001.1111111 as double), cast(100000.1111111 as varchar(20))); + least(cast(100001.1111111 as double), cast(100000.1111111 as var | +=================================================================== + 100000.1111111 | + +taos> select LEAST(cast(100001.1111111 as double), cast(100000.1111111 as nchar(20))); + least(cast(100001.1111111 as double), cast(100000.1111111 as nch | +=================================================================== + 100000.1111111 | + +taos> select LEAST(cast('中文测试' as varchar(20)), cast('中文测试一' as varchar(20))); + least(cast('中文测试' as varchar(20)), cast('中文测试一 | +=================================================================== + 中文测试 | + +taos> select LEAST(cast('中文测试' as varchar(20)), cast('中文测试一' as nchar(20))); + least(cast('中文测试' as varchar(20)), cast('中文测试一 | +=================================================================== + 中文测试 | + +taos> select LEAST(cast('中文测试一' as varchar(20)), cast('中文测试' as varchar(20))); + least(cast('中文测试一' as varchar(20)), cast('中文测试 | +=================================================================== + 中文测试 | + +taos> select LEAST(cast('中文测试一' as varchar(20)), cast('中文测试' as nchar(20))); + least(cast('中文测试一' as varchar(20)), cast('中文测试 | +=================================================================== + 中文测试 | + +taos> select LEAST(cast('abc123abc' as varchar(20)), cast('abc124abc' as varchar(20))); + least(cast('abc123abc' as varchar(20)), cast('abc124abc' as varc | +=================================================================== + abc123abc | + +taos> select LEAST(cast('abc123abc' as varchar(20)), cast('abc124abc' as nchar(20))); + least(cast('abc123abc' as varchar(20)), cast('abc124abc' as ncha | +=================================================================== + abc123abc | + +taos> select LEAST(cast('abc124abc' as varchar(20)), cast('abc123abc' as varchar(20))); + least(cast('abc124abc' as varchar(20)), cast('abc123abc' as varc | +=================================================================== + abc123abc | + +taos> select LEAST(cast('abc124abc' as varchar(20)), cast('abc123abc' as nchar(20))); + least(cast('abc124abc' as varchar(20)), cast('abc123abc' as ncha | +=================================================================== + abc123abc | + +taos> select LEAST(cast('abc123abc' as nchar(20)), cast('abc124abc' as nchar(20))); + least(cast('abc123abc' as nchar(20)), cast('abc124abc' as nchar( | +=================================================================== + abc123abc | + +taos> select LEAST(cast(100 as tinyint), cast(101 as float), cast(102 as varchar(20))); + least(cast(100 as tinyint), cast(101 as float), cast(102 as varc | +=================================================================== + 100 | + +taos> select LEAST(cast(100 as varchar(20)), cast(101 as float), cast(102 as tinyint)); + least(cast(100 as varchar(20)), cast(101 as float), cast(102 as | +=================================================================== + 100 | + +taos> select LEAST('a','b','c','d','e','f','g','h','1231','15155'); + least('a','b','c','d','e','f','g','h','1231','15155') | +======================================================== + 1231 | + +taos> select LEAST(current, voltage, phase, id, nch1, nch2, var1, var2) from ts_4893.meters order by ts limit 10; + least(current, voltage, phase, id, nch1, nch2, var1, var2) | +============================================================= + 0 | + 0.138830 | + 0.796942 | + 0.537330 | + 0.313430 | + 0.332767 | + 0.846763 | + 0.637813 | + 0.115989 | + 0.373575 | + +taos> select LEAST(current, voltage, phase, id) from ts_4893.meters order by ts limit 10; + least(current, voltage, phase, id) | +===================================== + 0 | + 0.13883 | + 0.796942 | + 0.53733 | + 0.31343 | + 0.332767 | + 0.846763 | + 0.637813 | + 0.115989 | + 0.373575 | + +taos> select LEAST(nch1, nch2, var1, var2) from ts_4893.meters order by ts limit 10; + least(nch1, nch2, var1, var2) | +================================= + e | + f | + c | + e | + b | + g | + again | + b | + c | + again | + +taos> select LEAST(221, voltage) from ts_4893.meters order by ts limit 10; + least(221, voltage) | +======================== + 221 | + 220 | + 215 | + 216 | + 219 | + 221 | + 215 | + 217 | + 216 | + 221 | + +taos> select LEAST(5, id) from ts_4893.meters order by ts limit 10; + least(5, id) | +======================== + 0 | + 1 | + 2 | + 3 | + 4 | + 5 | + 5 | + 5 | + 5 | + 5 | + +taos> select LEAST('r', nch1) from ts_4893.meters order by ts limit 10; + least('r', nch1) | +================================= + novel | + r | + r | + prision | + novel | + novel | + again | + r | + novel | + again | + +taos> select LEAST('r', nch1, nch2) from ts_4893.meters order by ts limit 10; + least('r', nch1, nch2) | +================================= + novel | + f | + c | + prision | + e | + novel | + again | + d | + c | + again | + +taos> select LEAST('r', var1) from ts_4893.meters order by ts limit 10; + least('r', var1) | +================================= + novel | + person | + novel | + plate | + r | + r | + prision | + person | + prision | + plate | + +taos> select LEAST('r', var1, var2) from ts_4893.meters order by ts limit 10; + least('r', var1, var2) | +================================= + e | + person | + novel | + e | + b | + g | + b | + b | + e | + plate | + +taos> select LEAST('二中文测试', nch1) from ts_4893.meters order by ts limit 10; + least('二中文测试', nch1) | +================================= + novel | + 一二三四五六七八九十 | + update | + prision | + novel | + novel | + again | + 一二三四五六七八九十 | + novel | + again | + +taos> select LEAST('二中文测试', nch1, nch2) from ts_4893.meters order by ts limit 10; + least('二中文测试', nch1, nch2) | +======================================= + novel | + f | + c | + prision | + e | + novel | + again | + d | + c | + again | + +taos> select LEAST('二中文测试', var1) from ts_4893.meters order by ts limit 10; + least('二中文测试', var1) | +================================= + novel | + person | + novel | + plate | + 一二三四五六七八九十 | + update | + prision | + person | + prision | + plate | + +taos> select LEAST('二中文测试', var1, var2) from ts_4893.meters order by ts limit 10; + least('二中文测试', var1, var2) | +======================================= + e | + person | + novel | + e | + b | + g | + b | + b | + e | + plate | + +taos> select LEAST('23', 3443434343434343); + least('23', 3443434343434343) | +================================ + 23 | + +taos> select LEAST(co, 3443434343434343) from ts_4893.n1; + least(co, 3443434343434343) | +================================= + 23 | + 23 | + 23 | + +taos> select LEAST('23', 3443434343434343) from ts_4893.n1; + least('23', 3443434343434343) | +================================ + 23 | + 23 | + 23 | + +taos> select LEAST('23', 1443434343434343) from ts_4893.n1; + least('23', 1443434343434343) | +================================ + 1443434343434343 | + 1443434343434343 | + 1443434343434343 | + +taos> select LEAST(current, voltage) from ts_4893.n1; + least(current, voltage) | +========================== + NULL | + NULL | + 3 | + +taos> select LEAST(current, voltage, '15') from ts_4893.n1; + least(current, voltage, '15') | +================================ + NULL | + NULL | + 15 | + +taos> alter local 'transToStrWhenMixTypeInLeast' '0'; + +taos> select LEAST(1,2,3,4,5,6,7,8,9,10); + least(1,2,3,4,5,6,7,8,9,10) | +============================== + 1 | + +taos> select LEAST(1,1.1,2.23,3.4444,5.66666666,6.21241241,7.999999999999); + least(1,1.1,2.23,3.4444,5.66666666,6.21241241,7.999999999999) | +================================================================ + 1 | + +taos> select LEAST(1,'2',3.3,4.4,5); + least(1,'2',3.3,4.4,5) | +============================ + 1 | + +taos> select LEAST(1,2,3,4,5,'5.1'); + least(1,2,3,4,5,'5.1') | +========================= + 1 | + +taos> select LEAST('1','2','3','4',5); + least('1','2','3','4',5) | +=========================== + 1 | + +taos> select LEAST('1','2','3','4','5'); + least('1','2','3','4','5') | +============================= + 1 | + +taos> select LEAST(1,2,3,4,5,6,7,'a','b','一','二','三'); + least(1,2,3,4,5,6,7,'a','b','一','二','三') | +================================================= + 0 | + +taos> select LEAST(1,2,3,4,5,6,7,'a','b','c','1','2','1231213'); + least(1,2,3,4,5,6,7,'a','b','c','1','2','1231213') | +===================================================== + 0 | + +taos> select LEAST(1,2,3,4,5,6,7,'a','b','c','1','2','1231213','1231213.123123'); + least(1,2,3,4,5,6,7,'a','b','c','1','2','1231213','1231213.12312 | +=================================================================== + 0 | + +taos> select LEAST(cast(0 as bool), cast(1 as tinyint), cast(2 as smallint), cast(3 as int), cast(4 as bigint), cast(5 as float), cast(6 as double), cast(8 as varchar(20)), cast(9 as nchar(20))); + least(cast(0 as bool), cast(1 as tinyint), cast(2 as smallint), | +=================================================================== + 0 | + +taos> select LEAST(cast(0 as bool), cast(1 as tinyint unsigned), cast(2 as smallint unsigned), cast(3 as int unsigned), cast(4 as bigint unsigned), cast(5 as float), cast(6 as double), cast(8 as varchar(20)), cast(9 as nchar(20))); + least(cast(0 as bool), cast(1 as tinyint unsigned), cast(2 as sm | +=================================================================== + 0 | + +taos> select LEAST(cast(0 as bool), cast(1 as tinyint), cast(2 as smallint), cast(3 as int), cast(4 as bigint), cast(5 as float), cast(6 as double), cast(8 as varchar(20)), cast(9 as nchar(20)), cast(0 as bool), cast(1 as tinyint unsigned), cast(2 as smallint unsigned), cast(3 as int unsigned), cast(4 as bigint unsigned), cast(5 as float), cast(6 as double)); + least(cast(0 as bool), cast(1 as tinyint), cast(2 as smallint), | +=================================================================== + 0 | + +taos> select LEAST(cast(100 as tinyint), cast(101 as tinyint)); + least(cast(100 as tinyint), cast(101 as tinyint)) | +==================================================== + 100 | + +taos> select LEAST(cast(100 as tinyint), cast(101 as smallint)); + least(cast(100 as tinyint), cast(101 as smallint)) | +===================================================== + 100 | + +taos> select LEAST(cast(100 as tinyint), cast(101 as int)); + least(cast(100 as tinyint), cast(101 as int)) | +================================================ + 100 | + +taos> select LEAST(cast(100 as tinyint), cast(101 as bigint)); + least(cast(100 as tinyint), cast(101 as bigint)) | +=================================================== + 100 | + +taos> select LEAST(cast(100 as tinyint), cast(101 as float)); + least(cast(100 as tinyint), cast(101 as float)) | +================================================== + 100 | + +taos> select LEAST(cast(100 as tinyint), cast(101 as double)); + least(cast(100 as tinyint), cast(101 as double)) | +=================================================== + 100 | + +taos> select LEAST(cast(100 as tinyint), cast(101 as varchar(20))); + least(cast(100 as tinyint), cast(101 as varchar(20))) | +======================================================== + 100 | + +taos> select LEAST(cast(100 as tinyint), cast(101 as nchar(20))); + least(cast(100 as tinyint), cast(101 as nchar(20))) | +====================================================== + 100 | + +taos> select LEAST(cast(101 as tinyint), cast(100 as tinyint)); + least(cast(101 as tinyint), cast(100 as tinyint)) | +==================================================== + 100 | + +taos> select LEAST(cast(101 as tinyint), cast(100 as smallint)); + least(cast(101 as tinyint), cast(100 as smallint)) | +===================================================== + 100 | + +taos> select LEAST(cast(101 as tinyint), cast(100 as int)); + least(cast(101 as tinyint), cast(100 as int)) | +================================================ + 100 | + +taos> select LEAST(cast(101 as tinyint), cast(100 as bigint)); + least(cast(101 as tinyint), cast(100 as bigint)) | +=================================================== + 100 | + +taos> select LEAST(cast(101 as tinyint), cast(100 as float)); + least(cast(101 as tinyint), cast(100 as float)) | +================================================== + 100 | + +taos> select LEAST(cast(101 as tinyint), cast(100 as double)); + least(cast(101 as tinyint), cast(100 as double)) | +=================================================== + 100 | + +taos> select LEAST(cast(101 as tinyint), cast(100 as varchar(20))); + least(cast(101 as tinyint), cast(100 as varchar(20))) | +======================================================== + 100 | + +taos> select LEAST(cast(101 as tinyint), cast(100 as nchar(20))); + least(cast(101 as tinyint), cast(100 as nchar(20))) | +====================================================== + 100 | + +taos> select LEAST(cast(1000 as smallint), cast(1001 as smallint)); + least(cast(1000 as smallint), cast(1001 as smallint)) | +======================================================== + 1000 | + +taos> select LEAST(cast(1000 as smallint), cast(1001 as int)); + least(cast(1000 as smallint), cast(1001 as int)) | +=================================================== + 1000 | + +taos> select LEAST(cast(1000 as smallint), cast(1001 as bigint)); + least(cast(1000 as smallint), cast(1001 as bigint)) | +====================================================== + 1000 | + +taos> select LEAST(cast(1000 as smallint), cast(1001 as float)); + least(cast(1000 as smallint), cast(1001 as float)) | +===================================================== + 1000 | + +taos> select LEAST(cast(1000 as smallint), cast(1001 as double)); + least(cast(1000 as smallint), cast(1001 as double)) | +====================================================== + 1000 | + +taos> select LEAST(cast(1000 as smallint), cast(1001 as varchar(20))); + least(cast(1000 as smallint), cast(1001 as varchar(20))) | +=========================================================== + 1000 | + +taos> select LEAST(cast(1000 as smallint), cast(1001 as nchar(20))); + least(cast(1000 as smallint), cast(1001 as nchar(20))) | +========================================================= + 1000 | + +taos> select LEAST(cast(1001 as smallint), cast(1000 as smallint)); + least(cast(1001 as smallint), cast(1000 as smallint)) | +======================================================== + 1000 | + +taos> select LEAST(cast(1001 as smallint), cast(1000 as int)); + least(cast(1001 as smallint), cast(1000 as int)) | +=================================================== + 1000 | + +taos> select LEAST(cast(1001 as smallint), cast(1000 as bigint)); + least(cast(1001 as smallint), cast(1000 as bigint)) | +====================================================== + 1000 | + +taos> select LEAST(cast(1001 as smallint), cast(1000 as float)); + least(cast(1001 as smallint), cast(1000 as float)) | +===================================================== + 1000 | + +taos> select LEAST(cast(1001 as smallint), cast(1000 as double)); + least(cast(1001 as smallint), cast(1000 as double)) | +====================================================== + 1000 | + +taos> select LEAST(cast(1001 as smallint), cast(1000 as varchar(20))); + least(cast(1001 as smallint), cast(1000 as varchar(20))) | +=========================================================== + 1000 | + +taos> select LEAST(cast(1001 as smallint), cast(1000 as nchar(20))); + least(cast(1001 as smallint), cast(1000 as nchar(20))) | +========================================================= + 1000 | + +taos> select LEAST(cast(1000000 as int), cast(1000001 as int)); + least(cast(1000000 as int), cast(1000001 as int)) | +==================================================== + 1000000 | + +taos> select LEAST(cast(1000000 as int), cast(1000001 as bigint)); + least(cast(1000000 as int), cast(1000001 as bigint)) | +======================================================= + 1000000 | + +taos> select LEAST(cast(1000000 as int), cast(1000001 as float)); + least(cast(1000000 as int), cast(1000001 as float)) | +====================================================== + 1e+06 | + +taos> select LEAST(cast(1000000 as int), cast(1000001 as double)); + least(cast(1000000 as int), cast(1000001 as double)) | +======================================================= + 1000000 | + +taos> select LEAST(cast(1000000 as int), cast(1000001 as varchar(20))); + least(cast(1000000 as int), cast(1000001 as varchar(20))) | +============================================================ + 1000000 | + +taos> select LEAST(cast(1000000 as int), cast(1000001 as nchar(20))); + least(cast(1000000 as int), cast(1000001 as nchar(20))) | +========================================================== + 1000000 | + +taos> select LEAST(cast(1000001 as int), cast(1000000 as int)); + least(cast(1000001 as int), cast(1000000 as int)) | +==================================================== + 1000000 | + +taos> select LEAST(cast(1000001 as int), cast(1000000 as bigint)); + least(cast(1000001 as int), cast(1000000 as bigint)) | +======================================================= + 1000000 | + +taos> select LEAST(cast(1000001 as int), cast(1000000 as float)); + least(cast(1000001 as int), cast(1000000 as float)) | +====================================================== + 1e+06 | + +taos> select LEAST(cast(1000001 as int), cast(1000000 as double)); + least(cast(1000001 as int), cast(1000000 as double)) | +======================================================= + 1000000 | + +taos> select LEAST(cast(1000001 as int), cast(1000000 as varchar(20))); + least(cast(1000001 as int), cast(1000000 as varchar(20))) | +============================================================ + 1000000 | + +taos> select LEAST(cast(1000001 as int), cast(1000000 as nchar(20))); + least(cast(1000001 as int), cast(1000000 as nchar(20))) | +========================================================== + 1000000 | + +taos> select LEAST(cast(1000000000 as bigint), cast(1000000001 as bigint)); + least(cast(1000000000 as bigint), cast(1000000001 as bigint)) | +================================================================ + 1000000000 | + +taos> select LEAST(cast(1000000000 as bigint), cast(1000000001 as float)); + least(cast(1000000000 as bigint), cast(1000000001 as float)) | +=============================================================== + 1e+09 | + +taos> select LEAST(cast(1000000000 as bigint), cast(1000000001 as double)); + least(cast(1000000000 as bigint), cast(1000000001 as double)) | +================================================================ + 1000000000 | + +taos> select LEAST(cast(1000000000 as bigint), cast(1000000001 as varchar(20))); + least(cast(1000000000 as bigint), cast(1000000001 as varchar(20) | +=================================================================== + 1000000000 | + +taos> select LEAST(cast(1000000000 as bigint), cast(1000000001 as nchar(20))); + least(cast(1000000000 as bigint), cast(1000000001 as nchar(20))) | +=================================================================== + 1000000000 | + +taos> select LEAST(cast(1000000001 as bigint), cast(1000000000 as bigint)); + least(cast(1000000001 as bigint), cast(1000000000 as bigint)) | +================================================================ + 1000000000 | + +taos> select LEAST(cast(1000000001 as bigint), cast(1000000000 as float)); + least(cast(1000000001 as bigint), cast(1000000000 as float)) | +=============================================================== + 1e+09 | + +taos> select LEAST(cast(1000000001 as bigint), cast(1000000000 as double)); + least(cast(1000000001 as bigint), cast(1000000000 as double)) | +================================================================ + 1000000000 | + +taos> select LEAST(cast(1000000001 as bigint), cast(1000000000 as varchar(20))); + least(cast(1000000001 as bigint), cast(1000000000 as varchar(20) | +=================================================================== + 1000000000 | + +taos> select LEAST(cast(1000000001 as bigint), cast(1000000000 as nchar(20))); + least(cast(1000000001 as bigint), cast(1000000000 as nchar(20))) | +=================================================================== + 1000000000 | + +taos> select LEAST(cast(100000.1111111 as float), cast(100001.1111111 as float)); + least(cast(100000.1111111 as float), cast(100001.1111111 as floa | +=================================================================== + 100000 | + +taos> select LEAST(cast(100000.1111111 as float), cast(100001.1111111 as double)); + least(cast(100000.1111111 as float), cast(100001.1111111 as doub | +=================================================================== + 100000.109375 | + +taos> select LEAST(cast(100000.1111111 as float), cast(100001.1111111 as timestamp)); + least(cast(100000.1111111 as float), cast(100001.1111111 as time | +=================================================================== + 100000 | + +taos> select LEAST(cast(100000.1111111 as float), cast(100001.1111111 as varchar(20))); + least(cast(100000.1111111 as float), cast(100001.1111111 as varc | +=================================================================== + 100000 | + +taos> select LEAST(cast(100000.1111111 as float), cast(100001.1111111 as nchar(20))); + least(cast(100000.1111111 as float), cast(100001.1111111 as ncha | +=================================================================== + 100000 | + +taos> select LEAST(cast(100001.1111111 as float), cast(100000.1111111 as float)); + least(cast(100001.1111111 as float), cast(100000.1111111 as floa | +=================================================================== + 100000 | + +taos> select LEAST(cast(100001.1111111 as float), cast(100000.1111111 as double)); + least(cast(100001.1111111 as float), cast(100000.1111111 as doub | +=================================================================== + 100000.1111111 | + +taos> select LEAST(cast(100001.1111111 as float), cast(100000.1111111 as timestamp)); + least(cast(100001.1111111 as float), cast(100000.1111111 as time | +=================================================================== + 100000 | + +taos> select LEAST(cast(100001.1111111 as float), cast(100000.1111111 as varchar(20))); + least(cast(100001.1111111 as float), cast(100000.1111111 as varc | +=================================================================== + 100000 | + +taos> select LEAST(cast(100001.1111111 as float), cast(100000.1111111 as nchar(20))); + least(cast(100001.1111111 as float), cast(100000.1111111 as ncha | +=================================================================== + 100000 | + +taos> select LEAST(cast(100000.1111111 as double), cast(100001.1111111 as double)); + least(cast(100000.1111111 as double), cast(100001.1111111 as dou | +=================================================================== + 100000.1111111 | + +taos> select LEAST(cast(100000.1111111 as double), cast(100001.1111111 as timestamp)); + least(cast(100000.1111111 as double), cast(100001.1111111 as tim | +=================================================================== + 100000.1111111 | + +taos> select LEAST(cast(100000.1111111 as double), cast(100001.1111111 as varchar(20))); + least(cast(100000.1111111 as double), cast(100001.1111111 as var | +=================================================================== + 100000.1111111 | + +taos> select LEAST(cast(100000.1111111 as double), cast(100001.1111111 as nchar(20))); + least(cast(100000.1111111 as double), cast(100001.1111111 as nch | +=================================================================== + 100000.1111111 | + +taos> select LEAST(cast(100001.1111111 as double), cast(100000.1111111 as double)); + least(cast(100001.1111111 as double), cast(100000.1111111 as dou | +=================================================================== + 100000.1111111 | + +taos> select LEAST(cast(100001.1111111 as double), cast(100000.1111111 as timestamp)); + least(cast(100001.1111111 as double), cast(100000.1111111 as tim | +=================================================================== + 100000 | + +taos> select LEAST(cast(100001.1111111 as double), cast(100000.1111111 as varchar(20))); + least(cast(100001.1111111 as double), cast(100000.1111111 as var | +=================================================================== + 100000.1111111 | + +taos> select LEAST(cast(100001.1111111 as double), cast(100000.1111111 as nchar(20))); + least(cast(100001.1111111 as double), cast(100000.1111111 as nch | +=================================================================== + 100000.1111111 | + +taos> select LEAST(cast('中文测试' as varchar(20)), cast('中文测试一' as varchar(20))); + least(cast('中文测试' as varchar(20)), cast('中文测试一 | +=================================================================== + 中文测试 | + +taos> select LEAST(cast('中文测试' as varchar(20)), cast('中文测试一' as nchar(20))); + least(cast('中文测试' as varchar(20)), cast('中文测试一 | +=================================================================== + 中文测试 | + +taos> select LEAST(cast('中文测试一' as varchar(20)), cast('中文测试' as varchar(20))); + least(cast('中文测试一' as varchar(20)), cast('中文测试 | +=================================================================== + 中文测试 | + +taos> select LEAST(cast('中文测试一' as varchar(20)), cast('中文测试' as nchar(20))); + least(cast('中文测试一' as varchar(20)), cast('中文测试 | +=================================================================== + 中文测试 | + +taos> select LEAST(cast('abc123abc' as varchar(20)), cast('abc124abc' as varchar(20))); + least(cast('abc123abc' as varchar(20)), cast('abc124abc' as varc | +=================================================================== + abc123abc | + +taos> select LEAST(cast('abc123abc' as varchar(20)), cast('abc124abc' as nchar(20))); + least(cast('abc123abc' as varchar(20)), cast('abc124abc' as ncha | +=================================================================== + abc123abc | + +taos> select LEAST(cast('abc124abc' as varchar(20)), cast('abc123abc' as varchar(20))); + least(cast('abc124abc' as varchar(20)), cast('abc123abc' as varc | +=================================================================== + abc123abc | + +taos> select LEAST(cast('abc124abc' as varchar(20)), cast('abc123abc' as nchar(20))); + least(cast('abc124abc' as varchar(20)), cast('abc123abc' as ncha | +=================================================================== + abc123abc | + +taos> select LEAST(cast('abc123abc' as nchar(20)), cast('abc124abc' as nchar(20))); + least(cast('abc123abc' as nchar(20)), cast('abc124abc' as nchar( | +=================================================================== + abc123abc | + +taos> select LEAST(cast(100 as tinyint), cast(101 as float), cast(102 as varchar(20))); + least(cast(100 as tinyint), cast(101 as float), cast(102 as varc | +=================================================================== + 100 | + +taos> select LEAST(cast(100 as varchar(20)), cast(101 as float), cast(102 as tinyint)); + least(cast(100 as varchar(20)), cast(101 as float), cast(102 as | +=================================================================== + 100 | + +taos> select LEAST('a','b','c','d','e','f','g','h','1231','15155'); + least('a','b','c','d','e','f','g','h','1231','15155') | +======================================================== + 1231 | + +taos> select LEAST(current, voltage, phase, id, nch1, nch2, var1, var2) from ts_4893.meters order by ts limit 10; + least(current, voltage, phase, id, nch1, nch2, var1, var2) | +============================================================= + 0 | + 0 | + 0 | + 0 | + 0 | + 0 | + 0 | + 0 | + 0 | + 0 | + +taos> select LEAST(current, voltage, phase, id) from ts_4893.meters order by ts limit 10; + least(current, voltage, phase, id) | +===================================== + 0 | + 0.13883 | + 0.796942 | + 0.53733 | + 0.31343 | + 0.332767 | + 0.846763 | + 0.637813 | + 0.115989 | + 0.373575 | + +taos> select LEAST(nch1, nch2, var1, var2) from ts_4893.meters order by ts limit 10; + least(nch1, nch2, var1, var2) | +================================= + e | + f | + c | + e | + b | + g | + again | + b | + c | + again | + +taos> select LEAST(221, voltage) from ts_4893.meters order by ts limit 10; + least(221, voltage) | +======================== + 221 | + 220 | + 215 | + 216 | + 219 | + 221 | + 215 | + 217 | + 216 | + 221 | + +taos> select LEAST(5, id) from ts_4893.meters order by ts limit 10; + least(5, id) | +======================== + 0 | + 1 | + 2 | + 3 | + 4 | + 5 | + 5 | + 5 | + 5 | + 5 | + +taos> select LEAST('r', nch1) from ts_4893.meters order by ts limit 10; + least('r', nch1) | +================================= + novel | + r | + r | + prision | + novel | + novel | + again | + r | + novel | + again | + +taos> select LEAST('r', nch1, nch2) from ts_4893.meters order by ts limit 10; + least('r', nch1, nch2) | +================================= + novel | + f | + c | + prision | + e | + novel | + again | + d | + c | + again | + +taos> select LEAST('r', var1) from ts_4893.meters order by ts limit 10; + least('r', var1) | +================================= + novel | + person | + novel | + plate | + r | + r | + prision | + person | + prision | + plate | + +taos> select LEAST('r', var1, var2) from ts_4893.meters order by ts limit 10; + least('r', var1, var2) | +================================= + e | + person | + novel | + e | + b | + g | + b | + b | + e | + plate | + +taos> select LEAST('二中文测试', nch1) from ts_4893.meters order by ts limit 10; + least('二中文测试', nch1) | +================================= + novel | + 一二三四五六七八九十 | + update | + prision | + novel | + novel | + again | + 一二三四五六七八九十 | + novel | + again | + +taos> select LEAST('二中文测试', nch1, nch2) from ts_4893.meters order by ts limit 10; + least('二中文测试', nch1, nch2) | +======================================= + novel | + f | + c | + prision | + e | + novel | + again | + d | + c | + again | + +taos> select LEAST('二中文测试', var1) from ts_4893.meters order by ts limit 10; + least('二中文测试', var1) | +================================= + novel | + person | + novel | + plate | + 一二三四五六七八九十 | + update | + prision | + person | + prision | + plate | + +taos> select LEAST('二中文测试', var1, var2) from ts_4893.meters order by ts limit 10; + least('二中文测试', var1, var2) | +======================================= + e | + person | + novel | + e | + b | + g | + b | + b | + e | + plate | + +taos> select LEAST('23', 3443434343434343); + least('23', 3443434343434343) | +================================ + 23 | + +taos> select LEAST(co, 3443434343434343) from ts_4893.n1; + least(co, 3443434343434343) | +============================== + 23 | + 23 | + 23 | + +taos> select LEAST('23', 3443434343434343) from ts_4893.n1; + least('23', 3443434343434343) | +================================ + 23 | + 23 | + 23 | + +taos> select LEAST('23', 1443434343434343) from ts_4893.n1; + least('23', 1443434343434343) | +================================ + 23 | + 23 | + 23 | + +taos> select LEAST(current, voltage) from ts_4893.n1; + least(current, voltage) | +========================== + NULL | + NULL | + 3 | + +taos> select LEAST(current, voltage, '15') from ts_4893.n1; + least(current, voltage, '15') | +================================ + NULL | + NULL | + 3 | + diff --git a/tests/army/query/function/ans/mod.csv b/tests/army/query/function/ans/mod.csv index 05770289db..c222257308 100644 --- a/tests/army/query/function/ans/mod.csv +++ b/tests/army/query/function/ans/mod.csv @@ -42,12 +42,12 @@ taos> select MOD(10.55, 1) + 1 taos> select MOD(MOD(MOD(MOD(MOD(MOD(MOD(123.123456789, 9), 8), 7), 6), 5), 4), 3) mod(mod(mod(mod(mod(mod(mod(123.123456789, 9), 8), 7), 6), 5), 4 | =================================================================== - 0.12345678900000 | + 0.123456789000002 | taos> select MOD(MOD(MOD(MOD(MOD(MOD(MOD(123456789.123456789, -1), -2), -3), -4), -5), -6), -7) mod(mod(mod(mod(mod(mod(mod(123456789.123456789, -1), -2), -3), | =================================================================== - 0.12345679104328 | + 0.123456791043282 | taos> select MOD(87654321.123456789, id + 1) from ts_4893.meters order by ts limit 10 mod(87654321.123456789, id + 1) | diff --git a/tests/army/query/function/ans/round.csv b/tests/army/query/function/ans/round.csv index 023c1fd43d..4ae3a85376 100644 --- a/tests/army/query/function/ans/round.csv +++ b/tests/army/query/function/ans/round.csv @@ -67,12 +67,12 @@ taos> select ROUND(10.55, 1) + 1 taos> select ROUND(ROUND(ROUND(ROUND(ROUND(ROUND(ROUND(123.123456789, 9), 8), 7), 6), 5), 4)) round(round(round(round(round(round(round(123.123456789, 9), 8), | =================================================================== - 12 | + 123 | taos> select ROUND(ROUND(ROUND(ROUND(ROUND(ROUND(ROUND(123456789.123456789, -1), -2), -3), -4), -5), -6)) round(round(round(round(round(round(round(123456789.123456789, - | =================================================================== - 12300000 | + 123000000 | taos> select ROUND(current) from ts_4893.meters order by ts limit 20 round(current) | diff --git a/tests/army/query/function/ans/trunc.csv b/tests/army/query/function/ans/trunc.csv index 77b6127cba..8d1316b760 100644 --- a/tests/army/query/function/ans/trunc.csv +++ b/tests/army/query/function/ans/trunc.csv @@ -47,12 +47,12 @@ taos> select TRUNCATE(10.55, 1) + 1 taos> select TRUNCATE(TRUNCATE(TRUNCATE(TRUNCATE(TRUNCATE(TRUNCATE(TRUNCATE(123.123456789, 9), 8), 7), 6), 5), 4), 3) truncate(truncate(truncate(truncate(truncate(truncate(truncate(1 | =================================================================== - 123.12 | + 123.123 | taos> select TRUNCATE(TRUNCATE(TRUNCATE(TRUNCATE(TRUNCATE(TRUNCATE(TRUNCATE(123456789.123456789, -1), -2), -3), -4), -5), -6), -7) truncate(truncate(truncate(truncate(truncate(truncate(truncate(1 | =================================================================== - 12000000 | + 120000000 | taos> select TRUNCATE(87654321.123456789, id) from ts_4893.meters order by ts limit 10 truncate(87654321.123456789, id) | @@ -144,12 +144,12 @@ taos> select TRUNC(10.55, 1) + 1 taos> select TRUNC(TRUNC(TRUNC(TRUNC(TRUNC(TRUNC(TRUNC(123.123456789, 9), 8), 7), 6), 5), 4), 3) trunc(trunc(trunc(trunc(trunc(trunc(trunc(123.123456789, 9), 8), | =================================================================== - 123.12 | + 123.123 | taos> select TRUNC(TRUNC(TRUNC(TRUNC(TRUNC(TRUNC(TRUNC(123456789.123456789, -1), -2), -3), -4), -5), -6), -7) trunc(trunc(trunc(trunc(trunc(trunc(trunc(123456789.123456789, - | =================================================================== - 12000000 | + 120000000 | taos> select TRUNC(87654321.123456789, id) from ts_4893.meters order by ts limit 10 trunc(87654321.123456789, id) | diff --git a/tests/army/query/function/in/greatest.in b/tests/army/query/function/in/greatest.in new file mode 100644 index 0000000000..1ae8bee06b --- /dev/null +++ b/tests/army/query/function/in/greatest.in @@ -0,0 +1,232 @@ +alter local 'transToStrWhenMixTypeInLeast' '1'; +select GREATEST(1,2,3,4,5,6,7,8,9,10); +select GREATEST(1,1.1,2.23,3.4444,5.66666666,6.21241241,7.999999999999); +select GREATEST(1,'2',3.3,4.4,5); +select GREATEST(121,'18'); +select GREATEST(18888,'18'); +select GREATEST(1,2,3,4,5,'5.1'); +select GREATEST('1','2','3','4',5); +select GREATEST('1','2','3','4','5'); +select GREATEST(1,2,3,4,5,6,7,'a','b','一','二','三'); +select GREATEST(1,2,3,4,5,6,7,'a','b','c','1','2','1231213'); +select GREATEST(1,2,3,4,5,6,7,'a','b','c','1','2','1231213','1231213.123123'); +select GREATEST(cast(0 as bool), cast(1 as tinyint), cast(2 as smallint), cast(3 as int), cast(4 as bigint), cast(5 as float), cast(6 as double), cast(8 as varchar(20)), cast(9 as nchar(20))); +select GREATEST(cast(0 as bool), cast(1 as tinyint unsigned), cast(2 as smallint unsigned), cast(3 as int unsigned), cast(4 as bigint unsigned), cast(5 as float), cast(6 as double), cast(8 as varchar(20)), cast(9 as nchar(20))); +select GREATEST(cast(0 as bool), cast(1 as tinyint), cast(2 as smallint), cast(3 as int), cast(4 as bigint), cast(5 as float), cast(6 as double), cast(8 as varchar(20)), cast(9 as nchar(20)), cast(0 as bool), cast(1 as tinyint unsigned), cast(2 as smallint unsigned), cast(3 as int unsigned), cast(4 as bigint unsigned), cast(5 as float), cast(6 as double)); +select GREATEST(cast(100 as tinyint), cast(101 as tinyint)); +select GREATEST(cast(100 as tinyint), cast(101 as smallint)); +select GREATEST(cast(100 as tinyint), cast(101 as int)); +select GREATEST(cast(100 as tinyint), cast(101 as bigint)); +select GREATEST(cast(100 as tinyint), cast(101 as float)); +select GREATEST(cast(100 as tinyint), cast(101 as double)); +select GREATEST(cast(100 as tinyint), cast(101 as varchar(20))); +select GREATEST(cast(100 as tinyint), cast(101 as nchar(20))); +select GREATEST(cast(101 as tinyint), cast(100 as tinyint)); +select GREATEST(cast(101 as tinyint), cast(100 as smallint)); +select GREATEST(cast(101 as tinyint), cast(100 as int)); +select GREATEST(cast(101 as tinyint), cast(100 as bigint)); +select GREATEST(cast(101 as tinyint), cast(100 as float)); +select GREATEST(cast(101 as tinyint), cast(100 as double)); +select GREATEST(cast(101 as tinyint), cast(100 as varchar(20))); +select GREATEST(cast(101 as tinyint), cast(100 as nchar(20))); +select GREATEST(cast(1000 as smallint), cast(1001 as smallint)); +select GREATEST(cast(1000 as smallint), cast(1001 as int)); +select GREATEST(cast(1000 as smallint), cast(1001 as bigint)); +select GREATEST(cast(1000 as smallint), cast(1001 as float)); +select GREATEST(cast(1000 as smallint), cast(1001 as double)); +select GREATEST(cast(1000 as smallint), cast(1001 as varchar(20))); +select GREATEST(cast(1000 as smallint), cast(1001 as nchar(20))); +select GREATEST(cast(1001 as smallint), cast(1000 as smallint)); +select GREATEST(cast(1001 as smallint), cast(1000 as int)); +select GREATEST(cast(1001 as smallint), cast(1000 as bigint)); +select GREATEST(cast(1001 as smallint), cast(1000 as float)); +select GREATEST(cast(1001 as smallint), cast(1000 as double)); +select GREATEST(cast(1001 as smallint), cast(1000 as varchar(20))); +select GREATEST(cast(1001 as smallint), cast(1000 as nchar(20))); +select GREATEST(cast(1000000 as int), cast(1000001 as int)); +select GREATEST(cast(1000000 as int), cast(1000001 as bigint)); +select GREATEST(cast(1000000 as int), cast(1000001 as float)); +select GREATEST(cast(1000000 as int), cast(1000001 as double)); +select GREATEST(cast(1000000 as int), cast(1000001 as varchar(20))); +select GREATEST(cast(1000000 as int), cast(1000001 as nchar(20))); +select GREATEST(cast(1000001 as int), cast(1000000 as int)); +select GREATEST(cast(1000001 as int), cast(1000000 as bigint)); +select GREATEST(cast(1000001 as int), cast(1000000 as float)); +select GREATEST(cast(1000001 as int), cast(1000000 as double)); +select GREATEST(cast(1000001 as int), cast(1000000 as varchar(20))); +select GREATEST(cast(1000001 as int), cast(1000000 as nchar(20))); +select GREATEST(cast(1000000000 as bigint), cast(1000000001 as bigint)); +select GREATEST(cast(1000000000 as bigint), cast(1000000001 as float)); +select GREATEST(cast(1000000000 as bigint), cast(1000000001 as double)); +select GREATEST(cast(1000000000 as bigint), cast(1000000001 as varchar(20))); +select GREATEST(cast(1000000000 as bigint), cast(1000000001 as nchar(20))); +select GREATEST(cast(1000000001 as bigint), cast(1000000000 as bigint)); +select GREATEST(cast(1000000001 as bigint), cast(1000000000 as float)); +select GREATEST(cast(1000000001 as bigint), cast(1000000000 as double)); +select GREATEST(cast(1000000001 as bigint), cast(1000000000 as varchar(20))); +select GREATEST(cast(1000000001 as bigint), cast(1000000000 as nchar(20))); +select GREATEST(cast(100000.1111111 as float), cast(100001.1111111 as float)); +select GREATEST(cast(100000.1111111 as float), cast(100001.1111111 as double)); +select GREATEST(cast(100000.1111111 as float), cast(100001.1111111 as timestamp)); +select GREATEST(cast(100000.1111111 as float), cast(100001.1111111 as varchar(20))); +select GREATEST(cast(100000.1111111 as float), cast(100001.1111111 as nchar(20))); +select GREATEST(cast(100001.1111111 as float), cast(100000.1111111 as float)); +select GREATEST(cast(100001.1111111 as float), cast(100000.1111111 as double)); +select GREATEST(cast(100001.1111111 as float), cast(100000.1111111 as timestamp)); +select GREATEST(cast(100001.1111111 as float), cast(100000.1111111 as varchar(20))); +select GREATEST(cast(100001.1111111 as float), cast(100000.1111111 as nchar(20))); +select GREATEST(cast(100000.1111111 as double), cast(100001.1111111 as double)); +select GREATEST(cast(100000.1111111 as double), cast(100001.1111111 as timestamp)); +select GREATEST(cast(100000.1111111 as double), cast(100001.1111111 as varchar(20))); +select GREATEST(cast(100000.1111111 as double), cast(100001.1111111 as nchar(20))); +select GREATEST(cast(100001.1111111 as double), cast(100000.1111111 as double)); +select GREATEST(cast(100001.1111111 as double), cast(100000.1111111 as timestamp)); +select GREATEST(cast(100001.1111111 as double), cast(100000.1111111 as varchar(20))); +select GREATEST(cast(100001.1111111 as double), cast(100000.1111111 as nchar(20))); +select GREATEST(cast('中文测试' as varchar(20)), cast('中文测试一' as varchar(20))); +select GREATEST(cast('中文测试' as varchar(20)), cast('中文测试一' as nchar(20))); +select GREATEST(cast('中文测试一' as varchar(20)), cast('中文测试' as varchar(20))); +select GREATEST(cast('中文测试一' as varchar(20)), cast('中文测试' as nchar(20))); +select GREATEST(cast('abc123abc' as varchar(20)), cast('abc124abc' as varchar(20))); +select GREATEST(cast('abc123abc' as varchar(20)), cast('abc124abc' as nchar(20))); +select GREATEST(cast('abc124abc' as varchar(20)), cast('abc123abc' as varchar(20))); +select GREATEST(cast('abc124abc' as varchar(20)), cast('abc123abc' as nchar(20))); +select GREATEST(cast('abc123abc' as nchar(20)), cast('abc124abc' as nchar(20))); +select GREATEST(cast(100 as tinyint), cast(101 as float), cast(102 as varchar(20))); +select GREATEST(cast(100 as float), cast(101 as tinyint), cast(102 as varchar(20))); +select GREATEST(cast(100 as float), cast(101 as varchar(20)), cast(102 as tinyint)); +select GREATEST(cast(100 as varchar(20)), cast(101 as float), cast(102 as tinyint)); +select GREATEST('a','b','c','d','e','f','g','h','1231','15155'); +select GREATEST(current, voltage, phase, id, nch1, nch2, var1, var2) from ts_4893.meters order by ts limit 10; +select GREATEST(current, voltage, phase, id) from ts_4893.meters order by ts limit 10; +select GREATEST(nch1, nch2, var1, var2) from ts_4893.meters order by ts limit 10; +select GREATEST(221, voltage) from ts_4893.meters order by ts limit 10; +select GREATEST(5, id) from ts_4893.meters order by ts limit 10; +select GREATEST('r', nch1) from ts_4893.meters order by ts limit 10; +select GREATEST('r', nch1, nch2) from ts_4893.meters order by ts limit 10; +select GREATEST('r', var1) from ts_4893.meters order by ts limit 10; +select GREATEST('r', var1, var2) from ts_4893.meters order by ts limit 10; +select GREATEST('二中文测试', nch1) from ts_4893.meters order by ts limit 10; +select GREATEST('二中文测试', nch1, nch2) from ts_4893.meters order by ts limit 10; +select GREATEST('二中文测试', var1) from ts_4893.meters order by ts limit 10; +select GREATEST('二中文测试', var1, var2) from ts_4893.meters order by ts limit 10; +select GREATEST('23', 3443434343434343); +select GREATEST(co, 3443434343434343) from ts_4893.n1; +select GREATEST('23', 3443434343434343) from ts_4893.n1; +select GREATEST('23', 1443434343434343) from ts_4893.n1; +select GREATEST(current, voltage) from ts_4893.n1; +select GREATEST(current, voltage, '15') from ts_4893.n1; +alter local 'transToStrWhenMixTypeInLeast' '0'; +select GREATEST(1,'2',3.3,4.4,5); +select GREATEST(1,2,3,4,5,'5.1'); +select GREATEST(121,'18'); +select GREATEST('1','2','3','4','5'); +select GREATEST(1,2,3,4,5,6,7,'a','b','一','二','三'); +select GREATEST(1,2,3,4,5,6,7,'a','b','c','1','2','1231213'); +select GREATEST(1,2,3,4,5,6,7,'a','b','c','1','2','1231213','1231213.123123'); +select GREATEST(cast(0 as bool), cast(1 as tinyint), cast(2 as smallint), cast(3 as int), cast(4 as bigint), cast(5 as float), cast(6 as double), cast(8 as varchar(20)), cast(9 as nchar(20))); +select GREATEST(cast(0 as bool), cast(1 as tinyint unsigned), cast(2 as smallint unsigned), cast(3 as int unsigned), cast(4 as bigint unsigned), cast(5 as float), cast(6 as double), cast(8 as varchar(20)), cast(9 as nchar(20))); +select GREATEST(cast(0 as bool), cast(1 as tinyint), cast(2 as smallint), cast(3 as int), cast(4 as bigint), cast(5 as float), cast(6 as double), cast(8 as varchar(20)), cast(9 as nchar(20)), cast(0 as bool), cast(1 as tinyint unsigned), cast(2 as smallint unsigned), cast(3 as int unsigned), cast(4 as bigint unsigned), cast(5 as float), cast(6 as double)); +select GREATEST(cast(100 as tinyint), cast(101 as tinyint)); +select GREATEST(cast(100 as tinyint), cast(101 as smallint)); +select GREATEST(cast(100 as tinyint), cast(101 as int)); +select GREATEST(cast(100 as tinyint), cast(101 as bigint)); +select GREATEST(cast(100 as tinyint), cast(101 as float)); +select GREATEST(cast(100 as tinyint), cast(101 as double)); +select GREATEST(cast(100 as tinyint), cast(101 as varchar(20))); +select GREATEST(cast(100 as tinyint), cast(101 as nchar(20))); +select GREATEST(cast(101 as tinyint), cast(100 as tinyint)); +select GREATEST(cast(101 as tinyint), cast(100 as smallint)); +select GREATEST(cast(101 as tinyint), cast(100 as int)); +select GREATEST(cast(101 as tinyint), cast(100 as bigint)); +select GREATEST(cast(101 as tinyint), cast(100 as float)); +select GREATEST(cast(101 as tinyint), cast(100 as double)); +select GREATEST(cast(101 as tinyint), cast(100 as varchar(20))); +select GREATEST(cast(101 as tinyint), cast(100 as nchar(20))); +select GREATEST(cast(1000 as smallint), cast(1001 as smallint)); +select GREATEST(cast(1000 as smallint), cast(1001 as int)); +select GREATEST(cast(1000 as smallint), cast(1001 as bigint)); +select GREATEST(cast(1000 as smallint), cast(1001 as float)); +select GREATEST(cast(1000 as smallint), cast(1001 as double)); +select GREATEST(cast(1000 as smallint), cast(1001 as varchar(20))); +select GREATEST(cast(1000 as smallint), cast(1001 as nchar(20))); +select GREATEST(cast(1001 as smallint), cast(1000 as smallint)); +select GREATEST(cast(1001 as smallint), cast(1000 as int)); +select GREATEST(cast(1001 as smallint), cast(1000 as bigint)); +select GREATEST(cast(1001 as smallint), cast(1000 as float)); +select GREATEST(cast(1001 as smallint), cast(1000 as double)); +select GREATEST(cast(1001 as smallint), cast(1000 as varchar(20))); +select GREATEST(cast(1001 as smallint), cast(1000 as nchar(20))); +select GREATEST(cast(1000000 as int), cast(1000001 as int)); +select GREATEST(cast(1000000 as int), cast(1000001 as bigint)); +select GREATEST(cast(1000000 as int), cast(1000001 as float)); +select GREATEST(cast(1000000 as int), cast(1000001 as double)); +select GREATEST(cast(1000000 as int), cast(1000001 as varchar(20))); +select GREATEST(cast(1000000 as int), cast(1000001 as nchar(20))); +select GREATEST(cast(1000001 as int), cast(1000000 as int)); +select GREATEST(cast(1000001 as int), cast(1000000 as bigint)); +select GREATEST(cast(1000001 as int), cast(1000000 as float)); +select GREATEST(cast(1000001 as int), cast(1000000 as double)); +select GREATEST(cast(1000001 as int), cast(1000000 as varchar(20))); +select GREATEST(cast(1000001 as int), cast(1000000 as nchar(20))); +select GREATEST(cast(1000000000 as bigint), cast(1000000001 as bigint)); +select GREATEST(cast(1000000000 as bigint), cast(1000000001 as float)); +select GREATEST(cast(1000000000 as bigint), cast(1000000001 as double)); +select GREATEST(cast(1000000000 as bigint), cast(1000000001 as varchar(20))); +select GREATEST(cast(1000000000 as bigint), cast(1000000001 as nchar(20))); +select GREATEST(cast(1000000001 as bigint), cast(1000000000 as bigint)); +select GREATEST(cast(1000000001 as bigint), cast(1000000000 as float)); +select GREATEST(cast(1000000001 as bigint), cast(1000000000 as double)); +select GREATEST(cast(1000000001 as bigint), cast(1000000000 as varchar(20))); +select GREATEST(cast(1000000001 as bigint), cast(1000000000 as nchar(20))); +select GREATEST(cast(100000.1111111 as float), cast(100001.1111111 as float)); +select GREATEST(cast(100000.1111111 as float), cast(100001.1111111 as double)); +select GREATEST(cast(100000.1111111 as float), cast(100001.1111111 as timestamp)); +select GREATEST(cast(100000.1111111 as float), cast(100001.1111111 as varchar(20))); +select GREATEST(cast(100000.1111111 as float), cast(100001.1111111 as nchar(20))); +select GREATEST(cast(100001.1111111 as float), cast(100000.1111111 as float)); +select GREATEST(cast(100001.1111111 as float), cast(100000.1111111 as double)); +select GREATEST(cast(100001.1111111 as float), cast(100000.1111111 as timestamp)); +select GREATEST(cast(100001.1111111 as float), cast(100000.1111111 as varchar(20))); +select GREATEST(cast(100001.1111111 as float), cast(100000.1111111 as nchar(20))); +select GREATEST(cast(100000.1111111 as double), cast(100001.1111111 as double)); +select GREATEST(cast(100000.1111111 as double), cast(100001.1111111 as timestamp)); +select GREATEST(cast(100000.1111111 as double), cast(100001.1111111 as varchar(20))); +select GREATEST(cast(100000.1111111 as double), cast(100001.1111111 as nchar(20))); +select GREATEST(cast(100001.1111111 as double), cast(100000.1111111 as double)); +select GREATEST(cast(100001.1111111 as double), cast(100000.1111111 as timestamp)); +select GREATEST(cast(100001.1111111 as double), cast(100000.1111111 as varchar(20))); +select GREATEST(cast(100001.1111111 as double), cast(100000.1111111 as nchar(20))); +select GREATEST(cast('中文测试' as varchar(20)), cast('中文测试一' as varchar(20))); +select GREATEST(cast('中文测试' as varchar(20)), cast('中文测试一' as nchar(20))); +select GREATEST(cast('中文测试一' as varchar(20)), cast('中文测试' as varchar(20))); +select GREATEST(cast('中文测试一' as varchar(20)), cast('中文测试' as nchar(20))); +select GREATEST(cast('abc123abc' as varchar(20)), cast('abc124abc' as varchar(20))); +select GREATEST(cast('abc123abc' as varchar(20)), cast('abc124abc' as nchar(20))); +select GREATEST(cast('abc124abc' as varchar(20)), cast('abc123abc' as varchar(20))); +select GREATEST(cast('abc124abc' as varchar(20)), cast('abc123abc' as nchar(20))); +select GREATEST(cast('abc123abc' as nchar(20)), cast('abc124abc' as nchar(20))); +select GREATEST(cast(100 as tinyint), cast(101 as float), cast(102 as varchar(20))); +select GREATEST(cast(100 as float), cast(101 as tinyint), cast(102 as varchar(20))); +select GREATEST(cast(100 as float), cast(101 as varchar(20)), cast(102 as tinyint)); +select GREATEST(cast(100 as varchar(20)), cast(101 as float), cast(102 as tinyint)); +select GREATEST('a','b','c','d','e','f','g','h','1231','15155'); +select GREATEST(current, voltage, phase, id, nch1, nch2, var1, var2) from ts_4893.meters order by ts limit 10; +select GREATEST(current, voltage, phase, id) from ts_4893.meters order by ts limit 10; +select GREATEST(nch1, nch2, var1, var2) from ts_4893.meters order by ts limit 10; +select GREATEST(221, voltage) from ts_4893.meters order by ts limit 10; +select GREATEST(5, id) from ts_4893.meters order by ts limit 10; +select GREATEST('r', nch1) from ts_4893.meters order by ts limit 10; +select GREATEST('r', nch1, nch2) from ts_4893.meters order by ts limit 10; +select GREATEST('r', var1) from ts_4893.meters order by ts limit 10; +select GREATEST('r', var1, var2) from ts_4893.meters order by ts limit 10; +select GREATEST('二中文测试', nch1) from ts_4893.meters order by ts limit 10; +select GREATEST('二中文测试', nch1, nch2) from ts_4893.meters order by ts limit 10; +select GREATEST('二中文测试', var1) from ts_4893.meters order by ts limit 10; +select GREATEST('二中文测试', var1, var2) from ts_4893.meters order by ts limit 10; +select GREATEST('23', 3443434343434343); +select GREATEST(co, 3443434343434343) from ts_4893.n1; +select GREATEST('23', 1443434343434343) from ts_4893.n1; +select GREATEST('23', 3443434343434343) from ts_4893.n1 +select GREATEST(current, voltage) from ts_4893.n1; +select GREATEST(current, voltage, '15') from ts_4893.n1; \ No newline at end of file diff --git a/tests/army/query/function/in/least.in b/tests/army/query/function/in/least.in new file mode 100644 index 0000000000..1dd664824d --- /dev/null +++ b/tests/army/query/function/in/least.in @@ -0,0 +1,228 @@ +alter local 'transToStrWhenMixTypeInLeast' '1'; +select LEAST(1,2,3,4,5,6,7,8,9,10); +select LEAST(1,1.1,2.23,3.4444,5.66666666,6.21241241,7.999999999999); +select LEAST(1,'2',3.3,4.4,5); +select LEAST(1,2,3,4,5,'5.1'); +select LEAST('1','2','3','4',5); +select LEAST('1','2','3','4','5'); +select LEAST(1,2,3,4,5,6,7,'a','b','一','二','三'); +select LEAST(1,2,3,4,5,6,7,'a','b','c','1','2','1231213'); +select LEAST(1,2,3,4,5,6,7,'a','b','c','1','2','1231213','1231213.123123'); +select LEAST(cast(0 as bool), cast(1 as tinyint), cast(2 as smallint), cast(3 as int), cast(4 as bigint), cast(5 as float), cast(6 as double), cast(8 as varchar(20)), cast(9 as nchar(20))); +select LEAST(cast(0 as bool), cast(1 as tinyint unsigned), cast(2 as smallint unsigned), cast(3 as int unsigned), cast(4 as bigint unsigned), cast(5 as float), cast(6 as double), cast(8 as varchar(20)), cast(9 as nchar(20))); +select LEAST(cast(0 as bool), cast(1 as tinyint), cast(2 as smallint), cast(3 as int), cast(4 as bigint), cast(5 as float), cast(6 as double), cast(8 as varchar(20)), cast(9 as nchar(20)), cast(0 as bool), cast(1 as tinyint unsigned), cast(2 as smallint unsigned), cast(3 as int unsigned), cast(4 as bigint unsigned), cast(5 as float), cast(6 as double)); +select LEAST(cast(100 as tinyint), cast(101 as tinyint)); +select LEAST(cast(100 as tinyint), cast(101 as smallint)); +select LEAST(cast(100 as tinyint), cast(101 as int)); +select LEAST(cast(100 as tinyint), cast(101 as bigint)); +select LEAST(cast(100 as tinyint), cast(101 as float)); +select LEAST(cast(100 as tinyint), cast(101 as double)); +select LEAST(cast(100 as tinyint), cast(101 as varchar(20))); +select LEAST(cast(100 as tinyint), cast(101 as nchar(20))); +select LEAST(cast(101 as tinyint), cast(100 as tinyint)); +select LEAST(cast(101 as tinyint), cast(100 as smallint)); +select LEAST(cast(101 as tinyint), cast(100 as int)); +select LEAST(cast(101 as tinyint), cast(100 as bigint)); +select LEAST(cast(101 as tinyint), cast(100 as float)); +select LEAST(cast(101 as tinyint), cast(100 as double)); +select LEAST(cast(101 as tinyint), cast(100 as varchar(20))); +select LEAST(cast(101 as tinyint), cast(100 as nchar(20))); +select LEAST(cast(1000 as smallint), cast(1001 as smallint)); +select LEAST(cast(1000 as smallint), cast(1001 as int)); +select LEAST(cast(1000 as smallint), cast(1001 as bigint)); +select LEAST(cast(1000 as smallint), cast(1001 as float)); +select LEAST(cast(1000 as smallint), cast(1001 as double)); +select LEAST(cast(1000 as smallint), cast(1001 as varchar(20))); +select LEAST(cast(1000 as smallint), cast(1001 as nchar(20))); +select LEAST(cast(1001 as smallint), cast(1000 as smallint)); +select LEAST(cast(1001 as smallint), cast(1000 as int)); +select LEAST(cast(1001 as smallint), cast(1000 as bigint)); +select LEAST(cast(1001 as smallint), cast(1000 as float)); +select LEAST(cast(1001 as smallint), cast(1000 as double)); +select LEAST(cast(1001 as smallint), cast(1000 as varchar(20))); +select LEAST(cast(1001 as smallint), cast(1000 as nchar(20))); +select LEAST(cast(1000000 as int), cast(1000001 as int)); +select LEAST(cast(1000000 as int), cast(1000001 as bigint)); +select LEAST(cast(1000000 as int), cast(1000001 as float)); +select LEAST(cast(1000000 as int), cast(1000001 as double)); +select LEAST(cast(1000000 as int), cast(1000001 as varchar(20))); +select LEAST(cast(1000000 as int), cast(1000001 as nchar(20))); +select LEAST(cast(1000001 as int), cast(1000000 as int)); +select LEAST(cast(1000001 as int), cast(1000000 as bigint)); +select LEAST(cast(1000001 as int), cast(1000000 as float)); +select LEAST(cast(1000001 as int), cast(1000000 as double)); +select LEAST(cast(1000001 as int), cast(1000000 as varchar(20))); +select LEAST(cast(1000001 as int), cast(1000000 as nchar(20))); +select LEAST(cast(1000000000 as bigint), cast(1000000001 as bigint)); +select LEAST(cast(1000000000 as bigint), cast(1000000001 as float)); +select LEAST(cast(1000000000 as bigint), cast(1000000001 as double)); +select LEAST(cast(1000000000 as bigint), cast(1000000001 as varchar(20))); +select LEAST(cast(1000000000 as bigint), cast(1000000001 as nchar(20))); +select LEAST(cast(1000000001 as bigint), cast(1000000000 as bigint)); +select LEAST(cast(1000000001 as bigint), cast(1000000000 as float)); +select LEAST(cast(1000000001 as bigint), cast(1000000000 as double)); +select LEAST(cast(1000000001 as bigint), cast(1000000000 as varchar(20))); +select LEAST(cast(1000000001 as bigint), cast(1000000000 as nchar(20))); +select LEAST(cast(100000.1111111 as float), cast(100001.1111111 as float)); +select LEAST(cast(100000.1111111 as float), cast(100001.1111111 as double)); +select LEAST(cast(100000.1111111 as float), cast(100001.1111111 as timestamp)); +select LEAST(cast(100000.1111111 as float), cast(100001.1111111 as varchar(20))); +select LEAST(cast(100000.1111111 as float), cast(100001.1111111 as nchar(20))); +select LEAST(cast(100001.1111111 as float), cast(100000.1111111 as float)); +select LEAST(cast(100001.1111111 as float), cast(100000.1111111 as double)); +select LEAST(cast(100001.1111111 as float), cast(100000.1111111 as timestamp)); +select LEAST(cast(100001.1111111 as float), cast(100000.1111111 as varchar(20))); +select LEAST(cast(100001.1111111 as float), cast(100000.1111111 as nchar(20))); +select LEAST(cast(100000.1111111 as double), cast(100001.1111111 as double)); +select LEAST(cast(100000.1111111 as double), cast(100001.1111111 as timestamp)); +select LEAST(cast(100000.1111111 as double), cast(100001.1111111 as varchar(20))); +select LEAST(cast(100000.1111111 as double), cast(100001.1111111 as nchar(20))); +select LEAST(cast(100001.1111111 as double), cast(100000.1111111 as double)); +select LEAST(cast(100001.1111111 as double), cast(100000.1111111 as timestamp)); +select LEAST(cast(100001.1111111 as double), cast(100000.1111111 as varchar(20))); +select LEAST(cast(100001.1111111 as double), cast(100000.1111111 as nchar(20))); +select LEAST(cast('中文测试' as varchar(20)), cast('中文测试一' as varchar(20))); +select LEAST(cast('中文测试' as varchar(20)), cast('中文测试一' as nchar(20))); +select LEAST(cast('中文测试一' as varchar(20)), cast('中文测试' as varchar(20))); +select LEAST(cast('中文测试一' as varchar(20)), cast('中文测试' as nchar(20))); +select LEAST(cast('abc123abc' as varchar(20)), cast('abc124abc' as varchar(20))); +select LEAST(cast('abc123abc' as varchar(20)), cast('abc124abc' as nchar(20))); +select LEAST(cast('abc124abc' as varchar(20)), cast('abc123abc' as varchar(20))); +select LEAST(cast('abc124abc' as varchar(20)), cast('abc123abc' as nchar(20))); +select LEAST(cast('abc123abc' as nchar(20)), cast('abc124abc' as nchar(20))); +select LEAST(cast(100 as tinyint), cast(101 as float), cast(102 as varchar(20))); +select LEAST(cast(100 as varchar(20)), cast(101 as float), cast(102 as tinyint)); +select LEAST('a','b','c','d','e','f','g','h','1231','15155'); +select LEAST(current, voltage, phase, id, nch1, nch2, var1, var2) from ts_4893.meters order by ts limit 10; +select LEAST(current, voltage, phase, id) from ts_4893.meters order by ts limit 10; +select LEAST(nch1, nch2, var1, var2) from ts_4893.meters order by ts limit 10; +select LEAST(221, voltage) from ts_4893.meters order by ts limit 10; +select LEAST(5, id) from ts_4893.meters order by ts limit 10; +select LEAST('r', nch1) from ts_4893.meters order by ts limit 10; +select LEAST('r', nch1, nch2) from ts_4893.meters order by ts limit 10; +select LEAST('r', var1) from ts_4893.meters order by ts limit 10; +select LEAST('r', var1, var2) from ts_4893.meters order by ts limit 10; +select LEAST('二中文测试', nch1) from ts_4893.meters order by ts limit 10; +select LEAST('二中文测试', nch1, nch2) from ts_4893.meters order by ts limit 10; +select LEAST('二中文测试', var1) from ts_4893.meters order by ts limit 10; +select LEAST('二中文测试', var1, var2) from ts_4893.meters order by ts limit 10; +select LEAST('23', 3443434343434343); +select LEAST(co, 3443434343434343) from ts_4893.n1; +select LEAST('23', 3443434343434343) from ts_4893.n1; +select LEAST('23', 1443434343434343) from ts_4893.n1; +select LEAST(current, voltage) from ts_4893.n1; +select LEAST(current, voltage, '15') from ts_4893.n1; +alter local 'transToStrWhenMixTypeInLeast' '0'; +select LEAST(1,2,3,4,5,6,7,8,9,10); +select LEAST(1,1.1,2.23,3.4444,5.66666666,6.21241241,7.999999999999); +select LEAST(1,'2',3.3,4.4,5); +select LEAST(1,2,3,4,5,'5.1'); +select LEAST('1','2','3','4',5); +select LEAST('1','2','3','4','5'); +select LEAST(1,2,3,4,5,6,7,'a','b','一','二','三'); +select LEAST(1,2,3,4,5,6,7,'a','b','c','1','2','1231213'); +select LEAST(1,2,3,4,5,6,7,'a','b','c','1','2','1231213','1231213.123123'); +select LEAST(cast(0 as bool), cast(1 as tinyint), cast(2 as smallint), cast(3 as int), cast(4 as bigint), cast(5 as float), cast(6 as double), cast(8 as varchar(20)), cast(9 as nchar(20))); +select LEAST(cast(0 as bool), cast(1 as tinyint unsigned), cast(2 as smallint unsigned), cast(3 as int unsigned), cast(4 as bigint unsigned), cast(5 as float), cast(6 as double), cast(8 as varchar(20)), cast(9 as nchar(20))); +select LEAST(cast(0 as bool), cast(1 as tinyint), cast(2 as smallint), cast(3 as int), cast(4 as bigint), cast(5 as float), cast(6 as double), cast(8 as varchar(20)), cast(9 as nchar(20)), cast(0 as bool), cast(1 as tinyint unsigned), cast(2 as smallint unsigned), cast(3 as int unsigned), cast(4 as bigint unsigned), cast(5 as float), cast(6 as double)); +select LEAST(cast(100 as tinyint), cast(101 as tinyint)); +select LEAST(cast(100 as tinyint), cast(101 as smallint)); +select LEAST(cast(100 as tinyint), cast(101 as int)); +select LEAST(cast(100 as tinyint), cast(101 as bigint)); +select LEAST(cast(100 as tinyint), cast(101 as float)); +select LEAST(cast(100 as tinyint), cast(101 as double)); +select LEAST(cast(100 as tinyint), cast(101 as varchar(20))); +select LEAST(cast(100 as tinyint), cast(101 as nchar(20))); +select LEAST(cast(101 as tinyint), cast(100 as tinyint)); +select LEAST(cast(101 as tinyint), cast(100 as smallint)); +select LEAST(cast(101 as tinyint), cast(100 as int)); +select LEAST(cast(101 as tinyint), cast(100 as bigint)); +select LEAST(cast(101 as tinyint), cast(100 as float)); +select LEAST(cast(101 as tinyint), cast(100 as double)); +select LEAST(cast(101 as tinyint), cast(100 as varchar(20))); +select LEAST(cast(101 as tinyint), cast(100 as nchar(20))); +select LEAST(cast(1000 as smallint), cast(1001 as smallint)); +select LEAST(cast(1000 as smallint), cast(1001 as int)); +select LEAST(cast(1000 as smallint), cast(1001 as bigint)); +select LEAST(cast(1000 as smallint), cast(1001 as float)); +select LEAST(cast(1000 as smallint), cast(1001 as double)); +select LEAST(cast(1000 as smallint), cast(1001 as varchar(20))); +select LEAST(cast(1000 as smallint), cast(1001 as nchar(20))); +select LEAST(cast(1001 as smallint), cast(1000 as smallint)); +select LEAST(cast(1001 as smallint), cast(1000 as int)); +select LEAST(cast(1001 as smallint), cast(1000 as bigint)); +select LEAST(cast(1001 as smallint), cast(1000 as float)); +select LEAST(cast(1001 as smallint), cast(1000 as double)); +select LEAST(cast(1001 as smallint), cast(1000 as varchar(20))); +select LEAST(cast(1001 as smallint), cast(1000 as nchar(20))); +select LEAST(cast(1000000 as int), cast(1000001 as int)); +select LEAST(cast(1000000 as int), cast(1000001 as bigint)); +select LEAST(cast(1000000 as int), cast(1000001 as float)); +select LEAST(cast(1000000 as int), cast(1000001 as double)); +select LEAST(cast(1000000 as int), cast(1000001 as varchar(20))); +select LEAST(cast(1000000 as int), cast(1000001 as nchar(20))); +select LEAST(cast(1000001 as int), cast(1000000 as int)); +select LEAST(cast(1000001 as int), cast(1000000 as bigint)); +select LEAST(cast(1000001 as int), cast(1000000 as float)); +select LEAST(cast(1000001 as int), cast(1000000 as double)); +select LEAST(cast(1000001 as int), cast(1000000 as varchar(20))); +select LEAST(cast(1000001 as int), cast(1000000 as nchar(20))); +select LEAST(cast(1000000000 as bigint), cast(1000000001 as bigint)); +select LEAST(cast(1000000000 as bigint), cast(1000000001 as float)); +select LEAST(cast(1000000000 as bigint), cast(1000000001 as double)); +select LEAST(cast(1000000000 as bigint), cast(1000000001 as varchar(20))); +select LEAST(cast(1000000000 as bigint), cast(1000000001 as nchar(20))); +select LEAST(cast(1000000001 as bigint), cast(1000000000 as bigint)); +select LEAST(cast(1000000001 as bigint), cast(1000000000 as float)); +select LEAST(cast(1000000001 as bigint), cast(1000000000 as double)); +select LEAST(cast(1000000001 as bigint), cast(1000000000 as varchar(20))); +select LEAST(cast(1000000001 as bigint), cast(1000000000 as nchar(20))); +select LEAST(cast(100000.1111111 as float), cast(100001.1111111 as float)); +select LEAST(cast(100000.1111111 as float), cast(100001.1111111 as double)); +select LEAST(cast(100000.1111111 as float), cast(100001.1111111 as timestamp)); +select LEAST(cast(100000.1111111 as float), cast(100001.1111111 as varchar(20))); +select LEAST(cast(100000.1111111 as float), cast(100001.1111111 as nchar(20))); +select LEAST(cast(100001.1111111 as float), cast(100000.1111111 as float)); +select LEAST(cast(100001.1111111 as float), cast(100000.1111111 as double)); +select LEAST(cast(100001.1111111 as float), cast(100000.1111111 as timestamp)); +select LEAST(cast(100001.1111111 as float), cast(100000.1111111 as varchar(20))); +select LEAST(cast(100001.1111111 as float), cast(100000.1111111 as nchar(20))); +select LEAST(cast(100000.1111111 as double), cast(100001.1111111 as double)); +select LEAST(cast(100000.1111111 as double), cast(100001.1111111 as timestamp)); +select LEAST(cast(100000.1111111 as double), cast(100001.1111111 as varchar(20))); +select LEAST(cast(100000.1111111 as double), cast(100001.1111111 as nchar(20))); +select LEAST(cast(100001.1111111 as double), cast(100000.1111111 as double)); +select LEAST(cast(100001.1111111 as double), cast(100000.1111111 as timestamp)); +select LEAST(cast(100001.1111111 as double), cast(100000.1111111 as varchar(20))); +select LEAST(cast(100001.1111111 as double), cast(100000.1111111 as nchar(20))); +select LEAST(cast('中文测试' as varchar(20)), cast('中文测试一' as varchar(20))); +select LEAST(cast('中文测试' as varchar(20)), cast('中文测试一' as nchar(20))); +select LEAST(cast('中文测试一' as varchar(20)), cast('中文测试' as varchar(20))); +select LEAST(cast('中文测试一' as varchar(20)), cast('中文测试' as nchar(20))); +select LEAST(cast('abc123abc' as varchar(20)), cast('abc124abc' as varchar(20))); +select LEAST(cast('abc123abc' as varchar(20)), cast('abc124abc' as nchar(20))); +select LEAST(cast('abc124abc' as varchar(20)), cast('abc123abc' as varchar(20))); +select LEAST(cast('abc124abc' as varchar(20)), cast('abc123abc' as nchar(20))); +select LEAST(cast('abc123abc' as nchar(20)), cast('abc124abc' as nchar(20))); +select LEAST(cast(100 as tinyint), cast(101 as float), cast(102 as varchar(20))); +select LEAST(cast(100 as varchar(20)), cast(101 as float), cast(102 as tinyint)); +select LEAST('a','b','c','d','e','f','g','h','1231','15155'); +select LEAST(current, voltage, phase, id, nch1, nch2, var1, var2) from ts_4893.meters order by ts limit 10; +select LEAST(current, voltage, phase, id) from ts_4893.meters order by ts limit 10; +select LEAST(nch1, nch2, var1, var2) from ts_4893.meters order by ts limit 10; +select LEAST(221, voltage) from ts_4893.meters order by ts limit 10; +select LEAST(5, id) from ts_4893.meters order by ts limit 10; +select LEAST('r', nch1) from ts_4893.meters order by ts limit 10; +select LEAST('r', nch1, nch2) from ts_4893.meters order by ts limit 10; +select LEAST('r', var1) from ts_4893.meters order by ts limit 10; +select LEAST('r', var1, var2) from ts_4893.meters order by ts limit 10; +select LEAST('二中文测试', nch1) from ts_4893.meters order by ts limit 10; +select LEAST('二中文测试', nch1, nch2) from ts_4893.meters order by ts limit 10; +select LEAST('二中文测试', var1) from ts_4893.meters order by ts limit 10; +select LEAST('二中文测试', var1, var2) from ts_4893.meters order by ts limit 10; +select LEAST('23', 3443434343434343); +select LEAST(co, 3443434343434343) from ts_4893.n1; +select LEAST('23', 3443434343434343) from ts_4893.n1; +select LEAST('23', 1443434343434343) from ts_4893.n1; +select LEAST(current, voltage) from ts_4893.n1; +select LEAST(current, voltage, '15') from ts_4893.n1; \ No newline at end of file diff --git a/tests/army/query/function/test_function.py b/tests/army/query/function/test_function.py index c583d08cec..0925a0e066 100644 --- a/tests/army/query/function/test_function.py +++ b/tests/army/query/function/test_function.py @@ -40,37 +40,10 @@ class TDTestCase(TBase): "`var2` VARCHAR(50)) TAGS (`groupid` TINYINT, `location` VARCHAR(16));") tdSql.execute("CREATE table d0 using meters tags(1, 'beijing')") tdSql.execute("insert into d0 file '%s'" % datafile) - - def test_normal_query(self, testCase): - # read sql from .sql file and execute - tdLog.info(f"test normal query.") - sqlFile = etool.curFile(__file__, f"in/{testCase}.in") - ansFile = etool.curFile(__file__, f"ans/{testCase}.csv") - with open(sqlFile, 'r') as sql_file: - sql_statement = '' - tdSql.csvLine = 0 - for line in sql_file: - if not line.strip() or line.strip().startswith('--'): - continue - - sql_statement += line.strip() - if sql_statement.endswith(';'): - sql_statement = sql_statement.rstrip(';') - tdSql.checkDataCsvByLine(sql_statement, ansFile) - sql_statement = '' - err_file_path = etool.curFile(__file__, f"in/{testCase}.err") - if not os.path.isfile(err_file_path): - return None - with open(err_file_path, 'r') as err_file: - err_statement = '' - for line in err_file: - if not line.strip() or line.strip().startswith('--'): - continue - - err_statement += line.strip() - if err_statement.endswith(';'): - tdSql.error(err_statement) - err_statement = '' + tdSql.execute("CREATE TABLE `n1` (`ts` TIMESTAMP, `current` FLOAT, `voltage` INT, co NCHAR(10))") + tdSql.execute("insert into n1 values(now, 1, null, '23')") + tdSql.execute("insert into n1 values(now, null, 3, '23')") + tdSql.execute("insert into n1 values(now, 5, 3, '23')") def test_normal_query_new(self, testCase): # read sql from .sql file and execute @@ -310,6 +283,150 @@ class TDTestCase(TBase): tdSql.error("select * from (select to_iso8601(ts, timezone()), timezone() from ts_4893.meters \ order by ts desc) limit 1000;", expectErrInfo="Invalid parameter data type : to_iso8601") # TS-5340 + def test_greatest(self): + self.test_normal_query_new("greatest") + + tdSql.execute("alter local 'transToStrWhenMixTypeInLeast' '1';") + + tdSql.query("select GREATEST(NULL, NULL, NULL, NULL);") + tdSql.checkRows(1) + tdSql.checkData(0, 0, None) + + tdSql.query("select GREATEST(1, NULL, NULL, NULL);") + tdSql.checkRows(1) + tdSql.checkData(0, 0, None) + + tdSql.query("select GREATEST(id, NULL, 1) from ts_4893.meters order by ts limit 10;") + tdSql.checkRows(10) + tdSql.checkData(0, 0, None) + + tdSql.query("select GREATEST(cast(100 as tinyint), cast(101 as timestamp));") + tdSql.checkRows(1) + tdSql.checkData(0, 0, "1970-01-01 08:00:00.101") + + tdSql.query("select GREATEST(cast(101 as tinyint), cast(100 as timestamp));") + tdSql.checkRows(1) + tdSql.checkData(0, 0, "1970-01-01 08:00:00.101") + + tdSql.query("select GREATEST(cast(1000 as smallint), cast(1001 as timestamp));") + tdSql.checkRows(1) + tdSql.checkData(0, 0, "1970-01-01 08:00:01.001") + + tdSql.query("select GREATEST(cast(1001 as smallint), cast(1000 as timestamp));") + tdSql.checkRows(1) + tdSql.checkData(0, 0, "1970-01-01 08:00:01.001") + + tdSql.query("select GREATEST(cast(1000000 as int), cast(1000001 as timestamp));") + tdSql.checkRows(1) + tdSql.checkData(0, 0, "1970-01-01 08:16:40.001") + + tdSql.query("select GREATEST(cast(1000001 as int), cast(1000000 as timestamp));") + tdSql.checkRows(1) + tdSql.checkData(0, 0, "1970-01-01 08:16:40.001") + + tdSql.query("select GREATEST(cast(1000000000 as bigint), cast(1000000001 as timestamp));") + tdSql.checkRows(1) + tdSql.checkData(0, 0, "1970-01-12 21:46:40.001") + + tdSql.query("select GREATEST(cast(1000000001 as bigint), cast(1000000000 as timestamp));") + tdSql.checkRows(1) + tdSql.checkData(0, 0, "1970-01-12 21:46:40.001") + + tdSql.query("select GREATEST(cast(1725506504000 as timestamp), cast(1725506510000 as timestamp));") + tdSql.checkRows(1) + tdSql.checkData(0, 0, "2024-09-05 11:21:50") + + tdSql.query("select GREATEST(cast(1725506510000 as timestamp), cast(1725506504000 as timestamp));") + tdSql.checkRows(1) + tdSql.checkData(0, 0, "2024-09-05 11:21:50") + + tdSql.query("select GREATEST(cast(100 as tinyint), cast(101 as varchar(20)), cast(102 as float));") + tdSql.checkRows(1) + tdSql.checkData(0, 0, "102.000000") + + tdSql.query("select GREATEST(cast(100 as varchar(20)), cast(101 as tinyint), cast(102 as float));") + tdSql.checkRows(1) + tdSql.checkData(0, 0, "102.000000") + + tdSql.error("select GREATEST(cast('a' as varbinary), cast('b' as varbinary), 'c', 'd');") + tdSql.error("select GREATEST(6, cast('f' as varbinary), cast('b' as varbinary), 'c', 'd');") + + def test_least(self): + self.test_normal_query_new("least") + + tdSql.execute("alter local 'transToStrWhenMixTypeInLeast' '1';") + + tdSql.query("select LEAST(NULL, NULL, NULL, NULL);") + tdSql.checkRows(1) + tdSql.checkData(0, 0, None) + + tdSql.query("select LEAST(1, NULL, NULL, NULL);") + tdSql.checkRows(1) + tdSql.checkData(0, 0, None) + + tdSql.query("select LEAST(id, NULL, 1) from ts_4893.meters order by ts limit 10;") + tdSql.checkRows(10) + tdSql.checkData(0, 0, None) + + tdSql.query("select LEAST(cast(100 as tinyint), cast(101 as timestamp));") + tdSql.checkRows(1) + tdSql.checkData(0, 0, "1970-01-01 08:00:00.100") + + tdSql.query("select LEAST(cast(101 as tinyint), cast(100 as timestamp));") + tdSql.checkRows(1) + tdSql.checkData(0, 0, "1970-01-01 08:00:00.100") + + tdSql.query("select LEAST(cast(1000 as smallint), cast(1001 as timestamp));") + tdSql.checkRows(1) + tdSql.checkData(0, 0, "1970-01-01 08:00:01.000") + + tdSql.query("select LEAST(cast(1001 as smallint), cast(1000 as timestamp));") + tdSql.checkRows(1) + tdSql.checkData(0, 0, "1970-01-01 08:00:01.000") + + tdSql.query("select LEAST(cast(1000000 as int), cast(1000001 as timestamp));") + tdSql.checkRows(1) + tdSql.checkData(0, 0, "1970-01-01 08:16:40.000") + + tdSql.query("select LEAST(cast(1000001 as int), cast(1000000 as timestamp));") + tdSql.checkRows(1) + tdSql.checkData(0, 0, "1970-01-01 08:16:40.000") + + tdSql.query("select LEAST(cast(1000000000 as bigint), cast(1000000001 as timestamp));") + tdSql.checkRows(1) + tdSql.checkData(0, 0, "1970-01-12 21:46:40.000") + + tdSql.query("select LEAST(cast(1000000001 as bigint), cast(1000000000 as timestamp));") + tdSql.checkRows(1) + tdSql.checkData(0, 0, "1970-01-12 21:46:40.000") + + tdSql.query("select LEAST(cast(1725506504000 as timestamp), cast(1725506510000 as timestamp));") + tdSql.checkRows(1) + tdSql.checkData(0, 0, "2024-09-05 11:21:44") + + tdSql.query("select LEAST(cast(1725506510000 as timestamp), cast(1725506504000 as timestamp));") + tdSql.checkRows(1) + tdSql.checkData(0, 0, "2024-09-05 11:21:44") + + tdSql.query("select LEAST(cast(100 as tinyint), cast(101 as varchar(20)), cast(102 as float));") + tdSql.checkRows(1) + tdSql.checkData(0, 0, "100") + + tdSql.query("select LEAST(cast(100 as varchar(20)), cast(101 as tinyint), cast(102 as float));") + tdSql.checkRows(1) + tdSql.checkData(0, 0, "100") + + tdSql.query("select LEAST(cast(100 as float), cast(101 as tinyint), cast(102 as varchar(20)));") + tdSql.checkRows(1) + tdSql.checkData(0, 0, "100.000000") + + tdSql.query("select LEAST(cast(100 as float), cast(101 as varchar(20)), cast(102 as tinyint));") + tdSql.checkRows(1) + tdSql.checkData(0, 0, "100.000000") + + tdSql.error("select LEAST(cast('a' as varbinary), cast('b' as varbinary), 'c', 'd');") + tdSql.error("select LEAST(cast('f' as varbinary), cast('b' as varbinary), 'c', 'd');") + def run(self): tdLog.debug(f"start to excute {__file__}") @@ -326,7 +443,9 @@ class TDTestCase(TBase): self.test_degrees() self.test_radians() self.test_rand() - + self.test_greatest() + self.test_least() + # char function self.test_char_length() self.test_char() diff --git a/tools/shell/src/shellEngine.c b/tools/shell/src/shellEngine.c index b1e21e8f62..99b1c204a2 100644 --- a/tools/shell/src/shellEngine.c +++ b/tools/shell/src/shellEngine.c @@ -668,6 +668,7 @@ void shellPrintField(const char *val, TAOS_FIELD *field, int32_t width, int32_t printf("%*" PRIu64, width, *((uint64_t *)val)); break; case TSDB_DATA_TYPE_FLOAT: + width = width >= LENGTH ? LENGTH - 1 : width; if (tsEnableScience) { printf("%*.7e", width, GET_FLOAT_VAL(val)); } else { @@ -676,6 +677,7 @@ void shellPrintField(const char *val, TAOS_FIELD *field, int32_t width, int32_t } break; case TSDB_DATA_TYPE_DOUBLE: + width = width >= LENGTH ? LENGTH - 1 : width; if (tsEnableScience) { snprintf(buf, LENGTH, "%*.15e", width, GET_DOUBLE_VAL(val)); printf("%s", buf); From 0959423d5b229b82385e53f08af5eb0bd0e63286 Mon Sep 17 00:00:00 2001 From: factosea <285808407@qq.com> Date: Sat, 8 Mar 2025 13:26:22 +0800 Subject: [PATCH 59/70] docs: greatest/least function desc --- .../14-reference/03-taos-sql/10-function.md | 34 +++++++++ .../14-reference/03-taos-sql/10-function.md | 34 +++++++++ tests/army/query/function/test_function.py | 73 +++++++++++++++++++ 3 files changed, 141 insertions(+) diff --git a/docs/en/14-reference/03-taos-sql/10-function.md b/docs/en/14-reference/03-taos-sql/10-function.md index 7875501e7e..7ddcea6897 100644 --- a/docs/en/14-reference/03-taos-sql/10-function.md +++ b/docs/en/14-reference/03-taos-sql/10-function.md @@ -600,6 +600,40 @@ taos> select radians(180); 3.141592653589793 | ``` +#### GREATEST +```sql +GREATEST(expr1, expr2[, expr]...) +``` + +**Function Description**: Get the maximum value of all input parameters. The minimum number of parameters for this function is 2. + +**Version**:ver-3.3.6.0 + +**Return Type**:Refer to the comparison rules. The comparison type is the final return type. + +**Applicable Data Types**: +- Numeric types: including bool, integer and floating point types +- nchar and varchar types. + +**Comparison rules**: The following rules describe the conversion method of the comparison operation: +- If any parameter is NULL, the comparison result is NULL. +- If all parameters in the comparison operation are string types, compare them as string types +- If all parameters are numeric types, compare them as numeric types. +- TIMESTAMP type is also a numeric type. When the types involved in the comparison with TIMESTAMP are all integer types, compare them as TIMESTAMP; +- If there are both string types and numeric types in the parameters, according to the `compareAsStrInGreatest` configuration item, they are uniformly compared as strings or numeric values. By default, they are compared as strings. +- In all cases, when different types are compared, the comparison type will choose the type with a larger range for comparison. For example, when comparing integer types, if there is a BIGINT type, BIGINT will definitely be selected as the comparison type. + +**Related configuration items**: Client configuration, compareAsStrInGreatest is 1, which means that both string types and numeric types are converted to string comparisons, and 0 means that they are converted to numeric types. The default is 1. + + +#### LEAST +```sql +LEAST(expr1, expr2[, expr]...) +``` + +**Function Description**:Get the minimum value of all input parameters. The arguments are compared using the same rules as for LEAST(). The rest of the description is the same as the greatest function. + + ### String Functions The input parameters for string functions are of string type, and the return results are of numeric type or string type. diff --git a/docs/zh/14-reference/03-taos-sql/10-function.md b/docs/zh/14-reference/03-taos-sql/10-function.md index ca7fb18b9f..296eab698b 100644 --- a/docs/zh/14-reference/03-taos-sql/10-function.md +++ b/docs/zh/14-reference/03-taos-sql/10-function.md @@ -578,6 +578,40 @@ taos> select truncate(8888.88, -1); 8880.000000000000000 | ``` +#### GREATEST +```sql +GREATEST(expr1, expr2[, expr]...) +``` + +**功能说明**:获得输入的所有参数中的最大值。该函数最小参数个数为 2 个。 + +**使用说明**:ver-3.3.6.0 + +**返回结果类型**:参考比较规则,比较类型即为最终返回类型。 + +**适用数据类型**: +- 数值类型:包括 bool 型,整型和浮点型 +- nchar 和 varchar 类型。 + +**比较规则**:以下规则描述了比较操作的转换方式: +- 如果有任何一个参数为 NULL,则比较结果为 NULL。 +- 如果比较操作中的所有参数都是字符串类型,按照字符串类型比较 +- 如果所有参数都是数值类型,则将它们作为数值类型进行比较。 +- TIMESTAMP 类型也是数值类型,当和 TIMESTAMP 参与比较的类型都是整数类型时,按照 TIMESTAMP 进行比较; +- 如果参数中既有字符串类型,也有数值类型,根据 compareAsStrInGreatest 配置项,统一作为字符串或者数值进行比较。默认按照字符串比较。 +- 在所有情况下,不同类型比较,比较类型会选择范围更大的类型进行比较,例如作为整数类型比较时,如果存在 BIGINT 类型,必定会选择 BIGINT 作为比较类型。 + +**相关配置项**:客户端配置,compareAsStrInGreatest 为 1 表示同时存在字符串类型和数值类型统一转为字符串比较,为 0 表示统一转为数值类型比较。默认为 1。 + + +#### LEAST +```sql +LEAST(expr1, expr2[, expr]...) +``` + +**功能说明**:获得输入的所有参数中的最小值。其余部分说明同 greatest 函数。 + + ### 字符串函数 字符串函数的输入参数为字符串类型,返回结果为数值类型或字符串类型。 diff --git a/tests/army/query/function/test_function.py b/tests/army/query/function/test_function.py index 0925a0e066..545007bd22 100644 --- a/tests/army/query/function/test_function.py +++ b/tests/army/query/function/test_function.py @@ -348,6 +348,10 @@ class TDTestCase(TBase): tdSql.checkRows(1) tdSql.checkData(0, 0, "102.000000") + tdSql.query("select GREATEST(now, 1);") + tdSql.query("select GREATEST(now, 1.0);") + tdSql.query("select GREATEST(now, '1');") + tdSql.error("select GREATEST(cast('a' as varbinary), cast('b' as varbinary), 'c', 'd');") tdSql.error("select GREATEST(6, cast('f' as varbinary), cast('b' as varbinary), 'c', 'd');") @@ -423,10 +427,78 @@ class TDTestCase(TBase): tdSql.query("select LEAST(cast(100 as float), cast(101 as varchar(20)), cast(102 as tinyint));") tdSql.checkRows(1) tdSql.checkData(0, 0, "100.000000") + + tdSql.query("select LEAST(now, 1);") + tdSql.checkRows(1) + tdSql.checkCols(1) + tdSql.checkData(0, 0, "1970-01-01 08:00:00.001") + + tdSql.query("select LEAST(now, 1.0);") + tdSql.checkRows(1) + tdSql.checkCols(1) + tdSql.checkData(0, 0, 1) + + tdSql.query("select LEAST(now, '1');") + tdSql.checkRows(1) + tdSql.checkCols(1) + tdSql.checkData(0, 0, "1") tdSql.error("select LEAST(cast('a' as varbinary), cast('b' as varbinary), 'c', 'd');") tdSql.error("select LEAST(cast('f' as varbinary), cast('b' as varbinary), 'c', 'd');") + def test_greatest_large_table(self): + tdLog.info("test greatest large table.") + + ts = 1741341251000 + create_table_sql = "CREATE TABLE `large_table` (`ts` TIMESTAMP" + for i in range(1, 1001): + if i % 5 == 1: + create_table_sql += f", `col{i}` INT" + elif i % 5 == 2: + create_table_sql += f", `col{i}` FLOAT" + elif i % 5 == 3: + create_table_sql += f", `col{i}` DOUBLE" + elif i % 5 == 4: + create_table_sql += f", `col{i}` VARCHAR(64)" + else: + create_table_sql += f", `col{i}` NCHAR(50)" + create_table_sql += ");" + tdSql.execute(create_table_sql) + + for j in range(1000): + insert_sql = f"INSERT INTO `large_table` VALUES ({ts +j}" + for i in range(1, 1001): + if i % 5 == 1: + insert_sql += f", {j + i}" + elif i % 5 == 2: + insert_sql += f", {j + i}.1" + elif i % 5 == 3: + insert_sql += f", {j + i}.2" + elif i % 5 == 4: + insert_sql += f", '{j + i}'" + else: + insert_sql += f", '{j + i}'" + insert_sql += ");" + tdSql.execute(insert_sql) + + greatest_query = "SELECT GREATEST(" + for i in range(1, 1001): + greatest_query += f"`col{i}`" + if i < 1000: + greatest_query += ", " + greatest_query += ") FROM `large_table` LIMIT 1;" + tdLog.info(f"greatest_query: {greatest_query}") + tdSql.execute(greatest_query) + + greatest_query = "SELECT " + for i in range(1, 1001): + greatest_query += f"`col{i}` > `col5`" + if i < 1000: + greatest_query += ", " + greatest_query += " FROM `large_table` LIMIT 1;" + tdLog.info(f"greatest_query: {greatest_query}") + tdSql.execute(greatest_query) + def run(self): tdLog.debug(f"start to excute {__file__}") @@ -445,6 +517,7 @@ class TDTestCase(TBase): self.test_rand() self.test_greatest() self.test_least() + self.test_greatest_large_table() # char function self.test_char_length() From 1d707a4318fca0e6b3bb0490f9eadbe516ce6353 Mon Sep 17 00:00:00 2001 From: factosea <285808407@qq.com> Date: Sat, 8 Mar 2025 13:54:00 +0800 Subject: [PATCH 60/70] fix: config variable rename to tsCompareAsStrInGreatest --- include/common/tglobal.h | 2 +- source/common/src/tglobal.c | 10 +++++----- source/libs/function/src/builtins.c | 2 +- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/include/common/tglobal.h b/include/common/tglobal.h index da62966493..8e9367d7c2 100644 --- a/include/common/tglobal.h +++ b/include/common/tglobal.h @@ -299,7 +299,7 @@ extern bool tsStreamCoverage; extern int8_t tsS3EpNum; extern int32_t tsStreamNotifyMessageSize; extern int32_t tsStreamNotifyFrameSize; -extern bool tsTransToStrWhenMixTypeInLeast; +extern bool tsCompareAsStrInGreatest; extern bool tsExperimental; // #define NEEDTO_COMPRESSS_MSG(size) (tsCompressMsgSize != -1 && (size) > tsCompressMsgSize) diff --git a/source/common/src/tglobal.c b/source/common/src/tglobal.c index 235843c041..756d03fd8a 100644 --- a/source/common/src/tglobal.c +++ b/source/common/src/tglobal.c @@ -130,7 +130,7 @@ uint32_t tsEncryptionKeyChksum = 0; int8_t tsEncryptionKeyStat = ENCRYPT_KEY_STAT_UNSET; int8_t tsGrant = 1; -bool tsTransToStrWhenMixTypeInLeast = true; +bool tsCompareAsStrInGreatest = true; // monitor bool tsEnableMonitor = true; @@ -749,7 +749,7 @@ static int32_t taosAddClientCfg(SConfig *pCfg) { TAOS_CHECK_RETURN( cfgAddBool(pCfg, "streamCoverage", tsStreamCoverage, CFG_DYN_CLIENT, CFG_DYN_CLIENT, CFG_CATEGORY_LOCAL)); - TAOS_CHECK_RETURN(cfgAddBool(pCfg, "transToStrWhenMixTypeInLeast", tsTransToStrWhenMixTypeInLeast, CFG_SCOPE_CLIENT, CFG_DYN_CLIENT,CFG_CATEGORY_LOCAL)); + TAOS_CHECK_RETURN(cfgAddBool(pCfg, "compareAsStrInGreatest", tsCompareAsStrInGreatest, CFG_SCOPE_CLIENT, CFG_DYN_CLIENT,CFG_CATEGORY_LOCAL)); TAOS_RETURN(TSDB_CODE_SUCCESS); } @@ -1484,8 +1484,8 @@ static int32_t taosSetClientCfg(SConfig *pCfg) { TAOS_CHECK_GET_CFG_ITEM(pCfg, pItem, "streamCoverage"); tsStreamCoverage = pItem->bval; - TAOS_CHECK_GET_CFG_ITEM(pCfg, pItem, "transToStrWhenMixTypeInLeast"); - tsTransToStrWhenMixTypeInLeast = pItem->bval; + TAOS_CHECK_GET_CFG_ITEM(pCfg, pItem, "compareAsStrInGreatest"); + tsCompareAsStrInGreatest = pItem->bval; TAOS_RETURN(TSDB_CODE_SUCCESS); } @@ -2791,7 +2791,7 @@ static int32_t taosCfgDynamicOptionsForClient(SConfig *pCfg, const char *name) { {"bypassFlag", &tsBypassFlag}, {"safetyCheckLevel", &tsSafetyCheckLevel}, {"streamCoverage", &tsStreamCoverage}, - {"transToStrWhenMixTypeInLeast", &tsTransToStrWhenMixTypeInLeast}}; + {"compareAsStrInGreatest", &tsCompareAsStrInGreatest}}; if ((code = taosCfgSetOption(debugOptions, tListLen(debugOptions), pItem, true)) != TSDB_CODE_SUCCESS) { code = taosCfgSetOption(options, tListLen(options), pItem, false); diff --git a/source/libs/function/src/builtins.c b/source/libs/function/src/builtins.c index 94ccf799bb..67ec8ab504 100644 --- a/source/libs/function/src/builtins.c +++ b/source/libs/function/src/builtins.c @@ -1754,7 +1754,7 @@ static int32_t translateGreatestleast(SFunctionNode* pFunc, char* pErrBuf, int32 return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName); } - bool mixTypeToStrings = tsTransToStrWhenMixTypeInLeast; + bool mixTypeToStrings = tsCompareAsStrInGreatest; SDataType res = {.type = 0}; bool resInit = false; From 3bcbc1200117afd978047f5f93bcf98c8cc5719c Mon Sep 17 00:00:00 2001 From: factosea <285808407@qq.com> Date: Sat, 8 Mar 2025 14:01:15 +0800 Subject: [PATCH 61/70] fix: rename compareAsStrInGreatest in test case --- tests/army/query/function/ans/greatest.csv | 4 +- tests/army/query/function/ans/least.csv | 4 +- tests/army/query/function/in/greatest.in | 4 +- tests/army/query/function/in/least.in | 4 +- tests/army/query/function/test_function.py | 4 +- tests/army/temp_exp.result | 260 +++++++++++++++++++++ 6 files changed, 270 insertions(+), 10 deletions(-) create mode 100644 tests/army/temp_exp.result diff --git a/tests/army/query/function/ans/greatest.csv b/tests/army/query/function/ans/greatest.csv index ced28474f8..b6854a95d6 100644 --- a/tests/army/query/function/ans/greatest.csv +++ b/tests/army/query/function/ans/greatest.csv @@ -1,5 +1,5 @@ -taos> alter local 'transToStrWhenMixTypeInLeast' '1'; +taos> alter local 'compareAsStrInGreatest' '1'; taos> select GREATEST(1,2,3,4,5,6,7,8,9,10); greatest(1,2,3,4,5,6,7,8,9,10) | @@ -713,7 +713,7 @@ taos> select GREATEST(current, voltage, '15') from ts_4893.n1; NULL | 5.000000 | -taos> alter local 'transToStrWhenMixTypeInLeast' '0'; +taos> alter local 'compareAsStrInGreatest' '0'; taos> select GREATEST(1,'2',3.3,4.4,5); greatest(1,'2',3.3,4.4,5) | diff --git a/tests/army/query/function/ans/least.csv b/tests/army/query/function/ans/least.csv index e1ee1095ce..de66426bd2 100644 --- a/tests/army/query/function/ans/least.csv +++ b/tests/army/query/function/ans/least.csv @@ -1,5 +1,5 @@ -taos> alter local 'transToStrWhenMixTypeInLeast' '1'; +taos> alter local 'compareAsStrInGreatest' '1'; taos> select LEAST(1,2,3,4,5,6,7,8,9,10); least(1,2,3,4,5,6,7,8,9,10) | @@ -693,7 +693,7 @@ taos> select LEAST(current, voltage, '15') from ts_4893.n1; NULL | 15 | -taos> alter local 'transToStrWhenMixTypeInLeast' '0'; +taos> alter local 'compareAsStrInGreatest' '0'; taos> select LEAST(1,2,3,4,5,6,7,8,9,10); least(1,2,3,4,5,6,7,8,9,10) | diff --git a/tests/army/query/function/in/greatest.in b/tests/army/query/function/in/greatest.in index 1ae8bee06b..098ef626e9 100644 --- a/tests/army/query/function/in/greatest.in +++ b/tests/army/query/function/in/greatest.in @@ -1,4 +1,4 @@ -alter local 'transToStrWhenMixTypeInLeast' '1'; +alter local 'compareAsStrInGreatest' '1'; select GREATEST(1,2,3,4,5,6,7,8,9,10); select GREATEST(1,1.1,2.23,3.4444,5.66666666,6.21241241,7.999999999999); select GREATEST(1,'2',3.3,4.4,5); @@ -116,7 +116,7 @@ select GREATEST('23', 3443434343434343) from ts_4893.n1; select GREATEST('23', 1443434343434343) from ts_4893.n1; select GREATEST(current, voltage) from ts_4893.n1; select GREATEST(current, voltage, '15') from ts_4893.n1; -alter local 'transToStrWhenMixTypeInLeast' '0'; +alter local 'compareAsStrInGreatest' '0'; select GREATEST(1,'2',3.3,4.4,5); select GREATEST(1,2,3,4,5,'5.1'); select GREATEST(121,'18'); diff --git a/tests/army/query/function/in/least.in b/tests/army/query/function/in/least.in index 1dd664824d..946d00b59b 100644 --- a/tests/army/query/function/in/least.in +++ b/tests/army/query/function/in/least.in @@ -1,4 +1,4 @@ -alter local 'transToStrWhenMixTypeInLeast' '1'; +alter local 'compareAsStrInGreatest' '1'; select LEAST(1,2,3,4,5,6,7,8,9,10); select LEAST(1,1.1,2.23,3.4444,5.66666666,6.21241241,7.999999999999); select LEAST(1,'2',3.3,4.4,5); @@ -112,7 +112,7 @@ select LEAST('23', 3443434343434343) from ts_4893.n1; select LEAST('23', 1443434343434343) from ts_4893.n1; select LEAST(current, voltage) from ts_4893.n1; select LEAST(current, voltage, '15') from ts_4893.n1; -alter local 'transToStrWhenMixTypeInLeast' '0'; +alter local 'compareAsStrInGreatest' '0'; select LEAST(1,2,3,4,5,6,7,8,9,10); select LEAST(1,1.1,2.23,3.4444,5.66666666,6.21241241,7.999999999999); select LEAST(1,'2',3.3,4.4,5); diff --git a/tests/army/query/function/test_function.py b/tests/army/query/function/test_function.py index 545007bd22..f8566914cd 100644 --- a/tests/army/query/function/test_function.py +++ b/tests/army/query/function/test_function.py @@ -286,7 +286,7 @@ class TDTestCase(TBase): def test_greatest(self): self.test_normal_query_new("greatest") - tdSql.execute("alter local 'transToStrWhenMixTypeInLeast' '1';") + tdSql.execute("alter local 'compareAsStrInGreatest' '1';") tdSql.query("select GREATEST(NULL, NULL, NULL, NULL);") tdSql.checkRows(1) @@ -358,7 +358,7 @@ class TDTestCase(TBase): def test_least(self): self.test_normal_query_new("least") - tdSql.execute("alter local 'transToStrWhenMixTypeInLeast' '1';") + tdSql.execute("alter local 'compareAsStrInGreatest' '1';") tdSql.query("select LEAST(NULL, NULL, NULL, NULL);") tdSql.checkRows(1) diff --git a/tests/army/temp_exp.result b/tests/army/temp_exp.result new file mode 100644 index 0000000000..7831bd8b06 --- /dev/null +++ b/tests/army/temp_exp.result @@ -0,0 +1,260 @@ + +taos> select EXP(0) + exp(0) | +============================ + 1 | + +taos> select EXP(1) + exp(1) | +============================ + 2.71828182845905 | + +taos> select EXP(1.5) + exp(1.5) | +============================ + 4.48168907033806 | + +taos> select EXP(100) + exp(100) | +============================ + 2.68811714181614e+43 | + +taos> select EXP(-1) + exp(-1) | +============================ + 0.367879441171442 | + +taos> select EXP(-1.5) + exp(-1.5) | +============================ + 0.22313016014843 | + +taos> select EXP(-100) + exp(-100) | +============================ + 3.72007597602084e-44 | + +taos> select EXP(1) + 1 + exp(1) + 1 | +============================ + 3.71828182845905 | + +taos> select EXP(1) - 1 + exp(1) - 1 | +============================ + 1.71828182845905 | + +taos> select EXP(1) * 1 + exp(1) * 1 | +============================ + 2.71828182845905 | + +taos> select EXP(1) / 1 + exp(1) / 1 | +============================ + 2.71828182845905 | + +taos> select exp(1) from ts_4893.meters limit 5 + exp(1) | +============================ + 2.71828182845905 | + 2.71828182845905 | + 2.71828182845905 | + 2.71828182845905 | + 2.71828182845905 | + +taos> select exp(1) + 1 from ts_4893.meters limit 1 + exp(1) + 1 | +============================ + 3.71828182845905 | + +taos> select exp(1) - 1 from ts_4893.meters limit 1 + exp(1) - 1 | +============================ + 1.71828182845905 | + +taos> select exp(1) * 2 from ts_4893.meters limit 1 + exp(1) * 2 | +============================ + 5.43656365691809 | + +taos> select exp(1) / 2 from ts_4893.meters limit 1 + exp(1) / 2 | +============================ + 1.35914091422952 | + +taos> select exp(2) + exp(1) from ts_4893.meters limit 1 + exp(2) + exp(1) | +============================ + 10.1073379273897 | + +taos> select exp(2) - exp(1) from ts_4893.meters limit 1 + exp(2) - exp(1) | +============================ + 4.67077427047161 | + +taos> select exp(2) * exp(1) from ts_4893.meters limit 1 + exp(2) * exp(1) | +============================ + 20.0855369231877 | + +taos> select exp(2) / exp(1) from ts_4893.meters limit 1 + exp(2) / exp(1) | +============================ + 2.71828182845905 | + +taos> select exp(1) + id from ts_4893.meters order by ts limit 5 + exp(1) + id | +============================ + 2.71828182845905 | + 3.71828182845905 | + 4.71828182845904 | + 5.71828182845904 | + 6.71828182845904 | + +taos> select exp(id) + id from ts_4893.meters order by ts limit 5 + exp(id) + id | +============================ + 1 | + 3.71828182845905 | + 9.38905609893065 | + 23.0855369231877 | + 58.5981500331442 | + +taos> select abs(EXP(10)) + abs(exp(10)) | +============================ + 22026.4657948067 | + +taos> select pow(EXP(10), 2) + pow(exp(10), 2) | +============================ + 485165195.40979 | + +taos> select sqrt(EXP(10)) + sqrt(exp(10)) | +============================ + 148.413159102577 | + +taos> select cast(EXP(10) as int) + cast(exp(10) as int) | +======================= + 22026 | + +taos> select EXP(sqrt(id)) from ts_4893.meters order by ts limit 5 + exp(sqrt(id)) | +============================ + 1 | + 2.71828182845905 | + 4.11325037878293 | + 5.65223367403409 | + 7.38905609893065 | + +taos> select EXP(EXP(EXP(EXP(0)))) + exp(exp(exp(exp(0)))) | +============================ + 3814279.10476021 | + +taos> select exp(2) + exp(2) | +============================ + 7.38905609893065 | + +taos> select exp(0.5) + exp(0.5) | +============================ + 1.64872127070013 | + +taos> select exp(current) from ts_4893.d0 order by ts limit 10 + exp(current) | +============================ + 42192.5784536358 | + 5292.25843238073 | + 17943.8026187706 | + 75583.9925987174 | + 44622.8049048128 | + 4954.24653595498 | + 14705.8362489581 | + 57641.6048971866 | + 75056.7544355615 | + 30853.3277793953 | + +taos> select exp(current) from ts_4893.meters order by ts limit 10 + exp(current) | +============================ + 42192.5784536358 | + 5292.25843238073 | + 17943.8026187706 | + 75583.9925987174 | + 44622.8049048128 | + 4954.24653595498 | + 14705.8362489581 | + 57641.6048971866 | + 75056.7544355615 | + 30853.3277793953 | + +taos> select exp(null) + exp(null) | +============================ + NULL | + +taos> select exp(100000) + exp(100000) | +============================ + NULL | + +taos> select exp(-1000) + exp(-1000) | +============================ + 0 | + +taos> select exp(-9999999999) + exp(-9999999999) | +============================ + 0 | + +taos> select exp(0.0001) + exp(0.0001) | +============================ + 1.00010000500017 | + +taos> select exp(pi()) + exp(pi()) | +============================ + 23.1406926327793 | + +taos> select exp(voltage) from ts_4893.meters limit 1 + exp(voltage) | +============================ + 9.5297279023672e+95 | + +taos> select exp(current) from ts_4893.meters limit 1 + exp(current) | +============================ + 42192.5784536358 | + +taos> select exp(phase) from ts_4893.meters limit 1 + exp(phase) | +============================ + 1.66345708776676 | + +taos> select exp(voltage + current) from ts_4893.meters limit 1 + exp(voltage + current) | +============================ + 4.02083792162431e+100 | + +taos> select exp(abs(current)) from ts_4893.meters limit 1 + exp(abs(current)) | +============================ + 42192.5784536358 | + +taos> select exp(log(voltage)) from ts_4893.meters limit 1 + exp(log(voltage)) | +============================ + 221 | + +taos> select round(exp(voltage), 2) from ts_4893.meters limit 1 + round(exp(voltage), 2) | +============================ + 9.5297279023672e+95 | + From cc0a14f5ab5822adf8450105c98bc5885e3ad18e Mon Sep 17 00:00:00 2001 From: factosea <285808407@qq.com> Date: Sat, 8 Mar 2025 14:02:45 +0800 Subject: [PATCH 62/70] fix: delete unused file --- tests/army/temp_exp.result | 260 ------------------------------------- 1 file changed, 260 deletions(-) delete mode 100644 tests/army/temp_exp.result diff --git a/tests/army/temp_exp.result b/tests/army/temp_exp.result deleted file mode 100644 index 7831bd8b06..0000000000 --- a/tests/army/temp_exp.result +++ /dev/null @@ -1,260 +0,0 @@ - -taos> select EXP(0) - exp(0) | -============================ - 1 | - -taos> select EXP(1) - exp(1) | -============================ - 2.71828182845905 | - -taos> select EXP(1.5) - exp(1.5) | -============================ - 4.48168907033806 | - -taos> select EXP(100) - exp(100) | -============================ - 2.68811714181614e+43 | - -taos> select EXP(-1) - exp(-1) | -============================ - 0.367879441171442 | - -taos> select EXP(-1.5) - exp(-1.5) | -============================ - 0.22313016014843 | - -taos> select EXP(-100) - exp(-100) | -============================ - 3.72007597602084e-44 | - -taos> select EXP(1) + 1 - exp(1) + 1 | -============================ - 3.71828182845905 | - -taos> select EXP(1) - 1 - exp(1) - 1 | -============================ - 1.71828182845905 | - -taos> select EXP(1) * 1 - exp(1) * 1 | -============================ - 2.71828182845905 | - -taos> select EXP(1) / 1 - exp(1) / 1 | -============================ - 2.71828182845905 | - -taos> select exp(1) from ts_4893.meters limit 5 - exp(1) | -============================ - 2.71828182845905 | - 2.71828182845905 | - 2.71828182845905 | - 2.71828182845905 | - 2.71828182845905 | - -taos> select exp(1) + 1 from ts_4893.meters limit 1 - exp(1) + 1 | -============================ - 3.71828182845905 | - -taos> select exp(1) - 1 from ts_4893.meters limit 1 - exp(1) - 1 | -============================ - 1.71828182845905 | - -taos> select exp(1) * 2 from ts_4893.meters limit 1 - exp(1) * 2 | -============================ - 5.43656365691809 | - -taos> select exp(1) / 2 from ts_4893.meters limit 1 - exp(1) / 2 | -============================ - 1.35914091422952 | - -taos> select exp(2) + exp(1) from ts_4893.meters limit 1 - exp(2) + exp(1) | -============================ - 10.1073379273897 | - -taos> select exp(2) - exp(1) from ts_4893.meters limit 1 - exp(2) - exp(1) | -============================ - 4.67077427047161 | - -taos> select exp(2) * exp(1) from ts_4893.meters limit 1 - exp(2) * exp(1) | -============================ - 20.0855369231877 | - -taos> select exp(2) / exp(1) from ts_4893.meters limit 1 - exp(2) / exp(1) | -============================ - 2.71828182845905 | - -taos> select exp(1) + id from ts_4893.meters order by ts limit 5 - exp(1) + id | -============================ - 2.71828182845905 | - 3.71828182845905 | - 4.71828182845904 | - 5.71828182845904 | - 6.71828182845904 | - -taos> select exp(id) + id from ts_4893.meters order by ts limit 5 - exp(id) + id | -============================ - 1 | - 3.71828182845905 | - 9.38905609893065 | - 23.0855369231877 | - 58.5981500331442 | - -taos> select abs(EXP(10)) - abs(exp(10)) | -============================ - 22026.4657948067 | - -taos> select pow(EXP(10), 2) - pow(exp(10), 2) | -============================ - 485165195.40979 | - -taos> select sqrt(EXP(10)) - sqrt(exp(10)) | -============================ - 148.413159102577 | - -taos> select cast(EXP(10) as int) - cast(exp(10) as int) | -======================= - 22026 | - -taos> select EXP(sqrt(id)) from ts_4893.meters order by ts limit 5 - exp(sqrt(id)) | -============================ - 1 | - 2.71828182845905 | - 4.11325037878293 | - 5.65223367403409 | - 7.38905609893065 | - -taos> select EXP(EXP(EXP(EXP(0)))) - exp(exp(exp(exp(0)))) | -============================ - 3814279.10476021 | - -taos> select exp(2) - exp(2) | -============================ - 7.38905609893065 | - -taos> select exp(0.5) - exp(0.5) | -============================ - 1.64872127070013 | - -taos> select exp(current) from ts_4893.d0 order by ts limit 10 - exp(current) | -============================ - 42192.5784536358 | - 5292.25843238073 | - 17943.8026187706 | - 75583.9925987174 | - 44622.8049048128 | - 4954.24653595498 | - 14705.8362489581 | - 57641.6048971866 | - 75056.7544355615 | - 30853.3277793953 | - -taos> select exp(current) from ts_4893.meters order by ts limit 10 - exp(current) | -============================ - 42192.5784536358 | - 5292.25843238073 | - 17943.8026187706 | - 75583.9925987174 | - 44622.8049048128 | - 4954.24653595498 | - 14705.8362489581 | - 57641.6048971866 | - 75056.7544355615 | - 30853.3277793953 | - -taos> select exp(null) - exp(null) | -============================ - NULL | - -taos> select exp(100000) - exp(100000) | -============================ - NULL | - -taos> select exp(-1000) - exp(-1000) | -============================ - 0 | - -taos> select exp(-9999999999) - exp(-9999999999) | -============================ - 0 | - -taos> select exp(0.0001) - exp(0.0001) | -============================ - 1.00010000500017 | - -taos> select exp(pi()) - exp(pi()) | -============================ - 23.1406926327793 | - -taos> select exp(voltage) from ts_4893.meters limit 1 - exp(voltage) | -============================ - 9.5297279023672e+95 | - -taos> select exp(current) from ts_4893.meters limit 1 - exp(current) | -============================ - 42192.5784536358 | - -taos> select exp(phase) from ts_4893.meters limit 1 - exp(phase) | -============================ - 1.66345708776676 | - -taos> select exp(voltage + current) from ts_4893.meters limit 1 - exp(voltage + current) | -============================ - 4.02083792162431e+100 | - -taos> select exp(abs(current)) from ts_4893.meters limit 1 - exp(abs(current)) | -============================ - 42192.5784536358 | - -taos> select exp(log(voltage)) from ts_4893.meters limit 1 - exp(log(voltage)) | -============================ - 221 | - -taos> select round(exp(voltage), 2) from ts_4893.meters limit 1 - round(exp(voltage), 2) | -============================ - 9.5297279023672e+95 | - From 7e6edb82ba05eef52dc706a4b287fa3a1c3901e2 Mon Sep 17 00:00:00 2001 From: factosea <285808407@qq.com> Date: Mon, 10 Mar 2025 15:08:55 +0800 Subject: [PATCH 63/70] fix: param check --- .../14-reference/03-taos-sql/10-function.md | 7 ++--- .../14-reference/03-taos-sql/10-function.md | 3 +- source/libs/function/src/builtins.c | 30 +++++++++++++++---- tests/army/query/function/test_function.py | 1 + 4 files changed, 29 insertions(+), 12 deletions(-) diff --git a/docs/en/14-reference/03-taos-sql/10-function.md b/docs/en/14-reference/03-taos-sql/10-function.md index 7ddcea6897..43772ffb45 100644 --- a/docs/en/14-reference/03-taos-sql/10-function.md +++ b/docs/en/14-reference/03-taos-sql/10-function.md @@ -612,14 +612,13 @@ GREATEST(expr1, expr2[, expr]...) **Return Type**:Refer to the comparison rules. The comparison type is the final return type. **Applicable Data Types**: -- Numeric types: including bool, integer and floating point types -- nchar and varchar types. +- Numeric types: timestamp, bool, integer and floating point types +- Strings types: nchar and varchar types. **Comparison rules**: The following rules describe the conversion method of the comparison operation: - If any parameter is NULL, the comparison result is NULL. - If all parameters in the comparison operation are string types, compare them as string types - If all parameters are numeric types, compare them as numeric types. -- TIMESTAMP type is also a numeric type. When the types involved in the comparison with TIMESTAMP are all integer types, compare them as TIMESTAMP; - If there are both string types and numeric types in the parameters, according to the `compareAsStrInGreatest` configuration item, they are uniformly compared as strings or numeric values. By default, they are compared as strings. - In all cases, when different types are compared, the comparison type will choose the type with a larger range for comparison. For example, when comparing integer types, if there is a BIGINT type, BIGINT will definitely be selected as the comparison type. @@ -631,7 +630,7 @@ GREATEST(expr1, expr2[, expr]...) LEAST(expr1, expr2[, expr]...) ``` -**Function Description**:Get the minimum value of all input parameters. The arguments are compared using the same rules as for LEAST(). The rest of the description is the same as the greatest function. +**Function Description**:Get the minimum value of all input parameters. The arguments are compared using the same rules as for GREATEST(). The rest of the description is the same as the greatest function. ### String Functions diff --git a/docs/zh/14-reference/03-taos-sql/10-function.md b/docs/zh/14-reference/03-taos-sql/10-function.md index 296eab698b..a076c5d81e 100644 --- a/docs/zh/14-reference/03-taos-sql/10-function.md +++ b/docs/zh/14-reference/03-taos-sql/10-function.md @@ -591,13 +591,12 @@ GREATEST(expr1, expr2[, expr]...) **适用数据类型**: - 数值类型:包括 bool 型,整型和浮点型 -- nchar 和 varchar 类型。 +- 字符串类型:包括 nchar 和 varchar 类型。 **比较规则**:以下规则描述了比较操作的转换方式: - 如果有任何一个参数为 NULL,则比较结果为 NULL。 - 如果比较操作中的所有参数都是字符串类型,按照字符串类型比较 - 如果所有参数都是数值类型,则将它们作为数值类型进行比较。 -- TIMESTAMP 类型也是数值类型,当和 TIMESTAMP 参与比较的类型都是整数类型时,按照 TIMESTAMP 进行比较; - 如果参数中既有字符串类型,也有数值类型,根据 compareAsStrInGreatest 配置项,统一作为字符串或者数值进行比较。默认按照字符串比较。 - 在所有情况下,不同类型比较,比较类型会选择范围更大的类型进行比较,例如作为整数类型比较时,如果存在 BIGINT 类型,必定会选择 BIGINT 作为比较类型。 diff --git a/source/libs/function/src/builtins.c b/source/libs/function/src/builtins.c index 67ec8ab504..34ff535486 100644 --- a/source/libs/function/src/builtins.c +++ b/source/libs/function/src/builtins.c @@ -1750,9 +1750,7 @@ static int32_t translateHistogramPartial(SFunctionNode* pFunc, char* pErrBuf, in #define NUMERIC_TO_STRINGS_LEN 25 static int32_t translateGreatestleast(SFunctionNode* pFunc, char* pErrBuf, int32_t len) { - if (LIST_LENGTH(pFunc->pParameterList) < 2) { - return invaildFuncParaNumErrMsg(pErrBuf, len, pFunc->functionName); - } + FUNC_ERR_RET(validateParam(pFunc, pErrBuf, len)); bool mixTypeToStrings = tsCompareAsStrInGreatest; @@ -1785,7 +1783,7 @@ static int32_t translateGreatestleast(SFunctionNode* pFunc, char* pErrBuf, int32 // last res is strings, para is numeric and mixTypeToStrings is true res.bytes = TMAX(res.bytes, NUMERIC_TO_STRINGS_LEN); } - } else if (IS_COMPARE_STR_DATA_TYPE(para->type)) { + } else { if (IS_COMPARE_STR_DATA_TYPE(res.type)) { int32_t resType = vectorGetConvertType(res.type, para->type); res.type = resType == 0 ? res.type : resType; @@ -1800,8 +1798,6 @@ static int32_t translateGreatestleast(SFunctionNode* pFunc, char* pErrBuf, int32 res.type = resType == 0 ? res.type : resType; res.bytes = tDataTypes[resType].bytes; } - } else { - return invaildFuncParaTypeErrMsg(pErrBuf, len, pFunc->functionName); } } pFunc->node.resType = res; @@ -5723,6 +5719,17 @@ const SBuiltinFuncDefinition funcMgtBuiltins[] = { .name = "greatest", .type = FUNCTION_TYPE_GREATEST, .classification = FUNC_MGT_SCALAR_FUNC, + .parameters = {.minParamNum = 2, + .maxParamNum = -1, + .paramInfoPattern = 1, + .inputParaInfo[0][0] = {.isLastParam = true, + .startParam = 1, + .endParam = -1, + .validDataType = FUNC_PARAM_SUPPORT_NUMERIC_TYPE | FUNC_PARAM_SUPPORT_NULL_TYPE | FUNC_PARAM_SUPPORT_BOOL_TYPE | FUNC_PARAM_SUPPORT_TIMESTAMP_TYPE | FUNC_PARAM_SUPPORT_VARCHAR_TYPE | FUNC_PARAM_SUPPORT_NCHAR_TYPE, + .validNodeType = FUNC_PARAM_SUPPORT_EXPR_NODE, + .paramAttribute = FUNC_PARAM_NO_SPECIFIC_ATTRIBUTE, + .valueRangeFlag = FUNC_PARAM_NO_SPECIFIC_VALUE,}, + .outputParaInfo = {.validDataType = FUNC_PARAM_SUPPORT_ALL_TYPE}}, .translateFunc = translateGreatestleast, .getEnvFunc = NULL, .initFunc = NULL, @@ -5733,6 +5740,17 @@ const SBuiltinFuncDefinition funcMgtBuiltins[] = { .name = "least", .type = FUNCTION_TYPE_LEAST, .classification = FUNC_MGT_SCALAR_FUNC, + .parameters = {.minParamNum = 2, + .maxParamNum = -1, + .paramInfoPattern = 1, + .inputParaInfo[0][0] = {.isLastParam = true, + .startParam = 1, + .endParam = -1, + .validDataType = FUNC_PARAM_SUPPORT_NUMERIC_TYPE | FUNC_PARAM_SUPPORT_NULL_TYPE | FUNC_PARAM_SUPPORT_BOOL_TYPE | FUNC_PARAM_SUPPORT_TIMESTAMP_TYPE | FUNC_PARAM_SUPPORT_VARCHAR_TYPE | FUNC_PARAM_SUPPORT_NCHAR_TYPE, + .validNodeType = FUNC_PARAM_SUPPORT_EXPR_NODE, + .paramAttribute = FUNC_PARAM_NO_SPECIFIC_ATTRIBUTE, + .valueRangeFlag = FUNC_PARAM_NO_SPECIFIC_VALUE,}, + .outputParaInfo = {.validDataType = FUNC_PARAM_SUPPORT_ALL_TYPE}}, .translateFunc = translateGreatestleast, .getEnvFunc = NULL, .initFunc = NULL, diff --git a/tests/army/query/function/test_function.py b/tests/army/query/function/test_function.py index f8566914cd..27e54407cc 100644 --- a/tests/army/query/function/test_function.py +++ b/tests/army/query/function/test_function.py @@ -352,6 +352,7 @@ class TDTestCase(TBase): tdSql.query("select GREATEST(now, 1.0);") tdSql.query("select GREATEST(now, '1');") + tdSql.error("select GREATEST(1)") tdSql.error("select GREATEST(cast('a' as varbinary), cast('b' as varbinary), 'c', 'd');") tdSql.error("select GREATEST(6, cast('f' as varbinary), cast('b' as varbinary), 'c', 'd');") From 1ee1636d9f07c31ed6bce5b2e341c2b569359cc5 Mon Sep 17 00:00:00 2001 From: factosea <285808407@qq.com> Date: Mon, 10 Mar 2025 15:46:41 +0800 Subject: [PATCH 64/70] docs: config --- docs/en/14-reference/01-components/02-taosc.md | 1 + docs/zh/14-reference/01-components/02-taosc.md | 6 ++++++ source/libs/function/src/builtins.c | 2 +- 3 files changed, 8 insertions(+), 1 deletion(-) diff --git a/docs/en/14-reference/01-components/02-taosc.md b/docs/en/14-reference/01-components/02-taosc.md index 7b8d3ae78c..83cb3d765b 100644 --- a/docs/en/14-reference/01-components/02-taosc.md +++ b/docs/en/14-reference/01-components/02-taosc.md @@ -44,6 +44,7 @@ The TDengine client driver provides all the APIs needed for application programm |enableQueryHb | |Supported, effective immediately |Internal parameter, whether to send query heartbeat messages| |minSlidingTime | |Supported, effective immediately |Internal parameter, minimum allowable value for sliding| |minIntervalTime | |Supported, effective immediately |Internal parameter, minimum allowable value for interval| +|compareAsStrInGreatest | v3.3.6.0 |Supported, effective immediately |When the greatest and least functions have both numeric and string types as parameters, the comparison type conversion rules are as follows: Integer; 1: uniformly converted to string comparison, 0: uniformly converted to numeric type comparison.| ### Writing Related diff --git a/docs/zh/14-reference/01-components/02-taosc.md b/docs/zh/14-reference/01-components/02-taosc.md index 262027bff6..8952e35664 100755 --- a/docs/zh/14-reference/01-components/02-taosc.md +++ b/docs/zh/14-reference/01-components/02-taosc.md @@ -221,6 +221,12 @@ TDengine 客户端驱动提供了应用编程所需要的全部 API,并且在 - 动态修改:不支持 - 支持版本:从 v3.0.0.0 版本开始引入 +#### compareAsStrInGreatest +- 说明:用于决定 greatest、least 函数的参数既有数值类型又有字符串类型时,比较类型的转换规则。 +- 类型:整数;1:统一转为字符串比较,0:统一转为数值类型比较。 +- 动态修改:支持通过 SQL 修改,立即生效 +- 支持版本:从 v3.3.6.0 版本开始引入 + ### 写入相关 #### smlChildTableName diff --git a/source/libs/function/src/builtins.c b/source/libs/function/src/builtins.c index 34ff535486..8233b2d450 100644 --- a/source/libs/function/src/builtins.c +++ b/source/libs/function/src/builtins.c @@ -1778,7 +1778,7 @@ static int32_t translateGreatestleast(SFunctionNode* pFunc, char* pErrBuf, int32 } else if (IS_MATHABLE_TYPE(res.type) || !mixTypeToStrings) { int32_t resType = vectorGetConvertType(res.type, para->type); res.type = resType == 0 ? res.type : resType; - res.bytes = tDataTypes[resType].bytes; + res.bytes = tDataTypes[res.type].bytes; } else { // last res is strings, para is numeric and mixTypeToStrings is true res.bytes = TMAX(res.bytes, NUMERIC_TO_STRINGS_LEN); From 3248f1cc828a1306276990b939cd6a46db8394d0 Mon Sep 17 00:00:00 2001 From: WANG Xu Date: Tue, 11 Mar 2025 13:47:19 +0800 Subject: [PATCH 65/70] ci: add 3.3.6 branch --- .github/workflows/taosd-ci-build.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.github/workflows/taosd-ci-build.yml b/.github/workflows/taosd-ci-build.yml index 0f65684027..2179200bb3 100644 --- a/.github/workflows/taosd-ci-build.yml +++ b/.github/workflows/taosd-ci-build.yml @@ -6,6 +6,7 @@ on: - 'main' - '3.0' - '3.1' + - '3.3.6' - 'enh/cmake-TD-33848' paths-ignore: From 90aa27ee23ceb9cafc28797f7b229cdcd7bc22d1 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 11 Mar 2025 13:57:31 +0800 Subject: [PATCH 66/70] fix(stream): handle TDMT_STREAM_CHKPT_EXEC in stream queue. --- source/dnode/mgmt/mgmt_snode/inc/smInt.h | 2 +- source/dnode/mgmt/mgmt_snode/src/smHandle.c | 2 ++ source/dnode/mgmt/mgmt_snode/src/smWorker.c | 11 ++++++++++- source/dnode/snode/src/snode.c | 2 ++ tests/system-test/8-stream/checkpoint_info2.py | 2 ++ 5 files changed, 17 insertions(+), 2 deletions(-) diff --git a/source/dnode/mgmt/mgmt_snode/inc/smInt.h b/source/dnode/mgmt/mgmt_snode/inc/smInt.h index 9d519e88f0..0df4b1c58a 100644 --- a/source/dnode/mgmt/mgmt_snode/inc/smInt.h +++ b/source/dnode/mgmt/mgmt_snode/inc/smInt.h @@ -47,7 +47,7 @@ int32_t smPutMsgToQueue(SSnodeMgmt *pMgmt, EQueueType qtype, SRpcMsg *pMsg); int32_t smPutNodeMsgToMgmtQueue(SSnodeMgmt *pMgmt, SRpcMsg *pMsg); int32_t smPutNodeMsgToWriteQueue(SSnodeMgmt *pMgmt, SRpcMsg *pMsg); int32_t smPutNodeMsgToStreamQueue(SSnodeMgmt *pMgmt, SRpcMsg *pMsg); -void sndEnqueueStreamDispatch(SSnode *pSnode, SRpcMsg *pMsg); +int32_t smPutNodeMsgToChkptQueue(SSnodeMgmt *pMgmt, SRpcMsg *pMsg); #ifdef __cplusplus } diff --git a/source/dnode/mgmt/mgmt_snode/src/smHandle.c b/source/dnode/mgmt/mgmt_snode/src/smHandle.c index 024e2e4e99..11710d7b39 100644 --- a/source/dnode/mgmt/mgmt_snode/src/smHandle.c +++ b/source/dnode/mgmt/mgmt_snode/src/smHandle.c @@ -102,6 +102,8 @@ SArray *smGetMsgHandles() { if (dmSetMgmtHandle(pArray, TDMT_MND_STREAM_REQ_CHKPT_RSP, smPutNodeMsgToStreamQueue, 1) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_MND_STREAM_CHKPT_REPORT_RSP, smPutNodeMsgToStreamQueue, 1) == NULL) goto _OVER; + if (dmSetMgmtHandle(pArray, TDMT_STREAM_CHKPT_EXEC, smPutNodeMsgToStreamQueue, 0) == NULL) goto _OVER; + code = 0; _OVER: if (code != 0) { diff --git a/source/dnode/mgmt/mgmt_snode/src/smWorker.c b/source/dnode/mgmt/mgmt_snode/src/smWorker.c index 1e882fc656..1255542454 100644 --- a/source/dnode/mgmt/mgmt_snode/src/smWorker.c +++ b/source/dnode/mgmt/mgmt_snode/src/smWorker.c @@ -162,6 +162,9 @@ int32_t smPutMsgToQueue(SSnodeMgmt *pMgmt, EQueueType qtype, SRpcMsg *pRpc) { case WRITE_QUEUE: code = smPutNodeMsgToWriteQueue(pMgmt, pMsg); break; + case STREAM_CHKPT_QUEUE: + code = smPutNodeMsgToStreamQueue(pMgmt, pMsg); + break; default: code = TSDB_CODE_INVALID_PARA; rpcFreeCont(pMsg->pCont); @@ -172,7 +175,6 @@ int32_t smPutMsgToQueue(SSnodeMgmt *pMgmt, EQueueType qtype, SRpcMsg *pRpc) { } int32_t smPutNodeMsgToMgmtQueue(SSnodeMgmt *pMgmt, SRpcMsg *pMsg) { - int32_t code = 0; SMultiWorker *pWorker = taosArrayGetP(pMgmt->writeWroker, 0); if (pWorker == NULL) { return TSDB_CODE_INVALID_MSG; @@ -198,3 +200,10 @@ int32_t smPutNodeMsgToStreamQueue(SSnodeMgmt *pMgmt, SRpcMsg *pMsg) { dTrace("msg:%p, put into worker %s", pMsg, pWorker->name); return taosWriteQitem(pWorker->queue, pMsg); } + +//int32_t smPutNodeMsgToChkptQueue(SSnodeMgmt *pMgmt, SRpcMsg *pMsg) { +// SSingleWorker *pWorker = &pMgmt->chkptWorker; +// +// dTrace("msg:%p, put into worker %s", pMsg, pWorker->name); +// return taosWriteQitem(pWorker->queue, pMsg); +//} diff --git a/source/dnode/snode/src/snode.c b/source/dnode/snode/src/snode.c index 326dcd712e..dcdc70da68 100644 --- a/source/dnode/snode/src/snode.c +++ b/source/dnode/snode/src/snode.c @@ -138,6 +138,8 @@ int32_t sndProcessStreamMsg(SSnode *pSnode, SRpcMsg *pMsg) { return tqStreamTaskProcessRetrieveTriggerReq(pSnode->pMeta, pMsg); case TDMT_STREAM_RETRIEVE_TRIGGER_RSP: return tqStreamTaskProcessRetrieveTriggerRsp(pSnode->pMeta, pMsg); + case TDMT_STREAM_CHKPT_EXEC: + return tqStreamTaskProcessRunReq(pSnode->pMeta, pMsg, true); default: sndError("invalid snode msg:%d", pMsg->msgType); return TSDB_CODE_INVALID_MSG; diff --git a/tests/system-test/8-stream/checkpoint_info2.py b/tests/system-test/8-stream/checkpoint_info2.py index 3dc57477f7..f4c8da8c9d 100644 --- a/tests/system-test/8-stream/checkpoint_info2.py +++ b/tests/system-test/8-stream/checkpoint_info2.py @@ -22,6 +22,8 @@ from util.cluster import * # should be used by -N option class TDTestCase: updatecfgDict = {'checkpointInterval': 60 , + 'vdebugflag':143, + 'ddebugflag':143 } def init(self, conn, logSql, replicaVar=1): self.replicaVar = int(replicaVar) From fb2583f14ca741098f2d48812cd31720f25f79b9 Mon Sep 17 00:00:00 2001 From: factosea <285808407@qq.com> Date: Tue, 11 Mar 2025 14:00:42 +0800 Subject: [PATCH 67/70] docs: function description sorting --- .../14-reference/03-taos-sql/10-function.md | 67 +++++++++---------- .../14-reference/03-taos-sql/10-function.md | 66 +++++++++--------- 2 files changed, 65 insertions(+), 68 deletions(-) diff --git a/docs/en/14-reference/03-taos-sql/10-function.md b/docs/en/14-reference/03-taos-sql/10-function.md index 43772ffb45..993b1176fe 100644 --- a/docs/en/14-reference/03-taos-sql/10-function.md +++ b/docs/en/14-reference/03-taos-sql/10-function.md @@ -124,7 +124,39 @@ FLOOR(expr) ``` **Function Description**: Gets the floor of the specified field. - Other usage notes see CEIL function description. + Other usage notes see [CEIL](#ceil) function description. + +#### GREATEST +```sql +GREATEST(expr1, expr2[, expr]...) +``` + +**Function Description**: Get the maximum value of all input parameters. The minimum number of parameters for this function is 2. + +**Version**:ver-3.3.6.0 + +**Return Type**:Refer to the comparison rules. The comparison type is the final return type. + +**Applicable Data Types**: +- Numeric types: timestamp, bool, integer and floating point types +- Strings types: nchar and varchar types. + +**Comparison rules**: The following rules describe the conversion method of the comparison operation: +- If any parameter is NULL, the comparison result is NULL. +- If all parameters in the comparison operation are string types, compare them as string types +- If all parameters are numeric types, compare them as numeric types. +- If there are both string types and numeric types in the parameters, according to the `compareAsStrInGreatest` configuration item, they are uniformly compared as strings or numeric values. By default, they are compared as strings. +- In all cases, when different types are compared, the comparison type will choose the type with a larger range for comparison. For example, when comparing integer types, if there is a BIGINT type, BIGINT will definitely be selected as the comparison type. + +**Related configuration items**: Client configuration, compareAsStrInGreatest is 1, which means that both string types and numeric types are converted to string comparisons, and 0 means that they are converted to numeric types. The default is 1. + + +#### LEAST +```sql +LEAST(expr1, expr2[, expr]...) +``` + +**Function Description**:Get the minimum value of all input parameters. The rest of the description is the same as the [GREATEST](#greatest) function. #### LOG @@ -600,39 +632,6 @@ taos> select radians(180); 3.141592653589793 | ``` -#### GREATEST -```sql -GREATEST(expr1, expr2[, expr]...) -``` - -**Function Description**: Get the maximum value of all input parameters. The minimum number of parameters for this function is 2. - -**Version**:ver-3.3.6.0 - -**Return Type**:Refer to the comparison rules. The comparison type is the final return type. - -**Applicable Data Types**: -- Numeric types: timestamp, bool, integer and floating point types -- Strings types: nchar and varchar types. - -**Comparison rules**: The following rules describe the conversion method of the comparison operation: -- If any parameter is NULL, the comparison result is NULL. -- If all parameters in the comparison operation are string types, compare them as string types -- If all parameters are numeric types, compare them as numeric types. -- If there are both string types and numeric types in the parameters, according to the `compareAsStrInGreatest` configuration item, they are uniformly compared as strings or numeric values. By default, they are compared as strings. -- In all cases, when different types are compared, the comparison type will choose the type with a larger range for comparison. For example, when comparing integer types, if there is a BIGINT type, BIGINT will definitely be selected as the comparison type. - -**Related configuration items**: Client configuration, compareAsStrInGreatest is 1, which means that both string types and numeric types are converted to string comparisons, and 0 means that they are converted to numeric types. The default is 1. - - -#### LEAST -```sql -LEAST(expr1, expr2[, expr]...) -``` - -**Function Description**:Get the minimum value of all input parameters. The arguments are compared using the same rules as for GREATEST(). The rest of the description is the same as the greatest function. - - ### String Functions The input parameters for string functions are of string type, and the return results are of numeric type or string type. diff --git a/docs/zh/14-reference/03-taos-sql/10-function.md b/docs/zh/14-reference/03-taos-sql/10-function.md index a076c5d81e..a8959d1d22 100644 --- a/docs/zh/14-reference/03-taos-sql/10-function.md +++ b/docs/zh/14-reference/03-taos-sql/10-function.md @@ -186,7 +186,38 @@ FLOOR(expr) ``` **功能说明**:获得指定字段的向下取整数的结果。 - 其他使用说明参见 CEIL 函数描述。 + 其他使用说明参见 [CEIL](#ceil) 函数描述。 + +#### GREATEST +```sql +GREATEST(expr1, expr2[, expr]...) +``` + +**功能说明**:获得输入的所有参数中的最大值。该函数最小参数个数为 2 个。 + +**使用说明**:ver-3.3.6.0 + +**返回结果类型**:参考比较规则,比较类型即为最终返回类型。 + +**适用数据类型**: +- 数值类型:包括 bool 型,整型和浮点型 +- 字符串类型:包括 nchar 和 varchar 类型。 + +**比较规则**:以下规则描述了比较操作的转换方式: +- 如果有任何一个参数为 NULL,则比较结果为 NULL。 +- 如果比较操作中的所有参数都是字符串类型,按照字符串类型比较 +- 如果所有参数都是数值类型,则将它们作为数值类型进行比较。 +- 如果参数中既有字符串类型,也有数值类型,根据 compareAsStrInGreatest 配置项,统一作为字符串或者数值进行比较。默认按照字符串比较。 +- 在所有情况下,不同类型比较,比较类型会选择范围更大的类型进行比较,例如作为整数类型比较时,如果存在 BIGINT 类型,必定会选择 BIGINT 作为比较类型。 + +**相关配置项**:客户端配置,compareAsStrInGreatest 为 1 表示同时存在字符串类型和数值类型统一转为字符串比较,为 0 表示统一转为数值类型比较。默认为 1。 + +#### LEAST +```sql +LEAST(expr1, expr2[, expr]...) +``` + +**功能说明**:获得输入的所有参数中的最小值。其余部分说明同 [GREATEST](#greatest) 函数。 #### LN @@ -578,39 +609,6 @@ taos> select truncate(8888.88, -1); 8880.000000000000000 | ``` -#### GREATEST -```sql -GREATEST(expr1, expr2[, expr]...) -``` - -**功能说明**:获得输入的所有参数中的最大值。该函数最小参数个数为 2 个。 - -**使用说明**:ver-3.3.6.0 - -**返回结果类型**:参考比较规则,比较类型即为最终返回类型。 - -**适用数据类型**: -- 数值类型:包括 bool 型,整型和浮点型 -- 字符串类型:包括 nchar 和 varchar 类型。 - -**比较规则**:以下规则描述了比较操作的转换方式: -- 如果有任何一个参数为 NULL,则比较结果为 NULL。 -- 如果比较操作中的所有参数都是字符串类型,按照字符串类型比较 -- 如果所有参数都是数值类型,则将它们作为数值类型进行比较。 -- 如果参数中既有字符串类型,也有数值类型,根据 compareAsStrInGreatest 配置项,统一作为字符串或者数值进行比较。默认按照字符串比较。 -- 在所有情况下,不同类型比较,比较类型会选择范围更大的类型进行比较,例如作为整数类型比较时,如果存在 BIGINT 类型,必定会选择 BIGINT 作为比较类型。 - -**相关配置项**:客户端配置,compareAsStrInGreatest 为 1 表示同时存在字符串类型和数值类型统一转为字符串比较,为 0 表示统一转为数值类型比较。默认为 1。 - - -#### LEAST -```sql -LEAST(expr1, expr2[, expr]...) -``` - -**功能说明**:获得输入的所有参数中的最小值。其余部分说明同 greatest 函数。 - - ### 字符串函数 字符串函数的输入参数为字符串类型,返回结果为数值类型或字符串类型。 From a4eec90ba4fd9ac667c69dff2e66f3e1f339bb4e Mon Sep 17 00:00:00 2001 From: WANG Xu Date: Tue, 11 Mar 2025 15:24:06 +0800 Subject: [PATCH 68/70] fix: working directory at job level Signed-off-by: WANG Xu --- .github/workflows/tdgpt-ci.yml | 54 ++++++++++++++++++---------------- 1 file changed, 29 insertions(+), 25 deletions(-) diff --git a/.github/workflows/tdgpt-ci.yml b/.github/workflows/tdgpt-ci.yml index 02c1c5889f..4a438938dc 100644 --- a/.github/workflows/tdgpt-ci.yml +++ b/.github/workflows/tdgpt-ci.yml @@ -1,6 +1,6 @@ -name: TDGpt CI Pipeline +name: TDgpt CI -on: +on: pull_request: branches: - '3.0' @@ -10,38 +10,42 @@ on: jobs: build: runs-on: ubuntu-latest - working-directory: ${{ github.workspace }}/tools/tdgpt + strategy: fail-fast: false matrix: python-version: ["3.10"] + defaults: + run: + working-directory: ${{ github.workspace }}/tools/tdgpt + steps: - - uses: actions/checkout@v4 + - uses: actions/checkout@v4 - - name: Set up Python ${{ matrix.python-version }} - uses: actions/setup-python@v3 - with: - python-version: ${{ matrix.python-version }} - cache: 'pip' + - name: Set up Python ${{ matrix.python-version }} + uses: actions/setup-python@v3 + with: + python-version: ${{ matrix.python-version }} + cache: 'pip' - - name: Install dependencies - run: | - python -m pip install --upgrade pip - python -m pip install flake8 pytest pylint - if [ -f requirements.txt ]; then pip install -r requirements.txt; fi + - name: Install dependencies + run: | + python -m pip install --upgrade pip + python -m pip install flake8 pytest pylint + if [ -f requirements.txt ]; then pip install -r requirements.txt; fi - - name: Checking the code with pylint - run: | - pylint $(git ls-files '*.py') --exit-zero + - name: Checking the code with pylint + run: | + pylint $(git ls-files '*.py') --exit-zero - - name: Checking the code with flake8 - run: | - # stop the build if there are Python syntax errors or undefined names - flake8 . --count --select=E9,F63,F7,F82 --show-source --statistics - # exit-zero treats all errors as warnings. The GitHub editor is 127 chars wide - flake8 . --count --exit-zero --max-complexity=10 --max-line-length=127 --statistics + - name: Checking the code with flake8 + run: | + # stop the build if there are Python syntax errors or undefined names + flake8 . --count --select=E9,F63,F7,F82 --show-source --statistics + # exit-zero treats all errors as warnings. The GitHub editor is 127 chars wide + flake8 . --count --exit-zero --max-complexity=10 --max-line-length=127 --statistics - - name: Run test cases with pytest - run: | + - name: Run test cases with pytest + run: | pytest From 5934e75c5aa4cecded688f487fe146e626ecc5aa Mon Sep 17 00:00:00 2001 From: WANG Xu Date: Tue, 11 Mar 2025 15:31:05 +0800 Subject: [PATCH 69/70] ci: modify workflow name --- .github/workflows/tdgpt-update-service.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/tdgpt-update-service.yml b/.github/workflows/tdgpt-update-service.yml index 03bff257c1..fa7344ce74 100644 --- a/.github/workflows/tdgpt-update-service.yml +++ b/.github/workflows/tdgpt-update-service.yml @@ -1,4 +1,4 @@ -name: TDGpt Update Service +name: TDgpt Update Service on: schedule: From d038633ec2e71c43b8f9be7d9c8a06f94e0bc4f8 Mon Sep 17 00:00:00 2001 From: Feng Chao Date: Tue, 11 Mar 2025 20:57:43 +0800 Subject: [PATCH 70/70] udpate ci workflow --- .github/workflows/taosd-ci.yml | 19 +++++-------------- 1 file changed, 5 insertions(+), 14 deletions(-) diff --git a/.github/workflows/taosd-ci.yml b/.github/workflows/taosd-ci.yml index b7b97cee8e..31eda597ad 100644 --- a/.github/workflows/taosd-ci.yml +++ b/.github/workflows/taosd-ci.yml @@ -70,33 +70,24 @@ jobs: echo ${{ github.event.pull_request.number }} run-tests-on-linux: - uses: taosdata/.github/.github/workflows/run-tests-on-linux.yml + uses: taosdata/.github/.github/workflows/run-tests-on-linux.yml@main needs: fetch-parameters if: ${{ needs.fetch-parameters.outputs.run_tdgpt_test == 'true' || needs.fetch-parameters.outputs.run_function_test == 'true' }} with: tdinternal: false - run_function_test: ${{ needs.fetch-parameters.outputs.run_function_test == 'true' || false }} - run_tdgpt_test: ${{ needs.fetch-parameters.outputs.run_tdgpt_test == 'true' || false }} - source_branch: ${{ github.event.pull_request.head.ref }} - target_branch: ${{ github.event.pull_request.base.ref }} - pr_number: ${{ github.event.pull_request.number }} + run_function_test: ${{ needs.fetch-parameters.outputs.run_function_test == 'true' }} + run_tdgpt_test: ${{ needs.fetch-parameters.outputs.run_tdgpt_test == 'true' }} run-tests-on-mac: - uses: taosdata/.github/.github/workflows/run-tests-on-macos.yml + uses: taosdata/.github/.github/workflows/run-tests-on-macos.yml@main needs: fetch-parameters if: ${{ needs.fetch-parameters.outputs.run_function_test == 'true' }} with: tdinternal: false - source_branch: ${{ github.event.pull_request.head.ref }} - target_branch: ${{ github.event.pull_request.base.ref }} - pr_number: ${{ github.event.pull_request.number }} run-tests-on-windows: - uses: taosdata/.github/.github/workflows/run-tests-on-windows.yml + uses: taosdata/.github/.github/workflows/run-tests-on-windows.yml@main needs: fetch-parameters if: ${{ needs.fetch-parameters.outputs.run_function_test == 'true' }} with: tdinternal: false - source_branch: ${{ github.event.pull_request.head.ref }} - target_branch: ${{ github.event.pull_request.base.ref }} - pr_number: ${{ github.event.pull_request.number }}