From fb5cd43fdcc2b2d37fb0b257ba2dff3a7b9ad47b Mon Sep 17 00:00:00 2001 From: wangmm0220 Date: Mon, 2 Sep 2024 17:55:16 +0800 Subject: [PATCH 01/13] 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/13] 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/13] 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/13] 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/13] 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/13] 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/13] 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/13] 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/13] 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/13] 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/13] 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/13] 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 65f268929358724634d1f72010e209954d65d7e1 Mon Sep 17 00:00:00 2001 From: wangmm0220 Date: Tue, 8 Oct 2024 13:48:22 +0800 Subject: [PATCH 13/13] 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);