diff --git a/include/common/tmsg.h b/include/common/tmsg.h index e9cde9e99b..7c635d8689 100644 --- a/include/common/tmsg.h +++ b/include/common/tmsg.h @@ -3605,7 +3605,7 @@ static FORCE_INLINE void tqOffsetResetToLog(STqOffsetVal* pOffsetVal, int64_t ve int32_t tEncodeSTqOffsetVal(SEncoder* pEncoder, const STqOffsetVal* pOffsetVal); int32_t tDecodeSTqOffsetVal(SDecoder* pDecoder, STqOffsetVal* pOffsetVal); -int32_t tFormatOffset(char* buf, int32_t maxLen, const STqOffsetVal* pVal); +void tFormatOffset(char* buf, int32_t maxLen, const STqOffsetVal* pVal); bool tOffsetEqual(const STqOffsetVal* pLeft, const STqOffsetVal* pRight); void tOffsetCopy(STqOffsetVal* pLeft, const STqOffsetVal* pOffsetVal); void tOffsetDestroy(void* pVal); diff --git a/include/libs/executor/executor.h b/include/libs/executor/executor.h index 082e824e5a..324804aa7c 100644 --- a/include/libs/executor/executor.h +++ b/include/libs/executor/executor.h @@ -211,7 +211,7 @@ const char* qExtractTbnameFromTask(qTaskInfo_t tinfo); void* qExtractReaderFromStreamScanner(void* scanner); -int32_t qExtractStreamScanner(qTaskInfo_t tinfo, void** scanner); +void qExtractStreamScanner(qTaskInfo_t tinfo, void** scanner); int32_t qSetStreamOperatorOptionForScanHistory(qTaskInfo_t tinfo); int32_t qStreamSourceScanParamForHistoryScanStep1(qTaskInfo_t tinfo, SVersionRange *pVerRange, STimeWindow* pWindow); diff --git a/include/libs/executor/storageapi.h b/include/libs/executor/storageapi.h index 05ef14657b..a0ff3353bc 100644 --- a/include/libs/executor/storageapi.h +++ b/include/libs/executor/storageapi.h @@ -218,10 +218,10 @@ typedef struct SStoreTqReader { int32_t (*tqGetStreamExecProgress)(); void (*tqReaderSetColIdList)(); - int32_t (*tqReaderSetQueryTableList)(); + void (*tqReaderSetQueryTableList)(); - int32_t (*tqReaderAddTables)(); - int32_t (*tqReaderRemoveTables)(); + void (*tqReaderAddTables)(); + void (*tqReaderRemoveTables)(); void (*tqSetTablePrimaryKey)(); bool (*tqGetTablePrimaryKey)(); @@ -239,8 +239,8 @@ typedef struct SStoreSnapshotFn { bool (*taosXGetTablePrimaryKey)(SSnapContext* ctx); void (*taosXSetTablePrimaryKey)(SSnapContext* ctx, int64_t uid); int32_t (*setForSnapShot)(SSnapContext* ctx, int64_t uid); - int32_t (*destroySnapshot)(SSnapContext* ctx); - SMetaTableInfo (*getMetaTableInfoFromSnapshot)(SSnapContext* ctx); + void (*destroySnapshot)(SSnapContext* ctx); + int32_t (*getMetaTableInfoFromSnapshot)(SSnapContext* ctx, SMetaTableInfo* info); int32_t (*getTableInfoFromSnapshot)(SSnapContext* ctx, void** pBuf, int32_t* contLen, int16_t* type, int64_t* uid); } SStoreSnapshotFn; diff --git a/include/util/tutil.h b/include/util/tutil.h index 5af79dfc49..84d42482cb 100644 --- a/include/util/tutil.h +++ b/include/util/tutil.h @@ -152,6 +152,9 @@ static FORCE_INLINE int32_t taosGetTbHashVal(const char *tbname, int32_t tblen, #define TCONTAINER_OF(ptr, type, member) ((type *)((char *)(ptr)-offsetof(type, member))) +#define TAOS_GET_TERRNO(code) \ + (terrno == 0 ? code : terrno) \ + #define TAOS_RETURN(code) \ do { \ return (terrno = (code)); \ diff --git a/source/client/src/clientRawBlockWrite.c b/source/client/src/clientRawBlockWrite.c index 9e1ff6e5e5..0e45c5f322 100644 --- a/source/client/src/clientRawBlockWrite.c +++ b/source/client/src/clientRawBlockWrite.c @@ -484,7 +484,7 @@ static void buildChildElement(cJSON* json, SVCreateTbReq* pCreateReq) { end: RAW_FALSE_CHECK(cJSON_AddItemToObject(json, "tags", tags)); - (void)taosArrayDestroy(pTagVals); + taosArrayDestroy(pTagVals); } static void buildCreateCTableJson(SVCreateTbReq* pCreateReq, int32_t nReqs, cJSON** pJson) { @@ -588,7 +588,7 @@ end: tDecoderClear(&decoder[i]); taosMemoryFreeClear(pCreateReq[i].comment); if (pCreateReq[i].type == TSDB_CHILD_TABLE) { - (void)taosArrayDestroy(pCreateReq[i].ctb.tagName); + taosArrayDestroy(pCreateReq[i].ctb.tagName); } } taosMemoryFree(decoder); @@ -1097,7 +1097,7 @@ typedef struct SVgroupCreateTableBatch { static void destroyCreateTbReqBatch(void* data) { SVgroupCreateTableBatch* pTbBatch = (SVgroupCreateTableBatch*)data; - (void)taosArrayDestroy(pTbBatch->req.pArray); + taosArrayDestroy(pTbBatch->req.pArray); } static int32_t taosCreateTable(TAOS* taos, void* meta, int32_t metaLen) { @@ -1250,7 +1250,7 @@ typedef struct SVgroupDropTableBatch { static void destroyDropTbReqBatch(void* data) { SVgroupDropTableBatch* pTbBatch = (SVgroupDropTableBatch*)data; - (void)taosArrayDestroy(pTbBatch->req.pArray); + taosArrayDestroy(pTbBatch->req.pArray); } static int32_t taosDropTable(TAOS* taos, void* meta, int32_t metaLen) { @@ -1513,7 +1513,7 @@ static int32_t taosAlterTable(TAOS* taos, void* meta, int32_t metaLen) { } end: uDebug(LOG_ID_TAG " alter table return, meta:%p, len:%d, msg:%s", LOG_ID_VALUE, meta, metaLen, tstrerror(code)); - (void)taosArrayDestroy(pArray); + taosArrayDestroy(pArray); if (pVgData) taosMemoryFreeClear(pVgData->pData); taosMemoryFreeClear(pVgData); destroyRequest(pRequest); diff --git a/source/client/src/clientSml.c b/source/client/src/clientSml.c index 00445b4d12..112cd2d7ed 100644 --- a/source/client/src/clientSml.c +++ b/source/client/src/clientSml.c @@ -197,8 +197,8 @@ static void smlDestroySTableMeta(void *para) { SSmlSTableMeta *meta = *(SSmlSTableMeta **)para; taosHashCleanup(meta->tagHash); taosHashCleanup(meta->colHash); - (void)taosArrayDestroy(meta->tags); - (void)taosArrayDestroy(meta->cols); + taosArrayDestroy(meta->tags); + taosArrayDestroy(meta->cols); taosMemoryFreeClear(meta->tableMeta); taosMemoryFree(meta); } @@ -568,7 +568,7 @@ int32_t smlSetCTableName(SSmlTableInfo *oneTable) { if (code != TSDB_CODE_SUCCESS){ return code; } - (void)taosArrayDestroy(dst); + taosArrayDestroy(dst); } return TSDB_CODE_SUCCESS; } @@ -1198,25 +1198,25 @@ static int32_t smlModifyDBSchemas(SSmlHandle *info) { SArray *pTags = taosArrayInit(taosArrayGetSize(sTableData->tags), sizeof(SField)); if (pTags == NULL) { code = TSDB_CODE_OUT_OF_MEMORY; - (void)taosArrayDestroy(pColumns); + taosArrayDestroy(pColumns); goto end; } code = smlBuildFieldsList(info, NULL, NULL, sTableData->tags, pTags, 0, true); if (code != TSDB_CODE_SUCCESS) { uError("SML:0x%" PRIx64 " smlBuildFieldsList tag1 failed. %s", info->id, pName.tname); - (void)taosArrayDestroy(pColumns); - (void)taosArrayDestroy(pTags); + taosArrayDestroy(pColumns); + taosArrayDestroy(pTags); goto end; } code = smlBuildFieldsList(info, NULL, NULL, sTableData->cols, pColumns, 0, false); if (code != TSDB_CODE_SUCCESS) { uError("SML:0x%" PRIx64 " smlBuildFieldsList col1 failed. %s", info->id, pName.tname); - (void)taosArrayDestroy(pColumns); - (void)taosArrayDestroy(pTags); + taosArrayDestroy(pColumns); + taosArrayDestroy(pTags); goto end; } code = smlSendMetaMsg(info, &pName, pColumns, pTags, NULL, SCHEMA_ACTION_CREATE_STABLE); - (void)taosArrayDestroy(pColumns); + taosArrayDestroy(pColumns); if (code != TSDB_CODE_SUCCESS) { uError("SML:0x%" PRIx64 " smlSendMetaMsg failed. can not create %s", info->id, pName.tname); goto end; @@ -1271,7 +1271,7 @@ static int32_t smlModifyDBSchemas(SSmlHandle *info) { SArray *pTags = taosArrayInit(taosArrayGetSize(sTableData->tags) + pTableMeta->tableInfo.numOfTags, sizeof(SField)); if (pTags == NULL){ - (void)taosArrayDestroy(pColumns); + taosArrayDestroy(pColumns); code = TSDB_CODE_OUT_OF_MEMORY; goto end; } @@ -1282,15 +1282,15 @@ static int32_t smlModifyDBSchemas(SSmlHandle *info) { tstrncpy(field.name, pTableMeta->schema[i].name, sizeof(field.name)); if (i < pTableMeta->tableInfo.numOfColumns) { if (taosArrayPush(pColumns, &field) == NULL){ - (void)taosArrayDestroy(pColumns); - (void)taosArrayDestroy(pTags); + taosArrayDestroy(pColumns); + taosArrayDestroy(pTags); code = TSDB_CODE_OUT_OF_MEMORY; goto end; } } else { if (taosArrayPush(pTags, &field) == NULL){ - (void)taosArrayDestroy(pColumns); - (void)taosArrayDestroy(pTags); + taosArrayDestroy(pColumns); + taosArrayDestroy(pTags); code = TSDB_CODE_OUT_OF_MEMORY; goto end; } @@ -1300,28 +1300,28 @@ static int32_t smlModifyDBSchemas(SSmlHandle *info) { pTableMeta->tableInfo.numOfColumns, true); if (code != TSDB_CODE_SUCCESS) { uError("SML:0x%" PRIx64 " smlBuildFieldsList tag2 failed. %s", info->id, pName.tname); - (void)taosArrayDestroy(pColumns); - (void)taosArrayDestroy(pTags); + taosArrayDestroy(pColumns); + taosArrayDestroy(pTags); goto end; } if (taosArrayGetSize(pTags) + pTableMeta->tableInfo.numOfColumns > TSDB_MAX_COLUMNS) { uError("SML:0x%" PRIx64 " too many columns than 4096", info->id); code = TSDB_CODE_PAR_TOO_MANY_COLUMNS; - (void)taosArrayDestroy(pColumns); - (void)taosArrayDestroy(pTags); + taosArrayDestroy(pColumns); + taosArrayDestroy(pTags); goto end; } if (taosArrayGetSize(pTags) > TSDB_MAX_TAGS) { uError("SML:0x%" PRIx64 " too many tags than 128", info->id); code = TSDB_CODE_PAR_INVALID_TAGS_NUM; - (void)taosArrayDestroy(pColumns); - (void)taosArrayDestroy(pTags); + taosArrayDestroy(pColumns); + taosArrayDestroy(pTags); goto end; } code = smlSendMetaMsg(info, &pName, pColumns, pTags, pTableMeta, action); - (void)taosArrayDestroy(pColumns); + taosArrayDestroy(pColumns); if (code != TSDB_CODE_SUCCESS) { uError("SML:0x%" PRIx64 " smlSendMetaMsg failed. can not create %s", info->id, pName.tname); goto end; @@ -1367,7 +1367,7 @@ static int32_t smlModifyDBSchemas(SSmlHandle *info) { SArray *pTags = taosArrayInit(taosArrayGetSize(sTableData->tags) + pTableMeta->tableInfo.numOfTags, sizeof(SField)); if (pTags == NULL){ - (void)taosArrayDestroy(pColumns); + taosArrayDestroy(pColumns); code = TSDB_CODE_OUT_OF_MEMORY; goto end; } @@ -1378,15 +1378,15 @@ static int32_t smlModifyDBSchemas(SSmlHandle *info) { tstrncpy(field.name, pTableMeta->schema[i].name, sizeof(field.name)); if (i < pTableMeta->tableInfo.numOfColumns) { if (taosArrayPush(pColumns, &field) == NULL){ - (void)taosArrayDestroy(pColumns); - (void)taosArrayDestroy(pTags); + taosArrayDestroy(pColumns); + taosArrayDestroy(pTags); code = TSDB_CODE_OUT_OF_MEMORY; goto end; } } else { if (taosArrayPush(pTags, &field) == NULL){ - (void)taosArrayDestroy(pColumns); - (void)taosArrayDestroy(pTags); + taosArrayDestroy(pColumns); + taosArrayDestroy(pTags); code = TSDB_CODE_OUT_OF_MEMORY; goto end; } @@ -1397,21 +1397,21 @@ static int32_t smlModifyDBSchemas(SSmlHandle *info) { pTableMeta->tableInfo.numOfColumns, false); if (code != TSDB_CODE_SUCCESS) { uError("SML:0x%" PRIx64 " smlBuildFieldsList col2 failed. %s", info->id, pName.tname); - (void)taosArrayDestroy(pColumns); - (void)taosArrayDestroy(pTags); + taosArrayDestroy(pColumns); + taosArrayDestroy(pTags); goto end; } if (taosArrayGetSize(pColumns) + pTableMeta->tableInfo.numOfTags > TSDB_MAX_COLUMNS) { uError("SML:0x%" PRIx64 " too many columns than 4096", info->id); code = TSDB_CODE_PAR_TOO_MANY_COLUMNS; - (void)taosArrayDestroy(pColumns); - (void)taosArrayDestroy(pTags); + taosArrayDestroy(pColumns); + taosArrayDestroy(pTags); goto end; } code = smlSendMetaMsg(info, &pName, pColumns, pTags, pTableMeta, action); - (void)taosArrayDestroy(pColumns); + taosArrayDestroy(pColumns); if (code != TSDB_CODE_SUCCESS) { uError("SML:0x%" PRIx64 " smlSendMetaMsg failed. can not create %s", info->id, pName.tname); goto end; @@ -1555,7 +1555,7 @@ void smlDestroyTableInfo(void *para) { taosHashCleanup(kvHash); } - (void)taosArrayDestroy(tag->cols); + taosArrayDestroy(tag->cols); taosArrayDestroyEx(tag->tags, freeSSmlKv); taosMemoryFree(tag); } @@ -1581,13 +1581,13 @@ void smlDestroyInfo(SSmlHandle *info) { cJSON *tags = (cJSON *)taosArrayGetP(info->tagJsonArray, i); cJSON_Delete(tags); } - (void)taosArrayDestroy(info->tagJsonArray); + taosArrayDestroy(info->tagJsonArray); for (int i = 0; i < taosArrayGetSize(info->valueJsonArray); i++) { cJSON *value = (cJSON *)taosArrayGetP(info->valueJsonArray, i); cJSON_Delete(value); } - (void)taosArrayDestroy(info->valueJsonArray); + taosArrayDestroy(info->valueJsonArray); taosArrayDestroyEx(info->preLineTagKV, freeSSmlKv); diff --git a/source/client/test/smlTest.cpp b/source/client/test/smlTest.cpp index 77079ed863..dc6a302924 100644 --- a/source/client/test/smlTest.cpp +++ b/source/client/test/smlTest.cpp @@ -65,7 +65,7 @@ TEST(testCase, smlParseInfluxString_Test) { ASSERT_EQ(elements.timestamp, sql + elements.measureTagsLen + 1 + elements.colsLen + 1); ASSERT_EQ(elements.timestampLen, strlen("1626006833639000000")); - (void)taosArrayDestroy(elements.colArray); + taosArrayDestroy(elements.colArray); elements.colArray = nullptr; // case 2 false @@ -74,7 +74,7 @@ TEST(testCase, smlParseInfluxString_Test) { (void)memset(&elements, 0, sizeof(SSmlLineInfo)); ret = smlParseInfluxString(info, sql, sql + strlen(sql), &elements); ASSERT_NE(ret, 0); - (void)taosArrayDestroy(elements.colArray); + taosArrayDestroy(elements.colArray); elements.colArray = nullptr; // case 4 tag is null @@ -95,7 +95,7 @@ TEST(testCase, smlParseInfluxString_Test) { ASSERT_EQ(elements.timestamp, sql + elements.measureTagsLen + 1 + elements.colsLen + 1); ASSERT_EQ(elements.timestampLen, strlen("1626006833639000000")); - (void)taosArrayDestroy(elements.colArray); + taosArrayDestroy(elements.colArray); elements.colArray = nullptr; // case 5 tag is null @@ -115,7 +115,7 @@ TEST(testCase, smlParseInfluxString_Test) { ASSERT_EQ(elements.timestamp, sql + 1 + elements.measureTagsLen + 3 + elements.colsLen + 2); ASSERT_EQ(elements.timestampLen, strlen("1626006833639000000")); - (void)taosArrayDestroy(elements.colArray); + taosArrayDestroy(elements.colArray); elements.colArray = nullptr; // case 6 @@ -124,7 +124,7 @@ TEST(testCase, smlParseInfluxString_Test) { (void)memset(&elements, 0, sizeof(SSmlLineInfo)); ret = smlParseInfluxString(info, sql, sql + strlen(sql), &elements); ASSERT_EQ(ret, 0); - (void)taosArrayDestroy(elements.colArray); + taosArrayDestroy(elements.colArray); elements.colArray = nullptr; ASSERT_EQ(smlClearForRerun(info), 0); @@ -134,7 +134,7 @@ TEST(testCase, smlParseInfluxString_Test) { (void)memset(&elements, 0, sizeof(SSmlLineInfo)); ret = smlParseInfluxString(info, sql, sql + strlen(sql), &elements); ASSERT_NE(ret, 0); - (void)taosArrayDestroy(elements.colArray); + taosArrayDestroy(elements.colArray); elements.colArray = nullptr; // case 8 false @@ -143,7 +143,7 @@ TEST(testCase, smlParseInfluxString_Test) { (void)memset(&elements, 0, sizeof(SSmlLineInfo)); ret = smlParseInfluxString(info, sql, sql + strlen(sql), &elements); ASSERT_NE(ret, 0); - (void)taosArrayDestroy(elements.colArray); + taosArrayDestroy(elements.colArray); elements.colArray = nullptr; taosMemoryFree(sql); @@ -250,7 +250,7 @@ TEST(testCase, smlParseCols_Error_Test) { // printf("i:%d\n", i); ASSERT_NE(ret, TSDB_CODE_SUCCESS); taosMemoryFree(sql); - (void)taosArrayDestroy(elements.colArray); + taosArrayDestroy(elements.colArray); } smlDestroyInfo(info); } @@ -458,7 +458,7 @@ TEST(testCase, smlParseCols_Test) { ASSERT_EQ(kv->length, 4); ASSERT_EQ(strncasecmp(kv->value, "iuwq", 4), 0); - (void)taosArrayDestroy(elements.colArray); + taosArrayDestroy(elements.colArray); taosMemoryFree(sql); smlDestroyInfo(info); } diff --git a/source/common/src/tdatablock.c b/source/common/src/tdatablock.c index 6746d0343b..cfcf383954 100644 --- a/source/common/src/tdatablock.c +++ b/source/common/src/tdatablock.c @@ -1510,6 +1510,9 @@ int32_t blockDataEnsureCapacity(SSDataBlock* pDataBlock, uint32_t numOfRows) { } void blockDataFreeRes(SSDataBlock* pBlock) { + if (pBlock == NULL){ + return; + } int32_t numOfOutput = taosArrayGetSize(pBlock->pDataBlock); for (int32_t i = 0; i < numOfOutput; ++i) { SColumnInfoData* pColInfoData = (SColumnInfoData*)taosArrayGet(pBlock->pDataBlock, i); diff --git a/source/common/src/tmsg.c b/source/common/src/tmsg.c index a15afa64db..fb881abca5 100644 --- a/source/common/src/tmsg.c +++ b/source/common/src/tmsg.c @@ -9195,7 +9195,7 @@ int32_t tDecodeSTqOffsetVal(SDecoder *pDecoder, STqOffsetVal *pOffsetVal) { return 0; } -int32_t tFormatOffset(char *buf, int32_t maxLen, const STqOffsetVal *pVal) { +void tFormatOffset(char *buf, int32_t maxLen, const STqOffsetVal *pVal) { if (pVal->type == TMQ_OFFSET__RESET_NONE) { (void)snprintf(buf, maxLen, "none"); } else if (pVal->type == TMQ_OFFSET__RESET_EARLIEST) { @@ -9207,7 +9207,9 @@ int32_t tFormatOffset(char *buf, int32_t maxLen, const STqOffsetVal *pVal) { } else if (pVal->type == TMQ_OFFSET__SNAPSHOT_DATA || pVal->type == TMQ_OFFSET__SNAPSHOT_META) { if (IS_VAR_DATA_TYPE(pVal->primaryKey.type)) { char *tmp = taosMemoryCalloc(1, pVal->primaryKey.nData + 1); - if (tmp == NULL) return TSDB_CODE_OUT_OF_MEMORY; + if (tmp == NULL) { + return; + } (void)memcpy(tmp, pVal->primaryKey.pData, pVal->primaryKey.nData); (void)snprintf(buf, maxLen, "tsdb:%" PRId64 "|%" PRId64 ",pk type:%d,val:%s", pVal->uid, pVal->ts, pVal->primaryKey.type, tmp); @@ -9216,11 +9218,7 @@ int32_t tFormatOffset(char *buf, int32_t maxLen, const STqOffsetVal *pVal) { (void)snprintf(buf, maxLen, "tsdb:%" PRId64 "|%" PRId64 ",pk type:%d,val:%" PRId64, pVal->uid, pVal->ts, pVal->primaryKey.type, pVal->primaryKey.val); } - } else { - return TSDB_CODE_INVALID_PARA; } - - return 0; } bool tOffsetEqual(const STqOffsetVal *pLeft, const STqOffsetVal *pRight) { @@ -9248,7 +9246,7 @@ void tOffsetCopy(STqOffsetVal *pLeft, const STqOffsetVal *pRight) { *pLeft = *pRight; if (IS_VAR_DATA_TYPE(pRight->primaryKey.type)) { pLeft->primaryKey.pData = taosMemoryMalloc(pRight->primaryKey.nData); - memcpy(pLeft->primaryKey.pData, pRight->primaryKey.pData, pRight->primaryKey.nData); + (void)memcpy(pLeft->primaryKey.pData, pRight->primaryKey.pData, pRight->primaryKey.nData); } } @@ -9310,7 +9308,7 @@ int32_t tDecodeSTqCheckInfo(SDecoder *pDecoder, STqCheckInfo *pInfo) { for (int32_t i = 0; i < sz; i++) { int16_t colId = 0; if (tDecodeI16(pDecoder, &colId) < 0) return -1; - taosArrayPush(pInfo->colIdList, &colId); + if (taosArrayPush(pInfo->colIdList, &colId) == NULL) return -1; } return 0; } diff --git a/source/dnode/mnode/impl/src/mndConsumer.c b/source/dnode/mnode/impl/src/mndConsumer.c index 1b77a0ad6a..8ab50bbf70 100644 --- a/source/dnode/mnode/impl/src/mndConsumer.c +++ b/source/dnode/mnode/impl/src/mndConsumer.c @@ -942,7 +942,7 @@ static int32_t mndRetrieveConsumer(SRpcMsg *pReq, SShowObj *pShow, SSDataBlock * char buf[TSDB_OFFSET_LEN] = {0}; STqOffsetVal pVal = {.type = pConsumer->resetOffsetCfg}; - MND_TMQ_RETURN_CHECK(tFormatOffset(buf, TSDB_OFFSET_LEN, &pVal)); + tFormatOffset(buf, TSDB_OFFSET_LEN, &pVal); char parasStr[64 + TSDB_OFFSET_LEN + VARSTR_HEADER_SIZE] = {0}; (void)sprintf(varDataVal(parasStr), "tbname:%d,commit:%d,interval:%dms,reset:%s", pConsumer->withTbName, diff --git a/source/dnode/mnode/impl/src/mndSubscribe.c b/source/dnode/mnode/impl/src/mndSubscribe.c index a110d442cf..9eda7625e5 100644 --- a/source/dnode/mnode/impl/src/mndSubscribe.c +++ b/source/dnode/mnode/impl/src/mndSubscribe.c @@ -1371,7 +1371,7 @@ static int32_t buildResult(SSDataBlock *pBlock, int32_t *numOfRows, int64_t cons if (data) { // vg id char buf[TSDB_OFFSET_LEN * 2 + VARSTR_HEADER_SIZE] = {0}; - MND_TMQ_RETURN_CHECK(tFormatOffset(varDataVal(buf), TSDB_OFFSET_LEN, &data->offset)); + tFormatOffset(varDataVal(buf), TSDB_OFFSET_LEN, &data->offset); (void)sprintf(varDataVal(buf) + strlen(varDataVal(buf)), "/%" PRId64, data->ever); varDataSetLen(buf, strlen(varDataVal(buf))); pColInfo = taosArrayGet(pBlock->pDataBlock, cols++); diff --git a/source/dnode/vnode/inc/vnode.h b/source/dnode/vnode/inc/vnode.h index 6dc777b6df..d01db56013 100644 --- a/source/dnode/vnode/inc/vnode.h +++ b/source/dnode/vnode/inc/vnode.h @@ -223,9 +223,9 @@ bool tqGetTablePrimaryKey(STqReader *pReader); void tqSetTablePrimaryKey(STqReader *pReader, int64_t uid); void tqReaderSetColIdList(STqReader *pReader, SArray *pColIdList); -int32_t tqReaderSetTbUidList(STqReader *pReader, const SArray *tbUidList, const char *id); -int32_t tqReaderAddTbUidList(STqReader *pReader, const SArray *pTableUidList); -int32_t tqReaderRemoveTbUidList(STqReader *pReader, const SArray *tbUidList); +void tqReaderSetTbUidList(STqReader *pReader, const SArray *tbUidList, const char *id); +void tqReaderAddTbUidList(STqReader *pReader, const SArray *pTableUidList); +void tqReaderRemoveTbUidList(STqReader *pReader, const SArray *tbUidList); bool tqReaderIsQueriedTable(STqReader *pReader, uint64_t uid); bool tqCurrentBlockConsumed(const STqReader *pReader); @@ -261,9 +261,9 @@ void taosXSetTablePrimaryKey(SSnapContext *ctx, int64_t uid); int32_t buildSnapContext(SVnode *pVnode, int64_t snapVersion, int64_t suid, int8_t subType, int8_t withMeta, SSnapContext **ctxRet); int32_t getTableInfoFromSnapshot(SSnapContext *ctx, void **pBuf, int32_t *contLen, int16_t *type, int64_t *uid); -SMetaTableInfo getMetaTableInfoFromSnapshot(SSnapContext *ctx); +int32_t getMetaTableInfoFromSnapshot(SSnapContext *ctx, SMetaTableInfo* info); int32_t setForSnapShot(SSnapContext *ctx, int64_t uid); -int32_t destroySnapContext(SSnapContext *ctx); +void destroySnapContext(SSnapContext *ctx); // structs struct STsdbCfg { diff --git a/source/dnode/vnode/src/inc/tq.h b/source/dnode/vnode/src/inc/tq.h index 56a3c00fee..050c6d09dc 100644 --- a/source/dnode/vnode/src/inc/tq.h +++ b/source/dnode/vnode/src/inc/tq.h @@ -121,11 +121,11 @@ int32_t tqTaosxScanLog(STQ* pTq, STqHandle* pHandle, SPackedData submit, STaosxR int32_t tqAddBlockDataToRsp(const SSDataBlock* pBlock, void* pRsp, int32_t numOfCols, int8_t precision); int32_t tqSendDataRsp(STqHandle* pHandle, const SRpcMsg* pMsg, const SMqPollReq* pReq, const void* pRsp, int32_t type, int32_t vgId); -int32_t tqPushEmptyDataRsp(STqHandle* pHandle, int32_t vgId); +void tqPushEmptyDataRsp(STqHandle* pHandle, int32_t vgId); // tqMeta int32_t tqMetaOpen(STQ* pTq); -int32_t tqMetaClose(STQ* pTq); +void tqMetaClose(STQ* pTq); int32_t tqMetaSaveHandle(STQ* pTq, const char* key, const STqHandle* pHandle); int32_t tqMetaSaveInfo(STQ* pTq, TTB* ttb, const void* key, int32_t kLen, const void* value, int32_t vLen); int32_t tqMetaDeleteInfo(STQ* pTq, TTB* ttb, const void* key, int32_t kLen); @@ -133,7 +133,7 @@ int32_t tqMetaCreateHandle(STQ* pTq, SMqRebVgReq* req, STqHandle* handle); int32_t tqMetaDecodeCheckInfo(STqCheckInfo *info, void *pVal, int32_t vLen); int32_t tqMetaDecodeOffsetInfo(STqOffset *info, void *pVal, int32_t vLen); int32_t tqMetaGetHandle(STQ* pTq, const char* key, STqHandle** pHandle); -void* tqMetaGetOffset(STQ* pTq, const char* subkey); +int32_t tqMetaGetOffset(STQ* pTq, const char* subkey, STqOffset** pOffset); int32_t tqMetaTransform(STQ* pTq); // tqSink int32_t tqBuildDeleteReq(STQ* pTq, const char* stbFullName, const SSDataBlock* pDataBlock, SBatchDeleteReq* deleteReq, @@ -166,12 +166,12 @@ int32_t buildAutoCreateTableReq(const char* stbFullName, int64_t suid, int32_t n } \ } while (0) -#define TQ_ERR_RETURN(c) \ - do { \ - code = c; \ - if (code != TSDB_CODE_SUCCESS) { \ - return code; \ - } \ +#define TQ_NULL_GO_TO_END(c) \ + do { \ + if (c == NULL) { \ + code = (terrno == 0 ? TSDB_CODE_OUT_OF_MEMORY : terrno); \ + goto END; \ + } \ } while (0) #define TQ_SUBSCRIBE_NAME "subscribe" diff --git a/source/dnode/vnode/src/inc/vnodeInt.h b/source/dnode/vnode/src/inc/vnodeInt.h index 32e4fea336..6bcacdf66f 100644 --- a/source/dnode/vnode/src/inc/vnodeInt.h +++ b/source/dnode/vnode/src/inc/vnodeInt.h @@ -239,12 +239,12 @@ int32_t tsdbSetKeepCfg(STsdb* pTsdb, STsdbCfg* pCfg); int64_t tsdbGetEarliestTs(STsdb* pTsdb); // tq -STQ* tqOpen(const char* path, SVnode* pVnode); +int32_t tqOpen(const char* path, SVnode* pVnode); void tqNotifyClose(STQ*); void tqClose(STQ*); int tqPushMsg(STQ*, tmsg_t msgType); int tqRegisterPushHandle(STQ* pTq, void* handle, SRpcMsg* pMsg); -int tqUnregisterPushHandle(STQ* pTq, void* pHandle); +void tqUnregisterPushHandle(STQ* pTq, void* pHandle); int tqScanWalAsync(STQ* pTq, bool ckPause); int32_t tqStopStreamTasksAsync(STQ* pTq); int32_t tqProcessTaskCheckPointSourceReq(STQ* pTq, SRpcMsg* pMsg, SRpcMsg* pRsp); diff --git a/source/dnode/vnode/src/meta/metaSnapshot.c b/source/dnode/vnode/src/meta/metaSnapshot.c index 95c65e5e80..d219c2859e 100644 --- a/source/dnode/vnode/src/meta/metaSnapshot.c +++ b/source/dnode/vnode/src/meta/metaSnapshot.c @@ -219,49 +219,92 @@ static void destroySTableInfoForChildTable(void* data) { tDeleteSchemaWrapper(pData->tagRow); } -static void MoveToSnapShotVersion(SSnapContext* ctx) { - tdbTbcClose((TBC*)ctx->pCur); - tdbTbcOpen(ctx->pMeta->pTbDb, (TBC**)&ctx->pCur, NULL); +static int32_t MoveToSnapShotVersion(SSnapContext* ctx) { + int32_t code = 0; + (void)tdbTbcClose((TBC*)ctx->pCur); + code = tdbTbcOpen(ctx->pMeta->pTbDb, (TBC**)&ctx->pCur, NULL); + if (code != 0){ + return TAOS_GET_TERRNO(code); + } STbDbKey key = {.version = ctx->snapVersion, .uid = INT64_MAX}; int c = 0; - tdbTbcMoveTo((TBC*)ctx->pCur, &key, sizeof(key), &c); + code = tdbTbcMoveTo((TBC*)ctx->pCur, &key, sizeof(key), &c); + if (code != 0){ + return TAOS_GET_TERRNO(code); + } if (c < 0) { tdbTbcMoveToPrev((TBC*)ctx->pCur); } + return 0; } static int32_t MoveToPosition(SSnapContext* ctx, int64_t ver, int64_t uid) { - tdbTbcClose((TBC*)ctx->pCur); - tdbTbcOpen(ctx->pMeta->pTbDb, (TBC**)&ctx->pCur, NULL); + (void)tdbTbcClose((TBC*)ctx->pCur); + int32_t code = tdbTbcOpen(ctx->pMeta->pTbDb, (TBC**)&ctx->pCur, NULL); + if (code != 0){ + return TAOS_GET_TERRNO(code); + } STbDbKey key = {.version = ver, .uid = uid}; int c = 0; - tdbTbcMoveTo((TBC*)ctx->pCur, &key, sizeof(key), &c); + code = tdbTbcMoveTo((TBC*)ctx->pCur, &key, sizeof(key), &c); + if (code != 0){ + return TAOS_GET_TERRNO(code); + } return c; } -static void MoveToFirst(SSnapContext* ctx) { - tdbTbcClose((TBC*)ctx->pCur); - tdbTbcOpen(ctx->pMeta->pTbDb, (TBC**)&ctx->pCur, NULL); - tdbTbcMoveToFirst((TBC*)ctx->pCur); +static int32_t MoveToFirst(SSnapContext* ctx) { + (void)tdbTbcClose((TBC*)ctx->pCur); + int32_t code = tdbTbcOpen(ctx->pMeta->pTbDb, (TBC**)&ctx->pCur, NULL); + if (code != 0){ + return TAOS_GET_TERRNO(code); + } + code = tdbTbcMoveToFirst((TBC*)ctx->pCur); + if (code != 0){ + return TAOS_GET_TERRNO(code); + } + return 0; } -static void saveSuperTableInfoForChildTable(SMetaEntry* me, SHashObj* suidInfo) { +static int32_t saveSuperTableInfoForChildTable(SMetaEntry* me, SHashObj* suidInfo) { STableInfoForChildTable* data = (STableInfoForChildTable*)taosHashGet(suidInfo, &me->uid, sizeof(tb_uid_t)); if (data) { - return; + return 0; } + int32_t code = 0; STableInfoForChildTable dataTmp = {0}; dataTmp.tableName = taosStrdup(me->name); - + if (dataTmp.tableName == NULL){ + code = TSDB_CODE_OUT_OF_MEMORY; + goto END; + } dataTmp.schemaRow = tCloneSSchemaWrapper(&me->stbEntry.schemaRow); + if (dataTmp.schemaRow == NULL){ + code = TSDB_CODE_OUT_OF_MEMORY; + goto END; + } dataTmp.tagRow = tCloneSSchemaWrapper(&me->stbEntry.schemaTag); - taosHashPut(suidInfo, &me->uid, sizeof(tb_uid_t), &dataTmp, sizeof(STableInfoForChildTable)); + if (dataTmp.tagRow == NULL){ + code = TSDB_CODE_OUT_OF_MEMORY; + goto END; + } + code = taosHashPut(suidInfo, &me->uid, sizeof(tb_uid_t), &dataTmp, sizeof(STableInfoForChildTable)); + if (code != 0){ + goto END; + } + return 0; + +END: + destroySTableInfoForChildTable(&dataTmp); + return TAOS_GET_TERRNO(code);; } int32_t buildSnapContext(SVnode* pVnode, int64_t snapVersion, int64_t suid, int8_t subType, int8_t withMeta, SSnapContext** ctxRet) { SSnapContext* ctx = taosMemoryCalloc(1, sizeof(SSnapContext)); - if (ctx == NULL) return -1; + if (ctx == NULL){ + return TAOS_GET_TERRNO(TSDB_CODE_OUT_OF_MEMORY); + } *ctxRet = ctx; ctx->pMeta = pVnode->pMeta; ctx->snapVersion = snapVersion; @@ -271,23 +314,29 @@ int32_t buildSnapContext(SVnode* pVnode, int64_t snapVersion, int64_t suid, int8 ctx->withMeta = withMeta; ctx->idVersion = taosHashInit(100, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BIGINT), true, HASH_NO_LOCK); if (ctx->idVersion == NULL) { - return -1; + return TAOS_GET_TERRNO(TSDB_CODE_OUT_OF_MEMORY); } ctx->suidInfo = taosHashInit(100, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BIGINT), true, HASH_NO_LOCK); if (ctx->suidInfo == NULL) { - return -1; + return TAOS_GET_TERRNO(TSDB_CODE_OUT_OF_MEMORY);; } taosHashSetFreeFp(ctx->suidInfo, destroySTableInfoForChildTable); ctx->index = 0; ctx->idList = taosArrayInit(100, sizeof(int64_t)); + if (ctx->idList == NULL){ + return TAOS_GET_TERRNO(TSDB_CODE_OUT_OF_MEMORY);; + } void* pKey = NULL; void* pVal = NULL; int vLen = 0, kLen = 0; metaDebug("tmqsnap init snapVersion:%" PRIi64, ctx->snapVersion); - MoveToFirst(ctx); + int32_t code = MoveToFirst(ctx); + if (code != 0){ + return code; + } while (1) { int32_t ret = tdbTbcNext((TBC*)ctx->pCur, &pKey, &kLen, &pVal, &vLen); if (ret < 0) break; @@ -299,33 +348,41 @@ int32_t buildSnapContext(SVnode* pVnode, int64_t snapVersion, int64_t suid, int8 continue; } - if (tdbTbGet(ctx->pMeta->pUidIdx, &tmp->uid, sizeof(tb_uid_t), NULL, NULL) < - 0) { // check if table exist for now, need optimize later + if (tdbTbGet(ctx->pMeta->pUidIdx, &tmp->uid, sizeof(tb_uid_t), NULL, NULL) < 0) { // check if table exist for now, need optimize later continue; } SDecoder dc = {0}; SMetaEntry me = {0}; tDecoderInit(&dc, pVal, vLen); - metaDecodeEntry(&dc, &me); + ret = metaDecodeEntry(&dc, &me); + tDecoderClear(&dc); + + if (ret < 0){ + return TAOS_GET_TERRNO(ret); + } if (ctx->subType == TOPIC_SUB_TYPE__TABLE) { if ((me.uid != ctx->suid && me.type == TSDB_SUPER_TABLE) || (me.ctbEntry.suid != ctx->suid && me.type == TSDB_CHILD_TABLE)) { - tDecoderClear(&dc); continue; } } - taosArrayPush(ctx->idList, &tmp->uid); + if (taosArrayPush(ctx->idList, &tmp->uid) == NULL){ + return TAOS_GET_TERRNO(TSDB_CODE_OUT_OF_MEMORY); + } metaDebug("tmqsnap init idlist name:%s, uid:%" PRIi64, me.name, tmp->uid); SIdInfo info = {0}; - taosHashPut(ctx->idVersion, &tmp->uid, sizeof(tb_uid_t), &info, sizeof(SIdInfo)); - - tDecoderClear(&dc); + if (taosHashPut(ctx->idVersion, &tmp->uid, sizeof(tb_uid_t), &info, sizeof(SIdInfo)) != 0) { + return TAOS_GET_TERRNO(TSDB_CODE_OUT_OF_MEMORY); + } } taosHashClear(ctx->idVersion); - MoveToSnapShotVersion(ctx); + code = MoveToSnapShotVersion(ctx); + if (code != 0){ + return code; + } while (1) { int32_t ret = tdbTbcPrev((TBC*)ctx->pCur, &pKey, &kLen, &pVal, &vLen); if (ret < 0) break; @@ -336,33 +393,45 @@ int32_t buildSnapContext(SVnode* pVnode, int64_t snapVersion, int64_t suid, int8 continue; } SIdInfo info = {.version = tmp->version, .index = 0}; - taosHashPut(ctx->idVersion, &tmp->uid, sizeof(tb_uid_t), &info, sizeof(SIdInfo)); + ret = taosHashPut(ctx->idVersion, &tmp->uid, sizeof(tb_uid_t), &info, sizeof(SIdInfo)); + if (ret != 0) { + return TAOS_GET_TERRNO(ret); + } SDecoder dc = {0}; SMetaEntry me = {0}; tDecoderInit(&dc, pVal, vLen); - metaDecodeEntry(&dc, &me); + ret = metaDecodeEntry(&dc, &me); + tDecoderClear(&dc); + if (ret < 0){ + return TAOS_GET_TERRNO(ret); + } + if (ctx->subType == TOPIC_SUB_TYPE__TABLE) { if ((me.uid != ctx->suid && me.type == TSDB_SUPER_TABLE) || (me.ctbEntry.suid != ctx->suid && me.type == TSDB_CHILD_TABLE)) { - tDecoderClear(&dc); continue; } } if ((ctx->subType == TOPIC_SUB_TYPE__DB && me.type == TSDB_SUPER_TABLE) || (ctx->subType == TOPIC_SUB_TYPE__TABLE && me.uid == ctx->suid)) { - saveSuperTableInfoForChildTable(&me, ctx->suidInfo); + ret = saveSuperTableInfoForChildTable(&me, ctx->suidInfo); + if (ret != 0){ + return ret; + } } - tDecoderClear(&dc); } for (int i = 0; i < taosArrayGetSize(ctx->idList); i++) { int64_t* uid = taosArrayGet(ctx->idList, i); + if (uid == NULL){ + return TAOS_GET_TERRNO(TSDB_CODE_OUT_OF_MEMORY); + } SIdInfo* idData = (SIdInfo*)taosHashGet(ctx->idVersion, uid, sizeof(int64_t)); if (!idData) { metaError("meta/snap: null idData"); - return TSDB_CODE_FAILED; + return TAOS_GET_TERRNO(TSDB_CODE_OUT_OF_MEMORY); } idData->index = i; @@ -375,13 +444,12 @@ int32_t buildSnapContext(SVnode* pVnode, int64_t snapVersion, int64_t suid, int8 return TDB_CODE_SUCCESS; } -int32_t destroySnapContext(SSnapContext* ctx) { - tdbTbcClose((TBC*)ctx->pCur); +void destroySnapContext(SSnapContext* ctx) { + (void)tdbTbcClose((TBC*)ctx->pCur); taosArrayDestroy(ctx->idList); taosHashCleanup(ctx->idVersion); taosHashCleanup(ctx->suidInfo); taosMemoryFree(ctx); - return 0; } static int32_t buildNormalChildTableInfo(SVCreateTbReq* req, void** pBuf, int32_t* contLen) { @@ -390,32 +458,36 @@ static int32_t buildNormalChildTableInfo(SVCreateTbReq* req, void** pBuf, int32_ reqs.pArray = taosArrayInit(1, sizeof(struct SVCreateTbReq)); if (NULL == reqs.pArray) { - ret = -1; + ret = TAOS_GET_TERRNO(TSDB_CODE_OUT_OF_MEMORY); + goto end; + } + if (taosArrayPush(reqs.pArray, req) == NULL){ + ret = TAOS_GET_TERRNO(TSDB_CODE_OUT_OF_MEMORY); goto end; } - taosArrayPush(reqs.pArray, req); reqs.nReqs = 1; tEncodeSize(tEncodeSVCreateTbBatchReq, &reqs, *contLen, ret); if (ret < 0) { - ret = -1; + ret = TAOS_GET_TERRNO(ret); goto end; } *contLen += sizeof(SMsgHead); *pBuf = taosMemoryMalloc(*contLen); if (NULL == *pBuf) { - ret = -1; + ret = TAOS_GET_TERRNO(TSDB_CODE_OUT_OF_MEMORY); goto end; } SEncoder coder = {0}; tEncoderInit(&coder, POINTER_SHIFT(*pBuf, sizeof(SMsgHead)), *contLen); - if (tEncodeSVCreateTbBatchReq(&coder, &reqs) < 0) { + ret = tEncodeSVCreateTbBatchReq(&coder, &reqs); + tEncoderClear(&coder); + + if (ret < 0) { taosMemoryFreeClear(*pBuf); - tEncoderClear(&coder); - ret = -1; + ret = TAOS_GET_TERRNO(ret); goto end; } - tEncoderClear(&coder); end: taosArrayDestroy(reqs.pArray); @@ -426,48 +498,46 @@ static int32_t buildSuperTableInfo(SVCreateStbReq* req, void** pBuf, int32_t* co int32_t ret = 0; tEncodeSize(tEncodeSVCreateStbReq, req, *contLen, ret); if (ret < 0) { - return -1; + return TAOS_GET_TERRNO(ret); } *contLen += sizeof(SMsgHead); *pBuf = taosMemoryMalloc(*contLen); if (NULL == *pBuf) { - return -1; + return TAOS_GET_TERRNO(TSDB_CODE_OUT_OF_MEMORY); } SEncoder encoder = {0}; tEncoderInit(&encoder, POINTER_SHIFT(*pBuf, sizeof(SMsgHead)), *contLen); - if (tEncodeSVCreateStbReq(&encoder, req) < 0) { - taosMemoryFreeClear(*pBuf); - tEncoderClear(&encoder); - return -1; - } + ret = tEncodeSVCreateStbReq(&encoder, req); tEncoderClear(&encoder); + if (ret < 0) { + taosMemoryFreeClear(*pBuf); + return TAOS_GET_TERRNO(ret); + } return 0; } int32_t setForSnapShot(SSnapContext* ctx, int64_t uid) { - int c = 0; - if (uid == 0) { ctx->index = 0; - return c; + return 0; } SIdInfo* idInfo = (SIdInfo*)taosHashGet(ctx->idVersion, &uid, sizeof(tb_uid_t)); - if (!idInfo) { - return -1; + if (idInfo == NULL) { + return terrno; } ctx->index = idInfo->index; - return c; + return 0; } -void taosXSetTablePrimaryKey(SSnapContext* ctx, int64_t uid) { - bool ret = false; - SSchemaWrapper* schema = metaGetTableSchema(ctx->pMeta, uid, -1, 1); - if (schema->nCols >= 2 && schema->pSchema[1].flags & COL_IS_KEY) { +void taosXSetTablePrimaryKey(SSnapContext* ctx, int64_t uid){ + bool ret = false; + SSchemaWrapper *schema = metaGetTableSchema(ctx->pMeta, uid, -1, 1); + if (schema && schema->nCols >= 2 && schema->pSchema[1].flags & COL_IS_KEY){ ret = true; } tDeleteSchemaWrapper(schema); @@ -491,11 +561,15 @@ int32_t getTableInfoFromSnapshot(SSnapContext* ctx, void** pBuf, int32_t* contLe } int64_t* uidTmp = taosArrayGet(ctx->idList, ctx->index); + if (uidTmp == NULL) { + metaError("tmqsnap get meta null uid"); + return TAOS_GET_TERRNO(TSDB_CODE_OUT_OF_MEMORY); + } ctx->index++; SIdInfo* idInfo = (SIdInfo*)taosHashGet(ctx->idVersion, uidTmp, sizeof(tb_uid_t)); if (!idInfo) { metaError("meta/snap: null idInfo"); - return TSDB_CODE_FAILED; + return TAOS_GET_TERRNO(TSDB_CODE_OUT_OF_MEMORY); } *uid = *uidTmp; @@ -506,11 +580,18 @@ int32_t getTableInfoFromSnapshot(SSnapContext* ctx, void** pBuf, int32_t* contLe metaDebug("tmqsnap get meta not exist uid:%" PRIi64 " version:%" PRIi64, *uid, idInfo->version); } - tdbTbcGet((TBC*)ctx->pCur, (const void**)&pKey, &kLen, (const void**)&pVal, &vLen); + ret = tdbTbcGet((TBC*)ctx->pCur, (const void**)&pKey, &kLen, (const void**)&pVal, &vLen); + if (ret < 0) { + return TAOS_GET_TERRNO(ret); + } SDecoder dc = {0}; SMetaEntry me = {0}; tDecoderInit(&dc, pVal, vLen); - metaDecodeEntry(&dc, &me); + ret = metaDecodeEntry(&dc, &me); + if (ret < 0) { + ret = TAOS_GET_TERRNO(ret); + goto END; + } metaDebug("tmqsnap get meta uid:%" PRIi64 " name:%s index:%d", *uid, me.name, ctx->index - 1); if ((ctx->subType == TOPIC_SUB_TYPE__DB && me.type == TSDB_SUPER_TABLE) || @@ -526,14 +607,14 @@ int32_t getTableInfoFromSnapshot(SSnapContext* ctx, void** pBuf, int32_t* contLe ret = buildSuperTableInfo(&req, pBuf, contLen); *type = TDMT_VND_CREATE_STB; - } else if ((ctx->subType == TOPIC_SUB_TYPE__DB && me.type == TSDB_CHILD_TABLE) || (ctx->subType == TOPIC_SUB_TYPE__TABLE && me.type == TSDB_CHILD_TABLE && me.ctbEntry.suid == ctx->suid)) { STableInfoForChildTable* data = (STableInfoForChildTable*)taosHashGet(ctx->suidInfo, &me.ctbEntry.suid, sizeof(tb_uid_t)); if (!data) { metaError("meta/snap: null data"); - return TSDB_CODE_FAILED; + ret = TAOS_GET_TERRNO(TSDB_CODE_OUT_OF_MEMORY); + goto END; } SVCreateTbReq req = {0}; @@ -547,46 +628,48 @@ int32_t getTableInfoFromSnapshot(SSnapContext* ctx, void** pBuf, int32_t* contLe req.ctb.stbName = data->tableName; SArray* tagName = taosArrayInit(req.ctb.tagNum, TSDB_COL_NAME_LEN); + if (tagName == NULL) { + metaError("meta/snap: init tag name failed."); + ret = TAOS_GET_TERRNO(TSDB_CODE_OUT_OF_MEMORY); + goto END; + } STag* p = (STag*)me.ctbEntry.pTags; if (tTagIsJson(p)) { if (p->nTag != 0) { SSchema* schema = &data->tagRow->pSchema[0]; - taosArrayPush(tagName, schema->name); + if (taosArrayPush(tagName, schema->name) == NULL) { + ret = TAOS_GET_TERRNO(TSDB_CODE_OUT_OF_MEMORY); + taosArrayDestroy(tagName); + goto END; + } } } else { SArray* pTagVals = NULL; - if (tTagToValArray((const STag*)p, &pTagVals) != 0) { + ret = tTagToValArray((const STag*)p, &pTagVals); + if (ret != 0) { metaError("meta/snap: tag to val array failed."); - return TSDB_CODE_FAILED; + taosArrayDestroy(pTagVals); + taosArrayDestroy(tagName); + goto END; } int16_t nCols = taosArrayGetSize(pTagVals); for (int j = 0; j < nCols; ++j) { STagVal* pTagVal = (STagVal*)taosArrayGet(pTagVals, j); - for (int i = 0; i < data->tagRow->nCols; i++) { + for (int i = 0; pTagVal && i < data->tagRow->nCols; i++) { SSchema* schema = &data->tagRow->pSchema[i]; if (schema->colId == pTagVal->cid) { - taosArrayPush(tagName, schema->name); + if (taosArrayPush(tagName, schema->name) == NULL) { + ret = TAOS_GET_TERRNO(TSDB_CODE_OUT_OF_MEMORY); + taosArrayDestroy(pTagVals); + taosArrayDestroy(tagName); + goto END; + } } } } taosArrayDestroy(pTagVals); } - // SIdInfo* sidInfo = (SIdInfo*)taosHashGet(ctx->idVersion, &me.ctbEntry.suid, sizeof(tb_uid_t)); - // if(sidInfo->version >= idInfo->version){ - // // need parse tag - // STag* p = (STag*)me.ctbEntry.pTags; - // SArray* pTagVals = NULL; - // if (tTagToValArray((const STag*)p, &pTagVals) != 0) { - // } - // - // int16_t nCols = taosArrayGetSize(pTagVals); - // for (int j = 0; j < nCols; ++j) { - // STagVal* pTagVal = (STagVal*)taosArrayGet(pTagVals, j); - // } - // }else{ req.ctb.pTag = me.ctbEntry.pTags; - // } - req.ctb.tagName = tagName; ret = buildNormalChildTableInfo(&req, pBuf, contLen); *type = TDMT_VND_CREATE_TABLE; @@ -603,15 +686,15 @@ int32_t getTableInfoFromSnapshot(SSnapContext* ctx, void** pBuf, int32_t* contLe *type = TDMT_VND_CREATE_TABLE; } else { metaError("meta/snap: invalid topic sub type: %" PRId8 " get meta from snap failed.", ctx->subType); - ret = -1; + ret = TSDB_CODE_SDB_INVALID_TABLE_TYPE; } - tDecoderClear(&dc); +END: + tDecoderClear(&dc); return ret; } -SMetaTableInfo getMetaTableInfoFromSnapshot(SSnapContext* ctx) { - SMetaTableInfo result = {0}; +int32_t getMetaTableInfoFromSnapshot(SSnapContext* ctx, SMetaTableInfo* result) { void* pKey = NULL; void* pVal = NULL; int vLen, kLen; @@ -619,14 +702,17 @@ SMetaTableInfo getMetaTableInfoFromSnapshot(SSnapContext* ctx) { while (1) { if (ctx->index >= taosArrayGetSize(ctx->idList)) { metaDebug("tmqsnap get uid info end"); - return result; + return 0; } int64_t* uidTmp = taosArrayGet(ctx->idList, ctx->index); + if (uidTmp == NULL){ + return TAOS_GET_TERRNO(TSDB_CODE_OUT_OF_MEMORY); + } ctx->index++; SIdInfo* idInfo = (SIdInfo*)taosHashGet(ctx->idVersion, uidTmp, sizeof(tb_uid_t)); if (!idInfo) { metaError("meta/snap: null idInfo"); - return result; + return TAOS_GET_TERRNO(TSDB_CODE_OUT_OF_MEMORY); } int32_t ret = MoveToPosition(ctx, idInfo->version, *uidTmp); @@ -635,44 +721,46 @@ SMetaTableInfo getMetaTableInfoFromSnapshot(SSnapContext* ctx) { idInfo->version); continue; } - tdbTbcGet((TBC*)ctx->pCur, (const void**)&pKey, &kLen, (const void**)&pVal, &vLen); + ret = tdbTbcGet((TBC*)ctx->pCur, (const void**)&pKey, &kLen, (const void**)&pVal, &vLen); + if (ret != 0){ + return TAOS_GET_TERRNO(ret); + } SDecoder dc = {0}; SMetaEntry me = {0}; tDecoderInit(&dc, pVal, vLen); - metaDecodeEntry(&dc, &me); + ret = metaDecodeEntry(&dc, &me); + if (ret != 0){ + tDecoderClear(&dc); + return TAOS_GET_TERRNO(ret); + } metaDebug("tmqsnap get uid info uid:%" PRIi64 " name:%s index:%d", me.uid, me.name, ctx->index - 1); - if (ctx->subType == TOPIC_SUB_TYPE__DB && me.type == TSDB_CHILD_TABLE) { + if ((ctx->subType == TOPIC_SUB_TYPE__DB && me.type == TSDB_CHILD_TABLE) || + (ctx->subType == TOPIC_SUB_TYPE__TABLE && me.type == TSDB_CHILD_TABLE && me.ctbEntry.suid == ctx->suid)){ STableInfoForChildTable* data = (STableInfoForChildTable*)taosHashGet(ctx->suidInfo, &me.ctbEntry.suid, sizeof(tb_uid_t)); - result.uid = me.uid; - result.suid = me.ctbEntry.suid; - result.schema = tCloneSSchemaWrapper(data->schemaRow); - strcpy(result.tbName, me.name); - tDecoderClear(&dc); - break; + if (data == NULL) { + tDecoderClear(&dc); + metaError("meta/snap: null data"); + return TAOS_GET_TERRNO(TSDB_CODE_OUT_OF_MEMORY); + } + result->suid = me.ctbEntry.suid; + result->schema = tCloneSSchemaWrapper(data->schemaRow); } else if (ctx->subType == TOPIC_SUB_TYPE__DB && me.type == TSDB_NORMAL_TABLE) { - result.uid = me.uid; - result.suid = 0; - strcpy(result.tbName, me.name); - result.schema = tCloneSSchemaWrapper(&me.ntbEntry.schemaRow); - tDecoderClear(&dc); - break; - } else if (ctx->subType == TOPIC_SUB_TYPE__TABLE && me.type == TSDB_CHILD_TABLE && me.ctbEntry.suid == ctx->suid) { - STableInfoForChildTable* data = - (STableInfoForChildTable*)taosHashGet(ctx->suidInfo, &me.ctbEntry.suid, sizeof(tb_uid_t)); - result.uid = me.uid; - result.suid = me.ctbEntry.suid; - strcpy(result.tbName, me.name); - result.schema = tCloneSSchemaWrapper(data->schemaRow); - tDecoderClear(&dc); - break; + result->suid = 0; + result->schema = tCloneSSchemaWrapper(&me.ntbEntry.schemaRow); } else { metaDebug("tmqsnap get uid continue"); tDecoderClear(&dc); continue; } + result->uid = me.uid; + tstrncpy(result->tbName, me.name, TSDB_TABLE_NAME_LEN); + tDecoderClear(&dc); + if(result->schema == NULL){ + return TAOS_GET_TERRNO(TSDB_CODE_OUT_OF_MEMORY); + } + break; } - - return result; + return 0; } diff --git a/source/dnode/vnode/src/tq/tq.c b/source/dnode/vnode/src/tq/tq.c index 45aa845696..c883a9d9dc 100644 --- a/source/dnode/vnode/src/tq/tq.c +++ b/source/dnode/vnode/src/tq/tq.c @@ -60,35 +60,41 @@ static bool tqOffsetEqual(const STqOffset* pLeft, const STqOffset* pRight) { pLeft->val.version == pRight->val.version; } -STQ* tqOpen(const char* path, SVnode* pVnode) { +int32_t tqOpen(const char* path, SVnode* pVnode) { STQ* pTq = taosMemoryCalloc(1, sizeof(STQ)); if (pTq == NULL) { - terrno = TSDB_CODE_OUT_OF_MEMORY; - return NULL; + return TSDB_CODE_OUT_OF_MEMORY; } - + pVnode->pTq = pTq; pTq->path = taosStrdup(path); pTq->pVnode = pVnode; pTq->pHandle = taosHashInit(64, MurmurHash3_32, true, HASH_ENTRY_LOCK); + if (pTq->pHandle == NULL) { + return TSDB_CODE_OUT_OF_MEMORY; + } taosHashSetFreeFp(pTq->pHandle, tqDestroyTqHandle); taosInitRWLatch(&pTq->lock); + pTq->pPushMgr = taosHashInit(64, MurmurHash3_32, false, HASH_NO_LOCK); + if (pTq->pPushMgr == NULL) { + return TSDB_CODE_OUT_OF_MEMORY; + } pTq->pCheckInfo = taosHashInit(64, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY), true, HASH_ENTRY_LOCK); + if (pTq->pCheckInfo == NULL) { + return TSDB_CODE_OUT_OF_MEMORY; + } taosHashSetFreeFp(pTq->pCheckInfo, (FDelete)tDeleteSTqCheckInfo); pTq->pOffset = taosHashInit(64, taosGetDefaultHashFunction(TSDB_DATA_TYPE_VARCHAR), true, HASH_ENTRY_LOCK); + if (pTq->pOffset == NULL) { + return TSDB_CODE_OUT_OF_MEMORY; + } taosHashSetFreeFp(pTq->pOffset, (FDelete)tDeleteSTqOffset); - int32_t code = tqInitialize(pTq); - if (code != TSDB_CODE_SUCCESS) { - tqClose(pTq); - return NULL; - } else { - return pTq; - } + return tqInitialize(pTq); } int32_t tqInitialize(STQ* pTq) { @@ -102,11 +108,7 @@ int32_t tqInitialize(STQ* pTq) { streamMetaLoadAllTasks(pTq->pStreamMeta); - if (tqMetaOpen(pTq) < 0) { - return -1; - } - - return 0; + return tqMetaOpen(pTq); } void tqClose(STQ* pTq) { @@ -150,25 +152,32 @@ void tqNotifyClose(STQ* pTq) { streamMetaNotifyClose(pTq->pStreamMeta); } -int32_t tqPushEmptyDataRsp(STqHandle* pHandle, int32_t vgId) { +void tqPushEmptyDataRsp(STqHandle* pHandle, int32_t vgId) { + int32_t code = 0; SMqPollReq req = {0}; - if (tDeserializeSMqPollReq(pHandle->msg->pCont, pHandle->msg->contLen, &req) < 0) { - tqError("tDeserializeSMqPollReq %d failed", pHandle->msg->contLen); - terrno = TSDB_CODE_INVALID_MSG; - return -1; + code = tDeserializeSMqPollReq(pHandle->msg->pCont, pHandle->msg->contLen, &req); + if (code < 0) { + tqError("tDeserializeSMqPollReq %d failed, code:%d", pHandle->msg->contLen, code); + return; } SMqDataRsp dataRsp = {0}; - tqInitDataRsp(&dataRsp.common, req.reqOffset); + code = tqInitDataRsp(&dataRsp.common, req.reqOffset); + if (code != 0) { + tqError("tqInitDataRsp failed, code:%d", code); + return; + } dataRsp.common.blockNum = 0; char buf[TSDB_OFFSET_LEN] = {0}; tFormatOffset(buf, TSDB_OFFSET_LEN, &dataRsp.common.reqOffset); tqInfo("tqPushEmptyDataRsp to consumer:0x%" PRIx64 " vgId:%d, offset:%s, reqId:0x%" PRIx64, req.consumerId, vgId, buf, req.reqId); - tqSendDataRsp(pHandle, pHandle->msg, &req, &dataRsp, TMQ_MSG_TYPE__POLL_DATA_RSP, vgId); + code = tqSendDataRsp(pHandle, pHandle->msg, &req, &dataRsp, TMQ_MSG_TYPE__POLL_DATA_RSP, vgId); + if (code != 0) { + tqError("tqSendDataRsp failed, code:%d", code); + } tDeleteMqDataRsp(&dataRsp); - return 0; } int32_t tqSendDataRsp(STqHandle* pHandle, const SRpcMsg* pMsg, const SMqPollReq* pReq, const void* pRsp, @@ -176,8 +185,6 @@ int32_t tqSendDataRsp(STqHandle* pHandle, const SRpcMsg* pMsg, const SMqPollReq* int64_t sver = 0, ever = 0; walReaderValidVersionRange(pHandle->execHandle.pTqReader->pWalReader, &sver, &ever); - tqDoSendDataRsp(&pMsg->info, pRsp, pReq->epoch, pReq->consumerId, type, sver, ever); - char buf1[TSDB_OFFSET_LEN] = {0}; char buf2[TSDB_OFFSET_LEN] = {0}; tFormatOffset(buf1, TSDB_OFFSET_LEN, &((SMqDataRspCommon*)pRsp)->reqOffset); @@ -186,7 +193,7 @@ int32_t tqSendDataRsp(STqHandle* pHandle, const SRpcMsg* pMsg, const SMqPollReq* tqDebug("tmq poll vgId:%d consumer:0x%" PRIx64 " (epoch %d) send rsp, block num:%d, req:%s, rsp:%s, reqId:0x%" PRIx64, vgId, pReq->consumerId, pReq->epoch, ((SMqDataRspCommon*)pRsp)->blockNum, buf1, buf2, pReq->reqId); - return 0; + return tqDoSendDataRsp(&pMsg->info, pRsp, pReq->epoch, pReq->consumerId, type, sver, ever); } int32_t tqProcessOffsetCommitReq(STQ* pTq, int64_t sversion, char* msg, int32_t msgLen) { @@ -217,8 +224,9 @@ int32_t tqProcessOffsetCommitReq(STQ* pTq, int64_t sversion, char* msg, int32_t goto end; } - STqOffset* pSavedOffset = (STqOffset*)tqMetaGetOffset(pTq, pOffset->subKey); - if (pSavedOffset != NULL && tqOffsetEqual(pOffset, pSavedOffset)) { + STqOffset* pSavedOffset = NULL; + code = tqMetaGetOffset(pTq, pOffset->subKey, &pSavedOffset); + if (code == 0 && tqOffsetEqual(pOffset, pSavedOffset)) { tqInfo("not update the offset, vgId:%d sub:%s since committed:%" PRId64 " less than/equal to existed:%" PRId64, vgId, pOffset->subKey, pOffset->val.version, pSavedOffset->val.version); goto end; // no need to update the offset value @@ -434,14 +442,13 @@ int32_t tqProcessVgCommittedInfoReq(STQ* pTq, SRpcMsg* pMsg) { tDecoderClear(&decoder); - STqOffset* pSavedOffset = (STqOffset*)tqMetaGetOffset(pTq, vgOffset.offset.subKey); - if (pSavedOffset == NULL) { - terrno = TSDB_CODE_TMQ_NO_COMMITTED; - return terrno; + STqOffset* pSavedOffset = NULL; + int32_t code = tqMetaGetOffset(pTq, vgOffset.offset.subKey, &pSavedOffset); + if (code != 0) { + return TSDB_CODE_TMQ_NO_COMMITTED; } vgOffset.offset = *pSavedOffset; - int32_t code = 0; tEncodeSize(tEncodeMqVgOffset, &vgOffset, len, code); if (code < 0) { terrno = TSDB_CODE_INVALID_PARA; @@ -465,11 +472,11 @@ int32_t tqProcessVgCommittedInfoReq(STQ* pTq, SRpcMsg* pMsg) { } int32_t tqProcessVgWalInfoReq(STQ* pTq, SRpcMsg* pMsg) { + int32_t code = 0; SMqPollReq req = {0}; if (tDeserializeSMqPollReq(pMsg->pCont, pMsg->contLen, &req) < 0) { tqError("tDeserializeSMqPollReq %d failed", pMsg->contLen); - terrno = TSDB_CODE_INVALID_MSG; - return -1; + return TSDB_CODE_INVALID_MSG; } int64_t consumerId = req.consumerId; @@ -481,18 +488,17 @@ int32_t tqProcessVgWalInfoReq(STQ* pTq, SRpcMsg* pMsg) { STqHandle* pHandle = taosHashGet(pTq->pHandle, req.subKey, strlen(req.subKey)); if (pHandle == NULL) { tqError("consumer:0x%" PRIx64 " vgId:%d subkey:%s not found", consumerId, vgId, req.subKey); - terrno = TSDB_CODE_INVALID_MSG; taosRUnLockLatch(&pTq->lock); - return -1; + return TSDB_CODE_INVALID_MSG; } // 2. check rebalance status if (pHandle->consumerId != consumerId) { tqDebug("ERROR consumer:0x%" PRIx64 " vgId:%d, subkey %s, mismatch for saved handle consumer:0x%" PRIx64, consumerId, vgId, req.subKey, pHandle->consumerId); - terrno = TSDB_CODE_TMQ_CONSUMER_MISMATCH; taosRUnLockLatch(&pTq->lock); - return -1; + return TSDB_CODE_TMQ_CONSUMER_MISMATCH; + } int64_t sver = 0, ever = 0; @@ -500,13 +506,15 @@ int32_t tqProcessVgWalInfoReq(STQ* pTq, SRpcMsg* pMsg) { taosRUnLockLatch(&pTq->lock); SMqDataRsp dataRsp = {0}; - tqInitDataRsp(&dataRsp.common, req.reqOffset); + code = tqInitDataRsp(&dataRsp.common, req.reqOffset); + if (code != 0) { + return code; + } if (req.useSnapshot == true) { tqError("consumer:0x%" PRIx64 " vgId:%d subkey:%s snapshot not support wal info", consumerId, vgId, req.subKey); - terrno = TSDB_CODE_INVALID_PARA; - tDeleteMqDataRsp(&dataRsp); - return -1; + code = TSDB_CODE_INVALID_PARA; + goto END; } dataRsp.common.rspOffset.type = TMQ_OFFSET__LOG; @@ -514,13 +522,13 @@ int32_t tqProcessVgWalInfoReq(STQ* pTq, SRpcMsg* pMsg) { if (reqOffset.type == TMQ_OFFSET__LOG) { dataRsp.common.rspOffset.version = reqOffset.version; } else if (reqOffset.type < 0) { - STqOffset* pOffset = (STqOffset*)(STqOffset*)tqMetaGetOffset(pTq, req.subKey); - if (pOffset != NULL) { + STqOffset* pOffset = NULL; + code = tqMetaGetOffset(pTq, req.subKey, &pOffset); + if (code == 0) { if (pOffset->val.type != TMQ_OFFSET__LOG) { tqError("consumer:0x%" PRIx64 " vgId:%d subkey:%s, no valid wal info", consumerId, vgId, req.subKey); - terrno = TSDB_CODE_INVALID_PARA; - tDeleteMqDataRsp(&dataRsp); - return -1; + code = TSDB_CODE_INVALID_PARA; + goto END; } dataRsp.common.rspOffset.version = pOffset->val.version; @@ -538,14 +546,15 @@ int32_t tqProcessVgWalInfoReq(STQ* pTq, SRpcMsg* pMsg) { } else { tqError("consumer:0x%" PRIx64 " vgId:%d subkey:%s invalid offset type:%d", consumerId, vgId, req.subKey, reqOffset.type); - terrno = TSDB_CODE_INVALID_PARA; - tDeleteMqDataRsp(&dataRsp); - return -1; + code = TSDB_CODE_INVALID_PARA; + goto END; } - tqDoSendDataRsp(&pMsg->info, &dataRsp, req.epoch, req.consumerId, TMQ_MSG_TYPE__WALINFO_RSP, sver, ever); + code = tqDoSendDataRsp(&pMsg->info, &dataRsp, req.epoch, req.consumerId, TMQ_MSG_TYPE__WALINFO_RSP, sver, ever); + +END: tDeleteMqDataRsp(&dataRsp); - return 0; + return code; } int32_t tqProcessDeleteSubReq(STQ* pTq, int64_t sversion, char* msg, int32_t msgLen) { @@ -568,9 +577,7 @@ int32_t tqProcessDeleteSubReq(STQ* pTq, int64_t sversion, char* msg, int32_t msg taosMsleep(10); continue; } - tqUnregisterPushHandle(pTq, pHandle); - code = taosHashRemove(pTq->pHandle, pReq->subKey, strlen(pReq->subKey)); if (code != 0) { tqError("cannot process tq delete req %s, since no such handle", pReq->subKey); @@ -598,33 +605,25 @@ int32_t tqProcessDeleteSubReq(STQ* pTq, int64_t sversion, char* msg, int32_t msg int32_t tqProcessAddCheckInfoReq(STQ* pTq, int64_t sversion, char* msg, int32_t msgLen) { STqCheckInfo info = {0}; - if(tqMetaDecodeCheckInfo(&info, msg, msgLen) != 0){ - return -1; + int32_t code = tqMetaDecodeCheckInfo(&info, msg, msgLen); + if(code != 0){ + return code; } - if (taosHashPut(pTq->pCheckInfo, info.topic, strlen(info.topic), &info, sizeof(STqCheckInfo)) < 0) { - terrno = TSDB_CODE_OUT_OF_MEMORY; + code = taosHashPut(pTq->pCheckInfo, info.topic, strlen(info.topic), &info, sizeof(STqCheckInfo)); + if (code != 0) { tDeleteSTqCheckInfo(&info); - return -1; + return code; } - if (tqMetaSaveInfo(pTq, pTq->pCheckStore, info.topic, strlen(info.topic), msg, msgLen) < 0) { - terrno = TSDB_CODE_OUT_OF_MEMORY; - return -1; - } - return 0; + + return tqMetaSaveInfo(pTq, pTq->pCheckStore, info.topic, strlen(info.topic), msg, msgLen); } int32_t tqProcessDelCheckInfoReq(STQ* pTq, int64_t sversion, char* msg, int32_t msgLen) { if (taosHashRemove(pTq->pCheckInfo, msg, strlen(msg)) < 0) { - terrno = TSDB_CODE_OUT_OF_MEMORY; - return -1; + return TSDB_CODE_TSC_INTERNAL_ERROR; } - if (tqMetaDeleteInfo(pTq, pTq->pCheckStore, msg, strlen(msg)) < 0) { - tqError("cannot process tq delete check info req %s, since no such check info", msg); - terrno = TSDB_CODE_OUT_OF_MEMORY; - return -1; - } - return 0; + return tqMetaDeleteInfo(pTq, pTq->pCheckStore, msg, strlen(msg)); } int32_t tqProcessSubscribeReq(STQ* pTq, int64_t sversion, char* msg, int32_t msgLen) { @@ -633,12 +632,10 @@ int32_t tqProcessSubscribeReq(STQ* pTq, int64_t sversion, char* msg, int32_t msg SDecoder dc = {0}; tDecoderInit(&dc, (uint8_t*)msg, msgLen); - + ret = tDecodeSMqRebVgReq(&dc, &req); // decode req - if (tDecodeSMqRebVgReq(&dc, &req) < 0) { - terrno = TSDB_CODE_INVALID_MSG; - tDecoderClear(&dc); - return -1; + if (ret < 0) { + goto end; } tqInfo("vgId:%d, tq process sub req:%s, Id:0x%" PRIx64 " -> Id:0x%" PRIx64, pTq->pVnode->config.vgId, req.subKey, @@ -777,18 +774,14 @@ int32_t tqBuildStreamTask(void* pTqObj, SStreamTask* pTask, int64_t nextProcessV return 0; } -int32_t tqProcessTaskCheckReq(STQ* pTq, SRpcMsg* pMsg) { return tqStreamTaskProcessCheckReq(pTq->pStreamMeta, pMsg); } +int32_t tqProcessTaskCheckReq(STQ* pTq, SRpcMsg* pMsg) { + return tqStreamTaskProcessCheckReq(pTq->pStreamMeta, pMsg); } int32_t tqProcessTaskCheckRsp(STQ* pTq, SRpcMsg* pMsg) { return tqStreamTaskProcessCheckRsp(pTq->pStreamMeta, pMsg, vnodeIsRoleLeader(pTq->pVnode)); } int32_t tqProcessTaskDeployReq(STQ* pTq, int64_t sversion, char* msg, int32_t msgLen) { -// if (!pTq->pVnode->restored) { -// tqDebug("vgId:%d not restored, ignore the stream task deploy msg", TD_VID(pTq->pVnode)); -// return TSDB_CODE_SUCCESS; -// } - return tqStreamTaskProcessDeployReq(pTq->pStreamMeta, &pTq->pVnode->msgCb, sversion, msg, msgLen, vnodeIsRoleLeader(pTq->pVnode), pTq->pVnode->restored); } diff --git a/source/dnode/vnode/src/tq/tqMeta.c b/source/dnode/vnode/src/tq/tqMeta.c index d0519ec5e7..f78084352d 100644 --- a/source/dnode/vnode/src/tq/tqMeta.c +++ b/source/dnode/vnode/src/tq/tqMeta.c @@ -58,12 +58,13 @@ int32_t tDecodeSTqHandle(SDecoder* pDecoder, STqHandle* pHandle) { } else if (pHandle->execHandle.subType == TOPIC_SUB_TYPE__DB) { pHandle->execHandle.execDb.pFilterOutTbUid = taosHashInit(64, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BIGINT), false, HASH_ENTRY_LOCK); + if (pHandle->execHandle.execDb.pFilterOutTbUid == NULL) return -1; int32_t size = 0; if (tDecodeI32(pDecoder, &size) < 0) return -1; for (int32_t i = 0; i < size; i++) { int64_t tbUid = 0; if (tDecodeI64(pDecoder, &tbUid) < 0) return -1; - taosHashPut(pHandle->execHandle.execDb.pFilterOutTbUid, &tbUid, sizeof(int64_t), NULL, 0); + if (taosHashPut(pHandle->execHandle.execDb.pFilterOutTbUid, &tbUid, sizeof(int64_t), NULL, 0) != 0) return -1; } } else if (pHandle->execHandle.subType == TOPIC_SUB_TYPE__TABLE) { if (tDecodeI64(pDecoder, &pHandle->execHandle.execTb.suid) < 0) return -1; @@ -79,11 +80,12 @@ int32_t tqMetaDecodeCheckInfo(STqCheckInfo* info, void* pVal, int32_t vLen) { SDecoder decoder = {0}; tDecoderInit(&decoder, (uint8_t*)pVal, vLen); int32_t code = tDecodeSTqCheckInfo(&decoder, info); + tDecoderClear(&decoder); + if (code != 0) { tDeleteSTqCheckInfo(info); return TSDB_CODE_OUT_OF_MEMORY; } - tDecoderClear(&decoder); return code; } @@ -91,11 +93,12 @@ int32_t tqMetaDecodeOffsetInfo(STqOffset* info, void* pVal, int32_t vLen) { SDecoder decoder = {0}; tDecoderInit(&decoder, (uint8_t*)pVal, vLen); int32_t code = tDecodeSTqOffset(&decoder, info); + tDecoderClear(&decoder); + if (code != 0) { tDeleteSTqOffset(info); return TSDB_CODE_OUT_OF_MEMORY; } - tDecoderClear(&decoder); return code; } @@ -131,32 +134,36 @@ END: return code; } -void* tqMetaGetOffset(STQ* pTq, const char* subkey) { +int32_t tqMetaGetOffset(STQ* pTq, const char* subkey, STqOffset** pOffset){ void* data = taosHashGet(pTq->pOffset, subkey, strlen(subkey)); if (data == NULL) { int vLen = 0; if (tdbTbGet(pTq->pOffsetStore, subkey, strlen(subkey), &data, &vLen) < 0) { tdbFree(data); - return NULL; + return TSDB_CODE_OUT_OF_MEMORY; } STqOffset offset = {0}; if (tqMetaDecodeOffsetInfo(&offset, data, vLen) != TDB_CODE_SUCCESS) { tdbFree(data); - return NULL; + return TSDB_CODE_OUT_OF_MEMORY; } if (taosHashPut(pTq->pOffset, subkey, strlen(subkey), &offset, sizeof(STqOffset)) != 0) { tDeleteSTqOffset(&offset); tdbFree(data); - return NULL; + return TSDB_CODE_OUT_OF_MEMORY; } tdbFree(data); - return taosHashGet(pTq->pOffset, subkey, strlen(subkey)); + *pOffset = taosHashGet(pTq->pOffset, subkey, strlen(subkey)); + if(*pOffset == NULL){ + return TSDB_CODE_OUT_OF_MEMORY; + } } else { - return data; + *pOffset = data; } + return 0; } int32_t tqMetaSaveHandle(STQ* pTq, const char* key, const STqHandle* pHandle) { @@ -199,10 +206,9 @@ static int tqMetaInitHandle(STQ* pTq, STqHandle* handle) { int32_t vgId = TD_VID(pVnode); handle->pRef = walOpenRef(pVnode->pWal); - if (handle->pRef == NULL) { - TAOS_RETURN(TSDB_CODE_OUT_OF_MEMORY); - } - TQ_ERR_RETURN(walSetRefVer(handle->pRef, handle->snapshotVer)); + + TQ_NULL_GO_TO_END(handle->pRef); + TQ_ERR_GO_TO_END(walSetRefVer(handle->pRef, handle->snapshotVer)); SReadHandle reader = { .vnode = pVnode, @@ -214,43 +220,36 @@ static int tqMetaInitHandle(STQ* pTq, STqHandle* handle) { initStorageAPI(&reader.api); if (handle->execHandle.subType == TOPIC_SUB_TYPE__COLUMN) { - handle->execHandle.task = qCreateQueueExecTaskInfo(handle->execHandle.execCol.qmsg, &reader, vgId, - &handle->execHandle.numOfCols, handle->consumerId); - if (handle->execHandle.task == NULL) { - tqError("cannot create exec task for %s", handle->subKey); - return TSDB_CODE_OUT_OF_MEMORY; - } + handle->execHandle.task = + qCreateQueueExecTaskInfo(handle->execHandle.execCol.qmsg, &reader, vgId, &handle->execHandle.numOfCols, handle->consumerId); + TQ_NULL_GO_TO_END(handle->execHandle.task); void* scanner = NULL; - (void)qExtractStreamScanner(handle->execHandle.task, &scanner); - if (scanner == NULL) { - tqError("cannot extract stream scanner for %s", handle->subKey); - return TSDB_CODE_SCH_INTERNAL_ERROR; - } + qExtractStreamScanner(handle->execHandle.task, &scanner); + TQ_NULL_GO_TO_END(scanner); handle->execHandle.pTqReader = qExtractReaderFromStreamScanner(scanner); - if (handle->execHandle.pTqReader == NULL) { - tqError("cannot extract exec reader for %s", handle->subKey); - return TSDB_CODE_SCH_INTERNAL_ERROR; - } + TQ_NULL_GO_TO_END(handle->execHandle.pTqReader); } else if (handle->execHandle.subType == TOPIC_SUB_TYPE__DB) { handle->pWalReader = walOpenReader(pVnode->pWal, NULL, 0); + TQ_NULL_GO_TO_END(handle->pWalReader); handle->execHandle.pTqReader = tqReaderOpen(pVnode); - - buildSnapContext(reader.vnode, reader.version, 0, handle->execHandle.subType, handle->fetchMeta, - (SSnapContext**)(&reader.sContext)); + TQ_NULL_GO_TO_END(handle->execHandle.pTqReader); + TQ_ERR_GO_TO_END(buildSnapContext(reader.vnode, reader.version, 0, handle->execHandle.subType, handle->fetchMeta, + (SSnapContext**)(&reader.sContext))); handle->execHandle.task = qCreateQueueExecTaskInfo(NULL, &reader, vgId, NULL, handle->consumerId); + TQ_NULL_GO_TO_END(handle->execHandle.task); } else if (handle->execHandle.subType == TOPIC_SUB_TYPE__TABLE) { handle->pWalReader = walOpenReader(pVnode->pWal, NULL, 0); - - if (handle->execHandle.execTb.qmsg != NULL && strcmp(handle->execHandle.execTb.qmsg, "") != 0) { + TQ_NULL_GO_TO_END(handle->pWalReader); + if(handle->execHandle.execTb.qmsg != NULL && strcmp(handle->execHandle.execTb.qmsg, "") != 0) { if (nodesStringToNode(handle->execHandle.execTb.qmsg, &handle->execHandle.execTb.node) != 0) { tqError("nodesStringToNode error in sub stable, since %s", terrstr()); return TSDB_CODE_SCH_INTERNAL_ERROR; } } - buildSnapContext(reader.vnode, reader.version, handle->execHandle.execTb.suid, handle->execHandle.subType, - handle->fetchMeta, (SSnapContext**)(&reader.sContext)); + TQ_ERR_GO_TO_END(buildSnapContext(reader.vnode, reader.version, handle->execHandle.execTb.suid, handle->execHandle.subType, + handle->fetchMeta, (SSnapContext**)(&reader.sContext))); handle->execHandle.task = qCreateQueueExecTaskInfo(NULL, &reader, vgId, NULL, handle->consumerId); - + TQ_NULL_GO_TO_END(handle->execHandle.task); SArray* tbUidList = NULL; int ret = qGetTableList(handle->execHandle.execTb.suid, pVnode, handle->execHandle.execTb.node, &tbUidList, handle->execHandle.task); @@ -262,10 +261,13 @@ static int tqMetaInitHandle(STQ* pTq, STqHandle* handle) { tqInfo("vgId:%d, tq try to get ctb for stb subscribe, suid:%" PRId64, pVnode->config.vgId, handle->execHandle.execTb.suid); handle->execHandle.pTqReader = tqReaderOpen(pVnode); + TQ_NULL_GO_TO_END(handle->execHandle.pTqReader); tqReaderSetTbUidList(handle->execHandle.pTqReader, tbUidList, NULL); taosArrayDestroy(tbUidList); } - return 0; + +END: + return code; } static int32_t tqMetaRestoreHandle(STQ* pTq, void* pVal, int vLen, STqHandle* handle) { @@ -287,7 +289,7 @@ END: int32_t tqMetaCreateHandle(STQ* pTq, SMqRebVgReq* req, STqHandle* handle) { int32_t vgId = TD_VID(pTq->pVnode); - memcpy(handle->subKey, req->subKey, TSDB_SUBSCRIBE_KEY_LEN); + (void)memcpy(handle->subKey, req->subKey, TSDB_SUBSCRIBE_KEY_LEN); handle->consumerId = req->newConsumerId; handle->execHandle.subType = req->subType; @@ -297,15 +299,19 @@ int32_t tqMetaCreateHandle(STQ* pTq, SMqRebVgReq* req, STqHandle* handle) { } else if (req->subType == TOPIC_SUB_TYPE__DB) { handle->execHandle.execDb.pFilterOutTbUid = taosHashInit(64, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BIGINT), false, HASH_ENTRY_LOCK); - } else if (req->subType == TOPIC_SUB_TYPE__TABLE) { + if(handle->execHandle.execDb.pFilterOutTbUid == NULL){ + return TSDB_CODE_OUT_OF_MEMORY; + } + }else if(req->subType == TOPIC_SUB_TYPE__TABLE){ handle->execHandle.execTb.suid = req->suid; handle->execHandle.execTb.qmsg = taosStrdup(req->qmsg); } handle->snapshotVer = walGetCommittedVer(pTq->pVnode->pWal); - if (tqMetaInitHandle(pTq, handle) < 0) { - return -1; + int32_t code = tqMetaInitHandle(pTq, handle); + if (code != 0){ + return code; } tqInfo("tqMetaCreateHandle %s consumer 0x%" PRIx64 " vgId:%d, snapshotVer:%" PRId64, handle->subKey, handle->consumerId, vgId, handle->snapshotVer); @@ -377,10 +383,12 @@ END: return code; } -static int32_t replaceTqPath(char** path) { - char* tpath = NULL; - int32_t code = TDB_CODE_SUCCESS; - TQ_ERR_RETURN(tqBuildFName(&tpath, *path, TQ_SUBSCRIBE_NAME)); +static int32_t replaceTqPath(char** path){ + char* tpath = NULL; + int32_t code = tqBuildFName(&tpath, *path, TQ_SUBSCRIBE_NAME); + if (code != 0){ + return code; + } taosMemoryFree(*path); *path = tpath; return TDB_CODE_SUCCESS; @@ -480,7 +488,7 @@ END: return code; } -int32_t tqMetaClose(STQ* pTq) { +void tqMetaClose(STQ* pTq) { if (pTq->pExecStore) { (void)tdbTbClose(pTq->pExecStore); } @@ -491,5 +499,4 @@ int32_t tqMetaClose(STQ* pTq) { (void)tdbTbClose(pTq->pOffsetStore); } (void)tdbClose(pTq->pMetaDB); - return 0; } diff --git a/source/dnode/vnode/src/tq/tqPush.c b/source/dnode/vnode/src/tq/tqPush.c index 7375478e61..386b61cce3 100644 --- a/source/dnode/vnode/src/tq/tqPush.c +++ b/source/dnode/vnode/src/tq/tqPush.c @@ -22,17 +22,28 @@ int32_t tqProcessSubmitReqForSubscribe(STQ* pTq) { } SRpcMsg msg = {.msgType = TDMT_VND_TMQ_CONSUME_PUSH}; msg.pCont = rpcMallocCont(sizeof(SMsgHead)); + if (msg.pCont == NULL) { + return TAOS_GET_TERRNO(TSDB_CODE_OUT_OF_MEMORY); + } msg.contLen = sizeof(SMsgHead); SMsgHead *pHead = msg.pCont; pHead->vgId = TD_VID(pTq->pVnode); pHead->contLen = msg.contLen; - tmsgPutToQueue(&pTq->pVnode->msgCb, QUERY_QUEUE, &msg); - return 0; + int32_t code = tmsgPutToQueue(&pTq->pVnode->msgCb, QUERY_QUEUE, &msg); + if (code != 0){ + tqError("vgId:%d failed to push msg to queue, code:%d", TD_VID(pTq->pVnode), code); + rpcFreeCont(msg.pCont); + } + return code; } int32_t tqPushMsg(STQ* pTq, tmsg_t msgType) { + int32_t code = 0; if (msgType == TDMT_VND_SUBMIT) { - tqProcessSubmitReqForSubscribe(pTq); + code = tqProcessSubmitReqForSubscribe(pTq); + if (code != 0){ + tqError("vgId:%d failed to process submit request for subscribe, code:%d", TD_VID(pTq->pVnode), code); + } } streamMetaRLock(pTq->pStreamMeta); @@ -46,10 +57,10 @@ int32_t tqPushMsg(STQ* pTq, tmsg_t msgType) { // 2. the vnode should be the leader. // 3. the stream is not suspended yet. if ((!tsDisableStream) && (numOfTasks > 0)) { - tqScanWalAsync(pTq, true); + code = tqScanWalAsync(pTq, true); } - return 0; + return code; } int32_t tqRegisterPushHandle(STQ* pTq, void* handle, SRpcMsg* pMsg) { @@ -58,43 +69,52 @@ int32_t tqRegisterPushHandle(STQ* pTq, void* handle, SRpcMsg* pMsg) { if (pHandle->msg == NULL) { pHandle->msg = taosMemoryCalloc(1, sizeof(SRpcMsg)); - memcpy(pHandle->msg, pMsg, sizeof(SRpcMsg)); + if (pHandle->msg == NULL) { + return TAOS_GET_TERRNO(TSDB_CODE_OUT_OF_MEMORY); + } + (void)memcpy(pHandle->msg, pMsg, sizeof(SRpcMsg)); pHandle->msg->pCont = rpcMallocCont(pMsg->contLen); + if (pHandle->msg->pCont == NULL) { + taosMemoryFree(pHandle->msg); + pHandle->msg = NULL; + return TAOS_GET_TERRNO(TSDB_CODE_OUT_OF_MEMORY); + } } else { -// tqPushDataRsp(pHandle, vgId); tqPushEmptyDataRsp(pHandle, vgId); void* tmp = pHandle->msg->pCont; - memcpy(pHandle->msg, pMsg, sizeof(SRpcMsg)); + (void)memcpy(pHandle->msg, pMsg, sizeof(SRpcMsg)); pHandle->msg->pCont = tmp; } - memcpy(pHandle->msg->pCont, pMsg->pCont, pMsg->contLen); + (void)memcpy(pHandle->msg->pCont, pMsg->pCont, pMsg->contLen); pHandle->msg->contLen = pMsg->contLen; int32_t ret = taosHashPut(pTq->pPushMgr, pHandle->subKey, strlen(pHandle->subKey), &pHandle, POINTER_BYTES); tqDebug("vgId:%d data is over, ret:%d, consumerId:0x%" PRIx64 ", register to pHandle:%p, pCont:%p, len:%d", vgId, ret, pHandle->consumerId, pHandle, pHandle->msg->pCont, pHandle->msg->contLen); - return 0; + if (ret != 0) { + rpcFreeCont(pHandle->msg->pCont); + taosMemoryFree(pHandle->msg); + pHandle->msg = NULL; + } + return ret; } -int tqUnregisterPushHandle(STQ* pTq, void *handle) { +void tqUnregisterPushHandle(STQ* pTq, void *handle) { STqHandle *pHandle = (STqHandle*)handle; int32_t vgId = TD_VID(pTq->pVnode); if(taosHashGetSize(pTq->pPushMgr) <= 0) { - return 0; + return; } int32_t ret = taosHashRemove(pTq->pPushMgr, pHandle->subKey, strlen(pHandle->subKey)); tqInfo("vgId:%d remove pHandle:%p,ret:%d consumer Id:0x%" PRIx64, vgId, pHandle, ret, pHandle->consumerId); if(ret == 0 && pHandle->msg != NULL) { -// tqPushDataRsp(pHandle, vgId); tqPushEmptyDataRsp(pHandle, vgId); rpcFreeCont(pHandle->msg->pCont); taosMemoryFree(pHandle->msg); pHandle->msg = NULL; } - - return 0; } diff --git a/source/dnode/vnode/src/tq/tqRead.c b/source/dnode/vnode/src/tq/tqRead.c index 0cdfbdb50a..7c6194a3ea 100644 --- a/source/dnode/vnode/src/tq/tqRead.c +++ b/source/dnode/vnode/src/tq/tqRead.c @@ -27,26 +27,26 @@ bool isValValidForTable(STqHandle* pHandle, SWalCont* pHead) { int64_t tbSuid = pHandle->execHandle.execTb.suid; int64_t realTbSuid = 0; - SDecoder coder; + SDecoder dcoder = {0}; void* data = POINTER_SHIFT(body, sizeof(SMsgHead)); int32_t len = bodyLen - sizeof(SMsgHead); - tDecoderInit(&coder, data, len); + tDecoderInit(&dcoder, data, len); if (msgType == TDMT_VND_CREATE_STB || msgType == TDMT_VND_ALTER_STB) { SVCreateStbReq req = {0}; - if (tDecodeSVCreateStbReq(&coder, &req) < 0) { + if (tDecodeSVCreateStbReq(&dcoder, &req) < 0) { goto end; } realTbSuid = req.suid; } else if (msgType == TDMT_VND_DROP_STB) { SVDropStbReq req = {0}; - if (tDecodeSVDropStbReq(&coder, &req) < 0) { + if (tDecodeSVDropStbReq(&dcoder, &req) < 0) { goto end; } realTbSuid = req.suid; } else if (msgType == TDMT_VND_CREATE_TABLE) { SVCreateTbBatchReq req = {0}; - if (tDecodeSVCreateTbBatchReq(&coder, &req) < 0) { + if (tDecodeSVCreateTbBatchReq(&dcoder, &req) < 0) { goto end; } @@ -66,34 +66,42 @@ bool isValValidForTable(STqHandle* pHandle, SWalCont* pHead) { realTbSuid = tbSuid; SVCreateTbBatchReq reqNew = {0}; reqNew.pArray = taosArrayInit(req.nReqs, sizeof(struct SVCreateTbReq)); + if (reqNew.pArray == NULL) { + tDeleteSVCreateTbBatchReq(&req); + goto end; + } for (int32_t iReq = 0; iReq < req.nReqs; iReq++) { pCreateReq = req.pReqs + iReq; if (pCreateReq->type == TSDB_CHILD_TABLE && pCreateReq->ctb.suid == tbSuid) { reqNew.nReqs++; - taosArrayPush(reqNew.pArray, pCreateReq); + if (taosArrayPush(reqNew.pArray, pCreateReq) == NULL){ + taosArrayDestroy(reqNew.pArray); + tDeleteSVCreateTbBatchReq(&req); + goto end; + } } } - int tlen; + int tlen = 0; int32_t ret = 0; tEncodeSize(tEncodeSVCreateTbBatchReq, &reqNew, tlen, ret); void* buf = taosMemoryMalloc(tlen); if (NULL == buf) { taosArrayDestroy(reqNew.pArray); - for (int32_t iReq = 0; iReq < req.nReqs; iReq++) { - pCreateReq = req.pReqs + iReq; - taosMemoryFreeClear(pCreateReq->comment); - if (pCreateReq->type == TSDB_CHILD_TABLE) { - taosArrayDestroy(pCreateReq->ctb.tagName); - } - } + tDeleteSVCreateTbBatchReq(&req); goto end; } SEncoder coderNew = {0}; tEncoderInit(&coderNew, buf, tlen - sizeof(SMsgHead)); - tEncodeSVCreateTbBatchReq(&coderNew, &reqNew); + ret = tEncodeSVCreateTbBatchReq(&coderNew, &reqNew); tEncoderClear(&coderNew); - memcpy(pHead->body + sizeof(SMsgHead), buf, tlen); + if (ret < 0) { + taosMemoryFree(buf); + taosArrayDestroy(reqNew.pArray); + tDeleteSVCreateTbBatchReq(&req); + goto end; + } + (void)memcpy(pHead->body + sizeof(SMsgHead), buf, tlen); pHead->bodyLen = tlen + sizeof(SMsgHead); taosMemoryFree(buf); taosArrayDestroy(reqNew.pArray); @@ -103,7 +111,7 @@ bool isValValidForTable(STqHandle* pHandle, SWalCont* pHead) { } else if (msgType == TDMT_VND_ALTER_TABLE) { SVAlterTbReq req = {0}; - if (tDecodeSVAlterTbReq(&coder, &req) < 0) { + if (tDecodeSVAlterTbReq(&dcoder, &req) < 0) { goto end; } @@ -119,7 +127,7 @@ bool isValValidForTable(STqHandle* pHandle, SWalCont* pHead) { } else if (msgType == TDMT_VND_DROP_TABLE) { SVDropTbBatchReq req = {0}; - if (tDecodeSVDropTbBatchReq(&coder, &req) < 0) { + if (tDecodeSVDropTbBatchReq(&dcoder, &req) < 0) { goto end; } @@ -140,15 +148,21 @@ bool isValValidForTable(STqHandle* pHandle, SWalCont* pHead) { realTbSuid = tbSuid; SVDropTbBatchReq reqNew = {0}; reqNew.pArray = taosArrayInit(req.nReqs, sizeof(SVDropTbReq)); + if (reqNew.pArray == NULL) { + goto end; + } for (int32_t iReq = 0; iReq < req.nReqs; iReq++) { pDropReq = req.pReqs + iReq; if (pDropReq->suid == tbSuid) { reqNew.nReqs++; - taosArrayPush(reqNew.pArray, pDropReq); + if (taosArrayPush(reqNew.pArray, pDropReq) == NULL){ + taosArrayDestroy(reqNew.pArray); + goto end; + } } } - int tlen; + int tlen = 0; int32_t ret = 0; tEncodeSize(tEncodeSVDropTbBatchReq, &reqNew, tlen, ret); void* buf = taosMemoryMalloc(tlen); @@ -158,23 +172,28 @@ bool isValValidForTable(STqHandle* pHandle, SWalCont* pHead) { } SEncoder coderNew = {0}; tEncoderInit(&coderNew, buf, tlen - sizeof(SMsgHead)); - tEncodeSVDropTbBatchReq(&coderNew, &reqNew); + ret = tEncodeSVDropTbBatchReq(&coderNew, &reqNew); tEncoderClear(&coderNew); - memcpy(pHead->body + sizeof(SMsgHead), buf, tlen); + if (ret != 0) { + taosMemoryFree(buf); + taosArrayDestroy(reqNew.pArray); + goto end; + } + (void)memcpy(pHead->body + sizeof(SMsgHead), buf, tlen); pHead->bodyLen = tlen + sizeof(SMsgHead); taosMemoryFree(buf); taosArrayDestroy(reqNew.pArray); } } else if (msgType == TDMT_VND_DELETE) { SDeleteRes req = {0}; - if (tDecodeDeleteRes(&coder, &req) < 0) { + if (tDecodeDeleteRes(&dcoder, &req) < 0) { goto end; } realTbSuid = req.suid; } end: - tDecoderClear(&coder); + tDecoderClear(&dcoder); return tbSuid == realTbSuid; } @@ -242,10 +261,10 @@ END: bool tqGetTablePrimaryKey(STqReader* pReader) { return pReader->hasPrimaryKey; } -void tqSetTablePrimaryKey(STqReader* pReader, int64_t uid) { - bool ret = false; - SSchemaWrapper* schema = metaGetTableSchema(pReader->pVnodeMeta, uid, -1, 1); - if (schema->nCols >= 2 && schema->pSchema[1].flags & COL_IS_KEY) { +void tqSetTablePrimaryKey(STqReader* pReader, int64_t uid){ + bool ret = false; + SSchemaWrapper *schema = metaGetTableSchema(pReader->pVnodeMeta, uid, -1, 1); + if (schema && schema->nCols >= 2 && schema->pSchema[1].flags & COL_IS_KEY){ ret = true; } tDeleteSchemaWrapper(schema); @@ -333,7 +352,7 @@ int32_t extractMsgFromWal(SWalReader* pReader, void** pItem, int64_t maxVer, con return code; } - memcpy(data, pBody, len); + (void)memcpy(data, pBody, len); SPackedData data1 = (SPackedData){.ver = ver, .msgLen = len, .msgStr = data}; code = streamDataSubmitNew(&data1, STREAM_INPUT__DATA_SUBMIT, (SStreamDataSubmit**)pItem); @@ -379,6 +398,11 @@ bool tqNextBlockInWal(STqReader* pReader, const char* id, int sourceExcluded) { pReader->msg.ver); SSubmitTbData* pSubmitTbData = taosArrayGet(pReader->submit.aSubmitTbData, pReader->nextBlk); + if (pSubmitTbData == NULL) { + tqError("tq reader next data block %d/%d, len:%d %" PRId64, pReader->nextBlk, numOfBlocks, pReader->msg.msgLen, + pReader->msg.ver); + return false; + } if ((pSubmitTbData->flags & sourceExcluded) != 0) { pReader->nextBlk += 1; continue; @@ -412,7 +436,9 @@ bool tqNextBlockInWal(STqReader* pReader, const char* id, int sourceExcluded) { void* pBody = POINTER_SHIFT(pWalReader->pHead->head.body, sizeof(SSubmitReq2Msg)); int32_t bodyLen = pWalReader->pHead->head.bodyLen - sizeof(SSubmitReq2Msg); int64_t ver = pWalReader->pHead->head.version; - tqReaderSetSubmitMsg(pReader, pBody, bodyLen, ver); + if (tqReaderSetSubmitMsg(pReader, pBody, bodyLen, ver) != 0) { + return false; + } pReader->nextBlk = 0; } } @@ -423,13 +449,14 @@ int32_t tqReaderSetSubmitMsg(STqReader* pReader, void* msgStr, int32_t msgLen, i pReader->msg.ver = ver; tqDebug("tq reader set msg %p %d", msgStr, msgLen); - SDecoder decoder; + SDecoder decoder = {0}; tDecoderInit(&decoder, pReader->msg.msgStr, pReader->msg.msgLen); - if (tDecodeSubmitReq(&decoder, &pReader->submit) < 0) { + int32_t code = tDecodeSubmitReq(&decoder, &pReader->submit); + if (code != 0) { tDecoderClear(&decoder); tqError("DecodeSSubmitReq2 error, msgLen:%d, ver:%" PRId64, msgLen, ver); - return -1; + return code; } tDecoderClear(&decoder); @@ -453,6 +480,9 @@ bool tqNextBlockImpl(STqReader* pReader, const char* idstr) { (pReader->nextBlk + 1), numOfBlocks, idstr); SSubmitTbData* pSubmitTbData = taosArrayGet(pReader->submit.aSubmitTbData, pReader->nextBlk); + if (pSubmitTbData == NULL){ + return false; + } if (pReader->tbIdHash == NULL) { return true; } @@ -482,6 +512,7 @@ bool tqNextDataBlockFilterOut(STqReader* pReader, SHashObj* filterOutUids) { int32_t blockSz = taosArrayGetSize(pReader->submit.aSubmitTbData); while (pReader->nextBlk < blockSz) { SSubmitTbData* pSubmitTbData = taosArrayGet(pReader->submit.aSubmitTbData, pReader->nextBlk); + if (pSubmitTbData == NULL) return false; if (filterOutUids == NULL) return true; void* ret = taosHashGet(filterOutUids, &pSubmitTbData->uid, sizeof(int64_t)); @@ -499,7 +530,7 @@ bool tqNextDataBlockFilterOut(STqReader* pReader, SHashObj* filterOutUids) { } int32_t tqMaskBlock(SSchemaWrapper* pDst, SSDataBlock* pBlock, const SSchemaWrapper* pSrc, char* mask) { - int32_t code; + int32_t code = 0; int32_t cnt = 0; for (int32_t i = 0; i < pSrc->nCols; i++) { @@ -509,7 +540,7 @@ int32_t tqMaskBlock(SSchemaWrapper* pDst, SSDataBlock* pBlock, const SSchemaWrap pDst->nCols = cnt; pDst->pSchema = taosMemoryCalloc(cnt, sizeof(SSchema)); if (pDst->pSchema == NULL) { - return -1; + return TAOS_GET_TERRNO(TSDB_CODE_OUT_OF_MEMORY); } int32_t j = 0; @@ -520,7 +551,7 @@ int32_t tqMaskBlock(SSchemaWrapper* pDst, SSDataBlock* pBlock, const SSchemaWrap createColumnInfoData(pSrc->pSchema[i].type, pSrc->pSchema[i].bytes, pSrc->pSchema[i].colId); code = blockDataAppendColInfo(pBlock, &colInfo); if (code != 0) { - return -1; + return code; } } } @@ -556,10 +587,13 @@ static int32_t buildResSDataBlock(SSDataBlock* pBlock, SSchemaWrapper* pSchema, SSchema* pColSchema = &pSchema->pSchema[i]; col_id_t colIdSchema = pColSchema->colId; - col_id_t colIdNeed = *(col_id_t*)taosArrayGet(pColIdList, j); - if (colIdSchema < colIdNeed) { + col_id_t* pColIdNeed = (col_id_t*)taosArrayGet(pColIdList, j); + if (pColIdNeed == NULL) { + break; + } + if (colIdSchema < *pColIdNeed) { i++; - } else if (colIdSchema > colIdNeed) { + } else if (colIdSchema > *pColIdNeed) { j++; } else { SColumnInfoData colInfo = createColumnInfoData(pColSchema->type, pColSchema->bytes, pColSchema->colId); @@ -583,7 +617,7 @@ static int32_t doSetVal(SColumnInfoData* pColumnInfoData, int32_t rowIndex, SCol char val[65535 + 2] = {0}; if (COL_VAL_IS_VALUE(pColVal)) { if (pColVal->value.pData != NULL) { - memcpy(varDataVal(val), pColVal->value.pData, pColVal->value.nData); + (void)memcpy(varDataVal(val), pColVal->value.pData, pColVal->value.nData); } varDataSetLen(val, pColVal->value.nData); code = colDataSetVal(pColumnInfoData, rowIndex, val, false); @@ -599,8 +633,11 @@ static int32_t doSetVal(SColumnInfoData* pColumnInfoData, int32_t rowIndex, SCol int32_t tqRetrieveDataBlock(STqReader* pReader, SSDataBlock** pRes, const char* id) { tqTrace("tq reader retrieve data block %p, index:%d", pReader->msg.msgStr, pReader->nextBlk); + int32_t code = 0; + int32_t line = 0; + STSchema* pTSchema = NULL; SSubmitTbData* pSubmitTbData = taosArrayGet(pReader->submit.aSubmitTbData, pReader->nextBlk++); - + TSDB_CHECK_NULL(pSubmitTbData, code, line, END, terrno); SSDataBlock* pBlock = pReader->pResBlock; *pRes = pBlock; @@ -625,8 +662,7 @@ int32_t tqRetrieveDataBlock(STqReader* pReader, SSDataBlock** pRes, const char* "version %d, possibly dropped table", vgId, suid, uid, pReader->cachedSchemaVer); pReader->cachedSchemaSuid = 0; - terrno = TSDB_CODE_TQ_TABLE_SCHEMA_NOT_FOUND; - return -1; + return TSDB_CODE_TQ_TABLE_SCHEMA_NOT_FOUND; } pReader->cachedSchemaUid = uid; @@ -635,29 +671,23 @@ int32_t tqRetrieveDataBlock(STqReader* pReader, SSDataBlock** pRes, const char* ASSERT(pReader->cachedSchemaVer == pReader->pSchemaWrapper->version); if (blockDataGetNumOfCols(pBlock) == 0) { - int32_t code = buildResSDataBlock(pReader->pResBlock, pReader->pSchemaWrapper, pReader->pColIdList); - if (code != TSDB_CODE_SUCCESS) { - tqError("vgId:%d failed to build data block, code:%s", vgId, tstrerror(code)); - return code; - } + code = buildResSDataBlock(pReader->pResBlock, pReader->pSchemaWrapper, pReader->pColIdList); + TSDB_CHECK_CODE(code, line, END); } } int32_t numOfRows = 0; if (pSubmitTbData->flags & SUBMIT_REQ_COLUMN_DATA_FORMAT) { SColData* pCol = taosArrayGet(pSubmitTbData->aCol, 0); + TSDB_CHECK_NULL(pCol, code, line, END, terrno); numOfRows = pCol->nVal; } else { numOfRows = taosArrayGetSize(pSubmitTbData->aRowP); } - if (blockDataEnsureCapacity(pBlock, numOfRows) < 0) { - terrno = TSDB_CODE_OUT_OF_MEMORY; - return -1; - } - + code = blockDataEnsureCapacity(pBlock, numOfRows); + TSDB_CHECK_CODE(code, line, END); pBlock->info.rows = numOfRows; - int32_t colActual = blockDataGetNumOfCols(pBlock); // convert and scan one block @@ -668,7 +698,7 @@ int32_t tqRetrieveDataBlock(STqReader* pReader, SSDataBlock** pRes, const char* int32_t sourceIdx = 0; while (targetIdx < colActual) { SColumnInfoData* pColData = taosArrayGet(pBlock->pDataBlock, targetIdx); - + TSDB_CHECK_NULL(pColData, code, line, END, terrno); if (sourceIdx >= numOfCols) { tqError("lostdata tqRetrieveDataBlock sourceIdx:%d >= numOfCols:%d", sourceIdx, numOfCols); colDataSetNNULL(pColData, 0, numOfRows); @@ -676,20 +706,17 @@ int32_t tqRetrieveDataBlock(STqReader* pReader, SSDataBlock** pRes, const char* continue; } - SColData* pCol = taosArrayGet(pCols, sourceIdx); - SColVal colVal; - - tqTrace("lostdata colActual:%d, sourceIdx:%d, targetIdx:%d, numOfCols:%d, source cid:%d, dst cid:%d", colActual, - sourceIdx, targetIdx, numOfCols, pCol->cid, pColData->info.colId); + SColData* pCol = taosArrayGet(pCols, sourceIdx); + TSDB_CHECK_NULL(pCol, code, line, END, terrno); + SColVal colVal = {0}; + tqTrace("lostdata colActual:%d, sourceIdx:%d, targetIdx:%d, numOfCols:%d, source cid:%d, dst cid:%d", colActual, sourceIdx, targetIdx, numOfCols, pCol->cid, pColData->info.colId); if (pCol->cid < pColData->info.colId) { sourceIdx++; } else if (pCol->cid == pColData->info.colId) { for (int32_t i = 0; i < pCol->nVal; i++) { tColDataGetValue(pCol, i, &colVal); - int32_t code = doSetVal(pColData, i, &colVal); - if (code != TSDB_CODE_SUCCESS) { - return code; - } + code = doSetVal(pColData, i, &colVal); + TSDB_CHECK_CODE(code, line, END); } sourceIdx++; targetIdx++; @@ -701,26 +728,28 @@ int32_t tqRetrieveDataBlock(STqReader* pReader, SSDataBlock** pRes, const char* } else { SArray* pRows = pSubmitTbData->aRowP; SSchemaWrapper* pWrapper = pReader->pSchemaWrapper; - STSchema* pTSchema = tBuildTSchema(pWrapper->pSchema, pWrapper->nCols, pWrapper->version); + pTSchema = tBuildTSchema(pWrapper->pSchema, pWrapper->nCols, pWrapper->version); + TSDB_CHECK_NULL(pTSchema, code, line, END, terrno); for (int32_t i = 0; i < numOfRows; i++) { SRow* pRow = taosArrayGetP(pRows, i); + TSDB_CHECK_NULL(pRow, code, line, END, terrno); int32_t sourceIdx = 0; - for (int32_t j = 0; j < colActual; j++) { SColumnInfoData* pColData = taosArrayGet(pBlock->pDataBlock, j); + TSDB_CHECK_NULL(pColData, code, line, END, terrno); + while (1) { - SColVal colVal; - tRowGet(pRow, pTSchema, sourceIdx, &colVal); + SColVal colVal = {0}; + code = tRowGet(pRow, pTSchema, sourceIdx, &colVal); + TSDB_CHECK_CODE(code, line, END); + if (colVal.cid < pColData->info.colId) { sourceIdx++; continue; } else if (colVal.cid == pColData->info.colId) { - int32_t code = doSetVal(pColData, i, &colVal); - if (code != TSDB_CODE_SUCCESS) { - return code; - } - + code = doSetVal(pColData, i, &colVal); + TSDB_CHECK_CODE(code, line, END); sourceIdx++; break; } else { @@ -731,17 +760,196 @@ int32_t tqRetrieveDataBlock(STqReader* pReader, SSDataBlock** pRes, const char* } } - taosMemoryFreeClear(pTSchema); } - return 0; +END: + if (code != 0){ + tqError("tqRetrieveDataBlock failed, line:%d, code:%d", line, code); + } + taosMemoryFreeClear(pTSchema); + return code; +} + +#define PROCESS_VAL \ + if (curRow == 0) {\ + assigned[j] = !COL_VAL_IS_NONE(&colVal);\ + buildNew = true;\ + } else {\ + bool currentRowAssigned = !COL_VAL_IS_NONE(&colVal);\ + if (currentRowAssigned != assigned[j]) {\ + assigned[j] = currentRowAssigned;\ + buildNew = true;\ + }\ + } + +#define SET_DATA \ + if (colVal.cid < pColData->info.colId) {\ + sourceIdx++;\ + } else if (colVal.cid == pColData->info.colId) {\ + TQ_ERR_GO_TO_END(doSetVal(pColData, curRow - lastRow, &colVal));\ + sourceIdx++;\ + targetIdx++;\ + } + +static int32_t processBuildNew(STqReader* pReader, SSubmitTbData* pSubmitTbData, SArray* blocks, + SArray* schemas, SSchemaWrapper* pSchemaWrapper, char* assigned, + int32_t numOfRows, int32_t curRow, int32_t* lastRow){ + int32_t code = 0; + SSchemaWrapper* pSW = NULL; + SSDataBlock* block = NULL; + if (taosArrayGetSize(blocks) > 0) { + SSDataBlock* pLastBlock = taosArrayGetLast(blocks); + TQ_NULL_GO_TO_END(pLastBlock); + pLastBlock->info.rows = curRow - *lastRow; + *lastRow = curRow; + } + + block = taosMemoryCalloc(1, sizeof(SSDataBlock)); + TQ_NULL_GO_TO_END(block); + + pSW = taosMemoryCalloc(1, sizeof(SSchemaWrapper)); + TQ_NULL_GO_TO_END(pSW); + + TQ_ERR_GO_TO_END(tqMaskBlock(pSW, block, pSchemaWrapper, assigned)); + tqTrace("vgId:%d, build new block, col %d", pReader->pWalReader->pWal->cfg.vgId, + (int32_t)taosArrayGetSize(block->pDataBlock)); + + block->info.id.uid = pSubmitTbData->uid; + block->info.version = pReader->msg.ver; + TQ_ERR_GO_TO_END(blockDataEnsureCapacity(block, numOfRows - curRow)); + TQ_NULL_GO_TO_END(taosArrayPush(blocks, block)); + TQ_NULL_GO_TO_END(taosArrayPush(schemas, &pSW)); + pSW = NULL; + taosMemoryFreeClear(block); + +END: + tDeleteSchemaWrapper(pSW); + blockDataFreeRes(block); + taosMemoryFree(block); + return code; +} +static int32_t tqProcessColData(STqReader* pReader, SSubmitTbData* pSubmitTbData, SArray* blocks, SArray* schemas){ + int32_t code = 0; + int32_t curRow = 0; + int32_t lastRow = 0; + + SSchemaWrapper* pSchemaWrapper = pReader->pSchemaWrapper; + char* assigned = taosMemoryCalloc(1, pSchemaWrapper->nCols); + TQ_NULL_GO_TO_END(assigned); + + SArray* pCols = pSubmitTbData->aCol; + SColData* pCol = taosArrayGet(pCols, 0); + TQ_NULL_GO_TO_END(pCol); + int32_t numOfRows = pCol->nVal; + int32_t numOfCols = taosArrayGetSize(pCols); + for (int32_t i = 0; i < numOfRows; i++) { + bool buildNew = false; + + for (int32_t j = 0; j < numOfCols; j++) { + pCol = taosArrayGet(pCols, j); + TQ_NULL_GO_TO_END(pCol); + SColVal colVal = {0}; + tColDataGetValue(pCol, i, &colVal); + PROCESS_VAL + } + + if (buildNew) { + TQ_ERR_GO_TO_END(processBuildNew(pReader, pSubmitTbData, blocks, schemas, pSchemaWrapper, assigned, numOfRows, curRow, &lastRow)); + } + + SSDataBlock* pBlock = taosArrayGetLast(blocks); + TQ_NULL_GO_TO_END(pBlock); + + tqTrace("vgId:%d, taosx scan, block num: %d", pReader->pWalReader->pWal->cfg.vgId, + (int32_t)taosArrayGetSize(blocks)); + + int32_t targetIdx = 0; + int32_t sourceIdx = 0; + int32_t colActual = blockDataGetNumOfCols(pBlock); + while (targetIdx < colActual) { + pCol = taosArrayGet(pCols, sourceIdx); + TQ_NULL_GO_TO_END(pCol); + SColumnInfoData* pColData = taosArrayGet(pBlock->pDataBlock, targetIdx); + TQ_NULL_GO_TO_END(pColData); + SColVal colVal = {0}; + tColDataGetValue(pCol, i, &colVal); + SET_DATA + } + + curRow++; + } + SSDataBlock* pLastBlock = taosArrayGetLast(blocks); + pLastBlock->info.rows = curRow - lastRow; + +END: + taosMemoryFree(assigned); + return code; +} + +int32_t tqProcessRowData(STqReader* pReader, SSubmitTbData* pSubmitTbData, SArray* blocks, SArray* schemas){ + int32_t code = 0; + STSchema* pTSchema = NULL; + + SSchemaWrapper* pSchemaWrapper = pReader->pSchemaWrapper; + char* assigned = taosMemoryCalloc(1, pSchemaWrapper->nCols); + TQ_NULL_GO_TO_END(assigned); + + int32_t curRow = 0; + int32_t lastRow = 0; + SArray* pRows = pSubmitTbData->aRowP; + int32_t numOfRows = taosArrayGetSize(pRows); + pTSchema = tBuildTSchema(pSchemaWrapper->pSchema, pSchemaWrapper->nCols, pSchemaWrapper->version); + + for (int32_t i = 0; i < numOfRows; i++) { + bool buildNew = false; + SRow* pRow = taosArrayGetP(pRows, i); + TQ_NULL_GO_TO_END(pRow); + + for (int32_t j = 0; j < pTSchema->numOfCols; j++) { + SColVal colVal = {0}; + TQ_ERR_GO_TO_END(tRowGet(pRow, pTSchema, j, &colVal)); + PROCESS_VAL + } + + if (buildNew) { + TQ_ERR_GO_TO_END(processBuildNew(pReader, pSubmitTbData, blocks, schemas, pSchemaWrapper, assigned, numOfRows, curRow, &lastRow)); + } + + SSDataBlock* pBlock = taosArrayGetLast(blocks); + TQ_NULL_GO_TO_END(pBlock); + + tqTrace("vgId:%d, taosx scan, block num: %d", pReader->pWalReader->pWal->cfg.vgId, + (int32_t)taosArrayGetSize(blocks)); + + int32_t targetIdx = 0; + int32_t sourceIdx = 0; + int32_t colActual = blockDataGetNumOfCols(pBlock); + while (targetIdx < colActual) { + SColumnInfoData* pColData = taosArrayGet(pBlock->pDataBlock, targetIdx); + SColVal colVal = {0}; + TQ_ERR_GO_TO_END(tRowGet(pRow, pTSchema, sourceIdx, &colVal)); + SET_DATA + } + + curRow++; + } + SSDataBlock* pLastBlock = taosArrayGetLast(blocks); + pLastBlock->info.rows = curRow - lastRow; + +END: + taosMemoryFreeClear(pTSchema); + taosMemoryFree(assigned); + return code; } -// todo refactor: int32_t tqRetrieveTaosxBlock(STqReader* pReader, SArray* blocks, SArray* schemas, SSubmitTbData** pSubmitTbDataRet) { tqDebug("tq reader retrieve data block %p, %d", pReader->msg.msgStr, pReader->nextBlk); + SSDataBlock* block = NULL; SSubmitTbData* pSubmitTbData = taosArrayGet(pReader->submit.aSubmitTbData, pReader->nextBlk); + if(pSubmitTbData == NULL){ + return terrno; + } pReader->nextBlk++; if (pSubmitTbDataRet) { @@ -758,248 +966,57 @@ int32_t tqRetrieveTaosxBlock(STqReader* pReader, SArray* blocks, SArray* schemas tqWarn("vgId:%d, cannot found schema wrapper for table: suid:%" PRId64 ", version %d, possibly dropped table", pReader->pWalReader->pWal->cfg.vgId, uid, pReader->cachedSchemaVer); pReader->cachedSchemaSuid = 0; - terrno = TSDB_CODE_TQ_TABLE_SCHEMA_NOT_FOUND; - return -1; + return TSDB_CODE_TQ_TABLE_SCHEMA_NOT_FOUND; } - SSchemaWrapper* pSchemaWrapper = pReader->pSchemaWrapper; - int32_t numOfRows = 0; - if (pSubmitTbData->flags & SUBMIT_REQ_COLUMN_DATA_FORMAT) { - SArray* pCols = pSubmitTbData->aCol; - SColData* pCol = taosArrayGet(pCols, 0); - numOfRows = pCol->nVal; + return tqProcessColData(pReader, pSubmitTbData, blocks, schemas); } else { - SArray* pRows = pSubmitTbData->aRowP; - numOfRows = taosArrayGetSize(pRows); + return tqProcessRowData(pReader, pSubmitTbData, blocks, schemas); } - - int32_t curRow = 0; - int32_t lastRow = 0; - char* assigned = taosMemoryCalloc(1, pSchemaWrapper->nCols); - if (assigned == NULL) return -1; - - // convert and scan one block - if (pSubmitTbData->flags & SUBMIT_REQ_COLUMN_DATA_FORMAT) { - SArray* pCols = pSubmitTbData->aCol; - int32_t numOfCols = taosArrayGetSize(pCols); - for (int32_t i = 0; i < numOfRows; i++) { - bool buildNew = false; - - for (int32_t j = 0; j < numOfCols; j++) { - SColData* pCol = taosArrayGet(pCols, j); - SColVal colVal; - tColDataGetValue(pCol, i, &colVal); - if (curRow == 0) { - assigned[j] = !COL_VAL_IS_NONE(&colVal); - buildNew = true; - } else { - bool currentRowAssigned = !COL_VAL_IS_NONE(&colVal); - if (currentRowAssigned != assigned[j]) { - assigned[j] = currentRowAssigned; - buildNew = true; - } - } - } - - if (buildNew) { - if (taosArrayGetSize(blocks) > 0) { - SSDataBlock* pLastBlock = taosArrayGetLast(blocks); - pLastBlock->info.rows = curRow - lastRow; - lastRow = curRow; - } - - SSDataBlock block = {0}; - SSchemaWrapper* pSW = taosMemoryCalloc(1, sizeof(SSchemaWrapper)); - if (pSW == NULL) { - terrno = TSDB_CODE_OUT_OF_MEMORY; - goto FAIL; - } - - if (tqMaskBlock(pSW, &block, pSchemaWrapper, assigned) < 0) { - blockDataFreeRes(&block); - tDeleteSchemaWrapper(pSW); - goto FAIL; - } - tqTrace("vgId:%d, build new block, col %d", pReader->pWalReader->pWal->cfg.vgId, - (int32_t)taosArrayGetSize(block.pDataBlock)); - - block.info.id.uid = uid; - block.info.version = pReader->msg.ver; - if (blockDataEnsureCapacity(&block, numOfRows - curRow) < 0) { - terrno = TSDB_CODE_OUT_OF_MEMORY; - blockDataFreeRes(&block); - tDeleteSchemaWrapper(pSW); - goto FAIL; - } - taosArrayPush(blocks, &block); - taosArrayPush(schemas, &pSW); - } - - SSDataBlock* pBlock = taosArrayGetLast(blocks); - - tqTrace("vgId:%d, taosx scan, block num: %d", pReader->pWalReader->pWal->cfg.vgId, - (int32_t)taosArrayGetSize(blocks)); - - int32_t targetIdx = 0; - int32_t sourceIdx = 0; - int32_t colActual = blockDataGetNumOfCols(pBlock); - while (targetIdx < colActual) { - SColData* pCol = taosArrayGet(pCols, sourceIdx); - SColumnInfoData* pColData = taosArrayGet(pBlock->pDataBlock, targetIdx); - SColVal colVal; - - if (pCol->cid < pColData->info.colId) { - sourceIdx++; - } else if (pCol->cid == pColData->info.colId) { - tColDataGetValue(pCol, i, &colVal); - if (doSetVal(pColData, curRow - lastRow, &colVal) != TDB_CODE_SUCCESS) { - goto FAIL; - } - sourceIdx++; - targetIdx++; - } - } - - curRow++; - } - } else { - SSchemaWrapper* pWrapper = pReader->pSchemaWrapper; - STSchema* pTSchema = tBuildTSchema(pWrapper->pSchema, pWrapper->nCols, pWrapper->version); - SArray* pRows = pSubmitTbData->aRowP; - - for (int32_t i = 0; i < numOfRows; i++) { - SRow* pRow = taosArrayGetP(pRows, i); - bool buildNew = false; - - for (int32_t j = 0; j < pTSchema->numOfCols; j++) { - SColVal colVal; - tRowGet(pRow, pTSchema, j, &colVal); - if (curRow == 0) { - assigned[j] = !COL_VAL_IS_NONE(&colVal); - buildNew = true; - } else { - bool currentRowAssigned = !COL_VAL_IS_NONE(&colVal); - if (currentRowAssigned != assigned[j]) { - assigned[j] = currentRowAssigned; - buildNew = true; - } - } - } - - if (buildNew) { - if (taosArrayGetSize(blocks) > 0) { - SSDataBlock* pLastBlock = taosArrayGetLast(blocks); - pLastBlock->info.rows = curRow - lastRow; - lastRow = curRow; - } - - SSDataBlock block = {0}; - SSchemaWrapper* pSW = taosMemoryCalloc(1, sizeof(SSchemaWrapper)); - if (pSW == NULL) { - terrno = TSDB_CODE_OUT_OF_MEMORY; - goto FAIL; - } - - if (tqMaskBlock(pSW, &block, pSchemaWrapper, assigned) < 0) { - blockDataFreeRes(&block); - tDeleteSchemaWrapper(pSW); - goto FAIL; - } - tqTrace("vgId:%d, build new block, col %d", pReader->pWalReader->pWal->cfg.vgId, - (int32_t)taosArrayGetSize(block.pDataBlock)); - - block.info.id.uid = uid; - block.info.version = pReader->msg.ver; - if (blockDataEnsureCapacity(&block, numOfRows - curRow) < 0) { - terrno = TSDB_CODE_OUT_OF_MEMORY; - blockDataFreeRes(&block); - tDeleteSchemaWrapper(pSW); - goto FAIL; - } - taosArrayPush(blocks, &block); - taosArrayPush(schemas, &pSW); - } - - SSDataBlock* pBlock = taosArrayGetLast(blocks); - - tqTrace("vgId:%d, taosx scan, block num: %d", pReader->pWalReader->pWal->cfg.vgId, - (int32_t)taosArrayGetSize(blocks)); - - int32_t targetIdx = 0; - int32_t sourceIdx = 0; - int32_t colActual = blockDataGetNumOfCols(pBlock); - while (targetIdx < colActual) { - SColumnInfoData* pColData = taosArrayGet(pBlock->pDataBlock, targetIdx); - SColVal colVal; - tRowGet(pRow, pTSchema, sourceIdx, &colVal); - - if (colVal.cid < pColData->info.colId) { - sourceIdx++; - } else if (colVal.cid == pColData->info.colId) { - if (doSetVal(pColData, curRow - lastRow, &colVal) != TDB_CODE_SUCCESS) { - goto FAIL; - } - sourceIdx++; - targetIdx++; - } - } - curRow++; - } - - taosMemoryFreeClear(pTSchema); - } - - SSDataBlock* pLastBlock = taosArrayGetLast(blocks); - pLastBlock->info.rows = curRow - lastRow; - - taosMemoryFree(assigned); - return 0; - -FAIL: - taosMemoryFree(assigned); - return -1; } void tqReaderSetColIdList(STqReader* pReader, SArray* pColIdList) { pReader->pColIdList = pColIdList; } -int tqReaderSetTbUidList(STqReader* pReader, const SArray* tbUidList, const char* id) { +void tqReaderSetTbUidList(STqReader* pReader, const SArray* tbUidList, const char* id) { if (pReader->tbIdHash) { taosHashClear(pReader->tbIdHash); } else { pReader->tbIdHash = taosHashInit(64, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BIGINT), true, HASH_ENTRY_LOCK); - } - - if (pReader->tbIdHash == NULL) { - terrno = TSDB_CODE_OUT_OF_MEMORY; - return -1; + if (pReader->tbIdHash == NULL) { + tqError("s-task:%s failed to init hash table", id); + return; + } } for (int i = 0; i < taosArrayGetSize(tbUidList); i++) { int64_t* pKey = (int64_t*)taosArrayGet(tbUidList, i); - taosHashPut(pReader->tbIdHash, pKey, sizeof(int64_t), NULL, 0); + if (pKey && taosHashPut(pReader->tbIdHash, pKey, sizeof(int64_t), NULL, 0) != 0) { + tqError("s-task:%s failed to add table uid:%" PRId64 " to hash", id, *pKey); + continue; + } } tqDebug("s-task:%s %d tables are set to be queried target table", id, (int32_t)taosArrayGetSize(tbUidList)); - return 0; } -int tqReaderAddTbUidList(STqReader* pReader, const SArray* pTableUidList) { +void tqReaderAddTbUidList(STqReader* pReader, const SArray* pTableUidList) { if (pReader->tbIdHash == NULL) { pReader->tbIdHash = taosHashInit(64, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BIGINT), true, HASH_ENTRY_LOCK); if (pReader->tbIdHash == NULL) { - terrno = TSDB_CODE_OUT_OF_MEMORY; - return -1; + tqError("failed to init hash table"); + return; } } int32_t numOfTables = taosArrayGetSize(pTableUidList); for (int i = 0; i < numOfTables; i++) { int64_t* pKey = (int64_t*)taosArrayGet(pTableUidList, i); - taosHashPut(pReader->tbIdHash, pKey, sizeof(int64_t), NULL, 0); + if (taosHashPut(pReader->tbIdHash, pKey, sizeof(int64_t), NULL, 0) != 0) { + tqError("failed to add table uid:%" PRId64 " to hash", *pKey); + continue; + } } - - return 0; } bool tqReaderIsQueriedTable(STqReader* pReader, uint64_t uid) { @@ -1008,16 +1025,15 @@ bool tqReaderIsQueriedTable(STqReader* pReader, uint64_t uid) { bool tqCurrentBlockConsumed(const STqReader* pReader) { return pReader->msg.msgStr == NULL; } -int tqReaderRemoveTbUidList(STqReader* pReader, const SArray* tbUidList) { +void tqReaderRemoveTbUidList(STqReader* pReader, const SArray* tbUidList) { for (int32_t i = 0; i < taosArrayGetSize(tbUidList); i++) { int64_t* pKey = (int64_t*)taosArrayGet(tbUidList, i); - taosHashRemove(pReader->tbIdHash, pKey, sizeof(int64_t)); + if (pKey && taosHashRemove(pReader->tbIdHash, pKey, sizeof(int64_t)) != 0){ + tqError("failed to remove table uid:%" PRId64 " from hash", *pKey); + } } - - return 0; } -// todo update the table list in wal reader int32_t tqUpdateTbUidList(STQ* pTq, const SArray* tbUidList, bool isAdd) { void* pIter = NULL; int32_t vgId = TD_VID(pTq->pVnode); @@ -1041,8 +1057,11 @@ int32_t tqUpdateTbUidList(STQ* pTq, const SArray* tbUidList, bool isAdd) { if (!isAdd) { int32_t sz = taosArrayGetSize(tbUidList); for (int32_t i = 0; i < sz; i++) { - int64_t tbUid = *(int64_t*)taosArrayGet(tbUidList, i); - taosHashPut(pTqHandle->execHandle.execDb.pFilterOutTbUid, &tbUid, sizeof(int64_t), NULL, 0); + int64_t* tbUid = (int64_t*)taosArrayGet(tbUidList, i); + if (tbUid && taosHashPut(pTqHandle->execHandle.execDb.pFilterOutTbUid, tbUid, sizeof(int64_t), NULL, 0) != 0){ + tqError("failed to add table uid:%" PRId64 " to hash", *tbUid); + continue; + } } } } else if (pTqHandle->execHandle.subType == TOPIC_SUB_TYPE__TABLE) { diff --git a/source/dnode/vnode/src/tq/tqScan.c b/source/dnode/vnode/src/tq/tqScan.c index b9c9595c74..820a621b1f 100644 --- a/source/dnode/vnode/src/tq/tqScan.c +++ b/source/dnode/vnode/src/tq/tqScan.c @@ -30,8 +30,14 @@ int32_t tqAddBlockDataToRsp(const SSDataBlock* pBlock, void* pRsp, int32_t numOf int32_t actualLen = blockEncode(pBlock, pRetrieve->data, numOfCols); actualLen += sizeof(SRetrieveTableRspForTmq); - taosArrayPush(((SMqDataRspCommon*)pRsp)->blockDataLen, &actualLen); - taosArrayPush(((SMqDataRspCommon*)pRsp)->blockData, &buf); + if (taosArrayPush(((SMqDataRspCommon*)pRsp)->blockDataLen, &actualLen) == NULL){ + taosMemoryFree(buf); + return TSDB_CODE_OUT_OF_MEMORY; + } + if (taosArrayPush(((SMqDataRspCommon*)pRsp)->blockData, &buf) == NULL) { + taosMemoryFree(buf); + return TSDB_CODE_OUT_OF_MEMORY; + } return TSDB_CODE_SUCCESS; } @@ -39,9 +45,11 @@ int32_t tqAddBlockDataToRsp(const SSDataBlock* pBlock, void* pRsp, int32_t numOf static int32_t tqAddBlockSchemaToRsp(const STqExecHandle* pExec, void* pRsp) { SSchemaWrapper* pSW = tCloneSSchemaWrapper(pExec->pTqReader->pSchemaWrapper); if (pSW == NULL) { - return -1; + return TSDB_CODE_OUT_OF_MEMORY; + } + if (taosArrayPush(((SMqDataRspCommon*)pRsp)->blockSchema, &pSW) == NULL) { + return TSDB_CODE_OUT_OF_MEMORY; } - taosArrayPush(((SMqDataRspCommon*)pRsp)->blockSchema, &pSW); return 0; } @@ -49,15 +57,17 @@ static int32_t tqAddTbNameToRsp(const STQ* pTq, int64_t uid, void* pRsp, int32_t SMetaReader mr = {0}; metaReaderDoInit(&mr, pTq->pVnode->pMeta, META_READER_LOCK); - // TODO add reference to gurantee success - if (metaReaderGetTableEntryByUidCache(&mr, uid) < 0) { + int32_t code = metaReaderGetTableEntryByUidCache(&mr, uid); + if (code < 0) { metaReaderClear(&mr); - return -1; + return code; } for (int32_t i = 0; i < n; i++) { char* tbName = taosStrdup(mr.me.name); - taosArrayPush(((SMqDataRspCommon*)pRsp)->blockTbName, &tbName); + if(taosArrayPush(((SMqDataRspCommon*)pRsp)->blockTbName, &tbName) == NULL){ + continue; + } } metaReaderClear(&mr); return 0; @@ -71,8 +81,7 @@ int32_t getDataBlock(qTaskInfo_t task, const STqHandle* pHandle, int32_t vgId, S int32_t code = qExecTask(task, res, &ts); if (code != TSDB_CODE_SUCCESS) { tqError("consumer:0x%" PRIx64 " vgId:%d, task exec error since %s", pHandle->consumerId, vgId, tstrerror(code)); - terrno = code; - return -1; + return code; } tqDebug("consumer:0x%" PRIx64 " vgId:%d tmq one task end executed, pDataBlock:%p", pHandle->consumerId, vgId, *res); @@ -82,22 +91,20 @@ int32_t getDataBlock(qTaskInfo_t task, const STqHandle* pHandle, int32_t vgId, S int32_t tqScanData(STQ* pTq, STqHandle* pHandle, SMqDataRsp* pRsp, STqOffsetVal* pOffset, const SMqPollReq* pRequest) { int32_t vgId = TD_VID(pTq->pVnode); int32_t code = 0; + int32_t line = 0; int32_t totalRows = 0; const STqExecHandle* pExec = &pHandle->execHandle; qTaskInfo_t task = pExec->task; - if (qStreamPrepareScan(task, pOffset, pHandle->execHandle.subType) < 0) { - return -1; - } + code = qStreamPrepareScan(task, pOffset, pHandle->execHandle.subType); + TSDB_CHECK_CODE(code, line, END); qStreamSetSourceExcluded(task, pRequest->sourceExcluded); while (1) { SSDataBlock* pDataBlock = NULL; code = getDataBlock(task, pHandle, vgId, &pDataBlock); - if (code != 0) { - return code; - } + TSDB_CHECK_CODE(code, line, END); if (pRequest->enableReplay) { if (IS_OFFSET_RESET_TYPE(pRequest->reqOffset.type) && pHandle->block != NULL) { @@ -111,27 +118,23 @@ int32_t tqScanData(STQ* pTq, STqHandle* pHandle, SMqDataRsp* pRsp, STqOffsetVal* STqOffsetVal offset = {0}; qStreamExtractOffset(task, &offset); pHandle->block = createOneDataBlock(pDataBlock, true); + TSDB_CHECK_NULL(pDataBlock, code, line, END, terrno); pHandle->blockTime = offset.ts; tOffsetDestroy(&offset); code = getDataBlock(task, pHandle, vgId, &pDataBlock); - if (code != 0) { - return code; - } + TSDB_CHECK_CODE(code, line, END); } code = tqAddBlockDataToRsp(pHandle->block, pRsp, pExec->numOfCols, pTq->pVnode->config.tsdbCfg.precision); - if (code != TSDB_CODE_SUCCESS) { - tqError("vgId:%d, failed to add block to rsp msg", vgId); - return code; - } + TSDB_CHECK_CODE(code, line, END); pRsp->common.blockNum++; if (pDataBlock == NULL) { blockDataDestroy(pHandle->block); pHandle->block = NULL; } else { - copyDataBlock(pHandle->block, pDataBlock); - + code = copyDataBlock(pHandle->block, pDataBlock); + TSDB_CHECK_CODE(code, line, END); STqOffsetVal offset = {0}; qStreamExtractOffset(task, &offset); pRsp->sleepTime = offset.ts - pHandle->blockTime; @@ -144,10 +147,7 @@ int32_t tqScanData(STQ* pTq, STqHandle* pHandle, SMqDataRsp* pRsp, STqOffsetVal* break; } code = tqAddBlockDataToRsp(pDataBlock, pRsp, pExec->numOfCols, pTq->pVnode->config.tsdbCfg.precision); - if (code != TSDB_CODE_SUCCESS) { - tqError("vgId:%d, failed to add block to rsp msg", vgId); - return code; - } + TSDB_CHECK_CODE(code, line, END); pRsp->common.blockNum++; totalRows += pDataBlock->info.rows; @@ -160,15 +160,19 @@ int32_t tqScanData(STQ* pTq, STqHandle* pHandle, SMqDataRsp* pRsp, STqOffsetVal* tqDebug("consumer:0x%" PRIx64 " vgId:%d tmq task executed finished, total blocks:%d, totalRows:%d", pHandle->consumerId, vgId, pRsp->common.blockNum, totalRows); qStreamExtractOffset(task, &pRsp->common.rspOffset); - return 0; +END: + if ( code!= 0){ + tqError("consumer:0x%" PRIx64 " vgId:%d tmq task executed error, line:%d code:%d", pHandle->consumerId, vgId, line, code); + } + return code; } int32_t tqScanTaosx(STQ* pTq, const STqHandle* pHandle, STaosxRsp* pRsp, SMqBatchMetaRsp* pBatchMetaRsp, STqOffsetVal* pOffset) { const STqExecHandle* pExec = &pHandle->execHandle; qTaskInfo_t task = pExec->task; - - if (qStreamPrepareScan(task, pOffset, pHandle->execHandle.subType) < 0) { - return -1; + int code = qStreamPrepareScan(task, pOffset, pHandle->execHandle.subType); + if (code != 0) { + return code; } int32_t rowCnt = 0; @@ -176,11 +180,10 @@ int32_t tqScanTaosx(STQ* pTq, const STqHandle* pHandle, STaosxRsp* pRsp, SMqBatc SSDataBlock* pDataBlock = NULL; uint64_t ts = 0; tqDebug("tmqsnap task start to execute"); - int code = qExecTask(task, &pDataBlock, &ts); + code = qExecTask(task, &pDataBlock, &ts); if (code != 0) { tqError("vgId:%d, task exec error since %s", pTq->pVnode->config.vgId, tstrerror(code)); - terrno = code; - return -1; + return code; } tqDebug("tmqsnap task execute end, get %p", pDataBlock); @@ -190,24 +193,37 @@ int32_t tqScanTaosx(STQ* pTq, const STqHandle* pHandle, STaosxRsp* pRsp, SMqBatc if (pOffset->type == TMQ_OFFSET__LOG) { int64_t uid = pExec->pTqReader->lastBlkUid; if (tqAddTbNameToRsp(pTq, uid, pRsp, 1) < 0) { + tqError("vgId:%d, failed to add tbname to rsp msg", pTq->pVnode->config.vgId); continue; } } else { char* tbName = taosStrdup(qExtractTbnameFromTask(task)); - taosArrayPush(pRsp->common.blockTbName, &tbName); + if (taosArrayPush(pRsp->common.blockTbName, &tbName) == NULL){ + tqError("vgId:%d, failed to add tbname to rsp msg", pTq->pVnode->config.vgId); + continue; + } } } if (pRsp->common.withSchema) { if (pOffset->type == TMQ_OFFSET__LOG) { - tqAddBlockSchemaToRsp(pExec, pRsp); + if (tqAddBlockSchemaToRsp(pExec, pRsp) != 0){ + tqError("vgId:%d, failed to add schema to rsp msg", pTq->pVnode->config.vgId); + continue; + } } else { SSchemaWrapper* pSW = tCloneSSchemaWrapper(qExtractSchemaFromTask(task)); - taosArrayPush(pRsp->common.blockSchema, &pSW); + if(taosArrayPush(pRsp->common.blockSchema, &pSW) == NULL){ + tqError("vgId:%d, failed to add schema to rsp msg", pTq->pVnode->config.vgId); + continue; + } } } - tqAddBlockDataToRsp(pDataBlock, (SMqDataRsp*)pRsp, taosArrayGetSize(pDataBlock->pDataBlock), - pTq->pVnode->config.tsdbCfg.precision); + if (tqAddBlockDataToRsp(pDataBlock, (SMqDataRsp*)pRsp, taosArrayGetSize(pDataBlock->pDataBlock), + pTq->pVnode->config.tsdbCfg.precision) != 0) { + tqError("vgId:%d, failed to add block to rsp msg", pTq->pVnode->config.vgId); + continue; + } pRsp->common.blockNum++; if (pOffset->type == TMQ_OFFSET__LOG) { continue; @@ -222,7 +238,6 @@ int32_t tqScanTaosx(STQ* pTq, const STqHandle* pHandle, STaosxRsp* pRsp, SMqBatc if (taosArrayGetSize(tmp->batchMetaReq) > 0) { qStreamExtractOffset(task, &tmp->rspOffset); *pBatchMetaRsp = *tmp; - tqDebug("tmqsnap task get meta"); break; } @@ -248,150 +263,128 @@ int32_t tqScanTaosx(STQ* pTq, const STqHandle* pHandle, STaosxRsp* pRsp, SMqBatc return 0; } + +static void tqProcessSubData(STQ* pTq, STqHandle* pHandle, STaosxRsp* pRsp, int32_t* totalRows, int8_t sourceExcluded){ + int32_t code = 0; + STqExecHandle* pExec = &pHandle->execHandle; + STqReader* pReader = pExec->pTqReader; + SArray* pBlocks = NULL; + SArray* pSchemas = NULL; + pBlocks = taosArrayInit(0, sizeof(SSDataBlock)); + if (pBlocks == NULL) { + code = terrno; + goto END; + } + pSchemas = taosArrayInit(0, sizeof(void*)); + if(pSchemas == NULL){ + code = terrno; + goto END; + } + + SSubmitTbData* pSubmitTbDataRet = NULL; + code = tqRetrieveTaosxBlock(pReader, pBlocks, pSchemas, &pSubmitTbDataRet); + if (code != 0) { + tqError("vgId:%d, failed to retrieve block", pTq->pVnode->config.vgId); + goto END; + } + + if ((pSubmitTbDataRet->flags & sourceExcluded) != 0) { + goto END; + } + if (pRsp->common.withTbName) { + int64_t uid = pExec->pTqReader->lastBlkUid; + code = tqAddTbNameToRsp(pTq, uid, pRsp, taosArrayGetSize(pBlocks)); + if (code != 0) { + tqError("vgId:%d, failed to add tbname to rsp msg", pTq->pVnode->config.vgId); + goto END; + } + } + if (pHandle->fetchMeta != WITH_DATA && pSubmitTbDataRet->pCreateTbReq != NULL) { + if (pRsp->createTableNum == 0) { + pRsp->createTableLen = taosArrayInit(0, sizeof(int32_t)); + if (pRsp->createTableLen == NULL) { + code = TSDB_CODE_OUT_OF_MEMORY; + goto END; + } + pRsp->createTableReq = taosArrayInit(0, sizeof(void*)); + if (pRsp->createTableReq == NULL) { + code = TSDB_CODE_OUT_OF_MEMORY; + goto END; + } + } + + uint32_t len = 0; + tEncodeSize(tEncodeSVCreateTbReq, pSubmitTbDataRet->pCreateTbReq, len, code); + if (TSDB_CODE_SUCCESS != code) { + goto END; + } + void* createReq = taosMemoryCalloc(1, len); + if (createReq == NULL){ + code = TSDB_CODE_OUT_OF_MEMORY; + goto END; + } + SEncoder encoder = {0}; + tEncoderInit(&encoder, createReq, len); + code = tEncodeSVCreateTbReq(&encoder, pSubmitTbDataRet->pCreateTbReq); + tEncoderClear(&encoder); + if (code < 0) { + taosMemoryFree(createReq); + goto END; + } + if (taosArrayPush(pRsp->createTableLen, &len) == NULL){ + taosMemoryFree(createReq); + goto END; + } + if (taosArrayPush(pRsp->createTableReq, &createReq) == NULL){ + taosMemoryFree(createReq); + goto END; + } + pRsp->createTableNum++; + } + if (pHandle->fetchMeta == ONLY_META && pSubmitTbDataRet->pCreateTbReq == NULL) { + goto END; + } + for (int32_t i = 0; i < taosArrayGetSize(pBlocks); i++) { + SSDataBlock* pBlock = taosArrayGet(pBlocks, i); + if (pBlock == NULL) { + continue; + } + if (tqAddBlockDataToRsp(pBlock, (SMqDataRsp*)pRsp, taosArrayGetSize(pBlock->pDataBlock), + pTq->pVnode->config.tsdbCfg.precision) != 0){ + tqError("vgId:%d, failed to add block to rsp msg", pTq->pVnode->config.vgId); + continue; + } + *totalRows += pBlock->info.rows; + blockDataFreeRes(pBlock); + SSchemaWrapper* pSW = taosArrayGetP(pSchemas, i); + (void) taosArrayPush(pRsp->common.blockSchema, &pSW); + pRsp->common.blockNum++; + } + +END: + taosArrayDestroyEx(pBlocks, (FDelete)blockDataFreeRes); + taosArrayDestroyP(pSchemas, (FDelete)tDeleteSchemaWrapper); +} + int32_t tqTaosxScanLog(STQ* pTq, STqHandle* pHandle, SPackedData submit, STaosxRsp* pRsp, int32_t* totalRows, int8_t sourceExcluded) { STqExecHandle* pExec = &pHandle->execHandle; - SArray* pBlocks = taosArrayInit(0, sizeof(SSDataBlock)); - SArray* pSchemas = taosArrayInit(0, sizeof(void*)); + int32_t code = 0; + STqReader* pReader = pExec->pTqReader; + code = tqReaderSetSubmitMsg(pReader, submit.msgStr, submit.msgLen, submit.ver); + if (code != 0) { + return code; + } if (pExec->subType == TOPIC_SUB_TYPE__TABLE) { - STqReader* pReader = pExec->pTqReader; - tqReaderSetSubmitMsg(pReader, submit.msgStr, submit.msgLen, submit.ver); while (tqNextBlockImpl(pReader, NULL)) { - taosArrayClear(pBlocks); - taosArrayClear(pSchemas); - SSubmitTbData* pSubmitTbDataRet = NULL; - if (tqRetrieveTaosxBlock(pReader, pBlocks, pSchemas, &pSubmitTbDataRet) < 0) { - if (terrno == TSDB_CODE_TQ_TABLE_SCHEMA_NOT_FOUND) goto loop_table; - } - - if ((pSubmitTbDataRet->flags & sourceExcluded) != 0) { - goto loop_table; - } - if (pRsp->common.withTbName) { - int64_t uid = pExec->pTqReader->lastBlkUid; - if (tqAddTbNameToRsp(pTq, uid, pRsp, taosArrayGetSize(pBlocks)) < 0) { - goto loop_table; - } - } - if (pHandle->fetchMeta != WITH_DATA && pSubmitTbDataRet->pCreateTbReq != NULL) { - if (pRsp->createTableNum == 0) { - pRsp->createTableLen = taosArrayInit(0, sizeof(int32_t)); - pRsp->createTableReq = taosArrayInit(0, sizeof(void*)); - } - - int32_t code = TSDB_CODE_SUCCESS; - uint32_t len = 0; - tEncodeSize(tEncodeSVCreateTbReq, pSubmitTbDataRet->pCreateTbReq, len, code); - if (TSDB_CODE_SUCCESS != code) { - goto loop_table; - } - void* createReq = taosMemoryCalloc(1, len); - SEncoder encoder = {0}; - tEncoderInit(&encoder, createReq, len); - code = tEncodeSVCreateTbReq(&encoder, pSubmitTbDataRet->pCreateTbReq); - if (code < 0) { - tEncoderClear(&encoder); - taosMemoryFree(createReq); - goto loop_table; - } - - taosArrayPush(pRsp->createTableLen, &len); - taosArrayPush(pRsp->createTableReq, &createReq); - pRsp->createTableNum++; - - tEncoderClear(&encoder); - } - if (pHandle->fetchMeta == ONLY_META && pSubmitTbDataRet->pCreateTbReq == NULL) { - goto loop_table; - } - for (int32_t i = 0; i < taosArrayGetSize(pBlocks); i++) { - SSDataBlock* pBlock = taosArrayGet(pBlocks, i); - tqAddBlockDataToRsp(pBlock, (SMqDataRsp*)pRsp, taosArrayGetSize(pBlock->pDataBlock), - pTq->pVnode->config.tsdbCfg.precision); - *totalRows += pBlock->info.rows; - blockDataFreeRes(pBlock); - SSchemaWrapper* pSW = taosArrayGetP(pSchemas, i); - taosArrayPush(pRsp->common.blockSchema, &pSW); - pRsp->common.blockNum++; - } - continue; - loop_table: - taosArrayDestroyEx(pBlocks, (FDelete)blockDataFreeRes); - taosArrayDestroyP(pSchemas, (FDelete)tDeleteSchemaWrapper); - pBlocks = taosArrayInit(0, sizeof(SSDataBlock)); - pSchemas = taosArrayInit(0, sizeof(void*)); + tqProcessSubData(pTq, pHandle, pRsp, totalRows, sourceExcluded); } } else if (pExec->subType == TOPIC_SUB_TYPE__DB) { - STqReader* pReader = pExec->pTqReader; - tqReaderSetSubmitMsg(pReader, submit.msgStr, submit.msgLen, submit.ver); while (tqNextDataBlockFilterOut(pReader, pExec->execDb.pFilterOutTbUid)) { - taosArrayClear(pBlocks); - taosArrayClear(pSchemas); - SSubmitTbData* pSubmitTbDataRet = NULL; - if (tqRetrieveTaosxBlock(pReader, pBlocks, pSchemas, &pSubmitTbDataRet) < 0) { - if (terrno == TSDB_CODE_TQ_TABLE_SCHEMA_NOT_FOUND) goto loop_db; - } - - if ((pSubmitTbDataRet->flags & sourceExcluded) != 0) { - goto loop_db; - } - if (pRsp->common.withTbName) { - int64_t uid = pExec->pTqReader->lastBlkUid; - if (tqAddTbNameToRsp(pTq, uid, pRsp, taosArrayGetSize(pBlocks)) < 0) { - goto loop_db; - } - } - if (pHandle->fetchMeta != WITH_DATA && pSubmitTbDataRet->pCreateTbReq != NULL) { - if (pRsp->createTableNum == 0) { - pRsp->createTableLen = taosArrayInit(0, sizeof(int32_t)); - pRsp->createTableReq = taosArrayInit(0, sizeof(void*)); - } - - int32_t code = TSDB_CODE_SUCCESS; - uint32_t len = 0; - tEncodeSize(tEncodeSVCreateTbReq, pSubmitTbDataRet->pCreateTbReq, len, code); - if (TSDB_CODE_SUCCESS != code) { - goto loop_db; - } - void* createReq = taosMemoryCalloc(1, len); - SEncoder encoder = {0}; - tEncoderInit(&encoder, createReq, len); - code = tEncodeSVCreateTbReq(&encoder, pSubmitTbDataRet->pCreateTbReq); - if (code < 0) { - tEncoderClear(&encoder); - taosMemoryFree(createReq); - goto loop_db; - } - - taosArrayPush(pRsp->createTableLen, &len); - taosArrayPush(pRsp->createTableReq, &createReq); - pRsp->createTableNum++; - - tEncoderClear(&encoder); - } - if (pHandle->fetchMeta == ONLY_META && pSubmitTbDataRet->pCreateTbReq == NULL) { - goto loop_db; - } - for (int32_t i = 0; i < taosArrayGetSize(pBlocks); i++) { - SSDataBlock* pBlock = taosArrayGet(pBlocks, i); - tqAddBlockDataToRsp(pBlock, (SMqDataRsp*)pRsp, taosArrayGetSize(pBlock->pDataBlock), - pTq->pVnode->config.tsdbCfg.precision); - *totalRows += pBlock->info.rows; - blockDataFreeRes(pBlock); - SSchemaWrapper* pSW = taosArrayGetP(pSchemas, i); - taosArrayPush(pRsp->common.blockSchema, &pSW); - pRsp->common.blockNum++; - } - continue; - loop_db: - taosArrayDestroyEx(pBlocks, (FDelete)blockDataFreeRes); - taosArrayDestroyP(pSchemas, (FDelete)tDeleteSchemaWrapper); - pBlocks = taosArrayInit(0, sizeof(SSDataBlock)); - pSchemas = taosArrayInit(0, sizeof(void*)); + tqProcessSubData(pTq, pHandle, pRsp, totalRows, sourceExcluded); } } - taosArrayDestroy(pBlocks); - taosArrayDestroy(pSchemas); - return 0; + + return code; } diff --git a/source/dnode/vnode/src/tq/tqUtil.c b/source/dnode/vnode/src/tq/tqUtil.c index edd15098e2..5e92e95105 100644 --- a/source/dnode/vnode/src/tq/tqUtil.c +++ b/source/dnode/vnode/src/tq/tqUtil.c @@ -21,16 +21,15 @@ static int32_t tqSendBatchMetaPollRsp(STqHandle* pHandle, const SRpcMsg* pMsg, c const SMqBatchMetaRsp* pRsp, int32_t vgId); int32_t tqInitDataRsp(SMqDataRspCommon* pRsp, STqOffsetVal pOffset) { - tOffsetCopy(&pRsp->reqOffset, &pOffset); - tOffsetCopy(&pRsp->rspOffset, &pOffset); - pRsp->blockData = taosArrayInit(0, sizeof(void*)); pRsp->blockDataLen = taosArrayInit(0, sizeof(int32_t)); if (pRsp->blockData == NULL || pRsp->blockDataLen == NULL) { - return -1; + return TSDB_CODE_OUT_OF_MEMORY; } + tOffsetCopy(&pRsp->reqOffset, &pOffset); + tOffsetCopy(&pRsp->rspOffset, &pOffset); pRsp->withTbName = 0; pRsp->withSchema = false; return 0; @@ -72,7 +71,7 @@ static int32_t tqInitTaosxRsp(SMqDataRspCommon* pRsp, STqOffsetVal pOffset) { taosArrayDestroy(pRsp->blockSchema); pRsp->blockSchema = NULL; } - return -1; + return TSDB_CODE_OUT_OF_MEMORY; } return 0; @@ -81,12 +80,13 @@ static int32_t tqInitTaosxRsp(SMqDataRspCommon* pRsp, STqOffsetVal pOffset) { static int32_t extractResetOffsetVal(STqOffsetVal* pOffsetVal, STQ* pTq, STqHandle* pHandle, const SMqPollReq* pRequest, SRpcMsg* pMsg, bool* pBlockReturned) { uint64_t consumerId = pRequest->consumerId; - STqOffset* pOffset = (STqOffset*)tqMetaGetOffset(pTq, pRequest->subKey); + STqOffset* pOffset = NULL; + int32_t code = tqMetaGetOffset(pTq, pRequest->subKey, &pOffset); int32_t vgId = TD_VID(pTq->pVnode); *pBlockReturned = false; // In this vnode, data has been polled by consumer for this topic, so let's continue from the last offset value. - if (pOffset != NULL) { + if (code == 0) { tOffsetCopy(pOffsetVal, &pOffset->val); char formatBuf[TSDB_OFFSET_LEN] = {0}; @@ -116,10 +116,13 @@ static int32_t extractResetOffsetVal(STqOffsetVal* pOffsetVal, STQ* pTq, STqHand SMqDataRsp dataRsp = {0}; tqOffsetResetToLog(pOffsetVal, pHandle->pRef->refVer + 1); - tqInitDataRsp(&dataRsp.common, *pOffsetVal); + code = tqInitDataRsp(&dataRsp.common, *pOffsetVal); + if (code != 0){ + return code; + } tqDebug("tmq poll: consumer:0x%" PRIx64 ", subkey %s, vgId:%d, (latest) offset reset to %" PRId64, consumerId, pHandle->subKey, vgId, dataRsp.common.rspOffset.version); - int32_t code = tqSendDataRsp(pHandle, pMsg, pRequest, &dataRsp, TMQ_MSG_TYPE__POLL_DATA_RSP, vgId); + code = tqSendDataRsp(pHandle, pMsg, pRequest, &dataRsp, TMQ_MSG_TYPE__POLL_DATA_RSP, vgId); tDeleteMqDataRsp(&dataRsp); *pBlockReturned = true; @@ -128,8 +131,7 @@ static int32_t extractResetOffsetVal(STqOffsetVal* pOffsetVal, STQ* pTq, STqHand tqError("tmq poll: subkey:%s, no offset committed for consumer:0x%" PRIx64 " in vg %d, subkey %s, reset none failed", pHandle->subKey, consumerId, vgId, pRequest->subKey); - terrno = TSDB_CODE_TQ_NO_COMMITTED_OFFSET; - return -1; + return TSDB_CODE_TQ_NO_COMMITTED_OFFSET; } } @@ -143,10 +145,13 @@ static int32_t extractDataAndRspForNormalSubscribe(STQ* pTq, STqHandle* pHandle, terrno = 0; SMqDataRsp dataRsp = {0}; - tqInitDataRsp(&dataRsp.common, *pOffset); + int code = tqInitDataRsp(&dataRsp.common, *pOffset); + if (code != 0) { + goto end; + } qSetTaskId(pHandle->execHandle.task, consumerId, pRequest->reqId); - int code = tqScanData(pTq, pHandle, &dataRsp, pOffset, pRequest); + code = tqScanData(pTq, pHandle, &dataRsp, pOffset, pRequest); if (code != 0 && terrno != TSDB_CODE_WAL_LOG_NOT_EXIST) { goto end; } @@ -201,17 +206,14 @@ static void tDeleteCommon(void* parm) {} static int32_t extractDataAndRspForDbStbSubscribe(STQ* pTq, STqHandle* pHandle, const SMqPollReq* pRequest, SRpcMsg* pMsg, STqOffsetVal* offset) { - int code = 0; - int32_t vgId = TD_VID(pTq->pVnode); - STaosxRsp taosxRsp = {0}; - SMqBatchMetaRsp btMetaRsp = {0}; - tqInitTaosxRsp(&taosxRsp.common, *offset); + int32_t vgId = TD_VID(pTq->pVnode); + STaosxRsp taosxRsp = {0}; + SMqBatchMetaRsp btMetaRsp = {0}; + int32_t code = 0; + TQ_ERR_GO_TO_END(tqInitTaosxRsp(&taosxRsp.common, *offset)); if (offset->type != TMQ_OFFSET__LOG) { - if (tqScanTaosx(pTq, pHandle, &taosxRsp, &btMetaRsp, offset) < 0) { - code = -1; - goto end; - } + TQ_ERR_GO_TO_END(tqScanTaosx(pTq, pHandle, &taosxRsp, &btMetaRsp, offset)); if (taosArrayGetSize(btMetaRsp.batchMetaReq) > 0) { code = tqSendBatchMetaPollRsp(pHandle, pMsg, pRequest, &btMetaRsp, vgId); @@ -219,8 +221,7 @@ static int32_t extractDataAndRspForDbStbSubscribe(STQ* pTq, STqHandle* pHandle, ",ts:%" PRId64, pRequest->consumerId, pHandle->subKey, vgId, btMetaRsp.rspOffset.type, btMetaRsp.rspOffset.uid, btMetaRsp.rspOffset.ts); - tDeleteMqBatchMetaRsp(&btMetaRsp); - goto end; + goto END; } tqDebug("taosx poll: consumer:0x%" PRIx64 " subkey:%s vgId:%d, send data blockNum:%d, offset type:%d,uid:%" PRId64 @@ -229,7 +230,7 @@ static int32_t extractDataAndRspForDbStbSubscribe(STQ* pTq, STqHandle* pHandle, taosxRsp.common.rspOffset.uid, taosxRsp.common.rspOffset.ts); if (taosxRsp.common.blockNum > 0) { code = tqSendDataRsp(pHandle, pMsg, pRequest, &taosxRsp, TMQ_MSG_TYPE__POLL_DATA_RSP, vgId); - goto end; + goto END; } else { tOffsetCopy(offset, &taosxRsp.common.rspOffset); } @@ -249,15 +250,15 @@ static int32_t extractDataAndRspForDbStbSubscribe(STQ* pTq, STqHandle* pHandle, if (tqFetchLog(pTq, pHandle, &fetchVer, pRequest->reqId) < 0) { if (totalMetaRows > 0) { tqOffsetResetToLog(&btMetaRsp.rspOffset, fetchVer); - tqSendBatchMetaPollRsp(pHandle, pMsg, pRequest, &btMetaRsp, vgId); + code = tqSendBatchMetaPollRsp(pHandle, pMsg, pRequest, &btMetaRsp, vgId); ASSERT(totalRows == 0); - goto end; + goto END; } tqOffsetResetToLog(&taosxRsp.common.rspOffset, fetchVer); code = tqSendDataRsp( pHandle, pMsg, pRequest, &taosxRsp, taosxRsp.createTableNum > 0 ? TMQ_MSG_TYPE__POLL_DATA_META_RSP : TMQ_MSG_TYPE__POLL_DATA_RSP, vgId); - goto end; + goto END; } SWalCont* pHead = &pHandle->pWalReader->pHead->head; @@ -271,7 +272,7 @@ static int32_t extractDataAndRspForDbStbSubscribe(STQ* pTq, STqHandle* pHandle, code = tqSendDataRsp( pHandle, pMsg, pRequest, &taosxRsp, taosxRsp.createTableNum > 0 ? TMQ_MSG_TYPE__POLL_DATA_META_RSP : TMQ_MSG_TYPE__POLL_DATA_RSP, vgId); - goto end; + goto END; } if ((pRequest->sourceExcluded & TD_REQ_FROM_TAOX) != 0) { @@ -295,12 +296,20 @@ static int32_t extractDataAndRspForDbStbSubscribe(STQ* pTq, STqHandle* pHandle, metaRsp.metaRspLen = pHead->bodyLen; metaRsp.metaRsp = pHead->body; code = tqSendMetaPollRsp(pHandle, pMsg, pRequest, &metaRsp, vgId); - goto end; + goto END; } if (!btMetaRsp.batchMetaReq) { btMetaRsp.batchMetaReq = taosArrayInit(4, POINTER_BYTES); + if (btMetaRsp.batchMetaReq == NULL) { + code = TAOS_GET_TERRNO(TSDB_CODE_OUT_OF_MEMORY); + goto END; + } btMetaRsp.batchMetaLen = taosArrayInit(4, sizeof(int32_t)); + if (btMetaRsp.batchMetaLen == NULL) { + code = TAOS_GET_TERRNO(TSDB_CODE_OUT_OF_MEMORY); + goto END; + } } fetchVer++; @@ -316,30 +325,40 @@ static int32_t extractDataAndRspForDbStbSubscribe(STQ* pTq, STqHandle* pHandle, } int32_t tLen = sizeof(SMqRspHead) + len; void* tBuf = taosMemoryCalloc(1, tLen); + if (tBuf == NULL){ + code = TAOS_GET_TERRNO(TSDB_CODE_OUT_OF_MEMORY); + goto END; + } void* metaBuff = POINTER_SHIFT(tBuf, sizeof(SMqRspHead)); SEncoder encoder = {0}; tEncoderInit(&encoder, metaBuff, len); code = tEncodeMqMetaRsp(&encoder, &tmpMetaRsp); + tEncoderClear(&encoder); + if (code < 0) { - tEncoderClear(&encoder); tqError("tmq extract meta from log, tEncodeMqMetaRsp error"); continue; } - taosArrayPush(btMetaRsp.batchMetaReq, &tBuf); - taosArrayPush(btMetaRsp.batchMetaLen, &tLen); - totalMetaRows++; + if (taosArrayPush(btMetaRsp.batchMetaReq, &tBuf) == NULL){ + code = TAOS_GET_TERRNO(TSDB_CODE_OUT_OF_MEMORY); + goto END; + } + if (taosArrayPush(btMetaRsp.batchMetaLen, &tLen) == NULL){ + code = TAOS_GET_TERRNO(TSDB_CODE_OUT_OF_MEMORY); + goto END; + } if ((taosArrayGetSize(btMetaRsp.batchMetaReq) >= tmqRowSize) || (taosGetTimestampMs() - st > 1000)) { tqOffsetResetToLog(&btMetaRsp.rspOffset, fetchVer); - tqSendBatchMetaPollRsp(pHandle, pMsg, pRequest, &btMetaRsp, vgId); - goto end; + code = tqSendBatchMetaPollRsp(pHandle, pMsg, pRequest, &btMetaRsp, vgId); + goto END; } continue; } if (totalMetaRows > 0) { tqOffsetResetToLog(&btMetaRsp.rspOffset, fetchVer); - tqSendBatchMetaPollRsp(pHandle, pMsg, pRequest, &btMetaRsp, vgId); - goto end; + code = tqSendBatchMetaPollRsp(pHandle, pMsg, pRequest, &btMetaRsp, vgId); + goto END; } // process data @@ -353,7 +372,7 @@ static int32_t extractDataAndRspForDbStbSubscribe(STQ* pTq, STqHandle* pHandle, if (code < 0) { tqError("tmq poll: tqTaosxScanLog error %" PRId64 ", in vgId:%d, subkey %s", pRequest->consumerId, vgId, pRequest->subKey); - goto end; + goto END; } if (totalRows >= tmqRowSize || (taosGetTimestampMs() - st > 1000)) { @@ -361,14 +380,14 @@ static int32_t extractDataAndRspForDbStbSubscribe(STQ* pTq, STqHandle* pHandle, code = tqSendDataRsp( pHandle, pMsg, pRequest, &taosxRsp, taosxRsp.createTableNum > 0 ? TMQ_MSG_TYPE__POLL_DATA_META_RSP : TMQ_MSG_TYPE__POLL_DATA_RSP, vgId); - goto end; + goto END; } else { fetchVer++; } } } -end: +END: tDeleteMqBatchMetaRsp(&btMetaRsp); tDeleteSTaosxRsp(&taosxRsp); return code; @@ -423,12 +442,12 @@ int32_t tqSendBatchMetaPollRsp(STqHandle* pHandle, const SRpcMsg* pMsg, const SM int32_t code = 0; tEncodeSize(tEncodeMqBatchMetaRsp, pRsp, len, code); if (code < 0) { - return -1; + return TAOS_GET_TERRNO(code); } int32_t tlen = sizeof(SMqRspHead) + len; void* buf = rpcMallocCont(tlen); if (buf == NULL) { - return -1; + return TAOS_GET_TERRNO(TSDB_CODE_OUT_OF_MEMORY); } int64_t sver = 0, ever = 0; @@ -439,9 +458,12 @@ int32_t tqSendBatchMetaPollRsp(STqHandle* pHandle, const SRpcMsg* pMsg, const SM SEncoder encoder = {0}; tEncoderInit(&encoder, abuf, len); - tEncodeMqBatchMetaRsp(&encoder, pRsp); + code = tEncodeMqBatchMetaRsp(&encoder, pRsp); tEncoderClear(&encoder); - + if (code < 0) { + rpcFreeCont(buf); + return TAOS_GET_TERRNO(code); + } SRpcMsg resp = {.info = pMsg->info, .pCont = buf, .contLen = tlen, .code = 0}; tmsgSendRsp(&resp); @@ -457,12 +479,12 @@ int32_t tqSendMetaPollRsp(STqHandle* pHandle, const SRpcMsg* pMsg, const SMqPoll int32_t code = 0; tEncodeSize(tEncodeMqMetaRsp, pRsp, len, code); if (code < 0) { - return -1; + return TAOS_GET_TERRNO(code); } int32_t tlen = sizeof(SMqRspHead) + len; void* buf = rpcMallocCont(tlen); if (buf == NULL) { - return -1; + return TAOS_GET_TERRNO(TSDB_CODE_OUT_OF_MEMORY); } int64_t sver = 0, ever = 0; @@ -473,8 +495,12 @@ int32_t tqSendMetaPollRsp(STqHandle* pHandle, const SRpcMsg* pMsg, const SMqPoll SEncoder encoder = {0}; tEncoderInit(&encoder, abuf, len); - tEncodeMqMetaRsp(&encoder, pRsp); + code = tEncodeMqMetaRsp(&encoder, pRsp); tEncoderClear(&encoder); + if (code < 0) { + rpcFreeCont(buf); + return TAOS_GET_TERRNO(code); + } SRpcMsg resp = {.info = pMsg->info, .pCont = buf, .contLen = tlen, .code = 0}; @@ -497,13 +523,13 @@ int32_t tqDoSendDataRsp(const SRpcHandleInfo* pRpcHandleInfo, const void* pRsp, } if (code < 0) { - return -1; + return TAOS_GET_TERRNO(code); } int32_t tlen = sizeof(SMqRspHead) + len; void* buf = rpcMallocCont(tlen); if (buf == NULL) { - return -1; + return TSDB_CODE_OUT_OF_MEMORY; } SMqRspHead* pHead = (SMqRspHead*)buf; @@ -515,12 +541,15 @@ int32_t tqDoSendDataRsp(const SRpcHandleInfo* pRpcHandleInfo, const void* pRsp, tEncoderInit(&encoder, abuf, len); if (type == TMQ_MSG_TYPE__POLL_DATA_RSP || type == TMQ_MSG_TYPE__WALINFO_RSP) { - tEncodeMqDataRsp(&encoder, pRsp); + code = tEncodeMqDataRsp(&encoder, pRsp); } else if (type == TMQ_MSG_TYPE__POLL_DATA_META_RSP) { - tEncodeSTaosxRsp(&encoder, pRsp); + code = tEncodeSTaosxRsp(&encoder, pRsp); } - tEncoderClear(&encoder); + if (code < 0) { + rpcFreeCont(buf); + return TAOS_GET_TERRNO(code); + } SRpcMsg rsp = {.info = *pRpcHandleInfo, .pCont = buf, .contLen = tlen, .code = 0}; tmsgSendRsp(&rsp); @@ -528,46 +557,59 @@ int32_t tqDoSendDataRsp(const SRpcHandleInfo* pRpcHandleInfo, const void* pRsp, } int32_t tqExtractDelDataBlock(const void* pData, int32_t len, int64_t ver, void** pRefBlock, int32_t type) { - int32_t code; + int32_t code = 0; + int32_t line = 0; SDecoder* pCoder = &(SDecoder){0}; SDeleteRes* pRes = &(SDeleteRes){0}; *pRefBlock = NULL; pRes->uidList = taosArrayInit(0, sizeof(tb_uid_t)); - if (pRes->uidList == NULL) { - return TSDB_CODE_OUT_OF_MEMORY; - } + TSDB_CHECK_NULL(pRes->uidList, code, line, END, terrno) tDecoderInit(pCoder, (uint8_t*)pData, len); - tDecodeDeleteRes(pCoder, pRes); - tDecoderClear(pCoder); + code = tDecodeDeleteRes(pCoder, pRes); + TSDB_CHECK_CODE(code, line, END); int32_t numOfTables = taosArrayGetSize(pRes->uidList); if (numOfTables == 0 || pRes->affectedRows == 0) { - taosArrayDestroy(pRes->uidList); - return TSDB_CODE_SUCCESS; + goto END; } SSDataBlock* pDelBlock = createSpecialDataBlock(STREAM_DELETE_DATA); - blockDataEnsureCapacity(pDelBlock, numOfTables); + TSDB_CHECK_NULL(pDelBlock, code, line, END, terrno) + code = blockDataEnsureCapacity(pDelBlock, numOfTables); + TSDB_CHECK_CODE(code, line, END); + pDelBlock->info.rows = numOfTables; pDelBlock->info.version = ver; for (int32_t i = 0; i < numOfTables; i++) { // start key column SColumnInfoData* pStartCol = taosArrayGet(pDelBlock->pDataBlock, START_TS_COLUMN_INDEX); - colDataSetVal(pStartCol, i, (const char*)&pRes->skey, false); // end key column + TSDB_CHECK_NULL(pStartCol, code, line, END, terrno) + code = colDataSetVal(pStartCol, i, (const char*)&pRes->skey, false); // end key column + TSDB_CHECK_CODE(code, line, END); SColumnInfoData* pEndCol = taosArrayGet(pDelBlock->pDataBlock, END_TS_COLUMN_INDEX); - colDataSetVal(pEndCol, i, (const char*)&pRes->ekey, false); + TSDB_CHECK_NULL(pEndCol, code, line, END, terrno) + code = colDataSetVal(pEndCol, i, (const char*)&pRes->ekey, false); + TSDB_CHECK_CODE(code, line, END); // uid column SColumnInfoData* pUidCol = taosArrayGet(pDelBlock->pDataBlock, UID_COLUMN_INDEX); - int64_t* pUid = taosArrayGet(pRes->uidList, i); - colDataSetVal(pUidCol, i, (const char*)pUid, false); + TSDB_CHECK_NULL(pUidCol, code, line, END, terrno) - colDataSetNULL(taosArrayGet(pDelBlock->pDataBlock, GROUPID_COLUMN_INDEX), i); - colDataSetNULL(taosArrayGet(pDelBlock->pDataBlock, CALCULATE_START_TS_COLUMN_INDEX), i); - colDataSetNULL(taosArrayGet(pDelBlock->pDataBlock, CALCULATE_END_TS_COLUMN_INDEX), i); + int64_t* pUid = taosArrayGet(pRes->uidList, i); + code = colDataSetVal(pUidCol, i, (const char*)pUid, false); + TSDB_CHECK_CODE(code, line, END); + void* tmp = taosArrayGet(pDelBlock->pDataBlock, GROUPID_COLUMN_INDEX); + TSDB_CHECK_NULL(tmp, code, line, END, terrno) + colDataSetNULL(tmp, i); + tmp = taosArrayGet(pDelBlock->pDataBlock, CALCULATE_START_TS_COLUMN_INDEX); + TSDB_CHECK_NULL(tmp, code, line, END, terrno) + colDataSetNULL(tmp, i); + tmp = taosArrayGet(pDelBlock->pDataBlock, CALCULATE_END_TS_COLUMN_INDEX); + TSDB_CHECK_NULL(tmp, code, line, END, terrno) + colDataSetNULL(tmp, i); } taosArrayDestroy(pRes->uidList); @@ -587,7 +629,13 @@ int32_t tqExtractDelDataBlock(const void* pData, int32_t len, int64_t ver, void* ASSERTS(0, "unknown type:%d", type); } - return TSDB_CODE_SUCCESS; +END: + if (code != 0){ + tqError("failed to extract delete data block, line:%d code:%d", line, code); + } + tDecoderClear(pCoder); + taosArrayDestroy(pRes->uidList); + return code; } int32_t tqGetStreamExecInfo(SVnode* pVnode, int64_t streamId, int64_t* pDelay, bool* fhFinished) { @@ -612,6 +660,9 @@ int32_t tqGetStreamExecInfo(SVnode* pVnode, int64_t streamId, int64_t* pDelay, b for (int32_t i = 0; i < numOfTasks; ++i) { SStreamTaskId* pId = taosArrayGet(pMeta->pTaskList, i); + if (pId == NULL){ + continue; + } if (pId->streamId != streamId) { continue; } diff --git a/source/dnode/vnode/src/vnd/vnodeOpen.c b/source/dnode/vnode/src/vnd/vnodeOpen.c index ea9209c6b4..0911b1046c 100644 --- a/source/dnode/vnode/src/vnd/vnodeOpen.c +++ b/source/dnode/vnode/src/vnd/vnodeOpen.c @@ -458,8 +458,7 @@ SVnode *vnodeOpen(const char *path, int32_t diskPrimary, STfs *pTfs, SMsgCb msgC } // sma required the tq is initialized before the vnode open - pVnode->pTq = tqOpen(tdir, pVnode); - if (pVnode->pTq == NULL) { + if (tqOpen(tdir, pVnode)) { vError("vgId:%d, failed to open vnode tq since %s", TD_VID(pVnode), tstrerror(terrno)); goto _err; } diff --git a/source/libs/executor/src/executor.c b/source/libs/executor/src/executor.c index d5757d2fc3..f8c394db85 100644 --- a/source/libs/executor/src/executor.c +++ b/source/libs/executor/src/executor.c @@ -470,11 +470,7 @@ int32_t qUpdateTableListForStreamScanner(qTaskInfo_t tinfo, const SArray* tableI } int32_t numOfQualifiedTables = taosArrayGetSize(qa); qDebug("%d qualified child tables added into stream scanner, %s", numOfQualifiedTables, id); - code = pTaskInfo->storageAPI.tqReaderFn.tqReaderAddTables(pScanInfo->tqReader, qa); - if (code != TSDB_CODE_SUCCESS) { - taosArrayDestroy(qa); - return code; - } + pTaskInfo->storageAPI.tqReaderFn.tqReaderAddTables(pScanInfo->tqReader, qa); bool assignUid = false; size_t bufLen = (pScanInfo->pGroupTags != NULL) ? getTableTagsBufLen(pScanInfo->pGroupTags) : 0; @@ -528,7 +524,7 @@ int32_t qUpdateTableListForStreamScanner(qTaskInfo_t tinfo, const SArray* tableI } else { // remove the table id in current list qDebug("%d remove child tables from the stream scanner, %s", (int32_t)taosArrayGetSize(tableIdList), id); taosWLockLatch(&pTaskInfo->lock); - code = pTaskInfo->storageAPI.tqReaderFn.tqReaderRemoveTables(pScanInfo->tqReader, tableIdList); + pTaskInfo->storageAPI.tqReaderFn.tqReaderRemoveTables(pScanInfo->tqReader, tableIdList); taosWUnLockLatch(&pTaskInfo->lock); } @@ -958,7 +954,7 @@ int32_t qGetExplainExecInfo(qTaskInfo_t tinfo, SArray* pExecInfoList) { return getOperatorExplainExecInfo(pTaskInfo->pRoot, pExecInfoList); } -int32_t qExtractStreamScanner(qTaskInfo_t tinfo, void** scanner) { +void qExtractStreamScanner(qTaskInfo_t tinfo, void** scanner) { SExecTaskInfo* pTaskInfo = (SExecTaskInfo*)tinfo; SOperatorInfo* pOperator = pTaskInfo->pRoot; @@ -966,7 +962,7 @@ int32_t qExtractStreamScanner(qTaskInfo_t tinfo, void** scanner) { uint16_t type = pOperator->operatorType; if (type == QUERY_NODE_PHYSICAL_PLAN_STREAM_SCAN) { *scanner = pOperator->info; - return 0; + break; } else { ASSERT(pOperator->numOfDownstream == 1); pOperator = pOperator->pDownstream[0]; @@ -1215,7 +1211,7 @@ int32_t qStreamPrepareScan(qTaskInfo_t tinfo, STqOffsetVal* pOffset, int8_t subT if (subType == TOPIC_SUB_TYPE__COLUMN && pOffset->type == TMQ_OFFSET__LOG) { pOperator = extractOperatorInTree(pOperator, QUERY_NODE_PHYSICAL_PLAN_STREAM_SCAN, id); if (pOperator == NULL) { - return -1; + return TAOS_GET_TERRNO(TSDB_CODE_OUT_OF_MEMORY); } SStreamScanInfo* pInfo = pOperator->info; SStoreTqReader* pReaderAPI = &pTaskInfo->storageAPI.tqReaderFn; @@ -1230,7 +1226,7 @@ int32_t qStreamPrepareScan(qTaskInfo_t tinfo, STqOffsetVal* pOffset, int8_t subT if (subType == TOPIC_SUB_TYPE__COLUMN) { pOperator = extractOperatorInTree(pOperator, QUERY_NODE_PHYSICAL_PLAN_STREAM_SCAN, id); if (pOperator == NULL) { - return -1; + return TAOS_GET_TERRNO(TSDB_CODE_OUT_OF_MEMORY); } SStreamScanInfo* pInfo = pOperator->info; @@ -1245,9 +1241,10 @@ int32_t qStreamPrepareScan(qTaskInfo_t tinfo, STqOffsetVal* pOffset, int8_t subT SStoreTqReader* pReaderAPI = &pTaskInfo->storageAPI.tqReaderFn; SWalReader* pWalReader = pReaderAPI->tqReaderGetWalReader(pInfo->tqReader); walReaderVerifyOffset(pWalReader, pOffset); - if (pReaderAPI->tqReaderSeek(pInfo->tqReader, pOffset->version, id) < 0) { + code = pReaderAPI->tqReaderSeek(pInfo->tqReader, pOffset->version, id); + if (code < 0) { qError("tqReaderSeek failed ver:%" PRId64 ", %s", pOffset->version, id); - return -1; + return code; } } else if (pOffset->type == TMQ_OFFSET__SNAPSHOT_DATA) { // iterate all tables from tableInfoList, and retrieve rows from each table one-by-one @@ -1262,15 +1259,14 @@ int32_t qStreamPrepareScan(qTaskInfo_t tinfo, STqOffsetVal* pOffset, int8_t subT if (uid == 0) { if (numOfTables != 0) { - STableKeyInfo* pTableInfo = tableListGetInfo(pTableListInfo, 0); - uid = pTableInfo->uid; + STableKeyInfo* tmp = tableListGetInfo(pTableListInfo, 0); + if (tmp) uid = tmp->uid; ts = INT64_MIN; pScanInfo->currentTable = 0; } else { taosRUnLockLatch(&pTaskInfo->lock); qError("no table in table list, %s", id); - terrno = TSDB_CODE_TMQ_NO_TABLE_QUALIFIED; - return -1; + return TSDB_CODE_TMQ_NO_TABLE_QUALIFIED; } } pTaskInfo->storageAPI.tqReaderFn.tqSetTablePrimaryKey(pInfo->tqReader, uid); @@ -1290,8 +1286,7 @@ int32_t qStreamPrepareScan(qTaskInfo_t tinfo, STqOffsetVal* pOffset, int8_t subT } else { qError("vgId:%d uid:%" PRIu64 " not found in table list, total:%d, index:%d %s", pTaskInfo->id.vgId, uid, numOfTables, pScanInfo->currentTable, id); - terrno = TSDB_CODE_TMQ_NO_TABLE_QUALIFIED; - return -1; + return TSDB_CODE_TMQ_NO_TABLE_QUALIFIED; } STableKeyInfo keyInfo = {.uid = uid}; @@ -1311,8 +1306,7 @@ int32_t qStreamPrepareScan(qTaskInfo_t tinfo, STqOffsetVal* pOffset, int8_t subT (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; - return -1; + return code; } qDebug("tsdb reader created with offset(snapshot) uid:%" PRId64 " ts:%" PRId64 " table index:%d, total:%d, %s", @@ -1337,8 +1331,7 @@ int32_t qStreamPrepareScan(qTaskInfo_t tinfo, STqOffsetVal* pOffset, int8_t subT pScanBaseInfo->cond.twindows.skey = oldSkey; } else { qError("invalid pOffset->type:%d, %s", pOffset->type, id); - terrno = TSDB_CODE_PAR_INTERNAL_ERROR; - return -1; + return TSDB_CODE_PAR_INTERNAL_ERROR; } } else { // subType == TOPIC_SUB_TYPE__TABLE/TOPIC_SUB_TYPE__DB @@ -1351,11 +1344,14 @@ int32_t qStreamPrepareScan(qTaskInfo_t tinfo, STqOffsetVal* pOffset, int8_t subT if (pAPI->snapshotFn.setForSnapShot(sContext, pOffset->uid) != 0) { qError("setDataForSnapShot error. uid:%" PRId64 " , %s", pOffset->uid, id); - terrno = TSDB_CODE_PAR_INTERNAL_ERROR; - return -1; + return TSDB_CODE_PAR_INTERNAL_ERROR; } - SMetaTableInfo mtInfo = pTaskInfo->storageAPI.snapshotFn.getMetaTableInfoFromSnapshot(sContext); + SMetaTableInfo mtInfo = {0}; + code = pTaskInfo->storageAPI.snapshotFn.getMetaTableInfoFromSnapshot(sContext, &mtInfo); + if (code != 0){ + return code; + } pTaskInfo->storageAPI.tsdReader.tsdReaderClose(pInfo->dataReader); pInfo->dataReader = NULL; @@ -1363,6 +1359,7 @@ int32_t qStreamPrepareScan(qTaskInfo_t tinfo, STqOffsetVal* pOffset, int8_t subT tableListClear(pTableListInfo); if (mtInfo.uid == 0) { + tDeleteSchemaWrapper(mtInfo.schema); goto end; // no data } @@ -1370,6 +1367,7 @@ int32_t qStreamPrepareScan(qTaskInfo_t tinfo, STqOffsetVal* pOffset, int8_t subT code = initQueryTableDataCondForTmq(&pTaskInfo->streamInfo.tableCond, sContext, &mtInfo); if (code != TSDB_CODE_SUCCESS) { qError("%s failed at line %d since %s", __func__, __LINE__, tstrerror(code)); + tDeleteSchemaWrapper(mtInfo.schema); return code; } if (pAPI->snapshotFn.taosXGetTablePrimaryKey(sContext)) { @@ -1380,6 +1378,7 @@ int32_t qStreamPrepareScan(qTaskInfo_t tinfo, STqOffsetVal* pOffset, int8_t subT code = tableListAddTableInfo(pTableListInfo, mtInfo.uid, 0); if (code != TSDB_CODE_SUCCESS) { + tDeleteSchemaWrapper(mtInfo.schema); qError("%s failed at line %d since %s", __func__, __LINE__, tstrerror(code)); return code; } @@ -1391,11 +1390,12 @@ int32_t qStreamPrepareScan(qTaskInfo_t tinfo, STqOffsetVal* pOffset, int8_t subT NULL, (void**)&pInfo->dataReader, NULL, NULL); if (code != TSDB_CODE_SUCCESS) { qError("%s failed at line %d since %s", __func__, __LINE__, tstrerror(code)); + tDeleteSchemaWrapper(mtInfo.schema); return code; } cleanupQueryTableDataCond(&pTaskInfo->streamInfo.tableCond); - strcpy(pTaskInfo->streamInfo.tbName, mtInfo.tbName); + tstrncpy(pTaskInfo->streamInfo.tbName, mtInfo.tbName, TSDB_TABLE_NAME_LEN); tDeleteSchemaWrapper(pTaskInfo->streamInfo.schema); pTaskInfo->streamInfo.schema = mtInfo.schema; @@ -1403,10 +1403,10 @@ int32_t qStreamPrepareScan(qTaskInfo_t tinfo, STqOffsetVal* pOffset, int8_t subT } else if (pOffset->type == TMQ_OFFSET__SNAPSHOT_META) { SStreamRawScanInfo* pInfo = pOperator->info; SSnapContext* sContext = pInfo->sContext; - if (pTaskInfo->storageAPI.snapshotFn.setForSnapShot(sContext, pOffset->uid) != 0) { + code = pTaskInfo->storageAPI.snapshotFn.setForSnapShot(sContext, pOffset->uid); + if (code != 0) { qError("setForSnapShot error. uid:%" PRIu64 " ,version:%" PRId64, pOffset->uid, pOffset->version); - terrno = TSDB_CODE_PAR_INTERNAL_ERROR; - return -1; + return code; } qDebug("tmqsnap qStreamPrepareScan snapshot meta uid:%" PRId64 " ts %" PRId64 " %s", pOffset->uid, pOffset->ts, id); diff --git a/source/libs/executor/src/scanoperator.c b/source/libs/executor/src/scanoperator.c index 20c88cbdb4..de0ff0e595 100644 --- a/source/libs/executor/src/scanoperator.c +++ b/source/libs/executor/src/scanoperator.c @@ -3394,7 +3394,9 @@ static SSDataBlock* doRawScan(SOperatorInfo* pOperator) { } } - SMetaTableInfo mtInfo = pAPI->snapshotFn.getMetaTableInfoFromSnapshot(pInfo->sContext); + SMetaTableInfo mtInfo = {0}; + code = pAPI->snapshotFn.getMetaTableInfoFromSnapshot(pInfo->sContext, &mtInfo); + QUERY_CHECK_CODE(code, lino, _end); STqOffsetVal offset = {0}; if (mtInfo.uid == 0 || pInfo->sContext->withMeta == ONLY_META) { // read snapshot done, change to get data from wal qDebug("tmqsnap read snapshot done, change to get data from wal"); @@ -3404,9 +3406,9 @@ static SSDataBlock* doRawScan(SOperatorInfo* pOperator) { tqOffsetResetToData(&offset, mtInfo.uid, INT64_MIN, val); qDebug("tmqsnap change get data uid:%" PRId64 "", mtInfo.uid); } + tDeleteSchemaWrapper(mtInfo.schema); code = qStreamPrepareScan(pTaskInfo, &offset, pInfo->sContext->subType); QUERY_CHECK_CODE(code, lino, _end); - tDeleteSchemaWrapper(mtInfo.schema); return NULL; } else if (pTaskInfo->streamInfo.currentOffset.type == TMQ_OFFSET__SNAPSHOT_META) { SSnapContext* sContext = pInfo->sContext; @@ -3441,7 +3443,6 @@ static SSDataBlock* doRawScan(SOperatorInfo* pOperator) { pTaskInfo->streamInfo.btMetaRsp.batchMetaLen = taosArrayInit(4, sizeof(int32_t)); QUERY_CHECK_NULL(pTaskInfo->streamInfo.btMetaRsp.batchMetaLen, code, lino, _end, TSDB_CODE_OUT_OF_MEMORY); } - int32_t code = TSDB_CODE_SUCCESS; uint32_t len = 0; tEncodeSize(tEncodeMqMetaRsp, &tmpMetaRsp, len, code); if (TSDB_CODE_SUCCESS != code) { @@ -3486,7 +3487,7 @@ _end: static void destroyRawScanOperatorInfo(void* param) { SStreamRawScanInfo* pRawScan = (SStreamRawScanInfo*)param; pRawScan->pAPI->tsdReader.tsdReaderClose(pRawScan->dataReader); - (void)pRawScan->pAPI->snapshotFn.destroySnapshot(pRawScan->sContext); + pRawScan->pAPI->snapshotFn.destroySnapshot(pRawScan->sContext); tableListDestroy(pRawScan->pTableListInfo); taosMemoryFree(pRawScan); } @@ -3811,13 +3812,7 @@ SOperatorInfo* createStreamScanOperatorInfo(SReadHandle* pHandle, STableScanPhys SArray* tableIdList = NULL; code = extractTableIdList(((STableScanInfo*)(pInfo->pTableScanOp->info))->base.pTableListInfo, &tableIdList); QUERY_CHECK_CODE(code, lino, _error); - - code = pAPI->tqReaderFn.tqReaderSetQueryTableList(pInfo->tqReader, tableIdList, idstr); - if (code != 0) { - taosArrayDestroy(tableIdList); - goto _error; - } - + pAPI->tqReaderFn.tqReaderSetQueryTableList(pInfo->tqReader, tableIdList, idstr); taosArrayDestroy(tableIdList); memcpy(&pTaskInfo->streamInfo.tableCond, &pTSInfo->base.cond, sizeof(SQueryTableDataCond)); } else {