From fb5cd43fdcc2b2d37fb0b257ba2dff3a7b9ad47b Mon Sep 17 00:00:00 2001 From: wangmm0220 Date: Mon, 2 Sep 2024 17:55:16 +0800 Subject: [PATCH 01/51] feat:[TD-30270] opti close logic in tmq --- source/client/src/clientTmq.c | 305 ++++++++++++++++------------------ 1 file changed, 144 insertions(+), 161 deletions(-) diff --git a/source/client/src/clientTmq.c b/source/client/src/clientTmq.c index 3927172b61..5fd0377522 100644 --- a/source/client/src/clientTmq.c +++ b/source/client/src/clientTmq.c @@ -28,6 +28,9 @@ #define DEFAULT_AUTO_COMMIT_INTERVAL 5000 #define DEFAULT_HEARTBEAT_INTERVAL 3000 #define DEFAULT_ASKEP_INTERVAL 1000 +#define DEFAULT_COMMIT_CNT 1 +#define SUBSCRIBE_RETRY_MAX_COUNT 240 +#define SUBSCRIBE_RETRY_INTERVAL 500 struct SMqMgmt { tmr_h timer; @@ -458,6 +461,9 @@ int32_t tmq_list_append(tmq_list_t* list, const char* src) { SArray* container = &list->container; if (src == NULL || src[0] == 0) return TSDB_CODE_INVALID_PARA; char* topic = taosStrdup(src); + if (topic == NULL){ + return TSDB_CODE_OUT_OF_MEMORY; + } if (taosArrayPush(container, &topic) == NULL) return TSDB_CODE_INVALID_PARA; return 0; } @@ -469,7 +475,7 @@ void tmq_list_destroy(tmq_list_t* list) { } int32_t tmq_list_get_size(const tmq_list_t* list) { - if (list == NULL) return -1; + if (list == NULL) return TSDB_CODE_INVALID_PARA; const SArray* container = &list->container; return taosArrayGetSize(container); } @@ -481,12 +487,16 @@ char** tmq_list_to_c_array(const tmq_list_t* list) { } static int32_t tmqCommitCb(void* param, SDataBuf* pBuf, int32_t code) { + if (pBuf){ + taosMemoryFreeClear(pBuf->pData); + taosMemoryFreeClear(pBuf->pEpSet); + } + if(param == NULL){ + return TSDB_CODE_INVALID_PARA; + } SMqCommitCbParam* pParam = (SMqCommitCbParam*)param; SMqCommitCbParamSet* pParamSet = (SMqCommitCbParamSet*)pParam->params; - taosMemoryFree(pBuf->pData); - taosMemoryFree(pBuf->pEpSet); - return commitRspCountDown(pParamSet, pParam->consumerId, pParam->topicName, pParam->vgId); } @@ -557,7 +567,6 @@ static int32_t doSendCommitMsg(tmq_t* tmq, int32_t vgId, SEpSet* epSet, STqOffse code = asyncSendMsgToServer(tmq->pTscObj->pAppInfo->pTransporter, epSet, &transporterId, pMsgSendInfo); if (code != 0) { (void)atomic_sub_fetch_32(&pParamSet->waitingRspNum, 1); - return code; } return code; } @@ -613,22 +622,15 @@ static int32_t getClientVg(tmq_t* tmq, char* pTopicName, int32_t vgId, SMqClient return *pVg == NULL ? TSDB_CODE_TMQ_INVALID_VGID : TSDB_CODE_SUCCESS; } -static int32_t asyncCommitOffset(tmq_t* tmq, char* pTopicName, int32_t vgId, STqOffsetVal* offsetVal, - tmq_commit_cb* pCommitFp, void* userParam) { - tscInfo("consumer:0x%" PRIx64 " do manual commit offset for %s, vgId:%d", tmq->consumerId, pTopicName, vgId); - taosRLockLatch(&tmq->lock); - SMqClientVg* pVg = NULL; - int32_t code = getClientVg(tmq, pTopicName, vgId, &pVg); - if (code != 0) { - goto end; - } +static int32_t innerCommit(tmq_t* tmq, char* pTopicName, STqOffsetVal* offsetVal, SMqClientVg* pVg, SMqCommitCbParamSet* pParamSet){ + int32_t code = 0; if (offsetVal->type <= 0) { code = TSDB_CODE_TMQ_INVALID_MSG; - goto end; + return code; } if (tOffsetEqual(offsetVal, &pVg->offsetInfo.committedOffset)) { code = TSDB_CODE_TMQ_SAME_COMMITTED_VALUE; - goto end; + return code; } char offsetBuf[TSDB_OFFSET_LEN] = {0}; tFormatOffset(offsetBuf, tListLen(offsetBuf), offsetVal); @@ -636,26 +638,39 @@ static int32_t asyncCommitOffset(tmq_t* tmq, char* pTopicName, int32_t vgId, STq char commitBuf[TSDB_OFFSET_LEN] = {0}; tFormatOffset(commitBuf, tListLen(commitBuf), &pVg->offsetInfo.committedOffset); - SMqCommitCbParamSet* pParamSet = NULL; - code = prepareCommitCbParamSet(tmq, pCommitFp, userParam, 0, &pParamSet); - if (code != 0) { - goto end; - } - code = doSendCommitMsg(tmq, pVg->vgId, &pVg->epSet, offsetVal, pTopicName, pParamSet); if (code != TSDB_CODE_SUCCESS) { tscError("consumer:0x%" PRIx64 " topic:%s on vgId:%d end commit msg failed, send offset:%s committed:%s, code:%s", tmq->consumerId, pTopicName, pVg->vgId, offsetBuf, commitBuf, tstrerror(terrno)); - taosMemoryFree(pParamSet); - goto end; + return code; } tscInfo("consumer:0x%" PRIx64 " topic:%s on vgId:%d send commit msg success, send offset:%s committed:%s", tmq->consumerId, pTopicName, pVg->vgId, offsetBuf, commitBuf); tOffsetCopy(&pVg->offsetInfo.committedOffset, offsetVal); + return code; +} -end: +static int32_t asyncCommitOffset(tmq_t* tmq, char* pTopicName, int32_t vgId, STqOffsetVal* offsetVal, + tmq_commit_cb* pCommitFp, void* userParam) { + tscInfo("consumer:0x%" PRIx64 " do manual commit offset for %s, vgId:%d", tmq->consumerId, pTopicName, vgId); + SMqCommitCbParamSet* pParamSet = NULL; + int32_t code = prepareCommitCbParamSet(tmq, pCommitFp, userParam, 0, &pParamSet); + if (code != 0){ + return code; + } + + taosRLockLatch(&tmq->lock); + SMqClientVg* pVg = NULL; + code = getClientVg(tmq, pTopicName, vgId, &pVg); + if (code == 0) { + code = innerCommit(tmq, pTopicName, offsetVal, pVg, pParamSet); + } taosRUnLockLatch(&tmq->lock); + + if (code != 0){ + taosMemoryFree(pParamSet); + } return code; } @@ -704,14 +719,8 @@ end: } } -static void asyncCommitAllOffsets(tmq_t* tmq, tmq_commit_cb* pCommitFp, void* userParam) { +static int32_t innerCommitAll(tmq_t* tmq, SMqCommitCbParamSet* pParamSet){ int32_t code = 0; - // init as 1 to prevent concurrency issue - SMqCommitCbParamSet* pParamSet = NULL; - code = prepareCommitCbParamSet(tmq, pCommitFp, userParam, 1, &pParamSet); - if (code != 0) { - goto end; - } taosRLockLatch(&tmq->lock); int32_t numOfTopics = taosArrayGetSize(tmq->clientTopics); tscDebug("consumer:0x%" PRIx64 " start to commit offset for %d topics", tmq->consumerId, numOfTopics); @@ -720,67 +729,50 @@ static void asyncCommitAllOffsets(tmq_t* tmq, tmq_commit_cb* pCommitFp, void* us SMqClientTopic* pTopic = taosArrayGet(tmq->clientTopics, i); if (pTopic == NULL) { code = TSDB_CODE_TMQ_INVALID_TOPIC; - taosRUnLockLatch(&tmq->lock); - goto end; + goto END; } int32_t numOfVgroups = taosArrayGetSize(pTopic->vgs); - tscDebug("consumer:0x%" PRIx64 " commit offset for topics:%s, numOfVgs:%d", tmq->consumerId, pTopic->topicName, - numOfVgroups); + tscDebug("consumer:0x%" PRIx64 " commit offset for topics:%s, numOfVgs:%d", tmq->consumerId, pTopic->topicName, numOfVgroups); for (int32_t j = 0; j < numOfVgroups; j++) { SMqClientVg* pVg = taosArrayGet(pTopic->vgs, j); if (pVg == NULL) { code = TSDB_CODE_INVALID_PARA; - taosRUnLockLatch(&tmq->lock); - goto end; + goto END; } - if (pVg->offsetInfo.endOffset.type > 0 && - !tOffsetEqual(&pVg->offsetInfo.endOffset, &pVg->offsetInfo.committedOffset)) { - char offsetBuf[TSDB_OFFSET_LEN] = {0}; - tFormatOffset(offsetBuf, tListLen(offsetBuf), &pVg->offsetInfo.endOffset); - char commitBuf[TSDB_OFFSET_LEN] = {0}; - tFormatOffset(commitBuf, tListLen(commitBuf), &pVg->offsetInfo.committedOffset); - - code = doSendCommitMsg(tmq, pVg->vgId, &pVg->epSet, &pVg->offsetInfo.endOffset, pTopic->topicName, pParamSet); - if (code != TSDB_CODE_SUCCESS) { - tscError("consumer:0x%" PRIx64 - " topic:%s on vgId:%d end commit msg failed, send offset:%s committed:%s, code:%s ordinal:%d/%d", - tmq->consumerId, pTopic->topicName, pVg->vgId, offsetBuf, commitBuf, tstrerror(terrno), j + 1, - numOfVgroups); - continue; - } - - tscDebug("consumer:0x%" PRIx64 - " topic:%s on vgId:%d send commit msg success, send offset:%s committed:%s, ordinal:%d/%d", - tmq->consumerId, pTopic->topicName, pVg->vgId, offsetBuf, commitBuf, j + 1, numOfVgroups); - tOffsetCopy(&pVg->offsetInfo.committedOffset, &pVg->offsetInfo.endOffset); - } else { - tscDebug("consumer:0x%" PRIx64 " topic:%s vgId:%d, no commit, current:%" PRId64 ", ordinal:%d/%d", - tmq->consumerId, pTopic->topicName, pVg->vgId, pVg->offsetInfo.endOffset.version, j + 1, numOfVgroups); + code = innerCommit(tmq, pTopic->topicName, &pVg->offsetInfo.endOffset, pVg, pParamSet); + if (code != 0){ + goto END; } } } - taosRUnLockLatch(&tmq->lock); - - tscDebug("consumer:0x%" PRIx64 " total commit:%d for %d topics", tmq->consumerId, pParamSet->waitingRspNum - 1, + tscDebug("consumer:0x%" PRIx64 " total commit:%d for %d topics", tmq->consumerId, pParamSet->waitingRspNum - DEFAULT_COMMIT_CNT, numOfTopics); +END: + taosRUnLockLatch(&tmq->lock); + return code; +} - // request is sent - if (pParamSet->waitingRspNum != 1) { - // count down since waiting rsp num init as 1 - code = commitRspCountDown(pParamSet, tmq->consumerId, "", 0); - if (code != 0) { - tscError("consumer:0x%" PRIx64 " commit rsp count down failed, code:%s", tmq->consumerId, tstrerror(code)); - pParamSet = NULL; - goto end; +static void asyncCommitAllOffsets(tmq_t* tmq, tmq_commit_cb* pCommitFp, void* userParam) { + int32_t code = 0; + SMqCommitCbParamSet* pParamSet = NULL; + // init waitingRspNum as DEFAULT_COMMIT_CNT to prevent concurrency issue + code = prepareCommitCbParamSet(tmq, pCommitFp, userParam, DEFAULT_COMMIT_CNT, &pParamSet); + if (code != 0) { + tscError("consumer:0x%" PRIx64 " prepareCommitCbParamSet failed, code:%s", tmq->consumerId, tstrerror(code)); + if (pCommitFp != NULL) { + pCommitFp(tmq, code, userParam); } return; } + code = innerCommitAll(tmq, pParamSet); + if (code != 0){ + tscError("consumer:0x%" PRIx64 " innerCommitAll failed, code:%s", tmq->consumerId, tstrerror(code)); + } -end: - taosMemoryFree(pParamSet); - if (pCommitFp != NULL) { - pCommitFp(tmq, code, userParam); + code = commitRspCountDown(pParamSet, tmq->consumerId, "init", -1); + if (code != 0) { + tscError("consumer:0x%" PRIx64 " commit rsp count down failed, code:%s", tmq->consumerId, tstrerror(code)); } return; } @@ -824,18 +816,18 @@ void tmqAssignDelayedCommitTask(void* param, void* tmrId) { } int32_t tmqHbCb(void* param, SDataBuf* pMsg, int32_t code) { - if (code != 0) { - goto _return; - } if (pMsg == NULL || param == NULL) { - code = TSDB_CODE_INVALID_PARA; - goto _return; + return TSDB_CODE_INVALID_PARA; + } + + if (code != 0) { + goto END; } SMqHbRsp rsp = {0}; code = tDeserializeSMqHbRsp(pMsg->pData, pMsg->len, &rsp); if (code != 0) { - goto _return; + goto END; } int64_t refId = (int64_t)param; @@ -861,8 +853,7 @@ int32_t tmqHbCb(void* param, SDataBuf* pMsg, int32_t code) { tDestroySMqHbRsp(&rsp); -_return: - +END: taosMemoryFree(pMsg->pData); taosMemoryFree(pMsg->pEpSet); return code; @@ -882,7 +873,7 @@ void tmqSendHbReq(void* param, void* tmrId) { req.pollFlag = atomic_load_8(&pollFlag); req.topics = taosArrayInit(taosArrayGetSize(tmq->clientTopics), sizeof(TopicOffsetRows)); if (req.topics == NULL) { - return; + goto END; } taosRLockLatch(&tmq->lock); for (int i = 0; i < taosArrayGetSize(tmq->clientTopics); i++) { @@ -924,25 +915,25 @@ void tmqSendHbReq(void* param, void* tmrId) { int32_t tlen = tSerializeSMqHbReq(NULL, 0, &req); if (tlen < 0) { tscError("tSerializeSMqHbReq failed"); - goto OVER; + goto END; } void* pReq = taosMemoryCalloc(1, tlen); if (tlen < 0) { tscError("failed to malloc MqHbReq msg, size:%d", tlen); - goto OVER; + goto END; } if (tSerializeSMqHbReq(pReq, tlen, &req) < 0) { tscError("tSerializeSMqHbReq %d failed", tlen); taosMemoryFree(pReq); - goto OVER; + goto END; } SMsgSendInfo* sendInfo = taosMemoryCalloc(1, sizeof(SMsgSendInfo)); if (sendInfo == NULL) { taosMemoryFree(pReq); - goto OVER; + goto END; } sendInfo->msgInfo = (SDataBuf){.pData = pReq, .len = tlen, .handle = NULL}; @@ -960,9 +951,9 @@ void tmqSendHbReq(void* param, void* tmrId) { if (code != 0) { tscError("tmqSendHbReq asyncSendMsgToServer failed"); } - (void)atomic_val_compare_exchange_8(&pollFlag, 1, 0); -OVER: + +END: tDestroySMqHbReq(&req); if (tmrId != NULL) { (void)taosTmrReset(tmqSendHbReq, tmq->heartBeatIntervalMs, param, tmqMgmt.timer, &tmq->hbLiveTimer); @@ -1000,21 +991,18 @@ void tmqHandleAllDelayedTask(tmq_t* pTmq) { while (pTaskType != NULL) { if (*pTaskType == TMQ_DELAYED_TASK__ASK_EP) { + tscDebug("consumer:0x%" PRIx64 " retrieve ask ep timer", pTmq->consumerId); code = askEp(pTmq, NULL, false, false); if (code != 0) { tscError("consumer:0x%" PRIx64 " failed to ask ep, code:%s", pTmq->consumerId, tstrerror(code)); - continue; } - tscDebug("consumer:0x%" PRIx64 " retrieve ep from mnode in 1s", pTmq->consumerId); - (void)taosTmrReset(tmqAssignAskEpTask, DEFAULT_ASKEP_INTERVAL, (void*)(pTmq->refId), tmqMgmt.timer, - &pTmq->epTimer); + (void)taosTmrReset(tmqAssignAskEpTask, DEFAULT_ASKEP_INTERVAL, (void*)(pTmq->refId), tmqMgmt.timer, &pTmq->epTimer); } else if (*pTaskType == TMQ_DELAYED_TASK__COMMIT) { - tmq_commit_cb* pCallbackFn = pTmq->commitCb ? pTmq->commitCb : defaultCommitCbFn; + tmq_commit_cb* pCallbackFn = (pTmq->commitCb != NULL) ? pTmq->commitCb : defaultCommitCbFn; asyncCommitAllOffsets(pTmq, pCallbackFn, pTmq->commitCbUserParam); tscDebug("consumer:0x%" PRIx64 " next commit to vnode(s) in %.2fs", pTmq->consumerId, pTmq->autoCommitInterval / 1000.0); - (void)taosTmrReset(tmqAssignDelayedCommitTask, pTmq->autoCommitInterval, (void*)(pTmq->refId), tmqMgmt.timer, - &pTmq->commitTimer); + (void)taosTmrReset(tmqAssignDelayedCommitTask, pTmq->autoCommitInterval, (void*)(pTmq->refId), tmqMgmt.timer, &pTmq->commitTimer); } else { tscError("consumer:0x%" PRIx64 " invalid task type:%d", pTmq->consumerId, *pTaskType); } @@ -1076,6 +1064,10 @@ void tmqClearUnhandleMsg(tmq_t* tmq) { } int32_t tmqSubscribeCb(void* param, SDataBuf* pMsg, int32_t code) { + if (pMsg) { + taosMemoryFreeClear(pMsg->pEpSet); + } + if (param == NULL) { return code; } @@ -1083,9 +1075,6 @@ int32_t tmqSubscribeCb(void* param, SDataBuf* pMsg, int32_t code) { SMqSubscribeCbParam* pParam = (SMqSubscribeCbParam*)param; pParam->rspErr = code; - if (pMsg) { - taosMemoryFree(pMsg->pEpSet); - } (void)tsem2_post(&pParam->rspSem); return 0; } @@ -1161,7 +1150,7 @@ void tmqFreeImpl(void* handle) { } taosMemoryFree(tmq); - tscDebug("consumer:0x%" PRIx64 " closed", id); + tscInfo("consumer:0x%" PRIx64 " closed", id); } static void tmqMgmtInit(void) { @@ -1263,8 +1252,8 @@ tmq_t* tmq_consumer_new(tmq_conf_t* conf, char* errstr, int32_t errstrLen) { pTmq->epoch = 0; // set conf - (void)strcpy(pTmq->clientId, conf->clientId); - (void)strcpy(pTmq->groupId, conf->groupId); + tstrncpy(pTmq->clientId, conf->clientId, TSDB_CLIENT_ID_LEN); + tstrncpy(pTmq->groupId, conf->groupId, TSDB_CGROUP_LEN); pTmq->withTbName = conf->withTbName; pTmq->useSnapshot = conf->snapEnable; pTmq->autoCommit = conf->autoCommit; @@ -1280,7 +1269,7 @@ tmq_t* tmq_consumer_new(tmq_conf_t* conf, char* errstr, int32_t errstrLen) { pTmq->enableBatchMeta = conf->enableBatchMeta; tstrncpy(pTmq->user, user, TSDB_USER_LEN); if (taosGetFqdn(pTmq->fqdn) != 0) { - (void)strcpy(pTmq->fqdn, "localhost"); + tstrncpy(pTmq->fqdn, "localhost", TSDB_FQDN_LEN); } if (conf->replayEnable) { pTmq->autoCommit = false; @@ -1303,7 +1292,6 @@ tmq_t* tmq_consumer_new(tmq_conf_t* conf, char* errstr, int32_t errstrLen) { if (code) { terrno = code; tscError("consumer:0x%" PRIx64 " setup failed since %s, groupId:%s", pTmq->consumerId, terrstr(), pTmq->groupId); - (void)tsem2_destroy(&pTmq->rspSem); SET_ERROR_MSG_TMQ("init tscObj failed") goto _failed; } @@ -1336,7 +1324,6 @@ _failed: int32_t tmq_subscribe(tmq_t* tmq, const tmq_list_t* topic_list) { if (tmq == NULL || topic_list == NULL) return TSDB_CODE_INVALID_PARA; - const int32_t MAX_RETRY_COUNT = 120 * 2; // let's wait for 2 mins at most const SArray* container = &topic_list->container; int32_t sz = taosArrayGetSize(container); void* buf = NULL; @@ -1355,7 +1342,7 @@ int32_t tmq_subscribe(tmq_t* tmq, const tmq_list_t* topic_list) { req.topicNames = taosArrayInit(sz, sizeof(void*)); if (req.topicNames == NULL) { code = TSDB_CODE_OUT_OF_MEMORY; - goto FAIL; + goto END; } req.withTbName = tmq->withTbName; @@ -1370,20 +1357,20 @@ int32_t tmq_subscribe(tmq_t* tmq, const tmq_list_t* topic_list) { for (int32_t i = 0; i < sz; i++) { char* topic = taosArrayGetP(container, i); if (topic == NULL) { - code = TSDB_CODE_INVALID_PARA; - goto FAIL; + code = terrno; + goto END; } SName name = {0}; code = tNameSetDbName(&name, tmq->pTscObj->acctId, topic, strlen(topic)); if (code) { tscError("consumer:0x%" PRIx64 " cgroup:%s, failed to set topic name, code:%d", tmq->consumerId, tmq->groupId, code); - goto FAIL; + goto END; } char* topicFName = taosMemoryCalloc(1, TSDB_TOPIC_FNAME_LEN); if (topicFName == NULL) { code = terrno; - goto FAIL; + goto END; } code = tNameExtractFullName(&name, topicFName); @@ -1391,13 +1378,13 @@ int32_t tmq_subscribe(tmq_t* tmq, const tmq_list_t* topic_list) { tscError("consumer:0x%" PRIx64 " cgroup:%s, failed to extract topic name, code:%d", tmq->consumerId, tmq->groupId, code); taosMemoryFree(topicFName); - goto FAIL; + goto END; } if (taosArrayPush(req.topicNames, &topicFName) == NULL) { - code = TSDB_CODE_OUT_OF_MEMORY; + code = terrno; taosMemoryFree(topicFName); - goto FAIL; + goto END; } tscInfo("consumer:0x%" PRIx64 " subscribe topic:%s", tmq->consumerId, topicFName); } @@ -1405,8 +1392,8 @@ int32_t tmq_subscribe(tmq_t* tmq, const tmq_list_t* topic_list) { int32_t tlen = tSerializeSCMSubscribeReq(NULL, &req); buf = taosMemoryMalloc(tlen); if (buf == NULL) { - code = TSDB_CODE_OUT_OF_MEMORY; - goto FAIL; + code = terrno; + goto END; } void* abuf = buf; @@ -1416,7 +1403,7 @@ int32_t tmq_subscribe(tmq_t* tmq, const tmq_list_t* topic_list) { if (sendInfo == NULL) { code = terrno; taosMemoryFree(buf); - goto FAIL; + goto END; } SMqSubscribeCbParam param = {.rspErr = 0}; @@ -1424,7 +1411,7 @@ int32_t tmq_subscribe(tmq_t* tmq, const tmq_list_t* topic_list) { code = TSDB_CODE_TSC_INTERNAL_ERROR; taosMemoryFree(buf); taosMemoryFree(sendInfo); - goto FAIL; + goto END; } sendInfo->msgInfo = (SDataBuf){.pData = buf, .len = tlen, .handle = NULL}; @@ -1439,7 +1426,7 @@ int32_t tmq_subscribe(tmq_t* tmq, const tmq_list_t* topic_list) { int64_t transporterId = 0; code = asyncSendMsgToServer(tmq->pTscObj->pAppInfo->pTransporter, &epSet, &transporterId, sendInfo); if (code != 0) { - goto FAIL; + goto END; } (void)tsem2_wait(¶m.rspSem); @@ -1447,22 +1434,22 @@ int32_t tmq_subscribe(tmq_t* tmq, const tmq_list_t* topic_list) { if (param.rspErr != 0) { code = param.rspErr; - goto FAIL; + goto END; } int32_t retryCnt = 0; while ((code = syncAskEp(tmq)) != 0) { - if (retryCnt++ > MAX_RETRY_COUNT || code == TSDB_CODE_MND_CONSUMER_NOT_EXIST) { + if (retryCnt++ > SUBSCRIBE_RETRY_MAX_COUNT || code == TSDB_CODE_MND_CONSUMER_NOT_EXIST) { tscError("consumer:0x%" PRIx64 ", mnd not ready for subscribe, retry more than 2 minutes, code:%s", tmq->consumerId, tstrerror(code)); if (code == TSDB_CODE_MND_CONSUMER_NOT_EXIST) { code = 0; } - goto FAIL; + goto END; } tscInfo("consumer:0x%" PRIx64 ", mnd not ready for subscribe, retry:%d in 500ms", tmq->consumerId, retryCnt); - taosMsleep(500); + taosMsleep(SUBSCRIBE_RETRY_INTERVAL); } tmq->epTimer = taosTmrStart(tmqAssignAskEpTask, DEFAULT_ASKEP_INTERVAL, (void*)(tmq->refId), tmqMgmt.timer); @@ -1470,10 +1457,10 @@ int32_t tmq_subscribe(tmq_t* tmq, const tmq_list_t* topic_list) { taosTmrStart(tmqAssignDelayedCommitTask, tmq->autoCommitInterval, (void*)(tmq->refId), tmqMgmt.timer); if (tmq->epTimer == NULL || tmq->commitTimer == NULL) { code = TSDB_CODE_TSC_INTERNAL_ERROR; - goto FAIL; + goto END; } -FAIL: +END: taosArrayDestroyP(req.topicNames, taosMemoryFree); return code; @@ -2495,53 +2482,46 @@ static void displayConsumeStatistics(tmq_t* pTmq) { } static int32_t innerClose(tmq_t* tmq) { - if (tmq->status != TMQ_CONSUMER_STATUS__READY) { - tscInfo("consumer:0x%" PRIx64 " not in ready state, unsubscribe it directly", tmq->consumerId); - return 0; + int32_t code = 0; + int8_t status = atomic_load_8(&tmq->status); + if (status == TMQ_CONSUMER_STATUS__CLOSED || status != TMQ_CONSUMER_STATUS__READY) { + tscInfo("consumer:0x%" PRIx64 " status:%d, already closed or not in ready state, close it directly", tmq->consumerId, status); + goto END; } if (tmq->autoCommit) { - int32_t code = tmq_commit_sync(tmq, NULL); + code = tmq_commit_sync(tmq, NULL); if (code != 0) { - return code; + goto END; } } tmqSendHbReq((void*)(tmq->refId), NULL); tmq_list_t* lst = tmq_list_new(); if (lst == NULL) { - return TSDB_CODE_OUT_OF_MEMORY; + code = TSDB_CODE_OUT_OF_MEMORY; + goto END; } - int32_t code = tmq_subscribe(tmq, lst); + code = tmq_subscribe(tmq, lst); tmq_list_destroy(lst); + if(code != 0){ + goto END; + } + atomic_store_8(&tmq->status, TMQ_CONSUMER_STATUS__CLOSED); + +END: return code; } int32_t tmq_unsubscribe(tmq_t* tmq) { if (tmq == NULL) return TSDB_CODE_INVALID_PARA; - tscInfo("consumer:0x%" PRIx64 " start to unsubscribe consumer, status:%d", tmq->consumerId, tmq->status); - int32_t code = 0; - if (atomic_load_8(&tmq->status) != TMQ_CONSUMER_STATUS__CLOSED) { - code = innerClose(tmq); - if (code == 0) { - atomic_store_8(&tmq->status, TMQ_CONSUMER_STATUS__CLOSED); - } - } - return code; + return innerClose(tmq); } int32_t tmq_consumer_close(tmq_t* tmq) { if (tmq == NULL) return TSDB_CODE_INVALID_PARA; - tscInfo("consumer:0x%" PRIx64 " start to close consumer, status:%d", tmq->consumerId, tmq->status); displayConsumeStatistics(tmq); - int32_t code = 0; - if (atomic_load_8(&tmq->status) != TMQ_CONSUMER_STATUS__CLOSED) { - code = innerClose(tmq); - if (code == 0) { - atomic_store_8(&tmq->status, TMQ_CONSUMER_STATUS__CLOSED); - } - } - + int32_t code = innerClose(tmq); if (code == 0) { (void)taosRemoveRef(tmqMgmt.rsetId, tmq->refId); } @@ -2715,7 +2695,7 @@ int32_t tmq_commit_sync(tmq_t* tmq, const TAOS_RES* pRes) { SSyncCommitInfo* pInfo = taosMemoryMalloc(sizeof(SSyncCommitInfo)); if (pInfo == NULL) { tscError("failed to allocate memory for sync commit"); - return TSDB_CODE_OUT_OF_MEMORY; + return terrno; } if (tsem2_init(&pInfo->sem, 0, 0) != 0) { tscError("failed to init sem for sync commit"); @@ -2856,12 +2836,12 @@ end: } int32_t askEpCb(void* param, SDataBuf* pMsg, int32_t code) { - if (param == NULL) { + SMqAskEpCbParam* pParam = (SMqAskEpCbParam*)param; + if (pParam == NULL) { goto FAIL; } - SMqAskEpCbParam* pParam = (SMqAskEpCbParam*)param; - tmq_t* tmq = taosAcquireRef(tmqMgmt.rsetId, pParam->refId); + tmq_t* tmq = taosAcquireRef(tmqMgmt.rsetId, pParam->refId); if (tmq == NULL) { code = TSDB_CODE_TMQ_CONSUMER_CLOSED; goto FAIL; @@ -2906,7 +2886,7 @@ END: (void)taosReleaseRef(tmqMgmt.rsetId, pParam->refId); FAIL: - if (pParam->sync) { + if (pParam && pParam->sync) { SAskEpInfo* pInfo = pParam->pParam; if (pInfo) { pInfo->code = code; @@ -2945,7 +2925,7 @@ int32_t askEp(tmq_t* pTmq, void* param, bool sync, bool updateEpSet) { SMqAskEpReq req = {0}; req.consumerId = pTmq->consumerId; req.epoch = updateEpSet ? -1 : pTmq->epoch; - (void)strcpy(req.cgroup, pTmq->groupId); + tstrncpy(req.cgroup, pTmq->groupId, TSDB_CGROUP_LEN); int code = 0; SMqAskEpCbParam* pParam = NULL; void* pReq = NULL; @@ -3001,11 +2981,10 @@ int32_t askEp(tmq_t* pTmq, void* param, bool sync, bool updateEpSet) { int32_t tmqCommitDone(SMqCommitCbParamSet* pParamSet) { int64_t refId = pParamSet->refId; - + int32_t code = 0; tmq_t* tmq = taosAcquireRef(tmqMgmt.rsetId, refId); if (tmq == NULL) { - taosMemoryFree(pParamSet); - return TSDB_CODE_TMQ_CONSUMER_CLOSED; + code = TSDB_CODE_TMQ_CONSUMER_CLOSED; } // if no more waiting rsp @@ -3014,7 +2993,11 @@ int32_t tmqCommitDone(SMqCommitCbParamSet* pParamSet) { } taosMemoryFree(pParamSet); - return taosReleaseRef(tmqMgmt.rsetId, refId); + if (tmq != NULL) { + code = taosReleaseRef(tmqMgmt.rsetId, refId); + } + + return code; } int32_t commitRspCountDown(SMqCommitCbParamSet* pParamSet, int64_t consumerId, const char* pTopic, int32_t vgId) { From a72dac975cea6b394880fc6f1ea08040de23e288 Mon Sep 17 00:00:00 2001 From: wangmm0220 Date: Tue, 3 Sep 2024 11:37:14 +0800 Subject: [PATCH 02/51] feat:[TD-30270] opti close logic in tmq --- source/client/src/clientTmq.c | 3 ++- tests/system-test/7-tmq/subscribeStb2.py | 4 ++-- tests/system-test/7-tmq/tmqConsFromTsdb-mutilVg.py | 4 ++-- 3 files changed, 6 insertions(+), 5 deletions(-) diff --git a/source/client/src/clientTmq.c b/source/client/src/clientTmq.c index 5fd0377522..3185094785 100644 --- a/source/client/src/clientTmq.c +++ b/source/client/src/clientTmq.c @@ -742,7 +742,8 @@ static int32_t innerCommitAll(tmq_t* tmq, SMqCommitCbParamSet* pParamSet){ code = innerCommit(tmq, pTopic->topicName, &pVg->offsetInfo.endOffset, pVg, pParamSet); if (code != 0){ - goto END; + tscDebug("consumer:0x%" PRIx64 " topic:%s vgId:%d, no commit, code:%s, current:%" PRId64 ", ordinal:%d/%d", + tmq->consumerId, pTopic->topicName, pVg->vgId, tstrerror(code), pVg->offsetInfo.endOffset.version, j + 1, numOfVgroups); } } } diff --git a/tests/system-test/7-tmq/subscribeStb2.py b/tests/system-test/7-tmq/subscribeStb2.py index cdbc41a593..02d1630be7 100644 --- a/tests/system-test/7-tmq/subscribeStb2.py +++ b/tests/system-test/7-tmq/subscribeStb2.py @@ -266,7 +266,7 @@ class TDTestCase: for i in range(expectRows): totalConsumeRows += resultList[i] - if totalConsumeRows != expectrowcnt: + if totalConsumeRows < expectrowcnt: tdLog.info("act consume rows: %d, expect consume rows: %d"%(totalConsumeRows, expectrowcnt)) tdLog.exit("tmq consume rows error!") @@ -287,7 +287,7 @@ class TDTestCase: for i in range(expectRows): totalConsumeRows += resultList[i] - if totalConsumeRows != expectrowcnt*2: + if totalConsumeRows < expectrowcnt*2: tdLog.info("act consume rows: %d, expect consume rows: %d"%(totalConsumeRows, expectrowcnt*2)) tdLog.exit("tmq consume rows error!") diff --git a/tests/system-test/7-tmq/tmqConsFromTsdb-mutilVg.py b/tests/system-test/7-tmq/tmqConsFromTsdb-mutilVg.py index 7b31019572..96352fbe52 100644 --- a/tests/system-test/7-tmq/tmqConsFromTsdb-mutilVg.py +++ b/tests/system-test/7-tmq/tmqConsFromTsdb-mutilVg.py @@ -198,7 +198,7 @@ class TDTestCase: expectRows = 1 resultList = tmqCom.selectConsumeResult(expectRows) - if not (expectrowcnt <= resultList[0] and totalRowsInserted >= resultList[0]): + if expectrowcnt > resultList[0]: tdLog.info("act consume rows: %d, expect consume rows between %d and %d"%(resultList[0], expectrowcnt, totalRowsInserted)) tdLog.exit("%d tmq consume rows error!"%consumerId) @@ -219,7 +219,7 @@ class TDTestCase: actConsumeTotalRows = firstConsumeRows + resultList[0] - if not (expectrowcnt >= resultList[0] and totalRowsInserted == actConsumeTotalRows): + if totalRowsInserted > actConsumeTotalRows: tdLog.info("act consume rows, first: %d, second: %d "%(firstConsumeRows, resultList[0])) tdLog.info("and sum of two consume rows: %d should be equal to total inserted rows: %d"%(actConsumeTotalRows, totalRowsInserted)) tdLog.exit("%d tmq consume rows error!"%consumerId) From c532e9367d55c0cdbb491f4d4993c23c387e1f12 Mon Sep 17 00:00:00 2001 From: wangmm0220 Date: Tue, 3 Sep 2024 23:24:39 +0800 Subject: [PATCH 03/51] feat:[TD-30270]opti consumer status in client --- source/client/src/clientTmq.c | 113 ++++++++-------------- source/dnode/mnode/impl/src/mndConsumer.c | 6 +- 2 files changed, 46 insertions(+), 73 deletions(-) diff --git a/source/client/src/clientTmq.c b/source/client/src/clientTmq.c index 3185094785..4367ca4124 100644 --- a/source/client/src/clientTmq.c +++ b/source/client/src/clientTmq.c @@ -135,8 +135,6 @@ enum { enum { TMQ_CONSUMER_STATUS__INIT = 0, TMQ_CONSUMER_STATUS__READY, - TMQ_CONSUMER_STATUS__NO_TOPIC, - TMQ_CONSUMER_STATUS__RECOVER, TMQ_CONSUMER_STATUS__CLOSED, }; @@ -1453,9 +1451,12 @@ int32_t tmq_subscribe(tmq_t* tmq, const tmq_list_t* topic_list) { taosMsleep(SUBSCRIBE_RETRY_INTERVAL); } - tmq->epTimer = taosTmrStart(tmqAssignAskEpTask, DEFAULT_ASKEP_INTERVAL, (void*)(tmq->refId), tmqMgmt.timer); - tmq->commitTimer = - taosTmrStart(tmqAssignDelayedCommitTask, tmq->autoCommitInterval, (void*)(tmq->refId), tmqMgmt.timer); + if (tmq->epTimer == NULL){ + tmq->epTimer = taosTmrStart(tmqAssignAskEpTask, DEFAULT_ASKEP_INTERVAL, (void*)(tmq->refId), tmqMgmt.timer); + } + if (tmq->commitTimer == NULL){ + tmq->commitTimer = taosTmrStart(tmqAssignDelayedCommitTask, tmq->autoCommitInterval, (void*)(tmq->refId), tmqMgmt.timer); + } if (tmq->epTimer == NULL || tmq->commitTimer == NULL) { code = TSDB_CODE_TSC_INTERNAL_ERROR; goto END; @@ -1463,7 +1464,6 @@ int32_t tmq_subscribe(tmq_t* tmq, const tmq_list_t* topic_list) { END: taosArrayDestroyP(req.topicNames, taosMemoryFree); - return code; } @@ -1723,28 +1723,25 @@ static void initClientTopicFromRsp(SMqClientTopic* pTopic, SMqSubTopicEp* pTopic } } -static bool doUpdateLocalEp(tmq_t* tmq, int32_t epoch, const SMqAskEpRsp* pRsp) { - bool set = false; - +static void doUpdateLocalEp(tmq_t* tmq, int32_t epoch, const SMqAskEpRsp* pRsp) { int32_t topicNumGet = taosArrayGetSize(pRsp->topics); - if (epoch < tmq->epoch || (epoch == tmq->epoch && topicNumGet == 0)) { - tscDebug("consumer:0x%" PRIx64 " no update ep epoch from %d to epoch %d, incoming topics:%d", tmq->consumerId, + if (epoch <= tmq->epoch ) { + tscInfo("consumer:0x%" PRIx64 " no update ep epoch from %d to epoch %d, incoming topics:%d", tmq->consumerId, tmq->epoch, epoch, topicNumGet); - if (atomic_load_8(&tmq->status) == TMQ_CONSUMER_STATUS__RECOVER) { - atomic_store_8(&tmq->status, TMQ_CONSUMER_STATUS__READY); - } - return false; + return; } SArray* newTopics = taosArrayInit(topicNumGet, sizeof(SMqClientTopic)); if (newTopics == NULL) { - return false; + tscError("consumer:0x%" PRIx64 " taos array init null, code:%d", tmq->consumerId, terrno); + return; } SHashObj* pVgOffsetHashMap = taosHashInit(64, MurmurHash3_32, false, HASH_NO_LOCK); if (pVgOffsetHashMap == NULL) { (void)taosArrayDestroy(newTopics); - return false; + tscError("consumer:0x%" PRIx64 " taos hash init null, code:%d", tmq->consumerId, terrno); + return; } taosWLockLatch(&tmq->lock); @@ -1805,12 +1802,10 @@ static bool doUpdateLocalEp(tmq_t* tmq, int32_t epoch, const SMqAskEpRsp* pRsp) tmq->clientTopics = newTopics; taosWUnLockLatch(&tmq->lock); - int8_t flag = (topicNumGet == 0) ? TMQ_CONSUMER_STATUS__NO_TOPIC : TMQ_CONSUMER_STATUS__READY; - atomic_store_8(&tmq->status, flag); + atomic_store_8(&tmq->status, TMQ_CONSUMER_STATUS__READY); atomic_store_32(&tmq->epoch, epoch); tscInfo("consumer:0x%" PRIx64 " update topic info completed", tmq->consumerId); - return set; } void tmqBuildConsumeReqImpl(SMqPollReq* pReq, tmq_t* tmq, int64_t timeout, SMqClientTopic* pTopic, SMqClientVg* pVg) { @@ -2051,9 +2046,6 @@ static int32_t doTmqPollImpl(tmq_t* pTmq, SMqClientTopic* pTopic, SMqClientVg* p // broadcast the poll request to all related vnodes static int32_t tmqPollImpl(tmq_t* tmq, int64_t timeout) { - if (atomic_load_8(&tmq->status) == TMQ_CONSUMER_STATUS__RECOVER) { - return 0; - } int32_t code = 0; taosWLockLatch(&tmq->lock); @@ -2150,26 +2142,26 @@ static void* tmqHandleAllRsp(tmq_t* tmq, int64_t timeout) { if (pRspWrapper->code != 0) { SMqPollRspWrapper* pollRspWrapper = (SMqPollRspWrapper*)pRspWrapper; - if (pRspWrapper->code == TSDB_CODE_TMQ_CONSUMER_MISMATCH) { - atomic_store_8(&tmq->status, TMQ_CONSUMER_STATUS__RECOVER); - tscDebug("consumer:0x%" PRIx64 " wait for the rebalance, set status to be RECOVER", tmq->consumerId); - } else if (pRspWrapper->code == TSDB_CODE_TQ_NO_COMMITTED_OFFSET) { - tscInfo("consumer:0x%" PRIx64 " return null since no committed offset", tmq->consumerId); - } else { - if (pRspWrapper->code == TSDB_CODE_VND_INVALID_VGROUP_ID) { // for vnode transform - int32_t code = askEp(tmq, NULL, false, true); - if (code != 0) { - tscError("consumer:0x%" PRIx64 " failed to ask ep, code:%s", tmq->consumerId, tstrerror(code)); - } + + if (pRspWrapper->code == TSDB_CODE_VND_INVALID_VGROUP_ID) { // for vnode transform + int32_t code = askEp(tmq, NULL, false, true); + if (code != 0) { + tscError("consumer:0x%" PRIx64 " failed to ask ep, code:%s", tmq->consumerId, tstrerror(code)); + } + } else if (pRspWrapper->code == TSDB_CODE_TMQ_CONSUMER_MISMATCH) { + int32_t code = askEp(tmq, NULL, false, false); + if (code != 0) { + tscError("consumer:0x%" PRIx64 " failed to ask ep, code:%s", tmq->consumerId, tstrerror(code)); } - tscError("consumer:0x%" PRIx64 " msg from vgId:%d discarded, since %s", tmq->consumerId, pollRspWrapper->vgId, - tstrerror(pRspWrapper->code)); - taosWLockLatch(&tmq->lock); - SMqClientVg* pVg = NULL; - getVgInfo(tmq, pollRspWrapper->topicName, pollRspWrapper->vgId, &pVg); - if (pVg) pVg->emptyBlockReceiveTs = taosGetTimestampMs(); - taosWUnLockLatch(&tmq->lock); } + tscInfo("consumer:0x%" PRIx64 " msg from vgId:%d discarded, since %s", tmq->consumerId, pollRspWrapper->vgId, + tstrerror(pRspWrapper->code)); + taosWLockLatch(&tmq->lock); + SMqClientVg* pVg = NULL; + getVgInfo(tmq, pollRspWrapper->topicName, pollRspWrapper->vgId, &pVg); + if (pVg) pVg->emptyBlockReceiveTs = taosGetTimestampMs(); + taosWUnLockLatch(&tmq->lock); + setVgIdle(tmq, pollRspWrapper->topicName, pollRspWrapper->vgId); taosMemoryFreeClear(pollRspWrapper->pEpset); tmqFreeRspWrapper(pRspWrapper); @@ -2388,7 +2380,7 @@ static void* tmqHandleAllRsp(tmq_t* tmq, int64_t timeout) { tscDebug("consumer:0x%" PRIx64 " ep msg received", tmq->consumerId); SMqAskEpRspWrapper* pEpRspWrapper = (SMqAskEpRspWrapper*)pRspWrapper; SMqAskEpRsp* rspMsg = &pEpRspWrapper->msg; - (void)doUpdateLocalEp(tmq, pEpRspWrapper->epoch, rspMsg); + doUpdateLocalEp(tmq, pEpRspWrapper->epoch, rspMsg); tmqFreeRspWrapper(pRspWrapper); taosFreeQitem(pRspWrapper); } else { @@ -2413,23 +2405,6 @@ TAOS_RES* tmq_consumer_poll(tmq_t* tmq, int64_t timeout) { return NULL; } - while (1) { - if (atomic_load_8(&tmq->status) != TMQ_CONSUMER_STATUS__RECOVER) { - break; - } - tscInfo("consumer:0x%" PRIx64 " tmq status is recover", tmq->consumerId); - - int32_t retryCnt = 0; - while (syncAskEp(tmq) != 0) { - if (retryCnt++ > 40) { - return NULL; - } - - tscInfo("consumer:0x%" PRIx64 " not ready, retry:%d/40 in 500ms", tmq->consumerId, retryCnt); - taosMsleep(500); - } - } - (void)atomic_val_compare_exchange_8(&pollFlag, 0, 1); while (1) { @@ -2482,11 +2457,14 @@ static void displayConsumeStatistics(tmq_t* pTmq) { tscDebug("consumer:0x%" PRIx64 " rows dist end", pTmq->consumerId); } -static int32_t innerClose(tmq_t* tmq) { +int32_t tmq_unsubscribe(tmq_t* tmq) { + if (tmq == NULL) return TSDB_CODE_INVALID_PARA; int32_t code = 0; int8_t status = atomic_load_8(&tmq->status); - if (status == TMQ_CONSUMER_STATUS__CLOSED || status != TMQ_CONSUMER_STATUS__READY) { - tscInfo("consumer:0x%" PRIx64 " status:%d, already closed or not in ready state, close it directly", tmq->consumerId, status); + tscInfo("consumer:0x%" PRIx64 " start to unsubscribe consumer, status:%d", tmq->consumerId, status); + + if (status != TMQ_CONSUMER_STATUS__READY) { + tscInfo("consumer:0x%" PRIx64 " status:%d, already closed or not in ready state, no need unsubscribe", tmq->consumerId, status); goto END; } if (tmq->autoCommit) { @@ -2507,23 +2485,18 @@ static int32_t innerClose(tmq_t* tmq) { if(code != 0){ goto END; } - atomic_store_8(&tmq->status, TMQ_CONSUMER_STATUS__CLOSED); END: return code; } -int32_t tmq_unsubscribe(tmq_t* tmq) { - if (tmq == NULL) return TSDB_CODE_INVALID_PARA; - tscInfo("consumer:0x%" PRIx64 " start to unsubscribe consumer, status:%d", tmq->consumerId, tmq->status); - return innerClose(tmq); -} int32_t tmq_consumer_close(tmq_t* tmq) { if (tmq == NULL) return TSDB_CODE_INVALID_PARA; tscInfo("consumer:0x%" PRIx64 " start to close consumer, status:%d", tmq->consumerId, tmq->status); displayConsumeStatistics(tmq); - int32_t code = innerClose(tmq); + int32_t code = tmq_unsubscribe(tmq); if (code == 0) { + atomic_store_8(&tmq->status, TMQ_CONSUMER_STATUS__CLOSED); (void)taosRemoveRef(tmqMgmt.rsetId, tmq->refId); } return code; @@ -2862,7 +2835,7 @@ int32_t askEpCb(void* param, SDataBuf* pMsg, int32_t code) { if (pParam->sync) { SMqAskEpRsp rsp = {0}; if (tDecodeSMqAskEpRsp(POINTER_SHIFT(pMsg->pData, sizeof(SMqRspHead)), &rsp) != NULL) { - (void)doUpdateLocalEp(tmq, head->epoch, &rsp); + doUpdateLocalEp(tmq, head->epoch, &rsp); } tDeleteSMqAskEpRsp(&rsp); } else { diff --git a/source/dnode/mnode/impl/src/mndConsumer.c b/source/dnode/mnode/impl/src/mndConsumer.c index fe33d666b8..57963e62fd 100644 --- a/source/dnode/mnode/impl/src/mndConsumer.c +++ b/source/dnode/mnode/impl/src/mndConsumer.c @@ -577,15 +577,15 @@ int32_t mndProcessSubscribeReq(SRpcMsg *pMsg) { SCMSubscribeReq subscribe = {0}; MND_TMQ_RETURN_CHECK(tDeserializeSCMSubscribeReq(msgStr, &subscribe, pMsg->contLen)); - bool ubSubscribe = (taosArrayGetSize(subscribe.topicNames) == 0); - if(ubSubscribe){ + bool unSubscribe = (taosArrayGetSize(subscribe.topicNames) == 0); + if(unSubscribe){ SMqConsumerObj *pConsumerTmp = NULL; MND_TMQ_RETURN_CHECK(mndAcquireConsumer(pMnode, subscribe.consumerId, &pConsumerTmp)); mndReleaseConsumer(pMnode, pConsumerTmp); } MND_TMQ_RETURN_CHECK(checkAndSortTopic(pMnode, subscribe.topicNames)); pTrans = mndTransCreate(pMnode, TRN_POLICY_RETRY, - (ubSubscribe ? TRN_CONFLICT_NOTHING :TRN_CONFLICT_DB_INSIDE), + (unSubscribe ? TRN_CONFLICT_NOTHING :TRN_CONFLICT_DB_INSIDE), pMsg, "subscribe"); MND_TMQ_NULL_CHECK(pTrans); From 0336d3aa646699864dcd64af5348435d40bb1d9d Mon Sep 17 00:00:00 2001 From: wangmm0220 Date: Wed, 4 Sep 2024 11:35:01 +0800 Subject: [PATCH 04/51] feat:[TD-30270] checkout subscribe logic if unsubscribe twice --- source/dnode/mnode/impl/src/mndConsumer.c | 4 ++++ tests/system-test/7-tmq/subscribeStb3.py | 9 +++------ 2 files changed, 7 insertions(+), 6 deletions(-) diff --git a/source/dnode/mnode/impl/src/mndConsumer.c b/source/dnode/mnode/impl/src/mndConsumer.c index 57963e62fd..15ca11f938 100644 --- a/source/dnode/mnode/impl/src/mndConsumer.c +++ b/source/dnode/mnode/impl/src/mndConsumer.c @@ -581,6 +581,10 @@ int32_t mndProcessSubscribeReq(SRpcMsg *pMsg) { if(unSubscribe){ SMqConsumerObj *pConsumerTmp = NULL; MND_TMQ_RETURN_CHECK(mndAcquireConsumer(pMnode, subscribe.consumerId, &pConsumerTmp)); + if (taosArrayGetSize(pConsumerTmp->assignedTopics) == 0){ + mndReleaseConsumer(pMnode, pConsumerTmp); + return 0; + } mndReleaseConsumer(pMnode, pConsumerTmp); } MND_TMQ_RETURN_CHECK(checkAndSortTopic(pMnode, subscribe.topicNames)); diff --git a/tests/system-test/7-tmq/subscribeStb3.py b/tests/system-test/7-tmq/subscribeStb3.py index ed44ab1fb1..5df58d8122 100644 --- a/tests/system-test/7-tmq/subscribeStb3.py +++ b/tests/system-test/7-tmq/subscribeStb3.py @@ -20,12 +20,9 @@ class actionType(Enum): class TDTestCase: hostname = socket.gethostname() - #rpcDebugFlagVal = '143' - #clientCfgDict = {'serverPort': '', 'firstEp': '', 'secondEp':'', 'rpcDebugFlag':'135', 'fqdn':''} - #clientCfgDict["rpcDebugFlag"] = rpcDebugFlagVal - #updatecfgDict = {'clientCfg': {}, 'serverPort': '', 'firstEp': '', 'secondEp':'', 'rpcDebugFlag':'135', 'fqdn':''} - #updatecfgDict["rpcDebugFlag"] = rpcDebugFlagVal - #print ("===================: ", updatecfgDict) + clientCfgDict = {'debugFlag': 135} + updatecfgDict = {'debugFlag': 135, 'asynclog': 0} + updatecfgDict["clientCfg"] = clientCfgDict def init(self, conn, logSql, replicaVar=1): self.replicaVar = int(replicaVar) From 4554b3b33e716eb8e274c24a324cbf463b97fcb2 Mon Sep 17 00:00:00 2001 From: wangmm0220 Date: Wed, 4 Sep 2024 13:58:19 +0800 Subject: [PATCH 05/51] feat:[TD-30270] checkout subscribe logic if unsubscribe twice --- source/dnode/mnode/impl/src/mndConsumer.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/dnode/mnode/impl/src/mndConsumer.c b/source/dnode/mnode/impl/src/mndConsumer.c index 15ca11f938..c5a440b549 100644 --- a/source/dnode/mnode/impl/src/mndConsumer.c +++ b/source/dnode/mnode/impl/src/mndConsumer.c @@ -583,7 +583,7 @@ int32_t mndProcessSubscribeReq(SRpcMsg *pMsg) { MND_TMQ_RETURN_CHECK(mndAcquireConsumer(pMnode, subscribe.consumerId, &pConsumerTmp)); if (taosArrayGetSize(pConsumerTmp->assignedTopics) == 0){ mndReleaseConsumer(pMnode, pConsumerTmp); - return 0; + goto END; } mndReleaseConsumer(pMnode, pConsumerTmp); } From bd83023b09335d441e5e543b6b9f666dce173ec3 Mon Sep 17 00:00:00 2001 From: wangmm0220 Date: Wed, 4 Sep 2024 18:26:24 +0800 Subject: [PATCH 06/51] feat:[TD-30270] checkout subscribe logic if unsubscribe twice --- include/util/taoserror.h | 1 + source/dnode/mnode/impl/src/mndConsumer.c | 8 ++- source/util/src/terror.c | 1 + tests/parallel_test/cases.task | 1 + tests/system-test/7-tmq/td-30270.py | 80 +++++++++++++++++++++++ 5 files changed, 90 insertions(+), 1 deletion(-) create mode 100644 tests/system-test/7-tmq/td-30270.py diff --git a/include/util/taoserror.h b/include/util/taoserror.h index 4591c7fbcc..16027730f7 100644 --- a/include/util/taoserror.h +++ b/include/util/taoserror.h @@ -962,6 +962,7 @@ int32_t taosGetErrSize(); #define TSDB_CODE_TMQ_REPLAY_NEED_ONE_VGROUP TAOS_DEF_ERROR_CODE(0, 0x4013) #define TSDB_CODE_TMQ_REPLAY_NOT_SUPPORT TAOS_DEF_ERROR_CODE(0, 0x4014) #define TSDB_CODE_TMQ_NO_TABLE_QUALIFIED TAOS_DEF_ERROR_CODE(0, 0x4015) +#define TSDB_CODE_TMQ_NO_NEED_REBALANCE TAOS_DEF_ERROR_CODE(0, 0x4016) // stream #define TSDB_CODE_STREAM_TASK_NOT_EXIST TAOS_DEF_ERROR_CODE(0, 0x4100) diff --git a/source/dnode/mnode/impl/src/mndConsumer.c b/source/dnode/mnode/impl/src/mndConsumer.c index c5a440b549..7639349bac 100644 --- a/source/dnode/mnode/impl/src/mndConsumer.c +++ b/source/dnode/mnode/impl/src/mndConsumer.c @@ -511,6 +511,11 @@ static int32_t getTopicAddDelete(SMqConsumerObj *pExistedConsumer, SMqConsumerOb } } } + // no topics need to be rebalanced + if (taosArrayGetSize(pConsumerNew->rebNewTopics) == 0 && taosArrayGetSize(pConsumerNew->rebRemovedTopics) == 0) { + code = TSDB_CODE_TMQ_NO_NEED_REBALANCE; + } + END: return code; } @@ -603,7 +608,8 @@ END: mndTransDrop(pTrans); tDeleteSMqConsumerObj(pConsumerNew); taosArrayDestroyP(subscribe.topicNames, (FDelete)taosMemoryFree); - return code; + + return code == TSDB_CODE_TMQ_NO_NEED_REBALANCE ? 0 : code; } SSdbRaw *mndConsumerActionEncode(SMqConsumerObj *pConsumer) { diff --git a/source/util/src/terror.c b/source/util/src/terror.c index 58dde5cd23..f85c76f157 100644 --- a/source/util/src/terror.c +++ b/source/util/src/terror.c @@ -806,6 +806,7 @@ TAOS_DEFINE_ERROR(TSDB_CODE_TMQ_SAME_COMMITTED_VALUE, "Same committed valu TAOS_DEFINE_ERROR(TSDB_CODE_TMQ_REPLAY_NEED_ONE_VGROUP, "Replay need only one vgroup if subscribe super table") TAOS_DEFINE_ERROR(TSDB_CODE_TMQ_REPLAY_NOT_SUPPORT, "Replay is disabled if subscribe db or stable") TAOS_DEFINE_ERROR(TSDB_CODE_TMQ_NO_TABLE_QUALIFIED, "No table qualified for query") +TAOS_DEFINE_ERROR(TSDB_CODE_TMQ_NO_NEED_REBALANCE, "No need rebalance") // stream TAOS_DEFINE_ERROR(TSDB_CODE_STREAM_TASK_NOT_EXIST, "Stream task not exist") diff --git a/tests/parallel_test/cases.task b/tests/parallel_test/cases.task index 43193128fa..00b17ec73f 100644 --- a/tests/parallel_test/cases.task +++ b/tests/parallel_test/cases.task @@ -229,6 +229,7 @@ ,,y,system-test,./pytest.sh python3 ./test.py -f 7-tmq/basic5.py ,,y,system-test,./pytest.sh python3 ./test.py -f 7-tmq/subscribeDb.py -N 3 -n 3 ,,y,system-test,./pytest.sh python3 ./test.py -f 7-tmq/ts-4674.py -N 3 -n 3 +,,y,system-test,./pytest.sh python3 ./test.py -f 7-tmq/td-30270.py ,,y,system-test,./pytest.sh python3 ./test.py -f 7-tmq/subscribeDb1.py ,,y,system-test,./pytest.sh python3 ./test.py -f 7-tmq/subscribeDb2.py ,,y,system-test,./pytest.sh python3 ./test.py -f 7-tmq/subscribeDb3.py diff --git a/tests/system-test/7-tmq/td-30270.py b/tests/system-test/7-tmq/td-30270.py new file mode 100644 index 0000000000..73c7a88f65 --- /dev/null +++ b/tests/system-test/7-tmq/td-30270.py @@ -0,0 +1,80 @@ + +import taos +import sys +import time +import socket +import os +import threading + +from util.log import * +from util.sql import * +from util.cases import * +from util.dnodes import * +from util.common import * +from taos.tmq import * +sys.path.append("./7-tmq") +from tmqCommon import * + +class TDTestCase: + clientCfgDict = {'debugFlag': 135} + updatecfgDict = {'debugFlag': 135, 'clientCfg':clientCfgDict} + # updatecfgDict = {'debugFlag': 135, 'clientCfg':clientCfgDict, 'tmqRowSize':1} + + def init(self, conn, logSql, replicaVar=1): + tdLog.debug(f"start to excute {__file__}") + tdSql.init(conn.cursor()) + + def consume_test(self): + + tdSql.execute(f'create database if not exists d1') + tdSql.execute(f'use d1') + tdSql.execute(f'create table st(ts timestamp, i int) tags(t int)') + tdSql.execute(f'insert into t1 using st tags(1) values(now, 1) (now+1s, 2)') + tdSql.execute(f'insert into t2 using st tags(2) values(now, 1) (now+1s, 2)') + tdSql.execute(f'insert into t3 using st tags(3) values(now, 1) (now+1s, 2)') + + + tdSql.execute(f'create topic topic_all as select * from st') + consumer_dict = { + "group.id": "g1", + "td.connect.user": "root", + "td.connect.pass": "taosdata", + "auto.offset.reset": "earliest", + } + consumer = Consumer(consumer_dict) + + try: + consumer.unsubscribe() + consumer.unsubscribe() + consumer.subscribe(["topic_all"]) + consumer.subscribe(["topic_all"]) + except TmqError: + tdLog.exit(f"subscribe error") + + cnt = 0 + try: + while True: + res = consumer.poll(2) + if not res: + break + val = res.value() + if val is None: + print(f"null val") + continue + for block in val: + cnt += len(block.fetchall()) + + print(f"block {cnt} rows") + + finally: + consumer.unsubscribe(); + consumer.close() + def run(self): + self.consume_test() + + def stop(self): + tdSql.close() + tdLog.success(f"{__file__} successfully executed") + +tdCases.addLinux(__file__, TDTestCase()) +tdCases.addWindows(__file__, TDTestCase()) From 0cd9e274ab8c302171a489bed0ce2ef62af4748a Mon Sep 17 00:00:00 2001 From: wangmm0220 Date: Thu, 5 Sep 2024 14:32:11 +0800 Subject: [PATCH 07/51] feat:[TD-30270] opti close logic in tmq --- include/common/tmsg.h | 1 + source/client/src/clientTmq.c | 376 +++++++++++----------- source/common/src/tmsg.c | 5 + source/dnode/mnode/impl/src/mndConsumer.c | 3 +- 4 files changed, 202 insertions(+), 183 deletions(-) diff --git a/include/common/tmsg.h b/include/common/tmsg.h index 010e538fb6..9db4496377 100644 --- a/include/common/tmsg.h +++ b/include/common/tmsg.h @@ -4158,6 +4158,7 @@ typedef struct { typedef struct { SArray* topicPrivileges; // SArray + int32_t debugFlag; } SMqHbRsp; typedef struct { diff --git a/source/client/src/clientTmq.c b/source/client/src/clientTmq.c index 4367ca4124..b702f79ddf 100644 --- a/source/client/src/clientTmq.c +++ b/source/client/src/clientTmq.c @@ -24,6 +24,13 @@ #include "tref.h" #include "ttimer.h" +#define tqFatalC(...) do { if (tqDebugFlag & DEBUG_FATAL) { taosPrintLog("TQ FATAL ", DEBUG_FATAL, tqDebugFlag, __VA_ARGS__); }} while(0) +#define tqErrorC(...) do { if (tqDebugFlag & DEBUG_ERROR) { taosPrintLog("TQ ERROR ", DEBUG_ERROR, tqDebugFlag, __VA_ARGS__); }} while(0) +#define tqWarnC(...) do { if (tqDebugFlag & DEBUG_WARN) { taosPrintLog("TQ WARN ", DEBUG_WARN, tqDebugFlag, __VA_ARGS__); }} while(0) +#define tqInfoC(...) do { if (tqDebugFlag & DEBUG_INFO) { taosPrintLog("TQ ", DEBUG_INFO, tqDebugFlag, __VA_ARGS__); }} while(0) +#define tqDebugC(...) do { if (tqDebugFlag & DEBUG_DEBUG) { taosPrintLog("TQ ", DEBUG_DEBUG, tqDebugFlag, __VA_ARGS__); }} while(0) +#define tqTraceC(...) do { if (tqDebugFlag & DEBUG_TRACE) { taosPrintLog("TQ ", DEBUG_TRACE, tqDebugFlag, __VA_ARGS__); }} while(0) + #define EMPTY_BLOCK_POLL_IDLE_DURATION 10 #define DEFAULT_AUTO_COMMIT_INTERVAL 5000 #define DEFAULT_HEARTBEAT_INTERVAL 3000 @@ -580,7 +587,7 @@ static int32_t getTopicByName(tmq_t* tmq, const char* pTopicName, SMqClientTopic return 0; } - tscError("consumer:0x%" PRIx64 ", total:%d, failed to find topic:%s", tmq->consumerId, numOfTopics, pTopicName); + tqErrorC("consumer:0x%" PRIx64 ", total:%d, failed to find topic:%s", tmq->consumerId, numOfTopics, pTopicName); return TSDB_CODE_TMQ_INVALID_TOPIC; } @@ -604,7 +611,7 @@ static int32_t getClientVg(tmq_t* tmq, char* pTopicName, int32_t vgId, SMqClient SMqClientTopic* pTopic = NULL; int32_t code = getTopicByName(tmq, pTopicName, &pTopic); if (code != 0) { - tscError("consumer:0x%" PRIx64 " invalid topic name:%s", tmq->consumerId, pTopicName); + tqErrorC("consumer:0x%" PRIx64 " invalid topic name:%s", tmq->consumerId, pTopicName); return code; } @@ -638,12 +645,12 @@ static int32_t innerCommit(tmq_t* tmq, char* pTopicName, STqOffsetVal* offsetVal code = doSendCommitMsg(tmq, pVg->vgId, &pVg->epSet, offsetVal, pTopicName, pParamSet); if (code != TSDB_CODE_SUCCESS) { - tscError("consumer:0x%" PRIx64 " topic:%s on vgId:%d end commit msg failed, send offset:%s committed:%s, code:%s", + tqErrorC("consumer:0x%" PRIx64 " topic:%s on vgId:%d end commit msg failed, send offset:%s committed:%s, code:%s", tmq->consumerId, pTopicName, pVg->vgId, offsetBuf, commitBuf, tstrerror(terrno)); return code; } - tscInfo("consumer:0x%" PRIx64 " topic:%s on vgId:%d send commit msg success, send offset:%s committed:%s", + tqDebugC("consumer:0x%" PRIx64 " topic:%s on vgId:%d send commit msg success, send offset:%s committed:%s", tmq->consumerId, pTopicName, pVg->vgId, offsetBuf, commitBuf); tOffsetCopy(&pVg->offsetInfo.committedOffset, offsetVal); return code; @@ -651,7 +658,7 @@ static int32_t innerCommit(tmq_t* tmq, char* pTopicName, STqOffsetVal* offsetVal static int32_t asyncCommitOffset(tmq_t* tmq, char* pTopicName, int32_t vgId, STqOffsetVal* offsetVal, tmq_commit_cb* pCommitFp, void* userParam) { - tscInfo("consumer:0x%" PRIx64 " do manual commit offset for %s, vgId:%d", tmq->consumerId, pTopicName, vgId); + tqInfoC("consumer:0x%" PRIx64 " do manual commit offset for %s, vgId:%d", tmq->consumerId, pTopicName, vgId); SMqCommitCbParamSet* pParamSet = NULL; int32_t code = prepareCommitCbParamSet(tmq, pCommitFp, userParam, 0, &pParamSet); if (code != 0){ @@ -721,7 +728,7 @@ static int32_t innerCommitAll(tmq_t* tmq, SMqCommitCbParamSet* pParamSet){ int32_t code = 0; taosRLockLatch(&tmq->lock); int32_t numOfTopics = taosArrayGetSize(tmq->clientTopics); - tscDebug("consumer:0x%" PRIx64 " start to commit offset for %d topics", tmq->consumerId, numOfTopics); + tqDebugC("consumer:0x%" PRIx64 " start to commit offset for %d topics", tmq->consumerId, numOfTopics); for (int32_t i = 0; i < numOfTopics; i++) { SMqClientTopic* pTopic = taosArrayGet(tmq->clientTopics, i); @@ -730,7 +737,7 @@ static int32_t innerCommitAll(tmq_t* tmq, SMqCommitCbParamSet* pParamSet){ goto END; } int32_t numOfVgroups = taosArrayGetSize(pTopic->vgs); - tscDebug("consumer:0x%" PRIx64 " commit offset for topics:%s, numOfVgs:%d", tmq->consumerId, pTopic->topicName, numOfVgroups); + tqDebugC("consumer:0x%" PRIx64 " commit offset for topics:%s, numOfVgs:%d", tmq->consumerId, pTopic->topicName, numOfVgroups); for (int32_t j = 0; j < numOfVgroups; j++) { SMqClientVg* pVg = taosArrayGet(pTopic->vgs, j); if (pVg == NULL) { @@ -740,12 +747,12 @@ static int32_t innerCommitAll(tmq_t* tmq, SMqCommitCbParamSet* pParamSet){ code = innerCommit(tmq, pTopic->topicName, &pVg->offsetInfo.endOffset, pVg, pParamSet); if (code != 0){ - tscDebug("consumer:0x%" PRIx64 " topic:%s vgId:%d, no commit, code:%s, current:%" PRId64 ", ordinal:%d/%d", + tqDebugC("consumer:0x%" PRIx64 " topic:%s vgId:%d, no commit, code:%s, current offset version:%" PRId64 ", ordinal:%d/%d", tmq->consumerId, pTopic->topicName, pVg->vgId, tstrerror(code), pVg->offsetInfo.endOffset.version, j + 1, numOfVgroups); } } } - tscDebug("consumer:0x%" PRIx64 " total commit:%d for %d topics", tmq->consumerId, pParamSet->waitingRspNum - DEFAULT_COMMIT_CNT, + tqDebugC("consumer:0x%" PRIx64 " total commit:%d for %d topics", tmq->consumerId, pParamSet->waitingRspNum - DEFAULT_COMMIT_CNT, numOfTopics); END: taosRUnLockLatch(&tmq->lock); @@ -758,7 +765,7 @@ static void asyncCommitAllOffsets(tmq_t* tmq, tmq_commit_cb* pCommitFp, void* us // init waitingRspNum as DEFAULT_COMMIT_CNT to prevent concurrency issue code = prepareCommitCbParamSet(tmq, pCommitFp, userParam, DEFAULT_COMMIT_CNT, &pParamSet); if (code != 0) { - tscError("consumer:0x%" PRIx64 " prepareCommitCbParamSet failed, code:%s", tmq->consumerId, tstrerror(code)); + tqErrorC("consumer:0x%" PRIx64 " prepareCommitCbParamSet failed, code:%s", tmq->consumerId, tstrerror(code)); if (pCommitFp != NULL) { pCommitFp(tmq, code, userParam); } @@ -766,12 +773,12 @@ static void asyncCommitAllOffsets(tmq_t* tmq, tmq_commit_cb* pCommitFp, void* us } code = innerCommitAll(tmq, pParamSet); if (code != 0){ - tscError("consumer:0x%" PRIx64 " innerCommitAll failed, code:%s", tmq->consumerId, tstrerror(code)); + tqErrorC("consumer:0x%" PRIx64 " innerCommitAll failed, code:%s", tmq->consumerId, tstrerror(code)); } code = commitRspCountDown(pParamSet, tmq->consumerId, "init", -1); if (code != 0) { - tscError("consumer:0x%" PRIx64 " commit rsp count down failed, code:%s", tmq->consumerId, tstrerror(code)); + tqErrorC("consumer:0x%" PRIx64 " commit rsp count down failed, code:%s", tmq->consumerId, tstrerror(code)); } return; } @@ -840,7 +847,7 @@ int32_t tmqHbCb(void* param, SDataBuf* pMsg, int32_t code) { for (int32_t j = 0; j < topicNumCur; j++) { SMqClientTopic* pTopicCur = taosArrayGet(tmq->clientTopics, j); if (pTopicCur && strcmp(pTopicCur->topicName, privilege->topic) == 0) { - tscInfo("consumer:0x%" PRIx64 ", has no privilege, topic:%s", tmq->consumerId, privilege->topic); + tqInfoC("consumer:0x%" PRIx64 ", has no privilege, topic:%s", tmq->consumerId, privilege->topic); pTopicCur->noPrivilege = 1; } } @@ -850,6 +857,7 @@ int32_t tmqHbCb(void* param, SDataBuf* pMsg, int32_t code) { (void)taosReleaseRef(tmqMgmt.rsetId, refId); } + tqDebugFlag = rsp.debugFlag; tDestroySMqHbRsp(&rsp); END: @@ -905,7 +913,7 @@ void tmqSendHbReq(void* param, void* tmrId) { offRows->ever = pVg->offsetInfo.walVerEnd == -1 ? 0 : pVg->offsetInfo.walVerEnd; char buf[TSDB_OFFSET_LEN] = {0}; tFormatOffset(buf, TSDB_OFFSET_LEN, &offRows->offset); - tscDebug("consumer:0x%" PRIx64 ",report offset, group:%s vgId:%d, offset:%s/%" PRId64 ", rows:%" PRId64, + tqDebugC("consumer:0x%" PRIx64 ",report offset, group:%s vgId:%d, offset:%s/%" PRId64 ", rows:%" PRId64, tmq->consumerId, tmq->groupId, offRows->vgId, buf, offRows->ever, offRows->rows); } } @@ -913,18 +921,18 @@ void tmqSendHbReq(void* param, void* tmrId) { int32_t tlen = tSerializeSMqHbReq(NULL, 0, &req); if (tlen < 0) { - tscError("tSerializeSMqHbReq failed"); + tqErrorC("tSerializeSMqHbReq failed"); goto END; } void* pReq = taosMemoryCalloc(1, tlen); if (tlen < 0) { - tscError("failed to malloc MqHbReq msg, size:%d", tlen); + tqErrorC("failed to malloc MqHbReq msg, size:%d", tlen); goto END; } if (tSerializeSMqHbReq(pReq, tlen, &req) < 0) { - tscError("tSerializeSMqHbReq %d failed", tlen); + tqErrorC("tSerializeSMqHbReq %d failed", tlen); taosMemoryFree(pReq); goto END; } @@ -948,7 +956,7 @@ void tmqSendHbReq(void* param, void* tmrId) { int64_t transporterId = 0; int32_t code = asyncSendMsgToServer(tmq->pTscObj->pAppInfo->pTransporter, &epSet, &transporterId, sendInfo); if (code != 0) { - tscError("tmqSendHbReq asyncSendMsgToServer failed"); + tqErrorC("tmqSendHbReq asyncSendMsgToServer failed"); } (void)atomic_val_compare_exchange_8(&pollFlag, 1, 0); @@ -962,7 +970,7 @@ END: static void defaultCommitCbFn(tmq_t* pTmq, int32_t code, void* param) { if (code != 0) { - tscError("consumer:0x%" PRIx64 ", failed to commit offset, code:%s", pTmq->consumerId, tstrerror(code)); + tqErrorC("consumer:0x%" PRIx64 ", failed to commit offset, code:%s", pTmq->consumerId, tstrerror(code)); } } @@ -972,7 +980,7 @@ void tmqHandleAllDelayedTask(tmq_t* pTmq) { code = taosAllocateQall(&qall); if (code) { - tscError("consumer:0x%" PRIx64 ", failed to allocate qall, code:%s", pTmq->consumerId, tstrerror(code)); + tqErrorC("consumer:0x%" PRIx64 ", failed to allocate qall, code:%s", pTmq->consumerId, tstrerror(code)); return; } @@ -984,26 +992,26 @@ void tmqHandleAllDelayedTask(tmq_t* pTmq) { return; } - tscDebug("consumer:0x%" PRIx64 " handle delayed %d tasks before poll data", pTmq->consumerId, numOfItems); + tqDebugC("consumer:0x%" PRIx64 " handle delayed %d tasks before poll data", pTmq->consumerId, numOfItems); int8_t* pTaskType = NULL; (void)taosGetQitem(qall, (void**)&pTaskType); while (pTaskType != NULL) { if (*pTaskType == TMQ_DELAYED_TASK__ASK_EP) { - tscDebug("consumer:0x%" PRIx64 " retrieve ask ep timer", pTmq->consumerId); + tqDebugC("consumer:0x%" PRIx64 " retrieve ask ep timer", pTmq->consumerId); code = askEp(pTmq, NULL, false, false); if (code != 0) { - tscError("consumer:0x%" PRIx64 " failed to ask ep, code:%s", pTmq->consumerId, tstrerror(code)); + tqErrorC("consumer:0x%" PRIx64 " failed to ask ep, code:%s", pTmq->consumerId, tstrerror(code)); } (void)taosTmrReset(tmqAssignAskEpTask, DEFAULT_ASKEP_INTERVAL, (void*)(pTmq->refId), tmqMgmt.timer, &pTmq->epTimer); } else if (*pTaskType == TMQ_DELAYED_TASK__COMMIT) { tmq_commit_cb* pCallbackFn = (pTmq->commitCb != NULL) ? pTmq->commitCb : defaultCommitCbFn; asyncCommitAllOffsets(pTmq, pCallbackFn, pTmq->commitCbUserParam); - tscDebug("consumer:0x%" PRIx64 " next commit to vnode(s) in %.2fs", pTmq->consumerId, + tqDebugC("consumer:0x%" PRIx64 " next commit to vnode(s) in %.2fs", pTmq->consumerId, pTmq->autoCommitInterval / 1000.0); (void)taosTmrReset(tmqAssignDelayedCommitTask, pTmq->autoCommitInterval, (void*)(pTmq->refId), tmqMgmt.timer, &pTmq->commitTimer); } else { - tscError("consumer:0x%" PRIx64 " invalid task type:%d", pTmq->consumerId, *pTaskType); + tqErrorC("consumer:0x%" PRIx64 " invalid task type:%d", pTmq->consumerId, *pTaskType); } taosFreeQitem(pTaskType); @@ -1090,16 +1098,16 @@ int32_t tmq_subscription(tmq_t* tmq, tmq_list_t** topics) { for (int i = 0; i < taosArrayGetSize(tmq->clientTopics); i++) { SMqClientTopic* topic = taosArrayGet(tmq->clientTopics, i); if (topic == NULL) { - tscError("topic is null"); + tqErrorC("topic is null"); continue; } char* tmp = strchr(topic->topicName, '.'); if (tmp == NULL) { - tscError("topic name is invalid:%s", topic->topicName); + tqErrorC("topic name is invalid:%s", topic->topicName); continue; } if (tmq_list_append(*topics, tmp + 1) != 0) { - tscError("failed to append topic:%s", tmp + 1); + tqErrorC("failed to append topic:%s", tmp + 1); continue; } } @@ -1149,7 +1157,7 @@ void tmqFreeImpl(void* handle) { } taosMemoryFree(tmq); - tscInfo("consumer:0x%" PRIx64 " closed", id); + tqInfoC("consumer:0x%" PRIx64 " closed", id); } static void tmqMgmtInit(void) { @@ -1200,7 +1208,7 @@ tmq_t* tmq_consumer_new(tmq_conf_t* conf, char* errstr, int32_t errstrLen) { tmq_t* pTmq = taosMemoryCalloc(1, sizeof(tmq_t)); if (pTmq == NULL) { - tscError("failed to create consumer, groupId:%s", conf->groupId); + tqErrorC("failed to create consumer, groupId:%s", conf->groupId); SET_ERROR_MSG_TMQ("malloc tmq failed") return NULL; } @@ -1210,13 +1218,13 @@ tmq_t* tmq_consumer_new(tmq_conf_t* conf, char* errstr, int32_t errstrLen) { pTmq->clientTopics = taosArrayInit(0, sizeof(SMqClientTopic)); if (pTmq->clientTopics == NULL) { - tscError("failed to create consumer, groupId:%s", conf->groupId); + tqErrorC("failed to create consumer, groupId:%s", conf->groupId); SET_ERROR_MSG_TMQ("malloc client topics failed") goto _failed; } code = taosOpenQueue(&pTmq->mqueue); if (code) { - tscError("consumer:0x%" PRIx64 " setup failed since %s, groupId:%s", pTmq->consumerId, tstrerror(code), + tqErrorC("consumer:0x%" PRIx64 " setup failed since %s, groupId:%s", pTmq->consumerId, tstrerror(code), pTmq->groupId); SET_ERROR_MSG_TMQ("open queue failed") goto _failed; @@ -1224,7 +1232,7 @@ tmq_t* tmq_consumer_new(tmq_conf_t* conf, char* errstr, int32_t errstrLen) { code = taosOpenQueue(&pTmq->delayedTask); if (code) { - tscError("consumer:0x%" PRIx64 " setup failed since %s, groupId:%s", pTmq->consumerId, tstrerror(code), + tqErrorC("consumer:0x%" PRIx64 " setup failed since %s, groupId:%s", pTmq->consumerId, tstrerror(code), pTmq->groupId); SET_ERROR_MSG_TMQ("open delayed task queue failed") goto _failed; @@ -1232,14 +1240,14 @@ tmq_t* tmq_consumer_new(tmq_conf_t* conf, char* errstr, int32_t errstrLen) { code = taosAllocateQall(&pTmq->qall); if (code) { - tscError("consumer:0x%" PRIx64 " setup failed since %s, groupId:%s", pTmq->consumerId, tstrerror(code), + tqErrorC("consumer:0x%" PRIx64 " setup failed since %s, groupId:%s", pTmq->consumerId, tstrerror(code), pTmq->groupId); SET_ERROR_MSG_TMQ("allocate qall failed") goto _failed; } if (conf->groupId[0] == 0) { - tscError("consumer:0x%" PRIx64 " setup failed since %s, groupId:%s", pTmq->consumerId, tstrerror(code), + tqErrorC("consumer:0x%" PRIx64 " setup failed since %s, groupId:%s", pTmq->consumerId, tstrerror(code), pTmq->groupId); SET_ERROR_MSG_TMQ("malloc tmq element failed or group is empty") goto _failed; @@ -1280,7 +1288,7 @@ tmq_t* tmq_consumer_new(tmq_conf_t* conf, char* errstr, int32_t errstrLen) { // init semaphore if (tsem2_init(&pTmq->rspSem, 0, 0) != 0) { - tscError("consumer:0x %" PRIx64 " setup failed since %s, consumer group %s", pTmq->consumerId, + tqErrorC("consumer:0x %" PRIx64 " setup failed since %s, consumer group %s", pTmq->consumerId, tstrerror(TAOS_SYSTEM_ERROR(errno)), pTmq->groupId); SET_ERROR_MSG_TMQ("init t_sem failed") goto _failed; @@ -1290,7 +1298,7 @@ tmq_t* tmq_consumer_new(tmq_conf_t* conf, char* errstr, int32_t errstrLen) { code = taos_connect_internal(conf->ip, user, pass, NULL, NULL, conf->port, CONN_TYPE__TMQ, &pTmq->pTscObj); if (code) { terrno = code; - tscError("consumer:0x%" PRIx64 " setup failed since %s, groupId:%s", pTmq->consumerId, terrstr(), pTmq->groupId); + tqErrorC("consumer:0x%" PRIx64 " setup failed since %s, groupId:%s", pTmq->consumerId, terrstr(), pTmq->groupId); SET_ERROR_MSG_TMQ("init tscObj failed") goto _failed; } @@ -1309,7 +1317,7 @@ tmq_t* tmq_consumer_new(tmq_conf_t* conf, char* errstr, int32_t errstrLen) { char buf[TSDB_OFFSET_LEN] = {0}; STqOffsetVal offset = {.type = pTmq->resetOffsetCfg}; tFormatOffset(buf, tListLen(buf), &offset); - tscInfo("consumer:0x%" PRIx64 " is setup, refId:%" PRId64 + tqInfoC("consumer:0x%" PRIx64 " is setup, refId:%" PRId64 ", groupId:%s, snapshot:%d, autoCommit:%d, commitInterval:%dms, offset:%s", pTmq->consumerId, pTmq->refId, pTmq->groupId, pTmq->useSnapshot, pTmq->autoCommit, pTmq->autoCommitInterval, buf); @@ -1330,7 +1338,7 @@ int32_t tmq_subscribe(tmq_t* tmq, const tmq_list_t* topic_list) { SCMSubscribeReq req = {0}; int32_t code = 0; - tscInfo("consumer:0x%" PRIx64 " cgroup:%s, subscribe %d topics", tmq->consumerId, tmq->groupId, sz); + tqInfoC("consumer:0x%" PRIx64 " cgroup:%s, subscribe %d topics", tmq->consumerId, tmq->groupId, sz); req.consumerId = tmq->consumerId; tstrncpy(req.clientId, tmq->clientId, TSDB_CLIENT_ID_LEN); @@ -1362,7 +1370,7 @@ int32_t tmq_subscribe(tmq_t* tmq, const tmq_list_t* topic_list) { SName name = {0}; code = tNameSetDbName(&name, tmq->pTscObj->acctId, topic, strlen(topic)); if (code) { - tscError("consumer:0x%" PRIx64 " cgroup:%s, failed to set topic name, code:%d", tmq->consumerId, tmq->groupId, + tqErrorC("consumer:0x%" PRIx64 " cgroup:%s, failed to set topic name, code:%d", tmq->consumerId, tmq->groupId, code); goto END; } @@ -1374,7 +1382,7 @@ int32_t tmq_subscribe(tmq_t* tmq, const tmq_list_t* topic_list) { code = tNameExtractFullName(&name, topicFName); if (code) { - tscError("consumer:0x%" PRIx64 " cgroup:%s, failed to extract topic name, code:%d", tmq->consumerId, tmq->groupId, + tqErrorC("consumer:0x%" PRIx64 " cgroup:%s, failed to extract topic name, code:%d", tmq->consumerId, tmq->groupId, code); taosMemoryFree(topicFName); goto END; @@ -1385,7 +1393,7 @@ int32_t tmq_subscribe(tmq_t* tmq, const tmq_list_t* topic_list) { taosMemoryFree(topicFName); goto END; } - tscInfo("consumer:0x%" PRIx64 " subscribe topic:%s", tmq->consumerId, topicFName); + tqInfoC("consumer:0x%" PRIx64 " subscribe topic:%s", tmq->consumerId, topicFName); } int32_t tlen = tSerializeSCMSubscribeReq(NULL, &req); @@ -1439,7 +1447,7 @@ int32_t tmq_subscribe(tmq_t* tmq, const tmq_list_t* topic_list) { int32_t retryCnt = 0; while ((code = syncAskEp(tmq)) != 0) { if (retryCnt++ > SUBSCRIBE_RETRY_MAX_COUNT || code == TSDB_CODE_MND_CONSUMER_NOT_EXIST) { - tscError("consumer:0x%" PRIx64 ", mnd not ready for subscribe, retry more than 2 minutes, code:%s", + tqErrorC("consumer:0x%" PRIx64 ", mnd not ready for subscribe, retry more than 2 minutes, code:%s", tmq->consumerId, tstrerror(code)); if (code == TSDB_CODE_MND_CONSUMER_NOT_EXIST) { code = 0; @@ -1447,7 +1455,7 @@ int32_t tmq_subscribe(tmq_t* tmq, const tmq_list_t* topic_list) { goto END; } - tscInfo("consumer:0x%" PRIx64 ", mnd not ready for subscribe, retry:%d in 500ms", tmq->consumerId, retryCnt); + tqInfoC("consumer:0x%" PRIx64 ", mnd not ready for subscribe, retry:%d in 500ms", tmq->consumerId, retryCnt); taosMsleep(SUBSCRIBE_RETRY_INTERVAL); } @@ -1548,7 +1556,7 @@ int32_t tmqPollCb(void* param, SDataBuf* pMsg, int32_t code) { } if (pMsg->pData == NULL) { - tscError("consumer:0x%" PRIx64 " msg discard from vgId:%d, since msg is NULL", tmq->consumerId, vgId); + tqErrorC("consumer:0x%" PRIx64 " msg discard from vgId:%d, since msg is NULL", tmq->consumerId, vgId); code = TSDB_CODE_TSC_INTERNAL_ERROR; goto END; } @@ -1565,7 +1573,7 @@ int32_t tmqPollCb(void* param, SDataBuf* pMsg, int32_t code) { } if (msgEpoch != clientEpoch) { - tscError("consumer:0x%" PRIx64 + tqErrorC("consumer:0x%" PRIx64 " msg discard from vgId:%d since from earlier epoch, rsp epoch %d, current epoch %d, reqId:0x%" PRIx64, tmq->consumerId, vgId, msgEpoch, clientEpoch, requestId); code = TSDB_CODE_TMQ_CONSUMER_MISMATCH; @@ -1591,7 +1599,7 @@ int32_t tmqPollCb(void* param, SDataBuf* pMsg, int32_t code) { char buf[TSDB_OFFSET_LEN] = {0}; tFormatOffset(buf, TSDB_OFFSET_LEN, &pRspWrapper->dataRsp.common.rspOffset); - tscDebug("consumer:0x%" PRIx64 " recv poll rsp, vgId:%d, req ver:%" PRId64 ", rsp:%s type %d,QID:0x%" PRIx64, + tqDebugC("consumer:0x%" PRIx64 " recv poll rsp, vgId:%d, req ver:%" PRId64 ", rsp:%s type %d,QID:0x%" PRIx64, tmq->consumerId, vgId, pRspWrapper->dataRsp.common.reqOffset.version, buf, rspType, requestId); } else if (rspType == TMQ_MSG_TYPE__POLL_META_RSP) { SDecoder decoder = {0}; @@ -1623,10 +1631,10 @@ int32_t tmqPollCb(void* param, SDataBuf* pMsg, int32_t code) { } tDecoderClear(&decoder); (void)memcpy(&pRspWrapper->batchMetaRsp, pMsg->pData, sizeof(SMqRspHead)); - tscDebug("consumer:0x%" PRIx64 " recv poll batchmeta rsp, vgId:%d,QID:0x%" PRIx64, tmq->consumerId, vgId, + tqDebugC("consumer:0x%" PRIx64 " recv poll batchmeta rsp, vgId:%d,QID:0x%" PRIx64, tmq->consumerId, vgId, requestId); } else { // invalid rspType - tscError("consumer:0x%" PRIx64 " invalid rsp msg received, type:%d ignored", tmq->consumerId, rspType); + tqErrorC("consumer:0x%" PRIx64 " invalid rsp msg received, type:%d ignored", tmq->consumerId, rspType); } END: @@ -1636,11 +1644,11 @@ END: (void)strcpy(pRspWrapper->topicName, pParam->topicName); code = taosWriteQitem(tmq->mqueue, pRspWrapper); if (code != 0) { - tscError("consumer:0x%" PRIx64 " put poll res into mqueue failed, code:%d", tmq->consumerId, code); + tqErrorC("consumer:0x%" PRIx64 " put poll res into mqueue failed, code:%d", tmq->consumerId, code); } } int32_t total = taosQueueItemSize(tmq->mqueue); - tscDebug("consumer:0x%" PRIx64 " put poll res into mqueue, type:%d, vgId:%d, total in queue:%d,QID:0x%" PRIx64, + tqDebugC("consumer:0x%" PRIx64 " put poll res into mqueue, type:%d, vgId:%d, total in queue:%d,QID:0x%" PRIx64, tmq ? tmq->consumerId : 0, rspType, vgId, total, requestId); if (tmq) (void)tsem2_post(&tmq->rspSem); @@ -1671,10 +1679,10 @@ static void initClientTopicFromRsp(SMqClientTopic* pTopic, SMqSubTopicEp* pTopic tstrncpy(pTopic->topicName, pTopicEp->topic, TSDB_TOPIC_FNAME_LEN); tstrncpy(pTopic->db, pTopicEp->db, TSDB_DB_FNAME_LEN); - tscInfo("consumer:0x%" PRIx64 ", update topic:%s, new numOfVgs:%d", tmq->consumerId, pTopic->topicName, vgNumGet); + tqInfoC("consumer:0x%" PRIx64 ", update topic:%s, new numOfVgs:%d", tmq->consumerId, pTopic->topicName, vgNumGet); pTopic->vgs = taosArrayInit(vgNumGet, sizeof(SMqClientVg)); if (pTopic->vgs == NULL) { - tscError("consumer:0x%" PRIx64 ", failed to init vgs for topic:%s", tmq->consumerId, pTopic->topicName); + tqErrorC("consumer:0x%" PRIx64 ", failed to init vgs for topic:%s", tmq->consumerId, pTopic->topicName); return; } for (int32_t j = 0; j < vgNumGet; j++) { @@ -1688,7 +1696,7 @@ static void initClientTopicFromRsp(SMqClientTopic* pTopic, SMqSubTopicEp* pTopic STqOffsetVal offsetNew = {0}; offsetNew.type = tmq->resetOffsetCfg; - tscInfo("consumer:0x%" PRIx64 ", update topic:%s, new numOfVgs:%d, num:%d, port:%d", tmq->consumerId, + tqInfoC("consumer:0x%" PRIx64 ", update topic:%s, new numOfVgs:%d, num:%d, port:%d", tmq->consumerId, pTopic->topicName, vgNumGet, pVgEp->epSet.numOfEps, pVgEp->epSet.eps[pVgEp->epSet.inUse].port); SMqClientVg clientVg = { @@ -1716,71 +1724,52 @@ static void initClientTopicFromRsp(SMqClientTopic* pTopic, SMqSubTopicEp* pTopic clientVg.offsetInfo.beginOffset = offsetNew; } if (taosArrayPush(pTopic->vgs, &clientVg) == NULL) { - tscError("consumer:0x%" PRIx64 ", failed to push vg:%d into topic:%s", tmq->consumerId, pVgEp->vgId, + tqErrorC("consumer:0x%" PRIx64 ", failed to push vg:%d into topic:%s", tmq->consumerId, pVgEp->vgId, pTopic->topicName); freeClientVg(&clientVg); } } } -static void doUpdateLocalEp(tmq_t* tmq, int32_t epoch, const SMqAskEpRsp* pRsp) { - int32_t topicNumGet = taosArrayGetSize(pRsp->topics); - if (epoch <= tmq->epoch ) { - tscInfo("consumer:0x%" PRIx64 " no update ep epoch from %d to epoch %d, incoming topics:%d", tmq->consumerId, - tmq->epoch, epoch, topicNumGet); - return; - } - - SArray* newTopics = taosArrayInit(topicNumGet, sizeof(SMqClientTopic)); - if (newTopics == NULL) { - tscError("consumer:0x%" PRIx64 " taos array init null, code:%d", tmq->consumerId, terrno); - return; - } - +static void buildNewTopicList(tmq_t* tmq, SArray* newTopics, const SMqAskEpRsp* pRsp){ SHashObj* pVgOffsetHashMap = taosHashInit(64, MurmurHash3_32, false, HASH_NO_LOCK); if (pVgOffsetHashMap == NULL) { - (void)taosArrayDestroy(newTopics); - tscError("consumer:0x%" PRIx64 " taos hash init null, code:%d", tmq->consumerId, terrno); + tqErrorC("consumer:0x%" PRIx64 " taos hash init null, code:%d", tmq->consumerId, terrno); return; } - taosWLockLatch(&tmq->lock); int32_t topicNumCur = taosArrayGetSize(tmq->clientTopics); - - char vgKey[TSDB_TOPIC_FNAME_LEN + 22] = {0}; - tscInfo("consumer:0x%" PRIx64 " update ep epoch from %d to epoch %d, incoming topics:%d, existed topics:%d", - tmq->consumerId, tmq->epoch, epoch, topicNumGet, topicNumCur); for (int32_t i = 0; i < topicNumCur; i++) { // find old topic SMqClientTopic* pTopicCur = taosArrayGet(tmq->clientTopics, i); if (pTopicCur && pTopicCur->vgs) { int32_t vgNumCur = taosArrayGetSize(pTopicCur->vgs); - tscInfo("consumer:0x%" PRIx64 ", current vg num: %d", tmq->consumerId, vgNumCur); + tqInfoC("consumer:0x%" PRIx64 ", current vg num: %d", tmq->consumerId, vgNumCur); for (int32_t j = 0; j < vgNumCur; j++) { SMqClientVg* pVgCur = taosArrayGet(pTopicCur->vgs, j); if (pVgCur == NULL) { continue; } + char vgKey[TSDB_TOPIC_FNAME_LEN + 22] = {0}; (void)sprintf(vgKey, "%s:%d", pTopicCur->topicName, pVgCur->vgId); char buf[TSDB_OFFSET_LEN] = {0}; tFormatOffset(buf, TSDB_OFFSET_LEN, &pVgCur->offsetInfo.endOffset); - tscInfo("consumer:0x%" PRIx64 ", epoch:%d vgId:%d vgKey:%s, offset:%s", tmq->consumerId, epoch, pVgCur->vgId, - vgKey, buf); + tqInfoC("consumer:0x%" PRIx64 ", vgId:%d vgKey:%s, offset:%s", tmq->consumerId, pVgCur->vgId, vgKey, buf); SVgroupSaveInfo info = {.currentOffset = pVgCur->offsetInfo.endOffset, - .seekOffset = pVgCur->offsetInfo.beginOffset, - .commitOffset = pVgCur->offsetInfo.committedOffset, - .numOfRows = pVgCur->numOfRows, - .vgStatus = pVgCur->vgStatus}; + .seekOffset = pVgCur->offsetInfo.beginOffset, + .commitOffset = pVgCur->offsetInfo.committedOffset, + .numOfRows = pVgCur->numOfRows, + .vgStatus = pVgCur->vgStatus}; if (taosHashPut(pVgOffsetHashMap, vgKey, strlen(vgKey), &info, sizeof(SVgroupSaveInfo)) != 0) { - tscError("consumer:0x%" PRIx64 ", failed to put vg:%d into hashmap", tmq->consumerId, pVgCur->vgId); + tqErrorC("consumer:0x%" PRIx64 ", failed to put vg:%d into hashmap", tmq->consumerId, pVgCur->vgId); } } } } - for (int32_t i = 0; i < topicNumGet; i++) { + for (int32_t i = 0; i < taosArrayGetSize(pRsp->topics); i++) { SMqClientTopic topic = {0}; SMqSubTopicEp* pTopicEp = taosArrayGet(pRsp->topics, i); if (pTopicEp == NULL) { @@ -1788,13 +1777,36 @@ static void doUpdateLocalEp(tmq_t* tmq, int32_t epoch, const SMqAskEpRsp* pRsp) } initClientTopicFromRsp(&topic, pTopicEp, pVgOffsetHashMap, tmq); if (taosArrayPush(newTopics, &topic) == NULL) { - tscError("consumer:0x%" PRIx64 ", failed to push topic:%s into new topics", tmq->consumerId, topic.topicName); + tqErrorC("consumer:0x%" PRIx64 ", failed to push topic:%s into new topics", tmq->consumerId, topic.topicName); freeClientTopic(&topic); } } taosHashCleanup(pVgOffsetHashMap); +} +static void doUpdateLocalEp(tmq_t* tmq, int32_t epoch, const SMqAskEpRsp* pRsp) { + int32_t topicNumGet = taosArrayGetSize(pRsp->topics); + // vnode transform (epoch == tmq->epoch && topicNumGet != 0) + // ask ep rsp (epoch == tmq->epoch && topicNumGet == 0) + if (epoch < tmq->epoch || (epoch == tmq->epoch && topicNumGet == 0)) { + tqDebugC("consumer:0x%" PRIx64 " no update ep epoch from %d to epoch %d, incoming topics:%d", tmq->consumerId, + tmq->epoch, epoch, topicNumGet); + return; + } + + SArray* newTopics = taosArrayInit(topicNumGet, sizeof(SMqClientTopic)); + if (newTopics == NULL) { + tqErrorC("consumer:0x%" PRIx64 " taos array init null, code:%d", tmq->consumerId, terrno); + return; + } + tqInfoC("consumer:0x%" PRIx64 " update ep epoch from %d to epoch %d, incoming topics:%d, existed topics:%d", + tmq->consumerId, tmq->epoch, epoch, topicNumGet, (int)taosArrayGetSize(tmq->clientTopics)); + + taosWLockLatch(&tmq->lock); + if (topicNumGet > 0){ + buildNewTopicList(tmq, newTopics, pRsp); + } // destroy current buffered existed topics info if (tmq->clientTopics) { taosArrayDestroyEx(tmq->clientTopics, freeClientTopic); @@ -1805,7 +1817,7 @@ static void doUpdateLocalEp(tmq_t* tmq, int32_t epoch, const SMqAskEpRsp* pRsp) atomic_store_8(&tmq->status, TMQ_CONSUMER_STATUS__READY); atomic_store_32(&tmq->epoch, epoch); - tscInfo("consumer:0x%" PRIx64 " update topic info completed", tmq->consumerId); + tqInfoC("consumer:0x%" PRIx64 " update topic info completed", tmq->consumerId); } void tmqBuildConsumeReqImpl(SMqPollReq* pReq, tmq_t* tmq, int64_t timeout, SMqClientTopic* pTopic, SMqClientVg* pVg) { @@ -1849,7 +1861,7 @@ int32_t tmqBuildBatchMetaRspFromWrapper(SMqPollRspWrapper* pWrapper, SMqBatchMet pRspObj->common.vgId = pWrapper->vgHandle->vgId; (void)memcpy(&pRspObj->rsp, &pWrapper->batchMetaRsp, sizeof(SMqBatchMetaRsp)); - tscDebug("build batchmeta Rsp from wrapper"); + tqDebugC("build batchmeta Rsp from wrapper"); *ppRspObj = pRspObj; return 0; } @@ -1864,7 +1876,7 @@ void changeByteEndian(char* pData) { // length | version: int32_t blockVersion = *(int32_t*)p; if (blockVersion != BLOCK_VERSION_1) { - tscError("invalid block version:%d", blockVersion); + tqErrorC("invalid block version:%d", blockVersion); return; } *(int32_t*)p = BLOCK_VERSION_2; @@ -1921,7 +1933,7 @@ static void tmqBuildRspFromWrapperInner(SMqPollRspWrapper* pWrapper, SMqClientVg pDataRsp->withSchema = true; pDataRsp->blockSchema = taosArrayInit(pDataRsp->blockNum, sizeof(void*)); if (pDataRsp->blockSchema == NULL) { - tscError("failed to allocate memory for blockSchema"); + tqErrorC("failed to allocate memory for blockSchema"); return; } } @@ -1940,7 +1952,7 @@ static void tmqBuildRspFromWrapperInner(SMqPollRspWrapper* pWrapper, SMqClientVg SSchemaWrapper* schema = tCloneSSchemaWrapper(&pWrapper->topicHandle->schema); if (schema) { if (taosArrayPush(pDataRsp->blockSchema, &schema) == NULL) { - tscError("failed to push schema into blockSchema"); + tqErrorC("failed to push schema into blockSchema"); continue; } } @@ -2031,7 +2043,7 @@ static int32_t doTmqPollImpl(tmq_t* pTmq, SMqClientTopic* pTopic, SMqClientVg* p char offsetFormatBuf[TSDB_OFFSET_LEN] = {0}; tFormatOffset(offsetFormatBuf, tListLen(offsetFormatBuf), &pVg->offsetInfo.endOffset); code = asyncSendMsgToServer(pTmq->pTscObj->pAppInfo->pTransporter, &pVg->epSet, &transporterId, sendInfo); - tscDebug("consumer:0x%" PRIx64 " send poll to %s vgId:%d, code:%d, epoch %d, req:%s,QID:0x%" PRIx64, pTmq->consumerId, + tqDebugC("consumer:0x%" PRIx64 " send poll to %s vgId:%d, code:%d, epoch %d, req:%s,QID:0x%" PRIx64, pTmq->consumerId, pTopic->topicName, pVg->vgId, code, pTmq->epoch, offsetFormatBuf, req.reqId); if (code != 0) { return code; @@ -2050,7 +2062,7 @@ static int32_t tmqPollImpl(tmq_t* tmq, int64_t timeout) { taosWLockLatch(&tmq->lock); int32_t numOfTopics = taosArrayGetSize(tmq->clientTopics); - tscDebug("consumer:0x%" PRIx64 " start to poll data, numOfTopics:%d", tmq->consumerId, numOfTopics); + tqDebugC("consumer:0x%" PRIx64 " start to poll data, numOfTopics:%d", tmq->consumerId, numOfTopics); for (int i = 0; i < numOfTopics; i++) { SMqClientTopic* pTopic = taosArrayGet(tmq->clientTopics, i); @@ -2059,7 +2071,7 @@ static int32_t tmqPollImpl(tmq_t* tmq, int64_t timeout) { } int32_t numOfVg = taosArrayGetSize(pTopic->vgs); if (pTopic->noPrivilege) { - tscDebug("consumer:0x%" PRIx64 " has no privilegr for topic:%s", tmq->consumerId, pTopic->topicName); + tqDebugC("consumer:0x%" PRIx64 " has no privilegr for topic:%s", tmq->consumerId, pTopic->topicName); continue; } for (int j = 0; j < numOfVg; j++) { @@ -2069,14 +2081,14 @@ static int32_t tmqPollImpl(tmq_t* tmq, int64_t timeout) { } int64_t elapsed = taosGetTimestampMs() - pVg->emptyBlockReceiveTs; if (elapsed < EMPTY_BLOCK_POLL_IDLE_DURATION && elapsed >= 0) { // less than 10ms - tscDebug("consumer:0x%" PRIx64 " epoch %d, vgId:%d idle for 10ms before start next poll", tmq->consumerId, + tqDebugC("consumer:0x%" PRIx64 " epoch %d, vgId:%d idle for 10ms before start next poll", tmq->consumerId, tmq->epoch, pVg->vgId); continue; } elapsed = taosGetTimestampMs() - pVg->blockReceiveTs; if (tmq->replayEnable && elapsed < pVg->blockSleepForReplay && elapsed >= 0) { - tscDebug("consumer:0x%" PRIx64 " epoch %d, vgId:%d idle for %" PRId64 "ms before start next poll when replay", + tqDebugC("consumer:0x%" PRIx64 " epoch %d, vgId:%d idle for %" PRId64 "ms before start next poll when replay", tmq->consumerId, tmq->epoch, pVg->vgId, pVg->blockSleepForReplay); continue; } @@ -2084,7 +2096,7 @@ static int32_t tmqPollImpl(tmq_t* tmq, int64_t timeout) { int32_t vgStatus = atomic_val_compare_exchange_32(&pVg->vgStatus, TMQ_VG_STATUS__IDLE, TMQ_VG_STATUS__WAIT); if (vgStatus == TMQ_VG_STATUS__WAIT) { int32_t vgSkipCnt = atomic_add_fetch_32(&pVg->vgSkipCnt, 1); - tscDebug("consumer:0x%" PRIx64 " epoch %d wait poll-rsp, skip vgId:%d skip cnt %d", tmq->consumerId, tmq->epoch, + tqDebugC("consumer:0x%" PRIx64 " epoch %d wait poll-rsp, skip vgId:%d skip cnt %d", tmq->consumerId, tmq->epoch, pVg->vgId, vgSkipCnt); continue; } @@ -2099,20 +2111,20 @@ static int32_t tmqPollImpl(tmq_t* tmq, int64_t timeout) { end: taosWUnLockLatch(&tmq->lock); - tscDebug("consumer:0x%" PRIx64 " end to poll data, code:%d", tmq->consumerId, code); + tqDebugC("consumer:0x%" PRIx64 " end to poll data, code:%d", tmq->consumerId, code); return code; } static void updateVgInfo(SMqClientVg* pVg, STqOffsetVal* reqOffset, STqOffsetVal* rspOffset, int64_t sver, int64_t ever, int64_t consumerId, bool hasData) { if (!pVg->seekUpdated) { - tscDebug("consumer:0x%" PRIx64 " local offset is update, since seekupdate not set", consumerId); + tqDebugC("consumer:0x%" PRIx64 " local offset is update, since seekupdate not set", consumerId); if (hasData) { tOffsetCopy(&pVg->offsetInfo.beginOffset, reqOffset); } tOffsetCopy(&pVg->offsetInfo.endOffset, rspOffset); } else { - tscDebug("consumer:0x%" PRIx64 " local offset is NOT update, since seekupdate is set", consumerId); + tqDebugC("consumer:0x%" PRIx64 " local offset is NOT update, since seekupdate is set", consumerId); } // update the status @@ -2124,7 +2136,7 @@ static void updateVgInfo(SMqClientVg* pVg, STqOffsetVal* reqOffset, STqOffsetVal } static void* tmqHandleAllRsp(tmq_t* tmq, int64_t timeout) { - tscDebug("consumer:0x%" PRIx64 " start to handle the rsp, total:%d", tmq->consumerId, taosQallItemSize(tmq->qall)); + tqDebugC("consumer:0x%" PRIx64 " start to handle the rsp, total:%d", tmq->consumerId, taosQallItemSize(tmq->qall)); while (1) { SMqRspWrapper* pRspWrapper = NULL; @@ -2138,7 +2150,7 @@ static void* tmqHandleAllRsp(tmq_t* tmq, int64_t timeout) { } } - tscDebug("consumer:0x%" PRIx64 " handle rsp, type:%d", tmq->consumerId, pRspWrapper->tmqRspType); + tqDebugC("consumer:0x%" PRIx64 " handle rsp, type:%d", tmq->consumerId, pRspWrapper->tmqRspType); if (pRspWrapper->code != 0) { SMqPollRspWrapper* pollRspWrapper = (SMqPollRspWrapper*)pRspWrapper; @@ -2146,15 +2158,15 @@ static void* tmqHandleAllRsp(tmq_t* tmq, int64_t timeout) { if (pRspWrapper->code == TSDB_CODE_VND_INVALID_VGROUP_ID) { // for vnode transform int32_t code = askEp(tmq, NULL, false, true); if (code != 0) { - tscError("consumer:0x%" PRIx64 " failed to ask ep, code:%s", tmq->consumerId, tstrerror(code)); + tqErrorC("consumer:0x%" PRIx64 " failed to ask ep, code:%s", tmq->consumerId, tstrerror(code)); } } else if (pRspWrapper->code == TSDB_CODE_TMQ_CONSUMER_MISMATCH) { int32_t code = askEp(tmq, NULL, false, false); if (code != 0) { - tscError("consumer:0x%" PRIx64 " failed to ask ep, code:%s", tmq->consumerId, tstrerror(code)); + tqErrorC("consumer:0x%" PRIx64 " failed to ask ep, code:%s", tmq->consumerId, tstrerror(code)); } } - tscInfo("consumer:0x%" PRIx64 " msg from vgId:%d discarded, since %s", tmq->consumerId, pollRspWrapper->vgId, + tqInfoC("consumer:0x%" PRIx64 " msg from vgId:%d discarded, since %s", tmq->consumerId, pollRspWrapper->vgId, tstrerror(pRspWrapper->code)); taosWLockLatch(&tmq->lock); SMqClientVg* pVg = NULL; @@ -2179,7 +2191,7 @@ static void* tmqHandleAllRsp(tmq_t* tmq, int64_t timeout) { pollRspWrapper->vgHandle = pVg; pollRspWrapper->topicHandle = getTopicInfo(tmq, pollRspWrapper->topicName); if (pollRspWrapper->vgHandle == NULL || pollRspWrapper->topicHandle == NULL) { - tscError("consumer:0x%" PRIx64 " get vg or topic error, topic:%s vgId:%d", tmq->consumerId, + tqErrorC("consumer:0x%" PRIx64 " get vg or topic error, topic:%s vgId:%d", tmq->consumerId, pollRspWrapper->topicName, pollRspWrapper->vgId); tmqFreeRspWrapper(pRspWrapper); taosFreeQitem(pRspWrapper); @@ -2190,7 +2202,7 @@ static void* tmqHandleAllRsp(tmq_t* tmq, int64_t timeout) { if (pollRspWrapper->pEpset != NULL) { SEp* pEp = GET_ACTIVE_EP(pollRspWrapper->pEpset); SEp* pOld = GET_ACTIVE_EP(&(pVg->epSet)); - tscDebug("consumer:0x%" PRIx64 " update epset vgId:%d, ep:%s:%d, old ep:%s:%d", tmq->consumerId, pVg->vgId, + tqDebugC("consumer:0x%" PRIx64 " update epset vgId:%d, ep:%s:%d, old ep:%s:%d", tmq->consumerId, pVg->vgId, pEp->fqdn, pEp->port, pOld->fqdn, pOld->port); pVg->epSet = *pollRspWrapper->pEpset; } @@ -2201,7 +2213,7 @@ static void* tmqHandleAllRsp(tmq_t* tmq, int64_t timeout) { char buf[TSDB_OFFSET_LEN] = {0}; tFormatOffset(buf, TSDB_OFFSET_LEN, &pDataRsp->rspOffset); if (pDataRsp->blockNum == 0) { - tscDebug("consumer:0x%" PRIx64 " empty block received, vgId:%d, offset:%s, vg total:%" PRId64 + tqDebugC("consumer:0x%" PRIx64 " empty block received, vgId:%d, offset:%s, vg total:%" PRId64 ", total:%" PRId64 ",QID:0x%" PRIx64, tmq->consumerId, pVg->vgId, buf, pVg->numOfRows, tmq->totalRows, pollRspWrapper->reqId); pVg->emptyBlockReceiveTs = taosGetTimestampMs(); @@ -2219,12 +2231,12 @@ static void* tmqHandleAllRsp(tmq_t* tmq, int64_t timeout) { pVg->blockSleepForReplay = pRsp->rsp.sleepTime; if (pVg->blockSleepForReplay > 0) { if (taosTmrStart(tmqReplayTask, pVg->blockSleepForReplay, (void*)(tmq->refId), tmqMgmt.timer) == NULL) { - tscError("consumer:0x%" PRIx64 " failed to start replay timer, vgId:%d, sleep:%" PRId64, + tqErrorC("consumer:0x%" PRIx64 " failed to start replay timer, vgId:%d, sleep:%" PRId64, tmq->consumerId, pVg->vgId, pVg->blockSleepForReplay); } } } - tscDebug("consumer:0x%" PRIx64 " process poll rsp, vgId:%d, offset:%s, blocks:%d, rows:%" PRId64 + tqDebugC("consumer:0x%" PRIx64 " process poll rsp, vgId:%d, offset:%s, blocks:%d, rows:%" PRId64 ", vg total:%" PRId64 ", total:%" PRId64 ",QID:0x%" PRIx64, tmq->consumerId, pVg->vgId, buf, pDataRsp->blockNum, numOfRows, pVg->numOfRows, tmq->totalRows, pollRspWrapper->reqId); @@ -2234,7 +2246,7 @@ static void* tmqHandleAllRsp(tmq_t* tmq, int64_t timeout) { return pRsp; } } else { - tscInfo("consumer:0x%" PRIx64 " vgId:%d msg discard since epoch mismatch: msg epoch %d, consumer epoch %d", + tqInfoC("consumer:0x%" PRIx64 " vgId:%d msg discard since epoch mismatch: msg epoch %d, consumer epoch %d", tmq->consumerId, pollRspWrapper->vgId, pDataRsp->head.epoch, consumerEpoch); setVgIdle(tmq, pollRspWrapper->topicName, pollRspWrapper->vgId); tmqFreeRspWrapper(pRspWrapper); @@ -2244,7 +2256,7 @@ static void* tmqHandleAllRsp(tmq_t* tmq, int64_t timeout) { SMqPollRspWrapper* pollRspWrapper = (SMqPollRspWrapper*)pRspWrapper; int32_t consumerEpoch = atomic_load_32(&tmq->epoch); - tscDebug("consumer:0x%" PRIx64 " process meta rsp", tmq->consumerId); + tqDebugC("consumer:0x%" PRIx64 " process meta rsp", tmq->consumerId); if (pollRspWrapper->metaRsp.head.epoch == consumerEpoch) { taosWLockLatch(&tmq->lock); @@ -2253,7 +2265,7 @@ static void* tmqHandleAllRsp(tmq_t* tmq, int64_t timeout) { pollRspWrapper->vgHandle = pVg; pollRspWrapper->topicHandle = getTopicInfo(tmq, pollRspWrapper->topicName); if (pollRspWrapper->vgHandle == NULL || pollRspWrapper->topicHandle == NULL) { - tscError("consumer:0x%" PRIx64 " get vg or topic error, topic:%s vgId:%d", tmq->consumerId, + tqErrorC("consumer:0x%" PRIx64 " get vg or topic error, topic:%s vgId:%d", tmq->consumerId, pollRspWrapper->topicName, pollRspWrapper->vgId); tmqFreeRspWrapper(pRspWrapper); taosFreeQitem(pRspWrapper); @@ -2271,7 +2283,7 @@ static void* tmqHandleAllRsp(tmq_t* tmq, int64_t timeout) { taosWUnLockLatch(&tmq->lock); return pRsp; } else { - tscInfo("consumer:0x%" PRIx64 " vgId:%d msg discard since epoch mismatch: msg epoch %d, consumer epoch %d", + tqInfoC("consumer:0x%" PRIx64 " vgId:%d msg discard since epoch mismatch: msg epoch %d, consumer epoch %d", tmq->consumerId, pollRspWrapper->vgId, pollRspWrapper->metaRsp.head.epoch, consumerEpoch); setVgIdle(tmq, pollRspWrapper->topicName, pollRspWrapper->vgId); tmqFreeRspWrapper(pRspWrapper); @@ -2281,7 +2293,7 @@ static void* tmqHandleAllRsp(tmq_t* tmq, int64_t timeout) { SMqPollRspWrapper* pollRspWrapper = (SMqPollRspWrapper*)pRspWrapper; int32_t consumerEpoch = atomic_load_32(&tmq->epoch); - tscDebug("consumer:0x%" PRIx64 " process meta rsp", tmq->consumerId); + tqDebugC("consumer:0x%" PRIx64 " process meta rsp", tmq->consumerId); if (pollRspWrapper->batchMetaRsp.head.epoch == consumerEpoch) { taosWLockLatch(&tmq->lock); @@ -2290,7 +2302,7 @@ static void* tmqHandleAllRsp(tmq_t* tmq, int64_t timeout) { pollRspWrapper->vgHandle = pVg; pollRspWrapper->topicHandle = getTopicInfo(tmq, pollRspWrapper->topicName); if (pollRspWrapper->vgHandle == NULL || pollRspWrapper->topicHandle == NULL) { - tscError("consumer:0x%" PRIx64 " get vg or topic error, topic:%s vgId:%d", tmq->consumerId, + tqErrorC("consumer:0x%" PRIx64 " get vg or topic error, topic:%s vgId:%d", tmq->consumerId, pollRspWrapper->topicName, pollRspWrapper->vgId); tmqFreeRspWrapper(pRspWrapper); taosFreeQitem(pRspWrapper); @@ -2309,7 +2321,7 @@ static void* tmqHandleAllRsp(tmq_t* tmq, int64_t timeout) { taosWUnLockLatch(&tmq->lock); return pRsp; } else { - tscInfo("consumer:0x%" PRIx64 " vgId:%d msg discard since epoch mismatch: msg epoch %d, consumer epoch %d", + tqInfoC("consumer:0x%" PRIx64 " vgId:%d msg discard since epoch mismatch: msg epoch %d, consumer epoch %d", tmq->consumerId, pollRspWrapper->vgId, pollRspWrapper->batchMetaRsp.head.epoch, consumerEpoch); setVgIdle(tmq, pollRspWrapper->topicName, pollRspWrapper->vgId); tmqFreeRspWrapper(pRspWrapper); @@ -2327,7 +2339,7 @@ static void* tmqHandleAllRsp(tmq_t* tmq, int64_t timeout) { pollRspWrapper->vgHandle = pVg; pollRspWrapper->topicHandle = getTopicInfo(tmq, pollRspWrapper->topicName); if (pollRspWrapper->vgHandle == NULL || pollRspWrapper->topicHandle == NULL) { - tscError("consumer:0x%" PRIx64 " get vg or topic error, topic:%s vgId:%d", tmq->consumerId, + tqErrorC("consumer:0x%" PRIx64 " get vg or topic error, topic:%s vgId:%d", tmq->consumerId, pollRspWrapper->topicName, pollRspWrapper->vgId); tmqFreeRspWrapper(pRspWrapper); taosFreeQitem(pRspWrapper); @@ -2339,7 +2351,7 @@ static void* tmqHandleAllRsp(tmq_t* tmq, int64_t timeout) { tmq->consumerId, pDataRsp->blockNum != 0); if (pDataRsp->blockNum == 0) { - tscDebug("consumer:0x%" PRIx64 " taosx empty block received, vgId:%d, vg total:%" PRId64 ",QID:0x%" PRIx64, + tqDebugC("consumer:0x%" PRIx64 " taosx empty block received, vgId:%d, vg total:%" PRId64 ",QID:0x%" PRIx64, tmq->consumerId, pVg->vgId, pVg->numOfRows, pollRspWrapper->reqId); pVg->emptyBlockReceiveTs = taosGetTimestampMs(); tmqFreeRspWrapper(pRspWrapper); @@ -2354,13 +2366,13 @@ static void* tmqHandleAllRsp(tmq_t* tmq, int64_t timeout) { int64_t numOfRows = 0; SMqTaosxRspObj* pRsp = NULL; if (tmqBuildTaosxRspFromWrapper(pollRspWrapper, pVg, &numOfRows, &pRsp) != 0) { - tscError("consumer:0x%" PRIx64 " build taosx rsp failed, vgId:%d", tmq->consumerId, pVg->vgId); + tqErrorC("consumer:0x%" PRIx64 " build taosx rsp failed, vgId:%d", tmq->consumerId, pVg->vgId); } tmq->totalRows += numOfRows; char buf[TSDB_OFFSET_LEN] = {0}; tFormatOffset(buf, TSDB_OFFSET_LEN, &pVg->offsetInfo.endOffset); - tscDebug("consumer:0x%" PRIx64 " process taosx poll rsp, vgId:%d, offset:%s, blocks:%d, rows:%" PRId64 + tqDebugC("consumer:0x%" PRIx64 " process taosx poll rsp, vgId:%d, offset:%s, blocks:%d, rows:%" PRId64 ", vg total:%" PRId64 ", total:%" PRId64 ",QID:0x%" PRIx64, tmq->consumerId, pVg->vgId, buf, pDataRsp->blockNum, numOfRows, pVg->numOfRows, tmq->totalRows, pollRspWrapper->reqId); @@ -2370,21 +2382,21 @@ static void* tmqHandleAllRsp(tmq_t* tmq, int64_t timeout) { taosWUnLockLatch(&tmq->lock); return pRsp; } else { - tscInfo("consumer:0x%" PRIx64 " vgId:%d msg discard since epoch mismatch: msg epoch %d, consumer epoch %d", + tqInfoC("consumer:0x%" PRIx64 " vgId:%d msg discard since epoch mismatch: msg epoch %d, consumer epoch %d", tmq->consumerId, pollRspWrapper->vgId, pDataRsp->head.epoch, consumerEpoch); setVgIdle(tmq, pollRspWrapper->topicName, pollRspWrapper->vgId); tmqFreeRspWrapper(pRspWrapper); taosFreeQitem(pRspWrapper); } } else if (pRspWrapper->tmqRspType == TMQ_MSG_TYPE__EP_RSP) { - tscDebug("consumer:0x%" PRIx64 " ep msg received", tmq->consumerId); + tqDebugC("consumer:0x%" PRIx64 " ep msg received", tmq->consumerId); SMqAskEpRspWrapper* pEpRspWrapper = (SMqAskEpRspWrapper*)pRspWrapper; SMqAskEpRsp* rspMsg = &pEpRspWrapper->msg; doUpdateLocalEp(tmq, pEpRspWrapper->epoch, rspMsg); tmqFreeRspWrapper(pRspWrapper); taosFreeQitem(pRspWrapper); } else { - tscError("consumer:0x%" PRIx64 " invalid msg received:%d", tmq->consumerId, pRspWrapper->tmqRspType); + tqErrorC("consumer:0x%" PRIx64 " invalid msg received:%d", tmq->consumerId, pRspWrapper->tmqRspType); } } } @@ -2395,12 +2407,12 @@ TAOS_RES* tmq_consumer_poll(tmq_t* tmq, int64_t timeout) { void* rspObj = NULL; int64_t startTime = taosGetTimestampMs(); - tscDebug("consumer:0x%" PRIx64 " start to poll at %" PRId64 ", timeout:%" PRId64, tmq->consumerId, startTime, + tqDebugC("consumer:0x%" PRIx64 " start to poll at %" PRId64 ", timeout:%" PRId64, tmq->consumerId, startTime, timeout); // in no topic status, delayed task also need to be processed if (atomic_load_8(&tmq->status) == TMQ_CONSUMER_STATUS__INIT) { - tscInfo("consumer:0x%" PRIx64 " poll return since consumer is init", tmq->consumerId); + tqInfoC("consumer:0x%" PRIx64 " poll return since consumer is init", tmq->consumerId); taosMsleep(500); // sleep for a while return NULL; } @@ -2411,12 +2423,12 @@ TAOS_RES* tmq_consumer_poll(tmq_t* tmq, int64_t timeout) { tmqHandleAllDelayedTask(tmq); if (tmqPollImpl(tmq, timeout) < 0) { - tscError("consumer:0x%" PRIx64 " return due to poll error", tmq->consumerId); + tqErrorC("consumer:0x%" PRIx64 " return due to poll error", tmq->consumerId); } rspObj = tmqHandleAllRsp(tmq, timeout); if (rspObj) { - tscDebug("consumer:0x%" PRIx64 " return rsp %p", tmq->consumerId, rspObj); + tqDebugC("consumer:0x%" PRIx64 " return rsp %p", tmq->consumerId, rspObj); return (TAOS_RES*)rspObj; } @@ -2424,7 +2436,7 @@ TAOS_RES* tmq_consumer_poll(tmq_t* tmq, int64_t timeout) { int64_t currentTime = taosGetTimestampMs(); int64_t elapsedTime = currentTime - startTime; if (elapsedTime > timeout || elapsedTime < 0) { - tscDebug("consumer:0x%" PRIx64 " (epoch %d) timeout, no rsp, start time %" PRId64 ", current time %" PRId64, + tqDebugC("consumer:0x%" PRIx64 " (epoch %d) timeout, no rsp, start time %" PRId64 ", current time %" PRId64, tmq->consumerId, tmq->epoch, startTime, currentTime); return NULL; } @@ -2438,33 +2450,34 @@ TAOS_RES* tmq_consumer_poll(tmq_t* tmq, int64_t timeout) { static void displayConsumeStatistics(tmq_t* pTmq) { taosRLockLatch(&pTmq->lock); int32_t numOfTopics = taosArrayGetSize(pTmq->clientTopics); - tscDebug("consumer:0x%" PRIx64 " closing poll:%" PRId64 " rows:%" PRId64 " topics:%d, final epoch:%d", + tqInfoC("consumer:0x%" PRIx64 " closing poll:%" PRId64 " rows:%" PRId64 " topics:%d, final epoch:%d", pTmq->consumerId, pTmq->pollCnt, pTmq->totalRows, numOfTopics, pTmq->epoch); - tscDebug("consumer:0x%" PRIx64 " rows dist begin: ", pTmq->consumerId); + tqInfoC("consumer:0x%" PRIx64 " rows dist begin: ", pTmq->consumerId); for (int32_t i = 0; i < numOfTopics; ++i) { SMqClientTopic* pTopics = taosArrayGet(pTmq->clientTopics, i); if (pTopics == NULL) continue; - tscDebug("consumer:0x%" PRIx64 " topic:%d", pTmq->consumerId, i); + tqInfoC("consumer:0x%" PRIx64 " topic:%d", pTmq->consumerId, i); int32_t numOfVgs = taosArrayGetSize(pTopics->vgs); for (int32_t j = 0; j < numOfVgs; ++j) { SMqClientVg* pVg = taosArrayGet(pTopics->vgs, j); if (pVg == NULL) continue; - tscDebug("topic:%s, %d. vgId:%d rows:%" PRId64, pTopics->topicName, j, pVg->vgId, pVg->numOfRows); + tqInfoC("topic:%s, %d. vgId:%d rows:%" PRId64, pTopics->topicName, j, pVg->vgId, pVg->numOfRows); } } taosRUnLockLatch(&pTmq->lock); - tscDebug("consumer:0x%" PRIx64 " rows dist end", pTmq->consumerId); + tqInfoC("consumer:0x%" PRIx64 " rows dist end", pTmq->consumerId); } int32_t tmq_unsubscribe(tmq_t* tmq) { if (tmq == NULL) return TSDB_CODE_INVALID_PARA; int32_t code = 0; int8_t status = atomic_load_8(&tmq->status); - tscInfo("consumer:0x%" PRIx64 " start to unsubscribe consumer, status:%d", tmq->consumerId, status); + tqInfoC("consumer:0x%" PRIx64 " start to unsubscribe consumer, status:%d", tmq->consumerId, status); + displayConsumeStatistics(tmq); if (status != TMQ_CONSUMER_STATUS__READY) { - tscInfo("consumer:0x%" PRIx64 " status:%d, already closed or not in ready state, no need unsubscribe", tmq->consumerId, status); + tqInfoC("consumer:0x%" PRIx64 " status:%d, already closed or not in ready state, no need unsubscribe", tmq->consumerId, status); goto END; } if (tmq->autoCommit) { @@ -2492,8 +2505,7 @@ END: int32_t tmq_consumer_close(tmq_t* tmq) { if (tmq == NULL) return TSDB_CODE_INVALID_PARA; - tscInfo("consumer:0x%" PRIx64 " start to close consumer, status:%d", tmq->consumerId, tmq->status); - displayConsumeStatistics(tmq); + tqInfoC("consumer:0x%" PRIx64 " start to close consumer, status:%d", tmq->consumerId, tmq->status); int32_t code = tmq_unsubscribe(tmq); if (code == 0) { atomic_store_8(&tmq->status, TMQ_CONSUMER_STATUS__CLOSED); @@ -2600,7 +2612,7 @@ int64_t tmq_get_vgroup_offset(TAOS_RES* res) { if (common->reqOffset.type == TMQ_OFFSET__LOG) { return common->reqOffset.version; } else { - tscError("invalid offset type:%d", common->reqOffset.type); + tqErrorC("invalid offset type:%d", common->reqOffset.type); } } else if (TD_RES_TMQ_META(res)) { SMqMetaRspObj* pRspObj = (SMqMetaRspObj*)res; @@ -2613,7 +2625,7 @@ int64_t tmq_get_vgroup_offset(TAOS_RES* res) { return pBtRspObj->rsp.rspOffset.version; } } else { - tscError("invalid tmq type:%d", *(int8_t*)res); + tqErrorC("invalid tmq type:%d", *(int8_t*)res); } // data from tsdb, no valid offset info @@ -2639,7 +2651,7 @@ const char* tmq_get_table_name(TAOS_RES* res) { void tmq_commit_async(tmq_t* tmq, const TAOS_RES* pRes, tmq_commit_cb* cb, void* param) { if (tmq == NULL) { - tscError("invalid tmq handle, null"); + tqErrorC("invalid tmq handle, null"); if (cb != NULL) { cb(tmq, TSDB_CODE_INVALID_PARA, param); } @@ -2660,7 +2672,7 @@ static void commitCallBackFn(tmq_t* UNUSED_PARAM(tmq), int32_t code, void* param int32_t tmq_commit_sync(tmq_t* tmq, const TAOS_RES* pRes) { if (tmq == NULL) { - tscError("invalid tmq handle, null"); + tqErrorC("invalid tmq handle, null"); return TSDB_CODE_INVALID_PARA; } @@ -2668,11 +2680,11 @@ int32_t tmq_commit_sync(tmq_t* tmq, const TAOS_RES* pRes) { SSyncCommitInfo* pInfo = taosMemoryMalloc(sizeof(SSyncCommitInfo)); if (pInfo == NULL) { - tscError("failed to allocate memory for sync commit"); + tqErrorC("failed to allocate memory for sync commit"); return terrno; } if (tsem2_init(&pInfo->sem, 0, 0) != 0) { - tscError("failed to init sem for sync commit"); + tqErrorC("failed to init sem for sync commit"); taosMemoryFree(pInfo); return TSDB_CODE_OUT_OF_MEMORY; } @@ -2690,19 +2702,19 @@ int32_t tmq_commit_sync(tmq_t* tmq, const TAOS_RES* pRes) { (void)tsem2_destroy(&pInfo->sem); taosMemoryFree(pInfo); - tscInfo("consumer:0x%" PRIx64 " sync res commit done, code:%s", tmq->consumerId, tstrerror(code)); + tqInfoC("consumer:0x%" PRIx64 " sync res commit done, code:%s", tmq->consumerId, tstrerror(code)); return code; } // wal range will be ok after calling tmq_get_topic_assignment or poll interface static int32_t checkWalRange(SVgOffsetInfo* offset, int64_t value) { if (offset->walVerBegin == -1 || offset->walVerEnd == -1) { - tscError("Assignment or poll interface need to be called first"); + tqErrorC("Assignment or poll interface need to be called first"); return TSDB_CODE_TMQ_NEED_INITIALIZED; } if (value != -1 && (value < offset->walVerBegin || value > offset->walVerEnd)) { - tscError("invalid seek params, offset:%" PRId64 ", valid range:[%" PRId64 ", %" PRId64 "]", value, + tqErrorC("invalid seek params, offset:%" PRId64 ", valid range:[%" PRId64 ", %" PRId64 "]", value, offset->walVerBegin, offset->walVerEnd); return TSDB_CODE_TMQ_VERSION_OUT_OF_RANGE; } @@ -2712,7 +2724,7 @@ static int32_t checkWalRange(SVgOffsetInfo* offset, int64_t value) { int32_t tmq_commit_offset_sync(tmq_t* tmq, const char* pTopicName, int32_t vgId, int64_t offset) { if (tmq == NULL || pTopicName == NULL) { - tscError("invalid tmq handle, null"); + tqErrorC("invalid tmq handle, null"); return TSDB_CODE_INVALID_PARA; } @@ -2740,7 +2752,7 @@ int32_t tmq_commit_offset_sync(tmq_t* tmq, const char* pTopicName, int32_t vgId, SSyncCommitInfo* pInfo = taosMemoryMalloc(sizeof(SSyncCommitInfo)); if (pInfo == NULL) { - tscError("consumer:0x%" PRIx64 " failed to prepare seek operation", tmq->consumerId); + tqErrorC("consumer:0x%" PRIx64 " failed to prepare seek operation", tmq->consumerId); return TSDB_CODE_OUT_OF_MEMORY; } @@ -2760,7 +2772,7 @@ int32_t tmq_commit_offset_sync(tmq_t* tmq, const char* pTopicName, int32_t vgId, (void)tsem2_destroy(&pInfo->sem); taosMemoryFree(pInfo); - tscInfo("consumer:0x%" PRIx64 " sync send commit to vgId:%d, offset:%" PRId64 " code:%s", tmq->consumerId, vgId, + tqInfoC("consumer:0x%" PRIx64 " sync send commit to vgId:%d, offset:%" PRId64 " code:%s", tmq->consumerId, vgId, offset, tstrerror(code)); return code; @@ -2770,7 +2782,7 @@ void tmq_commit_offset_async(tmq_t* tmq, const char* pTopicName, int32_t vgId, i void* param) { int32_t code = 0; if (tmq == NULL || pTopicName == NULL) { - tscError("invalid tmq handle, null"); + tqErrorC("invalid tmq handle, null"); code = TSDB_CODE_INVALID_PARA; goto end; } @@ -2799,7 +2811,7 @@ void tmq_commit_offset_async(tmq_t* tmq, const char* pTopicName, int32_t vgId, i code = asyncCommitOffset(tmq, tname, vgId, &offsetVal, cb, param); - tscInfo("consumer:0x%" PRIx64 " async send commit to vgId:%d, offset:%" PRId64 " code:%s", tmq->consumerId, vgId, + tqInfoC("consumer:0x%" PRIx64 " async send commit to vgId:%d, offset:%" PRId64 " code:%s", tmq->consumerId, vgId, offset, tstrerror(code)); end: @@ -2822,7 +2834,7 @@ int32_t askEpCb(void* param, SDataBuf* pMsg, int32_t code) { } if (code != TSDB_CODE_SUCCESS) { - tscError("consumer:0x%" PRIx64 ", get topic endpoint error, code:%s", tmq->consumerId, tstrerror(code)); + tqErrorC("consumer:0x%" PRIx64 ", get topic endpoint error, code:%s", tmq->consumerId, tstrerror(code)); goto END; } @@ -2831,7 +2843,7 @@ int32_t askEpCb(void* param, SDataBuf* pMsg, int32_t code) { } SMqRspHead* head = pMsg->pData; int32_t epoch = atomic_load_32(&tmq->epoch); - tscDebug("consumer:0x%" PRIx64 ", recv ep, msg epoch %d, current epoch %d", tmq->consumerId, head->epoch, epoch); + tqDebugC("consumer:0x%" PRIx64 ", recv ep, msg epoch %d, current epoch %d", tmq->consumerId, head->epoch, epoch); if (pParam->sync) { SMqAskEpRsp rsp = {0}; if (tDecodeSMqAskEpRsp(POINTER_SHIFT(pMsg->pData, sizeof(SMqRspHead)), &rsp) != NULL) { @@ -2906,25 +2918,25 @@ int32_t askEp(tmq_t* pTmq, void* param, bool sync, bool updateEpSet) { int32_t tlen = tSerializeSMqAskEpReq(NULL, 0, &req); if (tlen < 0) { - tscError("consumer:0x%" PRIx64 ", tSerializeSMqAskEpReq failed", pTmq->consumerId); + tqErrorC("consumer:0x%" PRIx64 ", tSerializeSMqAskEpReq failed", pTmq->consumerId); return TSDB_CODE_INVALID_PARA; } pReq = taosMemoryCalloc(1, tlen); if (pReq == NULL) { - tscError("consumer:0x%" PRIx64 ", failed to malloc askEpReq msg, size:%d", pTmq->consumerId, tlen); + tqErrorC("consumer:0x%" PRIx64 ", failed to malloc askEpReq msg, size:%d", pTmq->consumerId, tlen); return terrno; } if (tSerializeSMqAskEpReq(pReq, tlen, &req) < 0) { - tscError("consumer:0x%" PRIx64 ", tSerializeSMqAskEpReq %d failed", pTmq->consumerId, tlen); + tqErrorC("consumer:0x%" PRIx64 ", tSerializeSMqAskEpReq %d failed", pTmq->consumerId, tlen); taosMemoryFree(pReq); return TSDB_CODE_INVALID_PARA; } pParam = taosMemoryCalloc(1, sizeof(SMqAskEpCbParam)); if (pParam == NULL) { - tscError("consumer:0x%" PRIx64 ", failed to malloc subscribe param", pTmq->consumerId); + tqErrorC("consumer:0x%" PRIx64 ", failed to malloc subscribe param", pTmq->consumerId); taosMemoryFree(pReq); return terrno; } @@ -2949,7 +2961,7 @@ int32_t askEp(tmq_t* pTmq, void* param, bool sync, bool updateEpSet) { sendInfo->msgType = TDMT_MND_TMQ_ASK_EP; SEpSet epSet = getEpSet_s(&pTmq->pTscObj->pAppInfo->mgmtEp); - tscDebug("consumer:0x%" PRIx64 " ask ep from mnode,QID:0x%" PRIx64, pTmq->consumerId, sendInfo->requestId); + tqDebugC("consumer:0x%" PRIx64 " ask ep from mnode,QID:0x%" PRIx64, pTmq->consumerId, sendInfo->requestId); return asyncSendMsgToServer(pTmq->pTscObj->pAppInfo->pTransporter, &epSet, NULL, sendInfo); } @@ -2977,11 +2989,11 @@ int32_t tmqCommitDone(SMqCommitCbParamSet* pParamSet) { int32_t commitRspCountDown(SMqCommitCbParamSet* pParamSet, int64_t consumerId, const char* pTopic, int32_t vgId) { int32_t waitingRspNum = atomic_sub_fetch_32(&pParamSet->waitingRspNum, 1); if (waitingRspNum == 0) { - tscInfo("consumer:0x%" PRIx64 " topic:%s vgId:%d all commit-rsp received, commit completed", consumerId, pTopic, + tqDebugC("consumer:0x%" PRIx64 " topic:%s vgId:%d all commit-rsp received, commit completed", consumerId, pTopic, vgId); return tmqCommitDone(pParamSet); } else { - tscInfo("consumer:0x%" PRIx64 " topic:%s vgId:%d commit-rsp received, remain:%d", consumerId, pTopic, vgId, + tqDebugC("consumer:0x%" PRIx64 " topic:%s vgId:%d commit-rsp received, remain:%d", consumerId, pTopic, vgId, waitingRspNum); } return 0; @@ -3033,7 +3045,7 @@ static int32_t tmqGetWalInfoCb(void* param, SDataBuf* pMsg, int32_t code) { int32_t total = atomic_add_fetch_32(&pCommon->numOfRsp, 1); if (code != TSDB_CODE_SUCCESS) { - tscError("consumer:0x%" PRIx64 " failed to get the wal info from vgId:%d for topic:%s", pCommon->consumerId, + tqErrorC("consumer:0x%" PRIx64 " failed to get the wal info from vgId:%d for topic:%s", pCommon->consumerId, pParam->vgId, pCommon->pTopicName); } else { @@ -3054,7 +3066,7 @@ static int32_t tmqGetWalInfoCb(void* param, SDataBuf* pMsg, int32_t code) { (void)taosThreadMutexLock(&pCommon->mutex); if (taosArrayPush(pCommon->pList, &assignment) == NULL) { - tscError("consumer:0x%" PRIx64 " failed to push the wal info from vgId:%d for topic:%s", pCommon->consumerId, + tqErrorC("consumer:0x%" PRIx64 " failed to push the wal info from vgId:%d for topic:%s", pCommon->consumerId, pParam->vgId, pCommon->pTopicName); code = TSDB_CODE_TSC_INTERNAL_ERROR; } @@ -3204,7 +3216,7 @@ int64_t getCommittedFromServer(tmq_t* tmq, char* tname, int32_t vgId, SEpSet* ep int64_t tmq_position(tmq_t* tmq, const char* pTopicName, int32_t vgId) { if (tmq == NULL || pTopicName == NULL) { - tscError("invalid tmq handle, null"); + tqErrorC("invalid tmq handle, null"); return TSDB_CODE_INVALID_PARA; } @@ -3224,7 +3236,7 @@ int64_t tmq_position(tmq_t* tmq, const char* pTopicName, int32_t vgId) { SVgOffsetInfo* pOffsetInfo = &pVg->offsetInfo; int32_t type = pOffsetInfo->endOffset.type; if (isInSnapshotMode(type, tmq->useSnapshot)) { - tscError("consumer:0x%" PRIx64 " offset type:%d not wal version, position error", tmq->consumerId, type); + tqErrorC("consumer:0x%" PRIx64 " offset type:%d not wal version, position error", tmq->consumerId, type); taosWUnLockLatch(&tmq->lock); return TSDB_CODE_TMQ_SNAPSHOT_ERROR; } @@ -3254,16 +3266,16 @@ int64_t tmq_position(tmq_t* tmq, const char* pTopicName, int32_t vgId) { position = code; } } else { - tscError("consumer:0x%" PRIx64 " offset type:%d can not be reach here", tmq->consumerId, type); + tqErrorC("consumer:0x%" PRIx64 " offset type:%d can not be reach here", tmq->consumerId, type); } - tscInfo("consumer:0x%" PRIx64 " tmq_position vgId:%d position:%" PRId64, tmq->consumerId, vgId, position); + tqInfoC("consumer:0x%" PRIx64 " tmq_position vgId:%d position:%" PRId64, tmq->consumerId, vgId, position); return position; } int64_t tmq_committed(tmq_t* tmq, const char* pTopicName, int32_t vgId) { if (tmq == NULL || pTopicName == NULL) { - tscError("invalid tmq handle, null"); + tqErrorC("invalid tmq handle, null"); return TSDB_CODE_INVALID_PARA; } @@ -3282,14 +3294,14 @@ int64_t tmq_committed(tmq_t* tmq, const char* pTopicName, int32_t vgId) { SVgOffsetInfo* pOffsetInfo = &pVg->offsetInfo; if (isInSnapshotMode(pOffsetInfo->endOffset.type, tmq->useSnapshot)) { - tscError("consumer:0x%" PRIx64 " offset type:%d not wal version, committed error", tmq->consumerId, + tqErrorC("consumer:0x%" PRIx64 " offset type:%d not wal version, committed error", tmq->consumerId, pOffsetInfo->endOffset.type); taosWUnLockLatch(&tmq->lock); return TSDB_CODE_TMQ_SNAPSHOT_ERROR; } if (isInSnapshotMode(pOffsetInfo->committedOffset.type, tmq->useSnapshot)) { - tscError("consumer:0x%" PRIx64 " offset type:%d not wal version, committed error", tmq->consumerId, + tqErrorC("consumer:0x%" PRIx64 " offset type:%d not wal version, committed error", tmq->consumerId, pOffsetInfo->committedOffset.type); taosWUnLockLatch(&tmq->lock); return TSDB_CODE_TMQ_SNAPSHOT_ERROR; @@ -3307,14 +3319,14 @@ int64_t tmq_committed(tmq_t* tmq, const char* pTopicName, int32_t vgId) { committed = getCommittedFromServer(tmq, tname, vgId, &epSet); end: - tscInfo("consumer:0x%" PRIx64 " tmq_committed vgId:%d committed:%" PRId64, tmq->consumerId, vgId, committed); + tqInfoC("consumer:0x%" PRIx64 " tmq_committed vgId:%d committed:%" PRId64, tmq->consumerId, vgId, committed); return committed; } int32_t tmq_get_topic_assignment(tmq_t* tmq, const char* pTopicName, tmq_topic_assignment** assignment, int32_t* numOfAssignment) { if (tmq == NULL || pTopicName == NULL || assignment == NULL || numOfAssignment == NULL) { - tscError("invalid tmq handle, null"); + tqErrorC("invalid tmq handle, null"); return TSDB_CODE_INVALID_PARA; } *numOfAssignment = 0; @@ -3330,7 +3342,7 @@ int32_t tmq_get_topic_assignment(tmq_t* tmq, const char* pTopicName, tmq_topic_a SMqClientTopic* pTopic = NULL; int32_t code = getTopicByName(tmq, tname, &pTopic); if (code != 0) { - tscError("consumer:0x%" PRIx64 " invalid topic name:%s", tmq->consumerId, pTopicName); + tqErrorC("consumer:0x%" PRIx64 " invalid topic name:%s", tmq->consumerId, pTopicName); goto end; } @@ -3343,7 +3355,7 @@ int32_t tmq_get_topic_assignment(tmq_t* tmq, const char* pTopicName, tmq_topic_a } int32_t type = pClientVg->offsetInfo.beginOffset.type; if (isInSnapshotMode(type, tmq->useSnapshot)) { - tscError("consumer:0x%" PRIx64 " offset type:%d not wal version, assignment not allowed", tmq->consumerId, type); + tqErrorC("consumer:0x%" PRIx64 " offset type:%d not wal version, assignment not allowed", tmq->consumerId, type); code = TSDB_CODE_TMQ_SNAPSHOT_ERROR; goto end; } @@ -3351,7 +3363,7 @@ int32_t tmq_get_topic_assignment(tmq_t* tmq, const char* pTopicName, tmq_topic_a *assignment = taosMemoryCalloc(*numOfAssignment, sizeof(tmq_topic_assignment)); if (*assignment == NULL) { - tscError("consumer:0x%" PRIx64 " failed to malloc buffer, size:%" PRIzu, tmq->consumerId, + tqErrorC("consumer:0x%" PRIx64 " failed to malloc buffer, size:%" PRIzu, tmq->consumerId, (*numOfAssignment) * sizeof(tmq_topic_assignment)); code = terrno; goto end; @@ -3374,7 +3386,7 @@ int32_t tmq_get_topic_assignment(tmq_t* tmq, const char* pTopicName, tmq_topic_a pAssignment->begin = pClientVg->offsetInfo.walVerBegin; pAssignment->end = pClientVg->offsetInfo.walVerEnd; pAssignment->vgId = pClientVg->vgId; - tscInfo("consumer:0x%" PRIx64 " get assignment from local:%d->%" PRId64, tmq->consumerId, pAssignment->vgId, + tqInfoC("consumer:0x%" PRIx64 " get assignment from local:%d->%" PRId64, tmq->consumerId, pAssignment->vgId, pAssignment->currentOffset); } @@ -3459,7 +3471,7 @@ int32_t tmq_get_topic_assignment(tmq_t* tmq, const char* pTopicName, tmq_topic_a char offsetFormatBuf[TSDB_OFFSET_LEN] = {0}; tFormatOffset(offsetFormatBuf, tListLen(offsetFormatBuf), &pClientVg->offsetInfo.beginOffset); - tscInfo("consumer:0x%" PRIx64 " %s retrieve wal info vgId:%d, epoch %d, req:%s,QID:0x%" PRIx64, tmq->consumerId, + tqInfoC("consumer:0x%" PRIx64 " %s retrieve wal info vgId:%d, epoch %d, req:%s,QID:0x%" PRIx64, tmq->consumerId, pTopic->topicName, pClientVg->vgId, tmq->epoch, offsetFormatBuf, req.reqId); code = asyncSendMsgToServer(tmq->pTscObj->pAppInfo->pTransporter, &pClientVg->epSet, &transporterId, sendInfo); if (code != 0) { @@ -3492,7 +3504,7 @@ int32_t tmq_get_topic_assignment(tmq_t* tmq, const char* pTopicName, tmq_topic_a } SVgOffsetInfo* pOffsetInfo = &pClientVg->offsetInfo; - tscInfo("consumer:0x%" PRIx64 " %s vgId:%d offset is update to:%" PRId64, tmq->consumerId, pTopic->topicName, + tqInfoC("consumer:0x%" PRIx64 " %s vgId:%d offset is update to:%" PRId64, tmq->consumerId, pTopic->topicName, p->vgId, p->currentOffset); pOffsetInfo->walVerBegin = p->begin; @@ -3538,7 +3550,7 @@ static int32_t tmqSeekCb(void* param, SDataBuf* pMsg, int32_t code) { // there is no data to poll int32_t tmq_offset_seek(tmq_t* tmq, const char* pTopicName, int32_t vgId, int64_t offset) { if (tmq == NULL || pTopicName == NULL) { - tscError("invalid tmq handle, null"); + tqErrorC("invalid tmq handle, null"); return TSDB_CODE_INVALID_PARA; } @@ -3559,7 +3571,7 @@ int32_t tmq_offset_seek(tmq_t* tmq, const char* pTopicName, int32_t vgId, int64_ int32_t type = pOffsetInfo->endOffset.type; if (isInSnapshotMode(type, tmq->useSnapshot)) { - tscError("consumer:0x%" PRIx64 " offset type:%d not wal version, seek not allowed", tmq->consumerId, type); + tqErrorC("consumer:0x%" PRIx64 " offset type:%d not wal version, seek not allowed", tmq->consumerId, type); taosWUnLockLatch(&tmq->lock); return TSDB_CODE_TMQ_SNAPSHOT_ERROR; } @@ -3570,7 +3582,7 @@ int32_t tmq_offset_seek(tmq_t* tmq, const char* pTopicName, int32_t vgId, int64_ return code; } - tscInfo("consumer:0x%" PRIx64 " seek to %" PRId64 " on vgId:%d", tmq->consumerId, offset, vgId); + tqInfoC("consumer:0x%" PRIx64 " seek to %" PRId64 " on vgId:%d", tmq->consumerId, offset, vgId); // update the offset, and then commit to vnode pOffsetInfo->endOffset.type = TMQ_OFFSET__LOG; pOffsetInfo->endOffset.version = offset; @@ -3638,7 +3650,7 @@ int32_t tmq_offset_seek(tmq_t* tmq, const char* pTopicName, int32_t vgId, int64_ (void)tsem2_destroy(&pParam->sem); taosMemoryFree(pParam); - tscInfo("consumer:0x%" PRIx64 "send seek to vgId:%d, return code:%s", tmq->consumerId, vgId, tstrerror(code)); + tqInfoC("consumer:0x%" PRIx64 "send seek to vgId:%d, return code:%s", tmq->consumerId, vgId, tstrerror(code)); return code; } diff --git a/source/common/src/tmsg.c b/source/common/src/tmsg.c index 7973aa1b46..dd10aa5155 100644 --- a/source/common/src/tmsg.c +++ b/source/common/src/tmsg.c @@ -7050,6 +7050,7 @@ int32_t tSerializeSMqHbRsp(void *buf, int32_t bufLen, SMqHbRsp *pRsp) { if (tEncodeI8(&encoder, privilege->noPrivilege) < 0) return -1; } + if (tEncodeI32(&encoder, pRsp->debugFlag) < 0) return -1; tEndEncode(&encoder); int32_t tlen = encoder.pos; @@ -7075,6 +7076,10 @@ int32_t tDeserializeSMqHbRsp(void *buf, int32_t bufLen, SMqHbRsp *pRsp) { if (tDecodeI8(&decoder, &data->noPrivilege) < 0) return -1; } } + + if (!tDecodeIsEnd(&decoder)) { + if (tDecodeI32(&decoder, &pRsp->debugFlag) < 0) return -1; + } tEndDecode(&decoder); tDecoderClear(&decoder); diff --git a/source/dnode/mnode/impl/src/mndConsumer.c b/source/dnode/mnode/impl/src/mndConsumer.c index 7639349bac..136d3f9c6c 100644 --- a/source/dnode/mnode/impl/src/mndConsumer.c +++ b/source/dnode/mnode/impl/src/mndConsumer.c @@ -244,6 +244,7 @@ static int32_t mndProcessMqHbReq(SRpcMsg *pMsg) { } storeOffsetRows(pMnode, &req, pConsumer); + rsp.debugFlag = tqDebugFlag; code = buildMqHbRsp(pMsg, &rsp); END: @@ -609,7 +610,7 @@ END: tDeleteSMqConsumerObj(pConsumerNew); taosArrayDestroyP(subscribe.topicNames, (FDelete)taosMemoryFree); - return code == TSDB_CODE_TMQ_NO_NEED_REBALANCE ? 0 : code; + return (code == TSDB_CODE_TMQ_NO_NEED_REBALANCE || code == TSDB_CODE_MND_CONSUMER_NOT_EXIST) ? 0 : code; } SSdbRaw *mndConsumerActionEncode(SMqConsumerObj *pConsumer) { From d870d0a9670e319fb091e93bc17168dc3ad89e85 Mon Sep 17 00:00:00 2001 From: wangmm0220 Date: Mon, 9 Sep 2024 16:05:32 +0800 Subject: [PATCH 08/51] enh:[TD-30270]opti data struct in tmq --- include/client/taos.h | 2 +- include/common/tmsg.h | 83 +- source/client/inc/clientInt.h | 58 +- source/client/src/clientMain.c | 32 +- source/client/src/clientRawBlockWrite.c | 113 +- source/client/src/clientTmq.c | 1545 ++++++++----------- source/common/src/tmsg.c | 28 +- source/dnode/vnode/src/inc/tq.h | 12 +- source/dnode/vnode/src/tq/tq.c | 30 +- source/dnode/vnode/src/tq/tqScan.c | 54 +- source/dnode/vnode/src/tq/tqUtil.c | 42 +- tests/parallel_test/cases.task | 1 - tests/system-test/7-tmq/tmqClientConsLog.py | 231 --- tests/system-test/win-test-file | 1 - 14 files changed, 869 insertions(+), 1363 deletions(-) delete mode 100644 tests/system-test/7-tmq/tmqClientConsLog.py diff --git a/include/client/taos.h b/include/client/taos.h index 55b24c8721..1cd0a44315 100644 --- a/include/client/taos.h +++ b/include/client/taos.h @@ -339,7 +339,7 @@ DLL_EXPORT int32_t tmq_get_vgroup_id(TAOS_RES *res); DLL_EXPORT int64_t tmq_get_vgroup_offset(TAOS_RES* res); DLL_EXPORT const char *tmq_err2str(int32_t code); -/* ------------------------------ TAOSX -----------------------------------*/ +/* ------------------------------ TAOSX INTERFACE -----------------------------------*/ typedef struct tmq_raw_data { void *raw; uint32_t raw_len; diff --git a/include/common/tmsg.h b/include/common/tmsg.h index 9db4496377..c5b70d0175 100644 --- a/include/common/tmsg.h +++ b/include/common/tmsg.h @@ -649,6 +649,20 @@ void tFreeSSubmitRsp(SSubmitRsp* pRsp); (s)->flags &= (~COL_IDX_ON); \ } while (0) +enum { + RES_TYPE__QUERY = 1, + RES_TYPE__TMQ, + RES_TYPE__TMQ_META, + RES_TYPE__TMQ_METADATA, + RES_TYPE__TMQ_BATCH_META, +}; + +#define TD_RES_QUERY(res) (*(int8_t*)(res) == RES_TYPE__QUERY) +#define TD_RES_TMQ(res) (*(int8_t*)(res) == RES_TYPE__TMQ) +#define TD_RES_TMQ_META(res) (*(int8_t*)(res) == RES_TYPE__TMQ_META) +#define TD_RES_TMQ_METADATA(res) (*(int8_t*)(res) == RES_TYPE__TMQ_METADATA) +#define TD_RES_TMQ_BATCH_META(res) (*(int8_t*)(res) == RES_TYPE__TMQ_BATCH_META) + #define SSCHMEA_TYPE(s) ((s)->type) #define SSCHMEA_FLAGS(s) ((s)->flags) #define SSCHMEA_COLID(s) ((s)->colId) @@ -4042,38 +4056,53 @@ void tDeleteMqMetaRsp(SMqMetaRsp* pRsp); #define MQ_DATA_RSP_VERSION 100 -typedef struct { - SMqRspHead head; - STqOffsetVal reqOffset; - STqOffsetVal rspOffset; - int32_t blockNum; - int8_t withTbName; - int8_t withSchema; - SArray* blockDataLen; - SArray* blockData; - SArray* blockTbName; - SArray* blockSchema; -} SMqDataRspCommon; +//typedef struct { +// SMqRspHead head; +// STqOffsetVal rspOffset; +// STqOffsetVal reqOffset; +// int32_t blockNum; +// int8_t withTbName; +// int8_t withSchema; +// SArray* blockDataLen; +// SArray* blockData; +// SArray* blockTbName; +// SArray* blockSchema; +//} SMqDataRspCommon; typedef struct { - SMqDataRspCommon common; - int64_t sleepTime; + struct { + SMqRspHead head; + STqOffsetVal rspOffset; + STqOffsetVal reqOffset; + int32_t blockNum; + int8_t withTbName; + int8_t withSchema; + SArray* blockDataLen; + SArray* blockData; + SArray* blockTbName; + SArray* blockSchema; + }; + + union{ + struct{ + int64_t sleepTime; + }; + struct{ + int32_t createTableNum; + SArray* createTableLen; + SArray* createTableReq; + }; + }; + } SMqDataRsp; -int32_t tEncodeMqDataRsp(SEncoder* pEncoder, const void* pRsp); -int32_t tDecodeMqDataRsp(SDecoder* pDecoder, void* pRsp); -void tDeleteMqDataRsp(void* pRsp); +int32_t tEncodeMqDataRsp(SEncoder *pEncoder, const SMqDataRsp *pObj); +int32_t tDecodeMqDataRsp(SDecoder* pDecoder, SMqDataRsp* pRsp); +void tDeleteMqDataRsp(SMqDataRsp* pRsp); -typedef struct { - SMqDataRspCommon common; - int32_t createTableNum; - SArray* createTableLen; - SArray* createTableReq; -} STaosxRsp; - -int32_t tEncodeSTaosxRsp(SEncoder* pEncoder, const void* pRsp); -int32_t tDecodeSTaosxRsp(SDecoder* pDecoder, void* pRsp); -void tDeleteSTaosxRsp(void* pRsp); +int32_t tEncodeSTaosxRsp(SEncoder* pEncoder, const SMqDataRsp* pRsp); +int32_t tDecodeSTaosxRsp(SDecoder* pDecoder, SMqDataRsp* pRsp); +void tDeleteSTaosxRsp(SMqDataRsp* pRsp); typedef struct SMqBatchMetaRsp { SMqRspHead head; // not serialize diff --git a/source/client/inc/clientInt.h b/source/client/inc/clientInt.h index 9811003254..b198762733 100644 --- a/source/client/inc/clientInt.h +++ b/source/client/inc/clientInt.h @@ -39,24 +39,24 @@ extern "C" { #define ERROR_MSG_BUF_DEFAULT_SIZE 512 #define HEARTBEAT_INTERVAL 1500 // ms -enum { - RES_TYPE__QUERY = 1, - RES_TYPE__TMQ, - RES_TYPE__TMQ_META, - RES_TYPE__TMQ_METADATA, - RES_TYPE__TMQ_BATCH_META, -}; +//enum { +// RES_TYPE__QUERY = 1, +// RES_TYPE__TMQ, +// RES_TYPE__TMQ_META, +// RES_TYPE__TMQ_METADATA, +// RES_TYPE__TMQ_BATCH_META, +//}; #define SHOW_VARIABLES_RESULT_COLS 3 #define SHOW_VARIABLES_RESULT_FIELD1_LEN (TSDB_CONFIG_OPTION_LEN + VARSTR_HEADER_SIZE) #define SHOW_VARIABLES_RESULT_FIELD2_LEN (TSDB_CONFIG_VALUE_LEN + VARSTR_HEADER_SIZE) #define SHOW_VARIABLES_RESULT_FIELD3_LEN (TSDB_CONFIG_SCOPE_LEN + VARSTR_HEADER_SIZE) -#define TD_RES_QUERY(res) (*(int8_t*)(res) == RES_TYPE__QUERY) -#define TD_RES_TMQ(res) (*(int8_t*)(res) == RES_TYPE__TMQ) -#define TD_RES_TMQ_META(res) (*(int8_t*)(res) == RES_TYPE__TMQ_META) -#define TD_RES_TMQ_METADATA(res) (*(int8_t*)(res) == RES_TYPE__TMQ_METADATA) -#define TD_RES_TMQ_BATCH_META(res) (*(int8_t*)(res) == RES_TYPE__TMQ_BATCH_META) +//#define TD_RES_QUERY(res) (*(int8_t*)(res) == RES_TYPE__QUERY) +//#define TD_RES_TMQ(res) (*(int8_t*)(res) == RES_TYPE__TMQ) +//#define TD_RES_TMQ_META(res) (*(int8_t*)(res) == RES_TYPE__TMQ_META) +//#define TD_RES_TMQ_METADATA(res) (*(int8_t*)(res) == RES_TYPE__TMQ_METADATA) +//#define TD_RES_TMQ_BATCH_META(res) (*(int8_t*)(res) == RES_TYPE__TMQ_BATCH_META) typedef struct SAppInstInfo SAppInstInfo; @@ -226,31 +226,17 @@ typedef struct { SSchemaWrapper schema; int32_t resIter; SReqResultInfo resInfo; -} SMqRspObjCommon; - -typedef struct { - SMqRspObjCommon common; - SMqDataRsp rsp; + union{ + struct{ + SMqRspHead head; + STqOffsetVal rspOffset; + }; + SMqDataRsp dataRsp; + SMqMetaRsp metaRsp; + SMqBatchMetaRsp batchMetaRsp; + }; } SMqRspObj; -typedef struct { - int8_t resType; - char topic[TSDB_TOPIC_FNAME_LEN]; - char db[TSDB_DB_FNAME_LEN]; - int32_t vgId; - SMqMetaRsp metaRsp; -} SMqMetaRspObj; - -typedef struct { - SMqRspObjCommon common; - STaosxRsp rsp; -} SMqTaosxRspObj; - -typedef struct { - SMqRspObjCommon common; - SMqBatchMetaRsp rsp; -} SMqBatchMetaRspObj; - typedef struct SReqRelInfo { uint64_t userRefId; uint64_t prevRefId; @@ -330,7 +316,7 @@ int32_t getVersion1BlockMetaSize(const char* p, int32_t numOfCols); static FORCE_INLINE SReqResultInfo* tmqGetCurResInfo(TAOS_RES* res) { SMqRspObj* msg = (SMqRspObj*)res; - return (SReqResultInfo*)&msg->common.resInfo; + return (SReqResultInfo*)&msg->resInfo; } int32_t tmqGetNextResInfo(TAOS_RES* res, bool convertUcs4, SReqResultInfo** pResInfo); diff --git a/source/client/src/clientMain.c b/source/client/src/clientMain.c index 27c1878dc8..7439aa64d3 100644 --- a/source/client/src/clientMain.c +++ b/source/client/src/clientMain.c @@ -371,25 +371,19 @@ void taos_free_result(TAOS_RES *res) { SRequestObj *pRequest = (SRequestObj *)res; tscDebug("0x%" PRIx64 " taos_free_result start to free query", pRequest->requestId); destroyRequest(pRequest); - } else if (TD_RES_TMQ_METADATA(res)) { - SMqTaosxRspObj *pRsp = (SMqTaosxRspObj *)res; - tDeleteSTaosxRsp(&pRsp->rsp); - doFreeReqResultInfo(&pRsp->common.resInfo); - taosMemoryFree(pRsp); - } else if (TD_RES_TMQ(res)) { - SMqRspObj *pRsp = (SMqRspObj *)res; - tDeleteMqDataRsp(&pRsp->rsp); - doFreeReqResultInfo(&pRsp->common.resInfo); - taosMemoryFree(pRsp); - } else if (TD_RES_TMQ_META(res)) { - SMqMetaRspObj *pRspObj = (SMqMetaRspObj *)res; - tDeleteMqMetaRsp(&pRspObj->metaRsp); - taosMemoryFree(pRspObj); - } else if (TD_RES_TMQ_BATCH_META(res)) { - SMqBatchMetaRspObj *pBtRspObj = (SMqBatchMetaRspObj *)res; - tDeleteMqBatchMetaRsp(&pBtRspObj->rsp); - taosMemoryFree(pBtRspObj); + return; } + SMqRspObj *pRsp = (SMqRspObj *)res; + if (TD_RES_TMQ_METADATA(res) || TD_RES_TMQ(res)) { + tDeleteMqDataRsp(&pRsp->dataRsp); + doFreeReqResultInfo(&pRsp->resInfo); + } else if (TD_RES_TMQ_META(res)) { + tDeleteMqMetaRsp(&pRsp->metaRsp); + } else if (TD_RES_TMQ_BATCH_META(res)) { + tDeleteMqBatchMetaRsp(&pRsp->batchMetaRsp); + } + taosMemoryFree(pRsp); + } void taos_kill_query(TAOS *taos) { @@ -452,7 +446,7 @@ TAOS_ROW taos_fetch_row(TAOS_RES *res) { } else if (TD_RES_TMQ(res) || TD_RES_TMQ_METADATA(res)) { SMqRspObj *msg = ((SMqRspObj *)res); SReqResultInfo *pResultInfo = NULL; - if (msg->common.resIter == -1) { + if (msg->resIter == -1) { if (tmqGetNextResInfo(res, true, &pResultInfo) != 0) { return NULL; } diff --git a/source/client/src/clientRawBlockWrite.c b/source/client/src/clientRawBlockWrite.c index d5785cce6b..d481ede616 100644 --- a/source/client/src/clientRawBlockWrite.c +++ b/source/client/src/clientRawBlockWrite.c @@ -544,7 +544,7 @@ end: tDecoderClear(&decoder); } -static void processAutoCreateTable(STaosxRsp* rsp, char** string) { +static void processAutoCreateTable(SMqDataRsp* rsp, char** string) { SDecoder* decoder = NULL; SVCreateTbReq* pCreateReq = NULL; int32_t code = 0; @@ -1677,8 +1677,8 @@ static int32_t tmqWriteRawDataImpl(TAOS* taos, void* data, int32_t dataLen) { uDebug(LOG_ID_TAG " write raw data, data:%p, dataLen:%d", LOG_ID_VALUE, data, dataLen); pRequest->syncQuery = true; - rspObj.common.resIter = -1; - rspObj.common.resType = RES_TYPE__TMQ; + rspObj.resIter = -1; + rspObj.resType = RES_TYPE__TMQ; int8_t dataVersion = *(int8_t*)data; if (dataVersion >= MQ_DATA_RSP_VERSION) { @@ -1686,7 +1686,7 @@ static int32_t tmqWriteRawDataImpl(TAOS* taos, void* data, int32_t dataLen) { dataLen -= sizeof(int8_t) + sizeof(int32_t); } tDecoderInit(&decoder, data, dataLen); - code = tDecodeMqDataRsp(&decoder, &rspObj.rsp); + code = tDecodeMqDataRsp(&decoder, &rspObj.dataRsp); if (code != 0) { SET_ERROR_MSG("decode mq data rsp failed"); code = TSDB_CODE_INVALID_MSG; @@ -1710,14 +1710,14 @@ static int32_t tmqWriteRawDataImpl(TAOS* taos, void* data, int32_t dataLen) { RAW_RETURN_CHECK(smlInitHandle(&pQuery)); pVgHash = taosHashInit(16, taosGetDefaultHashFunction(TSDB_DATA_TYPE_INT), true, HASH_NO_LOCK); RAW_NULL_CHECK(pVgHash); - while (++rspObj.common.resIter < rspObj.rsp.common.blockNum) { - void* pRetrieve = taosArrayGetP(rspObj.rsp.common.blockData, rspObj.common.resIter); + while (++rspObj.resIter < rspObj.dataRsp.blockNum) { + void* pRetrieve = taosArrayGetP(rspObj.dataRsp.blockData, rspObj.resIter); RAW_NULL_CHECK(pRetrieve); - if (!rspObj.rsp.common.withSchema) { + if (!rspObj.dataRsp.withSchema) { goto end; } - const char* tbName = (const char*)taosArrayGetP(rspObj.rsp.common.blockTbName, rspObj.common.resIter); + const char* tbName = (const char*)taosArrayGetP(rspObj.dataRsp.blockTbName, rspObj.resIter); RAW_NULL_CHECK(tbName); SName pName = {TSDB_TABLE_NAME_T, pRequest->pTscObj->acctId, {0}, {0}}; @@ -1734,7 +1734,7 @@ static int32_t tmqWriteRawDataImpl(TAOS* taos, void* data, int32_t dataLen) { RAW_RETURN_CHECK(taosHashPut(pVgHash, (const char*)&vg.vgId, sizeof(vg.vgId), (char*)&vg, sizeof(vg))); } - SSchemaWrapper* pSW = (SSchemaWrapper*)taosArrayGetP(rspObj.rsp.common.blockSchema, rspObj.common.resIter); + SSchemaWrapper* pSW = (SSchemaWrapper*)taosArrayGetP(rspObj.dataRsp.blockSchema, rspObj.resIter); RAW_NULL_CHECK(pSW); TAOS_FIELD* fields = taosMemoryCalloc(pSW->nCols, sizeof(TAOS_FIELD)); RAW_NULL_CHECK(fields); @@ -1761,7 +1761,7 @@ static int32_t tmqWriteRawDataImpl(TAOS* taos, void* data, int32_t dataLen) { end: uDebug(LOG_ID_TAG " write raw data return, msg:%s", LOG_ID_VALUE, tstrerror(code)); - tDeleteMqDataRsp(&rspObj.rsp); + tDeleteMqDataRsp(&rspObj.dataRsp); tDecoderClear(&decoder); qDestroyQuery(pQuery); destroyRequest(pRequest); @@ -1778,7 +1778,7 @@ static int32_t tmqWriteRawMetaDataImpl(TAOS* taos, void* data, int32_t dataLen) int32_t code = TSDB_CODE_SUCCESS; SHashObj* pVgHash = NULL; SQuery* pQuery = NULL; - SMqTaosxRspObj rspObj = {0}; + SMqRspObj rspObj = {0}; SDecoder decoder = {0}; STableMeta* pTableMeta = NULL; SVCreateTbReq* pCreateReqDst = NULL; @@ -1788,8 +1788,8 @@ static int32_t tmqWriteRawMetaDataImpl(TAOS* taos, void* data, int32_t dataLen) uDebug(LOG_ID_TAG " write raw metadata, data:%p, dataLen:%d", LOG_ID_VALUE, data, dataLen); pRequest->syncQuery = true; - rspObj.common.resIter = -1; - rspObj.common.resType = RES_TYPE__TMQ_METADATA; + rspObj.resIter = -1; + rspObj.resType = RES_TYPE__TMQ_METADATA; int8_t dataVersion = *(int8_t*)data; if (dataVersion >= MQ_DATA_RSP_VERSION) { @@ -1798,7 +1798,7 @@ static int32_t tmqWriteRawMetaDataImpl(TAOS* taos, void* data, int32_t dataLen) } tDecoderInit(&decoder, data, dataLen); - code = tDecodeSTaosxRsp(&decoder, &rspObj.rsp); + code = tDecodeSTaosxRsp(&decoder, &rspObj.dataRsp); if (code != 0) { SET_ERROR_MSG("decode mq taosx data rsp failed"); code = TSDB_CODE_INVALID_MSG; @@ -1823,15 +1823,15 @@ static int32_t tmqWriteRawMetaDataImpl(TAOS* taos, void* data, int32_t dataLen) pVgHash = taosHashInit(16, taosGetDefaultHashFunction(TSDB_DATA_TYPE_INT), true, HASH_NO_LOCK); RAW_NULL_CHECK(pVgHash); - uDebug(LOG_ID_TAG " write raw metadata block num:%d", LOG_ID_VALUE, rspObj.rsp.common.blockNum); - while (++rspObj.common.resIter < rspObj.rsp.common.blockNum) { - void* pRetrieve = taosArrayGetP(rspObj.rsp.common.blockData, rspObj.common.resIter); + uDebug(LOG_ID_TAG " write raw metadata block num:%d", LOG_ID_VALUE, rspObj.dataRsp.blockNum); + while (++rspObj.resIter < rspObj.dataRsp.blockNum) { + void* pRetrieve = taosArrayGetP(rspObj.dataRsp.blockData, rspObj.resIter); RAW_NULL_CHECK(pRetrieve); - if (!rspObj.rsp.common.withSchema) { + if (!rspObj.dataRsp.withSchema) { goto end; } - const char* tbName = (const char*)taosArrayGetP(rspObj.rsp.common.blockTbName, rspObj.common.resIter); + const char* tbName = (const char*)taosArrayGetP(rspObj.dataRsp.blockTbName, rspObj.resIter); if (!tbName) { SET_ERROR_MSG("block tbname is null"); code = TSDB_CODE_TMQ_INVALID_MSG; @@ -1844,10 +1844,10 @@ static int32_t tmqWriteRawMetaDataImpl(TAOS* taos, void* data, int32_t dataLen) (void)strcpy(pName.tname, tbName); // find schema data info - for (int j = 0; j < rspObj.rsp.createTableNum; j++) { - void** dataTmp = taosArrayGet(rspObj.rsp.createTableReq, j); + for (int j = 0; j < rspObj.dataRsp.createTableNum; j++) { + void** dataTmp = taosArrayGet(rspObj.dataRsp.createTableReq, j); RAW_NULL_CHECK(dataTmp); - int32_t* lenTmp = taosArrayGet(rspObj.rsp.createTableLen, j); + int32_t* lenTmp = taosArrayGet(rspObj.dataRsp.createTableLen, j); RAW_NULL_CHECK(dataTmp); SDecoder decoderTmp = {0}; @@ -1895,7 +1895,7 @@ static int32_t tmqWriteRawMetaDataImpl(TAOS* taos, void* data, int32_t dataLen) RAW_RETURN_CHECK(taosHashPut(pVgHash, (const char*)&vg.vgId, sizeof(vg.vgId), (char*)&vg, sizeof(vg))); } - SSchemaWrapper* pSW = (SSchemaWrapper*)taosArrayGetP(rspObj.rsp.common.blockSchema, rspObj.common.resIter); + SSchemaWrapper* pSW = (SSchemaWrapper*)taosArrayGetP(rspObj.dataRsp.blockSchema, rspObj.resIter); RAW_NULL_CHECK(pSW); TAOS_FIELD* fields = taosMemoryCalloc(pSW->nCols, sizeof(TAOS_FIELD)); if (fields == NULL) { @@ -1926,7 +1926,7 @@ static int32_t tmqWriteRawMetaDataImpl(TAOS* taos, void* data, int32_t dataLen) end: uDebug(LOG_ID_TAG " write raw metadata return, msg:%s", LOG_ID_VALUE, tstrerror(code)); - tDeleteSTaosxRsp(&rspObj.rsp); + tDeleteSTaosxRsp(&rspObj.dataRsp); tDecoderClear(&decoder); qDestroyQuery(pQuery); destroyRequest(pRequest); @@ -2011,30 +2011,26 @@ char* tmq_get_json_meta(TAOS_RES* res) { return NULL; } + char* string = NULL; + SMqRspObj* rspObj = (SMqRspObj*)res; if (TD_RES_TMQ_METADATA(res)) { - SMqTaosxRspObj* pMetaDataRspObj = (SMqTaosxRspObj*)res; - char* string = NULL; - processAutoCreateTable(&pMetaDataRspObj->rsp, &string); - return string; + processAutoCreateTable(&rspObj->dataRsp, &string); } else if (TD_RES_TMQ_BATCH_META(res)) { - SMqBatchMetaRspObj* pBatchMetaRspObj = (SMqBatchMetaRspObj*)res; - char* string = NULL; - processBatchMetaToJson(&pBatchMetaRspObj->rsp, &string); - return string; + processBatchMetaToJson(&rspObj->batchMetaRsp, &string); + } else if (TD_RES_TMQ_META(res)) { + cJSON* pJson = NULL; + processSimpleMeta(&rspObj->metaRsp, &pJson); + string = cJSON_PrintUnformatted(pJson); + cJSON_Delete(pJson); + } else{ + uError("tmq_get_json_meta res:%d, invalid type", *(int8_t*)res); } - - SMqMetaRspObj* pMetaRspObj = (SMqMetaRspObj*)res; - cJSON* pJson = NULL; - processSimpleMeta(&pMetaRspObj->metaRsp, &pJson); - char* string = cJSON_PrintUnformatted(pJson); - cJSON_Delete(pJson); return string; } void tmq_free_json_meta(char* jsonMeta) { taosMemoryFreeClear(jsonMeta); } -static int32_t getOffSetLen(const void* rsp) { - const SMqDataRspCommon* pRsp = rsp; +static int32_t getOffSetLen(const SMqDataRsp* pRsp) { SEncoder coder = {0}; tEncoderInit(&coder, NULL, 0); if (tEncodeSTqOffsetVal(&coder, &pRsp->reqOffset) < 0) return -1; @@ -2044,9 +2040,8 @@ static int32_t getOffSetLen(const void* rsp) { return pos; } -typedef int32_t __encode_func__(SEncoder* pEncoder, const void* pRsp); - -static int32_t encodeMqDataRsp(__encode_func__* encodeFunc, void* rspObj, tmq_raw_data* raw) { +typedef int32_t __encode_func__(SEncoder* pEncoder, const SMqDataRsp* pRsp); +static int32_t encodeMqDataRsp(__encode_func__* encodeFunc, SMqDataRsp* rspObj, tmq_raw_data* raw) { int32_t len = 0; int32_t code = 0; SEncoder encoder = {0}; @@ -2095,36 +2090,24 @@ int32_t tmq_get_raw(TAOS_RES* res, tmq_raw_data* raw) { if (!raw || !res) { return TSDB_CODE_INVALID_PARA; } + SMqRspObj* rspObj = ((SMqRspObj*)res); if (TD_RES_TMQ_META(res)) { - SMqMetaRspObj* pMetaRspObj = (SMqMetaRspObj*)res; - raw->raw = pMetaRspObj->metaRsp.metaRsp; - raw->raw_len = pMetaRspObj->metaRsp.metaRspLen; - raw->raw_type = pMetaRspObj->metaRsp.resMsgType; + raw->raw = rspObj->metaRsp.metaRsp; + raw->raw_len = rspObj->metaRsp.metaRspLen; + raw->raw_type = rspObj->metaRsp.resMsgType; uDebug("tmq get raw type meta:%p", raw); - } else if (TD_RES_TMQ(res)) { - SMqRspObj* rspObj = ((SMqRspObj*)res); - int32_t code = encodeMqDataRsp(tEncodeMqDataRsp, &rspObj->rsp, raw); + } else if (TD_RES_TMQ(res) || TD_RES_TMQ_METADATA(res)) { + int32_t code = encodeMqDataRsp(tEncodeMqDataRsp, &rspObj->dataRsp, raw); if (code != 0) { uError("tmq get raw type error:%d", terrno); return code; } - raw->raw_type = RES_TYPE__TMQ; - uDebug("tmq get raw type data:%p", raw); - } else if (TD_RES_TMQ_METADATA(res)) { - SMqTaosxRspObj* rspObj = ((SMqTaosxRspObj*)res); - - int32_t code = encodeMqDataRsp(tEncodeSTaosxRsp, &rspObj->rsp, raw); - if (code != 0) { - uError("tmq get raw type error:%d", terrno); - return code; - } - raw->raw_type = RES_TYPE__TMQ_METADATA; + raw->raw_type = rspObj->resType; uDebug("tmq get raw type metadata:%p", raw); } else if (TD_RES_TMQ_BATCH_META(res)) { - SMqBatchMetaRspObj* pBtMetaRspObj = (SMqBatchMetaRspObj*)res; - raw->raw = pBtMetaRspObj->rsp.pMetaBuff; - raw->raw_len = pBtMetaRspObj->rsp.metaBuffLen; - raw->raw_type = RES_TYPE__TMQ_BATCH_META; + raw->raw = rspObj->batchMetaRsp.pMetaBuff; + raw->raw_len = rspObj->batchMetaRsp.metaBuffLen; + raw->raw_type = rspObj->resType; uDebug("tmq get raw batch meta:%p", raw); } else { uError("tmq get raw error type:%d", *(int8_t*)res); diff --git a/source/client/src/clientTmq.c b/source/client/src/clientTmq.c index cb3896f81a..2ee9396f7f 100644 --- a/source/client/src/clientTmq.c +++ b/source/client/src/clientTmq.c @@ -39,28 +39,46 @@ #define SUBSCRIBE_RETRY_MAX_COUNT 240 #define SUBSCRIBE_RETRY_INTERVAL 500 -struct SMqMgmt { - tmr_h timer; - int32_t rsetId; + +#define SET_ERROR_MSG_TMQ(MSG) \ + if (errstr != NULL) (void)snprintf(errstr, errstrLen, MSG); + +#define PROCESS_POLL_RSP(FUNC,DATA) \ + SDecoder decoder = {0}; \ + tDecoderInit(&decoder, POINTER_SHIFT(pMsg->pData, sizeof(SMqRspHead)), pMsg->len - sizeof(SMqRspHead)); \ + if (FUNC(&decoder, DATA) < 0) { \ + tDecoderClear(&decoder); \ + code = TSDB_CODE_OUT_OF_MEMORY; \ + goto END;\ + }\ + tDecoderClear(&decoder);\ + (void)memcpy(DATA, pMsg->pData, sizeof(SMqRspHead)); + +#define DELETE_POLL_RSP(FUNC,DATA) \ + SMqPollRspWrapper* pRsp = (SMqPollRspWrapper*)rspWrapper;\ + taosMemoryFreeClear(pRsp->pEpset);\ + FUNC(DATA); + +enum { + TMQ_VG_STATUS__IDLE = 0, + TMQ_VG_STATUS__WAIT, }; -static TdThreadOnce tmqInit = PTHREAD_ONCE_INIT; // initialize only once -volatile int32_t tmqInitRes = 0; // initialize rsp code -static struct SMqMgmt tmqMgmt = {0}; -static int8_t pollFlag = 0; +enum { + TMQ_CONSUMER_STATUS__INIT = 0, + TMQ_CONSUMER_STATUS__READY, + TMQ_CONSUMER_STATUS__CLOSED, +}; + +enum { + TMQ_DELAYED_TASK__ASK_EP = 1, + TMQ_DELAYED_TASK__COMMIT, +}; typedef struct { - int32_t code; - int8_t tmqRspType; - int32_t epoch; -} SMqRspWrapper; - -typedef struct { - int32_t code; - int8_t tmqRspType; - int32_t epoch; - SMqAskEpRsp msg; -} SMqAskEpRspWrapper; + tmr_h timer; + int32_t rsetId; +} SMqMgmt; struct tmq_list_t { SArray container; @@ -129,28 +147,12 @@ struct tmq_t { tsem2_t rspSem; }; -typedef struct SAskEpInfo { +typedef struct { int32_t code; tsem2_t sem; } SAskEpInfo; -enum { - TMQ_VG_STATUS__IDLE = 0, - TMQ_VG_STATUS__WAIT, -}; - -enum { - TMQ_CONSUMER_STATUS__INIT = 0, - TMQ_CONSUMER_STATUS__READY, - TMQ_CONSUMER_STATUS__CLOSED, -}; - -enum { - TMQ_DELAYED_TASK__ASK_EP = 1, - TMQ_DELAYED_TASK__COMMIT, -}; - -typedef struct SVgOffsetInfo { +typedef struct { STqOffsetVal committedOffset; STqOffsetVal endOffset; // the last version in TAOS_RES + 1 STqOffsetVal beginOffset; // the first version in TAOS_RES @@ -181,23 +183,32 @@ typedef struct { } SMqClientTopic; typedef struct { - int32_t code; - int8_t tmqRspType; - int32_t epoch; // epoch can be used to guard the vgHandle int32_t vgId; char topicName[TSDB_TOPIC_FNAME_LEN]; - SMqClientVg* vgHandle; SMqClientTopic* topicHandle; uint64_t reqId; SEpSet* pEpset; union { + struct{ + SMqRspHead head; + STqOffsetVal rspOffset; + }; SMqDataRsp dataRsp; SMqMetaRsp metaRsp; - STaosxRsp taosxRsp; SMqBatchMetaRsp batchMetaRsp; }; } SMqPollRspWrapper; +typedef struct { + int32_t code; + int8_t tmqRspType; + int32_t epoch; + union{ + SMqPollRspWrapper pollRsp; + SMqAskEpRsp epRsp; + }; +} SMqRspWrapper; + typedef struct { tsem2_t rspSem; int32_t rspErr; @@ -216,7 +227,7 @@ typedef struct { uint64_t requestId; // request id for debug purpose } SMqPollCbParam; -typedef struct SMqVgCommon { +typedef struct { tsem2_t rsp; int32_t numOfRsp; SArray* pList; @@ -226,18 +237,18 @@ typedef struct SMqVgCommon { int32_t code; } SMqVgCommon; -typedef struct SMqSeekParam { +typedef struct { tsem2_t sem; int32_t code; } SMqSeekParam; -typedef struct SMqCommittedParam { +typedef struct { tsem2_t sem; int32_t code; SMqVgOffset vgOffset; } SMqCommittedParam; -typedef struct SMqVgWalInfoParam { +typedef struct { int32_t vgId; int32_t epoch; int32_t totalReq; @@ -260,17 +271,23 @@ typedef struct { int64_t consumerId; } SMqCommitCbParam; -typedef struct SSyncCommitInfo { +typedef struct { tsem2_t sem; int32_t code; } SSyncCommitInfo; -static int32_t syncAskEp(tmq_t* tmq); -static int32_t tmqCommitDone(SMqCommitCbParamSet* pParamSet); -static int32_t doSendCommitMsg(tmq_t* tmq, int32_t vgId, SEpSet* epSet, STqOffsetVal* offset, const char* pTopicName, - SMqCommitCbParamSet* pParamSet); -static int32_t commitRspCountDown(SMqCommitCbParamSet* pParamSet, int64_t consumerId, const char* pTopic, int32_t vgId); -static int32_t askEp(tmq_t* pTmq, void* param, bool sync, bool updateEpset); +typedef struct { + STqOffsetVal currentOffset; + STqOffsetVal commitOffset; + STqOffsetVal seekOffset; + int64_t numOfRows; + int32_t vgStatus; +} SVgroupSaveInfo; + +static TdThreadOnce tmqInit = PTHREAD_ONCE_INIT; // initialize only once +volatile int32_t tmqInitRes = 0; // initialize rsp code +static SMqMgmt tmqMgmt = {0}; +static int8_t pollFlag = 0; tmq_conf_t* tmq_conf_new() { tmq_conf_t* conf = taosMemoryCalloc(1, sizeof(tmq_conf_t)); @@ -459,17 +476,25 @@ tmq_conf_res_t tmq_conf_set(tmq_conf_t* conf, const char* key, const char* value return TMQ_CONF_UNKNOWN; } -tmq_list_t* tmq_list_new() { return (tmq_list_t*)taosArrayInit(0, sizeof(void*)); } +tmq_list_t* tmq_list_new() { + return (tmq_list_t*)taosArrayInit(0, sizeof(void*)); +} int32_t tmq_list_append(tmq_list_t* list, const char* src) { - if (list == NULL) return TSDB_CODE_INVALID_PARA; + if (list == NULL) { + return TSDB_CODE_INVALID_PARA; + } SArray* container = &list->container; - if (src == NULL || src[0] == 0) return TSDB_CODE_INVALID_PARA; + if (src == NULL || src[0] == 0) { + return TSDB_CODE_INVALID_PARA; + } char* topic = taosStrdup(src); if (topic == NULL){ return TSDB_CODE_OUT_OF_MEMORY; } - if (taosArrayPush(container, &topic) == NULL) return TSDB_CODE_INVALID_PARA; + if (taosArrayPush(container, &topic) == NULL) { + return TSDB_CODE_INVALID_PARA; + } return 0; } @@ -480,17 +505,55 @@ void tmq_list_destroy(tmq_list_t* list) { } int32_t tmq_list_get_size(const tmq_list_t* list) { - if (list == NULL) return TSDB_CODE_INVALID_PARA; + if (list == NULL) { + return TSDB_CODE_INVALID_PARA; + } const SArray* container = &list->container; return taosArrayGetSize(container); } char** tmq_list_to_c_array(const tmq_list_t* list) { - if (list == NULL) return NULL; + if (list == NULL) { + return NULL; + } const SArray* container = &list->container; return container->pData; } +static int32_t tmqCommitDone(SMqCommitCbParamSet* pParamSet) { + int64_t refId = pParamSet->refId; + int32_t code = 0; + tmq_t* tmq = taosAcquireRef(tmqMgmt.rsetId, refId); + if (tmq == NULL) { + code = TSDB_CODE_TMQ_CONSUMER_CLOSED; + } + + // if no more waiting rsp + if (pParamSet->callbackFn != NULL) { + pParamSet->callbackFn(tmq, pParamSet->code, pParamSet->userParam); + } + + taosMemoryFree(pParamSet); + if (tmq != NULL) { + code = taosReleaseRef(tmqMgmt.rsetId, refId); + } + + return code; +} + +static int32_t commitRspCountDown(SMqCommitCbParamSet* pParamSet, int64_t consumerId, const char* pTopic, int32_t vgId) { + int32_t waitingRspNum = atomic_sub_fetch_32(&pParamSet->waitingRspNum, 1); + if (waitingRspNum == 0) { + tqDebugC("consumer:0x%" PRIx64 " topic:%s vgId:%d all commit-rsp received, commit completed", consumerId, pTopic, + vgId); + return tmqCommitDone(pParamSet); + } else { + tqDebugC("consumer:0x%" PRIx64 " topic:%s vgId:%d commit-rsp received, remain:%d", consumerId, pTopic, vgId, + waitingRspNum); + } + return 0; +} + static int32_t tmqCommitCb(void* param, SDataBuf* pBuf, int32_t code) { if (pBuf){ taosMemoryFreeClear(pBuf->pData); @@ -690,26 +753,12 @@ static void asyncCommitFromResult(tmq_t* tmq, const TAOS_RES* pRes, tmq_commit_c goto end; } - if (TD_RES_TMQ(pRes)) { + if (TD_RES_TMQ(pRes) || TD_RES_TMQ_META(pRes) || + TD_RES_TMQ_METADATA(pRes) || TD_RES_TMQ_BATCH_META(pRes)) { SMqRspObj* pRspObj = (SMqRspObj*)pRes; - pTopicName = pRspObj->common.topic; - vgId = pRspObj->common.vgId; - offsetVal = pRspObj->rsp.common.rspOffset; - } else if (TD_RES_TMQ_META(pRes)) { - SMqMetaRspObj* pMetaRspObj = (SMqMetaRspObj*)pRes; - pTopicName = pMetaRspObj->topic; - vgId = pMetaRspObj->vgId; - offsetVal = pMetaRspObj->metaRsp.rspOffset; - } else if (TD_RES_TMQ_METADATA(pRes)) { - SMqTaosxRspObj* pRspObj = (SMqTaosxRspObj*)pRes; - pTopicName = pRspObj->common.topic; - vgId = pRspObj->common.vgId; - offsetVal = pRspObj->rsp.common.rspOffset; - } else if (TD_RES_TMQ_BATCH_META(pRes)) { - SMqBatchMetaRspObj* pBtRspObj = (SMqBatchMetaRspObj*)pRes; - pTopicName = pBtRspObj->common.topic; - vgId = pBtRspObj->common.vgId; - offsetVal = pBtRspObj->rsp.rspOffset; + pTopicName = pRspObj->topic; + vgId = pRspObj->vgId; + offsetVal = pRspObj->rspOffset; } else { code = TSDB_CODE_TMQ_INVALID_MSG; goto end; @@ -822,10 +871,14 @@ void tmqAssignDelayedCommitTask(void* param, void* tmrId) { } int32_t tmqHbCb(void* param, SDataBuf* pMsg, int32_t code) { - if (pMsg == NULL || param == NULL) { + if (pMsg == NULL) { return TSDB_CODE_INVALID_PARA; } + if (param == NULL){ + goto END; + } + if (code != 0) { goto END; } @@ -973,6 +1026,309 @@ static void defaultCommitCbFn(tmq_t* pTmq, int32_t code, void* param) { } } +static void tmqFreeRspWrapper(SMqRspWrapper* rspWrapper) { + if (rspWrapper->tmqRspType == TMQ_MSG_TYPE__EP_RSP) { + tDeleteSMqAskEpRsp(&rspWrapper->epRsp); + } else if (rspWrapper->tmqRspType == TMQ_MSG_TYPE__POLL_DATA_RSP || + rspWrapper->tmqRspType == TMQ_MSG_TYPE__POLL_DATA_META_RSP) { + DELETE_POLL_RSP(tDeleteMqDataRsp,&pRsp->dataRsp) + } else if (rspWrapper->tmqRspType == TMQ_MSG_TYPE__POLL_META_RSP) { + DELETE_POLL_RSP(tDeleteMqMetaRsp,&pRsp->metaRsp) + } else if (rspWrapper->tmqRspType == TMQ_MSG_TYPE__POLL_BATCH_META_RSP) { + DELETE_POLL_RSP(tDeleteMqBatchMetaRsp,&pRsp->batchMetaRsp) + } +} + +static void freeClientVg(void* param) { + SMqClientVg* pVg = param; + tOffsetDestroy(&pVg->offsetInfo.endOffset); + tOffsetDestroy(&pVg->offsetInfo.beginOffset); + tOffsetDestroy(&pVg->offsetInfo.committedOffset); +} +static void freeClientTopic(void* param) { + SMqClientTopic* pTopic = param; + taosMemoryFreeClear(pTopic->schema.pSchema); + taosArrayDestroyEx(pTopic->vgs, freeClientVg); +} + +static void initClientTopicFromRsp(SMqClientTopic* pTopic, SMqSubTopicEp* pTopicEp, SHashObj* pVgOffsetHashMap, + tmq_t* tmq) { + pTopic->schema = pTopicEp->schema; + pTopicEp->schema.nCols = 0; + pTopicEp->schema.pSchema = NULL; + + char vgKey[TSDB_TOPIC_FNAME_LEN + 22] = {0}; + int32_t vgNumGet = taosArrayGetSize(pTopicEp->vgs); + + tstrncpy(pTopic->topicName, pTopicEp->topic, TSDB_TOPIC_FNAME_LEN); + tstrncpy(pTopic->db, pTopicEp->db, TSDB_DB_FNAME_LEN); + + tqInfoC("consumer:0x%" PRIx64 ", update topic:%s, new numOfVgs:%d", tmq->consumerId, pTopic->topicName, vgNumGet); + pTopic->vgs = taosArrayInit(vgNumGet, sizeof(SMqClientVg)); + if (pTopic->vgs == NULL) { + tqErrorC("consumer:0x%" PRIx64 ", failed to init vgs for topic:%s", tmq->consumerId, pTopic->topicName); + return; + } + for (int32_t j = 0; j < vgNumGet; j++) { + SMqSubVgEp* pVgEp = taosArrayGet(pTopicEp->vgs, j); + if (pVgEp == NULL) { + continue; + } + (void)sprintf(vgKey, "%s:%d", pTopic->topicName, pVgEp->vgId); + SVgroupSaveInfo* pInfo = taosHashGet(pVgOffsetHashMap, vgKey, strlen(vgKey)); + + STqOffsetVal offsetNew = {0}; + offsetNew.type = tmq->resetOffsetCfg; + + tqInfoC("consumer:0x%" PRIx64 ", update topic:%s, new numOfVgs:%d, num:%d, port:%d", tmq->consumerId, + pTopic->topicName, vgNumGet, pVgEp->epSet.numOfEps, pVgEp->epSet.eps[pVgEp->epSet.inUse].port); + + SMqClientVg clientVg = { + .pollCnt = 0, + .vgId = pVgEp->vgId, + .epSet = pVgEp->epSet, + .vgStatus = pInfo ? pInfo->vgStatus : TMQ_VG_STATUS__IDLE, + .vgSkipCnt = 0, + .emptyBlockReceiveTs = 0, + .blockReceiveTs = 0, + .blockSleepForReplay = 0, + .numOfRows = pInfo ? pInfo->numOfRows : 0, + }; + + clientVg.offsetInfo.walVerBegin = -1; + clientVg.offsetInfo.walVerEnd = -1; + clientVg.seekUpdated = false; + if (pInfo) { + tOffsetCopy(&clientVg.offsetInfo.endOffset, &pInfo->currentOffset); + tOffsetCopy(&clientVg.offsetInfo.committedOffset, &pInfo->commitOffset); + tOffsetCopy(&clientVg.offsetInfo.beginOffset, &pInfo->seekOffset); + } else { + clientVg.offsetInfo.endOffset = offsetNew; + clientVg.offsetInfo.committedOffset = offsetNew; + clientVg.offsetInfo.beginOffset = offsetNew; + } + if (taosArrayPush(pTopic->vgs, &clientVg) == NULL) { + tqErrorC("consumer:0x%" PRIx64 ", failed to push vg:%d into topic:%s", tmq->consumerId, pVgEp->vgId, + pTopic->topicName); + freeClientVg(&clientVg); + } + } +} + +static void buildNewTopicList(tmq_t* tmq, SArray* newTopics, const SMqAskEpRsp* pRsp){ + SHashObj* pVgOffsetHashMap = taosHashInit(64, MurmurHash3_32, false, HASH_NO_LOCK); + if (pVgOffsetHashMap == NULL) { + tqErrorC("consumer:0x%" PRIx64 " taos hash init null, code:%d", tmq->consumerId, terrno); + return; + } + + int32_t topicNumCur = taosArrayGetSize(tmq->clientTopics); + for (int32_t i = 0; i < topicNumCur; i++) { + // find old topic + SMqClientTopic* pTopicCur = taosArrayGet(tmq->clientTopics, i); + if (pTopicCur && pTopicCur->vgs) { + int32_t vgNumCur = taosArrayGetSize(pTopicCur->vgs); + tqInfoC("consumer:0x%" PRIx64 ", current vg num: %d", tmq->consumerId, vgNumCur); + for (int32_t j = 0; j < vgNumCur; j++) { + SMqClientVg* pVgCur = taosArrayGet(pTopicCur->vgs, j); + if (pVgCur == NULL) { + continue; + } + char vgKey[TSDB_TOPIC_FNAME_LEN + 22] = {0}; + (void)sprintf(vgKey, "%s:%d", pTopicCur->topicName, pVgCur->vgId); + + char buf[TSDB_OFFSET_LEN] = {0}; + tFormatOffset(buf, TSDB_OFFSET_LEN, &pVgCur->offsetInfo.endOffset); + tqInfoC("consumer:0x%" PRIx64 ", vgId:%d vgKey:%s, offset:%s", tmq->consumerId, pVgCur->vgId, vgKey, buf); + + SVgroupSaveInfo info = {.currentOffset = pVgCur->offsetInfo.endOffset, + .seekOffset = pVgCur->offsetInfo.beginOffset, + .commitOffset = pVgCur->offsetInfo.committedOffset, + .numOfRows = pVgCur->numOfRows, + .vgStatus = pVgCur->vgStatus}; + if (taosHashPut(pVgOffsetHashMap, vgKey, strlen(vgKey), &info, sizeof(SVgroupSaveInfo)) != 0) { + tqErrorC("consumer:0x%" PRIx64 ", failed to put vg:%d into hashmap", tmq->consumerId, pVgCur->vgId); + } + } + } + } + + for (int32_t i = 0; i < taosArrayGetSize(pRsp->topics); i++) { + SMqClientTopic topic = {0}; + SMqSubTopicEp* pTopicEp = taosArrayGet(pRsp->topics, i); + if (pTopicEp == NULL) { + continue; + } + initClientTopicFromRsp(&topic, pTopicEp, pVgOffsetHashMap, tmq); + if (taosArrayPush(newTopics, &topic) == NULL) { + tqErrorC("consumer:0x%" PRIx64 ", failed to push topic:%s into new topics", tmq->consumerId, topic.topicName); + freeClientTopic(&topic); + } + } + + taosHashCleanup(pVgOffsetHashMap); +} + +static void doUpdateLocalEp(tmq_t* tmq, int32_t epoch, const SMqAskEpRsp* pRsp) { + int32_t topicNumGet = taosArrayGetSize(pRsp->topics); + // vnode transform (epoch == tmq->epoch && topicNumGet != 0) + // ask ep rsp (epoch == tmq->epoch && topicNumGet == 0) + if (epoch < tmq->epoch || (epoch == tmq->epoch && topicNumGet == 0)) { + tqDebugC("consumer:0x%" PRIx64 " no update ep epoch from %d to epoch %d, incoming topics:%d", tmq->consumerId, + tmq->epoch, epoch, topicNumGet); + return; + } + + SArray* newTopics = taosArrayInit(topicNumGet, sizeof(SMqClientTopic)); + if (newTopics == NULL) { + tqErrorC("consumer:0x%" PRIx64 " taos array init null, code:%d", tmq->consumerId, terrno); + return; + } + tqInfoC("consumer:0x%" PRIx64 " update ep epoch from %d to epoch %d, incoming topics:%d, existed topics:%d", + tmq->consumerId, tmq->epoch, epoch, topicNumGet, (int)taosArrayGetSize(tmq->clientTopics)); + + taosWLockLatch(&tmq->lock); + if (topicNumGet > 0){ + buildNewTopicList(tmq, newTopics, pRsp); + } + // destroy current buffered existed topics info + if (tmq->clientTopics) { + taosArrayDestroyEx(tmq->clientTopics, freeClientTopic); + } + tmq->clientTopics = newTopics; + taosWUnLockLatch(&tmq->lock); + + atomic_store_8(&tmq->status, TMQ_CONSUMER_STATUS__READY); + atomic_store_32(&tmq->epoch, epoch); + + tqInfoC("consumer:0x%" PRIx64 " update topic info completed", tmq->consumerId); +} + +static int32_t askEpCb(void* param, SDataBuf* pMsg, int32_t code) { + SMqAskEpCbParam* pParam = (SMqAskEpCbParam*)param; + if (pParam == NULL) { + goto FAIL; + } + + tmq_t* tmq = taosAcquireRef(tmqMgmt.rsetId, pParam->refId); + if (tmq == NULL) { + code = TSDB_CODE_TMQ_CONSUMER_CLOSED; + goto FAIL; + } + + if (code != TSDB_CODE_SUCCESS) { + tqErrorC("consumer:0x%" PRIx64 ", get topic endpoint error, code:%s", tmq->consumerId, tstrerror(code)); + goto END; + } + + if (pMsg == NULL) { + goto END; + } + SMqRspHead* head = pMsg->pData; + int32_t epoch = atomic_load_32(&tmq->epoch); + tqDebugC("consumer:0x%" PRIx64 ", recv ep, msg epoch %d, current epoch %d", tmq->consumerId, head->epoch, epoch); + if (pParam->sync) { + SMqAskEpRsp rsp = {0}; + if (tDecodeSMqAskEpRsp(POINTER_SHIFT(pMsg->pData, sizeof(SMqRspHead)), &rsp) != NULL) { + doUpdateLocalEp(tmq, head->epoch, &rsp); + } + tDeleteSMqAskEpRsp(&rsp); + } else { + SMqRspWrapper* pWrapper = NULL; + code = taosAllocateQitem(sizeof(SMqRspWrapper), DEF_QITEM, 0, (void**)&pWrapper); + if (code) { + goto END; + } + + pWrapper->tmqRspType = TMQ_MSG_TYPE__EP_RSP; + pWrapper->epoch = head->epoch; + (void)memcpy(&pWrapper->epRsp, pMsg->pData, sizeof(SMqRspHead)); + if (tDecodeSMqAskEpRsp(POINTER_SHIFT(pMsg->pData, sizeof(SMqRspHead)), &pWrapper->epRsp) == NULL) { + tmqFreeRspWrapper((SMqRspWrapper*)pWrapper); + taosFreeQitem(pWrapper); + } else { + (void)taosWriteQitem(tmq->mqueue, pWrapper); + } + } + + END: + (void)taosReleaseRef(tmqMgmt.rsetId, pParam->refId); + + FAIL: + if (pParam && pParam->sync) { + SAskEpInfo* pInfo = pParam->pParam; + if (pInfo) { + pInfo->code = code; + (void)tsem2_post(&pInfo->sem); + } + } + + if (pMsg) { + taosMemoryFree(pMsg->pEpSet); + taosMemoryFree(pMsg->pData); + } + + return code; +} + +static int32_t askEp(tmq_t* pTmq, void* param, bool sync, bool updateEpSet) { + SMqAskEpReq req = {0}; + req.consumerId = pTmq->consumerId; + req.epoch = updateEpSet ? -1 : pTmq->epoch; + tstrncpy(req.cgroup, pTmq->groupId, TSDB_CGROUP_LEN); + int code = 0; + SMqAskEpCbParam* pParam = NULL; + void* pReq = NULL; + + int32_t tlen = tSerializeSMqAskEpReq(NULL, 0, &req); + if (tlen < 0) { + tqErrorC("consumer:0x%" PRIx64 ", tSerializeSMqAskEpReq failed", pTmq->consumerId); + return TSDB_CODE_INVALID_PARA; + } + + pReq = taosMemoryCalloc(1, tlen); + if (pReq == NULL) { + tqErrorC("consumer:0x%" PRIx64 ", failed to malloc askEpReq msg, size:%d", pTmq->consumerId, tlen); + return terrno; + } + + if (tSerializeSMqAskEpReq(pReq, tlen, &req) < 0) { + tqErrorC("consumer:0x%" PRIx64 ", tSerializeSMqAskEpReq %d failed", pTmq->consumerId, tlen); + taosMemoryFree(pReq); + return TSDB_CODE_INVALID_PARA; + } + + pParam = taosMemoryCalloc(1, sizeof(SMqAskEpCbParam)); + if (pParam == NULL) { + tqErrorC("consumer:0x%" PRIx64 ", failed to malloc subscribe param", pTmq->consumerId); + taosMemoryFree(pReq); + return terrno; + } + + pParam->refId = pTmq->refId; + pParam->sync = sync; + pParam->pParam = param; + + SMsgSendInfo* sendInfo = taosMemoryCalloc(1, sizeof(SMsgSendInfo)); + if (sendInfo == NULL) { + taosMemoryFree(pReq); + taosMemoryFree(pParam); + return terrno; + } + + sendInfo->msgInfo = (SDataBuf){.pData = pReq, .len = tlen, .handle = NULL}; + sendInfo->requestId = generateRequestId(); + sendInfo->requestObjRefId = 0; + sendInfo->param = pParam; + sendInfo->paramFreeFp = taosMemoryFree; + sendInfo->fp = askEpCb; + sendInfo->msgType = TDMT_MND_TMQ_ASK_EP; + + SEpSet epSet = getEpSet_s(&pTmq->pTscObj->pAppInfo->mgmtEp); + tqDebugC("consumer:0x%" PRIx64 " ask ep from mnode,QID:0x%" PRIx64, pTmq->consumerId, sendInfo->requestId); + return asyncSendMsgToServer(pTmq->pTscObj->pAppInfo->pTransporter, &epSet, NULL, sendInfo); +} + void tmqHandleAllDelayedTask(tmq_t* pTmq) { STaosQall* qall = NULL; int32_t code = 0; @@ -1020,30 +1376,6 @@ void tmqHandleAllDelayedTask(tmq_t* pTmq) { taosFreeQall(qall); } -static void tmqFreeRspWrapper(SMqRspWrapper* rspWrapper) { - if (rspWrapper->tmqRspType == TMQ_MSG_TYPE__EP_RSP) { - SMqAskEpRspWrapper* pEpRspWrapper = (SMqAskEpRspWrapper*)rspWrapper; - tDeleteSMqAskEpRsp(&pEpRspWrapper->msg); - } else if (rspWrapper->tmqRspType == TMQ_MSG_TYPE__POLL_DATA_RSP) { - SMqPollRspWrapper* pRsp = (SMqPollRspWrapper*)rspWrapper; - taosMemoryFreeClear(pRsp->pEpset); - - tDeleteMqDataRsp(&pRsp->dataRsp); - } else if (rspWrapper->tmqRspType == TMQ_MSG_TYPE__POLL_META_RSP) { - SMqPollRspWrapper* pRsp = (SMqPollRspWrapper*)rspWrapper; - taosMemoryFreeClear(pRsp->pEpset); - tDeleteMqMetaRsp(&pRsp->metaRsp); - } else if (rspWrapper->tmqRspType == TMQ_MSG_TYPE__POLL_DATA_META_RSP) { - SMqPollRspWrapper* pRsp = (SMqPollRspWrapper*)rspWrapper; - taosMemoryFreeClear(pRsp->pEpset); - tDeleteSTaosxRsp(&pRsp->taosxRsp); - } else if (rspWrapper->tmqRspType == TMQ_MSG_TYPE__POLL_BATCH_META_RSP) { - SMqPollRspWrapper* pRsp = (SMqPollRspWrapper*)rspWrapper; - taosMemoryFreeClear(pRsp->pEpset); - tDeleteMqBatchMetaRsp(&pRsp->batchMetaRsp); - } -} - void tmqClearUnhandleMsg(tmq_t* tmq) { SMqRspWrapper* rspWrapper = NULL; while (1) { @@ -1114,18 +1446,6 @@ int32_t tmq_subscription(tmq_t* tmq, tmq_list_t** topics) { return 0; } -static void freeClientVg(void* param) { - SMqClientVg* pVg = param; - tOffsetDestroy(&pVg->offsetInfo.endOffset); - tOffsetDestroy(&pVg->offsetInfo.beginOffset); - tOffsetDestroy(&pVg->offsetInfo.committedOffset); -} -static void freeClientTopic(void* param) { - SMqClientTopic* pTopic = param; - taosMemoryFreeClear(pTopic->schema.pSchema); - taosArrayDestroyEx(pTopic->vgs, freeClientVg); -} - void tmqFreeImpl(void* handle) { tmq_t* tmq = (tmq_t*)handle; int64_t id = tmq->consumerId; @@ -1185,9 +1505,6 @@ void tmqMgmtClose(void) { } } -#define SET_ERROR_MSG_TMQ(MSG) \ - if (errstr != NULL) (void)snprintf(errstr, errstrLen, MSG); - tmq_t* tmq_consumer_new(tmq_conf_t* conf, char* errstr, int32_t errstrLen) { int32_t code = 0; @@ -1328,6 +1645,25 @@ _failed: return NULL; } +static int32_t syncAskEp(tmq_t* pTmq) { + SAskEpInfo* pInfo = taosMemoryMalloc(sizeof(SAskEpInfo)); + if (pInfo == NULL) return TSDB_CODE_OUT_OF_MEMORY; + if (tsem2_init(&pInfo->sem, 0, 0) != 0) { + taosMemoryFree(pInfo); + return TSDB_CODE_TSC_INTERNAL_ERROR; + } + + int32_t code = askEp(pTmq, pInfo, true, false); + if (code == 0) { + (void)tsem2_wait(&pInfo->sem); + code = pInfo->code; + } + + (void)tsem2_destroy(&pInfo->sem); + taosMemoryFree(pInfo); + return code; +} + int32_t tmq_subscribe(tmq_t* tmq, const tmq_list_t* topic_list) { if (tmq == NULL || topic_list == NULL) return TSDB_CODE_INVALID_PARA; const SArray* container = &topic_list->container; @@ -1507,19 +1843,9 @@ static SMqClientTopic* getTopicInfo(tmq_t* tmq, char* topicName) { return NULL; } -static void setVgIdle(tmq_t* tmq, char* topicName, int32_t vgId) { - taosWLockLatch(&tmq->lock); - SMqClientVg* pVg = NULL; - getVgInfo(tmq, topicName, vgId, &pVg); - if (pVg) { - atomic_store_32(&pVg->vgStatus, TMQ_VG_STATUS__IDLE); - } - taosWUnLockLatch(&tmq->lock); -} - int32_t tmqPollCb(void* param, SDataBuf* pMsg, int32_t code) { tmq_t* tmq = NULL; - SMqPollRspWrapper* pRspWrapper = NULL; + SMqRspWrapper* pRspWrapper = NULL; int8_t rspType = 0; int32_t vgId = 0; uint64_t requestId = 0; @@ -1528,21 +1854,19 @@ int32_t tmqPollCb(void* param, SDataBuf* pMsg, int32_t code) { return TSDB_CODE_TSC_INTERNAL_ERROR; } if (pParam == NULL) { - taosMemoryFreeClear(pMsg->pData); - taosMemoryFreeClear(pMsg->pEpSet); - return TSDB_CODE_TSC_INTERNAL_ERROR; + code = TSDB_CODE_TSC_INTERNAL_ERROR; + goto EXIT; } int64_t refId = pParam->refId; vgId = pParam->vgId; requestId = pParam->requestId; tmq = taosAcquireRef(tmqMgmt.rsetId, refId); if (tmq == NULL) { - taosMemoryFreeClear(pMsg->pData); - taosMemoryFreeClear(pMsg->pEpSet); - return TSDB_CODE_TMQ_CONSUMER_CLOSED; + code = TSDB_CODE_TMQ_CONSUMER_CLOSED; + goto EXIT; } - int32_t ret = taosAllocateQitem(sizeof(SMqPollRspWrapper), DEF_QITEM, 0, (void**)&pRspWrapper); + int32_t ret = taosAllocateQitem(sizeof(SMqRspWrapper), DEF_QITEM, 0, (void**)&pRspWrapper); if (ret) { code = ret; tscWarn("consumer:0x%" PRIx64 " msg discard from vgId:%d, since out of memory", tmq->consumerId, vgId); @@ -1561,263 +1885,58 @@ int32_t tmqPollCb(void* param, SDataBuf* pMsg, int32_t code) { int32_t msgEpoch = ((SMqRspHead*)pMsg->pData)->epoch; int32_t clientEpoch = atomic_load_32(&tmq->epoch); - if (msgEpoch < clientEpoch) { - // do not write into queue since updating epoch reset - tscWarn("consumer:0x%" PRIx64 - " msg discard from vgId:%d since from earlier epoch, rsp epoch %d, current epoch %d,QID:0x%" PRIx64, - tmq->consumerId, vgId, msgEpoch, clientEpoch, requestId); - code = TSDB_CODE_TMQ_CONSUMER_MISMATCH; - goto END; - } if (msgEpoch != clientEpoch) { tqErrorC("consumer:0x%" PRIx64 - " msg discard from vgId:%d since from earlier epoch, rsp epoch %d, current epoch %d, reqId:0x%" PRIx64, + " msg discard from vgId:%d since epoch not equal, rsp epoch %d, current epoch %d, reqId:0x%" PRIx64, tmq->consumerId, vgId, msgEpoch, clientEpoch, requestId); code = TSDB_CODE_TMQ_CONSUMER_MISMATCH; goto END; } - // handle meta rsp rspType = ((SMqRspHead*)pMsg->pData)->mqMsgType; - pRspWrapper->tmqRspType = rspType; - pRspWrapper->reqId = requestId; - pRspWrapper->pEpset = pMsg->pEpSet; - pMsg->pEpSet = NULL; - + tqDebugC("consumer:0x%" PRIx64 " recv poll rsp, vgId:%d, type %d,QID:0x%" PRIx64, tmq->consumerId, vgId, rspType, requestId); if (rspType == TMQ_MSG_TYPE__POLL_DATA_RSP) { - SDecoder decoder = {0}; - tDecoderInit(&decoder, POINTER_SHIFT(pMsg->pData, sizeof(SMqRspHead)), pMsg->len - sizeof(SMqRspHead)); - if (tDecodeMqDataRsp(&decoder, &pRspWrapper->dataRsp) < 0) { - tDecoderClear(&decoder); - code = TSDB_CODE_OUT_OF_MEMORY; - goto END; - } - tDecoderClear(&decoder); - (void)memcpy(&pRspWrapper->dataRsp, pMsg->pData, sizeof(SMqRspHead)); - - char buf[TSDB_OFFSET_LEN] = {0}; - tFormatOffset(buf, TSDB_OFFSET_LEN, &pRspWrapper->dataRsp.common.rspOffset); - tqDebugC("consumer:0x%" PRIx64 " recv poll rsp, vgId:%d, req ver:%" PRId64 ", rsp:%s type %d,QID:0x%" PRIx64, - tmq->consumerId, vgId, pRspWrapper->dataRsp.common.reqOffset.version, buf, rspType, requestId); + PROCESS_POLL_RSP(tDecodeMqDataRsp, &pRspWrapper->pollRsp.dataRsp) } else if (rspType == TMQ_MSG_TYPE__POLL_META_RSP) { - SDecoder decoder = {0}; - tDecoderInit(&decoder, POINTER_SHIFT(pMsg->pData, sizeof(SMqRspHead)), pMsg->len - sizeof(SMqRspHead)); - if (tDecodeMqMetaRsp(&decoder, &pRspWrapper->metaRsp) < 0) { - tDecoderClear(&decoder); - code = TSDB_CODE_OUT_OF_MEMORY; - goto END; - } - tDecoderClear(&decoder); - (void)memcpy(&pRspWrapper->metaRsp, pMsg->pData, sizeof(SMqRspHead)); + PROCESS_POLL_RSP(tDecodeMqMetaRsp, &pRspWrapper->pollRsp.metaRsp) } else if (rspType == TMQ_MSG_TYPE__POLL_DATA_META_RSP) { - SDecoder decoder = {0}; - tDecoderInit(&decoder, POINTER_SHIFT(pMsg->pData, sizeof(SMqRspHead)), pMsg->len - sizeof(SMqRspHead)); - if (tDecodeSTaosxRsp(&decoder, &pRspWrapper->taosxRsp) < 0) { - tDecoderClear(&decoder); - code = TSDB_CODE_OUT_OF_MEMORY; - goto END; - } - tDecoderClear(&decoder); - (void)memcpy(&pRspWrapper->taosxRsp, pMsg->pData, sizeof(SMqRspHead)); + PROCESS_POLL_RSP(tDecodeSTaosxRsp, &pRspWrapper->pollRsp.dataRsp) } else if (rspType == TMQ_MSG_TYPE__POLL_BATCH_META_RSP) { - SDecoder decoder = {0}; - tDecoderInit(&decoder, POINTER_SHIFT(pMsg->pData, sizeof(SMqRspHead)), pMsg->len - sizeof(SMqRspHead)); - if (tSemiDecodeMqBatchMetaRsp(&decoder, &pRspWrapper->batchMetaRsp) < 0) { - tDecoderClear(&decoder); - code = TSDB_CODE_OUT_OF_MEMORY; - goto END; - } - tDecoderClear(&decoder); - (void)memcpy(&pRspWrapper->batchMetaRsp, pMsg->pData, sizeof(SMqRspHead)); - tqDebugC("consumer:0x%" PRIx64 " recv poll batchmeta rsp, vgId:%d,QID:0x%" PRIx64, tmq->consumerId, vgId, - requestId); + PROCESS_POLL_RSP(tSemiDecodeMqBatchMetaRsp, &pRspWrapper->pollRsp.batchMetaRsp) } else { // invalid rspType tqErrorC("consumer:0x%" PRIx64 " invalid rsp msg received, type:%d ignored", tmq->consumerId, rspType); + code = TSDB_CODE_TSC_INTERNAL_ERROR; + goto END; } + pRspWrapper->tmqRspType = rspType; + pRspWrapper->pollRsp.reqId = requestId; + pRspWrapper->pollRsp.pEpset = pMsg->pEpSet; + pMsg->pEpSet = NULL; END: if (pRspWrapper) { pRspWrapper->code = code; - pRspWrapper->vgId = vgId; - (void)strcpy(pRspWrapper->topicName, pParam->topicName); + pRspWrapper->pollRsp.vgId = vgId; + (void)strcpy(pRspWrapper->pollRsp.topicName, pParam->topicName); code = taosWriteQitem(tmq->mqueue, pRspWrapper); if (code != 0) { tqErrorC("consumer:0x%" PRIx64 " put poll res into mqueue failed, code:%d", tmq->consumerId, code); + } else { + tqDebugC("consumer:0x%" PRIx64 " put poll res into mqueue, type:%d, vgId:%d, total in queue:%d,QID:0x%" PRIx64, + tmq ? tmq->consumerId : 0, rspType, vgId, taosQueueItemSize(tmq->mqueue), requestId); } } - int32_t total = taosQueueItemSize(tmq->mqueue); - tqDebugC("consumer:0x%" PRIx64 " put poll res into mqueue, type:%d, vgId:%d, total in queue:%d,QID:0x%" PRIx64, - tmq ? tmq->consumerId : 0, rspType, vgId, total, requestId); - if (tmq) (void)tsem2_post(&tmq->rspSem); - if (pMsg) taosMemoryFreeClear(pMsg->pData); - if (pMsg) taosMemoryFreeClear(pMsg->pEpSet); + + (void)tsem2_post(&tmq->rspSem); (void)taosReleaseRef(tmqMgmt.rsetId, refId); +EXIT: + taosMemoryFreeClear(pMsg->pData); + taosMemoryFreeClear(pMsg->pEpSet); return code; } -typedef struct SVgroupSaveInfo { - STqOffsetVal currentOffset; - STqOffsetVal commitOffset; - STqOffsetVal seekOffset; - int64_t numOfRows; - int32_t vgStatus; -} SVgroupSaveInfo; - -static void initClientTopicFromRsp(SMqClientTopic* pTopic, SMqSubTopicEp* pTopicEp, SHashObj* pVgOffsetHashMap, - tmq_t* tmq) { - pTopic->schema = pTopicEp->schema; - pTopicEp->schema.nCols = 0; - pTopicEp->schema.pSchema = NULL; - - char vgKey[TSDB_TOPIC_FNAME_LEN + 22] = {0}; - int32_t vgNumGet = taosArrayGetSize(pTopicEp->vgs); - - tstrncpy(pTopic->topicName, pTopicEp->topic, TSDB_TOPIC_FNAME_LEN); - tstrncpy(pTopic->db, pTopicEp->db, TSDB_DB_FNAME_LEN); - - tqInfoC("consumer:0x%" PRIx64 ", update topic:%s, new numOfVgs:%d", tmq->consumerId, pTopic->topicName, vgNumGet); - pTopic->vgs = taosArrayInit(vgNumGet, sizeof(SMqClientVg)); - if (pTopic->vgs == NULL) { - tqErrorC("consumer:0x%" PRIx64 ", failed to init vgs for topic:%s", tmq->consumerId, pTopic->topicName); - return; - } - for (int32_t j = 0; j < vgNumGet; j++) { - SMqSubVgEp* pVgEp = taosArrayGet(pTopicEp->vgs, j); - if (pVgEp == NULL) { - continue; - } - (void)sprintf(vgKey, "%s:%d", pTopic->topicName, pVgEp->vgId); - SVgroupSaveInfo* pInfo = taosHashGet(pVgOffsetHashMap, vgKey, strlen(vgKey)); - - STqOffsetVal offsetNew = {0}; - offsetNew.type = tmq->resetOffsetCfg; - - tqInfoC("consumer:0x%" PRIx64 ", update topic:%s, new numOfVgs:%d, num:%d, port:%d", tmq->consumerId, - pTopic->topicName, vgNumGet, pVgEp->epSet.numOfEps, pVgEp->epSet.eps[pVgEp->epSet.inUse].port); - - SMqClientVg clientVg = { - .pollCnt = 0, - .vgId = pVgEp->vgId, - .epSet = pVgEp->epSet, - .vgStatus = pInfo ? pInfo->vgStatus : TMQ_VG_STATUS__IDLE, - .vgSkipCnt = 0, - .emptyBlockReceiveTs = 0, - .blockReceiveTs = 0, - .blockSleepForReplay = 0, - .numOfRows = pInfo ? pInfo->numOfRows : 0, - }; - - clientVg.offsetInfo.walVerBegin = -1; - clientVg.offsetInfo.walVerEnd = -1; - clientVg.seekUpdated = false; - if (pInfo) { - tOffsetCopy(&clientVg.offsetInfo.endOffset, &pInfo->currentOffset); - tOffsetCopy(&clientVg.offsetInfo.committedOffset, &pInfo->commitOffset); - tOffsetCopy(&clientVg.offsetInfo.beginOffset, &pInfo->seekOffset); - } else { - clientVg.offsetInfo.endOffset = offsetNew; - clientVg.offsetInfo.committedOffset = offsetNew; - clientVg.offsetInfo.beginOffset = offsetNew; - } - if (taosArrayPush(pTopic->vgs, &clientVg) == NULL) { - tqErrorC("consumer:0x%" PRIx64 ", failed to push vg:%d into topic:%s", tmq->consumerId, pVgEp->vgId, - pTopic->topicName); - freeClientVg(&clientVg); - } - } -} - -static void buildNewTopicList(tmq_t* tmq, SArray* newTopics, const SMqAskEpRsp* pRsp){ - SHashObj* pVgOffsetHashMap = taosHashInit(64, MurmurHash3_32, false, HASH_NO_LOCK); - if (pVgOffsetHashMap == NULL) { - tqErrorC("consumer:0x%" PRIx64 " taos hash init null, code:%d", tmq->consumerId, terrno); - return; - } - - int32_t topicNumCur = taosArrayGetSize(tmq->clientTopics); - for (int32_t i = 0; i < topicNumCur; i++) { - // find old topic - SMqClientTopic* pTopicCur = taosArrayGet(tmq->clientTopics, i); - if (pTopicCur && pTopicCur->vgs) { - int32_t vgNumCur = taosArrayGetSize(pTopicCur->vgs); - tqInfoC("consumer:0x%" PRIx64 ", current vg num: %d", tmq->consumerId, vgNumCur); - for (int32_t j = 0; j < vgNumCur; j++) { - SMqClientVg* pVgCur = taosArrayGet(pTopicCur->vgs, j); - if (pVgCur == NULL) { - continue; - } - char vgKey[TSDB_TOPIC_FNAME_LEN + 22] = {0}; - (void)sprintf(vgKey, "%s:%d", pTopicCur->topicName, pVgCur->vgId); - - char buf[TSDB_OFFSET_LEN] = {0}; - tFormatOffset(buf, TSDB_OFFSET_LEN, &pVgCur->offsetInfo.endOffset); - tqInfoC("consumer:0x%" PRIx64 ", vgId:%d vgKey:%s, offset:%s", tmq->consumerId, pVgCur->vgId, vgKey, buf); - - SVgroupSaveInfo info = {.currentOffset = pVgCur->offsetInfo.endOffset, - .seekOffset = pVgCur->offsetInfo.beginOffset, - .commitOffset = pVgCur->offsetInfo.committedOffset, - .numOfRows = pVgCur->numOfRows, - .vgStatus = pVgCur->vgStatus}; - if (taosHashPut(pVgOffsetHashMap, vgKey, strlen(vgKey), &info, sizeof(SVgroupSaveInfo)) != 0) { - tqErrorC("consumer:0x%" PRIx64 ", failed to put vg:%d into hashmap", tmq->consumerId, pVgCur->vgId); - } - } - } - } - - for (int32_t i = 0; i < taosArrayGetSize(pRsp->topics); i++) { - SMqClientTopic topic = {0}; - SMqSubTopicEp* pTopicEp = taosArrayGet(pRsp->topics, i); - if (pTopicEp == NULL) { - continue; - } - initClientTopicFromRsp(&topic, pTopicEp, pVgOffsetHashMap, tmq); - if (taosArrayPush(newTopics, &topic) == NULL) { - tqErrorC("consumer:0x%" PRIx64 ", failed to push topic:%s into new topics", tmq->consumerId, topic.topicName); - freeClientTopic(&topic); - } - } - - taosHashCleanup(pVgOffsetHashMap); -} - -static void doUpdateLocalEp(tmq_t* tmq, int32_t epoch, const SMqAskEpRsp* pRsp) { - int32_t topicNumGet = taosArrayGetSize(pRsp->topics); - // vnode transform (epoch == tmq->epoch && topicNumGet != 0) - // ask ep rsp (epoch == tmq->epoch && topicNumGet == 0) - if (epoch < tmq->epoch || (epoch == tmq->epoch && topicNumGet == 0)) { - tqDebugC("consumer:0x%" PRIx64 " no update ep epoch from %d to epoch %d, incoming topics:%d", tmq->consumerId, - tmq->epoch, epoch, topicNumGet); - return; - } - - SArray* newTopics = taosArrayInit(topicNumGet, sizeof(SMqClientTopic)); - if (newTopics == NULL) { - tqErrorC("consumer:0x%" PRIx64 " taos array init null, code:%d", tmq->consumerId, terrno); - return; - } - tqInfoC("consumer:0x%" PRIx64 " update ep epoch from %d to epoch %d, incoming topics:%d, existed topics:%d", - tmq->consumerId, tmq->epoch, epoch, topicNumGet, (int)taosArrayGetSize(tmq->clientTopics)); - - taosWLockLatch(&tmq->lock); - if (topicNumGet > 0){ - buildNewTopicList(tmq, newTopics, pRsp); - } - // destroy current buffered existed topics info - if (tmq->clientTopics) { - taosArrayDestroyEx(tmq->clientTopics, freeClientTopic); - } - tmq->clientTopics = newTopics; - taosWUnLockLatch(&tmq->lock); - - atomic_store_8(&tmq->status, TMQ_CONSUMER_STATUS__READY); - atomic_store_32(&tmq->epoch, epoch); - - tqInfoC("consumer:0x%" PRIx64 " update topic info completed", tmq->consumerId); -} - void tmqBuildConsumeReqImpl(SMqPollReq* pReq, tmq_t* tmq, int64_t timeout, SMqClientTopic* pTopic, SMqClientVg* pVg) { (void)snprintf(pReq->subKey, TSDB_SUBSCRIBE_KEY_LEN, "%s%s%s", tmq->groupId, TMQ_SEPARATOR, pTopic->topicName); pReq->withTbName = tmq->withTbName; @@ -1833,37 +1952,6 @@ void tmqBuildConsumeReqImpl(SMqPollReq* pReq, tmq_t* tmq, int64_t timeout, SMqCl pReq->enableBatchMeta = tmq->enableBatchMeta; } -int32_t tmqBuildMetaRspFromWrapper(SMqPollRspWrapper* pWrapper, SMqMetaRspObj** ppRspObj) { - SMqMetaRspObj* pRspObj = taosMemoryCalloc(1, sizeof(SMqMetaRspObj)); - if (pRspObj == NULL) { - return terrno; - } - pRspObj->resType = RES_TYPE__TMQ_META; - tstrncpy(pRspObj->topic, pWrapper->topicHandle->topicName, TSDB_TOPIC_FNAME_LEN); - tstrncpy(pRspObj->db, pWrapper->topicHandle->db, TSDB_DB_FNAME_LEN); - pRspObj->vgId = pWrapper->vgHandle->vgId; - - (void)memcpy(&pRspObj->metaRsp, &pWrapper->metaRsp, sizeof(SMqMetaRsp)); - *ppRspObj = pRspObj; - return 0; -} - -int32_t tmqBuildBatchMetaRspFromWrapper(SMqPollRspWrapper* pWrapper, SMqBatchMetaRspObj** ppRspObj) { - SMqBatchMetaRspObj* pRspObj = taosMemoryCalloc(1, sizeof(SMqBatchMetaRspObj)); - if (pRspObj == NULL) { - return terrno; - } - pRspObj->common.resType = RES_TYPE__TMQ_BATCH_META; - tstrncpy(pRspObj->common.topic, pWrapper->topicHandle->topicName, TSDB_TOPIC_FNAME_LEN); - tstrncpy(pRspObj->common.db, pWrapper->topicHandle->db, TSDB_DB_FNAME_LEN); - pRspObj->common.vgId = pWrapper->vgHandle->vgId; - - (void)memcpy(&pRspObj->rsp, &pWrapper->batchMetaRsp, sizeof(SMqBatchMetaRsp)); - tqDebugC("build batchmeta Rsp from wrapper"); - *ppRspObj = pRspObj; - return 0; -} - void changeByteEndian(char* pData) { if (pData == NULL) { return; @@ -1915,14 +2003,8 @@ static void tmqGetRawDataRowsPrecisionFromRes(void* pRetrieve, void** rawData, i } static void tmqBuildRspFromWrapperInner(SMqPollRspWrapper* pWrapper, SMqClientVg* pVg, int64_t* numOfRows, - SMqRspObjCommon* pRspObj, SMqDataRspCommon* pDataRsp) { - (*numOfRows) = 0; - tstrncpy(pRspObj->topic, pWrapper->topicHandle->topicName, TSDB_TOPIC_FNAME_LEN); - tstrncpy(pRspObj->db, pWrapper->topicHandle->db, TSDB_DB_FNAME_LEN); - - pRspObj->vgId = pWrapper->vgHandle->vgId; + SMqRspObj* pRspObj, SMqDataRsp* pDataRsp) { pRspObj->resIter = -1; - pRspObj->resInfo.totalRows = 0; pRspObj->resInfo.precision = TSDB_TIME_PRECISION_MILLI; @@ -1958,33 +2040,6 @@ static void tmqBuildRspFromWrapperInner(SMqPollRspWrapper* pWrapper, SMqClientVg } } -int32_t tmqBuildRspFromWrapper(SMqPollRspWrapper* pWrapper, SMqClientVg* pVg, int64_t* numOfRows, - SMqRspObj** ppRspObj) { - SMqRspObj* pRspObj = taosMemoryCalloc(1, sizeof(SMqRspObj)); - if (pRspObj == NULL) { - return terrno; - } - pRspObj->common.resType = RES_TYPE__TMQ; - (void)memcpy(&pRspObj->rsp, &pWrapper->dataRsp, sizeof(SMqDataRsp)); - tmqBuildRspFromWrapperInner(pWrapper, pVg, numOfRows, &pRspObj->common, &pRspObj->rsp.common); - *ppRspObj = pRspObj; - return 0; -} - -int32_t tmqBuildTaosxRspFromWrapper(SMqPollRspWrapper* pWrapper, SMqClientVg* pVg, int64_t* numOfRows, - SMqTaosxRspObj** ppRspObj) { - SMqTaosxRspObj* pRspObj = taosMemoryCalloc(1, sizeof(SMqTaosxRspObj)); - if (pRspObj == NULL) { - return terrno; - } - pRspObj->common.resType = RES_TYPE__TMQ_METADATA; - (void)memcpy(&pRspObj->rsp, &pWrapper->taosxRsp, sizeof(STaosxRsp)); - - tmqBuildRspFromWrapperInner(pWrapper, pVg, numOfRows, &pRspObj->common, &pRspObj->rsp.common); - *ppRspObj = pRspObj; - return 0; -} - static int32_t doTmqPollImpl(tmq_t* pTmq, SMqClientTopic* pTopic, SMqClientVg* pVg, int64_t timeout) { SMqPollReq req = {0}; char* msg = NULL; @@ -2132,11 +2187,135 @@ static void updateVgInfo(SMqClientVg* pVg, STqOffsetVal* reqOffset, STqOffsetVal pVg->offsetInfo.walVerEnd = ever + 1; } +static SMqRspObj* buildRsp(SMqPollRspWrapper* pollRspWrapper){ + typedef union { + SMqDataRsp dataRsp; + SMqMetaRsp metaRsp; + SMqBatchMetaRsp batchMetaRsp; + } MEMSIZE; + + SMqRspObj* pRspObj = taosMemoryCalloc(1, sizeof(SMqRspObj)); + if (pRspObj == NULL) { + tqErrorC("buildRsp:failed to allocate memory"); + return NULL; + } + (void)memcpy(&pRspObj->dataRsp, &pollRspWrapper->dataRsp, sizeof(MEMSIZE)); + tstrncpy(pRspObj->topic, pollRspWrapper->topicName, TSDB_TOPIC_FNAME_LEN); + tstrncpy(pRspObj->db, pollRspWrapper->topicHandle->db, TSDB_DB_FNAME_LEN); + pRspObj->vgId = pollRspWrapper->vgId; + (void)memset(&pollRspWrapper->dataRsp, 0, sizeof(MEMSIZE)); + return pRspObj; +} + +static void processMqRspError(tmq_t* tmq, SMqRspWrapper* pRspWrapper){ + SMqPollRspWrapper* pollRspWrapper = &pRspWrapper->pollRsp; + + if (pRspWrapper->code == TSDB_CODE_VND_INVALID_VGROUP_ID) { // for vnode transform + int32_t code = askEp(tmq, NULL, false, true); + if (code != 0) { + tqErrorC("consumer:0x%" PRIx64 " failed to ask ep, code:%s", tmq->consumerId, tstrerror(code)); + } + } else if (pRspWrapper->code == TSDB_CODE_TMQ_CONSUMER_MISMATCH) { + int32_t code = askEp(tmq, NULL, false, false); + if (code != 0) { + tqErrorC("consumer:0x%" PRIx64 " failed to ask ep, code:%s", tmq->consumerId, tstrerror(code)); + } + } + tqInfoC("consumer:0x%" PRIx64 " msg from vgId:%d discarded, since %s", tmq->consumerId, pollRspWrapper->vgId, + tstrerror(pRspWrapper->code)); + taosWLockLatch(&tmq->lock); + SMqClientVg* pVg = NULL; + getVgInfo(tmq, pollRspWrapper->topicName, pollRspWrapper->vgId, &pVg); + if (pVg) { + pVg->emptyBlockReceiveTs = taosGetTimestampMs(); + atomic_store_32(&pVg->vgStatus, TMQ_VG_STATUS__IDLE); + } + taosWUnLockLatch(&tmq->lock); +} +static SMqRspObj* processMqRsp(tmq_t* tmq, SMqRspWrapper* pRspWrapper){ + SMqRspObj* pRspObj = NULL; + + if (pRspWrapper->tmqRspType == TMQ_MSG_TYPE__EP_RSP) { + tqDebugC("consumer:0x%" PRIx64 " ep msg received", tmq->consumerId); + SMqAskEpRsp* rspMsg = &pRspWrapper->epRsp; + doUpdateLocalEp(tmq, pRspWrapper->epoch, rspMsg); + return pRspObj; + } + + SMqPollRspWrapper* pollRspWrapper = &pRspWrapper->pollRsp; + taosWLockLatch(&tmq->lock); + SMqClientVg* pVg = NULL; + getVgInfo(tmq, pollRspWrapper->topicName, pollRspWrapper->vgId, &pVg); + if(pVg == NULL) { + tqErrorC("consumer:0x%" PRIx64 " get vg or topic error, topic:%s vgId:%d", tmq->consumerId, + pollRspWrapper->topicName, pollRspWrapper->vgId); + goto END; + } + pollRspWrapper->topicHandle = getTopicInfo(tmq, pollRspWrapper->topicName); + if (pollRspWrapper->pEpset != NULL) { + pVg->epSet = *pollRspWrapper->pEpset; + } + + if (pRspWrapper->tmqRspType == TMQ_MSG_TYPE__POLL_DATA_RSP || pRspWrapper->tmqRspType == TMQ_MSG_TYPE__POLL_DATA_META_RSP) { + updateVgInfo(pVg, &pollRspWrapper->dataRsp.reqOffset, &pollRspWrapper->dataRsp.rspOffset, pollRspWrapper->head.walsver, pollRspWrapper->head.walever, + tmq->consumerId, pollRspWrapper->dataRsp.blockNum != 0); + + char buf[TSDB_OFFSET_LEN] = {0}; + tFormatOffset(buf, TSDB_OFFSET_LEN, &pollRspWrapper->rspOffset); + if (pollRspWrapper->dataRsp.blockNum == 0) { + tqDebugC("consumer:0x%" PRIx64 " empty block received, vgId:%d, offset:%s, vg total:%" PRId64 + ", total:%" PRId64 ",QID:0x%" PRIx64, + tmq->consumerId, pVg->vgId, buf, pVg->numOfRows, tmq->totalRows, pollRspWrapper->reqId); + pVg->emptyBlockReceiveTs = taosGetTimestampMs(); + } else { + pRspObj = buildRsp(pollRspWrapper); + if (pRspObj == NULL) { + tqErrorC("consumer:0x%" PRIx64 " failed to allocate memory for meta rsp", tmq->consumerId); + goto END; + } + pRspObj->resType = pRspWrapper->tmqRspType == TMQ_MSG_TYPE__POLL_DATA_RSP ? RES_TYPE__TMQ : RES_TYPE__TMQ_METADATA; + int64_t numOfRows = 0; + tmqBuildRspFromWrapperInner(pollRspWrapper, pVg, &numOfRows, pRspObj, &pRspObj->dataRsp); + tmq->totalRows += numOfRows; + pVg->emptyBlockReceiveTs = 0; + if (tmq->replayEnable) { + pVg->blockReceiveTs = taosGetTimestampMs(); + pVg->blockSleepForReplay = pollRspWrapper->dataRsp.sleepTime; + if (pVg->blockSleepForReplay > 0) { + if (taosTmrStart(tmqReplayTask, pVg->blockSleepForReplay, (void*)(tmq->refId), tmqMgmt.timer) == NULL) { + tqErrorC("consumer:0x%" PRIx64 " failed to start replay timer, vgId:%d, sleep:%" PRId64, + tmq->consumerId, pVg->vgId, pVg->blockSleepForReplay); + } + } + } + tqDebugC("consumer:0x%" PRIx64 " process poll rsp, vgId:%d, offset:%s, blocks:%d, rows:%" PRId64 + ", vg total:%" PRId64 ", total:%" PRId64 ",QID:0x%" PRIx64, + tmq->consumerId, pVg->vgId, buf, pollRspWrapper->dataRsp.blockNum, numOfRows, pVg->numOfRows, tmq->totalRows, + pollRspWrapper->reqId); + } + } else if (pRspWrapper->tmqRspType == TMQ_MSG_TYPE__POLL_META_RSP || pRspWrapper->tmqRspType == TMQ_MSG_TYPE__POLL_BATCH_META_RSP) { + updateVgInfo(pVg, &pollRspWrapper->rspOffset, &pollRspWrapper->rspOffset, + pollRspWrapper->head.walsver, pollRspWrapper->head.walever, tmq->consumerId, true); + + + pRspObj = buildRsp(pollRspWrapper); + if (pRspObj == NULL) { + tqErrorC("consumer:0x%" PRIx64 " failed to allocate memory for meta rsp", tmq->consumerId); + goto END; + } + pRspObj->resType = pRspWrapper->tmqRspType == TMQ_MSG_TYPE__POLL_META_RSP ? RES_TYPE__TMQ_META : RES_TYPE__TMQ_BATCH_META; + } + + END: + taosWUnLockLatch(&tmq->lock); + return pRspObj; +} static void* tmqHandleAllRsp(tmq_t* tmq, int64_t timeout) { tqDebugC("consumer:0x%" PRIx64 " start to handle the rsp, total:%d", tmq->consumerId, taosQallItemSize(tmq->qall)); + SMqRspWrapper* pRspWrapper = NULL; + void* returnVal = NULL; while (1) { - SMqRspWrapper* pRspWrapper = NULL; (void)taosGetQitem(tmq->qall, (void**)&pRspWrapper); if (pRspWrapper == NULL) { @@ -2148,254 +2327,20 @@ static void* tmqHandleAllRsp(tmq_t* tmq, int64_t timeout) { } tqDebugC("consumer:0x%" PRIx64 " handle rsp, type:%d", tmq->consumerId, pRspWrapper->tmqRspType); - if (pRspWrapper->code != 0) { - SMqPollRspWrapper* pollRspWrapper = (SMqPollRspWrapper*)pRspWrapper; + processMqRspError(tmq, pRspWrapper); + }else{ + returnVal = processMqRsp(tmq, pRspWrapper); + } - if (pRspWrapper->code == TSDB_CODE_VND_INVALID_VGROUP_ID) { // for vnode transform - int32_t code = askEp(tmq, NULL, false, true); - if (code != 0) { - tqErrorC("consumer:0x%" PRIx64 " failed to ask ep, code:%s", tmq->consumerId, tstrerror(code)); - } - } else if (pRspWrapper->code == TSDB_CODE_TMQ_CONSUMER_MISMATCH) { - int32_t code = askEp(tmq, NULL, false, false); - if (code != 0) { - tqErrorC("consumer:0x%" PRIx64 " failed to ask ep, code:%s", tmq->consumerId, tstrerror(code)); - } - } - tqInfoC("consumer:0x%" PRIx64 " msg from vgId:%d discarded, since %s", tmq->consumerId, pollRspWrapper->vgId, - tstrerror(pRspWrapper->code)); - taosWLockLatch(&tmq->lock); - SMqClientVg* pVg = NULL; - getVgInfo(tmq, pollRspWrapper->topicName, pollRspWrapper->vgId, &pVg); - if (pVg) pVg->emptyBlockReceiveTs = taosGetTimestampMs(); - taosWUnLockLatch(&tmq->lock); - - setVgIdle(tmq, pollRspWrapper->topicName, pollRspWrapper->vgId); - taosMemoryFreeClear(pollRspWrapper->pEpset); - tmqFreeRspWrapper(pRspWrapper); - taosFreeQitem(pRspWrapper); - } else if (pRspWrapper->tmqRspType == TMQ_MSG_TYPE__POLL_DATA_RSP) { - SMqPollRspWrapper* pollRspWrapper = (SMqPollRspWrapper*)pRspWrapper; - - int32_t consumerEpoch = atomic_load_32(&tmq->epoch); - SMqDataRspCommon* pDataRsp = (SMqDataRspCommon*)&pollRspWrapper->dataRsp; - - if (pDataRsp->head.epoch == consumerEpoch) { - taosWLockLatch(&tmq->lock); - SMqClientVg* pVg = NULL; - getVgInfo(tmq, pollRspWrapper->topicName, pollRspWrapper->vgId, &pVg); - pollRspWrapper->vgHandle = pVg; - pollRspWrapper->topicHandle = getTopicInfo(tmq, pollRspWrapper->topicName); - if (pollRspWrapper->vgHandle == NULL || pollRspWrapper->topicHandle == NULL) { - tqErrorC("consumer:0x%" PRIx64 " get vg or topic error, topic:%s vgId:%d", tmq->consumerId, - pollRspWrapper->topicName, pollRspWrapper->vgId); - tmqFreeRspWrapper(pRspWrapper); - taosFreeQitem(pRspWrapper); - taosWUnLockLatch(&tmq->lock); - return NULL; - } - // update the epset - if (pollRspWrapper->pEpset != NULL) { - SEp* pEp = GET_ACTIVE_EP(pollRspWrapper->pEpset); - SEp* pOld = GET_ACTIVE_EP(&(pVg->epSet)); - tqDebugC("consumer:0x%" PRIx64 " update epset vgId:%d, ep:%s:%d, old ep:%s:%d", tmq->consumerId, pVg->vgId, - pEp->fqdn, pEp->port, pOld->fqdn, pOld->port); - pVg->epSet = *pollRspWrapper->pEpset; - } - - updateVgInfo(pVg, &pDataRsp->reqOffset, &pDataRsp->rspOffset, pDataRsp->head.walsver, pDataRsp->head.walever, - tmq->consumerId, pDataRsp->blockNum != 0); - - char buf[TSDB_OFFSET_LEN] = {0}; - tFormatOffset(buf, TSDB_OFFSET_LEN, &pDataRsp->rspOffset); - if (pDataRsp->blockNum == 0) { - tqDebugC("consumer:0x%" PRIx64 " empty block received, vgId:%d, offset:%s, vg total:%" PRId64 - ", total:%" PRId64 ",QID:0x%" PRIx64, - tmq->consumerId, pVg->vgId, buf, pVg->numOfRows, tmq->totalRows, pollRspWrapper->reqId); - pVg->emptyBlockReceiveTs = taosGetTimestampMs(); - tmqFreeRspWrapper(pRspWrapper); - taosFreeQitem(pRspWrapper); - taosWUnLockLatch(&tmq->lock); - } else { // build rsp - int64_t numOfRows = 0; - SMqRspObj* pRsp = NULL; - (void)tmqBuildRspFromWrapper(pollRspWrapper, pVg, &numOfRows, &pRsp); - tmq->totalRows += numOfRows; - pVg->emptyBlockReceiveTs = 0; - if (pRsp && tmq->replayEnable) { - pVg->blockReceiveTs = taosGetTimestampMs(); - pVg->blockSleepForReplay = pRsp->rsp.sleepTime; - if (pVg->blockSleepForReplay > 0) { - if (taosTmrStart(tmqReplayTask, pVg->blockSleepForReplay, (void*)(tmq->refId), tmqMgmt.timer) == NULL) { - tqErrorC("consumer:0x%" PRIx64 " failed to start replay timer, vgId:%d, sleep:%" PRId64, - tmq->consumerId, pVg->vgId, pVg->blockSleepForReplay); - } - } - } - tqDebugC("consumer:0x%" PRIx64 " process poll rsp, vgId:%d, offset:%s, blocks:%d, rows:%" PRId64 - ", vg total:%" PRId64 ", total:%" PRId64 ",QID:0x%" PRIx64, - tmq->consumerId, pVg->vgId, buf, pDataRsp->blockNum, numOfRows, pVg->numOfRows, tmq->totalRows, - pollRspWrapper->reqId); - taosMemoryFreeClear(pollRspWrapper->pEpset); - taosFreeQitem(pRspWrapper); - taosWUnLockLatch(&tmq->lock); - return pRsp; - } - } else { - tqInfoC("consumer:0x%" PRIx64 " vgId:%d msg discard since epoch mismatch: msg epoch %d, consumer epoch %d", - tmq->consumerId, pollRspWrapper->vgId, pDataRsp->head.epoch, consumerEpoch); - setVgIdle(tmq, pollRspWrapper->topicName, pollRspWrapper->vgId); - tmqFreeRspWrapper(pRspWrapper); - taosFreeQitem(pRspWrapper); - } - } else if (pRspWrapper->tmqRspType == TMQ_MSG_TYPE__POLL_META_RSP) { - SMqPollRspWrapper* pollRspWrapper = (SMqPollRspWrapper*)pRspWrapper; - int32_t consumerEpoch = atomic_load_32(&tmq->epoch); - - tqDebugC("consumer:0x%" PRIx64 " process meta rsp", tmq->consumerId); - - if (pollRspWrapper->metaRsp.head.epoch == consumerEpoch) { - taosWLockLatch(&tmq->lock); - SMqClientVg* pVg = NULL; - getVgInfo(tmq, pollRspWrapper->topicName, pollRspWrapper->vgId, &pVg); - pollRspWrapper->vgHandle = pVg; - pollRspWrapper->topicHandle = getTopicInfo(tmq, pollRspWrapper->topicName); - if (pollRspWrapper->vgHandle == NULL || pollRspWrapper->topicHandle == NULL) { - tqErrorC("consumer:0x%" PRIx64 " get vg or topic error, topic:%s vgId:%d", tmq->consumerId, - pollRspWrapper->topicName, pollRspWrapper->vgId); - tmqFreeRspWrapper(pRspWrapper); - taosFreeQitem(pRspWrapper); - taosWUnLockLatch(&tmq->lock); - return NULL; - } - - updateVgInfo(pVg, &pollRspWrapper->metaRsp.rspOffset, &pollRspWrapper->metaRsp.rspOffset, - pollRspWrapper->metaRsp.head.walsver, pollRspWrapper->metaRsp.head.walever, tmq->consumerId, true); - // build rsp - SMqMetaRspObj* pRsp = NULL; - (void)tmqBuildMetaRspFromWrapper(pollRspWrapper, &pRsp); - taosMemoryFreeClear(pollRspWrapper->pEpset); - taosFreeQitem(pRspWrapper); - taosWUnLockLatch(&tmq->lock); - return pRsp; - } else { - tqInfoC("consumer:0x%" PRIx64 " vgId:%d msg discard since epoch mismatch: msg epoch %d, consumer epoch %d", - tmq->consumerId, pollRspWrapper->vgId, pollRspWrapper->metaRsp.head.epoch, consumerEpoch); - setVgIdle(tmq, pollRspWrapper->topicName, pollRspWrapper->vgId); - tmqFreeRspWrapper(pRspWrapper); - taosFreeQitem(pRspWrapper); - } - } else if (pRspWrapper->tmqRspType == TMQ_MSG_TYPE__POLL_BATCH_META_RSP) { - SMqPollRspWrapper* pollRspWrapper = (SMqPollRspWrapper*)pRspWrapper; - int32_t consumerEpoch = atomic_load_32(&tmq->epoch); - - tqDebugC("consumer:0x%" PRIx64 " process meta rsp", tmq->consumerId); - - if (pollRspWrapper->batchMetaRsp.head.epoch == consumerEpoch) { - taosWLockLatch(&tmq->lock); - SMqClientVg* pVg = NULL; - getVgInfo(tmq, pollRspWrapper->topicName, pollRspWrapper->vgId, &pVg); - pollRspWrapper->vgHandle = pVg; - pollRspWrapper->topicHandle = getTopicInfo(tmq, pollRspWrapper->topicName); - if (pollRspWrapper->vgHandle == NULL || pollRspWrapper->topicHandle == NULL) { - tqErrorC("consumer:0x%" PRIx64 " get vg or topic error, topic:%s vgId:%d", tmq->consumerId, - pollRspWrapper->topicName, pollRspWrapper->vgId); - tmqFreeRspWrapper(pRspWrapper); - taosFreeQitem(pRspWrapper); - taosWUnLockLatch(&tmq->lock); - return NULL; - } - - // build rsp - updateVgInfo(pVg, &pollRspWrapper->batchMetaRsp.rspOffset, &pollRspWrapper->batchMetaRsp.rspOffset, - pollRspWrapper->batchMetaRsp.head.walsver, pollRspWrapper->batchMetaRsp.head.walever, - tmq->consumerId, true); - SMqBatchMetaRspObj* pRsp = NULL; - (void)tmqBuildBatchMetaRspFromWrapper(pollRspWrapper, &pRsp); - taosMemoryFreeClear(pollRspWrapper->pEpset); - taosFreeQitem(pRspWrapper); - taosWUnLockLatch(&tmq->lock); - return pRsp; - } else { - tqInfoC("consumer:0x%" PRIx64 " vgId:%d msg discard since epoch mismatch: msg epoch %d, consumer epoch %d", - tmq->consumerId, pollRspWrapper->vgId, pollRspWrapper->batchMetaRsp.head.epoch, consumerEpoch); - setVgIdle(tmq, pollRspWrapper->topicName, pollRspWrapper->vgId); - tmqFreeRspWrapper(pRspWrapper); - taosFreeQitem(pRspWrapper); - } - } else if (pRspWrapper->tmqRspType == TMQ_MSG_TYPE__POLL_DATA_META_RSP) { - SMqPollRspWrapper* pollRspWrapper = (SMqPollRspWrapper*)pRspWrapper; - int32_t consumerEpoch = atomic_load_32(&tmq->epoch); - SMqDataRspCommon* pDataRsp = (SMqDataRspCommon*)&pollRspWrapper->taosxRsp; - - if (pDataRsp->head.epoch == consumerEpoch) { - taosWLockLatch(&tmq->lock); - SMqClientVg* pVg = NULL; - getVgInfo(tmq, pollRspWrapper->topicName, pollRspWrapper->vgId, &pVg); - pollRspWrapper->vgHandle = pVg; - pollRspWrapper->topicHandle = getTopicInfo(tmq, pollRspWrapper->topicName); - if (pollRspWrapper->vgHandle == NULL || pollRspWrapper->topicHandle == NULL) { - tqErrorC("consumer:0x%" PRIx64 " get vg or topic error, topic:%s vgId:%d", tmq->consumerId, - pollRspWrapper->topicName, pollRspWrapper->vgId); - tmqFreeRspWrapper(pRspWrapper); - taosFreeQitem(pRspWrapper); - taosWUnLockLatch(&tmq->lock); - return NULL; - } - - updateVgInfo(pVg, &pDataRsp->reqOffset, &pDataRsp->rspOffset, pDataRsp->head.walsver, pDataRsp->head.walever, - tmq->consumerId, pDataRsp->blockNum != 0); - - if (pDataRsp->blockNum == 0) { - tqDebugC("consumer:0x%" PRIx64 " taosx empty block received, vgId:%d, vg total:%" PRId64 ",QID:0x%" PRIx64, - tmq->consumerId, pVg->vgId, pVg->numOfRows, pollRspWrapper->reqId); - pVg->emptyBlockReceiveTs = taosGetTimestampMs(); - tmqFreeRspWrapper(pRspWrapper); - taosFreeQitem(pRspWrapper); - taosWUnLockLatch(&tmq->lock); - continue; - } else { - pVg->emptyBlockReceiveTs = 0; // reset the ts - } - - // build rsp - int64_t numOfRows = 0; - SMqTaosxRspObj* pRsp = NULL; - if (tmqBuildTaosxRspFromWrapper(pollRspWrapper, pVg, &numOfRows, &pRsp) != 0) { - tqErrorC("consumer:0x%" PRIx64 " build taosx rsp failed, vgId:%d", tmq->consumerId, pVg->vgId); - } - tmq->totalRows += numOfRows; - - char buf[TSDB_OFFSET_LEN] = {0}; - tFormatOffset(buf, TSDB_OFFSET_LEN, &pVg->offsetInfo.endOffset); - tqDebugC("consumer:0x%" PRIx64 " process taosx poll rsp, vgId:%d, offset:%s, blocks:%d, rows:%" PRId64 - ", vg total:%" PRId64 ", total:%" PRId64 ",QID:0x%" PRIx64, - tmq->consumerId, pVg->vgId, buf, pDataRsp->blockNum, numOfRows, pVg->numOfRows, tmq->totalRows, - pollRspWrapper->reqId); - - taosMemoryFreeClear(pollRspWrapper->pEpset); - taosFreeQitem(pRspWrapper); - taosWUnLockLatch(&tmq->lock); - return pRsp; - } else { - tqInfoC("consumer:0x%" PRIx64 " vgId:%d msg discard since epoch mismatch: msg epoch %d, consumer epoch %d", - tmq->consumerId, pollRspWrapper->vgId, pDataRsp->head.epoch, consumerEpoch); - setVgIdle(tmq, pollRspWrapper->topicName, pollRspWrapper->vgId); - tmqFreeRspWrapper(pRspWrapper); - taosFreeQitem(pRspWrapper); - } - } else if (pRspWrapper->tmqRspType == TMQ_MSG_TYPE__EP_RSP) { - tqDebugC("consumer:0x%" PRIx64 " ep msg received", tmq->consumerId); - SMqAskEpRspWrapper* pEpRspWrapper = (SMqAskEpRspWrapper*)pRspWrapper; - SMqAskEpRsp* rspMsg = &pEpRspWrapper->msg; - doUpdateLocalEp(tmq, pEpRspWrapper->epoch, rspMsg); - tmqFreeRspWrapper(pRspWrapper); - taosFreeQitem(pRspWrapper); - } else { - tqErrorC("consumer:0x%" PRIx64 " invalid msg received:%d", tmq->consumerId, pRspWrapper->tmqRspType); + tmqFreeRspWrapper(pRspWrapper); + taosFreeQitem(pRspWrapper); + if(returnVal != NULL){ + break; } } + + return returnVal; } TAOS_RES* tmq_consumer_poll(tmq_t* tmq, int64_t timeout) { @@ -2547,14 +2492,9 @@ const char* tmq_get_topic_name(TAOS_RES* res) { if (res == NULL) { return NULL; } - if (TD_RES_TMQ(res) || TD_RES_TMQ_METADATA(res) || TD_RES_TMQ_BATCH_META(res)) { - char* tmp = strchr(((SMqRspObjCommon*)res)->topic, '.'); - if (tmp == NULL) { - return NULL; - } - return tmp + 1; - } else if (TD_RES_TMQ_META(res)) { - char* tmp = strchr(((SMqMetaRspObj*)res)->topic, '.'); + if (TD_RES_TMQ(res) || TD_RES_TMQ_METADATA(res) || + TD_RES_TMQ_META(res) || TD_RES_TMQ_BATCH_META(res)) { + char* tmp = strchr(((SMqRspObj*)res)->topic, '.'); if (tmp == NULL) { return NULL; } @@ -2569,14 +2509,9 @@ const char* tmq_get_db_name(TAOS_RES* res) { return NULL; } - if (TD_RES_TMQ(res) || TD_RES_TMQ_METADATA(res) || TD_RES_TMQ_BATCH_META(res)) { - char* tmp = strchr(((SMqRspObjCommon*)res)->db, '.'); - if (tmp == NULL) { - return NULL; - } - return tmp + 1; - } else if (TD_RES_TMQ_META(res)) { - char* tmp = strchr(((SMqMetaRspObj*)res)->db, '.'); + if (TD_RES_TMQ(res) || TD_RES_TMQ_METADATA(res) || + TD_RES_TMQ_BATCH_META(res) || TD_RES_TMQ_META(res)) { + char* tmp = strchr(((SMqRspObj*)res)->db, '.'); if (tmp == NULL) { return NULL; } @@ -2590,10 +2525,9 @@ int32_t tmq_get_vgroup_id(TAOS_RES* res) { if (res == NULL) { return TSDB_CODE_INVALID_PARA; } - if (TD_RES_TMQ(res) || TD_RES_TMQ_METADATA(res) || TD_RES_TMQ_BATCH_META(res)) { - return ((SMqRspObjCommon*)res)->vgId; - } else if (TD_RES_TMQ_META(res)) { - return ((SMqMetaRspObj*)res)->vgId; + if (TD_RES_TMQ(res) || TD_RES_TMQ_METADATA(res) || + TD_RES_TMQ_BATCH_META(res) || TD_RES_TMQ_META(res)) { + return ((SMqRspObj*)res)->vgId; } else { return TSDB_CODE_INVALID_PARA; } @@ -2604,22 +2538,17 @@ int64_t tmq_get_vgroup_offset(TAOS_RES* res) { return TSDB_CODE_INVALID_PARA; } if (TD_RES_TMQ(res) || TD_RES_TMQ_METADATA(res)) { - SMqDataRspCommon* common = (SMqDataRspCommon*)POINTER_SHIFT(res, sizeof(SMqRspObjCommon)); - STqOffsetVal* pOffset = &common->reqOffset; - if (common->reqOffset.type == TMQ_OFFSET__LOG) { - return common->reqOffset.version; + SMqRspObj* pRspObj = (SMqRspObj*)res; + STqOffsetVal* pOffset = &pRspObj->dataRsp.reqOffset; + if (pOffset->type == TMQ_OFFSET__LOG) { + return pOffset->version; } else { - tqErrorC("invalid offset type:%d", common->reqOffset.type); + tqErrorC("invalid offset type:%d", pOffset->type); } - } else if (TD_RES_TMQ_META(res)) { - SMqMetaRspObj* pRspObj = (SMqMetaRspObj*)res; - if (pRspObj->metaRsp.rspOffset.type == TMQ_OFFSET__LOG) { - return pRspObj->metaRsp.rspOffset.version; - } - } else if (TD_RES_TMQ_BATCH_META(res)) { - SMqBatchMetaRspObj* pBtRspObj = (SMqBatchMetaRspObj*)res; - if (pBtRspObj->rsp.rspOffset.type == TMQ_OFFSET__LOG) { - return pBtRspObj->rsp.rspOffset.version; + } else if (TD_RES_TMQ_META(res) || TD_RES_TMQ_BATCH_META(res)) { + SMqRspObj* pRspObj = (SMqRspObj*)res; + if (pRspObj->rspOffset.type == TMQ_OFFSET__LOG) { + return pRspObj->rspOffset.version; } } else { tqErrorC("invalid tmq type:%d", *(int8_t*)res); @@ -2634,14 +2563,13 @@ const char* tmq_get_table_name(TAOS_RES* res) { return NULL; } if (TD_RES_TMQ(res) || TD_RES_TMQ_METADATA(res)) { - SMqDataRspCommon* common = (SMqDataRspCommon*)POINTER_SHIFT(res, sizeof(SMqRspObjCommon)); - - SMqRspObjCommon* pRspObj = (SMqRspObjCommon*)res; - if (!common->withTbName || common->blockTbName == NULL || pRspObj->resIter < 0 || - pRspObj->resIter >= common->blockNum) { + SMqRspObj* pRspObj = (SMqRspObj*)res; + SMqDataRsp* data = &pRspObj->dataRsp; + if (!data->withTbName || data->blockTbName == NULL || pRspObj->resIter < 0 || + pRspObj->resIter >= data->blockNum) { return NULL; } - return (const char*)taosArrayGetP(common->blockTbName, pRspObj->resIter); + return (const char*)taosArrayGetP(data->blockTbName, pRspObj->resIter); } return NULL; } @@ -2818,198 +2746,21 @@ end: } } -int32_t askEpCb(void* param, SDataBuf* pMsg, int32_t code) { - SMqAskEpCbParam* pParam = (SMqAskEpCbParam*)param; - if (pParam == NULL) { - goto FAIL; - } - - tmq_t* tmq = taosAcquireRef(tmqMgmt.rsetId, pParam->refId); - if (tmq == NULL) { - code = TSDB_CODE_TMQ_CONSUMER_CLOSED; - goto FAIL; - } - - if (code != TSDB_CODE_SUCCESS) { - tqErrorC("consumer:0x%" PRIx64 ", get topic endpoint error, code:%s", tmq->consumerId, tstrerror(code)); - goto END; - } - - if (pMsg == NULL) { - goto END; - } - SMqRspHead* head = pMsg->pData; - int32_t epoch = atomic_load_32(&tmq->epoch); - tqDebugC("consumer:0x%" PRIx64 ", recv ep, msg epoch %d, current epoch %d", tmq->consumerId, head->epoch, epoch); - if (pParam->sync) { - SMqAskEpRsp rsp = {0}; - if (tDecodeSMqAskEpRsp(POINTER_SHIFT(pMsg->pData, sizeof(SMqRspHead)), &rsp) != NULL) { - doUpdateLocalEp(tmq, head->epoch, &rsp); - } - tDeleteSMqAskEpRsp(&rsp); - } else { - SMqAskEpRspWrapper* pWrapper = NULL; - code = taosAllocateQitem(sizeof(SMqAskEpRspWrapper), DEF_QITEM, 0, (void**)&pWrapper); - if (code) { - goto END; - } - - pWrapper->tmqRspType = TMQ_MSG_TYPE__EP_RSP; - pWrapper->epoch = head->epoch; - (void)memcpy(&pWrapper->msg, pMsg->pData, sizeof(SMqRspHead)); - if (tDecodeSMqAskEpRsp(POINTER_SHIFT(pMsg->pData, sizeof(SMqRspHead)), &pWrapper->msg) == NULL) { - tmqFreeRspWrapper((SMqRspWrapper*)pWrapper); - taosFreeQitem(pWrapper); - } else { - (void)taosWriteQitem(tmq->mqueue, pWrapper); - } - } - -END: - (void)taosReleaseRef(tmqMgmt.rsetId, pParam->refId); - -FAIL: - if (pParam && pParam->sync) { - SAskEpInfo* pInfo = pParam->pParam; - if (pInfo) { - pInfo->code = code; - (void)tsem2_post(&pInfo->sem); - } - } - - if (pMsg) { - taosMemoryFree(pMsg->pEpSet); - taosMemoryFree(pMsg->pData); - } - - return code; -} - -int32_t syncAskEp(tmq_t* pTmq) { - SAskEpInfo* pInfo = taosMemoryMalloc(sizeof(SAskEpInfo)); - if (pInfo == NULL) return TSDB_CODE_OUT_OF_MEMORY; - if (tsem2_init(&pInfo->sem, 0, 0) != 0) { - taosMemoryFree(pInfo); - return TSDB_CODE_TSC_INTERNAL_ERROR; - } - - int32_t code = askEp(pTmq, pInfo, true, false); - if (code == 0) { - (void)tsem2_wait(&pInfo->sem); - code = pInfo->code; - } - - (void)tsem2_destroy(&pInfo->sem); - taosMemoryFree(pInfo); - return code; -} - -int32_t askEp(tmq_t* pTmq, void* param, bool sync, bool updateEpSet) { - SMqAskEpReq req = {0}; - req.consumerId = pTmq->consumerId; - req.epoch = updateEpSet ? -1 : pTmq->epoch; - tstrncpy(req.cgroup, pTmq->groupId, TSDB_CGROUP_LEN); - int code = 0; - SMqAskEpCbParam* pParam = NULL; - void* pReq = NULL; - - int32_t tlen = tSerializeSMqAskEpReq(NULL, 0, &req); - if (tlen < 0) { - tqErrorC("consumer:0x%" PRIx64 ", tSerializeSMqAskEpReq failed", pTmq->consumerId); - return TSDB_CODE_INVALID_PARA; - } - - pReq = taosMemoryCalloc(1, tlen); - if (pReq == NULL) { - tqErrorC("consumer:0x%" PRIx64 ", failed to malloc askEpReq msg, size:%d", pTmq->consumerId, tlen); - return terrno; - } - - if (tSerializeSMqAskEpReq(pReq, tlen, &req) < 0) { - tqErrorC("consumer:0x%" PRIx64 ", tSerializeSMqAskEpReq %d failed", pTmq->consumerId, tlen); - taosMemoryFree(pReq); - return TSDB_CODE_INVALID_PARA; - } - - pParam = taosMemoryCalloc(1, sizeof(SMqAskEpCbParam)); - if (pParam == NULL) { - tqErrorC("consumer:0x%" PRIx64 ", failed to malloc subscribe param", pTmq->consumerId); - taosMemoryFree(pReq); - return terrno; - } - - pParam->refId = pTmq->refId; - pParam->sync = sync; - pParam->pParam = param; - - SMsgSendInfo* sendInfo = taosMemoryCalloc(1, sizeof(SMsgSendInfo)); - if (sendInfo == NULL) { - taosMemoryFree(pReq); - taosMemoryFree(pParam); - return terrno; - } - - sendInfo->msgInfo = (SDataBuf){.pData = pReq, .len = tlen, .handle = NULL}; - sendInfo->requestId = generateRequestId(); - sendInfo->requestObjRefId = 0; - sendInfo->param = pParam; - sendInfo->paramFreeFp = taosMemoryFree; - sendInfo->fp = askEpCb; - sendInfo->msgType = TDMT_MND_TMQ_ASK_EP; - - SEpSet epSet = getEpSet_s(&pTmq->pTscObj->pAppInfo->mgmtEp); - tqDebugC("consumer:0x%" PRIx64 " ask ep from mnode,QID:0x%" PRIx64, pTmq->consumerId, sendInfo->requestId); - return asyncSendMsgToServer(pTmq->pTscObj->pAppInfo->pTransporter, &epSet, NULL, sendInfo); -} - -int32_t tmqCommitDone(SMqCommitCbParamSet* pParamSet) { - int64_t refId = pParamSet->refId; - int32_t code = 0; - tmq_t* tmq = taosAcquireRef(tmqMgmt.rsetId, refId); - if (tmq == NULL) { - code = TSDB_CODE_TMQ_CONSUMER_CLOSED; - } - - // if no more waiting rsp - if (pParamSet->callbackFn != NULL) { - pParamSet->callbackFn(tmq, pParamSet->code, pParamSet->userParam); - } - - taosMemoryFree(pParamSet); - if (tmq != NULL) { - code = taosReleaseRef(tmqMgmt.rsetId, refId); - } - - return code; -} - -int32_t commitRspCountDown(SMqCommitCbParamSet* pParamSet, int64_t consumerId, const char* pTopic, int32_t vgId) { - int32_t waitingRspNum = atomic_sub_fetch_32(&pParamSet->waitingRspNum, 1); - if (waitingRspNum == 0) { - tqDebugC("consumer:0x%" PRIx64 " topic:%s vgId:%d all commit-rsp received, commit completed", consumerId, pTopic, - vgId); - return tmqCommitDone(pParamSet); - } else { - tqDebugC("consumer:0x%" PRIx64 " topic:%s vgId:%d commit-rsp received, remain:%d", consumerId, pTopic, vgId, - waitingRspNum); - } - return 0; -} - int32_t tmqGetNextResInfo(TAOS_RES* res, bool convertUcs4, SReqResultInfo** pResInfo) { - SMqDataRspCommon* common = (SMqDataRspCommon*)POINTER_SHIFT(res, sizeof(SMqRspObjCommon)); - SMqRspObjCommon* pRspObj = (SMqRspObjCommon*)res; + SMqRspObj* pRspObj = (SMqRspObj*)res; + SMqDataRsp* data = &pRspObj->dataRsp; + pRspObj->resIter++; - if (pRspObj->resIter < common->blockNum) { - if (common->withSchema) { + if (pRspObj->resIter < data->blockNum) { + if (data->withSchema) { doFreeReqResultInfo(&pRspObj->resInfo); - SSchemaWrapper* pSW = (SSchemaWrapper*)taosArrayGetP(common->blockSchema, pRspObj->resIter); + SSchemaWrapper* pSW = (SSchemaWrapper*)taosArrayGetP(data->blockSchema, pRspObj->resIter); if (pSW) { TAOS_CHECK_RETURN(setResSchemaInfo(&pRspObj->resInfo, pSW->pSchema, pSW->nCols)); } } - void* pRetrieve = taosArrayGetP(common->blockData, pRspObj->resIter); + void* pRetrieve = taosArrayGetP(data->blockData, pRspObj->resIter); void* rawData = NULL; int64_t rows = 0; int32_t precision = 0; @@ -3058,7 +2809,7 @@ static int32_t tmqGetWalInfoCb(void* param, SDataBuf* pMsg, int32_t code) { SMqRspHead* pHead = pMsg->pData; tmq_topic_assignment assignment = {.begin = pHead->walsver, .end = pHead->walever + 1, - .currentOffset = rsp.common.rspOffset.version, + .currentOffset = rsp.rspOffset.version, .vgId = pParam->vgId}; (void)taosThreadMutexLock(&pCommon->mutex); diff --git a/source/common/src/tmsg.c b/source/common/src/tmsg.c index bdbfcd6750..7c2cfd83c9 100644 --- a/source/common/src/tmsg.c +++ b/source/common/src/tmsg.c @@ -10703,7 +10703,7 @@ int32_t tDecodeMqMetaRsp(SDecoder *pDecoder, SMqMetaRsp *pRsp) { void tDeleteMqMetaRsp(SMqMetaRsp *pRsp) { taosMemoryFree(pRsp->metaRsp); } -int32_t tEncodeMqDataRspCommon(SEncoder *pEncoder, const SMqDataRspCommon *pRsp) { +int32_t tEncodeMqDataRspCommon(SEncoder *pEncoder, const SMqDataRsp *pRsp) { if (tEncodeSTqOffsetVal(pEncoder, &pRsp->reqOffset) < 0) return -1; if (tEncodeSTqOffsetVal(pEncoder, &pRsp->rspOffset) < 0) return -1; if (tEncodeI32(pEncoder, pRsp->blockNum) < 0) return -1; @@ -10728,13 +10728,13 @@ int32_t tEncodeMqDataRspCommon(SEncoder *pEncoder, const SMqDataRspCommon *pRsp) return 0; } -int32_t tEncodeMqDataRsp(SEncoder *pEncoder, const void *pRsp) { +int32_t tEncodeMqDataRsp(SEncoder *pEncoder, const SMqDataRsp *pRsp) { if (tEncodeMqDataRspCommon(pEncoder, pRsp) < 0) return -1; if (tEncodeI64(pEncoder, ((SMqDataRsp *)pRsp)->sleepTime) < 0) return -1; return 0; } -int32_t tDecodeMqDataRspCommon(SDecoder *pDecoder, SMqDataRspCommon *pRsp) { +int32_t tDecodeMqDataRspCommon(SDecoder *pDecoder, SMqDataRsp *pRsp) { if (tDecodeSTqOffsetVal(pDecoder, &pRsp->reqOffset) < 0) return -1; if (tDecodeSTqOffsetVal(pDecoder, &pRsp->rspOffset) < 0) return -1; if (tDecodeI32(pDecoder, &pRsp->blockNum) < 0) return -1; @@ -10783,7 +10783,7 @@ int32_t tDecodeMqDataRspCommon(SDecoder *pDecoder, SMqDataRspCommon *pRsp) { return 0; } -int32_t tDecodeMqDataRsp(SDecoder *pDecoder, void *pRsp) { +int32_t tDecodeMqDataRsp(SDecoder *pDecoder, SMqDataRsp *pRsp) { if (tDecodeMqDataRspCommon(pDecoder, pRsp) < 0) return -1; if (!tDecodeIsEnd(pDecoder)) { if (tDecodeI64(pDecoder, &((SMqDataRsp *)pRsp)->sleepTime) < 0) return -1; @@ -10792,8 +10792,7 @@ int32_t tDecodeMqDataRsp(SDecoder *pDecoder, void *pRsp) { return 0; } -static void tDeleteMqDataRspCommon(void *rsp) { - SMqDataRspCommon *pRsp = rsp; +static void tDeleteMqDataRspCommon(SMqDataRsp *pRsp) { taosArrayDestroy(pRsp->blockDataLen); pRsp->blockDataLen = NULL; taosArrayDestroyP(pRsp->blockData, (FDelete)taosMemoryFree); @@ -10806,12 +10805,11 @@ static void tDeleteMqDataRspCommon(void *rsp) { tOffsetDestroy(&pRsp->rspOffset); } -void tDeleteMqDataRsp(void *rsp) { tDeleteMqDataRspCommon(rsp); } +void tDeleteMqDataRsp(SMqDataRsp *rsp) { tDeleteMqDataRspCommon(rsp); } -int32_t tEncodeSTaosxRsp(SEncoder *pEncoder, const void *rsp) { - if (tEncodeMqDataRspCommon(pEncoder, rsp) < 0) return -1; +int32_t tEncodeSTaosxRsp(SEncoder *pEncoder, const SMqDataRsp *pRsp) { + if (tEncodeMqDataRspCommon(pEncoder, pRsp) < 0) return -1; - const STaosxRsp *pRsp = (const STaosxRsp *)rsp; if (tEncodeI32(pEncoder, pRsp->createTableNum) < 0) return -1; if (pRsp->createTableNum) { for (int32_t i = 0; i < pRsp->createTableNum; i++) { @@ -10823,10 +10821,9 @@ int32_t tEncodeSTaosxRsp(SEncoder *pEncoder, const void *rsp) { return 0; } -int32_t tDecodeSTaosxRsp(SDecoder *pDecoder, void *rsp) { - if (tDecodeMqDataRspCommon(pDecoder, rsp) < 0) return -1; +int32_t tDecodeSTaosxRsp(SDecoder *pDecoder, SMqDataRsp *pRsp) { + if (tDecodeMqDataRspCommon(pDecoder, pRsp) < 0) return -1; - STaosxRsp *pRsp = (STaosxRsp *)rsp; if (tDecodeI32(pDecoder, &pRsp->createTableNum) < 0) return -1; if (pRsp->createTableNum) { if ((pRsp->createTableLen = taosArrayInit(pRsp->createTableNum, sizeof(int32_t))) == NULL) return -1; @@ -10844,10 +10841,9 @@ int32_t tDecodeSTaosxRsp(SDecoder *pDecoder, void *rsp) { return 0; } -void tDeleteSTaosxRsp(void *rsp) { - tDeleteMqDataRspCommon(rsp); +void tDeleteSTaosxRsp(SMqDataRsp *pRsp) { + tDeleteMqDataRspCommon(pRsp); - STaosxRsp *pRsp = (STaosxRsp *)rsp; taosArrayDestroy(pRsp->createTableLen); pRsp->createTableLen = NULL; taosArrayDestroyP(pRsp->createTableReq, (FDelete)taosMemoryFree); diff --git a/source/dnode/vnode/src/inc/tq.h b/source/dnode/vnode/src/inc/tq.h index 141aff0337..653b47ff14 100644 --- a/source/dnode/vnode/src/inc/tq.h +++ b/source/dnode/vnode/src/inc/tq.h @@ -112,14 +112,14 @@ int32_t tDecodeSTqHandle(SDecoder* pDecoder, STqHandle* pHandle); void tqDestroyTqHandle(void* data); // tqRead -int32_t tqScanTaosx(STQ* pTq, const STqHandle* pHandle, STaosxRsp* pRsp, SMqBatchMetaRsp* pBatchMetaRsp, STqOffsetVal* offset); +int32_t tqScanTaosx(STQ* pTq, const STqHandle* pHandle, SMqDataRsp* pRsp, SMqBatchMetaRsp* pBatchMetaRsp, STqOffsetVal* offset); int32_t tqScanData(STQ* pTq, STqHandle* pHandle, SMqDataRsp* pRsp, STqOffsetVal* pOffset, const SMqPollReq* pRequest); int32_t tqFetchLog(STQ* pTq, STqHandle* pHandle, int64_t* fetchOffset, uint64_t reqId); // tqExec -int32_t tqTaosxScanLog(STQ* pTq, STqHandle* pHandle, SPackedData submit, STaosxRsp* pRsp, int32_t* totalRows, int8_t sourceExcluded); -int32_t tqAddBlockDataToRsp(const SSDataBlock* pBlock, void* pRsp, int32_t numOfCols, int8_t precision); -int32_t tqSendDataRsp(STqHandle* pHandle, const SRpcMsg* pMsg, const SMqPollReq* pReq, const void* pRsp, +int32_t tqTaosxScanLog(STQ* pTq, STqHandle* pHandle, SPackedData submit, SMqDataRsp* pRsp, int32_t* totalRows, int8_t sourceExcluded); +int32_t tqAddBlockDataToRsp(const SSDataBlock* pBlock, SMqDataRsp* pRsp, int32_t numOfCols, int8_t precision); +int32_t tqSendDataRsp(STqHandle* pHandle, const SRpcMsg* pMsg, const SMqPollReq* pReq, const SMqDataRsp* pRsp, int32_t type, int32_t vgId); void tqPushEmptyDataRsp(STqHandle* pHandle, int32_t vgId); @@ -148,9 +148,9 @@ int32_t tqOffsetRestoreFromFile(STQ* pTq, char* name); // tq util int32_t tqExtractDelDataBlock(const void* pData, int32_t len, int64_t ver, void** pRefBlock, int32_t type); int32_t tqExtractDataForMq(STQ* pTq, STqHandle* pHandle, const SMqPollReq* pRequest, SRpcMsg* pMsg); -int32_t tqDoSendDataRsp(const SRpcHandleInfo* pRpcHandleInfo, const void* pRsp, int32_t epoch, int64_t consumerId, +int32_t tqDoSendDataRsp(const SRpcHandleInfo* pRpcHandleInfo, const SMqDataRsp* pRsp, int32_t epoch, int64_t consumerId, int32_t type, int64_t sver, int64_t ever); -int32_t tqInitDataRsp(SMqDataRspCommon* pRsp, STqOffsetVal pOffset); +int32_t tqInitDataRsp(SMqDataRsp* pRsp, STqOffsetVal pOffset); void tqUpdateNodeStage(STQ* pTq, bool isLeader); int32_t tqSetDstTableDataPayload(uint64_t suid, const STSchema* pTSchema, int32_t blockIndex, SSDataBlock* pDataBlock, SSubmitTbData* pTableData, int64_t earlyTs, const char* id); diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 07daab4459..e319c146c2 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -165,14 +165,14 @@ void tqPushEmptyDataRsp(STqHandle* pHandle, int32_t vgId) { } SMqDataRsp dataRsp = {0}; - code = tqInitDataRsp(&dataRsp.common, req.reqOffset); + code = tqInitDataRsp(&dataRsp, req.reqOffset); if (code != 0) { tqError("tqInitDataRsp failed, code:%d", code); return; } - dataRsp.common.blockNum = 0; + dataRsp.blockNum = 0; char buf[TSDB_OFFSET_LEN] = {0}; - (void)tFormatOffset(buf, TSDB_OFFSET_LEN, &dataRsp.common.reqOffset); + (void)tFormatOffset(buf, TSDB_OFFSET_LEN, &dataRsp.reqOffset); tqInfo("tqPushEmptyDataRsp to consumer:0x%" PRIx64 " vgId:%d, offset:%s,QID:0x%" PRIx64, req.consumerId, vgId, buf, req.reqId); @@ -183,18 +183,18 @@ void tqPushEmptyDataRsp(STqHandle* pHandle, int32_t vgId) { tDeleteMqDataRsp(&dataRsp); } -int32_t tqSendDataRsp(STqHandle* pHandle, const SRpcMsg* pMsg, const SMqPollReq* pReq, const void* pRsp, int32_t type, +int32_t tqSendDataRsp(STqHandle* pHandle, const SRpcMsg* pMsg, const SMqPollReq* pReq, const SMqDataRsp* pRsp, int32_t type, int32_t vgId) { int64_t sver = 0, ever = 0; walReaderValidVersionRange(pHandle->execHandle.pTqReader->pWalReader, &sver, &ever); char buf1[TSDB_OFFSET_LEN] = {0}; char buf2[TSDB_OFFSET_LEN] = {0}; - (void)tFormatOffset(buf1, TSDB_OFFSET_LEN, &((SMqDataRspCommon*)pRsp)->reqOffset); - (void)tFormatOffset(buf2, TSDB_OFFSET_LEN, &((SMqDataRspCommon*)pRsp)->rspOffset); + (void)tFormatOffset(buf1, TSDB_OFFSET_LEN, &(pRsp->reqOffset)); + (void)tFormatOffset(buf2, TSDB_OFFSET_LEN, &(pRsp->rspOffset)); tqDebug("tmq poll vgId:%d consumer:0x%" PRIx64 " (epoch %d) send rsp, block num:%d, req:%s, rsp:%s,QID:0x%" PRIx64, - vgId, pReq->consumerId, pReq->epoch, ((SMqDataRspCommon*)pRsp)->blockNum, buf1, buf2, pReq->reqId); + vgId, pReq->consumerId, pReq->epoch, pRsp->blockNum, buf1, buf2, pReq->reqId); return tqDoSendDataRsp(&pMsg->info, pRsp, pReq->epoch, pReq->consumerId, type, sver, ever); } @@ -516,7 +516,7 @@ int32_t tqProcessVgWalInfoReq(STQ* pTq, SRpcMsg* pMsg) { taosRUnLockLatch(&pTq->lock); SMqDataRsp dataRsp = {0}; - code = tqInitDataRsp(&dataRsp.common, req.reqOffset); + code = tqInitDataRsp(&dataRsp, req.reqOffset); if (code != 0) { return code; } @@ -527,10 +527,10 @@ int32_t tqProcessVgWalInfoReq(STQ* pTq, SRpcMsg* pMsg) { goto END; } - dataRsp.common.rspOffset.type = TMQ_OFFSET__LOG; + dataRsp.rspOffset.type = TMQ_OFFSET__LOG; if (reqOffset.type == TMQ_OFFSET__LOG) { - dataRsp.common.rspOffset.version = reqOffset.version; + dataRsp.rspOffset.version = reqOffset.version; } else if (reqOffset.type < 0) { STqOffset* pOffset = NULL; code = tqMetaGetOffset(pTq, req.subKey, &pOffset); @@ -541,17 +541,17 @@ int32_t tqProcessVgWalInfoReq(STQ* pTq, SRpcMsg* pMsg) { goto END; } - dataRsp.common.rspOffset.version = pOffset->val.version; + dataRsp.rspOffset.version = pOffset->val.version; tqInfo("consumer:0x%" PRIx64 " vgId:%d subkey:%s get assignment from store:%" PRId64, consumerId, vgId, - req.subKey, dataRsp.common.rspOffset.version); + req.subKey, dataRsp.rspOffset.version); } else { if (reqOffset.type == TMQ_OFFSET__RESET_EARLIEST) { - dataRsp.common.rspOffset.version = sver; // not consume yet, set the earliest position + dataRsp.rspOffset.version = sver; // not consume yet, set the earliest position } else if (reqOffset.type == TMQ_OFFSET__RESET_LATEST) { - dataRsp.common.rspOffset.version = ever; + dataRsp.rspOffset.version = ever; } tqInfo("consumer:0x%" PRIx64 " vgId:%d subkey:%s get assignment from init:%" PRId64, consumerId, vgId, req.subKey, - dataRsp.common.rspOffset.version); + dataRsp.rspOffset.version); } } else { tqError("consumer:0x%" PRIx64 " vgId:%d subkey:%s invalid offset type:%d", consumerId, vgId, req.subKey, diff --git a/source/dnode/vnode/src/tq/tqScan.c b/source/dnode/vnode/src/tq/tqScan.c index 527001e679..fe3f520da5 100644 --- a/source/dnode/vnode/src/tq/tqScan.c +++ b/source/dnode/vnode/src/tq/tqScan.c @@ -15,7 +15,7 @@ #include "tq.h" -int32_t tqAddBlockDataToRsp(const SSDataBlock* pBlock, void* pRsp, int32_t numOfCols, int8_t precision) { +int32_t tqAddBlockDataToRsp(const SSDataBlock* pBlock, SMqDataRsp* pRsp, int32_t numOfCols, int8_t precision) { int32_t dataStrLen = sizeof(SRetrieveTableRspForTmq) + blockGetEncodeSize(pBlock); void* buf = taosMemoryCalloc(1, dataStrLen); if (buf == NULL) { @@ -34,11 +34,11 @@ int32_t tqAddBlockDataToRsp(const SSDataBlock* pBlock, void* pRsp, int32_t numOf return terrno; } actualLen += sizeof(SRetrieveTableRspForTmq); - if (taosArrayPush(((SMqDataRspCommon*)pRsp)->blockDataLen, &actualLen) == NULL){ + if (taosArrayPush(pRsp->blockDataLen, &actualLen) == NULL){ taosMemoryFree(buf); return TSDB_CODE_OUT_OF_MEMORY; } - if (taosArrayPush(((SMqDataRspCommon*)pRsp)->blockData, &buf) == NULL) { + if (taosArrayPush(pRsp->blockData, &buf) == NULL) { taosMemoryFree(buf); return TSDB_CODE_OUT_OF_MEMORY; } @@ -46,18 +46,18 @@ int32_t tqAddBlockDataToRsp(const SSDataBlock* pBlock, void* pRsp, int32_t numOf return TSDB_CODE_SUCCESS; } -static int32_t tqAddBlockSchemaToRsp(const STqExecHandle* pExec, void* pRsp) { +static int32_t tqAddBlockSchemaToRsp(const STqExecHandle* pExec, SMqDataRsp* pRsp) { SSchemaWrapper* pSW = tCloneSSchemaWrapper(pExec->pTqReader->pSchemaWrapper); if (pSW == NULL) { return TSDB_CODE_OUT_OF_MEMORY; } - if (taosArrayPush(((SMqDataRspCommon*)pRsp)->blockSchema, &pSW) == NULL) { + if (taosArrayPush(pRsp->blockSchema, &pSW) == NULL) { return TSDB_CODE_OUT_OF_MEMORY; } return 0; } -static int32_t tqAddTbNameToRsp(const STQ* pTq, int64_t uid, void* pRsp, int32_t n) { +static int32_t tqAddTbNameToRsp(const STQ* pTq, int64_t uid, SMqDataRsp* pRsp, int32_t n) { SMetaReader mr = {0}; metaReaderDoInit(&mr, pTq->pVnode->pMeta, META_READER_LOCK); @@ -69,7 +69,7 @@ static int32_t tqAddTbNameToRsp(const STQ* pTq, int64_t uid, void* pRsp, int32_t for (int32_t i = 0; i < n; i++) { char* tbName = taosStrdup(mr.me.name); - if(taosArrayPush(((SMqDataRspCommon*)pRsp)->blockTbName, &tbName) == NULL){ + if(taosArrayPush(pRsp->blockTbName, &tbName) == NULL){ continue; } } @@ -139,7 +139,7 @@ int32_t tqScanData(STQ* pTq, STqHandle* pHandle, SMqDataRsp* pRsp, STqOffsetVal* code = tqAddBlockDataToRsp(pHandle->block, pRsp, pExec->numOfCols, pTq->pVnode->config.tsdbCfg.precision); TSDB_CHECK_CODE(code, line, END); - pRsp->common.blockNum++; + pRsp->blockNum++; if (pDataBlock == NULL) { blockDataDestroy(pHandle->block); pHandle->block = NULL; @@ -163,7 +163,7 @@ int32_t tqScanData(STQ* pTq, STqHandle* pHandle, SMqDataRsp* pRsp, STqOffsetVal* code = tqAddBlockDataToRsp(pDataBlock, pRsp, pExec->numOfCols, pTq->pVnode->config.tsdbCfg.precision); TSDB_CHECK_CODE(code, line, END); - pRsp->common.blockNum++; + pRsp->blockNum++; totalRows += pDataBlock->info.rows; if (totalRows >= tmqRowSize || (taosGetTimestampMs() - st > 1000)) { break; @@ -172,8 +172,8 @@ int32_t tqScanData(STQ* pTq, STqHandle* pHandle, SMqDataRsp* pRsp, STqOffsetVal* } tqDebug("consumer:0x%" PRIx64 " vgId:%d tmq task executed finished, total blocks:%d, totalRows:%d", - pHandle->consumerId, vgId, pRsp->common.blockNum, totalRows); - code = qStreamExtractOffset(task, &pRsp->common.rspOffset); + pHandle->consumerId, vgId, pRsp->blockNum, totalRows); + code = qStreamExtractOffset(task, &pRsp->rspOffset); END: if (code != 0) { tqError("consumer:0x%" PRIx64 " vgId:%d tmq task executed error, line:%d code:%d", pHandle->consumerId, vgId, line, @@ -182,7 +182,7 @@ END: return code; } -int32_t tqScanTaosx(STQ* pTq, const STqHandle* pHandle, STaosxRsp* pRsp, SMqBatchMetaRsp* pBatchMetaRsp, STqOffsetVal* pOffset) { +int32_t tqScanTaosx(STQ* pTq, const STqHandle* pHandle, SMqDataRsp* pRsp, SMqBatchMetaRsp* pBatchMetaRsp, STqOffsetVal* pOffset) { const STqExecHandle* pExec = &pHandle->execHandle; qTaskInfo_t task = pExec->task; int code = qStreamPrepareScan(task, pOffset, pHandle->execHandle.subType); @@ -204,7 +204,7 @@ int32_t tqScanTaosx(STQ* pTq, const STqHandle* pHandle, STaosxRsp* pRsp, SMqBatc tqDebug("tmqsnap task execute end, get %p", pDataBlock); if (pDataBlock != NULL && pDataBlock->info.rows > 0) { - if (pRsp->common.withTbName) { + if (pRsp->withTbName) { if (pOffset->type == TMQ_OFFSET__LOG) { int64_t uid = pExec->pTqReader->lastBlkUid; if (tqAddTbNameToRsp(pTq, uid, pRsp, 1) < 0) { @@ -213,13 +213,13 @@ int32_t tqScanTaosx(STQ* pTq, const STqHandle* pHandle, STaosxRsp* pRsp, SMqBatc } } else { char* tbName = taosStrdup(qExtractTbnameFromTask(task)); - if (taosArrayPush(pRsp->common.blockTbName, &tbName) == NULL){ + if (taosArrayPush(pRsp->blockTbName, &tbName) == NULL){ tqError("vgId:%d, failed to add tbname to rsp msg", pTq->pVnode->config.vgId); continue; } } } - if (pRsp->common.withSchema) { + if (pRsp->withSchema) { if (pOffset->type == TMQ_OFFSET__LOG) { if (tqAddBlockSchemaToRsp(pExec, pRsp) != 0){ tqError("vgId:%d, failed to add schema to rsp msg", pTq->pVnode->config.vgId); @@ -227,19 +227,19 @@ int32_t tqScanTaosx(STQ* pTq, const STqHandle* pHandle, STaosxRsp* pRsp, SMqBatc } } else { SSchemaWrapper* pSW = tCloneSSchemaWrapper(qExtractSchemaFromTask(task)); - if(taosArrayPush(pRsp->common.blockSchema, &pSW) == NULL){ + if(taosArrayPush(pRsp->blockSchema, &pSW) == NULL){ tqError("vgId:%d, failed to add schema to rsp msg", pTq->pVnode->config.vgId); continue; } } } - if (tqAddBlockDataToRsp(pDataBlock, (SMqDataRsp*)pRsp, taosArrayGetSize(pDataBlock->pDataBlock), + if (tqAddBlockDataToRsp(pDataBlock, pRsp, taosArrayGetSize(pDataBlock->pDataBlock), pTq->pVnode->config.tsdbCfg.precision) != 0) { tqError("vgId:%d, failed to add block to rsp msg", pTq->pVnode->config.vgId); continue; } - pRsp->common.blockNum++; + pRsp->blockNum++; if (pOffset->type == TMQ_OFFSET__LOG) { continue; } else { @@ -273,13 +273,13 @@ int32_t tqScanTaosx(STQ* pTq, const STqHandle* pHandle, STaosxRsp* pRsp, SMqBatc tqDebug("tmqsnap vgId: %d, tsdb consume over, switch to wal, ver %" PRId64, TD_VID(pTq->pVnode), pHandle->snapshotVer + 1); - code = qStreamExtractOffset(task, &pRsp->common.rspOffset); + code = qStreamExtractOffset(task, &pRsp->rspOffset); break; } - if (pRsp->common.blockNum > 0) { + if (pRsp->blockNum > 0) { tqDebug("tmqsnap task exec exited, get data"); - code = qStreamExtractOffset(task, &pRsp->common.rspOffset); + code = qStreamExtractOffset(task, &pRsp->rspOffset); break; } } @@ -288,7 +288,7 @@ int32_t tqScanTaosx(STQ* pTq, const STqHandle* pHandle, STaosxRsp* pRsp, SMqBatc } -static void tqProcessSubData(STQ* pTq, STqHandle* pHandle, STaosxRsp* pRsp, int32_t* totalRows, int8_t sourceExcluded){ +static void tqProcessSubData(STQ* pTq, STqHandle* pHandle, SMqDataRsp* pRsp, int32_t* totalRows, int8_t sourceExcluded){ int32_t code = 0; STqExecHandle* pExec = &pHandle->execHandle; STqReader* pReader = pExec->pTqReader; @@ -315,7 +315,7 @@ static void tqProcessSubData(STQ* pTq, STqHandle* pHandle, STaosxRsp* pRsp, int3 if ((pSubmitTbDataRet->flags & sourceExcluded) != 0) { goto END; } - if (pRsp->common.withTbName) { + if (pRsp->withTbName) { int64_t uid = pExec->pTqReader->lastBlkUid; code = tqAddTbNameToRsp(pTq, uid, pRsp, taosArrayGetSize(pBlocks)); if (code != 0) { @@ -373,7 +373,7 @@ static void tqProcessSubData(STQ* pTq, STqHandle* pHandle, STaosxRsp* pRsp, int3 if (pBlock == NULL) { continue; } - if (tqAddBlockDataToRsp(pBlock, (SMqDataRsp*)pRsp, taosArrayGetSize(pBlock->pDataBlock), + if (tqAddBlockDataToRsp(pBlock, pRsp, taosArrayGetSize(pBlock->pDataBlock), pTq->pVnode->config.tsdbCfg.precision) != 0){ tqError("vgId:%d, failed to add block to rsp msg", pTq->pVnode->config.vgId); continue; @@ -381,11 +381,11 @@ static void tqProcessSubData(STQ* pTq, STqHandle* pHandle, STaosxRsp* pRsp, int3 *totalRows += pBlock->info.rows; blockDataFreeRes(pBlock); SSchemaWrapper* pSW = taosArrayGetP(pSchemas, i); - if (taosArrayPush(pRsp->common.blockSchema, &pSW) == NULL){ + if (taosArrayPush(pRsp->blockSchema, &pSW) == NULL){ tqError("vgId:%d, failed to add schema to rsp msg", pTq->pVnode->config.vgId); continue; } - pRsp->common.blockNum++; + pRsp->blockNum++; } taosArrayDestroy(pBlocks); @@ -397,7 +397,7 @@ END: taosArrayDestroyP(pSchemas, (FDelete)tDeleteSchemaWrapper); } -int32_t tqTaosxScanLog(STQ* pTq, STqHandle* pHandle, SPackedData submit, STaosxRsp* pRsp, int32_t* totalRows, +int32_t tqTaosxScanLog(STQ* pTq, STqHandle* pHandle, SPackedData submit, SMqDataRsp* pRsp, int32_t* totalRows, int8_t sourceExcluded) { STqExecHandle* pExec = &pHandle->execHandle; int32_t code = 0; diff --git a/source/dnode/vnode/src/tq/tqUtil.c b/source/dnode/vnode/src/tq/tqUtil.c index ff9f1e524e..5276fbc7ed 100644 --- a/source/dnode/vnode/src/tq/tqUtil.c +++ b/source/dnode/vnode/src/tq/tqUtil.c @@ -20,7 +20,7 @@ static int32_t tqSendMetaPollRsp(STqHandle* pHandle, const SRpcMsg* pMsg, const static int32_t tqSendBatchMetaPollRsp(STqHandle* pHandle, const SRpcMsg* pMsg, const SMqPollReq* pReq, const SMqBatchMetaRsp* pRsp, int32_t vgId); -int32_t tqInitDataRsp(SMqDataRspCommon* pRsp, STqOffsetVal pOffset) { +int32_t tqInitDataRsp(SMqDataRsp* pRsp, STqOffsetVal pOffset) { pRsp->blockData = taosArrayInit(0, sizeof(void*)); pRsp->blockDataLen = taosArrayInit(0, sizeof(int32_t)); @@ -40,7 +40,7 @@ void tqUpdateNodeStage(STQ* pTq, bool isLeader) { streamMetaUpdateStageRole(pTq->pStreamMeta, state.term, isLeader); } -static int32_t tqInitTaosxRsp(SMqDataRspCommon* pRsp, STqOffsetVal pOffset) { +static int32_t tqInitTaosxRsp(SMqDataRsp* pRsp, STqOffsetVal pOffset) { tOffsetCopy(&pRsp->reqOffset, &pOffset); tOffsetCopy(&pRsp->rspOffset, &pOffset); @@ -116,12 +116,12 @@ static int32_t extractResetOffsetVal(STqOffsetVal* pOffsetVal, STQ* pTq, STqHand SMqDataRsp dataRsp = {0}; tqOffsetResetToLog(pOffsetVal, pHandle->pRef->refVer + 1); - code = tqInitDataRsp(&dataRsp.common, *pOffsetVal); + code = tqInitDataRsp(&dataRsp, *pOffsetVal); if (code != 0) { return code; } tqDebug("tmq poll: consumer:0x%" PRIx64 ", subkey %s, vgId:%d, (latest) offset reset to %" PRId64, consumerId, - pHandle->subKey, vgId, dataRsp.common.rspOffset.version); + pHandle->subKey, vgId, dataRsp.rspOffset.version); code = tqSendDataRsp(pHandle, pMsg, pRequest, &dataRsp, TMQ_MSG_TYPE__POLL_DATA_RSP, vgId); tDeleteMqDataRsp(&dataRsp); @@ -145,7 +145,7 @@ static int32_t extractDataAndRspForNormalSubscribe(STQ* pTq, STqHandle* pHandle, terrno = 0; SMqDataRsp dataRsp = {0}; - int code = tqInitDataRsp(&dataRsp.common, *pOffset); + int code = tqInitDataRsp(&dataRsp, *pOffset); if (code != 0) { goto end; } @@ -157,11 +157,11 @@ static int32_t extractDataAndRspForNormalSubscribe(STQ* pTq, STqHandle* pHandle, } // till now, all data has been transferred to consumer, new data needs to push client once arrived. - if (terrno == TSDB_CODE_WAL_LOG_NOT_EXIST && dataRsp.common.blockNum == 0) { + if (terrno == TSDB_CODE_WAL_LOG_NOT_EXIST && dataRsp.blockNum == 0) { // lock taosWLockLatch(&pTq->lock); int64_t ver = walGetCommittedVer(pTq->pVnode->pWal); - if (dataRsp.common.rspOffset.version > ver) { // check if there are data again to avoid lost data + if (dataRsp.rspOffset.version > ver) { // check if there are data again to avoid lost data code = tqRegisterPushHandle(pTq, pHandle, pMsg); taosWUnLockLatch(&pTq->lock); goto end; @@ -169,16 +169,16 @@ static int32_t extractDataAndRspForNormalSubscribe(STQ* pTq, STqHandle* pHandle, taosWUnLockLatch(&pTq->lock); } - tOffsetCopy(&dataRsp.common.reqOffset, - pOffset); // reqOffset represents the current date offset, may be changed if wal not exists + // reqOffset represents the current date offset, may be changed if wal not exists + tOffsetCopy(&dataRsp.reqOffset, pOffset); code = tqSendDataRsp(pHandle, pMsg, pRequest, &dataRsp, TMQ_MSG_TYPE__POLL_DATA_RSP, vgId); end : { char buf[TSDB_OFFSET_LEN] = {0}; - tFormatOffset(buf, TSDB_OFFSET_LEN, &dataRsp.common.rspOffset); + tFormatOffset(buf, TSDB_OFFSET_LEN, &dataRsp.rspOffset); tqDebug("tmq poll: consumer:0x%" PRIx64 ", subkey %s, vgId:%d, rsp block:%d, rsp offset type:%s,QID:0x%" PRIx64 " code:%d", - consumerId, pHandle->subKey, vgId, dataRsp.common.blockNum, buf, pRequest->reqId, code); + consumerId, pHandle->subKey, vgId, dataRsp.blockNum, buf, pRequest->reqId, code); tDeleteMqDataRsp(&dataRsp); return code; } @@ -207,11 +207,11 @@ static void tDeleteCommon(void* parm) {} static int32_t extractDataAndRspForDbStbSubscribe(STQ* pTq, STqHandle* pHandle, const SMqPollReq* pRequest, SRpcMsg* pMsg, STqOffsetVal* offset) { int32_t vgId = TD_VID(pTq->pVnode); - STaosxRsp taosxRsp = {0}; + SMqDataRsp taosxRsp = {0}; SMqBatchMetaRsp btMetaRsp = {0}; int32_t code = 0; - TQ_ERR_GO_TO_END(tqInitTaosxRsp(&taosxRsp.common, *offset)); + TQ_ERR_GO_TO_END(tqInitTaosxRsp(&taosxRsp, *offset)); if (offset->type != TMQ_OFFSET__LOG) { TQ_ERR_GO_TO_END(tqScanTaosx(pTq, pHandle, &taosxRsp, &btMetaRsp, offset)); @@ -226,13 +226,13 @@ static int32_t extractDataAndRspForDbStbSubscribe(STQ* pTq, STqHandle* pHandle, tqDebug("taosx poll: consumer:0x%" PRIx64 " subkey:%s vgId:%d, send data blockNum:%d, offset type:%d,uid:%" PRId64 ",ts:%" PRId64, - pRequest->consumerId, pHandle->subKey, vgId, taosxRsp.common.blockNum, taosxRsp.common.rspOffset.type, - taosxRsp.common.rspOffset.uid, taosxRsp.common.rspOffset.ts); - if (taosxRsp.common.blockNum > 0) { + pRequest->consumerId, pHandle->subKey, vgId, taosxRsp.blockNum, taosxRsp.rspOffset.type, + taosxRsp.rspOffset.uid, taosxRsp.rspOffset.ts); + if (taosxRsp.blockNum > 0) { code = tqSendDataRsp(pHandle, pMsg, pRequest, &taosxRsp, TMQ_MSG_TYPE__POLL_DATA_RSP, vgId); goto END; } else { - tOffsetCopy(offset, &taosxRsp.common.rspOffset); + tOffsetCopy(offset, &taosxRsp.rspOffset); } } @@ -263,7 +263,7 @@ static int32_t extractDataAndRspForDbStbSubscribe(STQ* pTq, STqHandle* pHandle, } goto END; } - tqOffsetResetToLog(&taosxRsp.common.rspOffset, fetchVer); + tqOffsetResetToLog(&taosxRsp.rspOffset, fetchVer); code = tqSendDataRsp( pHandle, pMsg, pRequest, &taosxRsp, taosxRsp.createTableNum > 0 ? TMQ_MSG_TYPE__POLL_DATA_META_RSP : TMQ_MSG_TYPE__POLL_DATA_RSP, vgId); @@ -277,7 +277,7 @@ static int32_t extractDataAndRspForDbStbSubscribe(STQ* pTq, STqHandle* pHandle, // process meta if (pHead->msgType != TDMT_VND_SUBMIT) { if (totalRows > 0) { - tqOffsetResetToLog(&taosxRsp.common.rspOffset, fetchVer); + tqOffsetResetToLog(&taosxRsp.rspOffset, fetchVer); code = tqSendDataRsp( pHandle, pMsg, pRequest, &taosxRsp, taosxRsp.createTableNum > 0 ? TMQ_MSG_TYPE__POLL_DATA_META_RSP : TMQ_MSG_TYPE__POLL_DATA_RSP, vgId); @@ -386,7 +386,7 @@ static int32_t extractDataAndRspForDbStbSubscribe(STQ* pTq, STqHandle* pHandle, } if (totalRows >= tmqRowSize || (taosGetTimestampMs() - st > 1000)) { - tqOffsetResetToLog(&taosxRsp.common.rspOffset, fetchVer + 1); + tqOffsetResetToLog(&taosxRsp.rspOffset, fetchVer + 1); code = tqSendDataRsp( pHandle, pMsg, pRequest, &taosxRsp, taosxRsp.createTableNum > 0 ? TMQ_MSG_TYPE__POLL_DATA_META_RSP : TMQ_MSG_TYPE__POLL_DATA_RSP, vgId); @@ -521,7 +521,7 @@ int32_t tqSendMetaPollRsp(STqHandle* pHandle, const SRpcMsg* pMsg, const SMqPoll return 0; } -int32_t tqDoSendDataRsp(const SRpcHandleInfo* pRpcHandleInfo, const void* pRsp, int32_t epoch, int64_t consumerId, +int32_t tqDoSendDataRsp(const SRpcHandleInfo* pRpcHandleInfo, const SMqDataRsp* pRsp, int32_t epoch, int64_t consumerId, int32_t type, int64_t sver, int64_t ever) { int32_t len = 0; int32_t code = 0; diff --git a/tests/parallel_test/cases.task b/tests/parallel_test/cases.task index e8c765a218..9431774112 100644 --- a/tests/parallel_test/cases.task +++ b/tests/parallel_test/cases.task @@ -157,7 +157,6 @@ ,,y,system-test,./pytest.sh python3 ./test.py -f 7-tmq/tmqMaxTopic.py ,,y,system-test,./pytest.sh python3 ./test.py -f 7-tmq/tmqParamsTest.py ,,y,system-test,./pytest.sh python3 ./test.py -f 7-tmq/tmqParamsTest.py -R -,,y,system-test,./pytest.sh python3 ./test.py -f 7-tmq/tmqClientConsLog.py ,,y,system-test,./pytest.sh python3 ./test.py -f 7-tmq/tmqMaxGroupIds.py ,,y,system-test,./pytest.sh python3 ./test.py -f 7-tmq/tmqConsumeDiscontinuousData.py ,,y,system-test,./pytest.sh python3 ./test.py -f 7-tmq/tmqOffset.py diff --git a/tests/system-test/7-tmq/tmqClientConsLog.py b/tests/system-test/7-tmq/tmqClientConsLog.py deleted file mode 100644 index 83d6f93be1..0000000000 --- a/tests/system-test/7-tmq/tmqClientConsLog.py +++ /dev/null @@ -1,231 +0,0 @@ - -import taos -import sys -import time -import socket -import os -import threading -import math - -from util.log import * -from util.sql import * -from util.cases import * -from util.dnodes import * -from util.common import * -sys.path.append("./7-tmq") -from tmqCommon import * - -class TDTestCase: - - clientCfgDict = {'debugFlag': 135} - updatecfgDict = {'debugFlag': 131, 'clientCfg':clientCfgDict} - def __init__(self): - self.vgroups = 3 - self.ctbNum = 10 - self.rowsPerTbl = 1000 - - def init(self, conn, logSql, replicaVar=1): - self.replicaVar = int(replicaVar) - tdLog.debug(f"start to excute {__file__}") - tdSql.init(conn.cursor(), True) - - def prepareTestEnv(self): - tdLog.printNoPrefix("======== prepare test env include database, stable, ctables, and insert data: ") - paraDict = {'dbName': 'dbt', - 'dropFlag': 1, - 'event': '', - 'vgroups': 2, - 'stbName': 'stb', - 'colPrefix': 'c', - 'tagPrefix': 't', - 'colSchema': [{'type': 'INT', 'count':1},{'type': 'BIGINT', 'count':1},{'type': 'DOUBLE', 'count':1},{'type': 'BINARY', 'len':32, 'count':1},{'type': 'NCHAR', 'len':32, 'count':1},{'type': 'TIMESTAMP', 'count':1}], - 'tagSchema': [{'type': 'INT', 'count':1},{'type': 'BIGINT', 'count':1},{'type': 'DOUBLE', 'count':1},{'type': 'BINARY', 'len':32, 'count':1},{'type': 'NCHAR', 'len':32, 'count':1}], - 'ctbPrefix': 'ctb', - 'ctbStartIdx': 0, - 'ctbNum': 10, - 'rowsPerTbl': 1000, - 'batchNum': 100, - 'startTs': 1640966400000, # 2022-01-01 00:00:00.000 - 'pollDelay': 10, - 'showMsg': 1, - 'showRow': 1, - 'snapshot': 0} - - paraDict['vgroups'] = self.vgroups - paraDict['ctbNum'] = self.ctbNum - paraDict['rowsPerTbl'] = self.rowsPerTbl - - tmqCom.initConsumerTable() - tdCom.create_database(tdSql, paraDict["dbName"],paraDict["dropFlag"], vgroups=paraDict["vgroups"],replica=self.replicaVar) - tdLog.info("create stb") - tmqCom.create_stable(tdSql, dbName=paraDict["dbName"],stbName=paraDict["stbName"]) - tdLog.info("create ctb") - tmqCom.create_ctable(tdSql, dbName=paraDict["dbName"],stbName=paraDict["stbName"],ctbPrefix=paraDict['ctbPrefix'], - ctbNum=paraDict["ctbNum"],ctbStartIdx=paraDict['ctbStartIdx']) - tdLog.info("insert data") - tmqCom.insert_data_interlaceByMultiTbl(tsql=tdSql,dbName=paraDict["dbName"],ctbPrefix=paraDict["ctbPrefix"], - ctbNum=paraDict["ctbNum"],rowsPerTbl=paraDict["rowsPerTbl"],batchNum=paraDict["batchNum"], - startTs=paraDict["startTs"],ctbStartIdx=paraDict['ctbStartIdx']) - - # tdLog.info("restart taosd to ensure that the data falls into the disk") - # tdDnodes.stop(1) - # tdDnodes.start(1) - # tdSql.query("flush database %s"%(paraDict['dbName'])) - return - - def updateRowsOfConsumer(self, consumerDict, consumerId, totalRowsOfConsumer): - for key in consumerDict: - if key == consumerId: - consumerDict[key] = totalRowsOfConsumer - return - - consumerDict[consumerId] = totalRowsOfConsumer - return - - def checkClientLog(self, actConsumeTotalRows, numOfConsumer): - # 01931245 TSC consumer:0x5ee20f124420000c process poll rsp, vgId:5, offset:log:3399, blocks:2, rows:6000 vg total:330000 total:654000, reqId:0xa77d2245ae20112 - # 01931245 TSC consumer:0x5ee20f124420000c process poll rsp, vgId:7, offset:log:3384, blocks:1, rows:2000 vg total:326000 total:656000, reqId:0xa77d2245b050113 - # 01931246 TSC consumer:0x5ee20f124420000d process poll rsp, vgId:6, offset:log:3400, blocks:2, rows:6000 vg total:330000 total:330000, reqId:0xa77d2245b380116 - # 01931246 TSC consumer:0x5ee20f124420000d process poll rsp, vgId:6, offset:log:3460, blocks:2, rows:6000 vg total:336000 total:336000, reqId:0xa77d2245b8f011a - # 01931246 TSC consumer:0x5ee20f124420000d process poll rsp, vgId:6, offset:log:3520, blocks:2, rows:6000 vg total:342000 total:342000, reqId:0xa77d2245beb011f - # 01931246 TSC consumer:0x5ee20f124420000d process poll rsp, vgId:6, offset:log:3567, blocks:1, rows:2000 vg total:344000 total:344000, reqId:0xa77d2245c430121 - # filter key: process poll rsp, vgId - - tdLog.printNoPrefix("======== start filter key info from client log file") - - cfgPath = tdCom.getClientCfgPath() - taosLogFile = '%s/../log/taoslog*'%(cfgPath) - filterResultFile = '%s/../log/filter'%(cfgPath) - cmdStr = 'grep -h "process poll rsp, vgId:" %s >> %s'%(taosLogFile, filterResultFile) - tdLog.info(cmdStr) - os.system(cmdStr) - - consumerDict = {} - for index, line in enumerate(open(filterResultFile,'r')): - - # tdLog.info("row[%d]: %s"%(index, line)) - valueList = line.split(',') - # for i in range(len(valueList)): - # tdLog.info("index[%d]: %s"%(i, valueList[i])) - # get consumer id - list2 = valueList[0].split(':') - list3 = list2[3].split() - consumerId = list3[0] - print("consumerId: %s"%(consumerId)) - - # # get vg id - # list2 = valueList[1].split(':') - # vgId = list2[1] - # print("vgId: %s"%(vgId)) - - # get total rows of a certain consuer - list2 = valueList[6].split(':') - totalRowsOfConsumer = list2[1] - print("totalRowsOfConsumer: %s"%(totalRowsOfConsumer)) - - # update a certain info - self.updateRowsOfConsumer(consumerDict, consumerId, totalRowsOfConsumer) - - # print(consumerDict) - if numOfConsumer != len(consumerDict): - tdLog.info("expect consumer num: %d, act consumer num: %d"%(numOfConsumer, len(consumerDict))) - tdLog.exit("act consumer error!") - - # total rows of all consumers - totalRows = 0 - for key in consumerDict: - totalRows += int(consumerDict[key]) - - if totalRows < actConsumeTotalRows: - tdLog.info("expect consume total rows: %d, act consume total rows: %d"%(actConsumeTotalRows, totalRows)) - tdLog.exit("act consume rows error!") - return - - def tmqCase1(self): - tdLog.printNoPrefix("======== test case 1: ") - paraDict = {'dbName': 'dbt', - 'dropFlag': 1, - 'event': '', - 'vgroups': 2, - 'stbName': 'stb', - 'colPrefix': 'c', - 'tagPrefix': 't', - 'colSchema': [{'type': 'INT', 'count':1},{'type': 'BIGINT', 'count':1},{'type': 'DOUBLE', 'count':1},{'type': 'BINARY', 'len':32, 'count':1},{'type': 'NCHAR', 'len':32, 'count':1},{'type': 'TIMESTAMP', 'count':1}], - 'tagSchema': [{'type': 'INT', 'count':1},{'type': 'BIGINT', 'count':1},{'type': 'DOUBLE', 'count':1},{'type': 'BINARY', 'len':32, 'count':1},{'type': 'NCHAR', 'len':32, 'count':1}], - 'ctbPrefix': 'ctb', - 'ctbStartIdx': 0, - 'ctbNum': 10, - 'rowsPerTbl': 1000, - 'batchNum': 100, - 'startTs': 1640966400000, # 2022-01-01 00:00:00.000 - 'pollDelay': 10, - 'showMsg': 1, - 'showRow': 1, - 'snapshot': 0} - paraDict['vgroups'] = self.vgroups - paraDict['ctbNum'] = self.ctbNum - paraDict['rowsPerTbl'] = self.rowsPerTbl - - topicNameList = ['topic1'] - expectRowsList = [] - tmqCom.initConsumerTable() - - tdLog.info("create topics from stb with filter") - queryString = "select * from %s.%s"%(paraDict['dbName'], paraDict['stbName']) - # sqlString = "create topic %s as stable %s" %(topicNameList[0], paraDict['stbName']) - sqlString = "create topic %s as %s" %(topicNameList[0], queryString) - tdLog.info("create topic sql: %s"%sqlString) - tdSql.execute(sqlString) - tdSql.query(queryString) - expectRowsList.append(tdSql.getRows()) - totalRowsInserted = expectRowsList[0] - - # init consume info, and start tmq_sim, then check consume result - tdLog.info("insert consume info to consume processor") - consumerId = 0 - expectrowcnt = paraDict["rowsPerTbl"] * paraDict["ctbNum"] - topicList = topicNameList[0] - ifcheckdata = 0 - ifManualCommit = 1 - keyList = 'group.id:cgrp1, enable.auto.commit:true, auto.commit.interval.ms:500, auto.offset.reset:earliest' - tmqCom.insertConsumerInfo(consumerId, expectrowcnt,topicList,keyList,ifcheckdata,ifManualCommit) - - consumerId = 1 - tmqCom.insertConsumerInfo(consumerId, expectrowcnt,topicList,keyList,ifcheckdata,ifManualCommit) - - tdLog.info("start consume processor 0") - tmqCom.startTmqSimProcess(pollDelay=paraDict['pollDelay'],dbName=paraDict["dbName"],showMsg=paraDict['showMsg'], showRow=paraDict['showRow'],snapshot=paraDict['snapshot']) - tdLog.info("wait the consume result") - - expectRows = 2 - resultList = tmqCom.selectConsumeResult(expectRows) - actConsumeTotalRows = resultList[0] + resultList[1] - - tdLog.info("two consumers poll rows: %d, %d"%(resultList[0], resultList[1])) - - tdLog.info("the consume rows: %d should be equal to total inserted rows: %d"%(actConsumeTotalRows, totalRowsInserted)) - if not (totalRowsInserted <= actConsumeTotalRows): - tdLog.exit("%d tmq consume rows error!"%consumerId) - - self.checkClientLog(actConsumeTotalRows, 2) - - time.sleep(10) - for i in range(len(topicNameList)): - tdSql.query("drop topic %s"%topicNameList[i]) - - tdLog.printNoPrefix("======== test case 1 end ...... ") - - def run(self): - tdSql.prepare() - self.prepareTestEnv() - self.tmqCase1() - - - def stop(self): - tdSql.close() - tdLog.success(f"{__file__} successfully executed") - -event = threading.Event() - -tdCases.addLinux(__file__, TDTestCase()) -tdCases.addWindows(__file__, TDTestCase()) diff --git a/tests/system-test/win-test-file b/tests/system-test/win-test-file index 1f2b3f476c..e86047bca8 100644 --- a/tests/system-test/win-test-file +++ b/tests/system-test/win-test-file @@ -87,7 +87,6 @@ python3 ./test.py -f 7-tmq/ins_topics_test.py python3 ./test.py -f 7-tmq/tmqMaxTopic.py python3 ./test.py -f 7-tmq/tmqParamsTest.py python3 ./test.py -f 7-tmq/tmqParamsTest.py -R -python3 ./test.py -f 7-tmq/tmqClientConsLog.py python3 ./test.py -f 7-tmq/tmqMaxGroupIds.py python3 ./test.py -f 7-tmq/tmqConsumeDiscontinuousData.py python3 ./test.py -f 7-tmq/tmqOffset.py From 221733edb26fa75d25309bd7d99e488281b032b7 Mon Sep 17 00:00:00 2001 From: wangmm0220 Date: Mon, 9 Sep 2024 16:49:56 +0800 Subject: [PATCH 09/51] enh:[TD-30270]opti data struct in tmq --- source/client/src/clientRawBlockWrite.c | 14 +++++++++++--- source/client/src/clientTmq.c | 9 +++++---- 2 files changed, 16 insertions(+), 7 deletions(-) diff --git a/source/client/src/clientRawBlockWrite.c b/source/client/src/clientRawBlockWrite.c index d481ede616..809d1442ba 100644 --- a/source/client/src/clientRawBlockWrite.c +++ b/source/client/src/clientRawBlockWrite.c @@ -2096,13 +2096,21 @@ int32_t tmq_get_raw(TAOS_RES* res, tmq_raw_data* raw) { raw->raw_len = rspObj->metaRsp.metaRspLen; raw->raw_type = rspObj->metaRsp.resMsgType; uDebug("tmq get raw type meta:%p", raw); - } else if (TD_RES_TMQ(res) || TD_RES_TMQ_METADATA(res)) { - int32_t code = encodeMqDataRsp(tEncodeMqDataRsp, &rspObj->dataRsp, raw); + } else if (TD_RES_TMQ(res)) { + int32_t code = encodeMqDataRsp(tEncodeMqDataRsp, &rspObj->dataRsp, raw); if (code != 0) { uError("tmq get raw type error:%d", terrno); return code; } - raw->raw_type = rspObj->resType; + raw->raw_type = RES_TYPE__TMQ; + uDebug("tmq get raw type data:%p", raw); + } else if (TD_RES_TMQ_METADATA(res)) { + int32_t code = encodeMqDataRsp(tEncodeSTaosxRsp, &rspObj->dataRsp, raw); + if (code != 0) { + uError("tmq get raw type error:%d", terrno); + return code; + } + raw->raw_type = RES_TYPE__TMQ_METADATA; uDebug("tmq get raw type metadata:%p", raw); } else if (TD_RES_TMQ_BATCH_META(res)) { raw->raw = rspObj->batchMetaRsp.pMetaBuff; diff --git a/source/client/src/clientTmq.c b/source/client/src/clientTmq.c index 2ee9396f7f..20798fbdeb 100644 --- a/source/client/src/clientTmq.c +++ b/source/client/src/clientTmq.c @@ -55,7 +55,7 @@ (void)memcpy(DATA, pMsg->pData, sizeof(SMqRspHead)); #define DELETE_POLL_RSP(FUNC,DATA) \ - SMqPollRspWrapper* pRsp = (SMqPollRspWrapper*)rspWrapper;\ + SMqPollRspWrapper* pRsp = &rspWrapper->pollRsp;\ taosMemoryFreeClear(pRsp->pEpset);\ FUNC(DATA); @@ -1029,9 +1029,10 @@ static void defaultCommitCbFn(tmq_t* pTmq, int32_t code, void* param) { static void tmqFreeRspWrapper(SMqRspWrapper* rspWrapper) { if (rspWrapper->tmqRspType == TMQ_MSG_TYPE__EP_RSP) { tDeleteSMqAskEpRsp(&rspWrapper->epRsp); - } else if (rspWrapper->tmqRspType == TMQ_MSG_TYPE__POLL_DATA_RSP || - rspWrapper->tmqRspType == TMQ_MSG_TYPE__POLL_DATA_META_RSP) { - DELETE_POLL_RSP(tDeleteMqDataRsp,&pRsp->dataRsp) + } else if (rspWrapper->tmqRspType == TMQ_MSG_TYPE__POLL_DATA_RSP) { + DELETE_POLL_RSP(tDeleteMqDataRsp, &pRsp->dataRsp) + } else if (rspWrapper->tmqRspType == TMQ_MSG_TYPE__POLL_DATA_META_RSP){ + DELETE_POLL_RSP(tDeleteSTaosxRsp, &pRsp->dataRsp) } else if (rspWrapper->tmqRspType == TMQ_MSG_TYPE__POLL_META_RSP) { DELETE_POLL_RSP(tDeleteMqMetaRsp,&pRsp->metaRsp) } else if (rspWrapper->tmqRspType == TMQ_MSG_TYPE__POLL_BATCH_META_RSP) { From a1fbf769a824df548bc794b8935def5a018f9e05 Mon Sep 17 00:00:00 2001 From: wangmm0220 Date: Mon, 9 Sep 2024 17:15:07 +0800 Subject: [PATCH 10/51] enh:[TD-30270]opti data struct in tmq --- include/common/tmsg.h | 27 --------------------------- source/client/inc/clientInt.h | 24 ++++++++++++------------ 2 files changed, 12 insertions(+), 39 deletions(-) diff --git a/include/common/tmsg.h b/include/common/tmsg.h index c5b70d0175..8962265fa2 100644 --- a/include/common/tmsg.h +++ b/include/common/tmsg.h @@ -649,20 +649,6 @@ void tFreeSSubmitRsp(SSubmitRsp* pRsp); (s)->flags &= (~COL_IDX_ON); \ } while (0) -enum { - RES_TYPE__QUERY = 1, - RES_TYPE__TMQ, - RES_TYPE__TMQ_META, - RES_TYPE__TMQ_METADATA, - RES_TYPE__TMQ_BATCH_META, -}; - -#define TD_RES_QUERY(res) (*(int8_t*)(res) == RES_TYPE__QUERY) -#define TD_RES_TMQ(res) (*(int8_t*)(res) == RES_TYPE__TMQ) -#define TD_RES_TMQ_META(res) (*(int8_t*)(res) == RES_TYPE__TMQ_META) -#define TD_RES_TMQ_METADATA(res) (*(int8_t*)(res) == RES_TYPE__TMQ_METADATA) -#define TD_RES_TMQ_BATCH_META(res) (*(int8_t*)(res) == RES_TYPE__TMQ_BATCH_META) - #define SSCHMEA_TYPE(s) ((s)->type) #define SSCHMEA_FLAGS(s) ((s)->flags) #define SSCHMEA_COLID(s) ((s)->colId) @@ -4056,19 +4042,6 @@ void tDeleteMqMetaRsp(SMqMetaRsp* pRsp); #define MQ_DATA_RSP_VERSION 100 -//typedef struct { -// SMqRspHead head; -// STqOffsetVal rspOffset; -// STqOffsetVal reqOffset; -// int32_t blockNum; -// int8_t withTbName; -// int8_t withSchema; -// SArray* blockDataLen; -// SArray* blockData; -// SArray* blockTbName; -// SArray* blockSchema; -//} SMqDataRspCommon; - typedef struct { struct { SMqRspHead head; diff --git a/source/client/inc/clientInt.h b/source/client/inc/clientInt.h index b198762733..afb0686845 100644 --- a/source/client/inc/clientInt.h +++ b/source/client/inc/clientInt.h @@ -39,24 +39,24 @@ extern "C" { #define ERROR_MSG_BUF_DEFAULT_SIZE 512 #define HEARTBEAT_INTERVAL 1500 // ms -//enum { -// RES_TYPE__QUERY = 1, -// RES_TYPE__TMQ, -// RES_TYPE__TMQ_META, -// RES_TYPE__TMQ_METADATA, -// RES_TYPE__TMQ_BATCH_META, -//}; +enum { + RES_TYPE__QUERY = 1, + RES_TYPE__TMQ, + RES_TYPE__TMQ_META, + RES_TYPE__TMQ_METADATA, + RES_TYPE__TMQ_BATCH_META, +}; #define SHOW_VARIABLES_RESULT_COLS 3 #define SHOW_VARIABLES_RESULT_FIELD1_LEN (TSDB_CONFIG_OPTION_LEN + VARSTR_HEADER_SIZE) #define SHOW_VARIABLES_RESULT_FIELD2_LEN (TSDB_CONFIG_VALUE_LEN + VARSTR_HEADER_SIZE) #define SHOW_VARIABLES_RESULT_FIELD3_LEN (TSDB_CONFIG_SCOPE_LEN + VARSTR_HEADER_SIZE) -//#define TD_RES_QUERY(res) (*(int8_t*)(res) == RES_TYPE__QUERY) -//#define TD_RES_TMQ(res) (*(int8_t*)(res) == RES_TYPE__TMQ) -//#define TD_RES_TMQ_META(res) (*(int8_t*)(res) == RES_TYPE__TMQ_META) -//#define TD_RES_TMQ_METADATA(res) (*(int8_t*)(res) == RES_TYPE__TMQ_METADATA) -//#define TD_RES_TMQ_BATCH_META(res) (*(int8_t*)(res) == RES_TYPE__TMQ_BATCH_META) +#define TD_RES_QUERY(res) (*(int8_t*)(res) == RES_TYPE__QUERY) +#define TD_RES_TMQ(res) (*(int8_t*)(res) == RES_TYPE__TMQ) +#define TD_RES_TMQ_META(res) (*(int8_t*)(res) == RES_TYPE__TMQ_META) +#define TD_RES_TMQ_METADATA(res) (*(int8_t*)(res) == RES_TYPE__TMQ_METADATA) +#define TD_RES_TMQ_BATCH_META(res) (*(int8_t*)(res) == RES_TYPE__TMQ_BATCH_META) typedef struct SAppInstInfo SAppInstInfo; From 88f8f62f8797514a3903258e14b984ad23419896 Mon Sep 17 00:00:00 2001 From: wangmm0220 Date: Tue, 10 Sep 2024 15:11:43 +0800 Subject: [PATCH 11/51] enh:[TD-30270]opti data struct in tmq --- include/util/tlog.h | 2 ++ source/client/src/clientMain.c | 5 ++++- source/client/src/clientTmq.c | 23 ++++++++++++----------- source/common/src/tglobal.c | 3 ++- source/common/src/tmsg.c | 2 +- source/dnode/mnode/impl/src/mndConsumer.c | 2 +- source/util/src/tlog.c | 2 ++ 7 files changed, 24 insertions(+), 15 deletions(-) diff --git a/include/util/tlog.h b/include/util/tlog.h index 832dc7dbc1..e80e94de32 100644 --- a/include/util/tlog.h +++ b/include/util/tlog.h @@ -69,6 +69,8 @@ extern int32_t tdbDebugFlag; extern int32_t sndDebugFlag; extern int32_t simDebugFlag; +extern int32_t tqClientDebug; + int32_t taosInitLog(const char *logName, int32_t maxFiles, bool tsc); void taosCloseLog(); void taosResetLog(); diff --git a/source/client/src/clientMain.c b/source/client/src/clientMain.c index 78135b245f..0b50ac6ea0 100644 --- a/source/client/src/clientMain.c +++ b/source/client/src/clientMain.c @@ -377,9 +377,12 @@ void taos_free_result(TAOS_RES *res) { return; } SMqRspObj *pRsp = (SMqRspObj *)res; - if (TD_RES_TMQ_METADATA(res) || TD_RES_TMQ(res)) { + if (TD_RES_TMQ(res)) { tDeleteMqDataRsp(&pRsp->dataRsp); doFreeReqResultInfo(&pRsp->resInfo); + } else if (TD_RES_TMQ_METADATA(res)) { + tDeleteSTaosxRsp(&pRsp->dataRsp); + doFreeReqResultInfo(&pRsp->resInfo); } else if (TD_RES_TMQ_META(res)) { tDeleteMqMetaRsp(&pRsp->metaRsp); } else if (TD_RES_TMQ_BATCH_META(res)) { diff --git a/source/client/src/clientTmq.c b/source/client/src/clientTmq.c index 20798fbdeb..3803ac0c7a 100644 --- a/source/client/src/clientTmq.c +++ b/source/client/src/clientTmq.c @@ -24,12 +24,12 @@ #include "tref.h" #include "ttimer.h" -#define tqFatalC(...) do { if (tqDebugFlag & DEBUG_FATAL) { taosPrintLog("TQ FATAL ", DEBUG_FATAL, tqDebugFlag, __VA_ARGS__); }} while(0) -#define tqErrorC(...) do { if (tqDebugFlag & DEBUG_ERROR) { taosPrintLog("TQ ERROR ", DEBUG_ERROR, tqDebugFlag, __VA_ARGS__); }} while(0) -#define tqWarnC(...) do { if (tqDebugFlag & DEBUG_WARN) { taosPrintLog("TQ WARN ", DEBUG_WARN, tqDebugFlag, __VA_ARGS__); }} while(0) -#define tqInfoC(...) do { if (tqDebugFlag & DEBUG_INFO) { taosPrintLog("TQ ", DEBUG_INFO, tqDebugFlag, __VA_ARGS__); }} while(0) -#define tqDebugC(...) do { if (tqDebugFlag & DEBUG_DEBUG) { taosPrintLog("TQ ", DEBUG_DEBUG, tqDebugFlag, __VA_ARGS__); }} while(0) -#define tqTraceC(...) do { if (tqDebugFlag & DEBUG_TRACE) { taosPrintLog("TQ ", DEBUG_TRACE, tqDebugFlag, __VA_ARGS__); }} while(0) +#define tqFatalC(...) do { if (cDebugFlag & DEBUG_FATAL || tqClientDebug) { taosPrintLog("TQ FATAL ", DEBUG_FATAL, tqDebugFlag, __VA_ARGS__); }} while(0) +#define tqErrorC(...) do { if (cDebugFlag & DEBUG_ERROR || tqClientDebug) { taosPrintLog("TQ ERROR ", DEBUG_ERROR, tqDebugFlag, __VA_ARGS__); }} while(0) +#define tqWarnC(...) do { if (cDebugFlag & DEBUG_WARN || tqClientDebug) { taosPrintLog("TQ WARN ", DEBUG_WARN, tqDebugFlag, __VA_ARGS__); }} while(0) +#define tqInfoC(...) do { if (cDebugFlag & DEBUG_INFO || tqClientDebug) { taosPrintLog("TQ ", DEBUG_INFO, tqDebugFlag, __VA_ARGS__); }} while(0) +#define tqDebugC(...) do { if (cDebugFlag & DEBUG_DEBUG || tqClientDebug) { taosPrintLog("TQ ", DEBUG_DEBUG, tqDebugFlag, __VA_ARGS__); }} while(0) +#define tqTraceC(...) do { if (cDebugFlag & DEBUG_TRACE || tqClientDebug) { taosPrintLog("TQ ", DEBUG_TRACE, tqDebugFlag, __VA_ARGS__); }} while(0) #define EMPTY_BLOCK_POLL_IDLE_DURATION 10 #define DEFAULT_AUTO_COMMIT_INTERVAL 5000 @@ -910,7 +910,7 @@ int32_t tmqHbCb(void* param, SDataBuf* pMsg, int32_t code) { (void)taosReleaseRef(tmqMgmt.rsetId, refId); } - tqDebugFlag = rsp.debugFlag; + tqClientDebug = rsp.debugFlag; tDestroySMqHbRsp(&rsp); END: @@ -2004,11 +2004,12 @@ static void tmqGetRawDataRowsPrecisionFromRes(void* pRetrieve, void** rawData, i } static void tmqBuildRspFromWrapperInner(SMqPollRspWrapper* pWrapper, SMqClientVg* pVg, int64_t* numOfRows, - SMqRspObj* pRspObj, SMqDataRsp* pDataRsp) { + SMqRspObj* pRspObj) { pRspObj->resIter = -1; pRspObj->resInfo.totalRows = 0; pRspObj->resInfo.precision = TSDB_TIME_PRECISION_MILLI; + SMqDataRsp* pDataRsp = &pRspObj->dataRsp; bool needTransformSchema = !pDataRsp->withSchema; if (!pDataRsp->withSchema) { // withSchema is false if subscribe subquery, true if subscribe db or stable pDataRsp->withSchema = true; @@ -2276,12 +2277,12 @@ static SMqRspObj* processMqRsp(tmq_t* tmq, SMqRspWrapper* pRspWrapper){ } pRspObj->resType = pRspWrapper->tmqRspType == TMQ_MSG_TYPE__POLL_DATA_RSP ? RES_TYPE__TMQ : RES_TYPE__TMQ_METADATA; int64_t numOfRows = 0; - tmqBuildRspFromWrapperInner(pollRspWrapper, pVg, &numOfRows, pRspObj, &pRspObj->dataRsp); + tmqBuildRspFromWrapperInner(pollRspWrapper, pVg, &numOfRows, pRspObj); tmq->totalRows += numOfRows; pVg->emptyBlockReceiveTs = 0; if (tmq->replayEnable) { pVg->blockReceiveTs = taosGetTimestampMs(); - pVg->blockSleepForReplay = pollRspWrapper->dataRsp.sleepTime; + pVg->blockSleepForReplay = pRspObj->dataRsp.sleepTime; if (pVg->blockSleepForReplay > 0) { if (taosTmrStart(tmqReplayTask, pVg->blockSleepForReplay, (void*)(tmq->refId), tmqMgmt.timer) == NULL) { tqErrorC("consumer:0x%" PRIx64 " failed to start replay timer, vgId:%d, sleep:%" PRId64, @@ -2291,7 +2292,7 @@ static SMqRspObj* processMqRsp(tmq_t* tmq, SMqRspWrapper* pRspWrapper){ } tqDebugC("consumer:0x%" PRIx64 " process poll rsp, vgId:%d, offset:%s, blocks:%d, rows:%" PRId64 ", vg total:%" PRId64 ", total:%" PRId64 ",QID:0x%" PRIx64, - tmq->consumerId, pVg->vgId, buf, pollRspWrapper->dataRsp.blockNum, numOfRows, pVg->numOfRows, tmq->totalRows, + tmq->consumerId, pVg->vgId, buf, pRspObj->dataRsp.blockNum, numOfRows, pVg->numOfRows, tmq->totalRows, pollRspWrapper->reqId); } } else if (pRspWrapper->tmqRspType == TMQ_MSG_TYPE__POLL_META_RSP || pRspWrapper->tmqRspType == TMQ_MSG_TYPE__POLL_BATCH_META_RSP) { diff --git a/source/common/src/tglobal.c b/source/common/src/tglobal.c index 5b67e1267b..6b0da468a2 100644 --- a/source/common/src/tglobal.c +++ b/source/common/src/tglobal.c @@ -541,6 +541,7 @@ static int32_t taosAddServerLogCfg(SConfig *pCfg) { TAOS_CHECK_RETURN(cfgAddInt32(pCfg, "sDebugFlag", sDebugFlag, 0, 255, CFG_SCOPE_SERVER, CFG_DYN_SERVER)); TAOS_CHECK_RETURN(cfgAddInt32(pCfg, "tsdbDebugFlag", tsdbDebugFlag, 0, 255, CFG_SCOPE_SERVER, CFG_DYN_SERVER)); TAOS_CHECK_RETURN(cfgAddInt32(pCfg, "tqDebugFlag", tqDebugFlag, 0, 255, CFG_SCOPE_SERVER, CFG_DYN_SERVER)); + TAOS_CHECK_RETURN(cfgAddInt32(pCfg, "tqClientDebug", tqClientDebug, 0, 255, CFG_SCOPE_SERVER, CFG_DYN_SERVER)); TAOS_CHECK_RETURN(cfgAddInt32(pCfg, "fsDebugFlag", fsDebugFlag, 0, 255, CFG_SCOPE_SERVER, CFG_DYN_SERVER)); TAOS_CHECK_RETURN(cfgAddInt32(pCfg, "udfDebugFlag", udfDebugFlag, 0, 255, CFG_SCOPE_SERVER, CFG_DYN_SERVER)); TAOS_CHECK_RETURN(cfgAddInt32(pCfg, "smaDebugFlag", smaDebugFlag, 0, 255, CFG_SCOPE_SERVER, CFG_DYN_SERVER)); @@ -1935,7 +1936,7 @@ static int32_t taosCfgDynamicOptionsForServer(SConfig *pCfg, const char *name) { {"smaDebugFlag", &smaDebugFlag}, {"idxDebugFlag", &idxDebugFlag}, {"tdbDebugFlag", &tdbDebugFlag}, {"tmrDebugFlag", &tmrDebugFlag}, {"uDebugFlag", &uDebugFlag}, {"smaDebugFlag", &smaDebugFlag}, {"rpcDebugFlag", &rpcDebugFlag}, {"qDebugFlag", &qDebugFlag}, {"metaDebugFlag", &metaDebugFlag}, - {"stDebugFlag", &stDebugFlag}, {"sndDebugFlag", &sndDebugFlag}, + {"stDebugFlag", &stDebugFlag}, {"sndDebugFlag", &sndDebugFlag}, {"tqClientDebug", &tqClientDebug}, }; static OptionNameAndVar options[] = {{"audit", &tsEnableAudit}, diff --git a/source/common/src/tmsg.c b/source/common/src/tmsg.c index bf4a91b224..38b793c0c9 100644 --- a/source/common/src/tmsg.c +++ b/source/common/src/tmsg.c @@ -10780,7 +10780,7 @@ _exit: int32_t tDecodeMqDataRsp(SDecoder *pDecoder, SMqDataRsp *pRsp) { TAOS_CHECK_RETURN(tDecodeMqDataRspCommon(pDecoder, pRsp)); if (!tDecodeIsEnd(pDecoder)) { - TAOS_CHECK_RETURN(tDecodeI64(pDecoder, &((SMqDataRsp *)pRsp)->sleepTime)); + TAOS_CHECK_RETURN(tDecodeI64(pDecoder, &pRsp->sleepTime)); } return 0; diff --git a/source/dnode/mnode/impl/src/mndConsumer.c b/source/dnode/mnode/impl/src/mndConsumer.c index d37ab090b5..1bffc016bd 100644 --- a/source/dnode/mnode/impl/src/mndConsumer.c +++ b/source/dnode/mnode/impl/src/mndConsumer.c @@ -244,7 +244,7 @@ static int32_t mndProcessMqHbReq(SRpcMsg *pMsg) { } storeOffsetRows(pMnode, &req, pConsumer); - rsp.debugFlag = tqDebugFlag; + rsp.debugFlag = tqClientDebug; code = buildMqHbRsp(pMsg, &rsp); END: diff --git a/source/util/src/tlog.c b/source/util/src/tlog.c index 49e1d87d80..66c49d9ac5 100644 --- a/source/util/src/tlog.c +++ b/source/util/src/tlog.c @@ -125,6 +125,8 @@ int32_t idxDebugFlag = 131; int32_t sndDebugFlag = 131; int32_t simDebugFlag = 131; +int32_t tqClientDebug = 0; + int64_t dbgEmptyW = 0; int64_t dbgWN = 0; int64_t dbgSmallWN = 0; From c9c5c24d85ce07a07acadf32c06720e162b724f7 Mon Sep 17 00:00:00 2001 From: wangmm0220 Date: Mon, 23 Sep 2024 10:27:38 +0800 Subject: [PATCH 12/51] fix:conflicts from 3.0 --- source/client/src/clientTmq.c | 72 +++++++++++++++++------------------ 1 file changed, 36 insertions(+), 36 deletions(-) diff --git a/source/client/src/clientTmq.c b/source/client/src/clientTmq.c index 8c44a8bfa0..975d14f3ee 100644 --- a/source/client/src/clientTmq.c +++ b/source/client/src/clientTmq.c @@ -1303,7 +1303,7 @@ static int32_t askEpCb(void* param, SDataBuf* pMsg, int32_t code) { if (code != 0) { tmqFreeRspWrapper((SMqRspWrapper*)pWrapper); taosFreeQitem(pWrapper); - tscError("consumer:0x%" PRIx64 " put ep res into mqueue failed, code:%d", tmq->consumerId, code); + tqErrorC("consumer:0x%" PRIx64 " put ep res into mqueue failed, code:%d", tmq->consumerId, code); } } } @@ -1312,7 +1312,7 @@ static int32_t askEpCb(void* param, SDataBuf* pMsg, int32_t code) { { int32_t ret = taosReleaseRef(tmqMgmt.rsetId, pParam->refId); if (ret != 0){ - tscError("failed to release ref:%"PRId64 ", code:%d", pParam->refId, ret); + tqErrorC("failed to release ref:%"PRId64 ", code:%d", pParam->refId, ret); } } @@ -1322,7 +1322,7 @@ static int32_t askEpCb(void* param, SDataBuf* pMsg, int32_t code) { if (pInfo) { pInfo->code = code; if (tsem2_post(&pInfo->sem) != 0){ - tscError("failed to post rsp sem askep cb"); + tqErrorC("failed to post rsp sem askep cb"); } } } @@ -1409,7 +1409,7 @@ void tmqHandleAllDelayedTask(tmq_t* pTmq) { return; } - tscDebug("consumer:0x%" PRIx64 " handle delayed %d tasks before poll data", pTmq->consumerId, numOfItems); + tqDebugC("consumer:0x%" PRIx64 " handle delayed %d tasks before poll data", pTmq->consumerId, numOfItems); int8_t* pTaskType = NULL; while (taosGetQitem(qall, (void**)&pTaskType) != 0) { if (*pTaskType == TMQ_DELAYED_TASK__ASK_EP) { @@ -1419,10 +1419,10 @@ void tmqHandleAllDelayedTask(tmq_t* pTmq) { tqErrorC("consumer:0x%" PRIx64 " failed to ask ep, code:%s", pTmq->consumerId, tstrerror(code)); continue; } - tscDebug("consumer:0x%" PRIx64 " retrieve ep from mnode in 1s", pTmq->consumerId); + tqDebugC("consumer:0x%" PRIx64 " retrieve ep from mnode in 1s", pTmq->consumerId); bool ret = taosTmrReset(tmqAssignAskEpTask, DEFAULT_ASKEP_INTERVAL, (void*)(pTmq->refId), tmqMgmt.timer, &pTmq->epTimer); - tscDebug("reset timer fo tmq ask ep:%d", ret); + tqDebugC("reset timer fo tmq ask ep:%d", ret); } else if (*pTaskType == TMQ_DELAYED_TASK__COMMIT) { tmq_commit_cb* pCallbackFn = (pTmq->commitCb != NULL) ? pTmq->commitCb : defaultCommitCbFn; asyncCommitAllOffsets(pTmq, pCallbackFn, pTmq->commitCbUserParam); @@ -1430,7 +1430,7 @@ void tmqHandleAllDelayedTask(tmq_t* pTmq) { pTmq->autoCommitInterval / 1000.0); bool ret = taosTmrReset(tmqAssignDelayedCommitTask, pTmq->autoCommitInterval, (void*)(pTmq->refId), tmqMgmt.timer, &pTmq->commitTimer); - tscDebug("reset timer fo commit:%d", ret); + tqDebugC("reset timer fo commit:%d", ret); } else { tqErrorC("consumer:0x%" PRIx64 " invalid task type:%d", pTmq->consumerId, *pTaskType); } @@ -1471,7 +1471,7 @@ int32_t tmqSubscribeCb(void* param, SDataBuf* pMsg, int32_t code) { pParam->rspErr = code; if (tsem2_post(&pParam->rspSem) != 0){ - tscError("failed to post sem, subscribe cb"); + tqErrorC("failed to post sem, subscribe cb"); } return 0; } @@ -1520,7 +1520,7 @@ void tmqFreeImpl(void* handle) { taosFreeQall(tmq->qall); if(tsem2_destroy(&tmq->rspSem) != 0) { - tscError("failed to destroy sem in free tmq"); + tqErrorC("failed to destroy sem in free tmq"); } taosArrayDestroyEx(tmq->clientTopics, freeClientTopic); @@ -1528,17 +1528,17 @@ void tmqFreeImpl(void* handle) { if (tmq->commitTimer) { if (!taosTmrStopA(&tmq->commitTimer)) { - tscError("failed to stop commit timer"); + tqErrorC("failed to stop commit timer"); } } if (tmq->epTimer) { if (!taosTmrStopA(&tmq->epTimer)) { - tscError("failed to stop ep timer"); + tqErrorC("failed to stop ep timer"); } } if (tmq->hbLiveTimer) { if (!taosTmrStopA(&tmq->hbLiveTimer)) { - tscError("failed to stop hb timer"); + tqErrorC("failed to stop hb timer"); } } taosMemoryFree(tmq); @@ -1723,13 +1723,13 @@ static int32_t syncAskEp(tmq_t* pTmq) { int32_t code = askEp(pTmq, pInfo, true, false); if (code == 0) { if (tsem2_wait(&pInfo->sem) != 0){ - tscError("consumer:0x%" PRIx64 ", failed to wait for sem", pTmq->consumerId); + tqErrorC("consumer:0x%" PRIx64 ", failed to wait for sem", pTmq->consumerId); } code = pInfo->code; } if(tsem2_destroy(&pInfo->sem) != 0) { - tscError("failed to destroy sem sync ask ep"); + tqErrorC("failed to destroy sem sync ask ep"); } taosMemoryFree(pInfo); return code; @@ -1842,10 +1842,10 @@ int32_t tmq_subscribe(tmq_t* tmq, const tmq_list_t* topic_list) { } if (tsem2_wait(¶m.rspSem) != 0){ - tscError("consumer:0x%" PRIx64 ", failed to wait semaphore in subscribe", tmq->consumerId); + tqErrorC("consumer:0x%" PRIx64 ", failed to wait semaphore in subscribe", tmq->consumerId); } if(tsem2_destroy(¶m.rspSem) != 0) { - tscError("consumer:0x%" PRIx64 ", failed to destroy semaphore in subscribe", tmq->consumerId); + tqErrorC("consumer:0x%" PRIx64 ", failed to destroy semaphore in subscribe", tmq->consumerId); } if (param.rspErr != 0) { @@ -2006,11 +2006,11 @@ END: if (tsem2_post(&tmq->rspSem) != 0){ - tscError("failed to post rsp sem, consumer:0x%" PRIx64, tmq->consumerId); + tqErrorC("failed to post rsp sem, consumer:0x%" PRIx64, tmq->consumerId); } ret = taosReleaseRef(tmqMgmt.rsetId, refId); if (ret != 0){ - tscError("failed to release ref:%"PRId64 ", code:%d", refId, ret); + tqErrorC("failed to release ref:%"PRId64 ", code:%d", refId, ret); } EXIT: @@ -2178,7 +2178,7 @@ static int32_t doTmqPollImpl(tmq_t* pTmq, SMqClientTopic* pTopic, SMqClientVg* p char offsetFormatBuf[TSDB_OFFSET_LEN] = {0}; tFormatOffset(offsetFormatBuf, tListLen(offsetFormatBuf), &pVg->offsetInfo.endOffset); code = asyncSendMsgToServer(pTmq->pTscObj->pAppInfo->pTransporter, &pVg->epSet, NULL, sendInfo); - tscDebug("consumer:0x%" PRIx64 " send poll to %s vgId:%d, code:%d, epoch %d, req:%s,QID:0x%" PRIx64, pTmq->consumerId, + tqDebugC("consumer:0x%" PRIx64 " send poll to %s vgId:%d, code:%d, epoch %d, req:%s,QID:0x%" PRIx64, pTmq->consumerId, pTopic->topicName, pVg->vgId, code, pTmq->epoch, offsetFormatBuf, req.reqId); if (code != 0) { return code; @@ -2535,7 +2535,7 @@ int32_t tmq_consumer_close(tmq_t* tmq) { atomic_store_8(&tmq->status, TMQ_CONSUMER_STATUS__CLOSED); code = taosRemoveRef(tmqMgmt.rsetId, tmq->refId); if (code != 0){ - tscError("tmq close failed to remove ref:%" PRId64 ", code:%d", tmq->refId, code); + tqErrorC("tmq close failed to remove ref:%" PRId64 ", code:%d", tmq->refId, code); } } return code; @@ -2678,7 +2678,7 @@ static void commitCallBackFn(tmq_t* UNUSED_PARAM(tmq), int32_t code, void* param SSyncCommitInfo* pInfo = (SSyncCommitInfo*)param; pInfo->code = code; if (tsem2_post(&pInfo->sem) != 0){ - tscError("failed to post rsp sem in commit cb"); + tqErrorC("failed to post rsp sem in commit cb"); } } @@ -2709,12 +2709,12 @@ int32_t tmq_commit_sync(tmq_t* tmq, const TAOS_RES* pRes) { } if (tsem2_wait(&pInfo->sem) != 0){ - tscError("failed to wait sem for sync commit"); + tqErrorC("failed to wait sem for sync commit"); } code = pInfo->code; if(tsem2_destroy(&pInfo->sem) != 0) { - tscError("failed to destroy sem for sync commit"); + tqErrorC("failed to destroy sem for sync commit"); } taosMemoryFree(pInfo); @@ -2781,14 +2781,14 @@ int32_t tmq_commit_offset_sync(tmq_t* tmq, const char* pTopicName, int32_t vgId, code = asyncCommitOffset(tmq, tname, vgId, &offsetVal, commitCallBackFn, pInfo); if (code == 0) { if (tsem2_wait(&pInfo->sem) != 0){ - tscError("failed to wait sem for sync commit offset"); + tqErrorC("failed to wait sem for sync commit offset"); } code = pInfo->code; } if (code == TSDB_CODE_TMQ_SAME_COMMITTED_VALUE) code = TSDB_CODE_SUCCESS; if(tsem2_destroy(&pInfo->sem) != 0) { - tscError("failed to destroy sem for sync commit offset"); + tqErrorC("failed to destroy sem for sync commit offset"); } taosMemoryFree(pInfo); @@ -2920,7 +2920,7 @@ END: pCommon->code = code; if (total == pParam->totalReq) { if (tsem2_post(&pCommon->rsp) != 0) { - tscError("failed to post semaphore in get wal cb"); + tqErrorC("failed to post semaphore in get wal cb"); } } @@ -2938,7 +2938,7 @@ static void destroyCommonInfo(SMqVgCommon* pCommon) { } taosArrayDestroy(pCommon->pList); if(tsem2_destroy(&pCommon->rsp) != 0) { - tscError("failed to destroy semaphore for topic:%s", pCommon->pTopicName); + tqErrorC("failed to destroy semaphore for topic:%s", pCommon->pTopicName); } (void)taosThreadMutexDestroy(&pCommon->mutex); taosMemoryFree(pCommon->pTopicName); @@ -2976,7 +2976,7 @@ end: } pParam->code = code; if (tsem2_post(&pParam->sem) != 0){ - tscError("failed to post semaphore in tmCommittedCb"); + tqErrorC("failed to post semaphore in tmCommittedCb"); } return code; } @@ -3042,14 +3042,14 @@ int64_t getCommittedFromServer(tmq_t* tmq, char* tname, int32_t vgId, SEpSet* ep code = asyncSendMsgToServer(tmq->pTscObj->pAppInfo->pTransporter, epSet, NULL, sendInfo); if (code != 0) { if(tsem2_destroy(&pParam->sem) != 0) { - tscError("failed to destroy semaphore in get committed from server1"); + tqErrorC("failed to destroy semaphore in get committed from server1"); } taosMemoryFree(pParam); return code; } if (tsem2_wait(&pParam->sem) != 0){ - tscError("failed to wait semaphore in get committed from server"); + tqErrorC("failed to wait semaphore in get committed from server"); } code = pParam->code; if (code == TSDB_CODE_SUCCESS) { @@ -3061,7 +3061,7 @@ int64_t getCommittedFromServer(tmq_t* tmq, char* tname, int32_t vgId, SEpSet* ep } } if(tsem2_destroy(&pParam->sem) != 0) { - tscError("failed to destroy semaphore in get committed from server2"); + tqErrorC("failed to destroy semaphore in get committed from server2"); } taosMemoryFree(pParam); @@ -3338,7 +3338,7 @@ int32_t tmq_get_topic_assignment(tmq_t* tmq, const char* pTopicName, tmq_topic_a } if (tsem2_wait(&pCommon->rsp) != 0){ - tscError("consumer:0x%" PRIx64 " failed to wait sem in get assignment", tmq->consumerId); + tqErrorC("consumer:0x%" PRIx64 " failed to wait sem in get assignment", tmq->consumerId); } code = pCommon->code; @@ -3403,7 +3403,7 @@ static int32_t tmqSeekCb(void* param, SDataBuf* pMsg, int32_t code) { SMqSeekParam* pParam = param; pParam->code = code; if (tsem2_post(&pParam->sem) != 0){ - tscError("failed to post sem in tmqSeekCb"); + tqErrorC("failed to post sem in tmqSeekCb"); } return 0; } @@ -3502,18 +3502,18 @@ int32_t tmq_offset_seek(tmq_t* tmq, const char* pTopicName, int32_t vgId, int64_ code = asyncSendMsgToServer(tmq->pTscObj->pAppInfo->pTransporter, &epSet, NULL, sendInfo); if (code != 0) { if(tsem2_destroy(&pParam->sem) != 0) { - tscError("consumer:0x%" PRIx64 "destroy rsp sem failed in seek offset", tmq->consumerId); + tqErrorC("consumer:0x%" PRIx64 "destroy rsp sem failed in seek offset", tmq->consumerId); } taosMemoryFree(pParam); return code; } if (tsem2_wait(&pParam->sem) != 0){ - tscError("consumer:0x%" PRIx64 "wait rsp sem failed in seek offset", tmq->consumerId); + tqErrorC("consumer:0x%" PRIx64 "wait rsp sem failed in seek offset", tmq->consumerId); } code = pParam->code; if(tsem2_destroy(&pParam->sem) != 0) { - tscError("consumer:0x%" PRIx64 "destroy rsp sem failed in seek offset", tmq->consumerId); + tqErrorC("consumer:0x%" PRIx64 "destroy rsp sem failed in seek offset", tmq->consumerId); } taosMemoryFree(pParam); From 6d0d2ae03930e4b5d33887072088818bbdfcb53c Mon Sep 17 00:00:00 2001 From: xsren <285808407@qq.com> Date: Sun, 29 Sep 2024 15:29:09 +0800 Subject: [PATCH 13/51] fix: ostime --- source/os/src/osTimezone.c | 22 ++++++++++++++++++++-- 1 file changed, 20 insertions(+), 2 deletions(-) diff --git a/source/os/src/osTimezone.c b/source/os/src/osTimezone.c index 89ced69f97..6e7c22c7f1 100644 --- a/source/os/src/osTimezone.c +++ b/source/os/src/osTimezone.c @@ -742,6 +742,20 @@ char *tz_win[554][2] = {{"Asia/Shanghai", "China Standard Time"}, static int isdst_now = 0; +void parseTimeStr(char *p, char to[5]) { + for (int i = 0; i < 5; ++i) { + if (strlen(p) > i) { + to[i] = p[i]; + } else { + to[i] = '0'; + } + } + if (strlen(p) == 2) { + to[1] = '0'; + to[2] = p[1]; + } +} + int32_t taosSetSystemTimezone(const char *inTimezoneStr, char *outTimezoneStr, int8_t *outDaylight, enum TdTimezone *tsTimezone) { if (inTimezoneStr == NULL || inTimezoneStr[0] == 0) { @@ -798,7 +812,9 @@ int32_t taosSetSystemTimezone(const char *inTimezoneStr, char *outTimezoneStr, i memcpy(&winStr[3], pp, ppp - pp); indexStr = ppp - pp + 3; } - sprintf(&winStr[indexStr], "%c%c%c:%c%c:00", (p[0] == '+' ? '+' : '-'), p[1], p[2], p[3], p[4]); + char to[5]; + parseTimeStr(p, to); + sprintf(&winStr[indexStr], "%c%c%c:%c%c:00", (to[0] == '+' ? '+' : '-'), to[1], to[2], to[3], to[4]); *tsTimezone = -taosStr2Int32(p, NULL, 10); } else { *tsTimezone = 0; @@ -806,7 +822,9 @@ int32_t taosSetSystemTimezone(const char *inTimezoneStr, char *outTimezoneStr, i } _putenv(winStr); _tzset(); - strcpy(outTimezoneStr, inTimezoneStr); + if (outTimezoneStr != inTimezoneStr) { + strcpy(outTimezoneStr, inTimezoneStr); + } *outDaylight = 0; #elif defined(_TD_DARWIN_64) From 5a85303c373441ef778f6a9f66de7b91b027a7cc Mon Sep 17 00:00:00 2001 From: xsren <285808407@qq.com> Date: Sun, 29 Sep 2024 23:40:19 +0800 Subject: [PATCH 14/51] fix: semphore leak --- source/libs/catalog/src/ctgCache.c | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/source/libs/catalog/src/ctgCache.c b/source/libs/catalog/src/ctgCache.c index eafd85a504..a5ed165118 100644 --- a/source/libs/catalog/src/ctgCache.c +++ b/source/libs/catalog/src/ctgCache.c @@ -906,7 +906,7 @@ int32_t ctgEnqueue(SCatalog *pCtg, SCtgCacheOperation *operation) { if (gCtgMgmt.queue.stopQueue) { ctgFreeQNode(node); CTG_UNLOCK(CTG_WRITE, &gCtgMgmt.queue.qlock); - CTG_RET(TSDB_CODE_CTG_EXIT); + CTG_ERR_JRET(TSDB_CODE_CTG_EXIT); } gCtgMgmt.queue.tail->next = node; @@ -924,7 +924,7 @@ int32_t ctgEnqueue(SCatalog *pCtg, SCtgCacheOperation *operation) { code = tsem_post(&gCtgMgmt.queue.reqSem); if (TSDB_CODE_SUCCESS != code) { qError("tsem_post failed, code:%x", code); - CTG_RET(code); + CTG_ERR_JRET(code); } if (syncOp) { @@ -935,9 +935,14 @@ int32_t ctgEnqueue(SCatalog *pCtg, SCtgCacheOperation *operation) { if (!operation->unLocked) { CTG_LOCK(CTG_READ, &gCtgMgmt.lock); } + tsem_destroy(&operation->rspSem); taosMemoryFree(operation); } +_return: + if (syncOp) { + tsem_destroy(&operation->rspSem); + } return code; } From 165875ac4f98f39a99a8ed5d01aab503bc5a8668 Mon Sep 17 00:00:00 2001 From: xsren <285808407@qq.com> Date: Mon, 30 Sep 2024 10:31:54 +0800 Subject: [PATCH 15/51] unused pointer --- source/libs/catalog/src/ctgCache.c | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/source/libs/catalog/src/ctgCache.c b/source/libs/catalog/src/ctgCache.c index a5ed165118..48b949cc28 100644 --- a/source/libs/catalog/src/ctgCache.c +++ b/source/libs/catalog/src/ctgCache.c @@ -935,13 +935,13 @@ int32_t ctgEnqueue(SCatalog *pCtg, SCtgCacheOperation *operation) { if (!operation->unLocked) { CTG_LOCK(CTG_READ, &gCtgMgmt.lock); } - tsem_destroy(&operation->rspSem); + TAOS_UNUSED(tsem_destroy(&operation->rspSem)); taosMemoryFree(operation); } _return: if (syncOp) { - tsem_destroy(&operation->rspSem); + TAOS_UNUSED(tsem_destroy(&operation->rspSem)); } return code; } From bac78203756c2ae39388ad74a2c50ba8054aa014 Mon Sep 17 00:00:00 2001 From: xsren <285808407@qq.com> Date: Mon, 30 Sep 2024 17:48:30 +0800 Subject: [PATCH 16/51] fix: free error --- source/libs/catalog/src/ctgCache.c | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/source/libs/catalog/src/ctgCache.c b/source/libs/catalog/src/ctgCache.c index 48b949cc28..ef8399d723 100644 --- a/source/libs/catalog/src/ctgCache.c +++ b/source/libs/catalog/src/ctgCache.c @@ -936,11 +936,12 @@ int32_t ctgEnqueue(SCatalog *pCtg, SCtgCacheOperation *operation) { CTG_LOCK(CTG_READ, &gCtgMgmt.lock); } TAOS_UNUSED(tsem_destroy(&operation->rspSem)); - taosMemoryFree(operation); + taosMemoryFreeClear(operation); } + return code; _return: - if (syncOp) { + if (syncOp && operation) { TAOS_UNUSED(tsem_destroy(&operation->rspSem)); } return code; From c6ad2a1e21c37554b0087772e1bb3aea2ddeaed1 Mon Sep 17 00:00:00 2001 From: wangmeng Date: Mon, 30 Sep 2024 17:48:42 +0800 Subject: [PATCH 17/51] fix:[TD-32198] add a test case to verify HAVING key --- tests/army/query/test_having.py | 378 ++++++++++++++++++++++++++++++++ tests/parallel_test/cases.task | 1 + 2 files changed, 379 insertions(+) create mode 100644 tests/army/query/test_having.py diff --git a/tests/army/query/test_having.py b/tests/army/query/test_having.py new file mode 100644 index 0000000000..ff8f6a1c1d --- /dev/null +++ b/tests/army/query/test_having.py @@ -0,0 +1,378 @@ +from frame.log import * +from frame.cases import * +from frame.sql import * +from frame.caseBase import * +from frame import * +from frame.eos import * +import random +import string + +""" + TD-32198: https://jira.taosdata.com:18080/browse/TD-32198 + Having关键字的专项测试,主要覆盖以下 4 种场景: + 1、普通聚合查询 + 2、关联查询 + 3、窗口切分查询 + 4、流计算中的窗口切分查询 +""" + + +class TDTestCase(TBase): + + def init(self, conn, logSql, replicaVar=1): + self.replicaVar = int(replicaVar) + tdLog.debug("start to execute %s" % __file__) + tdSql.init(conn.cursor()) + + def prepare_global_data(self): + tdSql.execute("DROP DATABASE IF EXISTS db_td32198;") + tdSql.execute("create database db_td32198;") + tdSql.execute("use db_td32198;") + + def prepare_agg_data(self): + # database for case TD-32198 + # super table + tdSql.execute("DROP STABLE IF EXISTS meters") + tdSql.execute("CREATE STABLE `meters` (`ts` TIMESTAMP , `current` FLOAT , `voltage` INT , `phase` FLOAT ) \ + TAGS (`groupid` TINYINT, `location` VARCHAR(16));") + + # child table + tdSql.execute("CREATE TABLE `ct_1` USING `meters` (`groupid`, `location`) TAGS (1, 'beijing');") + # tdSql.execute("CREATE TABLE `ct_2` USING `meters` (`groupid`, `location`) TAGS (2, 'shanghai');") + + data = [ + ('2020-06-01 00:00:00.000', 0.1000000, 12, 0.0200000), + ('2020-06-01 00:15:00.000', 0.3614670, 18, 0.1071560), + ('2020-06-01 00:30:00.000', 0.5209450, 18, 0.1736480), + ('2020-06-01 00:45:00.000', 0.8764570, 18, 0.2588190), + ('2020-06-01 01:00:00.000', 1.0260600, 14, 0.3620200), + ('2020-06-01 01:15:00.000', 1.3678550, 0, 0.4226180), + ('2020-06-01 01:30:00.000', 1.6000000, 12, 0.5200000), + ('2020-06-01 01:45:00.000', 1.8207290, 2, 0.5835760), + ('2020-06-01 02:00:00.000', 1.9283630, 18, 0.6527880), + ('2020-06-01 02:15:00.000', 2.1213200, 18, 0.7271070), + ('2020-06-01 02:30:00.000', 2.3981330, 12, 0.7760440), + ('2020-06-01 02:45:00.000', 2.4574561, 14, 0.8291520), + ('2020-06-01 03:00:00.000', 2.6980760, 14, 0.8760250), + ('2020-06-01 03:15:00.000', 2.8189230, 10, 0.9063080), + ('2020-06-01 03:30:00.000', 2.8190780, 6, 0.9396930), + ('2020-06-01 03:45:00.000', 2.8977780, 10, 0.9859260), + ('2020-06-01 04:00:00.000', 2.9544230, 4, 1.0048079), + ('2020-06-01 04:15:00.000', 2.9885840, 14, 1.0061949), + ('2020-06-01 04:30:00.000', 3.0999999, 6, 1.0200000), + ('2020-06-01 04:45:00.000', 3.0885839, 10, 1.0161951), + ('2020-06-01 05:00:00.000', 2.9544230, 18, 0.9848080), + ('2020-06-01 05:15:00.000', 2.9977770, 2, 0.9859260), + ('2020-06-01 05:30:00.000', 2.8190780, 0, 0.9496930), + ('2020-06-01 05:45:00.000', 2.7189231, 18, 0.9163080), + ('2020-06-01 06:00:00.000', 2.5980761, 10, 0.8860250) + ] + + sql = "insert into ct_1 values"; + for t in data: + sql += "('{}', {}, {}, {}),".format(t[0], t[1], t[2], t[3]) + sql += ";" + tdSql.execute(sql) + tdLog.debug("sql: %s" % sql) + + def test_agg_having(self): + tdSql.query("select voltage, sum(voltage), count(*) from ct_1 group by voltage;") + tdSql.checkRows(8) + tdSql.checkData(7, 2, 7) + tdSql.checkData(7, 1, 126) + + tdSql.query("select voltage, sum(voltage), count(*) from ct_1 group by voltage having count(voltage)>=4;"); + tdSql.checkRows(3) + tdSql.checkData(2, 2, 7) + tdSql.checkData(2, 1, 126) + + tdSql.query("select voltage, sum(voltage), count(*) from ct_1 group by voltage having count(current)>=4;"); + tdSql.checkRows(3) + tdSql.checkData(2, 2, 7) + tdSql.checkData(2, 1, 126) + + tdSql.query("select voltage, sum(voltage), count(*) from ct_1 group by voltage having voltage >=14;"); + tdSql.checkRows(2) + tdSql.checkData(0, 2, 4) + tdSql.checkData(1, 1, 126) + + tdSql.error("select voltage, count(*) from ct_1 group by voltage having current >1.0260600;"); + + def prepare_join_data(self): + # super table + tdSql.execute("DROP STABLE IF EXISTS meters") + tdSql.execute("CREATE STABLE `meters` (`ts` TIMESTAMP , `current` FLOAT , `voltage` INT , `phase` FLOAT ) \ + TAGS (`groupid` TINYINT, `location` VARCHAR(16));") + + # child table + tdSql.execute("CREATE TABLE `ct_join_1` USING `meters` (`groupid`, `location`) TAGS (1, 'beijing');") + tdSql.execute("CREATE TABLE `ct_join_2` USING `meters` (`groupid`, `location`) TAGS (2, 'shanghai');") + + # insert data for ts4806 + data_join_1 = [ + ('2020-06-01 00:00:00.000', 0.1000000, 12, 0.0200000), + ('2020-06-01 00:10:00.000', 1.2278550, 9, 0.4226180), + ('2020-06-01 00:15:00.000', 0.3614670, 18, 0.1071560), + ('2020-06-01 00:30:00.000', 0.5209450, 18, 0.1736480), + ('2020-06-01 00:40:00.000', 1.5230000, 10, 0.5200000), + ('2020-06-01 00:45:00.000', 0.8764570, 18, 0.2588190), + ('2020-06-01 00:50:00.000', 1.6507290, 11, 0.5835760), + ('2020-06-01 01:00:00.000', 1.0260600, 14, 0.3620200), + ('2020-06-01 01:15:00.000', 1.3678550, 0, 0.4226180), + ('2020-06-01 01:20:00.000', 1.1213200, 13, 0.7271070), + ('2020-06-01 01:30:00.000', 1.6000000, 12, 0.5200000), + ('2020-06-01 01:45:00.000', 1.8207290, 2, 0.5835760), + ('2020-06-01 02:00:00.000', 1.9283630, 18, 0.6527880), + ('2020-06-01 02:05:00.000', 0.9283630, 6, 0.6527880), + ('2020-06-01 02:15:00.000', 2.1213200, 18, 0.7271070) + ] + + data_join_2 = [ + ('2020-06-01 00:00:00.000', 0.3614670, 9, 0.0200000), + ('2020-06-01 00:15:00.000', 0.1000000, 12, 0.1071560), + ('2020-06-01 00:30:00.000', 0.5209450, 15, 0.1736480), + ('2020-06-01 00:45:00.000', 0.8764570, 18, 0.2588190), + ('2020-06-01 01:00:00.000', 1.0260600, 15, 0.3620200), + ('2020-06-01 01:15:00.000', 1.3678550, 7, 0.4226180), + ('2020-06-01 01:30:00.000', 1.6000000, 12, 0.5200000), + ('2020-06-01 01:45:00.000', 1.8207290, 7, 0.5835760), + ('2020-06-01 02:00:00.000', 1.0260600, 13, 0.6527880), + ('2020-06-01 02:15:00.000', 0.5209450, 18, 0.7271070) + ] + + sql = "insert into ct_join_1 values"; + for t in data_join_1: + sql += "('{}', {}, {}, {}),".format(t[0], t[1], t[2], t[3]) + sql += ";" + tdSql.execute(sql) + tdLog.debug("ct_join_1 sql: %s" % sql) + + sql = "insert into ct_join_2 values"; + for t in data_join_2: + sql += "('{}', {}, {}, {}),".format(t[0], t[1], t[2], t[3]) + sql += ";" + tdSql.execute(sql) + tdLog.debug("ct_join_2 sql: %s" % sql) + + def test_join_having(self): + tdSql.query("SELECT a.voltage, count(*) FROM ct_join_1 a JOIN ct_join_2 b ON a.ts = b.ts \ + group by a.voltage having count(*) > 4;") + tdSql.checkRows(1) + tdSql.checkData(0, 1, 5) + tdSql.checkData(0, 0, 18) + + tdSql.error("SELECT a.voltage, count(*) FROM ct_join_1 a JOIN ct_join_2 b ON a.ts = b.ts \ + group by a.voltage having b.voltage > 14;") + + tdSql.query("SELECT a.voltage, count(*) FROM ct_join_1 a left JOIN ct_join_2 b ON a.ts = b.ts \ + group by a.voltage having count(*) > 4;"); + tdSql.checkRows(1) + tdSql.checkData(0, 1, 5) + tdSql.checkData(0, 0, 18) + + tdSql.error("SELECT a.voltage, count(*) FROM ct_join_1 a left JOIN ct_join_2 b ON a.ts = b.ts \ + group by a.voltage having b.voltage > 14;"); + + tdSql.query("SELECT a.ts, a.voltage, avg(b.voltage) FROM ct_join_2 a LEFT WINDOW JOIN ct_join_1 b \ + WINDOW_OFFSET(-15m, 15m) where a.voltage >=18 and b.voltage > 11 having avg(b.voltage) > 17;"); + tdSql.checkRows(1) + + tdSql.error("SELECT a.ts, a.voltage, avg(b.voltage) FROM ct_join_2 a LEFT WINDOW JOIN ct_join_1 b \ + WINDOW_OFFSET(-15m, 15m) where a.voltage >=18 and b.voltage > 11 having b.voltage > 17;"); + + def prepare_window_data(self): + # super table + tdSql.execute("DROP STABLE IF EXISTS meters") + tdSql.execute("CREATE STABLE `meters` (`ts` TIMESTAMP , `current` FLOAT , `voltage` INT , `phase` FLOAT ) \ + TAGS (`groupid` TINYINT, `location` VARCHAR(16));") + + # child table + tdSql.execute("CREATE TABLE `ct_win` USING `meters` (`groupid`, `location`) TAGS (1, 'beijing');") + + # insert data for ts4806 + data_win = [ + ('2020-06-01 00:00:00.000', 0.1000000, 12, 0.0200000), + ('2020-06-01 00:10:00.000', 1.2278550, 9, 0.4226180), + ('2020-06-01 00:15:00.000', 0.3614670, 18, 0.1071560), + ('2020-06-01 00:30:00.000', 0.5209450, 18, 0.1736480), + ('2020-06-01 00:40:00.000', 1.5230000, 18, 0.5200000), + ('2020-06-01 00:45:00.000', 0.8764570, 18, 0.2588190), + ('2020-06-01 00:50:00.000', 1.6507290, 11, 0.5835760), + ('2020-06-01 01:00:00.000', 1.0260600, 14, 0.3620200), + ('2020-06-01 01:15:00.000', 1.3678550, 14, 0.4226180), + ('2020-06-01 01:20:00.000', 1.1213200, 13, 0.7271070), + ('2020-06-01 01:30:00.000', 1.6000000, 12, 0.5200000), + ('2020-06-01 01:45:00.000', 1.8207290, 12, 0.5835760), + ('2020-06-01 02:00:00.000', 1.9283630, 18, 0.6527880), + ('2020-06-01 02:05:00.000', 0.9283630, 18, 0.6527880), + ('2020-06-01 02:15:00.000', 2.1213200, 18, 0.7271070) + ] + + sql = "insert into ct_win values"; + for t in data_win: + sql += "('{}', {}, {}, {}),".format(t[0], t[1], t[2], t[3]) + sql += ";" + tdSql.execute(sql) + tdLog.debug("data_win sql: %s" % sql) + + def test_window_having(self): + tdSql.query("SELECT _WSTART, _WEND, COUNT(*) FROM ct_win INTERVAL(15m) having count(*) > 1;") + tdSql.checkRows(5) + tdSql.checkData(0, 2, 2) + + tdSql.error("SELECT _WSTART, _WEND, COUNT(*) FROM ct_win INTERVAL(15m) having voltage > 12;"); + + tdSql.query("SELECT _wstart, _wend, COUNT(*) AS cnt, FIRST(ts) AS fst, voltage FROM ct_win \ + STATE_WINDOW(voltage) having count(*) > 3;"); + tdSql.checkRows(1) + tdSql.checkData(0, 2, 4) + + tdSql.error("SELECT _wstart, _wend, COUNT(*) AS cnt, FIRST(ts) AS fst, voltage FROM ct_win \ + STATE_WINDOW(voltage) having phase > 0.26;"); + + tdSql.query("SELECT _wstart, _wend, COUNT(*), FIRST(ts) FROM ct_win SESSION(ts, 10m) having count(*) > 3;"); + tdSql.checkRows(1) + tdSql.checkData(0, 2, 5) + + tdSql.error("SELECT _wstart, _wend, COUNT(*), FIRST(ts) FROM ct_win SESSION(ts, 10m) having voltage > 12;"); + + tdSql.query("select _wstart, _wend, count(*), first(voltage), last(voltage) from ct_win \ + event_window start with voltage <= 12 end with voltage >= 17 having count(*) > 3;"); + tdSql.checkRows(1) + tdSql.checkData(0, 2, 7) + tdSql.checkData(0, 3, 11) + tdSql.checkData(0, 4, 18) + + tdSql.error("select _wstart, _wend, count(*) from ct_win \ + event_window start with voltage <=12 end with voltage >= 17 having phase > 0.2;"); + + tdSql.query( + "select _wstart, _wend, count(*), sum(voltage) from ct_win count_window(4) having sum(voltage) > 57;"); + tdSql.checkRows(1) + tdSql.checkData(0, 2, 4) + tdSql.checkData(0, 3, 61) + + tdSql.error("select _wstart, _wend, count(*), sum(voltage) from ct_win count_window(4) having voltage > 12;"); + + + def prepare_stream_window_data(self): + # super table + tdSql.execute("DROP STABLE IF EXISTS meters") + tdSql.execute("CREATE STABLE `meters` (`ts` TIMESTAMP , `current` FLOAT , `voltage` INT , `phase` FLOAT ) \ + TAGS (`groupid` TINYINT, `location` VARCHAR(16));") + + # child table + tdSql.execute("CREATE TABLE `ct_steam_win` USING `meters` (`groupid`, `location`) TAGS (1, 'beijing');") + + # insert data for ts4806 + data_win = [ + ('2020-06-01 00:00:00.000', 0.1000000, 12, 0.0200000), + ('2020-06-01 00:10:00.000', 1.2278550, 9, 0.4226180), + ('2020-06-01 00:15:00.000', 0.3614670, 18, 0.1071560), + ('2020-06-01 00:30:00.000', 0.5209450, 18, 0.1736480), + ('2020-06-01 00:40:00.000', 1.5230000, 18, 0.5200000), + ('2020-06-01 00:45:00.000', 0.8764570, 18, 0.2588190), + ('2020-06-01 00:50:00.000', 1.6507290, 11, 0.5835760), + ('2020-06-01 01:00:00.000', 1.0260600, 14, 0.3620200), + ('2020-06-01 01:15:00.000', 1.3678550, 14, 0.4226180), + ('2020-06-01 01:20:00.000', 1.1213200, 13, 0.7271070), + ('2020-06-01 01:30:00.000', 1.6000000, 12, 0.5200000), + ('2020-06-01 01:45:00.000', 1.8207290, 12, 0.5835760), + ('2020-06-01 02:00:00.000', 1.9283630, 18, 0.6527880), + ('2020-06-01 02:05:00.000', 0.9283630, 18, 0.6527880), + ('2020-06-01 02:15:00.000', 2.1213200, 18, 0.7271070) + ] + + sql = "insert into ct_win values"; + for t in data_win: + sql += "('{}', {}, {}, {}),".format(t[0], t[1], t[2], t[3]) + sql += ";" + tdSql.execute(sql) + tdLog.debug("data_win sql: %s" % sql) + + # 支持会话窗口、状态窗口、滑动窗口、事件窗口和计数窗口, + # 其中,状态窗口、事件窗口 和 计数窗口 搭配超级表时必须与 partition by tbname 一起使用 + def test_stream_window_having(self): + tdSql.execute("CREATE STREAM streams0 fill_history 1 INTO streamt0 AS \ + SELECT _WSTART, _WEND, COUNT(*) FROM meters PARTITION BY tbname INTERVAL(15m) having count(*) > 1;") + tdSql.query("select * from streamt0;"); + tdSql.checkRows(5) + tdSql.checkData(0, 2, 2) + + tdSql.error("CREATE STREAM streams10 fill_history 1 INTO streamt10 AS SELECT _WSTART, _WEND, COUNT(*) \ + FROM meters PARTITION BY tbname INTERVAL(15m) having voltage > 12;"); + + + tdSql.execute("CREATE STREAM streams1 fill_history 1 INTO streamt1 AS \ + SELECT _wstart, _wend, COUNT(*) AS cnt, FIRST(ts) AS fst, voltage FROM meters PARTITION BY tbname \ + STATE_WINDOW(voltage) having count(*) > 3;"); + tdSql.query("select * from streamt1;"); + tdSql.checkRows(1) + tdSql.checkData(0, 2, 4) + + tdSql.error("CREATE STREAM streams11 fill_history 1 INTO streamt11 AS \ + SELECT _wstart, _wend, COUNT(*) AS cnt, FIRST(ts) AS fst, voltage FROM meters PARTITION BY tbname \ + STATE_WINDOW(voltage) having phase > 0.26;"); + + + tdSql.execute("CREATE STREAM streams2 fill_history 1 INTO streamt2 AS \ + SELECT _wstart, _wend, COUNT(*), FIRST(ts) FROM meters SESSION(ts, 10m) having count(*) > 3;"); + tdSql.query("select * from streamt2;"); + tdSql.checkRows(1) + tdSql.checkData(0, 2, 5) + + tdSql.error("CREATE STREAM streams12 fill_history 1 INTO streamt12 AS \ + SELECT _wstart, _wend, COUNT(*), FIRST(ts) FROM meters SESSION(ts, 10m) having voltage > 12;"); + + tdSql.execute("CREATE STREAM streams3 fill_history 1 INTO streamt3 AS \ + select _wstart, _wend, count(*), first(voltage), last(voltage) from meters PARTITION BY tbname \ + event_window start with voltage <= 12 end with voltage >= 17 having count(*) > 3;"); + tdSql.query("select * from streamt3;"); + tdSql.checkRows(1) + tdSql.checkData(0, 2, 7) + tdSql.checkData(0, 3, 11) + tdSql.checkData(0, 4, 18) + + tdSql.error("CREATE STREAM streams13 fill_history 1 INTO streamt13 AS \ + select _wstart, _wend, count(*), first(voltage), last(voltage) from meters PARTITION BY tbname \ + event_window start with voltage <= 12 end with voltage >= 17 having phase > 0.2;"); + + tdSql.execute("CREATE STREAM streams4 fill_history 1 INTO streamt4 AS \ + select _wstart, _wend, count(*), sum(voltage) from meters PARTITION BY tbname \ + count_window(4) having sum(voltage) > 57;"); + tdSql.query("select * from streamt4;"); + tdSql.checkRows(1) + tdSql.checkData(0, 2, 4) + tdSql.checkData(0, 3, 61) + + tdSql.error("CREATE STREAM streams14 fill_history 1 INTO streamt14 AS \ + select _wstart, _wend, count(*), sum(voltage) from meters PARTITION BY tbname \ + count_window(4) having voltage > 12;"); + + + + def run(self): + self.prepare_global_data() + + self.prepare_agg_data() + self.test_agg_having() + + self.prepare_join_data() + self.test_join_having() + + self.prepare_window_data() + self.test_window_having() + + ''' + self.prepare_stream_window_data() + self.test_stream_window_having() + ''' + + def stop(self): + tdSql.execute("drop database db_td32198;") + tdSql.close() + tdLog.success("%s successfully executed" % __file__) + + +tdCases.addWindows(__file__, TDTestCase()) +tdCases.addLinux(__file__, TDTestCase()) diff --git a/tests/parallel_test/cases.task b/tests/parallel_test/cases.task index 72b470f509..53f3317ca9 100644 --- a/tests/parallel_test/cases.task +++ b/tests/parallel_test/cases.task @@ -46,6 +46,7 @@ ,,y,army,./pytest.sh python3 ./test.py -f query/last/test_last.py ,,y,army,./pytest.sh python3 ./test.py -f query/window/base.py ,,y,army,./pytest.sh python3 ./test.py -f query/sys/tb_perf_queries_exist_test.py -N 3 +,,y,army,./pytest.sh python3 ./test.py -f query/test_having.py # # system test From c150465da22495d62e6932153a85704ea2ad3cf4 Mon Sep 17 00:00:00 2001 From: Ping Xiao Date: Tue, 8 Oct 2024 06:38:56 +0800 Subject: [PATCH 18/51] docs: release ver-3.3.3.0 --- cmake/cmake.version | 2 +- docs/en/28-releases/01-tdengine.md | 4 ++++ docs/zh/28-releases/01-tdengine.md | 4 ++++ 3 files changed, 9 insertions(+), 1 deletion(-) diff --git a/cmake/cmake.version b/cmake/cmake.version index 3bb764612e..c600c084fd 100644 --- a/cmake/cmake.version +++ b/cmake/cmake.version @@ -2,7 +2,7 @@ IF (DEFINED VERNUMBER) SET(TD_VER_NUMBER ${VERNUMBER}) ELSE () - SET(TD_VER_NUMBER "3.3.3.0.alpha") + SET(TD_VER_NUMBER "3.3.4.0.alpha") ENDIF () IF (DEFINED VERCOMPATIBLE) diff --git a/docs/en/28-releases/01-tdengine.md b/docs/en/28-releases/01-tdengine.md index a6e157cf74..486fe2c015 100644 --- a/docs/en/28-releases/01-tdengine.md +++ b/docs/en/28-releases/01-tdengine.md @@ -20,6 +20,10 @@ For TDengine 2.x installation packages by version, please visit [here](https://t import Release from "/components/ReleaseV3"; +## 3.3.3.0 + + + ## 3.3.2.0 diff --git a/docs/zh/28-releases/01-tdengine.md b/docs/zh/28-releases/01-tdengine.md index 5b3abcb341..0f9ceada50 100644 --- a/docs/zh/28-releases/01-tdengine.md +++ b/docs/zh/28-releases/01-tdengine.md @@ -24,6 +24,10 @@ TDengine 3.x 各版本安装包下载链接如下: import Release from "/components/ReleaseV3"; +## 3.3.3.0 + + + ## 3.3.2.0 From 65f268929358724634d1f72010e209954d65d7e1 Mon Sep 17 00:00:00 2001 From: wangmm0220 Date: Tue, 8 Oct 2024 13:48:22 +0800 Subject: [PATCH 19/51] fix:[TD-30270]conflicts from 3.0 --- source/client/src/clientRawBlockWrite.c | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/source/client/src/clientRawBlockWrite.c b/source/client/src/clientRawBlockWrite.c index 9a3d5625ce..efd8836be4 100644 --- a/source/client/src/clientRawBlockWrite.c +++ b/source/client/src/clientRawBlockWrite.c @@ -1814,7 +1814,7 @@ end: return code; } -static int32_t buildCreateTbMap(STaosxRsp* rsp, SHashObj* pHashObj) { +static int32_t buildCreateTbMap(SMqDataRsp* rsp, SHashObj* pHashObj) { // find schema data info int32_t code = 0; SVCreateTbReq pCreateReq = {0}; @@ -1905,7 +1905,7 @@ static int32_t tmqWriteRawMetaDataImpl(TAOS* taos, void* data, int32_t dataLen) RAW_NULL_CHECK(pVgHash); pCreateTbHash = taosHashInit(16, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY), true, HASH_NO_LOCK); RAW_NULL_CHECK(pCreateTbHash); - RAW_RETURN_CHECK(buildCreateTbMap(&rspObj.rsp, pCreateTbHash)); + RAW_RETURN_CHECK(buildCreateTbMap(&rspObj.dataRsp, pCreateTbHash)); uDebug(LOG_ID_TAG " write raw metadata block num:%d", LOG_ID_VALUE, rspObj.dataRsp.blockNum); while (++rspObj.resIter < rspObj.dataRsp.blockNum) { @@ -1991,7 +1991,7 @@ end: pIter = taosHashIterate(pCreateTbHash, pIter); } taosHashCleanup(pCreateTbHash); - tDeleteSTaosxRsp(&rspObj.rsp); + tDeleteSTaosxRsp(&rspObj.dataRsp); tDecoderClear(&decoder); qDestroyQuery(pQuery); destroyRequest(pRequest); From 7a0d49a747a8a3e8c3cea01d3f9a86cc74e6cef1 Mon Sep 17 00:00:00 2001 From: sheyanjie-qq <249478495@qq.com> Date: Tue, 8 Oct 2024 14:01:07 +0800 Subject: [PATCH 20/51] fix taos/taosws dll path issue --- docs/en/14-reference/05-connectors/10-cpp.mdx | 2 +- docs/zh/14-reference/05-connector/10-cpp.mdx | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/14-reference/05-connectors/10-cpp.mdx b/docs/en/14-reference/05-connectors/10-cpp.mdx index 6a570b2490..ca32660ac7 100644 --- a/docs/en/14-reference/05-connectors/10-cpp.mdx +++ b/docs/en/14-reference/05-connectors/10-cpp.mdx @@ -19,7 +19,7 @@ After TDengine server or client installation, `taos.h` is located at The dynamic libraries for the TDengine client driver are located in. - Linux: `/usr/local/taos/driver/libtaos.so` -- Windows: `C:\TDengine\taos.dll` +- Windows: `C:\TDengine\driver\taos.dll` - macOS: `/usr/local/lib/libtaos.dylib` ## Supported platforms diff --git a/docs/zh/14-reference/05-connector/10-cpp.mdx b/docs/zh/14-reference/05-connector/10-cpp.mdx index 0df6ed924c..c618601fb9 100644 --- a/docs/zh/14-reference/05-connector/10-cpp.mdx +++ b/docs/zh/14-reference/05-connector/10-cpp.mdx @@ -27,7 +27,7 @@ TDengine 服务端或客户端安装后,`taosws.h` 位于: TDengine 客户端驱动的动态库位于: - Linux: `/usr/local/taos/driver/libtaosws.so` -- Windows: `C:\TDengine\taosws.dll` +- Windows: `C:\TDengine\driver\taosws.dll` - macOS: `/usr/local/lib/libtaosws.dylib` ### 支持的平台 @@ -626,7 +626,7 @@ TDengine 服务端或客户端安装后,`taos.h` 位于: TDengine 客户端驱动的动态库位于: - Linux: `/usr/local/taos/driver/libtaos.so` -- Windows: `C:\TDengine\taos.dll` +- Windows: `C:\TDengine\driver\taos.dll` - macOS: `/usr/local/lib/libtaos.dylib` ### 支持的平台 From 7f2a2cfbd07ded9de79f61c621c2b4f1f969787c Mon Sep 17 00:00:00 2001 From: xsren <285808407@qq.com> Date: Tue, 8 Oct 2024 14:25:51 +0800 Subject: [PATCH 21/51] enh: safe function --- include/os/osString.h | 1 + source/client/src/clientRawBlockWrite.c | 4 +- source/libs/command/src/command.c | 163 ++++++++++++++++-------- source/libs/function/src/tudf.c | 7 +- source/libs/function/src/udfd.c | 2 +- source/libs/qcom/src/queryUtil.c | 26 ++-- source/libs/qcom/src/querymsg.c | 16 +-- 7 files changed, 137 insertions(+), 82 deletions(-) diff --git a/include/os/osString.h b/include/os/osString.h index 30bfd61b62..b184e7efdb 100644 --- a/include/os/osString.h +++ b/include/os/osString.h @@ -68,6 +68,7 @@ typedef enum { M2C = 0, C2M } ConvType; #define TAOS_STRCPY(_dst, _src) ((void)strcpy(_dst, _src)) #define TAOS_STRNCPY(_dst, _src, _size) ((void)strncpy(_dst, _src, _size)) #define TAOS_STRCAT(_dst, _src) ((void)strcat(_dst, _src)) +#define TAOS_STRNCAT(_dst, _src, len) ((void)strncat(_dst, _src, len)) char *tstrdup(const char *src); int32_t taosUcs4len(TdUcs4 *ucs4); diff --git a/source/client/src/clientRawBlockWrite.c b/source/client/src/clientRawBlockWrite.c index 8ce4685716..d2578ac8e0 100644 --- a/source/client/src/clientRawBlockWrite.c +++ b/source/client/src/clientRawBlockWrite.c @@ -460,8 +460,10 @@ static void buildChildElement(cJSON* json, SVCreateTbReq* pCreateReq) { char* buf = NULL; if (pTagVal->type == TSDB_DATA_TYPE_VARBINARY) { buf = taosMemoryCalloc(pTagVal->nData * 2 + 2 + 3, 1); - } else { + } else if (IS_VAR_DATA_TYPE(pTagVal->type)) { buf = taosMemoryCalloc(pTagVal->nData + 3, 1); + } else { + buf = taosMemoryCalloc(32, 1); } RAW_NULL_CHECK(buf); diff --git a/source/libs/command/src/command.c b/source/libs/command/src/command.c index d24f830ea7..28ff8e7436 100644 --- a/source/libs/command/src/command.c +++ b/source/libs/command/src/command.c @@ -291,7 +291,8 @@ static int32_t buildRetension(SArray* pRetension, char** ppRetentions) { return TSDB_CODE_SUCCESS; } - char* p1 = taosMemoryCalloc(1, 100); + const int lMaxLen = 128; + char* p1 = taosMemoryCalloc(1, lMaxLen); if (NULL == p1) { return terrno; } @@ -302,13 +303,13 @@ static int32_t buildRetension(SArray* pRetension, char** ppRetentions) { int64_t v1 = getValOfDiffPrecision(p->freqUnit, p->freq); int64_t v2 = getValOfDiffPrecision(p->keepUnit, p->keep); if (i == 0) { - len += sprintf(p1 + len, "-:%" PRId64 "%c", v2, p->keepUnit); + len += snprintf(p1 + len, lMaxLen - len, "-:%" PRId64 "%c", v2, p->keepUnit); } else { - len += sprintf(p1 + len, "%" PRId64 "%c:%" PRId64 "%c", v1, p->freqUnit, v2, p->keepUnit); + len += snprintf(p1 + len, lMaxLen - len, "%" PRId64 "%c:%" PRId64 "%c", v1, p->freqUnit, v2, p->keepUnit); } if (i < size - 1) { - len += sprintf(p1 + len, ","); + len += snprintf(p1 + len, lMaxLen - len, ","); } } @@ -345,15 +346,19 @@ static const char* encryptAlgorithmStr(int8_t encryptAlgorithm) { } int32_t formatDurationOrKeep(char* buffer, int32_t timeInMinutes) { + if (buffer == NULL) { + return 0; + } + int lMaxLen = 32; int32_t len = 0; if (timeInMinutes % 1440 == 0) { int32_t days = timeInMinutes / 1440; - len = sprintf(buffer, "%dd", days); + len = snprintf(buffer, lMaxLen,"%dd", days); } else if (timeInMinutes % 60 == 0) { int32_t hours = timeInMinutes / 60; - len = sprintf(buffer, "%dh", hours); + len = snprintf(buffer, lMaxLen,"%dh", hours); } else { - len = sprintf(buffer, "%dm", timeInMinutes); + len = snprintf(buffer, lMaxLen,"%dm", timeInMinutes); } return len; } @@ -406,9 +411,9 @@ static int32_t setCreateDBResultIntoDataBlock(SSDataBlock* pBlock, char* dbName, int32_t lenKeep2 = formatDurationOrKeep(keep2Str, pCfg->daysToKeep2); if (IS_SYS_DBNAME(dbName)) { - len += sprintf(buf2 + VARSTR_HEADER_SIZE, "CREATE DATABASE `%s`", dbName); + len += snprintf(buf2 + VARSTR_HEADER_SIZE, SHOW_CREATE_DB_RESULT_FIELD2_LEN - VARSTR_HEADER_SIZE, "CREATE DATABASE `%s`", dbName); } else { - len += sprintf(buf2 + VARSTR_HEADER_SIZE, + len += snprintf(buf2 + VARSTR_HEADER_SIZE, SHOW_CREATE_DB_RESULT_FIELD2_LEN - VARSTR_HEADER_SIZE, "CREATE DATABASE `%s` BUFFER %d CACHESIZE %d CACHEMODEL '%s' COMP %d DURATION %s " "WAL_FSYNC_PERIOD %d MAXROWS %d MINROWS %d STT_TRIGGER %d KEEP %s,%s,%s PAGES %d PAGESIZE %d " "PRECISION '%s' REPLICA %d " @@ -426,7 +431,7 @@ static int32_t setCreateDBResultIntoDataBlock(SSDataBlock* pBlock, char* dbName, pCfg->s3KeepLocal, pCfg->s3Compact); if (pRetentions) { - len += sprintf(buf2 + VARSTR_HEADER_SIZE + len, " RETENTIONS %s", pRetentions); + len += snprintf(buf2 + VARSTR_HEADER_SIZE + len, SHOW_CREATE_DB_RESULT_FIELD2_LEN - VARSTR_HEADER_SIZE, " RETENTIONS %s", pRetentions); } } @@ -503,28 +508,32 @@ static int32_t buildCreateViewResultDataBlock(SSDataBlock** pOutput) { void appendColumnFields(char* buf, int32_t* len, STableCfg* pCfg) { for (int32_t i = 0; i < pCfg->numOfColumns; ++i) { SSchema* pSchema = pCfg->pSchemas + i; - char type[32 + 60]; // 60 byte for compress info - sprintf(type, "%s", tDataTypes[pSchema->type].name); +#define LTYPE_LEN (32 + 60) // 60 byte for compress info + char type[LTYPE_LEN]; + snprintf(type, LTYPE_LEN, "%s", tDataTypes[pSchema->type].name); if (TSDB_DATA_TYPE_VARCHAR == pSchema->type || TSDB_DATA_TYPE_VARBINARY == pSchema->type || TSDB_DATA_TYPE_GEOMETRY == pSchema->type) { - sprintf(type + strlen(type), "(%d)", (int32_t)(pSchema->bytes - VARSTR_HEADER_SIZE)); + snprintf(type + strlen(type), LTYPE_LEN - strlen(type), "(%d)", (int32_t)(pSchema->bytes - VARSTR_HEADER_SIZE)); } else if (TSDB_DATA_TYPE_NCHAR == pSchema->type) { - sprintf(type + strlen(type), "(%d)", (int32_t)((pSchema->bytes - VARSTR_HEADER_SIZE) / TSDB_NCHAR_SIZE)); + snprintf(type + strlen(type), LTYPE_LEN - strlen(type), "(%d)", + (int32_t)((pSchema->bytes - VARSTR_HEADER_SIZE) / TSDB_NCHAR_SIZE)); } if (useCompress(pCfg->tableType) && pCfg->pSchemaExt) { - sprintf(type + strlen(type), " ENCODE \'%s\'", - columnEncodeStr(COMPRESS_L1_TYPE_U32(pCfg->pSchemaExt[i].compress))); - sprintf(type + strlen(type), " COMPRESS \'%s\'", - columnCompressStr(COMPRESS_L2_TYPE_U32(pCfg->pSchemaExt[i].compress))); - sprintf(type + strlen(type), " LEVEL \'%s\'", - columnLevelStr(COMPRESS_L2_TYPE_LEVEL_U32(pCfg->pSchemaExt[i].compress))); + snprintf(type + strlen(type), LTYPE_LEN - strlen(type), " ENCODE \'%s\'", + columnEncodeStr(COMPRESS_L1_TYPE_U32(pCfg->pSchemaExt[i].compress))); + snprintf(type + strlen(type), LTYPE_LEN - strlen(type), " COMPRESS \'%s\'", + columnCompressStr(COMPRESS_L2_TYPE_U32(pCfg->pSchemaExt[i].compress))); + snprintf(type + strlen(type), LTYPE_LEN - strlen(type), " LEVEL \'%s\'", + columnLevelStr(COMPRESS_L2_TYPE_LEVEL_U32(pCfg->pSchemaExt[i].compress))); } if (!(pSchema->flags & COL_IS_KEY)) { - *len += sprintf(buf + VARSTR_HEADER_SIZE + *len, "%s`%s` %s", ((i > 0) ? ", " : ""), pSchema->name, type); + *len += snprintf(buf + VARSTR_HEADER_SIZE + *len, SHOW_CREATE_TB_RESULT_FIELD2_LEN - (VARSTR_HEADER_SIZE + *len), "%s`%s` %s", + ((i > 0) ? ", " : ""), pSchema->name, type); } else { char* pk = "PRIMARY KEY"; - *len += sprintf(buf + VARSTR_HEADER_SIZE + *len, "%s`%s` %s %s", ((i > 0) ? ", " : ""), pSchema->name, type, pk); + *len += snprintf(buf + VARSTR_HEADER_SIZE + *len, SHOW_CREATE_TB_RESULT_FIELD2_LEN - (VARSTR_HEADER_SIZE + *len), "%s`%s` %s %s", + ((i > 0) ? ", " : ""), pSchema->name, type, pk); } } } @@ -533,22 +542,25 @@ void appendTagFields(char* buf, int32_t* len, STableCfg* pCfg) { for (int32_t i = 0; i < pCfg->numOfTags; ++i) { SSchema* pSchema = pCfg->pSchemas + pCfg->numOfColumns + i; char type[32]; - sprintf(type, "%s", tDataTypes[pSchema->type].name); + snprintf(type, 32, "%s", tDataTypes[pSchema->type].name); if (TSDB_DATA_TYPE_VARCHAR == pSchema->type || TSDB_DATA_TYPE_VARBINARY == pSchema->type || TSDB_DATA_TYPE_GEOMETRY == pSchema->type) { - sprintf(type + strlen(type), "(%d)", (int32_t)(pSchema->bytes - VARSTR_HEADER_SIZE)); + snprintf(type + strlen(type), 32 - strlen(type), "(%d)", (int32_t)(pSchema->bytes - VARSTR_HEADER_SIZE)); } else if (TSDB_DATA_TYPE_NCHAR == pSchema->type) { - sprintf(type + strlen(type), "(%d)", (int32_t)((pSchema->bytes - VARSTR_HEADER_SIZE) / TSDB_NCHAR_SIZE)); + snprintf(type + strlen(type), 32 - strlen(type), "(%d)", + (int32_t)((pSchema->bytes - VARSTR_HEADER_SIZE) / TSDB_NCHAR_SIZE)); } - *len += sprintf(buf + VARSTR_HEADER_SIZE + *len, "%s`%s` %s", ((i > 0) ? ", " : ""), pSchema->name, type); + *len += snprintf(buf + VARSTR_HEADER_SIZE + *len, 32 - (VARSTR_HEADER_SIZE + *len), "%s`%s` %s", + ((i > 0) ? ", " : ""), pSchema->name, type); } } void appendTagNameFields(char* buf, int32_t* len, STableCfg* pCfg) { for (int32_t i = 0; i < pCfg->numOfTags; ++i) { SSchema* pSchema = pCfg->pSchemas + pCfg->numOfColumns + i; - *len += sprintf(buf + VARSTR_HEADER_SIZE + *len, "%s`%s`", ((i > 0) ? ", " : ""), pSchema->name); + *len += snprintf(buf + VARSTR_HEADER_SIZE + *len, SHOW_CREATE_TB_RESULT_FIELD2_LEN - (VARSTR_HEADER_SIZE + *len), + "%s`%s`", ((i > 0) ? ", " : ""), pSchema->name); } } @@ -565,11 +577,12 @@ int32_t appendTagValues(char* buf, int32_t* len, STableCfg* pCfg) { if (tTagIsJson(pTag)) { char* pJson = NULL; parseTagDatatoJson(pTag, &pJson); - if(NULL == pJson) { + if (NULL == pJson) { qError("failed to parse tag to json, pJson is NULL"); return terrno; } - *len += sprintf(buf + VARSTR_HEADER_SIZE + *len, "%s", pJson); + *len += snprintf(buf + VARSTR_HEADER_SIZE + *len, SHOW_CREATE_TB_RESULT_FIELD2_LEN - (VARSTR_HEADER_SIZE + *len), + "%s", pJson); taosMemoryFree(pJson); return TSDB_CODE_SUCCESS; @@ -582,11 +595,13 @@ int32_t appendTagValues(char* buf, int32_t* len, STableCfg* pCfg) { for (int32_t i = 0; i < pCfg->numOfTags; ++i) { SSchema* pSchema = pCfg->pSchemas + pCfg->numOfColumns + i; if (i > 0) { - *len += sprintf(buf + VARSTR_HEADER_SIZE + *len, ", "); + *len += snprintf(buf + VARSTR_HEADER_SIZE + *len, SHOW_CREATE_TB_RESULT_FIELD2_LEN - (VARSTR_HEADER_SIZE + *len), + ", "); } if (j >= valueNum) { - *len += sprintf(buf + VARSTR_HEADER_SIZE + *len, "NULL"); + *len += snprintf(buf + VARSTR_HEADER_SIZE + *len, SHOW_CREATE_TB_RESULT_FIELD2_LEN - (VARSTR_HEADER_SIZE + *len), + "NULL"); continue; } @@ -609,7 +624,8 @@ int32_t appendTagValues(char* buf, int32_t* len, STableCfg* pCfg) { *len += tlen; j++; } else { - *len += sprintf(buf + VARSTR_HEADER_SIZE + *len, "NULL"); + *len += snprintf(buf + VARSTR_HEADER_SIZE + *len, SHOW_CREATE_TB_RESULT_FIELD2_LEN - (VARSTR_HEADER_SIZE + *len), + "NULL"); } } _exit: @@ -620,37 +636,47 @@ _exit: void appendTableOptions(char* buf, int32_t* len, SDbCfgInfo* pDbCfg, STableCfg* pCfg) { if (pCfg->commentLen > 0) { - *len += sprintf(buf + VARSTR_HEADER_SIZE + *len, " COMMENT '%s'", pCfg->pComment); + *len += snprintf(buf + VARSTR_HEADER_SIZE + *len, SHOW_CREATE_TB_RESULT_FIELD2_LEN - (VARSTR_HEADER_SIZE + *len), + " COMMENT '%s'", pCfg->pComment); } else if (0 == pCfg->commentLen) { - *len += sprintf(buf + VARSTR_HEADER_SIZE + *len, " COMMENT ''"); + *len += snprintf(buf + VARSTR_HEADER_SIZE + *len, SHOW_CREATE_TB_RESULT_FIELD2_LEN - (VARSTR_HEADER_SIZE + *len), + " COMMENT ''"); } if (NULL != pDbCfg->pRetensions && pCfg->watermark1 > 0) { - *len += sprintf(buf + VARSTR_HEADER_SIZE + *len, " WATERMARK %" PRId64 "a", pCfg->watermark1); + *len += snprintf(buf + VARSTR_HEADER_SIZE + *len, SHOW_CREATE_TB_RESULT_FIELD2_LEN - (VARSTR_HEADER_SIZE + *len), + " WATERMARK %" PRId64 "a", pCfg->watermark1); if (pCfg->watermark2 > 0) { - *len += sprintf(buf + VARSTR_HEADER_SIZE + *len, ", %" PRId64 "a", pCfg->watermark2); + *len += snprintf(buf + VARSTR_HEADER_SIZE + *len, SHOW_CREATE_TB_RESULT_FIELD2_LEN - (VARSTR_HEADER_SIZE + *len), + ", %" PRId64 "a", pCfg->watermark2); } } if (NULL != pDbCfg->pRetensions && pCfg->delay1 > 0) { - *len += sprintf(buf + VARSTR_HEADER_SIZE + *len, " MAX_DELAY %" PRId64 "a", pCfg->delay1); + *len += snprintf(buf + VARSTR_HEADER_SIZE + *len, SHOW_CREATE_TB_RESULT_FIELD2_LEN - (VARSTR_HEADER_SIZE + *len), + " MAX_DELAY %" PRId64 "a", pCfg->delay1); if (pCfg->delay2 > 0) { - *len += sprintf(buf + VARSTR_HEADER_SIZE + *len, ", %" PRId64 "a", pCfg->delay2); + *len += snprintf(buf + VARSTR_HEADER_SIZE + *len, SHOW_CREATE_TB_RESULT_FIELD2_LEN - (VARSTR_HEADER_SIZE + *len), + ", %" PRId64 "a", pCfg->delay2); } } int32_t funcNum = taosArrayGetSize(pCfg->pFuncs); if (NULL != pDbCfg->pRetensions && funcNum > 0) { - *len += sprintf(buf + VARSTR_HEADER_SIZE + *len, " ROLLUP("); + *len += snprintf(buf + VARSTR_HEADER_SIZE + *len, SHOW_CREATE_TB_RESULT_FIELD2_LEN - (VARSTR_HEADER_SIZE + *len), + " ROLLUP("); for (int32_t i = 0; i < funcNum; ++i) { char* pFunc = taosArrayGet(pCfg->pFuncs, i); - *len += sprintf(buf + VARSTR_HEADER_SIZE + *len, "%s%s", ((i > 0) ? ", " : ""), pFunc); + *len += snprintf(buf + VARSTR_HEADER_SIZE + *len, SHOW_CREATE_TB_RESULT_FIELD2_LEN - (VARSTR_HEADER_SIZE + *len), + "%s%s", ((i > 0) ? ", " : ""), pFunc); } - *len += sprintf(buf + VARSTR_HEADER_SIZE + *len, ")"); + *len += + snprintf(buf + VARSTR_HEADER_SIZE + *len, SHOW_CREATE_TB_RESULT_FIELD2_LEN - (VARSTR_HEADER_SIZE + *len), ")"); } if (pCfg->ttl > 0) { - *len += sprintf(buf + VARSTR_HEADER_SIZE + *len, " TTL %d", pCfg->ttl); + *len += snprintf(buf + VARSTR_HEADER_SIZE + *len, SHOW_CREATE_TB_RESULT_FIELD2_LEN - (VARSTR_HEADER_SIZE + *len), + " TTL %d", pCfg->ttl); } if (TSDB_SUPER_TABLE == pCfg->tableType || TSDB_NORMAL_TABLE == pCfg->tableType) { @@ -663,18 +689,23 @@ void appendTableOptions(char* buf, int32_t* len, SDbCfgInfo* pDbCfg, STableCfg* if (nSma < pCfg->numOfColumns && nSma > 0) { bool smaOn = false; - *len += sprintf(buf + VARSTR_HEADER_SIZE + *len, " SMA("); + *len += snprintf(buf + VARSTR_HEADER_SIZE + *len, SHOW_CREATE_TB_RESULT_FIELD2_LEN - (VARSTR_HEADER_SIZE + *len), + " SMA("); for (int32_t i = 0; i < pCfg->numOfColumns; ++i) { if (IS_BSMA_ON(pCfg->pSchemas + i)) { if (smaOn) { - *len += sprintf(buf + VARSTR_HEADER_SIZE + *len, ",`%s`", (pCfg->pSchemas + i)->name); + *len += snprintf(buf + VARSTR_HEADER_SIZE + *len, + SHOW_CREATE_TB_RESULT_FIELD2_LEN - (VARSTR_HEADER_SIZE + *len), ",`%s`", + (pCfg->pSchemas + i)->name); } else { smaOn = true; - *len += sprintf(buf + VARSTR_HEADER_SIZE + *len, "`%s`", (pCfg->pSchemas + i)->name); + *len += snprintf(buf + VARSTR_HEADER_SIZE + *len, + SHOW_CREATE_TB_RESULT_FIELD2_LEN - (VARSTR_HEADER_SIZE + *len), "`%s`", + (pCfg->pSchemas + i)->name); } } } - *len += sprintf(buf + VARSTR_HEADER_SIZE + *len, ")"); + *len += snprintf(buf + VARSTR_HEADER_SIZE + *len, SHOW_CREATE_TB_RESULT_FIELD2_LEN - VARSTR_HEADER_SIZE, ")"); } } } @@ -698,24 +729,32 @@ static int32_t setCreateTBResultIntoDataBlock(SSDataBlock* pBlock, SDbCfgInfo* p int32_t len = 0; if (TSDB_SUPER_TABLE == pCfg->tableType) { - len += sprintf(buf2 + VARSTR_HEADER_SIZE, "CREATE STABLE `%s` (", tbName); + len += snprintf(buf2 + VARSTR_HEADER_SIZE, SHOW_CREATE_TB_RESULT_FIELD2_LEN - VARSTR_HEADER_SIZE, + "CREATE STABLE `%s` (", tbName); appendColumnFields(buf2, &len, pCfg); - len += sprintf(buf2 + VARSTR_HEADER_SIZE + len, ") TAGS ("); + len += snprintf(buf2 + VARSTR_HEADER_SIZE + len, SHOW_CREATE_TB_RESULT_FIELD2_LEN - (VARSTR_HEADER_SIZE + len), + ") TAGS ("); appendTagFields(buf2, &len, pCfg); - len += sprintf(buf2 + VARSTR_HEADER_SIZE + len, ")"); + len += + snprintf(buf2 + VARSTR_HEADER_SIZE + len, SHOW_CREATE_TB_RESULT_FIELD2_LEN - (VARSTR_HEADER_SIZE + len), ")"); appendTableOptions(buf2, &len, pDbCfg, pCfg); } else if (TSDB_CHILD_TABLE == pCfg->tableType) { - len += sprintf(buf2 + VARSTR_HEADER_SIZE, "CREATE TABLE `%s` USING `%s` (", tbName, pCfg->stbName); + len += snprintf(buf2 + VARSTR_HEADER_SIZE, SHOW_CREATE_TB_RESULT_FIELD2_LEN - VARSTR_HEADER_SIZE, + "CREATE TABLE `%s` USING `%s` (", tbName, pCfg->stbName); appendTagNameFields(buf2, &len, pCfg); - len += sprintf(buf2 + VARSTR_HEADER_SIZE + len, ") TAGS ("); + len += snprintf(buf2 + VARSTR_HEADER_SIZE + len, SHOW_CREATE_TB_RESULT_FIELD2_LEN - (VARSTR_HEADER_SIZE + len), + ") TAGS ("); code = appendTagValues(buf2, &len, pCfg); TAOS_CHECK_ERRNO(code); - len += sprintf(buf2 + VARSTR_HEADER_SIZE + len, ")"); + len += + snprintf(buf2 + VARSTR_HEADER_SIZE + len, SHOW_CREATE_TB_RESULT_FIELD2_LEN - (VARSTR_HEADER_SIZE + len), ")"); appendTableOptions(buf2, &len, pDbCfg, pCfg); } else { - len += sprintf(buf2 + VARSTR_HEADER_SIZE, "CREATE TABLE `%s` (", tbName); + len += snprintf(buf2 + VARSTR_HEADER_SIZE, SHOW_CREATE_TB_RESULT_FIELD2_LEN - VARSTR_HEADER_SIZE, + "CREATE TABLE `%s` (", tbName); appendColumnFields(buf2, &len, pCfg); - len += sprintf(buf2 + VARSTR_HEADER_SIZE + len, ")"); + len += + snprintf(buf2 + VARSTR_HEADER_SIZE + len, SHOW_CREATE_TB_RESULT_FIELD2_LEN - (VARSTR_HEADER_SIZE + len), ")"); appendTableOptions(buf2, &len, pDbCfg, pCfg); } @@ -792,9 +831,21 @@ static int32_t execAlterCmd(char* cmd, char* value, bool* processed) { taosResetLog(); cfgDumpCfg(tsCfg, 0, false); } else if (0 == strcasecmp(cmd, COMMAND_SCHEDULE_POLICY)) { - code = schedulerUpdatePolicy(atoi(value)); + int32_t tmp = 0; + code = taosStr2int32(value, &tmp); + if (code) { + qError("invalid value:%s, error:%s", value, tstrerror(code)); + return code; + } + code = schedulerUpdatePolicy(tmp); } else if (0 == strcasecmp(cmd, COMMAND_ENABLE_RESCHEDULE)) { - code = schedulerEnableReSchedule(atoi(value)); + int32_t tmp = 0; + code = taosStr2int32(value, &tmp); + if (code) { + qError("invalid value:%s, error:%s", value, tstrerror(code)); + return code; + } + code = schedulerEnableReSchedule(tmp != 0); } else if (0 == strcasecmp(cmd, COMMAND_CATALOG_DEBUG)) { code = ctgdHandleDbgCommand(value); } else if (0 == strcasecmp(cmd, COMMAND_ENABLE_MEM_DEBUG)) { diff --git a/source/libs/function/src/tudf.c b/source/libs/function/src/tudf.c index ffdd4ea500..bc8237992d 100644 --- a/source/libs/function/src/tudf.c +++ b/source/libs/function/src/tudf.c @@ -158,10 +158,11 @@ static int32_t udfSpawnUdfd(SUdfdData *pData) { char *taosFqdnEnvItem = NULL; char *taosFqdn = getenv("TAOS_FQDN"); if (taosFqdn != NULL) { - taosFqdnEnvItem = taosMemoryMalloc(strlen("TAOS_FQDN=") + strlen(taosFqdn) + 1); + int len = strlen("TAOS_FQDN=") + strlen(taosFqdn) + 1; + taosFqdnEnvItem = taosMemoryMalloc(len); if (taosFqdnEnvItem != NULL) { - strcpy(taosFqdnEnvItem, "TAOS_FQDN="); - TAOS_STRCAT(taosFqdnEnvItem, taosFqdn); + TAOS_STRNCPY(taosFqdnEnvItem, "TAOS_FQDN=", len); + TAOS_STRNCAT(taosFqdnEnvItem, taosFqdn, strlen(taosFqdn)); fnInfo("[UDFD]Succsess to set TAOS_FQDN:%s", taosFqdn); } else { fnError("[UDFD]Failed to allocate memory for TAOS_FQDN"); diff --git a/source/libs/function/src/udfd.c b/source/libs/function/src/udfd.c index e1dfd686d4..255cf1a4de 100644 --- a/source/libs/function/src/udfd.c +++ b/source/libs/function/src/udfd.c @@ -396,7 +396,7 @@ int32_t udfdLoadSharedLib(char *libPath, uv_lib_t *pLib, const char *funcName[], int32_t udfdInitializePythonPlugin(SUdfScriptPlugin *plugin) { plugin->scriptType = TSDB_FUNC_SCRIPT_PYTHON; // todo: windows support - sprintf(plugin->libPath, "%s", "libtaospyudf.so"); + snprintf(plugin->libPath, PATH_MAX, "%s", "libtaospyudf.so"); plugin->libLoaded = false; const char *funcName[UDFD_MAX_PLUGIN_FUNCS] = {"pyOpen", "pyClose", "pyUdfInit", "pyUdfDestroy", "pyUdfScalarProc", "pyUdfAggStart", diff --git a/source/libs/qcom/src/queryUtil.c b/source/libs/qcom/src/queryUtil.c index ff20211af2..dfc2596a29 100644 --- a/source/libs/qcom/src/queryUtil.c +++ b/source/libs/qcom/src/queryUtil.c @@ -313,42 +313,42 @@ void destroyQueryExecRes(SExecResult* pRes) { } } // clang-format on - +#define MAX_NUMERICAL_LENGTH (32) int32_t dataConverToStr(char* str, int type, void* buf, int32_t bufSize, int32_t* len) { int32_t n = 0; switch (type) { case TSDB_DATA_TYPE_NULL: - n = sprintf(str, "null"); + n = snprintf(str, MAX_NUMERICAL_LENGTH, "null"); break; case TSDB_DATA_TYPE_BOOL: - n = sprintf(str, (*(int8_t*)buf) ? "true" : "false"); + n = snprintf(str, MAX_NUMERICAL_LENGTH, (*(int8_t*)buf) ? "true" : "false"); break; case TSDB_DATA_TYPE_TINYINT: - n = sprintf(str, "%d", *(int8_t*)buf); + n = snprintf(str, MAX_NUMERICAL_LENGTH, "%d", *(int8_t*)buf); break; case TSDB_DATA_TYPE_SMALLINT: - n = sprintf(str, "%d", *(int16_t*)buf); + n = snprintf(str, MAX_NUMERICAL_LENGTH, "%d", *(int16_t*)buf); break; case TSDB_DATA_TYPE_INT: - n = sprintf(str, "%d", *(int32_t*)buf); + n = snprintf(str, MAX_NUMERICAL_LENGTH, "%d", *(int32_t*)buf); break; case TSDB_DATA_TYPE_BIGINT: case TSDB_DATA_TYPE_TIMESTAMP: - n = sprintf(str, "%" PRId64, *(int64_t*)buf); + n = snprintf(str, MAX_NUMERICAL_LENGTH, "%" PRId64, *(int64_t*)buf); break; case TSDB_DATA_TYPE_FLOAT: - n = sprintf(str, "%e", GET_FLOAT_VAL(buf)); + n = snprintf(str, MAX_NUMERICAL_LENGTH, "%e", GET_FLOAT_VAL(buf)); break; case TSDB_DATA_TYPE_DOUBLE: - n = sprintf(str, "%e", GET_DOUBLE_VAL(buf)); + n = snprintf(str, MAX_NUMERICAL_LENGTH, "%e", GET_DOUBLE_VAL(buf)); break; case TSDB_DATA_TYPE_VARBINARY: { @@ -395,19 +395,19 @@ int32_t dataConverToStr(char* str, int type, void* buf, int32_t bufSize, int32_t n = length + 2; break; case TSDB_DATA_TYPE_UTINYINT: - n = sprintf(str, "%d", *(uint8_t*)buf); + n = snprintf(str, MAX_NUMERICAL_LENGTH, "%d", *(uint8_t*)buf); break; case TSDB_DATA_TYPE_USMALLINT: - n = sprintf(str, "%d", *(uint16_t*)buf); + n = snprintf(str, MAX_NUMERICAL_LENGTH, "%d", *(uint16_t*)buf); break; case TSDB_DATA_TYPE_UINT: - n = sprintf(str, "%u", *(uint32_t*)buf); + n = snprintf(str, MAX_NUMERICAL_LENGTH, "%u", *(uint32_t*)buf); break; case TSDB_DATA_TYPE_UBIGINT: - n = sprintf(str, "%" PRIu64, *(uint64_t*)buf); + n = snprintf(str, MAX_NUMERICAL_LENGTH, "%" PRIu64, *(uint64_t*)buf); break; default: diff --git a/source/libs/qcom/src/querymsg.c b/source/libs/qcom/src/querymsg.c index 542e549d40..a3abadec01 100644 --- a/source/libs/qcom/src/querymsg.c +++ b/source/libs/qcom/src/querymsg.c @@ -688,14 +688,14 @@ int32_t queryProcessTableMetaRsp(void *output, char *msg, int32_t msgSize) { } STableMetaOutput *pOut = output; - strcpy(pOut->dbFName, metaRsp.dbFName); + strncpy(pOut->dbFName, metaRsp.dbFName, TSDB_DB_FNAME_LEN); pOut->dbId = metaRsp.dbId; if (metaRsp.tableType == TSDB_CHILD_TABLE) { SET_META_TYPE_BOTH_TABLE(pOut->metaType); - strcpy(pOut->ctbName, metaRsp.tbName); - strcpy(pOut->tbName, metaRsp.stbName); + strncpy(pOut->ctbName, metaRsp.tbName, TSDB_TABLE_NAME_LEN); + strncpy(pOut->tbName, metaRsp.stbName, TSDB_TABLE_NAME_LEN); pOut->ctbMeta.vgId = metaRsp.vgId; pOut->ctbMeta.tableType = metaRsp.tableType; @@ -705,7 +705,7 @@ int32_t queryProcessTableMetaRsp(void *output, char *msg, int32_t msgSize) { code = queryCreateTableMetaFromMsg(&metaRsp, true, &pOut->tbMeta); } else { SET_META_TYPE_TABLE(pOut->metaType); - strcpy(pOut->tbName, metaRsp.tbName); + strncpy(pOut->tbName, metaRsp.tbName, TSDB_TABLE_NAME_LEN); code = queryCreateTableMetaFromMsg(&metaRsp, (metaRsp.tableType == TSDB_SUPER_TABLE), &pOut->tbMeta); } @@ -744,14 +744,14 @@ static int32_t queryProcessTableNameRsp(void *output, char *msg, int32_t msgSize } STableMetaOutput *pOut = output; - strcpy(pOut->dbFName, metaRsp.dbFName); + strncpy(pOut->dbFName, metaRsp.dbFName, TSDB_DB_FNAME_LEN); pOut->dbId = metaRsp.dbId; if (metaRsp.tableType == TSDB_CHILD_TABLE) { SET_META_TYPE_BOTH_TABLE(pOut->metaType); - strcpy(pOut->ctbName, metaRsp.tbName); - strcpy(pOut->tbName, metaRsp.stbName); + strncpy(pOut->ctbName, metaRsp.tbName, TSDB_TABLE_NAME_LEN); + strncpy(pOut->tbName, metaRsp.stbName, TSDB_TABLE_NAME_LEN); pOut->ctbMeta.vgId = metaRsp.vgId; pOut->ctbMeta.tableType = metaRsp.tableType; @@ -761,7 +761,7 @@ static int32_t queryProcessTableNameRsp(void *output, char *msg, int32_t msgSize code = queryCreateTableMetaExFromMsg(&metaRsp, true, &pOut->tbMeta); } else { SET_META_TYPE_TABLE(pOut->metaType); - strcpy(pOut->tbName, metaRsp.tbName); + strncpy(pOut->tbName, metaRsp.tbName, TSDB_TABLE_NAME_LEN); code = queryCreateTableMetaExFromMsg(&metaRsp, (metaRsp.tableType == TSDB_SUPER_TABLE), &pOut->tbMeta); } From 83f0601ea9058b89d5beb2e1c3a4e8f288ca967f Mon Sep 17 00:00:00 2001 From: Jing Sima Date: Tue, 8 Oct 2024 14:57:04 +0800 Subject: [PATCH 22/51] fix:[TD-32452] Clean up when error occurs to avoid mem leak. --- source/libs/function/src/builtinsimpl.c | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/source/libs/function/src/builtinsimpl.c b/source/libs/function/src/builtinsimpl.c index 157d44b3de..a7e2b28de2 100644 --- a/source/libs/function/src/builtinsimpl.c +++ b/source/libs/function/src/builtinsimpl.c @@ -3185,7 +3185,8 @@ int32_t firstLastPartialFinalize(SqlFunctionCtx* pCtx, SSDataBlock* pBlock) { } else { code = colDataSetVal(pCol, pBlock->info.rows, res, false); if (TSDB_CODE_SUCCESS != code) { - return TSDB_CODE_OUT_OF_MEMORY; + taosMemoryFree(res); + return code; } code = setSelectivityValue(pCtx, pBlock, &pRes->pos, pBlock->info.rows); } From e51199d541217f2b00cf340f757e87d62e2bccd7 Mon Sep 17 00:00:00 2001 From: dapan1121 Date: Tue, 8 Oct 2024 15:16:48 +0800 Subject: [PATCH 23/51] fix: data deleter memory leak issue --- source/libs/executor/src/dataDeleter.c | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/source/libs/executor/src/dataDeleter.c b/source/libs/executor/src/dataDeleter.c index 57f4289ebf..c284e9a8a9 100644 --- a/source/libs/executor/src/dataDeleter.c +++ b/source/libs/executor/src/dataDeleter.c @@ -273,10 +273,18 @@ static int32_t getCacheSize(struct SDataSinkHandle* pHandle, uint64_t* size) { int32_t createDataDeleter(SDataSinkManager* pManager, const SDataSinkNode* pDataSink, DataSinkHandle* pHandle, void* pParam) { int32_t code = TSDB_CODE_SUCCESS; + if (pParam == NULL) { + code = TSDB_CODE_QRY_INVALID_INPUT; + qError("invalid input param in creating data deleter, code%s", tstrerror(code)); + goto _end; + } + + SDeleterParam* pDeleterParam = (SDeleterParam*)pParam; SDataDeleterHandle* deleter = taosMemoryCalloc(1, sizeof(SDataDeleterHandle)); if (NULL == deleter) { code = terrno; + taosArrayDestroy(pDeleterParam->pUidList); taosMemoryFree(pParam); goto _end; } @@ -292,12 +300,6 @@ int32_t createDataDeleter(SDataSinkManager* pManager, const SDataSinkNode* pData deleter->pDeleter = pDeleterNode; deleter->pSchema = pDataSink->pInputDataBlockDesc; - if (pParam == NULL) { - code = TSDB_CODE_QRY_INVALID_INPUT; - qError("invalid input param in creating data deleter, code%s", tstrerror(code)); - goto _end; - } - deleter->pParam = pParam; deleter->status = DS_BUF_EMPTY; deleter->queryEnd = false; From f1508db3be0f5cd4ed91826491ce691e01a582ce Mon Sep 17 00:00:00 2001 From: Jing Sima Date: Tue, 8 Oct 2024 15:26:26 +0800 Subject: [PATCH 24/51] fix:[TD-32450] Use taosArrayDestroyEx when error occurs to avoid mem leak. --- source/libs/scalar/src/filter.c | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/source/libs/scalar/src/filter.c b/source/libs/scalar/src/filter.c index a9a765c0fa..a3608cc1dc 100644 --- a/source/libs/scalar/src/filter.c +++ b/source/libs/scalar/src/filter.c @@ -3809,13 +3809,13 @@ int32_t fltInitFromNode(SNode *tree, SFilterInfo *info, uint32_t options) { SFltBuildGroupCtx tctx = {.info = info, .group = group}; nodesWalkExpr(tree, fltTreeToGroup, (void *)&tctx); if (TSDB_CODE_SUCCESS != tctx.code) { - taosArrayDestroy(group); + taosArrayDestroyEx(group, filterFreeGroup); code = tctx.code; goto _return; } code = filterConvertGroupFromArray(info, group); if (TSDB_CODE_SUCCESS != code) { - taosArrayDestroy(group); + taosArrayDestroyEx(group, filterFreeGroup); goto _return; } taosArrayDestroy(group); From f5872e6cecb3108e93d6295dab4ae4e72537f0ab Mon Sep 17 00:00:00 2001 From: Jing Sima Date: Tue, 8 Oct 2024 17:02:25 +0800 Subject: [PATCH 25/51] fix:[TD-32454] Use SMA when using min/max on numeric column. --- include/libs/nodes/querynodes.h | 1 - source/libs/function/src/builtins.c | 1 - source/libs/planner/src/planOptimizer.c | 6 +----- 3 files changed, 1 insertion(+), 7 deletions(-) diff --git a/include/libs/nodes/querynodes.h b/include/libs/nodes/querynodes.h index 5f9a4a1110..f5567c735e 100644 --- a/include/libs/nodes/querynodes.h +++ b/include/libs/nodes/querynodes.h @@ -190,7 +190,6 @@ typedef struct SFunctionNode { bool hasOriginalFunc; int32_t originalFuncId; ETrimType trimType; - bool hasSMA; bool dual; // whether select stmt without from stmt, true for without. } SFunctionNode; diff --git a/source/libs/function/src/builtins.c b/source/libs/function/src/builtins.c index 604375aed2..21fb57f5bb 100644 --- a/source/libs/function/src/builtins.c +++ b/source/libs/function/src/builtins.c @@ -327,7 +327,6 @@ static int32_t translateMinMax(SFunctionNode* pFunc, char* pErrBuf, int32_t len) } else if (IS_NULL_TYPE(paraType)) { paraType = TSDB_DATA_TYPE_BIGINT; } - pFunc->hasSMA = !IS_VAR_DATA_TYPE(paraType); int32_t bytes = IS_STR_DATA_TYPE(paraType) ? dataType->bytes : tDataTypes[paraType].bytes; pFunc->node.resType = (SDataType){.bytes = bytes, .type = paraType}; return TSDB_CODE_SUCCESS; diff --git a/source/libs/planner/src/planOptimizer.c b/source/libs/planner/src/planOptimizer.c index 401b4f93d1..1bcec86385 100644 --- a/source/libs/planner/src/planOptimizer.c +++ b/source/libs/planner/src/planOptimizer.c @@ -294,9 +294,6 @@ static bool scanPathOptIsSpecifiedFuncType(const SFunctionNode* pFunc, bool (*ty return true; } -static bool isMinMaxFunction(int32_t funcType) { - return funcType == FUNCTION_TYPE_MIN || funcType == FUNCTION_TYPE_MAX; -} static int32_t scanPathOptGetRelatedFuncs(SScanLogicNode* pScan, SNodeList** pSdrFuncs, SNodeList** pDsoFuncs) { SNodeList* pAllFuncs = scanPathOptGetAllFuncs(pScan->node.pParent); SNodeList* pTmpSdrFuncs = NULL; @@ -306,8 +303,7 @@ static int32_t scanPathOptGetRelatedFuncs(SScanLogicNode* pScan, SNodeList** pSd FOREACH(pNode, pAllFuncs) { SFunctionNode* pFunc = (SFunctionNode*)pNode; int32_t code = TSDB_CODE_SUCCESS; - if ((!isMinMaxFunction(pFunc->funcType) && scanPathOptIsSpecifiedFuncType(pFunc, fmIsSpecialDataRequiredFunc)) || - (isMinMaxFunction(pFunc->funcType) && pFunc->hasSMA)) { + if (scanPathOptIsSpecifiedFuncType(pFunc, fmIsSpecialDataRequiredFunc)) { SNode* pNew = NULL; code = nodesCloneNode(pNode, &pNew); if (TSDB_CODE_SUCCESS == code) { From 8acdbe597d4305037361bcd35992a53bec577554 Mon Sep 17 00:00:00 2001 From: Jing Sima Date: Mon, 30 Sep 2024 11:16:42 +0800 Subject: [PATCH 26/51] fix:[TD-32372] Clean up in a right way. --- source/libs/executor/inc/executorInt.h | 3 ++ source/libs/executor/src/aggregateoperator.c | 40 ++++++++++++++++++- source/libs/executor/src/timewindowoperator.c | 21 +++++----- 3 files changed, 50 insertions(+), 14 deletions(-) diff --git a/source/libs/executor/inc/executorInt.h b/source/libs/executor/inc/executorInt.h index e391d274e3..572ff88be9 100644 --- a/source/libs/executor/inc/executorInt.h +++ b/source/libs/executor/inc/executorInt.h @@ -833,6 +833,9 @@ void cleanupResultInfoInStream(SExecTaskInfo* pTaskInfo, void* pState, SExpr SGroupResInfo* pGroupResInfo); void cleanupResultInfo(SExecTaskInfo* pTaskInfo, SExprSupp* pSup, SDiskbasedBuf* pBuf, SGroupResInfo* pGroupResInfo, SSHashObj* pHashmap); +void cleanupResultInfoWithoutHash(SExecTaskInfo* pTaskInfo, SExprSupp* pSup, SDiskbasedBuf* pBuf, + SGroupResInfo* pGroupResInfo); + int32_t initAggSup(SExprSupp* pSup, SAggSupporter* pAggSup, SExprInfo* pExprInfo, int32_t numOfCols, size_t keyBufSize, const char* pkey, void* pState, SFunctionStateStore* pStore); void cleanupAggSup(SAggSupporter* pAggSup); diff --git a/source/libs/executor/src/aggregateoperator.c b/source/libs/executor/src/aggregateoperator.c index 9e5ad132f7..91b435fbec 100644 --- a/source/libs/executor/src/aggregateoperator.c +++ b/source/libs/executor/src/aggregateoperator.c @@ -159,8 +159,8 @@ void destroyAggOperatorInfo(void* param) { cleanupBasicInfo(&pInfo->binfo); if (pInfo->pOperator) { - cleanupResultInfo(pInfo->pOperator->pTaskInfo, &pInfo->pOperator->exprSupp, pInfo->aggSup.pResultBuf, - &pInfo->groupResInfo, pInfo->aggSup.pResultRowHashTable); + cleanupResultInfoWithoutHash(pInfo->pOperator->pTaskInfo, &pInfo->pOperator->exprSupp, pInfo->aggSup.pResultBuf, + &pInfo->groupResInfo); pInfo->pOperator = NULL; } cleanupAggSup(&pInfo->aggSup); @@ -627,6 +627,42 @@ void cleanupResultInfoInStream(SExecTaskInfo* pTaskInfo, void* pState, SExprSupp } } +void cleanupResultInfoWithoutHash(SExecTaskInfo* pTaskInfo, SExprSupp* pSup, SDiskbasedBuf* pBuf, + SGroupResInfo* pGroupResInfo) { + int32_t numOfExprs = pSup->numOfExprs; + int32_t* rowEntryOffset = pSup->rowEntryInfoOffset; + SqlFunctionCtx* pCtx = pSup->pCtx; + int32_t numOfRows = getNumOfTotalRes(pGroupResInfo); + bool needCleanup = false; + + for (int32_t j = 0; j < numOfExprs; ++j) { + needCleanup |= pCtx[j].needCleanup; + } + if (!needCleanup) { + return; + } + + for (int32_t i = pGroupResInfo->index; i < numOfRows; i += 1) { + SResultRow* pRow = NULL; + SResKeyPos* pPos = taosArrayGetP(pGroupResInfo->pRows, i); + SFilePage* page = getBufPage(pBuf, pPos->pos.pageId); + if (page == NULL) { + qError("failed to get buffer, code:%s, %s", tstrerror(terrno), GET_TASKID(pTaskInfo)); + continue; + } + pRow = (SResultRow*)((char*)page + pPos->pos.offset); + + + for (int32_t j = 0; j < numOfExprs; ++j) { + pCtx[j].resultInfo = getResultEntryInfo(pRow, j, rowEntryOffset); + if (pCtx[j].fpSet.cleanup) { + pCtx[j].fpSet.cleanup(&pCtx[j]); + } + } + releaseBufPage(pBuf, page); + } +} + void cleanupResultInfo(SExecTaskInfo* pTaskInfo, SExprSupp* pSup, SDiskbasedBuf* pBuf, SGroupResInfo* pGroupResInfo, SSHashObj* pHashmap) { int32_t numOfExprs = pSup->numOfExprs; diff --git a/source/libs/executor/src/timewindowoperator.c b/source/libs/executor/src/timewindowoperator.c index 8164281871..3817ef5b69 100644 --- a/source/libs/executor/src/timewindowoperator.c +++ b/source/libs/executor/src/timewindowoperator.c @@ -1229,10 +1229,9 @@ static void destroyStateWindowOperatorInfo(void* param) { SStateWindowOperatorInfo* pInfo = (SStateWindowOperatorInfo*)param; cleanupBasicInfo(&pInfo->binfo); taosMemoryFreeClear(pInfo->stateKey.pData); - - if (pInfo->pOperator != NULL) { - cleanupResultInfo(pInfo->pOperator->pTaskInfo, &pInfo->pOperator->exprSupp, pInfo->aggSup.pResultBuf, - &pInfo->groupResInfo, pInfo->aggSup.pResultRowHashTable); + if (pInfo->pOperator) { + cleanupResultInfoWithoutHash(pInfo->pOperator->pTaskInfo, &pInfo->pOperator->exprSupp, pInfo->aggSup.pResultBuf, + &pInfo->groupResInfo); pInfo->pOperator = NULL; } @@ -1257,10 +1256,9 @@ void destroyIntervalOperatorInfo(void* param) { SIntervalAggOperatorInfo* pInfo = (SIntervalAggOperatorInfo*)param; cleanupBasicInfo(&pInfo->binfo); - - if (pInfo->pOperator != NULL) { - cleanupResultInfo(pInfo->pOperator->pTaskInfo, &pInfo->pOperator->exprSupp, pInfo->aggSup.pResultBuf, - &pInfo->groupResInfo, pInfo->aggSup.pResultRowHashTable); + if (pInfo->pOperator) { + cleanupResultInfoWithoutHash(pInfo->pOperator->pTaskInfo, &pInfo->pOperator->exprSupp, pInfo->aggSup.pResultBuf, + &pInfo->groupResInfo); pInfo->pOperator = NULL; } @@ -1764,10 +1762,9 @@ void destroySWindowOperatorInfo(void* param) { cleanupBasicInfo(&pInfo->binfo); colDataDestroy(&pInfo->twAggSup.timeWindowData); - - if (pInfo->pOperator != NULL) { - cleanupResultInfo(pInfo->pOperator->pTaskInfo, &pInfo->pOperator->exprSupp, pInfo->aggSup.pResultBuf, - &pInfo->groupResInfo, pInfo->aggSup.pResultRowHashTable); + if (pInfo->pOperator) { + cleanupResultInfoWithoutHash(pInfo->pOperator->pTaskInfo, &pInfo->pOperator->exprSupp, pInfo->aggSup.pResultBuf, + &pInfo->groupResInfo); pInfo->pOperator = NULL; } From e88a8317afeda7c469338a59dc08b28b5f510ea4 Mon Sep 17 00:00:00 2001 From: xsren <285808407@qq.com> Date: Wed, 9 Oct 2024 09:45:56 +0800 Subject: [PATCH 27/51] fix: buff size --- include/libs/command/command.h | 2 +- include/libs/qcom/query.h | 2 +- source/client/src/clientRawBlockWrite.c | 18 ++++++----- source/dnode/mnode/impl/src/mndDb.c | 8 ++--- source/libs/command/src/command.c | 41 ++++++++++++++----------- source/libs/function/src/tudf.c | 2 +- source/libs/qcom/src/queryUtil.c | 27 ++++++++-------- 7 files changed, 53 insertions(+), 47 deletions(-) diff --git a/include/libs/command/command.h b/include/libs/command/command.h index 284f54e5ae..9fb2ca40b9 100644 --- a/include/libs/command/command.h +++ b/include/libs/command/command.h @@ -29,6 +29,6 @@ int32_t qExecExplainBegin(SQueryPlan *pDag, SExplainCtx **pCtx, int64_t startTs) int32_t qExecExplainEnd(SExplainCtx *pCtx, SRetrieveTableRsp **pRsp); int32_t qExplainUpdateExecInfo(SExplainCtx *pCtx, SExplainRsp *pRspMsg, int32_t groupId, SRetrieveTableRsp **pRsp); void qExplainFreeCtx(SExplainCtx *pCtx); -int32_t formatDurationOrKeep(char* buffer, int32_t timeInMinutes); +int32_t formatDurationOrKeep(char* buffer, int64_t bufSize, int32_t timeInMinutes); #endif diff --git a/include/libs/qcom/query.h b/include/libs/qcom/query.h index bcb1aa90d1..e0c7ff9a20 100644 --- a/include/libs/qcom/query.h +++ b/include/libs/qcom/query.h @@ -336,7 +336,7 @@ char* jobTaskStatusStr(int32_t status); SSchema createSchema(int8_t type, int32_t bytes, col_id_t colId, const char* name); void destroyQueryExecRes(SExecResult* pRes); -int32_t dataConverToStr(char* str, int type, void* buf, int32_t bufSize, int32_t* len); +int32_t dataConverToStr(char* str, int64_t capacity, int type, void* buf, int32_t bufSize, int32_t* len); void parseTagDatatoJson(void* p, char** jsonStr); int32_t cloneTableMeta(STableMeta* pSrc, STableMeta** pDst); void getColumnTypeFromMeta(STableMeta* pMeta, char* pName, ETableColumnType* pType); diff --git a/source/client/src/clientRawBlockWrite.c b/source/client/src/clientRawBlockWrite.c index d2578ac8e0..0804771a1b 100644 --- a/source/client/src/clientRawBlockWrite.c +++ b/source/client/src/clientRawBlockWrite.c @@ -458,17 +458,17 @@ static void buildChildElement(cJSON* json, SVCreateTbReq* pCreateReq) { cJSON* tvalue = NULL; if (IS_VAR_DATA_TYPE(pTagVal->type)) { char* buf = NULL; + int64_t bufSize = 0; if (pTagVal->type == TSDB_DATA_TYPE_VARBINARY) { - buf = taosMemoryCalloc(pTagVal->nData * 2 + 2 + 3, 1); - } else if (IS_VAR_DATA_TYPE(pTagVal->type)) { - buf = taosMemoryCalloc(pTagVal->nData + 3, 1); + bufSize = pTagVal->nData * 2 + 2 + 3; } else { - buf = taosMemoryCalloc(32, 1); + bufSize = pTagVal->nData + 3; } + buf = taosMemoryCalloc(bufSize, 1); RAW_NULL_CHECK(buf); if (!buf) goto end; - if (dataConverToStr(buf, pTagVal->type, pTagVal->pData, pTagVal->nData, NULL) != TSDB_CODE_SUCCESS) { + if (dataConverToStr(buf, bufSize, pTagVal->type, pTagVal->pData, pTagVal->nData, NULL) != TSDB_CODE_SUCCESS) { taosMemoryFree(buf); goto end; } @@ -738,13 +738,15 @@ static void processAlterTable(SMqMetaRsp* metaRsp, cJSON** pJson) { goto end; } } else { + int64_t bufSize = 0; if (vAlterTbReq.tagType == TSDB_DATA_TYPE_VARBINARY) { - buf = taosMemoryCalloc(vAlterTbReq.nTagVal * 2 + 2 + 3, 1); + bufSize = vAlterTbReq.nTagVal * 2 + 2 + 3; } else { - buf = taosMemoryCalloc(vAlterTbReq.nTagVal + 3, 1); + bufSize = vAlterTbReq.nTagVal + 3; } + buf = taosMemoryCalloc(bufSize, 1); RAW_NULL_CHECK(buf); - if (dataConverToStr(buf, vAlterTbReq.tagType, vAlterTbReq.pTagVal, vAlterTbReq.nTagVal, NULL) != + if (dataConverToStr(buf, bufSize, vAlterTbReq.tagType, vAlterTbReq.pTagVal, vAlterTbReq.nTagVal, NULL) != TSDB_CODE_SUCCESS) { taosMemoryFree(buf); goto end; diff --git a/source/dnode/mnode/impl/src/mndDb.c b/source/dnode/mnode/impl/src/mndDb.c index 0403029f74..7c42564f4c 100644 --- a/source/dnode/mnode/impl/src/mndDb.c +++ b/source/dnode/mnode/impl/src/mndDb.c @@ -2366,7 +2366,7 @@ static void mndDumpDbInfoData(SMnode *pMnode, SSDataBlock *pBlock, SDbObj *pDb, TAOS_CHECK_GOTO(colDataSetVal(pColInfo, rows, (const char *)strictVstr, false), &lino, _OVER); char durationVstr[128] = {0}; - int32_t len = formatDurationOrKeep(&durationVstr[VARSTR_HEADER_SIZE], pDb->cfg.daysPerFile); + int32_t len = formatDurationOrKeep(&durationVstr[VARSTR_HEADER_SIZE], sizeof(durationVstr) - VARSTR_HEADER_SIZE, pDb->cfg.daysPerFile); varDataSetLen(durationVstr, len); pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); @@ -2377,9 +2377,9 @@ static void mndDumpDbInfoData(SMnode *pMnode, SSDataBlock *pBlock, SDbObj *pDb, char keep1Str[128] = {0}; char keep2Str[128] = {0}; - int32_t lenKeep0 = formatDurationOrKeep(keep0Str, pDb->cfg.daysToKeep0); - int32_t lenKeep1 = formatDurationOrKeep(keep1Str, pDb->cfg.daysToKeep1); - int32_t lenKeep2 = formatDurationOrKeep(keep2Str, pDb->cfg.daysToKeep2); + int32_t lenKeep0 = formatDurationOrKeep(keep0Str, sizeof(keep0Str), pDb->cfg.daysToKeep0); + int32_t lenKeep1 = formatDurationOrKeep(keep1Str, sizeof(keep1Str), pDb->cfg.daysToKeep1); + int32_t lenKeep2 = formatDurationOrKeep(keep2Str, sizeof(keep2Str), pDb->cfg.daysToKeep2); if (pDb->cfg.daysToKeep0 > pDb->cfg.daysToKeep1 || pDb->cfg.daysToKeep0 > pDb->cfg.daysToKeep2) { len = sprintf(&keepVstr[VARSTR_HEADER_SIZE], "%s,%s,%s", keep1Str, keep2Str, keep0Str); diff --git a/source/libs/command/src/command.c b/source/libs/command/src/command.c index 28ff8e7436..1ab568905f 100644 --- a/source/libs/command/src/command.c +++ b/source/libs/command/src/command.c @@ -345,20 +345,19 @@ static const char* encryptAlgorithmStr(int8_t encryptAlgorithm) { return TSDB_CACHE_MODEL_NONE_STR; } -int32_t formatDurationOrKeep(char* buffer, int32_t timeInMinutes) { - if (buffer == NULL) { +int32_t formatDurationOrKeep(char* buffer, int64_t bufSize, int32_t timeInMinutes) { + if (buffer == NULL || bufSize <= 0) { return 0; } - int lMaxLen = 32; int32_t len = 0; if (timeInMinutes % 1440 == 0) { - int32_t days = timeInMinutes / 1440; - len = snprintf(buffer, lMaxLen,"%dd", days); + int32_t days = timeInMinutes / 1440; + len = snprintf(buffer, bufSize, "%dd", days); } else if (timeInMinutes % 60 == 0) { - int32_t hours = timeInMinutes / 60; - len = snprintf(buffer, lMaxLen,"%dh", hours); + int32_t hours = timeInMinutes / 60; + len = snprintf(buffer, bufSize, "%dh", hours); } else { - len = snprintf(buffer, lMaxLen,"%dm", timeInMinutes); + len = snprintf(buffer, bufSize, "%dm", timeInMinutes); } return len; } @@ -405,10 +404,10 @@ static int32_t setCreateDBResultIntoDataBlock(SSDataBlock* pBlock, char* dbName, char keep1Str[128] = {0}; char keep2Str[128] = {0}; - int32_t lenDuration = formatDurationOrKeep(durationStr, pCfg->daysPerFile); - int32_t lenKeep0 = formatDurationOrKeep(keep0Str, pCfg->daysToKeep0); - int32_t lenKeep1 = formatDurationOrKeep(keep1Str, pCfg->daysToKeep1); - int32_t lenKeep2 = formatDurationOrKeep(keep2Str, pCfg->daysToKeep2); + int32_t lenDuration = formatDurationOrKeep(durationStr, sizeof(durationStr), pCfg->daysPerFile); + int32_t lenKeep0 = formatDurationOrKeep(keep0Str, sizeof(keep0Str), pCfg->daysToKeep0); + int32_t lenKeep1 = formatDurationOrKeep(keep1Str, sizeof(keep1Str), pCfg->daysToKeep1); + int32_t lenKeep2 = formatDurationOrKeep(keep2Str, sizeof(keep2Str), pCfg->daysToKeep2); if (IS_SYS_DBNAME(dbName)) { len += snprintf(buf2 + VARSTR_HEADER_SIZE, SHOW_CREATE_DB_RESULT_FIELD2_LEN - VARSTR_HEADER_SIZE, "CREATE DATABASE `%s`", dbName); @@ -542,16 +541,16 @@ void appendTagFields(char* buf, int32_t* len, STableCfg* pCfg) { for (int32_t i = 0; i < pCfg->numOfTags; ++i) { SSchema* pSchema = pCfg->pSchemas + pCfg->numOfColumns + i; char type[32]; - snprintf(type, 32, "%s", tDataTypes[pSchema->type].name); + snprintf(type, sizeof(type), "%s", tDataTypes[pSchema->type].name); if (TSDB_DATA_TYPE_VARCHAR == pSchema->type || TSDB_DATA_TYPE_VARBINARY == pSchema->type || TSDB_DATA_TYPE_GEOMETRY == pSchema->type) { - snprintf(type + strlen(type), 32 - strlen(type), "(%d)", (int32_t)(pSchema->bytes - VARSTR_HEADER_SIZE)); + snprintf(type + strlen(type), sizeof(type) - strlen(type), "(%d)", (int32_t)(pSchema->bytes - VARSTR_HEADER_SIZE)); } else if (TSDB_DATA_TYPE_NCHAR == pSchema->type) { - snprintf(type + strlen(type), 32 - strlen(type), "(%d)", + snprintf(type + strlen(type), sizeof(type) - strlen(type), "(%d)", (int32_t)((pSchema->bytes - VARSTR_HEADER_SIZE) / TSDB_NCHAR_SIZE)); } - *len += snprintf(buf + VARSTR_HEADER_SIZE + *len, 32 - (VARSTR_HEADER_SIZE + *len), "%s`%s` %s", + *len += snprintf(buf + VARSTR_HEADER_SIZE + *len, sizeof(type) - (VARSTR_HEADER_SIZE + *len), "%s`%s` %s", ((i > 0) ? ", " : ""), pSchema->name, type); } } @@ -614,11 +613,17 @@ int32_t appendTagValues(char* buf, int32_t* len, STableCfg* pCfg) { char type = pTagVal->type; int32_t tlen = 0; + int64_t leftSize = SHOW_CREATE_TB_RESULT_FIELD2_LEN - (VARSTR_HEADER_SIZE + *len); + if (leftSize <= 0) { + qError("no enough space to store tag value, leftSize:%" PRId64, leftSize); + code = TSDB_CODE_APP_ERROR; + TAOS_CHECK_ERRNO(code); + } if (IS_VAR_DATA_TYPE(type)) { - code = dataConverToStr(buf + VARSTR_HEADER_SIZE + *len, type, pTagVal->pData, pTagVal->nData, &tlen); + code = dataConverToStr(buf + VARSTR_HEADER_SIZE + *len, leftSize, type, pTagVal->pData, pTagVal->nData, &tlen); TAOS_CHECK_ERRNO(code); } else { - code = dataConverToStr(buf + VARSTR_HEADER_SIZE + *len, type, &pTagVal->i64, tDataTypes[type].bytes, &tlen); + code = dataConverToStr(buf + VARSTR_HEADER_SIZE + *len, leftSize, type, &pTagVal->i64, tDataTypes[type].bytes, &tlen); TAOS_CHECK_ERRNO(code); } *len += tlen; diff --git a/source/libs/function/src/tudf.c b/source/libs/function/src/tudf.c index bc8237992d..74a1919850 100644 --- a/source/libs/function/src/tudf.c +++ b/source/libs/function/src/tudf.c @@ -161,7 +161,7 @@ static int32_t udfSpawnUdfd(SUdfdData *pData) { int len = strlen("TAOS_FQDN=") + strlen(taosFqdn) + 1; taosFqdnEnvItem = taosMemoryMalloc(len); if (taosFqdnEnvItem != NULL) { - TAOS_STRNCPY(taosFqdnEnvItem, "TAOS_FQDN=", len); + tstrncpy(taosFqdnEnvItem, "TAOS_FQDN=", len); TAOS_STRNCAT(taosFqdnEnvItem, taosFqdn, strlen(taosFqdn)); fnInfo("[UDFD]Succsess to set TAOS_FQDN:%s", taosFqdn); } else { diff --git a/source/libs/qcom/src/queryUtil.c b/source/libs/qcom/src/queryUtil.c index dfc2596a29..7d6b0f99a1 100644 --- a/source/libs/qcom/src/queryUtil.c +++ b/source/libs/qcom/src/queryUtil.c @@ -313,42 +313,41 @@ void destroyQueryExecRes(SExecResult* pRes) { } } // clang-format on -#define MAX_NUMERICAL_LENGTH (32) -int32_t dataConverToStr(char* str, int type, void* buf, int32_t bufSize, int32_t* len) { +int32_t dataConverToStr(char* str, int64_t capacity, int type, void* buf, int32_t bufSize, int32_t* len) { int32_t n = 0; switch (type) { case TSDB_DATA_TYPE_NULL: - n = snprintf(str, MAX_NUMERICAL_LENGTH, "null"); + n = snprintf(str, capacity, "null"); break; case TSDB_DATA_TYPE_BOOL: - n = snprintf(str, MAX_NUMERICAL_LENGTH, (*(int8_t*)buf) ? "true" : "false"); + n = snprintf(str, capacity, (*(int8_t*)buf) ? "true" : "false"); break; case TSDB_DATA_TYPE_TINYINT: - n = snprintf(str, MAX_NUMERICAL_LENGTH, "%d", *(int8_t*)buf); + n = snprintf(str, capacity, "%d", *(int8_t*)buf); break; case TSDB_DATA_TYPE_SMALLINT: - n = snprintf(str, MAX_NUMERICAL_LENGTH, "%d", *(int16_t*)buf); + n = snprintf(str, capacity, "%d", *(int16_t*)buf); break; case TSDB_DATA_TYPE_INT: - n = snprintf(str, MAX_NUMERICAL_LENGTH, "%d", *(int32_t*)buf); + n = snprintf(str, capacity, "%d", *(int32_t*)buf); break; case TSDB_DATA_TYPE_BIGINT: case TSDB_DATA_TYPE_TIMESTAMP: - n = snprintf(str, MAX_NUMERICAL_LENGTH, "%" PRId64, *(int64_t*)buf); + n = snprintf(str, capacity, "%" PRId64, *(int64_t*)buf); break; case TSDB_DATA_TYPE_FLOAT: - n = snprintf(str, MAX_NUMERICAL_LENGTH, "%e", GET_FLOAT_VAL(buf)); + n = snprintf(str, capacity, "%e", GET_FLOAT_VAL(buf)); break; case TSDB_DATA_TYPE_DOUBLE: - n = snprintf(str, MAX_NUMERICAL_LENGTH, "%e", GET_DOUBLE_VAL(buf)); + n = snprintf(str, capacity, "%e", GET_DOUBLE_VAL(buf)); break; case TSDB_DATA_TYPE_VARBINARY: { @@ -395,19 +394,19 @@ int32_t dataConverToStr(char* str, int type, void* buf, int32_t bufSize, int32_t n = length + 2; break; case TSDB_DATA_TYPE_UTINYINT: - n = snprintf(str, MAX_NUMERICAL_LENGTH, "%d", *(uint8_t*)buf); + n = snprintf(str, capacity, "%d", *(uint8_t*)buf); break; case TSDB_DATA_TYPE_USMALLINT: - n = snprintf(str, MAX_NUMERICAL_LENGTH, "%d", *(uint16_t*)buf); + n = snprintf(str, capacity, "%d", *(uint16_t*)buf); break; case TSDB_DATA_TYPE_UINT: - n = snprintf(str, MAX_NUMERICAL_LENGTH, "%u", *(uint32_t*)buf); + n = snprintf(str, capacity, "%u", *(uint32_t*)buf); break; case TSDB_DATA_TYPE_UBIGINT: - n = snprintf(str, MAX_NUMERICAL_LENGTH, "%" PRIu64, *(uint64_t*)buf); + n = snprintf(str, capacity, "%" PRIu64, *(uint64_t*)buf); break; default: From 95a26d7ee9889801e1885498cb86fd1d529054ca Mon Sep 17 00:00:00 2001 From: xsren <285808407@qq.com> Date: Wed, 9 Oct 2024 09:53:32 +0800 Subject: [PATCH 28/51] tstrncpy --- source/libs/qcom/src/querymsg.c | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/source/libs/qcom/src/querymsg.c b/source/libs/qcom/src/querymsg.c index a3abadec01..70309ace69 100644 --- a/source/libs/qcom/src/querymsg.c +++ b/source/libs/qcom/src/querymsg.c @@ -107,7 +107,7 @@ int32_t queryBuildUseDbMsg(void *input, char **msg, int32_t msgSize, int32_t *ms } SUseDbReq usedbReq = {0}; - strncpy(usedbReq.db, pInput->db, sizeof(usedbReq.db)); + tstrncpy(usedbReq.db, pInput->db, TSDB_DB_FNAME_LEN); usedbReq.db[sizeof(usedbReq.db) - 1] = 0; usedbReq.vgVersion = pInput->vgVersion; usedbReq.dbId = pInput->dbId; @@ -688,14 +688,14 @@ int32_t queryProcessTableMetaRsp(void *output, char *msg, int32_t msgSize) { } STableMetaOutput *pOut = output; - strncpy(pOut->dbFName, metaRsp.dbFName, TSDB_DB_FNAME_LEN); + tstrncpy(pOut->dbFName, metaRsp.dbFName, TSDB_DB_FNAME_LEN); pOut->dbId = metaRsp.dbId; if (metaRsp.tableType == TSDB_CHILD_TABLE) { SET_META_TYPE_BOTH_TABLE(pOut->metaType); - strncpy(pOut->ctbName, metaRsp.tbName, TSDB_TABLE_NAME_LEN); - strncpy(pOut->tbName, metaRsp.stbName, TSDB_TABLE_NAME_LEN); + tstrncpy(pOut->ctbName, metaRsp.tbName, TSDB_TABLE_NAME_LEN); + tstrncpy(pOut->tbName, metaRsp.stbName, TSDB_TABLE_NAME_LEN); pOut->ctbMeta.vgId = metaRsp.vgId; pOut->ctbMeta.tableType = metaRsp.tableType; @@ -705,7 +705,7 @@ int32_t queryProcessTableMetaRsp(void *output, char *msg, int32_t msgSize) { code = queryCreateTableMetaFromMsg(&metaRsp, true, &pOut->tbMeta); } else { SET_META_TYPE_TABLE(pOut->metaType); - strncpy(pOut->tbName, metaRsp.tbName, TSDB_TABLE_NAME_LEN); + tstrncpy(pOut->tbName, metaRsp.tbName, TSDB_TABLE_NAME_LEN); code = queryCreateTableMetaFromMsg(&metaRsp, (metaRsp.tableType == TSDB_SUPER_TABLE), &pOut->tbMeta); } @@ -744,14 +744,14 @@ static int32_t queryProcessTableNameRsp(void *output, char *msg, int32_t msgSize } STableMetaOutput *pOut = output; - strncpy(pOut->dbFName, metaRsp.dbFName, TSDB_DB_FNAME_LEN); + tstrncpy(pOut->dbFName, metaRsp.dbFName, TSDB_DB_FNAME_LEN); pOut->dbId = metaRsp.dbId; if (metaRsp.tableType == TSDB_CHILD_TABLE) { SET_META_TYPE_BOTH_TABLE(pOut->metaType); - strncpy(pOut->ctbName, metaRsp.tbName, TSDB_TABLE_NAME_LEN); - strncpy(pOut->tbName, metaRsp.stbName, TSDB_TABLE_NAME_LEN); + tstrncpy(pOut->ctbName, metaRsp.tbName, TSDB_TABLE_NAME_LEN); + tstrncpy(pOut->tbName, metaRsp.stbName, TSDB_TABLE_NAME_LEN); pOut->ctbMeta.vgId = metaRsp.vgId; pOut->ctbMeta.tableType = metaRsp.tableType; @@ -761,7 +761,7 @@ static int32_t queryProcessTableNameRsp(void *output, char *msg, int32_t msgSize code = queryCreateTableMetaExFromMsg(&metaRsp, true, &pOut->tbMeta); } else { SET_META_TYPE_TABLE(pOut->metaType); - strncpy(pOut->tbName, metaRsp.tbName, TSDB_TABLE_NAME_LEN); + tstrncpy(pOut->tbName, metaRsp.tbName, TSDB_TABLE_NAME_LEN); code = queryCreateTableMetaExFromMsg(&metaRsp, (metaRsp.tableType == TSDB_SUPER_TABLE), &pOut->tbMeta); } From 283e88196c90609e598db19da22f0fef50ce2e7d Mon Sep 17 00:00:00 2001 From: yihaoDeng Date: Wed, 9 Oct 2024 11:22:28 +0800 Subject: [PATCH 29/51] fix double free --- source/dnode/mnode/impl/src/mndIndex.c | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/source/dnode/mnode/impl/src/mndIndex.c b/source/dnode/mnode/impl/src/mndIndex.c index 0b3a0998f0..718c34e85a 100644 --- a/source/dnode/mnode/impl/src/mndIndex.c +++ b/source/dnode/mnode/impl/src/mndIndex.c @@ -157,7 +157,7 @@ static void *mndBuildDropIdxReq(SMnode *pMnode, SVgObj *pVgroup, SStbObj *pStbOb pHead->contLen = htonl(len); pHead->vgId = htonl(pVgroup->vgId); - void *pBuf = POINTER_SHIFT(pHead, sizeof(SMsgHead)); + void *pBuf = POINTER_SHIFT(pHead, sizeof(SMsgHead)); int32_t ret = 0; if ((ret = tSerializeSDropIdxReq(pBuf, len - sizeof(SMsgHead), &req)) < 0) { terrno = ret; @@ -662,6 +662,8 @@ static int32_t mndSetUpdateIdxStbCommitLogs(SMnode *pMnode, STrans *pTrans, SStb pNew->pTags = NULL; pNew->pColumns = NULL; + pNew->pCmpr = NULL; + pNew->pTags = NULL; pNew->updateTime = taosGetTimestampMs(); pNew->lock = 0; From eaf8bf0ca65a254a09bc9cead2d1b2ca284909d9 Mon Sep 17 00:00:00 2001 From: xsren <285808407@qq.com> Date: Wed, 9 Oct 2024 11:26:35 +0800 Subject: [PATCH 30/51] tstrncpy --- include/common/tglobal.h | 2 +- source/libs/function/src/tudf.c | 14 +++++++------- source/libs/function/src/udfd.c | 8 ++++---- source/libs/qcom/src/querymsg.c | 18 +++++++++--------- 4 files changed, 21 insertions(+), 21 deletions(-) diff --git a/include/common/tglobal.h b/include/common/tglobal.h index bece14c17d..cf918c6e0d 100644 --- a/include/common/tglobal.h +++ b/include/common/tglobal.h @@ -214,7 +214,7 @@ extern int64_t tsMinDiskFreeSize; // udf extern bool tsStartUdfd; extern char tsUdfdResFuncs[]; -extern char tsUdfdLdLibPath[]; +extern char tsUdfdLdLibPath[512]; // schemaless extern char tsSmlChildTableName[]; diff --git a/source/libs/function/src/tudf.c b/source/libs/function/src/tudf.c index 74a1919850..3a0cff7cf3 100644 --- a/source/libs/function/src/tudf.c +++ b/source/libs/function/src/tudf.c @@ -143,10 +143,10 @@ static int32_t udfSpawnUdfd(SUdfdData *pData) { char udfdPathLdLib[1024] = {0}; size_t udfdLdLibPathLen = strlen(tsUdfdLdLibPath); - strncpy(udfdPathLdLib, tsUdfdLdLibPath, tListLen(udfdPathLdLib)); + tstrncpy(udfdPathLdLib, tsUdfdLdLibPath, sizeof(udfdPathLdLib) < sizeof(tsUdfdLdLibPath) ? sizeof(udfdPathLdLib) : sizeof(tsUdfdLdLibPath)); udfdPathLdLib[udfdLdLibPathLen] = ':'; - strncpy(udfdPathLdLib + udfdLdLibPathLen + 1, pathTaosdLdLib, sizeof(udfdPathLdLib) - udfdLdLibPathLen - 1); + tstrncpy(udfdPathLdLib + udfdLdLibPathLen + 1, pathTaosdLdLib, sizeof(udfdPathLdLib) - udfdLdLibPathLen - 1); if (udfdLdLibPathLen + taosdLdLibPathLen < 1024) { fnInfo("[UDFD]udfd LD_LIBRARY_PATH: %s", udfdPathLdLib); } else { @@ -1073,7 +1073,7 @@ int32_t acquireUdfFuncHandle(char *udfName, UdfcFuncHandle *pHandle) { int32_t code = 0, line = 0; uv_mutex_lock(&gUdfcProxy.udfStubsMutex); SUdfcFuncStub key = {0}; - strncpy(key.udfName, udfName, TSDB_FUNC_NAME_LEN); + tstrncpy(key.udfName, udfName, TSDB_FUNC_NAME_LEN); int32_t stubIndex = taosArraySearchIdx(gUdfcProxy.udfStubs, &key, compareUdfcFuncSub, TD_EQ); if (stubIndex != -1) { SUdfcFuncStub *foundStub = taosArrayGet(gUdfcProxy.udfStubs, stubIndex); @@ -1106,7 +1106,7 @@ int32_t acquireUdfFuncHandle(char *udfName, UdfcFuncHandle *pHandle) { code = doSetupUdf(udfName, pHandle); if (code == TSDB_CODE_SUCCESS) { SUdfcFuncStub stub = {0}; - strncpy(stub.udfName, udfName, TSDB_FUNC_NAME_LEN); + tstrncpy(stub.udfName, udfName, TSDB_FUNC_NAME_LEN); stub.handle = *pHandle; ++stub.refCount; stub.createTime = taosGetTimestampUs(); @@ -1130,7 +1130,7 @@ _exit: void releaseUdfFuncHandle(char *udfName, UdfcFuncHandle handle) { uv_mutex_lock(&gUdfcProxy.udfStubsMutex); SUdfcFuncStub key = {0}; - strncpy(key.udfName, udfName, TSDB_FUNC_NAME_LEN); + tstrncpy(key.udfName, udfName, TSDB_FUNC_NAME_LEN); SUdfcFuncStub *foundStub = taosArraySearch(gUdfcProxy.udfStubs, &key, compareUdfcFuncSub, TD_EQ); SUdfcFuncStub *expiredStub = taosArraySearch(gUdfcProxy.expiredUdfStubs, &key, compareUdfcFuncSub, TD_EQ); if (!foundStub && !expiredStub) { @@ -2021,7 +2021,7 @@ int32_t doSetupUdf(char udfName[], UdfcFuncHandle *funcHandle) { task->type = UDF_TASK_SETUP; SUdfSetupRequest *req = &task->_setup.req; - strncpy(req->udfName, udfName, TSDB_FUNC_NAME_LEN); + tstrncpy(req->udfName, udfName, TSDB_FUNC_NAME_LEN); code = udfcRunUdfUvTask(task, UV_TASK_CONNECT); TAOS_CHECK_GOTO(code, &lino, _exit); @@ -2034,7 +2034,7 @@ int32_t doSetupUdf(char udfName[], UdfcFuncHandle *funcHandle) { task->session->outputType = rsp->outputType; task->session->bytes = rsp->bytes; task->session->bufSize = rsp->bufSize; - strncpy(task->session->udfName, udfName, TSDB_FUNC_NAME_LEN); + tstrncpy(task->session->udfName, udfName, TSDB_FUNC_NAME_LEN); fnInfo("successfully setup udf func handle. udfName: %s, handle: %p", udfName, task->session); *funcHandle = task->session; taosMemoryFree(task); diff --git a/source/libs/function/src/udfd.c b/source/libs/function/src/udfd.c index 255cf1a4de..c360cf6894 100644 --- a/source/libs/function/src/udfd.c +++ b/source/libs/function/src/udfd.c @@ -617,7 +617,7 @@ int32_t udfdNewUdf(SUdf **pUdf, const char *udfName) { } udfNew->refCount = 1; udfNew->lastFetchTime = taosGetTimestampMs(); - strncpy(udfNew->name, udfName, TSDB_FUNC_NAME_LEN); + tstrncpy(udfNew->name, udfName, TSDB_FUNC_NAME_LEN); udfNew->state = UDF_STATE_INIT; if (uv_mutex_init(&udfNew->lock) != 0) return TSDB_CODE_UDF_UV_EXEC_FAILURE; @@ -997,7 +997,7 @@ int32_t udfdSaveFuncBodyToFile(SFuncInfo *pFuncInfo, SUdf *udf) { udfdGetFuncBodyPath(udf, path); bool fileExist = !(taosStatFile(path, NULL, NULL, NULL) < 0); if (fileExist) { - strncpy(udf->path, path, PATH_MAX); + tstrncpy(udf->path, path, PATH_MAX); fnInfo("udfd func body file. reuse existing file %s", path); return TSDB_CODE_SUCCESS; } @@ -1017,7 +1017,7 @@ int32_t udfdSaveFuncBodyToFile(SFuncInfo *pFuncInfo, SUdf *udf) { return TSDB_CODE_FILE_CORRUPTED; } - strncpy(udf->path, path, PATH_MAX); + tstrncpy(udf->path, path, PATH_MAX); return TSDB_CODE_SUCCESS; } @@ -1612,7 +1612,7 @@ int32_t udfdInitResidentFuncs() { char *token; while ((token = strtok_r(pSave, ",", &pSave)) != NULL) { char func[TSDB_FUNC_NAME_LEN + 1] = {0}; - strncpy(func, token, TSDB_FUNC_NAME_LEN); + tstrncpy(func, token, TSDB_FUNC_NAME_LEN); fnInfo("udfd add resident function %s", func); if(taosArrayPush(global.residentFuncs, func) == NULL) { diff --git a/source/libs/qcom/src/querymsg.c b/source/libs/qcom/src/querymsg.c index 70309ace69..60c760a60e 100644 --- a/source/libs/qcom/src/querymsg.c +++ b/source/libs/qcom/src/querymsg.c @@ -207,7 +207,7 @@ int32_t queryBuildGetDBCfgMsg(void *input, char **msg, int32_t msgSize, int32_t } SDbCfgReq dbCfgReq = {0}; - strncpy(dbCfgReq.db, input, sizeof(dbCfgReq.db) - 1); + tstrncpy(dbCfgReq.db, input, TSDB_DB_FNAME_LEN); int32_t bufLen = tSerializeSDbCfgReq(NULL, 0, &dbCfgReq); void *pBuf = (*mallcFp)(bufLen); @@ -231,7 +231,7 @@ int32_t queryBuildGetIndexMsg(void *input, char **msg, int32_t msgSize, int32_t } SUserIndexReq indexReq = {0}; - strncpy(indexReq.indexFName, input, sizeof(indexReq.indexFName) - 1); + tstrncpy(indexReq.indexFName, input, TSDB_INDEX_FNAME_LEN); int32_t bufLen = tSerializeSUserIndexReq(NULL, 0, &indexReq); void *pBuf = (*mallcFp)(bufLen); @@ -293,7 +293,7 @@ int32_t queryBuildGetUserAuthMsg(void *input, char **msg, int32_t msgSize, int32 } SGetUserAuthReq req = {0}; - strncpy(req.user, input, sizeof(req.user) - 1); + tstrncpy(req.user, input, TSDB_USER_LEN); int32_t bufLen = tSerializeSGetUserAuthReq(NULL, 0, &req); void *pBuf = (*mallcFp)(bufLen); @@ -316,7 +316,7 @@ int32_t queryBuildGetTbIndexMsg(void *input, char **msg, int32_t msgSize, int32_ } STableIndexReq indexReq = {0}; - strncpy(indexReq.tbFName, input, sizeof(indexReq.tbFName) - 1); + tstrncpy(indexReq.tbFName, input, TSDB_TABLE_FNAME_LEN); int32_t bufLen = tSerializeSTableIndexReq(NULL, 0, &indexReq); void *pBuf = (*mallcFp)(bufLen); @@ -342,8 +342,8 @@ int32_t queryBuildGetTbCfgMsg(void *input, char **msg, int32_t msgSize, int32_t SBuildTableInput *pInput = input; STableCfgReq cfgReq = {0}; cfgReq.header.vgId = pInput->vgId; - strncpy(cfgReq.dbFName, pInput->dbFName, sizeof(cfgReq.dbFName) - 1); - strncpy(cfgReq.tbName, pInput->tbName, sizeof(cfgReq.tbName) - 1); + tstrncpy(cfgReq.dbFName, pInput->dbFName, TSDB_DB_FNAME_LEN); + tstrncpy(cfgReq.tbName, pInput->tbName, TSDB_TABLE_NAME_LEN); int32_t bufLen = tSerializeSTableCfgReq(NULL, 0, &cfgReq); void *pBuf = (*mallcFp)(bufLen); @@ -367,7 +367,7 @@ int32_t queryBuildGetViewMetaMsg(void *input, char **msg, int32_t msgSize, int32 } SViewMetaReq req = {0}; - strncpy(req.fullname, input, sizeof(req.fullname) - 1); + tstrncpy(req.fullname, input, TSDB_VIEW_FNAME_LEN); int32_t bufLen = tSerializeSViewMetaReq(NULL, 0, &req); void *pBuf = (*mallcFp)(bufLen); @@ -392,7 +392,7 @@ int32_t queryBuildGetTableTSMAMsg(void *input, char **msg, int32_t msgSize, int3 } STableTSMAInfoReq req = {0}; - strncpy(req.name, input, sizeof(req.name) - 1); + tstrncpy(req.name, input, TSDB_TABLE_FNAME_LEN); int32_t bufLen = tSerializeTableTSMAInfoReq(NULL, 0, &req); void * pBuf = (*mallcFp)(bufLen); @@ -417,7 +417,7 @@ int32_t queryBuildGetTSMAMsg(void *input, char **msg, int32_t msgSize, int32_t * STableTSMAInfoReq req = {0}; req.fetchingWithTsmaName = true; - strncpy(req.name, input, sizeof(req.name) - 1); + tstrncpy(req.name, input, TSDB_TABLE_FNAME_LEN); int32_t bufLen = tSerializeTableTSMAInfoReq(NULL, 0, &req); void * pBuf = (*mallcFp)(bufLen); From 3fdf6696a9f289409f004c9656a4b7019dbee02d Mon Sep 17 00:00:00 2001 From: Hongze Cheng Date: Wed, 9 Oct 2024 14:32:40 +0800 Subject: [PATCH 31/51] fix: memory leak on windows --- source/libs/tdb/src/db/tdbPage.c | 2 ++ 1 file changed, 2 insertions(+) diff --git a/source/libs/tdb/src/db/tdbPage.c b/source/libs/tdb/src/db/tdbPage.c index be391a75f1..6bc0c711ae 100644 --- a/source/libs/tdb/src/db/tdbPage.c +++ b/source/libs/tdb/src/db/tdbPage.c @@ -102,6 +102,8 @@ void tdbPageDestroy(SPage *pPage, void (*xFree)(void *arg, void *ptr), void *arg tdbOsFree(pPage->apOvfl[iOvfl]); } + TDB_DESTROY_PAGE_LOCK(pPage); + ptr = pPage->pData; xFree(arg, ptr); From 676f8759e23395d0ed5bbd6dabc2a18c6c08d197 Mon Sep 17 00:00:00 2001 From: lyh250-666 Date: Wed, 9 Oct 2024 15:37:00 +0800 Subject: [PATCH 32/51] enh:modify error code passing --- source/client/src/clientHb.c | 10 +++++----- source/client/src/clientImpl.c | 2 +- source/client/src/clientRawBlockWrite.c | 2 +- source/client/src/clientSml.c | 8 ++++---- source/client/src/clientStmt.c | 2 +- source/client/src/clientStmt2.c | 2 +- source/client/src/clientTmq.c | 2 +- source/common/src/tglobal.c | 2 +- source/common/src/tmisce.c | 6 +++--- source/dnode/mgmt/node_mgmt/src/dmMgmt.c | 2 +- source/dnode/mnode/impl/src/mndMain.c | 2 +- source/dnode/mnode/impl/src/mndSma.c | 2 +- source/dnode/mnode/impl/src/mndUser.c | 4 ++-- source/dnode/vnode/src/tsdb/tsdbRead2.c | 2 +- source/dnode/vnode/src/vnd/vnodeQuery.c | 6 +++--- source/libs/catalog/src/ctgUtil.c | 10 +++++----- source/libs/executor/src/groupcacheoperator.c | 2 +- source/libs/function/src/builtinsimpl.c | 20 +++++++++---------- source/libs/parser/src/parInsertSml.c | 6 +++--- 19 files changed, 46 insertions(+), 46 deletions(-) diff --git a/source/client/src/clientHb.c b/source/client/src/clientHb.c index 6ee6d753e4..62d8d470ba 100644 --- a/source/client/src/clientHb.c +++ b/source/client/src/clientHb.c @@ -55,7 +55,7 @@ static int32_t hbProcessUserAuthInfoRsp(void *value, int32_t valueLen, struct SC for (int32_t i = 0; i < numOfBatchs; ++i) { SGetUserAuthRsp *rsp = taosArrayGet(batchRsp.pArray, i); if (NULL == rsp) { - code = TSDB_CODE_OUT_OF_RANGE; + code = terrno; goto _return; } tscDebug("hb to update user auth, user:%s, version:%d", rsp->user, rsp->version); @@ -217,7 +217,7 @@ static int32_t hbProcessDBInfoRsp(void *value, int32_t valueLen, struct SCatalog for (int32_t i = 0; i < numOfBatchs; ++i) { SDbHbRsp *rsp = taosArrayGet(batchRsp.pArray, i); if (NULL == rsp) { - code = TSDB_CODE_OUT_OF_RANGE; + code = terrno; goto _return; } if (rsp->useDbRsp) { @@ -291,7 +291,7 @@ static int32_t hbProcessStbInfoRsp(void *value, int32_t valueLen, struct SCatalo for (int32_t i = 0; i < numOfMeta; ++i) { STableMetaRsp *rsp = taosArrayGet(hbRsp.pMetaRsp, i); if (NULL == rsp) { - code = TSDB_CODE_OUT_OF_RANGE; + code = terrno; goto _return; } if (rsp->numOfColumns < 0) { @@ -313,7 +313,7 @@ static int32_t hbProcessStbInfoRsp(void *value, int32_t valueLen, struct SCatalo for (int32_t i = 0; i < numOfIndex; ++i) { STableIndexRsp *rsp = taosArrayGet(hbRsp.pIndexRsp, i); if (NULL == rsp) { - code = TSDB_CODE_OUT_OF_RANGE; + code = terrno; goto _return; } TSC_ERR_JRET(catalogUpdateTableIndex(pCatalog, rsp)); @@ -354,7 +354,7 @@ static int32_t hbProcessViewInfoRsp(void *value, int32_t valueLen, struct SCatal for (int32_t i = 0; i < numOfMeta; ++i) { SViewMetaRsp *rsp = taosArrayGetP(hbRsp.pViewRsp, i); if (NULL == rsp) { - code = TSDB_CODE_OUT_OF_RANGE; + code = terrno; goto _return; } if (rsp->numOfCols < 0) { diff --git a/source/client/src/clientImpl.c b/source/client/src/clientImpl.c index 15bd5795e2..774cac750b 100644 --- a/source/client/src/clientImpl.c +++ b/source/client/src/clientImpl.c @@ -949,7 +949,7 @@ int32_t handleQueryExecRes(SRequestObj* pRequest, void* res, SCatalog* pCatalog, for (int32_t i = 0; i < tbNum; ++i) { STbVerInfo* tbInfo = taosArrayGet(pTbArray, i); if (NULL == tbInfo) { - code = TSDB_CODE_OUT_OF_RANGE; + code = terrno; goto _return; } STbSVersion tbSver = {.tbFName = tbInfo->tbFName, .sver = tbInfo->sversion, .tver = tbInfo->tversion}; diff --git a/source/client/src/clientRawBlockWrite.c b/source/client/src/clientRawBlockWrite.c index efd8836be4..ba846734fa 100644 --- a/source/client/src/clientRawBlockWrite.c +++ b/source/client/src/clientRawBlockWrite.c @@ -1918,7 +1918,7 @@ static int32_t tmqWriteRawMetaDataImpl(TAOS* taos, void* data, int32_t dataLen) const char* tbName = (const char*)taosArrayGetP(rspObj.dataRsp.blockTbName, rspObj.resIter); if (!tbName) { SET_ERROR_MSG("block tbname is null"); - code = TSDB_CODE_TMQ_INVALID_MSG; + code = terrno; goto end; } diff --git a/source/client/src/clientSml.c b/source/client/src/clientSml.c index f3a22bff75..e4e5a54a0b 100644 --- a/source/client/src/clientSml.c +++ b/source/client/src/clientSml.c @@ -393,7 +393,7 @@ int32_t smlProcessChildTable(SSmlHandle *info, SSmlLineInfo *elements) { tinfo->tags = taosArrayDup(info->preLineTagKV, NULL); if (tinfo->tags == NULL) { smlDestroyTableInfo(&tinfo); - return TSDB_CODE_OUT_OF_MEMORY; + return terrno; } for (size_t i = 0; i < taosArrayGetSize(info->preLineTagKV); i++) { SSmlKv *kv = (SSmlKv *)taosArrayGet(info->preLineTagKV, i); @@ -561,7 +561,7 @@ int32_t smlSetCTableName(SSmlTableInfo *oneTable, char *tbnameKey) { if (strlen(oneTable->childTableName) == 0) { SArray *dst = taosArrayDup(oneTable->tags, NULL); if (dst == NULL) { - return TSDB_CODE_OUT_OF_MEMORY; + return terrno; } if (oneTable->sTableNameLen >= TSDB_TABLE_NAME_LEN) { uError("SML:smlSetCTableName super table name is too long"); @@ -957,7 +957,7 @@ static int32_t smlCheckMeta(SSchema *schema, int32_t length, SArray *cols, bool for (; i < taosArrayGetSize(cols); i++) { SSmlKv *kv = (SSmlKv *)taosArrayGet(cols, i); if (kv == NULL) { - code = TSDB_CODE_SML_INVALID_DATA; + code = terrno; goto END; } if (taosHashGet(hashTmp, kv->key, kv->keyLen) == NULL) { @@ -1053,7 +1053,7 @@ static int32_t smlSendMetaMsg(SSmlHandle *info, SName *pName, SArray *pColumns, for (int32_t i = 0; i < pReq.numOfColumns; ++i) { SField *pField = taosArrayGet(pColumns, i); if (pField == NULL) { - code = TSDB_CODE_SML_INVALID_DATA; + code = terrno; goto end; } SFieldWithOptions fieldWithOption = {0}; diff --git a/source/client/src/clientStmt.c b/source/client/src/clientStmt.c index 866d0cc272..f3d765af2f 100644 --- a/source/client/src/clientStmt.c +++ b/source/client/src/clientStmt.c @@ -983,7 +983,7 @@ int stmtSetDbName(TAOS_STMT* stmt, const char* dbName) { taosMemoryFreeClear(pStmt->exec.pRequest->pDb); pStmt->exec.pRequest->pDb = taosStrdup(dbName); if (pStmt->exec.pRequest->pDb == NULL) { - return TSDB_CODE_OUT_OF_MEMORY; + return terrno; } return TSDB_CODE_SUCCESS; } diff --git a/source/client/src/clientStmt2.c b/source/client/src/clientStmt2.c index a0fd49ac86..841171bacf 100644 --- a/source/client/src/clientStmt2.c +++ b/source/client/src/clientStmt2.c @@ -850,7 +850,7 @@ static int stmtSetDbName2(TAOS_STMT2* stmt, const char* dbName) { taosMemoryFreeClear(pStmt->exec.pRequest->pDb); pStmt->exec.pRequest->pDb = taosStrdup(dbName); if (pStmt->exec.pRequest->pDb == NULL) { - return TSDB_CODE_OUT_OF_MEMORY; + return terrno; } return TSDB_CODE_SUCCESS; } diff --git a/source/client/src/clientTmq.c b/source/client/src/clientTmq.c index 975d14f3ee..42de2925da 100644 --- a/source/client/src/clientTmq.c +++ b/source/client/src/clientTmq.c @@ -826,7 +826,7 @@ static int32_t innerCommitAll(tmq_t* tmq, SMqCommitCbParamSet* pParamSet){ for (int32_t j = 0; j < numOfVgroups; j++) { SMqClientVg* pVg = taosArrayGet(pTopic->vgs, j); if (pVg == NULL) { - code = TSDB_CODE_INVALID_PARA; + code = terrno; goto END; } diff --git a/source/common/src/tglobal.c b/source/common/src/tglobal.c index af1a8ccfbe..ce152c8e10 100644 --- a/source/common/src/tglobal.c +++ b/source/common/src/tglobal.c @@ -362,7 +362,7 @@ static int32_t taosSplitS3Cfg(SConfig *pCfg, const char *name, char gVarible[TSD char *strDup = NULL; if ((strDup = taosStrdup(pItem->str))== NULL){ - code = TSDB_CODE_OUT_OF_MEMORY; + code = terrno; goto _exit; } diff --git a/source/common/src/tmisce.c b/source/common/src/tmisce.c index 8de557a881..10375ba857 100644 --- a/source/common/src/tmisce.c +++ b/source/common/src/tmisce.c @@ -284,7 +284,7 @@ int32_t dumpConfToDataBlock(SSDataBlock* pBlock, int32_t startCol) { SColumnInfoData* pColInfo = taosArrayGet(pBlock->pDataBlock, col++); if (pColInfo == NULL) { - code = TSDB_CODE_OUT_OF_RANGE; + code = terrno; TAOS_CHECK_GOTO(code, NULL, _exit); } @@ -297,7 +297,7 @@ int32_t dumpConfToDataBlock(SSDataBlock* pBlock, int32_t startCol) { pColInfo = taosArrayGet(pBlock->pDataBlock, col++); if (pColInfo == NULL) { - code = TSDB_CODE_OUT_OF_RANGE; + code = terrno; TAOS_CHECK_GOTO(code, NULL, _exit); } @@ -309,7 +309,7 @@ int32_t dumpConfToDataBlock(SSDataBlock* pBlock, int32_t startCol) { pColInfo = taosArrayGet(pBlock->pDataBlock, col++); if (pColInfo == NULL) { - code = TSDB_CODE_OUT_OF_RANGE; + code = terrno; TAOS_CHECK_GOTO(code, NULL, _exit); } TAOS_CHECK_GOTO(colDataSetVal(pColInfo, numOfRows, scope, false), NULL, _exit); diff --git a/source/dnode/mgmt/node_mgmt/src/dmMgmt.c b/source/dnode/mgmt/node_mgmt/src/dmMgmt.c index f77571c665..277dd2e02a 100644 --- a/source/dnode/mgmt/node_mgmt/src/dmMgmt.c +++ b/source/dnode/mgmt/node_mgmt/src/dmMgmt.c @@ -65,7 +65,7 @@ int32_t dmInitDnode(SDnode *pDnode) { snprintf(path, sizeof(path), "%s%s%s", tsDataDir, TD_DIRSEP, pWrapper->name); pWrapper->path = taosStrdup(path); if (pWrapper->path == NULL) { - code = TSDB_CODE_OUT_OF_MEMORY; + code = terrno; goto _OVER; } diff --git a/source/dnode/mnode/impl/src/mndMain.c b/source/dnode/mnode/impl/src/mndMain.c index eb855d28a8..3f160d8541 100644 --- a/source/dnode/mnode/impl/src/mndMain.c +++ b/source/dnode/mnode/impl/src/mndMain.c @@ -495,7 +495,7 @@ static int32_t mndCreateDir(SMnode *pMnode, const char *path) { int32_t code = 0; pMnode->path = taosStrdup(path); if (pMnode->path == NULL) { - code = TSDB_CODE_OUT_OF_MEMORY; + code = terrno; TAOS_RETURN(code); } diff --git a/source/dnode/mnode/impl/src/mndSma.c b/source/dnode/mnode/impl/src/mndSma.c index a258155223..a3b3ec01fb 100644 --- a/source/dnode/mnode/impl/src/mndSma.c +++ b/source/dnode/mnode/impl/src/mndSma.c @@ -2350,7 +2350,7 @@ int32_t dumpTSMAInfoFromSmaObj(const SSmaObj* pSma, const SStbObj* pDestStb, STa nodesDestroyNode(pNode); } pInfo->ast = taosStrdup(pSma->ast); - if (!pInfo->ast) code = TSDB_CODE_OUT_OF_MEMORY; + if (!pInfo->ast) code = terrno; if (code == TSDB_CODE_SUCCESS && pDestStb->numOfTags > 0) { pInfo->pTags = taosArrayInit(pDestStb->numOfTags, sizeof(SSchema)); diff --git a/source/dnode/mnode/impl/src/mndUser.c b/source/dnode/mnode/impl/src/mndUser.c index 99472ca457..63390d4772 100644 --- a/source/dnode/mnode/impl/src/mndUser.c +++ b/source/dnode/mnode/impl/src/mndUser.c @@ -594,7 +594,7 @@ int32_t mndFetchAllIpWhite(SMnode *pMnode, SHashObj **ppIpWhiteTab) { if (name == NULL) { sdbRelease(pSdb, pUser); sdbCancelFetch(pSdb, pIter); - TAOS_CHECK_GOTO(TSDB_CODE_OUT_OF_MEMORY, &lino, _OVER); + TAOS_CHECK_GOTO(terrno, &lino, _OVER); } if (taosArrayPush(pUserNames, &name) == NULL) { taosMemoryFree(name); @@ -617,7 +617,7 @@ int32_t mndFetchAllIpWhite(SMnode *pMnode, SHashObj **ppIpWhiteTab) { if (found == false) { char *name = taosStrdup(TSDB_DEFAULT_USER); if (name == NULL) { - TAOS_CHECK_GOTO(TSDB_CODE_OUT_OF_MEMORY, &lino, _OVER); + TAOS_CHECK_GOTO(terrno, &lino, _OVER); } if (taosArrayPush(pUserNames, &name) == NULL) { taosMemoryFree(name); diff --git a/source/dnode/vnode/src/tsdb/tsdbRead2.c b/source/dnode/vnode/src/tsdb/tsdbRead2.c index 36bfb56120..d4b906fe2a 100644 --- a/source/dnode/vnode/src/tsdb/tsdbRead2.c +++ b/source/dnode/vnode/src/tsdb/tsdbRead2.c @@ -596,7 +596,7 @@ static int32_t tsdbReaderCreate(SVnode* pVnode, SQueryTableDataCond* pCond, void pReader->status.pPrimaryTsCol = taosArrayGet(pReader->resBlockInfo.pResBlock->pDataBlock, pSup->slotId[0]); if (pReader->status.pPrimaryTsCol == NULL) { - code = TSDB_CODE_INVALID_PARA; + code = terrno; goto _end; } diff --git a/source/dnode/vnode/src/vnd/vnodeQuery.c b/source/dnode/vnode/src/vnd/vnodeQuery.c index d616bfd4ce..7c6a2e7313 100644 --- a/source/dnode/vnode/src/vnd/vnodeQuery.c +++ b/source/dnode/vnode/src/vnd/vnodeQuery.c @@ -254,7 +254,7 @@ int32_t vnodeGetTableCfg(SVnode *pVnode, SRpcMsg *pMsg, bool direct) { if (mer1.me.ctbEntry.commentLen > 0) { cfgRsp.pComment = taosStrdup(mer1.me.ctbEntry.comment); if (NULL == cfgRsp.pComment) { - code = TSDB_CODE_OUT_OF_MEMORY; + code = terrno; goto _exit; } } @@ -273,7 +273,7 @@ int32_t vnodeGetTableCfg(SVnode *pVnode, SRpcMsg *pMsg, bool direct) { if (mer1.me.ntbEntry.commentLen > 0) { cfgRsp.pComment = taosStrdup(mer1.me.ntbEntry.comment); if (NULL == cfgRsp.pComment) { - code = TSDB_CODE_OUT_OF_MEMORY; + code = terrno; goto _exit; } } @@ -399,7 +399,7 @@ int32_t vnodeGetBatchMeta(SVnode *pVnode, SRpcMsg *pMsg) { for (int32_t i = 0; i < msgNum; ++i) { req = taosArrayGet(batchReq.pMsgs, i); if (req == NULL) { - code = TSDB_CODE_OUT_OF_RANGE; + code = terrno; goto _exit; } diff --git a/source/libs/catalog/src/ctgUtil.c b/source/libs/catalog/src/ctgUtil.c index 86a38017bd..e7759bcc7d 100644 --- a/source/libs/catalog/src/ctgUtil.c +++ b/source/libs/catalog/src/ctgUtil.c @@ -1108,7 +1108,7 @@ int32_t ctgUpdateMsgCtx(SCtgMsgCtx* pCtx, int32_t reqType, void* out, char* targ if (target) { pCtx->target = taosStrdup(target); if (NULL == pCtx->target) { - CTG_ERR_RET(TSDB_CODE_OUT_OF_MEMORY); + CTG_ERR_RET(terrno); } } else { pCtx->target = NULL; @@ -1125,7 +1125,7 @@ int32_t ctgAddMsgCtx(SArray* pCtxs, int32_t reqType, void* out, char* target) { if (target) { ctx.target = taosStrdup(target); if (NULL == ctx.target) { - CTG_ERR_RET(TSDB_CODE_OUT_OF_MEMORY); + CTG_ERR_RET(terrno); } } @@ -1631,7 +1631,7 @@ int32_t ctgCloneVgInfo(SDBVgInfo* src, SDBVgInfo** dst) { if (NULL == (*dst)->vgArray) { taosHashCleanup((*dst)->vgHash); taosMemoryFreeClear(*dst); - CTG_ERR_RET(TSDB_CODE_OUT_OF_MEMORY); + CTG_ERR_RET(terrno); } } @@ -1698,7 +1698,7 @@ int32_t ctgCloneTableIndex(SArray* pIndex, SArray** pRes) { } pInfo->expr = taosStrdup(pInfo->expr); if (NULL == pInfo->expr) { - CTG_ERR_RET(TSDB_CODE_OUT_OF_MEMORY); + CTG_ERR_RET(terrno); } } @@ -1712,7 +1712,7 @@ int32_t ctgUpdateSendTargetInfo(SMsgSendInfo* pMsgSendInfo, int32_t msgType, cha pMsgSendInfo->target.vgId = vgId; pMsgSendInfo->target.dbFName = taosStrdup(dbFName); if (NULL == pMsgSendInfo->target.dbFName) { - CTG_ERR_RET(TSDB_CODE_OUT_OF_MEMORY); + CTG_ERR_RET(terrno); } } else { pMsgSendInfo->target.type = TARGET_TYPE_MNODE; diff --git a/source/libs/executor/src/groupcacheoperator.c b/source/libs/executor/src/groupcacheoperator.c index d785a1e619..13aff27d68 100644 --- a/source/libs/executor/src/groupcacheoperator.c +++ b/source/libs/executor/src/groupcacheoperator.c @@ -522,7 +522,7 @@ static int32_t buildGroupCacheBaseBlock(SSDataBlock** ppDst, SSDataBlock* pSrc) (*ppDst)->pDataBlock = taosArrayDup(pSrc->pDataBlock, NULL); if (NULL == (*ppDst)->pDataBlock) { taosMemoryFree(*ppDst); - return TSDB_CODE_OUT_OF_MEMORY; + return terrno; } TAOS_MEMCPY(&(*ppDst)->info, &pSrc->info, sizeof(pSrc->info)); blockDataDeepClear(*ppDst); diff --git a/source/libs/function/src/builtinsimpl.c b/source/libs/function/src/builtinsimpl.c index a7e2b28de2..a44b9e3ac2 100644 --- a/source/libs/function/src/builtinsimpl.c +++ b/source/libs/function/src/builtinsimpl.c @@ -2153,7 +2153,7 @@ int32_t percentileFinalize(SqlFunctionCtx* pCtx, SSDataBlock* pBlock) { int32_t slotId = pCtx->pExpr->base.resSchema.slotId; SColumnInfoData* pCol = taosArrayGet(pBlock->pDataBlock, slotId); if (NULL == pCol) { - code = TSDB_CODE_OUT_OF_RANGE; + code = terrno; goto _fin_error; } @@ -3676,7 +3676,7 @@ int32_t diffFunctionByRow(SArray* pCtxArray) { for (int i = 0; i < diffColNum; ++i) { SqlFunctionCtx* pCtx = *(SqlFunctionCtx**)taosArrayGet(pCtxArray, i); if (NULL == pCtx) { - code = TSDB_CODE_OUT_OF_RANGE; + code = terrno; goto _exit; } funcInputUpdate(pCtx); @@ -3690,7 +3690,7 @@ int32_t diffFunctionByRow(SArray* pCtxArray) { SqlFunctionCtx* pCtx0 = *(SqlFunctionCtx**)taosArrayGet(pCtxArray, 0); SFuncInputRow* pRow0 = (SFuncInputRow*)taosArrayGet(pRows, 0); if (NULL == pCtx0 || NULL == pRow0) { - code = TSDB_CODE_OUT_OF_RANGE; + code = terrno; goto _exit; } int32_t startOffset = pCtx0->offset; @@ -3708,7 +3708,7 @@ int32_t diffFunctionByRow(SArray* pCtxArray) { SqlFunctionCtx* pCtx = *(SqlFunctionCtx**)taosArrayGet(pCtxArray, i); SFuncInputRow* pRow = (SFuncInputRow*)taosArrayGet(pRows, i); if (NULL == pCtx || NULL == pRow) { - code = TSDB_CODE_OUT_OF_RANGE; + code = terrno; goto _exit; } code = funcInputGetNextRow(pCtx, pRow, &result); @@ -3731,7 +3731,7 @@ int32_t diffFunctionByRow(SArray* pCtxArray) { SqlFunctionCtx* pCtx = *(SqlFunctionCtx**)taosArrayGet(pCtxArray, i); SFuncInputRow* pRow = (SFuncInputRow*)taosArrayGet(pRows, i); if (NULL == pCtx || NULL == pRow) { - code = TSDB_CODE_OUT_OF_RANGE; + code = terrno; goto _exit; } if ((keepNull || hasNotNullValue) && !isFirstRow(pCtx, pRow)){ @@ -3753,7 +3753,7 @@ int32_t diffFunctionByRow(SArray* pCtxArray) { for (int i = 0; i < diffColNum; ++i) { SqlFunctionCtx* pCtx = *(SqlFunctionCtx**)taosArrayGet(pCtxArray, i); if (NULL == pCtx) { - code = TSDB_CODE_OUT_OF_RANGE; + code = terrno; goto _exit; } SResultRowEntryInfo* pResInfo = GET_RES_INFO(pCtx); @@ -4430,7 +4430,7 @@ int32_t spreadPartialFinalize(SqlFunctionCtx* pCtx, SSDataBlock* pBlock) { int32_t code = TSDB_CODE_SUCCESS; SColumnInfoData* pCol = taosArrayGet(pBlock->pDataBlock, slotId); if (NULL == pCol) { - code = TSDB_CODE_OUT_OF_RANGE; + code = terrno; goto _exit; } @@ -4620,7 +4620,7 @@ int32_t elapsedPartialFinalize(SqlFunctionCtx* pCtx, SSDataBlock* pBlock) { int32_t code = TSDB_CODE_SUCCESS; SColumnInfoData* pCol = taosArrayGet(pBlock->pDataBlock, slotId); if (NULL == pCol) { - code = TSDB_CODE_OUT_OF_RANGE; + code = terrno; goto _exit; } @@ -5003,7 +5003,7 @@ int32_t histogramPartialFinalize(SqlFunctionCtx* pCtx, SSDataBlock* pBlock) { int32_t code = TSDB_CODE_SUCCESS; SColumnInfoData* pCol = taosArrayGet(pBlock->pDataBlock, slotId); if (NULL == pCol) { - code = TSDB_CODE_OUT_OF_RANGE; + code = terrno; goto _exit; } code = colDataSetVal(pCol, pBlock->info.rows, res, false); @@ -5236,7 +5236,7 @@ int32_t hllPartialFinalize(SqlFunctionCtx* pCtx, SSDataBlock* pBlock) { int32_t code = TSDB_CODE_SUCCESS; SColumnInfoData* pCol = taosArrayGet(pBlock->pDataBlock, slotId); if (NULL == pCol) { - code = TSDB_CODE_OUT_OF_RANGE; + code = terrno; goto _exit; } diff --git a/source/libs/parser/src/parInsertSml.c b/source/libs/parser/src/parInsertSml.c index da9c9d5b8d..cca35d9c9a 100644 --- a/source/libs/parser/src/parInsertSml.c +++ b/source/libs/parser/src/parInsertSml.c @@ -113,7 +113,7 @@ static int32_t smlBuildTagRow(SArray* cols, SBoundColInfo* tags, SSchema* pSchem SSchema* pTagSchema = &pSchema[tags->pColIndex[i]]; SSmlKv* kv = taosArrayGet(cols, i); if (kv == NULL){ - code = TSDB_CODE_SML_INVALID_DATA; + code = terrno; uError("SML smlBuildTagRow error kv is null"); goto end; } @@ -381,7 +381,7 @@ int32_t smlBindData(SQuery* query, bool dataFormat, SArray* tags, SArray* colsSc for (int32_t r = 0; r < rowNum; ++r) { void* rowData = taosArrayGetP(cols, r); if (rowData == NULL) { - ret = TSDB_CODE_SML_INVALID_DATA; + ret = terrno; goto end; } // 1. set the parsed value from sql string @@ -389,7 +389,7 @@ int32_t smlBindData(SQuery* query, bool dataFormat, SArray* tags, SArray* colsSc SSchema* pColSchema = &pSchema[pTableCxt->boundColsInfo.pColIndex[c]]; SColVal* pVal = taosArrayGet(pTableCxt->pValues, pTableCxt->boundColsInfo.pColIndex[c]); if (pVal == NULL) { - ret = TSDB_CODE_SML_INVALID_DATA; + ret = terrno; goto end; } void** p = taosHashGet(rowData, pColSchema->name, strlen(pColSchema->name)); From 3924b1fbc69cd2913335e29d67293bce585393a0 Mon Sep 17 00:00:00 2001 From: Jing Sima Date: Mon, 30 Sep 2024 14:39:44 +0800 Subject: [PATCH 33/51] enh:[TD-32414] replace unsafe function in scalar and function. --- source/libs/function/src/builtins.c | 2 +- source/libs/function/src/builtinsimpl.c | 20 ++-- source/libs/function/src/functionMgt.c | 18 ++-- source/libs/function/src/tscript.c | 8 +- source/libs/scalar/src/filter.c | 93 ++++++++++++------- source/libs/scalar/src/scalar.c | 8 +- source/libs/scalar/src/sclfunc.c | 16 ++-- source/libs/scalar/src/sclvector.c | 6 +- .../libs/scalar/test/filter/filterTests.cpp | 6 +- .../libs/scalar/test/scalar/scalarTests.cpp | 2 +- 10 files changed, 102 insertions(+), 77 deletions(-) diff --git a/source/libs/function/src/builtins.c b/source/libs/function/src/builtins.c index 21fb57f5bb..643c4bba82 100644 --- a/source/libs/function/src/builtins.c +++ b/source/libs/function/src/builtins.c @@ -237,7 +237,7 @@ static int32_t addTimezoneParam(SNodeList* pList) { return terrno; } varDataSetLen(pVal->datum.p, len); - (void)strncpy(varDataVal(pVal->datum.p), pVal->literal, len); + tstrncpy(varDataVal(pVal->datum.p), pVal->literal, len + 1); code = nodesListAppend(pList, (SNode*)pVal); if (TSDB_CODE_SUCCESS != code) { diff --git a/source/libs/function/src/builtinsimpl.c b/source/libs/function/src/builtinsimpl.c index a7e2b28de2..ce4ff8d295 100644 --- a/source/libs/function/src/builtinsimpl.c +++ b/source/libs/function/src/builtinsimpl.c @@ -4970,10 +4970,10 @@ int32_t histogramFinalize(SqlFunctionCtx* pCtx, SSDataBlock* pBlock) { int32_t len; char buf[512] = {0}; if (!pInfo->normalized) { - len = sprintf(varDataVal(buf), "{\"lower_bin\":%g, \"upper_bin\":%g, \"count\":%" PRId64 "}", + len = snprintf(varDataVal(buf), sizeof(buf) - VARSTR_HEADER_SIZE, "{\"lower_bin\":%g, \"upper_bin\":%g, \"count\":%" PRId64 "}", pInfo->bins[i].lower, pInfo->bins[i].upper, pInfo->bins[i].count); } else { - len = sprintf(varDataVal(buf), "{\"lower_bin\":%g, \"upper_bin\":%g, \"count\":%lf}", pInfo->bins[i].lower, + len = snprintf(varDataVal(buf), sizeof(buf) - VARSTR_HEADER_SIZE, "{\"lower_bin\":%g, \"upper_bin\":%g, \"count\":%lf}", pInfo->bins[i].lower, pInfo->bins[i].upper, pInfo->bins[i].percentage); } varDataSetLen(buf, len); @@ -6601,7 +6601,7 @@ int32_t blockDistFinalize(SqlFunctionCtx* pCtx, SSDataBlock* pBlock) { compRatio = pData->totalSize * 100 / (double)totalRawSize; } - int32_t len = sprintf(st + VARSTR_HEADER_SIZE, + int32_t len = snprintf(varDataVal(st), sizeof(st) - VARSTR_HEADER_SIZE, "Total_Blocks=[%d] Total_Size=[%.2f KiB] Average_size=[%.2f KiB] Compression_Ratio=[%.2f %c]", pData->numOfBlocks, pData->totalSize / 1024.0, averageSize / 1024.0, compRatio, '%'); @@ -6616,7 +6616,7 @@ int32_t blockDistFinalize(SqlFunctionCtx* pCtx, SSDataBlock* pBlock) { avgRows = pData->totalRows / pData->numOfBlocks; } - len = sprintf(st + VARSTR_HEADER_SIZE, "Block_Rows=[%" PRId64 "] MinRows=[%d] MaxRows=[%d] AvgRows=[%" PRId64 "]", + len = snprintf(varDataVal(st), sizeof(st) - VARSTR_HEADER_SIZE, "Block_Rows=[%" PRId64 "] MinRows=[%d] MaxRows=[%d] AvgRows=[%" PRId64 "]", pData->totalRows, pData->minRows, pData->maxRows, avgRows); varDataSetLen(st, len); code = colDataSetVal(pColInfo, row++, st, false); @@ -6624,14 +6624,14 @@ int32_t blockDistFinalize(SqlFunctionCtx* pCtx, SSDataBlock* pBlock) { return code; } - len = sprintf(st + VARSTR_HEADER_SIZE, "Inmem_Rows=[%d] Stt_Rows=[%d] ", pData->numOfInmemRows, pData->numOfSttRows); + len = snprintf(varDataVal(st), sizeof(st) - VARSTR_HEADER_SIZE, "Inmem_Rows=[%d] Stt_Rows=[%d] ", pData->numOfInmemRows, pData->numOfSttRows); varDataSetLen(st, len); code = colDataSetVal(pColInfo, row++, st, false); if (TSDB_CODE_SUCCESS != code) { return code; } - len = sprintf(st + VARSTR_HEADER_SIZE, "Total_Tables=[%d] Total_Filesets=[%d] Total_Vgroups=[%d]", pData->numOfTables, + len = snprintf(varDataVal(st), sizeof(st) - VARSTR_HEADER_SIZE, "Total_Tables=[%d] Total_Filesets=[%d] Total_Vgroups=[%d]", pData->numOfTables, pData->numOfFiles, pData->numOfVgroups); varDataSetLen(st, len); @@ -6640,7 +6640,7 @@ int32_t blockDistFinalize(SqlFunctionCtx* pCtx, SSDataBlock* pBlock) { return code; } - len = sprintf(st + VARSTR_HEADER_SIZE, + len = snprintf(varDataVal(st), sizeof(st) - VARSTR_HEADER_SIZE, "--------------------------------------------------------------------------------"); varDataSetLen(st, len); code = colDataSetVal(pColInfo, row++, st, false); @@ -6667,7 +6667,7 @@ int32_t blockDistFinalize(SqlFunctionCtx* pCtx, SSDataBlock* pBlock) { int32_t bucketRange = ceil(((double) (pData->defMaxRows - pData->defMinRows)) / numOfBuckets); for (int32_t i = 0; i < tListLen(pData->blockRowsHisto); ++i) { - len = sprintf(st + VARSTR_HEADER_SIZE, "%04d |", pData->defMinRows + bucketRange * (i + 1)); + len = snprintf(varDataVal(st), sizeof(st) - VARSTR_HEADER_SIZE, "%04d |", pData->defMinRows + bucketRange * (i + 1)); int32_t num = 0; if (pData->blockRowsHisto[i] > 0) { @@ -6675,13 +6675,13 @@ int32_t blockDistFinalize(SqlFunctionCtx* pCtx, SSDataBlock* pBlock) { } for (int32_t j = 0; j < num; ++j) { - int32_t x = sprintf(st + VARSTR_HEADER_SIZE + len, "%c", '|'); + int32_t x = snprintf(varDataVal(st) + len, sizeof(st) - VARSTR_HEADER_SIZE - len, "%c", '|'); len += x; } if (pData->blockRowsHisto[i] > 0) { double v = pData->blockRowsHisto[i] * 100.0 / pData->numOfBlocks; - len += sprintf(st + VARSTR_HEADER_SIZE + len, " %d (%.2f%c)", pData->blockRowsHisto[i], v, '%'); + len += snprintf(varDataVal(st) + len, sizeof(st) - VARSTR_HEADER_SIZE - len, " %d (%.2f%c)", pData->blockRowsHisto[i], v, '%'); } varDataSetLen(st, len); diff --git a/source/libs/function/src/functionMgt.c b/source/libs/function/src/functionMgt.c index 886772b36c..d436925046 100644 --- a/source/libs/function/src/functionMgt.c +++ b/source/libs/function/src/functionMgt.c @@ -408,7 +408,7 @@ static int32_t createColumnByFunc(const SFunctionNode* pFunc, SColumnNode** ppCo if (NULL == *ppCol) { return code; } - (void)strcpy((*ppCol)->colName, pFunc->node.aliasName); + tstrncpy((*ppCol)->colName, pFunc->node.aliasName, TSDB_COL_NAME_LEN); (*ppCol)->node.resType = pFunc->node.resType; return TSDB_CODE_SUCCESS; } @@ -437,11 +437,11 @@ static int32_t createPartialFunction(const SFunctionNode* pSrcFunc, SFunctionNod (*pPartialFunc)->hasOriginalFunc = true; (*pPartialFunc)->originalFuncId = pSrcFunc->hasOriginalFunc ? pSrcFunc->originalFuncId : pSrcFunc->funcId; char name[TSDB_FUNC_NAME_LEN + TSDB_NAME_DELIMITER_LEN + TSDB_POINTER_PRINT_BYTES + 1] = {0}; - int32_t len = snprintf(name, sizeof(name) - 1, "%s.%p", (*pPartialFunc)->functionName, pSrcFunc); + int32_t len = snprintf(name, sizeof(name), "%s.%p", (*pPartialFunc)->functionName, pSrcFunc); if (taosHashBinary(name, len) < 0) { return TSDB_CODE_FAILED; } - (void)strncpy((*pPartialFunc)->node.aliasName, name, TSDB_COL_NAME_LEN - 1); + tstrncpy((*pPartialFunc)->node.aliasName, name, TSDB_COL_NAME_LEN); (*pPartialFunc)->hasPk = pSrcFunc->hasPk; (*pPartialFunc)->pkBytes = pSrcFunc->pkBytes; return TSDB_CODE_SUCCESS; @@ -475,7 +475,7 @@ static int32_t createMidFunction(const SFunctionNode* pSrcFunc, const SFunctionN } } if (TSDB_CODE_SUCCESS == code) { - (void)strcpy(pFunc->node.aliasName, pPartialFunc->node.aliasName); + tstrncpy(pFunc->node.aliasName, pPartialFunc->node.aliasName, TSDB_COL_NAME_LEN); } if (TSDB_CODE_SUCCESS == code) { @@ -504,7 +504,7 @@ static int32_t createMergeFunction(const SFunctionNode* pSrcFunc, const SFunctio if (fmIsSameInOutType(pSrcFunc->funcId)) { pFunc->node.resType = pSrcFunc->node.resType; } - (void)strcpy(pFunc->node.aliasName, pSrcFunc->node.aliasName); + tstrncpy(pFunc->node.aliasName, pSrcFunc->node.aliasName, TSDB_COL_NAME_LEN); } if (TSDB_CODE_SUCCESS == code) { @@ -558,8 +558,8 @@ static int32_t fmCreateStateFunc(const SFunctionNode* pFunc, SFunctionNode** pSt nodesDestroyList(pParams); return code; } - (void)strcpy((*pStateFunc)->node.aliasName, pFunc->node.aliasName); - (void)strcpy((*pStateFunc)->node.userAlias, pFunc->node.userAlias); + tstrncpy((*pStateFunc)->node.aliasName, pFunc->node.aliasName, TSDB_COL_NAME_LEN); + tstrncpy((*pStateFunc)->node.userAlias, pFunc->node.userAlias, TSDB_COL_NAME_LEN); } return TSDB_CODE_SUCCESS; } @@ -605,8 +605,8 @@ static int32_t fmCreateStateMergeFunc(SFunctionNode* pFunc, SFunctionNode** pSta nodesDestroyList(pParams); return code; } - (void)strcpy((*pStateMergeFunc)->node.aliasName, pFunc->node.aliasName); - (void)strcpy((*pStateMergeFunc)->node.userAlias, pFunc->node.userAlias); + tstrncpy((*pStateMergeFunc)->node.aliasName, pFunc->node.aliasName, TSDB_COL_NAME_LEN); + tstrncpy((*pStateMergeFunc)->node.userAlias, pFunc->node.userAlias, TSDB_COL_NAME_LEN); } return TSDB_CODE_SUCCESS; } diff --git a/source/libs/function/src/tscript.c b/source/libs/function/src/tscript.c index 768581285b..eecc66d6d6 100644 --- a/source/libs/function/src/tscript.c +++ b/source/libs/function/src/tscript.c @@ -92,7 +92,7 @@ void taosValueToLuaType(lua_State *lua, int32_t type, char *val) { int taosLoadScriptInit(void* pInit) { ScriptCtx *pCtx = pInit; char funcName[MAX_FUNC_NAME] = {0}; - sprintf(funcName, "%s_init", pCtx->funcName); + snprintf(funcName, MAX_FUNC_NAME, "%s_init", pCtx->funcName); lua_State* lua = pCtx->pEnv->lua_state; lua_getglobal(lua, funcName); @@ -106,7 +106,7 @@ void taosLoadScriptNormal(void *pInit, char *pInput, int16_t iType, int16_t iByt int64_t *ptsList, int64_t key, char* pOutput, char *ptsOutput, int32_t *numOfOutput, int16_t oType, int16_t oBytes) { ScriptCtx* pCtx = pInit; char funcName[MAX_FUNC_NAME] = {0}; - sprintf(funcName, "%s_add", pCtx->funcName); + snprintf(funcName, MAX_FUNC_NAME, "%s_add", pCtx->funcName); lua_State* lua = pCtx->pEnv->lua_state; lua_getglobal(lua, funcName); @@ -143,7 +143,7 @@ void taosLoadScriptNormal(void *pInit, char *pInput, int16_t iType, int16_t iByt void taosLoadScriptMerge(void *pInit, char* data, int32_t numOfRows, char* pOutput, int32_t* numOfOutput) { ScriptCtx *pCtx = pInit; char funcName[MAX_FUNC_NAME] = {0}; - sprintf(funcName, "%s_merge", pCtx->funcName); + snprintf(funcName, MAX_FUNC_NAME, "%s_merge", pCtx->funcName); lua_State* lua = pCtx->pEnv->lua_state; lua_getglobal(lua, funcName); @@ -167,7 +167,7 @@ void taosLoadScriptMerge(void *pInit, char* data, int32_t numOfRows, char* pOutp void taosLoadScriptFinalize(void *pInit,int64_t key, char *pOutput, int32_t* numOfOutput) { ScriptCtx *pCtx = pInit; char funcName[MAX_FUNC_NAME] = {0}; - sprintf(funcName, "%s_finalize", pCtx->funcName); + snprintf(funcName, MAX_FUNC_NAME, "%s_finalize", pCtx->funcName); lua_State* lua = pCtx->pEnv->lua_state; lua_getglobal(lua, funcName); diff --git a/source/libs/scalar/src/filter.c b/source/libs/scalar/src/filter.c index a3608cc1dc..e07ef69990 100644 --- a/source/libs/scalar/src/filter.c +++ b/source/libs/scalar/src/filter.c @@ -1764,41 +1764,41 @@ _return: return DEAL_RES_ERROR; } -int32_t fltConverToStr(char *str, int type, void *buf, int32_t bufSize, int32_t *len) { +int32_t fltConverToStr(char *str, int32_t strMaxLen, int type, void *buf, int32_t bufSize, int32_t *len) { int32_t n = 0; switch (type) { case TSDB_DATA_TYPE_NULL: - n = sprintf(str, "null"); + n = snprintf(str, strMaxLen, "null"); break; case TSDB_DATA_TYPE_BOOL: - n = sprintf(str, (*(int8_t *)buf) ? "true" : "false"); + n = snprintf(str, strMaxLen, (*(int8_t *)buf) ? "true" : "false"); break; case TSDB_DATA_TYPE_TINYINT: - n = sprintf(str, "%d", *(int8_t *)buf); + n = snprintf(str, strMaxLen, "%d", *(int8_t *)buf); break; case TSDB_DATA_TYPE_SMALLINT: - n = sprintf(str, "%d", *(int16_t *)buf); + n = snprintf(str, strMaxLen, "%d", *(int16_t *)buf); break; case TSDB_DATA_TYPE_INT: - n = sprintf(str, "%d", *(int32_t *)buf); + n = snprintf(str, strMaxLen, "%d", *(int32_t *)buf); break; case TSDB_DATA_TYPE_BIGINT: case TSDB_DATA_TYPE_TIMESTAMP: - n = sprintf(str, "%" PRId64, *(int64_t *)buf); + n = snprintf(str, strMaxLen, "%" PRId64, *(int64_t *)buf); break; case TSDB_DATA_TYPE_FLOAT: - n = sprintf(str, "%e", GET_FLOAT_VAL(buf)); + n = snprintf(str, strMaxLen, "%e", GET_FLOAT_VAL(buf)); break; case TSDB_DATA_TYPE_DOUBLE: - n = sprintf(str, "%e", GET_DOUBLE_VAL(buf)); + n = snprintf(str, strMaxLen, "%e", GET_DOUBLE_VAL(buf)); break; case TSDB_DATA_TYPE_BINARY: @@ -1817,19 +1817,19 @@ int32_t fltConverToStr(char *str, int type, void *buf, int32_t bufSize, int32_t break; case TSDB_DATA_TYPE_UTINYINT: - n = sprintf(str, "%d", *(uint8_t *)buf); + n = snprintf(str, strMaxLen, "%d", *(uint8_t *)buf); break; case TSDB_DATA_TYPE_USMALLINT: - n = sprintf(str, "%d", *(uint16_t *)buf); + n = snprintf(str, strMaxLen, "%d", *(uint16_t *)buf); break; case TSDB_DATA_TYPE_UINT: - n = sprintf(str, "%u", *(uint32_t *)buf); + n = snprintf(str, strMaxLen, "%u", *(uint32_t *)buf); break; case TSDB_DATA_TYPE_UBIGINT: - n = sprintf(str, "%" PRIu64, *(uint64_t *)buf); + n = snprintf(str, strMaxLen, "%" PRIu64, *(uint64_t *)buf); break; default: @@ -1886,8 +1886,8 @@ int32_t filterDumpInfoToString(SFilterInfo *info, const char *msg, int32_t optio SFilterField *left = FILTER_UNIT_LEFT_FIELD(info, unit); SColumnNode *refNode = (SColumnNode *)left->desc; if (unit->compare.optr <= OP_TYPE_JSON_CONTAINS) { - len = sprintf(str, "UNIT[%d] => [%d][%d] %s [", i, refNode->dataBlockId, refNode->slotId, - operatorTypeStr(unit->compare.optr)); + len += snprintf(str, sizeof(str), "UNIT[%d] => [%d][%d] %s [", i, refNode->dataBlockId, refNode->slotId, + operatorTypeStr(unit->compare.optr)); } if (unit->right.type == FLD_TYPE_VALUE && FILTER_UNIT_OPTR(unit) != OP_TYPE_IN) { @@ -1898,18 +1898,22 @@ int32_t filterDumpInfoToString(SFilterInfo *info, const char *msg, int32_t optio data += VARSTR_HEADER_SIZE; } if (data) { - FLT_ERR_RET(fltConverToStr(str + len, type, data, tlen > 32 ? 32 : tlen, &tlen)); + FLT_ERR_RET(fltConverToStr(str + len, sizeof(str) - len, type, data, tlen > 32 ? 32 : tlen, &tlen)); + len += tlen; } } else { - (void)strcat(str, "NULL"); + (void)strncat(str, "NULL", sizeof(str) - len - 1); + len += 4; } - (void)strcat(str, "]"); + (void)strncat(str, "]", sizeof(str) - len - 1); + len += 1; if (unit->compare.optr2) { - (void)strcat(str, " && "); + (void)strncat(str, " && ", sizeof(str) - len - 1); + len += 4; if (unit->compare.optr2 <= OP_TYPE_JSON_CONTAINS) { - (void)sprintf(str + strlen(str), "[%d][%d] %s [", refNode->dataBlockId, refNode->slotId, - operatorTypeStr(unit->compare.optr2)); + len += snprintf(str + len, sizeof(str) - len, "[%d][%d] %s [", refNode->dataBlockId, + refNode->slotId, operatorTypeStr(unit->compare.optr2)); } if (unit->right2.type == FLD_TYPE_VALUE && FILTER_UNIT_OPTR(unit) != OP_TYPE_IN) { @@ -1919,11 +1923,14 @@ int32_t filterDumpInfoToString(SFilterInfo *info, const char *msg, int32_t optio tlen = varDataLen(data); data += VARSTR_HEADER_SIZE; } - FLT_ERR_RET(fltConverToStr(str + strlen(str), type, data, tlen > 32 ? 32 : tlen, &tlen)); + FLT_ERR_RET(fltConverToStr(str + len, sizeof(str) - len, type, data, tlen > 32 ? 32 : tlen, &tlen)); + len += tlen; } else { - (void)strcat(str, "NULL"); + (void)strncat(str, "NULL", sizeof(str) - len - 1); + len += 4; } - (void)strcat(str, "]"); + (void)strncat(str, "]", sizeof(str) - len - 1); + len += 1; } qDebug("%s", str); // TODO @@ -1955,21 +1962,39 @@ int32_t filterDumpInfoToString(SFilterInfo *info, const char *msg, int32_t optio SFilterRangeNode *r = ctx->rs; int32_t tlen = 0; while (r) { - char str[256] = {0}; + char str[256] = {0}; + int32_t len = 0; if (FILTER_GET_FLAG(r->ra.sflag, RANGE_FLG_NULL)) { - (void)strcat(str, "(NULL)"); + (void)strncat(str, "(NULL)", sizeof(str) - len - 1); + len += 6; } else { - FILTER_GET_FLAG(r->ra.sflag, RANGE_FLG_EXCLUDE) ? strcat(str, "(") : strcat(str, "["); - FLT_ERR_RET(fltConverToStr(str + strlen(str), ctx->type, &r->ra.s, tlen > 32 ? 32 : tlen, &tlen)); - FILTER_GET_FLAG(r->ra.sflag, RANGE_FLG_EXCLUDE) ? strcat(str, ")") : strcat(str, "]"); + FILTER_GET_FLAG(r->ra.sflag, RANGE_FLG_EXCLUDE) ? + (void)strncat(str, "(", sizeof(str) - len - 1) : + (void)strncat(str, "[", sizeof(str) - len - 1); + len += 1; + FLT_ERR_RET(fltConverToStr(str + len, sizeof(str) - len, ctx->type, &r->ra.s, tlen > 32 ? 32 : tlen, &tlen)); + len += tlen; + FILTER_GET_FLAG(r->ra.sflag, RANGE_FLG_EXCLUDE) ? + (void)strncat(str, ")", sizeof(str) - len - 1) : + (void)strncat(str, "]", sizeof(str) - len - 1); + len += 1; } - (void)strcat(str, " - "); + (void)strncat(str, " - ", sizeof(str) - len - 1); + len += 3; if (FILTER_GET_FLAG(r->ra.eflag, RANGE_FLG_NULL)) { - (void)strcat(str, "(NULL)"); + (void)strncat(str, "(NULL)", sizeof(str) - len - 1); + len += 6; } else { - FILTER_GET_FLAG(r->ra.eflag, RANGE_FLG_EXCLUDE) ? strcat(str, "(") : strcat(str, "["); - FLT_ERR_RET(fltConverToStr(str + strlen(str), ctx->type, &r->ra.e, tlen > 32 ? 32 : tlen, &tlen)); - FILTER_GET_FLAG(r->ra.eflag, RANGE_FLG_EXCLUDE) ? strcat(str, ")") : strcat(str, "]"); + FILTER_GET_FLAG(r->ra.eflag, RANGE_FLG_EXCLUDE) ? + (void)strncat(str, "(", sizeof(str) - len - 1) : + (void)strncat(str, "[", sizeof(str) - len - 1); + len += 1; + FLT_ERR_RET(fltConverToStr(str + len, sizeof(str) - len, ctx->type, &r->ra.e, tlen > 32 ? 32 : tlen, &tlen)); + len += tlen; + FILTER_GET_FLAG(r->ra.eflag, RANGE_FLG_EXCLUDE) ? + (void)strncat(str, ")", sizeof(str) - len - 1) : + (void)strncat(str, "]", sizeof(str) - len - 1); + len += 1; } qDebug("range: %s", str); diff --git a/source/libs/scalar/src/scalar.c b/source/libs/scalar/src/scalar.c index 2a4951d237..209110b014 100644 --- a/source/libs/scalar/src/scalar.c +++ b/source/libs/scalar/src/scalar.c @@ -1211,7 +1211,7 @@ EDealRes sclRewriteFunction(SNode **pNode, SScalarCtx *ctx) { res->translate = true; - (void)strcpy(res->node.aliasName, node->node.aliasName); + tstrncpy(res->node.aliasName, node->node.aliasName, TSDB_COL_NAME_LEN); res->node.resType.type = output.columnData->info.type; res->node.resType.bytes = output.columnData->info.bytes; res->node.resType.scale = output.columnData->info.scale; @@ -1286,7 +1286,7 @@ EDealRes sclRewriteLogic(SNode **pNode, SScalarCtx *ctx) { res->node.resType = node->node.resType; res->translate = true; - (void)strcpy(res->node.aliasName, node->node.aliasName); + tstrncpy(res->node.aliasName, node->node.aliasName, TSDB_COL_NAME_LEN); int32_t type = output.columnData->info.type; if (IS_VAR_DATA_TYPE(type)) { res->datum.p = output.columnData->pData; @@ -1356,7 +1356,7 @@ EDealRes sclRewriteOperator(SNode **pNode, SScalarCtx *ctx) { res->translate = true; - (void)strcpy(res->node.aliasName, node->node.aliasName); + tstrncpy(res->node.aliasName, node->node.aliasName, TSDB_COL_NAME_LEN); res->node.resType = node->node.resType; if (colDataIsNull_s(output.columnData, 0)) { res->isNull = true; @@ -1419,7 +1419,7 @@ EDealRes sclRewriteCaseWhen(SNode **pNode, SScalarCtx *ctx) { res->translate = true; - (void)strcpy(res->node.aliasName, node->node.aliasName); + tstrncpy(res->node.aliasName, node->node.aliasName, TSDB_COL_NAME_LEN); res->node.resType = node->node.resType; if (colDataIsNull_s(output.columnData, 0)) { res->isNull = true; diff --git a/source/libs/scalar/src/sclfunc.c b/source/libs/scalar/src/sclfunc.c index 377009a07f..303cdc6f09 100644 --- a/source/libs/scalar/src/sclfunc.c +++ b/source/libs/scalar/src/sclfunc.c @@ -2067,9 +2067,9 @@ int32_t castFunction(SScalarParam *pInput, int32_t inputNum, SScalarParam *pOutp case TSDB_DATA_TYPE_BINARY: case TSDB_DATA_TYPE_GEOMETRY: { if (inputType == TSDB_DATA_TYPE_BOOL) { - // NOTE: sprintf will append '\0' at the end of string - int32_t len = sprintf(varDataVal(output), "%.*s", (int32_t)(outputLen - VARSTR_HEADER_SIZE), - *(int8_t *)input ? "true" : "false"); + // NOTE: snprintf will append '\0' at the end of string + int32_t len = snprintf(varDataVal(output), outputLen + TSDB_NCHAR_SIZE - VARSTR_HEADER_SIZE, "%.*s", + (int32_t)(outputLen - VARSTR_HEADER_SIZE), *(int8_t *)input ? "true" : "false"); varDataSetLen(output, len); } else if (inputType == TSDB_DATA_TYPE_BINARY) { int32_t len = TMIN(varDataLen(input), outputLen - VARSTR_HEADER_SIZE); @@ -2109,7 +2109,7 @@ int32_t castFunction(SScalarParam *pInput, int32_t inputNum, SScalarParam *pOutp int32_t len; if (inputType == TSDB_DATA_TYPE_BOOL) { char tmp[8] = {0}; - len = sprintf(tmp, "%.*s", outputCharLen, *(int8_t *)input ? "true" : "false"); + len = snprintf(tmp, sizeof(tmp), "%.*s", outputCharLen, *(int8_t *)input ? "true" : "false"); bool ret = taosMbsToUcs4(tmp, len, (TdUcs4 *)varDataVal(output), outputLen - VARSTR_HEADER_SIZE, &len); if (!ret) { code = TSDB_CODE_SCALAR_CONVERT_ERROR; @@ -4407,11 +4407,11 @@ int32_t histogramScalarFunction(SScalarParam *pInput, int32_t inputNum, SScalarP int32_t len; char buf[512] = {0}; if (!normalized) { - len = sprintf(varDataVal(buf), "{\"lower_bin\":%g, \"upper_bin\":%g, \"count\":%" PRId64 "}", bins[k].lower, - bins[k].upper, bins[k].count); + len = snprintf(varDataVal(buf), sizeof(buf) - VARSTR_HEADER_SIZE, "{\"lower_bin\":%g, \"upper_bin\":%g, \"count\":%" PRId64 "}", + bins[k].lower, bins[k].upper, bins[k].count); } else { - len = sprintf(varDataVal(buf), "{\"lower_bin\":%g, \"upper_bin\":%g, \"count\":%lf}", bins[k].lower, - bins[k].upper, bins[k].percentage); + len = snprintf(varDataVal(buf), sizeof(buf) - VARSTR_HEADER_SIZE, "{\"lower_bin\":%g, \"upper_bin\":%g, \"count\":%lf}", + bins[k].lower, bins[k].upper, bins[k].percentage); } varDataSetLen(buf, len); SCL_ERR_JRET(colDataSetVal(pOutputData, k, buf, false)); diff --git a/source/libs/scalar/src/sclvector.c b/source/libs/scalar/src/sclvector.c index 230454483d..a7c842172a 100644 --- a/source/libs/scalar/src/sclvector.c +++ b/source/libs/scalar/src/sclvector.c @@ -734,7 +734,7 @@ int32_t vectorConvertToVarData(SSclVectorConvCtx *pCtx) { int64_t value = 0; GET_TYPED_DATA(value, int64_t, pCtx->inType, colDataGetData(pInputCol, i)); - int32_t len = sprintf(varDataVal(tmp), "%" PRId64, value); + int32_t len = snprintf(varDataVal(tmp), sizeof(tmp) - VARSTR_HEADER_SIZE, "%" PRId64, value); varDataLen(tmp) = len; if (pCtx->outType == TSDB_DATA_TYPE_NCHAR) { SCL_ERR_RET(varToNchar(tmp, pCtx->pOut, i, NULL)); @@ -751,7 +751,7 @@ int32_t vectorConvertToVarData(SSclVectorConvCtx *pCtx) { uint64_t value = 0; GET_TYPED_DATA(value, uint64_t, pCtx->inType, colDataGetData(pInputCol, i)); - int32_t len = sprintf(varDataVal(tmp), "%" PRIu64, value); + int32_t len = snprintf(varDataVal(tmp), sizeof(tmp) - VARSTR_HEADER_SIZE, "%" PRIu64, value); varDataLen(tmp) = len; if (pCtx->outType == TSDB_DATA_TYPE_NCHAR) { SCL_ERR_RET(varToNchar(tmp, pCtx->pOut, i, NULL)); @@ -768,7 +768,7 @@ int32_t vectorConvertToVarData(SSclVectorConvCtx *pCtx) { double value = 0; GET_TYPED_DATA(value, double, pCtx->inType, colDataGetData(pInputCol, i)); - int32_t len = sprintf(varDataVal(tmp), "%lf", value); + int32_t len = snprintf(varDataVal(tmp), sizeof(tmp) - VARSTR_HEADER_SIZE, "%lf", value); varDataLen(tmp) = len; if (pCtx->outType == TSDB_DATA_TYPE_NCHAR) { SCL_ERR_RET(varToNchar(tmp, pCtx->pOut, i, NULL)); diff --git a/source/libs/scalar/test/filter/filterTests.cpp b/source/libs/scalar/test/filter/filterTests.cpp index 70d6f7d0ae..8bbadd0e22 100644 --- a/source/libs/scalar/test/filter/filterTests.cpp +++ b/source/libs/scalar/test/filter/filterTests.cpp @@ -55,7 +55,7 @@ void flttInitLogFile() { tsAsyncLog = 0; qDebugFlag = 159; - (void)strcpy(tsLogDir, TD_LOG_DIR_PATH); + tstrncpy(tsLogDir, TD_LOG_DIR_PATH, PATH_MAX); if (taosInitLog(defaultLogFileNamePrefix, maxLogFileNum, false) < 0) { printf("failed to open log file in directory:%s\n", tsLogDir); @@ -101,7 +101,7 @@ int32_t flttMakeColumnNode(SNode **pNode, SSDataBlock **block, int32_t dataType, rnode->node.resType.bytes = dataBytes; rnode->dataBlockId = 0; - sprintf(rnode->dbName, "%" PRIu64, dbidx++); + snprintf(rnode->dbName, TSDB_DB_NAME_LEN, "%" PRIu64, dbidx++); if (NULL == block) { rnode->slotId = 2; @@ -666,7 +666,7 @@ TEST(columnTest, binary_column_like_binary) { int32_t rowNum = sizeof(leftv) / sizeof(leftv[0]); flttMakeColumnNode(&pLeft, &src, TSDB_DATA_TYPE_BINARY, 3, rowNum, leftv); - sprintf(&rightv[2], "%s", "__0"); + snprintf(&rightv[2], sizeof(rightv) - 2, "%s", "__0"); varDataSetLen(rightv, strlen(&rightv[2])); flttMakeValueNode(&pRight, TSDB_DATA_TYPE_BINARY, rightv); flttMakeOpNode(&opNode, OP_TYPE_LIKE, TSDB_DATA_TYPE_BOOL, pLeft, pRight); diff --git a/source/libs/scalar/test/scalar/scalarTests.cpp b/source/libs/scalar/test/scalar/scalarTests.cpp index e14b772ea8..4cab644582 100644 --- a/source/libs/scalar/test/scalar/scalarTests.cpp +++ b/source/libs/scalar/test/scalar/scalarTests.cpp @@ -81,7 +81,7 @@ void scltInitLogFile() { tsAsyncLog = 0; qDebugFlag = 159; - (void)strcpy(tsLogDir, TD_LOG_DIR_PATH); + tstrncpy(tsLogDir, TD_LOG_DIR_PATH, PATH_MAX); if (taosInitLog(defaultLogFileNamePrefix, maxLogFileNum, false) < 0) { (void)printf("failed to open log file in directory:%s\n", tsLogDir); From 4dc0c2f47e9dd473869772a67c0d17af59d4c166 Mon Sep 17 00:00:00 2001 From: Shengliang Guan Date: Wed, 9 Oct 2024 17:09:43 +0800 Subject: [PATCH 34/51] feat: data analysis --- cmake/cmake.options | 13 + include/common/systable.h | 2 + include/common/tanal.h | 96 +++ include/common/tmsg.h | 54 ++ include/common/tmsgdef.h | 5 + include/libs/function/functionMgt.h | 6 + include/libs/nodes/cmdnodes.h | 15 + include/libs/nodes/plannodes.h | 22 +- include/libs/nodes/querynodes.h | 9 + include/libs/qcom/query.h | 40 +- include/libs/scalar/scalar.h | 1 + include/util/taoserror.h | 22 +- include/util/tdef.h | 13 + include/util/tjson.h | 2 + source/common/CMakeLists.txt | 4 + source/dnode/mnode/impl/inc/mndAnode.h | 32 + source/dnode/mnode/impl/inc/mndDef.h | 21 + source/libs/executor/inc/operator.h | 7 + .../libs/executor/src/eventwindowoperator.c | 16 - source/libs/executor/src/timewindowoperator.c | 4 +- source/libs/function/inc/builtinsimpl.h | 2 + source/libs/function/inc/functionMgtInt.h | 1 + source/libs/parser/inc/parAst.h | 4 + source/libs/scalar/src/sclfunc.c | 4 + source/util/src/tanal.c | 737 ++++++++++++++++++ source/util/src/terror.c | 20 + source/util/src/tjson.c | 4 + 27 files changed, 1116 insertions(+), 40 deletions(-) create mode 100644 include/common/tanal.h create mode 100644 source/dnode/mnode/impl/inc/mndAnode.h create mode 100644 source/util/src/tanal.c diff --git a/cmake/cmake.options b/cmake/cmake.options index 2158157780..e3b5782d85 100644 --- a/cmake/cmake.options +++ b/cmake/cmake.options @@ -144,6 +144,12 @@ option( OFF ) +option( + BUILD_WITH_ANALYSIS + "If build with analysis" + ON +) + ENDIF () IF(NOT TD_ENTERPRISE) @@ -151,8 +157,15 @@ MESSAGE("switch s3 off with community version") set(BUILD_S3 OFF) set(BUILD_WITH_S3 OFF) set(BUILD_WITH_COS OFF) +set(BUILD_WITH_ANALYSIS OFF) ENDIF () +IF(${BUILD_WITH_ANALYSIS}) + message("build with analysis") + set(BUILD_S3 ON) + set(BUILD_WITH_S3 ON) +ENDIF() + IF(${BUILD_S3}) IF(${BUILD_WITH_S3}) diff --git a/include/common/systable.h b/include/common/systable.h index 65b3b36af8..0acafbfc30 100644 --- a/include/common/systable.h +++ b/include/common/systable.h @@ -29,6 +29,8 @@ extern "C" { #define TSDB_INS_TABLE_QNODES "ins_qnodes" #define TSDB_INS_TABLE_BNODES "ins_bnodes" // no longer used #define TSDB_INS_TABLE_SNODES "ins_snodes" +#define TSDB_INS_TABLE_ANODES "ins_anodes" +#define TSDB_INS_TABLE_ANODES_FULL "ins_anodes_full" #define TSDB_INS_TABLE_ARBGROUPS "ins_arbgroups" #define TSDB_INS_TABLE_CLUSTER "ins_cluster" #define TSDB_INS_TABLE_DATABASES "ins_databases" diff --git a/include/common/tanal.h b/include/common/tanal.h new file mode 100644 index 0000000000..59a28ddbe3 --- /dev/null +++ b/include/common/tanal.h @@ -0,0 +1,96 @@ +/* + * Copyright (c) 2019 TAOS Data, Inc. + * + * This program is free software: you can use, redistribute, and/or modify + * it under the terms of the GNU Affero General Public License, version 3 + * or later ("AGPL"), as published by the Free Software Foundation. + * + * This program is distributed in the hope that it will be useful, but WITHOUT + * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or + * FITNESS FOR A PARTICULAR PURPOSE. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + */ + +#ifndef _TD_UTIL_ANAL_H_ +#define _TD_UTIL_ANAL_H_ + +#include "os.h" +#include "tdef.h" +#include "thash.h" +#include "tjson.h" + +#ifdef __cplusplus +extern "C" { +#endif + +#define ANAL_FORECAST_DEFAULT_PERIOD 10 +#define ANAL_FORECAST_DEFAULT_ROWS 10 +#define ANAL_FORECAST_DEFAULT_CONF 95 +#define ANAL_FORECAST_DEFAULT_ALPHA 0.05 +#define ANAL_FORECAST_DEFAULT_PARAM "diff" + +typedef struct { + EAnalAlgoType type; + int32_t anode; + int32_t urlLen; + char *url; +} SAnalUrl; + +typedef enum { + ANAL_BUF_TYPE_JSON = 0, + ANAL_BUF_TYPE_JSON_COL = 1, + ANAL_BUF_TYPE_OTHERS, +} EAnalBufType; + +typedef enum { + ANAL_HTTP_TYPE_GET = 0, + ANAL_HTTP_TYPE_POST, +} EAnalHttpType; + +typedef struct { + TdFilePtr filePtr; + char fileName[TSDB_FILENAME_LEN + 10]; + int64_t numOfRows; +} SAnalColBuf; + +typedef struct { + EAnalBufType bufType; + TdFilePtr filePtr; + char fileName[TSDB_FILENAME_LEN]; + int32_t numOfCols; + SAnalColBuf *pCols; +} SAnalBuf; + +int32_t taosAnalInit(); +void taosAnalCleanup(); +SJson *taosAnalSendReqRetJson(const char *url, EAnalHttpType type, SAnalBuf *pBuf); + +int32_t taosAnalGetAlgoUrl(const char *algoName, EAnalAlgoType type, char *url, int32_t urlLen); +bool taosAnalGetOptStr(const char *option, const char *optName, char *optValue, int32_t optMaxLen); +bool taosAnalGetOptInt(const char *option, const char *optName, int32_t *optValue); +int64_t taosAnalGetVersion(); +void taosAnalUpdate(int64_t newVer, SHashObj *pHash); + +int32_t tsosAnalBufOpen(SAnalBuf *pBuf, int32_t numOfCols); +int32_t taosAnalBufWriteOptStr(SAnalBuf *pBuf, const char *optName, const char *optVal); +int32_t taosAnalBufWriteOptInt(SAnalBuf *pBuf, const char *optName, int64_t optVal); +int32_t taosAnalBufWriteOptFloat(SAnalBuf *pBuf, const char *optName, float optVal); +int32_t taosAnalBufWriteColMeta(SAnalBuf *pBuf, int32_t colIndex, int32_t colType, const char *colName); +int32_t taosAnalBufWriteDataBegin(SAnalBuf *pBuf); +int32_t taosAnalBufWriteColBegin(SAnalBuf *pBuf, int32_t colIndex); +int32_t taosAnalBufWriteColData(SAnalBuf *pBuf, int32_t colIndex, int32_t colType, void *colValue); +int32_t taosAnalBufWriteColEnd(SAnalBuf *pBuf, int32_t colIndex); +int32_t taosAnalBufWriteDataEnd(SAnalBuf *pBuf); +int32_t taosAnalBufClose(SAnalBuf *pBuf); +void taosAnalBufDestroy(SAnalBuf *pBuf); + +const char *taosAnalAlgoStr(EAnalAlgoType algoType); +EAnalAlgoType taosAnalAlgoInt(const char *algoName); +const char *taosAnalAlgoUrlStr(EAnalAlgoType algoType); + +#ifdef __cplusplus +} +#endif +#endif /*_TD_UTIL_ANAL_H_*/ \ No newline at end of file diff --git a/include/common/tmsg.h b/include/common/tmsg.h index 1a10f02c96..ae9df866c5 100644 --- a/include/common/tmsg.h +++ b/include/common/tmsg.h @@ -159,6 +159,8 @@ typedef enum _mgmt_table { TSDB_MGMT_TABLE_ARBGROUP, TSDB_MGMT_TABLE_ENCRYPTIONS, TSDB_MGMT_TABLE_USER_FULL, + TSDB_MGMT_TABLE_ANODE, + TSDB_MGMT_TABLE_ANODE_FULL, TSDB_MGMT_TABLE_MAX, } EShowType; @@ -260,6 +262,7 @@ typedef enum ENodeType { QUERY_NODE_COUNT_WINDOW, QUERY_NODE_COLUMN_OPTIONS, QUERY_NODE_TSMA_OPTIONS, + QUERY_NODE_ANOMALY_WINDOW, // Statement nodes are used in parser and planner module. QUERY_NODE_SET_OPERATOR = 100, @@ -345,6 +348,9 @@ typedef enum ENodeType { QUERY_NODE_CREATE_VIEW_STMT, QUERY_NODE_DROP_VIEW_STMT, QUERY_NODE_CREATE_SUBTABLE_FROM_FILE_CLAUSE, + QUERY_NODE_CREATE_ANODE_STMT, + QUERY_NODE_DROP_ANODE_STMT, + QUERY_NODE_UPDATE_ANODE_STMT, // show statement nodes // see 'sysTableShowAdapter', 'SYSTABLE_SHOW_TYPE_OFFSET' @@ -386,6 +392,8 @@ typedef enum ENodeType { QUERY_NODE_SHOW_CLUSTER_MACHINES_STMT, QUERY_NODE_SHOW_ENCRYPTIONS_STMT, QUERY_NODE_SHOW_TSMAS_STMT, + QUERY_NODE_SHOW_ANODES_STMT, + QUERY_NODE_SHOW_ANODES_FULL_STMT, QUERY_NODE_CREATE_TSMA_STMT, QUERY_NODE_SHOW_CREATE_TSMA_STMT, QUERY_NODE_DROP_TSMA_STMT, @@ -408,6 +416,7 @@ typedef enum ENodeType { QUERY_NODE_LOGIC_PLAN, QUERY_NODE_LOGIC_PLAN_GROUP_CACHE, QUERY_NODE_LOGIC_PLAN_DYN_QUERY_CTRL, + QUERY_NODE_LOGIC_PLAN_FORECAST_FUNC, // physical plan node QUERY_NODE_PHYSICAL_PLAN_TAG_SCAN = 1100, @@ -458,6 +467,9 @@ typedef enum ENodeType { QUERY_NODE_PHYSICAL_PLAN_MERGE_COUNT, QUERY_NODE_PHYSICAL_PLAN_STREAM_COUNT, QUERY_NODE_PHYSICAL_PLAN_STREAM_MID_INTERVAL, + QUERY_NODE_PHYSICAL_PLAN_MERGE_ANOMALY, + QUERY_NODE_PHYSICAL_PLAN_STREAM_ANOMALY, + QUERY_NODE_PHYSICAL_PLAN_FORECAST_FUNC, } ENodeType; typedef struct { @@ -1092,6 +1104,22 @@ typedef struct { int32_t tSerializeRetrieveIpWhite(void* buf, int32_t bufLen, SRetrieveIpWhiteReq* pReq); int32_t tDeserializeRetrieveIpWhite(void* buf, int32_t bufLen, SRetrieveIpWhiteReq* pReq); +typedef struct { + int32_t dnodeId; + int64_t analVer; +} SRetrieveAnalAlgoReq; + +typedef struct { + int64_t ver; + SHashObj* hash; // algoname:algotype -> SAnalUrl +} SRetrieveAnalAlgoRsp; + +int32_t tSerializeRetrieveAnalAlgoReq(void* buf, int32_t bufLen, SRetrieveAnalAlgoReq* pReq); +int32_t tDeserializeRetrieveAnalAlgoReq(void* buf, int32_t bufLen, SRetrieveAnalAlgoReq* pReq); +int32_t tSerializeRetrieveAnalAlgoRsp(void* buf, int32_t bufLen, SRetrieveAnalAlgoRsp* pRsp); +int32_t tDeserializeRetrieveAnalAlgoRsp(void* buf, int32_t bufLen, SRetrieveAnalAlgoRsp* pRsp); +void tFreeRetrieveAnalAlgoRsp(SRetrieveAnalAlgoRsp* pRsp); + typedef struct { int8_t alterType; int8_t superUser; @@ -1766,6 +1794,7 @@ typedef struct { SArray* pVloads; // array of SVnodeLoad int32_t statusSeq; int64_t ipWhiteVer; + int64_t analVer; } SStatusReq; int32_t tSerializeSStatusReq(void* buf, int32_t bufLen, SStatusReq* pReq); @@ -1831,6 +1860,7 @@ typedef struct { SArray* pDnodeEps; // Array of SDnodeEp int32_t statusSeq; int64_t ipWhiteVer; + int64_t analVer; } SStatusRsp; int32_t tSerializeSStatusRsp(void* buf, int32_t bufLen, SStatusRsp* pRsp); @@ -2377,6 +2407,30 @@ typedef struct { int32_t tSerializeSDCreateMnodeReq(void* buf, int32_t bufLen, SDCreateMnodeReq* pReq); int32_t tDeserializeSDCreateMnodeReq(void* buf, int32_t bufLen, SDCreateMnodeReq* pReq); +typedef struct { + int32_t urlLen; + int32_t sqlLen; + char* url; + char* sql; +} SMCreateAnodeReq; + +int32_t tSerializeSMCreateAnodeReq(void* buf, int32_t bufLen, SMCreateAnodeReq* pReq); +int32_t tDeserializeSMCreateAnodeReq(void* buf, int32_t bufLen, SMCreateAnodeReq* pReq); +void tFreeSMCreateAnodeReq(SMCreateAnodeReq* pReq); + +typedef struct { + int32_t anodeId; + int32_t sqlLen; + char* sql; +} SMDropAnodeReq, SMUpdateAnodeReq; + +int32_t tSerializeSMDropAnodeReq(void* buf, int32_t bufLen, SMDropAnodeReq* pReq); +int32_t tDeserializeSMDropAnodeReq(void* buf, int32_t bufLen, SMDropAnodeReq* pReq); +void tFreeSMDropAnodeReq(SMDropAnodeReq* pReq); +int32_t tSerializeSMUpdateAnodeReq(void* buf, int32_t bufLen, SMUpdateAnodeReq* pReq); +int32_t tDeserializeSMUpdateAnodeReq(void* buf, int32_t bufLen, SMUpdateAnodeReq* pReq); +void tFreeSMUpdateAnodeReq(SMUpdateAnodeReq* pReq); + typedef struct { int32_t vgId; int32_t hbSeq; diff --git a/include/common/tmsgdef.h b/include/common/tmsgdef.h index 40464dc29a..6540e7b135 100644 --- a/include/common/tmsgdef.h +++ b/include/common/tmsgdef.h @@ -125,6 +125,11 @@ TD_DEF_MSG_TYPE(TDMT_DND_ALTER_VNODE_TYPE, "dnode-alter-vnode-type", NULL, NULL) TD_DEF_MSG_TYPE(TDMT_DND_CHECK_VNODE_LEARNER_CATCHUP, "dnode-check-vnode-learner-catchup", NULL, NULL) TD_DEF_MSG_TYPE(TDMT_DND_CREATE_ENCRYPT_KEY, "create-encrypt-key", NULL, NULL) + // mnode msg overload + TD_DEF_MSG_TYPE(TDMT_MND_CREATE_ANODE, "create-anode", NULL, NULL) + TD_DEF_MSG_TYPE(TDMT_MND_UPDATE_ANODE, "update-anode", NULL, NULL) + TD_DEF_MSG_TYPE(TDMT_MND_DROP_ANODE, "drop-anode", NULL, NULL) + TD_DEF_MSG_TYPE(TDMT_MND_RETRIEVE_ANAL_ALGO, "retrieve-anal-algo", NULL, NULL) TD_CLOSE_MSG_SEG(TDMT_DND_MSG) TD_NEW_MSG_SEG(TDMT_MND_MSG) // 1<<8 diff --git a/include/libs/function/functionMgt.h b/include/libs/function/functionMgt.h index 519207377b..e5bacf85b2 100644 --- a/include/libs/function/functionMgt.h +++ b/include/libs/function/functionMgt.h @@ -62,6 +62,7 @@ typedef enum EFunctionType { FUNCTION_TYPE_UNIQUE, FUNCTION_TYPE_STATE_COUNT, FUNCTION_TYPE_STATE_DURATION, + FUNCTION_TYPE_FORECAST, // math function FUNCTION_TYPE_ABS = 1000, @@ -149,6 +150,9 @@ typedef enum EFunctionType { FUNCTION_TYPE_TBUID, FUNCTION_TYPE_VGID, FUNCTION_TYPE_VGVER, + FUNCTION_TYPE_FORECAST_LOW, + FUNCTION_TYPE_FORECAST_HIGH, + FUNCTION_TYPE_FORECAST_ROWTS, // internal function FUNCTION_TYPE_SELECT_VALUE = 3750, @@ -263,6 +267,7 @@ bool fmIsForbidSysTableFunc(int32_t funcId); bool fmIsIntervalInterpoFunc(int32_t funcId); bool fmIsInterpFunc(int32_t funcId); bool fmIsLastRowFunc(int32_t funcId); +bool fmIsForecastFunc(int32_t funcId); bool fmIsNotNullOutputFunc(int32_t funcId); bool fmIsSelectValueFunc(int32_t funcId); bool fmIsSystemInfoFunc(int32_t funcId); @@ -272,6 +277,7 @@ bool fmIsMultiRowsFunc(int32_t funcId); bool fmIsKeepOrderFunc(int32_t funcId); bool fmIsCumulativeFunc(int32_t funcId); bool fmIsInterpPseudoColumnFunc(int32_t funcId); +bool fmIsForecastPseudoColumnFunc(int32_t funcId); bool fmIsGroupKeyFunc(int32_t funcId); bool fmIsBlockDistFunc(int32_t funcId); bool fmIsIgnoreNullFunc(int32_t funcId); diff --git a/include/libs/nodes/cmdnodes.h b/include/libs/nodes/cmdnodes.h index 2a18e800b8..ba1e21b897 100644 --- a/include/libs/nodes/cmdnodes.h +++ b/include/libs/nodes/cmdnodes.h @@ -318,6 +318,21 @@ typedef struct SAlterDnodeStmt { char value[TSDB_DNODE_VALUE_LEN]; } SAlterDnodeStmt; +typedef struct { + ENodeType type; + char url[TSDB_ANAL_ANODE_URL_LEN]; +} SCreateAnodeStmt; + +typedef struct { + ENodeType type; + int32_t anodeId; +} SDropAnodeStmt; + +typedef struct { + ENodeType type; + int32_t anodeId; +} SUpdateAnodeStmt; + typedef struct SShowStmt { ENodeType type; SNode* pDbName; // SValueNode diff --git a/include/libs/nodes/plannodes.h b/include/libs/nodes/plannodes.h index a691433ee6..8e4a3ea32b 100644 --- a/include/libs/nodes/plannodes.h +++ b/include/libs/nodes/plannodes.h @@ -204,6 +204,11 @@ typedef struct SInterpFuncLogicNode { SNode* pTimeSeries; // SColumnNode } SInterpFuncLogicNode; +typedef struct SForecastFuncLogicNode { + SLogicNode node; + SNodeList* pFuncs; +} SForecastFuncLogicNode; + typedef struct SGroupCacheLogicNode { SLogicNode node; bool grpColsMayBeNull; @@ -274,7 +279,8 @@ typedef enum EWindowType { WINDOW_TYPE_SESSION, WINDOW_TYPE_STATE, WINDOW_TYPE_EVENT, - WINDOW_TYPE_COUNT + WINDOW_TYPE_COUNT, + WINDOW_TYPE_ANOMALY } EWindowType; typedef enum EWindowAlgorithm { @@ -315,6 +321,8 @@ typedef struct SWindowLogicNode { int64_t windowCount; int64_t windowSliding; SNodeList* pTsmaSubplans; + SNode* pAnomalyExpr; + char anomalyOpt[TSDB_ANAL_ALGO_OPTION_LEN]; } SWindowLogicNode; typedef struct SFillLogicNode { @@ -507,6 +515,12 @@ typedef struct SInterpFuncPhysiNode { SNode* pTimeSeries; // SColumnNode } SInterpFuncPhysiNode; +typedef struct SForecastFuncPhysiNode { + SPhysiNode node; + SNodeList* pExprs; + SNodeList* pFuncs; +} SForecastFuncPhysiNode; + typedef struct SSortMergeJoinPhysiNode { SPhysiNode node; EJoinType joinType; @@ -704,6 +718,12 @@ typedef struct SCountWinodwPhysiNode { typedef SCountWinodwPhysiNode SStreamCountWinodwPhysiNode; +typedef struct SAnomalyWindowPhysiNode { + SWindowPhysiNode window; + SNode* pAnomalyKey; + char anomalyOpt[TSDB_ANAL_ALGO_OPTION_LEN]; +} SAnomalyWindowPhysiNode; + typedef struct SSortPhysiNode { SPhysiNode node; SNodeList* pExprs; // these are expression list of order_by_clause and parameter expression of aggregate function diff --git a/include/libs/nodes/querynodes.h b/include/libs/nodes/querynodes.h index f5567c735e..4763077ed9 100644 --- a/include/libs/nodes/querynodes.h +++ b/include/libs/nodes/querynodes.h @@ -347,6 +347,13 @@ typedef struct SCountWindowNode { int64_t windowSliding; } SCountWindowNode; +typedef struct SAnomalyWindowNode { + ENodeType type; // QUERY_NODE_ANOMALY_WINDOW + SNode* pCol; // timestamp primary key + SNode* pExpr; + char anomalyOpt[TSDB_ANAL_ALGO_OPTION_LEN]; +} SAnomalyWindowNode; + typedef enum EFillMode { FILL_MODE_NONE = 1, FILL_MODE_VALUE, @@ -442,6 +449,8 @@ typedef struct SSelectStmt { bool hasTailFunc; bool hasInterpFunc; bool hasInterpPseudoColFunc; + bool hasForecastFunc; + bool hasForecastPseudoColFunc; bool hasLastRowFunc; bool hasLastFunc; bool hasTimeLineFunc; diff --git a/include/libs/qcom/query.h b/include/libs/qcom/query.h index e0c7ff9a20..81a3952463 100644 --- a/include/libs/qcom/query.h +++ b/include/libs/qcom/query.h @@ -407,29 +407,29 @@ void* getTaskPoolWorkerCb(); #define IS_AUDIT_CTB_NAME(_ctbname) \ ((*(_ctbname) == 't') && (0 == strncmp(_ctbname, TSDB_AUDIT_CTB_OPERATION, TSDB_AUDIT_CTB_OPERATION_LEN))) -#define qFatal(...) \ - do { \ - if (qDebugFlag & DEBUG_FATAL) { \ - taosPrintLog("QRY FATAL ", DEBUG_FATAL, qDebugFlag, __VA_ARGS__); \ - } \ +#define qFatal(...) \ + do { \ + if (qDebugFlag & DEBUG_FATAL) { \ + taosPrintLog("QRY FATAL ", DEBUG_FATAL, tsLogEmbedded ? 255 : qDebugFlag, __VA_ARGS__); \ + } \ } while (0) -#define qError(...) \ - do { \ - if (qDebugFlag & DEBUG_ERROR) { \ - taosPrintLog("QRY ERROR ", DEBUG_ERROR, qDebugFlag, __VA_ARGS__); \ - } \ +#define qError(...) \ + do { \ + if (qDebugFlag & DEBUG_ERROR) { \ + taosPrintLog("QRY ERROR ", DEBUG_ERROR, tsLogEmbedded ? 255 : qDebugFlag, __VA_ARGS__); \ + } \ } while (0) -#define qWarn(...) \ - do { \ - if (qDebugFlag & DEBUG_WARN) { \ - taosPrintLog("QRY WARN ", DEBUG_WARN, qDebugFlag, __VA_ARGS__); \ - } \ +#define qWarn(...) \ + do { \ + if (qDebugFlag & DEBUG_WARN) { \ + taosPrintLog("QRY WARN ", DEBUG_WARN, tsLogEmbedded ? 255 : qDebugFlag, __VA_ARGS__); \ + } \ } while (0) -#define qInfo(...) \ - do { \ - if (qDebugFlag & DEBUG_INFO) { \ - taosPrintLog("QRY ", DEBUG_INFO, qDebugFlag, __VA_ARGS__); \ - } \ +#define qInfo(...) \ + do { \ + if (qDebugFlag & DEBUG_INFO) { \ + taosPrintLog("QRY ", DEBUG_INFO, tsLogEmbedded ? 255 : qDebugFlag, __VA_ARGS__); \ + } \ } while (0) #define qDebug(...) \ do { \ diff --git a/include/libs/scalar/scalar.h b/include/libs/scalar/scalar.h index b06b7c74c7..fd936dd087 100644 --- a/include/libs/scalar/scalar.h +++ b/include/libs/scalar/scalar.h @@ -139,6 +139,7 @@ int32_t mavgScalarFunction(SScalarParam *pInput, int32_t inputNum, SScalarParam int32_t hllScalarFunction(SScalarParam *pInput, int32_t inputNum, SScalarParam *pOutput); int32_t csumScalarFunction(SScalarParam *pInput, int32_t inputNum, SScalarParam *pOutput); int32_t diffScalarFunction(SScalarParam *pInput, int32_t inputNum, SScalarParam *pOutput); +int32_t forecastScalarFunction(SScalarParam *pInput, int32_t inputNum, SScalarParam *pOutput); int32_t stateCountScalarFunction(SScalarParam *pInput, int32_t inputNum, SScalarParam *pOutput); int32_t stateDurationScalarFunction(SScalarParam *pInput, int32_t inputNum, SScalarParam *pOutput); int32_t histogramScalarFunction(SScalarParam *pInput, int32_t inputNum, SScalarParam *pOutput); diff --git a/include/util/taoserror.h b/include/util/taoserror.h index 558902075b..603207d8c0 100644 --- a/include/util/taoserror.h +++ b/include/util/taoserror.h @@ -474,8 +474,24 @@ int32_t taosGetErrSize(); #define TSDB_CODE_DNODE_INVALID_TTL_CHG_ON_WR TAOS_DEF_ERROR_CODE(0, 0x0427) #define TSDB_CODE_DNODE_INVALID_EN_WHITELIST TAOS_DEF_ERROR_CODE(0, 0x0428) #define TSDB_CODE_DNODE_INVALID_MONITOR_PARAS TAOS_DEF_ERROR_CODE(0, 0x0429) -#define TSDB_CODE_MNODE_STOPPED TAOS_DEF_ERROR_CODE(0, 0x042A) +#define TSDB_CODE_MNODE_STOPPED TAOS_DEF_ERROR_CODE(0, 0x042A) +// anode +#define TSDB_CODE_MND_ANODE_ALREADY_EXIST TAOS_DEF_ERROR_CODE(0, 0x0430) +#define TSDB_CODE_MND_ANODE_NOT_EXIST TAOS_DEF_ERROR_CODE(0, 0x0431) +#define TSDB_CODE_MND_ANODE_TOO_LONG_URL TAOS_DEF_ERROR_CODE(0, 0x0432) +#define TSDB_CODE_MND_ANODE_INVALID_PROTOCOL TAOS_DEF_ERROR_CODE(0, 0x0433) +#define TSDB_CODE_MND_ANODE_INVALID_VERSION TAOS_DEF_ERROR_CODE(0, 0x0434) +#define TSDB_CODE_MND_ANODE_TOO_MANY_ALGO TAOS_DEF_ERROR_CODE(0, 0x0435) +#define TSDB_CODE_MND_ANODE_TOO_LONG_ALGO_NAME TAOS_DEF_ERROR_CODE(0, 0x0436) +#define TSDB_CODE_MND_ANODE_TOO_MANY_ALGO_TYPE TAOS_DEF_ERROR_CODE(0, 0x0437) + +// analysis +#define TSDB_CODE_ANAL_URL_RSP_IS_NULL TAOS_DEF_ERROR_CODE(0, 0x0440) +#define TSDB_CODE_ANAL_URL_CANT_ACCESS TAOS_DEF_ERROR_CODE(0, 0x0441) +#define TSDB_CODE_ANAL_ALGO_NOT_FOUND TAOS_DEF_ERROR_CODE(0, 0x0442) +#define TSDB_CODE_ANAL_ALGO_NOT_LOAD TAOS_DEF_ERROR_CODE(0, 0x0443) +#define TSDB_CODE_ANAL_BUF_INVALID_TYPE TAOS_DEF_ERROR_CODE(0, 0x0444) // mnode-sma #define TSDB_CODE_MND_SMA_ALREADY_EXIST TAOS_DEF_ERROR_CODE(0, 0x0480) @@ -868,6 +884,10 @@ int32_t taosGetErrSize(); #define TSDB_CODE_PAR_TAG_NAME_DUPLICATED TAOS_DEF_ERROR_CODE(0, 0x267F) #define TSDB_CODE_PAR_NOT_ALLOWED_DIFFERENT_BY_ROW_FUNC TAOS_DEF_ERROR_CODE(0, 0x2680) #define TSDB_CODE_PAR_REGULAR_EXPRESSION_ERROR TAOS_DEF_ERROR_CODE(0, 0x2681) +#define TSDB_CODE_PAR_INVALID_ANOMALY_WIN_TYPE TAOS_DEF_ERROR_CODE(0, 0x2682) +#define TSDB_CODE_PAR_INVALID_ANOMALY_WIN_COL TAOS_DEF_ERROR_CODE(0, 0x2683) +#define TSDB_CODE_PAR_INVALID_ANOMALY_WIN_OPT TAOS_DEF_ERROR_CODE(0, 0x2684) +#define TSDB_CODE_PAR_INVALID_FORECAST_CLAUSE TAOS_DEF_ERROR_CODE(0, 0x2685) #define TSDB_CODE_PAR_INTERNAL_ERROR TAOS_DEF_ERROR_CODE(0, 0x26FF) //planner diff --git a/include/util/tdef.h b/include/util/tdef.h index 46c84ab26a..768ff82ade 100644 --- a/include/util/tdef.h +++ b/include/util/tdef.h @@ -293,6 +293,12 @@ typedef enum ELogicConditionType { #define TSDB_SLOW_QUERY_SQL_LEN 512 #define TSDB_SHOW_SUBQUERY_LEN 1000 #define TSDB_LOG_VAR_LEN 32 +#define TSDB_ANAL_ANODE_URL_LEN 128 +#define TSDB_ANAL_ALGO_NAME_LEN 64 +#define TSDB_ANAL_ALGO_TYPE_LEN 24 +#define TSDB_ANAL_ALGO_KEY_LEN (TSDB_ANAL_ALGO_NAME_LEN + 9) +#define TSDB_ANAL_ALGO_URL_LEN (TSDB_ANAL_ANODE_URL_LEN + TSDB_ANAL_ALGO_TYPE_LEN + 1) +#define TSDB_ANAL_ALGO_OPTION_LEN 256 #define TSDB_MAX_EP_NUM 10 @@ -603,6 +609,13 @@ enum { RAND_ERR_MEMORY = 1, RAND_ERR_FILE = 2, RAND_ERR_NETWORK = 4 }; #define MONITOR_TAG_NAME_LEN 100 #define MONITOR_TAG_VALUE_LEN 300 #define MONITOR_METRIC_NAME_LEN 100 + +typedef enum { + ANAL_ALGO_TYPE_ANOMALY_DETECT = 0, + ANAL_ALGO_TYPE_FORECAST = 1, + ANAL_ALGO_TYPE_END, +} EAnalAlgoType; + #ifdef __cplusplus } #endif diff --git a/include/util/tjson.h b/include/util/tjson.h index b9ea72b4bb..50d1a4d438 100644 --- a/include/util/tjson.h +++ b/include/util/tjson.h @@ -68,6 +68,8 @@ int32_t tjsonAddItemToArray(SJson* pJson, SJson* pItem); SJson* tjsonGetObjectItem(const SJson* pJson, const char* pName); int32_t tjsonGetObjectName(const SJson* pJson, char** pName); int32_t tjsonGetObjectValueString(const SJson* pJson, char** pStringValue); +void tjsonGetObjectValueBigInt(const SJson* pJson, int64_t* pVal); +void tjsonGetObjectValueDouble(const SJson* pJson, double* pVal); int32_t tjsonGetStringValue(const SJson* pJson, const char* pName, char* pVal); int32_t tjsonDupStringValue(const SJson* pJson, const char* pName, char** pVal); int32_t tjsonGetBigIntValue(const SJson* pJson, const char* pName, int64_t* pVal); diff --git a/source/common/CMakeLists.txt b/source/common/CMakeLists.txt index eb3dd95e95..f01c8dcbb9 100644 --- a/source/common/CMakeLists.txt +++ b/source/common/CMakeLists.txt @@ -47,6 +47,10 @@ target_link_libraries( INTERFACE api ) +if(${BUILD_WITH_ANALYSIS}) + add_definitions(-DUSE_ANAL) +endif() + if(${BUILD_S3}) if(${BUILD_WITH_S3}) diff --git a/source/dnode/mnode/impl/inc/mndAnode.h b/source/dnode/mnode/impl/inc/mndAnode.h new file mode 100644 index 0000000000..63e8f9090e --- /dev/null +++ b/source/dnode/mnode/impl/inc/mndAnode.h @@ -0,0 +1,32 @@ +/* + * Copyright (c) 2019 TAOS Data, Inc. + * + * This program is free software: you can use, redistribute, and/or modify + * it under the terms of the GNU Affero General Public License, version 3 + * or later ("AGPL"), as published by the Free Software Foundation. + * + * This program is distributed in the hope that it will be useful, but WITHOUT + * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or + * FITNESS FOR A PARTICULAR PURPOSE. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + */ + +#ifndef _TD_MND_ANODE_H_ +#define _TD_MND_ANODE_H_ + +#include "mndInt.h" + +#ifdef __cplusplus +extern "C" { +#endif + +int32_t mndInitAnode(SMnode *pMnode); +void mndCleanupAnode(SMnode *pMnode); + +#ifdef __cplusplus +} +#endif + +#endif /*_TD_MND_ANODE_H_*/ \ No newline at end of file diff --git a/source/dnode/mnode/impl/inc/mndDef.h b/source/dnode/mnode/impl/inc/mndDef.h index 60b732f817..742db8f450 100644 --- a/source/dnode/mnode/impl/inc/mndDef.h +++ b/source/dnode/mnode/impl/inc/mndDef.h @@ -78,6 +78,9 @@ typedef enum { MND_OPER_DROP_VIEW, MND_OPER_CONFIG_CLUSTER, MND_OPER_BALANCE_VGROUP_LEADER, + MND_OPER_CREATE_ANODE, + MND_OPER_UPDATE_ANODE, + MND_OPER_DROP_ANODE } EOperType; typedef enum { @@ -232,6 +235,24 @@ typedef struct { char machineId[TSDB_MACHINE_ID_LEN + 1]; } SDnodeObj; +typedef struct { + int32_t nameLen; + char* name; +} SAnodeAlgo; + +typedef struct { + int32_t id; + int64_t createdTime; + int64_t updateTime; + int32_t version; + int32_t urlLen; + int32_t numOfAlgos; + int32_t status; + SRWLatch lock; + char* url; + SArray** algos; +} SAnodeObj; + typedef struct { int32_t id; int64_t createdTime; diff --git a/source/libs/executor/inc/operator.h b/source/libs/executor/inc/operator.h index 0df676c6e2..7dfc7080d6 100644 --- a/source/libs/executor/inc/operator.h +++ b/source/libs/executor/inc/operator.h @@ -133,6 +133,8 @@ int32_t createStreamPartitionOperatorInfo(SOperatorInfo* downstream, SStreamPart int32_t createTimeSliceOperatorInfo(SOperatorInfo* downstream, SPhysiNode* pNode, SExecTaskInfo* pTaskInfo, SOperatorInfo** pInfo); +int32_t createForecastOperatorInfo(SOperatorInfo* downstream, SPhysiNode* pNode, SExecTaskInfo* pTaskInfo, SOperatorInfo** pInfo); + int32_t createMergeJoinOperatorInfo(SOperatorInfo** pDownstream, int32_t numOfDownstream, SSortMergeJoinPhysiNode* pJoinNode, SExecTaskInfo* pTaskInfo, SOperatorInfo** pInfo); int32_t createHashJoinOperatorInfo(SOperatorInfo** pDownstream, int32_t numOfDownstream, SHashJoinPhysiNode* pJoinNode, SExecTaskInfo* pTaskInfo, SOperatorInfo** pInfo); @@ -159,6 +161,8 @@ int32_t createCountwindowOperatorInfo(SOperatorInfo* downstream, SPhysiNode* phy int32_t createGroupCacheOperatorInfo(SOperatorInfo** pDownstream, int32_t numOfDownstream, SGroupCachePhysiNode* pPhyciNode, SExecTaskInfo* pTaskInfo, SOperatorInfo** pInfo); +int32_t createAnomalywindowOperatorInfo(SOperatorInfo* downstream, SPhysiNode* physiNode, SExecTaskInfo* pTaskInfo, SOperatorInfo** pInfo); + int32_t createDynQueryCtrlOperatorInfo(SOperatorInfo** pDownstream, int32_t numOfDownstream, SDynQueryCtrlPhysiNode* pPhyciNode, SExecTaskInfo* pTaskInfo, SOperatorInfo** pInfo); // clang-format on @@ -190,6 +194,9 @@ int32_t stopTableScanOperator(SOperatorInfo* pOperator, const char* pIdSt int32_t getOperatorExplainExecInfo(struct SOperatorInfo* operatorInfo, SArray* pExecInfoList); void * getOperatorParam(int32_t opType, SOperatorParam* param, int32_t idx); +void doKeepTuple(SWindowRowsSup* pRowSup, int64_t ts, uint64_t groupId); +void doKeepNewWindowStartInfo(SWindowRowsSup* pRowSup, const int64_t* tsList, int32_t rowIndex, uint64_t groupId); + #ifdef __cplusplus } #endif diff --git a/source/libs/executor/src/eventwindowoperator.c b/source/libs/executor/src/eventwindowoperator.c index f473626953..e68a91d97d 100644 --- a/source/libs/executor/src/eventwindowoperator.c +++ b/source/libs/executor/src/eventwindowoperator.c @@ -44,22 +44,6 @@ static int32_t eventWindowAggregateNext(SOperatorInfo* pOperator, SSDataBlock** static void destroyEWindowOperatorInfo(void* param); static int32_t eventWindowAggImpl(SOperatorInfo* pOperator, SEventWindowOperatorInfo* pInfo, SSDataBlock* pBlock); -// todo : move to util -static void doKeepNewWindowStartInfo(SWindowRowsSup* pRowSup, const int64_t* tsList, int32_t rowIndex, - uint64_t groupId) { - pRowSup->startRowIndex = rowIndex; - pRowSup->numOfRows = 0; - pRowSup->win.skey = tsList[rowIndex]; - pRowSup->groupId = groupId; -} - -static void doKeepTuple(SWindowRowsSup* pRowSup, int64_t ts, uint64_t groupId) { - pRowSup->win.ekey = ts; - pRowSup->prevTs = ts; - pRowSup->numOfRows += 1; - pRowSup->groupId = groupId; -} - int32_t createEventwindowOperatorInfo(SOperatorInfo* downstream, SPhysiNode* physiNode, SExecTaskInfo* pTaskInfo, SOperatorInfo** pOptrInfo) { QRY_PARAM_CHECK(pOptrInfo); diff --git a/source/libs/executor/src/timewindowoperator.c b/source/libs/executor/src/timewindowoperator.c index 3817ef5b69..34ecda6ce7 100644 --- a/source/libs/executor/src/timewindowoperator.c +++ b/source/libs/executor/src/timewindowoperator.c @@ -89,14 +89,14 @@ static int32_t setTimeWindowOutputBuf(SResultRowInfo* pResultRowInfo, STimeWindo return setResultRowInitCtx(pResultRow, pCtx, numOfOutput, rowEntryInfoOffset); } -static void doKeepTuple(SWindowRowsSup* pRowSup, int64_t ts, uint64_t groupId) { +void doKeepTuple(SWindowRowsSup* pRowSup, int64_t ts, uint64_t groupId) { pRowSup->win.ekey = ts; pRowSup->prevTs = ts; pRowSup->numOfRows += 1; pRowSup->groupId = groupId; } -static void doKeepNewWindowStartInfo(SWindowRowsSup* pRowSup, const int64_t* tsList, int32_t rowIndex, +void doKeepNewWindowStartInfo(SWindowRowsSup* pRowSup, const int64_t* tsList, int32_t rowIndex, uint64_t groupId) { pRowSup->startRowIndex = rowIndex; pRowSup->numOfRows = 0; diff --git a/source/libs/function/inc/builtinsimpl.h b/source/libs/function/inc/builtinsimpl.h index 0b2fb70eba..77905792b8 100644 --- a/source/libs/function/inc/builtinsimpl.h +++ b/source/libs/function/inc/builtinsimpl.h @@ -138,6 +138,8 @@ int32_t diffFunctionSetup(SqlFunctionCtx* pCtx, SResultRowEntryInfo* pResInfo); int32_t diffFunction(SqlFunctionCtx* pCtx); int32_t diffFunctionByRow(SArray* pCtx); +bool getForecastConfEnv(SFunctionNode* UNUSED_PARAM(pFunc), SFuncExecEnv* pEnv); + bool getDerivativeFuncEnv(struct SFunctionNode* pFunc, SFuncExecEnv* pEnv); int32_t derivativeFuncSetup(SqlFunctionCtx* pCtx, SResultRowEntryInfo* pResInfo); int32_t derivativeFunction(SqlFunctionCtx* pCtx); diff --git a/source/libs/function/inc/functionMgtInt.h b/source/libs/function/inc/functionMgtInt.h index a50562d78d..3112245de9 100644 --- a/source/libs/function/inc/functionMgtInt.h +++ b/source/libs/function/inc/functionMgtInt.h @@ -58,6 +58,7 @@ extern "C" { #define FUNC_MGT_TSMA_FUNC FUNC_MGT_FUNC_CLASSIFICATION_MASK(29) #define FUNC_MGT_COUNT_LIKE_FUNC FUNC_MGT_FUNC_CLASSIFICATION_MASK(30) // funcs that should also return 0 when no rows found #define FUNC_MGT_PROCESS_BY_ROW FUNC_MGT_FUNC_CLASSIFICATION_MASK(31) +#define FUNC_MGT_FORECAST_PC_FUNC FUNC_MGT_FUNC_CLASSIFICATION_MASK(32) #define FUNC_MGT_TEST_MASK(val, mask) (((val) & (mask)) != 0) diff --git a/source/libs/parser/inc/parAst.h b/source/libs/parser/inc/parAst.h index a2aec77c2e..28e867965f 100644 --- a/source/libs/parser/inc/parAst.h +++ b/source/libs/parser/inc/parAst.h @@ -154,6 +154,7 @@ SNode* createSessionWindowNode(SAstCreateContext* pCxt, SNode* pCol, SNode* SNode* createStateWindowNode(SAstCreateContext* pCxt, SNode* pExpr); SNode* createEventWindowNode(SAstCreateContext* pCxt, SNode* pStartCond, SNode* pEndCond); SNode* createCountWindowNode(SAstCreateContext* pCxt, const SToken* pCountToken, const SToken* pSlidingToken); +SNode* createAnomalyWindowNode(SAstCreateContext* pCxt, SNode* pExpr, const SToken* pFuncOpt); SNode* createIntervalWindowNode(SAstCreateContext* pCxt, SNode* pInterval, SNode* pOffset, SNode* pSliding, SNode* pFill); SNode* createWindowOffsetNode(SAstCreateContext* pCxt, SNode* pStartOffset, SNode* pEndOffset); @@ -251,6 +252,9 @@ SNode* createDropUserStmt(SAstCreateContext* pCxt, SToken* pUserName); SNode* createCreateDnodeStmt(SAstCreateContext* pCxt, const SToken* pFqdn, const SToken* pPort); SNode* createDropDnodeStmt(SAstCreateContext* pCxt, const SToken* pDnode, bool force, bool unsafe); SNode* createAlterDnodeStmt(SAstCreateContext* pCxt, const SToken* pDnode, const SToken* pConfig, const SToken* pValue); +SNode* createCreateAnodeStmt(SAstCreateContext* pCxt, const SToken* pUrl); +SNode* createDropAnodeStmt(SAstCreateContext* pCxt, const SToken* pAnode); +SNode* createUpdateAnodeStmt(SAstCreateContext* pCxt, const SToken* pAnode, bool updateAll); SNode* createEncryptKeyStmt(SAstCreateContext* pCxt, const SToken* pValue); SNode* createRealTableNodeForIndexName(SAstCreateContext* pCxt, SToken* pDbName, SToken* pIndexName); SNode* createCreateIndexStmt(SAstCreateContext* pCxt, EIndexType type, bool ignoreExists, SNode* pIndexName, diff --git a/source/libs/scalar/src/sclfunc.c b/source/libs/scalar/src/sclfunc.c index 377009a07f..ff47c091b7 100644 --- a/source/libs/scalar/src/sclfunc.c +++ b/source/libs/scalar/src/sclfunc.c @@ -3972,6 +3972,10 @@ int32_t diffScalarFunction(SScalarParam *pInput, int32_t inputNum, SScalarParam return nonCalcScalarFunction(pInput, inputNum, pOutput); } +int32_t forecastScalarFunction(SScalarParam *pInput, int32_t inputNum, SScalarParam *pOutput) { + return nonCalcScalarFunction(pInput, inputNum, pOutput); +} + int32_t twaScalarFunction(SScalarParam *pInput, int32_t inputNum, SScalarParam *pOutput) { return avgScalarFunction(pInput, inputNum, pOutput); } diff --git a/source/util/src/tanal.c b/source/util/src/tanal.c new file mode 100644 index 0000000000..19d26e8a0a --- /dev/null +++ b/source/util/src/tanal.c @@ -0,0 +1,737 @@ +/* + * Copyright (c) 2019 TAOS Data, Inc. + * + * This program is free software: you can use, redistribute, and/or modify + * it under the terms of the GNU Affero General Public License, version 3 + * or later ("AGPL"), as published by the Free Software Foundation. + * + * This program is distributed in the hope that it will be useful, but WITHOUT + * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or + * FITNESS FOR A PARTICULAR PURPOSE. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + */ + +#define _DEFAULT_SOURCE +#include "tanal.h" +#include "tmsg.h" +#include "ttypes.h" +#include "tutil.h" + +#ifdef USE_ANAL +#include +#define ANAL_ALGO_SPLIT "," + +typedef struct { + int64_t ver; + SHashObj *hash; // algoname:algotype -> SAnalUrl + TdThreadMutex lock; +} SAlgoMgmt; + +typedef struct { + char *data; + int64_t dataLen; +} SCurlResp; + +static SAlgoMgmt tsAlgos = {0}; +static int32_t taosAnalBufGetCont(SAnalBuf *pBuf, char **ppCont, int64_t *pContLen); + +const char *taosAnalAlgoStr(EAnalAlgoType type) { + switch (type) { + case ANAL_ALGO_TYPE_ANOMALY_DETECT: + return "anomaly-detection"; + case ANAL_ALGO_TYPE_FORECAST: + return "forecast"; + default: + return "unknown"; + } +} + +const char *taosAnalAlgoUrlStr(EAnalAlgoType type) { + switch (type) { + case ANAL_ALGO_TYPE_ANOMALY_DETECT: + return "anomaly-detect"; + case ANAL_ALGO_TYPE_FORECAST: + return "forecast"; + default: + return "unknown"; + } +} + +EAnalAlgoType taosAnalAlgoInt(const char *name) { + for (EAnalAlgoType i = 0; i < ANAL_ALGO_TYPE_END; ++i) { + if (strcasecmp(name, taosAnalAlgoStr(i)) == 0) { + return i; + } + } + + return ANAL_ALGO_TYPE_END; +} + +int32_t taosAnalInit() { + if (curl_global_init(CURL_GLOBAL_ALL) != 0) { + uError("failed to init curl"); + return -1; + } + + tsAlgos.ver = 0; + taosThreadMutexInit(&tsAlgos.lock, NULL); + tsAlgos.hash = taosHashInit(64, MurmurHash3_32, true, HASH_ENTRY_LOCK); + if (tsAlgos.hash == NULL) { + uError("failed to init algo hash"); + return -1; + } + + uInfo("analysis env is initialized"); + return 0; +} + +static void taosAnalFreeHash(SHashObj *hash) { + void *pIter = taosHashIterate(hash, NULL); + while (pIter != NULL) { + SAnalUrl *pUrl = (SAnalUrl *)pIter; + taosMemoryFree(pUrl->url); + pIter = taosHashIterate(hash, pIter); + } + taosHashCleanup(hash); +} + +void taosAnalCleanup() { + curl_global_cleanup(); + taosThreadMutexDestroy(&tsAlgos.lock); + taosAnalFreeHash(tsAlgos.hash); + tsAlgos.hash = NULL; + uInfo("analysis env is cleaned up"); +} + +void taosAnalUpdate(int64_t newVer, SHashObj *pHash) { + if (newVer > tsAlgos.ver) { + taosThreadMutexLock(&tsAlgos.lock); + SHashObj *hash = tsAlgos.hash; + tsAlgos.ver = newVer; + tsAlgos.hash = pHash; + taosThreadMutexUnlock(&tsAlgos.lock); + taosAnalFreeHash(hash); + } else { + taosAnalFreeHash(pHash); + } +} + +bool taosAnalGetOptStr(const char *option, const char *optName, char *optValue, int32_t optMaxLen) { + char buf[TSDB_ANAL_ALGO_OPTION_LEN] = {0}; + int32_t bufLen = snprintf(buf, sizeof(buf), "%s=", optName); + + char *pos1 = strstr(option, buf); + char *pos2 = strstr(option, ANAL_ALGO_SPLIT); + if (pos1 != NULL) { + if (optMaxLen > 0) { + int32_t copyLen = optMaxLen; + if (pos2 != NULL) { + copyLen = (int32_t)(pos2 - pos1 - strlen(optName) + 1); + copyLen = MIN(copyLen, optMaxLen); + } + tstrncpy(optValue, pos1 + bufLen, copyLen); + } + return true; + } else { + return false; + } +} + +bool taosAnalGetOptInt(const char *option, const char *optName, int32_t *optValue) { + char buf[TSDB_ANAL_ALGO_OPTION_LEN] = {0}; + int32_t bufLen = snprintf(buf, sizeof(buf), "%s=", optName); + + char *pos1 = strstr(option, buf); + char *pos2 = strstr(option, ANAL_ALGO_SPLIT); + if (pos1 != NULL) { + *optValue = taosStr2Int32(pos1 + bufLen + 1, NULL, 10); + return true; + } else { + return false; + } +} + +int32_t taosAnalGetAlgoUrl(const char *algoName, EAnalAlgoType type, char *url, int32_t urlLen) { + int32_t code = 0; + char name[TSDB_ANAL_ALGO_KEY_LEN] = {0}; + int32_t nameLen = 1 + snprintf(name, sizeof(name) - 1, "%d:%s", type, algoName); + + taosThreadMutexLock(&tsAlgos.lock); + SAnalUrl *pUrl = taosHashAcquire(tsAlgos.hash, name, nameLen); + if (pUrl != NULL) { + tstrncpy(url, pUrl->url, urlLen); + uDebug("algo:%s, type:%s, url:%s", algoName, taosAnalAlgoStr(type), url); + } else { + url[0] = 0; + terrno = TSDB_CODE_ANAL_ALGO_NOT_FOUND; + code = terrno; + uError("algo:%s, type:%s, url not found", algoName, taosAnalAlgoStr(type)); + } + taosThreadMutexUnlock(&tsAlgos.lock); + + return code; +} + +int64_t taosAnalGetVersion() { return tsAlgos.ver; } + +static size_t taosCurlWriteData(char *pCont, size_t contLen, size_t nmemb, void *userdata) { + SCurlResp *pRsp = userdata; + if (contLen == 0 || nmemb == 0 || pCont == NULL) { + pRsp->dataLen = 0; + pRsp->data = NULL; + uError("curl response is received, len:%" PRId64, pRsp->dataLen); + return 0; + } + + pRsp->dataLen = (int64_t)contLen * (int64_t)nmemb; + pRsp->data = taosMemoryMalloc(pRsp->dataLen + 1); + + if (pRsp->data != NULL) { + (void)memcpy(pRsp->data, pCont, pRsp->dataLen); + pRsp->data[pRsp->dataLen] = 0; + uDebug("curl response is received, len:%" PRId64 ", content:%s", pRsp->dataLen, pRsp->data); + return pRsp->dataLen; + } else { + pRsp->dataLen = 0; + uError("failed to malloc curl response"); + return 0; + } +} + +static int32_t taosCurlGetRequest(const char *url, SCurlResp *pRsp) { + CURL *curl = NULL; + CURLcode code = 0; + + curl = curl_easy_init(); + if (curl == NULL) { + uError("failed to create curl handle"); + return -1; + } + + curl_easy_setopt(curl, CURLOPT_URL, url); + curl_easy_setopt(curl, CURLOPT_WRITEFUNCTION, taosCurlWriteData); + curl_easy_setopt(curl, CURLOPT_WRITEDATA, pRsp); + curl_easy_setopt(curl, CURLOPT_TIMEOUT_MS, 100); + + uDebug("curl get request will sent, url:%s", url); + code = curl_easy_perform(curl); + if (code != CURLE_OK) { + uError("failed to perform curl action, code:%d", code); + } + +_OVER: + if (curl != NULL) curl_easy_cleanup(curl); + return code; +} + +static int32_t taosCurlPostRequest(const char *url, SCurlResp *pRsp, const char *buf, int32_t bufLen) { + struct curl_slist *headers = NULL; + CURL *curl = NULL; + CURLcode code = 0; + + curl = curl_easy_init(); + if (curl == NULL) { + uError("failed to create curl handle"); + return -1; + } + + headers = curl_slist_append(headers, "Content-Type:application/json;charset=UTF-8"); + curl_easy_setopt(curl, CURLOPT_HTTPHEADER, headers); + curl_easy_setopt(curl, CURLOPT_URL, url); + curl_easy_setopt(curl, CURLOPT_WRITEFUNCTION, taosCurlWriteData); + curl_easy_setopt(curl, CURLOPT_WRITEDATA, pRsp); + curl_easy_setopt(curl, CURLOPT_TIMEOUT_MS, 60000); + curl_easy_setopt(curl, CURLOPT_POST, 1); + curl_easy_setopt(curl, CURLOPT_POSTFIELDSIZE, bufLen); + curl_easy_setopt(curl, CURLOPT_POSTFIELDS, buf); + + uDebug("curl post request will sent, url:%s len:%d", url, bufLen); + code = curl_easy_perform(curl); + if (code != CURLE_OK) { + uError("failed to perform curl action, code:%d", code); + } + +_OVER: + if (curl != NULL) { + curl_slist_free_all(headers); + curl_easy_cleanup(curl); + } + return code; +} + +SJson *taosAnalSendReqRetJson(const char *url, EAnalHttpType type, SAnalBuf *pBuf) { + int32_t code = -1; + char *pCont = NULL; + int64_t contentLen; + SJson *pJson = NULL; + SCurlResp curlRsp = {0}; + + if (type == ANAL_HTTP_TYPE_GET) { + if (taosCurlGetRequest(url, &curlRsp) != 0) { + terrno = TSDB_CODE_ANAL_URL_CANT_ACCESS; + goto _OVER; + } + } else { + code = taosAnalBufGetCont(pBuf, &pCont, &contentLen); + if (code != 0) { + terrno = code; + goto _OVER; + } + if (taosCurlPostRequest(url, &curlRsp, pCont, contentLen) != 0) { + terrno = TSDB_CODE_ANAL_URL_CANT_ACCESS; + goto _OVER; + } + } + + if (curlRsp.data == NULL || curlRsp.dataLen == 0) { + terrno = TSDB_CODE_ANAL_URL_RSP_IS_NULL; + goto _OVER; + } + + pJson = tjsonParse(curlRsp.data); + if (pJson == NULL) { + terrno = TSDB_CODE_INVALID_JSON_FORMAT; + goto _OVER; + } + +_OVER: + if (curlRsp.data != NULL) taosMemoryFreeClear(curlRsp.data); + if (pCont != NULL) taosMemoryFree(pCont); + return pJson; +} + +static int32_t taosAnalJsonBufGetCont(const char *fileName, char **ppCont, int64_t *pContLen) { + int32_t code = 0; + int64_t contLen; + char *pCont = NULL; + TdFilePtr pFile = NULL; + + pFile = taosOpenFile(fileName, TD_FILE_READ); + if (pFile == NULL) { + code = terrno; + goto _OVER; + } + + code = taosFStatFile(pFile, &contLen, NULL); + if (code != 0) goto _OVER; + + pCont = taosMemoryMalloc(contLen + 1); + if (pCont == NULL) { + code = TSDB_CODE_OUT_OF_MEMORY; + goto _OVER; + } + + if (taosReadFile(pFile, pCont, contLen) != contLen) { + code = terrno; + goto _OVER; + } + + pCont[contLen] = '\0'; + +_OVER: + if (code == 0) { + *ppCont = pCont; + *pContLen = contLen; + } else { + if (pCont != NULL) taosMemoryFree(pCont); + } + if (pFile != NULL) taosCloseFile(&pFile); + return code; +} + +static int32_t taosAnalJsonBufWriteOptInt(SAnalBuf *pBuf, const char *optName, int64_t optVal) { + char buf[64] = {0}; + int32_t bufLen = snprintf(buf, sizeof(buf), "\"%s\": %" PRId64 ",\n", optName, optVal); + if (taosWriteFile(pBuf->filePtr, buf, bufLen) != bufLen) { + return terrno; + } + return 0; +} + +static int32_t taosAnalJsonBufWriteOptStr(SAnalBuf *pBuf, const char *optName, const char *optVal) { + char buf[128] = {0}; + int32_t bufLen = snprintf(buf, sizeof(buf), "\"%s\": \"%s\",\n", optName, optVal); + if (taosWriteFile(pBuf->filePtr, buf, bufLen) != bufLen) { + return terrno; + } + return 0; +} + +static int32_t taosAnalJsonBufWriteOptFloat(SAnalBuf *pBuf, const char *optName, float optVal) { + char buf[128] = {0}; + int32_t bufLen = snprintf(buf, sizeof(buf), "\"%s\": %f,\n", optName, optVal); + if (taosWriteFile(pBuf->filePtr, buf, bufLen) != bufLen) { + return terrno; + } + return 0; +} + +static int32_t taosAnalJsonBufWriteStr(SAnalBuf *pBuf, const char *buf, int32_t bufLen) { + if (bufLen <= 0) { + bufLen = strlen(buf); + } + if (taosWriteFile(pBuf->filePtr, buf, bufLen) != bufLen) { + return terrno; + } + return 0; +} + +static int32_t taosAnalJsonBufWriteStart(SAnalBuf *pBuf) { return taosAnalJsonBufWriteStr(pBuf, "{\n", 0); } + +static int32_t tsosAnalJsonBufOpen(SAnalBuf *pBuf, int32_t numOfCols) { + pBuf->filePtr = taosOpenFile(pBuf->fileName, TD_FILE_CREATE | TD_FILE_WRITE | TD_FILE_TRUNC | TD_FILE_WRITE_THROUGH); + if (pBuf->filePtr == NULL) { + return terrno; + } + + pBuf->pCols = taosMemoryCalloc(numOfCols, sizeof(SAnalColBuf)); + if (pBuf->pCols == NULL) return TSDB_CODE_OUT_OF_MEMORY; + pBuf->numOfCols = numOfCols; + + if (pBuf->bufType == ANAL_BUF_TYPE_JSON) { + return taosAnalJsonBufWriteStart(pBuf); + } + + for (int32_t i = 0; i < numOfCols; ++i) { + SAnalColBuf *pCol = &pBuf->pCols[i]; + snprintf(pCol->fileName, sizeof(pCol->fileName), "%s-c%d", pBuf->fileName, i); + pCol->filePtr = + taosOpenFile(pCol->fileName, TD_FILE_CREATE | TD_FILE_WRITE | TD_FILE_TRUNC | TD_FILE_WRITE_THROUGH); + if (pCol->filePtr == NULL) { + return terrno; + } + } + + return taosAnalJsonBufWriteStart(pBuf); +} + +static int32_t taosAnalJsonBufWriteColMeta(SAnalBuf *pBuf, int32_t colIndex, int32_t colType, const char *colName) { + char buf[128] = {0}; + bool first = (colIndex == 0); + bool last = (colIndex == pBuf->numOfCols - 1); + + if (first) { + if (taosAnalJsonBufWriteStr(pBuf, "\"schema\": [\n", 0) != 0) { + return terrno; + } + } + + int32_t bufLen = snprintf(buf, sizeof(buf), " [\"%s\", \"%s\", %d]%s\n", colName, tDataTypes[colType].name, + tDataTypes[colType].bytes, last ? "" : ","); + if (taosWriteFile(pBuf->filePtr, buf, bufLen) != bufLen) { + return terrno; + } + + if (last) { + if (taosAnalJsonBufWriteStr(pBuf, "],\n", 0) != 0) { + return terrno; + } + } + + return 0; +} + +static int32_t taosAnalJsonBufWriteDataBegin(SAnalBuf *pBuf) { + return taosAnalJsonBufWriteStr(pBuf, "\"data\": [\n", 0); +} + +static int32_t taosAnalJsonBufWriteStrUseCol(SAnalBuf *pBuf, const char *buf, int32_t bufLen, int32_t colIndex) { + if (bufLen <= 0) { + bufLen = strlen(buf); + } + + if (pBuf->bufType == ANAL_BUF_TYPE_JSON) { + if (taosWriteFile(pBuf->filePtr, buf, bufLen) != bufLen) { + return terrno; + } + } else { + if (taosWriteFile(pBuf->pCols[colIndex].filePtr, buf, bufLen) != bufLen) { + return terrno; + } + } + + return 0; +} + +static int32_t taosAnalJsonBufWriteColBegin(SAnalBuf *pBuf, int32_t colIndex) { + return taosAnalJsonBufWriteStrUseCol(pBuf, "[\n", 0, colIndex); +} + +static int32_t taosAnalJsonBufWriteColEnd(SAnalBuf *pBuf, int32_t colIndex) { + if (colIndex == pBuf->numOfCols - 1) { + return taosAnalJsonBufWriteStrUseCol(pBuf, "\n]\n", 0, colIndex); + + } else { + return taosAnalJsonBufWriteStrUseCol(pBuf, "\n],\n", 0, colIndex); + } +} + +static int32_t taosAnalJsonBufWriteColData(SAnalBuf *pBuf, int32_t colIndex, int32_t colType, void *colValue) { + char buf[64]; + int32_t bufLen = 0; + + if (pBuf->pCols[colIndex].numOfRows != 0) { + buf[bufLen] = ','; + buf[bufLen + 1] = '\n'; + buf[bufLen + 2] = 0; + bufLen += 2; + } + + switch (colType) { + case TSDB_DATA_TYPE_BOOL: + bufLen += snprintf(buf + bufLen, sizeof(buf) - bufLen, "%d", (*((int8_t *)colValue) == 1) ? 1 : 0); + break; + case TSDB_DATA_TYPE_TINYINT: + bufLen += snprintf(buf + bufLen, sizeof(buf) - bufLen, "%d", *(int8_t *)colValue); + break; + case TSDB_DATA_TYPE_UTINYINT: + bufLen += snprintf(buf + bufLen, sizeof(buf) - bufLen, "%u", *(uint8_t *)colValue); + break; + case TSDB_DATA_TYPE_SMALLINT: + bufLen += snprintf(buf + bufLen, sizeof(buf) - bufLen, "%d", *(int16_t *)colValue); + break; + case TSDB_DATA_TYPE_USMALLINT: + bufLen += snprintf(buf + bufLen, sizeof(buf) - bufLen, "%u", *(uint16_t *)colValue); + break; + case TSDB_DATA_TYPE_INT: + bufLen += snprintf(buf + bufLen, sizeof(buf) - bufLen, "%d", *(int32_t *)colValue); + break; + case TSDB_DATA_TYPE_UINT: + bufLen += snprintf(buf + bufLen, sizeof(buf) - bufLen, "%u", *(uint32_t *)colValue); + break; + case TSDB_DATA_TYPE_BIGINT: + case TSDB_DATA_TYPE_TIMESTAMP: + bufLen += snprintf(buf + bufLen, sizeof(buf) - bufLen, "%" PRId64 "", *(int64_t *)colValue); + break; + case TSDB_DATA_TYPE_UBIGINT: + bufLen += snprintf(buf + bufLen, sizeof(buf) - bufLen, "%" PRIu64 "", *(uint64_t *)colValue); + break; + case TSDB_DATA_TYPE_FLOAT: + bufLen += snprintf(buf + bufLen, sizeof(buf) - bufLen, "%f", GET_FLOAT_VAL(colValue)); + break; + case TSDB_DATA_TYPE_DOUBLE: + bufLen += snprintf(buf + bufLen, sizeof(buf) - bufLen, "%f", GET_DOUBLE_VAL(colValue)); + break; + default: + buf[bufLen] = '\0'; + } + + pBuf->pCols[colIndex].numOfRows++; + return taosAnalJsonBufWriteStrUseCol(pBuf, buf, bufLen, colIndex); +} + +static int32_t taosAnalJsonBufWriteDataEnd(SAnalBuf *pBuf) { + int32_t code = 0; + char *pCont = NULL; + int64_t contLen = 0; + + if (pBuf->bufType == ANAL_BUF_TYPE_JSON_COL) { + for (int32_t i = 0; i < pBuf->numOfCols; ++i) { + SAnalColBuf *pCol = &pBuf->pCols[i]; + + code = taosFsyncFile(pCol->filePtr); + if (code != 0) return code; + + code = taosCloseFile(&pCol->filePtr); + if (code != 0) return code; + + code = taosAnalJsonBufGetCont(pBuf->pCols[i].fileName, &pCont, &contLen); + if (code != 0) return code; + + code = taosAnalJsonBufWriteStr(pBuf, pCont, contLen); + if (code != 0) return code; + + taosMemoryFreeClear(pCont); + contLen = 0; + } + } + + return taosAnalJsonBufWriteStr(pBuf, "],\n", 0); +} + +static int32_t taosAnalJsonBufWriteEnd(SAnalBuf *pBuf) { + int32_t code = taosAnalJsonBufWriteOptInt(pBuf, "rows", pBuf->pCols[0].numOfRows); + if (code != 0) return code; + + return taosAnalJsonBufWriteStr(pBuf, "\"protocol\": 1.0\n}", 0); +} + +int32_t taosAnalJsonBufClose(SAnalBuf *pBuf) { + int32_t code = taosAnalJsonBufWriteEnd(pBuf); + if (code != 0) return code; + + if (pBuf->filePtr != NULL) { + code = taosFsyncFile(pBuf->filePtr); + if (code != 0) return code; + code = taosCloseFile(&pBuf->filePtr); + if (code != 0) return code; + } + + if (pBuf->bufType == ANAL_BUF_TYPE_JSON_COL) { + for (int32_t i = 0; i < pBuf->numOfCols; ++i) { + SAnalColBuf *pCol = &pBuf->pCols[i]; + if (pCol->filePtr != NULL) { + code = taosFsyncFile(pCol->filePtr); + if (code != 0) return code; + code = taosCloseFile(&pCol->filePtr); + if (code != 0) return code; + } + } + } + + return 0; +} + +void taosAnalBufDestroy(SAnalBuf *pBuf) { + if (pBuf->fileName[0] != 0) { + if (pBuf->filePtr != NULL) (void)taosCloseFile(&pBuf->filePtr); + // taosRemoveFile(pBuf->fileName); + pBuf->fileName[0] = 0; + } + + if (pBuf->bufType == ANAL_BUF_TYPE_JSON_COL) { + for (int32_t i = 0; i < pBuf->numOfCols; ++i) { + SAnalColBuf *pCol = &pBuf->pCols[i]; + if (pCol->fileName[0] != 0) { + if (pCol->filePtr != NULL) (void)taosCloseFile(&pCol->filePtr); + taosRemoveFile(pCol->fileName); + pCol->fileName[0] = 0; + } + } + } + + taosMemoryFreeClear(pBuf->pCols); + pBuf->numOfCols = 0; +} + +int32_t tsosAnalBufOpen(SAnalBuf *pBuf, int32_t numOfCols) { + if (pBuf->bufType == ANAL_BUF_TYPE_JSON || pBuf->bufType == ANAL_BUF_TYPE_JSON_COL) { + return tsosAnalJsonBufOpen(pBuf, numOfCols); + } else { + return TSDB_CODE_ANAL_BUF_INVALID_TYPE; + } +} + +int32_t taosAnalBufWriteOptStr(SAnalBuf *pBuf, const char *optName, const char *optVal) { + if (pBuf->bufType == ANAL_BUF_TYPE_JSON || pBuf->bufType == ANAL_BUF_TYPE_JSON_COL) { + return taosAnalJsonBufWriteOptStr(pBuf, optName, optVal); + } else { + return TSDB_CODE_ANAL_BUF_INVALID_TYPE; + } +} + +int32_t taosAnalBufWriteOptInt(SAnalBuf *pBuf, const char *optName, int64_t optVal) { + if (pBuf->bufType == ANAL_BUF_TYPE_JSON || pBuf->bufType == ANAL_BUF_TYPE_JSON_COL) { + return taosAnalJsonBufWriteOptInt(pBuf, optName, optVal); + } else { + return TSDB_CODE_ANAL_BUF_INVALID_TYPE; + } +} + +int32_t taosAnalBufWriteOptFloat(SAnalBuf *pBuf, const char *optName, float optVal) { + if (pBuf->bufType == ANAL_BUF_TYPE_JSON || pBuf->bufType == ANAL_BUF_TYPE_JSON_COL) { + return taosAnalJsonBufWriteOptFloat(pBuf, optName, optVal); + } else { + return TSDB_CODE_ANAL_BUF_INVALID_TYPE; + } +} + +int32_t taosAnalBufWriteColMeta(SAnalBuf *pBuf, int32_t colIndex, int32_t colType, const char *colName) { + if (pBuf->bufType == ANAL_BUF_TYPE_JSON || pBuf->bufType == ANAL_BUF_TYPE_JSON_COL) { + return taosAnalJsonBufWriteColMeta(pBuf, colIndex, colType, colName); + } else { + return TSDB_CODE_ANAL_BUF_INVALID_TYPE; + } +} + +int32_t taosAnalBufWriteDataBegin(SAnalBuf *pBuf) { + if (pBuf->bufType == ANAL_BUF_TYPE_JSON || pBuf->bufType == ANAL_BUF_TYPE_JSON_COL) { + return taosAnalJsonBufWriteDataBegin(pBuf); + } else { + return TSDB_CODE_ANAL_BUF_INVALID_TYPE; + } +} + +int32_t taosAnalBufWriteColBegin(SAnalBuf *pBuf, int32_t colIndex) { + if (pBuf->bufType == ANAL_BUF_TYPE_JSON || pBuf->bufType == ANAL_BUF_TYPE_JSON_COL) { + return taosAnalJsonBufWriteColBegin(pBuf, colIndex); + } else { + return TSDB_CODE_ANAL_BUF_INVALID_TYPE; + } +} + +int32_t taosAnalBufWriteColData(SAnalBuf *pBuf, int32_t colIndex, int32_t colType, void *colValue) { + if (pBuf->bufType == ANAL_BUF_TYPE_JSON || pBuf->bufType == ANAL_BUF_TYPE_JSON_COL) { + return taosAnalJsonBufWriteColData(pBuf, colIndex, colType, colValue); + } else { + return TSDB_CODE_ANAL_BUF_INVALID_TYPE; + } +} + +int32_t taosAnalBufWriteColEnd(SAnalBuf *pBuf, int32_t colIndex) { + if (pBuf->bufType == ANAL_BUF_TYPE_JSON || pBuf->bufType == ANAL_BUF_TYPE_JSON_COL) { + return taosAnalJsonBufWriteColEnd(pBuf, colIndex); + } else { + return TSDB_CODE_ANAL_BUF_INVALID_TYPE; + } +} + +int32_t taosAnalBufWriteDataEnd(SAnalBuf *pBuf) { + if (pBuf->bufType == ANAL_BUF_TYPE_JSON || pBuf->bufType == ANAL_BUF_TYPE_JSON_COL) { + return taosAnalJsonBufWriteDataEnd(pBuf); + } else { + return TSDB_CODE_ANAL_BUF_INVALID_TYPE; + } +} + +int32_t taosAnalBufClose(SAnalBuf *pBuf) { + if (pBuf->bufType == ANAL_BUF_TYPE_JSON || pBuf->bufType == ANAL_BUF_TYPE_JSON_COL) { + return taosAnalJsonBufClose(pBuf); + } else { + return TSDB_CODE_ANAL_BUF_INVALID_TYPE; + } +} + +static int32_t taosAnalBufGetCont(SAnalBuf *pBuf, char **ppCont, int64_t *pContLen) { + *ppCont = NULL; + *pContLen = 0; + + if (pBuf->bufType == ANAL_BUF_TYPE_JSON || pBuf->bufType == ANAL_BUF_TYPE_JSON_COL) { + return taosAnalJsonBufGetCont(pBuf->fileName, ppCont, pContLen); + } else { + return TSDB_CODE_ANAL_BUF_INVALID_TYPE; + } +} + +#else + +int32_t taosAnalInit() { return 0; } +void taosAnalCleanup() {} +SJson *taosAnalSendReqRetJson(const char *url, EAnalHttpType type, SAnalBuf *pBuf) { return NULL; } + +int32_t taosAnalGetAlgoUrl(const char *algoName, EAnalAlgoType type, char *url, int32_t urlLen) { return 0; } +bool taosAnalGetOptStr(const char *option, const char *optName, char *optValue, int32_t optMaxLen) { return true; } +bool taosAnalGetOptInt(const char *option, const char *optName, int32_t *optValue) { return true; } +int64_t taosAnalGetVersion() { return 0; } +void taosAnalUpdate(int64_t newVer, SHashObj *pHash) {} + +int32_t tsosAnalBufOpen(SAnalBuf *pBuf, int32_t numOfCols) { return 0; } +int32_t taosAnalBufWriteOptStr(SAnalBuf *pBuf, const char *optName, const char *optVal) { return 0; } +int32_t taosAnalBufWriteOptInt(SAnalBuf *pBuf, const char *optName, int64_t optVal) { return 0; } +int32_t taosAnalBufWriteOptFloat(SAnalBuf *pBuf, const char *optName, float optVal) { return 0; } +int32_t taosAnalBufWriteColMeta(SAnalBuf *pBuf, int32_t colIndex, int32_t colType, const char *colName) { return 0; } +int32_t taosAnalBufWriteDataBegin(SAnalBuf *pBuf) { return 0; } +int32_t taosAnalBufWriteColBegin(SAnalBuf *pBuf, int32_t colIndex) { return 0; } +int32_t taosAnalBufWriteColData(SAnalBuf *pBuf, int32_t colIndex, int32_t colType, void *colValue) { return 0; } +int32_t taosAnalBufWriteColEnd(SAnalBuf *pBuf, int32_t colIndex) { return 0; } +int32_t taosAnalBufWriteDataEnd(SAnalBuf *pBuf) { return 0; } +int32_t taosAnalBufClose(SAnalBuf *pBuf) { return 0; } +void taosAnalBufDestroy(SAnalBuf *pBuf) {} + +const char *taosAnalAlgoStr(EAnalAlgoType algoType) { return 0; } +EAnalAlgoType taosAnalAlgoInt(const char *algoName) { return 0; } +const char *taosAnalAlgoUrlStr(EAnalAlgoType algoType) { return 0; } + +#endif \ No newline at end of file diff --git a/source/util/src/terror.c b/source/util/src/terror.c index df104508da..3598262d5d 100644 --- a/source/util/src/terror.c +++ b/source/util/src/terror.c @@ -345,6 +345,21 @@ TAOS_DEFINE_ERROR(TSDB_CODE_MND_MULTI_REPLICA_SOURCE_DB, "Stream temporarily do TAOS_DEFINE_ERROR(TSDB_CODE_MND_TOO_MANY_STREAMS, "Too many streams") TAOS_DEFINE_ERROR(TSDB_CODE_MND_INVALID_TARGET_TABLE, "Cannot write the same stable as other stream") +TAOS_DEFINE_ERROR(TSDB_CODE_MND_ANODE_ALREADY_EXIST, "Anode already exists") +TAOS_DEFINE_ERROR(TSDB_CODE_MND_ANODE_NOT_EXIST, "Anode not there") +TAOS_DEFINE_ERROR(TSDB_CODE_MND_ANODE_TOO_LONG_URL, "Anode too long url") +TAOS_DEFINE_ERROR(TSDB_CODE_MND_ANODE_INVALID_PROTOCOL, "Anode invalid protocol") +TAOS_DEFINE_ERROR(TSDB_CODE_MND_ANODE_INVALID_VERSION, "Anode invalid version") +TAOS_DEFINE_ERROR(TSDB_CODE_MND_ANODE_TOO_MANY_ALGO, "Anode too many algorithm") +TAOS_DEFINE_ERROR(TSDB_CODE_MND_ANODE_TOO_LONG_ALGO_NAME, "Anode too long algorithm name") +TAOS_DEFINE_ERROR(TSDB_CODE_MND_ANODE_TOO_MANY_ALGO_TYPE, "Anode too many algorithm type") + +TAOS_DEFINE_ERROR(TSDB_CODE_ANAL_URL_RSP_IS_NULL, "Analysis url response is NULL") +TAOS_DEFINE_ERROR(TSDB_CODE_ANAL_URL_CANT_ACCESS, "Analysis url can't access") +TAOS_DEFINE_ERROR(TSDB_CODE_ANAL_ALGO_NOT_FOUND, "Analysis algorithm not found") +TAOS_DEFINE_ERROR(TSDB_CODE_ANAL_ALGO_NOT_LOAD, "Analysis algorithm not loaded") +TAOS_DEFINE_ERROR(TSDB_CODE_ANAL_BUF_INVALID_TYPE, "Analysis invalid buffer type") + // mnode-sma TAOS_DEFINE_ERROR(TSDB_CODE_MND_SMA_ALREADY_EXIST, "SMA already exists") TAOS_DEFINE_ERROR(TSDB_CODE_MND_SMA_NOT_EXIST, "sma not exist") @@ -708,6 +723,11 @@ TAOS_DEFINE_ERROR(TSDB_CODE_PAR_TBNAME_ERROR, "Pseudo tag tbname n TAOS_DEFINE_ERROR(TSDB_CODE_PAR_TBNAME_DUPLICATED, "Table name duplicated") TAOS_DEFINE_ERROR(TSDB_CODE_PAR_TAG_NAME_DUPLICATED, "Tag name duplicated") TAOS_DEFINE_ERROR(TSDB_CODE_PAR_NOT_ALLOWED_DIFFERENT_BY_ROW_FUNC, "Some functions cannot appear in the select list at the same time") +TAOS_DEFINE_ERROR(TSDB_CODE_PAR_REGULAR_EXPRESSION_ERROR, "Syntax error in regular expression") +TAOS_DEFINE_ERROR(TSDB_CODE_PAR_INVALID_ANOMALY_WIN_TYPE, "ANOMALY_WINDOW only support mathable column") +TAOS_DEFINE_ERROR(TSDB_CODE_PAR_INVALID_ANOMALY_WIN_COL, "ANOMALY_WINDOW not support on tag column") +TAOS_DEFINE_ERROR(TSDB_CODE_PAR_INVALID_ANOMALY_WIN_OPT, "ANOMALY_WINDOW option should include algo field") +TAOS_DEFINE_ERROR(TSDB_CODE_PAR_INVALID_FORECAST_CLAUSE, "Invalid forecast clause") TAOS_DEFINE_ERROR(TSDB_CODE_PAR_REGULAR_EXPRESSION_ERROR, "Syntax error in regular expression") TAOS_DEFINE_ERROR(TSDB_CODE_PAR_INTERNAL_ERROR, "Parser internal error") diff --git a/source/util/src/tjson.c b/source/util/src/tjson.c index 0f2504ff5e..314f205057 100644 --- a/source/util/src/tjson.c +++ b/source/util/src/tjson.c @@ -194,6 +194,10 @@ int32_t tjsonGetObjectValueString(const SJson* pJson, char** pValueString) { return TSDB_CODE_SUCCESS; } +void tjsonGetObjectValueBigInt(const SJson* pJson, int64_t* pVal) { *pVal = (int64_t)((cJSON*)pJson)->valuedouble; } + +void tjsonGetObjectValueDouble(const SJson* pJson, double* pVal) { *pVal = ((cJSON*)pJson)->valuedouble; } + int32_t tjsonGetStringValue(const SJson* pJson, const char* pName, char* pVal) { char* p = cJSON_GetStringValue(tjsonGetObjectItem((cJSON*)pJson, pName)); if (NULL == p) { From a5b5bd61f6f375ca4fa26631f4a2e416d92a9470 Mon Sep 17 00:00:00 2001 From: Shengliang Guan Date: Wed, 9 Oct 2024 17:20:13 +0800 Subject: [PATCH 35/51] feat: anode --- source/dnode/mgmt/mgmt_dnode/src/dmInt.c | 7 +- source/dnode/mgmt/mgmt_mnode/src/mmHandle.c | 4 + source/dnode/mgmt/node_mgmt/src/dmEnv.c | 2 + source/dnode/mnode/impl/src/mndAnode.c | 902 ++++++++++++++++++++ source/dnode/mnode/impl/src/mndDnode.c | 8 +- source/dnode/mnode/impl/src/mndMain.c | 2 + source/dnode/mnode/impl/src/mndShow.c | 4 + source/dnode/mnode/sdb/inc/sdb.h | 3 +- source/dnode/mnode/sdb/src/sdbFile.c | 57 +- source/dnode/mnode/sdb/src/sdbHash.c | 2 + 10 files changed, 984 insertions(+), 7 deletions(-) create mode 100644 source/dnode/mnode/impl/src/mndAnode.c diff --git a/source/dnode/mgmt/mgmt_dnode/src/dmInt.c b/source/dnode/mgmt/mgmt_dnode/src/dmInt.c index 1561ab0a6b..74ef67ff1d 100644 --- a/source/dnode/mgmt/mgmt_dnode/src/dmInt.c +++ b/source/dnode/mgmt/mgmt_dnode/src/dmInt.c @@ -16,6 +16,7 @@ #define _DEFAULT_SOURCE #include "dmInt.h" #include "libs/function/tudf.h" +#include "tanal.h" static int32_t dmStartMgmt(SDnodeMgmt *pMgmt) { int32_t code = 0; @@ -77,7 +78,11 @@ static int32_t dmOpenMgmt(SMgmtInputOpt *pInput, SMgmtOutputOpt *pOutput) { } if ((code = udfStartUdfd(pMgmt->pData->dnodeId)) != 0) { - dError("failed to start udfd"); + dError("failed to start udfd since %s", tstrerror(code)); + } + + if ((code = taosAnalInit()) != 0) { + dError("failed to init analysis env since %s", tstrerror(code)); } pOutput->pMgmt = pMgmt; diff --git a/source/dnode/mgmt/mgmt_mnode/src/mmHandle.c b/source/dnode/mgmt/mgmt_mnode/src/mmHandle.c index 7204cde8f7..d9aa4614b6 100644 --- a/source/dnode/mgmt/mgmt_mnode/src/mmHandle.c +++ b/source/dnode/mgmt/mgmt_mnode/src/mmHandle.c @@ -141,6 +141,9 @@ SArray *mmGetMsgHandles() { if (dmSetMgmtHandle(pArray, TDMT_MND_DNODE_LIST, mmPutMsgToReadQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_MND_CREATE_SNODE, mmPutMsgToWriteQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_MND_DROP_SNODE, mmPutMsgToWriteQueue, 0) == NULL) goto _OVER; + if (dmSetMgmtHandle(pArray, TDMT_MND_CREATE_ANODE, mmPutMsgToWriteQueue, 0) == NULL) goto _OVER; + if (dmSetMgmtHandle(pArray, TDMT_MND_UPDATE_ANODE, mmPutMsgToWriteQueue, 0) == NULL) goto _OVER; + if (dmSetMgmtHandle(pArray, TDMT_MND_DROP_ANODE, mmPutMsgToWriteQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_MND_CREATE_DB, mmPutMsgToWriteQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_MND_DROP_DB, mmPutMsgToWriteQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_MND_USE_DB, mmPutMsgToReadQueue, 0) == NULL) goto _OVER; @@ -180,6 +183,7 @@ SArray *mmGetMsgHandles() { if (dmSetMgmtHandle(pArray, TDMT_VND_FETCH_TTL_EXPIRED_TBS_RSP, mmPutMsgToWriteQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_VND_DROP_TABLE_RSP, mmPutMsgToWriteQueue, 0) == NULL) goto _OVER; + if (dmSetMgmtHandle(pArray, TDMT_MND_RETRIEVE_ANAL_ALGO, mmPutMsgToReadQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_MND_RETRIEVE_IP_WHITE, mmPutMsgToReadQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_MND_GET_USER_WHITELIST, mmPutMsgToReadQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_MND_GET_INDEX, mmPutMsgToReadQueue, 0) == NULL) goto _OVER; diff --git a/source/dnode/mgmt/node_mgmt/src/dmEnv.c b/source/dnode/mgmt/node_mgmt/src/dmEnv.c index 2d0ad70adf..0c2bd2bc0f 100644 --- a/source/dnode/mgmt/node_mgmt/src/dmEnv.c +++ b/source/dnode/mgmt/node_mgmt/src/dmEnv.c @@ -20,6 +20,7 @@ #include "libs/function/tudf.h" #include "tgrant.h" #include "tcompare.h" +#include "tanal.h" // clang-format on #define DM_INIT_AUDIT() \ @@ -214,6 +215,7 @@ void dmCleanup() { dError("failed to close udfc"); } udfStopUdfd(); + taosAnalCleanup(); taosStopCacheRefreshWorker(); (void)dmDiskClose(); DestroyRegexCache(); diff --git a/source/dnode/mnode/impl/src/mndAnode.c b/source/dnode/mnode/impl/src/mndAnode.c new file mode 100644 index 0000000000..7e02db0e90 --- /dev/null +++ b/source/dnode/mnode/impl/src/mndAnode.c @@ -0,0 +1,902 @@ +/* + * Copyright (c) 2019 TAOS Data, Inc. + * + * This program is free software: you can use, redistribute, and/or modify + * it under the terms of the GNU Affero General Public License, version 3 + * or later ("AGPL"), as published by the Free Software Foundation. + * + * This program is distributed in the hope that it will be useful, but WITHOUT + * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or + * FITNESS FOR A PARTICULAR PURPOSE. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + */ + +#define _DEFAULT_SOURCE +#include "mndAnode.h" +#include "audit.h" +#include "mndDnode.h" +#include "mndPrivilege.h" +#include "mndShow.h" +#include "mndTrans.h" +#include "mndUser.h" +#include "tanal.h" +#include "tjson.h" + +#ifdef USE_ANAL + +#define TSDB_ANODE_VER_NUMBER 1 +#define TSDB_ANODE_RESERVE_SIZE 64 + +static SSdbRaw *mndAnodeActionEncode(SAnodeObj *pObj); +static SSdbRow *mndAnodeActionDecode(SSdbRaw *pRaw); +static int32_t mndAnodeActionInsert(SSdb *pSdb, SAnodeObj *pObj); +static int32_t mndAnodeActionUpdate(SSdb *pSdb, SAnodeObj *pOld, SAnodeObj *pNew); +static int32_t mndAnodeActionDelete(SSdb *pSdb, SAnodeObj *pObj); +static int32_t mndProcessCreateAnodeReq(SRpcMsg *pReq); +static int32_t mndProcessUpdateAnodeReq(SRpcMsg *pReq); +static int32_t mndProcessDropAnodeReq(SRpcMsg *pReq); +static int32_t mndProcessAnalAlgoReq(SRpcMsg *pReq); +static int32_t mndRetrieveAnodes(SRpcMsg *pReq, SShowObj *pShow, SSDataBlock *pBlock, int32_t rows); +static void mndCancelGetNextAnode(SMnode *pMnode, void *pIter); +static int32_t mndRetrieveAnodesFull(SRpcMsg *pReq, SShowObj *pShow, SSDataBlock *pBlock, int32_t rows); +static void mndCancelGetNextAnodeFull(SMnode *pMnode, void *pIter); +static int32_t mndGetAnodeAlgoList(const char *url, SAnodeObj *pObj); +static int32_t mndGetAnodeStatus(SAnodeObj *pObj, char *status); + +int32_t mndInitAnode(SMnode *pMnode) { + SSdbTable table = { + .sdbType = SDB_ANODE, + .keyType = SDB_KEY_INT32, + .encodeFp = (SdbEncodeFp)mndAnodeActionEncode, + .decodeFp = (SdbDecodeFp)mndAnodeActionDecode, + .insertFp = (SdbInsertFp)mndAnodeActionInsert, + .updateFp = (SdbUpdateFp)mndAnodeActionUpdate, + .deleteFp = (SdbDeleteFp)mndAnodeActionDelete, + }; + + mndSetMsgHandle(pMnode, TDMT_MND_CREATE_ANODE, mndProcessCreateAnodeReq); + mndSetMsgHandle(pMnode, TDMT_MND_UPDATE_ANODE, mndProcessUpdateAnodeReq); + mndSetMsgHandle(pMnode, TDMT_MND_DROP_ANODE, mndProcessDropAnodeReq); + mndSetMsgHandle(pMnode, TDMT_MND_RETRIEVE_ANAL_ALGO, mndProcessAnalAlgoReq); + + mndAddShowRetrieveHandle(pMnode, TSDB_MGMT_TABLE_ANODE, mndRetrieveAnodes); + mndAddShowFreeIterHandle(pMnode, TSDB_MGMT_TABLE_ANODE, mndCancelGetNextAnode); + mndAddShowRetrieveHandle(pMnode, TSDB_MGMT_TABLE_ANODE_FULL, mndRetrieveAnodesFull); + mndAddShowFreeIterHandle(pMnode, TSDB_MGMT_TABLE_ANODE_FULL, mndCancelGetNextAnodeFull); + + return sdbSetTable(pMnode->pSdb, table); +} + +void mndCleanupAnode(SMnode *pMnode) {} + +SAnodeObj *mndAcquireAnode(SMnode *pMnode, int32_t anodeId) { + SAnodeObj *pObj = sdbAcquire(pMnode->pSdb, SDB_ANODE, &anodeId); + if (pObj == NULL && terrno == TSDB_CODE_SDB_OBJ_NOT_THERE) { + terrno = TSDB_CODE_MND_ANODE_NOT_EXIST; + } + return pObj; +} + +void mndReleaseAnode(SMnode *pMnode, SAnodeObj *pObj) { + SSdb *pSdb = pMnode->pSdb; + sdbRelease(pSdb, pObj); +} + +static SSdbRaw *mndAnodeActionEncode(SAnodeObj *pObj) { + int32_t code = 0; + int32_t lino = 0; + terrno = TSDB_CODE_OUT_OF_MEMORY; + + int32_t rawDataLen = sizeof(SAnodeObj) + TSDB_ANODE_RESERVE_SIZE + pObj->urlLen; + for (int32_t t = 0; t < pObj->numOfAlgos; ++t) { + SArray *algos = pObj->algos[t]; + for (int32_t a = 0; a < (int32_t)taosArrayGetSize(algos); ++a) { + SAnodeAlgo *algo = taosArrayGet(algos, a); + rawDataLen += (2 * sizeof(int32_t) + algo->nameLen); + } + rawDataLen += sizeof(int32_t); + } + + SSdbRaw *pRaw = sdbAllocRaw(SDB_ANODE, TSDB_ANODE_VER_NUMBER, rawDataLen); + if (pRaw == NULL) goto _OVER; + + int32_t dataPos = 0; + SDB_SET_INT32(pRaw, dataPos, pObj->id, _OVER) + SDB_SET_INT64(pRaw, dataPos, pObj->createdTime, _OVER) + SDB_SET_INT64(pRaw, dataPos, pObj->updateTime, _OVER) + SDB_SET_INT32(pRaw, dataPos, pObj->version, _OVER) + SDB_SET_INT32(pRaw, dataPos, pObj->urlLen, _OVER) + SDB_SET_BINARY(pRaw, dataPos, pObj->url, pObj->urlLen, _OVER) + SDB_SET_INT32(pRaw, dataPos, pObj->numOfAlgos, _OVER) + for (int32_t i = 0; i < pObj->numOfAlgos; ++i) { + SArray *algos = pObj->algos[i]; + SDB_SET_INT32(pRaw, dataPos, (int32_t)taosArrayGetSize(algos), _OVER) + for (int32_t j = 0; j < (int32_t)taosArrayGetSize(algos); ++j) { + SAnodeAlgo *algo = taosArrayGet(algos, j); + SDB_SET_INT32(pRaw, dataPos, algo->nameLen, _OVER) + SDB_SET_BINARY(pRaw, dataPos, algo->name, algo->nameLen, _OVER) + SDB_SET_INT32(pRaw, dataPos, 0, _OVER) // reserved + } + } + + SDB_SET_RESERVE(pRaw, dataPos, TSDB_ANODE_RESERVE_SIZE, _OVER) + + terrno = 0; + +_OVER: + if (terrno != 0) { + mError("anode:%d, failed to encode to raw:%p since %s", pObj->id, pRaw, terrstr()); + sdbFreeRaw(pRaw); + return NULL; + } + + mTrace("anode:%d, encode to raw:%p, row:%p", pObj->id, pRaw, pObj); + return pRaw; +} + +static SSdbRow *mndAnodeActionDecode(SSdbRaw *pRaw) { + int32_t code = 0; + int32_t lino = 0; + terrno = TSDB_CODE_OUT_OF_MEMORY; + SSdbRow *pRow = NULL; + SAnodeObj *pObj = NULL; + + int8_t sver = 0; + if (sdbGetRawSoftVer(pRaw, &sver) != 0) goto _OVER; + + if (sver != TSDB_ANODE_VER_NUMBER) { + terrno = TSDB_CODE_SDB_INVALID_DATA_VER; + goto _OVER; + } + + pRow = sdbAllocRow(sizeof(SAnodeObj)); + if (pRow == NULL) goto _OVER; + + pObj = sdbGetRowObj(pRow); + if (pObj == NULL) goto _OVER; + + int32_t dataPos = 0; + SDB_GET_INT32(pRaw, dataPos, &pObj->id, _OVER) + SDB_GET_INT64(pRaw, dataPos, &pObj->createdTime, _OVER) + SDB_GET_INT64(pRaw, dataPos, &pObj->updateTime, _OVER) + SDB_GET_INT32(pRaw, dataPos, &pObj->version, _OVER) + SDB_GET_INT32(pRaw, dataPos, &pObj->urlLen, _OVER) + + if (pObj->urlLen > 0) { + pObj->url = taosMemoryCalloc(pObj->urlLen, 1); + if (pObj->url == NULL) goto _OVER; + SDB_GET_BINARY(pRaw, dataPos, pObj->url, pObj->urlLen, _OVER) + } + + SDB_GET_INT32(pRaw, dataPos, &pObj->numOfAlgos, _OVER) + if (pObj->numOfAlgos > 0) { + pObj->algos = taosMemoryCalloc(pObj->numOfAlgos, sizeof(SArray *)); + if (pObj->algos == NULL) { + goto _OVER; + } + } + + for (int32_t i = 0; i < pObj->numOfAlgos; ++i) { + int32_t numOfAlgos = 0; + SDB_GET_INT32(pRaw, dataPos, &numOfAlgos, _OVER) + + pObj->algos[i] = taosArrayInit(2, sizeof(SAnodeAlgo)); + if (pObj->algos[i] == NULL) goto _OVER; + + for (int32_t j = 0; j < numOfAlgos; ++j) { + SAnodeAlgo algoObj = {0}; + int32_t reserved = 0; + + SDB_GET_INT32(pRaw, dataPos, &algoObj.nameLen, _OVER) + if (algoObj.nameLen > 0) { + algoObj.name = taosMemoryCalloc(algoObj.nameLen, 1); + if (algoObj.name == NULL) goto _OVER; + } + + SDB_GET_BINARY(pRaw, dataPos, algoObj.name, algoObj.nameLen, _OVER) + SDB_GET_INT32(pRaw, dataPos, &reserved, _OVER); + + if (taosArrayPush(pObj->algos[i], &algoObj) == NULL) goto _OVER; + } + } + + SDB_GET_RESERVE(pRaw, dataPos, TSDB_ANODE_RESERVE_SIZE, _OVER) + + terrno = 0; + +_OVER: + if (terrno != 0) { + mError("anode:%d, failed to decode from raw:%p since %s", pObj == NULL ? 0 : pObj->id, pRaw, terrstr()); + if (pObj != NULL) { + taosMemoryFreeClear(pObj->url); + } + taosMemoryFreeClear(pRow); + return NULL; + } + + mTrace("anode:%d, decode from raw:%p, row:%p", pObj->id, pRaw, pObj); + return pRow; +} + +static void mndFreeAnode(SAnodeObj *pObj) { + taosMemoryFreeClear(pObj->url); + for (int32_t i = 0; i < pObj->numOfAlgos; ++i) { + SArray *algos = pObj->algos[i]; + for (int32_t j = 0; j < (int32_t)taosArrayGetSize(algos); ++j) { + SAnodeAlgo *algo = taosArrayGet(algos, j); + taosMemoryFreeClear(algo->name); + } + taosArrayDestroy(algos); + } + taosMemoryFreeClear(pObj->algos); +} + +static int32_t mndAnodeActionInsert(SSdb *pSdb, SAnodeObj *pObj) { + mTrace("anode:%d, perform insert action, row:%p", pObj->id, pObj); + return 0; +} + +static int32_t mndAnodeActionDelete(SSdb *pSdb, SAnodeObj *pObj) { + mTrace("anode:%d, perform delete action, row:%p", pObj->id, pObj); + mndFreeAnode(pObj); + return 0; +} + +static int32_t mndAnodeActionUpdate(SSdb *pSdb, SAnodeObj *pOld, SAnodeObj *pNew) { + mTrace("anode:%d, perform update action, old row:%p new row:%p", pOld->id, pOld, pNew); + + taosWLockLatch(&pOld->lock); + int32_t numOfAlgos = pNew->numOfAlgos; + void *algos = pNew->algos; + pNew->numOfAlgos = pOld->numOfAlgos; + pNew->algos = pOld->algos; + pOld->numOfAlgos = numOfAlgos; + pOld->algos = algos; + pOld->updateTime = pNew->updateTime; + pOld->version = pNew->version; + taosWUnLockLatch(&pOld->lock); + return 0; +} + +static int32_t mndSetCreateAnodeRedoLogs(STrans *pTrans, SAnodeObj *pObj) { + int32_t code = 0; + SSdbRaw *pRedoRaw = mndAnodeActionEncode(pObj); + if (pRedoRaw == NULL) { + code = TSDB_CODE_MND_RETURN_VALUE_NULL; + if (terrno != 0) code = terrno; + TAOS_RETURN(code); + } + TAOS_CHECK_RETURN(mndTransAppendRedolog(pTrans, pRedoRaw)); + TAOS_CHECK_RETURN(sdbSetRawStatus(pRedoRaw, SDB_STATUS_CREATING)); + TAOS_RETURN(code); +} + +static int32_t mndSetCreateAnodeUndoLogs(STrans *pTrans, SAnodeObj *pObj) { + int32_t code = 0; + SSdbRaw *pUndoRaw = mndAnodeActionEncode(pObj); + if (pUndoRaw == NULL) { + code = TSDB_CODE_MND_RETURN_VALUE_NULL; + if (terrno != 0) code = terrno; + TAOS_RETURN(code); + } + TAOS_CHECK_RETURN(mndTransAppendUndolog(pTrans, pUndoRaw)); + TAOS_CHECK_RETURN(sdbSetRawStatus(pUndoRaw, SDB_STATUS_DROPPED)); + TAOS_RETURN(code); +} + +static int32_t mndSetCreateAnodeCommitLogs(STrans *pTrans, SAnodeObj *pObj) { + int32_t code = 0; + SSdbRaw *pCommitRaw = mndAnodeActionEncode(pObj); + if (pCommitRaw == NULL) { + code = TSDB_CODE_MND_RETURN_VALUE_NULL; + if (terrno != 0) code = terrno; + TAOS_RETURN(code); + } + TAOS_CHECK_RETURN(mndTransAppendCommitlog(pTrans, pCommitRaw)); + TAOS_CHECK_RETURN(sdbSetRawStatus(pCommitRaw, SDB_STATUS_READY)); + TAOS_RETURN(code); +} + +static int32_t mndCreateAnode(SMnode *pMnode, SRpcMsg *pReq, SMCreateAnodeReq *pCreate) { + int32_t code = -1; + STrans *pTrans = NULL; + + SAnodeObj anodeObj = {0}; + anodeObj.id = sdbGetMaxId(pMnode->pSdb, SDB_ANODE); + anodeObj.createdTime = taosGetTimestampMs(); + anodeObj.updateTime = anodeObj.createdTime; + anodeObj.version = 0; + anodeObj.urlLen = pCreate->urlLen; + if (anodeObj.urlLen > TSDB_ANAL_ANODE_URL_LEN) { + code = TSDB_CODE_MND_ANODE_TOO_LONG_URL; + goto _OVER; + } + + anodeObj.url = taosMemoryCalloc(1, pCreate->urlLen); + if (anodeObj.url == NULL) goto _OVER; + (void)memcpy(anodeObj.url, pCreate->url, pCreate->urlLen); + + code = mndGetAnodeAlgoList(anodeObj.url, &anodeObj); + if (code != 0) goto _OVER; + + pTrans = mndTransCreate(pMnode, TRN_POLICY_ROLLBACK, TRN_CONFLICT_NOTHING, pReq, "create-anode"); + if (pTrans == NULL) { + code = TSDB_CODE_MND_RETURN_VALUE_NULL; + if (terrno != 0) code = terrno; + goto _OVER; + } + mndTransSetSerial(pTrans); + + mInfo("trans:%d, used to create anode:%s as anode:%d", pTrans->id, pCreate->url, anodeObj.id); + + TAOS_CHECK_GOTO(mndSetCreateAnodeRedoLogs(pTrans, &anodeObj), NULL, _OVER); + TAOS_CHECK_GOTO(mndSetCreateAnodeUndoLogs(pTrans, &anodeObj), NULL, _OVER); + TAOS_CHECK_GOTO(mndSetCreateAnodeCommitLogs(pTrans, &anodeObj), NULL, _OVER); + TAOS_CHECK_GOTO(mndTransPrepare(pMnode, pTrans), NULL, _OVER); + + code = 0; + +_OVER: + mndFreeAnode(&anodeObj); + mndTransDrop(pTrans); + TAOS_RETURN(code); +} + +static SAnodeObj *mndAcquireAnodeByURL(SMnode *pMnode, char *url) { + SSdb *pSdb = pMnode->pSdb; + + void *pIter = NULL; + while (1) { + SAnodeObj *pAnode = NULL; + pIter = sdbFetch(pSdb, SDB_ANODE, pIter, (void **)&pAnode); + if (pIter == NULL) break; + + if (strcasecmp(url, pAnode->url) == 0) { + sdbCancelFetch(pSdb, pIter); + return pAnode; + } + + sdbRelease(pSdb, pAnode); + } + + terrno = TSDB_CODE_MND_ANODE_NOT_EXIST; + return NULL; +} + +static int32_t mndProcessCreateAnodeReq(SRpcMsg *pReq) { + SMnode *pMnode = pReq->info.node; + int32_t code = -1; + SAnodeObj *pObj = NULL; + SMCreateAnodeReq createReq = {0}; + + TAOS_CHECK_GOTO(tDeserializeSMCreateAnodeReq(pReq->pCont, pReq->contLen, &createReq), NULL, _OVER); + + mInfo("anode:%s, start to create", createReq.url); + TAOS_CHECK_GOTO(mndCheckOperPrivilege(pMnode, pReq->info.conn.user, MND_OPER_CREATE_ANODE), NULL, _OVER); + + pObj = mndAcquireAnodeByURL(pMnode, createReq.url); + if (pObj != NULL) { + code = TSDB_CODE_MND_ANODE_ALREADY_EXIST; + goto _OVER; + } + + code = mndCreateAnode(pMnode, pReq, &createReq); + if (code == 0) code = TSDB_CODE_ACTION_IN_PROGRESS; + +_OVER: + if (code != 0 && code != TSDB_CODE_ACTION_IN_PROGRESS) { + mError("anode:%s, failed to create since %s", createReq.url, tstrerror(code)); + } + + mndReleaseAnode(pMnode, pObj); + tFreeSMCreateAnodeReq(&createReq); + TAOS_RETURN(code); +} + +static int32_t mndUpdateAnode(SMnode *pMnode, SAnodeObj *pAnode, SRpcMsg *pReq) { + mInfo("anode:%d, start to update", pAnode->id); + int32_t code = -1; + STrans *pTrans = NULL; + SAnodeObj anodeObj = {0}; + anodeObj.id = pAnode->id; + anodeObj.updateTime = taosGetTimestampMs(); + + code = mndGetAnodeAlgoList(pAnode->url, &anodeObj); + if (code != 0) goto _OVER; + + pTrans = mndTransCreate(pMnode, TRN_POLICY_ROLLBACK, TRN_CONFLICT_NOTHING, pReq, "update-anode"); + if (pTrans == NULL) { + code = TSDB_CODE_MND_RETURN_VALUE_NULL; + if (terrno != 0) code = terrno; + goto _OVER; + } + mInfo("trans:%d, used to update anode:%d", pTrans->id, anodeObj.id); + + TAOS_CHECK_GOTO(mndSetCreateAnodeCommitLogs(pTrans, &anodeObj), NULL, _OVER); + TAOS_CHECK_GOTO(mndTransPrepare(pMnode, pTrans), NULL, _OVER); + code = 0; + +_OVER: + mndFreeAnode(&anodeObj); + mndTransDrop(pTrans); + TAOS_RETURN(code); +} + +static int32_t mndUpdateAllAnodes(SMnode *pMnode, SRpcMsg *pReq) { + mInfo("update all anodes"); + SSdb *pSdb = pMnode->pSdb; + int32_t code = 0; + int32_t rows = 0; + int32_t numOfRows = sdbGetSize(pSdb, SDB_ANODE); + + void *pIter = NULL; + while (1) { + SAnodeObj *pObj = NULL; + ESdbStatus objStatus = 0; + pIter = sdbFetchAll(pSdb, SDB_ANODE, pIter, (void **)&pObj, &objStatus, true); + if (pIter == NULL) break; + + rows++; + void *transReq = NULL; + if (rows == numOfRows) transReq = pReq; + code = mndUpdateAnode(pMnode, pObj, transReq); + sdbRelease(pSdb, pObj); + + if (code != 0) break; + } + + if (code == 0 && rows == numOfRows) { + code = TSDB_CODE_ACTION_IN_PROGRESS; + } + + return code; +} + +static int32_t mndProcessUpdateAnodeReq(SRpcMsg *pReq) { + SMnode *pMnode = pReq->info.node; + int32_t code = -1; + SAnodeObj *pObj = NULL; + SMUpdateAnodeReq updateReq = {0}; + + TAOS_CHECK_GOTO(tDeserializeSMUpdateAnodeReq(pReq->pCont, pReq->contLen, &updateReq), NULL, _OVER); + TAOS_CHECK_GOTO(mndCheckOperPrivilege(pMnode, pReq->info.conn.user, MND_OPER_UPDATE_ANODE), NULL, _OVER); + + if (updateReq.anodeId == -1) { + code = mndUpdateAllAnodes(pMnode, pReq); + } else { + pObj = mndAcquireAnode(pMnode, updateReq.anodeId); + if (pObj == NULL) { + code = TSDB_CODE_MND_ANODE_NOT_EXIST; + goto _OVER; + } + code = mndUpdateAnode(pMnode, pObj, pReq); + if (code == 0) code = TSDB_CODE_ACTION_IN_PROGRESS; + } + +_OVER: + if (code != 0 && code != TSDB_CODE_ACTION_IN_PROGRESS) { + if (updateReq.anodeId != -1) { + mError("anode:%d, failed to update since %s", updateReq.anodeId, tstrerror(code)); + } + } + + mndReleaseAnode(pMnode, pObj); + tFreeSMUpdateAnodeReq(&updateReq); + TAOS_RETURN(code); +} + +static int32_t mndSetDropAnodeRedoLogs(STrans *pTrans, SAnodeObj *pObj) { + int32_t code = 0; + SSdbRaw *pRedoRaw = mndAnodeActionEncode(pObj); + if (pRedoRaw == NULL) { + code = TSDB_CODE_MND_RETURN_VALUE_NULL; + if (terrno != 0) code = terrno; + TAOS_RETURN(code); + } + TAOS_CHECK_RETURN(mndTransAppendRedolog(pTrans, pRedoRaw)); + TAOS_CHECK_RETURN(sdbSetRawStatus(pRedoRaw, SDB_STATUS_DROPPING)); + TAOS_RETURN(code); +} + +static int32_t mndSetDropAnodeCommitLogs(STrans *pTrans, SAnodeObj *pObj) { + int32_t code = 0; + SSdbRaw *pCommitRaw = mndAnodeActionEncode(pObj); + if (pCommitRaw == NULL) { + code = TSDB_CODE_MND_RETURN_VALUE_NULL; + if (terrno != 0) code = terrno; + TAOS_RETURN(code); + } + TAOS_CHECK_RETURN(mndTransAppendCommitlog(pTrans, pCommitRaw)); + TAOS_CHECK_RETURN(sdbSetRawStatus(pCommitRaw, SDB_STATUS_DROPPED)); + TAOS_RETURN(code); +} + +static int32_t mndSetDropAnodeInfoToTrans(SMnode *pMnode, STrans *pTrans, SAnodeObj *pObj, bool force) { + if (pObj == NULL) return 0; + TAOS_CHECK_RETURN(mndSetDropAnodeRedoLogs(pTrans, pObj)); + TAOS_CHECK_RETURN(mndSetDropAnodeCommitLogs(pTrans, pObj)); + return 0; +} + +static int32_t mndDropAnode(SMnode *pMnode, SRpcMsg *pReq, SAnodeObj *pObj) { + int32_t code = -1; + + STrans *pTrans = mndTransCreate(pMnode, TRN_POLICY_RETRY, TRN_CONFLICT_NOTHING, pReq, "drop-anode"); + if (pTrans == NULL) { + code = TSDB_CODE_MND_RETURN_VALUE_NULL; + if (terrno != 0) code = terrno; + goto _OVER; + } + mndTransSetSerial(pTrans); + + mInfo("trans:%d, used to drop anode:%d", pTrans->id, pObj->id); + TAOS_CHECK_GOTO(mndSetDropAnodeInfoToTrans(pMnode, pTrans, pObj, false), NULL, _OVER); + TAOS_CHECK_GOTO(mndTransPrepare(pMnode, pTrans), NULL, _OVER); + + code = 0; + +_OVER: + mndTransDrop(pTrans); + TAOS_RETURN(code); +} + +static int32_t mndProcessDropAnodeReq(SRpcMsg *pReq) { + SMnode *pMnode = pReq->info.node; + int32_t code = -1; + SAnodeObj *pObj = NULL; + SMDropAnodeReq dropReq = {0}; + + TAOS_CHECK_GOTO(tDeserializeSMDropAnodeReq(pReq->pCont, pReq->contLen, &dropReq), NULL, _OVER); + + mInfo("anode:%d, start to drop", dropReq.anodeId); + TAOS_CHECK_GOTO(mndCheckOperPrivilege(pMnode, pReq->info.conn.user, MND_OPER_DROP_ANODE), NULL, _OVER); + + if (dropReq.anodeId <= 0) { + code = TSDB_CODE_INVALID_MSG; + goto _OVER; + } + + pObj = mndAcquireAnode(pMnode, dropReq.anodeId); + if (pObj == NULL) { + code = TSDB_CODE_MND_RETURN_VALUE_NULL; + if (terrno != 0) code = terrno; + goto _OVER; + } + + code = mndDropAnode(pMnode, pReq, pObj); + if (code == 0) code = TSDB_CODE_ACTION_IN_PROGRESS; + +_OVER: + if (code != 0 && code != TSDB_CODE_ACTION_IN_PROGRESS) { + mError("anode:%d, failed to drop since %s", dropReq.anodeId, tstrerror(code)); + } + + mndReleaseAnode(pMnode, pObj); + tFreeSMDropAnodeReq(&dropReq); + TAOS_RETURN(code); +} + +static int32_t mndRetrieveAnodes(SRpcMsg *pReq, SShowObj *pShow, SSDataBlock *pBlock, int32_t rows) { + SMnode *pMnode = pReq->info.node; + SSdb *pSdb = pMnode->pSdb; + int32_t numOfRows = 0; + int32_t cols = 0; + SAnodeObj *pObj = NULL; + char buf[TSDB_ANAL_ANODE_URL_LEN + VARSTR_HEADER_SIZE]; + char status[64]; + int32_t code = 0; + + while (numOfRows < rows) { + pShow->pIter = sdbFetch(pSdb, SDB_ANODE, pShow->pIter, (void **)&pObj); + if (pShow->pIter == NULL) break; + + cols = 0; + SColumnInfoData *pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); + code = colDataSetVal(pColInfo, numOfRows, (const char *)&pObj->id, false); + if (code != 0) goto _end; + + STR_WITH_MAXSIZE_TO_VARSTR(buf, pObj->url, pShow->pMeta->pSchemas[cols].bytes); + pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); + code = colDataSetVal(pColInfo, numOfRows, (const char *)buf, false); + if (code != 0) goto _end; + + status[0] = 0; + if (mndGetAnodeStatus(pObj, status) == 0) { + STR_TO_VARSTR(buf, status); + } else { + STR_TO_VARSTR(buf, "offline"); + } + pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); + code = colDataSetVal(pColInfo, numOfRows, buf, false); + if (code != 0) goto _end; + + pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); + code = colDataSetVal(pColInfo, numOfRows, (const char *)&pObj->createdTime, false); + if (code != 0) goto _end; + + pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); + code = colDataSetVal(pColInfo, numOfRows, (const char *)&pObj->updateTime, false); + if (code != 0) goto _end; + + numOfRows++; + sdbRelease(pSdb, pObj); + } + +_end: + if (code != 0) sdbRelease(pSdb, pObj); + + pShow->numOfRows += numOfRows; + return numOfRows; +} + +static void mndCancelGetNextAnode(SMnode *pMnode, void *pIter) { + SSdb *pSdb = pMnode->pSdb; + sdbCancelFetchByType(pSdb, pIter, SDB_ANODE); +} + +static int32_t mndRetrieveAnodesFull(SRpcMsg *pReq, SShowObj *pShow, SSDataBlock *pBlock, int32_t rows) { + SMnode *pMnode = pReq->info.node; + SSdb *pSdb = pMnode->pSdb; + int32_t numOfRows = 0; + int32_t cols = 0; + SAnodeObj *pObj = NULL; + char buf[TSDB_ANAL_ALGO_NAME_LEN + VARSTR_HEADER_SIZE]; + int32_t code = 0; + + while (numOfRows < rows) { + pShow->pIter = sdbFetch(pSdb, SDB_ANODE, pShow->pIter, (void **)&pObj); + if (pShow->pIter == NULL) break; + + for (int32_t t = 0; t < pObj->numOfAlgos; ++t) { + SArray *algos = pObj->algos[t]; + + for (int32_t a = 0; a < taosArrayGetSize(algos); ++a) { + SAnodeAlgo *algo = taosArrayGet(algos, a); + + cols = 0; + SColumnInfoData *pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); + code = colDataSetVal(pColInfo, numOfRows, (const char *)&pObj->id, false); + if (code != 0) goto _end; + + STR_TO_VARSTR(buf, taosAnalAlgoStr(t)); + pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); + code = colDataSetVal(pColInfo, numOfRows, buf, false); + if (code != 0) goto _end; + + STR_TO_VARSTR(buf, algo->name); + pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); + code = colDataSetVal(pColInfo, numOfRows, buf, false); + if (code != 0) goto _end; + + numOfRows++; + } + } + + sdbRelease(pSdb, pObj); + } + +_end: + if (code != 0) sdbRelease(pSdb, pObj); + + pShow->numOfRows += numOfRows; + return numOfRows; +} + +static void mndCancelGetNextAnodeFull(SMnode *pMnode, void *pIter) { + SSdb *pSdb = pMnode->pSdb; + sdbCancelFetchByType(pSdb, pIter, SDB_ANODE); +} + +static int32_t mndDecodeAlgoList(SJson *pJson, SAnodeObj *pObj) { + int32_t code = 0; + int32_t protocol = 0; + double tmp = 0; + char buf[TSDB_ANAL_ALGO_NAME_LEN + 1] = {0}; + + code = tjsonGetDoubleValue(pJson, "protocol", &tmp); + if (code < 0) return TSDB_CODE_INVALID_JSON_FORMAT; + protocol = (int32_t)(tmp * 1000); + if (protocol != 100) return TSDB_CODE_MND_ANODE_INVALID_PROTOCOL; + + code = tjsonGetDoubleValue(pJson, "version", &tmp); + pObj->version = (int32_t)(tmp * 1000); + if (code < 0) return TSDB_CODE_INVALID_JSON_FORMAT; + if (pObj->version <= 0) return TSDB_CODE_MND_ANODE_INVALID_VERSION; + + SJson *details = tjsonGetObjectItem(pJson, "details"); + if (details == NULL) return TSDB_CODE_INVALID_JSON_FORMAT; + int32_t numOfDetails = tjsonGetArraySize(details); + + pObj->algos = taosMemoryCalloc(ANAL_ALGO_TYPE_END, sizeof(SArray *)); + if (pObj->algos == NULL) return TSDB_CODE_OUT_OF_MEMORY; + + pObj->numOfAlgos = ANAL_ALGO_TYPE_END; + for (int32_t i = 0; i < ANAL_ALGO_TYPE_END; ++i) { + pObj->algos[i] = taosArrayInit(4, sizeof(SAnodeAlgo)); + if (pObj->algos[i] == NULL) return TSDB_CODE_OUT_OF_MEMORY; + } + + for (int32_t d = 0; d < numOfDetails; ++d) { + SJson *detail = tjsonGetArrayItem(details, d); + if (detail == NULL) return TSDB_CODE_INVALID_JSON_FORMAT; + + code = tjsonGetStringValue(detail, "type", buf); + if (code < 0) return TSDB_CODE_INVALID_JSON_FORMAT; + EAnalAlgoType type = taosAnalAlgoInt(buf); + if (type < 0 || type >= ANAL_ALGO_TYPE_END) continue; + + SJson *algos = tjsonGetObjectItem(detail, "algo"); + if (algos == NULL) return TSDB_CODE_INVALID_JSON_FORMAT; + int32_t numOfAlgos = tjsonGetArraySize(algos); + for (int32_t a = 0; a < numOfAlgos; ++a) { + SJson *algo = tjsonGetArrayItem(algos, a); + if (algo == NULL) return TSDB_CODE_INVALID_JSON_FORMAT; + + code = tjsonGetStringValue(algo, "name", buf); + if (code < 0) return TSDB_CODE_INVALID_JSON_FORMAT; + + SAnodeAlgo algoObj = {0}; + algoObj.nameLen = strlen(buf) + 1; + if (algoObj.nameLen > TSDB_ANAL_ALGO_NAME_LEN) return TSDB_CODE_MND_ANODE_TOO_LONG_ALGO_NAME; + if (algoObj.nameLen <= 1) return TSDB_CODE_OUT_OF_MEMORY; + algoObj.name = taosMemoryCalloc(algoObj.nameLen, 1); + tstrncpy(algoObj.name, buf, algoObj.nameLen); + + if (taosArrayPush(pObj->algos[type], &algoObj) == NULL) return TSDB_CODE_OUT_OF_MEMORY; + } + } + + return 0; +} + +static int32_t mndGetAnodeAlgoList(const char *url, SAnodeObj *pObj) { + char anodeUrl[TSDB_ANAL_ANODE_URL_LEN + 1] = {0}; + snprintf(anodeUrl, TSDB_ANAL_ANODE_URL_LEN, "%s/%s", url, "list"); + + SJson *pJson = taosAnalSendReqRetJson(anodeUrl, ANAL_HTTP_TYPE_GET, NULL); + if (pJson == NULL) return terrno; + + int32_t code = mndDecodeAlgoList(pJson, pObj); + if (pJson != NULL) tjsonDelete(pJson); + + TAOS_RETURN(code); +} + +static int32_t mndGetAnodeStatus(SAnodeObj *pObj, char *status) { + int32_t code = 0; + int32_t protocol = 0; + double tmp = 0; + char anodeUrl[TSDB_ANAL_ANODE_URL_LEN + 1] = {0}; + snprintf(anodeUrl, TSDB_ANAL_ANODE_URL_LEN, "%s/%s", pObj->url, "status"); + + SJson *pJson = taosAnalSendReqRetJson(anodeUrl, ANAL_HTTP_TYPE_GET, NULL); + if (pJson == NULL) return terrno; + + code = tjsonGetDoubleValue(pJson, "protocol", &tmp); + if (code < 0) { + code = TSDB_CODE_INVALID_JSON_FORMAT; + goto _OVER; + } + protocol = (int32_t)(tmp * 1000); + if (protocol != 100) { + code = TSDB_CODE_MND_ANODE_INVALID_PROTOCOL; + goto _OVER; + } + + code = tjsonGetStringValue(pJson, "status", status); + if (code < 0) { + code = TSDB_CODE_INVALID_JSON_FORMAT; + goto _OVER; + } + if (strlen(status) == 0) { + code = TSDB_CODE_MND_ANODE_INVALID_PROTOCOL; + goto _OVER; + } + +_OVER: + if (pJson != NULL) tjsonDelete(pJson); + TAOS_RETURN(code); +} + +static int32_t mndProcessAnalAlgoReq(SRpcMsg *pReq) { + SMnode *pMnode = pReq->info.node; + SSdb *pSdb = pMnode->pSdb; + int32_t code = -1; + SAnodeObj *pObj = NULL; + SAnalUrl url; + int32_t nameLen; + char name[TSDB_ANAL_ALGO_KEY_LEN]; + SRetrieveAnalAlgoReq req = {0}; + SRetrieveAnalAlgoRsp rsp = {0}; + + TAOS_CHECK_GOTO(tDeserializeRetrieveAnalAlgoReq(pReq->pCont, pReq->contLen, &req), NULL, _OVER); + + rsp.ver = sdbGetTableVer(pSdb, SDB_ANODE); + if (req.analVer != rsp.ver) { + mInfo("dnode:%d, update analysis old ver:%" PRId64 " to new ver:%" PRId64, req.dnodeId, req.analVer, rsp.ver); + rsp.hash = taosHashInit(64, MurmurHash3_32, true, HASH_ENTRY_LOCK); + if (rsp.hash == NULL) { + terrno = TSDB_CODE_OUT_OF_MEMORY; + goto _OVER; + } + + void *pIter = NULL; + while (1) { + SAnodeObj *pAnode = NULL; + pIter = sdbFetch(pSdb, SDB_ANODE, pIter, (void **)&pAnode); + if (pIter == NULL) break; + + url.anode = pAnode->id; + for (int32_t t = 0; t < pAnode->numOfAlgos; ++t) { + SArray *algos = pAnode->algos[t]; + url.type = t; + + for (int32_t a = 0; a < taosArrayGetSize(algos); ++a) { + SAnodeAlgo *algo = taosArrayGet(algos, a); + nameLen = 1 + snprintf(name, sizeof(name) - 1, "%d:%s", url.type, algo->name); + + SAnalUrl *pOldUrl = taosHashAcquire(rsp.hash, name, nameLen); + if (pOldUrl == NULL || (pOldUrl != NULL && pOldUrl->anode < url.anode)) { + if (pOldUrl != NULL) { + taosMemoryFreeClear(pOldUrl->url); + if (taosHashRemove(rsp.hash, name, nameLen) != 0) { + sdbRelease(pSdb, pAnode); + goto _OVER; + } + } + url.url = taosMemoryMalloc(TSDB_ANAL_ANODE_URL_LEN + TSDB_ANAL_ALGO_TYPE_LEN + 1); + if (url.url == NULL) { + sdbRelease(pSdb, pAnode); + goto _OVER; + } + + url.urlLen = 1 + snprintf(url.url, TSDB_ANAL_ANODE_URL_LEN + TSDB_ANAL_ALGO_TYPE_LEN, "%s/%s", pAnode->url, + taosAnalAlgoUrlStr(url.type)); + if (taosHashPut(rsp.hash, name, nameLen, &url, sizeof(SAnalUrl)) != 0) { + taosMemoryFree(url.url); + sdbRelease(pSdb, pAnode); + goto _OVER; + } + } + } + + sdbRelease(pSdb, pAnode); + } + } + } + + int32_t contLen = tSerializeRetrieveAnalAlgoRsp(NULL, 0, &rsp); + void *pHead = rpcMallocCont(contLen); + (void)tSerializeRetrieveAnalAlgoRsp(pHead, contLen, &rsp); + + pReq->info.rspLen = contLen; + pReq->info.rsp = pHead; + +_OVER: + tFreeRetrieveAnalAlgoRsp(&rsp); + TAOS_RETURN(code); +} + +#else + +static int32_t mndProcessUnsupportReq(SRpcMsg *pReq) { return TSDB_CODE_OPS_NOT_SUPPORT; } +static int32_t mndRetrieveUnsupport(SRpcMsg *pReq, SShowObj *pShow, SSDataBlock *pBlock, int32_t rows) { + return TSDB_CODE_OPS_NOT_SUPPORT; +} + +int32_t mndInitAnode(SMnode *pMnode) { + mndSetMsgHandle(pMnode, TDMT_MND_CREATE_ANODE, mndProcessUnsupportReq); + mndSetMsgHandle(pMnode, TDMT_MND_UPDATE_ANODE, mndProcessUnsupportReq); + mndSetMsgHandle(pMnode, TDMT_MND_DROP_ANODE, mndProcessUnsupportReq); + mndSetMsgHandle(pMnode, TDMT_MND_RETRIEVE_ANAL_ALGO, mndProcessUnsupportReq); + + mndAddShowRetrieveHandle(pMnode, TSDB_MGMT_TABLE_ANODE, mndRetrieveUnsupport); + mndAddShowRetrieveHandle(pMnode, TSDB_MGMT_TABLE_ANODE_FULL, mndRetrieveUnsupport); + return 0; +} + +void mndCleanupAnode(SMnode *pMnode) {} + +#endif \ No newline at end of file diff --git a/source/dnode/mnode/impl/src/mndDnode.c b/source/dnode/mnode/impl/src/mndDnode.c index a94a471e4b..5e10583a0a 100644 --- a/source/dnode/mnode/impl/src/mndDnode.c +++ b/source/dnode/mnode/impl/src/mndDnode.c @@ -693,7 +693,7 @@ static int32_t mndProcessStatusReq(SRpcMsg *pReq) { int64_t clusterid = mndGetClusterId(pMnode); if (statusReq.clusterId != 0 && statusReq.clusterId != clusterid) { code = TSDB_CODE_MND_DNODE_DIFF_CLUSTER; - mWarn("dnode:%d, %s, its clusterid:%" PRId64 " differ from current cluster:%" PRId64 ", code:0x%x", + mWarn("dnode:%d, %s, its clusterid:%" PRId64 " differ from current clusterid:%" PRId64 ", code:0x%x", statusReq.dnodeId, statusReq.dnodeEp, statusReq.clusterId, clusterid, code); goto _OVER; } @@ -730,6 +730,7 @@ static int32_t mndProcessStatusReq(SRpcMsg *pReq) { pMnode->ipWhiteVer = mndGetIpWhiteVer(pMnode); + int64_t analVer = sdbGetTableVer(pMnode->pSdb, SDB_ANODE); int64_t dnodeVer = sdbGetTableVer(pMnode->pSdb, SDB_DNODE) + sdbGetTableVer(pMnode->pSdb, SDB_MNODE); int64_t curMs = taosGetTimestampMs(); bool online = mndIsDnodeOnline(pDnode, curMs); @@ -738,9 +739,9 @@ static int32_t mndProcessStatusReq(SRpcMsg *pReq) { bool supportVnodesChanged = pDnode->numOfSupportVnodes != statusReq.numOfSupportVnodes; bool encryptKeyChanged = pDnode->encryptionKeyChksum != statusReq.clusterCfg.encryptionKeyChksum; bool enableWhiteListChanged = statusReq.clusterCfg.enableWhiteList != (tsEnableWhiteList ? 1 : 0); - bool needCheck = !online || dnodeChanged || reboot || supportVnodesChanged || + bool analVerChanged = (analVer != statusReq.analVer); + 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); @@ -863,6 +864,7 @@ static int32_t mndProcessStatusReq(SRpcMsg *pReq) { SStatusRsp statusRsp = {0}; statusRsp.statusSeq++; + statusRsp.analVer = analVer; statusRsp.dnodeVer = dnodeVer; statusRsp.dnodeCfg.dnodeId = pDnode->id; statusRsp.dnodeCfg.clusterId = pMnode->clusterId; diff --git a/source/dnode/mnode/impl/src/mndMain.c b/source/dnode/mnode/impl/src/mndMain.c index eb855d28a8..451a83a02b 100644 --- a/source/dnode/mnode/impl/src/mndMain.c +++ b/source/dnode/mnode/impl/src/mndMain.c @@ -16,6 +16,7 @@ #define _DEFAULT_SOURCE #include "mndAcct.h" #include "mndArbGroup.h" +#include "mndAnode.h" #include "mndCluster.h" #include "mndCompact.h" #include "mndCompactDetail.h" @@ -605,6 +606,7 @@ static int32_t mndInitSteps(SMnode *pMnode) { TAOS_CHECK_RETURN(mndAllocStep(pMnode, "mnode-mnode", mndInitMnode, mndCleanupMnode)); TAOS_CHECK_RETURN(mndAllocStep(pMnode, "mnode-qnode", mndInitQnode, mndCleanupQnode)); TAOS_CHECK_RETURN(mndAllocStep(pMnode, "mnode-snode", mndInitSnode, mndCleanupSnode)); + TAOS_CHECK_RETURN(mndAllocStep(pMnode, "mnode-anode", mndInitAnode, mndCleanupAnode)); TAOS_CHECK_RETURN(mndAllocStep(pMnode, "mnode-arbgroup", mndInitArbGroup, mndCleanupArbGroup)); TAOS_CHECK_RETURN(mndAllocStep(pMnode, "mnode-dnode", mndInitDnode, mndCleanupDnode)); TAOS_CHECK_RETURN(mndAllocStep(pMnode, "mnode-user", mndInitUser, mndCleanupUser)); diff --git a/source/dnode/mnode/impl/src/mndShow.c b/source/dnode/mnode/impl/src/mndShow.c index 55687c00ba..264fea3476 100644 --- a/source/dnode/mnode/impl/src/mndShow.c +++ b/source/dnode/mnode/impl/src/mndShow.c @@ -68,6 +68,10 @@ static int32_t convertToRetrieveType(char *name, int32_t len) { type = TSDB_MGMT_TABLE_QNODE; } else if (strncasecmp(name, TSDB_INS_TABLE_SNODES, len) == 0) { type = TSDB_MGMT_TABLE_SNODE; + } else if (strncasecmp(name, TSDB_INS_TABLE_ANODES, len) == 0) { + type = TSDB_MGMT_TABLE_ANODE; + } else if (strncasecmp(name, TSDB_INS_TABLE_ANODES_FULL, len) == 0) { + type = TSDB_MGMT_TABLE_ANODE_FULL; } else if (strncasecmp(name, TSDB_INS_TABLE_ARBGROUPS, len) == 0) { type = TSDB_MGMT_TABLE_ARBGROUP; } else if (strncasecmp(name, TSDB_INS_TABLE_CLUSTER, len) == 0) { diff --git a/source/dnode/mnode/sdb/inc/sdb.h b/source/dnode/mnode/sdb/inc/sdb.h index c33b1d4366..f6d1587bb2 100644 --- a/source/dnode/mnode/sdb/inc/sdb.h +++ b/source/dnode/mnode/sdb/inc/sdb.h @@ -161,7 +161,8 @@ typedef enum { SDB_COMPACT_DETAIL = 25, SDB_GRANT = 26, // grant log SDB_ARBGROUP = 27, - SDB_MAX = 28 + SDB_ANODE = 28, + SDB_MAX = 29 } ESdbType; typedef struct SSdbRaw { diff --git a/source/dnode/mnode/sdb/src/sdbFile.c b/source/dnode/mnode/sdb/src/sdbFile.c index 34a017a907..d98c3e5a72 100644 --- a/source/dnode/mnode/sdb/src/sdbFile.c +++ b/source/dnode/mnode/sdb/src/sdbFile.c @@ -25,6 +25,9 @@ #define SDB_RESERVE_SIZE 512 #define SDB_FILE_VER 1 +#define SDB_TABLE_SIZE_EXTRA SDB_MAX +#define SDB_RESERVE_SIZE_EXTRA (512 - (SDB_TABLE_SIZE_EXTRA - SDB_TABLE_SIZE) * 2 * sizeof(int64_t)) + static int32_t sdbDeployData(SSdb *pSdb) { int32_t code = 0; mInfo("start to deploy sdb"); @@ -154,7 +157,38 @@ static int32_t sdbReadFileHead(SSdb *pSdb, TdFilePtr pFile) { } } - char reserve[SDB_RESERVE_SIZE] = {0}; + // for sdb compatibility + for (int32_t i = SDB_TABLE_SIZE; i < SDB_TABLE_SIZE_EXTRA; ++i) { + int64_t maxId = 0; + ret = taosReadFile(pFile, &maxId, sizeof(int64_t)); + if (ret < 0) { + code = TAOS_SYSTEM_ERROR(errno); + TAOS_RETURN(code); + } + if (ret != sizeof(int64_t)) { + code = TSDB_CODE_FILE_CORRUPTED; + TAOS_RETURN(code); + } + if (i < SDB_MAX) { + pSdb->maxId[i] = maxId; + } + + int64_t ver = 0; + ret = taosReadFile(pFile, &ver, sizeof(int64_t)); + if (ret < 0) { + code = TAOS_SYSTEM_ERROR(errno); + TAOS_RETURN(code); + } + if (ret != sizeof(int64_t)) { + code = TSDB_CODE_FILE_CORRUPTED; + TAOS_RETURN(code); + } + if (i < SDB_MAX) { + pSdb->tableVer[i] = ver; + } + } + + char reserve[SDB_RESERVE_SIZE_EXTRA] = {0}; ret = taosReadFile(pFile, reserve, sizeof(reserve)); if (ret < 0) { return terrno; @@ -205,7 +239,26 @@ static int32_t sdbWriteFileHead(SSdb *pSdb, TdFilePtr pFile) { } } - char reserve[SDB_RESERVE_SIZE] = {0}; + // for sdb compatibility + for (int32_t i = SDB_TABLE_SIZE; i < SDB_TABLE_SIZE_EXTRA; ++i) { + int64_t maxId = 0; + if (i < SDB_MAX) { + maxId = pSdb->maxId[i]; + } + if (taosWriteFile(pFile, &maxId, sizeof(int64_t)) != sizeof(int64_t)) { + return terrno; + } + + int64_t ver = 0; + if (i < SDB_MAX) { + ver = pSdb->tableVer[i]; + } + if (taosWriteFile(pFile, &ver, sizeof(int64_t)) != sizeof(int64_t)) { + return terrno; + } + } + + char reserve[SDB_RESERVE_SIZE_EXTRA] = {0}; if (taosWriteFile(pFile, reserve, sizeof(reserve)) != sizeof(reserve)) { return terrno; } diff --git a/source/dnode/mnode/sdb/src/sdbHash.c b/source/dnode/mnode/sdb/src/sdbHash.c index ea44a7c549..3f85ccb087 100644 --- a/source/dnode/mnode/sdb/src/sdbHash.c +++ b/source/dnode/mnode/sdb/src/sdbHash.c @@ -74,6 +74,8 @@ const char *sdbTableName(ESdbType type) { return "grant"; case SDB_ARBGROUP: return "arb_group"; + case SDB_ANODE: + return "anode"; default: return "undefine"; } From efcd56fb389697baa155e91fd1329ad187b57d9a Mon Sep 17 00:00:00 2001 From: dmchen Date: Wed, 9 Oct 2024 09:23:06 +0000 Subject: [PATCH 36/51] fix/TD-32475-reset-actionPos --- source/dnode/mnode/impl/src/mndTrans.c | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/source/dnode/mnode/impl/src/mndTrans.c b/source/dnode/mnode/impl/src/mndTrans.c index 40bb99d6b5..e16c6efa47 100644 --- a/source/dnode/mnode/impl/src/mndTrans.c +++ b/source/dnode/mnode/impl/src/mndTrans.c @@ -1495,7 +1495,7 @@ static int32_t mndTransExecuteActionsSerial(SMnode *pMnode, STrans *pTrans, SArr return code; } - mInfo("trans:%d, execute %d actions serial, current redoAction:%d", pTrans->id, numOfActions, pTrans->actionPos); + mInfo("trans:%d, execute %d actions serial, current action:%d", pTrans->id, numOfActions, pTrans->actionPos); for (int32_t action = pTrans->actionPos; action < numOfActions; ++action) { STransAction *pAction = taosArrayGet(pActions, action); @@ -1768,7 +1768,8 @@ static bool mndTransPerformRollbackStage(SMnode *pMnode, STrans *pTrans, bool to if (code == 0) { pTrans->stage = TRN_STAGE_UNDO_ACTION; - mInfo("trans:%d, stage from rollback to undoAction", pTrans->id); + pTrans->actionPos = 0; + mInfo("trans:%d, stage from rollback to undoAction, actionPos:%d", pTrans->id, pTrans->actionPos); continueExec = true; } else { pTrans->failedTimes++; From c36fe0af46b26f97fbc99260af559f4e72c744cf Mon Sep 17 00:00:00 2001 From: Shengliang Guan Date: Wed, 9 Oct 2024 17:33:27 +0800 Subject: [PATCH 37/51] feat: analysis msg --- source/common/src/systable.c | 17 ++ source/common/src/tmsg.c | 264 ++++++++++++++++++ source/dnode/mgmt/mgmt_dnode/src/dmHandle.c | 45 +++ source/dnode/mgmt/node_mgmt/src/dmTransport.c | 18 +- 4 files changed, 342 insertions(+), 2 deletions(-) diff --git a/source/common/src/systable.c b/source/common/src/systable.c index be841d9682..eef38bf18e 100644 --- a/source/common/src/systable.c +++ b/source/common/src/systable.c @@ -398,6 +398,21 @@ static const SSysDbTableSchema userCompactsDetailSchema[] = { {.name = "finished", .bytes = 4, .type = TSDB_DATA_TYPE_INT, .sysInfo = false}, {.name = "start_time", .bytes = 8, .type = TSDB_DATA_TYPE_TIMESTAMP, .sysInfo = false}, }; + +static const SSysDbTableSchema anodesSchema[] = { + {.name = "id", .bytes = 4, .type = TSDB_DATA_TYPE_INT, .sysInfo = false}, + {.name = "url", .bytes = TSDB_ANAL_ANODE_URL_LEN + VARSTR_HEADER_SIZE, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = true}, + {.name = "status", .bytes = 10 + VARSTR_HEADER_SIZE, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = true}, + {.name = "create_time", .bytes = 8, .type = TSDB_DATA_TYPE_TIMESTAMP, .sysInfo = true}, + {.name = "update_time", .bytes = 8, .type = TSDB_DATA_TYPE_TIMESTAMP, .sysInfo = true}, +}; + +static const SSysDbTableSchema anodesFullSchema[] = { + {.name = "id", .bytes = 4, .type = TSDB_DATA_TYPE_INT, .sysInfo = false}, + {.name = "type", .bytes = TSDB_ANAL_ALGO_TYPE_LEN + VARSTR_HEADER_SIZE, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = true}, + {.name = "algo", .bytes = TSDB_ANAL_ALGO_NAME_LEN + VARSTR_HEADER_SIZE, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = true}, +}; + static const SSysDbTableSchema tsmaSchema[] = { {.name = "tsma_name", .bytes = SYSTABLE_SCH_TABLE_NAME_LEN, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = false}, {.name = "db_name", .bytes = SYSTABLE_SCH_DB_NAME_LEN, .type = TSDB_DATA_TYPE_VARCHAR, .sysInfo = false}, @@ -472,6 +487,8 @@ static const SSysTableMeta infosMeta[] = { {TSDB_INS_TABLE_ARBGROUPS, arbGroupsSchema, tListLen(arbGroupsSchema), true}, {TSDB_INS_TABLE_ENCRYPTIONS, encryptionsSchema, tListLen(encryptionsSchema), true}, {TSDB_INS_TABLE_TSMAS, tsmaSchema, tListLen(tsmaSchema), false}, + {TSDB_INS_TABLE_ANODES, anodesSchema, tListLen(anodesSchema), true}, + {TSDB_INS_TABLE_ANODES_FULL, anodesFullSchema, tListLen(anodesFullSchema), true}, }; static const SSysDbTableSchema connectionsSchema[] = { diff --git a/source/common/src/tmsg.c b/source/common/src/tmsg.c index 986747fe58..63fcf900bf 100644 --- a/source/common/src/tmsg.c +++ b/source/common/src/tmsg.c @@ -40,6 +40,7 @@ #define TD_MSG_RANGE_CODE_ #include "tmsgdef.h" +#include "tanal.h" #include "tcol.h" #include "tlog.h" @@ -1453,6 +1454,7 @@ int32_t tSerializeSStatusReq(void *buf, int32_t bufLen, SStatusReq *pReq) { } TAOS_CHECK_EXIT(tEncodeI64(&encoder, pReq->ipWhiteVer)); + TAOS_CHECK_EXIT(tEncodeI64(&encoder, pReq->analVer)); TAOS_CHECK_EXIT(tSerializeSMonitorParas(&encoder, &pReq->clusterCfg.monitorParas)); tEndEncode(&encoder); @@ -1576,6 +1578,10 @@ int32_t tDeserializeSStatusReq(void *buf, int32_t bufLen, SStatusReq *pReq) { TAOS_CHECK_EXIT(tDecodeI64(&decoder, &pReq->ipWhiteVer)); } + if (!tDecodeIsEnd(&decoder)) { + TAOS_CHECK_EXIT(tDecodeI64(&decoder, &pReq->analVer)); + } + if (!tDecodeIsEnd(&decoder)) { TAOS_CHECK_EXIT(tDeserializeSMonitorParas(&decoder, &pReq->clusterCfg.monitorParas)); } @@ -1652,6 +1658,7 @@ int32_t tSerializeSStatusRsp(void *buf, int32_t bufLen, SStatusRsp *pRsp) { TAOS_CHECK_EXIT(tEncodeI32(&encoder, pRsp->statusSeq)); TAOS_CHECK_EXIT(tEncodeI64(&encoder, pRsp->ipWhiteVer)); + TAOS_CHECK_EXIT(tEncodeI64(&encoder, pRsp->analVer)); tEndEncode(&encoder); _exit: @@ -1703,6 +1710,11 @@ int32_t tDeserializeSStatusRsp(void *buf, int32_t bufLen, SStatusRsp *pRsp) { if (!tDecodeIsEnd(&decoder)) { TAOS_CHECK_EXIT(tDecodeI64(&decoder, &pRsp->ipWhiteVer)); } + + if (!tDecodeIsEnd(&decoder)) { + TAOS_CHECK_EXIT(tDecodeI64(&decoder, &pRsp->analVer)); + } + tEndDecode(&decoder); _exit: tDecoderClear(&decoder); @@ -2044,6 +2056,156 @@ _exit: return code; } +int32_t tSerializeRetrieveAnalAlgoReq(void *buf, int32_t bufLen, SRetrieveAnalAlgoReq *pReq) { + SEncoder encoder = {0}; + int32_t code = 0; + int32_t lino; + int32_t tlen; + tEncoderInit(&encoder, buf, bufLen); + + TAOS_CHECK_EXIT(tStartEncode(&encoder)); + TAOS_CHECK_EXIT(tEncodeI32(&encoder, pReq->dnodeId)); + TAOS_CHECK_EXIT(tEncodeI64(&encoder, pReq->analVer)); + tEndEncode(&encoder); + +_exit: + if (code) { + tlen = code; + } else { + tlen = encoder.pos; + } + tEncoderClear(&encoder); + return tlen; +} + +int32_t tDeserializeRetrieveAnalAlgoReq(void *buf, int32_t bufLen, SRetrieveAnalAlgoReq *pReq) { + SDecoder decoder = {0}; + int32_t code = 0; + int32_t lino; + + tDecoderInit(&decoder, buf, bufLen); + + TAOS_CHECK_EXIT(tStartDecode(&decoder)); + TAOS_CHECK_EXIT(tDecodeI32(&decoder, &pReq->dnodeId)); + TAOS_CHECK_EXIT(tDecodeI64(&decoder, &pReq->analVer)); + tEndDecode(&decoder); + +_exit: + tDecoderClear(&decoder); + return code; +} + +int32_t tSerializeRetrieveAnalAlgoRsp(void *buf, int32_t bufLen, SRetrieveAnalAlgoRsp *pRsp) { + SEncoder encoder = {0}; + int32_t code = 0; + int32_t lino; + int32_t tlen; + tEncoderInit(&encoder, buf, bufLen); + + int32_t numOfAlgos = 0; + void *pIter = taosHashIterate(pRsp->hash, NULL); + while (pIter != NULL) { + SAnalUrl *pUrl = pIter; + size_t nameLen = 0; + const char *name = taosHashGetKey(pIter, &nameLen); + if (nameLen > 0 && nameLen <= TSDB_ANAL_ALGO_KEY_LEN && pUrl->urlLen > 0) { + numOfAlgos++; + } + pIter = taosHashIterate(pRsp->hash, pIter); + } + + TAOS_CHECK_EXIT(tStartEncode(&encoder)); + TAOS_CHECK_EXIT(tEncodeI64(&encoder, pRsp->ver)); + TAOS_CHECK_EXIT(tEncodeI32(&encoder, numOfAlgos)); + + pIter = taosHashIterate(pRsp->hash, NULL); + while (pIter != NULL) { + SAnalUrl *pUrl = pIter; + size_t nameLen = 0; + const char *name = taosHashGetKey(pIter, &nameLen); + if (nameLen > 0 && pUrl->urlLen > 0) { + TAOS_CHECK_EXIT(tEncodeI32(&encoder, nameLen)); + TAOS_CHECK_EXIT(tEncodeBinary(&encoder, (const uint8_t *)name, nameLen)); + TAOS_CHECK_EXIT(tEncodeI32(&encoder, pUrl->anode)); + TAOS_CHECK_EXIT(tEncodeI32(&encoder, pUrl->type)); + TAOS_CHECK_EXIT(tEncodeI32(&encoder, pUrl->urlLen)); + TAOS_CHECK_EXIT(tEncodeBinary(&encoder, (const uint8_t *)pUrl->url, pUrl->urlLen)); + } + pIter = taosHashIterate(pRsp->hash, pIter); + } + + tEndEncode(&encoder); + +_exit: + if (code) { + tlen = code; + } else { + tlen = encoder.pos; + } + tEncoderClear(&encoder); + return tlen; +} + +int32_t tDeserializeRetrieveAnalAlgoRsp(void *buf, int32_t bufLen, SRetrieveAnalAlgoRsp *pRsp) { + if (pRsp->hash == NULL) { + pRsp->hash = taosHashInit(64, MurmurHash3_32, true, HASH_ENTRY_LOCK); + if (pRsp->hash == NULL) { + terrno = TSDB_CODE_OUT_OF_BUFFER; + return terrno; + } + } + + SDecoder decoder = {0}; + int32_t code = 0; + int32_t lino; + tDecoderInit(&decoder, buf, bufLen); + + int32_t numOfAlgos = 0; + int32_t nameLen; + int32_t type; + char name[TSDB_ANAL_ALGO_KEY_LEN]; + SAnalUrl url = {0}; + + TAOS_CHECK_EXIT(tStartDecode(&decoder)); + TAOS_CHECK_EXIT(tDecodeI64(&decoder, &pRsp->ver)); + TAOS_CHECK_EXIT(tDecodeI32(&decoder, &numOfAlgos)); + + for (int32_t f = 0; f < numOfAlgos; ++f) { + TAOS_CHECK_EXIT(tDecodeI32(&decoder, &nameLen)); + if (nameLen > 0 && nameLen <= TSDB_ANAL_ALGO_NAME_LEN) { + TAOS_CHECK_EXIT(tDecodeCStrTo(&decoder, name)); + } + + TAOS_CHECK_EXIT(tDecodeI32(&decoder, &url.anode)); + TAOS_CHECK_EXIT(tDecodeI32(&decoder, &type)); + url.type = (EAnalAlgoType)type; + TAOS_CHECK_EXIT(tDecodeI32(&decoder, &url.urlLen)); + if (url.urlLen > 0) { + TAOS_CHECK_EXIT(tDecodeBinaryAlloc(&decoder, (void **)&url.url, NULL) < 0); + } + + TAOS_CHECK_EXIT(taosHashPut(pRsp->hash, name, nameLen, &url, sizeof(SAnalUrl))); + } + + tEndDecode(&decoder); + +_exit: + tDecoderClear(&decoder); + return code; +} + +void tFreeRetrieveAnalAlgoRsp(SRetrieveAnalAlgoRsp *pRsp) { + void *pIter = taosHashIterate(pRsp->hash, NULL); + while (pIter != NULL) { + SAnalUrl *pUrl = (SAnalUrl *)pIter; + taosMemoryFree(pUrl->url); + pIter = taosHashIterate(pRsp->hash, pIter); + } + taosHashCleanup(pRsp->hash); + + pRsp->hash = NULL; +} + void tFreeSCreateUserReq(SCreateUserReq *pReq) { FREESQL(); taosMemoryFreeClear(pReq->pIpRanges); @@ -2961,6 +3123,108 @@ _exit: return code; } +int32_t tSerializeSMCreateAnodeReq(void *buf, int32_t bufLen, SMCreateAnodeReq *pReq) { + SEncoder encoder = {0}; + int32_t code = 0; + int32_t lino; + int32_t tlen; + tEncoderInit(&encoder, buf, bufLen); + + TAOS_CHECK_EXIT(tStartEncode(&encoder)); + TAOS_CHECK_EXIT(tEncodeI32(&encoder, pReq->urlLen)); + if (pReq->urlLen > 0) { + TAOS_CHECK_EXIT(tEncodeBinary(&encoder, (const uint8_t *)pReq->url, pReq->urlLen)); + } + ENCODESQL(); + tEndEncode(&encoder); + +_exit: + if (code) { + tlen = code; + } else { + tlen = encoder.pos; + } + tEncoderClear(&encoder); + return tlen; +} + +int32_t tDeserializeSMCreateAnodeReq(void *buf, int32_t bufLen, SMCreateAnodeReq *pReq) { + SDecoder decoder = {0}; + int32_t code = 0; + int32_t lino; + + tDecoderInit(&decoder, buf, bufLen); + + TAOS_CHECK_EXIT(tStartDecode(&decoder)); + TAOS_CHECK_EXIT(tDecodeI32(&decoder, &pReq->urlLen)); + if (pReq->urlLen > 0) { + TAOS_CHECK_EXIT(tDecodeBinaryAlloc(&decoder, (void **)&pReq->url, NULL)); + } + + DECODESQL(); + tEndDecode(&decoder); + +_exit: + tDecoderClear(&decoder); + return code; +} + +void tFreeSMCreateAnodeReq(SMCreateAnodeReq *pReq) { + taosMemoryFreeClear(pReq->url); + FREESQL(); +} + +int32_t tSerializeSMDropAnodeReq(void *buf, int32_t bufLen, SMDropAnodeReq *pReq) { + SEncoder encoder = {0}; + int32_t code = 0; + int32_t lino; + int32_t tlen; + tEncoderInit(&encoder, buf, bufLen); + + TAOS_CHECK_EXIT(tStartEncode(&encoder)); + TAOS_CHECK_EXIT(tEncodeI32(&encoder, pReq->anodeId)); + ENCODESQL(); + tEndEncode(&encoder); + +_exit: + if (code) { + tlen = code; + } else { + tlen = encoder.pos; + } + tEncoderClear(&encoder); + return tlen; +} + +int32_t tDeserializeSMDropAnodeReq(void *buf, int32_t bufLen, SMDropAnodeReq *pReq) { + SDecoder decoder = {0}; + int32_t code = 0; + int32_t lino; + + tDecoderInit(&decoder, buf, bufLen); + + TAOS_CHECK_EXIT(tStartDecode(&decoder)); + TAOS_CHECK_EXIT(tDecodeI32(&decoder, &pReq->anodeId)); + DECODESQL(); + tEndDecode(&decoder); + +_exit: + tDecoderClear(&decoder); + return code; +} + +void tFreeSMDropAnodeReq(SMDropAnodeReq *pReq) { FREESQL(); } + +int32_t tSerializeSMUpdateAnodeReq(void *buf, int32_t bufLen, SMUpdateAnodeReq *pReq) { + return tSerializeSMDropAnodeReq(buf, bufLen, pReq); +} + +int32_t tDeserializeSMUpdateAnodeReq(void *buf, int32_t bufLen, SMUpdateAnodeReq *pReq) { + return tDeserializeSMDropAnodeReq(buf, bufLen, pReq); +} + +void tFreeSMUpdateAnodeReq(SMUpdateAnodeReq *pReq) { tFreeSMDropAnodeReq(pReq); } + int32_t tSerializeSCreateDnodeReq(void *buf, int32_t bufLen, SCreateDnodeReq *pReq) { SEncoder encoder = {0}; int32_t code = 0; diff --git a/source/dnode/mgmt/mgmt_dnode/src/dmHandle.c b/source/dnode/mgmt/mgmt_dnode/src/dmHandle.c index 419c669103..bc33fc43dc 100644 --- a/source/dnode/mgmt/mgmt_dnode/src/dmHandle.c +++ b/source/dnode/mgmt/mgmt_dnode/src/dmHandle.c @@ -18,6 +18,7 @@ #include "dmInt.h" #include "monitor.h" #include "systable.h" +#include "tanal.h" #include "tchecksum.h" extern SConfig *tsCfg; @@ -39,6 +40,7 @@ static void dmUpdateDnodeCfg(SDnodeMgmt *pMgmt, SDnodeCfg *pCfg) { (void)taosThreadRwlockUnlock(&pMgmt->pData->lock); } } + static void dmMayShouldUpdateIpWhiteList(SDnodeMgmt *pMgmt, int64_t ver) { int32_t code = 0; dDebug("ip-white-list on dnode ver: %" PRId64 ", status ver: %" PRId64 "", pMgmt->pData->ipWhiteVer, ver); @@ -84,6 +86,47 @@ static void dmMayShouldUpdateIpWhiteList(SDnodeMgmt *pMgmt, int64_t ver) { dError("failed to send retrieve ip white list request since:%s", tstrerror(code)); } } + +static void dmMayShouldUpdateAnalFunc(SDnodeMgmt *pMgmt, int64_t newVer) { + int32_t code = 0; + int64_t oldVer = taosAnalGetVersion(); + if (oldVer == newVer) return; + dDebug("analysis on dnode ver:%" PRId64 ", status ver:%" PRId64, oldVer, newVer); + + SRetrieveAnalAlgoReq req = {.dnodeId = pMgmt->pData->dnodeId, .analVer = oldVer}; + int32_t contLen = tSerializeRetrieveAnalAlgoReq(NULL, 0, &req); + if (contLen < 0) { + dError("failed to serialize analysis function ver request since %s", tstrerror(contLen)); + return; + } + + void *pHead = rpcMallocCont(contLen); + contLen = tSerializeRetrieveAnalAlgoReq(pHead, contLen, &req); + if (contLen < 0) { + rpcFreeCont(pHead); + dError("failed to serialize analysis function ver request since %s", tstrerror(contLen)); + return; + } + + SRpcMsg rpcMsg = { + .pCont = pHead, + .contLen = contLen, + .msgType = TDMT_MND_RETRIEVE_ANAL_ALGO, + .info.ahandle = (void *)0x9527, + .info.refId = 0, + .info.noResp = 0, + .info.handle = 0, + }; + SEpSet epset = {0}; + + (void)dmGetMnodeEpSet(pMgmt->pData, &epset); + + code = rpcSendRequest(pMgmt->msgCb.clientRpc, &epset, &rpcMsg, NULL); + if (code != 0) { + dError("failed to send retrieve analysis func ver request since %s", tstrerror(code)); + } +} + static void dmProcessStatusRsp(SDnodeMgmt *pMgmt, SRpcMsg *pRsp) { const STraceId *trace = &pRsp->info.traceId; dGTrace("status rsp received from mnode, statusSeq:%d code:0x%x", pMgmt->statusSeq, pRsp->code); @@ -111,6 +154,7 @@ static void dmProcessStatusRsp(SDnodeMgmt *pMgmt, SRpcMsg *pRsp) { dmUpdateEps(pMgmt->pData, statusRsp.pDnodeEps); } dmMayShouldUpdateIpWhiteList(pMgmt, statusRsp.ipWhiteVer); + dmMayShouldUpdateAnalFunc(pMgmt, statusRsp.analVer); } tFreeSStatusRsp(&statusRsp); } @@ -172,6 +216,7 @@ void dmSendStatusReq(SDnodeMgmt *pMgmt) { pMgmt->statusSeq++; req.statusSeq = pMgmt->statusSeq; req.ipWhiteVer = pMgmt->pData->ipWhiteVer; + req.analVer = taosAnalGetVersion(); int32_t contLen = tSerializeSStatusReq(NULL, 0, &req); if (contLen < 0) { diff --git a/source/dnode/mgmt/node_mgmt/src/dmTransport.c b/source/dnode/mgmt/node_mgmt/src/dmTransport.c index b9f4ab54f4..e84d756e0a 100644 --- a/source/dnode/mgmt/node_mgmt/src/dmTransport.c +++ b/source/dnode/mgmt/node_mgmt/src/dmTransport.c @@ -17,6 +17,7 @@ #include "dmMgmt.h" #include "qworker.h" #include "tversion.h" +#include "tanal.h" static inline void dmSendRsp(SRpcMsg *pMsg) { if (rpcSendResponse(pMsg) != 0) { @@ -105,6 +106,17 @@ static bool dmIsForbiddenIp(int8_t forbidden, char *user, uint32_t clientIp) { return false; } } + +static void dmUpdateAnalFunc(SDnodeData *pData, void *pTrans, SRpcMsg *pRpc) { + SRetrieveAnalAlgoRsp rsp = {0}; + if (tDeserializeRetrieveAnalAlgoRsp(pRpc->pCont, pRpc->contLen, &rsp) == 0) { + taosAnalUpdate(rsp.ver, rsp.hash); + rsp.hash = NULL; + } + tFreeRetrieveAnalAlgoRsp(&rsp); + rpcFreeCont(pRpc->pCont); +} + static void dmProcessRpcMsg(SDnode *pDnode, SRpcMsg *pRpc, SEpSet *pEpSet) { SDnodeTrans *pTrans = &pDnode->trans; int32_t code = -1; @@ -150,10 +162,12 @@ static void dmProcessRpcMsg(SDnode *pDnode, SRpcMsg *pRpc, SEpSet *pEpSet) { dmSetMnodeEpSet(&pDnode->data, pEpSet); } break; - case TDMT_MND_RETRIEVE_IP_WHITE_RSP: { + case TDMT_MND_RETRIEVE_IP_WHITE_RSP: dmUpdateRpcIpWhite(&pDnode->data, pTrans->serverRpc, pRpc); return; - } break; + case TDMT_MND_RETRIEVE_ANAL_ALGO_RSP: + dmUpdateAnalFunc(&pDnode->data, pTrans->serverRpc, pRpc); + return; default: break; } From 22f50c4bbff8276f6ba4e220f091c85294b1a365 Mon Sep 17 00:00:00 2001 From: Shengliang Guan Date: Wed, 9 Oct 2024 17:46:40 +0800 Subject: [PATCH 38/51] feat: analysis operator --- .../libs/executor/src/anomalywindowoperator.c | 609 ++++++++++++++++ source/libs/executor/src/executil.c | 10 +- source/libs/executor/src/forecastoperator.c | 663 ++++++++++++++++++ source/libs/executor/src/operator.c | 4 + 4 files changed, 1283 insertions(+), 3 deletions(-) create mode 100644 source/libs/executor/src/anomalywindowoperator.c create mode 100644 source/libs/executor/src/forecastoperator.c diff --git a/source/libs/executor/src/anomalywindowoperator.c b/source/libs/executor/src/anomalywindowoperator.c new file mode 100644 index 0000000000..7267bbbe09 --- /dev/null +++ b/source/libs/executor/src/anomalywindowoperator.c @@ -0,0 +1,609 @@ +/* + * Copyright (c) 2019 TAOS Data, Inc. + * + * This program is free software: you can use, redistribute, and/or modify + * it under the terms of the GNU Affero General Public License, version 3 + * or later ("AGPL"), as published by the Free Software Foundation. + * + * This program is distributed in the hope that it will be useful, but WITHOUT + * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or + * FITNESS FOR A PARTICULAR PURPOSE. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + */ + +#include "executorInt.h" +#include "filter.h" +#include "function.h" +#include "functionMgt.h" +#include "operator.h" +#include "querytask.h" +#include "tanal.h" +#include "tcommon.h" +#include "tcompare.h" +#include "tdatablock.h" +#include "tjson.h" +#include "ttime.h" + +#ifdef USE_ANAL + +typedef struct { + SArray* blocks; // SSDataBlock* + SArray* windows; // STimeWindow + uint64_t groupId; + int64_t numOfRows; + int32_t curWinIndex; + STimeWindow curWin; + SResultRow* pResultRow; +} SAnomalyWindowSupp; + +typedef struct { + SOptrBasicInfo binfo; + SAggSupporter aggSup; + SExprSupp scalarSup; + int32_t tsSlotId; + STimeWindowAggSupp twAggSup; + char algoName[TSDB_ANAL_ALGO_NAME_LEN]; + char algoUrl[TSDB_ANAL_ALGO_URL_LEN]; + char anomalyOpt[TSDB_ANAL_ALGO_OPTION_LEN]; + SAnomalyWindowSupp anomalySup; + SWindowRowsSup anomalyWinRowSup; + SColumn anomalyCol; + SStateKeys anomalyKey; +} SAnomalyWindowOperatorInfo; + +static void anomalyDestroyOperatorInfo(void* param); +static int32_t anomalyAggregateNext(SOperatorInfo* pOperator, SSDataBlock** ppRes); +static void anomalyAggregateBlocks(SOperatorInfo* pOperator); +static int32_t anomalyCacheBlock(SAnomalyWindowOperatorInfo* pInfo, SSDataBlock* pBlock); + +int32_t createAnomalywindowOperatorInfo(SOperatorInfo* downstream, SPhysiNode* physiNode, SExecTaskInfo* pTaskInfo, + SOperatorInfo** pOptrInfo) { + QRY_PARAM_CHECK(pOptrInfo); + + int32_t code = TSDB_CODE_SUCCESS; + int32_t lino = 0; + SAnomalyWindowOperatorInfo* pInfo = taosMemoryCalloc(1, sizeof(SAnomalyWindowOperatorInfo)); + SOperatorInfo* pOperator = taosMemoryCalloc(1, sizeof(SOperatorInfo)); + SAnomalyWindowPhysiNode* pAnomalyNode = (SAnomalyWindowPhysiNode*)physiNode; + SColumnNode* pColNode = (SColumnNode*)(pAnomalyNode->pAnomalyKey); + if (pInfo == NULL || pOperator == NULL) { + code = terrno; + goto _error; + } + + if (!taosAnalGetOptStr(pAnomalyNode->anomalyOpt, "algo", pInfo->algoName, sizeof(pInfo->algoName))) { + qError("failed to get anomaly_window algorithm name from %s", pAnomalyNode->anomalyOpt); + code = TSDB_CODE_ANAL_ALGO_NOT_FOUND; + goto _error; + } + if (taosAnalGetAlgoUrl(pInfo->algoName, ANAL_ALGO_TYPE_ANOMALY_DETECT, pInfo->algoUrl, sizeof(pInfo->algoUrl)) != 0) { + qError("failed to get anomaly_window algorithm url from %s", pInfo->algoName); + code = TSDB_CODE_ANAL_ALGO_NOT_LOAD; + goto _error; + } + + pOperator->exprSupp.hasWindowOrGroup = true; + pInfo->tsSlotId = ((SColumnNode*)pAnomalyNode->window.pTspk)->slotId; + strncpy(pInfo->anomalyOpt, pAnomalyNode->anomalyOpt, sizeof(pInfo->anomalyOpt)); + + if (pAnomalyNode->window.pExprs != NULL) { + int32_t numOfScalarExpr = 0; + SExprInfo* pScalarExprInfo = NULL; + code = createExprInfo(pAnomalyNode->window.pExprs, NULL, &pScalarExprInfo, &numOfScalarExpr); + QUERY_CHECK_CODE(code, lino, _error); + code = initExprSupp(&pInfo->scalarSup, pScalarExprInfo, numOfScalarExpr, &pTaskInfo->storageAPI.functionStore); + QUERY_CHECK_CODE(code, lino, _error); + } + + size_t keyBufSize = 0; + int32_t num = 0; + SExprInfo* pExprInfo = NULL; + code = createExprInfo(pAnomalyNode->window.pFuncs, NULL, &pExprInfo, &num); + QUERY_CHECK_CODE(code, lino, _error); + + initResultSizeInfo(&pOperator->resultInfo, 4096); + + code = initAggSup(&pOperator->exprSupp, &pInfo->aggSup, pExprInfo, num, keyBufSize, pTaskInfo->id.str, + pTaskInfo->streamInfo.pState, &pTaskInfo->storageAPI.functionStore); + QUERY_CHECK_CODE(code, lino, _error); + + SSDataBlock* pResBlock = createDataBlockFromDescNode(pAnomalyNode->window.node.pOutputDataBlockDesc); + QUERY_CHECK_NULL(pResBlock, code, lino, _error, terrno); + initBasicInfo(&pInfo->binfo, pResBlock); + + code = blockDataEnsureCapacity(pResBlock, pOperator->resultInfo.capacity); + QUERY_CHECK_CODE(code, lino, _error); + + initResultRowInfo(&pInfo->binfo.resultRowInfo); + pInfo->binfo.inputTsOrder = pAnomalyNode->window.node.inputTsOrder; + pInfo->binfo.outputTsOrder = pAnomalyNode->window.node.outputTsOrder; + + pInfo->anomalyCol = extractColumnFromColumnNode(pColNode); + pInfo->anomalyKey.type = pInfo->anomalyCol.type; + pInfo->anomalyKey.bytes = pInfo->anomalyCol.bytes; + pInfo->anomalyKey.pData = taosMemoryCalloc(1, pInfo->anomalyCol.bytes); + if (pInfo->anomalyKey.pData == NULL) { + goto _error; + } + + int32_t itemSize = sizeof(int32_t) + pInfo->aggSup.resultRowSize + pInfo->anomalyKey.bytes; + pInfo->anomalySup.pResultRow = taosMemoryCalloc(1, itemSize); + pInfo->anomalySup.blocks = taosArrayInit(16, sizeof(SSDataBlock*)); + pInfo->anomalySup.windows = taosArrayInit(16, sizeof(STimeWindow)); + + if (pInfo->anomalySup.windows == NULL || pInfo->anomalySup.blocks == NULL || pInfo->anomalySup.pResultRow == NULL) { + code = TSDB_CODE_OUT_OF_MEMORY; + goto _error; + } + + code = filterInitFromNode((SNode*)pAnomalyNode->window.node.pConditions, &pOperator->exprSupp.pFilterInfo, 0); + QUERY_CHECK_CODE(code, lino, _error); + + code = initExecTimeWindowInfo(&pInfo->twAggSup.timeWindowData, &pTaskInfo->window); + QUERY_CHECK_CODE(code, lino, _error); + + setOperatorInfo(pOperator, "AnomalyWindowOperator", QUERY_NODE_PHYSICAL_PLAN_MERGE_ANOMALY, true, OP_NOT_OPENED, + pInfo, pTaskInfo); + pOperator->fpSet = createOperatorFpSet(optrDummyOpenFn, anomalyAggregateNext, NULL, anomalyDestroyOperatorInfo, + optrDefaultBufFn, NULL, optrDefaultGetNextExtFn, NULL); + + code = appendDownstream(pOperator, &downstream, 1); + QUERY_CHECK_CODE(code, lino, _error); + + *pOptrInfo = pOperator; + + qDebug("anomaly_window operator is created, algo:%s url:%s opt:%s", pInfo->algoName, pInfo->algoUrl, + pInfo->anomalyOpt); + return TSDB_CODE_SUCCESS; + +_error: + if (pInfo != NULL) { + anomalyDestroyOperatorInfo(pInfo); + } + + destroyOperatorAndDownstreams(pOperator, &downstream, 1); + pTaskInfo->code = code; + qError("failed to create anomaly_window operator, algo:%s code:0x%x", pInfo->algoName, code); + return code; +} + +static int32_t anomalyAggregateNext(SOperatorInfo* pOperator, SSDataBlock** ppRes) { + int32_t code = TSDB_CODE_SUCCESS; + int32_t lino = 0; + SAnomalyWindowOperatorInfo* pInfo = pOperator->info; + SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; + SOptrBasicInfo* pBInfo = &pInfo->binfo; + SAnomalyWindowSupp* pSupp = &pInfo->anomalySup; + SSDataBlock* pRes = pInfo->binfo.pRes; + int64_t st = taosGetTimestampUs(); + int32_t numOfBlocks = taosArrayGetSize(pSupp->blocks); + + blockDataCleanup(pRes); + + while (1) { + SSDataBlock* pBlock = getNextBlockFromDownstream(pOperator, 0); + if (pBlock == NULL) { + break; + } + + if (pSupp->groupId == 0 || pSupp->groupId == pBlock->info.id.groupId) { + pSupp->groupId = pBlock->info.id.groupId; + numOfBlocks++; + qDebug("group:%" PRId64 ", blocks:%d, cache block rows:%" PRId64, pSupp->groupId, numOfBlocks, pBlock->info.rows); + code = anomalyCacheBlock(pInfo, pBlock); + QUERY_CHECK_CODE(code, lino, _end); + } else { + qDebug("group:%" PRId64 ", read finish for new group coming, blocks:%d", pSupp->groupId, numOfBlocks); + anomalyAggregateBlocks(pOperator); + pSupp->groupId = pBlock->info.id.groupId; + numOfBlocks = 1; + qDebug("group:%" PRId64 ", new group, cache block rows:%" PRId64, pSupp->groupId, pBlock->info.rows); + code = anomalyCacheBlock(pInfo, pBlock); + QUERY_CHECK_CODE(code, lino, _end); + } + + if (pRes->info.rows > 0) { + (*ppRes) = pRes; + qDebug("group:%" PRId64 ", return to upstream, blocks:%d", pRes->info.id.groupId, numOfBlocks); + return code; + } + } + + if (numOfBlocks > 0) { + qDebug("group:%" PRId64 ", read finish, blocks:%d", pInfo->anomalySup.groupId, numOfBlocks); + anomalyAggregateBlocks(pOperator); + } + + int64_t cost = taosGetTimestampUs() - st; + qDebug("all groups finished, cost:%" PRId64 "us", cost); + +_end: + if (code != TSDB_CODE_SUCCESS) { + qError("%s failed at line %d since %s", __func__, lino, tstrerror(code)); + pTaskInfo->code = code; + T_LONG_JMP(pTaskInfo->env, code); + } + (*ppRes) = (pBInfo->pRes->info.rows == 0) ? NULL : pBInfo->pRes; + return code; +} + +static void anomalyDestroyOperatorInfo(void* param) { + SAnomalyWindowOperatorInfo* pInfo = (SAnomalyWindowOperatorInfo*)param; + if (pInfo == NULL) return; + + qDebug("anomaly_window operator is destroyed, algo:%s", pInfo->algoName); + + cleanupBasicInfo(&pInfo->binfo); + cleanupAggSup(&pInfo->aggSup); + cleanupExprSupp(&pInfo->scalarSup); + colDataDestroy(&pInfo->twAggSup.timeWindowData); + + for (int32_t i = 0; i < taosArrayGetSize(pInfo->anomalySup.blocks); ++i) { + SSDataBlock* pBlock = taosArrayGetP(pInfo->anomalySup.blocks, i); + blockDataDestroy(pBlock); + } + taosArrayDestroy(pInfo->anomalySup.blocks); + taosArrayDestroy(pInfo->anomalySup.windows); + taosMemoryFreeClear(pInfo->anomalySup.pResultRow); + taosMemoryFreeClear(pInfo->anomalyKey.pData); + + taosMemoryFreeClear(param); +} + +static int32_t anomalyCacheBlock(SAnomalyWindowOperatorInfo* pInfo, SSDataBlock* pSrc) { + SSDataBlock* pDst = NULL; + int32_t code = createOneDataBlock(pSrc, true, &pDst); + + if (code != 0) return code; + if (pDst == NULL) return TSDB_CODE_OUT_OF_MEMORY; + if (taosArrayPush(pInfo->anomalySup.blocks, &pDst) == NULL) return TSDB_CODE_OUT_OF_MEMORY; + + return 0; +} + +static int32_t anomalyFindWindow(SAnomalyWindowSupp* pSupp, TSKEY key) { + for (int32_t i = pSupp->curWinIndex; i < taosArrayGetSize(pSupp->windows); ++i) { + STimeWindow* pWindow = taosArrayGet(pSupp->windows, i); + if (key >= pWindow->skey && key < pWindow->ekey) { + pSupp->curWin = *pWindow; + pSupp->curWinIndex = i; + return 0; + } + } + return -1; +} + +static int32_t anomalyParseJson(SJson* pJson, SArray* pWindows) { + int32_t code = 0; + int32_t rows = 0; + STimeWindow win = {0}; + + taosArrayClear(pWindows); + + tjsonGetInt32ValueFromDouble(pJson, "rows", rows, code); + if (code < 0) return TSDB_CODE_INVALID_JSON_FORMAT; + if (rows <= 0) return 0; + + SJson* res = tjsonGetObjectItem(pJson, "res"); + if (res == NULL) return TSDB_CODE_INVALID_JSON_FORMAT; + + int32_t ressize = tjsonGetArraySize(res); + if (ressize != rows) return TSDB_CODE_INVALID_JSON_FORMAT; + + for (int32_t i = 0; i < rows; ++i) { + SJson* row = tjsonGetArrayItem(res, i); + if (row == NULL) return TSDB_CODE_INVALID_JSON_FORMAT; + + int32_t colsize = tjsonGetArraySize(row); + if (colsize != 2) return TSDB_CODE_INVALID_JSON_FORMAT; + + SJson* start = tjsonGetArrayItem(row, 0); + SJson* end = tjsonGetArrayItem(row, 1); + if (start == NULL || end == NULL) return TSDB_CODE_INVALID_JSON_FORMAT; + + tjsonGetObjectValueBigInt(start, &win.skey); + tjsonGetObjectValueBigInt(end, &win.ekey); + + if (win.skey >= win.ekey) { + win.ekey = win.skey + 1; + } + + if (taosArrayPush(pWindows, &win) == NULL) return TSDB_CODE_OUT_OF_BUFFER; + } + + int32_t numOfWins = taosArrayGetSize(pWindows); + qDebug("anomaly window recevied, total:%d", numOfWins); + for (int32_t i = 0; i < numOfWins; ++i) { + STimeWindow* pWindow = taosArrayGet(pWindows, i); + qDebug("anomaly win:%d [%" PRId64 ", %" PRId64 ")", i, pWindow->skey, pWindow->ekey); + } + + return 0; +} + +static int32_t anomalyAnalysisWindow(SOperatorInfo* pOperator) { + SAnomalyWindowOperatorInfo* pInfo = pOperator->info; + SAnomalyWindowSupp* pSupp = &pInfo->anomalySup; + SJson* pJson = NULL; + SAnalBuf analBuf = {.bufType = ANAL_BUF_TYPE_JSON}; + char dataBuf[64] = {0}; + int32_t code = 0; + + int64_t ts = 0; + // int64_t ts = taosGetTimestampMs(); + snprintf(analBuf.fileName, sizeof(analBuf.fileName), "%s/tdengine-anomaly-%" PRId64 "-%" PRId64, tsTempDir, ts, + pSupp->groupId); + code = tsosAnalBufOpen(&analBuf, 2); + if (code != 0) goto _OVER; + + const char* prec = TSDB_TIME_PRECISION_MILLI_STR; + if (pInfo->anomalyCol.precision == TSDB_TIME_PRECISION_MICRO) prec = TSDB_TIME_PRECISION_MICRO_STR; + if (pInfo->anomalyCol.precision == TSDB_TIME_PRECISION_NANO) prec = TSDB_TIME_PRECISION_NANO_STR; + + code = taosAnalBufWriteOptStr(&analBuf, "algo", pInfo->algoName); + if (code != 0) goto _OVER; + + code = taosAnalBufWriteOptStr(&analBuf, "prec", prec); + if (code != 0) goto _OVER; + + code = taosAnalBufWriteColMeta(&analBuf, 0, TSDB_DATA_TYPE_TIMESTAMP, "ts"); + if (code != 0) goto _OVER; + + code = taosAnalBufWriteColMeta(&analBuf, 1, pInfo->anomalyCol.type, "val"); + if (code != 0) goto _OVER; + + code = taosAnalBufWriteDataBegin(&analBuf); + if (code != 0) goto _OVER; + + int32_t numOfBlocks = (int32_t)taosArrayGetSize(pSupp->blocks); + + // timestamp + code = taosAnalBufWriteColBegin(&analBuf, 0); + if (code != 0) goto _OVER; + for (int32_t i = 0; i < numOfBlocks; ++i) { + SSDataBlock* pBlock = taosArrayGetP(pSupp->blocks, i); + if (pBlock == NULL) break; + SColumnInfoData* pTsCol = taosArrayGet(pBlock->pDataBlock, pInfo->tsSlotId); + if (pTsCol == NULL) break; + for (int32_t j = 0; j < pBlock->info.rows; ++j) { + code = taosAnalBufWriteColData(&analBuf, 0, TSDB_DATA_TYPE_TIMESTAMP, &((TSKEY*)pTsCol->pData)[j]); + if (code != 0) goto _OVER; + } + } + code = taosAnalBufWriteColEnd(&analBuf, 0); + if (code != 0) goto _OVER; + + // data + code = taosAnalBufWriteColBegin(&analBuf, 1); + if (code != 0) goto _OVER; + for (int32_t i = 0; i < numOfBlocks; ++i) { + SSDataBlock* pBlock = taosArrayGetP(pSupp->blocks, i); + if (pBlock == NULL) break; + SColumnInfoData* pValCol = taosArrayGet(pBlock->pDataBlock, pInfo->anomalyCol.slotId); + if (pValCol == NULL) break; + + for (int32_t j = 0; j < pBlock->info.rows; ++j) { + code = taosAnalBufWriteColData(&analBuf, 1, pValCol->info.type, colDataGetData(pValCol, j)); + if (code != 0) goto _OVER; + if (code != 0) goto _OVER; + } + } + code = taosAnalBufWriteColEnd(&analBuf, 1); + if (code != 0) goto _OVER; + + code = taosAnalBufWriteDataEnd(&analBuf); + if (code != 0) goto _OVER; + + code = taosAnalBufWriteOptStr(&analBuf, "option", pInfo->anomalyOpt); + if (code != 0) goto _OVER; + + code = taosAnalBufClose(&analBuf); + if (code != 0) goto _OVER; + + pJson = taosAnalSendReqRetJson(pInfo->algoUrl, ANAL_HTTP_TYPE_POST, &analBuf); + if (pJson == NULL) { + code = terrno; + goto _OVER; + } + + code = anomalyParseJson(pJson, pSupp->windows); + if (code != 0) goto _OVER; + +_OVER: + if (code != 0) { + qError("failed to analysis window since %s", tstrerror(code)); + } + taosAnalBufDestroy(&analBuf); + if (pJson != NULL) tjsonDelete(pJson); + return code; +} + +static void anomalyAggregateRows(SOperatorInfo* pOperator, SSDataBlock* pBlock) { + SAnomalyWindowOperatorInfo* pInfo = pOperator->info; + SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; + SExprSupp* pExprSup = &pOperator->exprSupp; + SAnomalyWindowSupp* pSupp = &pInfo->anomalySup; + SWindowRowsSup* pRowSup = &pInfo->anomalyWinRowSup; + SResultRow* pResRow = pSupp->pResultRow; + int32_t numOfOutput = pOperator->exprSupp.numOfExprs; + + if (setResultRowInitCtx(pResRow, pExprSup->pCtx, pExprSup->numOfExprs, pExprSup->rowEntryInfoOffset) == 0) { + updateTimeWindowInfo(&pInfo->twAggSup.timeWindowData, &pSupp->curWin, 0); + applyAggFunctionOnPartialTuples(pTaskInfo, pExprSup->pCtx, &pInfo->twAggSup.timeWindowData, pRowSup->startRowIndex, + pRowSup->numOfRows, pBlock->info.rows, numOfOutput); + } +} + +static void anomalyBuildResult(SOperatorInfo* pOperator) { + SAnomalyWindowOperatorInfo* pInfo = pOperator->info; + SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; + SExprSupp* pExprSup = &pOperator->exprSupp; + SSDataBlock* pRes = pInfo->binfo.pRes; + SResultRow* pResRow = pInfo->anomalySup.pResultRow; + + doUpdateNumOfRows(pExprSup->pCtx, pResRow, pExprSup->numOfExprs, pExprSup->rowEntryInfoOffset); + copyResultrowToDataBlock(pExprSup->pExprInfo, pExprSup->numOfExprs, pResRow, pExprSup->pCtx, pRes, + pExprSup->rowEntryInfoOffset, pTaskInfo); + pRes->info.rows += pResRow->numOfRows; + clearResultRowInitFlag(pExprSup->pCtx, pExprSup->numOfExprs); +} + +static void anomalyAggregateBlocks(SOperatorInfo* pOperator) { + int32_t code = TSDB_CODE_SUCCESS; + int32_t lino = 0; + SAnomalyWindowOperatorInfo* pInfo = pOperator->info; + SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; + SExprSupp* pExprSup = &pOperator->exprSupp; + SSDataBlock* pRes = pInfo->binfo.pRes; + SAnomalyWindowSupp* pSupp = &pInfo->anomalySup; + SWindowRowsSup* pRowSup = &pInfo->anomalyWinRowSup; + SResultRow* pResRow = pSupp->pResultRow; + int32_t numOfOutput = pOperator->exprSupp.numOfExprs; + int32_t rowsInWin = 0; + int32_t rowsInBlock = 0; + const int64_t gid = pSupp->groupId; + const int32_t order = pInfo->binfo.inputTsOrder; + + int32_t numOfBlocks = (int32_t)taosArrayGetSize(pSupp->blocks); + if (numOfBlocks == 0) goto _OVER; + + qDebug("group:%" PRId64 ", aggregate blocks, blocks:%d", pSupp->groupId, numOfBlocks); + pRes->info.id.groupId = pSupp->groupId; + + code = anomalyAnalysisWindow(pOperator); + QUERY_CHECK_CODE(code, lino, _OVER); + + int32_t numOfWins = taosArrayGetSize(pSupp->windows); + qDebug("group:%" PRId64 ", wins:%d, rows:%" PRId64, pSupp->groupId, numOfWins, pSupp->numOfRows); + for (int32_t w = 0; w < numOfWins; ++w) { + STimeWindow* pWindow = taosArrayGet(pSupp->windows, w); + if (w == 0) { + pSupp->curWin = *pWindow; + pRowSup->win.skey = pSupp->curWin.skey; + } + qDebug("group:%" PRId64 ", win:%d [%" PRId64 ", %" PRId64 ")", pSupp->groupId, w, pWindow->skey, pWindow->ekey); + } + + if (numOfWins <= 0) goto _OVER; + if (numOfWins > pRes->info.capacity) { + code = blockDataEnsureCapacity(pRes, numOfWins); + QUERY_CHECK_CODE(code, lino, _OVER); + } + + for (int32_t b = 0; b < numOfBlocks; ++b) { + SSDataBlock* pBlock = taosArrayGetP(pSupp->blocks, b); + if (pBlock == NULL) break; + + pRes->info.scanFlag = pBlock->info.scanFlag; + code = setInputDataBlock(pExprSup, pBlock, order, MAIN_SCAN, true); + if (code != 0) break; + + code = blockDataUpdateTsWindow(pBlock, pInfo->tsSlotId); + if (code != 0) break; + + // there is an scalar expression that needs to be calculated right before apply the group aggregation. + if (pInfo->scalarSup.pExprInfo != NULL) { + code = projectApplyFunctions(pInfo->scalarSup.pExprInfo, pBlock, pBlock, pInfo->scalarSup.pCtx, + pInfo->scalarSup.numOfExprs, NULL); + if (code != 0) break; + } + + SColumnInfoData* pValCol = taosArrayGet(pBlock->pDataBlock, pInfo->anomalyCol.slotId); + if (pValCol == NULL) break; + SColumnInfoData* pTsCol = taosArrayGet(pBlock->pDataBlock, pInfo->tsSlotId); + if (pTsCol == NULL) break; + TSKEY* tsList = (TSKEY*)pTsCol->pData; + bool lastBlock = (b == numOfBlocks - 1); + + qTrace("group:%" PRId64 ", block:%d win:%d, riwin:%d riblock:%d, rows:%" PRId64, pSupp->groupId, b, + pSupp->curWinIndex, rowsInWin, rowsInBlock, pBlock->info.rows); + + for (int32_t r = 0; r < pBlock->info.rows; ++r) { + TSKEY key = tsList[r]; + bool keyInWin = (key >= pSupp->curWin.skey && key < pSupp->curWin.ekey); + bool lastRow = (r == pBlock->info.rows - 1); + + if (keyInWin) { + if (r < 5) { + qTrace("group:%" PRId64 ", block:%d win:%d, row:%d ts:%" PRId64 ", riwin:%d riblock:%d", pSupp->groupId, b, + pSupp->curWinIndex, r, key, rowsInWin, rowsInBlock); + } + if (rowsInBlock == 0) { + doKeepNewWindowStartInfo(pRowSup, tsList, r, gid); + } + doKeepTuple(pRowSup, tsList[r], gid); + rowsInBlock++; + rowsInWin++; + } else { + if (rowsInBlock > 0) { + qTrace("group:%" PRId64 ", block:%d win:%d, row:%d ts:%" PRId64 ", riwin:%d riblock:%d, agg", pSupp->groupId, + b, pSupp->curWinIndex, r, key, rowsInWin, rowsInBlock); + anomalyAggregateRows(pOperator, pBlock); + rowsInBlock = 0; + } + if (rowsInWin > 0) { + qTrace("group:%" PRId64 ", block:%d win:%d, row:%d ts:%" PRId64 ", riwin:%d riblock:%d, build result", + pSupp->groupId, b, pSupp->curWinIndex, r, key, rowsInWin, rowsInBlock); + anomalyBuildResult(pOperator); + rowsInWin = 0; + } + if (anomalyFindWindow(pSupp, tsList[r]) == 0) { + qTrace("group:%" PRId64 ", block:%d win:%d, row:%d ts:%" PRId64 ", riwin:%d riblock:%d, new window detect", + pSupp->groupId, b, pSupp->curWinIndex, r, key, rowsInWin, rowsInBlock); + doKeepNewWindowStartInfo(pRowSup, tsList, r, gid); + doKeepTuple(pRowSup, tsList[r], gid); + rowsInBlock = 1; + rowsInWin = 1; + } else { + qTrace("group:%" PRId64 ", block:%d win:%d, row:%d ts:%" PRId64 ", riwin:%d riblock:%d, window not found", + pSupp->groupId, b, pSupp->curWinIndex, r, key, rowsInWin, rowsInBlock); + rowsInBlock = 0; + rowsInWin = 0; + } + } + + if (lastRow && rowsInBlock > 0) { + qTrace("group:%" PRId64 ", block:%d win:%d, row:%d ts:%" PRId64 ", riwin:%d riblock:%d, agg since lastrow", + pSupp->groupId, b, pSupp->curWinIndex, r, key, rowsInWin, rowsInBlock); + anomalyAggregateRows(pOperator, pBlock); + rowsInBlock = 0; + } + } + + if (lastBlock && rowsInWin > 0) { + qTrace("group:%" PRId64 ", block:%d win:%d, riwin:%d riblock:%d, build result since lastblock", pSupp->groupId, b, + pSupp->curWinIndex, rowsInWin, rowsInBlock); + anomalyBuildResult(pOperator); + rowsInWin = 0; + } + } + + code = doFilter(pRes, pOperator->exprSupp.pFilterInfo, NULL); + QUERY_CHECK_CODE(code, lino, _OVER); + +_OVER: + for (int32_t i = 0; i < numOfBlocks; ++i) { + SSDataBlock* pBlock = taosArrayGetP(pSupp->blocks, i); + qDebug("%s, clear block, pBlock:%p pBlock->pDataBlock:%p", __func__, pBlock, pBlock->pDataBlock); + blockDataDestroy(pBlock); + } + + taosArrayClear(pSupp->blocks); + taosArrayClear(pSupp->windows); + pSupp->numOfRows = 0; + pSupp->curWin.ekey = 0; + pSupp->curWin.skey = 0; + pSupp->curWinIndex = 0; +} + +#else + +int32_t createAnomalywindowOperatorInfo(SOperatorInfo* downstream, SPhysiNode* physiNode, SExecTaskInfo* pTaskInfo, + SOperatorInfo** pOptrInfo) { + return TSDB_CODE_OPS_NOT_SUPPORT; +} +void destroyForecastInfo(void* param) {} + +#endif \ No newline at end of file diff --git a/source/libs/executor/src/executil.c b/source/libs/executor/src/executil.c index c74aef3992..4fe45ff72e 100644 --- a/source/libs/executor/src/executil.c +++ b/source/libs/executor/src/executil.c @@ -1794,9 +1794,13 @@ int32_t createExprFromOneNode(SExprInfo* pExp, SNode* pNode, int16_t slotId) { pExp->pExpr->nodeType = QUERY_NODE_FUNCTION; SFunctionNode* pFuncNode = (SFunctionNode*)pNode; - SDataType* pType = &pFuncNode->node.resType; - pExp->base.resSchema = - createResSchema(pType->type, pType->bytes, slotId, pType->scale, pType->precision, pFuncNode->node.aliasName); + SDataType* pType = &pFuncNode->node.resType; + const char* pName = pFuncNode->node.aliasName; + if (pFuncNode->funcType == FUNCTION_TYPE_FORECAST_LOW || pFuncNode->funcType == FUNCTION_TYPE_FORECAST_HIGH || + pFuncNode->funcType == FUNCTION_TYPE_FORECAST_ROWTS) { + pName = pFuncNode->functionName; + } + pExp->base.resSchema = createResSchema(pType->type, pType->bytes, slotId, pType->scale, pType->precision, pName); tExprNode* pExprNode = pExp->pExpr; diff --git a/source/libs/executor/src/forecastoperator.c b/source/libs/executor/src/forecastoperator.c new file mode 100644 index 0000000000..599678106c --- /dev/null +++ b/source/libs/executor/src/forecastoperator.c @@ -0,0 +1,663 @@ +/* + * Copyright (c) 2019 TAOS Data, Inc. + * + * This program is free software: you can use, redistribute, and/or modify + * it under the terms of the GNU Affero General Public License, version 3 + * or later ("AGPL"), as published by the Free Software Foundation. + * + * This program is distributed in the hope that it will be useful, but WITHOUT + * ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or + * FITNESS FOR A PARTICULAR PURPOSE. + * + * You should have received a copy of the GNU Affero General Public License + * along with this program. If not, see . + */ +#include "executorInt.h" +#include "filter.h" +#include "function.h" +#include "functionMgt.h" +#include "operator.h" +#include "querytask.h" +#include "storageapi.h" +#include "tanal.h" +#include "tcommon.h" +#include "tcompare.h" +#include "tdatablock.h" +#include "tfill.h" +#include "ttime.h" + +#ifdef USE_ANAL + +typedef struct { + char algoName[TSDB_ANAL_ALGO_NAME_LEN]; + char algoUrl[TSDB_ANAL_ALGO_URL_LEN]; + char algoOpt[TSDB_ANAL_ALGO_OPTION_LEN]; + int64_t maxTs; + int64_t minTs; + int64_t numOfRows; + uint64_t groupId; + int32_t numOfBlocks; + int32_t optRows; + int16_t resTsSlot; + int16_t resValSlot; + int16_t resLowSlot; + int16_t resHighSlot; + int16_t inputTsSlot; + int16_t inputValSlot; + int8_t inputValType; + int8_t inputPrecision; + SAnalBuf analBuf; +} SForecastSupp; + +typedef struct SForecastOperatorInfo { + SSDataBlock* pRes; + SExprSupp scalarSup; // scalar calculation + SForecastSupp forecastSupp; +} SForecastOperatorInfo; + +static void destroyForecastInfo(void* param); + +static FORCE_INLINE int32_t forecastEnsureBlockCapacity(SSDataBlock* pBlock, int32_t newRowsNum) { + if (pBlock->info.rows < pBlock->info.capacity) { + return TSDB_CODE_SUCCESS; + } + + int32_t code = blockDataEnsureCapacity(pBlock, newRowsNum); + if (code != TSDB_CODE_SUCCESS) { + qError("%s failed at line %d since %s", __func__, __LINE__, tstrerror(code)); + return code; + } + + return TSDB_CODE_SUCCESS; +} + +static int32_t forecastCacheBlock(SForecastSupp* pSupp, SSDataBlock* pBlock) { + int32_t code = TSDB_CODE_SUCCESS; + int32_t lino = 0; + SAnalBuf* pBuf = &pSupp->analBuf; + + qDebug("block:%d, %p rows:%" PRId64, pSupp->numOfBlocks, pBlock, pBlock->info.rows); + pSupp->numOfBlocks++; + + for (int32_t j = 0; j < pBlock->info.rows; ++j) { + SColumnInfoData* pValCol = taosArrayGet(pBlock->pDataBlock, pSupp->inputValSlot); + SColumnInfoData* pTsCol = taosArrayGet(pBlock->pDataBlock, pSupp->inputTsSlot); + if (pTsCol == NULL || pValCol == NULL) break; + + int64_t ts = ((TSKEY*)pTsCol->pData)[j]; + char* val = colDataGetData(pValCol, j); + int16_t valType = pValCol->info.type; + + pSupp->minTs = MIN(pSupp->minTs, ts); + pSupp->maxTs = MAX(pSupp->maxTs, ts); + pSupp->numOfRows++; + + code = taosAnalBufWriteColData(pBuf, 0, TSDB_DATA_TYPE_TIMESTAMP, &ts); + if (TSDB_CODE_SUCCESS != code) return code; + + code = taosAnalBufWriteColData(pBuf, 1, valType, val); + if (TSDB_CODE_SUCCESS != code) return code; + } + + return 0; +} + +static int32_t forecastCloseBuf(SForecastSupp* pSupp) { + SAnalBuf* pBuf = &pSupp->analBuf; + int32_t code = 0; + + for (int32_t i = 0; i < 2; ++i) { + code = taosAnalBufWriteColEnd(pBuf, i); + if (code != 0) return code; + } + + code = taosAnalBufWriteDataEnd(pBuf); + if (code != 0) return code; + + int32_t len = strlen(pSupp->algoOpt); + int64_t every = (pSupp->maxTs - pSupp->minTs) / (pSupp->numOfRows + 1); + int64_t start = pSupp->maxTs + every; + bool hasStart = taosAnalGetOptStr(pSupp->algoOpt, "start", NULL, 0); + if (!hasStart) { + qDebug("forecast start not found from %s, use %" PRId64, pSupp->algoOpt, start); + code = taosAnalBufWriteOptInt(pBuf, "start", start); + if (code != 0) return code; + } + + bool hasEvery = taosAnalGetOptStr(pSupp->algoOpt, "every", NULL, 0); + if (!hasEvery) { + qDebug("forecast every not found from %s, use %" PRId64, pSupp->algoOpt, every); + code = taosAnalBufWriteOptInt(pBuf, "every", every); + if (code != 0) return code; + } + + code = taosAnalBufWriteOptStr(pBuf, "option", pSupp->algoOpt); + if (code != 0) return code; + + code = taosAnalBufClose(pBuf); + return code; +} + +static int32_t forecastAnalysis(SForecastSupp* pSupp, SSDataBlock* pBlock) { + SAnalBuf* pBuf = &pSupp->analBuf; + int32_t resCurRow = pBlock->info.rows; + int8_t tmpI8; + int16_t tmpI16; + int32_t tmpI32; + int64_t tmpI64; + float tmpFloat; + double tmpDouble; + int32_t code = 0; + + SColumnInfoData* pResValCol = taosArrayGet(pBlock->pDataBlock, pSupp->resValSlot); + if (NULL == pResValCol) return TSDB_CODE_OUT_OF_RANGE; + + SColumnInfoData* pResTsCol = (pSupp->resTsSlot != -1 ? taosArrayGet(pBlock->pDataBlock, pSupp->resTsSlot) : NULL); + SColumnInfoData* pResLowCol = (pSupp->resLowSlot != -1 ? taosArrayGet(pBlock->pDataBlock, pSupp->resLowSlot) : NULL); + SColumnInfoData* pResHighCol = + (pSupp->resHighSlot != -1 ? taosArrayGet(pBlock->pDataBlock, pSupp->resHighSlot) : NULL); + + SJson* pJson = taosAnalSendReqRetJson(pSupp->algoUrl, ANAL_HTTP_TYPE_POST, pBuf); + if (pJson == NULL) return terrno; + + int32_t rows = 0; + tjsonGetInt32ValueFromDouble(pJson, "rows", rows, code); + if (code < 0) goto _OVER; + if (rows <= 0) goto _OVER; + + SJson* res = tjsonGetObjectItem(pJson, "res"); + if (res == NULL) goto _OVER; + int32_t ressize = tjsonGetArraySize(res); + bool returnConf = (pSupp->resHighSlot != -1 || pSupp->resLowSlot != -1); + if (returnConf) { + if (ressize != 4) goto _OVER; + } else if (ressize != 2) { + goto _OVER; + } + + if (pResTsCol != NULL) { + resCurRow = pBlock->info.rows; + SJson* tsJsonArray = tjsonGetArrayItem(res, 0); + if (tsJsonArray == NULL) goto _OVER; + int32_t tsSize = tjsonGetArraySize(tsJsonArray); + if (tsSize != rows) goto _OVER; + for (int32_t i = 0; i < tsSize; ++i) { + SJson* tsJson = tjsonGetArrayItem(tsJsonArray, i); + tjsonGetObjectValueBigInt(tsJson, &tmpI64); + colDataSetInt64(pResTsCol, resCurRow, &tmpI64); + resCurRow++; + } + } + + if (pResLowCol != NULL) { + resCurRow = pBlock->info.rows; + SJson* lowJsonArray = tjsonGetArrayItem(res, 2); + if (lowJsonArray == NULL) goto _OVER; + int32_t lowSize = tjsonGetArraySize(lowJsonArray); + if (lowSize != rows) goto _OVER; + for (int32_t i = 0; i < lowSize; ++i) { + SJson* lowJson = tjsonGetArrayItem(lowJsonArray, i); + tjsonGetObjectValueDouble(lowJson, &tmpDouble); + tmpFloat = (float)tmpDouble; + colDataSetFloat(pResLowCol, resCurRow, &tmpFloat); + resCurRow++; + } + } + + if (pResHighCol != NULL) { + resCurRow = pBlock->info.rows; + SJson* highJsonArray = tjsonGetArrayItem(res, 3); + if (highJsonArray == NULL) goto _OVER; + int32_t highSize = tjsonGetArraySize(highJsonArray); + if (highSize != rows) goto _OVER; + for (int32_t i = 0; i < highSize; ++i) { + SJson* highJson = tjsonGetArrayItem(highJsonArray, i); + tjsonGetObjectValueDouble(highJson, &tmpDouble); + tmpFloat = (float)tmpDouble; + colDataSetFloat(pResHighCol, resCurRow, &tmpFloat); + resCurRow++; + } + } + + resCurRow = pBlock->info.rows; + SJson* valJsonArray = tjsonGetArrayItem(res, 1); + if (valJsonArray == NULL) goto _OVER; + int32_t valSize = tjsonGetArraySize(valJsonArray); + if (valSize != rows) goto _OVER; + for (int32_t i = 0; i < valSize; ++i) { + SJson* valJson = tjsonGetArrayItem(valJsonArray, i); + tjsonGetObjectValueDouble(valJson, &tmpDouble); + + switch (pSupp->inputValType) { + case TSDB_DATA_TYPE_BOOL: + case TSDB_DATA_TYPE_UTINYINT: + case TSDB_DATA_TYPE_TINYINT: { + tmpI8 = (int8_t)tmpDouble; + colDataSetInt8(pResValCol, resCurRow, &tmpI8); + break; + } + case TSDB_DATA_TYPE_USMALLINT: + case TSDB_DATA_TYPE_SMALLINT: { + tmpI16 = (int16_t)tmpDouble; + colDataSetInt16(pResValCol, resCurRow, &tmpI16); + break; + } + case TSDB_DATA_TYPE_INT: + case TSDB_DATA_TYPE_UINT: { + tmpI32 = (int32_t)tmpDouble; + colDataSetInt32(pResValCol, resCurRow, &tmpI32); + break; + } + case TSDB_DATA_TYPE_TIMESTAMP: + case TSDB_DATA_TYPE_UBIGINT: + case TSDB_DATA_TYPE_BIGINT: { + tmpI64 = (int64_t)tmpDouble; + colDataSetInt64(pResValCol, resCurRow, &tmpI64); + break; + } + case TSDB_DATA_TYPE_FLOAT: { + tmpFloat = (float)tmpDouble; + colDataSetFloat(pResValCol, resCurRow, &tmpFloat); + break; + } + case TSDB_DATA_TYPE_DOUBLE: { + colDataSetDouble(pResValCol, resCurRow, &tmpDouble); + break; + } + default: + code = TSDB_CODE_FUNC_FUNTION_PARA_TYPE; + goto _OVER; + } + resCurRow++; + } + + // for (int32_t i = rows; i < pSupp->optRows; ++i) { + // colDataSetNNULL(pResValCol, rows, (pSupp->optRows - rows)); + // if (pResTsCol != NULL) { + // colDataSetNNULL(pResTsCol, rows, (pSupp->optRows - rows)); + // } + // if (pResLowCol != NULL) { + // colDataSetNNULL(pResLowCol, rows, (pSupp->optRows - rows)); + // } + // if (pResHighCol != NULL) { + // colDataSetNNULL(pResHighCol, rows, (pSupp->optRows - rows)); + // } + // } + + // if (rows == pSupp->optRows) { + // pResValCol->hasNull = false; + // } + + pBlock->info.rows += rows; + + if (pJson != NULL) tjsonDelete(pJson); + return 0; + +_OVER: + if (pJson != NULL) tjsonDelete(pJson); + if (code == 0) { + code = TSDB_CODE_INVALID_JSON_FORMAT; + } + qError("failed to perform forecast finalize since %s", tstrerror(code)); + return TSDB_CODE_INVALID_JSON_FORMAT; +} + +static int32_t forecastAggregateBlocks(SForecastSupp* pSupp, SSDataBlock* pResBlock) { + int32_t code = TSDB_CODE_SUCCESS; + int32_t lino = 0; + SAnalBuf* pBuf = &pSupp->analBuf; + + code = forecastCloseBuf(pSupp); + QUERY_CHECK_CODE(code, lino, _end); + + code = forecastEnsureBlockCapacity(pResBlock, 1); + QUERY_CHECK_CODE(code, lino, _end); + + code = forecastAnalysis(pSupp, pResBlock); + QUERY_CHECK_CODE(code, lino, _end); + + uInfo("block:%d, forecast finalize", pSupp->numOfBlocks); + +_end: + pSupp->numOfBlocks = 0; + taosAnalBufDestroy(&pSupp->analBuf); + return code; +} + +static int32_t forecastNext(SOperatorInfo* pOperator, SSDataBlock** ppRes) { + int32_t code = TSDB_CODE_SUCCESS; + int32_t lino = 0; + SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; + SForecastOperatorInfo* pInfo = pOperator->info; + SSDataBlock* pResBlock = pInfo->pRes; + SForecastSupp* pSupp = &pInfo->forecastSupp; + SAnalBuf* pBuf = &pSupp->analBuf; + int64_t st = taosGetTimestampUs(); + int32_t numOfBlocks = pSupp->numOfBlocks; + + blockDataCleanup(pResBlock); + + while (1) { + SSDataBlock* pBlock = getNextBlockFromDownstream(pOperator, 0); + if (pBlock == NULL) { + break; + } + + if (pSupp->groupId == 0 || pSupp->groupId == pBlock->info.id.groupId) { + pSupp->groupId = pBlock->info.id.groupId; + numOfBlocks++; + qDebug("group:%" PRId64 ", blocks:%d, cache block rows:%" PRId64, pSupp->groupId, numOfBlocks, pBlock->info.rows); + code = forecastCacheBlock(pSupp, pBlock); + QUERY_CHECK_CODE(code, lino, _end); + } else { + qDebug("group:%" PRId64 ", read finish for new group coming, blocks:%d", pSupp->groupId, numOfBlocks); + forecastAggregateBlocks(pSupp, pResBlock); + pSupp->groupId = pBlock->info.id.groupId; + numOfBlocks = 1; + qDebug("group:%" PRId64 ", new group, cache block rows:%" PRId64, pSupp->groupId, pBlock->info.rows); + code = forecastCacheBlock(pSupp, pBlock); + QUERY_CHECK_CODE(code, lino, _end); + } + + if (pResBlock->info.rows > 0) { + (*ppRes) = pResBlock; + qDebug("group:%" PRId64 ", return to upstream, blocks:%d", pResBlock->info.id.groupId, numOfBlocks); + return code; + } + } + + if (numOfBlocks > 0) { + qDebug("group:%" PRId64 ", read finish, blocks:%d", pSupp->groupId, numOfBlocks); + forecastAggregateBlocks(pSupp, pResBlock); + } + + int64_t cost = taosGetTimestampUs() - st; + qDebug("all groups finished, cost:%" PRId64 "us", cost); + +_end: + if (code != TSDB_CODE_SUCCESS) { + qError("%s failed at line %d since %s", __func__, lino, tstrerror(code)); + pTaskInfo->code = code; + T_LONG_JMP(pTaskInfo->env, code); + } + (*ppRes) = (pResBlock->info.rows == 0) ? NULL : pResBlock; + return code; +} + +static int32_t forecastParseOutput(SForecastSupp* pSupp, SExprSupp* pExprSup) { + pSupp->resLowSlot = -1; + pSupp->resHighSlot = -1; + pSupp->resTsSlot = -1; + pSupp->resValSlot = -1; + + for (int32_t j = 0; j < pExprSup->numOfExprs; ++j) { + SExprInfo* pExprInfo = &pExprSup->pExprInfo[j]; + int32_t dstSlot = pExprInfo->base.resSchema.slotId; + if (pExprInfo->pExpr->_function.functionType == FUNCTION_TYPE_FORECAST) { + pSupp->resValSlot = dstSlot; + } else if (pExprInfo->pExpr->_function.functionType == FUNCTION_TYPE_FORECAST_ROWTS) { + pSupp->resTsSlot = dstSlot; + } else if (pExprInfo->pExpr->_function.functionType == FUNCTION_TYPE_FORECAST_LOW) { + pSupp->resLowSlot = dstSlot; + } else if (pExprInfo->pExpr->_function.functionType == FUNCTION_TYPE_FORECAST_HIGH) { + pSupp->resHighSlot = dstSlot; + } else { + } + } + + return 0; +} + +static int32_t forecastParseInput(SForecastSupp* pSupp, SNodeList* pFuncs) { + SNode* pNode = NULL; + + pSupp->inputTsSlot = -1; + pSupp->inputValSlot = -1; + pSupp->inputValType = -1; + pSupp->inputPrecision = -1; + + FOREACH(pNode, pFuncs) { + if ((nodeType(pNode) == QUERY_NODE_TARGET) && (nodeType(((STargetNode*)pNode)->pExpr) == QUERY_NODE_FUNCTION)) { + SFunctionNode* pFunc = (SFunctionNode*)((STargetNode*)pNode)->pExpr; + int32_t numOfParam = LIST_LENGTH(pFunc->pParameterList); + + if (pFunc->funcType == FUNCTION_TYPE_FORECAST) { + if (numOfParam == 3) { + SNode* p1 = nodesListGetNode(pFunc->pParameterList, 0); + SNode* p2 = nodesListGetNode(pFunc->pParameterList, 1); + SNode* p3 = nodesListGetNode(pFunc->pParameterList, 2); + if (p1 == NULL || p2 == NULL || p3 == NULL) return TSDB_CODE_PLAN_INTERNAL_ERROR; + if (p1->type != QUERY_NODE_COLUMN) return TSDB_CODE_PLAN_INTERNAL_ERROR; + if (p2->type != QUERY_NODE_VALUE) return TSDB_CODE_PLAN_INTERNAL_ERROR; + if (p3->type != QUERY_NODE_COLUMN) return TSDB_CODE_PLAN_INTERNAL_ERROR; + SColumnNode* pValNode = (SColumnNode*)p1; + SValueNode* pOptNode = (SValueNode*)p2; + SColumnNode* pTsNode = (SColumnNode*)p3; + pSupp->inputTsSlot = pTsNode->slotId; + pSupp->inputPrecision = pTsNode->node.resType.precision; + pSupp->inputValSlot = pValNode->slotId; + pSupp->inputValType = pValNode->node.resType.type; + tstrncpy(pSupp->algoOpt, pOptNode->literal, sizeof(pSupp->algoOpt)); + } else if (numOfParam == 2) { + SNode* p1 = nodesListGetNode(pFunc->pParameterList, 0); + SNode* p2 = nodesListGetNode(pFunc->pParameterList, 1); + if (p1 == NULL || p2 == NULL) return TSDB_CODE_PLAN_INTERNAL_ERROR; + if (p1->type != QUERY_NODE_COLUMN) return TSDB_CODE_PLAN_INTERNAL_ERROR; + if (p2->type != QUERY_NODE_COLUMN) return TSDB_CODE_PLAN_INTERNAL_ERROR; + SColumnNode* pValNode = (SColumnNode*)p1; + SColumnNode* pTsNode = (SColumnNode*)p2; + pSupp->inputTsSlot = pTsNode->slotId; + pSupp->inputPrecision = pTsNode->node.resType.precision; + pSupp->inputValSlot = pValNode->slotId; + pSupp->inputValType = pValNode->node.resType.type; + tstrncpy(pSupp->algoOpt, "algo=arima", TSDB_ANAL_ALGO_OPTION_LEN); + } else { + return TSDB_CODE_PLAN_INTERNAL_ERROR; + } + } + } + } + + return 0; +} + +static int32_t forecastParseAlgo(SForecastSupp* pSupp) { + pSupp->maxTs = 0; + pSupp->minTs = INT64_MAX; + pSupp->numOfRows = 0; + + if (!taosAnalGetOptStr(pSupp->algoOpt, "algo", pSupp->algoName, sizeof(pSupp->algoName))) { + qError("failed to get forecast algorithm name from %s", pSupp->algoOpt); + return TSDB_CODE_ANAL_ALGO_NOT_FOUND; + } + + if (taosAnalGetAlgoUrl(pSupp->algoName, ANAL_ALGO_TYPE_FORECAST, pSupp->algoUrl, sizeof(pSupp->algoUrl)) != 0) { + qError("failed to get forecast algorithm url from %s", pSupp->algoName); + return TSDB_CODE_ANAL_ALGO_NOT_LOAD; + } + + return 0; +} + +static int32_t forecastCreateBuf(SForecastSupp* pSupp) { + SAnalBuf* pBuf = &pSupp->analBuf; + int64_t ts = 0; // taosGetTimestampMs(); + + pBuf->bufType = ANAL_BUF_TYPE_JSON_COL; + snprintf(pBuf->fileName, sizeof(pBuf->fileName), "%s/tdengine-forecast-%" PRId64, tsTempDir, ts); + int32_t code = tsosAnalBufOpen(pBuf, 2); + if (code != 0) goto _OVER; + + code = taosAnalBufWriteOptStr(pBuf, "algo", pSupp->algoName); + if (code != 0) goto _OVER; + + bool returnConf = (pSupp->resHighSlot == -1 || pSupp->resLowSlot == -1); + code = taosAnalBufWriteOptStr(pBuf, "return_conf", returnConf ? "true" : "false"); + if (code != 0) goto _OVER; + + bool hasAlpha = taosAnalGetOptStr(pSupp->algoOpt, "alpha", NULL, 0); + if (!hasAlpha) { + qDebug("forecast alpha not found from %s, use default:%f", pSupp->algoOpt, ANAL_FORECAST_DEFAULT_ALPHA); + code = taosAnalBufWriteOptFloat(pBuf, "alpha", ANAL_FORECAST_DEFAULT_ALPHA); + if (code != 0) goto _OVER; + } + + char tmpOpt[32] = {0}; + bool hasParam = taosAnalGetOptStr(pSupp->algoOpt, "param", tmpOpt, sizeof(tmpOpt)); + if (!hasParam) { + qDebug("forecast param not found from %s, use default:%s", pSupp->algoOpt, ANAL_FORECAST_DEFAULT_PARAM); + code = taosAnalBufWriteOptStr(pBuf, "param", ANAL_FORECAST_DEFAULT_PARAM); + if (code != 0) goto _OVER; + } + + bool hasPeriod = taosAnalGetOptInt(pSupp->algoOpt, "period", NULL); + if (!hasPeriod) { + qDebug("forecast period not found from %s, use default:%d", pSupp->algoOpt, ANAL_FORECAST_DEFAULT_PERIOD); + code = taosAnalBufWriteOptInt(pBuf, "period", ANAL_FORECAST_DEFAULT_PERIOD); + if (code != 0) goto _OVER; + } + + bool hasRows = taosAnalGetOptInt(pSupp->algoOpt, "rows", &pSupp->optRows); + if (!hasRows) { + pSupp->optRows = ANAL_FORECAST_DEFAULT_ROWS; + qDebug("forecast rows not found from %s, use default:%d", pSupp->algoOpt, pSupp->optRows); + code = taosAnalBufWriteOptInt(pBuf, "forecast_rows", pSupp->optRows); + if (code != 0) goto _OVER; + } + + const char* prec = TSDB_TIME_PRECISION_MILLI_STR; + if (pSupp->inputPrecision == TSDB_TIME_PRECISION_MICRO) prec = TSDB_TIME_PRECISION_MICRO_STR; + if (pSupp->inputPrecision == TSDB_TIME_PRECISION_NANO) prec = TSDB_TIME_PRECISION_NANO_STR; + code = taosAnalBufWriteOptStr(pBuf, "prec", prec); + if (code != 0) goto _OVER; + + if (returnConf) { + bool hasConf = taosAnalGetOptStr(pSupp->algoOpt, "conf", NULL, 0); + if (!hasConf) { + qDebug("forecast conf not found from %s, use default:%d", pSupp->algoOpt, ANAL_FORECAST_DEFAULT_CONF); + code = taosAnalBufWriteOptInt(pBuf, "conf", ANAL_FORECAST_DEFAULT_CONF); + if (code != 0) goto _OVER; + } + } + + code = taosAnalBufWriteColMeta(pBuf, 0, TSDB_DATA_TYPE_TIMESTAMP, "ts"); + if (code != 0) goto _OVER; + + code = taosAnalBufWriteColMeta(pBuf, 1, pSupp->inputValType, "val"); + if (code != 0) goto _OVER; + + code = taosAnalBufWriteDataBegin(pBuf); + if (code != 0) goto _OVER; + + for (int32_t i = 0; i < 2; ++i) { + code = taosAnalBufWriteColBegin(pBuf, i); + if (code != 0) goto _OVER; + } + +_OVER: + if (code != 0) { + taosAnalBufClose(pBuf); + taosAnalBufDestroy(pBuf); + } + return code; +} + +int32_t createForecastOperatorInfo(SOperatorInfo* downstream, SPhysiNode* pPhyNode, SExecTaskInfo* pTaskInfo, + SOperatorInfo** pOptrInfo) { + QRY_PARAM_CHECK(pOptrInfo); + + int32_t code = 0; + int32_t lino = 0; + SForecastOperatorInfo* pInfo = taosMemoryCalloc(1, sizeof(SForecastOperatorInfo)); + SOperatorInfo* pOperator = taosMemoryCalloc(1, sizeof(SOperatorInfo)); + if (pOperator == NULL || pInfo == NULL) { + code = terrno; + goto _error; + } + + SForecastSupp* pSupp = &pInfo->forecastSupp; + SForecastFuncPhysiNode* pForecastPhyNode = (SForecastFuncPhysiNode*)pPhyNode; + SExprSupp* pExprSup = &pOperator->exprSupp; + int32_t numOfExprs = 0; + SExprInfo* pExprInfo = NULL; + + code = createExprInfo(pForecastPhyNode->pFuncs, NULL, &pExprInfo, &numOfExprs); + QUERY_CHECK_CODE(code, lino, _error); + + code = initExprSupp(pExprSup, pExprInfo, numOfExprs, &pTaskInfo->storageAPI.functionStore); + QUERY_CHECK_CODE(code, lino, _error); + + if (pForecastPhyNode->pExprs != NULL) { + int32_t num = 0; + SExprInfo* pScalarExprInfo = NULL; + code = createExprInfo(pForecastPhyNode->pExprs, NULL, &pScalarExprInfo, &num); + QUERY_CHECK_CODE(code, lino, _error); + + code = initExprSupp(&pInfo->scalarSup, pScalarExprInfo, num, &pTaskInfo->storageAPI.functionStore); + QUERY_CHECK_CODE(code, lino, _error); + } + + code = filterInitFromNode((SNode*)pForecastPhyNode->node.pConditions, &pOperator->exprSupp.pFilterInfo, 0); + QUERY_CHECK_CODE(code, lino, _error); + + code = forecastParseInput(pSupp, pForecastPhyNode->pFuncs); + QUERY_CHECK_CODE(code, lino, _error); + + code = forecastParseOutput(pSupp, pExprSup); + QUERY_CHECK_CODE(code, lino, _error); + + code = forecastParseAlgo(pSupp); + QUERY_CHECK_CODE(code, lino, _error); + + code = forecastCreateBuf(pSupp); + QUERY_CHECK_CODE(code, lino, _error); + + initResultSizeInfo(&pOperator->resultInfo, 4096); + + pInfo->pRes = createDataBlockFromDescNode(pPhyNode->pOutputDataBlockDesc); + QUERY_CHECK_NULL(pInfo->pRes, code, lino, _error, terrno); + + setOperatorInfo(pOperator, "ForecastOperator", QUERY_NODE_PHYSICAL_PLAN_FORECAST_FUNC, false, OP_NOT_OPENED, pInfo, + pTaskInfo); + pOperator->fpSet = createOperatorFpSet(optrDummyOpenFn, forecastNext, NULL, destroyForecastInfo, optrDefaultBufFn, + NULL, optrDefaultGetNextExtFn, NULL); + + code = blockDataEnsureCapacity(pInfo->pRes, pOperator->resultInfo.capacity); + QUERY_CHECK_CODE(code, lino, _error); + + code = appendDownstream(pOperator, &downstream, 1); + QUERY_CHECK_CODE(code, lino, _error); + + *pOptrInfo = pOperator; + + qDebug("forecast env is initialized, option:%s", pSupp->algoOpt); + return TSDB_CODE_SUCCESS; + +_error: + if (code != TSDB_CODE_SUCCESS) { + qError("%s failed at line %d since %s", __func__, lino, tstrerror(code)); + } + if (pInfo != NULL) destroyForecastInfo(pInfo); + destroyOperatorAndDownstreams(pOperator, &downstream, 1); + pTaskInfo->code = code; + return code; +} + +static void destroyForecastInfo(void* param) { + SForecastOperatorInfo* pInfo = (SForecastOperatorInfo*)param; + + blockDataDestroy(pInfo->pRes); + pInfo->pRes = NULL; + cleanupExprSupp(&pInfo->scalarSup); + taosAnalBufDestroy(&pInfo->forecastSupp.analBuf); + taosMemoryFreeClear(param); +} + +#else + +int32_t createForecastOperatorInfo(SOperatorInfo* downstream, SPhysiNode* pPhyNode, SExecTaskInfo* pTaskInfo, + SOperatorInfo** pOptrInfo) { + return TSDB_CODE_OPS_NOT_SUPPORT; +} + +#endif diff --git a/source/libs/executor/src/operator.c b/source/libs/executor/src/operator.c index 8daf4695db..7914f9f320 100644 --- a/source/libs/executor/src/operator.c +++ b/source/libs/executor/src/operator.c @@ -619,6 +619,8 @@ int32_t createOperator(SPhysiNode* pPhyNode, SExecTaskInfo* pTaskInfo, SReadHand code = createIndefinitOutputOperatorInfo(ops[0], pPhyNode, pTaskInfo, &pOptr); } else if (QUERY_NODE_PHYSICAL_PLAN_INTERP_FUNC == type) { code = createTimeSliceOperatorInfo(ops[0], pPhyNode, pTaskInfo, &pOptr); + } else if (QUERY_NODE_PHYSICAL_PLAN_FORECAST_FUNC == type) { + code = createForecastOperatorInfo(ops[0], pPhyNode, pTaskInfo, &pOptr); } else if (QUERY_NODE_PHYSICAL_PLAN_MERGE_EVENT == type) { code = createEventwindowOperatorInfo(ops[0], pPhyNode, pTaskInfo, &pOptr); } else if (QUERY_NODE_PHYSICAL_PLAN_GROUP_CACHE == type) { @@ -629,6 +631,8 @@ int32_t createOperator(SPhysiNode* pPhyNode, SExecTaskInfo* pTaskInfo, SReadHand code = createStreamCountAggOperatorInfo(ops[0], pPhyNode, pTaskInfo, pHandle, &pOptr); } else if (QUERY_NODE_PHYSICAL_PLAN_MERGE_COUNT == type) { code = createCountwindowOperatorInfo(ops[0], pPhyNode, pTaskInfo, &pOptr); + } else if (QUERY_NODE_PHYSICAL_PLAN_MERGE_ANOMALY == type) { + code = createAnomalywindowOperatorInfo(ops[0], pPhyNode, pTaskInfo, &pOptr); } else { code = TSDB_CODE_INVALID_PARA; pTaskInfo->code = code; From 560f2d6f24dba16018847daab683cd184cf8c3bc Mon Sep 17 00:00:00 2001 From: Hongze Cheng Date: Wed, 9 Oct 2024 18:05:46 +0800 Subject: [PATCH 39/51] check return code --- source/libs/tdb/src/db/tdbPage.c | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/source/libs/tdb/src/db/tdbPage.c b/source/libs/tdb/src/db/tdbPage.c index 6bc0c711ae..49a15070a6 100644 --- a/source/libs/tdb/src/db/tdbPage.c +++ b/source/libs/tdb/src/db/tdbPage.c @@ -102,7 +102,9 @@ void tdbPageDestroy(SPage *pPage, void (*xFree)(void *arg, void *ptr), void *arg tdbOsFree(pPage->apOvfl[iOvfl]); } - TDB_DESTROY_PAGE_LOCK(pPage); + if (TDB_DESTROY_PAGE_LOCK(pPage) != 0) { + tdbError("tdb/page-destroy: destroy page lock failed."); + } ptr = pPage->pData; xFree(arg, ptr); From a68da3c91c0077348c27ff1eeeeee94a59a67ed0 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 9 Oct 2024 21:52:20 +0800 Subject: [PATCH 40/51] fix(stream): fix memory leak. --- source/dnode/mnode/impl/src/mndStreamTransAct.c | 1 + 1 file changed, 1 insertion(+) diff --git a/source/dnode/mnode/impl/src/mndStreamTransAct.c b/source/dnode/mnode/impl/src/mndStreamTransAct.c index 3ecd192222..4e0bf97587 100644 --- a/source/dnode/mnode/impl/src/mndStreamTransAct.c +++ b/source/dnode/mnode/impl/src/mndStreamTransAct.c @@ -234,6 +234,7 @@ static int32_t doSetUpdateTaskAction(SMnode *pMnode, STrans *pTrans, SStreamTask code = extractNodeEpset(pMnode, &epset, &hasEpset, pTask->id.taskId, pTask->info.nodeId); if (code != TSDB_CODE_SUCCESS || !hasEpset) { mError("failed to extract epset during create update epset, code:%s", tstrerror(code)); + taosMemoryFree(pBuf); return code; } From a2f17c56da1d84d4dede3cb73303a912236a7665 Mon Sep 17 00:00:00 2001 From: Shengliang Guan Date: Thu, 10 Oct 2024 09:53:48 +0800 Subject: [PATCH 41/51] fix: ci test cases --- tests/develop-test/2-query/table_count_scan.py | 16 ++++++++-------- tests/script/tsim/query/sys_tbname.sim | 2 +- tests/script/tsim/query/tableCount.sim | 6 +++--- tests/system-test/0-others/information_schema.py | 4 ++-- 4 files changed, 14 insertions(+), 14 deletions(-) diff --git a/tests/develop-test/2-query/table_count_scan.py b/tests/develop-test/2-query/table_count_scan.py index 38e35a175e..b2b48c1f0b 100644 --- a/tests/develop-test/2-query/table_count_scan.py +++ b/tests/develop-test/2-query/table_count_scan.py @@ -65,7 +65,7 @@ class TDTestCase: tdSql.query('select count(*),db_name, stable_name from information_schema.ins_tables group by db_name, stable_name;') tdSql.checkRows(3) - tdSql.checkData(0, 0, 32) + tdSql.checkData(0, 0, 34) tdSql.checkData(0, 1, 'information_schema') tdSql.checkData(0, 2, None) tdSql.checkData(1, 0, 3) @@ -77,7 +77,7 @@ class TDTestCase: tdSql.query('select count(1) v,db_name, stable_name from information_schema.ins_tables group by db_name, stable_name order by v desc;') tdSql.checkRows(3) - tdSql.checkData(0, 0, 32) + tdSql.checkData(0, 0, 34) tdSql.checkData(0, 1, 'information_schema') tdSql.checkData(0, 2, None) tdSql.checkData(1, 0, 5) @@ -93,7 +93,7 @@ class TDTestCase: tdSql.checkData(1, 1, 'performance_schema') tdSql.checkData(0, 0, 3) tdSql.checkData(0, 1, 'tbl_count') - tdSql.checkData(2, 0, 32) + tdSql.checkData(2, 0, 34) tdSql.checkData(2, 1, 'information_schema') tdSql.query("select count(*) from information_schema.ins_tables where db_name='tbl_count'") @@ -106,7 +106,7 @@ class TDTestCase: tdSql.query('select count(*) from information_schema.ins_tables') tdSql.checkRows(1) - tdSql.checkData(0, 0, 40) + tdSql.checkData(0, 0, 42) tdSql.execute('create table stba (ts timestamp, c1 bool, c2 tinyint, c3 smallint, c4 int, c5 bigint, c6 float, c7 double, c8 binary(10), c9 nchar(10), c10 tinyint unsigned, c11 smallint unsigned, c12 int unsigned, c13 bigint unsigned) TAGS(t1 int, t2 binary(10), t3 double);') @@ -189,7 +189,7 @@ class TDTestCase: tdSql.checkData(2, 0, 5) tdSql.checkData(2, 1, 'performance_schema') tdSql.checkData(2, 2, None) - tdSql.checkData(3, 0, 32) + tdSql.checkData(3, 0, 34) tdSql.checkData(3, 1, 'information_schema') tdSql.checkData(3, 2, None) @@ -204,7 +204,7 @@ class TDTestCase: tdSql.checkData(2, 0, 5) tdSql.checkData(2, 1, 'performance_schema') tdSql.checkData(2, 2, None) - tdSql.checkData(3, 0, 32) + tdSql.checkData(3, 0, 34) tdSql.checkData(3, 1, 'information_schema') tdSql.checkData(3, 2, None) @@ -215,7 +215,7 @@ class TDTestCase: tdSql.checkData(0, 1, 'tbl_count') tdSql.checkData(1, 0, 5) tdSql.checkData(1, 1, 'performance_schema') - tdSql.checkData(2, 0, 32) + tdSql.checkData(2, 0, 34) tdSql.checkData(2, 1, 'information_schema') tdSql.query("select count(*) from information_schema.ins_tables where db_name='tbl_count'") @@ -228,7 +228,7 @@ class TDTestCase: tdSql.query('select count(*) from information_schema.ins_tables') tdSql.checkRows(1) - tdSql.checkData(0, 0, 41) + tdSql.checkData(0, 0, 43) tdSql.execute('drop database tbl_count') diff --git a/tests/script/tsim/query/sys_tbname.sim b/tests/script/tsim/query/sys_tbname.sim index dabe4fcdde..9736893428 100644 --- a/tests/script/tsim/query/sys_tbname.sim +++ b/tests/script/tsim/query/sys_tbname.sim @@ -58,7 +58,7 @@ endi sql select tbname from information_schema.ins_tables; print $rows $data00 -if $rows != 41 then +if $rows != 43 then return -1 endi if $data00 != @ins_tables@ then diff --git a/tests/script/tsim/query/tableCount.sim b/tests/script/tsim/query/tableCount.sim index 5a3dd0714f..87f72eb3b6 100644 --- a/tests/script/tsim/query/tableCount.sim +++ b/tests/script/tsim/query/tableCount.sim @@ -53,7 +53,7 @@ sql select stable_name,count(table_name) from information_schema.ins_tables grou if $rows != 3 then return -1 endi -if $data01 != 38 then +if $data01 != 40 then return -1 endi if $data11 != 10 then @@ -72,7 +72,7 @@ endi if $data11 != 5 then return -1 endi -if $data21 != 32 then +if $data21 != 34 then return -1 endi if $data31 != 5 then @@ -97,7 +97,7 @@ endi if $data42 != 3 then return -1 endi -if $data52 != 32 then +if $data52 != 34 then return -1 endi if $data62 != 5 then diff --git a/tests/system-test/0-others/information_schema.py b/tests/system-test/0-others/information_schema.py index f59410b552..01e416bb26 100644 --- a/tests/system-test/0-others/information_schema.py +++ b/tests/system-test/0-others/information_schema.py @@ -61,7 +61,7 @@ class TDTestCase: self.ins_list = ['ins_dnodes','ins_mnodes','ins_qnodes','ins_snodes','ins_cluster','ins_databases','ins_functions',\ 'ins_indexes','ins_stables','ins_tables','ins_tags','ins_columns','ins_users','ins_grants','ins_vgroups','ins_configs','ins_dnode_variables',\ 'ins_topics','ins_subscriptions','ins_streams','ins_stream_tasks','ins_vnodes','ins_user_privileges','ins_views', - 'ins_compacts', 'ins_compact_details', 'ins_grants_full','ins_grants_logs', 'ins_machines', 'ins_arbgroups', 'ins_tsmas', "ins_encryptions"] + 'ins_compacts', 'ins_compact_details', 'ins_grants_full','ins_grants_logs', 'ins_machines', 'ins_arbgroups', 'ins_tsmas', "ins_encryptions", "ins_anodes", "ins_anodes_full"] self.perf_list = ['perf_connections','perf_queries','perf_consumers','perf_trans','perf_apps'] def insert_data(self,column_dict,tbname,row_num): insert_sql = self.setsql.set_insertsql(column_dict,tbname,self.binary_str,self.nchar_str) @@ -222,7 +222,7 @@ class TDTestCase: tdSql.query("select * from information_schema.ins_columns where db_name ='information_schema'") tdLog.info(len(tdSql.queryResult)) - tdSql.checkEqual(True, len(tdSql.queryResult) in range(272, 273)) + tdSql.checkEqual(True, len(tdSql.queryResult) in range(280, 281)) tdSql.query("select * from information_schema.ins_columns where db_name ='performance_schema'") tdSql.checkEqual(56, len(tdSql.queryResult)) From 9f72bd7e70f599db1477c7553f3fc629f11c2ee9 Mon Sep 17 00:00:00 2001 From: 54liuyao <54liuyao@163.com> Date: Thu, 10 Oct 2024 10:42:04 +0800 Subject: [PATCH 42/51] add limit on number fo history window --- docs/zh/14-reference/03-taos-sql/14-stream.md | 2 +- source/libs/executor/src/streamtimewindowoperator.c | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/zh/14-reference/03-taos-sql/14-stream.md b/docs/zh/14-reference/03-taos-sql/14-stream.md index c0d14f0455..d995c2a09b 100644 --- a/docs/zh/14-reference/03-taos-sql/14-stream.md +++ b/docs/zh/14-reference/03-taos-sql/14-stream.md @@ -99,7 +99,7 @@ PARTITION 子句中,为 tbname 定义了一个别名 tname, 在PARTITION 子 ## 流式计算读取历史数据 -正常情况下,流式计算不会处理创建前已经写入源表中的数据,若要处理已经写入的数据,可以在创建流时设置 fill_history 1 选项,这样创建的流式计算会自动处理创建前、创建中、创建后写入的数据。例如: +正常情况下,流式计算不会处理创建前已经写入源表中的数据,若要处理已经写入的数据,可以在创建流时设置 fill_history 1 选项,这样创建的流式计算会自动处理创建前、创建中、创建后写入的数据。流计算处理历史数据的最大窗口数是2000万,超过限制会报错。例如: ```sql create stream if not exists s1 fill_history 1 into st1 as select count(*) from t1 interval(10s) diff --git a/source/libs/executor/src/streamtimewindowoperator.c b/source/libs/executor/src/streamtimewindowoperator.c index e8322d6911..6fc50bb860 100644 --- a/source/libs/executor/src/streamtimewindowoperator.c +++ b/source/libs/executor/src/streamtimewindowoperator.c @@ -49,7 +49,7 @@ #define STREAM_SESSION_OP_CHECKPOINT_NAME "StreamSessionOperator_Checkpoint" #define STREAM_STATE_OP_CHECKPOINT_NAME "StreamStateOperator_Checkpoint" -#define MAX_STREAM_HISTORY_RESULT 100000000 +#define MAX_STREAM_HISTORY_RESULT 20000000 typedef struct SStateWindowInfo { SResultWindowInfo winInfo; From bdb7e9558211e1081fbcb813c07d11f6b1c70388 Mon Sep 17 00:00:00 2001 From: Shengliang Guan Date: Thu, 10 Oct 2024 11:41:52 +0800 Subject: [PATCH 43/51] fix: unitest error --- source/libs/planner/src/planSpliter.c | 17 ++++++++++++++++- 1 file changed, 16 insertions(+), 1 deletion(-) diff --git a/source/libs/planner/src/planSpliter.c b/source/libs/planner/src/planSpliter.c index 706394507a..755dd8739b 100644 --- a/source/libs/planner/src/planSpliter.c +++ b/source/libs/planner/src/planSpliter.c @@ -939,6 +939,18 @@ static int32_t stbSplSplitCount(SSplitContext* pCxt, SStableSplitInfo* pInfo) { } } +static int32_t stbSplSplitAnomalyForStream(SSplitContext* pCxt, SStableSplitInfo* pInfo) { + return TSDB_CODE_PLAN_INTERNAL_ERROR; +} + +static int32_t stbSplSplitAnomaly(SSplitContext* pCxt, SStableSplitInfo* pInfo) { + if (pCxt->pPlanCxt->streamQuery) { + return stbSplSplitAnomalyForStream(pCxt, pInfo); + } else { + return stbSplSplitSessionOrStateForBatch(pCxt, pInfo); + } +} + static int32_t stbSplSplitWindowForCrossTable(SSplitContext* pCxt, SStableSplitInfo* pInfo) { switch (((SWindowLogicNode*)pInfo->pSplitNode)->winType) { case WINDOW_TYPE_INTERVAL: @@ -951,6 +963,8 @@ static int32_t stbSplSplitWindowForCrossTable(SSplitContext* pCxt, SStableSplitI return stbSplSplitEvent(pCxt, pInfo); case WINDOW_TYPE_COUNT: return stbSplSplitCount(pCxt, pInfo); + case WINDOW_TYPE_ANOMALY: + return stbSplSplitAnomaly(pCxt, pInfo); default: break; } @@ -2000,7 +2014,8 @@ typedef struct SQnodeSplitInfo { static bool qndSplFindSplitNode(SSplitContext* pCxt, SLogicSubplan* pSubplan, SLogicNode* pNode, SQnodeSplitInfo* pInfo) { if (QUERY_NODE_LOGIC_PLAN_SCAN == nodeType(pNode) && NULL != pNode->pParent && - QUERY_NODE_LOGIC_PLAN_INTERP_FUNC != nodeType(pNode->pParent) && ((SScanLogicNode*)pNode)->scanSeq[0] <= 1 && + QUERY_NODE_LOGIC_PLAN_INTERP_FUNC != nodeType(pNode->pParent) && + QUERY_NODE_LOGIC_PLAN_FORECAST_FUNC != nodeType(pNode->pParent) && ((SScanLogicNode*)pNode)->scanSeq[0] <= 1 && ((SScanLogicNode*)pNode)->scanSeq[1] <= 1) { pInfo->pSplitNode = pNode; pInfo->pSubplan = pSubplan; From e792c1a98662283c9616876432068017030c64d3 Mon Sep 17 00:00:00 2001 From: Shengliang Guan Date: Thu, 10 Oct 2024 11:49:26 +0800 Subject: [PATCH 44/51] fix: unitest error --- source/libs/function/src/functionMgt.c | 9 ++++ source/libs/parser/src/parAstCreater.c | 60 +++++++++++++++++++++++++ source/libs/parser/src/parUtil.c | 2 + source/libs/planner/src/planOptimizer.c | 2 + 4 files changed, 73 insertions(+) diff --git a/source/libs/function/src/functionMgt.c b/source/libs/function/src/functionMgt.c index d436925046..1717702df7 100644 --- a/source/libs/function/src/functionMgt.c +++ b/source/libs/function/src/functionMgt.c @@ -232,6 +232,15 @@ bool fmIsInterpFunc(int32_t funcId) { bool fmIsInterpPseudoColumnFunc(int32_t funcId) { return isSpecificClassifyFunc(funcId, FUNC_MGT_INTERP_PC_FUNC); } +bool fmIsForecastFunc(int32_t funcId) { + if (funcId < 0 || funcId >= funcMgtBuiltinsNum) { + return false; + } + return FUNCTION_TYPE_FORECAST == funcMgtBuiltins[funcId].type; +} + +bool fmIsForecastPseudoColumnFunc(int32_t funcId) { return isSpecificClassifyFunc(funcId, FUNC_MGT_FORECAST_PC_FUNC); } + bool fmIsLastRowFunc(int32_t funcId) { if (funcId < 0 || funcId >= funcMgtBuiltinsNum) { return false; diff --git a/source/libs/parser/src/parAstCreater.c b/source/libs/parser/src/parAstCreater.c index cd3095ede8..684bd24a9c 100644 --- a/source/libs/parser/src/parAstCreater.c +++ b/source/libs/parser/src/parAstCreater.c @@ -1367,6 +1367,25 @@ _err: return NULL; } +SNode* createAnomalyWindowNode(SAstCreateContext* pCxt, SNode* pExpr, const SToken* pFuncOpt) { + SAnomalyWindowNode* pAnomaly = NULL; + CHECK_PARSER_STATUS(pCxt); + pCxt->errCode = nodesMakeNode(QUERY_NODE_ANOMALY_WINDOW, (SNode**)&pAnomaly); + CHECK_MAKE_NODE(pAnomaly); + pAnomaly->pCol = createPrimaryKeyCol(pCxt, NULL); + CHECK_MAKE_NODE(pAnomaly->pCol); + pAnomaly->pExpr = pExpr; + if (pFuncOpt == NULL) { + tstrncpy(pAnomaly->anomalyOpt, "algo=iqr", TSDB_ANAL_ALGO_OPTION_LEN); + } else { + (void)trimString(pFuncOpt->z, pFuncOpt->n, pAnomaly->anomalyOpt, sizeof(pAnomaly->anomalyOpt)); + } + return (SNode*)pAnomaly; +_err: + nodesDestroyNode((SNode*)pAnomaly); + return NULL; +} + SNode* createIntervalWindowNode(SAstCreateContext* pCxt, SNode* pInterval, SNode* pOffset, SNode* pSliding, SNode* pFill) { SIntervalWindowNode* interval = NULL; @@ -2997,6 +3016,47 @@ _err: return NULL; } +SNode* createCreateAnodeStmt(SAstCreateContext* pCxt, const SToken* pUrl) { + CHECK_PARSER_STATUS(pCxt); + SCreateAnodeStmt* pStmt = NULL; + pCxt->errCode = nodesMakeNode(QUERY_NODE_CREATE_ANODE_STMT, (SNode**)&pStmt); + CHECK_MAKE_NODE(pStmt); + (void)trimString(pUrl->z, pUrl->n, pStmt->url, sizeof(pStmt->url)); + return (SNode*)pStmt; +_err: + return NULL; +} + +SNode* createDropAnodeStmt(SAstCreateContext* pCxt, const SToken* pAnode) { + CHECK_PARSER_STATUS(pCxt); + SUpdateAnodeStmt* pStmt = NULL; + pCxt->errCode = nodesMakeNode(QUERY_NODE_DROP_ANODE_STMT, (SNode**)&pStmt); + CHECK_MAKE_NODE(pStmt); + if (NULL != pAnode) { + pStmt->anodeId = taosStr2Int32(pAnode->z, NULL, 10); + } else { + pStmt->anodeId = -1; + } + return (SNode*)pStmt; +_err: + return NULL; +} + +SNode* createUpdateAnodeStmt(SAstCreateContext* pCxt, const SToken* pAnode, bool updateAll) { + CHECK_PARSER_STATUS(pCxt); + SUpdateAnodeStmt* pStmt = NULL; + pCxt->errCode = nodesMakeNode(QUERY_NODE_UPDATE_ANODE_STMT, (SNode**)&pStmt); + CHECK_MAKE_NODE(pStmt); + if (NULL != pAnode) { + pStmt->anodeId = taosStr2Int32(pAnode->z, NULL, 10); + } else { + pStmt->anodeId = -1; + } + return (SNode*)pStmt; +_err: + return NULL; +} + SNode* createEncryptKeyStmt(SAstCreateContext* pCxt, const SToken* pValue) { SToken config; config.type = TK_NK_STRING; diff --git a/source/libs/parser/src/parUtil.c b/source/libs/parser/src/parUtil.c index 1ce8b04324..189afdfcd3 100644 --- a/source/libs/parser/src/parUtil.c +++ b/source/libs/parser/src/parUtil.c @@ -185,6 +185,8 @@ static char* getSyntaxErrFormat(int32_t errCode) { return "%s is not supported in system table query"; case TSDB_CODE_PAR_INVALID_INTERP_CLAUSE: return "Invalid usage of RANGE clause, EVERY clause or FILL clause"; + case TSDB_CODE_PAR_INVALID_FORECAST_CLAUSE: + return "Invalid usage of forecast clause"; case TSDB_CODE_PAR_NO_VALID_FUNC_IN_WIN: return "No valid function in window query"; case TSDB_CODE_PAR_INVALID_OPTR_USAGE: diff --git a/source/libs/planner/src/planOptimizer.c b/source/libs/planner/src/planOptimizer.c index 1bcec86385..3b4e835465 100644 --- a/source/libs/planner/src/planOptimizer.c +++ b/source/libs/planner/src/planOptimizer.c @@ -2380,6 +2380,8 @@ static bool sortPriKeyOptHasUnsupportedPkFunc(SLogicNode* pLogicNode, EOrder sor case QUERY_NODE_LOGIC_PLAN_INTERP_FUNC: pFuncList = ((SInterpFuncLogicNode*)pLogicNode)->pFuncs; break; + case QUERY_NODE_LOGIC_PLAN_FORECAST_FUNC: + pFuncList = ((SForecastFuncLogicNode*)pLogicNode)->pFuncs; default: break; } From 3ad2eb939ca965afe68a4712f196eabbaf822bfa Mon Sep 17 00:00:00 2001 From: Shengliang Guan Date: Thu, 10 Oct 2024 11:54:32 +0800 Subject: [PATCH 45/51] fix: unitest error --- source/libs/function/src/builtins.c | 86 ++++++++++++++++++++++++- source/libs/function/src/builtinsimpl.c | 6 ++ 2 files changed, 91 insertions(+), 1 deletion(-) diff --git a/source/libs/function/src/builtins.c b/source/libs/function/src/builtins.c index 643c4bba82..86f1331e92 100644 --- a/source/libs/function/src/builtins.c +++ b/source/libs/function/src/builtins.c @@ -16,9 +16,10 @@ #include "builtins.h" #include "builtinsimpl.h" #include "cJSON.h" +#include "geomFunc.h" #include "querynodes.h" #include "scalar.h" -#include "geomFunc.h" +#include "tanal.h" #include "taoserror.h" #include "ttime.h" @@ -2078,6 +2079,47 @@ static int32_t translateMode(SFunctionNode* pFunc, char* pErrBuf, int32_t len) { return translateUniqueMode(pFunc, pErrBuf, len, false); } +static int32_t translateForecast(SFunctionNode* pFunc, char* pErrBuf, int32_t len) { + int32_t numOfParams = LIST_LENGTH(pFunc->pParameterList); + if (2 != numOfParams && 1 != numOfParams) { + return invaildFuncParaNumErrMsg(pErrBuf, len, "FORECAST require 1 or 2 parameters"); + } + + uint8_t valType = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 0))->type; + if (!IS_MATHABLE_TYPE(valType)) { + return invaildFuncParaTypeErrMsg(pErrBuf, len, "FORECAST only support mathable column"); + } + + if (numOfParams == 2) { + uint8_t optionType = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 1))->type; + if (TSDB_DATA_TYPE_BINARY != optionType) { + return invaildFuncParaTypeErrMsg(pErrBuf, len, "FORECAST option should be varchar"); + } + + SNode* pOption = nodesListGetNode(pFunc->pParameterList, 1); + if (QUERY_NODE_VALUE != nodeType(pOption)) { + return invaildFuncParaTypeErrMsg(pErrBuf, len, "FORECAST option should be value"); + } + + SValueNode* pValue = (SValueNode*)pOption; + if (!taosAnalGetOptStr(pValue->literal, "algo", NULL, 0) != 0) { + return invaildFuncParaValueErrMsg(pErrBuf, len, "FORECAST option should include algo field"); + } + + pValue->notReserved = true; + } + + pFunc->node.resType = (SDataType){.bytes = tDataTypes[valType].bytes, .type = valType}; + return TSDB_CODE_SUCCESS; +} + +static int32_t translateForecastConf(SFunctionNode* pFunc, char* pErrBuf, int32_t len) { + pFunc->node.resType = (SDataType){.bytes = tDataTypes[TSDB_DATA_TYPE_FLOAT].bytes, .type = TSDB_DATA_TYPE_FLOAT}; + return TSDB_CODE_SUCCESS; +} + +static EFuncReturnRows forecastEstReturnRows(SFunctionNode* pFunc) { return FUNC_RETURN_ROWS_N; } + static int32_t translateDiff(SFunctionNode* pFunc, char* pErrBuf, int32_t len) { int32_t numOfParams = LIST_LENGTH(pFunc->pParameterList); if (numOfParams > 2) { @@ -3623,6 +3665,48 @@ const SBuiltinFuncDefinition funcMgtBuiltins[] = { .estimateReturnRowsFunc = diffEstReturnRows, .processFuncByRow = diffFunctionByRow, }, + { + .name = "forecast", + .type = FUNCTION_TYPE_FORECAST, + .classification = FUNC_MGT_TIMELINE_FUNC | FUNC_MGT_INTERVAL_INTERPO_FUNC | FUNC_MGT_IMPLICIT_TS_FUNC | + FUNC_MGT_FORBID_STREAM_FUNC | FUNC_MGT_FORBID_SYSTABLE_FUNC | FUNC_MGT_KEEP_ORDER_FUNC | FUNC_MGT_PRIMARY_KEY_FUNC, + .translateFunc = translateForecast, + .getEnvFunc = getSelectivityFuncEnv, + .initFunc = functionSetup, + .processFunc = NULL, + .finalizeFunc = NULL, + .estimateReturnRowsFunc = forecastEstReturnRows, + }, + { + .name = "_frowts", + .type = FUNCTION_TYPE_FORECAST_ROWTS, + .classification = FUNC_MGT_PSEUDO_COLUMN_FUNC | FUNC_MGT_FORECAST_PC_FUNC | FUNC_MGT_KEEP_ORDER_FUNC, + .translateFunc = translateTimePseudoColumn, + .getEnvFunc = getTimePseudoFuncEnv, + .initFunc = NULL, + .sprocessFunc = NULL, + .finalizeFunc = NULL + }, + { + .name = "_flow", + .type = FUNCTION_TYPE_FORECAST_LOW, + .classification = FUNC_MGT_PSEUDO_COLUMN_FUNC | FUNC_MGT_FORECAST_PC_FUNC | FUNC_MGT_KEEP_ORDER_FUNC, + .translateFunc = translateForecastConf, + .getEnvFunc = getForecastConfEnv, + .initFunc = NULL, + .sprocessFunc = NULL, + .finalizeFunc = NULL + }, + { + .name = "_fhigh", + .type = FUNCTION_TYPE_FORECAST_HIGH, + .classification = FUNC_MGT_PSEUDO_COLUMN_FUNC | FUNC_MGT_FORECAST_PC_FUNC | FUNC_MGT_KEEP_ORDER_FUNC, + .translateFunc = translateForecastConf, + .getEnvFunc = getForecastConfEnv, + .initFunc = NULL, + .sprocessFunc = NULL, + .finalizeFunc = NULL + }, { .name = "statecount", .type = FUNCTION_TYPE_STATE_COUNT, diff --git a/source/libs/function/src/builtinsimpl.c b/source/libs/function/src/builtinsimpl.c index 9eda9e5b57..f13685239a 100644 --- a/source/libs/function/src/builtinsimpl.c +++ b/source/libs/function/src/builtinsimpl.c @@ -18,6 +18,7 @@ #include "function.h" #include "query.h" #include "querynodes.h" +#include "tanal.h" #include "tcompare.h" #include "tdatablock.h" #include "tdigest.h" @@ -3578,6 +3579,11 @@ bool funcInputGetNextRowIndex(SInputColumnInfoData* pInput, int32_t from, bool f } } +bool getForecastConfEnv(SFunctionNode* UNUSED_PARAM(pFunc), SFuncExecEnv* pEnv) { + pEnv->calcMemSize = sizeof(float); + return true; +} + int32_t diffResultIsNull(SqlFunctionCtx* pCtx, SFuncInputRow* pRow){ SResultRowEntryInfo* pResInfo = GET_RES_INFO(pCtx); SDiffInfo* pDiffInfo = GET_ROWCELL_INTERBUF(pResInfo); From 43fa9b4cc776e565bfd0ef94391cf7b07826d792 Mon Sep 17 00:00:00 2001 From: Shengliang Guan Date: Thu, 10 Oct 2024 13:04:17 +0800 Subject: [PATCH 46/51] fix: unitest error --- source/libs/function/src/builtins.c | 86 +---------------------------- 1 file changed, 1 insertion(+), 85 deletions(-) diff --git a/source/libs/function/src/builtins.c b/source/libs/function/src/builtins.c index 86f1331e92..643c4bba82 100644 --- a/source/libs/function/src/builtins.c +++ b/source/libs/function/src/builtins.c @@ -16,10 +16,9 @@ #include "builtins.h" #include "builtinsimpl.h" #include "cJSON.h" -#include "geomFunc.h" #include "querynodes.h" #include "scalar.h" -#include "tanal.h" +#include "geomFunc.h" #include "taoserror.h" #include "ttime.h" @@ -2079,47 +2078,6 @@ static int32_t translateMode(SFunctionNode* pFunc, char* pErrBuf, int32_t len) { return translateUniqueMode(pFunc, pErrBuf, len, false); } -static int32_t translateForecast(SFunctionNode* pFunc, char* pErrBuf, int32_t len) { - int32_t numOfParams = LIST_LENGTH(pFunc->pParameterList); - if (2 != numOfParams && 1 != numOfParams) { - return invaildFuncParaNumErrMsg(pErrBuf, len, "FORECAST require 1 or 2 parameters"); - } - - uint8_t valType = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 0))->type; - if (!IS_MATHABLE_TYPE(valType)) { - return invaildFuncParaTypeErrMsg(pErrBuf, len, "FORECAST only support mathable column"); - } - - if (numOfParams == 2) { - uint8_t optionType = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 1))->type; - if (TSDB_DATA_TYPE_BINARY != optionType) { - return invaildFuncParaTypeErrMsg(pErrBuf, len, "FORECAST option should be varchar"); - } - - SNode* pOption = nodesListGetNode(pFunc->pParameterList, 1); - if (QUERY_NODE_VALUE != nodeType(pOption)) { - return invaildFuncParaTypeErrMsg(pErrBuf, len, "FORECAST option should be value"); - } - - SValueNode* pValue = (SValueNode*)pOption; - if (!taosAnalGetOptStr(pValue->literal, "algo", NULL, 0) != 0) { - return invaildFuncParaValueErrMsg(pErrBuf, len, "FORECAST option should include algo field"); - } - - pValue->notReserved = true; - } - - pFunc->node.resType = (SDataType){.bytes = tDataTypes[valType].bytes, .type = valType}; - return TSDB_CODE_SUCCESS; -} - -static int32_t translateForecastConf(SFunctionNode* pFunc, char* pErrBuf, int32_t len) { - pFunc->node.resType = (SDataType){.bytes = tDataTypes[TSDB_DATA_TYPE_FLOAT].bytes, .type = TSDB_DATA_TYPE_FLOAT}; - return TSDB_CODE_SUCCESS; -} - -static EFuncReturnRows forecastEstReturnRows(SFunctionNode* pFunc) { return FUNC_RETURN_ROWS_N; } - static int32_t translateDiff(SFunctionNode* pFunc, char* pErrBuf, int32_t len) { int32_t numOfParams = LIST_LENGTH(pFunc->pParameterList); if (numOfParams > 2) { @@ -3665,48 +3623,6 @@ const SBuiltinFuncDefinition funcMgtBuiltins[] = { .estimateReturnRowsFunc = diffEstReturnRows, .processFuncByRow = diffFunctionByRow, }, - { - .name = "forecast", - .type = FUNCTION_TYPE_FORECAST, - .classification = FUNC_MGT_TIMELINE_FUNC | FUNC_MGT_INTERVAL_INTERPO_FUNC | FUNC_MGT_IMPLICIT_TS_FUNC | - FUNC_MGT_FORBID_STREAM_FUNC | FUNC_MGT_FORBID_SYSTABLE_FUNC | FUNC_MGT_KEEP_ORDER_FUNC | FUNC_MGT_PRIMARY_KEY_FUNC, - .translateFunc = translateForecast, - .getEnvFunc = getSelectivityFuncEnv, - .initFunc = functionSetup, - .processFunc = NULL, - .finalizeFunc = NULL, - .estimateReturnRowsFunc = forecastEstReturnRows, - }, - { - .name = "_frowts", - .type = FUNCTION_TYPE_FORECAST_ROWTS, - .classification = FUNC_MGT_PSEUDO_COLUMN_FUNC | FUNC_MGT_FORECAST_PC_FUNC | FUNC_MGT_KEEP_ORDER_FUNC, - .translateFunc = translateTimePseudoColumn, - .getEnvFunc = getTimePseudoFuncEnv, - .initFunc = NULL, - .sprocessFunc = NULL, - .finalizeFunc = NULL - }, - { - .name = "_flow", - .type = FUNCTION_TYPE_FORECAST_LOW, - .classification = FUNC_MGT_PSEUDO_COLUMN_FUNC | FUNC_MGT_FORECAST_PC_FUNC | FUNC_MGT_KEEP_ORDER_FUNC, - .translateFunc = translateForecastConf, - .getEnvFunc = getForecastConfEnv, - .initFunc = NULL, - .sprocessFunc = NULL, - .finalizeFunc = NULL - }, - { - .name = "_fhigh", - .type = FUNCTION_TYPE_FORECAST_HIGH, - .classification = FUNC_MGT_PSEUDO_COLUMN_FUNC | FUNC_MGT_FORECAST_PC_FUNC | FUNC_MGT_KEEP_ORDER_FUNC, - .translateFunc = translateForecastConf, - .getEnvFunc = getForecastConfEnv, - .initFunc = NULL, - .sprocessFunc = NULL, - .finalizeFunc = NULL - }, { .name = "statecount", .type = FUNCTION_TYPE_STATE_COUNT, From ad61980e40efe27f4659a2c0079aeb8e86e04a08 Mon Sep 17 00:00:00 2001 From: Shengliang Guan Date: Thu, 10 Oct 2024 13:17:24 +0800 Subject: [PATCH 47/51] fix: unitest error --- source/libs/function/src/builtins.c | 44 ++++++++++++++++++++++++++++- 1 file changed, 43 insertions(+), 1 deletion(-) diff --git a/source/libs/function/src/builtins.c b/source/libs/function/src/builtins.c index 643c4bba82..857e472ee0 100644 --- a/source/libs/function/src/builtins.c +++ b/source/libs/function/src/builtins.c @@ -16,9 +16,10 @@ #include "builtins.h" #include "builtinsimpl.h" #include "cJSON.h" +#include "geomFunc.h" #include "querynodes.h" #include "scalar.h" -#include "geomFunc.h" +#include "tanal.h" #include "taoserror.h" #include "ttime.h" @@ -2078,6 +2079,47 @@ static int32_t translateMode(SFunctionNode* pFunc, char* pErrBuf, int32_t len) { return translateUniqueMode(pFunc, pErrBuf, len, false); } +static int32_t translateForecast(SFunctionNode* pFunc, char* pErrBuf, int32_t len) { + int32_t numOfParams = LIST_LENGTH(pFunc->pParameterList); + if (2 != numOfParams && 1 != numOfParams) { + return invaildFuncParaNumErrMsg(pErrBuf, len, "FORECAST require 1 or 2 parameters"); + } + + uint8_t valType = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 0))->type; + if (!IS_MATHABLE_TYPE(valType)) { + return invaildFuncParaTypeErrMsg(pErrBuf, len, "FORECAST only support mathable column"); + } + + if (numOfParams == 2) { + uint8_t optionType = getSDataTypeFromNode(nodesListGetNode(pFunc->pParameterList, 1))->type; + if (TSDB_DATA_TYPE_BINARY != optionType) { + return invaildFuncParaTypeErrMsg(pErrBuf, len, "FORECAST option should be varchar"); + } + + SNode* pOption = nodesListGetNode(pFunc->pParameterList, 1); + if (QUERY_NODE_VALUE != nodeType(pOption)) { + return invaildFuncParaTypeErrMsg(pErrBuf, len, "FORECAST option should be value"); + } + + SValueNode* pValue = (SValueNode*)pOption; + if (!taosAnalGetOptStr(pValue->literal, "algo", NULL, 0) != 0) { + return invaildFuncParaValueErrMsg(pErrBuf, len, "FORECAST option should include algo field"); + } + + pValue->notReserved = true; + } + + pFunc->node.resType = (SDataType){.bytes = tDataTypes[valType].bytes, .type = valType}; + return TSDB_CODE_SUCCESS; +} + +static int32_t translateForecastConf(SFunctionNode* pFunc, char* pErrBuf, int32_t len) { + pFunc->node.resType = (SDataType){.bytes = tDataTypes[TSDB_DATA_TYPE_FLOAT].bytes, .type = TSDB_DATA_TYPE_FLOAT}; + return TSDB_CODE_SUCCESS; +} + +static EFuncReturnRows forecastEstReturnRows(SFunctionNode* pFunc) { return FUNC_RETURN_ROWS_N; } + static int32_t translateDiff(SFunctionNode* pFunc, char* pErrBuf, int32_t len) { int32_t numOfParams = LIST_LENGTH(pFunc->pParameterList); if (numOfParams > 2) { From 26eebde0efa1a137894c91043dd0e53821445153 Mon Sep 17 00:00:00 2001 From: Hongze Cheng Date: Thu, 10 Oct 2024 14:14:58 +0800 Subject: [PATCH 48/51] enh: change tsdb snapshot strategy to accelarate the snapshot process --- source/dnode/vnode/src/tsdb/tsdbSnapshot.c | 17 ++++++++++++++++- 1 file changed, 16 insertions(+), 1 deletion(-) diff --git a/source/dnode/vnode/src/tsdb/tsdbSnapshot.c b/source/dnode/vnode/src/tsdb/tsdbSnapshot.c index d0ea58c28a..e8740a0650 100644 --- a/source/dnode/vnode/src/tsdb/tsdbSnapshot.c +++ b/source/dnode/vnode/src/tsdb/tsdbSnapshot.c @@ -623,6 +623,7 @@ static int32_t tsdbSnapWriteFileSetOpenReader(STsdbSnapWriter* writer) { int32_t lino = 0; if (writer->ctx->fset) { +#if 0 // open data reader SDataFileReaderConfig dataFileReaderConfig = { .tsdb = writer->tsdb, @@ -650,6 +651,7 @@ static int32_t tsdbSnapWriteFileSetOpenReader(STsdbSnapWriter* writer) { code = tsdbDataFileReaderOpen(NULL, &dataFileReaderConfig, &writer->ctx->dataReader); TSDB_CHECK_CODE(code, lino, _exit); +#endif // open stt reader array SSttLvl* lvl; @@ -791,6 +793,15 @@ static int32_t tsdbSnapWriteFileSetOpenWriter(STsdbSnapWriter* writer) { .did = writer->ctx->did, .level = 0, }; + // merge stt files to either data or a new stt file + if (writer->ctx->fset) { + for (int32_t ftype = 0; ftype < TSDB_FTYPE_MAX; ++ftype) { + if (writer->ctx->fset->farr[ftype] != NULL) { + config.files[ftype].exist = true; + config.files[ftype].file = writer->ctx->fset->farr[ftype]->f[0]; + } + } + } code = tsdbFSetWriterOpen(&config, &writer->ctx->fsetWriter); TSDB_CHECK_CODE(code, lino, _exit); @@ -842,6 +853,8 @@ static int32_t tsdbSnapWriteFileSetBegin(STsdbSnapWriter* writer, int32_t fid) { _exit: if (code) { TSDB_ERROR_LOG(TD_VID(writer->tsdb->pVnode), lino, code); + } else { + tsdbInfo("vgId:%d %s succeeded, fid:%d", TD_VID(writer->tsdb->pVnode), __func__, fid); } return code; } @@ -922,6 +935,8 @@ static int32_t tsdbSnapWriteFileSetEnd(STsdbSnapWriter* writer) { _exit: if (code) { TSDB_ERROR_LOG(TD_VID(writer->tsdb->pVnode), lino, code); + } else { + tsdbInfo("vgId:%d %s succeeded, fid:%d", TD_VID(writer->tsdb->pVnode), __func__, writer->ctx->fid); } return code; } @@ -1175,7 +1190,7 @@ _exit: if (code) { TSDB_ERROR_LOG(TD_VID(tsdb->pVnode), lino, code); } else { - tsdbInfo("vgId:%d %s done", TD_VID(tsdb->pVnode), __func__); + tsdbInfo("vgId:%d %s done, rollback:%d", TD_VID(tsdb->pVnode), __func__, rollback); } return code; } From e9c620ccb69f66d6703f3579d0c35561f283375b Mon Sep 17 00:00:00 2001 From: gccgdb1234 Date: Thu, 10 Oct 2024 15:21:34 +0800 Subject: [PATCH 49/51] doc: fix broken links --- docs/zh/04-get-started/05-cloud.md | 2 +- docs/zh/08-operation/03-deployment.md | 4 ++-- docs/zh/08-operation/12-multi.md | 2 +- docs/zh/08-operation/18-dual.md | 2 +- 4 files changed, 5 insertions(+), 5 deletions(-) diff --git a/docs/zh/04-get-started/05-cloud.md b/docs/zh/04-get-started/05-cloud.md index bd76add527..c8bd0f5dac 100644 --- a/docs/zh/04-get-started/05-cloud.md +++ b/docs/zh/04-get-started/05-cloud.md @@ -15,7 +15,7 @@ TDengine Cloud 大幅减轻了用户在部署、运维等方面的人力负担 要在 TDengine Cloud 注册新用户,请遵循以下简易步骤完成注册流程: -1. 打开浏览器,访问 TDengine Cloud 的首页:https://cloud.taosdata.com,在右边的“注册”部分,填入自己的姓名以及企业邮箱地址,点击“获取验证码”按钮。 +1. 打开浏览器,访问 TDengine Cloud[https://cloud.taosdata.com],在右边的“注册”部分,填入自己的姓名以及企业邮箱地址,点击“获取验证码”按钮。 2. 检查企业邮箱,找到主题为“你的 TDengine Cloud 注册账户验证码”的邮件。从邮件内容中复制 6 位验证码,并将其粘贴到注册页面上的“验证码”输入框中。接着,点击“注册 TDengine Cloud”按钮,进入客户信息补全页面。 diff --git a/docs/zh/08-operation/03-deployment.md b/docs/zh/08-operation/03-deployment.md index 83b2c91843..2e0c2a7989 100644 --- a/docs/zh/08-operation/03-deployment.md +++ b/docs/zh/08-operation/03-deployment.md @@ -206,11 +206,11 @@ http { ### 部署 taosX -如果想使用 TDengine 的数据接入能力,需要部署 taosX 服务,关于它的详细说明和部署请参考[taosX 参考手册](../../reference/components/taosx)。 +如果想使用 TDengine 的数据接入能力,需要部署 taosX 服务,关于它的详细说明和部署请参考企业版参考手册。 ### 部署 taosX-Agent -有些数据源如 Pi, OPC 等,因为网络条件和数据源访问的限制,taosX 无法直接访问数据源,这种情况下需要部署一个代理服务 taosX-Agent,关于它的详细说明和部署请参考[taosX-Agent 参考手册](../../reference/components/taosx-agent)。 +有些数据源如 Pi, OPC 等,因为网络条件和数据源访问的限制,taosX 无法直接访问数据源,这种情况下需要部署一个代理服务 taosX-Agent,关于它的详细说明和部署请参考企业版参考手册。 ### 部署 taos-Explorer diff --git a/docs/zh/08-operation/12-multi.md b/docs/zh/08-operation/12-multi.md index 8f11ee4326..a5608ad5fa 100644 --- a/docs/zh/08-operation/12-multi.md +++ b/docs/zh/08-operation/12-multi.md @@ -70,7 +70,7 @@ dataDir /mnt/data6 2 0 |参数名称 | 参数含义 | |:-------------|:-----------------------------------------------| -|s3EndPoint | 用户所在地域的 COS 服务域名,支持 http 和 https,bucket 的区域需要与 endpoint 的保持一致,否则无法访问。例如:http://cos.ap-beijing.myqcloud.com | +|s3EndPoint | 用户所在地域的 COS 服务域名,支持 http 和 https,bucket 的区域需要与 endpoint 的保持一致,否则无法访问。 | |s3AccessKey |冒号分隔的用户 SecretId:SecretKey。例如:AKIDsQmwsfKxTo2A6nGVXZN0UlofKn6JRRSJ:lIdoy99ygEacU7iHfogaN2Xq0yumSm1E | |s3BucketName | 存储桶名称,减号后面是用户注册 COS 服务的 AppId。其中 AppId 是 COS 特有,AWS 和阿里云都没有,配置时需要作为 bucket name 的一部分,使用减号分隔。参数值均为字符串类型,但不需要引号。例如:test0711-1309024725 | |s3UploadDelaySec | data 文件持续多长时间不再变动后上传至 s3,单位:秒。最小值:1;最大值:2592000 (30天),默认值 60 秒 | diff --git a/docs/zh/08-operation/18-dual.md b/docs/zh/08-operation/18-dual.md index 354e715602..c7871a8e1e 100644 --- a/docs/zh/08-operation/18-dual.md +++ b/docs/zh/08-operation/18-dual.md @@ -83,7 +83,7 @@ taosx replica start ```shell taosx replica start -f td1:6030 -t td2:6030 ``` -该示例命令会自动创建除 information_schema、performance_schema、log、audit 库之外的同步任务。可以使用 http://td2:6041 指定该 endpoint 使用 websocket 接口(默认是原生接口)。也可以指定数据库同步:taosx replica start -f td1:6030 -t td2:6030 db1 仅创建指定的数据库同步任务。 +该示例命令会自动创建除 information_schema、performance_schema、log、audit 库之外的同步任务。可以使用 `http://td2:6041` 指定该 endpoint 使用 websocket 接口(默认是原生接口)。也可以指定数据库同步:taosx replica start -f td1:6030 -t td2:6030 db1 仅创建指定的数据库同步任务。 2. 方法二 From 6294023a8cc0d0d36083230436b8bc25f03f62b8 Mon Sep 17 00:00:00 2001 From: Ping Xiao Date: Thu, 10 Oct 2024 13:59:45 +0800 Subject: [PATCH 50/51] fix windows, mac installer issue --- .../tools/com.taosdata.taos-explorer.plist | 33 +++++++++++++++++++ packaging/tools/remove.sh | 26 ++++++++++----- packaging/tools/tdengine.iss | 4 +-- 3 files changed, 53 insertions(+), 10 deletions(-) create mode 100644 packaging/tools/com.taosdata.taos-explorer.plist diff --git a/packaging/tools/com.taosdata.taos-explorer.plist b/packaging/tools/com.taosdata.taos-explorer.plist new file mode 100644 index 0000000000..2edb5552ad --- /dev/null +++ b/packaging/tools/com.taosdata.taos-explorer.plist @@ -0,0 +1,33 @@ + + + + + Label + com.tdengine.taos-explorer + ProgramArguments + + /usr/local/bin/taos-explorer + + ProcessType + Interactive + Disabled + + RunAtLoad + + LaunchOnlyOnce + + SessionCreate + + ExitTimeOut + 600 + KeepAlive + + SuccessfulExit + + AfterInitialDemand + + + Program + /usr/local/bin/taos-explorer + + \ No newline at end of file diff --git a/packaging/tools/remove.sh b/packaging/tools/remove.sh index 58a17e2a50..c3f459ca9c 100755 --- a/packaging/tools/remove.sh +++ b/packaging/tools/remove.sh @@ -206,10 +206,17 @@ function clean_log() { } function clean_service_on_launchctl() { - ${csudouser}launchctl unload -w /Library/LaunchDaemons/com.taosdata.taosd.plist > /dev/null 2>&1 || : - ${csudo}rm /Library/LaunchDaemons/com.taosdata.taosd.plist > /dev/null 2>&1 || : - ${csudouser}launchctl unload -w /Library/LaunchDaemons/com.taosdata.${clientName2}adapter.plist > /dev/null 2>&1 || : - ${csudo}rm /Library/LaunchDaemons/com.taosdata.${clientName2}adapter.plist > /dev/null 2>&1 || : + ${csudo}launchctl unload -w /Library/LaunchDaemons/com.taosdata.taosd.plist || : + ${csudo}launchctl unload -w /Library/LaunchDaemons/com.taosdata.${PREFIX}adapter.plist || : + ${csudo}launchctl unload -w /Library/LaunchDaemons/com.taosdata.${PREFIX}keeper.plist || : + ${csudo}launchctl unload -w /Library/LaunchDaemons/com.taosdata.${PREFIX}-explorer.plist || : + + ${csudo}launchctl remove com.tdengine.taosd || : + ${csudo}launchctl remove com.tdengine.${PREFIX}adapter || : + ${csudo}launchctl remove com.tdengine.${PREFIX}keeper || : + ${csudo}launchctl remove com.tdengine.${PREFIX}-explorer || : + + ${csudo}rm /Library/LaunchDaemons/com.taosdata.* > /dev/null 2>&1 || : } function remove_data_and_config() { @@ -250,6 +257,12 @@ if [ -e ${install_main_dir}/uninstall_${PREFIX}x.sh ]; then fi fi + +if [ "$osType" = "Darwin" ]; then + clean_service_on_launchctl + ${csudo}rm -rf /Applications/TDengine.app +fi + remove_bin clean_header # Remove lib file @@ -282,10 +295,7 @@ elif echo $osinfo | grep -qwi "centos"; then # echo "this is centos system" ${csudo}rpm -e --noscripts tdengine >/dev/null 2>&1 || : fi -if [ "$osType" = "Darwin" ]; then - clean_service_on_launchctl - ${csudo}rm -rf /Applications/TDengine.app -fi + command -v systemctl >/dev/null 2>&1 && ${csudo}systemctl daemon-reload >/dev/null 2>&1 || true echo diff --git a/packaging/tools/tdengine.iss b/packaging/tools/tdengine.iss index 8085c55e3e..c3eb6f9f68 100644 --- a/packaging/tools/tdengine.iss +++ b/packaging/tools/tdengine.iss @@ -71,8 +71,8 @@ Source: {#MyAppSourceDir}\taosdump.exe; DestDir: "{app}"; DestName: "{#CusPrompt Filename: {sys}\sc.exe; Parameters: "create taosd start= DEMAND binPath= ""C:\\TDengine\\taosd.exe --win_service""" ; Flags: runhidden Filename: {sys}\sc.exe; Parameters: "create taosadapter start= DEMAND binPath= ""C:\\TDengine\\taosadapter.exe""" ; Flags: runhidden -Filename: "C:\Windows\System32\odbcconf.exe"; Parameters: "/S /F win_odbcinst.ini"; WorkingDir: "{app}\taos_odbc\x64"; Flags: runhidden; StatusMsg: "Configuring ODBC x64" -Filename: "C:\Windows\SysWOW64\odbcconf.exe"; Parameters: "/S /F win_odbcinst.ini"; WorkingDir: "{app}\taos_odbc\x86"; Flags: runhidden; StatusMsg: "Configuring ODBC x86" +Filename: "C:\Windows\System32\odbcconf.exe"; Parameters: "/S /F win_odbc_install.ini"; WorkingDir: "{app}\taos_odbc\x64"; Flags: runhidden; StatusMsg: "Configuring ODBC x64" +Filename: "C:\Windows\SysWOW64\odbcconf.exe"; Parameters: "/S /F win_odbc_install.ini"; WorkingDir: "{app}\taos_odbc\x86"; Flags: runhidden; StatusMsg: "Configuring ODBC x86" [UninstallRun] RunOnceId: "stoptaosd"; Filename: {sys}\sc.exe; Parameters: "stop taosd" ; Flags: runhidden From 1bb191cfca6d2d23cb86c3201f98572b58db7e32 Mon Sep 17 00:00:00 2001 From: wade zhang <95411902+gccgdb1234@users.noreply.github.com> Date: Thu, 10 Oct 2024 15:31:29 +0800 Subject: [PATCH 51/51] Update 05-cloud.md --- docs/zh/04-get-started/05-cloud.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/zh/04-get-started/05-cloud.md b/docs/zh/04-get-started/05-cloud.md index c8bd0f5dac..1bca09ee91 100644 --- a/docs/zh/04-get-started/05-cloud.md +++ b/docs/zh/04-get-started/05-cloud.md @@ -15,7 +15,7 @@ TDengine Cloud 大幅减轻了用户在部署、运维等方面的人力负担 要在 TDengine Cloud 注册新用户,请遵循以下简易步骤完成注册流程: -1. 打开浏览器,访问 TDengine Cloud[https://cloud.taosdata.com],在右边的“注册”部分,填入自己的姓名以及企业邮箱地址,点击“获取验证码”按钮。 +1. 打开浏览器,访问 [TDengine Cloud](https://cloud.taosdata.com),在右边的“注册”部分,填入自己的姓名以及企业邮箱地址,点击“获取验证码”按钮。 2. 检查企业邮箱,找到主题为“你的 TDengine Cloud 注册账户验证码”的邮件。从邮件内容中复制 6 位验证码,并将其粘贴到注册页面上的“验证码”输入框中。接着,点击“注册 TDengine Cloud”按钮,进入客户信息补全页面。 @@ -32,4 +32,4 @@ TDengine Cloud 大幅减轻了用户在部署、运维等方面的人力负担 3. 第 3 步,创建实例。在此步骤中,你需要填写实例的区域、名称、是否选择高可用选项以及计费方案等必填信息。确认无误后,点击“创建”按钮。大约等待 1min,新的TDengine 实例便会创建完成。随后,你可以在控制台中对该实例进行各种操作,如查询数据、创建订阅、创建流等。 -TDengine Cloud 提供多种级别的计费方案,包括入门版、基础版、标准版、专业版和旗舰版,以满足不同客户的需求。如果你觉得现有计费方案无法满足自己的特定需求,请联系 TDengine Cloud 的客户支持团队,他们将为你量身定制计费方案。注册后,你将获得一定的免费额度,以便体验服务 \ No newline at end of file +TDengine Cloud 提供多种级别的计费方案,包括入门版、基础版、标准版、专业版和旗舰版,以满足不同客户的需求。如果你觉得现有计费方案无法满足自己的特定需求,请联系 TDengine Cloud 的客户支持团队,他们将为你量身定制计费方案。注册后,你将获得一定的免费额度,以便体验服务