From 1350af5267d57a9f4754dce505d90e99ce8bc401 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sun, 16 Apr 2023 22:48:22 +0800 Subject: [PATCH 01/20] fix(stream): set the correct initial checkpoint version to restore the operators state and add check for the initial destination tables. --- source/dnode/vnode/src/inc/tq.h | 1 + source/dnode/vnode/src/tq/tq.c | 35 ++++++++++++++----------- source/dnode/vnode/src/tq/tqRestore.c | 7 ++--- source/dnode/vnode/src/tq/tqUtil.c | 6 +++++ source/dnode/vnode/src/vnd/vnodeSync.c | 2 +- source/libs/executor/inc/executil.h | 1 + source/libs/executor/inc/executorimpl.h | 1 - source/libs/executor/src/executil.c | 11 +++++++- source/libs/executor/src/executor.c | 26 +++++++++++++++--- source/libs/executor/src/scanoperator.c | 1 - source/util/src/tworker.c | 4 +-- 11 files changed, 66 insertions(+), 29 deletions(-) diff --git a/source/dnode/vnode/src/inc/tq.h b/source/dnode/vnode/src/inc/tq.h index c007f84790..db17e4f533 100644 --- a/source/dnode/vnode/src/inc/tq.h +++ b/source/dnode/vnode/src/inc/tq.h @@ -179,6 +179,7 @@ int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t ver); int32_t tqStreamTasksScanWal(STQ* pTq); // tq util +char* createStreamTaskIdStr(int64_t streamId, int32_t taskId); void createStreamTaskOffsetKey(char* dst, uint64_t streamId, uint32_t taskId); int32_t tqAddInputBlockNLaunchTask(SStreamTask* pTask, SStreamQueueItem* pQueueItem, int64_t ver); int32_t launchTaskForWalBlock(SStreamTask* pTask, SFetchRet* pRet, STqOffset* pOffset); diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 1230a352d9..a641d44dba 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -20,6 +20,8 @@ // 2: wait to be inited or cleaup #define WAL_READ_TASKS_ID (-1) +static int32_t tqInitialize(STQ* pTq); + int32_t tqInit() { int8_t old; while (1) { @@ -109,25 +111,30 @@ STQ* tqOpen(const char* path, SVnode* pVnode) { pTq->pCheckInfo = taosHashInit(64, MurmurHash3_32, true, HASH_ENTRY_LOCK); taosHashSetFreeFp(pTq->pCheckInfo, (FDelete)tDeleteSTqCheckInfo); + tqInitialize(pVnode->pTq); + return pTq; +} + +int32_t tqInitialize(STQ* pTq) { if (tqMetaOpen(pTq) < 0) { - return NULL; + return -1; } pTq->pOffsetStore = tqOffsetOpen(pTq); if (pTq->pOffsetStore == NULL) { - return NULL; + return -1; } - pTq->pStreamMeta = streamMetaOpen(path, pTq, (FTaskExpand*)tqExpandTask, pTq->pVnode->config.vgId); + pTq->pStreamMeta = streamMetaOpen(pTq->path, pTq, (FTaskExpand*)tqExpandTask, pTq->pVnode->config.vgId); if (pTq->pStreamMeta == NULL) { - return NULL; + return -1; } - if (streamLoadTasks(pTq->pStreamMeta, walGetCommittedVer(pVnode->pWal)) < 0) { - return NULL; + if (streamLoadTasks(pTq->pStreamMeta, walGetCommittedVer(pTq->pVnode->pWal)) < 0) { + return -1; } - return pTq; + return 0; } void tqClose(STQ* pTq) { @@ -547,13 +554,9 @@ int32_t tqProcessSubscribeReq(STQ* pTq, int64_t sversion, char* msg, int32_t msg return 0; } -int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t ver) { - // todo extract method - char buf[128] = {0}; - sprintf(buf, "0x%"PRIx64"-%d", pTask->id.streamId, pTask->id.taskId); - +int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t UNUSED_PARAM(ver)) { int32_t vgId = TD_VID(pTq->pVnode); - pTask->id.idStr = taosStrdup(buf); + pTask->id.idStr = createStreamTaskIdStr(pTask->id.streamId, pTask->id.taskId); pTask->refCnt = 1; pTask->status.schedStatus = TASK_SCHED_STATUS__INACTIVE; pTask->inputQueue = streamQueueOpen(); @@ -567,7 +570,6 @@ int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t ver) { pTask->outputStatus = TASK_OUTPUT_STATUS__NORMAL; pTask->pMsgCb = &pTq->pVnode->msgCb; pTask->pMeta = pTq->pStreamMeta; - pTask->chkInfo.version = ver; // expand executor if (pTask->fillHistory) { @@ -633,8 +635,11 @@ int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t ver) { } streamSetupTrigger(pTask); - tqInfo("vgId:%d expand stream task, s-task:%s, ver:%" PRId64 " child id:%d, level:%d", vgId, pTask->id.idStr, + tqInfo("vgId:%d expand stream task, s-task:%s, checkpoint ver:%" PRId64 " child id:%d, level:%d", vgId, pTask->id.idStr, pTask->chkInfo.version, pTask->selfChildId, pTask->taskLevel); + + // next valid version will add one + pTask->chkInfo.version += 1; return 0; } diff --git a/source/dnode/vnode/src/tq/tqRestore.c b/source/dnode/vnode/src/tq/tqRestore.c index 6ed74ddcc3..cba51cdee4 100644 --- a/source/dnode/vnode/src/tq/tqRestore.c +++ b/source/dnode/vnode/src/tq/tqRestore.c @@ -52,9 +52,6 @@ int tqStreamTasksScanWal(STQ* pTq) { double el = (taosGetTimestampMs() - st) / 1000.0; tqInfo("vgId:%d scan wal for stream tasks completed, elapsed time:%.2f sec", vgId, el); - - // restore wal scan flag -// atomic_store_8(&pTq->pStreamMeta->walScan, 0); return 0; } @@ -99,8 +96,8 @@ int32_t streamTaskReplayWal(SStreamMeta* pStreamMeta, STqOffsetStore* pOffsetSto continue; } - if (pTask->status.taskStatus == TASK_STATUS__RECOVER_PREPARE || - pTask->status.taskStatus == TASK_STATUS__WAIT_DOWNSTREAM) { + int8_t status = pTask->status.taskStatus; + if (status == TASK_STATUS__RECOVER_PREPARE || status == TASK_STATUS__WAIT_DOWNSTREAM) { tqDebug("s-task:%s skip push data, not ready for processing, status %d", pTask->id.idStr, pTask->status.taskStatus); continue; diff --git a/source/dnode/vnode/src/tq/tqUtil.c b/source/dnode/vnode/src/tq/tqUtil.c index 4c37e1052f..00bff5da5d 100644 --- a/source/dnode/vnode/src/tq/tqUtil.c +++ b/source/dnode/vnode/src/tq/tqUtil.c @@ -19,6 +19,12 @@ static int32_t tqSendMetaPollRsp(STQ* pTq, const SRpcMsg* pMsg, const SMqPollReq* pReq, const SMqMetaRsp* pRsp); +char* createStreamTaskIdStr(int64_t streamId, int32_t taskId) { + char buf[128] = {0}; + sprintf(buf, "0x%" PRIx64 "-%d", streamId, taskId); + return taosStrdup(buf); +} + // stream_task:stream_id:task_id void createStreamTaskOffsetKey(char* dst, uint64_t streamId, uint32_t taskId) { int32_t n = 12; diff --git a/source/dnode/vnode/src/vnd/vnodeSync.c b/source/dnode/vnode/src/vnd/vnodeSync.c index d4a394b584..dc2d709d76 100644 --- a/source/dnode/vnode/src/vnd/vnodeSync.c +++ b/source/dnode/vnode/src/vnd/vnodeSync.c @@ -552,7 +552,7 @@ static void vnodeRestoreFinish(const SSyncFSM *pFsm, const SyncIndex commitIdx) walApplyVer(pVnode->pWal, commitIdx); pVnode->restored = true; - vInfo("vgId:%d, sync restore finished", pVnode->config.vgId); + vInfo("vgId:%d, sync restore finished, start to restore stream tasks by replay wal", pVnode->config.vgId); // start to restore all stream tasks tqStartStreamTasks(pVnode->pTq); diff --git a/source/libs/executor/inc/executil.h b/source/libs/executor/inc/executil.h index 9b8f034e44..c50fc86dfa 100644 --- a/source/libs/executor/inc/executil.h +++ b/source/libs/executor/inc/executil.h @@ -108,6 +108,7 @@ uint64_t tableListGetSize(const STableListInfo* pTableList); uint64_t tableListGetSuid(const STableListInfo* pTableList); STableKeyInfo* tableListGetInfo(const STableListInfo* pTableList, int32_t index); int32_t tableListFind(const STableListInfo* pTableList, uint64_t uid, int32_t startIndex); +void tableListGetSourceTableInfo(const STableListInfo* pTableList, uint64_t* psuid, int32_t* type); size_t getResultRowSize(struct SqlFunctionCtx* pCtx, int32_t numOfOutput); void initResultRowInfo(SResultRowInfo* pResultRowInfo); diff --git a/source/libs/executor/inc/executorimpl.h b/source/libs/executor/inc/executorimpl.h index 2cb6626b03..85424fd7de 100644 --- a/source/libs/executor/inc/executorimpl.h +++ b/source/libs/executor/inc/executorimpl.h @@ -427,7 +427,6 @@ typedef struct STimeWindowAggSupp { } STimeWindowAggSupp; typedef struct SStreamScanInfo { - uint64_t tableUid; // queried super table uid SExprInfo* pPseudoExpr; int32_t numOfPseudoExpr; SExprSupp tbnameCalSup; diff --git a/source/libs/executor/src/executil.c b/source/libs/executor/src/executil.c index 7d318786ba..33698522cd 100644 --- a/source/libs/executor/src/executil.c +++ b/source/libs/executor/src/executil.c @@ -36,6 +36,7 @@ struct STableListInfo { SArray* pTableList; SHashObj* map; // speedup acquire the tableQueryInfo by table uid uint64_t suid; + int32_t tableType; // queried table type }; typedef struct tagFilterAssist { @@ -1026,14 +1027,17 @@ int32_t getTableList(void* metaHandle, void* pVnode, SScanPhysiNode* pScanNode, size_t numOfTables = 0; pListInfo->suid = pScanNode->suid; + pListInfo->tableType = pScanNode->tableType; + SArray* pUidList = taosArrayInit(8, sizeof(uint64_t)); SIdxFltStatus status = SFLT_NOT_INDEX; if (pScanNode->tableType != TSDB_SUPER_TABLE) { + pListInfo->suid = pScanNode->uid; + if (metaIsTableExist(metaHandle, pScanNode->uid)) { taosArrayPush(pUidList, &pScanNode->uid); } - code = doFilterByTagCond(pListInfo, pUidList, pTagCond, metaHandle, status); if (code != TSDB_CODE_SUCCESS) { goto _end; @@ -1819,6 +1823,11 @@ int32_t tableListFind(const STableListInfo* pTableList, uint64_t uid, int32_t st return -1; } +void tableListGetSourceTableInfo(const STableListInfo* pTableList, uint64_t* psuid, int32_t* type) { + *psuid = pTableList->suid; + *type = pTableList->tableType; +} + uint64_t getTableGroupId(const STableListInfo* pTableList, uint64_t tableUid) { int32_t* slot = taosHashGet(pTableList->map, &tableUid, sizeof(tableUid)); ASSERT(pTableList->map != NULL && slot != NULL); diff --git a/source/libs/executor/src/executor.c b/source/libs/executor/src/executor.c index 6e3a7d8725..6b6ee931ba 100644 --- a/source/libs/executor/src/executor.c +++ b/source/libs/executor/src/executor.c @@ -14,6 +14,7 @@ */ #include "executor.h" +#include #include "executorimpl.h" #include "planner.h" #include "tdatablock.h" @@ -327,6 +328,13 @@ static SArray* filterUnqualifiedTables(const SStreamScanInfo* pScanInfo, const S return qa; } + STableScanInfo* pTableScanInfo = pScanInfo->pTableScanOp->info; + + uint64_t suid = 0; + int32_t type = 0; + tableListGetSourceTableInfo(pTableScanInfo->base.pTableListInfo, &suid, &type); + int32_t numOfExisted = tableListGetSize(pTableScanInfo->base.pTableListInfo); + // let's discard the tables those are not created according to the queried super table. SMetaReader mr = {0}; metaReaderInit(&mr, pScanInfo->readHandle.meta, 0); @@ -341,9 +349,21 @@ static SArray* filterUnqualifiedTables(const SStreamScanInfo* pScanInfo, const S tDecoderClear(&mr.coder); - // TODO handle ntb case - if (mr.me.type != TSDB_CHILD_TABLE || mr.me.ctbEntry.suid != pScanInfo->tableUid) { + if (mr.me.type == TSDB_SUPER_TABLE) { continue; + } else { + if (type == TSDB_SUPER_TABLE) { + // this new created child table does not belong to the scanned super table. + if (mr.me.type != TSDB_CHILD_TABLE || mr.me.ctbEntry.suid != suid) { + continue; + } + } else { // ordinary table + // In case that the scanned target table is an ordinary table. When replay the WAL during restore the vnode, we + // should check all newly created ordinary table to make sure that this table isn't the destination table. + if (mr.me.uid != suid) { + continue; + } + } } if (pScanInfo->pTagCond != NULL) { @@ -382,7 +402,7 @@ int32_t qUpdateTableListForStreamScanner(qTaskInfo_t tinfo, const SArray* tableI SStreamScanInfo* pScanInfo = pInfo->info; if (isAdd) { // add new table id - SArray* qa = filterUnqualifiedTables(pScanInfo, tableIdList, GET_TASKID(pTaskInfo)); + SArray* qa = filterUnqualifiedTables(pScanInfo, tableIdList, id); int32_t numOfQualifiedTables = taosArrayGetSize(qa); qDebug("%d qualified child tables added into stream scanner, %s", numOfQualifiedTables, id); code = tqReaderAddTbUidList(pScanInfo->tqReader, qa); diff --git a/source/libs/executor/src/scanoperator.c b/source/libs/executor/src/scanoperator.c index ae396a4c68..52c04aecd8 100644 --- a/source/libs/executor/src/scanoperator.c +++ b/source/libs/executor/src/scanoperator.c @@ -2441,7 +2441,6 @@ SOperatorInfo* createStreamScanOperatorInfo(SReadHandle* pHandle, STableScanPhys } pInfo->readHandle = *pHandle; - pInfo->tableUid = pScanPhyNode->uid; pTaskInfo->streamInfo.snapshotVer = pHandle->version; pInfo->pCreateTbRes = buildCreateTableBlock(&pInfo->tbnameCalSup, &pInfo->tagCalSup); blockDataEnsureCapacity(pInfo->pCreateTbRes, 8); diff --git a/source/util/src/tworker.c b/source/util/src/tworker.c index a49ff0cd5b..6edee27c05 100644 --- a/source/util/src/tworker.c +++ b/source/util/src/tworker.c @@ -217,8 +217,8 @@ STaosQueue *tAutoQWorkerAllocQueue(SAutoQWorkerPool *pool, void *ahandle, FItem int32_t queueNum = taosGetQueueNumber(pool->qset); int32_t curWorkerNum = taosArrayGetSize(pool->workers); - int32_t dstWorkerNum = ceil(queueNum * pool->ratio); - if (dstWorkerNum < 2) dstWorkerNum = 2; + int32_t dstWorkerNum = ceilf(queueNum * pool->ratio); + if (dstWorkerNum < 1) dstWorkerNum = 1; // spawn a thread to process queue while (curWorkerNum < dstWorkerNum) { From be90d2c511a3cd3756eea011cbc75f4b9fe3de6d Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sun, 16 Apr 2023 23:07:54 +0800 Subject: [PATCH 02/20] fix(stream): disable stream task when no tasks exist. --- include/libs/stream/tstream.h | 1 - source/dnode/vnode/src/tq/tq.c | 11 ++++++++--- 2 files changed, 8 insertions(+), 4 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 103f807191..9e0a2826c5 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -346,7 +346,6 @@ typedef struct SStreamMeta { int32_t vgId; SRWLatch lock; int8_t walScan; - bool quit; } SStreamMeta; int32_t tEncodeStreamEpInfo(SEncoder* pEncoder, const SStreamChildEpInfo* pInfo); diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index a641d44dba..190a0893a8 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -111,7 +111,7 @@ STQ* tqOpen(const char* path, SVnode* pVnode) { pTq->pCheckInfo = taosHashInit(64, MurmurHash3_32, true, HASH_ENTRY_LOCK); taosHashSetFreeFp(pTq->pCheckInfo, (FDelete)tDeleteSTqCheckInfo); - tqInitialize(pVnode->pTq); + tqInitialize(pTq); return pTq; } @@ -1281,6 +1281,13 @@ int32_t tqStartStreamTasks(STQ* pTq) { SStreamMeta* pMeta = pTq->pStreamMeta; taosWLockLatch(&pMeta->lock); + int32_t numOfTasks = taosHashGetSize(pTq->pStreamMeta->pTasks); + if (numOfTasks == 0) { + tqInfo("vgId:%d no stream tasks exists", vgId); + taosWUnLockLatch(&pTq->pStreamMeta->lock); + return 0; + } + pMeta->walScan += 1; if (pMeta->walScan > 1) { @@ -1297,8 +1304,6 @@ int32_t tqStartStreamTasks(STQ* pTq) { return -1; } - int32_t numOfTasks = taosHashGetSize(pTq->pStreamMeta->pTasks); - tqInfo("vgId:%d start wal scan stream tasks, tasks:%d", vgId, numOfTasks); initOffsetForAllRestoreTasks(pTq); From 10b3fd9426a945159c95f56aff165f35b2595069 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 17 Apr 2023 09:19:48 +0800 Subject: [PATCH 03/20] other: merge main. --- source/libs/stream/src/stream.c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/source/libs/stream/src/stream.c b/source/libs/stream/src/stream.c index 7171b52912..3b9306c2cf 100644 --- a/source/libs/stream/src/stream.c +++ b/source/libs/stream/src/stream.c @@ -16,7 +16,7 @@ #include "streamInc.h" #include "ttimer.h" -#define STREAM_TASK_INPUT_QUEUEU_CAPACITY 2000 +#define STREAM_TASK_INPUT_QUEUEU_CAPACITY 100000 int32_t streamInit() { int8_t old; From ac137b4b33502fba06715e5b0827b219a3a4c9ad Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 17 Apr 2023 09:46:58 +0800 Subject: [PATCH 04/20] refactor: do some internal refactor. --- include/libs/stream/tstream.h | 27 ++++++++++++--------------- source/libs/stream/src/stream.c | 4 ++++ 2 files changed, 16 insertions(+), 15 deletions(-) diff --git a/include/libs/stream/tstream.h b/include/libs/stream/tstream.h index 9e0a2826c5..aade34e965 100644 --- a/include/libs/stream/tstream.h +++ b/include/libs/stream/tstream.h @@ -335,17 +335,17 @@ struct SStreamTask { // meta typedef struct SStreamMeta { - char* path; - TDB* db; - TTB* pTaskDb; - TTB* pCheckpointDb; - SHashObj* pTasks; - void* ahandle; - TXN* txn; - FTaskExpand* expandFunc; - int32_t vgId; - SRWLatch lock; - int8_t walScan; + char* path; + TDB* db; + TTB* pTaskDb; + TTB* pCheckpointDb; + SHashObj* pTasks; + void* ahandle; + TXN* txn; + FTaskExpand* expandFunc; + int32_t vgId; + SRWLatch lock; + int8_t walScan; } SStreamMeta; int32_t tEncodeStreamEpInfo(SEncoder* pEncoder, const SStreamChildEpInfo* pInfo); @@ -358,10 +358,6 @@ void tFreeStreamTask(SStreamTask* pTask); int32_t tAppendDataToInputQueue(SStreamTask* pTask, SStreamQueueItem* pItem); bool tInputQueueIsFull(const SStreamTask* pTask); -static FORCE_INLINE void streamTaskInputFail(SStreamTask* pTask) { - atomic_store_8(&pTask->inputStatus, TASK_INPUT_STATUS__FAILED); -} - typedef struct { SMsgHead head; int64_t streamId; @@ -537,6 +533,7 @@ int32_t streamProcessDispatchRsp(SStreamTask* pTask, SStreamDispatchRsp* pRsp, i int32_t streamProcessRetrieveReq(SStreamTask* pTask, SStreamRetrieveReq* pReq, SRpcMsg* pMsg); // int32_t streamProcessRetrieveRsp(SStreamTask* pTask, SStreamRetrieveRsp* pRsp); +void streamTaskInputFail(SStreamTask* pTask); int32_t streamTryExec(SStreamTask* pTask); int32_t streamSchedExec(SStreamTask* pTask); int32_t streamTaskOutput(SStreamTask* pTask, SStreamDataBlock* pBlock); diff --git a/source/libs/stream/src/stream.c b/source/libs/stream/src/stream.c index 3b9306c2cf..0f000f1f50 100644 --- a/source/libs/stream/src/stream.c +++ b/source/libs/stream/src/stream.c @@ -352,4 +352,8 @@ void* streamQueueNextItem(SStreamQueue* queue) { } return streamQueueCurItem(queue); } +} + +void streamTaskInputFail(SStreamTask* pTask) { + atomic_store_8(&pTask->inputStatus, TASK_INPUT_STATUS__FAILED); } \ No newline at end of file From eb7f510ccbfcbe3e92ca90c70a2c91e004642c4a Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 17 Apr 2023 10:59:24 +0800 Subject: [PATCH 05/20] fix(query): return correct suid to delete sink. --- include/libs/executor/dataSinkMgt.h | 1 - source/libs/executor/src/executil.c | 18 +++++++++++++----- source/libs/executor/src/executor.c | 9 ++++----- source/libs/executor/src/scanoperator.c | 2 +- 4 files changed, 18 insertions(+), 12 deletions(-) diff --git a/include/libs/executor/dataSinkMgt.h b/include/libs/executor/dataSinkMgt.h index ed7cbc8125..ce7d038d42 100644 --- a/include/libs/executor/dataSinkMgt.h +++ b/include/libs/executor/dataSinkMgt.h @@ -29,7 +29,6 @@ extern "C" { #define DS_BUF_FULL 2 #define DS_BUF_EMPTY 3 -struct SDataSink; struct SSDataBlock; typedef struct SDeleterRes { diff --git a/source/libs/executor/src/executil.c b/source/libs/executor/src/executil.c index 33698522cd..af68cd5990 100644 --- a/source/libs/executor/src/executil.c +++ b/source/libs/executor/src/executil.c @@ -35,8 +35,11 @@ struct STableListInfo { int32_t* groupOffset; // keep the offset value for each group in the tableList SArray* pTableList; SHashObj* map; // speedup acquire the tableQueryInfo by table uid - uint64_t suid; - int32_t tableType; // queried table type + union { + uint64_t suid; + uint64_t uid; + }; // this maybe the super table or ordinary table + int32_t tableType; // queried table type }; typedef struct tagFilterAssist { @@ -1033,8 +1036,7 @@ int32_t getTableList(void* metaHandle, void* pVnode, SScanPhysiNode* pScanNode, SIdxFltStatus status = SFLT_NOT_INDEX; if (pScanNode->tableType != TSDB_SUPER_TABLE) { - pListInfo->suid = pScanNode->uid; - + pListInfo->uid = pScanNode->uid; if (metaIsTableExist(metaHandle, pScanNode->uid)) { taosArrayPush(pUidList, &pScanNode->uid); } @@ -1798,7 +1800,13 @@ uint64_t tableListGetSize(const STableListInfo* pTableList) { return taosArrayGetSize(pTableList->pTableList); } -uint64_t tableListGetSuid(const STableListInfo* pTableList) { return pTableList->suid; } +uint64_t tableListGetSuid(const STableListInfo* pTableList) { + if (pTableList->tableType == TSDB_SUPER_TABLE) { + return pTableList->suid; + } else { // query normal table, no suid exists. + return 0; + } +} STableKeyInfo* tableListGetInfo(const STableListInfo* pTableList, int32_t index) { if (taosArrayGetSize(pTableList->pTableList) == 0) { diff --git a/source/libs/executor/src/executor.c b/source/libs/executor/src/executor.c index 6b6ee931ba..33697a4033 100644 --- a/source/libs/executor/src/executor.c +++ b/source/libs/executor/src/executor.c @@ -330,10 +330,9 @@ static SArray* filterUnqualifiedTables(const SStreamScanInfo* pScanInfo, const S STableScanInfo* pTableScanInfo = pScanInfo->pTableScanOp->info; - uint64_t suid = 0; + uint64_t uid = 0; int32_t type = 0; - tableListGetSourceTableInfo(pTableScanInfo->base.pTableListInfo, &suid, &type); - int32_t numOfExisted = tableListGetSize(pTableScanInfo->base.pTableListInfo); + tableListGetSourceTableInfo(pTableScanInfo->base.pTableListInfo, &uid, &type); // let's discard the tables those are not created according to the queried super table. SMetaReader mr = {0}; @@ -354,13 +353,13 @@ static SArray* filterUnqualifiedTables(const SStreamScanInfo* pScanInfo, const S } else { if (type == TSDB_SUPER_TABLE) { // this new created child table does not belong to the scanned super table. - if (mr.me.type != TSDB_CHILD_TABLE || mr.me.ctbEntry.suid != suid) { + if (mr.me.type != TSDB_CHILD_TABLE || mr.me.ctbEntry.suid != uid) { continue; } } else { // ordinary table // In case that the scanned target table is an ordinary table. When replay the WAL during restore the vnode, we // should check all newly created ordinary table to make sure that this table isn't the destination table. - if (mr.me.uid != suid) { + if (mr.me.uid != uid) { continue; } } diff --git a/source/libs/executor/src/scanoperator.c b/source/libs/executor/src/scanoperator.c index 52c04aecd8..2389c7252e 100644 --- a/source/libs/executor/src/scanoperator.c +++ b/source/libs/executor/src/scanoperator.c @@ -690,7 +690,7 @@ static SSDataBlock* doTableScanImpl(SOperatorInfo* pOperator) { } uint32_t status = 0; - int32_t code = loadDataBlock(pOperator, &pTableScanInfo->base, pBlock, &status); + code = loadDataBlock(pOperator, &pTableScanInfo->base, pBlock, &status); if (code != TSDB_CODE_SUCCESS) { T_LONG_JMP(pTaskInfo->env, code); } From 4ed26bbc1998a4b62a0f3316defe9a82703b9afc Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 17 Apr 2023 14:08:54 +0800 Subject: [PATCH 06/20] fix(stream): update the table list api. --- source/libs/executor/inc/executil.h | 2 +- source/libs/executor/src/executil.c | 51 ++++++++++++++--------------- source/libs/executor/src/executor.c | 5 +-- 3 files changed, 29 insertions(+), 29 deletions(-) diff --git a/source/libs/executor/inc/executil.h b/source/libs/executor/inc/executil.h index c50fc86dfa..2e92f9e396 100644 --- a/source/libs/executor/inc/executil.h +++ b/source/libs/executor/inc/executil.h @@ -108,7 +108,7 @@ uint64_t tableListGetSize(const STableListInfo* pTableList); uint64_t tableListGetSuid(const STableListInfo* pTableList); STableKeyInfo* tableListGetInfo(const STableListInfo* pTableList, int32_t index); int32_t tableListFind(const STableListInfo* pTableList, uint64_t uid, int32_t startIndex); -void tableListGetSourceTableInfo(const STableListInfo* pTableList, uint64_t* psuid, int32_t* type); +void tableListGetSourceTableInfo(const STableListInfo* pTableList, uint64_t* psuid, uint64_t* uid, int32_t* type); size_t getResultRowSize(struct SqlFunctionCtx* pCtx, int32_t numOfOutput); void initResultRowInfo(SResultRowInfo* pResultRowInfo); diff --git a/source/libs/executor/src/executil.c b/source/libs/executor/src/executil.c index af68cd5990..f61fd1ae01 100644 --- a/source/libs/executor/src/executil.c +++ b/source/libs/executor/src/executil.c @@ -27,19 +27,21 @@ #include "executorimpl.h" #include "tcompression.h" +typedef struct STableListIdInfo { + uint64_t suid; + uint64_t uid; + int32_t tableType; +} STableListIdInfo; + // If the numOfOutputGroups is 1, the data blocks that belongs to different groups will be provided randomly // The numOfOutputGroups is specified by physical plan. and will not be affect by numOfGroups struct STableListInfo { - bool oneTableForEachGroup; - int32_t numOfOuputGroups; // the data block will be generated one by one - int32_t* groupOffset; // keep the offset value for each group in the tableList - SArray* pTableList; - SHashObj* map; // speedup acquire the tableQueryInfo by table uid - union { - uint64_t suid; - uint64_t uid; - }; // this maybe the super table or ordinary table - int32_t tableType; // queried table type + bool oneTableForEachGroup; + int32_t numOfOuputGroups; // the data block will be generated one by one + int32_t* groupOffset; // keep the offset value for each group in the tableList + SArray* pTableList; + SHashObj* map; // speedup acquire the tableQueryInfo by table uid + STableListIdInfo idInfo; // this maybe the super table or ordinary table }; typedef struct tagFilterAssist { @@ -474,7 +476,7 @@ int32_t getColInfoResultForGroupby(void* metaHandle, SNodeList* group, STableLis } // int64_t stt = taosGetTimestampUs(); - code = metaGetTableTags(metaHandle, pTableListInfo->suid, pUidTagList); + code = metaGetTableTags(metaHandle, pTableListInfo->idInfo.suid, pUidTagList); if (code != TSDB_CODE_SUCCESS) { goto end; } @@ -957,7 +959,7 @@ static int32_t doFilterByTagCond(STableListInfo* pListInfo, SArray* pUidList, SN FilterCondType condType = checkTagCond(pTagCond); - int32_t filter = optimizeTbnameInCond(metaHandle, pListInfo->suid, pUidTagList, pTagCond); + int32_t filter = optimizeTbnameInCond(metaHandle, pListInfo->idInfo.suid, pUidTagList, pTagCond); if (filter == 0) { // tbname in filter is activated, do nothing and return taosArrayClear(pUidList); @@ -970,12 +972,12 @@ static int32_t doFilterByTagCond(STableListInfo* pListInfo, SArray* pUidList, SN terrno = 0; } else { if ((condType == FILTER_NO_LOGIC || condType == FILTER_AND) && status != SFLT_NOT_INDEX) { - code = metaGetTableTagsByUids(metaHandle, pListInfo->suid, pUidTagList); + code = metaGetTableTagsByUids(metaHandle, pListInfo->idInfo.suid, pUidTagList); } else { - code = metaGetTableTags(metaHandle, pListInfo->suid, pUidTagList); + code = metaGetTableTags(metaHandle, pListInfo->idInfo.suid, pUidTagList); } if (code != TSDB_CODE_SUCCESS) { - qError("failed to get table tags from meta, reason:%s, suid:%" PRIu64, tstrerror(code), pListInfo->suid); + qError("failed to get table tags from meta, reason:%s, suid:%" PRIu64, tstrerror(code), pListInfo->idInfo.suid); terrno = code; goto end; } @@ -1029,14 +1031,14 @@ int32_t getTableList(void* metaHandle, void* pVnode, SScanPhysiNode* pScanNode, int32_t code = TSDB_CODE_SUCCESS; size_t numOfTables = 0; - pListInfo->suid = pScanNode->suid; - pListInfo->tableType = pScanNode->tableType; + pListInfo->idInfo.suid = pScanNode->suid; + pListInfo->idInfo.tableType = pScanNode->tableType; SArray* pUidList = taosArrayInit(8, sizeof(uint64_t)); SIdxFltStatus status = SFLT_NOT_INDEX; if (pScanNode->tableType != TSDB_SUPER_TABLE) { - pListInfo->uid = pScanNode->uid; + pListInfo->idInfo.uid = pScanNode->uid; if (metaIsTableExist(metaHandle, pScanNode->uid)) { taosArrayPush(pUidList, &pScanNode->uid); } @@ -1801,11 +1803,7 @@ uint64_t tableListGetSize(const STableListInfo* pTableList) { } uint64_t tableListGetSuid(const STableListInfo* pTableList) { - if (pTableList->tableType == TSDB_SUPER_TABLE) { - return pTableList->suid; - } else { // query normal table, no suid exists. - return 0; - } + return pTableList->idInfo.suid; } STableKeyInfo* tableListGetInfo(const STableListInfo* pTableList, int32_t index) { @@ -1831,9 +1829,10 @@ int32_t tableListFind(const STableListInfo* pTableList, uint64_t uid, int32_t st return -1; } -void tableListGetSourceTableInfo(const STableListInfo* pTableList, uint64_t* psuid, int32_t* type) { - *psuid = pTableList->suid; - *type = pTableList->tableType; +void tableListGetSourceTableInfo(const STableListInfo* pTableList, uint64_t* psuid, uint64_t* uid, int32_t* type) { + *psuid = pTableList->idInfo.suid; + *uid = pTableList->idInfo.uid; + *type = pTableList->idInfo.tableType; } uint64_t getTableGroupId(const STableListInfo* pTableList, uint64_t tableUid) { diff --git a/source/libs/executor/src/executor.c b/source/libs/executor/src/executor.c index 33697a4033..181ab3d44c 100644 --- a/source/libs/executor/src/executor.c +++ b/source/libs/executor/src/executor.c @@ -330,9 +330,10 @@ static SArray* filterUnqualifiedTables(const SStreamScanInfo* pScanInfo, const S STableScanInfo* pTableScanInfo = pScanInfo->pTableScanOp->info; + uint64_t suid = 0; uint64_t uid = 0; int32_t type = 0; - tableListGetSourceTableInfo(pTableScanInfo->base.pTableListInfo, &uid, &type); + tableListGetSourceTableInfo(pTableScanInfo->base.pTableListInfo, &suid, &uid, &type); // let's discard the tables those are not created according to the queried super table. SMetaReader mr = {0}; @@ -353,7 +354,7 @@ static SArray* filterUnqualifiedTables(const SStreamScanInfo* pScanInfo, const S } else { if (type == TSDB_SUPER_TABLE) { // this new created child table does not belong to the scanned super table. - if (mr.me.type != TSDB_CHILD_TABLE || mr.me.ctbEntry.suid != uid) { + if (mr.me.type != TSDB_CHILD_TABLE || mr.me.ctbEntry.suid != suid) { continue; } } else { // ordinary table From 930b267a75fe975a88b75af01f8f676176eecd38 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 17 Apr 2023 22:59:00 +0800 Subject: [PATCH 07/20] fix(stream): set the correct start offset for stream task. --- source/libs/stream/src/streamDispatch.c | 12 +++++------- source/libs/stream/src/streamExec.c | 5 ++++- source/libs/stream/src/streamMeta.c | 3 +-- source/libs/stream/src/streamRecover.c | 8 ++++++-- 4 files changed, 16 insertions(+), 12 deletions(-) diff --git a/source/libs/stream/src/streamDispatch.c b/source/libs/stream/src/streamDispatch.c index a9f6d29bf5..549374ed94 100644 --- a/source/libs/stream/src/streamDispatch.c +++ b/source/libs/stream/src/streamDispatch.c @@ -230,23 +230,21 @@ int32_t streamDispatchOneCheckReq(SStreamTask* pTask, const SStreamTaskCheckReq* SEncoder encoder; tEncoderInit(&encoder, abuf, tlen); if ((code = tEncodeSStreamTaskCheckReq(&encoder, pReq)) < 0) { - goto FAIL; + rpcFreeCont(buf); + return code; } + tEncoderClear(&encoder); msg.contLen = tlen + sizeof(SMsgHead); msg.pCont = buf; msg.msgType = TDMT_STREAM_TASK_CHECK; - qDebug("dispatch from s-task:%s to downstream s-task:%"PRIx64":%d node %d: check msg", pTask->id.idStr, - pReq->streamId, pReq->downstreamTaskId, nodeId); + qDebug("dispatch from s-task:%s to downstream s-task:%" PRIx64 ":%d node %d: check msg", pTask->id.idStr, + pReq->streamId, pReq->downstreamTaskId, nodeId); tmsgSendReq(pEpSet, &msg); - return 0; -FAIL: - if (buf) rpcFreeCont(buf); - return code; } int32_t streamDispatchOneRecoverFinishReq(SStreamTask* pTask, const SStreamRecoverFinishReq* pReq, int32_t vgId, diff --git a/source/libs/stream/src/streamExec.c b/source/libs/stream/src/streamExec.c index 3d896c08ac..9a6ff302ef 100644 --- a/source/libs/stream/src/streamExec.c +++ b/source/libs/stream/src/streamExec.c @@ -189,7 +189,10 @@ int32_t streamScanExec(SStreamTask* pTask, int32_t batchSz) { qDebug("task %d scan exec dispatch block num %d", pTask->id.taskId, batchCnt); streamDispatch(pTask); } - if (finished) break; + + if (finished) { + break; + } } return 0; } diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 51cc315780..860514bb44 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -296,8 +296,7 @@ int32_t streamLoadTasks(SStreamMeta* pMeta, int64_t ver) { tDecodeStreamTask(&decoder, pTask); tDecoderClear(&decoder); - // todo set correct initial version. - if (pMeta->expandFunc(pMeta->ahandle, pTask, 0) < 0) { + if (pMeta->expandFunc(pMeta->ahandle, pTask, pTask->chkInfo.version) < 0) { tdbFree(pKey); tdbFree(pVal); tdbTbcClose(pCur); diff --git a/source/libs/stream/src/streamRecover.c b/source/libs/stream/src/streamRecover.c index 03afc0692d..55c745e417 100644 --- a/source/libs/stream/src/streamRecover.c +++ b/source/libs/stream/src/streamRecover.c @@ -102,8 +102,10 @@ int32_t streamRecheckOneDownstream(SStreamTask* pTask, const SStreamTaskCheckRsp .downstreamNodeId = pRsp->downstreamNodeId, .childId = pRsp->childId, }; - qDebug("task %d at node %d check downstream task %d at node %d (recheck)", pTask->id.taskId, pTask->nodeId, + + qDebug("s-task:%s at node %d check downstream task %d at node %d (recheck)", pTask->id.idStr, pTask->nodeId, req.downstreamTaskId, req.downstreamNodeId); + if (pTask->outputType == TASK_OUTPUT__FIXED_DISPATCH) { streamDispatchOneCheckReq(pTask, &req, pRsp->downstreamNodeId, &pTask->fixedEpDispatcher.epSet); } else if (pTask->outputType == TASK_OUTPUT__SHUFFLE_DISPATCH) { @@ -116,6 +118,7 @@ int32_t streamRecheckOneDownstream(SStreamTask* pTask, const SStreamTaskCheckRsp } } } + return 0; } @@ -158,9 +161,10 @@ int32_t streamProcessTaskCheckRsp(SStreamTask* pTask, const SStreamTaskCheckRsp* } else { ASSERT(0); } - } else { + } else { // not ready, it should wait for at least 100ms and then retry streamRecheckOneDownstream(pTask, pRsp); } + return 0; } From 2ae05b8cf6e962a6a709678e54781ba8429666b4 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 18 Apr 2023 08:56:31 +0800 Subject: [PATCH 08/20] fix(stream): set the correct initial offset value. --- source/dnode/vnode/src/tq/tq.c | 7 +++++-- tests/system-test/1-insert/delete_childtable.py | 2 +- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 190a0893a8..7c7f59b6b7 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -130,6 +130,8 @@ int32_t tqInitialize(STQ* pTq) { return -1; } + // the version is kept in task's meta data + // todo check if this version is required or not if (streamLoadTasks(pTq->pStreamMeta, walGetCommittedVer(pTq->pVnode->pWal)) < 0) { return -1; } @@ -554,7 +556,7 @@ int32_t tqProcessSubscribeReq(STQ* pTq, int64_t sversion, char* msg, int32_t msg return 0; } -int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t UNUSED_PARAM(ver)) { +int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t ver) { int32_t vgId = TD_VID(pTq->pVnode); pTask->id.idStr = createStreamTaskIdStr(pTask->id.streamId, pTask->id.taskId); pTask->refCnt = 1; @@ -570,6 +572,7 @@ int32_t tqExpandTask(STQ* pTq, SStreamTask* pTask, int64_t UNUSED_PARAM(ver)) { pTask->outputStatus = TASK_OUTPUT_STATUS__NORMAL; pTask->pMsgCb = &pTq->pVnode->msgCb; pTask->pMeta = pTq->pStreamMeta; + pTask->chkInfo.version = ver; // expand executor if (pTask->fillHistory) { @@ -755,7 +758,7 @@ int32_t tqProcessTaskDeployReq(STQ* pTq, int64_t sversion, char* msg, int32_t ms tDecoderClear(&decoder); - // 2.save task + // 2.save task, use the newest commit version as the initial start version of stream task. code = streamMetaAddDeployedTask(pTq->pStreamMeta, sversion, pTask); if (code < 0) { tqError("vgId:%d failed to add s-task:%s, total:%d", TD_VID(pTq->pVnode), pTask->id.idStr, diff --git a/tests/system-test/1-insert/delete_childtable.py b/tests/system-test/1-insert/delete_childtable.py index e3144edb45..a12f884981 100644 --- a/tests/system-test/1-insert/delete_childtable.py +++ b/tests/system-test/1-insert/delete_childtable.py @@ -27,7 +27,7 @@ class TDTestCase: def init(self, conn, logSql, replicaVar=1): self.replicaVar = int(replicaVar) tdLog.debug("start to execute %s" % __file__) - tdSql.init(conn.cursor()) + tdSql.init(conn.cursor(), True) self.dbname = 'db_test' self.setsql = TDSetSql() self.stbname = 'stb' From 7d3e5aa3d389cf4ac165f0eacd6fb8115e58c679 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 18 Apr 2023 09:03:30 +0800 Subject: [PATCH 09/20] fix(query): set the table schema correctly when the table is dropped. --- source/dnode/vnode/src/tsdb/tsdbRead.c | 50 ++++++++++++++++---------- 1 file changed, 32 insertions(+), 18 deletions(-) diff --git a/source/dnode/vnode/src/tsdb/tsdbRead.c b/source/dnode/vnode/src/tsdb/tsdbRead.c index 89686c3d33..15a605151c 100644 --- a/source/dnode/vnode/src/tsdb/tsdbRead.c +++ b/source/dnode/vnode/src/tsdb/tsdbRead.c @@ -1881,8 +1881,8 @@ static FORCE_INLINE STSchema* getLatestTableSchema(STsdbReader* pReader, uint64_ return pReader->pSchema; } - pReader->pSchema = metaGetTbTSchema(pReader->pTsdb->pVnode->pMeta, uid, -1, 1); - if (pReader->pSchema == NULL) { + int32_t code = metaGetTbTSchemaEx(pReader->pTsdb->pVnode->pMeta, uid, -1, 1, &pReader->pSchema); + if (code != TSDB_CODE_SUCCESS || pReader->pSchema == NULL) { tsdbError("failed to get table schema, uid:%" PRIu64 ", it may have been dropped, ver:-1, %s", uid, pReader->idStr); } @@ -1890,9 +1890,15 @@ static FORCE_INLINE STSchema* getLatestTableSchema(STsdbReader* pReader, uint64_ } static FORCE_INLINE STSchema* doGetSchemaForTSRow(int32_t sversion, STsdbReader* pReader, uint64_t uid) { + int32_t code = 0; + // always set the newest schema version in pReader->pSchema if (pReader->pSchema == NULL) { - pReader->pSchema = metaGetTbTSchema(pReader->pTsdb->pVnode->pMeta, uid, -1, 1); + code = metaGetTbTSchemaEx(pReader->pTsdb->pVnode->pMeta, uid, -1, 1, &pReader->pSchema); + if (code != TSDB_CODE_SUCCESS) { + terrno = code; + return NULL; + } } if (pReader->pSchema && sversion == pReader->pSchema->version) { @@ -1905,7 +1911,7 @@ static FORCE_INLINE STSchema* doGetSchemaForTSRow(int32_t sversion, STsdbReader* } STSchema* ptr = NULL; - int32_t code = metaGetTbTSchemaEx(pReader->pTsdb->pVnode->pMeta, pReader->suid, uid, sversion, &ptr); + code = metaGetTbTSchemaEx(pReader->pTsdb->pVnode->pMeta, pReader->suid, uid, sversion, &ptr); if (code != TSDB_CODE_SUCCESS) { terrno = code; return NULL; @@ -2014,6 +2020,10 @@ static int32_t doMergeBufAndFileRows(STsdbReader* pReader, STableBlockScanInfo* if (minKey == k.ts) { init = true; STSchema* pSchema = doGetSchemaForTSRow(TSDBROW_SVERSION(pRow), pReader, pBlockScanInfo->uid); + if (pSchema == NULL) { + return terrno; + } + int32_t code = tsdbRowMergerInit(&merge, NULL, pRow, pSchema); if (code != TSDB_CODE_SUCCESS) { return code; @@ -2222,6 +2232,7 @@ static int32_t doMergeMultiLevelRows(STsdbReader* pReader, STableBlockScanInfo* if (pSchema == NULL) { return code; } + STSchema* piSchema = doGetSchemaForTSRow(TSDBROW_SVERSION(piRow), pReader, pBlockScanInfo->uid); if (piSchema == NULL) { return code; @@ -3843,11 +3854,8 @@ int32_t doMergeMemTableMultiRows(TSDBROW* pRow, uint64_t uid, SIterInfo* pIter, return terrno; } - if (pReader->pSchema == NULL) { - pReader->pSchema = pTSchema; - } - - code = tsdbRowMergerInit(&merge, pReader->pSchema, ¤t, pTSchema); + STSchema* ps = (pReader->pSchema != NULL)? pReader->pSchema:pTSchema; + code = tsdbRowMergerInit(&merge, ps, ¤t, pTSchema); if (code != TSDB_CODE_SUCCESS) { return code; } @@ -3891,7 +3899,14 @@ int32_t doMergeMemIMemRows(TSDBROW* pRow, TSDBROW* piRow, STableBlockScanInfo* p TSDBKEY k = TSDBROW_KEY(pRow); TSDBKEY ik = TSDBROW_KEY(piRow); STSchema* pSchema = doGetSchemaForTSRow(TSDBROW_SVERSION(pRow), pReader, pBlockScanInfo->uid); + if (pSchema == NULL) { + return terrno; + } + STSchema* piSchema = doGetSchemaForTSRow(TSDBROW_SVERSION(piRow), pReader, pBlockScanInfo->uid); + if (piSchema == NULL) { + return terrno; + } if (ASCENDING_TRAVERSE(pReader->order)) { // ascending order imem --> mem int32_t code = tsdbRowMergerInit(&merge, pSchema, piRow, piSchema); @@ -4000,10 +4015,11 @@ int32_t doAppendRowFromTSRow(SSDataBlock* pBlock, STsdbReader* pReader, SRow* pT int64_t uid = pScanInfo->uid; int32_t code = TSDB_CODE_SUCCESS; - int32_t numOfCols = (int32_t)taosArrayGetSize(pBlock->pDataBlock); - SBlockLoadSuppInfo* pSupInfo = &pReader->suppInfo; STSchema* pSchema = doGetSchemaForTSRow(pTSRow->sver, pReader, uid); + if (pSchema == NULL) { + return terrno; + } SColVal colVal = {0}; int32_t i = 0, j = 0; @@ -5187,8 +5203,6 @@ int64_t tsdbGetNumOfRowsInMemTable(STsdbReader* pReader) { } int32_t tsdbGetTableSchema(SVnode* pVnode, int64_t uid, STSchema** pSchema, int64_t* suid) { - int32_t sversion = 1; - SMetaReader mr = {0}; metaReaderInit(&mr, pVnode->pMeta, 0); int32_t code = metaGetTableEntryByUidCache(&mr, uid); @@ -5200,6 +5214,7 @@ int32_t tsdbGetTableSchema(SVnode* pVnode, int64_t uid, STSchema** pSchema, int6 *suid = 0; + // only child table and ordinary table is allowed, super table is not allowed. if (mr.me.type == TSDB_CHILD_TABLE) { tDecoderClear(&mr.coder); *suid = mr.me.ctbEntry.suid; @@ -5209,9 +5224,7 @@ int32_t tsdbGetTableSchema(SVnode* pVnode, int64_t uid, STSchema** pSchema, int6 metaReaderClear(&mr); return terrno; } - sversion = mr.me.stbEntry.schemaRow.version; - } else if (mr.me.type == TSDB_NORMAL_TABLE) { - sversion = mr.me.ntbEntry.schemaRow.version; + } else if (mr.me.type == TSDB_NORMAL_TABLE) { // do nothing } else { terrno = TSDB_CODE_INVALID_PARA; metaReaderClear(&mr); @@ -5219,9 +5232,10 @@ int32_t tsdbGetTableSchema(SVnode* pVnode, int64_t uid, STSchema** pSchema, int6 } metaReaderClear(&mr); - *pSchema = metaGetTbTSchema(pVnode->pMeta, uid, sversion, 1); - return TSDB_CODE_SUCCESS; + // get the newest table schema version + code = metaGetTbTSchemaEx(pVnode->pMeta, uid, -1, 1, pSchema); + return code; } int32_t tsdbTakeReadSnap(STsdbReader* pReader, _query_reseek_func_t reseek, STsdbReadSnap** ppSnap) { From 97da34050b9300d1f19242ade11bc8a5dc7dbbaf Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 18 Apr 2023 10:40:53 +0800 Subject: [PATCH 10/20] fix(query): fix invalid free. --- source/dnode/vnode/src/tq/tqUtil.c | 17 +++++++++++++++++ source/dnode/vnode/src/tsdb/tsdbRead.c | 8 ++++---- source/libs/executor/src/dataDeleter.c | 1 + source/libs/executor/src/dataInserter.c | 1 + source/libs/executor/src/executor.c | 4 +--- 5 files changed, 24 insertions(+), 7 deletions(-) diff --git a/source/dnode/vnode/src/tq/tqUtil.c b/source/dnode/vnode/src/tq/tqUtil.c index 00bff5da5d..5ac747947f 100644 --- a/source/dnode/vnode/src/tq/tqUtil.c +++ b/source/dnode/vnode/src/tq/tqUtil.c @@ -150,6 +150,21 @@ static int32_t tqInitTaosxRsp(STaosxRsp* pRsp, const SMqPollReq* pReq) { pRsp->blockSchema = taosArrayInit(0, sizeof(void*)); if (pRsp->blockData == NULL || pRsp->blockDataLen == NULL || pRsp->blockTbName == NULL || pRsp->blockSchema == NULL) { + if (pRsp->blockData != NULL) { + pRsp->blockData = taosArrayDestroy(pRsp->blockData); + } + + if (pRsp->blockDataLen != NULL) { + pRsp->blockDataLen = taosArrayDestroy(pRsp->blockDataLen); + } + + if (pRsp->blockTbName != NULL) { + pRsp->blockTbName = taosArrayDestroy(pRsp->blockTbName); + } + + if (pRsp->blockSchema != NULL) { + pRsp->blockSchema = taosArrayDestroy(pRsp->blockSchema); + } return -1; } @@ -283,6 +298,7 @@ static int32_t extractDataAndRspForDbStbSubscribe(STQ* pTq, STqHandle* pHandle, if (offset->type != TMQ_OFFSET__LOG) { if (tqScanTaosx(pTq, pHandle, &taosxRsp, &metaRsp, offset) < 0) { + tDeleteSTaosxRsp(&taosxRsp); return -1; } @@ -358,6 +374,7 @@ static int32_t extractDataAndRspForDbStbSubscribe(STQ* pTq, STqHandle* pHandle, tDeleteSTaosxRsp(&taosxRsp); return code; } + code = 0; taosMemoryFreeClear(pCkHead); tDeleteSTaosxRsp(&taosxRsp); diff --git a/source/dnode/vnode/src/tsdb/tsdbRead.c b/source/dnode/vnode/src/tsdb/tsdbRead.c index 15a605151c..362dc51ad5 100644 --- a/source/dnode/vnode/src/tsdb/tsdbRead.c +++ b/source/dnode/vnode/src/tsdb/tsdbRead.c @@ -1881,7 +1881,7 @@ static FORCE_INLINE STSchema* getLatestTableSchema(STsdbReader* pReader, uint64_ return pReader->pSchema; } - int32_t code = metaGetTbTSchemaEx(pReader->pTsdb->pVnode->pMeta, uid, -1, 1, &pReader->pSchema); + int32_t code = metaGetTbTSchemaEx(pReader->pTsdb->pVnode->pMeta, pReader->suid, uid, -1, &pReader->pSchema); if (code != TSDB_CODE_SUCCESS || pReader->pSchema == NULL) { tsdbError("failed to get table schema, uid:%" PRIu64 ", it may have been dropped, ver:-1, %s", uid, pReader->idStr); } @@ -1894,7 +1894,7 @@ static FORCE_INLINE STSchema* doGetSchemaForTSRow(int32_t sversion, STsdbReader* // always set the newest schema version in pReader->pSchema if (pReader->pSchema == NULL) { - code = metaGetTbTSchemaEx(pReader->pTsdb->pVnode->pMeta, uid, -1, 1, &pReader->pSchema); + code = metaGetTbTSchemaEx(pReader->pTsdb->pVnode->pMeta, pReader->suid, uid, -1, &pReader->pSchema); if (code != TSDB_CODE_SUCCESS) { terrno = code; return NULL; @@ -1975,7 +1975,7 @@ static int32_t doMergeBufAndFileRows(STsdbReader* pReader, STableBlockScanInfo* // DESC: mem -----> imem -----> last block -----> file block if (pReader->order == TSDB_ORDER_ASC) { if (minKey == key) { - init = true; + init = true; // todo check if pReader->pSchema is null or not int32_t code = tsdbRowMergerInit(&merge, NULL, &fRow, pReader->pSchema); if (code != TSDB_CODE_SUCCESS) { return code; @@ -5234,7 +5234,7 @@ int32_t tsdbGetTableSchema(SVnode* pVnode, int64_t uid, STSchema** pSchema, int6 metaReaderClear(&mr); // get the newest table schema version - code = metaGetTbTSchemaEx(pVnode->pMeta, uid, -1, 1, pSchema); + code = metaGetTbTSchemaEx(pVnode->pMeta, *suid, uid, -1, pSchema); return code; } diff --git a/source/libs/executor/src/dataDeleter.c b/source/libs/executor/src/dataDeleter.c index dc3bf83a91..d693faf7f1 100644 --- a/source/libs/executor/src/dataDeleter.c +++ b/source/libs/executor/src/dataDeleter.c @@ -240,6 +240,7 @@ int32_t createDataDeleter(SDataSinkManager* pManager, const SDataSinkNode* pData SDataDeleterHandle* deleter = taosMemoryCalloc(1, sizeof(SDataDeleterHandle)); if (NULL == deleter) { + taosMemoryFree(pParam); code = TSDB_CODE_OUT_OF_MEMORY; goto _end; } diff --git a/source/libs/executor/src/dataInserter.c b/source/libs/executor/src/dataInserter.c index e9c46843c0..90d740bebd 100644 --- a/source/libs/executor/src/dataInserter.c +++ b/source/libs/executor/src/dataInserter.c @@ -408,6 +408,7 @@ int32_t createDataInserter(SDataSinkManager* pManager, const SDataSinkNode* pDat void* pParam) { SDataInserterHandle* inserter = taosMemoryCalloc(1, sizeof(SDataInserterHandle)); if (NULL == inserter) { + taosMemoryFree(pParam); terrno = TSDB_CODE_OUT_OF_MEMORY; return TSDB_CODE_OUT_OF_MEMORY; } diff --git a/source/libs/executor/src/executor.c b/source/libs/executor/src/executor.c index 181ab3d44c..1732ec04a7 100644 --- a/source/libs/executor/src/executor.c +++ b/source/libs/executor/src/executor.c @@ -517,10 +517,8 @@ int32_t qCreateExecTask(SReadHandle* readHandle, int32_t vgId, uint64_t taskId, goto _error; } + // pSinkParam has been freed during create sinker. code = dsCreateDataSinker(pSubplan->pDataSink, handle, pSinkParam, (*pTask)->id.str); - if (code != TSDB_CODE_SUCCESS) { - taosMemoryFreeClear(pSinkParam); - } } qDebug("subplan task create completed, TID:0x%" PRIx64 " QID:0x%" PRIx64, taskId, pSubplan->id.queryId); From 9627e8105b819626229a4ef9be61375d6486e248 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 18 Apr 2023 13:52:47 +0800 Subject: [PATCH 11/20] enh(tmq): support seek offset. --- include/client/taos.h | 8 ++ source/client/src/clientTmq.c | 205 +++++++++++++++++++++++++++++----- source/common/src/tmsg.c | 6 +- 3 files changed, 190 insertions(+), 29 deletions(-) diff --git a/include/client/taos.h b/include/client/taos.h index cf410a42da..05f3c5d11d 100644 --- a/include/client/taos.h +++ b/include/client/taos.h @@ -262,6 +262,12 @@ DLL_EXPORT tmq_t *tmq_consumer_new(tmq_conf_t *conf, char *errstr, int32_t errst DLL_EXPORT const char *tmq_err2str(int32_t code); /* ------------------------TMQ CONSUMER INTERFACE------------------------ */ +typedef struct tmq_topic_assignment { + int32_t vgroupHandle; + int64_t currentOffset; + int64_t begin; + int64_t end; +} tmq_topic_assignment; DLL_EXPORT int32_t tmq_subscribe(tmq_t *tmq, const tmq_list_t *topic_list); DLL_EXPORT int32_t tmq_unsubscribe(tmq_t *tmq); @@ -270,6 +276,8 @@ DLL_EXPORT TAOS_RES *tmq_consumer_poll(tmq_t *tmq, int64_t timeout); DLL_EXPORT int32_t tmq_consumer_close(tmq_t *tmq); DLL_EXPORT int32_t tmq_commit_sync(tmq_t *tmq, const TAOS_RES *msg); DLL_EXPORT void tmq_commit_async(tmq_t *tmq, const TAOS_RES *msg, tmq_commit_cb *cb, void *param); +DLL_EXPORT int32_t tmq_get_topic_assignment(tmq_t *tmq, const char* pTopicName, tmq_topic_assignment **assignment, int32_t *numOfAssignment); +DLL_EXPORT int32_t tmq_offset_seek(tmq_t *tmq, const char* pTopicName, int32_t vgroupHandle, int64_t offset); /* ----------------------TMQ CONFIGURATION INTERFACE---------------------- */ diff --git a/source/client/src/clientTmq.c b/source/client/src/clientTmq.c index ceca06e309..e6dc9881fe 100644 --- a/source/client/src/clientTmq.c +++ b/source/client/src/clientTmq.c @@ -133,16 +133,22 @@ enum { TMQ_DELAYED_TASK__COMMIT, }; -typedef struct { - int64_t pollCnt; - int64_t numOfRows; +typedef struct SVgOffsetInfo { STqOffsetVal committedOffset; STqOffsetVal currentOffset; - int32_t vgId; - int32_t vgStatus; - int32_t vgSkipCnt; - int64_t emptyBlockReceiveTs; // once empty block is received, idle for ignoreCnt then start to poll data - SEpSet epSet; + int64_t walVerBegin; + int64_t walVerEnd; +} SVgOffsetInfo; + +typedef struct { + int64_t pollCnt; + int64_t numOfRows; + SVgOffsetInfo offsetInfo; + int32_t vgId; + int32_t vgStatus; + int32_t vgSkipCnt; // here used to mark the slow vgroups + int64_t emptyBlockReceiveTs; // once empty block is received, idle for ignoreCnt then start to poll data + SEpSet epSet; } SMqClientVg; typedef struct { @@ -441,7 +447,7 @@ static int32_t tmqCommitCb(void* param, SDataBuf* pBuf, int32_t code) { // if (code1 != TSDB_CODE_SUCCESS) { // retry failed. // tscError("consumer:0x%" PRIx64 " topic:%s vgId:%d offset:%" PRId64 // " retry failed, ignore this commit. code:%s ordinal:%d/%d", - // pParam->pTmq->consumerId, pParam->topicName, pVg->vgId, pVg->committedOffset.version, + // pParam->pTmq->consumerId, pParam->topicName, pVg->vgId, pVg->offsetInfo.committedOffset.version, // tstrerror(terrno), index + 1, numOfVgroups); // } // } @@ -473,7 +479,7 @@ static int32_t doSendCommitMsg(tmq_t* tmq, SMqClientVg* pVg, const char* pTopicN return TSDB_CODE_OUT_OF_MEMORY; } - pOffset->val = pVg->currentOffset; + pOffset->val = pVg->offsetInfo.currentOffset; int32_t groupLen = strlen(tmq->groupId); memcpy(pOffset->subKey, tmq->groupId, groupLen); @@ -543,7 +549,7 @@ static int32_t doSendCommitMsg(tmq_t* tmq, SMqClientVg* pVg, const char* pTopicN tFormatOffset(offsetBuf, tListLen(offsetBuf), &pOffset->val); char commitBuf[80] = {0}; - tFormatOffset(commitBuf, tListLen(commitBuf), &pVg->committedOffset); + tFormatOffset(commitBuf, tListLen(commitBuf), &pVg->offsetInfo.committedOffset); tscDebug("consumer:0x%" PRIx64 " topic:%s on vgId:%d send offset:%s prev:%s, ep:%s:%d, ordinal:%d/%d, req:0x%" PRIx64, tmq->consumerId, pOffset->subKey, pVg->vgId, offsetBuf, commitBuf, pEp->fqdn, pEp->port, index + 1, totalVgroups, pMsgSendInfo->requestId); @@ -632,7 +638,7 @@ static void asyncCommitOffset(tmq_t* tmq, const TAOS_RES* pRes, tmq_commit_cb* p } SMqClientVg* pVg = taosArrayGet(pTopic->vgs, j); - if (pVg->currentOffset.type > 0 && !tOffsetEqual(&pVg->currentOffset, &pVg->committedOffset)) { + if (pVg->offsetInfo.currentOffset.type > 0 && !tOffsetEqual(&pVg->offsetInfo.currentOffset, &pVg->offsetInfo.committedOffset)) { code = doSendCommitMsg(tmq, pVg, pTopic->topicName, pParamSet, j, numOfVgroups); // failed to commit, callback user function directly. @@ -673,20 +679,20 @@ static void asyncCommitAllOffsets(tmq_t* tmq, tmq_commit_cb* pCommitFp, void* us for (int32_t j = 0; j < numOfVgroups; j++) { SMqClientVg* pVg = taosArrayGet(pTopic->vgs, j); - if (pVg->currentOffset.type > 0 && !tOffsetEqual(&pVg->currentOffset, &pVg->committedOffset)) { + if (pVg->offsetInfo.currentOffset.type > 0 && !tOffsetEqual(&pVg->offsetInfo.currentOffset, &pVg->offsetInfo.committedOffset)) { int32_t code = doSendCommitMsg(tmq, pVg, pTopic->topicName, pParamSet, j, numOfVgroups); if (code != TSDB_CODE_SUCCESS) { tscError("consumer:0x%" PRIx64 " topic:%s vgId:%d offset:%" PRId64 " failed, code:%s ordinal:%d/%d", - tmq->consumerId, pTopic->topicName, pVg->vgId, pVg->committedOffset.version, tstrerror(terrno), + tmq->consumerId, pTopic->topicName, pVg->vgId, pVg->offsetInfo.committedOffset.version, tstrerror(terrno), j + 1, numOfVgroups); continue; } // update the offset value. - pVg->committedOffset = pVg->currentOffset; + pVg->offsetInfo.committedOffset = pVg->offsetInfo.currentOffset; } else { tscDebug("consumer:0x%" PRIx64 " topic:%s vgId:%d, no commit, current:%" PRId64 ", ordinal:%d/%d", - tmq->consumerId, pTopic->topicName, pVg->vgId, pVg->currentOffset.version, j + 1, numOfVgroups); + tmq->consumerId, pTopic->topicName, pVg->vgId, pVg->offsetInfo.currentOffset.version, j + 1, numOfVgroups); } } } @@ -1398,7 +1404,6 @@ static void initClientTopicFromRsp(SMqClientTopic* pTopic, SMqSubTopicEp* pTopic SMqClientVg clientVg = { .pollCnt = 0, - .currentOffset = offsetNew, .vgId = pVgEp->vgId, .epSet = pVgEp->epSet, .vgStatus = TMQ_VG_STATUS__IDLE, @@ -1407,6 +1412,10 @@ static void initClientTopicFromRsp(SMqClientTopic* pTopic, SMqSubTopicEp* pTopic .numOfRows = numOfRows, }; + clientVg.offsetInfo.currentOffset = offsetNew; + clientVg.offsetInfo.committedOffset = offsetNew; + clientVg.offsetInfo.walVerBegin = -1; + clientVg.offsetInfo.walVerEnd = -1; taosArrayPush(pTopic->vgs, &clientVg); } } @@ -1456,11 +1465,11 @@ static bool doUpdateLocalEp(tmq_t* tmq, int32_t epoch, const SMqAskEpRsp* pRsp) makeTopicVgroupKey(vgKey, pTopicCur->topicName, pVgCur->vgId); char buf[80]; - tFormatOffset(buf, 80, &pVgCur->currentOffset); + tFormatOffset(buf, 80, &pVgCur->offsetInfo.currentOffset); tscDebug("consumer:0x%" PRIx64 ", epoch:%d vgId:%d vgKey:%s, offset:%s", tmq->consumerId, epoch, pVgCur->vgId, vgKey, buf); - SVgroupSaveInfo info = {.offset = pVgCur->currentOffset, .numOfRows = pVgCur->numOfRows}; + SVgroupSaveInfo info = {.offset = pVgCur->offsetInfo.currentOffset, .numOfRows = pVgCur->numOfRows}; taosHashPut(pVgOffsetHashMap, vgKey, strlen(vgKey), &info, sizeof(SVgroupSaveInfo)); } } @@ -1557,7 +1566,7 @@ void tmqBuildConsumeReqImpl(SMqPollReq* pReq, tmq_t* tmq, int64_t timeout, SMqCl pReq->timeout = timeout; pReq->epoch = tmq->epoch; /*pReq->currentOffset = reqOffset;*/ - pReq->reqOffset = pVg->currentOffset; + pReq->reqOffset = pVg->offsetInfo.currentOffset; pReq->head.vgId = pVg->vgId; pReq->useSnapshot = tmq->useSnapshot; pReq->reqId = generateRequestId(); @@ -1681,7 +1690,7 @@ static int32_t doTmqPollImpl(tmq_t* pTmq, SMqClientTopic* pTopic, SMqClientVg* p int64_t transporterId = 0; char offsetFormatBuf[80]; - tFormatOffset(offsetFormatBuf, tListLen(offsetFormatBuf), &pVg->currentOffset); + tFormatOffset(offsetFormatBuf, tListLen(offsetFormatBuf), &pVg->offsetInfo.currentOffset); tscDebug("consumer:0x%" PRIx64 " send poll to %s vgId:%d, epoch %d, req:%s, reqId:0x%" PRIx64, pTmq->consumerId, pTopic->topicName, pVg->vgId, pTmq->epoch, offsetFormatBuf, req.reqId); @@ -1798,7 +1807,7 @@ static void* tmqHandleAllRsp(tmq_t* tmq, int64_t timeout, bool pollIfReset) { } // update the local offset value only for the returned values. - pVg->currentOffset = pDataRsp->rspOffset; + pVg->offsetInfo.currentOffset = pDataRsp->rspOffset; atomic_store_32(&pVg->vgStatus, TMQ_VG_STATUS__IDLE); char buf[80]; @@ -1835,7 +1844,7 @@ static void* tmqHandleAllRsp(tmq_t* tmq, int64_t timeout, bool pollIfReset) { if (pollRspWrapper->metaRsp.head.epoch == consumerEpoch) { SMqClientVg* pVg = pollRspWrapper->vgHandle; - pVg->currentOffset = pollRspWrapper->metaRsp.rspOffset; + pVg->offsetInfo.currentOffset = pollRspWrapper->metaRsp.rspOffset; atomic_store_32(&pVg->vgStatus, TMQ_VG_STATUS__IDLE); // build rsp SMqMetaRspObj* pRsp = tmqBuildMetaRspFromWrapper(pollRspWrapper); @@ -1853,7 +1862,7 @@ static void* tmqHandleAllRsp(tmq_t* tmq, int64_t timeout, bool pollIfReset) { if (pollRspWrapper->taosxRsp.head.epoch == consumerEpoch) { SMqClientVg* pVg = pollRspWrapper->vgHandle; - pVg->currentOffset = pollRspWrapper->taosxRsp.rspOffset; + pVg->offsetInfo.currentOffset = pollRspWrapper->taosxRsp.rspOffset; atomic_store_32(&pVg->vgStatus, TMQ_VG_STATUS__IDLE); if (pollRspWrapper->taosxRsp.blockNum == 0) { @@ -1879,7 +1888,7 @@ static void* tmqHandleAllRsp(tmq_t* tmq, int64_t timeout, bool pollIfReset) { tmq->totalRows += numOfRows; char buf[80]; - tFormatOffset(buf, 80, &pVg->currentOffset); + tFormatOffset(buf, 80, &pVg->offsetInfo.currentOffset); tscDebug("consumer:0x%" PRIx64 " process taosx poll rsp, vgId:%d, offset:%s, blocks:%d, rows:%" PRId64 ", vg total:%" PRId64 " total:%" PRId64 " reqId:0x%" PRIx64, tmq->consumerId, pVg->vgId, buf, pollRspWrapper->dataRsp.blockNum, numOfRows, pVg->numOfRows, @@ -2323,4 +2332,150 @@ SReqResultInfo* tmqGetNextResInfo(TAOS_RES* res, bool convertUcs4) { } return NULL; +} + +static SMqClientTopic* getTopicByName(tmq_t* tmq, const char* pTopicName) { + int32_t numOfTopics = taosArrayGetSize(tmq->clientTopics); + for (int32_t i = 0; i < numOfTopics; ++i) { + SMqClientTopic* pTopic = taosArrayGet(tmq->clientTopics, i); + if (strcmp(pTopic->topicName, pTopicName) != 0) { + continue; + } + + return pTopic; + } + + tscError("consumer:0x%" PRIx64 ", failed to find topic:%s", tmq->consumerId, pTopicName); + return NULL; +} + +int32_t tmq_get_topic_assignment(tmq_t* tmq, const char* pTopicName, tmq_topic_assignment** assignment, + int32_t* numOfAssignment) { + *numOfAssignment = 0; + *assignment = NULL; + + SMqClientTopic* pTopic = getTopicByName(tmq, pTopicName); + if (pTopic == NULL) { + return TSDB_CODE_INVALID_PARA; + } + + // in case of snapshot is opened, no valid offset will return + *numOfAssignment = taosArrayGetSize(pTopic->vgs); + for (int32_t j = 0; j < (*numOfAssignment); ++j) { + SMqClientVg* pClientVg = taosArrayGet(pTopic->vgs, j); + + tmq_topic_assignment* pAssignment = &(*assignment)[j]; + if (pClientVg->offsetInfo.currentOffset.type == TMQ_OFFSET__LOG) { + pAssignment->currentOffset = pClientVg->offsetInfo.currentOffset.version; + } else { + pAssignment->currentOffset = 0; + } + + pAssignment->begin = pClientVg->offsetInfo.walVerBegin; + pAssignment->end = pClientVg->offsetInfo.walVerEnd; + pAssignment->vgroupHandle = pClientVg->vgId; + } + + return TSDB_CODE_SUCCESS; +} + +int32_t tmq_offset_seek(tmq_t* tmq, const char* pTopicName, int32_t vgroupHandle, int64_t offset) { + if (tmq == NULL) { + return TSDB_CODE_INVALID_PARA; + } + + SMqClientTopic* pTopic = getTopicByName(tmq, pTopicName); + if (pTopic == NULL) { + return TSDB_CODE_INVALID_PARA; + } + + SMqClientVg* pVg = NULL; + + int32_t numOfVgs = taosArrayGetSize(pTopic->vgs); + for(int32_t i= 0; i < numOfVgs; ++i) { + SMqClientVg* pClientVg = taosArrayGet(pTopic->vgs, i); + if (pClientVg->vgId == vgroupHandle) { + pVg = pClientVg; + break; + } + } + + if (pVg == NULL) { + return TSDB_CODE_INVALID_PARA; + } + + if (offset < pVg->offsetInfo.walVerBegin|| offset > pVg->offsetInfo.walVerEnd) { + return TSDB_CODE_INVALID_PARA; + } + + return 0; +#if 0 + // tmq_commit_sync(tmq, ); + { + SMqCommitCbParamSet* pParamSet = taosMemoryCalloc(1, sizeof(SMqCommitCbParamSet)); + if (pParamSet == NULL) { +// pCommitFp(tmq, TSDB_CODE_OUT_OF_MEMORY, userParam); + return -1; + } + + pParamSet->refId = tmq->refId; + pParamSet->epoch = tmq->epoch; + pParamSet->callbackFn = pCommitFp; + pParamSet->userParam = userParam; + + int32_t numOfTopics = taosArrayGetSize(tmq->clientTopics); + + tscDebug("consumer:0x%" PRIx64 " do manual commit offset for %s, vgId:%d", tmq->consumerId, pTopicName, vgId); + + int32_t i = 0; + for (; i < numOfTopics; i++) { + SMqClientTopic* pTopic = taosArrayGet(tmq->clientTopics, i); + if (strcmp(pTopic->topicName, pTopicName) == 0) { + break; + } + } + + if (i == numOfTopics) { + tscWarn("consumer:0x%" PRIx64 " failed to find the specified topic:%s, total topics:%d", tmq->consumerId, + pTopicName, numOfTopics); + taosMemoryFree(pParamSet); + pCommitFp(tmq, TSDB_CODE_SUCCESS, userParam); + return; + } + + SMqClientTopic* pTopic = taosArrayGet(tmq->clientTopics, i); + + int32_t j = 0; + int32_t numOfVgroups = taosArrayGetSize(pTopic->vgs); + for (j = 0; j < numOfVgroups; j++) { + SMqClientVg* pVg = taosArrayGet(pTopic->vgs, j); + if (pVg->vgId == vgId) { + break; + } + } + + if (j == numOfVgroups) { + tscWarn("consumer:0x%" PRIx64 " failed to find the specified vgId:%d, total Vgs:%d, topic:%s", tmq->consumerId, + vgId, numOfVgroups, pTopicName); + taosMemoryFree(pParamSet); + pCommitFp(tmq, TSDB_CODE_SUCCESS, userParam); + return; + } + + SMqClientVg* pVg = taosArrayGet(pTopic->vgs, j); + if (pVg->offsetInfo.currentOffset.type > 0 && !tOffsetEqual(&pVg->offsetInfo.currentOffset, &pVg->offsetInfo.committedOffset)) { + code = doSendCommitMsg(tmq, pVg, pTopic->topicName, pParamSet, j, numOfVgroups); + + // failed to commit, callback user function directly. + if (code != TSDB_CODE_SUCCESS) { + taosMemoryFree(pParamSet); + pCommitFp(tmq, code, userParam); + } + } else { // do not perform commit, callback user function directly. + taosMemoryFree(pParamSet); + pCommitFp(tmq, code, userParam); + } + } +#endif + } \ No newline at end of file diff --git a/source/common/src/tmsg.c b/source/common/src/tmsg.c index d9802244b7..0a18941780 100644 --- a/source/common/src/tmsg.c +++ b/source/common/src/tmsg.c @@ -7158,8 +7158,7 @@ int32_t tDecodeSTaosxRsp(SDecoder *pDecoder, STaosxRsp *pRsp) { } void tDeleteSTaosxRsp(STaosxRsp *pRsp) { - taosArrayDestroy(pRsp->blockDataLen); - pRsp->blockDataLen = NULL; + pRsp->blockDataLen = taosArrayDestroy(pRsp->blockDataLen); taosArrayDestroyP(pRsp->blockData, (FDelete)taosMemoryFree); pRsp->blockData = NULL; taosArrayDestroyP(pRsp->blockSchema, (FDelete)tDeleteSSchemaWrapper); @@ -7167,8 +7166,7 @@ void tDeleteSTaosxRsp(STaosxRsp *pRsp) { taosArrayDestroyP(pRsp->blockTbName, (FDelete)taosMemoryFree); pRsp->blockTbName = NULL; - taosArrayDestroy(pRsp->createTableLen); - pRsp->createTableLen = NULL; + pRsp->createTableLen = taosArrayDestroy(pRsp->createTableLen); taosArrayDestroyP(pRsp->createTableReq, (FDelete)taosMemoryFree); pRsp->createTableReq = NULL; } From 3f12156c537d968005966b4571036cfbc07caf9b Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 19 Apr 2023 13:56:42 +0800 Subject: [PATCH 12/20] enh(tmq): support seek in tmq. --- source/client/src/clientTmq.c | 183 +++++++++++++------------- source/dnode/vnode/src/tq/tqRestore.c | 21 --- 2 files changed, 90 insertions(+), 114 deletions(-) diff --git a/source/client/src/clientTmq.c b/source/client/src/clientTmq.c index e6dc9881fe..d6ad4c3db9 100644 --- a/source/client/src/clientTmq.c +++ b/source/client/src/clientTmq.c @@ -560,6 +560,21 @@ static int32_t doSendCommitMsg(tmq_t* tmq, SMqClientVg* pVg, const char* pTopicN return TSDB_CODE_SUCCESS; } +static SMqClientTopic* getTopicByName(tmq_t* tmq, const char* pTopicName) { + int32_t numOfTopics = taosArrayGetSize(tmq->clientTopics); + for (int32_t i = 0; i < numOfTopics; ++i) { + SMqClientTopic* pTopic = taosArrayGet(tmq->clientTopics, i); + if (strcmp(pTopic->topicName, pTopicName) != 0) { + continue; + } + + return pTopic; + } + + tscError("consumer:0x%" PRIx64 ", failed to find topic:%s", tmq->consumerId, pTopicName); + return NULL; +} + static void asyncCommitOffset(tmq_t* tmq, const TAOS_RES* pRes, tmq_commit_cb* pCommitFp, void* userParam) { char* pTopicName = NULL; int32_t vgId = 0; @@ -602,15 +617,8 @@ static void asyncCommitOffset(tmq_t* tmq, const TAOS_RES* pRes, tmq_commit_cb* p tscDebug("consumer:0x%" PRIx64 " do manual commit offset for %s, vgId:%d", tmq->consumerId, pTopicName, vgId); - int32_t i = 0; - for (; i < numOfTopics; i++) { - SMqClientTopic* pTopic = taosArrayGet(tmq->clientTopics, i); - if (strcmp(pTopic->topicName, pTopicName) == 0) { - break; - } - } - - if (i == numOfTopics) { + SMqClientTopic* pTopic = getTopicByName(tmq, pTopicName); + if (pTopic == NULL) { tscWarn("consumer:0x%" PRIx64 " failed to find the specified topic:%s, total topics:%d", tmq->consumerId, pTopicName, numOfTopics); taosMemoryFree(pParamSet); @@ -618,8 +626,6 @@ static void asyncCommitOffset(tmq_t* tmq, const TAOS_RES* pRes, tmq_commit_cb* p return; } - SMqClientTopic* pTopic = taosArrayGet(tmq->clientTopics, i); - int32_t j = 0; int32_t numOfVgroups = taosArrayGetSize(pTopic->vgs); for (j = 0; j < numOfVgroups; j++) { @@ -2334,20 +2340,7 @@ SReqResultInfo* tmqGetNextResInfo(TAOS_RES* res, bool convertUcs4) { return NULL; } -static SMqClientTopic* getTopicByName(tmq_t* tmq, const char* pTopicName) { - int32_t numOfTopics = taosArrayGetSize(tmq->clientTopics); - for (int32_t i = 0; i < numOfTopics; ++i) { - SMqClientTopic* pTopic = taosArrayGet(tmq->clientTopics, i); - if (strcmp(pTopic->topicName, pTopicName) != 0) { - continue; - } - return pTopic; - } - - tscError("consumer:0x%" PRIx64 ", failed to find topic:%s", tmq->consumerId, pTopicName); - return NULL; -} int32_t tmq_get_topic_assignment(tmq_t* tmq, const char* pTopicName, tmq_topic_assignment** assignment, int32_t* numOfAssignment) { @@ -2408,74 +2401,78 @@ int32_t tmq_offset_seek(tmq_t* tmq, const char* pTopicName, int32_t vgroupHandle return TSDB_CODE_INVALID_PARA; } - return 0; -#if 0 - // tmq_commit_sync(tmq, ); - { - SMqCommitCbParamSet* pParamSet = taosMemoryCalloc(1, sizeof(SMqCommitCbParamSet)); - if (pParamSet == NULL) { -// pCommitFp(tmq, TSDB_CODE_OUT_OF_MEMORY, userParam); - return -1; - } - - pParamSet->refId = tmq->refId; - pParamSet->epoch = tmq->epoch; - pParamSet->callbackFn = pCommitFp; - pParamSet->userParam = userParam; - - int32_t numOfTopics = taosArrayGetSize(tmq->clientTopics); - - tscDebug("consumer:0x%" PRIx64 " do manual commit offset for %s, vgId:%d", tmq->consumerId, pTopicName, vgId); - - int32_t i = 0; - for (; i < numOfTopics; i++) { - SMqClientTopic* pTopic = taosArrayGet(tmq->clientTopics, i); - if (strcmp(pTopic->topicName, pTopicName) == 0) { - break; - } - } - - if (i == numOfTopics) { - tscWarn("consumer:0x%" PRIx64 " failed to find the specified topic:%s, total topics:%d", tmq->consumerId, - pTopicName, numOfTopics); - taosMemoryFree(pParamSet); - pCommitFp(tmq, TSDB_CODE_SUCCESS, userParam); - return; - } - - SMqClientTopic* pTopic = taosArrayGet(tmq->clientTopics, i); - - int32_t j = 0; - int32_t numOfVgroups = taosArrayGetSize(pTopic->vgs); - for (j = 0; j < numOfVgroups; j++) { - SMqClientVg* pVg = taosArrayGet(pTopic->vgs, j); - if (pVg->vgId == vgId) { - break; - } - } - - if (j == numOfVgroups) { - tscWarn("consumer:0x%" PRIx64 " failed to find the specified vgId:%d, total Vgs:%d, topic:%s", tmq->consumerId, - vgId, numOfVgroups, pTopicName); - taosMemoryFree(pParamSet); - pCommitFp(tmq, TSDB_CODE_SUCCESS, userParam); - return; - } - - SMqClientVg* pVg = taosArrayGet(pTopic->vgs, j); - if (pVg->offsetInfo.currentOffset.type > 0 && !tOffsetEqual(&pVg->offsetInfo.currentOffset, &pVg->offsetInfo.committedOffset)) { - code = doSendCommitMsg(tmq, pVg, pTopic->topicName, pParamSet, j, numOfVgroups); - - // failed to commit, callback user function directly. - if (code != TSDB_CODE_SUCCESS) { - taosMemoryFree(pParamSet); - pCommitFp(tmq, code, userParam); - } - } else { // do not perform commit, callback user function directly. - taosMemoryFree(pParamSet); - pCommitFp(tmq, code, userParam); - } - } -#endif +// return 0; +//#if 0 + SMqRspObj rspObj = {.resType = RES_TYPE__TMQ, .vgId = pVg->vgId}; + tstrncpy(rspObj.topic, pTopicName, tListLen(rspObj.topic)); + tmq_commit_sync(tmq, &rspObj); + // { + // SMqCommitCbParamSet* pParamSet = taosMemoryCalloc(1, sizeof(SMqCommitCbParamSet)); + // if (pParamSet == NULL) { + //// pCommitFp(tmq, TSDB_CODE_OUT_OF_MEMORY, userParam); + // return -1; + // } + // + // pParamSet->refId = tmq->refId; + // pParamSet->epoch = tmq->epoch; + // pParamSet->callbackFn = pCommitFp; + // pParamSet->userParam = userParam; + // + // int32_t numOfTopics = taosArrayGetSize(tmq->clientTopics); + // + // tscDebug("consumer:0x%" PRIx64 " do manual commit offset for %s, vgId:%d", tmq->consumerId, pTopicName, vgId); + // + // int32_t i = 0; + // for (; i < numOfTopics; i++) { + // SMqClientTopic* pTopic = taosArrayGet(tmq->clientTopics, i); + // if (strcmp(pTopic->topicName, pTopicName) == 0) { + // break; + // } + // } + // + // if (i == numOfTopics) { + // tscWarn("consumer:0x%" PRIx64 " failed to find the specified topic:%s, total topics:%d", tmq->consumerId, + // pTopicName, numOfTopics); + // taosMemoryFree(pParamSet); + // pCommitFp(tmq, TSDB_CODE_SUCCESS, userParam); + // return; + // } + // + // SMqClientTopic* pTopic = taosArrayGet(tmq->clientTopics, i); + // + // int32_t j = 0; + // int32_t numOfVgroups = taosArrayGetSize(pTopic->vgs); + // for (j = 0; j < numOfVgroups; j++) { + // SMqClientVg* pVg = taosArrayGet(pTopic->vgs, j); + // if (pVg->vgId == vgId) { + // break; + // } + // } + // + // if (j == numOfVgroups) { + // tscWarn("consumer:0x%" PRIx64 " failed to find the specified vgId:%d, total Vgs:%d, topic:%s", + // tmq->consumerId, + // vgId, numOfVgroups, pTopicName); + // taosMemoryFree(pParamSet); + // pCommitFp(tmq, TSDB_CODE_SUCCESS, userParam); + // return; + // } + // + // SMqClientVg* pVg = taosArrayGet(pTopic->vgs, j); + // if (pVg->offsetInfo.currentOffset.type > 0 && !tOffsetEqual(&pVg->offsetInfo.currentOffset, + // &pVg->offsetInfo.committedOffset)) { + // code = doSendCommitMsg(tmq, pVg, pTopic->topicName, pParamSet, j, numOfVgroups); + // + // // failed to commit, callback user function directly. + // if (code != TSDB_CODE_SUCCESS) { + // taosMemoryFree(pParamSet); + // pCommitFp(tmq, code, userParam); + // } + // } else { // do not perform commit, callback user function directly. + // taosMemoryFree(pParamSet); + // pCommitFp(tmq, code, userParam); + // } + // } + //#endif } \ No newline at end of file diff --git a/source/dnode/vnode/src/tq/tqRestore.c b/source/dnode/vnode/src/tq/tqRestore.c index cba51cdee4..54fcc04b62 100644 --- a/source/dnode/vnode/src/tq/tqRestore.c +++ b/source/dnode/vnode/src/tq/tqRestore.c @@ -55,27 +55,6 @@ int tqStreamTasksScanWal(STQ* pTq) { return 0; } -//int32_t transferToNormalTask(SStreamMeta* pStreamMeta, SArray* pTaskList) { -// int32_t numOfTask = taosArrayGetSize(pTaskList); -// if (numOfTask <= 0) { -// return TSDB_CODE_SUCCESS; -// } -// -// // todo: add lock -// for (int32_t i = 0; i < numOfTask; ++i) { -// SStreamTask* pTask = taosArrayGetP(pTaskList, i); -// tqDebug("vgId:%d transfer s-task:%s state restore -> ready, checkpoint:%" PRId64 " checkpoint id:%" PRId64, -// pStreamMeta->vgId, pTask->id.idStr, pTask->chkInfo.version, pTask->chkInfo.id); -// taosHashRemove(pStreamMeta->pWalReadTasks, &pTask->id.taskId, sizeof(pTask->id.taskId)); -// -// // NOTE: do not change the following order -// atomic_store_8(&pTask->status.taskStatus, TASK_STATUS__NORMAL); -// taosHashPut(pStreamMeta->pTasks, &pTask->id.taskId, sizeof(pTask->id.taskId), &pTask, POINTER_BYTES); -// } -// -// return TSDB_CODE_SUCCESS; -//} - int32_t streamTaskReplayWal(SStreamMeta* pStreamMeta, STqOffsetStore* pOffsetStore, bool* pScanIdle) { void* pIter = NULL; int32_t vgId = pStreamMeta->vgId; From d5c57ca2449fd5059bce578d2ce73b6cdbe1e277 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 19 Apr 2023 14:46:48 +0800 Subject: [PATCH 13/20] enh(tmq): support the seek. --- source/client/src/clientTmq.c | 116 ++++++++++------------------------ 1 file changed, 32 insertions(+), 84 deletions(-) diff --git a/source/client/src/clientTmq.c b/source/client/src/clientTmq.c index d6ad4c3db9..6f4152ca97 100644 --- a/source/client/src/clientTmq.c +++ b/source/client/src/clientTmq.c @@ -571,7 +571,7 @@ static SMqClientTopic* getTopicByName(tmq_t* tmq, const char* pTopicName) { return pTopic; } - tscError("consumer:0x%" PRIx64 ", failed to find topic:%s", tmq->consumerId, pTopicName); + tscError("consumer:0x%" PRIx64 ", total:%d, failed to find topic:%s", tmq->consumerId, numOfTopics, pTopicName); return NULL; } @@ -1571,7 +1571,6 @@ void tmqBuildConsumeReqImpl(SMqPollReq* pReq, tmq_t* tmq, int64_t timeout, SMqCl pReq->consumerId = tmq->consumerId; pReq->timeout = timeout; pReq->epoch = tmq->epoch; - /*pReq->currentOffset = reqOffset;*/ pReq->reqOffset = pVg->offsetInfo.currentOffset; pReq->head.vgId = pVg->vgId; pReq->useSnapshot = tmq->useSnapshot; @@ -1670,7 +1669,7 @@ static int32_t doTmqPollImpl(tmq_t* pTmq, SMqClientTopic* pTopic, SMqClientVg* p pParam->refId = pTmq->refId; pParam->epoch = pTmq->epoch; - pParam->pVg = pVg; // pVg may be released,fix it + pParam->pVg = pVg; pParam->pTopic = pTopic; pParam->vgId = pVg->vgId; pParam->requestId = req.reqId; @@ -1682,12 +1681,7 @@ static int32_t doTmqPollImpl(tmq_t* pTmq, SMqClientTopic* pTopic, SMqClientVg* p return handleErrorBeforePoll(pVg, pTmq); } - sendInfo->msgInfo = (SDataBuf){ - .pData = msg, - .len = msgSize, - .handle = NULL, - }; - + sendInfo->msgInfo = (SDataBuf){.pData = msg, .len = msgSize, .handle = NULL}; sendInfo->requestId = req.reqId; sendInfo->requestObjRefId = 0; sendInfo->param = pParam; @@ -2374,18 +2368,19 @@ int32_t tmq_get_topic_assignment(tmq_t* tmq, const char* pTopicName, tmq_topic_a int32_t tmq_offset_seek(tmq_t* tmq, const char* pTopicName, int32_t vgroupHandle, int64_t offset) { if (tmq == NULL) { + tscError("invalid tmq handle, null"); return TSDB_CODE_INVALID_PARA; } SMqClientTopic* pTopic = getTopicByName(tmq, pTopicName); if (pTopic == NULL) { + tscError("consumer:0x:" PRIx64 " invalid topic name:%s", tmq->consumerId, pTopicName); return TSDB_CODE_INVALID_PARA; } SMqClientVg* pVg = NULL; - - int32_t numOfVgs = taosArrayGetSize(pTopic->vgs); - for(int32_t i= 0; i < numOfVgs; ++i) { + int32_t numOfVgs = taosArrayGetSize(pTopic->vgs); + for (int32_t i = 0; i < numOfVgs; ++i) { SMqClientVg* pClientVg = taosArrayGet(pTopic->vgs, i); if (pClientVg->vgId == vgroupHandle) { pVg = pClientVg; @@ -2394,85 +2389,38 @@ int32_t tmq_offset_seek(tmq_t* tmq, const char* pTopicName, int32_t vgroupHandle } if (pVg == NULL) { + tscError("consumer:0x:" PRIx64 " invalid vgroup id:%d", tmq->consumerId, vgroupHandle); return TSDB_CODE_INVALID_PARA; } - if (offset < pVg->offsetInfo.walVerBegin|| offset > pVg->offsetInfo.walVerEnd) { + SVgOffsetInfo* pOffsetInfo = &pVg->offsetInfo; + + int32_t type = pOffsetInfo->currentOffset.type; + if (type != TMQ_OFFSET__LOG) { + tscError("consumer:0x:" PRIx64 " offset type:%d not wal version, seek not allowed", tmq->consumerId, type); return TSDB_CODE_INVALID_PARA; } -// return 0; -//#if 0 + if (offset < pOffsetInfo->walVerBegin || offset > pOffsetInfo->walVerEnd) { + tscError("consumer:0x:" PRIx64 " invalid seek params, offset:%" PRId64, tmq->consumerId, offset); + return TSDB_CODE_INVALID_PARA; + } + + // update the offset, and then commit to vnode + if (pOffsetInfo->currentOffset.type == TMQ_OFFSET__LOG) { + pOffsetInfo->currentOffset.version = offset; + pOffsetInfo->committedOffset.version = offset; + } + SMqRspObj rspObj = {.resType = RES_TYPE__TMQ, .vgId = pVg->vgId}; tstrncpy(rspObj.topic, pTopicName, tListLen(rspObj.topic)); - tmq_commit_sync(tmq, &rspObj); - // { - // SMqCommitCbParamSet* pParamSet = taosMemoryCalloc(1, sizeof(SMqCommitCbParamSet)); - // if (pParamSet == NULL) { - //// pCommitFp(tmq, TSDB_CODE_OUT_OF_MEMORY, userParam); - // return -1; - // } - // - // pParamSet->refId = tmq->refId; - // pParamSet->epoch = tmq->epoch; - // pParamSet->callbackFn = pCommitFp; - // pParamSet->userParam = userParam; - // - // int32_t numOfTopics = taosArrayGetSize(tmq->clientTopics); - // - // tscDebug("consumer:0x%" PRIx64 " do manual commit offset for %s, vgId:%d", tmq->consumerId, pTopicName, vgId); - // - // int32_t i = 0; - // for (; i < numOfTopics; i++) { - // SMqClientTopic* pTopic = taosArrayGet(tmq->clientTopics, i); - // if (strcmp(pTopic->topicName, pTopicName) == 0) { - // break; - // } - // } - // - // if (i == numOfTopics) { - // tscWarn("consumer:0x%" PRIx64 " failed to find the specified topic:%s, total topics:%d", tmq->consumerId, - // pTopicName, numOfTopics); - // taosMemoryFree(pParamSet); - // pCommitFp(tmq, TSDB_CODE_SUCCESS, userParam); - // return; - // } - // - // SMqClientTopic* pTopic = taosArrayGet(tmq->clientTopics, i); - // - // int32_t j = 0; - // int32_t numOfVgroups = taosArrayGetSize(pTopic->vgs); - // for (j = 0; j < numOfVgroups; j++) { - // SMqClientVg* pVg = taosArrayGet(pTopic->vgs, j); - // if (pVg->vgId == vgId) { - // break; - // } - // } - // - // if (j == numOfVgroups) { - // tscWarn("consumer:0x%" PRIx64 " failed to find the specified vgId:%d, total Vgs:%d, topic:%s", - // tmq->consumerId, - // vgId, numOfVgroups, pTopicName); - // taosMemoryFree(pParamSet); - // pCommitFp(tmq, TSDB_CODE_SUCCESS, userParam); - // return; - // } - // - // SMqClientVg* pVg = taosArrayGet(pTopic->vgs, j); - // if (pVg->offsetInfo.currentOffset.type > 0 && !tOffsetEqual(&pVg->offsetInfo.currentOffset, - // &pVg->offsetInfo.committedOffset)) { - // code = doSendCommitMsg(tmq, pVg, pTopic->topicName, pParamSet, j, numOfVgroups); - // - // // failed to commit, callback user function directly. - // if (code != TSDB_CODE_SUCCESS) { - // taosMemoryFree(pParamSet); - // pCommitFp(tmq, code, userParam); - // } - // } else { // do not perform commit, callback user function directly. - // taosMemoryFree(pParamSet); - // pCommitFp(tmq, code, userParam); - // } - // } - //#endif + tscDebug("consumer:0x%" PRIx64 " seek to %" PRId64 " on vgId:%d", tmq->consumerId, offset, pVg->vgId); + int32_t code = tmq_commit_sync(tmq, &rspObj); + if (code != TSDB_CODE_SUCCESS) { + tscError("consumer:0x%" PRIx64 " failed to send seek to vgId:%d, code:%s", tmq->consumerId, pVg->vgId, + tstrerror(code)); + } + + return code; } \ No newline at end of file From 95346cd4d870f1c9045c5cbcffab295b18267933 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 19 Apr 2023 15:17:46 +0800 Subject: [PATCH 14/20] test:update the test cases. --- source/client/src/clientTmq.c | 4 +--- source/client/test/clientTests.cpp | 9 ++++----- 2 files changed, 5 insertions(+), 8 deletions(-) diff --git a/source/client/src/clientTmq.c b/source/client/src/clientTmq.c index 6f4152ca97..d24fae4211 100644 --- a/source/client/src/clientTmq.c +++ b/source/client/src/clientTmq.c @@ -2334,10 +2334,8 @@ SReqResultInfo* tmqGetNextResInfo(TAOS_RES* res, bool convertUcs4) { return NULL; } - - int32_t tmq_get_topic_assignment(tmq_t* tmq, const char* pTopicName, tmq_topic_assignment** assignment, - int32_t* numOfAssignment) { + int32_t* numOfAssignment) { *numOfAssignment = 0; *assignment = NULL; diff --git a/source/client/test/clientTests.cpp b/source/client/test/clientTests.cpp index 055ac450dc..2674bc0c59 100644 --- a/source/client/test/clientTests.cpp +++ b/source/client/test/clientTests.cpp @@ -112,7 +112,7 @@ void createNewTable(TAOS* pConn, int32_t index) { } taos_free_result(pRes); - for (int32_t i = 0; i < 100; i += 20) { + for (int32_t i = 0; i < 10000; i += 20) { char sql[1024] = {0}; sprintf(sql, "insert into tu%d values(now+%da, %d)(now+%da, %d)(now+%da, %d)(now+%da, %d)" @@ -803,7 +803,7 @@ TEST(clientCase, projection_query_tables) { } taos_free_result(pRes); - for (int32_t i = 0; i < 10000; ++i) { + for (int32_t i = 0; i < 1; ++i) { printf("create table :%d\n", i); createNewTable(pConn, i); } @@ -990,7 +990,7 @@ TEST(clientCase, sub_db_test) { tmq_conf_set(conf, "td.connect.user", "root"); tmq_conf_set(conf, "td.connect.pass", "taosdata"); tmq_conf_set(conf, "auto.offset.reset", "earliest"); - tmq_conf_set(conf, "experimental.snapshot.enable", "true"); + tmq_conf_set(conf, "experimental.snapshot.enable", "false"); tmq_conf_set(conf, "msg.with.table.name", "true"); tmq_conf_set_auto_commit_cb(conf, tmq_commit_cb_print, NULL); @@ -1000,7 +1000,7 @@ TEST(clientCase, sub_db_test) { // 创建订阅 topics 列表 tmq_list_t* topicList = tmq_list_new(); tmq_list_append(topicList, "topic_t1"); - tmq_list_append(topicList, "topic_s2"); +// tmq_list_append(topicList, "topic_s2"); // 启动订阅 tmq_subscribe(tmq, topicList); @@ -1078,7 +1078,6 @@ TEST(clientCase, sub_tb_test) { // 启动订阅 tmq_subscribe(tmq, topicList); - tmq_list_destroy(topicList); TAOS_FIELD* fields = NULL; From c654f1145d7a86688a3abcf98947ee96616ba8f9 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Thu, 20 Apr 2023 10:59:16 +0800 Subject: [PATCH 15/20] enh(stream): add new msg for seek, and do some internal refactor. --- include/common/tmsg.h | 58 ++------ include/common/tmsgdef.h | 1 + include/libs/wal/wal.h | 3 +- source/client/src/clientRawBlockWrite.c | 8 +- source/client/src/clientTmq.c | 80 ++++++++--- source/client/test/clientTests.cpp | 66 +++++---- source/common/src/tmsg.c | 48 ++++++- source/dnode/mgmt/mgmt_vnode/src/vmHandle.c | 1 + source/dnode/mnode/impl/src/mndConsumer.c | 11 +- source/dnode/vnode/src/inc/tq.h | 10 +- source/dnode/vnode/src/inc/vnodeInt.h | 1 + source/dnode/vnode/src/tq/tq.c | 124 +++++++++-------- source/dnode/vnode/src/tq/tqPush.c | 6 +- source/dnode/vnode/src/tq/tqUtil.c | 144 +++++++++++++------- source/dnode/vnode/src/vnd/vnodeSvr.c | 5 + source/libs/wal/src/walRead.c | 7 + 16 files changed, 357 insertions(+), 216 deletions(-) diff --git a/include/common/tmsg.h b/include/common/tmsg.h index bb2450e8f7..9e2dbe2f7a 100644 --- a/include/common/tmsg.h +++ b/include/common/tmsg.h @@ -3101,6 +3101,8 @@ typedef struct { int32_t code; int32_t epoch; int64_t consumerId; + int64_t walsver; + int64_t walever; } SMqRspHead; typedef struct { @@ -3147,43 +3149,9 @@ typedef struct { SSchemaWrapper schema; } SMqSubTopicEp; -static FORCE_INLINE int32_t tEncodeSMqSubTopicEp(void** buf, const SMqSubTopicEp* pTopicEp) { - int32_t tlen = 0; - tlen += taosEncodeString(buf, pTopicEp->topic); - tlen += taosEncodeString(buf, pTopicEp->db); - int32_t sz = taosArrayGetSize(pTopicEp->vgs); - tlen += taosEncodeFixedI32(buf, sz); - for (int32_t i = 0; i < sz; i++) { - SMqSubVgEp* pVgEp = (SMqSubVgEp*)taosArrayGet(pTopicEp->vgs, i); - tlen += tEncodeSMqSubVgEp(buf, pVgEp); - } - tlen += taosEncodeSSchemaWrapper(buf, &pTopicEp->schema); - return tlen; -} - -static FORCE_INLINE void* tDecodeSMqSubTopicEp(void* buf, SMqSubTopicEp* pTopicEp) { - buf = taosDecodeStringTo(buf, pTopicEp->topic); - buf = taosDecodeStringTo(buf, pTopicEp->db); - int32_t sz; - buf = taosDecodeFixedI32(buf, &sz); - pTopicEp->vgs = taosArrayInit(sz, sizeof(SMqSubVgEp)); - if (pTopicEp->vgs == NULL) { - return NULL; - } - for (int32_t i = 0; i < sz; i++) { - SMqSubVgEp vgEp; - buf = tDecodeSMqSubVgEp(buf, &vgEp); - taosArrayPush(pTopicEp->vgs, &vgEp); - } - buf = taosDecodeSSchemaWrapper(buf, &pTopicEp->schema); - return buf; -} - -static FORCE_INLINE void tDeleteSMqSubTopicEp(SMqSubTopicEp* pSubTopicEp) { - taosMemoryFreeClear(pSubTopicEp->schema.pSchema); - pSubTopicEp->schema.nCols = 0; - taosArrayDestroy(pSubTopicEp->vgs); -} +int32_t tEncodeMqSubTopicEp(void** buf, const SMqSubTopicEp* pTopicEp); +void* tDecodeMqSubTopicEp(void* buf, SMqSubTopicEp* pTopicEp); +void tDeleteMqSubTopicEp(SMqSubTopicEp* pSubTopicEp); typedef struct { SMqRspHead head; @@ -3193,8 +3161,8 @@ typedef struct { void* metaRsp; } SMqMetaRsp; -int32_t tEncodeSMqMetaRsp(SEncoder* pEncoder, const SMqMetaRsp* pRsp); -int32_t tDecodeSMqMetaRsp(SDecoder* pDecoder, SMqMetaRsp* pRsp); +int32_t tEncodeMqMetaRsp(SEncoder* pEncoder, const SMqMetaRsp* pRsp); +int32_t tDecodeMqMetaRsp(SDecoder* pDecoder, SMqMetaRsp* pRsp); typedef struct { SMqRspHead head; @@ -3209,9 +3177,9 @@ typedef struct { SArray* blockSchema; } SMqDataRsp; -int32_t tEncodeSMqDataRsp(SEncoder* pEncoder, const SMqDataRsp* pRsp); -int32_t tDecodeSMqDataRsp(SDecoder* pDecoder, SMqDataRsp* pRsp); -void tDeleteSMqDataRsp(SMqDataRsp* pRsp); +int32_t tEncodeMqDataRsp(SEncoder* pEncoder, const SMqDataRsp* pRsp); +int32_t tDecodeMqDataRsp(SDecoder* pDecoder, SMqDataRsp* pRsp); +void tDeleteMqDataRsp(SMqDataRsp* pRsp); typedef struct { SMqRspHead head; @@ -3247,7 +3215,7 @@ static FORCE_INLINE int32_t tEncodeSMqAskEpRsp(void** buf, const SMqAskEpRsp* pR tlen += taosEncodeFixedI32(buf, sz); for (int32_t i = 0; i < sz; i++) { SMqSubTopicEp* pVgEp = (SMqSubTopicEp*)taosArrayGet(pRsp->topics, i); - tlen += tEncodeSMqSubTopicEp(buf, pVgEp); + tlen += tEncodeMqSubTopicEp(buf, pVgEp); } return tlen; } @@ -3262,14 +3230,14 @@ static FORCE_INLINE void* tDecodeSMqAskEpRsp(void* buf, SMqAskEpRsp* pRsp) { } for (int32_t i = 0; i < sz; i++) { SMqSubTopicEp topicEp; - buf = tDecodeSMqSubTopicEp(buf, &topicEp); + buf = tDecodeMqSubTopicEp(buf, &topicEp); taosArrayPush(pRsp->topics, &topicEp); } return buf; } static FORCE_INLINE void tDeleteSMqAskEpRsp(SMqAskEpRsp* pRsp) { - taosArrayDestroyEx(pRsp->topics, (FDelete)tDeleteSMqSubTopicEp); + taosArrayDestroyEx(pRsp->topics, (FDelete)tDeleteMqSubTopicEp); } #define TD_AUTO_CREATE_TABLE 0x1 diff --git a/include/common/tmsgdef.h b/include/common/tmsgdef.h index 6cf6140815..c10e6415e1 100644 --- a/include/common/tmsgdef.h +++ b/include/common/tmsgdef.h @@ -300,6 +300,7 @@ enum { TD_DEF_MSG_TYPE(TDMT_VND_TMQ_SUBSCRIBE, "vnode-tmq-subscribe", SMqRebVgReq, SMqRebVgRsp) TD_DEF_MSG_TYPE(TDMT_VND_TMQ_DELETE_SUB, "vnode-tmq-delete-sub", SMqVDeleteReq, SMqVDeleteRsp) TD_DEF_MSG_TYPE(TDMT_VND_TMQ_COMMIT_OFFSET, "vnode-tmq-commit-offset", STqOffset, STqOffset) + TD_DEF_MSG_TYPE(TDMT_VND_TMQ_SEEK_TO_OFFSET, "vnode-tmq-seekto-offset", STqOffset, STqOffset) TD_DEF_MSG_TYPE(TDMT_VND_TMQ_ADD_CHECKINFO, "vnode-tmq-add-checkinfo", NULL, NULL) TD_DEF_MSG_TYPE(TDMT_VND_TMQ_DEL_CHECKINFO, "vnode-del-checkinfo", NULL, NULL) TD_DEF_MSG_TYPE(TDMT_VND_TMQ_CONSUME, "vnode-tmq-consume", SMqPollReq, SMqDataBlkRsp) diff --git a/include/libs/wal/wal.h b/include/libs/wal/wal.h index b51289de5e..1e3974f5cc 100644 --- a/include/libs/wal/wal.h +++ b/include/libs/wal/wal.h @@ -190,7 +190,7 @@ int32_t walApplyVer(SWal *, int64_t ver); // int32_t walDataCorrupted(SWal*); -// read +// wal reader SWalReader *walOpenReader(SWal *, SWalFilterCond *pCond); void walCloseReader(SWalReader *pRead); void walReadReset(SWalReader *pReader); @@ -198,6 +198,7 @@ int32_t walReadVer(SWalReader *pRead, int64_t ver); int32_t walReadSeekVer(SWalReader *pRead, int64_t ver); int32_t walNextValidMsg(SWalReader *pRead); int64_t walReaderGetCurrentVer(const SWalReader* pReader); +void walReaderValidVersionRange(SWalReader* pReader, int64_t *sver, int64_t *ever); // only for tq usage void walSetReaderCapacity(SWalReader *pRead, int32_t capacity); diff --git a/source/client/src/clientRawBlockWrite.c b/source/client/src/clientRawBlockWrite.c index a09780dc15..32f28e4563 100644 --- a/source/client/src/clientRawBlockWrite.c +++ b/source/client/src/clientRawBlockWrite.c @@ -1511,7 +1511,7 @@ static int32_t tmqWriteRawDataImpl(TAOS* taos, void* data, int32_t dataLen) { rspObj.resType = RES_TYPE__TMQ; tDecoderInit(&decoder, data, dataLen); - code = tDecodeSMqDataRsp(&decoder, &rspObj.rsp); + code = tDecodeMqDataRsp(&decoder, &rspObj.rsp); if (code != 0) { uError("WriteRaw:decode smqDataRsp error"); code = TSDB_CODE_INVALID_MSG; @@ -1615,7 +1615,7 @@ static int32_t tmqWriteRawDataImpl(TAOS* taos, void* data, int32_t dataLen) { code = pRequest->code; end: - tDeleteSMqDataRsp(&rspObj.rsp); + tDeleteMqDataRsp(&rspObj.rsp); tDecoderClear(&decoder); qDestroyQuery(pQuery); destroyRequest(pRequest); @@ -1858,7 +1858,7 @@ int32_t tmq_get_raw(TAOS_RES* res, tmq_raw_data* raw) { int32_t len = 0; int32_t code = 0; - tEncodeSize(tEncodeSMqDataRsp, &rspObj->rsp, len, code); + tEncodeSize(tEncodeMqDataRsp, &rspObj->rsp, len, code); if (code < 0) { return -1; } @@ -1866,7 +1866,7 @@ int32_t tmq_get_raw(TAOS_RES* res, tmq_raw_data* raw) { void* buf = taosMemoryCalloc(1, len); SEncoder encoder = {0}; tEncoderInit(&encoder, buf, len); - tEncodeSMqDataRsp(&encoder, &rspObj->rsp); + tEncodeMqDataRsp(&encoder, &rspObj->rsp); tEncoderClear(&encoder); raw->raw = buf; diff --git a/source/client/src/clientTmq.c b/source/client/src/clientTmq.c index d24fae4211..693daee1fc 100644 --- a/source/client/src/clientTmq.c +++ b/source/client/src/clientTmq.c @@ -225,7 +225,7 @@ static int32_t doAskEp(tmq_t* tmq); static int32_t makeTopicVgroupKey(char* dst, const char* topicName, int32_t vg); static int32_t tmqCommitDone(SMqCommitCbParamSet* pParamSet); static int32_t doSendCommitMsg(tmq_t* tmq, SMqClientVg* pVg, const char* pTopicName, SMqCommitCbParamSet* pParamSet, - int32_t index, int32_t totalVgroups); + int32_t index, int32_t totalVgroups, int32_t type); static void commitRspCountDown(SMqCommitCbParamSet* pParamSet, int64_t consumerId, const char* pTopic, int32_t vgId); static void asyncAskEp(tmq_t* pTmq, __tmq_askep_fn_t askEpFn, void* param); static void addToQueueCallbackFn(tmq_t* pTmq, int32_t code, SDataBuf* pDataBuf, void* param); @@ -473,7 +473,7 @@ static int32_t tmqCommitCb(void* param, SDataBuf* pBuf, int32_t code) { } static int32_t doSendCommitMsg(tmq_t* tmq, SMqClientVg* pVg, const char* pTopicName, SMqCommitCbParamSet* pParamSet, - int32_t index, int32_t totalVgroups) { + int32_t index, int32_t totalVgroups, int32_t type) { STqOffset* pOffset = taosMemoryCalloc(1, sizeof(STqOffset)); if (pOffset == NULL) { return TSDB_CODE_OUT_OF_MEMORY; @@ -539,7 +539,7 @@ static int32_t doSendCommitMsg(tmq_t* tmq, SMqClientVg* pVg, const char* pTopicN pMsgSendInfo->param = pParam; pMsgSendInfo->paramFreeFp = taosMemoryFree; pMsgSendInfo->fp = tmqCommitCb; - pMsgSendInfo->msgType = TDMT_VND_TMQ_COMMIT_OFFSET; + pMsgSendInfo->msgType = type; atomic_add_fetch_32(&pParamSet->waitingRspNum, 1); atomic_add_fetch_32(&pParamSet->totalRspNum, 1); @@ -575,7 +575,7 @@ static SMqClientTopic* getTopicByName(tmq_t* tmq, const char* pTopicName) { return NULL; } -static void asyncCommitOffset(tmq_t* tmq, const TAOS_RES* pRes, tmq_commit_cb* pCommitFp, void* userParam) { +static void asyncCommitOffset(tmq_t* tmq, const TAOS_RES* pRes, int32_t type, tmq_commit_cb* pCommitFp, void* userParam) { char* pTopicName = NULL; int32_t vgId = 0; int32_t code = 0; @@ -645,7 +645,7 @@ static void asyncCommitOffset(tmq_t* tmq, const TAOS_RES* pRes, tmq_commit_cb* p SMqClientVg* pVg = taosArrayGet(pTopic->vgs, j); if (pVg->offsetInfo.currentOffset.type > 0 && !tOffsetEqual(&pVg->offsetInfo.currentOffset, &pVg->offsetInfo.committedOffset)) { - code = doSendCommitMsg(tmq, pVg, pTopic->topicName, pParamSet, j, numOfVgroups); + code = doSendCommitMsg(tmq, pVg, pTopic->topicName, pParamSet, j, numOfVgroups, type); // failed to commit, callback user function directly. if (code != TSDB_CODE_SUCCESS) { @@ -686,7 +686,7 @@ static void asyncCommitAllOffsets(tmq_t* tmq, tmq_commit_cb* pCommitFp, void* us SMqClientVg* pVg = taosArrayGet(pTopic->vgs, j); if (pVg->offsetInfo.currentOffset.type > 0 && !tOffsetEqual(&pVg->offsetInfo.currentOffset, &pVg->offsetInfo.committedOffset)) { - int32_t code = doSendCommitMsg(tmq, pVg, pTopic->topicName, pParamSet, j, numOfVgroups); + int32_t code = doSendCommitMsg(tmq, pVg, pTopic->topicName, pParamSet, j, numOfVgroups, TDMT_VND_TMQ_COMMIT_OFFSET); if (code != TSDB_CODE_SUCCESS) { tscError("consumer:0x%" PRIx64 " topic:%s vgId:%d offset:%" PRId64 " failed, code:%s ordinal:%d/%d", tmq->consumerId, pTopic->topicName, pVg->vgId, pVg->offsetInfo.committedOffset.version, tstrerror(terrno), @@ -1328,7 +1328,7 @@ int32_t tmqPollCb(void* param, SDataBuf* pMsg, int32_t code) { if (rspType == TMQ_MSG_TYPE__POLL_RSP) { SDecoder decoder; tDecoderInit(&decoder, POINTER_SHIFT(pMsg->pData, sizeof(SMqRspHead)), pMsg->len - sizeof(SMqRspHead)); - tDecodeSMqDataRsp(&decoder, &pRspWrapper->dataRsp); + tDecodeMqDataRsp(&decoder, &pRspWrapper->dataRsp); tDecoderClear(&decoder); memcpy(&pRspWrapper->dataRsp, pMsg->pData, sizeof(SMqRspHead)); @@ -1339,7 +1339,7 @@ int32_t tmqPollCb(void* param, SDataBuf* pMsg, int32_t code) { } else if (rspType == TMQ_MSG_TYPE__POLL_META_RSP) { SDecoder decoder; tDecoderInit(&decoder, POINTER_SHIFT(pMsg->pData, sizeof(SMqRspHead)), pMsg->len - sizeof(SMqRspHead)); - tDecodeSMqMetaRsp(&decoder, &pRspWrapper->metaRsp); + tDecodeMqMetaRsp(&decoder, &pRspWrapper->metaRsp); tDecoderClear(&decoder); memcpy(&pRspWrapper->metaRsp, pMsg->pData, sizeof(SMqRspHead)); } else if (rspType == TMQ_MSG_TYPE__TAOSX_RSP) { @@ -1808,8 +1808,14 @@ static void* tmqHandleAllRsp(tmq_t* tmq, int64_t timeout, bool pollIfReset) { // update the local offset value only for the returned values. pVg->offsetInfo.currentOffset = pDataRsp->rspOffset; + + // update the status atomic_store_32(&pVg->vgStatus, TMQ_VG_STATUS__IDLE); + // update the valid wal version range + pVg->offsetInfo.walVerBegin = pDataRsp->head.walsver; + pVg->offsetInfo.walVerEnd = pDataRsp->head.walever; + char buf[80]; tFormatOffset(buf, 80, &pDataRsp->rspOffset); if (pDataRsp->blockNum == 0) { @@ -1837,6 +1843,7 @@ static void* tmqHandleAllRsp(tmq_t* tmq, int64_t timeout, bool pollIfReset) { taosFreeQitem(pollRspWrapper); } } else if (pRspWrapper->tmqRspType == TMQ_MSG_TYPE__POLL_META_RSP) { + // todo handle the wal range and epset for each vgroup SMqPollRspWrapper* pollRspWrapper = (SMqPollRspWrapper*)pRspWrapper; int32_t consumerEpoch = atomic_load_32(&tmq->epoch); @@ -2128,11 +2135,11 @@ void tmq_commit_async(tmq_t* tmq, const TAOS_RES* pRes, tmq_commit_cb* cb, void* if (pRes == NULL) { // here needs to commit all offsets. asyncCommitAllOffsets(tmq, cb, param); } else { // only commit one offset - asyncCommitOffset(tmq, pRes, cb, param); + asyncCommitOffset(tmq, pRes, TDMT_VND_TMQ_COMMIT_OFFSET, cb, param); } } -static void commitCallBackFn(tmq_t *pTmq, int32_t code, void* param) { +static void commitCallBackFn(tmq_t *UNUSED_PARAM(pTmq), int32_t code, void* param) { SSyncCommitInfo* pInfo = (SSyncCommitInfo*) param; pInfo->code = code; tsem_post(&pInfo->sem); @@ -2148,7 +2155,7 @@ int32_t tmq_commit_sync(tmq_t* tmq, const TAOS_RES* pRes) { if (pRes == NULL) { asyncCommitAllOffsets(tmq, commitCallBackFn, pInfo); } else { - asyncCommitOffset(tmq, pRes, commitCallBackFn, pInfo); + asyncCommitOffset(tmq, pRes, TDMT_VND_TMQ_COMMIT_OFFSET, commitCallBackFn, pInfo); } tsem_wait(&pInfo->sem); @@ -2339,13 +2346,25 @@ int32_t tmq_get_topic_assignment(tmq_t* tmq, const char* pTopicName, tmq_topic_a *numOfAssignment = 0; *assignment = NULL; - SMqClientTopic* pTopic = getTopicByName(tmq, pTopicName); + int32_t accId = tmq->pTscObj->acctId; + char tname[128] = {0}; + sprintf(tname, "%d.%s", accId, pTopicName); + + SMqClientTopic* pTopic = getTopicByName(tmq, tname); if (pTopic == NULL) { return TSDB_CODE_INVALID_PARA; } // in case of snapshot is opened, no valid offset will return *numOfAssignment = taosArrayGetSize(pTopic->vgs); + + *assignment = taosMemoryCalloc(*numOfAssignment, sizeof(tmq_topic_assignment)); + if (*assignment == NULL) { + tscError("consumer:0x%" PRIx64 " failed to malloc buffer, size:%" PRIzu, tmq->consumerId, + (*numOfAssignment) * sizeof(tmq_topic_assignment)); + return TSDB_CODE_OUT_OF_MEMORY; + } + for (int32_t j = 0; j < (*numOfAssignment); ++j) { SMqClientVg* pClientVg = taosArrayGet(pTopic->vgs, j); @@ -2370,9 +2389,13 @@ int32_t tmq_offset_seek(tmq_t* tmq, const char* pTopicName, int32_t vgroupHandle return TSDB_CODE_INVALID_PARA; } - SMqClientTopic* pTopic = getTopicByName(tmq, pTopicName); + int32_t accId = tmq->pTscObj->acctId; + char tname[128] = {0}; + sprintf(tname, "%d.%s", accId, pTopicName); + + SMqClientTopic* pTopic = getTopicByName(tmq, tname); if (pTopic == NULL) { - tscError("consumer:0x:" PRIx64 " invalid topic name:%s", tmq->consumerId, pTopicName); + tscError("consumer:0x%" PRIx64 " invalid topic name:%s", tmq->consumerId, pTopicName); return TSDB_CODE_INVALID_PARA; } @@ -2387,7 +2410,7 @@ int32_t tmq_offset_seek(tmq_t* tmq, const char* pTopicName, int32_t vgroupHandle } if (pVg == NULL) { - tscError("consumer:0x:" PRIx64 " invalid vgroup id:%d", tmq->consumerId, vgroupHandle); + tscError("consumer:0x%" PRIx64 " invalid vgroup id:%d", tmq->consumerId, vgroupHandle); return TSDB_CODE_INVALID_PARA; } @@ -2395,26 +2418,43 @@ int32_t tmq_offset_seek(tmq_t* tmq, const char* pTopicName, int32_t vgroupHandle int32_t type = pOffsetInfo->currentOffset.type; if (type != TMQ_OFFSET__LOG) { - tscError("consumer:0x:" PRIx64 " offset type:%d not wal version, seek not allowed", tmq->consumerId, type); + tscError("consumer:0x%" PRIx64 " offset type:%d not wal version, seek not allowed", tmq->consumerId, type); return TSDB_CODE_INVALID_PARA; } if (offset < pOffsetInfo->walVerBegin || offset > pOffsetInfo->walVerEnd) { - tscError("consumer:0x:" PRIx64 " invalid seek params, offset:%" PRId64, tmq->consumerId, offset); + tscError("consumer:0x%" PRIx64 " invalid seek params, offset:%" PRId64, tmq->consumerId, offset); return TSDB_CODE_INVALID_PARA; } // update the offset, and then commit to vnode if (pOffsetInfo->currentOffset.type == TMQ_OFFSET__LOG) { pOffsetInfo->currentOffset.version = offset; - pOffsetInfo->committedOffset.version = offset; + pOffsetInfo->committedOffset.version = INT64_MIN; } SMqRspObj rspObj = {.resType = RES_TYPE__TMQ, .vgId = pVg->vgId}; - tstrncpy(rspObj.topic, pTopicName, tListLen(rspObj.topic)); + tstrncpy(rspObj.topic, tname, tListLen(rspObj.topic)); tscDebug("consumer:0x%" PRIx64 " seek to %" PRId64 " on vgId:%d", tmq->consumerId, offset, pVg->vgId); - int32_t code = tmq_commit_sync(tmq, &rspObj); + + SSyncCommitInfo* pInfo = taosMemoryMalloc(sizeof(SSyncCommitInfo)); + if (pInfo == NULL) { + tscError("consumer:0x%"PRIx64" failed to prepare seek operation", tmq->consumerId); + return TSDB_CODE_OUT_OF_MEMORY; + } + + tsem_init(&pInfo->sem, 0, 0); + pInfo->code = 0; + + asyncCommitOffset(tmq, &rspObj, TDMT_VND_TMQ_SEEK_TO_OFFSET, commitCallBackFn, pInfo); + + tsem_wait(&pInfo->sem); + int32_t code = pInfo->code; + + tsem_destroy(&pInfo->sem); + taosMemoryFree(pInfo); + if (code != TSDB_CODE_SUCCESS) { tscError("consumer:0x%" PRIx64 " failed to send seek to vgId:%d, code:%s", tmq->consumerId, pVg->vgId, tstrerror(code)); diff --git a/source/client/test/clientTests.cpp b/source/client/test/clientTests.cpp index 2674bc0c59..734577bd1d 100644 --- a/source/client/test/clientTests.cpp +++ b/source/client/test/clientTests.cpp @@ -30,6 +30,27 @@ #include "taos.h" namespace { + +void printSubResults(void* pRes, int32_t* totalRows) { + char buf[1024]; + + while (1) { + TAOS_ROW row = taos_fetch_row(pRes); + if (row == NULL) { + break; + } + + TAOS_FIELD* fields = taos_fetch_fields(pRes); + int32_t numOfFields = taos_field_count(pRes); + int32_t precision = taos_result_precision(pRes); + taos_print_row(buf, row, fields, numOfFields); + *totalRows += 1; + printf("precision: %d, row content: %s\n", precision, buf); + } + +// taos_free_result(pRes); +} + void showDB(TAOS* pConn) { TAOS_RES* pRes = taos_query(pConn, "show databases"); TAOS_ROW pRow = NULL; @@ -1059,13 +1080,13 @@ TEST(clientCase, sub_tb_test) { ASSERT_NE(pConn, nullptr); tmq_conf_t* conf = tmq_conf_new(); - tmq_conf_set(conf, "enable.auto.commit", "true"); + tmq_conf_set(conf, "enable.auto.commit", "false"); tmq_conf_set(conf, "auto.commit.interval.ms", "1000"); tmq_conf_set(conf, "group.id", "cgrpName45"); tmq_conf_set(conf, "td.connect.user", "root"); tmq_conf_set(conf, "td.connect.pass", "taosdata"); tmq_conf_set(conf, "auto.offset.reset", "earliest"); - tmq_conf_set(conf, "experimental.snapshot.enable", "true"); + tmq_conf_set(conf, "experimental.snapshot.enable", "false"); tmq_conf_set(conf, "msg.with.table.name", "true"); tmq_conf_set_auto_commit_cb(conf, tmq_commit_cb_print, NULL); @@ -1074,7 +1095,7 @@ TEST(clientCase, sub_tb_test) { // 创建订阅 topics 列表 tmq_list_t* topicList = tmq_list_new(); - tmq_list_append(topicList, "topic_t2"); + tmq_list_append(topicList, "topic_t1"); // 启动订阅 tmq_subscribe(tmq, topicList); @@ -1089,11 +1110,15 @@ TEST(clientCase, sub_tb_test) { int32_t count = 0; + tmq_topic_assignment* pAssign = NULL; + int32_t numOfAssign = 0; + + TAOS_RES* p = tmq_consumer_poll(tmq, timeout); + int32_t code = tmq_get_topic_assignment(tmq, "topic_t1", &pAssign, &numOfAssign); + while (1) { TAOS_RES* pRes = tmq_consumer_poll(tmq, timeout); - if (pRes) { - char buf[1024]; - + if (pRes != NULL) { const char* topicName = tmq_get_topic_name(pRes); const char* dbName = tmq_get_db_name(pRes); int32_t vgroupId = tmq_get_vgroup_id(pRes); @@ -1102,27 +1127,18 @@ TEST(clientCase, sub_tb_test) { printf("db: %s\n", dbName); printf("vgroup id: %d\n", vgroupId); - while (1) { - TAOS_ROW row = taos_fetch_row(pRes); - if (row == NULL) { - break; - } - - fields = taos_fetch_fields(pRes); - numOfFields = taos_field_count(pRes); - precision = taos_result_precision(pRes); - taos_print_row(buf, row, fields, numOfFields); - totalRows += 1; - printf("precision: %d, row content: %s\n", precision, buf); - } - - taos_free_result(pRes); - // if ((++count) > 1) { - // break; - // } + printSubResults(pRes, &totalRows); } else { - break; +// tmq_offset_seek(tmq, "topic_t1", pAssign[0].vgroupHandle, pAssign[0].begin); +// break; } + + tmq_commit_sync(tmq, pRes); + if (pRes != NULL) { + taos_free_result(pRes); + } + + tmq_offset_seek(tmq, "topic_t1", pAssign[0].vgroupHandle, pAssign[0].begin); } tmq_consumer_close(tmq); diff --git a/source/common/src/tmsg.c b/source/common/src/tmsg.c index 0a18941780..70c9ae4732 100644 --- a/source/common/src/tmsg.c +++ b/source/common/src/tmsg.c @@ -6973,21 +6973,21 @@ int32_t tDecodeDeleteRes(SDecoder *pCoder, SDeleteRes *pRes) { return 0; } -int32_t tEncodeSMqMetaRsp(SEncoder *pEncoder, const SMqMetaRsp *pRsp) { +int32_t tEncodeMqMetaRsp(SEncoder *pEncoder, const SMqMetaRsp *pRsp) { if (tEncodeSTqOffsetVal(pEncoder, &pRsp->rspOffset) < 0) return -1; if (tEncodeI16(pEncoder, pRsp->resMsgType)) return -1; if (tEncodeBinary(pEncoder, pRsp->metaRsp, pRsp->metaRspLen)) return -1; return 0; } -int32_t tDecodeSMqMetaRsp(SDecoder *pDecoder, SMqMetaRsp *pRsp) { +int32_t tDecodeMqMetaRsp(SDecoder *pDecoder, SMqMetaRsp *pRsp) { if (tDecodeSTqOffsetVal(pDecoder, &pRsp->rspOffset) < 0) return -1; if (tDecodeI16(pDecoder, &pRsp->resMsgType) < 0) return -1; if (tDecodeBinaryAlloc(pDecoder, &pRsp->metaRsp, (uint64_t *)&pRsp->metaRspLen) < 0) return -1; return 0; } -int32_t tEncodeSMqDataRsp(SEncoder *pEncoder, const SMqDataRsp *pRsp) { +int32_t tEncodeMqDataRsp(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; @@ -7012,7 +7012,7 @@ int32_t tEncodeSMqDataRsp(SEncoder *pEncoder, const SMqDataRsp *pRsp) { return 0; } -int32_t tDecodeSMqDataRsp(SDecoder *pDecoder, SMqDataRsp *pRsp) { +int32_t tDecodeMqDataRsp(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; @@ -7057,7 +7057,7 @@ int32_t tDecodeSMqDataRsp(SDecoder *pDecoder, SMqDataRsp *pRsp) { return 0; } -void tDeleteSMqDataRsp(SMqDataRsp *pRsp) { +void tDeleteMqDataRsp(SMqDataRsp *pRsp) { pRsp->blockDataLen = taosArrayDestroy(pRsp->blockDataLen); taosArrayDestroyP(pRsp->blockData, (FDelete)taosMemoryFree); pRsp->blockData = NULL; @@ -7539,3 +7539,41 @@ void tDestroySSubmitRsp2(SSubmitRsp2 *pRsp, int32_t flag) { } } } + +int32_t tEncodeMqSubTopicEp(void **buf, const SMqSubTopicEp *pTopicEp) { + int32_t tlen = 0; + tlen += taosEncodeString(buf, pTopicEp->topic); + tlen += taosEncodeString(buf, pTopicEp->db); + int32_t sz = taosArrayGetSize(pTopicEp->vgs); + tlen += taosEncodeFixedI32(buf, sz); + for (int32_t i = 0; i < sz; i++) { + SMqSubVgEp *pVgEp = (SMqSubVgEp *)taosArrayGet(pTopicEp->vgs, i); + tlen += tEncodeSMqSubVgEp(buf, pVgEp); + } + tlen += taosEncodeSSchemaWrapper(buf, &pTopicEp->schema); + return tlen; +} + +void *tDecodeMqSubTopicEp(void *buf, SMqSubTopicEp *pTopicEp) { + buf = taosDecodeStringTo(buf, pTopicEp->topic); + buf = taosDecodeStringTo(buf, pTopicEp->db); + int32_t sz; + buf = taosDecodeFixedI32(buf, &sz); + pTopicEp->vgs = taosArrayInit(sz, sizeof(SMqSubVgEp)); + if (pTopicEp->vgs == NULL) { + return NULL; + } + for (int32_t i = 0; i < sz; i++) { + SMqSubVgEp vgEp; + buf = tDecodeSMqSubVgEp(buf, &vgEp); + taosArrayPush(pTopicEp->vgs, &vgEp); + } + buf = taosDecodeSSchemaWrapper(buf, &pTopicEp->schema); + return buf; +} + +void tDeleteMqSubTopicEp(SMqSubTopicEp *pSubTopicEp) { + taosMemoryFreeClear(pSubTopicEp->schema.pSchema); + pSubTopicEp->schema.nCols = 0; + taosArrayDestroy(pSubTopicEp->vgs); +} \ No newline at end of file diff --git a/source/dnode/mgmt/mgmt_vnode/src/vmHandle.c b/source/dnode/mgmt/mgmt_vnode/src/vmHandle.c index d61eb3ec03..7323d23486 100644 --- a/source/dnode/mgmt/mgmt_vnode/src/vmHandle.c +++ b/source/dnode/mgmt/mgmt_vnode/src/vmHandle.c @@ -517,6 +517,7 @@ SArray *vmGetMsgHandles() { if (dmSetMgmtHandle(pArray, TDMT_VND_TMQ_SUBSCRIBE, vmPutMsgToWriteQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_VND_TMQ_DELETE_SUB, vmPutMsgToWriteQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_VND_TMQ_COMMIT_OFFSET, vmPutMsgToWriteQueue, 0) == NULL) goto _OVER; + if (dmSetMgmtHandle(pArray, TDMT_VND_TMQ_SEEK_TO_OFFSET, vmPutMsgToWriteQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_VND_TMQ_ADD_CHECKINFO, vmPutMsgToWriteQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_VND_TMQ_DEL_CHECKINFO, vmPutMsgToWriteQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_VND_TMQ_CONSUME, vmPutMsgToFetchQueue, 0) == NULL) goto _OVER; diff --git a/source/dnode/mnode/impl/src/mndConsumer.c b/source/dnode/mnode/impl/src/mndConsumer.c index 1b146506a2..0d75b5fd68 100644 --- a/source/dnode/mnode/impl/src/mndConsumer.c +++ b/source/dnode/mnode/impl/src/mndConsumer.c @@ -556,9 +556,14 @@ static int32_t mndProcessAskEpReq(SRpcMsg *pMsg) { return -1; } - ((SMqRspHead *)buf)->mqMsgType = TMQ_MSG_TYPE__EP_RSP; - ((SMqRspHead *)buf)->epoch = serverEpoch; - ((SMqRspHead *)buf)->consumerId = pConsumer->consumerId; + SMqRspHead* pHead = buf; + + pHead->mqMsgType = TMQ_MSG_TYPE__EP_RSP; + pHead->epoch = serverEpoch; + pHead->consumerId = pConsumer->consumerId; + pHead->walsver = 0; + pHead->walever = 0; + void *abuf = POINTER_SHIFT(buf, sizeof(SMqRspHead)); tEncodeSMqAskEpRsp(&abuf, &rsp); diff --git a/source/dnode/vnode/src/inc/tq.h b/source/dnode/vnode/src/inc/tq.h index db17e4f533..5a6f5249b3 100644 --- a/source/dnode/vnode/src/inc/tq.h +++ b/source/dnode/vnode/src/inc/tq.h @@ -106,6 +106,7 @@ typedef struct { SMqDataRsp* pDataRsp; char subKey[TSDB_SUBSCRIBE_KEY_LEN]; SRpcHandleInfo info; + STqHandle* pHandle; } STqPushEntry; struct STQ { @@ -145,8 +146,9 @@ int32_t tqFetchLog(STQ* pTq, STqHandle* pHandle, int64_t* fetchOffset, SWalCkHea // tqExec int32_t tqTaosxScanLog(STQ* pTq, STqHandle* pHandle, SPackedData submit, STaosxRsp* pRsp, int32_t* totalRows); int32_t tqAddBlockDataToRsp(const SSDataBlock* pBlock, SMqDataRsp* pRsp, int32_t numOfCols, int8_t precision); -int32_t tqSendDataRsp(STQ* pTq, const SRpcMsg* pMsg, const SMqPollReq* pReq, const SMqDataRsp* pRsp, int32_t type); -int32_t tqPushDataRsp(STQ* pTq, STqPushEntry* pPushEntry); +int32_t tqSendDataRsp(STqHandle* pHandle, const SRpcMsg* pMsg, const SMqPollReq* pReq, const SMqDataRsp* pRsp, + int32_t type, int32_t vgId); +int32_t tqPushDataRsp(STqPushEntry* pPushEntry, int32_t vgId); // tqMeta int32_t tqMetaOpen(STQ* pTq); @@ -182,13 +184,13 @@ int32_t tqStreamTasksScanWal(STQ* pTq); char* createStreamTaskIdStr(int64_t streamId, int32_t taskId); void createStreamTaskOffsetKey(char* dst, uint64_t streamId, uint32_t taskId); int32_t tqAddInputBlockNLaunchTask(SStreamTask* pTask, SStreamQueueItem* pQueueItem, int64_t ver); -int32_t launchTaskForWalBlock(SStreamTask* pTask, SFetchRet* pRet, STqOffset* pOffset); int32_t tqExtractDataForMq(STQ* pTq, STqHandle* pHandle, const SMqPollReq* pRequest, SRpcMsg* pMsg); +int32_t tqDoSendDataRsp(const SRpcHandleInfo* pRpcHandleInfo, const SMqDataRsp* pRsp, int32_t epoch, int64_t consumerId, + int32_t type, int64_t sver, int64_t ever); void doSaveTaskOffset(STqOffsetStore* pOffsetStore, const char* pKey, int64_t ver); void saveOffsetForAllTasks(STQ* pTq, int64_t ver); void initOffsetForAllRestoreTasks(STQ* pTq); -int32_t transferToWalReadTask(SStreamMeta* pStreamMeta, SArray* pTaskList); #ifdef __cplusplus } diff --git a/source/dnode/vnode/src/inc/vnodeInt.h b/source/dnode/vnode/src/inc/vnodeInt.h index 81f7c3d52a..764e57eb09 100644 --- a/source/dnode/vnode/src/inc/vnodeInt.h +++ b/source/dnode/vnode/src/inc/vnodeInt.h @@ -206,6 +206,7 @@ int32_t tqProcessDelCheckInfoReq(STQ* pTq, int64_t version, char* msg, int32_t m int32_t tqProcessSubscribeReq(STQ* pTq, int64_t version, char* msg, int32_t msgLen); int32_t tqProcessDeleteSubReq(STQ* pTq, int64_t version, char* msg, int32_t msgLen); int32_t tqProcessOffsetCommitReq(STQ* pTq, int64_t version, char* msg, int32_t msgLen); +int32_t tqProcessSeekReq(STQ* pTq, int64_t sversion, char* msg, int32_t msgLen); int32_t tqProcessPollReq(STQ* pTq, SRpcMsg* pMsg); // tq-stream int32_t tqProcessTaskDeployReq(STQ* pTq, int64_t version, char* msg, int32_t msgLen); diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 7c7f59b6b7..5e4ede914a 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -76,7 +76,7 @@ static void destroyTqHandle(void* data) { static void tqPushEntryFree(void* data) { STqPushEntry* p = *(void**)data; if (p->pDataRsp->head.mqMsgType == TMQ_MSG_TYPE__POLL_RSP) { - tDeleteSMqDataRsp(p->pDataRsp); + tDeleteMqDataRsp(p->pDataRsp); } else if (p->pDataRsp->head.mqMsgType == TMQ_MSG_TYPE__TAOSX_RSP) { tDeleteSTaosxRsp((STaosxRsp*)p->pDataRsp); } @@ -154,71 +154,30 @@ void tqClose(STQ* pTq) { taosMemoryFree(pTq); } -static int32_t doSendDataRsp(const SRpcHandleInfo* pRpcHandleInfo, const SMqDataRsp* pRsp, int32_t epoch, - int64_t consumerId, int32_t type) { - int32_t len = 0; - int32_t code = 0; - - if (type == TMQ_MSG_TYPE__POLL_RSP) { - tEncodeSize(tEncodeSMqDataRsp, pRsp, len, code); - } else if (type == TMQ_MSG_TYPE__TAOSX_RSP) { - tEncodeSize(tEncodeSTaosxRsp, (STaosxRsp*)pRsp, len, code); - } - - if (code < 0) { - return -1; - } - - int32_t tlen = sizeof(SMqRspHead) + len; - void* buf = rpcMallocCont(tlen); - if (buf == NULL) { - return -1; - } - - ((SMqRspHead*)buf)->mqMsgType = type; - ((SMqRspHead*)buf)->epoch = epoch; - ((SMqRspHead*)buf)->consumerId = consumerId; - - void* abuf = POINTER_SHIFT(buf, sizeof(SMqRspHead)); - - SEncoder encoder = {0}; - tEncoderInit(&encoder, abuf, len); - - if (type == TMQ_MSG_TYPE__POLL_RSP) { - tEncodeSMqDataRsp(&encoder, pRsp); - } else if (type == TMQ_MSG_TYPE__TAOSX_RSP) { - tEncodeSTaosxRsp(&encoder, (STaosxRsp*)pRsp); - } - - tEncoderClear(&encoder); - - SRpcMsg rsp = { - .info = *pRpcHandleInfo, - .pCont = buf, - .contLen = tlen, - .code = 0, - }; - - tmsgSendRsp(&rsp); - return 0; -} - -int32_t tqPushDataRsp(STQ* pTq, STqPushEntry* pPushEntry) { +int32_t tqPushDataRsp(STqPushEntry* pPushEntry, int32_t vgId) { SMqDataRsp* pRsp = pPushEntry->pDataRsp; SMqRspHead* pHeader = &pPushEntry->pDataRsp->head; - doSendDataRsp(&pPushEntry->info, pRsp, pHeader->epoch, pHeader->consumerId, pHeader->mqMsgType); + + int64_t sver = 0, ever = 0; + walReaderValidVersionRange(pPushEntry->pHandle->execHandle.pTqReader->pWalReader, &sver, &ever); + + tqDoSendDataRsp(&pPushEntry->info, pRsp, pHeader->epoch, pHeader->consumerId, pHeader->mqMsgType, sver, ever); char buf1[80] = {0}; char buf2[80] = {0}; tFormatOffset(buf1, tListLen(buf1), &pRsp->reqOffset); tFormatOffset(buf2, tListLen(buf2), &pRsp->rspOffset); tqDebug("vgId:%d, from consumer:0x%" PRIx64 " (epoch %d) push rsp, block num: %d, req:%s, rsp:%s", - TD_VID(pTq->pVnode), pRsp->head.consumerId, pRsp->head.epoch, pRsp->blockNum, buf1, buf2); + vgId, pRsp->head.consumerId, pRsp->head.epoch, pRsp->blockNum, buf1, buf2); return 0; } -int32_t tqSendDataRsp(STQ* pTq, const SRpcMsg* pMsg, const SMqPollReq* pReq, const SMqDataRsp* pRsp, int32_t type) { - doSendDataRsp(&pMsg->info, pRsp, pReq->epoch, pReq->consumerId, 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); + + tqDoSendDataRsp(&pMsg->info, pRsp, pReq->epoch, pReq->consumerId, type, sver, ever); char buf1[80] = {0}; char buf2[80] = {0}; @@ -226,7 +185,7 @@ int32_t tqSendDataRsp(STQ* pTq, const SRpcMsg* pMsg, const SMqPollReq* pReq, con tFormatOffset(buf2, 80, &pRsp->rspOffset); tqDebug("vgId:%d consumer:0x%" PRIx64 " (epoch %d) send rsp, block num:%d, req:%s, rsp:%s, reqId:0x%" PRIx64, - TD_VID(pTq->pVnode), pReq->consumerId, pReq->epoch, pRsp->blockNum, buf1, buf2, pReq->reqId); + vgId, pReq->consumerId, pReq->epoch, pRsp->blockNum, buf1, buf2, pReq->reqId); return 0; } @@ -259,6 +218,8 @@ int32_t tqProcessOffsetCommitReq(STQ* pTq, int64_t sversion, char* msg, int32_t STqOffset* pSavedOffset = tqOffsetRead(pTq->pOffsetStore, offset.subKey); if (pSavedOffset != NULL && tqOffsetLessOrEqual(&offset, pSavedOffset)) { + tqDebug("not update the offset, vgId:%d sub:%s since committed:%" PRId64 " less than/equal to existed:%" PRId64, + vgId, offset.subKey, offset.val.version, pSavedOffset->val.version); return 0; // no need to update the offset value } @@ -277,6 +238,57 @@ int32_t tqProcessOffsetCommitReq(STQ* pTq, int64_t sversion, char* msg, int32_t return 0; } +int32_t tqProcessSeekReq(STQ* pTq, int64_t sversion, char* msg, int32_t msgLen) { + STqOffset offset = {0}; + int32_t vgId = TD_VID(pTq->pVnode); + + SDecoder decoder; + tDecoderInit(&decoder, (uint8_t*)msg, msgLen); + if (tDecodeSTqOffset(&decoder, &offset) < 0) { + return -1; + } + + tDecoderClear(&decoder); + + if (offset.val.type != TMQ_OFFSET__LOG) { + tqError("vgId:%d, subKey:%s invalid seek offset type:%d", vgId, offset.subKey, offset.val.type); + return -1; + } + + STqOffset* pSavedOffset = tqOffsetRead(pTq->pOffsetStore, offset.subKey); + if (pSavedOffset != NULL && pSavedOffset->val.type != TMQ_OFFSET__LOG) { + tqError("invalid saved offset type, vgId:%d sub:%s", vgId, offset.subKey); + return 0; // no need to update the offset value + } + + if (pSavedOffset->val.version == offset.val.version) { + tqDebug("vgId:%d subKey:%s no need to seek to %" PRId64 " prev offset:%" PRId64, vgId, offset.subKey, offset.val.version, + pSavedOffset->val.version); + return 0; + } + + STqHandle* pHandle = taosHashGet(pTq->pHandle, offset.subKey, strlen(offset.subKey)); + + int64_t sver = 0, ever = 0; + walReaderValidVersionRange(pHandle->execHandle.pTqReader->pWalReader, &sver, &ever); + if (offset.val.version < sver) { + offset.val.version = sver; + } else if (offset.val.version > ever) { + offset.val.version = ever; + } + + // save the new offset value + tqDebug("vgId:%d sub:%s seek to %" PRId64 " prev offset:%" PRId64, vgId, offset.subKey, offset.val.version, + pSavedOffset->val.version); + + if (tqOffsetWrite(pTq->pOffsetStore, &offset) < 0) { + tqError("failed to save offset, vgId:%d sub:%s seek to %" PRId64, vgId, offset.subKey, offset.val.version); + return -1; + } + + return 0; +} + int32_t tqCheckColModifiable(STQ* pTq, int64_t tbUid, int32_t colId) { void* pIter = NULL; diff --git a/source/dnode/vnode/src/tq/tqPush.c b/source/dnode/vnode/src/tq/tqPush.c index 7a1a6b7454..43463f67b7 100644 --- a/source/dnode/vnode/src/tq/tqPush.c +++ b/source/dnode/vnode/src/tq/tqPush.c @@ -263,7 +263,7 @@ static void doPushDataForEntry(void* pIter, STqExecHandle* pExec, STQ* pTq, int6 if (pRsp->blockNum > 0) { tqOffsetResetToLog(&pRsp->rspOffset, ver); - tqPushDataRsp(pTq, pPushEntry); + tqPushDataRsp(pPushEntry, vgId); recordPushedEntry(pCachedKey, pIter); } } @@ -376,6 +376,7 @@ int32_t tqRegisterPushHandle(STQ* pTq, void* pHandle, const SMqPollReq* pRequest return -1; } + pPushEntry->pHandle = pHandle; pPushEntry->info = pRpcMsg->info; memcpy(pPushEntry->subKey, pTqHandle->subKey, TSDB_SUBSCRIBE_KEY_LEN); @@ -388,6 +389,7 @@ int32_t tqRegisterPushHandle(STQ* pTq, void* pHandle, const SMqPollReq* pRequest } SMqRspHead* pHead = &pPushEntry->pDataRsp->head; + pHead->consumerId = consumerId; pHead->epoch = pRequest->epoch; pHead->mqMsgType = type; @@ -411,7 +413,7 @@ int32_t tqUnregisterPushHandle(STQ* pTq, const char* pKey, int32_t keyLen, uint6 (*pEntry)->subKey, vgId, taosHashGetSize(pTq->pPushMgr) - 1); if (rspConsumer) { // rsp the old consumer with empty block. - tqPushDataRsp(pTq, *pEntry); + tqPushDataRsp(*pEntry, vgId); } taosHashRemove(pTq->pPushMgr, pKey, keyLen); diff --git a/source/dnode/vnode/src/tq/tqUtil.c b/source/dnode/vnode/src/tq/tqUtil.c index 5ac747947f..b34adcec56 100644 --- a/source/dnode/vnode/src/tq/tqUtil.c +++ b/source/dnode/vnode/src/tq/tqUtil.c @@ -17,7 +17,7 @@ #define IS_OFFSET_RESET_TYPE(_t) ((_t) < 0) -static int32_t tqSendMetaPollRsp(STQ* pTq, const SRpcMsg* pMsg, const SMqPollReq* pReq, const SMqMetaRsp* pRsp); +static int32_t tqSendMetaPollRsp(STqHandle* pHandle, const SRpcMsg* pMsg, const SMqPollReq* pReq, const SMqMetaRsp* pRsp, int32_t vgId); char* createStreamTaskIdStr(int64_t streamId, int32_t taskId) { char buf[128] = {0}; @@ -219,8 +219,8 @@ static int32_t extractResetOffsetVal(STqOffsetVal* pOffsetVal, STQ* pTq, STqHand tqOffsetResetToLog(&dataRsp.rspOffset, walGetLastVer(pTq->pVnode->pWal)); tqDebug("tmq poll: consumer:0x%" PRIx64 ", subkey %s, vgId:%d, (latest) offset reset to %" PRId64, consumerId, pHandle->subKey, vgId, dataRsp.rspOffset.version); - int32_t code = tqSendDataRsp(pTq, pMsg, pRequest, &dataRsp, TMQ_MSG_TYPE__POLL_RSP); - tDeleteSMqDataRsp(&dataRsp); + int32_t code = tqSendDataRsp(pHandle, pMsg, pRequest, &dataRsp, TMQ_MSG_TYPE__POLL_RSP, vgId); + tDeleteMqDataRsp(&dataRsp); *pBlockReturned = true; return code; @@ -228,7 +228,7 @@ static int32_t extractResetOffsetVal(STqOffsetVal* pOffsetVal, STQ* pTq, STqHand STaosxRsp taosxRsp = {0}; tqInitTaosxRsp(&taosxRsp, pRequest); tqOffsetResetToLog(&taosxRsp.rspOffset, walGetLastVer(pTq->pVnode->pWal)); - int32_t code = tqSendDataRsp(pTq, pMsg, pRequest, (SMqDataRsp*)&taosxRsp, TMQ_MSG_TYPE__TAOSX_RSP); + int32_t code = tqSendDataRsp(pHandle, pMsg, pRequest, (SMqDataRsp*)&taosxRsp, TMQ_MSG_TYPE__TAOSX_RSP, vgId); tDeleteSTaosxRsp(&taosxRsp); *pBlockReturned = true; @@ -247,6 +247,7 @@ static int32_t extractResetOffsetVal(STqOffsetVal* pOffsetVal, STQ* pTq, STqHand static int32_t extractDataAndRspForNormalSubscribe(STQ* pTq, STqHandle* pHandle, const SMqPollReq* pRequest, SRpcMsg* pMsg, STqOffsetVal* pOffset) { + char buf[80] = {0}; uint64_t consumerId = pRequest->consumerId; int32_t vgId = TD_VID(pTq->pVnode); @@ -257,37 +258,32 @@ static int32_t extractDataAndRspForNormalSubscribe(STQ* pTq, STqHandle* pHandle, taosWLockLatch(&pTq->lock); qSetTaskId(pHandle->execHandle.task, consumerId, pRequest->reqId); + int code = tqScanData(pTq, pHandle, &dataRsp, pOffset); - if(code != 0) { - goto end; + if (code == 0) { + + // till now, all data has been transferred to consumer, new data needs to push client once arrived. + if (dataRsp.blockNum == 0 && dataRsp.reqOffset.type == TMQ_OFFSET__LOG && + dataRsp.reqOffset.version == dataRsp.rspOffset.version && pHandle->consumerId == pRequest->consumerId) { + code = tqRegisterPushHandle(pTq, pHandle, pRequest, pMsg, &dataRsp, TMQ_MSG_TYPE__POLL_RSP); + taosWUnLockLatch(&pTq->lock); + return code; + } + + // NOTE: this pHandle->consumerId may have been changed already. + code = tqSendDataRsp(pHandle, pMsg, pRequest, (SMqDataRsp*)&dataRsp, TMQ_MSG_TYPE__POLL_RSP, vgId); } - // till now, all data has been transferred to consumer, new data needs to push client once arrived. - if (dataRsp.blockNum == 0 && dataRsp.reqOffset.type == TMQ_OFFSET__LOG && - dataRsp.reqOffset.version == dataRsp.rspOffset.version && pHandle->consumerId == pRequest->consumerId) { - code = tqRegisterPushHandle(pTq, pHandle, pRequest, pMsg, &dataRsp, TMQ_MSG_TYPE__POLL_RSP); - taosWUnLockLatch(&pTq->lock); - return code; - } + tFormatOffset(buf, 80, &dataRsp.rspOffset); + tqDebug("tmq poll: consumer:0x%" PRIx64 ", subkey %s, vgId:%d, rsp block:%d, rsp offset type:%s, reqId:0x%" PRIx64 + " code:%d", + consumerId, pHandle->subKey, vgId, dataRsp.blockNum, buf, pRequest->reqId, code); + taosWUnLockLatch(&pTq->lock); + tDeleteMqDataRsp(&dataRsp); - - code = tqSendDataRsp(pTq, pMsg, pRequest, (SMqDataRsp*)&dataRsp, TMQ_MSG_TYPE__POLL_RSP); - - // NOTE: this pHandle->consumerId may have been changed already. - - end: - { - char buf[80] = {0}; - tFormatOffset(buf, 80, &dataRsp.rspOffset); - tqDebug("tmq poll: consumer:0x%" PRIx64 ", subkey %s, vgId:%d, rsp block:%d, rsp offset type:%s, reqId:0x%" PRIx64 " code:%d", - consumerId, pHandle->subKey, vgId, dataRsp.blockNum, buf, pRequest->reqId, code); - taosWUnLockLatch(&pTq->lock); - tDeleteSMqDataRsp(&dataRsp); - } return code; } - static int32_t extractDataAndRspForDbStbSubscribe(STQ* pTq, STqHandle* pHandle, const SMqPollReq* pRequest, SRpcMsg* pMsg, STqOffsetVal *offset) { int code = 0; int32_t vgId = TD_VID(pTq->pVnode); @@ -303,7 +299,7 @@ static int32_t extractDataAndRspForDbStbSubscribe(STQ* pTq, STqHandle* pHandle, } if (metaRsp.metaRspLen > 0) { - code = tqSendMetaPollRsp(pTq, pMsg, pRequest, &metaRsp); + code = tqSendMetaPollRsp(pHandle, pMsg, pRequest, &metaRsp, vgId); tqDebug("tmq poll: consumer:0x%" PRIx64 " subkey:%s vgId:%d, send meta offset type:%d,uid:%" PRId64 ",ts:%" PRId64, pRequest->consumerId, pHandle->subKey, vgId, metaRsp.rspOffset.type, metaRsp.rspOffset.uid, metaRsp.rspOffset.ts); taosMemoryFree(metaRsp.metaRsp); @@ -314,7 +310,7 @@ 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.blockNum, taosxRsp.rspOffset.type, taosxRsp.rspOffset.uid,taosxRsp.rspOffset.ts); if (taosxRsp.blockNum > 0) { - code = tqSendDataRsp(pTq, pMsg, pRequest, (SMqDataRsp*)&taosxRsp, TMQ_MSG_TYPE__TAOSX_RSP); + code = tqSendDataRsp(pHandle, pMsg, pRequest, (SMqDataRsp*)&taosxRsp, TMQ_MSG_TYPE__TAOSX_RSP, vgId); tDeleteSTaosxRsp(&taosxRsp); return code; }else { @@ -322,7 +318,6 @@ static int32_t extractDataAndRspForDbStbSubscribe(STQ* pTq, STqHandle* pHandle, } } - if (offset->type == TMQ_OFFSET__LOG) { int64_t fetchVer = offset->version + 1; pCkHead = taosMemoryMalloc(sizeof(SWalCkHead) + 2048); @@ -337,13 +332,14 @@ static int32_t extractDataAndRspForDbStbSubscribe(STQ* pTq, STqHandle* pHandle, int32_t savedEpoch = atomic_load_32(&pHandle->epoch); if (savedEpoch > pRequest->epoch) { tqWarn("tmq poll: consumer:0x%" PRIx64 " (epoch %d), subkey:%s vgId:%d offset %" PRId64 - ", found new consumer epoch %d, discard req epoch %d", pRequest->consumerId, pRequest->epoch, pHandle->subKey, vgId, fetchVer, savedEpoch, pRequest->epoch); + ", found new consumer epoch %d, discard req epoch %d", + pRequest->consumerId, pRequest->epoch, pHandle->subKey, vgId, fetchVer, savedEpoch, pRequest->epoch); break; } if (tqFetchLog(pTq, pHandle, &fetchVer, &pCkHead, pRequest->reqId) < 0) { tqOffsetResetToLog(&taosxRsp.rspOffset, fetchVer); - code = tqSendDataRsp(pTq, pMsg, pRequest, (SMqDataRsp*)&taosxRsp, TMQ_MSG_TYPE__TAOSX_RSP); + code = tqSendDataRsp(pHandle, pMsg, pRequest, (SMqDataRsp*)&taosxRsp, TMQ_MSG_TYPE__TAOSX_RSP, vgId); tDeleteSTaosxRsp(&taosxRsp); taosMemoryFreeClear(pCkHead); return code; @@ -357,7 +353,7 @@ static int32_t extractDataAndRspForDbStbSubscribe(STQ* pTq, STqHandle* pHandle, if (pHead->msgType != TDMT_VND_SUBMIT) { if(totalRows > 0) { tqOffsetResetToLog(&taosxRsp.rspOffset, fetchVer - 1); - code = tqSendDataRsp(pTq, pMsg, pRequest, (SMqDataRsp*)&taosxRsp, TMQ_MSG_TYPE__TAOSX_RSP); + code = tqSendDataRsp(pHandle, pMsg, pRequest, (SMqDataRsp*)&taosxRsp, TMQ_MSG_TYPE__TAOSX_RSP, vgId); tDeleteSTaosxRsp(&taosxRsp); taosMemoryFreeClear(pCkHead); return code; @@ -368,7 +364,7 @@ static int32_t extractDataAndRspForDbStbSubscribe(STQ* pTq, STqHandle* pHandle, metaRsp.resMsgType = pHead->msgType; metaRsp.metaRspLen = pHead->bodyLen; metaRsp.metaRsp = pHead->body; - if (tqSendMetaPollRsp(pTq, pMsg, pRequest, &metaRsp) < 0) { + if (tqSendMetaPollRsp(pHandle, pMsg, pRequest, &metaRsp, vgId) < 0) { code = -1; taosMemoryFreeClear(pCkHead); tDeleteSTaosxRsp(&taosxRsp); @@ -398,7 +394,7 @@ static int32_t extractDataAndRspForDbStbSubscribe(STQ* pTq, STqHandle* pHandle, if (totalRows >= 4096 || taosxRsp.createTableNum > 0) { tqOffsetResetToLog(&taosxRsp.rspOffset, fetchVer); - code = tqSendDataRsp(pTq, pMsg, pRequest, (SMqDataRsp*)&taosxRsp, TMQ_MSG_TYPE__TAOSX_RSP); + code = tqSendDataRsp(pHandle, pMsg, pRequest, (SMqDataRsp*)&taosxRsp, TMQ_MSG_TYPE__TAOSX_RSP, vgId); tDeleteSTaosxRsp(&taosxRsp); taosMemoryFreeClear(pCkHead); return code; @@ -446,10 +442,19 @@ int32_t tqExtractDataForMq(STQ* pTq, STqHandle* pHandle, const SMqPollReq* pRequ return extractDataAndRspForDbStbSubscribe(pTq, pHandle, pRequest, pMsg, &offset); } -int32_t tqSendMetaPollRsp(STQ* pTq, const SRpcMsg* pMsg, const SMqPollReq* pReq, const SMqMetaRsp* pRsp) { +static void initMqRspHead(SMqRspHead* pMsgHead, int32_t type, int32_t epoch, int64_t consumerId, int64_t sver, + int64_t ever) { + pMsgHead->consumerId = consumerId; + pMsgHead->epoch = epoch; + pMsgHead->mqMsgType = type; + pMsgHead->walsver = sver; + pMsgHead->walever = ever; +} + +int32_t tqSendMetaPollRsp(STqHandle* pHandle, const SRpcMsg* pMsg, const SMqPollReq* pReq, const SMqMetaRsp* pRsp, int32_t vgId) { int32_t len = 0; int32_t code = 0; - tEncodeSize(tEncodeSMqMetaRsp, pRsp, len, code); + tEncodeSize(tEncodeMqMetaRsp, pRsp, len, code); if (code < 0) { return -1; } @@ -459,27 +464,64 @@ int32_t tqSendMetaPollRsp(STQ* pTq, const SRpcMsg* pMsg, const SMqPollReq* pReq, return -1; } - ((SMqRspHead*)buf)->mqMsgType = TMQ_MSG_TYPE__POLL_META_RSP; - ((SMqRspHead*)buf)->epoch = pReq->epoch; - ((SMqRspHead*)buf)->consumerId = pReq->consumerId; + int64_t sver = 0, ever = 0; + walReaderValidVersionRange(pHandle->execHandle.pTqReader->pWalReader, &sver, &ever); + initMqRspHead(buf, TMQ_MSG_TYPE__POLL_META_RSP, pReq->epoch, pReq->consumerId, sver, ever); void* abuf = POINTER_SHIFT(buf, sizeof(SMqRspHead)); SEncoder encoder = {0}; tEncoderInit(&encoder, abuf, len); - tEncodeSMqMetaRsp(&encoder, pRsp); + tEncodeMqMetaRsp(&encoder, pRsp); tEncoderClear(&encoder); - SRpcMsg resp = { - .info = pMsg->info, - .pCont = buf, - .contLen = tlen, - .code = 0, - }; - tmsgSendRsp(&resp); + SRpcMsg resp = { .info = pMsg->info, .pCont = buf, .contLen = tlen, .code = 0 }; - tqDebug("vgId:%d, from consumer:0x%" PRIx64 " (epoch %d) send rsp, res msg type %d, offset type:%d", - TD_VID(pTq->pVnode), pReq->consumerId, pReq->epoch, pRsp->resMsgType, pRsp->rspOffset.type); + tmsgSendRsp(&resp); + tqDebug("vgId:%d, from consumer:0x%" PRIx64 " (epoch %d) send rsp, res msg type %d, offset type:%d", vgId, + pReq->consumerId, pReq->epoch, pRsp->resMsgType, pRsp->rspOffset.type); return 0; } + +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; + + if (type == TMQ_MSG_TYPE__POLL_RSP) { + tEncodeSize(tEncodeMqDataRsp, pRsp, len, code); + } else if (type == TMQ_MSG_TYPE__TAOSX_RSP) { + tEncodeSize(tEncodeSTaosxRsp, (STaosxRsp*)pRsp, len, code); + } + + if (code < 0) { + return -1; + } + + int32_t tlen = sizeof(SMqRspHead) + len; + void* buf = rpcMallocCont(tlen); + if (buf == NULL) { + return -1; + } + + SMqRspHead* pHead = (SMqRspHead*)buf; + initMqRspHead(pHead, type, epoch, consumerId, sver, ever); + + void* abuf = POINTER_SHIFT(buf, sizeof(SMqRspHead)); + + SEncoder encoder = {0}; + tEncoderInit(&encoder, abuf, len); + + if (type == TMQ_MSG_TYPE__POLL_RSP) { + tEncodeMqDataRsp(&encoder, pRsp); + } else if (type == TMQ_MSG_TYPE__TAOSX_RSP) { + tEncodeSTaosxRsp(&encoder, (STaosxRsp*)pRsp); + } + + tEncoderClear(&encoder); + SRpcMsg rsp = { .info = *pRpcHandleInfo, .pCont = buf, .contLen = tlen, .code = 0 }; + + tmsgSendRsp(&rsp); + return 0; +} \ No newline at end of file diff --git a/source/dnode/vnode/src/vnd/vnodeSvr.c b/source/dnode/vnode/src/vnd/vnodeSvr.c index b62bf27def..d6e8e19043 100644 --- a/source/dnode/vnode/src/vnd/vnodeSvr.c +++ b/source/dnode/vnode/src/vnd/vnodeSvr.c @@ -389,6 +389,11 @@ int32_t vnodeProcessWriteMsg(SVnode *pVnode, SRpcMsg *pMsg, int64_t version, SRp goto _err; } break; + case TDMT_VND_TMQ_SEEK_TO_OFFSET: + if (tqProcessSeekReq(pVnode->pTq, version, pReq, pMsg->contLen - sizeof(SMsgHead)) < 0) { + goto _err; + } + break; case TDMT_VND_TMQ_ADD_CHECKINFO: if (tqProcessAddCheckInfoReq(pVnode->pTq, version, pReq, len) < 0) { goto _err; diff --git a/source/libs/wal/src/walRead.c b/source/libs/wal/src/walRead.c index dc3ff3e6de..19694a6126 100644 --- a/source/libs/wal/src/walRead.c +++ b/source/libs/wal/src/walRead.c @@ -102,6 +102,13 @@ int32_t walNextValidMsg(SWalReader *pReader) { int64_t walReaderGetCurrentVer(const SWalReader *pReader) { return pReader->curVersion; } +void walReaderValidVersionRange(SWalReader *pReader, int64_t *sver, int64_t *ever) { + *sver = walGetFirstVer(pReader->pWal); + int64_t lastVer = walGetLastVer(pReader->pWal); + int64_t committedVer = walGetCommittedVer(pReader->pWal); + *ever = pReader->cond.scanUncommited ? lastVer : committedVer; +} + static int64_t walReadSeekFilePos(SWalReader *pReader, int64_t fileFirstVer, int64_t ver) { int64_t ret = 0; From 3dbe2087121af62fee82e62599012789d76edf61 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sat, 22 Apr 2023 15:34:46 +0800 Subject: [PATCH 16/20] enh(tmq): fix memory error and add retrieve wal info as required. --- include/common/tcommon.h | 1 + include/common/tmsg.h | 17 +- include/common/tmsgdef.h | 1 + source/client/src/clientTmq.c | 217 ++++++++++++++++---- source/client/test/clientTests.cpp | 2 +- source/dnode/mgmt/mgmt_vnode/src/vmHandle.c | 1 + source/dnode/vnode/src/inc/tq.h | 1 + source/dnode/vnode/src/inc/vnodeInt.h | 2 + source/dnode/vnode/src/tq/tq.c | 75 +++++++ source/dnode/vnode/src/tq/tqUtil.c | 10 +- source/dnode/vnode/src/vnd/vnodeSvr.c | 2 + 11 files changed, 275 insertions(+), 54 deletions(-) diff --git a/include/common/tcommon.h b/include/common/tcommon.h index a97c68be49..3d0dac7c3d 100644 --- a/include/common/tcommon.h +++ b/include/common/tcommon.h @@ -128,6 +128,7 @@ enum { TMQ_MSG_TYPE__POLL_META_RSP, TMQ_MSG_TYPE__EP_RSP, TMQ_MSG_TYPE__TAOSX_RSP, + TMQ_MSG_TYPE__WALINFO_RSP, TMQ_MSG_TYPE__END_RSP, }; diff --git a/include/common/tmsg.h b/include/common/tmsg.h index 9e2dbe2f7a..bc77b47f78 100644 --- a/include/common/tmsg.h +++ b/include/common/tmsg.h @@ -3106,15 +3106,14 @@ typedef struct { } SMqRspHead; typedef struct { - SMsgHead head; - char subKey[TSDB_SUBSCRIBE_KEY_LEN]; - int8_t withTbName; - int8_t useSnapshot; - int32_t epoch; - uint64_t reqId; - int64_t consumerId; - int64_t timeout; - // int64_t currentOffset; + SMsgHead head; + char subKey[TSDB_SUBSCRIBE_KEY_LEN]; + int8_t withTbName; + int8_t useSnapshot; + int32_t epoch; + uint64_t reqId; + int64_t consumerId; + int64_t timeout; STqOffsetVal reqOffset; } SMqPollReq; diff --git a/include/common/tmsgdef.h b/include/common/tmsgdef.h index c10e6415e1..a19156200f 100644 --- a/include/common/tmsgdef.h +++ b/include/common/tmsgdef.h @@ -304,6 +304,7 @@ enum { TD_DEF_MSG_TYPE(TDMT_VND_TMQ_ADD_CHECKINFO, "vnode-tmq-add-checkinfo", NULL, NULL) TD_DEF_MSG_TYPE(TDMT_VND_TMQ_DEL_CHECKINFO, "vnode-del-checkinfo", NULL, NULL) TD_DEF_MSG_TYPE(TDMT_VND_TMQ_CONSUME, "vnode-tmq-consume", SMqPollReq, SMqDataBlkRsp) + TD_DEF_MSG_TYPE(TDMT_VND_TMQ_VG_WALINFO, "vnode-tmq-vg-walinfo", SMqPollReq, SMqDataBlkRsp) TD_DEF_MSG_TYPE(TDMT_VND_TMQ_MAX_MSG, "vnd-tmq-max", NULL, NULL) diff --git a/source/client/src/clientTmq.c b/source/client/src/clientTmq.c index 693daee1fc..6b599e9ae6 100644 --- a/source/client/src/clientTmq.c +++ b/source/client/src/clientTmq.c @@ -147,6 +147,7 @@ typedef struct { int32_t vgId; int32_t vgStatus; int32_t vgSkipCnt; // here used to mark the slow vgroups + bool receiveInfo; int64_t emptyBlockReceiveTs; // once empty block is received, idle for ignoreCnt then start to poll data SEpSet epSet; } SMqClientVg; @@ -196,6 +197,23 @@ typedef struct { uint64_t requestId; // request id for debug purpose } SMqPollCbParam; +typedef struct SMqVgCommon { + tsem_t rsp; + int32_t numOfRsp; + SArray* pList; + TdThreadMutex mutex; + int64_t consumerId; + char* pTopicName; + int32_t code; +} SMqVgCommon; + +typedef struct SMqVgWalInfoParam { + int32_t vgId; + int32_t epoch; + int32_t totalReq; + SMqVgCommon* pCommon; +} SMqVgWalInfoParam; + typedef struct { int64_t refId; int32_t epoch; @@ -1100,7 +1118,7 @@ _failed: } int32_t tmq_subscribe(tmq_t* tmq, const tmq_list_t* topic_list) { - const int32_t MAX_RETRY_COUNT = 120 * 2; // let's wait for 2 mins at most + const int32_t MAX_RETRY_COUNT = 120 * 4; // let's wait for 4 mins at most const SArray* container = &topic_list->container; int32_t sz = taosArrayGetSize(container); void* buf = NULL; @@ -1153,22 +1171,13 @@ int32_t tmq_subscribe(tmq_t* tmq, const tmq_list_t* topic_list) { goto FAIL; } - SMqSubscribeCbParam param = { - .rspErr = 0, - .refId = tmq->refId, - .epoch = tmq->epoch, - }; - + SMqSubscribeCbParam param = { .rspErr = 0, .refId = tmq->refId, .epoch = tmq->epoch }; if (tsem_init(¶m.rspSem, 0, 0) != 0) { code = TSDB_CODE_TSC_INTERNAL_ERROR; goto FAIL; } - sendInfo->msgInfo = (SDataBuf){ - .pData = buf, - .len = tlen, - .handle = NULL, - }; + sendInfo->msgInfo = (SDataBuf){.pData = buf, .len = tlen, .handle = NULL}; sendInfo->requestId = generateRequestId(); sendInfo->requestObjRefId = 0; @@ -1196,7 +1205,7 @@ int32_t tmq_subscribe(tmq_t* tmq, const tmq_list_t* topic_list) { int32_t retryCnt = 0; while (TSDB_CODE_MND_CONSUMER_NOT_READY == doAskEp(tmq)) { if (retryCnt++ > MAX_RETRY_COUNT) { - tscError("consumer:0x%" PRIx64 ", mnd not ready for subscribe, retry:%d in 500ms", tmq->consumerId, retryCnt); + tscError("consumer:0x%" PRIx64 ", mnd not ready for subscribe, max retry reached:%d", tmq->consumerId, retryCnt); code = TSDB_CODE_TSC_INTERNAL_ERROR; goto FAIL; } @@ -1232,7 +1241,7 @@ void tmq_conf_set_auto_commit_cb(tmq_conf_t* conf, tmq_commit_cb* cb, void* para conf->commitCbUserParam = param; } -int32_t tmqPollCb(void* param, SDataBuf* pMsg, int32_t code) { +static int32_t tmqPollCb(void* param, SDataBuf* pMsg, int32_t code) { SMqPollCbParam* pParam = (SMqPollCbParam*)param; int64_t refId = pParam->refId; @@ -1285,12 +1294,12 @@ int32_t tmqPollCb(void* param, SDataBuf* pMsg, int32_t code) { } int32_t msgEpoch = ((SMqRspHead*)pMsg->pData)->epoch; - int32_t tmqEpoch = atomic_load_32(&tmq->epoch); - if (msgEpoch < tmqEpoch) { + 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, reqId:0x%" PRIx64, - tmq->consumerId, vgId, msgEpoch, tmqEpoch, requestId); + tmq->consumerId, vgId, msgEpoch, clientEpoch, requestId); tsem_post(&tmq->rspSem); taosReleaseRef(tmqMgmt.rsetId, refId); @@ -1300,9 +1309,9 @@ int32_t tmqPollCb(void* param, SDataBuf* pMsg, int32_t code) { return 0; } - if (msgEpoch != tmqEpoch) { + if (msgEpoch != clientEpoch) { tscWarn("consumer:0x%" PRIx64 " mismatch rsp from vgId:%d, epoch %d, current epoch %d, reqId:0x%" PRIx64, - tmq->consumerId, vgId, msgEpoch, tmqEpoch, requestId); + tmq->consumerId, vgId, msgEpoch, clientEpoch, requestId); } // handle meta rsp @@ -1551,8 +1560,8 @@ int32_t askEpCallbackFn(void* param, SDataBuf* pMsg, int32_t code) { tscDebug("consumer:0x%" PRIx64 ", recv ep, msg epoch %d, current epoch %d, update local ep", tmq->consumerId, head->epoch, epoch); } - pParam->pUserFn(tmq, code, pMsg, pParam->pParam); + pParam->pUserFn(tmq, code, pMsg, pParam->pParam); taosReleaseRef(tmqMgmt.rsetId, pParam->refId); taosMemoryFree(pMsg->pEpSet); @@ -1725,13 +1734,6 @@ static int32_t tmqPollImpl(tmq_t* tmq, int64_t timeout) { tscTrace("consumer:0x%" PRIx64 " epoch %d wait poll-rsp, skip vgId:%d skip cnt %d", tmq->consumerId, tmq->epoch, pVg->vgId, vgSkipCnt); continue; -#if 0 - if (skipCnt < 30000) { - continue; - } else { - tscDebug("consumer:0x%" PRIx64 ",skip vgId:%d skip too much reset", tmq->consumerId, pVg->vgId); - } -#endif } atomic_store_32(&pVg->vgSkipCnt, 0); @@ -1815,6 +1817,7 @@ static void* tmqHandleAllRsp(tmq_t* tmq, int64_t timeout, bool pollIfReset) { // update the valid wal version range pVg->offsetInfo.walVerBegin = pDataRsp->head.walsver; pVg->offsetInfo.walVerEnd = pDataRsp->head.walever; + pVg->receiveInfo = true; char buf[80]; tFormatOffset(buf, 80, &pDataRsp->rspOffset); @@ -1931,15 +1934,6 @@ TAOS_RES* tmq_consumer_poll(tmq_t* tmq, int64_t timeout) { tscDebug("consumer:0x%" PRIx64 " start to poll at %" PRId64 ", timeout:%" PRId64, tmq->consumerId, startTime, timeout); -#if 0 - tmqHandleAllDelayedTask(tmq); - tmqPollImpl(tmq, timeout); - rspObj = tmqHandleAllRsp(tmq, timeout, false); - if (rspObj) { - return (TAOS_RES*)rspObj; - } -#endif - // in no topic status, delayed task also need to be processed if (atomic_load_8(&tmq->status) == TMQ_CONSUMER_STATUS__INIT) { tscDebug("consumer:0x%" PRIx64 " poll return since consumer is init", tmq->consumerId); @@ -2139,7 +2133,7 @@ void tmq_commit_async(tmq_t* tmq, const TAOS_RES* pRes, tmq_commit_cb* cb, void* } } -static void commitCallBackFn(tmq_t *UNUSED_PARAM(pTmq), int32_t code, void* param) { +static void commitCallBackFn(tmq_t *UNUSED_PARAM(tmq), int32_t code, void* param) { SSyncCommitInfo* pInfo = (SSyncCommitInfo*) param; pInfo->code = code; tsem_post(&pInfo->sem); @@ -2341,13 +2335,57 @@ SReqResultInfo* tmqGetNextResInfo(TAOS_RES* res, bool convertUcs4) { return NULL; } +static int32_t tmqGetWalInfoCb(void* param, SDataBuf* pMsg, int32_t code) { + SMqVgWalInfoParam* pParam = param; + SMqVgCommon* pCommon = pParam->pCommon; + + 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, + pParam->vgId, pCommon->pTopicName); + pCommon->code = code; + } else { + SMqDataRsp rsp; + SDecoder decoder; + tDecoderInit(&decoder, POINTER_SHIFT(pMsg->pData, sizeof(SMqRspHead)), pMsg->len - sizeof(SMqRspHead)); + tDecodeMqDataRsp(&decoder, &rsp); + tDecoderClear(&decoder); + + SMqRspHead* pHead = pMsg->pData; + + tmq_topic_assignment assignment = {.begin = pHead->walsver, + .end = pHead->walever, + .currentOffset = rsp.rspOffset.version, + .vgroupHandle = pParam->vgId}; + + taosThreadMutexLock(&pCommon->mutex); + taosArrayPush(pCommon->pList, &assignment); + taosThreadMutexUnlock(&pCommon->mutex); + } + + if (total == pParam->totalReq) { + tsem_post(&pCommon->rsp); + } + + taosMemoryFree(pParam); + return 0; +} + +static void destroyCommonInfo(SMqVgCommon* pCommon) { + taosArrayDestroy(pCommon->pList); + tsem_destroy(&pCommon->rsp); + taosThreadMutexDestroy(&pCommon->mutex); + taosMemoryFree(pCommon->pTopicName); + taosMemoryFree(pCommon); +} + int32_t tmq_get_topic_assignment(tmq_t* tmq, const char* pTopicName, tmq_topic_assignment** assignment, int32_t* numOfAssignment) { *numOfAssignment = 0; *assignment = NULL; int32_t accId = tmq->pTscObj->acctId; - char tname[128] = {0}; + char tname[128] = {0}; sprintf(tname, "%d.%s", accId, pTopicName); SMqClientTopic* pTopic = getTopicByName(tmq, tname); @@ -2365,8 +2403,14 @@ int32_t tmq_get_topic_assignment(tmq_t* tmq, const char* pTopicName, tmq_topic_a return TSDB_CODE_OUT_OF_MEMORY; } + bool needFetch = false; + for (int32_t j = 0; j < (*numOfAssignment); ++j) { SMqClientVg* pClientVg = taosArrayGet(pTopic->vgs, j); + if (!pClientVg->receiveInfo) { + needFetch = true; + break; + } tmq_topic_assignment* pAssignment = &(*assignment)[j]; if (pClientVg->offsetInfo.currentOffset.type == TMQ_OFFSET__LOG) { @@ -2380,7 +2424,102 @@ int32_t tmq_get_topic_assignment(tmq_t* tmq, const char* pTopicName, tmq_topic_a pAssignment->vgroupHandle = pClientVg->vgId; } - return TSDB_CODE_SUCCESS; + if (needFetch) { + SMqVgCommon* pCommon = taosMemoryCalloc(1, sizeof(SMqVgCommon)); + if (pCommon == NULL) { + terrno = TSDB_CODE_OUT_OF_MEMORY; + return terrno; + } + + pCommon->pList= taosArrayInit(4, sizeof(tmq_topic_assignment)); + tsem_init(&pCommon->rsp, 0, 0); + taosThreadMutexInit(&pCommon->mutex, 0); + pCommon->pTopicName = taosStrdup(pTopic->topicName); + pCommon->consumerId = tmq->consumerId; + + terrno = TSDB_CODE_OUT_OF_MEMORY; + for (int32_t i = 0; i < (*numOfAssignment); ++i) { + SMqClientVg* pClientVg = taosArrayGet(pTopic->vgs, i); + + SMqVgWalInfoParam* pParam = taosMemoryMalloc(sizeof(SMqVgWalInfoParam)); + if (pParam == NULL) { + destroyCommonInfo(pCommon); + return terrno; + } + + pParam->epoch = tmq->epoch; + pParam->vgId = pClientVg->vgId; + pParam->totalReq = *numOfAssignment; + pParam->pCommon = pCommon; + + SMqPollReq req = {0}; + tmqBuildConsumeReqImpl(&req, tmq, 10, pTopic, pClientVg); + + int32_t msgSize = tSerializeSMqPollReq(NULL, 0, &req); + if (msgSize < 0) { + taosMemoryFree(pParam); + destroyCommonInfo(pCommon); + return terrno; + } + + char* msg = taosMemoryCalloc(1, msgSize); + if (NULL == msg) { + taosMemoryFree(pParam); + destroyCommonInfo(pCommon); + return terrno; + } + + if (tSerializeSMqPollReq(msg, msgSize, &req) < 0) { + taosMemoryFree(msg); + taosMemoryFree(pParam); + destroyCommonInfo(pCommon); + return terrno; + } + + SMsgSendInfo* sendInfo = taosMemoryCalloc(1, sizeof(SMsgSendInfo)); + if (sendInfo == NULL) { + taosMemoryFree(pParam); + taosMemoryFree(msg); + destroyCommonInfo(pCommon); + return terrno; + } + + sendInfo->msgInfo = (SDataBuf){.pData = msg, .len = msgSize, .handle = NULL}; + sendInfo->requestId = req.reqId; + sendInfo->requestObjRefId = 0; + sendInfo->param = pParam; + sendInfo->fp = tmqGetWalInfoCb; + sendInfo->msgType = TDMT_VND_TMQ_VG_WALINFO; + + int64_t transporterId = 0; + char offsetFormatBuf[80]; + tFormatOffset(offsetFormatBuf, tListLen(offsetFormatBuf), &pClientVg->offsetInfo.currentOffset); + + tscDebug("consumer:0x%" PRIx64 " %s retrieve wal info vgId:%d, epoch %d, req:%s, reqId:0x%" PRIx64, + tmq->consumerId, pTopic->topicName, pClientVg->vgId, tmq->epoch, offsetFormatBuf, req.reqId); + asyncSendMsgToServer(tmq->pTscObj->pAppInfo->pTransporter, &pClientVg->epSet, &transporterId, sendInfo); + } + + tsem_wait(&pCommon->rsp); + int32_t code = pCommon->code; + + terrno = code; + if (code != TSDB_CODE_SUCCESS) { + taosMemoryFree(*assignment); + *numOfAssignment = 0; + } else { + int32_t num = taosArrayGetSize(pCommon->pList); + for(int32_t i = 0; i < num; ++i) { + (*assignment)[i] = *(tmq_topic_assignment*)taosArrayGet(pCommon->pList, i); + } + *numOfAssignment = num; + } + + destroyCommonInfo(pCommon); + return code; + } else { + return TSDB_CODE_SUCCESS; + } } int32_t tmq_offset_seek(tmq_t* tmq, const char* pTopicName, int32_t vgroupHandle, int64_t offset) { diff --git a/source/client/test/clientTests.cpp b/source/client/test/clientTests.cpp index 734577bd1d..9c0b98c4f1 100644 --- a/source/client/test/clientTests.cpp +++ b/source/client/test/clientTests.cpp @@ -1113,7 +1113,7 @@ TEST(clientCase, sub_tb_test) { tmq_topic_assignment* pAssign = NULL; int32_t numOfAssign = 0; - TAOS_RES* p = tmq_consumer_poll(tmq, timeout); +// TAOS_RES* p = tmq_consumer_poll(tmq, timeout); int32_t code = tmq_get_topic_assignment(tmq, "topic_t1", &pAssign, &numOfAssign); while (1) { diff --git a/source/dnode/mgmt/mgmt_vnode/src/vmHandle.c b/source/dnode/mgmt/mgmt_vnode/src/vmHandle.c index 7323d23486..8b3cf4bb2d 100644 --- a/source/dnode/mgmt/mgmt_vnode/src/vmHandle.c +++ b/source/dnode/mgmt/mgmt_vnode/src/vmHandle.c @@ -521,6 +521,7 @@ SArray *vmGetMsgHandles() { if (dmSetMgmtHandle(pArray, TDMT_VND_TMQ_ADD_CHECKINFO, vmPutMsgToWriteQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_VND_TMQ_DEL_CHECKINFO, vmPutMsgToWriteQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_VND_TMQ_CONSUME, vmPutMsgToFetchQueue, 0) == NULL) goto _OVER; + if (dmSetMgmtHandle(pArray, TDMT_VND_TMQ_VG_WALINFO, vmPutMsgToFetchQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_VND_DELETE, vmPutMsgToWriteQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_VND_BATCH_DEL, vmPutMsgToWriteQueue, 0) == NULL) goto _OVER; if (dmSetMgmtHandle(pArray, TDMT_VND_COMMIT, vmPutMsgToWriteQueue, 0) == NULL) goto _OVER; diff --git a/source/dnode/vnode/src/inc/tq.h b/source/dnode/vnode/src/inc/tq.h index 5a6f5249b3..142a1f6fb3 100644 --- a/source/dnode/vnode/src/inc/tq.h +++ b/source/dnode/vnode/src/inc/tq.h @@ -188,6 +188,7 @@ int32_t tqExtractDataForMq(STQ* pTq, STqHandle* pHandle, const SMqPollReq* pRequ 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(SMqDataRsp* pRsp, const SMqPollReq* pReq); void doSaveTaskOffset(STqOffsetStore* pOffsetStore, const char* pKey, int64_t ver); void saveOffsetForAllTasks(STQ* pTq, int64_t ver); void initOffsetForAllRestoreTasks(STQ* pTq); diff --git a/source/dnode/vnode/src/inc/vnodeInt.h b/source/dnode/vnode/src/inc/vnodeInt.h index 764e57eb09..441ad8d31f 100644 --- a/source/dnode/vnode/src/inc/vnodeInt.h +++ b/source/dnode/vnode/src/inc/vnodeInt.h @@ -208,6 +208,8 @@ int32_t tqProcessDeleteSubReq(STQ* pTq, int64_t version, char* msg, int32_t msgL int32_t tqProcessOffsetCommitReq(STQ* pTq, int64_t version, char* msg, int32_t msgLen); int32_t tqProcessSeekReq(STQ* pTq, int64_t sversion, char* msg, int32_t msgLen); int32_t tqProcessPollReq(STQ* pTq, SRpcMsg* pMsg); +int32_t tqProcessVgWalInfoReq(STQ* pTq, SRpcMsg* pMsg); + // tq-stream int32_t tqProcessTaskDeployReq(STQ* pTq, int64_t version, char* msg, int32_t msgLen); int32_t tqProcessTaskDropReq(STQ* pTq, int64_t version, char* msg, int32_t msgLen); diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 5e4ede914a..7455bc021e 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -365,6 +365,81 @@ int32_t tqProcessPollReq(STQ* pTq, SRpcMsg* pMsg) { return tqExtractDataForMq(pTq, pHandle, &req, pMsg); } +int32_t tqProcessVgWalInfoReq(STQ* pTq, SRpcMsg* pMsg) { + SMqPollReq req = {0}; + if (tDeserializeSMqPollReq(pMsg->pCont, pMsg->contLen, &req) < 0) { + tqError("tDeserializeSMqPollReq %d failed", pMsg->contLen); + terrno = TSDB_CODE_INVALID_MSG; + return -1; + } + + int64_t consumerId = req.consumerId; + STqOffsetVal reqOffset = req.reqOffset; + int32_t vgId = TD_VID(pTq->pVnode); + + // 1. find handle + STqHandle* pHandle = taosHashGet(pTq->pHandle, req.subKey, strlen(req.subKey)); + if (pHandle == NULL) { + tqError("consumer:0x%" PRIx64 " vgId:%d subkey:%s not found", consumerId, vgId, req.subKey); + terrno = TSDB_CODE_INVALID_MSG; + return -1; + } + + // 2. check re-balance status + taosRLockLatch(&pTq->lock); + if (pHandle->consumerId != consumerId) { + tqDebug("ERROR consumer:0x%" PRIx64 " vgId:%d, subkey %s, mismatch for saved handle consumer:0x%" PRIx64, + consumerId, vgId, req.subKey, pHandle->consumerId); + terrno = TSDB_CODE_TMQ_CONSUMER_MISMATCH; + taosRUnLockLatch(&pTq->lock); + return -1; + } + taosRUnLockLatch(&pTq->lock); + + int64_t sver = 0, ever = 0; + walReaderValidVersionRange(pHandle->execHandle.pTqReader->pWalReader, &sver, &ever); + + SMqDataRsp dataRsp = {0}; + tqInitDataRsp(&dataRsp, &req); + + STqOffset* pOffset = tqOffsetRead(pTq->pOffsetStore, req.subKey); + if (pOffset != NULL) { + if (pOffset->val.type != TMQ_OFFSET__LOG) { + tqError("consumer:0x%" PRIx64 " vgId:%d subkey:%s use snapshot, no valid wal info", consumerId, vgId, req.subKey); + terrno = TSDB_CODE_INVALID_PARA; + tDeleteMqDataRsp(&dataRsp); + return -1; + } + + dataRsp.rspOffset.type = TMQ_OFFSET__LOG; + dataRsp.rspOffset.version = pOffset->val.version; + } else { + if (req.useSnapshot == true) { + tqError("consumer:0x%" PRIx64 " vgId:%d subkey:%s snapshot not support wal info", consumerId, vgId, req.subKey); + terrno = TSDB_CODE_INVALID_PARA; + tDeleteMqDataRsp(&dataRsp); + return -1; + } + + dataRsp.rspOffset.type = TMQ_OFFSET__LOG; + + if (reqOffset.type == TMQ_OFFSET__RESET_EARLIEAST) { + dataRsp.rspOffset.version = sver; + } else if (reqOffset.type == TMQ_OFFSET__RESET_LATEST) { + dataRsp.rspOffset.version = ever; + } else { + tqError("consumer:0x%" PRIx64 " vgId:%d subkey:%s invalid offset type:%d", consumerId, vgId, req.subKey, + reqOffset.type); + terrno = TSDB_CODE_INVALID_PARA; + tDeleteMqDataRsp(&dataRsp); + return -1; + } + } + + tqDoSendDataRsp(&pMsg->info, &dataRsp, req.epoch, req.consumerId, TMQ_MSG_TYPE__WALINFO_RSP, sver, ever); + return 0; +} + int32_t tqProcessDeleteSubReq(STQ* pTq, int64_t sversion, char* msg, int32_t msgLen) { SMqVDeleteReq* pReq = (SMqVDeleteReq*)msg; diff --git a/source/dnode/vnode/src/tq/tqUtil.c b/source/dnode/vnode/src/tq/tqUtil.c index b34adcec56..c09a8a030e 100644 --- a/source/dnode/vnode/src/tq/tqUtil.c +++ b/source/dnode/vnode/src/tq/tqUtil.c @@ -124,7 +124,7 @@ void doSaveTaskOffset(STqOffsetStore* pOffsetStore, const char* pKey, int64_t ve tqOffsetWrite(pOffsetStore, &offset); } -static int32_t tqInitDataRsp(SMqDataRsp* pRsp, const SMqPollReq* pReq, int8_t subType) { +int32_t tqInitDataRsp(SMqDataRsp* pRsp, const SMqPollReq* pReq) { pRsp->reqOffset = pReq->reqOffset; pRsp->blockData = taosArrayInit(0, sizeof(void*)); @@ -214,7 +214,7 @@ static int32_t extractResetOffsetVal(STqOffsetVal* pOffsetVal, STQ* pTq, STqHand } else if (reqOffset.type == TMQ_OFFSET__RESET_LATEST) { if (pHandle->execHandle.subType == TOPIC_SUB_TYPE__COLUMN) { SMqDataRsp dataRsp = {0}; - tqInitDataRsp(&dataRsp, pRequest, pHandle->execHandle.subType); + tqInitDataRsp(&dataRsp, pRequest); tqOffsetResetToLog(&dataRsp.rspOffset, walGetLastVer(pTq->pVnode->pWal)); tqDebug("tmq poll: consumer:0x%" PRIx64 ", subkey %s, vgId:%d, (latest) offset reset to %" PRId64, consumerId, @@ -252,7 +252,7 @@ static int32_t extractDataAndRspForNormalSubscribe(STQ* pTq, STqHandle* pHandle, int32_t vgId = TD_VID(pTq->pVnode); SMqDataRsp dataRsp = {0}; - tqInitDataRsp(&dataRsp, pRequest, pHandle->execHandle.subType); + tqInitDataRsp(&dataRsp, pRequest); // lock taosWLockLatch(&pTq->lock); @@ -489,7 +489,7 @@ int32_t tqDoSendDataRsp(const SRpcHandleInfo* pRpcHandleInfo, const SMqDataRsp* int32_t len = 0; int32_t code = 0; - if (type == TMQ_MSG_TYPE__POLL_RSP) { + if (type == TMQ_MSG_TYPE__POLL_RSP || type == TMQ_MSG_TYPE__WALINFO_RSP) { tEncodeSize(tEncodeMqDataRsp, pRsp, len, code); } else if (type == TMQ_MSG_TYPE__TAOSX_RSP) { tEncodeSize(tEncodeSTaosxRsp, (STaosxRsp*)pRsp, len, code); @@ -513,7 +513,7 @@ int32_t tqDoSendDataRsp(const SRpcHandleInfo* pRpcHandleInfo, const SMqDataRsp* SEncoder encoder = {0}; tEncoderInit(&encoder, abuf, len); - if (type == TMQ_MSG_TYPE__POLL_RSP) { + if (type == TMQ_MSG_TYPE__POLL_RSP || type == TMQ_MSG_TYPE__WALINFO_RSP) { tEncodeMqDataRsp(&encoder, pRsp); } else if (type == TMQ_MSG_TYPE__TAOSX_RSP) { tEncodeSTaosxRsp(&encoder, (STaosxRsp*)pRsp); diff --git a/source/dnode/vnode/src/vnd/vnodeSvr.c b/source/dnode/vnode/src/vnd/vnodeSvr.c index d6e8e19043..668020c36d 100644 --- a/source/dnode/vnode/src/vnd/vnodeSvr.c +++ b/source/dnode/vnode/src/vnd/vnodeSvr.c @@ -544,6 +544,8 @@ int32_t vnodeProcessFetchMsg(SVnode *pVnode, SRpcMsg *pMsg, SQueueInfo *pInfo) { return vnodeGetBatchMeta(pVnode, pMsg); case TDMT_VND_TMQ_CONSUME: return tqProcessPollReq(pVnode->pTq, pMsg); + case TDMT_VND_TMQ_VG_WALINFO: + return tqProcessVgWalInfoReq(pVnode->pTq, pMsg); case TDMT_STREAM_TASK_RUN: return tqProcessTaskRunReq(pVnode->pTq, pMsg); case TDMT_STREAM_TASK_DISPATCH: From 9cf89c6b6f618f77915dc3c26f84d2fbeeb68dec Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sun, 23 Apr 2023 11:51:43 +0800 Subject: [PATCH 17/20] enh(tmq): add check for consumer Id; --- include/common/tmsg.h | 9 +++ source/client/src/clientTmq.c | 22 ++++--- source/client/test/clientTests.cpp | 10 ++- source/common/src/tmsg.c | 12 ++++ source/dnode/vnode/src/tq/tq.c | 101 ++++++++++++++++++----------- 5 files changed, 103 insertions(+), 51 deletions(-) diff --git a/include/common/tmsg.h b/include/common/tmsg.h index bc77b47f78..42f2cb8f54 100644 --- a/include/common/tmsg.h +++ b/include/common/tmsg.h @@ -2805,6 +2805,7 @@ typedef struct { } SMqOffset; typedef struct { + int64_t consumerId; int32_t num; SMqOffset* offsets; } SMqCMCommitOffsetReq; @@ -2872,6 +2873,14 @@ typedef struct { int32_t tEncodeSTqOffset(SEncoder* pEncoder, const STqOffset* pOffset); int32_t tDecodeSTqOffset(SDecoder* pDecoder, STqOffset* pOffset); +typedef struct SMqVgOffset { + int64_t consumerId; + STqOffset offset; +} SMqVgOffset; + +int32_t tEncodeMqVgOffset(SEncoder* pEncoder, const SMqVgOffset* pOffset); +int32_t tDecodeMqVgOffset(SDecoder* pDecoder, SMqVgOffset* pOffset); + typedef struct { char name[TSDB_TABLE_FNAME_LEN]; char stb[TSDB_TABLE_FNAME_LEN]; diff --git a/source/client/src/clientTmq.c b/source/client/src/clientTmq.c index 6b599e9ae6..87c58ab32f 100644 --- a/source/client/src/clientTmq.c +++ b/source/client/src/clientTmq.c @@ -228,7 +228,7 @@ typedef struct { typedef struct { SMqCommitCbParamSet* params; - STqOffset* pOffset; + SMqVgOffset* pOffset; char topicName[TSDB_TOPIC_FNAME_LEN]; int32_t vgId; tmq_t* pTmq; @@ -492,21 +492,22 @@ static int32_t tmqCommitCb(void* param, SDataBuf* pBuf, int32_t code) { static int32_t doSendCommitMsg(tmq_t* tmq, SMqClientVg* pVg, const char* pTopicName, SMqCommitCbParamSet* pParamSet, int32_t index, int32_t totalVgroups, int32_t type) { - STqOffset* pOffset = taosMemoryCalloc(1, sizeof(STqOffset)); + SMqVgOffset* pOffset = taosMemoryCalloc(1, sizeof(SMqVgOffset)); if (pOffset == NULL) { return TSDB_CODE_OUT_OF_MEMORY; } - pOffset->val = pVg->offsetInfo.currentOffset; + pOffset->consumerId = tmq->consumerId; + pOffset->offset.val = pVg->offsetInfo.currentOffset; int32_t groupLen = strlen(tmq->groupId); - memcpy(pOffset->subKey, tmq->groupId, groupLen); - pOffset->subKey[groupLen] = TMQ_SEPARATOR; - strcpy(pOffset->subKey + groupLen + 1, pTopicName); + memcpy(pOffset->offset.subKey, tmq->groupId, groupLen); + pOffset->offset.subKey[groupLen] = TMQ_SEPARATOR; + strcpy(pOffset->offset.subKey + groupLen + 1, pTopicName); int32_t len = 0; int32_t code = 0; - tEncodeSize(tEncodeSTqOffset, pOffset, len, code); + tEncodeSize(tEncodeMqVgOffset, pOffset, len, code); if (code < 0) { return TSDB_CODE_INVALID_PARA; } @@ -523,7 +524,7 @@ static int32_t doSendCommitMsg(tmq_t* tmq, SMqClientVg* pVg, const char* pTopicN SEncoder encoder; tEncoderInit(&encoder, abuf, len); - tEncodeSTqOffset(&encoder, pOffset); + tEncodeMqVgOffset(&encoder, pOffset); tEncoderClear(&encoder); // build param @@ -564,12 +565,12 @@ static int32_t doSendCommitMsg(tmq_t* tmq, SMqClientVg* pVg, const char* pTopicN SEp* pEp = GET_ACTIVE_EP(&pVg->epSet); char offsetBuf[80] = {0}; - tFormatOffset(offsetBuf, tListLen(offsetBuf), &pOffset->val); + tFormatOffset(offsetBuf, tListLen(offsetBuf), &pOffset->offset.val); char commitBuf[80] = {0}; tFormatOffset(commitBuf, tListLen(commitBuf), &pVg->offsetInfo.committedOffset); tscDebug("consumer:0x%" PRIx64 " topic:%s on vgId:%d send offset:%s prev:%s, ep:%s:%d, ordinal:%d/%d, req:0x%" PRIx64, - tmq->consumerId, pOffset->subKey, pVg->vgId, offsetBuf, commitBuf, pEp->fqdn, pEp->port, index + 1, + tmq->consumerId, pOffset->offset.subKey, pVg->vgId, offsetBuf, commitBuf, pEp->fqdn, pEp->port, index + 1, totalVgroups, pMsgSendInfo->requestId); int64_t transporterId = 0; @@ -2506,6 +2507,7 @@ int32_t tmq_get_topic_assignment(tmq_t* tmq, const char* pTopicName, tmq_topic_a terrno = code; if (code != TSDB_CODE_SUCCESS) { taosMemoryFree(*assignment); + *assignment = NULL; *numOfAssignment = 0; } else { int32_t num = taosArrayGetSize(pCommon->pList); diff --git a/source/client/test/clientTests.cpp b/source/client/test/clientTests.cpp index 9c0b98c4f1..08d62054e3 100644 --- a/source/client/test/clientTests.cpp +++ b/source/client/test/clientTests.cpp @@ -1082,7 +1082,7 @@ TEST(clientCase, sub_tb_test) { tmq_conf_t* conf = tmq_conf_new(); tmq_conf_set(conf, "enable.auto.commit", "false"); tmq_conf_set(conf, "auto.commit.interval.ms", "1000"); - tmq_conf_set(conf, "group.id", "cgrpName45"); + tmq_conf_set(conf, "group.id", "cgrpName1024"); tmq_conf_set(conf, "td.connect.user", "root"); tmq_conf_set(conf, "td.connect.pass", "taosdata"); tmq_conf_set(conf, "auto.offset.reset", "earliest"); @@ -1113,8 +1113,14 @@ TEST(clientCase, sub_tb_test) { tmq_topic_assignment* pAssign = NULL; int32_t numOfAssign = 0; -// TAOS_RES* p = tmq_consumer_poll(tmq, timeout); int32_t code = tmq_get_topic_assignment(tmq, "topic_t1", &pAssign, &numOfAssign); + if (code != 0) { + printf("error occurs:%s\n", tmq_err2str(code)); + tmq_consumer_close(tmq); + taos_close(pConn); + fprintf(stderr, "%d msg consumed, include %d rows\n", msgCnt, totalRows); + return; + } while (1) { TAOS_RES* pRes = tmq_consumer_poll(tmq, timeout); diff --git a/source/common/src/tmsg.c b/source/common/src/tmsg.c index 70c9ae4732..7a087c450e 100644 --- a/source/common/src/tmsg.c +++ b/source/common/src/tmsg.c @@ -6909,6 +6909,18 @@ int32_t tDecodeSTqOffset(SDecoder *pDecoder, STqOffset *pOffset) { return 0; } +int32_t tEncodeMqVgOffset(SEncoder* pEncoder, const SMqVgOffset* pOffset) { + if (tEncodeI64(pEncoder, pOffset->consumerId) < 0) return -1; + if (tEncodeSTqOffset(pEncoder, &pOffset->offset) < 0) return -1; + return 0; +} + +int32_t tDecodeMqVgOffset(SDecoder* pDecoder, SMqVgOffset* pOffset) { + if (tDecodeI64(pDecoder, &pOffset->consumerId) < 0) return -1; + if (tDecodeSTqOffset(pDecoder, &pOffset->offset) < 0) return -1; + return 0; +} + int32_t tEncodeSTqCheckInfo(SEncoder *pEncoder, const STqCheckInfo *pInfo) { if (tEncodeCStr(pEncoder, pInfo->topic) < 0) return -1; if (tEncodeI64(pEncoder, pInfo->ntbUid) < 0) return -1; diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 7455bc021e..4a9bf07429 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -191,46 +191,48 @@ int32_t tqSendDataRsp(STqHandle* pHandle, const SRpcMsg* pMsg, const SMqPollReq* } int32_t tqProcessOffsetCommitReq(STQ* pTq, int64_t sversion, char* msg, int32_t msgLen) { - STqOffset offset = {0}; - int32_t vgId = TD_VID(pTq->pVnode); + SMqVgOffset vgOffset = {0}; + int32_t vgId = TD_VID(pTq->pVnode); SDecoder decoder; tDecoderInit(&decoder, (uint8_t*)msg, msgLen); - if (tDecodeSTqOffset(&decoder, &offset) < 0) { + if (tDecodeMqVgOffset(&decoder, &vgOffset) < 0) { return -1; } tDecoderClear(&decoder); - if (offset.val.type == TMQ_OFFSET__SNAPSHOT_DATA || offset.val.type == TMQ_OFFSET__SNAPSHOT_META) { + STqOffset* pOffset = &vgOffset.offset; + + if (pOffset->val.type == TMQ_OFFSET__SNAPSHOT_DATA || pOffset->val.type == TMQ_OFFSET__SNAPSHOT_META) { tqDebug("receive offset commit msg to %s on vgId:%d, offset(type:snapshot) uid:%" PRId64 ", ts:%" PRId64, - offset.subKey, vgId, offset.val.uid, offset.val.ts); - } else if (offset.val.type == TMQ_OFFSET__LOG) { - tqDebug("receive offset commit msg to %s on vgId:%d, offset(type:log) version:%" PRId64, offset.subKey, vgId, - offset.val.version); - if (offset.val.version + 1 == sversion) { - offset.val.version += 1; + pOffset->subKey, vgId, pOffset->val.uid, pOffset->val.ts); + } else if (pOffset->val.type == TMQ_OFFSET__LOG) { + tqDebug("receive offset commit msg to %s on vgId:%d, offset(type:log) version:%" PRId64, pOffset->subKey, vgId, + pOffset->val.version); + if (pOffset->val.version + 1 == sversion) { + pOffset->val.version += 1; } } else { - tqError("invalid commit offset type:%d", offset.val.type); + tqError("invalid commit offset type:%d", pOffset->val.type); return -1; } - STqOffset* pSavedOffset = tqOffsetRead(pTq->pOffsetStore, offset.subKey); - if (pSavedOffset != NULL && tqOffsetLessOrEqual(&offset, pSavedOffset)) { + STqOffset* pSavedOffset = tqOffsetRead(pTq->pOffsetStore, pOffset->subKey); + if (pSavedOffset != NULL && tqOffsetLessOrEqual(pOffset, pSavedOffset)) { tqDebug("not update the offset, vgId:%d sub:%s since committed:%" PRId64 " less than/equal to existed:%" PRId64, - vgId, offset.subKey, offset.val.version, pSavedOffset->val.version); + vgId, pOffset->subKey, pOffset->val.version, pSavedOffset->val.version); return 0; // no need to update the offset value } // save the new offset value - if (tqOffsetWrite(pTq->pOffsetStore, &offset) < 0) { + if (tqOffsetWrite(pTq->pOffsetStore, pOffset) < 0) { return -1; } - if (offset.val.type == TMQ_OFFSET__LOG) { - STqHandle* pHandle = taosHashGet(pTq->pHandle, offset.subKey, strlen(offset.subKey)); - if (pHandle && (walRefVer(pHandle->pRef, offset.val.version) < 0)) { + if (pOffset->val.type == TMQ_OFFSET__LOG) { + STqHandle* pHandle = taosHashGet(pTq->pHandle, pOffset->subKey, strlen(pOffset->subKey)); + if (pHandle && (walRefVer(pHandle->pRef, pOffset->val.version) < 0)) { return -1; } } @@ -239,50 +241,71 @@ int32_t tqProcessOffsetCommitReq(STQ* pTq, int64_t sversion, char* msg, int32_t } int32_t tqProcessSeekReq(STQ* pTq, int64_t sversion, char* msg, int32_t msgLen) { - STqOffset offset = {0}; - int32_t vgId = TD_VID(pTq->pVnode); + SMqVgOffset vgOffset = {0}; + int32_t vgId = TD_VID(pTq->pVnode); SDecoder decoder; tDecoderInit(&decoder, (uint8_t*)msg, msgLen); - if (tDecodeSTqOffset(&decoder, &offset) < 0) { + if (tDecodeMqVgOffset(&decoder, &vgOffset) < 0) { return -1; } tDecoderClear(&decoder); - if (offset.val.type != TMQ_OFFSET__LOG) { - tqError("vgId:%d, subKey:%s invalid seek offset type:%d", vgId, offset.subKey, offset.val.type); + STqOffset* pOffset = &vgOffset.offset; + if (pOffset->val.type != TMQ_OFFSET__LOG) { + tqError("vgId:%d, subKey:%s invalid seek offset type:%d", vgId, pOffset->subKey, pOffset->val.type); return -1; } - STqOffset* pSavedOffset = tqOffsetRead(pTq->pOffsetStore, offset.subKey); - if (pSavedOffset != NULL && pSavedOffset->val.type != TMQ_OFFSET__LOG) { - tqError("invalid saved offset type, vgId:%d sub:%s", vgId, offset.subKey); - return 0; // no need to update the offset value + STqHandle* pHandle = taosHashGet(pTq->pHandle, pOffset->subKey, strlen(pOffset->subKey)); + if (pHandle == NULL) { + tqError("tmq seek: consumer:0x%" PRIx64 " vgId:%d subkey %s not found", vgOffset.consumerId, vgId, + pOffset->subKey); + terrno = TSDB_CODE_INVALID_MSG; + return -1; } - if (pSavedOffset->val.version == offset.val.version) { - tqDebug("vgId:%d subKey:%s no need to seek to %" PRId64 " prev offset:%" PRId64, vgId, offset.subKey, offset.val.version, - pSavedOffset->val.version); - return 0; + // 2. check consumer-vg assignment status + taosRLockLatch(&pTq->lock); + if (pHandle->consumerId != vgOffset.consumerId) { + tqDebug("ERROR tmq seek: consumer:0x%" PRIx64 " vgId:%d, subkey %s, mismatch for saved handle consumer:0x%" PRIx64, + vgOffset.consumerId, vgId, pOffset->subKey, pHandle->consumerId); + terrno = TSDB_CODE_TMQ_CONSUMER_MISMATCH; + taosRUnLockLatch(&pTq->lock); + return -1; } + taosRUnLockLatch(&pTq->lock); - STqHandle* pHandle = taosHashGet(pTq->pHandle, offset.subKey, strlen(offset.subKey)); + //3. check the offset info + STqOffset* pSavedOffset = tqOffsetRead(pTq->pOffsetStore, pOffset->subKey); + if (pSavedOffset != NULL) { + if (pSavedOffset->val.type != TMQ_OFFSET__LOG) { + tqError("invalid saved offset type, vgId:%d sub:%s", vgId, pOffset->subKey); + return 0; // no need to update the offset value + } + + if (pSavedOffset->val.version == pOffset->val.version) { + tqDebug("vgId:%d subKey:%s no need to seek to %" PRId64 " prev offset:%" PRId64, vgId, pOffset->subKey, + pOffset->val.version, pSavedOffset->val.version); + return 0; + } + } int64_t sver = 0, ever = 0; walReaderValidVersionRange(pHandle->execHandle.pTqReader->pWalReader, &sver, &ever); - if (offset.val.version < sver) { - offset.val.version = sver; - } else if (offset.val.version > ever) { - offset.val.version = ever; + if (pOffset->val.version < sver) { + pOffset->val.version = sver; + } else if (pOffset->val.version > ever) { + pOffset->val.version = ever; } // save the new offset value - tqDebug("vgId:%d sub:%s seek to %" PRId64 " prev offset:%" PRId64, vgId, offset.subKey, offset.val.version, + tqDebug("vgId:%d sub:%s seek to %" PRId64 " prev offset:%" PRId64, vgId, pOffset->subKey, pOffset->val.version, pSavedOffset->val.version); - if (tqOffsetWrite(pTq->pOffsetStore, &offset) < 0) { - tqError("failed to save offset, vgId:%d sub:%s seek to %" PRId64, vgId, offset.subKey, offset.val.version); + if (tqOffsetWrite(pTq->pOffsetStore, pOffset) < 0) { + tqError("failed to save offset, vgId:%d sub:%s seek to %" PRId64, vgId, pOffset->subKey, pOffset->val.version); return -1; } From 2e2e3a61e5b1a8a782b692a71dd60c11cad175c0 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sun, 23 Apr 2023 11:58:40 +0800 Subject: [PATCH 18/20] fix(tmq): update the seek info serialization. --- source/common/src/tmsg.c | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/source/common/src/tmsg.c b/source/common/src/tmsg.c index 7a087c450e..6da5200749 100644 --- a/source/common/src/tmsg.c +++ b/source/common/src/tmsg.c @@ -6910,14 +6910,14 @@ int32_t tDecodeSTqOffset(SDecoder *pDecoder, STqOffset *pOffset) { } int32_t tEncodeMqVgOffset(SEncoder* pEncoder, const SMqVgOffset* pOffset) { - if (tEncodeI64(pEncoder, pOffset->consumerId) < 0) return -1; if (tEncodeSTqOffset(pEncoder, &pOffset->offset) < 0) return -1; + if (tEncodeI64(pEncoder, pOffset->consumerId) < 0) return -1; return 0; } int32_t tDecodeMqVgOffset(SDecoder* pDecoder, SMqVgOffset* pOffset) { - if (tDecodeI64(pDecoder, &pOffset->consumerId) < 0) return -1; if (tDecodeSTqOffset(pDecoder, &pOffset->offset) < 0) return -1; + if (tDecodeI64(pDecoder, &pOffset->consumerId) < 0) return -1; return 0; } From 72ddd1676deddb29a60555491484e0cf4882c433 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Sun, 23 Apr 2023 13:31:41 +0800 Subject: [PATCH 19/20] refactor: do some internal refactor. --- tests/pytest/util/sql.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/pytest/util/sql.py b/tests/pytest/util/sql.py index 783ee476cb..c041282bfc 100644 --- a/tests/pytest/util/sql.py +++ b/tests/pytest/util/sql.py @@ -431,7 +431,7 @@ class TDSql: time.sleep(1) continue - def execute(self, sql,queryTimes=10): + def execute(self, sql,queryTimes=30): self.sql = sql i=1 while i <= queryTimes: From 104781124aff2e9550e0975b30361f7c8398efb0 Mon Sep 17 00:00:00 2001 From: Benguang Zhao Date: Sun, 23 Apr 2023 17:59:43 +0800 Subject: [PATCH 20/20] fix: update commit index from heartbeat on learner in the same way as on follower --- source/libs/sync/src/syncMain.c | 46 +++++---------------------------- 1 file changed, 7 insertions(+), 39 deletions(-) diff --git a/source/libs/sync/src/syncMain.c b/source/libs/sync/src/syncMain.c index 8a0a6be8fd..e3342d76ee 100644 --- a/source/libs/sync/src/syncMain.c +++ b/source/libs/sync/src/syncMain.c @@ -2414,18 +2414,19 @@ int32_t syncNodeOnHeartbeat(SSyncNode* ths, const SRpcMsg* pRpcMsg) { (void)syncBuildLocalCmd(&rpcMsgLocalCmd, ths->vgId); SyncLocalCmd* pSyncMsg = rpcMsgLocalCmd.pCont; - pSyncMsg->cmd = SYNC_LOCAL_CMD_FOLLOWER_CMT; + pSyncMsg->cmd = + (ths->state == TAOS_SYNC_STATE_LEARNER) ? SYNC_LOCAL_CMD_LEARNER_CMT : SYNC_LOCAL_CMD_FOLLOWER_CMT; pSyncMsg->commitIndex = pMsg->commitIndex; pSyncMsg->currentTerm = pMsg->term; - SyncIndex fcIndex = pSyncMsg->commitIndex; if (ths->syncEqMsg != NULL && ths->msgcb != NULL) { int32_t code = ths->syncEqMsg(ths->msgcb, &rpcMsgLocalCmd); if (code != 0) { - sError("vgId:%d, sync enqueue fc-commit msg error, code:%d", ths->vgId, code); + sError("vgId:%d, failed to enqueue commit msg from heartbeat since %s, code:%d", ths->vgId, terrstr(), code); rpcFreeCont(rpcMsgLocalCmd.pCont); } else { - sTrace("vgId:%d, sync enqueue fc-commit msg, fc-index:%" PRId64, ths->vgId, fcIndex); + sTrace("vgId:%d, enqueue commit msg from heartbeat, commit-index:%" PRId64 ", term:%" PRId64, ths->vgId, + pMsg->commitIndex, pMsg->term); } } } @@ -2451,26 +2452,6 @@ int32_t syncNodeOnHeartbeat(SSyncNode* ths, const SRpcMsg* pRpcMsg) { } } - if (pMsg->term >= currentTerm && ths->state == TAOS_SYNC_STATE_LEARNER) { - SRpcMsg rpcMsgLocalCmd = {0}; - (void)syncBuildLocalCmd(&rpcMsgLocalCmd, ths->vgId); - - SyncLocalCmd* pSyncMsg = rpcMsgLocalCmd.pCont; - pSyncMsg->cmd = SYNC_LOCAL_CMD_LEARNER_CMT; - pSyncMsg->currentTerm = pMsg->term; - pSyncMsg->commitIndex = pMsg->commitIndex; - - if (ths->syncEqMsg != NULL && ths->msgcb != NULL) { - int32_t code = ths->syncEqMsg(ths->msgcb, &rpcMsgLocalCmd); - if (code != 0) { - sError("vgId:%d, sync enqueue step-down msg error, code:%d", ths->vgId, code); - rpcFreeCont(rpcMsgLocalCmd.pCont); - } else { - sTrace("vgId:%d, sync enqueue step-down msg, new-term:%" PRId64, ths->vgId, pMsg->term); - } - } - } - // reply syncNodeSendMsgById(&pMsgReply->destId, ths, &rpcMsg); @@ -2521,7 +2502,7 @@ int32_t syncNodeOnLocalCmd(SSyncNode* ths, const SRpcMsg* pRpcMsg) { if (pMsg->cmd == SYNC_LOCAL_CMD_STEP_DOWN) { syncNodeStepDown(ths, pMsg->currentTerm); - } else if (pMsg->cmd == SYNC_LOCAL_CMD_FOLLOWER_CMT) { + } else if (pMsg->cmd == SYNC_LOCAL_CMD_FOLLOWER_CMT || pMsg->cmd == SYNC_LOCAL_CMD_LEARNER_CMT) { if (syncLogBufferIsEmpty(ths->pLogBuf)) { sError("vgId:%d, sync log buffer is empty.", ths->vgId); return 0; @@ -2534,20 +2515,7 @@ int32_t syncNodeOnLocalCmd(SSyncNode* ths, const SRpcMsg* pRpcMsg) { sError("vgId:%d, failed to commit raft log since %s. commit index:%" PRId64 "", ths->vgId, terrstr(), ths->commitIndex); } - } else if (pMsg->cmd == SYNC_LOCAL_CMD_LEARNER_CMT){ - if (syncLogBufferIsEmpty(ths->pLogBuf)) { - sError("vgId:%d, sync log buffer is empty.", ths->vgId); - return 0; - } - raftStoreSetTerm(ths, pMsg->currentTerm); - (void)syncNodeUpdateCommitIndex(ths, pMsg->commitIndex); - sTrace("vgId:%d, start to commit raft log in heartbeat. commit index:%" PRId64 "", ths->vgId, ths->commitIndex); - if (syncLogBufferCommit(ths->pLogBuf, ths, ths->commitIndex) < 0) { - sError("vgId:%d, failed to commit raft log since %s. commit index:%" PRId64 "", ths->vgId, terrstr(), - ths->commitIndex); - } - } - else { + } else { sError("error local cmd"); }