feat(stream): support stream processing for virtual tables (#30144)

* enh: add client processing

* enh: add mnode vtables processing

* enh: add mnode vtable processing

* enh: add normal child vtable support

* fix: compile issues

* fix: compile issues

* fix: create stream issues

* fix: multi stream scan issue

* fix: remove debug info

* fix: agg task and task level issues

* fix: correct task output type

* fix: split vtablescan from agg

* fix: memory leak issues

* fix: add limitations

* Update 09-error-code.md

* Update 09-error-code.md

* fix: remove usless case

* feat(stream): extract original table data in source scan task

Implemented functionality in the source task to extract data
corresponding to the virtual table from the original table using WAL.
The extracted data is then sent to the downstream merge task for further
processing.

* feat(stream): multi-way merge using loser tree in virtual merge task

Implemented multi-way merge in the merge task using a loser tree to
combine data from multiple original table into a single virtual table.
The merged virtual table data is then pushed downstream for further
processing.  Introduced memory limit handling during the merge process
with configurable behavior when the memory limit is reached.

* fix(test): remove useless cases

---------

Co-authored-by: dapan1121 <wpan@taosdata.com>
Co-authored-by: Pan Wei <72057773+dapan1121@users.noreply.github.com>
This commit is contained in:
Jinqing Kuang 2025-03-15 17:06:48 +08:00 committed by GitHub
parent 6913d271d9
commit ac23a4fc55
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
76 changed files with 4467 additions and 1151 deletions

View File

@ -111,6 +111,7 @@ This document details the server error codes that may be encountered when using
| 0x8000030C | Invalid query id | Internal error | Report issue |
| 0x8000030E | Invalid connection id | Internal error | Report issue |
| 0x80000315 | User is disabled | User is unavailable | Grant permissions |
| 0x80000318 | Mnode internal error | Internal error | Report issue |
| 0x80000320 | Object already there | Internal error | Report issue |
| 0x80000322 | Invalid table type | Internal error | Report issue |
| 0x80000323 | Object not there | Internal error | Report issue |

View File

@ -117,6 +117,7 @@ description: TDengine 服务端的错误码列表和详细说明
| 0x8000030C | Invalid query id | 内部错误 | 上报 issue |
| 0x8000030E | Invalid connection id | 内部错误 | 上报 issue |
| 0x80000315 | User is disabled | 该用户不可用 | 赋权 |
| 0x80000318 | Mnode internal error | 内部错误 | 上报 issue |
| 0x80000320 | Object already there | 内部错误 | 上报 issue |
| 0x80000322 | Invalid table type | 内部错误 | 上报 issue |
| 0x80000323 | Object not there | 内部错误 | 上报 issue |

View File

@ -302,6 +302,9 @@ extern int8_t tsS3EpNum;
extern int32_t tsStreamNotifyMessageSize;
extern int32_t tsStreamNotifyFrameSize;
extern bool tsCompareAsStrInGreatest;
extern int32_t tsStreamVirtualMergeMaxDelayMs;
extern int32_t tsStreamVirtualMergeMaxMemKb;
extern int32_t tsStreamVirtualMergeWaitMode;
extern char tsAdapterFqdn[];
extern uint16_t tsAdapterPort;

View File

@ -624,6 +624,35 @@ typedef struct {
SColRef* pColRef;
} SColRefWrapper;
typedef struct {
int32_t vgId;
SColRef colRef;
} SColRefEx;
typedef struct {
int16_t colId;
char refDbName[TSDB_DB_NAME_LEN];
char refTableName[TSDB_TABLE_NAME_LEN];
char refColName[TSDB_COL_NAME_LEN];
} SRefColInfo;
typedef struct SVCTableRefCols {
uint64_t uid;
int32_t numOfSrcTbls;
int32_t numOfColRefs;
SRefColInfo* refCols;
} SVCTableRefCols;
typedef struct SVCTableMergeInfo {
uint64_t uid;
int32_t numOfSrcTbls;
} SVCTableMergeInfo;
typedef struct {
int32_t nCols;
SColRefEx* pColRefEx;
} SColRefExWrapper;
struct SSchema {
int8_t type;
int8_t flags;
@ -1421,6 +1450,24 @@ int32_t tSerializeSTableCfgRsp(void* buf, int32_t bufLen, STableCfgRsp* pRsp);
int32_t tDeserializeSTableCfgRsp(void* buf, int32_t bufLen, STableCfgRsp* pRsp);
void tFreeSTableCfgRsp(STableCfgRsp* pRsp);
typedef struct {
SMsgHead header;
tb_uid_t suid;
} SVSubTablesReq;
int32_t tSerializeSVSubTablesReq(void *buf, int32_t bufLen, SVSubTablesReq *pReq);
int32_t tDeserializeSVSubTablesReq(void *buf, int32_t bufLen, SVSubTablesReq *pReq);
typedef struct {
int32_t vgId;
SArray* pTables; //SArray<SVCTableRefCols*>
} SVSubTablesRsp;
int32_t tSerializeSVSubTablesRsp(void *buf, int32_t bufLen, SVSubTablesRsp *pRsp);
int32_t tDeserializeSVSubTablesRsp(void *buf, int32_t bufLen, SVSubTablesRsp *pRsp);
void tDestroySVSubTablesRsp(void* rsp);
typedef struct {
char db[TSDB_DB_FNAME_LEN];
int32_t numOfVgroups;
@ -3126,6 +3173,7 @@ typedef struct {
char pWendName[TSDB_COL_NAME_LEN];
char pGroupIdName[TSDB_COL_NAME_LEN];
char pIsWindowFilledName[TSDB_COL_NAME_LEN];
SArray* pVSubTables; // array of SVSubTablesRsp
} SCMCreateStreamReq;
typedef struct STaskNotifyEventStat {

View File

@ -323,6 +323,7 @@
TD_DEF_MSG_TYPE(TDMT_VND_ARB_CHECK_SYNC, "vnode-arb-check-sync", NULL, NULL)
TD_DEF_MSG_TYPE(TDMT_VND_FETCH_TTL_EXPIRED_TBS, "vnode-fetch-ttl-expired-tbs", NULL, NULL)
TD_DEF_MSG_TYPE(TDMT_VND_TABLE_NAME, "vnode-table-name", NULL, NULL)
TD_DEF_MSG_TYPE(TDMT_VND_VSUBTABLES_META, "vnode-virtual_stables-meta", NULL, NULL)
TD_CLOSE_MSG_SEG(TDMT_VND_MSG)
TD_NEW_MSG_SEG(TDMT_SCH_MSG) // 3<<8

View File

@ -98,6 +98,7 @@ typedef struct SCatalogReq {
SArray* pTableTSMAs; // element is STablesReq
SArray* pTSMAs; // element is STablesReq
SArray* pTableName; // element is STablesReq
SArray* pVSubTable; // element is SName
bool qNodeRequired; // valid qnode
bool dNodeRequired; // valid dnode
bool svrVerRequired;
@ -129,6 +130,7 @@ typedef struct SMetaData {
SArray* pView; // pRes = SViewMeta*
SArray* pTableTsmas; // pRes = SArray<STableTSMAInfo*>
SArray* pTsmas; // pRes = SArray<STableTSMAInfo*>
SArray* pVSubTables; // pRes = SVSubTablesRsp
SMetaRes* pSvrVer; // pRes = char*
} SMetaData;

View File

@ -240,8 +240,8 @@ typedef struct SStoreTqReader {
int64_t (*tqGetResultBlockTime)();
int32_t (*tqGetStreamExecProgress)();
void (*tqReaderSetColIdList)();
void (*tqReaderSetQueryTableList)();
int32_t (*tqReaderSetColIdList)();
int32_t (*tqReaderSetQueryTableList)();
void (*tqReaderAddTables)();
void (*tqReaderRemoveTables)();
@ -256,6 +256,8 @@ typedef struct SStoreTqReader {
int32_t (*tqReaderSetSubmitMsg)(); // todo remove it
// bool (*tqReaderNextBlockFilterOut)();
int32_t (*tqReaderSetVtableInfo)();
} SStoreTqReader;
typedef struct SStoreSnapshotFn {

View File

@ -176,6 +176,7 @@ void nodesSortList(SNodeList** pList, int32_t (*)(SNode* pNode1, SNode* pNode
void destroyFuncParam(void* pFuncStruct);
int32_t nodesListDeduplicate(SNodeList** pList);
#ifdef __cplusplus
}
#endif

View File

@ -500,6 +500,10 @@ typedef struct SVirtualScanPhysiNode {
bool groupSort;
bool scanAllCols;
SNodeList* pTargets;
SNodeList* pTags;
SNode* pSubtable;
int8_t igExpired;
int8_t igCheckUpdate;
}SVirtualScanPhysiNode;
typedef struct SLastRowScanPhysiNode {
@ -896,6 +900,7 @@ typedef struct SSubplan {
SDataSinkNode* pDataSink; // data of the subplan flow into the datasink
SNode* pTagCond;
SNode* pTagIndexCond;
SSHashObj* pVTables; // for stream virtual tables
bool showRewrite;
bool isView;
bool isAudit;

View File

@ -29,7 +29,7 @@ extern "C" {
#define TABLE_TOTAL_COL_NUM(pMeta) ((pMeta)->tableInfo.numOfColumns + (pMeta)->tableInfo.numOfTags)
#define TABLE_META_SIZE(pMeta) \
(NULL == (pMeta) ? 0 : (sizeof(STableMeta) + TABLE_TOTAL_COL_NUM((pMeta)) * sizeof(SSchema)))
(NULL == (pMeta) ? 0 : (sizeof(STableMeta) + TABLE_TOTAL_COL_NUM((pMeta)) * sizeof(SSchema) + (pMeta)->numOfColRefs * sizeof(SColRef)))
#define VGROUPS_INFO_SIZE(pInfo) \
(NULL == (pInfo) ? 0 : (sizeof(SVgroupsInfo) + (pInfo)->numOfVgroups * sizeof(SVgroupInfo)))

View File

@ -138,7 +138,7 @@ bool qParseDbName(const char* pStr, size_t length, char** pDbName);
// for async mode
int32_t qParseSqlSyntax(SParseContext* pCxt, SQuery** pQuery, struct SCatalogReq* pCatalogReq);
int32_t qAnalyseSqlSemantic(SParseContext* pCxt, const struct SCatalogReq* pCatalogReq,
const struct SMetaData* pMetaData, SQuery* pQuery);
struct SMetaData* pMetaData, SQuery* pQuery);
int32_t qContinueParseSql(SParseContext* pCxt, struct SCatalogReq* pCatalogReq, const struct SMetaData* pMetaData,
SQuery* pQuery);
int32_t qContinueParsePostQuery(SParseContext* pCxt, SQuery* pQuery, SSDataBlock* pBlock);

View File

@ -164,6 +164,34 @@ typedef struct SDBVgInfo {
SArray* vgArray; // SVgroupInfo
} SDBVgInfo;
typedef struct SVGroupHashInfo {
int32_t vgId;
uint32_t hashBegin;
uint32_t hashEnd;
} SVGroupHashInfo;
typedef struct SDBVgHashInfo {
int16_t hashPrefix;
int16_t hashSuffix;
int8_t hashMethod;
bool vgSorted;
SArray* vgArray; //SArray<SVGroupHashInfo>
} SDBVgHashInfo;
typedef struct SColIdName {
int16_t colId;
char* colName;
} SColIdName;
typedef struct SStreamVBuildCtx {
int64_t lastUid;
SRefColInfo* lastCol;
SSHashObj* lastVg;
SSHashObj* lastVtable;
SArray* lastOtable;
} SStreamVBuildCtx;
typedef struct SUseDbOutput {
char db[TSDB_DB_FNAME_LEN];
uint64_t dbId;
@ -372,6 +400,10 @@ int32_t cloneSVreateTbReq(SVCreateTbReq* pSrc, SVCreateTbReq** pDst);
void freeVgInfo(SDBVgInfo* vgInfo);
void freeDbCfgInfo(SDbCfgInfo* pInfo);
void tFreeStreamVtbOtbInfo(void* param);
void tFreeStreamVtbVtbInfo(void* param);
void tFreeStreamVtbDbVgInfo(void* param);
extern int32_t (*queryBuildMsg[TDMT_MAX])(void* input, char** msg, int32_t msgSize, int32_t* msgLen,
void* (*mallocFp)(int64_t));
extern int32_t (*queryProcessMsgRsp[TDMT_MAX])(void* output, char* msg, int32_t msgSize);

View File

@ -127,6 +127,7 @@ typedef enum {
TASK_LEVEL__AGG,
TASK_LEVEL__SINK,
TASK_LEVEL_SMA,
TASK_LEVEL__MERGE,
} ETASK_LEVEL;
enum {
@ -135,6 +136,7 @@ enum {
TASK_OUTPUT__TABLE,
TASK_OUTPUT__SMA,
TASK_OUTPUT__FETCH,
TASK_OUTPUT__VTABLE_MAP,
};
enum {
@ -211,6 +213,11 @@ typedef struct {
SUseDbRsp dbInfo;
} STaskDispatcherShuffle;
typedef struct {
SArray* taskInfos; // SArray<STaskDispatcherFixed>
SSHashObj* vtableMap;
} STaskDispatcherVtableMap;
typedef struct {
int32_t nodeId;
} SDownstreamTaskEpset;
@ -396,8 +403,9 @@ typedef struct SHistoryTaskInfo {
typedef struct STaskOutputInfo {
union {
STaskDispatcherFixed fixedDispatcher;
STaskDispatcherShuffle shuffleDispatcher;
STaskDispatcherFixed fixedDispatcher;
STaskDispatcherShuffle shuffleDispatcher;
STaskDispatcherVtableMap vtableMapDispatcher;
STaskSinkTb tbSink;
STaskSinkSma smaSink;
@ -466,6 +474,7 @@ struct SStreamTask {
STaskCheckInfo taskCheckInfo;
SNotifyInfo notifyInfo;
STaskNotifyEventStat notifyEventStat;
SArray* pVTables; // for merge task, SArray<SVCTableMergeInfo>
// the followings attributes don't be serialized
SScanhistorySchedInfo schedHistoryInfo;

View File

@ -246,6 +246,7 @@ int32_t taosGetErrSize();
#define TSDB_CODE_MND_USER_DISABLED TAOS_DEF_ERROR_CODE(0, 0x0315)
#define TSDB_CODE_MND_INVALID_PLATFORM TAOS_DEF_ERROR_CODE(0, 0x0316)
#define TSDB_CODE_MND_RETURN_VALUE_NULL TAOS_DEF_ERROR_CODE(0, 0x0317)
#define TSDB_CODE_MND_INTERNAL_ERROR TAOS_DEF_ERROR_CODE(0, 0x0318)
// mnode-sdb
#define TSDB_CODE_SDB_OBJ_ALREADY_THERE TAOS_DEF_ERROR_CODE(0, 0x0320) // internal

View File

@ -55,6 +55,7 @@ bool taosLRUCacheRef(SLRUCache *cache, LRUHandle *handle);
bool taosLRUCacheRelease(SLRUCache *cache, LRUHandle *handle, bool eraseIfLastRef);
void *taosLRUCacheValue(SLRUCache *cache, LRUHandle *handle);
void taosLRUCacheUpdate(SLRUCache *cache, LRUHandle *handle, void *value);
size_t taosLRUCacheGetUsage(SLRUCache *cache);
size_t taosLRUCacheGetPinnedUsage(SLRUCache *cache);

View File

@ -4528,6 +4528,174 @@ _exit:
return code;
}
int32_t tSerializeSVSubTablesReq(void *buf, int32_t bufLen, SVSubTablesReq *pReq) {
SEncoder encoder = {0};
int32_t code = 0;
int32_t lino;
int32_t tlen;
tEncoderInit(&encoder, buf, bufLen);
TAOS_CHECK_EXIT(tStartEncode(&encoder));
TAOS_CHECK_EXIT(tEncodeI64(&encoder, pReq->suid));
tEndEncode(&encoder);
_exit:
if (code) {
tlen = code;
} else {
tlen = encoder.pos;
}
tEncoderClear(&encoder);
return tlen;
}
int32_t tDeserializeSVSubTablesReq(void *buf, int32_t bufLen, SVSubTablesReq *pReq) {
SDecoder decoder = {0};
int32_t code = 0;
int32_t lino;
tDecoderInit(&decoder, buf, bufLen);
TAOS_CHECK_EXIT(tStartDecode(&decoder));
TAOS_CHECK_EXIT(tDecodeI64(&decoder, &pReq->suid));
tEndDecode(&decoder);
_exit:
tDecoderClear(&decoder);
return code;
}
int32_t tSerializeSVSubTablesRspImpl(SEncoder* pEncoder, SVSubTablesRsp *pRsp) {
int32_t code = 0;
int32_t lino;
TAOS_CHECK_EXIT(tEncodeI32(pEncoder, pRsp->vgId));
int32_t numOfTables = taosArrayGetSize(pRsp->pTables);
TAOS_CHECK_EXIT(tEncodeI32(pEncoder, numOfTables));
for (int32_t i = 0; i < numOfTables; ++i) {
SVCTableRefCols *pTb = (SVCTableRefCols *)taosArrayGetP(pRsp->pTables, i);
TAOS_CHECK_EXIT(tEncodeU64(pEncoder, pTb->uid));
TAOS_CHECK_EXIT(tEncodeI32(pEncoder, pTb->numOfSrcTbls));
TAOS_CHECK_EXIT(tEncodeI32(pEncoder, pTb->numOfColRefs));
for (int32_t n = 0; n < pTb->numOfColRefs; ++n) {
SRefColInfo* pCol = pTb->refCols + n;
TAOS_CHECK_EXIT(tEncodeI16(pEncoder, pCol->colId));
TAOS_CHECK_EXIT(tEncodeCStr(pEncoder, pCol->refDbName));
TAOS_CHECK_EXIT(tEncodeCStr(pEncoder, pCol->refTableName));
TAOS_CHECK_EXIT(tEncodeCStr(pEncoder, pCol->refColName));
}
}
_exit:
return code;
}
int32_t tSerializeSVSubTablesRsp(void *buf, int32_t bufLen, SVSubTablesRsp *pRsp) {
SEncoder encoder = {0};
int32_t code = 0;
int32_t lino;
int32_t tlen;
tEncoderInit(&encoder, buf, bufLen);
TAOS_CHECK_EXIT(tStartEncode(&encoder));
TAOS_CHECK_EXIT(tSerializeSVSubTablesRspImpl(&encoder, pRsp));
tEndEncode(&encoder);
_exit:
if (code) {
tlen = code;
} else {
tlen = encoder.pos;
}
tEncoderClear(&encoder);
return tlen;
}
int32_t tDeserializeSVSubTablesRspImpl(SDecoder* pDecoder, SVSubTablesRsp *pRsp) {
int32_t code = 0;
int32_t lino;
SVCTableRefCols tb = {0};
TAOS_CHECK_EXIT(tDecodeI32(pDecoder, &pRsp->vgId));
int32_t numOfTables = 0;
TAOS_CHECK_EXIT(tDecodeI32(pDecoder, &numOfTables));
if (numOfTables > 0) {
pRsp->pTables = taosArrayInit(numOfTables, POINTER_BYTES);
if (NULL == pRsp->pTables) {
code = terrno;
return code;
}
for (int32_t i = 0; i < numOfTables; ++i) {
TAOS_CHECK_EXIT(tDecodeU64(pDecoder, &tb.uid));
TAOS_CHECK_EXIT(tDecodeI32(pDecoder, &tb.numOfSrcTbls));
TAOS_CHECK_EXIT(tDecodeI32(pDecoder, &tb.numOfColRefs));
if (tb.numOfColRefs > 0) {
SVCTableRefCols* pTb = taosMemoryCalloc(1, sizeof(tb) + tb.numOfColRefs * sizeof(SRefColInfo));
if (NULL == pTb) {
code = terrno;
return code;
}
if (NULL == taosArrayPush(pRsp->pTables, &pTb)) {
code = terrno;
taosMemoryFree(pTb);
return code;
}
pTb->uid = tb.uid;
pTb->numOfSrcTbls = tb.numOfSrcTbls;
pTb->numOfColRefs = tb.numOfColRefs;
pTb->refCols = (SRefColInfo*)(pTb + 1);
for (int32_t n = 0; n < tb.numOfColRefs; ++n) {
TAOS_CHECK_EXIT(tDecodeI16(pDecoder, &pTb->refCols[n].colId));
TAOS_CHECK_EXIT(tDecodeCStrTo(pDecoder, pTb->refCols[n].refDbName));
TAOS_CHECK_EXIT(tDecodeCStrTo(pDecoder, pTb->refCols[n].refTableName));
TAOS_CHECK_EXIT(tDecodeCStrTo(pDecoder, pTb->refCols[n].refColName));
}
}
}
}
_exit:
return code;
}
int32_t tDeserializeSVSubTablesRsp(void *buf, int32_t bufLen, SVSubTablesRsp *pRsp) {
SDecoder decoder = {0};
int32_t code = 0;
int32_t lino;
tDecoderInit(&decoder, buf, bufLen);
TAOS_CHECK_EXIT(tStartDecode(&decoder));
TAOS_CHECK_EXIT(tDeserializeSVSubTablesRspImpl(&decoder, pRsp));
tEndDecode(&decoder);
_exit:
tDecoderClear(&decoder);
return code;
}
void tFreeSVCTableRefCols(void *pParam) {
SVCTableRefCols* pCols = *(SVCTableRefCols**)pParam;
if (NULL == pCols) {
return;
}
taosMemoryFree(pCols);
}
void tDestroySVSubTablesRsp(void* rsp) {
if (NULL == rsp) {
return;
}
SVSubTablesRsp *pRsp = (SVSubTablesRsp*)rsp;
taosArrayDestroyEx(pRsp->pTables, tFreeSVCTableRefCols);
}
int32_t tSerializeSQnodeListReq(void *buf, int32_t bufLen, SQnodeListReq *pReq) {
SEncoder encoder = {0};
int32_t code = 0;
@ -10296,6 +10464,13 @@ int32_t tSerializeSCMCreateStreamReq(void *buf, int32_t bufLen, const SCMCreateS
TAOS_CHECK_EXIT(tEncodeCStr(&encoder, pReq->pWendName));
TAOS_CHECK_EXIT(tEncodeCStr(&encoder, pReq->pGroupIdName));
TAOS_CHECK_EXIT(tEncodeCStr(&encoder, pReq->pIsWindowFilledName));
int32_t vgNum = taosArrayGetSize(pReq->pVSubTables);
TAOS_CHECK_EXIT(tEncodeI32(&encoder, vgNum));
for (int32_t i = 0; i < vgNum; ++i) {
SVSubTablesRsp* pVgTables = taosArrayGet(pReq->pVSubTables, i);
TAOS_CHECK_EXIT(tSerializeSVSubTablesRspImpl(&encoder, pVgTables));
}
tEndEncode(&encoder);
_exit:
@ -10460,9 +10635,27 @@ int32_t tDeserializeSCMCreateStreamReq(void *buf, int32_t bufLen, SCMCreateStrea
TAOS_CHECK_EXIT(tDecodeCStrTo(&decoder, pReq->pWendName));
TAOS_CHECK_EXIT(tDecodeCStrTo(&decoder, pReq->pGroupIdName));
TAOS_CHECK_EXIT(tDecodeCStrTo(&decoder, pReq->pIsWindowFilledName));
int32_t vgNum = 0;
TAOS_CHECK_EXIT(tDecodeI32(&decoder, &vgNum));
if (vgNum > 0) {
pReq->pVSubTables = taosArrayInit(vgNum, sizeof(SVSubTablesRsp));
if (pReq->pVSubTables == NULL) {
TAOS_CHECK_EXIT(terrno);
}
SVSubTablesRsp vgTables = {0};
for (int32_t i = 0; i < vgNum; ++i) {
vgTables.pTables = NULL;
TAOS_CHECK_EXIT(tDeserializeSVSubTablesRspImpl(&decoder, &vgTables));
if (taosArrayPush(pReq->pVSubTables, &vgTables) == NULL) {
tDestroySVSubTablesRsp(&vgTables);
TAOS_CHECK_EXIT(terrno);
}
}
}
}
tEndDecode(&decoder);
_exit:
tDecoderClear(&decoder);
return code;
@ -10525,6 +10718,7 @@ void tFreeSCMCreateStreamReq(SCMCreateStreamReq *pReq) {
taosArrayDestroy(pReq->pVgroupVerList);
taosArrayDestroy(pReq->pCols);
taosArrayDestroyP(pReq->pNotifyAddrUrls, NULL);
taosArrayDestroyEx(pReq->pVSubTables, tDestroySVSubTablesRsp);
}
int32_t tEncodeSRSmaParam(SEncoder *pCoder, const SRSmaParam *pRSmaParam) {

View File

@ -390,6 +390,10 @@ void *pTimezoneNameMap = NULL;
int32_t tsStreamNotifyMessageSize = 8 * 1024; // KB, default 8MB
int32_t tsStreamNotifyFrameSize = 256; // KB, default 256KB
int32_t tsStreamVirtualMergeMaxDelayMs = 10 * 1000; // 10s
int32_t tsStreamVirtualMergeMaxMemKb = 16 * 1024; // 16MB
int32_t tsStreamVirtualMergeWaitMode = 0; // 0 wait forever, 1 wait for max delay, 2 wait for max mem
int32_t taosCheckCfgStrValueLen(const char *name, const char *value, int32_t len);
#define TAOS_CHECK_GET_CFG_ITEM(pCfg, pItem, pName) \
@ -1011,6 +1015,10 @@ static int32_t taosAddServerCfg(SConfig *pCfg) {
TAOS_CHECK_RETURN(cfgAddInt32(pCfg, "adapterPort", tsAdapterPort, 1, 65056, CFG_SCOPE_SERVER, CFG_DYN_SERVER_LAZY, CFG_CATEGORY_LOCAL));
TAOS_CHECK_RETURN(cfgAddString(pCfg, "adapterToken", tsAdapterToken, CFG_SCOPE_SERVER, CFG_DYN_SERVER_LAZY, CFG_CATEGORY_LOCAL));
TAOS_CHECK_RETURN(cfgAddInt32(pCfg, "streamVirtualMergeMaxDelay", tsStreamVirtualMergeMaxDelayMs, 500, 10 * 60 * 1000, CFG_SCOPE_SERVER, CFG_DYN_NONE,CFG_CATEGORY_LOCAL));
TAOS_CHECK_RETURN(cfgAddInt32(pCfg, "streamVirtualMergeMaxMem", tsStreamVirtualMergeMaxMemKb, 8 * 1024, 1 * 1024 * 1024, CFG_SCOPE_SERVER, CFG_DYN_NONE,CFG_CATEGORY_LOCAL));
TAOS_CHECK_RETURN(cfgAddInt32(pCfg, "streamVirtualMergeWaitMode", tsStreamVirtualMergeWaitMode, 0, 2, CFG_SCOPE_SERVER, CFG_DYN_NONE,CFG_CATEGORY_LOCAL));
// clang-format on
// GRANT_CFG_ADD;
@ -1928,6 +1936,15 @@ static int32_t taosSetServerCfg(SConfig *pCfg) {
TAOS_CHECK_RETURN(taosCheckCfgStrValueLen(pItem->name, pItem->str, tListLen(tsAdapterToken)));
tstrncpy(tsAdapterToken, pItem->str, tListLen(tsAdapterToken));
TAOS_CHECK_GET_CFG_ITEM(pCfg, pItem, "streamVirtualMergeMaxDelay");
tsStreamVirtualMergeMaxDelayMs = pItem->i32;
TAOS_CHECK_GET_CFG_ITEM(pCfg, pItem, "streamVirtualMergeMaxMem");
tsStreamVirtualMergeMaxMemKb = pItem->i32;
TAOS_CHECK_GET_CFG_ITEM(pCfg, pItem, "streamVirtualMergeWaitMode");
tsStreamVirtualMergeWaitMode = pItem->i32;
// GRANT_CFG_GET;
TAOS_RETURN(TSDB_CODE_SUCCESS);
}

View File

@ -966,6 +966,7 @@ SArray *vmGetMsgHandles() {
if (dmSetMgmtHandle(pArray, TDMT_VND_ALTER_TABLE, vmPutMsgToWriteQueue, 0) == NULL) goto _OVER;
if (dmSetMgmtHandle(pArray, TDMT_VND_UPDATE_TAG_VAL, vmPutMsgToWriteQueue, 0) == NULL) goto _OVER;
if (dmSetMgmtHandle(pArray, TDMT_VND_TABLE_META, vmPutMsgToFetchQueue, 0) == NULL) goto _OVER;
if (dmSetMgmtHandle(pArray, TDMT_VND_VSUBTABLES_META, vmPutMsgToFetchQueue, 0) == NULL) goto _OVER;
if (dmSetMgmtHandle(pArray, TDMT_VND_TABLE_CFG, vmPutMsgToFetchQueue, 0) == NULL) goto _OVER;
if (dmSetMgmtHandle(pArray, TDMT_VND_BATCH_META, vmPutMsgToFetchQueue, 0) == NULL) goto _OVER;
if (dmSetMgmtHandle(pArray, TDMT_VND_TABLES_META, vmPutMsgToFetchQueue, 0) == NULL) goto _OVER;

View File

@ -788,6 +788,7 @@ typedef struct SStreamConf {
int64_t watermark;
} SStreamConf;
typedef struct {
char name[TSDB_STREAM_FNAME_LEN];
SRWLatch lock;
@ -839,6 +840,8 @@ typedef struct {
int8_t subTableWithoutMd5;
char reserve[TSDB_RESERVE_VALUE_LEN];
SSHashObj* pVTableMap; // do not serialize
SQueryPlan* pPlan; // do not serialize
} SStreamObj;
typedef struct SStreamSeq {

View File

@ -27,7 +27,7 @@ int32_t mndSchedInitSubEp(SMnode* pMnode, const SMqTopicObj* pTopic, SMqSubscrib
int32_t mndConvertRsmaTask(char** pDst, int32_t* pDstLen, const char* ast, int64_t uid, int8_t triggerType,
int64_t watermark, int64_t deleteMark);
int32_t mndScheduleStream(SMnode* pMnode, SStreamObj* pStream, int64_t skey, SArray* pVerList);
int32_t mndScheduleStream(SMnode* pMnode, SStreamObj* pStream, SCMCreateStreamReq* pCreate);
#ifdef __cplusplus
}

View File

@ -234,6 +234,12 @@ void tFreeStreamObj(SStreamObj *pStream) {
if (pStream->tagSchema.nCols > 0) {
taosMemoryFree(pStream->tagSchema.pSchema);
}
qDestroyQueryPlan(pStream->pPlan);
pStream->pPlan = NULL;
tSimpleHashCleanup(pStream->pVTableMap);
pStream->pVTableMap = NULL;
}
#endif

View File

@ -371,10 +371,14 @@ static void haltInitialTaskStatus(SStreamTask* pTask, SSubplan* pPlan, bool isFi
}
static int32_t buildSourceTask(SStreamObj* pStream, SEpSet* pEpset, EStreamTaskType type, bool useTriggerParam,
int8_t hasAggTasks, SStreamTask** pTask) {
int8_t hasAggTasks, SStreamTask** pTask, SArray* pSourceTaskList) {
uint64_t uid = 0;
SArray** pTaskList = NULL;
streamGetUidTaskList(pStream, type, &uid, &pTaskList);
if (pSourceTaskList) {
pTaskList = &pSourceTaskList;
} else {
streamGetUidTaskList(pStream, type, &uid, &pTaskList);
}
int32_t trigger = 0;
if (type == STREAM_RECALCUL_TASK) {
@ -440,10 +444,78 @@ static void setHTasksId(SStreamObj* pStream) {
}
}
static int32_t addSourceTaskVTableOutput(SStreamTask* pTask, SSHashObj* pVgTasks, SSHashObj* pVtables) {
int32_t code = 0;
int32_t lino = 0;
int32_t taskNum = tSimpleHashGetSize(pVgTasks);
int32_t tbNum = tSimpleHashGetSize(pVtables);
SSHashObj *pTaskMap = tSimpleHashInit(taskNum, taosGetDefaultHashFunction(TSDB_DATA_TYPE_INT));
TSDB_CHECK_NULL(pTaskMap, code, lino, _end, terrno);
pTask->outputInfo.type = TASK_OUTPUT__VTABLE_MAP;
STaskDispatcherVtableMap *pDispatcher = &pTask->outputInfo.vtableMapDispatcher;
pDispatcher->taskInfos = taosArrayInit(taskNum, sizeof(STaskDispatcherFixed));
TSDB_CHECK_NULL(pDispatcher->taskInfos, code, lino, _end, terrno);
pDispatcher->vtableMap = tSimpleHashInit(tbNum, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BIGINT));
TSDB_CHECK_NULL(pDispatcher->vtableMap, code, lino, _end, terrno);
int32_t iter = 0, vgId = 0;
uint64_t uid = 0;
STaskDispatcherFixed* pAddr = NULL;
void* p = NULL;
while (NULL != (p = tSimpleHashIterate(pVtables, p, &iter))) {
char* vgUid = tSimpleHashGetKey(p, NULL);
vgId = *(int32_t*)vgUid;
uid = *(uint64_t*)((int32_t*)vgUid + 1);
pAddr = tSimpleHashGet(pVgTasks, &vgId, sizeof(vgId));
if (NULL == pAddr) {
mError("tSimpleHashGet vgId %d not found", vgId);
return code;
}
void* px = tSimpleHashGet(pTaskMap, &pAddr->taskId, sizeof(int32_t));
int32_t idx = 0;
if (px == NULL) {
px = taosArrayPush(pDispatcher->taskInfos, pAddr);
TSDB_CHECK_NULL(px, code, lino, _end, terrno);
idx = taosArrayGetSize(pDispatcher->taskInfos) - 1;
code = tSimpleHashPut(pTaskMap, &pAddr->taskId, sizeof(int32_t), &idx, sizeof(int32_t));
if (code) {
mError("tSimpleHashPut uid to task idx failed, error:%d", code);
return code;
}
} else {
idx = *(int32_t*)px;
}
code = tSimpleHashPut(pDispatcher->vtableMap, &uid, sizeof(int64_t), &idx, sizeof(int32_t));
if (code) {
mError("tSimpleHashPut uid to STaskDispatcherFixed failed, error:%d", code);
return code;
}
mDebug("source task[%s,vg:%d] add vtable output map, vuid %" PRIu64 " => [%d, vg:%d]",
pTask->id.idStr, pTask->info.nodeId, uid, pAddr->taskId, pAddr->nodeId);
}
_end:
if (code != TSDB_CODE_SUCCESS) {
mError("source task[%s,vg:%d] add vtable output map failed, lino:%d, error:%s", pTask->id.idStr, pTask->info.nodeId,
lino, tstrerror(code));
}
if (pTaskMap != NULL) {
tSimpleHashCleanup(pTaskMap);
}
return code;
}
static int32_t doAddSourceTask(SMnode* pMnode, SSubplan* plan, SStreamObj* pStream, SEpSet* pEpset, int64_t skey,
SArray* pVerList, SVgObj* pVgroup, EStreamTaskType type, bool useTriggerParam, int8_t hasAggTasks) {
SArray* pVerList, SVgObj* pVgroup, EStreamTaskType type, bool useTriggerParam,
int8_t hasAggTasks, SSHashObj* pVgTasks, SArray* pSourceTaskList) {
SStreamTask* pTask = NULL;
int32_t code = buildSourceTask(pStream, pEpset, type, useTriggerParam, hasAggTasks, &pTask);
int32_t code = buildSourceTask(pStream, pEpset, type, useTriggerParam, hasAggTasks, &pTask, pSourceTaskList);
if (code != TSDB_CODE_SUCCESS) {
return code;
}
@ -461,7 +533,13 @@ static int32_t doAddSourceTask(SMnode* pMnode, SSubplan* plan, SStreamObj* pStre
return code;
}
return TDB_CODE_SUCCESS;
mTrace("souce task plan:%s", pTask->exec.qmsg);
if (pVgTasks) {
code = addSourceTaskVTableOutput(pTask, pVgTasks, plan->pVTables);
}
return code;
}
static SSubplan* getScanSubPlan(const SQueryPlan* pPlan) {
@ -480,6 +558,66 @@ static SSubplan* getScanSubPlan(const SQueryPlan* pPlan) {
return plan;
}
static int32_t doAddMergeTask(SMnode* pMnode, SSubplan* plan, SStreamObj* pStream, SEpSet* pEpset, SVgObj* pVgroup,
bool isHistoryTask, bool useTriggerParam, int8_t hasAggTasks, SArray* pVtables) {
SStreamTask* pTask = NULL;
SArray** pTaskList = taosArrayGetLast(pStream->pTaskList);
int32_t code = tNewStreamTask(pStream->uid, TASK_LEVEL__MERGE, pEpset, isHistoryTask, pStream->conf.trigger,
useTriggerParam ? pStream->conf.triggerParam : 0, *pTaskList, pStream->conf.fillHistory,
pStream->subTableWithoutMd5, hasAggTasks, &pTask);
if (code != TSDB_CODE_SUCCESS) {
return code;
}
int32_t vtbNum = taosArrayGetSize(pVtables);
pTask->pVTables = taosArrayInit(vtbNum, sizeof(SVCTableMergeInfo));
if (NULL == pTask->pVTables) {
code = terrno;
mError("taosArrayInit %d SVCTableMergeInfo failed, error:%d", vtbNum, terrno);
return code;
}
SVCTableMergeInfo tbInfo;
for (int32_t i = 0; i < vtbNum; ++i) {
SVCTableRefCols** pTb = taosArrayGet(pVtables, i);
tbInfo.uid = (*pTb)->uid;
tbInfo.numOfSrcTbls = (*pTb)->numOfSrcTbls;
if (NULL == taosArrayPush(pTask->pVTables, &tbInfo)) {
code = terrno;
mError("taosArrayPush SVCTableMergeInfo failed, error:%d", terrno);
return code;
}
mDebug("merge task[%s, vg:%d] add vtable info: vuid %" PRIu64 ", numOfSrcTbls:%d",
pTask->id.idStr, pVgroup->vgId, tbInfo.uid, tbInfo.numOfSrcTbls);
}
code = mndAssignStreamTaskToVgroup(pMnode, pTask, plan, pVgroup);
if (code != TSDB_CODE_SUCCESS) {
return code;
}
return TDB_CODE_SUCCESS;
}
static SSubplan* getVTbScanSubPlan(const SQueryPlan* pPlan) {
int32_t numOfPlanLevel = LIST_LENGTH(pPlan->pSubplans);
SNodeListNode* inner = (SNodeListNode*)nodesListGetNode(pPlan->pSubplans, numOfPlanLevel - 2);
if (LIST_LENGTH(inner->pNodeList) != 1) {
terrno = TSDB_CODE_QRY_INVALID_INPUT;
return NULL;
}
SSubplan* plan = (SSubplan*)nodesListGetNode(inner->pNodeList, 0);
if (plan->subplanType != SUBPLAN_TYPE_MERGE) {
terrno = TSDB_CODE_QRY_INVALID_INPUT;
return NULL;
}
return plan;
}
static SSubplan* getAggSubPlan(const SQueryPlan* pPlan, int index) {
SNodeListNode* inner = (SNodeListNode*)nodesListGetNode(pPlan->pSubplans, index);
if (LIST_LENGTH(inner->pNodeList) != 1) {
@ -495,6 +633,118 @@ static SSubplan* getAggSubPlan(const SQueryPlan* pPlan, int index) {
return plan;
}
static int32_t addVTableMergeTask(SMnode* pMnode, SSubplan* plan, SStreamObj* pStream, SEpSet* pEpset,
bool useTriggerParam, bool hasAggTasks, SCMCreateStreamReq* pCreate) {
SVgObj* pVgroup = NULL;
int32_t code = TSDB_CODE_SUCCESS;
int32_t vgNum = taosArrayGetSize(pCreate->pVSubTables);
for (int32_t i = 0; i < vgNum; ++i) {
SVSubTablesRsp* pVg = (SVSubTablesRsp*)taosArrayGet(pCreate->pVSubTables, i);
pVgroup = mndAcquireVgroup(pMnode, pVg->vgId);
if (NULL == pVgroup) {
mWarn("vnode %d in pVSubTables not found", pVg->vgId);
continue;
}
code = doAddMergeTask(pMnode, plan, pStream, pEpset, pVgroup, false, useTriggerParam, hasAggTasks, pVg->pTables);
if (code != 0) {
mError("failed to create stream task, code:%s", tstrerror(code));
mndReleaseVgroup(pMnode, pVgroup);
return code;
}
mndReleaseVgroup(pMnode, pVgroup);
}
return code;
}
static int32_t buildMergeTaskHash(SArray* pMergeTaskList, SSHashObj** ppVgTasks) {
STaskDispatcherFixed addr;
int32_t code = 0;
int32_t taskNum = taosArrayGetSize(pMergeTaskList);
*ppVgTasks = tSimpleHashInit(taskNum, taosGetDefaultHashFunction(TSDB_DATA_TYPE_INT));
if (NULL == *ppVgTasks) {
code = terrno;
mError("tSimpleHashInit %d failed", taskNum);
return code;
}
for (int32_t i = 0; i < taskNum; ++i) {
SStreamTask* pTask = taosArrayGetP(pMergeTaskList, i);
addr.taskId = pTask->id.taskId;
addr.nodeId = pTask->info.nodeId;
addr.epSet = pTask->info.epSet;
code = tSimpleHashPut(*ppVgTasks, &addr.nodeId, sizeof(addr.nodeId), &addr, sizeof(addr));
if (code) {
mError("tSimpleHashPut %d STaskDispatcherFixed failed", i);
return code;
}
}
return code;
}
static int32_t addVTableSourceTask(SMnode* pMnode, SSubplan* plan, SStreamObj* pStream, SEpSet* pEpset,
int64_t nextWindowSkey, SArray* pVerList, bool useTriggerParam, bool hasAggTasks,
SCMCreateStreamReq* pCreate, SSHashObj* pVTableMap, SArray* pSourceTaskList,
SArray* pMergeTaskList) {
int32_t code = 0;
SSHashObj* pVgTasks = NULL;
int32_t vgId = 0;
int32_t iter = 0;
SVgObj* pVgroup = NULL;
void* p = NULL;
code = buildMergeTaskHash(pMergeTaskList, &pVgTasks);
if (code) {
tSimpleHashCleanup(pVgTasks);
return code;
}
while (NULL != (p = tSimpleHashIterate(pVTableMap, p, &iter))) {
char* pDbVg = tSimpleHashGetKey(p, NULL);
char* pVgStr = strrchr(pDbVg, '.');
if (NULL == pVgStr) {
mError("Invalid DbVg string: %s", pDbVg);
tSimpleHashCleanup(pVgTasks);
return TSDB_CODE_MND_INTERNAL_ERROR;
}
(void)taosStr2int32(pVgStr + 1, &vgId);
pVgroup = mndAcquireVgroup(pMnode, vgId);
if (NULL == pVgroup) {
mWarn("vnode %d not found", vgId);
continue;
}
plan->pVTables = *(SSHashObj**)p;
*(SSHashObj**)p = NULL;
code = doAddSourceTask(pMnode, plan, pStream, pEpset, nextWindowSkey, pVerList, pVgroup, false, useTriggerParam,
hasAggTasks, pVgTasks, pSourceTaskList);
if (code != 0) {
mError("failed to create stream task, code:%s", tstrerror(code));
mndReleaseVgroup(pMnode, pVgroup);
tSimpleHashCleanup(pVgTasks);
return code;
}
mndReleaseVgroup(pMnode, pVgroup);
}
tSimpleHashCleanup(pVgTasks);
return TSDB_CODE_SUCCESS;
}
static int32_t addSourceTask(SMnode* pMnode, SSubplan* plan, SStreamObj* pStream, SEpSet* pEpset,
int64_t nextWindowSkey, SArray* pVerList, bool useTriggerParam, bool hasAggTasks) {
void* pIter = NULL;
@ -517,12 +767,11 @@ static int32_t addSourceTask(SMnode* pMnode, SSubplan* plan, SStreamObj* pStream
}
code = doAddSourceTask(pMnode, plan, pStream, pEpset, nextWindowSkey, pVerList, pVgroup, STREAM_NORMAL_TASK,
useTriggerParam, hasAggTasks);
useTriggerParam, hasAggTasks, NULL, NULL);
if (code != 0) {
mError("failed to create stream task, code:%s", tstrerror(code));
// todo drop the added source tasks.
sdbRelease(pSdb, pVgroup);
mndReleaseVgroup(pMnode, pVgroup);
return code;
}
@ -535,7 +784,7 @@ static int32_t addSourceTask(SMnode* pMnode, SSubplan* plan, SStreamObj* pStream
}
code = doAddSourceTask(pMnode, plan, pStream, pEpset, nextWindowSkey, pVerList, pVgroup, type,
useTriggerParam, hasAggTasks);
useTriggerParam, hasAggTasks, NULL, NULL);
if (code != 0) {
sdbRelease(pSdb, pVgroup);
return code;
@ -696,6 +945,26 @@ static void bindSourceSink(SStreamObj* pStream, SMnode* pMnode, SArray* tasks, b
}
}
static void bindVtableMergeSink(SStreamObj* pStream, SMnode* pMnode, SArray* tasks, bool hasExtraSink) {
int32_t code = 0;
SArray* pSinkTaskList = taosArrayGetP(tasks, SINK_NODE_LEVEL);
SArray* pMergeTaskList = taosArrayGetP(tasks, hasExtraSink ? SINK_NODE_LEVEL + 2 : SINK_NODE_LEVEL + 1);
for (int i = 0; i < taosArrayGetSize(pMergeTaskList); i++) {
SStreamTask* pMergeTask = taosArrayGetP(pMergeTaskList, i);
mDebug("bindVtableMergeSink taskId:%s to sink task list", pMergeTask->id.idStr);
if (hasExtraSink) {
bindTaskToSinkTask(pStream, pMnode, pSinkTaskList, pMergeTask);
} else {
if ((code = mndSetSinkTaskInfo(pStream, pMergeTask)) != 0) {
mError("failed bind task to sink task since %s", tstrerror(code));
}
}
}
}
static void bindTwoLevel(SArray* tasks, int32_t begin, int32_t end) {
int32_t code = 0;
size_t size = taosArrayGetSize(tasks);
@ -719,13 +988,370 @@ static void bindTwoLevel(SArray* tasks, int32_t begin, int32_t end) {
mDebug("bindTwoLevel task list(%d-%d) to taskId:%s", begin, end - 1, (*(pDownTask))->id.idStr);
}
static int32_t doScheduleStream(SStreamObj* pStream, SMnode* pMnode, SQueryPlan* pPlan, SEpSet* pEpset, int64_t skey,
SArray* pVerList) {
int32_t tableHashValueComp(void const* lp, void const* rp) {
uint32_t* key = (uint32_t*)lp;
SVgroupInfo* pVg = (SVgroupInfo*)rp;
if (*key < pVg->hashBegin) {
return -1;
} else if (*key > pVg->hashEnd) {
return 1;
}
return 0;
}
int dbVgInfoComp(const void* lp, const void* rp) {
SVGroupHashInfo* pLeft = (SVGroupHashInfo*)lp;
SVGroupHashInfo* pRight = (SVGroupHashInfo*)rp;
if (pLeft->hashBegin < pRight->hashBegin) {
return -1;
} else if (pLeft->hashBegin > pRight->hashBegin) {
return 1;
}
return 0;
}
int32_t getTableVgId(SDBVgHashInfo* dbInfo, int32_t acctId, char* dbFName, int32_t* vgId, char *tbName) {
int32_t code = 0;
int32_t lino = 0;
SVgroupInfo* vgInfo = NULL;
char tbFullName[TSDB_TABLE_FNAME_LEN];
(void)snprintf(tbFullName, sizeof(tbFullName), "%s.%s", dbFName, tbName);
uint32_t hashValue = taosGetTbHashVal(tbFullName, (uint32_t)strlen(tbFullName), dbInfo->hashMethod,
dbInfo->hashPrefix, dbInfo->hashSuffix);
if (!dbInfo->vgSorted) {
taosArraySort(dbInfo->vgArray, dbVgInfoComp);
dbInfo->vgSorted = true;
}
vgInfo = taosArraySearch(dbInfo->vgArray, &hashValue, tableHashValueComp, TD_EQ);
if (NULL == vgInfo) {
qError("no hash range found for hash value [%u], dbFName:%s, numOfVgId:%d", hashValue, dbFName,
(int32_t)taosArrayGetSize(dbInfo->vgArray));
return TSDB_CODE_INVALID_PARA;
}
*vgId = vgInfo->vgId;
_return:
return code;
}
static void destroyVSubtableVtb(SSHashObj *pVtable) {
int32_t iter = 0;
void* p = NULL;
while (NULL != (p = tSimpleHashIterate(pVtable, p, &iter))) {
taosArrayDestroy(*(SArray**)p);
}
tSimpleHashCleanup(pVtable);
}
static void destroyVSubtableVgHash(SSHashObj *pVg) {
int32_t iter = 0;
SSHashObj** pVtable = NULL;
void* p = NULL;
while (NULL != (p = tSimpleHashIterate(pVg, p, &iter))) {
pVtable = (SSHashObj**)p;
destroyVSubtableVtb(*pVtable);
}
tSimpleHashCleanup(pVg);
}
static void destroyDbVgroupsHash(SSHashObj *pDbVgs) {
int32_t iter = 0;
SDBVgHashInfo* pVg = NULL;
void* p = NULL;
while (NULL != (p = tSimpleHashIterate(pDbVgs, p, &iter))) {
pVg = (SDBVgHashInfo*)p;
taosArrayDestroy(pVg->vgArray);
}
tSimpleHashCleanup(pDbVgs);
}
static int32_t buildDBVgroupsMap(SMnode* pMnode, SSHashObj* pDbVgroup) {
void* pIter = NULL;
SSdb* pSdb = pMnode->pSdb;
int32_t code = TSDB_CODE_SUCCESS;
char key[TSDB_DB_NAME_LEN + 32];
SArray* pTarget = NULL;
SArray* pNew = NULL;
SDbObj* pDb = NULL;
SDBVgHashInfo dbInfo = {0}, *pDbInfo = NULL;
while (1) {
SVgObj* pVgroup = NULL;
pIter = sdbFetch(pSdb, SDB_VGROUP, pIter, (void**)&pVgroup);
if (pIter == NULL) {
break;
}
pDbInfo = (SDBVgHashInfo*)tSimpleHashGet(pDbVgroup, pVgroup->dbName, strlen(pVgroup->dbName) + 1);
if (NULL == pDbInfo) {
pNew = taosArrayInit(20, sizeof(SVGroupHashInfo));
if (NULL == pNew) {
code = terrno;
mError("taosArrayInit SVGroupHashInfo failed, code:%s", tstrerror(terrno));
sdbRelease(pSdb, pVgroup);
return code;
}
pDb = mndAcquireDb(pMnode, pVgroup->dbName);
if (pDb == NULL) {
code = terrno;
mError("mndAcquireDb %s failed, code:%s", pVgroup->dbName, tstrerror(terrno));
sdbRelease(pSdb, pVgroup);
return code;
}
dbInfo.vgSorted = false;
dbInfo.hashMethod = pDb->cfg.hashMethod;
dbInfo.hashPrefix = pDb->cfg.hashPrefix;
dbInfo.hashSuffix = pDb->cfg.hashSuffix;
dbInfo.vgArray = pNew;
mndReleaseDb(pMnode, pDb);
pTarget = pNew;
} else {
pTarget = pDbInfo->vgArray;
}
SVGroupHashInfo vgInfo = {.vgId = pVgroup->vgId, .hashBegin = pVgroup->hashBegin, .hashEnd = pVgroup->hashEnd};
if (NULL == taosArrayPush(pTarget, &vgInfo)) {
code = terrno;
mError("taosArrayPush SVGroupHashInfo failed, code:%s", tstrerror(terrno));
taosArrayDestroy(pNew);
sdbRelease(pSdb, pVgroup);
return code;
}
if (NULL == pDbInfo) {
code = tSimpleHashPut(pDbVgroup, pVgroup->dbName, strlen(pVgroup->dbName) + 1, &dbInfo, sizeof(dbInfo));
if (code != 0) {
mError("tSimpleHashPut SDBVgHashInfo failed, code:%s", tstrerror(code));
taosArrayDestroy(pNew);
sdbRelease(pSdb, pVgroup);
return code;
}
pNew = NULL;
}
sdbRelease(pSdb, pVgroup);
}
return code;
}
static int32_t addVTableToVnode(SSHashObj* pVg, int32_t vvgId, uint64_t vuid, SRefColInfo* pCol, SStreamVBuildCtx* pCtx) {
int32_t code = TSDB_CODE_SUCCESS;
int32_t lino = 0;
SSHashObj* pNewVtable = NULL;
SArray* pNewOtable = NULL, *pTarOtable = NULL;
SColIdName col;
char vId[sizeof(int32_t) + sizeof(uint64_t)];
*(int32_t*)vId = vvgId;
*(uint64_t*)((int32_t*)vId + 1) = vuid;
pCtx->lastUid = vuid;
SSHashObj** pVtable = (SSHashObj**)tSimpleHashGet(pVg, vId, sizeof(vId));
if (NULL == pVtable) {
pNewVtable = (SSHashObj*)tSimpleHashInit(0, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY));
TSDB_CHECK_NULL(pNewVtable, code, lino, _return, terrno);
pNewOtable = taosArrayInit(4, sizeof(SColIdName));
TSDB_CHECK_NULL(pNewOtable, code, lino, _return, terrno);
tSimpleHashSetFreeFp(pNewVtable, tFreeStreamVtbOtbInfo);
col.colId = pCol->colId;
col.colName = taosStrdup(pCol->refColName);
TSDB_CHECK_NULL(col.colName, code, lino, _return, terrno);
TSDB_CHECK_NULL(taosArrayPush(pNewOtable, &col), code, lino, _return, terrno);
TSDB_CHECK_CODE(tSimpleHashPut(pNewVtable, pCol->refTableName, strlen(pCol->refTableName) + 1, &pNewOtable, POINTER_BYTES), lino, _return);
TSDB_CHECK_CODE(tSimpleHashPut(pVg, vId, sizeof(vId), &pNewVtable, POINTER_BYTES), lino, _return);
pCtx->lastVtable = pNewVtable;
pCtx->lastOtable = pNewOtable;
return code;
}
SArray** pOtable = tSimpleHashGet(*pVtable, pCol->refTableName, strlen(pCol->refTableName) + 1);
if (NULL == pOtable) {
pNewOtable = taosArrayInit(4, sizeof(SColIdName));
TSDB_CHECK_NULL(pNewOtable, code, lino, _return, terrno);
pTarOtable = pNewOtable;
} else {
pTarOtable = *pOtable;
}
col.colId = pCol->colId;
col.colName = taosStrdup(pCol->refColName);
TSDB_CHECK_NULL(col.colName, code, lino, _return, terrno);
TSDB_CHECK_NULL(taosArrayPush(pTarOtable, &col), code, lino, _return, terrno);
if (NULL == pOtable) {
TSDB_CHECK_CODE(tSimpleHashPut(*pVtable, pCol->refTableName, strlen(pCol->refTableName) + 1, &pNewOtable, POINTER_BYTES), lino, _return);
}
pCtx->lastVtable = *pVtable;
pCtx->lastOtable = pTarOtable;
_return:
if (code) {
mError("%s failed at line %d since %s", __func__, lino, tstrerror(code));
}
return code;
}
static int32_t addVgroupToRes(char* fDBName, int32_t vvgId, uint64_t vuid, SRefColInfo* pCol, SDBVgHashInfo* pDb, SSHashObj* pRes, SStreamVBuildCtx* pCtx) {
int32_t code = TSDB_CODE_SUCCESS;
int32_t lino = 0;
int32_t vgId = 0;
char dbVgId[TSDB_DB_NAME_LEN + 32];
SSHashObj *pTarVg = NULL, *pNewVg = NULL;
TSDB_CHECK_CODE(getTableVgId(pDb, 1, fDBName, &vgId, pCol->refColName), lino, _return);
snprintf(dbVgId, sizeof(dbVgId), "%s.%d", pCol->refDbName, vgId);
SSHashObj** pVg = (SSHashObj**)tSimpleHashGet(pRes, dbVgId, strlen(dbVgId) + 1);
if (NULL == pVg) {
pNewVg = tSimpleHashInit(10, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BIGINT));
TSDB_CHECK_NULL(pNewVg, code, lino, _return, terrno);
tSimpleHashSetFreeFp(pNewVg, tFreeStreamVtbVtbInfo);
pTarVg = pNewVg;
} else {
pTarVg = *pVg;
}
TSDB_CHECK_CODE(addVTableToVnode(pTarVg, vvgId, vuid, pCol, pCtx), lino, _return);
if (NULL == pVg) {
TSDB_CHECK_CODE(tSimpleHashPut(pRes, dbVgId, strlen(dbVgId) + 1, &pNewVg, POINTER_BYTES), lino, _return);
pNewVg = NULL;
}
pCtx->lastVg = pTarVg;
_return:
if (code) {
mError("%s failed at line %d since %s", __func__, lino, tstrerror(code));
}
destroyVSubtableVgHash(pNewVg);
return code;
}
static int32_t addRefColToMap(int32_t vvgId, uint64_t vuid, SRefColInfo* pCol, SSHashObj* pDbVgroups, SSHashObj* pRes, SStreamVBuildCtx* pCtx) {
int32_t code = TSDB_CODE_SUCCESS;
int32_t lino = 0;
bool isLastVtable = vuid == pCtx->lastUid;
SSHashObj* currOtable = NULL;
SColIdName col;
char fDBName[TSDB_DB_FNAME_LEN];
if (pCtx->lastCol && pCtx->lastCol->refDbName[0] == pCol->refDbName[0] && pCtx->lastCol->refTableName[0] == pCol->refTableName[0] &&
0 == strcmp(pCtx->lastCol->refDbName, pCol->refDbName) && 0 == strcmp(pCtx->lastCol->refTableName, pCol->refTableName)) {
if (isLastVtable) {
col.colId = pCol->colId;
col.colName = taosStrdup(pCol->refColName);
TSDB_CHECK_NULL(col.colName, code, lino, _return, terrno);
TSDB_CHECK_NULL(taosArrayPush(pCtx->lastOtable, &col), code, lino, _return, terrno);
return code;
}
TSDB_CHECK_CODE(addVTableToVnode(pCtx->lastVg, vvgId, vuid, pCol, pCtx), lino, _return);
return code;
}
snprintf(fDBName, sizeof(fDBName), "1.%s", pCol->refDbName);
SDBVgHashInfo* pDb = (SDBVgHashInfo*)tSimpleHashGet(pDbVgroups, fDBName, strlen(fDBName) + 1);
if (NULL == pDb) {
mError("refDb %s does not exist", pCol->refDbName);
code = TSDB_CODE_MND_DB_NOT_EXIST;
goto _return;
}
TSDB_CHECK_CODE(addVgroupToRes(fDBName, vvgId, vuid, pCol, pDb, pRes, pCtx), lino, _return);
pCtx->lastCol = pCol;
_return:
if (code) {
mError("%s failed at line %d since %s", __func__, lino, tstrerror(code));
}
return code;
}
static int32_t buildVSubtableMap(SMnode* pMnode, SArray* pVSubTables, SSHashObj** ppRes) {
int32_t code = 0;
int32_t lino = 0;
SSHashObj* pDbVgroups = tSimpleHashInit(20, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY));
if (NULL == pDbVgroups) {
mError("tSimpleHashInit failed, error:%s", tstrerror(terrno));
return terrno;
}
TAOS_CHECK_EXIT(buildDBVgroupsMap(pMnode, pDbVgroups));
*ppRes = tSimpleHashInit(100, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY));
if (NULL == *ppRes) {
code = terrno;
mError("tSimpleHashInit failed, error:%s", tstrerror(terrno));
goto _exit;
}
tSimpleHashSetFreeFp(*ppRes, tFreeStreamVtbDbVgInfo);
SStreamVBuildCtx ctx = {0};
int32_t vgNum = taosArrayGetSize(pVSubTables);
for (int32_t i = 0; i < vgNum; ++i) {
SVSubTablesRsp* pVgTbs = taosArrayGet(pVSubTables, i);
int32_t tbNum = taosArrayGetSize(pVgTbs->pTables);
for (int32_t n = 0; n < tbNum; ++n) {
SVCTableRefCols* pTb = (SVCTableRefCols*)taosArrayGetP(pVgTbs->pTables, n);
for (int32_t m = 0; m < pTb->numOfColRefs; ++m) {
SRefColInfo* pCol = pTb->refCols + m;
TAOS_CHECK_EXIT(addRefColToMap(pVgTbs->vgId, pTb->uid, pCol, pDbVgroups, *ppRes, &ctx));
}
}
}
_exit:
if (code) {
mError("%s failed at line %d since %s", __func__, lino, tstrerror(code));
}
destroyDbVgroupsHash(pDbVgroups);
return code;
}
static int32_t doScheduleStream(SStreamObj* pStream, SMnode* pMnode, SQueryPlan* pPlan, SEpSet* pEpset, SCMCreateStreamReq* pCreate) {
int32_t code = 0;
bool isVTableStream = (NULL != pCreate->pVSubTables);
int64_t skey = pCreate->lastTs;
SArray* pVerList = pCreate->pVgroupVerList;
SSdb* pSdb = pMnode->pSdb;
int32_t numOfPlanLevel = LIST_LENGTH(pPlan->pSubplans);
bool hasExtraSink = false;
bool externalTargetDB = strcmp(pStream->sourceDb, pStream->targetDb) != 0;
SSubplan* plan = NULL;
SDbObj* pDbObj = mndAcquireDb(pMnode, pStream->targetDb);
if (pDbObj == NULL) {
@ -746,7 +1372,16 @@ static int32_t doScheduleStream(SStreamObj* pStream, SMnode* pMnode, SQueryPlan*
return terrno;
}
if (numOfPlanLevel > 1 || externalTargetDB || multiTarget || pStream->fixedSinkVgId) {
if (pCreate->pVSubTables) {
code = buildVSubtableMap(pMnode, pCreate->pVSubTables, &pStream->pVTableMap);
if (TSDB_CODE_SUCCESS != code) {
mError("failed to buildVSubtableMap, code:%s", tstrerror(terrno));
return code;
}
}
if ((numOfPlanLevel > 1 && !isVTableStream) || (numOfPlanLevel > 2 && isVTableStream) || externalTargetDB ||
multiTarget || pStream->fixedSinkVgId) {
// add extra sink
hasExtraSink = true;
code = addSinkTask(pMnode, pStream, pEpset);
@ -757,20 +1392,57 @@ static int32_t doScheduleStream(SStreamObj* pStream, SMnode* pMnode, SQueryPlan*
pStream->totalLevel = numOfPlanLevel + hasExtraSink;
SSubplan* plan = getScanSubPlan(pPlan); // source plan
if (plan == NULL) {
code = TSDB_CODE_MND_RETURN_VALUE_NULL;
if (terrno != 0) code = terrno;
TAOS_RETURN(code);
}
int8_t hasAggTasks = (numOfPlanLevel > 1) ? 1 : 0; // task level is greater than 1, which means agg existing
code = addSourceTask(pMnode, plan, pStream, pEpset, skey, pVerList, (numOfPlanLevel == 1), hasAggTasks);
if (pStream->pVTableMap) {
code = addNewTaskList(pStream);
if (code) {
return code;
}
plan = getVTbScanSubPlan(pPlan);
if (plan == NULL) {
mError("fail to get vtable scan plan");
code = TSDB_CODE_MND_RETURN_VALUE_NULL;
if (terrno != 0) code = terrno;
TAOS_RETURN(code);
}
SArray** pSourceTaskList = taosArrayGetLast(pStream->pTaskList);
code = addNewTaskList(pStream);
if (code) {
return code;
}
code = addVTableMergeTask(pMnode, plan, pStream, pEpset, (numOfPlanLevel == 1), hasAggTasks, pCreate);
if (code) {
return code;
}
plan = getScanSubPlan(pPlan); // source plan
if (plan == NULL) {
code = TSDB_CODE_MND_RETURN_VALUE_NULL;
if (terrno != 0) code = terrno;
TAOS_RETURN(code);
}
SArray** pMergeTaskList = taosArrayGetLast(pStream->pTaskList);
code = addVTableSourceTask(pMnode, plan, pStream, pEpset, skey, pVerList, (numOfPlanLevel == 1), hasAggTasks,
pCreate, pStream->pVTableMap, *pSourceTaskList, *pMergeTaskList);
} else {
plan = getScanSubPlan(pPlan); // source plan
if (plan == NULL) {
code = TSDB_CODE_MND_RETURN_VALUE_NULL;
if (terrno != 0) code = terrno;
TAOS_RETURN(code);
}
code = addSourceTask(pMnode, plan, pStream, pEpset, skey, pVerList, (numOfPlanLevel == 1), hasAggTasks);
}
if (code != TSDB_CODE_SUCCESS) {
return code;
}
if (numOfPlanLevel == 1) {
if ((numOfPlanLevel == 1 && !isVTableStream)) {
bindSourceSink(pStream, pMnode, pStream->pTaskList, hasExtraSink);
if (needHistoryTask(pStream)) {
bindSourceSink(pStream, pMnode, pStream->pHTaskList, hasExtraSink);
@ -778,8 +1450,14 @@ static int32_t doScheduleStream(SStreamObj* pStream, SMnode* pMnode, SQueryPlan*
return TDB_CODE_SUCCESS;
}
if (numOfPlanLevel == 3) {
plan = getAggSubPlan(pPlan, 1); // middle agg plan
if (numOfPlanLevel == 2 && isVTableStream) {
bindVtableMergeSink(pStream, pMnode, pStream->pTaskList, hasExtraSink);
return TDB_CODE_SUCCESS;
}
if ((numOfPlanLevel == 3 && !isVTableStream) || (numOfPlanLevel == 4 && isVTableStream)) {
int32_t idx = isVTableStream ? 2 : 1;
plan = getAggSubPlan(pPlan, idx); // middle agg plan
if (plan == NULL) {
code = TSDB_CODE_MND_RETURN_VALUE_NULL;
if (terrno != 0) code = terrno;
@ -844,10 +1522,10 @@ static int32_t doScheduleStream(SStreamObj* pStream, SMnode* pMnode, SQueryPlan*
TAOS_RETURN(code);
}
int32_t mndScheduleStream(SMnode* pMnode, SStreamObj* pStream, int64_t skey, SArray* pVgVerList) {
int32_t mndScheduleStream(SMnode* pMnode, SStreamObj* pStream, SCMCreateStreamReq* pCreate) {
int32_t code = 0;
SQueryPlan* pPlan = qStringToQueryPlan(pStream->physicalPlan);
if (pPlan == NULL) {
pStream->pPlan = qStringToQueryPlan(pStream->physicalPlan);
if (pStream->pPlan == NULL) {
code = TSDB_CODE_QRY_INVALID_INPUT;
TAOS_RETURN(code);
}
@ -855,8 +1533,7 @@ int32_t mndScheduleStream(SMnode* pMnode, SStreamObj* pStream, int64_t skey, SAr
SEpSet mnodeEpset = {0};
mndGetMnodeEpSet(pMnode, &mnodeEpset);
code = doScheduleStream(pStream, pMnode, pPlan, &mnodeEpset, skey, pVgVerList);
qDestroyQueryPlan(pPlan);
code = doScheduleStream(pStream, pMnode, pStream->pPlan, &mnodeEpset, pCreate);
TAOS_RETURN(code);
}

View File

@ -89,7 +89,7 @@ int32_t mndInitSma(SMnode *pMnode) {
.deleteFp = (SdbDeleteFp)mndSmaActionDelete,
};
mndSetMsgHandle(pMnode, TDMT_MND_CREATE_SMA, mndProcessCreateSmaReq);
// mndSetMsgHandle(pMnode, TDMT_MND_CREATE_SMA, mndProcessCreateSmaReq);
mndSetMsgHandle(pMnode, TDMT_MND_DROP_SMA, mndProcessDropIdxReq);
mndSetMsgHandle(pMnode, TDMT_VND_CREATE_SMA_RSP, mndTransProcessRsp);
mndSetMsgHandle(pMnode, TDMT_VND_DROP_SMA_RSP, mndTransProcessRsp);
@ -590,6 +590,7 @@ static void mndDestroySmaObj(SSmaObj *pSmaObj) {
}
}
#if 0
static int32_t mndCreateSma(SMnode *pMnode, SRpcMsg *pReq, SMCreateSmaReq *pCreate, SDbObj *pDb, SStbObj *pStb,
const char *streamName) {
int32_t code = 0;
@ -760,6 +761,7 @@ _OVER:
mndTransDrop(pTrans);
TAOS_RETURN(code);
}
#endif
static int32_t mndCheckCreateSmaReq(SMCreateSmaReq *pCreate) {
int32_t code = TSDB_CODE_MND_INVALID_SMA_OPTION;
@ -799,6 +801,7 @@ static int32_t mndGetStreamNameFromSmaName(char *streamName, char *smaName) {
return TSDB_CODE_SUCCESS;
}
#if 0
static int32_t mndProcessCreateSmaReq(SRpcMsg *pReq) {
SMnode *pMnode = pReq->info.node;
int32_t code = -1;
@ -889,6 +892,7 @@ _OVER:
TAOS_RETURN(code);
}
#endif
static int32_t mndSetDropSmaRedoLogs(SMnode *pMnode, STrans *pTrans, SSmaObj *pSma) {
int32_t code = 0;

View File

@ -1001,15 +1001,18 @@ static int32_t mndProcessCreateStreamReq(SRpcMsg *pReq) {
// schedule stream task for stream obj
if (!buildEmptyStream) {
code = mndScheduleStream(pMnode, &streamObj, createReq.lastTs, createReq.pVgroupVerList);
code = mndScheduleStream(pMnode, &streamObj, &createReq);
if (code != TSDB_CODE_SUCCESS && code != TSDB_CODE_ACTION_IN_PROGRESS) {
mError("stream:%s, failed to schedule since %s", createReq.name, tstrerror(code));
mndTransDrop(pTrans);
goto _OVER;
}
// add notify info into all stream tasks
code = addStreamNotifyInfo(&createReq, &streamObj);
if (code != TSDB_CODE_SUCCESS) {
mError("stream:%s failed to add stream notify info since %s", createReq.name, tstrerror(code));
mndTransDrop(pTrans);
goto _OVER;
}

View File

@ -207,6 +207,21 @@ typedef struct SIdInfo {
int32_t index;
} SIdInfo;
typedef struct SVTColInfo {
int32_t vColId; // column id of virtual table
int32_t pColId; // column id of physical table
int64_t pTbUid; // uid of physical table
} SVTColInfo;
typedef struct SVTSourceScanInfo {
SHashObj *pVirtualTables; // source column info of each vtable column. key: vtUid, value: SArray<SVTColInfo>*
SHashObj *pPhysicalTables; // set of vtables for each ptable. Key: ptUid, value: SArray<vtUid>*
SLRUCache *pPhyTblSchemaCache; // cache for physical table schema
int32_t nextVirtualTableIdx; // index in the value of pVirtualTables
uint64_t metaFetch;
uint64_t cacheHit;
} SVTSourceScanInfo;
typedef struct STqReader {
SPackedData msg;
SSubmitReq2 submit;
@ -224,6 +239,7 @@ typedef struct STqReader {
int64_t lastTs;
bool hasPrimaryKey;
SExtSchema *extSchema;
SVTSourceScanInfo vtSourceScanInfo;
} STqReader;
STqReader *tqReaderOpen(SVnode *pVnode);
@ -232,8 +248,8 @@ void tqReaderClose(STqReader *);
bool tqGetTablePrimaryKey(STqReader *pReader);
void tqSetTablePrimaryKey(STqReader *pReader, int64_t uid);
void tqReaderSetColIdList(STqReader *pReader, SArray *pColIdList);
void tqReaderSetTbUidList(STqReader *pReader, const SArray *tbUidList, const char *id);
int32_t tqReaderSetColIdList(STqReader *pReader, SArray *pColIdList, const char *id);
int32_t tqReaderSetTbUidList(STqReader *pReader, const SArray *tbUidList, const char *id);
void tqReaderAddTbUidList(STqReader *pReader, const SArray *pTableUidList);
void tqReaderRemoveTbUidList(STqReader *pReader, const SArray *tbUidList);
@ -255,6 +271,12 @@ int32_t tqRetrieveDataBlock(STqReader *pReader, SSDataBlock **pRes, const char *
int32_t tqRetrieveTaosxBlock(STqReader *pReader, SMqDataRsp* pRsp, SArray *blocks, SArray *schemas, SSubmitTbData **pSubmitTbDataRet, SArray* rawList, int8_t fetchMeta);
int32_t tqGetStreamExecInfo(SVnode *pVnode, int64_t streamId, int64_t *pDelay, bool *fhFinished);
int32_t tqReaderSetVtableInfo(STqReader *pReader, void *vnode, void *pAPI, SSHashObj *pVtableInfos,
SSDataBlock **ppResBlock, const char *idstr);
int32_t tqRetrieveVTableDataBlock(STqReader *pReader, SSDataBlock **pRes, const char *idstr);
bool tqNextVTableSourceBlockImpl(STqReader *pReader, const char *idstr);
bool tqReaderIsQueriedSourceTable(STqReader *pReader, uint64_t uid);
// sma
int32_t smaGetTSmaDays(SVnodeCfg *pCfg, void *pCont, uint32_t contLen, int32_t *days);

View File

@ -127,6 +127,7 @@ void vnodeQueryClose(SVnode* pVnode);
int32_t vnodeGetTableMeta(SVnode* pVnode, SRpcMsg* pMsg, bool direct);
int vnodeGetTableCfg(SVnode* pVnode, SRpcMsg* pMsg, bool direct);
int32_t vnodeGetBatchMeta(SVnode* pVnode, SRpcMsg* pMsg);
int32_t vnodeGetVSubtablesMeta(SVnode *pVnode, SRpcMsg *pMsg);
int32_t vnodeGetStreamProgress(SVnode* pVnode, SRpcMsg* pMsg, bool direct);
// vnodeCommit.c

View File

@ -16,6 +16,8 @@
#include "tmsg.h"
#include "tq.h"
static int32_t tqCollectPhysicalTables(STqReader* pReader, const char* idstr);
bool isValValidForTable(STqHandle* pHandle, SWalCont* pHead) {
if (pHandle == NULL || pHead == NULL) {
return false;
@ -198,7 +200,7 @@ bool isValValidForTable(STqHandle* pHandle, SWalCont* pHead) {
end:
tDecoderClear(&dcoder);
bool tmp = tbSuid == realTbSuid;
tqDebug("%s suid:%"PRId64" realSuid:%"PRId64" return:%d", __FUNCTION__, tbSuid, realTbSuid, tmp);
tqDebug("%s suid:%" PRId64 " realSuid:%" PRId64 " return:%d", __FUNCTION__, tbSuid, realTbSuid, tmp);
return tmp;
}
@ -264,7 +266,8 @@ int32_t tqFetchLog(STQ* pTq, STqHandle* pHandle, int64_t* fetchOffset, uint64_t
END:
*fetchOffset = offset;
tqDebug("vgId:%d, end to fetch wal, code:%d , index:%" PRId64 ", last:%" PRId64 " commit:%" PRId64 ", applied:%" PRId64 ", 0x%" PRIx64,
tqDebug("vgId:%d, end to fetch wal, code:%d , index:%" PRId64 ", last:%" PRId64 " commit:%" PRId64
", applied:%" PRId64 ", 0x%" PRIx64,
vgId, code, offset, lastVer, committedVer, appliedVer, id);
return code;
}
@ -277,7 +280,7 @@ bool tqGetTablePrimaryKey(STqReader* pReader) {
}
void tqSetTablePrimaryKey(STqReader* pReader, int64_t uid) {
tqDebug("%s:%p uid:%"PRId64, __FUNCTION__ , pReader, uid);
tqDebug("%s:%p uid:%" PRId64, __FUNCTION__, pReader, uid);
if (pReader == NULL) {
return;
@ -292,7 +295,7 @@ void tqSetTablePrimaryKey(STqReader* pReader, int64_t uid) {
}
STqReader* tqReaderOpen(SVnode* pVnode) {
tqDebug("%s:%p", __FUNCTION__ , pVnode);
tqDebug("%s:%p", __FUNCTION__, pVnode);
if (pVnode == NULL) {
return NULL;
}
@ -324,7 +327,7 @@ STqReader* tqReaderOpen(SVnode* pVnode) {
}
void tqReaderClose(STqReader* pReader) {
tqDebug("%s:%p", __FUNCTION__ , pReader);
tqDebug("%s:%p", __FUNCTION__, pReader);
if (pReader == NULL) return;
// close wal reader
@ -345,6 +348,10 @@ void tqReaderClose(STqReader* pReader) {
blockDataDestroy(pReader->pResBlock);
taosHashCleanup(pReader->tbIdHash);
tDestroySubmitReq(&pReader->submit, TSDB_MSG_FLG_DECODE);
taosHashCleanup(pReader->vtSourceScanInfo.pVirtualTables);
taosHashCleanup(pReader->vtSourceScanInfo.pPhysicalTables);
taosLRUCacheCleanup(pReader->vtSourceScanInfo.pPhyTblSchemaCache);
taosMemoryFree(pReader);
}
@ -393,8 +400,8 @@ int32_t extractMsgFromWal(SWalReader* pReader, void** pItem, int64_t maxVer, con
return code;
}
} else if (pCont->msgType == TDMT_VND_DELETE) {
void* pBody = POINTER_SHIFT(pCont->body, sizeof(SMsgHead));
int32_t len = pCont->bodyLen - sizeof(SMsgHead);
void* pBody = POINTER_SHIFT(pCont->body, sizeof(SMsgHead));
int32_t len = pCont->bodyLen - sizeof(SMsgHead);
EStreamType blockType = STREAM_DELETE_DATA;
code = tqExtractDelDataBlock(pBody, len, ver, (void**)pItem, 0, blockType);
if (code == TSDB_CODE_SUCCESS) {
@ -412,14 +419,14 @@ int32_t extractMsgFromWal(SWalReader* pReader, void** pItem, int64_t maxVer, con
}
} else if (pCont->msgType == TDMT_VND_DROP_TABLE && pReader->cond.scanDropCtb) {
void* pBody = POINTER_SHIFT(pCont->body, sizeof(SMsgHead));
void* pBody = POINTER_SHIFT(pCont->body, sizeof(SMsgHead));
int32_t len = pCont->bodyLen - sizeof(SMsgHead);
code = tqExtractDropCtbDataBlock(pBody, len, ver, (void**)pItem, 0);
if (TSDB_CODE_SUCCESS == code) {
if (!*pItem) {
continue;
} else {
tqDebug("s-task:%s drop ctb msg extract from WAL, len:%d, ver:%"PRId64, id, len, ver);
tqDebug("s-task:%s drop ctb msg extract from WAL, len:%d, ver:%" PRId64, id, len, ver);
}
} else {
terrno = code;
@ -515,13 +522,12 @@ int32_t tqReaderSetSubmitMsg(STqReader* pReader, void* msgStr, int32_t msgLen, i
return code;
}
void tqReaderClearSubmitMsg(STqReader *pReader) {
void tqReaderClearSubmitMsg(STqReader* pReader) {
tDestroySubmitReq(&pReader->submit, TSDB_MSG_FLG_DECODE);
pReader->nextBlk = 0;
pReader->msg.msgStr = NULL;
}
SWalReader* tqGetWalReader(STqReader* pReader) {
if (pReader == NULL) {
return NULL;
@ -564,10 +570,10 @@ bool tqNextBlockImpl(STqReader* pReader, const char* idstr) {
}
tqReaderClearSubmitMsg(pReader);
tqTrace("iterator data block end, total block num:%d, uid:%"PRId64, blockSz, uid);
tqTrace("iterator data block end, total block num:%d, uid:%" PRId64, blockSz, uid);
END:
tqTrace("%s:%d return:%s, uid:%"PRId64, __FUNCTION__, lino, code?"true":"false", uid);
tqTrace("%s:%d return:%s, uid:%" PRId64, __FUNCTION__, lino, code ? "true" : "false", uid);
return code;
}
@ -591,10 +597,10 @@ bool tqNextDataBlockFilterOut(STqReader* pReader, SHashObj* filterOutUids) {
pReader->nextBlk++;
}
tqReaderClearSubmitMsg(pReader);
tqTrace("iterator data block end, total block num:%d, uid:%"PRId64, blockSz, uid);
tqTrace("iterator data block end, total block num:%d, uid:%" PRId64, blockSz, uid);
END:
tqTrace("%s:%d get data:%s, uid:%"PRId64, __FUNCTION__, lino, code?"true":"false", uid);
tqTrace("%s:%d get data:%s, uid:%" PRId64, __FUNCTION__, lino, code ? "true" : "false", uid);
return code;
}
@ -639,15 +645,15 @@ static int32_t buildResSDataBlock(STqReader* pReader, SSchemaWrapper* pSchema, c
}
SSDataBlock* pBlock = pReader->pResBlock;
if (blockDataGetNumOfCols(pBlock) > 0) {
blockDataDestroy(pBlock);
int32_t code = createDataBlock(&pReader->pResBlock);
if (code) {
return code;
}
pBlock = pReader->pResBlock;
blockDataDestroy(pBlock);
int32_t code = createDataBlock(&pReader->pResBlock);
if (code) {
return code;
}
pBlock = pReader->pResBlock;
pBlock->info.id.uid = pReader->cachedSchemaUid;
pBlock->info.version = pReader->msg.ver;
pBlock->info.id.uid = pReader->cachedSchemaUid;
pBlock->info.version = pReader->msg.ver;
}
int32_t numOfCols = taosArrayGetSize(pColIdList);
@ -947,7 +953,8 @@ static int32_t tqProcessColData(STqReader* pReader, SSubmitTbData* pSubmitTbData
TQ_NULL_GO_TO_END(pCol);
int32_t numOfRows = pCol->nVal;
int32_t numOfCols = taosArrayGetSize(pCols);
tqTrace("vgId:%d, tqProcessColData start, col num: %d, rows:%d", pReader->pWalReader->pWal->cfg.vgId, numOfCols, numOfRows);
tqTrace("vgId:%d, tqProcessColData start, col num: %d, rows:%d", pReader->pWalReader->pWal->cfg.vgId, numOfCols,
numOfRows);
for (int32_t i = 0; i < numOfRows; i++) {
bool buildNew = false;
@ -987,7 +994,8 @@ static int32_t tqProcessColData(STqReader* pReader, SSubmitTbData* pSubmitTbData
}
SSDataBlock* pLastBlock = taosArrayGetLast(blocks);
pLastBlock->info.rows = curRow - lastRow;
tqTrace("vgId:%d, tqProcessColData end, col num: %d, rows:%d, block num:%d", pReader->pWalReader->pWal->cfg.vgId, numOfCols, numOfRows, (int)taosArrayGetSize(blocks));
tqTrace("vgId:%d, tqProcessColData end, col num: %d, rows:%d, block num:%d", pReader->pWalReader->pWal->cfg.vgId,
numOfCols, numOfRows, (int)taosArrayGetSize(blocks));
END:
if (code != TSDB_CODE_SUCCESS) {
tqError("vgId:%d, process col data failed, code:%d", pReader->pWalReader->pWal->cfg.vgId, code);
@ -1049,7 +1057,8 @@ int32_t tqProcessRowData(STqReader* pReader, SSubmitTbData* pSubmitTbData, SArra
SSDataBlock* pLastBlock = taosArrayGetLast(blocks);
pLastBlock->info.rows = curRow - lastRow;
tqTrace("vgId:%d, tqProcessRowData end, rows:%d, block num:%d", pReader->pWalReader->pWal->cfg.vgId, numOfRows, (int)taosArrayGetSize(blocks));
tqTrace("vgId:%d, tqProcessRowData end, rows:%d, block num:%d", pReader->pWalReader->pWal->cfg.vgId, numOfRows,
(int)taosArrayGetSize(blocks));
END:
if (code != TSDB_CODE_SUCCESS) {
tqError("vgId:%d, process row data failed, code:%d", pReader->pWalReader->pWal->cfg.vgId, code);
@ -1059,7 +1068,7 @@ END:
return code;
}
static int32_t buildCreateTbInfo(SMqDataRsp* pRsp, SVCreateTbReq* pCreateTbReq){
static int32_t buildCreateTbInfo(SMqDataRsp* pRsp, SVCreateTbReq* pCreateTbReq) {
int32_t code = 0;
int32_t lino = 0;
void* createReq = NULL;
@ -1089,15 +1098,16 @@ static int32_t buildCreateTbInfo(SMqDataRsp* pRsp, SVCreateTbReq* pCreateTbReq){
pRsp->createTableNum++;
tqTrace("build create table info msg success");
END:
if (code != 0){
END:
if (code != 0) {
tqError("%s failed at %d, failed to build create table info msg:%s", __FUNCTION__, lino, tstrerror(code));
taosMemoryFree(createReq);
}
return code;
}
int32_t tqRetrieveTaosxBlock(STqReader* pReader, SMqDataRsp* pRsp, SArray* blocks, SArray* schemas, SSubmitTbData** pSubmitTbDataRet, SArray* rawList, int8_t fetchMeta) {
int32_t tqRetrieveTaosxBlock(STqReader* pReader, SMqDataRsp* pRsp, SArray* blocks, SArray* schemas,
SSubmitTbData** pSubmitTbDataRet, SArray* rawList, int8_t fetchMeta) {
tqTrace("tq reader retrieve data block msg pointer:%p, index:%d", pReader->msg.msgStr, pReader->nextBlk);
SSubmitTbData* pSubmitTbData = taosArrayGet(pReader->submit.aSubmitTbData, pReader->nextBlk);
if (pSubmitTbData == NULL) {
@ -1111,13 +1121,13 @@ int32_t tqRetrieveTaosxBlock(STqReader* pReader, SMqDataRsp* pRsp, SArray* block
if (fetchMeta == ONLY_META) {
if (pSubmitTbData->pCreateTbReq != NULL) {
if (pRsp->createTableReq == NULL){
if (pRsp->createTableReq == NULL) {
pRsp->createTableReq = taosArrayInit(0, POINTER_BYTES);
if (pRsp->createTableReq == NULL){
if (pRsp->createTableReq == NULL) {
return terrno;
}
}
if (taosArrayPush(pRsp->createTableReq, &pSubmitTbData->pCreateTbReq) == NULL){
if (taosArrayPush(pRsp->createTableReq, &pSubmitTbData->pCreateTbReq) == NULL) {
return terrno;
}
pSubmitTbData->pCreateTbReq = NULL;
@ -1145,7 +1155,7 @@ int32_t tqRetrieveTaosxBlock(STqReader* pReader, SMqDataRsp* pRsp, SArray* block
return code;
}
} else if (rawList != NULL) {
if (taosArrayPush(schemas, &pReader->pSchemaWrapper) == NULL){
if (taosArrayPush(schemas, &pReader->pSchemaWrapper) == NULL) {
return terrno;
}
pReader->pSchemaWrapper = NULL;
@ -1159,16 +1169,17 @@ int32_t tqRetrieveTaosxBlock(STqReader* pReader, SMqDataRsp* pRsp, SArray* block
}
}
void tqReaderSetColIdList(STqReader* pReader, SArray* pColIdList) {
if (pReader == NULL){
return;
int32_t tqReaderSetColIdList(STqReader* pReader, SArray* pColIdList, const char* id) {
if (pReader == NULL) {
return TSDB_CODE_SUCCESS;
}
pReader->pColIdList = pColIdList;
return tqCollectPhysicalTables(pReader, id);
}
void tqReaderSetTbUidList(STqReader* pReader, const SArray* tbUidList, const char* id) {
int32_t tqReaderSetTbUidList(STqReader* pReader, const SArray* tbUidList, const char* id) {
if (pReader == NULL || tbUidList == NULL) {
return;
return TSDB_CODE_SUCCESS;
}
if (pReader->tbIdHash) {
taosHashClear(pReader->tbIdHash);
@ -1176,7 +1187,7 @@ void tqReaderSetTbUidList(STqReader* pReader, const SArray* tbUidList, const cha
pReader->tbIdHash = taosHashInit(64, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BIGINT), true, HASH_ENTRY_LOCK);
if (pReader->tbIdHash == NULL) {
tqError("s-task:%s failed to init hash table", id);
return;
return terrno;
}
}
@ -1189,6 +1200,7 @@ void tqReaderSetTbUidList(STqReader* pReader, const SArray* tbUidList, const cha
}
tqDebug("s-task:%s %d tables are set to be queried target table", id, (int32_t)taosArrayGetSize(tbUidList));
return tqCollectPhysicalTables(pReader, id);
}
void tqReaderAddTbUidList(STqReader* pReader, const SArray* pTableUidList) {
@ -1325,3 +1337,535 @@ int32_t tqUpdateTbUidList(STQ* pTq, const SArray* tbUidList, bool isAdd) {
streamMetaWUnLock(pTq->pStreamMeta);
return 0;
}
static void destroySourceScanTables(void* ptr) {
SArray** pTables = ptr;
if (pTables && *pTables) {
taosArrayDestroy(*pTables);
*pTables = NULL;
}
}
static int32_t compareSVTColInfo(const void* p1, const void* p2) {
SVTColInfo* pCol1 = (SVTColInfo*)p1;
SVTColInfo* pCol2 = (SVTColInfo*)p2;
if (pCol1->vColId == pCol2->vColId) {
return 0;
} else if (pCol1->vColId < pCol2->vColId) {
return -1;
} else {
return 1;
}
}
int32_t tqReaderSetVtableInfo(STqReader* pReader, void* vnode, void* ptr, SSHashObj* pVtableInfos,
SSDataBlock** ppResBlock, const char* idstr) {
int32_t code = TSDB_CODE_SUCCESS;
int32_t lino = 0;
SStorageAPI* pAPI = ptr;
SVTSourceScanInfo* pScanInfo = NULL;
SHashObj* pVirtualTables = NULL;
SMetaReader metaReader = {0};
SVTColInfo colInfo = {0};
SSchemaWrapper* schema = NULL;
TSDB_CHECK_NULL(pReader, code, lino, _end, TSDB_CODE_INVALID_PARA);
TSDB_CHECK_NULL(vnode, code, lino, _end, TSDB_CODE_INVALID_PARA);
TSDB_CHECK_NULL(pAPI, code, lino, _end, TSDB_CODE_INVALID_PARA);
pScanInfo = &pReader->vtSourceScanInfo;
taosHashCleanup(pScanInfo->pVirtualTables);
pScanInfo->pVirtualTables = NULL;
if (tSimpleHashGetSize(pVtableInfos) == 0) {
goto _end;
}
pVirtualTables = taosHashInit(64, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BIGINT), false, HASH_NO_LOCK);
TSDB_CHECK_NULL(pVirtualTables, code, lino, _end, terrno);
taosHashSetFreeFp(pVirtualTables, destroySourceScanTables);
int32_t iter = 0;
void* px = tSimpleHashIterate(pVtableInfos, NULL, &iter);
while (px != NULL) {
int64_t vTbUid = *(int64_t*)tSimpleHashGetKey(px, NULL);
SArray* pColInfos = taosArrayInit(8, sizeof(SVTColInfo));
TSDB_CHECK_NULL(pColInfos, code, lino, _end, terrno);
code = taosHashPut(pVirtualTables, &vTbUid, sizeof(int64_t), &pColInfos, POINTER_BYTES);
TSDB_CHECK_CODE(code, lino, _end);
SSHashObj* pPhysicalTables = *(SSHashObj**)px;
int32_t iterIn = 0;
void* pxIn = tSimpleHashIterate(pPhysicalTables, NULL, &iterIn);
while (pxIn != NULL) {
char* physicalTableName = tSimpleHashGetKey(pxIn, NULL);
pAPI->metaReaderFn.clearReader(&metaReader);
pAPI->metaReaderFn.initReader(&metaReader, vnode, META_READER_LOCK, &pAPI->metaFn);
code = pAPI->metaReaderFn.getTableEntryByName(&metaReader, physicalTableName);
TSDB_CHECK_CODE(code, lino, _end);
pAPI->metaReaderFn.readerReleaseLock(&metaReader);
colInfo.pTbUid = metaReader.me.uid;
switch (metaReader.me.type) {
case TSDB_CHILD_TABLE: {
int64_t suid = metaReader.me.ctbEntry.suid;
pAPI->metaReaderFn.clearReader(&metaReader);
pAPI->metaReaderFn.initReader(&metaReader, vnode, META_READER_LOCK, &pAPI->metaFn);
code = pAPI->metaReaderFn.getTableEntryByUid(&metaReader, suid);
TSDB_CHECK_CODE(code, lino, _end);
pAPI->metaReaderFn.readerReleaseLock(&metaReader);
schema = &metaReader.me.stbEntry.schemaRow;
break;
}
case TSDB_NORMAL_TABLE: {
schema = &metaReader.me.ntbEntry.schemaRow;
break;
}
default: {
tqError("invalid table type: %d", metaReader.me.type);
code = TSDB_CODE_INVALID_PARA;
TSDB_CHECK_CODE(code, lino, _end);
}
}
SArray* pCols = *(SArray**)pxIn;
int32_t ncols = taosArrayGetSize(pCols);
for (int32_t i = 0; i < ncols; ++i) {
SColIdName* pCol = taosArrayGet(pCols, i);
colInfo.vColId = pCol->colId;
for (int32_t j = 0; j < schema->nCols; ++j) {
if (strncmp(pCol->colName, schema->pSchema[j].name, strlen(schema->pSchema[j].name)) == 0) {
colInfo.pColId = schema->pSchema[j].colId;
void* px = taosArrayPush(pColInfos, &colInfo);
TSDB_CHECK_NULL(px, code, lino, _end, terrno);
break;
}
}
}
taosArraySort(pColInfos, compareSVTColInfo);
pxIn = tSimpleHashIterate(pPhysicalTables, pxIn, &iterIn);
}
px = tSimpleHashIterate(pVtableInfos, px, &iter);
}
pScanInfo->pVirtualTables = pVirtualTables;
pVirtualTables = NULL;
// set the result data block
if (pReader->pResBlock) {
blockDataDestroy(pReader->pResBlock);
}
pReader->pResBlock = *ppResBlock;
*ppResBlock = NULL;
// update reader callback for vtable source scan
pAPI->tqReaderFn.tqRetrieveBlock = tqRetrieveVTableDataBlock;
pAPI->tqReaderFn.tqNextBlockImpl = tqNextVTableSourceBlockImpl;
pAPI->tqReaderFn.tqReaderIsQueriedTable = tqReaderIsQueriedSourceTable;
_end:
if (code != TSDB_CODE_SUCCESS) {
tqError("%s failed at line %d since %s, id: %s", __func__, lino, tstrerror(code), idstr);
}
pAPI->metaReaderFn.clearReader(&metaReader);
if (pVirtualTables != NULL) {
taosHashCleanup(pVirtualTables);
}
return code;
}
static int32_t tqCollectPhysicalTables(STqReader* pReader, const char* idstr) {
int32_t code = TSDB_CODE_SUCCESS;
int32_t lino = 0;
SVTSourceScanInfo* pScanInfo = NULL;
SHashObj* pVirtualTables = NULL;
SHashObj* pPhysicalTables = NULL;
void* pIter = NULL;
void* px = NULL;
TSDB_CHECK_NULL(pReader, code, lino, _end, TSDB_CODE_INVALID_PARA);
pScanInfo = &pReader->vtSourceScanInfo;
taosHashCleanup(pScanInfo->pPhysicalTables);
pScanInfo->pPhysicalTables = NULL;
taosLRUCacheCleanup(pScanInfo->pPhyTblSchemaCache);
pScanInfo->pPhyTblSchemaCache = NULL;
pScanInfo->nextVirtualTableIdx = -1;
pScanInfo->metaFetch = 0;
pScanInfo->cacheHit = 0;
pVirtualTables = pScanInfo->pVirtualTables;
if (taosHashGetSize(pVirtualTables) == 0 || taosHashGetSize(pReader->tbIdHash) == 0 ||
taosArrayGetSize(pReader->pColIdList) == 0) {
goto _end;
}
pPhysicalTables = taosHashInit(64, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BIGINT), false, HASH_NO_LOCK);
TSDB_CHECK_NULL(pPhysicalTables, code, lino, _end, terrno);
taosHashSetFreeFp(pPhysicalTables, destroySourceScanTables);
pIter = taosHashIterate(pReader->tbIdHash, NULL);
while (pIter != NULL) {
int64_t vTbUid = *(int64_t*)taosHashGetKey(pIter, NULL);
px = taosHashGet(pVirtualTables, &vTbUid, sizeof(int64_t));
TSDB_CHECK_NULL(px, code, lino, _end, terrno);
SArray* pColInfos = *(SArray**)px;
TSDB_CHECK_NULL(pColInfos, code, lino, _end, TSDB_CODE_INTERNAL_ERROR);
// Traverse all required columns and collect corresponding physical tables
int32_t nColInfos = taosArrayGetSize(pColInfos);
int32_t nOutputCols = taosArrayGetSize(pReader->pColIdList);
for (int32_t i = 0, j = 0; i < nColInfos && j < nOutputCols;) {
SVTColInfo* pCol = taosArrayGet(pColInfos, i);
col_id_t colIdNeed = *(col_id_t*)taosArrayGet(pReader->pColIdList, j);
if (pCol->vColId < colIdNeed) {
i++;
} else if (pCol->vColId > colIdNeed) {
j++;
} else {
SArray* pRelatedVTs = NULL;
px = taosHashGet(pPhysicalTables, &pCol->pTbUid, sizeof(int64_t));
if (px == NULL) {
pRelatedVTs = taosArrayInit(8, sizeof(int64_t));
TSDB_CHECK_NULL(pRelatedVTs, code, lino, _end, terrno);
code = taosHashPut(pPhysicalTables, &pCol->pTbUid, sizeof(int64_t), &pRelatedVTs, POINTER_BYTES);
if (code != TSDB_CODE_SUCCESS) {
taosArrayDestroy(pRelatedVTs);
TSDB_CHECK_CODE(code, lino, _end);
}
} else {
pRelatedVTs = *(SArray**)px;
}
if (taosArrayGetSize(pRelatedVTs) == 0 || *(int64_t*)taosArrayGetLast(pRelatedVTs) != vTbUid) {
px = taosArrayPush(pRelatedVTs, &vTbUid);
TSDB_CHECK_NULL(px, code, lino, _end, terrno);
}
i++;
j++;
}
}
pIter = taosHashIterate(pReader->tbIdHash, pIter);
}
pScanInfo->pPhysicalTables = pPhysicalTables;
pPhysicalTables = NULL;
if (taosHashGetSize(pScanInfo->pPhysicalTables) > 0) {
pScanInfo->pPhyTblSchemaCache = taosLRUCacheInit(1024 * 128, -1, .5);
TSDB_CHECK_NULL(pScanInfo->pPhyTblSchemaCache, code, lino, _end, terrno);
}
_end:
if (code != TSDB_CODE_SUCCESS) {
tqError("%s failed at line %d since %s, id: %s", __func__, lino, tstrerror(code), idstr);
}
if (pIter != NULL) {
taosHashCancelIterate(pReader->tbIdHash, pIter);
}
if (pPhysicalTables != NULL) {
taosHashCleanup(pPhysicalTables);
}
return code;
}
static void freeTableSchemaCache(const void* key, size_t keyLen, void* value, void* ud) {
if (value) {
SSchemaWrapper** ppSchemaWrapper = value;
tDeleteSchemaWrapper(*ppSchemaWrapper);
*ppSchemaWrapper = NULL;
}
}
int32_t tqRetrieveVTableDataBlock(STqReader* pReader, SSDataBlock** pRes, const char* idstr) {
int32_t code = TSDB_CODE_SUCCESS;
int32_t lino = 0;
SVTSourceScanInfo* pScanInfo = NULL;
SSubmitTbData* pSubmitTbData = NULL;
SSDataBlock* pBlock = NULL;
void* px = NULL;
int64_t vTbUid = 0;
int64_t pTbUid = 0;
LRUHandle* h = NULL;
STSchema* pPhyTblSchema = NULL;
TSDB_CHECK_NULL(pReader, code, lino, _end, TSDB_CODE_INVALID_PARA);
TSDB_CHECK_NULL(pRes, code, lino, _end, TSDB_CODE_INVALID_PARA);
pScanInfo = &pReader->vtSourceScanInfo;
tqDebug("tq reader retrieve vtable data block from %p, nextBlk:%d, vtbIdx:%d, id:%s", pReader->msg.msgStr,
pReader->nextBlk, pScanInfo->nextVirtualTableIdx, idstr);
*pRes = NULL;
pBlock = pReader->pResBlock;
blockDataCleanup(pBlock);
pSubmitTbData = taosArrayGet(pReader->submit.aSubmitTbData, pReader->nextBlk);
TSDB_CHECK_NULL(pSubmitTbData, code, lino, _end, terrno);
pReader->lastTs = pSubmitTbData->ctimeMs;
pTbUid = pSubmitTbData->uid;
px = taosHashGet(pScanInfo->pPhysicalTables, &pTbUid, sizeof(int64_t));
TSDB_CHECK_NULL(px, code, lino, _end, terrno);
SArray* pRelatedVTs = *(SArray**)px;
vTbUid = *(int64_t*)taosArrayGet(pRelatedVTs, pScanInfo->nextVirtualTableIdx);
px = taosHashGet(pScanInfo->pVirtualTables, &vTbUid, sizeof(int64_t));
TSDB_CHECK_NULL(px, code, lino, _end, terrno);
SArray* pColInfos = *(SArray**)px;
TSDB_CHECK_NULL(pColInfos, code, lino, _end, TSDB_CODE_INTERNAL_ERROR);
int32_t nColInfos = taosArrayGetSize(pColInfos);
int32_t nOutputCols = taosArrayGetSize(pBlock->pDataBlock);
int32_t numOfRows = 0;
int32_t nInputCols = 0;
if (pSubmitTbData->flags & SUBMIT_REQ_COLUMN_DATA_FORMAT) {
SColData* pCol = taosArrayGet(pSubmitTbData->aCol, 0);
TSDB_CHECK_NULL(pCol, code, lino, _end, terrno);
numOfRows = pCol->nVal;
nInputCols = taosArrayGetSize(pSubmitTbData->aCol);
} else {
// try to get physical table schema from cache
pScanInfo->metaFetch++;
int64_t cacheKey = (pSubmitTbData->suid == 0) ? pTbUid : pSubmitTbData->suid;
SSchemaWrapper* pWrapper = NULL;
h = taosLRUCacheLookup(pScanInfo->pPhyTblSchemaCache, &cacheKey, sizeof(int64_t));
if (h != NULL) {
pWrapper = taosLRUCacheValue(pScanInfo->pPhyTblSchemaCache, h);
TSDB_CHECK_NULL(pWrapper, code, lino, _end, terrno);
}
if (pWrapper != NULL && pWrapper->version != pSubmitTbData->sver) {
// reset outdated schema
tDeleteSchemaWrapper(pWrapper);
pWrapper = NULL;
taosLRUCacheUpdate(pScanInfo->pPhyTblSchemaCache, h, pWrapper);
}
if (pWrapper == NULL) {
// get physical table schema from meta
pWrapper = metaGetTableSchema(pReader->pVnodeMeta, pTbUid, pSubmitTbData->sver, 1, NULL);
if (pWrapper == NULL) {
tqWarn("vgId:%d, cannot found schema wrapper for table: suid:%" PRId64 ", uid:%" PRId64
"version %d, possibly dropped table",
pReader->pWalReader->pWal->cfg.vgId, pSubmitTbData->suid, pTbUid, pSubmitTbData->sver);
TSDB_CHECK_NULL(pWrapper, code, lino, _end, TSDB_CODE_TQ_TABLE_SCHEMA_NOT_FOUND);
}
if (h == NULL) {
// insert schema to cache
code = taosLRUCacheInsert(pScanInfo->pPhyTblSchemaCache, &cacheKey, sizeof(int64_t), pWrapper, POINTER_BYTES,
freeTableSchemaCache, NULL, NULL, TAOS_LRU_PRIORITY_LOW, NULL);
if (code != TSDB_CODE_SUCCESS) {
tDeleteSchemaWrapper(pWrapper);
}
TSDB_CHECK_CODE(code, lino, _end);
} else {
// update schema in cache
taosLRUCacheUpdate(pScanInfo->pPhyTblSchemaCache, h, pWrapper);
}
} else {
pScanInfo->cacheHit++;
}
TSDB_CHECK_NULL(pWrapper, code, lino, _end, TSDB_CODE_INTERNAL_ERROR);
pPhyTblSchema = tBuildTSchema(pWrapper->pSchema, pWrapper->nCols, pWrapper->version);
TSDB_CHECK_NULL(pPhyTblSchema, code, lino, _end, terrno);
numOfRows = taosArrayGetSize(pSubmitTbData->aRowP);
nInputCols = pPhyTblSchema->numOfCols;
}
code = blockDataEnsureCapacity(pBlock, numOfRows);
TSDB_CHECK_CODE(code, lino, _end);
// convert one block
for (int32_t i = 0, j = 1; j < nOutputCols;) {
SColumnInfoData* pOutCol = taosArrayGet(pBlock->pDataBlock, j);
TSDB_CHECK_NULL(pOutCol, code, lino, _end, terrno);
if (i >= nColInfos) {
tqInfo("%s has %d column info, but vtable column %d is missing, id: %s", __func__, nColInfos, pOutCol->info.colId,
idstr);
colDataSetNNULL(pOutCol, 0, numOfRows);
j++;
continue;
}
SVTColInfo* pCol = taosArrayGet(pColInfos, i);
TSDB_CHECK_NULL(pCol, code, lino, _end, terrno);
if (pCol->vColId < pOutCol->info.colId) {
i++;
continue;
} else if (pCol->vColId > pOutCol->info.colId) {
tqInfo("%s does not find column info for vtable column %d, closest vtable column is %d, id: %s", __func__,
pOutCol->info.colId, pCol->vColId, idstr);
colDataSetNNULL(pOutCol, 0, numOfRows);
j++;
continue;
}
// copy data from physical table to the result block of virtual table
if (pCol->pTbUid != pTbUid) {
// skip this column since it is from another physical table
} else if (pSubmitTbData->flags & SUBMIT_REQ_COLUMN_DATA_FORMAT) {
// try to find the corresponding column data of physical table
SColData* pColData = NULL;
for (int32_t k = 0; k < nInputCols; ++k) {
pColData = taosArrayGet(pSubmitTbData->aCol, k);
TSDB_CHECK_NULL(pColData, code, lino, _end, terrno);
if (pColData->cid == pCol->pColId) {
break;
}
pColData = NULL;
}
if (pColData == NULL) {
tqError("%s does not find data of physical table %" PRId64 " column %d, virtual table: %" PRId64
" column: %d, id: %s",
__func__, pTbUid, pCol->pColId, vTbUid, pCol->vColId, idstr);
colDataSetNNULL(pOutCol, 0, numOfRows);
i++;
j++;
continue;
}
SColVal colVal = {0};
for (int32_t k = 0; k < pColData->nVal; ++k) {
code = tColDataGetValue(pColData, k, &colVal);
TSDB_CHECK_CODE(code, lino, _end);
code = doSetVal(pOutCol, k, &colVal);
TSDB_CHECK_CODE(code, lino, _end);
}
} else {
SArray* pRows = pSubmitTbData->aRowP;
TSDB_CHECK_NULL(pPhyTblSchema, code, lino, _end, TSDB_CODE_INTERNAL_ERROR);
SColVal colVal = {0};
for (int32_t k = 0; k < numOfRows; ++k) {
SRow* pRow = taosArrayGetP(pRows, k);
TSDB_CHECK_NULL(pRow, code, lino, _end, terrno);
for (int32_t l = 0; l < nInputCols; ++l) {
code = tRowGet(pRow, pPhyTblSchema, l, &colVal);
TSDB_CHECK_CODE(code, lino, _end);
if (colVal.cid == pCol->pColId) {
code = doSetVal(pOutCol, k, &colVal);
TSDB_CHECK_CODE(code, lino, _end);
break;
} else if (colVal.cid > pCol->pColId || l == (nInputCols - 1)) {
colDataSetNULL(pOutCol, k);
break;
}
}
}
}
i++;
j++;
}
// enforce to fill the first ts column
if (pSubmitTbData->flags & SUBMIT_REQ_COLUMN_DATA_FORMAT) {
SColumnInfoData* pOutCol = taosArrayGet(pBlock->pDataBlock, 0);
SColData* pColData = taosArrayGet(pSubmitTbData->aCol, 0);
TSDB_CHECK_NULL(pColData, code, lino, _end, terrno);
SColVal colVal = {0};
for (int32_t k = 0; k < pColData->nVal; ++k) {
code = tColDataGetValue(pColData, k, &colVal);
TSDB_CHECK_CODE(code, lino, _end);
code = doSetVal(pOutCol, k, &colVal);
TSDB_CHECK_CODE(code, lino, _end);
}
} else {
SColumnInfoData* pOutCol = taosArrayGet(pBlock->pDataBlock, 0);
SArray* pRows = pSubmitTbData->aRowP;
TSDB_CHECK_NULL(pPhyTblSchema, code, lino, _end, TSDB_CODE_INTERNAL_ERROR);
SColVal colVal = {0};
for (int32_t k = 0; k < numOfRows; ++k) {
SRow* pRow = taosArrayGetP(pRows, k);
TSDB_CHECK_NULL(pRow, code, lino, _end, terrno);
code = tRowGet(pRow, pPhyTblSchema, 0, &colVal);
TSDB_CHECK_CODE(code, lino, _end);
code = doSetVal(pOutCol, k, &colVal);
TSDB_CHECK_CODE(code, lino, _end);
}
}
pBlock->info.rows = numOfRows;
pBlock->info.id.uid = vTbUid;
pBlock->info.id.groupId = pTbUid;
pBlock->info.version = pReader->msg.ver;
pScanInfo->nextVirtualTableIdx++;
if (pScanInfo->nextVirtualTableIdx >= taosArrayGetSize(pRelatedVTs)) {
pReader->nextBlk++;
pScanInfo->nextVirtualTableIdx = -1;
}
tqDebug("tq reader will retrieve next vtable data block from %p, nextBlk:%d, vtbIdx:%d, id:%s", pReader->msg.msgStr,
pReader->nextBlk, pScanInfo->nextVirtualTableIdx, idstr);
*pRes = pBlock;
_end:
if (code != TSDB_CODE_SUCCESS) {
tqError("%s failed at line %d since %s, id: %s", __func__, lino, tstrerror(code), idstr);
}
if (h != NULL) {
bool bRes = taosLRUCacheRelease(pScanInfo->pPhyTblSchemaCache, h, false);
tqTrace("release LRU cache, res %d, id: %s", bRes, idstr);
}
if (pPhyTblSchema != NULL) {
taosMemoryFreeClear(pPhyTblSchema);
}
return code;
}
bool tqNextVTableSourceBlockImpl(STqReader* pReader, const char* idstr) {
int32_t code = TSDB_CODE_SUCCESS;
int32_t lino = 0;
SVTSourceScanInfo* pScanInfo = NULL;
TSDB_CHECK_NULL(pReader, code, lino, _end, TSDB_CODE_INVALID_PARA);
pScanInfo = &pReader->vtSourceScanInfo;
if (pReader->msg.msgStr == NULL || taosHashGetSize(pScanInfo->pPhysicalTables) == 0) {
return false;
}
if (pScanInfo->nextVirtualTableIdx >= 0) {
// The data still needs to be converted into the virtual table result block
return true;
}
int32_t blockSz = taosArrayGetSize(pReader->submit.aSubmitTbData);
while (pReader->nextBlk < blockSz) {
SSubmitTbData* pSubmitTbData = taosArrayGet(pReader->submit.aSubmitTbData, pReader->nextBlk);
TSDB_CHECK_NULL(pSubmitTbData, code, lino, _end, terrno);
int64_t pTbUid = pSubmitTbData->uid;
void* px = taosHashGet(pScanInfo->pPhysicalTables, &pTbUid, sizeof(int64_t));
if (px != NULL) {
SArray* pRelatedVTs = *(SArray**)px;
if (taosArrayGetSize(pRelatedVTs) > 0) {
pScanInfo->nextVirtualTableIdx = 0;
return true;
}
}
tqTrace("iterator data block in hash jump block, progress:%d/%d, uid:%" PRId64 "", pReader->nextBlk, blockSz,
pTbUid);
pReader->nextBlk++;
}
tqReaderClearSubmitMsg(pReader);
tqTrace("iterator data block end, total block num:%d", blockSz);
_end:
if (code != TSDB_CODE_SUCCESS) {
tqError("%s failed at line %d since %s, id: %s", __func__, lino, tstrerror(code), idstr);
}
return (code == TSDB_CODE_SUCCESS);
}
bool tqReaderIsQueriedSourceTable(STqReader* pReader, uint64_t uid) {
if (pReader == NULL) {
return false;
}
return taosHashGet(pReader->vtSourceScanInfo.pPhysicalTables, &uid, sizeof(uint64_t)) != NULL;
}

View File

@ -150,6 +150,8 @@ void initTqAPI(SStoreTqReader* pTq) {
pTq->tqGetResultBlockTime = tqGetResultBlockTime;
pTq->tqGetStreamExecProgress = tqGetStreamExecInfo;
pTq->tqReaderSetVtableInfo = tqReaderSetVtableInfo;
#endif
}

View File

@ -609,6 +609,159 @@ _exit:
(void)taosThreadRwlockUnlock(&(pVnode)->metaRWLock); \
} while (0)
int32_t vnodeReadVSubtables(SReadHandle* pHandle, int64_t suid, SArray** ppRes) {
int32_t code = TSDB_CODE_SUCCESS;
int32_t line = 0;
SMetaReader mr = {0};
bool readerInit = false;
SVCTableRefCols* pTb = NULL;
int32_t refColsNum = 0;
char tbFName[TSDB_TABLE_FNAME_LEN];
SArray *pList = taosArrayInit(10, sizeof(uint64_t));
QUERY_CHECK_NULL(pList, code, line, _return, terrno);
QUERY_CHECK_CODE(pHandle->api.metaFn.getChildTableList(pHandle->vnode, suid, pList), line, _return);
size_t num = taosArrayGetSize(pList);
*ppRes = taosArrayInit(num, POINTER_BYTES);
QUERY_CHECK_NULL(*ppRes, code, line, _return, terrno);
SSHashObj* pSrcTbls = tSimpleHashInit(10, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY));
QUERY_CHECK_NULL(pSrcTbls, code, line, _return, terrno);
for (int32_t i = 0; i < num; ++i) {
uint64_t* id = taosArrayGet(pList, i);
QUERY_CHECK_NULL(id, code, line, _return, terrno);
pHandle->api.metaReaderFn.initReader(&mr, pHandle->vnode, META_READER_LOCK, &pHandle->api.metaFn);
QUERY_CHECK_CODE(pHandle->api.metaReaderFn.getTableEntryByUid(&mr, *id), line, _return);
readerInit = true;
refColsNum = 0;
for (int32_t j = 0; j < mr.me.colRef.nCols; j++) {
if (mr.me.colRef.pColRef[j].hasRef) {
refColsNum++;
}
}
if (refColsNum <= 0) {
pHandle->api.metaReaderFn.clearReader(&mr);
readerInit = false;
continue;
}
pTb = taosMemoryCalloc(1, refColsNum * sizeof(SRefColInfo) + sizeof(*pTb));
QUERY_CHECK_NULL(pTb, code, line, _return, terrno);
pTb->uid = mr.me.uid;
pTb->numOfColRefs = refColsNum;
pTb->refCols = (SRefColInfo*)(pTb + 1);
refColsNum = 0;
tSimpleHashClear(pSrcTbls);
for (int32_t j = 0; j < mr.me.colRef.nCols; j++) {
if (!mr.me.colRef.pColRef[j].hasRef) {
continue;
}
pTb->refCols[refColsNum].colId = mr.me.colRef.pColRef[j].id;
tstrncpy(pTb->refCols[refColsNum].refColName, mr.me.colRef.pColRef[j].refColName, TSDB_COL_NAME_LEN);
tstrncpy(pTb->refCols[refColsNum].refTableName, mr.me.colRef.pColRef[j].refTableName, TSDB_TABLE_NAME_LEN);
tstrncpy(pTb->refCols[refColsNum].refDbName, mr.me.colRef.pColRef[j].refDbName, TSDB_DB_NAME_LEN);
snprintf(tbFName, sizeof(tbFName), "%s.%s", pTb->refCols[refColsNum].refDbName, pTb->refCols[refColsNum].refTableName);
if (NULL == tSimpleHashGet(pSrcTbls, tbFName, strlen(tbFName))) {
QUERY_CHECK_CODE(tSimpleHashPut(pSrcTbls, tbFName, strlen(tbFName), &code, sizeof(code)), line, _return);
}
refColsNum++;
}
pTb->numOfSrcTbls = tSimpleHashGetSize(pSrcTbls);
QUERY_CHECK_NULL(taosArrayPush(*ppRes, &pTb), code, line, _return, terrno);
pTb = NULL;
pHandle->api.metaReaderFn.clearReader(&mr);
readerInit = false;
}
_return:
if (readerInit) {
pHandle->api.metaReaderFn.clearReader(&mr);
}
taosArrayDestroy(pList);
taosMemoryFree(pTb);
tSimpleHashCleanup(pSrcTbls);
if (code) {
qError("%s failed since %s", __func__, tstrerror(code));
}
return code;
}
int32_t vnodeGetVSubtablesMeta(SVnode *pVnode, SRpcMsg *pMsg) {
int32_t code = 0;
int32_t rspSize = 0;
SVSubTablesReq req = {0};
SVSubTablesRsp rsp = {0};
SRpcMsg rspMsg = {0};
void *pRsp = NULL;
int32_t line = 0;
if (tDeserializeSVSubTablesReq(pMsg->pCont, pMsg->contLen, &req)) {
code = terrno;
qError("tDeserializeSVSubTablesReq failed");
goto _return;
}
SReadHandle handle = {.vnode = pVnode};
initStorageAPI(&handle.api);
QUERY_CHECK_CODE(vnodeReadVSubtables(&handle, req.suid, &rsp.pTables), line, _return);
rsp.vgId = TD_VID(pVnode);
rspSize = tSerializeSVSubTablesRsp(NULL, 0, &rsp);
if (rspSize < 0) {
code = rspSize;
qError("tSerializeSVSubTablesRsp failed, error:%d", rspSize);
goto _return;
}
pRsp = rpcMallocCont(rspSize);
if (pRsp == NULL) {
code = terrno;
qError("rpcMallocCont %d failed, error:%d", rspSize, terrno);
goto _return;
}
rspSize = tSerializeSVSubTablesRsp(pRsp, rspSize, &rsp);
if (rspSize < 0) {
code = rspSize;
qError("tSerializeSVSubTablesRsp failed, error:%d", rspSize);
goto _return;
}
_return:
rspMsg.info = pMsg->info;
rspMsg.pCont = pRsp;
rspMsg.contLen = rspSize;
rspMsg.code = code;
rspMsg.msgType = pMsg->msgType;
if (code) {
qError("vnd get virtual subtables failed cause of %s", tstrerror(code));
}
tDestroySVSubTablesRsp(&rsp);
tmsgSendRsp(&rspMsg);
return code;
}
int32_t vnodeGetLoad(SVnode *pVnode, SVnodeLoad *pLoad) {
SSyncState state = syncGetState(pVnode->sync);
pLoad->syncAppliedIndex = pVnode->state.applied;

View File

@ -898,7 +898,7 @@ int32_t vnodeProcessQueryMsg(SVnode *pVnode, SRpcMsg *pMsg, SQueueInfo *pInfo) {
int32_t vnodeProcessFetchMsg(SVnode *pVnode, SRpcMsg *pMsg, SQueueInfo *pInfo) {
vTrace("vgId:%d, msg:%p in fetch queue is processing", pVnode->config.vgId, pMsg);
if ((pMsg->msgType == TDMT_SCH_FETCH || pMsg->msgType == TDMT_VND_TABLE_META || pMsg->msgType == TDMT_VND_TABLE_CFG ||
pMsg->msgType == TDMT_VND_BATCH_META || pMsg->msgType == TDMT_VND_TABLE_NAME) &&
pMsg->msgType == TDMT_VND_BATCH_META || pMsg->msgType == TDMT_VND_TABLE_NAME || pMsg->msgType == TDMT_VND_VSUBTABLES_META) &&
!syncIsReadyForRead(pVnode->sync)) {
vnodeRedirectRpcMsg(pVnode, pMsg, terrno);
return 0;
@ -925,6 +925,8 @@ int32_t vnodeProcessFetchMsg(SVnode *pVnode, SRpcMsg *pMsg, SQueueInfo *pInfo) {
return vnodeGetTableCfg(pVnode, pMsg, true);
case TDMT_VND_BATCH_META:
return vnodeGetBatchMeta(pVnode, pMsg);
case TDMT_VND_VSUBTABLES_META:
return vnodeGetVSubtablesMeta(pVnode, pMsg);
#ifdef TD_ENTERPRISE
case TDMT_VND_QUERY_COMPACT_PROGRESS:
return vnodeQueryCompactProgress(pVnode, pMsg);

View File

@ -72,6 +72,7 @@ typedef enum {
CTG_CI_SVR_VER,
CTG_CI_VIEW,
CTG_CI_TBL_TSMA,
CTG_CI_VSUB_TBLS,
CTG_CI_MAX_VALUE,
} CTG_CACHE_ITEM;
@ -133,6 +134,7 @@ typedef enum {
CTG_TASK_GET_TB_TSMA,
CTG_TASK_GET_TSMA,
CTG_TASK_GET_TB_NAME,
CTG_TASK_GET_V_SUBTABLES,
} CTG_TASK_TYPE;
typedef enum {
@ -307,6 +309,20 @@ typedef struct SCtgTbTSMACtx {
SArray* pFetches;
} SCtgTbTSMACtx;
typedef struct SCtgVSubTablesCtx {
SArray* pNames;
STableMeta* pMeta;
int32_t vgNum;
bool clonedVgroups;
SArray* pVgroups;
SVSubTablesRsp* pResList;
int32_t resIdx;
} SCtgVSubTablesCtx;
typedef STableIndexRsp STableIndex;
typedef STableTSMAInfo STSMACache;
@ -433,6 +449,7 @@ typedef struct SCtgJob {
int32_t tbTsmaNum;
int32_t tsmaNum; // currently, only 1 is possible
int32_t tbNameNum;
int32_t vsubTbNum;
} SCtgJob;
typedef struct SCtgMsgCtx {
@ -1076,6 +1093,7 @@ int32_t ctgOpUpdateTbIndex(SCtgCacheOperation* operation);
int32_t ctgOpClearCache(SCtgCacheOperation* operation);
int32_t ctgOpUpdateViewMeta(SCtgCacheOperation* operation);
int32_t ctgReadTbTypeFromCache(SCatalog* pCtg, char* dbFName, char* tableName, int32_t* tbType);
int32_t ctgReadTbTypeSuidFromCache(SCatalog *pCtg, char *dbFName, char *tbName, int32_t *tbType, int64_t* suid);
int32_t ctgGetTbHashVgroupFromCache(SCatalog* pCtg, const SName* pTableName, SVgroupInfo** pVgroup);
int32_t ctgGetViewsFromCache(SCatalog* pCtg, SRequestConnInfo* pConn, SCtgViewsCtx* ctx, int32_t dbIdx,
int32_t* fetchIdx, int32_t baseResIdx, SArray* pList);
@ -1114,6 +1132,7 @@ int32_t ctgLaunchJob(SCtgJob* pJob);
int32_t ctgMakeAsyncRes(SCtgJob* pJob);
int32_t ctgLaunchSubTask(SCtgTask** ppTask, CTG_TASK_TYPE type, ctgSubTaskCbFp fp, void* param);
int32_t ctgGetTbCfgCb(SCtgTask* pTask);
int32_t ctgGetVSubTablesCb(SCtgTask* pTask);
void ctgFreeHandle(SCatalog* pCatalog);
void ctgFreeSViewMeta(SViewMeta* pMeta);
@ -1212,8 +1231,10 @@ bool isCtgTSMACacheOutOfDate(STSMACache* pTsmaCache);
int32_t ctgGetStreamProgressFromVnode(SCatalog* pCtg, SRequestConnInfo* pConn, const SName* pTbName,
SVgroupInfo* vgroupInfo, SStreamProgressRsp* out, SCtgTaskReq* tReq,
void* bInput);
int32_t ctgGetVSubTablesFromVnode(SCatalog* pCtg, SRequestConnInfo* pConn, int64_t suid, SVgroupInfo* vgroupInfo, SCtgTaskReq* tReq);
int32_t ctgAddTSMAFetch(SArray** pFetchs, int32_t dbIdx, int32_t tbIdx, int32_t* fetchIdx, int32_t resIdx, int32_t flag,
CTG_TSMA_FETCH_TYPE fetchType, const SName* sourceTbName);
int32_t ctgBuildNormalChildVtbList(SCtgVSubTablesCtx* pCtx);
int32_t ctgOpUpdateDbTsmaVersion(SCtgCacheOperation* pOper);
int32_t ctgUpdateDbTsmaVersionEnqueue(SCatalog* pCtg, int32_t tsmaVersion, const char* dbFName, int64_t dbId,
bool syncOper);

View File

@ -623,6 +623,29 @@ static int32_t ctgInitGetTbNamesTask(SCtgJob* pJob, int32_t taskId, void* param)
return TSDB_CODE_SUCCESS;
}
int32_t ctgInitGetVSubTablesTask(SCtgJob* pJob, int32_t taskId, void* param) {
SCtgTask task = {0};
task.type = CTG_TASK_GET_V_SUBTABLES;
task.taskId = taskId;
task.pJob = pJob;
SCtgVSubTablesCtx* pTaskCtx = taosMemoryCalloc(1, sizeof(SCtgVSubTablesCtx));
if (NULL == pTaskCtx) {
CTG_ERR_RET(terrno);
}
task.taskCtx = pTaskCtx;
pTaskCtx->pNames = param;
if (NULL == taosArrayPush(pJob->pTasks, &task)) {
ctgFreeTask(&task, true);
CTG_ERR_RET(terrno);
}
return TSDB_CODE_SUCCESS;
}
int32_t ctgHandleForceUpdateView(SCatalog* pCtg, const SCatalogReq* pReq) {
int32_t viewNum = taosArrayGetSize(pReq->pView);
for (int32_t i = 0; i < viewNum; ++i) {
@ -858,9 +881,10 @@ int32_t ctgInitJob(SCatalog* pCtg, SRequestConnInfo* pConn, SCtgJob** job, const
int32_t tbTsmaNum = tsQuerySmaOptimize ? (int32_t)taosArrayGetSize(pReq->pTableTSMAs) : 0;
int32_t tsmaNum = (int32_t)taosArrayGetSize(pReq->pTSMAs);
int32_t tbNameNum = (int32_t)ctgGetTablesReqNum(pReq->pTableName);
int32_t vsubTbNum = (int32_t)taosArrayGetSize(pReq->pVSubTable);
int32_t taskNum = tbMetaNum + dbVgNum + udfNum + tbHashNum + qnodeNum + dnodeNum + svrVerNum + dbCfgNum + indexNum +
userNum + dbInfoNum + tbIndexNum + tbCfgNum + tbTagNum + viewNum + tbTsmaNum + tbNameNum;
userNum + dbInfoNum + tbIndexNum + tbCfgNum + tbTagNum + viewNum + tbTsmaNum + tbNameNum + vsubTbNum;
int32_t taskNumWithSubTasks = tbMetaNum * gCtgAsyncFps[CTG_TASK_GET_TB_META].subTaskFactor + dbVgNum * gCtgAsyncFps[CTG_TASK_GET_DB_VGROUP].subTaskFactor +
udfNum * gCtgAsyncFps[CTG_TASK_GET_UDF].subTaskFactor + tbHashNum * gCtgAsyncFps[CTG_TASK_GET_TB_HASH].subTaskFactor +
qnodeNum * gCtgAsyncFps[CTG_TASK_GET_QNODE].subTaskFactor + dnodeNum * gCtgAsyncFps[CTG_TASK_GET_DNODE].subTaskFactor +
@ -869,7 +893,8 @@ int32_t ctgInitJob(SCatalog* pCtg, SRequestConnInfo* pConn, SCtgJob** job, const
dbInfoNum * gCtgAsyncFps[CTG_TASK_GET_DB_INFO].subTaskFactor + tbIndexNum * gCtgAsyncFps[CTG_TASK_GET_TB_SMA_INDEX].subTaskFactor +
tbCfgNum * gCtgAsyncFps[CTG_TASK_GET_TB_CFG].subTaskFactor + tbTagNum * gCtgAsyncFps[CTG_TASK_GET_TB_TAG].subTaskFactor +
viewNum * gCtgAsyncFps[CTG_TASK_GET_VIEW].subTaskFactor + tbTsmaNum * gCtgAsyncFps[CTG_TASK_GET_TB_TSMA].subTaskFactor +
tsmaNum * gCtgAsyncFps[CTG_TASK_GET_TSMA].subTaskFactor + tbNameNum * gCtgAsyncFps[CTG_TASK_GET_TB_NAME].subTaskFactor;
tsmaNum * gCtgAsyncFps[CTG_TASK_GET_TSMA].subTaskFactor + tbNameNum * gCtgAsyncFps[CTG_TASK_GET_TB_NAME].subTaskFactor +
vsubTbNum * gCtgAsyncFps[CTG_TASK_GET_V_SUBTABLES].subTaskFactor;
*job = taosMemoryCalloc(1, sizeof(SCtgJob));
if (NULL == *job) {
@ -905,6 +930,7 @@ int32_t ctgInitJob(SCatalog* pCtg, SRequestConnInfo* pConn, SCtgJob** job, const
pJob->tbTsmaNum = tbTsmaNum;
pJob->tsmaNum = tsmaNum;
pJob->tbNameNum = tbNameNum;
pJob->vsubTbNum = vsubTbNum;
#if CTG_BATCH_FETCH
pJob->pBatchs =
@ -1050,6 +1076,9 @@ int32_t ctgInitJob(SCatalog* pCtg, SRequestConnInfo* pConn, SCtgJob** job, const
if (svrVerNum) {
CTG_ERR_JRET(ctgInitTask(pJob, CTG_TASK_GET_SVR_VER, NULL, NULL));
}
if (vsubTbNum) {
CTG_ERR_JRET(ctgInitTask(pJob, CTG_TASK_GET_V_SUBTABLES, pReq->pVSubTable, NULL));
}
pJob->refId = taosAddRef(gCtgMgmt.jobPool, pJob);
if (pJob->refId < 0) {
@ -1440,6 +1469,41 @@ int32_t ctgDumpViewsRes(SCtgTask* pTask) {
return TSDB_CODE_SUCCESS;
}
int32_t ctgDumpVSubTablesRes(SCtgTask* pTask) {
if (pTask->subTask) {
return TSDB_CODE_SUCCESS;
}
SCtgVSubTablesCtx* pCtx = (SCtgVSubTablesCtx*)pTask->taskCtx;
SCatalog* pCtg = pTask->pJob->pCtg;
if (NULL == pCtx->pResList) {
return TSDB_CODE_SUCCESS;
}
SCtgJob* pJob = pTask->pJob;
int32_t resVgNum = (TSDB_SUPER_TABLE == pCtx->pMeta->tableType) ? pCtx->vgNum : 1;
pJob->jobRes.pVSubTables = taosArrayInit(resVgNum, sizeof(SVSubTablesRsp));
if (NULL == pJob->jobRes.pVSubTables) {
ctgError("taosArrayInit %d SVSubTablesRsp failed, error:%d", resVgNum, terrno);
return terrno;
}
SVSubTablesRsp* pVg = NULL;
for (int32_t i = 0; i < resVgNum; ++i) {
pVg = (SVSubTablesRsp*)pCtx->pResList + i;
if (NULL == taosArrayPush(pJob->jobRes.pVSubTables, pVg)) {
ctgError("taosArrayPush failed, error:%d", terrno);
return terrno;
}
pVg->pTables = NULL;
}
return TSDB_CODE_SUCCESS;
}
int32_t ctgCallSubCb(SCtgTask* pTask) {
int32_t code = 0;
@ -3076,6 +3140,30 @@ _return:
CTG_RET(code);
}
int32_t ctgHandleGetVSubTablesRsp(SCtgTaskReq* tReq, int32_t reqType, const SDataBuf* pMsg, int32_t rspCode) {
int32_t code = 0;
SCtgTask* pTask = tReq->pTask;
int32_t newCode = TSDB_CODE_SUCCESS;
SCtgVSubTablesCtx* pCtx = (SCtgVSubTablesCtx*)pTask->taskCtx;
CTG_ERR_JRET(ctgProcessRspMsg(pCtx->pResList + atomic_fetch_add_32(&pCtx->resIdx, 1), reqType, pMsg->pData, pMsg->len, rspCode, pTask->msgCtx.target));
if (atomic_load_32(&pCtx->resIdx) < pCtx->vgNum) {
CTG_RET(code);
}
_return:
newCode = ctgHandleTaskEnd(pTask, code);
if (newCode && TSDB_CODE_SUCCESS == code) {
code = newCode;
}
CTG_RET(code);
}
int32_t ctgAsyncRefreshTbMeta(SCtgTaskReq* tReq, int32_t flag, SName* pName, int32_t* vgId) {
SCtgTask* pTask = tReq->pTask;
SCatalog* pCtg = pTask->pJob->pCtg;
@ -4133,6 +4221,122 @@ static int32_t ctgLaunchGetTbNamesTask(SCtgTask* pTask) {
return TSDB_CODE_SUCCESS;
}
int32_t ctgLaunchGetVSubTablesTask(SCtgTask* pTask) {
SCatalog* pCtg = pTask->pJob->pCtg;
SRequestConnInfo* pConn = &pTask->pJob->conn;
SCtgVSubTablesCtx* pCtx = (SCtgVSubTablesCtx*)pTask->taskCtx;
SCtgJob* pJob = pTask->pJob;
SName* pName = NULL;
char dbFName[TSDB_DB_FNAME_LEN];
SCtgDBCache* dbCache = NULL;
int32_t code = TSDB_CODE_SUCCESS;
SCtgMsgCtx* pMsgCtx = CTG_GET_TASK_MSGCTX(pTask, -1);
if (NULL == pMsgCtx) {
ctgError("fail to get the %dth pMsgCtx", -1);
CTG_ERR_RET(TSDB_CODE_CTG_INTERNAL_ERROR);
}
if (NULL == pMsgCtx->pBatchs) {
pMsgCtx->pBatchs = pJob->pBatchs;
}
int32_t tbNum = taosArrayGetSize(pCtx->pNames);
if (tbNum > 1) {
ctgError("only 1 virtual table supported now, tbnum:%d", tbNum);
CTG_ERR_RET(TSDB_CODE_CTG_INVALID_INPUT);
}
pName = taosArrayGet(pCtx->pNames, 0);
if (NULL == pName) {
ctgError("fail to get SName in VSubtables req, num:%d", tbNum);
CTG_ERR_RET(TSDB_CODE_CTG_INVALID_INPUT);
}
if (NULL == pMsgCtx->target) {
pMsgCtx->target = taosMemoryMalloc(TSDB_TABLE_FNAME_LEN);
if (NULL == pMsgCtx->target) {
ctgError("taosMemoryMalloc %d failed", TSDB_TABLE_FNAME_LEN);
CTG_ERR_RET(terrno);
}
tNameExtractFullName(pName, pMsgCtx->target);
}
(void)tNameGetFullDbName(pName, dbFName);
if (NULL == pCtx->pMeta) {
SCtgTbMetaCtx metaCtx = {0};
metaCtx.pName = pName;
CTG_ERR_RET(ctgReadTbMetaFromCache(pCtg, &metaCtx, &pCtx->pMeta));
if (NULL == pCtx->pMeta) {
SCtgTbMetaParam param;
param.pName = pName;
param.flag = 0;
CTG_ERR_RET(ctgLaunchSubTask(&pTask, CTG_TASK_GET_TB_META, ctgGetVSubTablesCb, &param));
return TSDB_CODE_SUCCESS;
}
}
if (TSDB_VIRTUAL_CHILD_TABLE == pCtx->pMeta->tableType || TSDB_VIRTUAL_NORMAL_TABLE == pCtx->pMeta->tableType) {
CTG_ERR_RET(ctgBuildNormalChildVtbList(pCtx));
CTG_ERR_RET(ctgHandleTaskEnd(pTask, 0));
return TSDB_CODE_SUCCESS;
}
if (TSDB_SUPER_TABLE != pCtx->pMeta->tableType || !pCtx->pMeta->virtualStb) {
CTG_ERR_RET(ctgHandleTaskEnd(pTask, 0));
return TSDB_CODE_SUCCESS;
}
if (NULL == pCtx->pVgroups) {
CTG_ERR_RET(ctgAcquireVgInfoFromCache(pCtg, dbFName, &dbCache));
if (NULL == dbCache) {
CTG_ERR_RET(ctgLaunchSubTask(&pTask, CTG_TASK_GET_DB_VGROUP, ctgGetVSubTablesCb, dbFName));
return TSDB_CODE_SUCCESS;
}
pCtx->pVgroups = dbCache->vgCache.vgInfo->vgArray;
}
pCtx->vgNum = taosArrayGetSize(pCtx->pVgroups);
pCtx->pResList = taosMemoryCalloc(pCtx->vgNum, sizeof(SVSubTablesRsp));
if (NULL == pCtx->pResList) {
ctgError("taosMemoryMalloc %d SVSubTablesRsp failed", pCtx->vgNum);
CTG_ERR_JRET(terrno);
}
for (int32_t i = 0; i < pCtx->vgNum; ++i) {
SVgroupInfo* pVg = (SVgroupInfo*)taosArrayGet(pCtx->pVgroups, i);
ctgDebug("will try to get vsubtable from vg %d", pVg->vgId);
SCtgTaskReq tReq;
tReq.pTask = pTask;
tReq.msgIdx = -1;
CTG_ERR_JRET(ctgGetVSubTablesFromVnode(pCtg, pConn, pCtx->pMeta->suid, pVg, &tReq));
}
if (dbCache) {
ctgReleaseVgInfoToCache(pCtg, dbCache);
}
return TSDB_CODE_SUCCESS;
_return:
if (dbCache) {
ctgReleaseVgInfoToCache(pCtg, dbCache);
}
if (CTG_TASK_LAUNCHED == pTask->status) {
int32_t newCode = ctgHandleTaskEnd(pTask, code);
if (newCode && TSDB_CODE_SUCCESS == code) {
code = newCode;
}
}
CTG_RET(code);
}
int32_t ctgDumpTbTSMARes(SCtgTask* pTask) {
if (pTask->subTask) {
return TSDB_CODE_SUCCESS;
@ -4188,6 +4392,33 @@ _return:
CTG_RET(ctgHandleTaskEnd(pTask, pTask->subRes.code));
}
int32_t ctgGetVSubTablesCb(SCtgTask* pTask) {
int32_t code = 0;
CTG_ERR_JRET(pTask->subRes.code);
SCtgVSubTablesCtx* pCtx = (SCtgVSubTablesCtx*)pTask->taskCtx;
if (CTG_TASK_GET_TB_META == pTask->subRes.type) {
TSWAP(pCtx->pMeta, pTask->subRes.res);
} else if (CTG_TASK_GET_DB_VGROUP == pTask->subRes.type) {
SDBVgInfo* pDb = (SDBVgInfo*)pTask->subRes.res;
pCtx->pVgroups = taosArrayDup(pDb->vgArray, NULL);
if (NULL == pCtx->pVgroups) {
CTG_ERR_JRET(terrno);
}
pCtx->clonedVgroups = true;
}
CTG_RET(ctgLaunchGetVSubTablesTask(pTask));
_return:
CTG_RET(ctgHandleTaskEnd(pTask, pTask->subRes.code));
}
int32_t ctgGetTbTagCb(SCtgTask* pTask) {
int32_t code = 0;
@ -4280,6 +4511,7 @@ SCtgAsyncFps gCtgAsyncFps[] = {
{ctgInitGetTbTSMATask, ctgLaunchGetTbTSMATask, ctgHandleGetTbTSMARsp, ctgDumpTbTSMARes, NULL, NULL, 1},
{ctgInitGetTSMATask, ctgLaunchGetTSMATask, ctgHandleGetTSMARsp, ctgDumpTSMARes, NULL, NULL, 1},
{ctgInitGetTbNamesTask, ctgLaunchGetTbNamesTask, ctgHandleGetTbNamesRsp, ctgDumpTbNamesRes, NULL, NULL, 1},
{ctgInitGetVSubTablesTask, ctgLaunchGetVSubTablesTask, ctgHandleGetVSubTablesRsp, ctgDumpVSubTablesRes, NULL, NULL, 2},
};
int32_t ctgMakeAsyncRes(SCtgJob* pJob) {

View File

@ -776,6 +776,26 @@ int32_t ctgReadTbTypeFromCache(SCatalog *pCtg, char *dbFName, char *tbName, int3
return TSDB_CODE_SUCCESS;
}
int32_t ctgReadTbTypeSuidFromCache(SCatalog *pCtg, char *dbFName, char *tbName, int32_t *tbType, int64_t* suid) {
SCtgDBCache *dbCache = NULL;
SCtgTbCache *tbCache = NULL;
CTG_ERR_RET(ctgAcquireTbMetaFromCache(pCtg, dbFName, tbName, &dbCache, &tbCache));
if (NULL == tbCache) {
ctgReleaseTbMetaToCache(pCtg, dbCache, tbCache);
return TSDB_CODE_SUCCESS;
}
*tbType = tbCache->pMeta->tableType;
*suid = tbCache->pMeta->suid;
ctgReleaseTbMetaToCache(pCtg, dbCache, tbCache);
ctgDebug("Got tb %s tbType %d suid %" PRIu64 " from cache, dbFName:%s", tbName, *tbType, *suid, dbFName);
return TSDB_CODE_SUCCESS;
}
int32_t ctgReadTbIndexFromCache(SCatalog *pCtg, SName *pTableName, SArray **pRes) {
int32_t code = 0;
SCtgDBCache *dbCache = NULL;

View File

@ -412,6 +412,17 @@ int32_t ctgProcessRspMsg(void* out, int32_t reqType, char* msg, int32_t msgSize,
}
break;
}
case TDMT_VND_VSUBTABLES_META: {
if (TSDB_CODE_SUCCESS != rspCode) {
CTG_ERR_RET(rspCode);
}
code = queryProcessMsgRsp[TMSG_INDEX(reqType)](out, msg, msgSize);
if (code) {
qError("Process get vnode virtual subtables rsp failed, err: %s, tbFName: %s", tstrerror(code), target);
CTG_ERR_RET(code);
}
break;
}
default:
if (TSDB_CODE_SUCCESS != rspCode) {
qError("get error rsp, error:%s", tstrerror(rspCode));
@ -1828,3 +1839,30 @@ int32_t ctgGetStreamProgressFromVnode(SCatalog* pCtg, SRequestConnInfo* pConn, c
return TSDB_CODE_SUCCESS;
}
int32_t ctgGetVSubTablesFromVnode(SCatalog* pCtg, SRequestConnInfo* pConn, int64_t suid, SVgroupInfo* vgroupInfo, SCtgTaskReq* tReq) {
SCtgTask* pTask = tReq ? tReq->pTask : NULL;
void* (*mallocFp)(int64_t) = pTask ? (MallocType)taosMemMalloc : (MallocType)rpcMallocCont;
int32_t reqType = TDMT_VND_VSUBTABLES_META;
SEp* pEp = &vgroupInfo->epSet.eps[vgroupInfo->epSet.inUse];
ctgDebug("try to get vsubtables meta from vnode, vgId:%d, ep num:%d, ep %s:%d, suid:%" PRIu64, vgroupInfo->vgId,
vgroupInfo->epSet.numOfEps, pEp->fqdn, pEp->port, suid);
char* msg = NULL;
int32_t msgLen = 0;
int32_t code = queryBuildMsg[TMSG_INDEX(reqType)](&suid, &msg, 0, &msgLen, mallocFp);
if (code) {
ctgError("Build vnode vsubtables meta msg failed, code:%x, suid:%" PRIu64, code, suid);
CTG_ERR_RET(code);
}
SRequestConnInfo vConn = {.pTrans = pConn->pTrans,
.requestId = pConn->requestId,
.requestObjRefId = pConn->requestObjRefId,
.mgmtEps = vgroupInfo->epSet};
return ctgAddBatch(pCtg, vgroupInfo->vgId, &vConn, tReq, reqType, msg, msgLen);
}

View File

@ -202,6 +202,9 @@ void ctgFreeSMetaData(SMetaData* pData) {
taosArrayDestroy(pData->pTsmas);
pData->pTsmas = NULL;
taosArrayDestroyEx(pData->pVSubTables, tDestroySVSubTablesRsp);
pData->pVSubTables = NULL;
taosMemoryFreeClear(pData->pSvrVer);
}
@ -657,6 +660,10 @@ void ctgFreeMsgCtx(SCtgMsgCtx* pCtx) {
}
break;
}
case TDMT_VND_VSUBTABLES_META: {
taosMemoryFreeClear(pCtx->target);
break;
}
default:
qError("invalid reqType %d", pCtx->reqType);
break;
@ -859,6 +866,9 @@ void ctgFreeTaskRes(CTG_TASK_TYPE type, void** pRes) {
*pRes = NULL; // no need to free it
break;
}
case CTG_TASK_GET_V_SUBTABLES: {
break;
}
default:
qError("invalid task type %d", type);
break;
@ -925,6 +935,9 @@ void ctgFreeSubTaskRes(CTG_TASK_TYPE type, void** pRes) {
taosArrayDestroyEx(*pRes, ctgFreeBatchMeta);
*pRes = NULL;
break;
}
case CTG_TASK_GET_V_SUBTABLES: {
}
default:
qError("invalid task type %d", type);
@ -1051,6 +1064,23 @@ void ctgFreeTaskCtx(SCtgTask* pTask) {
taosMemoryFreeClear(pTask->taskCtx);
break;
}
case CTG_TASK_GET_V_SUBTABLES: {
SCtgVSubTablesCtx* taskCtx = (SCtgVSubTablesCtx*)pTask->taskCtx;
if (taskCtx->clonedVgroups) {
taosArrayDestroy(taskCtx->pVgroups);
taskCtx->pVgroups = NULL;
}
if (taskCtx->pResList) {
for (int32_t i = 0; i < taskCtx->vgNum; ++i) {
SVSubTablesRsp* pVg = taskCtx->pResList + i;
tDestroySVSubTablesRsp(pVg);
}
taosMemoryFreeClear(taskCtx->pResList);
}
taosMemoryFreeClear(taskCtx->pMeta);
taosMemoryFreeClear(pTask->taskCtx);
break;
}
default:
qError("invalid task type %d", pTask->type);
break;
@ -1776,7 +1806,7 @@ int32_t ctgCloneTableIndex(SArray* pIndex, SArray** pRes) {
int32_t ctgUpdateSendTargetInfo(SMsgSendInfo* pMsgSendInfo, int32_t msgType, char* dbFName, int32_t vgId) {
if (msgType == TDMT_VND_TABLE_META || msgType == TDMT_VND_TABLE_CFG || msgType == TDMT_VND_BATCH_META ||
msgType == TDMT_VND_TABLE_NAME) {
msgType == TDMT_VND_TABLE_NAME || msgType == TDMT_VND_VSUBTABLES_META || msgType == TDMT_VND_GET_STREAM_PROGRESS) {
pMsgSendInfo->target.type = TARGET_TYPE_VNODE;
pMsgSendInfo->target.vgId = vgId;
pMsgSendInfo->target.dbFName = taosStrdup(dbFName);
@ -2812,3 +2842,70 @@ int32_t ctgAddTSMAFetch(SArray** pFetchs, int32_t dbIdx, int32_t tbIdx, int32_t*
return TSDB_CODE_SUCCESS;
}
int32_t ctgBuildNormalChildVtbList(SCtgVSubTablesCtx* pCtx) {
int32_t code = TSDB_CODE_SUCCESS, line = 0;
char tbFName[TSDB_TABLE_FNAME_LEN];
pCtx->pResList = taosMemoryCalloc(1, sizeof(*pCtx->pResList));
QUERY_CHECK_NULL(pCtx->pResList, code, line, _return, terrno);
pCtx->pResList->vgId = pCtx->pMeta->vgId;
pCtx->pResList->pTables = taosArrayInit(1, POINTER_BYTES);
QUERY_CHECK_NULL(pCtx->pResList->pTables, code, line, _return, terrno);
SSHashObj* pSrcTbls = tSimpleHashInit(10, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY));
QUERY_CHECK_NULL(pSrcTbls, code, line, _return, terrno);
int32_t refColsNum = 0;
for (int32_t i = 0; i < pCtx->pMeta->numOfColRefs; ++i) {
if (!pCtx->pMeta->colRef[i].hasRef) {
continue;
}
refColsNum++;
}
SVCTableRefCols* pTb = (SVCTableRefCols*)taosMemoryCalloc(1, refColsNum * sizeof(SRefColInfo) + sizeof(SVCTableRefCols));
QUERY_CHECK_NULL(pTb, code, line, _return, terrno);
pTb->uid = pCtx->pMeta->uid;
pTb->numOfColRefs = refColsNum;
pTb->refCols = (SRefColInfo*)(pTb + 1);
refColsNum = 0;
for (int32_t j = 0; j < pCtx->pMeta->numOfColRefs; j++) {
if (!pCtx->pMeta->colRef[j].hasRef) {
continue;
}
pTb->refCols[refColsNum].colId = pCtx->pMeta->colRef[j].id;
tstrncpy(pTb->refCols[refColsNum].refColName, pCtx->pMeta->colRef[j].refColName, TSDB_COL_NAME_LEN);
tstrncpy(pTb->refCols[refColsNum].refTableName, pCtx->pMeta->colRef[j].refTableName, TSDB_TABLE_NAME_LEN);
tstrncpy(pTb->refCols[refColsNum].refDbName,pCtx->pMeta->colRef[j].refDbName, TSDB_DB_NAME_LEN);
snprintf(tbFName, sizeof(tbFName), "%s.%s", pTb->refCols[refColsNum].refDbName, pTb->refCols[refColsNum].refTableName);
if (NULL == tSimpleHashGet(pSrcTbls, tbFName, strlen(tbFName))) {
QUERY_CHECK_CODE(tSimpleHashPut(pSrcTbls, tbFName, strlen(tbFName), &code, sizeof(code)), line, _return);
}
refColsNum++;
}
pTb->numOfSrcTbls = tSimpleHashGetSize(pSrcTbls);
QUERY_CHECK_NULL(taosArrayPush(pCtx->pResList->pTables, &pTb), code, line, _return, terrno);
pTb = NULL;
_return:
tSimpleHashCleanup(pSrcTbls);
taosMemoryFree(pTb);
if (code) {
qError("%s failed since %s", __func__, tstrerror(code));
}
return code;
}

View File

@ -31,6 +31,7 @@ extern "C" {
#include "executor.h"
#include "planner.h"
#include "scalar.h"
#include "streamVtableMerge.h"
#include "taosdef.h"
#include "tarray.h"
#include "tfill.h"
@ -545,6 +546,8 @@ typedef struct SStreamScanInfo {
uint64_t numOfExec; // execution times
STqReader* tqReader;
SHashObj* pVtableMergeHandles; // key: vtable uid, value: SStreamVtableMergeHandle
uint64_t groupId;
bool igCheckGroupId;
struct SUpdateInfo* pUpdateInfo;
@ -1224,6 +1227,21 @@ bool getIgoreNullRes(SExprSupp* pExprSup);
bool checkNullRow(SExprSupp* pExprSup, SSDataBlock* pSrcBlock, int32_t index, bool ignoreNull);
int64_t getMinWindowSize(struct SOperatorInfo* pOperator);
void destroyStreamScanOperatorInfo(void* param);
void prepareRangeScan(SStreamScanInfo* pInfo, SSDataBlock* pBlock, int32_t* pRowIndex, bool* pRes);
int32_t setBlockGroupIdByUid(SStreamScanInfo* pInfo, SSDataBlock* pBlock);
bool hasScanRange(SStreamScanInfo* pInfo);
bool isStreamWindow(SStreamScanInfo* pInfo);
int32_t copyGetResultBlock(SSDataBlock* dest, TSKEY start, TSKEY end);
int32_t generateDeleteResultBlock(SStreamScanInfo* pInfo, SSDataBlock* pSrcBlock, SSDataBlock* pDestBlock);
int32_t doCheckUpdate(SStreamScanInfo* pInfo, TSKEY endKey, SSDataBlock* pBlock);
int32_t checkUpdateData(SStreamScanInfo* pInfo, bool invertible, SSDataBlock* pBlock, bool out);
int32_t calBlockTbName(SStreamScanInfo* pInfo, SSDataBlock* pBlock, int32_t rowId);
int32_t setBlockIntoRes(SStreamScanInfo* pInfo, const SSDataBlock* pBlock, STimeWindow* pTimeWindow, bool filter);
int32_t generateScanRange(SStreamScanInfo* pInfo, SSDataBlock* pSrcBlock, SSDataBlock* pDestBlock, EStreamType type);
int32_t doRangeScan(SStreamScanInfo* pInfo, SSDataBlock* pSDB, int32_t tsColIndex, int32_t* pRowIndex,
SSDataBlock** ppRes);
#ifdef __cplusplus
}
#endif

View File

@ -188,6 +188,7 @@ int32_t createEventNonblockOperatorInfo(SOperatorInfo* downstream, SPhysiNode* p
int32_t createVirtualTableMergeOperatorInfo(SOperatorInfo** pDownstream, SReadHandle* readHandle, STableListInfo* pTableListInfo, int32_t numOfDownstream, SVirtualScanPhysiNode * pJoinNode, SExecTaskInfo* pTaskInfo, SOperatorInfo** pOptrInfo);
int32_t createStreamVtableMergeOperatorInfo(SReadHandle* pHandle, SVirtualScanPhysiNode* pVirtualScanNode, SNode* pTagCond, SExecTaskInfo* pTaskInfo, SOperatorInfo** pInfo);
// clang-format on
SOperatorFpSet createOperatorFpSet(__optr_open_fn_t openFn, __optr_fn_t nextFn, __optr_fn_t cleanup,
@ -207,7 +208,8 @@ SSDataBlock* getNextBlockFromDownstreamRemain(struct SOperatorInfo* pOperator,
int16_t getOperatorResultBlockId(struct SOperatorInfo* pOperator, int32_t idx);
int32_t createOperator(SPhysiNode* pPhyNode, SExecTaskInfo* pTaskInfo, SReadHandle* pHandle, SNode* pTagCond,
SNode* pTagIndexCond, const char* pUser, const char* dbname, SOperatorInfo** pOptrInfo);
SNode* pTagIndexCond, const char* pUser, const char* dbname, SOperatorInfo** pOptrInfo,
EOPTR_EXEC_MODEL model);
void destroyOperator(SOperatorInfo* pOperator);
void destroyOperatorAndDownstreams(SOperatorInfo* pOperator, SOperatorInfo** stream, int32_t num);

View File

@ -0,0 +1,47 @@
/*
* Copyright (c) 2019 TAOS Data, Inc. <jhtao@taosdata.com>
*
* This program is free software: you can use, redistribute, and/or modify
* it under the terms of the GNU Affero General Public License, version 3
* or later ("AGPL"), as published by the Free Software Foundation.
*
* This program is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE.
*
* You should have received a copy of the GNU Affero General Public License
* along with this program. If not, see <http://www.gnu.org/licenses/>.
*/
#ifndef TDENGINE_STREAM_VTABLE_MERGE_H
#define TDENGINE_STREAM_VTABLE_MERGE_H
#ifdef __cplusplus
extern "C" {
#endif
#include "tcommon.h"
#include "tpagedbuf.h"
typedef struct SStreamVtableMergeHandle SStreamVtableMergeHandle;
typedef enum {
SVM_NEXT_NOT_READY = 0,
SVM_NEXT_FOUND = 1,
} SVM_NEXT_RESULT;
int32_t streamVtableMergeCreateHandle(SStreamVtableMergeHandle **ppHandle, int32_t nSrcTbls, int32_t numPageLimit,
SDiskbasedBuf *pBuf, SSDataBlock *pResBlock, const char *idstr);
void streamVtableMergeDestroyHandle(SStreamVtableMergeHandle **ppHandle);
int32_t streamVtableMergeAddBlock(SStreamVtableMergeHandle *pHandle, SSDataBlock *pDataBlock, const char *idstr);
int32_t streamVtableMergeNextTuple(SStreamVtableMergeHandle *pHandle, SSDataBlock *pResBlock, SVM_NEXT_RESULT *pRes,
const char *idstr);
#ifdef __cplusplus
}
#endif
#endif // TDENGINE_STREAM_VTABLE_MERGE_H

View File

@ -141,7 +141,8 @@ static int32_t doSetStreamBlock(SOperatorInfo* pOperator, void* input, size_t nu
const char* id) {
int32_t code = TSDB_CODE_SUCCESS;
int32_t lino = 0;
if (pOperator->operatorType != QUERY_NODE_PHYSICAL_PLAN_STREAM_SCAN) {
if (pOperator->operatorType != QUERY_NODE_PHYSICAL_PLAN_STREAM_SCAN &&
pOperator->operatorType != QUERY_NODE_PHYSICAL_PLAN_VIRTUAL_TABLE_SCAN) {
if (pOperator->numOfDownstream == 0) {
qError("failed to find stream scan operator to set the input data block, %s" PRIx64, id);
return TSDB_CODE_APP_ERROR;

View File

@ -280,7 +280,8 @@ int32_t stopTableScanOperator(SOperatorInfo* pOperator, const char* pIdStr, SSto
}
int32_t createOperator(SPhysiNode* pPhyNode, SExecTaskInfo* pTaskInfo, SReadHandle* pHandle, SNode* pTagCond,
SNode* pTagIndexCond, const char* pUser, const char* dbname, SOperatorInfo** pOptrInfo) {
SNode* pTagIndexCond, const char* pUser, const char* dbname, SOperatorInfo** pOptrInfo,
EOPTR_EXEC_MODEL model) {
QRY_PARAM_CHECK(pOptrInfo);
int32_t code = 0;
@ -512,8 +513,10 @@ int32_t createOperator(SPhysiNode* pPhyNode, SExecTaskInfo* pTaskInfo, SReadHand
}
} else if (QUERY_NODE_PHYSICAL_PLAN_PROJECT == type) {
code = createProjectOperatorInfo(NULL, (SProjectPhysiNode*)pPhyNode, pTaskInfo, &pOperator);
} else if (QUERY_NODE_PHYSICAL_PLAN_VIRTUAL_TABLE_SCAN == type) {
} else if (QUERY_NODE_PHYSICAL_PLAN_VIRTUAL_TABLE_SCAN == type && model != OPTR_EXEC_MODEL_STREAM) {
code = createVirtualTableMergeOperatorInfo(NULL, pHandle, NULL, 0, (SVirtualScanPhysiNode*)pPhyNode, pTaskInfo, &pOperator);
} else if (QUERY_NODE_PHYSICAL_PLAN_VIRTUAL_TABLE_SCAN == type && model == OPTR_EXEC_MODEL_STREAM) {
code = createStreamVtableMergeOperatorInfo(pHandle, (SVirtualScanPhysiNode*)pPhyNode, pTagCond, pTaskInfo, &pOperator);
} else {
code = TSDB_CODE_INVALID_PARA;
pTaskInfo->code = code;
@ -538,7 +541,7 @@ int32_t createOperator(SPhysiNode* pPhyNode, SExecTaskInfo* pTaskInfo, SReadHand
for (int32_t i = 0; i < size; ++i) {
SPhysiNode* pChildNode = (SPhysiNode*)nodesListGetNode(pPhyNode->pChildren, i);
code = createOperator(pChildNode, pTaskInfo, pHandle, pTagCond, pTagIndexCond, pUser, dbname, &ops[i]);
code = createOperator(pChildNode, pTaskInfo, pHandle, pTagCond, pTagIndexCond, pUser, dbname, &ops[i], model);
if (ops[i] == NULL || code != 0) {
for (int32_t j = 0; j < i; ++j) {
destroyOperator(ops[j]);
@ -653,7 +656,7 @@ int32_t createOperator(SPhysiNode* pPhyNode, SExecTaskInfo* pTaskInfo, SReadHand
code = createEventNonblockOperatorInfo(ops[0], pPhyNode, pTaskInfo, pHandle, &pOptr);
} else if (QUERY_NODE_PHYSICAL_PLAN_STREAM_CONTINUE_COUNT == type) {
//todo (liuyao) add
} else if (QUERY_NODE_PHYSICAL_PLAN_VIRTUAL_TABLE_SCAN == type) {
} else if (QUERY_NODE_PHYSICAL_PLAN_VIRTUAL_TABLE_SCAN == type && model != OPTR_EXEC_MODEL_STREAM) {
SVirtualScanPhysiNode* pVirtualTableScanNode = (SVirtualScanPhysiNode*)pPhyNode;
// NOTE: this is an patch to fix the physical plan

View File

@ -126,7 +126,7 @@ int32_t createExecTaskInfo(SSubplan* pPlan, SExecTaskInfo** pTaskInfo, SReadHand
(*pTaskInfo)->pSubplan = pPlan;
(*pTaskInfo)->pWorkerCb = pHandle->pWorkerCb;
code = createOperator(pPlan->pNode, *pTaskInfo, pHandle, pPlan->pTagCond, pPlan->pTagIndexCond, pPlan->user,
pPlan->dbFName, &((*pTaskInfo)->pRoot));
pPlan->dbFName, &((*pTaskInfo)->pRoot), model);
if (NULL == (*pTaskInfo)->pRoot || code != 0) {
doDestroyTask(*pTaskInfo);

View File

@ -2844,7 +2844,7 @@ _end:
return code;
}
static int32_t generateDeleteResultBlock(SStreamScanInfo* pInfo, SSDataBlock* pSrcBlock, SSDataBlock* pDestBlock) {
int32_t generateDeleteResultBlock(SStreamScanInfo* pInfo, SSDataBlock* pSrcBlock, SSDataBlock* pDestBlock) {
blockDataCleanup(pDestBlock);
int32_t rows = pSrcBlock->info.rows;
if (rows == 0) {
@ -2860,8 +2860,7 @@ static int32_t generateDeleteResultBlock(SStreamScanInfo* pInfo, SSDataBlock* pS
return generateDeleteResultBlockImpl(pInfo, pSrcBlock, pDestBlock);
}
static int32_t generateScanRange(SStreamScanInfo* pInfo, SSDataBlock* pSrcBlock, SSDataBlock* pDestBlock,
EStreamType type) {
int32_t generateScanRange(SStreamScanInfo* pInfo, SSDataBlock* pSrcBlock, SSDataBlock* pDestBlock, EStreamType type) {
int32_t code = TSDB_CODE_SUCCESS;
int32_t lino = 0;
if (isIntervalWindow(pInfo)) {
@ -2908,7 +2907,7 @@ bool checkExpiredData(SStateStore* pAPI, SUpdateInfo* pUpdateInfo, STimeWindowAg
return isExpired;
}
static int32_t checkUpdateData(SStreamScanInfo* pInfo, bool invertible, SSDataBlock* pBlock, bool out) {
int32_t checkUpdateData(SStreamScanInfo* pInfo, bool invertible, SSDataBlock* pBlock, bool out) {
int32_t code = TSDB_CODE_SUCCESS;
int32_t lino = 0;
if (out) {
@ -3151,14 +3150,14 @@ int32_t colIdComparFn(const void* param1, const void* param2) {
}
}
int32_t setBlockIntoRes(SStreamScanInfo* pInfo, const SSDataBlock* pBlock, STimeWindow* pTimeWindow,
bool filter) {
int32_t setBlockIntoRes(SStreamScanInfo* pInfo, const SSDataBlock* pBlock, STimeWindow* pTimeWindow, bool filter) {
int32_t code = TSDB_CODE_SUCCESS;
int32_t lino = 0;
SDataBlockInfo* pBlockInfo = &pInfo->pRes->info;
SOperatorInfo* pOperator = pInfo->pStreamScanOp;
SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo;
const char* id = GET_TASKID(pTaskInfo);
SSHashObj* pVtableInfos = pTaskInfo->pSubplan->pVTables;
code = blockDataEnsureCapacity(pInfo->pRes, pBlock->info.rows);
QUERY_CHECK_CODE(code, lino, _end);
@ -3169,7 +3168,12 @@ int32_t setBlockIntoRes(SStreamScanInfo* pInfo, const SSDataBlock* pBlock, STime
pBlockInfo->version = pBlock->info.version;
STableScanInfo* pTableScanInfo = pInfo->pTableScanOp->info;
pBlockInfo->id.groupId = tableListGetTableGroupId(pTableScanInfo->base.pTableListInfo, pBlock->info.id.uid);
if (pVtableInfos == NULL) {
pBlockInfo->id.groupId = tableListGetTableGroupId(pTableScanInfo->base.pTableListInfo, pBlock->info.id.uid);
} else {
// use original table uid as groupId for vtable
pBlockInfo->id.groupId = pBlock->info.id.groupId;
}
SArray* pColList = taosArrayInit(4, sizeof(int32_t));
QUERY_CHECK_NULL(pColList, code, lino, _end, terrno);
@ -3491,7 +3495,7 @@ _end:
return code;
}
static int32_t doCheckUpdate(SStreamScanInfo* pInfo, TSKEY endKey, SSDataBlock* pBlock) {
int32_t doCheckUpdate(SStreamScanInfo* pInfo, TSKEY endKey, SSDataBlock* pBlock) {
int32_t code = TSDB_CODE_SUCCESS;
int32_t lino = 0;
if (pInfo->pUpdateInfo) {
@ -3690,17 +3694,17 @@ _end:
qInfo("%s end at line %d", __func__, lino);
}
static bool hasScanRange(SStreamScanInfo* pInfo) {
bool hasScanRange(SStreamScanInfo* pInfo) {
SStreamAggSupporter* pSup = pInfo->windowSup.pStreamAggSup;
return pSup && pSup->pScanBlock->info.rows > 0 && (isStateWindow(pInfo) || isCountWindow(pInfo));
}
static bool isStreamWindow(SStreamScanInfo* pInfo) {
bool isStreamWindow(SStreamScanInfo* pInfo) {
return isIntervalWindow(pInfo) || isSessionWindow(pInfo) || isStateWindow(pInfo) || isCountWindow(pInfo) ||
isTimeSlice(pInfo);
}
static int32_t copyGetResultBlock(SSDataBlock* dest, TSKEY start, TSKEY end) {
int32_t copyGetResultBlock(SSDataBlock* dest, TSKEY start, TSKEY end) {
int32_t code = blockDataEnsureCapacity(dest, 1);
if (code != TSDB_CODE_SUCCESS) {
return code;
@ -4473,7 +4477,7 @@ _end:
return code;
}
static void destroyStreamScanOperatorInfo(void* param) {
void destroyStreamScanOperatorInfo(void* param) {
if (param == NULL) {
return;
}
@ -4486,6 +4490,10 @@ static void destroyStreamScanOperatorInfo(void* param) {
if (pStreamScan->tqReader != NULL && pStreamScan->readerFn.tqReaderClose != NULL) {
pStreamScan->readerFn.tqReaderClose(pStreamScan->tqReader);
}
if (pStreamScan->pVtableMergeHandles) {
taosHashCleanup(pStreamScan->pVtableMergeHandles);
pStreamScan->pVtableMergeHandles = NULL;
}
if (pStreamScan->matchInfo.pList) {
taosArrayDestroy(pStreamScan->matchInfo.pList);
}
@ -4673,6 +4681,42 @@ _end:
return code;
}
static int32_t createStreamVtableBlock(SColMatchInfo *pMatchInfo, SSDataBlock **ppRes, const char *idstr) {
int32_t code = TSDB_CODE_SUCCESS;
int32_t lino = 0;
SSDataBlock *pRes = NULL;
QUERY_CHECK_NULL(pMatchInfo, code, lino, _end, TSDB_CODE_INVALID_PARA);
*ppRes = NULL;
code = createDataBlock(&pRes);
QUERY_CHECK_CODE(code, lino, _end);
int32_t numOfOutput = taosArrayGetSize(pMatchInfo->pList);
for (int32_t i = 0; i < numOfOutput; ++i) {
SColMatchItem* pItem = taosArrayGet(pMatchInfo->pList, i);
if (!pItem->needOutput) {
continue;
}
SColumnInfoData colInfo = createColumnInfoData(pItem->dataType.type, pItem->dataType.bytes, pItem->colId);
code = blockDataAppendColInfo(pRes, &colInfo);
QUERY_CHECK_CODE(code, lino, _end);
}
*ppRes = pRes;
pRes = NULL;
_end:
if (code != TSDB_CODE_SUCCESS) {
qError("%s failed at line %d since %s, id: %s", __func__, lino, tstrerror(code), idstr);
}
if (pRes != NULL) {
blockDataDestroy(pRes);
}
return code;
}
static int32_t createStreamNormalScanOperatorInfo(SReadHandle* pHandle, STableScanPhysiNode* pTableScanNode,
SNode* pTagCond, STableListInfo* pTableListInfo,
SExecTaskInfo* pTaskInfo, SOperatorInfo** pOptrInfo) {
@ -4685,6 +4729,7 @@ static int32_t createStreamNormalScanOperatorInfo(SReadHandle* pHandle, STableSc
SOperatorInfo* pOperator = taosMemoryCalloc(1, sizeof(SOperatorInfo));
SStorageAPI* pAPI = &pTaskInfo->storageAPI;
const char* idstr = pTaskInfo->id.str;
SSHashObj* pVtableInfos = pTaskInfo->pSubplan->pVTables;
if (pInfo == NULL || pOperator == NULL) {
code = terrno;
@ -4791,6 +4836,16 @@ static int32_t createStreamNormalScanOperatorInfo(SReadHandle* pHandle, STableSc
QUERY_CHECK_NULL(pInfo->tqReader, code, lino, _error, TSDB_CODE_QRY_EXECUTOR_INTERNAL_ERROR);
}
if (pVtableInfos != NULL) {
// save vtable info into tqReader for vtable source scan
SSDataBlock* pResBlock = NULL;
code = createStreamVtableBlock(&pInfo->matchInfo, &pResBlock, idstr);
QUERY_CHECK_CODE(code, lino, _error);
code = pAPI->tqReaderFn.tqReaderSetVtableInfo(pInfo->tqReader, pHandle->vnode, pAPI, pVtableInfos, &pResBlock,
idstr);
QUERY_CHECK_CODE(code, lino, _error);
}
pInfo->pUpdateInfo = NULL;
pInfo->pTableScanOp = pTableScanOp;
if (pInfo->pTableScanOp->pTaskInfo->streamInfo.pState) {
@ -4808,12 +4863,14 @@ static int32_t createStreamNormalScanOperatorInfo(SReadHandle* pHandle, STableSc
QUERY_CHECK_CODE(code, lino, _error);
// set the extract column id to streamHandle
pAPI->tqReaderFn.tqReaderSetColIdList(pInfo->tqReader, pColIds);
code = pAPI->tqReaderFn.tqReaderSetColIdList(pInfo->tqReader, pColIds, idstr);
QUERY_CHECK_CODE(code, lino, _error);
SArray* tableIdList = NULL;
code = extractTableIdList(((STableScanInfo*)(pInfo->pTableScanOp->info))->base.pTableListInfo, &tableIdList);
QUERY_CHECK_CODE(code, lino, _error);
pAPI->tqReaderFn.tqReaderSetQueryTableList(pInfo->tqReader, tableIdList, idstr);
code = pAPI->tqReaderFn.tqReaderSetQueryTableList(pInfo->tqReader, tableIdList, idstr);
QUERY_CHECK_CODE(code, lino, _error);
taosArrayDestroy(tableIdList);
memcpy(&pTaskInfo->streamInfo.tableCond, &pTSInfo->base.cond, sizeof(SQueryTableDataCond));
} else {

View File

@ -0,0 +1,607 @@
/*
* Copyright (c) 2019 TAOS Data, Inc. <jhtao@taosdata.com>
*
* This program is free software: you can use, redistribute, and/or modify
* it under the terms of the GNU Affero General Public License, version 3
* or later ("AGPL"), as published by the Free Software Foundation.
*
* This program is distributed in the hope that it will be useful, but WITHOUT
* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or
* FITNESS FOR A PARTICULAR PURPOSE.
*
* You should have received a copy of the GNU Affero General Public License
* along with this program. If not, see <http://www.gnu.org/licenses/>.
*/
#include "streamVtableMerge.h"
#include "query.h"
#include "tdatablock.h"
#include "tglobal.h"
#include "tlosertree.h"
typedef struct SVMBufPageInfo {
int32_t pageId;
int64_t expireTimeMs;
} SVMBufPageInfo;
typedef struct SStreamVtableMergeSource {
SDiskbasedBuf* pBuf; // buffer for storing data
int32_t* pTotalPages; // total pages of all sources in the buffer
SSDataBlock* pInputDataBlock; // data block to be written to the buffer
int64_t currentExpireTimeMs; // expire time of the input data block
SList* pageInfoList; // info of current source's page in the buffer
SSDataBlock* pOutputDataBlock; // data block read from the buffer
int32_t rowIndex; // current row index of the output data block
int64_t latestTs; // latest timestamp of the source
int64_t* pGlobalLatestTs; // global latest timestamp of all sources
} SStreamVtableMergeSource;
typedef struct SStreamVtableMergeHandle {
SDiskbasedBuf* pBuf;
int32_t numOfPages;
int32_t numPageLimit;
int32_t nSrcTbls;
SHashObj* pSources;
SSDataBlock* datablock; // Does not store data, only used to save the schema of input/output data blocks
SMultiwayMergeTreeInfo* pMergeTree;
int64_t globalLatestTs;
} SStreamVtableMergeHandle;
typedef enum StreamVirtualMergeMode {
STREAM_VIRTUAL_MERGE_WAIT_FOREVER,
STREAM_VIRTUAL_MERGE_MAX_DELAY,
STREAM_VIRTUAL_MERGE_MAX_MEMORY,
} StreamVirtualMergeMode;
static void svmSourceDestroy(void* ptr) {
SStreamVtableMergeSource** ppSource = ptr;
if (ppSource == NULL || *ppSource == NULL) {
return;
}
SStreamVtableMergeSource* pSource = *ppSource;
if (pSource->pInputDataBlock) {
blockDataDestroy(pSource->pInputDataBlock);
pSource->pInputDataBlock = NULL;
}
pSource->pageInfoList = tdListFree(pSource->pageInfoList);
if (pSource->pOutputDataBlock) {
blockDataDestroy(pSource->pOutputDataBlock);
pSource->pOutputDataBlock = NULL;
}
taosMemoryFreeClear(*ppSource);
}
static int32_t svmSourceFlushInput(SStreamVtableMergeSource* pSource, const char* idstr) {
int32_t code = TSDB_CODE_SUCCESS;
int32_t lino = 0;
SSDataBlock* pBlock = NULL;
SVMBufPageInfo pageInfo = {0};
void* page = NULL;
QUERY_CHECK_NULL(pSource, code, lino, _end, TSDB_CODE_INVALID_PARA);
// check data block size
pBlock = pSource->pInputDataBlock;
int32_t size = blockDataGetSize(pBlock) + sizeof(int32_t) + taosArrayGetSize(pBlock->pDataBlock) * sizeof(int32_t);
QUERY_CHECK_CONDITION(size <= getBufPageSize(pSource->pBuf), code, lino, _end, TSDB_CODE_INTERNAL_ERROR);
// allocate a new page and write data to it
page = getNewBufPage(pSource->pBuf, &pageInfo.pageId);
QUERY_CHECK_NULL(page, code, lino, _end, terrno);
code = blockDataToBuf(page, pBlock);
QUERY_CHECK_CODE(code, lino, _end);
setBufPageDirty(page, true);
// save page info
pageInfo.expireTimeMs = pSource->currentExpireTimeMs;
code = tdListAppend(pSource->pageInfoList, &pageInfo);
QUERY_CHECK_CODE(code, lino, _end);
releaseBufPage(pSource->pBuf, page);
page = NULL;
blockDataCleanup(pBlock);
(*pSource->pTotalPages)++;
_end:
if (code != TSDB_CODE_SUCCESS) {
qError("%s failed at line %d since %s, id: %s", __func__, lino, tstrerror(code), idstr);
}
if (page != NULL) {
dBufSetBufPageRecycled(pSource->pBuf, page);
}
return code;
}
static int32_t svmSourceAddBlock(SStreamVtableMergeSource* pSource, SSDataBlock* pDataBlock, const char* idstr) {
int32_t code = TSDB_CODE_SUCCESS;
int32_t lino = 0;
int32_t pageSize = 0;
int32_t holdSize = 0;
SSDataBlock* pInputDataBlock = NULL;
QUERY_CHECK_NULL(pDataBlock, code, lino, _end, TSDB_CODE_INVALID_PARA);
QUERY_CHECK_NULL(pSource, code, lino, _end, TSDB_CODE_INVALID_PARA);
pInputDataBlock = pSource->pInputDataBlock;
if (pInputDataBlock == NULL) {
code = createOneDataBlock(pDataBlock, false, &pInputDataBlock);
QUERY_CHECK_CODE(code, lino, _end);
pSource->pInputDataBlock = pInputDataBlock;
}
int32_t start = 0;
int32_t nrows = blockDataGetNumOfRows(pDataBlock);
while (start < nrows) {
int32_t holdSize = blockDataGetSize(pInputDataBlock);
QUERY_CHECK_CONDITION(holdSize < pageSize, code, lino, _end, TSDB_CODE_INTERNAL_ERROR);
int32_t stop = 0;
code = blockDataSplitRows(pDataBlock, pDataBlock->info.hasVarCol, start, &stop, pageSize - holdSize);
QUERY_CHECK_CODE(code, lino, _end);
if (stop == start - 1) {
// If pInputDataBlock cannot hold new rows, ignore the error and write pInputDataBlock to the buffer
} else {
// append new rows to pInputDataBlock
if (blockDataGetNumOfRows(pInputDataBlock) == 0) {
// set expires time for the first block
pSource->currentExpireTimeMs = taosGetTimestampMs() + tsStreamVirtualMergeMaxDelayMs;
}
int32_t numOfRows = stop - start + 1;
code = blockDataMergeNRows(pInputDataBlock, pDataBlock, start, numOfRows);
QUERY_CHECK_CODE(code, lino, _end);
}
start = stop + 1;
if (stop == nrows - 1) {
break;
} else {
// pInputDataBlock is full, write it to the buffer
code = svmSourceFlushInput(pSource, idstr);
QUERY_CHECK_CODE(code, lino, _end);
}
}
_end:
if (code != TSDB_CODE_SUCCESS) {
qError("%s failed at line %d since %s, id: %s", __func__, lino, tstrerror(code), idstr);
}
return code;
}
static bool svmSourceIsEmpty(SStreamVtableMergeSource* pSource) { return listNEles(pSource->pageInfoList) == 0; }
static int32_t svmSourceReadBuf(SStreamVtableMergeSource* pSource, const char* idstr) {
int32_t code = TSDB_CODE_SUCCESS;
int32_t lino = 0;
SListNode* pn = NULL;
SVMBufPageInfo* pageInfo = NULL;
void* page = NULL;
QUERY_CHECK_NULL(pSource, code, lino, _end, TSDB_CODE_INVALID_PARA);
QUERY_CHECK_CONDITION(!svmSourceIsEmpty(pSource), code, lino, _end, TSDB_CODE_INVALID_PARA);
QUERY_CHECK_NULL(pSource->pOutputDataBlock, code, lino, _end, TSDB_CODE_INVALID_PARA);
blockDataCleanup(pSource->pOutputDataBlock);
pn = tdListGetHead(pSource->pageInfoList);
QUERY_CHECK_NULL(pn, code, lino, _end, TSDB_CODE_INTERNAL_ERROR);
pageInfo = (SVMBufPageInfo*)pn->data;
QUERY_CHECK_NULL(pageInfo, code, lino, _end, TSDB_CODE_INTERNAL_ERROR);
page = getBufPage(pSource->pBuf, pageInfo->pageId);
QUERY_CHECK_NULL(page, code, lino, _end, terrno);
code = blockDataFromBuf(pSource->pOutputDataBlock, page);
QUERY_CHECK_CODE(code, lino, _end);
_end:
if (code != TSDB_CODE_SUCCESS) {
qError("%s failed at line %d since %s, id: %s", __func__, lino, tstrerror(code), idstr);
}
if (page) {
releaseBufPage(pSource->pBuf, page);
}
return code;
}
static int32_t svmSourceCurrentTs(SStreamVtableMergeSource* pSource, const char* idstr, int64_t* pTs) {
int32_t code = TSDB_CODE_SUCCESS;
int32_t lino = 0;
SColumnInfoData* tsCol = NULL;
QUERY_CHECK_NULL(pSource, code, lino, _end, TSDB_CODE_INVALID_PARA);
QUERY_CHECK_CONDITION(!svmSourceIsEmpty(pSource), code, lino, _end, TSDB_CODE_INVALID_PARA);
QUERY_CHECK_NULL(pSource->pOutputDataBlock, code, lino, _end, TSDB_CODE_INVALID_PARA);
if (blockDataGetNumOfRows(pSource->pOutputDataBlock) == 0) {
code = svmSourceReadBuf(pSource, idstr);
QUERY_CHECK_CODE(code, lino, _end);
}
QUERY_CHECK_CONDITION(pSource->rowIndex < blockDataGetNumOfRows(pSource->pOutputDataBlock), code, lino, _end,
TSDB_CODE_INVALID_PARA);
tsCol = taosArrayGet(pSource->pOutputDataBlock->pDataBlock, 0);
QUERY_CHECK_NULL(tsCol, code, lino, _end, terrno);
*pTs = ((int64_t*)tsCol->pData)[pSource->rowIndex];
pSource->latestTs = TMAX(*pTs, pSource->latestTs);
_end:
if (code != TSDB_CODE_SUCCESS) {
qError("%s failed at line %d since %s, id: %s", __func__, lino, tstrerror(code), idstr);
}
return code;
}
static int32_t svmSourceMoveNext(SStreamVtableMergeSource* pSource, const char* idstr, SVM_NEXT_RESULT* pRes) {
int32_t code = TSDB_CODE_SUCCESS;
int32_t lino = 0;
SListNode* pn = NULL;
void* page = NULL;
int64_t latestTs = 0;
QUERY_CHECK_NULL(pSource, code, lino, _end, TSDB_CODE_INVALID_PARA);
QUERY_CHECK_NULL(pSource->pOutputDataBlock, code, lino, _end, TSDB_CODE_INVALID_PARA);
*pRes = SVM_NEXT_NOT_READY;
latestTs = pSource->latestTs;
while (true) {
if (svmSourceIsEmpty(pSource)) {
pSource->rowIndex = 0;
break;
}
QUERY_CHECK_CONDITION(pSource->rowIndex < blockDataGetNumOfRows(pSource->pOutputDataBlock), code, lino, _end,
TSDB_CODE_INVALID_PARA);
pSource->rowIndex++;
if (pSource->rowIndex >= blockDataGetNumOfRows(pSource->pOutputDataBlock)) {
// Pop the page from the list and recycle it
pn = tdListPopHead(pSource->pageInfoList);
QUERY_CHECK_NULL(pn, code, lino, _end, TSDB_CODE_INTERNAL_ERROR);
QUERY_CHECK_NULL(pn->data, code, lino, _end, TSDB_CODE_INTERNAL_ERROR);
SVMBufPageInfo* pageInfo = (SVMBufPageInfo*)pn->data;
page = getBufPage(pSource->pBuf, pageInfo->pageId);
QUERY_CHECK_NULL(page, code, lino, _end, terrno);
code = dBufSetBufPageRecycled(pSource->pBuf, page);
QUERY_CHECK_CODE(code, lino, _end);
(*pSource->pTotalPages)--;
taosMemoryFreeClear(pn);
pSource->rowIndex = 0;
}
if (svmSourceIsEmpty(pSource)) {
pSource->rowIndex = 0;
break;
}
int64_t ts = 0;
code = svmSourceCurrentTs(pSource, idstr, &ts);
QUERY_CHECK_CODE(code, lino, _end);
if (ts > latestTs && ts >= *pSource->pGlobalLatestTs) {
*pRes = SVM_NEXT_FOUND;
break;
}
}
_end:
if (code != TSDB_CODE_SUCCESS) {
qError("%s failed at line %d since %s, id: %s", __func__, lino, tstrerror(code), idstr);
}
return code;
}
static int32_t svmSourceCompare(const void* pLeft, const void* pRight, void* param) {
int32_t left = *(int32_t*)pLeft;
int32_t right = *(int32_t*)pRight;
int32_t code = TSDB_CODE_SUCCESS;
SArray* pValidSources = param;
SStreamVtableMergeSource* pLeftSource = *(SStreamVtableMergeSource**)taosArrayGet(pValidSources, left);
SStreamVtableMergeSource* pRightSource = *(SStreamVtableMergeSource**)taosArrayGet(pValidSources, right);
int64_t leftTs = 0;
code = svmSourceCurrentTs(pLeftSource, "", &leftTs);
if (code != TSDB_CODE_SUCCESS) {
return -1;
}
int64_t rightTs = 0;
code = svmSourceCurrentTs(pRightSource, "", &rightTs);
if (code != TSDB_CODE_SUCCESS) {
return 1;
}
if (leftTs < rightTs) {
return -1;
} else if (leftTs > rightTs) {
return 1;
} else {
return 0;
}
}
static SStreamVtableMergeSource* svmAddSource(SStreamVtableMergeHandle* pHandle, int64_t uid, const char* idstr) {
int32_t code = TSDB_CODE_SUCCESS;
int32_t lino = 0;
SStreamVtableMergeSource* pSource = NULL;
pSource = taosMemoryCalloc(1, sizeof(SStreamVtableMergeSource));
QUERY_CHECK_NULL(pSource, code, lino, _end, terrno);
pSource->pBuf = pHandle->pBuf;
pSource->pTotalPages = &pHandle->numOfPages;
pSource->pageInfoList = tdListNew(sizeof(SVMBufPageInfo));
QUERY_CHECK_NULL(pSource->pageInfoList, code, lino, _end, terrno);
code = createOneDataBlock(pHandle->datablock, false, &pSource->pOutputDataBlock);
QUERY_CHECK_CODE(code, lino, _end);
pSource->latestTs = INT64_MIN;
pSource->pGlobalLatestTs = &pHandle->globalLatestTs;
code = taosHashPut(pHandle->pSources, &uid, sizeof(uid), &pSource, POINTER_BYTES);
QUERY_CHECK_CODE(code, lino, _end);
_end:
if (code != TSDB_CODE_SUCCESS) {
qError("%s failed at line %d since %s, id: %s", __func__, lino, tstrerror(code), idstr);
svmSourceDestroy(&pSource);
pSource = NULL;
terrno = code;
}
return pSource;
}
static void svmDestroyTree(void* ptr) {
SMultiwayMergeTreeInfo** ppTree = ptr;
if (ppTree == NULL || *ppTree == NULL) {
return;
}
SMultiwayMergeTreeInfo* pTree = *ppTree;
if (pTree->param != NULL) {
taosArrayDestroy(pTree->param);
pTree->param = NULL;
}
tMergeTreeDestroy(ppTree);
}
static int32_t svmBuildTree(SStreamVtableMergeHandle* pHandle, SVM_NEXT_RESULT* pRes, const char* idstr) {
int32_t code = TSDB_CODE_SUCCESS;
int32_t lino = 0;
SArray* pReadySources = NULL;
void* pIter = NULL;
void* px = NULL;
int64_t globalExpireTimeMs = INT64_MAX;
QUERY_CHECK_NULL(pHandle, code, lino, _end, TSDB_CODE_INVALID_PARA);
QUERY_CHECK_NULL(pRes, code, lino, _end, TSDB_CODE_INVALID_PARA);
*pRes = SVM_NEXT_NOT_READY;
// find all non-empty sources
pReadySources = taosArrayInit(pHandle->nSrcTbls, POINTER_BYTES);
pIter = taosHashIterate(pHandle->pSources, NULL);
while (pIter != NULL) {
SStreamVtableMergeSource* pSource = *(SStreamVtableMergeSource**)pIter;
if (svmSourceIsEmpty(pSource)) {
code = svmSourceFlushInput(pSource, idstr);
QUERY_CHECK_CODE(code, lino, _end);
}
if (!svmSourceIsEmpty(pSource)) {
px = taosArrayPush(pReadySources, &pSource);
QUERY_CHECK_NULL(px, code, lino, _end, terrno);
globalExpireTimeMs = TMIN(globalExpireTimeMs, pSource->currentExpireTimeMs);
}
pIter = taosHashIterate(pHandle->pSources, pIter);
}
if (taosArrayGetSize(pReadySources) == 0) {
// no available sources
goto _end;
}
if (taosArrayGetSize(pReadySources) < pHandle->nSrcTbls) {
// some sources are still empty
switch (tsStreamVirtualMergeWaitMode) {
case STREAM_VIRTUAL_MERGE_WAIT_FOREVER:
goto _end; // wait forever
case STREAM_VIRTUAL_MERGE_MAX_DELAY:
if (taosGetTimestampMs() < globalExpireTimeMs) {
goto _end; // wait for max delay
}
break;
case STREAM_VIRTUAL_MERGE_MAX_MEMORY:
if (pHandle->numOfPages < pHandle->numPageLimit) {
goto _end; // wait for max memory
}
break;
};
}
void* param = NULL;
code = tMergeTreeCreate(&pHandle->pMergeTree, taosArrayGetSize(pReadySources), pReadySources, svmSourceCompare);
QUERY_CHECK_CODE(code, lino, _end);
pReadySources = NULL;
*pRes = SVM_NEXT_FOUND;
_end:
if (code != TSDB_CODE_SUCCESS) {
qError("%s failed at line %d since %s, id: %s", __func__, lino, tstrerror(code), idstr);
}
if (pReadySources != NULL) {
taosArrayDestroy(pReadySources);
}
if (pIter != NULL) {
taosHashCancelIterate(pHandle->pSources, pIter);
}
return code;
}
int32_t streamVtableMergeAddBlock(SStreamVtableMergeHandle* pHandle, SSDataBlock* pDataBlock, const char* idstr) {
int32_t code = TSDB_CODE_SUCCESS;
int32_t lino = 0;
int64_t pTbUid = 0;
void* px = 0;
SStreamVtableMergeSource* pSource = NULL;
QUERY_CHECK_NULL(pHandle, code, lino, _end, TSDB_CODE_INVALID_PARA);
QUERY_CHECK_NULL(pDataBlock, code, lino, _end, TSDB_CODE_INVALID_PARA);
pTbUid = pDataBlock->info.id.uid;
px = taosHashGet(pHandle->pSources, &pTbUid, sizeof(int64_t));
if (px == NULL) {
if (taosHashGetSize(pHandle->pSources) >= pHandle->nSrcTbls) {
qError("Number of source tables exceeded the limit %d, table uid: %" PRId64, pHandle->nSrcTbls, pTbUid);
code = TSDB_CODE_INTERNAL_ERROR;
QUERY_CHECK_CODE(code, lino, _end);
}
// try to allocate a new source
pSource = svmAddSource(pHandle, pTbUid, idstr);
QUERY_CHECK_NULL(pSource, code, lino, _end, terrno);
} else {
pSource = *(SStreamVtableMergeSource**)px;
QUERY_CHECK_NULL(pSource, code, lino, _end, TSDB_CODE_INTERNAL_ERROR);
}
code = svmSourceAddBlock(pSource, pDataBlock, idstr);
QUERY_CHECK_CODE(code, lino, _end);
_end:
if (code != TSDB_CODE_SUCCESS) {
qError("%s failed at line %d since %s, id: %s", __func__, lino, tstrerror(code), idstr);
}
return code;
}
int32_t streamVtableMergeNextTuple(SStreamVtableMergeHandle* pHandle, SSDataBlock* pResBlock, SVM_NEXT_RESULT* pRes,
const char* idstr) {
int32_t code = TSDB_CODE_SUCCESS;
int32_t lino = 0;
void* px = NULL;
SArray* pReadySources = NULL;
SStreamVtableMergeSource* pSource = NULL;
QUERY_CHECK_NULL(pHandle, code, lino, _end, TSDB_CODE_INVALID_PARA);
QUERY_CHECK_NULL(pResBlock, code, lino, _end, TSDB_CODE_INVALID_PARA);
QUERY_CHECK_NULL(pRes, code, lino, _end, TSDB_CODE_INVALID_PARA);
*pRes = SVM_NEXT_NOT_READY;
if (pHandle->pMergeTree == NULL) {
SVM_NEXT_RESULT buildRes = SVM_NEXT_NOT_READY;
code = svmBuildTree(pHandle, &buildRes, idstr);
QUERY_CHECK_CODE(code, lino, _end);
if (buildRes == SVM_NEXT_NOT_READY) {
goto _end;
}
}
QUERY_CHECK_NULL(pHandle->pMergeTree, code, lino, _end, TSDB_CODE_INTERNAL_ERROR);
int32_t idx = tMergeTreeGetChosenIndex(pHandle->pMergeTree);
pReadySources = pHandle->pMergeTree->param;
px = taosArrayGet(pReadySources, idx);
QUERY_CHECK_NULL(px, code, lino, _end, terrno);
pSource = *(SStreamVtableMergeSource**)px;
code = blockCopyOneRow(pSource->pOutputDataBlock, pSource->rowIndex, &pResBlock);
QUERY_CHECK_CODE(code, lino, _end);
*pRes = SVM_NEXT_FOUND;
pHandle->globalLatestTs = TMAX(pSource->latestTs, pHandle->globalLatestTs);
SVM_NEXT_RESULT nextRes = SVM_NEXT_NOT_READY;
int32_t origNumOfPages = pHandle->numOfPages;
code = svmSourceMoveNext(pSource, idstr, &nextRes);
QUERY_CHECK_CODE(code, lino, _end);
bool needDestroy = false;
if (nextRes == SVM_NEXT_NOT_READY) {
needDestroy = true;
} else if (taosArrayGetSize((SArray*)pHandle->pMergeTree->param) != pHandle->nSrcTbls &&
pHandle->numOfPages != origNumOfPages) {
// The original data for this portion is incomplete. Its merge was forcibly triggered by certain conditions, so we
// must recheck if those conditions are still met.
if (tsStreamVirtualMergeWaitMode == STREAM_VIRTUAL_MERGE_MAX_DELAY) {
int64_t globalExpireTimeMs = INT64_MAX;
for (int32_t i = 0; i < taosArrayGetSize(pReadySources); ++i) {
px = taosArrayGet(pReadySources, i);
QUERY_CHECK_NULL(px, code, lino, _end, TSDB_CODE_INTERNAL_ERROR);
pSource = *(SStreamVtableMergeSource**)px;
globalExpireTimeMs = TMIN(globalExpireTimeMs, pSource->currentExpireTimeMs);
}
needDestroy = taosGetTimestampMs() < globalExpireTimeMs;
} else if (tsStreamVirtualMergeWaitMode == STREAM_VIRTUAL_MERGE_MAX_MEMORY) {
needDestroy = pHandle->numOfPages < pHandle->numPageLimit;
} else {
code = TSDB_CODE_INTERNAL_ERROR;
QUERY_CHECK_CODE(code, lino, _end);
}
}
if (needDestroy) {
svmDestroyTree(&pHandle->pMergeTree);
}
_end:
if (code != TSDB_CODE_SUCCESS) {
qError("%s failed at line %d since %s, id: %s", __func__, lino, tstrerror(code), idstr);
}
return code;
}
int32_t streamVtableMergeCreateHandle(SStreamVtableMergeHandle** ppHandle, int32_t nSrcTbls, int32_t numPageLimit,
SDiskbasedBuf* pBuf, SSDataBlock* pResBlock, const char* idstr) {
int32_t code = TSDB_CODE_SUCCESS;
int32_t lino = 0;
SStreamVtableMergeHandle* pHandle = NULL;
QUERY_CHECK_NULL(ppHandle, code, lino, _end, TSDB_CODE_INVALID_PARA);
QUERY_CHECK_CONDITION(nSrcTbls > 0, code, lino, _end, TSDB_CODE_INVALID_PARA);
QUERY_CHECK_NULL(pBuf, code, lino, _end, TSDB_CODE_INVALID_PARA);
*ppHandle = NULL;
pHandle = taosMemoryCalloc(1, sizeof(SStreamVtableMergeHandle));
QUERY_CHECK_NULL(pHandle, code, lino, _end, terrno);
pHandle->pBuf = pBuf;
pHandle->numPageLimit = numPageLimit;
pHandle->nSrcTbls = nSrcTbls;
pHandle->pSources = taosHashInit(8, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BIGINT), false, HASH_NO_LOCK);
QUERY_CHECK_NULL(pHandle->pSources, code, lino, _end, terrno);
taosHashSetFreeFp(pHandle->pSources, svmSourceDestroy);
code = createOneDataBlock(pResBlock, false, &pHandle->datablock);
QUERY_CHECK_CODE(code, lino, _end);
pHandle->globalLatestTs = INT64_MIN;
*ppHandle = pHandle;
pHandle = NULL;
_end:
if (code != TSDB_CODE_SUCCESS) {
qError("%s failed at line %d since %s, id: %s", __func__, lino, tstrerror(code), idstr);
}
if (pHandle != NULL) {
streamVtableMergeDestroyHandle(&pHandle);
}
return code;
}
void streamVtableMergeDestroyHandle(SStreamVtableMergeHandle** ppHandle) {
if (ppHandle == NULL || *ppHandle == NULL) {
return;
}
SStreamVtableMergeHandle* pHandle = *ppHandle;
if (pHandle->pSources != NULL) {
taosHashCleanup(pHandle->pSources);
pHandle->pSources = NULL;
}
svmDestroyTree(&pHandle->pMergeTree);
taosMemoryFreeClear(*ppHandle);
}

View File

@ -17,10 +17,15 @@
#include "filter.h"
#include "operator.h"
#include "querytask.h"
#include "streamexecutorInt.h"
#include "tdatablock.h"
#include "ttime.h"
#include "virtualtablescan.h"
#include "tsort.h"
#define STREAM_VTABLE_MERGE_OP_NAME "StreamVtableMergeOperator"
#define STREAM_VTABLE_MERGE_OP_CHECKPOINT_NAME "StreamVtableMergeOperator_Checkpoint"
typedef struct SVirtualTableScanInfo {
STableScanBase base;
SArray* pSortInfo;
@ -759,3 +764,591 @@ _return:
destroyOperatorAndDownstreams(pOperator, pDownstream, numOfDownstream);
return code;
}
static int32_t doStreamVtableMergeNext(SOperatorInfo* pOperator, SSDataBlock** ppRes) {
// NOTE: this operator does never check if current status is done or not
int32_t code = TSDB_CODE_SUCCESS;
int32_t lino = 0;
SExecTaskInfo* pTaskInfo = pOperator->pTaskInfo;
const char* id = GET_TASKID(pTaskInfo);
SStorageAPI* pAPI = &pTaskInfo->storageAPI;
SStreamScanInfo* pInfo = pOperator->info;
SStreamTaskInfo* pStreamInfo = &pTaskInfo->streamInfo;
qDebug("stream scan started, %s", id);
// TODO(kjq): add fill history recover step
size_t total = taosArrayGetSize(pInfo->pBlockLists);
// TODO: refactor
FETCH_NEXT_BLOCK:
if (pInfo->blockType == STREAM_INPUT__DATA_BLOCK) {
if (pInfo->validBlockIndex >= total) {
doClearBufferedBlocks(pInfo);
(*ppRes) = NULL;
return code;
}
int32_t current = pInfo->validBlockIndex++;
qDebug("process %d/%d input data blocks, %s", current, (int32_t)total, id);
SPackedData* pPacked = taosArrayGet(pInfo->pBlockLists, current);
QUERY_CHECK_NULL(pPacked, code, lino, _end, terrno);
SSDataBlock* pBlock = pPacked->pDataBlock;
if (pBlock->info.parTbName[0]) {
code =
pAPI->stateStore.streamStatePutParName(pStreamInfo->pState, pBlock->info.id.groupId, pBlock->info.parTbName);
QUERY_CHECK_CODE(code, lino, _end);
}
// TODO move into scan
pBlock->info.calWin.skey = INT64_MIN;
pBlock->info.calWin.ekey = INT64_MAX;
pBlock->info.dataLoad = 1;
if (pInfo->pUpdateInfo) {
pInfo->pUpdateInfo->maxDataVersion = TMAX(pInfo->pUpdateInfo->maxDataVersion, pBlock->info.version);
}
code = blockDataUpdateTsWindow(pBlock, 0);
QUERY_CHECK_CODE(code, lino, _end);
switch (pBlock->info.type) {
case STREAM_NORMAL:
case STREAM_GET_ALL:
printDataBlock(pBlock, getStreamOpName(pOperator->operatorType), GET_TASKID(pTaskInfo));
setStreamOperatorState(&pInfo->basic, pBlock->info.type);
(*ppRes) = pBlock;
return code;
case STREAM_RETRIEVE: {
pInfo->blockType = STREAM_INPUT__DATA_SUBMIT;
pInfo->scanMode = STREAM_SCAN_FROM_DATAREADER_RETRIEVE;
code = copyDataBlock(pInfo->pUpdateRes, pBlock);
QUERY_CHECK_CODE(code, lino, _end);
pInfo->updateResIndex = 0;
prepareRangeScan(pInfo, pInfo->pUpdateRes, &pInfo->updateResIndex, NULL);
pAPI->stateStore.updateInfoAddCloseWindowSBF(pInfo->pUpdateInfo);
} break;
case STREAM_DELETE_DATA: {
printSpecDataBlock(pBlock, getStreamOpName(pOperator->operatorType), "delete recv", GET_TASKID(pTaskInfo));
SSDataBlock* pDelBlock = NULL;
if (pInfo->tqReader) {
code = createSpecialDataBlock(STREAM_DELETE_DATA, &pDelBlock);
QUERY_CHECK_CODE(code, lino, _end);
code = filterDelBlockByUid(pDelBlock, pBlock, pInfo->tqReader, &pInfo->readerFn);
QUERY_CHECK_CODE(code, lino, _end);
} else {
pDelBlock = pBlock;
}
code = setBlockGroupIdByUid(pInfo, pDelBlock);
QUERY_CHECK_CODE(code, lino, _end);
code = rebuildDeleteBlockData(pDelBlock, &pStreamInfo->fillHistoryWindow, id);
QUERY_CHECK_CODE(code, lino, _end);
printSpecDataBlock(pDelBlock, getStreamOpName(pOperator->operatorType), "delete recv filtered",
GET_TASKID(pTaskInfo));
if (pDelBlock->info.rows == 0) {
if (pInfo->tqReader) {
blockDataDestroy(pDelBlock);
}
goto FETCH_NEXT_BLOCK;
}
if (!isStreamWindow(pInfo)) {
code = generateDeleteResultBlock(pInfo, pDelBlock, pInfo->pDeleteDataRes);
QUERY_CHECK_CODE(code, lino, _end);
if (pInfo->partitionSup.needCalc) {
pInfo->pDeleteDataRes->info.type = STREAM_DELETE_DATA;
} else {
pInfo->pDeleteDataRes->info.type = STREAM_DELETE_RESULT;
}
blockDataDestroy(pDelBlock);
if (pInfo->pDeleteDataRes->info.rows > 0) {
printSpecDataBlock(pInfo->pDeleteDataRes, getStreamOpName(pOperator->operatorType), "delete result",
GET_TASKID(pTaskInfo));
setStreamOperatorState(&pInfo->basic, pInfo->pDeleteDataRes->info.type);
(*ppRes) = pInfo->pDeleteDataRes;
return code;
} else {
goto FETCH_NEXT_BLOCK;
}
} else {
pInfo->blockType = STREAM_INPUT__DATA_SUBMIT;
pInfo->updateResIndex = 0;
code = generateScanRange(pInfo, pDelBlock, pInfo->pUpdateRes, STREAM_DELETE_DATA);
QUERY_CHECK_CODE(code, lino, _end);
prepareRangeScan(pInfo, pInfo->pUpdateRes, &pInfo->updateResIndex, NULL);
code = copyDataBlock(pInfo->pDeleteDataRes, pInfo->pUpdateRes);
QUERY_CHECK_CODE(code, lino, _end);
pInfo->pDeleteDataRes->info.type = STREAM_DELETE_DATA;
if (pInfo->tqReader) {
blockDataDestroy(pDelBlock);
}
if (pInfo->pDeleteDataRes->info.rows > 0) {
pInfo->scanMode = STREAM_SCAN_FROM_DATAREADER_RANGE;
printSpecDataBlock(pInfo->pDeleteDataRes, getStreamOpName(pOperator->operatorType), "delete result",
GET_TASKID(pTaskInfo));
setStreamOperatorState(&pInfo->basic, pInfo->pDeleteDataRes->info.type);
(*ppRes) = pInfo->pDeleteDataRes;
return code;
} else {
goto FETCH_NEXT_BLOCK;
}
}
} break;
case STREAM_GET_RESULT: {
pInfo->blockType = STREAM_INPUT__DATA_SUBMIT;
pInfo->updateResIndex = 0;
pInfo->lastScanRange = pBlock->info.window;
TSKEY endKey = taosTimeGetIntervalEnd(pBlock->info.window.skey, &pInfo->interval);
if (pInfo->useGetResultRange == true) {
endKey = pBlock->info.window.ekey;
}
code = copyGetResultBlock(pInfo->pUpdateRes, pBlock->info.window.skey, endKey);
QUERY_CHECK_CODE(code, lino, _end);
pInfo->pUpdateInfo->maxDataVersion = -1;
prepareRangeScan(pInfo, pInfo->pUpdateRes, &pInfo->updateResIndex, NULL);
pInfo->scanMode = STREAM_SCAN_FROM_DATAREADER_RANGE;
} break;
case STREAM_DROP_CHILD_TABLE: {
int32_t deleteNum = 0;
code = deletePartName(&pInfo->stateStore, pInfo->pStreamScanOp->pTaskInfo->streamInfo.pState, pBlock, &deleteNum);
QUERY_CHECK_CODE(code, lino, _end);
if (deleteNum == 0) {
printSpecDataBlock(pBlock, getStreamOpName(pOperator->operatorType), "block recv", GET_TASKID(pTaskInfo));
qDebug("===stream=== ignore block type 18, delete num is 0");
goto FETCH_NEXT_BLOCK;
}
} break;
case STREAM_CHECKPOINT: {
qError("stream check point error. msg type: STREAM_INPUT__DATA_BLOCK");
} break;
default:
break;
}
printSpecDataBlock(pBlock, getStreamOpName(pOperator->operatorType), "block recv", GET_TASKID(pTaskInfo));
setStreamOperatorState(&pInfo->basic, pBlock->info.type);
(*ppRes) = pBlock;
return code;
} else if (pInfo->blockType == STREAM_INPUT__DATA_SUBMIT) {
qDebug("stream scan mode:%d, %s", pInfo->scanMode, id);
switch (pInfo->scanMode) {
case STREAM_SCAN_FROM_RES: {
pInfo->scanMode = STREAM_SCAN_FROM_READERHANDLE;
code = doCheckUpdate(pInfo, pInfo->pRes->info.window.ekey, pInfo->pRes);
QUERY_CHECK_CODE(code, lino, _end);
setStreamOperatorState(&pInfo->basic, pInfo->pRes->info.type);
code = doFilter(pInfo->pRes, pOperator->exprSupp.pFilterInfo, NULL);
QUERY_CHECK_CODE(code, lino, _end);
pInfo->pRes->info.dataLoad = 1;
code = blockDataUpdateTsWindow(pInfo->pRes, pInfo->primaryTsIndex);
QUERY_CHECK_CODE(code, lino, _end);
if (pInfo->pRes->info.rows > 0) {
printDataBlock(pInfo->pRes, getStreamOpName(pOperator->operatorType), GET_TASKID(pTaskInfo));
(*ppRes) = pInfo->pRes;
return code;
}
} break;
case STREAM_SCAN_FROM_DELETE_DATA: {
code = generateScanRange(pInfo, pInfo->pUpdateDataRes, pInfo->pUpdateRes, STREAM_PARTITION_DELETE_DATA);
QUERY_CHECK_CODE(code, lino, _end);
if (pInfo->pUpdateRes->info.rows > 0) {
prepareRangeScan(pInfo, pInfo->pUpdateRes, &pInfo->updateResIndex, NULL);
pInfo->scanMode = STREAM_SCAN_FROM_DATAREADER_RANGE;
code = copyDataBlock(pInfo->pDeleteDataRes, pInfo->pUpdateRes);
QUERY_CHECK_CODE(code, lino, _end);
pInfo->pDeleteDataRes->info.type = STREAM_DELETE_DATA;
(*ppRes) = pInfo->pDeleteDataRes;
return code;
}
qError("%s===stream=== %s failed at line %d since pInfo->pUpdateRes is empty", GET_TASKID(pTaskInfo), __func__,
__LINE__);
blockDataCleanup(pInfo->pUpdateDataRes);
pInfo->scanMode = STREAM_SCAN_FROM_READERHANDLE;
} break;
case STREAM_SCAN_FROM_UPDATERES: {
code = generateScanRange(pInfo, pInfo->pUpdateDataRes, pInfo->pUpdateRes, STREAM_CLEAR);
QUERY_CHECK_CODE(code, lino, _end);
if (pInfo->pUpdateRes->info.rows > 0) {
prepareRangeScan(pInfo, pInfo->pUpdateRes, &pInfo->updateResIndex, NULL);
pInfo->scanMode = STREAM_SCAN_FROM_DATAREADER_RANGE;
(*ppRes) = pInfo->pUpdateRes;
return code;
}
qError("%s===stream=== %s failed at line %d since pInfo->pUpdateRes is empty", GET_TASKID(pTaskInfo), __func__,
__LINE__);
blockDataCleanup(pInfo->pUpdateDataRes);
pInfo->scanMode = STREAM_SCAN_FROM_READERHANDLE;
} break;
case STREAM_SCAN_FROM_DATAREADER_RANGE:
case STREAM_SCAN_FROM_DATAREADER_RETRIEVE: {
if (pInfo->pRangeScanRes != NULL) {
(*ppRes) = pInfo->pRangeScanRes;
pInfo->pRangeScanRes = NULL;
return code;
}
SSDataBlock* pSDB = NULL;
code = doRangeScan(pInfo, pInfo->pUpdateRes, pInfo->primaryTsIndex, &pInfo->updateResIndex, &pSDB);
QUERY_CHECK_CODE(code, lino, _end);
if (pSDB) {
STableScanInfo* pTableScanInfo = pInfo->pTableScanOp->info;
pSDB->info.type = pInfo->scanMode == STREAM_SCAN_FROM_DATAREADER_RANGE ? STREAM_NORMAL : STREAM_PULL_DATA;
if (!pInfo->igCheckUpdate && pInfo->pUpdateInfo) {
code = checkUpdateData(pInfo, true, pSDB, false);
QUERY_CHECK_CODE(code, lino, _end);
}
printSpecDataBlock(pSDB, getStreamOpName(pOperator->operatorType), "update", GET_TASKID(pTaskInfo));
code = calBlockTbName(pInfo, pSDB, 0);
QUERY_CHECK_CODE(code, lino, _end);
if (pInfo->pCreateTbRes->info.rows > 0) {
printSpecDataBlock(pInfo->pCreateTbRes, getStreamOpName(pOperator->operatorType), "update",
GET_TASKID(pTaskInfo));
(*ppRes) = pInfo->pCreateTbRes;
pInfo->pRangeScanRes = pSDB;
return code;
}
(*ppRes) = pSDB;
return code;
}
blockDataCleanup(pInfo->pUpdateDataRes);
pInfo->scanMode = STREAM_SCAN_FROM_READERHANDLE;
} break;
default:
break;
}
if (hasScanRange(pInfo)) {
pInfo->scanMode = STREAM_SCAN_FROM_DATAREADER_RANGE;
pInfo->updateResIndex = 0;
SStreamAggSupporter* pSup = pInfo->windowSup.pStreamAggSup;
code = copyDataBlock(pInfo->pUpdateRes, pSup->pScanBlock);
QUERY_CHECK_CODE(code, lino, _end);
blockDataCleanup(pSup->pScanBlock);
prepareRangeScan(pInfo, pInfo->pUpdateRes, &pInfo->updateResIndex, NULL);
pInfo->pUpdateRes->info.type = STREAM_DELETE_DATA;
printSpecDataBlock(pInfo->pUpdateRes, getStreamOpName(pOperator->operatorType), "rebuild", GET_TASKID(pTaskInfo));
(*ppRes) = pInfo->pUpdateRes;
return code;
}
SDataBlockInfo* pBlockInfo = &pInfo->pRes->info;
int32_t totalBlocks = taosArrayGetSize(pInfo->pBlockLists);
NEXT_SUBMIT_BLK:
while (1) {
if (pInfo->readerFn.tqReaderCurrentBlockConsumed(pInfo->tqReader)) {
if (pInfo->validBlockIndex >= totalBlocks) {
pAPI->stateStore.updateInfoDestoryColseWinSBF(pInfo->pUpdateInfo);
doClearBufferedBlocks(pInfo);
qDebug("stream scan return empty, all %d submit blocks consumed, %s", totalBlocks, id);
(*ppRes) = NULL;
return code;
}
int32_t current = pInfo->validBlockIndex++;
SPackedData* pSubmit = taosArrayGet(pInfo->pBlockLists, current);
QUERY_CHECK_NULL(pSubmit, code, lino, _end, terrno);
qDebug("set %d/%d as the input submit block, %s", current + 1, totalBlocks, id);
if (pAPI->tqReaderFn.tqReaderSetSubmitMsg(pInfo->tqReader, pSubmit->msgStr, pSubmit->msgLen, pSubmit->ver,
NULL) < 0) {
qError("submit msg messed up when initializing stream submit block %p, current %d/%d, %s", pSubmit, current,
totalBlocks, id);
continue;
}
}
blockDataCleanup(pInfo->pRes);
while (pAPI->tqReaderFn.tqNextBlockImpl(pInfo->tqReader, id)) {
SSDataBlock* pRes = NULL;
code = pAPI->tqReaderFn.tqRetrieveBlock(pInfo->tqReader, &pRes, id);
qDebug("retrieve data from submit completed code:%s rows:%" PRId64 " %s", tstrerror(code), pRes->info.rows, id);
if (code != TSDB_CODE_SUCCESS || pRes->info.rows == 0) {
qDebug("retrieve data failed, try next block in submit block, %s", id);
continue;
}
code = setBlockIntoRes(pInfo, pRes, &pStreamInfo->fillHistoryWindow, false);
if (code == TSDB_CODE_PAR_TABLE_NOT_EXIST) {
pInfo->pRes->info.rows = 0;
code = TSDB_CODE_SUCCESS;
} else {
QUERY_CHECK_CODE(code, lino, _end);
}
if (pInfo->pRes->info.rows == 0) {
continue;
}
if (pInfo->pCreateTbRes->info.rows > 0) {
pInfo->scanMode = STREAM_SCAN_FROM_RES;
qDebug("create table res exists, rows:%" PRId64 " return from stream scan, %s",
pInfo->pCreateTbRes->info.rows, id);
(*ppRes) = pInfo->pCreateTbRes;
return code;
}
code = doCheckUpdate(pInfo, pBlockInfo->window.ekey, pInfo->pRes);
QUERY_CHECK_CODE(code, lino, _end);
setStreamOperatorState(&pInfo->basic, pInfo->pRes->info.type);
code = doFilter(pInfo->pRes, pOperator->exprSupp.pFilterInfo, NULL);
QUERY_CHECK_CODE(code, lino, _end);
code = blockDataUpdateTsWindow(pInfo->pRes, pInfo->primaryTsIndex);
QUERY_CHECK_CODE(code, lino, _end);
int64_t numOfUpdateRes = pInfo->pUpdateDataRes->info.rows;
qDebug("%s %" PRId64 " rows in datablock, update res:%" PRId64, id, pBlockInfo->rows, numOfUpdateRes);
if (pBlockInfo->rows > 0 || numOfUpdateRes > 0) {
break;
}
}
if (pBlockInfo->rows > 0 || pInfo->pUpdateDataRes->info.rows > 0) {
break;
} else {
continue;
}
}
// record the scan action.
pInfo->numOfExec++;
pOperator->resultInfo.totalRows += pBlockInfo->rows;
qDebug("stream scan completed, and return source rows:%" PRId64 ", %s", pBlockInfo->rows, id);
if (pBlockInfo->rows > 0) {
printDataBlock(pInfo->pRes, getStreamOpName(pOperator->operatorType), GET_TASKID(pTaskInfo));
(*ppRes) = pInfo->pRes;
return code;
}
if (pInfo->pUpdateDataRes->info.rows > 0) {
goto FETCH_NEXT_BLOCK;
}
goto NEXT_SUBMIT_BLK;
} else if (pInfo->blockType == STREAM_INPUT__CHECKPOINT) {
if (pInfo->validBlockIndex >= total) {
doClearBufferedBlocks(pInfo);
(*ppRes) = NULL;
return code;
}
int32_t current = pInfo->validBlockIndex++;
qDebug("process %d/%d input data blocks, %s", current, (int32_t)total, id);
SPackedData* pData = taosArrayGet(pInfo->pBlockLists, current);
QUERY_CHECK_NULL(pData, code, lino, _end, terrno);
SSDataBlock* pBlock = taosArrayGet(pData->pDataBlock, 0);
QUERY_CHECK_NULL(pBlock, code, lino, _end, terrno);
if (pBlock->info.type == STREAM_CHECKPOINT) {
// todo(kjq): serialize checkpoint
}
// printDataBlock(pInfo->pCheckpointRes, "stream scan ck", GET_TASKID(pTaskInfo));
(*ppRes) = pInfo->pCheckpointRes;
return code;
} else {
qError("stream scan error, invalid block type %d, %s", pInfo->blockType, id);
code = TSDB_CODE_QRY_EXECUTOR_INTERNAL_ERROR;
}
_end:
if (code != TSDB_CODE_SUCCESS) {
qError("%s failed at line %d since %s", __func__, lino, tstrerror(code));
pTaskInfo->code = code;
T_LONG_JMP(pTaskInfo->env, code);
}
(*ppRes) = NULL;
return code;
}
int32_t createStreamVtableMergeOperatorInfo(SReadHandle* pHandle, SVirtualScanPhysiNode* pVirtualScanNode,
SNode* pTagCond, SExecTaskInfo* pTaskInfo, SOperatorInfo** pOptrInfo) {
QRY_PARAM_CHECK(pOptrInfo);
int32_t code = TSDB_CODE_SUCCESS;
int32_t lino = 0;
SStreamScanInfo* pInfo = taosMemoryCalloc(1, sizeof(SStreamScanInfo));
SOperatorInfo* pOperator = taosMemoryCalloc(1, sizeof(SOperatorInfo));
SStorageAPI* pAPI = &pTaskInfo->storageAPI;
if (pInfo == NULL || pOperator == NULL) {
code = terrno;
goto _error;
}
SScanPhysiNode* pScanPhyNode = &pVirtualScanNode->scan;
SDataBlockDescNode* pDescNode = pScanPhyNode->node.pOutputDataBlockDesc;
pInfo->pTagCond = pTagCond;
pInfo->pGroupTags = pVirtualScanNode->pGroupTags;
int32_t numOfCols = 0;
code = extractColMatchInfo(pScanPhyNode->pScanCols, pDescNode, &numOfCols, COL_MATCH_FROM_COL_ID, &pInfo->matchInfo);
if (code != TSDB_CODE_SUCCESS) {
goto _error;
}
SDataType pkType = {0};
pInfo->primaryKeyIndex = -1;
pInfo->basic.primaryPkIndex = -1;
int32_t numOfOutput = taosArrayGetSize(pInfo->matchInfo.pList);
for (int32_t i = 0; i < numOfOutput; ++i) {
SColMatchItem* id = taosArrayGet(pInfo->matchInfo.pList, i);
QUERY_CHECK_NULL(id, code, lino, _error, terrno);
if (id->colId == PRIMARYKEY_TIMESTAMP_COL_ID) {
pInfo->primaryTsIndex = id->dstSlotId;
}
if (id->isPk) {
pInfo->primaryKeyIndex = id->dstSlotId;
pInfo->basic.primaryPkIndex = id->dstSlotId;
pkType = id->dataType;
}
}
pInfo->pPartTbnameSup = NULL;
if (pVirtualScanNode->pSubtable != NULL) {
SExprInfo* pSubTableExpr = taosMemoryCalloc(1, sizeof(SExprInfo));
if (pSubTableExpr == NULL) {
code = terrno;
goto _error;
}
pInfo->tbnameCalSup.pExprInfo = pSubTableExpr;
code = createExprFromOneNode(pSubTableExpr, pVirtualScanNode->pSubtable, 0);
QUERY_CHECK_CODE(code, lino, _error);
if (initExprSupp(&pInfo->tbnameCalSup, pSubTableExpr, 1, &pTaskInfo->storageAPI.functionStore) != 0) {
goto _error;
}
}
if (pVirtualScanNode->pTags != NULL) {
int32_t numOfTags;
SExprInfo* pTagExpr = createExpr(pVirtualScanNode->pTags, &numOfTags);
if (pTagExpr == NULL) {
goto _error;
}
code = initExprSupp(&pInfo->tagCalSup, pTagExpr, numOfTags, &pTaskInfo->storageAPI.functionStore);
if (code != 0) {
goto _error;
}
}
// todo(kjq): add partition table name generation
// todo(kjq): add tag column generation
pInfo->pBlockLists = taosArrayInit(4, sizeof(SPackedData));
QUERY_CHECK_NULL(pInfo->pBlockLists, code, lino, _error, terrno);
// TODO(kjq): support virtual table re-scan from tsdb
if (pTaskInfo->streamInfo.pState) {
pAPI->stateStore.streamStateSetNumber(pTaskInfo->streamInfo.pState, -1, pInfo->primaryTsIndex);
}
pInfo->readHandle = *pHandle;
pTaskInfo->streamInfo.snapshotVer = pHandle->version;
pInfo->pCreateTbRes = buildCreateTableBlock(&pInfo->tbnameCalSup, &pInfo->tagCalSup);
QUERY_CHECK_NULL(pInfo->pCreateTbRes, code, lino, _error, terrno);
// create the pseduo columns info
if (pVirtualScanNode->scan.pScanPseudoCols != NULL) {
code = createExprInfo(pVirtualScanNode->scan.pScanPseudoCols, NULL, &pInfo->pPseudoExpr, &pInfo->numOfPseudoExpr);
QUERY_CHECK_CODE(code, lino, _error);
}
code = filterInitFromNode((SNode*)pScanPhyNode->node.pConditions, &pOperator->exprSupp.pFilterInfo, 0);
QUERY_CHECK_CODE(code, lino, _error);
pInfo->pRes = createDataBlockFromDescNode(pDescNode);
QUERY_CHECK_NULL(pInfo->pRes, code, lino, _error, terrno);
code = createSpecialDataBlock(STREAM_CLEAR, &pInfo->pUpdateRes);
QUERY_CHECK_CODE(code, lino, _error);
pInfo->scanMode = STREAM_SCAN_FROM_READERHANDLE;
pInfo->windowSup = (SWindowSupporter){.pStreamAggSup = NULL, .gap = -1, .parentType = QUERY_NODE_PHYSICAL_PLAN};
pInfo->groupId = 0;
pInfo->igCheckGroupId = false;
pInfo->pStreamScanOp = pOperator;
pInfo->deleteDataIndex = 0;
code = createSpecialDataBlock(STREAM_DELETE_DATA, &pInfo->pDeleteDataRes);
QUERY_CHECK_CODE(code, lino, _error);
pInfo->updateWin = (STimeWindow){.skey = INT64_MAX, .ekey = INT64_MAX};
code = createSpecialDataBlock(STREAM_CLEAR, &pInfo->pUpdateDataRes);
QUERY_CHECK_CODE(code, lino, _error);
if (pInfo->primaryKeyIndex != -1) {
pInfo->pUpdateDataRes->info.rowSize += pkType.bytes;
SColumnInfoData infoData = {0};
infoData.info.type = pkType.type;
infoData.info.bytes = pkType.bytes;
void* px = taosArrayPush(pInfo->pUpdateDataRes->pDataBlock, &infoData);
QUERY_CHECK_NULL(px, code, lino, _error, terrno);
pInfo->pkColType = pkType.type;
pInfo->pkColLen = pkType.bytes;
}
pInfo->partitionSup.needCalc = false;
pInfo->igCheckUpdate = pVirtualScanNode->igCheckUpdate;
pInfo->igExpired = pVirtualScanNode->igExpired;
pInfo->twAggSup.maxTs = INT64_MIN;
pInfo->pState = pTaskInfo->streamInfo.pState;
pInfo->stateStore = pTaskInfo->storageAPI.stateStore;
pInfo->readerFn = pTaskInfo->storageAPI.tqReaderFn;
pInfo->pFillSup = NULL;
pInfo->useGetResultRange = false;
pInfo->pRangeScanRes = NULL;
code = createSpecialDataBlock(STREAM_CHECKPOINT, &pInfo->pCheckpointRes);
QUERY_CHECK_CODE(code, lino, _error);
pInfo->lastScanRange.skey = INT64_MIN;
pInfo->lastScanRange.ekey = INT64_MIN;
// for stream
if (pTaskInfo->streamInfo.pState) {
void* buff = NULL;
int32_t len = 0;
int32_t res =
pAPI->stateStore.streamStateGetInfo(pTaskInfo->streamInfo.pState, STREAM_VTABLE_MERGE_OP_CHECKPOINT_NAME,
strlen(STREAM_VTABLE_MERGE_OP_CHECKPOINT_NAME), &buff, &len);
if (res == TSDB_CODE_SUCCESS) {
// todo(kjq): deserialize checkpoint here
taosMemoryFree(buff);
}
}
setOperatorInfo(pOperator, STREAM_VTABLE_MERGE_OP_NAME, QUERY_NODE_PHYSICAL_PLAN_VIRTUAL_TABLE_SCAN, false,
OP_NOT_OPENED, pInfo, pTaskInfo);
pOperator->exprSupp.numOfExprs = taosArrayGetSize(pInfo->pRes->pDataBlock);
pOperator->fpSet = createOperatorFpSet(optrDummyOpenFn, doStreamVtableMergeNext, NULL, destroyStreamScanOperatorInfo,
optrDefaultBufFn, NULL, optrDefaultGetNextExtFn, NULL);
// TODO(kjq): save and load fill history state
*pOptrInfo = pOperator;
return code;
_error:
if (pInfo != NULL) {
destroyStreamScanOperatorInfo(pInfo);
}
if (pOperator != NULL) {
pOperator->info = NULL;
destroyOperator(pOperator);
}
pTaskInfo->code = code;
return code;
}

View File

@ -639,10 +639,60 @@ static int32_t jsonToSchema(const SJson* pJson, void* pObj) {
return code;
}
static const char* jkRefColHasRef = "HasRef";
static const char* jkRefColColId = "ColId";
static const char* jkRefColDbName = "DbName";
static const char* jkRefColTableName = "TableName";
static const char* jkRefColColName = "ColName";
static int32_t refColToJson(const void* pObj, SJson* pJson) {
const SColRef* pCol = (const SColRef*)pObj;
int32_t code = tjsonAddBoolToObject(pJson, jkRefColHasRef, pCol->hasRef);
if (TSDB_CODE_SUCCESS == code) {
code = tjsonAddIntegerToObject(pJson, jkRefColColId, pCol->id);
}
if (TSDB_CODE_SUCCESS == code) {
code = tjsonAddStringToObject(pJson, jkRefColDbName, pCol->refDbName);
}
if (TSDB_CODE_SUCCESS == code) {
code = tjsonAddStringToObject(pJson, jkRefColTableName, pCol->refTableName);
}
if (TSDB_CODE_SUCCESS == code) {
code = tjsonAddStringToObject(pJson, jkRefColColName, pCol->refColName);
}
return code;
}
static int32_t jsonToRefCol(const SJson* pJson, void* pObj) {
SColRef* pCol = (SColRef*)pObj;
int32_t code = tjsonGetBoolValue(pJson, jkRefColHasRef, &pCol->hasRef);
if (TSDB_CODE_SUCCESS == code) {
code = tjsonGetSmallIntValue(pJson, jkRefColColId, &pCol->id);
}
if (TSDB_CODE_SUCCESS == code) {
code = tjsonGetStringValue(pJson, jkRefColDbName, pCol->refDbName);
}
if (TSDB_CODE_SUCCESS == code) {
code = tjsonGetStringValue(pJson, jkRefColTableName, pCol->refTableName);
}
if (TSDB_CODE_SUCCESS == code) {
code = tjsonGetStringValue(pJson, jkRefColColName, pCol->refColName);
}
return code;
}
static const char* jkTableMetaVgId = "VgId";
static const char* jkTableMetaTableType = "TableType";
static const char* jkTableMetaUid = "Uid";
static const char* jkTableMetaSuid = "Suid";
static const char* jkTableMetaColRefNum = "ColRefNum";
static const char* jkTableMetaRefCols = "RefCols";
static const char* jkTableMetaSversion = "Sversion";
static const char* jkTableMetaTversion = "Tversion";
static const char* jkTableMetaComInfo = "ComInfo";
@ -661,6 +711,13 @@ static int32_t tableMetaToJson(const void* pObj, SJson* pJson) {
if (TSDB_CODE_SUCCESS == code) {
code = tjsonAddIntegerToObject(pJson, jkTableMetaSuid, pNode->suid);
}
if (TSDB_CODE_SUCCESS == code) {
code = tjsonAddIntegerToObject(pJson, jkTableMetaColRefNum, pNode->colRef ? pNode->numOfColRefs : 0);
}
if (TSDB_CODE_SUCCESS == code && pNode->numOfColRefs > 0 && pNode->colRef) {
code = tjsonAddArray(pJson, jkTableMetaRefCols, refColToJson, pNode->colRef, sizeof(SColRef),
pNode->numOfColRefs);
}
if (TSDB_CODE_SUCCESS == code) {
code = tjsonAddIntegerToObject(pJson, jkTableMetaSversion, pNode->sversion);
}
@ -692,6 +749,9 @@ static int32_t jsonToTableMeta(const SJson* pJson, void* pObj) {
if (TSDB_CODE_SUCCESS == code) {
tjsonGetNumberValue(pJson, jkTableMetaSuid, pNode->suid, code);
}
if (TSDB_CODE_SUCCESS == code) {
tjsonGetNumberValue(pJson, jkTableMetaColRefNum, pNode->numOfColRefs, code);
}
if (TSDB_CODE_SUCCESS == code) {
tjsonGetNumberValue(pJson, jkTableMetaSversion, pNode->sversion, code);
}
@ -704,6 +764,10 @@ static int32_t jsonToTableMeta(const SJson* pJson, void* pObj) {
if (TSDB_CODE_SUCCESS == code) {
code = tjsonToArray(pJson, jkTableMetaColSchemas, jsonToSchema, pNode->schema, sizeof(SSchema));
}
if (TSDB_CODE_SUCCESS == code && pNode->numOfColRefs > 0) {
pNode->colRef = (SColRef*)((char*)(pNode + 1) + TABLE_TOTAL_COL_NUM(pNode) * sizeof(SSchema));
code = tjsonToArray(pJson, jkTableMetaRefCols, jsonToRefCol, pNode->colRef, sizeof(SColRef));
}
return code;
}
@ -4059,6 +4123,110 @@ static int32_t jsonToQueryNodeAddr(const SJson* pJson, void* pObj) {
return code;
}
static const char* jkColOtableName = "colOtableName";
static const char* jkColVtableId = "colVtableId";
static int32_t colIdNameToJson(const void* pObj, SJson* pJson) {
const SColIdName* pCol = (const SColIdName*)pObj;
int32_t code = tjsonAddIntegerToObject(pJson, jkColVtableId, pCol->colId);
if (TSDB_CODE_SUCCESS == code) {
code = tjsonAddStringToObject(pJson, jkColOtableName, pCol->colName);
}
return code;
}
static const char* jkOtableHashSize = "otbHashSize";
static const char* jkOtableHashKV = "otbHashKeyValue";
static const char* jkOtableHashName = "otbHashName";
static const char* jkOtableHashValue = "otbHashValue";
static int32_t oTableHashToJson(const void* pObj, SJson* pJson) {
const SSHashObj* pHash = (const SSHashObj*)pObj;
int32_t code = tjsonAddIntegerToObject(pJson, jkOtableHashSize, tSimpleHashGetSize(pHash));
if (code) {
return code;
}
SJson* pJsonArray = tjsonAddArrayToObject(pJson, jkOtableHashKV);
if (NULL == pJsonArray) {
return terrno;
}
int32_t iter = 0;
SArray** pCols = NULL;
char* pKey = NULL;
void* p = NULL;
while (NULL != (p = tSimpleHashIterate(pHash, p, &iter))) {
pKey = (char*)tSimpleHashGetKey(p, NULL);
pCols = (SArray**)p;
SJson* pJobj = tjsonCreateObject();
if (pJobj == NULL) {
return terrno;
}
code = tjsonAddStringToObject(pJobj, jkOtableHashName, pKey);
if (code) {
return code;
}
code = tjsonAddArray(pJobj, jkOtableHashValue, colIdNameToJson, TARRAY_GET_ELEM(*pCols, 0), sizeof(SColIdName), taosArrayGetSize(*pCols));
if (code) {
return code;
}
code = tjsonAddItemToArray(pJsonArray, pJobj);
if (code) {
return code;
}
}
return code;
}
static const char* jkVtablesHashSize = "vtbHashSize";
static const char* jkVtablesHashKV = "vtbHashKeyValue";
static const char* jkVtablesVuid = "vtbHashVuid";
static const char* jkVtablesVValue = "vtbHashVValue";
static int32_t vtablesHashToJson(const void* pObj, SJson* pJson) {
const SSHashObj* pHash = (const SSHashObj*)pObj;
int32_t code = tjsonAddIntegerToObject(pJson, jkVtablesHashSize, tSimpleHashGetSize(pHash));
if (code) {
return code;
}
SJson* pJsonArray = tjsonAddArrayToObject(pJson, jkVtablesHashKV);
if (NULL == pJsonArray) {
return terrno;
}
int32_t iter = 0;
SSHashObj** pOtable = NULL;
char* pKey = NULL;
void* p = NULL;
while (NULL != (p = tSimpleHashIterate(pHash, p, &iter))) {
pKey = (char*)tSimpleHashGetKey(p, NULL);
pOtable = (SSHashObj**)p;
SJson* pJobj = tjsonCreateObject();
if (pJobj == NULL) {
return terrno;
}
code = tjsonAddIntegerToObject(pJobj, jkVtablesVuid, *(uint64_t*)((int32_t*)pKey + 1));
if (code) {
return code;
}
code = tjsonAddObject(pJobj, jkVtablesVValue, oTableHashToJson, *pOtable);
if (code) {
return code;
}
code = tjsonAddItemToArray(pJsonArray, pJobj);
if (code) {
return code;
}
}
return code;
}
static const char* jkSubplanId = "Id";
static const char* jkSubplanType = "SubplanType";
static const char* jkSubplanMsgType = "MsgType";
@ -4070,12 +4238,14 @@ static const char* jkSubplanRootNode = "RootNode";
static const char* jkSubplanDataSink = "DataSink";
static const char* jkSubplanTagCond = "TagCond";
static const char* jkSubplanTagIndexCond = "TagIndexCond";
static const char* jkSubplanVTables = "VTables";
static const char* jkSubplanShowRewrite = "ShowRewrite";
static const char* jkSubplanRowsThreshold = "RowThreshold";
static const char* jkSubplanDynamicRowsThreshold = "DyRowThreshold";
static const char* jkSubplanIsView = "IsView";
static const char* jkSubplanIsAudit = "IsAudit";
static int32_t subplanToJson(const void* pObj, SJson* pJson) {
const SSubplan* pNode = (const SSubplan*)pObj;
@ -4110,6 +4280,9 @@ static int32_t subplanToJson(const void* pObj, SJson* pJson) {
if (TSDB_CODE_SUCCESS == code) {
code = tjsonAddObject(pJson, jkSubplanTagIndexCond, nodeToJson, pNode->pTagIndexCond);
}
if (TSDB_CODE_SUCCESS == code) {
code = tjsonAddObject(pJson, jkSubplanVTables, vtablesHashToJson, pNode->pVTables);
}
if (TSDB_CODE_SUCCESS == code) {
code = tjsonAddBoolToObject(pJson, jkSubplanShowRewrite, pNode->showRewrite);
}
@ -4129,6 +4302,100 @@ static int32_t subplanToJson(const void* pObj, SJson* pJson) {
return code;
}
static int32_t jsonToOtableCols(const SJson* pJson, void* pObj) {
SArray** pCols = (SArray**)pObj;
SColIdName col;
char colName[TSDB_COL_NAME_LEN];
int32_t code = 0;
int32_t colNum = tjsonGetArraySize(pJson);
*pCols = taosArrayInit(colNum, sizeof(SColIdName));
if (NULL == *pCols) {
return terrno;
}
for (int32_t i = 0; i < colNum; ++i) {
SJson* pCol = tjsonGetArrayItem(pJson, i);
code = tjsonGetSmallIntValue(pCol, jkColVtableId, &col.colId);
if (code < 0) return TSDB_CODE_INVALID_JSON_FORMAT;
code = tjsonGetStringValue(pCol, jkColOtableName, colName);
if (code < 0) return TSDB_CODE_INVALID_JSON_FORMAT;
col.colName = taosStrdup(colName);
if (NULL == col.colName) {
return terrno;
}
if (NULL == taosArrayPush(*pCols, &col)) {
return terrno;
}
}
return code;
}
static int32_t jsonToOtableHash(const SJson* pJson, void* pObj) {
SSHashObj** pHash = (SSHashObj**)pObj;
int32_t hashSize = 0;
int32_t code = tjsonGetIntValue(pJson, jkOtableHashSize, &hashSize);
if (TSDB_CODE_SUCCESS == code && hashSize > 0) {
*pHash = tSimpleHashInit(hashSize, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BINARY));
if (NULL == *pHash) {
return terrno;
}
tSimpleHashSetFreeFp(*pHash, tFreeStreamVtbOtbInfo);
SJson *ovalues = tjsonGetObjectItem(pJson, jkOtableHashKV);
if (ovalues == NULL) return TSDB_CODE_INVALID_JSON_FORMAT;
char tbName[TSDB_TABLE_NAME_LEN];
SArray* pCols = NULL;
for (int32_t d = 0; d < hashSize; ++d) {
SJson *okeyValue = tjsonGetArrayItem(ovalues, d);
if (okeyValue == NULL) return TSDB_CODE_INVALID_JSON_FORMAT;
code = tjsonGetStringValue(okeyValue, jkOtableHashName, tbName);
if (code < 0) return TSDB_CODE_INVALID_JSON_FORMAT;
SJson *ovalue = tjsonGetObjectItem(okeyValue, jkOtableHashValue);
code = jsonToOtableCols(ovalue, &pCols);
if (code < 0) return code;
code = tSimpleHashPut(*pHash, tbName, strlen(tbName) + 1, &pCols, POINTER_BYTES);
if (code < 0) return code;
}
}
return code;
}
static int32_t jsonToVtablesHash(const SJson* pJson, void* pObj) {
SSHashObj** pHash = (SSHashObj**)pObj;
int32_t hashSize = 0;
int32_t code = tjsonGetIntValue(pJson, jkVtablesHashSize, &hashSize);
if (TSDB_CODE_SUCCESS == code && hashSize > 0) {
*pHash = tSimpleHashInit(hashSize, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BIGINT));
if (NULL == *pHash) {
return terrno;
}
tSimpleHashSetFreeFp(*pHash, tFreeStreamVtbVtbInfo);
SJson *vvalues = tjsonGetObjectItem(pJson, jkVtablesHashKV);
if (vvalues == NULL) return TSDB_CODE_INVALID_JSON_FORMAT;
uint64_t vuid = 0;
SSHashObj* pOtable = NULL;
for (int32_t d = 0; d < hashSize; ++d) {
SJson *vkeyValue = tjsonGetArrayItem(vvalues, d);
if (vkeyValue == NULL) return TSDB_CODE_INVALID_JSON_FORMAT;
code = tjsonGetUBigIntValue(vkeyValue, jkVtablesVuid, &vuid);
if (code < 0) return TSDB_CODE_INVALID_JSON_FORMAT;
SJson *vvalue = tjsonGetObjectItem(vkeyValue, jkVtablesVValue);
code = jsonToOtableHash(vvalue, &pOtable);
if (code < 0) return code;
code = tSimpleHashPut(*pHash, &vuid, sizeof(vuid), &pOtable, POINTER_BYTES);
if (code < 0) return code;
}
}
return code;
}
static int32_t jsonToSubplan(const SJson* pJson, void* pObj) {
SSubplan* pNode = (SSubplan*)pObj;
@ -4163,6 +4430,9 @@ static int32_t jsonToSubplan(const SJson* pJson, void* pObj) {
if (TSDB_CODE_SUCCESS == code) {
code = jsonToNodeObject(pJson, jkSubplanTagIndexCond, (SNode**)&pNode->pTagIndexCond);
}
if (TSDB_CODE_SUCCESS == code) {
code = tjsonToObject(pJson, jkSubplanVTables, jsonToVtablesHash, &pNode->pVTables);
}
if (TSDB_CODE_SUCCESS == code) {
code = tjsonGetBoolValue(pJson, jkSubplanShowRewrite, &pNode->showRewrite);
}

View File

@ -2143,6 +2143,7 @@ void nodesDestroyNode(SNode* pNode) {
nodesDestroyNode((SNode*)pSubplan->pDataSink);
nodesDestroyNode((SNode*)pSubplan->pTagCond);
nodesDestroyNode((SNode*)pSubplan->pTagIndexCond);
tSimpleHashCleanup(pSubplan->pVTables);
nodesClearList(pSubplan->pParents);
break;
}
@ -3422,8 +3423,6 @@ bool nodesContainsColumn(SNode* pNode) {
return cxt.containsCol;
}
int32_t mergeNodeToLogic(SNode** pDst, SNode** pSrc) {
SLogicConditionNode* pLogicCond = NULL;
int32_t code = nodesMakeNode(QUERY_NODE_LOGIC_CONDITION, (SNode**)&pLogicCond);
@ -3467,5 +3466,43 @@ int32_t nodesMergeNode(SNode** pCond, SNode** pAdditionalCond) {
return code;
}
void tFreeStreamVtbColName(void* param) {
if (NULL == param) {
return;
}
SColIdName* pColId = (SColIdName*)param;
taosMemoryFreeClear(pColId->colName);
}
void tFreeStreamVtbOtbInfo(void* param) {
SArray** ppArray = (SArray**)param;
if (NULL == param || NULL == *ppArray) {
return;
}
taosArrayDestroyEx(*ppArray, tFreeStreamVtbColName);
}
void tFreeStreamVtbVtbInfo(void* param) {
SSHashObj** ppHash = (SSHashObj**)param;
if (NULL == param || NULL == *ppHash) {
return;
}
tSimpleHashCleanup(*ppHash);
}
void tFreeStreamVtbDbVgInfo(void* param) {
SSHashObj** ppHash = (SSHashObj**)param;
if (NULL == param || NULL == *ppHash) {
return;
}
tSimpleHashCleanup(*ppHash);
}

View File

@ -113,6 +113,7 @@ typedef struct SParseMetaCache {
SHashObj* pTableTSMAs; // key is tbFName, elements are SArray<STableTSMAInfo*>
SHashObj* pTSMAs; // key is tsmaFName, elements are STableTSMAInfo*
SHashObj* pTableName; // key is tbFUid, elements is STableMeta*(append with tbName)
SArray* pVSubTables; // element is SVSubTablesRsp
SArray* pDnodes; // element is SEpSet
bool dnodeRequired;
bool forceFetchViewMeta;
@ -140,8 +141,8 @@ int32_t parseTagValue(SMsgBuf* pMsgBuf, const char** pSql, uint8_t precision, SS
SArray* pTagName, SArray* pTagVals, STag** pTag, timezone_t tz, void *charsetCxt);
int32_t parseTbnameToken(SMsgBuf* pMsgBuf, char* tname, SToken* pToken, bool* pFoundCtbName);
int32_t buildCatalogReq(const SParseMetaCache* pMetaCache, SCatalogReq* pCatalogReq);
int32_t putMetaDataToCache(const SCatalogReq* pCatalogReq, const SMetaData* pMetaData, SParseMetaCache* pMetaCache);
int32_t buildCatalogReq(SParseMetaCache* pMetaCache, SCatalogReq* pCatalogReq);
int32_t putMetaDataToCache(const SCatalogReq* pCatalogReq, SMetaData* pMetaData, SParseMetaCache* pMetaCache);
int32_t reserveTableMetaInCache(int32_t acctId, const char* pDb, const char* pTable, SParseMetaCache* pMetaCache);
int32_t reserveTableMetaInCacheExt(const SName* pName, SParseMetaCache* pMetaCache);
int32_t reserveTableUidInCache(int32_t acctId, const char* pDb, const char* pTable, SParseMetaCache* pMetaCache);
@ -162,6 +163,7 @@ int32_t reserveTableCfgInCache(int32_t acctId, const char* pDb, const char* pTab
int32_t reserveDnodeRequiredInCache(SParseMetaCache* pMetaCache);
int32_t reserveTableTSMAInfoInCache(int32_t acctId, const char* pDb, const char* pTable, SParseMetaCache* pMetaCache);
int32_t reserveTSMAInfoInCache(int32_t acctId, const char* pDb, const char* pTsmaName, SParseMetaCache* pMetaCache);
int32_t reserveVSubTableInCache(int32_t acctId, const char* pDb, const char* pTable, SParseMetaCache* pMetaCache);
int32_t getTableMetaFromCache(SParseMetaCache* pMetaCache, const SName* pName, STableMeta** pMeta);
int32_t getTableNameFromCache(SParseMetaCache* pMetaCache, const SName* pName, char* pTbName);
int32_t getViewMetaFromCache(SParseMetaCache* pMetaCache, const SName* pName, STableMeta** pMeta);

View File

@ -103,6 +103,7 @@ typedef struct SCollectMetaKeyCxt {
SParseContext* pParseCxt;
SParseMetaCache* pMetaCache;
SNode* pStmt;
bool collectVSubTables;
} SCollectMetaKeyCxt;
typedef struct SCollectMetaKeyFromExprCxt {
@ -199,6 +200,9 @@ static int32_t collectMetaKeyFromRealTableImpl(SCollectMetaKeyCxt* pCxt, const c
static EDealRes collectMetaKeyFromRealTable(SCollectMetaKeyFromExprCxt* pCxt, SRealTableNode* pRealTable) {
pCxt->errCode = collectMetaKeyFromRealTableImpl(pCxt->pComCxt, pRealTable->table.dbName, pRealTable->table.tableName,
AUTH_TYPE_READ);
if (TSDB_CODE_SUCCESS == pCxt->errCode && pCxt->pComCxt->collectVSubTables) {
pCxt->errCode = reserveVSubTableInCache(pCxt->pComCxt->pParseCxt->acctId, pRealTable->table.dbName, pRealTable->table.tableName, pCxt->pComCxt->pMetaCache);
}
return TSDB_CODE_SUCCESS == pCxt->errCode ? DEAL_RES_CONTINUE : DEAL_RES_ERROR;
}
@ -606,6 +610,8 @@ static int32_t collectMetaKeyFromDescribe(SCollectMetaKeyCxt* pCxt, SDescribeStm
}
static int32_t collectMetaKeyFromCreateStream(SCollectMetaKeyCxt* pCxt, SCreateStreamStmt* pStmt) {
pCxt->collectVSubTables = true;
int32_t code =
reserveTableMetaInCache(pCxt->pParseCxt->acctId, pStmt->targetDbName, pStmt->targetTabName, pCxt->pMetaCache);
if (TSDB_CODE_SUCCESS == code && NULL != pStmt->pSubtable && NULL != pStmt->pQuery) {

View File

@ -5052,7 +5052,7 @@ static int32_t translateVirtualSuperTable(STranslateContext* pCxt, SNode** pTabl
STableMeta* pMeta = pRealTable->pMeta;
int32_t code = TSDB_CODE_SUCCESS;
if (!pMeta->virtualStb) {
if (!pMeta->virtualStb && !pCxt->createStream) {
PAR_ERR_JRET(TSDB_CODE_PAR_INVALID_TABLE_TYPE);
}
@ -5137,7 +5137,11 @@ static int32_t translateVirtualTable(STranslateContext* pCxt, SNode** pTable, SN
break;
case TSDB_VIRTUAL_CHILD_TABLE:
case TSDB_VIRTUAL_NORMAL_TABLE:
PAR_ERR_JRET(translateVirtualNormalChildTable(pCxt, pTable, pName, pVTable, pRTNode));
if (pCxt->createStream) {
PAR_ERR_JRET(translateVirtualSuperTable(pCxt, pTable, pName, pVTable, pRTNode));
} else {
PAR_ERR_JRET(translateVirtualNormalChildTable(pCxt, pTable, pName, pVTable, pRTNode));
}
break;
default:
PAR_ERR_JRET(TSDB_CODE_PAR_INVALID_TABLE_TYPE);
@ -12005,6 +12009,16 @@ static bool crossTableWithUdaf(SSelectStmt* pSelect) {
!hasTbnameFunction(pSelect->pPartitionByList);
}
static bool isVirtualTable(int8_t tableType) {
if (tableType == TSDB_VIRTUAL_CHILD_TABLE || tableType == TSDB_VIRTUAL_NORMAL_TABLE) {
return true;
} else {
return false;
}
}
static int32_t checkCreateStream(STranslateContext* pCxt, SCreateStreamStmt* pStmt) {
if (NULL == pStmt->pQuery) {
return TSDB_CODE_SUCCESS;
@ -12015,20 +12029,38 @@ static int32_t checkCreateStream(STranslateContext* pCxt, SCreateStreamStmt* pSt
return generateSyntaxErrMsgExt(&pCxt->msgBuf, TSDB_CODE_PAR_INVALID_STREAM_QUERY, "Unsupported stream query");
}
#ifdef TD_ENTERPRISE
SRealTableNode* pRealTable = (SRealTableNode*)((SSelectStmt*)pStmt->pQuery)->pFromTable;
SName name = {0};
STableMeta* pMeta = NULL;
int8_t tableType = 0;
toName(pCxt->pParseCxt->acctId, pRealTable->table.dbName, pRealTable->table.tableName, &name);
int32_t code = getTargetMeta(pCxt, &name, &pMeta, true);
if (NULL != pMeta) {
tableType = pMeta->tableType;
taosMemoryFree(pMeta);
}
if (TSDB_CODE_SUCCESS != code) {
return generateSyntaxErrMsg(&pCxt->msgBuf, TSDB_CODE_PAR_GET_META_ERROR, tstrerror(code));
}
if (NULL != pMeta) {
tableType = pMeta->tableType;
}
if (isVirtualTable(tableType) || (tableType == TSDB_SUPER_TABLE && pMeta->virtualStb)) {
if (0 == pStmt->pOptions->ignoreExpired) {
taosMemoryFree(pMeta);
return generateSyntaxErrMsgExt(&pCxt->msgBuf, TSDB_CODE_PAR_INVALID_STREAM_QUERY, "For virtual table IGNORE EXPIRED must be 1");
}
if (0 == pStmt->pOptions->ignoreUpdate) {
taosMemoryFree(pMeta);
return generateSyntaxErrMsgExt(&pCxt->msgBuf, TSDB_CODE_PAR_INVALID_STREAM_QUERY, "For virtual table IGNORE UPDATE must be 1");
}
if (pStmt->pOptions->fillHistory) {
taosMemoryFree(pMeta);
return generateSyntaxErrMsgExt(&pCxt->msgBuf, TSDB_CODE_PAR_INVALID_STREAM_QUERY, "For virtual table FILL HISTORY must be 0");
}
}
taosMemoryFree(pMeta);
#ifdef TD_ENTERPRISE
if (TSDB_VIEW_TABLE == tableType) {
return generateSyntaxErrMsgExt(&pCxt->msgBuf, TSDB_CODE_PAR_INVALID_STREAM_QUERY, "Unsupported stream query");
}
@ -13508,7 +13540,9 @@ static int32_t buildCreateStreamReq(STranslateContext* pCxt, SCreateStreamStmt*
if (TSDB_CODE_SUCCESS == code) {
code = buildStreamNotifyOptions(pCxt, pStmt->pNotifyOptions, pReq);
}
if (TSDB_CODE_SUCCESS == code && pCxt->pMetaCache != NULL) {
TSWAP(pReq->pVSubTables, pCxt->pMetaCache->pVSubTables);
}
return code;
}
@ -17060,13 +17094,6 @@ static int32_t addDropTbReqIntoVgroup(SHashObj* pVgroupHashmap, SVgroupInfo* pVg
return code;
}
static bool isVirtualTable(int8_t tableType) {
if (tableType == TSDB_VIRTUAL_CHILD_TABLE || tableType == TSDB_VIRTUAL_NORMAL_TABLE) {
return true;
} else {
return false;
}
}
static int32_t buildDropTableVgroupHashmap(STranslateContext* pCxt, SDropTableClause* pClause, const SName* name,
int8_t* tableType, SHashObj* pVgroupHashmap) {
STableMeta* pTableMeta = NULL;

View File

@ -799,7 +799,7 @@ static int32_t buildUdfReq(SHashObj* pUdfHash, SArray** pUdf) {
return TSDB_CODE_SUCCESS;
}
int32_t buildCatalogReq(const SParseMetaCache* pMetaCache, SCatalogReq* pCatalogReq) {
int32_t buildCatalogReq(SParseMetaCache* pMetaCache, SCatalogReq* pCatalogReq) {
int32_t code = buildTableReqFromDb(pMetaCache->pTableMeta, &pCatalogReq->pTableMeta);
if (TSDB_CODE_SUCCESS == code) {
code = buildDbReq(pMetaCache->pDbVgroup, &pCatalogReq->pDbVgroup);
@ -839,6 +839,8 @@ int32_t buildCatalogReq(const SParseMetaCache* pMetaCache, SCatalogReq* pCatalog
code = buildTableReqFromDb(pMetaCache->pTableMeta, &pCatalogReq->pView);
}
#endif
TSWAP(pCatalogReq->pVSubTable, pMetaCache->pVSubTables);
pCatalogReq->dNodeRequired = pMetaCache->dnodeRequired;
pCatalogReq->forceFetchViewMeta = pMetaCache->forceFetchViewMeta;
return code;
@ -960,7 +962,7 @@ static int32_t putUdfToCache(const SArray* pUdfReq, const SArray* pUdfData, SHas
return TSDB_CODE_SUCCESS;
}
int32_t putMetaDataToCache(const SCatalogReq* pCatalogReq, const SMetaData* pMetaData, SParseMetaCache* pMetaCache) {
int32_t putMetaDataToCache(const SCatalogReq* pCatalogReq, SMetaData* pMetaData, SParseMetaCache* pMetaCache) {
int32_t code = putDbTableDataToCache(pCatalogReq->pTableMeta, pMetaData->pTableMeta, &pMetaCache->pTableMeta);
if (TSDB_CODE_SUCCESS == code) {
code = putDbDataToCache(pCatalogReq->pDbVgroup, pMetaData->pDbVgroup, &pMetaCache->pDbVgroup);
@ -1000,6 +1002,10 @@ int32_t putMetaDataToCache(const SCatalogReq* pCatalogReq, const SMetaData* pMet
code = putDbTableDataToCache(pCatalogReq->pView, pMetaData->pView, &pMetaCache->pViews);
}
#endif
pMetaCache->pVSubTables = pMetaData->pVSubTables;
pMetaData->pVSubTables = NULL;
pMetaCache->pDnodes = pMetaData->pDnodeList;
return code;
}
@ -1337,6 +1343,25 @@ int32_t reserveTSMAInfoInCache(int32_t acctId, const char* pDb, const char* pTsm
return reserveTableReqInDbCache(acctId, pDb, pTsmaName, &pMetaCache->pTSMAs);
}
int32_t reserveVSubTableInCache(int32_t acctId, const char* pDb, const char* pTable, SParseMetaCache* pMetaCache) {
SName fullName = {0};
toName(acctId, pDb, pTable, &fullName);
if (NULL == pMetaCache->pVSubTables) {
pMetaCache->pVSubTables = taosArrayInit(1, sizeof(fullName));
if (NULL == pMetaCache->pVSubTables) {
return terrno;
}
}
if (NULL == taosArrayPush(pMetaCache->pVSubTables, &fullName)) {
return terrno;
}
return TSDB_CODE_SUCCESS;
}
int32_t getTableIndexFromCache(SParseMetaCache* pMetaCache, const SName* pName, SArray** pIndexes) {
char fullName[TSDB_TABLE_FNAME_LEN];
int32_t code = tNameExtractFullName(pName, fullName);
@ -1517,6 +1542,7 @@ void destoryParseMetaCache(SParseMetaCache* pMetaCache, bool request) {
taosHashCleanup(pMetaCache->pTableIndex);
taosHashCleanup(pMetaCache->pTableCfg);
taosHashCleanup(pMetaCache->pTableTSMAs);
taosArrayDestroyEx(pMetaCache->pVSubTables, tDestroySVSubTablesRsp);
}
int64_t int64SafeSub(int64_t a, int64_t b) {

View File

@ -292,7 +292,7 @@ int32_t qParseSqlSyntax(SParseContext* pCxt, SQuery** pQuery, struct SCatalogReq
}
int32_t qAnalyseSqlSemantic(SParseContext* pCxt, const struct SCatalogReq* pCatalogReq,
const struct SMetaData* pMetaData, SQuery* pQuery) {
struct SMetaData* pMetaData, SQuery* pQuery) {
SParseMetaCache metaCache = {0};
int32_t code = nodesAcquireAllocator(pCxt->allocatorId);
if (TSDB_CODE_SUCCESS == code && pCatalogReq) {
@ -371,6 +371,7 @@ void destoryCatalogReq(SCatalogReq* pCatalogReq) {
taosArrayDestroy(pCatalogReq->pTableIndex);
taosArrayDestroy(pCatalogReq->pTableCfg);
taosArrayDestroy(pCatalogReq->pTableTag);
taosArrayDestroy(pCatalogReq->pVSubTable);
}
void tfreeSParseQueryRes(void* p) {

View File

@ -242,7 +242,7 @@ class ParserTestBaseImpl {
DO_WITH_THROW(collectMetaKey, pCxt, pQuery, pMetaCache);
}
void doBuildCatalogReq(SParseContext* pCxt, const SParseMetaCache* pMetaCache, SCatalogReq* pCatalogReq) {
void doBuildCatalogReq(SParseContext* pCxt, SParseMetaCache* pMetaCache, SCatalogReq* pCatalogReq) {
DO_WITH_THROW(buildCatalogReq, pMetaCache, pCatalogReq);
}
@ -250,7 +250,7 @@ class ParserTestBaseImpl {
DO_WITH_THROW(g_mockCatalogService->catalogGetAllMeta, pCatalogReq, pMetaData);
}
void doPutMetaDataToCache(const SCatalogReq* pCatalogReq, const SMetaData* pMetaData, SParseMetaCache* pMetaCache) {
void doPutMetaDataToCache(const SCatalogReq* pCatalogReq, SMetaData* pMetaData, SParseMetaCache* pMetaCache) {
DO_WITH_THROW(putMetaDataToCache, pCatalogReq, pMetaData, pMetaCache);
}
@ -283,7 +283,7 @@ class ParserTestBaseImpl {
}
}
void doAnalyseSqlSemantic(SParseContext* pCxt, const SCatalogReq* pCatalogReq, const SMetaData* pMetaData,
void doAnalyseSqlSemantic(SParseContext* pCxt, const SCatalogReq* pCatalogReq, SMetaData* pMetaData,
SQuery* pQuery) {
DO_WITH_THROW(qAnalyseSqlSemantic, pCxt, pCatalogReq, pMetaData, pQuery);
res_.calcConstAst_ = toString(pQuery->pRoot);

View File

@ -1013,6 +1013,11 @@ static int32_t createVirtualSuperTableLogicNode(SLogicPlanContext* pCxt, SSelect
PLAN_ERR_JRET(createColumnByRewriteExprs(pVtableScan->pScanCols, &pVtableScan->node.pTargets));
PLAN_ERR_JRET(createColumnByRewriteExprs(pVtableScan->pScanPseudoCols, &pVtableScan->node.pTargets));
if (pCxt->pPlanCxt->streamQuery) {
*pLogicNode = (SLogicNode*)pVtableScan;
return code;
}
// Dynamic query control node -> Virtual table scan node -> Real table scan node
PLAN_ERR_JRET(nodesMakeNode(QUERY_NODE_LOGIC_PLAN_DYN_QUERY_CTRL, (SNode**)&pDynCtrl));
pDynCtrl->qType = DYN_QTYPE_VTB_SCAN;
@ -1134,7 +1139,11 @@ static int32_t createVirtualTableLogicNode(SLogicPlanContext* pCxt, SSelectStmt*
break;
case TSDB_VIRTUAL_NORMAL_TABLE:
case TSDB_VIRTUAL_CHILD_TABLE:
PLAN_ERR_JRET(createVirtualNormalChildTableLogicNode(pCxt, pSelect, pVirtualTable, pVtableScan, pLogicNode));
if (pCxt->pPlanCxt->streamQuery) {
PLAN_ERR_JRET(createVirtualSuperTableLogicNode(pCxt, pSelect, pVirtualTable, pVtableScan, pLogicNode));
} else {
PLAN_ERR_JRET(createVirtualNormalChildTableLogicNode(pCxt, pSelect, pVirtualTable, pVtableScan, pLogicNode));
}
break;
default:
PLAN_ERR_JRET(TSDB_CODE_PLAN_INVALID_TABLE_TYPE);

View File

@ -2035,6 +2035,44 @@ static int32_t insertSelectSplit(SSplitContext* pCxt, SLogicSubplan* pSubplan) {
return code;
}
typedef struct SStreamVTableSplitInfo {
SLogicNode* pSplitNode;
SLogicSubplan* pSubplan;
} SStreamVTableSplitInfo;
static bool streamVTableFindSplitNode(SSplitContext* pCxt, SLogicSubplan* pSubplan, SLogicNode* pNode,
SStreamVTableSplitInfo* pInfo) {
if (!pCxt->pPlanCxt->streamQuery) {
return false;
}
if (1 == LIST_LENGTH(pNode->pChildren) && QUERY_NODE_LOGIC_PLAN_VIRTUAL_TABLE_SCAN == nodeType(nodesListGetNode(pNode->pChildren, 0))) {
pInfo->pSplitNode = (SLogicNode*)nodesListGetNode(pNode->pChildren, 0);
pInfo->pSubplan = pSubplan;
return true;
}
return false;
}
static int32_t streamVTableSplit(SSplitContext* pCxt, SLogicSubplan* pSubplan) {
int32_t code = TSDB_CODE_SUCCESS;
SStreamVTableSplitInfo info = {0};
if (!splMatch(pCxt, pSubplan, 0, (FSplFindSplitNode)streamVTableFindSplitNode, &info)) {
return TSDB_CODE_SUCCESS;
}
PLAN_ERR_JRET(splCreateExchangeNodeForSubplan(pCxt, info.pSubplan, info.pSplitNode, info.pSubplan->subplanType, false));
SLogicSubplan *sub = splCreateScanSubplan(pCxt, info.pSplitNode, 0);
PLAN_ERR_JRET(nodesListMakeStrictAppend(&info.pSubplan->pChildren, (SNode*)sub));
info.pSubplan->subplanType = SUBPLAN_TYPE_MERGE;
++(pCxt->groupId);
_return:
pCxt->split = true;
return code;
}
typedef struct SVirtualTableSplitInfo {
SVirtualScanLogicNode *pVirtual;
SLogicSubplan *pSubplan;
@ -2212,6 +2250,7 @@ static const SSplitRule splitRuleSet[] = {
{.pName = "UnionDistinctSplit", .splitFunc = unionDistinctSplit},
{.pName = "SmaIndexSplit", .splitFunc = smaIndexSplit}, // not used yet
{.pName = "InsertSelectSplit", .splitFunc = insertSelectSplit},
{.pName = "StreamVtableSplit", .splitFunc = streamVTableSplit},
{.pName = "VirtualtableSplit", .splitFunc = virtualTableSplit},
{.pName = "MergeAggColsSplit", .splitFunc = mergeAggColsSplit},
{.pName = "DynVirtualScanSplit", .splitFunc = dynVirtualScanSplit},

View File

@ -752,3 +752,10 @@ void freeDbCfgInfo(SDbCfgInfo* pInfo) {
void* getTaskPoolWorkerCb() {
return taskQueue.wrokrerPool.pCb;
}
void tFreeStreamVtbOtbInfo(void* param);
void tFreeStreamVtbVtbInfo(void* param);
void tFreeStreamVtbDbVgInfo(void* param);

View File

@ -429,6 +429,31 @@ int32_t queryBuildGetStreamProgressMsg(void* input, char** msg, int32_t msgSize,
return TSDB_CODE_SUCCESS;
}
int32_t queryBuildVSubTablesMsg(void* input, char** msg, int32_t msgSize, int32_t *msgLen, void*(*mallcFp)(int64_t)) {
QUERY_PARAM_CHECK(input);
QUERY_PARAM_CHECK(msg);
QUERY_PARAM_CHECK(msgLen);
SVSubTablesReq req = {0};
req.suid = *(int64_t*)input;
int32_t bufLen = tSerializeSVSubTablesReq(NULL, 0, &req);
void *pBuf = (*mallcFp)(bufLen);
if (NULL == pBuf) {
return terrno;
}
if(tSerializeSVSubTablesReq(pBuf, bufLen, &req) < 0) {
return TSDB_CODE_TSC_INVALID_INPUT;
}
*msg = pBuf;
*msgLen = bufLen;
return TSDB_CODE_SUCCESS;
}
int32_t queryProcessUseDBRsp(void *output, char *msg, int32_t msgSize) {
SUseDbOutput *pOut = output;
SUseDbRsp usedbRsp = {0};
@ -1062,6 +1087,21 @@ int32_t queryProcessStreamProgressRsp(void* output, char* msg, int32_t msgSize)
return TSDB_CODE_SUCCESS;
}
int32_t queryProcessVSubTablesRsp(void* output, char* msg, int32_t msgSize) {
if (!output || !msg || msgSize <= 0) {
qError("queryProcessVSubTablesRsp input error, output:%p, msg:%p, msgSize:%d", output, msg, msgSize);
return TSDB_CODE_TSC_INVALID_INPUT;
}
SVSubTablesRsp* pRsp = (SVSubTablesRsp*)output;
int32_t code = tDeserializeSVSubTablesRsp(msg, msgSize, pRsp);
if (code != 0) {
qError("tDeserializeSVSubTablesRsp failed, msgSize: %d, error:%d", msgSize, code);
return code;
}
return TSDB_CODE_SUCCESS;
}
void initQueryModuleMsgHandle() {
queryBuildMsg[TMSG_INDEX(TDMT_VND_TABLE_META)] = queryBuildTableMetaReqMsg;
@ -1082,6 +1122,7 @@ void initQueryModuleMsgHandle() {
queryBuildMsg[TMSG_INDEX(TDMT_MND_GET_TABLE_TSMA)] = queryBuildGetTableTSMAMsg;
queryBuildMsg[TMSG_INDEX(TDMT_MND_GET_TSMA)] = queryBuildGetTSMAMsg;
queryBuildMsg[TMSG_INDEX(TDMT_VND_GET_STREAM_PROGRESS)] = queryBuildGetStreamProgressMsg;
queryBuildMsg[TMSG_INDEX(TDMT_VND_VSUBTABLES_META)] = queryBuildVSubTablesMsg;
queryProcessMsgRsp[TMSG_INDEX(TDMT_VND_TABLE_META)] = queryProcessTableMetaRsp;
queryProcessMsgRsp[TMSG_INDEX(TDMT_VND_TABLE_NAME)] = queryProcessTableNameRsp;
@ -1101,6 +1142,7 @@ void initQueryModuleMsgHandle() {
queryProcessMsgRsp[TMSG_INDEX(TDMT_MND_GET_TABLE_TSMA)] = queryProcessGetTbTSMARsp;
queryProcessMsgRsp[TMSG_INDEX(TDMT_MND_GET_TSMA)] = queryProcessGetTbTSMARsp;
queryProcessMsgRsp[TMSG_INDEX(TDMT_VND_GET_STREAM_PROGRESS)] = queryProcessStreamProgressRsp;
queryProcessMsgRsp[TMSG_INDEX(TDMT_VND_VSUBTABLES_META)] = queryProcessVSubTablesRsp;
}
#pragma GCC diagnostic pop

View File

@ -132,6 +132,31 @@ void streamTaskSendCheckMsg(SStreamTask* pTask) {
idstr, pTask->info.nodeId, req.stage, req.downstreamTaskId, req.downstreamNodeId, i, req.reqId);
code = streamSendCheckMsg(pTask, &req, pVgInfo->vgId, &pVgInfo->epSet);
}
} else if (pTask->outputInfo.type == TASK_OUTPUT__VTABLE_MAP) {
streamTaskStartMonitorCheckRsp(pTask);
SArray* pTaskInfos = pTask->outputInfo.vtableMapDispatcher.taskInfos;
int32_t numTasks = taosArrayGetSize(pTaskInfos);
stDebug("s-task:%s check %d vtable downstream tasks, ver:%" PRId64 "-%" PRId64 " window:%" PRId64 "-%" PRId64,
idstr, numTasks, pRange->range.minVer, pRange->range.maxVer, pWindow->skey, pWindow->ekey);
for (int32_t i = 0; i < numTasks; ++i) {
STaskDispatcherFixed* pAddr = taosArrayGet(pTaskInfos, i);
if (pAddr == NULL) {
continue;
}
setCheckDownstreamReqInfo(&req, tGenIdPI64(), pAddr->taskId, pAddr->nodeId);
streamTaskAddReqInfo(&pTask->taskCheckInfo, req.reqId, pAddr->taskId, pAddr->nodeId, idstr);
stDebug("s-task:%s (vgId:%d) stage:%" PRId64 " check vtable downstream task:0x%x (vgId:%d), QID:0x%" PRIx64,
idstr, pTask->info.nodeId, req.stage, req.downstreamTaskId, req.downstreamNodeId, req.reqId);
code = streamSendCheckMsg(pTask, &req, pAddr->nodeId, &pAddr->epSet);
if (code != TSDB_CODE_SUCCESS) {
stError("s-task:%s failed to send check msg to vtable downstream task:0x%x (vgId:%d), code:%s", idstr,
req.downstreamTaskId, req.downstreamNodeId, tstrerror(code));
}
}
} else { // for sink task, set it ready directly.
stDebug("s-task:%s (vgId:%d) set downstream ready, since no downstream", idstr, pTask->info.nodeId);
streamTaskStopMonitorCheckRsp(&pTask->taskCheckInfo, idstr);
@ -414,6 +439,8 @@ void streamTaskInitTaskCheckInfo(STaskCheckInfo* pInfo, STaskOutputInfo* pOutput
pInfo->notReadyTasks = 1;
} else if (pOutputInfo->type == TASK_OUTPUT__SHUFFLE_DISPATCH) {
pInfo->notReadyTasks = taosArrayGetSize(pOutputInfo->shuffleDispatcher.dbInfo.pVgroupInfos);
} else if (pOutputInfo->type == TASK_OUTPUT__VTABLE_MAP) {
pInfo->notReadyTasks = taosArrayGetSize(pOutputInfo->vtableMapDispatcher.taskInfos);
}
pInfo->startTs = startTs;
@ -581,6 +608,26 @@ int32_t doSendCheckMsg(SStreamTask* pTask, SDownstreamStatusInfo* p) {
break;
}
}
} else if (pOutputInfo->type == TASK_OUTPUT__VTABLE_MAP) {
SArray* pTaskInfos = pTask->outputInfo.vtableMapDispatcher.taskInfos;
int32_t numTasks = taosArrayGetSize(pTaskInfos);
for (int32_t i = 0; i < numTasks; ++i) {
STaskDispatcherFixed* pAddr = taosArrayGet(pTaskInfos, i);
if (pAddr == NULL) {
continue;
}
if (p->taskId == pAddr->taskId) {
setCheckDownstreamReqInfo(&req, p->reqId, pAddr->taskId, pAddr->nodeId);
stDebug("s-task:%s (vgId:%d) stage:%" PRId64
" re-send check vtable downstream task:0x%x(vgId:%d), QID:0x%" PRIx64,
id, pTask->info.nodeId, req.stage, req.downstreamTaskId, req.downstreamNodeId, p->reqId);
code = streamSendCheckMsg(pTask, &req, pAddr->nodeId, &pAddr->epSet);
break;
}
}
}
if (code) {

View File

@ -407,7 +407,8 @@ int32_t streamProcessCheckpointTriggerBlock(SStreamTask* pTask, SStreamDataBlock
chkptFailedByRetrieveReqToSource(pTask, checkpointId);
#endif
if (type == TASK_OUTPUT__FIXED_DISPATCH || type == TASK_OUTPUT__SHUFFLE_DISPATCH) {
if (type == TASK_OUTPUT__FIXED_DISPATCH || type == TASK_OUTPUT__SHUFFLE_DISPATCH ||
type == TASK_OUTPUT__VTABLE_MAP) {
stDebug("s-task:%s set childIdx:%d, and add checkpoint-trigger block into outputQ", id, pTask->info.selfChildId);
code = continueDispatchCheckpointTriggerBlock(pBlock, pTask); // todo handle this failure
} else { // only one task exists, no need to dispatch downstream info
@ -1374,6 +1375,20 @@ int32_t streamTaskInitTriggerDispatchInfo(SStreamTask* pTask, int64_t sendingChk
if (px == NULL) { // pause the stream task, if memory not enough
code = terrno;
}
} else if (pTask->outputInfo.type == TASK_OUTPUT__VTABLE_MAP) {
for (int32_t i = 0; i < streamTaskGetNumOfDownstream(pTask); ++i) {
STaskDispatcherFixed* pAddr = taosArrayGet(pTask->outputInfo.vtableMapDispatcher.taskInfos, i);
if (pAddr == NULL) {
continue;
}
STaskTriggerSendInfo p = {.sendTs = now, .recved = false, .nodeId = pAddr->nodeId, .taskId = pAddr->taskId};
void* px = taosArrayPush(pInfo->pDispatchTriggerList, &p);
if (px == NULL) { // pause the stream task, if memory not enough
code = terrno;
break;
}
}
} else {
for (int32_t i = 0; i < streamTaskGetNumOfDownstream(pTask); ++i) {
SVgroupInfo* pVgInfo = taosArrayGet(pTask->outputInfo.shuffleDispatcher.dbInfo.pVgroupInfos, i);

View File

@ -28,6 +28,8 @@ static int32_t doSendDispatchMsg(SStreamTask* pTask, const SStreamDispatchReq* p
static int32_t streamAddBlockIntoDispatchMsg(const SSDataBlock* pBlock, SStreamDispatchReq* pReq);
static int32_t streamSearchAndAddBlock(SStreamTask* pTask, SStreamDispatchReq* pReqs, SSDataBlock* pDataBlock,
int64_t groupId, int64_t now);
static int32_t streamMapAndAddBlock(SStreamTask* pTask, SStreamDispatchReq* pReqs, SSDataBlock* pDataBlock,
int64_t now);
static int32_t tInitStreamDispatchReq(SStreamDispatchReq* pReq, const SStreamTask* pTask, int32_t vgId,
int32_t numOfBlocks, int64_t dstTaskId, int32_t type);
static int32_t getFailedDispatchInfo(SDispatchMsgInfo* pMsgInfo, int64_t now);
@ -265,7 +267,7 @@ static SStreamDispatchReq* createDispatchDataReq(SStreamTask* pTask, const SStre
int32_t type = pTask->outputInfo.type;
int32_t num = streamTaskGetNumOfDownstream(pTask);
if (type != TASK_OUTPUT__SHUFFLE_DISPATCH && type != TASK_OUTPUT__FIXED_DISPATCH) {
if (type != TASK_OUTPUT__SHUFFLE_DISPATCH && type != TASK_OUTPUT__FIXED_DISPATCH && type != TASK_OUTPUT__VTABLE_MAP) {
terrno = TSDB_CODE_INVALID_PARA;
stError("s-task:%s invalid dispatch type:%d not dispatch data", pTask->id.idStr, type);
return NULL;
@ -294,6 +296,23 @@ static SStreamDispatchReq* createDispatchDataReq(SStreamTask* pTask, const SStre
return NULL;
}
}
} else if (type == TASK_OUTPUT__VTABLE_MAP) {
SArray *pTaskInfos = pTask->outputInfo.vtableMapDispatcher.taskInfos;
int32_t numOfTasks = taosArrayGetSize(pTaskInfos);
for (int32_t i = 0; i < numOfTasks; ++i) {
STaskDispatcherFixed *pAddr = taosArrayGet(pTaskInfos, i);
if (pAddr == NULL) {
continue;
}
code = tInitStreamDispatchReq(&pReqs[i], pTask, pData->srcTaskId, 0, pAddr->taskId, pData->type);
if (code != TSDB_CODE_SUCCESS) {
destroyDispatchMsg(pReqs, numOfTasks);
terrno = code;
return NULL;
}
}
} else { // shuffle dispatch
int32_t numOfBlocks = taosArrayGetSize(pData->blocks);
int32_t downstreamTaskId = pTask->outputInfo.fixedDispatcher.taskId;
@ -401,6 +420,51 @@ static int32_t doBuildDispatchMsg(SStreamTask* pTask, const SStreamDataBlock* pD
}
}
pTask->msgInfo.pData = pReqs;
} else if (pTask->outputInfo.type == TASK_OUTPUT__VTABLE_MAP) {
SArray *pTaskInfos = pTask->outputInfo.vtableMapDispatcher.taskInfos;
int32_t numOfTasks = taosArrayGetSize(pTaskInfos);
for (int32_t i = 0; i < numOfBlocks; i++) {
SSDataBlock* pDataBlock = taosArrayGet(pData->blocks, i);
if (pDataBlock == NULL) {
destroyDispatchMsg(pReqs, numOfTasks);
return terrno;
}
// TODO: do not use broadcast
if (pDataBlock->info.type == STREAM_DELETE_RESULT || pDataBlock->info.type == STREAM_CHECKPOINT ||
pDataBlock->info.type == STREAM_TRANS_STATE) {
for (int32_t j = 0; j < numOfTasks; j++) {
code = streamAddBlockIntoDispatchMsg(pDataBlock, &pReqs[j]);
if (code != 0) {
destroyDispatchMsg(pReqs, numOfTasks);
return code;
}
// it's a new vnode to receive dispatch msg, so add one
if (pReqs[j].blockNum == 0) {
STaskDispatcherFixed *pAddr = taosArrayGet(pTaskInfos, j);
if (pAddr == NULL) {
destroyDispatchMsg(pReqs, numOfTasks);
return terrno;
}
addDispatchEntry(&pTask->msgInfo, pAddr->nodeId, now, true);
}
pReqs[j].blockNum++;
}
continue;
}
code = streamMapAndAddBlock(pTask, pReqs, pDataBlock, now);
if (code != 0) {
destroyDispatchMsg(pReqs, numOfTasks);
return code;
}
}
pTask->msgInfo.pData = pReqs;
}
@ -430,6 +494,39 @@ static int32_t sendDispatchMsg(SStreamTask* pTask, SStreamDispatchReq* pDispatch
pTask->info.selfChildId, 1, downstreamTaskId, vgId, msgId);
code = doSendDispatchMsg(pTask, pDispatchMsg, vgId, pEpSet);
} else if (pTask->outputInfo.type == TASK_OUTPUT__VTABLE_MAP) {
SArray *pTaskInfos = pTask->outputInfo.vtableMapDispatcher.taskInfos;
int32_t numOfTasks = taosArrayGetSize(pTaskInfos);
int32_t numOfBranches = taosArrayGetSize(pTask->msgInfo.pSendInfo);
stDebug("s-task:%s (child taskId:%d) start to vtable-map-dispatch blocks to %d/%d vgroup(s), msgId:%d", id,
pTask->info.selfChildId, numOfBranches, numOfTasks, msgId);
int32_t numOfSend = 0;
for (int32_t i = 0; i < numOfTasks; ++i) {
if (pDispatchMsg[i].blockNum > 0) {
STaskDispatcherFixed *pAddr = taosArrayGet(pTaskInfos, i);
if (pAddr == NULL) {
code = terrno;
break;
}
stDebug("s-task:%s (child taskId:%d) vtable-map-dispatch blocks:%d to vgId:%d", id, pTask->info.selfChildId,
pDispatchMsg[i].blockNum, pAddr->nodeId);
code = doSendDispatchMsg(pTask, &pDispatchMsg[i], pAddr->nodeId, &pAddr->epSet);
if (code != TSDB_CODE_SUCCESS) {
break;
}
// no need to try remain, all already send.
if (++numOfSend == numOfBranches) {
break;
}
}
}
stDebug("s-task:%s complete vtable-map-dispatch blocks to all %d vnodes, msgId:%d", id, numOfTasks, msgId);
} else {
SArray* vgInfo = pTask->outputInfo.shuffleDispatcher.dbInfo.pVgroupInfos;
int32_t numOfVgroups = taosArrayGetSize(vgInfo);
@ -798,6 +895,32 @@ int32_t streamSearchAndAddBlock(SStreamTask* pTask, SStreamDispatchReq* pReqs, S
}
}
int32_t streamMapAndAddBlock(SStreamTask* pTask, SStreamDispatchReq* pReqs, SSDataBlock* pDataBlock, int64_t now) {
int32_t code = TSDB_CODE_SUCCESS;
int32_t lino = 0;
SArray* pTaskInfos = pTask->outputInfo.vtableMapDispatcher.taskInfos;
SSHashObj* pVtableMap = pTask->outputInfo.vtableMapDispatcher.vtableMap;
int32_t* pIdx = tSimpleHashGet(pVtableMap, &pDataBlock->info.id.uid, sizeof(int64_t));
QUERY_CHECK_NULL(pIdx, code, lino, _end, terrno);
STaskDispatcherFixed* pAddr = taosArrayGet(pTaskInfos, *pIdx);
QUERY_CHECK_NULL(pAddr, code, lino, _end, terrno);
code = streamAddBlockIntoDispatchMsg(pDataBlock, &pReqs[*pIdx]);
QUERY_CHECK_CODE(code, lino, _end);
if (pReqs[*pIdx].blockNum == 0) {
addDispatchEntry(&pTask->msgInfo, pAddr->nodeId, now, false);
}
pReqs[*pIdx].blockNum++;
_end:
if (code != TSDB_CODE_SUCCESS) {
stError("s-task:%s failed at %s lino %d since %s", pTask->id.idStr, __func__, lino, tstrerror(code));
}
return code;
}
int32_t streamDispatchStreamBlock(SStreamTask* pTask) {
const char* id = pTask->id.idStr;
int32_t code = 0;

View File

@ -48,7 +48,8 @@ static int32_t doOutputResultBlockImpl(SStreamTask* pTask, SStreamDataBlock* pBl
pTask->outputInfo.smaSink.smaSink(pTask->outputInfo.smaSink.vnode, pTask->outputInfo.smaSink.smaId, pBlock->blocks);
destroyStreamDataBlock(pBlock);
} else {
if (type != TASK_OUTPUT__FIXED_DISPATCH && type != TASK_OUTPUT__SHUFFLE_DISPATCH) {
if (type != TASK_OUTPUT__FIXED_DISPATCH && type != TASK_OUTPUT__SHUFFLE_DISPATCH &&
type != TASK_OUTPUT__VTABLE_MAP) {
stError("s-task:%s invalid stream output type:%d, internal error", pTask->id.idStr, type);
return TSDB_CODE_STREAM_INTERNAL_ERROR;
}
@ -631,7 +632,7 @@ void streamProcessTransstateBlock(SStreamTask* pTask, SStreamDataBlock* pBlock)
}
// transfer the ownership of executor state
if (type == TASK_OUTPUT__FIXED_DISPATCH || type == TASK_OUTPUT__SHUFFLE_DISPATCH) {
if (type == TASK_OUTPUT__FIXED_DISPATCH || type == TASK_OUTPUT__SHUFFLE_DISPATCH || type == TASK_OUTPUT__VTABLE_MAP) {
if (level == TASK_LEVEL__SOURCE) {
stDebug("s-task:%s add transfer-state block into outputQ", id);
} else {

View File

@ -294,6 +294,9 @@ void tFreeStreamTask(void* pParam) {
tDeleteSchemaWrapper(pTask->outputInfo.tbSink.pTagSchema);
} else if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) {
taosArrayDestroy(pTask->outputInfo.shuffleDispatcher.dbInfo.pVgroupInfos);
} else if (pTask->outputInfo.type == TASK_OUTPUT__VTABLE_MAP) {
taosArrayDestroy(pTask->outputInfo.vtableMapDispatcher.taskInfos);
tSimpleHashCleanup(pTask->outputInfo.vtableMapDispatcher.vtableMap);
}
streamTaskCleanupCheckInfo(&pTask->taskCheckInfo);
@ -309,6 +312,9 @@ void tFreeStreamTask(void* pParam) {
streamMutexDestroy(&pTask->taskCheckInfo.checkInfoLock);
}
taosArrayDestroy(pTask->pVTables);
pTask->pVTables = NULL;
streamDestroyStateMachine(pTask->status.pSM);
pTask->status.pSM = NULL;
@ -584,6 +590,9 @@ int32_t streamTaskGetNumOfDownstream(const SStreamTask* pTask) {
return 0;
} else if (type == TASK_OUTPUT__FIXED_DISPATCH) {
return 1;
} else if (type == TASK_OUTPUT__VTABLE_MAP) {
SArray* pTaskInfos = pTask->outputInfo.vtableMapDispatcher.taskInfos;
return taosArrayGetSize(pTaskInfos);
} else {
SArray* vgInfo = pTask->outputInfo.shuffleDispatcher.dbInfo.pVgroupInfos;
return taosArrayGetSize(vgInfo);
@ -727,6 +736,37 @@ int32_t streamTaskUpdateDownstreamInfo(SStreamTask* pTask, int32_t nodeId, const
pDispatcher->taskId, nodeId, buf);
}
}
} else if (type == TASK_OUTPUT__VTABLE_MAP) {
SArray* pTaskInfos = pTask->outputInfo.vtableMapDispatcher.taskInfos;
for (int32_t i = 0; i < taosArrayGetSize(pTaskInfos); ++i) {
STaskDispatcherFixed* pAddr = taosArrayGet(pTaskInfos, i);
if (pAddr == NULL) {
continue;
}
if (pAddr->nodeId == nodeId) {
bool isEqual = isEpsetEqual(&pAddr->epSet, pEpSet);
if (!isEqual) {
*pUpdated = true;
char tmp[512] = {0};
code = epsetToStr(&pAddr->epSet, tmp, tListLen(tmp));
if (code != 0) { // print error and continue
stError("%s failed to convert epset to str, code:%s", pTask->id.idStr, tstrerror(code));
return code;
}
epsetAssign(&pAddr->epSet, pEpSet);
stDebug("s-task:0x%x update dispatch info, task:0x%x(nodeId:%d) newEpset:%s old:%s", id, pAddr->taskId,
nodeId, buf, tmp);
} else {
stDebug("s-task:0x%x not update dispatch info, since identical, task:0x%x(nodeId:%d) epset:%s", id,
pAddr->taskId, nodeId, buf);
}
break;
}
}
}
return code;
@ -1173,6 +1213,18 @@ SEpSet* streamTaskGetDownstreamEpInfo(SStreamTask* pTask, int32_t taskId) {
return &pVgInfo->epSet;
}
}
} else if (pTask->info.taskLevel == TASK_OUTPUT__VTABLE_MAP) {
SArray* pTaskInfos = pTask->outputInfo.vtableMapDispatcher.taskInfos;
for (int32_t i = 0; i < taosArrayGetSize(pTaskInfos); ++i) {
STaskDispatcherFixed* pAddr = taosArrayGet(pTaskInfos, i);
if (pAddr == NULL) {
continue;
}
if (pAddr->taskId == taskId) {
return &pAddr->epSet;
}
}
}
return NULL;
@ -1436,6 +1488,149 @@ _exit:
return code;
}
int32_t tEncodeVTablesInfo(SEncoder* pEncoder, SArray* pVTables) {
int32_t code = TSDB_CODE_SUCCESS;
int32_t lino = 0;
SVCTableMergeInfo* pMergeInfo = NULL;
int32_t mergeNum = taosArrayGetSize(pVTables);
TAOS_CHECK_EXIT(tEncodeI32(pEncoder, mergeNum));
for (int32_t i = 0; i < mergeNum; ++i) {
pMergeInfo = (SVCTableMergeInfo*)taosArrayGet(pVTables, i);
TAOS_CHECK_EXIT(tEncodeI64(pEncoder, pMergeInfo->uid));
TAOS_CHECK_EXIT(tEncodeI32(pEncoder, pMergeInfo->numOfSrcTbls));
}
_exit:
if (code != TSDB_CODE_SUCCESS) {
stError("%s failed at line %d since %s", __func__, lino, tstrerror(code));
}
return code;
}
static int32_t tDecodeVTablesInfo(SDecoder* pDecoder, SArray** pTables) {
int32_t code = TSDB_CODE_SUCCESS;
int32_t lino = 0;
int32_t mergeNum = 0;
TAOS_CHECK_EXIT(tDecodeI32(pDecoder, &mergeNum));
if (mergeNum > 0) {
*pTables = taosArrayInit(mergeNum, sizeof(SVCTableMergeInfo));
QUERY_CHECK_NULL(*pTables, code, lino, _exit, terrno);
}
SVCTableMergeInfo mergeInfo;
for (int32_t i = 0; i < mergeNum; ++i) {
TAOS_CHECK_EXIT(tDecodeI64(pDecoder, &mergeInfo.uid));
TAOS_CHECK_EXIT(tDecodeI32(pDecoder, &mergeInfo.numOfSrcTbls));
if (taosArrayPush(*pTables, &mergeInfo) == NULL) {
TAOS_CHECK_EXIT(terrno);
}
}
_exit:
if (code != TSDB_CODE_SUCCESS) {
stError("%s failed at line %d since %s", __func__, lino, tstrerror(code));
}
return code;
}
int32_t tSerializeDispatcherTaskInfo(SEncoder* pEncoder, const SArray* pTaskInfos) {
int32_t code = TSDB_CODE_SUCCESS;
int32_t lino = 0;
int32_t nTasks = taosArrayGetSize(pTaskInfos);
TAOS_CHECK_EXIT(tEncodeI32(pEncoder, nTasks));
for (int32_t i = 0; i < nTasks; ++i) {
STaskDispatcherFixed* pAddr = taosArrayGet(pTaskInfos, i);
TAOS_CHECK_EXIT(tEncodeI32(pEncoder, pAddr->taskId));
TAOS_CHECK_EXIT(tEncodeI32(pEncoder, pAddr->nodeId));
TAOS_CHECK_EXIT(tEncodeSEpSet(pEncoder, &pAddr->epSet));
}
_exit:
if (code != TSDB_CODE_SUCCESS) {
stError("%s failed at line %d since %s", __func__, lino, tstrerror(code));
}
return code;
}
int32_t tSerializeDispatcherVtableMap(SEncoder* pEncoder, const SSHashObj* pVtables) {
int32_t code = TSDB_CODE_SUCCESS;
int32_t lino = 0;
int32_t tbNum = tSimpleHashGetSize(pVtables);
TAOS_CHECK_EXIT(tEncodeI32(pEncoder, tbNum));
int32_t iter = 0;
void* p = tSimpleHashIterate(pVtables, NULL, &iter);
while (p != NULL) {
int64_t* pUid = tSimpleHashGetKey(p, NULL);
int32_t* pIdx = p;
TAOS_CHECK_EXIT(tEncodeI64(pEncoder, *pUid));
TAOS_CHECK_EXIT(tEncodeI32(pEncoder, *pIdx));
p = tSimpleHashIterate(pVtables, p, &iter);
}
_exit:
if (code != TSDB_CODE_SUCCESS) {
stError("%s failed at line %d since %s", __func__, lino, tstrerror(code));
}
return code;
}
int32_t tDeserializeDispatcherTaskInfo(SDecoder* pDecoder, SArray** ppTaskInfos) {
int32_t code = TSDB_CODE_SUCCESS;
int32_t lino = 0;
int32_t nTasks = 0;
TAOS_CHECK_EXIT(tDecodeI32(pDecoder, &nTasks));
if (nTasks <= 0) {
return code;
}
*ppTaskInfos = taosArrayInit(nTasks, sizeof(STaskDispatcherFixed));
QUERY_CHECK_NULL(*ppTaskInfos, code, lino, _exit, terrno);
STaskDispatcherFixed addr;
for (int32_t i = 0; i < nTasks; ++i) {
TAOS_CHECK_EXIT(tDecodeI32(pDecoder, &addr.taskId));
TAOS_CHECK_EXIT(tDecodeI32(pDecoder, &addr.nodeId));
TAOS_CHECK_EXIT(tDecodeSEpSet(pDecoder, &addr.epSet));
void* px = taosArrayPush(*ppTaskInfos, &addr);
QUERY_CHECK_NULL(px, code, lino, _exit, terrno);
}
_exit:
if (code != TSDB_CODE_SUCCESS) {
stError("%s failed at line %d since %s", __func__, lino, tstrerror(code));
}
return code;
}
int32_t tDeserializeDispatcherVtableMap(SDecoder* pDecoder, SSHashObj** ppVtables) {
int32_t code = TSDB_CODE_SUCCESS;
int32_t lino = 0;
int32_t tbNum = 0;
TAOS_CHECK_EXIT(tDecodeI32(pDecoder, &tbNum));
if (tbNum <= 0) {
return code;
}
*ppVtables = tSimpleHashInit(tbNum, taosGetDefaultHashFunction(TSDB_DATA_TYPE_BIGINT));
QUERY_CHECK_NULL(*ppVtables, code, lino, _exit, terrno);
uint64_t uid = 0;
int32_t idx = 0;
for (int32_t i = 0; i < tbNum; ++i) {
TAOS_CHECK_EXIT(tDecodeU64(pDecoder, &uid));
TAOS_CHECK_EXIT(tDecodeI32(pDecoder, &idx));
TAOS_CHECK_EXIT(tSimpleHashPut(*ppVtables, &uid, sizeof(uid), &idx, sizeof(idx)));
}
_exit:
if (code != TSDB_CODE_SUCCESS) {
stError("%s failed at line %d since %s", __func__, lino, tstrerror(code));
}
return code;
}
int32_t tEncodeStreamTask(SEncoder* pEncoder, const SStreamTask* pTask) {
int32_t code = 0;
int32_t lino;
@ -1500,6 +1695,9 @@ int32_t tEncodeStreamTask(SEncoder* pEncoder, const SStreamTask* pTask) {
} else if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) {
TAOS_CHECK_EXIT(tSerializeSUseDbRspImp(pEncoder, &pTask->outputInfo.shuffleDispatcher.dbInfo));
TAOS_CHECK_EXIT(tEncodeCStr(pEncoder, pTask->outputInfo.shuffleDispatcher.stbFullName));
} else if (pTask->outputInfo.type == TASK_OUTPUT__VTABLE_MAP) {
TAOS_CHECK_EXIT(tSerializeDispatcherTaskInfo(pEncoder, pTask->outputInfo.vtableMapDispatcher.taskInfos));
TAOS_CHECK_EXIT(tSerializeDispatcherVtableMap(pEncoder, pTask->outputInfo.vtableMapDispatcher.vtableMap));
}
TAOS_CHECK_EXIT(tEncodeI64(pEncoder, pTask->info.delaySchedParam));
TAOS_CHECK_EXIT(tEncodeI8(pEncoder, pTask->subtableWithoutMd5));
@ -1508,6 +1706,7 @@ int32_t tEncodeStreamTask(SEncoder* pEncoder, const SStreamTask* pTask) {
if (pTask->ver >= SSTREAM_TASK_ADD_NOTIFY_VER) {
TAOS_CHECK_EXIT(tEncodeI8(pEncoder, pTask->info.hasAggTasks));
TAOS_CHECK_EXIT(tEncodeStreamNotifyInfo(pEncoder, &pTask->notifyInfo));
TAOS_CHECK_EXIT(tEncodeVTablesInfo(pEncoder, pTask->pVTables));
}
tEndEncode(pEncoder);
@ -1601,6 +1800,9 @@ int32_t tDecodeStreamTask(SDecoder* pDecoder, SStreamTask* pTask) {
} else if (pTask->outputInfo.type == TASK_OUTPUT__SHUFFLE_DISPATCH) {
TAOS_CHECK_EXIT(tDeserializeSUseDbRspImp(pDecoder, &pTask->outputInfo.shuffleDispatcher.dbInfo));
TAOS_CHECK_EXIT(tDecodeCStrTo(pDecoder, pTask->outputInfo.shuffleDispatcher.stbFullName));
} else if (pTask->outputInfo.type == TASK_OUTPUT__VTABLE_MAP) {
TAOS_CHECK_EXIT(tDeserializeDispatcherTaskInfo(pDecoder, &pTask->outputInfo.vtableMapDispatcher.taskInfos));
TAOS_CHECK_EXIT(tDeserializeDispatcherVtableMap(pDecoder, &pTask->outputInfo.vtableMapDispatcher.vtableMap));
}
TAOS_CHECK_EXIT(tDecodeI64(pDecoder, &pTask->info.delaySchedParam));
if (pTask->ver >= SSTREAM_TASK_SUBTABLE_CHANGED_VER) {
@ -1611,6 +1813,7 @@ int32_t tDecodeStreamTask(SDecoder* pDecoder, SStreamTask* pTask) {
if (pTask->ver >= SSTREAM_TASK_ADD_NOTIFY_VER) {
TAOS_CHECK_EXIT(tDecodeI8(pDecoder, &pTask->info.hasAggTasks));
TAOS_CHECK_EXIT(tDecodeStreamNotifyInfo(pDecoder, &pTask->notifyInfo));
TAOS_CHECK_EXIT(tDecodeVTablesInfo(pDecoder, &pTask->pVTables));
}
tEndDecode(pDecoder);

View File

@ -183,6 +183,7 @@ TAOS_DEFINE_ERROR(TSDB_CODE_MND_INVALID_CONN_ID, "Invalid connection id
TAOS_DEFINE_ERROR(TSDB_CODE_MND_USER_DISABLED, "User is disabled")
TAOS_DEFINE_ERROR(TSDB_CODE_MND_INVALID_PLATFORM, "Unsupported feature on this platform")
TAOS_DEFINE_ERROR(TSDB_CODE_MND_RETURN_VALUE_NULL, "Return value is null")
TAOS_DEFINE_ERROR(TSDB_CODE_MND_INTERNAL_ERROR, "Mnode internal error")
// mnode-sdb
TAOS_DEFINE_ERROR(TSDB_CODE_SDB_OBJ_ALREADY_THERE, "Object already there")

View File

@ -807,6 +807,8 @@ bool taosLRUCacheRelease(SLRUCache *cache, LRUHandle *handle, bool eraseIfLastRe
void *taosLRUCacheValue(SLRUCache *cache, LRUHandle *handle) { return ((SLRUEntry *)handle)->value; }
void taosLRUCacheUpdate(SLRUCache *cache, LRUHandle *handle, void *value) { ((SLRUEntry *)handle)->value = value; }
size_t taosLRUCacheGetUsage(SLRUCache *cache) {
size_t usage = 0;

View File

@ -851,7 +851,6 @@
,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/smaBasic.py -N 3
,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/smaTest.py
,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/smaTest.py -R
,,y,system-test,./pytest.sh python3 ./test.py -f 0-others/sma_index.py
,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/sml_TS-3724.py
,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/sml-TD19291.py
,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/varbinary.py
@ -1704,9 +1703,6 @@
,,y,script,./test.sh -f tsim/stable/tag_rename.sim
,,y,script,./test.sh -f tsim/stable/values.sim
,,y,script,./test.sh -f tsim/stable/vnode3.sim
,,n,script,./test.sh -f tsim/sma/drop_sma.sim
,,y,script,./test.sh -f tsim/sma/sma_leak.sim
,,y,script,./test.sh -f tsim/sma/tsmaCreateInsertQuery.sim
,,y,script,./test.sh -f tsim/sma/rsmaCreateInsertQuery.sim
,,y,script,./test.sh -f tsim/sma/rsmaCreateInsertQueryDelete.sim
@ -1828,7 +1824,6 @@
,,y,script,./test.sh -f tsim/tag/change_multi_tag.sim
,,y,script,./test.sh -f tmp/monitor.sim
,,y,script,./test.sh -f tsim/tagindex/add_index.sim
,,n,script,./test.sh -f tsim/tagindex/sma_and_tag_index.sim
,,y,script,./test.sh -f tsim/tagindex/indexOverflow.sim
,,y,script,./test.sh -f tsim/view/view.sim
,,y,script,./test.sh -f tsim/query/cache_last.sim

View File

@ -604,7 +604,6 @@
,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/smaBasic.py -N 3
,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/smaTest.py
,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/smaTest.py -R
,,y,system-test,./pytest.sh python3 ./test.py -f 0-others/sma_index.py
,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/sml-TD19291.py
,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/varbinary.py
,,y,system-test,./pytest.sh python3 ./test.py -f 2-query/spread.py
@ -1416,9 +1415,6 @@
,,y,script,./test.sh -f tsim/stable/tag_rename.sim
,,y,script,./test.sh -f tsim/stable/values.sim
,,y,script,./test.sh -f tsim/stable/vnode3.sim
,,n,script,./test.sh -f tsim/sma/drop_sma.sim
,,y,script,./test.sh -f tsim/sma/sma_leak.sim
,,y,script,./test.sh -f tsim/sma/tsmaCreateInsertQuery.sim
,,y,script,./test.sh -f tsim/sma/rsmaCreateInsertQuery.sim
,,y,script,./test.sh -f tsim/sma/rsmaCreateInsertQueryDelete.sim
@ -1533,7 +1529,6 @@
,,y,script,./test.sh -f tsim/tag/change_multi_tag.sim
,,y,script,./test.sh -f tmp/monitor.sim
,,y,script,./test.sh -f tsim/tagindex/add_index.sim
,,n,script,./test.sh -f tsim/tagindex/sma_and_tag_index.sim
,,y,script,./test.sh -f tsim/tagindex/indexOverflow.sim
,,y,script,./test.sh -f tsim/query/cache_last.sim
,,y,script,./test.sh -f tsim/query/const.sim

View File

@ -14,7 +14,6 @@ sed -i '$a\%%FINISHED%%' ${case_file}
#unsupported case:sma
sed -i '/tsim\/sync\/vnodesnapshot-rsma-test.sim/d' ${case_file}
sed -i '/tsim\/sma/d' ${case_file}
sed -i '/tsim\/tagindex\/sma_and_tag_index.sim/d' ${case_file}
#unsupported case:to_char
sed -i '/tsim\/query\/sort-pre-cols.sim/d' ${case_file}
sed -i '/2-query\/nestedQueryInterval.py/d' ${case_file}

View File

@ -1,183 +0,0 @@
system sh/stop_dnodes.sh
system sh/deploy.sh -n dnode1 -i 1
system sh/deploy.sh -n dnode2 -i 2
system sh/deploy.sh -n dnode3 -i 3
system sh/cfg.sh -n dnode1 -c supportVnodes -v 0
system sh/cfg.sh -n dnode2 -c supportVnodes -v 4
system sh/cfg.sh -n dnode3 -c supportVnodes -v 4
print ========== step1
system sh/exec.sh -n dnode1 -s start
sql connect
print ========== step2
sql create dnode $hostname port 7200
sql create dnode $hostname port 7300
system sh/exec.sh -n dnode2 -s start
system sh/exec.sh -n dnode3 -s start
$x = 0
step2:
$x = $x + 1
sleep 1000
if $x == 10 then
print ====> dnode not ready!
return -1
endi
sql select * from information_schema.ins_dnodes
print ===> $data00 $data01 $data02 $data03 $data04 $data05
print ===> $data10 $data11 $data12 $data13 $data14 $data15
if $rows != 3 then
return -1
endi
if $data(1)[4] != ready then
goto step2
endi
if $data(2)[4] != ready then
goto step2
endi
if $data(3)[4] != ready then
goto step2
endi
print ========== step3
sql create database d1 vgroups 1
sql use d1;
print --> create stb
sql create table if not exists stb (ts timestamp, c1 int, c2 float, c3 double) tags (t1 int unsigned);
print --> create sma
sql create sma index sma_index_name1 on stb function(max(c1),max(c2),min(c1)) interval(6m,10s) sliding(6m);
print --> show sma
sql show indexes from stb from d1;
if $rows != 2 then
return -1
endi
if $data[0][6] == tag_index then
if $data[1][0] != sma_index_name1 then
return -1
endi
if $data[1][1] != d1 then
return -1
endi
if $data[1][2] != stb then
return -1
endi
else
if $data[0][0] != sma_index_name1 then
return -1
endi
if $data[0][1] != d1 then
return -1
endi
if $data[0][2] != stb then
return -1
endi
endi
sql drop index sma_index_name1
print --> drop stb
sql drop table stb;
print ========== step4 repeat
print --> create stb
sql create table if not exists stb (ts timestamp, c1 int, c2 float, c3 double) tags (t1 int unsigned);
print --> create sma
sql create sma index sma_index_name1 on stb function(max(c1),max(c2),min(c1)) interval(6m,10s) sliding(6m);
print --> show sma
sql show indexes from stb from d1;
if $rows != 2 then
return -1
endi
if $data[0][6] == tag_index then
if $data[1][0] != sma_index_name1 then
return -1
endi
if $data[1][1] != d1 then
return -1
endi
if $data[1][2] != stb then
return -1
endi
else
if $data[0][0] != sma_index_name1 then
return -1
endi
if $data[0][1] != d1 then
return -1
endi
if $data[0][2] != stb then
return -1
endi
endi
sql drop index sma_index_name1
print --> drop stb
sql drop table stb;
print ========== step5
sql drop database if exists db;
sql create database db duration 120;
sql use db;
sql create table stb1(ts timestamp, c_int int, c_bint bigint, c_sint smallint, c_tint tinyint, c_float float, c_double double, c_bool bool, c_binary binary(16), c_nchar nchar(32), c_ts timestamp, c_tint_un tinyint unsigned, c_sint_un smallint unsigned, c_int_un int unsigned, c_bint_un bigint unsigned) tags (t_int int);
sql CREATE SMA INDEX sma_index_1 ON stb1 function(min(c_int), max(c_int)) interval(6m, 10s) sliding(6m) watermark 5s;
print ========== step6 repeat
sql drop database if exists db;
sql create database db duration 120;
sql use db;
sql create table stb1(ts timestamp, c_int int, c_bint bigint ) tags (t_int int);
sql CREATE SMA INDEX sma_index_1 ON stb1 function(min(c_int), max(c_int)) interval(6m, 10s) sliding(6m) watermark 5s;
print ========== step7
sql drop database if exists db;
sql create database db duration 120;
sql use db;
sql create table stb1(ts timestamp, c_int int, c_bint bigint, c_sint smallint, c_tint tinyint,c_float float, c_double double, c_bool bool,c_binary binary(16), c_nchar nchar(32), c_ts timestamp,c_tint_un tinyint unsigned, c_sint_un smallint unsigned,c_int_un int unsigned, c_bint_un bigint unsigned) tags (t_int int);
sql create table ct1 using stb1 tags ( 1 );
sql create table ct2 using stb1 tags ( 2 );
sql create table ct3 using stb1 tags ( 3 );
sql create table ct4 using stb1 tags ( 4 );
sql CREATE SMA INDEX sma_index_1 ON stb1 function(min(c_int), max(c_int)) interval(6m, 10s) sliding(6m) watermark 5s;
sql CREATE SMA INDEX sma_index_2 ON stb1 function(min(c_int), max(c_int)) interval(6m, 10s) sliding(6m) max_delay 6m;
sql CREATE SMA INDEX sma_index_3 ON stb1 function(min(c_int), max(c_int)) interval(6m, 10s) watermark 5s max_delay 6m;
sql DROP INDEX sma_index_1 ;
sql DROP INDEX sma_index_2 ;
sql DROP INDEX sma_index_3 ;
print ========== step8
sql drop database if exists db;
sleep 2000
sql create database db duration 120;
sql use db;
sql create table stb1(ts timestamp, c_int int, c_bint bigint, c_sint smallint, c_tint tinyint,c_float float, c_double double, c_bool bool,c_binary binary(16), c_nchar nchar(32), c_ts timestamp,c_tint_un tinyint unsigned, c_sint_un smallint unsigned,c_int_un int unsigned, c_bint_un bigint unsigned) tags (t_int int);
sql create table ct1 using stb1 tags ( 1 );
sql create table ct2 using stb1 tags ( 2 );
sql create table ct3 using stb1 tags ( 3 );
sql create table ct4 using stb1 tags ( 4 );
sql CREATE SMA INDEX sma_index_1 ON stb1 function(min(c_int), max(c_int)) interval(6m, 10s) sliding(6m) watermark 5s;
sql CREATE SMA INDEX sma_index_2 ON stb1 function(min(c_int), max(c_int)) interval(6m, 10s) sliding(6m) max_delay 6m;
sql CREATE SMA INDEX sma_index_3 ON stb1 function(min(c_int), max(c_int)) interval(6m, 10s) watermark 5s max_delay 6m;
sql DROP INDEX sma_index_1 ;
sql DROP INDEX sma_index_2 ;
sql DROP INDEX sma_index_3 ;
system sh/exec.sh -n dnode1 -s stop -x SIGINT
system sh/exec.sh -n dnode2 -s stop -x SIGINT
system sh/exec.sh -n dnode3 -s stop -x SIGINT
system sh/exec.sh -n dnode4 -s stop -x SIGINT

View File

@ -1,154 +0,0 @@
system sh/stop_dnodes.sh
system sh/deploy.sh -n dnode1 -i 1
system sh/deploy.sh -n dnode2 -i 2
system sh/deploy.sh -n dnode3 -i 3
system sh/cfg.sh -n dnode1 -c supportVnodes -v 0
system sh/cfg.sh -n dnode2 -c supportVnodes -v 4
system sh/cfg.sh -n dnode3 -c supportVnodes -v 4
print ========== step1
system sh/exec.sh -n dnode1 -s start
sql connect
print ========== step2
sql create dnode $hostname port 7200
sql create dnode $hostname port 7300
system sh/exec.sh -n dnode2 -s start
system sh/exec.sh -n dnode3 -s start
$x = 0
step2:
$x = $x + 1
sleep 1000
if $x == 10 then
print ====> dnode not ready!
return -1
endi
sql select * from information_schema.ins_dnodes
print ===> $data00 $data01 $data02 $data03 $data04 $data05
print ===> $data10 $data11 $data12 $data13 $data14 $data15
if $rows != 3 then
return -1
endi
if $data(1)[4] != ready then
goto step2
endi
if $data(2)[4] != ready then
goto step2
endi
if $data(3)[4] != ready then
goto step2
endi
print ========== step3
sql create database d1 vgroups 1
sql use d1;
print --> create stb
sql create table if not exists stb (ts timestamp, c1 int, c2 float, c3 double) tags (t1 int unsigned);
print --> create sma
sql create sma index sma_index_name1 on stb function(max(c1),max(c2),min(c1)) interval(6m,10s) sliding(6m);
return
print --> show sma
sql show indexes from stb from d1;
if $rows != 1 then
return -1
endi
if $data[0][0] != sma_index_name1 then
return -1
endi
if $data[0][1] != d1 then
return -1
endi
if $data[0][2] != stb then
return -1
endi
print --> drop stb
sql drop table stb;
print ========== step4 repeat
print --> create stb
sql create table if not exists stb (ts timestamp, c1 int, c2 float, c3 double) tags (t1 int unsigned);
print --> create sma
sql create sma index sma_index_name1 on stb function(max(c1),max(c2),min(c1)) interval(6m,10s) sliding(6m);
print --> show sma
sql show indexes from stb from d1;
if $rows != 1 then
return -1
endi
if $data[0][0] != sma_index_name1 then
return -1
endi
if $data[0][1] != d1 then
return -1
endi
if $data[0][2] != stb then
return -1
endi
print --> drop stb
sql drop table stb;
print ========== step5
sql drop database if exists db;
sql create database db duration 120;
sql use db;
sql create table stb1(ts timestamp, c_int int, c_bint bigint, c_sint smallint, c_tint tinyint, c_float float, c_double double, c_bool bool, c_binary binary(16), c_nchar nchar(32), c_ts timestamp, c_tint_un tinyint unsigned, c_sint_un smallint unsigned, c_int_un int unsigned, c_bint_un bigint unsigned) tags (t_int int);
sql CREATE SMA INDEX sma_index_1 ON stb1 function(min(c_int), max(c_int)) interval(6m, 10s) sliding(6m) watermark 5s;
print ========== step6 repeat
sql drop database if exists db;
sql create database db duration 120;
sql use db;
sql create table stb1(ts timestamp, c_int int, c_bint bigint ) tags (t_int int);
sql CREATE SMA INDEX sma_index_1 ON stb1 function(min(c_int), max(c_int)) interval(6m, 10s) sliding(6m) watermark 5s;
print ========== step7
sql drop database if exists db;
sql create database db duration 120;
sql use db;
sql create table stb1(ts timestamp, c_int int, c_bint bigint, c_sint smallint, c_tint tinyint,c_float float, c_double double, c_bool bool,c_binary binary(16), c_nchar nchar(32), c_ts timestamp,c_tint_un tinyint unsigned, c_sint_un smallint unsigned,c_int_un int unsigned, c_bint_un bigint unsigned) tags (t_int int);
sql create table ct1 using stb1 tags ( 1 );
sql create table ct2 using stb1 tags ( 2 );
sql create table ct3 using stb1 tags ( 3 );
sql create table ct4 using stb1 tags ( 4 );
sql CREATE SMA INDEX sma_index_1 ON stb1 function(min(c_int), max(c_int)) interval(6m, 10s) sliding(6m) watermark 5s;
sql CREATE SMA INDEX sma_index_2 ON stb1 function(min(c_int), max(c_int)) interval(6m, 10s) sliding(6m) max_delay 6m;
sql CREATE SMA INDEX sma_index_3 ON stb1 function(min(c_int), max(c_int)) interval(6m, 10s) watermark 5s max_delay 6m;
sql DROP INDEX sma_index_1 ;
sql DROP INDEX sma_index_2 ;
sql DROP INDEX sma_index_3 ;
print ========== step8
sql drop database if exists db;
sql create database db duration 120;
sql use db;
sql create table stb1(ts timestamp, c_int int, c_bint bigint, c_sint smallint, c_tint tinyint,c_float float, c_double double, c_bool bool,c_binary binary(16), c_nchar nchar(32), c_ts timestamp,c_tint_un tinyint unsigned, c_sint_un smallint unsigned,c_int_un int unsigned, c_bint_un bigint unsigned) tags (t_int int);
sql create table ct1 using stb1 tags ( 1 );
sql create table ct2 using stb1 tags ( 2 );
sql create table ct3 using stb1 tags ( 3 );
sql create table ct4 using stb1 tags ( 4 );
sql CREATE SMA INDEX sma_index_1 ON stb1 function(min(c_int), max(c_int)) interval(6m, 10s) sliding(6m) watermark 5s;
sql CREATE SMA INDEX sma_index_2 ON stb1 function(min(c_int), max(c_int)) interval(6m, 10s) sliding(6m) max_delay 6m;
sql CREATE SMA INDEX sma_index_3 ON stb1 function(min(c_int), max(c_int)) interval(6m, 10s) watermark 5s max_delay 6m;
sql DROP INDEX sma_index_1 ;
sql DROP INDEX sma_index_2 ;
sql DROP INDEX sma_index_3 ;
system sh/exec.sh -n dnode1 -s stop -x SIGINT
system sh/exec.sh -n dnode2 -s stop -x SIGINT
system sh/exec.sh -n dnode3 -s stop -x SIGINT
system sh/exec.sh -n dnode4 -s stop -x SIGINT

View File

@ -1,430 +0,0 @@
system sh/stop_dnodes.sh
system sh/deploy.sh -n dnode1 -i 1
system sh/exec.sh -n dnode1 -s start
sleep 50
sql connect
print =============== create database
sql create database d1 keep 36500d vgroups 1
sql use d1
sql alter local 'querySmaOptimize' '1';
print =============== create super table, include column type for count/sum/min/max/first
sql create table if not exists stb (ts timestamp, c1 int, c2 float, c3 double, c4 binary(10),c5 nchar(10)) tags (t1 int unsigned)
sql show stables
if $rows != 1 then
return -1
endi
print =============== create child table
sql create table ct1 using stb tags(1000)
sql show tables
if $rows != 1 then
return -1
endi
print =============== insert data, mode1: one row one table in sql
sql insert into ct1 values('2022-10-19 09:55:45.682', 10, 2.0, 3.0, "a", "n0")
sql insert into ct1 values('2022-10-19 09:55:46.682', 11, 2.1, 3.1,"b","n1")('2022-10-19 09:55:47.682', -12, -2.2, -3.2,"c","n2")('2022-10-19 09:55:48.682', -13, -2.3, -3.3,"d","n3")
print =============== create sma index from super table
sql create sma index sma_index_name1 on stb function(max(c1),max(c2),min(c1)) interval(5m,10s) sliding(5m) watermark 1s max_delay 1s
sql create sma index sma_index_name2 on stb function(sum(c1),first(c1), last(c1), first(c4),last(c4),count(c4),first(c5),last(c5),count(c5),apercentile(c1,80,"t-digest"), avg(c2),count(c3), spread(c3), stddev(c2), hyperloglog(c2), hyperloglog(c4), hyperloglog(c5)) interval(5m,10s) sliding(5m);
# for varchar/binary
sql_error create sma index sma_index_name3 on stb function(sum(c4)) interval(5m,10s) sliding(5m);
sql_error create sma index sma_index_name3 on stb function(avg(c4)) interval(5m,10s) sliding(5m);
sql_error create sma index sma_index_name3 on stb function(apercentile(c4)) interval(5m,10s) sliding(5m);
sql_error create sma index sma_index_name3 on stb function(spread(c4)) interval(5m,10s) sliding(5m);
sql_error create sma index sma_index_name3 on stb function(stddev(c4)) interval(5m,10s) sliding(5m);
# for nchar
sql_error create sma index sma_index_name3 on stb function(sum(c5)) interval(5m,10s) sliding(5m);
sql_error create sma index sma_index_name3 on stb function(avg(c5)) interval(5m,10s) sliding(5m);
sql_error create sma index sma_index_name3 on stb function(apercentile(c5)) interval(5m,10s) sliding(5m);
sql_error create sma index sma_index_name3 on stb function(spread(c5)) interval(5m,10s) sliding(5m);
sql_error create sma index sma_index_name3 on stb function(stddev(c5)) interval(5m,10s) sliding(5m);
print =============== trigger stream to execute sma aggr task and insert sma data into sma store
sql insert into ct1 values('2022-10-19 09:55:50.682', 20, 20.0, 30.0,"e","n5")
#==================== sleep 2s to wait for tsma result
sleep 2000
print =============== show streams ================================
sql show streams;
print $data00 $data01 $data02
if $data00 != sma_index_name1 then
if $data00 != sma_index_name2 then
print $data00
return -1
endi
endi
print =============== select * from ct1 from memory
sql select * from ct1;
print $data00 $data01 $data02 $data03
print $data10 $data11 $data12 $data13
print $data20 $data21 $data22 $data23
print $data30 $data31 $data32 $data33
print $data40 $data41 $data42 $data43
if $rows != 5 then
print rows $rows != 5
return -1
endi
print =============== select with _wstart from stb from memory in designated vgroup
sql select _wstart, _wend, min(c1),max(c2),max(c1) from stb interval(5m,10s) sliding(5m);
print $data00 $data01 $data02 $data03 $data04
print $data10 $data11 $data12 $data13 $data14
if $rows != 1 then
print rows $rows != 1
return -1
endi
if $data02 != -13 then
print data02 $data02 != -13
return -1
endi
if $data03 != 20.00000 then
print data03 $data03 != 20.00000
return -1
endi
if $data04 != 20 then
print data04 $data04 != 20
return -1
endi
print =============== select without _wstart from stb from memory in designated vgroup
sql select min(c1),max(c2),max(c1) from stb interval(5m,10s) sliding(5m);
print $data00 $data01 $data02
print $data10 $data11 $data12
if $rows != 1 then
print rows $rows != 1
return -1
endi
if $data00 != -13 then
print data00 $data00 != -13
return -1
endi
if $data01 != 20.00000 then
print data01 $data01 != 20.00000
return -1
endi
if $data02 != 20 then
print data02 $data02 != 20
return -1
endi
print =============== select * from stb from memory in common vgroups
sql select _wstart, _wend, min(c1),max(c2),max(c1),max(c3) from stb interval(5m,10s) sliding(5m);
print $data00 $data01 $data02 $data03 $data04 $data05
if $rows != 1 then
print rows $rows != 1
return -1
endi
if $data02 != -13 then
print data02 $data02 != -13
return -1
endi
if $data03 != 20.00000 then
print data03 $data03 != 20.00000
return -1
endi
if $data04 != 20 then
print data04 $data04 != 20
return -1
endi
if $data05 != 30.000000000 then
print data05 $data05 != 30.000000000
return -1
endi
system sh/exec.sh -n dnode1 -s stop -x SIGINT
system sh/exec.sh -n dnode1 -s start
sleep 50
print =============== select * from ct1 from memory after reboot
sql select * from ct1;
print $data00 $data01 $data02 $data03
print $data10 $data11 $data12 $data13
print $data20 $data21 $data22 $data23
print $data30 $data31 $data32 $data33
print $data40 $data41 $data42 $data43
if $rows != 5 then
print rows $rows != 5
return -1
endi
print =============== select with _wstart from stb from memory in designated vgroup after reboot
sql select _wstart, _wend, min(c1),max(c2),max(c1) from stb interval(5m,10s) sliding(5m);
print $data00 $data01 $data02 $data03 $data04
print $data10 $data11 $data12 $data13 $data14
if $rows != 1 then
print rows $rows != 1
return -1
endi
if $data02 != -13 then
print data02 $data02 != -13
return -1
endi
if $data03 != 20.00000 then
print data03 $data03 != 20.00000
return -1
endi
if $data04 != 20 then
print data04 $data04 != 20
return -1
endi
print =============== select without _wstart from stb from memory in designated vgroup after reboot
sql select min(c1),max(c2),max(c1) from stb interval(5m,10s) sliding(5m);
print $data00 $data01 $data02
print $data10 $data11 $data12
if $rows != 1 then
print rows $rows != 1
return -1
endi
if $data00 != -13 then
print data00 $data00 != -13
return -1
endi
if $data01 != 20.00000 then
print data01 $data01 != 20.00000
return -1
endi
if $data02 != 20 then
print data02 $data02 != 20
return -1
endi
print =============== select * from stb from memory in common vgroups after reboot
sql select _wstart, _wend, min(c1),max(c2),max(c1),max(c3) from stb interval(5m,10s) sliding(5m);
print $data00 $data01 $data02 $data03 $data04 $data05
if $rows != 1 then
print rows $rows != 1
return -1
endi
if $data02 != -13 then
print data02 $data02 != -13
return -1
endi
if $data03 != 20.00000 then
print data03 $data03 != 20.00000
return -1
endi
if $data04 != 20 then
print data04 $data04 != 20
return -1
endi
if $data05 != 30.000000000 then
print data05 $data05 != 30.000000000
return -1
endi
#==================== flush database to trigger commit data to file
sql flush database d1;
system sh/exec.sh -n dnode1 -s stop -x SIGINT
system sh/exec.sh -n dnode1 -s start
sleep 50
print =============== select * from ct1 from file
sql select * from ct1;
print $data00 $data01 $data02 $data03
print $data10 $data11 $data12 $data13
print $data20 $data21 $data22 $data23
print $data30 $data31 $data32 $data33
print $data40 $data41 $data42 $data43
if $rows != 5 then
print rows $rows != 5
return -1
endi
print =============== select with _wstart from stb from file in designated vgroup
sql select _wstart, _wend, min(c1),max(c2),max(c1) from stb interval(5m,10s) sliding(5m);
print $data00 $data01 $data02 $data03 $data04
if $rows != 1 then
print rows $rows != 1
return -1
endi
if $data02 != -13 then
print data02 $data02 != -13
return -1
endi
if $data03 != 20.00000 then
print data03 $data03 != 20.00000
return -1
endi
if $data04 != 20 then
print data04 $data04 != 20
return -1
endi
print =============== select without _wstart from stb from file in designated vgroup
sql select _wend, min(c1),max(c2),max(c1) from stb interval(5m,10s) sliding(5m);
print $data00 $data01 $data02 $data03
if $rows != 1 then
print rows $rows != 1
return -1
endi
if $data01 != -13 then
print data01 $data01 != -13
return -1
endi
if $data02 != 20.00000 then
print data02 $data02 != 20.00000
return -1
endi
if $data03 != 20 then
print data03 $data03 != 20
return -1
endi
print =============== select * from stb from file in common vgroups
sql select _wstart, _wend, min(c1),max(c2),max(c1),max(c3) from stb interval(5m,10s) sliding(5m);
print $data00 $data01 $data02 $data03 $data04 $data05
if $rows != 1 then
print rows $rows != 1
return -1
endi
if $data02 != -13 then
print data02 $data02 != -13
return -1
endi
if $data03 != 20.00000 then
print data03 $data03 != 20.00000
return -1
endi
if $data04 != 20 then
print data04 $data04 != 20
return -1
endi
if $data05 != 30.000000000 then
print data05 $data05 != 30.000000000
return -1
endi
print =============== select with _wstart/order by _wstart from stb from file in designated vgroup
sql select _wstart, _wend, min(c1),max(c2),max(c1) from stb interval(5m,10s) sliding(5m) order by _wstart;
print $data00 $data01 $data02 $data03 $data04
if $rows != 1 then
print rows $rows != 1
return -1
endi
if $data02 != -13 then
print data02 $data02 != -13
return -1
endi
if $data03 != 20.00000 then
print data03 $data03 != 20.00000
return -1
endi
if $data04 != 20 then
print data04 $data04 != 20
return -1
endi
print =============== select without _wstart/with order by _wstart from stb from file in designated vgroup
sql select _wend, min(c1),max(c2),max(c1) from stb interval(5m,10s) sliding(5m) order by _wstart;
print $data00 $data01 $data02 $data03
if $rows != 1 then
print rows $rows != 1
return -1
endi
if $data01 != -13 then
print data01 $data01 != -13
return -1
endi
if $data02 != 20.00000 then
print data02 $data02 != 20.00000
return -1
endi
if $data03 != 20 then
print data03 $data03 != 20
return -1
endi
print =============== select * from stb from file in common vgroups
sql select _wstart, _wend, min(c1),max(c2),max(c1),max(c3) from stb interval(5m,10s) sliding(5m) order by _wstart;
print $data00 $data01 $data02 $data03 $data04 $data05
if $rows != 1 then
print rows $rows != 1
return -1
endi
if $data02 != -13 then
print data02 $data02 != -13
return -1
endi
if $data03 != 20.00000 then
print data03 $data03 != 20.00000
return -1
endi
if $data04 != 20 then
print data04 $data04 != 20
return -1
endi
if $data05 != 30.000000000 then
print data05 $data05 != 30.000000000
return -1
endi
sql delete from stb;
print =============== query after delete in common vgroups
sql select _wstart, _wend, min(c1),max(c2),max(c1),max(c3) from stb interval(5m,10s) sliding(5m) order by _wstart;
if $rows != 0 then
print rows $rows != 0
return -1
endi
sleep 2000
print =============== query after delete in designated vgroups
sql select _wend, min(c1),max(c2),max(c1) from stb interval(5m,10s) sliding(5m) order by _wstart;
if $rows != 0 then
print rows $rows != 0
return -1
endi
system sh/exec.sh -n dnode1 -s stop -x SIGINT

View File

@ -1,173 +0,0 @@
system sh/stop_dnodes.sh
system sh/deploy.sh -n dnode1 -i 1
system sh/exec.sh -n dnode1 -s start
sql connect
print ======== step0
$dbPrefix = ta_3_db
$dbPrefix1 = ta_3_db1
$tbPrefix = ta_3_tb
$mtPrefix = ta_3_mt
$tbNum = 1
$rowNum = 20
$totalNum = 200
print =============== create database
sql create database $dbPrefix
sql use $dbPrefix
print =============== create super table and register tag index
sql create table if not exists $mtPrefix (ts timestamp, c1 int) tags (t1 int, t2 int, t3 int, t4 int, t5 int)
sql show stables
if $rows != 1 then
return -1
endi
print =============== create child table
$i = 0
while $i < $tbNum
$tb = $tbPrefix . $i
sql create table $tb using $mtPrefix tags( $i , $i , $i , $i , $i );
$i = $i + 1
endw
sql show tables
if $rows != $tbNum then
return -1
endi
print =============== insert data into each table
$i = 0
while $i < $tbNum
$tb = $tbPrefix . $i
sql insert into $tb values(now, $i );
$i = $i + 1
endw
print ==== create sma and tag index, global name conflict
sql create sma index t2i on $mtPrefix function(max(c1)) interval(6m,10s) sliding(6m);
sql_error create index t2i on $mtPrefix (t2)
sql drop index t2i
print ==== create tagindex and sma index, global name conflict
sql create index t2i on $mtPrefix (t2)
sql_error create sma index t2i on $mtPrefix function(max(c1)) interval(6m,10s) sliding(6m);
sql drop index t2i
print ===== iter sma and tag index
sql create index tagt2i on $mtPrefix (t2)
sql create sma index smat2i on $mtPrefix function(max(c1)) interval(6m,10s) sliding(6m);
sql select * from information_schema.ins_indexes
if $rows != 3 then
return -1
endi
sql drop index smat2i
$i = 0
$smaPre = sma3
while $i < 5
$sma = $smaPre . $i
$i = $i + 1
sql create sma index $sma on $mtPrefix function(max(c1)) interval(6m,10s) sliding(6m);
endw
sql select * from information_schema.ins_indexes
if $rows != 7 then
return -1
endi
$i = 0
$smaPre = sma3
while $i < 5
$sma = $smaPre . $i
$i = $i + 1
sql drop index $sma
endw
#sleep 5000
sql drop stable $mtPrefix
sql select * from information_schema.ins_indexes
if $rows != 0 then
return -1
endi
sql create table if not exists $mtPrefix (ts timestamp, c1 int) tags (t1 int, t2 int, t3 int, t4 int, t5 int)
sql create index tagt2i on $mtPrefix (t2)
sql drop database $dbPrefix
sql select * from information_schema.ins_indexes
if $rows != 0 then
return -1
endi
print ===== drop tag and del tag index
sql create database $dbPrefix
sql use $dbPrefix
sql create table if not exists $mtPrefix (ts timestamp, c1 int) tags (t1 int, t2 int, t3 int, t4 int, t5 int)
sql create index tagt2i on $mtPrefix (t2)
sql select * from information_schema.ins_indexes
if $rows != 2 then
return -1
endi
sql alter table $mtPrefix drop tag t2
sql select * from information_schema.ins_indexes
if $rows != 1 then
return -1
endi
print ==== rename tag name, and update index colName
sql create index tagt3i on $mtPrefix (t3)
sql select * from information_schema.ins_indexes
if $rows != 2 then
return -1
endi
sql alter table $mtPrefix rename tag t3 txxx
sql select * from information_schema.ins_indexes
if $rows != 2 then
return -1
endi
if $data05 == txxx then
print "manual created index"
elif $data15 == txxx then
print "auto created index at tag0"
else
return -1;
endi
print ====== diff db has same index name
sql create database $dbPrefix1
sql use $dbPrefix1
sql create table if not exists $mtPrefix (ts timestamp, c1 int) tags (t1 int, t2 int, t3 int, t4 int, t5 int)
sql create index tagt3i on $mtPrefix (t3)
sql select * from information_schema.ins_indexes
if $rows != 4 then
return -1
endi
system sh/exec.sh -n dnode1 -s stop -x SIGINT

View File

@ -134,7 +134,6 @@ run tsim/sync/insertDataByRunBack.sim
run tsim/sync/oneReplica5VgElect.sim
run tsim/sync/3Replica1VgElect.sim
run tsim/sync/threeReplica1VgElectWihtInsert.sim
run tsim/sma/tsmaCreateInsertQuery.sim
run tsim/sma/rsmaCreateInsertQuery.sim
run tsim/sma/rsmaCreateInsertQueryDelete.sim
run tsim/valgrind/basic.sim

View File

@ -1,48 +0,0 @@
import taos
import sys
import time
import socket
import os
import threading
from util.log import *
from util.sql import *
from util.cases import *
from util.dnodes import *
class TDTestCase:
hostname = socket.gethostname()
def init(self, conn, logSql, replicaVar=1):
self.replicaVar = int(replicaVar)
tdLog.debug(f"start to excute {__file__}")
#tdSql.init(conn.cursor())
tdSql.init(conn.cursor(), logSql) # output sql.txt file
def create_databases(self):
tdSql.execute("create database db_ms precision 'ms'")
tdSql.execute("create database db_us precision 'us'")
tdSql.execute("create database db_ns precision 'ns'")
def create_stables(self):
tdSql.execute("CREATE STABLE db_ms.`meters` (`ts` TIMESTAMP, `c0` INT, `c1` TINYINT, `c2` DOUBLE, `c3` VARCHAR(64), `c4` NCHAR(64)) TAGS (`cc` VARCHAR(16))")
tdSql.execute("CREATE STABLE db_us.`meters` (`ts` TIMESTAMP, `c0` INT, `c1` TINYINT, `c2` DOUBLE, `c3` VARCHAR(64), `c4` NCHAR(64)) TAGS (`cc` VARCHAR(16))")
tdSql.execute("CREATE STABLE db_ns.`meters` (`ts` TIMESTAMP, `c0` INT, `c1` TINYINT, `c2` DOUBLE, `c3` VARCHAR(64), `c4` NCHAR(64)) TAGS (`cc` VARCHAR(16))")
def create_sma_index(self):
tdSql.execute("create sma index sma_index_ms on db_ms.meters function(max(c1), max(c2), min(c1)) interval(6m, 10s) sliding(6m)" )
tdSql.execute("create sma index sma_index_us on db_us.meters function(max(c1), max(c2), min(c1)) interval(6m, 10s) sliding(6m)" )
tdSql.execute("create sma index sma_index_ns on db_ns.meters function(max(c1), max(c2), min(c1)) interval(6m, 10s) sliding(6m)" )
def run(self):
tdSql.prepare()
self.create_databases()
self.create_stables()
self.create_sma_index()
def stop(self):
tdSql.close()
tdLog.success(f"{__file__} successfully executed")
tdCases.addLinux(__file__, TDTestCase())
tdCases.addWindows(__file__, TDTestCase())

View File

@ -144,7 +144,6 @@ class TDTestCase:
tdSql.execute("insert into db0.ct1 values(now+0s, 10, 2.0, 3.0);")
tdSql.execute("insert into db0.ct1 values(now+1s, 11, 2.1, 3.1)(now+2s, 12, 2.2, 3.2)(now+3s, 13, 2.3, 3.3);")
tdSql.execute("insert into db0.ntb values(now+2s, 10, 2.0, 3.0);")
tdSql.execute("create sma index sma_index_name1 on db0.stb function(max(c1),max(c2),min(c1)) interval(6m,10s) sliding(6m);")
tdSql.execute("create topic tpc1 as select * from db0.ct2; ")

View File

@ -168,11 +168,6 @@ class TDTestCase:
sql = f"create table db.st(ts timestamp, c1 int, c2 bigint, ts1 timestamp) tags(area int)"
tdSql.execute(sql, 5, True)
sql = f"create sma index sma_test on db.st function(max(c1),max(c2),min(c1),min(c2)) {self.smaClause};"
tdLog.info(sql)
tdSql.error(sql)
# create database db
sql = f"create database @db_name vgroups {self.vgroups1} replica 1"
self.exeDouble(sql)
@ -186,13 +181,6 @@ class TDTestCase:
sql = f"create table @db_name.t{i} using @db_name.st tags({i}) "
self.exeDouble(sql)
# create sma index on db2
sql = f"use {self.db2}"
tdSql.execute(sql)
sql = f"create sma index sma_index_maxmin on {self.db2}.st function(max(c1),max(c2),min(c1),min(c2)) {self.smaClause};"
tdLog.info(sql)
tdSql.execute(sql)
# insert data
self.insertData()

View File

@ -493,7 +493,7 @@ class TDTestCase:
tdSql.query('select case 值 when 标签1 then 标签1 else 标签2 end from sel_null.stable1', queryTimes=1)
tdSql.query('select count(*) from sel_null.stable1 group by 值 having sum(标签1) > 0', queryTimes=1)
tdSql.query('show table tags `标签1` 标签n from sel_null.stable1', queryTimes=1)
tdSql.query('create sma index a on sel_null.stable1 FUNCTION (sum(值)) interval(1s)', queryTimes=1)
#tdSql.query('create sma index a on sel_null.stable1 FUNCTION (sum(值)) interval(1s)', queryTimes=1)
tdSql.query('select count(值) from sel_null.stable1', queryTimes=1)
tdSql.query('select stable1.值 from sel_null.stable1', queryTimes=1)
tdSql.query('select stable1.值 from sel_null.stable1 order by 值', queryTimes=1)

View File

@ -502,7 +502,6 @@ python3 ./test.py -f 2-query/sin.py -R
python3 ./test.py -f 2-query/smaBasic.py -N 3
python3 ./test.py -f 2-query/smaTest.py
python3 ./test.py -f 2-query/smaTest.py -R
python3 ./test.py -f 0-others/sma_index.py
python3 ./test.py -f 2-query/sml_TS-3724.py
python3 ./test.py -f 2-query/sml-TD19291.py
python3 ./test.py -f 2-query/varbinary.py