From 3af9da8f51751d1f74bf274bc6f6ff0b25f37eec Mon Sep 17 00:00:00 2001 From: dapan1121 Date: Fri, 21 Jan 2022 13:49:12 +0800 Subject: [PATCH 01/11] feature/qnode --- source/libs/qworker/inc/qworkerInt.h | 2 +- source/libs/qworker/src/qworker.c | 123 ++++++++++++++------------- source/libs/qworker/src/qworkerMsg.c | 7 +- 3 files changed, 67 insertions(+), 65 deletions(-) diff --git a/source/libs/qworker/inc/qworkerInt.h b/source/libs/qworker/inc/qworkerInt.h index 5f9b33f7e3..9a33268472 100644 --- a/source/libs/qworker/inc/qworkerInt.h +++ b/source/libs/qworker/inc/qworkerInt.h @@ -83,7 +83,7 @@ typedef struct SQWMsg { } SQWMsg; typedef struct SQWPhaseInput { - int8_t status; + int8_t taskStatus; int8_t taskType; int32_t code; qTaskInfo_t taskHandle; diff --git a/source/libs/qworker/src/qworker.c b/source/libs/qworker/src/qworker.c index 566356e255..cedb3fa926 100644 --- a/source/libs/qworker/src/qworker.c +++ b/source/libs/qworker/src/qworker.c @@ -240,14 +240,14 @@ void qwReleaseTaskStatus(int32_t rwType, SQWSchStatus *sch) { } -int32_t qwAcquireTaskCtx(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint64_t tId, int32_t rwType, SQWTaskCtx **ctx) { +int32_t qwAcquireTaskCtx(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint64_t tId, SQWTaskCtx **ctx) { char id[sizeof(qId) + sizeof(tId)] = {0}; QW_SET_QTID(id, qId, tId); - QW_LOCK(rwType, &mgmt->ctxLock); - *ctx = taosHashGet(mgmt->ctxHash, id, sizeof(id)); + //QW_LOCK(rwType, &mgmt->ctxLock); + *ctx = taosHashAcquire(mgmt->ctxHash, id, sizeof(id)); if (NULL == (*ctx)) { - QW_UNLOCK(rwType, &mgmt->ctxLock); + //QW_UNLOCK(rwType, &mgmt->ctxLock); QW_TASK_ELOG("ctx not in ctxHash, id:%s", id); QW_ERR_RET(TSDB_CODE_QRY_RES_CACHE_NOT_EXIST); } @@ -268,19 +268,19 @@ int32_t qwGetTaskCtx(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint64_t tI return TSDB_CODE_SUCCESS; } -int32_t qwAddTaskCtxImpl(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint64_t tId, int32_t rwType, int32_t status, SQWTaskCtx **ctx) { +int32_t qwAddTaskCtxImpl(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint64_t tId, bool acquire, int32_t status, SQWTaskCtx **ctx) { char id[sizeof(qId) + sizeof(tId)] = {0}; QW_SET_QTID(id, qId, tId); SQWTaskCtx nctx = {0}; - QW_LOCK(QW_WRITE, &mgmt->ctxLock); + //QW_LOCK(QW_WRITE, &mgmt->ctxLock); int32_t code = taosHashPut(mgmt->ctxHash, id, sizeof(id), &nctx, sizeof(SQWTaskCtx)); if (0 != code) { - QW_UNLOCK(QW_WRITE, &mgmt->ctxLock); + //QW_UNLOCK(QW_WRITE, &mgmt->ctxLock); if (HASH_NODE_EXIST(code)) { - if (rwType && ctx) { + if (acquire && ctx) { QW_RET(qwAcquireTaskCtx(QW_FPARAMS(), rwType, ctx)); } else if (ctx) { QW_RET(qwGetTaskCtx(QW_FPARAMS(), ctx)); @@ -293,9 +293,9 @@ int32_t qwAddTaskCtxImpl(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint64_ QW_ERR_RET(TSDB_CODE_QRY_OUT_OF_MEMORY); } } - QW_UNLOCK(QW_WRITE, &mgmt->ctxLock); + //QW_UNLOCK(QW_WRITE, &mgmt->ctxLock); - if (rwType && ctx) { + if (acquire && ctx) { QW_RET(qwAcquireTaskCtx(QW_FPARAMS(), rwType, ctx)); } else if (ctx) { QW_RET(qwGetTaskCtx(QW_FPARAMS(), ctx)); @@ -305,22 +305,23 @@ int32_t qwAddTaskCtxImpl(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint64_ } int32_t qwAddTaskCtx(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint64_t tId) { - QW_RET(qwAddTaskCtxImpl(QW_FPARAMS(), 0, 0, NULL)); + QW_RET(qwAddTaskCtxImpl(QW_FPARAMS(), false, 0, NULL)); } -int32_t qwAddAcquireTaskCtx(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint64_t tId, int32_t rwType, SQWTaskCtx **ctx) { - return qwAddTaskCtxImpl(QW_FPARAMS(), rwType, 0, ctx); +int32_t qwAddAcquireTaskCtx(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint64_t tId, SQWTaskCtx **ctx) { + return qwAddTaskCtxImpl(QW_FPARAMS(), true, 0, ctx); } int32_t qwAddGetTaskCtx(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint64_t tId, SQWTaskCtx **ctx) { - return qwAddTaskCtxImpl(QW_FPARAMS(), 0, 0, ctx); + return qwAddTaskCtxImpl(QW_FPARAMS(), false, 0, ctx); } -void qwReleaseTaskCtx(int32_t rwType, SQWorkerMgmt *mgmt) { - QW_UNLOCK(rwType, &mgmt->ctxLock); +void qwReleaseTaskCtx(SQWorkerMgmt *mgmt, void *ctx) { + //QW_UNLOCK(rwType, &mgmt->ctxLock); + taosHashRelease(mgmt->ctxHash, ctx); } void qwFreeTaskHandle(QW_FPARAMS_DEF, qTaskInfo_t *taskHandle) { @@ -355,7 +356,7 @@ void qwFreeTask(QW_FPARAMS_DEF, SQWTaskCtx *ctx) { // Note: NEED CTX HASH LOCKED BEFORE ENTRANCE -int32_t qwDropTaskCtx(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint64_t tId) { +int32_t qwDropTaskCtx(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint64_t tId, int32_t rwType) { char id[sizeof(qId) + sizeof(tId)] = {0}; QW_SET_QTID(id, qId, tId); SQWTaskCtx octx; @@ -367,6 +368,13 @@ int32_t qwDropTaskCtx(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint64_t t octx = *ctx; + atomic_store_ptr(&ctx->taskHandle, NULL); + atomic_store_ptr(&ctx->sinkHandle, NULL); + + if (rwType) { + QW_UNLOCK(rwType, &ctx->lock); + } + if (taosHashRemove(mgmt->ctxHash, id, sizeof(id))) { QW_TASK_ELOG_E("taosHashRemove from ctx hash failed"); QW_ERR_RET(TSDB_CODE_QRY_RES_CACHE_NOT_EXIST); @@ -439,7 +447,7 @@ _return: QW_RET(code); } -int32_t qwExecTask(QW_FPARAMS_DEF, qTaskInfo_t *taskHandle, DataSinkHandle sinkHandle, int8_t taskType) { +int32_t qwExecTask(QW_FPARAMS_DEF, qTaskInfo_t *taskHandle, DataSinkHandle sinkHandle, int8_t taskType, bool execOnce) { int32_t code = 0; bool qcontinue = true; SSDataBlock* pRes = NULL; @@ -463,6 +471,7 @@ int32_t qwExecTask(QW_FPARAMS_DEF, qTaskInfo_t *taskHandle, DataSinkHandle sinkH if (TASK_TYPE_TEMP == taskType) { qwFreeTaskHandle(QW_FPARAMS(), taskHandle); } + break; } @@ -475,7 +484,7 @@ int32_t qwExecTask(QW_FPARAMS_DEF, qTaskInfo_t *taskHandle, DataSinkHandle sinkH QW_TASK_DLOG("data put into sink, rows:%d, continueExecTask:%d", pRes->info.rows, qcontinue); - if (!qcontinue) { + if (execOnce || (!qcontinue)) { break; } } @@ -573,7 +582,6 @@ int32_t qwHandleTaskEvent(QW_FPARAMS_DEF, int8_t phase, SQWPhaseInput *input, SQ int8_t status = 0; SQWTaskCtx *ctx = NULL; bool locked = false; - bool ctxAcquired = false; void *readyConnection = NULL; void *dropConnection = NULL; void *cancelConnection = NULL; @@ -582,7 +590,9 @@ int32_t qwHandleTaskEvent(QW_FPARAMS_DEF, int8_t phase, SQWPhaseInput *input, SQ switch (phase) { case QW_PHASE_PRE_QUERY: { - QW_ERR_JRET(qwAddGetTaskCtx(QW_FPARAMS(), &ctx)); + QW_ERR_JRET(qwAddAcquireTaskCtx(QW_FPARAMS(), &ctx)); + + QW_LOCK(QW_WRITE, &ctx->lock); atomic_store_32(&ctx->phase, phase); atomic_store_8(&ctx->taskType, input->taskType); @@ -597,7 +607,7 @@ int32_t qwHandleTaskEvent(QW_FPARAMS_DEF, int8_t phase, SQWPhaseInput *input, SQ if (QW_IS_EVENT_RECEIVED(ctx, QW_EVENT_DROP)) { QW_ERR_JRET(qwDropTaskStatus(QW_FPARAMS())); - QW_ERR_JRET(qwDropTaskCtx(QW_FPARAMS())); + QW_ERR_JRET(qwDropTaskCtx(QW_FPARAMS(), QW_WRITE)); output->needStop = true; output->rspCode = TSDB_CODE_QRY_TASK_DROPPED; @@ -621,7 +631,7 @@ int32_t qwHandleTaskEvent(QW_FPARAMS_DEF, int8_t phase, SQWPhaseInput *input, SQ break; } case QW_PHASE_POST_QUERY: { - QW_ERR_JRET(qwGetTaskCtx(QW_FPARAMS(), &ctx)); + QW_ERR_JRET(qwAcquireTaskCtx(QW_FPARAMS(), &ctx)); QW_LOCK(QW_WRITE, &ctx->lock); @@ -641,8 +651,8 @@ int32_t qwHandleTaskEvent(QW_FPARAMS_DEF, int8_t phase, SQWPhaseInput *input, SQ if (QW_IS_EVENT_RECEIVED(ctx, QW_EVENT_DROP)) { output->needStop = true; - QW_ERR_JRET(qwDropTaskStatus(QW_FPARAMS())); - QW_ERR_JRET(qwDropTaskCtx(QW_FPARAMS())); + QW_ERR_JRET(qwDropTaskStatus(QW_FPARAMS())); + QW_ERR_JRET(qwDropTaskCtx(QW_FPARAMS(), QW_WRITE)); output->rspCode = TSDB_CODE_QRY_TASK_DROPPED; dropConnection = ctx->dropConnection; @@ -665,13 +675,12 @@ int32_t qwHandleTaskEvent(QW_FPARAMS_DEF, int8_t phase, SQWPhaseInput *input, SQ } if (!output->needStop) { - QW_ERR_JRET(qwUpdateTaskStatus(QW_FPARAMS(), input->status)); + QW_ERR_JRET(qwUpdateTaskStatus(QW_FPARAMS(), input->taskStatus)); } break; } case QW_PHASE_PRE_FETCH: { QW_ERR_JRET(qwAcquireTaskCtx(QW_FPARAMS(), QW_READ, &ctx)); - ctxAcquired = true; QW_LOCK(QW_WRITE, &ctx->lock); @@ -686,6 +695,13 @@ int32_t qwHandleTaskEvent(QW_FPARAMS_DEF, int8_t phase, SQWPhaseInput *input, SQ QW_ERR_JRET(TSDB_CODE_QRY_TASK_CANCELLED); } + if (QW_IS_EVENT_PROCESSED(ctx, QW_EVENT_DROP)) { + QW_TASK_WLOG("task already dropped, phase:%d", phase); + output->needStop = true; + output->rspCode = TSDB_CODE_QRY_TASK_DROPPED; + QW_ERR_JRET(TSDB_CODE_QRY_TASK_DROPPED); + } + if (QW_IS_EVENT_RECEIVED(ctx, QW_EVENT_DROP)) { QW_TASK_ELOG("drop event at wrong phase, phase:%d", phase); output->needStop = true; @@ -721,7 +737,7 @@ int32_t qwHandleTaskEvent(QW_FPARAMS_DEF, int8_t phase, SQWPhaseInput *input, SQ break; } case QW_PHASE_POST_FETCH: { - QW_ERR_JRET(qwGetTaskCtx(QW_FPARAMS(), &ctx)); + QW_ERR_JRET(qwAcquireTaskCtx(QW_FPARAMS(), &ctx)); QW_LOCK(QW_WRITE, &ctx->lock); @@ -743,7 +759,7 @@ int32_t qwHandleTaskEvent(QW_FPARAMS_DEF, int8_t phase, SQWPhaseInput *input, SQ output->needStop = true; QW_ERR_JRET(qwDropTaskStatus(QW_FPARAMS())); - QW_ERR_JRET(qwDropTaskCtx(QW_FPARAMS())); + QW_ERR_JRET(qwDropTaskCtx(QW_FPARAMS(), QW_WRITE)); output->rspCode = TSDB_CODE_QRY_TASK_DROPPED; dropConnection = ctx->dropConnection; @@ -772,7 +788,6 @@ int32_t qwHandleTaskEvent(QW_FPARAMS_DEF, int8_t phase, SQWPhaseInput *input, SQ } case QW_PHASE_PRE_CQUERY: { QW_ERR_JRET(qwAcquireTaskCtx(QW_FPARAMS(), QW_READ, &ctx)); - ctxAcquired = true; QW_LOCK(QW_WRITE, &ctx->lock); @@ -808,7 +823,7 @@ int32_t qwHandleTaskEvent(QW_FPARAMS_DEF, int8_t phase, SQWPhaseInput *input, SQ break; } case QW_PHASE_POST_CQUERY: { - QW_ERR_JRET(qwGetTaskCtx(QW_FPARAMS(), &ctx)); + QW_ERR_JRET(qwAcquireTaskCtx(QW_FPARAMS(), &ctx)); QW_LOCK(QW_WRITE, &ctx->lock); @@ -829,8 +844,8 @@ int32_t qwHandleTaskEvent(QW_FPARAMS_DEF, int8_t phase, SQWPhaseInput *input, SQ QW_TASK_WLOG("start to drop task, phase:%d", phase); output->needStop = true; - QW_ERR_JRET(qwDropTaskStatus(QW_FPARAMS())); - QW_ERR_JRET(qwDropTaskCtx(QW_FPARAMS())); + QW_ERR_JRET(qwDropTaskStatus(QW_FPARAMS())); + QW_ERR_JRET(qwDropTaskCtx(QW_FPARAMS(), QW_WRITE)); output->rspCode = TSDB_CODE_QRY_TASK_DROPPED; dropConnection = ctx->dropConnection; @@ -871,8 +886,8 @@ _return: QW_UNLOCK(QW_WRITE, &ctx->lock); } - if (ctxAcquired && ctx) { - qwReleaseTaskCtx(QW_READ, mgmt); + if (ctx) { + qwReleaseTaskCtx(mgmt, ctx); } if (readyConnection) { @@ -896,7 +911,7 @@ _return: } -int32_t qwProcessQuery(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint64_t tId, SQWMsg *qwMsg, int8_t taskType) { +int32_t qwProcessQuery(QW_FPARAMS_DEF, SQWMsg *qwMsg, int8_t taskType) { int32_t code = 0; bool queryRsped = false; bool needStop = false; @@ -944,7 +959,7 @@ int32_t qwProcessQuery(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint64_t queryRsped = true; if (pTaskInfo && sinkHandle) { - QW_ERR_JRET(qwExecTask(QW_FPARAMS(), &pTaskInfo, sinkHandle, taskType)); + QW_ERR_JRET(qwExecTask(QW_FPARAMS(), &pTaskInfo, sinkHandle, taskType, true)); } _return: @@ -954,14 +969,8 @@ _return: } if (!queryRsped) { - code = qwBuildAndSendQueryRsp(qwMsg->connection, rspCode); - if (TSDB_CODE_SUCCESS == code) { - QW_TASK_DLOG("query msg rsped, code:%d", rspCode); - } - - if (TSDB_CODE_SUCCESS == rspCode && code) { - rspCode = code; - } + qwBuildAndSendQueryRsp(qwMsg->connection, rspCode); + QW_TASK_DLOG("query msg rsped, code:%x", rspCode); } if (needStop) { @@ -971,12 +980,7 @@ _return: input.code = rspCode; input.taskHandle = pTaskInfo; input.sinkHandle = sinkHandle; - - if (TSDB_CODE_SUCCESS != rspCode) { - input.status = JOB_TASK_STATUS_FAILED; - } else { - input.status = JOB_TASK_STATUS_PARTIAL_SUCCEED; - } + input.taskStatus = rspCode ? JOB_TASK_STATUS_FAILED : JOB_TASK_STATUS_PARTIAL_SUCCEED; QW_ERR_RET(qwHandleTaskEvent(QW_FPARAMS(), QW_PHASE_POST_QUERY, &input, &output)); @@ -989,8 +993,8 @@ int32_t qwProcessReady(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint64_t int8_t phase = 0; bool needRsp = false; int32_t rspCode = 0; - - QW_ERR_JRET(qwGetTaskCtx(QW_FPARAMS(), &ctx)); + + QW_ERR_JRET(qwAcquireTaskCtx(QW_FPARAMS(), &ctx)); QW_LOCK(QW_WRITE, &ctx->lock); @@ -1020,6 +1024,7 @@ _return: if (ctx) { QW_UNLOCK(QW_WRITE, &ctx->lock); + qwReleaseTaskCtx(mgmt, ctx); } if (needRsp) { @@ -1059,7 +1064,7 @@ int32_t qwProcessCQuery(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint64_t DataSinkHandle sinkHandle = ctx->sinkHandle; - QW_ERR_JRET(qwExecTask(QW_FPARAMS(), &ctx->taskHandle, sinkHandle, ctx->taskType)); + QW_ERR_JRET(qwExecTask(QW_FPARAMS(), &ctx->taskHandle, sinkHandle, ctx->taskType, QW_IS_EVENT_RECEIVED(ctx, QW_EVENT_FETCH))); if (QW_IS_EVENT_RECEIVED(ctx, QW_EVENT_FETCH)) { SOutputData sOutput = {0}; @@ -1191,7 +1196,7 @@ int32_t qwProcessDrop(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint64_t t SQWTaskCtx *ctx = NULL; bool locked = false; - QW_ERR_JRET(qwAddAcquireTaskCtx(QW_FPARAMS(), QW_WRITE, &ctx)); + QW_ERR_JRET(qwAddAcquireTaskCtx(QW_FPARAMS(), &ctx)); QW_LOCK(QW_WRITE, &ctx->lock); @@ -1210,7 +1215,7 @@ int32_t qwProcessDrop(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint64_t t ctx->dropConnection = qwMsg->connection; } else if (ctx->phase > 0) { QW_ERR_JRET(qwDropTaskStatus(QW_FPARAMS())); - QW_ERR_JRET(qwDropTaskCtx(QW_FPARAMS())); + QW_ERR_JRET(qwDropTaskCtx(QW_FPARAMS(), QW_WRITE)); locked = false; needRsp = true; @@ -1226,12 +1231,8 @@ _return: QW_SET_RSP_CODE(ctx, code); } - if (locked) { - QW_UNLOCK(QW_WRITE, &ctx->lock); - } - if (ctx) { - qwReleaseTaskCtx(QW_WRITE, mgmt); + qwReleaseTaskCtx(mgmt, ctx); } if (TSDB_CODE_SUCCESS != code || needRsp) { diff --git a/source/libs/qworker/src/qworkerMsg.c b/source/libs/qworker/src/qworkerMsg.c index feb8fd645e..0b1200745b 100644 --- a/source/libs/qworker/src/qworkerMsg.c +++ b/source/libs/qworker/src/qworkerMsg.c @@ -273,7 +273,7 @@ int32_t qWorkerProcessQueryMsg(void *node, void *qWorkerMgmt, SRpcMsg *pMsg) { SQWorkerMgmt *mgmt = (SQWorkerMgmt *)qWorkerMgmt; if (NULL == msg || pMsg->contLen <= sizeof(*msg)) { - QW_ELOG("invalid query msg, contLen:%d", pMsg->contLen); + QW_ELOG("invalid query msg, msg:%p, msgLen:%d", msg, pMsg->contLen); QW_ERR_RET(TSDB_CODE_QRY_INVALID_INPUT); } @@ -307,7 +307,7 @@ int32_t qWorkerProcessCQueryMsg(void *node, void *qWorkerMgmt, SRpcMsg *pMsg) { SQWorkerMgmt *mgmt = (SQWorkerMgmt *)qWorkerMgmt; if (NULL == msg || pMsg->contLen <= sizeof(*msg)) { - QW_ELOG("invalid cquery msg, contLen:%d", pMsg->contLen); + QW_ELOG("invalid cquery msg, msg:%p, msgLen:%d", msg, pMsg->contLen); QW_ERR_RET(TSDB_CODE_QRY_INVALID_INPUT); } @@ -337,7 +337,7 @@ int32_t qWorkerProcessReadyMsg(void *node, void *qWorkerMgmt, SRpcMsg *pMsg){ SResReadyReq *msg = pMsg->pCont; if (NULL == msg || pMsg->contLen < sizeof(*msg)) { - qError("invalid task status msg"); + QW_ELOG("invalid task ready msg, msg:%p, msgLen:%d", msg, pMsg->contLen); QW_ERR_RET(TSDB_CODE_QRY_INVALID_INPUT); } @@ -398,6 +398,7 @@ int32_t qWorkerProcessFetchMsg(void *node, void *qWorkerMgmt, SRpcMsg *pMsg) { SQWorkerMgmt *mgmt = (SQWorkerMgmt *)qWorkerMgmt; if (NULL == msg || pMsg->contLen < sizeof(*msg)) { + QW_ELOG("invalid fetch msg, msg:%p, msgLen:%d", msg, pMsg->contLen); QW_ERR_RET(TSDB_CODE_QRY_INVALID_INPUT); } From ff0200ae2885cf4789b2a9b9a3af2025dd5db1d8 Mon Sep 17 00:00:00 2001 From: dapan1121 Date: Fri, 21 Jan 2022 19:52:01 +0800 Subject: [PATCH 02/11] feature/qnode --- include/libs/scheduler/scheduler.h | 8 +- include/util/taoserror.h | 2 +- source/client/src/clientImpl.c | 8 +- source/dnode/mgmt/impl/src/dndTransport.c | 1 + source/dnode/mgmt/impl/src/dndVnodes.c | 2 +- source/dnode/vnode/src/vnd/vnodeQuery.c | 4 +- source/libs/qworker/inc/qworkerInt.h | 21 +- source/libs/qworker/inc/qworkerMsg.h | 1 + source/libs/qworker/src/qworker.c | 542 +++++++++--------- source/libs/qworker/src/qworkerMsg.c | 9 +- source/libs/qworker/test/qworkerTests.cpp | 321 ++++++++++- source/libs/scheduler/src/scheduler.c | 49 +- source/libs/scheduler/test/schedulerTests.cpp | 20 +- source/util/src/terror.c | 2 +- 14 files changed, 646 insertions(+), 344 deletions(-) diff --git a/include/libs/scheduler/scheduler.h b/include/libs/scheduler/scheduler.h index 3262b9437c..2eb768713e 100644 --- a/include/libs/scheduler/scheduler.h +++ b/include/libs/scheduler/scheduler.h @@ -72,7 +72,7 @@ int32_t schedulerInit(SSchedulerCfg *cfg); * @param nodeList Qnode/Vnode address list, element is SQueryNodeAddr * @return */ -int32_t scheduleExecJob(void *transport, SArray *nodeList, SQueryDag* pDag, struct SSchJob** pJob, SQueryResult *pRes); +int32_t schedulerExecJob(void *transport, SArray *nodeList, SQueryDag* pDag, struct SSchJob** pJob, SQueryResult *pRes); /** * Process the query job, generated according to the query physical plan. @@ -80,7 +80,7 @@ int32_t scheduleExecJob(void *transport, SArray *nodeList, SQueryDag* pDag, stru * @param nodeList Qnode/Vnode address list, element is SQueryNodeAddr * @return */ -int32_t scheduleAsyncExecJob(void *transport, SArray *nodeList, SQueryDag* pDag, struct SSchJob** pJob); +int32_t schedulerAsyncExecJob(void *transport, SArray *nodeList, SQueryDag* pDag, struct SSchJob** pJob); /** * Fetch query result from the remote query executor @@ -88,7 +88,7 @@ int32_t scheduleAsyncExecJob(void *transport, SArray *nodeList, SQueryDag* pDag, * @param data * @return */ -int32_t scheduleFetchRows(struct SSchJob *pJob, void **data); +int32_t schedulerFetchRows(struct SSchJob *pJob, void **data); /** @@ -102,7 +102,7 @@ int32_t scheduleFetchRows(struct SSchJob *pJob, void **data); * Free the query job * @param pJob */ -void scheduleFreeJob(void *pJob); +void schedulerFreeJob(void *pJob); void schedulerDestroy(void); diff --git a/include/util/taoserror.h b/include/util/taoserror.h index 570c1d8375..8c048690c0 100644 --- a/include/util/taoserror.h +++ b/include/util/taoserror.h @@ -354,7 +354,7 @@ int32_t* taosGetErrno(); #define TSDB_CODE_QRY_SCH_NOT_EXIST TAOS_DEF_ERROR_CODE(0, 0x0710) //"Scheduler not exist") #define TSDB_CODE_QRY_TASK_NOT_EXIST TAOS_DEF_ERROR_CODE(0, 0x0711) //"Task not exist") #define TSDB_CODE_QRY_TASK_ALREADY_EXIST TAOS_DEF_ERROR_CODE(0, 0x0712) //"Task already exist") -#define TSDB_CODE_QRY_RES_CACHE_NOT_EXIST TAOS_DEF_ERROR_CODE(0, 0x0713) //"Task result cache not exist") +#define TSDB_CODE_QRY_TASK_CTX_NOT_EXIST TAOS_DEF_ERROR_CODE(0, 0x0713) //"Task context not exist") #define TSDB_CODE_QRY_TASK_CANCELLED TAOS_DEF_ERROR_CODE(0, 0x0714) //"Task cancelled") #define TSDB_CODE_QRY_TASK_DROPPED TAOS_DEF_ERROR_CODE(0, 0x0715) //"Task dropped") #define TSDB_CODE_QRY_TASK_CANCELLING TAOS_DEF_ERROR_CODE(0, 0x0716) //"Task cancelling") diff --git a/source/client/src/clientImpl.c b/source/client/src/clientImpl.c index 34ab1fb05a..1726a066d3 100644 --- a/source/client/src/clientImpl.c +++ b/source/client/src/clientImpl.c @@ -242,12 +242,12 @@ int32_t scheduleQuery(SRequestObj* pRequest, SQueryDag* pDag) { if (TSDB_SQL_INSERT == pRequest->type || TSDB_SQL_CREATE_TABLE == pRequest->type) { SQueryResult res = {.code = 0, .numOfRows = 0, .msgSize = ERROR_MSG_BUF_DEFAULT_SIZE, .msg = pRequest->msgBuf}; - int32_t code = scheduleExecJob(pRequest->pTscObj->pAppInfo->pTransporter, NULL, pDag, &pRequest->body.pQueryJob, &res); + int32_t code = schedulerExecJob(pRequest->pTscObj->pAppInfo->pTransporter, NULL, pDag, &pRequest->body.pQueryJob, &res); if (code != TSDB_CODE_SUCCESS) { // handle error and retry } else { if (pRequest->body.pQueryJob != NULL) { - scheduleFreeJob(pRequest->body.pQueryJob); + schedulerFreeJob(pRequest->body.pQueryJob); } } @@ -263,7 +263,7 @@ int32_t scheduleQuery(SRequestObj* pRequest, SQueryDag* pDag) { strcpy(addr.epAddr[0].fqdn, "localhost"); taosArrayPush(execNode, &addr); - return scheduleAsyncExecJob(pRequest->pTscObj->pAppInfo->pTransporter, execNode, pDag, &pRequest->body.pQueryJob); + return schedulerAsyncExecJob(pRequest->pTscObj->pAppInfo->pTransporter, execNode, pDag, &pRequest->body.pQueryJob); } typedef struct tmq_t tmq_t; @@ -714,7 +714,7 @@ void* doFetchRow(SRequestObj* pRequest) { return NULL; } - int32_t code = scheduleFetchRows(pRequest->body.pQueryJob, (void **)&pRequest->body.resInfo.pData); + int32_t code = schedulerFetchRows(pRequest->body.pQueryJob, (void **)&pRequest->body.resInfo.pData); if (code != TSDB_CODE_SUCCESS) { pRequest->code = code; return NULL; diff --git a/source/dnode/mgmt/impl/src/dndTransport.c b/source/dnode/mgmt/impl/src/dndTransport.c index f4fda75bd8..63ed87c0a3 100644 --- a/source/dnode/mgmt/impl/src/dndTransport.c +++ b/source/dnode/mgmt/impl/src/dndTransport.c @@ -116,6 +116,7 @@ static void dndInitMsgFp(STransMgmt *pMgmt) { // Requests handled by VNODE pMgmt->msgFp[TMSG_INDEX(TDMT_VND_SUBMIT)] = dndProcessVnodeWriteMsg; pMgmt->msgFp[TMSG_INDEX(TDMT_VND_QUERY)] = dndProcessVnodeQueryMsg; + pMgmt->msgFp[TMSG_INDEX(TDMT_VND_QUERY_CONTINUE)] = dndProcessVnodeQueryMsg; pMgmt->msgFp[TMSG_INDEX(TDMT_VND_FETCH)] = dndProcessVnodeFetchMsg; pMgmt->msgFp[TMSG_INDEX(TDMT_VND_ALTER_TABLE)] = dndProcessVnodeWriteMsg; pMgmt->msgFp[TMSG_INDEX(TDMT_VND_UPDATE_TAG_VAL)] = dndProcessVnodeWriteMsg; diff --git a/source/dnode/mgmt/impl/src/dndVnodes.c b/source/dnode/mgmt/impl/src/dndVnodes.c index 0d4b9c803d..d4ff853e85 100644 --- a/source/dnode/mgmt/impl/src/dndVnodes.c +++ b/source/dnode/mgmt/impl/src/dndVnodes.c @@ -892,7 +892,7 @@ int32_t dndPutReqToVQueryQ(SDnode *pDnode, SRpcMsg *pMsg) { SVnodeObj *pVnode = dndAcquireVnode(pDnode, pHead->vgId); if (pVnode == NULL) return -1; - int32_t code = dndWriteRpcMsgToVnodeQueue(pVnode->pFetchQ, pMsg, false); + int32_t code = dndWriteRpcMsgToVnodeQueue(pVnode->pQueryQ, pMsg, false); dndReleaseVnode(pDnode, pVnode); return code; } diff --git a/source/dnode/vnode/src/vnd/vnodeQuery.c b/source/dnode/vnode/src/vnd/vnodeQuery.c index dd1e5ba9ae..79ec7fda26 100644 --- a/source/dnode/vnode/src/vnd/vnodeQuery.c +++ b/source/dnode/vnode/src/vnd/vnodeQuery.c @@ -22,7 +22,7 @@ static int vnodeGetTableMeta(SVnode *pVnode, SRpcMsg *pMsg, SRpcMsg **pRsp); int vnodeQueryOpen(SVnode *pVnode) { return qWorkerInit(NODE_TYPE_VNODE, pVnode->vgId, NULL, &pVnode->pQuery, pVnode, vnodePutReqToVQueryQ); } int vnodeProcessQueryReq(SVnode *pVnode, SRpcMsg *pMsg, SRpcMsg **pRsp) { - vTrace("query message is processing"); + vTrace("message in query queue is processing"); switch (pMsg->msgType) { case TDMT_VND_QUERY: @@ -36,7 +36,7 @@ int vnodeProcessQueryReq(SVnode *pVnode, SRpcMsg *pMsg, SRpcMsg **pRsp) { } int vnodeProcessFetchReq(SVnode *pVnode, SRpcMsg *pMsg, SRpcMsg **pRsp) { - vTrace("fetch message is processed"); + vTrace("message in fetch queue is processing"); switch (pMsg->msgType) { case TDMT_VND_FETCH: return qWorkerProcessFetchMsg(pVnode, pVnode->pQuery, pMsg); diff --git a/source/libs/qworker/inc/qworkerInt.h b/source/libs/qworker/inc/qworkerInt.h index 9a33268472..9ecce3f5f9 100644 --- a/source/libs/qworker/inc/qworkerInt.h +++ b/source/libs/qworker/inc/qworkerInt.h @@ -22,17 +22,17 @@ extern "C" { #include "tlockfree.h" -#define QWORKER_DEFAULT_SCHEDULER_NUMBER 10000 -#define QWORKER_DEFAULT_TASK_NUMBER 10000 -#define QWORKER_DEFAULT_SCH_TASK_NUMBER 10000 - +#define QW_DEFAULT_SCHEDULER_NUMBER 10000 +#define QW_DEFAULT_TASK_NUMBER 10000 +#define QW_DEFAULT_SCH_TASK_NUMBER 10000 +#define QW_DEFAULT_SHORT_RUN_TIMES 2 enum { QW_PHASE_PRE_QUERY = 1, QW_PHASE_POST_QUERY, - QW_PHASE_PRE_CQUERY, - QW_PHASE_POST_CQUERY, QW_PHASE_PRE_FETCH, QW_PHASE_POST_FETCH, + QW_PHASE_PRE_CQUERY, + QW_PHASE_POST_CQUERY, }; enum { @@ -133,7 +133,7 @@ typedef struct SQWorkerMgmt { int8_t nodeType; int32_t nodeId; SRWLatch schLock; - SRWLatch ctxLock; + //SRWLatch ctxLock; SHashObj *schHash; //key: schedulerId, value: SQWSchStatus SHashObj *ctxHash; //key: queryId+taskId, value: SQWTaskCtx void *nodeObj; @@ -144,6 +144,8 @@ typedef struct SQWorkerMgmt { #define QW_IDS() sId, qId, tId #define QW_FPARAMS() mgmt, QW_IDS() +#define QW_GET_EVENT_VALUE(ctx, event) atomic_load_8(&(ctx)->events[event]) + #define QW_IS_EVENT_RECEIVED(ctx, event) (atomic_load_8(&(ctx)->events[event]) == QW_EVENT_RECEIVED) #define QW_IS_EVENT_PROCESSED(ctx, event) (atomic_load_8(&(ctx)->events[event]) == QW_EVENT_PROCESSED) #define QW_SET_EVENT_RECEIVED(ctx, event) atomic_store_8(&(ctx)->events[event], QW_EVENT_RECEIVED) @@ -151,9 +153,10 @@ typedef struct SQWorkerMgmt { #define QW_GET_PHASE(ctx) atomic_load_8(&(ctx)->phase) -#define QW_SET_RSP_CODE(ctx, code) atomic_val_compare_exchange_32(&(ctx)->rspCode, 0, code) +#define QW_SET_RSP_CODE(ctx, code) atomic_store_32(&(ctx)->rspCode, code) +#define QW_UPDATE_RSP_CODE(ctx, code) atomic_val_compare_exchange_32(&(ctx)->rspCode, 0, code) -#define QW_IN_EXECUTOR(ctx) (QW_GET_PHASE(ctx) == QW_PHASE_PRE_QUERY || QW_GET_PHASE(ctx) == QW_PHASE_PRE_CQUERY || QW_GET_PHASE(ctx) == QW_PHASE_PRE_FETCH) +#define QW_IS_QUERY_RUNNING(ctx) (QW_GET_PHASE(ctx) == QW_PHASE_PRE_QUERY || QW_GET_PHASE(ctx) == QW_PHASE_PRE_CQUERY) #define QW_TASK_NOT_EXIST(code) (TSDB_CODE_QRY_SCH_NOT_EXIST == (code) || TSDB_CODE_QRY_TASK_NOT_EXIST == (code)) #define QW_TASK_ALREADY_EXIST(code) (TSDB_CODE_QRY_TASK_ALREADY_EXIST == (code)) diff --git a/source/libs/qworker/inc/qworkerMsg.h b/source/libs/qworker/inc/qworkerMsg.h index 7ecc2b2b20..7735e1a1ee 100644 --- a/source/libs/qworker/inc/qworkerMsg.h +++ b/source/libs/qworker/inc/qworkerMsg.h @@ -30,6 +30,7 @@ int32_t qwProcessFetch(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint64_t int32_t qwProcessDrop(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint64_t tId, SQWMsg *qwMsg); int32_t qwBuildAndSendDropRsp(void *connection, int32_t code); +int32_t qwBuildAndSendCancelRsp(SRpcMsg *pMsg, int32_t code); int32_t qwBuildAndSendFetchRsp(void *connection, SRetrieveTableRsp *pRsp, int32_t dataLength, int32_t code); void qwBuildFetchRsp(void *msg, SOutputData *input, int32_t len); int32_t qwBuildAndSendCQueryMsg(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint64_t tId, void *connection); diff --git a/source/libs/qworker/src/qworker.c b/source/libs/qworker/src/qworker.c index cedb3fa926..bcfbafd4c9 100644 --- a/source/libs/qworker/src/qworker.c +++ b/source/libs/qworker/src/qworker.c @@ -11,7 +11,7 @@ SQWDebug gQWDebug = {0}; -int32_t qwValidateStatus(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint64_t tId, int8_t oriStatus, int8_t newStatus) { +int32_t qwValidateStatus(QW_FPARAMS_DEF, int8_t oriStatus, int8_t newStatus) { int32_t code = 0; if (oriStatus == newStatus) { @@ -35,6 +35,7 @@ int32_t qwValidateStatus(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint64_ break; case JOB_TASK_STATUS_EXECUTING: if (newStatus != JOB_TASK_STATUS_PARTIAL_SUCCEED + && newStatus != JOB_TASK_STATUS_SUCCEED && newStatus != JOB_TASK_STATUS_FAILED && newStatus != JOB_TASK_STATUS_CANCELLING && newStatus != JOB_TASK_STATUS_CANCELLED @@ -77,7 +78,7 @@ _return: QW_RET(code); } -int32_t qwSetTaskStatus(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint64_t tId, SQWTaskStatus *task, int8_t status) { +int32_t qwSetTaskStatus(QW_FPARAMS_DEF, SQWTaskStatus *task, int8_t status) { int32_t code = 0; int8_t origStatus = 0; @@ -99,7 +100,7 @@ int32_t qwSetTaskStatus(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint64_t } -int32_t qwAddSchedulerImpl(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint64_t tId, int32_t rwType, SQWSchStatus **sch) { +int32_t qwAddSchedulerImpl(QW_FPARAMS_DEF, int32_t rwType, SQWSchStatus **sch) { SQWSchStatus newSch = {0}; newSch.tasksHash = taosHashInit(mgmt->cfg.maxSchTaskNum, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY), false, HASH_NO_LOCK); if (NULL == newSch.tasksHash) { @@ -125,7 +126,7 @@ int32_t qwAddSchedulerImpl(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint6 return TSDB_CODE_SUCCESS; } -int32_t qwAcquireSchedulerImpl(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint64_t tId, int32_t rwType, SQWSchStatus **sch, int32_t nOpt) { +int32_t qwAcquireSchedulerImpl(QW_FPARAMS_DEF, int32_t rwType, SQWSchStatus **sch, int32_t nOpt) { while (true) { QW_LOCK(rwType, &mgmt->schLock); *sch = taosHashGet(mgmt->schHash, &sId, sizeof(sId)); @@ -152,11 +153,11 @@ int32_t qwAcquireSchedulerImpl(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, u return TSDB_CODE_SUCCESS; } -int32_t qwAcquireAddScheduler(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint64_t tId, int32_t rwType, SQWSchStatus **sch) { +int32_t qwAcquireAddScheduler(QW_FPARAMS_DEF, int32_t rwType, SQWSchStatus **sch) { return qwAcquireSchedulerImpl(QW_FPARAMS(), rwType, sch, QW_NOT_EXIST_ADD); } -int32_t qwAcquireScheduler(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint64_t tId, int32_t rwType, SQWSchStatus **sch) { +int32_t qwAcquireScheduler(QW_FPARAMS_DEF, int32_t rwType, SQWSchStatus **sch) { return qwAcquireSchedulerImpl(QW_FPARAMS(), rwType, sch, QW_NOT_EXIST_RET_ERR); } @@ -165,7 +166,7 @@ void qwReleaseScheduler(int32_t rwType, SQWorkerMgmt *mgmt) { } -int32_t qwAcquireTaskStatus(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint64_t tId, int32_t rwType, SQWSchStatus *sch, SQWTaskStatus **task) { +int32_t qwAcquireTaskStatus(QW_FPARAMS_DEF, int32_t rwType, SQWSchStatus *sch, SQWTaskStatus **task) { char id[sizeof(qId) + sizeof(tId)] = {0}; QW_SET_QTID(id, qId, tId); @@ -181,7 +182,7 @@ int32_t qwAcquireTaskStatus(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint -int32_t qwAddTaskStatusImpl(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint64_t tId, SQWSchStatus *sch, int32_t rwType, int32_t status, SQWTaskStatus **task) { +int32_t qwAddTaskStatusImpl(QW_FPARAMS_DEF, SQWSchStatus *sch, int32_t rwType, int32_t status, SQWTaskStatus **task) { int32_t code = 0; char id[sizeof(qId) + sizeof(tId)] = {0}; @@ -215,7 +216,7 @@ int32_t qwAddTaskStatusImpl(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint return TSDB_CODE_SUCCESS; } -int32_t qwAddTaskStatus(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint64_t tId, int32_t status) { +int32_t qwAddTaskStatus(QW_FPARAMS_DEF, int32_t status) { SQWSchStatus *tsch = NULL; int32_t code = 0; QW_ERR_RET(qwAcquireAddScheduler(QW_FPARAMS(), QW_READ, &tsch)); @@ -230,7 +231,7 @@ _return: } -int32_t qwAddAcquireTaskStatus(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint64_t tId, int32_t rwType, SQWSchStatus *sch, int32_t status, SQWTaskStatus **task) { +int32_t qwAddAcquireTaskStatus(QW_FPARAMS_DEF, int32_t rwType, SQWSchStatus *sch, int32_t status, SQWTaskStatus **task) { return qwAddTaskStatusImpl(QW_FPARAMS(), sch, rwType, status, task); } @@ -240,7 +241,7 @@ void qwReleaseTaskStatus(int32_t rwType, SQWSchStatus *sch) { } -int32_t qwAcquireTaskCtx(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint64_t tId, SQWTaskCtx **ctx) { +int32_t qwAcquireTaskCtx(QW_FPARAMS_DEF, SQWTaskCtx **ctx) { char id[sizeof(qId) + sizeof(tId)] = {0}; QW_SET_QTID(id, qId, tId); @@ -249,26 +250,26 @@ int32_t qwAcquireTaskCtx(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint64_ if (NULL == (*ctx)) { //QW_UNLOCK(rwType, &mgmt->ctxLock); QW_TASK_ELOG("ctx not in ctxHash, id:%s", id); - QW_ERR_RET(TSDB_CODE_QRY_RES_CACHE_NOT_EXIST); + QW_ERR_RET(TSDB_CODE_QRY_TASK_CTX_NOT_EXIST); } return TSDB_CODE_SUCCESS; } -int32_t qwGetTaskCtx(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint64_t tId, SQWTaskCtx **ctx) { +int32_t qwGetTaskCtx(QW_FPARAMS_DEF, SQWTaskCtx **ctx) { char id[sizeof(qId) + sizeof(tId)] = {0}; QW_SET_QTID(id, qId, tId); *ctx = taosHashGet(mgmt->ctxHash, id, sizeof(id)); if (NULL == (*ctx)) { QW_TASK_ELOG("ctx not in ctxHash, ctxHashSize:%d", taosHashGetSize(mgmt->ctxHash)); - QW_ERR_RET(TSDB_CODE_QRY_RES_CACHE_NOT_EXIST); + QW_ERR_RET(TSDB_CODE_QRY_TASK_CTX_NOT_EXIST); } return TSDB_CODE_SUCCESS; } -int32_t qwAddTaskCtxImpl(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint64_t tId, bool acquire, int32_t status, SQWTaskCtx **ctx) { +int32_t qwAddTaskCtxImpl(QW_FPARAMS_DEF, bool acquire, int32_t status, SQWTaskCtx **ctx) { char id[sizeof(qId) + sizeof(tId)] = {0}; QW_SET_QTID(id, qId, tId); @@ -281,7 +282,7 @@ int32_t qwAddTaskCtxImpl(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint64_ if (HASH_NODE_EXIST(code)) { if (acquire && ctx) { - QW_RET(qwAcquireTaskCtx(QW_FPARAMS(), rwType, ctx)); + QW_RET(qwAcquireTaskCtx(QW_FPARAMS(), ctx)); } else if (ctx) { QW_RET(qwGetTaskCtx(QW_FPARAMS(), ctx)); } else { @@ -296,7 +297,7 @@ int32_t qwAddTaskCtxImpl(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint64_ //QW_UNLOCK(QW_WRITE, &mgmt->ctxLock); if (acquire && ctx) { - QW_RET(qwAcquireTaskCtx(QW_FPARAMS(), rwType, ctx)); + QW_RET(qwAcquireTaskCtx(QW_FPARAMS(), ctx)); } else if (ctx) { QW_RET(qwGetTaskCtx(QW_FPARAMS(), ctx)); } @@ -304,17 +305,17 @@ int32_t qwAddTaskCtxImpl(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint64_ return TSDB_CODE_SUCCESS; } -int32_t qwAddTaskCtx(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint64_t tId) { +int32_t qwAddTaskCtx(QW_FPARAMS_DEF) { QW_RET(qwAddTaskCtxImpl(QW_FPARAMS(), false, 0, NULL)); } -int32_t qwAddAcquireTaskCtx(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint64_t tId, SQWTaskCtx **ctx) { +int32_t qwAddAcquireTaskCtx(QW_FPARAMS_DEF, SQWTaskCtx **ctx) { return qwAddTaskCtxImpl(QW_FPARAMS(), true, 0, ctx); } -int32_t qwAddGetTaskCtx(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint64_t tId, SQWTaskCtx **ctx) { +int32_t qwAddGetTaskCtx(QW_FPARAMS_DEF, SQWTaskCtx **ctx) { return qwAddTaskCtxImpl(QW_FPARAMS(), false, 0, ctx); } @@ -356,14 +357,14 @@ void qwFreeTask(QW_FPARAMS_DEF, SQWTaskCtx *ctx) { // Note: NEED CTX HASH LOCKED BEFORE ENTRANCE -int32_t qwDropTaskCtx(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint64_t tId, int32_t rwType) { +int32_t qwDropTaskCtx(QW_FPARAMS_DEF, int32_t rwType) { char id[sizeof(qId) + sizeof(tId)] = {0}; QW_SET_QTID(id, qId, tId); SQWTaskCtx octx; SQWTaskCtx *ctx = taosHashGet(mgmt->ctxHash, id, sizeof(id)); if (NULL == ctx) { - QW_ERR_RET(TSDB_CODE_QRY_RES_CACHE_NOT_EXIST); + QW_ERR_RET(TSDB_CODE_QRY_TASK_CTX_NOT_EXIST); } octx = *ctx; @@ -371,13 +372,15 @@ int32_t qwDropTaskCtx(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint64_t t atomic_store_ptr(&ctx->taskHandle, NULL); atomic_store_ptr(&ctx->sinkHandle, NULL); + QW_SET_EVENT_PROCESSED(ctx, QW_EVENT_DROP); + if (rwType) { QW_UNLOCK(rwType, &ctx->lock); } if (taosHashRemove(mgmt->ctxHash, id, sizeof(id))) { QW_TASK_ELOG_E("taosHashRemove from ctx hash failed"); - QW_ERR_RET(TSDB_CODE_QRY_RES_CACHE_NOT_EXIST); + QW_ERR_RET(TSDB_CODE_QRY_TASK_CTX_NOT_EXIST); } if (octx.taskHandle) { @@ -394,7 +397,7 @@ int32_t qwDropTaskCtx(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint64_t t } -int32_t qwDropTaskStatus(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint64_t tId) { +int32_t qwDropTaskStatus(QW_FPARAMS_DEF) { SQWSchStatus *sch = NULL; SQWTaskStatus *task = NULL; int32_t code = 0; @@ -429,7 +432,7 @@ _return: QW_RET(code); } -int32_t qwUpdateTaskStatus(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint64_t tId, int8_t status) { +int32_t qwUpdateTaskStatus(QW_FPARAMS_DEF, int8_t status) { SQWSchStatus *sch = NULL; SQWTaskStatus *task = NULL; int32_t code = 0; @@ -447,12 +450,13 @@ _return: QW_RET(code); } -int32_t qwExecTask(QW_FPARAMS_DEF, qTaskInfo_t *taskHandle, DataSinkHandle sinkHandle, int8_t taskType, bool execOnce) { +int32_t qwExecTask(QW_FPARAMS_DEF, qTaskInfo_t *taskHandle, DataSinkHandle sinkHandle, int8_t taskType, bool shortRun) { int32_t code = 0; bool qcontinue = true; SSDataBlock* pRes = NULL; uint64_t useconds = 0; int32_t i = 0; + int32_t leftRun = QW_DEFAULT_SHORT_RUN_TIMES; while (true) { QW_TASK_DLOG("start to execTask in executor, loopIdx:%d", i++); @@ -484,7 +488,11 @@ int32_t qwExecTask(QW_FPARAMS_DEF, qTaskInfo_t *taskHandle, DataSinkHandle sinkH QW_TASK_DLOG("data put into sink, rows:%d, continueExecTask:%d", pRes->info.rows, qcontinue); - if (execOnce || (!qcontinue)) { + if (!qcontinue) { + break; + } + + if (shortRun && ((--leftRun) <= 0)) { break; } } @@ -576,29 +584,35 @@ int32_t qwGetResFromSink(QW_FPARAMS_DEF, SQWTaskCtx *ctx, int32_t *dataLen, void return TSDB_CODE_SUCCESS; } - -int32_t qwHandleTaskEvent(QW_FPARAMS_DEF, int8_t phase, SQWPhaseInput *input, SQWPhaseOutput *output) { +int32_t qwHandlePrePhaseEvents(QW_FPARAMS_DEF, int8_t phase, SQWPhaseInput *input, SQWPhaseOutput *output) { int32_t code = 0; int8_t status = 0; SQWTaskCtx *ctx = NULL; bool locked = false; - void *readyConnection = NULL; void *dropConnection = NULL; void *cancelConnection = NULL; QW_SCH_TASK_DLOG("start to handle event at phase %d", phase); + output->needStop = false; + + if (QW_PHASE_PRE_QUERY == phase) { + QW_ERR_JRET(qwAddAcquireTaskCtx(QW_FPARAMS(), &ctx)); + } else { + QW_ERR_JRET(qwAcquireTaskCtx(QW_FPARAMS(), &ctx)); + } + + QW_LOCK(QW_WRITE, &ctx->lock); + locked = true; + + atomic_store_32(&ctx->phase, phase); + switch (phase) { case QW_PHASE_PRE_QUERY: { - QW_ERR_JRET(qwAddAcquireTaskCtx(QW_FPARAMS(), &ctx)); - - QW_LOCK(QW_WRITE, &ctx->lock); - - atomic_store_32(&ctx->phase, phase); atomic_store_8(&ctx->taskType, input->taskType); - if (QW_IS_EVENT_PROCESSED(ctx, QW_EVENT_CANCEL)) { - QW_TASK_ELOG("task already cancelled at wrong phase, phase:%d", phase); + if (QW_IS_EVENT_PROCESSED(ctx, QW_EVENT_CANCEL) || QW_IS_EVENT_PROCESSED(ctx, QW_EVENT_DROP)) { + QW_TASK_ELOG("task already cancelled/dropped at wrong phase, phase:%d", phase); output->needStop = true; output->rspCode = TSDB_CODE_QRY_TASK_STATUS_ERROR; @@ -611,18 +625,27 @@ int32_t qwHandleTaskEvent(QW_FPARAMS_DEF, int8_t phase, SQWPhaseInput *input, SQ output->needStop = true; output->rspCode = TSDB_CODE_QRY_TASK_DROPPED; + QW_SET_RSP_CODE(ctx, output->rspCode); dropConnection = ctx->dropConnection; // Note: ctx freed, no need to unlock it locked = false; } else if (QW_IS_EVENT_RECEIVED(ctx, QW_EVENT_CANCEL)) { - output->needStop = true; - QW_ERR_JRET(qwAddTaskStatus(QW_FPARAMS(), JOB_TASK_STATUS_CANCELLED)); QW_SET_EVENT_PROCESSED(ctx, QW_EVENT_CANCEL); - + output->needStop = true; output->rspCode = TSDB_CODE_QRY_TASK_CANCELLED; + QW_SET_RSP_CODE(ctx, output->rspCode); + + cancelConnection = ctx->cancelConnection; + } + + if (ctx->rspCode) { + QW_TASK_ELOG("task already failed at wrong phase, code:%x, phase:%d", ctx->rspCode, phase); + output->needStop = true; + output->rspCode = ctx->rspCode; + QW_ERR_JRET(output->rspCode); } if (!output->needStop) { @@ -630,64 +653,55 @@ int32_t qwHandleTaskEvent(QW_FPARAMS_DEF, int8_t phase, SQWPhaseInput *input, SQ } break; } - case QW_PHASE_POST_QUERY: { - QW_ERR_JRET(qwAcquireTaskCtx(QW_FPARAMS(), &ctx)); - - QW_LOCK(QW_WRITE, &ctx->lock); - - locked = true; - - ctx->taskHandle = input->taskHandle; - ctx->sinkHandle = input->sinkHandle; - - if (NULL == ctx->taskHandle && NULL == ctx->sinkHandle) { - ctx->emptyRes = true; + case QW_PHASE_PRE_FETCH: { + if (QW_IS_EVENT_PROCESSED(ctx, QW_EVENT_DROP)) { + QW_TASK_WLOG("task already dropped, phase:%d", phase); + output->needStop = true; + output->rspCode = TSDB_CODE_QRY_TASK_DROPPED; + QW_ERR_JRET(TSDB_CODE_QRY_TASK_DROPPED); } - - if (input->code) { - output->rspCode = input->code; + if (QW_IS_EVENT_PROCESSED(ctx, QW_EVENT_CANCEL)) { + QW_TASK_WLOG("task already cancelled, phase:%d", phase); + output->needStop = true; + output->rspCode = TSDB_CODE_QRY_TASK_CANCELLED; + QW_ERR_JRET(TSDB_CODE_QRY_TASK_CANCELLED); } if (QW_IS_EVENT_RECEIVED(ctx, QW_EVENT_DROP)) { + QW_TASK_ELOG("drop event at wrong phase, phase:%d", phase); output->needStop = true; - - QW_ERR_JRET(qwDropTaskStatus(QW_FPARAMS())); - QW_ERR_JRET(qwDropTaskCtx(QW_FPARAMS(), QW_WRITE)); - - output->rspCode = TSDB_CODE_QRY_TASK_DROPPED; - dropConnection = ctx->dropConnection; - - // Note: ctx freed, no need to unlock it - locked = false; + output->rspCode = TSDB_CODE_QRY_TASK_STATUS_ERROR; + QW_ERR_JRET(TSDB_CODE_QRY_TASK_CANCELLED); } else if (QW_IS_EVENT_RECEIVED(ctx, QW_EVENT_CANCEL)) { + QW_TASK_ELOG("cancel event at wrong phase, phase:%d", phase); output->needStop = true; - - QW_ERR_JRET(qwUpdateTaskStatus(QW_FPARAMS(), JOB_TASK_STATUS_CANCELLED)); - qwFreeTask(QW_FPARAMS(), ctx); - - QW_SET_EVENT_PROCESSED(ctx, QW_EVENT_CANCEL); - - output->rspCode = TSDB_CODE_QRY_TASK_CANCELLED; - } else if (QW_IS_EVENT_RECEIVED(ctx, QW_EVENT_READY)) { - readyConnection = ctx->readyConnection; - - QW_SET_EVENT_PROCESSED(ctx, QW_EVENT_READY); + output->rspCode = TSDB_CODE_QRY_TASK_STATUS_ERROR; + QW_ERR_JRET(TSDB_CODE_QRY_TASK_CANCELLED); } - if (!output->needStop) { - QW_ERR_JRET(qwUpdateTaskStatus(QW_FPARAMS(), input->taskStatus)); + if (ctx->rspCode) { + QW_TASK_ELOG("task already failed, code:%x, phase:%d", ctx->rspCode, phase); + output->needStop = true; + output->rspCode = ctx->rspCode; + QW_ERR_JRET(output->rspCode); + } + + if (QW_IS_EVENT_RECEIVED(ctx, QW_EVENT_FETCH)) { + QW_TASK_WLOG("last fetch not finished, phase:%d", phase); + output->needStop = true; + output->rspCode = TSDB_CODE_QRY_DUPLICATTED_OPERATION; + QW_ERR_JRET(TSDB_CODE_QRY_DUPLICATTED_OPERATION); + } + + if (!QW_IS_EVENT_PROCESSED(ctx, QW_EVENT_READY)) { + QW_TASK_ELOG("query rsp are not ready, phase:%d", phase); + output->needStop = true; + output->rspCode = TSDB_CODE_QRY_TASK_MSG_ERROR; + QW_ERR_JRET(TSDB_CODE_QRY_TASK_MSG_ERROR); } break; - } - case QW_PHASE_PRE_FETCH: { - QW_ERR_JRET(qwAcquireTaskCtx(QW_FPARAMS(), QW_READ, &ctx)); - - QW_LOCK(QW_WRITE, &ctx->lock); - - locked = true; - - atomic_store_32(&ctx->phase, phase); - + } + case QW_PHASE_PRE_CQUERY: { if (QW_IS_EVENT_PROCESSED(ctx, QW_EVENT_CANCEL)) { QW_TASK_WLOG("task already cancelled, phase:%d", phase); output->needStop = true; @@ -714,20 +728,6 @@ int32_t qwHandleTaskEvent(QW_FPARAMS_DEF, int8_t phase, SQWPhaseInput *input, SQ QW_ERR_JRET(TSDB_CODE_QRY_TASK_CANCELLED); } - if (QW_IS_EVENT_RECEIVED(ctx, QW_EVENT_FETCH)) { - QW_TASK_WLOG("last fetch not finished, phase:%d", phase); - output->needStop = true; - output->rspCode = TSDB_CODE_QRY_DUPLICATTED_OPERATION; - QW_ERR_JRET(TSDB_CODE_QRY_DUPLICATTED_OPERATION); - } - - if (!QW_IS_EVENT_PROCESSED(ctx, QW_EVENT_READY)) { - QW_TASK_ELOG("query rsp are not ready, phase:%d", phase); - output->needStop = true; - output->rspCode = TSDB_CODE_QRY_TASK_MSG_ERROR; - QW_ERR_JRET(TSDB_CODE_QRY_TASK_MSG_ERROR); - } - if (ctx->rspCode) { QW_TASK_ELOG("task already failed, code:%x, phase:%d", ctx->rspCode, phase); output->needStop = true; @@ -736,160 +736,156 @@ int32_t qwHandleTaskEvent(QW_FPARAMS_DEF, int8_t phase, SQWPhaseInput *input, SQ } break; } - case QW_PHASE_POST_FETCH: { - QW_ERR_JRET(qwAcquireTaskCtx(QW_FPARAMS(), &ctx)); - - QW_LOCK(QW_WRITE, &ctx->lock); - - locked = true; - - if (input->code) { - output->rspCode = input->code; - } - - if (QW_IS_EVENT_PROCESSED(ctx, QW_EVENT_CANCEL)) { - QW_TASK_WLOG("task already cancelled, phase:%d", phase); - output->needStop = true; - output->rspCode = TSDB_CODE_QRY_TASK_CANCELLED; - QW_ERR_JRET(TSDB_CODE_QRY_TASK_CANCELLED); - } - - if (QW_IS_EVENT_RECEIVED(ctx, QW_EVENT_DROP)) { - QW_TASK_WLOG("start to drop task, phase:%d", phase); - output->needStop = true; - - QW_ERR_JRET(qwDropTaskStatus(QW_FPARAMS())); - QW_ERR_JRET(qwDropTaskCtx(QW_FPARAMS(), QW_WRITE)); - - output->rspCode = TSDB_CODE_QRY_TASK_DROPPED; - dropConnection = ctx->dropConnection; - - // Note: ctx freed, no need to unlock it - locked = false; - } else if (QW_IS_EVENT_RECEIVED(ctx, QW_EVENT_CANCEL)) { - QW_TASK_WLOG("start to cancel task, phase:%d", phase); - output->needStop = true; - - QW_ERR_JRET(qwUpdateTaskStatus(QW_FPARAMS(), JOB_TASK_STATUS_CANCELLED)); - - QW_SET_EVENT_PROCESSED(ctx, QW_EVENT_CANCEL); - - output->rspCode = TSDB_CODE_QRY_TASK_CANCELLED; - cancelConnection = ctx->cancelConnection; - } - - if (ctx->rspCode) { - QW_TASK_ELOG("task failed, code:%x, phase:%d", ctx->rspCode, phase); - output->needStop = true; - output->rspCode = ctx->rspCode; - QW_ERR_JRET(output->rspCode); - } - break; - } - case QW_PHASE_PRE_CQUERY: { - QW_ERR_JRET(qwAcquireTaskCtx(QW_FPARAMS(), QW_READ, &ctx)); - - QW_LOCK(QW_WRITE, &ctx->lock); - - locked = true; - - atomic_store_32(&ctx->phase, phase); - - if (QW_IS_EVENT_PROCESSED(ctx, QW_EVENT_CANCEL)) { - QW_TASK_WLOG("task already cancelled, phase:%d", phase); - output->needStop = true; - output->rspCode = TSDB_CODE_QRY_TASK_CANCELLED; - QW_ERR_JRET(TSDB_CODE_QRY_TASK_CANCELLED); - } - - if (QW_IS_EVENT_RECEIVED(ctx, QW_EVENT_DROP)) { - QW_TASK_ELOG("drop event at wrong phase, phase:%d", phase); - output->needStop = true; - output->rspCode = TSDB_CODE_QRY_TASK_STATUS_ERROR; - QW_ERR_JRET(TSDB_CODE_QRY_TASK_CANCELLED); - } else if (QW_IS_EVENT_RECEIVED(ctx, QW_EVENT_CANCEL)) { - QW_TASK_ELOG("cancel event at wrong phase, phase:%d", phase); - output->needStop = true; - output->rspCode = TSDB_CODE_QRY_TASK_STATUS_ERROR; - QW_ERR_JRET(TSDB_CODE_QRY_TASK_CANCELLED); - } - - if (ctx->rspCode) { - QW_TASK_ELOG("task already failed, code:%x, phase:%d", ctx->rspCode, phase); - output->needStop = true; - output->rspCode = ctx->rspCode; - QW_ERR_JRET(output->rspCode); - } - break; - } - case QW_PHASE_POST_CQUERY: { - QW_ERR_JRET(qwAcquireTaskCtx(QW_FPARAMS(), &ctx)); - - QW_LOCK(QW_WRITE, &ctx->lock); - - locked = true; - - if (input->code) { - output->rspCode = input->code; - } - - if (QW_IS_EVENT_PROCESSED(ctx, QW_EVENT_CANCEL)) { - QW_TASK_WLOG("task already cancelled, phase:%d", phase); - output->needStop = true; - output->rspCode = TSDB_CODE_QRY_TASK_CANCELLED; - QW_ERR_JRET(TSDB_CODE_QRY_TASK_CANCELLED); - } - - if (QW_IS_EVENT_RECEIVED(ctx, QW_EVENT_DROP)) { - QW_TASK_WLOG("start to drop task, phase:%d", phase); - output->needStop = true; - - QW_ERR_JRET(qwDropTaskStatus(QW_FPARAMS())); - QW_ERR_JRET(qwDropTaskCtx(QW_FPARAMS(), QW_WRITE)); - - output->rspCode = TSDB_CODE_QRY_TASK_DROPPED; - dropConnection = ctx->dropConnection; - - // Note: ctx freed, no need to unlock it - locked = false; - } else if (QW_IS_EVENT_RECEIVED(ctx, QW_EVENT_CANCEL)) { - QW_TASK_WLOG("start to cancel task, phase:%d", phase); - output->needStop = true; - - QW_ERR_JRET(qwUpdateTaskStatus(QW_FPARAMS(), JOB_TASK_STATUS_CANCELLED)); - - QW_SET_EVENT_PROCESSED(ctx, QW_EVENT_CANCEL); - - output->rspCode = TSDB_CODE_QRY_TASK_CANCELLED; - cancelConnection = ctx->cancelConnection; - } - - if (ctx->rspCode) { - QW_TASK_ELOG("task failed, code:%x, phase:%d", ctx->rspCode, phase); - output->needStop = true; - output->rspCode = ctx->rspCode; - QW_ERR_JRET(output->rspCode); - } - break; - } } _return: - if (output->rspCode) { - QW_SET_RSP_CODE(ctx, output->rspCode); + if (ctx) { + if (output->rspCode) { + QW_UPDATE_RSP_CODE(ctx, output->rspCode); + } + + if (locked) { + QW_UNLOCK(QW_WRITE, &ctx->lock); + } + + qwReleaseTaskCtx(mgmt, ctx); } - if (locked) { - atomic_store_32(&ctx->phase, phase); - - QW_UNLOCK(QW_WRITE, &ctx->lock); + if (code) { + output->needStop = true; + if (TSDB_CODE_SUCCESS == output->rspCode) { + output->rspCode = code; + } } + if (dropConnection) { + qwBuildAndSendDropRsp(dropConnection, output->rspCode); + QW_TASK_DLOG("drop msg rsped, code:%x", output->rspCode); + } + + if (cancelConnection) { + qwBuildAndSendCancelRsp(cancelConnection, output->rspCode); + QW_TASK_DLOG("cancel msg rsped, code:%x", output->rspCode); + } + + QW_SCH_TASK_DLOG("end to handle event at phase %d", phase); + + QW_RET(code); +} + + +int32_t qwHandlePostPhaseEvents(QW_FPARAMS_DEF, int8_t phase, SQWPhaseInput *input, SQWPhaseOutput *output) { + int32_t code = 0; + int8_t status = 0; + SQWTaskCtx *ctx = NULL; + bool locked = false; + void *readyConnection = NULL; + void *dropConnection = NULL; + void *cancelConnection = NULL; + + QW_SCH_TASK_DLOG("start to handle event at phase %d", phase); + + output->needStop = false; + + QW_ERR_JRET(qwAcquireTaskCtx(QW_FPARAMS(), &ctx)); + + QW_LOCK(QW_WRITE, &ctx->lock); + locked = true; + + if (QW_IS_EVENT_PROCESSED(ctx, QW_EVENT_DROP)) { + QW_TASK_WLOG("task already dropped, phase:%d", phase); + output->needStop = true; + output->rspCode = TSDB_CODE_QRY_TASK_DROPPED; + QW_ERR_JRET(TSDB_CODE_QRY_TASK_DROPPED); + } + + if (QW_IS_EVENT_PROCESSED(ctx, QW_EVENT_CANCEL)) { + QW_TASK_WLOG("task already cancelled, phase:%d", phase); + output->needStop = true; + output->rspCode = TSDB_CODE_QRY_TASK_CANCELLED; + QW_ERR_JRET(TSDB_CODE_QRY_TASK_CANCELLED); + } + + if (input->code) { + output->rspCode = input->code; + } + + if (QW_PHASE_POST_QUERY == phase) { + ctx->taskHandle = input->taskHandle; + ctx->sinkHandle = input->sinkHandle; + + if (NULL == ctx->taskHandle && NULL == ctx->sinkHandle) { + ctx->emptyRes = true; + } + + if (QW_IS_EVENT_RECEIVED(ctx, QW_EVENT_READY)) { + readyConnection = ctx->readyConnection; + QW_SET_EVENT_PROCESSED(ctx, QW_EVENT_READY); + } + } + + if (QW_IS_EVENT_RECEIVED(ctx, QW_EVENT_DROP)) { + QW_ERR_JRET(qwDropTaskStatus(QW_FPARAMS())); + QW_ERR_JRET(qwDropTaskCtx(QW_FPARAMS(), QW_WRITE)); + + output->rspCode = TSDB_CODE_QRY_TASK_DROPPED; + output->needStop = true; + QW_SET_RSP_CODE(ctx, output->rspCode); + dropConnection = ctx->dropConnection; + + // Note: ctx freed, no need to unlock it + locked = false; + + QW_ERR_JRET(output->rspCode); + } else if (QW_IS_EVENT_RECEIVED(ctx, QW_EVENT_CANCEL)) { + QW_ERR_JRET(qwUpdateTaskStatus(QW_FPARAMS(), JOB_TASK_STATUS_CANCELLED)); + qwFreeTask(QW_FPARAMS(), ctx); + + QW_SET_EVENT_PROCESSED(ctx, QW_EVENT_CANCEL); + + output->needStop = true; + output->rspCode = TSDB_CODE_QRY_TASK_CANCELLED; + QW_SET_RSP_CODE(ctx, output->rspCode); + cancelConnection = ctx->cancelConnection; + + QW_ERR_JRET(output->rspCode); + } + + if (ctx->rspCode) { + QW_TASK_ELOG("task failed, code:%x, phase:%d", ctx->rspCode, phase); + output->needStop = true; + output->rspCode = ctx->rspCode; + QW_ERR_JRET(output->rspCode); + } + + if (QW_PHASE_POST_QUERY == phase && (!output->needStop)) { + QW_ERR_JRET(qwUpdateTaskStatus(QW_FPARAMS(), input->taskStatus)); + } + +_return: + if (ctx) { + if (output->rspCode) { + QW_UPDATE_RSP_CODE(ctx, output->rspCode); + } + + atomic_store_32(&ctx->phase, phase); + + if (locked) { + QW_UNLOCK(QW_WRITE, &ctx->lock); + } + qwReleaseTaskCtx(mgmt, ctx); } + if (code) { + output->needStop = true; + if (TSDB_CODE_SUCCESS == output->rspCode) { + output->rspCode = code; + } + } + if (readyConnection) { qwBuildAndSendReadyRsp(readyConnection, output->rspCode); QW_TASK_DLOG("ready msg rsped, code:%x", output->rspCode); @@ -924,7 +920,7 @@ int32_t qwProcessQuery(QW_FPARAMS_DEF, SQWMsg *qwMsg, int8_t taskType) { input.taskType = taskType; - QW_ERR_JRET(qwHandleTaskEvent(QW_FPARAMS(), QW_PHASE_PRE_QUERY, &input, &output)); + QW_ERR_JRET(qwHandlePrePhaseEvents(QW_FPARAMS(), QW_PHASE_PRE_QUERY, &input, &output)); needStop = output.needStop; code = output.rspCode; @@ -973,21 +969,17 @@ _return: QW_TASK_DLOG("query msg rsped, code:%x", rspCode); } - if (needStop) { - QW_RET(rspCode); - } - input.code = rspCode; input.taskHandle = pTaskInfo; input.sinkHandle = sinkHandle; input.taskStatus = rspCode ? JOB_TASK_STATUS_FAILED : JOB_TASK_STATUS_PARTIAL_SUCCEED; - QW_ERR_RET(qwHandleTaskEvent(QW_FPARAMS(), QW_PHASE_POST_QUERY, &input, &output)); + QW_ERR_RET(qwHandlePostPhaseEvents(QW_FPARAMS(), QW_PHASE_POST_QUERY, &input, &output)); QW_RET(rspCode); } -int32_t qwProcessReady(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint64_t tId, SQWMsg *qwMsg) { +int32_t qwProcessReady(QW_FPARAMS_DEF, SQWMsg *qwMsg) { int32_t code = 0; SQWTaskCtx *ctx = NULL; int8_t phase = 0; @@ -998,6 +990,12 @@ int32_t qwProcessReady(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint64_t QW_LOCK(QW_WRITE, &ctx->lock); + if (QW_IS_EVENT_PROCESSED(ctx, QW_EVENT_CANCEL) || QW_IS_EVENT_PROCESSED(ctx, QW_EVENT_DROP) || + QW_IS_EVENT_RECEIVED(ctx, QW_EVENT_CANCEL) || QW_IS_EVENT_RECEIVED(ctx, QW_EVENT_DROP)) { + QW_TASK_WLOG("task already cancelled/dropped, phase:%d", phase); + QW_ERR_JRET(TSDB_CODE_QRY_TASK_CANCELLED); + } + phase = QW_GET_PHASE(ctx); if (phase == QW_PHASE_PRE_QUERY) { @@ -1019,7 +1017,7 @@ int32_t qwProcessReady(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint64_t _return: if (code && ctx) { - QW_SET_RSP_CODE(ctx, code); + QW_UPDATE_RSP_CODE(ctx, code); } if (ctx) { @@ -1036,7 +1034,7 @@ _return: } -int32_t qwProcessCQuery(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint64_t tId, SQWMsg *qwMsg) { +int32_t qwProcessCQuery(QW_FPARAMS_DEF, SQWMsg *qwMsg) { SQWTaskCtx *ctx = NULL; int32_t code = 0; bool queryRsped = false; @@ -1048,7 +1046,7 @@ int32_t qwProcessCQuery(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint64_t int32_t dataLen = 0; do { - QW_ERR_JRET(qwHandleTaskEvent(QW_FPARAMS(), QW_PHASE_PRE_CQUERY, &input, &output)); + QW_ERR_JRET(qwHandlePrePhaseEvents(QW_FPARAMS(), QW_PHASE_PRE_CQUERY, &input, &output)); needStop = output.needStop; code = output.rspCode; @@ -1100,7 +1098,7 @@ int32_t qwProcessCQuery(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint64_t } input.code = code; - qwHandleTaskEvent(QW_FPARAMS(), QW_PHASE_POST_CQUERY, &input, &output); + qwHandlePostPhaseEvents(QW_FPARAMS(), QW_PHASE_POST_CQUERY, &input, &output); needStop = output.needStop; code = output.rspCode; @@ -1110,7 +1108,7 @@ int32_t qwProcessCQuery(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint64_t } -int32_t qwProcessFetch(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint64_t tId, SQWMsg *qwMsg) { +int32_t qwProcessFetch(QW_FPARAMS_DEF, SQWMsg *qwMsg) { int32_t code = 0; int32_t needRsp = true; void *data = NULL; @@ -1126,7 +1124,7 @@ int32_t qwProcessFetch(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint64_t SQWPhaseInput input = {0}; SQWPhaseOutput output = {0}; - QW_ERR_JRET(qwHandleTaskEvent(QW_FPARAMS(), QW_PHASE_PRE_FETCH, &input, &output)); + QW_ERR_JRET(qwHandlePrePhaseEvents(QW_FPARAMS(), QW_PHASE_PRE_FETCH, &input, &output)); needStop = output.needStop; code = output.rspCode; @@ -1154,7 +1152,7 @@ int32_t qwProcessFetch(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint64_t locked = true; // RC WARNING - if (QW_IN_EXECUTOR(ctx)) { + if (QW_IS_QUERY_RUNNING(ctx)) { atomic_store_8(&ctx->queryContinue, 1); } else if (0 == atomic_load_8(&ctx->queryInQueue)) { QW_ERR_JRET(qwUpdateTaskStatus(QW_FPARAMS(), JOB_TASK_STATUS_EXECUTING)); @@ -1162,6 +1160,8 @@ int32_t qwProcessFetch(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint64_t atomic_store_8(&ctx->queryInQueue, 1); QW_ERR_JRET(qwBuildAndSendCQueryMsg(QW_FPARAMS(), qwMsg->connection)); + + QW_TASK_DLOG("schedule query in queue, phase:%d", ctx->phase); } } @@ -1173,7 +1173,11 @@ _return: input.code = code; - qwHandleTaskEvent(QW_FPARAMS(), QW_PHASE_POST_FETCH, &input, &output); + qwHandlePostPhaseEvents(QW_FPARAMS(), QW_PHASE_POST_FETCH, &input, &output); + + if (output.rspCode) { + code = output.rspCode; + } if (code) { qwFreeFetchRsp(rsp); @@ -1190,7 +1194,7 @@ _return: } -int32_t qwProcessDrop(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint64_t tId, SQWMsg *qwMsg) { +int32_t qwProcessDrop(QW_FPARAMS_DEF, SQWMsg *qwMsg) { int32_t code = 0; bool needRsp = false; SQWTaskCtx *ctx = NULL; @@ -1207,7 +1211,7 @@ int32_t qwProcessDrop(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint64_t t QW_ERR_JRET(TSDB_CODE_QRY_DUPLICATTED_OPERATION); } - if (QW_IN_EXECUTOR(ctx)) { + if (QW_IS_QUERY_RUNNING(ctx)) { QW_ERR_JRET(qwKillTaskHandle(QW_FPARAMS(), ctx)); QW_ERR_JRET(qwUpdateTaskStatus(QW_FPARAMS(), JOB_TASK_STATUS_DROPPING)); @@ -1217,8 +1221,12 @@ int32_t qwProcessDrop(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint64_t t QW_ERR_JRET(qwDropTaskStatus(QW_FPARAMS())); QW_ERR_JRET(qwDropTaskCtx(QW_FPARAMS(), QW_WRITE)); + QW_SET_RSP_CODE(ctx, TSDB_CODE_QRY_TASK_DROPPED); + locked = false; needRsp = true; + + QW_ERR_JRET(TSDB_CODE_QRY_TASK_DROPPED); } if (!needRsp) { @@ -1228,7 +1236,7 @@ int32_t qwProcessDrop(SQWorkerMgmt *mgmt, uint64_t sId, uint64_t qId, uint64_t t _return: if (code) { - QW_SET_RSP_CODE(ctx, code); + QW_UPDATE_RSP_CODE(ctx, code); } if (ctx) { @@ -1259,18 +1267,18 @@ int32_t qWorkerInit(int8_t nodeType, int32_t nodeId, SQWorkerCfg *cfg, void **qW if (cfg) { mgmt->cfg = *cfg; if (0 == mgmt->cfg.maxSchedulerNum) { - mgmt->cfg.maxSchedulerNum = QWORKER_DEFAULT_SCHEDULER_NUMBER; + mgmt->cfg.maxSchedulerNum = QW_DEFAULT_SCHEDULER_NUMBER; } if (0 == mgmt->cfg.maxTaskNum) { - mgmt->cfg.maxTaskNum = QWORKER_DEFAULT_TASK_NUMBER; + mgmt->cfg.maxTaskNum = QW_DEFAULT_TASK_NUMBER; } if (0 == mgmt->cfg.maxSchTaskNum) { - mgmt->cfg.maxSchTaskNum = QWORKER_DEFAULT_SCH_TASK_NUMBER; + mgmt->cfg.maxSchTaskNum = QW_DEFAULT_SCH_TASK_NUMBER; } } else { - mgmt->cfg.maxSchedulerNum = QWORKER_DEFAULT_SCHEDULER_NUMBER; - mgmt->cfg.maxTaskNum = QWORKER_DEFAULT_TASK_NUMBER; - mgmt->cfg.maxSchTaskNum = QWORKER_DEFAULT_SCH_TASK_NUMBER; + mgmt->cfg.maxSchedulerNum = QW_DEFAULT_SCHEDULER_NUMBER; + mgmt->cfg.maxTaskNum = QW_DEFAULT_TASK_NUMBER; + mgmt->cfg.maxSchTaskNum = QW_DEFAULT_SCH_TASK_NUMBER; } mgmt->schHash = taosHashInit(mgmt->cfg.maxSchedulerNum, taosGetDefaultHashFunction(TSDB_DATA_TYPE_UBIGINT), false, HASH_ENTRY_LOCK); diff --git a/source/libs/qworker/src/qworkerMsg.c b/source/libs/qworker/src/qworkerMsg.c index 0b1200745b..baa4ad2a04 100644 --- a/source/libs/qworker/src/qworkerMsg.c +++ b/source/libs/qworker/src/qworkerMsg.c @@ -306,15 +306,11 @@ int32_t qWorkerProcessCQueryMsg(void *node, void *qWorkerMgmt, SRpcMsg *pMsg) { SQWTaskCtx *handles = NULL; SQWorkerMgmt *mgmt = (SQWorkerMgmt *)qWorkerMgmt; - if (NULL == msg || pMsg->contLen <= sizeof(*msg)) { + if (NULL == msg || pMsg->contLen < sizeof(*msg)) { QW_ELOG("invalid cquery msg, msg:%p, msgLen:%d", msg, pMsg->contLen); QW_ERR_RET(TSDB_CODE_QRY_INVALID_INPUT); } - msg->sId = be64toh(msg->sId); - msg->queryId = be64toh(msg->queryId); - msg->taskId = be64toh(msg->taskId); - uint64_t sId = msg->sId; uint64_t qId = msg->queryId; uint64_t tId = msg->taskId; @@ -335,14 +331,13 @@ int32_t qWorkerProcessReadyMsg(void *node, void *qWorkerMgmt, SRpcMsg *pMsg){ return TSDB_CODE_QRY_INVALID_INPUT; } + SQWorkerMgmt *mgmt = (SQWorkerMgmt *)qWorkerMgmt; SResReadyReq *msg = pMsg->pCont; if (NULL == msg || pMsg->contLen < sizeof(*msg)) { QW_ELOG("invalid task ready msg, msg:%p, msgLen:%d", msg, pMsg->contLen); QW_ERR_RET(TSDB_CODE_QRY_INVALID_INPUT); } - SQWorkerMgmt *mgmt = (SQWorkerMgmt *)qWorkerMgmt; - msg->sId = be64toh(msg->sId); msg->queryId = be64toh(msg->queryId); msg->taskId = be64toh(msg->taskId); diff --git a/source/libs/qworker/test/qworkerTests.cpp b/source/libs/qworker/test/qworkerTests.cpp index d1cc9f03d1..a94af4f69b 100644 --- a/source/libs/qworker/test/qworkerTests.cpp +++ b/source/libs/qworker/test/qworkerTests.cpp @@ -38,6 +38,10 @@ namespace { +#define qwtTestQueryQueueSize 1000 +#define qwtTestFetchQueueSize 1000 +#define qwtTestMaxExecTaskUsec 1000000 + bool qwtTestEnableSleep = true; bool qwtTestStop = false; bool qwtTestDeadLoop = true; @@ -45,6 +49,36 @@ int32_t qwtTestMTRunSec = 10; int32_t qwtTestPrintNum = 100000; int32_t qwtTestCaseIdx = 0; int32_t qwtTestCaseNum = 4; +bool qwtTestCaseFinished = false; +tsem_t qwtTestQuerySem; +tsem_t qwtTestFetchSem; + +int32_t qwtTestQueryQueueRIdx = 0; +int32_t qwtTestQueryQueueWIdx = 0; +int32_t qwtTestQueryQueueNum = 0; +SRWLatch qwtTestQueryQueueLock = 0; +struct SRpcMsg *qwtTestQueryQueue[qwtTestQueryQueueSize] = {0}; + +int32_t qwtTestFetchQueueRIdx = 0; +int32_t qwtTestFetchQueueWIdx = 0; +int32_t qwtTestFetchQueueNum = 0; +SRWLatch qwtTestFetchQueueLock = 0; +struct SRpcMsg *qwtTestFetchQueue[qwtTestFetchQueueSize] = {0}; + + +int32_t qwtTestSinkBlockNum = 0; +int32_t qwtTestSinkMaxBlockNum = 0; +bool qwtTestSinkQueryEnd = false; +SRWLatch qwtTestSinkLock = 0; + + +SRpcMsg qwtfetchRpc = {0}; +SResFetchReq qwtfetchMsg = {0}; +SRpcMsg qwtreadyRpc = {0}; +SResReadyReq qwtreadyMsg = {0}; +SRpcMsg qwtdropRpc = {0}; +STaskDropReq qwtdropMsg = {0}; + void qwtInitLogFile() { const char *defaultLogFileNamePrefix = "taosdlog"; @@ -103,30 +137,112 @@ void qwtBuildStatusReqMsg(SSchTasksStatusReq *statusMsg, SRpcMsg *statusRpc) { } int32_t qwtStringToPlan(const char* str, SSubplan** subplan) { + *subplan = 0x1; return 0; } -int32_t qwtPutReqToQueue(void *node, struct SRpcMsg *pMsg) { +int32_t qwtPutReqToFetchQueue(void *node, struct SRpcMsg *pMsg) { + taosWLockLatch(&qwtTestFetchQueueLock); + qwtTestFetchQueue[qwtTestFetchQueueWIdx++] = pMsg; + if (qwtTestFetchQueueWIdx >= qwtTestFetchQueueSize) { + qwtTestFetchQueueWIdx = 0; + } + + qwtTestFetchQueueNum++; + + if (qwtTestFetchQueueWIdx == qwtTestFetchQueueRIdx) { + printf("Fetch queue is full"); + assert(0); + } + taosWUnLockLatch(&qwtTestFetchQueueLock); + + tsem_post(&qwtTestFetchSem); + return 0; } +int32_t qwtPutReqToQueue(void *node, struct SRpcMsg *pMsg) { + taosWLockLatch(&qwtTestQueryQueueLock); + qwtTestQueryQueue[qwtTestQueryQueueWIdx++] = pMsg; + if (qwtTestQueryQueueWIdx >= qwtTestQueryQueueSize) { + qwtTestQueryQueueWIdx = 0; + } + + qwtTestQueryQueueNum++; + + if (qwtTestQueryQueueWIdx == qwtTestQueryQueueRIdx) { + printf("query queue is full"); + assert(0); + } + taosWUnLockLatch(&qwtTestQueryQueueLock); + + tsem_post(&qwtTestQuerySem); + + return 0; +} + + + void qwtRpcSendResponse(const SRpcMsg *pRsp) { -/* - if (TDMT_VND_TASKS_STATUS_RSP == pRsp->msgType) { - SSchedulerStatusRsp *rsp = (SSchedulerStatusRsp *)pRsp->pCont; - printf("task num:%d\n", rsp->num); - for (int32_t i = 0; i < rsp->num; ++i) { - STaskStatus *task = &rsp->status[i]; - printf("qId:%"PRIx64",tId:%"PRIx64",status:%d\n", task->queryId, task->taskId, task->status); + + switch (pRsp->msgType) { + case TDMT_VND_QUERY_RSP: { + SQueryTableRsp *rsp = (SQueryTableRsp *)pRsp->pCont; + + if (0 == pRsp->code) { + qwtBuildReadyReqMsg(&qwtreadyMsg, &qwtreadyRpc); + qwtPutReqToFetchQueue(0x1, &qwtreadyRpc); + } else { + qwtBuildDropReqMsg(&qwtdropMsg, &qwtdropRpc); + qwtPutReqToFetchQueue(0x1, &qwtdropRpc); + } + + break; + } + case TDMT_VND_RES_READY_RSP: { + SResReadyRsp *rsp = (SResReadyRsp *)pRsp->pCont; + + if (0 == pRsp->code) { + qwtBuildFetchReqMsg(&qwtfetchMsg, &qwtfetchRpc); + qwtPutReqToFetchQueue(0x1, &qwtfetchRpc); + } else { + qwtBuildDropReqMsg(&qwtdropMsg, &qwtdropRpc); + qwtPutReqToFetchQueue(0x1, &qwtdropRpc); + } + break; + } + case TDMT_VND_FETCH_RSP: { + SRetrieveTableRsp *rsp = (SRetrieveTableRsp *)pRsp->pCont; + + if (0 == pRsp->code && 0 == rsp->completed) { + qwtBuildFetchReqMsg(&qwtfetchMsg, &qwtfetchRpc); + qwtPutReqToFetchQueue(0x1, &qwtfetchRpc); + return; + } + + qwtBuildDropReqMsg(&qwtdropMsg, &qwtdropRpc); + qwtPutReqToFetchQueue(0x1, &qwtdropRpc); + + break; + } + case TDMT_VND_DROP_TASK: { + STaskDropRsp *rsp = (STaskDropRsp *)pRsp->pCont; + + qwtTestCaseFinished = true; + break; } } -*/ + return; } int32_t qwtCreateExecTask(void* tsdb, int32_t vgId, struct SSubplan* pPlan, qTaskInfo_t* pTaskInfo, DataSinkHandle* handle) { - int32_t idx = qwtTestCaseIdx % qwtTestCaseNum; + int32_t idx = abs((++qwtTestCaseIdx) % qwtTestCaseNum); + + qwtTestSinkBlockNum = 0; + qwtTestSinkMaxBlockNum = rand() % 100 + 1; + qwtTestSinkQueryEnd = false; if (0 == idx) { *pTaskInfo = (qTaskInfo_t)qwtTestCaseIdx; @@ -141,13 +257,30 @@ int32_t qwtCreateExecTask(void* tsdb, int32_t vgId, struct SSubplan* pPlan, qTas *pTaskInfo = NULL; *handle = (DataSinkHandle)qwtTestCaseIdx; } - - ++qwtTestCaseIdx; return 0; } int32_t qwtExecTask(qTaskInfo_t tinfo, SSDataBlock** pRes, uint64_t *useconds) { + int32_t endExec = 0; + + if (NULL == tinfo) { + *pRes = NULL; + *useconds = 0; + } else { + endExec = rand() % 5; + + if (endExec) { + usleep(rand() % qwtTestMaxExecTaskUsec); + + *pRes = (SSDataBlock*)0x1; + } else { + *pRes = NULL; + usleep(rand() % qwtTestMaxExecTaskUsec); + *useconds = rand() % 10; + } + } + return 0; } @@ -156,21 +289,85 @@ int32_t qwtKillTask(qTaskInfo_t qinfo) { } void qwtDestroyTask(qTaskInfo_t qHandle) { - } int32_t qwtPutDataBlock(DataSinkHandle handle, const SInputData* pInput, bool* pContinue) { + if (NULL == handle || NULL == pInput || NULL == pContinue) { + assert(0); + } + + taosWLockLatch(&qwtTestSinkLock); + + qwtTestSinkBlockNum++; + + if (qwtTestSinkBlockNum >= qwtTestSinkMaxBlockNum) { + *pContinue = true; + } else { + *pContinue = false; + } + taosWUnLockLatch(&qwtTestSinkLock); + return 0; } void qwtEndPut(DataSinkHandle handle, uint64_t useconds) { + if (NULL == handle) { + assert(0); + } + + qwtTestSinkQueryEnd = true; } void qwtGetDataLength(DataSinkHandle handle, int32_t* pLen, bool* pQueryEnd) { + static int32_t in = 0; + + if (in > 0) { + assert(0); + } + + atomic_add_fetch_32(&in, 1); + + if (NULL == handle) { + assert(0); + } + + taosWLockLatch(&qwtTestSinkLock); + if (qwtTestSinkBlockNum > 0) { + *pLen = rand() % 100 + 1; + qwtTestSinkBlockNum--; + } else { + *pLen = 0; + } + taosWUnLockLatch(&qwtTestSinkLock); + + *pQueryEnd = qwtTestSinkQueryEnd; + + atomic_sub_fetch_32(&in, 1); } int32_t qwtGetDataBlock(DataSinkHandle handle, SOutputData* pOutput) { + taosWLockLatch(&qwtTestSinkLock); + if (qwtTestSinkBlockNum > 0) { + qwtTestSinkBlockNum--; + pOutput->numOfRows = rand() % 10 + 1; + pOutput->compressed = 1; + pOutput->pData = malloc(pOutput->numOfRows); + pOutput->queryEnd = qwtTestSinkQueryEnd; + if (qwtTestSinkBlockNum == 0) { + pOutput->bufStatus = DS_BUF_EMPTY; + } else if (qwtTestSinkBlockNum <= qwtTestSinkMaxBlockNum*0.5) { + pOutput->bufStatus = DS_BUF_LOW; + } else { + pOutput->bufStatus = DS_BUF_FULL; + } + pOutput->useconds = rand() % 10 + 1; + pOutput->precision = 1; + } else { + assert(0); + } + taosWUnLockLatch(&qwtTestSinkLock); + return 0; } @@ -438,20 +635,31 @@ void *statusThread(void *param) { } -void *controlThread(void *param) { - SRpcMsg queryRpc = {0}; +void *clientThread(void *param) { int32_t code = 0; uint32_t n = 0; - void *mockPointer = (void *)0x1; void *mgmt = param; + void *mockPointer = (void *)0x1; + SRpcMsg queryRpc = {0}; + + sleep(1); while (!qwtTestStop) { + qwtTestCaseFinished = false; + qwtBuildQueryReqMsg(&queryRpc); - qWorkerProcessQueryMsg(mockPointer, mgmt, &queryRpc); + qwtPutReqToQueue(0x1, &queryRpc); + + while (!qwtTestCaseFinished) { + usleep(1); + } + free(queryRpc.pCont); + if (qwtTestEnableSleep) { usleep(rand()%5); } + if (++n % qwtTestPrintNum == 0) { printf("query:%d\n", n); } @@ -461,10 +669,79 @@ void *controlThread(void *param) { } void *queryQueueThread(void *param) { + void *mockPointer = (void *)0x1; + SRpcMsg *queryRpc = NULL; + void *mgmt = param; + + while (!qwtTestStop) { + tsem_wait(&qwtTestQuerySem); + + taosWLockLatch(&qwtTestQueryQueueLock); + if (qwtTestQueryQueueNum <= 0 || qwtTestQueryQueueRIdx == qwtTestQueryQueueWIdx) { + printf("query queue is empty\n"); + assert(0); + } + + queryRpc = qwtTestQueryQueue[qwtTestQueryQueueRIdx++]; + + if (qwtTestQueryQueueRIdx >= qwtTestQueryQueueSize) { + qwtTestQueryQueueRIdx = 0; + } + + qwtTestQueryQueueNum--; + taosWUnLockLatch(&qwtTestQueryQueueLock); + + if (TDMT_VND_QUERY == queryRpc->msgType) { + qWorkerProcessQueryMsg(mockPointer, mgmt, &queryRpc); + } else if (TDMT_VND_QUERY_CONTINUE == queryRpc->msgType) { + qWorkerProcessCQueryMsg(mockPointer, mgmt, &queryRpc) + } else { + printf("unknown msg in query queue, type:%d\n", queryRpc->msgType); + assert(0); + } + } } void *fetchQueueThread(void *param) { + void *mockPointer = (void *)0x1; + SRpcMsg *fetchRpc = NULL; + void *mgmt = param; + + while (!qwtTestStop) { + tsem_wait(&qwtTestFetchSem); + + taosWLockLatch(&qwtTestFetchQueueLock); + if (qwtTestFetchQueueNum <= 0 || qwtTestFetchQueueRIdx == qwtTestFetchQueueWIdx) { + printf("Fetch queue is empty\n"); + assert(0); + } + + fetchRpc = qwtTestFetchQueue[qwtTestFetchQueueRIdx++]; + + if (qwtTestFetchQueueRIdx >= qwtTestFetchQueueSize) { + qwtTestFetchQueueRIdx = 0; + } + + qwtTestFetchQueueNum--; + taosWUnLockLatch(&qwtTestFetchQueueLock); + + switch (fetchRpc->msgType) { + case TDMT_VND_FETCH: + qWorkerProcessFetchMsg(mockPointer, mgmt, fetchRpc); + case TDMT_VND_RES_READY: + qWorkerProcessReadyMsg(mockPointer, mgmt, fetchRpc); + case TDMT_VND_TASKS_STATUS: + qWorkerProcessStatusMsg(mockPointer, mgmt, fetchRpc); + case TDMT_VND_CANCEL_TASK: + qWorkerProcessCancelMsg(mockPointer, mgmt, fetchRpc); + case TDMT_VND_DROP_TASK: + qWorkerProcessDropMsg(mockPointer, mgmt, fetchRpc); + default: + printf("unknown msg type:%d in fetch queue", fetchRpc->msgType); + assert(0); + } + } } @@ -753,13 +1030,16 @@ TEST(rcTest, multithread) { code = qWorkerInit(NODE_TYPE_VNODE, 1, NULL, &mgmt, mockPointer, qwtPutReqToQueue); ASSERT_EQ(code, 0); + tsem_init(&qwtTestQuerySem, 0, 0); + tsem_init(&qwtTestFetchSem, 0, 0); + pthread_attr_t thattr; pthread_attr_init(&thattr); pthread_t t1,t2,t3,t4,t5; - pthread_create(&(t1), &thattr, controlThread, mgmt); - pthread_create(&(t2), &thattr, queryQueueThread, NULL); - pthread_create(&(t3), &thattr, fetchQueueThread, NULL); + pthread_create(&(t1), &thattr, clientThread, mgmt); + pthread_create(&(t2), &thattr, queryQueueThread, mgmt); + pthread_create(&(t3), &thattr, fetchQueueThread, mgmt); while (true) { if (qwtTestDeadLoop) { @@ -779,6 +1059,7 @@ TEST(rcTest, multithread) { int main(int argc, char** argv) { + srand(time(NULL)); testing::InitGoogleTest(&argc, argv); return RUN_ALL_TESTS(); } diff --git a/source/libs/scheduler/src/scheduler.c b/source/libs/scheduler/src/scheduler.c index d8904cdfa9..366cc62a7e 100644 --- a/source/libs/scheduler/src/scheduler.c +++ b/source/libs/scheduler/src/scheduler.c @@ -276,9 +276,12 @@ int32_t schBuildTaskRalation(SSchJob *pJob, SHashObj *planToTask) { int32_t schRecordTaskSucceedNode(SSchTask *pTask) { - SQueryNodeAddr *addr = taosArrayGet(pTask->candidateAddrs, atomic_load_8(&pTask->candidateIdx)); - - assert(NULL != addr); + int32_t idx = atomic_load_8(&pTask->candidateIdx); + SQueryNodeAddr *addr = taosArrayGet(pTask->candidateAddrs, idx); + if (NULL == addr) { + SCH_TASK_ELOG("taosArrayGet candidate addr failed, idx:%d, size:%d", idx, taosArrayGetSize(pTask->candidateAddrs)); + SCH_ERR_RET(TSDB_CODE_SCH_INTERNAL_ERROR); + } pTask->succeedAddr = *addr; @@ -578,9 +581,10 @@ int32_t schProcessOnJobFailureImpl(SSchJob *pJob, int32_t status, int32_t errCod tsem_post(&pJob->rspSem); } - SCH_ERR_RET(atomic_load_32(&pJob->errCode)); + int32_t code = atomic_load_32(&pJob->errCode); + SCH_ERR_RET(code); - assert(0); + SCH_JOB_ELOG("job errCode is invalid, errCode:%d", code); } @@ -725,7 +729,7 @@ int32_t schProcessOnTaskSuccess(SSchJob *pJob, SSchTask *pTask) { SCH_SET_TASK_STATUS(pTask, JOB_TASK_STATUS_PARTIAL_SUCCEED); - SCH_ERR_JRET(schRecordTaskSucceedNode(pTask)); + SCH_ERR_JRET(schRecordTaskSucceedNode(pJob, pTask)); int32_t parentNum = pTask->parents ? (int32_t)taosArrayGetSize(pTask->parents) : 0; if (parentNum == 0) { @@ -738,11 +742,11 @@ int32_t schProcessOnTaskSuccess(SSchJob *pJob, SSchTask *pTask) { SCH_UNLOCK(SCH_WRITE, &pTask->level->lock); if (taskDone < pTask->level->taskNum) { - SCH_TASK_ELOG("wait all tasks, done:%d, all:%d", taskDone, pTask->level->taskNum); + SCH_TASK_DLOG("wait all tasks, done:%d, all:%d", taskDone, pTask->level->taskNum); return TSDB_CODE_SUCCESS; } else if (taskDone > pTask->level->taskNum) { - assert(0); + SCH_TASK_ELOG("taskDone number invalid, done:%d, total:%d", taskDone, pTask->level->taskNum); } if (pTask->level->taskFailed > 0) { @@ -875,19 +879,22 @@ int32_t schHandleResponseMsg(SSchJob *pJob, SSchTask *pTask, int32_t msgType, ch } atomic_store_ptr(&pJob->res, rsp); - atomic_store_32(&pJob->resNumOfRows, rsp->numOfRows); + atomic_add_fetch_32(&pJob->resNumOfRows, htonl(rsp->numOfRows)); if (rsp->completed) { SCH_SET_TASK_STATUS(pTask, JOB_TASK_STATUS_SUCCEED); } + SCH_TASK_DLOG("got fetch rsp, rows:%d, complete:%d", htonl(rsp->numOfRows), rsp->completed); + SCH_ERR_JRET(schProcessOnDataFetched(pJob)); break; } case TDMT_VND_DROP_TASK: { // SHOULD NEVER REACH HERE - assert(0); + SCH_TASK_ELOG("invalid status to handle drop task rsp, ref:%d", atomic_load_32(&pJob->ref)); + SCH_ERR_JRET(TSDB_CODE_SCH_INTERNAL_ERROR); break; } default: @@ -936,7 +943,7 @@ int32_t schHandleCallback(void* param, const SDataBuf* pMsg, int32_t msgType, in pTask = *task; - SCH_TASK_DLOG("rsp msg received, type:%d, code:%x", msgType, rspCode); + SCH_TASK_DLOG("rsp msg received, type:%d, %s, code:%x", msgType, TMSG_INFO(msgType), rspCode); SCH_ERR_JRET(schHandleResponseMsg(pJob, pTask, msgType, pMsg->pData, pMsg->len, rspCode)); @@ -1037,6 +1044,8 @@ int32_t schAsyncSendMsg(void *transport, SEpSet* epSet, uint64_t qId, uint64_t t qError("QID:%"PRIx64 ",TID:%"PRIx64 " asyncSendMsgToServer failed, code:%x", qId, tId, code); SCH_ERR_JRET(code); } + + qDebug("QID:%"PRIx64 ",TID:%"PRIx64 " req msg sent, type:%d, %s", qId, tId, msgType, TMSG_INFO(msgType)); return TSDB_CODE_SUCCESS; @@ -1296,6 +1305,8 @@ void schDropJobAllTasks(SSchJob *pJob) { } int32_t schExecJobImpl(void *transport, SArray *nodeList, SQueryDag* pDag, struct SSchJob** job, bool syncSchedule) { + qDebug("QID:%"PRIx64" job started", pDag->queryId); + if (nodeList && taosArrayGetSize(nodeList) <= 0) { qInfo("QID:%"PRIx64" input nodeList is empty", pDag->queryId); } @@ -1363,7 +1374,7 @@ _return: *(SSchJob **)job = NULL; - scheduleFreeJob(pJob); + schedulerFreeJob(pJob); SCH_RET(code); } @@ -1408,7 +1419,7 @@ int32_t schedulerInit(SSchedulerCfg *cfg) { return TSDB_CODE_SUCCESS; } -int32_t scheduleExecJob(void *transport, SArray *nodeList, SQueryDag* pDag, struct SSchJob** pJob, SQueryResult *pRes) { +int32_t schedulerExecJob(void *transport, SArray *nodeList, SQueryDag* pDag, struct SSchJob** pJob, SQueryResult *pRes) { if (NULL == transport || NULL == pDag || NULL == pDag->pSubplans || NULL == pJob || NULL == pRes) { SCH_ERR_RET(TSDB_CODE_QRY_INVALID_INPUT); } @@ -1425,7 +1436,7 @@ int32_t scheduleExecJob(void *transport, SArray *nodeList, SQueryDag* pDag, stru return TSDB_CODE_SUCCESS; } -int32_t scheduleAsyncExecJob(void *transport, SArray *nodeList, SQueryDag* pDag, struct SSchJob** pJob) { +int32_t schedulerAsyncExecJob(void *transport, SArray *nodeList, SQueryDag* pDag, struct SSchJob** pJob) { if (NULL == transport || NULL == pDag || NULL == pDag->pSubplans || NULL == pJob) { SCH_ERR_RET(TSDB_CODE_QRY_INVALID_INPUT); } @@ -1558,7 +1569,7 @@ _return: } -int32_t scheduleFetchRows(SSchJob *pJob, void** pData) { +int32_t schedulerFetchRows(SSchJob *pJob, void** pData) { if (NULL == pJob || NULL == pData) { SCH_ERR_RET(TSDB_CODE_QRY_INVALID_INPUT); } @@ -1624,11 +1635,12 @@ _return: } *pData = rsp; + SCH_JOB_DLOG("empty res and set query complete, code:%x", code); } atomic_val_compare_exchange_8(&pJob->userFetch, 1, 0); - SCH_JOB_DLOG("fetch done, code:%x", code); + SCH_JOB_DLOG("fetch done, totalRows:%d, code:%x", pJob->resNumOfRows, code); atomic_sub_fetch_32(&pJob->ref, 1); @@ -1647,7 +1659,7 @@ int32_t scheduleCancelJob(void *job) { SCH_RET(code); } -void scheduleFreeJob(void *job) { +void schedulerFreeJob(void *job) { if (NULL == job) { return; } @@ -1676,7 +1688,8 @@ void scheduleFreeJob(void *job) { usleep(1); } else { - assert(0); + SCH_JOB_ELOG("invalid job ref number, ref:%d", ref); + break; } } diff --git a/source/libs/scheduler/test/schedulerTests.cpp b/source/libs/scheduler/test/schedulerTests.cpp index 1425ac0e6c..9348339efb 100644 --- a/source/libs/scheduler/test/schedulerTests.cpp +++ b/source/libs/scheduler/test/schedulerTests.cpp @@ -328,7 +328,7 @@ void schtFreeQueryJob(int32_t freeThread) { SSchJob *job = atomic_load_ptr(&pQueryJob); if (job && atomic_val_compare_exchange_ptr(&pQueryJob, job, NULL)) { - scheduleFreeJob(job); + schedulerFreeJob(job); if (freeThread) { if (++freeNum % schtTestPrintNum == 0) { printf("FreeNum:%d\n", freeNum); @@ -372,7 +372,7 @@ void* schtRunJobThread(void *aa) { qnodeAddr.port = 6031; taosArrayPush(qnodeList, &qnodeAddr); - code = scheduleAsyncExecJob(mockPointer, qnodeList, &dag, &job); + code = schedulerAsyncExecJob(mockPointer, qnodeList, &dag, &job); assert(code == 0); execTasks = taosHashInit(5, taosGetDefaultHashFunction(TSDB_DATA_TYPE_UBIGINT), false, HASH_ENTRY_LOCK); @@ -466,7 +466,7 @@ void* schtRunJobThread(void *aa) { atomic_store_32(&schtStartFetch, 1); void *data = NULL; - code = scheduleFetchRows(pQueryJob, &data); + code = schedulerFetchRows(pQueryJob, &data); assert(code == 0 || code); if (0 == code) { @@ -476,7 +476,7 @@ void* schtRunJobThread(void *aa) { } data = NULL; - code = scheduleFetchRows(pQueryJob, &data); + code = schedulerFetchRows(pQueryJob, &data); assert(code == 0 || code); schtFreeQueryJob(0); @@ -533,7 +533,7 @@ TEST(queryTest, normalCase) { schtSetExecNode(); schtSetAsyncSendMsgToServer(); - code = scheduleAsyncExecJob(mockPointer, qnodeList, &dag, &pJob); + code = schedulerAsyncExecJob(mockPointer, qnodeList, &dag, &pJob); ASSERT_EQ(code, 0); SSchJob *job = (SSchJob *)pJob; @@ -588,7 +588,7 @@ TEST(queryTest, normalCase) { pthread_create(&(thread1), &thattr, schtCreateFetchRspThread, job); void *data = NULL; - code = scheduleFetchRows(job, &data); + code = schedulerFetchRows(job, &data); ASSERT_EQ(code, 0); SRetrieveTableRsp *pRsp = (SRetrieveTableRsp *)data; @@ -597,11 +597,11 @@ TEST(queryTest, normalCase) { tfree(data); data = NULL; - code = scheduleFetchRows(job, &data); + code = schedulerFetchRows(job, &data); ASSERT_EQ(code, 0); ASSERT_TRUE(data); - scheduleFreeJob(pJob); + schedulerFreeJob(pJob); schtFreeQueryDag(&dag); @@ -643,11 +643,11 @@ TEST(insertTest, normalCase) { pthread_create(&(thread1), &thattr, schtSendRsp, &pInsertJob); SQueryResult res = {0}; - code = scheduleExecJob(mockPointer, qnodeList, &dag, &pInsertJob, &res); + code = schedulerExecJob(mockPointer, qnodeList, &dag, &pInsertJob, &res); ASSERT_EQ(code, 0); ASSERT_EQ(res.numOfRows, 20); - scheduleFreeJob(pInsertJob); + schedulerFreeJob(pInsertJob); schedulerDestroy(); } diff --git a/source/util/src/terror.c b/source/util/src/terror.c index a67d66efb0..3a67b6515b 100644 --- a/source/util/src/terror.c +++ b/source/util/src/terror.c @@ -353,7 +353,7 @@ TAOS_DEFINE_ERROR(TSDB_CODE_QRY_INVALID_INPUT, "invalid input") TAOS_DEFINE_ERROR(TSDB_CODE_QRY_SCH_NOT_EXIST, "Scheduler not exist") TAOS_DEFINE_ERROR(TSDB_CODE_QRY_TASK_NOT_EXIST, "Task not exist") TAOS_DEFINE_ERROR(TSDB_CODE_QRY_TASK_ALREADY_EXIST, "Task already exist") -TAOS_DEFINE_ERROR(TSDB_CODE_QRY_RES_CACHE_NOT_EXIST, "Task result cache not exist") +TAOS_DEFINE_ERROR(TSDB_CODE_QRY_TASK_CTX_NOT_EXIST, "Task context not exist") TAOS_DEFINE_ERROR(TSDB_CODE_QRY_TASK_CANCELLED, "Task cancelled") TAOS_DEFINE_ERROR(TSDB_CODE_QRY_TASK_DROPPED, "Task dropped") TAOS_DEFINE_ERROR(TSDB_CODE_QRY_TASK_CANCELLING, "Task cancelling") From 79ce13e88db2c7da4b841a3dab6380afe0e0b995 Mon Sep 17 00:00:00 2001 From: dapan Date: Sat, 22 Jan 2022 10:36:03 +0800 Subject: [PATCH 03/11] feature/qnode --- source/libs/qworker/src/qworker.c | 18 +++++++++++------ source/libs/qworker/test/qworkerTests.cpp | 24 ++++++++++++----------- source/libs/scheduler/src/scheduler.c | 2 +- 3 files changed, 26 insertions(+), 18 deletions(-) diff --git a/source/libs/qworker/src/qworker.c b/source/libs/qworker/src/qworker.c index bcfbafd4c9..021bd642bd 100644 --- a/source/libs/qworker/src/qworker.c +++ b/source/libs/qworker/src/qworker.c @@ -629,7 +629,9 @@ int32_t qwHandlePrePhaseEvents(QW_FPARAMS_DEF, int8_t phase, SQWPhaseInput *inpu dropConnection = ctx->dropConnection; // Note: ctx freed, no need to unlock it - locked = false; + locked = false; + + break; } else if (QW_IS_EVENT_RECEIVED(ctx, QW_EVENT_CANCEL)) { QW_ERR_JRET(qwAddTaskStatus(QW_FPARAMS(), JOB_TASK_STATUS_CANCELLED)); @@ -639,6 +641,8 @@ int32_t qwHandlePrePhaseEvents(QW_FPARAMS_DEF, int8_t phase, SQWPhaseInput *inpu QW_SET_RSP_CODE(ctx, output->rspCode); cancelConnection = ctx->cancelConnection; + + break; } if (ctx->rspCode) { @@ -1215,8 +1219,6 @@ int32_t qwProcessDrop(QW_FPARAMS_DEF, SQWMsg *qwMsg) { QW_ERR_JRET(qwKillTaskHandle(QW_FPARAMS(), ctx)); QW_ERR_JRET(qwUpdateTaskStatus(QW_FPARAMS(), JOB_TASK_STATUS_DROPPING)); - - ctx->dropConnection = qwMsg->connection; } else if (ctx->phase > 0) { QW_ERR_JRET(qwDropTaskStatus(QW_FPARAMS())); QW_ERR_JRET(qwDropTaskCtx(QW_FPARAMS(), QW_WRITE)); @@ -1225,11 +1227,11 @@ int32_t qwProcessDrop(QW_FPARAMS_DEF, SQWMsg *qwMsg) { locked = false; needRsp = true; - - QW_ERR_JRET(TSDB_CODE_QRY_TASK_DROPPED); } - if (!needRsp) { + if (!needRsp) { + ctx->dropConnection = qwMsg->connection; + QW_SET_EVENT_RECEIVED(ctx, QW_EVENT_DROP); } @@ -1239,6 +1241,10 @@ _return: QW_UPDATE_RSP_CODE(ctx, code); } + if (locked) { + QW_UNLOCK(QW_WRITE, &ctx->lock); + } + if (ctx) { qwReleaseTaskCtx(mgmt, ctx); } diff --git a/source/libs/qworker/test/qworkerTests.cpp b/source/libs/qworker/test/qworkerTests.cpp index a94af4f69b..5720c2d47c 100644 --- a/source/libs/qworker/test/qworkerTests.cpp +++ b/source/libs/qworker/test/qworkerTests.cpp @@ -137,7 +137,7 @@ void qwtBuildStatusReqMsg(SSchTasksStatusReq *statusMsg, SRpcMsg *statusRpc) { } int32_t qwtStringToPlan(const char* str, SSubplan** subplan) { - *subplan = 0x1; + *subplan = (SSubplan *)0x1; return 0; } @@ -192,10 +192,10 @@ void qwtRpcSendResponse(const SRpcMsg *pRsp) { if (0 == pRsp->code) { qwtBuildReadyReqMsg(&qwtreadyMsg, &qwtreadyRpc); - qwtPutReqToFetchQueue(0x1, &qwtreadyRpc); + qwtPutReqToFetchQueue((void *)0x1, &qwtreadyRpc); } else { qwtBuildDropReqMsg(&qwtdropMsg, &qwtdropRpc); - qwtPutReqToFetchQueue(0x1, &qwtdropRpc); + qwtPutReqToFetchQueue((void *)0x1, &qwtdropRpc); } break; @@ -205,10 +205,10 @@ void qwtRpcSendResponse(const SRpcMsg *pRsp) { if (0 == pRsp->code) { qwtBuildFetchReqMsg(&qwtfetchMsg, &qwtfetchRpc); - qwtPutReqToFetchQueue(0x1, &qwtfetchRpc); + qwtPutReqToFetchQueue((void *)0x1, &qwtfetchRpc); } else { qwtBuildDropReqMsg(&qwtdropMsg, &qwtdropRpc); - qwtPutReqToFetchQueue(0x1, &qwtdropRpc); + qwtPutReqToFetchQueue((void *)0x1, &qwtdropRpc); } break; } @@ -217,12 +217,12 @@ void qwtRpcSendResponse(const SRpcMsg *pRsp) { if (0 == pRsp->code && 0 == rsp->completed) { qwtBuildFetchReqMsg(&qwtfetchMsg, &qwtfetchRpc); - qwtPutReqToFetchQueue(0x1, &qwtfetchRpc); + qwtPutReqToFetchQueue((void *)0x1, &qwtfetchRpc); return; } qwtBuildDropReqMsg(&qwtdropMsg, &qwtdropRpc); - qwtPutReqToFetchQueue(0x1, &qwtdropRpc); + qwtPutReqToFetchQueue((void *)0x1, &qwtdropRpc); break; } @@ -352,7 +352,7 @@ int32_t qwtGetDataBlock(DataSinkHandle handle, SOutputData* pOutput) { qwtTestSinkBlockNum--; pOutput->numOfRows = rand() % 10 + 1; pOutput->compressed = 1; - pOutput->pData = malloc(pOutput->numOfRows); + pOutput->pData = (char *)malloc(pOutput->numOfRows); pOutput->queryEnd = qwtTestSinkQueryEnd; if (qwtTestSinkBlockNum == 0) { pOutput->bufStatus = DS_BUF_EMPTY; @@ -648,7 +648,7 @@ void *clientThread(void *param) { qwtTestCaseFinished = false; qwtBuildQueryReqMsg(&queryRpc); - qwtPutReqToQueue(0x1, &queryRpc); + qwtPutReqToQueue((void *)0x1, &queryRpc); while (!qwtTestCaseFinished) { usleep(1); @@ -692,15 +692,16 @@ void *queryQueueThread(void *param) { taosWUnLockLatch(&qwtTestQueryQueueLock); if (TDMT_VND_QUERY == queryRpc->msgType) { - qWorkerProcessQueryMsg(mockPointer, mgmt, &queryRpc); + qWorkerProcessQueryMsg(mockPointer, mgmt, queryRpc); } else if (TDMT_VND_QUERY_CONTINUE == queryRpc->msgType) { - qWorkerProcessCQueryMsg(mockPointer, mgmt, &queryRpc) + qWorkerProcessCQueryMsg(mockPointer, mgmt, queryRpc); } else { printf("unknown msg in query queue, type:%d\n", queryRpc->msgType); assert(0); } } + return NULL; } void *fetchQueueThread(void *param) { @@ -743,6 +744,7 @@ void *fetchQueueThread(void *param) { } } + return NULL; } diff --git a/source/libs/scheduler/src/scheduler.c b/source/libs/scheduler/src/scheduler.c index 366cc62a7e..0ad51d0b57 100644 --- a/source/libs/scheduler/src/scheduler.c +++ b/source/libs/scheduler/src/scheduler.c @@ -275,7 +275,7 @@ int32_t schBuildTaskRalation(SSchJob *pJob, SHashObj *planToTask) { } -int32_t schRecordTaskSucceedNode(SSchTask *pTask) { +int32_t schRecordTaskSucceedNode(SSchJob *pJob, SSchTask *pTask) { int32_t idx = atomic_load_8(&pTask->candidateIdx); SQueryNodeAddr *addr = taosArrayGet(pTask->candidateAddrs, idx); if (NULL == addr) { From e4338377c4d6881ba59a53736c3689170e387c6d Mon Sep 17 00:00:00 2001 From: dapan1121 <89396746@qq.com> Date: Sat, 22 Jan 2022 11:54:57 +0800 Subject: [PATCH 04/11] feature/qnode --- source/libs/qworker/test/qworkerTests.cpp | 130 ++++++++-------------- 1 file changed, 49 insertions(+), 81 deletions(-) diff --git a/source/libs/qworker/test/qworkerTests.cpp b/source/libs/qworker/test/qworkerTests.cpp index 5720c2d47c..362156ebcd 100644 --- a/source/libs/qworker/test/qworkerTests.cpp +++ b/source/libs/qworker/test/qworkerTests.cpp @@ -40,8 +40,9 @@ namespace { #define qwtTestQueryQueueSize 1000 #define qwtTestFetchQueueSize 1000 -#define qwtTestMaxExecTaskUsec 1000000 +#define qwtTestMaxExecTaskUsec 2 +uint64_t qwtTestQueryId = 0; bool qwtTestEnableSleep = true; bool qwtTestStop = false; bool qwtTestDeadLoop = true; @@ -70,14 +71,17 @@ int32_t qwtTestSinkBlockNum = 0; int32_t qwtTestSinkMaxBlockNum = 0; bool qwtTestSinkQueryEnd = false; SRWLatch qwtTestSinkLock = 0; +int32_t qwtTestSinkLastLen = 0; +SSubQueryMsg qwtqueryMsg = {0}; SRpcMsg qwtfetchRpc = {0}; SResFetchReq qwtfetchMsg = {0}; SRpcMsg qwtreadyRpc = {0}; SResReadyReq qwtreadyMsg = {0}; SRpcMsg qwtdropRpc = {0}; STaskDropReq qwtdropMsg = {0}; +SSchTasksStatusReq qwtstatusMsg = {0}; void qwtInitLogFile() { @@ -96,18 +100,17 @@ void qwtInitLogFile() { } void qwtBuildQueryReqMsg(SRpcMsg *queryRpc) { - SSubQueryMsg *queryMsg = (SSubQueryMsg *)calloc(1, sizeof(SSubQueryMsg) + 100); - queryMsg->queryId = htobe64(1); - queryMsg->sId = htobe64(1); - queryMsg->taskId = htobe64(1); - queryMsg->contentLen = htonl(100); - queryRpc->pCont = queryMsg; + qwtqueryMsg.queryId = htobe64(atomic_add_fetch_64(&qwtTestQueryId, 1)); + qwtqueryMsg.sId = htobe64(1); + qwtqueryMsg.taskId = htobe64(1); + qwtqueryMsg.contentLen = htonl(100); + queryRpc->pCont = &qwtqueryMsg; queryRpc->contLen = sizeof(SSubQueryMsg) + 100; } void qwtBuildReadyReqMsg(SResReadyReq *readyMsg, SRpcMsg *readyRpc) { readyMsg->sId = htobe64(1); - readyMsg->queryId = htobe64(1); + readyMsg->queryId = htobe64(atomic_load_64(&qwtTestQueryId)); readyMsg->taskId = htobe64(1); readyRpc->pCont = readyMsg; readyRpc->contLen = sizeof(SResReadyReq); @@ -115,7 +118,7 @@ void qwtBuildReadyReqMsg(SResReadyReq *readyMsg, SRpcMsg *readyRpc) { void qwtBuildFetchReqMsg(SResFetchReq *fetchMsg, SRpcMsg *fetchRpc) { fetchMsg->sId = htobe64(1); - fetchMsg->queryId = htobe64(1); + fetchMsg->queryId = htobe64(atomic_load_64(&qwtTestQueryId)); fetchMsg->taskId = htobe64(1); fetchRpc->pCont = fetchMsg; fetchRpc->contLen = sizeof(SResFetchReq); @@ -123,7 +126,7 @@ void qwtBuildFetchReqMsg(SResFetchReq *fetchMsg, SRpcMsg *fetchRpc) { void qwtBuildDropReqMsg(STaskDropReq *dropMsg, SRpcMsg *dropRpc) { dropMsg->sId = htobe64(1); - dropMsg->queryId = htobe64(1); + dropMsg->queryId = htobe64(atomic_load_64(&qwtTestQueryId)); dropMsg->taskId = htobe64(1); dropRpc->pCont = dropMsg; dropRpc->contLen = sizeof(STaskDropReq); @@ -273,7 +276,8 @@ int32_t qwtExecTask(qTaskInfo_t tinfo, SSDataBlock** pRes, uint64_t *useconds) { if (endExec) { usleep(rand() % qwtTestMaxExecTaskUsec); - *pRes = (SSDataBlock*)0x1; + *pRes = (SSDataBlock*)calloc(1, sizeof(SSDataBlock)); + (*pRes)->info.rows = rand() % 1000; } else { *pRes = NULL; usleep(rand() % qwtTestMaxExecTaskUsec); @@ -297,6 +301,8 @@ int32_t qwtPutDataBlock(DataSinkHandle handle, const SInputData* pInput, bool* p assert(0); } + free((void *)pInput->pData); + taosWLockLatch(&qwtTestSinkLock); qwtTestSinkBlockNum++; @@ -339,6 +345,7 @@ void qwtGetDataLength(DataSinkHandle handle, int32_t* pLen, bool* pQueryEnd) { } else { *pLen = 0; } + qwtTestSinkLastLen = *pLen; taosWUnLockLatch(&qwtTestSinkLock); *pQueryEnd = qwtTestSinkQueryEnd; @@ -348,11 +355,23 @@ void qwtGetDataLength(DataSinkHandle handle, int32_t* pLen, bool* pQueryEnd) { int32_t qwtGetDataBlock(DataSinkHandle handle, SOutputData* pOutput) { taosWLockLatch(&qwtTestSinkLock); - if (qwtTestSinkBlockNum > 0) { - qwtTestSinkBlockNum--; + if (qwtTestSinkLastLen > 0) { pOutput->numOfRows = rand() % 10 + 1; pOutput->compressed = 1; - pOutput->pData = (char *)malloc(pOutput->numOfRows); + pOutput->queryEnd = qwtTestSinkQueryEnd; + if (qwtTestSinkBlockNum == 0) { + pOutput->bufStatus = DS_BUF_EMPTY; + } else if (qwtTestSinkBlockNum <= qwtTestSinkMaxBlockNum*0.5) { + pOutput->bufStatus = DS_BUF_LOW; + } else { + pOutput->bufStatus = DS_BUF_FULL; + } + pOutput->useconds = rand() % 10 + 1; + pOutput->precision = 1; + } else if (qwtTestSinkLastLen == 0) { + pOutput->numOfRows = 0; + pOutput->compressed = 1; + pOutput->pData = NULL; pOutput->queryEnd = qwtTestSinkQueryEnd; if (qwtTestSinkBlockNum == 0) { pOutput->bufStatus = DS_BUF_EMPTY; @@ -534,7 +553,6 @@ void *queryThread(void *param) { while (!qwtTestStop) { qwtBuildQueryReqMsg(&queryRpc); qWorkerProcessQueryMsg(mockPointer, mgmt, &queryRpc); - free(queryRpc.pCont); if (qwtTestEnableSleep) { usleep(rand()%5); } @@ -654,8 +672,6 @@ void *clientThread(void *param) { usleep(1); } - free(queryRpc.pCont); - if (qwtTestEnableSleep) { usleep(rand()%5); } @@ -763,41 +779,11 @@ TEST(seqTest, normalCase) { SRpcMsg statusRpc = {0}; qwtInitLogFile(); - - SSubQueryMsg *queryMsg = (SSubQueryMsg *)calloc(1, sizeof(SSubQueryMsg) + 100); - queryMsg->queryId = htobe64(1); - queryMsg->sId = htobe64(1); - queryMsg->taskId = htobe64(1); - queryMsg->contentLen = htonl(100); - queryRpc.pCont = queryMsg; - queryRpc.contLen = sizeof(SSubQueryMsg) + 100; - SResReadyReq readyMsg = {0}; - readyMsg.sId = htobe64(1); - readyMsg.queryId = htobe64(1); - readyMsg.taskId = htobe64(1); - readyRpc.pCont = &readyMsg; - readyRpc.contLen = sizeof(SResReadyReq); - - SResFetchReq fetchMsg = {0}; - fetchMsg.sId = htobe64(1); - fetchMsg.queryId = htobe64(1); - fetchMsg.taskId = htobe64(1); - fetchRpc.pCont = &fetchMsg; - fetchRpc.contLen = sizeof(SResFetchReq); - - STaskDropReq dropMsg = {0}; - dropMsg.sId = htobe64(1); - dropMsg.queryId = htobe64(1); - dropMsg.taskId = htobe64(1); - dropRpc.pCont = &dropMsg; - dropRpc.contLen = sizeof(STaskDropReq); - - SSchTasksStatusReq statusMsg = {0}; - statusMsg.sId = htobe64(1); - statusRpc.pCont = &statusMsg; - statusRpc.contLen = sizeof(SSchTasksStatusReq); - statusRpc.msgType = TDMT_VND_TASKS_STATUS; + qwtBuildQueryReqMsg(&queryRpc); + qwtBuildReadyReqMsg(&qwtreadyMsg, &readyRpc); + qwtBuildFetchReqMsg(&qwtfetchMsg, &fetchRpc); + qwtBuildDropReqMsg(&qwtdropMsg, &dropRpc); stubSetStringToPlan(); stubSetRpcSendResponse(); @@ -814,35 +800,35 @@ TEST(seqTest, normalCase) { code = qWorkerInit(NODE_TYPE_VNODE, 1, NULL, &mgmt, mockPointer, qwtPutReqToQueue); ASSERT_EQ(code, 0); - statusMsg.sId = htobe64(1); + qwtBuildStatusReqMsg(&qwtstatusMsg, &statusRpc); code = qWorkerProcessStatusMsg(mockPointer, mgmt, &statusRpc); ASSERT_EQ(code, 0); code = qWorkerProcessQueryMsg(mockPointer, mgmt, &queryRpc); ASSERT_EQ(code, 0); - statusMsg.sId = htobe64(1); + qwtBuildStatusReqMsg(&qwtstatusMsg, &statusRpc); code = qWorkerProcessStatusMsg(mockPointer, mgmt, &statusRpc); ASSERT_EQ(code, 0); code = qWorkerProcessReadyMsg(mockPointer, mgmt, &readyRpc); ASSERT_EQ(code, 0); - statusMsg.sId = htobe64(1); + qwtBuildStatusReqMsg(&qwtstatusMsg, &statusRpc); code = qWorkerProcessStatusMsg(mockPointer, mgmt, &statusRpc); ASSERT_EQ(code, 0); code = qWorkerProcessFetchMsg(mockPointer, mgmt, &fetchRpc); ASSERT_EQ(code, 0); - statusMsg.sId = htobe64(1); + qwtBuildStatusReqMsg(&qwtstatusMsg, &statusRpc); code = qWorkerProcessStatusMsg(mockPointer, mgmt, &statusRpc); ASSERT_EQ(code, 0); code = qWorkerProcessDropMsg(mockPointer, mgmt, &dropRpc); ASSERT_EQ(code, 0); - statusMsg.sId = htobe64(1); + qwtBuildStatusReqMsg(&qwtstatusMsg, &statusRpc); code = qWorkerProcessStatusMsg(mockPointer, mgmt, &statusRpc); ASSERT_EQ(code, 0); @@ -859,26 +845,9 @@ TEST(seqTest, cancelFirst) { qwtInitLogFile(); - SSubQueryMsg *queryMsg = (SSubQueryMsg *)calloc(1, sizeof(SSubQueryMsg) + 100); - queryMsg->queryId = htobe64(1); - queryMsg->sId = htobe64(1); - queryMsg->taskId = htobe64(1); - queryMsg->contentLen = htonl(100); - queryRpc.pCont = queryMsg; - queryRpc.contLen = sizeof(SSubQueryMsg) + 100; - - STaskDropReq dropMsg = {0}; - dropMsg.sId = htobe64(1); - dropMsg.queryId = htobe64(1); - dropMsg.taskId = htobe64(1); - dropRpc.pCont = &dropMsg; - dropRpc.contLen = sizeof(STaskDropReq); - - SSchTasksStatusReq statusMsg = {0}; - statusMsg.sId = htobe64(1); - statusRpc.pCont = &statusMsg; - statusRpc.contLen = sizeof(SSchTasksStatusReq); - statusRpc.msgType = TDMT_VND_TASKS_STATUS; + qwtBuildQueryReqMsg(&queryRpc); + qwtBuildDropReqMsg(&qwtdropMsg, &dropRpc); + qwtBuildStatusReqMsg(&qwtstatusMsg, &statusRpc); stubSetStringToPlan(); stubSetRpcSendResponse(); @@ -886,21 +855,21 @@ TEST(seqTest, cancelFirst) { code = qWorkerInit(NODE_TYPE_VNODE, 1, NULL, &mgmt, mockPointer, qwtPutReqToQueue); ASSERT_EQ(code, 0); - statusMsg.sId = htobe64(1); + qwtBuildStatusReqMsg(&qwtstatusMsg, &statusRpc); code = qWorkerProcessStatusMsg(mockPointer, mgmt, &statusRpc); ASSERT_EQ(code, 0); code = qWorkerProcessDropMsg(mockPointer, mgmt, &dropRpc); ASSERT_EQ(code, 0); - statusMsg.sId = htobe64(1); + qwtBuildStatusReqMsg(&qwtstatusMsg, &statusRpc); code = qWorkerProcessStatusMsg(mockPointer, mgmt, &statusRpc); ASSERT_EQ(code, 0); code = qWorkerProcessQueryMsg(mockPointer, mgmt, &queryRpc); - ASSERT_EQ(code, TSDB_CODE_QRY_TASK_DROPPED); + ASSERT_TRUE(0 != code); - statusMsg.sId = htobe64(1); + qwtBuildStatusReqMsg(&qwtstatusMsg, &statusRpc); code = qWorkerProcessStatusMsg(mockPointer, mgmt, &statusRpc); ASSERT_EQ(code, 0); @@ -941,7 +910,6 @@ TEST(seqTest, randCase) { printf("Query,%d\n", t++); qwtBuildQueryReqMsg(&queryRpc); code = qWorkerProcessQueryMsg(mockPointer, mgmt, &queryRpc); - free(queryRpc.pCont); } else if (r >= maxr/5 && r < maxr * 2/5) { printf("Ready,%d\n", t++); qwtBuildReadyReqMsg(&readyMsg, &readyRpc); From e6f05d3889c592ab3a8d48cb16ce34ecbc053ed6 Mon Sep 17 00:00:00 2001 From: dapan1121 Date: Sat, 22 Jan 2022 22:50:00 +0800 Subject: [PATCH 05/11] feature/qnode --- source/libs/qworker/inc/qworkerInt.h | 1 + source/libs/qworker/src/qworker.c | 82 +++-- source/libs/qworker/src/qworkerMsg.c | 7 +- source/libs/qworker/test/qworkerTests.cpp | 375 +++++++++++++++++++--- source/libs/scheduler/src/scheduler.c | 2 +- 5 files changed, 409 insertions(+), 58 deletions(-) diff --git a/source/libs/qworker/inc/qworkerInt.h b/source/libs/qworker/inc/qworkerInt.h index 9ecce3f5f9..2765d7d5d7 100644 --- a/source/libs/qworker/inc/qworkerInt.h +++ b/source/libs/qworker/inc/qworkerInt.h @@ -111,6 +111,7 @@ typedef struct SQWTaskCtx { void *cancelConnection; bool emptyRes; + bool multiExec; int8_t queryContinue; int8_t queryInQueue; int32_t rspCode; diff --git a/source/libs/qworker/src/qworker.c b/source/libs/qworker/src/qworker.c index 021bd642bd..f1fd8aa6fb 100644 --- a/source/libs/qworker/src/qworker.c +++ b/source/libs/qworker/src/qworker.c @@ -53,6 +53,12 @@ int32_t qwValidateStatus(QW_FPARAMS_DEF, int8_t oriStatus, int8_t newStatus) { break; case JOB_TASK_STATUS_SUCCEED: + if (newStatus != JOB_TASK_STATUS_CANCELLED + && newStatus != JOB_TASK_STATUS_DROPPING) { + QW_ERR_JRET(TSDB_CODE_QRY_APP_ERROR); + } + + break; case JOB_TASK_STATUS_FAILED: case JOB_TASK_STATUS_CANCELLING: if (newStatus != JOB_TASK_STATUS_CANCELLED) { @@ -249,7 +255,7 @@ int32_t qwAcquireTaskCtx(QW_FPARAMS_DEF, SQWTaskCtx **ctx) { *ctx = taosHashAcquire(mgmt->ctxHash, id, sizeof(id)); if (NULL == (*ctx)) { //QW_UNLOCK(rwType, &mgmt->ctxLock); - QW_TASK_ELOG("ctx not in ctxHash, id:%s", id); + QW_TASK_DLOG_E("task ctx not exist, may be dropped"); QW_ERR_RET(TSDB_CODE_QRY_TASK_CTX_NOT_EXIST); } @@ -262,7 +268,7 @@ int32_t qwGetTaskCtx(QW_FPARAMS_DEF, SQWTaskCtx **ctx) { *ctx = taosHashGet(mgmt->ctxHash, id, sizeof(id)); if (NULL == (*ctx)) { - QW_TASK_ELOG("ctx not in ctxHash, ctxHashSize:%d", taosHashGetSize(mgmt->ctxHash)); + QW_TASK_DLOG_E("task ctx not exist, may be dropped"); QW_ERR_RET(TSDB_CODE_QRY_TASK_CTX_NOT_EXIST); } @@ -548,6 +554,8 @@ int32_t qwGetResFromSink(QW_FPARAMS_DEF, SQWTaskCtx *ctx, int32_t *dataLen, void return TSDB_CODE_SUCCESS; } + + pOutput->bufStatus = DS_BUF_EMPTY; QW_TASK_DLOG("no res data in sink, need response later, queryEnd:%d", queryEnd); @@ -605,10 +613,10 @@ int32_t qwHandlePrePhaseEvents(QW_FPARAMS_DEF, int8_t phase, SQWPhaseInput *inpu QW_LOCK(QW_WRITE, &ctx->lock); locked = true; - atomic_store_32(&ctx->phase, phase); - switch (phase) { case QW_PHASE_PRE_QUERY: { + atomic_store_8(&ctx->phase, phase); + atomic_store_8(&ctx->taskType, input->taskType); if (QW_IS_EVENT_PROCESSED(ctx, QW_EVENT_CANCEL) || QW_IS_EVENT_PROCESSED(ctx, QW_EVENT_DROP)) { @@ -706,6 +714,8 @@ int32_t qwHandlePrePhaseEvents(QW_FPARAMS_DEF, int8_t phase, SQWPhaseInput *inpu break; } case QW_PHASE_PRE_CQUERY: { + atomic_store_8(&ctx->phase, phase); + if (QW_IS_EVENT_PROCESSED(ctx, QW_EVENT_CANCEL)) { QW_TASK_WLOG("task already cancelled, phase:%d", phase); output->needStop = true; @@ -721,17 +731,33 @@ int32_t qwHandlePrePhaseEvents(QW_FPARAMS_DEF, int8_t phase, SQWPhaseInput *inpu } if (QW_IS_EVENT_RECEIVED(ctx, QW_EVENT_DROP)) { - QW_TASK_ELOG("drop event at wrong phase, phase:%d", phase); + QW_ERR_JRET(qwDropTaskStatus(QW_FPARAMS())); + QW_ERR_JRET(qwDropTaskCtx(QW_FPARAMS(), QW_WRITE)); + + output->rspCode = TSDB_CODE_QRY_TASK_DROPPED; output->needStop = true; - output->rspCode = TSDB_CODE_QRY_TASK_STATUS_ERROR; - QW_ERR_JRET(TSDB_CODE_QRY_TASK_CANCELLED); + QW_SET_RSP_CODE(ctx, output->rspCode); + dropConnection = ctx->dropConnection; + + // Note: ctx freed, no need to unlock it + locked = false; + + QW_ERR_JRET(output->rspCode); } else if (QW_IS_EVENT_RECEIVED(ctx, QW_EVENT_CANCEL)) { - QW_TASK_ELOG("cancel event at wrong phase, phase:%d", phase); - output->needStop = true; - output->rspCode = TSDB_CODE_QRY_TASK_STATUS_ERROR; - QW_ERR_JRET(TSDB_CODE_QRY_TASK_CANCELLED); + QW_ERR_JRET(qwUpdateTaskStatus(QW_FPARAMS(), JOB_TASK_STATUS_CANCELLED)); + qwFreeTask(QW_FPARAMS(), ctx); + + QW_SET_EVENT_PROCESSED(ctx, QW_EVENT_CANCEL); + + output->needStop = true; + output->rspCode = TSDB_CODE_QRY_TASK_CANCELLED; + QW_SET_RSP_CODE(ctx, output->rspCode); + cancelConnection = ctx->cancelConnection; + + QW_ERR_JRET(output->rspCode); } + if (ctx->rspCode) { QW_TASK_ELOG("task already failed, code:%x, phase:%d", ctx->rspCode, phase); output->needStop = true; @@ -874,7 +900,9 @@ _return: QW_UPDATE_RSP_CODE(ctx, output->rspCode); } - atomic_store_32(&ctx->phase, phase); + if (QW_PHASE_POST_FETCH != phase) { + atomic_store_8(&ctx->phase, phase); + } if (locked) { QW_UNLOCK(QW_WRITE, &ctx->lock); @@ -1063,6 +1091,7 @@ int32_t qwProcessCQuery(QW_FPARAMS_DEF, SQWMsg *qwMsg) { QW_ERR_JRET(qwGetTaskCtx(QW_FPARAMS(), &ctx)); atomic_store_8(&ctx->queryInQueue, 0); + atomic_store_8(&ctx->queryContinue, 0); DataSinkHandle sinkHandle = ctx->sinkHandle; @@ -1078,6 +1107,10 @@ int32_t qwProcessCQuery(QW_FPARAMS_DEF, SQWMsg *qwMsg) { // RC WARNING atomic_store_8(&ctx->queryContinue, 1); } + + if (sOutput.queryEnd) { + needStop = true; + } if (rsp) { qwBuildFetchRsp(rsp, &sOutput, dataLen); @@ -1093,6 +1126,10 @@ int32_t qwProcessCQuery(QW_FPARAMS_DEF, SQWMsg *qwMsg) { _return: + if (NULL == ctx) { + break; + } + if (code && QW_IS_EVENT_RECEIVED(ctx, QW_EVENT_FETCH)) { QW_SET_EVENT_PROCESSED(ctx, QW_EVENT_FETCH); qwFreeFetchRsp(rsp); @@ -1101,12 +1138,18 @@ int32_t qwProcessCQuery(QW_FPARAMS_DEF, SQWMsg *qwMsg) { QW_TASK_DLOG("fetch msg rsped, code:%x, dataLen:%d", code, 0); } - input.code = code; - qwHandlePostPhaseEvents(QW_FPARAMS(), QW_PHASE_POST_CQUERY, &input, &output); + QW_LOCK(QW_WRITE, &ctx->lock); + if (needStop || code || 0 == atomic_load_8(&ctx->queryContinue)) { + atomic_store_8(&ctx->phase, 0); + QW_UNLOCK(QW_WRITE,&ctx->lock); + break; + } + + QW_UNLOCK(QW_WRITE,&ctx->lock); + } while (true); - needStop = output.needStop; - code = output.rspCode; - } while ((!needStop) && (0 == code) && atomic_val_compare_exchange_8(&ctx->queryContinue, 1, 0)); + input.code = code; + qwHandlePostPhaseEvents(QW_FPARAMS(), QW_PHASE_POST_CQUERY, &input, &output); QW_RET(code); } @@ -1159,7 +1202,10 @@ int32_t qwProcessFetch(QW_FPARAMS_DEF, SQWMsg *qwMsg) { if (QW_IS_QUERY_RUNNING(ctx)) { atomic_store_8(&ctx->queryContinue, 1); } else if (0 == atomic_load_8(&ctx->queryInQueue)) { - QW_ERR_JRET(qwUpdateTaskStatus(QW_FPARAMS(), JOB_TASK_STATUS_EXECUTING)); + if (!ctx->multiExec) { + QW_ERR_JRET(qwUpdateTaskStatus(QW_FPARAMS(), JOB_TASK_STATUS_EXECUTING)); + ctx->multiExec = true; + } atomic_store_8(&ctx->queryInQueue, 1); diff --git a/source/libs/qworker/src/qworkerMsg.c b/source/libs/qworker/src/qworkerMsg.c index baa4ad2a04..d11bee6dce 100644 --- a/source/libs/qworker/src/qworkerMsg.c +++ b/source/libs/qworker/src/qworkerMsg.c @@ -50,6 +50,7 @@ int32_t qwBuildAndSendQueryRsp(void *connection, int32_t code) { pRsp->code = code; SRpcMsg rpcRsp = { + .msgType = TDMT_VND_QUERY_RSP, .handle = pMsg->handle, .ahandle = pMsg->ahandle, .pCont = pRsp, @@ -68,6 +69,7 @@ int32_t qwBuildAndSendReadyRsp(void *connection, int32_t code) { pRsp->code = code; SRpcMsg rpcRsp = { + .msgType = TDMT_VND_RES_READY_RSP, .handle = pMsg->handle, .ahandle = pMsg->ahandle, .pCont = pRsp, @@ -98,7 +100,7 @@ int32_t qwBuildAndSendStatusRsp(SRpcMsg *pMsg, SSchedulerStatusRsp *sStatus) { } SRpcMsg rpcRsp = { - .msgType = pMsg->msgType + 1, + .msgType = TDMT_VND_TASKS_STATUS_RSP, .handle = pMsg->handle, .ahandle = pMsg->ahandle, .pCont = pRsp, @@ -121,6 +123,7 @@ int32_t qwBuildAndSendFetchRsp(void *connection, SRetrieveTableRsp *pRsp, int32_ } SRpcMsg rpcRsp = { + .msgType = TDMT_VND_FETCH_RSP, .handle = pMsg->handle, .ahandle = pMsg->ahandle, .pCont = pRsp, @@ -138,6 +141,7 @@ int32_t qwBuildAndSendCancelRsp(SRpcMsg *pMsg, int32_t code) { pRsp->code = code; SRpcMsg rpcRsp = { + .msgType = TDMT_VND_CANCEL_TASK_RSP, .handle = pMsg->handle, .ahandle = pMsg->ahandle, .pCont = pRsp, @@ -155,6 +159,7 @@ int32_t qwBuildAndSendDropRsp(void *connection, int32_t code) { pRsp->code = code; SRpcMsg rpcRsp = { + .msgType = TDMT_VND_DROP_TASK_RSP, .handle = pMsg->handle, .ahandle = pMsg->ahandle, .pCont = pRsp, diff --git a/source/libs/qworker/test/qworkerTests.cpp b/source/libs/qworker/test/qworkerTests.cpp index 362156ebcd..5812719c51 100644 --- a/source/libs/qworker/test/qworkerTests.cpp +++ b/source/libs/qworker/test/qworkerTests.cpp @@ -38,21 +38,25 @@ namespace { -#define qwtTestQueryQueueSize 1000 -#define qwtTestFetchQueueSize 1000 -#define qwtTestMaxExecTaskUsec 2 +#define qwtTestQueryQueueSize 1000000 +#define qwtTestFetchQueueSize 1000000 + +int32_t qwtTestMaxExecTaskUsec = 2; +int32_t qwtTestReqMaxDelayUsec = 2; uint64_t qwtTestQueryId = 0; bool qwtTestEnableSleep = true; bool qwtTestStop = false; -bool qwtTestDeadLoop = true; -int32_t qwtTestMTRunSec = 10; +bool qwtTestDeadLoop = false; +int32_t qwtTestMTRunSec = 60; int32_t qwtTestPrintNum = 100000; int32_t qwtTestCaseIdx = 0; int32_t qwtTestCaseNum = 4; bool qwtTestCaseFinished = false; tsem_t qwtTestQuerySem; tsem_t qwtTestFetchSem; +int32_t qwtTestQuitThreadNum = 0; + int32_t qwtTestQueryQueueRIdx = 0; int32_t qwtTestQueryQueueWIdx = 0; @@ -104,6 +108,7 @@ void qwtBuildQueryReqMsg(SRpcMsg *queryRpc) { qwtqueryMsg.sId = htobe64(1); qwtqueryMsg.taskId = htobe64(1); qwtqueryMsg.contentLen = htonl(100); + queryRpc->msgType = TDMT_VND_QUERY; queryRpc->pCont = &qwtqueryMsg; queryRpc->contLen = sizeof(SSubQueryMsg) + 100; } @@ -112,6 +117,7 @@ void qwtBuildReadyReqMsg(SResReadyReq *readyMsg, SRpcMsg *readyRpc) { readyMsg->sId = htobe64(1); readyMsg->queryId = htobe64(atomic_load_64(&qwtTestQueryId)); readyMsg->taskId = htobe64(1); + readyRpc->msgType = TDMT_VND_RES_READY; readyRpc->pCont = readyMsg; readyRpc->contLen = sizeof(SResReadyReq); } @@ -120,6 +126,7 @@ void qwtBuildFetchReqMsg(SResFetchReq *fetchMsg, SRpcMsg *fetchRpc) { fetchMsg->sId = htobe64(1); fetchMsg->queryId = htobe64(atomic_load_64(&qwtTestQueryId)); fetchMsg->taskId = htobe64(1); + fetchRpc->msgType = TDMT_VND_FETCH; fetchRpc->pCont = fetchMsg; fetchRpc->contLen = sizeof(SResFetchReq); } @@ -128,6 +135,7 @@ void qwtBuildDropReqMsg(STaskDropReq *dropMsg, SRpcMsg *dropRpc) { dropMsg->sId = htobe64(1); dropMsg->queryId = htobe64(atomic_load_64(&qwtTestQueryId)); dropMsg->taskId = htobe64(1); + dropRpc->msgType = TDMT_VND_DROP_TASK; dropRpc->pCont = dropMsg; dropRpc->contLen = sizeof(STaskDropReq); } @@ -146,7 +154,9 @@ int32_t qwtStringToPlan(const char* str, SSubplan** subplan) { int32_t qwtPutReqToFetchQueue(void *node, struct SRpcMsg *pMsg) { taosWLockLatch(&qwtTestFetchQueueLock); - qwtTestFetchQueue[qwtTestFetchQueueWIdx++] = pMsg; + struct SRpcMsg *newMsg = (struct SRpcMsg *)calloc(1, sizeof(struct SRpcMsg)); + memcpy(newMsg, pMsg, sizeof(struct SRpcMsg)); + qwtTestFetchQueue[qwtTestFetchQueueWIdx++] = newMsg; if (qwtTestFetchQueueWIdx >= qwtTestFetchQueueSize) { qwtTestFetchQueueWIdx = 0; } @@ -167,7 +177,9 @@ int32_t qwtPutReqToFetchQueue(void *node, struct SRpcMsg *pMsg) { int32_t qwtPutReqToQueue(void *node, struct SRpcMsg *pMsg) { taosWLockLatch(&qwtTestQueryQueueLock); - qwtTestQueryQueue[qwtTestQueryQueueWIdx++] = pMsg; + struct SRpcMsg *newMsg = (struct SRpcMsg *)calloc(1, sizeof(struct SRpcMsg)); + memcpy(newMsg, pMsg, sizeof(struct SRpcMsg)); + qwtTestQueryQueue[qwtTestQueryQueueWIdx++] = newMsg; if (qwtTestQueryQueueWIdx >= qwtTestQueryQueueSize) { qwtTestQueryQueueWIdx = 0; } @@ -201,6 +213,7 @@ void qwtRpcSendResponse(const SRpcMsg *pRsp) { qwtPutReqToFetchQueue((void *)0x1, &qwtdropRpc); } + rpcFreeCont(rsp); break; } case TDMT_VND_RES_READY_RSP: { @@ -213,6 +226,7 @@ void qwtRpcSendResponse(const SRpcMsg *pRsp) { qwtBuildDropReqMsg(&qwtdropMsg, &qwtdropRpc); qwtPutReqToFetchQueue((void *)0x1, &qwtdropRpc); } + rpcFreeCont(rsp); break; } case TDMT_VND_FETCH_RSP: { @@ -226,16 +240,19 @@ void qwtRpcSendResponse(const SRpcMsg *pRsp) { qwtBuildDropReqMsg(&qwtdropMsg, &qwtdropRpc); qwtPutReqToFetchQueue((void *)0x1, &qwtdropRpc); + rpcFreeCont(rsp); break; } - case TDMT_VND_DROP_TASK: { + case TDMT_VND_DROP_TASK_RSP: { STaskDropRsp *rsp = (STaskDropRsp *)pRsp->pCont; + rpcFreeCont(rsp); qwtTestCaseFinished = true; break; } } + return; } @@ -271,16 +288,30 @@ int32_t qwtExecTask(qTaskInfo_t tinfo, SSDataBlock** pRes, uint64_t *useconds) { *pRes = NULL; *useconds = 0; } else { + if (qwtTestSinkQueryEnd) { + *pRes = NULL; + *useconds = rand() % 10; + return 0; + } + endExec = rand() % 5; - if (endExec) { - usleep(rand() % qwtTestMaxExecTaskUsec); + int32_t runTime = 0; + if (qwtTestEnableSleep && qwtTestMaxExecTaskUsec > 0) { + runTime = rand() % qwtTestMaxExecTaskUsec; + } + + if (qwtTestEnableSleep) { + if (runTime) { + usleep(runTime); + } + } + if (endExec) { *pRes = (SSDataBlock*)calloc(1, sizeof(SSDataBlock)); (*pRes)->info.rows = rand() % 1000; } else { *pRes = NULL; - usleep(rand() % qwtTestMaxExecTaskUsec); *useconds = rand() % 10; } } @@ -308,9 +339,9 @@ int32_t qwtPutDataBlock(DataSinkHandle handle, const SInputData* pInput, bool* p qwtTestSinkBlockNum++; if (qwtTestSinkBlockNum >= qwtTestSinkMaxBlockNum) { - *pContinue = true; - } else { *pContinue = false; + } else { + *pContinue = true; } taosWUnLockLatch(&qwtTestSinkLock); @@ -653,7 +684,7 @@ void *statusThread(void *param) { } -void *clientThread(void *param) { +void *qwtclientThread(void *param) { int32_t code = 0; uint32_t n = 0; void *mgmt = param; @@ -672,15 +703,14 @@ void *clientThread(void *param) { usleep(1); } - if (qwtTestEnableSleep) { - usleep(rand()%5); - } if (++n % qwtTestPrintNum == 0) { - printf("query:%d\n", n); + printf("case run:%d\n", n); } } + atomic_add_fetch_32(&qwtTestQuitThreadNum, 1); + return NULL; } @@ -689,9 +719,13 @@ void *queryQueueThread(void *param) { SRpcMsg *queryRpc = NULL; void *mgmt = param; - while (!qwtTestStop) { + while (true) { tsem_wait(&qwtTestQuerySem); + if (qwtTestStop && qwtTestQueryQueueNum <= 0) { + break; + } + taosWLockLatch(&qwtTestQueryQueueLock); if (qwtTestQueryQueueNum <= 0 || qwtTestQueryQueueRIdx == qwtTestQueryQueueWIdx) { printf("query queue is empty\n"); @@ -707,6 +741,15 @@ void *queryQueueThread(void *param) { qwtTestQueryQueueNum--; taosWUnLockLatch(&qwtTestQueryQueueLock); + + if (qwtTestEnableSleep && qwtTestReqMaxDelayUsec > 0) { + int32_t delay = rand() % qwtTestReqMaxDelayUsec; + + if (delay) { + usleep(delay); + } + } + if (TDMT_VND_QUERY == queryRpc->msgType) { qWorkerProcessQueryMsg(mockPointer, mgmt, queryRpc); } else if (TDMT_VND_QUERY_CONTINUE == queryRpc->msgType) { @@ -715,8 +758,16 @@ void *queryQueueThread(void *param) { printf("unknown msg in query queue, type:%d\n", queryRpc->msgType); assert(0); } + + free(queryRpc); + + if (qwtTestStop && qwtTestQueryQueueNum <= 0) { + break; + } } + atomic_add_fetch_32(&qwtTestQuitThreadNum, 1); + return NULL; } @@ -725,7 +776,7 @@ void *fetchQueueThread(void *param) { SRpcMsg *fetchRpc = NULL; void *mgmt = param; - while (!qwtTestStop) { + while (true) { tsem_wait(&qwtTestFetchSem); taosWLockLatch(&qwtTestFetchQueueLock); @@ -743,23 +794,45 @@ void *fetchQueueThread(void *param) { qwtTestFetchQueueNum--; taosWUnLockLatch(&qwtTestFetchQueueLock); + if (qwtTestEnableSleep && qwtTestReqMaxDelayUsec > 0) { + int32_t delay = rand() % qwtTestReqMaxDelayUsec; + + if (delay) { + usleep(delay); + } + } + switch (fetchRpc->msgType) { case TDMT_VND_FETCH: qWorkerProcessFetchMsg(mockPointer, mgmt, fetchRpc); + break; case TDMT_VND_RES_READY: qWorkerProcessReadyMsg(mockPointer, mgmt, fetchRpc); + break; case TDMT_VND_TASKS_STATUS: qWorkerProcessStatusMsg(mockPointer, mgmt, fetchRpc); + break; case TDMT_VND_CANCEL_TASK: qWorkerProcessCancelMsg(mockPointer, mgmt, fetchRpc); + break; case TDMT_VND_DROP_TASK: qWorkerProcessDropMsg(mockPointer, mgmt, fetchRpc); + break; default: printf("unknown msg type:%d in fetch queue", fetchRpc->msgType); assert(0); + break; } + + free(fetchRpc); + + if (qwtTestStop && qwtTestFetchQueueNum <= 0) { + break; + } } + atomic_add_fetch_32(&qwtTestQuitThreadNum, 1); + return NULL; } @@ -767,6 +840,7 @@ void *fetchQueueThread(void *param) { } +#if 0 TEST(seqTest, normalCase) { void *mgmt = NULL; @@ -800,31 +874,15 @@ TEST(seqTest, normalCase) { code = qWorkerInit(NODE_TYPE_VNODE, 1, NULL, &mgmt, mockPointer, qwtPutReqToQueue); ASSERT_EQ(code, 0); - qwtBuildStatusReqMsg(&qwtstatusMsg, &statusRpc); - code = qWorkerProcessStatusMsg(mockPointer, mgmt, &statusRpc); - ASSERT_EQ(code, 0); - code = qWorkerProcessQueryMsg(mockPointer, mgmt, &queryRpc); ASSERT_EQ(code, 0); - qwtBuildStatusReqMsg(&qwtstatusMsg, &statusRpc); - code = qWorkerProcessStatusMsg(mockPointer, mgmt, &statusRpc); - ASSERT_EQ(code, 0); - code = qWorkerProcessReadyMsg(mockPointer, mgmt, &readyRpc); ASSERT_EQ(code, 0); - qwtBuildStatusReqMsg(&qwtstatusMsg, &statusRpc); - code = qWorkerProcessStatusMsg(mockPointer, mgmt, &statusRpc); - ASSERT_EQ(code, 0); - code = qWorkerProcessFetchMsg(mockPointer, mgmt, &fetchRpc); ASSERT_EQ(code, 0); - qwtBuildStatusReqMsg(&qwtstatusMsg, &statusRpc); - code = qWorkerProcessStatusMsg(mockPointer, mgmt, &statusRpc); - ASSERT_EQ(code, 0); - code = qWorkerProcessDropMsg(mockPointer, mgmt, &dropRpc); ASSERT_EQ(code, 0); @@ -914,19 +972,31 @@ TEST(seqTest, randCase) { printf("Ready,%d\n", t++); qwtBuildReadyReqMsg(&readyMsg, &readyRpc); code = qWorkerProcessReadyMsg(mockPointer, mgmt, &readyRpc); + if (qwtTestEnableSleep) { + usleep(1); + } } else if (r >= maxr * 2/5 && r < maxr* 3/5) { printf("Fetch,%d\n", t++); qwtBuildFetchReqMsg(&fetchMsg, &fetchRpc); code = qWorkerProcessFetchMsg(mockPointer, mgmt, &fetchRpc); + if (qwtTestEnableSleep) { + usleep(1); + } } else if (r >= maxr * 3/5 && r < maxr * 4/5) { printf("Drop,%d\n", t++); qwtBuildDropReqMsg(&dropMsg, &dropRpc); code = qWorkerProcessDropMsg(mockPointer, mgmt, &dropRpc); + if (qwtTestEnableSleep) { + usleep(1); + } } else if (r >= maxr * 4/5 && r < maxr-1) { printf("Status,%d\n", t++); qwtBuildStatusReqMsg(&statusMsg, &statusRpc); code = qWorkerProcessStatusMsg(mockPointer, mgmt, &statusRpc); ASSERT_EQ(code, 0); + if (qwtTestEnableSleep) { + usleep(1); + } } else { printf("QUIT RAND NOW"); break; @@ -976,7 +1046,236 @@ TEST(seqTest, multithreadRand) { qWorkerDestroy(&mgmt); } -TEST(rcTest, multithread) { +#endif + +TEST(rcTest, shortExecshortDelay) { + void *mgmt = NULL; + int32_t code = 0; + void *mockPointer = (void *)0x1; + + qwtInitLogFile(); + + stubSetStringToPlan(); + stubSetRpcSendResponse(); + stubSetExecTask(); + stubSetCreateExecTask(); + stubSetAsyncKillTask(); + stubSetDestroyTask(); + stubSetDestroyDataSinker(); + stubSetGetDataLength(); + stubSetEndPut(); + stubSetPutDataBlock(); + stubSetGetDataBlock(); + + srand(time(NULL)); + qwtTestStop = false; + qwtTestQuitThreadNum = 0; + + code = qWorkerInit(NODE_TYPE_VNODE, 1, NULL, &mgmt, mockPointer, qwtPutReqToQueue); + ASSERT_EQ(code, 0); + + qwtTestMaxExecTaskUsec = 0; + qwtTestReqMaxDelayUsec = 0; + + tsem_init(&qwtTestQuerySem, 0, 0); + tsem_init(&qwtTestFetchSem, 0, 0); + + pthread_attr_t thattr; + pthread_attr_init(&thattr); + + pthread_t t1,t2,t3,t4,t5; + pthread_create(&(t1), &thattr, qwtclientThread, mgmt); + pthread_create(&(t2), &thattr, queryQueueThread, mgmt); + pthread_create(&(t3), &thattr, fetchQueueThread, mgmt); + + while (true) { + if (qwtTestDeadLoop) { + sleep(1); + } else { + sleep(qwtTestMTRunSec); + break; + } + } + + qwtTestStop = true; + + while (true) { + if (qwtTestQuitThreadNum == 3) { + break; + } + + sleep(3); + + tsem_post(&qwtTestQuerySem); + usleep(10); + } + + qwtTestQueryQueueNum = 0; + qwtTestQueryQueueRIdx = 0; + qwtTestQueryQueueWIdx = 0; + qwtTestQueryQueueLock = 0; + qwtTestFetchQueueNum = 0; + qwtTestFetchQueueRIdx = 0; + qwtTestFetchQueueWIdx = 0; + qwtTestFetchQueueLock = 0; + + qWorkerDestroy(&mgmt); +} + +TEST(rcTest, longExecshortDelay) { + void *mgmt = NULL; + int32_t code = 0; + void *mockPointer = (void *)0x1; + + qwtInitLogFile(); + + stubSetStringToPlan(); + stubSetRpcSendResponse(); + stubSetExecTask(); + stubSetCreateExecTask(); + stubSetAsyncKillTask(); + stubSetDestroyTask(); + stubSetDestroyDataSinker(); + stubSetGetDataLength(); + stubSetEndPut(); + stubSetPutDataBlock(); + stubSetGetDataBlock(); + + srand(time(NULL)); + qwtTestStop = false; + qwtTestQuitThreadNum = 0; + + code = qWorkerInit(NODE_TYPE_VNODE, 1, NULL, &mgmt, mockPointer, qwtPutReqToQueue); + ASSERT_EQ(code, 0); + + qwtTestMaxExecTaskUsec = 1000000; + qwtTestReqMaxDelayUsec = 0; + + tsem_init(&qwtTestQuerySem, 0, 0); + tsem_init(&qwtTestFetchSem, 0, 0); + + pthread_attr_t thattr; + pthread_attr_init(&thattr); + + pthread_t t1,t2,t3,t4,t5; + pthread_create(&(t1), &thattr, qwtclientThread, mgmt); + pthread_create(&(t2), &thattr, queryQueueThread, mgmt); + pthread_create(&(t3), &thattr, fetchQueueThread, mgmt); + + while (true) { + if (qwtTestDeadLoop) { + sleep(1); + } else { + sleep(qwtTestMTRunSec); + break; + } + } + + qwtTestStop = true; + + + while (true) { + if (qwtTestQuitThreadNum == 3) { + break; + } + + sleep(3); + + tsem_post(&qwtTestQuerySem); + usleep(10); + } + + qwtTestQueryQueueNum = 0; + qwtTestQueryQueueRIdx = 0; + qwtTestQueryQueueWIdx = 0; + qwtTestQueryQueueLock = 0; + qwtTestFetchQueueNum = 0; + qwtTestFetchQueueRIdx = 0; + qwtTestFetchQueueWIdx = 0; + qwtTestFetchQueueLock = 0; + + qWorkerDestroy(&mgmt); +} + + +TEST(rcTest, shortExeclongDelay) { + void *mgmt = NULL; + int32_t code = 0; + void *mockPointer = (void *)0x1; + + qwtInitLogFile(); + + stubSetStringToPlan(); + stubSetRpcSendResponse(); + stubSetExecTask(); + stubSetCreateExecTask(); + stubSetAsyncKillTask(); + stubSetDestroyTask(); + stubSetDestroyDataSinker(); + stubSetGetDataLength(); + stubSetEndPut(); + stubSetPutDataBlock(); + stubSetGetDataBlock(); + + srand(time(NULL)); + qwtTestStop = false; + qwtTestQuitThreadNum = 0; + + code = qWorkerInit(NODE_TYPE_VNODE, 1, NULL, &mgmt, mockPointer, qwtPutReqToQueue); + ASSERT_EQ(code, 0); + + qwtTestMaxExecTaskUsec = 0; + qwtTestReqMaxDelayUsec = 1000000; + + tsem_init(&qwtTestQuerySem, 0, 0); + tsem_init(&qwtTestFetchSem, 0, 0); + + pthread_attr_t thattr; + pthread_attr_init(&thattr); + + pthread_t t1,t2,t3,t4,t5; + pthread_create(&(t1), &thattr, qwtclientThread, mgmt); + pthread_create(&(t2), &thattr, queryQueueThread, mgmt); + pthread_create(&(t3), &thattr, fetchQueueThread, mgmt); + + while (true) { + if (qwtTestDeadLoop) { + sleep(1); + } else { + sleep(qwtTestMTRunSec); + break; + } + } + + qwtTestStop = true; + + + while (true) { + if (qwtTestQuitThreadNum == 3) { + break; + } + + sleep(3); + + tsem_post(&qwtTestQuerySem); + usleep(10); + } + + qwtTestQueryQueueNum = 0; + qwtTestQueryQueueRIdx = 0; + qwtTestQueryQueueWIdx = 0; + qwtTestQueryQueueLock = 0; + qwtTestFetchQueueNum = 0; + qwtTestFetchQueueRIdx = 0; + qwtTestFetchQueueWIdx = 0; + qwtTestFetchQueueLock = 0; + + qWorkerDestroy(&mgmt); +} + + +#if 0 +TEST(rcTest, dropTest) { void *mgmt = NULL; int32_t code = 0; void *mockPointer = (void *)0x1; @@ -1025,7 +1324,7 @@ TEST(rcTest, multithread) { qWorkerDestroy(&mgmt); } - +#endif int main(int argc, char** argv) { diff --git a/source/libs/scheduler/src/scheduler.c b/source/libs/scheduler/src/scheduler.c index 0ad51d0b57..d5f7b85b1b 100644 --- a/source/libs/scheduler/src/scheduler.c +++ b/source/libs/scheduler/src/scheduler.c @@ -891,7 +891,7 @@ int32_t schHandleResponseMsg(SSchJob *pJob, SSchTask *pTask, int32_t msgType, ch break; } - case TDMT_VND_DROP_TASK: { + case TDMT_VND_DROP_TASK_RSP: { // SHOULD NEVER REACH HERE SCH_TASK_ELOG("invalid status to handle drop task rsp, ref:%d", atomic_load_32(&pJob->ref)); SCH_ERR_JRET(TSDB_CODE_SCH_INTERNAL_ERROR); From 54e59090ab52db3505fadfcbd1047cd07b5610e3 Mon Sep 17 00:00:00 2001 From: dapan1121 Date: Mon, 24 Jan 2022 09:43:27 +0800 Subject: [PATCH 06/11] feature/qnode --- source/libs/qworker/inc/qworkerInt.h | 2 - source/libs/qworker/src/qworker.c | 37 ++++++++++------- source/libs/qworker/test/qworkerTests.cpp | 49 +++++++++++++++++------ 3 files changed, 59 insertions(+), 29 deletions(-) diff --git a/source/libs/qworker/inc/qworkerInt.h b/source/libs/qworker/inc/qworkerInt.h index 2765d7d5d7..46e46d323e 100644 --- a/source/libs/qworker/inc/qworkerInt.h +++ b/source/libs/qworker/inc/qworkerInt.h @@ -86,8 +86,6 @@ typedef struct SQWPhaseInput { int8_t taskStatus; int8_t taskType; int32_t code; - qTaskInfo_t taskHandle; - DataSinkHandle sinkHandle; } SQWPhaseInput; typedef struct SQWPhaseOutput { diff --git a/source/libs/qworker/src/qworker.c b/source/libs/qworker/src/qworker.c index f1fd8aa6fb..63ae1e71f0 100644 --- a/source/libs/qworker/src/qworker.c +++ b/source/libs/qworker/src/qworker.c @@ -456,13 +456,15 @@ _return: QW_RET(code); } -int32_t qwExecTask(QW_FPARAMS_DEF, qTaskInfo_t *taskHandle, DataSinkHandle sinkHandle, int8_t taskType, bool shortRun) { +int32_t qwExecTask(QW_FPARAMS_DEF, SQWTaskCtx *ctx) { int32_t code = 0; bool qcontinue = true; SSDataBlock* pRes = NULL; uint64_t useconds = 0; int32_t i = 0; - int32_t leftRun = QW_DEFAULT_SHORT_RUN_TIMES; + int32_t execNum = 0; + qTaskInfo_t *taskHandle = &ctx->taskHandle; + DataSinkHandle sinkHandle = ctx->sinkHandle; while (true) { QW_TASK_DLOG("start to execTask in executor, loopIdx:%d", i++); @@ -473,12 +475,14 @@ int32_t qwExecTask(QW_FPARAMS_DEF, qTaskInfo_t *taskHandle, DataSinkHandle sinkH QW_ERR_JRET(code); } + ++execNum; + if (NULL == pRes) { QW_TASK_DLOG("task query done, useconds:%"PRIu64, useconds); dsEndPut(sinkHandle, useconds); - if (TASK_TYPE_TEMP == taskType) { + if (TASK_TYPE_TEMP == ctx->taskType) { qwFreeTaskHandle(QW_FPARAMS(), taskHandle); } @@ -498,7 +502,11 @@ int32_t qwExecTask(QW_FPARAMS_DEF, qTaskInfo_t *taskHandle, DataSinkHandle sinkH break; } - if (shortRun && ((--leftRun) <= 0)) { + if (QW_IS_EVENT_RECEIVED(ctx, QW_EVENT_READY) && execNum >= QW_DEFAULT_SHORT_RUN_TIMES) { + break; + } + + if (QW_IS_EVENT_RECEIVED(ctx, QW_EVENT_FETCH)) { break; } } @@ -617,8 +625,6 @@ int32_t qwHandlePrePhaseEvents(QW_FPARAMS_DEF, int8_t phase, SQWPhaseInput *inpu case QW_PHASE_PRE_QUERY: { atomic_store_8(&ctx->phase, phase); - atomic_store_8(&ctx->taskType, input->taskType); - if (QW_IS_EVENT_PROCESSED(ctx, QW_EVENT_CANCEL) || QW_IS_EVENT_PROCESSED(ctx, QW_EVENT_DROP)) { QW_TASK_ELOG("task already cancelled/dropped at wrong phase, phase:%d", phase); @@ -842,9 +848,6 @@ int32_t qwHandlePostPhaseEvents(QW_FPARAMS_DEF, int8_t phase, SQWPhaseInput *inp } if (QW_PHASE_POST_QUERY == phase) { - ctx->taskHandle = input->taskHandle; - ctx->sinkHandle = input->sinkHandle; - if (NULL == ctx->taskHandle && NULL == ctx->sinkHandle) { ctx->emptyRes = true; } @@ -949,8 +952,7 @@ int32_t qwProcessQuery(QW_FPARAMS_DEF, SQWMsg *qwMsg, int8_t taskType) { SQWPhaseOutput output = {0}; qTaskInfo_t pTaskInfo = NULL; DataSinkHandle sinkHandle = NULL; - - input.taskType = taskType; + SQWTaskCtx *ctx = NULL; QW_ERR_JRET(qwHandlePrePhaseEvents(QW_FPARAMS(), QW_PHASE_PRE_QUERY, &input, &output)); @@ -961,6 +963,10 @@ int32_t qwProcessQuery(QW_FPARAMS_DEF, SQWMsg *qwMsg, int8_t taskType) { QW_TASK_DLOG("task need stop, phase:%d", QW_PHASE_PRE_QUERY); QW_ERR_JRET(code); } + + QW_ERR_JRET(qwGetTaskCtx(QW_FPARAMS(), &ctx)); + + atomic_store_8(&ctx->taskType, taskType); code = qStringToSubplan(qwMsg->msg, &plan); if (TSDB_CODE_SUCCESS != code) { @@ -986,8 +992,11 @@ int32_t qwProcessQuery(QW_FPARAMS_DEF, SQWMsg *qwMsg, int8_t taskType) { queryRsped = true; + atomic_store_ptr(&ctx->taskHandle, pTaskInfo); + atomic_store_ptr(&ctx->sinkHandle, sinkHandle); + if (pTaskInfo && sinkHandle) { - QW_ERR_JRET(qwExecTask(QW_FPARAMS(), &pTaskInfo, sinkHandle, taskType, true)); + QW_ERR_JRET(qwExecTask(QW_FPARAMS(), ctx)); } _return: @@ -1002,8 +1011,6 @@ _return: } input.code = rspCode; - input.taskHandle = pTaskInfo; - input.sinkHandle = sinkHandle; input.taskStatus = rspCode ? JOB_TASK_STATUS_FAILED : JOB_TASK_STATUS_PARTIAL_SUCCEED; QW_ERR_RET(qwHandlePostPhaseEvents(QW_FPARAMS(), QW_PHASE_POST_QUERY, &input, &output)); @@ -1095,7 +1102,7 @@ int32_t qwProcessCQuery(QW_FPARAMS_DEF, SQWMsg *qwMsg) { DataSinkHandle sinkHandle = ctx->sinkHandle; - QW_ERR_JRET(qwExecTask(QW_FPARAMS(), &ctx->taskHandle, sinkHandle, ctx->taskType, QW_IS_EVENT_RECEIVED(ctx, QW_EVENT_FETCH))); + QW_ERR_JRET(qwExecTask(QW_FPARAMS(), ctx)); if (QW_IS_EVENT_RECEIVED(ctx, QW_EVENT_FETCH)) { SOutputData sOutput = {0}; diff --git a/source/libs/qworker/test/qworkerTests.cpp b/source/libs/qworker/test/qworkerTests.cpp index 5812719c51..23a742c6ec 100644 --- a/source/libs/qworker/test/qworkerTests.cpp +++ b/source/libs/qworker/test/qworkerTests.cpp @@ -722,7 +722,7 @@ void *queryQueueThread(void *param) { while (true) { tsem_wait(&qwtTestQuerySem); - if (qwtTestStop && qwtTestQueryQueueNum <= 0) { + if (qwtTestStop && qwtTestQueryQueueNum <= 0 && qwtTestCaseFinished) { break; } @@ -761,7 +761,7 @@ void *queryQueueThread(void *param) { free(queryRpc); - if (qwtTestStop && qwtTestQueryQueueNum <= 0) { + if (qwtTestStop && qwtTestQueryQueueNum <= 0 && qwtTestCaseFinished) { break; } } @@ -779,6 +779,10 @@ void *fetchQueueThread(void *param) { while (true) { tsem_wait(&qwtTestFetchSem); + if (qwtTestStop && qwtTestFetchQueueNum <= 0 && qwtTestCaseFinished) { + break; + } + taosWLockLatch(&qwtTestFetchQueueLock); if (qwtTestFetchQueueNum <= 0 || qwtTestFetchQueueRIdx == qwtTestFetchQueueWIdx) { printf("Fetch queue is empty\n"); @@ -826,7 +830,7 @@ void *fetchQueueThread(void *param) { free(fetchRpc); - if (qwtTestStop && qwtTestFetchQueueNum <= 0) { + if (qwtTestStop && qwtTestFetchQueueNum <= 0 && qwtTestCaseFinished) { break; } } @@ -1104,10 +1108,17 @@ TEST(rcTest, shortExecshortDelay) { break; } - sleep(3); + sleep(1); + + if (qwtTestCaseFinished) { + if (qwtTestQuitThreadNum < 3) { + tsem_post(&qwtTestQuerySem); + tsem_post(&qwtTestFetchSem); + + usleep(10); + } + } - tsem_post(&qwtTestQuerySem); - usleep(10); } qwtTestQueryQueueNum = 0; @@ -1179,10 +1190,17 @@ TEST(rcTest, longExecshortDelay) { break; } - sleep(3); + sleep(1); + + if (qwtTestCaseFinished) { + if (qwtTestQuitThreadNum < 3) { + tsem_post(&qwtTestQuerySem); + tsem_post(&qwtTestFetchSem); + + usleep(10); + } + } - tsem_post(&qwtTestQuerySem); - usleep(10); } qwtTestQueryQueueNum = 0; @@ -1255,10 +1273,17 @@ TEST(rcTest, shortExeclongDelay) { break; } - sleep(3); + sleep(1); + + if (qwtTestCaseFinished) { + if (qwtTestQuitThreadNum < 3) { + tsem_post(&qwtTestQuerySem); + tsem_post(&qwtTestFetchSem); + + usleep(10); + } + } - tsem_post(&qwtTestQuerySem); - usleep(10); } qwtTestQueryQueueNum = 0; From 09568ff6552ea216e3fe5dbed24114913a33371a Mon Sep 17 00:00:00 2001 From: dapan1121 Date: Mon, 24 Jan 2022 10:06:11 +0800 Subject: [PATCH 07/11] feature/qnode --- source/libs/scheduler/src/scheduler.c | 1 + 1 file changed, 1 insertion(+) diff --git a/source/libs/scheduler/src/scheduler.c b/source/libs/scheduler/src/scheduler.c index 15ef6b3d56..46851d722f 100644 --- a/source/libs/scheduler/src/scheduler.c +++ b/source/libs/scheduler/src/scheduler.c @@ -1743,3 +1743,4 @@ void schedulerDestroy(void) { } } + \ No newline at end of file From c7c702daabc9698346550ae3e507c0128905bae5 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 24 Jan 2022 10:07:35 +0800 Subject: [PATCH 08/11] [td-11818] update log. --- source/client/src/clientEnv.c | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/source/client/src/clientEnv.c b/source/client/src/clientEnv.c index d2696fb355..1f2048608a 100644 --- a/source/client/src/clientEnv.c +++ b/source/client/src/clientEnv.c @@ -67,7 +67,9 @@ static void deregisterRequest(SRequestObj* pRequest) { int32_t currentInst = atomic_sub_fetch_32(&pActivity->currentRequests, 1); int32_t num = atomic_sub_fetch_32(&pTscObj->numOfReqs, 1); - tscDebug("0x%"PRIx64" free Request from connObj: 0x%"PRIx64", current:%d, app current:%d", pRequest->self, pTscObj->id, num, currentInst); + int64_t duration = taosGetTimestampMs() - pRequest->metric.start; + tscDebug("0x%"PRIx64" free Request from connObj: 0x%"PRIx64", reqId:0x%"PRIx64" elapsed:%"PRIu64" ms, current:%d, app current:%d", pRequest->self, pTscObj->id, + pRequest->requestId, duration, num, currentInst); taosReleaseRef(clientConnRefPool, pTscObj->id); } From a00a2b27bf0d3b8548d703769e77a0a23cb11783 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Mon, 24 Jan 2022 10:16:45 +0800 Subject: [PATCH 09/11] [td-11818] update log. --- source/libs/catalog/src/catalog.c | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/source/libs/catalog/src/catalog.c b/source/libs/catalog/src/catalog.c index 43b8ae53a4..2197fdfd62 100644 --- a/source/libs/catalog/src/catalog.c +++ b/source/libs/catalog/src/catalog.c @@ -86,7 +86,7 @@ int32_t ctgGetDBVgroupFromMnode(struct SCatalog* pCatalog, void *pRpc, const SEp rpcSendRecv(pRpc, (SEpSet*)pMgmtEps, &rpcMsg, &rpcRsp); if (TSDB_CODE_SUCCESS != rpcRsp.code) { - ctgError("error rsp for use db, code:%x, db:%s", rpcRsp.code, input->db); + ctgError("error rsp for use db, code:%s, db:%s", tstrerror(rpcRsp.code), input->db); CTG_ERR_RET(rpcRsp.code); } @@ -258,7 +258,7 @@ int32_t ctgGetTableMetaFromMnodeImpl(struct SCatalog* pCatalog, void *pTransport return TSDB_CODE_SUCCESS; } - ctgError("error rsp for stablemeta from mnode, code:%x, tbName:%s", rpcRsp.code, tbFullName); + ctgError("error rsp for stablemeta from mnode, code:%s, tbName:%s", tstrerror(rpcRsp.code), tbFullName); CTG_ERR_RET(rpcRsp.code); } @@ -320,18 +320,17 @@ int32_t ctgGetTableMetaFromVnode(struct SCatalog* pCatalog, void *pTransporter, return TSDB_CODE_SUCCESS; } - ctgError("error rsp for table meta from vnode, code:%x, tbName:%s", rpcRsp.code, tNameGetTableName(pTableName)); + ctgError("error rsp for table meta from vnode, code:%s, tbName:%s", tstrerror(rpcRsp.code), tNameGetTableName(pTableName)); CTG_ERR_RET(rpcRsp.code); } code = queryProcessMsgRsp[TMSG_INDEX(TDMT_VND_TABLE_META)](output, rpcRsp.pCont, rpcRsp.contLen); if (code) { - ctgError("Process vnode tablemeta rsp failed, code:%x, tbName:%s", code, tNameGetTableName(pTableName)); + ctgError("Process vnode tablemeta rsp failed, code:%s, tbName:%s", tstrerror(code), tNameGetTableName(pTableName)); CTG_ERR_RET(code); } ctgDebug("Got table meta from vnode, db:%s, tbName:%s", dbFullName, tNameGetTableName(pTableName)); - return TSDB_CODE_SUCCESS; } From 989d35c79cee181a0dc5f0ae5fb340ac1e90165b Mon Sep 17 00:00:00 2001 From: dapan1121 Date: Mon, 24 Jan 2022 10:29:11 +0800 Subject: [PATCH 10/11] feature/qnode --- source/libs/scheduler/src/scheduler.c | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/source/libs/scheduler/src/scheduler.c b/source/libs/scheduler/src/scheduler.c index 46851d722f..f031d37743 100644 --- a/source/libs/scheduler/src/scheduler.c +++ b/source/libs/scheduler/src/scheduler.c @@ -279,7 +279,7 @@ int32_t schRecordTaskSucceedNode(SSchJob *pJob, SSchTask *pTask) { int32_t idx = atomic_load_8(&pTask->candidateIdx); SQueryNodeAddr *addr = taosArrayGet(pTask->candidateAddrs, idx); if (NULL == addr) { - SCH_TASK_ELOG("taosArrayGet candidate addr failed, idx:%d, size:%d", idx, taosArrayGetSize(pTask->candidateAddrs)); + SCH_TASK_ELOG("taosArrayGet candidate addr failed, idx:%d, size:%d", idx, (int32_t)taosArrayGetSize(pTask->candidateAddrs)); SCH_ERR_RET(TSDB_CODE_SCH_INTERNAL_ERROR); } @@ -1743,4 +1743,3 @@ void schedulerDestroy(void) { } } - \ No newline at end of file From 02e454a0d14f594a76a3f986e610c0e0cf54d4fc Mon Sep 17 00:00:00 2001 From: Liu Jicong Date: Mon, 24 Jan 2022 10:46:55 +0800 Subject: [PATCH 11/11] expose interfaces of mq --- include/client/taos.h | 29 +++ include/common/tmsg.h | 37 ++-- source/client/src/clientImpl.c | 223 ++++++++++++++++++--- source/client/src/clientMsgHandler.c | 2 +- source/client/test/clientTests.cpp | 73 ++++--- source/dnode/mnode/impl/inc/mndDef.h | 204 +++++++++++-------- source/dnode/mnode/impl/src/mndSubscribe.c | 18 +- source/dnode/vnode/inc/tq.h | 15 +- source/dnode/vnode/src/inc/tqInt.h | 6 +- source/dnode/vnode/src/tq/tq.c | 105 +++++++++- source/dnode/vnode/src/vnd/vnodeWrite.c | 4 +- 11 files changed, 531 insertions(+), 185 deletions(-) diff --git a/include/client/taos.h b/include/client/taos.h index 84f6255710..40772e9d2c 100644 --- a/include/client/taos.h +++ b/include/client/taos.h @@ -192,9 +192,38 @@ DLL_EXPORT void taos_close_stream(TAOS_STREAM *tstr); DLL_EXPORT int taos_load_table_info(TAOS *taos, const char* tableNameList); DLL_EXPORT TAOS_RES* taos_schemaless_insert(TAOS* taos, char* lines[], int numLines, int protocol, int precision); +typedef struct tmq_t tmq_t; +typedef struct tmq_conf_t tmq_conf_t; +typedef struct tmq_list_t tmq_list_t; + +typedef struct tmq_message_t tmq_message_t; +typedef struct tmq_message_topic_t tmq_message_topic_t; +typedef struct tmq_message_tb_t tmq_message_tb_t; +typedef struct tmq_tb_iter_t tmq_tb_iter_t; +typedef struct tmq_message_col_t tmq_message_col_t; +typedef struct tmq_col_iter_t tmq_col_iter_t; + +DLL_EXPORT tmq_list_t* tmq_list_new(); +DLL_EXPORT int32_t tmq_list_append(tmq_list_t*, char*); + +DLL_EXPORT tmq_conf_t* tmq_conf_new(); + +DLL_EXPORT int32_t tmq_conf_set(tmq_conf_t* conf, const char* key, const char* value); DLL_EXPORT TAOS_RES *taos_create_topic(TAOS* taos, const char* name, const char* sql, int sqlLen); +DLL_EXPORT tmq_t* taos_consumer_new(void* conn, tmq_conf_t* conf, char* errstr, int32_t errstrLen); + +DLL_EXPORT TAOS_RES* tmq_subscribe(tmq_t* tmq, tmq_list_t* topic_list); + +DLL_EXPORT tmq_message_t* tmq_consume_poll(tmq_t* tmq, int64_t blocking_time); + +DLL_EXPORT int32_t tmq_topic_num(tmq_message_t* msg); +DLL_EXPORT char* tmq_get_topic(tmq_message_topic_t* msg); +DLL_EXPORT int32_t tmq_get_vgId(tmq_message_topic_t* msg); +DLL_EXPORT tmq_message_tb_t* tmq_get_next_tb(tmq_message_topic_t* msg, tmq_tb_iter_t* iter); +DLL_EXPORT tmq_message_col_t* tmq_get_next_col(tmq_message_tb_t* msg, tmq_col_iter_t* iter); + #ifdef __cplusplus } #endif diff --git a/include/common/tmsg.h b/include/common/tmsg.h index 857de3671f..7630c5f5e5 100644 --- a/include/common/tmsg.h +++ b/include/common/tmsg.h @@ -1519,7 +1519,8 @@ static FORCE_INLINE void* taosDecodeSMqMsg(void* buf, SMqHbMsg* pMsg) { typedef struct SMqSetCVgReq { int32_t vgId; - int64_t consumerId; + int64_t oldConsumerId; + int64_t newConsumerId; char topicName[TSDB_TOPIC_FNAME_LEN]; char cgroup[TSDB_CONSUMER_GROUP_LEN]; char* sql; @@ -1550,7 +1551,8 @@ static FORCE_INLINE void* tDecodeSSubQueryMsg(void* buf, SSubQueryMsg* pMsg) { static FORCE_INLINE int32_t tEncodeSMqSetCVgReq(void** buf, const SMqSetCVgReq* pReq) { int32_t tlen = 0; tlen += taosEncodeFixedI32(buf, pReq->vgId); - tlen += taosEncodeFixedI64(buf, pReq->consumerId); + tlen += taosEncodeFixedI64(buf, pReq->oldConsumerId); + tlen += taosEncodeFixedI64(buf, pReq->newConsumerId); tlen += taosEncodeString(buf, pReq->topicName); tlen += taosEncodeString(buf, pReq->cgroup); tlen += taosEncodeString(buf, pReq->sql); @@ -1562,7 +1564,8 @@ static FORCE_INLINE int32_t tEncodeSMqSetCVgReq(void** buf, const SMqSetCVgReq* static FORCE_INLINE void* tDecodeSMqSetCVgReq(void* buf, SMqSetCVgReq* pReq) { buf = taosDecodeFixedI32(buf, &pReq->vgId); - buf = taosDecodeFixedI64(buf, &pReq->consumerId); + buf = taosDecodeFixedI64(buf, &pReq->oldConsumerId); + buf = taosDecodeFixedI64(buf, &pReq->newConsumerId); buf = taosDecodeStringTo(buf, pReq->topicName); buf = taosDecodeStringTo(buf, pReq->cgroup); buf = taosDecodeString(buf, &pReq->sql); @@ -1579,15 +1582,6 @@ typedef struct SMqSetCVgRsp { char cGroup[TSDB_CONSUMER_GROUP_LEN]; } SMqSetCVgRsp; -typedef struct SMqConsumeReq { - int64_t reqId; - int64_t offset; - int64_t consumerId; - int64_t blockingTime; - char topicName[TSDB_TOPIC_FNAME_LEN]; - char cgroup[TSDB_CONSUMER_GROUP_LEN]; -} SMqConsumeReq; - typedef struct SMqColData { int16_t colId; int16_t type; @@ -1615,12 +1609,29 @@ typedef struct SMqTopicBlk { typedef struct SMqConsumeRsp { int64_t reqId; - int64_t clientId; + int64_t consumerId; int32_t bodyLen; int32_t numOfTopics; SMqTopicData data[]; } SMqConsumeRsp; +// one req for one vg+topic +typedef struct SMqConsumeReq { + //0: commit only, current offset + //1: consume only, poll next offset + //2: commit current and consume next offset + int32_t reqType; + + int64_t reqId; + int64_t consumerId; + int64_t blockingTime; + char cgroup[TSDB_CONSUMER_GROUP_LEN]; + + int64_t offset; + char topic[TSDB_TOPIC_FNAME_LEN]; +} SMqConsumeReq; + + #ifdef __cplusplus } #endif diff --git a/source/client/src/clientImpl.c b/source/client/src/clientImpl.c index 7d5199dfaf..f5d7f169de 100644 --- a/source/client/src/clientImpl.c +++ b/source/client/src/clientImpl.c @@ -117,7 +117,7 @@ TAOS *taos_connect_internal(const char *ip, const char *user, const char *pass, SAppInstInfo* p = calloc(1, sizeof(struct SAppInstInfo)); p->mgmtEp = epSet; p->pTransporter = openTransporter(user, secretEncrypt, tsNumOfCores); - /*p->pAppHbMgr = appHbMgrInit(p);*/ + p->pAppHbMgr = appHbMgrInit(p); taosHashPut(appInfo.pInstMap, key, strlen(key), &p, POINTER_BYTES); pInst = &p; @@ -254,14 +254,8 @@ int32_t scheduleQuery(SRequestObj* pRequest, SQueryDag* pDag, SArray* pNodeList) return scheduleAsyncExecJob(pRequest->pTscObj->pAppInfo->pTransporter, pNodeList, pDag, &pRequest->body.pQueryJob); } -typedef struct tmq_t tmq_t; -typedef struct SMqClientTopic { - // subscribe info - int32_t sqlLen; - char* sql; - char* topicName; - int64_t topicId; +typedef struct SMqClientVg { // statistics int64_t consumeCnt; // offset @@ -270,36 +264,160 @@ typedef struct SMqClientTopic { //connection info int32_t vgId; SEpSet epSet; +} SMqClientVg; + +typedef struct SMqClientTopic { + // subscribe info + int32_t sqlLen; + char* sql; + char* topicName; + int64_t topicId; + int32_t nextVgIdx; + SArray* vgs; //SArray } SMqClientTopic; typedef struct tmq_resp_err_t { int32_t code; } tmq_resp_err_t; -typedef struct tmq_topic_vgroup_list_t { - char* topicName; +typedef struct tmq_topic_vgroup_t { + char* topic; int32_t vgId; - int64_t committedOffset; + int64_t commitOffset; +} tmq_topic_vgroup_t; + +typedef struct tmq_topic_vgroup_list_t { + int32_t cnt; + int32_t size; + tmq_topic_vgroup_t* elems; } tmq_topic_vgroup_list_t; typedef void (tmq_commit_cb(tmq_t*, tmq_resp_err_t, tmq_topic_vgroup_list_t*, void* param)); -typedef struct tmq_conf_t{ - char* clientId; - char* groupId; +struct tmq_conf_t { + char clientId[256]; + char groupId[256]; char* ip; uint16_t port; tmq_commit_cb* commit_cb; -} tmq_conf_t; +}; + +tmq_conf_t* tmq_conf_new() { + tmq_conf_t* conf = calloc(1, sizeof(tmq_conf_t)); + return conf; +} + +int32_t tmq_conf_set(tmq_conf_t* conf, const char* key, const char* value) { + if (strcmp(key, "group.id")) { + strcpy(conf->groupId, value); + } + if (strcmp(key, "client.id")) { + strcpy(conf->clientId, value); + } + return 0; +} struct tmq_t { char groupId[256]; char clientId[256]; + int64_t consumerId; + int64_t status; STscObj* pTscObj; tmq_commit_cb* commit_cb; - SArray* clientTopics; // SArray + int32_t nextTopicIdx; + SArray* clientTopics; //SArray }; +tmq_t* taos_consumer_new(void* conn, tmq_conf_t* conf, char* errstr, int32_t errstrLen) { + tmq_t* pTmq = calloc(sizeof(tmq_t), 1); + if (pTmq == NULL) { + return NULL; + } + pTmq->pTscObj = (STscObj*)conn; + pTmq->status = 0; + strcpy(pTmq->clientId, conf->clientId); + strcpy(pTmq->groupId, conf->groupId); + pTmq->commit_cb = conf->commit_cb; + pTmq->consumerId = generateRequestId() & ((uint64_t)-1 >> 1); + return pTmq; +} + +struct tmq_list_t { + int32_t cnt; + int32_t tot; + char* elems[]; +}; +tmq_list_t* tmq_list_new() { + tmq_list_t *ptr = malloc(sizeof(tmq_list_t) + 8 * sizeof(char*)); + if (ptr == NULL) { + return ptr; + } + ptr->cnt = 0; + ptr->tot = 8; + return ptr; +} + +int32_t tmq_list_append(tmq_list_t* ptr, char* src) { + if (ptr->cnt >= ptr->tot-1) return -1; + ptr->elems[ptr->cnt] = src; + ptr->cnt++; + return 0; +} + + +TAOS_RES* tmq_subscribe(tmq_t* tmq, tmq_list_t* topic_list) { + SRequestObj *pRequest = NULL; + tmq->status = 1; + int32_t sz = topic_list->cnt; + tmq->clientTopics = taosArrayInit(sz, sizeof(void*)); + for (int i = 0; i < sz; i++) { + char* topicName = strdup(topic_list->elems[i]); + taosArrayPush(tmq->clientTopics, &topicName); + } + SCMSubscribeReq req; + req.topicNum = taosArrayGetSize(tmq->clientTopics); + req.consumerId = tmq->consumerId; + req.consumerGroup = strdup(tmq->groupId); + req.topicNames = tmq->clientTopics; + + int tlen = tSerializeSCMSubscribeReq(NULL, &req); + void* buf = malloc(tlen); + if(buf == NULL) { + goto _return; + } + + void* abuf = buf; + tSerializeSCMSubscribeReq(&abuf, &req); + /*printf("formatted: %s\n", dagStr);*/ + + pRequest = createRequest(tmq->pTscObj, NULL, NULL, TSDB_SQL_SELECT); + if (pRequest == NULL) { + tscError("failed to malloc sqlObj"); + } + + pRequest->body.requestMsg = (SDataBuf){ .pData = buf, .len = tlen }; + pRequest->type = TDMT_MND_CREATE_TOPIC; + + SMsgSendInfo* body = buildMsgInfoImpl(pRequest); + SEpSet epSet = getEpSet_s(&tmq->pTscObj->pAppInfo->mgmtEp); + + int64_t transporterId = 0; + asyncSendMsgToServer(tmq->pTscObj->pAppInfo->pTransporter, &epSet, &transporterId, body); + + tsem_wait(&pRequest->body.rspSem); + +_return: + if (body != NULL) { + destroySendMsgInfo(body); + } + + if (pRequest != NULL && terrno != TSDB_CODE_SUCCESS) { + pRequest->code = terrno; + } + + return pRequest; +} + void tmq_conf_set_offset_commit_cb(tmq_conf_t* conf, tmq_commit_cb* cb) { conf->commit_cb = cb; } @@ -327,10 +445,10 @@ SArray* tmqGetConnInfo(SClientHbKey connKey, void* param) { int sz = taosArrayGetSize(clientTopics); for (int i = 0; i < sz; i++) { SMqClientTopic* pCTopic = taosArrayGet(clientTopics, i); - if (pCTopic->vgId == -1) { - pMqHb->status = 1; - break; - } + /*if (pCTopic->vgId == -1) {*/ + /*pMqHb->status = 1;*/ + /*break;*/ + /*}*/ } kv.value = pMqHb; kv.valueLen = sizeof(SMqHbMsg); @@ -451,22 +569,63 @@ _return: return pRequest; } -typedef struct tmq_message_t { - int32_t numOfRows; - char* topicName; - TAOS_ROW row[]; -} tmq_message_t; +/*typedef SMqConsumeRsp tmq_message_t;*/ -tmq_message_t* tmq_consume_poll(tmq_t* mq, int64_t blocking_time) { +struct tmq_message_t { + SMqConsumeRsp rsp; +}; + +tmq_message_t* tmq_consume_poll(tmq_t* tmq, int64_t blocking_time) { + if (tmq->clientTopics == NULL || taosArrayGetSize(tmq->clientTopics) == 0) { + return NULL; + } + SRequestObj *pRequest = NULL; + SMqConsumeReq req = {0}; + req.reqType = 1; + req.blockingTime = blocking_time; + req.consumerId = tmq->consumerId; + strcpy(req.cgroup, tmq->groupId); + + SMqClientTopic* pTopic = taosArrayGet(tmq->clientTopics, tmq->nextTopicIdx); + tmq->nextTopicIdx = (tmq->nextTopicIdx + 1) % taosArrayGetSize(tmq->clientTopics); + strcpy(req.topic, pTopic->topicName); + int32_t nextVgIdx = pTopic->nextVgIdx; + pTopic->nextVgIdx = (nextVgIdx + 1) % taosArrayGetSize(pTopic->vgs); + SMqClientVg* pVg = taosArrayGet(pTopic->vgs, nextVgIdx); + req.offset = pVg->currentOffset; + + pRequest->body.requestMsg = (SDataBuf){ .pData = &req, .len = sizeof(SMqConsumeReq) }; + pRequest->type = TDMT_VND_CONSUME; + + SMsgSendInfo* body = buildMsgInfoImpl(pRequest); + + int64_t transporterId = 0; + asyncSendMsgToServer(tmq->pTscObj->pAppInfo->pTransporter, &pVg->epSet, &transporterId, body); + + tsem_wait(&pRequest->body.rspSem); + + return (tmq_message_t*)pRequest->body.resInfo.pData; + + /*tsem_wait(&pRequest->body.rspSem);*/ + + /*if (body != NULL) {*/ + /*destroySendMsgInfo(body);*/ + /*}*/ + + /*if (pRequest != NULL && terrno != TSDB_CODE_SUCCESS) {*/ + /*pRequest->code = terrno;*/ + /*}*/ + + /*return pRequest;*/ +} + +tmq_resp_err_t* tmq_commit(tmq_t* tmq, tmq_topic_vgroup_list_t* tmq_topic_vgroup_list, int32_t async) { + SMqConsumeReq req = {0}; return NULL; } -tmq_resp_err_t* tmq_commit(tmq_t* mq, void* callback, int32_t async) { - return NULL; -} - -void tmq_message_destroy(tmq_message_t* mq_message) { - +void tmq_message_destroy(tmq_message_t* tmq_message) { + if (tmq_message == NULL) return; } diff --git a/source/client/src/clientMsgHandler.c b/source/client/src/clientMsgHandler.c index 81ea18fe08..ec088eb073 100644 --- a/source/client/src/clientMsgHandler.c +++ b/source/client/src/clientMsgHandler.c @@ -72,7 +72,7 @@ int processConnectRsp(void* param, const SDataBuf* pMsg, int32_t code) { atomic_add_fetch_64(&pTscObj->pAppInfo->numOfConns, 1); SClientHbKey connKey = {.connId = pConnect->connId, .hbType = HEARTBEAT_TYPE_QUERY}; - /*hbRegisterConn(pTscObj->pAppInfo->pAppHbMgr, connKey, NULL);*/ + hbRegisterConn(pTscObj->pAppInfo->pAppHbMgr, connKey, NULL); // pRequest->body.resInfo.pRspMsg = pMsg->pData; tscDebug("0x%" PRIx64 " clusterId:%" PRId64 ", totalConn:%" PRId64, pRequest->requestId, pConnect->clusterId, diff --git a/source/client/test/clientTests.cpp b/source/client/test/clientTests.cpp index ee23567705..a2a26cf6dc 100644 --- a/source/client/test/clientTests.cpp +++ b/source/client/test/clientTests.cpp @@ -526,29 +526,54 @@ TEST(testCase, show_table_Test) { // taosHashCleanup(phash); //} // -//TEST(testCase, create_topic_Test) { -// TAOS* pConn = taos_connect("localhost", "root", "taosdata", NULL, 0); -// assert(pConn != NULL); -// -// TAOS_RES* pRes = taos_query(pConn, "use abc1"); -// if (taos_errno(pRes) != 0) { -// printf("error in use db, reason:%s\n", taos_errstr(pRes)); -// } -// taos_free_result(pRes); -// -// TAOS_FIELD* pFields = taos_fetch_fields(pRes); -// ASSERT_TRUE(pFields == nullptr); -// -// int32_t numOfFields = taos_num_fields(pRes); -// ASSERT_EQ(numOfFields, 0); -// -// taos_free_result(pRes); -// -// char* sql = "select * from tu"; -// pRes = taos_create_topic(pConn, "test_topic_1", sql, strlen(sql)); -// taos_free_result(pRes); -// taos_close(pConn); -//} +TEST(testCase, create_topic_Test) { + TAOS* pConn = taos_connect("localhost", "root", "taosdata", NULL, 0); + assert(pConn != NULL); + + TAOS_RES* pRes = taos_query(pConn, "use abc1"); + if (taos_errno(pRes) != 0) { + printf("error in use db, reason:%s\n", taos_errstr(pRes)); + } + taos_free_result(pRes); + + TAOS_FIELD* pFields = taos_fetch_fields(pRes); + ASSERT_TRUE(pFields == nullptr); + + int32_t numOfFields = taos_num_fields(pRes); + ASSERT_EQ(numOfFields, 0); + + taos_free_result(pRes); + + char* sql = "select * from tu"; + pRes = taos_create_topic(pConn, "test_topic_1", sql, strlen(sql)); + taos_free_result(pRes); + taos_close(pConn); +} + +TEST(testCase, tmq_subscribe_Test) { + TAOS* pConn = taos_connect("localhost", "root", "taosdata", NULL, 0); + assert(pConn != NULL); + + tmq_conf_t* conf = tmq_conf_new(); + tmq_conf_set(conf, "group.id", "tg1"); + tmq_t* tmq = taos_consumer_new(pConn, conf, NULL, 0); + + tmq_list_t* topic_list = tmq_list_new(); + tmq_list_append(topic_list, "test_topic_1"); + tmq_subscribe(tmq, topic_list); + + while (1) { + tmq_message_t* msg = tmq_consume_poll(tmq, 0); + printf("get msg\n"); + if (msg == NULL) break; + } +} + +TEST(testCase, tmq_consume_Test) { +} + +TEST(testCase, tmq_commit_TEST) { +} //TEST(testCase, insert_test) { // TAOS* pConn = taos_connect("localhost", "root", "taosdata", NULL, 0); @@ -681,4 +706,4 @@ TEST(testCase, projection_query_tables) { // taos_close(pConn); //} -#pragma GCC diagnostic pop \ No newline at end of file +#pragma GCC diagnostic pop diff --git a/source/dnode/mnode/impl/inc/mndDef.h b/source/dnode/mnode/impl/inc/mndDef.h index 1507e2a30d..78f371133c 100644 --- a/source/dnode/mnode/impl/inc/mndDef.h +++ b/source/dnode/mnode/impl/inc/mndDef.h @@ -19,14 +19,14 @@ #include "os.h" #include "cJSON.h" +#include "scheduler.h" #include "sync.h" -#include "tmsg.h" #include "thash.h" #include "tlist.h" #include "tlog.h" +#include "tmsg.h" #include "trpc.h" #include "ttimer.h" -#include "scheduler.h" #include "mnode.h" @@ -37,12 +37,42 @@ extern "C" { extern int32_t mDebugFlag; // mnode log function -#define mFatal(...) { if (mDebugFlag & DEBUG_FATAL) { taosPrintLog("MND FATAL ", 255, __VA_ARGS__); }} -#define mError(...) { if (mDebugFlag & DEBUG_ERROR) { taosPrintLog("MND ERROR ", 255, __VA_ARGS__); }} -#define mWarn(...) { if (mDebugFlag & DEBUG_WARN) { taosPrintLog("MND WARN ", 255, __VA_ARGS__); }} -#define mInfo(...) { if (mDebugFlag & DEBUG_INFO) { taosPrintLog("MND ", 255, __VA_ARGS__); }} -#define mDebug(...) { if (mDebugFlag & DEBUG_DEBUG) { taosPrintLog("MND ", mDebugFlag, __VA_ARGS__); }} -#define mTrace(...) { if (mDebugFlag & DEBUG_TRACE) { taosPrintLog("MND ", mDebugFlag, __VA_ARGS__); }} +#define mFatal(...) \ + { \ + if (mDebugFlag & DEBUG_FATAL) { \ + taosPrintLog("MND FATAL ", 255, __VA_ARGS__); \ + } \ + } +#define mError(...) \ + { \ + if (mDebugFlag & DEBUG_ERROR) { \ + taosPrintLog("MND ERROR ", 255, __VA_ARGS__); \ + } \ + } +#define mWarn(...) \ + { \ + if (mDebugFlag & DEBUG_WARN) { \ + taosPrintLog("MND WARN ", 255, __VA_ARGS__); \ + } \ + } +#define mInfo(...) \ + { \ + if (mDebugFlag & DEBUG_INFO) { \ + taosPrintLog("MND ", 255, __VA_ARGS__); \ + } \ + } +#define mDebug(...) \ + { \ + if (mDebugFlag & DEBUG_DEBUG) { \ + taosPrintLog("MND ", mDebugFlag, __VA_ARGS__); \ + } \ + } +#define mTrace(...) \ + { \ + if (mDebugFlag & DEBUG_TRACE) { \ + taosPrintLog("MND ", mDebugFlag, __VA_ARGS__); \ + } \ + } typedef enum { MND_AUTH_ACCT_START = 0, @@ -96,13 +126,13 @@ typedef struct { ETrnPolicy policy; int32_t code; int32_t failedTimes; - void *rpcHandle; - void *rpcAHandle; - SArray *redoLogs; - SArray *undoLogs; - SArray *commitLogs; - SArray *redoActions; - SArray *undoActions; + void* rpcHandle; + void* rpcAHandle; + SArray* redoLogs; + SArray* undoLogs; + SArray* commitLogs; + SArray* redoActions; + SArray* undoActions; } STrans; typedef struct { @@ -135,28 +165,28 @@ typedef struct { ESyncState role; int32_t roleTerm; int64_t roleTime; - SDnodeObj *pDnode; + SDnodeObj* pDnode; } SMnodeObj; typedef struct { int32_t id; int64_t createdTime; int64_t updateTime; - SDnodeObj *pDnode; + SDnodeObj* pDnode; } SQnodeObj; typedef struct { int32_t id; int64_t createdTime; int64_t updateTime; - SDnodeObj *pDnode; + SDnodeObj* pDnode; } SSnodeObj; typedef struct { int32_t id; int64_t createdTime; int64_t updateTime; - SDnodeObj *pDnode; + SDnodeObj* pDnode; } SBnodeObj; typedef struct { @@ -201,7 +231,7 @@ typedef struct { int64_t updateTime; int8_t superUser; int32_t acctId; - SHashObj *prohibitDbHash; + SHashObj* prohibitDbHash; } SUserObj; typedef struct { @@ -226,15 +256,15 @@ typedef struct { } SDbCfg; typedef struct { - char name[TSDB_DB_FNAME_LEN]; - char acct[TSDB_USER_LEN]; - int64_t createdTime; - int64_t updateTime; + char name[TSDB_DB_FNAME_LEN]; + char acct[TSDB_USER_LEN]; + int64_t createdTime; + int64_t updateTime; uint64_t uid; - int32_t cfgVersion; - int32_t vgVersion; - int8_t hashMethod; // default is 1 - SDbCfg cfg; + int32_t cfgVersion; + int32_t vgVersion; + int8_t hashMethod; // default is 1 + SDbCfg cfg; } SDbObj; typedef struct { @@ -272,7 +302,7 @@ typedef struct { int32_t numOfColumns; int32_t numOfTags; SRWLatch lock; - SSchema *pSchema; + SSchema* pSchema; } SStbObj; typedef struct { @@ -287,8 +317,8 @@ typedef struct { int64_t sigature; int32_t commentSize; int32_t codeSize; - char *pComment; - char *pCode; + char* pComment; + char* pCode; char pData[]; } SFuncObj; @@ -301,8 +331,8 @@ typedef struct { int32_t numOfRows; int32_t numOfReads; int32_t payloadLen; - void *pIter; - SMnode *pMnode; + void* pIter; + SMnode* pMnode; char db[TSDB_DB_FNAME_LEN]; int16_t offset[TSDB_MAX_COLUMNS]; int32_t bytes[TSDB_MAX_COLUMNS]; @@ -327,9 +357,10 @@ typedef struct SMqTopicConsumer { #endif typedef struct SMqConsumerEp { - int32_t vgId; // -1 for unassigned + int32_t vgId; // -1 for unassigned + int32_t status; SEpSet epSet; - int64_t consumerId; // -1 for unassigned + int64_t consumerId; // -1 for unassigned int64_t lastConsumerHbTs; int64_t lastVgHbTs; int32_t execLen; @@ -339,6 +370,7 @@ typedef struct SMqConsumerEp { static FORCE_INLINE int32_t tEncodeSMqConsumerEp(void** buf, SMqConsumerEp* pConsumerEp) { int32_t tlen = 0; tlen += taosEncodeFixedI32(buf, pConsumerEp->vgId); + tlen += taosEncodeFixedI32(buf, pConsumerEp->status); tlen += taosEncodeSEpSet(buf, &pConsumerEp->epSet); tlen += taosEncodeFixedI64(buf, pConsumerEp->consumerId); tlen += tEncodeSSubQueryMsg(buf, &pConsumerEp->qExec); @@ -347,6 +379,7 @@ static FORCE_INLINE int32_t tEncodeSMqConsumerEp(void** buf, SMqConsumerEp* pCon static FORCE_INLINE void* tDecodeSMqConsumerEp(void** buf, SMqConsumerEp* pConsumerEp) { buf = taosDecodeFixedI32(buf, &pConsumerEp->vgId); + buf = taosDecodeFixedI32(buf, &pConsumerEp->status); buf = taosDecodeSEpSet(buf, &pConsumerEp->epSet); buf = taosDecodeFixedI64(buf, &pConsumerEp->consumerId); buf = tDecodeSSubQueryMsg(buf, &pConsumerEp->qExec); @@ -354,16 +387,17 @@ static FORCE_INLINE void* tDecodeSMqConsumerEp(void** buf, SMqConsumerEp* pConsu return buf; } -//unit for rebalance +// unit for rebalance typedef struct SMqSubscribeObj { char key[TSDB_SUBSCRIBE_KEY_LEN]; int32_t epoch; - //TODO: replace with priority queue + // TODO: replace with priority queue int32_t nextConsumerIdx; - SArray* availConsumer; // SArray (consumerId) - SArray* assigned; // SArray - SArray* unassignedConsumer; // SArray - SArray* unassignedVg; // SArray + SArray* availConsumer; // SArray (consumerId) + SArray* assigned; // SArray + SArray* idleConsumer; // SArray + SArray* lostConsumer; // SArray + SArray* unassignedVg; // SArray } SMqSubscribeObj; static FORCE_INLINE SMqSubscribeObj* tNewSubscribeObj() { @@ -384,17 +418,17 @@ static FORCE_INLINE SMqSubscribeObj* tNewSubscribeObj() { free(pSub); return NULL; } - pSub->unassignedConsumer = taosArrayInit(0, sizeof(SMqConsumerEp)); + pSub->idleConsumer = taosArrayInit(0, sizeof(SMqConsumerEp)); if (pSub->assigned == NULL) { taosArrayDestroy(pSub->availConsumer); - taosArrayDestroy(pSub->unassignedConsumer); + taosArrayDestroy(pSub->idleConsumer); free(pSub); return NULL; } pSub->unassignedVg = taosArrayInit(0, sizeof(SMqConsumerEp)); if (pSub->assigned == NULL) { taosArrayDestroy(pSub->availConsumer); - taosArrayDestroy(pSub->unassignedConsumer); + taosArrayDestroy(pSub->idleConsumer); taosArrayDestroy(pSub->unassignedVg); free(pSub); return NULL; @@ -422,10 +456,10 @@ static FORCE_INLINE int32_t tEncodeSubscribeObj(void** buf, const SMqSubscribeOb tlen += tEncodeSMqConsumerEp(buf, pCEp); } - sz = taosArrayGetSize(pSub->unassignedConsumer); + sz = taosArrayGetSize(pSub->idleConsumer); tlen += taosEncodeFixedI32(buf, sz); for (int32_t i = 0; i < sz; i++) { - SMqConsumerEp* pCEp = taosArrayGet(pSub->unassignedConsumer, i); + SMqConsumerEp* pCEp = taosArrayGet(pSub->idleConsumer, i); tlen += tEncodeSMqConsumerEp(buf, pCEp); } @@ -457,22 +491,22 @@ static FORCE_INLINE void* tDecodeSubscribeObj(void* buf, SMqSubscribeObj* pSub) } buf = taosDecodeFixedI32(buf, &sz); - pSub->unassignedConsumer = taosArrayInit(sz, sizeof(SMqConsumerEp)); - if (pSub->unassignedConsumer == NULL) { + pSub->idleConsumer = taosArrayInit(sz, sizeof(SMqConsumerEp)); + if (pSub->idleConsumer == NULL) { taosArrayDestroy(pSub->assigned); return NULL; } for (int32_t i = 0; i < sz; i++) { SMqConsumerEp cEp; buf = tDecodeSMqConsumerEp(buf, &cEp); - taosArrayPush(pSub->unassignedConsumer, &cEp); + taosArrayPush(pSub->idleConsumer, &cEp); } buf = taosDecodeFixedI32(buf, &sz); pSub->unassignedVg = taosArrayInit(sz, sizeof(SMqConsumerEp)); if (pSub->unassignedVg == NULL) { taosArrayDestroy(pSub->assigned); - taosArrayDestroy(pSub->unassignedConsumer); + taosArrayDestroy(pSub->idleConsumer); return NULL; } for (int32_t i = 0; i < sz; i++) { @@ -487,38 +521,37 @@ static FORCE_INLINE void* tDecodeSubscribeObj(void* buf, SMqSubscribeObj* pSub) typedef struct SMqCGroup { char name[TSDB_CONSUMER_GROUP_LEN]; int32_t status; // 0 - uninitialized, 1 - wait rebalance, 2- normal - SList *consumerIds; // SList - SList *idleVGroups; // SList + SList* consumerIds; // SList + SList* idleVGroups; // SList } SMqCGroup; typedef struct SMqTopicObj { - char name[TSDB_TOPIC_FNAME_LEN]; - char db[TSDB_DB_FNAME_LEN]; - int64_t createTime; - int64_t updateTime; - uint64_t uid; - uint64_t dbUid; - int32_t version; - SRWLatch lock; - int32_t sqlLen; - char *sql; - char *logicalPlan; - char *physicalPlan; - //SHashObj *cgroups; // SHashObj - //SHashObj *consumers; // SHashObj + char name[TSDB_TOPIC_FNAME_LEN]; + char db[TSDB_DB_FNAME_LEN]; + int64_t createTime; + int64_t updateTime; + uint64_t uid; + uint64_t dbUid; + int32_t version; + SRWLatch lock; + int32_t sqlLen; + char* sql; + char* logicalPlan; + char* physicalPlan; + // SHashObj *cgroups; // SHashObj + // SHashObj *consumers; // SHashObj } SMqTopicObj; // TODO: add cache and change name to id typedef struct SMqConsumerTopic { char name[TSDB_TOPIC_FNAME_LEN]; int32_t epoch; - //TODO: replace with something with ep - //SList *vgroups; // SList - //vg assigned to the consumer on the topic - SArray *pVgInfo; // SArray + // vg assigned to the consumer on the topic + SArray* pVgInfo; // SArray } SMqConsumerTopic; -static FORCE_INLINE SMqConsumerTopic* tNewConsumerTopic(int64_t consumerId, SMqTopicObj* pTopic, SMqSubscribeObj* pSub) { +static FORCE_INLINE SMqConsumerTopic* tNewConsumerTopic(int64_t consumerId, SMqTopicObj* pTopic, + SMqSubscribeObj* pSub) { SMqConsumerTopic* pCTopic = malloc(sizeof(SMqConsumerTopic)); if (pCTopic == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; @@ -567,10 +600,11 @@ static FORCE_INLINE void* tDecodeSMqConsumerTopic(void* buf, SMqConsumerTopic* p typedef struct SMqConsumerObj { int64_t consumerId; + int64_t connId; SRWLatch lock; char cgroup[TSDB_CONSUMER_GROUP_LEN]; - SArray *topics; // SArray - //SHashObj *topicHash; //SHashObj + SArray* topics; // SArray + // SHashObj *topicHash; //SHashObj } SMqConsumerObj; static FORCE_INLINE int32_t tEncodeSMqConsumerObj(void** buf, const SMqConsumerObj* pConsumer) { @@ -602,12 +636,12 @@ static FORCE_INLINE void* tDecodeSMqConsumerObj(void* buf, SMqConsumerObj* pCons typedef struct SMqSubConsumerObj { int64_t consumerUid; // if -1, unassigned - SList *vgId; // SList + SList* vgId; // SList } SMqSubConsumerObj; typedef struct SMqSubCGroupObj { char name[TSDB_CONSUMER_GROUP_LEN]; - SList *consumers; // SList + SList* consumers; // SList } SMqSubCGroupObj; typedef struct SMqSubTopicObj { @@ -620,30 +654,30 @@ typedef struct SMqSubTopicObj { int32_t version; SRWLatch lock; int32_t sqlLen; - char *sql; - char *logicalPlan; - char *physicalPlan; - SList *cgroups; // SList + char* sql; + char* logicalPlan; + char* physicalPlan; + SList* cgroups; // SList } SMqSubTopicObj; typedef struct SMqConsumerSubObj { int64_t topicUid; - SList *vgIds; // SList + SList* vgIds; // SList } SMqConsumerSubObj; typedef struct SMqConsumerHbObj { int64_t consumerId; - SList *consumerSubs; // SList + SList* consumerSubs; // SList } SMqConsumerHbObj; typedef struct SMqVGroupSubObj { int64_t topicUid; - SList *consumerIds; // SList + SList* consumerIds; // SList } SMqVGroupSubObj; typedef struct SMqVGroupHbObj { int64_t vgId; - SList *vgSubs; // SList + SList* vgSubs; // SList } SMqVGroupHbObj; #if 0 @@ -663,11 +697,11 @@ typedef struct SMnodeMsg { char user[TSDB_USER_LEN]; char db[TSDB_DB_FNAME_LEN]; int32_t acctId; - SMnode *pMnode; + SMnode* pMnode; int64_t createdTime; SRpcMsg rpcMsg; int32_t contLen; - void *pCont; + void* pCont; } SMnodeMsg; #ifdef __cplusplus diff --git a/source/dnode/mnode/impl/src/mndSubscribe.c b/source/dnode/mnode/impl/src/mndSubscribe.c index 817ca4f4be..9a573cbe2c 100644 --- a/source/dnode/mnode/impl/src/mndSubscribe.c +++ b/source/dnode/mnode/impl/src/mndSubscribe.c @@ -98,7 +98,8 @@ static int32_t mndProcessMqTimerMsg(SMnodeMsg *pMsg) { // build msg SMqSetCVgReq req = { .vgId = pCEp->vgId, - .consumerId = consumerId, + .oldConsumerId = -1, + .newConsumerId = consumerId, }; strcpy(req.cgroup, cgroup); strcpy(req.topicName, topic); @@ -152,6 +153,7 @@ static int mndInitUnassignedVg(SMnode *pMnode, SMqTopicObj *pTopic, SArray *unas //convert dag to msg for (int32_t i = 0; i < sz; i++) { SMqConsumerEp CEp; + CEp.status = 0; CEp.lastConsumerHbTs = CEp.lastVgHbTs = -1; STaskInfo* pTaskInfo = taosArrayGet(pArray, i); tConvertQueryAddrToEpSet(&CEp.epSet, &pTaskInfo->addr); @@ -171,7 +173,8 @@ static int mndBuildMqSetConsumerVgReq(SMnode *pMnode, STrans *pTrans, SMqConsume SVgObj *pVgObj = mndAcquireVgroup(pMnode, vgId); SMqSetCVgReq req = { .vgId = vgId, - .consumerId = pConsumer->consumerId, + .oldConsumerId = -1, + .newConsumerId = pConsumer->consumerId, }; strcpy(req.cgroup, pConsumer->cgroup); strcpy(req.topicName, pTopic->name); @@ -451,12 +454,13 @@ static int32_t mndProcessSubscribeReq(SMnodeMsg *pMsg) { SMqTopicObj *pTopic = mndAcquireTopic(pMnode, newTopicName); if (pTopic == NULL) { - /*terrno = */ + mError("topic being subscribed not exist: %s", newTopicName); continue; } SMqSubscribeObj *pSub = mndAcquireSubscribe(pMnode, consumerGroup, newTopicName); if (pSub == NULL) { + mDebug("create new subscription, group: %s, topic %s", consumerGroup, newTopicName); pSub = tNewSubscribeObj(); if (pSub == NULL) { terrno = TSDB_CODE_OUT_OF_MEMORY; @@ -464,14 +468,15 @@ static int32_t mndProcessSubscribeReq(SMnodeMsg *pMsg) { } // set unassigned vg mndInitUnassignedVg(pMnode, pTopic, pSub->unassignedVg); + //TODO: disable alter } taosArrayPush(pSub->availConsumer, &consumerId); - // TODO: no need SMqConsumerTopic *pConsumerTopic = tNewConsumerTopic(consumerId, pTopic, pSub); taosArrayPush(pConsumer->topics, pConsumerTopic); if (taosArrayGetSize(pConsumerTopic->pVgInfo) > 0) { + ASSERT(taosArrayGetSize(pConsumerTopic->pVgInfo) == 1); int32_t vgId = *(int32_t *)taosArrayGetLast(pConsumerTopic->pVgInfo); // send setmsg to vnode if (mndBuildMqSetConsumerVgReq(pMnode, pTrans, pConsumer, pConsumerTopic, pTopic) < 0) { @@ -479,8 +484,7 @@ static int32_t mndProcessSubscribeReq(SMnodeMsg *pMsg) { return -1; } } - taosArrayDestroy(pConsumerTopic->pVgInfo); - free(pConsumerTopic); + SSdbRaw *pRaw = mndSubActionEncode(pSub); /*sdbSetRawStatus(pRaw, SDB_STATUS_READY);*/ mndTransAppendRedolog(pTrans, pRaw); @@ -533,12 +537,12 @@ static int32_t mndProcessSubscribeReq(SMnodeMsg *pMsg) { if (mndTransPrepare(pMnode, pTrans) != 0) { mError("trans:%d, failed to prepare since %s", pTrans->id, terrstr()); + if (newSub) taosArrayDestroy(newSub); mndTransDrop(pTrans); mndReleaseConsumer(pMnode, pConsumer); return -1; } - // TODO: free memory if (newSub) taosArrayDestroy(newSub); mndTransDrop(pTrans); mndReleaseConsumer(pMnode, pConsumer); diff --git a/source/dnode/vnode/inc/tq.h b/source/dnode/vnode/inc/tq.h index 9cc987d731..0f318dea0b 100644 --- a/source/dnode/vnode/inc/tq.h +++ b/source/dnode/vnode/inc/tq.h @@ -18,7 +18,6 @@ #include "common.h" #include "executor.h" -#include "vnode.h" #include "mallocator.h" #include "meta.h" #include "os.h" @@ -29,6 +28,7 @@ #include "trpc.h" #include "ttimer.h" #include "tutil.h" +#include "vnode.h" #include "wal.h" #ifdef __cplusplus @@ -149,10 +149,11 @@ typedef struct STqGroup { } STqGroup; typedef struct STqTaskItem { - int8_t status; - int64_t offset; - void* dst; - qTaskInfo_t task; + int8_t status; + int64_t offset; + void* dst; + qTaskInfo_t task; + SSubQueryMsg* pQueryMsg; } STqTaskItem; // new version @@ -164,7 +165,6 @@ typedef struct STqBuffer { typedef struct STqTopicHandle { char topicName[TSDB_TOPIC_FNAME_LEN]; - char cgroup[TSDB_TOPIC_FNAME_LEN]; char* sql; char* logicalPlan; char* physicalPlan; @@ -177,6 +177,7 @@ typedef struct STqTopicHandle { typedef struct STqConsumerHandle { int64_t consumerId; int64_t epoch; + char cgroup[TSDB_TOPIC_FNAME_LEN]; SArray* topics; // SArray } STqConsumerHandle; @@ -318,7 +319,7 @@ int tqSendLaunchQuery(STqMsgItem*, int64_t offset); #endif int32_t tqProcessConsumeReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg** ppRsp); -int32_t tqProcessSetConnReq(STQ* pTq, SMqSetCVgReq* pReq); +int32_t tqProcessSetConnReq(STQ* pTq, char* msg); #ifdef __cplusplus } diff --git a/source/dnode/vnode/src/inc/tqInt.h b/source/dnode/vnode/src/inc/tqInt.h index 2b4200fce5..a9ba825a29 100644 --- a/source/dnode/vnode/src/inc/tqInt.h +++ b/source/dnode/vnode/src/inc/tqInt.h @@ -44,8 +44,10 @@ extern int32_t tqDebugFlag; // delete persistent storage for meta info // int tqDropTCGroup(STQ*, const char* topic, int cgId); -int tqSerializeGroup(const STqGroup*, STqSerializedHead**); -const void* tqDeserializeGroup(const STqSerializedHead* pHead, STqGroup** ppGroup); +//int tqSerializeGroup(const STqGroup*, STqSerializedHead**); +//const void* tqDeserializeGroup(const STqSerializedHead* pHead, STqGroup** ppGroup); +int tqSerializeConsumer(const STqConsumerHandle*, STqSerializedHead**); +const void* tqDeserializeConsumer(const STqSerializedHead* pHead, STqConsumerHandle**); static int FORCE_INLINE tqQueryExecuting(int32_t status) { return status; } #ifdef __cplusplus } diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index b18f50cd3f..49bbb77797 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -68,7 +68,7 @@ STQ* tqOpen(const char* path, SWal* pWal, SMeta* pMeta, STqCfg* tqConfig, SMemAl // TODO: error code of buffer pool } #endif - pTq->tqMeta = tqStoreOpen(path, (FTqSerialize)tqSerializeGroup, (FTqDeserialize)tqDeserializeGroup, free, 0); + pTq->tqMeta = tqStoreOpen(path, (FTqSerialize)tqSerializeConsumer, (FTqDeserialize)tqDeserializeConsumer, free, 0); if (pTq->tqMeta == NULL) { free(pTq); #if 0 @@ -478,6 +478,59 @@ int tqConsume(STQ* pTq, STqConsumeReq* pMsg) { } #endif +int tqSerializeConsumer(const STqConsumerHandle* pConsumer, STqSerializedHead** ppHead) { + int32_t num = taosArrayGetSize(pConsumer->topics); + int32_t sz = sizeof(STqSerializedHead) + sizeof(int64_t) * 2 + TSDB_TOPIC_FNAME_LEN + num * (sizeof(int64_t) + TSDB_TOPIC_FNAME_LEN); + if (sz > (*ppHead)->ssize) { + void* tmpPtr = realloc(*ppHead, sz); + if (tmpPtr == NULL) { + free(*ppHead); + return -1; + } + *ppHead = tmpPtr; + (*ppHead)->ssize = sz; + } + + void* ptr = (*ppHead)->content; + *(int64_t*)ptr = pConsumer->consumerId; + ptr = POINTER_SHIFT(ptr, sizeof(int64_t)); + *(int64_t*)ptr = pConsumer->epoch; + ptr = POINTER_SHIFT(ptr, sizeof(int64_t)); + memcpy(ptr, pConsumer->topics, TSDB_TOPIC_FNAME_LEN); + ptr = POINTER_SHIFT(ptr, TSDB_TOPIC_FNAME_LEN); + *(int32_t*)ptr = num; + ptr = POINTER_SHIFT(ptr, sizeof(int32_t)); + for (int32_t i = 0; i < num; i++) { + STqTopicHandle* pTopic = taosArrayGet(pConsumer->topics, i); + memcpy(ptr, pTopic->topicName, TSDB_TOPIC_FNAME_LEN); + ptr = POINTER_SHIFT(ptr, TSDB_TOPIC_FNAME_LEN); + *(int64_t*)ptr = pTopic->committedOffset; + POINTER_SHIFT(ptr, sizeof(int64_t)); + } + + return 0; +} + +const void* tqDeserializeConsumer(const STqSerializedHead* pHead, STqConsumerHandle** ppConsumer) { + STqConsumerHandle* pConsumer = *ppConsumer; + const void* ptr = pHead->content; + pConsumer->consumerId = *(int64_t*)ptr; + ptr = POINTER_SHIFT(ptr, sizeof(int64_t)); + pConsumer->epoch = *(int64_t*)ptr; + ptr = POINTER_SHIFT(ptr, sizeof(int64_t)); + memcpy(pConsumer->cgroup, ptr, TSDB_TOPIC_FNAME_LEN); + ptr = POINTER_SHIFT(ptr, TSDB_TOPIC_FNAME_LEN); + int32_t sz = *(int32_t*)ptr; + ptr = POINTER_SHIFT(ptr, sizeof(int32_t)); + pConsumer->topics = taosArrayInit(sz, sizeof(STqTopicHandle)); + for (int32_t i = 0; i < sz; i++) { + /*STqTopicHandle* topicHandle = */ + /*taosArrayPush(pConsumer->topics, );*/ + } + return NULL; +} + +#if 0 int tqSerializeGroup(const STqGroup* pGroup, STqSerializedHead** ppHead) { // calculate size int sz = tqGroupSSize(pGroup) + sizeof(STqSerializedHead); @@ -608,6 +661,7 @@ int tqItemSSize() { // mainly for executor return 0; } +#endif int32_t tqProcessConsumeReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg** ppRsp) { SMqConsumeReq* pReq = pMsg->pCont; @@ -625,7 +679,14 @@ int32_t tqProcessConsumeReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg** ppRsp) { for (int i = 0; i < sz; i++) { STqTopicHandle* pTopic = taosArrayGet(pConsumer->topics, i); + //TODO: support multiple topic in one req + if (strcmp(pTopic->topicName, pReq->topic) != 0) { + continue; + } + if (fetchOffset == -1) { + fetchOffset = pTopic->committedOffset + 1; + } int8_t pos; int8_t skip = 0; SWalHead* pHead; @@ -670,6 +731,23 @@ int32_t tqProcessConsumeReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg** ppRsp) { break; } if (pDataBlock != NULL) { + SMqTbData tbData = { + .uid = pDataBlock->info.uid, + .numOfCols = pDataBlock->info.numOfCols, + .numOfRows = pDataBlock->info.rows, + }; + for (int i = 0; i < pDataBlock->info.numOfCols; i++) { + SColumnInfoData* pColData = taosArrayGet(pDataBlock->pDataBlock, i); + int32_t sz = pColData->info.bytes * pDataBlock->info.rows; + SMqColData colData = { + .bytes = pColData->info.bytes, + .colId = pColData->info.colId, + .type = pColData->info.type, + }; + memcpy(colData.data, pColData->pData, colData.bytes * pDataBlock->info.rows); + memcpy(&tbData.colData[i], &colData, sz); + } + /*pDataBlock->info.*/ taosArrayPush(pRes, pDataBlock); } else { break; @@ -692,29 +770,34 @@ int32_t tqProcessConsumeReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg** ppRsp) { pTopic->buffer.lastOffset = pReq->offset; } // put output into rsp + SMqConsumeRsp rsp = { + .consumerId = consumerId, + .numOfTopics = 1 + }; } - // launch query - // get result return 0; } -int32_t tqProcessSetConnReq(STQ* pTq, SMqSetCVgReq* pReq) { +int32_t tqProcessSetConnReq(STQ* pTq, char* msg) { + SMqSetCVgReq req; + tDecodeSMqSetCVgReq(msg, &req); STqConsumerHandle* pConsumer = calloc(sizeof(STqConsumerHandle), 1); if (pConsumer == NULL) { return -1; } + strcpy(pConsumer->cgroup, req.cgroup); + pConsumer->topics = taosArrayInit(0, sizeof(STqTopicHandle)); STqTopicHandle* pTopic = calloc(sizeof(STqTopicHandle), 1); if (pTopic == NULL) { free(pConsumer); return -1; } - strcpy(pTopic->topicName, pReq->topicName); - strcpy(pTopic->cgroup, pReq->cgroup); - strcpy(pTopic->sql, pReq->sql); - strcpy(pTopic->logicalPlan, pReq->logicalPlan); - strcpy(pTopic->physicalPlan, pReq->physicalPlan); + strcpy(pTopic->topicName, req.topicName); + strcpy(pTopic->sql, req.sql); + strcpy(pTopic->logicalPlan, req.logicalPlan); + strcpy(pTopic->physicalPlan, req.physicalPlan); pTopic->buffer.firstOffset = -1; pTopic->buffer.lastOffset = -1; @@ -724,9 +807,9 @@ int32_t tqProcessSetConnReq(STQ* pTq, SMqSetCVgReq* pReq) { for (int i = 0; i < TQ_BUFFER_SIZE; i++) { pTopic->buffer.output[i].status = 0; STqReadHandle* pReadHandle = tqInitSubmitMsgScanner(pTq->pMeta); - pTopic->buffer.output[i].task = qCreateStreamExecTaskInfo(&pReq->msg, pReadHandle); + pTopic->buffer.output[i].task = qCreateStreamExecTaskInfo(&req.msg, pReadHandle); } - // write mq meta + taosArrayPush(pConsumer->topics, pTopic); return 0; } diff --git a/source/dnode/vnode/src/vnd/vnodeWrite.c b/source/dnode/vnode/src/vnd/vnodeWrite.c index a6c3f25c6f..ccddfd56d8 100644 --- a/source/dnode/vnode/src/vnd/vnodeWrite.c +++ b/source/dnode/vnode/src/vnd/vnodeWrite.c @@ -112,9 +112,7 @@ int vnodeApplyWMsg(SVnode *pVnode, SRpcMsg *pMsg, SRpcMsg **pRsp) { } break; case TDMT_VND_MQ_SET_CONN: { - SMqSetCVgReq req; - tDecodeSMqSetCVgReq(ptr, &req); - if (tqProcessSetConnReq(pVnode->pTq, &req) < 0) { + if (tqProcessSetConnReq(pVnode->pTq, ptr) < 0) { } } break; default: