From 7497f193c7639473c78e7174a8b5c1c80da8f173 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 23 Jul 2024 14:47:29 +0800 Subject: [PATCH 01/14] fix(stream): check return value, and do some internal refactor. --- include/libs/executor/storageapi.h | 2 +- source/common/src/tglobal.c | 2 +- source/dnode/mnode/impl/src/mndStream.c | 737 ++++++++++--------- source/dnode/mnode/impl/src/mndStreamHb.c | 62 +- source/dnode/mnode/impl/src/mndStreamTrans.c | 55 +- source/dnode/mnode/impl/src/mndStreamUtil.c | 14 +- source/dnode/vnode/inc/vnode.h | 2 +- source/dnode/vnode/src/tsdb/tsdbCacheRead.c | 205 ++++-- source/dnode/vnode/src/tsdb/tsdbRead2.c | 166 ++++- source/dnode/vnode/src/tsdb/tsdbReadUtil.c | 58 +- source/libs/executor/inc/tsort.h | 18 +- source/libs/executor/src/cachescanoperator.c | 6 +- source/libs/executor/src/mergeoperator.c | 19 +- source/libs/executor/src/scanoperator.c | 15 +- source/libs/executor/src/sortoperator.c | 44 +- source/libs/executor/src/tsort.c | 552 ++++++++++---- source/libs/stream/src/streamCheckStatus.c | 31 +- source/libs/stream/src/streamMeta.c | 2 +- source/libs/stream/src/streamSched.c | 4 +- 19 files changed, 1360 insertions(+), 634 deletions(-) mode change 100755 => 100644 source/libs/executor/src/mergeoperator.c diff --git a/include/libs/executor/storageapi.h b/include/libs/executor/storageapi.h index a746fa8443..05ef14657b 100644 --- a/include/libs/executor/storageapi.h +++ b/include/libs/executor/storageapi.h @@ -195,7 +195,7 @@ typedef struct SStoreCacheReader { int32_t (*openReader)(void *pVnode, int32_t type, void *pTableIdList, int32_t numOfTables, int32_t numOfCols, SArray *pCidList, int32_t *pSlotIds, uint64_t suid, void **pReader, const char *idstr, SArray *pFuncTypeList, SColumnInfo* pPkCol, int32_t numOfPks); - void *(*closeReader)(void *pReader); + void (*closeReader)(void *pReader); int32_t (*retrieveRows)(void *pReader, SSDataBlock *pResBlock, const int32_t *slotIds, const int32_t *dstSlotIds, SArray *pTableUidList); int32_t (*reuseReader)(void *pReader, void *pTableIdList, int32_t numOfTables); diff --git a/source/common/src/tglobal.c b/source/common/src/tglobal.c index 88b743d905..f126b2d6c9 100644 --- a/source/common/src/tglobal.c +++ b/source/common/src/tglobal.c @@ -756,7 +756,7 @@ static int32_t taosAddServerCfg(SConfig *pCfg) { if (cfgAddInt64(pCfg, "streamBufferSize", tsStreamBufferSize, 0, INT64_MAX, CFG_SCOPE_SERVER, CFG_DYN_NONE) != 0) return -1; if (cfgAddInt64(pCfg, "streamAggCnt", tsStreamAggCnt, 2, INT32_MAX, CFG_SCOPE_SERVER, CFG_DYN_NONE) != 0) return -1; - if (cfgAddInt32(pCfg, "checkpointInterval", tsStreamCheckpointInterval, 60, 1200, CFG_SCOPE_SERVER, CFG_DYN_ENT_SERVER) != 0) return -1; + if (cfgAddInt32(pCfg, "checkpointInterval", tsStreamCheckpointInterval, 60, 1800, CFG_SCOPE_SERVER, CFG_DYN_ENT_SERVER) != 0) return -1; if (cfgAddFloat(pCfg, "streamSinkDataRate", tsSinkDataRate, 0.1, 5, CFG_SCOPE_SERVER, CFG_DYN_NONE) != 0) return -1; if (cfgAddInt32(pCfg, "concurrentCheckpoint", tsMaxConcurrentCheckpoint, 1, 10, CFG_SCOPE_SERVER, CFG_DYN_ENT_SERVER) != 0) return -1; diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index df8800aee4..31d28d99a4 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -62,6 +62,9 @@ static int32_t mndProcessCheckpointReport(SRpcMsg *pReq); static int32_t mndProcessConsensusInTmr(SRpcMsg *pMsg); static void doSendQuickRsp(SRpcHandleInfo *pInfo, int32_t msgSize, int32_t vgId, int32_t code); + +static int32_t setStreamAttrInResBlock(SStreamObj *pStream, SSDataBlock *pBlock, int32_t numOfRows); + static SVgroupChangeInfo mndFindChangedNodeInfo(SMnode *pMnode, const SArray *pPrevNodeList, const SArray *pNodeList); static void addAllStreamTasksIntoBuf(SMnode *pMnode, SStreamExecInfo *pExecInfo); @@ -156,7 +159,7 @@ void mndCleanupStream(SMnode *pMnode) { taosHashCleanup(execInfo.pTransferStateStreams); taosHashCleanup(execInfo.pChkptStreams); taosHashCleanup(execInfo.pStreamConsensus); - taosThreadMutexDestroy(&execInfo.lock); + (void) taosThreadMutexDestroy(&execInfo.lock); mDebug("mnd stream exec info cleanup"); } @@ -238,8 +241,7 @@ static int32_t mndStreamActionDelete(SSdb *pSdb, SStreamObj *pStream) { static int32_t mndStreamActionUpdate(SSdb *pSdb, SStreamObj *pOldStream, SStreamObj *pNewStream) { mTrace("stream:%s, perform update action", pOldStream->name); - - atomic_exchange_32(&pOldStream->version, pNewStream->version); + (void) atomic_exchange_32(&pOldStream->version, pNewStream->version); taosWLockLatch(&pOldStream->lock); @@ -303,8 +305,7 @@ static void mndShowStreamTrigger(char *dst, SStreamObj *pStream) { static int32_t mndCheckCreateStreamReq(SCMCreateStreamReq *pCreate) { if (pCreate->name[0] == 0 || pCreate->sql == NULL || pCreate->sql[0] == 0 || pCreate->sourceDB[0] == 0 || pCreate->targetStbFullName[0] == 0) { - terrno = TSDB_CODE_MND_INVALID_STREAM_OPTION; - return -1; + return terrno = TSDB_CODE_MND_INVALID_STREAM_OPTION; } return 0; } @@ -383,7 +384,7 @@ static int32_t mndBuildStreamObjFromCreateReq(SMnode *pMnode, SStreamObj *pObj, SDbObj *pSourceDb = mndAcquireDb(pMnode, pCreate->sourceDB); if (pSourceDb == NULL) { mInfo("stream:%s failed to create, source db %s not exist since %s", pCreate->name, pObj->sourceDb, terrstr()); - return -1; + return terrno; } pObj->sourceDbUid = pSourceDb->uid; mndReleaseDb(pMnode, pSourceDb); @@ -393,7 +394,7 @@ static int32_t mndBuildStreamObjFromCreateReq(SMnode *pMnode, SStreamObj *pObj, SDbObj *pTargetDb = mndAcquireDbByStb(pMnode, pObj->targetSTbName); if (pTargetDb == NULL) { mInfo("stream:%s failed to create, target db %s not exist since %s", pCreate->name, pObj->targetDb, terrstr()); - return -1; + return terrno; } tstrncpy(pObj->targetDb, pTargetDb->name, TSDB_DB_FNAME_LEN); @@ -503,7 +504,11 @@ int32_t mndPersistTaskDeployReq(STrans *pTrans, SStreamTask *pTask) { if (pTask->ver < SSTREAM_TASK_SUBTABLE_CHANGED_VER) { pTask->ver = SSTREAM_TASK_VER; } - tEncodeStreamTask(&encoder, pTask); + + int32_t code = tEncodeStreamTask(&encoder, pTask); + if (code == -1) { + return TSDB_CODE_INVALID_MSG; + } int32_t size = encoder.pos; int32_t tlen = sizeof(SMsgHead) + size; @@ -511,25 +516,29 @@ int32_t mndPersistTaskDeployReq(STrans *pTrans, SStreamTask *pTask) { void *buf = taosMemoryCalloc(1, tlen); if (buf == NULL) { - terrno = TSDB_CODE_OUT_OF_MEMORY; - return -1; + return TSDB_CODE_OUT_OF_MEMORY; } ((SMsgHead *)buf)->vgId = htonl(pTask->info.nodeId); void *abuf = POINTER_SHIFT(buf, sizeof(SMsgHead)); tEncoderInit(&encoder, abuf, size); - - tEncodeStreamTask(&encoder, pTask); + code = tEncodeStreamTask(&encoder, pTask); tEncoderClear(&encoder); - int32_t code = setTransAction(pTrans, buf, tlen, TDMT_STREAM_TASK_DEPLOY, &pTask->info.epSet, 0, 0); - if (code != 0) { + if (code == -1) { + code = TSDB_CODE_INVALID_PARA; + mError("failed to encode stream task, code:%s", tstrerror(code)); taosMemoryFree(buf); - return -1; + return code; } - return 0; + code = setTransAction(pTrans, buf, tlen, TDMT_STREAM_TASK_DEPLOY, &pTask->info.epSet, 0, 0); + if (code) { + taosMemoryFree(buf); + } + + return code; } int32_t mndPersistStreamTasks(STrans *pTrans, SStreamObj *pStream) { @@ -548,9 +557,10 @@ int32_t mndPersistStreamTasks(STrans *pTrans, SStreamObj *pStream) { return code; } - if (mndPersistTaskDeployReq(pTrans, pTask) < 0) { + code = mndPersistTaskDeployReq(pTrans, pTask); + if (code) { destroyStreamTaskIter(pIter); - return -1; + return code; } } @@ -566,19 +576,21 @@ int32_t mndPersistStreamTasks(STrans *pTrans, SStreamObj *pStream) { int32_t numOfTasks = taosArrayGetSize(pLevel); for (int32_t j = 0; j < numOfTasks; j++) { SStreamTask *pTask = taosArrayGetP(pLevel, j); - if (mndPersistTaskDeployReq(pTrans, pTask) < 0) { - return -1; + code = mndPersistTaskDeployReq(pTrans, pTask); + if (code) { + return code; } } } } - return 0; + return code; } int32_t mndPersistStream(STrans *pTrans, SStreamObj *pStream) { - if (mndPersistStreamTasks(pTrans, pStream) < 0) { - return -1; + int32_t code = 0; + if ((code = mndPersistStreamTasks(pTrans, pStream)) < 0) { + return code; } return mndPersistTransLog(pStream, pTrans, SDB_STATUS_READY); @@ -726,7 +738,7 @@ static int32_t mndProcessCreateStreamReq(SRpcMsg *pReq) { SCMCreateStreamReq createReq = {0}; if (tDeserializeSCMCreateStreamReq(pReq->pCont, pReq->contLen, &createReq) != 0) { - terrno = TSDB_CODE_INVALID_MSG; + code = TSDB_CODE_INVALID_MSG; goto _OVER; } @@ -736,7 +748,7 @@ static int32_t mndProcessCreateStreamReq(SRpcMsg *pReq) { #endif mInfo("stream:%s, start to create stream, sql:%s", createReq.name, createReq.sql); - if (mndCheckCreateStreamReq(&createReq) != 0) { + if ((code = mndCheckCreateStreamReq(&createReq)) != 0) { mError("stream:%s, failed to create since %s", createReq.name, terrstr()); goto _OVER; } @@ -766,12 +778,12 @@ static int32_t mndProcessCreateStreamReq(SRpcMsg *pReq) { } // build stream obj from request - if (mndBuildStreamObjFromCreateReq(pMnode, &streamObj, &createReq) < 0) { + if ((code = mndBuildStreamObjFromCreateReq(pMnode, &streamObj, &createReq)) < 0) { mError("stream:%s, failed to create since %s", createReq.name, terrstr()); goto _OVER; } - if (doStreamCheck(pMnode, &streamObj) < 0) { + if ((code = doStreamCheck(pMnode, &streamObj)) < 0) { goto _OVER; } @@ -783,7 +795,7 @@ static int32_t mndProcessCreateStreamReq(SRpcMsg *pReq) { // create stb for stream if (createReq.createStb == STREAM_CREATE_STABLE_TRUE) { - if (mndCreateStbForStream(pMnode, pTrans, &streamObj, pReq->info.conn.user) < 0) { + if ((code = mndCreateStbForStream(pMnode, pTrans, &streamObj, pReq->info.conn.user)) < 0) { mError("trans:%d, failed to create stb for stream %s since %s", pTrans->id, createReq.name, terrstr()); mndTransDrop(pTrans); goto _OVER; @@ -793,25 +805,25 @@ static int32_t mndProcessCreateStreamReq(SRpcMsg *pReq) { } // schedule stream task for stream obj - if (mndScheduleStream(pMnode, &streamObj, createReq.lastTs, createReq.pVgroupVerList) < 0) { + if ((code = mndScheduleStream(pMnode, &streamObj, createReq.lastTs, createReq.pVgroupVerList)) < 0) { mError("stream:%s, failed to schedule since %s", createReq.name, terrstr()); mndTransDrop(pTrans); goto _OVER; } // add stream to trans - if (mndPersistStream(pTrans, &streamObj) < 0) { - mError("stream:%s, failed to schedule since %s", createReq.name, terrstr()); + if ((code = mndPersistStream(pTrans, &streamObj)) < 0) { + mError("stream:%s, failed to persist since %s", createReq.name, terrstr()); mndTransDrop(pTrans); goto _OVER; } - if (mndCheckDbPrivilegeByName(pMnode, pReq->info.conn.user, MND_OPER_READ_DB, streamObj.sourceDb) != 0) { + if ((code = mndCheckDbPrivilegeByName(pMnode, pReq->info.conn.user, MND_OPER_READ_DB, streamObj.sourceDb)) != 0) { mndTransDrop(pTrans); goto _OVER; } - if (mndCheckDbPrivilegeByName(pMnode, pReq->info.conn.user, MND_OPER_WRITE_DB, streamObj.targetDb) != 0) { + if ((code = mndCheckDbPrivilegeByName(pMnode, pReq->info.conn.user, MND_OPER_WRITE_DB, streamObj.targetDb)) != 0) { mndTransDrop(pTrans); goto _OVER; } @@ -824,7 +836,7 @@ static int32_t mndProcessCreateStreamReq(SRpcMsg *pReq) { streamMutexUnlock(&execInfo.lock); // execute creation - if (mndTransPrepare(pMnode, pTrans) != 0) { + if ((code = mndTransPrepare(pMnode, pTrans)) != 0) { mError("trans:%d, failed to prepare since %s", pTrans->id, terrstr()); mndTransDrop(pTrans); goto _OVER; @@ -833,12 +845,18 @@ static int32_t mndProcessCreateStreamReq(SRpcMsg *pReq) { mndTransDrop(pTrans); SName dbname = {0}; - tNameFromString(&dbname, createReq.sourceDB, T_NAME_ACCT | T_NAME_DB | T_NAME_TABLE); + code = tNameFromString(&dbname, createReq.sourceDB, T_NAME_ACCT | T_NAME_DB | T_NAME_TABLE); + if (code) { + goto _OVER; + } SName name = {0}; - tNameFromString(&name, createReq.name, T_NAME_ACCT | T_NAME_DB | T_NAME_TABLE); - // reuse this function for stream + code = tNameFromString(&name, createReq.name, T_NAME_ACCT | T_NAME_DB | T_NAME_TABLE); + if (code) { + goto _OVER; + } + // reuse this function for stream if (sql != NULL && sqlLen > 0) { auditRecord(pReq, pMnode->clusterId, "createStream", dbname.dbname, name.dbname, sql, sqlLen); } else { @@ -848,7 +866,7 @@ static int32_t mndProcessCreateStreamReq(SRpcMsg *pReq) { } _OVER: - if (terrno != TSDB_CODE_SUCCESS && terrno != TSDB_CODE_ACTION_IN_PROGRESS) { + if (code != TSDB_CODE_SUCCESS && code != TSDB_CODE_ACTION_IN_PROGRESS) { mError("stream:%s, failed to create since %s", createReq.name, terrstr()); } @@ -860,7 +878,7 @@ _OVER: taosMemoryFreeClear(sql); } - return terrno; + return code; } int64_t mndStreamGenChkptId(SMnode *pMnode, bool lock) { @@ -948,7 +966,10 @@ static int32_t mndBuildStreamCheckpointSourceReq(void **pBuf, int32_t *pLen, int void *abuf = POINTER_SHIFT(buf, sizeof(SMsgHead)); SEncoder encoder; tEncoderInit(&encoder, abuf, tlen); - tEncodeStreamCheckpointSourceReq(&encoder, &req); + int32_t pos = tEncodeStreamCheckpointSourceReq(&encoder, &req); + if (pos == -1) { + return TSDB_CODE_INVALID_MSG; + } SMsgHead *pMsgHead = (SMsgHead *)buf; pMsgHead->contLen = htonl(tlen); @@ -1001,7 +1022,7 @@ static int32_t mndProcessStreamCheckpointTrans(SMnode *pMnode, SStreamObj *pStre if (conflict) { mWarn("checkpoint conflict with other trans in %s, ignore the checkpoint for stream:%s %" PRIx64, pStream->sourceDb, pStream->name, pStream->uid); - return -1; + return TSDB_CODE_MND_TRANS_CONFLICT; } STrans *pTrans = NULL; @@ -1014,7 +1035,12 @@ static int32_t mndProcessStreamCheckpointTrans(SMnode *pMnode, SStreamObj *pStre goto _ERR; } - mndStreamRegisterTrans(pTrans, MND_STREAM_CHECKPOINT_NAME, pStream->uid); + code = mndStreamRegisterTrans(pTrans, MND_STREAM_CHECKPOINT_NAME, pStream->uid); + if (code) { + mError("failed to register checkpoint trans for stream:%s, checkpointId:%" PRId64, pStream->name, checkpointId); + goto _ERR; + } + mDebug("start to trigger checkpoint for stream:%s, checkpoint: %" PRId64 "", pStream->name, checkpointId); taosWLockLatch(&pStream->lock); @@ -1054,6 +1080,7 @@ static int32_t mndProcessStreamCheckpointTrans(SMnode *pMnode, SStreamObj *pStre } if ((code = mndTransPrepare(pMnode, pTrans)) != TSDB_CODE_SUCCESS) { + code = terrno; mError("failed to prepare checkpoint trans since %s", terrstr()); goto _ERR; } @@ -1066,7 +1093,11 @@ _ERR: int32_t extractStreamNodeList(SMnode *pMnode) { if (taosArrayGetSize(execInfo.pNodeList) == 0) { - extractNodeListFromStream(pMnode, execInfo.pNodeList); + int32_t code = extractNodeListFromStream(pMnode, execInfo.pNodeList); + if (code) { + mError("Failed to extract node list from stream, code:%s", tstrerror(code)); + return code; + } } return taosArrayGetSize(execInfo.pNodeList); @@ -1148,7 +1179,10 @@ static int32_t mndCheckTaskAndNodeStatus(SMnode *pMnode) { for (int32_t j = 0; j < taosArrayGetSize(pInvalidList); ++j) { STaskId *pId = taosArrayGet(pInvalidList, j); if (pEntry->id.streamId == pId->streamId) { - taosArrayPush(pInvalidList, &pEntry->id); + void* px = taosArrayPush(pInvalidList, &pEntry->id); + if (px == NULL) { + mError("failed to put stream into invalid list, code:%s", tstrerror(TSDB_CODE_OUT_OF_MEMORY)); + } break; } } @@ -1247,12 +1281,15 @@ static int32_t mndProcessStreamCheckpoint(SRpcMsg *pReq) { streamMutexUnlock(&execInfo.lock); SCheckpointInterval in = {.streamId = pStream->uid, .duration = duration}; - taosArrayPush(pList, &in); - - int32_t currentSize = taosArrayGetSize(pList); - mDebug("stream:%s (uid:0x%" PRIx64 ") checkpoint interval beyond threshold: %ds(%" PRId64 "s) beyond threshold:%d", - pStream->name, pStream->uid, tsStreamCheckpointInterval, duration / 1000, currentSize); - + void* p = taosArrayPush(pList, &in); + if (p) { + int32_t currentSize = taosArrayGetSize(pList); + mDebug("stream:%s (uid:0x%" PRIx64 ") checkpoint interval beyond threshold: %ds(%" PRId64 + "s) beyond concurrently launch threshold:%d", + pStream->name, pStream->uid, tsStreamCheckpointInterval, duration / 1000, currentSize); + } else { + mError("failed to record the checkpoint interval info, stream:0x%" PRIx64, pStream->uid); + } sdbRelease(pSdb, pStream); } @@ -1263,9 +1300,14 @@ static int32_t mndProcessStreamCheckpoint(SRpcMsg *pReq) { } taosArraySort(pList, streamWaitComparFn); - mndStreamClearFinishedTrans(pMnode, &numOfCheckpointTrans); - int32_t numOfQual = taosArrayGetSize(pList); + code = mndStreamClearFinishedTrans(pMnode, &numOfCheckpointTrans); + if (code) { + mError("failed to clear finish trans, code:%s", tstrerror(code)); + taosArrayDestroy(pList); + return code; + } + int32_t numOfQual = taosArrayGetSize(pList); if (numOfCheckpointTrans > tsMaxConcurrentCheckpoint) { mDebug( "%d stream(s) checkpoint interval longer than %ds, ongoing checkpoint trans:%d reach maximum allowed:%d, new " @@ -1310,9 +1352,9 @@ static int32_t mndProcessStreamCheckpoint(SRpcMsg *pReq) { } static int32_t mndProcessDropStreamReq(SRpcMsg *pReq) { - SMnode *pMnode = pReq->info.node; + SMnode * pMnode = pReq->info.node; SStreamObj *pStream = NULL; - int32_t code = 0; + int32_t code = 0; SMDropStreamReq dropReq = {0}; if (tDeserializeSMDropStreamReq(pReq->pCont, pReq->contLen, &dropReq) < 0) { @@ -1341,7 +1383,7 @@ static int32_t mndProcessDropStreamReq(SRpcMsg *pReq) { if (pStream->smaId != 0) { mDebug("stream:%s, uid:0x%" PRIx64 " try to drop sma related stream", dropReq.name, pStream->uid); - void *pIter = NULL; + void * pIter = NULL; SSmaObj *pSma = NULL; pIter = sdbFetch(pMnode->pSdb, SDB_SMA, pIter, (void **)&pSma); while (pIter) { @@ -1429,15 +1471,18 @@ static int32_t mndProcessDropStreamReq(SRpcMsg *pReq) { removeStreamTasksInBuf(pStream, &execInfo); SName name = {0}; - tNameFromString(&name, dropReq.name, T_NAME_ACCT | T_NAME_DB | T_NAME_TABLE); - + code = tNameFromString(&name, dropReq.name, T_NAME_ACCT | T_NAME_DB | T_NAME_TABLE); auditRecord(pReq, pMnode->clusterId, "dropStream", "", name.dbname, dropReq.sql, dropReq.sqlLen); sdbRelease(pMnode->pSdb, pStream); mndTransDrop(pTrans); tFreeMDropStreamReq(&dropReq); - return TSDB_CODE_ACTION_IN_PROGRESS; + if (code == 0) { + return TSDB_CODE_ACTION_IN_PROGRESS; + } else { + return code; + } } int32_t mndDropStreamByDb(SMnode *pMnode, STrans *pTrans, SDbObj *pDb) { @@ -1532,113 +1577,16 @@ static int32_t mndRetrieveStream(SRpcMsg *pReq, SShowObj *pShow, SSDataBlock *pB SSdb *pSdb = pMnode->pSdb; int32_t numOfRows = 0; SStreamObj *pStream = NULL; + int32_t code = 0; while (numOfRows < rows) { pShow->pIter = sdbFetch(pSdb, SDB_STREAM, pShow->pIter, (void **)&pStream); if (pShow->pIter == NULL) break; - SColumnInfoData *pColInfo; - int32_t cols = 0; - - char streamName[TSDB_TABLE_NAME_LEN + VARSTR_HEADER_SIZE] = {0}; - STR_WITH_MAXSIZE_TO_VARSTR(streamName, mndGetDbStr(pStream->name), sizeof(streamName)); - pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); - colDataSetVal(pColInfo, numOfRows, (const char *)streamName, false); - - // create time - pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); - colDataSetVal(pColInfo, numOfRows, (const char *)&pStream->createTime, false); - - // stream id - char buf[128] = {0}; - int64ToHexStr(pStream->uid, buf, tListLen(buf)); - pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); - colDataSetVal(pColInfo, numOfRows, buf, false); - - // related fill-history stream id - pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); - if (pStream->hTaskUid != 0) { - int64ToHexStr(pStream->hTaskUid, buf, tListLen(buf)); - colDataSetVal(pColInfo, numOfRows, buf, false); - } else { - colDataSetVal(pColInfo, numOfRows, buf, true); + code = setStreamAttrInResBlock(pStream, pBlock, numOfRows); + if (code == 0) { + numOfRows++; } - - // related fill-history stream id - char sql[TSDB_SHOW_SQL_LEN + VARSTR_HEADER_SIZE] = {0}; - STR_WITH_MAXSIZE_TO_VARSTR(sql, pStream->sql, sizeof(sql)); - pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); - colDataSetVal(pColInfo, numOfRows, (const char *)sql, false); - - char status[20 + VARSTR_HEADER_SIZE] = {0}; - char status2[20] = {0}; - mndShowStreamStatus(status2, pStream); - STR_WITH_MAXSIZE_TO_VARSTR(status, status2, sizeof(status)); - pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); - colDataSetVal(pColInfo, numOfRows, (const char *)&status, false); - - char sourceDB[TSDB_DB_NAME_LEN + VARSTR_HEADER_SIZE] = {0}; - STR_WITH_MAXSIZE_TO_VARSTR(sourceDB, mndGetDbStr(pStream->sourceDb), sizeof(sourceDB)); - pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); - colDataSetVal(pColInfo, numOfRows, (const char *)&sourceDB, false); - - char targetDB[TSDB_DB_NAME_LEN + VARSTR_HEADER_SIZE] = {0}; - STR_WITH_MAXSIZE_TO_VARSTR(targetDB, mndGetDbStr(pStream->targetDb), sizeof(targetDB)); - pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); - colDataSetVal(pColInfo, numOfRows, (const char *)&targetDB, false); - - if (pStream->targetSTbName[0] == 0) { - pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); - colDataSetVal(pColInfo, numOfRows, NULL, true); - } else { - char targetSTB[TSDB_TABLE_NAME_LEN + VARSTR_HEADER_SIZE] = {0}; - STR_WITH_MAXSIZE_TO_VARSTR(targetSTB, mndGetStbStr(pStream->targetSTbName), sizeof(targetSTB)); - pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); - colDataSetVal(pColInfo, numOfRows, (const char *)&targetSTB, false); - } - - pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); - colDataSetVal(pColInfo, numOfRows, (const char *)&pStream->conf.watermark, false); - - char trigger[20 + VARSTR_HEADER_SIZE] = {0}; - char trigger2[20] = {0}; - mndShowStreamTrigger(trigger2, pStream); - STR_WITH_MAXSIZE_TO_VARSTR(trigger, trigger2, sizeof(trigger)); - pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); - colDataSetVal(pColInfo, numOfRows, (const char *)&trigger, false); - - // sink_quota - char sinkQuota[20 + VARSTR_HEADER_SIZE] = {0}; - sinkQuota[0] = '0'; - char dstStr[20] = {0}; - STR_TO_VARSTR(dstStr, sinkQuota) - pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); - colDataSetVal(pColInfo, numOfRows, (const char *)dstStr, false); - - // checkpoint interval - char tmp[20 + VARSTR_HEADER_SIZE] = {0}; - sprintf(varDataVal(tmp), "%d sec", tsStreamCheckpointInterval); - varDataSetLen(tmp, strlen(varDataVal(tmp))); - - pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); - colDataSetVal(pColInfo, numOfRows, (const char *)tmp, false); - - // checkpoint backup type - char backup[20 + VARSTR_HEADER_SIZE] = {0}; - STR_TO_VARSTR(backup, "none") - pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); - colDataSetVal(pColInfo, numOfRows, (const char *)backup, false); - - // history scan idle - char scanHistoryIdle[20 + VARSTR_HEADER_SIZE] = {0}; - strcpy(scanHistoryIdle, "100a"); - - memset(dstStr, 0, tListLen(dstStr)); - STR_TO_VARSTR(dstStr, scanHistoryIdle) - pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); - colDataSetVal(pColInfo, numOfRows, (const char *)dstStr, false); - - numOfRows++; sdbRelease(pSdb, pStream); } @@ -1651,9 +1599,133 @@ static void mndCancelGetNextStream(SMnode *pMnode, void *pIter) { sdbCancelFetch(pSdb, pIter); } +int32_t setStreamAttrInResBlock(SStreamObj *pStream, SSDataBlock *pBlock, int32_t numOfRows) { + int32_t code = 0; + int32_t cols = 0; + int32_t lino = 0; + + char streamName[TSDB_TABLE_NAME_LEN + VARSTR_HEADER_SIZE] = {0}; + STR_WITH_MAXSIZE_TO_VARSTR(streamName, mndGetDbStr(pStream->name), sizeof(streamName)); + SColumnInfoData *pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); + + code = colDataSetVal(pColInfo, numOfRows, (const char *)streamName, false); + TSDB_CHECK_CODE(code, lino, _end); + + // create time + pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); + code = colDataSetVal(pColInfo, numOfRows, (const char *)&pStream->createTime, false); + TSDB_CHECK_CODE(code, lino, _end); + + // stream id + char buf[128] = {0}; + int64ToHexStr(pStream->uid, buf, tListLen(buf)); + pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); + code = colDataSetVal(pColInfo, numOfRows, buf, false); + TSDB_CHECK_CODE(code, lino, _end); + + // related fill-history stream id + pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); + if (pStream->hTaskUid != 0) { + int64ToHexStr(pStream->hTaskUid, buf, tListLen(buf)); + code = colDataSetVal(pColInfo, numOfRows, buf, false); + } else { + code = colDataSetVal(pColInfo, numOfRows, buf, true); + } + TSDB_CHECK_CODE(code, lino, _end); + + // related fill-history stream id + char sql[TSDB_SHOW_SQL_LEN + VARSTR_HEADER_SIZE] = {0}; + STR_WITH_MAXSIZE_TO_VARSTR(sql, pStream->sql, sizeof(sql)); + pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); + code = colDataSetVal(pColInfo, numOfRows, (const char *)sql, false); + TSDB_CHECK_CODE(code, lino, _end); + + char status[20 + VARSTR_HEADER_SIZE] = {0}; + char status2[20] = {0}; + mndShowStreamStatus(status2, pStream); + STR_WITH_MAXSIZE_TO_VARSTR(status, status2, sizeof(status)); + pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); + code = colDataSetVal(pColInfo, numOfRows, (const char *)&status, false); + TSDB_CHECK_CODE(code, lino, _end); + + char sourceDB[TSDB_DB_NAME_LEN + VARSTR_HEADER_SIZE] = {0}; + STR_WITH_MAXSIZE_TO_VARSTR(sourceDB, mndGetDbStr(pStream->sourceDb), sizeof(sourceDB)); + pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); + code = colDataSetVal(pColInfo, numOfRows, (const char *)&sourceDB, false); + TSDB_CHECK_CODE(code, lino, _end); + + char targetDB[TSDB_DB_NAME_LEN + VARSTR_HEADER_SIZE] = {0}; + STR_WITH_MAXSIZE_TO_VARSTR(targetDB, mndGetDbStr(pStream->targetDb), sizeof(targetDB)); + pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); + code = colDataSetVal(pColInfo, numOfRows, (const char *)&targetDB, false); + TSDB_CHECK_CODE(code, lino, _end); + + if (pStream->targetSTbName[0] == 0) { + pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); + code = colDataSetVal(pColInfo, numOfRows, NULL, true); + } else { + char targetSTB[TSDB_TABLE_NAME_LEN + VARSTR_HEADER_SIZE] = {0}; + STR_WITH_MAXSIZE_TO_VARSTR(targetSTB, mndGetStbStr(pStream->targetSTbName), sizeof(targetSTB)); + pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); + code = colDataSetVal(pColInfo, numOfRows, (const char *)&targetSTB, false); + } + TSDB_CHECK_CODE(code, lino, _end); + + pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); + code = colDataSetVal(pColInfo, numOfRows, (const char *)&pStream->conf.watermark, false); + TSDB_CHECK_CODE(code, lino, _end); + + char trigger[20 + VARSTR_HEADER_SIZE] = {0}; + char trigger2[20] = {0}; + mndShowStreamTrigger(trigger2, pStream); + STR_WITH_MAXSIZE_TO_VARSTR(trigger, trigger2, sizeof(trigger)); + pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); + code = colDataSetVal(pColInfo, numOfRows, (const char *)&trigger, false); + TSDB_CHECK_CODE(code, lino, _end); + + // sink_quota + char sinkQuota[20 + VARSTR_HEADER_SIZE] = {0}; + sinkQuota[0] = '0'; + char dstStr[20] = {0}; + STR_TO_VARSTR(dstStr, sinkQuota) + pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); + code = colDataSetVal(pColInfo, numOfRows, (const char *)dstStr, false); + TSDB_CHECK_CODE(code, lino, _end); + + // checkpoint interval + char tmp[20 + VARSTR_HEADER_SIZE] = {0}; + sprintf(varDataVal(tmp), "%d sec", tsStreamCheckpointInterval); + varDataSetLen(tmp, strlen(varDataVal(tmp))); + + pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); + code = colDataSetVal(pColInfo, numOfRows, (const char *)tmp, false); + TSDB_CHECK_CODE(code, lino, _end); + + // checkpoint backup type + char backup[20 + VARSTR_HEADER_SIZE] = {0}; + STR_TO_VARSTR(backup, "none") + pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); + code = colDataSetVal(pColInfo, numOfRows, (const char *)backup, false); + TSDB_CHECK_CODE(code, lino, _end); + + // history scan idle + char scanHistoryIdle[20 + VARSTR_HEADER_SIZE] = {0}; + strcpy(scanHistoryIdle, "100a"); + + memset(dstStr, 0, tListLen(dstStr)); + STR_TO_VARSTR(dstStr, scanHistoryIdle) + pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); + code = colDataSetVal(pColInfo, numOfRows, (const char *)dstStr, false); + + _end: + return code; +} + static int32_t setTaskAttrInResBlock(SStreamObj *pStream, SStreamTask *pTask, SSDataBlock *pBlock, int32_t numOfRows) { SColumnInfoData *pColInfo; int32_t cols = 0; + int32_t code = 0; + int32_t lino = 0; STaskId id = {.streamId = pTask->id.streamId, .taskId = pTask->id.taskId}; @@ -1662,7 +1734,7 @@ static int32_t setTaskAttrInResBlock(SStreamObj *pStream, SStreamTask *pTask, SS mError("task:0x%" PRIx64 " not exists in any vnodes, streamName:%s, streamId:0x%" PRIx64 " createTs:%" PRId64 " no valid status/stage info", id.taskId, pStream->name, pStream->uid, pStream->createTime); - return -1; + return TSDB_CODE_STREAM_TASK_NOT_EXIST; } // stream name @@ -1670,14 +1742,16 @@ static int32_t setTaskAttrInResBlock(SStreamObj *pStream, SStreamTask *pTask, SS STR_WITH_MAXSIZE_TO_VARSTR(streamName, mndGetDbStr(pStream->name), sizeof(streamName)); pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); - colDataSetVal(pColInfo, numOfRows, (const char *)streamName, false); + code = colDataSetVal(pColInfo, numOfRows, (const char *)streamName, false); + TSDB_CHECK_CODE(code, lino, _end); // task id pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); char idstr[128] = {0}; int64ToHexStr(pTask->id.taskId, idstr, tListLen(idstr)); - colDataSetVal(pColInfo, numOfRows, idstr, false); + code = colDataSetVal(pColInfo, numOfRows, idstr, false); + TSDB_CHECK_CODE(code, lino, _end); // node type char nodeType[20 + VARSTR_HEADER_SIZE] = {0}; @@ -1688,12 +1762,14 @@ static int32_t setTaskAttrInResBlock(SStreamObj *pStream, SStreamTask *pTask, SS } else { memcpy(varDataVal(nodeType), "snode", 5); } - colDataSetVal(pColInfo, numOfRows, nodeType, false); + code = colDataSetVal(pColInfo, numOfRows, nodeType, false); + TSDB_CHECK_CODE(code, lino, _end); // node id pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); int64_t nodeId = TMAX(pTask->info.nodeId, 0); - colDataSetVal(pColInfo, numOfRows, (const char *)&nodeId, false); + code = colDataSetVal(pColInfo, numOfRows, (const char *)&nodeId, false); + TSDB_CHECK_CODE(code, lino, _end); // level char level[20 + VARSTR_HEADER_SIZE] = {0}; @@ -1709,7 +1785,8 @@ static int32_t setTaskAttrInResBlock(SStreamObj *pStream, SStreamTask *pTask, SS } pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); - colDataSetVal(pColInfo, numOfRows, (const char *)level, false); + code = colDataSetVal(pColInfo, numOfRows, (const char *)level, false); + TSDB_CHECK_CODE(code, lino, _end); // status char status[20 + VARSTR_HEADER_SIZE] = {0}; @@ -1719,11 +1796,13 @@ static int32_t setTaskAttrInResBlock(SStreamObj *pStream, SStreamTask *pTask, SS // status pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); - colDataSetVal(pColInfo, numOfRows, (const char *)status, false); + code = colDataSetVal(pColInfo, numOfRows, (const char *)status, false); + TSDB_CHECK_CODE(code, lino, _end); // stage pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); - colDataSetVal(pColInfo, numOfRows, (const char *)&pe->stage, false); + code = colDataSetVal(pColInfo, numOfRows, (const char *)&pe->stage, false); + TSDB_CHECK_CODE(code, lino, _end); // input queue char vbuf[40] = {0}; @@ -1733,7 +1812,8 @@ static int32_t setTaskAttrInResBlock(SStreamObj *pStream, SStreamTask *pTask, SS STR_TO_VARSTR(vbuf, buf); pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); - colDataSetVal(pColInfo, numOfRows, (const char *)vbuf, false); + code = colDataSetVal(pColInfo, numOfRows, (const char *)vbuf, false); + TSDB_CHECK_CODE(code, lino, _end); // input total const char *formatTotalMb = "%7.2f MiB"; @@ -1748,7 +1828,8 @@ static int32_t setTaskAttrInResBlock(SStreamObj *pStream, SStreamTask *pTask, SS STR_TO_VARSTR(vbuf, buf); pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); - colDataSetVal(pColInfo, numOfRows, (const char *)vbuf, false); + code = colDataSetVal(pColInfo, numOfRows, (const char *)vbuf, false); + TSDB_CHECK_CODE(code, lino, _end); // process throughput const char *formatKb = "%7.2f KiB/s"; @@ -1763,7 +1844,8 @@ static int32_t setTaskAttrInResBlock(SStreamObj *pStream, SStreamTask *pTask, SS STR_TO_VARSTR(vbuf, buf); pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); - colDataSetVal(pColInfo, numOfRows, (const char *)vbuf, false); + code = colDataSetVal(pColInfo, numOfRows, (const char *)vbuf, false); + TSDB_CHECK_CODE(code, lino, _end); // output total pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); @@ -1775,7 +1857,8 @@ static int32_t setTaskAttrInResBlock(SStreamObj *pStream, SStreamTask *pTask, SS memset(vbuf, 0, tListLen(vbuf)); STR_TO_VARSTR(vbuf, buf); - colDataSetVal(pColInfo, numOfRows, (const char *)vbuf, false); + code = colDataSetVal(pColInfo, numOfRows, (const char *)vbuf, false); + TSDB_CHECK_CODE(code, lino, _end); } // output throughput @@ -1793,7 +1876,8 @@ static int32_t setTaskAttrInResBlock(SStreamObj *pStream, SStreamTask *pTask, SS memset(vbuf, 0, tListLen(vbuf)); STR_TO_VARSTR(vbuf, buf); - colDataSetVal(pColInfo, numOfRows, (const char *)vbuf, false); + code = colDataSetVal(pColInfo, numOfRows, (const char *)vbuf, false); + TSDB_CHECK_CODE(code, lino, _end); } // output queue @@ -1818,35 +1902,42 @@ static int32_t setTaskAttrInResBlock(SStreamObj *pStream, SStreamTask *pTask, SS STR_TO_VARSTR(vbuf, buf); pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); - colDataSetVal(pColInfo, numOfRows, (const char *)vbuf, false); + code = colDataSetVal(pColInfo, numOfRows, (const char *)vbuf, false); + TSDB_CHECK_CODE(code, lino, _end); // start_time pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); - colDataSetVal(pColInfo, numOfRows, (const char *)&pe->startTime, false); + code = colDataSetVal(pColInfo, numOfRows, (const char *)&pe->startTime, false); + TSDB_CHECK_CODE(code, lino, _end); // start id pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); - colDataSetVal(pColInfo, numOfRows, (const char *)&pe->startCheckpointId, false); + code = colDataSetVal(pColInfo, numOfRows, (const char *)&pe->startCheckpointId, false); + TSDB_CHECK_CODE(code, lino, _end); // start ver pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); - colDataSetVal(pColInfo, numOfRows, (const char *)&pe->startCheckpointVer, false); + code = colDataSetVal(pColInfo, numOfRows, (const char *)&pe->startCheckpointVer, false); + TSDB_CHECK_CODE(code, lino, _end); // checkpoint time pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); if (pe->checkpointInfo.latestTime != 0) { - colDataSetVal(pColInfo, numOfRows, (const char *)&pe->checkpointInfo.latestTime, false); + code = colDataSetVal(pColInfo, numOfRows, (const char *)&pe->checkpointInfo.latestTime, false); } else { - colDataSetVal(pColInfo, numOfRows, 0, true); + code = colDataSetVal(pColInfo, numOfRows, 0, true); } + TSDB_CHECK_CODE(code, lino, _end); // checkpoint_id pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); - colDataSetVal(pColInfo, numOfRows, (const char *)&pe->checkpointInfo.latestId, false); + code = colDataSetVal(pColInfo, numOfRows, (const char *)&pe->checkpointInfo.latestId, false); + TSDB_CHECK_CODE(code, lino, _end); // checkpoint version pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); - colDataSetVal(pColInfo, numOfRows, (const char *)&pe->checkpointInfo.latestVer, false); + code = colDataSetVal(pColInfo, numOfRows, (const char *)&pe->checkpointInfo.latestVer, false); + TSDB_CHECK_CODE(code, lino, _end); // checkpoint size pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); @@ -1854,26 +1945,31 @@ static int32_t setTaskAttrInResBlock(SStreamObj *pStream, SStreamTask *pTask, SS // checkpoint backup status pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); - colDataSetVal(pColInfo, numOfRows, 0, true); + code = colDataSetVal(pColInfo, numOfRows, 0, true); + TSDB_CHECK_CODE(code, lino, _end); // ds_err_info pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); - colDataSetVal(pColInfo, numOfRows, 0, true); + code = colDataSetVal(pColInfo, numOfRows, 0, true); + TSDB_CHECK_CODE(code, lino, _end); // history_task_id pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); if (pe->hTaskId != 0) { int64ToHexStr(pe->hTaskId, idstr, tListLen(idstr)); - colDataSetVal(pColInfo, numOfRows, idstr, false); + code = colDataSetVal(pColInfo, numOfRows, idstr, false); } else { - colDataSetVal(pColInfo, numOfRows, 0, true); + code = colDataSetVal(pColInfo, numOfRows, 0, true); } + TSDB_CHECK_CODE(code, lino, _end); // history_task_status pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); - colDataSetVal(pColInfo, numOfRows, 0, true); + code = colDataSetVal(pColInfo, numOfRows, 0, true); + TSDB_CHECK_CODE(code, lino, _end); - return TSDB_CODE_SUCCESS; + _end: + return code; } static int32_t mndRetrieveStreamTask(SRpcMsg *pReq, SShowObj *pShow, SSDataBlock *pBlock, int32_t rowsCapacity) { @@ -1881,6 +1977,7 @@ static int32_t mndRetrieveStreamTask(SRpcMsg *pReq, SShowObj *pShow, SSDataBlock SSdb *pSdb = pMnode->pSdb; int32_t numOfRows = 0; SStreamObj *pStream = NULL; + int32_t code = 0; streamMutexLock(&execInfo.lock); mndInitStreamExecInfo(pMnode, &execInfo); @@ -1897,12 +1994,18 @@ static int32_t mndRetrieveStreamTask(SRpcMsg *pReq, SShowObj *pShow, SSDataBlock int32_t count = mndGetNumOfStreamTasks(pStream); if (numOfRows + count > rowsCapacity) { - blockDataEnsureCapacity(pBlock, numOfRows + count); + code = blockDataEnsureCapacity(pBlock, numOfRows + count); + if (code) { + mError("failed to prepare the result block buffer, quit return value"); + taosRUnLockLatch(&pStream->lock); + sdbRelease(pSdb, pStream); + continue; + } } // add row for each task SStreamTaskIter *pIter = NULL; - int32_t code = createStreamTaskIter(pStream, &pIter); + code = createStreamTaskIter(pStream, &pIter); if (code) { taosRUnLockLatch(&pStream->lock); sdbRelease(pSdb, pStream); @@ -2123,7 +2226,7 @@ static int32_t mndProcessResumeStreamReq(SRpcMsg *pReq) { bool conflict = mndStreamTransConflictCheck(pMnode, pStream->uid, MND_STREAM_RESUME_NAME, true); if (conflict) { sdbRelease(pMnode->pSdb, pStream); - return -1; + return terrno; } STrans *pTrans = NULL; @@ -2209,7 +2312,7 @@ static SVgroupChangeInfo mndFindChangedNodeInfo(SMnode *pMnode, const SArray *pP const SEp *pPrevEp = GET_ACTIVE_EP(&pPrevEntry->epset); char buf[256] = {0}; - epsetToStr(&pCurrent->epset, buf, tListLen(buf)); + (void) epsetToStr(&pCurrent->epset, buf, tListLen(buf)); // ignore this error mDebug("nodeId:%d restart/epset changed detected, old:%s:%d -> new:%s, stageUpdate:%d", pCurrent->nodeId, pPrevEp->fqdn, pPrevEp->port, buf, pPrevEntry->stageUpdated); @@ -2217,14 +2320,21 @@ static SVgroupChangeInfo mndFindChangedNodeInfo(SMnode *pMnode, const SArray *pP SNodeUpdateInfo updateInfo = {.nodeId = pPrevEntry->nodeId}; epsetAssign(&updateInfo.prevEp, &pPrevEntry->epset); epsetAssign(&updateInfo.newEp, &pCurrent->epset); - taosArrayPush(info.pUpdateNodeList, &updateInfo); + + void* p = taosArrayPush(info.pUpdateNodeList, &updateInfo); + if (p == NULL) { + mError("failed to put update entry into node list, code:%s", tstrerror(TSDB_CODE_OUT_OF_MEMORY)); + } } // todo handle the snode info if (pCurrent->nodeId != SNODE_HANDLE) { SVgObj *pVgroup = mndAcquireVgroup(pMnode, pCurrent->nodeId); - taosHashPut(info.pDBMap, pVgroup->dbName, strlen(pVgroup->dbName), NULL, 0); + int32_t code = taosHashPut(info.pDBMap, pVgroup->dbName, strlen(pVgroup->dbName), NULL, 0); mndReleaseVgroup(pMnode, pVgroup); + if (code) { + mError("failed to put into dbmap, code:out of memory"); + } } break; @@ -2255,7 +2365,7 @@ static int32_t mndProcessVgroupChange(SMnode *pMnode, SVgroupChangeInfo *pChange if (conflict) { mError("nodeUpdate conflict with other trans, current nodeUpdate ignored"); sdbCancelFetch(pSdb, pIter); - return -1; + return terrno; } } @@ -2274,7 +2384,10 @@ static int32_t mndProcessVgroupChange(SMnode *pMnode, SVgroupChangeInfo *pChange return terrno = code; } - mndStreamRegisterTrans(pTrans, MND_STREAM_TASK_UPDATE_NAME, pStream->uid); + code = mndStreamRegisterTrans(pTrans, MND_STREAM_TASK_UPDATE_NAME, pStream->uid); + if (code) { + mError("failed to register trans, transId:%d, and continue", pTrans->id); + } } void *p = taosHashGet(pChangeInfo->pDBMap, pStream->targetDb, strlen(pStream->targetDb)); @@ -2329,8 +2442,13 @@ static int32_t extractNodeListFromStream(SMnode *pMnode, SArray *pNodeList) { SSdb *pSdb = pMnode->pSdb; SStreamObj *pStream = NULL; void *pIter = NULL; + int32_t code = 0; SHashObj *pHash = taosHashInit(64, taosGetDefaultHashFunction(TSDB_DATA_TYPE_INT), false, HASH_NO_LOCK); + if (pHash == NULL) { + return TSDB_CODE_OUT_OF_MEMORY; + } + while (1) { pIter = sdbFetch(pSdb, SDB_STREAM, pIter, (void **)&pStream); if (pIter == NULL) { @@ -2340,7 +2458,7 @@ static int32_t extractNodeListFromStream(SMnode *pMnode, SArray *pNodeList) { taosWLockLatch(&pStream->lock); SStreamTaskIter *pTaskIter = NULL; - int32_t code = createStreamTaskIter(pStream, &pTaskIter); + code = createStreamTaskIter(pStream, &pTaskIter); if (code) { taosWUnLockLatch(&pStream->lock); sdbRelease(pSdb, pStream); @@ -2357,7 +2475,10 @@ static int32_t extractNodeListFromStream(SMnode *pMnode, SArray *pNodeList) { SNodeEntry entry = {.hbTimestamp = -1, .nodeId = pTask->info.nodeId}; epsetAssign(&entry.epset, &pTask->info.epSet); - taosHashPut(pHash, &entry.nodeId, sizeof(entry.nodeId), &entry, sizeof(entry)); + code = taosHashPut(pHash, &entry.nodeId, sizeof(entry.nodeId), &entry, sizeof(entry)); + if (code) { + break; + } } destroyStreamTaskIter(pTaskIter); @@ -2372,22 +2493,32 @@ static int32_t extractNodeListFromStream(SMnode *pMnode, SArray *pNodeList) { pIter = NULL; while ((pIter = taosHashIterate(pHash, pIter)) != NULL) { SNodeEntry *pEntry = (SNodeEntry *)pIter; - taosArrayPush(pNodeList, pEntry); + + void *p = taosArrayPush(pNodeList, pEntry); + if (p == NULL) { + mError("failed to put entry into node list, nodeId:%d, code: out of memory", pEntry->nodeId); + if (code == 0) { + code = TSDB_CODE_OUT_OF_MEMORY; + } + continue; + } char buf[256] = {0}; - epsetToStr(&pEntry->epset, buf, tListLen(buf)); + (void) epsetToStr(&pEntry->epset, buf, tListLen(buf)); // ignore this error since it is only for log file mDebug("extract nodeInfo from stream obj, nodeId:%d, %s", pEntry->nodeId, buf); } taosHashCleanup(pHash); mDebug("numOfNodes:%d for stream after extract nodeInfo from stream", (int32_t)taosArrayGetSize(pNodeList)); - return TSDB_CODE_SUCCESS; + return code; } // this function runs by only one thread, so it is not multi-thread safe static int32_t mndProcessNodeCheckReq(SRpcMsg *pMsg) { int32_t code = 0; + bool allReady = true; + SArray *pNodeSnapshot = NULL; int32_t old = atomic_val_compare_exchange_32(&mndNodeCheckSentinel, 0, 1); if (old != 0) { @@ -2411,9 +2542,6 @@ static int32_t mndProcessNodeCheckReq(SRpcMsg *pMsg) { return 0; } - bool allReady = true; - SArray *pNodeSnapshot = NULL; - code = mndTakeVgroupSnapshot(pMnode, &allReady, &pNodeSnapshot); if (code) { mError("failed to take the vgroup snapshot, ignore it and continue"); @@ -2428,7 +2556,10 @@ static int32_t mndProcessNodeCheckReq(SRpcMsg *pMsg) { streamMutexLock(&execInfo.lock); - removeExpiredNodeEntryAndTaskInBuf(pNodeSnapshot); + code = removeExpiredNodeEntryAndTaskInBuf(pNodeSnapshot); + if (code) { + goto _end; + } SVgroupChangeInfo changeInfo = mndFindChangedNodeInfo(pMnode, execInfo.pNodeList, pNodeSnapshot); if (taosArrayGetSize(changeInfo.pUpdateNodeList) > 0) { @@ -2438,7 +2569,12 @@ static int32_t mndProcessNodeCheckReq(SRpcMsg *pMsg) { // keep the new vnode snapshot if success if (code == TSDB_CODE_SUCCESS || code == TSDB_CODE_ACTION_IN_PROGRESS) { - extractNodeListFromStream(pMnode, execInfo.pNodeList); + code = extractNodeListFromStream(pMnode, execInfo.pNodeList); + if (code) { + mError("failed to extract node list from stream, code:%s", tstrerror(code)); + goto _end; + } + execInfo.ts = ts; mDebug("create trans successfully, update cached node list, numOfNodes:%d", (int)taosArrayGetSize(execInfo.pNodeList)); @@ -2449,12 +2585,13 @@ static int32_t mndProcessNodeCheckReq(SRpcMsg *pMsg) { mDebug("no update found in nodeList"); } - taosArrayDestroy(pNodeSnapshot); - streamMutexUnlock(&execInfo.lock); - taosArrayDestroy(changeInfo.pUpdateNodeList); taosHashCleanup(changeInfo.pDBMap); + _end: + streamMutexUnlock(&execInfo.lock); + taosArrayDestroy(pNodeSnapshot); + mDebug("end to do stream task node change checking"); atomic_store_32(&mndNodeCheckSentinel, 0); return 0; @@ -2471,8 +2608,7 @@ static int32_t mndProcessNodeCheck(SRpcMsg *pReq) { SMStreamNodeCheckMsg *pMsg = rpcMallocCont(size); SRpcMsg rpcMsg = {.msgType = TDMT_MND_STREAM_NODECHANGE_CHECK, .pCont = pMsg, .contLen = size}; - tmsgPutToQueue(&pMnode->msgCb, WRITE_QUEUE, &rpcMsg); - return 0; + return tmsgPutToQueue(&pMnode->msgCb, WRITE_QUEUE, &rpcMsg); } void saveTaskAndNodeInfoIntoBuf(SStreamObj *pStream, SStreamExecInfo *pExecNode) { @@ -2496,11 +2632,18 @@ void saveTaskAndNodeInfoIntoBuf(SStreamObj *pStream, SStreamExecInfo *pExecNode) STaskStatusEntry entry = {0}; streamTaskStatusInit(&entry, pTask); - taosHashPut(pExecNode->pTaskMap, &id, sizeof(id), &entry, sizeof(entry)); - taosArrayPush(pExecNode->pTaskList, &id); - - int32_t num = (int32_t)taosArrayGetSize(pExecNode->pTaskList); - mInfo("s-task:0x%x add into task buffer, total:%d", (int32_t)entry.id.taskId, num); + code = taosHashPut(pExecNode->pTaskMap, &id, sizeof(id), &entry, sizeof(entry)); + if (code == 0) { + void * px = taosArrayPush(pExecNode->pTaskList, &id); + int32_t num = (int32_t)taosArrayGetSize(pExecNode->pTaskList); + if (px) { + mInfo("s-task:0x%x add into task buffer, total:%d", (int32_t)entry.id.taskId, num); + } else { + mError("s-task:0x%x failed to add into task buffer, total:%d", (int32_t)entry.id.taskId, num); + } + } else { + mError("s-task:0x%x failed to add into task map, since out of memory", (int32_t) entry.id.taskId); + } // add the new vgroups if not added yet bool exist = false; @@ -2516,8 +2659,12 @@ void saveTaskAndNodeInfoIntoBuf(SStreamObj *pStream, SStreamExecInfo *pExecNode) SNodeEntry nodeEntry = {.hbTimestamp = -1, .nodeId = pTask->info.nodeId}; epsetAssign(&nodeEntry.epset, &pTask->info.epSet); - taosArrayPush(pExecNode->pNodeList, &nodeEntry); - mInfo("vgId:%d added into nodeList, total:%d", nodeEntry.nodeId, (int)taosArrayGetSize(pExecNode->pNodeList)); + void* px = taosArrayPush(pExecNode->pNodeList, &nodeEntry); + if (px) { + mInfo("vgId:%d added into nodeList, total:%d", nodeEntry.nodeId, (int)taosArrayGetSize(pExecNode->pNodeList)); + } else { + mError("vgId:%d failed to add into nodeList, total:%d", nodeEntry.nodeId, (int)taosArrayGetSize(pExecNode->pNodeList)) + } } } } @@ -2534,10 +2681,15 @@ static void doAddTaskId(SArray *pList, int32_t taskId, int64_t uid, int32_t numO } } - taosArrayPush(pList, &taskId); - + void* p = taosArrayPush(pList, &taskId); int32_t numOfTasks = taosArrayGetSize(pList); - mDebug("stream:0x%" PRIx64 " receive %d reqs for checkpoint, remain:%d", uid, numOfTasks, numOfTotal - numOfTasks); + + if (p) { + mDebug("stream:0x%" PRIx64 " receive %d reqs for checkpoint, remain:%d", uid, numOfTasks, numOfTotal - numOfTasks); + } else { + mError("stream:0x%" PRIx64 " receive %d reqs for checkpoint, failed to added into task list, since out of memory", + uid, numOfTasks); + } } int32_t mndProcessStreamReqCheckpoint(SRpcMsg *pReq) { @@ -2549,9 +2701,8 @@ int32_t mndProcessStreamReqCheckpoint(SRpcMsg *pReq) { if (tDecodeStreamTaskCheckpointReq(&decoder, &req)) { tDecoderClear(&decoder); - terrno = TSDB_CODE_INVALID_MSG; mError("invalid task checkpoint req msg received"); - return -1; + return TSDB_CODE_INVALID_MSG; } tDecoderClear(&decoder); @@ -2572,9 +2723,8 @@ int32_t mndProcessStreamReqCheckpoint(SRpcMsg *pReq) { void *p = taosHashGet(execInfo.pTaskMap, &id, sizeof(id)); if (p == NULL) { mError("failed to find the stream:0x%" PRIx64 " in buf, not handle the checkpoint req", req.streamId); - terrno = TSDB_CODE_MND_STREAM_NOT_EXIST; streamMutexUnlock(&execInfo.lock); - return -1; + return TSDB_CODE_MND_STREAM_NOT_EXIST; } else { mDebug("s-task:0x%" PRIx64 "-0x%x in buf not in mnode/meta, create stream trans may not complete yet", req.streamId, req.taskId); @@ -2587,8 +2737,10 @@ int32_t mndProcessStreamReqCheckpoint(SRpcMsg *pReq) { if (pReqTaskList == NULL) { SArray *pList = taosArrayInit(4, sizeof(int32_t)); doAddTaskId(pList, req.taskId, req.streamId, numOfTasks); - taosHashPut(execInfo.pTransferStateStreams, &req.streamId, sizeof(int64_t), &pList, sizeof(void *)); - + code = taosHashPut(execInfo.pTransferStateStreams, &req.streamId, sizeof(int64_t), &pList, sizeof(void *)); + if (code) { + mError("failed to put into transfer state stream map, code: out of memory"); + } pReqTaskList = (SArray **)taosHashGet(execInfo.pTransferStateStreams, &req.streamId, sizeof(req.streamId)); } else { doAddTaskId(*pReqTaskList, req.taskId, req.streamId, numOfTasks); @@ -2600,7 +2752,10 @@ int32_t mndProcessStreamReqCheckpoint(SRpcMsg *pReq) { mInfo("stream:0x%" PRIx64 " all tasks req checkpoint, start checkpointId:%" PRId64, req.streamId, checkpointId); if (pStream != NULL) { // TODO:handle error - int32_t code = mndProcessStreamCheckpointTrans(pMnode, pStream, checkpointId, 0, false); + code = mndProcessStreamCheckpointTrans(pMnode, pStream, checkpointId, 0, false); + if (code) { + mError("failed to create checkpoint trans, code:%s", strerror(code)); + } } else { // todo: wait for the create stream trans completed, and launch the checkpoint trans // SStreamObj *pStream = mndGetStreamObj(pMnode, req.streamId); @@ -2608,7 +2763,7 @@ int32_t mndProcessStreamReqCheckpoint(SRpcMsg *pReq) { } // remove this entry - taosHashRemove(execInfo.pTransferStateStreams, &req.streamId, sizeof(int64_t)); + (void) taosHashRemove(execInfo.pTransferStateStreams, &req.streamId, sizeof(int64_t)); int32_t numOfStreams = taosHashGetSize(execInfo.pTransferStateStreams); mDebug("stream:0x%" PRIx64 " removed, remain streams:%d fill-history not completed", req.streamId, numOfStreams); @@ -2654,7 +2809,11 @@ static void doAddReportStreamTask(SArray* pList, const SCheckpointReport* pRepor .checkpointId = pReport->checkpointId, .nodeId = pReport->nodeId, }; - taosArrayPush(pList, &info); + + void* p = taosArrayPush(pList, &info); + if (p == NULL) { + mError("failed to put into task list, taskId:0x%x", pReport->taskId); + } } } @@ -2667,9 +2826,8 @@ int32_t mndProcessCheckpointReport(SRpcMsg *pReq) { if (tDecodeStreamTaskChkptReport(&decoder, &req)) { tDecoderClear(&decoder); - terrno = TSDB_CODE_INVALID_MSG; mError("invalid task checkpoint-report msg received"); - return -1; + return TSDB_CODE_INVALID_MSG; } tDecoderClear(&decoder); @@ -2693,7 +2851,7 @@ int32_t mndProcessCheckpointReport(SRpcMsg *pReq) { mError("failed to find the stream:0x%" PRIx64 " in buf, not handle the checkpoint-report", req.streamId); terrno = TSDB_CODE_MND_STREAM_NOT_EXIST; streamMutexUnlock(&execInfo.lock); - return -1; + return terrno; } else { mDebug("s-task:0x%" PRIx64 "-0x%x in buf not in mnode/meta, create stream trans may not complete yet", req.streamId, req.taskId); @@ -2706,7 +2864,11 @@ int32_t mndProcessCheckpointReport(SRpcMsg *pReq) { if (pReqTaskList == NULL) { SArray *pList = taosArrayInit(4, sizeof(STaskChkptInfo)); doAddReportStreamTask(pList, &req); - taosHashPut(execInfo.pChkptStreams, &req.streamId, sizeof(req.streamId), &pList, POINTER_BYTES); + + code = taosHashPut(execInfo.pChkptStreams, &req.streamId, sizeof(req.streamId), &pList, POINTER_BYTES); + if (code) { + mError("stream:0x%"PRIx64 " failed to put into checkpoint stream", req.streamId); + } pReqTaskList = (SArray **)taosHashGet(execInfo.pChkptStreams, &req.streamId, sizeof(req.streamId)); } else { @@ -2727,7 +2889,7 @@ int32_t mndProcessCheckpointReport(SRpcMsg *pReq) { streamMutexUnlock(&execInfo.lock); doSendQuickRsp(&pReq->info, sizeof(SMStreamUpdateChkptRsp), req.nodeId, TSDB_CODE_SUCCESS); - return 0; + return code; } static int64_t getConsensusId(int64_t streamId, int32_t numOfTasks, int32_t* pExistedTasks, bool *pAllSame) { @@ -2770,100 +2932,6 @@ static void doSendQuickRsp(SRpcHandleInfo *pInfo, int32_t msgSize, int32_t vgId, pInfo->handle = NULL; // disable auto rsp } -//static int32_t mndProcessConsensusCheckpointId(SRpcMsg *pMsg) { -// SMnode *pMnode = pMsg->info.node; -// SDecoder decoder = {0}; -// -// SRestoreCheckpointInfo req = {0}; -// tDecoderInit(&decoder, pMsg->pCont, pMsg->contLen); -// -// if (tDecodeRestoreCheckpointInfo(&decoder, &req)) { -// tDecoderClear(&decoder); -// terrno = TSDB_CODE_INVALID_MSG; -// mError("invalid task consensus-checkpoint msg received"); -// return -1; -// } -// tDecoderClear(&decoder); -// -// mDebug("receive stream task consensus-checkpoint msg, vgId:%d, s-task:0x%" PRIx64 "-0x%x, checkpointId:%" PRId64, -// req.nodeId, req.streamId, req.taskId, req.checkpointId); -// -// // register to the stream task done map, if all tasks has sent this kinds of message, start the checkpoint trans. -// streamMutexLock(&execInfo.lock); -// -// // mnode handle the create stream transaction too slow may cause this problem -// SStreamObj *pStream = mndGetStreamObj(pMnode, req.streamId); -// if (pStream == NULL) { -// mWarn("failed to find the stream:0x%" PRIx64 ", not handle consensus-checkpointId", req.streamId); -// -// // not in meta-store yet, try to acquire the task in exec buffer -// // the checkpoint req arrives too soon before the completion of the create stream trans. -// STaskId id = {.streamId = req.streamId, .taskId = req.taskId}; -// void *p = taosHashGet(execInfo.pTaskMap, &id, sizeof(id)); -// if (p == NULL) { -// mError("failed to find the stream:0x%" PRIx64 " in buf, not handle consensus-checkpointId", req.streamId); -// terrno = TSDB_CODE_MND_STREAM_NOT_EXIST; -// streamMutexUnlock(&execInfo.lock); -// -// doSendQuickRsp(&pMsg->info, sizeof(SMStreamReqConsensChkptRsp), req.nodeId, terrno); -// return -1; -// } else { -// mDebug("s-task:0x%" PRIx64 "-0x%x in buf not in mnode/meta, create stream trans may not complete yet", -// req.streamId, req.taskId); -// // todo wait for stream is created -// } -// } -// -// mInfo("vgId:%d stream:0x%" PRIx64 " %s meta-stored checkpointId:%" PRId64, req.nodeId, req.streamId, pStream->name, -// pStream->checkpointId); -// -// int32_t numOfTasks = (pStream == NULL) ? 0 : mndGetNumOfStreamTasks(pStream); -// if ((pStream != NULL) && (pStream->checkpointId == 0)) { // not generated checkpoint yet, return 0 directly -// streamMutexUnlock(&execInfo.lock); -// mndCreateSetConsensusChkptIdTrans(pMnode, pStream, req.taskId, 0, req.startTs); -// -// doSendQuickRsp(&pMsg->info, sizeof(SMStreamReqConsensChkptRsp), req.nodeId, terrno); -// return TSDB_CODE_SUCCESS; -// } -// -// int32_t num = 0; -// int64_t chkId = getConsensusId(req.streamId, numOfTasks, &num); -// -// // some tasks not send hbMsg to mnode yet, wait for 5s. -// if (chkId == -1) { -// mDebug("not all(%d/%d) task(s) send hbMsg yet, wait for a while and check again, s-task:0x%x", req.taskId, num, -// numOfTasks); -// SCheckpointConsensusInfo *pInfo = mndGetConsensusInfo(execInfo.pStreamConsensus, req.streamId, numOfTasks); -// mndAddConsensusTasks(pInfo, &req); -// -// streamMutexUnlock(&execInfo.lock); -// doSendQuickRsp(&pMsg->info, sizeof(SMStreamReqConsensChkptRsp), req.nodeId, terrno); -// return 0; -// } -// -// if (chkId == req.checkpointId) { -// mDebug("vgId:%d stream:0x%" PRIx64 " %s consensus-checkpointId is:%" PRId64 ", meta-stored checkpointId:%" PRId64, -// req.nodeId, req.streamId, pStream->name, chkId, pStream->checkpointId); -// mndCreateSetConsensusChkptIdTrans(pMnode, pStream, req.taskId, chkId, req.startTs); -// -// streamMutexUnlock(&execInfo.lock); -// doSendQuickRsp(&pMsg->info, sizeof(SMStreamReqConsensChkptRsp), req.nodeId, terrno); -// return 0; -// } -// -// // wait for 5s and check again -// SCheckpointConsensusInfo *pInfo = mndGetConsensusInfo(execInfo.pStreamConsensus, req.streamId, numOfTasks); -// mndAddConsensusTasks(pInfo, &req); -// -// if (pStream != NULL) { -// mndReleaseStream(pMnode, pStream); -// } -// -// streamMutexUnlock(&execInfo.lock); -// doSendQuickRsp(&pMsg->info, sizeof(SMStreamReqConsensChkptRsp), req.nodeId, terrno); -// return 0; -//} - int32_t mndProcessConsensusInTmr(SRpcMsg *pMsg) { SMnode *pMnode = pMsg->info.node; int64_t now = taosGetTimestampMs(); @@ -2921,9 +2989,15 @@ int32_t mndProcessConsensusInTmr(SRpcMsg *pMsg) { mDebug("s-task:0x%x sendTs:%" PRId64 " wait %.2fs and all tasks have same checkpointId", pe->req.taskId, pe->req.startTs, (now - pe->ts) / 1000.0); ASSERT(chkId <= pe->req.checkpointId); - mndCreateSetConsensusChkptIdTrans(pMnode, pStream, pe->req.taskId, chkId, pe->req.startTs); + code = mndCreateSetConsensusChkptIdTrans(pMnode, pStream, pe->req.taskId, chkId, pe->req.startTs); + if (code != TSDB_CODE_SUCCESS && code != TSDB_CODE_ACTION_IN_PROGRESS) { + mError("failed to create consensus-checkpoint trans, stream:0x%" PRIx64, pStream->uid); + } - taosArrayPush(pList, &pe->req.taskId); + void* p = taosArrayPush(pList, &pe->req.taskId); + if (p == NULL) { + mError("failed to put into task list, taskId:0x%x", pe->req.taskId); + } streamId = pe->req.streamId; } else { mDebug("s-task:0x%x sendTs:%" PRId64 " wait %.2fs already, wait for next round to check", pe->req.taskId, @@ -2951,7 +3025,10 @@ int32_t mndProcessConsensusInTmr(SRpcMsg *pMsg) { if (taosArrayGetSize(pInfo->pTaskList) == 0) { mndClearConsensusRspEntry(pInfo); ASSERT(streamId != -1); - taosArrayPush(pStreamList, &streamId); + void* p = taosArrayPush(pStreamList, &streamId); + if (p == NULL) { + mError("failed to put into stream list, stream:0x%" PRIx64, streamId); + } } } diff --git a/source/dnode/mnode/impl/src/mndStreamHb.c b/source/dnode/mnode/impl/src/mndStreamHb.c index c5297b5ba8..1ca46f128f 100644 --- a/source/dnode/mnode/impl/src/mndStreamHb.c +++ b/source/dnode/mnode/impl/src/mndStreamHb.c @@ -57,7 +57,10 @@ void addIntoCheckpointList(SArray *pList, const SFailedCheckpointInfo *pInfo) { } } - taosArrayPush(pList, pInfo); + void* p = taosArrayPush(pList, pInfo); + if (p == NULL) { + mError("failed to push failed checkpoint info checkpointId:%" PRId64 " in list", pInfo->checkpointId); + } } int32_t mndCreateStreamResetStatusTrans(SMnode *pMnode, SStreamObj *pStream) { @@ -208,6 +211,8 @@ int32_t suspendAllStreams(SMnode *pMnode, SRpcHandleInfo *info) { SSdb *pSdb = pMnode->pSdb; SStreamObj *pStream = NULL; void *pIter = NULL; + int32_t code = 0; + while (1) { pIter = sdbFetch(pSdb, SDB_STREAM, pIter, (void **)&pStream); if (pIter == NULL) break; @@ -219,7 +224,17 @@ int32_t suspendAllStreams(SMnode *pMnode, SRpcHandleInfo *info) { int32_t contLen = tSerializeSMPauseStreamReq(NULL, 0, &reqPause); void *pHead = rpcMallocCont(contLen); - tSerializeSMPauseStreamReq(pHead, contLen, &reqPause); + if (pHead == NULL) { + code = TSDB_CODE_OUT_OF_MEMORY; + sdbRelease(pSdb, pStream); + continue; + } + + code = tSerializeSMPauseStreamReq(pHead, contLen, &reqPause); + if (code) { + sdbRelease(pSdb, pStream); + continue; + } SRpcMsg rpcMsg = { .msgType = TDMT_MND_PAUSE_STREAM, @@ -228,14 +243,14 @@ int32_t suspendAllStreams(SMnode *pMnode, SRpcHandleInfo *info) { .info = *info, }; - tmsgPutToQueue(&pMnode->msgCb, WRITE_QUEUE, &rpcMsg); - mInfo("receive pause stream:%s, %s, %" PRId64 ", because grant expired", pStream->name, reqPause.name, - pStream->uid); + code = tmsgPutToQueue(&pMnode->msgCb, WRITE_QUEUE, &rpcMsg); + mInfo("receive pause stream:%s, %s, %" PRId64 ", because grant expired, code:%s", pStream->name, reqPause.name, + pStream->uid, tstrerror(code)); } sdbRelease(pSdb, pStream); } - return 0; + return code; } int32_t mndProcessStreamHb(SRpcMsg *pReq) { @@ -267,7 +282,7 @@ int32_t mndProcessStreamHb(SRpcMsg *pReq) { pFailedChkpt = taosArrayInit(4, sizeof(SFailedCheckpointInfo)); pOrphanTasks = taosArrayInit(4, sizeof(SOrphanTask)); - taosThreadMutexLock(&execInfo.lock); + streamMutexLock(&execInfo.lock); mndInitStreamExecInfo(pMnode, &execInfo); if (!validateHbMsg(execInfo.pNodeList, req.vgId)) { @@ -276,7 +291,7 @@ int32_t mndProcessStreamHb(SRpcMsg *pReq) { code = terrno = TSDB_CODE_INVALID_MSG; doSendHbMsgRsp(terrno, &pReq->info, req.vgId, req.msgId); - taosThreadMutexUnlock(&execInfo.lock); + streamMutexUnlock(&execInfo.lock); cleanupAfterProcessHbMsg(&req, pFailedChkpt, pOrphanTasks); return code; } @@ -284,7 +299,7 @@ int32_t mndProcessStreamHb(SRpcMsg *pReq) { int32_t numOfUpdated = taosArrayGetSize(req.pUpdateNodes); if (numOfUpdated > 0) { mDebug("%d stream node(s) need updated from hbMsg(vgId:%d)", numOfUpdated, req.vgId); - setNodeEpsetExpiredFlag(req.pUpdateNodes); + (void) setNodeEpsetExpiredFlag(req.pUpdateNodes); } bool snodeChanged = false; @@ -296,7 +311,10 @@ int32_t mndProcessStreamHb(SRpcMsg *pReq) { mError("s-task:0x%" PRIx64 " not found in mnode task list", p->id.taskId); SOrphanTask oTask = {.streamId = p->id.streamId, .taskId = p->id.taskId, .nodeId = p->nodeId}; - taosArrayPush(pOrphanTasks, &oTask); + void* px = taosArrayPush(pOrphanTasks, &oTask); + if (px == NULL) { + mError("Failed to put task into list, taskId:0x%" PRIx64, p->id.taskId); + } continue; } @@ -346,7 +364,10 @@ int32_t mndProcessStreamHb(SRpcMsg *pReq) { addIntoCheckpointList(pFailedChkpt, &info); // remove failed trans from pChkptStreams - taosHashRemove(execInfo.pChkptStreams, &p->id.streamId, sizeof(p->id.streamId)); + code = taosHashRemove(execInfo.pChkptStreams, &p->id.streamId, sizeof(p->id.streamId)); + if (code) { + mError("failed to remove stream:0x%"PRIx64" in checkpoint stream list", p->id.streamId); + } } } @@ -386,7 +407,10 @@ int32_t mndProcessStreamHb(SRpcMsg *pReq) { mInfo("checkpointId:%" PRId64 " transId:%d failed, issue task-reset trans to reset all tasks status", pInfo->checkpointId, pInfo->transId); - mndResetStatusFromCheckpoint(pMnode, pInfo->streamUid, pInfo->transId); + code = mndResetStatusFromCheckpoint(pMnode, pInfo->streamUid, pInfo->transId); + if (code) { + mError("failed to create reset task trans, code:%s", tstrerror(code)); + } } } else { mInfo("not all vgroups are ready, wait for next HB from stream tasks to reset the task status"); @@ -395,20 +419,19 @@ int32_t mndProcessStreamHb(SRpcMsg *pReq) { // handle the orphan tasks that are invalid but not removed in some vnodes or snode due to some unknown errors. if (taosArrayGetSize(pOrphanTasks) > 0) { - mndDropOrphanTasks(pMnode, pOrphanTasks); + code = mndDropOrphanTasks(pMnode, pOrphanTasks); } if (pMnode != NULL) { // make sure that the unit test case can work mndStreamStartUpdateCheckpointInfo(pMnode); } - taosThreadMutexUnlock(&execInfo.lock); + streamMutexUnlock(&execInfo.lock); - terrno = TSDB_CODE_SUCCESS; - doSendHbMsgRsp(terrno, &pReq->info, req.vgId, req.msgId); + doSendHbMsgRsp(TSDB_CODE_SUCCESS, &pReq->info, req.vgId, req.msgId); cleanupAfterProcessHbMsg(&req, pFailedChkpt, pOrphanTasks); - return terrno; + return code; } void mndStreamStartUpdateCheckpointInfo(SMnode *pMnode) { // here reuse the doCheckpointmsg @@ -416,7 +439,10 @@ void mndStreamStartUpdateCheckpointInfo(SMnode *pMnode) { // here reuse the doC if (pMsg != NULL) { int32_t size = sizeof(SMStreamDoCheckpointMsg); SRpcMsg rpcMsg = {.msgType = TDMT_MND_STREAM_UPDATE_CHKPT_EVT, .pCont = pMsg, .contLen = size}; - tmsgPutToQueue(&pMnode->msgCb, WRITE_QUEUE, &rpcMsg); + int32_t code = tmsgPutToQueue(&pMnode->msgCb, WRITE_QUEUE, &rpcMsg); + if (code) { + mError("failed to put into write Queue, code:%s", tstrerror(code)); + } } } diff --git a/source/dnode/mnode/impl/src/mndStreamTrans.c b/source/dnode/mnode/impl/src/mndStreamTrans.c index c0a869fb77..fe656ba9c4 100644 --- a/source/dnode/mnode/impl/src/mndStreamTrans.c +++ b/source/dnode/mnode/impl/src/mndStreamTrans.c @@ -24,8 +24,7 @@ typedef struct SKeyInfo { int32_t mndStreamRegisterTrans(STrans *pTrans, const char *pTransName, int64_t streamId) { SStreamTransInfo info = { .transId = pTrans->id, .startTime = taosGetTimestampMs(), .name = pTransName, .streamId = streamId}; - taosHashPut(execInfo.transMgmt.pDBTrans, &streamId, sizeof(streamId), &info, sizeof(SStreamTransInfo)); - return 0; + return taosHashPut(execInfo.transMgmt.pDBTrans, &streamId, sizeof(streamId), &info, sizeof(SStreamTransInfo)); } int32_t mndStreamClearFinishedTrans(SMnode *pMnode, int32_t *pNumOfActiveChkpt) { @@ -45,7 +44,10 @@ int32_t mndStreamClearFinishedTrans(SMnode *pMnode, int32_t *pNumOfActiveChkpt) SKeyInfo info = {.pKey = pKey, .keyLen = keyLen}; mDebug("transId:%d %s startTs:%" PRId64 " cleared since finished", pEntry->transId, pEntry->name, pEntry->startTime); - taosArrayPush(pList, &info); + void* p = taosArrayPush(pList, &info); + if (p == NULL) { + return TSDB_CODE_OUT_OF_MEMORY; + } } else { if (strcmp(pEntry->name, MND_STREAM_CHECKPOINT_NAME) == 0) { num++; @@ -57,7 +59,11 @@ int32_t mndStreamClearFinishedTrans(SMnode *pMnode, int32_t *pNumOfActiveChkpt) int32_t size = taosArrayGetSize(pList); for (int32_t i = 0; i < size; ++i) { SKeyInfo *pKey = taosArrayGet(pList, i); - taosHashRemove(execInfo.transMgmt.pDBTrans, pKey->pKey, pKey->keyLen); + int32_t code = taosHashRemove(execInfo.transMgmt.pDBTrans, pKey->pKey, pKey->keyLen); + if (code != 0) { + taosArrayDestroy(pList); + return code; + } } mDebug("clear %d finished stream-trans, remained:%d, active checkpoint trans:%d", size, @@ -79,25 +85,28 @@ int32_t mndStreamClearFinishedTrans(SMnode *pMnode, int32_t *pNumOfActiveChkpt) // 2. create/drop/reset/update trans are conflict with any other trans. bool mndStreamTransConflictCheck(SMnode *pMnode, int64_t streamId, const char *pTransName, bool lock) { if (lock) { - taosThreadMutexLock(&execInfo.lock); + streamMutexLock(&execInfo.lock); } int32_t num = taosHashGetSize(execInfo.transMgmt.pDBTrans); if (num <= 0) { if (lock) { - taosThreadMutexUnlock(&execInfo.lock); + streamMutexUnlock(&execInfo.lock); } return false; } - mndStreamClearFinishedTrans(pMnode, NULL); + int32_t code = mndStreamClearFinishedTrans(pMnode, NULL); + if (code) { + mError("failed to clear finish trans, code:%s", tstrerror(code)); + } SStreamTransInfo *pEntry = taosHashGet(execInfo.transMgmt.pDBTrans, &streamId, sizeof(streamId)); if (pEntry != NULL) { SStreamTransInfo tInfo = *pEntry; if (lock) { - taosThreadMutexUnlock(&execInfo.lock); + streamMutexUnlock(&execInfo.lock); } if (strcmp(tInfo.name, MND_STREAM_CHECKPOINT_NAME) == 0) { @@ -122,32 +131,36 @@ bool mndStreamTransConflictCheck(SMnode *pMnode, int64_t streamId, const char *p } if (lock) { - taosThreadMutexUnlock(&execInfo.lock); + streamMutexUnlock(&execInfo.lock); } return false; } int32_t mndStreamGetRelTrans(SMnode *pMnode, int64_t streamId) { - taosThreadMutexLock(&execInfo.lock); + streamMutexLock(&execInfo.lock); int32_t num = taosHashGetSize(execInfo.transMgmt.pDBTrans); if (num <= 0) { - taosThreadMutexUnlock(&execInfo.lock); + streamMutexUnlock(&execInfo.lock); return 0; } - mndStreamClearFinishedTrans(pMnode, NULL); + int32_t code = mndStreamClearFinishedTrans(pMnode, NULL); + if (code) { + mError("failed to clear finish trans, code:%s", tstrerror(code)); + } + SStreamTransInfo *pEntry = taosHashGet(execInfo.transMgmt.pDBTrans, &streamId, sizeof(streamId)); if (pEntry != NULL) { SStreamTransInfo tInfo = *pEntry; - taosThreadMutexUnlock(&execInfo.lock); + streamMutexUnlock(&execInfo.lock); if (strcmp(tInfo.name, MND_STREAM_CHECKPOINT_NAME) == 0 || strcmp(tInfo.name, MND_STREAM_TASK_UPDATE_NAME) == 0 || strcmp(tInfo.name, MND_STREAM_CHKPT_UPDATE_NAME) == 0) { return tInfo.transId; } } else { - taosThreadMutexUnlock(&execInfo.lock); + streamMutexUnlock(&execInfo.lock); } return 0; @@ -231,21 +244,21 @@ int32_t mndPersistTransLog(SStreamObj *pStream, STrans *pTrans, int32_t status) if (pCommitRaw == NULL) { mError("failed to encode stream since %s", terrstr()); mndTransDrop(pTrans); - return -1; + return terrno; } if (mndTransAppendCommitlog(pTrans, pCommitRaw) != 0) { mError("stream trans:%d, failed to append commit log since %s", pTrans->id, terrstr()); sdbFreeRaw(pCommitRaw); mndTransDrop(pTrans); - return -1; + return terrno; } if (sdbSetRawStatus(pCommitRaw, status) != 0) { mError("stream trans:%d failed to set raw status:%d since %s", pTrans->id, status, terrstr()); sdbFreeRaw(pCommitRaw); mndTransDrop(pTrans); - return -1; + return terrno; } return 0; @@ -303,8 +316,12 @@ void killAllCheckpointTrans(SMnode *pMnode, SVgroupChangeInfo *pChangeInfo) { void *pKey = taosHashGetKey(pDb, &len); char *p = strndup(pKey, len); - mDebug("clear checkpoint trans in Db:%s", p); - doKillCheckpointTrans(pMnode, pKey, len); + int32_t code = doKillCheckpointTrans(pMnode, pKey, len); + if (code) { + mError("failed to kill trans, transId:%p", pKey) + } else { + mDebug("clear checkpoint trans in Db:%s", p); + } taosMemoryFree(p); } diff --git a/source/dnode/mnode/impl/src/mndStreamUtil.c b/source/dnode/mnode/impl/src/mndStreamUtil.c index 0b96626536..e250034184 100644 --- a/source/dnode/mnode/impl/src/mndStreamUtil.c +++ b/source/dnode/mnode/impl/src/mndStreamUtil.c @@ -1203,13 +1203,17 @@ void mndClearConsensusRspEntry(SCheckpointConsensusInfo* pInfo) { } int64_t mndClearConsensusCheckpointId(SHashObj* pHash, int64_t streamId) { - int32_t code = taosHashRemove(pHash, &streamId, sizeof(streamId)); + int32_t code = 0; + int32_t numOfStreams = taosHashGetSize(pHash); + if (numOfStreams == 0) { + return TSDB_CODE_SUCCESS; + } + + code = taosHashRemove(pHash, &streamId, sizeof(streamId)); if (code == 0) { - int32_t numOfStreams = taosHashGetSize(pHash); - mDebug("drop stream:0x%" PRIx64 " in consensus-checkpointId list after new checkpoint generated, remain:%d", - streamId, numOfStreams); + mDebug("drop stream:0x%" PRIx64 " in consensus-checkpointId list, remain:%d", streamId, numOfStreams); } else { - mError("failed to remove stream:0x%"PRIx64" in consensus-checkpointId list", streamId); + mError("failed to remove stream:0x%"PRIx64" in consensus-checkpointId list, remain:%d", streamId, numOfStreams); } return code; diff --git a/source/dnode/vnode/inc/vnode.h b/source/dnode/vnode/inc/vnode.h index 990d03f940..6dc777b6df 100644 --- a/source/dnode/vnode/inc/vnode.h +++ b/source/dnode/vnode/inc/vnode.h @@ -185,7 +185,7 @@ int32_t tsdbCacherowsReaderOpen(void *pVnode, int32_t type, void *pTableIdList, SArray *pFuncTypeList, SColumnInfo *pkCol, int32_t numOfPks); int32_t tsdbRetrieveCacheRows(void *pReader, SSDataBlock *pResBlock, const int32_t *slotIds, const int32_t *dstSlotIds, SArray *pTableUids); -void *tsdbCacherowsReaderClose(void *pReader); +void tsdbCacherowsReaderClose(void *pReader); void tsdbCacheSetCapacity(SVnode *pVnode, size_t capacity); size_t tsdbCacheGetCapacity(SVnode *pVnode); diff --git a/source/dnode/vnode/src/tsdb/tsdbCacheRead.c b/source/dnode/vnode/src/tsdb/tsdbCacheRead.c index 46c3ba4785..e4f17a8b3c 100644 --- a/source/dnode/vnode/src/tsdb/tsdbCacheRead.c +++ b/source/dnode/vnode/src/tsdb/tsdbCacheRead.c @@ -23,20 +23,27 @@ #define HASTYPE(_type, _t) (((_type) & (_t)) == (_t)) -static void setFirstLastResColToNull(SColumnInfoData* pCol, int32_t row) { - char* buf = taosMemoryCalloc(1, pCol->info.bytes); +static int32_t setFirstLastResColToNull(SColumnInfoData* pCol, int32_t row) { + char* buf = taosMemoryCalloc(1, pCol->info.bytes); + if (buf == NULL) { + return TSDB_CODE_OUT_OF_MEMORY; + } + SFirstLastRes* pRes = (SFirstLastRes*)((char*)buf + VARSTR_HEADER_SIZE); pRes->bytes = 0; pRes->hasResult = true; pRes->isNull = true; varDataSetLen(buf, pCol->info.bytes - VARSTR_HEADER_SIZE); - colDataSetVal(pCol, row, buf, false); + int32_t code = colDataSetVal(pCol, row, buf, false); taosMemoryFree(buf); + + return code; } -static void saveOneRowForLastRaw(SLastCol* pColVal, SCacheRowsReader* pReader, const int32_t slotId, +static int32_t saveOneRowForLastRaw(SLastCol* pColVal, SCacheRowsReader* pReader, const int32_t slotId, SColumnInfoData* pColInfoData, int32_t numOfRows) { SColVal* pVal = &pColVal->colVal; + int32_t code = 0; // allNullRow = false; if (IS_VAR_DATA_TYPE(pColVal->colVal.value.type)) { @@ -46,17 +53,19 @@ static void saveOneRowForLastRaw(SLastCol* pColVal, SCacheRowsReader* pReader, c varDataSetLen(pReader->transferBuf[slotId], pVal->value.nData); memcpy(varDataVal(pReader->transferBuf[slotId]), pVal->value.pData, pVal->value.nData); - colDataSetVal(pColInfoData, numOfRows, pReader->transferBuf[slotId], false); + code = colDataSetVal(pColInfoData, numOfRows, pReader->transferBuf[slotId], false); } } else { - colDataSetVal(pColInfoData, numOfRows, (const char*)&pVal->value.val, !COL_VAL_IS_VALUE(pVal)); + code = colDataSetVal(pColInfoData, numOfRows, (const char*)&pVal->value.val, !COL_VAL_IS_VALUE(pVal)); } - return; + + return code; } static int32_t saveOneRow(SArray* pRow, SSDataBlock* pBlock, SCacheRowsReader* pReader, const int32_t* slotIds, const int32_t* dstSlotIds, void** pRes, const char* idStr) { int32_t numOfRows = pBlock->info.rows; + int32_t code = 0; if (HASTYPE(pReader->type, CACHESCAN_RETRIEVE_LAST)) { uint64_t ts = TSKEY_MIN; @@ -64,14 +73,33 @@ static int32_t saveOneRow(SArray* pRow, SSDataBlock* pBlock, SCacheRowsReader* p col_id_t colId = -1; SArray* funcTypeBlockArray = taosArrayInit(pReader->numOfCols, sizeof(int32_t)); + if (funcTypeBlockArray == NULL) { + return TSDB_CODE_OUT_OF_MEMORY; + } for (int32_t i = 0; i < pReader->numOfCols; ++i) { SColumnInfoData* pColInfoData = taosArrayGet(pBlock->pDataBlock, dstSlotIds[i]); - int32_t funcType = FUNCTION_TYPE_CACHE_LAST; + if (pColInfoData == NULL) { + return TSDB_CODE_INVALID_PARA; + } + int32_t funcType = FUNCTION_TYPE_CACHE_LAST; if (pReader->pFuncTypeList != NULL && taosArrayGetSize(pReader->pFuncTypeList) > i) { - funcType = *(int32_t*)taosArrayGet(pReader->pFuncTypeList, i); - taosArrayInsert(funcTypeBlockArray, dstSlotIds[i], taosArrayGet(pReader->pFuncTypeList, i)); + void* pVal = taosArrayGet(pReader->pFuncTypeList, i); + if (pVal == NULL) { + return TSDB_CODE_INVALID_PARA; + } + funcType = *(int32_t*) pVal; + + pVal = taosArrayGet(pReader->pFuncTypeList, i); + if (pVal == NULL) { + return TSDB_CODE_INVALID_PARA; + } + + void* px = taosArrayInsert(funcTypeBlockArray, dstSlotIds[i], pVal); + if (px == NULL) { + return TSDB_CODE_OUT_OF_MEMORY; + } } if (slotIds[i] == -1) { @@ -79,15 +107,27 @@ static int32_t saveOneRow(SArray* pRow, SSDataBlock* pBlock, SCacheRowsReader* p colDataSetNULL(pColInfoData, numOfRows); continue; } - setFirstLastResColToNull(pColInfoData, numOfRows); + + code = setFirstLastResColToNull(pColInfoData, numOfRows); + if (code) { + return code; + } continue; } + int32_t slotId = slotIds[i]; SLastCol* pColVal = (SLastCol*)taosArrayGet(pRow, i); - colId = pColVal->colVal.cid; + if (pColVal == NULL) { + return TSDB_CODE_INVALID_PARA; + } + colId = pColVal->colVal.cid; if (FUNCTION_TYPE_CACHE_LAST_ROW == funcType) { - saveOneRowForLastRaw(pColVal, pReader, slotId, pColInfoData, numOfRows); + code = saveOneRowForLastRaw(pColVal, pReader, slotId, pColInfoData, numOfRows); + if (code) { + return code; + } + continue; } @@ -112,13 +152,25 @@ static int32_t saveOneRow(SArray* pRow, SSDataBlock* pBlock, SCacheRowsReader* p // pColInfoData->info.bytes includes the VARSTR_HEADER_SIZE, need to subtract it p->hasResult = true; varDataSetLen(pRes[i], pColInfoData->info.bytes - VARSTR_HEADER_SIZE); - colDataSetVal(pColInfoData, numOfRows, (const char*)pRes[i], false); + code = colDataSetVal(pColInfoData, numOfRows, (const char*)pRes[i], false); + if (code) { + return code; + } } for (int32_t idx = 0; idx < taosArrayGetSize(pBlock->pDataBlock); ++idx) { SColumnInfoData* pCol = taosArrayGet(pBlock->pDataBlock, idx); + if (pCol == NULL) { + return TSDB_CODE_INVALID_PARA; + } + if (idx < funcTypeBlockArray->size) { - int32_t funcType = *(int32_t*)taosArrayGet(funcTypeBlockArray, idx); + void* pVal = taosArrayGet(funcTypeBlockArray, idx); + if (pVal == NULL) { + return TSDB_CODE_INVALID_PARA; + } + + int32_t funcType = *(int32_t*)pVal; if (FUNCTION_TYPE_CACHE_LAST_ROW == funcType) { continue; } @@ -128,12 +180,18 @@ static int32_t saveOneRow(SArray* pRow, SSDataBlock* pBlock, SCacheRowsReader* p if (ts == TSKEY_MIN) { colDataSetNULL(pCol, numOfRows); } else { - colDataSetVal(pCol, numOfRows, (const char*)&ts, false); + code = colDataSetVal(pCol, numOfRows, (const char*)&ts, false); + if (code) { + return code; + } } continue; } else if (pReader->numOfCols == 1 && idx != dstSlotIds[0] && (pCol->info.colId == colId || colId == -1)) { if (p && !p->isNull) { - colDataSetVal(pCol, numOfRows, p->buf, false); + code = colDataSetVal(pCol, numOfRows, p->buf, false); + if (code) { + return code; + } } else { colDataSetNULL(pCol, numOfRows); } @@ -146,15 +204,25 @@ static int32_t saveOneRow(SArray* pRow, SSDataBlock* pBlock, SCacheRowsReader* p } else if (HASTYPE(pReader->type, CACHESCAN_RETRIEVE_LAST_ROW)) { for (int32_t i = 0; i < pReader->numOfCols; ++i) { SColumnInfoData* pColInfoData = taosArrayGet(pBlock->pDataBlock, dstSlotIds[i]); + if (pColInfoData == NULL) { + return TSDB_CODE_INVALID_PARA; + } int32_t slotId = slotIds[i]; if (slotId == -1) { colDataSetNULL(pColInfoData, numOfRows); continue; } - SLastCol* pColVal = (SLastCol*)taosArrayGet(pRow, i); - saveOneRowForLastRaw(pColVal, pReader, slotId, pColInfoData, numOfRows); + SLastCol* pColVal = (SLastCol*)taosArrayGet(pRow, i); + if (pColVal == NULL) { + return TSDB_CODE_INVALID_PARA; + } + + code = saveOneRowForLastRaw(pColVal, pReader, slotId, pColInfoData, numOfRows); + if (code) { + return code; + } } // pBlock->info.rows += allNullRow ? 0 : 1; @@ -164,7 +232,7 @@ static int32_t saveOneRow(SArray* pRow, SSDataBlock* pBlock, SCacheRowsReader* p return TSDB_CODE_INVALID_PARA; } - return TSDB_CODE_SUCCESS; + return code; } static int32_t setTableSchema(SCacheRowsReader* p, uint64_t suid, const char* idstr) { @@ -206,7 +274,7 @@ int32_t tsdbReuseCacherowsReader(void* reader, void* pTableIdList, int32_t numOf destroySttBlockReader(pReader->pLDataIterArray, NULL); pReader->pLDataIterArray = taosArrayInit(4, POINTER_BYTES); - return TSDB_CODE_SUCCESS; + return (pReader->pLDataIterArray != NULL) ? TSDB_CODE_SUCCESS : TSDB_CODE_OUT_OF_MEMORY; } int32_t tsdbCacherowsReaderOpen(void* pVnode, int32_t type, void* pTableIdList, int32_t numOfTables, int32_t numOfCols, @@ -269,18 +337,22 @@ int32_t tsdbCacherowsReaderOpen(void* pVnode, int32_t type, void* pTableIdList, } p->idstr = taosStrdup(idstr); - taosThreadMutexInit(&p->readerMutex, NULL); + code = taosThreadMutexInit(&p->readerMutex, NULL); + if (code) { + tsdbCacherowsReaderClose(p); + return code; + } p->lastTs = INT64_MIN; *pReader = p; - return TSDB_CODE_SUCCESS; + return code; } -void* tsdbCacherowsReaderClose(void* pReader) { +void tsdbCacherowsReaderClose(void* pReader) { SCacheRowsReader* p = pReader; if (p == NULL) { - return NULL; + return; } if (p->pSchema != NULL) { @@ -299,12 +371,12 @@ void* tsdbCacherowsReaderClose(void* pReader) { } if (p->pFileReader) { - tsdbDataFileReaderClose(&p->pFileReader); + (void) tsdbDataFileReaderClose(&p->pFileReader); p->pFileReader = NULL; } taosMemoryFree((void*)p->idstr); - taosThreadMutexDestroy(&p->readerMutex); + (void) taosThreadMutexDestroy(&p->readerMutex); if (p->pTableMap) { void* pe = NULL; @@ -322,7 +394,6 @@ void* tsdbCacherowsReaderClose(void* pReader) { } taosMemoryFree(pReader); - return NULL; } static void freeItem(void* pItem) { @@ -342,8 +413,7 @@ static int32_t tsdbCacheQueryReseek(void* pQHandle) { // just wait for the big all tables' snapshot untaking for now code = TSDB_CODE_VND_QUERY_BUSY; - - taosThreadMutexUnlock(&pReader->readerMutex); + (void)taosThreadMutexUnlock(&pReader->readerMutex); return code; } else if (code == EBUSY) { @@ -360,9 +430,14 @@ int32_t tsdbRetrieveCacheRows(void* pReader, SSDataBlock* pResBlock, const int32 } SCacheRowsReader* pr = pReader; - int32_t code = TSDB_CODE_SUCCESS; - SArray* pRow = taosArrayInit(TARRAY_SIZE(pr->pCidList), sizeof(SLastCol)); - bool hasRes = false; + + int32_t code = TSDB_CODE_SUCCESS; + bool hasRes = false; + SArray* pRow = taosArrayInit(TARRAY_SIZE(pr->pCidList), sizeof(SLastCol)); + if (pRow == NULL) { + code = TSDB_CODE_OUT_OF_MEMORY; + goto _end; + } void** pRes = taosMemoryCalloc(pr->numOfCols, POINTER_BYTES); if (pRes == NULL) { @@ -370,7 +445,7 @@ int32_t tsdbRetrieveCacheRows(void* pReader, SSDataBlock* pResBlock, const int32 goto _end; } - int32_t pkBufLen = (pr->rowKey.numOfPKs > 0)? pr->pkColumn.bytes:0; + int32_t pkBufLen = (pr->rowKey.numOfPKs > 0) ? pr->pkColumn.bytes : 0; for (int32_t j = 0; j < pr->numOfCols; ++j) { int32_t bytes = (slotIds[j] == -1) ? 1 : pr->pSchema->columns[slotIds[j]].bytes; @@ -379,7 +454,7 @@ int32_t tsdbRetrieveCacheRows(void* pReader, SSDataBlock* pResBlock, const int32 p->ts = INT64_MIN; } - taosThreadMutexLock(&pr->readerMutex); + (void)taosThreadMutexLock(&pr->readerMutex); code = tsdbTakeReadSnap2((STsdbReader*)pr, tsdbCacheQueryReseek, &pr->pReadSnap); if (code != TSDB_CODE_SUCCESS) { goto _end; @@ -401,7 +476,11 @@ int32_t tsdbRetrieveCacheRows(void* pReader, SSDataBlock* pResBlock, const int32 int32_t slotId = slotIds[i]; if (slotId == -1) { SLastCol p = {.rowKey.ts = INT64_MIN, .colVal.value.type = TSDB_DATA_TYPE_BOOL, .colVal.flag = CV_FLAG_NULL}; - taosArrayPush(pLastCols, &p); + void* px = taosArrayPush(pLastCols, &p); + if (px == NULL) { + code = TSDB_CODE_OUT_OF_MEMORY; + goto _end; + } continue; } struct STColumn* pCol = &pr->pSchema->columns[slotId]; @@ -409,8 +488,17 @@ int32_t tsdbRetrieveCacheRows(void* pReader, SSDataBlock* pResBlock, const int32 if (IS_VAR_DATA_TYPE(pCol->type)) { p.colVal.value.pData = taosMemoryCalloc(pCol->bytes, sizeof(char)); + if (p.colVal.value.pData == NULL) { + code = TSDB_CODE_OUT_OF_MEMORY; + goto _end; + } + } + + void* px = taosArrayPush(pLastCols, &p); + if (px == NULL) { + code = TSDB_CODE_OUT_OF_MEMORY; + goto _end; } - taosArrayPush(pLastCols, &p); } int64_t st = taosGetTimestampUs(); @@ -418,7 +506,11 @@ int32_t tsdbRetrieveCacheRows(void* pReader, SSDataBlock* pResBlock, const int32 for (int32_t i = 0; i < pr->numOfTables; ++i) { tb_uid_t uid = pTableList[i].uid; - tsdbCacheGetBatch(pr->pTsdb, uid, pRow, pr, ltype); + code = tsdbCacheGetBatch(pr->pTsdb, uid, pRow, pr, ltype); + if (code) { + goto _end; + } + if (TARRAY_SIZE(pRow) <= 0 || COL_VAL_IS_NONE(&((SLastCol*)TARRAY_DATA(pRow))[0].colVal)) { taosArrayClearEx(pRow, freeItem); continue; @@ -430,7 +522,14 @@ int32_t tsdbRetrieveCacheRows(void* pReader, SSDataBlock* pResBlock, const int32 for (int32_t k = 0; k < pr->numOfCols; ++k) { if (slotIds[k] == -1) continue; SLastCol* p = taosArrayGet(pLastCols, k); + if (p == NULL) { + return TSDB_CODE_INVALID_PARA; + } + SLastCol* pColVal = (SLastCol*)taosArrayGet(pRow, k); + if (pColVal == NULL) { + return TSDB_CODE_INVALID_PARA; + } if (pColVal->rowKey.ts > p->rowKey.ts) { if (!COL_VAL_IS_VALUE(&pColVal->colVal) && HASTYPE(pr->type, CACHESCAN_RETRIEVE_LAST)) { @@ -447,7 +546,11 @@ int32_t tsdbRetrieveCacheRows(void* pReader, SSDataBlock* pResBlock, const int32 p->rowKey.ts = pColVal->rowKey.ts; if (k == 0) { if (TARRAY_SIZE(pTableUidList) == 0) { - taosArrayPush(pTableUidList, &uid); + void* px = taosArrayPush(pTableUidList, &uid); + if (px == NULL) { + code = TSDB_CODE_OUT_OF_MEMORY; + goto _end; + } } else { taosArraySet(pTableUidList, 0, &uid); } @@ -487,7 +590,10 @@ int32_t tsdbRetrieveCacheRows(void* pReader, SSDataBlock* pResBlock, const int32 } if (hasRes) { - saveOneRow(pLastCols, pResBlock, pr, slotIds, dstSlotIds, pRes, pr->idstr); + code = saveOneRow(pLastCols, pResBlock, pr, slotIds, dstSlotIds, pRes, pr->idstr); + if (code) { + goto _end; + } } taosArrayDestroyEx(pLastCols, freeItem); @@ -495,16 +601,27 @@ int32_t tsdbRetrieveCacheRows(void* pReader, SSDataBlock* pResBlock, const int32 for (int32_t i = pr->tableIndex; i < pr->numOfTables; ++i) { tb_uid_t uid = pTableList[i].uid; - tsdbCacheGetBatch(pr->pTsdb, uid, pRow, pr, ltype); + if ((code = tsdbCacheGetBatch(pr->pTsdb, uid, pRow, pr, ltype)) != 0) { + goto _end; + } + if (TARRAY_SIZE(pRow) <= 0 || COL_VAL_IS_NONE(&((SLastCol*)TARRAY_DATA(pRow))[0].colVal)) { taosArrayClearEx(pRow, freeItem); continue; } - saveOneRow(pRow, pResBlock, pr, slotIds, dstSlotIds, pRes, pr->idstr); + code = saveOneRow(pRow, pResBlock, pr, slotIds, dstSlotIds, pRes, pr->idstr); + if (code) { + goto _end; + } + taosArrayClearEx(pRow, freeItem); - taosArrayPush(pTableUidList, &uid); + void* px = taosArrayPush(pTableUidList, &uid); + if (px == NULL) { + code = TSDB_CODE_OUT_OF_MEMORY; + goto _end; + } ++pr->tableIndex; if (pResBlock->info.rows >= pResBlock->info.capacity) { @@ -521,7 +638,7 @@ _end: pr->pCurFileSet = NULL; } - taosThreadMutexUnlock(&pr->readerMutex); + (void)taosThreadMutexUnlock(&pr->readerMutex); if (pRes != NULL) { for (int32_t j = 0; j < pr->numOfCols; ++j) { diff --git a/source/dnode/vnode/src/tsdb/tsdbRead2.c b/source/dnode/vnode/src/tsdb/tsdbRead2.c index 23e396b6a4..95ae9bcdf0 100644 --- a/source/dnode/vnode/src/tsdb/tsdbRead2.c +++ b/source/dnode/vnode/src/tsdb/tsdbRead2.c @@ -48,7 +48,7 @@ typedef struct { static int32_t getCurrentBlockInfo(SDataBlockIter* pBlockIter, SFileDataBlockInfo** pInfo); static int32_t buildDataBlockFromBufImpl(STableBlockScanInfo* pBlockScanInfo, int64_t endKey, int32_t capacity, STsdbReader* pReader); -static int32_t getValidMemRow(SIterInfo* pIter, const SArray* pDelList, STsdbReader* pReader, TSDBROW** pRow); +static int32_t getValidMemRow(SIterInfo* pIter, const SArray* pDelList, STsdbReader* pReader, TSDBROW** pRes); static int32_t doMergeRowsInFileBlocks(SBlockData* pBlockData, STableBlockScanInfo* pScanInfo, SRowKey* pKey, STsdbReader* pReader); static int32_t doMergeRowsInSttBlock(SSttBlockReader* pSttBlockReader, STableBlockScanInfo* pScanInfo, @@ -73,8 +73,8 @@ static int32_t mergeRowsInSttBlocks(SSttBlockReader* pSttBlockReader, STableBloc STsdbReader* pReader); static int32_t initDelSkylineIterator(STableBlockScanInfo* pBlockScanInfo, int32_t order, SReadCostSummary* pCost); -static STsdb* getTsdbByRetentions(SVnode* pVnode, SQueryTableDataCond* pCond, SRetention* retentions, const char* idstr, - int8_t* pLevel); +static void getTsdbByRetentions(SVnode* pVnode, SQueryTableDataCond* pCond, SRetention* retentions, const char* idstr, + int8_t* pLevel, STsdb** pTsdb); static SVersionRange getQueryVerRange(SVnode* pVnode, SQueryTableDataCond* pCond, int8_t level); static int32_t doBuildDataBlock(STsdbReader* pReader); static TSDBKEY getCurrentKeyInBuf(STableBlockScanInfo* pScanInfo, STsdbReader* pReader); @@ -123,7 +123,7 @@ static void tColRowGetPriamyKeyDeepCopy(SBlockData* pBlock, int32_t irow, int32_ pKey->pks[0].val = cv.value.val; } else { pKey->pks[0].nData = cv.value.nData; - (void)(void)memcpy(pKey->pks[0].pData, cv.value.pData, cv.value.nData); + (void)memcpy(pKey->pks[0].pData, cv.value.pData, cv.value.nData); } } @@ -439,7 +439,6 @@ static int32_t tsdbUninitReaderLock(STsdbReader* pReader) { tsdbTrace("tsdb/read: %p, pre-uninit read mutex: %p, code: %d", pReader, &pReader->readerMutex, code); code = taosThreadMutexDestroy(&pReader->readerMutex); - tsdbTrace("tsdb/read: %p, post-uninit read mutex: %p, code: %d", pReader, &pReader->readerMutex, code); return code; @@ -450,7 +449,6 @@ static int32_t tsdbAcquireReader(STsdbReader* pReader) { tsdbTrace("tsdb/read: %p, pre-take read mutex: %p, code: %d", pReader, &pReader->readerMutex, code); code = taosThreadMutexLock(&pReader->readerMutex); - tsdbTrace("tsdb/read: %p, post-take read mutex: %p, code: %d", pReader, &pReader->readerMutex, code); return code; @@ -541,8 +539,7 @@ static int32_t tsdbReaderCreate(SVnode* pVnode, SQueryTableDataCond* pCond, void } initReaderStatus(&pReader->status); - - pReader->pTsdb = getTsdbByRetentions(pVnode, pCond, pVnode->config.tsdbCfg.retentions, idstr, &level); + getTsdbByRetentions(pVnode, pCond, pVnode->config.tsdbCfg.retentions, idstr, &level, &pReader->pTsdb); pReader->info.suid = pCond->suid; pReader->info.order = pCond->order; @@ -550,6 +547,11 @@ static int32_t tsdbReaderCreate(SVnode* pVnode, SQueryTableDataCond* pCond, void pReader->info.window = updateQueryTimeWindow(pReader->pTsdb, &pCond->twindows); pReader->idStr = (idstr != NULL) ? taosStrdup(idstr) : NULL; + if (pReader->idStr == NULL) { + code = TSDB_CODE_OUT_OF_MEMORY; + goto _end; + } + pReader->type = pCond->type; pReader->bFilesetDelimited = false; pReader->blockInfoBuf.numPerBucket = 1000; // 1000 tables per bucket @@ -586,6 +588,11 @@ static int32_t tsdbReaderCreate(SVnode* pVnode, SQueryTableDataCond* pCond, void } pReader->status.pPrimaryTsCol = taosArrayGet(pReader->resBlockInfo.pResBlock->pDataBlock, pSup->slotId[0]); + if (pReader->status.pPrimaryTsCol == NULL) { + code = TSDB_CODE_INVALID_PARA; + goto _end; + } + int32_t type = pReader->status.pPrimaryTsCol->info.type; if (type != TSDB_DATA_TYPE_TIMESTAMP) { tsdbError("the first column isn't primary timestamp in result block, actual: %s, %s", tDataTypes[type].name, @@ -651,7 +658,6 @@ static int32_t doLoadBlockIndex(STsdbReader* pReader, SDataFileReader* pFileRead STableUidList* pList = &pReader->status.uidList; int32_t i = 0; - while (i < TARRAY2_SIZE(pBlkArray)) { pBrinBlk = &pBlkArray->data[i]; if (pBrinBlk->maxTbid.suid < pReader->info.suid) { @@ -824,6 +830,10 @@ static int32_t loadFileBlockBrinInfo(STsdbReader* pReader, SArray* pIndexList, S p1 = taosArrayPush(pTableScanInfoList, &pScanInfo); } else { STableBlockScanInfo** p = taosArrayGetLast(pTableScanInfoList); + if (p == NULL) { + return TSDB_CODE_INVALID_PARA; + } + if ((*p)->uid != uid) { p1 = taosArrayPush(pTableScanInfoList, &pScanInfo); } @@ -1273,6 +1283,10 @@ static int32_t copyBlockDataToSDataBlock(STsdbReader* pReader, SRowKey* pLastPro int32_t rowIndex = 0; SColumnInfoData* pColData = taosArrayGet(pResBlock->pDataBlock, pSupInfo->slotId[i]); + if (pColData == NULL) { + return TSDB_CODE_INVALID_PARA; + } + if (pSupInfo->colId[i] == PRIMARYKEY_TIMESTAMP_COL_ID) { copyPrimaryTsCol(pBlockData, pDumpInfo, pColData, dumpedRows, asc); i += 1; @@ -1288,6 +1302,9 @@ static int32_t copyBlockDataToSDataBlock(STsdbReader* pReader, SRowKey* pLastPro colIndex += 1; } else if (pData->cid == pSupInfo->colId[i]) { pColData = taosArrayGet(pResBlock->pDataBlock, pSupInfo->slotId[i]); + if (pColData == NULL) { + return TSDB_CODE_INVALID_PARA; + } if (pData->flag == HAS_NONE || pData->flag == HAS_NULL || pData->flag == (HAS_NULL | HAS_NONE)) { colDataSetNNULL(pColData, 0, dumpedRows); @@ -1309,6 +1326,10 @@ static int32_t copyBlockDataToSDataBlock(STsdbReader* pReader, SRowKey* pLastPro i += 1; } else { // the specified column does not exist in file block, fill with null data pColData = taosArrayGet(pResBlock->pDataBlock, pSupInfo->slotId[i]); + if (pColData == NULL) { + return TSDB_CODE_INVALID_PARA; + } + colDataSetNNULL(pColData, 0, dumpedRows); i += 1; } @@ -1317,6 +1338,10 @@ static int32_t copyBlockDataToSDataBlock(STsdbReader* pReader, SRowKey* pLastPro // fill the mis-matched columns with null value while (i < numOfOutputCols) { pColData = taosArrayGet(pResBlock->pDataBlock, pSupInfo->slotId[i]); + if (pColData == NULL) { + return TSDB_CODE_INVALID_PARA; + } + colDataSetNNULL(pColData, 0, dumpedRows); i += 1; } @@ -1448,7 +1473,15 @@ static bool getNeighborBlockOfTable(SDataBlockIter* pBlockIter, SFileDataBlockIn } STableDataBlockIdx* pTableDataBlockIdx = taosArrayGet(pScanInfo->pBlockIdxList, pBlockInfo->tbBlockIdx + step); + if (pTableDataBlockIdx == NULL) { + return TSDB_CODE_INVALID_PARA; + } + SFileDataBlockInfo* p = taosArrayGet(pBlockIter->blockList, pTableDataBlockIdx->globalIndex); + if (p == NULL) { + return TSDB_CODE_INVALID_PARA; + } + blockInfoToRecord(pRecord, p, pSupInfo); *nextIndex = pBlockInfo->tbBlockIdx + step; @@ -1462,13 +1495,21 @@ static int32_t setFileBlockActiveInBlockIter(STsdbReader* pReader, SDataBlockIte return TSDB_CODE_FAILED; } - SFileDataBlockInfo fblock = *(SFileDataBlockInfo*)taosArrayGet(pBlockIter->blockList, index); + void* p = taosArrayGet(pBlockIter->blockList, index); + if (p == NULL) { + return TSDB_CODE_INVALID_PARA; + } + + SFileDataBlockInfo fblock = *(SFileDataBlockInfo*) p; pBlockIter->index += step; if (index != pBlockIter->index) { if (index > pBlockIter->index) { for (int32_t i = index - 1; i >= pBlockIter->index; --i) { SFileDataBlockInfo* pBlockInfo = taosArrayGet(pBlockIter->blockList, i); + if (pBlockInfo == NULL) { + return TSDB_CODE_INVALID_PARA; + } STableBlockScanInfo* pBlockScanInfo = NULL; code = getTableBlockScanInfo(pReader->status.pTableMap, pBlockInfo->uid, &pBlockScanInfo, pReader->idStr); @@ -1477,6 +1518,10 @@ static int32_t setFileBlockActiveInBlockIter(STsdbReader* pReader, SDataBlockIte } STableDataBlockIdx* pTableDataBlockIdx = taosArrayGet(pBlockScanInfo->pBlockIdxList, pBlockInfo->tbBlockIdx); + if (pTableDataBlockIdx == NULL) { + return TSDB_CODE_INVALID_PARA; + } + pTableDataBlockIdx->globalIndex = i + 1; taosArraySet(pBlockIter->blockList, i + 1, pBlockInfo); @@ -1484,6 +1529,9 @@ static int32_t setFileBlockActiveInBlockIter(STsdbReader* pReader, SDataBlockIte } else if (index < pBlockIter->index) { for (int32_t i = index + 1; i <= pBlockIter->index; ++i) { SFileDataBlockInfo* pBlockInfo = taosArrayGet(pBlockIter->blockList, i); + if (pBlockInfo == NULL) { + return TSDB_CODE_INVALID_PARA; + } STableBlockScanInfo* pBlockScanInfo = NULL; code = getTableBlockScanInfo(pReader->status.pTableMap, pBlockInfo->uid, &pBlockScanInfo, pReader->idStr); @@ -1492,8 +1540,11 @@ static int32_t setFileBlockActiveInBlockIter(STsdbReader* pReader, SDataBlockIte } STableDataBlockIdx* pTableDataBlockIdx = taosArrayGet(pBlockScanInfo->pBlockIdxList, pBlockInfo->tbBlockIdx); - pTableDataBlockIdx->globalIndex = i - 1; + if (pTableDataBlockIdx == NULL) { + return TSDB_CODE_INVALID_PARA; + } + pTableDataBlockIdx->globalIndex = i - 1; taosArraySet(pBlockIter->blockList, i - 1, pBlockInfo); } } @@ -1505,7 +1556,11 @@ static int32_t setFileBlockActiveInBlockIter(STsdbReader* pReader, SDataBlockIte return code; } - STableDataBlockIdx* pTableDataBlockIdx = taosArrayGet(pBlockScanInfo->pBlockIdxList, fblock.tbBlockIdx); + STableDataBlockIdx* pTableDataBlockIdx = taosArrayGet(pBlockScanInfo->pBlockIdxList, fblock.tbBlockIdx); + if (pTableDataBlockIdx == NULL) { + return TSDB_CODE_INVALID_PARA; + } + pTableDataBlockIdx->globalIndex = pBlockIter->index; } @@ -2338,7 +2393,7 @@ static bool initSttBlockReader(SSttBlockReader* pSttBlockReader, STableBlockScan pSttBlockReader->uid = pScanInfo->uid; // second time init stt block reader - if (pScanInfo->cleanSttBlocks && pReader->info.execMode == READER_EXEC_ROWS) { + if (pScanInfo->cleanSttBlocks && (pReader->info.execMode == READER_EXEC_ROWS)) { return !pScanInfo->sttBlockReturned; } @@ -2374,6 +2429,9 @@ static bool initSttBlockReader(SSttBlockReader* pSttBlockReader, STableBlockScan }; SSttDataInfoForTable info = {.pKeyRangeList = taosArrayInit(4, sizeof(SSttKeyRange))}; + if (info.pKeyRangeList == NULL) { + return false; + } int32_t code = tMergeTreeOpen2(&pSttBlockReader->mergeTree, &conf, &info); if (code != TSDB_CODE_SUCCESS) { @@ -2398,6 +2456,10 @@ static bool initSttBlockReader(SSttBlockReader* pSttBlockReader, STableBlockScan // calculate the time window for data in stt files for (int32_t i = 0; i < taosArrayGetSize(info.pKeyRangeList); ++i) { SSttKeyRange* pKeyRange = taosArrayGet(info.pKeyRangeList, i); + if (pKeyRange == NULL) { + return TSDB_CODE_INVALID_PARA; + } + if (pkCompEx(&pScanInfo->sttRange.skey, &pKeyRange->skey) > 0) { tRowKeyAssign(&pScanInfo->sttRange.skey, &pKeyRange->skey); } @@ -2618,6 +2680,9 @@ static int32_t loadNeighborIfOverlap(SFileDataBlockInfo* pBlockInfo, STableBlock // 1. find the next neighbor block in the scan block list STableDataBlockIdx* tableDataBlockIdx = taosArrayGet(pBlockScanInfo->pBlockIdxList, nextIndex); + if (tableDataBlockIdx == NULL) { + return TSDB_CODE_INVALID_PARA; + } // 2. remove it from the scan block list int32_t neighborIndex = tableDataBlockIdx->globalIndex; @@ -2787,6 +2852,9 @@ int32_t initDelSkylineIterator(STableBlockScanInfo* pBlockScanInfo, int32_t orde taosArrayClear(pBlockScanInfo->delSkyline); } else { pBlockScanInfo->delSkyline = taosArrayInit(4, sizeof(TSDBKEY)); + if (pBlockScanInfo->delSkyline == NULL) { + return TSDB_CODE_OUT_OF_MEMORY; + } } SArray* pSource = pBlockScanInfo->pFileDelData; @@ -3412,6 +3480,9 @@ static void initBlockDumpInfo(STsdbReader* pReader, SDataBlockIter* pBlockIter) static int32_t initForFirstBlockInFile(STsdbReader* pReader, SDataBlockIter* pBlockIter) { SBlockNumber num = {0}; SArray* pTableList = taosArrayInit(40, POINTER_BYTES); + if (pTableList == NULL) { + return TSDB_CODE_OUT_OF_MEMORY; + } int32_t code = moveToNextFile(pReader, &num, pTableList); if (code != TSDB_CODE_SUCCESS) { @@ -3563,8 +3634,13 @@ static int32_t buildBlockFromFiles(STsdbReader* pReader) { } } -static STsdb* getTsdbByRetentions(SVnode* pVnode, SQueryTableDataCond* pCond, SRetention* retentions, const char* idStr, - int8_t* pLevel) { +static void getTsdbByRetentions(SVnode* pVnode, SQueryTableDataCond* pCond, SRetention* retentions, const char* idStr, + int8_t* pLevel, STsdb** pTsdb) { + if (pTsdb == NULL) { + return; + } + + *pTsdb = NULL; if (VND_IS_RSMA(pVnode) && !pCond->skipRollup) { int8_t level = 0; int8_t precision = pVnode->config.tsdbCfg.precision; @@ -3573,7 +3649,7 @@ static STsdb* getTsdbByRetentions(SVnode* pVnode, SQueryTableDataCond* pCond, SR : (precision == TSDB_TIME_PRECISION_MICRO) ? 1000L : 1000000L); - for (int8_t i = 0; i < TSDB_RETENTION_MAX; ++i) { + for (int32_t i = 0; i < TSDB_RETENTION_MAX; ++i) { SRetention* pRetention = retentions + level; if (pRetention->keep <= 0) { if (level > 0) { @@ -3592,19 +3668,19 @@ static STsdb* getTsdbByRetentions(SVnode* pVnode, SQueryTableDataCond* pCond, SR if (level == TSDB_RETENTION_L0) { *pLevel = TSDB_RETENTION_L0; tsdbDebug("vgId:%d, rsma level %d is selected to query %s", TD_VID(pVnode), TSDB_RETENTION_L0, str); - return VND_RSMA0(pVnode); + *pTsdb = VND_RSMA0(pVnode); } else if (level == TSDB_RETENTION_L1) { *pLevel = TSDB_RETENTION_L1; tsdbDebug("vgId:%d, rsma level %d is selected to query %s", TD_VID(pVnode), TSDB_RETENTION_L1, str); - return VND_RSMA1(pVnode); + *pTsdb = VND_RSMA1(pVnode); } else { *pLevel = TSDB_RETENTION_L2; tsdbDebug("vgId:%d, rsma level %d is selected to query %s", TD_VID(pVnode), TSDB_RETENTION_L2, str); - return VND_RSMA2(pVnode); + *pTsdb = VND_RSMA2(pVnode); } } - return VND_TSDB(pVnode); + *pTsdb = VND_TSDB(pVnode); } SVersionRange getQueryVerRange(SVnode* pVnode, SQueryTableDataCond* pCond, int8_t level) { @@ -3631,6 +3707,10 @@ static int32_t reverseSearchStartPos(const SArray* pDelList, int32_t index, int6 } TSDBKEY* p = taosArrayGet(pDelList, start); + if (p == NULL) { + return TSDB_CODE_INVALID_PARA; + } + while (p->ts >= key && start > 0) { start -= 1; } @@ -3640,6 +3720,10 @@ static int32_t reverseSearchStartPos(const SArray* pDelList, int32_t index, int6 } TSDBKEY* p = taosArrayGet(pDelList, start); + if (p == NULL) { + return TSDB_CODE_INVALID_PARA; + } + while (p->ts <= key && start < num - 1) { start += 1; } @@ -3665,17 +3749,27 @@ bool hasBeenDropped(const SArray* pDelList, int32_t* index, int64_t key, int64_t if (asc) { if (*index >= num - 1) { TSDBKEY* last = taosArrayGetLast(pDelList); - ASSERT(key >= last->ts); + if (last == NULL) { + return false; + } + ASSERT(key >= last->ts); if (key > last->ts) { return false; } else if (key == last->ts) { TSDBKEY* prev = taosArrayGet(pDelList, num - 2); + if (prev == NULL) { + return false; + } + return (prev->version >= ver && prev->version <= pVerRange->maxVer && prev->version >= pVerRange->minVer); } } else { TSDBKEY* pCurrent = taosArrayGet(pDelList, *index); TSDBKEY* pNext = taosArrayGet(pDelList, (*index) + 1); + if (pCurrent == NULL || pNext == NULL) { + return false; + } if (key < pCurrent->ts) { return false; @@ -3692,6 +3786,9 @@ bool hasBeenDropped(const SArray* pDelList, int32_t* index, int64_t key, int64_t if ((*index) < num - 1) { pCurrent = taosArrayGet(pDelList, *index); pNext = taosArrayGet(pDelList, (*index) + 1); + if (pCurrent == NULL || pNext == NULL) { + return false; + } // it is not a consecutive deletion range, ignore it if (pCurrent->version == 0 && pNext->version > 0) { @@ -3710,6 +3807,9 @@ bool hasBeenDropped(const SArray* pDelList, int32_t* index, int64_t key, int64_t } else { if (*index <= 0) { TSDBKEY* pFirst = taosArrayGet(pDelList, 0); + if (pFirst == NULL) { + return false; + } if (key < pFirst->ts) { return false; @@ -3721,6 +3821,9 @@ bool hasBeenDropped(const SArray* pDelList, int32_t* index, int64_t key, int64_t } else { TSDBKEY* pCurrent = taosArrayGet(pDelList, *index); TSDBKEY* pPrev = taosArrayGet(pDelList, (*index) - 1); + if (pCurrent == NULL || pPrev == NULL) { + return false; + } if (key > pCurrent->ts) { return false; @@ -3736,6 +3839,9 @@ bool hasBeenDropped(const SArray* pDelList, int32_t* index, int64_t key, int64_t if ((*index) >= 1) { pCurrent = taosArrayGet(pDelList, *index); pPrev = taosArrayGet(pDelList, (*index) - 1); + if (pCurrent == NULL || pPrev == NULL) { + return false; + } // it is not a consecutive deletion range, ignore it if (pCurrent->version > 0 && pPrev->version == 0) { @@ -4229,6 +4335,10 @@ int32_t doAppendRowFromTSRow(SSDataBlock* pBlock, STsdbReader* pReader, SRow* pT if (pSupInfo->colId[i] == PRIMARYKEY_TIMESTAMP_COL_ID) { SColumnInfoData* pColData = taosArrayGet(pBlock->pDataBlock, pSupInfo->slotId[i]); + if (pColData == NULL) { + return TSDB_CODE_INVALID_PARA; + } + ((int64_t*)pColData->pData)[outputRowIndex] = pTSRow->ts; i += 1; } @@ -4238,6 +4348,9 @@ int32_t doAppendRowFromTSRow(SSDataBlock* pBlock, STsdbReader* pReader, SRow* pT if (colId == pSchema->columns[j].colId) { SColumnInfoData* pColInfoData = taosArrayGet(pBlock->pDataBlock, pSupInfo->slotId[i]); + if (pColInfoData == NULL) { + return TSDB_CODE_INVALID_PARA; + } code = tRowGet(pTSRow, pSchema, j, &colVal); if (code) { @@ -4252,6 +4365,9 @@ int32_t doAppendRowFromTSRow(SSDataBlock* pBlock, STsdbReader* pReader, SRow* pT j += 1; } else if (colId < pSchema->columns[j].colId) { SColumnInfoData* pColInfoData = taosArrayGet(pBlock->pDataBlock, pSupInfo->slotId[i]); + if (pColInfoData == NULL) { + return TSDB_CODE_INVALID_PARA; + } colDataSetNULL(pColInfoData, outputRowIndex); i += 1; @@ -4263,6 +4379,10 @@ int32_t doAppendRowFromTSRow(SSDataBlock* pBlock, STsdbReader* pReader, SRow* pT // set null value since current column does not exist in the "pSchema" while (i < pSupInfo->numOfCols) { SColumnInfoData* pColInfoData = taosArrayGet(pBlock->pDataBlock, pSupInfo->slotId[i]); + if (pColInfoData == NULL) { + return TSDB_CODE_INVALID_PARA; + } + colDataSetNULL(pColInfoData, outputRowIndex); i += 1; } @@ -4311,6 +4431,10 @@ int32_t doAppendRowFromFileBlock(SSDataBlock* pResBlock, STsdbReader* pReader, S while (i < numOfOutputCols) { SColumnInfoData* pCol = taosArrayGet(pResBlock->pDataBlock, pSupInfo->slotId[i]); + if (pCol == NULL) { + return TSDB_CODE_INVALID_PARA; + } + colDataSetNULL(pCol, outputRowIndex); i += 1; } diff --git a/source/dnode/vnode/src/tsdb/tsdbReadUtil.c b/source/dnode/vnode/src/tsdb/tsdbReadUtil.c index d7da6cbd9d..64569e63ac 100644 --- a/source/dnode/vnode/src/tsdb/tsdbReadUtil.c +++ b/source/dnode/vnode/src/tsdb/tsdbReadUtil.c @@ -122,7 +122,9 @@ void clearBlockScanInfoBuf(SBlockInfoBuf* pBuf) { size_t num = taosArrayGetSize(pBuf->pData); for (int32_t i = 0; i < num; ++i) { char** p = taosArrayGet(pBuf->pData, i); - taosMemoryFree(*p); + if (p != NULL) { + taosMemoryFree(*p); + } } taosArrayDestroy(pBuf->pData); @@ -446,8 +448,8 @@ void cleanupInfoForNextFileset(SSHashObj* pTableMap) { // brin records iterator void initBrinRecordIter(SBrinRecordIter* pIter, SDataFileReader* pReader, SArray* pList) { - memset(&pIter->block, 0, sizeof(SBrinBlock)); - memset(&pIter->record, 0, sizeof(SBrinRecord)); + (void) memset(&pIter->block, 0, sizeof(SBrinBlock)); + (void) memset(&pIter->record, 0, sizeof(SBrinRecord)); pIter->blockIndex = -1; pIter->recordIndex = -1; @@ -465,6 +467,9 @@ int32_t getNextBrinRecord(SBrinRecordIter* pIter, SBrinRecord** pRecord) { } pIter->pCurrentBlk = taosArrayGet(pIter->pBrinBlockList, pIter->blockIndex); + if (pIter->pCurrentBlk == NULL) { + return TSDB_CODE_INVALID_PARA; + } (void) tBrinBlockClear(&pIter->block); int32_t code = tsdbDataFileReadBrinBlock(pIter->pReader, pIter->pCurrentBlk, &pIter->block); @@ -631,6 +636,10 @@ int32_t initBlockIterator(STsdbReader* pReader, SDataBlockIter* pBlockIter, int3 for (int32_t k = 0; k < num; ++k) { SFileDataBlockInfo* pBlockInfo = taosArrayGet(pTableScanInfo->pBlockList, k); + if (pBlockInfo == NULL) { + return TSDB_CODE_INVALID_PARA; + } + sup.pDataBlockInfo[sup.numOfTables][k] = (SBlockOrderWrapper){.uid = pTableScanInfo->uid, .offset = pBlockInfo->blockOffset, .pInfo = pTableScanInfo}; cnt++; @@ -689,6 +698,10 @@ int32_t initBlockIterator(STsdbReader* pReader, SDataBlockIter* pBlockIter, int3 int32_t index = sup.indexPerTable[pos]++; SFileDataBlockInfo* pBlockInfo = taosArrayGet(sup.pDataBlockInfo[pos][index].pInfo->pBlockList, index); + if (pBlockInfo == NULL) { + return TSDB_CODE_INVALID_PARA; + } + void* px = taosArrayPush(pBlockIter->blockList, pBlockInfo); if (px == NULL) { return TSDB_CODE_OUT_OF_MEMORY; @@ -1208,6 +1221,10 @@ bool isCleanSttBlock(SArray* pKeyRangeList, STimeWindow* pQueryWindow, STableBlo } SSttKeyRange* pRange = taosArrayGet(pKeyRangeList, 0); + if (pRange == NULL) { + return false; + } + STimeWindow w = {.skey = pRange->skey.ts, .ekey = pRange->ekey.ts}; if (overlapWithTimeWindow(&w, pQueryWindow, pScanInfo, order)) { return false; @@ -1216,6 +1233,9 @@ bool isCleanSttBlock(SArray* pKeyRangeList, STimeWindow* pQueryWindow, STableBlo for (int32_t i = 0; i < num - 1; ++i) { SSttKeyRange* p1 = taosArrayGet(pKeyRangeList, i); SSttKeyRange* p2 = taosArrayGet(pKeyRangeList, i + 1); + if (p1 == NULL || p2 == NULL) { + return false; + } if (p1->ekey.ts >= p2->skey.ts) { return false; @@ -1237,6 +1257,10 @@ static bool doCheckDatablockOverlap(STableBlockScanInfo* pBlockScanInfo, const S for (int32_t i = startIndex; i < num; i += 1) { TSDBKEY* p = taosArrayGet(pBlockScanInfo->delSkyline, i); + if (p == NULL) { + return false; + } + if (p->ts >= pRecord->firstKey.key.ts && p->ts <= pRecord->lastKey.key.ts) { if (p->version >= pRecord->minVer) { return true; @@ -1245,6 +1269,10 @@ static bool doCheckDatablockOverlap(STableBlockScanInfo* pBlockScanInfo, const S if (p->version >= pRecord->minVer) { if (i < num - 1) { TSDBKEY* pnext = taosArrayGet(pBlockScanInfo->delSkyline, i + 1); + if (pnext == NULL) { + return false; + } + if (pnext->ts >= pRecord->firstKey.key.ts) { return true; } @@ -1266,11 +1294,19 @@ static bool doCheckDatablockOverlapWithoutVersion(STableBlockScanInfo* pBlockSca for (int32_t i = startIndex; i < num; i += 1) { TSDBKEY* p = taosArrayGet(pBlockScanInfo->delSkyline, i); + if (p == NULL) { + return false; + } + if (p->ts >= pRecord->firstKey.key.ts && p->ts <= pRecord->lastKey.key.ts) { return true; } else if (p->ts < pRecord->firstKey.key.ts) { // p->ts < pBlock->minKey.ts if (i < num - 1) { TSDBKEY* pnext = taosArrayGet(pBlockScanInfo->delSkyline, i + 1); + if (pnext == NULL) { + return false; + } + if (pnext->ts >= pRecord->firstKey.key.ts) { return true; } @@ -1291,6 +1327,10 @@ bool overlapWithDelSkyline(STableBlockScanInfo* pBlockScanInfo, const SBrinRecor // ts is not overlap TSDBKEY* pFirst = taosArrayGet(pBlockScanInfo->delSkyline, 0); TSDBKEY* pLast = taosArrayGetLast(pBlockScanInfo->delSkyline); + if (pFirst == NULL || pLast == NULL) { + return false; + } + if (pRecord->firstKey.key.ts > pLast->ts || pRecord->lastKey.key.ts < pFirst->ts) { return false; } @@ -1302,6 +1342,10 @@ bool overlapWithDelSkyline(STableBlockScanInfo* pBlockScanInfo, const SBrinRecor int32_t index = pBlockScanInfo->fileDelIndex; while (1) { TSDBKEY* p = taosArrayGet(pBlockScanInfo->delSkyline, index); + if (p == NULL) { + return false; + } + if (p->ts > pRecord->firstKey.key.ts && index > 0) { index -= 1; } else { // find the first point that is smaller than the minKey.ts of dataBlock. @@ -1324,6 +1368,10 @@ bool overlapWithDelSkylineWithoutVer(STableBlockScanInfo* pBlockScanInfo, const // ts is not overlap TSDBKEY* pFirst = taosArrayGet(pBlockScanInfo->delSkyline, 0); TSDBKEY* pLast = taosArrayGetLast(pBlockScanInfo->delSkyline); + if (pFirst == NULL || pLast == NULL) { + return false; + } + if (pRecord->firstKey.key.ts > pLast->ts || pRecord->lastKey.key.ts < pFirst->ts) { return false; } @@ -1335,6 +1383,10 @@ bool overlapWithDelSkylineWithoutVer(STableBlockScanInfo* pBlockScanInfo, const int32_t index = pBlockScanInfo->fileDelIndex; while (1) { TSDBKEY* p = taosArrayGet(pBlockScanInfo->delSkyline, index); + if (p == NULL) { + return false; + } + if (p->ts > pRecord->firstKey.key.ts && index > 0) { index -= 1; } else { // find the first point that is smaller than the minKey.ts of dataBlock. diff --git a/source/libs/executor/inc/tsort.h b/source/libs/executor/inc/tsort.h index b9118cf00d..c6f3986776 100644 --- a/source/libs/executor/inc/tsort.h +++ b/source/libs/executor/inc/tsort.h @@ -77,9 +77,9 @@ typedef int32_t (*_sort_merge_compar_fn_t)(const void* p1, const void* p2, void* * @param sortBufSize sort memory buf size, for check if heap sort is applicable * @return */ -SSortHandle* tsortCreateSortHandle(SArray* pOrderInfo, int32_t type, int32_t pageSize, int32_t numOfPages, +int32_t tsortCreateSortHandle(SArray* pOrderInfo, int32_t type, int32_t pageSize, int32_t numOfPages, SSDataBlock* pBlock, const char* idstr, uint64_t pqMaxRows, uint32_t pqMaxTupleLength, - uint32_t pqSortBufSize); + uint32_t pqSortBufSize, SSortHandle** pHandle); void tsortSetForceUsePQSort(SSortHandle* pHandle); @@ -101,7 +101,7 @@ int32_t tsortOpen(SSortHandle* pHandle); * @param pHandle * @return */ -int32_t tsortClose(SSortHandle* pHandle); +void tsortClose(SSortHandle* pHandle); /** * @@ -116,16 +116,17 @@ int32_t tsortSetFetchRawDataFp(SSortHandle* pHandle, _sort_fetch_block_fn_t fetc * @param fp * @return */ -int32_t tsortSetComparFp(SSortHandle* pHandle, _sort_merge_compar_fn_t fp); +void tsortSetComparFp(SSortHandle* pHandle, _sort_merge_compar_fn_t fp); /** * */ void tsortSetMergeLimit(SSortHandle* pHandle, int64_t mergeLimit); + /** * */ -int32_t tsortSetCompareGroupId(SSortHandle* pHandle, bool compareGroupId); +void tsortSetCompareGroupId(SSortHandle* pHandle, bool compareGroupId); /** * @@ -140,7 +141,7 @@ int32_t tsortAddSource(SSortHandle* pSortHandle, void* pSource); * @param pHandle * @return */ -STupleHandle* tsortNextTuple(SSortHandle* pHandle); +int32_t tsortNextTuple(SSortHandle* pHandle, STupleHandle** pTupleHandle); /** * @@ -156,7 +157,7 @@ bool tsortIsNullVal(STupleHandle* pVHandle, int32_t colId); * @param colId * @return */ -void* tsortGetValue(STupleHandle* pVHandle, int32_t colId); +void tsortGetValue(STupleHandle* pVHandle, int32_t colId, void** pVal); /** * @@ -197,7 +198,8 @@ void tsortSetAbortCheckFn(SSortHandle* pHandle, bool (*checkFn)(void* param), vo int32_t tsortSetSortByRowId(SSortHandle* pHandle, int32_t extRowsSize); -void tsortAppendTupleToBlock(SSortHandle* pHandle, SSDataBlock* pBlock, STupleHandle* pTupleHandle); +int32_t tsortAppendTupleToBlock(SSortHandle* pHandle, SSDataBlock* pBlock, STupleHandle* pTupleHandle); + /** * @brief comp the tuple with keyBuf, if not equal, new keys will be built in keyBuf, newLen will be stored in keyLen * @param [in] pSortCols cols to comp and build diff --git a/source/libs/executor/src/cachescanoperator.c b/source/libs/executor/src/cachescanoperator.c index ad7d089da9..bfa9dfd8d9 100644 --- a/source/libs/executor/src/cachescanoperator.c +++ b/source/libs/executor/src/cachescanoperator.c @@ -359,7 +359,8 @@ SSDataBlock* doScanCache(SOperatorInfo* pOperator) { } } - pInfo->pLastrowReader = pReaderFn->closeReader(pInfo->pLastrowReader); + pReaderFn->closeReader(pInfo->pLastrowReader); + pInfo->pLastrowReader = NULL; setOperatorCompleted(pOperator); return NULL; } @@ -378,7 +379,8 @@ void destroyCacheScanOperator(void* param) { tableListDestroy(pInfo->pTableList); if (pInfo->pLastrowReader != NULL) { - pInfo->pLastrowReader = pInfo->readHandle.api.cacheFn.closeReader(pInfo->pLastrowReader); + pInfo->readHandle.api.cacheFn.closeReader(pInfo->pLastrowReader); + pInfo->pLastrowReader = NULL; } cleanupExprSupp(&pInfo->pseudoExprSup); diff --git a/source/libs/executor/src/mergeoperator.c b/source/libs/executor/src/mergeoperator.c old mode 100755 new mode 100644 index 2816bae03c..cef1915d2c --- a/source/libs/executor/src/mergeoperator.c +++ b/source/libs/executor/src/mergeoperator.c @@ -70,8 +70,12 @@ int32_t openSortMergeOperator(SOperatorInfo* pOperator) { int32_t numOfBufPage = pSortMergeInfo->sortBufSize / pSortMergeInfo->bufPageSize; - pSortMergeInfo->pSortHandle = tsortCreateSortHandle(pSortMergeInfo->pSortInfo, SORT_MULTISOURCE_MERGE, pSortMergeInfo->bufPageSize, numOfBufPage, - pSortMergeInfo->pInputBlock, pTaskInfo->id.str, 0, 0, 0); + pSortMergeInfo->pSortHandle = NULL; + int32_t code = tsortCreateSortHandle(pSortMergeInfo->pSortInfo, SORT_MULTISOURCE_MERGE, pSortMergeInfo->bufPageSize, + numOfBufPage, pSortMergeInfo->pInputBlock, pTaskInfo->id.str, 0, 0, 0, &pSortMergeInfo->pSortHandle); + if (code) { + return code; + } tsortSetFetchRawDataFp(pSortMergeInfo->pSortHandle, sortMergeloadNextDataBlock, NULL, NULL); tsortSetCompareGroupId(pSortMergeInfo->pSortHandle, pInfo->groupMerge); @@ -96,12 +100,17 @@ static void doGetSortedBlockData(SMultiwayMergeOperatorInfo* pInfo, SSortHandle* SSDataBlock* p, bool* newgroup) { SSortMergeInfo* pSortMergeInfo = &pInfo->sortMergeInfo; *newgroup = false; + int32_t code = 0; while (1) { STupleHandle* pTupleHandle = NULL; if (pInfo->groupMerge || pInfo->inputWithGroupId) { if (pSortMergeInfo->prefetchedTuple == NULL) { - pTupleHandle = tsortNextTuple(pHandle); + pTupleHandle = NULL; + code = tsortNextTuple(pHandle, &pTupleHandle); + if (code) { + // todo handle error + } } else { pTupleHandle = pSortMergeInfo->prefetchedTuple; pSortMergeInfo->prefetchedTuple = NULL; @@ -112,11 +121,11 @@ static void doGetSortedBlockData(SMultiwayMergeOperatorInfo* pInfo, SSortHandle* } } } else { - pTupleHandle = tsortNextTuple(pHandle); + code = tsortNextTuple(pHandle, &pTupleHandle); pInfo->groupId = 0; } - if (pTupleHandle == NULL) { + if (pTupleHandle == NULL || (code != 0)) { break; } diff --git a/source/libs/executor/src/scanoperator.c b/source/libs/executor/src/scanoperator.c index 520cba92a1..591c225e19 100644 --- a/source/libs/executor/src/scanoperator.c +++ b/source/libs/executor/src/scanoperator.c @@ -5113,8 +5113,13 @@ int32_t startDurationForGroupTableMergeScan(SOperatorInfo* pOperator) { pInfo->sortBufSize = 2048 * pInfo->bufPageSize; int32_t numOfBufPage = pInfo->sortBufSize / pInfo->bufPageSize; - pInfo->pSortHandle = tsortCreateSortHandle(pInfo->pSortInfo, SORT_BLOCK_TS_MERGE, pInfo->bufPageSize, numOfBufPage, - pInfo->pSortInputBlock, pTaskInfo->id.str, 0, 0, 0); + pInfo->pSortHandle = NULL; + code = tsortCreateSortHandle(pInfo->pSortInfo, SORT_BLOCK_TS_MERGE, pInfo->bufPageSize, numOfBufPage, + pInfo->pSortInputBlock, pTaskInfo->id.str, 0, 0, 0, &pInfo->pSortHandle); + if (code) { + return code; + } + if (pInfo->bSortRowId && numOfTable != 1) { int32_t memSize = 512 * 1024 * 1024; code = tsortSetSortByRowId(pInfo->pSortHandle, memSize); @@ -5247,10 +5252,12 @@ SSDataBlock* getSortedTableMergeScanBlockData(SSortHandle* pHandle, SSDataBlock* STupleHandle* pTupleHandle = NULL; while (1) { while (1) { - pTupleHandle = tsortNextTuple(pHandle); - if (pTupleHandle == NULL) { + pTupleHandle = NULL; + int32_t code = tsortNextTuple(pHandle, &pTupleHandle); + if (pTupleHandle == NULL || code != 0) { break; } + tsortAppendTupleToBlock(pInfo->pSortHandle, pResBlock, pTupleHandle); if (pResBlock->info.rows >= capacity) { break; diff --git a/source/libs/executor/src/sortoperator.c b/source/libs/executor/src/sortoperator.c index 82eebf5310..67b993d702 100644 --- a/source/libs/executor/src/sortoperator.c +++ b/source/libs/executor/src/sortoperator.c @@ -157,7 +157,8 @@ void appendOneRowToDataBlock(SSDataBlock* pBlock, STupleHandle* pTupleHandle) { if (isNull) { colDataSetNULL(pColInfo, pBlock->info.rows); } else { - char* pData = tsortGetValue(pTupleHandle, i); + char* pData = NULL; + tsortGetValue(pTupleHandle, i, (void**) &pData); if (pData != NULL) { colDataSetVal(pColInfo, pBlock->info.rows, pData, false); } @@ -176,9 +177,13 @@ void appendOneRowToDataBlock(SSDataBlock* pBlock, STupleHandle* pTupleHandle) { * @retval NULL if no more tuples */ static STupleHandle* nextTupleWithGroupId(SSortHandle* pHandle, SSortOperatorInfo* pInfo, SSDataBlock* pBlock) { + int32_t code = 0; STupleHandle* retTuple = pInfo->pGroupIdCalc->pSavedTuple; if (!retTuple) { - retTuple = tsortNextTuple(pHandle); + code = tsortNextTuple(pHandle, &retTuple); + if (code) { + return NULL; + } } if (retTuple) { @@ -219,6 +224,7 @@ SSDataBlock* getSortedBlockData(SSortHandle* pHandle, SSDataBlock* pDataBlock, i SSortOperatorInfo* pInfo) { blockDataCleanup(pDataBlock); + int32_t code = 0; SSDataBlock* p = tsortGetSortedDataBlock(pHandle); if (p == NULL) { return NULL; @@ -231,9 +237,9 @@ SSDataBlock* getSortedBlockData(SSortHandle* pHandle, SSDataBlock* pDataBlock, i if (pInfo->pGroupIdCalc) { pTupleHandle = nextTupleWithGroupId(pHandle, pInfo, p); } else { - pTupleHandle = tsortNextTuple(pHandle); + code = tsortNextTuple(pHandle, &pTupleHandle); } - if (pTupleHandle == NULL) { + if (pTupleHandle == NULL || code != 0) { break; } @@ -295,8 +301,12 @@ int32_t doOpenSortOperator(SOperatorInfo* pOperator) { pInfo->startTs = taosGetTimestampUs(); // pInfo->binfo.pRes is not equalled to the input datablock. - pInfo->pSortHandle = tsortCreateSortHandle(pInfo->pSortInfo, SORT_SINGLESOURCE_SORT, -1, -1, NULL, pTaskInfo->id.str, - pInfo->maxRows, pInfo->maxTupleLength, tsPQSortMemThreshold * 1024 * 1024); + pInfo->pSortHandle = NULL; + int32_t code = tsortCreateSortHandle(pInfo->pSortInfo, SORT_SINGLESOURCE_SORT, -1, -1, NULL, pTaskInfo->id.str, + pInfo->maxRows, pInfo->maxTupleLength, tsPQSortMemThreshold * 1024 * 1024, &pInfo->pSortHandle); + if (code) { + return code; + } tsortSetFetchRawDataFp(pInfo->pSortHandle, loadNextDataBlock, applyScalarFunction, pOperator); @@ -305,8 +315,7 @@ int32_t doOpenSortOperator(SOperatorInfo* pOperator) { ps->onlyRef = true; tsortAddSource(pInfo->pSortHandle, ps); - int32_t code = tsortOpen(pInfo->pSortHandle); - + code = tsortOpen(pInfo->pSortHandle); if (code != TSDB_CODE_SUCCESS) { T_LONG_JMP(pTaskInfo->env, terrno); } @@ -442,8 +451,9 @@ SSDataBlock* getGroupSortedBlockData(SSortHandle* pHandle, SSDataBlock* pDataBlo blockDataEnsureCapacity(p, capacity); while (1) { - STupleHandle* pTupleHandle = tsortNextTuple(pHandle); - if (pTupleHandle == NULL) { + STupleHandle* pTupleHandle = NULL; + int32_t code = tsortNextTuple(pHandle, &pTupleHandle); + if (pTupleHandle == NULL || code != 0) { break; } @@ -508,8 +518,13 @@ int32_t beginSortGroup(SOperatorInfo* pOperator) { SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo; // pInfo->binfo.pRes is not equalled to the input datablock. - pInfo->pCurrSortHandle = - tsortCreateSortHandle(pInfo->pSortInfo, SORT_SINGLESOURCE_SORT, -1, -1, NULL, pTaskInfo->id.str, 0, 0, 0); + pInfo->pCurrSortHandle = NULL; + + int32_t code = tsortCreateSortHandle(pInfo->pSortInfo, SORT_SINGLESOURCE_SORT, -1, -1, NULL, pTaskInfo->id.str, 0, 0, + 0, &pInfo->pCurrSortHandle); + if (code) { + return code; + } tsortSetFetchRawDataFp(pInfo->pCurrSortHandle, fetchNextGroupSortDataBlock, applyScalarFunction, pOperator); @@ -521,13 +536,12 @@ int32_t beginSortGroup(SOperatorInfo* pOperator) { ps->onlyRef = false; tsortAddSource(pInfo->pCurrSortHandle, ps); - int32_t code = tsortOpen(pInfo->pCurrSortHandle); - + code = tsortOpen(pInfo->pCurrSortHandle); if (code != TSDB_CODE_SUCCESS) { T_LONG_JMP(pTaskInfo->env, terrno); } - return TSDB_CODE_SUCCESS; + return code; } int32_t finishSortGroup(SOperatorInfo* pOperator) { diff --git a/source/libs/executor/src/tsort.c b/source/libs/executor/src/tsort.c index 6d88eaef99..33596c4802 100644 --- a/source/libs/executor/src/tsort.c +++ b/source/libs/executor/src/tsort.c @@ -114,7 +114,7 @@ struct SSortHandle { void* mergeLimitReachedParam; }; -static int32_t destroySortMemFile(SSortHandle* pHandle); +static void destroySortMemFile(SSortHandle* pHandle); static int32_t getRowBufFromExtMemFile(SSortHandle* pHandle, int32_t regionId, int32_t tupleOffset, int32_t rowLen, char** ppRow, bool* pFreeRow); void tsortSetSingleTableMerge(SSortHandle* pHandle) { @@ -226,16 +226,21 @@ void destroyTuple(void* t) { } } - /** * * @param type * @return */ -SSortHandle* tsortCreateSortHandle(SArray* pSortInfo, int32_t type, int32_t pageSize, int32_t numOfPages, - SSDataBlock* pBlock, const char* idstr, uint64_t pqMaxRows, uint32_t pqMaxTupleLength, - uint32_t pqSortBufSize) { +int32_t tsortCreateSortHandle(SArray* pSortInfo, int32_t type, int32_t pageSize, int32_t numOfPages, + SSDataBlock* pBlock, const char* idstr, uint64_t pqMaxRows, uint32_t pqMaxTupleLength, + uint32_t pqSortBufSize, SSortHandle** pHandle) { + int32_t code = 0; + *pHandle = NULL; + SSortHandle* pSortHandle = taosMemoryCalloc(1, sizeof(SSortHandle)); + if (pSortHandle == NULL) { + return TSDB_CODE_OUT_OF_MEMORY; + } pSortHandle->type = type; pSortHandle->pageSize = pageSize; @@ -248,15 +253,24 @@ SSortHandle* tsortCreateSortHandle(SArray* pSortInfo, int32_t type, int32_t page pSortHandle->pqSortBufSize = pqSortBufSize; pSortHandle->pqMaxRows = pqMaxRows; } - pSortHandle->forceUsePQSort = false; + pSortHandle->forceUsePQSort = false; if (pBlock != NULL) { pSortHandle->pDataBlock = createOneDataBlock(pBlock, false); + if (pSortHandle->pDataBlock == NULL) { + code = TSDB_CODE_OUT_OF_MEMORY; + goto _err; + } } pSortHandle->mergeLimit = -1; pSortHandle->pOrderedSource = taosArrayInit(4, POINTER_BYTES); + if (pSortHandle->pOrderedSource == NULL) { + code = TSDB_CODE_OUT_OF_MEMORY; + goto _err; + } + pSortHandle->cmpParam.orderInfo = pSortInfo; pSortHandle->cmpParam.cmpGroupId = false; pSortHandle->cmpParam.sortType = type; @@ -278,9 +292,18 @@ SSortHandle* tsortCreateSortHandle(SArray* pSortInfo, int32_t type, int32_t page if (idstr != NULL) { pSortHandle->idStr = taosStrdup(idstr); + if (pSortHandle->idStr == NULL) { + code = TSDB_CODE_OUT_OF_MEMORY; + goto _err; + } } - return pSortHandle; + *pHandle = pSortHandle; + return code; + +_err: + tsortDestroySortHandle(pSortHandle); + return code; } static int32_t sortComparCleanup(SMsortComparParam* cmpParam) { @@ -336,6 +359,7 @@ void tsortDestroySortHandle(SSortHandle* pSortHandle) { if (pSortHandle == NULL) { return; } + tsortClose(pSortHandle); if (pSortHandle->pMergeTree != NULL) { tMergeTreeDestroy(&pSortHandle->pMergeTree); @@ -354,6 +378,7 @@ void tsortDestroySortHandle(SSortHandle* pSortHandle) { if (pSortHandle->pExtRowsMemFile != NULL) { destroySortMemFile(pSortHandle); } + taosArrayDestroy(pSortHandle->pSortInfo); taosArrayDestroy(pSortHandle->aExtRowsOrders); pSortHandle->aExtRowsOrders = NULL; @@ -361,8 +386,8 @@ void tsortDestroySortHandle(SSortHandle* pSortHandle) { } int32_t tsortAddSource(SSortHandle* pSortHandle, void* pSource) { - taosArrayPush(pSortHandle->pOrderedSource, &pSource); - return TSDB_CODE_SUCCESS; + void* p = taosArrayPush(pSortHandle->pOrderedSource, &pSource); + return (p != NULL)? TSDB_CODE_SUCCESS:TSDB_CODE_OUT_OF_MEMORY; } static int32_t doAddNewExternalMemSource(SDiskbasedBuf* pBuf, SArray* pAllSources, SSDataBlock* pBlock, @@ -375,7 +400,12 @@ static int32_t doAddNewExternalMemSource(SDiskbasedBuf* pBuf, SArray* pAllSource pSource->src.pBlock = pBlock; pSource->pageIdList = pPageIdList; - taosArrayPush(pAllSources, &pSource); + + void* p = taosArrayPush(pAllSources, &pSource); + if (p == NULL) { + taosArrayDestroy(pPageIdList); + return TSDB_CODE_OUT_OF_MEMORY; + } (*sourceId) += 1; @@ -410,7 +440,13 @@ static int32_t doAddToBuf(SSDataBlock* pDataBlock, SSortHandle* pHandle) { SArray* pPageIdList = taosArrayInit(4, sizeof(int32_t)); while (start < pDataBlock->info.rows) { int32_t stop = 0; - blockDataSplitRows(pDataBlock, pDataBlock->info.hasVarCol, start, &stop, pHandle->pageSize); + + int32_t code = blockDataSplitRows(pDataBlock, pDataBlock->info.hasVarCol, start, &stop, pHandle->pageSize); + if (code) { + taosArrayDestroy(pPageIdList); + return code; + } + SSDataBlock* p = blockDataExtractBlock(pDataBlock, start, stop - start + 1); if (p == NULL) { taosArrayDestroy(pPageIdList); @@ -425,12 +461,20 @@ static int32_t doAddToBuf(SSDataBlock* pDataBlock, SSortHandle* pHandle) { return terrno; } - taosArrayPush(pPageIdList, &pageId); + void* px = taosArrayPush(pPageIdList, &pageId); + if (px == NULL) { + taosArrayDestroy(pPageIdList); + blockDataDestroy(p); + return TSDB_CODE_OUT_OF_MEMORY; + } int32_t size = blockDataGetSize(p) + sizeof(int32_t) + taosArrayGetSize(p->pDataBlock) * sizeof(int32_t); ASSERT(size <= getBufPageSize(pHandle->pBuf)); - blockDataToBuf(pPage, p); + code = blockDataToBuf(pPage, p); + if (code) { + return code; + } setBufPageDirty(pPage, true); releaseBufPage(pHandle->pBuf, pPage); @@ -520,23 +564,32 @@ static int32_t sortComparInit(SMsortComparParam* pParam, SArray* pSources, int32 return code; } -static void appendOneRowToDataBlock(SSDataBlock* pBlock, const SSDataBlock* pSource, int32_t* rowIndex) { +static int32_t appendOneRowToDataBlock(SSDataBlock* pBlock, const SSDataBlock* pSource, int32_t* rowIndex) { + int32_t code = 0; + for (int32_t i = 0; i < taosArrayGetSize(pBlock->pDataBlock); ++i) { SColumnInfoData* pColInfo = taosArrayGet(pBlock->pDataBlock, i); SColumnInfoData* pSrcColInfo = taosArrayGet(pSource->pDataBlock, i); bool isNull = colDataIsNull(pSrcColInfo, pSource->info.rows, *rowIndex, NULL); if (isNull) { - colDataSetVal(pColInfo, pBlock->info.rows, NULL, true); + code = colDataSetVal(pColInfo, pBlock->info.rows, NULL, true); + if (code) { + return code; + } } else { if (!pSrcColInfo->pData) continue; char* pData = colDataGetData(pSrcColInfo, *rowIndex); - colDataSetVal(pColInfo, pBlock->info.rows, pData, false); + code = colDataSetVal(pColInfo, pBlock->info.rows, pData, false); + if (code) { + return code; + } } } pBlock->info.rows += 1; *rowIndex += 1; + return code; } static int32_t adjustMergeTreeForNextTuple(SSortSource* pSource, SMultiwayMergeTreeInfo* pTree, SSortHandle* pHandle, @@ -609,7 +662,11 @@ static int32_t adjustMergeTreeForNextTuple(SSortSource* pSource, SMultiwayMergeT return TSDB_CODE_SUCCESS; } -static SSDataBlock* getSortedBlockDataInner(SSortHandle* pHandle, SMsortComparParam* cmpParam, int32_t capacity) { +static int32_t getSortedBlockDataInner(SSortHandle* pHandle, SMsortComparParam* cmpParam, int32_t capacity, + SSDataBlock** pRes) { + *pRes = NULL; + + int32_t code = 0; blockDataCleanup(pHandle->pDataBlock); while (1) { @@ -620,20 +677,24 @@ static SSDataBlock* getSortedBlockDataInner(SSortHandle* pHandle, SMsortComparPa int32_t index = tMergeTreeGetChosenIndex(pHandle->pMergeTree); SSortSource* pSource = (*cmpParam).pSources[index]; - appendOneRowToDataBlock(pHandle->pDataBlock, pSource->src.pBlock, &pSource->src.rowIndex); + code = appendOneRowToDataBlock(pHandle->pDataBlock, pSource->src.pBlock, &pSource->src.rowIndex); + if (code) { + return code; + } - int32_t code = adjustMergeTreeForNextTuple(pSource, pHandle->pMergeTree, pHandle, &pHandle->numOfCompletedSources); + code = adjustMergeTreeForNextTuple(pSource, pHandle->pMergeTree, pHandle, &pHandle->numOfCompletedSources); if (code != TSDB_CODE_SUCCESS) { - terrno = code; - return NULL; + return terrno = code; } if (pHandle->pDataBlock->info.rows >= capacity) { - return pHandle->pDataBlock; + *pRes = pHandle->pDataBlock; + return code; } } - return (pHandle->pDataBlock->info.rows > 0) ? pHandle->pDataBlock : NULL; + *pRes = (pHandle->pDataBlock->info.rows > 0) ? pHandle->pDataBlock : NULL; + return code; } // TODO: improve this function performance @@ -819,7 +880,10 @@ static int32_t doInternalMergeSort(SSortHandle* pHandle) { int32_t numOfRows = blockDataGetCapacityInRow(pHandle->pDataBlock, pHandle->pageSize, blockDataGetSerialMetaSize(taosArrayGetSize(pHandle->pDataBlock->pDataBlock))); - blockDataEnsureCapacity(pHandle->pDataBlock, numOfRows); + int32_t code = blockDataEnsureCapacity(pHandle->pDataBlock, numOfRows); + if (code) { + return code; + } // the initial pass + sortPass + final mergePass pHandle->loops = sortPass + 2; @@ -845,7 +909,7 @@ static int32_t doInternalMergeSort(SSortHandle* pHandle) { pHandle->cmpParam.numOfSources = end - i * numOfInputSources + 1; - int32_t code = sortComparInit(&pHandle->cmpParam, pHandle->pOrderedSource, i * numOfInputSources, end, pHandle); + code = sortComparInit(&pHandle->cmpParam, pHandle->pOrderedSource, i * numOfInputSources, end, pHandle); if (code != TSDB_CODE_SUCCESS) { taosArrayDestroy(pResList); return code; @@ -866,8 +930,10 @@ static int32_t doInternalMergeSort(SSortHandle* pHandle) { code = terrno = TSDB_CODE_TSC_QUERY_CANCELLED; return code; } - SSDataBlock* pDataBlock = getSortedBlockDataInner(pHandle, &pHandle->cmpParam, numOfRows); - if (pDataBlock == NULL) { + + SSDataBlock* pDataBlock = NULL; + code = getSortedBlockDataInner(pHandle, &pHandle->cmpParam, numOfRows, &pDataBlock); + if (pDataBlock == NULL || code != 0) { break; } @@ -879,13 +945,21 @@ static int32_t doInternalMergeSort(SSortHandle* pHandle) { return terrno; } - taosArrayPush(pPageIdList, &pageId); + void* px = taosArrayPush(pPageIdList, &pageId); + if (px == NULL) { + taosArrayDestroy(pResList); + taosArrayDestroy(pPageIdList); + return terrno; + } int32_t size = blockDataGetSize(pDataBlock) + sizeof(int32_t) + taosArrayGetSize(pDataBlock->pDataBlock) * sizeof(int32_t); ASSERT(size <= getBufPageSize(pHandle->pBuf)); - blockDataToBuf(pPage, pDataBlock); + code= blockDataToBuf(pPage, pDataBlock); + if (code) { + return code; + } setBufPageDirty(pPage, true); releaseBufPage(pHandle->pBuf, pPage); @@ -897,7 +971,11 @@ static int32_t doInternalMergeSort(SSortHandle* pHandle) { } } - sortComparCleanup(&pHandle->cmpParam); + code = sortComparCleanup(&pHandle->cmpParam); + if (code) { + return code; + } + tMergeTreeDestroy(&pHandle->pMergeTree); pHandle->numOfCompletedSources = 0; @@ -910,7 +988,11 @@ static int32_t doInternalMergeSort(SSortHandle* pHandle) { } tsortClearOrderdSource(pHandle->pOrderedSource, NULL, NULL); - taosArrayAddAll(pHandle->pOrderedSource, pResList); + void* px = taosArrayAddAll(pHandle->pOrderedSource, pResList); + if (px == NULL) { + return TSDB_CODE_OUT_OF_MEMORY; + } + taosArrayDestroy(pResList); numOfSorted = taosArrayGetSize(pHandle->pOrderedSource); @@ -960,15 +1042,27 @@ static int32_t createPageBuf(SSortHandle* pHandle) { return 0; } -void tsortAppendTupleToBlock(SSortHandle* pHandle, SSDataBlock* pBlock, STupleHandle* pTupleHandle) { +int32_t tsortAppendTupleToBlock(SSortHandle* pHandle, SSDataBlock* pBlock, STupleHandle* pTupleHandle) { + int32_t code = 0; + if (pHandle->bSortByRowId) { - int32_t regionId = *(int32_t*)tsortGetValue(pTupleHandle, 1); - int32_t offset = *(int32_t*)tsortGetValue(pTupleHandle, 2); - int32_t length = *(int32_t*)tsortGetValue(pTupleHandle, 3); + int32_t *p1, *p2, *p3; + tsortGetValue(pTupleHandle, 1, (void**) &p1); + tsortGetValue(pTupleHandle, 2, (void**) &p2); + tsortGetValue(pTupleHandle, 3, (void**) &p3); + + int32_t regionId = *p1; + int32_t offset = *p2; + int32_t length = *p3; char* buf = NULL; bool bFreeRow = false; - getRowBufFromExtMemFile(pHandle, regionId, offset, length, &buf, &bFreeRow); + + code = getRowBufFromExtMemFile(pHandle, regionId, offset, length, &buf, &bFreeRow); + if (code) { + return code; + } + int32_t numOfCols = taosArrayGetSize(pBlock->pDataBlock); char* isNull = (char*)buf; char* pStart = (char*)buf + sizeof(int8_t) * numOfCols; @@ -976,7 +1070,11 @@ void tsortAppendTupleToBlock(SSortHandle* pHandle, SSDataBlock* pBlock, STupleHa SColumnInfoData* pColInfo = taosArrayGet(pBlock->pDataBlock, i); if (!isNull[i]) { - colDataSetVal(pColInfo, pBlock->info.rows, pStart, false); + code = colDataSetVal(pColInfo, pBlock->info.rows, pStart, false); + if (code) { + return code; + } + if (pColInfo->info.type == TSDB_DATA_TYPE_JSON) { int32_t dataLen = getJsonValueLen(pStart); pStart += dataLen; @@ -1011,9 +1109,13 @@ void tsortAppendTupleToBlock(SSortHandle* pHandle, SSDataBlock* pBlock, STupleHa if (isNull) { colDataSetNULL(pColInfo, pBlock->info.rows); } else { - char* pData = tsortGetValue(pTupleHandle, i); + char* pData = NULL; + tsortGetValue(pTupleHandle, i, (void**)&pData); if (pData != NULL) { - colDataSetVal(pColInfo, pBlock->info.rows, pData, false); + code = colDataSetVal(pColInfo, pBlock->info.rows, pData, false); + if (code) { + return code; + } } } } @@ -1022,6 +1124,8 @@ void tsortAppendTupleToBlock(SSortHandle* pHandle, SSDataBlock* pBlock, STupleHa pBlock->info.scanFlag = ((SDataBlockInfo*)tsortGetBlockInfo(pTupleHandle))->scanFlag; pBlock->info.rows += 1; } + + return code; } static int32_t blockRowToBuf(SSDataBlock* pBlock, int32_t rowIdx, char* buf) { @@ -1078,7 +1182,10 @@ static int32_t getRowBufFromExtMemFile(SSortHandle* pHandle, int32_t regionId, i if (pRegion->buf == NULL) { return TSDB_CODE_OUT_OF_MEMORY; } - taosSeekCFile(pMemFile->pTdFile, pRegion->fileOffset, SEEK_SET); + + // todo + (void)taosSeekCFile(pMemFile->pTdFile, pRegion->fileOffset, SEEK_SET); + int32_t readBytes = TMIN(pMemFile->blockSize, pRegion->regionSize); int32_t ret = taosReadFromCFile(pRegion->buf, readBytes, 1, pMemFile->pTdFile); if (ret != 1) { @@ -1098,7 +1205,9 @@ static int32_t getRowBufFromExtMemFile(SSortHandle* pHandle, int32_t regionId, i } int32_t szThisBlock = pRegion->bufLen - (tupleOffset - pRegion->bufRegOffset); memcpy(*ppRow, pRegion->buf + tupleOffset - pRegion->bufRegOffset, szThisBlock); - taosSeekCFile(pMemFile->pTdFile, pRegion->fileOffset + pRegion->bufRegOffset + pRegion->bufLen, SEEK_SET); + + // todo + (void) taosSeekCFile(pMemFile->pTdFile, pRegion->fileOffset + pRegion->bufRegOffset + pRegion->bufLen, SEEK_SET); int32_t readBytes = TMIN(pMemFile->blockSize, pRegion->regionSize - (pRegion->bufRegOffset + pRegion->bufLen)); int32_t ret = taosReadFromCFile(pRegion->buf, readBytes, 1, pMemFile->pTdFile); if (ret != 1) { @@ -1131,7 +1240,11 @@ static int32_t createSortMemFile(SSortHandle* pHandle) { } } if (code == TSDB_CODE_SUCCESS) { - taosSetAutoDelFile(pMemFile->memFilePath); + code = taosSetAutoDelFile(pMemFile->memFilePath); + if (code) { + qError("failed to set the auto-delete file attribute"); + return code; + } pMemFile->currRegionId = -1; pMemFile->currRegionOffset = -1; @@ -1145,6 +1258,7 @@ static int32_t createSortMemFile(SSortHandle* pHandle) { code = TSDB_CODE_OUT_OF_MEMORY; } } + if (code == TSDB_CODE_SUCCESS) { pMemFile->cacheSize = pHandle->extRowsMemSize; pMemFile->aFileRegions = taosArrayInit(64, sizeof(SSortMemFileRegion)); @@ -1152,6 +1266,7 @@ static int32_t createSortMemFile(SSortHandle* pHandle) { code = TSDB_CODE_OUT_OF_MEMORY; } } + if (code == TSDB_CODE_SUCCESS) { pHandle->pExtRowsMemFile = pMemFile; } else { @@ -1159,7 +1274,7 @@ static int32_t createSortMemFile(SSortHandle* pHandle) { if (pMemFile->aFileRegions) taosMemoryFreeClear(pMemFile->aFileRegions); if (pMemFile->writeBuf) taosMemoryFreeClear(pMemFile->writeBuf); if (pMemFile->pTdFile) { - taosCloseCFile(pMemFile->pTdFile); + (void) taosCloseCFile(pMemFile->pTdFile); pMemFile->pTdFile = NULL; } taosMemoryFreeClear(pMemFile); @@ -1168,8 +1283,10 @@ static int32_t createSortMemFile(SSortHandle* pHandle) { return code; } -static int32_t destroySortMemFile(SSortHandle* pHandle) { - if (pHandle->pExtRowsMemFile == NULL) return TSDB_CODE_SUCCESS; +static void destroySortMemFile(SSortHandle* pHandle) { + if (pHandle->pExtRowsMemFile == NULL) { + return; + } SSortMemFile* pMemFile = pHandle->pExtRowsMemFile; for (int32_t i = 0; i < taosArrayGetSize(pMemFile->aFileRegions); ++i) { @@ -1182,21 +1299,26 @@ static int32_t destroySortMemFile(SSortHandle* pHandle) { taosMemoryFree(pMemFile->writeBuf); pMemFile->writeBuf = NULL; - taosCloseCFile(pMemFile->pTdFile); + (void) taosCloseCFile(pMemFile->pTdFile); pMemFile->pTdFile = NULL; - taosRemoveFile(pMemFile->memFilePath); + (void) taosRemoveFile(pMemFile->memFilePath); taosMemoryFree(pMemFile); pHandle->pExtRowsMemFile = NULL; - return TSDB_CODE_SUCCESS; } static int32_t tsortOpenRegion(SSortHandle* pHandle) { SSortMemFile* pMemFile = pHandle->pExtRowsMemFile; + int32_t code = 0; + if (pMemFile->currRegionId == -1) { SSortMemFileRegion region = {0}; region.fileOffset = 0; region.bufRegOffset = 0; - taosArrayPush(pMemFile->aFileRegions, ®ion); + void* px = taosArrayPush(pMemFile->aFileRegions, ®ion); + if (px == NULL) { + code = TSDB_CODE_OUT_OF_MEMORY; + } + pMemFile->currRegionId = 0; pMemFile->currRegionOffset = 0; pMemFile->writeFileOffset = 0; @@ -1205,12 +1327,15 @@ static int32_t tsortOpenRegion(SSortHandle* pHandle) { SSortMemFileRegion* pRegion = taosArrayGet(pMemFile->aFileRegions, pMemFile->currRegionId); regionNew.fileOffset = pRegion->fileOffset + pRegion->regionSize; regionNew.bufRegOffset = 0; - taosArrayPush(pMemFile->aFileRegions, ®ionNew); + void* px = taosArrayPush(pMemFile->aFileRegions, ®ionNew); + if (px == NULL) { + code = TSDB_CODE_OUT_OF_MEMORY; + } ++pMemFile->currRegionId; pMemFile->currRegionOffset = 0; pMemFile->writeFileOffset = regionNew.fileOffset; } - return TSDB_CODE_SUCCESS; + return code; } static int32_t tsortCloseRegion(SSortHandle* pHandle) { @@ -1233,14 +1358,22 @@ static int32_t tsortFinalizeRegions(SSortHandle* pHandle) { SSortMemFile* pMemFile = pHandle->pExtRowsMemFile; size_t numRegions = taosArrayGetSize(pMemFile->aFileRegions); ASSERT(numRegions == (pMemFile->currRegionId + 1)); - if (numRegions == 0) return TSDB_CODE_SUCCESS; + if (numRegions == 0) { + return TSDB_CODE_SUCCESS; + } + int32_t blockReadBytes = (pMemFile->cacheSize / numRegions + 4095) & ~4095; pMemFile->blockSize = blockReadBytes; for (int32_t i = 0; i < numRegions; ++i) { SSortMemFileRegion* pRegion = taosArrayGet(pMemFile->aFileRegions, i); + if (pRegion == NULL) { + return TSDB_CODE_INVALID_PARA; + } + pRegion->bufRegOffset = 0; } + taosMemoryFree(pMemFile->writeBuf); pMemFile->writeBuf = NULL; return TSDB_CODE_SUCCESS; @@ -1274,18 +1407,26 @@ static int32_t saveBlockRowToExtRowsMemFile(SSortHandle* pHandle, SSDataBlock* p return TSDB_CODE_SUCCESS; } -static void appendToRowIndexDataBlock(SSortHandle* pHandle, SSDataBlock* pSource, int32_t* rowIndex) { +static int32_t appendToRowIndexDataBlock(SSortHandle* pHandle, SSDataBlock* pSource, int32_t* rowIndex) { int32_t pageId = -1; int32_t offset = -1; int32_t length = -1; - saveBlockRowToExtRowsMemFile(pHandle, pSource, *rowIndex, &pageId, &offset, &length); + int32_t code = 0; + + code = saveBlockRowToExtRowsMemFile(pHandle, pSource, *rowIndex, &pageId, &offset, &length); + if (code) { + return code; + } SSDataBlock* pBlock = pHandle->pDataBlock; SBlockOrderInfo* extRowsTsOrder = taosArrayGet(pHandle->aExtRowsOrders, 0); SColumnInfoData* pSrcTsCol = taosArrayGet(pSource->pDataBlock, extRowsTsOrder->slotId); SColumnInfoData* pTsCol = taosArrayGet(pBlock->pDataBlock, 0); char* pData = colDataGetData(pSrcTsCol, *rowIndex); - colDataSetVal(pTsCol, pBlock->info.rows, pData, false); + code = colDataSetVal(pTsCol, pBlock->info.rows, pData, false); + if (code) { + return code; + } SColumnInfoData* pRegionIdCol = taosArrayGet(pBlock->pDataBlock, 1); colDataSetInt32(pRegionIdCol, pBlock->info.rows, &pageId); @@ -1304,32 +1445,53 @@ static void appendToRowIndexDataBlock(SSortHandle* pHandle, SSDataBlock* pSource colDataSetNULL(pPkCol, pBlock->info.rows); } else { char* pPkData = colDataGetData(pSrcPkCol, *rowIndex); - colDataSetVal(pPkCol, pBlock->info.rows, pPkData, false); + code = colDataSetVal(pPkCol, pBlock->info.rows, pPkData, false); + if (code) { + return code; + } } } pBlock->info.rows += 1; *rowIndex += 1; + return code; } -static void initRowIdSort(SSortHandle* pHandle) { +static int32_t initRowIdSort(SSortHandle* pHandle) { SBlockOrderInfo* pkOrder = (pHandle->bSortPk) ? taosArrayGet(pHandle->aExtRowsOrders, 1) : NULL; SColumnInfoData* extPkCol = (pHandle->bSortPk) ? taosArrayGet(pHandle->pDataBlock->pDataBlock, pkOrder->slotId) : NULL; SColumnInfoData pkCol = {0}; SSDataBlock* pSortInput = createDataBlock(); + pHandle->pDataBlock = pSortInput; + SColumnInfoData tsCol = createColumnInfoData(TSDB_DATA_TYPE_TIMESTAMP, 8, 1); - blockDataAppendColInfo(pSortInput, &tsCol); + int32_t code = blockDataAppendColInfo(pSortInput, &tsCol); + if (code) { + return code; + } + SColumnInfoData regionIdCol = createColumnInfoData(TSDB_DATA_TYPE_INT, 4, 2); - blockDataAppendColInfo(pSortInput, ®ionIdCol); + code = blockDataAppendColInfo(pSortInput, ®ionIdCol); + if (code) { + return code; + } + SColumnInfoData offsetCol = createColumnInfoData(TSDB_DATA_TYPE_INT, 4, 3); - blockDataAppendColInfo(pSortInput, &offsetCol); + code = blockDataAppendColInfo(pSortInput, &offsetCol); + if (code) { + return code; + } + SColumnInfoData lengthCol = createColumnInfoData(TSDB_DATA_TYPE_INT, 4, 4); - blockDataAppendColInfo(pSortInput, &lengthCol); + code = blockDataAppendColInfo(pSortInput, &lengthCol); + if (code) { + return code; + } if (pHandle->bSortPk) { pkCol = createColumnInfoData(extPkCol->info.type, extPkCol->info.bytes, 5); - blockDataAppendColInfo(pSortInput, &pkCol); + code = blockDataAppendColInfo(pSortInput, &pkCol); } blockDataDestroy(pHandle->pDataBlock); @@ -1341,6 +1503,9 @@ static void initRowIdSort(SSortHandle* pHandle) { pHandle->numOfPages = 256; SArray* pOrderInfoList = taosArrayInit(1, sizeof(SBlockOrderInfo)); + if (pOrderInfoList == NULL) { + return TSDB_CODE_OUT_OF_MEMORY; + } int32_t tsOrder = ((SBlockOrderInfo*)taosArrayGet(pHandle->pSortInfo, 0))->order; @@ -1349,7 +1514,10 @@ static void initRowIdSort(SSortHandle* pHandle) { biTs.slotId = 0; biTs.nullFirst = (biTs.order == TSDB_ORDER_ASC); biTs.compFn = getKeyComparFunc(TSDB_DATA_TYPE_TIMESTAMP, biTs.order); - taosArrayPush(pOrderInfoList, &biTs); + void* p = taosArrayPush(pOrderInfoList, &biTs); + if (p == NULL) { + return TSDB_CODE_OUT_OF_MEMORY; + } if (pHandle->bSortPk) { SBlockOrderInfo biPk = {0}; @@ -1357,24 +1525,34 @@ static void initRowIdSort(SSortHandle* pHandle) { biPk.slotId = 4; biPk.nullFirst = (biPk.order == TSDB_ORDER_ASC); biPk.compFn = getKeyComparFunc(pkCol.info.type, biPk.order); - taosArrayPush(pOrderInfoList, &biPk); + + void* px = taosArrayPush(pOrderInfoList, &biPk); + if (px == NULL) { + return TSDB_CODE_OUT_OF_MEMORY; + } } taosArrayDestroy(pHandle->pSortInfo); pHandle->pSortInfo = pOrderInfoList; + return TSDB_CODE_SUCCESS; } int32_t tsortSetSortByRowId(SSortHandle* pHandle, int32_t extRowsMemSize) { pHandle->extRowBytes = blockDataGetRowSize(pHandle->pDataBlock) + taosArrayGetSize(pHandle->pDataBlock->pDataBlock) + sizeof(int32_t); pHandle->extRowsMemSize = extRowsMemSize; pHandle->aExtRowsOrders = taosArrayDup(pHandle->pSortInfo, NULL); - initRowIdSort(pHandle); - if (!osTempSpaceAvailable()) { - terrno = TSDB_CODE_NO_DISKSPACE; - qError("create sort mem file failed since %s, tempDir:%s", terrstr(), tsTempDir); - return terrno; + + int32_t code = initRowIdSort(pHandle); + if (code) { + return code; } - int32_t code = createSortMemFile(pHandle); + + if (!osTempSpaceAvailable()) { + qError("create sort mem file failed since %s, tempDir:%s", terrstr(), tsTempDir); + return TSDB_CODE_NO_DISKSPACE; + } + + code = createSortMemFile(pHandle); pHandle->bSortByRowId = true; return code; } @@ -1439,17 +1617,21 @@ static int32_t appendDataBlockToPageBuf(SSortHandle* pHandle, SSDataBlock* blk, if (pPage == NULL) { return terrno; } - taosArrayPush(aPgId, &pageId); + + void* px = taosArrayPush(aPgId, &pageId); + if (px == NULL) { + return TSDB_CODE_OUT_OF_MEMORY; + } int32_t size = blockDataGetSize(blk) + sizeof(int32_t) + taosArrayGetSize(blk->pDataBlock) * sizeof(int32_t); ASSERT(size <= getBufPageSize(pHandle->pBuf)); - blockDataToBuf(pPage, blk); + int32_t code = blockDataToBuf(pPage, blk); setBufPageDirty(pPage, true); releaseBufPage(pHandle->pBuf, pPage); - return 0; + return code; } static int32_t getPageBufIncForRow(SSDataBlock* pSrcBlock, int32_t srcRowIndex, int32_t dstRowIndex) { @@ -1590,9 +1772,12 @@ static int32_t sortBlocksToExtSource(SSortHandle* pHandle, SArray* aBlk, SArray* int32_t pageHeaderSize = sizeof(int32_t) + sizeof(int32_t) * blockDataGetNumOfCols(pHandle->pDataBlock); int32_t rowCap = blockDataGetCapacityInRow(pHandle->pDataBlock, pHandle->pageSize, pageHeaderSize); - blockDataEnsureCapacity(pHandle->pDataBlock, rowCap); - blockDataCleanup(pHandle->pDataBlock); + code = blockDataEnsureCapacity(pHandle->pDataBlock, rowCap); + if (code) { + return code; + } + blockDataCleanup(pHandle->pDataBlock); SBlkMergeSupport sup = {0}; SBlockOrderInfo* pOrigBlockTsOrder = @@ -1606,7 +1791,10 @@ static int32_t sortBlocksToExtSource(SSortHandle* pHandle, SArray* aBlk, SArray* (!pHandle->bSortByRowId) ? taosArrayGet(pHandle->pSortInfo, 1) : taosArrayGet(pHandle->aExtRowsOrders, 1); } - initMergeSup(&sup, aBlk, pOrigBlockTsOrder->order, pOrigBlockTsOrder->slotId, pOrigBlockPkOrder); + code = initMergeSup(&sup, aBlk, pOrigBlockTsOrder->order, pOrigBlockTsOrder->slotId, pOrigBlockPkOrder); + if (code) { + return code; + } int32_t totalRows = getTotalRows(aBlk); @@ -1661,11 +1849,19 @@ static int32_t sortBlocksToExtSource(SSortHandle* pHandle, SArray* aBlk, SArray* } } - blockDataEnsureCapacity(pHandle->pDataBlock, pHandle->pDataBlock->info.rows + 1); + code = blockDataEnsureCapacity(pHandle->pDataBlock, pHandle->pDataBlock->info.rows + 1); + if (code) { + return code; + } + if (pHandle->bSortByRowId) { - appendToRowIndexDataBlock(pHandle, minBlk, &minRow); + code = appendToRowIndexDataBlock(pHandle, minBlk, &minRow); } else { - appendOneRowToDataBlock(pHandle->pDataBlock, minBlk, &minRow); + code = appendOneRowToDataBlock(pHandle->pDataBlock, minBlk, &minRow); + } + + if (code) { + return code; } blkPgSz += bufInc; @@ -1705,22 +1901,30 @@ static int32_t sortBlocksToExtSource(SSortHandle* pHandle, SArray* aBlk, SArray* } SSDataBlock* pMemSrcBlk = createOneDataBlock(pHandle->pDataBlock, false); - doAddNewExternalMemSource(pHandle->pBuf, aExtSrc, pMemSrcBlk, &pHandle->sourceId, aPgId); + code = doAddNewExternalMemSource(pHandle->pBuf, aExtSrc, pMemSrcBlk, &pHandle->sourceId, aPgId); cleanupMergeSup(&sup); tMergeTreeDestroy(&pTree); - return 0; + return code; } -static SSDataBlock* getRowsBlockWithinMergeLimit(const SSortHandle* pHandle, SSHashObj* mTableNumRows, SSDataBlock* pOrigBlk, bool* pExtractedBlock, bool *pSkipBlock) { +static int32_t getRowsBlockWithinMergeLimit(const SSortHandle* pHandle, SSHashObj* mTableNumRows, SSDataBlock* pOrigBlk, + bool* pExtractedBlock, bool* pSkipBlock, SSDataBlock** pRes) { int64_t nRows = 0; int64_t prevRows = 0; + int32_t code = 0; + + *pRes = NULL; + void* pNum = tSimpleHashGet(mTableNumRows, &pOrigBlk->info.id.uid, sizeof(pOrigBlk->info.id.uid)); if (pNum == NULL) { prevRows = 0; nRows = pOrigBlk->info.rows; - tSimpleHashPut(mTableNumRows, &pOrigBlk->info.id.uid, sizeof(pOrigBlk->info.id.uid), &nRows, sizeof(nRows)); + code = tSimpleHashPut(mTableNumRows, &pOrigBlk->info.id.uid, sizeof(pOrigBlk->info.id.uid), &nRows, sizeof(nRows)); + if (code) { + return code; + } } else { prevRows = *(int64_t*)pNum; *(int64_t*)pNum = *(int64_t*)pNum + pOrigBlk->info.rows; @@ -1737,7 +1941,7 @@ static SSDataBlock* getRowsBlockWithinMergeLimit(const SSortHandle* pHandle, SSH if (keepRows == 0) { *pSkipBlock = true; - return pOrigBlk; + *pRes = pOrigBlk; } *pSkipBlock = false; @@ -1749,7 +1953,9 @@ static SSDataBlock* getRowsBlockWithinMergeLimit(const SSortHandle* pHandle, SSH *pExtractedBlock = false; pBlock = pOrigBlk; } - return pBlock; + + *pRes = pBlock; + return code; } static int32_t createBlocksMergeSortInitialSources(SSortHandle* pHandle) { @@ -1784,8 +1990,10 @@ static int32_t createBlocksMergeSortInitialSources(SSortHandle* pHandle) { bool bExtractedBlock = false; bool bSkipBlock = false; if (pBlk != NULL && pHandle->mergeLimit > 0) { - pBlk = getRowsBlockWithinMergeLimit(pHandle, mTableNumRows, pBlk, &bExtractedBlock, &bSkipBlock); - if (bSkipBlock) { + pBlk = NULL; + + code = getRowsBlockWithinMergeLimit(pHandle, mTableNumRows, pBlk, &bExtractedBlock, &bSkipBlock, &pBlk); + if (bSkipBlock || code != 0) { continue; } } @@ -1807,14 +2015,26 @@ static int32_t createBlocksMergeSortInitialSources(SSortHandle* pHandle) { void* ppBlk = tSimpleHashGet(mUidBlk, &pBlk->info.id.uid, sizeof(pBlk->info.id.uid)); if (ppBlk != NULL) { SSDataBlock* tBlk = *(SSDataBlock**)(ppBlk); - blockDataMerge(tBlk, pBlk); + + code = blockDataMerge(tBlk, pBlk); + if (code) { + return code; + } + if (bExtractedBlock) { blockDataDestroy(pBlk); } } else { SSDataBlock* tBlk = (bExtractedBlock) ? pBlk : createOneDataBlock(pBlk, true); - tSimpleHashPut(mUidBlk, &pBlk->info.id.uid, sizeof(pBlk->info.id.uid), &tBlk, POINTER_BYTES); - taosArrayPush(aBlkSort, &tBlk); + code = tSimpleHashPut(mUidBlk, &pBlk->info.id.uid, sizeof(pBlk->info.id.uid), &tBlk, POINTER_BYTES); + if (code) { + return code; + } + + void* px = taosArrayPush(aBlkSort, &tBlk); + if (px == NULL) { + return TSDB_CODE_OUT_OF_MEMORY; + } } } @@ -1823,10 +2043,13 @@ static int32_t createBlocksMergeSortInitialSources(SSortHandle* pHandle) { int64_t p = taosGetTimestampUs(); if (pHandle->bSortByRowId) { - tsortOpenRegion(pHandle); + code = tsortOpenRegion(pHandle); + if (code) { + return code; + } } - code = sortBlocksToExtSource(pHandle, aBlkSort, aExtSrc); + code = sortBlocksToExtSource(pHandle, aBlkSort, aExtSrc); if (code != TSDB_CODE_SUCCESS) { for (int32_t i = 0; i < taosArrayGetSize(aBlkSort); ++i) { blockDataDestroy(taosArrayGetP(aBlkSort, i)); @@ -1834,15 +2057,18 @@ static int32_t createBlocksMergeSortInitialSources(SSortHandle* pHandle) { taosArrayClear(aBlkSort); break; } + if (pHandle->bSortByRowId) { - tsortCloseRegion(pHandle); + code = tsortCloseRegion(pHandle); // ignore this error code } + int64_t el = taosGetTimestampUs() - p; pHandle->sortElapsed += el; for (int32_t i = 0; i < taosArrayGetSize(aBlkSort); ++i) { blockDataDestroy(taosArrayGetP(aBlkSort, i)); } + taosArrayClear(aBlkSort); szSort = 0; qDebug("source %zu created", taosArrayGetSize(aExtSrc)); @@ -1869,13 +2095,18 @@ static int32_t createBlocksMergeSortInitialSources(SSortHandle* pHandle) { taosArrayDestroy(aBlkSort); tsortClearOrderdSource(pHandle->pOrderedSource, NULL, NULL); if (!tsortIsClosed(pHandle)) { - taosArrayAddAll(pHandle->pOrderedSource, aExtSrc); + void* px = taosArrayAddAll(pHandle->pOrderedSource, aExtSrc); + if (px == NULL) { + code = TSDB_CODE_OUT_OF_MEMORY; + } } + taosArrayDestroy(aExtSrc); tSimpleHashCleanup(mTableNumRows); if (pHandle->bSortByRowId) { - tsortFinalizeRegions(pHandle); + code = tsortFinalizeRegions(pHandle); } + pHandle->type = SORT_SINGLESOURCE_SORT; return code; } @@ -2023,9 +2254,8 @@ static bool tsortOpenForBufMergeSort(SSortHandle* pHandle) { return tMergeTreeCreate(&pHandle->pMergeTree, pHandle->cmpParam.numOfSources, &pHandle->cmpParam, pHandle->comparFn); } -int32_t tsortClose(SSortHandle* pHandle) { - atomic_val_compare_exchange_8(&pHandle->closed, 0, 1); - return TSDB_CODE_SUCCESS; +void tsortClose(SSortHandle* pHandle) { + (void) atomic_val_compare_exchange_8(&pHandle->closed, 0, 1); } bool tsortIsClosed(SSortHandle* pHandle) { @@ -2048,22 +2278,28 @@ int32_t tsortSetFetchRawDataFp(SSortHandle* pHandle, _sort_fetch_block_fn_t fetc return TSDB_CODE_SUCCESS; } -int32_t tsortSetComparFp(SSortHandle* pHandle, _sort_merge_compar_fn_t fp) { - pHandle->comparFn = fp; - return TSDB_CODE_SUCCESS; -} - -int32_t tsortSetCompareGroupId(SSortHandle* pHandle, bool compareGroupId) { - pHandle->cmpParam.cmpGroupId = compareGroupId; - return TSDB_CODE_SUCCESS; -} - -static STupleHandle* tsortBufMergeSortNextTuple(SSortHandle* pHandle) { - if (tsortIsClosed(pHandle)) { - return NULL; +void tsortSetComparFp(SSortHandle* pHandle, _sort_merge_compar_fn_t fp) { + if (pHandle) { + pHandle->comparFn = fp; } +} + +void tsortSetCompareGroupId(SSortHandle* pHandle, bool compareGroupId) { + if (pHandle) { + pHandle->cmpParam.cmpGroupId = compareGroupId; + } +} + +static int32_t tsortBufMergeSortNextTuple(SSortHandle* pHandle, STupleHandle** pTupleHandle) { + *pTupleHandle = NULL; + int32_t code = 0; + + if (tsortIsClosed(pHandle)) { + return code; + } + if (pHandle->cmpParam.numOfSources == pHandle->numOfCompletedSources) { - return NULL; + return code; } // All the data are hold in the buffer, no external sort is invoked. @@ -2071,26 +2307,26 @@ static STupleHandle* tsortBufMergeSortNextTuple(SSortHandle* pHandle) { pHandle->tupleHandle.rowIndex += 1; if (pHandle->tupleHandle.rowIndex == pHandle->pDataBlock->info.rows) { pHandle->numOfCompletedSources = 1; - return NULL; + return code; } - return &pHandle->tupleHandle; + *pTupleHandle = &pHandle->tupleHandle; + return code; } int32_t index = tMergeTreeGetChosenIndex(pHandle->pMergeTree); SSortSource* pSource = pHandle->cmpParam.pSources[index]; if (pHandle->needAdjust) { - int32_t code = adjustMergeTreeForNextTuple(pSource, pHandle->pMergeTree, pHandle, &pHandle->numOfCompletedSources); + code = adjustMergeTreeForNextTuple(pSource, pHandle->pMergeTree, pHandle, &pHandle->numOfCompletedSources); if (code != TSDB_CODE_SUCCESS) { - terrno = code; - return NULL; + return code; } } // all sources are completed. if (pHandle->cmpParam.numOfSources == pHandle->numOfCompletedSources) { - return NULL; + return code; } // Get the adjusted value after the loser tree is updated. @@ -2105,7 +2341,8 @@ static STupleHandle* tsortBufMergeSortNextTuple(SSortHandle* pHandle) { pHandle->needAdjust = true; pSource->src.rowIndex += 1; - return &pHandle->tupleHandle; + *pTupleHandle = &pHandle->tupleHandle; + return code; } static bool tsortIsForceUsePQSort(SSortHandle* pHandle) { @@ -2216,12 +2453,20 @@ static int32_t tsortOpenForPQSort(SSortHandle* pHandle) { return TSDB_CODE_SUCCESS; } -static STupleHandle* tsortPQSortNextTuple(SSortHandle* pHandle) { +static int32_t tsortPQSortNextTuple(SSortHandle* pHandle, STupleHandle **pTupleHandle) { + int32_t code = 0; + + *pTupleHandle = NULL; if (pHandle->pDataBlock == NULL) { // when no input stream datablock - return NULL; + return code; } + blockDataCleanup(pHandle->pDataBlock); - blockDataEnsureCapacity(pHandle->pDataBlock, 1); + code = blockDataEnsureCapacity(pHandle->pDataBlock, 1); + if (code) { + return code; + } + // abandon the top tuple if queue size bigger than max size if (taosBQSize(pHandle->pBoundedQueue) == taosBQMaxSize(pHandle->pBoundedQueue) + 1) { taosBQPop(pHandle->pBoundedQueue); @@ -2241,36 +2486,55 @@ static STupleHandle* tsortPQSortNextTuple(SSortHandle* pHandle) { if (!pData) { colDataSetNULL(bdGetColumnInfoData(pHandle->pDataBlock, i), 0); } else { - colDataSetVal(bdGetColumnInfoData(pHandle->pDataBlock, i), 0, pData, false); + code = colDataSetVal(bdGetColumnInfoData(pHandle->pDataBlock, i), 0, pData, false); + if (code) { + return code; + } } } pHandle->pDataBlock->info.rows++; pHandle->tmpRowIdx++; taosBQPop(pHandle->pBoundedQueue); } - if (pHandle->pDataBlock->info.rows == 0) return NULL; + + if (pHandle->pDataBlock->info.rows == 0) { + return code; + } + pHandle->tupleHandle.pBlock = pHandle->pDataBlock; - return &pHandle->tupleHandle; + *pTupleHandle = &pHandle->tupleHandle; + return code; } -static STupleHandle* tsortSingleTableMergeNextTuple(SSortHandle* pHandle) { - if (1 == pHandle->numOfCompletedSources) return NULL; +static int32_t tsortSingleTableMergeNextTuple(SSortHandle* pHandle, STupleHandle** pTupleHandle) { + *pTupleHandle = NULL; + int32_t code = 0; + + if (1 == pHandle->numOfCompletedSources) { + return code; + } + if (pHandle->tupleHandle.pBlock && pHandle->tupleHandle.rowIndex + 1 < pHandle->tupleHandle.pBlock->info.rows) { pHandle->tupleHandle.rowIndex++; } else { - if (pHandle->tupleHandle.rowIndex == -1) return NULL; + if (pHandle->tupleHandle.rowIndex == -1) { + return code; + } + SSortSource** pSource = taosArrayGet(pHandle->pOrderedSource, 0); SSortSource* source = *pSource; SSDataBlock* pBlock = pHandle->fetchfp(source->param); if (!pBlock || pBlock->info.rows == 0) { setCurrentSourceDone(source, pHandle); pHandle->tupleHandle.pBlock = NULL; - return NULL; + return code; } pHandle->tupleHandle.pBlock = pBlock; pHandle->tupleHandle.rowIndex = 0; } - return &pHandle->tupleHandle; + + *pTupleHandle = &pHandle->tupleHandle; + return code; } int32_t tsortOpen(SSortHandle* pHandle) { @@ -2289,13 +2553,18 @@ int32_t tsortOpen(SSortHandle* pHandle) { return tsortOpenForBufMergeSort(pHandle); } -STupleHandle* tsortNextTuple(SSortHandle* pHandle) { - if (pHandle->singleTableMerge) - return tsortSingleTableMergeNextTuple(pHandle); - else if (pHandle->pBoundedQueue) - return tsortPQSortNextTuple(pHandle); - else - return tsortBufMergeSortNextTuple(pHandle); +int32_t tsortNextTuple(SSortHandle* pHandle, STupleHandle** pTupleHandle) { + int32_t code = 0; + + if (pHandle->singleTableMerge) { + code = tsortSingleTableMergeNextTuple(pHandle, pTupleHandle); + } else if (pHandle->pBoundedQueue) { + code = tsortPQSortNextTuple(pHandle, pTupleHandle); + } else { + code = tsortBufMergeSortNextTuple(pHandle, pTupleHandle); + } + + return code; } bool tsortIsNullVal(STupleHandle* pVHandle, int32_t colIndex) { @@ -2303,12 +2572,11 @@ bool tsortIsNullVal(STupleHandle* pVHandle, int32_t colIndex) { return colDataIsNull_s(pColInfoSrc, pVHandle->rowIndex); } -void* tsortGetValue(STupleHandle* pVHandle, int32_t colIndex) { +void tsortGetValue(STupleHandle* pVHandle, int32_t colIndex, void** pVal) { + *pVal = NULL; SColumnInfoData* pColInfo = TARRAY_GET_ELEM(pVHandle->pBlock->pDataBlock, colIndex); - if (pColInfo->pData == NULL) { - return NULL; - } else { - return colDataGetData(pColInfo, pVHandle->rowIndex); + if (pColInfo->pData != NULL) { + *pVal = colDataGetData(pColInfo, pVHandle->rowIndex); } } diff --git a/source/libs/stream/src/streamCheckStatus.c b/source/libs/stream/src/streamCheckStatus.c index 55209fc427..66c1941abb 100644 --- a/source/libs/stream/src/streamCheckStatus.c +++ b/source/libs/stream/src/streamCheckStatus.c @@ -36,7 +36,7 @@ static void setCheckDownstreamReqInfo(SStreamTaskCheckReq* pReq, int64_t reqId, static void getCheckRspStatus(STaskCheckInfo* pInfo, int64_t el, int32_t* numOfReady, int32_t* numOfFault, int32_t* numOfNotRsp, SArray* pTimeoutList, SArray* pNotReadyList, const char* id); static int32_t addDownstreamFailedStatusResultAsync(SMsgCb* pMsgCb, int32_t vgId, int64_t streamId, int32_t taskId); -static SDownstreamStatusInfo* findCheckRspStatus(STaskCheckInfo* pInfo, int32_t taskId); +static void findCheckRspStatus(STaskCheckInfo* pInfo, int32_t taskId, SDownstreamStatusInfo** pStatusInfo); int32_t streamTaskCheckStatus(SStreamTask* pTask, int32_t upstreamTaskId, int32_t vgId, int64_t stage, int64_t* oldStage) { @@ -402,26 +402,29 @@ void streamTaskInitTaskCheckInfo(STaskCheckInfo* pInfo, STaskOutputInfo* pOutput pInfo->stopCheckProcess = 0; } -SDownstreamStatusInfo* findCheckRspStatus(STaskCheckInfo* pInfo, int32_t taskId) { +void findCheckRspStatus(STaskCheckInfo* pInfo, int32_t taskId, SDownstreamStatusInfo** pStatusInfo) { + if (pStatusInfo == NULL) { + return; + } + + *pStatusInfo = NULL; for (int32_t j = 0; j < taosArrayGetSize(pInfo->pList); ++j) { SDownstreamStatusInfo* p = taosArrayGet(pInfo->pList, j); if (p->taskId == taskId) { - return p; + *pStatusInfo = p; } } - - return NULL; } int32_t streamTaskUpdateCheckInfo(STaskCheckInfo* pInfo, int32_t taskId, int32_t status, int64_t rspTs, int64_t reqId, int32_t* pNotReady, const char* id) { streamMutexLock(&pInfo->checkInfoLock); - SDownstreamStatusInfo* p = findCheckRspStatus(pInfo, taskId); + SDownstreamStatusInfo* p = NULL; + findCheckRspStatus(pInfo, taskId, &p); if (p != NULL) { if (reqId != p->reqId) { - stError("s-task:%s reqId:0x%" PRIx64 " expected:0x%" PRIx64 - " expired check-rsp recv from downstream task:0x%x, discarded", + stError("s-task:%s reqId:0x%" PRIx64 " expected:0x%" PRIx64 " expired check-rsp recv from downstream task:0x%x, discarded", id, reqId, p->reqId, taskId); streamMutexUnlock(&pInfo->checkInfoLock); return TSDB_CODE_FAILED; @@ -495,7 +498,8 @@ void streamTaskAddReqInfo(STaskCheckInfo* pInfo, int64_t reqId, int32_t taskId, SDownstreamStatusInfo info = {.taskId = taskId, .status = -1, .vgId = vgId, .reqId = reqId, .rspTs = 0}; streamMutexLock(&pInfo->checkInfoLock); - SDownstreamStatusInfo* p = findCheckRspStatus(pInfo, taskId); + SDownstreamStatusInfo* p = NULL; + findCheckRspStatus(pInfo, taskId, &p); if (p != NULL) { stDebug("s-task:%s check info to task:0x%x already sent", id, taskId); streamMutexUnlock(&pInfo->checkInfoLock); @@ -598,7 +602,8 @@ void handleTimeoutDownstreamTasks(SStreamTask* pTask, SArray* pTimeoutList) { for (int32_t i = 0; i < numOfTimeout; ++i) { int32_t taskId = *(int32_t*)taosArrayGet(pTimeoutList, i); - SDownstreamStatusInfo* p = findCheckRspStatus(pInfo, taskId); + SDownstreamStatusInfo* p = NULL; + findCheckRspStatus(pInfo, taskId, &p); if (p != NULL) { ASSERT(p->status == -1 && p->rspTs == 0); doSendCheckMsg(pTask, p); @@ -613,7 +618,8 @@ void handleTimeoutDownstreamTasks(SStreamTask* pTask, SArray* pTimeoutList) { for (int32_t i = 0; i < numOfTimeout; ++i) { int32_t taskId = *(int32_t*)taosArrayGet(pTimeoutList, i); - SDownstreamStatusInfo* p = findCheckRspStatus(pInfo, taskId); + SDownstreamStatusInfo* p = NULL; + findCheckRspStatus(pInfo, taskId, &p); if (p != NULL) { addIntoNodeUpdateList(pTask, p->vgId); stDebug("s-task:%s vgId:%d downstream task:0x%x (vgId:%d) timeout more than 100sec, add into nodeUpate list", @@ -640,7 +646,8 @@ void handleNotReadyDownstreamTask(SStreamTask* pTask, SArray* pNotReadyList) { for (int32_t i = 0; i < numOfNotReady; ++i) { int32_t taskId = *(int32_t*)taosArrayGet(pNotReadyList, i); - SDownstreamStatusInfo* p = findCheckRspStatus(pInfo, taskId); + SDownstreamStatusInfo* p = NULL; + findCheckRspStatus(pInfo, taskId, &p); if (p != NULL) { p->rspTs = 0; p->status = -1; diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 1283f8e20b..43a22e0637 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -726,7 +726,7 @@ int32_t streamMetaAcquireTaskNoLock(SStreamMeta* pMeta, int64_t streamId, int32_ SStreamTask** ppTask = (SStreamTask**)taosHashGet(pMeta->pTasksMap, &id, sizeof(id)); if (ppTask == NULL || streamTaskShouldStop(*ppTask)) { *pTask = NULL; - return TSDB_CODE_FAILED; + return TSDB_CODE_STREAM_TASK_NOT_EXIST; } int32_t ref = atomic_add_fetch_32(&(*ppTask)->refCnt, 1); diff --git a/source/libs/stream/src/streamSched.c b/source/libs/stream/src/streamSched.c index 74f370d199..f9fcf36668 100644 --- a/source/libs/stream/src/streamSched.c +++ b/source/libs/stream/src/streamSched.c @@ -104,10 +104,10 @@ void streamTaskResumeHelper(void* param, void* tmrId) { } int32_t code = streamTaskSchedTask(pTask->pMsgCb, pTask->info.nodeId, pId->streamId, pId->taskId, STREAM_EXEC_T_RESUME_TASK); + int32_t ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1); if (code) { - stError("s-task:%s sched task failed, code:%s", pTask->id.idStr, strerror(code)); + stError("s-task:%s sched task failed, code:%s, ref:%d", pId->idStr, strerror(code), ref); } else { - int32_t ref = atomic_sub_fetch_32(&pTask->status.timerActive, 1); stDebug("trigger to resume s-task:%s after being idled for %dms, ref:%d", pId->idStr, pTask->status.schedIdleTime, ref); From cd0722fe1f52641d1312d01369ae7e87d4a8a980 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 23 Jul 2024 16:38:05 +0800 Subject: [PATCH 02/14] fix(stream): check return value. --- source/libs/executor/inc/tsort.h | 8 ++--- source/libs/executor/src/tsort.c | 53 ++++++++++++++++++++++++-------- 2 files changed, 45 insertions(+), 16 deletions(-) diff --git a/source/libs/executor/inc/tsort.h b/source/libs/executor/inc/tsort.h index c6f3986776..cf1464ccd7 100644 --- a/source/libs/executor/inc/tsort.h +++ b/source/libs/executor/inc/tsort.h @@ -165,13 +165,13 @@ void tsortGetValue(STupleHandle* pVHandle, int32_t colId, void** pVal); * @return */ uint64_t tsortGetGroupId(STupleHandle* pVHandle); -void* tsortGetBlockInfo(STupleHandle* pVHandle); +void tsortGetBlockInfo(STupleHandle* pVHandle, SDataBlockInfo* pInfo); /** * * @param pSortHandle * @return */ -SSDataBlock* tsortGetSortedDataBlock(const SSortHandle* pSortHandle); +int32_t tsortGetSortedDataBlock(const SSortHandle* pSortHandle, SSDataBlock** pBlock); /** * return the sort execution information. @@ -215,8 +215,8 @@ int32_t tsortCompAndBuildKeys(const SArray* pSortCols, char* keyBuf, int32_t* ke */ void tsortSetMergeLimitReachedFp(SSortHandle* pHandle, void (*mergeLimitReached)(uint64_t tableUid, void* param), void* param); -int tsortComparBlockCell(SSDataBlock* pLeftBlock, SSDataBlock* pRightBlock, - int32_t leftRowIndex, int32_t rightRowIndex, void* pOrder); +int tsortComparBlockCell(SSDataBlock* pLeftBlock, SSDataBlock* pRightBlock, int32_t leftRowIndex, int32_t rightRowIndex, + void* pOrder); #ifdef __cplusplus } #endif diff --git a/source/libs/executor/src/tsort.c b/source/libs/executor/src/tsort.c index 33596c4802..b4e5a9a01a 100644 --- a/source/libs/executor/src/tsort.c +++ b/source/libs/executor/src/tsort.c @@ -167,8 +167,17 @@ static void* tupleGetField(char* t, uint32_t colIdx, uint32_t colNum) { return t + *tupleOffset(t, colIdx); } -SSDataBlock* tsortGetSortedDataBlock(const SSortHandle* pSortHandle) { - return createOneDataBlock(pSortHandle->pDataBlock, false); +int32_t tsortGetSortedDataBlock(const SSortHandle* pSortHandle, SSDataBlock** pBlock) { + if (pBlock == NULL) { + return TSDB_CODE_INVALID_PARA; + } + + *pBlock = createOneDataBlock(pSortHandle->pDataBlock, false); + if (*pBlock == NULL) { + return TSDB_CODE_OUT_OF_MEMORY; + } else { + return TSDB_CODE_SUCCESS; + } } #define AllocatedTupleType 0 @@ -1099,7 +1108,11 @@ int32_t tsortAppendTupleToBlock(SSortHandle* pHandle, SSDataBlock* pBlock, STupl } pBlock->info.dataLoad = 1; - pBlock->info.scanFlag = ((SDataBlockInfo*)tsortGetBlockInfo(pTupleHandle))->scanFlag; + + SDataBlockInfo info = {0}; + tsortGetBlockInfo(pTupleHandle, &info); + + pBlock->info.scanFlag = info.scanFlag; pBlock->info.rows += 1; } else { @@ -1121,7 +1134,10 @@ int32_t tsortAppendTupleToBlock(SSortHandle* pHandle, SSDataBlock* pBlock, STupl } pBlock->info.dataLoad = 1; - pBlock->info.scanFlag = ((SDataBlockInfo*)tsortGetBlockInfo(pTupleHandle))->scanFlag; + SDataBlockInfo info = {0}; + tsortGetBlockInfo(pTupleHandle, &info); + + pBlock->info.scanFlag = info.scanFlag; pBlock->info.rows += 1; } @@ -1463,35 +1479,41 @@ static int32_t initRowIdSort(SSortHandle* pHandle) { SColumnInfoData pkCol = {0}; SSDataBlock* pSortInput = createDataBlock(); - pHandle->pDataBlock = pSortInput; - SColumnInfoData tsCol = createColumnInfoData(TSDB_DATA_TYPE_TIMESTAMP, 8, 1); int32_t code = blockDataAppendColInfo(pSortInput, &tsCol); if (code) { + blockDataDestroy(pSortInput); return code; } SColumnInfoData regionIdCol = createColumnInfoData(TSDB_DATA_TYPE_INT, 4, 2); code = blockDataAppendColInfo(pSortInput, ®ionIdCol); if (code) { + blockDataDestroy(pSortInput); return code; } SColumnInfoData offsetCol = createColumnInfoData(TSDB_DATA_TYPE_INT, 4, 3); code = blockDataAppendColInfo(pSortInput, &offsetCol); if (code) { + blockDataDestroy(pSortInput); return code; } SColumnInfoData lengthCol = createColumnInfoData(TSDB_DATA_TYPE_INT, 4, 4); code = blockDataAppendColInfo(pSortInput, &lengthCol); if (code) { + blockDataDestroy(pSortInput); return code; } if (pHandle->bSortPk) { pkCol = createColumnInfoData(extPkCol->info.type, extPkCol->info.bytes, 5); code = blockDataAppendColInfo(pSortInput, &pkCol); + if (code) { + blockDataDestroy(pSortInput); + return code; + } } blockDataDestroy(pHandle->pDataBlock); @@ -1990,12 +2012,13 @@ static int32_t createBlocksMergeSortInitialSources(SSortHandle* pHandle) { bool bExtractedBlock = false; bool bSkipBlock = false; if (pBlk != NULL && pHandle->mergeLimit > 0) { - pBlk = NULL; - - code = getRowsBlockWithinMergeLimit(pHandle, mTableNumRows, pBlk, &bExtractedBlock, &bSkipBlock, &pBlk); + SSDataBlock* p = NULL; + code = getRowsBlockWithinMergeLimit(pHandle, mTableNumRows, pBlk, &bExtractedBlock, &bSkipBlock, &p); if (bSkipBlock || code != 0) { continue; } + + pBlk = p; } if (pBlk != NULL) { @@ -2416,15 +2439,21 @@ static int32_t tsortOpenForPQSort(SSortHandle* pHandle) { while (1) { // fetch data SSDataBlock* pBlock = pHandle->fetchfp(source->param); - if (NULL == pBlock) break; + if (NULL == pBlock) { + break; + } if (pHandle->beforeFp != NULL) { pHandle->beforeFp(pBlock, pHandle->param); } + if (pHandle->pDataBlock == NULL) { pHandle->pDataBlock = createOneDataBlock(pBlock, false); } - if (pHandle->pDataBlock == NULL) return TSDB_CODE_OUT_OF_MEMORY; + + if (pHandle->pDataBlock == NULL) { + return TSDB_CODE_OUT_OF_MEMORY; + } size_t colNum = blockDataGetNumOfCols(pBlock); @@ -2581,7 +2610,7 @@ void tsortGetValue(STupleHandle* pVHandle, int32_t colIndex, void** pVal) { } uint64_t tsortGetGroupId(STupleHandle* pVHandle) { return pVHandle->pBlock->info.id.groupId; } -void* tsortGetBlockInfo(STupleHandle* pVHandle) { return &pVHandle->pBlock->info; } +void tsortGetBlockInfo(STupleHandle* pVHandle, SDataBlockInfo* pBlockInfo) { *pBlockInfo = pVHandle->pBlock->info; } SSortExecInfo tsortGetSortExecInfo(SSortHandle* pHandle) { SSortExecInfo info = {0}; From eefee54c6aa64a6f871cd9a8f12d2497a7ff2626 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 23 Jul 2024 17:31:41 +0800 Subject: [PATCH 03/14] fix(stream): check return value. --- source/libs/executor/inc/operator.h | 4 +- source/libs/executor/inc/querytask.h | 21 +++---- source/libs/executor/src/executor.c | 30 ++++++++-- source/libs/executor/src/executorInt.c | 7 ++- source/libs/executor/src/mergeoperator.c | 6 +- source/libs/executor/src/querytask.c | 65 +++++++++++++--------- source/libs/executor/src/sortoperator.c | 19 ++++--- source/libs/stream/src/streamCheckStatus.c | 7 ++- source/libs/stream/src/streamCheckpoint.c | 42 ++++++++++++++ 9 files changed, 143 insertions(+), 58 deletions(-) diff --git a/source/libs/executor/inc/operator.h b/source/libs/executor/inc/operator.h index 95208545bd..37cdd5d7b3 100644 --- a/source/libs/executor/inc/operator.h +++ b/source/libs/executor/inc/operator.h @@ -140,7 +140,7 @@ SOperatorInfo* createTimeSliceOperatorInfo(SOperatorInfo* downstream, SPhysiNode SOperatorInfo* createMergeJoinOperatorInfo(SOperatorInfo** pDownstream, int32_t numOfDownstream, SSortMergeJoinPhysiNode* pJoinNode, SExecTaskInfo* pTaskInfo); -SOperatorInfo* createHashJoinOperatorInfo(SOperatorInfo** pDownstream, int32_t numOfDownstream, SHashJoinPhysiNode* pJoinNode, SExecTaskInfo* pTaskInfo); +SOperatorInfo* createHashJoinOperatorInfo(SOperatorInfo** pDownstream, int32_t numOfDownstream, SHashJoinPhysiNode* pJoinNode, SExecTaskInfo* pTaskInfo); SOperatorInfo* createStreamSessionAggOperatorInfo(SOperatorInfo* downstream, SPhysiNode* pPhyNode, SExecTaskInfo* pTaskInfo, SReadHandle* pHandle); @@ -164,7 +164,7 @@ SOperatorInfo* createCountwindowOperatorInfo(SOperatorInfo* downstream, SPhysiNo SOperatorInfo* createGroupCacheOperatorInfo(SOperatorInfo** pDownstream, int32_t numOfDownstream, SGroupCachePhysiNode* pPhyciNode, SExecTaskInfo* pTaskInfo); -SOperatorInfo* createDynQueryCtrlOperatorInfo(SOperatorInfo** pDownstream, int32_t numOfDownstream, SDynQueryCtrlPhysiNode* pPhyciNode, SExecTaskInfo* pTaskInfo); +SOperatorInfo* createDynQueryCtrlOperatorInfo(SOperatorInfo** pDownstream, int32_t numOfDownstream, SDynQueryCtrlPhysiNode* pPhyciNode, SExecTaskInfo* pTaskInfo); // clang-format on diff --git a/source/libs/executor/inc/querytask.h b/source/libs/executor/inc/querytask.h index 48de49f07c..e3bb9a1361 100644 --- a/source/libs/executor/inc/querytask.h +++ b/source/libs/executor/inc/querytask.h @@ -98,16 +98,17 @@ struct SExecTaskInfo { SQueryAutoQWorkerPoolCB* pWorkerCb; }; -void buildTaskId(uint64_t taskId, uint64_t queryId, char* dst); -SExecTaskInfo* doCreateTask(uint64_t queryId, uint64_t taskId, int32_t vgId, EOPTR_EXEC_MODEL model, SStorageAPI* pAPI); -void doDestroyTask(SExecTaskInfo* pTaskInfo); -bool isTaskKilled(void* pTaskInfo); -void setTaskKilled(SExecTaskInfo* pTaskInfo, int32_t rspCode); -void setTaskStatus(SExecTaskInfo* pTaskInfo, int8_t status); -int32_t createExecTaskInfo(SSubplan* pPlan, SExecTaskInfo** pTaskInfo, SReadHandle* pHandle, uint64_t taskId, - int32_t vgId, char* sql, EOPTR_EXEC_MODEL model); -int32_t qAppendTaskStopInfo(SExecTaskInfo* pTaskInfo, SExchangeOpStopInfo* pInfo); -SArray* getTableListInfo(const SExecTaskInfo* pTaskInfo); +void buildTaskId(uint64_t taskId, uint64_t queryId, char* dst); +int32_t doCreateTask(uint64_t queryId, uint64_t taskId, int32_t vgId, EOPTR_EXEC_MODEL model, SStorageAPI* pAPI, + SExecTaskInfo** pTaskInfo); +void doDestroyTask(SExecTaskInfo* pTaskInfo); +bool isTaskKilled(void* pTaskInfo); +void setTaskKilled(SExecTaskInfo* pTaskInfo, int32_t rspCode); +void setTaskStatus(SExecTaskInfo* pTaskInfo, int8_t status); +int32_t createExecTaskInfo(SSubplan* pPlan, SExecTaskInfo** pTaskInfo, SReadHandle* pHandle, uint64_t taskId, + int32_t vgId, char* sql, EOPTR_EXEC_MODEL model); +int32_t qAppendTaskStopInfo(SExecTaskInfo* pTaskInfo, SExchangeOpStopInfo* pInfo); +int32_t getTableListInfo(const SExecTaskInfo* pTaskInfo, SArray** pList); #ifdef __cplusplus } diff --git a/source/libs/executor/src/executor.c b/source/libs/executor/src/executor.c index 8cf44602f9..8c98df5c8d 100644 --- a/source/libs/executor/src/executor.c +++ b/source/libs/executor/src/executor.c @@ -288,8 +288,10 @@ int32_t qSetSMAInput(qTaskInfo_t tinfo, const void* pBlocks, size_t numOfBlocks, qTaskInfo_t qCreateQueueExecTaskInfo(void* msg, SReadHandle* pReaderHandle, int32_t vgId, int32_t* numOfCols, uint64_t id) { if (msg == NULL) { // create raw scan - SExecTaskInfo* pTaskInfo = doCreateTask(0, id, vgId, OPTR_EXEC_MODEL_QUEUE, &pReaderHandle->api); - if (NULL == pTaskInfo) { + SExecTaskInfo* pTaskInfo = NULL; + + int32_t code = doCreateTask(0, id, vgId, OPTR_EXEC_MODEL_QUEUE, &pReaderHandle->api, &pTaskInfo); + if (NULL == pTaskInfo || code != 0) { terrno = TSDB_CODE_OUT_OF_MEMORY; return NULL; } @@ -1450,7 +1452,13 @@ SArray* qGetQueriedTableListInfo(qTaskInfo_t tinfo) { int32_t code = TSDB_CODE_SUCCESS; int32_t lino = 0; SExecTaskInfo* pTaskInfo = tinfo; - SArray* plist = getTableListInfo(pTaskInfo); + SArray* plist = NULL; + + code = getTableListInfo(pTaskInfo, &plist); + if (code || plist == NULL) { + code = TSDB_CODE_OUT_OF_MEMORY; + return NULL; + } // only extract table in the first elements STableListInfo* pTableListInfo = taosArrayGetP(plist, 0); @@ -1502,11 +1510,21 @@ _end: } } -SArray* getTableListInfo(const SExecTaskInfo* pTaskInfo) { - SArray* pArray = taosArrayInit(0, POINTER_BYTES); +int32_t getTableListInfo(const SExecTaskInfo* pTaskInfo, SArray** pList) { + if (pList == NULL) { + return TSDB_CODE_INVALID_PARA; + } + + SArray* pArray = taosArrayInit(0, POINTER_BYTES); + if (pArray == NULL) { + return TSDB_CODE_OUT_OF_MEMORY; + } + SOperatorInfo* pOperator = pTaskInfo->pRoot; extractTableList(pArray, pOperator); - return pArray; + + *pList = pArray; + return TSDB_CODE_SUCCESS; } int32_t qStreamOperatorReleaseState(qTaskInfo_t tInfo) { diff --git a/source/libs/executor/src/executorInt.c b/source/libs/executor/src/executorInt.c index a3e3501114..b974384d85 100644 --- a/source/libs/executor/src/executorInt.c +++ b/source/libs/executor/src/executorInt.c @@ -1068,7 +1068,12 @@ int32_t createDataSinkParam(SDataSinkNode* pNode, void** pParam, SExecTaskInfo* return TSDB_CODE_OUT_OF_MEMORY; } - SArray* pInfoList = getTableListInfo(pTask); + SArray* pInfoList = NULL; + int32_t code = getTableListInfo(pTask, &pInfoList); + if (code || pInfoList == NULL) { + return code; + } + STableListInfo* pTableListInfo = taosArrayGetP(pInfoList, 0); taosArrayDestroy(pInfoList); diff --git a/source/libs/executor/src/mergeoperator.c b/source/libs/executor/src/mergeoperator.c index cef1915d2c..384b14ce41 100644 --- a/source/libs/executor/src/mergeoperator.c +++ b/source/libs/executor/src/mergeoperator.c @@ -168,8 +168,10 @@ SSDataBlock* doSortMerge(SOperatorInfo* pOperator) { blockDataCleanup(pDataBlock); if (pSortMergeInfo->pIntermediateBlock == NULL) { - pSortMergeInfo->pIntermediateBlock = tsortGetSortedDataBlock(pHandle); - if (pSortMergeInfo->pIntermediateBlock == NULL) { + pSortMergeInfo->pIntermediateBlock = NULL; + + int32_t code = tsortGetSortedDataBlock(pHandle, &pSortMergeInfo->pIntermediateBlock); + if (pSortMergeInfo->pIntermediateBlock == NULL || code != 0) { terrno = TSDB_CODE_OUT_OF_MEMORY; return NULL; } diff --git a/source/libs/executor/src/querytask.c b/source/libs/executor/src/querytask.c index 0bf4ac2b21..ddba556c7a 100644 --- a/source/libs/executor/src/querytask.c +++ b/source/libs/executor/src/querytask.c @@ -35,38 +35,51 @@ #define CLEAR_QUERY_STATUS(q, st) ((q)->status &= (~(st))) -SExecTaskInfo* doCreateTask(uint64_t queryId, uint64_t taskId, int32_t vgId, EOPTR_EXEC_MODEL model, SStorageAPI* pAPI) { - SExecTaskInfo* pTaskInfo = taosMemoryCalloc(1, sizeof(SExecTaskInfo)); +int32_t doCreateTask(uint64_t queryId, uint64_t taskId, int32_t vgId, EOPTR_EXEC_MODEL model, SStorageAPI* pAPI, + SExecTaskInfo** pTaskInfo) { if (pTaskInfo == NULL) { - terrno = TSDB_CODE_OUT_OF_MEMORY; - return NULL; + return TSDB_CODE_SUCCESS; } - setTaskStatus(pTaskInfo, TASK_NOT_COMPLETED); - pTaskInfo->cost.created = taosGetTimestampUs(); + SExecTaskInfo* p = taosMemoryCalloc(1, sizeof(SExecTaskInfo)); + if (p == NULL) { + return TSDB_CODE_OUT_OF_MEMORY; + } - pTaskInfo->execModel = model; - pTaskInfo->stopInfo.pStopInfo = taosArrayInit(4, sizeof(SExchangeOpStopInfo)); - pTaskInfo->pResultBlockList = taosArrayInit(128, POINTER_BYTES); - pTaskInfo->storageAPI = *pAPI; + setTaskStatus(p, TASK_NOT_COMPLETED); + p->cost.created = taosGetTimestampUs(); - taosInitRWLatch(&pTaskInfo->lock); + p->execModel = model; + p->stopInfo.pStopInfo = taosArrayInit(4, sizeof(SExchangeOpStopInfo)); + p->pResultBlockList = taosArrayInit(128, POINTER_BYTES); + if (p->stopInfo.pStopInfo == NULL || p->pResultBlockList == NULL) { + doDestroyTask(p); + return TSDB_CODE_OUT_OF_MEMORY; + } - pTaskInfo->id.vgId = vgId; - pTaskInfo->id.queryId = queryId; - pTaskInfo->id.taskId = taskId; - pTaskInfo->id.str = taosMemoryMalloc(64); - buildTaskId(taskId, queryId, pTaskInfo->id.str); - pTaskInfo->schemaInfos = taosArrayInit(1, sizeof(SSchemaInfo)); - - return pTaskInfo; + p->storageAPI = *pAPI; + taosInitRWLatch(&p->lock); + + p->id.vgId = vgId; + p->id.queryId = queryId; + p->id.taskId = taskId; + p->id.str = taosMemoryMalloc(64); + buildTaskId(taskId, queryId, p->id.str); + p->schemaInfos = taosArrayInit(1, sizeof(SSchemaInfo)); + if (p->id.str == NULL || p->schemaInfos == NULL) { + doDestroyTask(p); + return TSDB_CODE_OUT_OF_MEMORY; + } + + *pTaskInfo = p; + return TSDB_CODE_SUCCESS; } bool isTaskKilled(void* pTaskInfo) { return (0 != ((SExecTaskInfo*)pTaskInfo)->code); } void setTaskKilled(SExecTaskInfo* pTaskInfo, int32_t rspCode) { pTaskInfo->code = rspCode; - stopTableScanOperator(pTaskInfo->pRoot, pTaskInfo->id.str, &pTaskInfo->storageAPI); + (void) stopTableScanOperator(pTaskInfo->pRoot, pTaskInfo->id.str, &pTaskInfo->storageAPI); } void setTaskStatus(SExecTaskInfo* pTaskInfo, int8_t status) { @@ -81,10 +94,10 @@ void setTaskStatus(SExecTaskInfo* pTaskInfo, int8_t status) { int32_t createExecTaskInfo(SSubplan* pPlan, SExecTaskInfo** pTaskInfo, SReadHandle* pHandle, uint64_t taskId, int32_t vgId, char* sql, EOPTR_EXEC_MODEL model) { - *pTaskInfo = doCreateTask(pPlan->id.queryId, taskId, vgId, model, &pHandle->api); - if (*pTaskInfo == NULL) { + int32_t code = doCreateTask(pPlan->id.queryId, taskId, vgId, model, &pHandle->api, pTaskInfo); + if (*pTaskInfo == NULL || code != 0) { taosMemoryFree(sql); - return terrno; + return code; } if (pHandle) { @@ -165,12 +178,10 @@ int32_t initQueriedTableSchemaInfo(SReadHandle* pHandle, SScanPhysiNode* pScanNo } pAPI->metaReaderFn.clearReader(&mr); - schemaInfo.qsw = extractQueriedColumnSchema(pScanNode); - taosArrayPush(pTaskInfo->schemaInfos, &schemaInfo); - - return TSDB_CODE_SUCCESS; + void* p = taosArrayPush(pTaskInfo->schemaInfos, &schemaInfo); + return (p != NULL)? TSDB_CODE_SUCCESS:TSDB_CODE_OUT_OF_MEMORY; } SSchemaWrapper* extractQueriedColumnSchema(SScanPhysiNode* pScanNode) { diff --git a/source/libs/executor/src/sortoperator.c b/source/libs/executor/src/sortoperator.c index 67b993d702..9892114735 100644 --- a/source/libs/executor/src/sortoperator.c +++ b/source/libs/executor/src/sortoperator.c @@ -166,7 +166,11 @@ void appendOneRowToDataBlock(SSDataBlock* pBlock, STupleHandle* pTupleHandle) { } pBlock->info.dataLoad = 1; - pBlock->info.scanFlag = ((SDataBlockInfo*)tsortGetBlockInfo(pTupleHandle))->scanFlag; + + SDataBlockInfo info = {0}; + tsortGetBlockInfo(pTupleHandle, &info); + + pBlock->info.scanFlag = info.scanFlag; pBlock->info.rows += 1; } @@ -224,9 +228,9 @@ SSDataBlock* getSortedBlockData(SSortHandle* pHandle, SSDataBlock* pDataBlock, i SSortOperatorInfo* pInfo) { blockDataCleanup(pDataBlock); - int32_t code = 0; - SSDataBlock* p = tsortGetSortedDataBlock(pHandle); - if (p == NULL) { + SSDataBlock* p = NULL; + int32_t code = tsortGetSortedDataBlock(pHandle, &p); + if (p == NULL || (code != 0)) { return NULL; } @@ -443,8 +447,9 @@ SSDataBlock* getGroupSortedBlockData(SSortHandle* pHandle, SSDataBlock* pDataBlo blockDataCleanup(pDataBlock); blockDataEnsureCapacity(pDataBlock, capacity); - SSDataBlock* p = tsortGetSortedDataBlock(pHandle); - if (p == NULL) { + SSDataBlock* p = NULL; + int32_t code = tsortGetSortedDataBlock(pHandle, &p); + if (p == NULL || (code != 0)) { return NULL; } @@ -452,7 +457,7 @@ SSDataBlock* getGroupSortedBlockData(SSortHandle* pHandle, SSDataBlock* pDataBlo while (1) { STupleHandle* pTupleHandle = NULL; - int32_t code = tsortNextTuple(pHandle, &pTupleHandle); + code = tsortNextTuple(pHandle, &pTupleHandle); if (pTupleHandle == NULL || code != 0) { break; } diff --git a/source/libs/stream/src/streamCheckStatus.c b/source/libs/stream/src/streamCheckStatus.c index 66c1941abb..625305da03 100644 --- a/source/libs/stream/src/streamCheckStatus.c +++ b/source/libs/stream/src/streamCheckStatus.c @@ -418,13 +418,14 @@ void findCheckRspStatus(STaskCheckInfo* pInfo, int32_t taskId, SDownstreamStatus int32_t streamTaskUpdateCheckInfo(STaskCheckInfo* pInfo, int32_t taskId, int32_t status, int64_t rspTs, int64_t reqId, int32_t* pNotReady, const char* id) { - streamMutexLock(&pInfo->checkInfoLock); - SDownstreamStatusInfo* p = NULL; + + streamMutexLock(&pInfo->checkInfoLock); findCheckRspStatus(pInfo, taskId, &p); if (p != NULL) { if (reqId != p->reqId) { - stError("s-task:%s reqId:0x%" PRIx64 " expected:0x%" PRIx64 " expired check-rsp recv from downstream task:0x%x, discarded", + stError("s-task:%s reqId:0x%" PRIx64 " expected:0x%" PRIx64 + " expired check-rsp recv from downstream task:0x%x, discarded", id, reqId, p->reqId, taskId); streamMutexUnlock(&pInfo->checkInfoLock); return TSDB_CODE_FAILED; diff --git a/source/libs/stream/src/streamCheckpoint.c b/source/libs/stream/src/streamCheckpoint.c index 60019977cc..4dc7b14cc3 100644 --- a/source/libs/stream/src/streamCheckpoint.c +++ b/source/libs/stream/src/streamCheckpoint.c @@ -176,6 +176,10 @@ int32_t continueDispatchCheckpointTriggerBlock(SStreamDataBlock* pBlock, SStream int32_t streamProcessCheckpointTriggerBlock(SStreamTask* pTask, SStreamDataBlock* pBlock) { SSDataBlock* pDataBlock = taosArrayGet(pBlock->blocks, 0); + if (pDataBlock == NULL) { + return TSDB_CODE_INVALID_PARA; + } + int64_t checkpointId = pDataBlock->info.version; int32_t transId = pDataBlock->info.window.skey; const char* id = pTask->id.idStr; @@ -248,6 +252,10 @@ int32_t streamProcessCheckpointTriggerBlock(SStreamTask* pTask, SStreamDataBlock // check if already recv or not, and duplicated checkpoint-trigger msg recv, discard it for (int32_t i = 0; i < taosArrayGetSize(pActiveInfo->pReadyMsgList); ++i) { STaskCheckpointReadyInfo* p = taosArrayGet(pActiveInfo->pReadyMsgList, i); + if (p == NULL) { + return TSDB_CODE_INVALID_PARA; + } + if (p->upstreamTaskId == pBlock->srcTaskId) { ASSERT(p->checkpointId == checkpointId); stWarn("s-task:%s repeatly recv checkpoint-source msg from task:0x%x vgId:%d, checkpointId:%" PRId64 @@ -381,6 +389,10 @@ int32_t streamProcessCheckpointReadyMsg(SStreamTask* pTask, int64_t checkpointId int32_t size = taosArrayGetSize(pInfo->pCheckpointReadyRecvList); for (int32_t i = 0; i < size; ++i) { STaskDownstreamReadyInfo* p = taosArrayGet(pInfo->pCheckpointReadyRecvList, i); + if (p == NULL) { + return TSDB_CODE_INVALID_PARA; + } + if (p->downstreamTaskId == downstreamTaskId) { received = true; break; @@ -420,6 +432,10 @@ int32_t streamTaskProcessCheckpointReadyRsp(SStreamTask* pTask, int32_t upstream streamMutexLock(&pInfo->lock); for (int32_t i = 0; i < taosArrayGetSize(pInfo->pReadyMsgList); ++i) { STaskCheckpointReadyInfo* pReadyInfo = taosArrayGet(pInfo->pReadyMsgList, i); + if (pReadyInfo == NULL) { + return TSDB_CODE_INVALID_PARA; + } + if (pReadyInfo->upstreamTaskId == upstreamTaskId && pReadyInfo->checkpointId == checkpointId) { pReadyInfo->sendCompleted = 1; stDebug("s-task:%s send checkpoint-ready msg to upstream:0x%x confirmed, checkpointId:%" PRId64 " ts:%" PRId64, @@ -430,6 +446,10 @@ int32_t streamTaskProcessCheckpointReadyRsp(SStreamTask* pTask, int32_t upstream for (int32_t i = 0; i < taosArrayGetSize(pInfo->pReadyMsgList); ++i) { STaskCheckpointReadyInfo* pReadyInfo = taosArrayGet(pInfo->pReadyMsgList, i); + if (pReadyInfo == NULL) { + return TSDB_CODE_INVALID_PARA; + } + if (pReadyInfo->sendCompleted == 1) { numOfConfirmed += 1; } @@ -819,6 +839,10 @@ void checkpointTriggerMonitorFn(void* param, void* tmrId) { bool recved = false; for (int32_t j = 0; j < taosArrayGetSize(pActiveInfo->pReadyMsgList); ++j) { STaskCheckpointReadyInfo* pReady = taosArrayGet(pActiveInfo->pReadyMsgList, j); + if (pReady == NULL) { + continue; + } + if (pInfo->nodeId == pReady->upstreamNodeId) { recved = true; break; @@ -867,6 +891,9 @@ int32_t doSendRetrieveTriggerMsg(SStreamTask* pTask, SArray* pNotSendList) { for (int32_t i = 0; i < size; i++) { SStreamUpstreamEpInfo* pUpstreamTask = taosArrayGet(pNotSendList, i); + if (pUpstreamTask == NULL) { + return TSDB_CODE_INVALID_PARA; + } SRetrieveChkptTriggerReq* pReq = rpcMallocCont(sizeof(SRetrieveChkptTriggerReq)); if (pReq == NULL) { @@ -917,6 +944,10 @@ bool streamTaskAlreadySendTrigger(SStreamTask* pTask, int32_t downstreamNodeId) for (int32_t i = 0; i < taosArrayGetSize(pInfo->pDispatchTriggerList); ++i) { STaskTriggerSendInfo* pSendInfo = taosArrayGet(pInfo->pDispatchTriggerList, i); + if (pSendInfo == NULL) { + return TSDB_CODE_INVALID_PARA; + } + if (pSendInfo->nodeId != downstreamNodeId) { continue; } @@ -974,6 +1005,9 @@ void streamTaskInitTriggerDispatchInfo(SStreamTask* pTask) { } else { for (int32_t i = 0; i < streamTaskGetNumOfDownstream(pTask); ++i) { SVgroupInfo* pVgInfo = taosArrayGet(pTask->outputInfo.shuffleDispatcher.dbInfo.pVgroupInfos, i); + if (pVgInfo == NULL) { + continue; + } STaskTriggerSendInfo p = {.sendTs = now, .recved = false, .nodeId = pVgInfo->vgId, .taskId = pVgInfo->taskId}; void* px = taosArrayPush(pInfo->pDispatchTriggerList, &p); @@ -993,6 +1027,10 @@ int32_t streamTaskGetNumOfConfirmed(SStreamTask* pTask) { streamMutexLock(&pInfo->lock); for (int32_t i = 0; i < taosArrayGetSize(pInfo->pDispatchTriggerList); ++i) { STaskTriggerSendInfo* p = taosArrayGet(pInfo->pDispatchTriggerList, i); + if (p == NULL) { + return num; + } + if (p->recved) { num++; } @@ -1009,6 +1047,10 @@ void streamTaskSetTriggerDispatchConfirmed(SStreamTask* pTask, int32_t vgId) { for (int32_t i = 0; i < taosArrayGetSize(pInfo->pDispatchTriggerList); ++i) { STaskTriggerSendInfo* p = taosArrayGet(pInfo->pDispatchTriggerList, i); + if (p == NULL) { + continue; + } + if (p->nodeId == vgId) { ASSERT(p->recved == false); From 4d8064c950251925a2998b51729f6dc88e891ba9 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 23 Jul 2024 19:16:06 +0800 Subject: [PATCH 04/14] fix(tsdb):fix invalid return code. --- source/dnode/mnode/impl/src/mndStream.c | 15 +++++++------ source/dnode/vnode/src/tsdb/tsdbCacheRead.c | 22 +++++++++++++++++--- source/util/src/tcompression.c | 4 ++-- tests/script/tsim/parser/last_both_query.sim | 1 + 4 files changed, 29 insertions(+), 13 deletions(-) diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index 31d28d99a4..b3e81665e3 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -255,14 +255,13 @@ static int32_t mndStreamActionUpdate(SSdb *pSdb, SStreamObj *pOldStream, SStream } int32_t mndAcquireStream(SMnode *pMnode, char *streamName, SStreamObj **pStream) { - terrno = 0; - + int32_t code = 0; SSdb *pSdb = pMnode->pSdb; (*pStream) = sdbAcquire(pSdb, SDB_STREAM, streamName); if ((*pStream) == NULL && terrno == TSDB_CODE_SDB_OBJ_NOT_THERE) { - terrno = TSDB_CODE_MND_STREAM_NOT_EXIST; + code = TSDB_CODE_MND_STREAM_NOT_EXIST; } - return terrno; + return code; } void mndReleaseStream(SMnode *pMnode, SStreamObj *pStream) { @@ -754,19 +753,19 @@ static int32_t mndProcessCreateStreamReq(SRpcMsg *pReq) { } code = mndAcquireStream(pMnode, createReq.name, &pStream); - if (pStream != NULL || code == 0) { + if (pStream != NULL && code == 0) { if (createReq.igExists) { mInfo("stream:%s, already exist, ignore exist is set", createReq.name); goto _OVER; } else { - terrno = TSDB_CODE_MND_STREAM_ALREADY_EXIST; + code = TSDB_CODE_MND_STREAM_ALREADY_EXIST; goto _OVER; } - } else if (terrno != TSDB_CODE_MND_STREAM_NOT_EXIST) { + } else if (code != TSDB_CODE_MND_STREAM_NOT_EXIST) { goto _OVER; } - if ((terrno = grantCheck(TSDB_GRANT_STREAMS)) < 0) { + if ((code = grantCheck(TSDB_GRANT_STREAMS)) < 0) { goto _OVER; } diff --git a/source/dnode/vnode/src/tsdb/tsdbCacheRead.c b/source/dnode/vnode/src/tsdb/tsdbCacheRead.c index 97da98e9c6..38a2dd3ab2 100644 --- a/source/dnode/vnode/src/tsdb/tsdbCacheRead.c +++ b/source/dnode/vnode/src/tsdb/tsdbCacheRead.c @@ -89,8 +89,8 @@ static int32_t saveOneRow(SArray* pRow, SSDataBlock* pBlock, SCacheRowsReader* p if (pVal == NULL) { return TSDB_CODE_INVALID_PARA; } - funcType = *(int32_t*) pVal; + funcType = *(int32_t*) pVal; pVal = taosArrayGet(pReader->pFuncTypeList, i); if (pVal == NULL) { return TSDB_CODE_INVALID_PARA; @@ -471,6 +471,11 @@ int32_t tsdbRetrieveCacheRows(void* pReader, SSDataBlock* pResBlock, const int32 int32_t bytes = (slotIds[j] == -1) ? 1 : pr->pSchema->columns[slotIds[j]].bytes; pRes[j] = taosMemoryCalloc(1, sizeof(SFirstLastRes) + bytes + pkBufLen + VARSTR_HEADER_SIZE); + if (pRes[j] == NULL) { + code = TSDB_CODE_OUT_OF_MEMORY; + goto _end; + } + SFirstLastRes* p = (SFirstLastRes*)varDataVal(pRes[j]); p->ts = INT64_MIN; } @@ -512,7 +517,12 @@ int32_t tsdbRetrieveCacheRows(void* pReader, SSDataBlock* pResBlock, const int32 for (int32_t j = 0; j < pr->rowKey.numOfPKs; j++) { p.rowKey.pks[j].type = pr->pkColumn.type; if (IS_VAR_DATA_TYPE(pr->pkColumn.type)) { + p.rowKey.pks[j].pData = taosMemoryCalloc(1, pr->pkColumn.bytes); + if (p.rowKey.pks[j].pData == NULL) { + code = TSDB_CODE_OUT_OF_MEMORY; + goto _end; + } } } } @@ -538,7 +548,9 @@ int32_t tsdbRetrieveCacheRows(void* pReader, SSDataBlock* pResBlock, const int32 tb_uid_t uid = pTableList[i].uid; code = tsdbCacheGetBatch(pr->pTsdb, uid, pRow, pr, ltype); - if (code) { + if (code == -1) {// fix the invalid return code + code = 0; + } else if (code != 0) { goto _end; } @@ -642,7 +654,11 @@ int32_t tsdbRetrieveCacheRows(void* pReader, SSDataBlock* pResBlock, const int32 tb_uid_t uid = pTableList[i].uid; if ((code = tsdbCacheGetBatch(pr->pTsdb, uid, pRow, pr, ltype)) != 0) { - goto _end; + if (code == -1) {// fix the invalid return code + code = 0; + } else if (code != 0) { + goto _end; + } } if (TARRAY_SIZE(pRow) <= 0 || COL_VAL_IS_NONE(&((SLastCol*)TARRAY_DATA(pRow))[0].colVal)) { diff --git a/source/util/src/tcompression.c b/source/util/src/tcompression.c index 884d7ea1b6..d8656c0f60 100644 --- a/source/util/src/tcompression.c +++ b/source/util/src/tcompression.c @@ -252,7 +252,7 @@ int32_t l2ComressInitImpl_xz(char *lossyColumns, float fPrecision, double dPreci } int32_t l2CompressImpl_xz(const char *const input, const int32_t inputSize, char *const output, int32_t outputSize, const char type, int8_t lvl) { - size_t len = FL2_compress(output + 1, outputSize - 1, input, inputSize, lvl); + size_t len = 0;//FL2_compress(output + 1, outputSize - 1, input, inputSize, lvl); if (len > inputSize) { output[0] = 0; memcpy(output + 1, input, inputSize); @@ -264,7 +264,7 @@ int32_t l2CompressImpl_xz(const char *const input, const int32_t inputSize, char int32_t l2DecompressImpl_xz(const char *const input, const int32_t compressedSize, char *const output, int32_t outputSize, const char type) { if (input[0] == 1) { - return FL2_decompress(output, outputSize, input + 1, compressedSize - 1); + return 0;//FL2_decompress(output, outputSize, input + 1, compressedSize - 1); } else if (input[0] == 0) { memcpy(output, input + 1, compressedSize - 1); return compressedSize - 1; diff --git a/tests/script/tsim/parser/last_both_query.sim b/tests/script/tsim/parser/last_both_query.sim index 5f86412199..b082351e0b 100644 --- a/tests/script/tsim/parser/last_both_query.sim +++ b/tests/script/tsim/parser/last_both_query.sim @@ -16,6 +16,7 @@ endi sql select last(f1) from tb1 if $rows != 1 then + print expect 1, actual $rows return -1 endi if $data00 != 6 then From d8700998dc8d6ab2dc5ef17b49843290d0522d9e Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 23 Jul 2024 19:17:00 +0800 Subject: [PATCH 05/14] fix(util): fix error --- source/util/src/tcompression.c | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/source/util/src/tcompression.c b/source/util/src/tcompression.c index d8656c0f60..884d7ea1b6 100644 --- a/source/util/src/tcompression.c +++ b/source/util/src/tcompression.c @@ -252,7 +252,7 @@ int32_t l2ComressInitImpl_xz(char *lossyColumns, float fPrecision, double dPreci } int32_t l2CompressImpl_xz(const char *const input, const int32_t inputSize, char *const output, int32_t outputSize, const char type, int8_t lvl) { - size_t len = 0;//FL2_compress(output + 1, outputSize - 1, input, inputSize, lvl); + size_t len = FL2_compress(output + 1, outputSize - 1, input, inputSize, lvl); if (len > inputSize) { output[0] = 0; memcpy(output + 1, input, inputSize); @@ -264,7 +264,7 @@ int32_t l2CompressImpl_xz(const char *const input, const int32_t inputSize, char int32_t l2DecompressImpl_xz(const char *const input, const int32_t compressedSize, char *const output, int32_t outputSize, const char type) { if (input[0] == 1) { - return 0;//FL2_decompress(output, outputSize, input + 1, compressedSize - 1); + return FL2_decompress(output, outputSize, input + 1, compressedSize - 1); } else if (input[0] == 0) { memcpy(output, input + 1, compressedSize - 1); return compressedSize - 1; From d8993c16eeab721cc10b2ca845f1d1d3ead43ad9 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 23 Jul 2024 19:32:05 +0800 Subject: [PATCH 06/14] fix(stream): update test case. --- tests/system-test/1-insert/drop.py | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/tests/system-test/1-insert/drop.py b/tests/system-test/1-insert/drop.py index 493e1491b8..b467276fce 100644 --- a/tests/system-test/1-insert/drop.py +++ b/tests/system-test/1-insert/drop.py @@ -146,11 +146,15 @@ class TDTestCase: tdSql.execute(f'create table {stbname} (ts timestamp,c0 int) tags(t0 int)') tdSql.execute(f'create table tb using {stbname} tags(1)') tdSql.execute(f'create stream {stream_name} trigger at_once ignore expired 0 into stb as select * from {self.dbname}.{stbname} partition by tbname') + time.sleep(5) + tdSql.query(f'select * from information_schema.ins_streams where stream_name = "{stream_name}"') print(tdSql.queryResult) tdSql.checkEqual(tdSql.queryResult[0][4],f'create stream {stream_name} trigger at_once ignore expired 0 into stb as select * from {self.dbname}.{stbname} partition by tbname') tdSql.execute(f'drop stream {stream_name}') tdSql.execute(f'create stream {stream_name} trigger at_once ignore expired 0 into stb1 as select * from tb') + time.sleep(5) + tdSql.query(f'select * from information_schema.ins_streams where stream_name = "{stream_name}"') tdSql.checkEqual(tdSql.queryResult[0][4],f'create stream {stream_name} trigger at_once ignore expired 0 into stb1 as select * from tb') tdSql.execute(f'drop database {self.dbname}') From 923c7912014cf70fa7036c65a2ca93dc6e73d1a7 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Tue, 23 Jul 2024 19:53:32 +0800 Subject: [PATCH 07/14] other: update test cases. --- source/dnode/vnode/src/sma/smaRollup.c | 2 +- tests/system-test/0-others/test_hot_refresh_configurations.py | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/source/dnode/vnode/src/sma/smaRollup.c b/source/dnode/vnode/src/sma/smaRollup.c index 823f65a9fd..515272ddeb 100644 --- a/source/dnode/vnode/src/sma/smaRollup.c +++ b/source/dnode/vnode/src/sma/smaRollup.c @@ -1442,7 +1442,7 @@ static void tdRSmaFetchTrigger(void *param, void *tmrId) { SMA_VID(pSma), pItem->level, pRSmaInfo->suid); } break; case TASK_TRIGGER_STAT_INIT: { - smaDebug("vgId:%d, rsma fetch task not start for level:%" PRIi8 " suid::%" PRIi64 " since stat is init", + smaDebug("vgId:%d, rsma fetch task not start for level:%" PRIi8 " suid:%" PRIi64 " since stat is init", SMA_VID(pSma), pItem->level, pRSmaInfo->suid); } break; default: { diff --git a/tests/system-test/0-others/test_hot_refresh_configurations.py b/tests/system-test/0-others/test_hot_refresh_configurations.py index 71f6290469..15f291787a 100644 --- a/tests/system-test/0-others/test_hot_refresh_configurations.py +++ b/tests/system-test/0-others/test_hot_refresh_configurations.py @@ -123,8 +123,8 @@ class TDTestCase: { "name": "checkpointInterval", "alias": "tsStreamCheckpointInterval", - "values": [60, 1000, 1200], - "except_values": [59, 1201] + "values": [60, 1000, 1800], + "except_values": [59, 1801] }, { "name": "trimVDbIntervalSec", From cd000ba9817f65938c5960080231e62520a72ab6 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 24 Jul 2024 00:08:39 +0800 Subject: [PATCH 08/14] fix(stream): set correct check return value. --- source/dnode/vnode/src/tsdb/tsdbRead2.c | 32 +++++++------------------ source/libs/executor/src/executor.c | 6 ++--- 2 files changed, 12 insertions(+), 26 deletions(-) diff --git a/source/dnode/vnode/src/tsdb/tsdbRead2.c b/source/dnode/vnode/src/tsdb/tsdbRead2.c index 95ae9bcdf0..4aaa6c696c 100644 --- a/source/dnode/vnode/src/tsdb/tsdbRead2.c +++ b/source/dnode/vnode/src/tsdb/tsdbRead2.c @@ -547,7 +547,7 @@ static int32_t tsdbReaderCreate(SVnode* pVnode, SQueryTableDataCond* pCond, void pReader->info.window = updateQueryTimeWindow(pReader->pTsdb, &pCond->twindows); pReader->idStr = (idstr != NULL) ? taosStrdup(idstr) : NULL; - if (pReader->idStr == NULL) { + if (idstr != NULL && pReader->idStr == NULL) { code = TSDB_CODE_OUT_OF_MEMORY; goto _end; } @@ -4618,19 +4618,16 @@ int32_t tsdbReaderOpen2(void* pVnode, SQueryTableDataCond* pCond, void* pTableLi STimeWindow window = pCond->twindows; SVnodeCfg* pConf = &(((SVnode*)pVnode)->config); int32_t code = 0; + int32_t lino = 0; int32_t capacity = pConf->tsdbCfg.maxRows; if (pResBlock != NULL) { code = blockDataEnsureCapacity(pResBlock, capacity); - if (code != TSDB_CODE_SUCCESS) { - goto _err; - } + TSDB_CHECK_CODE(code, lino, _err); } code = tsdbReaderCreate(pVnode, pCond, ppReader, capacity, pResBlock, idstr); - if (code != TSDB_CODE_SUCCESS) { - goto _err; - } + TSDB_CHECK_CODE(code, lino, _err); // check for query time window STsdbReader* pReader = *ppReader; @@ -4654,9 +4651,7 @@ int32_t tsdbReaderOpen2(void* pVnode, SQueryTableDataCond* pCond, void* pTableLi // here we only need one more row, so the capacity is set to be ONE. code = tsdbReaderCreate(pVnode, pCond, (void**)&((STsdbReader*)pReader)->innerReader[0], 1, pResBlock, idstr); - if (code != TSDB_CODE_SUCCESS) { - goto _err; - } + TSDB_CHECK_CODE(code, lino, _err); if (order == TSDB_ORDER_ASC) { pCond->twindows.skey = window.ekey + 1; @@ -4668,9 +4663,7 @@ int32_t tsdbReaderOpen2(void* pVnode, SQueryTableDataCond* pCond, void* pTableLi pCond->order = order; code = tsdbReaderCreate(pVnode, pCond, (void**)&((STsdbReader*)pReader)->innerReader[1], 1, pResBlock, idstr); - if (code != TSDB_CODE_SUCCESS) { - goto _err; - } + TSDB_CHECK_CODE(code, lino, _err); } // NOTE: the endVersion in pCond is the data version not schema version, so pCond->endVersion is not correct here. @@ -4691,9 +4684,7 @@ int32_t tsdbReaderOpen2(void* pVnode, SQueryTableDataCond* pCond, void* pTableLi if (pReader->info.pSchema != NULL) { code = tsdbRowMergerInit(&pReader->status.merger, pReader->info.pSchema); - if (code != TSDB_CODE_SUCCESS) { - return code; - } + TSDB_CHECK_CODE(code, lino, _err); } pReader->pSchemaMap = tSimpleHashInit(8, taosFastHash); @@ -4706,18 +4697,13 @@ int32_t tsdbReaderOpen2(void* pVnode, SQueryTableDataCond* pCond, void* pTableLi tSimpleHashSetFreeFp(pReader->pSchemaMap, freeSchemaFunc); if (pReader->info.pSchema != NULL) { code = updateBlockSMAInfo(pReader->info.pSchema, &pReader->suppInfo); - if (code != TSDB_CODE_SUCCESS) { - goto _err; - } + TSDB_CHECK_CODE(code, lino, _err); } STsdbReader* p = (pReader->innerReader[0] != NULL) ? pReader->innerReader[0] : pReader; code = createDataBlockScanInfo(p, &pReader->blockInfoBuf, pTableList, &pReader->status.uidList, numOfTables, &pReader->status.pTableMap); - if (code != TSDB_CODE_SUCCESS) { - *ppReader = NULL; - goto _err; - } + TSDB_CHECK_CODE(code, lino, _err); pReader->status.pLDataIterArray = taosArrayInit(4, POINTER_BYTES); if (pReader->status.pLDataIterArray == NULL) { diff --git a/source/libs/executor/src/executor.c b/source/libs/executor/src/executor.c index 8c98df5c8d..cf539ba8f1 100644 --- a/source/libs/executor/src/executor.c +++ b/source/libs/executor/src/executor.c @@ -1306,9 +1306,9 @@ int32_t qStreamPrepareScan(qTaskInfo_t tinfo, STqOffsetVal* pOffset, int8_t subT pScanInfo->scanTimes = 0; if (pScanBaseInfo->dataReader == NULL) { - int32_t code = pTaskInfo->storageAPI.tsdReader.tsdReaderOpen( - pScanBaseInfo->readHandle.vnode, &pScanBaseInfo->cond, &keyInfo, 1, pScanInfo->pResBlock, - (void**)&pScanBaseInfo->dataReader, id, NULL); + code = pTaskInfo->storageAPI.tsdReader.tsdReaderOpen(pScanBaseInfo->readHandle.vnode, &pScanBaseInfo->cond, + &keyInfo, 1, pScanInfo->pResBlock, + (void**)&pScanBaseInfo->dataReader, id, NULL); if (code != TSDB_CODE_SUCCESS) { qError("prepare read tsdb snapshot failed, uid:%" PRId64 ", code:%s %s", pOffset->uid, tstrerror(code), id); terrno = code; From 9bfb0603d8342873b4f8bedfa30ccccb96bf1c0b Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 24 Jul 2024 00:27:22 +0800 Subject: [PATCH 09/14] test: update the test cases. --- source/libs/stream/src/streamMeta.c | 8 +++++--- tests/system-test/0-others/backquote_check.py | 4 ++++ 2 files changed, 9 insertions(+), 3 deletions(-) diff --git a/source/libs/stream/src/streamMeta.c b/source/libs/stream/src/streamMeta.c index 43a22e0637..7b410501ca 100644 --- a/source/libs/stream/src/streamMeta.c +++ b/source/libs/stream/src/streamMeta.c @@ -539,9 +539,11 @@ void streamMetaClear(SStreamMeta* pMeta) { streamMetaReleaseTask(pMeta, p); } - int32_t code = taosRemoveRef(streamBackendId, pMeta->streamBackendRid); - if (code) { - stError("vgId:%d remove stream backend Ref failed, rid:%"PRId64, pMeta->vgId, pMeta->streamBackendRid); + if (pMeta->streamBackendRid != 0) { + int32_t code = taosRemoveRef(streamBackendId, pMeta->streamBackendRid); + if (code) { + stError("vgId:%d remove stream backend Ref failed, rid:%" PRId64, pMeta->vgId, pMeta->streamBackendRid); + } } taosHashClear(pMeta->pTasksMap); diff --git a/tests/system-test/0-others/backquote_check.py b/tests/system-test/0-others/backquote_check.py index b35701f1df..2858373ced 100644 --- a/tests/system-test/0-others/backquote_check.py +++ b/tests/system-test/0-others/backquote_check.py @@ -67,11 +67,15 @@ class TDTestCase: tdSql.execute(f'use {self.dbname}') tdSql.execute(f'create stable {self.stbname} (ts timestamp,c0 int) tags(t0 int)') tdSql.execute(f'create stream `{self.streamname}` into `{self.streamtb}` as select count(*) from {self.stbname} interval(10s);') + + time.sleep(15) tdSql.query('show streams') tdSql.checkEqual(tdSql.queryResult[0][0],self.streamname) tdSql.execute(f'drop stream {self.streamname}') tdSql.execute(f'drop stable {self.streamtb}') tdSql.execute(f'create stream {self.streamname} into `{self.streamtb}` as select count(*) from {self.stbname} interval(10s);') + + time.sleep(10) tdSql.query('show streams') tdSql.checkEqual(tdSql.queryResult[0][0],self.streamname) tdSql.execute(f'drop stream `{self.streamname}`') From d34ed134b3502d1588a236df7607999565842d1c Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 24 Jul 2024 00:44:52 +0800 Subject: [PATCH 10/14] fix(tsdb): set the correct tsdbReader --- source/dnode/vnode/src/tsdb/tsdbRead2.c | 3 +++ 1 file changed, 3 insertions(+) diff --git a/source/dnode/vnode/src/tsdb/tsdbRead2.c b/source/dnode/vnode/src/tsdb/tsdbRead2.c index 4aaa6c696c..a0063cbf29 100644 --- a/source/dnode/vnode/src/tsdb/tsdbRead2.c +++ b/source/dnode/vnode/src/tsdb/tsdbRead2.c @@ -3669,14 +3669,17 @@ static void getTsdbByRetentions(SVnode* pVnode, SQueryTableDataCond* pCond, SRet *pLevel = TSDB_RETENTION_L0; tsdbDebug("vgId:%d, rsma level %d is selected to query %s", TD_VID(pVnode), TSDB_RETENTION_L0, str); *pTsdb = VND_RSMA0(pVnode); + return; } else if (level == TSDB_RETENTION_L1) { *pLevel = TSDB_RETENTION_L1; tsdbDebug("vgId:%d, rsma level %d is selected to query %s", TD_VID(pVnode), TSDB_RETENTION_L1, str); *pTsdb = VND_RSMA1(pVnode); + return; } else { *pLevel = TSDB_RETENTION_L2; tsdbDebug("vgId:%d, rsma level %d is selected to query %s", TD_VID(pVnode), TSDB_RETENTION_L2, str); *pTsdb = VND_RSMA2(pVnode); + return; } } From ee03a7eea79caf7c277935715c92af778038e137 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 24 Jul 2024 13:29:15 +0800 Subject: [PATCH 11/14] fix(stream): add some logs. --- source/dnode/mnode/impl/src/mndStreamUtil.c | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/source/dnode/mnode/impl/src/mndStreamUtil.c b/source/dnode/mnode/impl/src/mndStreamUtil.c index e250034184..5a17d659cd 100644 --- a/source/dnode/mnode/impl/src/mndStreamUtil.c +++ b/source/dnode/mnode/impl/src/mndStreamUtil.c @@ -744,8 +744,9 @@ int32_t mndInitExecInfo() { void removeExpiredNodeInfo(const SArray *pNodeSnapshot) { SArray *pValidList = taosArrayInit(4, sizeof(SNodeEntry)); int32_t size = taosArrayGetSize(pNodeSnapshot); + int32_t oldSize = taosArrayGetSize(execInfo.pNodeList); - for (int32_t i = 0; i < taosArrayGetSize(execInfo.pNodeList); ++i) { + for (int32_t i = 0; i < oldSize; ++i) { SNodeEntry *p = taosArrayGet(execInfo.pNodeList, i); for (int32_t j = 0; j < size; ++j) { @@ -763,7 +764,8 @@ void removeExpiredNodeInfo(const SArray *pNodeSnapshot) { taosArrayDestroy(execInfo.pNodeList); execInfo.pNodeList = pValidList; - mDebug("remain %d valid node entries after clean expired nodes info", (int32_t)taosArrayGetSize(pValidList)); + mDebug("remain %d valid node entries after clean expired nodes info, prev size:%d", + (int32_t)taosArrayGetSize(pValidList), oldSize); } int32_t doRemoveTasks(SStreamExecInfo *pExecNode, STaskId *pRemovedId) { From 12371207300dd8d82a6dbc9c2637ee2b5839e1a0 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 24 Jul 2024 14:31:16 +0800 Subject: [PATCH 12/14] fix(stream): ignore error. --- source/dnode/mnode/impl/src/mndStream.c | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index b3e81665e3..2c3f2c59c7 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -2474,10 +2474,7 @@ static int32_t extractNodeListFromStream(SMnode *pMnode, SArray *pNodeList) { SNodeEntry entry = {.hbTimestamp = -1, .nodeId = pTask->info.nodeId}; epsetAssign(&entry.epset, &pTask->info.epSet); - code = taosHashPut(pHash, &entry.nodeId, sizeof(entry.nodeId), &entry, sizeof(entry)); - if (code) { - break; - } + (void)taosHashPut(pHash, &entry.nodeId, sizeof(entry.nodeId), &entry, sizeof(entry)); } destroyStreamTaskIter(pTaskIter); From 3cb462cdf987db7d1f993ec257148d6efda5595f Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 24 Jul 2024 14:34:45 +0800 Subject: [PATCH 13/14] refactor: do some internal refactor. --- source/dnode/mnode/impl/src/mndStream.c | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index 2c3f2c59c7..d864326ed0 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -56,7 +56,7 @@ static int32_t mndBuildStreamCheckpointSourceReq(void **pBuf, int32_t *pLen, int int64_t streamId, int32_t taskId, int32_t transId, int8_t mndTrigger); static int32_t mndProcessNodeCheck(SRpcMsg *pReq); static int32_t mndProcessNodeCheckReq(SRpcMsg *pMsg); -static int32_t extractNodeListFromStream(SMnode *pMnode, SArray *pNodeList); +static int32_t refreshNodeListFromExistedStreams(SMnode *pMnode, SArray *pNodeList); static int32_t mndProcessStreamReqCheckpoint(SRpcMsg *pReq); static int32_t mndProcessCheckpointReport(SRpcMsg *pReq); static int32_t mndProcessConsensusInTmr(SRpcMsg *pMsg); @@ -1092,7 +1092,7 @@ _ERR: int32_t extractStreamNodeList(SMnode *pMnode) { if (taosArrayGetSize(execInfo.pNodeList) == 0) { - int32_t code = extractNodeListFromStream(pMnode, execInfo.pNodeList); + int32_t code = refreshNodeListFromExistedStreams(pMnode, execInfo.pNodeList); if (code) { mError("Failed to extract node list from stream, code:%s", tstrerror(code)); return code; @@ -2437,7 +2437,7 @@ static int32_t mndProcessVgroupChange(SMnode *pMnode, SVgroupChangeInfo *pChange return code; } -static int32_t extractNodeListFromStream(SMnode *pMnode, SArray *pNodeList) { +static int32_t refreshNodeListFromExistedStreams(SMnode *pMnode, SArray *pNodeList) { SSdb *pSdb = pMnode->pSdb; SStreamObj *pStream = NULL; void *pIter = NULL; @@ -2565,7 +2565,7 @@ static int32_t mndProcessNodeCheckReq(SRpcMsg *pMsg) { // keep the new vnode snapshot if success if (code == TSDB_CODE_SUCCESS || code == TSDB_CODE_ACTION_IN_PROGRESS) { - code = extractNodeListFromStream(pMnode, execInfo.pNodeList); + code = refreshNodeListFromExistedStreams(pMnode, execInfo.pNodeList); if (code) { mError("failed to extract node list from stream, code:%s", tstrerror(code)); goto _end; From 3e6d3c541680ec679b801e4c92ba6438d5574189 Mon Sep 17 00:00:00 2001 From: Haojun Liao Date: Wed, 24 Jul 2024 14:35:55 +0800 Subject: [PATCH 14/14] refactor: do some internal refactor. --- source/dnode/mnode/impl/src/mndStream.c | 2 ++ 1 file changed, 2 insertions(+) diff --git a/source/dnode/mnode/impl/src/mndStream.c b/source/dnode/mnode/impl/src/mndStream.c index d864326ed0..24f757ebe6 100644 --- a/source/dnode/mnode/impl/src/mndStream.c +++ b/source/dnode/mnode/impl/src/mndStream.c @@ -2443,6 +2443,8 @@ static int32_t refreshNodeListFromExistedStreams(SMnode *pMnode, SArray *pNodeLi void *pIter = NULL; int32_t code = 0; + mDebug("start to refresh node list by existed streams"); + SHashObj *pHash = taosHashInit(64, taosGetDefaultHashFunction(TSDB_DATA_TYPE_INT), false, HASH_NO_LOCK); if (pHash == NULL) { return TSDB_CODE_OUT_OF_MEMORY;